From 236893f452f39d7236b653e03d8f26c9810221ac Mon Sep 17 00:00:00 2001 From: Andrew Lamb Date: Fri, 29 Sep 2023 03:19:27 -0400 Subject: [PATCH 001/572] Minor: Improve `TableProviderFilterPushDown` docs (#7685) --- datafusion/expr/src/table_source.rs | 14 +++++++------- 1 file changed, 7 insertions(+), 7 deletions(-) diff --git a/datafusion/expr/src/table_source.rs b/datafusion/expr/src/table_source.rs index b83ce778133b..94f26d9158cd 100644 --- a/datafusion/expr/src/table_source.rs +++ b/datafusion/expr/src/table_source.rs @@ -30,14 +30,14 @@ use std::any::Any; pub enum TableProviderFilterPushDown { /// The expression cannot be used by the provider. Unsupported, - /// The expression can be used to help minimise the data retrieved, - /// but the provider cannot guarantee that all returned tuples - /// satisfy the filter. The Filter plan node containing this expression - /// will be preserved. + /// The expression can be used to reduce the data retrieved, + /// but the provider cannot guarantee it will omit all tuples that + /// may be filtered. In this case, DataFusion will apply an additional + /// `Filter` operation after the scan to ensure all rows are filtered correctly. Inexact, - /// The provider guarantees that all returned data satisfies this - /// filter expression. The Filter plan node containing this expression - /// will be removed. + /// The provider **guarantees** that it will omit **all** tuples that are + /// filtered by the filter expression. This is the fastest option, if available + /// as DataFusion will not apply additional filtering. Exact, } From c6d29e7854d78efca84ffd473131a674cb420f6d Mon Sep 17 00:00:00 2001 From: Jay Zhan Date: Fri, 29 Sep 2023 20:28:58 +0800 Subject: [PATCH 002/572] FIX: Test timestamp with table (#7701) * use table for test Signed-off-by: jayzhan211 * fmt Signed-off-by: jayzhan211 --------- Signed-off-by: jayzhan211 --- datafusion/core/tests/sql/timestamp.rs | 31 +++++++++++++++----------- 1 file changed, 18 insertions(+), 13 deletions(-) diff --git a/datafusion/core/tests/sql/timestamp.rs b/datafusion/core/tests/sql/timestamp.rs index 09bbd1754276..ada66503a181 100644 --- a/datafusion/core/tests/sql/timestamp.rs +++ b/datafusion/core/tests/sql/timestamp.rs @@ -567,25 +567,30 @@ async fn timestamp_sub_interval_days() -> Result<()> { #[tokio::test] async fn timestamp_add_interval_months() -> Result<()> { let ctx = SessionContext::new(); + let table_a = + make_timestamp_tz_table::(Some("+00:00".into()))?; + ctx.register_table("table_a", table_a)?; - let sql = "SELECT NOW(), NOW() + INTERVAL '17' MONTH;"; + let sql = "SELECT ts, ts + INTERVAL '17' MONTH FROM table_a;"; let results = execute_to_batches(&ctx, sql).await; - let actual = result_vec(&results); + let actual_vec = result_vec(&results); - let res1 = actual[0][0].as_str(); - let res2 = actual[0][1].as_str(); + for actual in actual_vec { + let res1 = actual[0].as_str(); + let res2 = actual[1].as_str(); - let format = "%Y-%m-%dT%H:%M:%S%.6fZ"; - let t1_naive = NaiveDateTime::parse_from_str(res1, format).unwrap(); - let t2_naive = NaiveDateTime::parse_from_str(res2, format).unwrap(); + let format = "%Y-%m-%dT%H:%M:%S%.6fZ"; + let t1_naive = NaiveDateTime::parse_from_str(res1, format).unwrap(); + let t2_naive = NaiveDateTime::parse_from_str(res2, format).unwrap(); - let year = t1_naive.year() + (t1_naive.month0() as i32 + 17) / 12; - let month = (t1_naive.month0() + 17) % 12 + 1; + let year = t1_naive.year() + (t1_naive.month0() as i32 + 17) / 12; + let month = (t1_naive.month0() + 17) % 12 + 1; - assert_eq!( - t1_naive.with_year(year).unwrap().with_month(month).unwrap(), - t2_naive - ); + assert_eq!( + t1_naive.with_year(year).unwrap().with_month(month).unwrap(), + t2_naive + ); + } Ok(()) } From 2abacf4a070a8cb48ac08da73d5e77331652903d Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Dani=C3=ABl=20Heres?= Date: Fri, 29 Sep 2023 15:24:57 +0200 Subject: [PATCH 003/572] Fix bug in simplify expressions (#214) (#7699) Co-authored-by: Dan Harris <1327726+thinkharderdev@users.noreply.github.com> --- .../simplify_expressions/simplify_exprs.rs | 51 +++++++++++++++++-- 1 file changed, 48 insertions(+), 3 deletions(-) diff --git a/datafusion/optimizer/src/simplify_expressions/simplify_exprs.rs b/datafusion/optimizer/src/simplify_expressions/simplify_exprs.rs index 35a698b709ac..355d556d5da4 100644 --- a/datafusion/optimizer/src/simplify_expressions/simplify_exprs.rs +++ b/datafusion/optimizer/src/simplify_expressions/simplify_exprs.rs @@ -65,10 +65,13 @@ impl SimplifyExpressions { ) -> Result { let schema = if !plan.inputs().is_empty() { DFSchemaRef::new(merge_schema(plan.inputs())) - } else if let LogicalPlan::TableScan(_) = plan { + } else if let LogicalPlan::TableScan(scan) = plan { // When predicates are pushed into a table scan, there needs to be // a schema to resolve the fields against. - Arc::clone(plan.schema()) + Arc::new(DFSchema::try_from_qualified_schema( + &scan.table_name, + &scan.source.schema(), + )?) } else { Arc::new(DFSchema::empty()) }; @@ -111,7 +114,7 @@ mod tests { use crate::simplify_expressions::utils::for_test::{ cast_to_int64_expr, now_expr, to_timestamp_expr, }; - use crate::test::test_table_scan_with_name; + use crate::test::{assert_fields_eq, test_table_scan_with_name}; use super::*; use arrow::datatypes::{DataType, Field, Schema}; @@ -174,6 +177,48 @@ mod tests { Ok(()) } + #[test] + fn test_simplify_table_full_filter_in_scan() -> Result<()> { + let fields = vec![ + Field::new("a", DataType::UInt32, false), + Field::new("b", DataType::UInt32, false), + Field::new("c", DataType::UInt32, false), + ]; + + let schema = Schema::new(fields); + + let table_scan = table_scan_with_filters( + Some("test"), + &schema, + Some(vec![0]), + vec![col("b").is_not_null()], + )? + .build()?; + assert_eq!(1, table_scan.schema().fields().len()); + assert_fields_eq(&table_scan, vec!["a"]); + + let expected = "TableScan: test projection=[a], full_filters=[Boolean(true) AS b IS NOT NULL]"; + + assert_optimized_plan_eq(&table_scan, expected) + } + + #[test] + fn test_simplify_filter_pushdown() -> Result<()> { + let table_scan = test_table_scan(); + let plan = LogicalPlanBuilder::from(table_scan) + .project(vec![col("a")])? + .filter(and(col("b").gt(lit(1)), col("b").gt(lit(1))))? + .build()?; + + assert_optimized_plan_eq( + &plan, + "\ + Filter: test.b > Int32(1)\ + \n Projection: test.a\ + \n TableScan: test", + ) + } + #[test] fn test_simplify_optimized_plan() -> Result<()> { let table_scan = test_table_scan(); From fcd94fb5b7a49597bde4b27742f1d54b97f149c6 Mon Sep 17 00:00:00 2001 From: Mustafa Akur <106137913+mustafasrepo@users.noreply.github.com> Date: Fri, 29 Sep 2023 16:27:21 +0300 Subject: [PATCH 004/572] Enhance Enforce Dist capabilities to fix, sub optimal bad plans (#7671) * Extend capabilities of enforcedist * Simplifications * Fix test * Do not use hard coded partition number * Add comments, Fix with_new_children of CustomPlan * Use sub-rule as separate rule. * Add unbounded method * Final review * Move util code to exectree file * Update variables and comments * Apply suggestions from code review Update comments Co-authored-by: Andrew Lamb * Address reviews * Add new tests, do not satisfy requirement if not absolutely necessary enforce dist --------- Co-authored-by: Mehmet Ozan Kabak Co-authored-by: Andrew Lamb --- .../core/src/datasource/listing/table.rs | 60 +-- .../enforce_distribution.rs | 417 +++++++++++++----- .../src/physical_optimizer/enforce_sorting.rs | 1 - datafusion/core/src/physical_optimizer/mod.rs | 1 + .../core/src/physical_optimizer/optimizer.rs | 7 + .../physical_optimizer/output_requirements.rs | 275 ++++++++++++ .../core/src/physical_optimizer/utils.rs | 24 + .../provider_filter_pushdown.rs | 19 +- .../physical-plan/src/coalesce_partitions.rs | 4 + datafusion/physical-plan/src/memory.rs | 9 +- datafusion/physical-plan/src/streaming.rs | 8 +- .../sqllogictest/test_files/explain.slt | 4 + .../sqllogictest/test_files/groupby.slt | 6 +- datafusion/sqllogictest/test_files/window.slt | 66 +++ 14 files changed, 758 insertions(+), 143 deletions(-) create mode 100644 datafusion/core/src/physical_optimizer/output_requirements.rs diff --git a/datafusion/core/src/datasource/listing/table.rs b/datafusion/core/src/datasource/listing/table.rs index 8360847e1bd1..797562e92d2e 100644 --- a/datafusion/core/src/datasource/listing/table.rs +++ b/datafusion/core/src/datasource/listing/table.rs @@ -20,16 +20,8 @@ use std::str::FromStr; use std::{any::Any, sync::Arc}; -use arrow::compute::SortOptions; -use arrow::datatypes::{DataType, Field, SchemaBuilder, SchemaRef}; -use arrow_schema::Schema; -use async_trait::async_trait; -use datafusion_common::FileTypeWriterOptions; -use datafusion_common::{internal_err, plan_err, project_schema, SchemaExt, ToDFSchema}; -use datafusion_expr::expr::Sort; -use datafusion_optimizer::utils::conjunction; -use datafusion_physical_expr::{create_physical_expr, LexOrdering, PhysicalSortExpr}; -use futures::{future, stream, StreamExt, TryStreamExt}; +use super::helpers::{expr_applicable_for_cols, pruned_partition_list, split_files}; +use super::PartitionedFile; use crate::datasource::file_format::file_compression_type::{ FileCompressionType, FileTypeExt, @@ -54,13 +46,21 @@ use crate::{ logical_expr::Expr, physical_plan::{empty::EmptyExec, ExecutionPlan, Statistics}, }; -use datafusion_common::FileType; +use arrow::compute::SortOptions; +use arrow::datatypes::{DataType, Field, SchemaBuilder, SchemaRef}; +use arrow_schema::Schema; +use datafusion_common::{ + internal_err, plan_err, project_schema, FileType, FileTypeWriterOptions, SchemaExt, + ToDFSchema, +}; use datafusion_execution::cache::cache_manager::FileStatisticsCache; use datafusion_execution::cache::cache_unit::DefaultFileStatisticsCache; +use datafusion_expr::expr::Sort; +use datafusion_optimizer::utils::conjunction; +use datafusion_physical_expr::{create_physical_expr, LexOrdering, PhysicalSortExpr}; -use super::PartitionedFile; - -use super::helpers::{expr_applicable_for_cols, pruned_partition_list, split_files}; +use async_trait::async_trait; +use futures::{future, stream, StreamExt, TryStreamExt}; /// Configuration for creating a [`ListingTable`] #[derive(Debug, Clone)] @@ -996,6 +996,9 @@ impl ListingTable { #[cfg(test)] mod tests { + use std::collections::HashMap; + use std::fs::File; + use super::*; use crate::datasource::{provider_as_source, MemTable}; use crate::execution::options::ArrowReadOptions; @@ -1010,14 +1013,13 @@ mod tests { logical_expr::{col, lit}, test::{columns, object_store::register_test_store}, }; + use arrow::datatypes::{DataType, Schema}; use arrow::record_batch::RecordBatch; - use datafusion_common::assert_contains; - use datafusion_common::GetExt; - use datafusion_expr::LogicalPlanBuilder; + use datafusion_common::{assert_contains, GetExt, ScalarValue}; + use datafusion_expr::{BinaryExpr, LogicalPlanBuilder, Operator}; + use rstest::*; - use std::collections::HashMap; - use std::fs::File; use tempfile::TempDir; /// It creates dummy file and checks if it can create unbounded input executors. @@ -2048,6 +2050,7 @@ mod tests { } None => SessionContext::new(), }; + let target_partition_number = session_ctx.state().config().target_partitions(); // Create a new schema with one field called "a" of type Int32 let schema = Arc::new(Schema::new(vec![Field::new( @@ -2056,6 +2059,12 @@ mod tests { false, )])); + let filter_predicate = Expr::BinaryExpr(BinaryExpr::new( + Box::new(Expr::Column("column1".into())), + Operator::GtEq, + Box::new(Expr::Literal(ScalarValue::Int32(Some(0)))), + )); + // Create a new batch of data to insert into the table let batch = RecordBatch::try_new( schema.clone(), @@ -2136,8 +2145,10 @@ mod tests { let source = provider_as_source(source_table); // Create a table scan logical plan to read from the source table let scan_plan = LogicalPlanBuilder::scan("source", source, None)? - .repartition(Partitioning::Hash(vec![Expr::Column("column1".into())], 6))? + .filter(filter_predicate)? .build()?; + // Since logical plan contains a filter, increasing parallelism is helpful. + // Therefore, we will have 8 partitions in the final plan. // Create an insert plan to insert the source data into the initial table let insert_into_table = LogicalPlanBuilder::insert_into(scan_plan, "t", &schema, false)?.build()?; @@ -2146,7 +2157,6 @@ mod tests { .state() .create_physical_plan(&insert_into_table) .await?; - // Execute the physical plan and collect the results let res = collect(plan, session_ctx.task_ctx()).await?; // Insert returns the number of rows written, in our case this would be 6. @@ -2178,9 +2188,9 @@ mod tests { // Assert that the batches read from the file match the expected result. assert_batches_eq!(expected, &batches); - // Assert that 6 files were added to the table + // Assert that `target_partition_number` many files were added to the table. let num_files = tmp_dir.path().read_dir()?.count(); - assert_eq!(num_files, 6); + assert_eq!(num_files, target_partition_number); // Create a physical plan from the insert plan let plan = session_ctx @@ -2221,9 +2231,9 @@ mod tests { // Assert that the batches read from the file after the second append match the expected result. assert_batches_eq!(expected, &batches); - // Assert that another 6 files were added to the table + // Assert that another `target_partition_number` many files were added to the table. let num_files = tmp_dir.path().read_dir()?.count(); - assert_eq!(num_files, 12); + assert_eq!(num_files, 2 * target_partition_number); // Return Ok if the function Ok(()) diff --git a/datafusion/core/src/physical_optimizer/enforce_distribution.rs b/datafusion/core/src/physical_optimizer/enforce_distribution.rs index b2a1a0338384..b3fb41ea100f 100644 --- a/datafusion/core/src/physical_optimizer/enforce_distribution.rs +++ b/datafusion/core/src/physical_optimizer/enforce_distribution.rs @@ -27,8 +27,11 @@ use std::sync::Arc; use crate::config::ConfigOptions; use crate::datasource::physical_plan::{CsvExec, ParquetExec}; -use crate::error::{DataFusionError, Result}; -use crate::physical_optimizer::utils::{add_sort_above, get_plan_string, ExecTree}; +use crate::error::Result; +use crate::physical_optimizer::utils::{ + add_sort_above, get_children_exectrees, get_plan_string, is_coalesce_partitions, + is_repartition, is_sort_preserving_merge, ExecTree, +}; use crate::physical_optimizer::PhysicalOptimizerRule; use crate::physical_plan::aggregates::{AggregateExec, AggregateMode, PhysicalGroupBy}; use crate::physical_plan::coalesce_partitions::CoalescePartitionsExec; @@ -44,7 +47,6 @@ use crate::physical_plan::windows::WindowAggExec; use crate::physical_plan::Partitioning; use crate::physical_plan::{with_new_children_if_necessary, Distribution, ExecutionPlan}; -use datafusion_common::internal_err; use datafusion_common::tree_node::{Transformed, TreeNode, VisitRecursion}; use datafusion_expr::logical_plan::JoinType; use datafusion_physical_expr::equivalence::EquivalenceProperties; @@ -57,6 +59,7 @@ use datafusion_physical_expr::{ }; use datafusion_physical_plan::unbounded_output; +use datafusion_physical_plan::windows::{get_best_fitting_window, BoundedWindowAggExec}; use itertools::izip; /// The `EnforceDistribution` rule ensures that distribution requirements are @@ -213,9 +216,7 @@ impl PhysicalOptimizerRule for EnforceDistribution { distribution_context.transform_up(&|distribution_context| { ensure_distribution(distribution_context, config) })?; - - // If output ordering is not necessary, removes it - update_plan_to_remove_unnecessary_final_order(distribution_context) + Ok(distribution_context.plan) } fn name(&self) -> &str { @@ -979,12 +980,6 @@ fn add_roundrobin_on_top( RepartitionExec::try_new(input, Partitioning::RoundRobinBatch(n_target))? .with_preserve_order(should_preserve_ordering), ) as Arc; - if let Some(exec_tree) = dist_onward { - return internal_err!( - "ExecTree should have been empty, but got:{:?}", - exec_tree - ); - } // update distribution onward with new operator update_distribution_onward(new_plan.clone(), dist_onward, input_idx); @@ -1112,8 +1107,9 @@ fn add_spm_on_top( } } -/// Updates the physical plan inside `distribution_context` if having a -/// `RepartitionExec(RoundRobin)` is not helpful. +/// Updates the physical plan inside `distribution_context` so that distribution +/// changing operators are removed from the top. If they are necessary, they will +/// be added in subsequent stages. /// /// Assume that following plan is given: /// ```text @@ -1122,14 +1118,13 @@ fn add_spm_on_top( /// " ParquetExec: file_groups={2 groups: \[\[x], \[y]]}, projection=\[a, b, c, d, e], output_ordering=\[a@0 ASC]", /// ``` /// -/// `RepartitionExec` at the top is unnecessary. Since it doesn't help with increasing parallelism. -/// This function removes top repartition, and returns following plan. +/// Since `RepartitionExec`s change the distribution, this function removes +/// them and returns following plan: /// /// ```text -/// "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=2", -/// " ParquetExec: file_groups={2 groups: \[\[x], \[y]]}, projection=\[a, b, c, d, e], output_ordering=\[a@0 ASC]", +/// "ParquetExec: file_groups={2 groups: \[\[x], \[y]]}, projection=\[a, b, c, d, e], output_ordering=\[a@0 ASC]", /// ``` -fn remove_unnecessary_repartition( +fn remove_dist_changing_operators( distribution_context: DistributionContext, ) -> Result { let DistributionContext { @@ -1137,22 +1132,17 @@ fn remove_unnecessary_repartition( mut distribution_onwards, } = distribution_context; - // Remove any redundant RoundRobin at the start: - if let Some(repartition) = plan.as_any().downcast_ref::() { - if let Partitioning::RoundRobinBatch(n_out) = repartition.partitioning() { - // Repartition is useless: - if *n_out <= repartition.input().output_partitioning().partition_count() { - let mut new_distribution_onwards = - vec![None; repartition.input().children().len()]; - if let Some(exec_tree) = &distribution_onwards[0] { - for child in &exec_tree.children { - new_distribution_onwards[child.idx] = Some(child.clone()); - } - } - plan = repartition.input().clone(); - distribution_onwards = new_distribution_onwards; - } - } + // Remove any distribution changing operators at the beginning: + // Note that they will be re-inserted later on if necessary or helpful. + while is_repartition(&plan) + || is_coalesce_partitions(&plan) + || is_sort_preserving_merge(&plan) + { + // All of above operators have a single child. When we remove the top + // operator, we take the first child. + plan = plan.children()[0].clone(); + distribution_onwards = + get_children_exectrees(plan.children().len(), &distribution_onwards[0]); } // Create a plan with the updated children: @@ -1162,29 +1152,6 @@ fn remove_unnecessary_repartition( }) } -/// Changes each child of the `dist_context.plan` such that they no longer -/// use order preserving variants, if no ordering is required at the output -/// of the physical plan (there is no global ordering requirement by the query). -fn update_plan_to_remove_unnecessary_final_order( - dist_context: DistributionContext, -) -> Result> { - let DistributionContext { - plan, - distribution_onwards, - } = dist_context; - let new_children = izip!(plan.children(), distribution_onwards) - .map(|(mut child, mut dist_onward)| { - replace_order_preserving_variants(&mut child, &mut dist_onward)?; - Ok(child) - }) - .collect::>>()?; - if !new_children.is_empty() { - plan.with_new_children(new_children) - } else { - Ok(plan) - } -} - /// Updates the physical plan `input` by using `dist_onward` replace order preserving operator variants /// with their corresponding operators that do not preserve order. It is a wrapper for `replace_order_preserving_variants_helper` fn replace_order_preserving_variants( @@ -1224,18 +1191,16 @@ fn replace_order_preserving_variants_helper( for child in &exec_tree.children { updated_children[child.idx] = replace_order_preserving_variants_helper(child)?; } - if let Some(spm) = exec_tree - .plan - .as_any() - .downcast_ref::() - { - return Ok(Arc::new(CoalescePartitionsExec::new(spm.input().clone()))); + if is_sort_preserving_merge(&exec_tree.plan) { + return Ok(Arc::new(CoalescePartitionsExec::new( + updated_children[0].clone(), + ))); } if let Some(repartition) = exec_tree.plan.as_any().downcast_ref::() { if repartition.preserve_order() { return Ok(Arc::new( RepartitionExec::try_new( - repartition.input().clone(), + updated_children[0].clone(), repartition.partitioning().clone(), )? .with_preserve_order(false), @@ -1275,12 +1240,29 @@ fn ensure_distribution( repartition_beneficial_stat = stats.num_rows.map(|num_rows| num_rows > 1).unwrap_or(true); } - // Remove unnecessary repartition from the physical plan if any let DistributionContext { - plan, + mut plan, mut distribution_onwards, - } = remove_unnecessary_repartition(dist_context)?; + } = remove_dist_changing_operators(dist_context)?; + + if let Some(exec) = plan.as_any().downcast_ref::() { + if let Some(updated_window) = get_best_fitting_window( + exec.window_expr(), + exec.input(), + &exec.partition_keys, + )? { + plan = updated_window; + } + } else if let Some(exec) = plan.as_any().downcast_ref::() { + if let Some(updated_window) = get_best_fitting_window( + exec.window_expr(), + exec.input(), + &exec.partition_keys, + )? { + plan = updated_window; + } + }; let n_children = plan.children().len(); // This loop iterates over all the children to: @@ -1369,19 +1351,23 @@ fn ensure_distribution( // Either: // - Ordering requirement cannot be satisfied by preserving ordering through repartitions, or // - using order preserving variant is not desirable. - if !ordering_satisfy_requirement_concrete( + let ordering_satisfied = ordering_satisfy_requirement_concrete( existing_ordering, required_input_ordering, || child.equivalence_properties(), || child.ordering_equivalence_properties(), - ) || !order_preserving_variants_desirable - { + ); + if !ordering_satisfied || !order_preserving_variants_desirable { replace_order_preserving_variants(&mut child, dist_onward)?; - let sort_expr = PhysicalSortRequirement::to_sort_exprs( - required_input_ordering.clone(), - ); - // Make sure to satisfy ordering requirement - add_sort_above(&mut child, sort_expr, None)?; + // If ordering requirements were satisfied before repartitioning, + // make sure ordering requirements are still satisfied after. + if ordering_satisfied { + // Make sure to satisfy ordering requirement: + let sort_expr = PhysicalSortRequirement::to_sort_exprs( + required_input_ordering.clone(), + ); + add_sort_above(&mut child, sort_expr, None)?; + } } // Stop tracking distribution changing operators *dist_onward = None; @@ -1690,6 +1676,7 @@ mod tests { use crate::datasource::object_store::ObjectStoreUrl; use crate::datasource::physical_plan::{FileScanConfig, ParquetExec}; use crate::physical_optimizer::enforce_sorting::EnforceSorting; + use crate::physical_optimizer::output_requirements::OutputRequirements; use crate::physical_plan::aggregates::{ AggregateExec, AggregateMode, PhysicalGroupBy, }; @@ -1703,9 +1690,12 @@ mod tests { use crate::physical_plan::sorts::sort_preserving_merge::SortPreservingMergeExec; use crate::physical_plan::{displayable, DisplayAs, DisplayFormatType, Statistics}; - use crate::physical_optimizer::test_utils::repartition_exec; + use crate::physical_optimizer::test_utils::{ + coalesce_partitions_exec, repartition_exec, + }; use crate::physical_plan::limit::{GlobalLimitExec, LocalLimitExec}; use crate::physical_plan::sorts::sort::SortExec; + use arrow::compute::SortOptions; use arrow::datatypes::{DataType, Field, Schema, SchemaRef}; use datafusion_common::ScalarValue; @@ -1714,7 +1704,7 @@ mod tests { use datafusion_physical_expr::expressions::{BinaryExpr, Literal}; use datafusion_physical_expr::{ expressions, expressions::binary, expressions::lit, expressions::Column, - PhysicalExpr, PhysicalSortExpr, + LexOrdering, PhysicalExpr, PhysicalSortExpr, }; /// Models operators like BoundedWindowExec that require an input @@ -1722,11 +1712,23 @@ mod tests { #[derive(Debug)] struct SortRequiredExec { input: Arc, + expr: LexOrdering, } impl SortRequiredExec { fn new(input: Arc) -> Self { - Self { input } + let expr = input.output_ordering().unwrap_or(&[]).to_vec(); + Self { input, expr } + } + + fn new_with_requirement( + input: Arc, + requirement: Vec, + ) -> Self { + Self { + input, + expr: requirement, + } } } @@ -1736,7 +1738,8 @@ mod tests { _t: DisplayFormatType, f: &mut std::fmt::Formatter, ) -> std::fmt::Result { - write!(f, "SortRequiredExec") + let expr: Vec = self.expr.iter().map(|e| e.to_string()).collect(); + write!(f, "SortRequiredExec: [{}]", expr.join(",")) } } @@ -1778,7 +1781,10 @@ mod tests { ) -> Result> { assert_eq!(children.len(), 1); let child = children.pop().unwrap(); - Ok(Arc::new(Self::new(child))) + Ok(Arc::new(Self::new_with_requirement( + child, + self.expr.clone(), + ))) } fn execute( @@ -2054,6 +2060,13 @@ mod tests { Arc::new(SortRequiredExec::new(input)) } + fn sort_required_exec_with_req( + input: Arc, + sort_exprs: LexOrdering, + ) -> Arc { + Arc::new(SortRequiredExec::new_with_requirement(input, sort_exprs)) + } + fn trim_plan_display(plan: &str) -> Vec<&str> { plan.split('\n') .map(|s| s.trim()) @@ -2118,10 +2131,15 @@ mod tests { // `EnforceSorting` and `EnforceDistribution`. // TODO: Orthogonalize the tests here just to verify `EnforceDistribution` and create // new tests for the cascade. + + // Add the ancillary output requirements operator at the start: + let optimizer = OutputRequirements::new_add_mode(); + let optimized = optimizer.optimize($PLAN.clone(), &config)?; + let optimized = if $FIRST_ENFORCE_DIST { // Run enforce distribution rule first: let optimizer = EnforceDistribution::new(); - let optimized = optimizer.optimize($PLAN.clone(), &config)?; + let optimized = optimizer.optimize(optimized, &config)?; // The rule should be idempotent. // Re-running this rule shouldn't introduce unnecessary operators. let optimizer = EnforceDistribution::new(); @@ -2133,7 +2151,7 @@ mod tests { } else { // Run the enforce sorting rule first: let optimizer = EnforceSorting::new(); - let optimized = optimizer.optimize($PLAN.clone(), &config)?; + let optimized = optimizer.optimize(optimized, &config)?; // Run enforce distribution rule: let optimizer = EnforceDistribution::new(); let optimized = optimizer.optimize(optimized, &config)?; @@ -2144,6 +2162,10 @@ mod tests { optimized }; + // Remove the ancillary output requirements operator when done: + let optimizer = OutputRequirements::new_remove_mode(); + let optimized = optimizer.optimize(optimized, &config)?; + // Now format correctly let plan = displayable(optimized.as_ref()).indent(true).to_string(); let actual_lines = trim_plan_display(&plan); @@ -2978,7 +3000,7 @@ mod tests { format!("SortMergeJoin: join_type={join_type}, on=[(a@0, c@2)]"); let expected = match join_type { - // Should include 6 RepartitionExecs 3 SortExecs + // Should include 6 RepartitionExecs (3 hash, 3 round-robin), 3 SortExecs JoinType::Inner | JoinType::Left | JoinType::LeftSemi | JoinType::LeftAnti => vec![ top_join_plan.as_str(), @@ -2997,9 +3019,18 @@ mod tests { "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", ], - // Should include 7 RepartitionExecs + // Should include 7 RepartitionExecs (4 hash, 3 round-robin), 4 SortExecs + // Since ordering of the left child is not preserved after SortMergeJoin + // when mode is Right, RgihtSemi, RightAnti, Full + // - We need to add one additional SortExec after SortMergeJoin in contrast the test cases + // when mode is Inner, Left, LeftSemi, LeftAnti + // Similarly, since partitioning of the left side is not preserved + // when mode is Right, RgihtSemi, RightAnti, Full + // - We need to add one additional Hash Repartition after SortMergeJoin in contrast the test + // cases when mode is Inner, Left, LeftSemi, LeftAnti _ => vec![ top_join_plan.as_str(), + // Below 2 operators are differences introduced, when join mode is changed "SortExec: expr=[a@0 ASC]", "RepartitionExec: partitioning=Hash([a@0], 10), input_partitions=10", join_plan.as_str(), @@ -3021,7 +3052,7 @@ mod tests { assert_optimized!(expected, top_join.clone(), true, true); let expected_first_sort_enforcement = match join_type { - // Should include 3 RepartitionExecs 3 SortExecs + // Should include 6 RepartitionExecs (3 hash, 3 round-robin), 3 SortExecs JoinType::Inner | JoinType::Left | JoinType::LeftSemi | JoinType::LeftAnti => vec![ top_join_plan.as_str(), @@ -3040,9 +3071,18 @@ mod tests { "SortExec: expr=[c@2 ASC]", "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", ], - // Should include 8 RepartitionExecs (4 of them preserves ordering) + // Should include 8 RepartitionExecs (4 hash, 8 round-robin), 4 SortExecs + // Since ordering of the left child is not preserved after SortMergeJoin + // when mode is Right, RgihtSemi, RightAnti, Full + // - We need to add one additional SortExec after SortMergeJoin in contrast the test cases + // when mode is Inner, Left, LeftSemi, LeftAnti + // Similarly, since partitioning of the left side is not preserved + // when mode is Right, RgihtSemi, RightAnti, Full + // - We need to add one additional Hash Repartition and Roundrobin repartition after + // SortMergeJoin in contrast the test cases when mode is Inner, Left, LeftSemi, LeftAnti _ => vec![ top_join_plan.as_str(), + // Below 4 operators are differences introduced, when join mode is changed "SortPreservingRepartitionExec: partitioning=Hash([a@0], 10), input_partitions=10", "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", "SortExec: expr=[a@0 ASC]", @@ -3083,7 +3123,7 @@ mod tests { format!("SortMergeJoin: join_type={join_type}, on=[(b1@6, c@2)]"); let expected = match join_type { - // Should include 3 RepartitionExecs and 3 SortExecs + // Should include 6 RepartitionExecs(3 hash, 3 round-robin) and 3 SortExecs JoinType::Inner | JoinType::Right => vec![ top_join_plan.as_str(), join_plan.as_str(), @@ -3101,8 +3141,8 @@ mod tests { "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", ], - // Should include 4 RepartitionExecs and 4 SortExecs - _ => vec![ + // Should include 7 RepartitionExecs (4 hash, 3 round-robin) and 4 SortExecs + JoinType::Left | JoinType::Full => vec![ top_join_plan.as_str(), "SortExec: expr=[b1@6 ASC]", "RepartitionExec: partitioning=Hash([b1@6], 10), input_partitions=10", @@ -3121,6 +3161,8 @@ mod tests { "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", ], + // this match arm cannot be reached + _ => unreachable!() }; assert_optimized!(expected, top_join.clone(), true, true); @@ -3144,7 +3186,7 @@ mod tests { "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", ], // Should include 8 RepartitionExecs (4 of them preserves order) and 4 SortExecs - _ => vec![ + JoinType::Left | JoinType::Full => vec![ top_join_plan.as_str(), "SortPreservingRepartitionExec: partitioning=Hash([b1@6], 10), input_partitions=10", "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", @@ -3165,6 +3207,8 @@ mod tests { "SortExec: expr=[c@2 ASC]", "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", ], + // this match arm cannot be reached + _ => unreachable!() }; assert_optimized!( expected_first_sort_enforcement, @@ -3301,6 +3345,16 @@ mod tests { "ParquetExec: file_groups={2 groups: [[x], [y]]}, projection=[a, b, c, d, e], output_ordering=[a@0 ASC]", ]; assert_optimized!(expected, exec, true); + // In this case preserving ordering through order preserving operators is not desirable + // (according to flag: bounded_order_preserving_variants) + // hence in this case ordering lost during CoalescePartitionsExec and re-introduced with + // SortExec at the top. + let expected = &[ + "SortExec: expr=[a@0 ASC]", + "CoalescePartitionsExec", + "CoalesceBatchesExec: target_batch_size=4096", + "ParquetExec: file_groups={2 groups: [[x], [y]]}, projection=[a, b, c, d, e], output_ordering=[a@0 ASC]", + ]; assert_optimized!(expected, exec, false); Ok(()) } @@ -3435,7 +3489,7 @@ mod tests { sort_required_exec(filter_exec(sort_exec(sort_key, parquet_exec(), false))); let expected = &[ - "SortRequiredExec", + "SortRequiredExec: [c@2 ASC]", "FilterExec: c@2 = 0", // We can use repartition here, ordering requirement by SortRequiredExec // is still satisfied. @@ -3541,6 +3595,12 @@ mod tests { ]; assert_optimized!(expected, plan.clone(), true); + + let expected = &[ + "SortExec: expr=[c@2 ASC]", + "CoalescePartitionsExec", + "ParquetExec: file_groups={2 groups: [[x], [y]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC]", + ]; assert_optimized!(expected, plan, false); Ok(()) } @@ -3565,6 +3625,14 @@ mod tests { ]; assert_optimized!(expected, plan.clone(), true); + + let expected = &[ + "SortExec: expr=[c@2 ASC]", + "CoalescePartitionsExec", + "UnionExec", + "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC]", + "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC]", + ]; assert_optimized!(expected, plan, false); Ok(()) } @@ -3583,7 +3651,7 @@ mod tests { // during repartitioning ordering is preserved let expected = &[ - "SortRequiredExec", + "SortRequiredExec: [c@2 ASC]", "FilterExec: c@2 = 0", "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC]", @@ -3619,7 +3687,7 @@ mod tests { let expected = &[ "UnionExec", // union input 1: no repartitioning - "SortRequiredExec", + "SortRequiredExec: [c@2 ASC]", "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC]", // union input 2: should repartition "FilterExec: c@2 = 0", @@ -3687,16 +3755,13 @@ mod tests { ("c".to_string(), "c".to_string()), ]; // sorted input - let plan = sort_preserving_merge_exec( - sort_key.clone(), - projection_exec_with_alias( - parquet_exec_multiple_sorted(vec![sort_key]), - alias, - ), - ); + let plan = sort_required_exec(projection_exec_with_alias( + parquet_exec_multiple_sorted(vec![sort_key]), + alias, + )); let expected = &[ - "SortPreservingMergeExec: [c@2 ASC]", + "SortRequiredExec: [c@2 ASC]", // Since this projection is trivial, increasing parallelism is not beneficial "ProjectionExec: expr=[a@0 as a, b@1 as b, c@2 as c]", "ParquetExec: file_groups={2 groups: [[x], [y]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC]", @@ -4186,11 +4251,11 @@ mod tests { // no parallelization, because SortRequiredExec doesn't benefit from increased parallelism let expected_parquet = &[ - "SortRequiredExec", + "SortRequiredExec: [c@2 ASC]", "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC]", ]; let expected_csv = &[ - "SortRequiredExec", + "SortRequiredExec: [c@2 ASC]", "CsvExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC], has_header=false", ]; @@ -4348,6 +4413,14 @@ mod tests { ]; assert_optimized!(expected, physical_plan.clone(), true); + + let expected = &[ + "SortExec: expr=[c@2 ASC]", + "CoalescePartitionsExec", + "FilterExec: c@2 = 0", + "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=2", + "ParquetExec: file_groups={2 groups: [[x], [y]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC]", + ]; assert_optimized!(expected, physical_plan, false); Ok(()) @@ -4377,6 +4450,15 @@ mod tests { ]; assert_optimized!(expected, physical_plan.clone(), true); + + let expected = &[ + "SortExec: expr=[a@0 ASC]", + "CoalescePartitionsExec", + "SortExec: expr=[a@0 ASC]", + "FilterExec: c@2 = 0", + "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=2", + "ParquetExec: file_groups={2 groups: [[x], [y]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC]", + ]; assert_optimized!(expected, physical_plan, false); Ok(()) @@ -4404,6 +4486,86 @@ mod tests { Ok(()) } + #[test] + fn do_not_put_sort_when_input_is_invalid() -> Result<()> { + let schema = schema(); + let sort_key = vec![PhysicalSortExpr { + expr: col("a", &schema).unwrap(), + options: SortOptions::default(), + }]; + let input = parquet_exec(); + let physical_plan = sort_required_exec_with_req(filter_exec(input), sort_key); + let expected = &[ + // Ordering requirement of sort required exec is NOT satisfied + // by existing ordering at the source. + "SortRequiredExec: [a@0 ASC]", + "FilterExec: c@2 = 0", + "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", + ]; + assert_plan_txt!(expected, physical_plan); + + let expected = &[ + "SortRequiredExec: [a@0 ASC]", + // Since at the start of the rule ordering requirement is not satisfied + // EnforceDistribution rule doesn't satisfy this requirement either. + "FilterExec: c@2 = 0", + "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", + "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", + ]; + + let mut config = ConfigOptions::new(); + config.execution.target_partitions = 10; + config.optimizer.enable_round_robin_repartition = true; + config.optimizer.bounded_order_preserving_variants = false; + let distribution_plan = + EnforceDistribution::new().optimize(physical_plan, &config)?; + assert_plan_txt!(expected, distribution_plan); + + Ok(()) + } + + #[test] + fn put_sort_when_input_is_valid() -> Result<()> { + let schema = schema(); + let sort_key = vec![PhysicalSortExpr { + expr: col("a", &schema).unwrap(), + options: SortOptions::default(), + }]; + let input = parquet_exec_multiple_sorted(vec![sort_key.clone()]); + let physical_plan = sort_required_exec_with_req(filter_exec(input), sort_key); + + let expected = &[ + // Ordering requirement of sort required exec is satisfied + // by existing ordering at the source. + "SortRequiredExec: [a@0 ASC]", + "FilterExec: c@2 = 0", + "ParquetExec: file_groups={2 groups: [[x], [y]]}, projection=[a, b, c, d, e], output_ordering=[a@0 ASC]", + ]; + assert_plan_txt!(expected, physical_plan); + + let expected = &[ + "SortRequiredExec: [a@0 ASC]", + // Since at the start of the rule ordering requirement is satisfied + // EnforceDistribution rule satisfy this requirement also. + // ordering is re-satisfied by introduction of SortExec. + "SortExec: expr=[a@0 ASC]", + "FilterExec: c@2 = 0", + // ordering is lost here + "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=2", + "ParquetExec: file_groups={2 groups: [[x], [y]]}, projection=[a, b, c, d, e], output_ordering=[a@0 ASC]", + ]; + + let mut config = ConfigOptions::new(); + config.execution.target_partitions = 10; + config.optimizer.enable_round_robin_repartition = true; + config.optimizer.bounded_order_preserving_variants = false; + let distribution_plan = + EnforceDistribution::new().optimize(physical_plan, &config)?; + assert_plan_txt!(expected, distribution_plan); + + Ok(()) + } + #[test] fn do_not_add_unnecessary_hash() -> Result<()> { let schema = schema(); @@ -4458,4 +4620,51 @@ mod tests { Ok(()) } + + #[test] + fn optimize_away_unnecessary_repartition() -> Result<()> { + let physical_plan = coalesce_partitions_exec(repartition_exec(parquet_exec())); + let expected = &[ + "CoalescePartitionsExec", + " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", + " ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", + ]; + plans_matches_expected!(expected, physical_plan.clone()); + + let expected = + &["ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]"]; + + assert_optimized!(expected, physical_plan.clone(), true); + assert_optimized!(expected, physical_plan, false); + + Ok(()) + } + + #[test] + fn optimize_away_unnecessary_repartition2() -> Result<()> { + let physical_plan = filter_exec(repartition_exec(coalesce_partitions_exec( + filter_exec(repartition_exec(parquet_exec())), + ))); + let expected = &[ + "FilterExec: c@2 = 0", + " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", + " CoalescePartitionsExec", + " FilterExec: c@2 = 0", + " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", + " ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", + ]; + plans_matches_expected!(expected, physical_plan.clone()); + + let expected = &[ + "FilterExec: c@2 = 0", + "FilterExec: c@2 = 0", + "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", + "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", + ]; + + assert_optimized!(expected, physical_plan.clone(), true); + assert_optimized!(expected, physical_plan, false); + + Ok(()) + } } diff --git a/datafusion/core/src/physical_optimizer/enforce_sorting.rs b/datafusion/core/src/physical_optimizer/enforce_sorting.rs index a149330181d9..c4b72a7cb31e 100644 --- a/datafusion/core/src/physical_optimizer/enforce_sorting.rs +++ b/datafusion/core/src/physical_optimizer/enforce_sorting.rs @@ -2035,7 +2035,6 @@ mod tests { let orig_plan = Arc::new(SortExec::new(sort_exprs, repartition)) as Arc; let actual = get_plan_string(&orig_plan); - println!("{:?}", actual); let expected_input = vec![ "SortExec: expr=[nullable_col@0 ASC]", " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", diff --git a/datafusion/core/src/physical_optimizer/mod.rs b/datafusion/core/src/physical_optimizer/mod.rs index 0801a9bc595c..9e22bff340c9 100644 --- a/datafusion/core/src/physical_optimizer/mod.rs +++ b/datafusion/core/src/physical_optimizer/mod.rs @@ -28,6 +28,7 @@ pub mod enforce_distribution; pub mod enforce_sorting; pub mod join_selection; pub mod optimizer; +pub mod output_requirements; pub mod pipeline_checker; pub mod pruning; pub mod replace_with_order_preserving_variants; diff --git a/datafusion/core/src/physical_optimizer/optimizer.rs b/datafusion/core/src/physical_optimizer/optimizer.rs index 5de70efe3c47..95035e5f81a0 100644 --- a/datafusion/core/src/physical_optimizer/optimizer.rs +++ b/datafusion/core/src/physical_optimizer/optimizer.rs @@ -26,6 +26,7 @@ use crate::physical_optimizer::combine_partial_final_agg::CombinePartialFinalAgg use crate::physical_optimizer::enforce_distribution::EnforceDistribution; use crate::physical_optimizer::enforce_sorting::EnforceSorting; use crate::physical_optimizer::join_selection::JoinSelection; +use crate::physical_optimizer::output_requirements::OutputRequirements; use crate::physical_optimizer::pipeline_checker::PipelineChecker; use crate::physical_optimizer::topk_aggregation::TopKAggregation; use crate::{error::Result, physical_plan::ExecutionPlan}; @@ -68,6 +69,9 @@ impl PhysicalOptimizer { /// Create a new optimizer using the recommended list of rules pub fn new() -> Self { let rules: Vec> = vec![ + // If there is a output requirement of the query, make sure that + // this information is not lost across different rules during optimization. + Arc::new(OutputRequirements::new_add_mode()), Arc::new(AggregateStatistics::new()), // Statistics-based join selection will change the Auto mode to a real join implementation, // like collect left, or hash join, or future sort merge join, which will influence the @@ -90,6 +94,9 @@ impl PhysicalOptimizer { // The CoalesceBatches rule will not influence the distribution and ordering of the // whole plan tree. Therefore, to avoid influencing other rules, it should run last. Arc::new(CoalesceBatches::new()), + // Remove the ancillary output requirement operator since we are done with the planning + // phase. + Arc::new(OutputRequirements::new_remove_mode()), // The PipelineChecker rule will reject non-runnable query plans that use // pipeline-breaking operators on infinite input(s). The rule generates a // diagnostic error message when this happens. It makes no changes to the diff --git a/datafusion/core/src/physical_optimizer/output_requirements.rs b/datafusion/core/src/physical_optimizer/output_requirements.rs new file mode 100644 index 000000000000..4b687d7f3536 --- /dev/null +++ b/datafusion/core/src/physical_optimizer/output_requirements.rs @@ -0,0 +1,275 @@ +// 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. + +//! The GlobalOrderRequire optimizer rule either: +//! - Adds an auxiliary `OutputRequirementExec` operator to keep track of global +//! ordering and distribution requirement across rules, or +//! - Removes the auxiliary `OutputRequirementExec` operator from the physical plan. +//! Since the `OutputRequirementExec` operator is only a helper operator, it +//! shouldn't occur in the final plan (i.e. the executed plan). + +use std::sync::Arc; + +use crate::physical_optimizer::PhysicalOptimizerRule; +use crate::physical_plan::sorts::sort::SortExec; +use crate::physical_plan::{DisplayAs, DisplayFormatType, ExecutionPlan}; + +use arrow_schema::SchemaRef; +use datafusion_common::config::ConfigOptions; +use datafusion_common::tree_node::{Transformed, TreeNode}; +use datafusion_common::{Result, Statistics}; +use datafusion_physical_expr::{ + Distribution, LexOrderingReq, PhysicalSortExpr, PhysicalSortRequirement, +}; +use datafusion_physical_plan::sorts::sort_preserving_merge::SortPreservingMergeExec; + +/// This rule either adds or removes [`OutputRequirements`]s to/from the physical +/// plan according to its `mode` attribute, which is set by the constructors +/// `new_add_mode` and `new_remove_mode`. With this rule, we can keep track of +/// the global requirements (ordering and distribution) across rules. +/// +/// The primary usecase of this node and rule is to specify and preserve the desired output +/// ordering and distribution the entire plan. When sending to a single client, a single partition may +/// be desirable, but when sending to a multi-partitioned writer, keeping multiple partitions may be +/// better. +#[derive(Debug)] +pub struct OutputRequirements { + mode: RuleMode, +} + +impl OutputRequirements { + /// Create a new rule which works in `Add` mode; i.e. it simply adds a + /// top-level [`OutputRequirementExec`] into the physical plan to keep track + /// of global ordering and distribution requirements if there are any. + /// Note that this rule should run at the beginning. + pub fn new_add_mode() -> Self { + Self { + mode: RuleMode::Add, + } + } + + /// Create a new rule which works in `Remove` mode; i.e. it simply removes + /// the top-level [`OutputRequirementExec`] from the physical plan if there is + /// any. We do this because a `OutputRequirementExec` is an ancillary, + /// non-executable operator whose sole purpose is to track global + /// requirements during optimization. Therefore, a + /// `OutputRequirementExec` should not appear in the final plan. + pub fn new_remove_mode() -> Self { + Self { + mode: RuleMode::Remove, + } + } +} + +#[derive(Debug, Ord, PartialOrd, PartialEq, Eq, Hash)] +enum RuleMode { + Add, + Remove, +} + +/// An ancillary, non-executable operator whose sole purpose is to track global +/// requirements during optimization. It imposes +/// - the ordering requirement in its `order_requirement` attribute. +/// - the distribution requirement in its `dist_requirement` attribute. +/// +/// See [`OutputRequirements`] for more details +#[derive(Debug)] +struct OutputRequirementExec { + input: Arc, + order_requirement: Option, + dist_requirement: Distribution, +} + +impl OutputRequirementExec { + fn new( + input: Arc, + requirements: Option, + dist_requirement: Distribution, + ) -> Self { + Self { + input, + order_requirement: requirements, + dist_requirement, + } + } + + fn input(&self) -> Arc { + self.input.clone() + } +} + +impl DisplayAs for OutputRequirementExec { + fn fmt_as( + &self, + _t: DisplayFormatType, + f: &mut std::fmt::Formatter, + ) -> std::fmt::Result { + write!(f, "OutputRequirementExec") + } +} + +impl ExecutionPlan for OutputRequirementExec { + fn as_any(&self) -> &dyn std::any::Any { + self + } + + fn schema(&self) -> SchemaRef { + self.input.schema() + } + + fn output_partitioning(&self) -> crate::physical_plan::Partitioning { + self.input.output_partitioning() + } + + fn benefits_from_input_partitioning(&self) -> Vec { + vec![false] + } + + fn required_input_distribution(&self) -> Vec { + vec![self.dist_requirement.clone()] + } + + fn output_ordering(&self) -> Option<&[PhysicalSortExpr]> { + self.input.output_ordering() + } + + fn children(&self) -> Vec> { + vec![self.input.clone()] + } + + fn required_input_ordering(&self) -> Vec>> { + vec![self.order_requirement.clone()] + } + + fn unbounded_output(&self, children: &[bool]) -> Result { + // Has a single child + Ok(children[0]) + } + + fn with_new_children( + self: Arc, + mut children: Vec>, + ) -> Result> { + Ok(Arc::new(Self::new( + children.remove(0), // has a single child + self.order_requirement.clone(), + self.dist_requirement.clone(), + ))) + } + + fn execute( + &self, + _partition: usize, + _context: Arc, + ) -> Result { + unreachable!(); + } + + fn statistics(&self) -> Statistics { + self.input.statistics() + } +} + +impl PhysicalOptimizerRule for OutputRequirements { + fn optimize( + &self, + plan: Arc, + _config: &ConfigOptions, + ) -> Result> { + match self.mode { + RuleMode::Add => require_top_ordering(plan), + RuleMode::Remove => plan.transform_up(&|plan| { + if let Some(sort_req) = + plan.as_any().downcast_ref::() + { + Ok(Transformed::Yes(sort_req.input())) + } else { + Ok(Transformed::No(plan)) + } + }), + } + } + + fn name(&self) -> &str { + "OutputRequirements" + } + + fn schema_check(&self) -> bool { + true + } +} + +/// This functions adds ancillary `OutputRequirementExec` to the the physical plan, so that +/// global requirements are not lost during optimization. +fn require_top_ordering(plan: Arc) -> Result> { + let (new_plan, is_changed) = require_top_ordering_helper(plan)?; + if is_changed { + Ok(new_plan) + } else { + // Add `OutputRequirementExec` to the top, with no specified ordering and distribution requirement. + Ok(Arc::new(OutputRequirementExec::new( + new_plan, + // there is no ordering requirement + None, + Distribution::UnspecifiedDistribution, + )) as _) + } +} + +/// Helper function that adds an ancillary `OutputRequirementExec` to the given plan. +/// First entry in the tuple is resulting plan, second entry indicates whether any +/// `OutputRequirementExec` is added to the plan. +fn require_top_ordering_helper( + plan: Arc, +) -> Result<(Arc, bool)> { + let mut children = plan.children(); + // Global ordering defines desired ordering in the final result. + if children.len() != 1 { + Ok((plan, false)) + } else if let Some(sort_exec) = plan.as_any().downcast_ref::() { + let req_ordering = sort_exec.output_ordering().unwrap_or(&[]); + let req_dist = sort_exec.required_input_distribution()[0].clone(); + let reqs = PhysicalSortRequirement::from_sort_exprs(req_ordering); + Ok(( + Arc::new(OutputRequirementExec::new(plan, Some(reqs), req_dist)) as _, + true, + )) + } else if let Some(spm) = plan.as_any().downcast_ref::() { + let reqs = PhysicalSortRequirement::from_sort_exprs(spm.expr()); + Ok(( + Arc::new(OutputRequirementExec::new( + plan, + Some(reqs), + Distribution::SinglePartition, + )) as _, + true, + )) + } else if plan.maintains_input_order()[0] + && plan.required_input_ordering()[0].is_none() + { + // Keep searching for a `SortExec` as long as ordering is maintained, + // and on-the-way operators do not themselves require an ordering. + // When an operator requires an ordering, any `SortExec` below can not + // be responsible for (i.e. the originator of) the global ordering. + let (new_child, is_changed) = + require_top_ordering_helper(children.swap_remove(0))?; + Ok((plan.with_new_children(vec![new_child])?, is_changed)) + } else { + // Stop searching, there is no global ordering desired for the query. + Ok((plan, false)) + } +} diff --git a/datafusion/core/src/physical_optimizer/utils.rs b/datafusion/core/src/physical_optimizer/utils.rs index 21c976e07a15..0d6c85f9f22b 100644 --- a/datafusion/core/src/physical_optimizer/utils.rs +++ b/datafusion/core/src/physical_optimizer/utils.rs @@ -73,6 +73,30 @@ impl ExecTree { } } +/// Get `ExecTree` for each child of the plan if they are tracked. +/// # Arguments +/// +/// * `n_children` - Children count of the plan of interest +/// * `onward` - Contains `Some(ExecTree)` of the plan tracked. +/// - Contains `None` is plan is not tracked. +/// +/// # Returns +/// +/// A `Vec>` that contains tracking information of each child. +/// If a child is `None`, it is not tracked. If `Some(ExecTree)` child is tracked also. +pub(crate) fn get_children_exectrees( + n_children: usize, + onward: &Option, +) -> Vec> { + let mut children_onward = vec![None; n_children]; + if let Some(exec_tree) = &onward { + for child in &exec_tree.children { + children_onward[child.idx] = Some(child.clone()); + } + } + children_onward +} + /// This utility function adds a `SortExec` above an operator according to the /// given ordering requirements while preserving the original partitioning. pub fn add_sort_above( diff --git a/datafusion/core/tests/custom_sources_cases/provider_filter_pushdown.rs b/datafusion/core/tests/custom_sources_cases/provider_filter_pushdown.rs index 79214092fa57..73085937cbca 100644 --- a/datafusion/core/tests/custom_sources_cases/provider_filter_pushdown.rs +++ b/datafusion/core/tests/custom_sources_cases/provider_filter_pushdown.rs @@ -15,10 +15,12 @@ // specific language governing permissions and limitations // under the License. +use std::ops::Deref; +use std::sync::Arc; + use arrow::array::{Int32Builder, Int64Array}; use arrow::datatypes::{DataType, Field, Schema, SchemaRef}; use arrow::record_batch::RecordBatch; -use async_trait::async_trait; use datafusion::datasource::provider::{TableProvider, TableType}; use datafusion::error::Result; use datafusion::execution::context::{SessionContext, SessionState, TaskContext}; @@ -32,10 +34,10 @@ use datafusion::physical_plan::{ use datafusion::prelude::*; use datafusion::scalar::ScalarValue; use datafusion_common::cast::as_primitive_array; -use datafusion_common::{not_impl_err, DataFusionError}; +use datafusion_common::{internal_err, not_impl_err, DataFusionError}; use datafusion_expr::expr::{BinaryExpr, Cast}; -use std::ops::Deref; -use std::sync::Arc; + +use async_trait::async_trait; fn create_batch(value: i32, num_rows: usize) -> Result { let mut builder = Int32Builder::with_capacity(num_rows); @@ -96,9 +98,14 @@ impl ExecutionPlan for CustomPlan { fn with_new_children( self: Arc, - _: Vec>, + children: Vec>, ) -> Result> { - unreachable!() + // CustomPlan has no children + if children.is_empty() { + Ok(self) + } else { + internal_err!("Children cannot be replaced in {self:?}") + } } fn execute( diff --git a/datafusion/physical-plan/src/coalesce_partitions.rs b/datafusion/physical-plan/src/coalesce_partitions.rs index 8eddf57ae551..646d42795ba4 100644 --- a/datafusion/physical-plan/src/coalesce_partitions.rs +++ b/datafusion/physical-plan/src/coalesce_partitions.rs @@ -104,6 +104,10 @@ impl ExecutionPlan for CoalescePartitionsExec { self.input.equivalence_properties() } + fn benefits_from_input_partitioning(&self) -> Vec { + vec![false] + } + fn with_new_children( self: Arc, children: Vec>, diff --git a/datafusion/physical-plan/src/memory.rs b/datafusion/physical-plan/src/memory.rs index b29c8e9c7bd9..1dcdae56cfa3 100644 --- a/datafusion/physical-plan/src/memory.rs +++ b/datafusion/physical-plan/src/memory.rs @@ -126,9 +126,14 @@ impl ExecutionPlan for MemoryExec { fn with_new_children( self: Arc, - _: Vec>, + children: Vec>, ) -> Result> { - internal_err!("Children cannot be replaced in {self:?}") + // MemoryExec has no children + if children.is_empty() { + Ok(self) + } else { + internal_err!("Children cannot be replaced in {self:?}") + } } fn execute( diff --git a/datafusion/physical-plan/src/streaming.rs b/datafusion/physical-plan/src/streaming.rs index 00809b71e443..cb972fa41e3e 100644 --- a/datafusion/physical-plan/src/streaming.rs +++ b/datafusion/physical-plan/src/streaming.rs @@ -163,9 +163,13 @@ impl ExecutionPlan for StreamingTableExec { fn with_new_children( self: Arc, - _children: Vec>, + children: Vec>, ) -> Result> { - internal_err!("Children cannot be replaced in {self:?}") + if children.is_empty() { + Ok(self) + } else { + internal_err!("Children cannot be replaced in {self:?}") + } } fn execute( diff --git a/datafusion/sqllogictest/test_files/explain.slt b/datafusion/sqllogictest/test_files/explain.slt index b1ba1eb36d11..27ab8671e939 100644 --- a/datafusion/sqllogictest/test_files/explain.slt +++ b/datafusion/sqllogictest/test_files/explain.slt @@ -242,12 +242,16 @@ logical_plan after eliminate_projection SAME TEXT AS ABOVE logical_plan after push_down_limit SAME TEXT AS ABOVE logical_plan TableScan: simple_explain_test projection=[a, b, c] initial_physical_plan CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/example.csv]]}, projection=[a, b, c], has_header=true +physical_plan after OutputRequirements +OutputRequirementExec +--CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/example.csv]]}, projection=[a, b, c], has_header=true physical_plan after aggregate_statistics SAME TEXT AS ABOVE physical_plan after join_selection SAME TEXT AS ABOVE physical_plan after EnforceDistribution SAME TEXT AS ABOVE physical_plan after CombinePartialFinalAggregate SAME TEXT AS ABOVE physical_plan after EnforceSorting SAME TEXT AS ABOVE physical_plan after coalesce_batches SAME TEXT AS ABOVE +physical_plan after OutputRequirements CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/example.csv]]}, projection=[a, b, c], has_header=true physical_plan after PipelineChecker SAME TEXT AS ABOVE physical_plan after LimitAggregation SAME TEXT AS ABOVE physical_plan CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/example.csv]]}, projection=[a, b, c], has_header=true diff --git a/datafusion/sqllogictest/test_files/groupby.slt b/datafusion/sqllogictest/test_files/groupby.slt index ffef93837b27..5bb0f31ed542 100644 --- a/datafusion/sqllogictest/test_files/groupby.slt +++ b/datafusion/sqllogictest/test_files/groupby.slt @@ -2014,9 +2014,9 @@ Sort: l.col0 ASC NULLS LAST ----------TableScan: tab0 projection=[col0, col1] physical_plan SortPreservingMergeExec: [col0@0 ASC NULLS LAST] ---ProjectionExec: expr=[col0@0 as col0, LAST_VALUE(r.col1) ORDER BY [r.col0 ASC NULLS LAST]@3 as last_col1] -----AggregateExec: mode=FinalPartitioned, gby=[col0@0 as col0, col1@1 as col1, col2@2 as col2], aggr=[LAST_VALUE(r.col1)], ordering_mode=PartiallyOrdered -------SortExec: expr=[col0@0 ASC NULLS LAST] +--SortExec: expr=[col0@0 ASC NULLS LAST] +----ProjectionExec: expr=[col0@0 as col0, LAST_VALUE(r.col1) ORDER BY [r.col0 ASC NULLS LAST]@3 as last_col1] +------AggregateExec: mode=FinalPartitioned, gby=[col0@0 as col0, col1@1 as col1, col2@2 as col2], aggr=[LAST_VALUE(r.col1)] --------CoalesceBatchesExec: target_batch_size=8192 ----------RepartitionExec: partitioning=Hash([col0@0, col1@1, col2@2], 4), input_partitions=4 ------------AggregateExec: mode=Partial, gby=[col0@0 as col0, col1@1 as col1, col2@2 as col2], aggr=[LAST_VALUE(r.col1)], ordering_mode=PartiallyOrdered diff --git a/datafusion/sqllogictest/test_files/window.slt b/datafusion/sqllogictest/test_files/window.slt index 3d9f7511be26..b6325fd889ec 100644 --- a/datafusion/sqllogictest/test_files/window.slt +++ b/datafusion/sqllogictest/test_files/window.slt @@ -3197,6 +3197,72 @@ SELECT a_new, d, rn1 FROM (SELECT d, a as a_new, 0 0 4 0 1 5 +query TT +EXPLAIN SELECT SUM(a) OVER(partition by a, b order by c) as sum1, +SUM(a) OVER(partition by b, a order by c) as sum2, + SUM(a) OVER(partition by a, d order by b) as sum3, + SUM(a) OVER(partition by d order by a) as sum4 +FROM annotated_data_infinite2; +---- +logical_plan +Projection: SUM(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.a, annotated_data_infinite2.b] ORDER BY [annotated_data_infinite2.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW AS sum1, SUM(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.b, annotated_data_infinite2.a] ORDER BY [annotated_data_infinite2.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW AS sum2, SUM(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.a, annotated_data_infinite2.d] ORDER BY [annotated_data_infinite2.b ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW AS sum3, SUM(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.d] ORDER BY [annotated_data_infinite2.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW AS sum4 +--WindowAggr: windowExpr=[[SUM(CAST(annotated_data_infinite2.a AS Int64)) PARTITION BY [annotated_data_infinite2.d] ORDER BY [annotated_data_infinite2.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW]] +----Projection: annotated_data_infinite2.a, annotated_data_infinite2.d, SUM(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.a, annotated_data_infinite2.b] ORDER BY [annotated_data_infinite2.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW, SUM(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.a, annotated_data_infinite2.d] ORDER BY [annotated_data_infinite2.b ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW, SUM(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.b, annotated_data_infinite2.a] ORDER BY [annotated_data_infinite2.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW +------WindowAggr: windowExpr=[[SUM(CAST(annotated_data_infinite2.a AS Int64)) PARTITION BY [annotated_data_infinite2.b, annotated_data_infinite2.a] ORDER BY [annotated_data_infinite2.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW]] +--------WindowAggr: windowExpr=[[SUM(CAST(annotated_data_infinite2.a AS Int64)) PARTITION BY [annotated_data_infinite2.a, annotated_data_infinite2.d] ORDER BY [annotated_data_infinite2.b ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW]] +----------WindowAggr: windowExpr=[[SUM(CAST(annotated_data_infinite2.a AS Int64)) PARTITION BY [annotated_data_infinite2.a, annotated_data_infinite2.b] ORDER BY [annotated_data_infinite2.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW]] +------------TableScan: annotated_data_infinite2 projection=[a, b, c, d] +physical_plan +ProjectionExec: expr=[SUM(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.a, annotated_data_infinite2.b] ORDER BY [annotated_data_infinite2.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@2 as sum1, SUM(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.b, annotated_data_infinite2.a] ORDER BY [annotated_data_infinite2.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@4 as sum2, SUM(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.a, annotated_data_infinite2.d] ORDER BY [annotated_data_infinite2.b ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@3 as sum3, SUM(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.d] ORDER BY [annotated_data_infinite2.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@5 as sum4] +--BoundedWindowAggExec: wdw=[SUM(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.d] ORDER BY [annotated_data_infinite2.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "SUM(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.d] ORDER BY [annotated_data_infinite2.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(NULL)), end_bound: CurrentRow }], mode=[Linear] +----ProjectionExec: expr=[a@0 as a, d@3 as d, SUM(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.a, annotated_data_infinite2.b] ORDER BY [annotated_data_infinite2.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@4 as SUM(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.a, annotated_data_infinite2.b] ORDER BY [annotated_data_infinite2.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW, SUM(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.a, annotated_data_infinite2.d] ORDER BY [annotated_data_infinite2.b ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@5 as SUM(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.a, annotated_data_infinite2.d] ORDER BY [annotated_data_infinite2.b ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW, SUM(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.b, annotated_data_infinite2.a] ORDER BY [annotated_data_infinite2.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@6 as SUM(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.b, annotated_data_infinite2.a] ORDER BY [annotated_data_infinite2.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW] +------BoundedWindowAggExec: wdw=[SUM(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.b, annotated_data_infinite2.a] ORDER BY [annotated_data_infinite2.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "SUM(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.b, annotated_data_infinite2.a] ORDER BY [annotated_data_infinite2.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(NULL)), end_bound: CurrentRow }], mode=[Sorted] +--------BoundedWindowAggExec: wdw=[SUM(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.a, annotated_data_infinite2.d] ORDER BY [annotated_data_infinite2.b ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "SUM(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.a, annotated_data_infinite2.d] ORDER BY [annotated_data_infinite2.b ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(NULL)), end_bound: CurrentRow }], mode=[PartiallySorted([0])] +----------BoundedWindowAggExec: wdw=[SUM(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.a, annotated_data_infinite2.b] ORDER BY [annotated_data_infinite2.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "SUM(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.a, annotated_data_infinite2.b] ORDER BY [annotated_data_infinite2.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(NULL)), end_bound: CurrentRow }], mode=[Sorted] +------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, b, c, d], infinite_source=true, output_ordering=[a@0 ASC NULLS LAST, b@1 ASC NULLS LAST, c@2 ASC NULLS LAST], has_header=true + +statement ok +set datafusion.execution.target_partitions = 2; + +# re-execute the same query in multi partitions. +# final plan should still be streamable +query TT +EXPLAIN SELECT SUM(a) OVER(partition by a, b order by c) as sum1, + SUM(a) OVER(partition by b, a order by c) as sum2, + SUM(a) OVER(partition by a, d order by b) as sum3, + SUM(a) OVER(partition by d order by a) as sum4 +FROM annotated_data_infinite2; +---- +logical_plan +Projection: SUM(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.a, annotated_data_infinite2.b] ORDER BY [annotated_data_infinite2.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW AS sum1, SUM(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.b, annotated_data_infinite2.a] ORDER BY [annotated_data_infinite2.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW AS sum2, SUM(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.a, annotated_data_infinite2.d] ORDER BY [annotated_data_infinite2.b ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW AS sum3, SUM(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.d] ORDER BY [annotated_data_infinite2.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW AS sum4 +--WindowAggr: windowExpr=[[SUM(CAST(annotated_data_infinite2.a AS Int64)) PARTITION BY [annotated_data_infinite2.d] ORDER BY [annotated_data_infinite2.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW]] +----Projection: annotated_data_infinite2.a, annotated_data_infinite2.d, SUM(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.a, annotated_data_infinite2.b] ORDER BY [annotated_data_infinite2.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW, SUM(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.a, annotated_data_infinite2.d] ORDER BY [annotated_data_infinite2.b ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW, SUM(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.b, annotated_data_infinite2.a] ORDER BY [annotated_data_infinite2.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW +------WindowAggr: windowExpr=[[SUM(CAST(annotated_data_infinite2.a AS Int64)) PARTITION BY [annotated_data_infinite2.b, annotated_data_infinite2.a] ORDER BY [annotated_data_infinite2.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW]] +--------WindowAggr: windowExpr=[[SUM(CAST(annotated_data_infinite2.a AS Int64)) PARTITION BY [annotated_data_infinite2.a, annotated_data_infinite2.d] ORDER BY [annotated_data_infinite2.b ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW]] +----------WindowAggr: windowExpr=[[SUM(CAST(annotated_data_infinite2.a AS Int64)) PARTITION BY [annotated_data_infinite2.a, annotated_data_infinite2.b] ORDER BY [annotated_data_infinite2.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW]] +------------TableScan: annotated_data_infinite2 projection=[a, b, c, d] +physical_plan +ProjectionExec: expr=[SUM(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.a, annotated_data_infinite2.b] ORDER BY [annotated_data_infinite2.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@2 as sum1, SUM(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.b, annotated_data_infinite2.a] ORDER BY [annotated_data_infinite2.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@4 as sum2, SUM(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.a, annotated_data_infinite2.d] ORDER BY [annotated_data_infinite2.b ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@3 as sum3, SUM(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.d] ORDER BY [annotated_data_infinite2.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@5 as sum4] +--BoundedWindowAggExec: wdw=[SUM(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.d] ORDER BY [annotated_data_infinite2.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "SUM(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.d] ORDER BY [annotated_data_infinite2.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(NULL)), end_bound: CurrentRow }], mode=[Linear] +----CoalesceBatchesExec: target_batch_size=4096 +------SortPreservingRepartitionExec: partitioning=Hash([d@1], 2), input_partitions=2 +--------ProjectionExec: expr=[a@0 as a, d@3 as d, SUM(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.a, annotated_data_infinite2.b] ORDER BY [annotated_data_infinite2.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@4 as SUM(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.a, annotated_data_infinite2.b] ORDER BY [annotated_data_infinite2.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW, SUM(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.a, annotated_data_infinite2.d] ORDER BY [annotated_data_infinite2.b ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@5 as SUM(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.a, annotated_data_infinite2.d] ORDER BY [annotated_data_infinite2.b ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW, SUM(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.b, annotated_data_infinite2.a] ORDER BY [annotated_data_infinite2.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@6 as SUM(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.b, annotated_data_infinite2.a] ORDER BY [annotated_data_infinite2.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW] +----------BoundedWindowAggExec: wdw=[SUM(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.b, annotated_data_infinite2.a] ORDER BY [annotated_data_infinite2.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "SUM(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.b, annotated_data_infinite2.a] ORDER BY [annotated_data_infinite2.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(NULL)), end_bound: CurrentRow }], mode=[Sorted] +------------CoalesceBatchesExec: target_batch_size=4096 +--------------SortPreservingRepartitionExec: partitioning=Hash([b@1, a@0], 2), input_partitions=2 +----------------BoundedWindowAggExec: wdw=[SUM(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.a, annotated_data_infinite2.d] ORDER BY [annotated_data_infinite2.b ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "SUM(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.a, annotated_data_infinite2.d] ORDER BY [annotated_data_infinite2.b ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(NULL)), end_bound: CurrentRow }], mode=[PartiallySorted([0])] +------------------CoalesceBatchesExec: target_batch_size=4096 +--------------------SortPreservingRepartitionExec: partitioning=Hash([a@0, d@3], 2), input_partitions=2 +----------------------BoundedWindowAggExec: wdw=[SUM(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.a, annotated_data_infinite2.b] ORDER BY [annotated_data_infinite2.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "SUM(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.a, annotated_data_infinite2.b] ORDER BY [annotated_data_infinite2.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(NULL)), end_bound: CurrentRow }], mode=[Sorted] +------------------------CoalesceBatchesExec: target_batch_size=4096 +--------------------------SortPreservingRepartitionExec: partitioning=Hash([a@0, b@1], 2), input_partitions=2 +----------------------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, b, c, d], infinite_source=true, output_ordering=[a@0 ASC NULLS LAST, b@1 ASC NULLS LAST, c@2 ASC NULLS LAST], has_header=true + +# reset the partition number 1 again +statement ok +set datafusion.execution.target_partitions = 1; + statement ok drop table annotated_data_finite2 From 6ed2e84da8154931c393a9885b80c4f8a9d47ebb Mon Sep 17 00:00:00 2001 From: Trent Hauck Date: Fri, 29 Sep 2023 07:06:48 -0700 Subject: [PATCH 005/572] docs: add section on supports_filters_pushdown (#7680) * docs: add section on supports_filters_pushdown * fix: fixup code * Apply suggestions from code review Co-authored-by: Andrew Lamb * fix: apply prettier --------- Co-authored-by: Andrew Lamb --- .../library-user-guide/custom-table-providers.md | 16 ++++++++++++++++ 1 file changed, 16 insertions(+) diff --git a/docs/source/library-user-guide/custom-table-providers.md b/docs/source/library-user-guide/custom-table-providers.md index 30721d6a5ba6..2de4b1ba055b 100644 --- a/docs/source/library-user-guide/custom-table-providers.md +++ b/docs/source/library-user-guide/custom-table-providers.md @@ -121,6 +121,22 @@ impl TableProvider for CustomDataSource { With this, and the implementation of the omitted methods, we can now use the `CustomDataSource` as a `TableProvider` in DataFusion. +##### Additional `TableProvider` Methods + +`scan` has no default implementation, so it needed to be written. There are other methods on the `TableProvider` that have default implementations, but can be overridden if needed to provide additional functionality. + +###### `supports_filters_pushdown` + +The `supports_filters_pushdown` method can be overridden to indicate which filter expressions support being pushed down to the data source and within that the specificity of the pushdown. + +This returns a `Vec` of `TableProviderFilterPushDown` enums where each enum represents a filter that can be pushed down. The `TableProviderFilterPushDown` enum has three variants: + +- `TableProviderFilterPushDown::Unsupported` - the filter cannot be pushed down +- `TableProviderFilterPushDown::Exact` - the filter can be pushed down and the data source can guarantee that the filter will be applied completely to all rows. This is the highest performance option. +- `TableProviderFilterPushDown::Inexact` - the filter can be pushed down, but the data source cannot guarantee that the filter will be applied to all rows. DataFusion will apply `Inexact` filters again after the scan to ensure correctness. + +For filters that can be pushed down, they'll be passed to the `scan` method as the `filters` parameter and they can be made use of there. + ## Using the Custom Table Provider In order to use the custom table provider, we need to register it with DataFusion. This is done by creating a `TableProvider` and registering it with the `ExecutionContext`. From de1591770445773ca5947f44c8b5fe58fec985d7 Mon Sep 17 00:00:00 2001 From: Kousuke Saruta Date: Fri, 29 Sep 2023 23:30:27 +0900 Subject: [PATCH 006/572] Improve cache usage in CI (#7678) * Improve cache usage in CI * Trigger GA to check the effect of improvement --- .github/workflows/rust.yml | 77 ++++++++++---------------------------- 1 file changed, 20 insertions(+), 57 deletions(-) diff --git a/.github/workflows/rust.yml b/.github/workflows/rust.yml index fa5c56b43e03..80de6e663fcd 100644 --- a/.github/workflows/rust.yml +++ b/.github/workflows/rust.yml @@ -47,18 +47,24 @@ jobs: image: amd64/rust steps: - uses: actions/checkout@v4 - - name: Cache Cargo - uses: actions/cache@v3 - with: - # these represent dependencies downloaded by cargo - # and thus do not depend on the OS, arch nor rust version. - path: /github/home/.cargo - key: cargo-cache- - name: Setup Rust toolchain uses: ./.github/actions/setup-builder with: rust-version: stable + - name: Cache Cargo + uses: actions/cache@v3 + with: + path: | + ~/.cargo/bin/ + ~/.cargo/registry/index/ + ~/.cargo/registry/cache/ + ~/.cargo/git/db/ + ./target/ + ./datafusion-cli/target/ + # this key equals the ones on `linux-build-lib` for re-use + key: cargo-cache-benchmark-${{ hashFiles('datafusion/**/Cargo.toml', 'benchmarks/Cargo.toml', 'datafusion-cli/Cargo.toml') }} + - name: Check workspace without default features run: cargo check --no-default-features -p datafusion @@ -84,12 +90,6 @@ jobs: - uses: actions/checkout@v4 with: submodules: true - - name: Cache Cargo - uses: actions/cache@v3 - with: - path: /github/home/.cargo - # this key equals the ones on `linux-build-lib` for re-use - key: cargo-cache- - name: Setup Rust toolchain uses: ./.github/actions/setup-builder with: @@ -109,12 +109,6 @@ jobs: - uses: actions/checkout@v4 with: submodules: true - - name: Cache Cargo - uses: actions/cache@v3 - with: - path: /github/home/.cargo - # this key equals the ones on `linux-build-lib` for re-use - key: cargo-cache- - name: Setup Rust toolchain uses: ./.github/actions/setup-builder with: @@ -211,12 +205,6 @@ jobs: image: amd64/rust steps: - uses: actions/checkout@v4 - - name: Cache Cargo - uses: actions/cache@v3 - with: - path: /github/home/.cargo - # this key equals the ones on `linux-build-lib` for re-use - key: cargo-cache- - name: Setup Rust toolchain uses: ./.github/actions/setup-builder with: @@ -241,9 +229,14 @@ jobs: - name: Cache Cargo uses: actions/cache@v3 with: - path: /github/home/.cargo + path: | + ~/.cargo/bin/ + ~/.cargo/registry/index/ + ~/.cargo/registry/cache/ + ~/.cargo/git/db/ + ./target/ # this key equals the ones on `linux-build-lib` for re-use - key: cargo-cache- + key: cargo-cache-benchmark-${{ hashFiles('datafusion/**/Cargo.toml', 'benchmarks/Cargo.toml') }} - name: Setup Rust toolchain uses: ./.github/actions/setup-builder with: @@ -377,12 +370,6 @@ jobs: - uses: actions/checkout@v4 with: submodules: true - - name: Cache Cargo - uses: actions/cache@v3 - with: - path: /github/home/.cargo - # this key equals the ones on `linux-build-lib` for re-use - key: cargo-cache- - uses: actions/setup-python@v4 with: python-version: "3.8" @@ -480,12 +467,6 @@ jobs: - uses: actions/checkout@v4 with: submodules: true - - name: Cache Cargo - uses: actions/cache@v3 - with: - path: /github/home/.cargo - # this key equals the ones on `linux-build-lib` for re-use - key: cargo-cache- - name: Setup Rust toolchain uses: ./.github/actions/setup-builder with: @@ -506,12 +487,6 @@ jobs: - uses: actions/checkout@v4 with: submodules: true - - name: Cache Cargo - uses: actions/cache@v3 - with: - path: /github/home/.cargo - # this key equals the ones on `linux-build-lib` for re-use - key: cargo-cache- - name: Setup Rust toolchain uses: ./.github/actions/setup-builder with: @@ -531,12 +506,6 @@ jobs: - uses: actions/checkout@v4 with: submodules: true - - name: Cache Cargo - uses: actions/cache@v3 - with: - path: /github/home/.cargo - # this key equals the ones on `linux-build-lib` for re-use - key: cargo-cache- - name: Setup Rust toolchain uses: ./.github/actions/setup-builder with: @@ -563,12 +532,6 @@ jobs: - uses: actions/checkout@v4 with: submodules: true - - name: Cache Cargo - uses: actions/cache@v3 - with: - path: /github/home/.cargo - # this key equals the ones on `linux-build-lib` for re-use - key: cargo-cache- - name: Setup Rust toolchain uses: ./.github/actions/setup-builder with: From 70cded6e3d95036d4150d4c77b7e57caa90d7a22 Mon Sep 17 00:00:00 2001 From: Ruihang Xia Date: Sat, 30 Sep 2023 00:30:26 +0800 Subject: [PATCH 007/572] fix: substrait limit when fetch is None (#7669) * fix: substrait limit when fetch is None Signed-off-by: Ruihang Xia * Add comments --------- Signed-off-by: Ruihang Xia Co-authored-by: Andrew Lamb --- datafusion/physical-plan/src/limit.rs | 2 +- datafusion/substrait/src/logical_plan/consumer.rs | 9 +++++++-- datafusion/substrait/src/logical_plan/producer.rs | 3 ++- .../substrait/tests/cases/roundtrip_logical_plan.rs | 5 +++++ 4 files changed, 15 insertions(+), 4 deletions(-) diff --git a/datafusion/physical-plan/src/limit.rs b/datafusion/physical-plan/src/limit.rs index 922c3db0efc8..31ed08399c2e 100644 --- a/datafusion/physical-plan/src/limit.rs +++ b/datafusion/physical-plan/src/limit.rs @@ -442,7 +442,7 @@ impl LimitStream { match &poll { Poll::Ready(Some(Ok(batch))) => { - if batch.num_rows() > 0 && self.skip == 0 { + if batch.num_rows() > 0 { break poll; } else { // continue to poll input stream diff --git a/datafusion/substrait/src/logical_plan/consumer.rs b/datafusion/substrait/src/logical_plan/consumer.rs index 32b8f8ea547f..e1dde39427a5 100644 --- a/datafusion/substrait/src/logical_plan/consumer.rs +++ b/datafusion/substrait/src/logical_plan/consumer.rs @@ -227,8 +227,13 @@ pub async fn from_substrait_rel( from_substrait_rel(ctx, input, extensions).await?, ); let offset = fetch.offset as usize; - let count = fetch.count as usize; - input.limit(offset, Some(count))?.build() + // Since protobuf can't directly distinguish `None` vs `0` `None` is encoded as `MAX` + let count = if fetch.count as usize == usize::MAX { + None + } else { + Some(fetch.count as usize) + }; + input.limit(offset, count)?.build() } else { not_impl_err!("Fetch without an input is not valid") } diff --git a/datafusion/substrait/src/logical_plan/producer.rs b/datafusion/substrait/src/logical_plan/producer.rs index e17b022f3b53..1124ea53a557 100644 --- a/datafusion/substrait/src/logical_plan/producer.rs +++ b/datafusion/substrait/src/logical_plan/producer.rs @@ -193,7 +193,8 @@ pub fn to_substrait_rel( } LogicalPlan::Limit(limit) => { let input = to_substrait_rel(limit.input.as_ref(), ctx, extension_info)?; - let limit_fetch = limit.fetch.unwrap_or(0); + // Since protobuf can't directly distinguish `None` vs `0` encode `None` as `MAX` + let limit_fetch = limit.fetch.unwrap_or(usize::MAX); Ok(Box::new(Rel { rel_type: Some(RelType::Fetch(Box::new(FetchRel { common: None, diff --git a/datafusion/substrait/tests/cases/roundtrip_logical_plan.rs b/datafusion/substrait/tests/cases/roundtrip_logical_plan.rs index f4d74ae42681..2554d0667e48 100644 --- a/datafusion/substrait/tests/cases/roundtrip_logical_plan.rs +++ b/datafusion/substrait/tests/cases/roundtrip_logical_plan.rs @@ -188,6 +188,11 @@ async fn select_with_limit() -> Result<()> { roundtrip_fill_na("SELECT * FROM data LIMIT 100").await } +#[tokio::test] +async fn select_without_limit() -> Result<()> { + roundtrip_fill_na("SELECT * FROM data OFFSET 10").await +} + #[tokio::test] async fn select_with_limit_offset() -> Result<()> { roundtrip("SELECT * FROM data LIMIT 200 OFFSET 10").await From 2d6e768c6084c4955ac6dfb9c389aa8849464f05 Mon Sep 17 00:00:00 2001 From: Ruihang Xia Date: Sat, 30 Sep 2023 00:35:10 +0800 Subject: [PATCH 008/572] minor: revert parsing precedence between Aggr and UDAF (#7682) * minor: revert parsing precedence between Aggr and UDAF Signed-off-by: Ruihang Xia * add unit test Signed-off-by: Ruihang Xia --------- Signed-off-by: Ruihang Xia --- .../user_defined/user_defined_aggregates.rs | 37 +++++++++++++++++-- datafusion/sql/src/expr/function.rs | 18 ++++----- 2 files changed, 42 insertions(+), 13 deletions(-) diff --git a/datafusion/core/tests/user_defined/user_defined_aggregates.rs b/datafusion/core/tests/user_defined/user_defined_aggregates.rs index 64547bbdfa36..3b7b4d0e87b7 100644 --- a/datafusion/core/tests/user_defined/user_defined_aggregates.rs +++ b/datafusion/core/tests/user_defined/user_defined_aggregates.rs @@ -169,6 +169,37 @@ async fn test_udaf_returning_struct_subquery() { assert_batches_eq!(expected, &execute(&ctx, sql).await.unwrap()); } +#[tokio::test] +async fn test_udaf_shadows_builtin_fn() { + let TestContext { + mut ctx, + test_state, + } = TestContext::new(); + let sql = "SELECT sum(arrow_cast(time, 'Int64')) from t"; + + // compute with builtin `sum` aggregator + let expected = [ + "+-------------+", + "| SUM(t.time) |", + "+-------------+", + "| 19000 |", + "+-------------+", + ]; + assert_batches_eq!(expected, &execute(&ctx, sql).await.unwrap()); + + // Register `TimeSum` with name `sum`. This will shadow the builtin one + let sql = "SELECT sum(time) from t"; + TimeSum::register(&mut ctx, test_state.clone(), "sum"); + let expected = [ + "+----------------------------+", + "| sum(t.time) |", + "+----------------------------+", + "| 1970-01-01T00:00:00.000019 |", + "+----------------------------+", + ]; + assert_batches_eq!(expected, &execute(&ctx, sql).await.unwrap()); +} + async fn execute(ctx: &SessionContext, sql: &str) -> Result> { ctx.sql(sql).await?.collect().await } @@ -214,7 +245,7 @@ impl TestContext { // Tell DataFusion about the "first" function FirstSelector::register(&mut ctx); // Tell DataFusion about the "time_sum" function - TimeSum::register(&mut ctx, Arc::clone(&test_state)); + TimeSum::register(&mut ctx, Arc::clone(&test_state), "time_sum"); Self { ctx, test_state } } @@ -281,7 +312,7 @@ impl TimeSum { Self { sum: 0, test_state } } - fn register(ctx: &mut SessionContext, test_state: Arc) { + fn register(ctx: &mut SessionContext, test_state: Arc, name: &str) { let timestamp_type = DataType::Timestamp(TimeUnit::Nanosecond, None); // Returns the same type as its input @@ -301,8 +332,6 @@ impl TimeSum { let accumulator: AccumulatorFactoryFunction = Arc::new(move |_| Ok(Box::new(Self::new(Arc::clone(&captured_state))))); - let name = "time_sum"; - let time_sum = AggregateUDF::new(name, &signature, &return_type, &accumulator, &state_type); diff --git a/datafusion/sql/src/expr/function.rs b/datafusion/sql/src/expr/function.rs index 05f80fcfafa9..3861b4848d9b 100644 --- a/datafusion/sql/src/expr/function.rs +++ b/datafusion/sql/src/expr/function.rs @@ -124,6 +124,15 @@ impl<'a, S: ContextProvider> SqlToRel<'a, S> { return Ok(expr); } } else { + // User defined aggregate functions (UDAF) have precedence in case it has the same name as a scalar built-in function + if let Some(fm) = self.schema_provider.get_aggregate_meta(&name) { + let args = + self.function_args_to_expr(function.args, schema, planner_context)?; + return Ok(Expr::AggregateUDF(expr::AggregateUDF::new( + fm, args, None, None, + ))); + } + // next, aggregate built-ins if let Ok(fun) = AggregateFunction::from_str(&name) { let distinct = function.distinct; @@ -141,15 +150,6 @@ impl<'a, S: ContextProvider> SqlToRel<'a, S> { ))); }; - // User defined aggregate functions (UDAF) - if let Some(fm) = self.schema_provider.get_aggregate_meta(&name) { - let args = - self.function_args_to_expr(function.args, schema, planner_context)?; - return Ok(Expr::AggregateUDF(expr::AggregateUDF::new( - fm, args, None, None, - ))); - } - // Special case arrow_cast (as its type is dependent on its argument value) if name == ARROW_CAST_NAME { let args = From 85f3578f5fb47d28a8bc3a7b9be0284b3ced0fcd Mon Sep 17 00:00:00 2001 From: Jay Zhan Date: Sat, 30 Sep 2023 00:51:39 +0800 Subject: [PATCH 009/572] Minor: Move hash utils to common (#7684) * move hash utils to common Signed-off-by: jayzhan211 * support backward compatibility Signed-off-by: jayzhan211 --------- Signed-off-by: jayzhan211 Co-authored-by: Andrew Lamb --- datafusion-cli/Cargo.lock | 4 ++++ datafusion/common/Cargo.toml | 4 ++++ .../{physical-expr => common}/src/hash_utils.rs | 16 ++++++++-------- datafusion/common/src/lib.rs | 1 + .../physical-expr/src/expressions/in_list.rs | 2 +- datafusion/physical-expr/src/lib.rs | 4 +++- .../src/aggregates/group_values/row.rs | 2 +- datafusion/physical-plan/src/lib.rs | 3 ++- .../src/windows/bounded_window_agg_exec.rs | 3 ++- 9 files changed, 26 insertions(+), 13 deletions(-) rename datafusion/{physical-expr => common}/src/hash_utils.rs (98%) diff --git a/datafusion-cli/Cargo.lock b/datafusion-cli/Cargo.lock index 0ca83452bd02..775f8ec87e38 100644 --- a/datafusion-cli/Cargo.lock +++ b/datafusion-cli/Cargo.lock @@ -1122,9 +1122,13 @@ dependencies = [ name = "datafusion-common" version = "31.0.0" dependencies = [ + "ahash", "arrow", "arrow-array", + "arrow-buffer", + "arrow-schema", "chrono", + "half", "num_cpus", "object_store", "parquet", diff --git a/datafusion/common/Cargo.toml b/datafusion/common/Cargo.toml index f2b8f1a1e4be..b5cdec1be17b 100644 --- a/datafusion/common/Cargo.toml +++ b/datafusion/common/Cargo.toml @@ -39,10 +39,14 @@ default = ["parquet"] pyarrow = ["pyo3", "arrow/pyarrow"] [dependencies] +ahash = { version = "0.8", default-features = false, features = ["runtime-rng"] } apache-avro = { version = "0.16", default-features = false, features = ["snappy"], optional = true } arrow = { workspace = true } arrow-array = { workspace = true } +arrow-buffer = { workspace = true } +arrow-schema = { workspace = true } chrono = { workspace = true } +half = { version = "2.1", default-features = false } num_cpus = "1.13.0" object_store = { version = "0.7.0", default-features = false, optional = true } parquet = { workspace = true, optional = true } diff --git a/datafusion/physical-expr/src/hash_utils.rs b/datafusion/common/src/hash_utils.rs similarity index 98% rename from datafusion/physical-expr/src/hash_utils.rs rename to datafusion/common/src/hash_utils.rs index 379e0eba5277..9198461e00bf 100644 --- a/datafusion/physical-expr/src/hash_utils.rs +++ b/datafusion/common/src/hash_utils.rs @@ -17,19 +17,19 @@ //! Functionality used both on logical and physical plans +use std::sync::Arc; + use ahash::RandomState; use arrow::array::*; use arrow::datatypes::*; use arrow::row::Rows; use arrow::{downcast_dictionary_array, downcast_primitive_array}; use arrow_buffer::i256; -use datafusion_common::{ - cast::{ - as_boolean_array, as_generic_binary_array, as_primitive_array, as_string_array, - }, - internal_err, DataFusionError, Result, + +use crate::cast::{ + as_boolean_array, as_generic_binary_array, as_primitive_array, as_string_array, }; -use std::sync::Arc; +use crate::error::{DataFusionError, Result, _internal_err}; // Combines two hashes into one hash #[inline] @@ -51,7 +51,7 @@ fn hash_null(random_state: &RandomState, hashes_buffer: &'_ mut [u64], mul_col: } } -pub(crate) trait HashValue { +pub trait HashValue { fn hash_one(&self, state: &RandomState) -> u64; } @@ -337,7 +337,7 @@ pub fn create_hashes<'a>( } _ => { // This is internal because we should have caught this before. - return internal_err!( + return _internal_err!( "Unsupported data type in hasher: {}", col.data_type() ); diff --git a/datafusion/common/src/lib.rs b/datafusion/common/src/lib.rs index eeb5b2681370..71782f67046d 100644 --- a/datafusion/common/src/lib.rs +++ b/datafusion/common/src/lib.rs @@ -25,6 +25,7 @@ mod error; pub mod file_options; pub mod format; mod functional_dependencies; +pub mod hash_utils; mod join_type; pub mod parsers; #[cfg(feature = "pyarrow")] diff --git a/datafusion/physical-expr/src/expressions/in_list.rs b/datafusion/physical-expr/src/expressions/in_list.rs index c92bbbb74f16..bdc476f5b3a1 100644 --- a/datafusion/physical-expr/src/expressions/in_list.rs +++ b/datafusion/physical-expr/src/expressions/in_list.rs @@ -24,7 +24,6 @@ use std::fmt::Debug; use std::hash::{Hash, Hasher}; use std::sync::Arc; -use crate::hash_utils::HashValue; use crate::physical_expr::down_cast_any_ref; use crate::utils::expr_list_eq_any_order; use crate::PhysicalExpr; @@ -37,6 +36,7 @@ use arrow::datatypes::*; use arrow::record_batch::RecordBatch; use arrow::util::bit_iterator::BitIndexIterator; use arrow::{downcast_dictionary_array, downcast_primitive_array}; +use datafusion_common::hash_utils::HashValue; use datafusion_common::{ cast::{as_boolean_array, as_generic_binary_array, as_string_array}, internal_err, not_impl_err, DataFusionError, Result, ScalarValue, diff --git a/datafusion/physical-expr/src/lib.rs b/datafusion/physical-expr/src/lib.rs index e83dee2e6c80..48d5f4e1308b 100644 --- a/datafusion/physical-expr/src/lib.rs +++ b/datafusion/physical-expr/src/lib.rs @@ -28,7 +28,6 @@ pub mod equivalence; pub mod execution_props; pub mod expressions; pub mod functions; -pub mod hash_utils; pub mod intervals; pub mod math_expressions; mod partitioning; @@ -49,6 +48,9 @@ pub mod utils; pub mod var_provider; pub mod window; +// For backwards compatibility +pub use datafusion_common::hash_utils; + pub use aggregate::groups_accumulator::{ EmitTo, GroupsAccumulator, GroupsAccumulatorAdapter, }; diff --git a/datafusion/physical-plan/src/aggregates/group_values/row.rs b/datafusion/physical-plan/src/aggregates/group_values/row.rs index 746537557d46..10ff9edb8912 100644 --- a/datafusion/physical-plan/src/aggregates/group_values/row.rs +++ b/datafusion/physical-plan/src/aggregates/group_values/row.rs @@ -22,9 +22,9 @@ use arrow::record_batch::RecordBatch; use arrow::row::{RowConverter, Rows, SortField}; use arrow_array::{Array, ArrayRef}; use arrow_schema::{DataType, SchemaRef}; +use datafusion_common::hash_utils::create_hashes; use datafusion_common::{DataFusionError, Result}; use datafusion_execution::memory_pool::proxy::{RawTableAllocExt, VecAllocExt}; -use datafusion_physical_expr::hash_utils::create_hashes; use datafusion_physical_expr::EmitTo; use hashbrown::raw::RawTable; diff --git a/datafusion/physical-plan/src/lib.rs b/datafusion/physical-plan/src/lib.rs index 76adf7611d6f..aca10893db3d 100644 --- a/datafusion/physical-plan/src/lib.rs +++ b/datafusion/physical-plan/src/lib.rs @@ -375,10 +375,11 @@ pub mod windows; use crate::repartition::RepartitionExec; use crate::sorts::sort_preserving_merge::SortPreservingMergeExec; +pub use datafusion_common::hash_utils; pub use datafusion_common::utils::project_schema; use datafusion_execution::TaskContext; pub use datafusion_physical_expr::{ - expressions, functions, hash_utils, ordering_equivalence_properties_helper, udf, + expressions, functions, ordering_equivalence_properties_helper, udf, }; #[cfg(test)] diff --git a/datafusion/physical-plan/src/windows/bounded_window_agg_exec.rs b/datafusion/physical-plan/src/windows/bounded_window_agg_exec.rs index 4108b4220599..dfef0ddefa03 100644 --- a/datafusion/physical-plan/src/windows/bounded_window_agg_exec.rs +++ b/datafusion/physical-plan/src/windows/bounded_window_agg_exec.rs @@ -43,6 +43,8 @@ use arrow::{ datatypes::{Schema, SchemaBuilder, SchemaRef}, record_batch::RecordBatch, }; + +use datafusion_common::hash_utils::create_hashes; use datafusion_common::utils::{ evaluate_partition_ranges, get_arrayref_at_indices, get_at_indices, get_record_batch_at_indices, get_row_at_idx, @@ -51,7 +53,6 @@ use datafusion_common::{exec_err, plan_err, DataFusionError, Result}; use datafusion_execution::TaskContext; use datafusion_expr::window_state::{PartitionBatchState, WindowAggState}; use datafusion_expr::ColumnarValue; -use datafusion_physical_expr::hash_utils::create_hashes; use datafusion_physical_expr::window::{ PartitionBatches, PartitionKey, PartitionWindowAggStates, WindowState, }; From 692ea24357d32b1242c476f0ed33498c815ac921 Mon Sep 17 00:00:00 2001 From: Devin D'Angelo Date: Sat, 30 Sep 2023 01:22:52 -0400 Subject: [PATCH 010/572] Update Default Parquet Write Compression (#7692) * update compression default * fix tests --------- Co-authored-by: Andrew Lamb --- datafusion/common/src/config.rs | 2 +- datafusion/sqllogictest/test_files/information_schema.slt | 2 +- docs/source/user-guide/configs.md | 2 +- 3 files changed, 3 insertions(+), 3 deletions(-) diff --git a/datafusion/common/src/config.rs b/datafusion/common/src/config.rs index b34c64ff8893..261c2bf435a4 100644 --- a/datafusion/common/src/config.rs +++ b/datafusion/common/src/config.rs @@ -307,7 +307,7 @@ config_namespace! { /// lzo, brotli(level), lz4, zstd(level), and lz4_raw. /// These values are not case sensitive. If NULL, uses /// default parquet writer setting - pub compression: Option, default = None + pub compression: Option, default = Some("zstd(3)".into()) /// Sets if dictionary encoding is enabled. If NULL, uses /// default parquet writer setting diff --git a/datafusion/sqllogictest/test_files/information_schema.slt b/datafusion/sqllogictest/test_files/information_schema.slt index f90901021637..12aa9089a0c9 100644 --- a/datafusion/sqllogictest/test_files/information_schema.slt +++ b/datafusion/sqllogictest/test_files/information_schema.slt @@ -156,7 +156,7 @@ datafusion.execution.parquet.bloom_filter_enabled false datafusion.execution.parquet.bloom_filter_fpp NULL datafusion.execution.parquet.bloom_filter_ndv NULL datafusion.execution.parquet.column_index_truncate_length NULL -datafusion.execution.parquet.compression NULL +datafusion.execution.parquet.compression zstd(3) datafusion.execution.parquet.created_by datafusion datafusion.execution.parquet.data_page_row_count_limit 18446744073709551615 datafusion.execution.parquet.data_pagesize_limit 1048576 diff --git a/docs/source/user-guide/configs.md b/docs/source/user-guide/configs.md index 7fe229b4d3c6..638ac5a36b83 100644 --- a/docs/source/user-guide/configs.md +++ b/docs/source/user-guide/configs.md @@ -58,7 +58,7 @@ Environment variables are read during `SessionConfig` initialisation so they mus | datafusion.execution.parquet.data_pagesize_limit | 1048576 | Sets best effort maximum size of data page in bytes | | datafusion.execution.parquet.write_batch_size | 1024 | Sets write_batch_size in bytes | | datafusion.execution.parquet.writer_version | 1.0 | Sets parquet writer version valid values are "1.0" and "2.0" | -| datafusion.execution.parquet.compression | NULL | Sets default parquet compression codec Valid values are: uncompressed, snappy, gzip(level), lzo, brotli(level), lz4, zstd(level), and lz4_raw. These values are not case sensitive. If NULL, uses default parquet writer setting | +| datafusion.execution.parquet.compression | zstd(3) | Sets default parquet compression codec Valid values are: uncompressed, snappy, gzip(level), lzo, brotli(level), lz4, zstd(level), and lz4_raw. These values are not case sensitive. If NULL, uses default parquet writer setting | | datafusion.execution.parquet.dictionary_enabled | NULL | Sets if dictionary encoding is enabled. If NULL, uses default parquet writer setting | | datafusion.execution.parquet.dictionary_page_size_limit | 1048576 | Sets best effort maximum dictionary page size, in bytes | | datafusion.execution.parquet.statistics_enabled | NULL | Sets if statistics are enabled for any column Valid values are: "none", "chunk", and "page" These values are not case sensitive. If NULL, uses default parquet writer setting | From 5d522fb55628de063ea59ee738aa341cb73b71ff Mon Sep 17 00:00:00 2001 From: Kousuke Saruta Date: Sun, 1 Oct 2023 19:33:25 +0900 Subject: [PATCH 011/572] Stop using cache for the benchmark job (#7706) --- .github/workflows/rust.yml | 11 ----------- 1 file changed, 11 deletions(-) diff --git a/.github/workflows/rust.yml b/.github/workflows/rust.yml index 80de6e663fcd..37533396e0d8 100644 --- a/.github/workflows/rust.yml +++ b/.github/workflows/rust.yml @@ -226,17 +226,6 @@ jobs: - uses: actions/checkout@v4 with: submodules: true - - name: Cache Cargo - uses: actions/cache@v3 - with: - path: | - ~/.cargo/bin/ - ~/.cargo/registry/index/ - ~/.cargo/registry/cache/ - ~/.cargo/git/db/ - ./target/ - # this key equals the ones on `linux-build-lib` for re-use - key: cargo-cache-benchmark-${{ hashFiles('datafusion/**/Cargo.toml', 'benchmarks/Cargo.toml') }} - name: Setup Rust toolchain uses: ./.github/actions/setup-builder with: From 14cdf72099bca3bb82c0103a5586b89ddb2feecc Mon Sep 17 00:00:00 2001 From: Kousuke Saruta Date: Sun, 1 Oct 2023 19:46:38 +0900 Subject: [PATCH 012/572] Change rust.yml to run benchmark (#7708) * Change rust.yml to run benchmark * Restore unrelated change --- .github/workflows/rust.yml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/.github/workflows/rust.yml b/.github/workflows/rust.yml index 37533396e0d8..0f572c9687dd 100644 --- a/.github/workflows/rust.yml +++ b/.github/workflows/rust.yml @@ -241,7 +241,7 @@ jobs: - name: Verify that benchmark queries return expected results run: | export TPCH_DATA=`realpath datafusion/sqllogictest/test_files/tpch/data` - cargo test serde_q --profile release-nonlto --features=ci -- --test-threads=1 + cargo test plan_q --package datafusion-benchmarks --profile release-nonlto --features=ci -- --test-threads=1 INCLUDE_TPCH=true cargo test --test sqllogictests - name: Verify Working Directory Clean run: git diff --exit-code From f9591275c1eb9d4df30f0ce0561992672f34af91 Mon Sep 17 00:00:00 2001 From: andrelmartins Date: Sun, 1 Oct 2023 11:59:52 +0100 Subject: [PATCH 013/572] Extend infer_placeholder_types to support BETWEEN predicates (#7703) --- datafusion/sql/src/expr/mod.rs | 10 ++++++++ datafusion/sql/tests/sql_integration.rs | 34 +++++++++++++++++++++++++ 2 files changed, 44 insertions(+) diff --git a/datafusion/sql/src/expr/mod.rs b/datafusion/sql/src/expr/mod.rs index be6cee9885aa..2f6266f29d6a 100644 --- a/datafusion/sql/src/expr/mod.rs +++ b/datafusion/sql/src/expr/mod.rs @@ -741,6 +741,16 @@ fn infer_placeholder_types(expr: Expr, schema: &DFSchema) -> Result { rewrite_placeholder(left.as_mut(), right.as_ref(), schema)?; rewrite_placeholder(right.as_mut(), left.as_ref(), schema)?; }; + if let Expr::Between(Between { + expr, + negated: _, + low, + high, + }) = &mut expr + { + rewrite_placeholder(low.as_mut(), expr.as_ref(), schema)?; + rewrite_placeholder(high.as_mut(), expr.as_ref(), schema)?; + } Ok(Transformed::Yes(expr)) }) } diff --git a/datafusion/sql/tests/sql_integration.rs b/datafusion/sql/tests/sql_integration.rs index 88a041a66145..f4ffea06b7a1 100644 --- a/datafusion/sql/tests/sql_integration.rs +++ b/datafusion/sql/tests/sql_integration.rs @@ -3894,6 +3894,40 @@ Projection: person.id, person.age prepare_stmt_replace_params_quick_test(plan, param_values, expected_plan); } +#[test] +fn test_prepare_statement_infer_types_from_between_predicate() { + let sql = "SELECT id, age FROM person WHERE age BETWEEN $1 AND $2"; + + let expected_plan = r#" +Projection: person.id, person.age + Filter: person.age BETWEEN $1 AND $2 + TableScan: person + "# + .trim(); + + let expected_dt = "[Int32]"; + let plan = prepare_stmt_quick_test(sql, expected_plan, expected_dt); + + let actual_types = plan.get_parameter_types().unwrap(); + let expected_types = HashMap::from([ + ("$1".to_string(), Some(DataType::Int32)), + ("$2".to_string(), Some(DataType::Int32)), + ]); + assert_eq!(actual_types, expected_types); + + // replace params with values + let param_values = vec![ScalarValue::Int32(Some(10)), ScalarValue::Int32(Some(30))]; + let expected_plan = r#" +Projection: person.id, person.age + Filter: person.age BETWEEN Int32(10) AND Int32(30) + TableScan: person + "# + .trim(); + let plan = plan.replace_params_with_values(¶m_values).unwrap(); + + prepare_stmt_replace_params_quick_test(plan, param_values, expected_plan); +} + #[test] fn test_prepare_statement_infer_types_subquery() { let sql = "SELECT id, age FROM person WHERE age = (select max(age) from person where id = $1)"; From 2ab0c0005f5aa9c835a0d7173bf7b6fec6ad191b Mon Sep 17 00:00:00 2001 From: Andrew Lamb Date: Mon, 2 Oct 2023 07:00:43 -0400 Subject: [PATCH 014/572] Minor: Add comment explaining why verify benchmark results uses release mode (#7712) While looking into https://github.com/apache/arrow-datafusion/issues/7709 I discovered why this test is run in release mode, and I wanted to leave a note for my future self --- .github/workflows/rust.yml | 1 + 1 file changed, 1 insertion(+) diff --git a/.github/workflows/rust.yml b/.github/workflows/rust.yml index 0f572c9687dd..27fdc9e37037 100644 --- a/.github/workflows/rust.yml +++ b/.github/workflows/rust.yml @@ -241,6 +241,7 @@ jobs: - name: Verify that benchmark queries return expected results run: | export TPCH_DATA=`realpath datafusion/sqllogictest/test_files/tpch/data` + # use release build for plan verificaton because debug build causes stack overflow cargo test plan_q --package datafusion-benchmarks --profile release-nonlto --features=ci -- --test-threads=1 INCLUDE_TPCH=true cargo test --test sqllogictests - name: Verify Working Directory Clean From e97ed6650eea65e2baa2239dfdc35e282846e132 Mon Sep 17 00:00:00 2001 From: Kousuke Saruta Date: Mon, 2 Oct 2023 21:14:49 +0900 Subject: [PATCH 015/572] Support all the codecs supported by Avro (#7718) * Support all the codes supported by Avro * Update testing dir --- datafusion/common/Cargo.toml | 2 +- datafusion/sqllogictest/test_files/avro.slt | 124 ++++++++++++++++++++ testing | 2 +- 3 files changed, 126 insertions(+), 2 deletions(-) diff --git a/datafusion/common/Cargo.toml b/datafusion/common/Cargo.toml index b5cdec1be17b..733aba1e1da1 100644 --- a/datafusion/common/Cargo.toml +++ b/datafusion/common/Cargo.toml @@ -40,7 +40,7 @@ pyarrow = ["pyo3", "arrow/pyarrow"] [dependencies] ahash = { version = "0.8", default-features = false, features = ["runtime-rng"] } -apache-avro = { version = "0.16", default-features = false, features = ["snappy"], optional = true } +apache-avro = { version = "0.16", default-features = false, features = ["bzip", "snappy", "xz", "zstandard"], optional = true } arrow = { workspace = true } arrow-array = { workspace = true } arrow-buffer = { workspace = true } diff --git a/datafusion/sqllogictest/test_files/avro.slt b/datafusion/sqllogictest/test_files/avro.slt index ede11406e1a9..3309cd1cf642 100644 --- a/datafusion/sqllogictest/test_files/avro.slt +++ b/datafusion/sqllogictest/test_files/avro.slt @@ -34,6 +34,78 @@ STORED AS AVRO WITH HEADER ROW LOCATION '../../testing/data/avro/alltypes_plain.avro' +statement ok +CREATE EXTERNAL TABLE alltypes_plain_snappy ( + id INT NOT NULL, + bool_col BOOLEAN NOT NULL, + tinyint_col TINYINT NOT NULL, + smallint_col SMALLINT NOT NULL, + int_col INT NOT NULL, + bigint_col BIGINT NOT NULL, + float_col FLOAT NOT NULL, + double_col DOUBLE NOT NULL, + date_string_col BYTEA NOT NULL, + string_col VARCHAR NOT NULL, + timestamp_col TIMESTAMP NOT NULL, +) +STORED AS AVRO +WITH HEADER ROW +LOCATION '../../testing/data/avro/alltypes_plain.snappy.avro' + +statement ok +CREATE EXTERNAL TABLE alltypes_plain_bzip2 ( + id INT NOT NULL, + bool_col BOOLEAN NOT NULL, + tinyint_col TINYINT NOT NULL, + smallint_col SMALLINT NOT NULL, + int_col INT NOT NULL, + bigint_col BIGINT NOT NULL, + float_col FLOAT NOT NULL, + double_col DOUBLE NOT NULL, + date_string_col BYTEA NOT NULL, + string_col VARCHAR NOT NULL, + timestamp_col TIMESTAMP NOT NULL, +) +STORED AS AVRO +WITH HEADER ROW +LOCATION '../../testing/data/avro/alltypes_plain.bzip2.avro' + +statement ok +CREATE EXTERNAL TABLE alltypes_plain_xz ( + id INT NOT NULL, + bool_col BOOLEAN NOT NULL, + tinyint_col TINYINT NOT NULL, + smallint_col SMALLINT NOT NULL, + int_col INT NOT NULL, + bigint_col BIGINT NOT NULL, + float_col FLOAT NOT NULL, + double_col DOUBLE NOT NULL, + date_string_col BYTEA NOT NULL, + string_col VARCHAR NOT NULL, + timestamp_col TIMESTAMP NOT NULL, +) +STORED AS AVRO +WITH HEADER ROW +LOCATION '../../testing/data/avro/alltypes_plain.xz.avro' + +statement ok +CREATE EXTERNAL TABLE alltypes_plain_zstandard ( + id INT NOT NULL, + bool_col BOOLEAN NOT NULL, + tinyint_col TINYINT NOT NULL, + smallint_col SMALLINT NOT NULL, + int_col INT NOT NULL, + bigint_col BIGINT NOT NULL, + float_col FLOAT NOT NULL, + double_col DOUBLE NOT NULL, + date_string_col BYTEA NOT NULL, + string_col VARCHAR NOT NULL, + timestamp_col TIMESTAMP NOT NULL, +) +STORED AS AVRO +WITH HEADER ROW +LOCATION '../../testing/data/avro/alltypes_plain.zstandard.avro' + statement ok CREATE EXTERNAL TABLE single_nan ( mycol FLOAT @@ -73,6 +145,58 @@ SELECT id, CAST(string_col AS varchar) FROM alltypes_plain 0 0 1 1 +# test avro query with snappy +query IT +SELECT id, CAST(string_col AS varchar) FROM alltypes_plain_snappy +---- +4 0 +5 1 +6 0 +7 1 +2 0 +3 1 +0 0 +1 1 + +# test avro query with bzip2 +query IT +SELECT id, CAST(string_col AS varchar) FROM alltypes_plain_bzip2 +---- +4 0 +5 1 +6 0 +7 1 +2 0 +3 1 +0 0 +1 1 + +# test avro query with xz +query IT +SELECT id, CAST(string_col AS varchar) FROM alltypes_plain_xz +---- +4 0 +5 1 +6 0 +7 1 +2 0 +3 1 +0 0 +1 1 + +# test avro query with zstandard +query IT +SELECT id, CAST(string_col AS varchar) FROM alltypes_plain_zstandard +---- +4 0 +5 1 +6 0 +7 1 +2 0 +3 1 +0 0 +1 1 + # test avro single nan schema query R SELECT mycol FROM single_nan diff --git a/testing b/testing index 37f29510ce97..98fceecd024d 160000 --- a/testing +++ b/testing @@ -1 +1 @@ -Subproject commit 37f29510ce97cd491b8e6ed75866c6533a5ea2a1 +Subproject commit 98fceecd024dccd2f8a00e32fc144975f218acf4 From 642e5a45752bb7889504a9fe39b461398901fb84 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Mon, 2 Oct 2023 12:19:24 -0400 Subject: [PATCH 016/572] Update substrait requirement from 0.14.0 to 0.15.0 (#7719) Updates the requirements on [substrait](https://github.com/substrait-io/substrait-rs) to permit the latest version. - [Release notes](https://github.com/substrait-io/substrait-rs/releases) - [Changelog](https://github.com/substrait-io/substrait-rs/blob/main/CHANGELOG.md) - [Commits](https://github.com/substrait-io/substrait-rs/compare/v0.14.0...v0.15.0) --- updated-dependencies: - dependency-name: substrait dependency-type: direct:production ... Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- datafusion/substrait/Cargo.toml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/datafusion/substrait/Cargo.toml b/datafusion/substrait/Cargo.toml index 2f7816c6488a..a9e2209404be 100644 --- a/datafusion/substrait/Cargo.toml +++ b/datafusion/substrait/Cargo.toml @@ -35,7 +35,7 @@ itertools = "0.11" object_store = "0.7.0" prost = "0.11" prost-types = "0.11" -substrait = "0.14.0" +substrait = "0.15.0" tokio = "1.17" [features] From 9225ce84b14a79d99624e43c182a4c519b0ad13b Mon Sep 17 00:00:00 2001 From: Martin Hilton Date: Mon, 2 Oct 2023 17:28:57 +0100 Subject: [PATCH 017/572] fix: coerce text to timestamps with timezones (#7720) * fix: coerce text to timestamps with timezones Extend the type coercion logic used with function calls to enable text respresentaions of timestamps to be coerced to a timestamp type with a timezone. If there is no other suitable choice the timezone will be "+00" (UTC). The specific use case for this change is to allow the third (offset) parameter of the date_bin function to be specified as a constant srting when the input array for the second parameter has a specitied time zone. * review comments Create a named constant for the placeholder timezone as suggested in the review. --- datafusion/expr/src/built_in_function.rs | 63 ++++++++++---- .../expr/src/type_coercion/functions.rs | 86 ++++++++++++++----- .../sqllogictest/test_files/timestamps.slt | 24 ++++++ 3 files changed, 136 insertions(+), 37 deletions(-) diff --git a/datafusion/expr/src/built_in_function.rs b/datafusion/expr/src/built_in_function.rs index a42963495617..58d84545dbb6 100644 --- a/datafusion/expr/src/built_in_function.rs +++ b/datafusion/expr/src/built_in_function.rs @@ -18,7 +18,7 @@ //! Built-in functions module contains all the built-in functions definitions. use crate::nullif::SUPPORTED_NULLIF_TYPES; -use crate::type_coercion::functions::data_types; +use crate::type_coercion::functions::{data_types, TIMEZONE_PLACEHOLDER}; use crate::{ conditional_expressions, struct_expressions, utils, Signature, TypeSignature, Volatility, @@ -1020,13 +1020,25 @@ impl BuiltinScalarFunction { BuiltinScalarFunction::DateTrunc => Signature::one_of( vec![ Exact(vec![Utf8, Timestamp(Nanosecond, None)]), - Exact(vec![Utf8, Timestamp(Nanosecond, Some("+TZ".into()))]), + Exact(vec![ + Utf8, + Timestamp(Nanosecond, Some(TIMEZONE_PLACEHOLDER.into())), + ]), Exact(vec![Utf8, Timestamp(Microsecond, None)]), - Exact(vec![Utf8, Timestamp(Microsecond, Some("+TZ".into()))]), + Exact(vec![ + Utf8, + Timestamp(Microsecond, Some(TIMEZONE_PLACEHOLDER.into())), + ]), Exact(vec![Utf8, Timestamp(Millisecond, None)]), - Exact(vec![Utf8, Timestamp(Millisecond, Some("+TZ".into()))]), + Exact(vec![ + Utf8, + Timestamp(Millisecond, Some(TIMEZONE_PLACEHOLDER.into())), + ]), Exact(vec![Utf8, Timestamp(Second, None)]), - Exact(vec![Utf8, Timestamp(Second, Some("+TZ".into()))]), + Exact(vec![ + Utf8, + Timestamp(Second, Some(TIMEZONE_PLACEHOLDER.into())), + ]), ], self.volatility(), ), @@ -1040,8 +1052,11 @@ impl BuiltinScalarFunction { ]), Exact(vec![ Interval(MonthDayNano), - Timestamp(array_type.clone(), Some("+TZ".into())), - Timestamp(Nanosecond, Some("+TZ".into())), + Timestamp( + array_type.clone(), + Some(TIMEZONE_PLACEHOLDER.into()), + ), + Timestamp(Nanosecond, Some(TIMEZONE_PLACEHOLDER.into())), ]), Exact(vec![ Interval(DayTime), @@ -1050,8 +1065,11 @@ impl BuiltinScalarFunction { ]), Exact(vec![ Interval(DayTime), - Timestamp(array_type.clone(), Some("+TZ".into())), - Timestamp(Nanosecond, Some("+TZ".into())), + Timestamp( + array_type.clone(), + Some(TIMEZONE_PLACEHOLDER.into()), + ), + Timestamp(Nanosecond, Some(TIMEZONE_PLACEHOLDER.into())), ]), Exact(vec![ Interval(MonthDayNano), @@ -1059,7 +1077,10 @@ impl BuiltinScalarFunction { ]), Exact(vec![ Interval(MonthDayNano), - Timestamp(array_type.clone(), Some("+TZ".into())), + Timestamp( + array_type.clone(), + Some(TIMEZONE_PLACEHOLDER.into()), + ), ]), Exact(vec![ Interval(DayTime), @@ -1067,7 +1088,7 @@ impl BuiltinScalarFunction { ]), Exact(vec![ Interval(DayTime), - Timestamp(array_type, Some("+TZ".into())), + Timestamp(array_type, Some(TIMEZONE_PLACEHOLDER.into())), ]), ] }; @@ -1085,13 +1106,25 @@ impl BuiltinScalarFunction { Exact(vec![Utf8, Date32]), Exact(vec![Utf8, Date64]), Exact(vec![Utf8, Timestamp(Second, None)]), - Exact(vec![Utf8, Timestamp(Second, Some("+TZ".into()))]), + Exact(vec![ + Utf8, + Timestamp(Second, Some(TIMEZONE_PLACEHOLDER.into())), + ]), Exact(vec![Utf8, Timestamp(Microsecond, None)]), - Exact(vec![Utf8, Timestamp(Microsecond, Some("+TZ".into()))]), + Exact(vec![ + Utf8, + Timestamp(Microsecond, Some(TIMEZONE_PLACEHOLDER.into())), + ]), Exact(vec![Utf8, Timestamp(Millisecond, None)]), - Exact(vec![Utf8, Timestamp(Millisecond, Some("+TZ".into()))]), + Exact(vec![ + Utf8, + Timestamp(Millisecond, Some(TIMEZONE_PLACEHOLDER.into())), + ]), Exact(vec![Utf8, Timestamp(Nanosecond, None)]), - Exact(vec![Utf8, Timestamp(Nanosecond, Some("+TZ".into()))]), + Exact(vec![ + Utf8, + Timestamp(Nanosecond, Some(TIMEZONE_PLACEHOLDER.into())), + ]), ], self.volatility(), ), diff --git a/datafusion/expr/src/type_coercion/functions.rs b/datafusion/expr/src/type_coercion/functions.rs index 883ca2b39362..5452c8a5c8f5 100644 --- a/datafusion/expr/src/type_coercion/functions.rs +++ b/datafusion/expr/src/type_coercion/functions.rs @@ -22,6 +22,16 @@ use arrow::{ }; use datafusion_common::{plan_err, DataFusionError, Result}; +/// Constant that is used as a placeholder for any valid timezone. +/// This is used where a function can accept a timestamp type with any +/// valid timezone, it exists to avoid the need to enumerate all possible +/// timezones. +/// +/// Type coercion always ensures that functions will be executed using +/// timestamp arrays that have a valid time zone. Functions must never +/// return results with this timezone. +pub(crate) const TIMEZONE_PLACEHOLDER: &str = "+TZ"; + /// Performs type coercion for function arguments. /// /// Returns the data types to which each argument must be coerced to @@ -121,7 +131,7 @@ fn maybe_data_types( } else { // attempt to coerce if let Some(valid_type) = coerced_from(valid_type, current_type) { - new_type.push(valid_type.clone()) + new_type.push(valid_type) } else { // not possible return None; @@ -140,7 +150,7 @@ pub fn can_coerce_from(type_into: &DataType, type_from: &DataType) -> bool { return true; } if let Some(coerced) = coerced_from(type_into, type_from) { - return coerced == type_into; + return coerced == *type_into; } false } @@ -148,15 +158,17 @@ pub fn can_coerce_from(type_into: &DataType, type_from: &DataType) -> bool { fn coerced_from<'a>( type_into: &'a DataType, type_from: &'a DataType, -) -> Option<&'a DataType> { +) -> Option { use self::DataType::*; match type_into { // coerced into type_into - Int8 if matches!(type_from, Null | Int8) => Some(type_into), - Int16 if matches!(type_from, Null | Int8 | Int16 | UInt8) => Some(type_into), + Int8 if matches!(type_from, Null | Int8) => Some(type_into.clone()), + Int16 if matches!(type_from, Null | Int8 | Int16 | UInt8) => { + Some(type_into.clone()) + } Int32 if matches!(type_from, Null | Int8 | Int16 | Int32 | UInt8 | UInt16) => { - Some(type_into) + Some(type_into.clone()) } Int64 if matches!( @@ -164,13 +176,15 @@ fn coerced_from<'a>( Null | Int8 | Int16 | Int32 | Int64 | UInt8 | UInt16 | UInt32 ) => { - Some(type_into) + Some(type_into.clone()) + } + UInt8 if matches!(type_from, Null | UInt8) => Some(type_into.clone()), + UInt16 if matches!(type_from, Null | UInt8 | UInt16) => Some(type_into.clone()), + UInt32 if matches!(type_from, Null | UInt8 | UInt16 | UInt32) => { + Some(type_into.clone()) } - UInt8 if matches!(type_from, Null | UInt8) => Some(type_into), - UInt16 if matches!(type_from, Null | UInt8 | UInt16) => Some(type_into), - UInt32 if matches!(type_from, Null | UInt8 | UInt16 | UInt32) => Some(type_into), UInt64 if matches!(type_from, Null | UInt8 | UInt16 | UInt32 | UInt64) => { - Some(type_into) + Some(type_into.clone()) } Float32 if matches!( @@ -186,7 +200,7 @@ fn coerced_from<'a>( | Float32 ) => { - Some(type_into) + Some(type_into.clone()) } Float64 if matches!( @@ -204,7 +218,7 @@ fn coerced_from<'a>( | Decimal128(_, _) ) => { - Some(type_into) + Some(type_into.clone()) } Timestamp(TimeUnit::Nanosecond, None) if matches!( @@ -212,17 +226,31 @@ fn coerced_from<'a>( Null | Timestamp(_, None) | Date32 | Utf8 | LargeUtf8 ) => { - Some(type_into) + Some(type_into.clone()) } - Interval(_) if matches!(type_from, Utf8 | LargeUtf8) => Some(type_into), - Utf8 | LargeUtf8 => Some(type_into), - Null if can_cast_types(type_from, type_into) => Some(type_into), + Interval(_) if matches!(type_from, Utf8 | LargeUtf8) => Some(type_into.clone()), + Utf8 | LargeUtf8 => Some(type_into.clone()), + Null if can_cast_types(type_from, type_into) => Some(type_into.clone()), - // Coerce to consistent timezones, if the `type_from` timezone exists. - Timestamp(TimeUnit::Nanosecond, Some(_)) - if matches!(type_from, Timestamp(TimeUnit::Nanosecond, Some(_))) => + Timestamp(unit, Some(tz)) if tz.as_ref() == TIMEZONE_PLACEHOLDER => { + match type_from { + Timestamp(_, Some(from_tz)) => { + Some(Timestamp(unit.clone(), Some(from_tz.clone()))) + } + Null | Date32 | Utf8 | LargeUtf8 => { + // In the absence of any other information assume the time zone is "+00" (UTC). + Some(Timestamp(unit.clone(), Some("+00".into()))) + } + _ => None, + } + } + Timestamp(_, Some(_)) + if matches!( + type_from, + Null | Timestamp(_, Some(_)) | Date32 | Utf8 | LargeUtf8 + ) => { - Some(type_from) + Some(type_into.clone()) } // cannot coerce @@ -233,7 +261,7 @@ fn coerced_from<'a>( #[cfg(test)] mod tests { use super::*; - use arrow::datatypes::DataType; + use arrow::datatypes::{DataType, TimeUnit}; #[test] fn test_maybe_data_types() { @@ -265,6 +293,20 @@ mod tests { vec![DataType::Boolean, DataType::UInt16], Some(vec![DataType::Boolean, DataType::UInt32]), ), + // UTF8 -> Timestamp + ( + vec![ + DataType::Timestamp(TimeUnit::Nanosecond, None), + DataType::Timestamp(TimeUnit::Nanosecond, Some("+TZ".into())), + DataType::Timestamp(TimeUnit::Nanosecond, Some("+01".into())), + ], + vec![DataType::Utf8, DataType::Utf8, DataType::Utf8], + Some(vec![ + DataType::Timestamp(TimeUnit::Nanosecond, None), + DataType::Timestamp(TimeUnit::Nanosecond, Some("+00".into())), + DataType::Timestamp(TimeUnit::Nanosecond, Some("+01".into())), + ]), + ), ]; for case in cases { diff --git a/datafusion/sqllogictest/test_files/timestamps.slt b/datafusion/sqllogictest/test_files/timestamps.slt index 88a024e0f9da..bb06c569f081 100644 --- a/datafusion/sqllogictest/test_files/timestamps.slt +++ b/datafusion/sqllogictest/test_files/timestamps.slt @@ -1448,6 +1448,30 @@ SELECT date_bin('1 day', TIMESTAMPTZ '2022-01-01 01:10:00+07', TIMESTAMPTZ '2020 ---- 2021-12-31T00:00:00Z +# postgresql: 2021-12-31 00:00:00+00 +query P +SELECT date_bin('1 day', TIMESTAMPTZ '2022-01-01 01:10:00+07', '2020-01-01') +---- +2021-12-31T00:00:00Z + +# postgresql: 2021-12-31 00:00:00+00 +query P +SELECT date_bin('1 day', TIMESTAMPTZ '2022-01-01 01:10:00+07', '2020-01-01T00:00:00Z') +---- +2021-12-31T00:00:00Z + +# postgresql: 2021-12-31 18:00:00+00 +query P +SELECT date_bin('2 hour', TIMESTAMPTZ '2022-01-01 01:10:00+07', '2020-01-01') +---- +2021-12-31T18:00:00Z + +# postgresql: 2021-12-31 18:00:00+00 +query P +SELECT date_bin('2 hour', TIMESTAMPTZ '2022-01-01 01:10:00+07', '2020-01-01T00:00:00Z') +---- +2021-12-31T18:00:00Z + # postgresql: 1 query R SELECT date_part('hour', TIMESTAMPTZ '2000-01-01T01:01:01') as part From 422e68ee684802b4f5639fd424d980b4a8ca76cd Mon Sep 17 00:00:00 2001 From: Andrew Lamb Date: Mon, 2 Oct 2023 16:27:13 -0400 Subject: [PATCH 018/572] Add LanceDB to the list of Known Users (#7716) * Add LanceDB to the list of Known Users * Update docs/source/user-guide/introduction.md Co-authored-by: Will Jones --------- Co-authored-by: Will Jones --- docs/source/user-guide/introduction.md | 1 + 1 file changed, 1 insertion(+) diff --git a/docs/source/user-guide/introduction.md b/docs/source/user-guide/introduction.md index 272f534613cf..da250fbb1f9c 100644 --- a/docs/source/user-guide/introduction.md +++ b/docs/source/user-guide/introduction.md @@ -106,6 +106,7 @@ Here are some active projects using DataFusion: - [GlareDB](https://github.com/GlareDB/glaredb) Fast SQL database for querying and analyzing distributed data. - [InfluxDB IOx](https://github.com/influxdata/influxdb_iox) Time Series Database - [Kamu](https://github.com/kamu-data/kamu-cli/) Planet-scale streaming data pipeline +- [Lance](https://github.com/lancedb/lance) Modern columnar data format for ML - [Parseable](https://github.com/parseablehq/parseable) Log storage and observability platform - [qv](https://github.com/timvw/qv) Quickly view your data - [bdt](https://github.com/andygrove/bdt) Boring Data Tool From 872fd16d263a72f68664218091792bba62366bd4 Mon Sep 17 00:00:00 2001 From: Andrew Lamb Date: Tue, 3 Oct 2023 06:00:52 -0400 Subject: [PATCH 019/572] Enable avro reading/writing in datafusion-cli (#7715) --- datafusion-cli/Cargo.lock | 271 ++++++++++++++++++++++++++----------- datafusion-cli/Cargo.toml | 2 +- datafusion/core/Cargo.toml | 4 +- 3 files changed, 195 insertions(+), 82 deletions(-) diff --git a/datafusion-cli/Cargo.lock b/datafusion-cli/Cargo.lock index 775f8ec87e38..1235c0b740bc 100644 --- a/datafusion-cli/Cargo.lock +++ b/datafusion-cli/Cargo.lock @@ -17,6 +17,12 @@ version = "1.0.2" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "f26201604c87b1e01bd3d98f8d5d9a8fcbb815e8cedb41ffccbeb4bf593a35fe" +[[package]] +name = "adler32" +version = "1.2.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "aae1277d39aeec15cb388266ecc24b11c80469deae6067e17a1a7aa9e5c1f234" + [[package]] name = "ahash" version = "0.8.3" @@ -77,9 +83,34 @@ dependencies = [ [[package]] name = "anstyle" -version = "1.0.3" +version = "1.0.4" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "b84bf0a05bbb2a83e5eb6fa36bb6e87baa08193c35ff52bbf6b38d8af2890e46" +checksum = "7079075b41f533b8c61d2a4d073c4676e1f8b249ff94a393b0595db304e0dd87" + +[[package]] +name = "apache-avro" +version = "0.16.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "ceb7c683b2f8f40970b70e39ff8be514c95b96fcb9c4af87e1ed2cb2e10801a0" +dependencies = [ + "crc32fast", + "digest", + "lazy_static", + "libflate", + "log", + "num-bigint", + "quad-rand", + "rand", + "regex-lite", + "serde", + "serde_json", + "snap", + "strum 0.25.0", + "strum_macros 0.25.2", + "thiserror", + "typed-builder", + "uuid", +] [[package]] name = "arrayref" @@ -143,7 +174,7 @@ dependencies = [ "chrono", "chrono-tz", "half", - "hashbrown 0.14.0", + "hashbrown 0.14.1", "num", ] @@ -234,7 +265,7 @@ dependencies = [ "arrow-schema", "chrono", "half", - "indexmap 2.0.0", + "indexmap 2.0.2", "lexical-core", "num", "serde", @@ -268,7 +299,7 @@ dependencies = [ "arrow-data", "arrow-schema", "half", - "hashbrown 0.14.0", + "hashbrown 0.14.1", ] [[package]] @@ -734,9 +765,9 @@ dependencies = [ [[package]] name = "brotli" -version = "3.3.4" +version = "3.4.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "a1a0b1dbcc8ae29329621f8d4f0d835787c1c38bb1401979b49d13b0b305ff68" +checksum = "516074a47ef4bce09577a3b379392300159ce5b1ba2e501ff1c819950066100f" dependencies = [ "alloc-no-stdlib", "alloc-stdlib", @@ -745,9 +776,9 @@ dependencies = [ [[package]] name = "brotli-decompressor" -version = "2.3.4" +version = "2.5.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "4b6561fd3f895a11e8f72af2cb7d22e08366bebc2b6b57f7744c4bda27034744" +checksum = "da74e2b81409b1b743f8f0c62cc6254afefb8b8e50bbfe3735550f7aeefa3448" dependencies = [ "alloc-no-stdlib", "alloc-stdlib", @@ -969,6 +1000,15 @@ version = "0.8.4" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "e496a50fda8aacccc86d7529e2c1e0892dbd0f898a6b5645b5561b89c3210efa" +[[package]] +name = "core2" +version = "0.4.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "b49ba7ef1ad6107f8824dbe97de947cbaac53c44e7f9756a1fba0d37c1eec505" +dependencies = [ + "memchr", +] + [[package]] name = "cpufeatures" version = "0.2.9" @@ -1026,14 +1066,20 @@ dependencies = [ [[package]] name = "ctor" -version = "0.2.4" +version = "0.2.5" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "1f34ba9a9bcb8645379e9de8cb3ecfcf4d1c85ba66d90deb3259206fa5aa193b" +checksum = "37e366bff8cd32dd8754b0991fb66b279dc48f598c3a18914852a6673deef583" dependencies = [ "quote", "syn 2.0.37", ] +[[package]] +name = "dary_heap" +version = "0.3.6" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "7762d17f1241643615821a8455a0b2c3e803784b058693d990b11f2dce25a0ca" + [[package]] name = "dashmap" version = "5.5.3" @@ -1041,7 +1087,7 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "978747c1d849a7d2ee5e8adc0159961c48fb7e5db2f06af6723b80123bb53856" dependencies = [ "cfg-if", - "hashbrown 0.14.0", + "hashbrown 0.14.1", "lock_api", "once_cell", "parking_lot_core", @@ -1052,6 +1098,7 @@ name = "datafusion" version = "31.0.0" dependencies = [ "ahash", + "apache-avro", "arrow", "arrow-array", "arrow-schema", @@ -1072,10 +1119,11 @@ dependencies = [ "futures", "glob", "half", - "hashbrown 0.14.0", - "indexmap 2.0.0", - "itertools 0.11.0", + "hashbrown 0.14.1", + "indexmap 2.0.2", + "itertools", "log", + "num-traits", "num_cpus", "object_store", "parking_lot", @@ -1123,6 +1171,7 @@ name = "datafusion-common" version = "31.0.0" dependencies = [ "ahash", + "apache-avro", "arrow", "arrow-array", "arrow-buffer", @@ -1145,7 +1194,7 @@ dependencies = [ "datafusion-common", "datafusion-expr", "futures", - "hashbrown 0.14.0", + "hashbrown 0.14.1", "log", "object_store", "parking_lot", @@ -1177,8 +1226,8 @@ dependencies = [ "datafusion-common", "datafusion-expr", "datafusion-physical-expr", - "hashbrown 0.14.0", - "itertools 0.11.0", + "hashbrown 0.14.1", + "itertools", "log", "regex-syntax", ] @@ -1199,10 +1248,10 @@ dependencies = [ "datafusion-common", "datafusion-expr", "half", - "hashbrown 0.14.0", + "hashbrown 0.14.1", "hex", - "indexmap 2.0.0", - "itertools 0.11.0", + "indexmap 2.0.2", + "itertools", "libc", "log", "md-5", @@ -1232,9 +1281,9 @@ dependencies = [ "datafusion-physical-expr", "futures", "half", - "hashbrown 0.14.0", - "indexmap 2.0.0", - "itertools 0.11.0", + "hashbrown 0.14.1", + "indexmap 2.0.2", + "itertools", "log", "once_cell", "parking_lot", @@ -1408,9 +1457,9 @@ dependencies = [ [[package]] name = "fastrand" -version = "2.0.0" +version = "2.0.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "6999dc1837253364c2ebb0704ba97994bd874e8f195d665c50b7548f6ea92764" +checksum = "25cbce373ec4653f1a01a31e8a5e5ec0c622dc27ff9c4e6606eefef5cbbed4a5" [[package]] name = "fd-lock" @@ -1639,9 +1688,18 @@ checksum = "8a9ee70c43aaf417c914396645a0fa852624801b24ebb7ae78fe8272889ac888" [[package]] name = "hashbrown" -version = "0.14.0" +version = "0.13.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "43a3c133739dddd0d2990f9a4bdf8eb4b21ef50e4851ca85ab661199821d510e" +dependencies = [ + "ahash", +] + +[[package]] +name = "hashbrown" +version = "0.14.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "2c6201b9ff9fd90a5a3bac2e56a830d0caa509576f0e503818ee82c181b3437a" +checksum = "7dfda62a12f55daeae5015f81b0baea145391cb4520f86c248fc615d72640d12" dependencies = [ "ahash", "allocator-api2", @@ -1821,12 +1879,12 @@ dependencies = [ [[package]] name = "indexmap" -version = "2.0.0" +version = "2.0.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "d5477fe2230a79769d8dc68e0eabf5437907c0457a5614a9e8dddb67f65eb65d" +checksum = "8adf3ddd720272c6ea8bf59463c04e0f93d0bbf7c5439b691bca2987e0270897" dependencies = [ "equivalent", - "hashbrown 0.14.0", + "hashbrown 0.14.1", ] [[package]] @@ -1850,15 +1908,6 @@ version = "2.8.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "28b29a3cd74f0f4598934efe3aeba42bae0eb4680554128851ebbecb02af14e6" -[[package]] -name = "itertools" -version = "0.10.5" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "b0fd2260e829bddf4cb6ea802289de2f86d6a7a690192fbe91b3f46e0f2c8473" -dependencies = [ - "either", -] - [[package]] name = "itertools" version = "0.11.0" @@ -1968,6 +2017,30 @@ version = "0.2.148" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "9cdc71e17332e86d2e1d38c1f99edcb6288ee11b815fb1a4b049eaa2114d369b" +[[package]] +name = "libflate" +version = "2.0.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "9f7d5654ae1795afc7ff76f4365c2c8791b0feb18e8996a96adad8ffd7c3b2bf" +dependencies = [ + "adler32", + "core2", + "crc32fast", + "dary_heap", + "libflate_lz77", +] + +[[package]] +name = "libflate_lz77" +version = "2.0.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "be5f52fb8c451576ec6b79d3f4deb327398bc05bbdbd99021a6e77a4c855d524" +dependencies = [ + "core2", + "hashbrown 0.13.2", + "rle-decode-fast", +] + [[package]] name = "libm" version = "0.2.7" @@ -1986,9 +2059,9 @@ dependencies = [ [[package]] name = "linux-raw-sys" -version = "0.4.7" +version = "0.4.8" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "1a9bad9f94746442c783ca431b22403b519cd7fbeed0533fdd6328b2f2212128" +checksum = "3852614a3bd9ca9804678ba6be5e3b8ce76dfc902cae004e3e0c44051b6e88db" [[package]] name = "lock_api" @@ -2039,10 +2112,11 @@ dependencies = [ [[package]] name = "md-5" -version = "0.10.5" +version = "0.10.6" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "6365506850d44bff6e2fbcb5176cf63650e48bd45ef2fe2665ae1570e0f4b9ca" +checksum = "d89e7ee0cfbedfc4da3340218492196241d89eefb6dab27de5df917a6d2e78cf" dependencies = [ + "cfg-if", "digest", ] @@ -2211,9 +2285,9 @@ dependencies = [ [[package]] name = "object_store" -version = "0.7.0" +version = "0.7.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "d359e231e5451f4f9fa889d56e3ce34f8724f1a61db2107739359717cf2bbf08" +checksum = "f930c88a43b1c3f6e776dfe495b4afab89882dbc81530c632db2ed65451ebcb4" dependencies = [ "async-trait", "base64", @@ -2222,7 +2296,7 @@ dependencies = [ "futures", "humantime", "hyper", - "itertools 0.10.5", + "itertools", "parking_lot", "percent-encoding", "quick-xml", @@ -2315,7 +2389,7 @@ dependencies = [ "chrono", "flate2", "futures", - "hashbrown 0.14.0", + "hashbrown 0.14.1", "lz4", "num", "num-bigint", @@ -2357,7 +2431,7 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "e1d3afd2628e69da2be385eb6f2fd57c8ac7977ceeff6dc166ff1657b0e386a9" dependencies = [ "fixedbitset", - "indexmap 2.0.0", + "indexmap 2.0.2", ] [[package]] @@ -2451,7 +2525,7 @@ dependencies = [ "anstyle", "difflib", "float-cmp", - "itertools 0.11.0", + "itertools", "normalize-line-endings", "predicates-core", "regex", @@ -2512,11 +2586,17 @@ dependencies = [ "unicode-ident", ] +[[package]] +name = "quad-rand" +version = "0.2.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "658fa1faf7a4cc5f057c9ee5ef560f717ad9d8dc66d975267f709624d6e1ab88" + [[package]] name = "quick-xml" -version = "0.28.2" +version = "0.30.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "0ce5e73202a820a31f8a0ee32ada5e21029c81fd9e3ebf668a40832e4219d9d1" +checksum = "eff6510e86862b57b210fd8cbe8ed3f0d7d600b9c2863cd4549a2e033c66e956" dependencies = [ "memchr", "serde", @@ -2602,9 +2682,9 @@ dependencies = [ [[package]] name = "regex" -version = "1.9.5" +version = "1.9.6" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "697061221ea1b4a94a624f67d0ae2bfe4e22b8a17b6a192afb11046542cc8c47" +checksum = "ebee201405406dbf528b8b672104ae6d6d63e6d118cb10e4d51abbc7b58044ff" dependencies = [ "aho-corasick", "memchr", @@ -2614,15 +2694,21 @@ dependencies = [ [[package]] name = "regex-automata" -version = "0.3.8" +version = "0.3.9" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "c2f401f4955220693b56f8ec66ee9c78abffd8d1c4f23dc41a23839eb88f0795" +checksum = "59b23e92ee4318893fa3fe3e6fb365258efbfe6ac6ab30f090cdcbb7aa37efa9" dependencies = [ "aho-corasick", "memchr", "regex-syntax", ] +[[package]] +name = "regex-lite" +version = "0.1.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "f96ede7f386ba6e910092e7ccdc04176cface62abebea07ed6b46d870ed95ca2" + [[package]] name = "regex-syntax" version = "0.7.5" @@ -2685,6 +2771,12 @@ dependencies = [ "winapi", ] +[[package]] +name = "rle-decode-fast" +version = "1.0.3" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "3582f63211428f83597b51b2ddb88e2a91a9d52d12831f9d08f5e624e8977422" + [[package]] name = "rstest" version = "0.17.0" @@ -2728,9 +2820,9 @@ dependencies = [ [[package]] name = "rustix" -version = "0.38.14" +version = "0.38.15" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "747c788e9ce8e92b12cd485c49ddf90723550b654b32508f979b71a7b1ecda4f" +checksum = "d2f9da0cbd88f9f09e7814e388301c8414c51c62aa6ce1e4b5c551d49d96e531" dependencies = [ "bitflags 2.4.0", "errno", @@ -2786,9 +2878,9 @@ dependencies = [ [[package]] name = "rustls-webpki" -version = "0.101.5" +version = "0.101.6" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "45a27e3b59326c16e23d30aeb7a36a24cc0d29e71d68ff611cdfb4a01d013bed" +checksum = "3c7d5dece342910d9ba34d259310cae3e0154b873b35408b787b59bce53d34fe" dependencies = [ "ring", "untrusted", @@ -2888,9 +2980,9 @@ dependencies = [ [[package]] name = "semver" -version = "1.0.18" +version = "1.0.19" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "b0293b4b29daaf487284529cc2f5675b8e57c61f70167ba415a463651fd6a918" +checksum = "ad977052201c6de01a8ef2aa3378c4bd23217a056337d1d6da40468d267a4fb0" [[package]] name = "seq-macro" @@ -2943,9 +3035,9 @@ dependencies = [ [[package]] name = "sha2" -version = "0.10.7" +version = "0.10.8" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "479fb9d862239e610720565ca91403019f2f00410f1864c5aa7479b950a76ed8" +checksum = "793db75ad2bcafc3ffa7c68b215fee268f537982cd901d132f89c6343f3a3dc8" dependencies = [ "cfg-if", "cpufeatures", @@ -3142,7 +3234,7 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "cb94d2f3cc536af71caac6b6fcebf65860b347e7ce0cc9ebe8f70d3e521054ef" dependencies = [ "cfg-if", - "fastrand 2.0.0", + "fastrand 2.0.1", "redox_syscall 0.3.5", "rustix", "windows-sys", @@ -3171,18 +3263,18 @@ checksum = "222a222a5bfe1bba4a77b45ec488a741b3cb8872e5e499451fd7d0129c9c7c3d" [[package]] name = "thiserror" -version = "1.0.48" +version = "1.0.49" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "9d6d7a740b8a666a7e828dd00da9c0dc290dff53154ea77ac109281de90589b7" +checksum = "1177e8c6d7ede7afde3585fd2513e611227efd6481bd78d2e82ba1ce16557ed4" dependencies = [ "thiserror-impl", ] [[package]] name = "thiserror-impl" -version = "1.0.48" +version = "1.0.49" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "49922ecae66cc8a249b77e68d1d0623c1b2c514f0060c27cdc68bd62a1219d35" +checksum = "10712f02019e9288794769fba95cd6847df9874d49d871d062172f9dd41bc4cc" dependencies = [ "proc-macro2", "quote", @@ -3202,9 +3294,9 @@ dependencies = [ [[package]] name = "time" -version = "0.3.28" +version = "0.3.29" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "17f6bb557fd245c28e6411aa56b6403c689ad95061f50e4be16c274e70a17e48" +checksum = "426f806f4089c493dcac0d24c29c01e2c38baf8e30f1b716ee37e83d200b18fe" dependencies = [ "deranged", "serde", @@ -3214,15 +3306,15 @@ dependencies = [ [[package]] name = "time-core" -version = "0.1.1" +version = "0.1.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "7300fbefb4dadc1af235a9cef3737cea692a9d97e1b9cbcd4ebdae6f8868e6fb" +checksum = "ef927ca75afb808a4d64dd374f00a2adf8d0fcff8e7b184af886c3c87ec4a3f3" [[package]] name = "time-macros" -version = "0.2.14" +version = "0.2.15" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "1a942f44339478ef67935ab2bbaec2fb0322496cf3cbe84b261e06ac3814c572" +checksum = "4ad70d68dba9e1f8aceda7aa6711965dfec1cac869f311a51bd08b3a2ccbce20" dependencies = [ "time-core", ] @@ -3403,6 +3495,26 @@ dependencies = [ "static_assertions", ] +[[package]] +name = "typed-builder" +version = "0.16.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "34085c17941e36627a879208083e25d357243812c30e7d7387c3b954f30ade16" +dependencies = [ + "typed-builder-macro", +] + +[[package]] +name = "typed-builder-macro" +version = "0.16.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "f03ca4cb38206e2bef0700092660bb74d696f808514dae47fa1467cbfe26e96e" +dependencies = [ + "proc-macro2", + "quote", + "syn 2.0.37", +] + [[package]] name = "typenum" version = "1.17.0" @@ -3478,6 +3590,7 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "79daa5ed5740825c40b389c5e50312b9c86df53fccd33f281df655642b43869d" dependencies = [ "getrandom", + "serde", ] [[package]] @@ -3617,9 +3730,9 @@ dependencies = [ [[package]] name = "webpki" -version = "0.22.1" +version = "0.22.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "f0e74f82d49d545ad128049b7e88f6576df2da6b02e9ce565c6f533be576957e" +checksum = "07ecc0cd7cac091bf682ec5efa18b1cff79d617b84181f38b3951dbe135f607f" dependencies = [ "ring", "untrusted", @@ -3749,9 +3862,9 @@ dependencies = [ [[package]] name = "xmlparser" -version = "0.13.5" +version = "0.13.6" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "4d25c75bf9ea12c4040a97f829154768bbbce366287e2dc044af160cd79a13fd" +checksum = "66fee0b777b0f5ac1c69bb06d361268faafa61cd4682ae064a171c16c433e9e4" [[package]] name = "xz2" diff --git a/datafusion-cli/Cargo.toml b/datafusion-cli/Cargo.toml index a550b487509d..9653c73d25bf 100644 --- a/datafusion-cli/Cargo.toml +++ b/datafusion-cli/Cargo.toml @@ -34,7 +34,7 @@ async-trait = "0.1.41" aws-config = "0.55" aws-credential-types = "0.55" clap = { version = "3", features = ["derive", "cargo"] } -datafusion = { path = "../datafusion/core", version = "31.0.0" } +datafusion = { path = "../datafusion/core", version = "31.0.0", features = ["avro", "crypto_expressions", "encoding_expressions", "regex_expressions", "unicode_expressions", "compression"] } dirs = "4.0.0" env_logger = "0.9" mimalloc = { version = "0.1", default-features = false } diff --git a/datafusion/core/Cargo.toml b/datafusion/core/Cargo.toml index 266ff855752b..d84d6a13c336 100644 --- a/datafusion/core/Cargo.toml +++ b/datafusion/core/Cargo.toml @@ -39,8 +39,8 @@ avro = ["apache-avro", "num-traits", "datafusion-common/avro"] backtrace = ["datafusion-common/backtrace"] compression = ["xz2", "bzip2", "flate2", "zstd", "async-compression"] crypto_expressions = ["datafusion-physical-expr/crypto_expressions", "datafusion-optimizer/crypto_expressions"] -default = ["crypto_expressions", "encoding__expressions", "regex_expressions", "unicode_expressions", "compression"] -encoding__expressions = ["datafusion-physical-expr/encoding_expressions"] +default = ["crypto_expressions", "encoding_expressions", "regex_expressions", "unicode_expressions", "compression"] +encoding_expressions = ["datafusion-physical-expr/encoding_expressions"] # Used for testing ONLY: causes all values to hash to the same value (test for collisions) force_hash_collisions = [] pyarrow = ["datafusion-common/pyarrow"] From b1587c1149c5a94e8acfa3c0d8d623be4fd2f37c Mon Sep 17 00:00:00 2001 From: Andrew Lamb Date: Tue, 3 Oct 2023 06:01:09 -0400 Subject: [PATCH 020/572] Document crate feature flags (#7713) * Document crate feature flags * prettier --- README.md | 28 +++++++++++++++++++++++-- datafusion/common/src/pyarrow.rs | 2 +- docs/source/user-guide/example-usage.md | 4 ---- 3 files changed, 27 insertions(+), 7 deletions(-) diff --git a/README.md b/README.md index ccb527a1f977..63da8c1c1a96 100644 --- a/README.md +++ b/README.md @@ -35,9 +35,33 @@ Here are links to some important information - [Python DataFrame API](https://arrow.apache.org/datafusion-python/) - [Architecture](https://docs.rs/datafusion/latest/datafusion/index.html#architecture) -## Building your project with DataFusion +## What can you do with this crate? -DataFusion is great for building projects and products like SQL interfaces, time series platforms, and domain specific query engines. [Click Here](https://arrow.apache.org/datafusion/user-guide/introduction.html#known-users) to see a list known users. +DataFusion is great for building projects such as domain specific query engines, new database platforms and data pipelines, query languages and more. +It lets you start quickly from a fully working engine, and then customize those features specific to your use. [Click Here](https://arrow.apache.org/datafusion/user-guide/introduction.html#known-users) to see a list known users. + +## Crate features + +Default features: + +- `compression`: reading files compressed with `xz2`, `bzip2`, `flate2`, and `zstd` +- `crypto_expressions`: cryptographic functions such as `md5` and `sha256` +- `encoding_expressions`: `encode` and `decode` functions +- `regex_expressions`: regular expression functions, such as `regexp_match` +- `unicode_expressions`: Include unicode aware functions such as `character_length` + +Optional features: + +- `avro`: support for reading the [Apache Avro] format +- `backtrace`: include backtrace information in error messages +- `pyarrow`: conversions between PyArrow and DataFusion types +- `simd`: enable arrow-rs's manual `SIMD` kernels (requires Rust `nightly`) + +[apache avro]: https://avro.apache.org/ + +## Rust Version Compatibility + +This crate is tested with the latest stable version of Rust. We do not currently test against other, older versions of the Rust compiler. ## Contributing to DataFusion diff --git a/datafusion/common/src/pyarrow.rs b/datafusion/common/src/pyarrow.rs index d18782e037ae..d78aa8b988f7 100644 --- a/datafusion/common/src/pyarrow.rs +++ b/datafusion/common/src/pyarrow.rs @@ -15,7 +15,7 @@ // specific language governing permissions and limitations // under the License. -//! PyArrow +//! Conversions between PyArrow and DataFusion types use arrow::array::ArrayData; use arrow::pyarrow::{FromPyArrow, ToPyArrow}; diff --git a/docs/source/user-guide/example-usage.md b/docs/source/user-guide/example-usage.md index adaf780558bc..c631d552dd73 100644 --- a/docs/source/user-guide/example-usage.md +++ b/docs/source/user-guide/example-usage.md @@ -187,10 +187,6 @@ DataFusion is designed to be extensible at all points. To that end, you can prov - [x] User Defined `LogicalPlan` nodes - [x] User Defined `ExecutionPlan` nodes -## Rust Version Compatibility - -This crate is tested with the latest stable version of Rust. We do not currently test against other, older versions of the Rust compiler. - ## Optimized Configuration For an optimized build several steps are required. First, use the below in your `Cargo.toml`. It is From 32fe1761287c9b58294a54805a0fcafc4ab046b8 Mon Sep 17 00:00:00 2001 From: Andrew Lamb Date: Tue, 3 Oct 2023 07:34:22 -0400 Subject: [PATCH 021/572] Minor: Consolidate UDF tests (#7704) * Minor: Consolidate user defined functions * cleanup * move more tests * more * cleanup use --- datafusion/core/src/execution/context.rs | 86 +-------- datafusion/core/tests/sql/expr.rs | 16 +- datafusion/core/tests/sql/mod.rs | 55 +----- datafusion/core/tests/user_defined/mod.rs | 3 + .../user_defined/user_defined_aggregates.rs | 94 ++++++++++ .../user_defined_scalar_functions.rs} | 174 ++++++++++++------ 6 files changed, 222 insertions(+), 206 deletions(-) rename datafusion/core/tests/{sql/udf.rs => user_defined/user_defined_scalar_functions.rs} (68%) diff --git a/datafusion/core/src/execution/context.rs b/datafusion/core/src/execution/context.rs index 6cfb73a5109a..4bdd40a91403 100644 --- a/datafusion/core/src/execution/context.rs +++ b/datafusion/core/src/execution/context.rs @@ -2222,16 +2222,13 @@ mod tests { use crate::execution::context::QueryPlanner; use crate::execution::memory_pool::MemoryConsumer; use crate::execution::runtime_env::RuntimeConfig; - use crate::physical_plan::expressions::AvgAccumulator; use crate::test; use crate::test_util::parquet_test_data; use crate::variable::VarType; - use arrow::array::ArrayRef; use arrow::record_batch::RecordBatch; use arrow_schema::{Field, Schema}; use async_trait::async_trait; - use datafusion_expr::{create_udaf, create_udf, Expr, Volatility}; - use datafusion_physical_expr::functions::make_scalar_function; + use datafusion_expr::Expr; use std::fs::File; use std::path::PathBuf; use std::sync::Weak; @@ -2330,87 +2327,6 @@ mod tests { Ok(()) } - #[tokio::test] - async fn case_sensitive_identifiers_user_defined_functions() -> Result<()> { - let ctx = SessionContext::new(); - ctx.register_table("t", test::table_with_sequence(1, 1).unwrap()) - .unwrap(); - - let myfunc = |args: &[ArrayRef]| Ok(Arc::clone(&args[0])); - let myfunc = make_scalar_function(myfunc); - - ctx.register_udf(create_udf( - "MY_FUNC", - vec![DataType::Int32], - Arc::new(DataType::Int32), - Volatility::Immutable, - myfunc, - )); - - // doesn't work as it was registered with non lowercase - let err = plan_and_collect(&ctx, "SELECT MY_FUNC(i) FROM t") - .await - .unwrap_err(); - assert!(err - .to_string() - .contains("Error during planning: Invalid function \'my_func\'")); - - // Can call it if you put quotes - let result = plan_and_collect(&ctx, "SELECT \"MY_FUNC\"(i) FROM t").await?; - - let expected = [ - "+--------------+", - "| MY_FUNC(t.i) |", - "+--------------+", - "| 1 |", - "+--------------+", - ]; - assert_batches_eq!(expected, &result); - - Ok(()) - } - - #[tokio::test] - async fn case_sensitive_identifiers_user_defined_aggregates() -> Result<()> { - let ctx = SessionContext::new(); - ctx.register_table("t", test::table_with_sequence(1, 1).unwrap()) - .unwrap(); - - // Note capitalization - let my_avg = create_udaf( - "MY_AVG", - vec![DataType::Float64], - Arc::new(DataType::Float64), - Volatility::Immutable, - Arc::new(|_| Ok(Box::::default())), - Arc::new(vec![DataType::UInt64, DataType::Float64]), - ); - - ctx.register_udaf(my_avg); - - // doesn't work as it was registered as non lowercase - let err = plan_and_collect(&ctx, "SELECT MY_AVG(i) FROM t") - .await - .unwrap_err(); - assert!(err - .to_string() - .contains("Error during planning: Invalid function \'my_avg\'")); - - // Can call it if you put quotes - let result = plan_and_collect(&ctx, "SELECT \"MY_AVG\"(i) FROM t").await?; - - let expected = [ - "+-------------+", - "| MY_AVG(t.i) |", - "+-------------+", - "| 1.0 |", - "+-------------+", - ]; - assert_batches_eq!(expected, &result); - - Ok(()) - } - #[tokio::test] async fn query_csv_with_custom_partition_extension() -> Result<()> { let tmp_dir = TempDir::new()?; diff --git a/datafusion/core/tests/sql/expr.rs b/datafusion/core/tests/sql/expr.rs index 044b3b57ea90..af33cfea65af 100644 --- a/datafusion/core/tests/sql/expr.rs +++ b/datafusion/core/tests/sql/expr.rs @@ -616,7 +616,7 @@ async fn test_array_cast_expressions() -> Result<()> { #[tokio::test] async fn test_random_expression() -> Result<()> { - let ctx = create_ctx(); + let ctx = SessionContext::new(); let sql = "SELECT random() r1"; let actual = execute(&ctx, sql).await; let r1 = actual[0][0].parse::().unwrap(); @@ -627,7 +627,7 @@ async fn test_random_expression() -> Result<()> { #[tokio::test] async fn test_uuid_expression() -> Result<()> { - let ctx = create_ctx(); + let ctx = SessionContext::new(); let sql = "SELECT uuid()"; let actual = execute(&ctx, sql).await; let uuid = actual[0][0].parse::().unwrap(); @@ -886,18 +886,6 @@ async fn csv_query_nullif_divide_by_0() -> Result<()> { Ok(()) } -#[tokio::test] -async fn csv_query_avg_sqrt() -> Result<()> { - let ctx = create_ctx(); - register_aggregate_csv(&ctx).await?; - let sql = "SELECT avg(custom_sqrt(c12)) FROM aggregate_test_100"; - let mut actual = execute(&ctx, sql).await; - actual.sort(); - let expected = vec![vec!["0.6706002946036462"]]; - assert_float_eq(&expected, &actual); - Ok(()) -} - #[tokio::test] async fn nested_subquery() -> Result<()> { let ctx = SessionContext::new(); diff --git a/datafusion/core/tests/sql/mod.rs b/datafusion/core/tests/sql/mod.rs index 7d175b65260f..4529889270c6 100644 --- a/datafusion/core/tests/sql/mod.rs +++ b/datafusion/core/tests/sql/mod.rs @@ -26,6 +26,7 @@ use chrono::prelude::*; use chrono::Duration; use datafusion::datasource::TableProvider; +use datafusion::error::{DataFusionError, Result}; use datafusion::logical_expr::{Aggregate, LogicalPlan, TableScan}; use datafusion::physical_plan::metrics::MetricValue; use datafusion::physical_plan::ExecutionPlan; @@ -34,15 +35,9 @@ use datafusion::prelude::*; use datafusion::test_util; use datafusion::{assert_batches_eq, assert_batches_sorted_eq}; use datafusion::{datasource::MemTable, physical_plan::collect}; -use datafusion::{ - error::{DataFusionError, Result}, - physical_plan::ColumnarValue, -}; use datafusion::{execution::context::SessionContext, physical_plan::displayable}; -use datafusion_common::cast::as_float64_array; use datafusion_common::plan_err; use datafusion_common::{assert_contains, assert_not_contains}; -use datafusion_expr::Volatility; use object_store::path::Path; use std::fs::File; use std::io::Write; @@ -101,54 +96,6 @@ pub mod select; mod sql_api; pub mod subqueries; pub mod timestamp; -pub mod udf; - -fn assert_float_eq(expected: &[Vec], received: &[Vec]) -where - T: AsRef, -{ - expected - .iter() - .flatten() - .zip(received.iter().flatten()) - .for_each(|(l, r)| { - let (l, r) = ( - l.as_ref().parse::().unwrap(), - r.as_str().parse::().unwrap(), - ); - if l.is_nan() || r.is_nan() { - assert!(l.is_nan() && r.is_nan()); - } else if (l - r).abs() > 2.0 * f64::EPSILON { - panic!("{l} != {r}") - } - }); -} - -fn create_ctx() -> SessionContext { - let ctx = SessionContext::new(); - - // register a custom UDF - ctx.register_udf(create_udf( - "custom_sqrt", - vec![DataType::Float64], - Arc::new(DataType::Float64), - Volatility::Immutable, - Arc::new(custom_sqrt), - )); - - ctx -} - -fn custom_sqrt(args: &[ColumnarValue]) -> Result { - let arg = &args[0]; - if let ColumnarValue::Array(v) = arg { - let input = as_float64_array(v).expect("cast failed"); - let array: Float64Array = input.iter().map(|v| v.map(|x| x.sqrt())).collect(); - Ok(ColumnarValue::Array(Arc::new(array))) - } else { - unimplemented!() - } -} fn create_join_context( column_left: &str, diff --git a/datafusion/core/tests/user_defined/mod.rs b/datafusion/core/tests/user_defined/mod.rs index ab6f51c47ba7..09c7c3d3266b 100644 --- a/datafusion/core/tests/user_defined/mod.rs +++ b/datafusion/core/tests/user_defined/mod.rs @@ -15,6 +15,9 @@ // specific language governing permissions and limitations // under the License. +/// Tests for user defined Scalar functions +mod user_defined_scalar_functions; + /// Tests for User Defined Aggregate Functions mod user_defined_aggregates; diff --git a/datafusion/core/tests/user_defined/user_defined_aggregates.rs b/datafusion/core/tests/user_defined/user_defined_aggregates.rs index 3b7b4d0e87b7..fb0ecd02c6b0 100644 --- a/datafusion/core/tests/user_defined/user_defined_aggregates.rs +++ b/datafusion/core/tests/user_defined/user_defined_aggregates.rs @@ -19,11 +19,14 @@ //! user defined aggregate functions use arrow::{array::AsArray, datatypes::Fields}; +use arrow_array::Int32Array; +use arrow_schema::Schema; use std::sync::{ atomic::{AtomicBool, Ordering}, Arc, }; +use datafusion::datasource::MemTable; use datafusion::{ arrow::{ array::{ArrayRef, Float64Array, TimestampNanosecondArray}, @@ -43,6 +46,8 @@ use datafusion::{ use datafusion_common::{ assert_contains, cast::as_primitive_array, exec_err, DataFusionError, }; +use datafusion_expr::create_udaf; +use datafusion_physical_expr::expressions::AvgAccumulator; /// Test to show the contents of the setup #[tokio::test] @@ -204,6 +209,95 @@ async fn execute(ctx: &SessionContext, sql: &str) -> Result> { ctx.sql(sql).await?.collect().await } +/// tests the creation, registration and usage of a UDAF +#[tokio::test] +async fn simple_udaf() -> Result<()> { + let schema = Schema::new(vec![Field::new("a", DataType::Int32, false)]); + + let batch1 = RecordBatch::try_new( + Arc::new(schema.clone()), + vec![Arc::new(Int32Array::from(vec![1, 2, 3]))], + )?; + let batch2 = RecordBatch::try_new( + Arc::new(schema.clone()), + vec![Arc::new(Int32Array::from(vec![4, 5]))], + )?; + + let ctx = SessionContext::new(); + + let provider = MemTable::try_new(Arc::new(schema), vec![vec![batch1], vec![batch2]])?; + ctx.register_table("t", Arc::new(provider))?; + + // define a udaf, using a DataFusion's accumulator + let my_avg = create_udaf( + "my_avg", + vec![DataType::Float64], + Arc::new(DataType::Float64), + Volatility::Immutable, + Arc::new(|_| Ok(Box::::default())), + Arc::new(vec![DataType::UInt64, DataType::Float64]), + ); + + ctx.register_udaf(my_avg); + + let result = ctx.sql("SELECT MY_AVG(a) FROM t").await?.collect().await?; + + let expected = [ + "+-------------+", + "| my_avg(t.a) |", + "+-------------+", + "| 3.0 |", + "+-------------+", + ]; + assert_batches_eq!(expected, &result); + + Ok(()) +} + +#[tokio::test] +async fn case_sensitive_identifiers_user_defined_aggregates() -> Result<()> { + let ctx = SessionContext::new(); + let arr = Int32Array::from(vec![1]); + let batch = RecordBatch::try_from_iter(vec![("i", Arc::new(arr) as _)])?; + ctx.register_batch("t", batch).unwrap(); + + // Note capitalization + let my_avg = create_udaf( + "MY_AVG", + vec![DataType::Float64], + Arc::new(DataType::Float64), + Volatility::Immutable, + Arc::new(|_| Ok(Box::::default())), + Arc::new(vec![DataType::UInt64, DataType::Float64]), + ); + + ctx.register_udaf(my_avg); + + // doesn't work as it was registered as non lowercase + let err = ctx.sql("SELECT MY_AVG(i) FROM t").await.unwrap_err(); + assert!(err + .to_string() + .contains("Error during planning: Invalid function \'my_avg\'")); + + // Can call it if you put quotes + let result = ctx + .sql("SELECT \"MY_AVG\"(i) FROM t") + .await? + .collect() + .await?; + + let expected = [ + "+-------------+", + "| MY_AVG(t.i) |", + "+-------------+", + "| 1.0 |", + "+-------------+", + ]; + assert_batches_eq!(expected, &result); + + Ok(()) +} + /// Returns an context with a table "t" and the "first" and "time_sum" /// aggregate functions registered. /// diff --git a/datafusion/core/tests/sql/udf.rs b/datafusion/core/tests/user_defined/user_defined_scalar_functions.rs similarity index 68% rename from datafusion/core/tests/sql/udf.rs rename to datafusion/core/tests/user_defined/user_defined_scalar_functions.rs index 97512d0249c4..1c7e7137290f 100644 --- a/datafusion/core/tests/sql/udf.rs +++ b/datafusion/core/tests/user_defined/user_defined_scalar_functions.rs @@ -15,26 +15,56 @@ // specific language governing permissions and limitations // under the License. -use super::*; use arrow::compute::kernels::numeric::add; +use arrow_array::{ArrayRef, Float64Array, Int32Array, RecordBatch}; +use arrow_schema::{DataType, Field, Schema}; +use datafusion::prelude::*; use datafusion::{ execution::registry::FunctionRegistry, - physical_plan::{expressions::AvgAccumulator, functions::make_scalar_function}, + physical_plan::functions::make_scalar_function, test_util, }; -use datafusion_common::{cast::as_int32_array, ScalarValue}; -use datafusion_expr::{create_udaf, Accumulator, LogicalPlanBuilder}; +use datafusion_common::cast::as_float64_array; +use datafusion_common::{assert_batches_eq, cast::as_int32_array, Result, ScalarValue}; +use datafusion_expr::{ + create_udaf, create_udf, Accumulator, ColumnarValue, LogicalPlanBuilder, Volatility, +}; +use std::sync::Arc; /// test that casting happens on udfs. /// c11 is f32, but `custom_sqrt` requires f64. Casting happens but the logical plan and /// physical plan have the same schema. #[tokio::test] async fn csv_query_custom_udf_with_cast() -> Result<()> { - let ctx = create_ctx(); + let ctx = create_udf_context(); register_aggregate_csv(&ctx).await?; let sql = "SELECT avg(custom_sqrt(c11)) FROM aggregate_test_100"; - let actual = execute(&ctx, sql).await; - let expected = vec![vec!["0.6584408483418833"]]; - assert_float_eq(&expected, &actual); + let actual = plan_and_collect(&ctx, sql).await.unwrap(); + let expected = [ + "+------------------------------------------+", + "| AVG(custom_sqrt(aggregate_test_100.c11)) |", + "+------------------------------------------+", + "| 0.6584408483418833 |", + "+------------------------------------------+", + ]; + assert_batches_eq!(&expected, &actual); + Ok(()) +} + +#[tokio::test] +async fn csv_query_avg_sqrt() -> Result<()> { + let ctx = create_udf_context(); + register_aggregate_csv(&ctx).await?; + // Note it is a different column (c12) than above (c11) + let sql = "SELECT avg(custom_sqrt(c12)) FROM aggregate_test_100"; + let actual = plan_and_collect(&ctx, sql).await.unwrap(); + let expected = [ + "+------------------------------------------+", + "| AVG(custom_sqrt(aggregate_test_100.c12)) |", + "+------------------------------------------+", + "| 0.6706002946036462 |", + "+------------------------------------------+", + ]; + assert_batches_eq!(&expected, &actual); Ok(()) } @@ -212,51 +242,6 @@ async fn scalar_udf_override_built_in_scalar_function() -> Result<()> { Ok(()) } -/// tests the creation, registration and usage of a UDAF -#[tokio::test] -async fn simple_udaf() -> Result<()> { - let schema = Schema::new(vec![Field::new("a", DataType::Int32, false)]); - - let batch1 = RecordBatch::try_new( - Arc::new(schema.clone()), - vec![Arc::new(Int32Array::from(vec![1, 2, 3]))], - )?; - let batch2 = RecordBatch::try_new( - Arc::new(schema.clone()), - vec![Arc::new(Int32Array::from(vec![4, 5]))], - )?; - - let ctx = SessionContext::new(); - - let provider = MemTable::try_new(Arc::new(schema), vec![vec![batch1], vec![batch2]])?; - ctx.register_table("t", Arc::new(provider))?; - - // define a udaf, using a DataFusion's accumulator - let my_avg = create_udaf( - "my_avg", - vec![DataType::Float64], - Arc::new(DataType::Float64), - Volatility::Immutable, - Arc::new(|_| Ok(Box::::default())), - Arc::new(vec![DataType::UInt64, DataType::Float64]), - ); - - ctx.register_udaf(my_avg); - - let result = plan_and_collect(&ctx, "SELECT MY_AVG(a) FROM t").await?; - - let expected = [ - "+-------------+", - "| my_avg(t.a) |", - "+-------------+", - "| 3.0 |", - "+-------------+", - ]; - assert_batches_eq!(expected, &result); - - Ok(()) -} - #[tokio::test] async fn udaf_as_window_func() -> Result<()> { #[derive(Debug)] @@ -314,3 +299,86 @@ async fn udaf_as_window_func() -> Result<()> { assert_eq!(format!("{:?}", dataframe.logical_plan()), expected); Ok(()) } + +#[tokio::test] +async fn case_sensitive_identifiers_user_defined_functions() -> Result<()> { + let ctx = SessionContext::new(); + let arr = Int32Array::from(vec![1]); + let batch = RecordBatch::try_from_iter(vec![("i", Arc::new(arr) as _)])?; + ctx.register_batch("t", batch).unwrap(); + + let myfunc = |args: &[ArrayRef]| Ok(Arc::clone(&args[0])); + let myfunc = make_scalar_function(myfunc); + + ctx.register_udf(create_udf( + "MY_FUNC", + vec![DataType::Int32], + Arc::new(DataType::Int32), + Volatility::Immutable, + myfunc, + )); + + // doesn't work as it was registered with non lowercase + let err = plan_and_collect(&ctx, "SELECT MY_FUNC(i) FROM t") + .await + .unwrap_err(); + assert!(err + .to_string() + .contains("Error during planning: Invalid function \'my_func\'")); + + // Can call it if you put quotes + let result = plan_and_collect(&ctx, "SELECT \"MY_FUNC\"(i) FROM t").await?; + + let expected = [ + "+--------------+", + "| MY_FUNC(t.i) |", + "+--------------+", + "| 1 |", + "+--------------+", + ]; + assert_batches_eq!(expected, &result); + + Ok(()) +} + +fn create_udf_context() -> SessionContext { + let ctx = SessionContext::new(); + // register a custom UDF + ctx.register_udf(create_udf( + "custom_sqrt", + vec![DataType::Float64], + Arc::new(DataType::Float64), + Volatility::Immutable, + Arc::new(custom_sqrt), + )); + + ctx +} + +fn custom_sqrt(args: &[ColumnarValue]) -> Result { + let arg = &args[0]; + if let ColumnarValue::Array(v) = arg { + let input = as_float64_array(v).expect("cast failed"); + let array: Float64Array = input.iter().map(|v| v.map(|x| x.sqrt())).collect(); + Ok(ColumnarValue::Array(Arc::new(array))) + } else { + unimplemented!() + } +} + +async fn register_aggregate_csv(ctx: &SessionContext) -> Result<()> { + let testdata = datafusion::test_util::arrow_test_data(); + let schema = test_util::aggr_test_schema(); + ctx.register_csv( + "aggregate_test_100", + &format!("{testdata}/csv/aggregate_test_100.csv"), + CsvReadOptions::new().schema(&schema), + ) + .await?; + Ok(()) +} + +/// Execute SQL and return results as a RecordBatch +async fn plan_and_collect(ctx: &SessionContext, sql: &str) -> Result> { + ctx.sql(sql).await?.collect().await +} From 2582b8a7c1dae7830aac939eed60107c147a8fdc Mon Sep 17 00:00:00 2001 From: Yijie Shen Date: Tue, 3 Oct 2023 09:40:44 -0700 Subject: [PATCH 022/572] Minor: fix CI failure due to Cargo.lock in cli (#7733) --- datafusion-cli/Cargo.lock | 45 ++++++++++++++++++++++++++++++--------- 1 file changed, 35 insertions(+), 10 deletions(-) diff --git a/datafusion-cli/Cargo.lock b/datafusion-cli/Cargo.lock index 1235c0b740bc..e442393ab08f 100644 --- a/datafusion-cli/Cargo.lock +++ b/datafusion-cli/Cargo.lock @@ -93,6 +93,7 @@ version = "0.16.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "ceb7c683b2f8f40970b70e39ff8be514c95b96fcb9c4af87e1ed2cb2e10801a0" dependencies = [ + "bzip2", "crc32fast", "digest", "lazy_static", @@ -110,6 +111,8 @@ dependencies = [ "thiserror", "typed-builder", "uuid", + "xz2", + "zstd", ] [[package]] @@ -1045,9 +1048,9 @@ dependencies = [ [[package]] name = "csv" -version = "1.2.2" +version = "1.3.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "626ae34994d3d8d668f4269922248239db4ae42d538b14c398b74a52208e8086" +checksum = "ac574ff4d437a7b5ad237ef331c17ccca63c46479e5b5453eb8e10bb99a759fe" dependencies = [ "csv-core", "itoa", @@ -1057,9 +1060,9 @@ dependencies = [ [[package]] name = "csv-core" -version = "0.1.10" +version = "0.1.11" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "2b2466559f260f48ad25fe6317b3c8dac77b5bdb5763ac7d9d6103530663bc90" +checksum = "5efa2b3d7902f4b634a20cae3c9c4e6209dc4779feb6863329607560143efa70" dependencies = [ "memchr", ] @@ -1417,9 +1420,9 @@ checksum = "5443807d6dff69373d433ab9ef5378ad8df50ca6298caf15de6e52e24aaf54d5" [[package]] name = "errno" -version = "0.3.3" +version = "0.3.4" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "136526188508e25c6fef639d7927dfb3e0e3084488bf202267829cf7fc23dbdd" +checksum = "add4f07d43996f76ef320709726a556a9d4f965d9410d8d0271132d2f8293480" dependencies = [ "errno-dragonfly", "libc", @@ -2122,9 +2125,9 @@ dependencies = [ [[package]] name = "memchr" -version = "2.6.3" +version = "2.6.4" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "8f232d6ef707e1956a43342693d2a31e72989554d58299d7a88738cc95b0d35c" +checksum = "f665ee40bc4a3c5590afb1e9677db74a508659dfd71e126420da8274909a0167" [[package]] name = "mimalloc" @@ -2717,9 +2720,9 @@ checksum = "dbb5fb1acd8a1a18b3dd5be62d25485eb770e05afb408a9627d14d451bae12da" [[package]] name = "reqwest" -version = "0.11.20" +version = "0.11.22" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "3e9ad3fe7488d7e34558a2033d45a0c90b72d97b4f80705666fea71472e2e6a1" +checksum = "046cd98826c46c2ac8ddecae268eb5c2e58628688a5fc7a2643704a73faba95b" dependencies = [ "base64", "bytes", @@ -2743,6 +2746,7 @@ dependencies = [ "serde", "serde_json", "serde_urlencoded", + "system-configuration", "tokio", "tokio-rustls 0.24.1", "tokio-util", @@ -3227,6 +3231,27 @@ dependencies = [ "unicode-ident", ] +[[package]] +name = "system-configuration" +version = "0.5.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "ba3a3adc5c275d719af8cb4272ea1c4a6d668a777f37e115f6d11ddbc1c8e0e7" +dependencies = [ + "bitflags 1.3.2", + "core-foundation", + "system-configuration-sys", +] + +[[package]] +name = "system-configuration-sys" +version = "0.5.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "a75fb188eb626b924683e3b95e3a48e63551fcfb51949de2f06a9d91dbee93c9" +dependencies = [ + "core-foundation-sys", + "libc", +] + [[package]] name = "tempfile" version = "3.8.0" From a64f36d8fbce35cb7146271fb5d9ab98a9f094e6 Mon Sep 17 00:00:00 2001 From: mwish <1506118561@qq.com> Date: Wed, 4 Oct 2023 04:46:51 +0800 Subject: [PATCH 023/572] MINOR: change file to column index in page_filter trace log (#7730) * MINOR: change file to column index in page_filter trace log * apply suggestion from review: add comma --- .../physical_plan/parquet/page_filter.rs | 20 ++++++++++--------- 1 file changed, 11 insertions(+), 9 deletions(-) diff --git a/datafusion/core/src/datasource/physical_plan/parquet/page_filter.rs b/datafusion/core/src/datasource/physical_plan/parquet/page_filter.rs index e5c1d8feb0ab..b5b5f154f7a0 100644 --- a/datafusion/core/src/datasource/physical_plan/parquet/page_filter.rs +++ b/datafusion/core/src/datasource/physical_plan/parquet/page_filter.rs @@ -147,17 +147,19 @@ impl PagePruningPredicate { let file_offset_indexes = file_metadata.offset_index(); let file_page_indexes = file_metadata.column_index(); - let (file_offset_indexes, file_page_indexes) = - match (file_offset_indexes, file_page_indexes) { - (Some(o), Some(i)) => (o, i), - _ => { - trace!( - "skip page pruning due to lack of indexes. Have offset: {} file: {}", + let (file_offset_indexes, file_page_indexes) = match ( + file_offset_indexes, + file_page_indexes, + ) { + (Some(o), Some(i)) => (o, i), + _ => { + trace!( + "skip page pruning due to lack of indexes. Have offset: {}, column index: {}", file_offset_indexes.is_some(), file_page_indexes.is_some() ); - return Ok(None); - } - }; + return Ok(None); + } + }; let mut row_selections = Vec::with_capacity(page_index_predicates.len()); for predicate in page_index_predicates { From 018ffbe14383256868e7298b8f3fe4d3ab841639 Mon Sep 17 00:00:00 2001 From: Martin Hilton Date: Wed, 4 Oct 2023 15:09:53 +0100 Subject: [PATCH 024/572] preserve array type / timezone in `date_bin` and `date_trunc` functions (#7729) * preserve array type in date_bin and date_trunc functions The result type of date_bin and date_trunc never includes any timezone information. Change this such that the timezone of the resulting array from these functions is copied from the input array. * Update datafusion/expr/src/built_in_function.rs Co-authored-by: Alex Huang * fix: syntax error * fix: datafusion-cli cargo update * review suggestions Add some additional tests suggested in code reviews. * fix formatting --------- Co-authored-by: Alex Huang --- datafusion/expr/src/built_in_function.rs | 17 +- .../physical-expr/src/datetime_expressions.rs | 299 +++++++++++++++++- .../sqllogictest/test_files/timestamps.slt | 56 ++++ 3 files changed, 355 insertions(+), 17 deletions(-) diff --git a/datafusion/expr/src/built_in_function.rs b/datafusion/expr/src/built_in_function.rs index 58d84545dbb6..70514f52d5f4 100644 --- a/datafusion/expr/src/built_in_function.rs +++ b/datafusion/expr/src/built_in_function.rs @@ -618,13 +618,20 @@ impl BuiltinScalarFunction { BuiltinScalarFunction::ConcatWithSeparator => Ok(Utf8), BuiltinScalarFunction::DatePart => Ok(Float64), BuiltinScalarFunction::DateBin | BuiltinScalarFunction::DateTrunc => { - match input_expr_types[1] { - Timestamp(Nanosecond, _) | Utf8 | Null => { + match &input_expr_types[1] { + Timestamp(Nanosecond, None) | Utf8 | Null => { Ok(Timestamp(Nanosecond, None)) } - Timestamp(Microsecond, _) => Ok(Timestamp(Microsecond, None)), - Timestamp(Millisecond, _) => Ok(Timestamp(Millisecond, None)), - Timestamp(Second, _) => Ok(Timestamp(Second, None)), + Timestamp(Nanosecond, tz_opt) => { + Ok(Timestamp(Nanosecond, tz_opt.clone())) + } + Timestamp(Microsecond, tz_opt) => { + Ok(Timestamp(Microsecond, tz_opt.clone())) + } + Timestamp(Millisecond, tz_opt) => { + Ok(Timestamp(Millisecond, tz_opt.clone())) + } + Timestamp(Second, tz_opt) => Ok(Timestamp(Second, tz_opt.clone())), _ => plan_err!( "The {self} function can only accept timestamp as the second arg." ), diff --git a/datafusion/physical-expr/src/datetime_expressions.rs b/datafusion/physical-expr/src/datetime_expressions.rs index 5ce71f4584bb..5cf1c21df5c2 100644 --- a/datafusion/physical-expr/src/datetime_expressions.rs +++ b/datafusion/physical-expr/src/datetime_expressions.rs @@ -433,7 +433,8 @@ pub fn date_trunc(args: &[ColumnarValue]) -> Result { granularity.as_str(), ) }) - .collect::>()?; + .collect::>()? + .with_timezone_opt(tz_opt.clone()); ColumnarValue::Array(Arc::new(array)) } DataType::Timestamp(TimeUnit::Millisecond, tz_opt) => { @@ -449,7 +450,8 @@ pub fn date_trunc(args: &[ColumnarValue]) -> Result { granularity.as_str(), ) }) - .collect::>()?; + .collect::>()? + .with_timezone_opt(tz_opt.clone()); ColumnarValue::Array(Arc::new(array)) } DataType::Timestamp(TimeUnit::Microsecond, tz_opt) => { @@ -465,7 +467,25 @@ pub fn date_trunc(args: &[ColumnarValue]) -> Result { granularity.as_str(), ) }) - .collect::>()?; + .collect::>()? + .with_timezone_opt(tz_opt.clone()); + ColumnarValue::Array(Arc::new(array)) + } + DataType::Timestamp(TimeUnit::Nanosecond, tz_opt) => { + let parsed_tz = parse_tz(tz_opt)?; + let array = as_timestamp_nanosecond_array(array)?; + let array = array + .iter() + .map(|x| { + _date_trunc( + TimeUnit::Nanosecond, + &x, + parsed_tz, + granularity.as_str(), + ) + }) + .collect::>()? + .with_timezone_opt(tz_opt.clone()); ColumnarValue::Array(Arc::new(array)) } _ => { @@ -713,35 +733,39 @@ fn date_bin_impl( )) } ColumnarValue::Array(array) => match array.data_type() { - DataType::Timestamp(TimeUnit::Nanosecond, _) => { + DataType::Timestamp(TimeUnit::Nanosecond, tz_opt) => { let array = as_timestamp_nanosecond_array(array)? .iter() .map(f_nanos) - .collect::(); + .collect::() + .with_timezone_opt(tz_opt.clone()); ColumnarValue::Array(Arc::new(array)) } - DataType::Timestamp(TimeUnit::Microsecond, _) => { + DataType::Timestamp(TimeUnit::Microsecond, tz_opt) => { let array = as_timestamp_microsecond_array(array)? .iter() .map(f_micros) - .collect::(); + .collect::() + .with_timezone_opt(tz_opt.clone()); ColumnarValue::Array(Arc::new(array)) } - DataType::Timestamp(TimeUnit::Millisecond, _) => { + DataType::Timestamp(TimeUnit::Millisecond, tz_opt) => { let array = as_timestamp_millisecond_array(array)? .iter() .map(f_millis) - .collect::(); + .collect::() + .with_timezone_opt(tz_opt.clone()); ColumnarValue::Array(Arc::new(array)) } - DataType::Timestamp(TimeUnit::Second, _) => { + DataType::Timestamp(TimeUnit::Second, tz_opt) => { let array = as_timestamp_second_array(array)? .iter() .map(f_secs) - .collect::(); + .collect::() + .with_timezone_opt(tz_opt.clone()); ColumnarValue::Array(Arc::new(array)) } @@ -925,7 +949,9 @@ where mod tests { use std::sync::Arc; - use arrow::array::{ArrayRef, Int64Array, IntervalDayTimeArray, StringBuilder}; + use arrow::array::{ + as_primitive_array, ArrayRef, Int64Array, IntervalDayTimeArray, StringBuilder, + }; use super::*; @@ -1051,6 +1077,125 @@ mod tests { }); } + #[test] + fn test_date_trunc_timezones() { + let cases = vec![ + ( + vec![ + "2020-09-08T00:00:00Z", + "2020-09-08T01:00:00Z", + "2020-09-08T02:00:00Z", + "2020-09-08T03:00:00Z", + "2020-09-08T04:00:00Z", + ], + Some("+00".into()), + vec![ + "2020-09-08T00:00:00Z", + "2020-09-08T00:00:00Z", + "2020-09-08T00:00:00Z", + "2020-09-08T00:00:00Z", + "2020-09-08T00:00:00Z", + ], + ), + ( + vec![ + "2020-09-08T00:00:00Z", + "2020-09-08T01:00:00Z", + "2020-09-08T02:00:00Z", + "2020-09-08T03:00:00Z", + "2020-09-08T04:00:00Z", + ], + None, + vec![ + "2020-09-08T00:00:00Z", + "2020-09-08T00:00:00Z", + "2020-09-08T00:00:00Z", + "2020-09-08T00:00:00Z", + "2020-09-08T00:00:00Z", + ], + ), + ( + vec![ + "2020-09-08T00:00:00Z", + "2020-09-08T01:00:00Z", + "2020-09-08T02:00:00Z", + "2020-09-08T03:00:00Z", + "2020-09-08T04:00:00Z", + ], + Some("-02".into()), + vec![ + "2020-09-07T02:00:00Z", + "2020-09-07T02:00:00Z", + "2020-09-08T02:00:00Z", + "2020-09-08T02:00:00Z", + "2020-09-08T02:00:00Z", + ], + ), + ( + vec![ + "2020-09-08T00:00:00+05", + "2020-09-08T01:00:00+05", + "2020-09-08T02:00:00+05", + "2020-09-08T03:00:00+05", + "2020-09-08T04:00:00+05", + ], + Some("+05".into()), + vec![ + "2020-09-08T00:00:00+05", + "2020-09-08T00:00:00+05", + "2020-09-08T00:00:00+05", + "2020-09-08T00:00:00+05", + "2020-09-08T00:00:00+05", + ], + ), + ( + vec![ + "2020-09-08T00:00:00+08", + "2020-09-08T01:00:00+08", + "2020-09-08T02:00:00+08", + "2020-09-08T03:00:00+08", + "2020-09-08T04:00:00+08", + ], + Some("+08".into()), + vec![ + "2020-09-08T00:00:00+08", + "2020-09-08T00:00:00+08", + "2020-09-08T00:00:00+08", + "2020-09-08T00:00:00+08", + "2020-09-08T00:00:00+08", + ], + ), + ]; + + cases.iter().for_each(|(original, tz_opt, expected)| { + let input = original + .iter() + .map(|s| Some(string_to_timestamp_nanos(s).unwrap())) + .collect::() + .with_timezone_opt(tz_opt.clone()); + let right = expected + .iter() + .map(|s| Some(string_to_timestamp_nanos(s).unwrap())) + .collect::() + .with_timezone_opt(tz_opt.clone()); + let result = date_trunc(&[ + ColumnarValue::Scalar(ScalarValue::Utf8(Some("day".to_string()))), + ColumnarValue::Array(Arc::new(input)), + ]) + .unwrap(); + if let ColumnarValue::Array(result) = result { + assert_eq!( + result.data_type(), + &DataType::Timestamp(TimeUnit::Nanosecond, tz_opt.clone()) + ); + let left = as_primitive_array::(&result); + assert_eq!(left, &right); + } else { + panic!("unexpected column type"); + } + }); + } + #[test] fn test_date_bin_single() { use chrono::Duration; @@ -1252,6 +1397,136 @@ mod tests { ); } + #[test] + fn test_date_bin_timezones() { + let cases = vec![ + ( + vec![ + "2020-09-08T00:00:00Z", + "2020-09-08T01:00:00Z", + "2020-09-08T02:00:00Z", + "2020-09-08T03:00:00Z", + "2020-09-08T04:00:00Z", + ], + Some("+00".into()), + "1970-01-01T00:00:00Z", + vec![ + "2020-09-08T00:00:00Z", + "2020-09-08T00:00:00Z", + "2020-09-08T00:00:00Z", + "2020-09-08T00:00:00Z", + "2020-09-08T00:00:00Z", + ], + ), + ( + vec![ + "2020-09-08T00:00:00Z", + "2020-09-08T01:00:00Z", + "2020-09-08T02:00:00Z", + "2020-09-08T03:00:00Z", + "2020-09-08T04:00:00Z", + ], + None, + "1970-01-01T00:00:00Z", + vec![ + "2020-09-08T00:00:00Z", + "2020-09-08T00:00:00Z", + "2020-09-08T00:00:00Z", + "2020-09-08T00:00:00Z", + "2020-09-08T00:00:00Z", + ], + ), + ( + vec![ + "2020-09-08T00:00:00Z", + "2020-09-08T01:00:00Z", + "2020-09-08T02:00:00Z", + "2020-09-08T03:00:00Z", + "2020-09-08T04:00:00Z", + ], + Some("-02".into()), + "1970-01-01T00:00:00Z", + vec![ + "2020-09-08T00:00:00Z", + "2020-09-08T00:00:00Z", + "2020-09-08T00:00:00Z", + "2020-09-08T00:00:00Z", + "2020-09-08T00:00:00Z", + ], + ), + ( + vec![ + "2020-09-08T00:00:00+05", + "2020-09-08T01:00:00+05", + "2020-09-08T02:00:00+05", + "2020-09-08T03:00:00+05", + "2020-09-08T04:00:00+05", + ], + Some("+05".into()), + "1970-01-01T00:00:00+05", + vec![ + "2020-09-08T00:00:00+05", + "2020-09-08T00:00:00+05", + "2020-09-08T00:00:00+05", + "2020-09-08T00:00:00+05", + "2020-09-08T00:00:00+05", + ], + ), + ( + vec![ + "2020-09-08T00:00:00+08", + "2020-09-08T01:00:00+08", + "2020-09-08T02:00:00+08", + "2020-09-08T03:00:00+08", + "2020-09-08T04:00:00+08", + ], + Some("+08".into()), + "1970-01-01T00:00:00+08", + vec![ + "2020-09-08T00:00:00+08", + "2020-09-08T00:00:00+08", + "2020-09-08T00:00:00+08", + "2020-09-08T00:00:00+08", + "2020-09-08T00:00:00+08", + ], + ), + ]; + + cases + .iter() + .for_each(|(original, tz_opt, origin, expected)| { + let input = original + .iter() + .map(|s| Some(string_to_timestamp_nanos(s).unwrap())) + .collect::() + .with_timezone_opt(tz_opt.clone()); + let right = expected + .iter() + .map(|s| Some(string_to_timestamp_nanos(s).unwrap())) + .collect::() + .with_timezone_opt(tz_opt.clone()); + let result = date_bin(&[ + ColumnarValue::Scalar(ScalarValue::new_interval_dt(1, 0)), + ColumnarValue::Array(Arc::new(input)), + ColumnarValue::Scalar(ScalarValue::TimestampNanosecond( + Some(string_to_timestamp_nanos(origin).unwrap()), + tz_opt.clone(), + )), + ]) + .unwrap(); + if let ColumnarValue::Array(result) = result { + assert_eq!( + result.data_type(), + &DataType::Timestamp(TimeUnit::Nanosecond, tz_opt.clone()) + ); + let left = as_primitive_array::(&result); + assert_eq!(left, &right); + } else { + panic!("unexpected column type"); + } + }); + } + #[test] fn to_timestamp_invalid_input_type() -> Result<()> { // pass the wrong type of input array to to_timestamp and test diff --git a/datafusion/sqllogictest/test_files/timestamps.slt b/datafusion/sqllogictest/test_files/timestamps.slt index bb06c569f081..edafe18caab5 100644 --- a/datafusion/sqllogictest/test_files/timestamps.slt +++ b/datafusion/sqllogictest/test_files/timestamps.slt @@ -1702,3 +1702,59 @@ SELECT TIMESTAMPTZ '2023-03-11 02:00:00 America/Los_Angeles' as ts_geo # postgresql: accepts statement error SELECT TIMESTAMPTZ '2023-03-12 02:00:00 America/Los_Angeles' as ts_geo + + + +########## +## Timezone column tests +########## + +# create a table with a non-UTC time zone. +statement ok +SET TIME ZONE = '+05:00' + +statement ok +CREATE TABLE foo (time TIMESTAMPTZ) AS VALUES + ('2020-01-01T00:00:00+05:00'), + ('2020-01-01T01:00:00+05:00'), + ('2020-01-01T02:00:00+05:00'), + ('2020-01-01T03:00:00+05:00') + +statement ok +SET TIME ZONE = '+00' + +# verify column type +query T +SELECT arrow_typeof(time) FROM foo LIMIT 1 +---- +Timestamp(Nanosecond, Some("+05:00")) + +# check date_trunc +query P +SELECT date_trunc('day', time) FROM foo +---- +2020-01-01T00:00:00+05:00 +2020-01-01T00:00:00+05:00 +2020-01-01T00:00:00+05:00 +2020-01-01T00:00:00+05:00 + +# verify date_trunc column type +query T +SELECT arrow_typeof(date_trunc('day', time)) FROM foo LIMIT 1 +---- +Timestamp(Nanosecond, Some("+05:00")) + +# check date_bin +query P +SELECT date_bin(INTERVAL '1 day', time, '1970-01-01T00:00:00+05:00') FROM foo +---- +2020-01-01T00:00:00+05:00 +2020-01-01T00:00:00+05:00 +2020-01-01T00:00:00+05:00 +2020-01-01T00:00:00+05:00 + +# verify date_trunc column type +query T +SELECT arrow_typeof(date_bin(INTERVAL '1 day', time, '1970-01-01T00:00:00+05:00')) FROM foo LIMIT 1 +---- +Timestamp(Nanosecond, Some("+05:00")) From 91a8025a0b0939b426d9a3e0867b14b5b9b15681 Mon Sep 17 00:00:00 2001 From: Kirill Zaborsky Date: Wed, 4 Oct 2023 17:14:47 +0300 Subject: [PATCH 025/572] Remove redundant is_numeric for DataType (#7734) It's available in arrow-rs since version 3.0.0 Resolves #1613 --- datafusion/expr/src/type_coercion/binary.rs | 27 ++++++++++--------- datafusion/expr/src/type_coercion/mod.rs | 9 ------- .../optimizer/src/analyzer/type_coercion.rs | 4 +-- 3 files changed, 17 insertions(+), 23 deletions(-) diff --git a/datafusion/expr/src/type_coercion/binary.rs b/datafusion/expr/src/type_coercion/binary.rs index 64f814cd958b..19bb16651da5 100644 --- a/datafusion/expr/src/type_coercion/binary.rs +++ b/datafusion/expr/src/type_coercion/binary.rs @@ -27,7 +27,6 @@ use arrow::datatypes::{ use datafusion_common::Result; use datafusion_common::{plan_err, DataFusionError}; -use crate::type_coercion::is_numeric; use crate::Operator; /// The type signature of an instantiation of binary expression @@ -310,10 +309,10 @@ pub fn comparison_coercion(lhs_type: &DataType, rhs_type: &DataType) -> Option Option { use arrow::datatypes::DataType::*; match (lhs_type, rhs_type) { - (Utf8, _) if is_numeric(rhs_type) => Some(Utf8), - (LargeUtf8, _) if is_numeric(rhs_type) => Some(LargeUtf8), - (_, Utf8) if is_numeric(lhs_type) => Some(Utf8), - (_, LargeUtf8) if is_numeric(lhs_type) => Some(LargeUtf8), + (Utf8, _) if rhs_type.is_numeric() => Some(Utf8), + (LargeUtf8, _) if rhs_type.is_numeric() => Some(LargeUtf8), + (_, Utf8) if lhs_type.is_numeric() => Some(Utf8), + (_, LargeUtf8) if lhs_type.is_numeric() => Some(LargeUtf8), _ => None, } } @@ -365,7 +364,7 @@ fn comparison_binary_numeric_coercion( rhs_type: &DataType, ) -> Option { use arrow::datatypes::DataType::*; - if !is_numeric(lhs_type) || !is_numeric(rhs_type) { + if !lhs_type.is_numeric() || !rhs_type.is_numeric() { return None; }; @@ -563,14 +562,18 @@ fn create_decimal256_type(precision: u8, scale: i8) -> DataType { fn both_numeric_or_null_and_numeric(lhs_type: &DataType, rhs_type: &DataType) -> bool { use arrow::datatypes::DataType::*; match (lhs_type, rhs_type) { - (_, Null) => is_numeric(lhs_type), - (Null, _) => is_numeric(rhs_type), + (_, Null) => lhs_type.is_numeric(), + (Null, _) => rhs_type.is_numeric(), (Dictionary(_, lhs_value_type), Dictionary(_, rhs_value_type)) => { - is_numeric(lhs_value_type) && is_numeric(rhs_value_type) + lhs_value_type.is_numeric() && rhs_value_type.is_numeric() } - (Dictionary(_, value_type), _) => is_numeric(value_type) && is_numeric(rhs_type), - (_, Dictionary(_, value_type)) => is_numeric(lhs_type) && is_numeric(value_type), - _ => is_numeric(lhs_type) && is_numeric(rhs_type), + (Dictionary(_, value_type), _) => { + value_type.is_numeric() && rhs_type.is_numeric() + } + (_, Dictionary(_, value_type)) => { + lhs_type.is_numeric() && value_type.is_numeric() + } + _ => lhs_type.is_numeric() && rhs_type.is_numeric(), } } diff --git a/datafusion/expr/src/type_coercion/mod.rs b/datafusion/expr/src/type_coercion/mod.rs index d72d9c50edd2..86005da3dafa 100644 --- a/datafusion/expr/src/type_coercion/mod.rs +++ b/datafusion/expr/src/type_coercion/mod.rs @@ -58,15 +58,6 @@ pub fn is_null(dt: &DataType) -> bool { *dt == DataType::Null } -/// Determine whether the given data type `dt` represents numeric values. -pub fn is_numeric(dt: &DataType) -> bool { - is_signed_numeric(dt) - || matches!( - dt, - DataType::UInt8 | DataType::UInt16 | DataType::UInt32 | DataType::UInt64 - ) -} - /// Determine whether the given data type `dt` is a `Timestamp`. pub fn is_timestamp(dt: &DataType) -> bool { matches!(dt, DataType::Timestamp(_, _)) diff --git a/datafusion/optimizer/src/analyzer/type_coercion.rs b/datafusion/optimizer/src/analyzer/type_coercion.rs index 5e239f8e9934..3b866173859c 100644 --- a/datafusion/optimizer/src/analyzer/type_coercion.rs +++ b/datafusion/optimizer/src/analyzer/type_coercion.rs @@ -41,7 +41,7 @@ use datafusion_expr::type_coercion::functions::data_types; use datafusion_expr::type_coercion::other::{ get_coerce_type_for_case_expression, get_coerce_type_for_list, }; -use datafusion_expr::type_coercion::{is_datetime, is_numeric, is_utf8_or_large_utf8}; +use datafusion_expr::type_coercion::{is_datetime, is_utf8_or_large_utf8}; use datafusion_expr::{ is_false, is_not_false, is_not_true, is_not_unknown, is_true, is_unknown, type_coercion, window_function, AggregateFunction, BuiltinScalarFunction, Expr, @@ -496,7 +496,7 @@ fn coerce_window_frame( let target_type = match window_frame.units { WindowFrameUnits::Range => { if let Some(col_type) = current_types.first() { - if is_numeric(col_type) || is_utf8_or_large_utf8(col_type) { + if col_type.is_numeric() || is_utf8_or_large_utf8(col_type) { col_type } else if is_datetime(col_type) { &DataType::Interval(IntervalUnit::MonthDayNano) From 5361d2ee090156eed4a1d91f6530695588a9405a Mon Sep 17 00:00:00 2001 From: Samrose Date: Wed, 4 Oct 2023 09:59:09 -0700 Subject: [PATCH 026/572] fix: avro_to_arrow: Handle avro nested nullable struct (union) (#7663) MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Corrects handling of a nullable struct union. Signed-off-by: 🐼 Samrose Ahmed 🐼 --- datafusion/core/Cargo.toml | 1 + .../avro_to_arrow/arrow_array_reader.rs | 669 ++++++++++++++++-- .../src/datasource/avro_to_arrow/schema.rs | 60 +- datafusion/sqllogictest/test_files/avro.slt | 8 +- 4 files changed, 670 insertions(+), 68 deletions(-) diff --git a/datafusion/core/Cargo.toml b/datafusion/core/Cargo.toml index d84d6a13c336..1db5d55baf10 100644 --- a/datafusion/core/Cargo.toml +++ b/datafusion/core/Cargo.toml @@ -109,6 +109,7 @@ rand_distr = "0.4.3" regex = "1.5.4" rstest = "0.18.0" rust_decimal = { version = "1.27.0", features = ["tokio-pg"] } +serde_json = "1" test-utils = { path = "../../test-utils" } thiserror = "1.0.37" tokio-postgres = "0.7.7" diff --git a/datafusion/core/src/datasource/avro_to_arrow/arrow_array_reader.rs b/datafusion/core/src/datasource/avro_to_arrow/arrow_array_reader.rs index f983e26d48a4..fd91ea1cc538 100644 --- a/datafusion/core/src/datasource/avro_to_arrow/arrow_array_reader.rs +++ b/datafusion/core/src/datasource/avro_to_arrow/arrow_array_reader.rs @@ -82,7 +82,7 @@ impl<'a, R: Read> AvroArrowArrayReader<'a, R> { fields, mut lookup, .. }) => { for field in fields { - Self::child_schema_lookup(&field.schema, &mut lookup)?; + Self::child_schema_lookup(&field.name, &field.schema, &mut lookup)?; } Ok(lookup) } @@ -93,27 +93,51 @@ impl<'a, R: Read> AvroArrowArrayReader<'a, R> { } fn child_schema_lookup<'b>( + parent_field_name: &str, schema: &AvroSchema, schema_lookup: &'b mut BTreeMap, ) -> Result<&'b BTreeMap> { match schema { - AvroSchema::Record(RecordSchema { - name, - fields, - lookup, - .. - }) => { + AvroSchema::Union(us) => { + let has_nullable = us + .find_schema_with_known_schemata::( + &Value::Null, + None, + &None, + ) + .is_some(); + let sub_schemas = us.variants(); + if has_nullable && sub_schemas.len() == 2 { + if let Some(sub_schema) = + sub_schemas.iter().find(|&s| !matches!(s, AvroSchema::Null)) + { + Self::child_schema_lookup( + parent_field_name, + sub_schema, + schema_lookup, + )?; + } + } + } + AvroSchema::Record(RecordSchema { fields, lookup, .. }) => { lookup.iter().for_each(|(field_name, pos)| { schema_lookup - .insert(format!("{}.{}", name.fullname(None), field_name), *pos); + .insert(format!("{}.{}", parent_field_name, field_name), *pos); }); for field in fields { - Self::child_schema_lookup(&field.schema, schema_lookup)?; + let sub_parent_field_name = + format!("{}.{}", parent_field_name, field.name); + Self::child_schema_lookup( + &sub_parent_field_name, + &field.schema, + schema_lookup, + )?; } } AvroSchema::Array(schema) => { - Self::child_schema_lookup(schema, schema_lookup)?; + let sub_parent_field_name = format!("{}.element", parent_field_name); + Self::child_schema_lookup(&sub_parent_field_name, schema, schema_lookup)?; } _ => (), } @@ -147,7 +171,8 @@ impl<'a, R: Read> AvroArrowArrayReader<'a, R> { let rows = rows.iter().collect::>>(); let projection = self.projection.clone().unwrap_or_default(); - let arrays = self.build_struct_array(&rows, self.schema.fields(), &projection); + let arrays = + self.build_struct_array(&rows, "", self.schema.fields(), &projection); let projected_fields = if projection.is_empty() { self.schema.fields().clone() } else { @@ -305,6 +330,7 @@ impl<'a, R: Read> AvroArrowArrayReader<'a, R> { for row in rows { if let Some(value) = self.field_lookup(col_name, row) { + let value = maybe_resolve_union(value); // value can be an array or a scalar let vals: Vec> = if let Value::String(v) = value { vec![Some(v.to_string())] @@ -444,6 +470,7 @@ impl<'a, R: Read> AvroArrowArrayReader<'a, R> { /// Build a nested GenericListArray from a list of unnested `Value`s fn build_nested_list_array( &self, + parent_field_name: &str, rows: &[&Value], list_field: &Field, ) -> ArrowResult { @@ -530,13 +557,19 @@ impl<'a, R: Read> AvroArrowArrayReader<'a, R> { .collect::() .into_data(), DataType::List(field) => { - let child = - self.build_nested_list_array::(&flatten_values(rows), field)?; + let child = self.build_nested_list_array::( + parent_field_name, + &flatten_values(rows), + field, + )?; child.to_data() } DataType::LargeList(field) => { - let child = - self.build_nested_list_array::(&flatten_values(rows), field)?; + let child = self.build_nested_list_array::( + parent_field_name, + &flatten_values(rows), + field, + )?; child.to_data() } DataType::Struct(fields) => { @@ -554,16 +587,22 @@ impl<'a, R: Read> AvroArrowArrayReader<'a, R> { let null_struct_array = vec![("null".to_string(), Value::Null)]; let rows: Vec<&Vec<(String, Value)>> = rows .iter() + .map(|v| maybe_resolve_union(v)) .flat_map(|row| { if let Value::Array(values) = row { - values.iter().for_each(|_| { - bit_util::set_bit(&mut null_buffer, struct_index); - struct_index += 1; - }); values .iter() + .map(maybe_resolve_union) .map(|v| match v { - Value::Record(record) => record, + Value::Record(record) => { + bit_util::set_bit(&mut null_buffer, struct_index); + struct_index += 1; + record + } + Value::Null => { + struct_index += 1; + &null_struct_array + } other => panic!("expected Record, got {other:?}"), }) .collect::>>() @@ -573,7 +612,11 @@ impl<'a, R: Read> AvroArrowArrayReader<'a, R> { } }) .collect(); - let arrays = self.build_struct_array(&rows, fields, &[])?; + + let sub_parent_field_name = + format!("{}.{}", parent_field_name, list_field.name()); + let arrays = + self.build_struct_array(&rows, &sub_parent_field_name, fields, &[])?; let data_type = DataType::Struct(fields.clone()); ArrayDataBuilder::new(data_type) .len(rows.len()) @@ -610,6 +653,7 @@ impl<'a, R: Read> AvroArrowArrayReader<'a, R> { fn build_struct_array( &self, rows: RecordSlice, + parent_field_name: &str, struct_fields: &Fields, projection: &[String], ) -> ArrowResult> { @@ -617,78 +661,83 @@ impl<'a, R: Read> AvroArrowArrayReader<'a, R> { .iter() .filter(|field| projection.is_empty() || projection.contains(field.name())) .map(|field| { + let field_path = if parent_field_name.is_empty() { + field.name().to_string() + } else { + format!("{}.{}", parent_field_name, field.name()) + }; let arr = match field.data_type() { DataType::Null => Arc::new(NullArray::new(rows.len())) as ArrayRef, - DataType::Boolean => self.build_boolean_array(rows, field.name()), + DataType::Boolean => self.build_boolean_array(rows, &field_path), DataType::Float64 => { - self.build_primitive_array::(rows, field.name()) + self.build_primitive_array::(rows, &field_path) } DataType::Float32 => { - self.build_primitive_array::(rows, field.name()) + self.build_primitive_array::(rows, &field_path) } DataType::Int64 => { - self.build_primitive_array::(rows, field.name()) + self.build_primitive_array::(rows, &field_path) } DataType::Int32 => { - self.build_primitive_array::(rows, field.name()) + self.build_primitive_array::(rows, &field_path) } DataType::Int16 => { - self.build_primitive_array::(rows, field.name()) + self.build_primitive_array::(rows, &field_path) } DataType::Int8 => { - self.build_primitive_array::(rows, field.name()) + self.build_primitive_array::(rows, &field_path) } DataType::UInt64 => { - self.build_primitive_array::(rows, field.name()) + self.build_primitive_array::(rows, &field_path) } DataType::UInt32 => { - self.build_primitive_array::(rows, field.name()) + self.build_primitive_array::(rows, &field_path) } DataType::UInt16 => { - self.build_primitive_array::(rows, field.name()) + self.build_primitive_array::(rows, &field_path) } DataType::UInt8 => { - self.build_primitive_array::(rows, field.name()) + self.build_primitive_array::(rows, &field_path) } // TODO: this is incomplete DataType::Timestamp(unit, _) => match unit { TimeUnit::Second => self .build_primitive_array::( rows, - field.name(), + &field_path, ), TimeUnit::Microsecond => self .build_primitive_array::( rows, - field.name(), + &field_path, ), TimeUnit::Millisecond => self .build_primitive_array::( rows, - field.name(), + &field_path, ), TimeUnit::Nanosecond => self .build_primitive_array::( rows, - field.name(), + &field_path, ), }, DataType::Date64 => { - self.build_primitive_array::(rows, field.name()) + self.build_primitive_array::(rows, &field_path) } DataType::Date32 => { - self.build_primitive_array::(rows, field.name()) + self.build_primitive_array::(rows, &field_path) } DataType::Time64(unit) => match unit { TimeUnit::Microsecond => self .build_primitive_array::( rows, - field.name(), + &field_path, ), TimeUnit::Nanosecond => self .build_primitive_array::( rows, - field.name(), + &field_path, ), t => { return Err(ArrowError::SchemaError(format!( @@ -698,14 +747,11 @@ impl<'a, R: Read> AvroArrowArrayReader<'a, R> { }, DataType::Time32(unit) => match unit { TimeUnit::Second => self - .build_primitive_array::( - rows, - field.name(), - ), + .build_primitive_array::(rows, &field_path), TimeUnit::Millisecond => self .build_primitive_array::( rows, - field.name(), + &field_path, ), t => { return Err(ArrowError::SchemaError(format!( @@ -716,7 +762,7 @@ impl<'a, R: Read> AvroArrowArrayReader<'a, R> { DataType::Utf8 | DataType::LargeUtf8 => Arc::new( rows.iter() .map(|row| { - let maybe_value = self.field_lookup(field.name(), row); + let maybe_value = self.field_lookup(&field_path, row); match maybe_value { None => Ok(None), Some(v) => resolve_string(v), @@ -728,7 +774,7 @@ impl<'a, R: Read> AvroArrowArrayReader<'a, R> { DataType::Binary | DataType::LargeBinary => Arc::new( rows.iter() .map(|row| { - let maybe_value = self.field_lookup(field.name(), row); + let maybe_value = self.field_lookup(&field_path, row); maybe_value.and_then(resolve_bytes) }) .collect::(), @@ -737,7 +783,7 @@ impl<'a, R: Read> AvroArrowArrayReader<'a, R> { DataType::FixedSizeBinary(ref size) => { Arc::new(FixedSizeBinaryArray::try_from_sparse_iter_with_size( rows.iter().map(|row| { - let maybe_value = self.field_lookup(field.name(), row); + let maybe_value = self.field_lookup(&field_path, row); maybe_value.and_then(|v| resolve_fixed(v, *size as usize)) }), *size, @@ -746,18 +792,19 @@ impl<'a, R: Read> AvroArrowArrayReader<'a, R> { DataType::List(ref list_field) => { match list_field.data_type() { DataType::Dictionary(ref key_ty, _) => { - self.build_wrapped_list_array(rows, field.name(), key_ty)? + self.build_wrapped_list_array(rows, &field_path, key_ty)? } _ => { // extract rows by name let extracted_rows = rows .iter() .map(|row| { - self.field_lookup(field.name(), row) + self.field_lookup(&field_path, row) .unwrap_or(&Value::Null) }) .collect::>(); self.build_nested_list_array::( + &field_path, &extracted_rows, list_field, )? @@ -767,7 +814,7 @@ impl<'a, R: Read> AvroArrowArrayReader<'a, R> { DataType::Dictionary(ref key_ty, ref val_ty) => self .build_string_dictionary_array( rows, - field.name(), + &field_path, key_ty, val_ty, )?, @@ -775,21 +822,31 @@ impl<'a, R: Read> AvroArrowArrayReader<'a, R> { let len = rows.len(); let num_bytes = bit_util::ceil(len, 8); let mut null_buffer = MutableBuffer::from_len_zeroed(num_bytes); + let empty_vec = vec![]; let struct_rows = rows .iter() .enumerate() - .map(|(i, row)| (i, self.field_lookup(field.name(), row))) + .map(|(i, row)| (i, self.field_lookup(&field_path, row))) .map(|(i, v)| { - if let Some(Value::Record(value)) = v { - bit_util::set_bit(&mut null_buffer, i); - value - } else { - panic!("expected struct got {v:?}"); + let v = v.map(maybe_resolve_union); + match v { + Some(Value::Record(value)) => { + bit_util::set_bit(&mut null_buffer, i); + value + } + None | Some(Value::Null) => &empty_vec, + other => { + panic!("expected struct got {other:?}"); + } } }) .collect::>>(); - let arrays = - self.build_struct_array(&struct_rows, fields, &[])?; + let arrays = self.build_struct_array( + &struct_rows, + &field_path, + fields, + &[], + )?; // construct a struct array's data in order to set null buffer let data_type = DataType::Struct(fields.clone()); let data = ArrayDataBuilder::new(data_type) @@ -1019,6 +1076,7 @@ mod test { use crate::arrow::datatypes::{Field, TimeUnit}; use crate::datasource::avro_to_arrow::{Reader, ReaderBuilder}; use arrow::datatypes::DataType; + use datafusion_common::assert_batches_eq; use datafusion_common::cast::{ as_int32_array, as_int64_array, as_list_array, as_timestamp_microsecond_array, }; @@ -1079,7 +1137,7 @@ mod test { let a_array = as_list_array(batch.column(col_id_index)).unwrap(); assert_eq!( *a_array.data_type(), - DataType::List(Arc::new(Field::new("bigint", DataType::Int64, true))) + DataType::List(Arc::new(Field::new("element", DataType::Int64, true))) ); let array = a_array.value(0); assert_eq!(*array.data_type(), DataType::Int64); @@ -1101,6 +1159,497 @@ mod test { assert_eq!(batch.num_rows(), 3); } + #[test] + fn test_complex_list() { + let schema = apache_avro::Schema::parse_str( + r#" + { + "type": "record", + "name": "r1", + "fields": [ + { + "name": "headers", + "type": ["null", { + "type": "array", + "items": ["null",{ + "name":"r2", + "type": "record", + "fields":[ + {"name":"name", "type": ["null", "string"], "default": null}, + {"name":"value", "type": ["null", "string"], "default": null} + ] + }] + }], + "default": null + } + ] + }"#, + ) + .unwrap(); + let r1 = apache_avro::to_value(serde_json::json!({ + "headers": [ + { + "name": "a", + "value": "b" + } + ] + })) + .unwrap() + .resolve(&schema) + .unwrap(); + + let mut w = apache_avro::Writer::new(&schema, vec![]); + w.append(r1).unwrap(); + let bytes = w.into_inner().unwrap(); + + let mut reader = ReaderBuilder::new() + .read_schema() + .with_batch_size(2) + .build(std::io::Cursor::new(bytes)) + .unwrap(); + + let batch = reader.next().unwrap().unwrap(); + assert_eq!(batch.num_rows(), 1); + assert_eq!(batch.num_columns(), 1); + let expected = [ + "+-----------------------+", + "| headers |", + "+-----------------------+", + "| [{name: a, value: b}] |", + "+-----------------------+", + ]; + assert_batches_eq!(expected, &[batch]); + } + + #[test] + fn test_complex_struct() { + let schema = apache_avro::Schema::parse_str( + r#" + { + "type": "record", + "name": "r1", + "fields": [ + { + "name": "dns", + "type": [ + "null", + { + "type": "record", + "name": "r13", + "fields": [ + { + "name": "answers", + "type": [ + "null", + { + "type": "array", + "items": [ + "null", + { + "type": "record", + "name": "r292", + "fields": [ + { + "name": "class", + "type": ["null", "string"], + "default": null + }, + { + "name": "data", + "type": ["null", "string"], + "default": null + }, + { + "name": "name", + "type": ["null", "string"], + "default": null + }, + { + "name": "ttl", + "type": ["null", "long"], + "default": null + }, + { + "name": "type", + "type": ["null", "string"], + "default": null + } + ] + } + ] + } + ], + "default": null + }, + { + "name": "header_flags", + "type": [ + "null", + { + "type": "array", + "items": ["null", "string"] + } + ], + "default": null + }, + { + "name": "id", + "type": ["null", "string"], + "default": null + }, + { + "name": "op_code", + "type": ["null", "string"], + "default": null + }, + { + "name": "question", + "type": [ + "null", + { + "type": "record", + "name": "r288", + "fields": [ + { + "name": "class", + "type": ["null", "string"], + "default": null + }, + { + "name": "name", + "type": ["null", "string"], + "default": null + }, + { + "name": "registered_domain", + "type": ["null", "string"], + "default": null + }, + { + "name": "subdomain", + "type": ["null", "string"], + "default": null + }, + { + "name": "top_level_domain", + "type": ["null", "string"], + "default": null + }, + { + "name": "type", + "type": ["null", "string"], + "default": null + } + ] + } + ], + "default": null + }, + { + "name": "resolved_ip", + "type": [ + "null", + { + "type": "array", + "items": ["null", "string"] + } + ], + "default": null + }, + { + "name": "response_code", + "type": ["null", "string"], + "default": null + }, + { + "name": "type", + "type": ["null", "string"], + "default": null + } + ] + } + ], + "default": null + } + ] + }"#, + ) + .unwrap(); + + let jv1 = serde_json::json!({ + "dns": { + "answers": [ + { + "data": "CHNlY3VyaXR5BnVidW50dQMjb20AAAEAAQAAAAgABLl9vic=", + "type": "1" + }, + { + "data": "CHNlY3VyaXR5BnVidW50dQNjb20AAAEAABAAAAgABLl9viQ=", + "type": "1" + }, + { + "data": "CHNlT3VyaXR5BnVidW50dQNjb20AAAEAAQAAAAgABFu9Wyc=", + "type": "1" + } + ], + "question": { + "name": "security.ubuntu.com", + "type": "A" + }, + "resolved_ip": [ + "67.43.156.1", + "67.43.156.2", + "67.43.156.3" + ], + "response_code": "0" + } + }); + let r1 = apache_avro::to_value(jv1) + .unwrap() + .resolve(&schema) + .unwrap(); + + let mut w = apache_avro::Writer::new(&schema, vec![]); + w.append(r1).unwrap(); + let bytes = w.into_inner().unwrap(); + + let mut reader = ReaderBuilder::new() + .read_schema() + .with_batch_size(1) + .build(std::io::Cursor::new(bytes)) + .unwrap(); + + let batch = reader.next().unwrap().unwrap(); + assert_eq!(batch.num_rows(), 1); + assert_eq!(batch.num_columns(), 1); + + let expected = [ + "+------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+", + "| dns |", + "+------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+", + "| {answers: [{class: , data: CHNlY3VyaXR5BnVidW50dQMjb20AAAEAAQAAAAgABLl9vic=, name: , ttl: , type: 1}, {class: , data: CHNlY3VyaXR5BnVidW50dQNjb20AAAEAABAAAAgABLl9viQ=, name: , ttl: , type: 1}, {class: , data: CHNlT3VyaXR5BnVidW50dQNjb20AAAEAAQAAAAgABFu9Wyc=, name: , ttl: , type: 1}], header_flags: , id: , op_code: , question: {class: , name: security.ubuntu.com, registered_domain: , subdomain: , top_level_domain: , type: A}, resolved_ip: [67.43.156.1, 67.43.156.2, 67.43.156.3], response_code: 0, type: } |", + "+------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+", + ]; + assert_batches_eq!(expected, &[batch]); + } + + #[test] + fn test_deep_nullable_struct() { + let schema = apache_avro::Schema::parse_str( + r#" + { + "type": "record", + "name": "r1", + "fields": [ + { + "name": "col1", + "type": [ + "null", + { + "type": "record", + "name": "r2", + "fields": [ + { + "name": "col2", + "type": [ + "null", + { + "type": "record", + "name": "r3", + "fields": [ + { + "name": "col3", + "type": [ + "null", + { + "type": "record", + "name": "r4", + "fields": [ + { + "name": "col4", + "type": [ + "null", + { + "type": "record", + "name": "r5", + "fields": [ + { + "name": "col5", + "type": ["null", "string"] + } + ] + } + ] + } + ] + } + ] + } + ] + } + ] + } + ] + } + ] + } + ] + } + "#, + ) + .unwrap(); + let r1 = apache_avro::to_value(serde_json::json!({ + "col1": { + "col2": { + "col3": { + "col4": { + "col5": "hello" + } + } + } + } + })) + .unwrap() + .resolve(&schema) + .unwrap(); + let r2 = apache_avro::to_value(serde_json::json!({ + "col1": { + "col2": { + "col3": { + "col4": { + "col5": null + } + } + } + } + })) + .unwrap() + .resolve(&schema) + .unwrap(); + let r3 = apache_avro::to_value(serde_json::json!({ + "col1": { + "col2": { + "col3": null + } + } + })) + .unwrap() + .resolve(&schema) + .unwrap(); + let r4 = apache_avro::to_value(serde_json::json!({ + "col1": null + })) + .unwrap() + .resolve(&schema) + .unwrap(); + + let mut w = apache_avro::Writer::new(&schema, vec![]); + w.append(r1).unwrap(); + w.append(r2).unwrap(); + w.append(r3).unwrap(); + w.append(r4).unwrap(); + let bytes = w.into_inner().unwrap(); + + let mut reader = ReaderBuilder::new() + .read_schema() + .with_batch_size(4) + .build(std::io::Cursor::new(bytes)) + .unwrap(); + + let batch = reader.next().unwrap().unwrap(); + + let expected = [ + "+---------------------------------------+", + "| col1 |", + "+---------------------------------------+", + "| {col2: {col3: {col4: {col5: hello}}}} |", + "| {col2: {col3: {col4: {col5: }}}} |", + "| {col2: {col3: }} |", + "| |", + "+---------------------------------------+", + ]; + assert_batches_eq!(expected, &[batch]); + } + + #[test] + fn test_avro_nullable_struct() { + let schema = apache_avro::Schema::parse_str( + r#" + { + "type": "record", + "name": "r1", + "fields": [ + { + "name": "col1", + "type": [ + "null", + { + "type": "record", + "name": "r2", + "fields": [ + { + "name": "col2", + "type": ["null", "string"] + } + ] + } + ], + "default": null + } + ] + }"#, + ) + .unwrap(); + let r1 = apache_avro::to_value(serde_json::json!({ + "col1": null + })) + .unwrap() + .resolve(&schema) + .unwrap(); + let r2 = apache_avro::to_value(serde_json::json!({ + "col1": { + "col2": "hello" + } + })) + .unwrap() + .resolve(&schema) + .unwrap(); + let r3 = apache_avro::to_value(serde_json::json!({ + "col1": { + "col2": null + } + })) + .unwrap() + .resolve(&schema) + .unwrap(); + + let mut w = apache_avro::Writer::new(&schema, vec![]); + w.append(r1).unwrap(); + w.append(r2).unwrap(); + w.append(r3).unwrap(); + let bytes = w.into_inner().unwrap(); + + let mut reader = ReaderBuilder::new() + .read_schema() + .with_batch_size(3) + .build(std::io::Cursor::new(bytes)) + .unwrap(); + let batch = reader.next().unwrap().unwrap(); + assert_eq!(batch.num_rows(), 3); + assert_eq!(batch.num_columns(), 1); + + let expected = [ + "+---------------+", + "| col1 |", + "+---------------+", + "| |", + "| {col2: hello} |", + "| {col2: } |", + "+---------------+", + ]; + assert_batches_eq!(expected, &[batch]); + } + #[test] fn test_avro_iterator() { let reader = build_reader("alltypes_plain.avro", 5); diff --git a/datafusion/core/src/datasource/avro_to_arrow/schema.rs b/datafusion/core/src/datasource/avro_to_arrow/schema.rs index f15e378cc699..761e6b62680f 100644 --- a/datafusion/core/src/datasource/avro_to_arrow/schema.rs +++ b/datafusion/core/src/datasource/avro_to_arrow/schema.rs @@ -35,7 +35,7 @@ pub fn to_arrow_schema(avro_schema: &apache_avro::Schema) -> Result { schema_fields.push(schema_to_field_with_props( &field.schema, Some(&field.name), - false, + field.is_nullable(), Some(external_props(&field.schema)), )?) } @@ -73,7 +73,7 @@ fn schema_to_field_with_props( AvroSchema::Bytes => DataType::Binary, AvroSchema::String => DataType::Utf8, AvroSchema::Array(item_schema) => DataType::List(Arc::new( - schema_to_field_with_props(item_schema, None, false, None)?, + schema_to_field_with_props(item_schema, Some("element"), false, None)?, )), AvroSchema::Map(value_schema) => { let value_field = @@ -116,7 +116,7 @@ fn schema_to_field_with_props( DataType::Union(UnionFields::new(type_ids, fields), UnionMode::Dense) } } - AvroSchema::Record(RecordSchema { name, fields, .. }) => { + AvroSchema::Record(RecordSchema { fields, .. }) => { let fields: Result<_> = fields .iter() .map(|field| { @@ -129,7 +129,7 @@ fn schema_to_field_with_props( }*/ schema_to_field_with_props( &field.schema, - Some(&format!("{}.{}", name.fullname(None), field.name)), + Some(&field.name), false, Some(props), ) @@ -442,6 +442,58 @@ mod test { assert_eq!(arrow_schema.unwrap(), expected); } + #[test] + fn test_nested_schema() { + let avro_schema = apache_avro::Schema::parse_str( + r#" + { + "type": "record", + "name": "r1", + "fields": [ + { + "name": "col1", + "type": [ + "null", + { + "type": "record", + "name": "r2", + "fields": [ + { + "name": "col2", + "type": "string" + }, + { + "name": "col3", + "type": ["null", "string"], + "default": null + } + ] + } + ], + "default": null + } + ] + }"#, + ) + .unwrap(); + // should not use Avro Record names. + let expected_arrow_schema = Schema::new(vec![Field::new( + "col1", + arrow::datatypes::DataType::Struct( + vec![ + Field::new("col2", Utf8, false), + Field::new("col3", Utf8, true), + ] + .into(), + ), + true, + )]); + assert_eq!( + to_arrow_schema(&avro_schema).unwrap(), + expected_arrow_schema + ); + } + #[test] fn test_non_record_schema() { let arrow_schema = to_arrow_schema(&AvroSchema::String); diff --git a/datafusion/sqllogictest/test_files/avro.slt b/datafusion/sqllogictest/test_files/avro.slt index 3309cd1cf642..5cd268e8ef7f 100644 --- a/datafusion/sqllogictest/test_files/avro.slt +++ b/datafusion/sqllogictest/test_files/avro.slt @@ -225,11 +225,11 @@ SELECT id, CAST(string_col AS varchar) FROM alltypes_plain_multi_files 1 1 # test avro nested records -query ?? -SELECT f1, f2 FROM nested_records +query ???? +SELECT f1, f2, f3, f4 FROM nested_records ---- -{ns2.record2.f1_1: aaa, ns2.record2.f1_2: 10, ns2.record2.f1_3: {ns3.record3.f1_3_1: 3.14}} [{ns4.record4.f2_1: true, ns4.record4.f2_2: 1.2}, {ns4.record4.f2_1: true, ns4.record4.f2_2: 2.2}] -{ns2.record2.f1_1: bbb, ns2.record2.f1_2: 20, ns2.record2.f1_3: {ns3.record3.f1_3_1: 3.14}} [{ns4.record4.f2_1: false, ns4.record4.f2_2: 10.2}] +{f1_1: aaa, f1_2: 10, f1_3: {f1_3_1: 3.14}} [{f2_1: true, f2_2: 1.2}, {f2_1: true, f2_2: 2.2}] {f3_1: xyz} [{f4_1: 200}, ] +{f1_1: bbb, f1_2: 20, f1_3: {f1_3_1: 3.14}} [{f2_1: false, f2_2: 10.2}] NULL [, {f4_1: 300}] # test avro enum query TTT From f16bc8bd93b9ea29c8f84f6e0ddf2f3500503c19 Mon Sep 17 00:00:00 2001 From: Andrew Lamb Date: Wed, 4 Oct 2023 14:32:40 -0400 Subject: [PATCH 027/572] Rename `SessionContext::with_config_rt` to `SessionContext::new_with_config_from_rt`, etc (#7631) * Rename `SessionContext::with_config_rt` to `SessionContext::new_with_config_from_rt`, etc * update other references * Apply suggestions from code review Co-authored-by: Liang-Chi Hsieh --------- Co-authored-by: Liang-Chi Hsieh --- benchmarks/src/bin/h2o.rs | 2 +- benchmarks/src/clickbench.rs | 2 +- benchmarks/src/parquet_filter.rs | 2 +- benchmarks/src/sort.rs | 2 +- benchmarks/src/tpch/convert.rs | 2 +- benchmarks/src/tpch/run.rs | 2 +- datafusion-cli/src/main.rs | 2 +- .../examples/flight_sql_server.rs | 2 +- datafusion/core/benches/parquet_query_sql.rs | 2 +- .../core/benches/sort_limit_query_sql.rs | 5 +- datafusion/core/benches/sql_query_with_io.rs | 2 +- datafusion/core/benches/topk_aggregate.rs | 2 +- datafusion/core/src/dataframe.rs | 4 +- .../core/src/datasource/file_format/avro.rs | 2 +- .../core/src/datasource/file_format/csv.rs | 18 ++-- .../core/src/datasource/file_format/json.rs | 2 +- .../src/datasource/file_format/parquet.rs | 4 +- .../core/src/datasource/listing/table.rs | 6 +- .../core/src/datasource/physical_plan/csv.rs | 5 +- .../core/src/datasource/physical_plan/json.rs | 5 +- .../src/datasource/physical_plan/parquet.rs | 5 +- datafusion/core/src/datasource/view.rs | 18 ++-- datafusion/core/src/execution/context.rs | 100 ++++++++++++------ .../src/physical_optimizer/enforce_sorting.rs | 2 +- datafusion/core/src/physical_planner.rs | 2 +- datafusion/core/tests/dataframe/mod.rs | 2 +- datafusion/core/tests/fifo.rs | 6 +- .../core/tests/fuzz_cases/aggregate_fuzz.rs | 2 +- datafusion/core/tests/fuzz_cases/join_fuzz.rs | 2 +- .../core/tests/fuzz_cases/merge_fuzz.rs | 2 +- datafusion/core/tests/fuzz_cases/sort_fuzz.rs | 4 +- .../sort_preserving_repartition_fuzz.rs | 2 +- .../core/tests/fuzz_cases/window_fuzz.rs | 2 +- datafusion/core/tests/memory_limit.rs | 4 +- .../core/tests/parquet/file_statistics.rs | 2 +- .../core/tests/parquet/filter_pushdown.rs | 2 +- datafusion/core/tests/parquet/mod.rs | 2 +- datafusion/core/tests/sql/aggregates.rs | 2 +- datafusion/core/tests/sql/create_drop.rs | 8 +- datafusion/core/tests/sql/displayable.rs | 2 +- datafusion/core/tests/sql/explain_analyze.rs | 12 +-- datafusion/core/tests/sql/group_by.rs | 2 +- datafusion/core/tests/sql/joins.rs | 6 +- datafusion/core/tests/sql/mod.rs | 7 +- datafusion/core/tests/sql/order.rs | 2 +- datafusion/core/tests/sql/partitioned_csv.rs | 3 +- datafusion/core/tests/sql/repartition.rs | 2 +- datafusion/core/tests/sql/select.rs | 5 +- datafusion/core/tests/tpcds_planning.rs | 2 +- .../tests/user_defined/user_defined_plan.rs | 4 +- .../tests/cases/roundtrip_logical_plan.rs | 4 +- datafusion/sqllogictest/src/test_context.rs | 2 +- .../tests/cases/roundtrip_logical_plan.rs | 4 +- 53 files changed, 169 insertions(+), 130 deletions(-) diff --git a/benchmarks/src/bin/h2o.rs b/benchmarks/src/bin/h2o.rs index d75f9a30b4e9..1bb8cb9d43e4 100644 --- a/benchmarks/src/bin/h2o.rs +++ b/benchmarks/src/bin/h2o.rs @@ -72,7 +72,7 @@ async fn group_by(opt: &GroupBy) -> Result<()> { let mut config = ConfigOptions::from_env()?; config.execution.batch_size = 65535; - let ctx = SessionContext::with_config(config.into()); + let ctx = SessionContext::new_with_config(config.into()); let schema = Schema::new(vec![ Field::new("id1", DataType::Utf8, false), diff --git a/benchmarks/src/clickbench.rs b/benchmarks/src/clickbench.rs index 98ef6dd805b0..a6d32eb39f31 100644 --- a/benchmarks/src/clickbench.rs +++ b/benchmarks/src/clickbench.rs @@ -81,7 +81,7 @@ impl RunOpt { }; let config = self.common.config(); - let ctx = SessionContext::with_config(config); + let ctx = SessionContext::new_with_config(config); self.register_hits(&ctx).await?; let iterations = self.common.iterations; diff --git a/benchmarks/src/parquet_filter.rs b/benchmarks/src/parquet_filter.rs index ceea12de9238..e19596b80f54 100644 --- a/benchmarks/src/parquet_filter.rs +++ b/benchmarks/src/parquet_filter.rs @@ -144,7 +144,7 @@ impl RunOpt { )); for i in 0..self.common.iterations { let config = self.common.update_config(scan_options.config()); - let ctx = SessionContext::with_config(config); + let ctx = SessionContext::new_with_config(config); let (rows, elapsed) = exec_scan( &ctx, diff --git a/benchmarks/src/sort.rs b/benchmarks/src/sort.rs index d1baae868780..5643c8561944 100644 --- a/benchmarks/src/sort.rs +++ b/benchmarks/src/sort.rs @@ -150,7 +150,7 @@ impl RunOpt { for i in 0..self.common.iterations { let config = SessionConfig::new().with_target_partitions(self.common.partitions); - let ctx = SessionContext::with_config(config); + let ctx = SessionContext::new_with_config(config); let (rows, elapsed) = exec_sort(&ctx, &expr, &test_file, self.common.debug).await?; let ms = elapsed.as_secs_f64() * 1000.0; diff --git a/benchmarks/src/tpch/convert.rs b/benchmarks/src/tpch/convert.rs index f1ed081c43f5..2fc74ce38888 100644 --- a/benchmarks/src/tpch/convert.rs +++ b/benchmarks/src/tpch/convert.rs @@ -78,7 +78,7 @@ impl ConvertOpt { .file_extension(".tbl"); let config = SessionConfig::new().with_batch_size(self.batch_size); - let ctx = SessionContext::with_config(config); + let ctx = SessionContext::new_with_config(config); // build plan to read the TBL file let mut csv = ctx.read_csv(&input_path, options).await?; diff --git a/benchmarks/src/tpch/run.rs b/benchmarks/src/tpch/run.rs index cf5c7b9f67e3..171b074d2a1b 100644 --- a/benchmarks/src/tpch/run.rs +++ b/benchmarks/src/tpch/run.rs @@ -110,7 +110,7 @@ impl RunOpt { .common .config() .with_collect_statistics(!self.disable_statistics); - let ctx = SessionContext::with_config(config); + let ctx = SessionContext::new_with_config(config); // register tables self.register_tables(&ctx).await?; diff --git a/datafusion-cli/src/main.rs b/datafusion-cli/src/main.rs index 33a1caeb1b5b..c069f458f196 100644 --- a/datafusion-cli/src/main.rs +++ b/datafusion-cli/src/main.rs @@ -178,7 +178,7 @@ pub async fn main() -> Result<()> { let runtime_env = create_runtime_env(rn_config.clone())?; let mut ctx = - SessionContext::with_config_rt(session_config.clone(), Arc::new(runtime_env)); + SessionContext::new_with_config_rt(session_config.clone(), Arc::new(runtime_env)); ctx.refresh_catalogs().await?; // install dynamic catalog provider that knows how to open files ctx.register_catalog_list(Arc::new(DynamicFileCatalog::new( diff --git a/datafusion-examples/examples/flight_sql_server.rs b/datafusion-examples/examples/flight_sql_server.rs index aad63fc2bca8..ed5b86d0b66c 100644 --- a/datafusion-examples/examples/flight_sql_server.rs +++ b/datafusion-examples/examples/flight_sql_server.rs @@ -105,7 +105,7 @@ impl FlightSqlServiceImpl { let session_config = SessionConfig::from_env() .map_err(|e| Status::internal(format!("Error building plan: {e}")))? .with_information_schema(true); - let ctx = Arc::new(SessionContext::with_config(session_config)); + let ctx = Arc::new(SessionContext::new_with_config(session_config)); let testdata = datafusion::test_util::parquet_test_data(); diff --git a/datafusion/core/benches/parquet_query_sql.rs b/datafusion/core/benches/parquet_query_sql.rs index 876b1fe7e198..6c9ab315761e 100644 --- a/datafusion/core/benches/parquet_query_sql.rs +++ b/datafusion/core/benches/parquet_query_sql.rs @@ -193,7 +193,7 @@ fn criterion_benchmark(c: &mut Criterion) { let partitions = 4; let config = SessionConfig::new().with_target_partitions(partitions); - let context = SessionContext::with_config(config); + let context = SessionContext::new_with_config(config); let local_rt = tokio::runtime::Builder::new_current_thread() .build() diff --git a/datafusion/core/benches/sort_limit_query_sql.rs b/datafusion/core/benches/sort_limit_query_sql.rs index 62160067143e..efed5a04e7a5 100644 --- a/datafusion/core/benches/sort_limit_query_sql.rs +++ b/datafusion/core/benches/sort_limit_query_sql.rs @@ -86,8 +86,9 @@ fn create_context() -> Arc> { rt.block_on(async { // create local session context - let ctx = - SessionContext::with_config(SessionConfig::new().with_target_partitions(1)); + let ctx = SessionContext::new_with_config( + SessionConfig::new().with_target_partitions(1), + ); let table_provider = Arc::new(csv.await); let mem_table = MemTable::load(table_provider, Some(partitions), &ctx.state()) diff --git a/datafusion/core/benches/sql_query_with_io.rs b/datafusion/core/benches/sql_query_with_io.rs index 1d96df0cecaa..1f9b4dc6ccf7 100644 --- a/datafusion/core/benches/sql_query_with_io.rs +++ b/datafusion/core/benches/sql_query_with_io.rs @@ -120,7 +120,7 @@ async fn setup_context(object_store: Arc) -> SessionContext { let config = SessionConfig::new().with_target_partitions(THREADS); let rt = Arc::new(RuntimeEnv::default()); rt.register_object_store(&Url::parse("data://my_store").unwrap(), object_store); - let context = SessionContext::with_config_rt(config, rt); + let context = SessionContext::new_with_config_rt(config, rt); for table_id in 0..TABLES { let table_name = table_name(table_id); diff --git a/datafusion/core/benches/topk_aggregate.rs b/datafusion/core/benches/topk_aggregate.rs index f50a8ec047da..eaafea248894 100644 --- a/datafusion/core/benches/topk_aggregate.rs +++ b/datafusion/core/benches/topk_aggregate.rs @@ -45,7 +45,7 @@ async fn create_context( let mut cfg = SessionConfig::new(); let opts = cfg.options_mut(); opts.optimizer.enable_topk_aggregation = use_topk; - let ctx = SessionContext::with_config(cfg); + let ctx = SessionContext::new_with_config(cfg); let _ = ctx.register_table("traces", mem_table)?; let sql = format!("select trace_id, max(timestamp_ms) from traces group by trace_id order by max(timestamp_ms) desc limit {limit};"); let df = ctx.sql(sql.as_str()).await?; diff --git a/datafusion/core/src/dataframe.rs b/datafusion/core/src/dataframe.rs index 640f57f3d5fc..f6490801126f 100644 --- a/datafusion/core/src/dataframe.rs +++ b/datafusion/core/src/dataframe.rs @@ -1238,7 +1238,7 @@ impl DataFrame { /// # } /// ``` pub async fn cache(self) -> Result { - let context = SessionContext::with_state(self.session_state.clone()); + let context = SessionContext::new_with_state(self.session_state.clone()); let mem_table = MemTable::try_new( SchemaRef::from(self.schema().clone()), self.collect_partitioned().await?, @@ -2011,7 +2011,7 @@ mod tests { "datafusion.sql_parser.enable_ident_normalization".to_owned(), "false".to_owned(), )]))?; - let mut ctx = SessionContext::with_config(config); + let mut ctx = SessionContext::new_with_config(config); let name = "aggregate_test_100"; register_aggregate_csv(&mut ctx, name).await?; let df = ctx.table(name); diff --git a/datafusion/core/src/datasource/file_format/avro.rs b/datafusion/core/src/datasource/file_format/avro.rs index e68a4cad2207..cc6a03ba3f08 100644 --- a/datafusion/core/src/datasource/file_format/avro.rs +++ b/datafusion/core/src/datasource/file_format/avro.rs @@ -112,7 +112,7 @@ mod tests { #[tokio::test] async fn read_small_batches() -> Result<()> { let config = SessionConfig::new().with_batch_size(2); - let session_ctx = SessionContext::with_config(config); + let session_ctx = SessionContext::new_with_config(config); let state = session_ctx.state(); let task_ctx = state.task_ctx(); let projection = None; diff --git a/datafusion/core/src/datasource/file_format/csv.rs b/datafusion/core/src/datasource/file_format/csv.rs index 897174659e13..c3295042b5e3 100644 --- a/datafusion/core/src/datasource/file_format/csv.rs +++ b/datafusion/core/src/datasource/file_format/csv.rs @@ -622,7 +622,7 @@ mod tests { #[tokio::test] async fn read_small_batches() -> Result<()> { let config = SessionConfig::new().with_batch_size(2); - let session_ctx = SessionContext::with_config(config); + let session_ctx = SessionContext::new_with_config(config); let state = session_ctx.state(); let task_ctx = state.task_ctx(); // skip column 9 that overflows the automaticly discovered column type of i64 (u64 would work) @@ -960,7 +960,7 @@ mod tests { .with_repartition_file_scans(true) .with_repartition_file_min_size(0) .with_target_partitions(n_partitions); - let ctx = SessionContext::with_config(config); + let ctx = SessionContext::new_with_config(config); let testdata = arrow_test_data(); ctx.register_csv( "aggr", @@ -997,7 +997,7 @@ mod tests { .has_header(true) .file_compression_type(FileCompressionType::GZIP) .file_extension("csv.gz"); - let ctx = SessionContext::with_config(config); + let ctx = SessionContext::new_with_config(config); let testdata = arrow_test_data(); ctx.register_csv( "aggr", @@ -1033,7 +1033,7 @@ mod tests { .with_repartition_file_scans(true) .with_repartition_file_min_size(0) .with_target_partitions(n_partitions); - let ctx = SessionContext::with_config(config); + let ctx = SessionContext::new_with_config(config); ctx.register_csv( "empty", "tests/data/empty_0_byte.csv", @@ -1066,7 +1066,7 @@ mod tests { .with_repartition_file_scans(true) .with_repartition_file_min_size(0) .with_target_partitions(n_partitions); - let ctx = SessionContext::with_config(config); + let ctx = SessionContext::new_with_config(config); ctx.register_csv( "empty", "tests/data/empty.csv", @@ -1104,7 +1104,7 @@ mod tests { .with_repartition_file_scans(true) .with_repartition_file_min_size(0) .with_target_partitions(n_partitions); - let ctx = SessionContext::with_config(config); + let ctx = SessionContext::new_with_config(config); let file_format = CsvFormat::default().with_has_header(false); let listing_options = ListingOptions::new(Arc::new(file_format)) .with_file_extension(FileType::CSV.get_ext()); @@ -1157,7 +1157,7 @@ mod tests { .with_repartition_file_scans(true) .with_repartition_file_min_size(0) .with_target_partitions(n_partitions); - let ctx = SessionContext::with_config(config); + let ctx = SessionContext::new_with_config(config); let file_format = CsvFormat::default().with_has_header(false); let listing_options = ListingOptions::new(Arc::new(file_format)) .with_file_extension(FileType::CSV.get_ext()); @@ -1202,7 +1202,7 @@ mod tests { .with_repartition_file_scans(true) .with_repartition_file_min_size(0) .with_target_partitions(n_partitions); - let ctx = SessionContext::with_config(config); + let ctx = SessionContext::new_with_config(config); ctx.register_csv( "one_col", @@ -1251,7 +1251,7 @@ mod tests { .with_repartition_file_scans(true) .with_repartition_file_min_size(0) .with_target_partitions(n_partitions); - let ctx = SessionContext::with_config(config); + let ctx = SessionContext::new_with_config(config); ctx.register_csv( "wide_rows", "tests/data/wide_rows.csv", diff --git a/datafusion/core/src/datasource/file_format/json.rs b/datafusion/core/src/datasource/file_format/json.rs index c715317a9527..96fd4daa2da6 100644 --- a/datafusion/core/src/datasource/file_format/json.rs +++ b/datafusion/core/src/datasource/file_format/json.rs @@ -389,7 +389,7 @@ mod tests { #[tokio::test] async fn read_small_batches() -> Result<()> { let config = SessionConfig::new().with_batch_size(2); - let session_ctx = SessionContext::with_config(config); + let session_ctx = SessionContext::new_with_config(config); let state = session_ctx.state(); let task_ctx = state.task_ctx(); let projection = None; diff --git a/datafusion/core/src/datasource/file_format/parquet.rs b/datafusion/core/src/datasource/file_format/parquet.rs index ebdf3ea444b1..16050d66db5d 100644 --- a/datafusion/core/src/datasource/file_format/parquet.rs +++ b/datafusion/core/src/datasource/file_format/parquet.rs @@ -1378,7 +1378,7 @@ mod tests { #[tokio::test] async fn read_small_batches() -> Result<()> { let config = SessionConfig::new().with_batch_size(2); - let session_ctx = SessionContext::with_config(config); + let session_ctx = SessionContext::new_with_config(config); let state = session_ctx.state(); let task_ctx = state.task_ctx(); let projection = None; @@ -1406,7 +1406,7 @@ mod tests { #[tokio::test] async fn capture_bytes_scanned_metric() -> Result<()> { let config = SessionConfig::new().with_batch_size(2); - let session = SessionContext::with_config(config); + let session = SessionContext::new_with_config(config); let ctx = session.state(); // Read the full file diff --git a/datafusion/core/src/datasource/listing/table.rs b/datafusion/core/src/datasource/listing/table.rs index 797562e92d2e..5b1710d344ee 100644 --- a/datafusion/core/src/datasource/listing/table.rs +++ b/datafusion/core/src/datasource/listing/table.rs @@ -1877,7 +1877,7 @@ mod tests { let session_ctx = match session_config_map { Some(cfg) => { let config = SessionConfig::from_string_hash_map(cfg)?; - SessionContext::with_config(config) + SessionContext::new_with_config(config) } None => SessionContext::new(), }; @@ -2046,7 +2046,7 @@ mod tests { let session_ctx = match session_config_map { Some(cfg) => { let config = SessionConfig::from_string_hash_map(cfg)?; - SessionContext::with_config(config) + SessionContext::new_with_config(config) } None => SessionContext::new(), }; @@ -2252,7 +2252,7 @@ mod tests { let session_ctx = match session_config_map { Some(cfg) => { let config = SessionConfig::from_string_hash_map(cfg)?; - SessionContext::with_config(config) + SessionContext::new_with_config(config) } None => SessionContext::new(), }; diff --git a/datafusion/core/src/datasource/physical_plan/csv.rs b/datafusion/core/src/datasource/physical_plan/csv.rs index dfc6acdde073..8277836a24ff 100644 --- a/datafusion/core/src/datasource/physical_plan/csv.rs +++ b/datafusion/core/src/datasource/physical_plan/csv.rs @@ -1079,8 +1079,9 @@ mod tests { async fn write_csv_results() -> Result<()> { // create partitioned input file and context let tmp_dir = TempDir::new()?; - let ctx = - SessionContext::with_config(SessionConfig::new().with_target_partitions(8)); + let ctx = SessionContext::new_with_config( + SessionConfig::new().with_target_partitions(8), + ); let schema = populate_csv_partitions(&tmp_dir, 8, ".csv")?; diff --git a/datafusion/core/src/datasource/physical_plan/json.rs b/datafusion/core/src/datasource/physical_plan/json.rs index 537855704a76..493110404119 100644 --- a/datafusion/core/src/datasource/physical_plan/json.rs +++ b/datafusion/core/src/datasource/physical_plan/json.rs @@ -675,8 +675,9 @@ mod tests { #[tokio::test] async fn write_json_results() -> Result<()> { // create partitioned input file and context - let ctx = - SessionContext::with_config(SessionConfig::new().with_target_partitions(8)); + let ctx = SessionContext::new_with_config( + SessionConfig::new().with_target_partitions(8), + ); let path = format!("{TEST_DATA_BASE}/1.json"); diff --git a/datafusion/core/src/datasource/physical_plan/parquet.rs b/datafusion/core/src/datasource/physical_plan/parquet.rs index d16c79a9692c..6f27acfe7f6c 100644 --- a/datafusion/core/src/datasource/physical_plan/parquet.rs +++ b/datafusion/core/src/datasource/physical_plan/parquet.rs @@ -1928,8 +1928,9 @@ mod tests { // create partitioned input file and context let tmp_dir = TempDir::new()?; // let mut ctx = create_ctx(&tmp_dir, 4).await?; - let ctx = - SessionContext::with_config(SessionConfig::new().with_target_partitions(8)); + let ctx = SessionContext::new_with_config( + SessionConfig::new().with_target_partitions(8), + ); let schema = populate_csv_partitions(&tmp_dir, 4, ".csv")?; // register csv file with the execution context ctx.register_csv( diff --git a/datafusion/core/src/datasource/view.rs b/datafusion/core/src/datasource/view.rs index d58284d1bac5..85fb8939886c 100644 --- a/datafusion/core/src/datasource/view.rs +++ b/datafusion/core/src/datasource/view.rs @@ -159,7 +159,7 @@ mod tests { #[tokio::test] async fn issue_3242() -> Result<()> { // regression test for https://github.com/apache/arrow-datafusion/pull/3242 - let session_ctx = SessionContext::with_config( + let session_ctx = SessionContext::new_with_config( SessionConfig::new().with_information_schema(true), ); @@ -199,7 +199,7 @@ mod tests { #[tokio::test] async fn query_view() -> Result<()> { - let session_ctx = SessionContext::with_config( + let session_ctx = SessionContext::new_with_config( SessionConfig::new().with_information_schema(true), ); @@ -237,7 +237,7 @@ mod tests { #[tokio::test] async fn query_view_with_alias() -> Result<()> { - let session_ctx = SessionContext::with_config(SessionConfig::new()); + let session_ctx = SessionContext::new_with_config(SessionConfig::new()); session_ctx .sql("CREATE TABLE abc AS VALUES (1,2,3), (4,5,6)") @@ -270,7 +270,7 @@ mod tests { #[tokio::test] async fn query_view_with_inline_alias() -> Result<()> { - let session_ctx = SessionContext::with_config(SessionConfig::new()); + let session_ctx = SessionContext::new_with_config(SessionConfig::new()); session_ctx .sql("CREATE TABLE abc AS VALUES (1,2,3), (4,5,6)") @@ -303,7 +303,7 @@ mod tests { #[tokio::test] async fn query_view_with_projection() -> Result<()> { - let session_ctx = SessionContext::with_config( + let session_ctx = SessionContext::new_with_config( SessionConfig::new().with_information_schema(true), ); @@ -341,7 +341,7 @@ mod tests { #[tokio::test] async fn query_view_with_filter() -> Result<()> { - let session_ctx = SessionContext::with_config( + let session_ctx = SessionContext::new_with_config( SessionConfig::new().with_information_schema(true), ); @@ -378,7 +378,7 @@ mod tests { #[tokio::test] async fn query_join_views() -> Result<()> { - let session_ctx = SessionContext::with_config( + let session_ctx = SessionContext::new_with_config( SessionConfig::new().with_information_schema(true), ); @@ -481,7 +481,7 @@ mod tests { #[tokio::test] async fn create_view_plan() -> Result<()> { - let session_ctx = SessionContext::with_config( + let session_ctx = SessionContext::new_with_config( SessionConfig::new().with_information_schema(true), ); @@ -534,7 +534,7 @@ mod tests { #[tokio::test] async fn create_or_replace_view() -> Result<()> { - let session_ctx = SessionContext::with_config( + let session_ctx = SessionContext::new_with_config( SessionConfig::new().with_information_schema(true), ); diff --git a/datafusion/core/src/execution/context.rs b/datafusion/core/src/execution/context.rs index 4bdd40a91403..ca6da6cfa047 100644 --- a/datafusion/core/src/execution/context.rs +++ b/datafusion/core/src/execution/context.rs @@ -258,7 +258,7 @@ impl Default for SessionContext { impl SessionContext { /// Creates a new `SessionContext` using the default [`SessionConfig`]. pub fn new() -> Self { - Self::with_config(SessionConfig::new()) + Self::new_with_config(SessionConfig::new()) } /// Finds any [`ListingSchemaProvider`]s and instructs them to reload tables from "disk" @@ -284,11 +284,18 @@ impl SessionContext { /// Creates a new `SessionContext` using the provided /// [`SessionConfig`] and a new [`RuntimeEnv`]. /// - /// See [`Self::with_config_rt`] for more details on resource + /// See [`Self::new_with_config_rt`] for more details on resource /// limits. - pub fn with_config(config: SessionConfig) -> Self { + pub fn new_with_config(config: SessionConfig) -> Self { let runtime = Arc::new(RuntimeEnv::default()); - Self::with_config_rt(config, runtime) + Self::new_with_config_rt(config, runtime) + } + + /// Creates a new `SessionContext` using the provided + /// [`SessionConfig`] and a new [`RuntimeEnv`]. + #[deprecated(since = "32.0.0", note = "Use SessionContext::new_with_config")] + pub fn with_config(config: SessionConfig) -> Self { + Self::new_with_config(config) } /// Creates a new `SessionContext` using the provided @@ -304,13 +311,20 @@ impl SessionContext { /// memory used) across all DataFusion queries in a process, /// all `SessionContext`'s should be configured with the /// same `RuntimeEnv`. + pub fn new_with_config_rt(config: SessionConfig, runtime: Arc) -> Self { + let state = SessionState::new_with_config_rt(config, runtime); + Self::new_with_state(state) + } + + /// Creates a new `SessionContext` using the provided + /// [`SessionConfig`] and a [`RuntimeEnv`]. + #[deprecated(since = "32.0.0", note = "Use SessionState::new_with_config_rt")] pub fn with_config_rt(config: SessionConfig, runtime: Arc) -> Self { - let state = SessionState::with_config_rt(config, runtime); - Self::with_state(state) + Self::new_with_config_rt(config, runtime) } /// Creates a new `SessionContext` using the provided [`SessionState`] - pub fn with_state(state: SessionState) -> Self { + pub fn new_with_state(state: SessionState) -> Self { Self { session_id: state.session_id.clone(), session_start_time: Utc::now(), @@ -318,6 +332,11 @@ impl SessionContext { } } + /// Creates a new `SessionContext` using the provided [`SessionState`] + #[deprecated(since = "32.0.0", note = "Use SessionState::new_with_state")] + pub fn with_state(state: SessionState) -> Self { + Self::new_with_state(state) + } /// Returns the time this `SessionContext` was created pub fn session_start_time(&self) -> DateTime { self.session_start_time @@ -1401,25 +1420,24 @@ impl Debug for SessionState { } } -/// Default session builder using the provided configuration -#[deprecated( - since = "23.0.0", - note = "See SessionContext::with_config() or SessionState::with_config_rt" -)] -pub fn default_session_builder(config: SessionConfig) -> SessionState { - SessionState::with_config_rt(config, Arc::new(RuntimeEnv::default())) -} - impl SessionState { /// Returns new [`SessionState`] using the provided /// [`SessionConfig`] and [`RuntimeEnv`]. - pub fn with_config_rt(config: SessionConfig, runtime: Arc) -> Self { + pub fn new_with_config_rt(config: SessionConfig, runtime: Arc) -> Self { let catalog_list = Arc::new(MemoryCatalogList::new()) as Arc; - Self::with_config_rt_and_catalog_list(config, runtime, catalog_list) + Self::new_with_config_rt_and_catalog_list(config, runtime, catalog_list) } - /// Returns new SessionState using the provided configuration, runtime and catalog list. - pub fn with_config_rt_and_catalog_list( + /// Returns new [`SessionState`] using the provided + /// [`SessionConfig`] and [`RuntimeEnv`]. + #[deprecated(since = "32.0.0", note = "Use SessionState::new_with_config_rt")] + pub fn with_config_rt(config: SessionConfig, runtime: Arc) -> Self { + Self::new_with_config_rt(config, runtime) + } + + /// Returns new [`SessionState`] using the provided + /// [`SessionConfig`], [`RuntimeEnv`], and [`CatalogList`] + pub fn new_with_config_rt_and_catalog_list( config: SessionConfig, runtime: Arc, catalog_list: Arc, @@ -1476,7 +1494,19 @@ impl SessionState { table_factories, } } - + /// Returns new [`SessionState`] using the provided + /// [`SessionConfig`] and [`RuntimeEnv`]. + #[deprecated( + since = "32.0.0", + note = "Use SessionState::new_with_config_rt_and_catalog_list" + )] + pub fn with_config_rt_and_catalog_list( + config: SessionConfig, + runtime: Arc, + catalog_list: Arc, + ) -> Self { + Self::new_with_config_rt_and_catalog_list(config, runtime, catalog_list) + } fn register_default_schema( config: &SessionConfig, table_factories: &HashMap>, @@ -2250,7 +2280,7 @@ mod tests { let disk_manager = ctx1.runtime_env().disk_manager.clone(); let ctx2 = - SessionContext::with_config_rt(SessionConfig::new(), ctx1.runtime_env()); + SessionContext::new_with_config_rt(SessionConfig::new(), ctx1.runtime_env()); assert_eq!(ctx1.runtime_env().memory_pool.reserved(), 100); assert_eq!(ctx2.runtime_env().memory_pool.reserved(), 100); @@ -2397,8 +2427,8 @@ mod tests { .set_str("datafusion.catalog.location", url.as_str()) .set_str("datafusion.catalog.format", "CSV") .set_str("datafusion.catalog.has_header", "true"); - let session_state = SessionState::with_config_rt(cfg, runtime); - let ctx = SessionContext::with_state(session_state); + let session_state = SessionState::new_with_config_rt(cfg, runtime); + let ctx = SessionContext::new_with_state(session_state); ctx.refresh_catalogs().await?; let result = @@ -2423,9 +2453,10 @@ mod tests { #[tokio::test] async fn custom_query_planner() -> Result<()> { let runtime = Arc::new(RuntimeEnv::default()); - let session_state = SessionState::with_config_rt(SessionConfig::new(), runtime) - .with_query_planner(Arc::new(MyQueryPlanner {})); - let ctx = SessionContext::with_state(session_state); + let session_state = + SessionState::new_with_config_rt(SessionConfig::new(), runtime) + .with_query_planner(Arc::new(MyQueryPlanner {})); + let ctx = SessionContext::new_with_state(session_state); let df = ctx.sql("SELECT 1").await?; df.collect().await.expect_err("query not supported"); @@ -2434,7 +2465,7 @@ mod tests { #[tokio::test] async fn disabled_default_catalog_and_schema() -> Result<()> { - let ctx = SessionContext::with_config( + let ctx = SessionContext::new_with_config( SessionConfig::new().with_create_default_catalog_and_schema(false), ); @@ -2477,7 +2508,7 @@ mod tests { } async fn catalog_and_schema_test(config: SessionConfig) { - let ctx = SessionContext::with_config(config); + let ctx = SessionContext::new_with_config(config); let catalog = MemoryCatalogProvider::new(); let schema = MemorySchemaProvider::new(); schema @@ -2554,7 +2585,7 @@ mod tests { #[tokio::test] async fn catalogs_not_leaked() { // the information schema used to introduce cyclic Arcs - let ctx = SessionContext::with_config( + let ctx = SessionContext::new_with_config( SessionConfig::new().with_information_schema(true), ); @@ -2577,7 +2608,7 @@ mod tests { #[tokio::test] async fn sql_create_schema() -> Result<()> { // the information schema used to introduce cyclic Arcs - let ctx = SessionContext::with_config( + let ctx = SessionContext::new_with_config( SessionConfig::new().with_information_schema(true), ); @@ -2600,7 +2631,7 @@ mod tests { #[tokio::test] async fn sql_create_catalog() -> Result<()> { // the information schema used to introduce cyclic Arcs - let ctx = SessionContext::with_config( + let ctx = SessionContext::new_with_config( SessionConfig::new().with_information_schema(true), ); @@ -2758,8 +2789,9 @@ mod tests { tmp_dir: &TempDir, partition_count: usize, ) -> Result { - let ctx = - SessionContext::with_config(SessionConfig::new().with_target_partitions(8)); + let ctx = SessionContext::new_with_config( + SessionConfig::new().with_target_partitions(8), + ); let schema = populate_csv_partitions(tmp_dir, partition_count, ".csv")?; diff --git a/datafusion/core/src/physical_optimizer/enforce_sorting.rs b/datafusion/core/src/physical_optimizer/enforce_sorting.rs index c4b72a7cb31e..6e3b160c8922 100644 --- a/datafusion/core/src/physical_optimizer/enforce_sorting.rs +++ b/datafusion/core/src/physical_optimizer/enforce_sorting.rs @@ -810,7 +810,7 @@ mod tests { macro_rules! assert_optimized { ($EXPECTED_PLAN_LINES: expr, $EXPECTED_OPTIMIZED_PLAN_LINES: expr, $PLAN: expr, $REPARTITION_SORTS: expr) => { let config = SessionConfig::new().with_repartition_sorts($REPARTITION_SORTS); - let session_ctx = SessionContext::with_config(config); + let session_ctx = SessionContext::new_with_config(config); let state = session_ctx.state(); let physical_plan = $PLAN; diff --git a/datafusion/core/src/physical_planner.rs b/datafusion/core/src/physical_planner.rs index 2328ffce235d..84b5b9afa7e7 100644 --- a/datafusion/core/src/physical_planner.rs +++ b/datafusion/core/src/physical_planner.rs @@ -2087,7 +2087,7 @@ mod tests { let runtime = Arc::new(RuntimeEnv::default()); let config = SessionConfig::new().with_target_partitions(4); let config = config.set_bool("datafusion.optimizer.skip_failed_rules", false); - SessionState::with_config_rt(config, runtime) + SessionState::new_with_config_rt(config, runtime) } async fn plan(logical_plan: &LogicalPlan) -> Result> { diff --git a/datafusion/core/tests/dataframe/mod.rs b/datafusion/core/tests/dataframe/mod.rs index e1982761f04c..845d77581b59 100644 --- a/datafusion/core/tests/dataframe/mod.rs +++ b/datafusion/core/tests/dataframe/mod.rs @@ -1567,7 +1567,7 @@ async fn use_var_provider() -> Result<()> { let config = SessionConfig::new() .with_target_partitions(4) .set_bool("datafusion.optimizer.skip_failed_rules", false); - let ctx = SessionContext::with_config(config); + let ctx = SessionContext::new_with_config(config); ctx.register_table("csv_table", mem_table)?; ctx.register_variable(VarType::UserDefined, Arc::new(HardcodedIntProvider {})); diff --git a/datafusion/core/tests/fifo.rs b/datafusion/core/tests/fifo.rs index 754389a61433..2c8b0b784f26 100644 --- a/datafusion/core/tests/fifo.rs +++ b/datafusion/core/tests/fifo.rs @@ -99,7 +99,7 @@ mod unix_test { .with_batch_size(TEST_BATCH_SIZE) .with_collect_statistics(false) .with_target_partitions(1); - let ctx = SessionContext::with_config(config); + let ctx = SessionContext::new_with_config(config); // To make unbounded deterministic let waiting = Arc::new(AtomicBool::new(unbounded_file)); // Create a new temporary FIFO file @@ -211,7 +211,7 @@ mod unix_test { .with_batch_size(TEST_BATCH_SIZE) .set_bool("datafusion.execution.coalesce_batches", false) .with_target_partitions(1); - let ctx = SessionContext::with_config(config); + let ctx = SessionContext::new_with_config(config); // Tasks let mut tasks: Vec> = vec![]; @@ -342,7 +342,7 @@ mod unix_test { let waiting_thread = waiting.clone(); // create local execution context let config = SessionConfig::new().with_batch_size(TEST_BATCH_SIZE); - let ctx = SessionContext::with_config(config); + let ctx = SessionContext::new_with_config(config); // Create a new temporary FIFO file let tmp_dir = TempDir::new()?; let source_fifo_path = create_fifo_file(&tmp_dir, "source.csv")?; diff --git a/datafusion/core/tests/fuzz_cases/aggregate_fuzz.rs b/datafusion/core/tests/fuzz_cases/aggregate_fuzz.rs index a0e9a50a22ae..50d3610deed3 100644 --- a/datafusion/core/tests/fuzz_cases/aggregate_fuzz.rs +++ b/datafusion/core/tests/fuzz_cases/aggregate_fuzz.rs @@ -77,7 +77,7 @@ mod tests { async fn run_aggregate_test(input1: Vec, group_by_columns: Vec<&str>) { let schema = input1[0].schema(); let session_config = SessionConfig::new().with_batch_size(50); - let ctx = SessionContext::with_config(session_config); + let ctx = SessionContext::new_with_config(session_config); let mut sort_keys = vec![]; for ordering_col in ["a", "b", "c"] { sort_keys.push(PhysicalSortExpr { diff --git a/datafusion/core/tests/fuzz_cases/join_fuzz.rs b/datafusion/core/tests/fuzz_cases/join_fuzz.rs index 9b741440ff13..ac86364f4255 100644 --- a/datafusion/core/tests/fuzz_cases/join_fuzz.rs +++ b/datafusion/core/tests/fuzz_cases/join_fuzz.rs @@ -102,7 +102,7 @@ async fn run_join_test( let batch_sizes = [1, 2, 7, 49, 50, 51, 100]; for batch_size in batch_sizes { let session_config = SessionConfig::new().with_batch_size(batch_size); - let ctx = SessionContext::with_config(session_config); + let ctx = SessionContext::new_with_config(session_config); let task_ctx = ctx.task_ctx(); let schema1 = input1[0].schema(); diff --git a/datafusion/core/tests/fuzz_cases/merge_fuzz.rs b/datafusion/core/tests/fuzz_cases/merge_fuzz.rs index 6411f31be0ce..c38ff41f5783 100644 --- a/datafusion/core/tests/fuzz_cases/merge_fuzz.rs +++ b/datafusion/core/tests/fuzz_cases/merge_fuzz.rs @@ -118,7 +118,7 @@ async fn run_merge_test(input: Vec>) { let merge = Arc::new(SortPreservingMergeExec::new(sort, Arc::new(exec))); let session_config = SessionConfig::new().with_batch_size(batch_size); - let ctx = SessionContext::with_config(session_config); + let ctx = SessionContext::new_with_config(session_config); let task_ctx = ctx.task_ctx(); let collected = collect(merge, task_ctx).await.unwrap(); diff --git a/datafusion/core/tests/fuzz_cases/sort_fuzz.rs b/datafusion/core/tests/fuzz_cases/sort_fuzz.rs index 6c427c7fb7b3..d74144b0abce 100644 --- a/datafusion/core/tests/fuzz_cases/sort_fuzz.rs +++ b/datafusion/core/tests/fuzz_cases/sort_fuzz.rs @@ -174,9 +174,9 @@ impl SortTest { let runtime_config = RuntimeConfig::new() .with_memory_pool(Arc::new(GreedyMemoryPool::new(pool_size))); let runtime = Arc::new(RuntimeEnv::new(runtime_config).unwrap()); - SessionContext::with_config_rt(session_config, runtime) + SessionContext::new_with_config_rt(session_config, runtime) } else { - SessionContext::with_config(session_config) + SessionContext::new_with_config(session_config) }; let task_ctx = session_ctx.task_ctx(); diff --git a/datafusion/core/tests/fuzz_cases/sort_preserving_repartition_fuzz.rs b/datafusion/core/tests/fuzz_cases/sort_preserving_repartition_fuzz.rs index 6304e01c6389..4d3a2a15c5e9 100644 --- a/datafusion/core/tests/fuzz_cases/sort_preserving_repartition_fuzz.rs +++ b/datafusion/core/tests/fuzz_cases/sort_preserving_repartition_fuzz.rs @@ -93,7 +93,7 @@ mod sp_repartition_fuzz_tests { ) { let schema = input1[0].schema(); let session_config = SessionConfig::new().with_batch_size(50); - let ctx = SessionContext::with_config(session_config); + let ctx = SessionContext::new_with_config(session_config); let mut sort_keys = vec![]; for ordering_col in ["a", "b", "c"] { sort_keys.push(PhysicalSortExpr { diff --git a/datafusion/core/tests/fuzz_cases/window_fuzz.rs b/datafusion/core/tests/fuzz_cases/window_fuzz.rs index 3d103ee70ee8..1f0a4b09b15f 100644 --- a/datafusion/core/tests/fuzz_cases/window_fuzz.rs +++ b/datafusion/core/tests/fuzz_cases/window_fuzz.rs @@ -396,7 +396,7 @@ async fn run_window_test( let mut rng = StdRng::seed_from_u64(random_seed); let schema = input1[0].schema(); let session_config = SessionConfig::new().with_batch_size(50); - let ctx = SessionContext::with_config(session_config); + let ctx = SessionContext::new_with_config(session_config); let (window_fn, args, fn_name) = get_random_function(&schema, &mut rng, is_linear); let window_frame = get_random_window_frame(&mut rng, is_linear); diff --git a/datafusion/core/tests/memory_limit.rs b/datafusion/core/tests/memory_limit.rs index 1041888b95d9..a98d097856fb 100644 --- a/datafusion/core/tests/memory_limit.rs +++ b/datafusion/core/tests/memory_limit.rs @@ -412,13 +412,13 @@ impl TestCase { let runtime = RuntimeEnv::new(rt_config).unwrap(); // Configure execution - let state = SessionState::with_config_rt(config, Arc::new(runtime)); + let state = SessionState::new_with_config_rt(config, Arc::new(runtime)); let state = match scenario.rules() { Some(rules) => state.with_physical_optimizer_rules(rules), None => state, }; - let ctx = SessionContext::with_state(state); + let ctx = SessionContext::new_with_state(state); ctx.register_table("t", table).expect("registering table"); let query = query.expect("Test error: query not specified"); diff --git a/datafusion/core/tests/parquet/file_statistics.rs b/datafusion/core/tests/parquet/file_statistics.rs index 90abbe9e2128..58f81cc57190 100644 --- a/datafusion/core/tests/parquet/file_statistics.rs +++ b/datafusion/core/tests/parquet/file_statistics.rs @@ -101,7 +101,7 @@ fn get_cache_runtime_state() -> (Arc, SessionState) let rt = Arc::new( RuntimeEnv::new(RuntimeConfig::new().with_cache_manager(cache_config)).unwrap(), ); - let state = SessionContext::with_config_rt(SessionConfig::default(), rt).state(); + let state = SessionContext::new_with_config_rt(SessionConfig::default(), rt).state(); (cache1, state) } diff --git a/datafusion/core/tests/parquet/filter_pushdown.rs b/datafusion/core/tests/parquet/filter_pushdown.rs index 885834f93979..61a8f87b9ea5 100644 --- a/datafusion/core/tests/parquet/filter_pushdown.rs +++ b/datafusion/core/tests/parquet/filter_pushdown.rs @@ -507,7 +507,7 @@ impl<'a> TestCase<'a> { ) -> RecordBatch { println!(" scan options: {scan_options:?}"); println!(" reading with filter {filter:?}"); - let ctx = SessionContext::with_config(scan_options.config()); + let ctx = SessionContext::new_with_config(scan_options.config()); let exec = self .test_parquet_file .create_scan(Some(filter.clone())) diff --git a/datafusion/core/tests/parquet/mod.rs b/datafusion/core/tests/parquet/mod.rs index 33a78660ab9d..3f003c077d6a 100644 --- a/datafusion/core/tests/parquet/mod.rs +++ b/datafusion/core/tests/parquet/mod.rs @@ -154,7 +154,7 @@ impl ContextWithParquet { let parquet_path = file.path().to_string_lossy(); // now, setup a the file as a data source and run a query against it - let ctx = SessionContext::with_config(config); + let ctx = SessionContext::new_with_config(config); ctx.register_parquet("t", &parquet_path, ParquetReadOptions::default()) .await diff --git a/datafusion/core/tests/sql/aggregates.rs b/datafusion/core/tests/sql/aggregates.rs index 5d42936232b5..63d5e58090eb 100644 --- a/datafusion/core/tests/sql/aggregates.rs +++ b/datafusion/core/tests/sql/aggregates.rs @@ -801,7 +801,7 @@ async fn aggregate_with_alias() -> Result<()> { #[tokio::test] async fn test_accumulator_row_accumulator() -> Result<()> { let config = SessionConfig::new(); - let ctx = SessionContext::with_config(config); + let ctx = SessionContext::new_with_config(config); register_aggregate_csv(&ctx).await?; let sql = "SELECT c1, c2, MIN(c13) as min1, MIN(c9) as min2, MAX(c13) as max1, MAX(c9) as max2, AVG(c9) as avg1, MIN(c13) as min3, COUNT(C9) as cnt1, 0.5*SUM(c9-c8) as sum1 diff --git a/datafusion/core/tests/sql/create_drop.rs b/datafusion/core/tests/sql/create_drop.rs index aa34552044d4..b1434dddee50 100644 --- a/datafusion/core/tests/sql/create_drop.rs +++ b/datafusion/core/tests/sql/create_drop.rs @@ -26,11 +26,11 @@ async fn create_custom_table() -> Result<()> { let cfg = RuntimeConfig::new(); let env = RuntimeEnv::new(cfg).unwrap(); let ses = SessionConfig::new(); - let mut state = SessionState::with_config_rt(ses, Arc::new(env)); + let mut state = SessionState::new_with_config_rt(ses, Arc::new(env)); state .table_factories_mut() .insert("DELTATABLE".to_string(), Arc::new(TestTableFactory {})); - let ctx = SessionContext::with_state(state); + let ctx = SessionContext::new_with_state(state); let sql = "CREATE EXTERNAL TABLE dt STORED AS DELTATABLE LOCATION 's3://bucket/schema/table';"; ctx.sql(sql).await.unwrap(); @@ -48,11 +48,11 @@ async fn create_external_table_with_ddl() -> Result<()> { let cfg = RuntimeConfig::new(); let env = RuntimeEnv::new(cfg).unwrap(); let ses = SessionConfig::new(); - let mut state = SessionState::with_config_rt(ses, Arc::new(env)); + let mut state = SessionState::new_with_config_rt(ses, Arc::new(env)); state .table_factories_mut() .insert("MOCKTABLE".to_string(), Arc::new(TestTableFactory {})); - let ctx = SessionContext::with_state(state); + let ctx = SessionContext::new_with_state(state); let sql = "CREATE EXTERNAL TABLE dt (a_id integer, a_str string, a_bool boolean) STORED AS MOCKTABLE LOCATION 'mockprotocol://path/to/table';"; ctx.sql(sql).await.unwrap(); diff --git a/datafusion/core/tests/sql/displayable.rs b/datafusion/core/tests/sql/displayable.rs index b736820009cc..3255d514c5e4 100644 --- a/datafusion/core/tests/sql/displayable.rs +++ b/datafusion/core/tests/sql/displayable.rs @@ -24,7 +24,7 @@ use datafusion_physical_plan::displayable; async fn teset_displayable() { // Hard code target_partitions as it appears in the RepartitionExec output let config = SessionConfig::new().with_target_partitions(3); - let ctx = SessionContext::with_config(config); + let ctx = SessionContext::new_with_config(config); // register the a table ctx.register_csv("example", "tests/data/example.csv", CsvReadOptions::new()) diff --git a/datafusion/core/tests/sql/explain_analyze.rs b/datafusion/core/tests/sql/explain_analyze.rs index 06120c01ce86..2b708df4f348 100644 --- a/datafusion/core/tests/sql/explain_analyze.rs +++ b/datafusion/core/tests/sql/explain_analyze.rs @@ -27,7 +27,7 @@ async fn explain_analyze_baseline_metrics() { let config = SessionConfig::new() .with_target_partitions(3) .with_batch_size(4096); - let ctx = SessionContext::with_config(config); + let ctx = SessionContext::new_with_config(config); register_aggregate_csv_by_sql(&ctx).await; // a query with as many operators as we have metrics for let sql = "EXPLAIN ANALYZE \ @@ -598,7 +598,7 @@ async fn test_physical_plan_display_indent() { let config = SessionConfig::new() .with_target_partitions(9000) .with_batch_size(4096); - let ctx = SessionContext::with_config(config); + let ctx = SessionContext::new_with_config(config); register_aggregate_csv(&ctx).await.unwrap(); let sql = "SELECT c1, MAX(c12), MIN(c12) as the_min \ FROM aggregate_test_100 \ @@ -642,7 +642,7 @@ async fn test_physical_plan_display_indent_multi_children() { let config = SessionConfig::new() .with_target_partitions(9000) .with_batch_size(4096); - let ctx = SessionContext::with_config(config); + let ctx = SessionContext::new_with_config(config); // ensure indenting works for nodes with multiple children register_aggregate_csv(&ctx).await.unwrap(); let sql = "SELECT c1 \ @@ -777,7 +777,7 @@ async fn csv_explain_analyze_verbose() { async fn explain_logical_plan_only() { let mut config = ConfigOptions::new(); config.explain.logical_plan_only = true; - let ctx = SessionContext::with_config(config.into()); + let ctx = SessionContext::new_with_config(config.into()); let sql = "EXPLAIN select count(*) from (values ('a', 1, 100), ('a', 2, 150)) as t (c1,c2,c3)"; let actual = execute(&ctx, sql).await; let actual = normalize_vec_for_explain(actual); @@ -797,7 +797,7 @@ async fn explain_logical_plan_only() { async fn explain_physical_plan_only() { let mut config = ConfigOptions::new(); config.explain.physical_plan_only = true; - let ctx = SessionContext::with_config(config.into()); + let ctx = SessionContext::new_with_config(config.into()); let sql = "EXPLAIN select count(*) from (values ('a', 1, 100), ('a', 2, 150)) as t (c1,c2,c3)"; let actual = execute(&ctx, sql).await; let actual = normalize_vec_for_explain(actual); @@ -816,7 +816,7 @@ async fn csv_explain_analyze_with_statistics() { let mut config = ConfigOptions::new(); config.explain.physical_plan_only = true; config.explain.show_statistics = true; - let ctx = SessionContext::with_config(config.into()); + let ctx = SessionContext::new_with_config(config.into()); register_aggregate_csv_by_sql(&ctx).await; let sql = "EXPLAIN ANALYZE SELECT c1 FROM aggregate_test_100"; diff --git a/datafusion/core/tests/sql/group_by.rs b/datafusion/core/tests/sql/group_by.rs index 862d2275afc2..49903857b796 100644 --- a/datafusion/core/tests/sql/group_by.rs +++ b/datafusion/core/tests/sql/group_by.rs @@ -149,7 +149,7 @@ async fn create_groupby_context(tmp_dir: &TempDir) -> Result { } let cfg = SessionConfig::new().with_target_partitions(1); - let ctx = SessionContext::with_config(cfg); + let ctx = SessionContext::new_with_config(cfg); ctx.register_csv( "traces", tmp_dir.path().to_str().unwrap(), diff --git a/datafusion/core/tests/sql/joins.rs b/datafusion/core/tests/sql/joins.rs index d08f09d3b6e1..528bde632355 100644 --- a/datafusion/core/tests/sql/joins.rs +++ b/datafusion/core/tests/sql/joins.rs @@ -81,7 +81,7 @@ async fn null_aware_left_anti_join() -> Result<()> { #[tokio::test] async fn join_change_in_planner() -> Result<()> { let config = SessionConfig::new().with_target_partitions(8); - let ctx = SessionContext::with_config(config); + let ctx = SessionContext::new_with_config(config); let tmp_dir = TempDir::new().unwrap(); let left_file_path = tmp_dir.path().join("left.csv"); File::create(left_file_path.clone()).unwrap(); @@ -152,7 +152,7 @@ async fn join_change_in_planner() -> Result<()> { #[tokio::test] async fn join_change_in_planner_without_sort() -> Result<()> { let config = SessionConfig::new().with_target_partitions(8); - let ctx = SessionContext::with_config(config); + let ctx = SessionContext::new_with_config(config); let tmp_dir = TempDir::new()?; let left_file_path = tmp_dir.path().join("left.csv"); File::create(left_file_path.clone())?; @@ -209,7 +209,7 @@ async fn join_change_in_planner_without_sort_not_allowed() -> Result<()> { let config = SessionConfig::new() .with_target_partitions(8) .with_allow_symmetric_joins_without_pruning(false); - let ctx = SessionContext::with_config(config); + let ctx = SessionContext::new_with_config(config); let tmp_dir = TempDir::new()?; let left_file_path = tmp_dir.path().join("left.csv"); File::create(left_file_path.clone())?; diff --git a/datafusion/core/tests/sql/mod.rs b/datafusion/core/tests/sql/mod.rs index 4529889270c6..20d715b32c69 100644 --- a/datafusion/core/tests/sql/mod.rs +++ b/datafusion/core/tests/sql/mod.rs @@ -102,7 +102,7 @@ fn create_join_context( column_right: &str, repartition_joins: bool, ) -> Result { - let ctx = SessionContext::with_config( + let ctx = SessionContext::new_with_config( SessionConfig::new() .with_repartition_joins(repartition_joins) .with_target_partitions(2) @@ -157,7 +157,7 @@ fn create_left_semi_anti_join_context_with_null_ids( column_right: &str, repartition_joins: bool, ) -> Result { - let ctx = SessionContext::with_config( + let ctx = SessionContext::new_with_config( SessionConfig::new() .with_repartition_joins(repartition_joins) .with_target_partitions(2) @@ -524,7 +524,8 @@ async fn create_ctx_with_partition( tmp_dir: &TempDir, partition_count: usize, ) -> Result { - let ctx = SessionContext::with_config(SessionConfig::new().with_target_partitions(8)); + let ctx = + SessionContext::new_with_config(SessionConfig::new().with_target_partitions(8)); let schema = populate_csv_partitions(tmp_dir, partition_count, ".csv")?; diff --git a/datafusion/core/tests/sql/order.rs b/datafusion/core/tests/sql/order.rs index c5497b4cc0f9..0142675bbd49 100644 --- a/datafusion/core/tests/sql/order.rs +++ b/datafusion/core/tests/sql/order.rs @@ -180,7 +180,7 @@ async fn test_issue5970_mini() -> Result<()> { let config = SessionConfig::new() .with_target_partitions(2) .with_repartition_sorts(true); - let ctx = SessionContext::with_config(config); + let ctx = SessionContext::new_with_config(config); let sql = " WITH m0(t) AS ( diff --git a/datafusion/core/tests/sql/partitioned_csv.rs b/datafusion/core/tests/sql/partitioned_csv.rs index 98cb3b189361..d5a1c2f0b4f8 100644 --- a/datafusion/core/tests/sql/partitioned_csv.rs +++ b/datafusion/core/tests/sql/partitioned_csv.rs @@ -78,7 +78,8 @@ pub async fn create_ctx( tmp_dir: &TempDir, partition_count: usize, ) -> Result { - let ctx = SessionContext::with_config(SessionConfig::new().with_target_partitions(8)); + let ctx = + SessionContext::new_with_config(SessionConfig::new().with_target_partitions(8)); let schema = populate_csv_partitions(tmp_dir, partition_count, ".csv")?; diff --git a/datafusion/core/tests/sql/repartition.rs b/datafusion/core/tests/sql/repartition.rs index 20e64b2eeefc..332f18e941aa 100644 --- a/datafusion/core/tests/sql/repartition.rs +++ b/datafusion/core/tests/sql/repartition.rs @@ -33,7 +33,7 @@ use std::sync::Arc; #[tokio::test] async fn unbounded_repartition() -> Result<()> { let config = SessionConfig::new(); - let ctx = SessionContext::with_config(config); + let ctx = SessionContext::new_with_config(config); let task = ctx.task_ctx(); let schema = Arc::new(Schema::new(vec![Field::new("a2", DataType::UInt32, false)])); let batch = RecordBatch::try_new( diff --git a/datafusion/core/tests/sql/select.rs b/datafusion/core/tests/sql/select.rs index cda5fba8051e..63f3e979305a 100644 --- a/datafusion/core/tests/sql/select.rs +++ b/datafusion/core/tests/sql/select.rs @@ -407,7 +407,8 @@ async fn sort_on_window_null_string() -> Result<()> { ]) .unwrap(); - let ctx = SessionContext::with_config(SessionConfig::new().with_target_partitions(1)); + let ctx = + SessionContext::new_with_config(SessionConfig::new().with_target_partitions(1)); ctx.register_batch("test", batch)?; let sql = @@ -590,7 +591,7 @@ async fn boolean_literal() -> Result<()> { #[tokio::test] async fn unprojected_filter() { let config = SessionConfig::new(); - let ctx = SessionContext::with_config(config); + let ctx = SessionContext::new_with_config(config); let df = ctx.read_table(table_with_sequence(1, 3).unwrap()).unwrap(); let df = df diff --git a/datafusion/core/tests/tpcds_planning.rs b/datafusion/core/tests/tpcds_planning.rs index 3f55049ecd3c..4db97c75cb33 100644 --- a/datafusion/core/tests/tpcds_planning.rs +++ b/datafusion/core/tests/tpcds_planning.rs @@ -1045,7 +1045,7 @@ async fn regression_test(query_no: u8, create_physical: bool) -> Result<()> { let sql = fs::read_to_string(filename).expect("Could not read query"); let config = SessionConfig::default(); - let ctx = SessionContext::with_config(config); + let ctx = SessionContext::new_with_config(config); let tables = get_table_definitions(); for table in &tables { ctx.register_table( diff --git a/datafusion/core/tests/user_defined/user_defined_plan.rs b/datafusion/core/tests/user_defined/user_defined_plan.rs index 21ec20f0d4d6..d8c4bea1744f 100644 --- a/datafusion/core/tests/user_defined/user_defined_plan.rs +++ b/datafusion/core/tests/user_defined/user_defined_plan.rs @@ -247,10 +247,10 @@ async fn topk_plan() -> Result<()> { fn make_topk_context() -> SessionContext { let config = SessionConfig::new().with_target_partitions(48); let runtime = Arc::new(RuntimeEnv::default()); - let state = SessionState::with_config_rt(config, runtime) + let state = SessionState::new_with_config_rt(config, runtime) .with_query_planner(Arc::new(TopKQueryPlanner {})) .add_optimizer_rule(Arc::new(TopKOptimizerRule {})); - SessionContext::with_state(state) + SessionContext::new_with_state(state) } // ------ The implementation of the TopK code follows ----- diff --git a/datafusion/proto/tests/cases/roundtrip_logical_plan.rs b/datafusion/proto/tests/cases/roundtrip_logical_plan.rs index cd294b0e535f..5b53946379c8 100644 --- a/datafusion/proto/tests/cases/roundtrip_logical_plan.rs +++ b/datafusion/proto/tests/cases/roundtrip_logical_plan.rs @@ -168,10 +168,10 @@ async fn roundtrip_custom_tables() -> Result<()> { let cfg = RuntimeConfig::new(); let env = RuntimeEnv::new(cfg).unwrap(); let ses = SessionConfig::new(); - let mut state = SessionState::with_config_rt(ses, Arc::new(env)); + let mut state = SessionState::new_with_config_rt(ses, Arc::new(env)); // replace factories *state.table_factories_mut() = table_factories; - let ctx = SessionContext::with_state(state); + let ctx = SessionContext::new_with_state(state); let sql = "CREATE EXTERNAL TABLE t STORED AS testtable LOCATION 's3://bucket/schema/table';"; ctx.sql(sql).await.unwrap(); diff --git a/datafusion/sqllogictest/src/test_context.rs b/datafusion/sqllogictest/src/test_context.rs index 9af2de1af49e..b2314f34f360 100644 --- a/datafusion/sqllogictest/src/test_context.rs +++ b/datafusion/sqllogictest/src/test_context.rs @@ -67,7 +67,7 @@ impl TestContext { // hardcode target partitions so plans are deterministic .with_target_partitions(4); - let test_ctx = TestContext::new(SessionContext::with_config(config)); + let test_ctx = TestContext::new(SessionContext::new_with_config(config)); let file_name = relative_path.file_name().unwrap().to_str().unwrap(); match file_name { diff --git a/datafusion/substrait/tests/cases/roundtrip_logical_plan.rs b/datafusion/substrait/tests/cases/roundtrip_logical_plan.rs index 2554d0667e48..9b9afa159c20 100644 --- a/datafusion/substrait/tests/cases/roundtrip_logical_plan.rs +++ b/datafusion/substrait/tests/cases/roundtrip_logical_plan.rs @@ -726,12 +726,12 @@ async fn function_extension_info(sql: &str) -> Result<(Vec, Vec)> { } async fn create_context() -> Result { - let state = SessionState::with_config_rt( + let state = SessionState::new_with_config_rt( SessionConfig::default(), Arc::new(RuntimeEnv::default()), ) .with_serializer_registry(Arc::new(MockSerializerRegistry)); - let ctx = SessionContext::with_state(state); + let ctx = SessionContext::new_with_state(state); let mut explicit_options = CsvReadOptions::new(); let schema = Schema::new(vec![ Field::new("a", DataType::Int64, true), From 0408c2b1596417ba55a636fa3c8a601ffbdb0e60 Mon Sep 17 00:00:00 2001 From: Andrew Lamb Date: Wed, 4 Oct 2023 17:20:01 -0400 Subject: [PATCH 028/572] Rename `bounded_order_preserving_variants` config to `prefer_exising_sort` and update docs (#7723) * Improve documentation for bounded_order_preserving_variants config * update docs * fmt * update config * fix typo :facepalm * prettier * Reword for clarity --- datafusion/common/src/config.rs | 12 ++++++----- .../enforce_distribution.rs | 11 +++++----- .../replace_with_order_preserving_variants.rs | 4 ++-- datafusion/execution/src/config.rs | 20 +++++++++++-------- .../test_files/information_schema.slt | 2 +- docs/source/user-guide/configs.md | 2 +- 6 files changed, 28 insertions(+), 23 deletions(-) diff --git a/datafusion/common/src/config.rs b/datafusion/common/src/config.rs index 261c2bf435a4..281da1f69e69 100644 --- a/datafusion/common/src/config.rs +++ b/datafusion/common/src/config.rs @@ -453,11 +453,13 @@ config_namespace! { /// ``` pub repartition_sorts: bool, default = true - /// When true, DataFusion will opportunistically remove sorts by replacing - /// `RepartitionExec` with `SortPreservingRepartitionExec`, and - /// `CoalescePartitionsExec` with `SortPreservingMergeExec`, - /// even when the query is bounded. - pub bounded_order_preserving_variants: bool, default = false + /// When true, DataFusion will opportunistically remove sorts when the data is already sorted, + /// (i.e. setting `preserve_order` to true on `RepartitionExec` and + /// using `SortPreservingMergeExec`) + /// + /// When false, DataFusion will maximize plan parallelism using + /// `RepartitionExec` even if this requires subsequently resorting data using a `SortExec`. + pub prefer_existing_sort: bool, default = false /// When set to true, the logical plan optimizer will produce warning /// messages if any optimization rules produce errors and then proceed to the next diff --git a/datafusion/core/src/physical_optimizer/enforce_distribution.rs b/datafusion/core/src/physical_optimizer/enforce_distribution.rs index b3fb41ea100f..3463f3a31376 100644 --- a/datafusion/core/src/physical_optimizer/enforce_distribution.rs +++ b/datafusion/core/src/physical_optimizer/enforce_distribution.rs @@ -1228,7 +1228,7 @@ fn ensure_distribution( // - it is desired according to config // - when plan is unbounded let order_preserving_variants_desirable = - is_unbounded || config.optimizer.bounded_order_preserving_variants; + is_unbounded || config.optimizer.prefer_existing_sort; if dist_context.plan.children().is_empty() { return Ok(Transformed::No(dist_context)); @@ -2085,8 +2085,7 @@ mod tests { config.optimizer.enable_round_robin_repartition = false; config.optimizer.repartition_file_scans = false; config.optimizer.repartition_file_min_size = 1024; - config.optimizer.bounded_order_preserving_variants = - bounded_order_preserving_variants; + config.optimizer.prefer_existing_sort = bounded_order_preserving_variants; ensure_distribution(distribution_context, &config).map(|item| item.into().plan) } @@ -2124,7 +2123,7 @@ mod tests { config.execution.target_partitions = $TARGET_PARTITIONS; config.optimizer.repartition_file_scans = $REPARTITION_FILE_SCANS; config.optimizer.repartition_file_min_size = $REPARTITION_FILE_MIN_SIZE; - config.optimizer.bounded_order_preserving_variants = $BOUNDED_ORDER_PRESERVING_VARIANTS; + config.optimizer.prefer_existing_sort = $BOUNDED_ORDER_PRESERVING_VARIANTS; // NOTE: These tests verify the joint `EnforceDistribution` + `EnforceSorting` cascade // because they were written prior to the separation of `BasicEnforcement` into @@ -4516,7 +4515,7 @@ mod tests { let mut config = ConfigOptions::new(); config.execution.target_partitions = 10; config.optimizer.enable_round_robin_repartition = true; - config.optimizer.bounded_order_preserving_variants = false; + config.optimizer.prefer_existing_sort = false; let distribution_plan = EnforceDistribution::new().optimize(physical_plan, &config)?; assert_plan_txt!(expected, distribution_plan); @@ -4558,7 +4557,7 @@ mod tests { let mut config = ConfigOptions::new(); config.execution.target_partitions = 10; config.optimizer.enable_round_robin_repartition = true; - config.optimizer.bounded_order_preserving_variants = false; + config.optimizer.prefer_existing_sort = false; let distribution_plan = EnforceDistribution::new().optimize(physical_plan, &config)?; assert_plan_txt!(expected, distribution_plan); diff --git a/datafusion/core/src/physical_optimizer/replace_with_order_preserving_variants.rs b/datafusion/core/src/physical_optimizer/replace_with_order_preserving_variants.rs index b0ae199a2da4..cb3b6c3d0741 100644 --- a/datafusion/core/src/physical_optimizer/replace_with_order_preserving_variants.rs +++ b/datafusion/core/src/physical_optimizer/replace_with_order_preserving_variants.rs @@ -251,7 +251,7 @@ pub(crate) fn replace_with_order_preserving_variants( // any case, as doing so helps fix the pipeline. // Also do the replacement if opted-in via config options. let use_order_preserving_variant = - config.optimizer.bounded_order_preserving_variants || unbounded_output(plan); + config.optimizer.prefer_existing_sort || unbounded_output(plan); let updated_sort_input = get_updated_plan( exec_tree, is_spr_better || use_order_preserving_variant, @@ -336,7 +336,7 @@ mod tests { // Run the rule top-down // let optimized_physical_plan = physical_plan.transform_down(&replace_repartition_execs)?; - let config = SessionConfig::new().with_bounded_order_preserving_variants($ALLOW_BOUNDED); + let config = SessionConfig::new().with_prefer_existing_sort($ALLOW_BOUNDED); let plan_with_pipeline_fixer = OrderPreservationContext::new(physical_plan); let parallel = plan_with_pipeline_fixer.transform_up(&|plan_with_pipeline_fixer| replace_with_order_preserving_variants(plan_with_pipeline_fixer, false, false, config.options()))?; let optimized_physical_plan = parallel.plan; diff --git a/datafusion/execution/src/config.rs b/datafusion/execution/src/config.rs index 44fcc2ab49b4..cfcc205b5625 100644 --- a/datafusion/execution/src/config.rs +++ b/datafusion/execution/src/config.rs @@ -145,10 +145,12 @@ impl SessionConfig { self.options.optimizer.repartition_sorts } - /// Remove sorts by replacing with order-preserving variants of operators, - /// even when query is bounded? - pub fn bounded_order_preserving_variants(&self) -> bool { - self.options.optimizer.bounded_order_preserving_variants + /// Prefer existing sort (true) or maximize parallelism (false). See + /// [prefer_existing_sort] for more details + /// + /// [prefer_existing_sort]: datafusion_common::config::OptimizerOptions::prefer_existing_sort + pub fn prefer_existing_sort(&self) -> bool { + self.options.optimizer.prefer_existing_sort } /// Are statistics collected during execution? @@ -221,10 +223,12 @@ impl SessionConfig { self } - /// Enables or disables the use of order-preserving variants of `CoalescePartitions` - /// and `RepartitionExec` operators, even when the query is bounded - pub fn with_bounded_order_preserving_variants(mut self, enabled: bool) -> Self { - self.options.optimizer.bounded_order_preserving_variants = enabled; + /// Prefer existing sort (true) or maximize parallelism (false). See + /// [prefer_existing_sort] for more details + /// + /// [prefer_existing_sort]: datafusion_common::config::OptimizerOptions::prefer_existing_sort + pub fn with_prefer_existing_sort(mut self, enabled: bool) -> Self { + self.options.optimizer.prefer_existing_sort = enabled; self } diff --git a/datafusion/sqllogictest/test_files/information_schema.slt b/datafusion/sqllogictest/test_files/information_schema.slt index 12aa9089a0c9..74c1296fa4ef 100644 --- a/datafusion/sqllogictest/test_files/information_schema.slt +++ b/datafusion/sqllogictest/test_files/information_schema.slt @@ -183,12 +183,12 @@ datafusion.explain.logical_plan_only false datafusion.explain.physical_plan_only false datafusion.explain.show_statistics false datafusion.optimizer.allow_symmetric_joins_without_pruning true -datafusion.optimizer.bounded_order_preserving_variants false datafusion.optimizer.enable_round_robin_repartition true datafusion.optimizer.enable_topk_aggregation true datafusion.optimizer.filter_null_join_keys false datafusion.optimizer.hash_join_single_partition_threshold 1048576 datafusion.optimizer.max_passes 3 +datafusion.optimizer.prefer_existing_sort false datafusion.optimizer.prefer_hash_join true datafusion.optimizer.repartition_aggregations true datafusion.optimizer.repartition_file_min_size 10485760 diff --git a/docs/source/user-guide/configs.md b/docs/source/user-guide/configs.md index 638ac5a36b83..9eb0862de9c1 100644 --- a/docs/source/user-guide/configs.md +++ b/docs/source/user-guide/configs.md @@ -87,7 +87,7 @@ Environment variables are read during `SessionConfig` initialisation so they mus | datafusion.optimizer.repartition_file_scans | true | When set to `true`, file groups will be repartitioned to achieve maximum parallelism. Currently Parquet and CSV formats are supported. If set to `true`, all files will be repartitioned evenly (i.e., a single large file might be partitioned into smaller chunks) for parallel scanning. If set to `false`, different files will be read in parallel, but repartitioning won't happen within a single file. | | datafusion.optimizer.repartition_windows | true | Should DataFusion repartition data using the partitions keys to execute window functions in parallel using the provided `target_partitions` level | | datafusion.optimizer.repartition_sorts | true | Should DataFusion execute sorts in a per-partition fashion and merge afterwards instead of coalescing first and sorting globally. With this flag is enabled, plans in the form below `text "SortExec: [a@0 ASC]", " CoalescePartitionsExec", " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", ` would turn into the plan below which performs better in multithreaded environments `text "SortPreservingMergeExec: [a@0 ASC]", " SortExec: [a@0 ASC]", " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", ` | -| datafusion.optimizer.bounded_order_preserving_variants | false | When true, DataFusion will opportunistically remove sorts by replacing `RepartitionExec` with `SortPreservingRepartitionExec`, and `CoalescePartitionsExec` with `SortPreservingMergeExec`, even when the query is bounded. | +| datafusion.optimizer.prefer_existing_sort | false | When true, DataFusion will opportunistically remove sorts when the data is already sorted, (i.e. setting `preserve_order` to true on `RepartitionExec` and using `SortPreservingMergeExec`) When false, DataFusion will maximize plan parallelism using `RepartitionExec` even if this requires subsequently resorting data using a `SortExec`. | | datafusion.optimizer.skip_failed_rules | false | When set to true, the logical plan optimizer will produce warning messages if any optimization rules produce errors and then proceed to the next rule. When set to false, any rules that produce errors will cause the query to fail | | datafusion.optimizer.max_passes | 3 | Number of times that the optimizer will attempt to optimize the plan | | datafusion.optimizer.top_down_join_key_reordering | true | When set to true, the physical plan optimizer will run a top down process to reorder the join keys | From 1cf808d0e081221cc31b5f92fc414de7c76cf5c4 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Dani=C3=ABl=20Heres?= Date: Thu, 5 Oct 2023 13:27:31 +0200 Subject: [PATCH 029/572] Optimize "ORDER BY + LIMIT" queries for speed / memory with special TopK operator (#7721) * Prototype TopK operator * Avoid use of Row * start working on compaction * checkpoint * update * checkpoint * fmt * Fix compaction * add location for re-encoding * Start sketching dictionary interleave * checkpoint * initial specialized dictionary * finish initial special interleave * Complete dictionary order * Merge * fmt * Cleanup * Fix test * Cleanup * Make test deterministic * Clippy, doctest * Use into_sorted_vec * Fix nondeterministic tests * Update cargo.lock * Update datafusion/physical-plan/src/topk/mod.rs Co-authored-by: Andrew Lamb * Update datafusion/physical-plan/src/topk/mod.rs Co-authored-by: Andrew Lamb * Update datafusion/physical-plan/src/topk/mod.rs Co-authored-by: Andrew Lamb * Update datafusion/physical-plan/src/topk/mod.rs Co-authored-by: Andrew Lamb * Add / update some comments * Rename test file * Rename table as well * Update datafusion/sqllogictest/test_files/topk.slt Co-authored-by: Andrew Lamb --------- Co-authored-by: Andrew Lamb --- datafusion/physical-plan/src/lib.rs | 2 + datafusion/physical-plan/src/sorts/sort.rs | 79 ++- datafusion/physical-plan/src/topk/mod.rs | 644 ++++++++++++++++++ .../sqllogictest/test_files/decimal.slt | 15 +- datafusion/sqllogictest/test_files/topk.slt | 232 +++++++ datafusion/sqllogictest/test_files/window.slt | 111 +-- 6 files changed, 996 insertions(+), 87 deletions(-) create mode 100644 datafusion/physical-plan/src/topk/mod.rs create mode 100644 datafusion/sqllogictest/test_files/topk.slt diff --git a/datafusion/physical-plan/src/lib.rs b/datafusion/physical-plan/src/lib.rs index aca10893db3d..3071fadcb1a2 100644 --- a/datafusion/physical-plan/src/lib.rs +++ b/datafusion/physical-plan/src/lib.rs @@ -17,6 +17,7 @@ //! Traits for physical query plan, supporting parallel execution for partitioned relations. +mod topk; mod visitor; pub use self::metrics::Metric; use self::metrics::MetricsSet; @@ -26,6 +27,7 @@ use self::{ pub use datafusion_common::{internal_err, ColumnStatistics, Statistics}; use datafusion_common::{plan_err, Result}; use datafusion_physical_expr::PhysicalSortExpr; +pub use topk::TopK; pub use visitor::{accept, visit_execution_plan, ExecutionPlanVisitor}; use arrow::datatypes::SchemaRef; diff --git a/datafusion/physical-plan/src/sorts/sort.rs b/datafusion/physical-plan/src/sorts/sort.rs index 7d260d42d9cd..703f80d90d2b 100644 --- a/datafusion/physical-plan/src/sorts/sort.rs +++ b/datafusion/physical-plan/src/sorts/sort.rs @@ -26,6 +26,7 @@ use crate::metrics::{ }; use crate::sorts::merge::streaming_merge; use crate::stream::{RecordBatchReceiverStream, RecordBatchStreamAdapter}; +use crate::topk::TopK; use crate::{ DisplayAs, DisplayFormatType, Distribution, EmptyRecordBatchStream, ExecutionPlan, Partitioning, SendableRecordBatchStream, Statistics, @@ -765,7 +766,12 @@ impl DisplayAs for SortExec { let expr: Vec = self.expr.iter().map(|e| e.to_string()).collect(); match self.fetch { Some(fetch) => { - write!(f, "SortExec: fetch={fetch}, expr=[{}]", expr.join(",")) + write!( + f, + // TODO should this say topk? + "SortExec: fetch={fetch}, expr=[{}]", + expr.join(",") + ) } None => write!(f, "SortExec: expr=[{}]", expr.join(",")), } @@ -853,29 +859,54 @@ impl ExecutionPlan for SortExec { trace!("End SortExec's input.execute for partition: {}", partition); - let mut sorter = ExternalSorter::new( - partition, - input.schema(), - self.expr.clone(), - context.session_config().batch_size(), - self.fetch, - execution_options.sort_spill_reservation_bytes, - execution_options.sort_in_place_threshold_bytes, - &self.metrics_set, - context.runtime_env(), - ); + if let Some(fetch) = self.fetch.as_ref() { + let mut topk = TopK::try_new( + partition, + input.schema(), + self.expr.clone(), + *fetch, + context.session_config().batch_size(), + context.runtime_env(), + &self.metrics_set, + partition, + )?; + + Ok(Box::pin(RecordBatchStreamAdapter::new( + self.schema(), + futures::stream::once(async move { + while let Some(batch) = input.next().await { + let batch = batch?; + topk.insert_batch(batch)?; + } + topk.emit() + }) + .try_flatten(), + ))) + } else { + let mut sorter = ExternalSorter::new( + partition, + input.schema(), + self.expr.clone(), + context.session_config().batch_size(), + self.fetch, + execution_options.sort_spill_reservation_bytes, + execution_options.sort_in_place_threshold_bytes, + &self.metrics_set, + context.runtime_env(), + ); - Ok(Box::pin(RecordBatchStreamAdapter::new( - self.schema(), - futures::stream::once(async move { - while let Some(batch) = input.next().await { - let batch = batch?; - sorter.insert_batch(batch).await?; - } - sorter.sort() - }) - .try_flatten(), - ))) + Ok(Box::pin(RecordBatchStreamAdapter::new( + self.schema(), + futures::stream::once(async move { + while let Some(batch) = input.next().await { + let batch = batch?; + sorter.insert_batch(batch).await?; + } + sorter.sort() + }) + .try_flatten(), + ))) + } } fn metrics(&self) -> Option { @@ -1043,7 +1074,7 @@ mod tests { assert_eq!(result.len(), 1); let metrics = sort_exec.metrics().unwrap(); - let did_it_spill = metrics.spill_count().unwrap() > 0; + let did_it_spill = metrics.spill_count().unwrap_or(0) > 0; assert_eq!(did_it_spill, expect_spillage, "with fetch: {fetch:?}"); } Ok(()) diff --git a/datafusion/physical-plan/src/topk/mod.rs b/datafusion/physical-plan/src/topk/mod.rs new file mode 100644 index 000000000000..4638c0dcf264 --- /dev/null +++ b/datafusion/physical-plan/src/topk/mod.rs @@ -0,0 +1,644 @@ +// 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. + +//! TopK: Combination of Sort / LIMIT + +use arrow::{ + compute::interleave, + row::{RowConverter, Rows, SortField}, +}; +use std::{cmp::Ordering, collections::BinaryHeap, sync::Arc}; + +use arrow_array::{Array, ArrayRef, RecordBatch}; +use arrow_schema::SchemaRef; +use datafusion_common::Result; +use datafusion_execution::{ + memory_pool::{MemoryConsumer, MemoryReservation}, + runtime_env::RuntimeEnv, +}; +use datafusion_physical_expr::PhysicalSortExpr; +use hashbrown::HashMap; + +use crate::{stream::RecordBatchStreamAdapter, SendableRecordBatchStream}; + +use super::metrics::{BaselineMetrics, Count, ExecutionPlanMetricsSet, MetricBuilder}; + +/// Global TopK +/// +/// # Background +/// +/// "Top K" is a common query optimization used for queries such as +/// "find the top 3 customers by revenue". The (simplified) SQL for +/// such a query might be: +/// +/// ```sql +/// SELECT customer_id, revenue FROM 'sales.csv' ORDER BY revenue DESC limit 3; +/// ``` +/// +/// The simple plan would be: +/// +/// ```sql +/// > explain SELECT customer_id, revenue FROM sales ORDER BY revenue DESC limit 3; +/// +--------------+----------------------------------------+ +/// | plan_type | plan | +/// +--------------+----------------------------------------+ +/// | logical_plan | Limit: 3 | +/// | | Sort: revenue DESC NULLS FIRST | +/// | | Projection: customer_id, revenue | +/// | | TableScan: sales | +/// +--------------+----------------------------------------+ +/// ``` +/// +/// While this plan produces the correct answer, it will fully sorts the +/// input before discarding everything other than the top 3 elements. +/// +/// The same answer can be produced by simply keeping track of the top +/// K=3 elements, reducing the total amount of required buffer memory. +/// +/// # Structure +/// +/// This operator tracks the top K items using a `TopKHeap`. +pub struct TopK { + /// schema of the output (and the input) + schema: SchemaRef, + /// Runtime metrics + metrics: TopKMetrics, + /// Reservation + reservation: MemoryReservation, + /// The target number of rows for output batches + batch_size: usize, + /// sort expressions + expr: Arc<[PhysicalSortExpr]>, + /// row converter, for sort keys + row_converter: RowConverter, + /// scratch space for converting rows + scratch_rows: Rows, + /// stores the top k values and their sort key values, in order + heap: TopKHeap, +} + +impl TopK { + /// Create a new [`TopK`] that stores the top `k` values, as + /// defined by the sort expressions in `expr`. + // TOOD: make a builder or some other nicer API to avoid the + // clippy warning + #[allow(clippy::too_many_arguments)] + pub fn try_new( + partition_id: usize, + schema: SchemaRef, + expr: Vec, + k: usize, + batch_size: usize, + runtime: Arc, + metrics: &ExecutionPlanMetricsSet, + partition: usize, + ) -> Result { + let reservation = MemoryConsumer::new(format!("TopK[{partition_id}]")) + .register(&runtime.memory_pool); + + let expr: Arc<[PhysicalSortExpr]> = expr.into(); + + let sort_fields: Vec<_> = expr + .iter() + .map(|e| { + Ok(SortField::new_with_options( + e.expr.data_type(&schema)?, + e.options, + )) + }) + .collect::>()?; + + // TODO there is potential to add special cases for single column sort fields + // to improve performance + let row_converter = RowConverter::new(sort_fields)?; + let scratch_rows = row_converter.empty_rows( + batch_size, + 20 * batch_size, // guestimate 20 bytes per row + ); + + Ok(Self { + schema: schema.clone(), + metrics: TopKMetrics::new(metrics, partition), + reservation, + batch_size, + expr, + row_converter, + scratch_rows, + heap: TopKHeap::new(k, batch_size, schema), + }) + } + + /// Insert `batch`, remembering if any of its values are among + /// the top k seen so far. + pub fn insert_batch(&mut self, batch: RecordBatch) -> Result<()> { + // Updates on drop + let _timer = self.metrics.baseline.elapsed_compute().timer(); + + let sort_keys: Vec = self + .expr + .iter() + .map(|expr| { + let value = expr.expr.evaluate(&batch)?; + Ok(value.into_array(batch.num_rows())) + }) + .collect::>>()?; + + // reuse existing `Rows` to avoid reallocations + let rows = &mut self.scratch_rows; + rows.clear(); + self.row_converter.append(rows, &sort_keys)?; + + // TODO make this algorithmically better?: + // Idea: filter out rows >= self.heap.max() early (before passing to `RowConverter`) + // this avoids some work and also might be better vectorizable. + let mut batch_entry = self.heap.register_batch(batch); + for (index, row) in rows.iter().enumerate() { + match self.heap.max() { + // heap has k items, and the new row is greater than the + // current max in the heap ==> it is not a new topk + Some(max_row) if row.as_ref() >= max_row.row() => {} + // don't yet have k items or new item is lower than the currently k low values + None | Some(_) => { + self.heap.add(&mut batch_entry, row, index); + self.metrics.row_replacements.add(1); + } + } + } + self.heap.insert_batch_entry(batch_entry); + + // conserve memory + self.heap.maybe_compact()?; + + // update memory reservation + self.reservation.try_resize(self.size())?; + Ok(()) + } + + /// Returns the top k results broken into `batch_size` [`RecordBatch`]es, consuming the heap + pub fn emit(self) -> Result { + let Self { + schema, + metrics, + reservation: _, + batch_size, + expr: _, + row_converter: _, + scratch_rows: _, + mut heap, + } = self; + let _timer = metrics.baseline.elapsed_compute().timer(); // time updated on drop + + let mut batch = heap.emit()?; + metrics.baseline.output_rows().add(batch.num_rows()); + + // break into record batches as needed + let mut batches = vec![]; + loop { + if batch.num_rows() < batch_size { + batches.push(Ok(batch)); + break; + } else { + batches.push(Ok(batch.slice(0, batch_size))); + let remaining_length = batch.num_rows() - batch_size; + batch = batch.slice(batch_size, remaining_length); + } + } + Ok(Box::pin(RecordBatchStreamAdapter::new( + schema, + futures::stream::iter(batches), + ))) + } + + /// return the size of memory used by this operator, in bytes + fn size(&self) -> usize { + std::mem::size_of::() + + self.row_converter.size() + + self.scratch_rows.size() + + self.heap.size() + } +} + +struct TopKMetrics { + /// metrics + pub baseline: BaselineMetrics, + + /// count of how many rows were replaced in the heap + pub row_replacements: Count, +} + +impl TopKMetrics { + fn new(metrics: &ExecutionPlanMetricsSet, partition: usize) -> Self { + Self { + baseline: BaselineMetrics::new(metrics, partition), + row_replacements: MetricBuilder::new(metrics) + .counter("row_replacements", partition), + } + } +} + +/// This structure keeps at most the *smallest* k items, using the +/// [arrow::row] format for sort keys. While it is called "topK" for +/// values like `1, 2, 3, 4, 5` the "top 3" really means the +/// *smallest* 3 , `1, 2, 3`, not the *largest* 3 `3, 4, 5`. +/// +/// Using the `Row` format handles things such as ascending vs +/// descending and nulls first vs nulls last. +struct TopKHeap { + /// The maximum number of elemenents to store in this heap. + k: usize, + /// The target number of rows for output batches + batch_size: usize, + /// Storage for up at most `k` items using a BinaryHeap. Reverserd + /// so that the smallest k so far is on the top + inner: BinaryHeap, + /// Storage the original row values (TopKRow only has the sort key) + store: RecordBatchStore, + /// The size of all owned data held by this heap + owned_bytes: usize, +} + +impl TopKHeap { + fn new(k: usize, batch_size: usize, schema: SchemaRef) -> Self { + assert!(k > 0); + Self { + k, + batch_size, + inner: BinaryHeap::new(), + store: RecordBatchStore::new(schema), + owned_bytes: 0, + } + } + + /// Register a [`RecordBatch`] with the heap, returning the + /// appropriate entry + pub fn register_batch(&mut self, batch: RecordBatch) -> RecordBatchEntry { + self.store.register(batch) + } + + /// Insert a [`RecordBatchEntry`] created by a previous call to + /// [`Self::register_batch`] into storage. + pub fn insert_batch_entry(&mut self, entry: RecordBatchEntry) { + self.store.insert(entry) + } + + /// Returns the largest value stored by the heap if there are k + /// items, otherwise returns None. Remember this structure is + /// keeping the "smallest" k values + fn max(&self) -> Option<&TopKRow> { + if self.inner.len() < self.k { + None + } else { + self.inner.peek() + } + } + + /// Adds `row` to this heap. If inserting this new item would + /// increase the size past `k`, removes the previously smallest + /// item. + fn add( + &mut self, + batch_entry: &mut RecordBatchEntry, + row: impl AsRef<[u8]>, + index: usize, + ) { + let batch_id = batch_entry.id; + batch_entry.uses += 1; + + assert!(self.inner.len() <= self.k); + let row = row.as_ref(); + + // Reuse storage for evicted item if possible + let new_top_k = if self.inner.len() == self.k { + let prev_min = self.inner.pop().unwrap(); + + // Update batch use + if prev_min.batch_id == batch_entry.id { + batch_entry.uses -= 1; + } else { + self.store.unuse(prev_min.batch_id); + } + + // update memory accounting + self.owned_bytes -= prev_min.owned_size(); + prev_min.with_new_row(row, batch_id, index) + } else { + TopKRow::new(row, batch_id, index) + }; + + self.owned_bytes += new_top_k.owned_size(); + + // put the new row into the heap + self.inner.push(new_top_k) + } + + /// Returns the values stored in this heap, from values low to + /// high, as a single [`RecordBatch`], resetting the inner heap + pub fn emit(&mut self) -> Result { + Ok(self.emit_with_state()?.0) + } + + /// Returns the values stored in this heap, from values low to + /// high, as a single [`RecordBatch`], and a sorted vec of the + /// current heap's contents + pub fn emit_with_state(&mut self) -> Result<(RecordBatch, Vec)> { + let schema = self.store.schema().clone(); + + // generate sorted rows + let topk_rows = std::mem::take(&mut self.inner).into_sorted_vec(); + + if self.store.is_empty() { + return Ok((RecordBatch::new_empty(schema), topk_rows)); + } + + // Indices for each row within its respective RecordBatch + let indices: Vec<_> = topk_rows + .iter() + .enumerate() + .map(|(i, k)| (i, k.index)) + .collect(); + + let num_columns = schema.fields().len(); + + // build the output columns one at time, using the + // `interleave` kernel to pick rows from different arrays + let output_columns: Vec<_> = (0..num_columns) + .map(|col| { + let input_arrays: Vec<_> = topk_rows + .iter() + .map(|k| { + let entry = + self.store.get(k.batch_id).expect("invalid stored batch id"); + entry.batch.column(col) as &dyn Array + }) + .collect(); + + // at this point `indices` contains indexes within the + // rows and `input_arrays` contains a reference to the + // relevant Array for that index. `interleave` pulls + // them together into a single new array + Ok(interleave(&input_arrays, &indices)?) + }) + .collect::>()?; + + let new_batch = RecordBatch::try_new(schema, output_columns)?; + Ok((new_batch, topk_rows)) + } + + /// Compact this heap, rewriting all stored batches into a single + /// input batch + pub fn maybe_compact(&mut self) -> Result<()> { + // we compact if the number of "unused" rows in the store is + // past some pre-defined threshold. Target holding up to + // around 20 batches, but handle cases of large k where some + // batches might be partially full + let max_unused_rows = (20 * self.batch_size) + self.k; + let unused_rows = self.store.unused_rows(); + + // don't compact if the store has one extra batch or + // unused rows is under the threshold + if self.store.len() <= 2 || unused_rows < max_unused_rows { + return Ok(()); + } + // at first, compact the entire thing always into a new batch + // (maybe we can get fancier in the future about ignoring + // batches that have a high usage ratio already + + // Note: new batch is in the same order as inner + let num_rows = self.inner.len(); + let (new_batch, mut topk_rows) = self.emit_with_state()?; + + // clear all old entires in store (this invalidates all + // store_ids in `inner`) + self.store.clear(); + + let mut batch_entry = self.register_batch(new_batch); + batch_entry.uses = num_rows; + + // rewrite all existing entries to use the new batch, and + // remove old entries. The sortedness and their relative + // position do not change + for (i, topk_row) in topk_rows.iter_mut().enumerate() { + topk_row.batch_id = batch_entry.id; + topk_row.index = i; + } + self.insert_batch_entry(batch_entry); + // restore the heap + self.inner = BinaryHeap::from(topk_rows); + + Ok(()) + } + + /// return the size of memory used by this heap, in bytes + fn size(&self) -> usize { + std::mem::size_of::() + + (self.inner.capacity() * std::mem::size_of::()) + + self.store.size() + + self.owned_bytes + } +} + +/// Represents one of the top K rows held in this heap. Orders +/// according to memcmp of row (e.g. the arrow Row format, but could +/// also be primtive values) +/// +/// Reuses allocations to minimize runtime overhead of creating new Vecs +#[derive(Debug, PartialEq)] +struct TopKRow { + /// the value of the sort key for this row. This contains the + /// bytes that could be stored in `OwnedRow` but uses `Vec` to + /// reuse allocations. + row: Vec, + /// the RecordBatch this row came from: an id into a [`RecordBatchStore`] + batch_id: u32, + /// the index in this record batch the row came from + index: usize, +} + +impl TopKRow { + /// Create a new TopKRow with new allocation + fn new(row: impl AsRef<[u8]>, batch_id: u32, index: usize) -> Self { + Self { + row: row.as_ref().to_vec(), + batch_id, + index, + } + } + + /// Create a new TopKRow reusing the existing allocation + fn with_new_row( + self, + new_row: impl AsRef<[u8]>, + batch_id: u32, + index: usize, + ) -> Self { + let Self { + mut row, + batch_id: _, + index: _, + } = self; + row.clear(); + row.extend_from_slice(new_row.as_ref()); + + Self { + row, + batch_id, + index, + } + } + + /// Returns the number of bytes owned by this row in the heap (not + /// including itself) + fn owned_size(&self) -> usize { + self.row.capacity() + } + + /// Returns a slice to the owned row value + fn row(&self) -> &[u8] { + self.row.as_slice() + } +} + +impl Eq for TopKRow {} + +impl PartialOrd for TopKRow { + fn partial_cmp(&self, other: &Self) -> Option { + Some(self.cmp(other)) + } +} + +impl Ord for TopKRow { + fn cmp(&self, other: &Self) -> Ordering { + self.row.cmp(&other.row) + } +} + +#[derive(Debug)] +struct RecordBatchEntry { + id: u32, + batch: RecordBatch, + // for this batch, how many times has it been used + uses: usize, +} + +/// This structure tracks [`RecordBatch`] by an id so that: +/// +/// 1. The baches can be tracked via an id that can be copied cheaply +/// 2. The total memory held by all batches is tracked +#[derive(Debug)] +struct RecordBatchStore { + /// id generator + next_id: u32, + /// storage + batches: HashMap, + /// total size of all record batches tracked by this store + batches_size: usize, + /// schema of the batches + schema: SchemaRef, +} + +impl RecordBatchStore { + fn new(schema: SchemaRef) -> Self { + Self { + next_id: 0, + batches: HashMap::new(), + batches_size: 0, + schema, + } + } + + /// Register this batch with the store and assign an ID. No + /// attempt is made to compare this batch to other batches + pub fn register(&mut self, batch: RecordBatch) -> RecordBatchEntry { + let id = self.next_id; + self.next_id += 1; + RecordBatchEntry { id, batch, uses: 0 } + } + + /// Insert a record batch entry into this store, tracking its + /// memory use, if it has any uses + pub fn insert(&mut self, entry: RecordBatchEntry) { + // uses of 0 means that none of the rows in the batch were stored in the topk + if entry.uses > 0 { + self.batches_size += entry.batch.get_array_memory_size(); + self.batches.insert(entry.id, entry); + } + } + + /// Clear all values in this store, invalidating all previous batch ids + fn clear(&mut self) { + self.batches.clear(); + self.batches_size = 0; + } + + fn get(&self, id: u32) -> Option<&RecordBatchEntry> { + self.batches.get(&id) + } + + /// returns the total number of batches stored in this store + fn len(&self) -> usize { + self.batches.len() + } + + /// Returns the total number of rows in batches minus the number + /// which are in use + fn unused_rows(&self) -> usize { + self.batches + .values() + .map(|batch_entry| batch_entry.batch.num_rows() - batch_entry.uses) + .sum() + } + + /// returns true if the store has nothing stored + fn is_empty(&self) -> bool { + self.batches.is_empty() + } + + /// return the schema of batches stored + fn schema(&self) -> &SchemaRef { + &self.schema + } + + /// remove a use from the specified batch id. If the use count + /// reaches zero the batch entry is removed from the store + /// + /// panics if there were no remaining uses of id + pub fn unuse(&mut self, id: u32) { + let remove = if let Some(batch_entry) = self.batches.get_mut(&id) { + batch_entry.uses = batch_entry.uses.checked_sub(1).expect("underflow"); + batch_entry.uses == 0 + } else { + panic!("No entry for id {id}"); + }; + + if remove { + let old_entry = self.batches.remove(&id).unwrap(); + self.batches_size = self + .batches_size + .checked_sub(old_entry.batch.get_array_memory_size()) + .unwrap(); + } + } + + /// returns the size of memory used by this store, including all + /// referenced `RecordBatch`es, in bytes + pub fn size(&self) -> usize { + std::mem::size_of::() + + self.batches.capacity() + * (std::mem::size_of::() + std::mem::size_of::()) + + self.batches_size + } +} diff --git a/datafusion/sqllogictest/test_files/decimal.slt b/datafusion/sqllogictest/test_files/decimal.slt index a326a0cc4941..d7632138a84e 100644 --- a/datafusion/sqllogictest/test_files/decimal.slt +++ b/datafusion/sqllogictest/test_files/decimal.slt @@ -507,27 +507,26 @@ select * from decimal_simple where c1 >= 0.00004 order by c1; query RRIBR -select * from decimal_simple where c1 >= 0.00004 order by c1 limit 10; +select * from decimal_simple where c1 >= 0.00004 order by c1, c3 limit 10; ---- 0.00004 0.000000000004 5 true 0.000044 +0.00004 0.000000000004 8 false 0.000044 0.00004 0.000000000004 12 false 0.00004 0.00004 0.000000000004 14 true 0.00004 -0.00004 0.000000000004 8 false 0.000044 -0.00005 0.000000000005 9 true 0.000052 +0.00005 0.000000000005 1 false 0.0001 0.00005 0.000000000005 4 true 0.000078 0.00005 0.000000000005 8 false 0.000033 +0.00005 0.000000000005 9 true 0.000052 0.00005 0.000000000005 100 true 0.000068 -0.00005 0.000000000005 1 false 0.0001 - query RRIBR -select * from decimal_simple where c1 >= 0.00004 order by c1 limit 5; +select * from decimal_simple where c1 >= 0.00004 order by c1, c3 limit 5; ---- 0.00004 0.000000000004 5 true 0.000044 +0.00004 0.000000000004 8 false 0.000044 0.00004 0.000000000004 12 false 0.00004 0.00004 0.000000000004 14 true 0.00004 -0.00004 0.000000000004 8 false 0.000044 -0.00005 0.000000000005 9 true 0.000052 +0.00005 0.000000000005 1 false 0.0001 query RRIBR diff --git a/datafusion/sqllogictest/test_files/topk.slt b/datafusion/sqllogictest/test_files/topk.slt new file mode 100644 index 000000000000..8d3b70139d35 --- /dev/null +++ b/datafusion/sqllogictest/test_files/topk.slt @@ -0,0 +1,232 @@ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at + +# http://www.apache.org/licenses/LICENSE-2.0 + +# Unless required by applicable law or agreed to in writing, +# software distributed under the License is distributed on an +# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +# KIND, either express or implied. See the License for the +# specific language governing permissions and limitations +# under the License. + +# Tests for development + +statement ok +create table topk(x int) as values (10), (2), (3), (0), (5), (4), (3), (2), (1), (3), (8); + +query I +select * from topk order by x; +---- +0 +1 +2 +2 +3 +3 +3 +4 +5 +8 +10 + +query I +select * from topk order by x limit 3; +---- +0 +1 +2 + +query I +select * from topk order by x desc limit 3; +---- +10 +8 +5 + + + + +statement ok +CREATE EXTERNAL TABLE aggregate_test_100 ( + c1 VARCHAR NOT NULL, + c2 TINYINT NOT NULL, + c3 SMALLINT NOT NULL, + c4 SMALLINT, + c5 INT, + c6 BIGINT NOT NULL, + c7 SMALLINT NOT NULL, + c8 INT NOT NULL, + c9 BIGINT UNSIGNED NOT NULL, + c10 VARCHAR NOT NULL, + c11 FLOAT NOT NULL, + c12 DOUBLE NOT NULL, + c13 VARCHAR NOT NULL +) +STORED AS CSV +WITH HEADER ROW +LOCATION '../../testing/data/csv/aggregate_test_100.csv' + +query TT +explain select * from aggregate_test_100 ORDER BY c13 desc limit 5; +---- +logical_plan +Limit: skip=0, fetch=5 +--Sort: aggregate_test_100.c13 DESC NULLS FIRST, fetch=5 +----TableScan: aggregate_test_100 projection=[c1, c2, c3, c4, c5, c6, c7, c8, c9, c10, c11, c12, c13] +physical_plan +GlobalLimitExec: skip=0, fetch=5 +--SortExec: fetch=5, expr=[c13@12 DESC] +----CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1, c2, c3, c4, c5, c6, c7, c8, c9, c10, c11, c12, c13], has_header=true + + + + +query T +select c13 from aggregate_test_100 ORDER BY c13; +---- +0VVIHzxWtNOFLtnhjHEKjXaJOSLJfm +0keZ5G8BffGwgF2RwQD59TFzMStxCB +0og6hSkhbX8AC1ktFS4kounvTzy8Vo +1aOcrEGd0cOqZe2I5XBOm0nDcwtBZO +2T3wSlHdEmASmO0xcXHnndkKEt6bz8 +3BEOHQsMEFZ58VcNTOJYShTBpAPzbt +4HX6feIvmNXBN7XGqgO4YVBkhu8GDI +4JznSdBajNWhu4hRQwjV1FjTTxY68i +52mKlRE3aHCBZtjECq6sY9OqVf8Dze +56MZa5O1hVtX4c5sbnCfxuX5kDChqI +6FPJlLAcaQ5uokyOWZ9HGdLZObFvOZ +6WfVFBVGJSQb7FhA7E0lBwdvjfZnSW +6oIXZuIPIqEoPBvFmbt2Nxy3tryGUE +6x93sxYioWuq5c9Kkk8oTAAORM7cH0 +802bgTGl6Bk5TlkPYYTxp5JkKyaYUA +8LIh0b6jmDGm87BmIyjdxNIpX4ugjD +90gAtmGEeIqUTbo1ZrxCvWtsseukXC +9UbObCsVkmYpJGcGrgfK90qOnwb2Lj +AFGCj7OWlEB5QfniEFgonMq90Tq5uH +ALuRhobVWbnQTTWZdSOk0iVe8oYFhW +Amn2K87Db5Es3dFQO9cw9cvpAM6h35 +AyYVExXK6AR2qUTxNZ7qRHQOVGMLcz +BJqx5WokrmrrezZA0dUbleMYkG5U2O +BPtQMxnuSPpxMExYV9YkDa6cAN7GP3 +BsM5ZAYifRh5Lw3Y8X1r53I0cTJnfE +C2GT5KVyOPZpgKVl110TyZO0NcJ434 +DuJNG8tufSqW0ZstHqWj3aGvFLMg4A +EcCuckwsF3gV1Ecgmh5v4KM8g1ozif +ErJFw6hzZ5fmI5r8bhE4JzlscnhKZU +F7NSTjWvQJyBburN7CXRUlbgp2dIrA +Fi4rJeTQq4eXj8Lxg3Hja5hBVTVV5u +H5j5ZHy1FGesOAHjkQEDYCucbpKWRu +HKSMQ9nTnwXCJIte1JrM1dtYnDtJ8g +IWl0G3ZlMNf7WT8yjIB49cx7MmYOmr +IZTkHMLvIKuiLjhDjYMmIHxh166we4 +Ig1QcuKsjHXkproePdERo2w0mYzIqd +JHNgc2UCaiXOdmkxwDDyGhRlO0mnBQ +JN0VclewmjwYlSl8386MlWv5rEhWCz +JafwVLSVk5AVoXFuzclesQ000EE2k1 +KJFcmTVjdkCMv94wYCtfHMFhzyRsmH +Ktb7GQ0N1DrxwkCkEUsTaIXk0xYinn +Ld2ej8NEv5zNcqU60FwpHeZKBhfpiV +LiEBxds3X0Uw0lxiYjDqrkAaAwoiIW +MXhhH1Var3OzzJCtI9VNyYvA0q8UyJ +MeSTAXq8gVxVjbEjgkvU9YLte0X9uE +NEhyk8uIx4kEULJGa8qIyFjjBcP2G6 +O66j6PaYuZhEUtqV6fuU7TyjM2WxC5 +OF7fQ37GzaZ5ikA2oMyvleKtgnLjXh +OPwBqCEK5PWTjWaiOyL45u2NLTaDWv +Oq6J4Rx6nde0YlhOIJkFsX2MsSvAQ0 +Ow5PGpfTm4dXCfTDsXAOTatXRoAydR +QEHVvcP8gxI6EMJIrvcnIhgzPNjIvv +QJYm7YRA3YetcBHI5wkMZeLXVmfuNy +QYlaIAnJA6r8rlAb6f59wcxvcPcWFf +RilTlL1tKkPOUFuzmLydHAVZwv1OGl +Sfx0vxv1skzZWT1PqVdoRDdO6Sb6xH +TTQUwpMNSXZqVBKAFvXu7OlWvKXJKX +TtDKUZxzVxsq758G6AWPSYuZgVgbcl +VDhtJkYjAYPykCgOU9x3v7v3t4SO1a +VY0zXmXeksCT8BzvpzpPLbmU9Kp9Y4 +Vp3gmWunM5A7wOC9YW2JroFqTWjvTi +WHmjWk2AY4c6m7DA4GitUx6nmb1yYS +XemNcT1xp61xcM1Qz3wZ1VECCnq06O +Z2sWcQr0qyCJRMHDpRy3aQr7PkHtkK +aDxBtor7Icd9C5hnTvvw5NrIre740e +akiiY5N0I44CMwEnBL6RTBk7BRkxEj +b3b9esRhTzFEawbs6XhpKnD9ojutHB +bgK1r6v3BCTh0aejJUhkA1Hn6idXGp +cBGc0kSm32ylBDnxogG727C0uhZEYZ +cq4WSAIFwx3wwTUS5bp1wCe71R6U5I +dVdvo6nUD5FgCgsbOZLds28RyGTpnx +e2Gh6Ov8XkXoFdJWhl0EjwEHlMDYyG +f9ALCzwDAKmdu7Rk2msJaB1wxe5IBX +fuyvs0w7WsKSlXqJ1e6HFSoLmx03AG +gTpyQnEODMcpsPnJMZC66gh33i3m0b +gpo8K5qtYePve6jyPt6xgJx4YOVjms +gxfHWUF8XgY2KdFxigxvNEXe2V2XMl +i6RQVXKUh7MzuGMDaNclUYnFUAireU +ioEncce3mPOXD2hWhpZpCPWGATG6GU +jQimhdepw3GKmioWUlVSWeBVRKFkY3 +l7uwDoTepWwnAP0ufqtHJS3CRi7RfP +lqhzgLsXZ8JhtpeeUWWNbMz8PHI705 +m6jD0LBIQWaMfenwRCTANI9eOdyyto +mhjME0zBHbrK6NMkytMTQzOssOa1gF +mzbkwXKrPeZnxg2Kn1LRF5hYSsmksS +nYVJnVicpGRqKZibHyBAmtmzBXAFfT +oHJMNvWuunsIMIWFnYG31RCfkOo2V7 +oLZ21P2JEDooxV1pU31cIxQHEeeoLu +okOkcWflkNXIy4R8LzmySyY1EC3sYd +pLk3i59bZwd5KBZrI1FiweYTd5hteG +pTeu0WMjBRTaNRT15rLCuEh3tBJVc5 +qnPOOmslCJaT45buUisMRnM0rc77EK +t6fQUjJejPcjc04wHvHTPe55S65B4V +ukOiFGGFnQJDHFgZxHMpvhD3zybF0M +ukyD7b0Efj7tNlFSRmzZ0IqkEzg2a8 +waIGbOGl1PM6gnzZ4uuZt4E2yDWRHs +wwXqSGKLyBQyPkonlzBNYUJTCo4LRS +xipQ93429ksjNcXPX5326VSg1xJZcW +y7C453hRWd4E7ImjNDWlpexB8nUqjh +ydkwycaISlYSlEq3TlkS2m15I2pcp8 + + +query TIIIIIIIITRRT +select * from aggregate_test_100 ORDER BY c13 desc limit 5; +---- +a 4 -38 20744 762932956 308913475857409919 7 45465 1787652631 878137512938218976 0.7459874 0.021825780392 ydkwycaISlYSlEq3TlkS2m15I2pcp8 +d 1 -98 13630 -1991133944 1184110014998006843 220 2986 225513085 9634106610243643486 0.89651865 0.164088254508 y7C453hRWd4E7ImjNDWlpexB8nUqjh +e 2 52 -12056 -1090239422 9011500141803970147 238 4168 2013662838 12565360638488684051 0.6694766 0.391444365692 xipQ93429ksjNcXPX5326VSg1xJZcW +d 1 -72 25590 1188089983 3090286296481837049 241 832 3542840110 5885937420286765261 0.41980565 0.215354023438 wwXqSGKLyBQyPkonlzBNYUJTCo4LRS +a 1 -5 12636 794623392 2909750622865366631 15 24022 2669374863 4776679784701509574 0.29877836 0.253725340799 waIGbOGl1PM6gnzZ4uuZt4E2yDWRHs + + + +## -- make tiny batches to trigger batch compaction +statement ok +set datafusion.execution.batch_size = 2 + +query TIIIIIIIITRRT +select * from aggregate_test_100 ORDER BY c13 desc limit 5; +---- +a 4 -38 20744 762932956 308913475857409919 7 45465 1787652631 878137512938218976 0.7459874 0.021825780392 ydkwycaISlYSlEq3TlkS2m15I2pcp8 +d 1 -98 13630 -1991133944 1184110014998006843 220 2986 225513085 9634106610243643486 0.89651865 0.164088254508 y7C453hRWd4E7ImjNDWlpexB8nUqjh +e 2 52 -12056 -1090239422 9011500141803970147 238 4168 2013662838 12565360638488684051 0.6694766 0.391444365692 xipQ93429ksjNcXPX5326VSg1xJZcW +d 1 -72 25590 1188089983 3090286296481837049 241 832 3542840110 5885937420286765261 0.41980565 0.215354023438 wwXqSGKLyBQyPkonlzBNYUJTCo4LRS +a 1 -5 12636 794623392 2909750622865366631 15 24022 2669374863 4776679784701509574 0.29877836 0.253725340799 waIGbOGl1PM6gnzZ4uuZt4E2yDWRHs + + +## make an example for dictionary encoding + +statement ok +create table dict as select c1, c2, c3, c13, arrow_cast(c13, 'Dictionary(Int32, Utf8)') as c13_dict from aggregate_test_100; + +query TIIT? +select * from dict order by c13 desc limit 5; +---- +a 4 -38 ydkwycaISlYSlEq3TlkS2m15I2pcp8 ydkwycaISlYSlEq3TlkS2m15I2pcp8 +d 1 -98 y7C453hRWd4E7ImjNDWlpexB8nUqjh y7C453hRWd4E7ImjNDWlpexB8nUqjh +e 2 52 xipQ93429ksjNcXPX5326VSg1xJZcW xipQ93429ksjNcXPX5326VSg1xJZcW +d 1 -72 wwXqSGKLyBQyPkonlzBNYUJTCo4LRS wwXqSGKLyBQyPkonlzBNYUJTCo4LRS +a 1 -5 waIGbOGl1PM6gnzZ4uuZt4E2yDWRHs waIGbOGl1PM6gnzZ4uuZt4E2yDWRHs diff --git a/datafusion/sqllogictest/test_files/window.slt b/datafusion/sqllogictest/test_files/window.slt index b6325fd889ec..5fb5a04c6709 100644 --- a/datafusion/sqllogictest/test_files/window.slt +++ b/datafusion/sqllogictest/test_files/window.slt @@ -2601,6 +2601,7 @@ SELECT # test_source_sorted_builtin query TT EXPLAIN SELECT + ts, FIRST_VALUE(inc_col) OVER(ORDER BY ts RANGE BETWEEN 10 PRECEDING and 1 FOLLOWING) as fv1, FIRST_VALUE(inc_col) OVER(ORDER BY ts ROWS BETWEEN 10 PRECEDING and 1 FOLLOWING) as fv2, LAST_VALUE(inc_col) OVER(ORDER BY ts RANGE BETWEEN 10 PRECEDING and 1 FOLLOWING) as lv1, @@ -2630,24 +2631,23 @@ EXPLAIN SELECT LIMIT 5; ---- logical_plan -Projection: fv1, fv2, lv1, lv2, nv1, nv2, rn1, rn2, rank1, rank2, dense_rank1, dense_rank2, lag1, lag2, lead1, lead2, fvr1, fvr2, lvr1, lvr2, lagr1, lagr2, leadr1, leadr2 ---Limit: skip=0, fetch=5 -----Sort: annotated_data_finite.ts DESC NULLS FIRST, fetch=5 -------Projection: FIRST_VALUE(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING AS fv1, FIRST_VALUE(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING AS fv2, LAST_VALUE(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING AS lv1, LAST_VALUE(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING AS lv2, NTH_VALUE(annotated_data_finite.inc_col,Int64(5)) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING AS nv1, NTH_VALUE(annotated_data_finite.inc_col,Int64(5)) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING AS nv2, ROW_NUMBER() ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING AS rn1, ROW_NUMBER() ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING AS rn2, RANK() ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING AS rank1, RANK() ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING AS rank2, DENSE_RANK() ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING AS dense_rank1, DENSE_RANK() ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING AS dense_rank2, LAG(annotated_data_finite.inc_col,Int64(1),Int64(1001)) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING AS lag1, LAG(annotated_data_finite.inc_col,Int64(2),Int64(1002)) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING AS lag2, LEAD(annotated_data_finite.inc_col,Int64(-1),Int64(1001)) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING AS lead1, LEAD(annotated_data_finite.inc_col,Int64(4),Int64(1004)) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING AS lead2, FIRST_VALUE(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING AS fvr1, FIRST_VALUE(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING AS fvr2, LAST_VALUE(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING AS lvr1, LAST_VALUE(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING AS lvr2, LAG(annotated_data_finite.inc_col,Int64(1),Int64(1001)) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING AS lagr1, LAG(annotated_data_finite.inc_col,Int64(2),Int64(1002)) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING AS lagr2, LEAD(annotated_data_finite.inc_col,Int64(-1),Int64(1001)) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING AS leadr1, LEAD(annotated_data_finite.inc_col,Int64(4),Int64(1004)) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING AS leadr2, annotated_data_finite.ts ---------WindowAggr: windowExpr=[[FIRST_VALUE(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING, FIRST_VALUE(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING, LAST_VALUE(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING, LAST_VALUE(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING, NTH_VALUE(annotated_data_finite.inc_col, Int64(5)) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING, NTH_VALUE(annotated_data_finite.inc_col, Int64(5)) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING, ROW_NUMBER() ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING, ROW_NUMBER() ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING, RANK() ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING, RANK() ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING, DENSE_RANK() ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING, DENSE_RANK() ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING, LAG(annotated_data_finite.inc_col, Int64(1), Int64(1001)) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING, LAG(annotated_data_finite.inc_col, Int64(2), Int64(1002)) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING, LEAD(annotated_data_finite.inc_col, Int64(-1), Int64(1001)) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING, LEAD(annotated_data_finite.inc_col, Int64(4), Int64(1004)) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING]] -----------WindowAggr: windowExpr=[[FIRST_VALUE(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING, FIRST_VALUE(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING, LAST_VALUE(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING, LAST_VALUE(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING, LAG(annotated_data_finite.inc_col, Int64(1), Int64(1001)) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING, LAG(annotated_data_finite.inc_col, Int64(2), Int64(1002)) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING, LEAD(annotated_data_finite.inc_col, Int64(-1), Int64(1001)) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING, LEAD(annotated_data_finite.inc_col, Int64(4), Int64(1004)) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING]] -------------TableScan: annotated_data_finite projection=[ts, inc_col] +Limit: skip=0, fetch=5 +--Sort: annotated_data_finite.ts DESC NULLS FIRST, fetch=5 +----Projection: annotated_data_finite.ts, FIRST_VALUE(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING AS fv1, FIRST_VALUE(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING AS fv2, LAST_VALUE(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING AS lv1, LAST_VALUE(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING AS lv2, NTH_VALUE(annotated_data_finite.inc_col,Int64(5)) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING AS nv1, NTH_VALUE(annotated_data_finite.inc_col,Int64(5)) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING AS nv2, ROW_NUMBER() ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING AS rn1, ROW_NUMBER() ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING AS rn2, RANK() ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING AS rank1, RANK() ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING AS rank2, DENSE_RANK() ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING AS dense_rank1, DENSE_RANK() ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING AS dense_rank2, LAG(annotated_data_finite.inc_col,Int64(1),Int64(1001)) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING AS lag1, LAG(annotated_data_finite.inc_col,Int64(2),Int64(1002)) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING AS lag2, LEAD(annotated_data_finite.inc_col,Int64(-1),Int64(1001)) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING AS lead1, LEAD(annotated_data_finite.inc_col,Int64(4),Int64(1004)) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING AS lead2, FIRST_VALUE(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING AS fvr1, FIRST_VALUE(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING AS fvr2, LAST_VALUE(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING AS lvr1, LAST_VALUE(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING AS lvr2, LAG(annotated_data_finite.inc_col,Int64(1),Int64(1001)) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING AS lagr1, LAG(annotated_data_finite.inc_col,Int64(2),Int64(1002)) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING AS lagr2, LEAD(annotated_data_finite.inc_col,Int64(-1),Int64(1001)) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING AS leadr1, LEAD(annotated_data_finite.inc_col,Int64(4),Int64(1004)) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING AS leadr2 +------WindowAggr: windowExpr=[[FIRST_VALUE(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING, FIRST_VALUE(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING, LAST_VALUE(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING, LAST_VALUE(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING, NTH_VALUE(annotated_data_finite.inc_col, Int64(5)) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING, NTH_VALUE(annotated_data_finite.inc_col, Int64(5)) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING, ROW_NUMBER() ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING, ROW_NUMBER() ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING, RANK() ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING, RANK() ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING, DENSE_RANK() ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING, DENSE_RANK() ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING, LAG(annotated_data_finite.inc_col, Int64(1), Int64(1001)) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING, LAG(annotated_data_finite.inc_col, Int64(2), Int64(1002)) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING, LEAD(annotated_data_finite.inc_col, Int64(-1), Int64(1001)) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING, LEAD(annotated_data_finite.inc_col, Int64(4), Int64(1004)) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING]] +--------WindowAggr: windowExpr=[[FIRST_VALUE(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING, FIRST_VALUE(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING, LAST_VALUE(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING, LAST_VALUE(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING, LAG(annotated_data_finite.inc_col, Int64(1), Int64(1001)) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING, LAG(annotated_data_finite.inc_col, Int64(2), Int64(1002)) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING, LEAD(annotated_data_finite.inc_col, Int64(-1), Int64(1001)) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING, LEAD(annotated_data_finite.inc_col, Int64(4), Int64(1004)) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING]] +----------TableScan: annotated_data_finite projection=[ts, inc_col] physical_plan -ProjectionExec: expr=[fv1@0 as fv1, fv2@1 as fv2, lv1@2 as lv1, lv2@3 as lv2, nv1@4 as nv1, nv2@5 as nv2, rn1@6 as rn1, rn2@7 as rn2, rank1@8 as rank1, rank2@9 as rank2, dense_rank1@10 as dense_rank1, dense_rank2@11 as dense_rank2, lag1@12 as lag1, lag2@13 as lag2, lead1@14 as lead1, lead2@15 as lead2, fvr1@16 as fvr1, fvr2@17 as fvr2, lvr1@18 as lvr1, lvr2@19 as lvr2, lagr1@20 as lagr1, lagr2@21 as lagr2, leadr1@22 as leadr1, leadr2@23 as leadr2] ---GlobalLimitExec: skip=0, fetch=5 -----SortExec: fetch=5, expr=[ts@24 DESC] -------ProjectionExec: expr=[FIRST_VALUE(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING@10 as fv1, FIRST_VALUE(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING@11 as fv2, LAST_VALUE(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING@12 as lv1, LAST_VALUE(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING@13 as lv2, NTH_VALUE(annotated_data_finite.inc_col,Int64(5)) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING@14 as nv1, NTH_VALUE(annotated_data_finite.inc_col,Int64(5)) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING@15 as nv2, ROW_NUMBER() ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING@16 as rn1, ROW_NUMBER() ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING@17 as rn2, RANK() ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING@18 as rank1, RANK() ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING@19 as rank2, DENSE_RANK() ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING@20 as dense_rank1, DENSE_RANK() ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING@21 as dense_rank2, LAG(annotated_data_finite.inc_col,Int64(1),Int64(1001)) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING@22 as lag1, LAG(annotated_data_finite.inc_col,Int64(2),Int64(1002)) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING@23 as lag2, LEAD(annotated_data_finite.inc_col,Int64(-1),Int64(1001)) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING@24 as lead1, LEAD(annotated_data_finite.inc_col,Int64(4),Int64(1004)) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING@25 as lead2, FIRST_VALUE(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING@2 as fvr1, FIRST_VALUE(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING@3 as fvr2, LAST_VALUE(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING@4 as lvr1, LAST_VALUE(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING@5 as lvr2, LAG(annotated_data_finite.inc_col,Int64(1),Int64(1001)) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING@6 as lagr1, LAG(annotated_data_finite.inc_col,Int64(2),Int64(1002)) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING@7 as lagr2, LEAD(annotated_data_finite.inc_col,Int64(-1),Int64(1001)) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING@8 as leadr1, LEAD(annotated_data_finite.inc_col,Int64(4),Int64(1004)) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING@9 as leadr2, ts@0 as ts] ---------BoundedWindowAggExec: wdw=[FIRST_VALUE(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "FIRST_VALUE(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(10)), end_bound: Following(Int32(1)) }, FIRST_VALUE(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "FIRST_VALUE(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(10)), end_bound: Following(UInt64(1)) }, LAST_VALUE(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "LAST_VALUE(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(10)), end_bound: Following(Int32(1)) }, LAST_VALUE(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "LAST_VALUE(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(10)), end_bound: Following(UInt64(1)) }, NTH_VALUE(annotated_data_finite.inc_col,Int64(5)) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "NTH_VALUE(annotated_data_finite.inc_col,Int64(5)) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(10)), end_bound: Following(Int32(1)) }, NTH_VALUE(annotated_data_finite.inc_col,Int64(5)) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "NTH_VALUE(annotated_data_finite.inc_col,Int64(5)) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(10)), end_bound: Following(UInt64(1)) }, ROW_NUMBER() ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING: Ok(Field { name: "ROW_NUMBER() ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING", data_type: UInt64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(1)), end_bound: Following(Int32(10)) }, ROW_NUMBER() ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "ROW_NUMBER() ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING", data_type: UInt64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(10)), end_bound: Following(UInt64(1)) }, RANK() ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING: Ok(Field { name: "RANK() ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING", data_type: UInt64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(1)), end_bound: Following(Int32(10)) }, RANK() ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "RANK() ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING", data_type: UInt64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(10)), end_bound: Following(UInt64(1)) }, DENSE_RANK() ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING: Ok(Field { name: "DENSE_RANK() ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING", data_type: UInt64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(1)), end_bound: Following(Int32(10)) }, DENSE_RANK() ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "DENSE_RANK() ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING", data_type: UInt64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(10)), end_bound: Following(UInt64(1)) }, LAG(annotated_data_finite.inc_col,Int64(1),Int64(1001)) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING: Ok(Field { name: "LAG(annotated_data_finite.inc_col,Int64(1),Int64(1001)) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(1)), end_bound: Following(Int32(10)) }, LAG(annotated_data_finite.inc_col,Int64(2),Int64(1002)) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "LAG(annotated_data_finite.inc_col,Int64(2),Int64(1002)) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(10)), end_bound: Following(UInt64(1)) }, LEAD(annotated_data_finite.inc_col,Int64(-1),Int64(1001)) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING: Ok(Field { name: "LEAD(annotated_data_finite.inc_col,Int64(-1),Int64(1001)) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(1)), end_bound: Following(Int32(10)) }, LEAD(annotated_data_finite.inc_col,Int64(4),Int64(1004)) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "LEAD(annotated_data_finite.inc_col,Int64(4),Int64(1004)) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(10)), end_bound: Following(UInt64(1)) }], mode=[Sorted] -----------BoundedWindowAggExec: wdw=[FIRST_VALUE(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "FIRST_VALUE(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(1)), end_bound: Following(Int32(10)) }, FIRST_VALUE(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "FIRST_VALUE(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(1)), end_bound: Following(UInt64(10)) }, LAST_VALUE(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "LAST_VALUE(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(1)), end_bound: Following(Int32(10)) }, LAST_VALUE(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "LAST_VALUE(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(1)), end_bound: Following(UInt64(10)) }, LAG(annotated_data_finite.inc_col,Int64(1),Int64(1001)) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING: Ok(Field { name: "LAG(annotated_data_finite.inc_col,Int64(1),Int64(1001)) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(10)), end_bound: Following(Int32(1)) }, LAG(annotated_data_finite.inc_col,Int64(2),Int64(1002)) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "LAG(annotated_data_finite.inc_col,Int64(2),Int64(1002)) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(1)), end_bound: Following(UInt64(10)) }, LEAD(annotated_data_finite.inc_col,Int64(-1),Int64(1001)) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING: Ok(Field { name: "LEAD(annotated_data_finite.inc_col,Int64(-1),Int64(1001)) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(10)), end_bound: Following(Int32(1)) }, LEAD(annotated_data_finite.inc_col,Int64(4),Int64(1004)) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "LEAD(annotated_data_finite.inc_col,Int64(4),Int64(1004)) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(1)), end_bound: Following(UInt64(10)) }], mode=[Sorted] -------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_1.csv]]}, projection=[ts, inc_col], output_ordering=[ts@0 ASC NULLS LAST], has_header=true +GlobalLimitExec: skip=0, fetch=5 +--SortExec: fetch=5, expr=[ts@0 DESC] +----ProjectionExec: expr=[ts@0 as ts, FIRST_VALUE(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING@10 as fv1, FIRST_VALUE(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING@11 as fv2, LAST_VALUE(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING@12 as lv1, LAST_VALUE(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING@13 as lv2, NTH_VALUE(annotated_data_finite.inc_col,Int64(5)) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING@14 as nv1, NTH_VALUE(annotated_data_finite.inc_col,Int64(5)) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING@15 as nv2, ROW_NUMBER() ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING@16 as rn1, ROW_NUMBER() ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING@17 as rn2, RANK() ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING@18 as rank1, RANK() ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING@19 as rank2, DENSE_RANK() ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING@20 as dense_rank1, DENSE_RANK() ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING@21 as dense_rank2, LAG(annotated_data_finite.inc_col,Int64(1),Int64(1001)) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING@22 as lag1, LAG(annotated_data_finite.inc_col,Int64(2),Int64(1002)) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING@23 as lag2, LEAD(annotated_data_finite.inc_col,Int64(-1),Int64(1001)) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING@24 as lead1, LEAD(annotated_data_finite.inc_col,Int64(4),Int64(1004)) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING@25 as lead2, FIRST_VALUE(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING@2 as fvr1, FIRST_VALUE(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING@3 as fvr2, LAST_VALUE(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING@4 as lvr1, LAST_VALUE(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING@5 as lvr2, LAG(annotated_data_finite.inc_col,Int64(1),Int64(1001)) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING@6 as lagr1, LAG(annotated_data_finite.inc_col,Int64(2),Int64(1002)) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING@7 as lagr2, LEAD(annotated_data_finite.inc_col,Int64(-1),Int64(1001)) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING@8 as leadr1, LEAD(annotated_data_finite.inc_col,Int64(4),Int64(1004)) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING@9 as leadr2] +------BoundedWindowAggExec: wdw=[FIRST_VALUE(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "FIRST_VALUE(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(10)), end_bound: Following(Int32(1)) }, FIRST_VALUE(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "FIRST_VALUE(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(10)), end_bound: Following(UInt64(1)) }, LAST_VALUE(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "LAST_VALUE(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(10)), end_bound: Following(Int32(1)) }, LAST_VALUE(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "LAST_VALUE(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(10)), end_bound: Following(UInt64(1)) }, NTH_VALUE(annotated_data_finite.inc_col,Int64(5)) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "NTH_VALUE(annotated_data_finite.inc_col,Int64(5)) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(10)), end_bound: Following(Int32(1)) }, NTH_VALUE(annotated_data_finite.inc_col,Int64(5)) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "NTH_VALUE(annotated_data_finite.inc_col,Int64(5)) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(10)), end_bound: Following(UInt64(1)) }, ROW_NUMBER() ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING: Ok(Field { name: "ROW_NUMBER() ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING", data_type: UInt64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(1)), end_bound: Following(Int32(10)) }, ROW_NUMBER() ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "ROW_NUMBER() ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING", data_type: UInt64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(10)), end_bound: Following(UInt64(1)) }, RANK() ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING: Ok(Field { name: "RANK() ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING", data_type: UInt64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(1)), end_bound: Following(Int32(10)) }, RANK() ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "RANK() ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING", data_type: UInt64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(10)), end_bound: Following(UInt64(1)) }, DENSE_RANK() ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING: Ok(Field { name: "DENSE_RANK() ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING", data_type: UInt64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(1)), end_bound: Following(Int32(10)) }, DENSE_RANK() ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "DENSE_RANK() ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING", data_type: UInt64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(10)), end_bound: Following(UInt64(1)) }, LAG(annotated_data_finite.inc_col,Int64(1),Int64(1001)) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING: Ok(Field { name: "LAG(annotated_data_finite.inc_col,Int64(1),Int64(1001)) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(1)), end_bound: Following(Int32(10)) }, LAG(annotated_data_finite.inc_col,Int64(2),Int64(1002)) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "LAG(annotated_data_finite.inc_col,Int64(2),Int64(1002)) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(10)), end_bound: Following(UInt64(1)) }, LEAD(annotated_data_finite.inc_col,Int64(-1),Int64(1001)) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING: Ok(Field { name: "LEAD(annotated_data_finite.inc_col,Int64(-1),Int64(1001)) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(1)), end_bound: Following(Int32(10)) }, LEAD(annotated_data_finite.inc_col,Int64(4),Int64(1004)) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "LEAD(annotated_data_finite.inc_col,Int64(4),Int64(1004)) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(10)), end_bound: Following(UInt64(1)) }], mode=[Sorted] +--------BoundedWindowAggExec: wdw=[FIRST_VALUE(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "FIRST_VALUE(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(1)), end_bound: Following(Int32(10)) }, FIRST_VALUE(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "FIRST_VALUE(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(1)), end_bound: Following(UInt64(10)) }, LAST_VALUE(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "LAST_VALUE(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(1)), end_bound: Following(Int32(10)) }, LAST_VALUE(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "LAST_VALUE(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(1)), end_bound: Following(UInt64(10)) }, LAG(annotated_data_finite.inc_col,Int64(1),Int64(1001)) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING: Ok(Field { name: "LAG(annotated_data_finite.inc_col,Int64(1),Int64(1001)) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(10)), end_bound: Following(Int32(1)) }, LAG(annotated_data_finite.inc_col,Int64(2),Int64(1002)) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "LAG(annotated_data_finite.inc_col,Int64(2),Int64(1002)) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(1)), end_bound: Following(UInt64(10)) }, LEAD(annotated_data_finite.inc_col,Int64(-1),Int64(1001)) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING: Ok(Field { name: "LEAD(annotated_data_finite.inc_col,Int64(-1),Int64(1001)) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(10)), end_bound: Following(Int32(1)) }, LEAD(annotated_data_finite.inc_col,Int64(4),Int64(1004)) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "LEAD(annotated_data_finite.inc_col,Int64(4),Int64(1004)) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(1)), end_bound: Following(UInt64(10)) }], mode=[Sorted] +----------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_1.csv]]}, projection=[ts, inc_col], output_ordering=[ts@0 ASC NULLS LAST], has_header=true -query IIIIIIIIIIIIIIIIIIIIIIII +query IIIIIIIIIIIIIIIIIIIIIIIII SELECT + ts, FIRST_VALUE(inc_col) OVER(ORDER BY ts RANGE BETWEEN 10 PRECEDING and 1 FOLLOWING) as fv1, FIRST_VALUE(inc_col) OVER(ORDER BY ts ROWS BETWEEN 10 PRECEDING and 1 FOLLOWING) as fv2, LAST_VALUE(inc_col) OVER(ORDER BY ts RANGE BETWEEN 10 PRECEDING and 1 FOLLOWING) as lv1, @@ -2673,14 +2673,14 @@ SELECT LEAD(inc_col, -1, 1001) OVER(ORDER BY ts DESC RANGE BETWEEN 1 PRECEDING and 10 FOLLOWING) AS leadr1, LEAD(inc_col, 4, 1004) OVER(ORDER BY ts DESC ROWS BETWEEN 10 PRECEDING and 1 FOLLOWING) as leadr2 FROM annotated_data_finite - ORDER BY ts DESC + ORDER BY ts DESC, fv2 LIMIT 5; ---- -289 269 305 305 305 283 100 100 99 99 86 86 301 296 301 1004 305 305 301 301 1001 1002 1001 289 -289 266 305 305 305 278 99 99 99 99 86 86 296 291 296 1004 305 305 301 296 305 1002 305 286 -289 261 296 301 NULL 275 98 98 98 98 85 85 291 289 291 1004 305 305 296 291 301 305 301 283 -286 259 291 296 NULL 272 97 97 97 97 84 84 289 286 289 1004 305 305 291 289 296 301 296 278 -275 254 289 291 289 269 96 96 96 96 83 83 286 283 286 305 305 305 289 286 291 296 291 275 +264 289 266 305 305 305 278 99 99 99 99 86 86 296 291 296 1004 305 305 301 296 305 1002 305 286 +264 289 269 305 305 305 283 100 100 99 99 86 86 301 296 301 1004 305 305 301 301 1001 1002 1001 289 +262 289 261 296 301 NULL 275 98 98 98 98 85 85 291 289 291 1004 305 305 296 291 301 305 301 283 +258 286 259 291 296 NULL 272 97 97 97 97 84 84 289 286 289 1004 305 305 291 289 296 301 296 278 +254 275 254 289 291 289 269 96 96 96 96 83 83 286 283 286 305 305 305 289 286 291 296 291 275 # test_source_sorted_unbounded_preceding @@ -3271,25 +3271,27 @@ drop table annotated_data_infinite2 # window3 spec is not used in window functions. # The query should still work. -query RR +query IRR SELECT - MAX(c12) OVER window1, - MIN(c12) OVER window2 as max1 + C3, + MAX(c12) OVER window1 as max1, + MIN(c12) OVER window2 as max2 FROM aggregate_test_100 WINDOW window1 AS (ORDER BY C12), window2 AS (PARTITION BY C11), window3 AS (ORDER BY C1) - ORDER BY C3 + ORDER BY C3, max2 LIMIT 5 ---- -0.970671228336 0.970671228336 -0.850672105305 0.850672105305 -0.152498292972 0.152498292972 -0.369363046006 0.369363046006 -0.56535284223 0.56535284223 +-117 0.850672105305 0.850672105305 +-117 0.970671228336 0.970671228336 +-111 0.152498292972 0.152498292972 +-107 0.369363046006 0.369363046006 +-106 0.56535284223 0.56535284223 query TT EXPLAIN SELECT + C3, MAX(c12) OVER window1 as min1, MIN(c12) OVER window2 as max1 FROM aggregate_test_100 @@ -3300,42 +3302,41 @@ EXPLAIN SELECT LIMIT 5 ---- logical_plan -Projection: min1, max1 ---Limit: skip=0, fetch=5 -----Sort: aggregate_test_100.c3 ASC NULLS LAST, fetch=5 -------Projection: MAX(aggregate_test_100.c12) ORDER BY [aggregate_test_100.c12 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW AS min1, MIN(aggregate_test_100.c12) PARTITION BY [aggregate_test_100.c11] ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING AS max1, aggregate_test_100.c3 ---------WindowAggr: windowExpr=[[MAX(aggregate_test_100.c12) ORDER BY [aggregate_test_100.c12 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW]] -----------Projection: aggregate_test_100.c3, aggregate_test_100.c12, MIN(aggregate_test_100.c12) PARTITION BY [aggregate_test_100.c11] ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING -------------WindowAggr: windowExpr=[[MIN(aggregate_test_100.c12) PARTITION BY [aggregate_test_100.c11] ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING]] ---------------TableScan: aggregate_test_100 projection=[c3, c11, c12] +Limit: skip=0, fetch=5 +--Sort: aggregate_test_100.c3 ASC NULLS LAST, fetch=5 +----Projection: aggregate_test_100.c3, MAX(aggregate_test_100.c12) ORDER BY [aggregate_test_100.c12 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW AS min1, MIN(aggregate_test_100.c12) PARTITION BY [aggregate_test_100.c11] ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING AS max1 +------WindowAggr: windowExpr=[[MAX(aggregate_test_100.c12) ORDER BY [aggregate_test_100.c12 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW]] +--------Projection: aggregate_test_100.c3, aggregate_test_100.c12, MIN(aggregate_test_100.c12) PARTITION BY [aggregate_test_100.c11] ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING +----------WindowAggr: windowExpr=[[MIN(aggregate_test_100.c12) PARTITION BY [aggregate_test_100.c11] ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING]] +------------TableScan: aggregate_test_100 projection=[c3, c11, c12] physical_plan -ProjectionExec: expr=[min1@0 as min1, max1@1 as max1] ---GlobalLimitExec: skip=0, fetch=5 -----SortExec: fetch=5, expr=[c3@2 ASC NULLS LAST] -------ProjectionExec: expr=[MAX(aggregate_test_100.c12) ORDER BY [aggregate_test_100.c12 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@3 as min1, MIN(aggregate_test_100.c12) PARTITION BY [aggregate_test_100.c11] ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING@2 as max1, c3@0 as c3] ---------BoundedWindowAggExec: wdw=[MAX(aggregate_test_100.c12) ORDER BY [aggregate_test_100.c12 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "MAX(aggregate_test_100.c12) ORDER BY [aggregate_test_100.c12 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW", data_type: Float64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Float64(NULL)), end_bound: CurrentRow }], mode=[Sorted] -----------SortExec: expr=[c12@1 ASC NULLS LAST] -------------ProjectionExec: expr=[c3@0 as c3, c12@2 as c12, MIN(aggregate_test_100.c12) PARTITION BY [aggregate_test_100.c11] ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING@3 as MIN(aggregate_test_100.c12) PARTITION BY [aggregate_test_100.c11] ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING] ---------------WindowAggExec: wdw=[MIN(aggregate_test_100.c12) PARTITION BY [aggregate_test_100.c11] ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING: Ok(Field { name: "MIN(aggregate_test_100.c12) PARTITION BY [aggregate_test_100.c11] ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING", data_type: Float64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(NULL)), end_bound: Following(UInt64(NULL)) }] -----------------SortExec: expr=[c11@1 ASC NULLS LAST] -------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c3, c11, c12], has_header=true +GlobalLimitExec: skip=0, fetch=5 +--SortExec: fetch=5, expr=[c3@0 ASC NULLS LAST] +----ProjectionExec: expr=[c3@0 as c3, MAX(aggregate_test_100.c12) ORDER BY [aggregate_test_100.c12 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@3 as min1, MIN(aggregate_test_100.c12) PARTITION BY [aggregate_test_100.c11] ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING@2 as max1] +------BoundedWindowAggExec: wdw=[MAX(aggregate_test_100.c12) ORDER BY [aggregate_test_100.c12 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "MAX(aggregate_test_100.c12) ORDER BY [aggregate_test_100.c12 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW", data_type: Float64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Float64(NULL)), end_bound: CurrentRow }], mode=[Sorted] +--------SortExec: expr=[c12@1 ASC NULLS LAST] +----------ProjectionExec: expr=[c3@0 as c3, c12@2 as c12, MIN(aggregate_test_100.c12) PARTITION BY [aggregate_test_100.c11] ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING@3 as MIN(aggregate_test_100.c12) PARTITION BY [aggregate_test_100.c11] ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING] +------------WindowAggExec: wdw=[MIN(aggregate_test_100.c12) PARTITION BY [aggregate_test_100.c11] ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING: Ok(Field { name: "MIN(aggregate_test_100.c12) PARTITION BY [aggregate_test_100.c11] ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING", data_type: Float64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(NULL)), end_bound: Following(UInt64(NULL)) }] +--------------SortExec: expr=[c11@1 ASC NULLS LAST] +----------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c3, c11, c12], has_header=true # window1 spec is used multiple times under different aggregations. # The query should still work. -query RR +query IRR SELECT + C3, MAX(c12) OVER window1 as min1, MIN(c12) OVER window1 as max1 FROM aggregate_test_100 WINDOW window1 AS (ORDER BY C12) - ORDER BY C3 + ORDER BY C3, min1 LIMIT 5 ---- -0.970671228336 0.014793053078 -0.850672105305 0.014793053078 -0.152498292972 0.014793053078 -0.369363046006 0.014793053078 -0.56535284223 0.014793053078 +-117 0.850672105305 0.014793053078 +-117 0.970671228336 0.014793053078 +-111 0.152498292972 0.014793053078 +-107 0.369363046006 0.014793053078 +-106 0.56535284223 0.014793053078 query TT EXPLAIN SELECT From e95a24b5a260e0e2f603d52682d36cce192676f8 Mon Sep 17 00:00:00 2001 From: Andrew Lamb Date: Thu, 5 Oct 2023 14:07:08 -0400 Subject: [PATCH 030/572] Minor: Improve crate docs (#7740) * Minor: Improve crate docs * fix links --- datafusion/core/src/lib.rs | 22 ++++++++++++---------- 1 file changed, 12 insertions(+), 10 deletions(-) diff --git a/datafusion/core/src/lib.rs b/datafusion/core/src/lib.rs index 576f66a5ed7c..5e9f130eade5 100644 --- a/datafusion/core/src/lib.rs +++ b/datafusion/core/src/lib.rs @@ -132,18 +132,19 @@ //! //! ## Customization and Extension //! -//! DataFusion is designed to be a "disaggregated" query engine. This -//! means that developers can mix and extend the parts of DataFusion -//! they need for their usecase. For example, just the -//! [`ExecutionPlan`] operators, or the [`SqlToRel`] SQL planner and -//! optimizer. +//! DataFusion is a "disaggregated" query engine. This +//! means developers can start with a working, full featured engine, and then +//! extend the parts of DataFusion they need to specialize for their usecase. For example, +//! some projects may add custom [`ExecutionPlan`] operators, or create their own +//! query language that directly creates [`LogicalPlan`] rather than using the +//! built in SQL planner, [`SqlToRel`]. //! //! In order to achieve this, DataFusion supports extension at many points: //! //! * read from any datasource ([`TableProvider`]) //! * define your own catalogs, schemas, and table lists ([`CatalogProvider`]) -//! * build your own query langue or plans using the ([`LogicalPlanBuilder`]) -//! * declare and use user-defined functions: ([`ScalarUDF`], and [`AggregateUDF`]) +//! * build your own query language or plans ([`LogicalPlanBuilder`]) +//! * declare and use user-defined functions ([`ScalarUDF`], and [`AggregateUDF`], [`WindowUDF`]) //! * add custom optimizer rewrite passes ([`OptimizerRule`] and [`PhysicalOptimizerRule`]) //! * extend the planner to use user-defined logical and physical nodes ([`QueryPlanner`]) //! @@ -152,8 +153,9 @@ //! [`TableProvider`]: crate::datasource::TableProvider //! [`CatalogProvider`]: crate::catalog::CatalogProvider //! [`LogicalPlanBuilder`]: datafusion_expr::logical_plan::builder::LogicalPlanBuilder -//! [`ScalarUDF`]: physical_plan::udf::ScalarUDF -//! [`AggregateUDF`]: physical_plan::udaf::AggregateUDF +//! [`ScalarUDF`]: crate::logical_expr::ScalarUDF +//! [`AggregateUDF`]: crate::logical_expr::AggregateUDF +//! [`WindowUDF`]: crate::logical_expr::WindowUDF //! [`QueryPlanner`]: execution::context::QueryPlanner //! [`OptimizerRule`]: datafusion_optimizer::optimizer::OptimizerRule //! [`PhysicalOptimizerRule`]: crate::physical_optimizer::optimizer::PhysicalOptimizerRule @@ -279,7 +281,7 @@ //! [`MemTable`]: crate::datasource::memory::MemTable //! [`StreamingTable`]: crate::datasource::streaming::StreamingTable //! -//! ## Plans +//! ## Plan Representations //! //! Logical planning yields [`LogicalPlan`]s nodes and [`Expr`] //! expressions which are [`Schema`] aware and represent statements From 85046001da91d535f7ea417911cd51944f9820f4 Mon Sep 17 00:00:00 2001 From: Mustafa Akur <106137913+mustafasrepo@users.noreply.github.com> Date: Fri, 6 Oct 2023 14:21:32 +0300 Subject: [PATCH 031/572] Resolve linter errors (#7753) --- datafusion/core/benches/topk_aggregate.rs | 7 +++++-- datafusion/core/src/catalog/mod.rs | 4 ++-- datafusion/core/src/datasource/default_table_source.rs | 2 +- datafusion/core/src/datasource/file_format/parquet.rs | 2 +- .../core/src/physical_optimizer/enforce_distribution.rs | 2 +- datafusion/core/src/physical_optimizer/enforce_sorting.rs | 6 +++--- datafusion/core/src/physical_optimizer/pipeline_checker.rs | 2 +- .../replace_with_order_preserving_variants.rs | 2 +- datafusion/core/src/physical_optimizer/test_utils.rs | 7 +++---- datafusion/core/tests/custom_sources.rs | 2 +- datafusion/execution/src/memory_pool/mod.rs | 6 +++--- datafusion/execution/src/stream.rs | 2 +- datafusion/expr/src/expr.rs | 2 +- datafusion/optimizer/src/push_down_filter.rs | 2 +- datafusion/physical-expr/src/aggregate/mod.rs | 2 +- datafusion/physical-expr/src/aggregate/utils.rs | 2 +- datafusion/physical-expr/src/expressions/binary.rs | 4 ++-- datafusion/physical-expr/src/physical_expr.rs | 2 +- datafusion/physical-expr/src/sort_properties.rs | 2 +- .../src/window/built_in_window_function_expr.rs | 2 +- datafusion/physical-expr/src/window/window_expr.rs | 2 +- datafusion/physical-plan/src/aggregates/mod.rs | 4 ++-- datafusion/physical-plan/src/lib.rs | 2 +- datafusion/physical-plan/src/memory.rs | 2 +- datafusion/physical-plan/src/projection.rs | 4 +--- datafusion/physical-plan/src/visitor.rs | 2 +- datafusion/sql/src/expr/mod.rs | 2 +- datafusion/sql/src/parser.rs | 3 +-- 28 files changed, 41 insertions(+), 42 deletions(-) diff --git a/datafusion/core/benches/topk_aggregate.rs b/datafusion/core/benches/topk_aggregate.rs index eaafea248894..ef84d6e3cac8 100644 --- a/datafusion/core/benches/topk_aggregate.rs +++ b/datafusion/core/benches/topk_aggregate.rs @@ -28,6 +28,7 @@ use datafusion_execution::config::SessionConfig; use datafusion_execution::TaskContext; use rand_distr::Distribution; use rand_distr::{Normal, Pareto}; +use std::fmt::Write; use std::sync::Arc; use tokio::runtime::Runtime; @@ -130,8 +131,10 @@ fn make_data( let gen_id = |rng: &mut rand::rngs::SmallRng| { rng.gen::<[u8; 16]>() .iter() - .map(|b| format!("{:02x}", b)) - .collect::() + .fold(String::new(), |mut output, b| { + let _ = write!(output, "{b:02X}"); + output + }) }; let gen_sample_cnt = |mut rng: &mut rand::rngs::SmallRng| pareto.sample(&mut rng).ceil() as u32; diff --git a/datafusion/core/src/catalog/mod.rs b/datafusion/core/src/catalog/mod.rs index 6751edbd3a84..fe5bdc0ec6a9 100644 --- a/datafusion/core/src/catalog/mod.rs +++ b/datafusion/core/src/catalog/mod.rs @@ -31,7 +31,7 @@ use std::sync::Arc; /// Represent a list of named catalogs pub trait CatalogList: Sync + Send { - /// Returns the catalog list as [`Any`](std::any::Any) + /// Returns the catalog list as [`Any`] /// so that it can be downcast to a specific implementation. fn as_any(&self) -> &dyn Any; @@ -101,7 +101,7 @@ impl Default for MemoryCatalogProvider { /// Represents a catalog, comprising a number of named schemas. pub trait CatalogProvider: Sync + Send { - /// Returns the catalog provider as [`Any`](std::any::Any) + /// Returns the catalog provider as [`Any`] /// so that it can be downcast to a specific implementation. fn as_any(&self) -> &dyn Any; diff --git a/datafusion/core/src/datasource/default_table_source.rs b/datafusion/core/src/datasource/default_table_source.rs index 58d0997bb653..f93faa50a9b9 100644 --- a/datafusion/core/src/datasource/default_table_source.rs +++ b/datafusion/core/src/datasource/default_table_source.rs @@ -43,7 +43,7 @@ impl DefaultTableSource { } impl TableSource for DefaultTableSource { - /// Returns the table source as [`Any`](std::any::Any) so that it can be + /// Returns the table source as [`Any`] so that it can be /// downcast to a specific implementation. fn as_any(&self) -> &dyn Any { self diff --git a/datafusion/core/src/datasource/file_format/parquet.rs b/datafusion/core/src/datasource/file_format/parquet.rs index 16050d66db5d..8ddddab71fd5 100644 --- a/datafusion/core/src/datasource/file_format/parquet.rs +++ b/datafusion/core/src/datasource/file_format/parquet.rs @@ -1101,7 +1101,7 @@ pub(crate) mod test_util { Ok((meta, files)) } - //// write batches chunk_size rows at a time + /// write batches chunk_size rows at a time fn write_in_chunks( writer: &mut ArrowWriter, batch: &RecordBatch, diff --git a/datafusion/core/src/physical_optimizer/enforce_distribution.rs b/datafusion/core/src/physical_optimizer/enforce_distribution.rs index 3463f3a31376..a1f509d28733 100644 --- a/datafusion/core/src/physical_optimizer/enforce_distribution.rs +++ b/datafusion/core/src/physical_optimizer/enforce_distribution.rs @@ -1529,7 +1529,7 @@ impl DistributionContext { self.plan .children() .into_iter() - .map(|child| DistributionContext::new(child)) + .map(DistributionContext::new) .collect() } } diff --git a/datafusion/core/src/physical_optimizer/enforce_sorting.rs b/datafusion/core/src/physical_optimizer/enforce_sorting.rs index 6e3b160c8922..95ec1973d017 100644 --- a/datafusion/core/src/physical_optimizer/enforce_sorting.rs +++ b/datafusion/core/src/physical_optimizer/enforce_sorting.rs @@ -151,7 +151,7 @@ impl PlanWithCorrespondingSort { self.plan .children() .into_iter() - .map(|child| PlanWithCorrespondingSort::new(child)) + .map(PlanWithCorrespondingSort::new) .collect() } } @@ -267,7 +267,7 @@ impl PlanWithCorrespondingCoalescePartitions { self.plan .children() .into_iter() - .map(|child| PlanWithCorrespondingCoalescePartitions::new(child)) + .map(PlanWithCorrespondingCoalescePartitions::new) .collect() } } @@ -602,7 +602,7 @@ fn analyze_window_sort_removal( let reqs = window_exec .required_input_ordering() .swap_remove(0) - .unwrap_or(vec![]); + .unwrap_or_default(); let sort_expr = PhysicalSortRequirement::to_sort_exprs(reqs); // Satisfy the ordering requirement so that the window can run: add_sort_above(&mut window_child, sort_expr, None)?; diff --git a/datafusion/core/src/physical_optimizer/pipeline_checker.rs b/datafusion/core/src/physical_optimizer/pipeline_checker.rs index 44679647b5b2..3b994d5f893f 100644 --- a/datafusion/core/src/physical_optimizer/pipeline_checker.rs +++ b/datafusion/core/src/physical_optimizer/pipeline_checker.rs @@ -108,7 +108,7 @@ impl TreeNode for PipelineStatePropagator { if !children.is_empty() { let new_children = children .into_iter() - .map(|child| PipelineStatePropagator::new(child)) + .map(PipelineStatePropagator::new) .map(transform) .collect::>>()?; let children_unbounded = new_children diff --git a/datafusion/core/src/physical_optimizer/replace_with_order_preserving_variants.rs b/datafusion/core/src/physical_optimizer/replace_with_order_preserving_variants.rs index cb3b6c3d0741..ede95fc67721 100644 --- a/datafusion/core/src/physical_optimizer/replace_with_order_preserving_variants.rs +++ b/datafusion/core/src/physical_optimizer/replace_with_order_preserving_variants.rs @@ -112,7 +112,7 @@ impl OrderPreservationContext { self.plan .children() .into_iter() - .map(|child| OrderPreservationContext::new(child)) + .map(OrderPreservationContext::new) .collect() } } diff --git a/datafusion/core/src/physical_optimizer/test_utils.rs b/datafusion/core/src/physical_optimizer/test_utils.rs index e021cda2c868..0915fdbf1cd7 100644 --- a/datafusion/core/src/physical_optimizer/test_utils.rs +++ b/datafusion/core/src/physical_optimizer/test_utils.rs @@ -140,14 +140,13 @@ impl QueryCase { async fn run_case(&self, ctx: SessionContext, error: Option<&String>) -> Result<()> { let dataframe = ctx.sql(self.sql.as_str()).await?; let plan = dataframe.create_physical_plan().await; - if error.is_some() { + if let Some(error) = error { let plan_error = plan.unwrap_err(); - let initial = error.unwrap().to_string(); assert!( - plan_error.to_string().contains(initial.as_str()), + plan_error.to_string().contains(error.as_str()), "plan_error: {:?} doesn't contain message: {:?}", plan_error, - initial.as_str() + error.as_str() ); } else { assert!(plan.is_ok()) diff --git a/datafusion/core/tests/custom_sources.rs b/datafusion/core/tests/custom_sources.rs index 771da80aa6e7..3886ad3c42c0 100644 --- a/datafusion/core/tests/custom_sources.rs +++ b/datafusion/core/tests/custom_sources.rs @@ -49,7 +49,7 @@ mod custom_sources_cases; use async_trait::async_trait; -//// Custom source dataframe tests //// +//--- Custom source dataframe tests ---// struct CustomTableProvider; #[derive(Debug, Clone)] diff --git a/datafusion/execution/src/memory_pool/mod.rs b/datafusion/execution/src/memory_pool/mod.rs index f8fc9fcdbbbb..bbd8f4be4f1c 100644 --- a/datafusion/execution/src/memory_pool/mod.rs +++ b/datafusion/execution/src/memory_pool/mod.rs @@ -46,9 +46,9 @@ pub use pool::*; /// /// The following memory pool implementations are available: /// -/// * [`UnboundedMemoryPool`](pool::UnboundedMemoryPool) -/// * [`GreedyMemoryPool`](pool::GreedyMemoryPool) -/// * [`FairSpillPool`](pool::FairSpillPool) +/// * [`UnboundedMemoryPool`] +/// * [`GreedyMemoryPool`] +/// * [`FairSpillPool`] pub trait MemoryPool: Send + Sync + std::fmt::Debug { /// Registers a new [`MemoryConsumer`] /// diff --git a/datafusion/execution/src/stream.rs b/datafusion/execution/src/stream.rs index 5a1a9aaa2590..7fc5e458b86b 100644 --- a/datafusion/execution/src/stream.rs +++ b/datafusion/execution/src/stream.rs @@ -29,5 +29,5 @@ pub trait RecordBatchStream: Stream> { fn schema(&self) -> SchemaRef; } -/// Trait for a [`Stream`](futures::stream::Stream) of [`RecordBatch`]es +/// Trait for a [`Stream`] of [`RecordBatch`]es pub type SendableRecordBatchStream = Pin>; diff --git a/datafusion/expr/src/expr.rs b/datafusion/expr/src/expr.rs index 94ef69eb7933..3e4e3068977c 100644 --- a/datafusion/expr/src/expr.rs +++ b/datafusion/expr/src/expr.rs @@ -39,7 +39,7 @@ use std::sync::Arc; /// represent logical expressions such as `A + 1`, or `CAST(c1 AS /// int)`. /// -/// An `Expr` can compute its [DataType](arrow::datatypes::DataType) +/// An `Expr` can compute its [DataType] /// and nullability, and has functions for building up complex /// expressions. /// diff --git a/datafusion/optimizer/src/push_down_filter.rs b/datafusion/optimizer/src/push_down_filter.rs index fe726d5d7783..ee7b37979dc4 100644 --- a/datafusion/optimizer/src/push_down_filter.rs +++ b/datafusion/optimizer/src/push_down_filter.rs @@ -488,7 +488,7 @@ fn push_down_join( .filter .as_ref() .map(|e| utils::split_conjunction_owned(e.clone())) - .unwrap_or_else(Vec::new); + .unwrap_or_default(); let mut is_inner_join = false; let infer_predicates = if join.join_type == JoinType::Inner { diff --git a/datafusion/physical-expr/src/aggregate/mod.rs b/datafusion/physical-expr/src/aggregate/mod.rs index 4920f7a3e07f..442d018b87d5 100644 --- a/datafusion/physical-expr/src/aggregate/mod.rs +++ b/datafusion/physical-expr/src/aggregate/mod.rs @@ -68,7 +68,7 @@ pub(crate) mod variance; /// `PartialEq` to allows comparing equality between the /// trait objects. pub trait AggregateExpr: Send + Sync + Debug + PartialEq { - /// Returns the aggregate expression as [`Any`](std::any::Any) so that it can be + /// Returns the aggregate expression as [`Any`] so that it can be /// downcast to a specific implementation. fn as_any(&self) -> &dyn Any; diff --git a/datafusion/physical-expr/src/aggregate/utils.rs b/datafusion/physical-expr/src/aggregate/utils.rs index e1af67071260..2f473f7608b0 100644 --- a/datafusion/physical-expr/src/aggregate/utils.rs +++ b/datafusion/physical-expr/src/aggregate/utils.rs @@ -161,7 +161,7 @@ pub fn adjust_output_array( } /// Downcast a `Box` or `Arc` -/// and return the inner trait object as [`Any`](std::any::Any) so +/// and return the inner trait object as [`Any`] so /// that it can be downcast to a specific implementation. /// /// This method is used when implementing the `PartialEq` diff --git a/datafusion/physical-expr/src/expressions/binary.rs b/datafusion/physical-expr/src/expressions/binary.rs index f75c2f951f56..63fa98011fdd 100644 --- a/datafusion/physical-expr/src/expressions/binary.rs +++ b/datafusion/physical-expr/src/expressions/binary.rs @@ -299,7 +299,7 @@ impl PhysicalExpr for BinaryExpr { }; if let Some(result) = scalar_result { - return result.map(|a| ColumnarValue::Array(a)); + return result.map(ColumnarValue::Array); } // if both arrays or both literals - extract arrays and continue execution @@ -308,7 +308,7 @@ impl PhysicalExpr for BinaryExpr { rhs.into_array(batch.num_rows()), ); self.evaluate_with_resolved_args(left, &left_data_type, right, &right_data_type) - .map(|a| ColumnarValue::Array(a)) + .map(ColumnarValue::Array) } fn children(&self) -> Vec> { diff --git a/datafusion/physical-expr/src/physical_expr.rs b/datafusion/physical-expr/src/physical_expr.rs index 81702d8bfae0..00fc9ed5fa98 100644 --- a/datafusion/physical-expr/src/physical_expr.rs +++ b/datafusion/physical-expr/src/physical_expr.rs @@ -35,7 +35,7 @@ use std::sync::Arc; /// Expression that can be evaluated against a RecordBatch /// A Physical expression knows its type, nullability and how to evaluate itself. pub trait PhysicalExpr: Send + Sync + Display + Debug + PartialEq { - /// Returns the physical expression as [`Any`](std::any::Any) so that it can be + /// Returns the physical expression as [`Any`] so that it can be /// downcast to a specific implementation. fn as_any(&self) -> &dyn Any; /// Get the data type of this expression, given the schema of the input diff --git a/datafusion/physical-expr/src/sort_properties.rs b/datafusion/physical-expr/src/sort_properties.rs index 001b86e60a86..097f491cb979 100644 --- a/datafusion/physical-expr/src/sort_properties.rs +++ b/datafusion/physical-expr/src/sort_properties.rs @@ -172,7 +172,7 @@ impl ExprOrdering { self.expr .children() .into_iter() - .map(|e| ExprOrdering::new(e)) + .map(ExprOrdering::new) .collect() } diff --git a/datafusion/physical-expr/src/window/built_in_window_function_expr.rs b/datafusion/physical-expr/src/window/built_in_window_function_expr.rs index 1a060817d2e1..66ffa990b78b 100644 --- a/datafusion/physical-expr/src/window/built_in_window_function_expr.rs +++ b/datafusion/physical-expr/src/window/built_in_window_function_expr.rs @@ -37,7 +37,7 @@ use std::sync::Arc; /// `nth_value` need the value. #[allow(rustdoc::private_intra_doc_links)] pub trait BuiltInWindowFunctionExpr: Send + Sync + std::fmt::Debug { - /// Returns the aggregate expression as [`Any`](std::any::Any) so that it can be + /// Returns the aggregate expression as [`Any`] so that it can be /// downcast to a specific implementation. fn as_any(&self) -> &dyn Any; diff --git a/datafusion/physical-expr/src/window/window_expr.rs b/datafusion/physical-expr/src/window/window_expr.rs index da67bcabee0b..9b0a02d329c4 100644 --- a/datafusion/physical-expr/src/window/window_expr.rs +++ b/datafusion/physical-expr/src/window/window_expr.rs @@ -59,7 +59,7 @@ use std::sync::Arc; /// [`PlainAggregateWindowExpr`]: crate::window::PlainAggregateWindowExpr /// [`SlidingAggregateWindowExpr`]: crate::window::SlidingAggregateWindowExpr pub trait WindowExpr: Send + Sync + Debug { - /// Returns the window expression as [`Any`](std::any::Any) so that it can be + /// Returns the window expression as [`Any`] so that it can be /// downcast to a specific implementation. fn as_any(&self) -> &dyn Any; diff --git a/datafusion/physical-plan/src/aggregates/mod.rs b/datafusion/physical-plan/src/aggregates/mod.rs index b6e4b0a44dec..6ab991c6f596 100644 --- a/datafusion/physical-plan/src/aggregates/mod.rs +++ b/datafusion/physical-plan/src/aggregates/mod.rs @@ -681,7 +681,7 @@ impl AggregateExec { for (expression, name) in group_by.expr.iter() { if let Some(column) = expression.as_any().downcast_ref::() { let new_col_idx = schema.index_of(name)?; - let entry = columns_map.entry(column.clone()).or_insert_with(Vec::new); + let entry = columns_map.entry(column.clone()).or_default(); entry.push(Column::new(name, new_col_idx)); }; } @@ -1868,7 +1868,7 @@ mod tests { } } - //// Tests //// + //--- Tests ---// #[tokio::test] async fn aggregate_source_not_yielding() -> Result<()> { diff --git a/datafusion/physical-plan/src/lib.rs b/datafusion/physical-plan/src/lib.rs index 3071fadcb1a2..30ac2cb98366 100644 --- a/datafusion/physical-plan/src/lib.rs +++ b/datafusion/physical-plan/src/lib.rs @@ -62,7 +62,7 @@ pub use stream::EmptyRecordBatchStream; /// return value from [`displayable`] in addition to the (normally /// quite verbose) `Debug` output. pub trait ExecutionPlan: Debug + DisplayAs + Send + Sync { - /// Returns the execution plan as [`Any`](std::any::Any) so that it can be + /// Returns the execution plan as [`Any`] so that it can be /// downcast to a specific implementation. fn as_any(&self) -> &dyn Any; diff --git a/datafusion/physical-plan/src/memory.rs b/datafusion/physical-plan/src/memory.rs index 1dcdae56cfa3..4c83ff1528fa 100644 --- a/datafusion/physical-plan/src/memory.rs +++ b/datafusion/physical-plan/src/memory.rs @@ -81,7 +81,7 @@ impl DisplayAs for MemoryExec { output_ordering.iter().map(|e| e.to_string()).collect(); format!(", output_ordering={}", order_strings.join(",")) }) - .unwrap_or_else(|| "".to_string()); + .unwrap_or_default(); write!( f, diff --git a/datafusion/physical-plan/src/projection.rs b/datafusion/physical-plan/src/projection.rs index 4fc48e971ca9..f4a2cd339ee5 100644 --- a/datafusion/physical-plan/src/projection.rs +++ b/datafusion/physical-plan/src/projection.rs @@ -111,9 +111,7 @@ impl ProjectionExec { let idx = column.index(); let matching_input_field = input_schema.field(idx); let matching_input_column = Column::new(matching_input_field.name(), idx); - let entry = columns_map - .entry(matching_input_column) - .or_insert_with(Vec::new); + let entry = columns_map.entry(matching_input_column).or_default(); entry.push(Column::new(name, expr_idx)); }; } diff --git a/datafusion/physical-plan/src/visitor.rs b/datafusion/physical-plan/src/visitor.rs index 573e4f8b02be..ca826c50022d 100644 --- a/datafusion/physical-plan/src/visitor.rs +++ b/datafusion/physical-plan/src/visitor.rs @@ -38,7 +38,7 @@ pub fn accept( /// depth first walk of `ExecutionPlan` nodes. `pre_visit` is called /// before any children are visited, and then `post_visit` is called /// after all children have been visited. -//// +/// /// To use, define a struct that implements this trait and then invoke /// ['accept']. /// diff --git a/datafusion/sql/src/expr/mod.rs b/datafusion/sql/src/expr/mod.rs index 2f6266f29d6a..cb34b6ca36e8 100644 --- a/datafusion/sql/src/expr/mod.rs +++ b/datafusion/sql/src/expr/mod.rs @@ -719,7 +719,7 @@ fn rewrite_placeholder(expr: &mut Expr, other: &Expr, schema: &DFSchema) -> Resu let other_dt = other.get_type(schema); match other_dt { Err(e) => { - return Err(e.context(format!( + Err(e.context(format!( "Can not find type of {other} needed to infer type of {expr}" )))?; } diff --git a/datafusion/sql/src/parser.rs b/datafusion/sql/src/parser.rs index 0e3e47508904..f52fdec7f704 100644 --- a/datafusion/sql/src/parser.rs +++ b/datafusion/sql/src/parser.rs @@ -253,8 +253,7 @@ impl fmt::Display for Statement { /// Datafusion SQL Parser based on [`sqlparser`] /// -/// Parses DataFusion's SQL dialect, often delegating to [`sqlparser`]'s -/// [`Parser`](sqlparser::parser::Parser). +/// Parses DataFusion's SQL dialect, often delegating to [`sqlparser`]'s [`Parser`]. /// /// DataFusion mostly follows existing SQL dialects via /// `sqlparser`. However, certain statements such as `COPY` and From 2e7fd6266ed115b930b6ddc5134c82b37c635366 Mon Sep 17 00:00:00 2001 From: Jay Zhan Date: Sat, 7 Oct 2023 02:41:54 +0800 Subject: [PATCH 032/572] rewrite concat_internal (#7748) Signed-off-by: jayzhan211 --- .../physical-expr/src/array_expressions.rs | 82 ++++++++++--------- 1 file changed, 45 insertions(+), 37 deletions(-) diff --git a/datafusion/physical-expr/src/array_expressions.rs b/datafusion/physical-expr/src/array_expressions.rs index 34fbfc3c0269..790c0481e188 100644 --- a/datafusion/physical-expr/src/array_expressions.rs +++ b/datafusion/physical-expr/src/array_expressions.rs @@ -810,6 +810,7 @@ fn align_array_dimensions(args: Vec) -> Result> { aligned_args } +// Concatenate arrays on the same row. fn concat_internal(args: &[ArrayRef]) -> Result { let args = align_array_dimensions(args.to_vec())?; @@ -818,49 +819,56 @@ fn concat_internal(args: &[ArrayRef]) -> Result { // Assume number of rows is the same for all arrays let row_count = list_arrays[0].len(); - let capacity = Capacities::Array(list_arrays.iter().map(|a| a.len()).sum()); - let array_data: Vec<_> = list_arrays.iter().map(|a| a.to_data()).collect::>(); - let array_data: Vec<&ArrayData> = array_data.iter().collect(); - let mut mutable = MutableArrayData::with_capacities(array_data, true, capacity); - - let mut array_lens = vec![0; row_count]; - let mut null_bit_map: Vec = vec![true; row_count]; + let mut array_lengths = vec![]; + let mut arrays = vec![]; + let mut valid = BooleanBufferBuilder::new(row_count); + for i in 0..row_count { + let nulls = list_arrays + .iter() + .map(|arr| arr.is_null(i)) + .collect::>(); + + // If all the arrays are null, the concatenated array is null + let is_null = nulls.iter().all(|&x| x); + if is_null { + array_lengths.push(0); + valid.append(false); + } else { + // Get all the arrays on i-th row + let values = list_arrays + .iter() + .map(|arr| arr.value(i)) + .collect::>(); - for (i, array_len) in array_lens.iter_mut().enumerate().take(row_count) { - let null_count = mutable.null_count(); - for (j, a) in list_arrays.iter().enumerate() { - mutable.extend(j, i, i + 1); - *array_len += a.value_length(i); - } + let elements = values + .iter() + .map(|a| a.as_ref()) + .collect::>(); - // This means all arrays are null - if mutable.null_count() == null_count + list_arrays.len() { - null_bit_map[i] = false; + // Concatenated array on i-th row + let concated_array = arrow::compute::concat(elements.as_slice())?; + array_lengths.push(concated_array.len()); + arrays.push(concated_array); + valid.append(true); } } + // Assume all arrays have the same data type + let data_type = list_arrays[0].value_type().clone(); + let buffer = valid.finish(); - let mut buffer = BooleanBufferBuilder::new(row_count); - buffer.append_slice(null_bit_map.as_slice()); - let nulls = Some(NullBuffer::from(buffer.finish())); - - let offsets: Vec = std::iter::once(0) - .chain(array_lens.iter().scan(0, |state, &x| { - *state += x; - Some(*state) - })) - .collect(); - - let builder = mutable.into_builder(); - - let list = builder - .len(row_count) - .buffers(vec![Buffer::from_vec(offsets)]) - .nulls(nulls) - .build()?; - - let list = arrow::array::make_array(list); - Ok(Arc::new(list)) + let elements = arrays + .iter() + .map(|a| a.as_ref()) + .collect::>(); + + let list_arr = ListArray::new( + Arc::new(Field::new("item", data_type, true)), + OffsetBuffer::from_lengths(array_lengths), + Arc::new(arrow::compute::concat(elements.as_slice())?), + Some(NullBuffer::new(buffer)), + ); + Ok(Arc::new(list_arr)) } /// Array_concat/Array_cat SQL function From a8c01de49259df7ae36d643845b234c12932e473 Mon Sep 17 00:00:00 2001 From: Liang-Chi Hsieh Date: Fri, 6 Oct 2023 13:11:53 -0700 Subject: [PATCH 033/572] Minor: Add comment on input_schema from AggregateExec (#7727) * Remove input_schema * Revert "Remove input_schema" This reverts commit 5096883a82ab94d451974d04660e33deb1b560ea. * Add comment --- datafusion/physical-plan/src/aggregates/mod.rs | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/datafusion/physical-plan/src/aggregates/mod.rs b/datafusion/physical-plan/src/aggregates/mod.rs index 6ab991c6f596..546abcfc4a4d 100644 --- a/datafusion/physical-plan/src/aggregates/mod.rs +++ b/datafusion/physical-plan/src/aggregates/mod.rs @@ -285,7 +285,9 @@ pub struct AggregateExec { schema: SchemaRef, /// Input schema before any aggregation is applied. For partial aggregate this will be the /// same as input.schema() but for the final aggregate it will be the same as the input - /// to the partial aggregate + /// to the partial aggregate, i.e., partial and final aggregates have same `input_schema`. + /// We need the input schema of partial aggregate to be able to deserialize aggregate + /// expressions from protobuf for final aggregate. pub input_schema: SchemaRef, /// The columns map used to normalize out expressions like Partitioning and PhysicalSortExpr /// The key is the column from the input schema and the values are the columns from the output schema From c0409a7b0b45e98f43013dff71550b9fe864fa26 Mon Sep 17 00:00:00 2001 From: Kirill Zaborsky Date: Fri, 6 Oct 2023 23:14:23 +0300 Subject: [PATCH 034/572] Fix column name for COUNT(*) set by AggregateStatistics (#7757) This fixes the issue of column name COUNT(UInt8) appearing instead Resolves #104 --- datafusion/core/src/physical_optimizer/aggregate_statistics.rs | 2 +- datafusion/core/tests/custom_sources.rs | 2 +- datafusion/core/tests/sql/explain_analyze.rs | 2 +- datafusion/sqllogictest/test_files/explain.slt | 2 +- 4 files changed, 4 insertions(+), 4 deletions(-) diff --git a/datafusion/core/src/physical_optimizer/aggregate_statistics.rs b/datafusion/core/src/physical_optimizer/aggregate_statistics.rs index 396e66972f30..52fd60aa94bd 100644 --- a/datafusion/core/src/physical_optimizer/aggregate_statistics.rs +++ b/datafusion/core/src/physical_optimizer/aggregate_statistics.rs @@ -38,7 +38,7 @@ use crate::error::Result; pub struct AggregateStatistics {} /// The name of the column corresponding to [`COUNT_STAR_EXPANSION`] -const COUNT_STAR_NAME: &str = "COUNT(UInt8(1))"; +const COUNT_STAR_NAME: &str = "COUNT(*)"; impl AggregateStatistics { #[allow(missing_docs)] diff --git a/datafusion/core/tests/custom_sources.rs b/datafusion/core/tests/custom_sources.rs index 3886ad3c42c0..377c2ab2ab24 100644 --- a/datafusion/core/tests/custom_sources.rs +++ b/datafusion/core/tests/custom_sources.rs @@ -268,7 +268,7 @@ async fn optimizers_catch_all_statistics() { let expected = RecordBatch::try_new( Arc::new(Schema::new(vec![ - Field::new("COUNT(UInt8(1))", DataType::Int64, false), + Field::new("COUNT(*)", DataType::Int64, false), Field::new("MIN(test.c1)", DataType::Int32, false), Field::new("MAX(test.c1)", DataType::Int32, false), ])), diff --git a/datafusion/core/tests/sql/explain_analyze.rs b/datafusion/core/tests/sql/explain_analyze.rs index 2b708df4f348..46f94c348735 100644 --- a/datafusion/core/tests/sql/explain_analyze.rs +++ b/datafusion/core/tests/sql/explain_analyze.rs @@ -804,7 +804,7 @@ async fn explain_physical_plan_only() { let expected = vec![vec![ "physical_plan", - "ProjectionExec: expr=[2 as COUNT(UInt8(1))]\ + "ProjectionExec: expr=[2 as COUNT(*)]\ \n EmptyExec: produce_one_row=true\ \n", ]]; diff --git a/datafusion/sqllogictest/test_files/explain.slt b/datafusion/sqllogictest/test_files/explain.slt index 27ab8671e939..23055cd978ff 100644 --- a/datafusion/sqllogictest/test_files/explain.slt +++ b/datafusion/sqllogictest/test_files/explain.slt @@ -93,7 +93,7 @@ query TT EXPLAIN select count(*) from (values ('a', 1, 100), ('a', 2, 150)) as t (c1,c2,c3) ---- physical_plan -ProjectionExec: expr=[2 as COUNT(UInt8(1))] +ProjectionExec: expr=[2 as COUNT(*)] --EmptyExec: produce_one_row=true statement ok From 0809f45c7e7c28138704490fbf168598f99fe742 Mon Sep 17 00:00:00 2001 From: Andrew Lamb Date: Sat, 7 Oct 2023 06:22:47 -0400 Subject: [PATCH 035/572] Add documentation about type signatures, and export `TIMEZONE_WILDCARD` (#7726) * Add documentation and export `TIMEZONE_WILDCARD` * improve example * Apply suggestions from code review Co-authored-by: Yongting You <2010youy01@gmail.com> --------- Co-authored-by: Yongting You <2010youy01@gmail.com> --- datafusion/expr/src/built_in_function.rs | 40 +++---- datafusion/expr/src/lib.rs | 2 +- datafusion/expr/src/signature.rs | 102 +++++++++++++----- .../expr/src/type_coercion/functions.rs | 13 +-- 4 files changed, 95 insertions(+), 62 deletions(-) diff --git a/datafusion/expr/src/built_in_function.rs b/datafusion/expr/src/built_in_function.rs index 70514f52d5f4..3ce573eaf15c 100644 --- a/datafusion/expr/src/built_in_function.rs +++ b/datafusion/expr/src/built_in_function.rs @@ -18,7 +18,8 @@ //! Built-in functions module contains all the built-in functions definitions. use crate::nullif::SUPPORTED_NULLIF_TYPES; -use crate::type_coercion::functions::{data_types, TIMEZONE_PLACEHOLDER}; +use crate::signature::TIMEZONE_WILDCARD; +use crate::type_coercion::functions::data_types; use crate::{ conditional_expressions, struct_expressions, utils, Signature, TypeSignature, Volatility, @@ -1029,22 +1030,22 @@ impl BuiltinScalarFunction { Exact(vec![Utf8, Timestamp(Nanosecond, None)]), Exact(vec![ Utf8, - Timestamp(Nanosecond, Some(TIMEZONE_PLACEHOLDER.into())), + Timestamp(Nanosecond, Some(TIMEZONE_WILDCARD.into())), ]), Exact(vec![Utf8, Timestamp(Microsecond, None)]), Exact(vec![ Utf8, - Timestamp(Microsecond, Some(TIMEZONE_PLACEHOLDER.into())), + Timestamp(Microsecond, Some(TIMEZONE_WILDCARD.into())), ]), Exact(vec![Utf8, Timestamp(Millisecond, None)]), Exact(vec![ Utf8, - Timestamp(Millisecond, Some(TIMEZONE_PLACEHOLDER.into())), + Timestamp(Millisecond, Some(TIMEZONE_WILDCARD.into())), ]), Exact(vec![Utf8, Timestamp(Second, None)]), Exact(vec![ Utf8, - Timestamp(Second, Some(TIMEZONE_PLACEHOLDER.into())), + Timestamp(Second, Some(TIMEZONE_WILDCARD.into())), ]), ], self.volatility(), @@ -1059,11 +1060,8 @@ impl BuiltinScalarFunction { ]), Exact(vec![ Interval(MonthDayNano), - Timestamp( - array_type.clone(), - Some(TIMEZONE_PLACEHOLDER.into()), - ), - Timestamp(Nanosecond, Some(TIMEZONE_PLACEHOLDER.into())), + Timestamp(array_type.clone(), Some(TIMEZONE_WILDCARD.into())), + Timestamp(Nanosecond, Some(TIMEZONE_WILDCARD.into())), ]), Exact(vec![ Interval(DayTime), @@ -1072,11 +1070,8 @@ impl BuiltinScalarFunction { ]), Exact(vec![ Interval(DayTime), - Timestamp( - array_type.clone(), - Some(TIMEZONE_PLACEHOLDER.into()), - ), - Timestamp(Nanosecond, Some(TIMEZONE_PLACEHOLDER.into())), + Timestamp(array_type.clone(), Some(TIMEZONE_WILDCARD.into())), + Timestamp(Nanosecond, Some(TIMEZONE_WILDCARD.into())), ]), Exact(vec![ Interval(MonthDayNano), @@ -1084,10 +1079,7 @@ impl BuiltinScalarFunction { ]), Exact(vec![ Interval(MonthDayNano), - Timestamp( - array_type.clone(), - Some(TIMEZONE_PLACEHOLDER.into()), - ), + Timestamp(array_type.clone(), Some(TIMEZONE_WILDCARD.into())), ]), Exact(vec![ Interval(DayTime), @@ -1095,7 +1087,7 @@ impl BuiltinScalarFunction { ]), Exact(vec![ Interval(DayTime), - Timestamp(array_type, Some(TIMEZONE_PLACEHOLDER.into())), + Timestamp(array_type, Some(TIMEZONE_WILDCARD.into())), ]), ] }; @@ -1115,22 +1107,22 @@ impl BuiltinScalarFunction { Exact(vec![Utf8, Timestamp(Second, None)]), Exact(vec![ Utf8, - Timestamp(Second, Some(TIMEZONE_PLACEHOLDER.into())), + Timestamp(Second, Some(TIMEZONE_WILDCARD.into())), ]), Exact(vec![Utf8, Timestamp(Microsecond, None)]), Exact(vec![ Utf8, - Timestamp(Microsecond, Some(TIMEZONE_PLACEHOLDER.into())), + Timestamp(Microsecond, Some(TIMEZONE_WILDCARD.into())), ]), Exact(vec![Utf8, Timestamp(Millisecond, None)]), Exact(vec![ Utf8, - Timestamp(Millisecond, Some(TIMEZONE_PLACEHOLDER.into())), + Timestamp(Millisecond, Some(TIMEZONE_WILDCARD.into())), ]), Exact(vec![Utf8, Timestamp(Nanosecond, None)]), Exact(vec![ Utf8, - Timestamp(Nanosecond, Some(TIMEZONE_PLACEHOLDER.into())), + Timestamp(Nanosecond, Some(TIMEZONE_WILDCARD.into())), ]), ], self.volatility(), diff --git a/datafusion/expr/src/lib.rs b/datafusion/expr/src/lib.rs index d35233bc39d2..fee0bdf5c1a4 100644 --- a/datafusion/expr/src/lib.rs +++ b/datafusion/expr/src/lib.rs @@ -74,7 +74,7 @@ pub use logical_plan::*; pub use nullif::SUPPORTED_NULLIF_TYPES; pub use operator::Operator; pub use partition_evaluator::PartitionEvaluator; -pub use signature::{Signature, TypeSignature, Volatility}; +pub use signature::{Signature, TypeSignature, Volatility, TIMEZONE_WILDCARD}; pub use table_source::{TableProviderFilterPushDown, TableSource, TableType}; pub use udaf::AggregateUDF; pub use udf::ScalarUDF; diff --git a/datafusion/expr/src/signature.rs b/datafusion/expr/src/signature.rs index 988fe7c91d4f..0d732ee6e5b9 100644 --- a/datafusion/expr/src/signature.rs +++ b/datafusion/expr/src/signature.rs @@ -20,35 +20,82 @@ use arrow::datatypes::DataType; +/// Constant that is used as a placeholder for any valid timezone. +/// This is used where a function can accept a timestamp type with any +/// valid timezone, it exists to avoid the need to enumerate all possible +/// timezones. See [`TypeSignature`] for more details. +/// +/// Type coercion always ensures that functions will be executed using +/// timestamp arrays that have a valid time zone. Functions must never +/// return results with this timezone. +pub const TIMEZONE_WILDCARD: &str = "+TZ"; + ///A function's volatility, which defines the functions eligibility for certain optimizations #[derive(Debug, PartialEq, Eq, PartialOrd, Ord, Clone, Copy, Hash)] pub enum Volatility { - /// Immutable - An immutable function will always return the same output when given the same - /// input. An example of this is [super::BuiltinScalarFunction::Cos]. + /// An immutable function will always return the same output when given the same + /// input. An example of this is [super::BuiltinScalarFunction::Cos]. DataFusion + /// will attempt to inline immutable functions during planning. Immutable, - /// Stable - A stable function may return different values given the same input across different + /// A stable function may return different values given the same input across different /// queries but must return the same value for a given input within a query. An example of - /// this is [super::BuiltinScalarFunction::Now]. + /// this is [super::BuiltinScalarFunction::Now]. DataFusion + /// will attempt to inline `Stable` functions during planning, when possible. + /// For query `select col1, now() from t1`, it might take a while to execute but + /// `now()` column will be the same for each output row, which is evaluated + /// during planning. Stable, - /// Volatile - A volatile function may change the return value from evaluation to evaluation. + /// A volatile function may change the return value from evaluation to evaluation. /// Multiple invocations of a volatile function may return different results when used in the - /// same query. An example of this is [super::BuiltinScalarFunction::Random]. + /// same query. An example of this is [super::BuiltinScalarFunction::Random]. DataFusion + /// can not evaluate such functions during planning. + /// In the query `select col1, random() from t1`, `random()` function will be evaluated + /// for each output row, resulting in a unique random value for each row. Volatile, } -/// A function's type signature, which defines the function's supported argument types. +/// A function's type signature defines the types of arguments the function supports. +/// +/// Functions typically support only a few different types of arguments compared to the +/// different datatypes in Arrow. To make functions easy to use, when possible DataFusion +/// automatically coerces (add casts to) function arguments so they match the type signature. +/// +/// For example, a function like `cos` may only be implemented for `Float64` arguments. To support a query +/// that calles `cos` with a different argument type, such as `cos(int_column)`, type coercion automatically +/// adds a cast such as `cos(CAST int_column AS DOUBLE)` during planning. +/// +/// # Data Types +/// Types to match are represented using Arrow's [`DataType`]. [`DataType::Timestamp`] has an optional variable +/// timezone specification. To specify a function can handle a timestamp with *ANY* timezone, use +/// the [`TIMEZONE_WILDCARD`]. For example: +/// +/// ``` +/// # use arrow::datatypes::{DataType, TimeUnit}; +/// # use datafusion_expr::{TIMEZONE_WILDCARD, TypeSignature}; +/// let type_signature = TypeSignature::Exact(vec![ +/// // A nanosecond precision timestamp with ANY timezone +/// // matches Timestamp(Nanosecond, Some("+0:00")) +/// // matches Timestamp(Nanosecond, Some("+5:00")) +/// // does not match Timestamp(Nanosecond, None) +/// DataType::Timestamp(TimeUnit::Nanosecond, Some(TIMEZONE_WILDCARD.into())), +/// ]); +/// ``` #[derive(Debug, Clone, PartialEq, Eq, Hash)] pub enum TypeSignature { - /// arbitrary number of arguments of an common type out of a list of valid types - // A function such as `concat` is `Variadic(vec![DataType::Utf8, DataType::LargeUtf8])` + /// arbitrary number of arguments of an common type out of a list of valid types. + /// + /// # Examples + /// A function such as `concat` is `Variadic(vec![DataType::Utf8, DataType::LargeUtf8])` Variadic(Vec), - /// arbitrary number of arguments of an arbitrary but equal type - // A function such as `array` is `VariadicEqual` - // The first argument decides the type used for coercion + /// arbitrary number of arguments of an arbitrary but equal type. + /// DataFusion attempts to coerce all argument types to match the first argument's type + /// + /// # Examples + /// A function such as `array` is `VariadicEqual` VariadicEqual, /// arbitrary number of arguments with arbitrary types VariadicAny, - /// fixed number of arguments of an arbitrary but equal type out of a list of valid types + /// fixed number of arguments of an arbitrary but equal type out of a list of valid types. /// /// # Examples /// 1. A function of one argument of f64 is `Uniform(1, vec![DataType::Float64])` @@ -58,7 +105,8 @@ pub enum TypeSignature { Exact(Vec), /// fixed number of arguments of arbitrary types Any(usize), - /// One of a list of signatures + /// Matches exactly one of a list of [`TypeSignature`]s. Coercion is attempted to match + /// the signatures in order, and stops after the first success, if any. OneOf(Vec), } @@ -104,46 +152,48 @@ impl TypeSignature { } } -/// The signature of a function defines the supported argument types -/// and its volatility. +/// Defines the supported argument types ([`TypeSignature`]) and [`Volatility`] for a function. +/// +/// DataFusion will automatically coerce (cast) argument types to one of the supported +/// function signatures, if possible. #[derive(Debug, Clone, PartialEq, Eq, Hash)] pub struct Signature { - /// type_signature - The types that the function accepts. See [TypeSignature] for more information. + /// The data types that the function accepts. See [TypeSignature] for more information. pub type_signature: TypeSignature, - /// volatility - The volatility of the function. See [Volatility] for more information. + /// The volatility of the function. See [Volatility] for more information. pub volatility: Volatility, } impl Signature { - /// new - Creates a new Signature from any type signature and the volatility. + /// Creates a new Signature from a given type signature and volatility. pub fn new(type_signature: TypeSignature, volatility: Volatility) -> Self { Signature { type_signature, volatility, } } - /// variadic - Creates a variadic signature that represents an arbitrary number of arguments all from a type in common_types. + /// An arbitrary number of arguments with the same type, from those listed in `common_types`. pub fn variadic(common_types: Vec, volatility: Volatility) -> Self { Self { type_signature: TypeSignature::Variadic(common_types), volatility, } } - /// variadic_equal - Creates a variadic signature that represents an arbitrary number of arguments of the same type. + /// An arbitrary number of arguments of the same type. pub fn variadic_equal(volatility: Volatility) -> Self { Self { type_signature: TypeSignature::VariadicEqual, volatility, } } - /// variadic_any - Creates a variadic signature that represents an arbitrary number of arguments of any type. + /// An arbitrary number of arguments of any type. pub fn variadic_any(volatility: Volatility) -> Self { Self { type_signature: TypeSignature::VariadicAny, volatility, } } - /// uniform - Creates a function with a fixed number of arguments of the same type, which must be from valid_types. + /// A fixed number of arguments of the same type, from those listed in `valid_types`. pub fn uniform( arg_count: usize, valid_types: Vec, @@ -154,21 +204,21 @@ impl Signature { volatility, } } - /// exact - Creates a signature which must match the types in exact_types in order. + /// Exactly matches the types in `exact_types`, in order. pub fn exact(exact_types: Vec, volatility: Volatility) -> Self { Signature { type_signature: TypeSignature::Exact(exact_types), volatility, } } - /// any - Creates a signature which can a be made of any type but of a specified number + /// A specified number of arguments of any type pub fn any(arg_count: usize, volatility: Volatility) -> Self { Signature { type_signature: TypeSignature::Any(arg_count), volatility, } } - /// one_of Creates a signature which can match any of the [TypeSignature]s which are passed in. + /// Any one of a list of [TypeSignature]s. pub fn one_of(type_signatures: Vec, volatility: Volatility) -> Self { Signature { type_signature: TypeSignature::OneOf(type_signatures), diff --git a/datafusion/expr/src/type_coercion/functions.rs b/datafusion/expr/src/type_coercion/functions.rs index 5452c8a5c8f5..17ca40236d41 100644 --- a/datafusion/expr/src/type_coercion/functions.rs +++ b/datafusion/expr/src/type_coercion/functions.rs @@ -15,6 +15,7 @@ // specific language governing permissions and limitations // under the License. +use crate::signature::TIMEZONE_WILDCARD; use crate::{Signature, TypeSignature}; use arrow::{ compute::can_cast_types, @@ -22,16 +23,6 @@ use arrow::{ }; use datafusion_common::{plan_err, DataFusionError, Result}; -/// Constant that is used as a placeholder for any valid timezone. -/// This is used where a function can accept a timestamp type with any -/// valid timezone, it exists to avoid the need to enumerate all possible -/// timezones. -/// -/// Type coercion always ensures that functions will be executed using -/// timestamp arrays that have a valid time zone. Functions must never -/// return results with this timezone. -pub(crate) const TIMEZONE_PLACEHOLDER: &str = "+TZ"; - /// Performs type coercion for function arguments. /// /// Returns the data types to which each argument must be coerced to @@ -232,7 +223,7 @@ fn coerced_from<'a>( Utf8 | LargeUtf8 => Some(type_into.clone()), Null if can_cast_types(type_from, type_into) => Some(type_into.clone()), - Timestamp(unit, Some(tz)) if tz.as_ref() == TIMEZONE_PLACEHOLDER => { + Timestamp(unit, Some(tz)) if tz.as_ref() == TIMEZONE_WILDCARD => { match type_from { Timestamp(_, Some(from_tz)) => { Some(Timestamp(unit.clone(), Some(from_tz.clone()))) From 093b775adc3593e9e5cb7343e28406ed458551ad Mon Sep 17 00:00:00 2001 From: Yang Jiang Date: Sat, 7 Oct 2023 18:23:49 +0800 Subject: [PATCH 036/572] [feat] Support cache ListFiles result cache in session level (#7620) * support cache list files under path in session * add test * fix clippy * fix clippy2 * fix comment * Update datafusion/execution/src/cache/cache_manager.rs Co-authored-by: Andrew Lamb * fix fmt * fix api change * fix api change2 --------- Co-authored-by: Andrew Lamb --- .../core/src/datasource/listing/helpers.rs | 16 +- .../core/src/datasource/listing/table.rs | 8 +- datafusion/core/src/datasource/listing/url.rs | 34 +++- datafusion/core/src/test/object_store.rs | 15 +- .../core/tests/parquet/file_statistics.rs | 160 ++++++++++++++---- .../execution/src/cache/cache_manager.rs | 30 ++++ datafusion/execution/src/cache/cache_unit.rs | 83 ++++++++- 7 files changed, 296 insertions(+), 50 deletions(-) diff --git a/datafusion/core/src/datasource/listing/helpers.rs b/datafusion/core/src/datasource/listing/helpers.rs index 476c58b698d8..d6a0add9b253 100644 --- a/datafusion/core/src/datasource/listing/helpers.rs +++ b/datafusion/core/src/datasource/listing/helpers.rs @@ -36,6 +36,7 @@ use crate::{error::Result, scalar::ScalarValue}; use super::PartitionedFile; use crate::datasource::listing::ListingTableUrl; +use crate::execution::context::SessionState; use datafusion_common::tree_node::{TreeNode, VisitRecursion}; use datafusion_common::{Column, DFField, DFSchema, DataFusionError}; use datafusion_expr::expr::ScalarUDF; @@ -315,6 +316,7 @@ async fn prune_partitions( /// `filters` might contain expressions that can be resolved only at the /// file level (e.g. Parquet row group pruning). pub async fn pruned_partition_list<'a>( + ctx: &'a SessionState, store: &'a dyn ObjectStore, table_path: &'a ListingTableUrl, filters: &'a [Expr], @@ -325,7 +327,8 @@ pub async fn pruned_partition_list<'a>( if partition_cols.is_empty() { return Ok(Box::pin( table_path - .list_all_files(store, file_extension) + .list_all_files(ctx, store, file_extension) + .await? .map_ok(|object_meta| object_meta.into()), )); } @@ -422,7 +425,7 @@ mod tests { use futures::StreamExt; use crate::logical_expr::{case, col, lit}; - use crate::test::object_store::make_test_store; + use crate::test::object_store::make_test_store_and_state; use super::*; @@ -468,12 +471,13 @@ mod tests { #[tokio::test] async fn test_pruned_partition_list_empty() { - let store = make_test_store(&[ + let (store, state) = make_test_store_and_state(&[ ("tablepath/mypartition=val1/notparquetfile", 100), ("tablepath/file.parquet", 100), ]); let filter = Expr::eq(col("mypartition"), lit("val1")); let pruned = pruned_partition_list( + &state, store.as_ref(), &ListingTableUrl::parse("file:///tablepath/").unwrap(), &[filter], @@ -490,13 +494,14 @@ mod tests { #[tokio::test] async fn test_pruned_partition_list() { - let store = make_test_store(&[ + let (store, state) = make_test_store_and_state(&[ ("tablepath/mypartition=val1/file.parquet", 100), ("tablepath/mypartition=val2/file.parquet", 100), ("tablepath/mypartition=val1/other=val3/file.parquet", 100), ]); let filter = Expr::eq(col("mypartition"), lit("val1")); let pruned = pruned_partition_list( + &state, store.as_ref(), &ListingTableUrl::parse("file:///tablepath/").unwrap(), &[filter], @@ -532,7 +537,7 @@ mod tests { #[tokio::test] async fn test_pruned_partition_list_multi() { - let store = make_test_store(&[ + let (store, state) = make_test_store_and_state(&[ ("tablepath/part1=p1v1/file.parquet", 100), ("tablepath/part1=p1v2/part2=p2v1/file1.parquet", 100), ("tablepath/part1=p1v2/part2=p2v1/file2.parquet", 100), @@ -544,6 +549,7 @@ mod tests { // filter3 cannot be resolved at partition pruning let filter3 = Expr::eq(col("part2"), col("other")); let pruned = pruned_partition_list( + &state, store.as_ref(), &ListingTableUrl::parse("file:///tablepath/").unwrap(), &[filter1, filter2, filter3], diff --git a/datafusion/core/src/datasource/listing/table.rs b/datafusion/core/src/datasource/listing/table.rs index 5b1710d344ee..7834de2b19f6 100644 --- a/datafusion/core/src/datasource/listing/table.rs +++ b/datafusion/core/src/datasource/listing/table.rs @@ -165,7 +165,8 @@ impl ListingTableConfig { .table_paths .get(0) .unwrap() - .list_all_files(store.as_ref(), "") + .list_all_files(state, store.as_ref(), "") + .await? .next() .await .ok_or_else(|| DataFusionError::Internal("No files for table".into()))??; @@ -507,7 +508,8 @@ impl ListingOptions { let store = state.runtime_env().object_store(table_path)?; let files: Vec<_> = table_path - .list_all_files(store.as_ref(), &self.file_extension) + .list_all_files(state, store.as_ref(), &self.file_extension) + .await? .try_collect() .await?; @@ -844,6 +846,7 @@ impl TableProvider for ListingTable { let store = state.runtime_env().object_store(table_path)?; let file_list_stream = pruned_partition_list( + state, store.as_ref(), table_path, &[], @@ -933,6 +936,7 @@ impl ListingTable { // list files (with partitions) let file_list = future::try_join_all(self.table_paths.iter().map(|table_path| { pruned_partition_list( + ctx, store.as_ref(), table_path, filters, diff --git a/datafusion/core/src/datasource/listing/url.rs b/datafusion/core/src/datasource/listing/url.rs index 96998de17b5d..4d1ca4853a73 100644 --- a/datafusion/core/src/datasource/listing/url.rs +++ b/datafusion/core/src/datasource/listing/url.rs @@ -18,14 +18,17 @@ use std::fs; use crate::datasource::object_store::ObjectStoreUrl; +use crate::execution::context::SessionState; use datafusion_common::{DataFusionError, Result}; use futures::stream::BoxStream; use futures::{StreamExt, TryStreamExt}; use glob::Pattern; use itertools::Itertools; +use log::debug; use object_store::path::Path; use object_store::{ObjectMeta, ObjectStore}; use percent_encoding; +use std::sync::Arc; use url::Url; /// A parsed URL identifying files for a listing table, see [`ListingTableUrl::parse`] @@ -185,28 +188,43 @@ impl ListingTableUrl { } /// List all files identified by this [`ListingTableUrl`] for the provided `file_extension` - pub(crate) fn list_all_files<'a>( + pub(crate) async fn list_all_files<'a>( &'a self, + ctx: &'a SessionState, store: &'a dyn ObjectStore, file_extension: &'a str, - ) -> BoxStream<'a, Result> { + ) -> Result>> { // If the prefix is a file, use a head request, otherwise list let is_dir = self.url.as_str().ends_with('/'); let list = match is_dir { - true => futures::stream::once(store.list(Some(&self.prefix))) - .try_flatten() - .boxed(), + true => match ctx.runtime_env().cache_manager.get_list_files_cache() { + None => futures::stream::once(store.list(Some(&self.prefix))) + .try_flatten() + .boxed(), + Some(cache) => { + if let Some(res) = cache.get(&self.prefix) { + debug!("Hit list all files cache"); + futures::stream::iter(res.as_ref().clone().into_iter().map(Ok)) + .boxed() + } else { + let list_res = store.list(Some(&self.prefix)).await; + let vec = list_res?.try_collect::>().await?; + cache.put(&self.prefix, Arc::new(vec.clone())); + futures::stream::iter(vec.into_iter().map(Ok)).boxed() + } + } + }, false => futures::stream::once(store.head(&self.prefix)).boxed(), }; - - list.map_err(Into::into) + Ok(list .try_filter(move |meta| { let path = &meta.location; let extension_match = path.as_ref().ends_with(file_extension); let glob_match = self.contains(path); futures::future::ready(extension_match && glob_match) }) - .boxed() + .map_err(DataFusionError::ObjectStore) + .boxed()) } /// Returns this [`ListingTableUrl`] as a string diff --git a/datafusion/core/src/test/object_store.rs b/datafusion/core/src/test/object_store.rs index 425d0724ea4f..08cebb56cc77 100644 --- a/datafusion/core/src/test/object_store.rs +++ b/datafusion/core/src/test/object_store.rs @@ -15,7 +15,10 @@ // specific language governing permissions and limitations // under the License. //! Object store implementation used for testing +use crate::execution::context::SessionState; use crate::prelude::SessionContext; +use datafusion_execution::config::SessionConfig; +use datafusion_execution::runtime_env::RuntimeEnv; use futures::FutureExt; use object_store::{memory::InMemory, path::Path, ObjectMeta, ObjectStore}; use std::sync::Arc; @@ -25,11 +28,11 @@ use url::Url; pub fn register_test_store(ctx: &SessionContext, files: &[(&str, u64)]) { let url = Url::parse("test://").unwrap(); ctx.runtime_env() - .register_object_store(&url, make_test_store(files)); + .register_object_store(&url, make_test_store_and_state(files).0); } /// Create a test object store with the provided files -pub fn make_test_store(files: &[(&str, u64)]) -> Arc { +pub fn make_test_store_and_state(files: &[(&str, u64)]) -> (Arc, SessionState) { let memory = InMemory::new(); for (name, size) in files { @@ -40,7 +43,13 @@ pub fn make_test_store(files: &[(&str, u64)]) -> Arc { .unwrap(); } - Arc::new(memory) + ( + Arc::new(memory), + SessionState::new_with_config_rt( + SessionConfig::default(), + Arc::new(RuntimeEnv::default()), + ), + ) } /// Helper method to fetch the file size and date at given path and create a `ObjectMeta` diff --git a/datafusion/core/tests/parquet/file_statistics.rs b/datafusion/core/tests/parquet/file_statistics.rs index 58f81cc57190..ac4a91720ed9 100644 --- a/datafusion/core/tests/parquet/file_statistics.rs +++ b/datafusion/core/tests/parquet/file_statistics.rs @@ -19,15 +19,20 @@ use datafusion::datasource::file_format::parquet::ParquetFormat; use datafusion::datasource::listing::{ ListingOptions, ListingTable, ListingTableConfig, ListingTableUrl, }; +use datafusion::datasource::physical_plan::ParquetExec; use datafusion::datasource::TableProvider; use datafusion::execution::context::SessionState; use datafusion::prelude::SessionContext; use datafusion_execution::cache::cache_manager::CacheManagerConfig; use datafusion_execution::cache::cache_unit; -use datafusion_execution::cache::cache_unit::DefaultFileStatisticsCache; +use datafusion_execution::cache::cache_unit::{ + DefaultFileStatisticsCache, DefaultListFilesCache, +}; use datafusion_execution::config::SessionConfig; use datafusion_execution::runtime_env::{RuntimeConfig, RuntimeEnv}; +use std::fs; use std::sync::Arc; +use tempfile::tempdir; #[tokio::test] async fn load_table_stats_with_session_level_cache() { @@ -35,78 +40,162 @@ async fn load_table_stats_with_session_level_cache() { let filename = format!("{}/{}", testdata, "alltypes_plain.parquet"); let table_path = ListingTableUrl::parse(filename).unwrap(); - let (cache1, state1) = get_cache_runtime_state(); + let (cache1, _, state1) = get_cache_runtime_state(); // Create a separate DefaultFileStatisticsCache - let (cache2, state2) = get_cache_runtime_state(); + let (cache2, _, state2) = get_cache_runtime_state(); let opt = ListingOptions::new(Arc::new(ParquetFormat::default())); - let table1 = get_listing_with_cache(&table_path, cache1, &state1, &opt).await; - let table2 = get_listing_with_cache(&table_path, cache2, &state2, &opt).await; + let table1 = get_listing_table(&table_path, Some(cache1), &opt).await; + let table2 = get_listing_table(&table_path, Some(cache2), &opt).await; //Session 1 first time list files - assert_eq!(get_cache_size(&state1), 0); + assert_eq!(get_static_cache_size(&state1), 0); let exec1 = table1.scan(&state1, None, &[], None).await.unwrap(); assert_eq!(exec1.statistics().num_rows, Some(8)); assert_eq!(exec1.statistics().total_byte_size, Some(671)); - assert_eq!(get_cache_size(&state1), 1); + assert_eq!(get_static_cache_size(&state1), 1); //Session 2 first time list files //check session 1 cache result not show in session 2 - assert_eq!( - state2 - .runtime_env() - .cache_manager - .get_file_statistic_cache() - .unwrap() - .len(), - 0 - ); + assert_eq!(get_static_cache_size(&state2), 0); let exec2 = table2.scan(&state2, None, &[], None).await.unwrap(); assert_eq!(exec2.statistics().num_rows, Some(8)); assert_eq!(exec2.statistics().total_byte_size, Some(671)); - assert_eq!(get_cache_size(&state2), 1); + assert_eq!(get_static_cache_size(&state2), 1); //Session 1 second time list files //check session 1 cache result not show in session 2 - assert_eq!(get_cache_size(&state1), 1); + assert_eq!(get_static_cache_size(&state1), 1); let exec3 = table1.scan(&state1, None, &[], None).await.unwrap(); assert_eq!(exec3.statistics().num_rows, Some(8)); assert_eq!(exec3.statistics().total_byte_size, Some(671)); // List same file no increase - assert_eq!(get_cache_size(&state1), 1); + assert_eq!(get_static_cache_size(&state1), 1); } -async fn get_listing_with_cache( +#[tokio::test] +async fn list_files_with_session_level_cache() { + let p_name = "alltypes_plain.parquet"; + let testdata = datafusion::test_util::parquet_test_data(); + let filename = format!("{}/{}", testdata, p_name); + + let temp_path1 = tempdir() + .unwrap() + .into_path() + .into_os_string() + .into_string() + .unwrap(); + let temp_filename1 = format!("{}/{}", temp_path1, p_name); + + let temp_path2 = tempdir() + .unwrap() + .into_path() + .into_os_string() + .into_string() + .unwrap(); + let temp_filename2 = format!("{}/{}", temp_path2, p_name); + + fs::copy(filename.clone(), temp_filename1).expect("panic"); + fs::copy(filename, temp_filename2).expect("panic"); + + let table_path = ListingTableUrl::parse(temp_path1).unwrap(); + + let (_, _, state1) = get_cache_runtime_state(); + + // Create a separate DefaultFileStatisticsCache + let (_, _, state2) = get_cache_runtime_state(); + + let opt = ListingOptions::new(Arc::new(ParquetFormat::default())); + + let table1 = get_listing_table(&table_path, None, &opt).await; + let table2 = get_listing_table(&table_path, None, &opt).await; + + //Session 1 first time list files + assert_eq!(get_list_file_cache_size(&state1), 0); + let exec1 = table1.scan(&state1, None, &[], None).await.unwrap(); + let parquet1 = exec1.as_any().downcast_ref::().unwrap(); + + assert_eq!(get_list_file_cache_size(&state1), 1); + let fg = &parquet1.base_config().file_groups; + assert_eq!(fg.len(), 1); + assert_eq!(fg.get(0).unwrap().len(), 1); + + //Session 2 first time list files + //check session 1 cache result not show in session 2 + assert_eq!(get_list_file_cache_size(&state2), 0); + let exec2 = table2.scan(&state2, None, &[], None).await.unwrap(); + let parquet2 = exec2.as_any().downcast_ref::().unwrap(); + + assert_eq!(get_list_file_cache_size(&state2), 1); + let fg2 = &parquet2.base_config().file_groups; + assert_eq!(fg2.len(), 1); + assert_eq!(fg2.get(0).unwrap().len(), 1); + + //Session 1 second time list files + //check session 1 cache result not show in session 2 + assert_eq!(get_list_file_cache_size(&state1), 1); + let exec3 = table1.scan(&state1, None, &[], None).await.unwrap(); + let parquet3 = exec3.as_any().downcast_ref::().unwrap(); + + assert_eq!(get_list_file_cache_size(&state1), 1); + let fg = &parquet3.base_config().file_groups; + assert_eq!(fg.len(), 1); + assert_eq!(fg.get(0).unwrap().len(), 1); + // List same file no increase + assert_eq!(get_list_file_cache_size(&state1), 1); +} + +async fn get_listing_table( table_path: &ListingTableUrl, - cache1: Arc, - state1: &SessionState, + static_cache: Option>, opt: &ListingOptions, ) -> ListingTable { - let schema = opt.infer_schema(state1, table_path).await.unwrap(); + let schema = opt + .infer_schema( + &SessionState::new_with_config_rt( + SessionConfig::default(), + Arc::new(RuntimeEnv::default()), + ), + table_path, + ) + .await + .unwrap(); let config1 = ListingTableConfig::new(table_path.clone()) .with_listing_options(opt.clone()) .with_schema(schema); - ListingTable::try_new(config1) - .unwrap() - .with_cache(Some(cache1)) + let table = ListingTable::try_new(config1).unwrap(); + if let Some(c) = static_cache { + table.with_cache(Some(c)) + } else { + table + } } -fn get_cache_runtime_state() -> (Arc, SessionState) { +fn get_cache_runtime_state() -> ( + Arc, + Arc, + SessionState, +) { let cache_config = CacheManagerConfig::default(); - let cache1 = Arc::new(cache_unit::DefaultFileStatisticsCache::default()); - let cache_config = cache_config.with_files_statistics_cache(Some(cache1.clone())); + let file_static_cache = Arc::new(cache_unit::DefaultFileStatisticsCache::default()); + let list_file_cache = Arc::new(cache_unit::DefaultListFilesCache::default()); + + let cache_config = cache_config + .with_files_statistics_cache(Some(file_static_cache.clone())) + .with_list_files_cache(Some(list_file_cache.clone())); + let rt = Arc::new( RuntimeEnv::new(RuntimeConfig::new().with_cache_manager(cache_config)).unwrap(), ); let state = SessionContext::new_with_config_rt(SessionConfig::default(), rt).state(); - (cache1, state) + (file_static_cache, list_file_cache, state) } -fn get_cache_size(state1: &SessionState) -> usize { +fn get_static_cache_size(state1: &SessionState) -> usize { state1 .runtime_env() .cache_manager @@ -114,3 +203,12 @@ fn get_cache_size(state1: &SessionState) -> usize { .unwrap() .len() } + +fn get_list_file_cache_size(state1: &SessionState) -> usize { + state1 + .runtime_env() + .cache_manager + .get_list_files_cache() + .unwrap() + .len() +} diff --git a/datafusion/execution/src/cache/cache_manager.rs b/datafusion/execution/src/cache/cache_manager.rs index 987b47bbb8f5..97529263688b 100644 --- a/datafusion/execution/src/cache/cache_manager.rs +++ b/datafusion/execution/src/cache/cache_manager.rs @@ -29,15 +29,25 @@ use std::sync::Arc; pub type FileStatisticsCache = Arc, Extra = ObjectMeta>>; +pub type ListFilesCache = + Arc>, Extra = ObjectMeta>>; + impl Debug for dyn CacheAccessor, Extra = ObjectMeta> { fn fmt(&self, f: &mut Formatter<'_>) -> std::fmt::Result { write!(f, "Cache name: {} with length: {}", self.name(), self.len()) } } +impl Debug for dyn CacheAccessor>, Extra = ObjectMeta> { + fn fmt(&self, f: &mut Formatter<'_>) -> std::fmt::Result { + write!(f, "Cache name: {} with length: {}", self.name(), self.len()) + } +} + #[derive(Default, Debug)] pub struct CacheManager { file_statistic_cache: Option, + list_files_cache: Option, } impl CacheManager { @@ -46,6 +56,9 @@ impl CacheManager { if let Some(cc) = &config.table_files_statistics_cache { manager.file_statistic_cache = Some(cc.clone()) } + if let Some(lc) = &config.list_files_cache { + manager.list_files_cache = Some(lc.clone()) + } Ok(Arc::new(manager)) } @@ -53,6 +66,11 @@ impl CacheManager { pub fn get_file_statistic_cache(&self) -> Option { self.file_statistic_cache.clone() } + + /// Get the cache of objectMeta under same path. + pub fn get_list_files_cache(&self) -> Option { + self.list_files_cache.clone() + } } #[derive(Clone, Default)] @@ -61,6 +79,13 @@ pub struct CacheManagerConfig { /// Avoid get same file statistics repeatedly in same datafusion session. /// Default is disable. Fow now only supports Parquet files. pub table_files_statistics_cache: Option, + /// Enable cache of file metadata when listing files. + /// This setting avoids listing file meta of the same path repeatedly + /// in same session, which may be expensive in certain situations (e.g. remote object storage). + /// Note that if this option is enabled, DataFusion will not see any updates to the underlying + /// location. + /// Default is disable. + pub list_files_cache: Option, } impl CacheManagerConfig { @@ -71,4 +96,9 @@ impl CacheManagerConfig { self.table_files_statistics_cache = cache; self } + + pub fn with_list_files_cache(mut self, cache: Option) -> Self { + self.list_files_cache = cache; + self + } } diff --git a/datafusion/execution/src/cache/cache_unit.rs b/datafusion/execution/src/cache/cache_unit.rs index 3ef699ac2360..c432a67587fe 100644 --- a/datafusion/execution/src/cache/cache_unit.rs +++ b/datafusion/execution/src/cache/cache_unit.rs @@ -94,9 +94,69 @@ impl CacheAccessor> for DefaultFileStatisticsCache { } } +/// Collected files metadata for listing files. +/// Cache will not invalided until user call remove or clear. +#[derive(Default)] +pub struct DefaultListFilesCache { + statistics: DashMap>>, +} + +impl CacheAccessor>> for DefaultListFilesCache { + type Extra = ObjectMeta; + + fn get(&self, k: &Path) -> Option>> { + self.statistics.get(k).map(|x| x.value().clone()) + } + + fn get_with_extra( + &self, + _k: &Path, + _e: &Self::Extra, + ) -> Option>> { + panic!("Not supported DefaultListFilesCache get_with_extra") + } + + fn put( + &self, + key: &Path, + value: Arc>, + ) -> Option>> { + self.statistics.insert(key.clone(), value) + } + + fn put_with_extra( + &self, + _key: &Path, + _value: Arc>, + _e: &Self::Extra, + ) -> Option>> { + panic!("Not supported DefaultListFilesCache put_with_extra") + } + + fn remove(&mut self, k: &Path) -> Option>> { + self.statistics.remove(k).map(|x| x.1) + } + + fn contains_key(&self, k: &Path) -> bool { + self.statistics.contains_key(k) + } + + fn len(&self) -> usize { + self.statistics.len() + } + + fn clear(&self) { + self.statistics.clear() + } + + fn name(&self) -> String { + "DefaultListFilesCache".to_string() + } +} + #[cfg(test)] mod tests { - use crate::cache::cache_unit::DefaultFileStatisticsCache; + use crate::cache::cache_unit::{DefaultFileStatisticsCache, DefaultListFilesCache}; use crate::cache::CacheAccessor; use chrono::DateTime; use datafusion_common::Statistics; @@ -137,4 +197,25 @@ mod tests { meta2.location = Path::from("test2"); assert!(cache.get_with_extra(&meta2.location, &meta2).is_none()); } + + #[test] + fn test_list_file_cache() { + let meta = ObjectMeta { + location: Path::from("test"), + last_modified: DateTime::parse_from_rfc3339("2022-09-27T22:36:00+02:00") + .unwrap() + .into(), + size: 1024, + e_tag: None, + }; + + let cache = DefaultListFilesCache::default(); + assert!(cache.get(&meta.location).is_none()); + + cache.put(&meta.location, vec![meta.clone()].into()); + assert_eq!( + cache.get(&meta.location).unwrap().get(0).unwrap().clone(), + meta.clone() + ); + } } From e23d34bae60bb2f9c496241e218bab795af3af83 Mon Sep 17 00:00:00 2001 From: comphead Date: Sat, 7 Oct 2023 04:00:53 -0700 Subject: [PATCH 037/572] Support `SHOW ALL VERBOSE` to show settings description (#7735) * Expand SHOW ALL stmt to show settings description * cli tests * comments --- datafusion-cli/tests/cli_integration.rs | 2 +- .../core/src/catalog/information_schema.rs | 15 ++- datafusion/sql/src/statement.rs | 30 +++--- .../test_files/information_schema.slt | 96 +++++++++++++++++++ .../sqllogictest/test_files/set_variable.slt | 2 +- docs/source/user-guide/cli.md | 29 ++++-- 6 files changed, 149 insertions(+), 25 deletions(-) diff --git a/datafusion-cli/tests/cli_integration.rs b/datafusion-cli/tests/cli_integration.rs index 28344ffa94f8..119a0aa39d3c 100644 --- a/datafusion-cli/tests/cli_integration.rs +++ b/datafusion-cli/tests/cli_integration.rs @@ -43,7 +43,7 @@ fn init() { )] #[case::set_batch_size( ["--command", "show datafusion.execution.batch_size", "--format", "json", "-q", "-b", "1"], - "[{\"name\":\"datafusion.execution.batch_size\",\"setting\":\"1\"}]\n" + "[{\"name\":\"datafusion.execution.batch_size\",\"value\":\"1\"}]\n" )] #[test] fn cli_quick_test<'a>( diff --git a/datafusion/core/src/catalog/information_schema.rs b/datafusion/core/src/catalog/information_schema.rs index b30683a3ea13..3a8fef2d25ab 100644 --- a/datafusion/core/src/catalog/information_schema.rs +++ b/datafusion/core/src/catalog/information_schema.rs @@ -626,7 +626,8 @@ impl InformationSchemaDfSettings { fn new(config: InformationSchemaConfig) -> Self { let schema = Arc::new(Schema::new(vec![ Field::new("name", DataType::Utf8, false), - Field::new("setting", DataType::Utf8, true), + Field::new("value", DataType::Utf8, true), + Field::new("description", DataType::Utf8, true), ])); Self { schema, config } @@ -635,7 +636,8 @@ impl InformationSchemaDfSettings { fn builder(&self) -> InformationSchemaDfSettingsBuilder { InformationSchemaDfSettingsBuilder { names: StringBuilder::new(), - settings: StringBuilder::new(), + values: StringBuilder::new(), + descriptions: StringBuilder::new(), schema: self.schema.clone(), } } @@ -664,13 +666,15 @@ impl PartitionStream for InformationSchemaDfSettings { struct InformationSchemaDfSettingsBuilder { schema: SchemaRef, names: StringBuilder, - settings: StringBuilder, + values: StringBuilder, + descriptions: StringBuilder, } impl InformationSchemaDfSettingsBuilder { fn add_setting(&mut self, entry: ConfigEntry) { self.names.append_value(entry.key); - self.settings.append_option(entry.value); + self.values.append_option(entry.value); + self.descriptions.append_value(entry.description); } fn finish(&mut self) -> RecordBatch { @@ -678,7 +682,8 @@ impl InformationSchemaDfSettingsBuilder { self.schema.clone(), vec![ Arc::new(self.names.finish()), - Arc::new(self.settings.finish()), + Arc::new(self.values.finish()), + Arc::new(self.descriptions.finish()), ], ) .unwrap() diff --git a/datafusion/sql/src/statement.rs b/datafusion/sql/src/statement.rs index ab19fa716c9b..de85820543ea 100644 --- a/datafusion/sql/src/statement.rs +++ b/datafusion/sql/src/statement.rs @@ -757,28 +757,34 @@ impl<'a, S: ContextProvider> SqlToRel<'a, S> { } fn show_variable_to_plan(&self, variable: &[Ident]) -> Result { - let variable = object_name_to_string(&ObjectName(variable.to_vec())); - if !self.has_table("information_schema", "df_settings") { return plan_err!( "SHOW [VARIABLE] is not supported unless information_schema is enabled" ); } - let variable_lower = variable.to_lowercase(); + let verbose = variable + .last() + .map(|s| ident_to_string(s) == "verbose") + .unwrap_or(false); + let mut variable_vec = variable.to_vec(); + let mut columns: String = "name, value".to_owned(); + + if verbose { + columns = format!("{columns}, description"); + variable_vec = variable_vec.split_at(variable_vec.len() - 1).0.to_vec(); + } - let query = if variable_lower == "all" { + let variable = object_name_to_string(&ObjectName(variable_vec)); + let base_query = format!("SELECT {columns} FROM information_schema.df_settings"); + let query = if variable == "all" { // Add an ORDER BY so the output comes out in a consistent order - String::from( - "SELECT name, setting FROM information_schema.df_settings ORDER BY name", - ) - } else if variable_lower == "timezone" || variable_lower == "time.zone" { + format!("{base_query} ORDER BY name") + } else if variable == "timezone" || variable == "time.zone" { // we could introduce alias in OptionDefinition if this string matching thing grows - String::from("SELECT name, setting FROM information_schema.df_settings WHERE name = 'datafusion.execution.time_zone'") + format!("{base_query} WHERE name = 'datafusion.execution.time_zone'") } else { - format!( - "SELECT name, setting FROM information_schema.df_settings WHERE name = '{variable}'" - ) + format!("{base_query} WHERE name = '{variable}'") }; let mut rewrite = DFParser::parse_sql(&query)?; diff --git a/datafusion/sqllogictest/test_files/information_schema.slt b/datafusion/sqllogictest/test_files/information_schema.slt index 74c1296fa4ef..d2ec21488d3d 100644 --- a/datafusion/sqllogictest/test_files/information_schema.slt +++ b/datafusion/sqllogictest/test_files/information_schema.slt @@ -202,12 +202,85 @@ datafusion.sql_parser.dialect generic datafusion.sql_parser.enable_ident_normalization true datafusion.sql_parser.parse_float_as_decimal false +# show all variables with verbose +query TTT rowsort +SHOW ALL VERBOSE +---- +datafusion.catalog.create_default_catalog_and_schema true Whether the default catalog and schema should be created automatically. +datafusion.catalog.default_catalog datafusion The default catalog name - this impacts what SQL queries use if not specified +datafusion.catalog.default_schema public The default schema name - this impacts what SQL queries use if not specified +datafusion.catalog.format NULL Type of `TableProvider` to use when loading `default` schema +datafusion.catalog.has_header false If the file has a header +datafusion.catalog.information_schema true Should DataFusion provide access to `information_schema` virtual tables for displaying schema information +datafusion.catalog.location NULL Location scanned to load tables for `default` schema +datafusion.execution.aggregate.scalar_update_factor 10 Specifies the threshold for using `ScalarValue`s to update accumulators during high-cardinality aggregations for each input batch. The aggregation is considered high-cardinality if the number of affected groups is greater than or equal to `batch_size / scalar_update_factor`. In such cases, `ScalarValue`s are utilized for updating accumulators, rather than the default batch-slice approach. This can lead to performance improvements. By adjusting the `scalar_update_factor`, you can balance the trade-off between more efficient accumulator updates and the number of groups affected. +datafusion.execution.batch_size 8192 Default batch size while creating new batches, it's especially useful for buffer-in-memory batches since creating tiny batches would result in too much metadata memory consumption +datafusion.execution.coalesce_batches true When set to true, record batches will be examined between each operator and small batches will be coalesced into larger batches. This is helpful when there are highly selective filters or joins that could produce tiny output batches. The target batch size is determined by the configuration setting +datafusion.execution.collect_statistics false Should DataFusion collect statistics after listing files +datafusion.execution.meta_fetch_concurrency 32 Number of files to read in parallel when inferring schema and statistics +datafusion.execution.parquet.allow_single_file_parallelism false Controls whether DataFusion will attempt to speed up writing large parquet files by first writing multiple smaller files and then stitching them together into a single large file. This will result in faster write speeds, but higher memory usage. Also currently unsupported are bloom filters and column indexes when single_file_parallelism is enabled. +datafusion.execution.parquet.bloom_filter_enabled false Sets if bloom filter is enabled for any column +datafusion.execution.parquet.bloom_filter_fpp NULL Sets bloom filter false positive probability. If NULL, uses default parquet writer setting +datafusion.execution.parquet.bloom_filter_ndv NULL Sets bloom filter number of distinct values. If NULL, uses default parquet writer setting +datafusion.execution.parquet.column_index_truncate_length NULL Sets column index trucate length +datafusion.execution.parquet.compression zstd(3) Sets default parquet compression codec Valid values are: uncompressed, snappy, gzip(level), lzo, brotli(level), lz4, zstd(level), and lz4_raw. These values are not case sensitive. If NULL, uses default parquet writer setting +datafusion.execution.parquet.created_by datafusion Sets "created by" property +datafusion.execution.parquet.data_page_row_count_limit 18446744073709551615 Sets best effort maximum number of rows in data page +datafusion.execution.parquet.data_pagesize_limit 1048576 Sets best effort maximum size of data page in bytes +datafusion.execution.parquet.dictionary_enabled NULL Sets if dictionary encoding is enabled. If NULL, uses default parquet writer setting +datafusion.execution.parquet.dictionary_page_size_limit 1048576 Sets best effort maximum dictionary page size, in bytes +datafusion.execution.parquet.enable_page_index true 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 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.max_row_group_size 1048576 Sets maximum number of rows in a row group +datafusion.execution.parquet.max_statistics_size NULL Sets max statistics size for any column. If NULL, uses default parquet writer setting +datafusion.execution.parquet.metadata_size_hint NULL If specified, the parquet reader will try and fetch the last `size_hint` bytes of the parquet file optimistically. If not specified, two reads are required: One read to fetch the 8-byte parquet footer and another to fetch the metadata length encoded in the footer +datafusion.execution.parquet.pruning true 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 If true, filter expressions are be applied during the parquet decoding operation to reduce the number of rows decoded +datafusion.execution.parquet.reorder_filters false 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.skip_metadata true If true, the parquet reader skip the optional embedded metadata that may be in the file Schema. This setting can help avoid schema conflicts when querying multiple parquet files with schemas containing compatible types but different metadata +datafusion.execution.parquet.statistics_enabled NULL Sets if statistics are enabled for any column Valid values are: "none", "chunk", and "page" These values are not case sensitive. If NULL, uses default parquet writer setting +datafusion.execution.parquet.write_batch_size 1024 Sets write_batch_size in bytes +datafusion.execution.parquet.writer_version 1.0 Sets parquet writer version valid values are "1.0" and "2.0" +datafusion.execution.planning_concurrency 13 Fan-out during initial physical planning. This is mostly use to plan `UNION` children in parallel. Defaults to the number of CPU cores on the system +datafusion.execution.sort_in_place_threshold_bytes 1048576 When sorting, below what size should data be concatenated and sorted in a single RecordBatch rather than sorted in batches and merged. +datafusion.execution.sort_spill_reservation_bytes 10485760 Specifies the reserved memory for each spillable sort operation to facilitate an in-memory merge. When a sort operation spills to disk, the in-memory data must be sorted and merged before being written to a file. This setting reserves a specific amount of memory for that in-memory sort/merge process. Note: This setting is irrelevant if the sort operation cannot spill (i.e., if there's no `DiskManager` configured). +datafusion.execution.target_partitions 7 Number of partitions for query execution. Increasing partitions can increase concurrency. Defaults to the number of CPU cores on the system +datafusion.execution.time_zone +00:00 The default time zone Some functions, e.g. `EXTRACT(HOUR from SOME_TIME)`, shift the underlying datetime according to this time zone, and then extract the hour +datafusion.explain.logical_plan_only false When set to true, the explain statement will only print logical plans +datafusion.explain.physical_plan_only false When set to true, the explain statement will only print physical plans +datafusion.explain.show_statistics false When set to true, the explain statement will print operator statistics for physical plans +datafusion.optimizer.allow_symmetric_joins_without_pruning true Should DataFusion allow symmetric hash joins for unbounded data sources even when its inputs do not have any ordering or filtering If the flag is not enabled, the SymmetricHashJoin operator will be unable to prune its internal buffers, resulting in certain join types - such as Full, Left, LeftAnti, LeftSemi, Right, RightAnti, and RightSemi - being produced only at the end of the execution. This is not typical in stream processing. Additionally, without proper design for long runner execution, all types of joins may encounter out-of-memory errors. +datafusion.optimizer.enable_round_robin_repartition true When set to true, the physical plan optimizer will try to add round robin repartitioning to increase parallelism to leverage more CPU cores +datafusion.optimizer.enable_topk_aggregation true When set to true, the optimizer will attempt to perform limit operations during aggregations, if possible +datafusion.optimizer.filter_null_join_keys false When set to true, the optimizer will insert filters before a join between a nullable and non-nullable column to filter out nulls on the nullable side. This filter can add additional overhead when the file format does not fully support predicate push down. +datafusion.optimizer.hash_join_single_partition_threshold 1048576 The maximum estimated size in bytes for one input side of a HashJoin will be collected into a single partition +datafusion.optimizer.max_passes 3 Number of times that the optimizer will attempt to optimize the plan +datafusion.optimizer.prefer_existing_sort false When true, DataFusion will opportunistically remove sorts when the data is already sorted, (i.e. setting `preserve_order` to true on `RepartitionExec` and using `SortPreservingMergeExec`) When false, DataFusion will maximize plan parallelism using `RepartitionExec` even if this requires subsequently resorting data using a `SortExec`. +datafusion.optimizer.prefer_hash_join true When set to true, the physical plan optimizer will prefer HashJoin over SortMergeJoin. HashJoin can work more efficiently than SortMergeJoin but consumes more memory +datafusion.optimizer.repartition_aggregations true Should DataFusion repartition data using the aggregate keys to execute aggregates in parallel using the provided `target_partitions` level +datafusion.optimizer.repartition_file_min_size 10485760 Minimum total files size in bytes to perform file scan repartitioning. +datafusion.optimizer.repartition_file_scans true When set to `true`, file groups will be repartitioned to achieve maximum parallelism. Currently Parquet and CSV formats are supported. If set to `true`, all files will be repartitioned evenly (i.e., a single large file might be partitioned into smaller chunks) for parallel scanning. If set to `false`, different files will be read in parallel, but repartitioning won't happen within a single file. +datafusion.optimizer.repartition_joins true Should DataFusion repartition data using the join keys to execute joins in parallel using the provided `target_partitions` level +datafusion.optimizer.repartition_sorts true Should DataFusion execute sorts in a per-partition fashion and merge afterwards instead of coalescing first and sorting globally. With this flag is enabled, plans in the form below ```text "SortExec: [a@0 ASC]", " CoalescePartitionsExec", " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", ``` would turn into the plan below which performs better in multithreaded environments ```text "SortPreservingMergeExec: [a@0 ASC]", " SortExec: [a@0 ASC]", " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", ``` +datafusion.optimizer.repartition_windows true Should DataFusion repartition data using the partitions keys to execute window functions in parallel using the provided `target_partitions` level +datafusion.optimizer.skip_failed_rules false When set to true, the logical plan optimizer will produce warning messages if any optimization rules produce errors and then proceed to the next rule. When set to false, any rules that produce errors will cause the query to fail +datafusion.optimizer.top_down_join_key_reordering true When set to true, the physical plan optimizer will run a top down process to reorder the join keys +datafusion.sql_parser.dialect generic Configure the SQL dialect used by DataFusion's parser; supported values include: Generic, MySQL, PostgreSQL, Hive, SQLite, Snowflake, Redshift, MsSQL, ClickHouse, BigQuery, and Ansi. +datafusion.sql_parser.enable_ident_normalization true When set to true, SQL parser will normalize ident (convert ident to lowercase when not quoted) +datafusion.sql_parser.parse_float_as_decimal false When set to true, SQL parser will parse float as decimal type + # show_variable_in_config_options query TT SHOW datafusion.execution.batch_size ---- datafusion.execution.batch_size 8192 +# show_variable_in_config_options_verbose +query TTT +SHOW datafusion.execution.batch_size VERBOSE +---- +datafusion.execution.batch_size 8192 Default batch size while creating new batches, it's especially useful for buffer-in-memory batches since creating tiny batches would result in too much metadata memory consumption + # show_time_zone_default_utc # https://github.com/apache/arrow-datafusion/issues/3255 query TT @@ -223,6 +296,26 @@ SHOW TIMEZONE datafusion.execution.time_zone +00:00 +# show_time_zone_default_utc_verbose +# https://github.com/apache/arrow-datafusion/issues/3255 +query TTT +SHOW TIME ZONE VERBOSE +---- +datafusion.execution.time_zone +00:00 The default time zone Some functions, e.g. `EXTRACT(HOUR from SOME_TIME)`, shift the underlying datetime according to this time zone, and then extract the hour + +# show_timezone_default_utc +# https://github.com/apache/arrow-datafusion/issues/3255 +query TTT +SHOW TIMEZONE VERBOSE +---- +datafusion.execution.time_zone +00:00 The default time zone Some functions, e.g. `EXTRACT(HOUR from SOME_TIME)`, shift the underlying datetime according to this time zone, and then extract the hour + + +# show empty verbose +query TTT +SHOW VERBOSE +---- + # information_schema_describe_table ## some_table @@ -372,6 +465,9 @@ set datafusion.catalog.information_schema = false; statement error Error during planning: SHOW \[VARIABLE\] is not supported unless information_schema is enabled SHOW SOMETHING +statement error Error during planning: SHOW \[VARIABLE\] is not supported unless information_schema is enabled +SHOW SOMETHING VERBOSE + statement ok set datafusion.catalog.information_schema = true; diff --git a/datafusion/sqllogictest/test_files/set_variable.slt b/datafusion/sqllogictest/test_files/set_variable.slt index 440fb2c6ef2b..714e1e995e26 100644 --- a/datafusion/sqllogictest/test_files/set_variable.slt +++ b/datafusion/sqllogictest/test_files/set_variable.slt @@ -243,4 +243,4 @@ statement ok SET TIME ZONE = 'Asia/Taipei2' statement error Arrow error: Parser error: Invalid timezone "Asia/Taipei2": 'Asia/Taipei2' is not a valid timezone -SELECT '2000-01-01T00:00:00'::TIMESTAMP::TIMESTAMPTZ +SELECT '2000-01-01T00:00:00'::TIMESTAMP::TIMESTAMPTZ \ No newline at end of file diff --git a/docs/source/user-guide/cli.md b/docs/source/user-guide/cli.md index 05b4165e612f..e8fdae7bb097 100644 --- a/docs/source/user-guide/cli.md +++ b/docs/source/user-guide/cli.md @@ -397,11 +397,13 @@ Available commands inside DataFusion CLI are: - Show configuration options +`SHOW ALL [VERBOSE]` + ```SQL > show all; +-------------------------------------------------+---------+ -| name | setting | +| name | value | +-------------------------------------------------+---------+ | datafusion.execution.batch_size | 8192 | | datafusion.execution.coalesce_batches | true | @@ -414,6 +416,21 @@ Available commands inside DataFusion CLI are: ``` +- Show specific configuration option + +`SHOW xyz.abc.qwe [VERBOSE]` + +```SQL +> show datafusion.execution.batch_size; + ++-------------------------------------------------+---------+ +| name | value | ++-------------------------------------------------+---------+ +| datafusion.execution.batch_size | 8192 | ++-------------------------------------------------+---------+ + +``` + - Set configuration options ```SQL @@ -432,12 +449,12 @@ For example, to set `datafusion.execution.batch_size` to `1024` you would set the `DATAFUSION_EXECUTION_BATCH_SIZE` environment variable appropriately: -```shell +```SQL $ DATAFUSION_EXECUTION_BATCH_SIZE=1024 datafusion-cli DataFusion CLI v12.0.0 ❯ show all; +-------------------------------------------------+---------+ -| name | setting | +| name | value | +-------------------------------------------------+---------+ | datafusion.execution.batch_size | 1024 | | datafusion.execution.coalesce_batches | true | @@ -452,13 +469,13 @@ DataFusion CLI v12.0.0 You can change the configuration options using `SET` statement as well -```shell +```SQL $ datafusion-cli DataFusion CLI v13.0.0 ❯ show datafusion.execution.batch_size; +---------------------------------+---------+ -| name | setting | +| name | value | +---------------------------------+---------+ | datafusion.execution.batch_size | 8192 | +---------------------------------+---------+ @@ -469,7 +486,7 @@ DataFusion CLI v13.0.0 ❯ show datafusion.execution.batch_size; +---------------------------------+---------+ -| name | setting | +| name | value | +---------------------------------+---------+ | datafusion.execution.batch_size | 1024 | +---------------------------------+---------+ From fb371058244c00325ec8f779623ce223b9b50b86 Mon Sep 17 00:00:00 2001 From: Andrew Lamb Date: Sat, 7 Oct 2023 15:51:44 -0400 Subject: [PATCH 038/572] Minor: Improve TableProvider document, and add ascii art (#7759) * Improve TableProvider document, and add ascii art * Update datafusion/core/src/datasource/provider.rs --- datafusion/core/src/datasource/provider.rs | 81 +++++++++++++++++++--- 1 file changed, 72 insertions(+), 9 deletions(-) diff --git a/datafusion/core/src/datasource/provider.rs b/datafusion/core/src/datasource/provider.rs index 5ebcc45b572b..af99c12d39c9 100644 --- a/datafusion/core/src/datasource/provider.rs +++ b/datafusion/core/src/datasource/provider.rs @@ -54,24 +54,87 @@ pub trait TableProvider: Sync + Send { None } - /// Get the Logical Plan of this table, if available. + /// Get the [`LogicalPlan`] of this table, if available fn get_logical_plan(&self) -> Option<&LogicalPlan> { None } - /// Create an ExecutionPlan that will scan the table. - /// The table provider will be usually responsible of grouping - /// the source data into partitions that can be efficiently - /// parallelized or distributed. + /// Create an [`ExecutionPlan`] for scanning the table with optionally + /// specified `projection`, `filter` and `limit`, described below. + /// + /// The `ExecutionPlan` is responsible scanning the datasource's + /// partitions in a streaming, parallelized fashion. + /// + /// # Projection + /// + /// If specified, only a subset of columns should be returned, in the order + /// specified. The projection is a set of indexes of the fields in + /// [`Self::schema`]. + /// + /// DataFusion provides the projection to scan only the columns actually + /// used in the query to improve performance, an optimization called + /// "Projection Pushdown". Some datasources, such as Parquet, can use this + /// information to go significantly faster when only a subset of columns is + /// required. + /// + /// # Filters + /// + /// A list of boolean filter [`Expr`]s to evaluate *during* the scan, in the + /// manner specified by [`Self::supports_filters_pushdown`]. Only rows for + /// which *all* of the `Expr`s evaluate to `true` must be returned (aka the + /// expressions are `AND`ed together). + /// + /// DataFusion pushes filtering into the scans whenever possible + /// ("Projection Pushdown"), and depending on the format and the + /// implementation of the format, evaluating the predicate during the scan + /// can increase performance significantly. + /// + /// ## Note: Some columns may appear *only* in Filters + /// + /// In certain cases, a query may only use a certain column in a Filter that + /// has been completely pushed down to the scan. In this case, the + /// projection will not contain all the columns found in the filter + /// expressions. + /// + /// For example, given the query `SELECT t.a FROM t WHERE t.b > 5`, + /// + /// ```text + /// ┌────────────────────┐ + /// │ Projection(t.a) │ + /// └────────────────────┘ + /// ▲ + /// │ + /// │ + /// ┌────────────────────┐ Filter ┌────────────────────┐ Projection ┌────────────────────┐ + /// │ Filter(t.b > 5) │────Pushdown──▶ │ Projection(t.a) │ ───Pushdown───▶ │ Projection(t.a) │ + /// └────────────────────┘ └────────────────────┘ └────────────────────┘ + /// ▲ ▲ ▲ + /// │ │ │ + /// │ │ ┌────────────────────┐ + /// ┌────────────────────┐ ┌────────────────────┐ │ Scan │ + /// │ Scan │ │ Scan │ │ filter=(t.b > 5) │ + /// └────────────────────┘ │ filter=(t.b > 5) │ │ projection=(t.a) │ + /// └────────────────────┘ └────────────────────┘ + /// + /// Initial Plan If `TableProviderFilterPushDown` Projection pushdown notes that + /// returns true, filter pushdown the scan only needs t.a + /// pushes the filter into the scan + /// BUT internally evaluating the + /// predicate still requires t.b + /// ``` + /// + /// # Limit + /// + /// If `limit` is specified, must only produce *at least* this many rows, + /// (though it may return more). Like Projection Pushdown and Filter + /// Pushdown, DataFusion pushes `LIMIT`s as far down in the plan as + /// possible, called "Limit Pushdown" as some sources can use this + /// information to improve their performance. async fn scan( &self, state: &SessionState, projection: Option<&Vec>, filters: &[Expr], - // limit can be used to reduce the amount scanned - // from the datasource as a performance optimization. - // If set, it contains the amount of rows needed by the `LogicalPlan`, - // The datasource should return *at least* this number of rows if available. limit: Option, ) -> Result>; From 219cfb4ccb36045c73409127db51377d66ca0f33 Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Sat, 7 Oct 2023 15:56:45 -0600 Subject: [PATCH 039/572] Prepare 32.0.0 Release (#7769) * bump version * revert unintentional change * changelog * update changelog --- Cargo.toml | 2 +- benchmarks/Cargo.toml | 8 +- datafusion-cli/Cargo.lock | 66 +++++----- datafusion-cli/Cargo.toml | 4 +- datafusion/CHANGELOG.md | 1 + datafusion/core/Cargo.toml | 14 +- datafusion/execution/Cargo.toml | 4 +- datafusion/expr/Cargo.toml | 2 +- datafusion/optimizer/Cargo.toml | 8 +- datafusion/physical-expr/Cargo.toml | 4 +- datafusion/physical-plan/Cargo.toml | 8 +- datafusion/proto/Cargo.toml | 6 +- datafusion/sql/Cargo.toml | 4 +- datafusion/sqllogictest/Cargo.toml | 4 +- datafusion/substrait/Cargo.toml | 2 +- datafusion/wasmtest/Cargo.toml | 2 +- dev/changelog/32.0.0.md | 195 ++++++++++++++++++++++++++++ dev/release/README.md | 2 +- dev/update_datafusion_versions.py | 2 + docs/source/user-guide/configs.md | 2 +- 20 files changed, 269 insertions(+), 71 deletions(-) create mode 100644 dev/changelog/32.0.0.md diff --git a/Cargo.toml b/Cargo.toml index 60ff770d0d13..d28ebd15a09e 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -45,7 +45,7 @@ license = "Apache-2.0" readme = "README.md" repository = "https://github.com/apache/arrow-datafusion" rust-version = "1.70" -version = "31.0.0" +version = "32.0.0" [workspace.dependencies] arrow = { version = "47.0.0", features = ["prettyprint"] } diff --git a/benchmarks/Cargo.toml b/benchmarks/Cargo.toml index 1531b857bcef..0def335521ce 100644 --- a/benchmarks/Cargo.toml +++ b/benchmarks/Cargo.toml @@ -18,7 +18,7 @@ [package] name = "datafusion-benchmarks" description = "DataFusion Benchmarks" -version = "31.0.0" +version = "32.0.0" edition = { workspace = true } authors = ["Apache Arrow "] homepage = "https://github.com/apache/arrow-datafusion" @@ -34,8 +34,8 @@ snmalloc = ["snmalloc-rs"] [dependencies] arrow = { workspace = true } -datafusion = { path = "../datafusion/core", version = "31.0.0" } -datafusion-common = { path = "../datafusion/common", version = "31.0.0" } +datafusion = { path = "../datafusion/core", version = "32.0.0" } +datafusion-common = { path = "../datafusion/common", version = "32.0.0" } env_logger = "0.10" futures = "0.3" log = "^0.4" @@ -50,4 +50,4 @@ test-utils = { path = "../test-utils/", version = "0.1.0" } tokio = { version = "^1.0", features = ["macros", "rt", "rt-multi-thread", "parking_lot"] } [dev-dependencies] -datafusion-proto = { path = "../datafusion/proto", version = "31.0.0" } +datafusion-proto = { path = "../datafusion/proto", version = "32.0.0" } diff --git a/datafusion-cli/Cargo.lock b/datafusion-cli/Cargo.lock index e442393ab08f..54e9a5ccc2dc 100644 --- a/datafusion-cli/Cargo.lock +++ b/datafusion-cli/Cargo.lock @@ -382,7 +382,7 @@ checksum = "bc00ceb34980c03614e35a3a4e218276a0a824e911d07651cd0d858a51e8c0f0" dependencies = [ "proc-macro2", "quote", - "syn 2.0.37", + "syn 2.0.38", ] [[package]] @@ -806,9 +806,9 @@ checksum = "7f30e7476521f6f8af1a1c4c0b8cc94f0bee37d91763d0ca2665f299b6cd8aec" [[package]] name = "byteorder" -version = "1.4.3" +version = "1.5.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "14c189c53d098945499cdfa7ecc63567cf3886b3332b312a5b4585d8d3a6a610" +checksum = "1fd0f2584146f6f2ef48085050886acf353beff7305ebd1ae69500e27c67f64b" [[package]] name = "bytes" @@ -1074,7 +1074,7 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "37e366bff8cd32dd8754b0991fb66b279dc48f598c3a18914852a6673deef583" dependencies = [ "quote", - "syn 2.0.37", + "syn 2.0.38", ] [[package]] @@ -1098,7 +1098,7 @@ dependencies = [ [[package]] name = "datafusion" -version = "31.0.0" +version = "32.0.0" dependencies = [ "ahash", "apache-avro", @@ -1146,7 +1146,7 @@ dependencies = [ [[package]] name = "datafusion-cli" -version = "31.0.0" +version = "32.0.0" dependencies = [ "arrow", "assert_cmd", @@ -1171,7 +1171,7 @@ dependencies = [ [[package]] name = "datafusion-common" -version = "31.0.0" +version = "32.0.0" dependencies = [ "ahash", "apache-avro", @@ -1189,7 +1189,7 @@ dependencies = [ [[package]] name = "datafusion-execution" -version = "31.0.0" +version = "32.0.0" dependencies = [ "arrow", "chrono", @@ -1208,7 +1208,7 @@ dependencies = [ [[package]] name = "datafusion-expr" -version = "31.0.0" +version = "32.0.0" dependencies = [ "ahash", "arrow", @@ -1221,7 +1221,7 @@ dependencies = [ [[package]] name = "datafusion-optimizer" -version = "31.0.0" +version = "32.0.0" dependencies = [ "arrow", "async-trait", @@ -1237,7 +1237,7 @@ dependencies = [ [[package]] name = "datafusion-physical-expr" -version = "31.0.0" +version = "32.0.0" dependencies = [ "ahash", "arrow", @@ -1269,7 +1269,7 @@ dependencies = [ [[package]] name = "datafusion-physical-plan" -version = "31.0.0" +version = "32.0.0" dependencies = [ "ahash", "arrow", @@ -1298,7 +1298,7 @@ dependencies = [ [[package]] name = "datafusion-sql" -version = "31.0.0" +version = "32.0.0" dependencies = [ "arrow", "arrow-schema", @@ -1581,7 +1581,7 @@ checksum = "89ca545a94061b6365f2c7355b4b32bd20df3ff95f02da9329b34ccc3bd6ee72" dependencies = [ "proc-macro2", "quote", - "syn 2.0.37", + "syn 2.0.38", ] [[package]] @@ -2016,9 +2016,9 @@ dependencies = [ [[package]] name = "libc" -version = "0.2.148" +version = "0.2.149" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "9cdc71e17332e86d2e1d38c1f99edcb6288ee11b815fb1a4b049eaa2114d369b" +checksum = "a08173bc88b7955d1b3145aa561539096c421ac8debde8cbc3612ec635fee29b" [[package]] name = "libflate" @@ -2046,9 +2046,9 @@ dependencies = [ [[package]] name = "libm" -version = "0.2.7" +version = "0.2.8" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "f7012b1bbb0719e1097c47611d3898568c546d597c2e74d66f6087edd5233ff4" +checksum = "4ec2a862134d2a7d32d7983ddcdd1c4923530833c9f2ea1a44fc5fa473989058" [[package]] name = "libmimalloc-sys" @@ -2492,7 +2492,7 @@ checksum = "4359fd9c9171ec6e8c62926d6faaf553a8dc3f64e1507e76da7911b4f6a04405" dependencies = [ "proc-macro2", "quote", - "syn 2.0.37", + "syn 2.0.38", ] [[package]] @@ -2582,9 +2582,9 @@ checksum = "dc375e1527247fe1a97d8b7156678dfe7c1af2fc075c9a4db3690ecd2a148068" [[package]] name = "proc-macro2" -version = "1.0.67" +version = "1.0.68" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "3d433d9f1a3e8c1263d9456598b16fec66f4acc9a74dacffd35c7bb09b3a1328" +checksum = "5b1106fec09662ec6dd98ccac0f81cef56984d0b49f75c92d8cbad76e20c005c" dependencies = [ "unicode-ident", ] @@ -2824,9 +2824,9 @@ dependencies = [ [[package]] name = "rustix" -version = "0.38.15" +version = "0.38.17" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "d2f9da0cbd88f9f09e7814e388301c8414c51c62aa6ce1e4b5c551d49d96e531" +checksum = "f25469e9ae0f3d0047ca8b93fc56843f38e6774f0914a107ff8b41be8be8e0b7" dependencies = [ "bitflags 2.4.0", "errno", @@ -3011,7 +3011,7 @@ checksum = "4eca7ac642d82aa35b60049a6eccb4be6be75e599bd2e9adb5f875a737654af2" dependencies = [ "proc-macro2", "quote", - "syn 2.0.37", + "syn 2.0.38", ] [[package]] @@ -3200,7 +3200,7 @@ dependencies = [ "proc-macro2", "quote", "rustversion", - "syn 2.0.37", + "syn 2.0.38", ] [[package]] @@ -3222,9 +3222,9 @@ dependencies = [ [[package]] name = "syn" -version = "2.0.37" +version = "2.0.38" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "7303ef2c05cd654186cb250d29049a24840ca25d2747c25c0381c8d9e2f582e8" +checksum = "e96b79aaa137db8f61e26363a0c9b47d8b4ec75da28b7d1d614c2303e232408b" dependencies = [ "proc-macro2", "quote", @@ -3303,7 +3303,7 @@ checksum = "10712f02019e9288794769fba95cd6847df9874d49d871d062172f9dd41bc4cc" dependencies = [ "proc-macro2", "quote", - "syn 2.0.37", + "syn 2.0.38", ] [[package]] @@ -3394,7 +3394,7 @@ checksum = "630bdcf245f78637c13ec01ffae6187cca34625e8c63150d424b59e55af2675e" dependencies = [ "proc-macro2", "quote", - "syn 2.0.37", + "syn 2.0.38", ] [[package]] @@ -3492,7 +3492,7 @@ checksum = "5f4f31f56159e98206da9efd823404b79b6ef3143b4a7ab76e67b1751b25a4ab" dependencies = [ "proc-macro2", "quote", - "syn 2.0.37", + "syn 2.0.38", ] [[package]] @@ -3537,7 +3537,7 @@ checksum = "f03ca4cb38206e2bef0700092660bb74d696f808514dae47fa1467cbfe26e96e" dependencies = [ "proc-macro2", "quote", - "syn 2.0.37", + "syn 2.0.38", ] [[package]] @@ -3685,7 +3685,7 @@ dependencies = [ "once_cell", "proc-macro2", "quote", - "syn 2.0.37", + "syn 2.0.38", "wasm-bindgen-shared", ] @@ -3719,7 +3719,7 @@ checksum = "54681b18a46765f095758388f2d0cf16eb8d4169b639ab575a8f5693af210c7b" dependencies = [ "proc-macro2", "quote", - "syn 2.0.37", + "syn 2.0.38", "wasm-bindgen-backend", "wasm-bindgen-shared", ] diff --git a/datafusion-cli/Cargo.toml b/datafusion-cli/Cargo.toml index 9653c73d25bf..b2a22cec987f 100644 --- a/datafusion-cli/Cargo.toml +++ b/datafusion-cli/Cargo.toml @@ -18,7 +18,7 @@ [package] name = "datafusion-cli" description = "Command Line Client for DataFusion query engine." -version = "31.0.0" +version = "32.0.0" authors = ["Apache Arrow "] edition = "2021" keywords = ["arrow", "datafusion", "query", "sql"] @@ -34,7 +34,7 @@ async-trait = "0.1.41" aws-config = "0.55" aws-credential-types = "0.55" clap = { version = "3", features = ["derive", "cargo"] } -datafusion = { path = "../datafusion/core", version = "31.0.0", features = ["avro", "crypto_expressions", "encoding_expressions", "regex_expressions", "unicode_expressions", "compression"] } +datafusion = { path = "../datafusion/core", version = "32.0.0", features = ["avro", "crypto_expressions", "encoding_expressions", "regex_expressions", "unicode_expressions", "compression"] } dirs = "4.0.0" env_logger = "0.9" mimalloc = { version = "0.1", default-features = false } diff --git a/datafusion/CHANGELOG.md b/datafusion/CHANGELOG.md index 0a25b747c739..d26081dcb6df 100644 --- a/datafusion/CHANGELOG.md +++ b/datafusion/CHANGELOG.md @@ -19,6 +19,7 @@ # Changelog +- [32.0.0](../dev/changelog/32.0.0.md) - [31.0.0](../dev/changelog/31.0.0.md) - [30.0.0](../dev/changelog/30.0.0.md) - [29.0.0](../dev/changelog/29.0.0.md) diff --git a/datafusion/core/Cargo.toml b/datafusion/core/Cargo.toml index 1db5d55baf10..484d203be848 100644 --- a/datafusion/core/Cargo.toml +++ b/datafusion/core/Cargo.toml @@ -60,13 +60,13 @@ bytes = "1.4" bzip2 = { version = "0.4.3", optional = true } chrono = { workspace = true } dashmap = "5.4.0" -datafusion-common = { path = "../common", version = "31.0.0", features = ["parquet", "object_store"] } -datafusion-execution = { path = "../execution", version = "31.0.0" } -datafusion-expr = { path = "../expr", version = "31.0.0" } -datafusion-optimizer = { path = "../optimizer", version = "31.0.0", default-features = false } -datafusion-physical-expr = { path = "../physical-expr", version = "31.0.0", default-features = false } -datafusion-physical-plan = { path = "../physical-plan", version = "31.0.0", default-features = false } -datafusion-sql = { path = "../sql", version = "31.0.0" } +datafusion-common = { path = "../common", version = "32.0.0", features = ["parquet", "object_store"] } +datafusion-execution = { path = "../execution", version = "32.0.0" } +datafusion-expr = { path = "../expr", version = "32.0.0" } +datafusion-optimizer = { path = "../optimizer", version = "32.0.0", default-features = false } +datafusion-physical-expr = { path = "../physical-expr", version = "32.0.0", default-features = false } +datafusion-physical-plan = { path = "../physical-plan", version = "32.0.0", default-features = false } +datafusion-sql = { path = "../sql", version = "32.0.0" } flate2 = { version = "1.0.24", optional = true } futures = "0.3" glob = "0.3.0" diff --git a/datafusion/execution/Cargo.toml b/datafusion/execution/Cargo.toml index cf4eb5ef1f25..6ae8bccdae38 100644 --- a/datafusion/execution/Cargo.toml +++ b/datafusion/execution/Cargo.toml @@ -36,8 +36,8 @@ path = "src/lib.rs" arrow = { workspace = true } chrono = { version = "0.4", default-features = false } dashmap = "5.4.0" -datafusion-common = { path = "../common", version = "31.0.0" } -datafusion-expr = { path = "../expr", version = "31.0.0" } +datafusion-common = { path = "../common", version = "32.0.0" } +datafusion-expr = { path = "../expr", version = "32.0.0" } futures = "0.3" hashbrown = { version = "0.14", features = ["raw"] } log = "^0.4" diff --git a/datafusion/expr/Cargo.toml b/datafusion/expr/Cargo.toml index 4d69ce747518..c5cf6a1ac11f 100644 --- a/datafusion/expr/Cargo.toml +++ b/datafusion/expr/Cargo.toml @@ -38,7 +38,7 @@ path = "src/lib.rs" ahash = { version = "0.8", default-features = false, features = ["runtime-rng"] } arrow = { workspace = true } arrow-array = { workspace = true } -datafusion-common = { path = "../common", version = "31.0.0", default-features = false } +datafusion-common = { path = "../common", version = "32.0.0", default-features = false } sqlparser = { workspace = true } strum = { version = "0.25.0", features = ["derive"] } strum_macros = "0.25.0" diff --git a/datafusion/optimizer/Cargo.toml b/datafusion/optimizer/Cargo.toml index e1ffcb41ba6e..d60b229716e0 100644 --- a/datafusion/optimizer/Cargo.toml +++ b/datafusion/optimizer/Cargo.toml @@ -42,9 +42,9 @@ unicode_expressions = ["datafusion-physical-expr/unicode_expressions"] arrow = { workspace = true } async-trait = "0.1.41" chrono = { workspace = true } -datafusion-common = { path = "../common", version = "31.0.0", default-features = false } -datafusion-expr = { path = "../expr", version = "31.0.0" } -datafusion-physical-expr = { path = "../physical-expr", version = "31.0.0", default-features = false } +datafusion-common = { path = "../common", version = "32.0.0", default-features = false } +datafusion-expr = { path = "../expr", version = "32.0.0" } +datafusion-physical-expr = { path = "../physical-expr", version = "32.0.0", default-features = false } hashbrown = { version = "0.14", features = ["raw"] } itertools = "0.11" log = "^0.4" @@ -52,5 +52,5 @@ regex-syntax = "0.7.1" [dev-dependencies] ctor = "0.2.0" -datafusion-sql = { path = "../sql", version = "31.0.0" } +datafusion-sql = { path = "../sql", version = "32.0.0" } env_logger = "0.10.0" diff --git a/datafusion/physical-expr/Cargo.toml b/datafusion/physical-expr/Cargo.toml index 2c0ddc692d28..6269f27310a6 100644 --- a/datafusion/physical-expr/Cargo.toml +++ b/datafusion/physical-expr/Cargo.toml @@ -49,8 +49,8 @@ base64 = { version = "0.21", optional = true } blake2 = { version = "^0.10.2", optional = true } blake3 = { version = "1.0", optional = true } chrono = { workspace = true } -datafusion-common = { path = "../common", version = "31.0.0", default-features = false } -datafusion-expr = { path = "../expr", version = "31.0.0" } +datafusion-common = { path = "../common", version = "32.0.0", default-features = false } +datafusion-expr = { path = "../expr", version = "32.0.0" } half = { version = "2.1", default-features = false } hashbrown = { version = "0.14", features = ["raw"] } hex = { version = "0.4", optional = true } diff --git a/datafusion/physical-plan/Cargo.toml b/datafusion/physical-plan/Cargo.toml index ace6f5d95483..2dfcf12e350a 100644 --- a/datafusion/physical-plan/Cargo.toml +++ b/datafusion/physical-plan/Cargo.toml @@ -40,10 +40,10 @@ arrow-buffer = { workspace = true } arrow-schema = { workspace = true } async-trait = "0.1.41" chrono = { version = "0.4.23", default-features = false } -datafusion-common = { path = "../common", version = "31.0.0", default-features = false } -datafusion-execution = { path = "../execution", version = "31.0.0" } -datafusion-expr = { path = "../expr", version = "31.0.0" } -datafusion-physical-expr = { path = "../physical-expr", version = "31.0.0" } +datafusion-common = { path = "../common", version = "32.0.0", default-features = false } +datafusion-execution = { path = "../execution", version = "32.0.0" } +datafusion-expr = { path = "../expr", version = "32.0.0" } +datafusion-physical-expr = { path = "../physical-expr", version = "32.0.0" } futures = "0.3" half = { version = "2.1", default-features = false } hashbrown = { version = "0.14", features = ["raw"] } diff --git a/datafusion/proto/Cargo.toml b/datafusion/proto/Cargo.toml index e27eff537ba6..94e77088a7e8 100644 --- a/datafusion/proto/Cargo.toml +++ b/datafusion/proto/Cargo.toml @@ -42,9 +42,9 @@ json = ["pbjson", "serde", "serde_json"] [dependencies] arrow = { workspace = true } chrono = { workspace = true } -datafusion = { path = "../core", version = "31.0.0" } -datafusion-common = { path = "../common", version = "31.0.0", default-features = false } -datafusion-expr = { path = "../expr", version = "31.0.0" } +datafusion = { path = "../core", version = "32.0.0" } +datafusion-common = { path = "../common", version = "32.0.0", default-features = false } +datafusion-expr = { path = "../expr", version = "32.0.0" } object_store = { version = "0.7.0" } pbjson = { version = "0.5", optional = true } prost = "0.12.0" diff --git a/datafusion/sql/Cargo.toml b/datafusion/sql/Cargo.toml index c2cdc4c52dbd..a00a7f021352 100644 --- a/datafusion/sql/Cargo.toml +++ b/datafusion/sql/Cargo.toml @@ -39,8 +39,8 @@ unicode_expressions = [] [dependencies] arrow = { workspace = true } arrow-schema = { workspace = true } -datafusion-common = { path = "../common", version = "31.0.0", default-features = false } -datafusion-expr = { path = "../expr", version = "31.0.0" } +datafusion-common = { path = "../common", version = "32.0.0", default-features = false } +datafusion-expr = { path = "../expr", version = "32.0.0" } log = "^0.4" sqlparser = { workspace = true } diff --git a/datafusion/sqllogictest/Cargo.toml b/datafusion/sqllogictest/Cargo.toml index 2cd16927be2c..c7fabffa0aa0 100644 --- a/datafusion/sqllogictest/Cargo.toml +++ b/datafusion/sqllogictest/Cargo.toml @@ -34,8 +34,8 @@ path = "src/lib.rs" arrow = {workspace = true} async-trait = "0.1.41" bigdecimal = "0.4.1" -datafusion = {path = "../core", version = "31.0.0"} -datafusion-common = {path = "../common", version = "31.0.0", default-features = false} +datafusion = {path = "../core", version = "32.0.0"} +datafusion-common = {path = "../common", version = "32.0.0", default-features = false} half = "2.2.1" itertools = "0.11" object_store = "0.7.0" diff --git a/datafusion/substrait/Cargo.toml b/datafusion/substrait/Cargo.toml index a9e2209404be..a793df40e2ae 100644 --- a/datafusion/substrait/Cargo.toml +++ b/datafusion/substrait/Cargo.toml @@ -30,7 +30,7 @@ rust-version = "1.70" [dependencies] async-recursion = "1.0" chrono = { workspace = true } -datafusion = { version = "31.0.0", path = "../core" } +datafusion = { version = "32.0.0", path = "../core" } itertools = "0.11" object_store = "0.7.0" prost = "0.11" diff --git a/datafusion/wasmtest/Cargo.toml b/datafusion/wasmtest/Cargo.toml index 0aee45a511f3..691031866af2 100644 --- a/datafusion/wasmtest/Cargo.toml +++ b/datafusion/wasmtest/Cargo.toml @@ -38,7 +38,7 @@ crate-type = ["cdylib", "rlib",] # code size when deploying. console_error_panic_hook = { version = "0.1.1", optional = true } -datafusion-common = { path = "../common", version = "31.0.0", default-features = false } +datafusion-common = { path = "../common", version = "32.0.0", default-features = false } datafusion-expr = { path = "../expr" } datafusion-optimizer = { path = "../optimizer" } datafusion-physical-expr = { path = "../physical-expr" } diff --git a/dev/changelog/32.0.0.md b/dev/changelog/32.0.0.md new file mode 100644 index 000000000000..781fd5001552 --- /dev/null +++ b/dev/changelog/32.0.0.md @@ -0,0 +1,195 @@ + + +## [32.0.0](https://github.com/apache/arrow-datafusion/tree/32.0.0) (2023-10-07) + +[Full Changelog](https://github.com/apache/arrow-datafusion/compare/31.0.0...32.0.0) + +**Breaking changes:** + +- Remove implicit interval type coercion from ScalarValue comparison [#7514](https://github.com/apache/arrow-datafusion/pull/7514) (tustvold) +- Remove get_scan_files and ExecutionPlan::file_scan_config (#7357) [#7487](https://github.com/apache/arrow-datafusion/pull/7487) (tustvold) +- Move `FileCompressionType` out of `common` and into `core` [#7596](https://github.com/apache/arrow-datafusion/pull/7596) (haohuaijin) +- Update arrow 47.0.0 in DataFusion [#7587](https://github.com/apache/arrow-datafusion/pull/7587) (tustvold) +- Rename `bounded_order_preserving_variants` config to `prefer_exising_sort` and update docs [#7723](https://github.com/apache/arrow-datafusion/pull/7723) (alamb) + +**Implemented enhancements:** + +- Parallelize Stateless (CSV/JSON) File Write Serialization [#7452](https://github.com/apache/arrow-datafusion/pull/7452) (devinjdangelo) +- Create a Priority Queue based Aggregation with `limit` [#7192](https://github.com/apache/arrow-datafusion/pull/7192) (avantgardnerio) +- feat: add guarantees to simplification [#7467](https://github.com/apache/arrow-datafusion/pull/7467) (wjones127) +- [Minor]: Produce better plan when group by contains all of the ordering requirements [#7542](https://github.com/apache/arrow-datafusion/pull/7542) (mustafasrepo) +- Make AvroArrowArrayReader possible to scan Avro backed table which contains nested records [#7525](https://github.com/apache/arrow-datafusion/pull/7525) (sarutak) +- feat: Support spilling for hash aggregation [#7400](https://github.com/apache/arrow-datafusion/pull/7400) (kazuyukitanimura) +- Parallelize Parquet Serialization [#7562](https://github.com/apache/arrow-datafusion/pull/7562) (devinjdangelo) +- feat: natively support more data types for the `abs` function. [#7568](https://github.com/apache/arrow-datafusion/pull/7568) (jonahgao) +- feat: Parallel collecting parquet files statistics #7573 [#7595](https://github.com/apache/arrow-datafusion/pull/7595) (hengfeiyang) +- Support hashing List columns [#7616](https://github.com/apache/arrow-datafusion/pull/7616) (jonmmease) +- feat: Better large output display in datafusion-cli with --maxrows option [#7617](https://github.com/apache/arrow-datafusion/pull/7617) (2010YOUY01) +- feat: make parse_float_as_decimal work on negative numbers [#7648](https://github.com/apache/arrow-datafusion/pull/7648) (jonahgao) +- Update Default Parquet Write Compression [#7692](https://github.com/apache/arrow-datafusion/pull/7692) (devinjdangelo) +- Support all the codecs supported by Avro [#7718](https://github.com/apache/arrow-datafusion/pull/7718) (sarutak) +- Optimize "ORDER BY + LIMIT" queries for speed / memory with special TopK operator [#7721](https://github.com/apache/arrow-datafusion/pull/7721) (Dandandan) + +**Fixed bugs:** + +- fix: inconsistent behaviors when dividing floating numbers by zero [#7503](https://github.com/apache/arrow-datafusion/pull/7503) (jonahgao) +- fix: skip EliminateCrossJoin rule if inner join with filter is found [#7529](https://github.com/apache/arrow-datafusion/pull/7529) (epsio-banay) +- fix: check for precision overflow when parsing float as decimal [#7627](https://github.com/apache/arrow-datafusion/pull/7627) (jonahgao) +- fix: substrait limit when fetch is None [#7669](https://github.com/apache/arrow-datafusion/pull/7669) (waynexia) +- fix: coerce text to timestamps with timezones [#7720](https://github.com/apache/arrow-datafusion/pull/7720) (mhilton) +- fix: avro_to_arrow: Handle avro nested nullable struct (union) [#7663](https://github.com/apache/arrow-datafusion/pull/7663) (Samrose-Ahmed) + +**Documentation updates:** + +- Documentation Updates for New Write Related Features [#7520](https://github.com/apache/arrow-datafusion/pull/7520) (devinjdangelo) +- Create 2023 Q4 roadmap [#7551](https://github.com/apache/arrow-datafusion/pull/7551) (graydenshand) +- docs: add section on supports_filters_pushdown [#7680](https://github.com/apache/arrow-datafusion/pull/7680) (tshauck) +- Add LanceDB to the list of Known Users [#7716](https://github.com/apache/arrow-datafusion/pull/7716) (alamb) +- Document crate feature flags [#7713](https://github.com/apache/arrow-datafusion/pull/7713) (alamb) + +**Merged pull requests:** + +- Prepare 31.0.0 release [#7508](https://github.com/apache/arrow-datafusion/pull/7508) (andygrove) +- Minor(proto): Implement `TryFrom<&DFSchema>` for `protobuf::DfSchema` [#7505](https://github.com/apache/arrow-datafusion/pull/7505) (jonahgao) +- fix: inconsistent behaviors when dividing floating numbers by zero [#7503](https://github.com/apache/arrow-datafusion/pull/7503) (jonahgao) +- Parallelize Stateless (CSV/JSON) File Write Serialization [#7452](https://github.com/apache/arrow-datafusion/pull/7452) (devinjdangelo) +- Minor: Remove stray comment markings from encoding error message [#7512](https://github.com/apache/arrow-datafusion/pull/7512) (devinjdangelo) +- Remove implicit interval type coercion from ScalarValue comparison [#7514](https://github.com/apache/arrow-datafusion/pull/7514) (tustvold) +- Minor: deprecate ScalarValue::get_datatype() [#7507](https://github.com/apache/arrow-datafusion/pull/7507) (Weijun-H) +- Propagate error from spawned task reading spills [#7510](https://github.com/apache/arrow-datafusion/pull/7510) (viirya) +- Refactor the EnforceDistribution Rule [#7488](https://github.com/apache/arrow-datafusion/pull/7488) (mustafasrepo) +- Remove get_scan_files and ExecutionPlan::file_scan_config (#7357) [#7487](https://github.com/apache/arrow-datafusion/pull/7487) (tustvold) +- Simplify ScalarValue::distance (#7517) [#7519](https://github.com/apache/arrow-datafusion/pull/7519) (tustvold) +- typo: change `delimeter` to `delimiter` [#7521](https://github.com/apache/arrow-datafusion/pull/7521) (Weijun-H) +- Fix some simplification rules for floating-point arithmetic operations [#7515](https://github.com/apache/arrow-datafusion/pull/7515) (jonahgao) +- Documentation Updates for New Write Related Features [#7520](https://github.com/apache/arrow-datafusion/pull/7520) (devinjdangelo) +- [MINOR]: Move tests from repartition to enforce_distribution file [#7539](https://github.com/apache/arrow-datafusion/pull/7539) (mustafasrepo) +- Update the async-trait crate to resolve clippy bug [#7541](https://github.com/apache/arrow-datafusion/pull/7541) (metesynnada) +- Fix flaky `test_sort_fetch_memory_calculation` test [#7534](https://github.com/apache/arrow-datafusion/pull/7534) (viirya) +- Move common code to utils [#7545](https://github.com/apache/arrow-datafusion/pull/7545) (mustafasrepo) +- Minor: Add comments and clearer constructors to `Interval` [#7526](https://github.com/apache/arrow-datafusion/pull/7526) (alamb) +- fix: skip EliminateCrossJoin rule if inner join with filter is found [#7529](https://github.com/apache/arrow-datafusion/pull/7529) (epsio-banay) +- Create a Priority Queue based Aggregation with `limit` [#7192](https://github.com/apache/arrow-datafusion/pull/7192) (avantgardnerio) +- feat: add guarantees to simplification [#7467](https://github.com/apache/arrow-datafusion/pull/7467) (wjones127) +- [Minor]: Produce better plan when group by contains all of the ordering requirements [#7542](https://github.com/apache/arrow-datafusion/pull/7542) (mustafasrepo) +- Minor: beautify interval display [#7554](https://github.com/apache/arrow-datafusion/pull/7554) (Weijun-H) +- replace ptree with termtree [#7560](https://github.com/apache/arrow-datafusion/pull/7560) (avantgardnerio) +- Make AvroArrowArrayReader possible to scan Avro backed table which contains nested records [#7525](https://github.com/apache/arrow-datafusion/pull/7525) (sarutak) +- Fix a race condition issue on reading spilled file [#7538](https://github.com/apache/arrow-datafusion/pull/7538) (sarutak) +- [MINOR]: Add is single method [#7558](https://github.com/apache/arrow-datafusion/pull/7558) (mustafasrepo) +- Fix `describe ` to work without SessionContext [#7441](https://github.com/apache/arrow-datafusion/pull/7441) (alamb) +- Make the tests in SHJ faster [#7543](https://github.com/apache/arrow-datafusion/pull/7543) (metesynnada) +- feat: Support spilling for hash aggregation [#7400](https://github.com/apache/arrow-datafusion/pull/7400) (kazuyukitanimura) +- Make backtrace as a cargo feature [#7527](https://github.com/apache/arrow-datafusion/pull/7527) (comphead) +- Minor: Fix `clippy` by switching to `timestamp_nanos_opt` instead of (deprecated) `timestamp_nanos` [#7572](https://github.com/apache/arrow-datafusion/pull/7572) (alamb) +- Update sqllogictest requirement from 0.15.0 to 0.16.0 [#7569](https://github.com/apache/arrow-datafusion/pull/7569) (dependabot[bot]) +- extract `datafusion-physical-plan` to its own crate [#7432](https://github.com/apache/arrow-datafusion/pull/7432) (alamb) +- First and Last Accumulators should update with state row excluding is_set flag [#7565](https://github.com/apache/arrow-datafusion/pull/7565) (viirya) +- refactor: simplify code of eliminate_cross_join.rs [#7561](https://github.com/apache/arrow-datafusion/pull/7561) (jackwener) +- Update release instructions for datafusion-physical-plan crate [#7576](https://github.com/apache/arrow-datafusion/pull/7576) (alamb) +- Minor: Update chrono pin to `0.4.31` [#7575](https://github.com/apache/arrow-datafusion/pull/7575) (alamb) +- [feat] Introduce cacheManager in session ctx and make StatisticsCache share in session [#7570](https://github.com/apache/arrow-datafusion/pull/7570) (Ted-Jiang) +- Enhance/Refactor Ordering Equivalence Properties [#7566](https://github.com/apache/arrow-datafusion/pull/7566) (mustafasrepo) +- fix misplaced statements in sqllogictest [#7586](https://github.com/apache/arrow-datafusion/pull/7586) (jonahgao) +- Update substrait requirement from 0.13.1 to 0.14.0 [#7585](https://github.com/apache/arrow-datafusion/pull/7585) (dependabot[bot]) +- chore: use the `create_udwf` function in `simple_udwf`, consistent with `simple_udf` and `simple_udaf` [#7579](https://github.com/apache/arrow-datafusion/pull/7579) (tanruixiang) +- Implement protobuf serialization for AnalyzeExec [#7574](https://github.com/apache/arrow-datafusion/pull/7574) (adhish20) +- chore: fix catalog's usage docs error and add docs about `CatalogList` trait [#7582](https://github.com/apache/arrow-datafusion/pull/7582) (tanruixiang) +- Implement `CardinalityAwareRowConverter` while doing streaming merge [#7401](https://github.com/apache/arrow-datafusion/pull/7401) (JayjeetAtGithub) +- Parallelize Parquet Serialization [#7562](https://github.com/apache/arrow-datafusion/pull/7562) (devinjdangelo) +- feat: natively support more data types for the `abs` function. [#7568](https://github.com/apache/arrow-datafusion/pull/7568) (jonahgao) +- implement string_to_array [#7577](https://github.com/apache/arrow-datafusion/pull/7577) (casperhart) +- Create 2023 Q4 roadmap [#7551](https://github.com/apache/arrow-datafusion/pull/7551) (graydenshand) +- chore: reduce `physical-plan` dependencies [#7599](https://github.com/apache/arrow-datafusion/pull/7599) (crepererum) +- Minor: add githubs start/fork buttons to documentation page [#7588](https://github.com/apache/arrow-datafusion/pull/7588) (alamb) +- Minor: add more examples for `CREATE EXTERNAL TABLE` doc [#7594](https://github.com/apache/arrow-datafusion/pull/7594) (comphead) +- Update nix requirement from 0.26.1 to 0.27.1 [#7438](https://github.com/apache/arrow-datafusion/pull/7438) (dependabot[bot]) +- Update sqllogictest requirement from 0.16.0 to 0.17.0 [#7606](https://github.com/apache/arrow-datafusion/pull/7606) (dependabot[bot]) +- Fix panic in TopK [#7609](https://github.com/apache/arrow-datafusion/pull/7609) (avantgardnerio) +- Move `FileCompressionType` out of `common` and into `core` [#7596](https://github.com/apache/arrow-datafusion/pull/7596) (haohuaijin) +- Expose contents of Constraints [#7603](https://github.com/apache/arrow-datafusion/pull/7603) (tv42) +- Change the unbounded_output API default [#7605](https://github.com/apache/arrow-datafusion/pull/7605) (metesynnada) +- feat: Parallel collecting parquet files statistics #7573 [#7595](https://github.com/apache/arrow-datafusion/pull/7595) (hengfeiyang) +- Support hashing List columns [#7616](https://github.com/apache/arrow-datafusion/pull/7616) (jonmmease) +- [MINOR] Make the sink input aware of its plan [#7610](https://github.com/apache/arrow-datafusion/pull/7610) (metesynnada) +- [MINOR] Reduce complexity on SHJ [#7607](https://github.com/apache/arrow-datafusion/pull/7607) (metesynnada) +- feat: Better large output display in datafusion-cli with --maxrows option [#7617](https://github.com/apache/arrow-datafusion/pull/7617) (2010YOUY01) +- Minor: add examples for `arrow_cast` and `arrow_typeof` to user guide [#7615](https://github.com/apache/arrow-datafusion/pull/7615) (alamb) +- [MINOR]: Fix stack overflow bug for get field access expr [#7623](https://github.com/apache/arrow-datafusion/pull/7623) (mustafasrepo) +- Group By All [#7622](https://github.com/apache/arrow-datafusion/pull/7622) (berkaysynnada) +- Implement protobuf serialization for `(Bounded)WindowAggExec`. [#7557](https://github.com/apache/arrow-datafusion/pull/7557) (vrongmeal) +- Make it possible to compile datafusion-common without default features [#7625](https://github.com/apache/arrow-datafusion/pull/7625) (jonmmease) +- Minor: Adding backtrace documentation [#7628](https://github.com/apache/arrow-datafusion/pull/7628) (comphead) +- fix(5975/5976): timezone handling for timestamps and `date_trunc`, `date_part` and `date_bin` [#7614](https://github.com/apache/arrow-datafusion/pull/7614) (wiedld) +- Minor: remove unecessary `Arc`s in datetime_expressions [#7630](https://github.com/apache/arrow-datafusion/pull/7630) (alamb) +- fix: check for precision overflow when parsing float as decimal [#7627](https://github.com/apache/arrow-datafusion/pull/7627) (jonahgao) +- Update arrow 47.0.0 in DataFusion [#7587](https://github.com/apache/arrow-datafusion/pull/7587) (tustvold) +- Add test crate to compile DataFusion with wasm-pack [#7633](https://github.com/apache/arrow-datafusion/pull/7633) (jonmmease) +- Minor: Update documentation of case expression [#7646](https://github.com/apache/arrow-datafusion/pull/7646) (ongchi) +- Minor: improve docstrings on `SessionState` [#7654](https://github.com/apache/arrow-datafusion/pull/7654) (alamb) +- Update example in the DataFrame documentation. [#7650](https://github.com/apache/arrow-datafusion/pull/7650) (jsimpson-gro) +- Add HTTP object store example [#7602](https://github.com/apache/arrow-datafusion/pull/7602) (pka) +- feat: make parse_float_as_decimal work on negative numbers [#7648](https://github.com/apache/arrow-datafusion/pull/7648) (jonahgao) +- Minor: add doc comments to `ExtractEquijoinPredicate` [#7658](https://github.com/apache/arrow-datafusion/pull/7658) (alamb) +- [MINOR]: Do not add unnecessary hash repartition to the physical plan [#7667](https://github.com/apache/arrow-datafusion/pull/7667) (mustafasrepo) +- Minor: add ticket references to parallel parquet writing code [#7592](https://github.com/apache/arrow-datafusion/pull/7592) (alamb) +- Minor: Add ticket reference and add test comment [#7593](https://github.com/apache/arrow-datafusion/pull/7593) (alamb) +- Support Avro's Enum type and Fixed type [#7635](https://github.com/apache/arrow-datafusion/pull/7635) (sarutak) +- Minor: Migrate datafusion-proto tests into it own binary [#7668](https://github.com/apache/arrow-datafusion/pull/7668) (ongchi) +- Upgrade apache-avro to 0.16 [#7674](https://github.com/apache/arrow-datafusion/pull/7674) (sarutak) +- Move window analysis to the window method [#7672](https://github.com/apache/arrow-datafusion/pull/7672) (mustafasrepo) +- Don't add filters to projection in TableScan [#7670](https://github.com/apache/arrow-datafusion/pull/7670) (Dandandan) +- Minor: Improve `TableProviderFilterPushDown` docs [#7685](https://github.com/apache/arrow-datafusion/pull/7685) (alamb) +- FIX: Test timestamp with table [#7701](https://github.com/apache/arrow-datafusion/pull/7701) (jayzhan211) +- Fix bug in `SimplifyExpressions` [#7699](https://github.com/apache/arrow-datafusion/pull/7699) (Dandandan) +- Enhance Enforce Dist capabilities to fix, sub optimal bad plans [#7671](https://github.com/apache/arrow-datafusion/pull/7671) (mustafasrepo) +- docs: add section on supports_filters_pushdown [#7680](https://github.com/apache/arrow-datafusion/pull/7680) (tshauck) +- Improve cache usage in CI [#7678](https://github.com/apache/arrow-datafusion/pull/7678) (sarutak) +- fix: substrait limit when fetch is None [#7669](https://github.com/apache/arrow-datafusion/pull/7669) (waynexia) +- minor: revert parsing precedence between Aggr and UDAF [#7682](https://github.com/apache/arrow-datafusion/pull/7682) (waynexia) +- Minor: Move hash utils to common [#7684](https://github.com/apache/arrow-datafusion/pull/7684) (jayzhan211) +- Update Default Parquet Write Compression [#7692](https://github.com/apache/arrow-datafusion/pull/7692) (devinjdangelo) +- Stop using cache for the benchmark job [#7706](https://github.com/apache/arrow-datafusion/pull/7706) (sarutak) +- Change rust.yml to run benchmark [#7708](https://github.com/apache/arrow-datafusion/pull/7708) (sarutak) +- Extend infer_placeholder_types to support BETWEEN predicates [#7703](https://github.com/apache/arrow-datafusion/pull/7703) (andrelmartins) +- Minor: Add comment explaining why verify benchmark results uses release mode [#7712](https://github.com/apache/arrow-datafusion/pull/7712) (alamb) +- Support all the codecs supported by Avro [#7718](https://github.com/apache/arrow-datafusion/pull/7718) (sarutak) +- Update substrait requirement from 0.14.0 to 0.15.0 [#7719](https://github.com/apache/arrow-datafusion/pull/7719) (dependabot[bot]) +- fix: coerce text to timestamps with timezones [#7720](https://github.com/apache/arrow-datafusion/pull/7720) (mhilton) +- Add LanceDB to the list of Known Users [#7716](https://github.com/apache/arrow-datafusion/pull/7716) (alamb) +- Enable avro reading/writing in datafusion-cli [#7715](https://github.com/apache/arrow-datafusion/pull/7715) (alamb) +- Document crate feature flags [#7713](https://github.com/apache/arrow-datafusion/pull/7713) (alamb) +- Minor: Consolidate UDF tests [#7704](https://github.com/apache/arrow-datafusion/pull/7704) (alamb) +- Minor: fix CI failure due to Cargo.lock in datafusioncli [#7733](https://github.com/apache/arrow-datafusion/pull/7733) (yjshen) +- MINOR: change file to column index in page_filter trace log [#7730](https://github.com/apache/arrow-datafusion/pull/7730) (mapleFU) +- preserve array type / timezone in `date_bin` and `date_trunc` functions [#7729](https://github.com/apache/arrow-datafusion/pull/7729) (mhilton) +- Remove redundant is_numeric for DataType [#7734](https://github.com/apache/arrow-datafusion/pull/7734) (qrilka) +- fix: avro_to_arrow: Handle avro nested nullable struct (union) [#7663](https://github.com/apache/arrow-datafusion/pull/7663) (Samrose-Ahmed) +- Rename `SessionContext::with_config_rt` to `SessionContext::new_with_config_from_rt`, etc [#7631](https://github.com/apache/arrow-datafusion/pull/7631) (alamb) +- Rename `bounded_order_preserving_variants` config to `prefer_exising_sort` and update docs [#7723](https://github.com/apache/arrow-datafusion/pull/7723) (alamb) +- Optimize "ORDER BY + LIMIT" queries for speed / memory with special TopK operator [#7721](https://github.com/apache/arrow-datafusion/pull/7721) (Dandandan) +- Minor: Improve crate docs [#7740](https://github.com/apache/arrow-datafusion/pull/7740) (alamb) +- [MINOR]: Resolve linter errors in the main [#7753](https://github.com/apache/arrow-datafusion/pull/7753) (mustafasrepo) +- Minor: Build concat_internal() with ListArray construction instead of ArrayData [#7748](https://github.com/apache/arrow-datafusion/pull/7748) (jayzhan211) +- Minor: Add comment on input_schema from AggregateExec [#7727](https://github.com/apache/arrow-datafusion/pull/7727) (viirya) +- Fix column name for COUNT(\*) set by AggregateStatistics [#7757](https://github.com/apache/arrow-datafusion/pull/7757) (qrilka) +- Add documentation about type signatures, and export `TIMEZONE_WILDCARD` [#7726](https://github.com/apache/arrow-datafusion/pull/7726) (alamb) +- [feat] Support cache ListFiles result cache in session level [#7620](https://github.com/apache/arrow-datafusion/pull/7620) (Ted-Jiang) +- Support `SHOW ALL VERBOSE` to show settings description [#7735](https://github.com/apache/arrow-datafusion/pull/7735) (comphead) diff --git a/dev/release/README.md b/dev/release/README.md index b44259ad560b..53487678aa69 100644 --- a/dev/release/README.md +++ b/dev/release/README.md @@ -82,7 +82,7 @@ You will need a GitHub Personal Access Token for the following steps. Follow [these instructions](https://docs.github.com/en/authentication/keeping-your-account-and-data-secure/creating-a-personal-access-token) to generate one if you do not already have one. -The changelog is generated using a Python script. There is a depency on `PyGitHub`, which can be installed using pip: +The changelog is generated using a Python script. There is a dependency on `PyGitHub`, which can be installed using pip: ```bash pip3 install PyGitHub diff --git a/dev/update_datafusion_versions.py b/dev/update_datafusion_versions.py index d9433915f7e2..7cbe39fdfb66 100755 --- a/dev/update_datafusion_versions.py +++ b/dev/update_datafusion_versions.py @@ -35,10 +35,12 @@ 'datafusion-execution': 'datafusion/execution/Cargo.toml', 'datafusion-optimizer': 'datafusion/optimizer/Cargo.toml', 'datafusion-physical-expr': 'datafusion/physical-expr/Cargo.toml', + 'datafusion-physical-plan': 'datafusion/physical-plan/Cargo.toml', 'datafusion-proto': 'datafusion/proto/Cargo.toml', 'datafusion-substrait': 'datafusion/substrait/Cargo.toml', 'datafusion-sql': 'datafusion/sql/Cargo.toml', 'datafusion-sqllogictest': 'datafusion/sqllogictest/Cargo.toml', + 'datafusion-wasmtest': 'datafusion/wasmtest/Cargo.toml', 'datafusion-benchmarks': 'benchmarks/Cargo.toml', 'datafusion-examples': 'datafusion-examples/Cargo.toml', } diff --git a/docs/source/user-guide/configs.md b/docs/source/user-guide/configs.md index 9eb0862de9c1..cab1e5c3e4a9 100644 --- a/docs/source/user-guide/configs.md +++ b/docs/source/user-guide/configs.md @@ -64,7 +64,7 @@ Environment variables are read during `SessionConfig` initialisation so they mus | datafusion.execution.parquet.statistics_enabled | NULL | Sets if statistics are enabled for any column Valid values are: "none", "chunk", and "page" These values are not case sensitive. If NULL, uses default parquet writer setting | | datafusion.execution.parquet.max_statistics_size | NULL | Sets max statistics size for any column. If NULL, uses default parquet writer setting | | datafusion.execution.parquet.max_row_group_size | 1048576 | Sets maximum number of rows in a row group | -| datafusion.execution.parquet.created_by | datafusion version 31.0.0 | Sets "created by" property | +| datafusion.execution.parquet.created_by | datafusion version 32.0.0 | Sets "created by" property | | datafusion.execution.parquet.column_index_truncate_length | NULL | Sets column index trucate length | | datafusion.execution.parquet.data_page_row_count_limit | 18446744073709551615 | Sets best effort maximum number of rows in data page | | datafusion.execution.parquet.encoding | NULL | 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 | From 5e9dd44edddfbd524b9d2dc7684d9880114a4fba Mon Sep 17 00:00:00 2001 From: Chih Wang Date: Sun, 8 Oct 2023 15:17:41 +0800 Subject: [PATCH 040/572] Minor: Change all file links to GitHub in document (#7768) --- docs/source/library-user-guide/adding-udfs.md | 10 +++++----- docs/source/library-user-guide/catalogs.md | 2 +- docs/source/library-user-guide/working-with-exprs.md | 4 ++-- docs/source/user-guide/example-usage.md | 4 ++-- 4 files changed, 10 insertions(+), 10 deletions(-) diff --git a/docs/source/library-user-guide/adding-udfs.md b/docs/source/library-user-guide/adding-udfs.md index d3f31bd45aee..a4b5ed0b40f1 100644 --- a/docs/source/library-user-guide/adding-udfs.md +++ b/docs/source/library-user-guide/adding-udfs.md @@ -23,11 +23,11 @@ User Defined Functions (UDFs) are functions that can be used in the context of D This page covers how to add UDFs to DataFusion. In particular, it covers how to add Scalar, Window, and Aggregate UDFs. -| UDF Type | Description | Example | -| --------- | ---------------------------------------------------------------------------------------------------------- | ---------------------------------------------------------------------- | -| Scalar | A function that takes a row of data and returns a single value. | [simple_udf.rs](../../../datafusion-examples/examples/simple_udf.rs) | -| Window | A function that takes a row of data and returns a single value, but also has access to the rows around it. | [simple_udwf.rs](../../../datafusion-examples/examples/simple_udwf.rs) | -| Aggregate | A function that takes a group of rows and returns a single value. | [simple_udaf.rs](../../../datafusion-examples/examples/simple_udaf.rs) | +| UDF Type | Description | Example | +| --------- | ---------------------------------------------------------------------------------------------------------- | ------------------------------------------------------------------------------------------------------------------ | +| Scalar | A function that takes a row of data and returns a single value. | [simple_udf.rs](https://github.com/apache/arrow-datafusion/blob/main/datafusion-examples/examples/simple_udf.rs) | +| Window | A function that takes a row of data and returns a single value, but also has access to the rows around it. | [simple_udwf.rs](https://github.com/apache/arrow-datafusion/blob/main/datafusion-examples/examples/simple_udwf.rs) | +| Aggregate | A function that takes a group of rows and returns a single value. | [simple_udaf.rs](https://github.com/apache/arrow-datafusion/blob/main/datafusion-examples/examples/simple_udaf.rs) | First we'll talk about adding an Scalar UDF end-to-end, then we'll talk about the differences between the different types of UDFs. diff --git a/docs/source/library-user-guide/catalogs.md b/docs/source/library-user-guide/catalogs.md index 1dd235f0a2d2..e53d16366350 100644 --- a/docs/source/library-user-guide/catalogs.md +++ b/docs/source/library-user-guide/catalogs.md @@ -19,7 +19,7 @@ # Catalogs, Schemas, and Tables -This section describes how to create and manage catalogs, schemas, and tables in DataFusion. For those wanting to dive into the code quickly please see the [example](../../../datafusion-examples/examples/catalog.rs). +This section describes how to create and manage catalogs, schemas, and tables in DataFusion. For those wanting to dive into the code quickly please see the [example](https://github.com/apache/arrow-datafusion/blob/main/datafusion-examples/examples/catalog.rs). ## General Concepts diff --git a/docs/source/library-user-guide/working-with-exprs.md b/docs/source/library-user-guide/working-with-exprs.md index 507e984acb0b..a8baf24d5f0a 100644 --- a/docs/source/library-user-guide/working-with-exprs.md +++ b/docs/source/library-user-guide/working-with-exprs.md @@ -52,8 +52,8 @@ As the writer of a library, you may want to use or create `Expr`s to represent c There are also executable examples for working with `Expr`s: -- [rewrite_expr.rs](../../../datafusion-examples/examples/catalog.rs) -- [expr_api.rs](../../../datafusion-examples/examples/expr_api.rs) +- [rewrite_expr.rs](https://github.com/apache/arrow-datafusion/blob/main/datafusion-examples/examples/rewrite_expr.rs) +- [expr_api.rs](https://github.com/apache/arrow-datafusion/blob/main/datafusion-examples/examples/expr_api.rs) ## A Scalar UDF Example diff --git a/docs/source/user-guide/example-usage.md b/docs/source/user-guide/example-usage.md index c631d552dd73..a7557f9b0bc3 100644 --- a/docs/source/user-guide/example-usage.md +++ b/docs/source/user-guide/example-usage.md @@ -19,9 +19,9 @@ # Example Usage -In this example some simple processing is performed on the [`example.csv`](../../../datafusion/core/tests/data/example.csv) file. +In this example some simple processing is performed on the [`example.csv`](https://github.com/apache/arrow-datafusion/blob/main/datafusion/core/tests/data/example.csv) file. -Even [`more code examples`](../../../datafusion-examples) attached to the project +Even [`more code examples`](https://github.com/apache/arrow-datafusion/tree/main/datafusion-examples) attached to the project. ## Update `Cargo.toml` From 3d1b23a04bdc04c526e2dcb06e0cf1995707587d Mon Sep 17 00:00:00 2001 From: Andrew Lamb Date: Sun, 8 Oct 2023 03:26:06 -0400 Subject: [PATCH 041/572] Minor: Improve `PruningPredicate` documentation (#7738) * Minor: Improve PruningPredicate documentation * Apply suggestions from code review * fix doc links --- .../core/src/physical_optimizer/pruning.rs | 81 ++++++++++++------- 1 file changed, 53 insertions(+), 28 deletions(-) diff --git a/datafusion/core/src/physical_optimizer/pruning.rs b/datafusion/core/src/physical_optimizer/pruning.rs index 2987ec6d6552..43a7dd8afe21 100644 --- a/datafusion/core/src/physical_optimizer/pruning.rs +++ b/datafusion/core/src/physical_optimizer/pruning.rs @@ -15,19 +15,10 @@ // specific language governing permissions and limitations // under the License. -//! This module contains code to prune "containers" of row groups -//! based on statistics prior to execution. This can lead to -//! significant performance improvements by avoiding the need -//! to evaluate a plan on entire containers (e.g. an entire file) +//! [`PruningPredicate`] to apply filter [`Expr`] to prune "containers" +//! based on statistics (e.g. Parquet Row Groups) //! -//! For example, DataFusion uses this code to prune (skip) row groups -//! while reading parquet files if it can be determined from the -//! predicate that nothing in the row group can match. -//! -//! This code can also be used by other systems to prune other -//! entities (e.g. entire files) if the statistics are known via some -//! other source (e.g. a catalog) - +//! [`Expr`]: crate::prelude::Expr use std::collections::HashSet; use std::convert::TryFrom; use std::sync::Arc; @@ -53,18 +44,21 @@ use datafusion_physical_expr::utils::collect_columns; use datafusion_physical_expr::{expressions as phys_expr, PhysicalExprRef}; use log::trace; -/// Interface to pass statistics information to [`PruningPredicate`] +/// Interface to pass statistics (min/max/nulls) information to [`PruningPredicate`]. /// -/// Returns statistics for containers / files of data in Arrays. +/// Returns statistics for containers / files as Arrow [`ArrayRef`], so the +/// evaluation happens once on a single `RecordBatch`, amortizing the overhead +/// of evaluating of the predicate. This is important when pruning 1000s of +/// containers which often happens in analytic systems. /// -/// For example, for the following three files with a single column +/// For example, for the following three files with a single column `a`: /// ```text /// file1: column a: min=5, max=10 /// file2: column a: No stats /// file2: column a: min=20, max=30 /// ``` /// -/// PruningStatistics should return: +/// PruningStatistics would return: /// /// ```text /// min_values("a") -> Some([5, Null, 20]) @@ -91,10 +85,44 @@ pub trait PruningStatistics { fn null_counts(&self, column: &Column) -> Option; } -/// Evaluates filter expressions on statistics in order to -/// prune data containers (e.g. parquet row group) +/// Evaluates filter expressions on statistics, rather than the actual data. If +/// no rows could possibly pass the filter entire containers can be "pruned" +/// (skipped), without reading any actual data, leading to significant +/// performance improvements. +/// +/// [`PruningPredicate`]s are used to prune (avoid scanning) Parquet Row Groups +/// based on the min/max values found in the Parquet metadata. If the +/// `PruningPredicate` can guarantee that no rows in the Row Group match the +/// filter, the entire Row Group is skipped during query execution. +/// +/// Note that this API is designed to be general, as it works: +/// +/// 1. Arbitrary expressions expressions (including user defined functions) +/// +/// 2. Anything that implements the [`PruningStatistics`] trait, not just +/// Parquet metadata, allowing it to be used by other systems to prune entities +/// (e.g. entire files) if the statistics are known via some other source, such +/// as a catalog. +/// +/// # Example +/// +/// Given an expression like `x = 5` and statistics for 3 containers (Row +/// Groups, files, etc) `A`, `B`, and `C`: +/// +/// ```text +/// A: {x_min = 0, x_max = 4} +/// B: {x_min = 2, x_max = 10} +/// C: {x_min = 5, x_max = 8} +/// ``` +/// +/// Applying the `PruningPredicate` will concludes that `A` can be pruned: /// -/// See [`PruningPredicate::try_new`] for more information. +/// ```text +/// A: false (no rows could possibly match x = 5) +/// B: true (rows might match x = 5) +/// C: true (rows might match x = 5) +/// ``` +/// See [`PruningPredicate::try_new`] and [`PruningPredicate::prune`] for more information. #[derive(Debug, Clone)] pub struct PruningPredicate { /// The input schema against which the predicate will be evaluated @@ -146,17 +174,14 @@ impl PruningPredicate { /// /// `true`: There MAY be rows that match the predicate /// - /// `false`: There are no rows that could match the predicate + /// `false`: There are no rows that could possibly match the predicate /// - /// Note this function takes a slice of statistics as a parameter - /// to amortize the cost of the evaluation of the predicate - /// against a single record batch. - /// - /// Note: the predicate passed to `prune` should be simplified as + /// Note: the predicate passed to `prune` should already be simplified as /// much as possible (e.g. this pass doesn't handle some /// expressions like `b = false`, but it does handle the - /// simplified version `b`. The predicates are simplified via the - /// ConstantFolding optimizer pass + /// simplified version `b`. See [`ExprSimplifier`] to simplify expressions. + /// + /// [`ExprSimplifier`]: crate::optimizer::simplify_expressions::ExprSimplifier pub fn prune(&self, statistics: &S) -> Result> { // build a RecordBatch that contains the min/max values in the // appropriate statistics columns @@ -909,7 +934,7 @@ fn build_statistics_expr( _ => { return plan_err!( "expressions other than (neq, eq, gt, gteq, lt, lteq) are not supported" - ) + ); } }; Ok(statistics_expr) From 258a7cd1cbf20d8f633db6e608649b98dde8b5ce Mon Sep 17 00:00:00 2001 From: Devin D'Angelo Date: Sun, 8 Oct 2023 13:04:35 -0400 Subject: [PATCH 042/572] Support InsertInto Sorted ListingTable (#7743) * enable insert into sorted listing table * add check for single file table * improve test to verify both order conditions * address feedback --- .../core/src/datasource/file_format/csv.rs | 10 +++- .../core/src/datasource/file_format/json.rs | 9 +++- .../core/src/datasource/file_format/mod.rs | 3 +- .../src/datasource/file_format/parquet.rs | 10 +++- .../core/src/datasource/listing/table.rs | 48 +++++++++++++------ datafusion/core/src/datasource/memory.rs | 1 + datafusion/core/src/physical_planner.rs | 2 +- datafusion/physical-plan/src/insert.rs | 19 ++++++-- .../test_files/insert_to_external.slt | 42 ++++++++++++++++ 9 files changed, 117 insertions(+), 27 deletions(-) diff --git a/datafusion/core/src/datasource/file_format/csv.rs b/datafusion/core/src/datasource/file_format/csv.rs index c3295042b5e3..4c625b7ed742 100644 --- a/datafusion/core/src/datasource/file_format/csv.rs +++ b/datafusion/core/src/datasource/file_format/csv.rs @@ -29,7 +29,7 @@ use arrow::{self, datatypes::SchemaRef}; use arrow_array::RecordBatch; use datafusion_common::{exec_err, not_impl_err, DataFusionError, FileType}; use datafusion_execution::TaskContext; -use datafusion_physical_expr::PhysicalExpr; +use datafusion_physical_expr::{PhysicalExpr, PhysicalSortRequirement}; use async_trait::async_trait; use bytes::{Buf, Bytes}; @@ -263,6 +263,7 @@ impl FileFormat for CsvFormat { input: Arc, _state: &SessionState, conf: FileSinkConfig, + order_requirements: Option>, ) -> Result> { if conf.overwrite { return not_impl_err!("Overwrites are not implemented yet for CSV"); @@ -275,7 +276,12 @@ impl FileFormat for CsvFormat { let sink_schema = conf.output_schema().clone(); let sink = Arc::new(CsvSink::new(conf)); - Ok(Arc::new(FileSinkExec::new(input, sink, sink_schema)) as _) + Ok(Arc::new(FileSinkExec::new( + input, + sink, + sink_schema, + order_requirements, + )) as _) } fn file_type(&self) -> FileType { diff --git a/datafusion/core/src/datasource/file_format/json.rs b/datafusion/core/src/datasource/file_format/json.rs index 96fd4daa2da6..6c260b9802ec 100644 --- a/datafusion/core/src/datasource/file_format/json.rs +++ b/datafusion/core/src/datasource/file_format/json.rs @@ -24,6 +24,7 @@ use datafusion_common::not_impl_err; use datafusion_common::DataFusionError; use datafusion_common::FileType; use datafusion_execution::TaskContext; +use datafusion_physical_expr::PhysicalSortRequirement; use rand::distributions::Alphanumeric; use rand::distributions::DistString; use std::fmt; @@ -173,6 +174,7 @@ impl FileFormat for JsonFormat { input: Arc, _state: &SessionState, conf: FileSinkConfig, + order_requirements: Option>, ) -> Result> { if conf.overwrite { return not_impl_err!("Overwrites are not implemented yet for Json"); @@ -184,7 +186,12 @@ impl FileFormat for JsonFormat { let sink_schema = conf.output_schema().clone(); let sink = Arc::new(JsonSink::new(conf, self.file_compression_type)); - Ok(Arc::new(FileSinkExec::new(input, sink, sink_schema)) as _) + Ok(Arc::new(FileSinkExec::new( + input, + sink, + sink_schema, + order_requirements, + )) as _) } fn file_type(&self) -> FileType { diff --git a/datafusion/core/src/datasource/file_format/mod.rs b/datafusion/core/src/datasource/file_format/mod.rs index 86f265ab9492..293f062d86a9 100644 --- a/datafusion/core/src/datasource/file_format/mod.rs +++ b/datafusion/core/src/datasource/file_format/mod.rs @@ -41,7 +41,7 @@ use crate::execution::context::SessionState; use crate::physical_plan::{ExecutionPlan, Statistics}; use datafusion_common::{not_impl_err, DataFusionError, FileType}; -use datafusion_physical_expr::PhysicalExpr; +use datafusion_physical_expr::{PhysicalExpr, PhysicalSortRequirement}; use async_trait::async_trait; use object_store::{ObjectMeta, ObjectStore}; @@ -99,6 +99,7 @@ pub trait FileFormat: Send + Sync + fmt::Debug { _input: Arc, _state: &SessionState, _conf: FileSinkConfig, + _order_requirements: Option>, ) -> Result> { not_impl_err!("Writer not implemented for this format") } diff --git a/datafusion/core/src/datasource/file_format/parquet.rs b/datafusion/core/src/datasource/file_format/parquet.rs index 8ddddab71fd5..062ec1329dc4 100644 --- a/datafusion/core/src/datasource/file_format/parquet.rs +++ b/datafusion/core/src/datasource/file_format/parquet.rs @@ -36,7 +36,7 @@ use async_trait::async_trait; use bytes::{BufMut, BytesMut}; use datafusion_common::{exec_err, not_impl_err, plan_err, DataFusionError, FileType}; use datafusion_execution::TaskContext; -use datafusion_physical_expr::PhysicalExpr; +use datafusion_physical_expr::{PhysicalExpr, PhysicalSortRequirement}; use futures::{StreamExt, TryStreamExt}; use hashbrown::HashMap; use object_store::{ObjectMeta, ObjectStore}; @@ -229,6 +229,7 @@ impl FileFormat for ParquetFormat { input: Arc, _state: &SessionState, conf: FileSinkConfig, + order_requirements: Option>, ) -> Result> { if conf.overwrite { return not_impl_err!("Overwrites are not implemented yet for Parquet"); @@ -237,7 +238,12 @@ impl FileFormat for ParquetFormat { let sink_schema = conf.output_schema().clone(); let sink = Arc::new(ParquetSink::new(conf)); - Ok(Arc::new(FileSinkExec::new(input, sink, sink_schema)) as _) + Ok(Arc::new(FileSinkExec::new( + input, + sink, + sink_schema, + order_requirements, + )) as _) } fn file_type(&self) -> FileType { diff --git a/datafusion/core/src/datasource/listing/table.rs b/datafusion/core/src/datasource/listing/table.rs index 7834de2b19f6..e2696d52333e 100644 --- a/datafusion/core/src/datasource/listing/table.rs +++ b/datafusion/core/src/datasource/listing/table.rs @@ -57,7 +57,9 @@ use datafusion_execution::cache::cache_manager::FileStatisticsCache; use datafusion_execution::cache::cache_unit::DefaultFileStatisticsCache; use datafusion_expr::expr::Sort; use datafusion_optimizer::utils::conjunction; -use datafusion_physical_expr::{create_physical_expr, LexOrdering, PhysicalSortExpr}; +use datafusion_physical_expr::{ + create_physical_expr, LexOrdering, PhysicalSortExpr, PhysicalSortRequirement, +}; use async_trait::async_trait; use futures::{future, stream, StreamExt, TryStreamExt}; @@ -828,19 +830,6 @@ impl TableProvider for ListingTable { ); } - // TODO support inserts to sorted tables which preserve sort_order - // Inserts currently make no effort to preserve sort_order. This could lead to - // incorrect query results on the table after inserting incorrectly sorted data. - let unsorted: Vec> = vec![]; - if self.options.file_sort_order != unsorted { - return Err( - DataFusionError::NotImplemented( - "Writing to a sorted listing table via insert into is not supported yet. \ - To write to this table in the meantime, register an equivalent table with \ - file_sort_order = vec![]".into()) - ); - } - let table_path = &self.table_paths()[0]; // Get the object store for the table path. let store = state.runtime_env().object_store(table_path)?; @@ -911,9 +900,38 @@ impl TableProvider for ListingTable { file_type_writer_options, }; + let unsorted: Vec> = vec![]; + let order_requirements = if self.options().file_sort_order != unsorted { + if matches!( + self.options().insert_mode, + ListingTableInsertMode::AppendToFile + ) { + return Err(DataFusionError::Plan( + "Cannot insert into a sorted ListingTable with mode append!".into(), + )); + } + // Multiple sort orders in outer vec are equivalent, so we pass only the first one + let ordering = self + .try_create_output_ordering()? + .get(0) + .ok_or(DataFusionError::Internal( + "Expected ListingTable to have a sort order, but none found!".into(), + ))? + .clone(); + // Converts Vec> into type required by execution plan to specify its required input ordering + Some( + ordering + .into_iter() + .map(PhysicalSortRequirement::from) + .collect::>(), + ) + } else { + None + }; + self.options() .format - .create_writer_physical_plan(input, state, config) + .create_writer_physical_plan(input, state, config, order_requirements) .await } } diff --git a/datafusion/core/src/datasource/memory.rs b/datafusion/core/src/datasource/memory.rs index 337a8cabc269..6231bd2c2f92 100644 --- a/datafusion/core/src/datasource/memory.rs +++ b/datafusion/core/src/datasource/memory.rs @@ -223,6 +223,7 @@ impl TableProvider for MemTable { input, sink, self.schema.clone(), + None, ))) } } diff --git a/datafusion/core/src/physical_planner.rs b/datafusion/core/src/physical_planner.rs index 84b5b9afa7e7..35119f374fa3 100644 --- a/datafusion/core/src/physical_planner.rs +++ b/datafusion/core/src/physical_planner.rs @@ -604,7 +604,7 @@ impl DefaultPhysicalPlanner { FileType::ARROW => Arc::new(ArrowFormat {}), }; - sink_format.create_writer_physical_plan(input_exec, session_state, config).await + sink_format.create_writer_physical_plan(input_exec, session_state, config, None).await } LogicalPlan::Dml(DmlStatement { table_name, diff --git a/datafusion/physical-plan/src/insert.rs b/datafusion/physical-plan/src/insert.rs index 8b467461ddad..a7b0d32c8eb8 100644 --- a/datafusion/physical-plan/src/insert.rs +++ b/datafusion/physical-plan/src/insert.rs @@ -73,6 +73,8 @@ pub struct FileSinkExec { sink_schema: SchemaRef, /// Schema describing the structure of the output data. count_schema: SchemaRef, + /// Optional required sort order for output data. + sort_order: Option>, } impl fmt::Debug for FileSinkExec { @@ -87,12 +89,14 @@ impl FileSinkExec { input: Arc, sink: Arc, sink_schema: SchemaRef, + sort_order: Option>, ) -> Self { Self { input, sink, sink_schema, count_schema: make_count_schema(), + sort_order, } } @@ -192,16 +196,20 @@ impl ExecutionPlan for FileSinkExec { } fn required_input_ordering(&self) -> Vec>> { - // Require that the InsertExec gets the data in the order the + // The input order is either exlicitly set (such as by a ListingTable), + // or require that the [FileSinkExec] gets the data in the order the // input produced it (otherwise the optimizer may chose to reorder // the input which could result in unintended / poor UX) // // More rationale: // https://github.com/apache/arrow-datafusion/pull/6354#discussion_r1195284178 - vec![self - .input - .output_ordering() - .map(PhysicalSortRequirement::from_sort_exprs)] + match &self.sort_order { + Some(requirements) => vec![Some(requirements.clone())], + None => vec![self + .input + .output_ordering() + .map(PhysicalSortRequirement::from_sort_exprs)], + } } fn maintains_input_order(&self) -> Vec { @@ -221,6 +229,7 @@ impl ExecutionPlan for FileSinkExec { sink: self.sink.clone(), sink_schema: self.sink_schema.clone(), count_schema: self.count_schema.clone(), + sort_order: self.sort_order.clone(), })) } diff --git a/datafusion/sqllogictest/test_files/insert_to_external.slt b/datafusion/sqllogictest/test_files/insert_to_external.slt index a29c230a466e..d1b73204e379 100644 --- a/datafusion/sqllogictest/test_files/insert_to_external.slt +++ b/datafusion/sqllogictest/test_files/insert_to_external.slt @@ -45,6 +45,48 @@ LOCATION '../../testing/data/csv/aggregate_test_100.csv' statement ok set datafusion.execution.target_partitions = 8; +statement ok +CREATE EXTERNAL TABLE +ordered_insert_test(a bigint, b bigint) +STORED AS csv +LOCATION 'test_files/scratch/insert_to_external/insert_to_ordered/' +WITH ORDER (a ASC, B DESC) +OPTIONS( +create_local_path 'true', +insert_mode 'append_new_files', +); + +query TT +EXPLAIN INSERT INTO ordered_insert_test values (5, 1), (4, 2), (7,7), (7,8), (7,9), (7,10), (3, 3), (2, 4), (1, 5); +---- +logical_plan +Dml: op=[Insert Into] table=[ordered_insert_test] +--Projection: column1 AS a, column2 AS b +----Values: (Int64(5), Int64(1)), (Int64(4), Int64(2)), (Int64(7), Int64(7)), (Int64(7), Int64(8)), (Int64(7), Int64(9))... +physical_plan +InsertExec: sink=CsvSink(writer_mode=PutMultipart, file_groups=[]) +--SortExec: expr=[a@0 ASC NULLS LAST,b@1 DESC] +----ProjectionExec: expr=[column1@0 as a, column2@1 as b] +------ValuesExec + +query II +INSERT INTO ordered_insert_test values (5, 1), (4, 2), (7,7), (7,8), (7,9), (7,10), (3, 3), (2, 4), (1, 5); +---- +9 + +query II +SELECT * from ordered_insert_test; +---- +1 5 +2 4 +3 3 +4 2 +5 1 +7 10 +7 9 +7 8 +7 7 + statement ok CREATE EXTERNAL TABLE single_file_test(a bigint, b bigint) From 6685583dfccf56967cf834d55846447263ec148d Mon Sep 17 00:00:00 2001 From: Andrew Lamb Date: Mon, 9 Oct 2023 02:44:23 -0400 Subject: [PATCH 043/572] Minor: improve documentation to `stagger_batch` (#7754) --- test-utils/src/lib.rs | 14 ++++++++++++-- 1 file changed, 12 insertions(+), 2 deletions(-) diff --git a/test-utils/src/lib.rs b/test-utils/src/lib.rs index dfd878275181..e3c96d16eeb9 100644 --- a/test-utils/src/lib.rs +++ b/test-utils/src/lib.rs @@ -70,13 +70,23 @@ pub fn add_empty_batches( } /// "stagger" batches: split the batches into random sized batches +/// +/// For example, if the input batch has 1000 rows, [`stagger_batch`] might return +/// multiple batches +/// ```text +/// [ +/// RecordBatch(123 rows), +/// RecordBatch(234 rows), +/// RecordBatch(634 rows), +/// ] +/// ``` pub fn stagger_batch(batch: RecordBatch) -> Vec { let seed = 42; stagger_batch_with_seed(batch, seed) } -/// "stagger" batches: split the batches into random sized batches -/// using the specified value for a rng seed +/// "stagger" batches: split the batches into random sized batches using the +/// specified value for a rng seed. See [`stagger_batch`] for more detail. pub fn stagger_batch_with_seed(batch: RecordBatch, seed: u64) -> Vec { let mut batches = vec![]; From 68fb90a9ef5c1f4779c4406507b4a99a663c4890 Mon Sep 17 00:00:00 2001 From: Mustafa Akur <106137913+mustafasrepo@users.noreply.github.com> Date: Mon, 9 Oct 2023 15:52:38 +0300 Subject: [PATCH 044/572] External Table Primary key support (#7755) * Initial commit * Add primary key external table support * Minor changes * Resolve proto errors * Minor changes * Review * Remove option from constraints * Add test for unsupported foreign key syntax * Add new round-trip tests * Minor changes * Simplify inline constraints helper functions --------- Co-authored-by: Mehmet Ozan Kabak --- .../common/src/functional_dependencies.rs | 32 +- datafusion/common/src/lib.rs | 31 +- datafusion/core/src/catalog/listing_schema.rs | 17 +- .../core/src/datasource/listing/table.rs | 33 +- .../src/datasource/listing_table_factory.rs | 37 +- datafusion/core/src/datasource/memory.rs | 10 +- datafusion/core/src/datasource/provider.rs | 5 + datafusion/expr/src/logical_plan/ddl.rs | 5 +- datafusion/proto/proto/datafusion.proto | 20 + datafusion/proto/src/generated/pbjson.rs | 406 ++++++++++++++++++ datafusion/proto/src/generated/prost.rs | 37 ++ .../proto/src/logical_plan/from_proto.rs | 31 +- datafusion/proto/src/logical_plan/mod.rs | 32 +- datafusion/proto/src/logical_plan/to_proto.rs | 33 +- .../tests/cases/roundtrip_logical_plan.rs | 88 ++++ datafusion/sql/src/parser.rs | 22 +- datafusion/sql/src/statement.rs | 95 ++-- datafusion/sql/tests/sql_integration.rs | 8 + .../sqllogictest/test_files/groupby.slt | 127 ++++++ 19 files changed, 963 insertions(+), 106 deletions(-) diff --git a/datafusion/common/src/functional_dependencies.rs b/datafusion/common/src/functional_dependencies.rs index 6a08c4fd3589..324374f557b4 100644 --- a/datafusion/common/src/functional_dependencies.rs +++ b/datafusion/common/src/functional_dependencies.rs @@ -18,11 +18,14 @@ //! FunctionalDependencies keeps track of functional dependencies //! inside DFSchema. -use crate::{DFSchema, DFSchemaRef, DataFusionError, JoinType, Result}; -use sqlparser::ast::TableConstraint; use std::collections::HashSet; use std::fmt::{Display, Formatter}; use std::ops::Deref; +use std::vec::IntoIter; + +use crate::{DFSchema, DFSchemaRef, DataFusionError, JoinType, Result}; + +use sqlparser::ast::TableConstraint; /// This object defines a constraint on a table. #[derive(Debug, Clone, PartialEq, Eq, Hash)] @@ -43,13 +46,15 @@ pub struct Constraints { impl Constraints { /// Create empty constraints pub fn empty() -> Self { - Constraints::new(vec![]) + Constraints::new_unverified(vec![]) } - // This method is private. - // Outside callers can either create empty constraint using `Constraints::empty` API. - // or create constraint from table constraints using `Constraints::new_from_table_constraints` API. - fn new(constraints: Vec) -> Self { + /// Create a new `Constraints` object from the given `constraints`. + /// Users should use the `empty` or `new_from_table_constraints` functions + /// for constructing `Constraints`. This constructor is for internal + /// purposes only and does not check whether the argument is valid. The user + /// is responsible for supplying a valid vector of `Constraint` objects. + pub fn new_unverified(constraints: Vec) -> Self { Self { inner: constraints } } @@ -104,7 +109,7 @@ impl Constraints { )), }) .collect::>>()?; - Ok(Constraints::new(constraints)) + Ok(Constraints::new_unverified(constraints)) } /// Check whether constraints is empty @@ -113,6 +118,15 @@ impl Constraints { } } +impl IntoIterator for Constraints { + type Item = Constraint; + type IntoIter = IntoIter; + + fn into_iter(self) -> Self::IntoIter { + self.inner.into_iter() + } +} + impl Display for Constraints { fn fmt(&self, f: &mut Formatter<'_>) -> std::fmt::Result { let pk: Vec = self.inner.iter().map(|c| format!("{:?}", c)).collect(); @@ -534,7 +548,7 @@ mod tests { #[test] fn constraints_iter() { - let constraints = Constraints::new(vec![ + let constraints = Constraints::new_unverified(vec![ Constraint::PrimaryKey(vec![10]), Constraint::Unique(vec![20]), ]); diff --git a/datafusion/common/src/lib.rs b/datafusion/common/src/lib.rs index 71782f67046d..a939cf73dc9c 100644 --- a/datafusion/common/src/lib.rs +++ b/datafusion/common/src/lib.rs @@ -15,45 +15,47 @@ // specific language governing permissions and limitations // under the License. +mod column; +mod dfschema; +mod error; +mod functional_dependencies; +mod join_type; +#[cfg(feature = "pyarrow")] +mod pyarrow; +mod schema_reference; +mod table_reference; +mod unnest; + pub mod alias; pub mod cast; -mod column; pub mod config; -mod dfschema; pub mod display; -mod error; pub mod file_options; pub mod format; -mod functional_dependencies; pub mod hash_utils; -mod join_type; pub mod parsers; -#[cfg(feature = "pyarrow")] -mod pyarrow; pub mod scalar; -mod schema_reference; pub mod stats; -mod table_reference; pub mod test_util; pub mod tree_node; -mod unnest; pub mod utils; +/// Reexport arrow crate +pub use arrow; pub use column::Column; pub use dfschema::{DFField, DFSchema, DFSchemaRef, ExprSchema, SchemaExt, ToDFSchema}; pub use error::{ field_not_found, unqualified_field_not_found, DataFusionError, Result, SchemaError, SharedResult, }; - pub use file_options::file_type::{ FileType, GetExt, DEFAULT_ARROW_EXTENSION, DEFAULT_AVRO_EXTENSION, DEFAULT_CSV_EXTENSION, DEFAULT_JSON_EXTENSION, DEFAULT_PARQUET_EXTENSION, }; pub use file_options::FileTypeWriterOptions; pub use functional_dependencies::{ - aggregate_functional_dependencies, get_target_functional_dependencies, Constraints, - Dependency, FunctionalDependence, FunctionalDependencies, + aggregate_functional_dependencies, get_target_functional_dependencies, Constraint, + Constraints, Dependency, FunctionalDependence, FunctionalDependencies, }; pub use join_type::{JoinConstraint, JoinType}; pub use scalar::{ScalarType, ScalarValue}; @@ -63,9 +65,6 @@ pub use table_reference::{OwnedTableReference, ResolvedTableReference, TableRefe pub use unnest::UnnestOptions; pub use utils::project_schema; -/// Reexport arrow crate -pub use arrow; - /// Downcast an Arrow Array to a concrete type, return an `DataFusionError::Internal` if the cast is /// not possible. In normal usage of DataFusion the downcast should always succeed. /// diff --git a/datafusion/core/src/catalog/listing_schema.rs b/datafusion/core/src/catalog/listing_schema.rs index e7b4d8dec03c..7e527642be16 100644 --- a/datafusion/core/src/catalog/listing_schema.rs +++ b/datafusion/core/src/catalog/listing_schema.rs @@ -16,21 +16,25 @@ // under the License. //! listing_schema contains a SchemaProvider that scans ObjectStores for tables automatically + +use std::any::Any; +use std::collections::{HashMap, HashSet}; +use std::path::Path; +use std::sync::{Arc, Mutex}; + use crate::catalog::schema::SchemaProvider; use crate::datasource::provider::TableProviderFactory; use crate::datasource::TableProvider; use crate::execution::context::SessionState; -use async_trait::async_trait; + use datafusion_common::parsers::CompressionTypeVariant; -use datafusion_common::{DFSchema, DataFusionError, OwnedTableReference}; +use datafusion_common::{Constraints, DFSchema, DataFusionError, OwnedTableReference}; use datafusion_expr::CreateExternalTable; + +use async_trait::async_trait; use futures::TryStreamExt; use itertools::Itertools; use object_store::ObjectStore; -use std::any::Any; -use std::collections::{HashMap, HashSet}; -use std::path::Path; -use std::sync::{Arc, Mutex}; /// A [`SchemaProvider`] that scans an [`ObjectStore`] to automatically discover tables /// @@ -149,6 +153,7 @@ impl ListingSchemaProvider { order_exprs: vec![], unbounded: false, options: Default::default(), + constraints: Constraints::empty(), }, ) .await?; diff --git a/datafusion/core/src/datasource/listing/table.rs b/datafusion/core/src/datasource/listing/table.rs index e2696d52333e..b6870ad2a827 100644 --- a/datafusion/core/src/datasource/listing/table.rs +++ b/datafusion/core/src/datasource/listing/table.rs @@ -23,19 +23,19 @@ use std::{any::Any, sync::Arc}; use super::helpers::{expr_applicable_for_cols, pruned_partition_list, split_files}; use super::PartitionedFile; -use crate::datasource::file_format::file_compression_type::{ - FileCompressionType, FileTypeExt, -}; -use crate::datasource::physical_plan::{ - is_plan_streaming, FileScanConfig, FileSinkConfig, -}; use crate::datasource::{ file_format::{ - arrow::ArrowFormat, avro::AvroFormat, csv::CsvFormat, json::JsonFormat, - parquet::ParquetFormat, FileFormat, + arrow::ArrowFormat, + avro::AvroFormat, + csv::CsvFormat, + file_compression_type::{FileCompressionType, FileTypeExt}, + json::JsonFormat, + parquet::ParquetFormat, + FileFormat, }, get_statistics_with_limit, listing::ListingTableUrl, + physical_plan::{is_plan_streaming, FileScanConfig, FileSinkConfig}, TableProvider, TableType, }; use crate::logical_expr::TableProviderFilterPushDown; @@ -46,12 +46,13 @@ use crate::{ logical_expr::Expr, physical_plan::{empty::EmptyExec, ExecutionPlan, Statistics}, }; + use arrow::compute::SortOptions; use arrow::datatypes::{DataType, Field, SchemaBuilder, SchemaRef}; use arrow_schema::Schema; use datafusion_common::{ - internal_err, plan_err, project_schema, FileType, FileTypeWriterOptions, SchemaExt, - ToDFSchema, + internal_err, plan_err, project_schema, Constraints, FileType, FileTypeWriterOptions, + SchemaExt, ToDFSchema, }; use datafusion_execution::cache::cache_manager::FileStatisticsCache; use datafusion_execution::cache::cache_unit::DefaultFileStatisticsCache; @@ -594,6 +595,7 @@ pub struct ListingTable { definition: Option, collected_statistics: FileStatisticsCache, infinite_source: bool, + constraints: Constraints, } impl ListingTable { @@ -631,11 +633,18 @@ impl ListingTable { definition: None, collected_statistics: Arc::new(DefaultFileStatisticsCache::default()), infinite_source, + constraints: Constraints::empty(), }; Ok(table) } + /// Assign constraints + pub fn with_constraints(mut self, constraints: Constraints) -> Self { + self.constraints = constraints; + self + } + /// Set the [`FileStatisticsCache`] used to cache parquet file statistics. /// /// Setting a statistics cache on the `SessionContext` can avoid refetching statistics @@ -707,6 +716,10 @@ impl TableProvider for ListingTable { Arc::clone(&self.table_schema) } + fn constraints(&self) -> Option<&Constraints> { + Some(&self.constraints) + } + fn table_type(&self) -> TableType { TableType::Base } diff --git a/datafusion/core/src/datasource/listing_table_factory.rs b/datafusion/core/src/datasource/listing_table_factory.rs index 9c438a47943f..ebfb589f179e 100644 --- a/datafusion/core/src/datasource/listing_table_factory.rs +++ b/datafusion/core/src/datasource/listing_table_factory.rs @@ -21,28 +21,26 @@ use std::path::Path; use std::str::FromStr; use std::sync::Arc; -use arrow::datatypes::{DataType, SchemaRef}; -use async_trait::async_trait; -use datafusion_common::file_options::{FileTypeWriterOptions, StatementOptions}; -use datafusion_common::DataFusionError; -use datafusion_expr::CreateExternalTable; +use super::listing::ListingTableInsertMode; -use crate::datasource::file_format::arrow::ArrowFormat; -use crate::datasource::file_format::avro::AvroFormat; -use crate::datasource::file_format::csv::CsvFormat; -use crate::datasource::file_format::file_compression_type::FileCompressionType; -use crate::datasource::file_format::json::JsonFormat; -use crate::datasource::file_format::parquet::ParquetFormat; -use crate::datasource::file_format::FileFormat; +use crate::datasource::file_format::{ + arrow::ArrowFormat, avro::AvroFormat, csv::CsvFormat, + file_compression_type::FileCompressionType, json::JsonFormat, parquet::ParquetFormat, + FileFormat, +}; use crate::datasource::listing::{ ListingOptions, ListingTable, ListingTableConfig, ListingTableUrl, }; use crate::datasource::provider::TableProviderFactory; use crate::datasource::TableProvider; use crate::execution::context::SessionState; -use datafusion_common::FileType; -use super::listing::ListingTableInsertMode; +use arrow::datatypes::{DataType, SchemaRef}; +use datafusion_common::file_options::{FileTypeWriterOptions, StatementOptions}; +use datafusion_common::{DataFusionError, FileType}; +use datafusion_expr::CreateExternalTable; + +use async_trait::async_trait; /// A `TableProviderFactory` capable of creating new `ListingTable`s pub struct ListingTableFactory {} @@ -232,7 +230,9 @@ impl TableProviderFactory for ListingTableFactory { .with_schema(resolved_schema); let provider = ListingTable::try_new(config)? .with_cache(state.runtime_env().cache_manager.get_file_statistic_cache()); - let table = provider.with_definition(cmd.definition.clone()); + let table = provider + .with_definition(cmd.definition.clone()) + .with_constraints(cmd.constraints.clone()); Ok(Arc::new(table)) } } @@ -248,13 +248,13 @@ fn get_extension(path: &str) -> String { #[cfg(test)] mod tests { - use super::*; - use std::collections::HashMap; + use super::*; use crate::execution::context::SessionContext; + use datafusion_common::parsers::CompressionTypeVariant; - use datafusion_common::{DFSchema, OwnedTableReference}; + use datafusion_common::{Constraints, DFSchema, OwnedTableReference}; #[tokio::test] async fn test_create_using_non_std_file_ext() { @@ -282,6 +282,7 @@ mod tests { order_exprs: vec![], unbounded: false, options: HashMap::new(), + constraints: Constraints::empty(), }; let table_provider = factory.create(&state, &cmd).await.unwrap(); let listing_table = table_provider diff --git a/datafusion/core/src/datasource/memory.rs b/datafusion/core/src/datasource/memory.rs index 6231bd2c2f92..2766e73d3319 100644 --- a/datafusion/core/src/datasource/memory.rs +++ b/datafusion/core/src/datasource/memory.rs @@ -54,7 +54,7 @@ pub type PartitionData = Arc>>; pub struct MemTable { schema: SchemaRef, pub(crate) batches: Vec, - constraints: Option, + constraints: Constraints, } impl MemTable { @@ -77,15 +77,13 @@ impl MemTable { .into_iter() .map(|e| Arc::new(RwLock::new(e))) .collect::>(), - constraints: None, + constraints: Constraints::empty(), }) } /// Assign constraints pub fn with_constraints(mut self, constraints: Constraints) -> Self { - if !constraints.is_empty() { - self.constraints = Some(constraints); - } + self.constraints = constraints; self } @@ -164,7 +162,7 @@ impl TableProvider for MemTable { } fn constraints(&self) -> Option<&Constraints> { - self.constraints.as_ref() + Some(&self.constraints) } fn table_type(&self) -> TableType { diff --git a/datafusion/core/src/datasource/provider.rs b/datafusion/core/src/datasource/provider.rs index af99c12d39c9..7d9f9e86d603 100644 --- a/datafusion/core/src/datasource/provider.rs +++ b/datafusion/core/src/datasource/provider.rs @@ -42,6 +42,11 @@ pub trait TableProvider: Sync + Send { fn schema(&self) -> SchemaRef; /// Get a reference to the constraints of the table. + /// Returns: + /// - `None` for tables that do not support constraints. + /// - `Some(&Constraints)` for tables supporting constraints. + /// Therefore, a `Some(&Constraints::empty())` return value indicates that + /// this table supports constraints, but there are no constraints. fn constraints(&self) -> Option<&Constraints> { None } diff --git a/datafusion/expr/src/logical_plan/ddl.rs b/datafusion/expr/src/logical_plan/ddl.rs index dc247da3642c..2c90a3aca754 100644 --- a/datafusion/expr/src/logical_plan/ddl.rs +++ b/datafusion/expr/src/logical_plan/ddl.rs @@ -112,9 +112,10 @@ impl DdlStatement { match self.0 { DdlStatement::CreateExternalTable(CreateExternalTable { ref name, + constraints, .. }) => { - write!(f, "CreateExternalTable: {name:?}") + write!(f, "CreateExternalTable: {name:?}{constraints}") } DdlStatement::CreateMemoryTable(CreateMemoryTable { name, @@ -191,6 +192,8 @@ pub struct CreateExternalTable { pub unbounded: bool, /// Table(provider) specific options pub options: HashMap, + /// The list of constraints in the schema, such as primary key, unique, etc. + pub constraints: Constraints, } // Hashing refers to a subset of fields considered in PartialEq. diff --git a/datafusion/proto/proto/datafusion.proto b/datafusion/proto/proto/datafusion.proto index 0ebcf2537dda..bda0f7828726 100644 --- a/datafusion/proto/proto/datafusion.proto +++ b/datafusion/proto/proto/datafusion.proto @@ -180,6 +180,25 @@ message EmptyRelationNode { bool produce_one_row = 1; } +message PrimaryKeyConstraint{ + repeated uint64 indices = 1; +} + +message UniqueConstraint{ + repeated uint64 indices = 1; +} + +message Constraint{ + oneof constraint_mode{ + PrimaryKeyConstraint primary_key = 1; + UniqueConstraint unique = 2; + } +} + +message Constraints{ + repeated Constraint constraints = 1; +} + message CreateExternalTableNode { reserved 1; // was string name OwnedTableReference name = 12; @@ -195,6 +214,7 @@ message CreateExternalTableNode { repeated LogicalExprNodeCollection order_exprs = 13; bool unbounded = 14; map options = 11; + Constraints constraints = 15; } message PrepareNode { diff --git a/datafusion/proto/src/generated/pbjson.rs b/datafusion/proto/src/generated/pbjson.rs index d1e9e886e7d5..ced0c8bd7c7a 100644 --- a/datafusion/proto/src/generated/pbjson.rs +++ b/datafusion/proto/src/generated/pbjson.rs @@ -3425,6 +3425,207 @@ impl<'de> serde::Deserialize<'de> for ColumnStats { deserializer.deserialize_struct("datafusion.ColumnStats", FIELDS, GeneratedVisitor) } } +impl serde::Serialize for Constraint { + #[allow(deprecated)] + fn serialize(&self, serializer: S) -> std::result::Result + where + S: serde::Serializer, + { + use serde::ser::SerializeStruct; + let mut len = 0; + if self.constraint_mode.is_some() { + len += 1; + } + let mut struct_ser = serializer.serialize_struct("datafusion.Constraint", len)?; + if let Some(v) = self.constraint_mode.as_ref() { + match v { + constraint::ConstraintMode::PrimaryKey(v) => { + struct_ser.serialize_field("primaryKey", v)?; + } + constraint::ConstraintMode::Unique(v) => { + struct_ser.serialize_field("unique", v)?; + } + } + } + struct_ser.end() + } +} +impl<'de> serde::Deserialize<'de> for Constraint { + #[allow(deprecated)] + fn deserialize(deserializer: D) -> std::result::Result + where + D: serde::Deserializer<'de>, + { + const FIELDS: &[&str] = &[ + "primary_key", + "primaryKey", + "unique", + ]; + + #[allow(clippy::enum_variant_names)] + enum GeneratedField { + PrimaryKey, + Unique, + } + impl<'de> serde::Deserialize<'de> for GeneratedField { + fn deserialize(deserializer: D) -> std::result::Result + where + D: serde::Deserializer<'de>, + { + struct GeneratedVisitor; + + impl<'de> serde::de::Visitor<'de> for GeneratedVisitor { + type Value = GeneratedField; + + fn expecting(&self, formatter: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + write!(formatter, "expected one of: {:?}", &FIELDS) + } + + #[allow(unused_variables)] + fn visit_str(self, value: &str) -> std::result::Result + where + E: serde::de::Error, + { + match value { + "primaryKey" | "primary_key" => Ok(GeneratedField::PrimaryKey), + "unique" => Ok(GeneratedField::Unique), + _ => Err(serde::de::Error::unknown_field(value, FIELDS)), + } + } + } + deserializer.deserialize_identifier(GeneratedVisitor) + } + } + struct GeneratedVisitor; + impl<'de> serde::de::Visitor<'de> for GeneratedVisitor { + type Value = Constraint; + + fn expecting(&self, formatter: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + formatter.write_str("struct datafusion.Constraint") + } + + fn visit_map(self, mut map_: V) -> std::result::Result + where + V: serde::de::MapAccess<'de>, + { + let mut constraint_mode__ = None; + while let Some(k) = map_.next_key()? { + match k { + GeneratedField::PrimaryKey => { + if constraint_mode__.is_some() { + return Err(serde::de::Error::duplicate_field("primaryKey")); + } + constraint_mode__ = map_.next_value::<::std::option::Option<_>>()?.map(constraint::ConstraintMode::PrimaryKey) +; + } + GeneratedField::Unique => { + if constraint_mode__.is_some() { + return Err(serde::de::Error::duplicate_field("unique")); + } + constraint_mode__ = map_.next_value::<::std::option::Option<_>>()?.map(constraint::ConstraintMode::Unique) +; + } + } + } + Ok(Constraint { + constraint_mode: constraint_mode__, + }) + } + } + deserializer.deserialize_struct("datafusion.Constraint", FIELDS, GeneratedVisitor) + } +} +impl serde::Serialize for Constraints { + #[allow(deprecated)] + fn serialize(&self, serializer: S) -> std::result::Result + where + S: serde::Serializer, + { + use serde::ser::SerializeStruct; + let mut len = 0; + if !self.constraints.is_empty() { + len += 1; + } + let mut struct_ser = serializer.serialize_struct("datafusion.Constraints", len)?; + if !self.constraints.is_empty() { + struct_ser.serialize_field("constraints", &self.constraints)?; + } + struct_ser.end() + } +} +impl<'de> serde::Deserialize<'de> for Constraints { + #[allow(deprecated)] + fn deserialize(deserializer: D) -> std::result::Result + where + D: serde::Deserializer<'de>, + { + const FIELDS: &[&str] = &[ + "constraints", + ]; + + #[allow(clippy::enum_variant_names)] + enum GeneratedField { + Constraints, + } + impl<'de> serde::Deserialize<'de> for GeneratedField { + fn deserialize(deserializer: D) -> std::result::Result + where + D: serde::Deserializer<'de>, + { + struct GeneratedVisitor; + + impl<'de> serde::de::Visitor<'de> for GeneratedVisitor { + type Value = GeneratedField; + + fn expecting(&self, formatter: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + write!(formatter, "expected one of: {:?}", &FIELDS) + } + + #[allow(unused_variables)] + fn visit_str(self, value: &str) -> std::result::Result + where + E: serde::de::Error, + { + match value { + "constraints" => Ok(GeneratedField::Constraints), + _ => Err(serde::de::Error::unknown_field(value, FIELDS)), + } + } + } + deserializer.deserialize_identifier(GeneratedVisitor) + } + } + struct GeneratedVisitor; + impl<'de> serde::de::Visitor<'de> for GeneratedVisitor { + type Value = Constraints; + + fn expecting(&self, formatter: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + formatter.write_str("struct datafusion.Constraints") + } + + fn visit_map(self, mut map_: V) -> std::result::Result + where + V: serde::de::MapAccess<'de>, + { + let mut constraints__ = None; + while let Some(k) = map_.next_key()? { + match k { + GeneratedField::Constraints => { + if constraints__.is_some() { + return Err(serde::de::Error::duplicate_field("constraints")); + } + constraints__ = Some(map_.next_value()?); + } + } + } + Ok(Constraints { + constraints: constraints__.unwrap_or_default(), + }) + } + } + deserializer.deserialize_struct("datafusion.Constraints", FIELDS, GeneratedVisitor) + } +} impl serde::Serialize for CreateCatalogNode { #[allow(deprecated)] fn serialize(&self, serializer: S) -> std::result::Result @@ -3726,6 +3927,9 @@ impl serde::Serialize for CreateExternalTableNode { if !self.options.is_empty() { len += 1; } + if self.constraints.is_some() { + len += 1; + } let mut struct_ser = serializer.serialize_struct("datafusion.CreateExternalTableNode", len)?; if let Some(v) = self.name.as_ref() { struct_ser.serialize_field("name", v)?; @@ -3766,6 +3970,9 @@ impl serde::Serialize for CreateExternalTableNode { if !self.options.is_empty() { struct_ser.serialize_field("options", &self.options)?; } + if let Some(v) = self.constraints.as_ref() { + struct_ser.serialize_field("constraints", v)?; + } struct_ser.end() } } @@ -3795,6 +4002,7 @@ impl<'de> serde::Deserialize<'de> for CreateExternalTableNode { "orderExprs", "unbounded", "options", + "constraints", ]; #[allow(clippy::enum_variant_names)] @@ -3812,6 +4020,7 @@ impl<'de> serde::Deserialize<'de> for CreateExternalTableNode { OrderExprs, Unbounded, Options, + Constraints, } impl<'de> serde::Deserialize<'de> for GeneratedField { fn deserialize(deserializer: D) -> std::result::Result @@ -3846,6 +4055,7 @@ impl<'de> serde::Deserialize<'de> for CreateExternalTableNode { "orderExprs" | "order_exprs" => Ok(GeneratedField::OrderExprs), "unbounded" => Ok(GeneratedField::Unbounded), "options" => Ok(GeneratedField::Options), + "constraints" => Ok(GeneratedField::Constraints), _ => Err(serde::de::Error::unknown_field(value, FIELDS)), } } @@ -3878,6 +4088,7 @@ impl<'de> serde::Deserialize<'de> for CreateExternalTableNode { let mut order_exprs__ = None; let mut unbounded__ = None; let mut options__ = None; + let mut constraints__ = None; while let Some(k) = map_.next_key()? { match k { GeneratedField::Name => { @@ -3960,6 +4171,12 @@ impl<'de> serde::Deserialize<'de> for CreateExternalTableNode { map_.next_value::>()? ); } + GeneratedField::Constraints => { + if constraints__.is_some() { + return Err(serde::de::Error::duplicate_field("constraints")); + } + constraints__ = map_.next_value()?; + } } } Ok(CreateExternalTableNode { @@ -3976,6 +4193,7 @@ impl<'de> serde::Deserialize<'de> for CreateExternalTableNode { order_exprs: order_exprs__.unwrap_or_default(), unbounded: unbounded__.unwrap_or_default(), options: options__.unwrap_or_default(), + constraints: constraints__, }) } } @@ -18228,6 +18446,100 @@ impl<'de> serde::Deserialize<'de> for PrepareNode { deserializer.deserialize_struct("datafusion.PrepareNode", FIELDS, GeneratedVisitor) } } +impl serde::Serialize for PrimaryKeyConstraint { + #[allow(deprecated)] + fn serialize(&self, serializer: S) -> std::result::Result + where + S: serde::Serializer, + { + use serde::ser::SerializeStruct; + let mut len = 0; + if !self.indices.is_empty() { + len += 1; + } + let mut struct_ser = serializer.serialize_struct("datafusion.PrimaryKeyConstraint", len)?; + if !self.indices.is_empty() { + struct_ser.serialize_field("indices", &self.indices.iter().map(ToString::to_string).collect::>())?; + } + struct_ser.end() + } +} +impl<'de> serde::Deserialize<'de> for PrimaryKeyConstraint { + #[allow(deprecated)] + fn deserialize(deserializer: D) -> std::result::Result + where + D: serde::Deserializer<'de>, + { + const FIELDS: &[&str] = &[ + "indices", + ]; + + #[allow(clippy::enum_variant_names)] + enum GeneratedField { + Indices, + } + impl<'de> serde::Deserialize<'de> for GeneratedField { + fn deserialize(deserializer: D) -> std::result::Result + where + D: serde::Deserializer<'de>, + { + struct GeneratedVisitor; + + impl<'de> serde::de::Visitor<'de> for GeneratedVisitor { + type Value = GeneratedField; + + fn expecting(&self, formatter: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + write!(formatter, "expected one of: {:?}", &FIELDS) + } + + #[allow(unused_variables)] + fn visit_str(self, value: &str) -> std::result::Result + where + E: serde::de::Error, + { + match value { + "indices" => Ok(GeneratedField::Indices), + _ => Err(serde::de::Error::unknown_field(value, FIELDS)), + } + } + } + deserializer.deserialize_identifier(GeneratedVisitor) + } + } + struct GeneratedVisitor; + impl<'de> serde::de::Visitor<'de> for GeneratedVisitor { + type Value = PrimaryKeyConstraint; + + fn expecting(&self, formatter: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + formatter.write_str("struct datafusion.PrimaryKeyConstraint") + } + + fn visit_map(self, mut map_: V) -> std::result::Result + where + V: serde::de::MapAccess<'de>, + { + let mut indices__ = None; + while let Some(k) = map_.next_key()? { + match k { + GeneratedField::Indices => { + if indices__.is_some() { + return Err(serde::de::Error::duplicate_field("indices")); + } + indices__ = + Some(map_.next_value::>>()? + .into_iter().map(|x| x.0).collect()) + ; + } + } + } + Ok(PrimaryKeyConstraint { + indices: indices__.unwrap_or_default(), + }) + } + } + deserializer.deserialize_struct("datafusion.PrimaryKeyConstraint", FIELDS, GeneratedVisitor) + } +} impl serde::Serialize for ProjectionColumns { #[allow(deprecated)] fn serialize(&self, serializer: S) -> std::result::Result @@ -23166,6 +23478,100 @@ impl<'de> serde::Deserialize<'de> for UnionNode { deserializer.deserialize_struct("datafusion.UnionNode", FIELDS, GeneratedVisitor) } } +impl serde::Serialize for UniqueConstraint { + #[allow(deprecated)] + fn serialize(&self, serializer: S) -> std::result::Result + where + S: serde::Serializer, + { + use serde::ser::SerializeStruct; + let mut len = 0; + if !self.indices.is_empty() { + len += 1; + } + let mut struct_ser = serializer.serialize_struct("datafusion.UniqueConstraint", len)?; + if !self.indices.is_empty() { + struct_ser.serialize_field("indices", &self.indices.iter().map(ToString::to_string).collect::>())?; + } + struct_ser.end() + } +} +impl<'de> serde::Deserialize<'de> for UniqueConstraint { + #[allow(deprecated)] + fn deserialize(deserializer: D) -> std::result::Result + where + D: serde::Deserializer<'de>, + { + const FIELDS: &[&str] = &[ + "indices", + ]; + + #[allow(clippy::enum_variant_names)] + enum GeneratedField { + Indices, + } + impl<'de> serde::Deserialize<'de> for GeneratedField { + fn deserialize(deserializer: D) -> std::result::Result + where + D: serde::Deserializer<'de>, + { + struct GeneratedVisitor; + + impl<'de> serde::de::Visitor<'de> for GeneratedVisitor { + type Value = GeneratedField; + + fn expecting(&self, formatter: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + write!(formatter, "expected one of: {:?}", &FIELDS) + } + + #[allow(unused_variables)] + fn visit_str(self, value: &str) -> std::result::Result + where + E: serde::de::Error, + { + match value { + "indices" => Ok(GeneratedField::Indices), + _ => Err(serde::de::Error::unknown_field(value, FIELDS)), + } + } + } + deserializer.deserialize_identifier(GeneratedVisitor) + } + } + struct GeneratedVisitor; + impl<'de> serde::de::Visitor<'de> for GeneratedVisitor { + type Value = UniqueConstraint; + + fn expecting(&self, formatter: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + formatter.write_str("struct datafusion.UniqueConstraint") + } + + fn visit_map(self, mut map_: V) -> std::result::Result + where + V: serde::de::MapAccess<'de>, + { + let mut indices__ = None; + while let Some(k) = map_.next_key()? { + match k { + GeneratedField::Indices => { + if indices__.is_some() { + return Err(serde::de::Error::duplicate_field("indices")); + } + indices__ = + Some(map_.next_value::>>()? + .into_iter().map(|x| x.0).collect()) + ; + } + } + } + Ok(UniqueConstraint { + indices: indices__.unwrap_or_default(), + }) + } + } + deserializer.deserialize_struct("datafusion.UniqueConstraint", FIELDS, GeneratedVisitor) + } +} impl serde::Serialize for ValuesNode { #[allow(deprecated)] fn serialize(&self, serializer: S) -> std::result::Result diff --git a/datafusion/proto/src/generated/prost.rs b/datafusion/proto/src/generated/prost.rs index 3382fa17fe58..ca20cd35cb55 100644 --- a/datafusion/proto/src/generated/prost.rs +++ b/datafusion/proto/src/generated/prost.rs @@ -291,6 +291,41 @@ pub struct EmptyRelationNode { } #[allow(clippy::derive_partial_eq_without_eq)] #[derive(Clone, PartialEq, ::prost::Message)] +pub struct PrimaryKeyConstraint { + #[prost(uint64, repeated, tag = "1")] + pub indices: ::prost::alloc::vec::Vec, +} +#[allow(clippy::derive_partial_eq_without_eq)] +#[derive(Clone, PartialEq, ::prost::Message)] +pub struct UniqueConstraint { + #[prost(uint64, repeated, tag = "1")] + pub indices: ::prost::alloc::vec::Vec, +} +#[allow(clippy::derive_partial_eq_without_eq)] +#[derive(Clone, PartialEq, ::prost::Message)] +pub struct Constraint { + #[prost(oneof = "constraint::ConstraintMode", tags = "1, 2")] + pub constraint_mode: ::core::option::Option, +} +/// Nested message and enum types in `Constraint`. +pub mod constraint { + #[allow(clippy::derive_partial_eq_without_eq)] + #[derive(Clone, PartialEq, ::prost::Oneof)] + pub enum ConstraintMode { + #[prost(message, tag = "1")] + PrimaryKey(super::PrimaryKeyConstraint), + #[prost(message, tag = "2")] + Unique(super::UniqueConstraint), + } +} +#[allow(clippy::derive_partial_eq_without_eq)] +#[derive(Clone, PartialEq, ::prost::Message)] +pub struct Constraints { + #[prost(message, repeated, tag = "1")] + pub constraints: ::prost::alloc::vec::Vec, +} +#[allow(clippy::derive_partial_eq_without_eq)] +#[derive(Clone, PartialEq, ::prost::Message)] pub struct CreateExternalTableNode { #[prost(message, optional, tag = "12")] pub name: ::core::option::Option, @@ -321,6 +356,8 @@ pub struct CreateExternalTableNode { ::prost::alloc::string::String, ::prost::alloc::string::String, >, + #[prost(message, optional, tag = "15")] + pub constraints: ::core::option::Option, } #[allow(clippy::derive_partial_eq_without_eq)] #[derive(Clone, PartialEq, ::prost::Message)] diff --git a/datafusion/proto/src/logical_plan/from_proto.rs b/datafusion/proto/src/logical_plan/from_proto.rs index f8746ef4fd6c..aa56c7b19d0c 100644 --- a/datafusion/proto/src/logical_plan/from_proto.rs +++ b/datafusion/proto/src/logical_plan/from_proto.rs @@ -31,8 +31,8 @@ use arrow::datatypes::{ }; use datafusion::execution::registry::FunctionRegistry; use datafusion_common::{ - internal_err, Column, DFField, DFSchema, DFSchemaRef, DataFusionError, - OwnedTableReference, Result, ScalarValue, + internal_err, Column, Constraint, Constraints, DFField, DFSchema, DFSchemaRef, + DataFusionError, OwnedTableReference, Result, ScalarValue, }; use datafusion_expr::{ abs, acos, acosh, array, array_append, array_concat, array_dims, array_element, @@ -880,6 +880,33 @@ impl From for JoinConstraint { } } +impl From for Constraints { + fn from(constraints: protobuf::Constraints) -> Self { + Constraints::new_unverified( + constraints + .constraints + .into_iter() + .map(|item| item.into()) + .collect(), + ) + } +} + +impl From for Constraint { + fn from(value: protobuf::Constraint) -> Self { + match value.constraint_mode.unwrap() { + protobuf::constraint::ConstraintMode::PrimaryKey(elem) => { + Constraint::PrimaryKey( + elem.indices.into_iter().map(|item| item as usize).collect(), + ) + } + protobuf::constraint::ConstraintMode::Unique(elem) => Constraint::Unique( + elem.indices.into_iter().map(|item| item as usize).collect(), + ), + } + } +} + pub fn parse_i32_to_time_unit(value: &i32) -> Result { protobuf::TimeUnit::try_from(*value) .map(|t| t.into()) diff --git a/datafusion/proto/src/logical_plan/mod.rs b/datafusion/proto/src/logical_plan/mod.rs index 099352cb589e..76d44186b5b5 100644 --- a/datafusion/proto/src/logical_plan/mod.rs +++ b/datafusion/proto/src/logical_plan/mod.rs @@ -15,6 +15,10 @@ // specific language governing permissions and limitations // under the License. +use std::fmt::Debug; +use std::str::FromStr; +use std::sync::Arc; + use crate::common::{byte_to_string, proto_error, str_to_byte}; use crate::protobuf::logical_plan_node::LogicalPlanType::CustomScan; use crate::protobuf::{CustomTableScanNode, LogicalExprNodeCollection}; @@ -25,6 +29,7 @@ use crate::{ logical_plan_node::LogicalPlanType, LogicalExtensionNode, LogicalPlanNode, }, }; + use arrow::datatypes::{DataType, Schema, SchemaRef}; use datafusion::{ datasource::{ @@ -38,27 +43,22 @@ use datafusion::{ datasource::{provider_as_source, source_as_provider}, prelude::SessionContext, }; -use datafusion_common::not_impl_err; use datafusion_common::{ - context, internal_err, parsers::CompressionTypeVariant, DataFusionError, - OwnedTableReference, Result, + context, internal_err, not_impl_err, parsers::CompressionTypeVariant, + DataFusionError, OwnedTableReference, Result, }; -use datafusion_expr::logical_plan::DdlStatement; -use datafusion_expr::DropView; use datafusion_expr::{ logical_plan::{ builder::project, Aggregate, CreateCatalog, CreateCatalogSchema, - CreateExternalTable, CreateView, CrossJoin, Distinct, EmptyRelation, Extension, - Join, JoinConstraint, Limit, Prepare, Projection, Repartition, Sort, - SubqueryAlias, TableScan, Values, Window, + CreateExternalTable, CreateView, CrossJoin, DdlStatement, Distinct, + EmptyRelation, Extension, Join, JoinConstraint, Limit, Prepare, Projection, + Repartition, Sort, SubqueryAlias, TableScan, Values, Window, }, - Expr, LogicalPlan, LogicalPlanBuilder, + DropView, Expr, LogicalPlan, LogicalPlanBuilder, }; + use prost::bytes::BufMut; use prost::Message; -use std::fmt::Debug; -use std::str::FromStr; -use std::sync::Arc; pub mod from_proto; pub mod to_proto; @@ -493,6 +493,11 @@ impl AsLogicalPlan for LogicalPlanNode { )) })?; + let constraints = (create_extern_table.constraints.clone()).ok_or_else(|| { + DataFusionError::Internal(String::from( + "Protobuf deserialization error, CreateExternalTableNode was missing required table constraints.", + )) + })?; let definition = if !create_extern_table.definition.is_empty() { Some(create_extern_table.definition.clone()) } else { @@ -532,6 +537,7 @@ impl AsLogicalPlan for LogicalPlanNode { definition, unbounded: create_extern_table.unbounded, options: create_extern_table.options.clone(), + constraints: constraints.into(), }))) } LogicalPlanType::CreateView(create_view) => { @@ -1205,6 +1211,7 @@ impl AsLogicalPlan for LogicalPlanNode { order_exprs, unbounded, options, + constraints, }, )) => { let mut converted_order_exprs: Vec = vec![]; @@ -1235,6 +1242,7 @@ impl AsLogicalPlan for LogicalPlanNode { file_compression_type: file_compression_type.to_string(), unbounded: *unbounded, options: options.clone(), + constraints: Some(constraints.clone().into()), }, )), }) diff --git a/datafusion/proto/src/logical_plan/to_proto.rs b/datafusion/proto/src/logical_plan/to_proto.rs index 8a8550d05d13..c10855bf2514 100644 --- a/datafusion/proto/src/logical_plan/to_proto.rs +++ b/datafusion/proto/src/logical_plan/to_proto.rs @@ -30,12 +30,14 @@ use crate::protobuf::{ AnalyzedLogicalPlanType, CubeNode, EmptyMessage, GroupingSetNode, LogicalExprList, OptimizedLogicalPlanType, OptimizedPhysicalPlanType, PlaceholderNode, RollupNode, }; + use arrow::datatypes::{ DataType, Field, IntervalMonthDayNanoType, IntervalUnit, Schema, SchemaRef, TimeUnit, UnionMode, }; use datafusion_common::{ - Column, DFField, DFSchema, DFSchemaRef, OwnedTableReference, ScalarValue, + Column, Constraint, Constraints, DFField, DFSchema, DFSchemaRef, OwnedTableReference, + ScalarValue, }; use datafusion_expr::expr::{ self, Alias, Between, BinaryExpr, Cast, GetFieldAccess, GetIndexedField, GroupingSet, @@ -1623,6 +1625,35 @@ impl From for protobuf::JoinConstraint { } } +impl From for protobuf::Constraints { + fn from(value: Constraints) -> Self { + let constraints = value.into_iter().map(|item| item.into()).collect(); + protobuf::Constraints { constraints } + } +} + +impl From for protobuf::Constraint { + fn from(value: Constraint) -> Self { + let res = match value { + Constraint::PrimaryKey(indices) => { + let indices = indices.into_iter().map(|item| item as u64).collect(); + protobuf::constraint::ConstraintMode::PrimaryKey( + protobuf::PrimaryKeyConstraint { indices }, + ) + } + Constraint::Unique(indices) => { + let indices = indices.into_iter().map(|item| item as u64).collect(); + protobuf::constraint::ConstraintMode::PrimaryKey( + protobuf::PrimaryKeyConstraint { indices }, + ) + } + }; + protobuf::Constraint { + constraint_mode: Some(res), + } + } +} + /// Creates a scalar protobuf value from an optional value (T), and /// encoding None as the appropriate datatype fn create_proto_scalar protobuf::scalar_value::Value>( diff --git a/datafusion/proto/tests/cases/roundtrip_logical_plan.rs b/datafusion/proto/tests/cases/roundtrip_logical_plan.rs index 5b53946379c8..11ee8c0876bc 100644 --- a/datafusion/proto/tests/cases/roundtrip_logical_plan.rs +++ b/datafusion/proto/tests/cases/roundtrip_logical_plan.rs @@ -185,6 +185,94 @@ async fn roundtrip_custom_tables() -> Result<()> { Ok(()) } +#[tokio::test] +async fn roundtrip_custom_memory_tables() -> Result<()> { + let ctx = SessionContext::new(); + // Make sure during round-trip, constraint information is preserved + let query = "CREATE TABLE sales_global_with_pk (zip_code INT, + country VARCHAR(3), + sn INT, + ts TIMESTAMP, + currency VARCHAR(3), + amount FLOAT, + primary key(sn) + ) as VALUES + (0, 'GRC', 0, '2022-01-01 06:00:00'::timestamp, 'EUR', 30.0), + (1, 'FRA', 1, '2022-01-01 08:00:00'::timestamp, 'EUR', 50.0), + (1, 'TUR', 2, '2022-01-01 11:30:00'::timestamp, 'TRY', 75.0), + (1, 'FRA', 3, '2022-01-02 12:00:00'::timestamp, 'EUR', 200.0), + (1, 'TUR', 4, '2022-01-03 10:00:00'::timestamp, 'TRY', 100.0)"; + + let plan = ctx.sql(query).await?.into_optimized_plan()?; + + let bytes = logical_plan_to_bytes(&plan)?; + let logical_round_trip = logical_plan_from_bytes(&bytes, &ctx)?; + assert_eq!(format!("{plan:?}"), format!("{logical_round_trip:?}")); + + Ok(()) +} + +#[tokio::test] +async fn roundtrip_custom_listing_tables() -> Result<()> { + let ctx = SessionContext::new(); + + // Make sure during round-trip, constraint information is preserved + let query = "CREATE EXTERNAL TABLE multiple_ordered_table_with_pk ( + a0 INTEGER, + a INTEGER, + b INTEGER, + c INTEGER, + d INTEGER, + primary key(c) + ) + STORED AS CSV + WITH HEADER ROW + WITH ORDER (a ASC, b ASC) + WITH ORDER (c ASC) + LOCATION '../core/tests/data/window_2.csv';"; + + let plan = ctx.sql(query).await?.into_optimized_plan()?; + + let bytes = logical_plan_to_bytes(&plan)?; + let logical_round_trip = logical_plan_from_bytes(&bytes, &ctx)?; + assert_eq!(format!("{plan:?}"), format!("{logical_round_trip:?}")); + + Ok(()) +} + +#[tokio::test] +async fn roundtrip_logical_plan_aggregation_with_pk() -> Result<()> { + let ctx = SessionContext::new(); + + ctx.sql( + "CREATE EXTERNAL TABLE multiple_ordered_table_with_pk ( + a0 INTEGER, + a INTEGER, + b INTEGER, + c INTEGER, + d INTEGER, + primary key(c) + ) + STORED AS CSV + WITH HEADER ROW + WITH ORDER (a ASC, b ASC) + WITH ORDER (c ASC) + LOCATION '../core/tests/data/window_2.csv';", + ) + .await?; + + let query = "SELECT c, b, SUM(d) + FROM multiple_ordered_table_with_pk + GROUP BY c"; + let plan = ctx.sql(query).await?.into_optimized_plan()?; + + let bytes = logical_plan_to_bytes(&plan)?; + let logical_round_trip = logical_plan_from_bytes(&bytes, &ctx)?; + assert_eq!(format!("{plan:?}"), format!("{logical_round_trip:?}")); + + Ok(()) +} + #[tokio::test] async fn roundtrip_logical_plan_aggregation() -> Result<()> { let ctx = SessionContext::new(); diff --git a/datafusion/sql/src/parser.rs b/datafusion/sql/src/parser.rs index f52fdec7f704..9c104ff18a9b 100644 --- a/datafusion/sql/src/parser.rs +++ b/datafusion/sql/src/parser.rs @@ -197,6 +197,8 @@ pub struct CreateExternalTable { pub unbounded: bool, /// Table(provider) specific options pub options: HashMap, + /// A table-level constraint + pub constraints: Vec, } impl fmt::Display for CreateExternalTable { @@ -629,7 +631,7 @@ impl<'a> DFParser<'a> { self.parser .parse_keywords(&[Keyword::IF, Keyword::NOT, Keyword::EXISTS]); let table_name = self.parser.parse_object_name()?; - let (columns, _) = self.parse_columns()?; + let (columns, constraints) = self.parse_columns()?; #[derive(Default)] struct Builder { @@ -748,6 +750,7 @@ impl<'a> DFParser<'a> { .unwrap_or(CompressionTypeVariant::UNCOMPRESSED), unbounded, options: builder.options.unwrap_or(HashMap::new()), + constraints, }; Ok(Statement::CreateExternalTable(create)) } @@ -899,6 +902,7 @@ mod tests { file_compression_type: UNCOMPRESSED, unbounded: false, options: HashMap::new(), + constraints: vec![], }); expect_parse_ok(sql, expected)?; @@ -917,6 +921,7 @@ mod tests { file_compression_type: UNCOMPRESSED, unbounded: false, options: HashMap::new(), + constraints: vec![], }); expect_parse_ok(sql, expected)?; @@ -936,6 +941,7 @@ mod tests { file_compression_type: UNCOMPRESSED, unbounded: false, options: HashMap::new(), + constraints: vec![], }); expect_parse_ok(sql, expected)?; @@ -955,6 +961,7 @@ mod tests { file_compression_type: UNCOMPRESSED, unbounded: false, options: HashMap::new(), + constraints: vec![], }); expect_parse_ok(sql, expected)?; @@ -974,6 +981,7 @@ mod tests { file_compression_type: UNCOMPRESSED, unbounded: false, options: HashMap::new(), + constraints: vec![], }); expect_parse_ok(sql, expected)?; @@ -996,6 +1004,7 @@ mod tests { file_compression_type: UNCOMPRESSED, unbounded: false, options: HashMap::new(), + constraints: vec![], }); expect_parse_ok(sql, expected)?; } @@ -1023,6 +1032,7 @@ mod tests { )?, unbounded: false, options: HashMap::new(), + constraints: vec![], }); expect_parse_ok(sql, expected)?; } @@ -1042,6 +1052,7 @@ mod tests { file_compression_type: UNCOMPRESSED, unbounded: false, options: HashMap::new(), + constraints: vec![], }); expect_parse_ok(sql, expected)?; @@ -1060,6 +1071,7 @@ mod tests { file_compression_type: UNCOMPRESSED, unbounded: false, options: HashMap::new(), + constraints: vec![], }); expect_parse_ok(sql, expected)?; @@ -1078,6 +1090,7 @@ mod tests { file_compression_type: UNCOMPRESSED, unbounded: false, options: HashMap::new(), + constraints: vec![], }); expect_parse_ok(sql, expected)?; @@ -1097,6 +1110,7 @@ mod tests { file_compression_type: UNCOMPRESSED, unbounded: false, options: HashMap::new(), + constraints: vec![], }); expect_parse_ok(sql, expected)?; @@ -1121,6 +1135,7 @@ mod tests { file_compression_type: UNCOMPRESSED, unbounded: false, options: HashMap::from([("k1".into(), "v1".into())]), + constraints: vec![], }); expect_parse_ok(sql, expected)?; @@ -1143,6 +1158,7 @@ mod tests { ("k1".into(), "v1".into()), ("k2".into(), "v2".into()), ]), + constraints: vec![], }); expect_parse_ok(sql, expected)?; @@ -1188,6 +1204,7 @@ mod tests { file_compression_type: UNCOMPRESSED, unbounded: false, options: HashMap::new(), + constraints: vec![], }); expect_parse_ok(sql, expected)?; } @@ -1228,6 +1245,7 @@ mod tests { file_compression_type: UNCOMPRESSED, unbounded: false, options: HashMap::new(), + constraints: vec![], }); expect_parse_ok(sql, expected)?; @@ -1264,6 +1282,7 @@ mod tests { file_compression_type: UNCOMPRESSED, unbounded: false, options: HashMap::new(), + constraints: vec![], }); expect_parse_ok(sql, expected)?; @@ -1312,6 +1331,7 @@ mod tests { ("ROW_GROUP_SIZE".into(), "1024".into()), ("TRUNCATE".into(), "NO".into()), ]), + constraints: vec![], }); expect_parse_ok(sql, expected)?; diff --git a/datafusion/sql/src/statement.rs b/datafusion/sql/src/statement.rs index de85820543ea..059a32be441b 100644 --- a/datafusion/sql/src/statement.rs +++ b/datafusion/sql/src/statement.rs @@ -15,6 +15,9 @@ // specific language governing permissions and limitations // under the License. +use std::collections::{BTreeMap, HashMap, HashSet}; +use std::sync::Arc; + use crate::parser::{ CopyToSource, CopyToStatement, CreateExternalTable, DFParser, DescribeTableStmt, ExplainStatement, LexOrdering, Statement as DFStatement, @@ -28,8 +31,8 @@ use arrow_schema::DataType; use datafusion_common::file_options::StatementOptions; use datafusion_common::parsers::CompressionTypeVariant; use datafusion_common::{ - not_impl_err, unqualified_field_not_found, Column, Constraints, DFField, DFSchema, - DFSchemaRef, DataFusionError, ExprSchema, OwnedTableReference, Result, + not_impl_err, plan_err, unqualified_field_not_found, Column, Constraints, DFField, + DFSchema, DFSchemaRef, DataFusionError, ExprSchema, OwnedTableReference, Result, SchemaReference, TableReference, ToDFSchema, }; use datafusion_expr::dml::{CopyOptions, CopyTo}; @@ -49,16 +52,12 @@ use datafusion_expr::{ }; use sqlparser::ast; use sqlparser::ast::{ - Assignment, Expr as SQLExpr, Expr, Ident, ObjectName, ObjectType, Query, SchemaName, - SetExpr, ShowCreateObject, ShowStatementFilter, Statement, TableFactor, - TableWithJoins, TransactionMode, UnaryOperator, Value, + Assignment, ColumnDef, Expr as SQLExpr, Expr, Ident, ObjectName, ObjectType, Query, + SchemaName, SetExpr, ShowCreateObject, ShowStatementFilter, Statement, + TableConstraint, TableFactor, TableWithJoins, TransactionMode, UnaryOperator, Value, }; use sqlparser::parser::ParserError::ParserError; -use datafusion_common::plan_err; -use std::collections::{BTreeMap, HashMap, HashSet}; -use std::sync::Arc; - fn ident_to_string(ident: &Ident) -> String { normalize_ident(ident.to_owned()) } @@ -84,6 +83,54 @@ fn get_schema_name(schema_name: &SchemaName) -> String { } } +/// Construct `TableConstraint`(s) for the given columns by iterating over +/// `columns` and extracting individual inline constraint definitions. +fn calc_inline_constraints_from_columns(columns: &[ColumnDef]) -> Vec { + let mut constraints = vec![]; + for column in columns { + for ast::ColumnOptionDef { name, option } in &column.options { + match option { + ast::ColumnOption::Unique { is_primary } => { + constraints.push(ast::TableConstraint::Unique { + name: name.clone(), + columns: vec![column.name.clone()], + is_primary: *is_primary, + }) + } + ast::ColumnOption::ForeignKey { + foreign_table, + referred_columns, + on_delete, + on_update, + } => constraints.push(ast::TableConstraint::ForeignKey { + name: name.clone(), + columns: vec![], + foreign_table: foreign_table.clone(), + referred_columns: referred_columns.to_vec(), + on_delete: *on_delete, + on_update: *on_update, + }), + ast::ColumnOption::Check(expr) => { + constraints.push(ast::TableConstraint::Check { + name: name.clone(), + expr: Box::new(expr.clone()), + }) + } + // Other options are not constraint related. + ast::ColumnOption::Default(_) + | ast::ColumnOption::Null + | ast::ColumnOption::NotNull + | ast::ColumnOption::DialectSpecific(_) + | ast::ColumnOption::CharacterSet(_) + | ast::ColumnOption::Generated { .. } + | ast::ColumnOption::Comment(_) + | ast::ColumnOption::OnUpdate(_) => {} + } + } + } + constraints +} + impl<'a, S: ContextProvider> SqlToRel<'a, S> { /// Generate a logical plan from an DataFusion SQL statement pub fn statement_to_plan(&self, statement: DFStatement) -> Result { @@ -154,18 +201,10 @@ impl<'a, S: ContextProvider> SqlToRel<'a, S> { or_replace, .. } if table_properties.is_empty() && with_options.is_empty() => { - let mut constraints = constraints; - for column in &columns { - for option in &column.options { - if let ast::ColumnOption::Unique { is_primary } = option.option { - constraints.push(ast::TableConstraint::Unique { - name: None, - columns: vec![column.name.clone()], - is_primary, - }) - } - } - } + // Merge inline constraints and existing constraints + let mut all_constraints = constraints; + let inline_constraints = calc_inline_constraints_from_columns(&columns); + all_constraints.extend(inline_constraints); match query { Some(query) => { let plan = self.query_to_plan(*query, planner_context)?; @@ -201,7 +240,7 @@ impl<'a, S: ContextProvider> SqlToRel<'a, S> { }; let constraints = Constraints::new_from_table_constraints( - &constraints, + &all_constraints, plan.schema(), )?; @@ -224,7 +263,7 @@ impl<'a, S: ContextProvider> SqlToRel<'a, S> { }; let plan = LogicalPlan::EmptyRelation(plan); let constraints = Constraints::new_from_table_constraints( - &constraints, + &all_constraints, plan.schema(), )?; Ok(LogicalPlan::Ddl(DdlStatement::CreateMemoryTable( @@ -681,8 +720,14 @@ impl<'a, S: ContextProvider> SqlToRel<'a, S> { order_exprs, unbounded, options, + constraints, } = statement; + // Merge inline constraints and existing constraints + let mut all_constraints = constraints; + let inline_constraints = calc_inline_constraints_from_columns(&columns); + all_constraints.extend(inline_constraints); + if (file_type == "PARQUET" || file_type == "AVRO" || file_type == "ARROW") && file_compression_type != CompressionTypeVariant::UNCOMPRESSED { @@ -699,7 +744,8 @@ impl<'a, S: ContextProvider> SqlToRel<'a, S> { // External tables do not support schemas at the moment, so the name is just a table name let name = OwnedTableReference::bare(name); - + let constraints = + Constraints::new_from_table_constraints(&all_constraints, &df_schema)?; Ok(LogicalPlan::Ddl(DdlStatement::CreateExternalTable( PlanCreateExternalTable { schema: df_schema, @@ -715,6 +761,7 @@ impl<'a, S: ContextProvider> SqlToRel<'a, S> { order_exprs: ordered_exprs, unbounded, options, + constraints, }, ))) } diff --git a/datafusion/sql/tests/sql_integration.rs b/datafusion/sql/tests/sql_integration.rs index f4ffea06b7a1..b1de5a12bcd0 100644 --- a/datafusion/sql/tests/sql_integration.rs +++ b/datafusion/sql/tests/sql_integration.rs @@ -1806,6 +1806,14 @@ fn create_external_table_csv() { quick_test(sql, expected); } +#[test] +fn create_external_table_with_pk() { + let sql = "CREATE EXTERNAL TABLE t(c1 int, primary key(c1)) STORED AS CSV LOCATION 'foo.csv'"; + let expected = + "CreateExternalTable: Bare { table: \"t\" } constraints=[PrimaryKey([0])]"; + quick_test(sql, expected); +} + #[test] fn create_schema_with_quoted_name() { let sql = "CREATE SCHEMA \"quoted_schema_name\""; diff --git a/datafusion/sqllogictest/test_files/groupby.slt b/datafusion/sqllogictest/test_files/groupby.slt index 5bb0f31ed542..b7070a8d7efb 100644 --- a/datafusion/sqllogictest/test_files/groupby.slt +++ b/datafusion/sqllogictest/test_files/groupby.slt @@ -3093,6 +3093,55 @@ CREATE TABLE sales_global_with_pk_alternate (zip_code INT, (1, 'FRA', 3, '2022-01-02 12:00:00'::timestamp, 'EUR', 200.0), (1, 'TUR', 4, '2022-01-03 10:00:00'::timestamp, 'TRY', 100.0) +# we do not currently support foreign key constraints. +statement error DataFusion error: Error during planning: Foreign key constraints are not currently supported +CREATE TABLE sales_global_with_foreign_key (zip_code INT, + country VARCHAR(3), + sn INT references sales_global_with_pk_alternate(sn), + ts TIMESTAMP, + currency VARCHAR(3), + amount FLOAT +) as VALUES + (0, 'GRC', 0, '2022-01-01 06:00:00'::timestamp, 'EUR', 30.0), + (1, 'FRA', 1, '2022-01-01 08:00:00'::timestamp, 'EUR', 50.0), + (1, 'TUR', 2, '2022-01-01 11:30:00'::timestamp, 'TRY', 75.0), + (1, 'FRA', 3, '2022-01-02 12:00:00'::timestamp, 'EUR', 200.0), + (1, 'TUR', 4, '2022-01-03 10:00:00'::timestamp, 'TRY', 100.0) + +# we do not currently support foreign key +statement error DataFusion error: Error during planning: Foreign key constraints are not currently supported +CREATE TABLE sales_global_with_foreign_key (zip_code INT, + country VARCHAR(3), + sn INT REFERENCES sales_global_with_pk_alternate(sn), + ts TIMESTAMP, + currency VARCHAR(3), + amount FLOAT +) as VALUES + (0, 'GRC', 0, '2022-01-01 06:00:00'::timestamp, 'EUR', 30.0), + (1, 'FRA', 1, '2022-01-01 08:00:00'::timestamp, 'EUR', 50.0), + (1, 'TUR', 2, '2022-01-01 11:30:00'::timestamp, 'TRY', 75.0), + (1, 'FRA', 3, '2022-01-02 12:00:00'::timestamp, 'EUR', 200.0), + (1, 'TUR', 4, '2022-01-03 10:00:00'::timestamp, 'TRY', 100.0) + +# we do not currently support foreign key +# foreign key can be defined with a different syntax. +# we should get the same error. +statement error DataFusion error: Error during planning: Foreign key constraints are not currently supported +CREATE TABLE sales_global_with_foreign_key (zip_code INT, + country VARCHAR(3), + sn INT, + ts TIMESTAMP, + currency VARCHAR(3), + amount FLOAT, + FOREIGN KEY (sn) + REFERENCES sales_global_with_pk_alternate(sn) +) as VALUES + (0, 'GRC', 0, '2022-01-01 06:00:00'::timestamp, 'EUR', 30.0), + (1, 'FRA', 1, '2022-01-01 08:00:00'::timestamp, 'EUR', 50.0), + (1, 'TUR', 2, '2022-01-01 11:30:00'::timestamp, 'TRY', 75.0), + (1, 'FRA', 3, '2022-01-02 12:00:00'::timestamp, 'EUR', 200.0), + (1, 'TUR', 4, '2022-01-03 10:00:00'::timestamp, 'TRY', 100.0) + # create a table for testing, where primary key is composite statement ok CREATE TABLE sales_global_with_composite_pk (zip_code INT, @@ -3563,3 +3612,81 @@ logical_plan Sort: multiple_ordered_table.c ASC NULLS LAST --TableScan: multiple_ordered_table projection=[c] physical_plan CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[c], output_ordering=[c@0 ASC NULLS LAST], has_header=true + +# Create an external table with primary key +# column c +statement ok +CREATE EXTERNAL TABLE multiple_ordered_table_with_pk ( + a0 INTEGER, + a INTEGER, + b INTEGER, + c INTEGER, + d INTEGER, + primary key(c) +) +STORED AS CSV +WITH HEADER ROW +WITH ORDER (a ASC, b ASC) +WITH ORDER (c ASC) +LOCATION '../core/tests/data/window_2.csv'; + +# We can use column b during selection +# even if it is not among group by expressions +# because column c is primary key. +query TT +EXPLAIN SELECT c, b, SUM(d) +FROM multiple_ordered_table_with_pk +GROUP BY c; +---- +logical_plan +Aggregate: groupBy=[[multiple_ordered_table_with_pk.c, multiple_ordered_table_with_pk.b]], aggr=[[SUM(CAST(multiple_ordered_table_with_pk.d AS Int64))]] +--TableScan: multiple_ordered_table_with_pk projection=[b, c, d] +physical_plan +AggregateExec: mode=FinalPartitioned, gby=[c@0 as c, b@1 as b], aggr=[SUM(multiple_ordered_table_with_pk.d)], ordering_mode=PartiallyOrdered +--SortExec: expr=[c@0 ASC NULLS LAST] +----CoalesceBatchesExec: target_batch_size=8192 +------RepartitionExec: partitioning=Hash([c@0, b@1], 8), input_partitions=8 +--------AggregateExec: mode=Partial, gby=[c@1 as c, b@0 as b], aggr=[SUM(multiple_ordered_table_with_pk.d)], ordering_mode=PartiallyOrdered +----------RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1 +------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[b, c, d], output_ordering=[c@1 ASC NULLS LAST], has_header=true + +# drop table multiple_ordered_table_with_pk +statement ok +drop table multiple_ordered_table_with_pk; + +# Create an external table with primary key +# column c, in this case use alternative syntax +# for defining primary key +statement ok +CREATE EXTERNAL TABLE multiple_ordered_table_with_pk ( + a0 INTEGER, + a INTEGER, + b INTEGER, + c INTEGER primary key, + d INTEGER +) +STORED AS CSV +WITH HEADER ROW +WITH ORDER (a ASC, b ASC) +WITH ORDER (c ASC) +LOCATION '../core/tests/data/window_2.csv'; + +# We can use column b during selection +# even if it is not among group by expressions +# because column c is primary key. +query TT +EXPLAIN SELECT c, b, SUM(d) +FROM multiple_ordered_table_with_pk +GROUP BY c; +---- +logical_plan +Aggregate: groupBy=[[multiple_ordered_table_with_pk.c, multiple_ordered_table_with_pk.b]], aggr=[[SUM(CAST(multiple_ordered_table_with_pk.d AS Int64))]] +--TableScan: multiple_ordered_table_with_pk projection=[b, c, d] +physical_plan +AggregateExec: mode=FinalPartitioned, gby=[c@0 as c, b@1 as b], aggr=[SUM(multiple_ordered_table_with_pk.d)], ordering_mode=PartiallyOrdered +--SortExec: expr=[c@0 ASC NULLS LAST] +----CoalesceBatchesExec: target_batch_size=8192 +------RepartitionExec: partitioning=Hash([c@0, b@1], 8), input_partitions=8 +--------AggregateExec: mode=Partial, gby=[c@1 as c, b@0 as b], aggr=[SUM(multiple_ordered_table_with_pk.d)], ordering_mode=PartiallyOrdered +----------RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1 +------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[b, c, d], output_ordering=[c@1 ASC NULLS LAST], has_header=true From 9ff35f659e686044b40f4ddc5fa8df4f7aafe2bb Mon Sep 17 00:00:00 2001 From: Jay Zhan Date: Tue, 10 Oct 2023 04:44:48 +0800 Subject: [PATCH 045/572] Minor: Build array_array() with ListArray construction instead of ArrayData (#7780) * rewrite array_array Signed-off-by: jayzhan211 * remove unwrap Signed-off-by: jayzhan211 --------- Signed-off-by: jayzhan211 --- .../physical-expr/src/array_expressions.rs | 120 +++++++++--------- 1 file changed, 58 insertions(+), 62 deletions(-) diff --git a/datafusion/physical-expr/src/array_expressions.rs b/datafusion/physical-expr/src/array_expressions.rs index 790c0481e188..0a9c2f56048e 100644 --- a/datafusion/physical-expr/src/array_expressions.rs +++ b/datafusion/physical-expr/src/array_expressions.rs @@ -18,7 +18,7 @@ //! Array expressions use arrow::array::*; -use arrow::buffer::{Buffer, OffsetBuffer}; +use arrow::buffer::OffsetBuffer; use arrow::compute; use arrow::datatypes::{DataType, Field, UInt64Type}; use arrow_buffer::NullBuffer; @@ -261,12 +261,6 @@ macro_rules! call_array_function { }}; } -#[derive(Debug)] -enum ListOrNull<'a> { - List(&'a dyn Array), - Null, -} - /// Convert one or more [`ArrayRef`] of the same type into a /// `ListArray` /// @@ -315,64 +309,66 @@ fn array_array(args: &[ArrayRef], data_type: DataType) -> Result { let res = match data_type { DataType::List(..) => { - let mut arrays = vec![]; - let mut row_count = 0; - - for arg in args { - let list_arr = arg.as_list_opt::(); - if let Some(list_arr) = list_arr { - // Assume number of rows is the same for all arrays - row_count = list_arr.len(); - arrays.push(ListOrNull::List(list_arr)); - } else if arg.as_any().downcast_ref::().is_some() { - arrays.push(ListOrNull::Null); - } else { - return internal_err!("Unsupported argument type for array"); - } - } - - let mut total_capacity = 0; - let mut array_data = vec![]; - for arr in arrays.iter() { - if let ListOrNull::List(arr) = arr { - total_capacity += arr.len(); - array_data.push(arr.to_data()); - } - } - let capacity = Capacities::Array(total_capacity); - let array_data = array_data.iter().collect(); - - let mut mutable = - MutableArrayData::with_capacities(array_data, true, capacity); - - for i in 0..row_count { - let mut nulls = 0; - for (j, arr) in arrays.iter().enumerate() { - match arr { - ListOrNull::List(_) => { - mutable.extend(j - nulls, i, i + 1); - } - ListOrNull::Null => { - mutable.extend_nulls(1); - nulls += 1; - } + let row_count = args[0].len(); + let column_count = args.len(); + let mut list_arrays = vec![]; + let mut list_array_lengths = vec![]; + let mut list_valid = BooleanBufferBuilder::new(row_count); + // Construct ListArray per row + for index in 0..row_count { + let mut arrays = vec![]; + let mut array_lengths = vec![]; + let mut valid = BooleanBufferBuilder::new(column_count); + for arg in args { + if arg.as_any().downcast_ref::().is_some() { + array_lengths.push(0); + valid.append(false); + } else { + let list_arr = as_list_array(arg)?; + let arr = list_arr.value(index); + array_lengths.push(arr.len()); + arrays.push(arr); + valid.append(true); } } + if arrays.is_empty() { + list_valid.append(false); + list_array_lengths.push(0); + } else { + let buffer = valid.finish(); + // Assume all list arrays have the same data type + let data_type = arrays[0].data_type(); + let field = Arc::new(Field::new("item", data_type.to_owned(), true)); + let elements = arrays.iter().map(|x| x.as_ref()).collect::>(); + let values = arrow::compute::concat(elements.as_slice())?; + let list_arr = ListArray::new( + field, + OffsetBuffer::from_lengths(array_lengths), + values, + Some(NullBuffer::new(buffer)), + ); + list_valid.append(true); + list_array_lengths.push(list_arr.len()); + list_arrays.push(list_arr); + } } - - let list_data_type = - DataType::List(Arc::new(Field::new("item", data_type, true))); - - let offsets: Vec = (0..row_count as i32 + 1) - .map(|i| i * arrays.len() as i32) - .collect(); - - let list_data = ArrayData::builder(list_data_type) - .len(row_count) - .buffers(vec![Buffer::from_vec(offsets)]) - .add_child_data(mutable.freeze()) - .build()?; - Arc::new(ListArray::from(list_data)) + // Construct ListArray for all rows + let buffer = list_valid.finish(); + // Assume all list arrays have the same data type + let data_type = list_arrays[0].data_type(); + let field = Arc::new(Field::new("item", data_type.to_owned(), true)); + let elements = list_arrays + .iter() + .map(|x| x as &dyn Array) + .collect::>(); + let values = arrow::compute::concat(elements.as_slice())?; + let list_arr = ListArray::new( + field, + OffsetBuffer::from_lengths(list_array_lengths), + values, + Some(NullBuffer::new(buffer)), + ); + Arc::new(list_arr) } DataType::Utf8 => array!(args, StringArray, StringBuilder), DataType::LargeUtf8 => array!(args, LargeStringArray, LargeStringBuilder), From b0ea758bb1f80a6c97b5ce6bdef6231aa25e3e88 Mon Sep 17 00:00:00 2001 From: Kousuke Saruta Date: Tue, 10 Oct 2023 06:18:21 +0900 Subject: [PATCH 046/572] Remove unnecessary cfg (#7773) --- datafusion/core/tests/sql/mod.rs | 1 - 1 file changed, 1 deletion(-) diff --git a/datafusion/core/tests/sql/mod.rs b/datafusion/core/tests/sql/mod.rs index 20d715b32c69..d44513e69a9f 100644 --- a/datafusion/core/tests/sql/mod.rs +++ b/datafusion/core/tests/sql/mod.rs @@ -74,7 +74,6 @@ macro_rules! test_expression { pub mod aggregates; pub mod arrow_files; -#[cfg(feature = "avro")] pub mod create_drop; pub mod csv_files; pub mod describe; From 96561a417e9aa24e36a4123278df333f2dfd3e01 Mon Sep 17 00:00:00 2001 From: Martin Hilton Date: Mon, 9 Oct 2023 22:56:45 +0100 Subject: [PATCH 047/572] add interval arithmetic for timestamp types (#7758) * add interval arithmetic for timestamp types Timestamp types have custom arithmetic and need special handling when attempting to determine potential interval ranges. Change the processing of comparison operator propagation to convert timestamp intervals into int64 intervals for processing. The results are converted back the the correct datatype at the end of the process. * apply review suggestion The review from @berkaysynnada showed that it was not necessary to have special handling for timestamp types, but to make sure the new_zero function for scalars of a duration type return 0 rather than null values. Apply the suggested change, leaving the test to ensure the functionality doesn't break in the future. * clippy fixes Apply a number of changes suggested by clippy. * fix: edit clash * Update datafusion/physical-expr/src/intervals/cp_solver.rs * fmt --------- Co-authored-by: Andrew Lamb --- datafusion/common/src/scalar.rs | 8 +- .../physical-expr/src/intervals/cp_solver.rs | 131 ++++++++++++++++++ 2 files changed, 135 insertions(+), 4 deletions(-) diff --git a/datafusion/common/src/scalar.rs b/datafusion/common/src/scalar.rs index 32343b98fa24..ae9c29b5a141 100644 --- a/datafusion/common/src/scalar.rs +++ b/datafusion/common/src/scalar.rs @@ -833,15 +833,15 @@ impl ScalarValue { DataType::Interval(IntervalUnit::MonthDayNano) => { ScalarValue::IntervalMonthDayNano(Some(0)) } - DataType::Duration(TimeUnit::Second) => ScalarValue::DurationSecond(None), + DataType::Duration(TimeUnit::Second) => ScalarValue::DurationSecond(Some(0)), DataType::Duration(TimeUnit::Millisecond) => { - ScalarValue::DurationMillisecond(None) + ScalarValue::DurationMillisecond(Some(0)) } DataType::Duration(TimeUnit::Microsecond) => { - ScalarValue::DurationMicrosecond(None) + ScalarValue::DurationMicrosecond(Some(0)) } DataType::Duration(TimeUnit::Nanosecond) => { - ScalarValue::DurationNanosecond(None) + ScalarValue::DurationNanosecond(Some(0)) } _ => { return _not_impl_err!( diff --git a/datafusion/physical-expr/src/intervals/cp_solver.rs b/datafusion/physical-expr/src/intervals/cp_solver.rs index 0a090636dc4b..e7515341c52c 100644 --- a/datafusion/physical-expr/src/intervals/cp_solver.rs +++ b/datafusion/physical-expr/src/intervals/cp_solver.rs @@ -688,6 +688,7 @@ mod tests { use crate::expressions::{BinaryExpr, Column}; use crate::intervals::test_utils::gen_conjunctive_numerical_expr; + use arrow::datatypes::TimeUnit; use datafusion_common::ScalarValue; use rand::rngs::StdRng; use rand::{Rng, SeedableRng}; @@ -1414,4 +1415,134 @@ mod tests { Ok(()) } + + #[test] + fn test_propagate_comparison() { + // In the examples below: + // `left` is unbounded: [?, ?], + // `right` is known to be [1000,1000] + // so `left` < `right` results in no new knowledge of `right` but knowing that `left` is now < 1000:` [?, 1000) + let left = Interval::new( + IntervalBound::make_unbounded(DataType::Int64).unwrap(), + IntervalBound::make_unbounded(DataType::Int64).unwrap(), + ); + let right = Interval::new( + IntervalBound::new(ScalarValue::Int64(Some(1000)), false), + IntervalBound::new(ScalarValue::Int64(Some(1000)), false), + ); + assert_eq!( + ( + Some(Interval::new( + IntervalBound::make_unbounded(DataType::Int64).unwrap(), + IntervalBound::new(ScalarValue::Int64(Some(1000)), true) + )), + Some(Interval::new( + IntervalBound::new(ScalarValue::Int64(Some(1000)), false), + IntervalBound::new(ScalarValue::Int64(Some(1000)), false) + )), + ), + propagate_comparison(&Operator::Lt, &left, &right).unwrap() + ); + + let left = Interval::new( + IntervalBound::make_unbounded(DataType::Timestamp( + TimeUnit::Nanosecond, + None, + )) + .unwrap(), + IntervalBound::make_unbounded(DataType::Timestamp( + TimeUnit::Nanosecond, + None, + )) + .unwrap(), + ); + let right = Interval::new( + IntervalBound::new(ScalarValue::TimestampNanosecond(Some(1000), None), false), + IntervalBound::new(ScalarValue::TimestampNanosecond(Some(1000), None), false), + ); + assert_eq!( + ( + Some(Interval::new( + IntervalBound::make_unbounded(DataType::Timestamp( + TimeUnit::Nanosecond, + None + )) + .unwrap(), + IntervalBound::new( + ScalarValue::TimestampNanosecond(Some(1000), None), + true + ) + )), + Some(Interval::new( + IntervalBound::new( + ScalarValue::TimestampNanosecond(Some(1000), None), + false + ), + IntervalBound::new( + ScalarValue::TimestampNanosecond(Some(1000), None), + false + ) + )), + ), + propagate_comparison(&Operator::Lt, &left, &right).unwrap() + ); + + let left = Interval::new( + IntervalBound::make_unbounded(DataType::Timestamp( + TimeUnit::Nanosecond, + Some("+05:00".into()), + )) + .unwrap(), + IntervalBound::make_unbounded(DataType::Timestamp( + TimeUnit::Nanosecond, + Some("+05:00".into()), + )) + .unwrap(), + ); + let right = Interval::new( + IntervalBound::new( + ScalarValue::TimestampNanosecond(Some(1000), Some("+05:00".into())), + false, + ), + IntervalBound::new( + ScalarValue::TimestampNanosecond(Some(1000), Some("+05:00".into())), + false, + ), + ); + assert_eq!( + ( + Some(Interval::new( + IntervalBound::make_unbounded(DataType::Timestamp( + TimeUnit::Nanosecond, + Some("+05:00".into()), + )) + .unwrap(), + IntervalBound::new( + ScalarValue::TimestampNanosecond( + Some(1000), + Some("+05:00".into()) + ), + true + ) + )), + Some(Interval::new( + IntervalBound::new( + ScalarValue::TimestampNanosecond( + Some(1000), + Some("+05:00".into()) + ), + false + ), + IntervalBound::new( + ScalarValue::TimestampNanosecond( + Some(1000), + Some("+05:00".into()) + ), + false + ) + )), + ), + propagate_comparison(&Operator::Lt, &left, &right).unwrap() + ); + } } From 9f25634414a2650ee8dd2b263f0e29900b13a370 Mon Sep 17 00:00:00 2001 From: Alex Huang Date: Mon, 9 Oct 2023 23:57:39 +0200 Subject: [PATCH 048/572] make tests deterministic (#7771) --- datafusion/sqllogictest/test_files/aggregate.slt | 15 ++++++++------- 1 file changed, 8 insertions(+), 7 deletions(-) diff --git a/datafusion/sqllogictest/test_files/aggregate.slt b/datafusion/sqllogictest/test_files/aggregate.slt index d0e41b12b8c9..64bf64d23c7a 100644 --- a/datafusion/sqllogictest/test_files/aggregate.slt +++ b/datafusion/sqllogictest/test_files/aggregate.slt @@ -2318,6 +2318,7 @@ CREATE TABLE traces(trace_id varchar, timestamp bigint, other bigint) AS VALUES (NULL, 0, 0), ('a', NULL, NULL), ('a', 1, 1), +('a', -1, -1), ('b', 0, 0), ('c', 1, 1), ('c', 2, 2), @@ -2357,26 +2358,26 @@ NULL 0 query TI select trace_id, MIN(timestamp) from traces group by trace_id order by MIN(timestamp) asc limit 4; ---- +a -1 NULL 0 b 0 c 1 -a 1 query TII select trace_id, other, MIN(timestamp) from traces group by trace_id, other order by MIN(timestamp) asc limit 4; ---- +a -1 -1 b 0 0 NULL 0 0 c 1 1 -a 1 1 query TII select trace_id, MIN(other), MIN(timestamp) from traces group by trace_id order by MIN(timestamp), MIN(other) limit 4; ---- +a -1 -1 NULL 0 0 b 0 0 c 1 1 -a 1 1 statement ok set datafusion.optimizer.enable_topk_aggregation = true; @@ -2468,10 +2469,10 @@ NULL 0 query TI select trace_id, MIN(timestamp) from traces group by trace_id order by MIN(timestamp) asc limit 4; ---- +a -1 NULL 0 b 0 c 1 -a 1 query TI select trace_id, MAX(timestamp) from traces group by trace_id order by MAX(timestamp) desc limit 3; @@ -2483,25 +2484,25 @@ a 1 query TI select trace_id, MIN(timestamp) from traces group by trace_id order by MIN(timestamp) asc limit 3; ---- +a -1 NULL 0 b 0 -c 1 query TII select trace_id, other, MIN(timestamp) from traces group by trace_id, other order by MIN(timestamp) asc limit 4; ---- +a -1 -1 b 0 0 NULL 0 0 c 1 1 -a 1 1 query TII select trace_id, MIN(other), MIN(timestamp) from traces group by trace_id order by MIN(timestamp), MIN(other) limit 4; ---- +a -1 -1 NULL 0 0 b 0 0 c 1 1 -a 1 1 # # regr_*() tests From b6f87edbbaaaedd696b2e448fcd7df7ad3cd2c93 Mon Sep 17 00:00:00 2001 From: Andrew Lamb Date: Tue, 10 Oct 2023 06:28:28 -0400 Subject: [PATCH 049/572] Minor: Improve `Interval` Docs (#7782) --- .../src/intervals/interval_aritmetic.rs | 37 +++++++++++++++---- 1 file changed, 29 insertions(+), 8 deletions(-) diff --git a/datafusion/physical-expr/src/intervals/interval_aritmetic.rs b/datafusion/physical-expr/src/intervals/interval_aritmetic.rs index 3ed228517fd2..210db0ebe6a2 100644 --- a/datafusion/physical-expr/src/intervals/interval_aritmetic.rs +++ b/datafusion/physical-expr/src/intervals/interval_aritmetic.rs @@ -174,10 +174,28 @@ impl Display for IntervalBound { } /// This type represents an interval, which is used to calculate reliable -/// bounds for expressions. Currently, we only support addition and -/// subtraction, but more capabilities will be added in the future. -/// Upper/lower bounds having NULL values indicate an unbounded side. For -/// example; [10, 20], [10, ∞), (-∞, 100] and (-∞, ∞) are all valid intervals. +/// bounds for expressions: +/// +/// * An *open* interval does not include the endpoint and is written using a +/// `(` or `)`. +/// +/// * A *closed* interval does include the endpoint and is written using `[` or +/// `]`. +/// +/// * If the interval's `lower` and/or `upper` bounds are not known, they are +/// called *unbounded* endpoint and represented using a `NULL` and written using +/// `∞`. +/// +/// # Examples +/// +/// A `Int64` `Interval` of `[10, 20)` represents the values `10, 11, ... 18, +/// 19` (includes 10, but does not include 20). +/// +/// A `Int64` `Interval` of `[10, ∞)` represents a value known to be either +/// `10` or higher. +/// +/// An `Interval` of `(-∞, ∞)` represents that the range is entirely unknown. +/// #[derive(Debug, Clone, PartialEq, Eq)] pub struct Interval { pub lower: IntervalBound, @@ -208,14 +226,17 @@ impl Display for Interval { impl Interval { /// Creates a new interval object using the given bounds. - /// For boolean intervals, having an open false lower bound is equivalent - /// to having a true closed lower bound. Similarly, open true upper bound - /// is equivalent to having a false closed upper bound. Also for boolean + /// + /// # Boolean intervals need special handling + /// + /// For boolean intervals, having an open false lower bound is equivalent to + /// having a true closed lower bound. Similarly, open true upper bound is + /// equivalent to having a false closed upper bound. Also for boolean /// intervals, having an unbounded left endpoint is equivalent to having a /// false closed lower bound, while having an unbounded right endpoint is /// equivalent to having a true closed upper bound. Therefore; input /// parameters to construct an Interval can have different types, but they - /// all result in [false, false], [false, true] or [true, true]. + /// all result in `[false, false]`, `[false, true]` or `[true, true]`. pub fn new(lower: IntervalBound, upper: IntervalBound) -> Interval { // Boolean intervals need a special handling. if let ScalarValue::Boolean(_) = lower.value { From d2cc4d253e13f32ac95545ae79a3cb2d4d59de78 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Dani=C3=ABl=20Heres?= Date: Tue, 10 Oct 2023 14:55:18 +0200 Subject: [PATCH 050/572] `DataSink` additions (#7778) * File sink additions * Fmt * Clippy * Update datafusion/physical-plan/src/insert.rs Co-authored-by: Andrew Lamb * Feedback * Fmt --------- Co-authored-by: Andrew Lamb --- .../core/src/datasource/file_format/csv.rs | 9 ++++++++ .../core/src/datasource/file_format/json.rs | 9 ++++++++ .../src/datasource/file_format/parquet.rs | 9 ++++++++ datafusion/core/src/datasource/memory.rs | 9 ++++++++ datafusion/physical-plan/src/insert.rs | 21 +++++++++++++++++++ 5 files changed, 57 insertions(+) diff --git a/datafusion/core/src/datasource/file_format/csv.rs b/datafusion/core/src/datasource/file_format/csv.rs index 4c625b7ed742..e77382ad9ce8 100644 --- a/datafusion/core/src/datasource/file_format/csv.rs +++ b/datafusion/core/src/datasource/file_format/csv.rs @@ -33,6 +33,7 @@ use datafusion_physical_expr::{PhysicalExpr, PhysicalSortRequirement}; use async_trait::async_trait; use bytes::{Buf, Bytes}; +use datafusion_physical_plan::metrics::MetricsSet; use futures::stream::BoxStream; use futures::{pin_mut, Stream, StreamExt, TryStreamExt}; use object_store::{delimited::newline_delimited_stream, ObjectMeta, ObjectStore}; @@ -484,6 +485,14 @@ impl CsvSink { #[async_trait] impl DataSink for CsvSink { + fn as_any(&self) -> &dyn Any { + self + } + + fn metrics(&self) -> Option { + None + } + async fn write_all( &self, data: Vec, diff --git a/datafusion/core/src/datasource/file_format/json.rs b/datafusion/core/src/datasource/file_format/json.rs index 6c260b9802ec..fa8fb5a72331 100644 --- a/datafusion/core/src/datasource/file_format/json.rs +++ b/datafusion/core/src/datasource/file_format/json.rs @@ -25,6 +25,7 @@ use datafusion_common::DataFusionError; use datafusion_common::FileType; use datafusion_execution::TaskContext; use datafusion_physical_expr::PhysicalSortRequirement; +use datafusion_physical_plan::metrics::MetricsSet; use rand::distributions::Alphanumeric; use rand::distributions::DistString; use std::fmt; @@ -276,6 +277,14 @@ impl JsonSink { #[async_trait] impl DataSink for JsonSink { + fn as_any(&self) -> &dyn Any { + self + } + + fn metrics(&self) -> Option { + None + } + async fn write_all( &self, data: Vec, diff --git a/datafusion/core/src/datasource/file_format/parquet.rs b/datafusion/core/src/datasource/file_format/parquet.rs index 062ec1329dc4..d946bfb0b90d 100644 --- a/datafusion/core/src/datasource/file_format/parquet.rs +++ b/datafusion/core/src/datasource/file_format/parquet.rs @@ -17,6 +17,7 @@ //! Parquet format abstractions +use datafusion_physical_plan::metrics::MetricsSet; use parquet::column::writer::ColumnCloseResult; use parquet::file::writer::SerializedFileWriter; use rand::distributions::DistString; @@ -757,6 +758,14 @@ impl ParquetSink { #[async_trait] impl DataSink for ParquetSink { + fn as_any(&self) -> &dyn Any { + self + } + + fn metrics(&self) -> Option { + None + } + async fn write_all( &self, mut data: Vec, diff --git a/datafusion/core/src/datasource/memory.rs b/datafusion/core/src/datasource/memory.rs index 2766e73d3319..ba99a2b695b6 100644 --- a/datafusion/core/src/datasource/memory.rs +++ b/datafusion/core/src/datasource/memory.rs @@ -17,6 +17,7 @@ //! [`MemTable`] for querying `Vec` by DataFusion. +use datafusion_physical_plan::metrics::MetricsSet; use futures::StreamExt; use log::debug; use std::any::Any; @@ -259,6 +260,14 @@ impl MemSink { #[async_trait] impl DataSink for MemSink { + fn as_any(&self) -> &dyn Any { + self + } + + fn metrics(&self) -> Option { + None + } + async fn write_all( &self, mut data: Vec, diff --git a/datafusion/physical-plan/src/insert.rs b/datafusion/physical-plan/src/insert.rs index a7b0d32c8eb8..bff20e85b7ff 100644 --- a/datafusion/physical-plan/src/insert.rs +++ b/datafusion/physical-plan/src/insert.rs @@ -35,6 +35,7 @@ use std::any::Any; use std::fmt::Debug; use std::sync::Arc; +use crate::metrics::MetricsSet; use crate::stream::RecordBatchStreamAdapter; use datafusion_common::{exec_err, internal_err, DataFusionError}; use datafusion_execution::TaskContext; @@ -46,6 +47,16 @@ use datafusion_execution::TaskContext; /// output. #[async_trait] pub trait DataSink: DisplayAs + Debug + Send + Sync { + /// Returns the data sink as [`Any`](std::any::Any) so that it can be + /// downcast to a specific implementation. + fn as_any(&self) -> &dyn Any; + + /// Return a snapshot of the [MetricsSet] for this + /// [DataSink]. + /// + /// See [ExecutionPlan::metrics()] for more details + fn metrics(&self) -> Option; + // TODO add desired input ordering // How does this sink want its input ordered? @@ -151,6 +162,16 @@ impl FileSinkExec { } Ok(streams) } + + /// Returns insert sink + pub fn sink(&self) -> &dyn DataSink { + self.sink.as_ref() + } + + /// Returns the metrics of the underlying [DataSink] + pub fn metrics(&self) -> Option { + self.sink.metrics() + } } impl DisplayAs for FileSinkExec { From a0c5affca271d67980286cb2ae08ea8eec75a326 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Tue, 10 Oct 2023 09:35:22 -0400 Subject: [PATCH 051/572] Update substrait requirement from 0.15.0 to 0.16.0 (#7783) Updates the requirements on [substrait](https://github.com/substrait-io/substrait-rs) to permit the latest version. - [Release notes](https://github.com/substrait-io/substrait-rs/releases) - [Changelog](https://github.com/substrait-io/substrait-rs/blob/main/CHANGELOG.md) - [Commits](https://github.com/substrait-io/substrait-rs/compare/v0.15.0...v0.16.0) --- updated-dependencies: - dependency-name: substrait dependency-type: direct:production ... Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- datafusion/substrait/Cargo.toml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/datafusion/substrait/Cargo.toml b/datafusion/substrait/Cargo.toml index a793df40e2ae..13b9c02cdd28 100644 --- a/datafusion/substrait/Cargo.toml +++ b/datafusion/substrait/Cargo.toml @@ -35,7 +35,7 @@ itertools = "0.11" object_store = "0.7.0" prost = "0.11" prost-types = "0.11" -substrait = "0.15.0" +substrait = "0.16.0" tokio = "1.17" [features] From 704e034abbc57645ae594288c49c550350540022 Mon Sep 17 00:00:00 2001 From: Eugene Marushchenko Date: Wed, 11 Oct 2023 02:14:13 +1000 Subject: [PATCH 052/572] Move nested union optimization from plan builder to logical optimizer (#7695) * Add naive implementation of eliminate_nested_union * Remove union optimization from LogicalPlanBuilder::union * Fix propagate_union_children_different_schema test * Add implementation of eliminate_one_union * Simplified eliminate_nested_union test * Fix * clippy --------- Co-authored-by: Evgeny Maruschenko Co-authored-by: Andrew Lamb --- datafusion/expr/src/logical_plan/builder.rs | 67 +++--- .../optimizer/src/eliminate_nested_union.rs | 211 ++++++++++++++++++ .../optimizer/src/eliminate_one_union.rs | 118 ++++++++++ datafusion/optimizer/src/lib.rs | 2 + datafusion/optimizer/src/optimizer.rs | 5 + .../optimizer/src/propagate_empty_relation.rs | 31 +-- datafusion/optimizer/src/test/mod.rs | 19 ++ datafusion/sql/tests/sql_integration.rs | 18 -- .../sqllogictest/test_files/explain.slt | 4 + 9 files changed, 404 insertions(+), 71 deletions(-) create mode 100644 datafusion/optimizer/src/eliminate_nested_union.rs create mode 100644 datafusion/optimizer/src/eliminate_one_union.rs diff --git a/datafusion/expr/src/logical_plan/builder.rs b/datafusion/expr/src/logical_plan/builder.rs index 6171d43b37f5..3d62bcf55d6c 100644 --- a/datafusion/expr/src/logical_plan/builder.rs +++ b/datafusion/expr/src/logical_plan/builder.rs @@ -54,6 +54,7 @@ use std::any::Any; use std::cmp::Ordering; use std::collections::{HashMap, HashSet}; use std::convert::TryFrom; +use std::iter::zip; use std::sync::Arc; /// Default table name for unnamed table @@ -1196,39 +1197,36 @@ pub fn union(left_plan: LogicalPlan, right_plan: LogicalPlan) -> Result>>()? - .to_dfschema()?; + })?; + + Ok(DFField::new( + left_field.qualifier().cloned(), + left_field.name(), + data_type, + nullable, + )) + }) + .collect::>>()? + .to_dfschema()?; let inputs = vec![left_plan, right_plan] .into_iter() - .flat_map(|p| match p { - LogicalPlan::Union(Union { inputs, .. }) => inputs, - other_plan => vec![Arc::new(other_plan)], - }) .map(|p| { let plan = coerce_plan_expr_for_schema(&p, &union_schema)?; match plan { @@ -1596,7 +1594,7 @@ mod tests { } #[test] - fn plan_builder_union_combined_single_union() -> Result<()> { + fn plan_builder_union() -> Result<()> { let plan = table_scan(Some("employee_csv"), &employee_schema(), Some(vec![3, 4]))?; @@ -1607,11 +1605,12 @@ mod tests { .union(plan.build()?)? .build()?; - // output has only one union let expected = "Union\ - \n TableScan: employee_csv projection=[state, salary]\ - \n TableScan: employee_csv projection=[state, salary]\ - \n TableScan: employee_csv projection=[state, salary]\ + \n Union\ + \n Union\ + \n TableScan: employee_csv projection=[state, salary]\ + \n TableScan: employee_csv projection=[state, salary]\ + \n TableScan: employee_csv projection=[state, salary]\ \n TableScan: employee_csv projection=[state, salary]"; assert_eq!(expected, format!("{plan:?}")); @@ -1620,7 +1619,7 @@ mod tests { } #[test] - fn plan_builder_union_distinct_combined_single_union() -> Result<()> { + fn plan_builder_union_distinct() -> Result<()> { let plan = table_scan(Some("employee_csv"), &employee_schema(), Some(vec![3, 4]))?; diff --git a/datafusion/optimizer/src/eliminate_nested_union.rs b/datafusion/optimizer/src/eliminate_nested_union.rs new file mode 100644 index 000000000000..e22c73e5794d --- /dev/null +++ b/datafusion/optimizer/src/eliminate_nested_union.rs @@ -0,0 +1,211 @@ +// 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. + +//! Optimizer rule to replace nested unions to single union. +use crate::{OptimizerConfig, OptimizerRule}; +use datafusion_common::Result; +use datafusion_expr::logical_plan::{LogicalPlan, Union}; + +use crate::optimizer::ApplyOrder; +use datafusion_expr::expr_rewriter::coerce_plan_expr_for_schema; +use std::sync::Arc; + +#[derive(Default)] +/// An optimization rule that replaces nested unions with a single union. +pub struct EliminateNestedUnion; + +impl EliminateNestedUnion { + #[allow(missing_docs)] + pub fn new() -> Self { + Self {} + } +} + +impl OptimizerRule for EliminateNestedUnion { + fn try_optimize( + &self, + plan: &LogicalPlan, + _config: &dyn OptimizerConfig, + ) -> Result> { + // TODO: Add optimization for nested distinct unions. + match plan { + LogicalPlan::Union(Union { inputs, schema }) => { + let inputs = inputs + .iter() + .flat_map(|plan| match plan.as_ref() { + LogicalPlan::Union(Union { inputs, schema }) => inputs + .iter() + .map(|plan| { + Arc::new( + coerce_plan_expr_for_schema(plan, schema).unwrap(), + ) + }) + .collect::>(), + _ => vec![plan.clone()], + }) + .collect::>(); + + Ok(Some(LogicalPlan::Union(Union { + inputs, + schema: schema.clone(), + }))) + } + _ => Ok(None), + } + } + + fn name(&self) -> &str { + "eliminate_nested_union" + } + + fn apply_order(&self) -> Option { + Some(ApplyOrder::BottomUp) + } +} + +#[cfg(test)] +mod tests { + use super::*; + use crate::test::*; + use arrow::datatypes::{DataType, Field, Schema}; + use datafusion_expr::{col, logical_plan::table_scan}; + + fn schema() -> Schema { + Schema::new(vec![ + Field::new("id", DataType::Int32, false), + Field::new("key", DataType::Utf8, false), + Field::new("value", DataType::Float64, false), + ]) + } + + fn assert_optimized_plan_equal(plan: &LogicalPlan, expected: &str) -> Result<()> { + assert_optimized_plan_eq(Arc::new(EliminateNestedUnion::new()), plan, expected) + } + + #[test] + fn eliminate_nothing() -> Result<()> { + let plan_builder = table_scan(Some("table"), &schema(), None)?; + + let plan = plan_builder + .clone() + .union(plan_builder.clone().build()?)? + .build()?; + + let expected = "\ + Union\ + \n TableScan: table\ + \n TableScan: table"; + assert_optimized_plan_equal(&plan, expected) + } + + #[test] + fn eliminate_nested_union() -> Result<()> { + let plan_builder = table_scan(Some("table"), &schema(), None)?; + + let plan = plan_builder + .clone() + .union(plan_builder.clone().build()?)? + .union(plan_builder.clone().build()?)? + .union(plan_builder.clone().build()?)? + .build()?; + + let expected = "\ + Union\ + \n TableScan: table\ + \n TableScan: table\ + \n TableScan: table\ + \n TableScan: table"; + assert_optimized_plan_equal(&plan, expected) + } + + // We don't need to use project_with_column_index in logical optimizer, + // after LogicalPlanBuilder::union, we already have all equal expression aliases + #[test] + fn eliminate_nested_union_with_projection() -> Result<()> { + let plan_builder = table_scan(Some("table"), &schema(), None)?; + + let plan = plan_builder + .clone() + .union( + plan_builder + .clone() + .project(vec![col("id").alias("table_id"), col("key"), col("value")])? + .build()?, + )? + .union( + plan_builder + .clone() + .project(vec![col("id").alias("_id"), col("key"), col("value")])? + .build()?, + )? + .build()?; + + let expected = "Union\ + \n TableScan: table\ + \n Projection: table.id AS id, table.key, table.value\ + \n TableScan: table\ + \n Projection: table.id AS id, table.key, table.value\ + \n TableScan: table"; + assert_optimized_plan_equal(&plan, expected) + } + + #[test] + fn eliminate_nested_union_with_type_cast_projection() -> Result<()> { + let table_1 = table_scan( + Some("table_1"), + &Schema::new(vec![ + Field::new("id", DataType::Int64, false), + Field::new("key", DataType::Utf8, false), + Field::new("value", DataType::Float64, false), + ]), + None, + )?; + + let table_2 = table_scan( + Some("table_1"), + &Schema::new(vec![ + Field::new("id", DataType::Int32, false), + Field::new("key", DataType::Utf8, false), + Field::new("value", DataType::Float32, false), + ]), + None, + )?; + + let table_3 = table_scan( + Some("table_1"), + &Schema::new(vec![ + Field::new("id", DataType::Int16, false), + Field::new("key", DataType::Utf8, false), + Field::new("value", DataType::Float32, false), + ]), + None, + )?; + + let plan = table_1 + .union(table_2.build()?)? + .union(table_3.build()?)? + .build()?; + + let expected = "Union\ + \n TableScan: table_1\ + \n Projection: CAST(table_1.id AS Int64) AS id, table_1.key, CAST(table_1.value AS Float64) AS value\ + \n TableScan: table_1\ + \n Projection: CAST(table_1.id AS Int64) AS id, table_1.key, CAST(table_1.value AS Float64) AS value\ + \n TableScan: table_1"; + assert_optimized_plan_equal(&plan, expected) + } +} diff --git a/datafusion/optimizer/src/eliminate_one_union.rs b/datafusion/optimizer/src/eliminate_one_union.rs new file mode 100644 index 000000000000..70ee490346ff --- /dev/null +++ b/datafusion/optimizer/src/eliminate_one_union.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. + +//! Optimizer rule to eliminate one union. +use crate::{OptimizerConfig, OptimizerRule}; +use datafusion_common::Result; +use datafusion_expr::logical_plan::{LogicalPlan, Union}; + +use crate::optimizer::ApplyOrder; + +#[derive(Default)] +/// An optimization rule that eliminates union with one element. +pub struct EliminateOneUnion; + +impl EliminateOneUnion { + #[allow(missing_docs)] + pub fn new() -> Self { + Self {} + } +} + +impl OptimizerRule for EliminateOneUnion { + fn try_optimize( + &self, + plan: &LogicalPlan, + _config: &dyn OptimizerConfig, + ) -> Result> { + match plan { + LogicalPlan::Union(Union { inputs, .. }) if inputs.len() == 1 => { + Ok(inputs.first().map(|input| input.as_ref().clone())) + } + _ => Ok(None), + } + } + + fn name(&self) -> &str { + "eliminate_one_union" + } + + fn apply_order(&self) -> Option { + Some(ApplyOrder::TopDown) + } +} + +#[cfg(test)] +mod tests { + use super::*; + use crate::test::*; + use arrow::datatypes::{DataType, Field, Schema}; + use datafusion_common::ToDFSchema; + use datafusion_expr::{ + expr_rewriter::coerce_plan_expr_for_schema, + logical_plan::{table_scan, Union}, + }; + use std::sync::Arc; + + fn schema() -> Schema { + Schema::new(vec![ + Field::new("id", DataType::Int32, false), + Field::new("key", DataType::Utf8, false), + Field::new("value", DataType::Int32, false), + ]) + } + + fn assert_optimized_plan_equal(plan: &LogicalPlan, expected: &str) -> Result<()> { + assert_optimized_plan_eq_with_rules( + vec![Arc::new(EliminateOneUnion::new())], + plan, + expected, + ) + } + + #[test] + fn eliminate_nothing() -> Result<()> { + let plan_builder = table_scan(Some("table"), &schema(), None)?; + + let plan = plan_builder + .clone() + .union(plan_builder.clone().build()?)? + .build()?; + + let expected = "\ + Union\ + \n TableScan: table\ + \n TableScan: table"; + assert_optimized_plan_equal(&plan, expected) + } + + #[test] + fn eliminate_one_union() -> Result<()> { + let table_plan = coerce_plan_expr_for_schema( + &table_scan(Some("table"), &schema(), None)?.build()?, + &schema().to_dfschema()?, + )?; + let schema = table_plan.schema().clone(); + let single_union_plan = LogicalPlan::Union(Union { + inputs: vec![Arc::new(table_plan)], + schema, + }); + + let expected = "TableScan: table"; + assert_optimized_plan_equal(&single_union_plan, expected) + } +} diff --git a/datafusion/optimizer/src/lib.rs b/datafusion/optimizer/src/lib.rs index 1d12ca7e3950..ede0ac5c7164 100644 --- a/datafusion/optimizer/src/lib.rs +++ b/datafusion/optimizer/src/lib.rs @@ -24,6 +24,8 @@ pub mod eliminate_duplicated_expr; pub mod eliminate_filter; pub mod eliminate_join; pub mod eliminate_limit; +pub mod eliminate_nested_union; +pub mod eliminate_one_union; pub mod eliminate_outer_join; pub mod eliminate_project; pub mod extract_equijoin_predicate; diff --git a/datafusion/optimizer/src/optimizer.rs b/datafusion/optimizer/src/optimizer.rs index d3bdd47c5cb3..5231dc869875 100644 --- a/datafusion/optimizer/src/optimizer.rs +++ b/datafusion/optimizer/src/optimizer.rs @@ -24,6 +24,8 @@ use crate::eliminate_duplicated_expr::EliminateDuplicatedExpr; use crate::eliminate_filter::EliminateFilter; use crate::eliminate_join::EliminateJoin; use crate::eliminate_limit::EliminateLimit; +use crate::eliminate_nested_union::EliminateNestedUnion; +use crate::eliminate_one_union::EliminateOneUnion; use crate::eliminate_outer_join::EliminateOuterJoin; use crate::eliminate_project::EliminateProjection; use crate::extract_equijoin_predicate::ExtractEquijoinPredicate; @@ -220,6 +222,7 @@ impl Optimizer { /// Create a new optimizer using the recommended list of rules pub fn new() -> Self { let rules: Vec> = vec![ + Arc::new(EliminateNestedUnion::new()), Arc::new(SimplifyExpressions::new()), Arc::new(UnwrapCastInComparison::new()), Arc::new(ReplaceDistinctWithAggregate::new()), @@ -239,6 +242,8 @@ impl Optimizer { Arc::new(CommonSubexprEliminate::new()), Arc::new(EliminateLimit::new()), Arc::new(PropagateEmptyRelation::new()), + // Must be after PropagateEmptyRelation + Arc::new(EliminateOneUnion::new()), Arc::new(FilterNullJoinKeys::default()), Arc::new(EliminateOuterJoin::new()), // Filters can't be pushed down past Limits, we should do PushDownFilter after PushDownLimit diff --git a/datafusion/optimizer/src/propagate_empty_relation.rs b/datafusion/optimizer/src/propagate_empty_relation.rs index 4de7596b329c..040b69fc8bf3 100644 --- a/datafusion/optimizer/src/propagate_empty_relation.rs +++ b/datafusion/optimizer/src/propagate_empty_relation.rs @@ -182,12 +182,11 @@ fn empty_child(plan: &LogicalPlan) -> Result> { #[cfg(test)] mod tests { use crate::eliminate_filter::EliminateFilter; - use crate::optimizer::Optimizer; + use crate::eliminate_nested_union::EliminateNestedUnion; use crate::test::{ - assert_optimized_plan_eq, test_table_scan, test_table_scan_fields, - test_table_scan_with_name, + assert_optimized_plan_eq, assert_optimized_plan_eq_with_rules, test_table_scan, + test_table_scan_fields, test_table_scan_with_name, }; - use crate::OptimizerContext; use arrow::datatypes::{DataType, Field, Schema}; use datafusion_common::{Column, DFField, DFSchema, ScalarValue}; use datafusion_expr::logical_plan::table_scan; @@ -206,21 +205,15 @@ mod tests { plan: &LogicalPlan, expected: &str, ) -> Result<()> { - fn observe(_plan: &LogicalPlan, _rule: &dyn OptimizerRule) {} - let optimizer = Optimizer::with_rules(vec![ - Arc::new(EliminateFilter::new()), - Arc::new(PropagateEmptyRelation::new()), - ]); - let config = &mut OptimizerContext::new() - .with_max_passes(1) - .with_skip_failing_rules(false); - let optimized_plan = optimizer - .optimize(plan, config, observe) - .expect("failed to optimize plan"); - let formatted_plan = format!("{optimized_plan:?}"); - assert_eq!(formatted_plan, expected); - assert_eq!(plan.schema(), optimized_plan.schema()); - Ok(()) + assert_optimized_plan_eq_with_rules( + vec![ + Arc::new(EliminateFilter::new()), + Arc::new(EliminateNestedUnion::new()), + Arc::new(PropagateEmptyRelation::new()), + ], + plan, + expected, + ) } #[test] diff --git a/datafusion/optimizer/src/test/mod.rs b/datafusion/optimizer/src/test/mod.rs index 7d334a80b682..3eac2317b849 100644 --- a/datafusion/optimizer/src/test/mod.rs +++ b/datafusion/optimizer/src/test/mod.rs @@ -169,6 +169,25 @@ pub fn assert_optimized_plan_eq( Ok(()) } +pub fn assert_optimized_plan_eq_with_rules( + rules: Vec>, + plan: &LogicalPlan, + expected: &str, +) -> Result<()> { + fn observe(_plan: &LogicalPlan, _rule: &dyn OptimizerRule) {} + let config = &mut OptimizerContext::new() + .with_max_passes(1) + .with_skip_failing_rules(false); + let optimizer = Optimizer::with_rules(rules); + let optimized_plan = optimizer + .optimize(plan, config, observe) + .expect("failed to optimize plan"); + let formatted_plan = format!("{optimized_plan:?}"); + assert_eq!(formatted_plan, expected); + assert_eq!(plan.schema(), optimized_plan.schema()); + Ok(()) +} + pub fn assert_optimized_plan_eq_display_indent( rule: Arc, plan: &LogicalPlan, diff --git a/datafusion/sql/tests/sql_integration.rs b/datafusion/sql/tests/sql_integration.rs index b1de5a12bcd0..661890e12533 100644 --- a/datafusion/sql/tests/sql_integration.rs +++ b/datafusion/sql/tests/sql_integration.rs @@ -2061,24 +2061,6 @@ fn union_all() { quick_test(sql, expected); } -#[test] -fn union_4_combined_in_one() { - let sql = "SELECT order_id from orders - UNION ALL SELECT order_id FROM orders - UNION ALL SELECT order_id FROM orders - UNION ALL SELECT order_id FROM orders"; - let expected = "Union\ - \n Projection: orders.order_id\ - \n TableScan: orders\ - \n Projection: orders.order_id\ - \n TableScan: orders\ - \n Projection: orders.order_id\ - \n TableScan: orders\ - \n Projection: orders.order_id\ - \n TableScan: orders"; - quick_test(sql, expected); -} - #[test] fn union_with_different_column_names() { let sql = "SELECT order_id from orders UNION ALL SELECT customer_id FROM orders"; diff --git a/datafusion/sqllogictest/test_files/explain.slt b/datafusion/sqllogictest/test_files/explain.slt index 23055cd978ff..0e190a6acd62 100644 --- a/datafusion/sqllogictest/test_files/explain.slt +++ b/datafusion/sqllogictest/test_files/explain.slt @@ -184,6 +184,7 @@ logical_plan after inline_table_scan SAME TEXT AS ABOVE logical_plan after type_coercion SAME TEXT AS ABOVE logical_plan after count_wildcard_rule SAME TEXT AS ABOVE analyzed_logical_plan SAME TEXT AS ABOVE +logical_plan after eliminate_nested_union SAME TEXT AS ABOVE logical_plan after simplify_expressions SAME TEXT AS ABOVE logical_plan after unwrap_cast_in_comparison SAME TEXT AS ABOVE logical_plan after replace_distinct_aggregate SAME TEXT AS ABOVE @@ -200,6 +201,7 @@ logical_plan after eliminate_cross_join SAME TEXT AS ABOVE logical_plan after common_sub_expression_eliminate SAME TEXT AS ABOVE logical_plan after eliminate_limit SAME TEXT AS ABOVE logical_plan after propagate_empty_relation SAME TEXT AS ABOVE +logical_plan after eliminate_one_union SAME TEXT AS ABOVE logical_plan after filter_null_join_keys SAME TEXT AS ABOVE logical_plan after eliminate_outer_join SAME TEXT AS ABOVE logical_plan after push_down_limit SAME TEXT AS ABOVE @@ -213,6 +215,7 @@ Projection: simple_explain_test.a, simple_explain_test.b, simple_explain_test.c --TableScan: simple_explain_test projection=[a, b, c] logical_plan after eliminate_projection TableScan: simple_explain_test projection=[a, b, c] logical_plan after push_down_limit SAME TEXT AS ABOVE +logical_plan after eliminate_nested_union SAME TEXT AS ABOVE logical_plan after simplify_expressions SAME TEXT AS ABOVE logical_plan after unwrap_cast_in_comparison SAME TEXT AS ABOVE logical_plan after replace_distinct_aggregate SAME TEXT AS ABOVE @@ -229,6 +232,7 @@ logical_plan after eliminate_cross_join SAME TEXT AS ABOVE logical_plan after common_sub_expression_eliminate SAME TEXT AS ABOVE logical_plan after eliminate_limit SAME TEXT AS ABOVE logical_plan after propagate_empty_relation SAME TEXT AS ABOVE +logical_plan after eliminate_one_union SAME TEXT AS ABOVE logical_plan after filter_null_join_keys SAME TEXT AS ABOVE logical_plan after eliminate_outer_join SAME TEXT AS ABOVE logical_plan after push_down_limit SAME TEXT AS ABOVE From 94153b4b0916b371bf869f6101cfa16aa11051db Mon Sep 17 00:00:00 2001 From: Andrew Lamb Date: Tue, 10 Oct 2023 13:40:39 -0400 Subject: [PATCH 053/572] Minor: comments that explain the schema used in simply_expressions (#7747) * Minor: comments that explain the schema used in simply_expressions * Apply suggestions from code review Co-authored-by: comphead --------- Co-authored-by: comphead --- .../src/simplify_expressions/simplify_exprs.rs | 12 ++++++++++-- 1 file changed, 10 insertions(+), 2 deletions(-) diff --git a/datafusion/optimizer/src/simplify_expressions/simplify_exprs.rs b/datafusion/optimizer/src/simplify_expressions/simplify_exprs.rs index 355d556d5da4..9dc83e0fadf5 100644 --- a/datafusion/optimizer/src/simplify_expressions/simplify_exprs.rs +++ b/datafusion/optimizer/src/simplify_expressions/simplify_exprs.rs @@ -66,8 +66,16 @@ impl SimplifyExpressions { let schema = if !plan.inputs().is_empty() { DFSchemaRef::new(merge_schema(plan.inputs())) } else if let LogicalPlan::TableScan(scan) = plan { - // When predicates are pushed into a table scan, there needs to be - // a schema to resolve the fields against. + // When predicates are pushed into a table scan, there is no input + // schema to resolve predicates against, so it must be handled specially + // + // Note that this is not `plan.schema()` which is the *output* + // schema, and reflects any pushed down projection. The output schema + // will not contain columns that *only* appear in pushed down predicates + // (and no where else) in the plan. + // + // Thus, use the full schema of the inner provider without any + // projection applied for simplification Arc::new(DFSchema::try_from_qualified_schema( &scan.table_name, &scan.source.schema(), From 92ba6c3cc1a801762bd157f88f997f70b5123084 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Tue, 10 Oct 2023 13:54:27 -0400 Subject: [PATCH 054/572] Update regex-syntax requirement from 0.7.1 to 0.8.0 (#7784) * Update regex-syntax requirement from 0.7.1 to 0.8.0 Updates the requirements on [regex-syntax](https://github.com/rust-lang/regex) to permit the latest version. - [Release notes](https://github.com/rust-lang/regex/releases) - [Changelog](https://github.com/rust-lang/regex/blob/master/CHANGELOG.md) - [Commits](https://github.com/rust-lang/regex/commits) --- updated-dependencies: - dependency-name: regex-syntax dependency-type: direct:production ... Signed-off-by: dependabot[bot] * Update datafusion-cli Cargo.lock * update datafusion-cli lock again --------- Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> Co-authored-by: Andrew Lamb --- datafusion-cli/Cargo.lock | 131 ++++++++++++++++++-------------- datafusion/optimizer/Cargo.toml | 2 +- 2 files changed, 77 insertions(+), 56 deletions(-) diff --git a/datafusion-cli/Cargo.lock b/datafusion-cli/Cargo.lock index 54e9a5ccc2dc..f695e81d9876 100644 --- a/datafusion-cli/Cargo.lock +++ b/datafusion-cli/Cargo.lock @@ -38,9 +38,9 @@ dependencies = [ [[package]] name = "aho-corasick" -version = "1.1.1" +version = "1.1.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "ea5d730647d4fadd988536d06fecce94b7b4f2a7efdae548f1cf4b63205518ab" +checksum = "b2969dcb958b36655471fc61f7e416fa76033bdd4bfed0678d8fee1e2d07a1f0" dependencies = [ "memchr", ] @@ -338,7 +338,7 @@ dependencies = [ "arrow-select", "num", "regex", - "regex-syntax", + "regex-syntax 0.7.5", ] [[package]] @@ -424,7 +424,7 @@ dependencies = [ "hex", "http", "hyper", - "ring", + "ring 0.16.20", "time", "tokio", "tower", @@ -789,9 +789,9 @@ dependencies = [ [[package]] name = "bstr" -version = "1.6.2" +version = "1.7.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "4c2f7349907b712260e64b0afe2f84692af14a454be26187d9df565c7f69266a" +checksum = "c79ad7fb2dd38f3dabd76b09c6a5a20c038fc0213ef1e9afd30eb777f120f019" dependencies = [ "memchr", "regex-automata", @@ -1232,7 +1232,7 @@ dependencies = [ "hashbrown 0.14.1", "itertools", "log", - "regex-syntax", + "regex-syntax 0.8.0", ] [[package]] @@ -1420,25 +1420,14 @@ checksum = "5443807d6dff69373d433ab9ef5378ad8df50ca6298caf15de6e52e24aaf54d5" [[package]] name = "errno" -version = "0.3.4" +version = "0.3.5" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "add4f07d43996f76ef320709726a556a9d4f965d9410d8d0271132d2f8293480" +checksum = "ac3e13f66a2f95e32a39eaa81f6b95d42878ca0e1db0c7543723dfe12557e860" dependencies = [ - "errno-dragonfly", "libc", "windows-sys", ] -[[package]] -name = "errno-dragonfly" -version = "0.1.2" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "aa68f1b12764fab894d2755d2518754e71b4fd80ecfb822714a1206c2aab39bf" -dependencies = [ - "cc", - "libc", -] - [[package]] name = "error-code" version = "2.3.1" @@ -1928,9 +1917,9 @@ checksum = "af150ab688ff2122fcef229be89cb50dd66af9e01a4ff320cc137eecc9bacc38" [[package]] name = "jobserver" -version = "0.1.26" +version = "0.1.27" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "936cfd212a0155903bcbc060e316fb6cc7cbf2e1907329391ebadc1fe0ce77c2" +checksum = "8c37f63953c4c63420ed5fd3d6d398c719489b9f872b9fa683262f8edd363c7d" dependencies = [ "libc", ] @@ -2062,9 +2051,9 @@ dependencies = [ [[package]] name = "linux-raw-sys" -version = "0.4.8" +version = "0.4.10" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "3852614a3bd9ca9804678ba6be5e3b8ce76dfc902cae004e3e0c44051b6e88db" +checksum = "da2479e8c062e40bf0066ffa0bc823de0a9368974af99c9f6df941d2c231e03f" [[package]] name = "lock_api" @@ -2259,9 +2248,9 @@ dependencies = [ [[package]] name = "num-traits" -version = "0.2.16" +version = "0.2.17" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "f30b0abd723be7e2ffca1272140fac1a2f084c77ec3e123c192b66af1ee9e6c2" +checksum = "39e3200413f237f41ab11ad6d161bc7239c84dcb631773ccd7de3dfe4b5c267c" dependencies = [ "autocfg", "libm", @@ -2305,7 +2294,7 @@ dependencies = [ "quick-xml", "rand", "reqwest", - "ring", + "ring 0.16.20", "rustls-pemfile", "serde", "serde_json", @@ -2582,9 +2571,9 @@ checksum = "dc375e1527247fe1a97d8b7156678dfe7c1af2fc075c9a4db3690ecd2a148068" [[package]] name = "proc-macro2" -version = "1.0.68" +version = "1.0.69" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "5b1106fec09662ec6dd98ccac0f81cef56984d0b49f75c92d8cbad76e20c005c" +checksum = "134c189feb4956b20f6f547d2cf727d4c0fe06722b20a0eec87ed445a97f92da" dependencies = [ "unicode-ident", ] @@ -2685,32 +2674,32 @@ dependencies = [ [[package]] name = "regex" -version = "1.9.6" +version = "1.10.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "ebee201405406dbf528b8b672104ae6d6d63e6d118cb10e4d51abbc7b58044ff" +checksum = "d119d7c7ca818f8a53c300863d4f87566aac09943aef5b355bb83969dae75d87" dependencies = [ "aho-corasick", "memchr", "regex-automata", - "regex-syntax", + "regex-syntax 0.8.0", ] [[package]] name = "regex-automata" -version = "0.3.9" +version = "0.4.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "59b23e92ee4318893fa3fe3e6fb365258efbfe6ac6ab30f090cdcbb7aa37efa9" +checksum = "465c6fc0621e4abc4187a2bda0937bfd4f722c2730b29562e19689ea796c9a4b" dependencies = [ "aho-corasick", "memchr", - "regex-syntax", + "regex-syntax 0.8.0", ] [[package]] name = "regex-lite" -version = "0.1.0" +version = "0.1.3" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "f96ede7f386ba6e910092e7ccdc04176cface62abebea07ed6b46d870ed95ca2" +checksum = "9a6ebcd15653947e6140f59a9811a06ed061d18a5c35dfca2e2e4c5525696878" [[package]] name = "regex-syntax" @@ -2718,6 +2707,12 @@ version = "0.7.5" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "dbb5fb1acd8a1a18b3dd5be62d25485eb770e05afb408a9627d14d451bae12da" +[[package]] +name = "regex-syntax" +version = "0.8.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "c3cbb081b9784b07cceb8824c8583f86db4814d172ab043f3c23f7dc600bf83d" + [[package]] name = "reqwest" version = "0.11.22" @@ -2769,12 +2764,26 @@ dependencies = [ "cc", "libc", "once_cell", - "spin", - "untrusted", + "spin 0.5.2", + "untrusted 0.7.1", "web-sys", "winapi", ] +[[package]] +name = "ring" +version = "0.17.3" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "9babe80d5c16becf6594aa32ad2be8fe08498e7ae60b77de8df700e67f191d7e" +dependencies = [ + "cc", + "getrandom", + "libc", + "spin 0.9.8", + "untrusted 0.9.0", + "windows-sys", +] + [[package]] name = "rle-decode-fast" version = "1.0.3" @@ -2824,9 +2833,9 @@ dependencies = [ [[package]] name = "rustix" -version = "0.38.17" +version = "0.38.18" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "f25469e9ae0f3d0047ca8b93fc56843f38e6774f0914a107ff8b41be8be8e0b7" +checksum = "5a74ee2d7c2581cd139b42447d7d9389b889bdaad3a73f1ebb16f2a3237bb19c" dependencies = [ "bitflags 2.4.0", "errno", @@ -2842,7 +2851,7 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "1b80e3dec595989ea8510028f30c408a4630db12c9cbb8de34203b89d6577e99" dependencies = [ "log", - "ring", + "ring 0.16.20", "sct", "webpki", ] @@ -2854,7 +2863,7 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "cd8d6c9f025a446bc4d18ad9632e69aec8f287aa84499ee335599fabd20c3fd8" dependencies = [ "log", - "ring", + "ring 0.16.20", "rustls-webpki", "sct", ] @@ -2886,8 +2895,8 @@ version = "0.101.6" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "3c7d5dece342910d9ba34d259310cae3e0154b873b35408b787b59bce53d34fe" dependencies = [ - "ring", - "untrusted", + "ring 0.16.20", + "untrusted 0.7.1", ] [[package]] @@ -2955,8 +2964,8 @@ version = "0.7.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "d53dcdb7c9f8158937a7981b48accfd39a43af418591a5d008c7b22b5e1b7ca4" dependencies = [ - "ring", - "untrusted", + "ring 0.16.20", + "untrusted 0.7.1", ] [[package]] @@ -2984,9 +2993,9 @@ dependencies = [ [[package]] name = "semver" -version = "1.0.19" +version = "1.0.20" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "ad977052201c6de01a8ef2aa3378c4bd23217a056337d1d6da40468d267a4fb0" +checksum = "836fa6a3e1e547f9a2c4040802ec865b5d85f4014efe00555d7090a3dcaa1090" [[package]] name = "seq-macro" @@ -3123,6 +3132,12 @@ version = "0.5.2" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "6e63cff320ae2c57904679ba7cb63280a3dc4613885beafb148ee7bf9aa9042d" +[[package]] +name = "spin" +version = "0.9.8" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "6980e8d7511241f8acf4aebddbb1ff938df5eebe98691418c4468d0b72a96a67" + [[package]] name = "sqlparser" version = "0.38.0" @@ -3370,9 +3385,9 @@ checksum = "1f3ccbac311fea05f86f61904b462b55fb3df8837a366dfc601a0161d0532f20" [[package]] name = "tokio" -version = "1.32.0" +version = "1.33.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "17ed6077ed6cd6c74735e21f37eb16dc3935f96878b1fe961074089cc80893f9" +checksum = "4f38200e3ef7995e5ef13baec2f432a6da0aa9ac495b2c0e8f3b7eec2c92d653" dependencies = [ "backtrace", "bytes", @@ -3585,6 +3600,12 @@ version = "0.7.1" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "a156c684c91ea7d62626509bce3cb4e1d9ed5c4d978f7b4352658f96a4c26b4a" +[[package]] +name = "untrusted" +version = "0.9.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "8ecb6da28b8a351d773b68d5825ac39017e680750f980f3a1a85cd8dd28a47c1" + [[package]] name = "url" version = "2.4.1" @@ -3755,12 +3776,12 @@ dependencies = [ [[package]] name = "webpki" -version = "0.22.2" +version = "0.22.4" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "07ecc0cd7cac091bf682ec5efa18b1cff79d617b84181f38b3951dbe135f607f" +checksum = "ed63aea5ce73d0ff405984102c42de94fc55a6b75765d621c65262469b3c9b53" dependencies = [ - "ring", - "untrusted", + "ring 0.17.3", + "untrusted 0.9.0", ] [[package]] diff --git a/datafusion/optimizer/Cargo.toml b/datafusion/optimizer/Cargo.toml index d60b229716e0..bf786686f474 100644 --- a/datafusion/optimizer/Cargo.toml +++ b/datafusion/optimizer/Cargo.toml @@ -48,7 +48,7 @@ datafusion-physical-expr = { path = "../physical-expr", version = "32.0.0", defa hashbrown = { version = "0.14", features = ["raw"] } itertools = "0.11" log = "^0.4" -regex-syntax = "0.7.1" +regex-syntax = "0.8.0" [dev-dependencies] ctor = "0.2.0" From 2a421757216c1779ea14426485e3fa7e0ee988c8 Mon Sep 17 00:00:00 2001 From: Andrew Lamb Date: Wed, 11 Oct 2023 10:14:21 -0400 Subject: [PATCH 055/572] Minor: Add sql test for `UNION` / `UNION ALL` + plans (#7787) --- datafusion/sqllogictest/test_files/union.slt | 74 ++++++++++++++++++++ 1 file changed, 74 insertions(+) diff --git a/datafusion/sqllogictest/test_files/union.slt b/datafusion/sqllogictest/test_files/union.slt index 05eaa10dabde..b11a687d8b9f 100644 --- a/datafusion/sqllogictest/test_files/union.slt +++ b/datafusion/sqllogictest/test_files/union.slt @@ -174,6 +174,80 @@ UNION ALL Alice John +# nested_union +query T rowsort +SELECT name FROM t1 UNION (SELECT name from t2 UNION SELECT name || '_new' from t2) +---- +Alex +Alex_new +Alice +Bob +Bob_new +John +John_new + +# should be un-nested +# https://github.com/apache/arrow-datafusion/issues/7786 +query TT +EXPLAIN SELECT name FROM t1 UNION (SELECT name from t2 UNION SELECT name || '_new' from t2) +---- +logical_plan +Aggregate: groupBy=[[t1.name]], aggr=[[]] +--Union +----TableScan: t1 projection=[name] +----Aggregate: groupBy=[[t2.name]], aggr=[[]] +------Union +--------TableScan: t2 projection=[name] +--------Projection: t2.name || Utf8("_new") AS name +----------TableScan: t2 projection=[name] +physical_plan +AggregateExec: mode=FinalPartitioned, gby=[name@0 as name], aggr=[] +--CoalesceBatchesExec: target_batch_size=8192 +----RepartitionExec: partitioning=Hash([name@0], 4), input_partitions=8 +------AggregateExec: mode=Partial, gby=[name@0 as name], aggr=[] +--------UnionExec +----------MemoryExec: partitions=4, partition_sizes=[1, 0, 0, 0] +----------AggregateExec: mode=FinalPartitioned, gby=[name@0 as name], aggr=[] +------------CoalesceBatchesExec: target_batch_size=8192 +--------------RepartitionExec: partitioning=Hash([name@0], 4), input_partitions=8 +----------------AggregateExec: mode=Partial, gby=[name@0 as name], aggr=[] +------------------UnionExec +--------------------MemoryExec: partitions=4, partition_sizes=[1, 0, 0, 0] +--------------------ProjectionExec: expr=[name@0 || _new as name] +----------------------MemoryExec: partitions=4, partition_sizes=[1, 0, 0, 0] + +# nested_union_all +query T rowsort +SELECT name FROM t1 UNION ALL (SELECT name from t2 UNION ALL SELECT name || '_new' from t2) +---- +Alex +Alex +Alex_new +Alice +Bob +Bob +Bob_new +John +John_new + +# Plan is unnested +query TT +EXPLAIN SELECT name FROM t1 UNION ALL (SELECT name from t2 UNION ALL SELECT name || '_new' from t2) +---- +logical_plan +Union +--TableScan: t1 projection=[name] +--TableScan: t2 projection=[name] +--Projection: t2.name || Utf8("_new") AS name +----TableScan: t2 projection=[name] +physical_plan +UnionExec +--MemoryExec: partitions=4, partition_sizes=[1, 0, 0, 0] +--MemoryExec: partitions=4, partition_sizes=[1, 0, 0, 0] +--ProjectionExec: expr=[name@0 || _new as name] +----MemoryExec: partitions=4, partition_sizes=[1, 0, 0, 0] + + # union_with_type_coercion query TT explain From 1bfe7404fd2c08d9a8717ccc8f04063a6e254b8e Mon Sep 17 00:00:00 2001 From: Jonah Gao Date: Thu, 12 Oct 2023 04:26:45 +0800 Subject: [PATCH 056/572] fix: preserve column qualifier for `DataFrame::with_column` (#7792) * fix: preserve column qualifier for `DataFrame::with_column` * fix test variable * review feedback: add self join test --- datafusion/core/src/dataframe.rs | 130 ++++++++++++++++++++++++++++++- 1 file changed, 126 insertions(+), 4 deletions(-) diff --git a/datafusion/core/src/dataframe.rs b/datafusion/core/src/dataframe.rs index f6490801126f..a06722f92cdb 100644 --- a/datafusion/core/src/dataframe.rs +++ b/datafusion/core/src/dataframe.rs @@ -1144,10 +1144,7 @@ impl DataFrame { col_exists = true; new_column.clone() } else { - Expr::Column(Column { - relation: None, - name: f.name().into(), - }) + col(f.qualified_column()) } }) .collect(); @@ -1855,6 +1852,131 @@ mod tests { Ok(()) } + // Test issue: https://github.com/apache/arrow-datafusion/issues/7790 + // The join operation outputs two identical column names, but they belong to different relations. + #[tokio::test] + async fn with_column_join_same_columns() -> Result<()> { + let df = test_table().await?.select_columns(&["c1"])?; + let ctx = SessionContext::new(); + + let table = df.into_view(); + ctx.register_table("t1", table.clone())?; + ctx.register_table("t2", table)?; + let df = ctx + .table("t1") + .await? + .join( + ctx.table("t2").await?, + JoinType::Inner, + &["c1"], + &["c1"], + None, + )? + .sort(vec![ + // make the test deterministic + col("t1.c1").sort(true, true), + ])? + .limit(0, Some(1))?; + + let df_results = df.clone().collect().await?; + assert_batches_sorted_eq!( + [ + "+----+----+", + "| c1 | c1 |", + "+----+----+", + "| a | a |", + "+----+----+", + ], + &df_results + ); + + let df_with_column = df.clone().with_column("new_column", lit(true))?; + + assert_eq!( + "\ + Projection: t1.c1, t2.c1, Boolean(true) AS new_column\ + \n Limit: skip=0, fetch=1\ + \n Sort: t1.c1 ASC NULLS FIRST\ + \n Inner Join: t1.c1 = t2.c1\ + \n TableScan: t1\ + \n TableScan: t2", + format!("{:?}", df_with_column.logical_plan()) + ); + + assert_eq!( + "\ + Projection: t1.c1, t2.c1, Boolean(true) AS new_column\ + \n Limit: skip=0, fetch=1\ + \n Sort: t1.c1 ASC NULLS FIRST, fetch=1\ + \n Inner Join: t1.c1 = t2.c1\ + \n SubqueryAlias: t1\ + \n TableScan: aggregate_test_100 projection=[c1]\ + \n SubqueryAlias: t2\ + \n TableScan: aggregate_test_100 projection=[c1]", + format!("{:?}", df_with_column.clone().into_optimized_plan()?) + ); + + let df_results = df_with_column.collect().await?; + + assert_batches_sorted_eq!( + [ + "+----+----+------------+", + "| c1 | c1 | new_column |", + "+----+----+------------+", + "| a | a | true |", + "+----+----+------------+", + ], + &df_results + ); + Ok(()) + } + + // Table 't1' self join + // Supplementary test of issue: https://github.com/apache/arrow-datafusion/issues/7790 + #[tokio::test] + async fn with_column_self_join() -> Result<()> { + let df = test_table().await?.select_columns(&["c1"])?; + let ctx = SessionContext::new(); + + ctx.register_table("t1", df.into_view())?; + + let df = ctx + .table("t1") + .await? + .join( + ctx.table("t1").await?, + JoinType::Inner, + &["c1"], + &["c1"], + None, + )? + .sort(vec![ + // make the test deterministic + col("t1.c1").sort(true, true), + ])? + .limit(0, Some(1))?; + + let df_results = df.clone().collect().await?; + assert_batches_sorted_eq!( + [ + "+----+----+", + "| c1 | c1 |", + "+----+----+", + "| a | a |", + "+----+----+", + ], + &df_results + ); + + let actual_err = df.clone().with_column("new_column", lit(true)).unwrap_err(); + let expected_err = "Error during planning: Projections require unique expression names \ + but the expression \"t1.c1\" at position 0 and \"t1.c1\" at position 1 have the same name. \ + Consider aliasing (\"AS\") one of them."; + assert_eq!(actual_err.strip_backtrace(), expected_err); + + Ok(()) + } + #[tokio::test] async fn with_column_renamed() -> Result<()> { let df = test_table() From 1f4442e0fb427b06a78854d39b07d68743b94d2a Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Berkay=20=C5=9Eahin?= <124376117+berkaysynnada@users.noreply.github.com> Date: Thu, 12 Oct 2023 18:34:51 +0300 Subject: [PATCH 057/572] Interval Arithmetic NegativeExpr Support (#7804) * Intervals can propagate over NegativeExpr's. * Addressing doc string reviews --- datafusion/common/src/scalar.rs | 3 +- .../src/physical_optimizer/join_selection.rs | 13 +++- .../physical_optimizer/pipeline_checker.rs | 2 +- .../physical-expr/src/expressions/negative.rs | 73 +++++++++++++++++-- .../src/intervals/interval_aritmetic.rs | 20 +++-- .../physical-expr/src/intervals/utils.rs | 39 +++++++--- datafusion/physical-plan/src/filter.rs | 2 +- 7 files changed, 120 insertions(+), 32 deletions(-) diff --git a/datafusion/common/src/scalar.rs b/datafusion/common/src/scalar.rs index ae9c29b5a141..242d784edc9d 100644 --- a/datafusion/common/src/scalar.rs +++ b/datafusion/common/src/scalar.rs @@ -1004,7 +1004,8 @@ impl ScalarValue { | ScalarValue::Int16(None) | ScalarValue::Int32(None) | ScalarValue::Int64(None) - | ScalarValue::Float32(None) => Ok(self.clone()), + | ScalarValue::Float32(None) + | ScalarValue::Float64(None) => Ok(self.clone()), ScalarValue::Float64(Some(v)) => Ok(ScalarValue::Float64(Some(-v))), ScalarValue::Float32(Some(v)) => Ok(ScalarValue::Float32(Some(-v))), ScalarValue::Int8(Some(v)) => Ok(ScalarValue::Int8(Some(-v))), diff --git a/datafusion/core/src/physical_optimizer/join_selection.rs b/datafusion/core/src/physical_optimizer/join_selection.rs index 4cff4a8f6c55..3d5c93d4b360 100644 --- a/datafusion/core/src/physical_optimizer/join_selection.rs +++ b/datafusion/core/src/physical_optimizer/join_selection.rs @@ -1173,6 +1173,7 @@ mod tests_statistical { #[cfg(test)] mod util_tests { + use arrow_schema::{DataType, Field, Schema}; use datafusion_expr::Operator; use datafusion_physical_expr::expressions::{BinaryExpr, Column, NegativeExpr}; use datafusion_physical_expr::intervals::utils::check_support; @@ -1181,26 +1182,30 @@ mod util_tests { #[test] fn check_expr_supported() { + let schema = Arc::new(Schema::new(vec![ + Field::new("a", DataType::Int32, false), + Field::new("b", DataType::Utf8, false), + ])); let supported_expr = Arc::new(BinaryExpr::new( Arc::new(Column::new("a", 0)), Operator::Plus, Arc::new(Column::new("a", 0)), )) as Arc; - assert!(check_support(&supported_expr)); + assert!(check_support(&supported_expr, &schema)); let supported_expr_2 = Arc::new(Column::new("a", 0)) as Arc; - assert!(check_support(&supported_expr_2)); + assert!(check_support(&supported_expr_2, &schema)); let unsupported_expr = Arc::new(BinaryExpr::new( Arc::new(Column::new("a", 0)), Operator::Or, Arc::new(Column::new("a", 0)), )) as Arc; - assert!(!check_support(&unsupported_expr)); + assert!(!check_support(&unsupported_expr, &schema)); let unsupported_expr_2 = Arc::new(BinaryExpr::new( Arc::new(Column::new("a", 0)), Operator::Or, Arc::new(NegativeExpr::new(Arc::new(Column::new("a", 0)))), )) as Arc; - assert!(!check_support(&unsupported_expr_2)); + assert!(!check_support(&unsupported_expr_2, &schema)); } } diff --git a/datafusion/core/src/physical_optimizer/pipeline_checker.rs b/datafusion/core/src/physical_optimizer/pipeline_checker.rs index 3b994d5f893f..fbc991017911 100644 --- a/datafusion/core/src/physical_optimizer/pipeline_checker.rs +++ b/datafusion/core/src/physical_optimizer/pipeline_checker.rs @@ -163,7 +163,7 @@ pub fn check_finiteness_requirements( /// [`Operator`]: datafusion_expr::Operator fn is_prunable(join: &SymmetricHashJoinExec) -> bool { join.filter().map_or(false, |filter| { - check_support(filter.expression()) + check_support(filter.expression(), &join.schema()) && filter .schema() .fields() diff --git a/datafusion/physical-expr/src/expressions/negative.rs b/datafusion/physical-expr/src/expressions/negative.rs index 90430cb2bbda..c66c91ef3152 100644 --- a/datafusion/physical-expr/src/expressions/negative.rs +++ b/datafusion/physical-expr/src/expressions/negative.rs @@ -17,26 +17,25 @@ //! Negation (-) expression -use std::any::Any; -use std::hash::{Hash, Hasher}; -use std::sync::Arc; - +use crate::intervals::Interval; use crate::physical_expr::down_cast_any_ref; use crate::sort_properties::SortProperties; use crate::PhysicalExpr; - use arrow::{ compute::kernels::numeric::neg_wrapping, datatypes::{DataType, Schema}, record_batch::RecordBatch, }; - use datafusion_common::{internal_err, DataFusionError, Result}; use datafusion_expr::{ type_coercion::{is_interval, is_null, is_signed_numeric}, ColumnarValue, }; +use std::any::Any; +use std::hash::{Hash, Hasher}; +use std::sync::Arc; + /// Negative expression #[derive(Debug, Hash)] pub struct NegativeExpr { @@ -105,6 +104,30 @@ impl PhysicalExpr for NegativeExpr { self.hash(&mut s); } + /// Given the child interval of a NegativeExpr, it calculates the NegativeExpr's interval. + /// It replaces the upper and lower bounds after multiplying them with -1. + /// Ex: `(a, b] => [-b, -a)`` + fn evaluate_bounds(&self, children: &[&Interval]) -> Result { + Ok(Interval::new( + children[0].upper.negate()?, + children[0].lower.negate()?, + )) + } + + /// Returns a new [`Interval`] of a NegativeExpr that has the existing `interval` given that + /// given the input interval is known to be `children`. + fn propagate_constraints( + &self, + interval: &Interval, + children: &[&Interval], + ) -> Result>> { + let child_interval = children[0]; + let negated_interval = + Interval::new(interval.upper.negate()?, interval.lower.negate()?); + + Ok(vec![child_interval.intersect(negated_interval)?]) + } + /// The ordering of a [`NegativeExpr`] is simply the reverse of its child. fn get_ordering(&self, children: &[SortProperties]) -> SortProperties { -children[0] @@ -144,7 +167,10 @@ pub fn negative( #[cfg(test)] mod tests { use super::*; - use crate::expressions::col; + use crate::{ + expressions::{col, Column}, + intervals::Interval, + }; #[allow(unused_imports)] use arrow::array::*; use arrow::datatypes::*; @@ -187,4 +213,37 @@ mod tests { test_array_negative_op!(Float64, 23456.0f64, 12345.0f64); Ok(()) } + + #[test] + fn test_evaluate_bounds() -> Result<()> { + let negative_expr = NegativeExpr { + arg: Arc::new(Column::new("a", 0)), + }; + let child_interval = Interval::make(Some(-2), Some(1), (true, false)); + let negative_expr_interval = Interval::make(Some(-1), Some(2), (false, true)); + assert_eq!( + negative_expr.evaluate_bounds(&[&child_interval])?, + negative_expr_interval + ); + Ok(()) + } + + #[test] + fn test_propagate_constraints() -> Result<()> { + let negative_expr = NegativeExpr { + arg: Arc::new(Column::new("a", 0)), + }; + let original_child_interval = Interval::make(Some(-2), Some(3), (false, false)); + let negative_expr_interval = Interval::make(Some(0), Some(4), (true, false)); + let after_propagation = + vec![Some(Interval::make(Some(-2), Some(0), (false, true)))]; + assert_eq!( + negative_expr.propagate_constraints( + &negative_expr_interval, + &[&original_child_interval] + )?, + after_propagation + ); + Ok(()) + } } diff --git a/datafusion/physical-expr/src/intervals/interval_aritmetic.rs b/datafusion/physical-expr/src/intervals/interval_aritmetic.rs index 210db0ebe6a2..277124ea223c 100644 --- a/datafusion/physical-expr/src/intervals/interval_aritmetic.rs +++ b/datafusion/physical-expr/src/intervals/interval_aritmetic.rs @@ -17,14 +17,8 @@ //! Interval arithmetic library -use std::borrow::Borrow; -use std::fmt; -use std::fmt::{Display, Formatter}; -use std::ops::{AddAssign, SubAssign}; - use crate::aggregate::min_max::{max, min}; use crate::intervals::rounding::{alter_fp_rounding_mode, next_down, next_up}; - use arrow::compute::{cast_with_options, CastOptions}; use arrow::datatypes::DataType; use arrow_array::ArrowNativeTypeOp; @@ -32,6 +26,11 @@ use datafusion_common::{exec_err, internal_err, DataFusionError, Result, ScalarV use datafusion_expr::type_coercion::binary::get_result_type; use datafusion_expr::Operator; +use std::borrow::Borrow; +use std::fmt; +use std::fmt::{Display, Formatter}; +use std::ops::{AddAssign, SubAssign}; + /// This type represents a single endpoint of an [`Interval`]. An /// endpoint can be open (does not include the endpoint) or closed /// (includes the endpoint). @@ -87,6 +86,15 @@ impl IntervalBound { .map(|value| IntervalBound::new(value, self.open)) } + /// Returns a new bound with a negated value, if any, and the same open/closed. + /// For example negating `[5` would return `[-5`, or `-1)` would return `1)` + pub fn negate(&self) -> Result { + self.value.arithmetic_negate().map(|value| IntervalBound { + value, + open: self.open, + }) + } + /// This function adds the given `IntervalBound` to this `IntervalBound`. /// The result is unbounded if either is; otherwise, their values are /// added. The result is closed if both original bounds are closed, or open diff --git a/datafusion/physical-expr/src/intervals/utils.rs b/datafusion/physical-expr/src/intervals/utils.rs index be3b17771303..2ddbca40733c 100644 --- a/datafusion/physical-expr/src/intervals/utils.rs +++ b/datafusion/physical-expr/src/intervals/utils.rs @@ -17,18 +17,17 @@ //! Utility functions for the interval arithmetic library -use std::sync::Arc; - use super::{Interval, IntervalBound}; use crate::{ - expressions::{BinaryExpr, CastExpr, Column, Literal}, + expressions::{BinaryExpr, CastExpr, Column, Literal, NegativeExpr}, PhysicalExpr, }; - -use arrow_schema::DataType; +use arrow_schema::{DataType, SchemaRef}; use datafusion_common::{DataFusionError, Result, ScalarValue}; use datafusion_expr::Operator; +use std::sync::Arc; + const MDN_DAY_MASK: i128 = 0xFFFF_FFFF_0000_0000_0000_0000; const MDN_NS_MASK: i128 = 0xFFFF_FFFF_FFFF_FFFF; const DT_MS_MASK: i64 = 0xFFFF_FFFF; @@ -37,16 +36,32 @@ const DT_MS_MASK: i64 = 0xFFFF_FFFF; /// Currently, we do not support all [`PhysicalExpr`]s for interval calculations. /// We do not support every type of [`Operator`]s either. Over time, this check /// will relax as more types of `PhysicalExpr`s and `Operator`s are supported. -/// Currently, [`CastExpr`], [`BinaryExpr`], [`Column`] and [`Literal`] are supported. -pub fn check_support(expr: &Arc) -> bool { +/// Currently, [`CastExpr`], [`NegativeExpr`], [`BinaryExpr`], [`Column`] and [`Literal`] are supported. +pub fn check_support(expr: &Arc, schema: &SchemaRef) -> bool { let expr_any = expr.as_any(); - let expr_supported = if let Some(binary_expr) = expr_any.downcast_ref::() - { + if let Some(binary_expr) = expr_any.downcast_ref::() { is_operator_supported(binary_expr.op()) + && check_support(binary_expr.left(), schema) + && check_support(binary_expr.right(), schema) + } else if let Some(column) = expr_any.downcast_ref::() { + if let Ok(field) = schema.field_with_name(column.name()) { + is_datatype_supported(field.data_type()) + } else { + return false; + } + } else if let Some(literal) = expr_any.downcast_ref::() { + if let Ok(dt) = literal.data_type(schema) { + is_datatype_supported(&dt) + } else { + return false; + } + } else if let Some(cast) = expr_any.downcast_ref::() { + check_support(cast.expr(), schema) + } else if let Some(negative) = expr_any.downcast_ref::() { + check_support(negative.arg(), schema) } else { - expr_any.is::() || expr_any.is::() || expr_any.is::() - }; - expr_supported && expr.children().iter().all(check_support) + false + } } // This function returns the inverse operator of the given operator. diff --git a/datafusion/physical-plan/src/filter.rs b/datafusion/physical-plan/src/filter.rs index 4a8b18914411..ccd0bd525f10 100644 --- a/datafusion/physical-plan/src/filter.rs +++ b/datafusion/physical-plan/src/filter.rs @@ -203,7 +203,7 @@ impl ExecutionPlan for FilterExec { fn statistics(&self) -> Statistics { let predicate = self.predicate(); - if !check_support(predicate) { + if !check_support(predicate, &self.schema()) { return Statistics::default(); } From cf7a9a0d7f27250806785a76c7144d8cb9b31749 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Berkay=20=C5=9Eahin?= <124376117+berkaysynnada@users.noreply.github.com> Date: Thu, 12 Oct 2023 18:35:40 +0300 Subject: [PATCH 058/572] Exactness Indicator of Parameters: Precision (#7809) * Sharpness enum is introduced. * Unit tests added * typo fix * Name changes to precision. --- datafusion/common/src/stats.rs | 297 ++++++++++++++++++++++++++++++++- 1 file changed, 294 insertions(+), 3 deletions(-) diff --git a/datafusion/common/src/stats.rs b/datafusion/common/src/stats.rs index ca76e14cb8ab..279ad434a817 100644 --- a/datafusion/common/src/stats.rs +++ b/datafusion/common/src/stats.rs @@ -17,11 +17,187 @@ //! This module provides data structures to represent statistics -use std::fmt::Display; - +use crate::ScalarValue; use arrow::datatypes::DataType; -use crate::ScalarValue; +use std::fmt::{self, Debug, Display}; + +/// Represents a value with a degree of certainty. `Precision` is used to +/// propagate information the precision of statistical values. +#[derive(Clone, PartialEq, Eq, Default)] +pub enum Precision { + /// The exact value is known + Exact(T), + /// The value is not known exactly, but is likely close to this value + Inexact(T), + /// Nothing is known about the value + #[default] + Absent, +} + +impl Precision { + /// If we have some value (exact or inexact), it returns that value. + /// Otherwise, it returns `None`. + pub fn get_value(&self) -> Option<&T> { + match self { + Precision::Exact(value) | Precision::Inexact(value) => Some(value), + Precision::Absent => None, + } + } + + /// Transform the value in this [`Precision`] object, if one exists, using + /// the given function. Preserves the exactness state. + pub fn map(self, f: F) -> Precision + where + F: Fn(T) -> T, + { + match self { + Precision::Exact(val) => Precision::Exact(f(val)), + Precision::Inexact(val) => Precision::Inexact(f(val)), + _ => self, + } + } + + /// Returns `Some(true)` if we have an exact value, `Some(false)` if we + /// have an inexact value, and `None` if there is no value. + pub fn is_exact(&self) -> Option { + match self { + Precision::Exact(_) => Some(true), + Precision::Inexact(_) => Some(false), + _ => None, + } + } + + /// Returns the maximum of two (possibly inexact) values, conservatively + /// propagating exactness information. If one of the input values is + /// [`Precision::Absent`], the result is `Absent` too. + pub fn max(&self, other: &Precision) -> Precision { + match (self, other) { + (Precision::Exact(a), Precision::Exact(b)) => { + Precision::Exact(if a >= b { a.clone() } else { b.clone() }) + } + (Precision::Inexact(a), Precision::Exact(b)) + | (Precision::Exact(a), Precision::Inexact(b)) + | (Precision::Inexact(a), Precision::Inexact(b)) => { + Precision::Inexact(if a >= b { a.clone() } else { b.clone() }) + } + (_, _) => Precision::Absent, + } + } + + /// Returns the minimum of two (possibly inexact) values, conservatively + /// propagating exactness information. If one of the input values is + /// [`Precision::Absent`], the result is `Absent` too. + pub fn min(&self, other: &Precision) -> Precision { + match (self, other) { + (Precision::Exact(a), Precision::Exact(b)) => { + Precision::Exact(if a >= b { b.clone() } else { a.clone() }) + } + (Precision::Inexact(a), Precision::Exact(b)) + | (Precision::Exact(a), Precision::Inexact(b)) + | (Precision::Inexact(a), Precision::Inexact(b)) => { + Precision::Inexact(if a >= b { b.clone() } else { a.clone() }) + } + (_, _) => Precision::Absent, + } + } + + /// Demotes the precision state from exact to inexact (if present). + pub fn to_inexact(self) -> Self { + match self { + Precision::Exact(value) => Precision::Inexact(value), + _ => self, + } + } +} + +impl Precision { + /// Calculates the sum of two (possibly inexact) [`usize`] values, + /// conservatively propagating exactness information. If one of the input + /// values is [`Precision::Absent`], the result is `Absent` too. + pub fn add(&self, other: &Precision) -> Precision { + match (self, other) { + (Precision::Exact(a), Precision::Exact(b)) => Precision::Exact(a + b), + (Precision::Inexact(a), Precision::Exact(b)) + | (Precision::Exact(a), Precision::Inexact(b)) + | (Precision::Inexact(a), Precision::Inexact(b)) => Precision::Inexact(a + b), + (_, _) => Precision::Absent, + } + } + + /// Calculates the difference of two (possibly inexact) [`usize`] values, + /// conservatively propagating exactness information. If one of the input + /// values is [`Precision::Absent`], the result is `Absent` too. + pub fn sub(&self, other: &Precision) -> Precision { + match (self, other) { + (Precision::Exact(a), Precision::Exact(b)) => Precision::Exact(a - b), + (Precision::Inexact(a), Precision::Exact(b)) + | (Precision::Exact(a), Precision::Inexact(b)) + | (Precision::Inexact(a), Precision::Inexact(b)) => Precision::Inexact(a - b), + (_, _) => Precision::Absent, + } + } + + /// Calculates the multiplication of two (possibly inexact) [`usize`] values, + /// conservatively propagating exactness information. If one of the input + /// values is [`Precision::Absent`], the result is `Absent` too. + pub fn multiply(&self, other: &Precision) -> Precision { + match (self, other) { + (Precision::Exact(a), Precision::Exact(b)) => Precision::Exact(a * b), + (Precision::Inexact(a), Precision::Exact(b)) + | (Precision::Exact(a), Precision::Inexact(b)) + | (Precision::Inexact(a), Precision::Inexact(b)) => Precision::Inexact(a * b), + (_, _) => Precision::Absent, + } + } +} + +impl Precision { + /// Calculates the sum of two (possibly inexact) [`ScalarValue`] values, + /// conservatively propagating exactness information. If one of the input + /// values is [`Precision::Absent`], the result is `Absent` too. + pub fn add(&self, other: &Precision) -> Precision { + match (self, other) { + (Precision::Exact(a), Precision::Exact(b)) => { + if let Ok(result) = a.add(b) { + Precision::Exact(result) + } else { + Precision::Absent + } + } + (Precision::Inexact(a), Precision::Exact(b)) + | (Precision::Exact(a), Precision::Inexact(b)) + | (Precision::Inexact(a), Precision::Inexact(b)) => { + if let Ok(result) = a.add(b) { + Precision::Inexact(result) + } else { + Precision::Absent + } + } + (_, _) => Precision::Absent, + } + } +} + +impl Debug for Precision { + fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result { + match self { + Precision::Exact(inner) => write!(f, "Exact({:?})", inner), + Precision::Inexact(inner) => write!(f, "Inexact({:?})", inner), + Precision::Absent => write!(f, "Absent"), + } + } +} + +impl Display for Precision { + fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result { + match self { + Precision::Exact(inner) => write!(f, "Exact({:?})", inner), + Precision::Inexact(inner) => write!(f, "Inexact({:?})", inner), + Precision::Absent => write!(f, "Absent"), + } + } +} /// Statistics for a relation /// Fields are optional and can be inexact because the sources @@ -94,3 +270,118 @@ impl ColumnStatistics { } } } + +#[cfg(test)] +mod tests { + use super::*; + + #[test] + fn test_get_value() { + let exact_precision = Precision::Exact(42); + let inexact_precision = Precision::Inexact(23); + let absent_precision = Precision::::Absent; + + assert_eq!(*exact_precision.get_value().unwrap(), 42); + assert_eq!(*inexact_precision.get_value().unwrap(), 23); + assert_eq!(absent_precision.get_value(), None); + } + + #[test] + fn test_map() { + let exact_precision = Precision::Exact(42); + let inexact_precision = Precision::Inexact(23); + let absent_precision = Precision::Absent; + + let squared = |x| x * x; + + assert_eq!(exact_precision.map(squared), Precision::Exact(1764)); + assert_eq!(inexact_precision.map(squared), Precision::Inexact(529)); + assert_eq!(absent_precision.map(squared), Precision::Absent); + } + + #[test] + fn test_is_exact() { + let exact_precision = Precision::Exact(42); + let inexact_precision = Precision::Inexact(23); + let absent_precision = Precision::::Absent; + + assert_eq!(exact_precision.is_exact(), Some(true)); + assert_eq!(inexact_precision.is_exact(), Some(false)); + assert_eq!(absent_precision.is_exact(), None); + } + + #[test] + fn test_max() { + let precision1 = Precision::Exact(42); + let precision2 = Precision::Inexact(23); + let precision3 = Precision::Exact(30); + let absent_precision = Precision::Absent; + + assert_eq!(precision1.max(&precision2), Precision::Inexact(42)); + assert_eq!(precision1.max(&precision3), Precision::Exact(42)); + assert_eq!(precision2.max(&precision3), Precision::Inexact(30)); + assert_eq!(precision1.max(&absent_precision), Precision::Absent); + } + + #[test] + fn test_min() { + let precision1 = Precision::Exact(42); + let precision2 = Precision::Inexact(23); + let precision3 = Precision::Exact(30); + let absent_precision = Precision::Absent; + + assert_eq!(precision1.min(&precision2), Precision::Inexact(23)); + assert_eq!(precision1.min(&precision3), Precision::Exact(30)); + assert_eq!(precision2.min(&precision3), Precision::Inexact(23)); + assert_eq!(precision1.min(&absent_precision), Precision::Absent); + } + + #[test] + fn test_to_inexact() { + let exact_precision = Precision::Exact(42); + let inexact_precision = Precision::Inexact(42); + let absent_precision = Precision::::Absent; + + assert_eq!(exact_precision.clone().to_inexact(), inexact_precision); + assert_eq!(inexact_precision.clone().to_inexact(), inexact_precision); + assert_eq!(absent_precision.clone().to_inexact(), absent_precision); + } + + #[test] + fn test_add() { + let precision1 = Precision::Exact(42); + let precision2 = Precision::Inexact(23); + let precision3 = Precision::Exact(30); + let absent_precision = Precision::Absent; + + assert_eq!(precision1.add(&precision2), Precision::Inexact(65)); + assert_eq!(precision1.add(&precision3), Precision::Exact(72)); + assert_eq!(precision2.add(&precision3), Precision::Inexact(53)); + assert_eq!(precision1.add(&absent_precision), Precision::Absent); + } + + #[test] + fn test_sub() { + let precision1 = Precision::Exact(42); + let precision2 = Precision::Inexact(23); + let precision3 = Precision::Exact(30); + let absent_precision = Precision::Absent; + + assert_eq!(precision1.sub(&precision2), Precision::Inexact(19)); + assert_eq!(precision1.sub(&precision3), Precision::Exact(12)); + assert_eq!(precision1.sub(&absent_precision), Precision::Absent); + } + + #[test] + fn test_multiply() { + let precision1 = Precision::Exact(6); + let precision2 = Precision::Inexact(3); + let precision3 = Precision::Exact(5); + let absent_precision = Precision::Absent; + + assert_eq!(precision1.multiply(&precision2), Precision::Inexact(18)); + assert_eq!(precision1.multiply(&precision3), Precision::Exact(30)); + assert_eq!(precision2.multiply(&precision3), Precision::Inexact(15)); + assert_eq!(precision1.multiply(&absent_precision), Precision::Absent); + } +} From dae12a40e3632b62e19ceaa0b373c4d8303c166c Mon Sep 17 00:00:00 2001 From: Huaijin Date: Fri, 13 Oct 2023 00:21:01 +0800 Subject: [PATCH 059/572] add LogicalPlanBuilder::join_on (#7805) --- datafusion/core/src/dataframe.rs | 7 +-- datafusion/expr/src/logical_plan/builder.rs | 48 +++++++++++++++++++++ 2 files changed, 49 insertions(+), 6 deletions(-) diff --git a/datafusion/core/src/dataframe.rs b/datafusion/core/src/dataframe.rs index a06722f92cdb..d704c7f30443 100644 --- a/datafusion/core/src/dataframe.rs +++ b/datafusion/core/src/dataframe.rs @@ -663,12 +663,7 @@ impl DataFrame { ) -> Result { let expr = on_exprs.into_iter().reduce(Expr::and); let plan = LogicalPlanBuilder::from(self.plan) - .join( - right.plan, - join_type, - (Vec::::new(), Vec::::new()), - expr, - )? + .join_on(right.plan, join_type, expr)? .build()?; Ok(DataFrame::new(self.session_state, plan)) } diff --git a/datafusion/expr/src/logical_plan/builder.rs b/datafusion/expr/src/logical_plan/builder.rs index 3d62bcf55d6c..539f6382adf3 100644 --- a/datafusion/expr/src/logical_plan/builder.rs +++ b/datafusion/expr/src/logical_plan/builder.rs @@ -626,6 +626,54 @@ impl LogicalPlanBuilder { self.join_detailed(right, join_type, join_keys, filter, false) } + /// Apply a join with on constraint. + /// + /// The `ExtractEquijoinPredicate` optimizer pass has the ability to split join predicates into + /// equijoin predicates and (other) filter predicates. Therefore, if you prefer not to manually split the + /// join predicates, it is recommended to use the `join_on` method instead of the `join` method. + /// + /// ``` + /// # use datafusion_expr::{Expr, col, LogicalPlanBuilder, + /// # logical_plan::builder::LogicalTableSource, logical_plan::JoinType,}; + /// # use std::sync::Arc; + /// # use arrow::datatypes::{Schema, DataType, Field}; + /// # use datafusion_common::Result; + /// # fn main() -> Result<()> { + /// let example_schema = Arc::new(Schema::new(vec![ + /// Field::new("a", DataType::Int32, false), + /// Field::new("b", DataType::Int32, false), + /// Field::new("c", DataType::Int32, false), + /// ])); + /// let table_source = Arc::new(LogicalTableSource::new(example_schema)); + /// let left_table = table_source.clone(); + /// let right_table = table_source.clone(); + /// + /// let right_plan = LogicalPlanBuilder::scan("right", right_table, None)?.build()?; + /// + /// let exprs = vec![col("left.a").eq(col("right.a")), col("left.b").not_eq(col("right.b"))] + /// .into_iter() + /// .reduce(Expr::and); + /// let plan = LogicalPlanBuilder::scan("left", left_table, None)? + /// .join_on(right_plan, JoinType::Inner, exprs)? + /// .build()?; + /// # Ok(()) + /// # } + /// ``` + pub fn join_on( + self, + right: LogicalPlan, + join_type: JoinType, + on_exprs: Option, + ) -> Result { + self.join_detailed( + right, + join_type, + (Vec::::new(), Vec::::new()), + on_exprs, + false, + ) + } + pub(crate) fn normalize( plan: &LogicalPlan, column: impl Into + Clone, From 036dc48ae17287939e5f9ba54e7da175ceb60910 Mon Sep 17 00:00:00 2001 From: Mustafa Akur <106137913+mustafasrepo@users.noreply.github.com> Date: Thu, 12 Oct 2023 21:49:00 +0300 Subject: [PATCH 060/572] Fix window sort removal wrong operator. (#7811) --- .../src/physical_optimizer/enforce_sorting.rs | 52 +++++++++++++++++-- .../core/src/physical_optimizer/test_utils.rs | 8 +++ 2 files changed, 56 insertions(+), 4 deletions(-) diff --git a/datafusion/core/src/physical_optimizer/enforce_sorting.rs b/datafusion/core/src/physical_optimizer/enforce_sorting.rs index 95ec1973d017..f84a05f0fdac 100644 --- a/datafusion/core/src/physical_optimizer/enforce_sorting.rs +++ b/datafusion/core/src/physical_optimizer/enforce_sorting.rs @@ -62,6 +62,7 @@ use datafusion_physical_expr::utils::{ }; use datafusion_physical_expr::{PhysicalSortExpr, PhysicalSortRequirement}; +use datafusion_physical_plan::repartition::RepartitionExec; use itertools::izip; /// This rule inspects [`SortExec`]'s in the given physical plan and removes the @@ -566,7 +567,6 @@ fn analyze_window_sort_removal( ); let mut window_child = remove_corresponding_sort_from_sub_plan(sort_tree, requires_single_partition)?; - let (window_expr, new_window) = if let Some(exec) = window_exec.as_any().downcast_ref::() { ( @@ -704,8 +704,18 @@ fn remove_corresponding_sort_from_sub_plan( children[item.idx] = remove_corresponding_sort_from_sub_plan(item, requires_single_partition)?; } + // Replace with variants that do not preserve order. if is_sort_preserving_merge(plan) { children[0].clone() + } else if let Some(repartition) = plan.as_any().downcast_ref::() + { + Arc::new( + RepartitionExec::try_new( + children[0].clone(), + repartition.partitioning().clone(), + )? + .with_preserve_order(false), + ) } else { plan.clone().with_new_children(children)? } @@ -758,7 +768,7 @@ mod tests { coalesce_partitions_exec, filter_exec, global_limit_exec, hash_join_exec, limit_exec, local_limit_exec, memory_exec, parquet_exec, parquet_exec_sorted, repartition_exec, sort_exec, sort_expr, sort_expr_options, sort_merge_join_exec, - sort_preserving_merge_exec, union_exec, + sort_preserving_merge_exec, spr_repartition_exec, union_exec, }; use crate::physical_optimizer::utils::get_plan_string; use crate::physical_plan::repartition::RepartitionExec; @@ -1635,14 +1645,16 @@ mod tests { // During the removal of `SortExec`s, it should be able to remove the // corresponding SortExecs together. Also, the inputs of these `SortExec`s // are not necessarily the same to be able to remove them. - let expected_input = ["BoundedWindowAggExec: wdw=[count: Ok(Field { name: \"count\", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(NULL), end_bound: CurrentRow }], mode=[Sorted]", + let expected_input = [ + "BoundedWindowAggExec: wdw=[count: Ok(Field { name: \"count\", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(NULL), end_bound: CurrentRow }], mode=[Sorted]", " SortPreservingMergeExec: [nullable_col@0 DESC NULLS LAST]", " UnionExec", " SortExec: expr=[nullable_col@0 DESC NULLS LAST]", " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], output_ordering=[nullable_col@0 ASC, non_nullable_col@1 ASC]", " SortExec: expr=[nullable_col@0 DESC NULLS LAST]", " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], output_ordering=[nullable_col@0 ASC]"]; - let expected_optimized = ["WindowAggExec: wdw=[count: Ok(Field { name: \"count\", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: CurrentRow, end_bound: Following(NULL) }]", + let expected_optimized = [ + "WindowAggExec: wdw=[count: Ok(Field { name: \"count\", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: CurrentRow, end_bound: Following(NULL) }]", " SortPreservingMergeExec: [nullable_col@0 ASC]", " UnionExec", " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], output_ordering=[nullable_col@0 ASC, non_nullable_col@1 ASC]", @@ -2234,4 +2246,36 @@ mod tests { assert_optimized!(expected_input, expected_optimized, physical_plan, false); Ok(()) } + + #[tokio::test] + async fn test_window_multi_layer_requirement() -> Result<()> { + let schema = create_test_schema3()?; + let sort_exprs = vec![sort_expr("a", &schema), sort_expr("b", &schema)]; + let source = csv_exec_sorted(&schema, vec![], false); + let sort = sort_exec(sort_exprs.clone(), source); + let repartition = repartition_exec(sort); + let repartition = spr_repartition_exec(repartition); + let spm = sort_preserving_merge_exec(sort_exprs.clone(), repartition); + + let physical_plan = bounded_window_exec("a", sort_exprs, spm); + + let expected_input = [ + "BoundedWindowAggExec: wdw=[count: Ok(Field { name: \"count\", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(NULL), end_bound: CurrentRow }], mode=[Sorted]", + " SortPreservingMergeExec: [a@0 ASC,b@1 ASC]", + " SortPreservingRepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=10", + " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", + " SortExec: expr=[a@0 ASC,b@1 ASC]", + " CsvExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], has_header=false", + ]; + let expected_optimized = [ + "BoundedWindowAggExec: wdw=[count: Ok(Field { name: \"count\", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(NULL), end_bound: CurrentRow }], mode=[Sorted]", + " SortExec: expr=[a@0 ASC,b@1 ASC]", + " CoalescePartitionsExec", + " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=10", + " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", + " CsvExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], has_header=false", + ]; + assert_optimized!(expected_input, expected_optimized, physical_plan, false); + Ok(()) + } } diff --git a/datafusion/core/src/physical_optimizer/test_utils.rs b/datafusion/core/src/physical_optimizer/test_utils.rs index 0915fdbf1cd7..9f966990b830 100644 --- a/datafusion/core/src/physical_optimizer/test_utils.rs +++ b/datafusion/core/src/physical_optimizer/test_utils.rs @@ -324,6 +324,14 @@ pub fn repartition_exec(input: Arc) -> Arc Arc::new(RepartitionExec::try_new(input, Partitioning::RoundRobinBatch(10)).unwrap()) } +pub fn spr_repartition_exec(input: Arc) -> Arc { + Arc::new( + RepartitionExec::try_new(input, Partitioning::RoundRobinBatch(10)) + .unwrap() + .with_preserve_order(true), + ) +} + pub fn aggregate_exec(input: Arc) -> Arc { let schema = input.schema(); Arc::new( From 3ccbcfc49cafffe5d25425ed56a281f74c6edae7 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Fri, 13 Oct 2023 07:05:49 +0200 Subject: [PATCH 061/572] Update zstd requirement from 0.12 to 0.13 (#7806) * Update zstd requirement from 0.12 to 0.13 Updates the requirements on [zstd](https://github.com/gyscos/zstd-rs) to permit the latest version. - [Release notes](https://github.com/gyscos/zstd-rs/releases) - [Commits](https://github.com/gyscos/zstd-rs/compare/v0.12.0...v0.13.0) --- updated-dependencies: - dependency-name: zstd dependency-type: direct:production ... Signed-off-by: dependabot[bot] * Update --------- Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> Co-authored-by: Liang-Chi Hsieh --- datafusion-cli/Cargo.lock | 49 +++++++++++++++++++++++++------------- datafusion/core/Cargo.toml | 2 +- 2 files changed, 34 insertions(+), 17 deletions(-) diff --git a/datafusion-cli/Cargo.lock b/datafusion-cli/Cargo.lock index f695e81d9876..1c872c28485c 100644 --- a/datafusion-cli/Cargo.lock +++ b/datafusion-cli/Cargo.lock @@ -112,7 +112,7 @@ dependencies = [ "typed-builder", "uuid", "xz2", - "zstd", + "zstd 0.12.4", ] [[package]] @@ -370,8 +370,8 @@ dependencies = [ "pin-project-lite", "tokio", "xz2", - "zstd", - "zstd-safe", + "zstd 0.12.4", + "zstd-safe 6.0.6", ] [[package]] @@ -1141,7 +1141,7 @@ dependencies = [ "url", "uuid", "xz2", - "zstd", + "zstd 0.13.0", ] [[package]] @@ -1232,7 +1232,7 @@ dependencies = [ "hashbrown 0.14.1", "itertools", "log", - "regex-syntax 0.8.0", + "regex-syntax 0.8.1", ] [[package]] @@ -2319,9 +2319,9 @@ checksum = "ff011a302c396a5197692431fc1948019154afc178baf7d8e37367442a4601cf" [[package]] name = "ordered-float" -version = "2.10.0" +version = "2.10.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "7940cf2ca942593318d07fcf2596cdca60a85c9e7fab408a5e21a4f9dcd40d87" +checksum = "68f19d67e5a2795c94e73e0bb1cc1a7edeb2e28efd39e2e1c9b7a40c1108b11c" dependencies = [ "num-traits", ] @@ -2392,7 +2392,7 @@ dependencies = [ "thrift", "tokio", "twox-hash", - "zstd", + "zstd 0.12.4", ] [[package]] @@ -2681,7 +2681,7 @@ dependencies = [ "aho-corasick", "memchr", "regex-automata", - "regex-syntax 0.8.0", + "regex-syntax 0.8.1", ] [[package]] @@ -2692,7 +2692,7 @@ checksum = "465c6fc0621e4abc4187a2bda0937bfd4f722c2730b29562e19689ea796c9a4b" dependencies = [ "aho-corasick", "memchr", - "regex-syntax 0.8.0", + "regex-syntax 0.8.1", ] [[package]] @@ -2709,9 +2709,9 @@ checksum = "dbb5fb1acd8a1a18b3dd5be62d25485eb770e05afb408a9627d14d451bae12da" [[package]] name = "regex-syntax" -version = "0.8.0" +version = "0.8.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "c3cbb081b9784b07cceb8824c8583f86db4814d172ab043f3c23f7dc600bf83d" +checksum = "56d84fdd47036b038fc80dd333d10b6aab10d5d31f4a366e20014def75328d33" [[package]] name = "reqwest" @@ -3933,7 +3933,16 @@ version = "0.12.4" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "1a27595e173641171fc74a1232b7b1c7a7cb6e18222c11e9dfb9888fa424c53c" dependencies = [ - "zstd-safe", + "zstd-safe 6.0.6", +] + +[[package]] +name = "zstd" +version = "0.13.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "bffb3309596d527cfcba7dfc6ed6052f1d39dfbd7c867aa2e865e4a449c10110" +dependencies = [ + "zstd-safe 7.0.0", ] [[package]] @@ -3946,13 +3955,21 @@ dependencies = [ "zstd-sys", ] +[[package]] +name = "zstd-safe" +version = "7.0.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "43747c7422e2924c11144d5229878b98180ef8b06cca4ab5af37afc8a8d8ea3e" +dependencies = [ + "zstd-sys", +] + [[package]] name = "zstd-sys" -version = "2.0.8+zstd.1.5.5" +version = "2.0.9+zstd.1.5.5" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "5556e6ee25d32df2586c098bbfa278803692a20d0ab9565e049480d52707ec8c" +checksum = "9e16efa8a874a0481a574084d34cc26fdb3b99627480f785888deb6386506656" dependencies = [ "cc", - "libc", "pkg-config", ] diff --git a/datafusion/core/Cargo.toml b/datafusion/core/Cargo.toml index 484d203be848..fa580c914ce2 100644 --- a/datafusion/core/Cargo.toml +++ b/datafusion/core/Cargo.toml @@ -90,7 +90,7 @@ tokio-util = { version = "0.7.4", features = ["io"] } url = "2.2" uuid = { version = "1.0", features = ["v4"] } xz2 = { version = "0.1", optional = true } -zstd = { version = "0.12", optional = true, default-features = false } +zstd = { version = "0.13", optional = true, default-features = false } [dev-dependencies] From 485b80ec49b75d9b181807c1a726eddf627cf4ab Mon Sep 17 00:00:00 2001 From: Mustafa Akur <106137913+mustafasrepo@users.noreply.github.com> Date: Fri, 13 Oct 2023 08:51:03 +0300 Subject: [PATCH 062/572] [Minor]: Remove input_schema field from window executor (#7810) * Initial commit * Remove input schema from proto --- .../src/physical_optimizer/enforce_sorting.rs | 3 --- .../core/src/physical_optimizer/test_utils.rs | 1 - datafusion/core/src/physical_planner.rs | 2 -- .../core/tests/fuzz_cases/window_fuzz.rs | 2 -- .../src/windows/bounded_window_agg_exec.rs | 14 ++------------ datafusion/physical-plan/src/windows/mod.rs | 3 --- .../src/windows/window_agg_exec.rs | 14 ++------------ datafusion/proto/proto/datafusion.proto | 1 - datafusion/proto/src/generated/pbjson.rs | 18 ------------------ datafusion/proto/src/generated/prost.rs | 2 -- datafusion/proto/src/physical_plan/mod.rs | 19 +------------------ .../tests/cases/roundtrip_physical_plan.rs | 1 - 12 files changed, 5 insertions(+), 75 deletions(-) diff --git a/datafusion/core/src/physical_optimizer/enforce_sorting.rs b/datafusion/core/src/physical_optimizer/enforce_sorting.rs index f84a05f0fdac..a381bbb501bc 100644 --- a/datafusion/core/src/physical_optimizer/enforce_sorting.rs +++ b/datafusion/core/src/physical_optimizer/enforce_sorting.rs @@ -608,12 +608,10 @@ fn analyze_window_sort_removal( add_sort_above(&mut window_child, sort_expr, None)?; let uses_bounded_memory = window_expr.iter().all(|e| e.uses_bounded_memory()); - let input_schema = window_child.schema(); let new_window = if uses_bounded_memory { Arc::new(BoundedWindowAggExec::try_new( window_expr.to_vec(), window_child, - input_schema, partitionby_exprs.to_vec(), PartitionSearchMode::Sorted, )?) as _ @@ -621,7 +619,6 @@ fn analyze_window_sort_removal( Arc::new(WindowAggExec::try_new( window_expr.to_vec(), window_child, - input_schema, partitionby_exprs.to_vec(), )?) as _ }; diff --git a/datafusion/core/src/physical_optimizer/test_utils.rs b/datafusion/core/src/physical_optimizer/test_utils.rs index 9f966990b830..ed7345651457 100644 --- a/datafusion/core/src/physical_optimizer/test_utils.rs +++ b/datafusion/core/src/physical_optimizer/test_utils.rs @@ -238,7 +238,6 @@ pub fn bounded_window_exec( ) .unwrap()], input.clone(), - input.schema(), vec![], crate::physical_plan::windows::PartitionSearchMode::Sorted, ) diff --git a/datafusion/core/src/physical_planner.rs b/datafusion/core/src/physical_planner.rs index 35119f374fa3..325927bb7381 100644 --- a/datafusion/core/src/physical_planner.rs +++ b/datafusion/core/src/physical_planner.rs @@ -751,7 +751,6 @@ impl DefaultPhysicalPlanner { Arc::new(BoundedWindowAggExec::try_new( window_expr, input_exec, - physical_input_schema, physical_partition_keys, PartitionSearchMode::Sorted, )?) @@ -759,7 +758,6 @@ impl DefaultPhysicalPlanner { Arc::new(WindowAggExec::try_new( window_expr, input_exec, - physical_input_schema, physical_partition_keys, )?) }) diff --git a/datafusion/core/tests/fuzz_cases/window_fuzz.rs b/datafusion/core/tests/fuzz_cases/window_fuzz.rs index 1f0a4b09b15f..83c8e1f57896 100644 --- a/datafusion/core/tests/fuzz_cases/window_fuzz.rs +++ b/datafusion/core/tests/fuzz_cases/window_fuzz.rs @@ -461,7 +461,6 @@ async fn run_window_test( ) .unwrap()], exec1, - schema.clone(), vec![], ) .unwrap(), @@ -484,7 +483,6 @@ async fn run_window_test( ) .unwrap()], exec2, - schema.clone(), vec![], search_mode, ) diff --git a/datafusion/physical-plan/src/windows/bounded_window_agg_exec.rs b/datafusion/physical-plan/src/windows/bounded_window_agg_exec.rs index dfef0ddefa03..800ea42b3562 100644 --- a/datafusion/physical-plan/src/windows/bounded_window_agg_exec.rs +++ b/datafusion/physical-plan/src/windows/bounded_window_agg_exec.rs @@ -88,8 +88,6 @@ pub struct BoundedWindowAggExec { window_expr: Vec>, /// Schema after the window is run schema: SchemaRef, - /// Schema before the window - input_schema: SchemaRef, /// Partition Keys pub partition_keys: Vec>, /// Execution metrics @@ -110,11 +108,10 @@ impl BoundedWindowAggExec { pub fn try_new( window_expr: Vec>, input: Arc, - input_schema: SchemaRef, partition_keys: Vec>, partition_search_mode: PartitionSearchMode, ) -> Result { - let schema = create_schema(&input_schema, &window_expr)?; + let schema = create_schema(&input.schema(), &window_expr)?; let schema = Arc::new(schema); let partition_by_exprs = window_expr[0].partition_by(); let ordered_partition_by_indices = match &partition_search_mode { @@ -140,7 +137,6 @@ impl BoundedWindowAggExec { input, window_expr, schema, - input_schema, partition_keys, metrics: ExecutionPlanMetricsSet::new(), partition_search_mode, @@ -158,11 +154,6 @@ impl BoundedWindowAggExec { &self.input } - /// Get the input schema before any window functions are applied - pub fn input_schema(&self) -> SchemaRef { - self.input_schema.clone() - } - /// Return the output sort order of partition keys: For example /// OVER(PARTITION BY a, ORDER BY b) -> would give sorting of the column a // We are sure that partition by columns are always at the beginning of sort_keys @@ -303,7 +294,6 @@ impl ExecutionPlan for BoundedWindowAggExec { Ok(Arc::new(BoundedWindowAggExec::try_new( self.window_expr.clone(), children[0].clone(), - self.input_schema.clone(), self.partition_keys.clone(), self.partition_search_mode.clone(), )?)) @@ -333,7 +323,7 @@ impl ExecutionPlan for BoundedWindowAggExec { fn statistics(&self) -> Statistics { let input_stat = self.input.statistics(); let win_cols = self.window_expr.len(); - let input_cols = self.input_schema.fields().len(); + let input_cols = self.input.schema().fields().len(); // TODO stats: some windowing function will maintain invariants such as min, max... let mut column_statistics = Vec::with_capacity(win_cols + input_cols); if let Some(input_col_stats) = input_stat.column_statistics { diff --git a/datafusion/physical-plan/src/windows/mod.rs b/datafusion/physical-plan/src/windows/mod.rs index 0f165f79354e..cc915e54af60 100644 --- a/datafusion/physical-plan/src/windows/mod.rs +++ b/datafusion/physical-plan/src/windows/mod.rs @@ -421,7 +421,6 @@ pub fn get_best_fitting_window( Ok(Some(Arc::new(BoundedWindowAggExec::try_new( window_expr, input.clone(), - input.schema(), physical_partition_keys.to_vec(), partition_search_mode, )?) as _)) @@ -435,7 +434,6 @@ pub fn get_best_fitting_window( Ok(Some(Arc::new(WindowAggExec::try_new( window_expr, input.clone(), - input.schema(), physical_partition_keys.to_vec(), )?) as _)) } @@ -759,7 +757,6 @@ mod tests { schema.as_ref(), )?], blocking_exec, - schema, vec![], )?); diff --git a/datafusion/physical-plan/src/windows/window_agg_exec.rs b/datafusion/physical-plan/src/windows/window_agg_exec.rs index b56a9c194c8f..b4dc8ec88c68 100644 --- a/datafusion/physical-plan/src/windows/window_agg_exec.rs +++ b/datafusion/physical-plan/src/windows/window_agg_exec.rs @@ -59,8 +59,6 @@ pub struct WindowAggExec { window_expr: Vec>, /// Schema after the window is run schema: SchemaRef, - /// Schema before the window - input_schema: SchemaRef, /// Partition Keys pub partition_keys: Vec>, /// Execution metrics @@ -75,10 +73,9 @@ impl WindowAggExec { pub fn try_new( window_expr: Vec>, input: Arc, - input_schema: SchemaRef, partition_keys: Vec>, ) -> Result { - let schema = create_schema(&input_schema, &window_expr)?; + let schema = create_schema(&input.schema(), &window_expr)?; let schema = Arc::new(schema); let ordered_partition_by_indices = @@ -87,7 +84,6 @@ impl WindowAggExec { input, window_expr, schema, - input_schema, partition_keys, metrics: ExecutionPlanMetricsSet::new(), ordered_partition_by_indices, @@ -104,11 +100,6 @@ impl WindowAggExec { &self.input } - /// Get the input schema before any window functions are applied - pub fn input_schema(&self) -> SchemaRef { - self.input_schema.clone() - } - /// Return the output sort order of partition keys: For example /// OVER(PARTITION BY a, ORDER BY b) -> would give sorting of the column a // We are sure that partition by columns are always at the beginning of sort_keys @@ -230,7 +221,6 @@ impl ExecutionPlan for WindowAggExec { Ok(Arc::new(WindowAggExec::try_new( self.window_expr.clone(), children[0].clone(), - self.input_schema.clone(), self.partition_keys.clone(), )?)) } @@ -259,7 +249,7 @@ impl ExecutionPlan for WindowAggExec { fn statistics(&self) -> Statistics { let input_stat = self.input.statistics(); let win_cols = self.window_expr.len(); - let input_cols = self.input_schema.fields().len(); + let input_cols = self.input.schema().fields().len(); // TODO stats: some windowing function will maintain invariants such as min, max... let mut column_statistics = Vec::with_capacity(win_cols + input_cols); if let Some(input_col_stats) = input_stat.column_statistics { diff --git a/datafusion/proto/proto/datafusion.proto b/datafusion/proto/proto/datafusion.proto index bda0f7828726..c60dae71ef86 100644 --- a/datafusion/proto/proto/datafusion.proto +++ b/datafusion/proto/proto/datafusion.proto @@ -1419,7 +1419,6 @@ message PartiallySortedPartitionSearchMode { message WindowAggExecNode { PhysicalPlanNode input = 1; repeated PhysicalWindowExprNode window_expr = 2; - Schema input_schema = 4; repeated PhysicalExprNode partition_keys = 5; // Set optional to `None` for `BoundedWindowAggExec`. oneof partition_search_mode { diff --git a/datafusion/proto/src/generated/pbjson.rs b/datafusion/proto/src/generated/pbjson.rs index ced0c8bd7c7a..266075e68922 100644 --- a/datafusion/proto/src/generated/pbjson.rs +++ b/datafusion/proto/src/generated/pbjson.rs @@ -23969,9 +23969,6 @@ impl serde::Serialize for WindowAggExecNode { if !self.window_expr.is_empty() { len += 1; } - if self.input_schema.is_some() { - len += 1; - } if !self.partition_keys.is_empty() { len += 1; } @@ -23985,9 +23982,6 @@ impl serde::Serialize for WindowAggExecNode { if !self.window_expr.is_empty() { struct_ser.serialize_field("windowExpr", &self.window_expr)?; } - if let Some(v) = self.input_schema.as_ref() { - struct_ser.serialize_field("inputSchema", v)?; - } if !self.partition_keys.is_empty() { struct_ser.serialize_field("partitionKeys", &self.partition_keys)?; } @@ -24017,8 +24011,6 @@ impl<'de> serde::Deserialize<'de> for WindowAggExecNode { "input", "window_expr", "windowExpr", - "input_schema", - "inputSchema", "partition_keys", "partitionKeys", "linear", @@ -24031,7 +24023,6 @@ impl<'de> serde::Deserialize<'de> for WindowAggExecNode { enum GeneratedField { Input, WindowExpr, - InputSchema, PartitionKeys, Linear, PartiallySorted, @@ -24059,7 +24050,6 @@ impl<'de> serde::Deserialize<'de> for WindowAggExecNode { match value { "input" => Ok(GeneratedField::Input), "windowExpr" | "window_expr" => Ok(GeneratedField::WindowExpr), - "inputSchema" | "input_schema" => Ok(GeneratedField::InputSchema), "partitionKeys" | "partition_keys" => Ok(GeneratedField::PartitionKeys), "linear" => Ok(GeneratedField::Linear), "partiallySorted" | "partially_sorted" => Ok(GeneratedField::PartiallySorted), @@ -24085,7 +24075,6 @@ impl<'de> serde::Deserialize<'de> for WindowAggExecNode { { let mut input__ = None; let mut window_expr__ = None; - let mut input_schema__ = None; let mut partition_keys__ = None; let mut partition_search_mode__ = None; while let Some(k) = map_.next_key()? { @@ -24102,12 +24091,6 @@ impl<'de> serde::Deserialize<'de> for WindowAggExecNode { } window_expr__ = Some(map_.next_value()?); } - GeneratedField::InputSchema => { - if input_schema__.is_some() { - return Err(serde::de::Error::duplicate_field("inputSchema")); - } - input_schema__ = map_.next_value()?; - } GeneratedField::PartitionKeys => { if partition_keys__.is_some() { return Err(serde::de::Error::duplicate_field("partitionKeys")); @@ -24140,7 +24123,6 @@ impl<'de> serde::Deserialize<'de> for WindowAggExecNode { Ok(WindowAggExecNode { input: input__, window_expr: window_expr__.unwrap_or_default(), - input_schema: input_schema__, partition_keys: partition_keys__.unwrap_or_default(), partition_search_mode: partition_search_mode__, }) diff --git a/datafusion/proto/src/generated/prost.rs b/datafusion/proto/src/generated/prost.rs index ca20cd35cb55..894afa570fb0 100644 --- a/datafusion/proto/src/generated/prost.rs +++ b/datafusion/proto/src/generated/prost.rs @@ -1994,8 +1994,6 @@ pub struct WindowAggExecNode { pub input: ::core::option::Option<::prost::alloc::boxed::Box>, #[prost(message, repeated, tag = "2")] pub window_expr: ::prost::alloc::vec::Vec, - #[prost(message, optional, tag = "4")] - pub input_schema: ::core::option::Option, #[prost(message, repeated, tag = "5")] pub partition_keys: ::prost::alloc::vec::Vec, /// Set optional to `None` for `BoundedWindowAggExec`. diff --git a/datafusion/proto/src/physical_plan/mod.rs b/datafusion/proto/src/physical_plan/mod.rs index 8257f9aa3458..08010a3151ee 100644 --- a/datafusion/proto/src/physical_plan/mod.rs +++ b/datafusion/proto/src/physical_plan/mod.rs @@ -282,16 +282,7 @@ impl AsExecutionPlan for PhysicalPlanNode { runtime, extension_codec, )?; - let input_schema = window_agg - .input_schema - .as_ref() - .ok_or_else(|| { - DataFusionError::Internal( - "input_schema in WindowAggrNode is missing.".to_owned(), - ) - })? - .clone(); - let input_schema: SchemaRef = SchemaRef::new((&input_schema).try_into()?); + let input_schema = input.schema(); let physical_window_expr: Vec> = window_agg .window_expr @@ -333,7 +324,6 @@ impl AsExecutionPlan for PhysicalPlanNode { Ok(Arc::new(BoundedWindowAggExec::try_new( physical_window_expr, input, - input_schema, partition_keys, partition_search_mode, )?)) @@ -341,7 +331,6 @@ impl AsExecutionPlan for PhysicalPlanNode { Ok(Arc::new(WindowAggExec::try_new( physical_window_expr, input, - input_schema, partition_keys, )?)) } @@ -1315,8 +1304,6 @@ impl AsExecutionPlan for PhysicalPlanNode { extension_codec, )?; - let input_schema = protobuf::Schema::try_from(exec.input_schema().as_ref())?; - let window_expr = exec.window_expr() .iter() @@ -1334,7 +1321,6 @@ impl AsExecutionPlan for PhysicalPlanNode { protobuf::WindowAggExecNode { input: Some(Box::new(input)), window_expr, - input_schema: Some(input_schema), partition_keys, partition_search_mode: None, }, @@ -1346,8 +1332,6 @@ impl AsExecutionPlan for PhysicalPlanNode { extension_codec, )?; - let input_schema = protobuf::Schema::try_from(exec.input_schema().as_ref())?; - let window_expr = exec.window_expr() .iter() @@ -1385,7 +1369,6 @@ impl AsExecutionPlan for PhysicalPlanNode { protobuf::WindowAggExecNode { input: Some(Box::new(input)), window_expr, - input_schema: Some(input_schema), partition_keys, partition_search_mode: Some(partition_search_mode), }, diff --git a/datafusion/proto/tests/cases/roundtrip_physical_plan.rs b/datafusion/proto/tests/cases/roundtrip_physical_plan.rs index 77e77630bcb2..e30d416bdc95 100644 --- a/datafusion/proto/tests/cases/roundtrip_physical_plan.rs +++ b/datafusion/proto/tests/cases/roundtrip_physical_plan.rs @@ -275,7 +275,6 @@ fn roundtrip_window() -> Result<()> { sliding_aggr_window_expr, ], input, - schema.clone(), vec![col("b", &schema)?], )?)) } From 20bc365e2908ee7731bda7a82b0e29da2665b8bf Mon Sep 17 00:00:00 2001 From: wiedld Date: Fri, 13 Oct 2023 02:24:49 -0400 Subject: [PATCH 063/572] refactor(7181): move streaming_merge() into separate mod from the merge mod. (#7799) Merge mod has the SortPreservingMergeStream, containing the loser tree. This SortPreservingMergeStream struct will be used repeatedly as part of the cascading merge; in turn, the cascading merge will be implemented for the streaming_merge() method. --- datafusion/physical-plan/src/sorts/merge.rs | 73 +-------------- datafusion/physical-plan/src/sorts/mod.rs | 5 +- datafusion/physical-plan/src/sorts/sort.rs | 2 +- .../src/sorts/streaming_merge.rs | 92 +++++++++++++++++++ 4 files changed, 101 insertions(+), 71 deletions(-) create mode 100644 datafusion/physical-plan/src/sorts/streaming_merge.rs diff --git a/datafusion/physical-plan/src/sorts/merge.rs b/datafusion/physical-plan/src/sorts/merge.rs index 67685509abe5..e60baf2cd806 100644 --- a/datafusion/physical-plan/src/sorts/merge.rs +++ b/datafusion/physical-plan/src/sorts/merge.rs @@ -21,84 +21,21 @@ use crate::metrics::BaselineMetrics; use crate::sorts::builder::BatchBuilder; use crate::sorts::cursor::Cursor; -use crate::sorts::stream::{FieldCursorStream, PartitionedStream, RowCursorStream}; -use crate::{PhysicalSortExpr, RecordBatchStream, SendableRecordBatchStream}; -use arrow::datatypes::{DataType, SchemaRef}; +use crate::sorts::stream::PartitionedStream; +use crate::RecordBatchStream; +use arrow::datatypes::SchemaRef; use arrow::record_batch::RecordBatch; -use arrow_array::*; use datafusion_common::Result; use datafusion_execution::memory_pool::MemoryReservation; use futures::Stream; use std::pin::Pin; use std::task::{ready, Context, Poll}; -macro_rules! primitive_merge_helper { - ($t:ty, $($v:ident),+) => { - merge_helper!(PrimitiveArray<$t>, $($v),+) - }; -} - -macro_rules! merge_helper { - ($t:ty, $sort:ident, $streams:ident, $schema:ident, $tracking_metrics:ident, $batch_size:ident, $fetch:ident, $reservation:ident) => {{ - let streams = FieldCursorStream::<$t>::new($sort, $streams); - return Ok(Box::pin(SortPreservingMergeStream::new( - Box::new(streams), - $schema, - $tracking_metrics, - $batch_size, - $fetch, - $reservation, - ))); - }}; -} - -/// Perform a streaming merge of [`SendableRecordBatchStream`] based on provided sort expressions -/// while preserving order. -pub fn streaming_merge( - streams: Vec, - schema: SchemaRef, - expressions: &[PhysicalSortExpr], - metrics: BaselineMetrics, - batch_size: usize, - fetch: Option, - reservation: MemoryReservation, -) -> Result { - // Special case single column comparisons with optimized cursor implementations - if expressions.len() == 1 { - let sort = expressions[0].clone(); - let data_type = sort.expr.data_type(schema.as_ref())?; - downcast_primitive! { - data_type => (primitive_merge_helper, sort, streams, schema, metrics, batch_size, fetch, reservation), - DataType::Utf8 => merge_helper!(StringArray, sort, streams, schema, metrics, batch_size, fetch, reservation) - DataType::LargeUtf8 => merge_helper!(LargeStringArray, sort, streams, schema, metrics, batch_size, fetch, reservation) - DataType::Binary => merge_helper!(BinaryArray, sort, streams, schema, metrics, batch_size, fetch, reservation) - DataType::LargeBinary => merge_helper!(LargeBinaryArray, sort, streams, schema, metrics, batch_size, fetch, reservation) - _ => {} - } - } - - let streams = RowCursorStream::try_new( - schema.as_ref(), - expressions, - streams, - reservation.new_empty(), - )?; - - Ok(Box::pin(SortPreservingMergeStream::new( - Box::new(streams), - schema, - metrics, - batch_size, - fetch, - reservation, - ))) -} - /// A fallible [`PartitionedStream`] of [`Cursor`] and [`RecordBatch`] type CursorStream = Box>>; #[derive(Debug)] -struct SortPreservingMergeStream { +pub(crate) struct SortPreservingMergeStream { in_progress: BatchBuilder, /// The sorted input streams to merge together @@ -162,7 +99,7 @@ struct SortPreservingMergeStream { } impl SortPreservingMergeStream { - fn new( + pub(crate) fn new( streams: CursorStream, schema: SchemaRef, metrics: BaselineMetrics, diff --git a/datafusion/physical-plan/src/sorts/mod.rs b/datafusion/physical-plan/src/sorts/mod.rs index dff39db423f0..8a1184d3c2b5 100644 --- a/datafusion/physical-plan/src/sorts/mod.rs +++ b/datafusion/physical-plan/src/sorts/mod.rs @@ -20,10 +20,11 @@ mod builder; mod cursor; mod index; -pub mod merge; +mod merge; pub mod sort; pub mod sort_preserving_merge; mod stream; +pub mod streaming_merge; pub use index::RowIndex; -pub(crate) use merge::streaming_merge; +pub(crate) use streaming_merge::streaming_merge; diff --git a/datafusion/physical-plan/src/sorts/sort.rs b/datafusion/physical-plan/src/sorts/sort.rs index 703f80d90d2b..a56f8fec6876 100644 --- a/datafusion/physical-plan/src/sorts/sort.rs +++ b/datafusion/physical-plan/src/sorts/sort.rs @@ -24,7 +24,7 @@ use crate::expressions::PhysicalSortExpr; use crate::metrics::{ BaselineMetrics, Count, ExecutionPlanMetricsSet, MetricBuilder, MetricsSet, }; -use crate::sorts::merge::streaming_merge; +use crate::sorts::streaming_merge::streaming_merge; use crate::stream::{RecordBatchReceiverStream, RecordBatchStreamAdapter}; use crate::topk::TopK; use crate::{ diff --git a/datafusion/physical-plan/src/sorts/streaming_merge.rs b/datafusion/physical-plan/src/sorts/streaming_merge.rs new file mode 100644 index 000000000000..96d180027eee --- /dev/null +++ b/datafusion/physical-plan/src/sorts/streaming_merge.rs @@ -0,0 +1,92 @@ +// 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. + +//! Merge that deals with an arbitrary size of streaming inputs. +//! This is an order-preserving merge. + +use crate::metrics::BaselineMetrics; +use crate::sorts::{ + merge::SortPreservingMergeStream, + stream::{FieldCursorStream, RowCursorStream}, +}; +use crate::{PhysicalSortExpr, SendableRecordBatchStream}; +use arrow::datatypes::{DataType, SchemaRef}; +use arrow_array::*; +use datafusion_common::Result; +use datafusion_execution::memory_pool::MemoryReservation; + +macro_rules! primitive_merge_helper { + ($t:ty, $($v:ident),+) => { + merge_helper!(PrimitiveArray<$t>, $($v),+) + }; +} + +macro_rules! merge_helper { + ($t:ty, $sort:ident, $streams:ident, $schema:ident, $tracking_metrics:ident, $batch_size:ident, $fetch:ident, $reservation:ident) => {{ + let streams = FieldCursorStream::<$t>::new($sort, $streams); + return Ok(Box::pin(SortPreservingMergeStream::new( + Box::new(streams), + $schema, + $tracking_metrics, + $batch_size, + $fetch, + $reservation, + ))); + }}; +} + +/// Perform a streaming merge of [`SendableRecordBatchStream`] based on provided sort expressions +/// while preserving order. +pub fn streaming_merge( + streams: Vec, + schema: SchemaRef, + expressions: &[PhysicalSortExpr], + metrics: BaselineMetrics, + batch_size: usize, + fetch: Option, + reservation: MemoryReservation, +) -> Result { + // Special case single column comparisons with optimized cursor implementations + if expressions.len() == 1 { + let sort = expressions[0].clone(); + let data_type = sort.expr.data_type(schema.as_ref())?; + downcast_primitive! { + data_type => (primitive_merge_helper, sort, streams, schema, metrics, batch_size, fetch, reservation), + DataType::Utf8 => merge_helper!(StringArray, sort, streams, schema, metrics, batch_size, fetch, reservation) + DataType::LargeUtf8 => merge_helper!(LargeStringArray, sort, streams, schema, metrics, batch_size, fetch, reservation) + DataType::Binary => merge_helper!(BinaryArray, sort, streams, schema, metrics, batch_size, fetch, reservation) + DataType::LargeBinary => merge_helper!(LargeBinaryArray, sort, streams, schema, metrics, batch_size, fetch, reservation) + _ => {} + } + } + + let streams = RowCursorStream::try_new( + schema.as_ref(), + expressions, + streams, + reservation.new_empty(), + )?; + + Ok(Box::pin(SortPreservingMergeStream::new( + Box::new(streams), + schema, + metrics, + batch_size, + fetch, + reservation, + ))) +} From 3fe07c4c7f56b6b5286cea63b64240114bd955fe Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E5=BC=A0=E6=9E=97=E4=BC=9F?= Date: Fri, 13 Oct 2023 17:57:05 +0800 Subject: [PATCH 064/572] Improve update error (#7777) * Improve update error * Update datafusion/expr/src/logical_plan/dml.rs Co-authored-by: Alex Huang * Update datafusion/core/src/physical_planner.rs Co-authored-by: Alex Huang * Update datafusion/core/src/physical_planner.rs Co-authored-by: Alex Huang --------- Co-authored-by: Alex Huang --- datafusion/core/src/physical_planner.rs | 4 ++-- datafusion/expr/src/logical_plan/plan.rs | 2 +- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/datafusion/core/src/physical_planner.rs b/datafusion/core/src/physical_planner.rs index 325927bb7381..4055f9f4ebd2 100644 --- a/datafusion/core/src/physical_planner.rs +++ b/datafusion/core/src/physical_planner.rs @@ -1249,10 +1249,10 @@ impl DefaultPhysicalPlanner { "Unsupported logical plan: Prepare" ) } - LogicalPlan::Dml(_) => { + LogicalPlan::Dml(dml) => { // DataFusion is a read-only query engine, but also a library, so consumers may implement this not_impl_err!( - "Unsupported logical plan: Dml" + "Unsupported logical plan: Dml({0})", dml.op ) } LogicalPlan::Statement(statement) => { diff --git a/datafusion/expr/src/logical_plan/plan.rs b/datafusion/expr/src/logical_plan/plan.rs index dfc83f9eec76..b865b6855724 100644 --- a/datafusion/expr/src/logical_plan/plan.rs +++ b/datafusion/expr/src/logical_plan/plan.rs @@ -142,7 +142,7 @@ pub enum LogicalPlan { Prepare(Prepare), /// Data Manipulaton Language (DML): Insert / Update / Delete Dml(DmlStatement), - /// Data Definition Language (DDL): CREATE / DROP TABLES / VIEWS / SCHEMAs + /// Data Definition Language (DDL): CREATE / DROP TABLES / VIEWS / SCHEMAS Ddl(DdlStatement), /// `COPY TO` for writing plan results to files Copy(CopyTo), From 05f0fa14628dab1cf27a3b15c3ae7d53cc01d1b2 Mon Sep 17 00:00:00 2001 From: Andrew Lamb Date: Fri, 13 Oct 2023 11:02:34 -0400 Subject: [PATCH 065/572] Minor: Update LogicalPlan::join_on API, use it more (#7814) * Minor: Update DataFrame::join_on API, use it more * fix doc --- datafusion/expr/src/logical_plan/builder.rs | 11 +++--- .../src/decorrelate_predicate_subquery.rs | 9 ++--- datafusion/optimizer/src/eliminate_join.rs | 8 ++--- .../src/extract_equijoin_predicate.rs | 36 +++++-------------- .../optimizer/src/scalar_subquery_to_join.rs | 14 ++------ datafusion/sql/src/relation/join.rs | 7 +--- .../substrait/src/logical_plan/consumer.rs | 7 +--- 7 files changed, 24 insertions(+), 68 deletions(-) diff --git a/datafusion/expr/src/logical_plan/builder.rs b/datafusion/expr/src/logical_plan/builder.rs index 539f6382adf3..c3f576195e2a 100644 --- a/datafusion/expr/src/logical_plan/builder.rs +++ b/datafusion/expr/src/logical_plan/builder.rs @@ -650,9 +650,8 @@ impl LogicalPlanBuilder { /// /// let right_plan = LogicalPlanBuilder::scan("right", right_table, None)?.build()?; /// - /// let exprs = vec![col("left.a").eq(col("right.a")), col("left.b").not_eq(col("right.b"))] - /// .into_iter() - /// .reduce(Expr::and); + /// let exprs = vec![col("left.a").eq(col("right.a")), col("left.b").not_eq(col("right.b"))]; + /// /// let plan = LogicalPlanBuilder::scan("left", left_table, None)? /// .join_on(right_plan, JoinType::Inner, exprs)? /// .build()?; @@ -663,13 +662,15 @@ impl LogicalPlanBuilder { self, right: LogicalPlan, join_type: JoinType, - on_exprs: Option, + on_exprs: impl IntoIterator, ) -> Result { + let filter = on_exprs.into_iter().reduce(Expr::and); + self.join_detailed( right, join_type, (Vec::::new(), Vec::::new()), - on_exprs, + filter, false, ) } diff --git a/datafusion/optimizer/src/decorrelate_predicate_subquery.rs b/datafusion/optimizer/src/decorrelate_predicate_subquery.rs index 432d7f053aef..96b46663d8e4 100644 --- a/datafusion/optimizer/src/decorrelate_predicate_subquery.rs +++ b/datafusion/optimizer/src/decorrelate_predicate_subquery.rs @@ -21,7 +21,7 @@ use crate::utils::{conjunction, replace_qualified_name, split_conjunction}; use crate::{OptimizerConfig, OptimizerRule}; use datafusion_common::alias::AliasGenerator; use datafusion_common::tree_node::TreeNode; -use datafusion_common::{plan_err, Column, DataFusionError, Result}; +use datafusion_common::{plan_err, DataFusionError, Result}; use datafusion_expr::expr::{Exists, InSubquery}; use datafusion_expr::expr_rewriter::create_col_from_scalar_expr; use datafusion_expr::logical_plan::{JoinType, Subquery}; @@ -282,12 +282,7 @@ fn build_join( false => JoinType::LeftSemi, }; let new_plan = LogicalPlanBuilder::from(left.clone()) - .join( - sub_query_alias, - join_type, - (Vec::::new(), Vec::::new()), - Some(join_filter), - )? + .join_on(sub_query_alias, join_type, Some(join_filter))? .build()?; debug!( "predicate subquery optimized:\n{}", diff --git a/datafusion/optimizer/src/eliminate_join.rs b/datafusion/optimizer/src/eliminate_join.rs index 00abcdcc68aa..0dbebcc8a051 100644 --- a/datafusion/optimizer/src/eliminate_join.rs +++ b/datafusion/optimizer/src/eliminate_join.rs @@ -77,7 +77,7 @@ impl OptimizerRule for EliminateJoin { mod tests { use crate::eliminate_join::EliminateJoin; use crate::test::*; - use datafusion_common::{Column, Result, ScalarValue}; + use datafusion_common::{Result, ScalarValue}; use datafusion_expr::JoinType::Inner; use datafusion_expr::{logical_plan::builder::LogicalPlanBuilder, Expr, LogicalPlan}; use std::sync::Arc; @@ -89,10 +89,9 @@ mod tests { #[test] fn join_on_false() -> Result<()> { let plan = LogicalPlanBuilder::empty(false) - .join( + .join_on( LogicalPlanBuilder::empty(false).build()?, Inner, - (Vec::::new(), Vec::::new()), Some(Expr::Literal(ScalarValue::Boolean(Some(false)))), )? .build()?; @@ -104,10 +103,9 @@ mod tests { #[test] fn join_on_true() -> Result<()> { let plan = LogicalPlanBuilder::empty(false) - .join( + .join_on( LogicalPlanBuilder::empty(false).build()?, Inner, - (Vec::::new(), Vec::::new()), Some(Expr::Literal(ScalarValue::Boolean(Some(true)))), )? .build()?; diff --git a/datafusion/optimizer/src/extract_equijoin_predicate.rs b/datafusion/optimizer/src/extract_equijoin_predicate.rs index e328eeeb00a1..575969fbf73c 100644 --- a/datafusion/optimizer/src/extract_equijoin_predicate.rs +++ b/datafusion/optimizer/src/extract_equijoin_predicate.rs @@ -161,7 +161,6 @@ mod tests { use super::*; use crate::test::*; use arrow::datatypes::DataType; - use datafusion_common::Column; use datafusion_expr::{ col, lit, logical_plan::builder::LogicalPlanBuilder, JoinType, }; @@ -182,12 +181,7 @@ mod tests { let t2 = test_table_scan_with_name("t2")?; let plan = LogicalPlanBuilder::from(t1) - .join( - t2, - JoinType::Left, - (Vec::::new(), Vec::::new()), - Some(col("t1.a").eq(col("t2.a"))), - )? + .join_on(t2, JoinType::Left, Some(col("t1.a").eq(col("t2.a"))))? .build()?; let expected = "Left Join: t1.a = t2.a [a:UInt32, b:UInt32, c:UInt32, a:UInt32;N, b:UInt32;N, c:UInt32;N]\ \n TableScan: t1 [a:UInt32, b:UInt32, c:UInt32]\ @@ -202,10 +196,9 @@ mod tests { let t2 = test_table_scan_with_name("t2")?; let plan = LogicalPlanBuilder::from(t1) - .join( + .join_on( t2, JoinType::Left, - (Vec::::new(), Vec::::new()), Some((col("t1.a") + lit(10i64)).eq(col("t2.a") * lit(2u32))), )? .build()?; @@ -222,10 +215,9 @@ mod tests { let t2 = test_table_scan_with_name("t2")?; let plan = LogicalPlanBuilder::from(t1) - .join( + .join_on( t2, JoinType::Left, - (Vec::::new(), Vec::::new()), Some( (col("t1.a") + lit(10i64)) .gt_eq(col("t2.a") * lit(2u32)) @@ -273,10 +265,9 @@ mod tests { let t2 = test_table_scan_with_name("t2")?; let plan = LogicalPlanBuilder::from(t1) - .join( + .join_on( t2, JoinType::Left, - (Vec::::new(), Vec::::new()), Some( col("t1.c") .eq(col("t2.c")) @@ -301,10 +292,9 @@ mod tests { let t3 = test_table_scan_with_name("t3")?; let input = LogicalPlanBuilder::from(t2) - .join( + .join_on( t3, JoinType::Left, - (Vec::::new(), Vec::::new()), Some( col("t2.a") .eq(col("t3.a")) @@ -313,10 +303,9 @@ mod tests { )? .build()?; let plan = LogicalPlanBuilder::from(t1) - .join( + .join_on( input, JoinType::Left, - (Vec::::new(), Vec::::new()), Some( col("t1.a") .eq(col("t2.a")) @@ -340,10 +329,9 @@ mod tests { let t3 = test_table_scan_with_name("t3")?; let input = LogicalPlanBuilder::from(t2) - .join( + .join_on( t3, JoinType::Left, - (Vec::::new(), Vec::::new()), Some( col("t2.a") .eq(col("t3.a")) @@ -352,10 +340,9 @@ mod tests { )? .build()?; let plan = LogicalPlanBuilder::from(t1) - .join( + .join_on( input, JoinType::Left, - (Vec::::new(), Vec::::new()), Some(col("t1.a").eq(col("t2.a")).and(col("t2.c").eq(col("t3.c")))), )? .build()?; @@ -383,12 +370,7 @@ mod tests { ) .alias("t1.a + 1 = t2.a + 2"); let plan = LogicalPlanBuilder::from(t1) - .join( - t2, - JoinType::Left, - (Vec::::new(), Vec::::new()), - Some(filter), - )? + .join_on(t2, JoinType::Left, Some(filter))? .build()?; let expected = "Left Join: t1.a + CAST(Int64(1) AS UInt32) = t2.a + CAST(Int32(2) AS UInt32) [a:UInt32, b:UInt32, c:UInt32, a:UInt32;N, b:UInt32;N, c:UInt32;N]\ \n TableScan: t1 [a:UInt32, b:UInt32, c:UInt32]\ diff --git a/datafusion/optimizer/src/scalar_subquery_to_join.rs b/datafusion/optimizer/src/scalar_subquery_to_join.rs index 96d2f45d808e..7ac0c25119c3 100644 --- a/datafusion/optimizer/src/scalar_subquery_to_join.rs +++ b/datafusion/optimizer/src/scalar_subquery_to_join.rs @@ -315,24 +315,14 @@ fn build_join( _ => { // if not correlated, group down to 1 row and left join on that (preserving row count) LogicalPlanBuilder::from(filter_input.clone()) - .join( - sub_query_alias, - JoinType::Left, - (Vec::::new(), Vec::::new()), - None, - )? + .join_on(sub_query_alias, JoinType::Left, None)? .build()? } } } else { // left join if correlated, grouping by the join keys so we don't change row count LogicalPlanBuilder::from(filter_input.clone()) - .join( - sub_query_alias, - JoinType::Left, - (Vec::::new(), Vec::::new()), - join_filter_opt, - )? + .join_on(sub_query_alias, JoinType::Left, join_filter_opt)? .build()? }; let mut computation_project_expr = HashMap::new(); diff --git a/datafusion/sql/src/relation/join.rs b/datafusion/sql/src/relation/join.rs index 0113f337e6dc..b119672eae5f 100644 --- a/datafusion/sql/src/relation/join.rs +++ b/datafusion/sql/src/relation/join.rs @@ -132,12 +132,7 @@ impl<'a, S: ContextProvider> SqlToRel<'a, S> { // parse ON expression let expr = self.sql_to_expr(sql_expr, &join_schema, planner_context)?; LogicalPlanBuilder::from(left) - .join( - right, - join_type, - (Vec::::new(), Vec::::new()), - Some(expr), - )? + .join_on(right, join_type, Some(expr))? .build() } JoinConstraint::Using(idents) => { diff --git a/datafusion/substrait/src/logical_plan/consumer.rs b/datafusion/substrait/src/logical_plan/consumer.rs index e1dde39427a5..8d99d1981b91 100644 --- a/datafusion/substrait/src/logical_plan/consumer.rs +++ b/datafusion/substrait/src/logical_plan/consumer.rs @@ -396,12 +396,7 @@ pub async fn from_substrait_rel( } None => match &join_filter { Some(_) => left - .join( - right.build()?, - join_type, - (Vec::::new(), Vec::::new()), - join_filter, - )? + .join_on(right.build()?, join_type, join_filter)? .build(), None => plan_err!("Join without join keys require a valid filter"), }, From 81c915c58e90967b1586fc0b3145194a6fddf6f5 Mon Sep 17 00:00:00 2001 From: Eugene Marushchenko Date: Sat, 14 Oct 2023 03:24:37 +1000 Subject: [PATCH 066/572] Add distinct union optimization (#7788) * Add eliminate_distinct_nested_union spagetti implementation * update test --------- Co-authored-by: Andrew Lamb --- .../optimizer/src/eliminate_nested_union.rs | 208 ++++++++++++++++-- datafusion/sqllogictest/test_files/union.slt | 24 +- 2 files changed, 201 insertions(+), 31 deletions(-) diff --git a/datafusion/optimizer/src/eliminate_nested_union.rs b/datafusion/optimizer/src/eliminate_nested_union.rs index e22c73e5794d..89bcc90bc075 100644 --- a/datafusion/optimizer/src/eliminate_nested_union.rs +++ b/datafusion/optimizer/src/eliminate_nested_union.rs @@ -16,12 +16,11 @@ // under the License. //! Optimizer rule to replace nested unions to single union. +use crate::optimizer::ApplyOrder; use crate::{OptimizerConfig, OptimizerRule}; use datafusion_common::Result; -use datafusion_expr::logical_plan::{LogicalPlan, Union}; - -use crate::optimizer::ApplyOrder; use datafusion_expr::expr_rewriter::coerce_plan_expr_for_schema; +use datafusion_expr::{Distinct, LogicalPlan, Union}; use std::sync::Arc; #[derive(Default)] @@ -41,22 +40,11 @@ impl OptimizerRule for EliminateNestedUnion { plan: &LogicalPlan, _config: &dyn OptimizerConfig, ) -> Result> { - // TODO: Add optimization for nested distinct unions. match plan { LogicalPlan::Union(Union { inputs, schema }) => { let inputs = inputs .iter() - .flat_map(|plan| match plan.as_ref() { - LogicalPlan::Union(Union { inputs, schema }) => inputs - .iter() - .map(|plan| { - Arc::new( - coerce_plan_expr_for_schema(plan, schema).unwrap(), - ) - }) - .collect::>(), - _ => vec![plan.clone()], - }) + .flat_map(extract_plans_from_union) .collect::>(); Ok(Some(LogicalPlan::Union(Union { @@ -64,6 +52,23 @@ impl OptimizerRule for EliminateNestedUnion { schema: schema.clone(), }))) } + LogicalPlan::Distinct(Distinct { input: plan }) => match plan.as_ref() { + LogicalPlan::Union(Union { inputs, schema }) => { + let inputs = inputs + .iter() + .map(extract_plan_from_distinct) + .flat_map(extract_plans_from_union) + .collect::>(); + + Ok(Some(LogicalPlan::Distinct(Distinct { + input: Arc::new(LogicalPlan::Union(Union { + inputs, + schema: schema.clone(), + })), + }))) + } + _ => Ok(None), + }, _ => Ok(None), } } @@ -77,6 +82,23 @@ impl OptimizerRule for EliminateNestedUnion { } } +fn extract_plans_from_union(plan: &Arc) -> Vec> { + match plan.as_ref() { + LogicalPlan::Union(Union { inputs, schema }) => inputs + .iter() + .map(|plan| Arc::new(coerce_plan_expr_for_schema(plan, schema).unwrap())) + .collect::>(), + _ => vec![plan.clone()], + } +} + +fn extract_plan_from_distinct(plan: &Arc) -> &Arc { + match plan.as_ref() { + LogicalPlan::Distinct(Distinct { input: plan }) => plan, + _ => plan, + } +} + #[cfg(test)] mod tests { use super::*; @@ -112,6 +134,22 @@ mod tests { assert_optimized_plan_equal(&plan, expected) } + #[test] + fn eliminate_distinct_nothing() -> Result<()> { + let plan_builder = table_scan(Some("table"), &schema(), None)?; + + let plan = plan_builder + .clone() + .union_distinct(plan_builder.clone().build()?)? + .build()?; + + let expected = "Distinct:\ + \n Union\ + \n TableScan: table\ + \n TableScan: table"; + assert_optimized_plan_equal(&plan, expected) + } + #[test] fn eliminate_nested_union() -> Result<()> { let plan_builder = table_scan(Some("table"), &schema(), None)?; @@ -132,6 +170,69 @@ mod tests { assert_optimized_plan_equal(&plan, expected) } + #[test] + fn eliminate_nested_union_with_distinct_union() -> Result<()> { + let plan_builder = table_scan(Some("table"), &schema(), None)?; + + let plan = plan_builder + .clone() + .union_distinct(plan_builder.clone().build()?)? + .union(plan_builder.clone().build()?)? + .union(plan_builder.clone().build()?)? + .build()?; + + let expected = "Union\ + \n Distinct:\ + \n Union\ + \n TableScan: table\ + \n TableScan: table\ + \n TableScan: table\ + \n TableScan: table"; + assert_optimized_plan_equal(&plan, expected) + } + + #[test] + fn eliminate_nested_distinct_union() -> Result<()> { + let plan_builder = table_scan(Some("table"), &schema(), None)?; + + let plan = plan_builder + .clone() + .union(plan_builder.clone().build()?)? + .union_distinct(plan_builder.clone().build()?)? + .union(plan_builder.clone().build()?)? + .union_distinct(plan_builder.clone().build()?)? + .build()?; + + let expected = "Distinct:\ + \n Union\ + \n TableScan: table\ + \n TableScan: table\ + \n TableScan: table\ + \n TableScan: table\ + \n TableScan: table"; + assert_optimized_plan_equal(&plan, expected) + } + + #[test] + fn eliminate_nested_distinct_union_with_distinct_table() -> Result<()> { + let plan_builder = table_scan(Some("table"), &schema(), None)?; + + let plan = plan_builder + .clone() + .union_distinct(plan_builder.clone().distinct()?.build()?)? + .union(plan_builder.clone().distinct()?.build()?)? + .union_distinct(plan_builder.clone().build()?)? + .build()?; + + let expected = "Distinct:\ + \n Union\ + \n TableScan: table\ + \n TableScan: table\ + \n TableScan: table\ + \n TableScan: table"; + assert_optimized_plan_equal(&plan, expected) + } + // We don't need to use project_with_column_index in logical optimizer, // after LogicalPlanBuilder::union, we already have all equal expression aliases #[test] @@ -163,6 +264,36 @@ mod tests { assert_optimized_plan_equal(&plan, expected) } + #[test] + fn eliminate_nested_distinct_union_with_projection() -> Result<()> { + let plan_builder = table_scan(Some("table"), &schema(), None)?; + + let plan = plan_builder + .clone() + .union_distinct( + plan_builder + .clone() + .project(vec![col("id").alias("table_id"), col("key"), col("value")])? + .build()?, + )? + .union_distinct( + plan_builder + .clone() + .project(vec![col("id").alias("_id"), col("key"), col("value")])? + .build()?, + )? + .build()?; + + let expected = "Distinct:\ + \n Union\ + \n TableScan: table\ + \n Projection: table.id AS id, table.key, table.value\ + \n TableScan: table\ + \n Projection: table.id AS id, table.key, table.value\ + \n TableScan: table"; + assert_optimized_plan_equal(&plan, expected) + } + #[test] fn eliminate_nested_union_with_type_cast_projection() -> Result<()> { let table_1 = table_scan( @@ -208,4 +339,51 @@ mod tests { \n TableScan: table_1"; assert_optimized_plan_equal(&plan, expected) } + + #[test] + fn eliminate_nested_distinct_union_with_type_cast_projection() -> Result<()> { + let table_1 = table_scan( + Some("table_1"), + &Schema::new(vec![ + Field::new("id", DataType::Int64, false), + Field::new("key", DataType::Utf8, false), + Field::new("value", DataType::Float64, false), + ]), + None, + )?; + + let table_2 = table_scan( + Some("table_1"), + &Schema::new(vec![ + Field::new("id", DataType::Int32, false), + Field::new("key", DataType::Utf8, false), + Field::new("value", DataType::Float32, false), + ]), + None, + )?; + + let table_3 = table_scan( + Some("table_1"), + &Schema::new(vec![ + Field::new("id", DataType::Int16, false), + Field::new("key", DataType::Utf8, false), + Field::new("value", DataType::Float32, false), + ]), + None, + )?; + + let plan = table_1 + .union_distinct(table_2.build()?)? + .union_distinct(table_3.build()?)? + .build()?; + + let expected = "Distinct:\ + \n Union\ + \n TableScan: table_1\ + \n Projection: CAST(table_1.id AS Int64) AS id, table_1.key, CAST(table_1.value AS Float64) AS value\ + \n TableScan: table_1\ + \n Projection: CAST(table_1.id AS Int64) AS id, table_1.key, CAST(table_1.value AS Float64) AS value\ + \n TableScan: table_1"; + assert_optimized_plan_equal(&plan, expected) + } } diff --git a/datafusion/sqllogictest/test_files/union.slt b/datafusion/sqllogictest/test_files/union.slt index b11a687d8b9f..cbb1896efb13 100644 --- a/datafusion/sqllogictest/test_files/union.slt +++ b/datafusion/sqllogictest/test_files/union.slt @@ -186,8 +186,7 @@ Bob_new John John_new -# should be un-nested -# https://github.com/apache/arrow-datafusion/issues/7786 +# should be un-nested, with a single (logical) aggregate query TT EXPLAIN SELECT name FROM t1 UNION (SELECT name from t2 UNION SELECT name || '_new' from t2) ---- @@ -195,26 +194,19 @@ logical_plan Aggregate: groupBy=[[t1.name]], aggr=[[]] --Union ----TableScan: t1 projection=[name] -----Aggregate: groupBy=[[t2.name]], aggr=[[]] -------Union ---------TableScan: t2 projection=[name] ---------Projection: t2.name || Utf8("_new") AS name -----------TableScan: t2 projection=[name] +----TableScan: t2 projection=[name] +----Projection: t2.name || Utf8("_new") AS name +------TableScan: t2 projection=[name] physical_plan AggregateExec: mode=FinalPartitioned, gby=[name@0 as name], aggr=[] --CoalesceBatchesExec: target_batch_size=8192 -----RepartitionExec: partitioning=Hash([name@0], 4), input_partitions=8 +----RepartitionExec: partitioning=Hash([name@0], 4), input_partitions=12 ------AggregateExec: mode=Partial, gby=[name@0 as name], aggr=[] --------UnionExec ----------MemoryExec: partitions=4, partition_sizes=[1, 0, 0, 0] -----------AggregateExec: mode=FinalPartitioned, gby=[name@0 as name], aggr=[] -------------CoalesceBatchesExec: target_batch_size=8192 ---------------RepartitionExec: partitioning=Hash([name@0], 4), input_partitions=8 -----------------AggregateExec: mode=Partial, gby=[name@0 as name], aggr=[] -------------------UnionExec ---------------------MemoryExec: partitions=4, partition_sizes=[1, 0, 0, 0] ---------------------ProjectionExec: expr=[name@0 || _new as name] -----------------------MemoryExec: partitions=4, partition_sizes=[1, 0, 0, 0] +----------MemoryExec: partitions=4, partition_sizes=[1, 0, 0, 0] +----------ProjectionExec: expr=[name@0 || _new as name] +------------MemoryExec: partitions=4, partition_sizes=[1, 0, 0, 0] # nested_union_all query T rowsort From e0fa75fc991f29f5c16d84cbbf656123b611162e Mon Sep 17 00:00:00 2001 From: Chih Wang Date: Sat, 14 Oct 2023 01:26:32 +0800 Subject: [PATCH 067/572] Make CI fail on any occurrence of rust-tomlfmt failed (#7774) * Make CI fail on any occurrence of rust-tomlfmt failed * Fix the format of datafusion/sqllogictest/Cargo.toml --- .github/workflows/rust.yml | 5 ++-- ci/scripts/rust_toml_fmt.sh | 8 +++++- datafusion/sqllogictest/Cargo.toml | 44 +++++++++++++++--------------- 3 files changed, 31 insertions(+), 26 deletions(-) diff --git a/.github/workflows/rust.yml b/.github/workflows/rust.yml index 27fdc9e37037..49ce70a5c0bc 100644 --- a/.github/workflows/rust.yml +++ b/.github/workflows/rust.yml @@ -505,12 +505,11 @@ jobs: - name: Check Cargo.toml formatting run: | - # if you encounter error, try rerun the command below, finally run 'git diff' to - # check which Cargo.toml introduces formatting violation + # if you encounter an error, try running 'cargo tomlfmt -p path/to/Cargo.toml' to fix the formatting automatically. + # If the error still persists, you need to manually edit the Cargo.toml file, which introduces formatting violation. # # ignore ./Cargo.toml because putting workspaces in multi-line lists make it easy to read ci/scripts/rust_toml_fmt.sh - git diff --exit-code config-docs-check: name: check configs.md is up-to-date diff --git a/ci/scripts/rust_toml_fmt.sh b/ci/scripts/rust_toml_fmt.sh index e297ef001594..0a8cc346a37d 100755 --- a/ci/scripts/rust_toml_fmt.sh +++ b/ci/scripts/rust_toml_fmt.sh @@ -17,5 +17,11 @@ # specific language governing permissions and limitations # under the License. +# Run cargo-tomlfmt with flag `-d` in dry run to check formatting +# without overwritng the file. If any error occur, you may want to +# rerun 'cargo tomlfmt -p path/to/Cargo.toml' without '-d' to fix +# the formatting automatically. set -ex -find . -mindepth 2 -name 'Cargo.toml' -exec cargo tomlfmt -p {} \; +for toml in $(find . -mindepth 2 -name 'Cargo.toml'); do + cargo tomlfmt -d -p $toml +done diff --git a/datafusion/sqllogictest/Cargo.toml b/datafusion/sqllogictest/Cargo.toml index c7fabffa0aa0..454f99942f52 100644 --- a/datafusion/sqllogictest/Cargo.toml +++ b/datafusion/sqllogictest/Cargo.toml @@ -16,46 +16,46 @@ # under the License. [package] -authors.workspace = true -edition.workspace = true -homepage.workspace = true -license.workspace = true +authors = { workspace = true } +edition = { workspace = true } +homepage = { workspace = true } +license = { workspace = true } name = "datafusion-sqllogictest" -readme.workspace = true -repository.workspace = true -rust-version.workspace = true -version.workspace = true +readme = { workspace = true } +repository = { workspace = true } +rust-version = { workspace = true } +version = { workspace = true } [lib] name = "datafusion_sqllogictest" path = "src/lib.rs" [dependencies] -arrow = {workspace = true} +arrow = { workspace = true } async-trait = "0.1.41" bigdecimal = "0.4.1" -datafusion = {path = "../core", version = "32.0.0"} -datafusion-common = {path = "../common", version = "32.0.0", default-features = false} +bytes = { version = "1.4.0", optional = true } +chrono = { workspace = true, optional = true } +datafusion = { path = "../core", version = "32.0.0" } +datafusion-common = { path = "../common", version = "32.0.0", default-features = false } +futures = { version = "0.3.28" } half = "2.2.1" itertools = "0.11" -object_store = "0.7.0" -rust_decimal = {version = "1.27.0"} log = "^0.4" +object_store = "0.7.0" +postgres-protocol = { version = "0.6.4", optional = true } +postgres-types = { version = "0.2.4", optional = true } +rust_decimal = { version = "1.27.0" } sqllogictest = "0.17.0" -sqlparser.workspace = true +sqlparser = { workspace = true } tempfile = "3" thiserror = "1.0.44" -tokio = {version = "1.0"} -bytes = {version = "1.4.0", optional = true} -futures = {version = "0.3.28"} -chrono = { workspace = true, optional = true } -tokio-postgres = {version = "0.7.7", optional = true} -postgres-types = {version = "0.2.4", optional = true} -postgres-protocol = {version = "0.6.4", optional = true} +tokio = { version = "1.0" } +tokio-postgres = { version = "0.7.7", optional = true } [features] -postgres = ["bytes", "chrono", "tokio-postgres", "postgres-types", "postgres-protocol"] avro = ["datafusion/avro"] +postgres = ["bytes", "chrono", "tokio-postgres", "postgres-types", "postgres-protocol"] [dev-dependencies] env_logger = "0.10" From f5a6d01a5ef9bc583ecad583c850aa30ca356ee2 Mon Sep 17 00:00:00 2001 From: Nuttiiya Seekhao <37189615+nseekhao@users.noreply.github.com> Date: Fri, 13 Oct 2023 14:10:08 -0400 Subject: [PATCH 068/572] Encode all join conditions in a single expression field (#7612) * Encode all join conditions in a single expression field * Removed all references to post_join_filter * Simplify from_substrait_rel() * Clippy fix * Added test to ensure that Substrait plans produced from DF do not contain a post_join_filter --------- Co-authored-by: Andrew Lamb --- .../substrait/src/logical_plan/consumer.rs | 108 +++++++++------- .../substrait/src/logical_plan/producer.rs | 33 +++-- .../tests/cases/roundtrip_logical_plan.rs | 118 +++++++++++++++++- 3 files changed, 205 insertions(+), 54 deletions(-) diff --git a/datafusion/substrait/src/logical_plan/consumer.rs b/datafusion/substrait/src/logical_plan/consumer.rs index 8d99d1981b91..82e457767bb4 100644 --- a/datafusion/substrait/src/logical_plan/consumer.rs +++ b/datafusion/substrait/src/logical_plan/consumer.rs @@ -18,6 +18,7 @@ use async_recursion::async_recursion; use datafusion::arrow::datatypes::{DataType, Field, TimeUnit}; use datafusion::common::{not_impl_err, DFField, DFSchema, DFSchemaRef}; + use datafusion::logical_expr::{ aggregate_function, window_function::find_df_window_func, BinaryExpr, BuiltinScalarFunction, Case, Expr, LogicalPlan, Operator, @@ -129,6 +130,51 @@ fn scalar_function_type_from_str(name: &str) -> Result { } } +fn split_eq_and_noneq_join_predicate_with_nulls_equality( + filter: &Expr, +) -> (Vec<(Column, Column)>, bool, Option) { + let exprs = split_conjunction(filter); + + let mut accum_join_keys: Vec<(Column, Column)> = vec![]; + let mut accum_filters: Vec = vec![]; + let mut nulls_equal_nulls = false; + + for expr in exprs { + match expr { + Expr::BinaryExpr(binary_expr) => match binary_expr { + x @ (BinaryExpr { + left, + op: Operator::Eq, + right, + } + | BinaryExpr { + left, + op: Operator::IsNotDistinctFrom, + right, + }) => { + nulls_equal_nulls = match x.op { + Operator::Eq => false, + Operator::IsNotDistinctFrom => true, + _ => unreachable!(), + }; + + match (left.as_ref(), right.as_ref()) { + (Expr::Column(l), Expr::Column(r)) => { + accum_join_keys.push((l.clone(), r.clone())); + } + _ => accum_filters.push(expr.clone()), + } + } + _ => accum_filters.push(expr.clone()), + }, + _ => accum_filters.push(expr.clone()), + } + } + + let join_filter = accum_filters.into_iter().reduce(Expr::and); + (accum_join_keys, nulls_equal_nulls, join_filter) +} + /// Convert Substrait Plan to DataFusion DataFrame pub async fn from_substrait_plan( ctx: &mut SessionContext, @@ -336,7 +382,13 @@ pub async fn from_substrait_rel( } } Some(RelType::Join(join)) => { - let left = LogicalPlanBuilder::from( + if join.post_join_filter.is_some() { + return not_impl_err!( + "JoinRel with post_join_filter is not yet supported" + ); + } + + let left: LogicalPlanBuilder = LogicalPlanBuilder::from( from_substrait_rel(ctx, join.left.as_ref().unwrap(), extensions).await?, ); let right = LogicalPlanBuilder::from( @@ -346,60 +398,32 @@ pub async fn from_substrait_rel( // The join condition expression needs full input schema and not the output schema from join since we lose columns from // certain join types such as semi and anti joins let in_join_schema = left.schema().join(right.schema())?; - // Parse post join filter if exists - let join_filter = match &join.post_join_filter { - Some(filter) => { - let parsed_filter = - from_substrait_rex(filter, &in_join_schema, extensions).await?; - Some(parsed_filter.as_ref().clone()) - } - None => None, - }; + // If join expression exists, parse the `on` condition expression, build join and return - // Otherwise, build join with koin filter, without join keys + // Otherwise, build join with only the filter, without join keys match &join.expression.as_ref() { Some(expr) => { let on = from_substrait_rex(expr, &in_join_schema, extensions).await?; - let predicates = split_conjunction(&on); - // TODO: collect only one null_eq_null - let join_exprs: Vec<(Column, Column, bool)> = predicates - .iter() - .map(|p| match p { - Expr::BinaryExpr(BinaryExpr { left, op, right }) => { - match (left.as_ref(), right.as_ref()) { - (Expr::Column(l), Expr::Column(r)) => match op { - Operator::Eq => Ok((l.clone(), r.clone(), false)), - Operator::IsNotDistinctFrom => { - Ok((l.clone(), r.clone(), true)) - } - _ => plan_err!("invalid join condition op"), - }, - _ => plan_err!("invalid join condition expression"), - } - } - _ => plan_err!( - "Non-binary expression is not supported in join condition" - ), - }) - .collect::>>()?; - let (left_cols, right_cols, null_eq_nulls): (Vec<_>, Vec<_>, Vec<_>) = - itertools::multiunzip(join_exprs); + // The join expression can contain both equal and non-equal ops. + // As of datafusion 31.0.0, the equal and non equal join conditions are in separate fields. + // So we extract each part as follows: + // - If an Eq or IsNotDistinctFrom op is encountered, add the left column, right column and is_null_equal_nulls to `join_ons` vector + // - Otherwise we add the expression to join_filter (use conjunction if filter already exists) + let (join_ons, nulls_equal_nulls, join_filter) = + split_eq_and_noneq_join_predicate_with_nulls_equality(&on); + let (left_cols, right_cols): (Vec<_>, Vec<_>) = + itertools::multiunzip(join_ons); left.join_detailed( right.build()?, join_type, (left_cols, right_cols), join_filter, - null_eq_nulls[0], + nulls_equal_nulls, )? .build() } - None => match &join_filter { - Some(_) => left - .join_on(right.build()?, join_type, join_filter)? - .build(), - None => plan_err!("Join without join keys require a valid filter"), - }, + None => plan_err!("JoinRel without join condition is not allowed"), } } Some(RelType::Read(read)) => match &read.as_ref().read_type { diff --git a/datafusion/substrait/src/logical_plan/producer.rs b/datafusion/substrait/src/logical_plan/producer.rs index 1124ea53a557..757bddf9fe58 100644 --- a/datafusion/substrait/src/logical_plan/producer.rs +++ b/datafusion/substrait/src/logical_plan/producer.rs @@ -278,14 +278,15 @@ pub fn to_substrait_rel( // parse filter if exists let in_join_schema = join.left.schema().join(join.right.schema())?; let join_filter = match &join.filter { - Some(filter) => Some(Box::new(to_substrait_rex( + Some(filter) => Some(to_substrait_rex( filter, &Arc::new(in_join_schema), 0, extension_info, - )?)), + )?), None => None, }; + // map the left and right columns to binary expressions in the form `l = r` // build a single expression for the ON condition, such as `l.a = r.a AND l.b = r.b` let eq_op = if join.null_equals_null { @@ -293,15 +294,31 @@ pub fn to_substrait_rel( } else { Operator::Eq }; - - let join_expr = to_substrait_join_expr( + let join_on = to_substrait_join_expr( &join.on, eq_op, join.left.schema(), join.right.schema(), extension_info, - )? - .map(Box::new); + )?; + + // create conjunction between `join_on` and `join_filter` to embed all join conditions, + // whether equal or non-equal in a single expression + let join_expr = match &join_on { + Some(on_expr) => match &join_filter { + Some(filter) => Some(Box::new(make_binary_op_scalar_func( + on_expr, + filter, + Operator::And, + extension_info, + ))), + None => join_on.map(Box::new), // the join expression will only contain `join_on` if filter doesn't exist + }, + None => match &join_filter { + Some(_) => join_filter.map(Box::new), // the join expression will only contain `join_filter` if the `on` condition doesn't exist + None => None, + }, + }; Ok(Box::new(Rel { rel_type: Some(RelType::Join(Box::new(JoinRel { @@ -309,8 +326,8 @@ pub fn to_substrait_rel( left: Some(left), right: Some(right), r#type: join_type as i32, - expression: join_expr, - post_join_filter: join_filter, + expression: join_expr.clone(), + post_join_filter: None, advanced_extension: None, }))), })) diff --git a/datafusion/substrait/tests/cases/roundtrip_logical_plan.rs b/datafusion/substrait/tests/cases/roundtrip_logical_plan.rs index 9b9afa159c20..32416125de24 100644 --- a/datafusion/substrait/tests/cases/roundtrip_logical_plan.rs +++ b/datafusion/substrait/tests/cases/roundtrip_logical_plan.rs @@ -23,15 +23,18 @@ use std::hash::Hash; use std::sync::Arc; use datafusion::arrow::datatypes::{DataType, Field, Schema, TimeUnit}; -use datafusion::common::{DFSchema, DFSchemaRef}; -use datafusion::error::Result; +use datafusion::common::{not_impl_err, plan_err, DFSchema, DFSchemaRef}; +use datafusion::error::{DataFusionError, Result}; use datafusion::execution::context::SessionState; use datafusion::execution::registry::SerializerRegistry; use datafusion::execution::runtime_env::RuntimeEnv; use datafusion::logical_expr::{Extension, LogicalPlan, UserDefinedLogicalNode}; use datafusion::optimizer::simplify_expressions::expr_simplifier::THRESHOLD_INLINE_INLIST; use datafusion::prelude::*; + use substrait::proto::extensions::simple_extension_declaration::MappingType; +use substrait::proto::rel::RelType; +use substrait::proto::{plan_rel, Plan, Rel}; struct MockSerializerRegistry; @@ -383,12 +386,15 @@ async fn roundtrip_inner_join() -> Result<()> { #[tokio::test] async fn roundtrip_non_equi_inner_join() -> Result<()> { - roundtrip("SELECT data.a FROM data JOIN data2 ON data.a <> data2.a").await + roundtrip_verify_post_join_filter( + "SELECT data.a FROM data JOIN data2 ON data.a <> data2.a", + ) + .await } #[tokio::test] async fn roundtrip_non_equi_join() -> Result<()> { - roundtrip( + roundtrip_verify_post_join_filter( "SELECT data.a FROM data, data2 WHERE data.a = data2.a AND data.e > data2.a", ) .await @@ -620,6 +626,91 @@ async fn extension_logical_plan() -> Result<()> { Ok(()) } +fn check_post_join_filters(rel: &Rel) -> Result<()> { + // search for target_rel and field value in proto + match &rel.rel_type { + Some(RelType::Join(join)) => { + // check if join filter is None + if join.post_join_filter.is_some() { + plan_err!( + "DataFusion generated Susbtrait plan cannot have post_join_filter in JoinRel" + ) + } else { + // recursively check JoinRels + match check_post_join_filters(join.left.as_ref().unwrap().as_ref()) { + Err(e) => Err(e), + Ok(_) => { + check_post_join_filters(join.right.as_ref().unwrap().as_ref()) + } + } + } + } + Some(RelType::Project(p)) => { + check_post_join_filters(p.input.as_ref().unwrap().as_ref()) + } + Some(RelType::Filter(filter)) => { + check_post_join_filters(filter.input.as_ref().unwrap().as_ref()) + } + Some(RelType::Fetch(fetch)) => { + check_post_join_filters(fetch.input.as_ref().unwrap().as_ref()) + } + Some(RelType::Sort(sort)) => { + check_post_join_filters(sort.input.as_ref().unwrap().as_ref()) + } + Some(RelType::Aggregate(agg)) => { + check_post_join_filters(agg.input.as_ref().unwrap().as_ref()) + } + Some(RelType::Set(set)) => { + for input in &set.inputs { + match check_post_join_filters(input) { + Err(e) => return Err(e), + Ok(_) => continue, + } + } + Ok(()) + } + Some(RelType::ExtensionSingle(ext)) => { + check_post_join_filters(ext.input.as_ref().unwrap().as_ref()) + } + Some(RelType::ExtensionMulti(ext)) => { + for input in &ext.inputs { + match check_post_join_filters(input) { + Err(e) => return Err(e), + Ok(_) => continue, + } + } + Ok(()) + } + Some(RelType::ExtensionLeaf(_)) | Some(RelType::Read(_)) => Ok(()), + _ => not_impl_err!( + "Unsupported RelType: {:?} in post join filter check", + rel.rel_type + ), + } +} + +async fn verify_post_join_filter_value(proto: Box) -> Result<()> { + for relation in &proto.relations { + match relation.rel_type.as_ref() { + Some(rt) => match rt { + plan_rel::RelType::Rel(rel) => match check_post_join_filters(rel) { + Err(e) => return Err(e), + Ok(_) => continue, + }, + plan_rel::RelType::Root(root) => { + match check_post_join_filters(root.input.as_ref().unwrap()) { + Err(e) => return Err(e), + Ok(_) => continue, + } + } + }, + None => return plan_err!("Cannot parse plan relation: None"), + } + } + + Ok(()) +} + async fn assert_expected_plan(sql: &str, expected_plan_str: &str) -> Result<()> { let mut ctx = create_context().await?; let df = ctx.sql(sql).await?; @@ -688,6 +779,25 @@ async fn roundtrip(sql: &str) -> Result<()> { Ok(()) } +async fn roundtrip_verify_post_join_filter(sql: &str) -> Result<()> { + let mut ctx = create_context().await?; + let df = ctx.sql(sql).await?; + let plan = df.into_optimized_plan()?; + let proto = to_substrait_plan(&plan, &ctx)?; + let plan2 = from_substrait_plan(&mut ctx, &proto).await?; + let plan2 = ctx.state().optimize(&plan2)?; + + println!("{plan:#?}"); + println!("{plan2:#?}"); + + let plan1str = format!("{plan:?}"); + let plan2str = format!("{plan2:?}"); + assert_eq!(plan1str, plan2str); + + // verify that the join filters are None + verify_post_join_filter_value(proto).await +} + async fn roundtrip_all_types(sql: &str) -> Result<()> { let mut ctx = create_all_type_context().await?; let df = ctx.sql(sql).await?; From 19d22d396c9bfa0c23ddf43d739cd43f44f311f3 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Fri, 13 Oct 2023 15:58:13 -0400 Subject: [PATCH 069/572] Update substrait requirement from 0.16.0 to 0.17.0 (#7808) * Update substrait requirement from 0.16.0 to 0.17.0 Updates the requirements on [substrait](https://github.com/substrait-io/substrait-rs) to permit the latest version. - [Release notes](https://github.com/substrait-io/substrait-rs/releases) - [Changelog](https://github.com/substrait-io/substrait-rs/blob/main/CHANGELOG.md) - [Commits](https://github.com/substrait-io/substrait-rs/compare/v0.16.0...v0.17.0) --- updated-dependencies: - dependency-name: substrait dependency-type: direct:production ... Signed-off-by: dependabot[bot] * Update prost dependencies, and new clippy --------- Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> Co-authored-by: Andrew Lamb --- datafusion/substrait/Cargo.toml | 6 +++--- datafusion/substrait/src/logical_plan/consumer.rs | 10 +++++----- 2 files changed, 8 insertions(+), 8 deletions(-) diff --git a/datafusion/substrait/Cargo.toml b/datafusion/substrait/Cargo.toml index 13b9c02cdd28..44a13cdcac37 100644 --- a/datafusion/substrait/Cargo.toml +++ b/datafusion/substrait/Cargo.toml @@ -33,9 +33,9 @@ chrono = { workspace = true } datafusion = { version = "32.0.0", path = "../core" } itertools = "0.11" object_store = "0.7.0" -prost = "0.11" -prost-types = "0.11" -substrait = "0.16.0" +prost = "0.12" +prost-types = "0.12" +substrait = "0.17.0" tokio = "1.17" [features] diff --git a/datafusion/substrait/src/logical_plan/consumer.rs b/datafusion/substrait/src/logical_plan/consumer.rs index 82e457767bb4..ae65a2c7d94a 100644 --- a/datafusion/substrait/src/logical_plan/consumer.rs +++ b/datafusion/substrait/src/logical_plan/consumer.rs @@ -480,8 +480,8 @@ pub async fn from_substrait_rel( } _ => not_impl_err!("Only NamedTable reads are supported"), }, - Some(RelType::Set(set)) => match set_rel::SetOp::from_i32(set.op) { - Some(set_op) => match set_op { + Some(RelType::Set(set)) => match set_rel::SetOp::try_from(set.op) { + Ok(set_op) => match set_op { set_rel::SetOp::UnionAll => { if !set.inputs.is_empty() { let mut union_builder = Ok(LogicalPlanBuilder::from( @@ -498,7 +498,7 @@ pub async fn from_substrait_rel( } _ => not_impl_err!("Unsupported set operator: {set_op:?}"), }, - None => not_impl_err!("Invalid set operation type None"), + Err(e) => not_impl_err!("Invalid set operation type {}: {e}", set.op), }, Some(RelType::ExtensionLeaf(extension)) => { let Some(ext_detail) = &extension.detail else { @@ -554,7 +554,7 @@ pub async fn from_substrait_rel( } fn from_substrait_jointype(join_type: i32) -> Result { - if let Some(substrait_join_type) = join_rel::JoinType::from_i32(join_type) { + if let Ok(substrait_join_type) = join_rel::JoinType::try_from(join_type) { match substrait_join_type { join_rel::JoinType::Inner => Ok(JoinType::Inner), join_rel::JoinType::Left => Ok(JoinType::Left), @@ -582,7 +582,7 @@ pub async fn from_substrait_sorts( let asc_nullfirst = match &s.sort_kind { Some(k) => match k { Direction(d) => { - let Some(direction) = SortDirection::from_i32(*d) else { + let Ok(direction) = SortDirection::try_from(*d) else { return not_impl_err!( "Unsupported Substrait SortDirection value {d}" ); From e1bef86240a99bd9a1032a63833b255922065fd3 Mon Sep 17 00:00:00 2001 From: Andrew Lamb Date: Fri, 13 Oct 2023 16:42:20 -0400 Subject: [PATCH 070/572] Minor: include `sort` expressions in `SortPreservingRepartitionExec` explain plan (#7796) * Minor: include sort expressions in SortPreservingRepartitionExec plan * update plan in new test --- .../enforce_distribution.rs | 41 ++++++++++--------- .../src/physical_optimizer/enforce_sorting.rs | 27 +++++++----- .../replace_with_order_preserving_variants.rs | 41 +++++++++++-------- datafusion/physical-expr/src/sort_expr.rs | 21 ++++++++++ datafusion/physical-plan/src/memory.rs | 7 ++-- .../physical-plan/src/repartition/mod.rs | 29 ++++++++++--- datafusion/physical-plan/src/sorts/sort.rs | 7 ++-- .../src/sorts/sort_preserving_merge.rs | 7 +++- datafusion/sqllogictest/test_files/window.slt | 8 ++-- 9 files changed, 124 insertions(+), 64 deletions(-) diff --git a/datafusion/core/src/physical_optimizer/enforce_distribution.rs b/datafusion/core/src/physical_optimizer/enforce_distribution.rs index a1f509d28733..9be566f10a72 100644 --- a/datafusion/core/src/physical_optimizer/enforce_distribution.rs +++ b/datafusion/core/src/physical_optimizer/enforce_distribution.rs @@ -1738,8 +1738,11 @@ mod tests { _t: DisplayFormatType, f: &mut std::fmt::Formatter, ) -> std::fmt::Result { - let expr: Vec = self.expr.iter().map(|e| e.to_string()).collect(); - write!(f, "SortRequiredExec: [{}]", expr.join(",")) + write!( + f, + "SortRequiredExec: [{}]", + PhysicalSortExpr::format_list(&self.expr) + ) } } @@ -3056,16 +3059,16 @@ mod tests { vec![ top_join_plan.as_str(), join_plan.as_str(), - "SortPreservingRepartitionExec: partitioning=Hash([a@0], 10), input_partitions=10", + "SortPreservingRepartitionExec: partitioning=Hash([a@0], 10), input_partitions=10, sort_exprs=a@0 ASC", "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", "SortExec: expr=[a@0 ASC]", "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", - "SortPreservingRepartitionExec: partitioning=Hash([b1@1], 10), input_partitions=10", + "SortPreservingRepartitionExec: partitioning=Hash([b1@1], 10), input_partitions=10, sort_exprs=b1@1 ASC", "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", "SortExec: expr=[b1@1 ASC]", "ProjectionExec: expr=[a@0 as a1, b@1 as b1, c@2 as c1, d@3 as d1, e@4 as e1]", "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", - "SortPreservingRepartitionExec: partitioning=Hash([c@2], 10), input_partitions=10", + "SortPreservingRepartitionExec: partitioning=Hash([c@2], 10), input_partitions=10, sort_exprs=c@2 ASC", "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", "SortExec: expr=[c@2 ASC]", "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", @@ -3082,21 +3085,21 @@ mod tests { _ => vec![ top_join_plan.as_str(), // Below 4 operators are differences introduced, when join mode is changed - "SortPreservingRepartitionExec: partitioning=Hash([a@0], 10), input_partitions=10", + "SortPreservingRepartitionExec: partitioning=Hash([a@0], 10), input_partitions=10, sort_exprs=a@0 ASC", "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", "SortExec: expr=[a@0 ASC]", "CoalescePartitionsExec", join_plan.as_str(), - "SortPreservingRepartitionExec: partitioning=Hash([a@0], 10), input_partitions=10", + "SortPreservingRepartitionExec: partitioning=Hash([a@0], 10), input_partitions=10, sort_exprs=a@0 ASC", "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", "SortExec: expr=[a@0 ASC]", "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", - "SortPreservingRepartitionExec: partitioning=Hash([b1@1], 10), input_partitions=10", + "SortPreservingRepartitionExec: partitioning=Hash([b1@1], 10), input_partitions=10, sort_exprs=b1@1 ASC", "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", "SortExec: expr=[b1@1 ASC]", "ProjectionExec: expr=[a@0 as a1, b@1 as b1, c@2 as c1, d@3 as d1, e@4 as e1]", "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", - "SortPreservingRepartitionExec: partitioning=Hash([c@2], 10), input_partitions=10", + "SortPreservingRepartitionExec: partitioning=Hash([c@2], 10), input_partitions=10, sort_exprs=c@2 ASC", "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", "SortExec: expr=[c@2 ASC]", "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", @@ -3170,16 +3173,16 @@ mod tests { JoinType::Inner | JoinType::Right => vec![ top_join_plan.as_str(), join_plan.as_str(), - "SortPreservingRepartitionExec: partitioning=Hash([a@0], 10), input_partitions=10", + "SortPreservingRepartitionExec: partitioning=Hash([a@0], 10), input_partitions=10, sort_exprs=a@0 ASC", "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", "SortExec: expr=[a@0 ASC]", "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", - "SortPreservingRepartitionExec: partitioning=Hash([b1@1], 10), input_partitions=10", + "SortPreservingRepartitionExec: partitioning=Hash([b1@1], 10), input_partitions=10, sort_exprs=b1@1 ASC", "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", "SortExec: expr=[b1@1 ASC]", "ProjectionExec: expr=[a@0 as a1, b@1 as b1, c@2 as c1, d@3 as d1, e@4 as e1]", "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", - "SortPreservingRepartitionExec: partitioning=Hash([c@2], 10), input_partitions=10", + "SortPreservingRepartitionExec: partitioning=Hash([c@2], 10), input_partitions=10, sort_exprs=c@2 ASC", "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", "SortExec: expr=[c@2 ASC]", "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", @@ -3187,21 +3190,21 @@ mod tests { // Should include 8 RepartitionExecs (4 of them preserves order) and 4 SortExecs JoinType::Left | JoinType::Full => vec![ top_join_plan.as_str(), - "SortPreservingRepartitionExec: partitioning=Hash([b1@6], 10), input_partitions=10", + "SortPreservingRepartitionExec: partitioning=Hash([b1@6], 10), input_partitions=10, sort_exprs=b1@6 ASC", "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", "SortExec: expr=[b1@6 ASC]", "CoalescePartitionsExec", join_plan.as_str(), - "SortPreservingRepartitionExec: partitioning=Hash([a@0], 10), input_partitions=10", + "SortPreservingRepartitionExec: partitioning=Hash([a@0], 10), input_partitions=10, sort_exprs=a@0 ASC", "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", "SortExec: expr=[a@0 ASC]", "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", - "SortPreservingRepartitionExec: partitioning=Hash([b1@1], 10), input_partitions=10", + "SortPreservingRepartitionExec: partitioning=Hash([b1@1], 10), input_partitions=10, sort_exprs=b1@1 ASC", "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", "SortExec: expr=[b1@1 ASC]", "ProjectionExec: expr=[a@0 as a1, b@1 as b1, c@2 as c1, d@3 as d1, e@4 as e1]", "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", - "SortPreservingRepartitionExec: partitioning=Hash([c@2], 10), input_partitions=10", + "SortPreservingRepartitionExec: partitioning=Hash([c@2], 10), input_partitions=10, sort_exprs=c@2 ASC", "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", "SortExec: expr=[c@2 ASC]", "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", @@ -3292,7 +3295,7 @@ mod tests { let expected_first_sort_enforcement = &[ "SortMergeJoin: join_type=Inner, on=[(b3@1, b2@1), (a3@0, a2@0)]", - "SortPreservingRepartitionExec: partitioning=Hash([b3@1, a3@0], 10), input_partitions=10", + "SortPreservingRepartitionExec: partitioning=Hash([b3@1, a3@0], 10), input_partitions=10, sort_exprs=b3@1 ASC,a3@0 ASC", "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", "SortExec: expr=[b3@1 ASC,a3@0 ASC]", "CoalescePartitionsExec", @@ -3303,7 +3306,7 @@ mod tests { "AggregateExec: mode=Partial, gby=[b@1 as b1, a@0 as a1], aggr=[]", "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", - "SortPreservingRepartitionExec: partitioning=Hash([b2@1, a2@0], 10), input_partitions=10", + "SortPreservingRepartitionExec: partitioning=Hash([b2@1, a2@0], 10), input_partitions=10, sort_exprs=b2@1 ASC,a2@0 ASC", "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", "SortExec: expr=[b2@1 ASC,a2@0 ASC]", "CoalescePartitionsExec", @@ -4382,7 +4385,7 @@ mod tests { let expected = &[ "SortPreservingMergeExec: [c@2 ASC]", "FilterExec: c@2 = 0", - "SortPreservingRepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=2", + "SortPreservingRepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=2, sort_exprs=c@2 ASC", "ParquetExec: file_groups={2 groups: [[x], [y]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC]", ]; diff --git a/datafusion/core/src/physical_optimizer/enforce_sorting.rs b/datafusion/core/src/physical_optimizer/enforce_sorting.rs index a381bbb501bc..92db3bbd053e 100644 --- a/datafusion/core/src/physical_optimizer/enforce_sorting.rs +++ b/datafusion/core/src/physical_optimizer/enforce_sorting.rs @@ -2156,15 +2156,19 @@ mod tests { let coalesce_partitions = coalesce_partitions_exec(repartition_hash); let physical_plan = sort_exec(vec![sort_expr("a", &schema)], coalesce_partitions); - let expected_input = ["SortExec: expr=[a@0 ASC]", + let expected_input = [ + "SortExec: expr=[a@0 ASC]", " CoalescePartitionsExec", " RepartitionExec: partitioning=Hash([c@2], 10), input_partitions=10", " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - " CsvExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], infinite_source=true, output_ordering=[a@0 ASC], has_header=false"]; - let expected_optimized = ["SortPreservingMergeExec: [a@0 ASC]", - " SortPreservingRepartitionExec: partitioning=Hash([c@2], 10), input_partitions=10", + " CsvExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], infinite_source=true, output_ordering=[a@0 ASC], has_header=false" + ]; + let expected_optimized = [ + "SortPreservingMergeExec: [a@0 ASC]", + " SortPreservingRepartitionExec: partitioning=Hash([c@2], 10), input_partitions=10, sort_exprs=a@0 ASC", " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - " CsvExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], infinite_source=true, output_ordering=[a@0 ASC], has_header=false"]; + " CsvExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], infinite_source=true, output_ordering=[a@0 ASC], has_header=false", + ]; assert_optimized!(expected_input, expected_optimized, physical_plan, true); Ok(()) } @@ -2186,11 +2190,14 @@ mod tests { " CoalescePartitionsExec", " RepartitionExec: partitioning=Hash([c@2], 10), input_partitions=10", " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - " CsvExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], infinite_source=true, output_ordering=[a@0 ASC], has_header=false"]; - let expected_optimized = ["SortPreservingMergeExec: [a@0 ASC]", - " SortPreservingRepartitionExec: partitioning=Hash([c@2], 10), input_partitions=10", + " CsvExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], infinite_source=true, output_ordering=[a@0 ASC], has_header=false" + ]; + let expected_optimized = [ + "SortPreservingMergeExec: [a@0 ASC]", + " SortPreservingRepartitionExec: partitioning=Hash([c@2], 10), input_partitions=10, sort_exprs=a@0 ASC", " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - " CsvExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], infinite_source=true, output_ordering=[a@0 ASC], has_header=false"]; + " CsvExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], infinite_source=true, output_ordering=[a@0 ASC], has_header=false", + ]; assert_optimized!(expected_input, expected_optimized, physical_plan, false); Ok(()) } @@ -2259,7 +2266,7 @@ mod tests { let expected_input = [ "BoundedWindowAggExec: wdw=[count: Ok(Field { name: \"count\", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(NULL), end_bound: CurrentRow }], mode=[Sorted]", " SortPreservingMergeExec: [a@0 ASC,b@1 ASC]", - " SortPreservingRepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=10", + " SortPreservingRepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=10, sort_exprs=a@0 ASC,b@1 ASC", " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", " SortExec: expr=[a@0 ASC,b@1 ASC]", " CsvExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], has_header=false", diff --git a/datafusion/core/src/physical_optimizer/replace_with_order_preserving_variants.rs b/datafusion/core/src/physical_optimizer/replace_with_order_preserving_variants.rs index ede95fc67721..f4b3608d00c7 100644 --- a/datafusion/core/src/physical_optimizer/replace_with_order_preserving_variants.rs +++ b/datafusion/core/src/physical_optimizer/replace_with_order_preserving_variants.rs @@ -368,7 +368,7 @@ mod tests { " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", " CsvExec: file_groups={1 group: [[file_path]]}, projection=[a, c, d], infinite_source=true, output_ordering=[a@0 ASC NULLS LAST], has_header=true"]; let expected_optimized = ["SortPreservingMergeExec: [a@0 ASC NULLS LAST]", - " SortPreservingRepartitionExec: partitioning=Hash([c1@0], 8), input_partitions=8", + " SortPreservingRepartitionExec: partitioning=Hash([c1@0], 8), input_partitions=8, sort_exprs=a@0 ASC NULLS LAST", " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", " CsvExec: file_groups={1 group: [[file_path]]}, projection=[a, c, d], infinite_source=true, output_ordering=[a@0 ASC NULLS LAST], has_header=true"]; assert_optimized!(expected_input, expected_optimized, physical_plan); @@ -412,10 +412,10 @@ mod tests { let expected_optimized = [ "SortPreservingMergeExec: [a@0 ASC]", " FilterExec: c@2 > 3", - " SortPreservingRepartitionExec: partitioning=Hash([c1@0], 8), input_partitions=8", + " SortPreservingRepartitionExec: partitioning=Hash([c1@0], 8), input_partitions=8, sort_exprs=a@0 ASC", " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", " SortPreservingMergeExec: [a@0 ASC]", - " SortPreservingRepartitionExec: partitioning=Hash([c1@0], 8), input_partitions=8", + " SortPreservingRepartitionExec: partitioning=Hash([c1@0], 8), input_partitions=8, sort_exprs=a@0 ASC", " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", " CsvExec: file_groups={1 group: [[file_path]]}, projection=[a, c, d], infinite_source=true, output_ordering=[a@0 ASC], has_header=true", ]; @@ -442,11 +442,14 @@ mod tests { " FilterExec: c@2 > 3", " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", " CsvExec: file_groups={1 group: [[file_path]]}, projection=[a, c, d], infinite_source=true, output_ordering=[a@0 ASC NULLS LAST], has_header=true"]; - let expected_optimized = ["SortPreservingMergeExec: [a@0 ASC NULLS LAST]", - " SortPreservingRepartitionExec: partitioning=Hash([c1@0], 8), input_partitions=8", + let expected_optimized = [ + "SortPreservingMergeExec: [a@0 ASC NULLS LAST]", + " SortPreservingRepartitionExec: partitioning=Hash([c1@0], 8), input_partitions=8, sort_exprs=a@0 ASC NULLS LAST", " FilterExec: c@2 > 3", " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", - " CsvExec: file_groups={1 group: [[file_path]]}, projection=[a, c, d], infinite_source=true, output_ordering=[a@0 ASC NULLS LAST], has_header=true"]; + " CsvExec: file_groups={1 group: [[file_path]]}, projection=[a, c, d], infinite_source=true, output_ordering=[a@0 ASC NULLS LAST], has_header=true", + + ]; assert_optimized!(expected_input, expected_optimized, physical_plan); Ok(()) } @@ -475,7 +478,7 @@ mod tests { let expected_optimized = ["SortPreservingMergeExec: [a@0 ASC NULLS LAST]", " CoalesceBatchesExec: target_batch_size=8192", " FilterExec: c@2 > 3", - " SortPreservingRepartitionExec: partitioning=Hash([c1@0], 8), input_partitions=8", + " SortPreservingRepartitionExec: partitioning=Hash([c1@0], 8), input_partitions=8, sort_exprs=a@0 ASC NULLS LAST", " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", " CsvExec: file_groups={1 group: [[file_path]]}, projection=[a, c, d], infinite_source=true, output_ordering=[a@0 ASC NULLS LAST], has_header=true"]; assert_optimized!(expected_input, expected_optimized, physical_plan); @@ -509,7 +512,7 @@ mod tests { let expected_optimized = ["SortPreservingMergeExec: [a@0 ASC NULLS LAST]", " CoalesceBatchesExec: target_batch_size=8192", " FilterExec: c@2 > 3", - " SortPreservingRepartitionExec: partitioning=Hash([c1@0], 8), input_partitions=8", + " SortPreservingRepartitionExec: partitioning=Hash([c1@0], 8), input_partitions=8, sort_exprs=a@0 ASC NULLS LAST", " CoalesceBatchesExec: target_batch_size=8192", " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", " CsvExec: file_groups={1 group: [[file_path]]}, projection=[a, c, d], infinite_source=true, output_ordering=[a@0 ASC NULLS LAST], has_header=true"]; @@ -561,21 +564,25 @@ mod tests { let physical_plan = sort_preserving_merge_exec(vec![sort_expr("a", &schema)], sort); - let expected_input = ["SortPreservingMergeExec: [a@0 ASC NULLS LAST]", + let expected_input = [ + "SortPreservingMergeExec: [a@0 ASC NULLS LAST]", " SortExec: expr=[a@0 ASC NULLS LAST]", " RepartitionExec: partitioning=Hash([c1@0], 8), input_partitions=8", " CoalesceBatchesExec: target_batch_size=8192", " FilterExec: c@2 > 3", " RepartitionExec: partitioning=Hash([c1@0], 8), input_partitions=8", " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", - " CsvExec: file_groups={1 group: [[file_path]]}, projection=[a, c, d], infinite_source=true, output_ordering=[a@0 ASC NULLS LAST], has_header=true"]; - let expected_optimized = ["SortPreservingMergeExec: [a@0 ASC NULLS LAST]", - " SortPreservingRepartitionExec: partitioning=Hash([c1@0], 8), input_partitions=8", + " CsvExec: file_groups={1 group: [[file_path]]}, projection=[a, c, d], infinite_source=true, output_ordering=[a@0 ASC NULLS LAST], has_header=true" + ]; + let expected_optimized = [ + "SortPreservingMergeExec: [a@0 ASC NULLS LAST]", + " SortPreservingRepartitionExec: partitioning=Hash([c1@0], 8), input_partitions=8, sort_exprs=a@0 ASC NULLS LAST", " CoalesceBatchesExec: target_batch_size=8192", " FilterExec: c@2 > 3", - " SortPreservingRepartitionExec: partitioning=Hash([c1@0], 8), input_partitions=8", + " SortPreservingRepartitionExec: partitioning=Hash([c1@0], 8), input_partitions=8, sort_exprs=a@0 ASC NULLS LAST", " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", - " CsvExec: file_groups={1 group: [[file_path]]}, projection=[a, c, d], infinite_source=true, output_ordering=[a@0 ASC NULLS LAST], has_header=true"]; + " CsvExec: file_groups={1 group: [[file_path]]}, projection=[a, c, d], infinite_source=true, output_ordering=[a@0 ASC NULLS LAST], has_header=true", + ]; assert_optimized!(expected_input, expected_optimized, physical_plan); Ok(()) } @@ -627,7 +634,7 @@ mod tests { " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", " CsvExec: file_groups={1 group: [[file_path]]}, projection=[a, c, d], infinite_source=true, output_ordering=[a@0 ASC NULLS LAST], has_header=true"]; let expected_optimized = ["SortPreservingMergeExec: [a@0 ASC NULLS LAST]", - " SortPreservingRepartitionExec: partitioning=Hash([c1@0], 8), input_partitions=8", + " SortPreservingRepartitionExec: partitioning=Hash([c1@0], 8), input_partitions=8, sort_exprs=a@0 ASC NULLS LAST", " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", " CsvExec: file_groups={1 group: [[file_path]]}, projection=[a, c, d], infinite_source=true, output_ordering=[a@0 ASC NULLS LAST], has_header=true"]; assert_optimized!(expected_input, expected_optimized, physical_plan); @@ -671,7 +678,7 @@ mod tests { let expected_optimized = [ "SortPreservingMergeExec: [c@2 ASC]", " FilterExec: c@2 > 3", - " SortPreservingRepartitionExec: partitioning=Hash([c1@0], 8), input_partitions=8", + " SortPreservingRepartitionExec: partitioning=Hash([c1@0], 8), input_partitions=8, sort_exprs=c@2 ASC", " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", " SortExec: expr=[c@2 ASC]", " CoalescePartitionsExec", @@ -756,7 +763,7 @@ mod tests { " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", " CsvExec: file_groups={1 group: [[file_path]]}, projection=[a, c, d], output_ordering=[a@0 ASC NULLS LAST], has_header=true"]; let expected_optimized = ["SortPreservingMergeExec: [a@0 ASC NULLS LAST]", - " SortPreservingRepartitionExec: partitioning=Hash([c1@0], 8), input_partitions=8", + " SortPreservingRepartitionExec: partitioning=Hash([c1@0], 8), input_partitions=8, sort_exprs=a@0 ASC NULLS LAST", " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", " CsvExec: file_groups={1 group: [[file_path]]}, projection=[a, c, d], output_ordering=[a@0 ASC NULLS LAST], has_header=true"]; assert_optimized!(expected_input, expected_optimized, physical_plan, true); diff --git a/datafusion/physical-expr/src/sort_expr.rs b/datafusion/physical-expr/src/sort_expr.rs index 83d32dfeec17..74179ba5947c 100644 --- a/datafusion/physical-expr/src/sort_expr.rs +++ b/datafusion/physical-expr/src/sort_expr.rs @@ -17,6 +17,7 @@ //! Sort expressions +use std::fmt::Display; use std::hash::{Hash, Hasher}; use std::sync::Arc; @@ -89,6 +90,26 @@ impl PhysicalSortExpr { .options .map_or(true, |opts| self.options == opts) } + + /// Returns a [`Display`]able list of `PhysicalSortExpr`. + pub fn format_list(input: &[PhysicalSortExpr]) -> impl Display + '_ { + struct DisplayableList<'a>(&'a [PhysicalSortExpr]); + impl<'a> Display for DisplayableList<'a> { + fn fmt(&self, f: &mut std::fmt::Formatter) -> std::fmt::Result { + let mut first = true; + for sort_expr in self.0 { + if first { + first = false; + } else { + write!(f, ",")?; + } + write!(f, "{}", sort_expr)?; + } + Ok(()) + } + } + DisplayableList(input) + } } /// Represents sort requirement associated with a plan diff --git a/datafusion/physical-plan/src/memory.rs b/datafusion/physical-plan/src/memory.rs index 4c83ff1528fa..d919ded8d0e2 100644 --- a/datafusion/physical-plan/src/memory.rs +++ b/datafusion/physical-plan/src/memory.rs @@ -77,9 +77,10 @@ impl DisplayAs for MemoryExec { .sort_information .first() .map(|output_ordering| { - let order_strings: Vec<_> = - output_ordering.iter().map(|e| e.to_string()).collect(); - format!(", output_ordering={}", order_strings.join(",")) + format!( + ", output_ordering={}", + PhysicalSortExpr::format_list(output_ordering) + ) }) .unwrap_or_default(); diff --git a/datafusion/physical-plan/src/repartition/mod.rs b/datafusion/physical-plan/src/repartition/mod.rs index 14b54dc0614d..bcb9c3afeef1 100644 --- a/datafusion/physical-plan/src/repartition/mod.rs +++ b/datafusion/physical-plan/src/repartition/mod.rs @@ -308,7 +308,8 @@ pub struct RepartitionExec { /// Execution metrics metrics: ExecutionPlanMetricsSet, - /// Boolean flag to decide whether to preserve ordering + /// Boolean flag to decide whether to preserve ordering. If true means + /// `SortPreservingRepartitionExec`, false means `RepartitionExec`. preserve_order: bool, } @@ -370,7 +371,7 @@ impl RepartitionExec { self.preserve_order } - /// Get name of the Executor + /// Get name used to display this Exec pub fn name(&self) -> &str { if self.preserve_order { "SortPreservingRepartitionExec" @@ -394,7 +395,16 @@ impl DisplayAs for RepartitionExec { self.name(), self.partitioning, self.input.output_partitioning().partition_count() - ) + )?; + + if let Some(sort_exprs) = self.sort_exprs() { + write!( + f, + ", sort_exprs={}", + PhysicalSortExpr::format_list(sort_exprs) + )?; + } + Ok(()) } } } @@ -576,8 +586,8 @@ impl ExecutionPlan for RepartitionExec { .collect::>(); // Note that receiver size (`rx.len()`) and `num_input_partitions` are same. - // Get existing ordering: - let sort_exprs = self.input.output_ordering().unwrap_or(&[]); + // Get existing ordering to use for merging + let sort_exprs = self.sort_exprs().unwrap_or(&[]); // Merge streams (while preserving ordering) coming from // input partitions to this partition: @@ -646,6 +656,15 @@ impl RepartitionExec { self } + /// Return the sort expressions that are used to merge + fn sort_exprs(&self) -> Option<&[PhysicalSortExpr]> { + if self.preserve_order { + self.input.output_ordering() + } else { + None + } + } + /// Pulls data from the specified input plan, feeding it to the /// output partitions based on the desired partitioning /// diff --git a/datafusion/physical-plan/src/sorts/sort.rs b/datafusion/physical-plan/src/sorts/sort.rs index a56f8fec6876..ffc4ef9dc32a 100644 --- a/datafusion/physical-plan/src/sorts/sort.rs +++ b/datafusion/physical-plan/src/sorts/sort.rs @@ -763,17 +763,16 @@ impl DisplayAs for SortExec { ) -> std::fmt::Result { match t { DisplayFormatType::Default | DisplayFormatType::Verbose => { - let expr: Vec = self.expr.iter().map(|e| e.to_string()).collect(); + let expr = PhysicalSortExpr::format_list(&self.expr); match self.fetch { Some(fetch) => { write!( f, // TODO should this say topk? - "SortExec: fetch={fetch}, expr=[{}]", - expr.join(",") + "SortExec: fetch={fetch}, expr=[{expr}]", ) } - None => write!(f, "SortExec: expr=[{}]", expr.join(",")), + None => write!(f, "SortExec: expr=[{expr}]"), } } } diff --git a/datafusion/physical-plan/src/sorts/sort_preserving_merge.rs b/datafusion/physical-plan/src/sorts/sort_preserving_merge.rs index 5b485e0b68e4..597b59f776d5 100644 --- a/datafusion/physical-plan/src/sorts/sort_preserving_merge.rs +++ b/datafusion/physical-plan/src/sorts/sort_preserving_merge.rs @@ -118,8 +118,11 @@ impl DisplayAs for SortPreservingMergeExec { ) -> std::fmt::Result { match t { DisplayFormatType::Default | DisplayFormatType::Verbose => { - let expr: Vec = self.expr.iter().map(|e| e.to_string()).collect(); - write!(f, "SortPreservingMergeExec: [{}]", expr.join(","))?; + write!( + f, + "SortPreservingMergeExec: [{}]", + PhysicalSortExpr::format_list(&self.expr) + )?; if let Some(fetch) = self.fetch { write!(f, ", fetch={fetch}")?; }; diff --git a/datafusion/sqllogictest/test_files/window.slt b/datafusion/sqllogictest/test_files/window.slt index 5fb5a04c6709..80e496a336f4 100644 --- a/datafusion/sqllogictest/test_files/window.slt +++ b/datafusion/sqllogictest/test_files/window.slt @@ -3245,17 +3245,17 @@ physical_plan ProjectionExec: expr=[SUM(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.a, annotated_data_infinite2.b] ORDER BY [annotated_data_infinite2.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@2 as sum1, SUM(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.b, annotated_data_infinite2.a] ORDER BY [annotated_data_infinite2.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@4 as sum2, SUM(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.a, annotated_data_infinite2.d] ORDER BY [annotated_data_infinite2.b ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@3 as sum3, SUM(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.d] ORDER BY [annotated_data_infinite2.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@5 as sum4] --BoundedWindowAggExec: wdw=[SUM(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.d] ORDER BY [annotated_data_infinite2.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "SUM(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.d] ORDER BY [annotated_data_infinite2.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(NULL)), end_bound: CurrentRow }], mode=[Linear] ----CoalesceBatchesExec: target_batch_size=4096 -------SortPreservingRepartitionExec: partitioning=Hash([d@1], 2), input_partitions=2 +------SortPreservingRepartitionExec: partitioning=Hash([d@1], 2), input_partitions=2, sort_exprs=a@0 ASC NULLS LAST,b ASC NULLS LAST,c ASC NULLS LAST --------ProjectionExec: expr=[a@0 as a, d@3 as d, SUM(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.a, annotated_data_infinite2.b] ORDER BY [annotated_data_infinite2.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@4 as SUM(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.a, annotated_data_infinite2.b] ORDER BY [annotated_data_infinite2.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW, SUM(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.a, annotated_data_infinite2.d] ORDER BY [annotated_data_infinite2.b ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@5 as SUM(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.a, annotated_data_infinite2.d] ORDER BY [annotated_data_infinite2.b ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW, SUM(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.b, annotated_data_infinite2.a] ORDER BY [annotated_data_infinite2.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@6 as SUM(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.b, annotated_data_infinite2.a] ORDER BY [annotated_data_infinite2.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW] ----------BoundedWindowAggExec: wdw=[SUM(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.b, annotated_data_infinite2.a] ORDER BY [annotated_data_infinite2.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "SUM(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.b, annotated_data_infinite2.a] ORDER BY [annotated_data_infinite2.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(NULL)), end_bound: CurrentRow }], mode=[Sorted] ------------CoalesceBatchesExec: target_batch_size=4096 ---------------SortPreservingRepartitionExec: partitioning=Hash([b@1, a@0], 2), input_partitions=2 +--------------SortPreservingRepartitionExec: partitioning=Hash([b@1, a@0], 2), input_partitions=2, sort_exprs=a@0 ASC NULLS LAST,b@1 ASC NULLS LAST,c@2 ASC NULLS LAST ----------------BoundedWindowAggExec: wdw=[SUM(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.a, annotated_data_infinite2.d] ORDER BY [annotated_data_infinite2.b ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "SUM(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.a, annotated_data_infinite2.d] ORDER BY [annotated_data_infinite2.b ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(NULL)), end_bound: CurrentRow }], mode=[PartiallySorted([0])] ------------------CoalesceBatchesExec: target_batch_size=4096 ---------------------SortPreservingRepartitionExec: partitioning=Hash([a@0, d@3], 2), input_partitions=2 +--------------------SortPreservingRepartitionExec: partitioning=Hash([a@0, d@3], 2), input_partitions=2, sort_exprs=a@0 ASC NULLS LAST,b@1 ASC NULLS LAST,c@2 ASC NULLS LAST ----------------------BoundedWindowAggExec: wdw=[SUM(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.a, annotated_data_infinite2.b] ORDER BY [annotated_data_infinite2.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "SUM(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.a, annotated_data_infinite2.b] ORDER BY [annotated_data_infinite2.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(NULL)), end_bound: CurrentRow }], mode=[Sorted] ------------------------CoalesceBatchesExec: target_batch_size=4096 ---------------------------SortPreservingRepartitionExec: partitioning=Hash([a@0, b@1], 2), input_partitions=2 +--------------------------SortPreservingRepartitionExec: partitioning=Hash([a@0, b@1], 2), input_partitions=2, sort_exprs=a@0 ASC NULLS LAST,b@1 ASC NULLS LAST,c@2 ASC NULLS LAST ----------------------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 ------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, b, c, d], infinite_source=true, output_ordering=[a@0 ASC NULLS LAST, b@1 ASC NULLS LAST, c@2 ASC NULLS LAST], has_header=true From a86ee16a934b9ffed1ee71fc55cee440636cd38e Mon Sep 17 00:00:00 2001 From: Ruihang Xia Date: Sat, 14 Oct 2023 23:03:31 +0800 Subject: [PATCH 071/572] minor: add more document to Wildcard expr (#7822) Signed-off-by: Ruihang Xia --- datafusion/expr/src/expr.rs | 10 ++++++++-- 1 file changed, 8 insertions(+), 2 deletions(-) diff --git a/datafusion/expr/src/expr.rs b/datafusion/expr/src/expr.rs index 3e4e3068977c..3949d25b3025 100644 --- a/datafusion/expr/src/expr.rs +++ b/datafusion/expr/src/expr.rs @@ -165,9 +165,15 @@ pub enum Expr { InSubquery(InSubquery), /// Scalar subquery ScalarSubquery(Subquery), - /// Represents a reference to all fields in a schema. + /// Represents a reference to all available fields. + /// + /// This expr has to be resolved to a list of columns before translating logical + /// plan into physical plan. Wildcard, - /// Represents a reference to all fields in a specific schema. + /// Represents a reference to all available fields in a specific schema. + /// + /// This expr has to be resolved to a list of columns before translating logical + /// plan into physical plan. QualifiedWildcard { qualifier: String }, /// List of grouping set expressions. Only valid in the context of an aggregate /// GROUP BY expression list From 4819e7a3e935c37d72557978ed76870d08288471 Mon Sep 17 00:00:00 2001 From: Yongting You <2010youy01@gmail.com> Date: Sun, 15 Oct 2023 02:33:49 -0700 Subject: [PATCH 072/572] Minor: Move `Monotonicity` to `expr` crate (#7820) * move Monotonicity datafusion-physical-expr -> datafusion-expr * review comments --- datafusion/expr/src/built_in_function.rs | 45 +++++++++++++- datafusion/expr/src/lib.rs | 4 +- datafusion/expr/src/signature.rs | 8 +++ datafusion/physical-expr/src/functions.rs | 58 +++---------------- .../physical-expr/src/scalar_function.rs | 2 +- 5 files changed, 64 insertions(+), 53 deletions(-) diff --git a/datafusion/expr/src/built_in_function.rs b/datafusion/expr/src/built_in_function.rs index 3ce573eaf15c..991ad0322087 100644 --- a/datafusion/expr/src/built_in_function.rs +++ b/datafusion/expr/src/built_in_function.rs @@ -21,8 +21,8 @@ use crate::nullif::SUPPORTED_NULLIF_TYPES; use crate::signature::TIMEZONE_WILDCARD; use crate::type_coercion::functions::data_types; use crate::{ - conditional_expressions, struct_expressions, utils, Signature, TypeSignature, - Volatility, + conditional_expressions, struct_expressions, utils, FuncMonotonicity, Signature, + TypeSignature, Volatility, }; use arrow::datatypes::{DataType, Field, Fields, IntervalUnit, TimeUnit}; use datafusion_common::{internal_err, plan_err, DataFusionError, Result}; @@ -1283,6 +1283,47 @@ impl BuiltinScalarFunction { } } } + + /// This function specifies monotonicity behaviors for built-in scalar functions. + /// The list can be extended, only mathematical and datetime functions are + /// considered for the initial implementation of this feature. + pub fn monotonicity(&self) -> Option { + if matches!( + &self, + BuiltinScalarFunction::Atan + | BuiltinScalarFunction::Acosh + | BuiltinScalarFunction::Asinh + | BuiltinScalarFunction::Atanh + | BuiltinScalarFunction::Ceil + | BuiltinScalarFunction::Degrees + | BuiltinScalarFunction::Exp + | BuiltinScalarFunction::Factorial + | BuiltinScalarFunction::Floor + | BuiltinScalarFunction::Ln + | BuiltinScalarFunction::Log10 + | BuiltinScalarFunction::Log2 + | BuiltinScalarFunction::Radians + | BuiltinScalarFunction::Round + | BuiltinScalarFunction::Signum + | BuiltinScalarFunction::Sinh + | BuiltinScalarFunction::Sqrt + | BuiltinScalarFunction::Cbrt + | BuiltinScalarFunction::Tanh + | BuiltinScalarFunction::Trunc + | BuiltinScalarFunction::Pi + ) { + Some(vec![Some(true)]) + } else if matches!( + &self, + BuiltinScalarFunction::DateTrunc | BuiltinScalarFunction::DateBin + ) { + Some(vec![None, Some(true)]) + } else if *self == BuiltinScalarFunction::Log { + Some(vec![Some(true), Some(false)]) + } else { + None + } + } } fn aliases(func: &BuiltinScalarFunction) -> &'static [&'static str] { diff --git a/datafusion/expr/src/lib.rs b/datafusion/expr/src/lib.rs index fee0bdf5c1a4..21c0d750a36d 100644 --- a/datafusion/expr/src/lib.rs +++ b/datafusion/expr/src/lib.rs @@ -74,7 +74,9 @@ pub use logical_plan::*; pub use nullif::SUPPORTED_NULLIF_TYPES; pub use operator::Operator; pub use partition_evaluator::PartitionEvaluator; -pub use signature::{Signature, TypeSignature, Volatility, TIMEZONE_WILDCARD}; +pub use signature::{ + FuncMonotonicity, Signature, TypeSignature, Volatility, TIMEZONE_WILDCARD, +}; pub use table_source::{TableProviderFilterPushDown, TableSource, TableType}; pub use udaf::AggregateUDF; pub use udf::ScalarUDF; diff --git a/datafusion/expr/src/signature.rs b/datafusion/expr/src/signature.rs index 0d732ee6e5b9..399aefc4b66e 100644 --- a/datafusion/expr/src/signature.rs +++ b/datafusion/expr/src/signature.rs @@ -226,3 +226,11 @@ impl Signature { } } } + +/// Monotonicity of the `ScalarFunctionExpr` with respect to its arguments. +/// Each element of this vector corresponds to an argument and indicates whether +/// the function's behavior is monotonic, or non-monotonic/unknown for that argument, namely: +/// - `None` signifies unknown monotonicity or non-monotonicity. +/// - `Some(true)` indicates that the function is monotonically increasing w.r.t. the argument in question. +/// - Some(false) indicates that the function is monotonically decreasing w.r.t. the argument in question. +pub type FuncMonotonicity = Vec>; diff --git a/datafusion/physical-expr/src/functions.rs b/datafusion/physical-expr/src/functions.rs index 5de0dc366b85..e04a68615a46 100644 --- a/datafusion/physical-expr/src/functions.rs +++ b/datafusion/physical-expr/src/functions.rs @@ -45,6 +45,7 @@ use arrow::{ datatypes::{DataType, Int32Type, Int64Type, Schema}, }; use datafusion_common::{internal_err, DataFusionError, Result, ScalarValue}; +pub use datafusion_expr::FuncMonotonicity; use datafusion_expr::{ BuiltinScalarFunction, ColumnarValue, ScalarFunctionImplementation, }; @@ -180,7 +181,7 @@ pub fn create_physical_expr( _ => create_physical_fun(fun, execution_props)?, }; - let monotonicity = get_func_monotonicity(fun); + let monotonicity = fun.monotonicity(); Ok(Arc::new(ScalarFunctionExpr::new( &format!("{fun}"), @@ -903,13 +904,13 @@ pub fn create_physical_fun( }) } -/// Monotonicity of the `ScalarFunctionExpr` with respect to its arguments. -/// Each element of this vector corresponds to an argument and indicates whether -/// the function's behavior is monotonic, or non-monotonic/unknown for that argument, namely: -/// - `None` signifies unknown monotonicity or non-monotonicity. -/// - `Some(true)` indicates that the function is monotonically increasing w.r.t. the argument in question. -/// - Some(false) indicates that the function is monotonically decreasing w.r.t. the argument in question. -pub type FuncMonotonicity = Vec>; +#[deprecated( + since = "32.0.0", + note = "Moved to `expr` crate. Please use `BuiltinScalarFunction::monotonicity()` instead" +)] +pub fn get_func_monotonicity(fun: &BuiltinScalarFunction) -> Option { + fun.monotonicity() +} /// Determines a [`ScalarFunctionExpr`]'s monotonicity for the given arguments /// and the function's behavior depending on its arguments. @@ -964,47 +965,6 @@ fn func_order_in_one_dimension( } } -/// This function specifies monotonicity behaviors for built-in scalar functions. -/// The list can be extended, only mathematical and datetime functions are -/// considered for the initial implementation of this feature. -pub fn get_func_monotonicity(fun: &BuiltinScalarFunction) -> Option { - if matches!( - fun, - BuiltinScalarFunction::Atan - | BuiltinScalarFunction::Acosh - | BuiltinScalarFunction::Asinh - | BuiltinScalarFunction::Atanh - | BuiltinScalarFunction::Ceil - | BuiltinScalarFunction::Degrees - | BuiltinScalarFunction::Exp - | BuiltinScalarFunction::Factorial - | BuiltinScalarFunction::Floor - | BuiltinScalarFunction::Ln - | BuiltinScalarFunction::Log10 - | BuiltinScalarFunction::Log2 - | BuiltinScalarFunction::Radians - | BuiltinScalarFunction::Round - | BuiltinScalarFunction::Signum - | BuiltinScalarFunction::Sinh - | BuiltinScalarFunction::Sqrt - | BuiltinScalarFunction::Cbrt - | BuiltinScalarFunction::Tanh - | BuiltinScalarFunction::Trunc - | BuiltinScalarFunction::Pi - ) { - Some(vec![Some(true)]) - } else if matches!( - fun, - BuiltinScalarFunction::DateTrunc | BuiltinScalarFunction::DateBin - ) { - Some(vec![None, Some(true)]) - } else if *fun == BuiltinScalarFunction::Log { - Some(vec![Some(true), Some(false)]) - } else { - None - } -} - #[cfg(test)] mod tests { use super::*; diff --git a/datafusion/physical-expr/src/scalar_function.rs b/datafusion/physical-expr/src/scalar_function.rs index 517553f90fb2..dc48baa23ab3 100644 --- a/datafusion/physical-expr/src/scalar_function.rs +++ b/datafusion/physical-expr/src/scalar_function.rs @@ -30,7 +30,6 @@ //! to a function that supports f64, it is coerced to f64. use crate::functions::out_ordering; -use crate::functions::FuncMonotonicity; use crate::physical_expr::down_cast_any_ref; use crate::sort_properties::SortProperties; use crate::utils::expr_list_eq_strict_order; @@ -41,6 +40,7 @@ use datafusion_common::Result; use datafusion_expr::expr_vec_fmt; use datafusion_expr::BuiltinScalarFunction; use datafusion_expr::ColumnarValue; +use datafusion_expr::FuncMonotonicity; use datafusion_expr::ScalarFunctionImplementation; use std::any::Any; use std::fmt::Debug; From 8208ff09935f3953a80c239c1c2e6ec073dd0069 Mon Sep 17 00:00:00 2001 From: Kirill Zaborsky Date: Sun, 15 Oct 2023 12:58:32 +0300 Subject: [PATCH 073/572] Use code block for better formatting of rustdoc for PhysicalGroupBy (#7823) --- datafusion/physical-plan/src/aggregates/mod.rs | 2 ++ 1 file changed, 2 insertions(+) diff --git a/datafusion/physical-plan/src/aggregates/mod.rs b/datafusion/physical-plan/src/aggregates/mod.rs index 546abcfc4a4d..c33f49049de1 100644 --- a/datafusion/physical-plan/src/aggregates/mod.rs +++ b/datafusion/physical-plan/src/aggregates/mod.rs @@ -140,6 +140,7 @@ pub enum GroupByOrderMode { /// into multiple groups, using null expressions to align each group. /// For example, with a group by clause `GROUP BY GROUPING SET ((a,b),(a),(b))` the planner should /// create a `PhysicalGroupBy` like +/// ```text /// PhysicalGroupBy { /// expr: [(col(a), a), (col(b), b)], /// null_expr: [(NULL, a), (NULL, b)], @@ -149,6 +150,7 @@ pub enum GroupByOrderMode { /// [true, false] // (b) <=> (NULL, b) /// ] /// } +/// ``` #[derive(Clone, Debug, Default)] pub struct PhysicalGroupBy { /// Distinct (Physical Expr, Alias) in the grouping set From 26e43acac3a96cec8dd4c8365f22dfb1a84306e9 Mon Sep 17 00:00:00 2001 From: Huaijin Date: Sun, 15 Oct 2023 18:13:56 +0800 Subject: [PATCH 074/572] Update explain plan to show `TopK` operator (#7826) * Updated sort.rs solves: #7750 Replaced `SortExec: fetch={fetch}, expr=[{}]` with 'SortExec: TopK(fetch={fetch}), expr=[{}]' in [sort.rs](https://github.com/apache/arrow-datafusion/blob/main/datafusion/physical-plan/src/sorts/sort.rs) file * fix: ci --------- Co-authored-by: Pratibhanu Jarngal <99665583+Night-Amber3301@users.noreply.github.com> --- datafusion/core/tests/sql/explain_analyze.rs | 2 +- datafusion/core/tests/sql/group_by.rs | 2 +- datafusion/physical-plan/src/sorts/sort.rs | 6 +---- .../sqllogictest/test_files/aggregate.slt | 10 ++++---- datafusion/sqllogictest/test_files/topk.slt | 2 +- .../sqllogictest/test_files/tpch/q10.slt.part | 2 +- .../sqllogictest/test_files/tpch/q11.slt.part | 2 +- .../sqllogictest/test_files/tpch/q13.slt.part | 2 +- .../sqllogictest/test_files/tpch/q16.slt.part | 2 +- .../sqllogictest/test_files/tpch/q2.slt.part | 2 +- .../sqllogictest/test_files/tpch/q3.slt.part | 2 +- .../sqllogictest/test_files/tpch/q9.slt.part | 2 +- datafusion/sqllogictest/test_files/window.slt | 24 +++++++++---------- 13 files changed, 28 insertions(+), 32 deletions(-) diff --git a/datafusion/core/tests/sql/explain_analyze.rs b/datafusion/core/tests/sql/explain_analyze.rs index 46f94c348735..de7ed19d3f34 100644 --- a/datafusion/core/tests/sql/explain_analyze.rs +++ b/datafusion/core/tests/sql/explain_analyze.rs @@ -611,7 +611,7 @@ async fn test_physical_plan_display_indent() { let expected = vec![ "GlobalLimitExec: skip=0, fetch=10", " SortPreservingMergeExec: [the_min@2 DESC], fetch=10", - " SortExec: fetch=10, expr=[the_min@2 DESC]", + " SortExec: TopK(fetch=10), expr=[the_min@2 DESC]", " ProjectionExec: expr=[c1@0 as c1, MAX(aggregate_test_100.c12)@1 as MAX(aggregate_test_100.c12), MIN(aggregate_test_100.c12)@2 as the_min]", " AggregateExec: mode=FinalPartitioned, gby=[c1@0 as c1], aggr=[MAX(aggregate_test_100.c12), MIN(aggregate_test_100.c12)]", " CoalesceBatchesExec: target_batch_size=4096", diff --git a/datafusion/core/tests/sql/group_by.rs b/datafusion/core/tests/sql/group_by.rs index 49903857b796..7c7703b69683 100644 --- a/datafusion/core/tests/sql/group_by.rs +++ b/datafusion/core/tests/sql/group_by.rs @@ -82,7 +82,7 @@ async fn group_by_limit() -> Result<()> { let physical_plan = dataframe.create_physical_plan().await?; let mut expected_physical_plan = r#" GlobalLimitExec: skip=0, fetch=4 - SortExec: fetch=4, expr=[MAX(traces.ts)@1 DESC] + SortExec: TopK(fetch=4), expr=[MAX(traces.ts)@1 DESC] AggregateExec: mode=Single, gby=[trace_id@0 as trace_id], aggr=[MAX(traces.ts)], lim=[4] "#.trim().to_string(); let actual_phys_plan = diff --git a/datafusion/physical-plan/src/sorts/sort.rs b/datafusion/physical-plan/src/sorts/sort.rs index ffc4ef9dc32a..fb6f460f1bcb 100644 --- a/datafusion/physical-plan/src/sorts/sort.rs +++ b/datafusion/physical-plan/src/sorts/sort.rs @@ -766,11 +766,7 @@ impl DisplayAs for SortExec { let expr = PhysicalSortExpr::format_list(&self.expr); match self.fetch { Some(fetch) => { - write!( - f, - // TODO should this say topk? - "SortExec: fetch={fetch}, expr=[{expr}]", - ) + write!(f, "SortExec: TopK(fetch={fetch}), expr=[{expr}]",) } None => write!(f, "SortExec: expr=[{expr}]"), } diff --git a/datafusion/sqllogictest/test_files/aggregate.slt b/datafusion/sqllogictest/test_files/aggregate.slt index 64bf64d23c7a..c6f9f42fa2d8 100644 --- a/datafusion/sqllogictest/test_files/aggregate.slt +++ b/datafusion/sqllogictest/test_files/aggregate.slt @@ -2338,7 +2338,7 @@ Limit: skip=0, fetch=4 physical_plan GlobalLimitExec: skip=0, fetch=4 --SortPreservingMergeExec: [MAX(traces.timestamp)@1 DESC], fetch=4 -----SortExec: fetch=4, expr=[MAX(traces.timestamp)@1 DESC] +----SortExec: TopK(fetch=4), expr=[MAX(traces.timestamp)@1 DESC] ------AggregateExec: mode=FinalPartitioned, gby=[trace_id@0 as trace_id], aggr=[MAX(traces.timestamp)] --------CoalesceBatchesExec: target_batch_size=8192 ----------RepartitionExec: partitioning=Hash([trace_id@0], 4), input_partitions=4 @@ -2393,7 +2393,7 @@ Limit: skip=0, fetch=4 physical_plan GlobalLimitExec: skip=0, fetch=4 --SortPreservingMergeExec: [MAX(traces.timestamp)@1 DESC], fetch=4 -----SortExec: fetch=4, expr=[MAX(traces.timestamp)@1 DESC] +----SortExec: TopK(fetch=4), expr=[MAX(traces.timestamp)@1 DESC] ------AggregateExec: mode=FinalPartitioned, gby=[trace_id@0 as trace_id], aggr=[MAX(traces.timestamp)], lim=[4] --------CoalesceBatchesExec: target_batch_size=8192 ----------RepartitionExec: partitioning=Hash([trace_id@0], 4), input_partitions=4 @@ -2412,7 +2412,7 @@ Limit: skip=0, fetch=4 physical_plan GlobalLimitExec: skip=0, fetch=4 --SortPreservingMergeExec: [MIN(traces.timestamp)@1 DESC], fetch=4 -----SortExec: fetch=4, expr=[MIN(traces.timestamp)@1 DESC] +----SortExec: TopK(fetch=4), expr=[MIN(traces.timestamp)@1 DESC] ------AggregateExec: mode=FinalPartitioned, gby=[trace_id@0 as trace_id], aggr=[MIN(traces.timestamp)] --------CoalesceBatchesExec: target_batch_size=8192 ----------RepartitionExec: partitioning=Hash([trace_id@0], 4), input_partitions=4 @@ -2431,7 +2431,7 @@ Limit: skip=0, fetch=4 physical_plan GlobalLimitExec: skip=0, fetch=4 --SortPreservingMergeExec: [MAX(traces.timestamp)@1 ASC NULLS LAST], fetch=4 -----SortExec: fetch=4, expr=[MAX(traces.timestamp)@1 ASC NULLS LAST] +----SortExec: TopK(fetch=4), expr=[MAX(traces.timestamp)@1 ASC NULLS LAST] ------AggregateExec: mode=FinalPartitioned, gby=[trace_id@0 as trace_id], aggr=[MAX(traces.timestamp)] --------CoalesceBatchesExec: target_batch_size=8192 ----------RepartitionExec: partitioning=Hash([trace_id@0], 4), input_partitions=4 @@ -2450,7 +2450,7 @@ Limit: skip=0, fetch=4 physical_plan GlobalLimitExec: skip=0, fetch=4 --SortPreservingMergeExec: [trace_id@0 ASC NULLS LAST], fetch=4 -----SortExec: fetch=4, expr=[trace_id@0 ASC NULLS LAST] +----SortExec: TopK(fetch=4), expr=[trace_id@0 ASC NULLS LAST] ------AggregateExec: mode=FinalPartitioned, gby=[trace_id@0 as trace_id], aggr=[MAX(traces.timestamp)] --------CoalesceBatchesExec: target_batch_size=8192 ----------RepartitionExec: partitioning=Hash([trace_id@0], 4), input_partitions=4 diff --git a/datafusion/sqllogictest/test_files/topk.slt b/datafusion/sqllogictest/test_files/topk.slt index 8d3b70139d35..5eba20fdc655 100644 --- a/datafusion/sqllogictest/test_files/topk.slt +++ b/datafusion/sqllogictest/test_files/topk.slt @@ -81,7 +81,7 @@ Limit: skip=0, fetch=5 ----TableScan: aggregate_test_100 projection=[c1, c2, c3, c4, c5, c6, c7, c8, c9, c10, c11, c12, c13] physical_plan GlobalLimitExec: skip=0, fetch=5 ---SortExec: fetch=5, expr=[c13@12 DESC] +--SortExec: TopK(fetch=5), expr=[c13@12 DESC] ----CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1, c2, c3, c4, c5, c6, c7, c8, c9, c10, c11, c12, c13], has_header=true diff --git a/datafusion/sqllogictest/test_files/tpch/q10.slt.part b/datafusion/sqllogictest/test_files/tpch/q10.slt.part index 708bcb3c9b6f..eb0b66f024de 100644 --- a/datafusion/sqllogictest/test_files/tpch/q10.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/q10.slt.part @@ -72,7 +72,7 @@ Limit: skip=0, fetch=10 physical_plan GlobalLimitExec: skip=0, fetch=10 --SortPreservingMergeExec: [revenue@2 DESC], fetch=10 -----SortExec: fetch=10, expr=[revenue@2 DESC] +----SortExec: TopK(fetch=10), expr=[revenue@2 DESC] ------ProjectionExec: expr=[c_custkey@0 as c_custkey, c_name@1 as c_name, SUM(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)@7 as revenue, c_acctbal@2 as c_acctbal, n_name@4 as n_name, c_address@5 as c_address, c_phone@3 as c_phone, c_comment@6 as c_comment] --------AggregateExec: mode=FinalPartitioned, gby=[c_custkey@0 as c_custkey, c_name@1 as c_name, c_acctbal@2 as c_acctbal, c_phone@3 as c_phone, n_name@4 as n_name, c_address@5 as c_address, c_comment@6 as c_comment], aggr=[SUM(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)] ----------CoalesceBatchesExec: target_batch_size=8192 diff --git a/datafusion/sqllogictest/test_files/tpch/q11.slt.part b/datafusion/sqllogictest/test_files/tpch/q11.slt.part index 0a045d4f77ca..4efa29e2c0ac 100644 --- a/datafusion/sqllogictest/test_files/tpch/q11.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/q11.slt.part @@ -76,7 +76,7 @@ Limit: skip=0, fetch=10 physical_plan GlobalLimitExec: skip=0, fetch=10 --SortPreservingMergeExec: [value@1 DESC], fetch=10 -----SortExec: fetch=10, expr=[value@1 DESC] +----SortExec: TopK(fetch=10), expr=[value@1 DESC] ------ProjectionExec: expr=[ps_partkey@0 as ps_partkey, SUM(partsupp.ps_supplycost * partsupp.ps_availqty)@1 as value] --------NestedLoopJoinExec: join_type=Inner, filter=CAST(SUM(partsupp.ps_supplycost * partsupp.ps_availqty)@0 AS Decimal128(38, 15)) > SUM(partsupp.ps_supplycost * partsupp.ps_availqty) * Float64(0.0001)@1 ----------AggregateExec: mode=FinalPartitioned, gby=[ps_partkey@0 as ps_partkey], aggr=[SUM(partsupp.ps_supplycost * partsupp.ps_availqty)] diff --git a/datafusion/sqllogictest/test_files/tpch/q13.slt.part b/datafusion/sqllogictest/test_files/tpch/q13.slt.part index bb33c2ad3419..5cf6ace8b27b 100644 --- a/datafusion/sqllogictest/test_files/tpch/q13.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/q13.slt.part @@ -56,7 +56,7 @@ Limit: skip=0, fetch=10 physical_plan GlobalLimitExec: skip=0, fetch=10 --SortPreservingMergeExec: [custdist@1 DESC,c_count@0 DESC], fetch=10 -----SortExec: fetch=10, expr=[custdist@1 DESC,c_count@0 DESC] +----SortExec: TopK(fetch=10), expr=[custdist@1 DESC,c_count@0 DESC] ------ProjectionExec: expr=[c_count@0 as c_count, COUNT(*)@1 as custdist] --------AggregateExec: mode=FinalPartitioned, gby=[c_count@0 as c_count], aggr=[COUNT(*)] ----------CoalesceBatchesExec: target_batch_size=8192 diff --git a/datafusion/sqllogictest/test_files/tpch/q16.slt.part b/datafusion/sqllogictest/test_files/tpch/q16.slt.part index 5247fbc90d7c..fb9d98b76fe3 100644 --- a/datafusion/sqllogictest/test_files/tpch/q16.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/q16.slt.part @@ -68,7 +68,7 @@ Limit: skip=0, fetch=10 physical_plan GlobalLimitExec: skip=0, fetch=10 --SortPreservingMergeExec: [supplier_cnt@3 DESC,p_brand@0 ASC NULLS LAST,p_type@1 ASC NULLS LAST,p_size@2 ASC NULLS LAST], fetch=10 -----SortExec: fetch=10, expr=[supplier_cnt@3 DESC,p_brand@0 ASC NULLS LAST,p_type@1 ASC NULLS LAST,p_size@2 ASC NULLS LAST] +----SortExec: TopK(fetch=10), expr=[supplier_cnt@3 DESC,p_brand@0 ASC NULLS LAST,p_type@1 ASC NULLS LAST,p_size@2 ASC NULLS LAST] ------ProjectionExec: expr=[group_alias_0@0 as part.p_brand, group_alias_1@1 as part.p_type, group_alias_2@2 as part.p_size, COUNT(alias1)@3 as supplier_cnt] --------AggregateExec: mode=FinalPartitioned, gby=[group_alias_0@0 as group_alias_0, group_alias_1@1 as group_alias_1, group_alias_2@2 as group_alias_2], aggr=[COUNT(alias1)] ----------CoalesceBatchesExec: target_batch_size=8192 diff --git a/datafusion/sqllogictest/test_files/tpch/q2.slt.part b/datafusion/sqllogictest/test_files/tpch/q2.slt.part index f98634033bf8..ed439348d22d 100644 --- a/datafusion/sqllogictest/test_files/tpch/q2.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/q2.slt.part @@ -102,7 +102,7 @@ Limit: skip=0, fetch=10 physical_plan GlobalLimitExec: skip=0, fetch=10 --SortPreservingMergeExec: [s_acctbal@0 DESC,n_name@2 ASC NULLS LAST,s_name@1 ASC NULLS LAST,p_partkey@3 ASC NULLS LAST], fetch=10 -----SortExec: fetch=10, expr=[s_acctbal@0 DESC,n_name@2 ASC NULLS LAST,s_name@1 ASC NULLS LAST,p_partkey@3 ASC NULLS LAST] +----SortExec: TopK(fetch=10), expr=[s_acctbal@0 DESC,n_name@2 ASC NULLS LAST,s_name@1 ASC NULLS LAST,p_partkey@3 ASC NULLS LAST] ------ProjectionExec: expr=[s_acctbal@5 as s_acctbal, s_name@2 as s_name, n_name@8 as n_name, p_partkey@0 as p_partkey, p_mfgr@1 as p_mfgr, s_address@3 as s_address, s_phone@4 as s_phone, s_comment@6 as s_comment] --------CoalesceBatchesExec: target_batch_size=8192 ----------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(p_partkey@0, ps_partkey@1), (ps_supplycost@7, MIN(partsupp.ps_supplycost)@0)] diff --git a/datafusion/sqllogictest/test_files/tpch/q3.slt.part b/datafusion/sqllogictest/test_files/tpch/q3.slt.part index 634f06d0bf50..85f2d9986c27 100644 --- a/datafusion/sqllogictest/test_files/tpch/q3.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/q3.slt.part @@ -61,7 +61,7 @@ Limit: skip=0, fetch=10 physical_plan GlobalLimitExec: skip=0, fetch=10 --SortPreservingMergeExec: [revenue@1 DESC,o_orderdate@2 ASC NULLS LAST], fetch=10 -----SortExec: fetch=10, expr=[revenue@1 DESC,o_orderdate@2 ASC NULLS LAST] +----SortExec: TopK(fetch=10), expr=[revenue@1 DESC,o_orderdate@2 ASC NULLS LAST] ------ProjectionExec: expr=[l_orderkey@0 as l_orderkey, SUM(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)@3 as revenue, o_orderdate@1 as o_orderdate, o_shippriority@2 as o_shippriority] --------AggregateExec: mode=FinalPartitioned, gby=[l_orderkey@0 as l_orderkey, o_orderdate@1 as o_orderdate, o_shippriority@2 as o_shippriority], aggr=[SUM(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)] ----------CoalesceBatchesExec: target_batch_size=8192 diff --git a/datafusion/sqllogictest/test_files/tpch/q9.slt.part b/datafusion/sqllogictest/test_files/tpch/q9.slt.part index fc5f82008dad..5db97f79bdb1 100644 --- a/datafusion/sqllogictest/test_files/tpch/q9.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/q9.slt.part @@ -78,7 +78,7 @@ Limit: skip=0, fetch=10 physical_plan GlobalLimitExec: skip=0, fetch=10 --SortPreservingMergeExec: [nation@0 ASC NULLS LAST,o_year@1 DESC], fetch=10 -----SortExec: fetch=10, expr=[nation@0 ASC NULLS LAST,o_year@1 DESC] +----SortExec: TopK(fetch=10), expr=[nation@0 ASC NULLS LAST,o_year@1 DESC] ------ProjectionExec: expr=[nation@0 as nation, o_year@1 as o_year, SUM(profit.amount)@2 as sum_profit] --------AggregateExec: mode=FinalPartitioned, gby=[nation@0 as nation, o_year@1 as o_year], aggr=[SUM(profit.amount)] ----------CoalesceBatchesExec: target_batch_size=8192 diff --git a/datafusion/sqllogictest/test_files/window.slt b/datafusion/sqllogictest/test_files/window.slt index 80e496a336f4..aa7b3d5bb99c 100644 --- a/datafusion/sqllogictest/test_files/window.slt +++ b/datafusion/sqllogictest/test_files/window.slt @@ -2015,7 +2015,7 @@ ProjectionExec: expr=[ARRAY_AGG(aggregate_test_100.c13)@0 as array_agg1] ------AggregateExec: mode=Partial, gby=[], aggr=[ARRAY_AGG(aggregate_test_100.c13)] --------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 ----------GlobalLimitExec: skip=0, fetch=1 -------------SortExec: fetch=1, expr=[c13@0 ASC NULLS LAST] +------------SortExec: TopK(fetch=1), expr=[c13@0 ASC NULLS LAST] --------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c13], has_header=true @@ -2075,7 +2075,7 @@ Limit: skip=0, fetch=5 ----------------TableScan: aggregate_test_100 projection=[c1, c2, c8, c9] physical_plan GlobalLimitExec: skip=0, fetch=5 ---SortExec: fetch=5, expr=[c9@0 ASC NULLS LAST] +--SortExec: TopK(fetch=5), expr=[c9@0 ASC NULLS LAST] ----ProjectionExec: expr=[c9@2 as c9, SUM(aggregate_test_100.c9) PARTITION BY [aggregate_test_100.c1, aggregate_test_100.c2] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING@4 as sum1, SUM(aggregate_test_100.c9) PARTITION BY [aggregate_test_100.c2, aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING@6 as sum2, SUM(aggregate_test_100.c9) PARTITION BY [aggregate_test_100.c1, aggregate_test_100.c2] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST, aggregate_test_100.c8 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND UNBOUNDED FOLLOWING@3 as sum3, SUM(aggregate_test_100.c9) PARTITION BY [aggregate_test_100.c2, aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST, aggregate_test_100.c8 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND UNBOUNDED FOLLOWING@5 as sum4] ------BoundedWindowAggExec: wdw=[SUM(aggregate_test_100.c9) PARTITION BY [aggregate_test_100.c2, aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING: Ok(Field { name: "SUM(aggregate_test_100.c9) PARTITION BY [aggregate_test_100.c2, aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING", data_type: UInt64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(1)), end_bound: Following(UInt64(5)) }], mode=[Sorted] --------ProjectionExec: expr=[c1@0 as c1, c2@1 as c2, c9@3 as c9, SUM(aggregate_test_100.c9) PARTITION BY [aggregate_test_100.c1, aggregate_test_100.c2] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST, aggregate_test_100.c8 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND UNBOUNDED FOLLOWING@4 as SUM(aggregate_test_100.c9) PARTITION BY [aggregate_test_100.c1, aggregate_test_100.c2] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST, aggregate_test_100.c8 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND UNBOUNDED FOLLOWING, SUM(aggregate_test_100.c9) PARTITION BY [aggregate_test_100.c1, aggregate_test_100.c2] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING@5 as SUM(aggregate_test_100.c9) PARTITION BY [aggregate_test_100.c1, aggregate_test_100.c2] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING, SUM(aggregate_test_100.c9) PARTITION BY [aggregate_test_100.c2, aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST, aggregate_test_100.c8 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND UNBOUNDED FOLLOWING@6 as SUM(aggregate_test_100.c9) PARTITION BY [aggregate_test_100.c2, aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST, aggregate_test_100.c8 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND UNBOUNDED FOLLOWING] @@ -2182,7 +2182,7 @@ Projection: sum1, sum2 physical_plan ProjectionExec: expr=[sum1@0 as sum1, sum2@1 as sum2] --GlobalLimitExec: skip=0, fetch=5 -----SortExec: fetch=5, expr=[c9@2 ASC NULLS LAST] +----SortExec: TopK(fetch=5), expr=[c9@2 ASC NULLS LAST] ------ProjectionExec: expr=[SUM(aggregate_test_100.c12) ORDER BY [aggregate_test_100.c1 ASC NULLS LAST, aggregate_test_100.c2 ASC NULLS LAST] GROUPS BETWEEN 1 PRECEDING AND 1 FOLLOWING@3 as sum1, SUM(aggregate_test_100.c12) ORDER BY [aggregate_test_100.c1 ASC NULLS LAST] GROUPS BETWEEN 5 PRECEDING AND 3 PRECEDING@4 as sum2, c9@1 as c9] --------BoundedWindowAggExec: wdw=[SUM(aggregate_test_100.c12) ORDER BY [aggregate_test_100.c1 ASC NULLS LAST] GROUPS BETWEEN 5 PRECEDING AND 3 PRECEDING: Ok(Field { name: "SUM(aggregate_test_100.c12) ORDER BY [aggregate_test_100.c1 ASC NULLS LAST] GROUPS BETWEEN 5 PRECEDING AND 3 PRECEDING", data_type: Float64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Groups, start_bound: Preceding(UInt64(5)), end_bound: Preceding(UInt64(3)) }], mode=[Sorted] ----------ProjectionExec: expr=[c1@0 as c1, c9@2 as c9, c12@3 as c12, SUM(aggregate_test_100.c12) ORDER BY [aggregate_test_100.c1 ASC NULLS LAST, aggregate_test_100.c2 ASC NULLS LAST] GROUPS BETWEEN 1 PRECEDING AND 1 FOLLOWING@4 as SUM(aggregate_test_100.c12) ORDER BY [aggregate_test_100.c1 ASC NULLS LAST, aggregate_test_100.c2 ASC NULLS LAST] GROUPS BETWEEN 1 PRECEDING AND 1 FOLLOWING] @@ -2299,7 +2299,7 @@ Limit: skip=0, fetch=5 ----------TableScan: aggregate_test_100 projection=[c9] physical_plan GlobalLimitExec: skip=0, fetch=5 ---SortExec: fetch=5, expr=[rn1@1 DESC] +--SortExec: TopK(fetch=5), expr=[rn1@1 DESC] ----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] ------BoundedWindowAggExec: wdw=[ROW_NUMBER() ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "ROW_NUMBER() ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW", data_type: UInt64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(UInt64(NULL)), end_bound: CurrentRow }], mode=[Sorted] --------SortExec: expr=[c9@0 DESC] @@ -2550,7 +2550,7 @@ Projection: sum1, sum2, sum3, min1, min2, min3, max1, max2, max3, cnt1, cnt2, su physical_plan ProjectionExec: expr=[sum1@0 as sum1, sum2@1 as sum2, sum3@2 as sum3, min1@3 as min1, min2@4 as min2, min3@5 as min3, max1@6 as max1, max2@7 as max2, max3@8 as max3, cnt1@9 as cnt1, cnt2@10 as cnt2, sumr1@11 as sumr1, sumr2@12 as sumr2, sumr3@13 as sumr3, minr1@14 as minr1, minr2@15 as minr2, minr3@16 as minr3, maxr1@17 as maxr1, maxr2@18 as maxr2, maxr3@19 as maxr3, cntr1@20 as cntr1, cntr2@21 as cntr2, sum4@22 as sum4, cnt3@23 as cnt3] --GlobalLimitExec: skip=0, fetch=5 -----SortExec: fetch=5, expr=[inc_col@24 DESC] +----SortExec: TopK(fetch=5), expr=[inc_col@24 DESC] ------ProjectionExec: expr=[SUM(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING@13 as sum1, SUM(annotated_data_finite.desc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 5 PRECEDING AND 1 FOLLOWING@14 as sum2, SUM(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 10 FOLLOWING@15 as sum3, MIN(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING@16 as min1, MIN(annotated_data_finite.desc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 5 PRECEDING AND 1 FOLLOWING@17 as min2, MIN(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 10 FOLLOWING@18 as min3, MAX(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING@19 as max1, MAX(annotated_data_finite.desc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 5 PRECEDING AND 1 FOLLOWING@20 as max2, MAX(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 10 FOLLOWING@21 as max3, COUNT(*) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 4 PRECEDING AND 8 FOLLOWING@22 as cnt1, COUNT(*) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 8 PRECEDING AND 1 FOLLOWING@23 as cnt2, SUM(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 1 PRECEDING AND 4 FOLLOWING@2 as sumr1, SUM(annotated_data_finite.desc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 1 PRECEDING AND 8 FOLLOWING@3 as sumr2, SUM(annotated_data_finite.desc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING@4 as sumr3, MIN(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING@5 as minr1, MIN(annotated_data_finite.desc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 5 PRECEDING AND 1 FOLLOWING@6 as minr2, MIN(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 1 PRECEDING AND 10 FOLLOWING@7 as minr3, MAX(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING@8 as maxr1, MAX(annotated_data_finite.desc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 5 PRECEDING AND 1 FOLLOWING@9 as maxr2, MAX(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 1 PRECEDING AND 10 FOLLOWING@10 as maxr3, COUNT(*) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING@11 as cntr1, COUNT(*) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 8 PRECEDING AND 1 FOLLOWING@12 as cntr2, SUM(annotated_data_finite.desc_col) ROWS BETWEEN 8 PRECEDING AND 1 FOLLOWING@24 as sum4, COUNT(*) ROWS BETWEEN 8 PRECEDING AND 1 FOLLOWING@25 as cnt3, inc_col@0 as inc_col] --------BoundedWindowAggExec: wdw=[SUM(annotated_data_finite.desc_col) ROWS BETWEEN 8 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "SUM(annotated_data_finite.desc_col) ROWS BETWEEN 8 PRECEDING AND 1 FOLLOWING", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(8)), end_bound: Following(UInt64(1)) }, COUNT(*) ROWS BETWEEN 8 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "COUNT(*) ROWS BETWEEN 8 PRECEDING AND 1 FOLLOWING", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(8)), end_bound: Following(UInt64(1)) }], mode=[Sorted] ----------ProjectionExec: expr=[inc_col@2 as inc_col, desc_col@3 as desc_col, SUM(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 1 PRECEDING AND 4 FOLLOWING@4 as SUM(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 1 PRECEDING AND 4 FOLLOWING, SUM(annotated_data_finite.desc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 1 PRECEDING AND 8 FOLLOWING@5 as SUM(annotated_data_finite.desc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 1 PRECEDING AND 8 FOLLOWING, SUM(annotated_data_finite.desc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING@6 as SUM(annotated_data_finite.desc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING, MIN(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING@7 as MIN(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING, MIN(annotated_data_finite.desc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 5 PRECEDING AND 1 FOLLOWING@8 as MIN(annotated_data_finite.desc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 5 PRECEDING AND 1 FOLLOWING, MIN(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 1 PRECEDING AND 10 FOLLOWING@9 as MIN(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 1 PRECEDING AND 10 FOLLOWING, MAX(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING@10 as MAX(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING, MAX(annotated_data_finite.desc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 5 PRECEDING AND 1 FOLLOWING@11 as MAX(annotated_data_finite.desc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 5 PRECEDING AND 1 FOLLOWING, MAX(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 1 PRECEDING AND 10 FOLLOWING@12 as MAX(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 1 PRECEDING AND 10 FOLLOWING, COUNT(*) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING@13 as COUNT(*) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING, COUNT(*) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 8 PRECEDING AND 1 FOLLOWING@14 as COUNT(*) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 8 PRECEDING AND 1 FOLLOWING, SUM(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING@15 as SUM(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING, SUM(annotated_data_finite.desc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 5 PRECEDING AND 1 FOLLOWING@16 as SUM(annotated_data_finite.desc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 5 PRECEDING AND 1 FOLLOWING, SUM(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 10 FOLLOWING@17 as SUM(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 10 FOLLOWING, MIN(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING@18 as MIN(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING, MIN(annotated_data_finite.desc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 5 PRECEDING AND 1 FOLLOWING@19 as MIN(annotated_data_finite.desc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 5 PRECEDING AND 1 FOLLOWING, MIN(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 10 FOLLOWING@20 as MIN(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 10 FOLLOWING, MAX(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING@21 as MAX(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING, MAX(annotated_data_finite.desc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 5 PRECEDING AND 1 FOLLOWING@22 as MAX(annotated_data_finite.desc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 5 PRECEDING AND 1 FOLLOWING, MAX(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 10 FOLLOWING@23 as MAX(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 10 FOLLOWING, COUNT(*) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 4 PRECEDING AND 8 FOLLOWING@24 as COUNT(*) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 4 PRECEDING AND 8 FOLLOWING, COUNT(*) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 8 PRECEDING AND 1 FOLLOWING@25 as COUNT(*) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 8 PRECEDING AND 1 FOLLOWING] @@ -2639,7 +2639,7 @@ Limit: skip=0, fetch=5 ----------TableScan: annotated_data_finite projection=[ts, inc_col] physical_plan GlobalLimitExec: skip=0, fetch=5 ---SortExec: fetch=5, expr=[ts@0 DESC] +--SortExec: TopK(fetch=5), expr=[ts@0 DESC] ----ProjectionExec: expr=[ts@0 as ts, FIRST_VALUE(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING@10 as fv1, FIRST_VALUE(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING@11 as fv2, LAST_VALUE(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING@12 as lv1, LAST_VALUE(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING@13 as lv2, NTH_VALUE(annotated_data_finite.inc_col,Int64(5)) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING@14 as nv1, NTH_VALUE(annotated_data_finite.inc_col,Int64(5)) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING@15 as nv2, ROW_NUMBER() ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING@16 as rn1, ROW_NUMBER() ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING@17 as rn2, RANK() ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING@18 as rank1, RANK() ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING@19 as rank2, DENSE_RANK() ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING@20 as dense_rank1, DENSE_RANK() ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING@21 as dense_rank2, LAG(annotated_data_finite.inc_col,Int64(1),Int64(1001)) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING@22 as lag1, LAG(annotated_data_finite.inc_col,Int64(2),Int64(1002)) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING@23 as lag2, LEAD(annotated_data_finite.inc_col,Int64(-1),Int64(1001)) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING@24 as lead1, LEAD(annotated_data_finite.inc_col,Int64(4),Int64(1004)) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING@25 as lead2, FIRST_VALUE(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING@2 as fvr1, FIRST_VALUE(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING@3 as fvr2, LAST_VALUE(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING@4 as lvr1, LAST_VALUE(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING@5 as lvr2, LAG(annotated_data_finite.inc_col,Int64(1),Int64(1001)) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING@6 as lagr1, LAG(annotated_data_finite.inc_col,Int64(2),Int64(1002)) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING@7 as lagr2, LEAD(annotated_data_finite.inc_col,Int64(-1),Int64(1001)) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING@8 as leadr1, LEAD(annotated_data_finite.inc_col,Int64(4),Int64(1004)) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING@9 as leadr2] ------BoundedWindowAggExec: wdw=[FIRST_VALUE(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "FIRST_VALUE(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(10)), end_bound: Following(Int32(1)) }, FIRST_VALUE(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "FIRST_VALUE(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(10)), end_bound: Following(UInt64(1)) }, LAST_VALUE(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "LAST_VALUE(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(10)), end_bound: Following(Int32(1)) }, LAST_VALUE(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "LAST_VALUE(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(10)), end_bound: Following(UInt64(1)) }, NTH_VALUE(annotated_data_finite.inc_col,Int64(5)) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "NTH_VALUE(annotated_data_finite.inc_col,Int64(5)) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(10)), end_bound: Following(Int32(1)) }, NTH_VALUE(annotated_data_finite.inc_col,Int64(5)) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "NTH_VALUE(annotated_data_finite.inc_col,Int64(5)) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(10)), end_bound: Following(UInt64(1)) }, ROW_NUMBER() ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING: Ok(Field { name: "ROW_NUMBER() ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING", data_type: UInt64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(1)), end_bound: Following(Int32(10)) }, ROW_NUMBER() ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "ROW_NUMBER() ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING", data_type: UInt64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(10)), end_bound: Following(UInt64(1)) }, RANK() ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING: Ok(Field { name: "RANK() ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING", data_type: UInt64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(1)), end_bound: Following(Int32(10)) }, RANK() ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "RANK() ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING", data_type: UInt64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(10)), end_bound: Following(UInt64(1)) }, DENSE_RANK() ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING: Ok(Field { name: "DENSE_RANK() ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING", data_type: UInt64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(1)), end_bound: Following(Int32(10)) }, DENSE_RANK() ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "DENSE_RANK() ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING", data_type: UInt64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(10)), end_bound: Following(UInt64(1)) }, LAG(annotated_data_finite.inc_col,Int64(1),Int64(1001)) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING: Ok(Field { name: "LAG(annotated_data_finite.inc_col,Int64(1),Int64(1001)) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(1)), end_bound: Following(Int32(10)) }, LAG(annotated_data_finite.inc_col,Int64(2),Int64(1002)) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "LAG(annotated_data_finite.inc_col,Int64(2),Int64(1002)) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(10)), end_bound: Following(UInt64(1)) }, LEAD(annotated_data_finite.inc_col,Int64(-1),Int64(1001)) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING: Ok(Field { name: "LEAD(annotated_data_finite.inc_col,Int64(-1),Int64(1001)) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(1)), end_bound: Following(Int32(10)) }, LEAD(annotated_data_finite.inc_col,Int64(4),Int64(1004)) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "LEAD(annotated_data_finite.inc_col,Int64(4),Int64(1004)) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(10)), end_bound: Following(UInt64(1)) }], mode=[Sorted] --------BoundedWindowAggExec: wdw=[FIRST_VALUE(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "FIRST_VALUE(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(1)), end_bound: Following(Int32(10)) }, FIRST_VALUE(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "FIRST_VALUE(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(1)), end_bound: Following(UInt64(10)) }, LAST_VALUE(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "LAST_VALUE(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(1)), end_bound: Following(Int32(10)) }, LAST_VALUE(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "LAST_VALUE(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(1)), end_bound: Following(UInt64(10)) }, LAG(annotated_data_finite.inc_col,Int64(1),Int64(1001)) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING: Ok(Field { name: "LAG(annotated_data_finite.inc_col,Int64(1),Int64(1001)) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(10)), end_bound: Following(Int32(1)) }, LAG(annotated_data_finite.inc_col,Int64(2),Int64(1002)) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "LAG(annotated_data_finite.inc_col,Int64(2),Int64(1002)) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(1)), end_bound: Following(UInt64(10)) }, LEAD(annotated_data_finite.inc_col,Int64(-1),Int64(1001)) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING: Ok(Field { name: "LEAD(annotated_data_finite.inc_col,Int64(-1),Int64(1001)) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(10)), end_bound: Following(Int32(1)) }, LEAD(annotated_data_finite.inc_col,Int64(4),Int64(1004)) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "LEAD(annotated_data_finite.inc_col,Int64(4),Int64(1004)) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(1)), end_bound: Following(UInt64(10)) }], mode=[Sorted] @@ -2712,7 +2712,7 @@ Projection: sum1, sum2, min1, min2, max1, max2, count1, count2, avg1, avg2 physical_plan ProjectionExec: expr=[sum1@0 as sum1, sum2@1 as sum2, min1@2 as min1, min2@3 as min2, max1@4 as max1, max2@5 as max2, count1@6 as count1, count2@7 as count2, avg1@8 as avg1, avg2@9 as avg2] --GlobalLimitExec: skip=0, fetch=5 -----SortExec: fetch=5, expr=[inc_col@10 ASC NULLS LAST] +----SortExec: TopK(fetch=5), expr=[inc_col@10 ASC NULLS LAST] ------ProjectionExec: expr=[SUM(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND 5 FOLLOWING@7 as sum1, SUM(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING@2 as sum2, MIN(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND 5 FOLLOWING@8 as min1, MIN(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING@3 as min2, MAX(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND 5 FOLLOWING@9 as max1, MAX(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING@4 as max2, COUNT(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND 5 FOLLOWING@10 as count1, COUNT(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING@5 as count2, AVG(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND 5 FOLLOWING@11 as avg1, AVG(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING@6 as avg2, inc_col@1 as inc_col] --------BoundedWindowAggExec: wdw=[SUM(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND 5 FOLLOWING: Ok(Field { name: "SUM(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND 5 FOLLOWING", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(NULL)), end_bound: Following(Int32(5)) }, MIN(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND 5 FOLLOWING: Ok(Field { name: "MIN(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND 5 FOLLOWING", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(NULL)), end_bound: Following(Int32(5)) }, MAX(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND 5 FOLLOWING: Ok(Field { name: "MAX(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND 5 FOLLOWING", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(NULL)), end_bound: Following(Int32(5)) }, COUNT(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND 5 FOLLOWING: Ok(Field { name: "COUNT(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND 5 FOLLOWING", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(NULL)), end_bound: Following(Int32(5)) }, AVG(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND 5 FOLLOWING: Ok(Field { name: "AVG(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND 5 FOLLOWING", data_type: Float64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(NULL)), end_bound: Following(Int32(5)) }], mode=[Sorted] ----------BoundedWindowAggExec: wdw=[SUM(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING: Ok(Field { name: "SUM(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(NULL)), end_bound: Following(Int32(3)) }, MIN(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING: Ok(Field { name: "MIN(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(NULL)), end_bound: Following(Int32(3)) }, MAX(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING: Ok(Field { name: "MAX(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(NULL)), end_bound: Following(Int32(3)) }, COUNT(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING: Ok(Field { name: "COUNT(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(NULL)), end_bound: Following(Int32(3)) }, AVG(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING: Ok(Field { name: "AVG(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING", data_type: Float64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(NULL)), end_bound: Following(Int32(3)) }], mode=[Sorted] @@ -2765,7 +2765,7 @@ Projection: first_value1, first_value2, last_value1, last_value2, nth_value1 physical_plan ProjectionExec: expr=[first_value1@0 as first_value1, first_value2@1 as first_value2, last_value1@2 as last_value1, last_value2@3 as last_value2, nth_value1@4 as nth_value1] --GlobalLimitExec: skip=0, fetch=5 -----SortExec: fetch=5, expr=[inc_col@5 ASC NULLS LAST] +----SortExec: TopK(fetch=5), expr=[inc_col@5 ASC NULLS LAST] ------ProjectionExec: expr=[FIRST_VALUE(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING@4 as first_value1, FIRST_VALUE(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING@2 as first_value2, LAST_VALUE(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING@5 as last_value1, LAST_VALUE(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING@3 as last_value2, NTH_VALUE(annotated_data_finite.inc_col,Int64(2)) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING@6 as nth_value1, inc_col@1 as inc_col] --------BoundedWindowAggExec: wdw=[FIRST_VALUE(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING: Ok(Field { name: "FIRST_VALUE(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(NULL)), end_bound: Following(UInt64(1)) }, LAST_VALUE(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING: Ok(Field { name: "LAST_VALUE(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(NULL)), end_bound: Following(UInt64(1)) }, NTH_VALUE(annotated_data_finite.inc_col,Int64(2)) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING: Ok(Field { name: "NTH_VALUE(annotated_data_finite.inc_col,Int64(2)) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(NULL)), end_bound: Following(UInt64(1)) }], mode=[Sorted] ----------BoundedWindowAggExec: wdw=[FIRST_VALUE(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING: Ok(Field { name: "FIRST_VALUE(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(NULL)), end_bound: Following(UInt64(3)) }, LAST_VALUE(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING: Ok(Field { name: "LAST_VALUE(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(NULL)), end_bound: Following(UInt64(3)) }], mode=[Sorted] @@ -3026,7 +3026,7 @@ Limit: skip=0, fetch=5 --------------------TableScan: annotated_data_finite2 projection=[a, b, c, d] physical_plan GlobalLimitExec: skip=0, fetch=5 ---SortExec: fetch=5, expr=[c@2 ASC NULLS LAST] +--SortExec: TopK(fetch=5), expr=[c@2 ASC NULLS LAST] ----ProjectionExec: expr=[a@1 as a, b@2 as b, c@3 as c, SUM(annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.a, annotated_data_finite2.d] ORDER BY [annotated_data_finite2.b ASC NULLS LAST, annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN 2 PRECEDING AND 1 FOLLOWING@9 as sum1, SUM(annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.a, annotated_data_finite2.d] ORDER BY [annotated_data_finite2.b ASC NULLS LAST, annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN 1 FOLLOWING AND 5 FOLLOWING@10 as sum2, SUM(annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.d] ORDER BY [annotated_data_finite2.a ASC NULLS LAST, annotated_data_finite2.b ASC NULLS LAST, annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN 2 PRECEDING AND 1 FOLLOWING@15 as sum3, SUM(annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.d] ORDER BY [annotated_data_finite2.a ASC NULLS LAST, annotated_data_finite2.b ASC NULLS LAST, annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN 5 PRECEDING AND 1 PRECEDING@16 as sum4, SUM(annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.a, annotated_data_finite2.b] ORDER BY [annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN 2 PRECEDING AND 1 FOLLOWING@5 as sum5, SUM(annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.a, annotated_data_finite2.b] ORDER BY [annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN 5 PRECEDING AND 5 FOLLOWING@6 as sum6, SUM(annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.b, annotated_data_finite2.a] ORDER BY [annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN 2 PRECEDING AND 1 FOLLOWING@11 as sum7, SUM(annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.b, annotated_data_finite2.a] ORDER BY [annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN 5 PRECEDING AND 5 FOLLOWING@12 as sum8, SUM(annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.a, annotated_data_finite2.b, annotated_data_finite2.d] ORDER BY [annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN 2 PRECEDING AND 1 FOLLOWING@7 as sum9, SUM(annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.a, annotated_data_finite2.b, annotated_data_finite2.d] ORDER BY [annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN 5 PRECEDING AND CURRENT ROW@8 as sum10, SUM(annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.b, annotated_data_finite2.a, annotated_data_finite2.d] ORDER BY [annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN 2 PRECEDING AND 1 FOLLOWING@13 as sum11, SUM(annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.b, annotated_data_finite2.a, annotated_data_finite2.d] ORDER BY [annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN CURRENT ROW AND 1 FOLLOWING@14 as sum12] ------BoundedWindowAggExec: wdw=[SUM(annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.d] ORDER BY [annotated_data_finite2.a ASC NULLS LAST, annotated_data_finite2.b ASC NULLS LAST, annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN 2 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "SUM(annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.d] ORDER BY [annotated_data_finite2.a ASC NULLS LAST, annotated_data_finite2.b ASC NULLS LAST, annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN 2 PRECEDING AND 1 FOLLOWING", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(2)), end_bound: Following(UInt64(1)) }, SUM(annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.d] ORDER BY [annotated_data_finite2.a ASC NULLS LAST, annotated_data_finite2.b ASC NULLS LAST, annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN 5 PRECEDING AND 1 PRECEDING: Ok(Field { name: "SUM(annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.d] ORDER BY [annotated_data_finite2.a ASC NULLS LAST, annotated_data_finite2.b ASC NULLS LAST, annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN 5 PRECEDING AND 1 PRECEDING", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(5)), end_bound: Preceding(UInt64(1)) }], mode=[Sorted] --------SortExec: expr=[d@4 ASC NULLS LAST,a@1 ASC NULLS LAST,b@2 ASC NULLS LAST,c@3 ASC NULLS LAST] @@ -3128,7 +3128,7 @@ Limit: skip=0, fetch=5 ----------TableScan: aggregate_test_100 projection=[c9] physical_plan GlobalLimitExec: skip=0, fetch=5 ---SortExec: fetch=5, expr=[sum1@1 ASC NULLS LAST,c9@0 DESC] +--SortExec: TopK(fetch=5), expr=[sum1@1 ASC NULLS LAST,c9@0 DESC] ----ProjectionExec: expr=[c9@0 as c9, SUM(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@1 as sum1] ------BoundedWindowAggExec: wdw=[SUM(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "SUM(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW", data_type: UInt64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(UInt64(NULL)), end_bound: CurrentRow }], mode=[Sorted] --------SortExec: expr=[c9@0 DESC] @@ -3311,7 +3311,7 @@ Limit: skip=0, fetch=5 ------------TableScan: aggregate_test_100 projection=[c3, c11, c12] physical_plan GlobalLimitExec: skip=0, fetch=5 ---SortExec: fetch=5, expr=[c3@0 ASC NULLS LAST] +--SortExec: TopK(fetch=5), expr=[c3@0 ASC NULLS LAST] ----ProjectionExec: expr=[c3@0 as c3, MAX(aggregate_test_100.c12) ORDER BY [aggregate_test_100.c12 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@3 as min1, MIN(aggregate_test_100.c12) PARTITION BY [aggregate_test_100.c11] ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING@2 as max1] ------BoundedWindowAggExec: wdw=[MAX(aggregate_test_100.c12) ORDER BY [aggregate_test_100.c12 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "MAX(aggregate_test_100.c12) ORDER BY [aggregate_test_100.c12 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW", data_type: Float64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Float64(NULL)), end_bound: CurrentRow }], mode=[Sorted] --------SortExec: expr=[c12@1 ASC NULLS LAST] @@ -3357,7 +3357,7 @@ Projection: min1, max1 physical_plan ProjectionExec: expr=[min1@0 as min1, max1@1 as max1] --GlobalLimitExec: skip=0, fetch=5 -----SortExec: fetch=5, expr=[c3@2 ASC NULLS LAST] +----SortExec: TopK(fetch=5), expr=[c3@2 ASC NULLS LAST] ------ProjectionExec: expr=[MAX(aggregate_test_100.c12) ORDER BY [aggregate_test_100.c12 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@2 as min1, MIN(aggregate_test_100.c12) ORDER BY [aggregate_test_100.c12 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@3 as max1, c3@0 as c3] --------BoundedWindowAggExec: wdw=[MAX(aggregate_test_100.c12) ORDER BY [aggregate_test_100.c12 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "MAX(aggregate_test_100.c12) ORDER BY [aggregate_test_100.c12 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW", data_type: Float64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Float64(NULL)), end_bound: CurrentRow }, MIN(aggregate_test_100.c12) ORDER BY [aggregate_test_100.c12 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "MIN(aggregate_test_100.c12) ORDER BY [aggregate_test_100.c12 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW", data_type: Float64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Float64(NULL)), end_bound: CurrentRow }], mode=[Sorted] ----------SortExec: expr=[c12@1 ASC NULLS LAST] From fa2bb6c4e80d80e3d1d26ce85f7c21232f036dd5 Mon Sep 17 00:00:00 2001 From: Raphael Taylor-Davies <1781103+tustvold@users.noreply.github.com> Date: Mon, 16 Oct 2023 10:02:05 +0100 Subject: [PATCH 075/572] Extract ReceiverStreamBuilder (#7817) * Extract ReceiverStreamBuilder * Docs and format * Update datafusion/physical-plan/src/stream.rs * fmt * Undo changes to testing pin --------- Co-authored-by: Andrew Lamb --- datafusion/physical-plan/src/stream.rs | 232 ++++++++++++++----------- 1 file changed, 132 insertions(+), 100 deletions(-) diff --git a/datafusion/physical-plan/src/stream.rs b/datafusion/physical-plan/src/stream.rs index a3fb856c326d..fdf32620ca50 100644 --- a/datafusion/physical-plan/src/stream.rs +++ b/datafusion/physical-plan/src/stream.rs @@ -38,6 +38,124 @@ use tokio::task::JoinSet; use super::metrics::BaselineMetrics; use super::{ExecutionPlan, RecordBatchStream, SendableRecordBatchStream}; +/// Creates a stream from a collection of producing tasks, routing panics to the stream. +/// +/// Note that this is similar to [`ReceiverStream` from tokio-stream], with the differences being: +/// +/// 1. Methods to bound and "detach" tasks (`spawn()` and `spawn_blocking()`). +/// +/// 2. Propagates panics, whereas the `tokio` version doesn't propagate panics to the receiver. +/// +/// 3. Automatically cancels any outstanding tasks when the receiver stream is dropped. +/// +/// [`ReceiverStream` from tokio-stream]: https://docs.rs/tokio-stream/latest/tokio_stream/wrappers/struct.ReceiverStream.html + +pub(crate) struct ReceiverStreamBuilder { + tx: Sender>, + rx: Receiver>, + join_set: JoinSet>, +} + +impl ReceiverStreamBuilder { + /// create new channels with the specified buffer size + pub fn new(capacity: usize) -> Self { + let (tx, rx) = tokio::sync::mpsc::channel(capacity); + + Self { + tx, + rx, + join_set: JoinSet::new(), + } + } + + /// Get a handle for sending data to the output + pub fn tx(&self) -> Sender> { + self.tx.clone() + } + + /// Spawn task that will be aborted if this builder (or the stream + /// built from it) are dropped + pub fn spawn(&mut self, task: F) + where + F: Future>, + F: Send + 'static, + { + self.join_set.spawn(task); + } + + /// Spawn a blocking task that will be aborted if this builder (or the stream + /// built from it) are dropped + /// + /// this is often used to spawn tasks that write to the sender + /// retrieved from `Self::tx` + pub fn spawn_blocking(&mut self, f: F) + where + F: FnOnce() -> Result<()>, + F: Send + 'static, + { + self.join_set.spawn_blocking(f); + } + + /// Create a stream of all data written to `tx` + pub fn build(self) -> BoxStream<'static, Result> { + let Self { + tx, + rx, + mut join_set, + } = self; + + // don't need tx + drop(tx); + + // future that checks the result of the join set, and propagates panic if seen + let check = async move { + while let Some(result) = join_set.join_next().await { + match result { + Ok(task_result) => { + match task_result { + // nothing to report + Ok(_) => continue, + // This means a blocking task error + Err(e) => { + return Some(exec_err!("Spawned Task error: {e}")); + } + } + } + // This means a tokio task error, likely a panic + Err(e) => { + if e.is_panic() { + // resume on the main thread + std::panic::resume_unwind(e.into_panic()); + } else { + // This should only occur if the task is + // cancelled, which would only occur if + // the JoinSet were aborted, which in turn + // would imply that the receiver has been + // dropped and this code is not running + return Some(internal_err!("Non Panic Task error: {e}")); + } + } + } + } + None + }; + + let check_stream = futures::stream::once(check) + // unwrap Option / only return the error + .filter_map(|item| async move { item }); + + // Convert the receiver into a stream + let rx_stream = futures::stream::unfold(rx, |mut rx| async move { + let next_item = rx.recv().await; + next_item.map(|next_item| (next_item, rx)) + }); + + // Merge the streams together so whichever is ready first + // produces the batch + futures::stream::select(rx_stream, check_stream).boxed() + } +} + /// Builder for [`RecordBatchReceiverStream`] that propagates errors /// and panic's correctly. /// @@ -47,28 +165,22 @@ use super::{ExecutionPlan, RecordBatchStream, SendableRecordBatchStream}; /// /// This also handles propagating panic`s and canceling the tasks. pub struct RecordBatchReceiverStreamBuilder { - tx: Sender>, - rx: Receiver>, schema: SchemaRef, - join_set: JoinSet>, + inner: ReceiverStreamBuilder, } impl RecordBatchReceiverStreamBuilder { /// create new channels with the specified buffer size pub fn new(schema: SchemaRef, capacity: usize) -> Self { - let (tx, rx) = tokio::sync::mpsc::channel(capacity); - Self { - tx, - rx, schema, - join_set: JoinSet::new(), + inner: ReceiverStreamBuilder::new(capacity), } } - /// Get a handle for sending [`RecordBatch`]es to the output + /// Get a handle for sending [`RecordBatch`] to the output pub fn tx(&self) -> Sender> { - self.tx.clone() + self.inner.tx() } /// Spawn task that will be aborted if this builder (or the stream @@ -81,7 +193,7 @@ impl RecordBatchReceiverStreamBuilder { F: Future>, F: Send + 'static, { - self.join_set.spawn(task); + self.inner.spawn(task) } /// Spawn a blocking task that will be aborted if this builder (or the stream @@ -94,7 +206,7 @@ impl RecordBatchReceiverStreamBuilder { F: FnOnce() -> Result<()>, F: Send + 'static, { - self.join_set.spawn_blocking(f); + self.inner.spawn_blocking(f) } /// runs the input_partition of the `input` ExecutionPlan on the @@ -110,7 +222,7 @@ impl RecordBatchReceiverStreamBuilder { ) { let output = self.tx(); - self.spawn(async move { + self.inner.spawn(async move { let mut stream = match input.execute(partition, context) { Err(e) => { // If send fails, the plan being torn down, there @@ -155,80 +267,17 @@ impl RecordBatchReceiverStreamBuilder { }); } - /// Create a stream of all `RecordBatch`es written to `tx` + /// Create a stream of all [`RecordBatch`] written to `tx` pub fn build(self) -> SendableRecordBatchStream { - let Self { - tx, - rx, - schema, - mut join_set, - } = self; - - // don't need tx - drop(tx); - - // future that checks the result of the join set, and propagates panic if seen - let check = async move { - while let Some(result) = join_set.join_next().await { - match result { - Ok(task_result) => { - match task_result { - // nothing to report - Ok(_) => continue, - // This means a blocking task error - Err(e) => { - return Some(exec_err!("Spawned Task error: {e}")); - } - } - } - // This means a tokio task error, likely a panic - Err(e) => { - if e.is_panic() { - // resume on the main thread - std::panic::resume_unwind(e.into_panic()); - } else { - // This should only occur if the task is - // cancelled, which would only occur if - // the JoinSet were aborted, which in turn - // would imply that the receiver has been - // dropped and this code is not running - return Some(internal_err!("Non Panic Task error: {e}")); - } - } - } - } - None - }; - - let check_stream = futures::stream::once(check) - // unwrap Option / only return the error - .filter_map(|item| async move { item }); - - // Convert the receiver into a stream - let rx_stream = futures::stream::unfold(rx, |mut rx| async move { - let next_item = rx.recv().await; - next_item.map(|next_item| (next_item, rx)) - }); - - // Merge the streams together so whichever is ready first - // produces the batch - let inner = futures::stream::select(rx_stream, check_stream).boxed(); - - Box::pin(RecordBatchReceiverStream { schema, inner }) + Box::pin(RecordBatchStreamAdapter::new( + self.schema, + self.inner.build(), + )) } } -/// A [`SendableRecordBatchStream`] that combines [`RecordBatch`]es from multiple inputs, -/// on new tokio Tasks, increasing the potential parallelism. -/// -/// This structure also handles propagating panics and cancelling the -/// underlying tasks correctly. -/// -/// Use [`Self::builder`] to construct one. -pub struct RecordBatchReceiverStream { - schema: SchemaRef, - inner: BoxStream<'static, Result>, -} +#[doc(hidden)] +pub struct RecordBatchReceiverStream {} impl RecordBatchReceiverStream { /// Create a builder with an internal buffer of capacity batches. @@ -240,23 +289,6 @@ impl RecordBatchReceiverStream { } } -impl Stream for RecordBatchReceiverStream { - type Item = Result; - - fn poll_next( - mut self: Pin<&mut Self>, - cx: &mut Context<'_>, - ) -> Poll> { - self.inner.poll_next_unpin(cx) - } -} - -impl RecordBatchStream for RecordBatchReceiverStream { - fn schema(&self) -> SchemaRef { - self.schema.clone() - } -} - pin_project! { /// Combines a [`Stream`] with a [`SchemaRef`] implementing /// [`RecordBatchStream`] for the combination From cb6f7fe3a627529d3614cf4ea126a646f1c76557 Mon Sep 17 00:00:00 2001 From: comphead Date: Mon, 16 Oct 2023 07:08:20 -0700 Subject: [PATCH 076/572] Extend backtrace coverage for `DatafusionError::Plan` errors errors (#7803) * improve backtrace coverag for plan errors * fix cli * cli fmt * fix tests * docs * doc fmt --- datafusion-cli/src/exec.rs | 5 +- datafusion/common/src/dfschema.rs | 13 +++-- datafusion/common/src/error.rs | 26 +++++++-- .../common/src/functional_dependencies.rs | 25 ++++---- .../core/src/datasource/listing/table.rs | 14 ++--- datafusion/core/src/execution/context.rs | 29 ++++------ .../core/src/physical_optimizer/pruning.rs | 4 +- .../src/physical_optimizer/sort_pushdown.rs | 6 +- datafusion/execution/src/task.rs | 10 +--- datafusion/expr/src/aggregate_function.rs | 15 +++-- datafusion/expr/src/built_in_function.rs | 17 ++++-- datafusion/expr/src/expr_schema.rs | 8 +-- datafusion/expr/src/field_util.rs | 6 +- datafusion/expr/src/logical_plan/builder.rs | 10 ++-- datafusion/expr/src/type_coercion/binary.rs | 30 +++++----- datafusion/expr/src/utils.rs | 8 +-- datafusion/expr/src/window_function.rs | 15 +++-- .../optimizer/src/analyzer/type_coercion.rs | 30 +++++----- datafusion/optimizer/src/push_down_filter.rs | 6 +- datafusion/physical-plan/src/joins/utils.rs | 12 ++-- datafusion/proto/src/bytes/mod.rs | 58 +++++++++---------- .../proto/src/logical_plan/from_proto.rs | 8 +-- datafusion/proto/src/logical_plan/mod.rs | 5 +- datafusion/sql/src/expr/arrow_cast.rs | 8 +-- datafusion/sql/src/expr/function.rs | 6 +- datafusion/sql/src/expr/identifier.rs | 7 +-- datafusion/sql/src/expr/order_by.rs | 16 ++--- datafusion/sql/src/statement.rs | 22 +++---- datafusion/sql/tests/sql_integration.rs | 35 ++++++----- 29 files changed, 236 insertions(+), 218 deletions(-) diff --git a/datafusion-cli/src/exec.rs b/datafusion-cli/src/exec.rs index 100d7bce440c..b62ad12dbfbb 100644 --- a/datafusion-cli/src/exec.rs +++ b/datafusion-cli/src/exec.rs @@ -26,6 +26,7 @@ use crate::{ }, print_options::{MaxRows, PrintOptions}, }; +use datafusion::common::plan_datafusion_err; use datafusion::sql::{parser::DFParser, sqlparser::dialect::dialect_from_str}; use datafusion::{ datasource::listing::ListingTableUrl, @@ -202,11 +203,11 @@ async fn exec_and_print( let task_ctx = ctx.task_ctx(); let dialect = &task_ctx.session_config().options().sql_parser.dialect; let dialect = dialect_from_str(dialect).ok_or_else(|| { - DataFusionError::Plan(format!( + plan_datafusion_err!( "Unsupported SQL dialect: {dialect}. Available dialects: \ Generic, MySQL, PostgreSQL, Hive, SQLite, Snowflake, Redshift, \ MsSQL, ClickHouse, BigQuery, Ansi." - )) + ) })?; let statements = DFParser::parse_sql_with_dialect(&sql, dialect.as_ref())?; for statement in statements { diff --git a/datafusion/common/src/dfschema.rs b/datafusion/common/src/dfschema.rs index e015ef5c4082..b1aee41978c2 100644 --- a/datafusion/common/src/dfschema.rs +++ b/datafusion/common/src/dfschema.rs @@ -24,7 +24,9 @@ use std::fmt::{Display, Formatter}; use std::hash::Hash; use std::sync::Arc; -use crate::error::{unqualified_field_not_found, DataFusionError, Result, SchemaError}; +use crate::error::{ + unqualified_field_not_found, DataFusionError, Result, SchemaError, _plan_err, +}; use crate::{ field_not_found, Column, FunctionalDependencies, OwnedTableReference, TableReference, }; @@ -187,10 +189,10 @@ impl DFSchema { match &self.fields[i].qualifier { Some(qualifier) => { if (qualifier.to_string() + "." + self.fields[i].name()) == name { - return Err(DataFusionError::Plan(format!( + return _plan_err!( "Fully qualified field name '{name}' was supplied to `index_of` \ which is deprecated. Please use `index_of_column_by_name` instead" - ))); + ); } } None => (), @@ -378,12 +380,11 @@ impl DFSchema { .zip(arrow_schema.fields().iter()) .try_for_each(|(l_field, r_field)| { if !can_cast_types(r_field.data_type(), l_field.data_type()) { - Err(DataFusionError::Plan( - format!("Column {} (type: {}) is not compatible with column {} (type: {})", + _plan_err!("Column {} (type: {}) is not compatible with column {} (type: {})", r_field.name(), r_field.data_type(), l_field.name(), - l_field.data_type()))) + l_field.data_type()) } else { Ok(()) } diff --git a/datafusion/common/src/error.rs b/datafusion/common/src/error.rs index d7a0e1b59dba..adf58e282ed9 100644 --- a/datafusion/common/src/error.rs +++ b/datafusion/common/src/error.rs @@ -477,12 +477,25 @@ macro_rules! with_dollar_sign { /// plan_err!("Error {:?}", val) /// plan_err!("Error {val}") /// plan_err!("Error {val:?}") +/// +/// `NAME_ERR` - macro name for wrapping Err(DataFusionError::*) +/// `NAME_DF_ERR` - macro name for wrapping DataFusionError::*. Needed to keep backtrace opportunity +/// in construction where DataFusionError::* used directly, like `map_err`, `ok_or_else`, etc macro_rules! make_error { - ($NAME:ident, $ERR:ident) => { + ($NAME_ERR:ident, $NAME_DF_ERR: ident, $ERR:ident) => { with_dollar_sign! { ($d:tt) => { + /// Macro wraps `$ERR` to add backtrace feature + #[macro_export] + macro_rules! $NAME_DF_ERR { + ($d($d args:expr),*) => { + DataFusionError::$ERR(format!("{}{}", format!($d($d args),*), DataFusionError::get_back_trace()).into()) + } + } + + /// Macro wraps Err(`$ERR`) to add backtrace feature #[macro_export] - macro_rules! $NAME { + macro_rules! $NAME_ERR { ($d($d args:expr),*) => { Err(DataFusionError::$ERR(format!("{}{}", format!($d($d args),*), DataFusionError::get_back_trace()).into())) } @@ -493,16 +506,16 @@ macro_rules! make_error { } // Exposes a macro to create `DataFusionError::Plan` -make_error!(plan_err, Plan); +make_error!(plan_err, plan_datafusion_err, Plan); // Exposes a macro to create `DataFusionError::Internal` -make_error!(internal_err, Internal); +make_error!(internal_err, internal_datafusion_err, Internal); // Exposes a macro to create `DataFusionError::NotImplemented` -make_error!(not_impl_err, NotImplemented); +make_error!(not_impl_err, not_impl_datafusion_err, NotImplemented); // Exposes a macro to create `DataFusionError::Execution` -make_error!(exec_err, Execution); +make_error!(exec_err, exec_datafusion_err, Execution); // Exposes a macro to create `DataFusionError::SQL` #[macro_export] @@ -517,6 +530,7 @@ macro_rules! sql_err { pub use exec_err as _exec_err; pub use internal_err as _internal_err; pub use not_impl_err as _not_impl_err; +pub use plan_err as _plan_err; #[cfg(test)] mod test { diff --git a/datafusion/common/src/functional_dependencies.rs b/datafusion/common/src/functional_dependencies.rs index 324374f557b4..869709bc8dfc 100644 --- a/datafusion/common/src/functional_dependencies.rs +++ b/datafusion/common/src/functional_dependencies.rs @@ -23,6 +23,7 @@ use std::fmt::{Display, Formatter}; use std::ops::Deref; use std::vec::IntoIter; +use crate::error::_plan_err; use crate::{DFSchema, DFSchemaRef, DataFusionError, JoinType, Result}; use sqlparser::ast::TableConstraint; @@ -95,18 +96,18 @@ impl Constraints { Constraint::Unique(indices) }) } - TableConstraint::ForeignKey { .. } => Err(DataFusionError::Plan( - "Foreign key constraints are not currently supported".to_string(), - )), - TableConstraint::Check { .. } => Err(DataFusionError::Plan( - "Check constraints are not currently supported".to_string(), - )), - TableConstraint::Index { .. } => Err(DataFusionError::Plan( - "Indexes are not currently supported".to_string(), - )), - TableConstraint::FulltextOrSpatial { .. } => Err(DataFusionError::Plan( - "Indexes are not currently supported".to_string(), - )), + TableConstraint::ForeignKey { .. } => { + _plan_err!("Foreign key constraints are not currently supported") + } + TableConstraint::Check { .. } => { + _plan_err!("Check constraints are not currently supported") + } + TableConstraint::Index { .. } => { + _plan_err!("Indexes are not currently supported") + } + TableConstraint::FulltextOrSpatial { .. } => { + _plan_err!("Indexes are not currently supported") + } }) .collect::>>()?; Ok(Constraints::new_unverified(constraints)) diff --git a/datafusion/core/src/datasource/listing/table.rs b/datafusion/core/src/datasource/listing/table.rs index b6870ad2a827..064624366938 100644 --- a/datafusion/core/src/datasource/listing/table.rs +++ b/datafusion/core/src/datasource/listing/table.rs @@ -233,10 +233,10 @@ impl FromStr for ListingTableInsertMode { "append_to_file" => Ok(ListingTableInsertMode::AppendToFile), "append_new_files" => Ok(ListingTableInsertMode::AppendNewFiles), "error" => Ok(ListingTableInsertMode::Error), - _ => Err(DataFusionError::Plan(format!( + _ => plan_err!( "Unknown or unsupported insert mode {s}. Supported options are \ append_to_file, append_new_files, and error." - ))), + ), } } } @@ -865,10 +865,10 @@ impl TableProvider for ListingTable { let writer_mode = match self.options.insert_mode { ListingTableInsertMode::AppendToFile => { if input_partitions > file_groups.len() { - return Err(DataFusionError::Plan(format!( + return plan_err!( "Cannot append {input_partitions} partitions to {} files!", file_groups.len() - ))); + ); } crate::datasource::file_format::write::FileWriterMode::Append @@ -919,9 +919,9 @@ impl TableProvider for ListingTable { self.options().insert_mode, ListingTableInsertMode::AppendToFile ) { - return Err(DataFusionError::Plan( - "Cannot insert into a sorted ListingTable with mode append!".into(), - )); + return plan_err!( + "Cannot insert into a sorted ListingTable with mode append!" + ); } // Multiple sort orders in outer vec are equivalent, so we pass only the first one let ordering = self diff --git a/datafusion/core/src/execution/context.rs b/datafusion/core/src/execution/context.rs index ca6da6cfa047..b393d363ca0b 100644 --- a/datafusion/core/src/execution/context.rs +++ b/datafusion/core/src/execution/context.rs @@ -30,7 +30,7 @@ use crate::{ }; use datafusion_common::{ alias::AliasGenerator, - exec_err, not_impl_err, plan_err, + exec_err, not_impl_err, plan_datafusion_err, plan_err, tree_node::{TreeNode, TreeNodeVisitor, VisitRecursion}, }; use datafusion_execution::registry::SerializerRegistry; @@ -1577,17 +1577,14 @@ impl SessionState { self.catalog_list .catalog(&resolved_ref.catalog) .ok_or_else(|| { - DataFusionError::Plan(format!( + plan_datafusion_err!( "failed to resolve catalog: {}", resolved_ref.catalog - )) + ) })? .schema(&resolved_ref.schema) .ok_or_else(|| { - DataFusionError::Plan(format!( - "failed to resolve schema: {}", - resolved_ref.schema - )) + plan_datafusion_err!("failed to resolve schema: {}", resolved_ref.schema) }) } @@ -1689,11 +1686,11 @@ impl SessionState { dialect: &str, ) -> Result { let dialect = dialect_from_str(dialect).ok_or_else(|| { - DataFusionError::Plan(format!( + plan_datafusion_err!( "Unsupported SQL dialect: {dialect}. Available dialects: \ Generic, MySQL, PostgreSQL, Hive, SQLite, Snowflake, Redshift, \ MsSQL, ClickHouse, BigQuery, Ansi." - )) + ) })?; let mut statements = DFParser::parse_sql_with_dialect(sql, dialect.as_ref())?; if statements.len() > 1 { @@ -2022,7 +2019,7 @@ impl<'a> ContextProvider for SessionContextProvider<'a> { self.tables .get(&name) .cloned() - .ok_or_else(|| DataFusionError::Plan(format!("table '{name}' not found"))) + .ok_or_else(|| plan_datafusion_err!("table '{name}' not found")) } fn get_function_meta(&self, name: &str) -> Option> { @@ -2069,9 +2066,7 @@ impl FunctionRegistry for SessionState { let result = self.scalar_functions.get(name); result.cloned().ok_or_else(|| { - DataFusionError::Plan(format!( - "There is no UDF named \"{name}\" in the registry" - )) + plan_datafusion_err!("There is no UDF named \"{name}\" in the registry") }) } @@ -2079,9 +2074,7 @@ impl FunctionRegistry for SessionState { let result = self.aggregate_functions.get(name); result.cloned().ok_or_else(|| { - DataFusionError::Plan(format!( - "There is no UDAF named \"{name}\" in the registry" - )) + plan_datafusion_err!("There is no UDAF named \"{name}\" in the registry") }) } @@ -2089,9 +2082,7 @@ impl FunctionRegistry for SessionState { let result = self.window_functions.get(name); result.cloned().ok_or_else(|| { - DataFusionError::Plan(format!( - "There is no UDWF named \"{name}\" in the registry" - )) + plan_datafusion_err!("There is no UDWF named \"{name}\" in the registry") }) } } diff --git a/datafusion/core/src/physical_optimizer/pruning.rs b/datafusion/core/src/physical_optimizer/pruning.rs index 43a7dd8afe21..de508327fade 100644 --- a/datafusion/core/src/physical_optimizer/pruning.rs +++ b/datafusion/core/src/physical_optimizer/pruning.rs @@ -35,7 +35,7 @@ use arrow::{ datatypes::{DataType, Field, Schema, SchemaRef}, record_batch::RecordBatch, }; -use datafusion_common::{downcast_value, ScalarValue}; +use datafusion_common::{downcast_value, plan_datafusion_err, ScalarValue}; use datafusion_common::{ internal_err, plan_err, tree_node::{Transformed, TreeNode}, @@ -451,7 +451,7 @@ fn build_statistics_record_batch( ); RecordBatch::try_new_with_options(schema, arrays, &options).map_err(|err| { - DataFusionError::Plan(format!("Can not create statistics record batch: {err}")) + plan_datafusion_err!("Can not create statistics record batch: {err}") }) } diff --git a/datafusion/core/src/physical_optimizer/sort_pushdown.rs b/datafusion/core/src/physical_optimizer/sort_pushdown.rs index 629011cb0faa..bb991115ec75 100644 --- a/datafusion/core/src/physical_optimizer/sort_pushdown.rs +++ b/datafusion/core/src/physical_optimizer/sort_pushdown.rs @@ -29,7 +29,7 @@ use crate::physical_plan::sorts::sort::SortExec; use crate::physical_plan::{with_new_children_if_necessary, ExecutionPlan}; use datafusion_common::tree_node::{Transformed, TreeNode, VisitRecursion}; -use datafusion_common::{plan_err, DataFusionError, Result}; +use datafusion_common::{plan_datafusion_err, plan_err, DataFusionError, Result}; use datafusion_expr::JoinType; use datafusion_physical_expr::expressions::Column; use datafusion_physical_expr::utils::{ @@ -127,7 +127,7 @@ pub(crate) fn pushdown_sorts( let plan = &requirements.plan; let parent_required = requirements.required_ordering.as_deref(); const ERR_MSG: &str = "Expects parent requirement to contain something"; - let err = || DataFusionError::Plan(ERR_MSG.to_string()); + let err = || plan_datafusion_err!("{}", ERR_MSG); if let Some(sort_exec) = plan.as_any().downcast_ref::() { let mut new_plan = plan.clone(); if !ordering_satisfy_requirement( @@ -199,7 +199,7 @@ fn pushdown_requirement_to_children( parent_required: Option<&[PhysicalSortRequirement]>, ) -> Result>>>> { const ERR_MSG: &str = "Expects parent requirement to contain something"; - let err = || DataFusionError::Plan(ERR_MSG.to_string()); + let err = || plan_datafusion_err!("{}", ERR_MSG); let maintains_input_order = plan.maintains_input_order(); if is_window(plan) { let required_input_ordering = plan.required_input_ordering(); diff --git a/datafusion/execution/src/task.rs b/datafusion/execution/src/task.rs index 72d804d7bb9a..52c183b1612c 100644 --- a/datafusion/execution/src/task.rs +++ b/datafusion/execution/src/task.rs @@ -22,7 +22,7 @@ use std::{ use datafusion_common::{ config::{ConfigOptions, Extensions}, - DataFusionError, Result, + plan_datafusion_err, DataFusionError, Result, }; use datafusion_expr::{AggregateUDF, ScalarUDF, WindowUDF}; @@ -182,9 +182,7 @@ impl FunctionRegistry for TaskContext { let result = self.scalar_functions.get(name); result.cloned().ok_or_else(|| { - DataFusionError::Plan(format!( - "There is no UDF named \"{name}\" in the TaskContext" - )) + plan_datafusion_err!("There is no UDF named \"{name}\" in the TaskContext") }) } @@ -192,9 +190,7 @@ impl FunctionRegistry for TaskContext { let result = self.aggregate_functions.get(name); result.cloned().ok_or_else(|| { - DataFusionError::Plan(format!( - "There is no UDAF named \"{name}\" in the TaskContext" - )) + plan_datafusion_err!("There is no UDAF named \"{name}\" in the TaskContext") }) } diff --git a/datafusion/expr/src/aggregate_function.rs b/datafusion/expr/src/aggregate_function.rs index 1c8f34ec1d02..eaf4ff5ad806 100644 --- a/datafusion/expr/src/aggregate_function.rs +++ b/datafusion/expr/src/aggregate_function.rs @@ -20,7 +20,7 @@ use crate::utils; use crate::{type_coercion::aggregates::*, Signature, TypeSignature, Volatility}; use arrow::datatypes::{DataType, Field}; -use datafusion_common::{plan_err, DataFusionError, Result}; +use datafusion_common::{plan_datafusion_err, plan_err, DataFusionError, Result}; use std::sync::Arc; use std::{fmt, str::FromStr}; use strum_macros::EnumIter; @@ -232,11 +232,14 @@ impl AggregateFunction { // original errors are all related to wrong function signature // aggregate them for better error message .map_err(|_| { - DataFusionError::Plan(utils::generate_signature_error_msg( - &format!("{self}"), - self.signature(), - input_expr_types, - )) + plan_datafusion_err!( + "{}", + utils::generate_signature_error_msg( + &format!("{self}"), + self.signature(), + input_expr_types, + ) + ) })?; match self { diff --git a/datafusion/expr/src/built_in_function.rs b/datafusion/expr/src/built_in_function.rs index 991ad0322087..d7cfd9f4209e 100644 --- a/datafusion/expr/src/built_in_function.rs +++ b/datafusion/expr/src/built_in_function.rs @@ -25,7 +25,9 @@ use crate::{ TypeSignature, Volatility, }; use arrow::datatypes::{DataType, Field, Fields, IntervalUnit, TimeUnit}; -use datafusion_common::{internal_err, plan_err, DataFusionError, Result}; +use datafusion_common::{ + internal_err, plan_datafusion_err, plan_err, DataFusionError, Result, +}; use std::collections::HashMap; use std::fmt; use std::str::FromStr; @@ -501,11 +503,14 @@ impl BuiltinScalarFunction { // verify that this is a valid set of data types for this function data_types(input_expr_types, &self.signature()).map_err(|_| { - DataFusionError::Plan(utils::generate_signature_error_msg( - &format!("{self}"), - self.signature(), - input_expr_types, - )) + plan_datafusion_err!( + "{}", + utils::generate_signature_error_msg( + &format!("{self}"), + self.signature(), + input_expr_types, + ) + ) })?; // the return type of the built in function. diff --git a/datafusion/expr/src/expr_schema.rs b/datafusion/expr/src/expr_schema.rs index 9651b377c5bd..025b74eb5009 100644 --- a/datafusion/expr/src/expr_schema.rs +++ b/datafusion/expr/src/expr_schema.rs @@ -27,8 +27,8 @@ use crate::{LogicalPlan, Projection, Subquery}; use arrow::compute::can_cast_types; use arrow::datatypes::{DataType, Field}; use datafusion_common::{ - internal_err, plan_err, Column, DFField, DFSchema, DataFusionError, ExprSchema, - Result, + internal_err, plan_datafusion_err, plan_err, Column, DFField, DFSchema, + DataFusionError, ExprSchema, Result, }; use std::collections::HashMap; use std::sync::Arc; @@ -141,9 +141,7 @@ impl ExprSchemable for Expr { Expr::Like { .. } | Expr::SimilarTo { .. } => Ok(DataType::Boolean), Expr::Placeholder(Placeholder { data_type, .. }) => { data_type.clone().ok_or_else(|| { - DataFusionError::Plan( - "Placeholder type could not be resolved".to_owned(), - ) + plan_datafusion_err!("Placeholder type could not be resolved") }) } Expr::Wildcard => { diff --git a/datafusion/expr/src/field_util.rs b/datafusion/expr/src/field_util.rs index 23260ea9c270..96fa901c2cb3 100644 --- a/datafusion/expr/src/field_util.rs +++ b/datafusion/expr/src/field_util.rs @@ -18,7 +18,9 @@ //! Utility functions for complex field access use arrow::datatypes::{DataType, Field}; -use datafusion_common::{plan_err, DataFusionError, Result, ScalarValue}; +use datafusion_common::{ + plan_datafusion_err, plan_err, DataFusionError, Result, ScalarValue, +}; /// Types of the field access expression of a nested type, such as `Field` or `List` pub enum GetFieldAccessSchema { @@ -52,7 +54,7 @@ impl GetFieldAccessSchema { ) } else { let field = fields.iter().find(|f| f.name() == s); - field.ok_or(DataFusionError::Plan(format!("Field {s} not found in struct"))).map(|f| f.as_ref().clone()) + field.ok_or(plan_datafusion_err!("Field {s} not found in struct")).map(|f| f.as_ref().clone()) } } (DataType::Struct(_), _) => plan_err!( diff --git a/datafusion/expr/src/logical_plan/builder.rs b/datafusion/expr/src/logical_plan/builder.rs index c3f576195e2a..770f39be4710 100644 --- a/datafusion/expr/src/logical_plan/builder.rs +++ b/datafusion/expr/src/logical_plan/builder.rs @@ -43,13 +43,13 @@ use crate::{ Expr, ExprSchemable, TableSource, }; use arrow::datatypes::{DataType, Schema, SchemaRef}; -use datafusion_common::plan_err; use datafusion_common::UnnestOptions; use datafusion_common::{ display::ToStringifiedPlan, Column, DFField, DFSchema, DFSchemaRef, DataFusionError, FileType, FunctionalDependencies, OwnedTableReference, Result, ScalarValue, TableReference, ToDFSchema, }; +use datafusion_common::{plan_datafusion_err, plan_err}; use std::any::Any; use std::cmp::Ordering; use std::collections::{HashMap, HashSet}; @@ -1075,9 +1075,9 @@ impl LogicalPlanBuilder { self.plan.schema().clone(), right.schema().clone(), )?.ok_or_else(|| - DataFusionError::Plan(format!( + plan_datafusion_err!( "can't create join plan, join key should belong to one input, error key: ({normalized_left_key},{normalized_right_key})" - ))) + )) }) .collect::>>()?; @@ -1255,13 +1255,13 @@ pub fn union(left_plan: LogicalPlan, right_plan: LogicalPlan) -> Result Result Operator::IsDistinctFrom | Operator::IsNotDistinctFrom => { comparison_coercion(lhs, rhs).map(Signature::comparison).ok_or_else(|| { - DataFusionError::Plan(format!( + plan_datafusion_err!( "Cannot infer common argument type for comparison operation {lhs} {op} {rhs}" - )) + ) }) } Operator::And | Operator::Or => match (lhs, rhs) { @@ -91,9 +91,9 @@ fn signature(lhs: &DataType, op: &Operator, rhs: &DataType) -> Result Operator::RegexNotMatch | Operator::RegexNotIMatch => { regex_coercion(lhs, rhs).map(Signature::comparison).ok_or_else(|| { - DataFusionError::Plan(format!( + plan_datafusion_err!( "Cannot infer common argument type for regex operation {lhs} {op} {rhs}" - )) + ) }) } Operator::BitwiseAnd @@ -102,24 +102,24 @@ fn signature(lhs: &DataType, op: &Operator, rhs: &DataType) -> Result | Operator::BitwiseShiftRight | Operator::BitwiseShiftLeft => { bitwise_coercion(lhs, rhs).map(Signature::uniform).ok_or_else(|| { - DataFusionError::Plan(format!( + plan_datafusion_err!( "Cannot infer common type for bitwise operation {lhs} {op} {rhs}" - )) + ) }) } Operator::StringConcat => { string_concat_coercion(lhs, rhs).map(Signature::uniform).ok_or_else(|| { - DataFusionError::Plan(format!( + plan_datafusion_err!( "Cannot infer common string type for string concat operation {lhs} {op} {rhs}" - )) + ) }) } Operator::AtArrow | Operator::ArrowAt => { array_coercion(lhs, rhs).map(Signature::uniform).ok_or_else(|| { - DataFusionError::Plan(format!( + plan_datafusion_err!( "Cannot infer common array type for arrow operation {lhs} {op} {rhs}" - )) + ) }) } Operator::Plus | @@ -154,9 +154,9 @@ fn signature(lhs: &DataType, op: &Operator, rhs: &DataType) -> Result // Temporal arithmetic by first coercing to a common time representation // e.g. Date32 - Timestamp let ret = get_result(&coerced, &coerced).map_err(|e| { - DataFusionError::Plan(format!( + plan_datafusion_err!( "Cannot get result type for temporal operation {coerced} {op} {coerced}: {e}" - )) + ) })?; Ok(Signature{ lhs: coerced.clone(), @@ -166,9 +166,9 @@ fn signature(lhs: &DataType, op: &Operator, rhs: &DataType) -> Result } else if let Some((lhs, rhs)) = math_decimal_coercion(lhs, rhs) { // Decimal arithmetic, e.g. Decimal(10, 2) + Decimal(10, 0) let ret = get_result(&lhs, &rhs).map_err(|e| { - DataFusionError::Plan(format!( + plan_datafusion_err!( "Cannot get result type for decimal operation {lhs} {op} {rhs}: {e}" - )) + ) })?; Ok(Signature{ lhs, diff --git a/datafusion/expr/src/utils.rs b/datafusion/expr/src/utils.rs index 54a1ce348bf9..5fc5b5b3f9c7 100644 --- a/datafusion/expr/src/utils.rs +++ b/datafusion/expr/src/utils.rs @@ -24,8 +24,8 @@ use crate::{Cast, Expr, ExprSchemable, GroupingSet, LogicalPlan, TryCast}; use arrow::datatypes::{DataType, TimeUnit}; use datafusion_common::tree_node::{TreeNode, VisitRecursion}; use datafusion_common::{ - internal_err, plan_err, Column, DFField, DFSchema, DFSchemaRef, DataFusionError, - Result, ScalarValue, TableReference, + internal_err, plan_datafusion_err, plan_err, Column, DFField, DFSchema, DFSchemaRef, + DataFusionError, Result, ScalarValue, TableReference, }; use sqlparser::ast::{ExceptSelectItem, ExcludeSelectItem, WildcardAdditionalOptions}; use std::cmp::Ordering; @@ -198,8 +198,8 @@ pub fn enumerate_grouping_sets(group_expr: Vec) -> Result> { grouping_sets.iter().map(|e| e.iter().collect()).collect() } Expr::GroupingSet(GroupingSet::Cube(group_exprs)) => { - let grouping_sets = - powerset(group_exprs).map_err(DataFusionError::Plan)?; + let grouping_sets = powerset(group_exprs) + .map_err(|e| plan_datafusion_err!("{}", e))?; check_grouping_sets_size_limit(grouping_sets.len())?; grouping_sets } diff --git a/datafusion/expr/src/window_function.rs b/datafusion/expr/src/window_function.rs index 1f36ebdd6b54..e5b00c8f298b 100644 --- a/datafusion/expr/src/window_function.rs +++ b/datafusion/expr/src/window_function.rs @@ -25,7 +25,7 @@ use crate::type_coercion::functions::data_types; use crate::utils; use crate::{AggregateUDF, Signature, TypeSignature, Volatility, WindowUDF}; use arrow::datatypes::DataType; -use datafusion_common::{plan_err, DataFusionError, Result}; +use datafusion_common::{plan_datafusion_err, plan_err, DataFusionError, Result}; use std::sync::Arc; use std::{fmt, str::FromStr}; use strum_macros::EnumIter; @@ -192,11 +192,14 @@ impl BuiltInWindowFunction { // original errors are all related to wrong function signature // aggregate them for better error message .map_err(|_| { - DataFusionError::Plan(utils::generate_signature_error_msg( - &format!("{self}"), - self.signature(), - input_expr_types, - )) + plan_datafusion_err!( + "{}", + utils::generate_signature_error_msg( + &format!("{self}"), + self.signature(), + input_expr_types, + ) + ) })?; match self { diff --git a/datafusion/optimizer/src/analyzer/type_coercion.rs b/datafusion/optimizer/src/analyzer/type_coercion.rs index 3b866173859c..bfdbec390199 100644 --- a/datafusion/optimizer/src/analyzer/type_coercion.rs +++ b/datafusion/optimizer/src/analyzer/type_coercion.rs @@ -24,8 +24,8 @@ use arrow::datatypes::{DataType, IntervalUnit}; use datafusion_common::config::ConfigOptions; use datafusion_common::tree_node::{RewriteRecursion, TreeNodeRewriter}; use datafusion_common::{ - exec_err, internal_err, plan_err, DFSchema, DFSchemaRef, DataFusionError, Result, - ScalarValue, + exec_err, internal_err, plan_datafusion_err, plan_err, DFSchema, DFSchemaRef, + DataFusionError, Result, ScalarValue, }; use datafusion_expr::expr::{ self, Between, BinaryExpr, Case, Exists, InList, InSubquery, Like, ScalarFunction, @@ -162,11 +162,10 @@ impl TreeNodeRewriter for TypeCoercionRewriter { let new_plan = analyze_internal(&self.schema, &subquery.subquery)?; let expr_type = expr.get_type(&self.schema)?; let subquery_type = new_plan.schema().field(0).data_type(); - let common_type = comparison_coercion(&expr_type, subquery_type).ok_or(DataFusionError::Plan( - format!( + let common_type = comparison_coercion(&expr_type, subquery_type).ok_or(plan_datafusion_err!( "expr type {expr_type:?} can't cast to {subquery_type:?} in InSubquery" ), - ))?; + )?; let new_subquery = Subquery { subquery: Arc::new(new_plan), outer_ref_columns: subquery.outer_ref_columns, @@ -218,9 +217,9 @@ impl TreeNodeRewriter for TypeCoercionRewriter { } else { "LIKE" }; - DataFusionError::Plan(format!( + plan_datafusion_err!( "There isn't a common type to coerce {left_type} and {right_type} in {op_name} expression" - )) + ) })?; let expr = Box::new(expr.cast_to(&coerced_type, &self.schema)?); let pattern = Box::new(pattern.cast_to(&coerced_type, &self.schema)?); @@ -709,20 +708,20 @@ fn coerce_case_expression(case: Case, schema: &DFSchemaRef) -> Result { let coerced_type = get_coerce_type_for_case_expression(&when_types, Some(case_type)); coerced_type.ok_or_else(|| { - DataFusionError::Plan(format!( + plan_datafusion_err!( "Failed to coerce case ({case_type:?}) and when ({when_types:?}) \ to common types in CASE WHEN expression" - )) + ) }) }) .transpose()?; let then_else_coerce_type = get_coerce_type_for_case_expression(&then_types, else_type.as_ref()).ok_or_else( || { - DataFusionError::Plan(format!( + plan_datafusion_err!( "Failed to coerce then ({then_types:?}) and else ({else_type:?}) \ to common types in CASE WHEN expression" - )) + ) }, )?; @@ -991,10 +990,13 @@ mod test { None, None, )); - let err = Projection::try_new(vec![agg_expr], empty).err().unwrap(); + let err = Projection::try_new(vec![agg_expr], empty) + .err() + .unwrap() + .strip_backtrace(); assert_eq!( - "Plan(\"No function matches the given name and argument types 'AVG(Utf8)'. You might need to add explicit type casts.\\n\\tCandidate functions:\\n\\tAVG(Int8/Int16/Int32/Int64/UInt8/UInt16/UInt32/UInt64/Float32/Float64)\")", - &format!("{err:?}") + "Error during planning: No function matches the given name and argument types 'AVG(Utf8)'. You might need to add explicit type casts.\n\tCandidate functions:\n\tAVG(Int8/Int16/Int32/Int64/UInt8/UInt16/UInt32/UInt64/Float32/Float64)", + err ); Ok(()) } diff --git a/datafusion/optimizer/src/push_down_filter.rs b/datafusion/optimizer/src/push_down_filter.rs index ee7b37979dc4..4c5cd3ab2855 100644 --- a/datafusion/optimizer/src/push_down_filter.rs +++ b/datafusion/optimizer/src/push_down_filter.rs @@ -18,7 +18,9 @@ use crate::optimizer::ApplyOrder; use crate::utils::{conjunction, split_conjunction}; use crate::{utils, OptimizerConfig, OptimizerRule}; use datafusion_common::tree_node::{Transformed, TreeNode, VisitRecursion}; -use datafusion_common::{internal_err, Column, DFSchema, DataFusionError, Result}; +use datafusion_common::{ + internal_err, plan_datafusion_err, Column, DFSchema, DataFusionError, Result, +}; use datafusion_expr::expr::Alias; use datafusion_expr::{ and, @@ -609,7 +611,7 @@ impl OptimizerRule for PushDownFilter { .map(|e| (*e).clone()) .collect::>(); let new_predicate = conjunction(new_predicates).ok_or_else(|| { - DataFusionError::Plan("at least one expression exists".to_string()) + plan_datafusion_err!("at least one expression exists") })?; let new_filter = LogicalPlan::Filter(Filter::try_new( new_predicate, diff --git a/datafusion/physical-plan/src/joins/utils.rs b/datafusion/physical-plan/src/joins/utils.rs index daaa16e0552d..1dbcf7617385 100644 --- a/datafusion/physical-plan/src/joins/utils.rs +++ b/datafusion/physical-plan/src/joins/utils.rs @@ -41,7 +41,8 @@ use arrow::record_batch::{RecordBatch, RecordBatchOptions}; use datafusion_common::cast::as_boolean_array; use datafusion_common::tree_node::{Transformed, TreeNode}; use datafusion_common::{ - exec_err, plan_err, DataFusionError, JoinType, Result, ScalarValue, SharedResult, + exec_err, plan_datafusion_err, plan_err, DataFusionError, JoinType, Result, + ScalarValue, SharedResult, }; use datafusion_physical_expr::expressions::Column; use datafusion_physical_expr::{ @@ -363,11 +364,7 @@ pub fn combine_join_ordering_equivalence_properties( let left_maintains = maintains_input_order[0]; let right_maintains = maintains_input_order[1]; match (left_maintains, right_maintains) { - (true, true) => { - return Err(DataFusionError::Plan( - "Cannot maintain ordering of both sides".to_string(), - )) - } + (true, true) => return plan_err!("Cannot maintain ordering of both sides"), (true, false) => { new_properties.extend(left_oeq_properties.oeq_class().cloned()); // In this special case, right side ordering can be prefixed with left side ordering. @@ -1345,8 +1342,7 @@ pub fn prepare_sorted_exprs( right_sort_exprs: &[PhysicalSortExpr], ) -> Result<(SortedFilterExpr, SortedFilterExpr, ExprIntervalGraph)> { // Build the filter order for the left side - let err = - || DataFusionError::Plan("Filter does not include the child order".to_owned()); + let err = || plan_datafusion_err!("Filter does not include the child order"); let left_temp_sorted_filter_expr = build_filter_input_order( JoinSide::Left, diff --git a/datafusion/proto/src/bytes/mod.rs b/datafusion/proto/src/bytes/mod.rs index d3ac33bab535..9377501499e2 100644 --- a/datafusion/proto/src/bytes/mod.rs +++ b/datafusion/proto/src/bytes/mod.rs @@ -24,7 +24,7 @@ use crate::physical_plan::{ }; use crate::protobuf; use datafusion::physical_plan::functions::make_scalar_function; -use datafusion_common::{DataFusionError, Result}; +use datafusion_common::{plan_datafusion_err, DataFusionError, Result}; use datafusion_expr::{ create_udaf, create_udf, create_udwf, AggregateUDF, Expr, LogicalPlan, Volatility, WindowUDF, @@ -88,13 +88,13 @@ pub trait Serializeable: Sized { impl Serializeable for Expr { fn to_bytes(&self) -> Result { let mut buffer = BytesMut::new(); - let protobuf: protobuf::LogicalExprNode = self.try_into().map_err(|e| { - DataFusionError::Plan(format!("Error encoding expr as protobuf: {e}")) - })?; + let protobuf: protobuf::LogicalExprNode = self + .try_into() + .map_err(|e| plan_datafusion_err!("Error encoding expr as protobuf: {e}"))?; - protobuf.encode(&mut buffer).map_err(|e| { - DataFusionError::Plan(format!("Error encoding protobuf as bytes: {e}")) - })?; + protobuf + .encode(&mut buffer) + .map_err(|e| plan_datafusion_err!("Error encoding protobuf as bytes: {e}"))?; let bytes: Bytes = buffer.into(); @@ -151,13 +151,11 @@ impl Serializeable for Expr { bytes: &[u8], registry: &dyn FunctionRegistry, ) -> Result { - let protobuf = protobuf::LogicalExprNode::decode(bytes).map_err(|e| { - DataFusionError::Plan(format!("Error decoding expr as protobuf: {e}")) - })?; + let protobuf = protobuf::LogicalExprNode::decode(bytes) + .map_err(|e| plan_datafusion_err!("Error decoding expr as protobuf: {e}"))?; - logical_plan::from_proto::parse_expr(&protobuf, registry).map_err(|e| { - DataFusionError::Plan(format!("Error parsing protobuf into Expr: {e}")) - }) + logical_plan::from_proto::parse_expr(&protobuf, registry) + .map_err(|e| plan_datafusion_err!("Error parsing protobuf into Expr: {e}")) } } @@ -173,9 +171,9 @@ pub fn logical_plan_to_json(plan: &LogicalPlan) -> Result { let extension_codec = DefaultLogicalExtensionCodec {}; let protobuf = protobuf::LogicalPlanNode::try_from_logical_plan(plan, &extension_codec) - .map_err(|e| DataFusionError::Plan(format!("Error serializing plan: {e}")))?; + .map_err(|e| plan_datafusion_err!("Error serializing plan: {e}"))?; serde_json::to_string(&protobuf) - .map_err(|e| DataFusionError::Plan(format!("Error serializing plan: {e}"))) + .map_err(|e| plan_datafusion_err!("Error serializing plan: {e}")) } /// Serialize a LogicalPlan as bytes, using the provided extension codec @@ -186,9 +184,9 @@ pub fn logical_plan_to_bytes_with_extension_codec( let protobuf = protobuf::LogicalPlanNode::try_from_logical_plan(plan, extension_codec)?; let mut buffer = BytesMut::new(); - protobuf.encode(&mut buffer).map_err(|e| { - DataFusionError::Plan(format!("Error encoding protobuf as bytes: {e}")) - })?; + protobuf + .encode(&mut buffer) + .map_err(|e| plan_datafusion_err!("Error encoding protobuf as bytes: {e}"))?; Ok(buffer.into()) } @@ -196,7 +194,7 @@ pub fn logical_plan_to_bytes_with_extension_codec( #[cfg(feature = "json")] pub fn logical_plan_from_json(json: &str, ctx: &SessionContext) -> Result { let back: protobuf::LogicalPlanNode = serde_json::from_str(json) - .map_err(|e| DataFusionError::Plan(format!("Error serializing plan: {e}")))?; + .map_err(|e| plan_datafusion_err!("Error serializing plan: {e}"))?; let extension_codec = DefaultLogicalExtensionCodec {}; back.try_into_logical_plan(ctx, &extension_codec) } @@ -216,9 +214,8 @@ pub fn logical_plan_from_bytes_with_extension_codec( ctx: &SessionContext, extension_codec: &dyn LogicalExtensionCodec, ) -> Result { - let protobuf = protobuf::LogicalPlanNode::decode(bytes).map_err(|e| { - DataFusionError::Plan(format!("Error decoding expr as protobuf: {e}")) - })?; + let protobuf = protobuf::LogicalPlanNode::decode(bytes) + .map_err(|e| plan_datafusion_err!("Error decoding expr as protobuf: {e}"))?; protobuf.try_into_logical_plan(ctx, extension_codec) } @@ -234,9 +231,9 @@ pub fn physical_plan_to_json(plan: Arc) -> Result { let extension_codec = DefaultPhysicalExtensionCodec {}; let protobuf = protobuf::PhysicalPlanNode::try_from_physical_plan(plan, &extension_codec) - .map_err(|e| DataFusionError::Plan(format!("Error serializing plan: {e}")))?; + .map_err(|e| plan_datafusion_err!("Error serializing plan: {e}"))?; serde_json::to_string(&protobuf) - .map_err(|e| DataFusionError::Plan(format!("Error serializing plan: {e}"))) + .map_err(|e| plan_datafusion_err!("Error serializing plan: {e}")) } /// Serialize a PhysicalPlan as bytes, using the provided extension codec @@ -247,9 +244,9 @@ pub fn physical_plan_to_bytes_with_extension_codec( let protobuf = protobuf::PhysicalPlanNode::try_from_physical_plan(plan, extension_codec)?; let mut buffer = BytesMut::new(); - protobuf.encode(&mut buffer).map_err(|e| { - DataFusionError::Plan(format!("Error encoding protobuf as bytes: {e}")) - })?; + protobuf + .encode(&mut buffer) + .map_err(|e| plan_datafusion_err!("Error encoding protobuf as bytes: {e}"))?; Ok(buffer.into()) } @@ -260,7 +257,7 @@ pub fn physical_plan_from_json( ctx: &SessionContext, ) -> Result> { let back: protobuf::PhysicalPlanNode = serde_json::from_str(json) - .map_err(|e| DataFusionError::Plan(format!("Error serializing plan: {e}")))?; + .map_err(|e| plan_datafusion_err!("Error serializing plan: {e}"))?; let extension_codec = DefaultPhysicalExtensionCodec {}; back.try_into_physical_plan(ctx, &ctx.runtime_env(), &extension_codec) } @@ -280,8 +277,7 @@ pub fn physical_plan_from_bytes_with_extension_codec( ctx: &SessionContext, extension_codec: &dyn PhysicalExtensionCodec, ) -> Result> { - let protobuf = protobuf::PhysicalPlanNode::decode(bytes).map_err(|e| { - DataFusionError::Plan(format!("Error decoding expr as protobuf: {e}")) - })?; + let protobuf = protobuf::PhysicalPlanNode::decode(bytes) + .map_err(|e| plan_datafusion_err!("Error decoding expr as protobuf: {e}"))?; protobuf.try_into_physical_plan(ctx, &ctx.runtime_env(), extension_codec) } diff --git a/datafusion/proto/src/logical_plan/from_proto.rs b/datafusion/proto/src/logical_plan/from_proto.rs index aa56c7b19d0c..2203016e08f1 100644 --- a/datafusion/proto/src/logical_plan/from_proto.rs +++ b/datafusion/proto/src/logical_plan/from_proto.rs @@ -31,8 +31,8 @@ use arrow::datatypes::{ }; use datafusion::execution::registry::FunctionRegistry; use datafusion_common::{ - internal_err, Column, Constraint, Constraints, DFField, DFSchema, DFSchemaRef, - DataFusionError, OwnedTableReference, Result, ScalarValue, + internal_err, plan_datafusion_err, Column, Constraint, Constraints, DFField, + DFSchema, DFSchemaRef, DataFusionError, OwnedTableReference, Result, ScalarValue, }; use datafusion_expr::{ abs, acos, acosh, array, array_append, array_concat, array_dims, array_element, @@ -1740,9 +1740,7 @@ fn parse_vec_expr( ) -> Result>, Error> { let res = p .iter() - .map(|elem| { - parse_expr(elem, registry).map_err(|e| DataFusionError::Plan(e.to_string())) - }) + .map(|elem| parse_expr(elem, registry).map_err(|e| plan_datafusion_err!("{}", e))) .collect::>>()?; // Convert empty vector to None. Ok((!res.is_empty()).then_some(res)) diff --git a/datafusion/proto/src/logical_plan/mod.rs b/datafusion/proto/src/logical_plan/mod.rs index 76d44186b5b5..f0999871f568 100644 --- a/datafusion/proto/src/logical_plan/mod.rs +++ b/datafusion/proto/src/logical_plan/mod.rs @@ -43,6 +43,7 @@ use datafusion::{ datasource::{provider_as_source, source_as_provider}, prelude::SessionContext, }; +use datafusion_common::plan_datafusion_err; use datafusion_common::{ context, internal_err, not_impl_err, parsers::CompressionTypeVariant, DataFusionError, OwnedTableReference, Result, @@ -65,13 +66,13 @@ pub mod to_proto; impl From for DataFusionError { fn from(e: from_proto::Error) -> Self { - DataFusionError::Plan(e.to_string()) + plan_datafusion_err!("{}", e) } } impl From for DataFusionError { fn from(e: to_proto::Error) -> Self { - DataFusionError::Plan(e.to_string()) + plan_datafusion_err!("{}", e) } } diff --git a/datafusion/sql/src/expr/arrow_cast.rs b/datafusion/sql/src/expr/arrow_cast.rs index 549d46c5e277..8c0184b6d119 100644 --- a/datafusion/sql/src/expr/arrow_cast.rs +++ b/datafusion/sql/src/expr/arrow_cast.rs @@ -21,7 +21,9 @@ use std::{fmt::Display, iter::Peekable, str::Chars, sync::Arc}; use arrow_schema::{DataType, Field, IntervalUnit, TimeUnit}; -use datafusion_common::{DFSchema, DataFusionError, Result, ScalarValue}; +use datafusion_common::{ + plan_datafusion_err, DFSchema, DataFusionError, Result, ScalarValue, +}; use datafusion_common::plan_err; use datafusion_expr::{Expr, ExprSchemable}; @@ -98,9 +100,7 @@ pub fn parse_data_type(val: &str) -> Result { } fn make_error(val: &str, msg: &str) -> DataFusionError { - DataFusionError::Plan( - format!("Unsupported type '{val}'. Must be a supported arrow type name such as 'Int32' or 'Timestamp(Nanosecond, None)'. Error {msg}" ) - ) + plan_datafusion_err!("Unsupported type '{val}'. Must be a supported arrow type name such as 'Int32' or 'Timestamp(Nanosecond, None)'. Error {msg}" ) } fn make_error_expected(val: &str, expected: &Token, actual: &Token) -> DataFusionError { diff --git a/datafusion/sql/src/expr/function.rs b/datafusion/sql/src/expr/function.rs index 3861b4848d9b..ab70853949fd 100644 --- a/datafusion/sql/src/expr/function.rs +++ b/datafusion/sql/src/expr/function.rs @@ -16,7 +16,9 @@ // under the License. use crate::planner::{ContextProvider, PlannerContext, SqlToRel}; -use datafusion_common::{not_impl_err, plan_err, DFSchema, DataFusionError, Result}; +use datafusion_common::{ + not_impl_err, plan_datafusion_err, plan_err, DFSchema, DataFusionError, Result, +}; use datafusion_expr::expr::{ScalarFunction, ScalarUDF}; use datafusion_expr::function::suggest_valid_function; use datafusion_expr::window_frame::regularize; @@ -189,7 +191,7 @@ impl<'a, S: ContextProvider> SqlToRel<'a, S> { .map(WindowFunction::WindowUDF) }) .ok_or_else(|| { - DataFusionError::Plan(format!("There is no window function named {name}")) + plan_datafusion_err!("There is no window function named {name}") }) } diff --git a/datafusion/sql/src/expr/identifier.rs b/datafusion/sql/src/expr/identifier.rs index 5e03f14e5337..4033c28148f8 100644 --- a/datafusion/sql/src/expr/identifier.rs +++ b/datafusion/sql/src/expr/identifier.rs @@ -17,7 +17,8 @@ use crate::planner::{ContextProvider, PlannerContext, SqlToRel}; use datafusion_common::{ - internal_err, Column, DFField, DFSchema, DataFusionError, Result, TableReference, + internal_err, plan_datafusion_err, Column, DFField, DFSchema, DataFusionError, + Result, TableReference, }; use datafusion_expr::{Case, Expr}; use sqlparser::ast::{Expr as SQLExpr, Ident}; @@ -36,9 +37,7 @@ impl<'a, S: ContextProvider> SqlToRel<'a, S> { .schema_provider .get_variable_type(&var_names) .ok_or_else(|| { - DataFusionError::Plan(format!( - "variable {var_names:?} has no type information" - )) + plan_datafusion_err!("variable {var_names:?} has no type information") })?; Ok(Expr::ScalarVariable(ty, var_names)) } else { diff --git a/datafusion/sql/src/expr/order_by.rs b/datafusion/sql/src/expr/order_by.rs index b32388f1bcdf..1dccc2376f0b 100644 --- a/datafusion/sql/src/expr/order_by.rs +++ b/datafusion/sql/src/expr/order_by.rs @@ -16,7 +16,9 @@ // under the License. use crate::planner::{ContextProvider, PlannerContext, SqlToRel}; -use datafusion_common::{DFSchema, DataFusionError, Result}; +use datafusion_common::{ + plan_datafusion_err, plan_err, DFSchema, DataFusionError, Result, +}; use datafusion_expr::expr::Sort; use datafusion_expr::Expr; use sqlparser::ast::{Expr as SQLExpr, OrderByExpr, Value}; @@ -41,18 +43,18 @@ impl<'a, S: ContextProvider> SqlToRel<'a, S> { SQLExpr::Value(Value::Number(v, _)) => { let field_index = v .parse::() - .map_err(|err| DataFusionError::Plan(err.to_string()))?; + .map_err(|err| plan_datafusion_err!("{}", err))?; if field_index == 0 { - return Err(DataFusionError::Plan( - "Order by index starts at 1 for column indexes".to_string(), - )); + return plan_err!( + "Order by index starts at 1 for column indexes" + ); } else if schema.fields().len() < field_index { - return Err(DataFusionError::Plan(format!( + return plan_err!( "Order by column out of bounds, specified: {}, max: {}", field_index, schema.fields().len() - ))); + ); } let field = schema.field(field_index - 1); diff --git a/datafusion/sql/src/statement.rs b/datafusion/sql/src/statement.rs index 059a32be441b..5ae1e2001d83 100644 --- a/datafusion/sql/src/statement.rs +++ b/datafusion/sql/src/statement.rs @@ -31,9 +31,9 @@ use arrow_schema::DataType; use datafusion_common::file_options::StatementOptions; use datafusion_common::parsers::CompressionTypeVariant; use datafusion_common::{ - not_impl_err, plan_err, unqualified_field_not_found, Column, Constraints, DFField, - DFSchema, DFSchemaRef, DataFusionError, ExprSchema, OwnedTableReference, Result, - SchemaReference, TableReference, ToDFSchema, + not_impl_err, plan_datafusion_err, plan_err, unqualified_field_not_found, Column, + Constraints, DFField, DFSchema, DFSchemaRef, DataFusionError, ExprSchema, + OwnedTableReference, Result, SchemaReference, TableReference, ToDFSchema, }; use datafusion_expr::dml::{CopyOptions, CopyTo}; use datafusion_expr::expr::Placeholder; @@ -980,9 +980,11 @@ impl<'a, S: ContextProvider> SqlToRel<'a, S> { let mut assign_map = assignments .iter() .map(|assign| { - let col_name: &Ident = assign.id.iter().last().ok_or_else(|| { - DataFusionError::Plan("Empty column id".to_string()) - })?; + let col_name: &Ident = assign + .id + .iter() + .last() + .ok_or_else(|| plan_datafusion_err!("Empty column id"))?; // Validate that the assignment target column exists table_schema.field_with_unqualified_name(&col_name.value)?; Ok((col_name.value.clone(), assign.value.clone())) @@ -1113,15 +1115,13 @@ impl<'a, S: ContextProvider> SqlToRel<'a, S> { if let ast::Expr::Value(Value::Placeholder(name)) = val { let name = name.replace('$', "").parse::().map_err(|_| { - DataFusionError::Plan(format!( - "Can't parse placeholder: {name}" - )) + plan_datafusion_err!("Can't parse placeholder: {name}") })? - 1; let field = fields.get(idx).ok_or_else(|| { - DataFusionError::Plan(format!( + plan_datafusion_err!( "Placeholder ${} refers to a non existent column", idx + 1 - )) + ) })?; let dt = field.field().data_type().clone(); let _ = prepare_param_data_types.insert(name, dt); diff --git a/datafusion/sql/tests/sql_integration.rs b/datafusion/sql/tests/sql_integration.rs index 661890e12533..d95598cc3d8e 100644 --- a/datafusion/sql/tests/sql_integration.rs +++ b/datafusion/sql/tests/sql_integration.rs @@ -1687,20 +1687,24 @@ fn select_order_by_multiple_index() { #[test] fn select_order_by_index_of_0() { let sql = "SELECT id FROM person ORDER BY 0"; - let err = logical_plan(sql).expect_err("query should have failed"); + let err = logical_plan(sql) + .expect_err("query should have failed") + .strip_backtrace(); assert_eq!( - "Plan(\"Order by index starts at 1 for column indexes\")", - format!("{err:?}") + "Error during planning: Order by index starts at 1 for column indexes", + err ); } #[test] fn select_order_by_index_oob() { let sql = "SELECT id FROM person ORDER BY 2"; - let err = logical_plan(sql).expect_err("query should have failed"); + let err = logical_plan(sql) + .expect_err("query should have failed") + .strip_backtrace(); assert_eq!( - "Plan(\"Order by column out of bounds, specified: 2, max: 1\")", - format!("{err:?}") + "Error during planning: Order by column out of bounds, specified: 2, max: 1", + err ); } @@ -2086,13 +2090,12 @@ fn union_values_with_no_alias() { #[test] fn union_with_incompatible_data_type() { let sql = "SELECT interval '1 year 1 day' UNION ALL SELECT 1"; - let err = logical_plan(sql).expect_err("query should have failed"); + let err = logical_plan(sql) + .expect_err("query should have failed") + .strip_backtrace(); assert_eq!( - "Plan(\"UNION Column Int64(1) (type: Int64) is \ - not compatible with column IntervalMonthDayNano\ - (\\\"950737950189618795196236955648\\\") \ - (type: Interval(MonthDayNano))\")", - format!("{err:?}") + "Error during planning: UNION Column Int64(1) (type: Int64) is not compatible with column IntervalMonthDayNano(\"950737950189618795196236955648\") (type: Interval(MonthDayNano))", + err ); } @@ -2195,10 +2198,12 @@ fn union_with_aliases() { #[test] fn union_with_incompatible_data_types() { let sql = "SELECT 'a' a UNION ALL SELECT true a"; - let err = logical_plan(sql).expect_err("query should have failed"); + let err = logical_plan(sql) + .expect_err("query should have failed") + .strip_backtrace(); assert_eq!( - "Plan(\"UNION Column a (type: Boolean) is not compatible with column a (type: Utf8)\")", - format!("{err:?}") + "Error during planning: UNION Column a (type: Boolean) is not compatible with column a (type: Utf8)", + err ); } From d33595ab55de6e4317c52bceedc0231b25af5fef Mon Sep 17 00:00:00 2001 From: Andrew Lamb Date: Mon, 16 Oct 2023 10:19:01 -0400 Subject: [PATCH 077/572] Add documentation and usability for prepared parameters (#7785) * Add documentation for prepared parameters + make it eaiser to use * Update datafusion/expr/src/expr.rs Co-authored-by: jakevin --------- Co-authored-by: jakevin --- datafusion/core/src/dataframe.rs | 37 ++++++++++++- datafusion/expr/src/expr.rs | 58 ++++++++++++++++++-- datafusion/expr/src/expr_fn.rs | 20 ++++++- datafusion/expr/src/logical_plan/plan.rs | 69 +++++++++++++++++------- datafusion/sql/src/expr/mod.rs | 48 +---------------- datafusion/sql/tests/sql_integration.rs | 13 +++++ 6 files changed, 175 insertions(+), 70 deletions(-) diff --git a/datafusion/core/src/dataframe.rs b/datafusion/core/src/dataframe.rs index d704c7f30443..79b8fcd519b5 100644 --- a/datafusion/core/src/dataframe.rs +++ b/datafusion/core/src/dataframe.rs @@ -1210,7 +1210,42 @@ impl DataFrame { Ok(DataFrame::new(self.session_state, project_plan)) } - /// Convert a prepare logical plan into its inner logical plan with all params replaced with their corresponding values + /// Replace all parameters in logical plan with the specified + /// values, in preparation for execution. + /// + /// # Example + /// + /// ``` + /// use datafusion::prelude::*; + /// # use datafusion::{error::Result, assert_batches_eq}; + /// # #[tokio::main] + /// # async fn main() -> Result<()> { + /// # use datafusion_common::ScalarValue; + /// let mut ctx = SessionContext::new(); + /// # ctx.register_csv("example", "tests/data/example.csv", CsvReadOptions::new()).await?; + /// let results = ctx + /// .sql("SELECT a FROM example WHERE b = $1") + /// .await? + /// // replace $1 with value 2 + /// .with_param_values(vec![ + /// // value at index 0 --> $1 + /// ScalarValue::from(2i64) + /// ])? + /// .collect() + /// .await?; + /// assert_batches_eq!( + /// &[ + /// "+---+", + /// "| a |", + /// "+---+", + /// "| 1 |", + /// "+---+", + /// ], + /// &results + /// ); + /// # Ok(()) + /// # } + /// ``` pub fn with_param_values(self, param_values: Vec) -> Result { let plan = self.plan.with_param_values(param_values)?; Ok(Self::new(self.session_state, plan)) diff --git a/datafusion/expr/src/expr.rs b/datafusion/expr/src/expr.rs index 3949d25b3025..0b166107fb4b 100644 --- a/datafusion/expr/src/expr.rs +++ b/datafusion/expr/src/expr.rs @@ -17,7 +17,6 @@ //! Expr module contains core type definition for `Expr`. -use crate::aggregate_function; use crate::built_in_function; use crate::expr_fn::binary_expr; use crate::logical_plan::Subquery; @@ -26,8 +25,10 @@ use crate::utils::{expr_to_columns, find_out_reference_exprs}; use crate::window_frame; use crate::window_function; use crate::Operator; +use crate::{aggregate_function, ExprSchemable}; use arrow::datatypes::DataType; -use datafusion_common::internal_err; +use datafusion_common::tree_node::{Transformed, TreeNode}; +use datafusion_common::{internal_err, DFSchema}; use datafusion_common::{plan_err, Column, DataFusionError, Result, ScalarValue}; use std::collections::HashSet; use std::fmt; @@ -605,10 +606,13 @@ impl InSubquery { } } -/// Placeholder +/// Placeholder, representing bind parameter values such as `$1`. +/// +/// The type of these parameters is inferred using [`Expr::infer_placeholder_types`] +/// or can be specified directly using `PREPARE` statements. #[derive(Clone, PartialEq, Eq, Hash, Debug)] pub struct Placeholder { - /// The identifier of the parameter (e.g, $1 or $foo) + /// The identifier of the parameter, including the leading `$` (e.g, `"$1"` or `"$foo"`) pub id: String, /// The type the parameter will be filled in with pub data_type: Option, @@ -1036,6 +1040,52 @@ impl Expr { pub fn contains_outer(&self) -> bool { !find_out_reference_exprs(self).is_empty() } + + /// Recursively find all [`Expr::Placeholder`] expressions, and + /// to infer their [`DataType`] from the context of their use. + /// + /// For example, gicen an expression like ` = $0` will infer `$0` to + /// have type `int32`. + pub fn infer_placeholder_types(self, schema: &DFSchema) -> Result { + self.transform(&|mut expr| { + // Default to assuming the arguments are the same type + if let Expr::BinaryExpr(BinaryExpr { left, op: _, right }) = &mut expr { + rewrite_placeholder(left.as_mut(), right.as_ref(), schema)?; + rewrite_placeholder(right.as_mut(), left.as_ref(), schema)?; + }; + if let Expr::Between(Between { + expr, + negated: _, + low, + high, + }) = &mut expr + { + rewrite_placeholder(low.as_mut(), expr.as_ref(), schema)?; + rewrite_placeholder(high.as_mut(), expr.as_ref(), schema)?; + } + Ok(Transformed::Yes(expr)) + }) + } +} + +// modifies expr if it is a placeholder with datatype of right +fn rewrite_placeholder(expr: &mut Expr, other: &Expr, schema: &DFSchema) -> Result<()> { + if let Expr::Placeholder(Placeholder { id: _, data_type }) = expr { + if data_type.is_none() { + let other_dt = other.get_type(schema); + match other_dt { + Err(e) => { + Err(e.context(format!( + "Can not find type of {other} needed to infer type of {expr}" + )))?; + } + Ok(dt) => { + *data_type = Some(dt); + } + } + }; + } + Ok(()) } #[macro_export] diff --git a/datafusion/expr/src/expr_fn.rs b/datafusion/expr/src/expr_fn.rs index 711dc123a4a4..79a43c2353db 100644 --- a/datafusion/expr/src/expr_fn.rs +++ b/datafusion/expr/src/expr_fn.rs @@ -19,7 +19,7 @@ use crate::expr::{ AggregateFunction, BinaryExpr, Cast, Exists, GroupingSet, InList, InSubquery, - ScalarFunction, TryCast, + Placeholder, ScalarFunction, TryCast, }; use crate::function::PartitionEvaluatorFactory; use crate::WindowUDF; @@ -80,6 +80,24 @@ pub fn ident(name: impl Into) -> Expr { Expr::Column(Column::from_name(name)) } +/// Create placeholder value that will be filled in (such as `$1`) +/// +/// Note the parameter type can be inferred using [`Expr::infer_placeholder_types`] +/// +/// # Example +/// +/// ```rust +/// # use datafusion_expr::{placeholder}; +/// let p = placeholder("$0"); // $0, refers to parameter 1 +/// assert_eq!(p.to_string(), "$0") +/// ``` +pub fn placeholder(id: impl Into) -> Expr { + Expr::Placeholder(Placeholder { + id: id.into(), + data_type: None, + }) +} + /// Return a new expression `left right` pub fn binary_expr(left: Expr, op: Operator, right: Expr) -> Expr { Expr::BinaryExpr(BinaryExpr::new(Box::new(left), op, Box::new(right))) diff --git a/datafusion/expr/src/logical_plan/plan.rs b/datafusion/expr/src/logical_plan/plan.rs index b865b6855724..1c526c7b4030 100644 --- a/datafusion/expr/src/logical_plan/plan.rs +++ b/datafusion/expr/src/logical_plan/plan.rs @@ -928,8 +928,40 @@ impl LogicalPlan { } } } - /// Convert a prepared [`LogicalPlan`] into its inner logical plan - /// with all params replaced with their corresponding values + /// Replaces placeholder param values (like `$1`, `$2`) in [`LogicalPlan`] + /// with the specified `param_values`. + /// + /// [`LogicalPlan::Prepare`] are + /// converted to their inner logical plan for execution. + /// + /// # Example + /// ``` + /// # use arrow::datatypes::{Field, Schema, DataType}; + /// use datafusion_common::ScalarValue; + /// # use datafusion_expr::{lit, col, LogicalPlanBuilder, logical_plan::table_scan, placeholder}; + /// # let schema = Schema::new(vec![ + /// # Field::new("id", DataType::Int32, false), + /// # ]); + /// // Build SELECT * FROM t1 WHRERE id = $1 + /// let plan = table_scan(Some("t1"), &schema, None).unwrap() + /// .filter(col("id").eq(placeholder("$1"))).unwrap() + /// .build().unwrap(); + /// + /// assert_eq!("Filter: t1.id = $1\ + /// \n TableScan: t1", + /// plan.display_indent().to_string() + /// ); + /// + /// // Fill in the parameter $1 with a literal 3 + /// let plan = plan.with_param_values(vec![ + /// ScalarValue::from(3i32) // value at index 0 --> $1 + /// ]).unwrap(); + /// + /// assert_eq!("Filter: t1.id = Int32(3)\ + /// \n TableScan: t1", + /// plan.display_indent().to_string() + /// ); + /// ``` pub fn with_param_values( self, param_values: Vec, @@ -961,7 +993,7 @@ impl LogicalPlan { let input_plan = prepare_lp.input; input_plan.replace_params_with_values(¶m_values) } - _ => Ok(self), + _ => self.replace_params_with_values(¶m_values), } } @@ -1060,7 +1092,7 @@ impl LogicalPlan { } impl LogicalPlan { - /// applies collect to any subqueries in the plan + /// applies `op` to any subqueries in the plan pub(crate) fn apply_subqueries(&self, op: &mut F) -> datafusion_common::Result<()> where F: FnMut(&Self) -> datafusion_common::Result, @@ -1112,9 +1144,11 @@ impl LogicalPlan { Ok(()) } - /// Return a logical plan with all placeholders/params (e.g $1 $2, - /// ...) replaced with corresponding values provided in the - /// params_values + /// Return a `LogicalPlan` with all placeholders (e.g $1 $2, + /// ...) replaced with corresponding values provided in + /// `params_values` + /// + /// See [`Self::with_param_values`] for examples and usage pub fn replace_params_with_values( &self, param_values: &[ScalarValue], @@ -1122,7 +1156,10 @@ impl LogicalPlan { let new_exprs = self .expressions() .into_iter() - .map(|e| Self::replace_placeholders_with_values(e, param_values)) + .map(|e| { + let e = e.infer_placeholder_types(self.schema())?; + Self::replace_placeholders_with_values(e, param_values) + }) .collect::>>()?; let new_inputs_with_values = self @@ -1219,7 +1256,9 @@ impl LogicalPlan { // Various implementations for printing out LogicalPlans impl LogicalPlan { /// Return a `format`able structure that produces a single line - /// per node. For example: + /// per node. + /// + /// # Example /// /// ```text /// Projection: employee.id @@ -2321,7 +2360,7 @@ pub struct Unnest { mod tests { use super::*; use crate::logical_plan::table_scan; - use crate::{col, exists, in_subquery, lit}; + use crate::{col, exists, in_subquery, lit, placeholder}; use arrow::datatypes::{DataType, Field, Schema}; use datafusion_common::tree_node::TreeNodeVisitor; use datafusion_common::{not_impl_err, DFSchema, TableReference}; @@ -2767,10 +2806,7 @@ digraph { let plan = table_scan(TableReference::none(), &schema, None) .unwrap() - .filter(col("id").eq(Expr::Placeholder(Placeholder::new( - "".into(), - Some(DataType::Int32), - )))) + .filter(col("id").eq(placeholder(""))) .unwrap() .build() .unwrap(); @@ -2783,10 +2819,7 @@ digraph { let plan = table_scan(TableReference::none(), &schema, None) .unwrap() - .filter(col("id").eq(Expr::Placeholder(Placeholder::new( - "$0".into(), - Some(DataType::Int32), - )))) + .filter(col("id").eq(placeholder("$0"))) .unwrap() .build() .unwrap(); diff --git a/datafusion/sql/src/expr/mod.rs b/datafusion/sql/src/expr/mod.rs index cb34b6ca36e8..a90a0f121f26 100644 --- a/datafusion/sql/src/expr/mod.rs +++ b/datafusion/sql/src/expr/mod.rs @@ -29,13 +29,12 @@ mod value; use crate::planner::{ContextProvider, PlannerContext, SqlToRel}; use arrow_schema::DataType; -use datafusion_common::tree_node::{Transformed, TreeNode}; use datafusion_common::{ internal_err, not_impl_err, plan_err, Column, DFSchema, DataFusionError, Result, ScalarValue, }; +use datafusion_expr::expr::InList; use datafusion_expr::expr::ScalarFunction; -use datafusion_expr::expr::{InList, Placeholder}; use datafusion_expr::{ col, expr, lit, AggregateFunction, Between, BinaryExpr, BuiltinScalarFunction, Cast, Expr, ExprSchemable, GetFieldAccess, GetIndexedField, Like, Operator, TryCast, @@ -122,7 +121,7 @@ impl<'a, S: ContextProvider> SqlToRel<'a, S> { let mut expr = self.sql_expr_to_logical_expr(sql, schema, planner_context)?; expr = self.rewrite_partial_qualifier(expr, schema); self.validate_schema_satisfies_exprs(schema, &[expr.clone()])?; - let expr = infer_placeholder_types(expr, schema)?; + let expr = expr.infer_placeholder_types(schema)?; Ok(expr) } @@ -712,49 +711,6 @@ impl<'a, S: ContextProvider> SqlToRel<'a, S> { } } -// modifies expr if it is a placeholder with datatype of right -fn rewrite_placeholder(expr: &mut Expr, other: &Expr, schema: &DFSchema) -> Result<()> { - if let Expr::Placeholder(Placeholder { id: _, data_type }) = expr { - if data_type.is_none() { - let other_dt = other.get_type(schema); - match other_dt { - Err(e) => { - Err(e.context(format!( - "Can not find type of {other} needed to infer type of {expr}" - )))?; - } - Ok(dt) => { - *data_type = Some(dt); - } - } - }; - } - Ok(()) -} - -/// Find all [`Expr::Placeholder`] tokens in a logical plan, and try -/// to infer their [`DataType`] from the context of their use. -fn infer_placeholder_types(expr: Expr, schema: &DFSchema) -> Result { - expr.transform(&|mut expr| { - // Default to assuming the arguments are the same type - if let Expr::BinaryExpr(BinaryExpr { left, op: _, right }) = &mut expr { - rewrite_placeholder(left.as_mut(), right.as_ref(), schema)?; - rewrite_placeholder(right.as_mut(), left.as_ref(), schema)?; - }; - if let Expr::Between(Between { - expr, - negated: _, - low, - high, - }) = &mut expr - { - rewrite_placeholder(low.as_mut(), expr.as_ref(), schema)?; - rewrite_placeholder(high.as_mut(), expr.as_ref(), schema)?; - } - Ok(Transformed::Yes(expr)) - }) -} - #[cfg(test)] mod tests { use super::*; diff --git a/datafusion/sql/tests/sql_integration.rs b/datafusion/sql/tests/sql_integration.rs index d95598cc3d8e..702d7dbce69e 100644 --- a/datafusion/sql/tests/sql_integration.rs +++ b/datafusion/sql/tests/sql_integration.rs @@ -3671,6 +3671,19 @@ fn test_prepare_statement_should_infer_types() { assert_eq!(actual_types, expected_types); } +#[test] +fn test_non_prepare_statement_should_infer_types() { + // Non prepared statements (like SELECT) should also have their parameter types inferred + let sql = "SELECT 1 + $1"; + let plan = logical_plan(sql).unwrap(); + let actual_types = plan.get_parameter_types().unwrap(); + let expected_types = HashMap::from([ + // constant 1 is inferred to be int64 + ("$1".to_string(), Some(DataType::Int64)), + ]); + assert_eq!(actual_types, expected_types); +} + #[test] #[should_panic( expected = "value: SQL(ParserError(\"Expected [NOT] NULL or TRUE|FALSE or [NOT] DISTINCT FROM after IS, found: $1\"" From e84b9994a2dadcf492a7a82a90589c15c7be5037 Mon Sep 17 00:00:00 2001 From: Sean Gillespie Date: Mon, 16 Oct 2023 08:17:45 -0700 Subject: [PATCH 078/572] Implement GetIndexedField for map-typed columns (#7825) * Implement GetIndexedField for map-typed columns * Drop explicit dep on arrow-ord and use re-exported kernel --- .../core/tests/data/parquet_map.parquet | Bin 0 -> 10546 bytes datafusion/expr/src/field_util.rs | 15 +++++- .../src/expressions/get_indexed_field.rs | 15 +++++- datafusion/sqllogictest/test_files/map.slt | 46 ++++++++++++++++++ 4 files changed, 73 insertions(+), 3 deletions(-) create mode 100644 datafusion/core/tests/data/parquet_map.parquet create mode 100644 datafusion/sqllogictest/test_files/map.slt diff --git a/datafusion/core/tests/data/parquet_map.parquet b/datafusion/core/tests/data/parquet_map.parquet new file mode 100644 index 0000000000000000000000000000000000000000..e7ffb5115c44f56577e6d26ca908eafed131b989 GIT binary patch literal 10546 zcmbt)30xFM)_B!SH}t?vF*D3e1_Y#)K~!kEdu~<gH zy|cQjv!z@?PwBOK!+gC=kF*FWB58zB@v25ogWk{Au}nWG*CQWL1Hw-emQPRg8n0Y{ zcu)?1Sk3^{D?_ZvPaWc*Dry=V{5>^|^%b55`3lb}z05EuH}qk{4je5;qu+4>Df&Y7 zdxGXqOb3E=tC+uSM>M7q|3v#U=#8ZchMNBXfQ(K6iQ52Nco0BO6@XD4fT5`X7z2PO z9|16y3&8(c1%tv=4r=ITg_>H@2B;(D0JanX_^)gLfyDrFX8^c+1Ax*p00)u)Bp(29 zn*gBs762D-18_JQK*wDGPPzcRvkSl*^8kF@0pQf5Fj<#s2<=y0Ky|7%s%fzzk_xPV zu80f(Cnj0KZTGHERJR76ACF56F5{4)l6lIc4ir7^%lv z0KMl<0K+g3b%q77DFVQ?W+07F9i_JB0moOSL#qT@1Afq2oCXA6*Z|1a0DhXvNX^m! ze%(w!HLU_*Z2)|u>H_t0qK|$7pqAAEY8%Y`mFhz(ElOcTYcgA?Hx&vsx{%3G?889F z?EymZMgR}p0MuWo>Zm)j0Ke3yh)-()^;BjI^_B>zkmJxwiOZ(s>!EeeV?c1gAHc;; z0A^~T^IFv~3Ifje-wx}1Wd~r4agiW_n%cPqW)H7MR6sDyuFvd4cY|L4dJoXGZ5?moznpG zCJ=p514M040#SE6pc3QS&}*u_)cOD*TDBE1kMtUmH}fcU^JEaunAt};?}G)=s&sUw zdX(CB46u*)_EU9(fPKFDEP6>bj#{gS(J?T?WpJt6s&`N_r@}#1Rs*+_ac9xH)eh>} zMZo_tH(*3Cz>`ma5OFJ@71^t%-in4+0SB#)%pp|PTaMnnA5jl11eg3X4cIf)MvCeO z8r%deq-sGMdlB+fGw4w7Idrj_M~|z@(bTwMv;t)5tdGC!D~aVbJbsa}BP^3sw4Xnt2nbZFNRV2i4c6%m*BC=WCxwMaOpauyOpP){Pm7s8 zV`gj|7taf#*s9aH1T~k|kV|_zo)5_+SRjb#mUDtY3o437VeP`E(?v0x^_w?Sp zrElxDTW-DW_U$|Vc*mW0-M#ak{(-@JckSMD-`;)u4;;MzPY(@Jng!w<_b{Ow$h85r_&Mf7igT3ECEw?*-i!SIdBu+1;} zG3={~|4M&g&++@PemH%;{!e}r7QMW1{0;(&6`T%Ea9TLQEOHh>4wyNqc|Oyo=jCO` z3miWa?#mk+>lzZ`;~Q$LJzPm`b$nxaMMJ!&rnIgW3Uz!PNY>a8UtL?{S;Yc*d~GpU ze5Jd%qN<{CRXn&$eMLzFkkytpl{8k=*7&Be@VsxDhH`g}XKqW1&rbvk&k2H+6GaQY zHO9A*$d>Yw(<4zxM^&?jtF2!i@AHrN;@a9ePknK1qaa$sfJ&u_1Y8~9;;ZS&6aa!> zW6Zn-zm{L=X=w12maYl}I-8lZIXMeJJDx;;trR()3$Kr=ch{8GR&zC;#?~UFv|4#3 z&Wiy$PPCdi!D`blQVN_98@|Ytv3emwTKtmhBZQxxW#P>V;5yG;Ur}9MyVAqXE)~sA z7$Jg2Z5(6}U$CU4&QHxt<-?nqvf6snaO(iZXY%9Sb#)E#rJlN~+Eu>TsrFRW2rC*S zYorsF4-yIvdId6uyJuCqS9rKmPkeJ-oV&ETq6XimS0NP}9t#s#L=aKnY&QHHlf5X9 zJtmgO*tf%5v3ps3gQuk4(->D#Q&t-f+Nz0b@HDuqJ=h&t?5XjT)mJpG#&`u;1Np~^}7UpHd z$ndebf6VQc;d>pT*$!OHU_OBp`5EE&rd~reHB=VK@pz=WyxLvruB@)W+j%p1oCW@U zmYx(okNre)&rU}nQqHo{+PIq9#yEFL30N(@#^Wh%h-A&$JbXmcz1si8~O1pFmvh(IH$l^RyYBC7b zo{D)MoXrecwDO!&WVb5QQ>7Gco=jZ>YHDCVn=Pk1EduDpVugNKl*2UJLN(Rl_<~9q zdwt~}F)oO-!%9V%#ldpO&RxWszo5oqCp|qH33!fLgqX4o`0GFv-ZDLWfS$wF7M{lR zwUoeT{wS4jT?iO@eyF@hhPx;;-+6|2h@9EN*+1Tcqt}3XC+;o>?Weut%<~ssh(<@H95nN$Yt~t6&55+ClCq;ag351|oKGEDb8O z<6C$ebPEC~1#D&K5>c-K^xBXcL>He%bea_AGW*f3O3<$j+toJEqH`cutf!xN z$h3m7i(JKi%E5E3{@z6}ds;DKXM%Vlt_Tw>FfH&JXk$d182-WZ?{EGP(H*AGbQ}l- z!cV5?4^~d5*lB7(0JBJKN(&?|j*?T&9=fBFlH=3TQhyfdiRwC2~{{odeWpwI}YzxIZyP z&hBQwAqL(y3OuOCA`ZN%gkfgxysY}I*3B|l_4ybxID`PB9Kz)Ak$oAiO)}aR1?v@g zhz-$90vQw?ZbQx5Sxij<#Rv|IbY^Uf0v|jrlkUJ@bA?zb!tzJzm7oYamqz2*dXK*( zlTJo~3v!UDeL2P@Gd%-=T6uzE?}-F*K#9Eb)L{5~(0hINc@OGRC`nS%+D;U9SA&NlqZ~!G*Q*R&|B|E5g*fs(WbBQ=GI5C`&G)yy|Q{-eXfk+mH8^ZgVYgE{@ zo^o1JKT(5+SgiP@MS#>VfC(IsC}L2wS7lOo&laqe;a*A*rN2>}NK!x{0#`_ZyVJ8; z<(TcIM1JG|c{4x_Hns2xg;Z9TA8nJqWNbDLGmI$O($+DoqU>=Qxg4m38JunvNNMH1 zLlsBM{($J#&I@b@j25g8?soe57FRIMt{Xc?IYrYjvVQa@O0=<;5l8zYo}1(63C1Ud z?@b4hNOm&hTmWM>ht59eJ^*%3M1kORummUX=?Aegrl{CA%RWS=G2|qVDH$tgA4$$i zM$$bilM#Lggs|gepc4oE>36z{XvV==$h{117a7IeabQ@m0wX)Se0Kq56SxL{dS%&S z83P_lUI?JfwvkKRW;*kHO$9{`Ds%#*b(@8(Rip(g*cV#0a%md=)H~>10#e^8qmSXV zf?MTyHb|1GL{7tYMQSM|ialUMLoY!hJyi{!VY6x<52RoT?;^SgT(0m2q+P-wo?Zk> zFx(FQq0MKw%XH|Bz??^KF(+2Pog-gNqr}SlKr+wgg#;1aurR!Ak8o z3yE3{h^^hJOx&+=cefg4Y&vNs$ZWV?$@q#3rb3@r%ud6NlwHtvD%B8bW~)msm*yna z`m+fmO07Krf{KC?I$0LCrcf@Y6Ws?%qDneO!PCRn3NKgixC{@mN0boIHsT53>hVVl zD!;%efZV86!#=~|WIsnyPlSG-=GsQH3(2_Z)G6{zP-3}^om#mP;fE-}%DxNplY7ki^MXT4W)q)MuGO=rL7bJmk zIek;smwwnzp4O7xItXi#b859pLTkMHq_YtIL*5E&O5M1o2vlN)<%U;j_~W}Dz0N-SIWud?#o0{F`y$zpyGKH8qE}(N4SbC zW!$^f;E0e~-o+c=FPkEBe=u-^78?m|M1p9dO>5Y!!YCP?gzSbtUt_QZ!al+zQ+xtJ z>eBm7Up4Y5NL8tUY-7)G(|A=bWGFDx3*snGm|EDVUHT%}5qwaYWpm1sR?FV`3VjkGPAJkUDs84zi_f(BZ?b8z79? z2CXR&k{`siVtSLbu5$B)hL18@jh2E0;F1~SL4^-s+ya&$I?B$G+jOVWh!{ZP2zv&s z!F`S5R3P!r;!V3Bu%@j`*vUP_k^|7Oml)09UtM@dHsS zkY!(*xK4KH#lm_-CxXl61S0dPeaMoUL6(M@{_a*IJB}K8j(iTAE}UCdm3IH@^fc`%Faf7bO{O;P9wB_ppZb!CLA5cz*)~Ffz0Fs#Z4EfQw3*hiS zOrC)SYs)h}xLv&3Kz_ZplrL(3f z#1sn3w(<$p{;PMRkuqX8Kx(kt6IKLEH4#-Rb}QIi6i-D0 zJqbb&c8F@qY)-qRz)!TV@F!NIuPEFNp25vjn4ZWu7iKz0FrlnYZA~bc4YOlkkXN)y zc)&A@-!ikGOl*UhgGTRu8Fq!_>?8^{ z#@|j+ZKyx5#tTX0HROjNa#00XdXN%?5fzbi;&K(4AHW`QrTJkiY!419y?&r{lg_gq z; zIMPhnc_PJ(?IjtEL=Z-QM&8sk5G%l4m+|Bv3%eX0Da-cNB8k}<}l{x+|ip`x;f+N2IdB4XNx zBV{s0a7^A%9q#B*hR&!OZIfL7LF(AGXe@GlcfizisxRB+-54B`%VbOas6!taHWWG3 z-?JVGsck59N@dm1jme0p^u~A3jTQN0Caxt?eeT2@mHb+@0}Zm> zG&?;^`FTzo-l`*;F7wAn(~*DrTAgESRHxiDFVa|~O-c%;vkGR7jG||@#Ta%`b-ZV; zDlku-zIU+yG&2*2E&F_E7#S$-gWdVOYcs)1Q|g>Arf#6(BMgTx%Um)eO=qotl$Cn6 z(8|+AeMwzOIzNv(oqy-vF0a`b)$iSUy8Vtx3J9f3r3ru0cs41hnQU`TfohHAi zNaKflbi3T=V%pH0Df5jjyH4OJl{GdcgB?-P(nT@P(=?N&!_O1i^vZ!QyQ znv%IWHz91t`C-q=>69!x2kgEG`R|{VXFL?(l`o!GnP1*?!aGy7tl*ey6y4q~GZfS6 z$Y+zW-%StYrZc#s7Z*&Yc1+10PP!k-s=hn=7ul;S#iK0Jr#{42^8sxTQWSFj}&KphL7z;o``_E z^1CEt)@0c%$)m#;wNsTlKT@OK_FGgh<~sWJjK0!ckq~xNqd9?&Op@)_FvY`n5XGx& zjUnE+&W@^xqsDW7u3;Nkt}c1B|F7FZE-4zhHxWHQ(N|7ly~kmKiTR&W`TfATx}6>-x5| zZB_kE&0Lsvd?Ve;M1A$;NLk0V;pB&rq7qz${ITBXWV1JPcq7gqj%-Ozk~CJEaXvdb zx^1bSdXKtK-4!ekXmaGIdpFZvkx`bH`}TAuQ?iIn{2@F@6b~2eI;0jlsg3iF_Uzbk z9H(jm)AHuCXp?{N)L6f|I?d^JjZu?+Hl57~?6pT#MHRhGok1l3GJk$>s!S2FZpit$ z#w+jBUC3t?$Gz#3E7E%AkF~wK%@UWfM~&vKSIDDzrBZGhui4xTetz;9WmH*VPEp`)BtP2`24Z{54J6xw04hr{3ZWM zfk{p2OvXSP9xzhMuSZL%9Hnf33s|l^!2h)6^#_YAZ5=E%!|1rfHT;Ols*F4pnY4#t z;yb&L;N5y_My!+H5CF#@ff@~(^FU?Nk`>bv9 z`$LQKFRVU+!;Q-d>N3ar5BLR7J=R_~z9=Qsuc+3+92hyv9Ad(HzP|BUbWx{$W(I%b z@lREFxjvR}v$eGy4mBLkK5$~BK(ov^l*tan^@J*4-%~ffa^prros^y*GuY-mYK+?P zMNx6bGPUge%A-fToF8u5ezK^@-q}IT>!~_=AX!!@OG|tA308W5vH5+|uI9UwyoR++ zjt>J*ob`K(iAw*nuk7SoIxYHpcQHGz!tLAZ1U`QQI!^T%W0bCTcYr!A>~v&X*9ME9 zx}>jf0d=Qg<3h)WkL+s0r;JfiQQ1W&-<~8hE=(`!7i5Z3OWuW1N9X!8p^^D*%HvY0 zDl03Uj21oU&lqCvQ}?BMZwbCl!wenlZ==z_JmUHJ6b}x0-aLg3N6Vt$P4}Jh{L2FD zJLd7%-!7Z4pK;}oN4`u4Wtq3dFDPk@hj%3A1i_wQH78iC1x7dz@d3LJIDEjl*oa^^ z1qetmR{^axJ;biWnOM{N8Z z)+aKoWaeI4i*#5g)5&#|j@J3<{B?{@p$pIj>XbT_j_87PYMllON3c$JRk21zV+Fj@ zZ**7JT{#l{LSeXzihfw;)8+4G`gik(Xk;NjeYkSs4}N6xV}Z&_ z8Lxh+qo=@|O~~l`V#kE$`_AU{6tu5+vTx|sW6b~QXdkyLqp$MwVSX{_PgwXkC?qUo zS#&=RyK41+{|!+>H1oS}hzw_M#(JG$bTihAD0;mi?0%sa&6n!%M7n!fB=c+<~`t<4NQ{GRF(GOnpua9{RUrjJR2aHot^rp^VY2&w?|4DLr zczN{sh<}q^3_V+XV&B?}^=T;j*rs2&|H>f0jxg|~=>9JJzYH?NE|@0N@ES~0gA9AW z$6(xO#&D?A49nx*9BQicx&S4r~q z`hGdkmZZ79TR>gCLMs#gQ+$+CmJdv@N^Xu{86TfE(QclA0Mb&f-T`M{f3k1Z^xV4Q z>lUvros-sB;_J^^Ju^PuvBY=#Gren$Z<3#+^i7qL;~O8JZI6#{nc#n=-`8`UkI*OT zRe2`pc_01UY#7rrvDBaGT{%C?C#CxOn-{@Im$>A*7NNjLnpz1+hY$ISJkp|{$4e;_ z{S%_P#l==h@~y&ERa}$1$_?xKslB>1J)^F;x^%TPH_ylKx9hti#Dr)QO1A(}eBFe+ zzggds38IPhE%pf|`RJ2NeDtd(=zZ-8r0*(84BfHNpSq^}BK>h$aAVg&`9R|BXI;{jNID59r4Cn>`yD mL+af1@a;=uTzy02N>6 { match (data_type, name) { + (DataType::Map(fields, _), _) => { + match fields.data_type() { + DataType::Struct(fields) if fields.len() == 2 => { + // Arrow's MapArray is essentially a ListArray of structs with two columns. They are + // often named "key", and "value", but we don't require any specific naming here; + // instead, we assume that the second columnis the "value" column both here and in + // execution. + let value_field = fields.get(1).expect("fields should have exactly two members"); + Ok(Field::new("map", value_field.data_type().clone(), true)) + }, + _ => plan_err!("Map fields must contain a Struct with exactly 2 fields"), + } + } (DataType::Struct(fields), ScalarValue::Utf8(Some(s))) => { if s.is_empty() { plan_err!( @@ -60,7 +73,7 @@ impl GetFieldAccessSchema { (DataType::Struct(_), _) => plan_err!( "Only utf8 strings are valid as an indexed field in a struct" ), - (other, _) => plan_err!("The expression to get an indexed field is only valid for `List` or `Struct` types, got {other}"), + (other, _) => plan_err!("The expression to get an indexed field is only valid for `List`, `Struct`, or `Map` types, got {other}"), } } Self::ListIndex{ key_dt } => { diff --git a/datafusion/physical-expr/src/expressions/get_indexed_field.rs b/datafusion/physical-expr/src/expressions/get_indexed_field.rs index ab15356dc212..df79e2835820 100644 --- a/datafusion/physical-expr/src/expressions/get_indexed_field.rs +++ b/datafusion/physical-expr/src/expressions/get_indexed_field.rs @@ -18,16 +18,19 @@ //! get field of a `ListArray` use crate::PhysicalExpr; -use arrow::array::Array; use datafusion_common::exec_err; use crate::array_expressions::{array_element, array_slice}; use crate::physical_expr::down_cast_any_ref; use arrow::{ + array::{Array, Scalar, StringArray}, datatypes::{DataType, Schema}, record_batch::RecordBatch, }; -use datafusion_common::{cast::as_struct_array, DataFusionError, Result, ScalarValue}; +use datafusion_common::{ + cast::{as_map_array, as_struct_array}, + DataFusionError, Result, ScalarValue, +}; use datafusion_expr::{field_util::GetFieldAccessSchema, ColumnarValue}; use std::fmt::Debug; use std::hash::{Hash, Hasher}; @@ -183,6 +186,14 @@ impl PhysicalExpr for GetIndexedFieldExpr { let array = self.arg.evaluate(batch)?.into_array(batch.num_rows()); match &self.field { GetFieldAccessExpr::NamedStructField{name} => match (array.data_type(), name) { + (DataType::Map(_, _), ScalarValue::Utf8(Some(k))) => { + let map_array = as_map_array(array.as_ref())?; + let key_scalar = Scalar::new(StringArray::from(vec![k.clone()])); + let keys = arrow::compute::kernels::cmp::eq(&key_scalar, map_array.keys())?; + let entries = arrow::compute::filter(map_array.entries(), &keys)?; + let entries_struct_array = as_struct_array(entries.as_ref())?; + Ok(ColumnarValue::Array(entries_struct_array.column(1).clone())) + } (DataType::Struct(_), ScalarValue::Utf8(Some(k))) => { let as_struct_array = as_struct_array(&array)?; match as_struct_array.column_by_name(k) { diff --git a/datafusion/sqllogictest/test_files/map.slt b/datafusion/sqllogictest/test_files/map.slt new file mode 100644 index 000000000000..c3d16fca904e --- /dev/null +++ b/datafusion/sqllogictest/test_files/map.slt @@ -0,0 +1,46 @@ +# 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 EXTERNAL TABLE data +STORED AS PARQUET +LOCATION '../core/tests/data/parquet_map.parquet'; + +query I +SELECT SUM(ints['bytes']) FROM data; +---- +5636785 + +query I +SELECT SUM(ints['bytes']) FROM data WHERE strings['method'] == 'GET'; +---- +649668 + +query TI +SELECT strings['method'] AS method, COUNT(*) as count FROM data GROUP BY method ORDER BY count DESC; +---- +POST 41 +HEAD 33 +PATCH 30 +OPTION 29 +GET 27 +PUT 25 +DELETE 24 + +query T +SELECT strings['not_found'] FROM data LIMIT 1; +---- From 7aa6b3659b33510203778c3978f0830c1b3f7c2c Mon Sep 17 00:00:00 2001 From: Andrew Lamb Date: Mon, 16 Oct 2023 11:18:08 -0400 Subject: [PATCH 079/572] Minor: Assert `streaming_merge` has non empty sort exprs (#7795) * Minor: Assert streaming_merge has non empty sort exprs * clippy * Add test * Apply suggestions from code review Co-authored-by: jakevin Co-authored-by: Liang-Chi Hsieh --------- Co-authored-by: jakevin Co-authored-by: Liang-Chi Hsieh --- datafusion/physical-plan/src/sorts/cursor.rs | 18 +++++++++++----- .../src/sorts/sort_preserving_merge.rs | 21 ++++++++++++++++++- .../src/sorts/streaming_merge.rs | 7 ++++++- 3 files changed, 39 insertions(+), 7 deletions(-) diff --git a/datafusion/physical-plan/src/sorts/cursor.rs b/datafusion/physical-plan/src/sorts/cursor.rs index baa417649fb0..52de880bae69 100644 --- a/datafusion/physical-plan/src/sorts/cursor.rs +++ b/datafusion/physical-plan/src/sorts/cursor.rs @@ -48,16 +48,17 @@ impl std::fmt::Debug for RowCursor { impl RowCursor { /// Create a new SortKeyCursor from `rows` and a `reservation` - /// that tracks its memory. + /// that tracks its memory. There must be at least one row /// - /// Panic's if the reservation is not for exactly `rows.size()` - /// bytes + /// Panics if the reservation is not for exactly `rows.size()` + /// bytes or if `rows` is empty. pub fn new(rows: Rows, reservation: MemoryReservation) -> Self { assert_eq!( rows.size(), reservation.size(), "memory reservation mismatch" ); + assert!(rows.num_rows() > 0); Self { cur_row: 0, num_rows: rows.num_rows(), @@ -92,7 +93,10 @@ impl Ord for RowCursor { } } -/// A cursor into a sorted batch of rows +/// A cursor into a sorted batch of rows. +/// +/// Each cursor must have at least one row so `advance` can be called at least +/// once prior to calling `is_finished`. pub trait Cursor: Ord { /// Returns true if there are no more rows in this cursor fn is_finished(&self) -> bool; @@ -207,8 +211,12 @@ pub struct FieldCursor { } impl FieldCursor { - /// Create a new [`FieldCursor`] from the provided `values` sorted according to `options` + /// Create a new [`FieldCursor`] from the provided `values` sorted according + /// to `options`. + /// + /// Panics if the array is empty pub fn new>(options: SortOptions, array: &A) -> Self { + assert!(array.len() > 0, "Empty array passed to FieldCursor"); let null_threshold = match options.nulls_first { true => array.null_count(), false => array.len() - array.null_count(), diff --git a/datafusion/physical-plan/src/sorts/sort_preserving_merge.rs b/datafusion/physical-plan/src/sorts/sort_preserving_merge.rs index 597b59f776d5..f7a9e193ee06 100644 --- a/datafusion/physical-plan/src/sorts/sort_preserving_merge.rs +++ b/datafusion/physical-plan/src/sorts/sort_preserving_merge.rs @@ -290,7 +290,7 @@ mod tests { use crate::test::{self, assert_is_pending, make_partition}; use crate::{collect, common}; use arrow::array::{Int32Array, StringArray, TimestampNanosecondArray}; - use datafusion_common::assert_batches_eq; + use datafusion_common::{assert_batches_eq, assert_contains}; use super::*; @@ -342,6 +342,25 @@ mod tests { .await; } + #[tokio::test] + async fn test_merge_no_exprs() { + let task_ctx = Arc::new(TaskContext::default()); + let a: ArrayRef = Arc::new(Int32Array::from(vec![1, 2, 7, 9, 3])); + let batch = RecordBatch::try_from_iter(vec![("a", a)]).unwrap(); + + let schema = batch.schema(); + let sort = vec![]; // no sort expressions + let exec = MemoryExec::try_new(&[vec![batch.clone()], vec![batch]], schema, None) + .unwrap(); + let merge = Arc::new(SortPreservingMergeExec::new(sort, Arc::new(exec))); + + let res = collect(merge, task_ctx).await.unwrap_err(); + assert_contains!( + res.to_string(), + "Internal error: Sort expressions cannot be empty for streaming merge" + ); + } + #[tokio::test] async fn test_merge_some_overlap() { let task_ctx = Arc::new(TaskContext::default()); diff --git a/datafusion/physical-plan/src/sorts/streaming_merge.rs b/datafusion/physical-plan/src/sorts/streaming_merge.rs index 96d180027eee..4f8d8063853b 100644 --- a/datafusion/physical-plan/src/sorts/streaming_merge.rs +++ b/datafusion/physical-plan/src/sorts/streaming_merge.rs @@ -26,7 +26,7 @@ use crate::sorts::{ use crate::{PhysicalSortExpr, SendableRecordBatchStream}; use arrow::datatypes::{DataType, SchemaRef}; use arrow_array::*; -use datafusion_common::Result; +use datafusion_common::{internal_err, DataFusionError, Result}; use datafusion_execution::memory_pool::MemoryReservation; macro_rules! primitive_merge_helper { @@ -60,6 +60,11 @@ pub fn streaming_merge( fetch: Option, reservation: MemoryReservation, ) -> Result { + // If there are no sort expressions, preserving the order + // doesn't mean anything (and result in infinite loops) + if expressions.is_empty() { + return internal_err!("Sort expressions cannot be empty for streaming merge"); + } // Special case single column comparisons with optimized cursor implementations if expressions.len() == 1 { let sort = expressions[0].clone(); From 90bb5dd05e2500f887d4da2a11f262c5f53c36e1 Mon Sep 17 00:00:00 2001 From: Andrew Lamb Date: Mon, 16 Oct 2023 11:59:27 -0400 Subject: [PATCH 080/572] Minor: Upgrade docs for `PhysicalExpr::{propagate_constraints, evaluate_bounds}` (#7812) MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit * Minor: Upgrade docs for `PhysicalExpr::{propagate_constraints, evaluate_bounds}` * Update datafusion/physical-expr/src/physical_expr.rs Co-authored-by: Berkay Şahin <124376117+berkaysynnada@users.noreply.github.com> --------- Co-authored-by: Berkay Şahin <124376117+berkaysynnada@users.noreply.github.com> --- datafusion/physical-expr/src/physical_expr.rs | 42 +++++++++++++++++-- 1 file changed, 38 insertions(+), 4 deletions(-) diff --git a/datafusion/physical-expr/src/physical_expr.rs b/datafusion/physical-expr/src/physical_expr.rs index 00fc9ed5fa98..0eff45b6b9f7 100644 --- a/datafusion/physical-expr/src/physical_expr.rs +++ b/datafusion/physical-expr/src/physical_expr.rs @@ -75,15 +75,49 @@ pub trait PhysicalExpr: Send + Sync + Display + Debug + PartialEq { children: Vec>, ) -> Result>; - /// Computes bounds for the expression using interval arithmetic. + /// Computes the output interval for the expression, given the input + /// intervals. + /// + /// # Arguments + /// + /// * `children` are the intervals for the children (inputs) of this + /// expression. + /// + /// # Example + /// + /// If the expression is `a + b`, and the input intervals are `a: [1, 2]` + /// and `b: [3, 4]`, then the output interval would be `[4, 6]`. fn evaluate_bounds(&self, _children: &[&Interval]) -> Result { not_impl_err!("Not implemented for {self}") } - /// Updates/shrinks bounds for the expression using interval arithmetic. + /// Updates bounds for child expressions, given a known interval for this + /// expression. + /// + /// This is used to propagate constraints down through an + /// expression tree. + /// + /// # Arguments + /// + /// * `interval` is the currently known interval for this expression. + /// * `children` are the current intervals for the children of this expression + /// + /// # Returns + /// + /// A Vec of new intervals for the children, in order. + /// /// If constraint propagation reveals an infeasibility, returns [None] for - /// the child causing infeasibility. If none of the children intervals - /// change, may return an empty vector instead of cloning `children`. + /// the child causing infeasibility. + /// + /// If none of the child intervals change as a result of propagation, may + /// return an empty vector instead of cloning `children`. + /// + /// # Example + /// + /// If the expression is `a + b`, the current `interval` is `[4, 5] and the + /// inputs are given [`a: [0, 2], `b: [-∞, 4]]`, then propagation would + /// would return `[a: [0, 2], b: [2, 4]]` as `b` must be at least 2 to + /// make the output at least `4`. fn propagate_constraints( &self, _interval: &Interval, From cb2d03c826c2104ee6960cd6c3652146f800ae47 Mon Sep 17 00:00:00 2001 From: Jay Zhan Date: Tue, 17 Oct 2023 02:41:18 +0800 Subject: [PATCH 081/572] Change ScalarValue::List to store ArrayRef (#7629) * first draft Signed-off-by: jayzhan211 * revert subquery Signed-off-by: jayzhan211 * Decimal128 in list_to_array Signed-off-by: jayzhan211 * refactor null in list_to_array Signed-off-by: jayzhan211 * cleanup iter to array v3 Signed-off-by: jayzhan211 * remove comment Signed-off-by: jayzhan211 * cleanup Signed-off-by: jayzhan211 * cleanup struct array Signed-off-by: jayzhan211 * remove new_list in array Signed-off-by: jayzhan211 * cleanup Signed-off-by: jayzhan211 * cleanup Signed-off-by: jayzhan211 * cleanup list to array Signed-off-by: jayzhan211 * cleanup list to array Signed-off-by: jayzhan211 * remove try from array v2 Signed-off-by: jayzhan211 * remove try from array v4 Signed-off-by: jayzhan211 * cleanup expr_simply Signed-off-by: jayzhan211 * cleanup iter to array v3 Signed-off-by: jayzhan211 * checkpoint Signed-off-by: jayzhan211 * remove try from array v3 Signed-off-by: jayzhan211 * remove iter to array v4 Signed-off-by: jayzhan211 * rewrite iter to array list v2 Signed-off-by: jayzhan211 * cleanup iter to array list v3 Signed-off-by: jayzhan211 * remove test Signed-off-by: jayzhan211 * refactor wrap_into_list_array Signed-off-by: jayzhan211 * cleanup Signed-off-by: jayzhan211 * cleanup Signed-off-by: jayzhan211 * revert math expressions Signed-off-by: jayzhan211 * remove iter to array v3 Signed-off-by: jayzhan211 * fmt Signed-off-by: jayzhan211 * cleanup Signed-off-by: jayzhan211 * cleanup Signed-off-by: jayzhan211 * cleanup Signed-off-by: jayzhan211 * cleanup Signed-off-by: jayzhan211 * fix check Signed-off-by: jayzhan211 * fix build_a_list_array_from_scalars Signed-off-by: jayzhan211 * fmt Signed-off-by: jayzhan211 * clippy fix Signed-off-by: jayzhan211 * fix scalars test Signed-off-by: jayzhan211 * fix array agg distinct Signed-off-by: jayzhan211 * fix agg test Signed-off-by: jayzhan211 * cleanup Signed-off-by: jayzhan211 * add hash for scalar::Listarr Signed-off-by: jayzhan211 * fix count distinct Signed-off-by: jayzhan211 * fmt Signed-off-by: jayzhan211 * comment out scalarvalue list inproto Signed-off-by: jayzhan211 * rename Signed-off-by: jayzhan211 * rename and add comment Signed-off-by: jayzhan211 * reduce tryFromArray for List Signed-off-by: jayzhan211 * reduce tryFromArray for FixSizeList Signed-off-by: jayzhan211 * fmt Signed-off-by: jayzhan211 * proto for list Signed-off-by: jayzhan211 * cleanup Signed-off-by: jayzhan211 * replace List with ListArr Signed-off-by: jayzhan211 * remove build a list ... func Signed-off-by: jayzhan211 * refactor merge_batch in DistinctBitXorAccumulator Signed-off-by: jayzhan211 * rewrite merge batch in ArrayAggAccumulator Signed-off-by: jayzhan211 * clippy Signed-off-by: jayzhan211 * cleanup Signed-off-by: jayzhan211 * return array directly in array() Signed-off-by: jayzhan211 * add expected message Signed-off-by: jayzhan211 * fmt Signed-off-by: jayzhan211 * rename add add doc test for core function Signed-off-by: jayzhan211 * fix ci Signed-off-by: jayzhan211 * fix type Signed-off-by: jayzhan211 * rename and address comment Signed-off-by: jayzhan211 * address comment Signed-off-by: jayzhan211 * rewrite roundtrip scalars Signed-off-by: jayzhan211 * fix flatten Signed-off-by: jayzhan211 * cleanup Signed-off-by: jayzhan211 * add partial cmp Signed-off-by: jayzhan211 * fmt Signed-off-by: jayzhan211 * add comment Signed-off-by: jayzhan211 * cleanup Signed-off-by: jayzhan211 --------- Signed-off-by: jayzhan211 Co-authored-by: Andrew Lamb --- datafusion/common/src/scalar.rs | 897 ++++++++++++------ datafusion/common/src/utils.rs | 16 +- datafusion/core/tests/sql/aggregates.rs | 36 +- .../simplify_expressions/expr_simplifier.rs | 10 +- .../physical-expr/src/aggregate/array_agg.rs | 161 ++-- .../src/aggregate/array_agg_distinct.rs | 336 ++++--- .../src/aggregate/array_agg_ordered.rs | 88 +- .../src/aggregate/bit_and_or_xor.rs | 17 +- .../src/aggregate/count_distinct.rs | 167 ++-- .../physical-expr/src/aggregate/median.rs | 6 +- .../src/aggregate/sum_distinct.rs | 8 +- .../physical-expr/src/aggregate/tdigest.rs | 25 +- .../physical-expr/src/array_expressions.rs | 117 +-- datafusion/physical-expr/src/functions.rs | 8 +- datafusion/physical-plan/src/values.rs | 3 +- datafusion/proto/proto/datafusion.proto | 11 +- datafusion/proto/src/generated/pbjson.rs | 81 +- datafusion/proto/src/generated/prost.rs | 15 +- .../proto/src/logical_plan/from_proto.rs | 70 +- datafusion/proto/src/logical_plan/to_proto.rs | 45 +- .../tests/cases/roundtrip_logical_plan.rs | 89 +- datafusion/sql/src/expr/value.rs | 7 +- 22 files changed, 1219 insertions(+), 994 deletions(-) diff --git a/datafusion/common/src/scalar.rs b/datafusion/common/src/scalar.rs index 242d784edc9d..2d47b3e31472 100644 --- a/datafusion/common/src/scalar.rs +++ b/datafusion/common/src/scalar.rs @@ -26,12 +26,13 @@ use std::{convert::TryFrom, fmt, iter::repeat, sync::Arc}; use crate::cast::{ as_decimal128_array, as_decimal256_array, as_dictionary_array, - as_fixed_size_binary_array, as_fixed_size_list_array, as_list_array, as_struct_array, + as_fixed_size_binary_array, as_fixed_size_list_array, as_struct_array, }; use crate::error::{DataFusionError, Result, _internal_err, _not_impl_err}; -use arrow::buffer::NullBuffer; +use crate::hash_utils::create_hashes; +use crate::utils::wrap_into_list_array; +use arrow::buffer::{NullBuffer, OffsetBuffer}; use arrow::compute::kernels::numeric::*; -use arrow::compute::nullif; use arrow::datatypes::{i256, FieldRef, Fields, SchemaBuilder}; use arrow::{ array::*, @@ -45,6 +46,7 @@ use arrow::{ DECIMAL128_MAX_PRECISION, }, }; +use arrow_array::cast::as_list_array; use arrow_array::{ArrowNativeTypeOp, Scalar}; /// Represents a dynamically typed, nullable single value. @@ -95,8 +97,8 @@ pub enum ScalarValue { LargeBinary(Option>), /// Fixed size list of nested ScalarValue Fixedsizelist(Option>, FieldRef, i32), - /// List of nested ScalarValue - List(Option>, FieldRef), + /// Represents a single element of a [`ListArray`] as an [`ArrayRef`] + List(ArrayRef), /// Date stored as a signed 32bit int days since UNIX epoch 1970-01-01 Date32(Option), /// Date stored as a signed 64bit int milliseconds since UNIX epoch 1970-01-01 @@ -198,8 +200,8 @@ impl PartialEq for ScalarValue { v1.eq(v2) && t1.eq(t2) && l1.eq(l2) } (Fixedsizelist(_, _, _), _) => false, - (List(v1, t1), List(v2, t2)) => v1.eq(v2) && t1.eq(t2), - (List(_, _), _) => false, + (List(v1), List(v2)) => v1.eq(v2), + (List(_), _) => false, (Date32(v1), Date32(v2)) => v1.eq(v2), (Date32(_), _) => false, (Date64(v1), Date64(v2)) => v1.eq(v2), @@ -316,14 +318,37 @@ impl PartialOrd for ScalarValue { } } (Fixedsizelist(_, _, _), _) => None, - (List(v1, t1), List(v2, t2)) => { - if t1.eq(t2) { - v1.partial_cmp(v2) + (List(arr1), List(arr2)) => { + if arr1.data_type() == arr2.data_type() { + let list_arr1 = as_list_array(arr1); + let list_arr2 = as_list_array(arr2); + if list_arr1.len() != list_arr2.len() { + return None; + } + for i in 0..list_arr1.len() { + let arr1 = list_arr1.value(i); + let arr2 = list_arr2.value(i); + + let lt_res = + arrow::compute::kernels::cmp::lt(&arr1, &arr2).unwrap(); + let eq_res = + arrow::compute::kernels::cmp::eq(&arr1, &arr2).unwrap(); + + for j in 0..lt_res.len() { + if lt_res.is_valid(j) && lt_res.value(j) { + return Some(Ordering::Less); + } + if eq_res.is_valid(j) && !eq_res.value(j) { + return Some(Ordering::Greater); + } + } + } + Some(Ordering::Equal) } else { None } } - (List(_, _), _) => None, + (List(_), _) => None, (Date32(v1), Date32(v2)) => v1.partial_cmp(v2), (Date32(_), _) => None, (Date64(v1), Date64(v2)) => v1.partial_cmp(v2), @@ -441,9 +466,14 @@ impl std::hash::Hash for ScalarValue { t.hash(state); l.hash(state); } - List(v, t) => { - v.hash(state); - t.hash(state); + List(arr) => { + let arrays = vec![arr.to_owned()]; + let hashes_buffer = &mut vec![0; arr.len()]; + let random_state = ahash::RandomState::with_seeds(0, 0, 0, 0); + let hashes = + create_hashes(&arrays, &random_state, hashes_buffer).unwrap(); + // Hash back to std::hash::Hasher + hashes.hash(state); } Date32(v) => v.hash(state), Date64(v) => v.hash(state), @@ -570,28 +600,6 @@ macro_rules! typed_cast { }}; } -// keep until https://github.com/apache/arrow-rs/issues/2054 is finished -macro_rules! build_list { - ($VALUE_BUILDER_TY:ident, $SCALAR_TY:ident, $VALUES:expr, $SIZE:expr) => {{ - match $VALUES { - // the return on the macro is necessary, to short-circuit and return ArrayRef - None => { - return new_null_array( - &DataType::List(Arc::new(Field::new( - "item", - DataType::$SCALAR_TY, - true, - ))), - $SIZE, - ) - } - Some(values) => { - build_values_list!($VALUE_BUILDER_TY, $SCALAR_TY, values, $SIZE) - } - } - }}; -} - macro_rules! build_timestamp_list { ($TIME_UNIT:expr, $TIME_ZONE:expr, $VALUES:expr, $SIZE:expr) => {{ match $VALUES { @@ -792,11 +800,6 @@ impl ScalarValue { ScalarValue::IntervalMonthDayNano(Some(val)) } - /// Create a new nullable ScalarValue::List with the specified child_type - pub fn new_list(scalars: Option>, child_type: DataType) -> Self { - Self::List(scalars, Arc::new(Field::new("item", child_type, true))) - } - /// Create a zero value in the given type. pub fn new_zero(datatype: &DataType) -> Result { assert!(datatype.is_primitive()); @@ -953,11 +956,7 @@ impl ScalarValue { Arc::new(Field::new("item", field.data_type().clone(), true)), *length, ), - ScalarValue::List(_, field) => DataType::List(Arc::new(Field::new( - "item", - field.data_type().clone(), - true, - ))), + ScalarValue::List(arr) => arr.data_type().to_owned(), ScalarValue::Date32(_) => DataType::Date32, ScalarValue::Date64(_) => DataType::Date64, ScalarValue::Time32Second(_) => DataType::Time32(TimeUnit::Second), @@ -1105,7 +1104,7 @@ impl ScalarValue { ScalarValue::FixedSizeBinary(_, v) => v.is_none(), ScalarValue::LargeBinary(v) => v.is_none(), ScalarValue::Fixedsizelist(v, ..) => v.is_none(), - ScalarValue::List(v, _) => v.is_none(), + ScalarValue::List(arr) => arr.len() == arr.null_count(), ScalarValue::Date32(v) => v.is_none(), ScalarValue::Date64(v) => v.is_none(), ScalarValue::Time32Second(v) => v.is_none(), @@ -1307,17 +1306,18 @@ impl ScalarValue { ($ARRAY_TY:ident, $SCALAR_TY:ident, $NATIVE_TYPE:ident) => {{ Arc::new(ListArray::from_iter_primitive::<$ARRAY_TY, _, _>( scalars.into_iter().map(|x| match x { - ScalarValue::List(xs, _) => xs.map(|x| { - x.iter().map(|x| match x { - ScalarValue::$SCALAR_TY(i) => *i, - sv => panic!( - "Inconsistent types in ScalarValue::iter_to_array. \ - Expected {:?}, got {:?}", - data_type, sv - ), - }) - .collect::>>() - }), + ScalarValue::List(arr) => { + if arr.as_any().downcast_ref::().is_some() { + None + } else { + let list_arr = as_list_array(&arr); + let primitive_arr = + list_arr.values().as_primitive::<$ARRAY_TY>(); + Some( + primitive_arr.into_iter().collect::>>(), + ) + } + } sv => panic!( "Inconsistent types in ScalarValue::iter_to_array. \ Expected {:?}, got {:?}", @@ -1329,33 +1329,28 @@ impl ScalarValue { } macro_rules! build_array_list_string { - ($BUILDER:ident, $SCALAR_TY:ident) => {{ + ($BUILDER:ident, $STRING_ARRAY:ident) => {{ let mut builder = ListBuilder::new($BUILDER::new()); for scalar in scalars.into_iter() { match scalar { - ScalarValue::List(Some(xs), _) => { - for s in xs { - match s { - ScalarValue::$SCALAR_TY(Some(val)) => { - builder.values().append_value(val); - } - ScalarValue::$SCALAR_TY(None) => { - builder.values().append_null(); - } - sv => { - return _internal_err!( - "Inconsistent types in ScalarValue::iter_to_array. \ - Expected Utf8, got {:?}", - sv - ) - } + ScalarValue::List(arr) => { + if arr.as_any().downcast_ref::().is_some() { + builder.append(false); + continue; + } + + let list_arr = as_list_array(&arr); + let string_arr = $STRING_ARRAY(list_arr.values()); + + for v in string_arr.iter() { + if let Some(v) = v { + builder.values().append_value(v); + } else { + builder.values().append_null(); } } builder.append(true); } - ScalarValue::List(None, _) => { - builder.append(false); - } sv => { return _internal_err!( "Inconsistent types in ScalarValue::iter_to_array. \ @@ -1474,14 +1469,14 @@ impl ScalarValue { build_array_list_primitive!(Float64Type, Float64, f64) } DataType::List(fields) if fields.data_type() == &DataType::Utf8 => { - build_array_list_string!(StringBuilder, Utf8) + build_array_list_string!(StringBuilder, as_string_array) } DataType::List(fields) if fields.data_type() == &DataType::LargeUtf8 => { - build_array_list_string!(LargeStringBuilder, LargeUtf8) + build_array_list_string!(LargeStringBuilder, as_largestring_array) } DataType::List(_) => { // Fallback case handling homogeneous lists with any ScalarValue element type - let list_array = ScalarValue::iter_to_array_list(scalars, &data_type)?; + let list_array = ScalarValue::iter_to_array_list(scalars)?; Arc::new(list_array) } DataType::Struct(fields) => { @@ -1529,7 +1524,7 @@ impl ScalarValue { .collect::>>()?; let array = StructArray::from(field_values); - nullif(&array, &null_mask_builder.finish())? + arrow::compute::nullif(&array, &null_mask_builder.finish())? } DataType::Dictionary(key_type, value_type) => { // create the values array @@ -1654,42 +1649,66 @@ impl ScalarValue { Ok(array) } - fn iter_to_array_list( - scalars: impl IntoIterator, + /// This function does not contains nulls but empty array instead. + fn iter_to_array_list_without_nulls( + values: &[ScalarValue], data_type: &DataType, ) -> Result> { - let mut offsets = Int32Array::builder(0); - offsets.append_value(0); + let mut elements: Vec = vec![]; + let mut offsets = vec![]; - let mut elements: Vec = Vec::new(); - let mut valid = BooleanBufferBuilder::new(0); - let mut flat_len = 0i32; - for scalar in scalars { - if let ScalarValue::List(values, field) = scalar { - match values { - Some(values) => { - let element_array = if !values.is_empty() { - ScalarValue::iter_to_array(values)? - } else { - arrow::array::new_empty_array(field.data_type()) - }; + if values.is_empty() { + offsets.push(0); + } else { + let arr = ScalarValue::iter_to_array(values.to_vec())?; + offsets.push(arr.len()); + elements.push(arr); + } - // Add new offset index - flat_len += element_array.len() as i32; - offsets.append_value(flat_len); + // Concatenate element arrays to create single flat array + let flat_array = if elements.is_empty() { + new_empty_array(data_type) + } else { + let element_arrays: Vec<&dyn Array> = + elements.iter().map(|a| a.as_ref()).collect(); + arrow::compute::concat(&element_arrays)? + }; - elements.push(element_array); + let list_array = ListArray::new( + Arc::new(Field::new("item", flat_array.data_type().to_owned(), true)), + OffsetBuffer::::from_lengths(offsets), + flat_array, + None, + ); - // Element is valid - valid.append(true); - } - None => { - // Repeat previous offset index - offsets.append_value(flat_len); + Ok(list_array) + } - // Element is null - valid.append(false); - } + /// This function build with nulls with nulls buffer. + fn iter_to_array_list( + scalars: impl IntoIterator, + ) -> Result> { + let mut elements: Vec = vec![]; + let mut valid = BooleanBufferBuilder::new(0); + let mut offsets = vec![]; + + for scalar in scalars { + if let ScalarValue::List(arr) = scalar { + // i.e. NullArray(1) + if arr.as_any().downcast_ref::().is_some() { + // Repeat previous offset index + offsets.push(0); + + // Element is null + valid.append(false); + } else { + let list_arr = as_list_array(&arr); + let arr = list_arr.values().to_owned(); + offsets.push(arr.len()); + elements.push(arr); + + // Element is valid + valid.append(true); } } else { return _internal_err!( @@ -1701,20 +1720,21 @@ impl ScalarValue { // Concatenate element arrays to create single flat array let element_arrays: Vec<&dyn Array> = elements.iter().map(|a| a.as_ref()).collect(); + let flat_array = match arrow::compute::concat(&element_arrays) { Ok(flat_array) => flat_array, Err(err) => return Err(DataFusionError::ArrowError(err)), }; - // Build ListArray using ArrayData so we can specify a flat inner array, and offset indices - let offsets_array = offsets.finish(); - let array_data = ArrayDataBuilder::new(data_type.clone()) - .len(offsets_array.len() - 1) - .nulls(Some(NullBuffer::new(valid.finish()))) - .add_buffer(offsets_array.values().inner().clone()) - .add_child_data(flat_array.to_data()); + let buffer = valid.finish(); + + let list_array = ListArray::new( + Arc::new(Field::new("item", flat_array.data_type().clone(), true)), + OffsetBuffer::::from_lengths(offsets), + flat_array, + Some(NullBuffer::new(buffer)), + ); - let list_array = ListArray::from(array_data.build()?); Ok(list_array) } @@ -1751,6 +1771,80 @@ impl ScalarValue { .unwrap() } + /// Converts `Vec` to ListArray, simplified version of ScalarValue::to_array + /// + /// Example + /// ``` + /// use datafusion_common::ScalarValue; + /// use arrow::array::{ListArray, Int32Array}; + /// use arrow::datatypes::{DataType, Int32Type}; + /// use datafusion_common::cast::as_list_array; + /// + /// let scalars = vec![ + /// ScalarValue::Int32(Some(1)), + /// ScalarValue::Int32(None), + /// ScalarValue::Int32(Some(2)) + /// ]; + /// + /// let array = ScalarValue::new_list(&scalars, &DataType::Int32); + /// let result = as_list_array(&array).unwrap(); + /// + /// let expected = ListArray::from_iter_primitive::( + /// vec![ + /// Some(vec![Some(1), None, Some(2)]) + /// ]); + /// + /// assert_eq!(result, &expected); + /// ``` + pub fn new_list(values: &[ScalarValue], data_type: &DataType) -> ArrayRef { + Arc::new(match data_type { + DataType::Boolean => build_values_list!(BooleanBuilder, Boolean, values, 1), + DataType::Int8 => build_values_list!(Int8Builder, Int8, values, 1), + DataType::Int16 => build_values_list!(Int16Builder, Int16, values, 1), + DataType::Int32 => build_values_list!(Int32Builder, Int32, values, 1), + DataType::Int64 => build_values_list!(Int64Builder, Int64, values, 1), + DataType::UInt8 => build_values_list!(UInt8Builder, UInt8, values, 1), + DataType::UInt16 => build_values_list!(UInt16Builder, UInt16, values, 1), + DataType::UInt32 => build_values_list!(UInt32Builder, UInt32, values, 1), + DataType::UInt64 => build_values_list!(UInt64Builder, UInt64, values, 1), + DataType::Utf8 => build_values_list!(StringBuilder, Utf8, values, 1), + DataType::LargeUtf8 => { + build_values_list!(LargeStringBuilder, LargeUtf8, values, 1) + } + DataType::Float32 => build_values_list!(Float32Builder, Float32, values, 1), + DataType::Float64 => build_values_list!(Float64Builder, Float64, values, 1), + DataType::Timestamp(unit, tz) => { + let values = Some(values); + build_timestamp_list!(unit.clone(), tz.clone(), values, 1) + } + DataType::List(_) | DataType::Struct(_) => { + ScalarValue::iter_to_array_list_without_nulls(values, data_type).unwrap() + } + DataType::Decimal128(precision, scale) => { + let mut vals = vec![]; + for value in values.iter() { + if let ScalarValue::Decimal128(v, _, _) = value { + vals.push(v.to_owned()) + } + } + + let arr = Decimal128Array::from(vals) + .with_precision_and_scale(*precision, *scale) + .unwrap(); + wrap_into_list_array(Arc::new(arr)) + } + + DataType::Null => { + let arr = new_null_array(&DataType::Null, values.len()); + wrap_into_list_array(arr) + } + _ => panic!( + "Unsupported data type {:?} for ScalarValue::list_to_array", + data_type + ), + }) + } + /// Converts a scalar value into an array of `size` rows. pub fn to_array_of_size(&self, size: usize) -> ArrayRef { match self { @@ -1873,35 +1967,12 @@ impl ScalarValue { ScalarValue::Fixedsizelist(..) => { unimplemented!("FixedSizeList is not supported yet") } - ScalarValue::List(values, field) => Arc::new(match field.data_type() { - DataType::Boolean => build_list!(BooleanBuilder, Boolean, values, size), - DataType::Int8 => build_list!(Int8Builder, Int8, values, size), - DataType::Int16 => build_list!(Int16Builder, Int16, values, size), - DataType::Int32 => build_list!(Int32Builder, Int32, values, size), - DataType::Int64 => build_list!(Int64Builder, Int64, values, size), - DataType::UInt8 => build_list!(UInt8Builder, UInt8, values, size), - DataType::UInt16 => build_list!(UInt16Builder, UInt16, values, size), - DataType::UInt32 => build_list!(UInt32Builder, UInt32, values, size), - DataType::UInt64 => build_list!(UInt64Builder, UInt64, values, size), - DataType::Utf8 => build_list!(StringBuilder, Utf8, values, size), - DataType::Float32 => build_list!(Float32Builder, Float32, values, size), - DataType::Float64 => build_list!(Float64Builder, Float64, values, size), - DataType::Timestamp(unit, tz) => { - build_timestamp_list!(unit.clone(), tz.clone(), values, size) - } - &DataType::LargeUtf8 => { - build_list!(LargeStringBuilder, LargeUtf8, values, size) - } - _ => ScalarValue::iter_to_array_list( - repeat(self.clone()).take(size), - &DataType::List(Arc::new(Field::new( - "item", - field.data_type().clone(), - true, - ))), - ) - .unwrap(), - }), + ScalarValue::List(arr) => { + let arrays = std::iter::repeat(arr.as_ref()) + .take(size) + .collect::>(); + arrow::compute::concat(arrays.as_slice()).unwrap() + } ScalarValue::Date32(e) => { build_array_from_option!(Date32, Date32Array, e, size) } @@ -2057,6 +2128,71 @@ impl ScalarValue { } } + /// Retrieve ScalarValue for each row in `array` + /// + /// Example + /// ``` + /// use datafusion_common::ScalarValue; + /// use arrow::array::ListArray; + /// use arrow::datatypes::{DataType, Int32Type}; + /// + /// let list_arr = ListArray::from_iter_primitive::(vec![ + /// Some(vec![Some(1), Some(2), Some(3)]), + /// None, + /// Some(vec![Some(4), Some(5)]) + /// ]); + /// + /// let scalar_vec = ScalarValue::convert_array_to_scalar_vec(&list_arr).unwrap(); + /// + /// let expected = vec![ + /// vec![ + /// ScalarValue::Int32(Some(1)), + /// ScalarValue::Int32(Some(2)), + /// ScalarValue::Int32(Some(3)), + /// ], + /// vec![], + /// vec![ScalarValue::Int32(Some(4)), ScalarValue::Int32(Some(5))] + /// ]; + /// + /// assert_eq!(scalar_vec, expected); + /// ``` + pub fn convert_array_to_scalar_vec(array: &dyn Array) -> Result>> { + let mut scalars = Vec::with_capacity(array.len()); + + for index in 0..array.len() { + let scalar_values = match array.data_type() { + DataType::List(_) => { + let list_array = as_list_array(array); + match list_array.is_null(index) { + true => Vec::new(), + false => { + let nested_array = list_array.value(index); + ScalarValue::convert_array_to_scalar_vec(&nested_array)? + .into_iter() + .flatten() + .collect() + } + } + } + _ => { + let scalar = ScalarValue::try_from_array(array, index)?; + vec![scalar] + } + }; + scalars.push(scalar_values); + } + Ok(scalars) + } + + // TODO: Support more types after other ScalarValue is wrapped with ArrayRef + /// Get raw data (inner array) inside ScalarValue + pub fn raw_data(&self) -> Result { + match self { + ScalarValue::List(arr) => Ok(arr.to_owned()), + _ => _internal_err!("ScalarValue is not a list"), + } + } + /// Converts a value in `array` at `index` into a ScalarValue pub fn try_from_array(array: &dyn Array, index: usize) -> Result { // handle NULL value @@ -2094,18 +2230,29 @@ impl ScalarValue { DataType::Utf8 => typed_cast!(array, index, StringArray, Utf8), DataType::LargeUtf8 => typed_cast!(array, index, LargeStringArray, LargeUtf8), DataType::List(nested_type) => { - let list_array = as_list_array(array)?; - let value = match list_array.is_null(index) { - true => None, + let list_array = as_list_array(array); + let arr = match list_array.is_null(index) { + true => new_null_array(nested_type.data_type(), 0), + false => { + let nested_array = list_array.value(index); + Arc::new(wrap_into_list_array(nested_array)) + } + }; + + ScalarValue::List(arr) + } + // TODO: There is no test for FixedSizeList now, add it later + DataType::FixedSizeList(nested_type, _len) => { + let list_array = as_fixed_size_list_array(array)?; + let arr = match list_array.is_null(index) { + true => new_null_array(nested_type.data_type(), 0), false => { let nested_array = list_array.value(index); - let scalar_vec = (0..nested_array.len()) - .map(|i| ScalarValue::try_from_array(&nested_array, i)) - .collect::>>()?; - Some(scalar_vec) + Arc::new(wrap_into_list_array(nested_array)) } }; - ScalarValue::new_list(value, nested_type.data_type().clone()) + + ScalarValue::List(arr) } DataType::Date32 => { typed_cast!(array, index, Date32Array, Date32) @@ -2194,20 +2341,6 @@ impl ScalarValue { } Self::Struct(Some(field_values), fields.clone()) } - DataType::FixedSizeList(nested_type, _len) => { - let list_array = as_fixed_size_list_array(array)?; - let value = match list_array.is_null(index) { - true => None, - false => { - let nested_array = list_array.value(index); - let scalar_vec = (0..nested_array.len()) - .map(|i| ScalarValue::try_from_array(&nested_array, i)) - .collect::>>()?; - Some(scalar_vec) - } - }; - ScalarValue::new_list(value, nested_type.data_type().clone()) - } DataType::FixedSizeBinary(_) => { let array = as_fixed_size_binary_array(array)?; let size = match array.data_type() { @@ -2383,7 +2516,7 @@ impl ScalarValue { eq_array_primitive!(array, index, LargeBinaryArray, val) } ScalarValue::Fixedsizelist(..) => unimplemented!(), - ScalarValue::List(_, _) => unimplemented!(), + ScalarValue::List(_) => unimplemented!("ListArr"), ScalarValue::Date32(val) => { eq_array_primitive!(array, index, Date32Array, val) } @@ -2504,14 +2637,14 @@ impl ScalarValue { | ScalarValue::LargeBinary(b) => { b.as_ref().map(|b| b.capacity()).unwrap_or_default() } - ScalarValue::Fixedsizelist(vals, field, _) - | ScalarValue::List(vals, field) => { + ScalarValue::Fixedsizelist(vals, field, _) => { vals.as_ref() .map(|vals| Self::size_of_vec(vals) - std::mem::size_of_val(vals)) .unwrap_or_default() // `field` is boxed, so it is NOT already included in `self` + field.size() } + ScalarValue::List(arr) => arr.get_array_memory_size(), ScalarValue::Struct(vals, fields) => { vals.as_ref() .map(|vals| { @@ -2735,8 +2868,8 @@ impl TryFrom<&DataType> for ScalarValue { type Error = DataFusionError; /// Create a Null instance of ScalarValue for this datatype - fn try_from(datatype: &DataType) -> Result { - Ok(match datatype { + fn try_from(data_type: &DataType) -> Result { + Ok(match data_type { DataType::Boolean => ScalarValue::Boolean(None), DataType::Float64 => ScalarValue::Float64(None), DataType::Float32 => ScalarValue::Float32(None), @@ -2806,14 +2939,13 @@ impl TryFrom<&DataType> for ScalarValue { index_type.clone(), Box::new(value_type.as_ref().try_into()?), ), - DataType::List(ref nested_type) => { - ScalarValue::new_list(None, nested_type.data_type().clone()) - } + DataType::List(_) => ScalarValue::List(new_null_array(&DataType::Null, 0)), + DataType::Struct(fields) => ScalarValue::Struct(None, fields.clone()), DataType::Null => ScalarValue::Null, _ => { return _not_impl_err!( - "Can't create a scalar from data_type \"{datatype:?}\"" + "Can't create a scalar from data_type \"{data_type:?}\"" ); } }) @@ -2868,7 +3000,7 @@ impl fmt::Display for ScalarValue { )?, None => write!(f, "NULL")?, }, - ScalarValue::Fixedsizelist(e, ..) | ScalarValue::List(e, _) => match e { + ScalarValue::Fixedsizelist(e, ..) => match e { Some(l) => write!( f, "{}", @@ -2879,6 +3011,12 @@ impl fmt::Display for ScalarValue { )?, None => write!(f, "NULL")?, }, + ScalarValue::List(arr) => write!( + f, + "{}", + arrow::util::pretty::pretty_format_columns("col", &[arr.to_owned()]) + .unwrap() + )?, ScalarValue::Date32(e) => format_option!(f, e)?, ScalarValue::Date64(e) => format_option!(f, e)?, ScalarValue::Time32Second(e) => format_option!(f, e)?, @@ -2954,7 +3092,7 @@ impl fmt::Debug for ScalarValue { ScalarValue::LargeBinary(None) => write!(f, "LargeBinary({self})"), ScalarValue::LargeBinary(Some(_)) => write!(f, "LargeBinary(\"{self}\")"), ScalarValue::Fixedsizelist(..) => write!(f, "FixedSizeList([{self}])"), - ScalarValue::List(_, _) => write!(f, "List([{self}])"), + ScalarValue::List(arr) => write!(f, "List([{arr:?}])"), ScalarValue::Date32(_) => write!(f, "Date32(\"{self}\")"), ScalarValue::Date64(_) => write!(f, "Date64(\"{self}\")"), ScalarValue::Time32Second(_) => write!(f, "Time32Second(\"{self}\")"), @@ -3060,6 +3198,104 @@ mod tests { use super::*; + #[test] + fn test_to_array_of_size_for_list() { + let arr = ListArray::from_iter_primitive::(vec![Some(vec![ + Some(1), + None, + Some(2), + ])]); + + let sv = ScalarValue::List(Arc::new(arr)); + let actual_arr = sv.to_array_of_size(2); + let actual_list_arr = as_list_array(&actual_arr); + + let arr = ListArray::from_iter_primitive::(vec![ + Some(vec![Some(1), None, Some(2)]), + Some(vec![Some(1), None, Some(2)]), + ]); + + assert_eq!(&arr, actual_list_arr); + } + + #[test] + fn test_list_to_array_string() { + let scalars = vec![ + ScalarValue::Utf8(Some(String::from("rust"))), + ScalarValue::Utf8(Some(String::from("arrow"))), + ScalarValue::Utf8(Some(String::from("data-fusion"))), + ]; + + let array = ScalarValue::new_list(scalars.as_slice(), &DataType::Utf8); + + let expected = wrap_into_list_array(Arc::new(StringArray::from(vec![ + "rust", + "arrow", + "data-fusion", + ]))); + let result = as_list_array(&array); + assert_eq!(result, &expected); + } + + #[test] + fn iter_to_array_primitive_test() { + let scalars = vec![ + ScalarValue::List(Arc::new( + ListArray::from_iter_primitive::(vec![Some(vec![ + Some(1), + Some(2), + Some(3), + ])]), + )), + ScalarValue::List(Arc::new( + ListArray::from_iter_primitive::(vec![Some(vec![ + Some(4), + Some(5), + ])]), + )), + ]; + + let array = ScalarValue::iter_to_array(scalars).unwrap(); + let list_array = as_list_array(&array); + let expected = ListArray::from_iter_primitive::(vec![ + Some(vec![Some(1), Some(2), Some(3)]), + Some(vec![Some(4), Some(5)]), + ]); + assert_eq!(list_array, &expected); + } + + #[test] + fn iter_to_array_string_test() { + let arr1 = + wrap_into_list_array(Arc::new(StringArray::from(vec!["foo", "bar", "baz"]))); + let arr2 = + wrap_into_list_array(Arc::new(StringArray::from(vec!["rust", "world"]))); + + let scalars = vec![ + ScalarValue::List(Arc::new(arr1)), + ScalarValue::List(Arc::new(arr2)), + ]; + + let array = ScalarValue::iter_to_array(scalars).unwrap(); + let result = as_list_array(&array); + + // build expected array + let string_builder = StringBuilder::with_capacity(5, 25); + let mut list_of_string_builder = ListBuilder::new(string_builder); + + list_of_string_builder.values().append_value("foo"); + list_of_string_builder.values().append_value("bar"); + list_of_string_builder.values().append_value("baz"); + list_of_string_builder.append(true); + + list_of_string_builder.values().append_value("rust"); + list_of_string_builder.values().append_value("world"); + list_of_string_builder.append(true); + let expected = list_of_string_builder.finish(); + + assert_eq!(result, &expected); + } + #[test] fn scalar_add_trait_test() -> Result<()> { let float_value = ScalarValue::Float64(Some(123.)); @@ -3304,6 +3540,81 @@ mod tests { Ok(()) } + #[test] + fn test_list_partial_cmp() { + let a = + ScalarValue::List(Arc::new( + ListArray::from_iter_primitive::(vec![Some(vec![ + Some(1), + Some(2), + Some(3), + ])]), + )); + let b = + ScalarValue::List(Arc::new( + ListArray::from_iter_primitive::(vec![Some(vec![ + Some(1), + Some(2), + Some(3), + ])]), + )); + assert_eq!(a.partial_cmp(&b), Some(Ordering::Equal)); + + let a = + ScalarValue::List(Arc::new( + ListArray::from_iter_primitive::(vec![Some(vec![ + Some(10), + Some(2), + Some(3), + ])]), + )); + let b = + ScalarValue::List(Arc::new( + ListArray::from_iter_primitive::(vec![Some(vec![ + Some(1), + Some(2), + Some(30), + ])]), + )); + assert_eq!(a.partial_cmp(&b), Some(Ordering::Greater)); + + let a = + ScalarValue::List(Arc::new( + ListArray::from_iter_primitive::(vec![Some(vec![ + Some(10), + Some(2), + Some(3), + ])]), + )); + let b = + ScalarValue::List(Arc::new( + ListArray::from_iter_primitive::(vec![Some(vec![ + Some(10), + Some(2), + Some(30), + ])]), + )); + assert_eq!(a.partial_cmp(&b), Some(Ordering::Less)); + + let a = + ScalarValue::List(Arc::new( + ListArray::from_iter_primitive::(vec![ + Some(vec![Some(10), Some(2), Some(3)]), + None, + Some(vec![Some(10), Some(2), Some(3)]), + ]), + )); + let b = + ScalarValue::List(Arc::new( + ListArray::from_iter_primitive::(vec![ + Some(vec![Some(10), Some(2), Some(3)]), + None, + Some(vec![Some(10), Some(2), Some(3)]), + ]), + )); + assert_eq!(a.partial_cmp(&b), Some(Ordering::Equal)); + } + #[test] fn scalar_value_to_array_u64() -> Result<()> { let value = ScalarValue::UInt64(Some(13u64)); @@ -3340,31 +3651,22 @@ mod tests { #[test] fn scalar_list_null_to_array() { - let list_array_ref = ScalarValue::List( - None, - Arc::new(Field::new("item", DataType::UInt64, false)), - ) - .to_array(); - let list_array = as_list_array(&list_array_ref).unwrap(); + let list_array_ref = ScalarValue::new_list(&[], &DataType::UInt64); + let list_array = as_list_array(&list_array_ref); - assert!(list_array.is_null(0)); assert_eq!(list_array.len(), 1); assert_eq!(list_array.values().len(), 0); } #[test] fn scalar_list_to_array() -> Result<()> { - let list_array_ref = ScalarValue::List( - Some(vec![ - ScalarValue::UInt64(Some(100)), - ScalarValue::UInt64(None), - ScalarValue::UInt64(Some(101)), - ]), - Arc::new(Field::new("item", DataType::UInt64, false)), - ) - .to_array(); - - let list_array = as_list_array(&list_array_ref)?; + let values = vec![ + ScalarValue::UInt64(Some(100)), + ScalarValue::UInt64(None), + ScalarValue::UInt64(Some(101)), + ]; + let list_array_ref = ScalarValue::new_list(&values, &DataType::UInt64); + let list_array = as_list_array(&list_array_ref); assert_eq!(list_array.len(), 1); assert_eq!(list_array.values().len(), 3); @@ -3864,55 +4166,6 @@ mod tests { assert_eq!(Int64(Some(33)).partial_cmp(&Int32(Some(33))), None); assert_eq!(Int32(Some(33)).partial_cmp(&Int64(Some(33))), None); - assert_eq!( - List( - Some(vec![Int32(Some(1)), Int32(Some(5))]), - Arc::new(Field::new("item", DataType::Int32, false)), - ) - .partial_cmp(&List( - Some(vec![Int32(Some(1)), Int32(Some(5))]), - Arc::new(Field::new("item", DataType::Int32, false)), - )), - Some(Ordering::Equal) - ); - - assert_eq!( - List( - Some(vec![Int32(Some(10)), Int32(Some(5))]), - Arc::new(Field::new("item", DataType::Int32, false)), - ) - .partial_cmp(&List( - Some(vec![Int32(Some(1)), Int32(Some(5))]), - Arc::new(Field::new("item", DataType::Int32, false)), - )), - Some(Ordering::Greater) - ); - - assert_eq!( - List( - Some(vec![Int32(Some(1)), Int32(Some(5))]), - Arc::new(Field::new("item", DataType::Int32, false)), - ) - .partial_cmp(&List( - Some(vec![Int32(Some(10)), Int32(Some(5))]), - Arc::new(Field::new("item", DataType::Int32, false)), - )), - Some(Ordering::Less) - ); - - // For different data type, `partial_cmp` returns None. - assert_eq!( - List( - Some(vec![Int64(Some(1)), Int64(Some(5))]), - Arc::new(Field::new("item", DataType::Int64, false)), - ) - .partial_cmp(&List( - Some(vec![Int32(Some(1)), Int32(Some(5))]), - Arc::new(Field::new("item", DataType::Int32, false)), - )), - None - ); - assert_eq!( ScalarValue::from(vec![ ("A", ScalarValue::from(1.0)), @@ -4125,24 +4378,26 @@ mod tests { )); // Define primitive list scalars - let l0 = ScalarValue::List( - Some(vec![ - ScalarValue::from(1i32), - ScalarValue::from(2i32), - ScalarValue::from(3i32), - ]), - Arc::new(Field::new("item", DataType::Int32, false)), - ); - - let l1 = ScalarValue::List( - Some(vec![ScalarValue::from(4i32), ScalarValue::from(5i32)]), - Arc::new(Field::new("item", DataType::Int32, false)), - ); - - let l2 = ScalarValue::List( - Some(vec![ScalarValue::from(6i32)]), - Arc::new(Field::new("item", DataType::Int32, false)), - ); + let l0 = + ScalarValue::List(Arc::new( + ListArray::from_iter_primitive::(vec![Some(vec![ + Some(1), + Some(2), + Some(3), + ])]), + )); + let l1 = + ScalarValue::List(Arc::new( + ListArray::from_iter_primitive::(vec![Some(vec![ + Some(4), + Some(5), + ])]), + )); + let l2 = ScalarValue::List(Arc::new(ListArray::from_iter_primitive::< + Int32Type, + _, + _, + >(vec![Some(vec![Some(6)])]))); // Define struct scalars let s0 = ScalarValue::from(vec![ @@ -4182,15 +4437,19 @@ mod tests { assert_eq!(array, &expected); // Define list-of-structs scalars - let nl0 = - ScalarValue::new_list(Some(vec![s0.clone(), s1.clone()]), s0.data_type()); - let nl1 = ScalarValue::new_list(Some(vec![s2]), s0.data_type()); + let nl0_array = ScalarValue::iter_to_array(vec![s0.clone(), s1.clone()]).unwrap(); + let nl0 = ScalarValue::List(Arc::new(wrap_into_list_array(nl0_array))); + + let nl1_array = ScalarValue::iter_to_array(vec![s2.clone()]).unwrap(); + let nl1 = ScalarValue::List(Arc::new(wrap_into_list_array(nl1_array))); + + let nl2_array = ScalarValue::iter_to_array(vec![s1.clone()]).unwrap(); + let nl2 = ScalarValue::List(Arc::new(wrap_into_list_array(nl2_array))); - let nl2 = ScalarValue::new_list(Some(vec![s1]), s0.data_type()); // iter_to_array for list-of-struct let array = ScalarValue::iter_to_array(vec![nl0, nl1, nl2]).unwrap(); - let array = as_list_array(&array).unwrap(); + let array = as_list_array(&array); // Construct expected array with array builders let field_a_builder = StringBuilder::with_capacity(4, 1024); @@ -4313,48 +4572,63 @@ mod tests { #[test] fn test_nested_lists() { // Define inner list scalars - let l1 = ScalarValue::new_list( - Some(vec![ - ScalarValue::new_list( - Some(vec![ - ScalarValue::from(1i32), - ScalarValue::from(2i32), - ScalarValue::from(3i32), - ]), - DataType::Int32, - ), - ScalarValue::new_list( - Some(vec![ScalarValue::from(4i32), ScalarValue::from(5i32)]), - DataType::Int32, - ), - ]), - DataType::List(Arc::new(Field::new("item", DataType::Int32, true))), + let a1 = ListArray::from_iter_primitive::(vec![Some(vec![ + Some(1), + Some(2), + Some(3), + ])]); + let a2 = ListArray::from_iter_primitive::(vec![Some(vec![ + Some(4), + Some(5), + ])]); + let l1 = ListArray::new( + Arc::new(Field::new( + "item", + DataType::List(Arc::new(Field::new("item", DataType::Int32, true))), + true, + )), + OffsetBuffer::::from_lengths([1, 1]), + arrow::compute::concat(&[&a1, &a2]).unwrap(), + None, ); - let l2 = ScalarValue::new_list( - Some(vec![ - ScalarValue::new_list( - Some(vec![ScalarValue::from(6i32)]), - DataType::Int32, - ), - ScalarValue::new_list( - Some(vec![ScalarValue::from(7i32), ScalarValue::from(8i32)]), - DataType::Int32, - ), - ]), - DataType::List(Arc::new(Field::new("item", DataType::Int32, true))), + let a1 = + ListArray::from_iter_primitive::(vec![Some(vec![Some(6)])]); + let a2 = ListArray::from_iter_primitive::(vec![Some(vec![ + Some(7), + Some(8), + ])]); + let l2 = ListArray::new( + Arc::new(Field::new( + "item", + DataType::List(Arc::new(Field::new("item", DataType::Int32, true))), + true, + )), + OffsetBuffer::::from_lengths([1, 1]), + arrow::compute::concat(&[&a1, &a2]).unwrap(), + None, ); - let l3 = ScalarValue::new_list( - Some(vec![ScalarValue::new_list( - Some(vec![ScalarValue::from(9i32)]), - DataType::Int32, - )]), - DataType::List(Arc::new(Field::new("item", DataType::Int32, true))), + let a1 = + ListArray::from_iter_primitive::(vec![Some(vec![Some(9)])]); + let l3 = ListArray::new( + Arc::new(Field::new( + "item", + DataType::List(Arc::new(Field::new("item", DataType::Int32, true))), + true, + )), + OffsetBuffer::::from_lengths([1]), + arrow::compute::concat(&[&a1]).unwrap(), + None, ); - let array = ScalarValue::iter_to_array(vec![l1, l2, l3]).unwrap(); - let array = as_list_array(&array).unwrap(); + let array = ScalarValue::iter_to_array(vec![ + ScalarValue::List(Arc::new(l1)), + ScalarValue::List(Arc::new(l2)), + ScalarValue::List(Arc::new(l3)), + ]) + .unwrap(); + let array = as_list_array(&array); // Construct expected array with array builders let inner_builder = Int32Array::builder(8); @@ -4904,12 +5178,11 @@ mod tests { #[test] fn test_build_timestamp_millisecond_list() { let values = vec![ScalarValue::TimestampMillisecond(Some(1), None)]; - let ts_list = ScalarValue::new_list( - Some(values), - DataType::Timestamp(TimeUnit::Millisecond, None), + let arr = ScalarValue::new_list( + &values, + &DataType::Timestamp(TimeUnit::Millisecond, None), ); - let list = ts_list.to_array_of_size(1); - assert_eq!(1, list.len()); + assert_eq!(1, arr.len()); } fn get_random_timestamps(sample_size: u64) -> Vec { diff --git a/datafusion/common/src/utils.rs b/datafusion/common/src/utils.rs index b7c80aa9ac44..b2f71e86f21e 100644 --- a/datafusion/common/src/utils.rs +++ b/datafusion/common/src/utils.rs @@ -19,10 +19,12 @@ use crate::{DataFusionError, Result, ScalarValue}; use arrow::array::{ArrayRef, PrimitiveArray}; +use arrow::buffer::OffsetBuffer; use arrow::compute; use arrow::compute::{partition, SortColumn, SortOptions}; -use arrow::datatypes::{SchemaRef, UInt32Type}; +use arrow::datatypes::{Field, SchemaRef, UInt32Type}; use arrow::record_batch::RecordBatch; +use arrow_array::ListArray; use sqlparser::ast::Ident; use sqlparser::dialect::GenericDialect; use sqlparser::parser::Parser; @@ -334,6 +336,18 @@ pub fn longest_consecutive_prefix>( count } +/// Wrap an array into a single element `ListArray`. +/// For example `[1, 2, 3]` would be converted into `[[1, 2, 3]]` +pub fn wrap_into_list_array(arr: ArrayRef) -> ListArray { + let offsets = OffsetBuffer::from_lengths([arr.len()]); + ListArray::new( + Arc::new(Field::new("item", arr.data_type().to_owned(), true)), + offsets, + arr, + None, + ) +} + /// An extension trait for smart pointers. Provides an interface to get a /// raw pointer to the data (with metadata stripped away). /// diff --git a/datafusion/core/tests/sql/aggregates.rs b/datafusion/core/tests/sql/aggregates.rs index 63d5e58090eb..03864e9efef8 100644 --- a/datafusion/core/tests/sql/aggregates.rs +++ b/datafusion/core/tests/sql/aggregates.rs @@ -47,25 +47,23 @@ async fn csv_query_array_agg_distinct() -> Result<()> { let column = actual[0].column(0); assert_eq!(column.len(), 1); - if let ScalarValue::List(Some(mut v), _) = ScalarValue::try_from_array(column, 0)? { - // workaround lack of Ord of ScalarValue - let cmp = |a: &ScalarValue, b: &ScalarValue| { - a.partial_cmp(b).expect("Can compare ScalarValues") - }; - v.sort_by(cmp); - assert_eq!( - *v, - vec![ - ScalarValue::UInt32(Some(1)), - ScalarValue::UInt32(Some(2)), - ScalarValue::UInt32(Some(3)), - ScalarValue::UInt32(Some(4)), - ScalarValue::UInt32(Some(5)) - ] - ); - } else { - unreachable!(); - } + let scalar_vec = ScalarValue::convert_array_to_scalar_vec(&column)?; + let mut scalars = scalar_vec[0].clone(); + // workaround lack of Ord of ScalarValue + let cmp = |a: &ScalarValue, b: &ScalarValue| { + a.partial_cmp(b).expect("Can compare ScalarValues") + }; + scalars.sort_by(cmp); + assert_eq!( + scalars, + vec![ + ScalarValue::UInt32(Some(1)), + ScalarValue::UInt32(Some(2)), + ScalarValue::UInt32(Some(3)), + ScalarValue::UInt32(Some(4)), + ScalarValue::UInt32(Some(5)) + ] + ); Ok(()) } diff --git a/datafusion/optimizer/src/simplify_expressions/expr_simplifier.rs b/datafusion/optimizer/src/simplify_expressions/expr_simplifier.rs index f5a6860299ab..cb3f13a51ec4 100644 --- a/datafusion/optimizer/src/simplify_expressions/expr_simplifier.rs +++ b/datafusion/optimizer/src/simplify_expressions/expr_simplifier.rs @@ -30,7 +30,10 @@ use arrow::{ error::ArrowError, record_batch::RecordBatch, }; -use datafusion_common::tree_node::{RewriteRecursion, TreeNode, TreeNodeRewriter}; +use datafusion_common::{ + cast::{as_large_list_array, as_list_array}, + tree_node::{RewriteRecursion, TreeNode, TreeNodeRewriter}, +}; use datafusion_common::{ exec_err, internal_err, DFSchema, DFSchemaRef, DataFusionError, Result, ScalarValue, }; @@ -392,8 +395,11 @@ impl<'a> ConstEvaluator<'a> { "Could not evaluate the expression, found a result of length {}", a.len() ) + } else if as_list_array(&a).is_ok() || as_large_list_array(&a).is_ok() { + Ok(ScalarValue::List(a)) } else { - Ok(ScalarValue::try_from_array(&a, 0)?) + // Non-ListArray + ScalarValue::try_from_array(&a, 0) } } ColumnarValue::Scalar(s) => Ok(s), diff --git a/datafusion/physical-expr/src/aggregate/array_agg.rs b/datafusion/physical-expr/src/aggregate/array_agg.rs index 0cf39888f133..834925b8d554 100644 --- a/datafusion/physical-expr/src/aggregate/array_agg.rs +++ b/datafusion/physical-expr/src/aggregate/array_agg.rs @@ -22,8 +22,11 @@ use crate::expressions::format_state_name; use crate::{AggregateExpr, PhysicalExpr}; use arrow::array::ArrayRef; use arrow::datatypes::{DataType, Field}; +use arrow_array::Array; +use datafusion_common::cast::as_list_array; +use datafusion_common::utils::wrap_into_list_array; +use datafusion_common::Result; use datafusion_common::ScalarValue; -use datafusion_common::{internal_err, DataFusionError, Result}; use datafusion_expr::Accumulator; use std::any::Any; use std::sync::Arc; @@ -102,7 +105,7 @@ impl PartialEq for ArrayAgg { #[derive(Debug)] pub(crate) struct ArrayAggAccumulator { - values: Vec, + values: Vec, datatype: DataType, } @@ -117,34 +120,29 @@ impl ArrayAggAccumulator { } impl Accumulator for ArrayAggAccumulator { + // Append value like Int64Array(1,2,3) fn update_batch(&mut self, values: &[ArrayRef]) -> Result<()> { if values.is_empty() { return Ok(()); } assert!(values.len() == 1, "array_agg can only take 1 param!"); - let arr = &values[0]; - (0..arr.len()).try_for_each(|index| { - let scalar = ScalarValue::try_from_array(arr, index)?; - self.values.push(scalar); - Ok(()) - }) + let val = values[0].clone(); + self.values.push(val); + Ok(()) } + // Append value like ListArray(Int64Array(1,2,3), Int64Array(4,5,6)) fn merge_batch(&mut self, states: &[ArrayRef]) -> Result<()> { if states.is_empty() { return Ok(()); } assert!(states.len() == 1, "array_agg states must be singleton!"); - let arr = &states[0]; - (0..arr.len()).try_for_each(|index| { - let scalar = ScalarValue::try_from_array(arr, index)?; - if let ScalarValue::List(Some(values), _) = scalar { - self.values.extend(values); - Ok(()) - } else { - internal_err!("array_agg state must be list!") - } - }) + + let list_arr = as_list_array(&states[0])?; + for arr in list_arr.iter().flatten() { + self.values.push(arr); + } + Ok(()) } fn state(&self) -> Result> { @@ -152,15 +150,30 @@ impl Accumulator for ArrayAggAccumulator { } fn evaluate(&self) -> Result { - Ok(ScalarValue::new_list( - Some(self.values.clone()), - self.datatype.clone(), - )) + // Transform Vec to ListArr + + let element_arrays: Vec<&dyn Array> = + self.values.iter().map(|a| a.as_ref()).collect(); + + if element_arrays.is_empty() { + let arr = ScalarValue::new_list(&[], &self.datatype); + return Ok(ScalarValue::List(arr)); + } + + let concated_array = arrow::compute::concat(&element_arrays)?; + let list_array = wrap_into_list_array(concated_array); + + Ok(ScalarValue::List(Arc::new(list_array))) } fn size(&self) -> usize { - std::mem::size_of_val(self) + ScalarValue::size_of_vec(&self.values) - - std::mem::size_of_val(&self.values) + std::mem::size_of_val(self) + + (std::mem::size_of::() * self.values.capacity()) + + self + .values + .iter() + .map(|arr| arr.get_array_memory_size()) + .sum::() + self.datatype.size() - std::mem::size_of_val(&self.datatype) } @@ -176,72 +189,78 @@ mod tests { use arrow::array::Int32Array; use arrow::datatypes::*; use arrow::record_batch::RecordBatch; + use arrow_array::Array; + use arrow_array::ListArray; + use arrow_buffer::OffsetBuffer; + use datafusion_common::DataFusionError; use datafusion_common::Result; #[test] fn array_agg_i32() -> Result<()> { let a: ArrayRef = Arc::new(Int32Array::from(vec![1, 2, 3, 4, 5])); - let list = ScalarValue::new_list( - Some(vec![ - ScalarValue::Int32(Some(1)), - ScalarValue::Int32(Some(2)), - ScalarValue::Int32(Some(3)), - ScalarValue::Int32(Some(4)), - ScalarValue::Int32(Some(5)), - ]), - DataType::Int32, - ); + let list = ListArray::from_iter_primitive::(vec![Some(vec![ + Some(1), + Some(2), + Some(3), + Some(4), + Some(5), + ])]); + let list = ScalarValue::List(Arc::new(list)); generic_test_op!(a, DataType::Int32, ArrayAgg, list, DataType::Int32) } #[test] fn array_agg_nested() -> Result<()> { - let l1 = ScalarValue::new_list( - Some(vec![ - ScalarValue::new_list( - Some(vec![ - ScalarValue::from(1i32), - ScalarValue::from(2i32), - ScalarValue::from(3i32), - ]), - DataType::Int32, - ), - ScalarValue::new_list( - Some(vec![ScalarValue::from(4i32), ScalarValue::from(5i32)]), - DataType::Int32, - ), - ]), - DataType::List(Arc::new(Field::new("item", DataType::Int32, true))), + let a1 = ListArray::from_iter_primitive::(vec![Some(vec![ + Some(1), + Some(2), + Some(3), + ])]); + let a2 = ListArray::from_iter_primitive::(vec![Some(vec![ + Some(4), + Some(5), + ])]); + let l1 = ListArray::new( + Arc::new(Field::new("item", a1.data_type().to_owned(), true)), + OffsetBuffer::from_lengths([a1.len() + a2.len()]), + arrow::compute::concat(&[&a1, &a2])?, + None, ); - let l2 = ScalarValue::new_list( - Some(vec![ - ScalarValue::new_list( - Some(vec![ScalarValue::from(6i32)]), - DataType::Int32, - ), - ScalarValue::new_list( - Some(vec![ScalarValue::from(7i32), ScalarValue::from(8i32)]), - DataType::Int32, - ), - ]), - DataType::List(Arc::new(Field::new("item", DataType::Int32, true))), + let a1 = + ListArray::from_iter_primitive::(vec![Some(vec![Some(6)])]); + let a2 = ListArray::from_iter_primitive::(vec![Some(vec![ + Some(7), + Some(8), + ])]); + let l2 = ListArray::new( + Arc::new(Field::new("item", a1.data_type().to_owned(), true)), + OffsetBuffer::from_lengths([a1.len() + a2.len()]), + arrow::compute::concat(&[&a1, &a2])?, + None, ); - let l3 = ScalarValue::new_list( - Some(vec![ScalarValue::new_list( - Some(vec![ScalarValue::from(9i32)]), - DataType::Int32, - )]), - DataType::List(Arc::new(Field::new("item", DataType::Int32, true))), + let a1 = + ListArray::from_iter_primitive::(vec![Some(vec![Some(9)])]); + let l3 = ListArray::new( + Arc::new(Field::new("item", a1.data_type().to_owned(), true)), + OffsetBuffer::from_lengths([a1.len()]), + arrow::compute::concat(&[&a1])?, + None, ); - let list = ScalarValue::new_list( - Some(vec![l1.clone(), l2.clone(), l3.clone()]), - DataType::List(Arc::new(Field::new("item", DataType::Int32, true))), + let list = ListArray::new( + Arc::new(Field::new("item", l1.data_type().to_owned(), true)), + OffsetBuffer::from_lengths([l1.len() + l2.len() + l3.len()]), + arrow::compute::concat(&[&l1, &l2, &l3])?, + None, ); + let list = ScalarValue::List(Arc::new(list)); + let l1 = ScalarValue::List(Arc::new(l1)); + let l2 = ScalarValue::List(Arc::new(l2)); + let l3 = ScalarValue::List(Arc::new(l3)); let array = ScalarValue::iter_to_array(vec![l1, l2, l3]).unwrap(); diff --git a/datafusion/physical-expr/src/aggregate/array_agg_distinct.rs b/datafusion/physical-expr/src/aggregate/array_agg_distinct.rs index 422eecd20155..21143ce54a20 100644 --- a/datafusion/physical-expr/src/aggregate/array_agg_distinct.rs +++ b/datafusion/physical-expr/src/aggregate/array_agg_distinct.rs @@ -22,13 +22,13 @@ use std::any::Any; use std::fmt::Debug; use std::sync::Arc; -use arrow::array::{Array, ArrayRef}; +use arrow::array::ArrayRef; use std::collections::HashSet; use crate::aggregate::utils::down_cast_any_ref; use crate::expressions::format_state_name; use crate::{AggregateExpr, PhysicalExpr}; -use datafusion_common::{internal_err, DataFusionError, Result, ScalarValue}; +use datafusion_common::{Result, ScalarValue}; use datafusion_expr::Accumulator; /// Expression for a ARRAY_AGG(DISTINCT) aggregation. @@ -125,22 +125,18 @@ impl DistinctArrayAggAccumulator { impl Accumulator for DistinctArrayAggAccumulator { fn state(&self) -> Result> { - Ok(vec![ScalarValue::new_list( - Some(self.values.clone().into_iter().collect()), - self.datatype.clone(), - )]) + Ok(vec![self.evaluate()?]) } fn update_batch(&mut self, values: &[ArrayRef]) -> Result<()> { assert_eq!(values.len(), 1, "batch input should only include 1 column!"); let array = &values[0]; - (0..array.len()).try_for_each(|i| { - if !array.is_null(i) { - self.values.insert(ScalarValue::try_from_array(array, i)?); - } - Ok(()) - }) + let scalars = ScalarValue::convert_array_to_scalar_vec(array)?; + for scalar in scalars { + self.values.extend(scalar) + } + Ok(()) } fn merge_batch(&mut self, states: &[ArrayRef]) -> Result<()> { @@ -154,25 +150,18 @@ impl Accumulator for DistinctArrayAggAccumulator { "array_agg_distinct states must contain single array" ); - let array = &states[0]; - (0..array.len()).try_for_each(|i| { - let scalar = ScalarValue::try_from_array(array, i)?; - if let ScalarValue::List(Some(values), _) = scalar { - self.values.extend(values); - Ok(()) - } else { - internal_err!("array_agg_distinct state must be list") - } - })?; + let scalar_vec = ScalarValue::convert_array_to_scalar_vec(&states[0])?; + for scalars in scalar_vec { + self.values.extend(scalars) + } Ok(()) } fn evaluate(&self) -> Result { - Ok(ScalarValue::new_list( - Some(self.values.clone().into_iter().collect()), - self.datatype.clone(), - )) + let values: Vec = self.values.iter().cloned().collect(); + let arr = ScalarValue::new_list(&values, &self.datatype); + Ok(ScalarValue::List(arr)) } fn size(&self) -> usize { @@ -185,34 +174,56 @@ impl Accumulator for DistinctArrayAggAccumulator { #[cfg(test)] mod tests { + use super::*; - use crate::aggregate::utils::get_accum_scalar_values_as_arrays; use crate::expressions::col; use crate::expressions::tests::aggregate; use arrow::array::{ArrayRef, Int32Array}; use arrow::datatypes::{DataType, Schema}; use arrow::record_batch::RecordBatch; + use arrow_array::cast::as_list_array; + use arrow_array::types::Int32Type; + use arrow_array::{Array, ListArray}; + use arrow_buffer::OffsetBuffer; + use datafusion_common::utils::wrap_into_list_array; + use datafusion_common::{internal_err, DataFusionError}; + + // arrow::compute::sort cann't sort ListArray directly, so we need to sort the inner primitive array and wrap it back into ListArray. + fn sort_list_inner(arr: ScalarValue) -> ScalarValue { + let arr = match arr { + ScalarValue::List(arr) => { + let list_arr = as_list_array(&arr); + list_arr.value(0) + } + _ => { + panic!("Expected ScalarValue::List, got {:?}", arr) + } + }; + + let arr = arrow::compute::sort(&arr, None).unwrap(); + let list_arr = wrap_into_list_array(arr); + ScalarValue::List(Arc::new(list_arr)) + } fn compare_list_contents(expected: ScalarValue, actual: ScalarValue) -> Result<()> { - match (expected, actual) { - (ScalarValue::List(Some(mut e), _), ScalarValue::List(Some(mut a), _)) => { - // workaround lack of Ord of ScalarValue - let cmp = |a: &ScalarValue, b: &ScalarValue| { - a.partial_cmp(b).expect("Can compare ScalarValues") - }; - - e.sort_by(cmp); - a.sort_by(cmp); - // Check that the inputs are the same - assert_eq!(e, a); + let actual = sort_list_inner(actual); + + match (&expected, &actual) { + (ScalarValue::List(arr1), ScalarValue::List(arr2)) => { + if arr1.eq(arr2) { + Ok(()) + } else { + internal_err!( + "Actual value {:?} not found in expected values {:?}", + actual, + expected + ) + } } _ => { - return Err(DataFusionError::Internal( - "Expected scalar lists as inputs".to_string(), - )); + internal_err!("Expected scalar lists as inputs") } } - Ok(()) } fn check_distinct_array_agg( @@ -252,8 +263,8 @@ mod tests { accum1.update_batch(&[input1])?; accum2.update_batch(&[input2])?; - let state = get_accum_scalar_values_as_arrays(accum2.as_ref())?; - accum1.merge_batch(&state)?; + let array = accum2.state()?[0].raw_data()?; + accum1.merge_batch(&[array])?; let actual = accum1.evaluate()?; @@ -263,19 +274,18 @@ mod tests { #[test] fn distinct_array_agg_i32() -> Result<()> { let col: ArrayRef = Arc::new(Int32Array::from(vec![1, 2, 7, 4, 5, 2])); - - let out = ScalarValue::new_list( - Some(vec![ - ScalarValue::Int32(Some(1)), - ScalarValue::Int32(Some(2)), - ScalarValue::Int32(Some(7)), - ScalarValue::Int32(Some(4)), - ScalarValue::Int32(Some(5)), - ]), - DataType::Int32, - ); - - check_distinct_array_agg(col, out, DataType::Int32) + let expected = + ScalarValue::List(Arc::new( + ListArray::from_iter_primitive::(vec![Some(vec![ + Some(1), + Some(2), + Some(4), + Some(5), + Some(7), + ])]), + )); + + check_distinct_array_agg(col, expected, DataType::Int32) } #[test] @@ -283,78 +293,90 @@ mod tests { let col1: ArrayRef = Arc::new(Int32Array::from(vec![1, 2, 7, 4, 5, 2])); let col2: ArrayRef = Arc::new(Int32Array::from(vec![1, 3, 7, 8, 4])); - let out = ScalarValue::new_list( - Some(vec![ - ScalarValue::Int32(Some(1)), - ScalarValue::Int32(Some(2)), - ScalarValue::Int32(Some(3)), - ScalarValue::Int32(Some(4)), - ScalarValue::Int32(Some(5)), - ScalarValue::Int32(Some(7)), - ScalarValue::Int32(Some(8)), - ]), - DataType::Int32, - ); - - check_merge_distinct_array_agg(col1, col2, out, DataType::Int32) + let expected = + ScalarValue::List(Arc::new( + ListArray::from_iter_primitive::(vec![Some(vec![ + Some(1), + Some(2), + Some(3), + Some(4), + Some(5), + Some(7), + Some(8), + ])]), + )); + + check_merge_distinct_array_agg(col1, col2, expected, DataType::Int32) } #[test] fn distinct_array_agg_nested() -> Result<()> { // [[1, 2, 3], [4, 5]] - let l1 = ScalarValue::new_list( - Some(vec![ - ScalarValue::new_list( - Some(vec![ - ScalarValue::from(1i32), - ScalarValue::from(2i32), - ScalarValue::from(3i32), - ]), - DataType::Int32, - ), - ScalarValue::new_list( - Some(vec![ScalarValue::from(4i32), ScalarValue::from(5i32)]), - DataType::Int32, - ), - ]), - DataType::List(Arc::new(Field::new("item", DataType::Int32, true))), + let a1 = ListArray::from_iter_primitive::(vec![Some(vec![ + Some(1), + Some(2), + Some(3), + ])]); + let a2 = ListArray::from_iter_primitive::(vec![Some(vec![ + Some(4), + Some(5), + ])]); + let l1 = ListArray::new( + Arc::new(Field::new("item", a1.data_type().to_owned(), true)), + OffsetBuffer::from_lengths([2]), + arrow::compute::concat(&[&a1, &a2]).unwrap(), + None, ); // [[6], [7, 8]] - let l2 = ScalarValue::new_list( - Some(vec![ - ScalarValue::new_list( - Some(vec![ScalarValue::from(6i32)]), - DataType::Int32, - ), - ScalarValue::new_list( - Some(vec![ScalarValue::from(7i32), ScalarValue::from(8i32)]), - DataType::Int32, - ), - ]), - DataType::List(Arc::new(Field::new("item", DataType::Int32, true))), + let a1 = + ListArray::from_iter_primitive::(vec![Some(vec![Some(6)])]); + let a2 = ListArray::from_iter_primitive::(vec![Some(vec![ + Some(7), + Some(8), + ])]); + let l2 = ListArray::new( + Arc::new(Field::new("item", a1.data_type().to_owned(), true)), + OffsetBuffer::from_lengths([2]), + arrow::compute::concat(&[&a1, &a2]).unwrap(), + None, ); // [[9]] - let l3 = ScalarValue::new_list( - Some(vec![ScalarValue::new_list( - Some(vec![ScalarValue::from(9i32)]), - DataType::Int32, - )]), - DataType::List(Arc::new(Field::new("item", DataType::Int32, true))), + let a1 = + ListArray::from_iter_primitive::(vec![Some(vec![Some(9)])]); + let l3 = ListArray::new( + Arc::new(Field::new("item", a1.data_type().to_owned(), true)), + OffsetBuffer::from_lengths([1]), + Arc::new(a1), + None, ); - let list = ScalarValue::new_list( - Some(vec![l1.clone(), l2.clone(), l3.clone()]), - DataType::List(Arc::new(Field::new("item", DataType::Int32, true))), - ); + let l1 = ScalarValue::List(Arc::new(l1)); + let l2 = ScalarValue::List(Arc::new(l2)); + let l3 = ScalarValue::List(Arc::new(l3)); // Duplicate l1 in the input array and check that it is deduped in the output. let array = ScalarValue::iter_to_array(vec![l1.clone(), l2, l3, l1]).unwrap(); + let expected = + ScalarValue::List(Arc::new( + ListArray::from_iter_primitive::(vec![Some(vec![ + Some(1), + Some(2), + Some(3), + Some(4), + Some(5), + Some(6), + Some(7), + Some(8), + Some(9), + ])]), + )); + check_distinct_array_agg( array, - list, + expected, DataType::List(Arc::new(Field::new_list( "item", Field::new("item", DataType::Int32, true), @@ -366,62 +388,66 @@ mod tests { #[test] fn merge_distinct_array_agg_nested() -> Result<()> { // [[1, 2], [3, 4]] - let l1 = ScalarValue::new_list( - Some(vec![ - ScalarValue::new_list( - Some(vec![ScalarValue::from(1i32), ScalarValue::from(2i32)]), - DataType::Int32, - ), - ScalarValue::new_list( - Some(vec![ScalarValue::from(3i32), ScalarValue::from(4i32)]), - DataType::Int32, - ), - ]), - DataType::List(Arc::new(Field::new("item", DataType::Int32, true))), + let a1 = ListArray::from_iter_primitive::(vec![Some(vec![ + Some(1), + Some(2), + ])]); + let a2 = ListArray::from_iter_primitive::(vec![Some(vec![ + Some(3), + Some(4), + ])]); + let l1 = ListArray::new( + Arc::new(Field::new("item", a1.data_type().to_owned(), true)), + OffsetBuffer::from_lengths([2]), + arrow::compute::concat(&[&a1, &a2]).unwrap(), + None, ); - // [[5]] - let l2 = ScalarValue::new_list( - Some(vec![ScalarValue::new_list( - Some(vec![ScalarValue::from(5i32)]), - DataType::Int32, - )]), - DataType::List(Arc::new(Field::new("item", DataType::Int32, true))), + let a1 = + ListArray::from_iter_primitive::(vec![Some(vec![Some(5)])]); + let l2 = ListArray::new( + Arc::new(Field::new("item", a1.data_type().to_owned(), true)), + OffsetBuffer::from_lengths([1]), + Arc::new(a1), + None, ); // [[6, 7], [8]] - let l3 = ScalarValue::new_list( - Some(vec![ - ScalarValue::new_list( - Some(vec![ScalarValue::from(6i32), ScalarValue::from(7i32)]), - DataType::Int32, - ), - ScalarValue::new_list( - Some(vec![ScalarValue::from(8i32)]), - DataType::Int32, - ), - ]), - DataType::List(Arc::new(Field::new("item", DataType::Int32, true))), + let a1 = ListArray::from_iter_primitive::(vec![Some(vec![ + Some(6), + Some(7), + ])]); + let a2 = + ListArray::from_iter_primitive::(vec![Some(vec![Some(8)])]); + let l3 = ListArray::new( + Arc::new(Field::new("item", a1.data_type().to_owned(), true)), + OffsetBuffer::from_lengths([2]), + arrow::compute::concat(&[&a1, &a2]).unwrap(), + None, ); - let expected = ScalarValue::new_list( - Some(vec![l1.clone(), l2.clone(), l3.clone()]), - DataType::List(Arc::new(Field::new("item", DataType::Int32, true))), - ); + let l1 = ScalarValue::List(Arc::new(l1)); + let l2 = ScalarValue::List(Arc::new(l2)); + let l3 = ScalarValue::List(Arc::new(l3)); // Duplicate l1 in the input array and check that it is deduped in the output. let input1 = ScalarValue::iter_to_array(vec![l1.clone(), l2]).unwrap(); let input2 = ScalarValue::iter_to_array(vec![l1, l3]).unwrap(); - check_merge_distinct_array_agg( - input1, - input2, - expected, - DataType::List(Arc::new(Field::new_list( - "item", - Field::new("item", DataType::Int32, true), - true, - ))), - ) + let expected = + ScalarValue::List(Arc::new( + ListArray::from_iter_primitive::(vec![Some(vec![ + Some(1), + Some(2), + Some(3), + Some(4), + Some(5), + Some(6), + Some(7), + Some(8), + ])]), + )); + + check_merge_distinct_array_agg(input1, input2, expected, DataType::Int32) } } diff --git a/datafusion/physical-expr/src/aggregate/array_agg_ordered.rs b/datafusion/physical-expr/src/aggregate/array_agg_ordered.rs index bf5dbfb4fda9..a53d53107add 100644 --- a/datafusion/physical-expr/src/aggregate/array_agg_ordered.rs +++ b/datafusion/physical-expr/src/aggregate/array_agg_ordered.rs @@ -30,10 +30,11 @@ use crate::{AggregateExpr, LexOrdering, PhysicalExpr, PhysicalSortExpr}; use arrow::array::ArrayRef; use arrow::datatypes::{DataType, Field}; -use arrow_array::{Array, ListArray}; +use arrow_array::Array; use arrow_schema::{Fields, SortOptions}; +use datafusion_common::cast::as_list_array; use datafusion_common::utils::{compare_rows, get_row_at_idx}; -use datafusion_common::{exec_err, internal_err, DataFusionError, Result, ScalarValue}; +use datafusion_common::{exec_err, DataFusionError, Result, ScalarValue}; use datafusion_expr::Accumulator; use itertools::izip; @@ -181,12 +182,14 @@ impl Accumulator for OrderSensitiveArrayAggAccumulator { if values.is_empty() { return Ok(()); } + let n_row = values[0].len(); for index in 0..n_row { let row = get_row_at_idx(values, index)?; self.values.push(row[0].clone()); self.ordering_values.push(row[1..].to_vec()); } + Ok(()) } @@ -197,10 +200,11 @@ impl Accumulator for OrderSensitiveArrayAggAccumulator { // First entry in the state is the aggregation result. let array_agg_values = &states[0]; // 2nd entry stores values received for ordering requirement columns, for each aggregation value inside ARRAY_AGG list. - // For each `ScalarValue` inside ARRAY_AGG list, we will receive a `Vec` that stores + // For each `StructArray` inside ARRAY_AGG list, we will receive an `Array` that stores // values received from its ordering requirement expression. (This information is necessary for during merging). let agg_orderings = &states[1]; - if agg_orderings.as_any().is::() { + + if as_list_array(agg_orderings).is_ok() { // Stores ARRAY_AGG results coming from each partition let mut partition_values = vec![]; // Stores ordering requirement expression results coming from each partition @@ -209,20 +213,21 @@ impl Accumulator for OrderSensitiveArrayAggAccumulator { // Existing values should be merged also. partition_values.push(self.values.clone()); partition_ordering_values.push(self.ordering_values.clone()); - for index in 0..agg_orderings.len() { - let ordering = ScalarValue::try_from_array(agg_orderings, index)?; - // Ordering requirement expression values for each entry in the ARRAY_AGG list - let other_ordering_values = - self.convert_array_agg_to_orderings(ordering)?; - // ARRAY_AGG result. (It is a `ScalarValue::List` under the hood, it stores `Vec`) - let array_agg_res = ScalarValue::try_from_array(array_agg_values, index)?; - if let ScalarValue::List(Some(other_values), _) = array_agg_res { - partition_values.push(other_values); - partition_ordering_values.push(other_ordering_values); - } else { - return internal_err!("ARRAY_AGG state must be list!"); - } + + let array_agg_res = + ScalarValue::convert_array_to_scalar_vec(array_agg_values)?; + + for v in array_agg_res.into_iter() { + partition_values.push(v); } + + let orderings = ScalarValue::convert_array_to_scalar_vec(agg_orderings)?; + // Ordering requirement expression values for each entry in the ARRAY_AGG list + let other_ordering_values = self.convert_array_agg_to_orderings(orderings)?; + for v in other_ordering_values.into_iter() { + partition_ordering_values.push(v); + } + let sort_options = self .ordering_req .iter() @@ -248,10 +253,8 @@ impl Accumulator for OrderSensitiveArrayAggAccumulator { } fn evaluate(&self) -> Result { - Ok(ScalarValue::new_list( - Some(self.values.clone()), - self.datatypes[0].clone(), - )) + let arr = ScalarValue::new_list(&self.values, &self.datatypes[0]); + Ok(ScalarValue::List(arr)) } fn size(&self) -> usize { @@ -280,33 +283,34 @@ impl Accumulator for OrderSensitiveArrayAggAccumulator { } impl OrderSensitiveArrayAggAccumulator { + /// Inner Vec\ in the ordering_values can be thought as ordering information for the each ScalarValue in the values array. + /// See [`merge_ordered_arrays`] for more information. fn convert_array_agg_to_orderings( &self, - in_data: ScalarValue, - ) -> Result>> { - if let ScalarValue::List(Some(list_vals), _field_ref) = in_data { - list_vals.into_iter().map(|struct_vals| { - if let ScalarValue::Struct(Some(orderings), _fields) = struct_vals { - Ok(orderings) - } else { - exec_err!( - "Expects to receive ScalarValue::Struct(Some(..), _) but got:{:?}", - struct_vals.data_type() - ) - } - }).collect::>>() - } else { - exec_err!( - "Expects to receive ScalarValue::List(Some(..), _) but got:{:?}", - in_data.data_type() - ) + array_agg: Vec>, + ) -> Result>>> { + let mut orderings = vec![]; + // in_data is Vec where ScalarValue does not include ScalarValue::List + for in_data in array_agg.into_iter() { + let ordering = in_data.into_iter().map(|struct_vals| { + if let ScalarValue::Struct(Some(orderings), _) = struct_vals { + Ok(orderings) + } else { + exec_err!( + "Expects to receive ScalarValue::Struct(Some(..), _) but got:{:?}", + struct_vals.data_type() + ) + } + }).collect::>>()?; + orderings.push(ordering); } + Ok(orderings) } fn evaluate_orderings(&self) -> Result { let fields = ordering_fields(&self.ordering_req, &self.datatypes[1..]); let struct_field = Fields::from(fields.clone()); - let orderings = self + let orderings: Vec = self .ordering_values .iter() .map(|ordering| { @@ -314,7 +318,9 @@ impl OrderSensitiveArrayAggAccumulator { }) .collect(); let struct_type = DataType::Struct(Fields::from(fields)); - Ok(ScalarValue::new_list(Some(orderings), struct_type)) + + let arr = ScalarValue::new_list(&orderings, &struct_type); + Ok(ScalarValue::List(arr)) } } diff --git a/datafusion/physical-expr/src/aggregate/bit_and_or_xor.rs b/datafusion/physical-expr/src/aggregate/bit_and_or_xor.rs index 93b911c939d6..d7934e79c366 100644 --- a/datafusion/physical-expr/src/aggregate/bit_and_or_xor.rs +++ b/datafusion/physical-expr/src/aggregate/bit_and_or_xor.rs @@ -18,6 +18,7 @@ //! Defines BitAnd, BitOr, and BitXor Aggregate accumulators use ahash::RandomState; +use datafusion_common::cast::as_list_array; use std::any::Any; use std::sync::Arc; @@ -637,13 +638,14 @@ where // 1. Stores aggregate state in `ScalarValue::List` // 2. Constructs `ScalarValue::List` state from distinct numeric stored in hash set let state_out = { - let values = self + let values: Vec = self .values .iter() .map(|x| ScalarValue::new_primitive::(Some(*x), &T::DATA_TYPE)) .collect(); - vec![ScalarValue::new_list(Some(values), T::DATA_TYPE)] + let arr = ScalarValue::new_list(&values, &T::DATA_TYPE); + vec![ScalarValue::List(arr)] }; Ok(state_out) } @@ -668,12 +670,11 @@ where } fn merge_batch(&mut self, states: &[ArrayRef]) -> Result<()> { - if states.is_empty() { - return Ok(()); - } - - for x in states[0].as_list::().iter().flatten() { - self.update_batch(&[x])? + if let Some(state) = states.first() { + let list_arr = as_list_array(state)?; + for arr in list_arr.iter().flatten() { + self.update_batch(&[arr])?; + } } Ok(()) } diff --git a/datafusion/physical-expr/src/aggregate/count_distinct.rs b/datafusion/physical-expr/src/aggregate/count_distinct.rs index 05be8cbccb5f..f5242d983d4c 100644 --- a/datafusion/physical-expr/src/aggregate/count_distinct.rs +++ b/datafusion/physical-expr/src/aggregate/count_distinct.rs @@ -16,6 +16,7 @@ // under the License. use arrow::datatypes::{DataType, Field}; + use std::any::Any; use std::fmt::Debug; use std::sync::Arc; @@ -27,8 +28,8 @@ use std::collections::HashSet; use crate::aggregate::utils::down_cast_any_ref; use crate::expressions::format_state_name; use crate::{AggregateExpr, PhysicalExpr}; +use datafusion_common::Result; use datafusion_common::ScalarValue; -use datafusion_common::{internal_err, DataFusionError, Result}; use datafusion_expr::Accumulator; type DistinctScalarValues = ScalarValue; @@ -142,18 +143,11 @@ impl DistinctCountAccumulator { impl Accumulator for DistinctCountAccumulator { fn state(&self) -> Result> { - let mut cols_out = - ScalarValue::new_list(Some(Vec::new()), self.state_data_type.clone()); - self.values - .iter() - .enumerate() - .for_each(|(_, distinct_values)| { - if let ScalarValue::List(Some(ref mut v), _) = cols_out { - v.push(distinct_values.clone()); - } - }); - Ok(vec![cols_out]) + let scalars = self.values.iter().cloned().collect::>(); + let arr = ScalarValue::new_list(scalars.as_slice(), &self.state_data_type); + Ok(vec![ScalarValue::List(arr)]) } + fn update_batch(&mut self, values: &[ArrayRef]) -> Result<()> { if values.is_empty() { return Ok(()); @@ -167,25 +161,17 @@ impl Accumulator for DistinctCountAccumulator { Ok(()) }) } + fn merge_batch(&mut self, states: &[ArrayRef]) -> Result<()> { if states.is_empty() { return Ok(()); } - let arr = &states[0]; - (0..arr.len()).try_for_each(|index| { - let scalar = ScalarValue::try_from_array(arr, index)?; - - if let ScalarValue::List(Some(scalar), _) = scalar { - scalar.iter().for_each(|scalar| { - if !ScalarValue::is_null(scalar) { - self.values.insert(scalar.clone()); - } - }); - } else { - return internal_err!("Unexpected accumulator state"); - } - Ok(()) - }) + assert_eq!(states.len(), 1, "array_agg states must be singleton!"); + let scalar_vec = ScalarValue::convert_array_to_scalar_vec(&states[0])?; + for scalars in scalar_vec.into_iter() { + self.values.extend(scalars) + } + Ok(()) } fn evaluate(&self) -> Result { @@ -211,33 +197,21 @@ mod tests { Int64Array, Int8Array, UInt16Array, UInt32Array, UInt64Array, UInt8Array, }; use arrow::datatypes::DataType; + use arrow::datatypes::{ + Float32Type, Float64Type, Int16Type, Int32Type, Int64Type, Int8Type, UInt16Type, + UInt32Type, UInt64Type, UInt8Type, + }; + use datafusion_common::cast::{as_boolean_array, as_list_array, as_primitive_array}; use datafusion_common::internal_err; - - macro_rules! state_to_vec { - ($LIST:expr, $DATA_TYPE:ident, $PRIM_TY:ty) => {{ - match $LIST { - ScalarValue::List(_, field) => match field.data_type() { - &DataType::$DATA_TYPE => (), - _ => panic!("Unexpected DataType for list"), - }, - _ => panic!("Expected a ScalarValue::List"), - } - - match $LIST { - ScalarValue::List(None, _) => None, - ScalarValue::List(Some(scalar_values), _) => { - let vec = scalar_values - .iter() - .map(|scalar_value| match scalar_value { - ScalarValue::$DATA_TYPE(value) => *value, - _ => panic!("Unexpected ScalarValue variant"), - }) - .collect::>>(); - - Some(vec) - } - _ => unreachable!(), - } + use datafusion_common::DataFusionError; + + macro_rules! state_to_vec_primitive { + ($LIST:expr, $DATA_TYPE:ident) => {{ + let arr = ScalarValue::raw_data($LIST).unwrap(); + let list_arr = as_list_array(&arr).unwrap(); + let arr = list_arr.values(); + let arr = as_primitive_array::<$DATA_TYPE>(arr)?; + arr.values().iter().cloned().collect::>() }}; } @@ -259,18 +233,25 @@ mod tests { let (states, result) = run_update_batch(&arrays)?; - let mut state_vec = - state_to_vec!(&states[0], $DATA_TYPE, $PRIM_TYPE).unwrap(); + let mut state_vec = state_to_vec_primitive!(&states[0], $DATA_TYPE); state_vec.sort(); assert_eq!(states.len(), 1); - assert_eq!(state_vec, vec![Some(1), Some(2), Some(3)]); + assert_eq!(state_vec, vec![1, 2, 3]); assert_eq!(result, ScalarValue::Int64(Some(3))); Ok(()) }}; } + fn state_to_vec_bool(sv: &ScalarValue) -> Result> { + let arr = ScalarValue::raw_data(sv)?; + let list_arr = as_list_array(&arr)?; + let arr = list_arr.values(); + let bool_arr = as_boolean_array(arr)?; + Ok(bool_arr.iter().flatten().collect()) + } + fn run_update_batch(arrays: &[ArrayRef]) -> Result<(Vec, ScalarValue)> { let agg = DistinctCount::new( arrays[0].data_type().clone(), @@ -353,13 +334,11 @@ mod tests { let (states, result) = run_update_batch(&arrays)?; - let mut state_vec = - state_to_vec!(&states[0], $DATA_TYPE, $PRIM_TYPE).unwrap(); + let mut state_vec = state_to_vec_primitive!(&states[0], $DATA_TYPE); dbg!(&state_vec); state_vec.sort_by(|a, b| match (a, b) { - (Some(lhs), Some(rhs)) => lhs.total_cmp(rhs), - _ => a.partial_cmp(b).unwrap(), + (lhs, rhs) => lhs.total_cmp(rhs), }); let nan_idx = state_vec.len() - 1; @@ -367,16 +346,16 @@ mod tests { assert_eq!( &state_vec[..nan_idx], vec![ - Some(<$PRIM_TYPE>::NEG_INFINITY), - Some(-4.5), - Some(<$PRIM_TYPE as SubNormal>::SUBNORMAL), - Some(1.0), - Some(2.0), - Some(3.0), - Some(<$PRIM_TYPE>::INFINITY) + <$PRIM_TYPE>::NEG_INFINITY, + -4.5, + <$PRIM_TYPE as SubNormal>::SUBNORMAL, + 1.0, + 2.0, + 3.0, + <$PRIM_TYPE>::INFINITY ] ); - assert!(state_vec[nan_idx].unwrap_or_default().is_nan()); + assert!(state_vec[nan_idx].is_nan()); assert_eq!(result, ScalarValue::Int64(Some(8))); Ok(()) @@ -385,61 +364,62 @@ mod tests { #[test] fn count_distinct_update_batch_i8() -> Result<()> { - test_count_distinct_update_batch_numeric!(Int8Array, Int8, i8) + test_count_distinct_update_batch_numeric!(Int8Array, Int8Type, i8) } #[test] fn count_distinct_update_batch_i16() -> Result<()> { - test_count_distinct_update_batch_numeric!(Int16Array, Int16, i16) + test_count_distinct_update_batch_numeric!(Int16Array, Int16Type, i16) } #[test] fn count_distinct_update_batch_i32() -> Result<()> { - test_count_distinct_update_batch_numeric!(Int32Array, Int32, i32) + test_count_distinct_update_batch_numeric!(Int32Array, Int32Type, i32) } #[test] fn count_distinct_update_batch_i64() -> Result<()> { - test_count_distinct_update_batch_numeric!(Int64Array, Int64, i64) + test_count_distinct_update_batch_numeric!(Int64Array, Int64Type, i64) } #[test] fn count_distinct_update_batch_u8() -> Result<()> { - test_count_distinct_update_batch_numeric!(UInt8Array, UInt8, u8) + test_count_distinct_update_batch_numeric!(UInt8Array, UInt8Type, u8) } #[test] fn count_distinct_update_batch_u16() -> Result<()> { - test_count_distinct_update_batch_numeric!(UInt16Array, UInt16, u16) + test_count_distinct_update_batch_numeric!(UInt16Array, UInt16Type, u16) } #[test] fn count_distinct_update_batch_u32() -> Result<()> { - test_count_distinct_update_batch_numeric!(UInt32Array, UInt32, u32) + test_count_distinct_update_batch_numeric!(UInt32Array, UInt32Type, u32) } #[test] fn count_distinct_update_batch_u64() -> Result<()> { - test_count_distinct_update_batch_numeric!(UInt64Array, UInt64, u64) + test_count_distinct_update_batch_numeric!(UInt64Array, UInt64Type, u64) } #[test] fn count_distinct_update_batch_f32() -> Result<()> { - test_count_distinct_update_batch_floating_point!(Float32Array, Float32, f32) + test_count_distinct_update_batch_floating_point!(Float32Array, Float32Type, f32) } #[test] fn count_distinct_update_batch_f64() -> Result<()> { - test_count_distinct_update_batch_floating_point!(Float64Array, Float64, f64) + test_count_distinct_update_batch_floating_point!(Float64Array, Float64Type, f64) } #[test] fn count_distinct_update_batch_boolean() -> Result<()> { - let get_count = |data: BooleanArray| -> Result<(Vec>, i64)> { + let get_count = |data: BooleanArray| -> Result<(Vec, i64)> { let arrays = vec![Arc::new(data) as ArrayRef]; let (states, result) = run_update_batch(&arrays)?; - let mut state_vec = state_to_vec!(&states[0], Boolean, bool).unwrap(); + let mut state_vec = state_to_vec_bool(&states[0])?; state_vec.sort(); + let count = match result { ScalarValue::Int64(c) => c.ok_or_else(|| { DataFusionError::Internal("Found None count".to_string()) @@ -467,22 +447,13 @@ mod tests { Some(false), ]); - assert_eq!( - get_count(zero_count_values)?, - (Vec::>::new(), 0) - ); - assert_eq!(get_count(one_count_values)?, (vec![Some(false)], 1)); - assert_eq!( - get_count(one_count_values_with_null)?, - (vec![Some(true)], 1) - ); - assert_eq!( - get_count(two_count_values)?, - (vec![Some(false), Some(true)], 2) - ); + assert_eq!(get_count(zero_count_values)?, (Vec::::new(), 0)); + assert_eq!(get_count(one_count_values)?, (vec![false], 1)); + assert_eq!(get_count(one_count_values_with_null)?, (vec![true], 1)); + assert_eq!(get_count(two_count_values)?, (vec![false, true], 2)); assert_eq!( get_count(two_count_values_with_null)?, - (vec![Some(false), Some(true)], 2) + (vec![false, true], 2) ); Ok(()) } @@ -494,9 +465,9 @@ mod tests { )) as ArrayRef]; let (states, result) = run_update_batch(&arrays)?; - + let state_vec = state_to_vec_primitive!(&states[0], Int32Type); assert_eq!(states.len(), 1); - assert_eq!(state_to_vec!(&states[0], Int32, i32), Some(vec![])); + assert!(state_vec.is_empty()); assert_eq!(result, ScalarValue::Int64(Some(0))); Ok(()) @@ -507,9 +478,9 @@ mod tests { let arrays = vec![Arc::new(Int32Array::from(vec![0_i32; 0])) as ArrayRef]; let (states, result) = run_update_batch(&arrays)?; - + let state_vec = state_to_vec_primitive!(&states[0], Int32Type); assert_eq!(states.len(), 1); - assert_eq!(state_to_vec!(&states[0], Int32, i32), Some(vec![])); + assert!(state_vec.is_empty()); assert_eq!(result, ScalarValue::Int64(Some(0))); Ok(()) diff --git a/datafusion/physical-expr/src/aggregate/median.rs b/datafusion/physical-expr/src/aggregate/median.rs index 1ec412402638..477dcadceee7 100644 --- a/datafusion/physical-expr/src/aggregate/median.rs +++ b/datafusion/physical-expr/src/aggregate/median.rs @@ -146,14 +146,14 @@ impl std::fmt::Debug for MedianAccumulator { impl Accumulator for MedianAccumulator { fn state(&self) -> Result> { - let all_values = self + let all_values: Vec = self .all_values .iter() .map(|x| ScalarValue::new_primitive::(Some(*x), &self.data_type)) .collect(); - let state = ScalarValue::new_list(Some(all_values), self.data_type.clone()); - Ok(vec![state]) + let arr = ScalarValue::new_list(&all_values, &self.data_type); + Ok(vec![ScalarValue::List(arr)]) } fn update_batch(&mut self, values: &[ArrayRef]) -> Result<()> { diff --git a/datafusion/physical-expr/src/aggregate/sum_distinct.rs b/datafusion/physical-expr/src/aggregate/sum_distinct.rs index c3d8d5e87068..742e24b99e71 100644 --- a/datafusion/physical-expr/src/aggregate/sum_distinct.rs +++ b/datafusion/physical-expr/src/aggregate/sum_distinct.rs @@ -166,10 +166,10 @@ impl Accumulator for DistinctSumAccumulator { &self.data_type, )) }); - vec![ScalarValue::new_list( - Some(distinct_values), - self.data_type.clone(), - )] + vec![ScalarValue::List(ScalarValue::new_list( + &distinct_values, + &self.data_type, + ))] }; Ok(state_out) } diff --git a/datafusion/physical-expr/src/aggregate/tdigest.rs b/datafusion/physical-expr/src/aggregate/tdigest.rs index 7e6d2dcf8f4f..90f5244f477d 100644 --- a/datafusion/physical-expr/src/aggregate/tdigest.rs +++ b/datafusion/physical-expr/src/aggregate/tdigest.rs @@ -28,6 +28,9 @@ //! [Facebook's Folly TDigest]: https://github.com/facebook/folly/blob/main/folly/stats/TDigest.h use arrow::datatypes::DataType; +use arrow_array::cast::as_list_array; +use arrow_array::types::Float64Type; +use datafusion_common::cast::as_primitive_array; use datafusion_common::Result; use datafusion_common::ScalarValue; use std::cmp::Ordering; @@ -566,20 +569,22 @@ impl TDigest { /// [`TDigest`]. pub(crate) fn to_scalar_state(&self) -> Vec { // Gather up all the centroids - let centroids: Vec<_> = self + let centroids: Vec = self .centroids .iter() .flat_map(|c| [c.mean(), c.weight()]) .map(|v| ScalarValue::Float64(Some(v))) .collect(); + let arr = ScalarValue::new_list(¢roids, &DataType::Float64); + vec![ ScalarValue::UInt64(Some(self.max_size as u64)), ScalarValue::Float64(Some(self.sum)), ScalarValue::Float64(Some(self.count)), ScalarValue::Float64(Some(self.max)), ScalarValue::Float64(Some(self.min)), - ScalarValue::new_list(Some(centroids), DataType::Float64), + ScalarValue::List(arr), ] } @@ -600,10 +605,18 @@ impl TDigest { }; let centroids: Vec<_> = match &state[5] { - ScalarValue::List(Some(c), f) if *f.data_type() == DataType::Float64 => c - .chunks(2) - .map(|v| Centroid::new(cast_scalar_f64!(v[0]), cast_scalar_f64!(v[1]))) - .collect(), + ScalarValue::List(arr) => { + let list_array = as_list_array(arr); + let arr = list_array.values(); + + let f64arr = + as_primitive_array::(arr).expect("expected f64 array"); + f64arr + .values() + .chunks(2) + .map(|v| Centroid::new(v[0], v[1])) + .collect() + } v => panic!("invalid centroids type {v:?}"), }; diff --git a/datafusion/physical-expr/src/array_expressions.rs b/datafusion/physical-expr/src/array_expressions.rs index 0a9c2f56048e..067a4cfdffc0 100644 --- a/datafusion/physical-expr/src/array_expressions.rs +++ b/datafusion/physical-expr/src/array_expressions.rs @@ -395,7 +395,7 @@ fn array_array(args: &[ArrayRef], data_type: DataType) -> Result { /// `ListArray` /// /// See [`array_array`] for more details. -fn array(values: &[ColumnarValue]) -> Result { +fn array(values: &[ColumnarValue]) -> Result { let arrays: Vec = values .iter() .map(|x| match x { @@ -417,26 +417,17 @@ fn array(values: &[ColumnarValue]) -> Result { match data_type { // empty array - None => Ok(ColumnarValue::Scalar(ScalarValue::new_list( - Some(vec![]), - DataType::Null, - ))), + None => { + let list_arr = ScalarValue::new_list(&[], &DataType::Null); + Ok(Arc::new(list_arr)) + } // all nulls, set default data type as int32 Some(DataType::Null) => { - let nulls = arrays.len(); - let null_arr = Int32Array::from(vec![None; nulls]); - let field = Arc::new(Field::new("item", DataType::Int32, true)); - let offsets = OffsetBuffer::from_lengths([nulls]); - let values = Arc::new(null_arr) as ArrayRef; - let nulls = None; - Ok(ColumnarValue::Array(Arc::new(ListArray::new( - field, offsets, values, nulls, - )))) + let null_arr = vec![ScalarValue::Int32(None); arrays.len()]; + let list_arr = ScalarValue::new_list(null_arr.as_slice(), &DataType::Int32); + Ok(Arc::new(list_arr)) } - Some(data_type) => Ok(ColumnarValue::Array(array_array( - arrays.as_slice(), - data_type, - )?)), + Some(data_type) => Ok(array_array(arrays.as_slice(), data_type)?), } } @@ -446,11 +437,7 @@ pub fn make_array(arrays: &[ArrayRef]) -> Result { .iter() .map(|x| ColumnarValue::Array(x.clone())) .collect(); - - match array(values.as_slice())? { - ColumnarValue::Array(array) => Ok(array), - ColumnarValue::Scalar(scalar) => Ok(scalar.to_array().clone()), - } + array(values.as_slice()) } fn return_empty(return_null: bool, data_type: DataType) -> Arc { @@ -671,9 +658,7 @@ pub fn array_append(args: &[ArrayRef]) -> Result { check_datatypes("array_append", &[arr.values(), element])?; let res = match arr.value_type() { DataType::List(_) => concat_internal(args)?, - DataType::Null => { - return Ok(array(&[ColumnarValue::Array(args[1].clone())])?.into_array(1)) - } + DataType::Null => return array(&[ColumnarValue::Array(args[1].clone())]), data_type => { macro_rules! array_function { ($ARRAY_TYPE:ident) => { @@ -747,9 +732,7 @@ pub fn array_prepend(args: &[ArrayRef]) -> Result { check_datatypes("array_prepend", &[element, arr.values()])?; let res = match arr.value_type() { DataType::List(_) => concat_internal(args)?, - DataType::Null => { - return Ok(array(&[ColumnarValue::Array(args[0].clone())])?.into_array(1)) - } + DataType::Null => return array(&[ColumnarValue::Array(args[0].clone())]), data_type => { macro_rules! array_function { ($ARRAY_TYPE:ident) => { @@ -1636,7 +1619,7 @@ fn flatten_internal( indexes: Option>, ) -> Result { let list_arr = as_list_array(array)?; - let (field, offsets, values, nulls) = list_arr.clone().into_parts(); + let (field, offsets, values, _) = list_arr.clone().into_parts(); let data_type = field.data_type(); match data_type { @@ -1653,7 +1636,7 @@ fn flatten_internal( _ => { if let Some(indexes) = indexes { let offsets = get_offsets_for_flatten(offsets, indexes); - let list_arr = ListArray::new(field, offsets, values, nulls); + let list_arr = ListArray::new(field, offsets, values, None); Ok(list_arr) } else { Ok(list_arr.clone()) @@ -1974,9 +1957,7 @@ mod tests { ColumnarValue::Scalar(ScalarValue::Int64(Some(2))), ColumnarValue::Scalar(ScalarValue::Int64(Some(3))), ]; - let array = array(&args) - .expect("failed to initialize function array") - .into_array(1); + let array = array(&args).expect("failed to initialize function array"); let result = as_list_array(&array).expect("failed to initialize function array"); assert_eq!(result.len(), 1); assert_eq!( @@ -1998,9 +1979,7 @@ mod tests { ColumnarValue::Array(Arc::new(Int64Array::from(vec![3, 4]))), ColumnarValue::Array(Arc::new(Int64Array::from(vec![5, 6]))), ]; - let array = array(&args) - .expect("failed to initialize function array") - .into_array(1); + let array = array(&args).expect("failed to initialize function array"); let result = as_list_array(&array).expect("failed to initialize function array"); assert_eq!(result.len(), 2); assert_eq!( @@ -3317,9 +3296,7 @@ mod tests { ColumnarValue::Scalar(ScalarValue::Int64(Some(3))), ColumnarValue::Scalar(ScalarValue::Int64(Some(4))), ]; - let result = array(&args) - .expect("failed to initialize function array") - .into_array(1); + let result = array(&args).expect("failed to initialize function array"); ColumnarValue::Array(result.clone()) } @@ -3331,9 +3308,7 @@ mod tests { ColumnarValue::Scalar(ScalarValue::Int64(Some(13))), ColumnarValue::Scalar(ScalarValue::Int64(Some(14))), ]; - let result = array(&args) - .expect("failed to initialize function array") - .into_array(1); + let result = array(&args).expect("failed to initialize function array"); ColumnarValue::Array(result.clone()) } @@ -3345,9 +3320,7 @@ mod tests { ColumnarValue::Scalar(ScalarValue::Int64(Some(3))), ColumnarValue::Scalar(ScalarValue::Int64(Some(4))), ]; - let arr1 = array(&args) - .expect("failed to initialize function array") - .into_array(1); + let arr1 = array(&args).expect("failed to initialize function array"); let args = [ ColumnarValue::Scalar(ScalarValue::Int64(Some(5))), @@ -3355,14 +3328,10 @@ mod tests { ColumnarValue::Scalar(ScalarValue::Int64(Some(7))), ColumnarValue::Scalar(ScalarValue::Int64(Some(8))), ]; - let arr2 = array(&args) - .expect("failed to initialize function array") - .into_array(1); + let arr2 = array(&args).expect("failed to initialize function array"); let args = [ColumnarValue::Array(arr1), ColumnarValue::Array(arr2)]; - let result = array(&args) - .expect("failed to initialize function array") - .into_array(1); + let result = array(&args).expect("failed to initialize function array"); ColumnarValue::Array(result.clone()) } @@ -3374,9 +3343,7 @@ mod tests { ColumnarValue::Scalar(ScalarValue::Int64(Some(3))), ColumnarValue::Scalar(ScalarValue::Null), ]; - let result = array(&args) - .expect("failed to initialize function array") - .into_array(1); + let result = array(&args).expect("failed to initialize function array"); ColumnarValue::Array(result.clone()) } @@ -3388,9 +3355,7 @@ mod tests { ColumnarValue::Scalar(ScalarValue::Int64(Some(3))), ColumnarValue::Scalar(ScalarValue::Null), ]; - let arr1 = array(&args) - .expect("failed to initialize function array") - .into_array(1); + let arr1 = array(&args).expect("failed to initialize function array"); let args = [ ColumnarValue::Scalar(ScalarValue::Null), @@ -3398,14 +3363,10 @@ mod tests { ColumnarValue::Scalar(ScalarValue::Int64(Some(7))), ColumnarValue::Scalar(ScalarValue::Null), ]; - let arr2 = array(&args) - .expect("failed to initialize function array") - .into_array(1); + let arr2 = array(&args).expect("failed to initialize function array"); let args = [ColumnarValue::Array(arr1), ColumnarValue::Array(arr2)]; - let result = array(&args) - .expect("failed to initialize function array") - .into_array(1); + let result = array(&args).expect("failed to initialize function array"); ColumnarValue::Array(result.clone()) } @@ -3419,9 +3380,7 @@ mod tests { ColumnarValue::Scalar(ScalarValue::Int64(Some(2))), ColumnarValue::Scalar(ScalarValue::Int64(Some(3))), ]; - let result = array(&args) - .expect("failed to initialize function array") - .into_array(1); + let result = array(&args).expect("failed to initialize function array"); ColumnarValue::Array(result.clone()) } @@ -3433,9 +3392,7 @@ mod tests { ColumnarValue::Scalar(ScalarValue::Int64(Some(3))), ColumnarValue::Scalar(ScalarValue::Int64(Some(4))), ]; - let arr1 = array(&args) - .expect("failed to initialize function array") - .into_array(1); + let arr1 = array(&args).expect("failed to initialize function array"); let args = [ ColumnarValue::Scalar(ScalarValue::Int64(Some(5))), @@ -3443,9 +3400,7 @@ mod tests { ColumnarValue::Scalar(ScalarValue::Int64(Some(7))), ColumnarValue::Scalar(ScalarValue::Int64(Some(8))), ]; - let arr2 = array(&args) - .expect("failed to initialize function array") - .into_array(1); + let arr2 = array(&args).expect("failed to initialize function array"); let args = [ ColumnarValue::Scalar(ScalarValue::Int64(Some(1))), @@ -3453,9 +3408,7 @@ mod tests { ColumnarValue::Scalar(ScalarValue::Int64(Some(3))), ColumnarValue::Scalar(ScalarValue::Int64(Some(4))), ]; - let arr3 = array(&args) - .expect("failed to initialize function array") - .into_array(1); + let arr3 = array(&args).expect("failed to initialize function array"); let args = [ ColumnarValue::Scalar(ScalarValue::Int64(Some(9))), @@ -3463,9 +3416,7 @@ mod tests { ColumnarValue::Scalar(ScalarValue::Int64(Some(11))), ColumnarValue::Scalar(ScalarValue::Int64(Some(12))), ]; - let arr4 = array(&args) - .expect("failed to initialize function array") - .into_array(1); + let arr4 = array(&args).expect("failed to initialize function array"); let args = [ ColumnarValue::Scalar(ScalarValue::Int64(Some(5))), @@ -3473,9 +3424,7 @@ mod tests { ColumnarValue::Scalar(ScalarValue::Int64(Some(7))), ColumnarValue::Scalar(ScalarValue::Int64(Some(8))), ]; - let arr5 = array(&args) - .expect("failed to initialize function array") - .into_array(1); + let arr5 = array(&args).expect("failed to initialize function array"); let args = [ ColumnarValue::Array(arr1), @@ -3484,9 +3433,7 @@ mod tests { ColumnarValue::Array(arr4), ColumnarValue::Array(arr5), ]; - let result = array(&args) - .expect("failed to initialize function array") - .into_array(1); + let result = array(&args).expect("failed to initialize function array"); ColumnarValue::Array(result.clone()) } } diff --git a/datafusion/physical-expr/src/functions.rs b/datafusion/physical-expr/src/functions.rs index e04a68615a46..f23b45e26a03 100644 --- a/datafusion/physical-expr/src/functions.rs +++ b/datafusion/physical-expr/src/functions.rs @@ -349,13 +349,7 @@ where .collect::>(); let result = (inner)(&args); - - // maybe back to scalar - if len.is_some() { - result.map(ColumnarValue::Array) - } else { - ScalarValue::try_from_array(&result?, 0).map(ColumnarValue::Scalar) - } + result.map(ColumnarValue::Array) }) } diff --git a/datafusion/physical-plan/src/values.rs b/datafusion/physical-plan/src/values.rs index 2cf341d1fe60..383726401c5a 100644 --- a/datafusion/physical-plan/src/values.rs +++ b/datafusion/physical-plan/src/values.rs @@ -66,10 +66,11 @@ impl ValuesExec { (0..n_row) .map(|i| { let r = data[i][j].evaluate(&batch); + match r { Ok(ColumnarValue::Scalar(scalar)) => Ok(scalar), Ok(ColumnarValue::Array(a)) if a.len() == 1 => { - ScalarValue::try_from_array(&a, 0) + Ok(ScalarValue::List(a)) } Ok(ColumnarValue::Array(a)) => { plan_err!( diff --git a/datafusion/proto/proto/datafusion.proto b/datafusion/proto/proto/datafusion.proto index c60dae71ef86..1819d1a4392d 100644 --- a/datafusion/proto/proto/datafusion.proto +++ b/datafusion/proto/proto/datafusion.proto @@ -883,12 +883,10 @@ message Union{ repeated int32 type_ids = 3; } -message ScalarListValue{ - // encode null explicitly to distinguish a list with a null value - // from a list with no values) - bool is_null = 3; - Field field = 1; - repeated ScalarValue values = 2; +message ScalarListValue { + bytes ipc_message = 1; + bytes arrow_data = 2; + Schema schema = 3; } message ScalarTime32Value { @@ -965,7 +963,6 @@ message ScalarValue{ int32 date_32_value = 14; ScalarTime32Value time32_value = 15; ScalarListValue list_value = 17; - //WAS: ScalarType null_list_value = 18; Decimal128 decimal128_value = 20; Decimal256 decimal256_value = 39; diff --git a/datafusion/proto/src/generated/pbjson.rs b/datafusion/proto/src/generated/pbjson.rs index 266075e68922..9aed987491f3 100644 --- a/datafusion/proto/src/generated/pbjson.rs +++ b/datafusion/proto/src/generated/pbjson.rs @@ -20003,24 +20003,26 @@ impl serde::Serialize for ScalarListValue { { use serde::ser::SerializeStruct; let mut len = 0; - if self.is_null { + if !self.ipc_message.is_empty() { len += 1; } - if self.field.is_some() { + if !self.arrow_data.is_empty() { len += 1; } - if !self.values.is_empty() { + if self.schema.is_some() { len += 1; } let mut struct_ser = serializer.serialize_struct("datafusion.ScalarListValue", len)?; - if self.is_null { - struct_ser.serialize_field("isNull", &self.is_null)?; + if !self.ipc_message.is_empty() { + #[allow(clippy::needless_borrow)] + struct_ser.serialize_field("ipcMessage", pbjson::private::base64::encode(&self.ipc_message).as_str())?; } - if let Some(v) = self.field.as_ref() { - struct_ser.serialize_field("field", v)?; + if !self.arrow_data.is_empty() { + #[allow(clippy::needless_borrow)] + struct_ser.serialize_field("arrowData", pbjson::private::base64::encode(&self.arrow_data).as_str())?; } - if !self.values.is_empty() { - struct_ser.serialize_field("values", &self.values)?; + if let Some(v) = self.schema.as_ref() { + struct_ser.serialize_field("schema", v)?; } struct_ser.end() } @@ -20032,17 +20034,18 @@ impl<'de> serde::Deserialize<'de> for ScalarListValue { D: serde::Deserializer<'de>, { const FIELDS: &[&str] = &[ - "is_null", - "isNull", - "field", - "values", + "ipc_message", + "ipcMessage", + "arrow_data", + "arrowData", + "schema", ]; #[allow(clippy::enum_variant_names)] enum GeneratedField { - IsNull, - Field, - Values, + IpcMessage, + ArrowData, + Schema, } impl<'de> serde::Deserialize<'de> for GeneratedField { fn deserialize(deserializer: D) -> std::result::Result @@ -20064,9 +20067,9 @@ impl<'de> serde::Deserialize<'de> for ScalarListValue { E: serde::de::Error, { match value { - "isNull" | "is_null" => Ok(GeneratedField::IsNull), - "field" => Ok(GeneratedField::Field), - "values" => Ok(GeneratedField::Values), + "ipcMessage" | "ipc_message" => Ok(GeneratedField::IpcMessage), + "arrowData" | "arrow_data" => Ok(GeneratedField::ArrowData), + "schema" => Ok(GeneratedField::Schema), _ => Err(serde::de::Error::unknown_field(value, FIELDS)), } } @@ -20086,35 +20089,39 @@ impl<'de> serde::Deserialize<'de> for ScalarListValue { where V: serde::de::MapAccess<'de>, { - let mut is_null__ = None; - let mut field__ = None; - let mut values__ = None; + let mut ipc_message__ = None; + let mut arrow_data__ = None; + let mut schema__ = None; while let Some(k) = map_.next_key()? { match k { - GeneratedField::IsNull => { - if is_null__.is_some() { - return Err(serde::de::Error::duplicate_field("isNull")); + GeneratedField::IpcMessage => { + if ipc_message__.is_some() { + return Err(serde::de::Error::duplicate_field("ipcMessage")); } - is_null__ = Some(map_.next_value()?); + ipc_message__ = + Some(map_.next_value::<::pbjson::private::BytesDeserialize<_>>()?.0) + ; } - GeneratedField::Field => { - if field__.is_some() { - return Err(serde::de::Error::duplicate_field("field")); + GeneratedField::ArrowData => { + if arrow_data__.is_some() { + return Err(serde::de::Error::duplicate_field("arrowData")); } - field__ = map_.next_value()?; + arrow_data__ = + Some(map_.next_value::<::pbjson::private::BytesDeserialize<_>>()?.0) + ; } - GeneratedField::Values => { - if values__.is_some() { - return Err(serde::de::Error::duplicate_field("values")); + GeneratedField::Schema => { + if schema__.is_some() { + return Err(serde::de::Error::duplicate_field("schema")); } - values__ = Some(map_.next_value()?); + schema__ = map_.next_value()?; } } } Ok(ScalarListValue { - is_null: is_null__.unwrap_or_default(), - field: field__, - values: values__.unwrap_or_default(), + ipc_message: ipc_message__.unwrap_or_default(), + arrow_data: arrow_data__.unwrap_or_default(), + schema: schema__, }) } } diff --git a/datafusion/proto/src/generated/prost.rs b/datafusion/proto/src/generated/prost.rs index 894afa570fb0..883799b1590d 100644 --- a/datafusion/proto/src/generated/prost.rs +++ b/datafusion/proto/src/generated/prost.rs @@ -1072,14 +1072,12 @@ pub struct Union { #[allow(clippy::derive_partial_eq_without_eq)] #[derive(Clone, PartialEq, ::prost::Message)] pub struct ScalarListValue { - /// encode null explicitly to distinguish a list with a null value - /// from a list with no values) - #[prost(bool, tag = "3")] - pub is_null: bool, - #[prost(message, optional, tag = "1")] - pub field: ::core::option::Option, - #[prost(message, repeated, tag = "2")] - pub values: ::prost::alloc::vec::Vec, + #[prost(bytes = "vec", tag = "1")] + pub ipc_message: ::prost::alloc::vec::Vec, + #[prost(bytes = "vec", tag = "2")] + pub arrow_data: ::prost::alloc::vec::Vec, + #[prost(message, optional, tag = "3")] + pub schema: ::core::option::Option, } #[allow(clippy::derive_partial_eq_without_eq)] #[derive(Clone, PartialEq, ::prost::Message)] @@ -1224,7 +1222,6 @@ pub mod scalar_value { Date32Value(i32), #[prost(message, tag = "15")] Time32Value(super::ScalarTime32Value), - /// WAS: ScalarType null_list_value = 18; #[prost(message, tag = "17")] ListValue(super::ScalarListValue), #[prost(message, tag = "20")] diff --git a/datafusion/proto/src/logical_plan/from_proto.rs b/datafusion/proto/src/logical_plan/from_proto.rs index 2203016e08f1..b3873c01dd06 100644 --- a/datafusion/proto/src/logical_plan/from_proto.rs +++ b/datafusion/proto/src/logical_plan/from_proto.rs @@ -25,9 +25,13 @@ use crate::protobuf::{ AnalyzedLogicalPlanType, CubeNode, GroupingSetNode, OptimizedLogicalPlanType, OptimizedPhysicalPlanType, PlaceholderNode, RollupNode, }; -use arrow::datatypes::{ - i256, DataType, Field, IntervalMonthDayNanoType, IntervalUnit, Schema, TimeUnit, - UnionFields, UnionMode, +use arrow::{ + buffer::{Buffer, MutableBuffer}, + datatypes::{ + i256, DataType, Field, IntervalMonthDayNanoType, IntervalUnit, Schema, TimeUnit, + UnionFields, UnionMode, + }, + ipc::{reader::read_record_batch, root_as_message}, }; use datafusion::execution::registry::FunctionRegistry; use datafusion_common::{ @@ -643,23 +647,39 @@ impl TryFrom<&protobuf::ScalarValue> for ScalarValue { Value::Date32Value(v) => Self::Date32(Some(*v)), Value::ListValue(scalar_list) => { let protobuf::ScalarListValue { - is_null, - values, - field, + ipc_message, + arrow_data, + schema, } = &scalar_list; - let field: Field = field.as_ref().required("field")?; - let field = Arc::new(field); - - let values: Result, Error> = - values.iter().map(|val| val.try_into()).collect(); - let values = values?; - - validate_list_values(field.as_ref(), &values)?; - - let values = if *is_null { None } else { Some(values) }; + let schema: Schema = if let Some(schema_ref) = schema { + schema_ref.try_into()? + } else { + return Err(Error::General("Unexpected schema".to_string())); + }; - Self::List(values, field) + let message = root_as_message(ipc_message.as_slice()).unwrap(); + + // TODO: Add comment to why adding 0 before arrow_data. + // This code is from https://github.com/apache/arrow-rs/blob/4320a753beaee0a1a6870c59ef46b59e88c9c323/arrow-ipc/src/reader.rs#L1670-L1674C45 + // Construct an unaligned buffer + let mut buffer = MutableBuffer::with_capacity(arrow_data.len() + 1); + buffer.push(0_u8); + buffer.extend_from_slice(arrow_data.as_slice()); + let b = Buffer::from(buffer).slice(1); + + let ipc_batch = message.header_as_record_batch().unwrap(); + let record_batch = read_record_batch( + &b, + ipc_batch, + Arc::new(schema), + &Default::default(), + None, + &message.version(), + ) + .unwrap(); + let arr = record_batch.column(0); + Self::List(arr.to_owned()) } Value::NullValue(v) => { let null_type: DataType = v.try_into()?; @@ -925,22 +945,6 @@ pub fn parse_i32_to_aggregate_function(value: &i32) -> Result Result<(), Error> { - for value in values { - let field_type = field.data_type(); - let value_type = value.data_type(); - - if field_type != &value_type { - return Err(proto_error(format!( - "Expected field type {field_type:?}, got scalar of type: {value_type:?}" - ))); - } - } - Ok(()) -} - pub fn parse_expr( proto: &protobuf::LogicalExprNode, registry: &dyn FunctionRegistry, diff --git a/datafusion/proto/src/logical_plan/to_proto.rs b/datafusion/proto/src/logical_plan/to_proto.rs index c10855bf2514..e80d60931cf6 100644 --- a/datafusion/proto/src/logical_plan/to_proto.rs +++ b/datafusion/proto/src/logical_plan/to_proto.rs @@ -30,10 +30,13 @@ use crate::protobuf::{ AnalyzedLogicalPlanType, CubeNode, EmptyMessage, GroupingSetNode, LogicalExprList, OptimizedLogicalPlanType, OptimizedPhysicalPlanType, PlaceholderNode, RollupNode, }; - -use arrow::datatypes::{ - DataType, Field, IntervalMonthDayNanoType, IntervalUnit, Schema, SchemaRef, TimeUnit, - UnionMode, +use arrow::{ + datatypes::{ + DataType, Field, IntervalMonthDayNanoType, IntervalUnit, Schema, SchemaRef, + TimeUnit, UnionMode, + }, + ipc::writer::{DictionaryTracker, IpcDataGenerator}, + record_batch::RecordBatch, }; use datafusion_common::{ Column, Constraint, Constraints, DFField, DFSchema, DFSchemaRef, OwnedTableReference, @@ -1142,27 +1145,27 @@ impl TryFrom<&ScalarValue> for protobuf::ScalarValue { "Proto serialization error: ScalarValue::Fixedsizelist not supported" .to_string(), )), - ScalarValue::List(values, boxed_field) => { - let is_null = values.is_none(); - - let values = if let Some(values) = values.as_ref() { - values - .iter() - .map(|v| v.try_into()) - .collect::, _>>()? - } else { - vec![] + ScalarValue::List(arr) => { + let batch = + RecordBatch::try_from_iter(vec![("field_name", arr.to_owned())]) + .unwrap(); + let gen = IpcDataGenerator {}; + let mut dict_tracker = DictionaryTracker::new(false); + let (_, encoded_message) = gen + .encoded_batch(&batch, &mut dict_tracker, &Default::default()) + .unwrap(); + + let schema: protobuf::Schema = batch.schema().try_into()?; + + let scalar_list_value = protobuf::ScalarListValue { + ipc_message: encoded_message.ipc_message, + arrow_data: encoded_message.arrow_data, + schema: Some(schema), }; - let field = boxed_field.as_ref().try_into()?; - Ok(protobuf::ScalarValue { value: Some(protobuf::scalar_value::Value::ListValue( - protobuf::ScalarListValue { - is_null, - field: Some(field), - values, - }, + scalar_list_value, )), }) } diff --git a/datafusion/proto/tests/cases/roundtrip_logical_plan.rs b/datafusion/proto/tests/cases/roundtrip_logical_plan.rs index 11ee8c0876bc..ca801df337f1 100644 --- a/datafusion/proto/tests/cases/roundtrip_logical_plan.rs +++ b/datafusion/proto/tests/cases/roundtrip_logical_plan.rs @@ -24,6 +24,7 @@ use arrow::datatypes::{ DataType, Field, Fields, IntervalDayTimeType, IntervalMonthDayNanoType, IntervalUnit, Schema, SchemaRef, TimeUnit, UnionFields, UnionMode, }; + use prost::Message; use datafusion::datasource::provider::TableProviderFactory; @@ -512,59 +513,6 @@ fn scalar_values_error_serialization() { Some(vec![]), vec![Field::new("item", DataType::Int16, true)].into(), ), - // Should fail due to inconsistent types in the list - ScalarValue::new_list( - Some(vec![ - ScalarValue::Int16(None), - ScalarValue::Float32(Some(32.0)), - ]), - DataType::List(new_arc_field("item", DataType::Int16, true)), - ), - ScalarValue::new_list( - Some(vec![ - ScalarValue::Float32(None), - ScalarValue::Float32(Some(32.0)), - ]), - DataType::List(new_arc_field("item", DataType::Int16, true)), - ), - ScalarValue::new_list( - Some(vec![ - ScalarValue::Float32(None), - ScalarValue::Float32(Some(32.0)), - ]), - DataType::Int16, - ), - ScalarValue::new_list( - Some(vec![ - ScalarValue::new_list( - None, - DataType::List(new_arc_field("level2", DataType::Float32, true)), - ), - ScalarValue::new_list( - Some(vec![ - ScalarValue::Float32(Some(-213.1)), - ScalarValue::Float32(None), - ScalarValue::Float32(Some(5.5)), - ScalarValue::Float32(Some(2.0)), - ScalarValue::Float32(Some(1.0)), - ]), - DataType::List(new_arc_field("level2", DataType::Float32, true)), - ), - ScalarValue::new_list( - None, - DataType::List(new_arc_field( - "lists are typed inconsistently", - DataType::Int16, - true, - )), - ), - ]), - DataType::List(new_arc_field( - "level1", - DataType::List(new_arc_field("level2", DataType::Float32, true)), - true, - )), - ), ]; for test_case in should_fail_on_seralize.into_iter() { @@ -599,7 +547,7 @@ fn round_trip_scalar_values() { ScalarValue::UInt64(None), ScalarValue::Utf8(None), ScalarValue::LargeUtf8(None), - ScalarValue::new_list(None, DataType::Boolean), + ScalarValue::List(ScalarValue::new_list(&[], &DataType::Boolean)), ScalarValue::Date32(None), ScalarValue::Boolean(Some(true)), ScalarValue::Boolean(Some(false)), @@ -690,32 +638,32 @@ fn round_trip_scalar_values() { i64::MAX, ))), ScalarValue::IntervalMonthDayNano(None), - ScalarValue::new_list( - Some(vec![ + ScalarValue::List(ScalarValue::new_list( + &[ ScalarValue::Float32(Some(-213.1)), ScalarValue::Float32(None), ScalarValue::Float32(Some(5.5)), ScalarValue::Float32(Some(2.0)), ScalarValue::Float32(Some(1.0)), - ]), - DataType::Float32, - ), - ScalarValue::new_list( - Some(vec![ - ScalarValue::new_list(None, DataType::Float32), - ScalarValue::new_list( - Some(vec![ + ], + &DataType::Float32, + )), + ScalarValue::List(ScalarValue::new_list( + &[ + ScalarValue::List(ScalarValue::new_list(&[], &DataType::Float32)), + ScalarValue::List(ScalarValue::new_list( + &[ ScalarValue::Float32(Some(-213.1)), ScalarValue::Float32(None), ScalarValue::Float32(Some(5.5)), ScalarValue::Float32(Some(2.0)), ScalarValue::Float32(Some(1.0)), - ]), - DataType::Float32, - ), - ]), - DataType::List(new_arc_field("item", DataType::Float32, true)), - ), + ], + &DataType::Float32, + )), + ], + &DataType::List(new_arc_field("item", DataType::Float32, true)), + )), ScalarValue::Dictionary( Box::new(DataType::Int32), Box::new(ScalarValue::Utf8(Some("foo".into()))), @@ -978,7 +926,6 @@ fn roundtrip_null_scalar_values() { ScalarValue::Date32(None), ScalarValue::TimestampMicrosecond(None, None), ScalarValue::TimestampNanosecond(None, None), - ScalarValue::List(None, Arc::new(Field::new("item", DataType::Boolean, false))), ]; for test_case in test_types.into_iter() { diff --git a/datafusion/sql/src/expr/value.rs b/datafusion/sql/src/expr/value.rs index c949904cd84c..3a06fdb158f7 100644 --- a/datafusion/sql/src/expr/value.rs +++ b/datafusion/sql/src/expr/value.rs @@ -16,6 +16,7 @@ // under the License. use crate::planner::{ContextProvider, PlannerContext, SqlToRel}; +use arrow::array::new_null_array; use arrow::compute::kernels::cast_utils::parse_interval_month_day_nano; use arrow::datatypes::DECIMAL128_MAX_PRECISION; use arrow_schema::DataType; @@ -153,13 +154,13 @@ impl<'a, S: ContextProvider> SqlToRel<'a, S> { values.iter().map(|e| e.data_type()).collect(); if data_types.is_empty() { - Ok(lit(ScalarValue::new_list(None, DataType::Utf8))) + Ok(lit(ScalarValue::List(new_null_array(&DataType::Null, 0)))) } else if data_types.len() > 1 { not_impl_err!("Arrays with different types are not supported: {data_types:?}") } else { let data_type = values[0].data_type(); - - Ok(lit(ScalarValue::new_list(Some(values), data_type))) + let arr = ScalarValue::new_list(&values, &data_type); + Ok(lit(ScalarValue::List(arr))) } } From 9aacdee13881f25335d8ea323311949b9799e6d3 Mon Sep 17 00:00:00 2001 From: Mustafa Akur <106137913+mustafasrepo@users.noreply.github.com> Date: Tue, 17 Oct 2023 11:18:44 +0300 Subject: [PATCH 082/572] [MINOR]:Do not introduce unnecessary repartition when row count is 1. (#7832) * Initial commit * Fix failing tests * More idiomatic expressions * Update tests, use batch size during partition benefit check * Fix failing tests * is_exact when row count is 1 --------- Co-authored-by: Mehmet Ozan Kabak --- .../core/src/datasource/listing/table.rs | 13 +- .../enforce_distribution.rs | 34 ++-- datafusion/core/tests/sql/order.rs | 8 +- .../physical-plan/src/aggregates/mod.rs | 21 ++- .../sqllogictest/test_files/aggregate.slt | 20 +-- datafusion/sqllogictest/test_files/copy.slt | 2 +- .../sqllogictest/test_files/groupby.slt | 53 ++++--- datafusion/sqllogictest/test_files/join.slt | 10 +- datafusion/sqllogictest/test_files/joins.slt | 148 +++++++++--------- .../sqllogictest/test_files/options.slt | 6 +- datafusion/sqllogictest/test_files/select.slt | 3 +- .../sqllogictest/test_files/subquery.slt | 32 ++-- .../sqllogictest/test_files/tpch/q15.slt.part | 27 ++-- datafusion/sqllogictest/test_files/union.slt | 107 +++++++------ 14 files changed, 270 insertions(+), 214 deletions(-) diff --git a/datafusion/core/src/datasource/listing/table.rs b/datafusion/core/src/datasource/listing/table.rs index 064624366938..6ef214c97fd5 100644 --- a/datafusion/core/src/datasource/listing/table.rs +++ b/datafusion/core/src/datasource/listing/table.rs @@ -1615,10 +1615,12 @@ mod tests { #[tokio::test] async fn test_insert_into_append_new_json_files() -> Result<()> { + let mut config_map: HashMap = HashMap::new(); + config_map.insert("datafusion.execution.batch_size".into(), "1".into()); helper_test_append_new_files_to_table( FileType::JSON, FileCompressionType::UNCOMPRESSED, - None, + Some(config_map), ) .await?; Ok(()) @@ -1637,10 +1639,12 @@ mod tests { #[tokio::test] async fn test_insert_into_append_new_csv_files() -> Result<()> { + let mut config_map: HashMap = HashMap::new(); + config_map.insert("datafusion.execution.batch_size".into(), "1".into()); helper_test_append_new_files_to_table( FileType::CSV, FileCompressionType::UNCOMPRESSED, - None, + Some(config_map), ) .await?; Ok(()) @@ -1648,10 +1652,12 @@ mod tests { #[tokio::test] async fn test_insert_into_append_new_parquet_files_defaults() -> Result<()> { + let mut config_map: HashMap = HashMap::new(); + config_map.insert("datafusion.execution.batch_size".into(), "1".into()); helper_test_append_new_files_to_table( FileType::PARQUET, FileCompressionType::UNCOMPRESSED, - None, + Some(config_map), ) .await?; Ok(()) @@ -1838,6 +1844,7 @@ mod tests { "datafusion.execution.parquet.write_batch_size".into(), "5".into(), ); + config_map.insert("datafusion.execution.batch_size".into(), "1".into()); helper_test_append_new_files_to_table( FileType::PARQUET, FileCompressionType::UNCOMPRESSED, diff --git a/datafusion/core/src/physical_optimizer/enforce_distribution.rs b/datafusion/core/src/physical_optimizer/enforce_distribution.rs index 9be566f10a72..52525d1fc44d 100644 --- a/datafusion/core/src/physical_optimizer/enforce_distribution.rs +++ b/datafusion/core/src/physical_optimizer/enforce_distribution.rs @@ -1021,6 +1021,7 @@ fn add_hash_on_top( // until Repartition(Hash). dist_onward: &mut Option, input_idx: usize, + repartition_beneficial_stats: bool, ) -> Result> { if n_target == input.output_partitioning().partition_count() && n_target == 1 { // In this case adding a hash repartition is unnecessary as the hash @@ -1044,9 +1045,13 @@ fn add_hash_on_top( // - Usage of order preserving variants is not desirable (per the flag // `config.optimizer.bounded_order_preserving_variants`). let should_preserve_ordering = input.output_ordering().is_some(); - // Since hashing benefits from partitioning, add a round-robin repartition - // before it: - let mut new_plan = add_roundrobin_on_top(input, n_target, dist_onward, 0)?; + let mut new_plan = if repartition_beneficial_stats { + // Since hashing benefits from partitioning, add a round-robin repartition + // before it: + add_roundrobin_on_top(input, n_target, dist_onward, 0)? + } else { + input + }; new_plan = Arc::new( RepartitionExec::try_new(new_plan, Partitioning::Hash(hash_exprs, n_target))? .with_preserve_order(should_preserve_ordering), @@ -1223,6 +1228,7 @@ fn ensure_distribution( let enable_round_robin = config.optimizer.enable_round_robin_repartition; let repartition_file_scans = config.optimizer.repartition_file_scans; let repartition_file_min_size = config.optimizer.repartition_file_min_size; + let batch_size = config.execution.batch_size; let is_unbounded = unbounded_output(&dist_context.plan); // Use order preserving variants either of the conditions true // - it is desired according to config @@ -1233,13 +1239,7 @@ fn ensure_distribution( if dist_context.plan.children().is_empty() { return Ok(Transformed::No(dist_context)); } - // Don't need to apply when the returned row count is not greater than 1: - let stats = dist_context.plan.statistics(); - let mut repartition_beneficial_stat = true; - if stats.is_exact { - repartition_beneficial_stat = - stats.num_rows.map(|num_rows| num_rows > 1).unwrap_or(true); - } + // Remove unnecessary repartition from the physical plan if any let DistributionContext { mut plan, @@ -1263,7 +1263,6 @@ fn ensure_distribution( plan = updated_window; } }; - let n_children = plan.children().len(); // This loop iterates over all the children to: // - Increase parallelism for every child if it is beneficial. @@ -1289,9 +1288,19 @@ fn ensure_distribution( maintains, child_idx, )| { + // Don't need to apply when the returned row count is not greater than 1: + let stats = child.statistics(); + let repartition_beneficial_stats = if stats.is_exact { + stats + .num_rows + .map(|num_rows| num_rows > batch_size) + .unwrap_or(true) + } else { + true + }; if enable_round_robin // Operator benefits from partitioning (e.g. filter): - && (would_benefit && repartition_beneficial_stat) + && (would_benefit && repartition_beneficial_stats) // Unless partitioning doesn't increase the partition count, it is not beneficial: && child.output_partitioning().partition_count() < target_partitions { @@ -1340,6 +1349,7 @@ fn ensure_distribution( target_partitions, dist_onward, child_idx, + repartition_beneficial_stats, )?; } Distribution::UnspecifiedDistribution => {} diff --git a/datafusion/core/tests/sql/order.rs b/datafusion/core/tests/sql/order.rs index 0142675bbd49..6e3f6319e119 100644 --- a/datafusion/core/tests/sql/order.rs +++ b/datafusion/core/tests/sql/order.rs @@ -209,16 +209,16 @@ ORDER BY 1, 2; " AggregateExec: mode=FinalPartitioned, gby=[Int64(0)@0 as Int64(0), t@1 as t], aggr=[]", " CoalesceBatchesExec: target_batch_size=8192", " RepartitionExec: partitioning=Hash([Int64(0)@0, t@1], 2), input_partitions=2", - " AggregateExec: mode=Partial, gby=[0 as Int64(0), t@0 as t], aggr=[]", - " RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1", + " RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1", + " AggregateExec: mode=Partial, gby=[0 as Int64(0), t@0 as t], aggr=[]", " ProjectionExec: expr=[column1@0 as t]", " ValuesExec", " ProjectionExec: expr=[Int64(1)@0 as m, t@1 as t]", " AggregateExec: mode=FinalPartitioned, gby=[Int64(1)@0 as Int64(1), t@1 as t], aggr=[]", " CoalesceBatchesExec: target_batch_size=8192", " RepartitionExec: partitioning=Hash([Int64(1)@0, t@1], 2), input_partitions=2", - " AggregateExec: mode=Partial, gby=[1 as Int64(1), t@0 as t], aggr=[]", - " RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1", + " RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1", + " AggregateExec: mode=Partial, gby=[1 as Int64(1), t@0 as t], aggr=[]", " ProjectionExec: expr=[column1@0 as t]", " ValuesExec", ]; diff --git a/datafusion/physical-plan/src/aggregates/mod.rs b/datafusion/physical-plan/src/aggregates/mod.rs index c33f49049de1..3c9dabf60590 100644 --- a/datafusion/physical-plan/src/aggregates/mod.rs +++ b/datafusion/physical-plan/src/aggregates/mod.rs @@ -1021,12 +1021,21 @@ impl ExecutionPlan for AggregateExec { ..Default::default() } } - _ => Statistics { - // the output row count is surely not larger than its input row count - num_rows: self.input.statistics().num_rows, - is_exact: false, - ..Default::default() - }, + _ => { + let input_stats = self.input.statistics(); + // Input statistics is exact and number of rows not greater than 1: + let is_exact = input_stats.is_exact + && (input_stats + .num_rows + .map(|num_rows| num_rows == 1) + .unwrap_or(false)); + Statistics { + // the output row count is surely not larger than its input row count + num_rows: self.input.statistics().num_rows, + is_exact, + ..Default::default() + } + } } } } diff --git a/datafusion/sqllogictest/test_files/aggregate.slt b/datafusion/sqllogictest/test_files/aggregate.slt index c6f9f42fa2d8..777b634e93b1 100644 --- a/datafusion/sqllogictest/test_files/aggregate.slt +++ b/datafusion/sqllogictest/test_files/aggregate.slt @@ -2342,8 +2342,8 @@ GlobalLimitExec: skip=0, fetch=4 ------AggregateExec: mode=FinalPartitioned, gby=[trace_id@0 as trace_id], aggr=[MAX(traces.timestamp)] --------CoalesceBatchesExec: target_batch_size=8192 ----------RepartitionExec: partitioning=Hash([trace_id@0], 4), input_partitions=4 -------------AggregateExec: mode=Partial, gby=[trace_id@0 as trace_id], aggr=[MAX(traces.timestamp)] ---------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +--------------AggregateExec: mode=Partial, gby=[trace_id@0 as trace_id], aggr=[MAX(traces.timestamp)] ----------------MemoryExec: partitions=1, partition_sizes=[1] @@ -2397,8 +2397,8 @@ GlobalLimitExec: skip=0, fetch=4 ------AggregateExec: mode=FinalPartitioned, gby=[trace_id@0 as trace_id], aggr=[MAX(traces.timestamp)], lim=[4] --------CoalesceBatchesExec: target_batch_size=8192 ----------RepartitionExec: partitioning=Hash([trace_id@0], 4), input_partitions=4 -------------AggregateExec: mode=Partial, gby=[trace_id@0 as trace_id], aggr=[MAX(traces.timestamp)], lim=[4] ---------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +--------------AggregateExec: mode=Partial, gby=[trace_id@0 as trace_id], aggr=[MAX(traces.timestamp)], lim=[4] ----------------MemoryExec: partitions=1, partition_sizes=[1] query TT @@ -2416,8 +2416,8 @@ GlobalLimitExec: skip=0, fetch=4 ------AggregateExec: mode=FinalPartitioned, gby=[trace_id@0 as trace_id], aggr=[MIN(traces.timestamp)] --------CoalesceBatchesExec: target_batch_size=8192 ----------RepartitionExec: partitioning=Hash([trace_id@0], 4), input_partitions=4 -------------AggregateExec: mode=Partial, gby=[trace_id@0 as trace_id], aggr=[MIN(traces.timestamp)] ---------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +--------------AggregateExec: mode=Partial, gby=[trace_id@0 as trace_id], aggr=[MIN(traces.timestamp)] ----------------MemoryExec: partitions=1, partition_sizes=[1] query TT @@ -2435,8 +2435,8 @@ GlobalLimitExec: skip=0, fetch=4 ------AggregateExec: mode=FinalPartitioned, gby=[trace_id@0 as trace_id], aggr=[MAX(traces.timestamp)] --------CoalesceBatchesExec: target_batch_size=8192 ----------RepartitionExec: partitioning=Hash([trace_id@0], 4), input_partitions=4 -------------AggregateExec: mode=Partial, gby=[trace_id@0 as trace_id], aggr=[MAX(traces.timestamp)] ---------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +--------------AggregateExec: mode=Partial, gby=[trace_id@0 as trace_id], aggr=[MAX(traces.timestamp)] ----------------MemoryExec: partitions=1, partition_sizes=[1] query TT @@ -2454,8 +2454,8 @@ GlobalLimitExec: skip=0, fetch=4 ------AggregateExec: mode=FinalPartitioned, gby=[trace_id@0 as trace_id], aggr=[MAX(traces.timestamp)] --------CoalesceBatchesExec: target_batch_size=8192 ----------RepartitionExec: partitioning=Hash([trace_id@0], 4), input_partitions=4 -------------AggregateExec: mode=Partial, gby=[trace_id@0 as trace_id], aggr=[MAX(traces.timestamp)] ---------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +--------------AggregateExec: mode=Partial, gby=[trace_id@0 as trace_id], aggr=[MAX(traces.timestamp)] ----------------MemoryExec: partitions=1, partition_sizes=[1] query TI diff --git a/datafusion/sqllogictest/test_files/copy.slt b/datafusion/sqllogictest/test_files/copy.slt index a41d1fca66a4..f2fe216ee864 100644 --- a/datafusion/sqllogictest/test_files/copy.slt +++ b/datafusion/sqllogictest/test_files/copy.slt @@ -33,7 +33,7 @@ CopyTo: format=parquet output_url=test_files/scratch/copy/table single_file_outp --TableScan: source_table projection=[col1, col2] physical_plan InsertExec: sink=ParquetSink(writer_mode=PutMultipart, file_groups=[]) ---MemoryExec: partitions=4, partition_sizes=[1, 0, 0, 0] +--MemoryExec: partitions=1, partition_sizes=[1] # Error case query error DataFusion error: Invalid or Unsupported Configuration: Format not explicitly set and unable to get file extension! diff --git a/datafusion/sqllogictest/test_files/groupby.slt b/datafusion/sqllogictest/test_files/groupby.slt index b7070a8d7efb..bf93c6633bfc 100644 --- a/datafusion/sqllogictest/test_files/groupby.slt +++ b/datafusion/sqllogictest/test_files/groupby.slt @@ -2024,13 +2024,11 @@ SortPreservingMergeExec: [col0@0 ASC NULLS LAST] ----------------CoalesceBatchesExec: target_batch_size=8192 ------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(col0@0, col0@0)] --------------------CoalesceBatchesExec: target_batch_size=8192 -----------------------RepartitionExec: partitioning=Hash([col0@0], 4), input_partitions=4 -------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 ---------------------------MemoryExec: partitions=1, partition_sizes=[3] +----------------------RepartitionExec: partitioning=Hash([col0@0], 4), input_partitions=1 +------------------------MemoryExec: partitions=1, partition_sizes=[3] --------------------CoalesceBatchesExec: target_batch_size=8192 -----------------------RepartitionExec: partitioning=Hash([col0@0], 4), input_partitions=4 -------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 ---------------------------MemoryExec: partitions=1, partition_sizes=[3] +----------------------RepartitionExec: partitioning=Hash([col0@0], 4), input_partitions=1 +------------------------MemoryExec: partitions=1, partition_sizes=[3] # Columns in the table are a,b,c,d. Source is CsvExec which is ordered by # a,b,c column. Column a has cardinality 2, column b has cardinality 4. @@ -2762,9 +2760,9 @@ SortPreservingMergeExec: [country@0 ASC NULLS LAST] ------AggregateExec: mode=FinalPartitioned, gby=[country@0 as country], aggr=[FIRST_VALUE(sales_global.amount), LAST_VALUE(sales_global.amount)] --------CoalesceBatchesExec: target_batch_size=8192 ----------RepartitionExec: partitioning=Hash([country@0], 8), input_partitions=8 -------------AggregateExec: mode=Partial, gby=[country@0 as country], aggr=[FIRST_VALUE(sales_global.amount), LAST_VALUE(sales_global.amount)] ---------------SortExec: expr=[ts@1 ASC NULLS LAST] -----------------RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1 +------------RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1 +--------------AggregateExec: mode=Partial, gby=[country@0 as country], aggr=[FIRST_VALUE(sales_global.amount), LAST_VALUE(sales_global.amount)] +----------------SortExec: expr=[ts@1 ASC NULLS LAST] ------------------MemoryExec: partitions=1, partition_sizes=[1] query TRR @@ -2799,9 +2797,9 @@ SortPreservingMergeExec: [country@0 ASC NULLS LAST] ------AggregateExec: mode=FinalPartitioned, gby=[country@0 as country], aggr=[FIRST_VALUE(sales_global.amount), FIRST_VALUE(sales_global.amount)] --------CoalesceBatchesExec: target_batch_size=8192 ----------RepartitionExec: partitioning=Hash([country@0], 8), input_partitions=8 -------------AggregateExec: mode=Partial, gby=[country@0 as country], aggr=[FIRST_VALUE(sales_global.amount), FIRST_VALUE(sales_global.amount)] ---------------SortExec: expr=[ts@1 ASC NULLS LAST] -----------------RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1 +------------RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1 +--------------AggregateExec: mode=Partial, gby=[country@0 as country], aggr=[FIRST_VALUE(sales_global.amount), FIRST_VALUE(sales_global.amount)] +----------------SortExec: expr=[ts@1 ASC NULLS LAST] ------------------MemoryExec: partitions=1, partition_sizes=[1] query TRR @@ -2815,6 +2813,11 @@ FRA 50 50 GRC 30 30 TUR 75 75 +# make sure that batch size is small. So that query below runs in multi partitions +# row number of the sales_global is 5. Hence we choose batch size 4 to make is smaller. +statement ok +set datafusion.execution.batch_size = 4; + # order-sensitive FIRST_VALUE and LAST_VALUE aggregators should work in # multi-partitions without group by also. query TT @@ -2958,7 +2961,7 @@ SortPreservingMergeExec: [country@0 ASC NULLS LAST] --SortExec: expr=[country@0 ASC NULLS LAST] ----ProjectionExec: expr=[country@0 as country, ARRAY_AGG(sales_global.amount) ORDER BY [sales_global.amount ASC NULLS LAST]@1 as array_agg1] ------AggregateExec: mode=FinalPartitioned, gby=[country@0 as country], aggr=[ARRAY_AGG(sales_global.amount)] ---------CoalesceBatchesExec: target_batch_size=8192 +--------CoalesceBatchesExec: target_batch_size=4 ----------RepartitionExec: partitioning=Hash([country@0], 8), input_partitions=8 ------------AggregateExec: mode=Partial, gby=[country@0 as country], aggr=[ARRAY_AGG(sales_global.amount)] --------------SortExec: expr=[amount@1 ASC NULLS LAST] @@ -2994,7 +2997,7 @@ SortPreservingMergeExec: [country@0 ASC NULLS LAST] --SortExec: expr=[country@0 ASC NULLS LAST] ----ProjectionExec: expr=[country@0 as country, ARRAY_AGG(sales_global.amount) ORDER BY [sales_global.amount DESC NULLS FIRST]@1 as amounts, FIRST_VALUE(sales_global.amount) ORDER BY [sales_global.amount ASC NULLS LAST]@2 as fv1, LAST_VALUE(sales_global.amount) ORDER BY [sales_global.amount DESC NULLS FIRST]@3 as fv2] ------AggregateExec: mode=FinalPartitioned, gby=[country@0 as country], aggr=[ARRAY_AGG(sales_global.amount), LAST_VALUE(sales_global.amount), LAST_VALUE(sales_global.amount)] ---------CoalesceBatchesExec: target_batch_size=8192 +--------CoalesceBatchesExec: target_batch_size=4 ----------RepartitionExec: partitioning=Hash([country@0], 8), input_partitions=8 ------------AggregateExec: mode=Partial, gby=[country@0 as country], aggr=[ARRAY_AGG(sales_global.amount), LAST_VALUE(sales_global.amount), LAST_VALUE(sales_global.amount)] --------------SortExec: expr=[amount@1 DESC] @@ -3195,7 +3198,7 @@ SortPreservingMergeExec: [sn@0 ASC NULLS LAST] --SortExec: expr=[sn@0 ASC NULLS LAST] ----ProjectionExec: expr=[sn@0 as sn, amount@1 as amount, 2 * CAST(sn@0 AS Int64) as Int64(2) * s.sn] ------AggregateExec: mode=FinalPartitioned, gby=[sn@0 as sn, amount@1 as amount], aggr=[] ---------CoalesceBatchesExec: target_batch_size=8192 +--------CoalesceBatchesExec: target_batch_size=4 ----------RepartitionExec: partitioning=Hash([sn@0, amount@1], 8), input_partitions=8 ------------AggregateExec: mode=Partial, gby=[sn@0 as sn, amount@1 as amount], aggr=[] --------------MemoryExec: partitions=8, partition_sizes=[1, 0, 0, 0, 0, 0, 0, 0] @@ -3248,7 +3251,7 @@ SortPreservingMergeExec: [sn@0 ASC NULLS LAST] --SortExec: expr=[sn@0 ASC NULLS LAST] ----ProjectionExec: expr=[sn@0 as sn, SUM(l.amount)@2 as SUM(l.amount), amount@1 as amount] ------AggregateExec: mode=FinalPartitioned, gby=[sn@0 as sn, amount@1 as amount], aggr=[SUM(l.amount)] ---------CoalesceBatchesExec: target_batch_size=8192 +--------CoalesceBatchesExec: target_batch_size=4 ----------RepartitionExec: partitioning=Hash([sn@0, amount@1], 8), input_partitions=8 ------------AggregateExec: mode=Partial, gby=[sn@1 as sn, amount@2 as amount], aggr=[SUM(l.amount)] --------------ProjectionExec: expr=[amount@1 as amount, sn@2 as sn, amount@3 as amount] @@ -3396,7 +3399,7 @@ SortPreservingMergeExec: [sn@2 ASC NULLS LAST] --SortExec: expr=[sn@2 ASC NULLS LAST] ----ProjectionExec: expr=[zip_code@1 as zip_code, country@2 as country, sn@0 as sn, ts@3 as ts, currency@4 as currency, amount@5 as amount, sum_amount@6 as sum_amount] ------AggregateExec: mode=FinalPartitioned, gby=[sn@0 as sn, zip_code@1 as zip_code, country@2 as country, ts@3 as ts, currency@4 as currency, amount@5 as amount, sum_amount@6 as sum_amount], aggr=[] ---------CoalesceBatchesExec: target_batch_size=8192 +--------CoalesceBatchesExec: target_batch_size=4 ----------RepartitionExec: partitioning=Hash([sn@0, zip_code@1, country@2, ts@3, currency@4, amount@5, sum_amount@6], 8), input_partitions=8 ------------AggregateExec: mode=Partial, gby=[sn@2 as sn, zip_code@0 as zip_code, country@1 as country, ts@3 as ts, currency@4 as currency, amount@5 as amount, sum_amount@6 as sum_amount], aggr=[] --------------RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1 @@ -3566,6 +3569,12 @@ ORDER BY y; 2 1 3 1 +# Make sure to choose a batch size smaller than, row number of the table. +# In this case we choose 2 (Row number of the table is 3). +# otherwise we won't see parallelism in tests. +statement ok +set datafusion.execution.batch_size = 2; + # plan of the query above should contain partial # and final aggregation stages query TT @@ -3579,7 +3588,8 @@ physical_plan AggregateExec: mode=Final, gby=[], aggr=[LAST_VALUE(foo.x)] --CoalescePartitionsExec ----AggregateExec: mode=Partial, gby=[], aggr=[LAST_VALUE(foo.x)] -------MemoryExec: partitions=8, partition_sizes=[1, 0, 0, 0, 0, 0, 0, 0] +------RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1 +--------MemoryExec: partitions=1, partition_sizes=[1] query I SELECT FIRST_VALUE(x) @@ -3600,7 +3610,8 @@ physical_plan AggregateExec: mode=Final, gby=[], aggr=[FIRST_VALUE(foo.x)] --CoalescePartitionsExec ----AggregateExec: mode=Partial, gby=[], aggr=[FIRST_VALUE(foo.x)] -------MemoryExec: partitions=8, partition_sizes=[1, 0, 0, 0, 0, 0, 0, 0] +------RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1 +--------MemoryExec: partitions=1, partition_sizes=[1] query TT @@ -3644,7 +3655,7 @@ Aggregate: groupBy=[[multiple_ordered_table_with_pk.c, multiple_ordered_table_wi physical_plan AggregateExec: mode=FinalPartitioned, gby=[c@0 as c, b@1 as b], aggr=[SUM(multiple_ordered_table_with_pk.d)], ordering_mode=PartiallyOrdered --SortExec: expr=[c@0 ASC NULLS LAST] -----CoalesceBatchesExec: target_batch_size=8192 +----CoalesceBatchesExec: target_batch_size=2 ------RepartitionExec: partitioning=Hash([c@0, b@1], 8), input_partitions=8 --------AggregateExec: mode=Partial, gby=[c@1 as c, b@0 as b], aggr=[SUM(multiple_ordered_table_with_pk.d)], ordering_mode=PartiallyOrdered ----------RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1 @@ -3685,7 +3696,7 @@ Aggregate: groupBy=[[multiple_ordered_table_with_pk.c, multiple_ordered_table_wi physical_plan AggregateExec: mode=FinalPartitioned, gby=[c@0 as c, b@1 as b], aggr=[SUM(multiple_ordered_table_with_pk.d)], ordering_mode=PartiallyOrdered --SortExec: expr=[c@0 ASC NULLS LAST] -----CoalesceBatchesExec: target_batch_size=8192 +----CoalesceBatchesExec: target_batch_size=2 ------RepartitionExec: partitioning=Hash([c@0, b@1], 8), input_partitions=8 --------AggregateExec: mode=Partial, gby=[c@1 as c, b@0 as b], aggr=[SUM(multiple_ordered_table_with_pk.d)], ordering_mode=PartiallyOrdered ----------RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1 diff --git a/datafusion/sqllogictest/test_files/join.slt b/datafusion/sqllogictest/test_files/join.slt index 283ff57a984c..874d849e9a29 100644 --- a/datafusion/sqllogictest/test_files/join.slt +++ b/datafusion/sqllogictest/test_files/join.slt @@ -558,6 +558,10 @@ explain select * from t1 join t2 on false; logical_plan EmptyRelation physical_plan EmptyExec: produce_one_row=false +# Make batch size smaller than table row number. to introduce parallelism to the plan. +statement ok +set datafusion.execution.batch_size = 1; + # test covert inner join to cross join when condition is true query TT explain select * from t1 inner join t2 on true; @@ -568,9 +572,9 @@ CrossJoin: --TableScan: t2 projection=[t2_id, t2_name, t2_int] physical_plan CrossJoinExec ---CoalescePartitionsExec -----MemoryExec: partitions=4, partition_sizes=[1, 0, 0, 0] ---MemoryExec: partitions=4, partition_sizes=[1, 0, 0, 0] +--MemoryExec: partitions=1, partition_sizes=[1] +--RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +----MemoryExec: partitions=1, partition_sizes=[1] statement ok drop table IF EXISTS t1; diff --git a/datafusion/sqllogictest/test_files/joins.slt b/datafusion/sqllogictest/test_files/joins.slt index 01c0131fdb62..cc90e6431389 100644 --- a/datafusion/sqllogictest/test_files/joins.slt +++ b/datafusion/sqllogictest/test_files/joins.slt @@ -24,7 +24,7 @@ statement ok set datafusion.execution.target_partitions = 2; statement ok -set datafusion.execution.batch_size = 4096; +set datafusion.execution.batch_size = 2; statement ok set datafusion.explain.logical_plan_only = true; @@ -185,6 +185,10 @@ FROM statement ok set datafusion.execution.target_partitions = 2; +# make sure to a batch size smaller than row number of the table. +statement ok +set datafusion.execution.batch_size = 2; + ########## ## Joins Tests ########## @@ -1311,13 +1315,13 @@ Aggregate: groupBy=[[join_t1.t1_id]], aggr=[[]] physical_plan AggregateExec: mode=SinglePartitioned, gby=[t1_id@0 as t1_id], aggr=[] --ProjectionExec: expr=[t1_id@0 as t1_id] -----CoalesceBatchesExec: target_batch_size=4096 +----CoalesceBatchesExec: target_batch_size=2 ------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(t1_id@0, t2_id@0)] ---------CoalesceBatchesExec: target_batch_size=4096 +--------CoalesceBatchesExec: target_batch_size=2 ----------RepartitionExec: partitioning=Hash([t1_id@0], 2), input_partitions=2 ------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 --------------MemoryExec: partitions=1, partition_sizes=[1] ---------CoalesceBatchesExec: target_batch_size=4096 +--------CoalesceBatchesExec: target_batch_size=2 ----------RepartitionExec: partitioning=Hash([t2_id@0], 2), input_partitions=2 ------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 --------------MemoryExec: partitions=1, partition_sizes=[1] @@ -1339,13 +1343,13 @@ physical_plan ProjectionExec: expr=[COUNT(*)@1 as COUNT(*)] --AggregateExec: mode=SinglePartitioned, gby=[t1_id@0 as t1_id], aggr=[COUNT(*)] ----ProjectionExec: expr=[t1_id@0 as t1_id] -------CoalesceBatchesExec: target_batch_size=4096 +------CoalesceBatchesExec: target_batch_size=2 --------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(t1_id@0, t2_id@0)] -----------CoalesceBatchesExec: target_batch_size=4096 +----------CoalesceBatchesExec: target_batch_size=2 ------------RepartitionExec: partitioning=Hash([t1_id@0], 2), input_partitions=2 --------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 ----------------MemoryExec: partitions=1, partition_sizes=[1] -----------CoalesceBatchesExec: target_batch_size=4096 +----------CoalesceBatchesExec: target_batch_size=2 ------------RepartitionExec: partitioning=Hash([t2_id@0], 2), input_partitions=2 --------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 ----------------MemoryExec: partitions=1, partition_sizes=[1] @@ -1372,13 +1376,13 @@ ProjectionExec: expr=[COUNT(alias1)@0 as COUNT(DISTINCT join_t1.t1_id)] --------AggregateExec: mode=FinalPartitioned, gby=[alias1@0 as alias1], aggr=[] ----------AggregateExec: mode=Partial, gby=[t1_id@0 as alias1], aggr=[] ------------ProjectionExec: expr=[t1_id@0 as t1_id] ---------------CoalesceBatchesExec: target_batch_size=4096 +--------------CoalesceBatchesExec: target_batch_size=2 ----------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(t1_id@0, t2_id@0)] -------------------CoalesceBatchesExec: target_batch_size=4096 +------------------CoalesceBatchesExec: target_batch_size=2 --------------------RepartitionExec: partitioning=Hash([t1_id@0], 2), input_partitions=2 ----------------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 ------------------------MemoryExec: partitions=1, partition_sizes=[1] -------------------CoalesceBatchesExec: target_batch_size=4096 +------------------CoalesceBatchesExec: target_batch_size=2 --------------------RepartitionExec: partitioning=Hash([t2_id@0], 2), input_partitions=2 ----------------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 ------------------------MemoryExec: partitions=1, partition_sizes=[1] @@ -1441,7 +1445,7 @@ Projection: join_t1.t1_id, join_t1.t1_name, join_t1.t1_int, join_t2.t2_id, join_ physical_plan ProjectionExec: expr=[t1_id@0 as t1_id, t1_name@1 as t1_name, t1_int@2 as t1_int, t2_id@3 as t2_id, t2_name@4 as t2_name, t2_int@5 as t2_int, CAST(t1_id@0 AS Int64) + 11 as join_t1.t1_id + Int64(11)] --ProjectionExec: expr=[t1_id@0 as t1_id, t1_name@1 as t1_name, t1_int@2 as t1_int, t2_id@4 as t2_id, t2_name@5 as t2_name, t2_int@6 as t2_int] -----CoalesceBatchesExec: target_batch_size=4096 +----CoalesceBatchesExec: target_batch_size=2 ------HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(join_t1.t1_id + Int64(11)@3, CAST(join_t2.t2_id AS Int64)@3)] --------CoalescePartitionsExec ----------ProjectionExec: expr=[t1_id@0 as t1_id, t1_name@1 as t1_name, t1_int@2 as t1_int, CAST(t1_id@0 AS Int64) + 11 as join_t1.t1_id + Int64(11)] @@ -1468,14 +1472,14 @@ Projection: join_t1.t1_id, join_t1.t1_name, join_t1.t1_int, join_t2.t2_id, join_ physical_plan ProjectionExec: expr=[t1_id@0 as t1_id, t1_name@1 as t1_name, t1_int@2 as t1_int, t2_id@3 as t2_id, t2_name@4 as t2_name, t2_int@5 as t2_int, CAST(t1_id@0 AS Int64) + 11 as join_t1.t1_id + Int64(11)] --ProjectionExec: expr=[t1_id@0 as t1_id, t1_name@1 as t1_name, t1_int@2 as t1_int, t2_id@4 as t2_id, t2_name@5 as t2_name, t2_int@6 as t2_int] -----CoalesceBatchesExec: target_batch_size=4096 +----CoalesceBatchesExec: target_batch_size=2 ------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(join_t1.t1_id + Int64(11)@3, CAST(join_t2.t2_id AS Int64)@3)] ---------CoalesceBatchesExec: target_batch_size=4096 +--------CoalesceBatchesExec: target_batch_size=2 ----------RepartitionExec: partitioning=Hash([join_t1.t1_id + Int64(11)@3], 2), input_partitions=2 ------------ProjectionExec: expr=[t1_id@0 as t1_id, t1_name@1 as t1_name, t1_int@2 as t1_int, CAST(t1_id@0 AS Int64) + 11 as join_t1.t1_id + Int64(11)] --------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 ----------------MemoryExec: partitions=1, partition_sizes=[1] ---------CoalesceBatchesExec: target_batch_size=4096 +--------CoalesceBatchesExec: target_batch_size=2 ----------RepartitionExec: partitioning=Hash([CAST(join_t2.t2_id AS Int64)@3], 2), input_partitions=2 ------------ProjectionExec: expr=[t2_id@0 as t2_id, t2_name@1 as t2_name, t2_int@2 as t2_int, CAST(t2_id@0 AS Int64) as CAST(join_t2.t2_id AS Int64)] --------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 @@ -1501,7 +1505,7 @@ physical_plan ProjectionExec: expr=[t1_id@0 as t1_id, t2_id@2 as t2_id, t1_name@1 as t1_name] --ProjectionExec: expr=[t1_id@0 as t1_id, t1_name@1 as t1_name, t2_id@3 as t2_id] ----ProjectionExec: expr=[t1_id@2 as t1_id, t1_name@3 as t1_name, join_t1.t1_id + UInt32(12)@4 as join_t1.t1_id + UInt32(12), t2_id@0 as t2_id, join_t2.t2_id + UInt32(1)@1 as join_t2.t2_id + UInt32(1)] -------CoalesceBatchesExec: target_batch_size=4096 +------CoalesceBatchesExec: target_batch_size=2 --------HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(join_t2.t2_id + UInt32(1)@1, join_t1.t1_id + UInt32(12)@2)] ----------CoalescePartitionsExec ------------ProjectionExec: expr=[t2_id@0 as t2_id, t2_id@0 + 1 as join_t2.t2_id + UInt32(1)] @@ -1529,14 +1533,14 @@ physical_plan ProjectionExec: expr=[t1_id@0 as t1_id, t2_id@2 as t2_id, t1_name@1 as t1_name] --ProjectionExec: expr=[t1_id@0 as t1_id, t1_name@1 as t1_name, t2_id@3 as t2_id] ----ProjectionExec: expr=[t1_id@2 as t1_id, t1_name@3 as t1_name, join_t1.t1_id + UInt32(12)@4 as join_t1.t1_id + UInt32(12), t2_id@0 as t2_id, join_t2.t2_id + UInt32(1)@1 as join_t2.t2_id + UInt32(1)] -------CoalesceBatchesExec: target_batch_size=4096 +------CoalesceBatchesExec: target_batch_size=2 --------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(join_t2.t2_id + UInt32(1)@1, join_t1.t1_id + UInt32(12)@2)] -----------CoalesceBatchesExec: target_batch_size=4096 +----------CoalesceBatchesExec: target_batch_size=2 ------------RepartitionExec: partitioning=Hash([join_t2.t2_id + UInt32(1)@1], 2), input_partitions=2 --------------ProjectionExec: expr=[t2_id@0 as t2_id, t2_id@0 + 1 as join_t2.t2_id + UInt32(1)] ----------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 ------------------MemoryExec: partitions=1, partition_sizes=[1] -----------CoalesceBatchesExec: target_batch_size=4096 +----------CoalesceBatchesExec: target_batch_size=2 ------------RepartitionExec: partitioning=Hash([join_t1.t1_id + UInt32(12)@2], 2), input_partitions=2 --------------ProjectionExec: expr=[t1_id@0 as t1_id, t1_name@1 as t1_name, t1_id@0 + 12 as join_t1.t1_id + UInt32(12)] ----------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 @@ -1562,7 +1566,7 @@ Projection: join_t1.t1_id, join_t2.t2_id, join_t1.t1_name physical_plan ProjectionExec: expr=[t1_id@0 as t1_id, t2_id@2 as t2_id, t1_name@1 as t1_name] --ProjectionExec: expr=[t1_id@0 as t1_id, t1_name@1 as t1_name, t2_id@3 as t2_id] -----CoalesceBatchesExec: target_batch_size=4096 +----CoalesceBatchesExec: target_batch_size=2 ------HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(join_t1.t1_id + UInt32(11)@2, t2_id@0)] --------CoalescePartitionsExec ----------ProjectionExec: expr=[t1_id@0 as t1_id, t1_name@1 as t1_name, t1_id@0 + 11 as join_t1.t1_id + UInt32(11)] @@ -1589,14 +1593,14 @@ Projection: join_t1.t1_id, join_t2.t2_id, join_t1.t1_name physical_plan ProjectionExec: expr=[t1_id@0 as t1_id, t2_id@2 as t2_id, t1_name@1 as t1_name] --ProjectionExec: expr=[t1_id@0 as t1_id, t1_name@1 as t1_name, t2_id@3 as t2_id] -----CoalesceBatchesExec: target_batch_size=4096 +----CoalesceBatchesExec: target_batch_size=2 ------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(join_t1.t1_id + UInt32(11)@2, t2_id@0)] ---------CoalesceBatchesExec: target_batch_size=4096 +--------CoalesceBatchesExec: target_batch_size=2 ----------RepartitionExec: partitioning=Hash([join_t1.t1_id + UInt32(11)@2], 2), input_partitions=2 ------------ProjectionExec: expr=[t1_id@0 as t1_id, t1_name@1 as t1_name, t1_id@0 + 11 as join_t1.t1_id + UInt32(11)] --------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 ----------------MemoryExec: partitions=1, partition_sizes=[1] ---------CoalesceBatchesExec: target_batch_size=4096 +--------CoalesceBatchesExec: target_batch_size=2 ----------RepartitionExec: partitioning=Hash([t2_id@0], 2), input_partitions=2 ------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 --------------MemoryExec: partitions=1, partition_sizes=[1] @@ -1622,7 +1626,7 @@ physical_plan ProjectionExec: expr=[t1_id@0 as t1_id, t2_id@2 as t2_id, t1_name@1 as t1_name] --ProjectionExec: expr=[t1_id@0 as t1_id, t1_name@1 as t1_name, t2_id@2 as t2_id] ----ProjectionExec: expr=[t1_id@2 as t1_id, t1_name@3 as t1_name, t2_id@0 as t2_id, join_t2.t2_id - UInt32(11)@1 as join_t2.t2_id - UInt32(11)] -------CoalesceBatchesExec: target_batch_size=4096 +------CoalesceBatchesExec: target_batch_size=2 --------HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(join_t2.t2_id - UInt32(11)@1, t1_id@0)] ----------CoalescePartitionsExec ------------ProjectionExec: expr=[t2_id@0 as t2_id, t2_id@0 - 11 as join_t2.t2_id - UInt32(11)] @@ -1650,14 +1654,14 @@ physical_plan ProjectionExec: expr=[t1_id@0 as t1_id, t2_id@2 as t2_id, t1_name@1 as t1_name] --ProjectionExec: expr=[t1_id@0 as t1_id, t1_name@1 as t1_name, t2_id@2 as t2_id] ----ProjectionExec: expr=[t1_id@2 as t1_id, t1_name@3 as t1_name, t2_id@0 as t2_id, join_t2.t2_id - UInt32(11)@1 as join_t2.t2_id - UInt32(11)] -------CoalesceBatchesExec: target_batch_size=4096 +------CoalesceBatchesExec: target_batch_size=2 --------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(join_t2.t2_id - UInt32(11)@1, t1_id@0)] -----------CoalesceBatchesExec: target_batch_size=4096 +----------CoalesceBatchesExec: target_batch_size=2 ------------RepartitionExec: partitioning=Hash([join_t2.t2_id - UInt32(11)@1], 2), input_partitions=2 --------------ProjectionExec: expr=[t2_id@0 as t2_id, t2_id@0 - 11 as join_t2.t2_id - UInt32(11)] ----------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 ------------------MemoryExec: partitions=1, partition_sizes=[1] -----------CoalesceBatchesExec: target_batch_size=4096 +----------CoalesceBatchesExec: target_batch_size=2 ------------RepartitionExec: partitioning=Hash([t1_id@0], 2), input_partitions=2 --------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 ----------------MemoryExec: partitions=1, partition_sizes=[1] @@ -1680,7 +1684,7 @@ Inner Join: join_t1.t1_id = join_t2.t2_id - UInt32(11) --TableScan: join_t2 projection=[t2_id, t2_name, t2_int] physical_plan ProjectionExec: expr=[t1_id@0 as t1_id, t1_name@1 as t1_name, t1_int@2 as t1_int, t2_id@3 as t2_id, t2_name@4 as t2_name, t2_int@5 as t2_int] ---CoalesceBatchesExec: target_batch_size=4096 +--CoalesceBatchesExec: target_batch_size=2 ----HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(t1_id@0, join_t2.t2_id - UInt32(11)@3)] ------MemoryExec: partitions=1, partition_sizes=[1] ------ProjectionExec: expr=[t2_id@0 as t2_id, t2_name@1 as t2_name, t2_int@2 as t2_int, t2_id@0 - 11 as join_t2.t2_id - UInt32(11)] @@ -1703,13 +1707,13 @@ Inner Join: join_t1.t1_id = join_t2.t2_id - UInt32(11) --TableScan: join_t2 projection=[t2_id, t2_name, t2_int] physical_plan ProjectionExec: expr=[t1_id@0 as t1_id, t1_name@1 as t1_name, t1_int@2 as t1_int, t2_id@3 as t2_id, t2_name@4 as t2_name, t2_int@5 as t2_int] ---CoalesceBatchesExec: target_batch_size=4096 +--CoalesceBatchesExec: target_batch_size=2 ----HashJoinExec: mode=Partitioned, join_type=Inner, on=[(t1_id@0, join_t2.t2_id - UInt32(11)@3)] -------CoalesceBatchesExec: target_batch_size=4096 +------CoalesceBatchesExec: target_batch_size=2 --------RepartitionExec: partitioning=Hash([t1_id@0], 2), input_partitions=2 ----------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 ------------MemoryExec: partitions=1, partition_sizes=[1] -------CoalesceBatchesExec: target_batch_size=4096 +------CoalesceBatchesExec: target_batch_size=2 --------RepartitionExec: partitioning=Hash([join_t2.t2_id - UInt32(11)@3], 2), input_partitions=2 ----------ProjectionExec: expr=[t2_id@0 as t2_id, t2_name@1 as t2_name, t2_int@2 as t2_int, t2_id@0 - 11 as join_t2.t2_id - UInt32(11)] ------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 @@ -2032,13 +2036,13 @@ Inner Join: Filter: join_t1.t1_id > join_t2.t2_id ------TableScan: join_t2 projection=[t2_id, t2_int] physical_plan NestedLoopJoinExec: join_type=Inner, filter=t1_id@0 > t2_id@1 ---CoalesceBatchesExec: target_batch_size=4096 +--CoalesceBatchesExec: target_batch_size=2 ----FilterExec: t1_id@0 > 10 ------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 --------MemoryExec: partitions=1, partition_sizes=[1] --CoalescePartitionsExec ----ProjectionExec: expr=[t2_id@0 as t2_id] -------CoalesceBatchesExec: target_batch_size=4096 +------CoalesceBatchesExec: target_batch_size=2 --------FilterExec: t2_int@1 > 1 ----------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 ------------MemoryExec: partitions=1, partition_sizes=[1] @@ -2073,11 +2077,11 @@ Right Join: Filter: join_t1.t1_id < join_t2.t2_id physical_plan NestedLoopJoinExec: join_type=Right, filter=t1_id@0 < t2_id@1 --CoalescePartitionsExec -----CoalesceBatchesExec: target_batch_size=4096 +----CoalesceBatchesExec: target_batch_size=2 ------FilterExec: t1_id@0 > 22 --------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 ----------MemoryExec: partitions=1, partition_sizes=[1] ---CoalesceBatchesExec: target_batch_size=4096 +--CoalesceBatchesExec: target_batch_size=2 ----FilterExec: t2_id@0 > 11 ------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 --------MemoryExec: partitions=1, partition_sizes=[1] @@ -2643,7 +2647,7 @@ statement ok set datafusion.execution.target_partitions = 2; statement ok -set datafusion.execution.batch_size = 4096; +set datafusion.execution.batch_size = 2; # explain sort_merge_join_on_date32 inner sort merge join on data type (Date32) query TT @@ -2658,12 +2662,12 @@ Inner Join: t1.c1 = t2.c1 physical_plan SortMergeJoin: join_type=Inner, on=[(c1@0, c1@0)] --SortExec: expr=[c1@0 ASC] -----CoalesceBatchesExec: target_batch_size=4096 +----CoalesceBatchesExec: target_batch_size=2 ------RepartitionExec: partitioning=Hash([c1@0], 2), input_partitions=2 --------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 ----------MemoryExec: partitions=1, partition_sizes=[1] --SortExec: expr=[c1@0 ASC] -----CoalesceBatchesExec: target_batch_size=4096 +----CoalesceBatchesExec: target_batch_size=2 ------RepartitionExec: partitioning=Hash([c1@0], 2), input_partitions=2 --------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 ----------MemoryExec: partitions=1, partition_sizes=[1] @@ -2689,13 +2693,13 @@ physical_plan ProjectionExec: expr=[c1@0 as c1, c2@1 as c2, c3@2 as c3, c4@3 as c4, c1@5 as c1, c2@6 as c2, c3@7 as c3, c4@8 as c4] --SortMergeJoin: join_type=Right, on=[(CAST(t1.c3 AS Decimal128(10, 2))@4, c3@2)] ----SortExec: expr=[CAST(t1.c3 AS Decimal128(10, 2))@4 ASC] -------CoalesceBatchesExec: target_batch_size=4096 +------CoalesceBatchesExec: target_batch_size=2 --------RepartitionExec: partitioning=Hash([CAST(t1.c3 AS Decimal128(10, 2))@4], 2), input_partitions=2 ----------ProjectionExec: expr=[c1@0 as c1, c2@1 as c2, c3@2 as c3, c4@3 as c4, CAST(c3@2 AS Decimal128(10, 2)) as CAST(t1.c3 AS Decimal128(10, 2))] ------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 --------------MemoryExec: partitions=1, partition_sizes=[1] ----SortExec: expr=[c3@2 ASC] -------CoalesceBatchesExec: target_batch_size=4096 +------CoalesceBatchesExec: target_batch_size=2 --------RepartitionExec: partitioning=Hash([c3@2], 2), input_partitions=2 ----------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 ------------MemoryExec: partitions=1, partition_sizes=[1] @@ -2722,7 +2726,7 @@ statement ok set datafusion.execution.target_partitions = 2; statement ok -set datafusion.execution.batch_size = 4096; +set datafusion.execution.batch_size = 2; @@ -2743,7 +2747,7 @@ statement ok set datafusion.execution.target_partitions = 2; statement ok -set datafusion.execution.batch_size = 4096; +set datafusion.execution.batch_size = 2; query TT explain SELECT t1_id, t1_name FROM left_semi_anti_join_table_t1 t1 WHERE t1_id IN (SELECT t2_id FROM left_semi_anti_join_table_t2 t2) ORDER BY t1_id @@ -2751,13 +2755,13 @@ explain SELECT t1_id, t1_name FROM left_semi_anti_join_table_t1 t1 WHERE t1_id I physical_plan SortPreservingMergeExec: [t1_id@0 ASC NULLS LAST] --SortExec: expr=[t1_id@0 ASC NULLS LAST] -----CoalesceBatchesExec: target_batch_size=4096 +----CoalesceBatchesExec: target_batch_size=2 ------HashJoinExec: mode=Partitioned, join_type=LeftSemi, on=[(t1_id@0, t2_id@0)] ---------CoalesceBatchesExec: target_batch_size=4096 +--------CoalesceBatchesExec: target_batch_size=2 ----------RepartitionExec: partitioning=Hash([t1_id@0], 2), input_partitions=2 ------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 --------------MemoryExec: partitions=1, partition_sizes=[1] ---------CoalesceBatchesExec: target_batch_size=4096 +--------CoalesceBatchesExec: target_batch_size=2 ----------RepartitionExec: partitioning=Hash([t2_id@0], 2), input_partitions=2 ------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 --------------MemoryExec: partitions=1, partition_sizes=[1] @@ -2792,13 +2796,13 @@ explain SELECT t1_id, t1_name FROM left_semi_anti_join_table_t1 t1 LEFT SEMI JOI physical_plan SortPreservingMergeExec: [t1_id@0 ASC NULLS LAST] --SortExec: expr=[t1_id@0 ASC NULLS LAST] -----CoalesceBatchesExec: target_batch_size=4096 +----CoalesceBatchesExec: target_batch_size=2 ------HashJoinExec: mode=Partitioned, join_type=LeftSemi, on=[(t1_id@0, t2_id@0)] ---------CoalesceBatchesExec: target_batch_size=4096 +--------CoalesceBatchesExec: target_batch_size=2 ----------RepartitionExec: partitioning=Hash([t1_id@0], 2), input_partitions=2 ------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 --------------MemoryExec: partitions=1, partition_sizes=[1] ---------CoalesceBatchesExec: target_batch_size=4096 +--------CoalesceBatchesExec: target_batch_size=2 ----------RepartitionExec: partitioning=Hash([t2_id@0], 2), input_partitions=2 ------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 --------------MemoryExec: partitions=1, partition_sizes=[1] @@ -2827,7 +2831,7 @@ statement ok set datafusion.execution.target_partitions = 2; statement ok -set datafusion.execution.batch_size = 4096; +set datafusion.execution.batch_size = 2; #Test the left_semi_join scenarios where the current repartition_joins parameter is set to false . #### @@ -2846,7 +2850,7 @@ statement ok set datafusion.execution.target_partitions = 2; statement ok -set datafusion.execution.batch_size = 4096; +set datafusion.execution.batch_size = 2; query TT explain SELECT t1_id, t1_name FROM left_semi_anti_join_table_t1 t1 WHERE t1_id IN (SELECT t2_id FROM left_semi_anti_join_table_t2 t2) ORDER BY t1_id @@ -2854,7 +2858,7 @@ explain SELECT t1_id, t1_name FROM left_semi_anti_join_table_t1 t1 WHERE t1_id I physical_plan SortPreservingMergeExec: [t1_id@0 ASC NULLS LAST] --SortExec: expr=[t1_id@0 ASC NULLS LAST] -----CoalesceBatchesExec: target_batch_size=4096 +----CoalesceBatchesExec: target_batch_size=2 ------HashJoinExec: mode=CollectLeft, join_type=LeftSemi, on=[(t1_id@0, t2_id@0)] --------MemoryExec: partitions=1, partition_sizes=[1] --------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 @@ -2890,7 +2894,7 @@ explain SELECT t1_id, t1_name FROM left_semi_anti_join_table_t1 t1 LEFT SEMI JOI physical_plan SortPreservingMergeExec: [t1_id@0 ASC NULLS LAST] --SortExec: expr=[t1_id@0 ASC NULLS LAST] -----CoalesceBatchesExec: target_batch_size=4096 +----CoalesceBatchesExec: target_batch_size=2 ------HashJoinExec: mode=CollectLeft, join_type=LeftSemi, on=[(t1_id@0, t2_id@0)] --------MemoryExec: partitions=1, partition_sizes=[1] --------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 @@ -2920,7 +2924,7 @@ statement ok set datafusion.execution.target_partitions = 2; statement ok -set datafusion.execution.batch_size = 4096; +set datafusion.execution.batch_size = 2; #Test the right_semi_join scenarios where the current repartition_joins parameter is set to true . @@ -2940,7 +2944,7 @@ statement ok set datafusion.execution.target_partitions = 2; statement ok -set datafusion.execution.batch_size = 4096; +set datafusion.execution.batch_size = 2; query TT explain SELECT t1_id, t1_name, t1_int FROM right_semi_anti_join_table_t1 t1 WHERE EXISTS (SELECT * FROM right_semi_anti_join_table_t2 t2 where t2.t2_id = t1.t1_id and t2.t2_name <> t1.t1_name) ORDER BY t1_id @@ -2948,13 +2952,13 @@ explain SELECT t1_id, t1_name, t1_int FROM right_semi_anti_join_table_t1 t1 WHER physical_plan SortPreservingMergeExec: [t1_id@0 ASC NULLS LAST] --SortExec: expr=[t1_id@0 ASC NULLS LAST] -----CoalesceBatchesExec: target_batch_size=4096 +----CoalesceBatchesExec: target_batch_size=2 ------HashJoinExec: mode=Partitioned, join_type=RightSemi, on=[(t2_id@0, t1_id@0)], filter=t2_name@1 != t1_name@0 ---------CoalesceBatchesExec: target_batch_size=4096 +--------CoalesceBatchesExec: target_batch_size=2 ----------RepartitionExec: partitioning=Hash([t2_id@0], 2), input_partitions=2 ------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 --------------MemoryExec: partitions=1, partition_sizes=[1] ---------CoalesceBatchesExec: target_batch_size=4096 +--------CoalesceBatchesExec: target_batch_size=2 ----------RepartitionExec: partitioning=Hash([t1_id@0], 2), input_partitions=2 ------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 --------------MemoryExec: partitions=1, partition_sizes=[1] @@ -2970,13 +2974,13 @@ explain SELECT t1_id, t1_name, t1_int FROM right_semi_anti_join_table_t2 t2 RIGH physical_plan SortPreservingMergeExec: [t1_id@0 ASC NULLS LAST] --SortExec: expr=[t1_id@0 ASC NULLS LAST] -----CoalesceBatchesExec: target_batch_size=4096 +----CoalesceBatchesExec: target_batch_size=2 ------HashJoinExec: mode=Partitioned, join_type=RightSemi, on=[(t2_id@0, t1_id@0)], filter=t2_name@0 != t1_name@1 ---------CoalesceBatchesExec: target_batch_size=4096 +--------CoalesceBatchesExec: target_batch_size=2 ----------RepartitionExec: partitioning=Hash([t2_id@0], 2), input_partitions=2 ------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 --------------MemoryExec: partitions=1, partition_sizes=[1] ---------CoalesceBatchesExec: target_batch_size=4096 +--------CoalesceBatchesExec: target_batch_size=2 ----------RepartitionExec: partitioning=Hash([t1_id@0], 2), input_partitions=2 ------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 --------------MemoryExec: partitions=1, partition_sizes=[1] @@ -3002,7 +3006,7 @@ statement ok set datafusion.execution.target_partitions = 2; statement ok -set datafusion.execution.batch_size = 4096; +set datafusion.execution.batch_size = 2; #Test the right_semi_join scenarios where the current repartition_joins parameter is set to false . @@ -3022,7 +3026,7 @@ statement ok set datafusion.execution.target_partitions = 2; statement ok -set datafusion.execution.batch_size = 4096; +set datafusion.execution.batch_size = 2; query TT explain SELECT t1_id, t1_name, t1_int FROM right_semi_anti_join_table_t1 t1 WHERE EXISTS (SELECT * FROM right_semi_anti_join_table_t2 t2 where t2.t2_id = t1.t1_id and t2.t2_name <> t1.t1_name) ORDER BY t1_id @@ -3030,7 +3034,7 @@ explain SELECT t1_id, t1_name, t1_int FROM right_semi_anti_join_table_t1 t1 WHER physical_plan SortPreservingMergeExec: [t1_id@0 ASC NULLS LAST] --SortExec: expr=[t1_id@0 ASC NULLS LAST] -----CoalesceBatchesExec: target_batch_size=4096 +----CoalesceBatchesExec: target_batch_size=2 ------HashJoinExec: mode=CollectLeft, join_type=RightSemi, on=[(t2_id@0, t1_id@0)], filter=t2_name@1 != t1_name@0 --------MemoryExec: partitions=1, partition_sizes=[1] --------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 @@ -3047,7 +3051,7 @@ explain SELECT t1_id, t1_name, t1_int FROM right_semi_anti_join_table_t2 t2 RIGH physical_plan SortPreservingMergeExec: [t1_id@0 ASC NULLS LAST] --SortExec: expr=[t1_id@0 ASC NULLS LAST] -----CoalesceBatchesExec: target_batch_size=4096 +----CoalesceBatchesExec: target_batch_size=2 ------HashJoinExec: mode=CollectLeft, join_type=RightSemi, on=[(t2_id@0, t1_id@0)], filter=t2_name@0 != t1_name@1 --------MemoryExec: partitions=1, partition_sizes=[1] --------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 @@ -3074,7 +3078,7 @@ statement ok set datafusion.execution.target_partitions = 2; statement ok -set datafusion.execution.batch_size = 4096; +set datafusion.execution.batch_size = 2; #### @@ -3126,14 +3130,14 @@ physical_plan SortPreservingMergeExec: [rn1@5 ASC NULLS LAST] --SortMergeJoin: join_type=Inner, on=[(a@1, a@1)] ----SortExec: expr=[rn1@5 ASC NULLS LAST] -------CoalesceBatchesExec: target_batch_size=4096 +------CoalesceBatchesExec: target_batch_size=2 --------RepartitionExec: partitioning=Hash([a@1], 2), input_partitions=2 ----------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 ------------ProjectionExec: expr=[a0@0 as a0, a@1 as a, b@2 as b, c@3 as c, d@4 as d, ROW_NUMBER() ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING@5 as rn1] --------------BoundedWindowAggExec: wdw=[ROW_NUMBER() ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING: Ok(Field { name: "ROW_NUMBER() ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING", data_type: UInt64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(NULL)), end_bound: Following(UInt64(NULL)) }], mode=[Sorted] ----------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a0, a, b, c, d], output_ordering=[a@1 ASC, b@2 ASC NULLS LAST, c@3 ASC NULLS LAST], has_header=true ----SortExec: expr=[a@1 ASC] -------CoalesceBatchesExec: target_batch_size=4096 +------CoalesceBatchesExec: target_batch_size=2 --------RepartitionExec: partitioning=Hash([a@1], 2), input_partitions=2 ----------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 ------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a0, a, b, c, d], output_ordering=[a@1 ASC, b@2 ASC NULLS LAST, c@3 ASC NULLS LAST], has_header=true @@ -3162,12 +3166,12 @@ physical_plan SortPreservingMergeExec: [rn1@10 ASC NULLS LAST] --SortMergeJoin: join_type=Right, on=[(a@1, a@1)] ----SortExec: expr=[a@1 ASC] -------CoalesceBatchesExec: target_batch_size=4096 +------CoalesceBatchesExec: target_batch_size=2 --------RepartitionExec: partitioning=Hash([a@1], 2), input_partitions=2 ----------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 ------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a0, a, b, c, d], output_ordering=[a@1 ASC, b@2 ASC NULLS LAST, c@3 ASC NULLS LAST], has_header=true ----SortExec: expr=[rn1@5 ASC NULLS LAST] -------CoalesceBatchesExec: target_batch_size=4096 +------CoalesceBatchesExec: target_batch_size=2 --------RepartitionExec: partitioning=Hash([a@1], 2), input_partitions=2 ----------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 ------------ProjectionExec: expr=[a0@0 as a0, a@1 as a, b@2 as b, c@3 as c, d@4 as d, ROW_NUMBER() ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING@5 as rn1] @@ -3203,14 +3207,14 @@ SortPreservingMergeExec: [a@1 ASC,b@2 ASC NULLS LAST,c@3 ASC NULLS LAST,rn1@11 A --SortExec: expr=[a@1 ASC,b@2 ASC NULLS LAST,c@3 ASC NULLS LAST,rn1@11 ASC NULLS LAST] ----SortMergeJoin: join_type=Inner, on=[(a@1, a@1)] ------SortExec: expr=[a@1 ASC] ---------CoalesceBatchesExec: target_batch_size=4096 +--------CoalesceBatchesExec: target_batch_size=2 ----------RepartitionExec: partitioning=Hash([a@1], 2), input_partitions=2 ------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 --------------ProjectionExec: expr=[a0@0 as a0, a@1 as a, b@2 as b, c@3 as c, d@4 as d, ROW_NUMBER() ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING@5 as rn1] ----------------BoundedWindowAggExec: wdw=[ROW_NUMBER() ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING: Ok(Field { name: "ROW_NUMBER() ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING", data_type: UInt64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(NULL)), end_bound: Following(UInt64(NULL)) }], mode=[Sorted] ------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a0, a, b, c, d], output_ordering=[a@1 ASC, b@2 ASC NULLS LAST, c@3 ASC NULLS LAST], has_header=true ------SortExec: expr=[a@1 ASC] ---------CoalesceBatchesExec: target_batch_size=4096 +--------CoalesceBatchesExec: target_batch_size=2 ----------RepartitionExec: partitioning=Hash([a@1], 2), input_partitions=2 ------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 --------------ProjectionExec: expr=[a0@0 as a0, a@1 as a, b@2 as b, c@3 as c, d@4 as d, ROW_NUMBER() ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING@5 as rn1] @@ -3245,7 +3249,7 @@ Sort: r_table.rn1 ASC NULLS LAST --------WindowAggr: windowExpr=[[ROW_NUMBER() ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING]] ----------TableScan: annotated_data projection=[a0, a, b, c, d] physical_plan -CoalesceBatchesExec: target_batch_size=4096 +CoalesceBatchesExec: target_batch_size=2 --HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(a@1, a@1)] ----CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a0, a, b, c, d], output_ordering=[a@1 ASC, b@2 ASC NULLS LAST, c@3 ASC NULLS LAST], has_header=true ----ProjectionExec: expr=[a0@0 as a0, a@1 as a, b@2 as b, c@3 as c, d@4 as d, ROW_NUMBER() ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING@5 as rn1] @@ -3272,7 +3276,7 @@ Sort: r_table.rn1 ASC NULLS LAST --------WindowAggr: windowExpr=[[ROW_NUMBER() ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING]] ----------TableScan: annotated_data projection=[a0, a, b, c, d] physical_plan -CoalesceBatchesExec: target_batch_size=4096 +CoalesceBatchesExec: target_batch_size=2 --HashJoinExec: mode=CollectLeft, join_type=RightAnti, on=[(a@0, a@1)] ----CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a], output_ordering=[a@0 ASC], has_header=true ----ProjectionExec: expr=[a0@0 as a0, a@1 as a, b@2 as b, c@3 as c, d@4 as d, ROW_NUMBER() ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING@5 as rn1] diff --git a/datafusion/sqllogictest/test_files/options.slt b/datafusion/sqllogictest/test_files/options.slt index 5fbb2102f4bf..83fe85745ef8 100644 --- a/datafusion/sqllogictest/test_files/options.slt +++ b/datafusion/sqllogictest/test_files/options.slt @@ -33,7 +33,7 @@ Filter: a.c0 < Int32(1) physical_plan CoalesceBatchesExec: target_batch_size=8192 --FilterExec: c0@0 < 1 -----MemoryExec: partitions=4, partition_sizes=[1, 0, 0, 0] +----MemoryExec: partitions=1, partition_sizes=[1] ## # test_disable_coalesce @@ -51,7 +51,7 @@ Filter: a.c0 < Int32(1) --TableScan: a projection=[c0] physical_plan FilterExec: c0@0 < 1 ---MemoryExec: partitions=4, partition_sizes=[1, 0, 0, 0] +--MemoryExec: partitions=1, partition_sizes=[1] statement ok set datafusion.execution.coalesce_batches = true @@ -74,7 +74,7 @@ Filter: a.c0 < Int32(1) physical_plan CoalesceBatchesExec: target_batch_size=1234 --FilterExec: c0@0 < 1 -----MemoryExec: partitions=4, partition_sizes=[1, 0, 0, 0] +----MemoryExec: partitions=1, partition_sizes=[1] statement ok diff --git a/datafusion/sqllogictest/test_files/select.slt b/datafusion/sqllogictest/test_files/select.slt index b09910735809..1d427479763a 100644 --- a/datafusion/sqllogictest/test_files/select.slt +++ b/datafusion/sqllogictest/test_files/select.slt @@ -485,8 +485,7 @@ Projection: select_between_data.c1 >= Int64(2) AND select_between_data.c1 <= Int --TableScan: select_between_data projection=[c1] physical_plan ProjectionExec: expr=[c1@0 >= 2 AND c1@0 <= 3 as select_between_data.c1 BETWEEN Int64(2) AND Int64(3)] ---RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -----MemoryExec: partitions=1, partition_sizes=[1] +--MemoryExec: partitions=1, partition_sizes=[1] # TODO: query_get_indexed_field diff --git a/datafusion/sqllogictest/test_files/subquery.slt b/datafusion/sqllogictest/test_files/subquery.slt index 2eccb60aad3e..7cbb848f3333 100644 --- a/datafusion/sqllogictest/test_files/subquery.slt +++ b/datafusion/sqllogictest/test_files/subquery.slt @@ -15,6 +15,10 @@ # specific language governing permissions and limitations # under the License. +# make sure to a batch size smaller than row number of the table. +statement ok +set datafusion.execution.batch_size = 2; + ############# ## Subquery Tests ############# @@ -178,15 +182,15 @@ Projection: t1.t1_id, __scalar_sq_1.SUM(t2.t2_int) AS t2_sum physical_plan ProjectionExec: expr=[t1_id@0 as t1_id, SUM(t2.t2_int)@1 as t2_sum] --ProjectionExec: expr=[t1_id@2 as t1_id, SUM(t2.t2_int)@0 as SUM(t2.t2_int), t2_id@1 as t2_id] -----CoalesceBatchesExec: target_batch_size=8192 +----CoalesceBatchesExec: target_batch_size=2 ------HashJoinExec: mode=Partitioned, join_type=Right, on=[(t2_id@1, t1_id@0)] --------ProjectionExec: expr=[SUM(t2.t2_int)@1 as SUM(t2.t2_int), t2_id@0 as t2_id] ----------AggregateExec: mode=FinalPartitioned, gby=[t2_id@0 as t2_id], aggr=[SUM(t2.t2_int)] -------------CoalesceBatchesExec: target_batch_size=8192 +------------CoalesceBatchesExec: target_batch_size=2 --------------RepartitionExec: partitioning=Hash([t2_id@0], 4), input_partitions=4 ----------------AggregateExec: mode=Partial, gby=[t2_id@0 as t2_id], aggr=[SUM(t2.t2_int)] ------------------MemoryExec: partitions=4, partition_sizes=[1, 0, 0, 0] ---------CoalesceBatchesExec: target_batch_size=8192 +--------CoalesceBatchesExec: target_batch_size=2 ----------RepartitionExec: partitioning=Hash([t1_id@0], 4), input_partitions=4 ------------MemoryExec: partitions=4, partition_sizes=[1, 0, 0, 0] @@ -213,15 +217,15 @@ Projection: t1.t1_id, __scalar_sq_1.SUM(t2.t2_int * Float64(1)) + Int64(1) AS t2 physical_plan ProjectionExec: expr=[t1_id@0 as t1_id, SUM(t2.t2_int * Float64(1)) + Int64(1)@1 as t2_sum] --ProjectionExec: expr=[t1_id@2 as t1_id, SUM(t2.t2_int * Float64(1)) + Int64(1)@0 as SUM(t2.t2_int * Float64(1)) + Int64(1), t2_id@1 as t2_id] -----CoalesceBatchesExec: target_batch_size=8192 +----CoalesceBatchesExec: target_batch_size=2 ------HashJoinExec: mode=Partitioned, join_type=Right, on=[(t2_id@1, t1_id@0)] --------ProjectionExec: expr=[SUM(t2.t2_int * Float64(1))@1 + 1 as SUM(t2.t2_int * Float64(1)) + Int64(1), t2_id@0 as t2_id] ----------AggregateExec: mode=FinalPartitioned, gby=[t2_id@0 as t2_id], aggr=[SUM(t2.t2_int * Float64(1))] -------------CoalesceBatchesExec: target_batch_size=8192 +------------CoalesceBatchesExec: target_batch_size=2 --------------RepartitionExec: partitioning=Hash([t2_id@0], 4), input_partitions=4 ----------------AggregateExec: mode=Partial, gby=[t2_id@0 as t2_id], aggr=[SUM(t2.t2_int * Float64(1))] ------------------MemoryExec: partitions=4, partition_sizes=[1, 0, 0, 0] ---------CoalesceBatchesExec: target_batch_size=8192 +--------CoalesceBatchesExec: target_batch_size=2 ----------RepartitionExec: partitioning=Hash([t1_id@0], 4), input_partitions=4 ------------MemoryExec: partitions=4, partition_sizes=[1, 0, 0, 0] @@ -247,16 +251,16 @@ Projection: t1.t1_id, __scalar_sq_1.SUM(t2.t2_int) AS t2_sum ----------TableScan: t2 projection=[t2_id, t2_int] physical_plan ProjectionExec: expr=[t1_id@0 as t1_id, SUM(t2.t2_int)@1 as t2_sum] ---CoalesceBatchesExec: target_batch_size=8192 +--CoalesceBatchesExec: target_batch_size=2 ----HashJoinExec: mode=Partitioned, join_type=Left, on=[(t1_id@0, t2_id@1)] -------CoalesceBatchesExec: target_batch_size=8192 +------CoalesceBatchesExec: target_batch_size=2 --------RepartitionExec: partitioning=Hash([t1_id@0], 4), input_partitions=4 ----------MemoryExec: partitions=4, partition_sizes=[1, 0, 0, 0] -------CoalesceBatchesExec: target_batch_size=8192 +------CoalesceBatchesExec: target_batch_size=2 --------RepartitionExec: partitioning=Hash([t2_id@1], 4), input_partitions=4 ----------ProjectionExec: expr=[SUM(t2.t2_int)@2 as SUM(t2.t2_int), t2_id@0 as t2_id] ------------AggregateExec: mode=FinalPartitioned, gby=[t2_id@0 as t2_id, Utf8("a")@1 as Utf8("a")], aggr=[SUM(t2.t2_int)] ---------------CoalesceBatchesExec: target_batch_size=8192 +--------------CoalesceBatchesExec: target_batch_size=2 ----------------RepartitionExec: partitioning=Hash([t2_id@0, Utf8("a")@1], 4), input_partitions=4 ------------------AggregateExec: mode=Partial, gby=[t2_id@0 as t2_id, a as Utf8("a")], aggr=[SUM(t2.t2_int)] --------------------MemoryExec: partitions=4, partition_sizes=[1, 0, 0, 0] @@ -285,17 +289,17 @@ Projection: t1.t1_id, __scalar_sq_1.SUM(t2.t2_int) AS t2_sum physical_plan ProjectionExec: expr=[t1_id@0 as t1_id, SUM(t2.t2_int)@1 as t2_sum] --ProjectionExec: expr=[t1_id@2 as t1_id, SUM(t2.t2_int)@0 as SUM(t2.t2_int), t2_id@1 as t2_id] -----CoalesceBatchesExec: target_batch_size=8192 +----CoalesceBatchesExec: target_batch_size=2 ------HashJoinExec: mode=Partitioned, join_type=Right, on=[(t2_id@1, t1_id@0)] --------ProjectionExec: expr=[SUM(t2.t2_int)@1 as SUM(t2.t2_int), t2_id@0 as t2_id] -----------CoalesceBatchesExec: target_batch_size=8192 +----------CoalesceBatchesExec: target_batch_size=2 ------------FilterExec: SUM(t2.t2_int)@1 < 3 --------------AggregateExec: mode=FinalPartitioned, gby=[t2_id@0 as t2_id], aggr=[SUM(t2.t2_int)] -----------------CoalesceBatchesExec: target_batch_size=8192 +----------------CoalesceBatchesExec: target_batch_size=2 ------------------RepartitionExec: partitioning=Hash([t2_id@0], 4), input_partitions=4 --------------------AggregateExec: mode=Partial, gby=[t2_id@0 as t2_id], aggr=[SUM(t2.t2_int)] ----------------------MemoryExec: partitions=4, partition_sizes=[1, 0, 0, 0] ---------CoalesceBatchesExec: target_batch_size=8192 +--------CoalesceBatchesExec: target_batch_size=2 ----------RepartitionExec: partitioning=Hash([t1_id@0], 4), input_partitions=4 ------------MemoryExec: partitions=4, partition_sizes=[1, 0, 0, 0] diff --git a/datafusion/sqllogictest/test_files/tpch/q15.slt.part b/datafusion/sqllogictest/test_files/tpch/q15.slt.part index 4515b8ae1fb4..a872e96acf04 100644 --- a/datafusion/sqllogictest/test_files/tpch/q15.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/q15.slt.part @@ -95,20 +95,19 @@ SortPreservingMergeExec: [s_suppkey@0 ASC NULLS LAST] ----------------------------------FilterExec: l_shipdate@3 >= 9496 AND l_shipdate@3 < 9587 ------------------------------------CsvExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:0..18561749], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:18561749..37123498], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:37123498..55685247], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:55685247..74246996]]}, projection=[l_suppkey, l_extendedprice, l_discount, l_shipdate], has_header=false ----------CoalesceBatchesExec: target_batch_size=8192 -------------RepartitionExec: partitioning=Hash([MAX(revenue0.total_revenue)@0], 4), input_partitions=4 ---------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -----------------AggregateExec: mode=Final, gby=[], aggr=[MAX(revenue0.total_revenue)] -------------------CoalescePartitionsExec ---------------------AggregateExec: mode=Partial, gby=[], aggr=[MAX(revenue0.total_revenue)] -----------------------ProjectionExec: expr=[SUM(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)@1 as total_revenue] -------------------------AggregateExec: mode=FinalPartitioned, gby=[l_suppkey@0 as l_suppkey], aggr=[SUM(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)] ---------------------------CoalesceBatchesExec: target_batch_size=8192 -----------------------------RepartitionExec: partitioning=Hash([l_suppkey@0], 4), input_partitions=4 -------------------------------AggregateExec: mode=Partial, gby=[l_suppkey@0 as l_suppkey], aggr=[SUM(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)] ---------------------------------ProjectionExec: expr=[l_suppkey@0 as l_suppkey, l_extendedprice@1 as l_extendedprice, l_discount@2 as l_discount] -----------------------------------CoalesceBatchesExec: target_batch_size=8192 -------------------------------------FilterExec: l_shipdate@3 >= 9496 AND l_shipdate@3 < 9587 ---------------------------------------CsvExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:0..18561749], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:18561749..37123498], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:37123498..55685247], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:55685247..74246996]]}, projection=[l_suppkey, l_extendedprice, l_discount, l_shipdate], has_header=false +------------RepartitionExec: partitioning=Hash([MAX(revenue0.total_revenue)@0], 4), input_partitions=1 +--------------AggregateExec: mode=Final, gby=[], aggr=[MAX(revenue0.total_revenue)] +----------------CoalescePartitionsExec +------------------AggregateExec: mode=Partial, gby=[], aggr=[MAX(revenue0.total_revenue)] +--------------------ProjectionExec: expr=[SUM(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)@1 as total_revenue] +----------------------AggregateExec: mode=FinalPartitioned, gby=[l_suppkey@0 as l_suppkey], aggr=[SUM(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)] +------------------------CoalesceBatchesExec: target_batch_size=8192 +--------------------------RepartitionExec: partitioning=Hash([l_suppkey@0], 4), input_partitions=4 +----------------------------AggregateExec: mode=Partial, gby=[l_suppkey@0 as l_suppkey], aggr=[SUM(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)] +------------------------------ProjectionExec: expr=[l_suppkey@0 as l_suppkey, l_extendedprice@1 as l_extendedprice, l_discount@2 as l_discount] +--------------------------------CoalesceBatchesExec: target_batch_size=8192 +----------------------------------FilterExec: l_shipdate@3 >= 9496 AND l_shipdate@3 < 9587 +------------------------------------CsvExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:0..18561749], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:18561749..37123498], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:37123498..55685247], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:55685247..74246996]]}, projection=[l_suppkey, l_extendedprice, l_discount, l_shipdate], has_header=false query ITTTR with revenue0 (supplier_no, total_revenue) as ( diff --git a/datafusion/sqllogictest/test_files/union.slt b/datafusion/sqllogictest/test_files/union.slt index cbb1896efb13..688774c906fe 100644 --- a/datafusion/sqllogictest/test_files/union.slt +++ b/datafusion/sqllogictest/test_files/union.slt @@ -200,13 +200,14 @@ Aggregate: groupBy=[[t1.name]], aggr=[[]] physical_plan AggregateExec: mode=FinalPartitioned, gby=[name@0 as name], aggr=[] --CoalesceBatchesExec: target_batch_size=8192 -----RepartitionExec: partitioning=Hash([name@0], 4), input_partitions=12 -------AggregateExec: mode=Partial, gby=[name@0 as name], aggr=[] ---------UnionExec -----------MemoryExec: partitions=4, partition_sizes=[1, 0, 0, 0] -----------MemoryExec: partitions=4, partition_sizes=[1, 0, 0, 0] -----------ProjectionExec: expr=[name@0 || _new as name] -------------MemoryExec: partitions=4, partition_sizes=[1, 0, 0, 0] +----RepartitionExec: partitioning=Hash([name@0], 4), input_partitions=4 +------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=3 +--------AggregateExec: mode=Partial, gby=[name@0 as name], aggr=[] +----------UnionExec +------------MemoryExec: partitions=1, partition_sizes=[1] +------------MemoryExec: partitions=1, partition_sizes=[1] +------------ProjectionExec: expr=[name@0 || _new as name] +--------------MemoryExec: partitions=1, partition_sizes=[1] # nested_union_all query T rowsort @@ -234,11 +235,14 @@ Union ----TableScan: t2 projection=[name] physical_plan UnionExec ---MemoryExec: partitions=4, partition_sizes=[1, 0, 0, 0] ---MemoryExec: partitions=4, partition_sizes=[1, 0, 0, 0] +--MemoryExec: partitions=1, partition_sizes=[1] +--MemoryExec: partitions=1, partition_sizes=[1] --ProjectionExec: expr=[name@0 || _new as name] -----MemoryExec: partitions=4, partition_sizes=[1, 0, 0, 0] +----MemoryExec: partitions=1, partition_sizes=[1] +# Make sure to choose a small batch size to introduce parallelism to the plan. +statement ok +set datafusion.execution.batch_size = 2; # union_with_type_coercion query TT @@ -269,32 +273,36 @@ Union physical_plan UnionExec --ProjectionExec: expr=[id@0 as id, name@1 as name] -----CoalesceBatchesExec: target_batch_size=8192 +----CoalesceBatchesExec: target_batch_size=2 ------HashJoinExec: mode=Partitioned, join_type=LeftAnti, on=[(id@0, CAST(t2.id AS Int32)@2), (name@1, name@1)] --------AggregateExec: mode=FinalPartitioned, gby=[id@0 as id, name@1 as name], aggr=[] -----------CoalesceBatchesExec: target_batch_size=8192 +----------CoalesceBatchesExec: target_batch_size=2 ------------RepartitionExec: partitioning=Hash([id@0, name@1], 4), input_partitions=4 --------------AggregateExec: mode=Partial, gby=[id@0 as id, name@1 as name], aggr=[] -----------------MemoryExec: partitions=4, partition_sizes=[1, 0, 0, 0] ---------CoalesceBatchesExec: target_batch_size=8192 +----------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +------------------MemoryExec: partitions=1, partition_sizes=[1] +--------CoalesceBatchesExec: target_batch_size=2 ----------RepartitionExec: partitioning=Hash([CAST(t2.id AS Int32)@2, name@1], 4), input_partitions=4 ------------ProjectionExec: expr=[id@0 as id, name@1 as name, CAST(id@0 AS Int32) as CAST(t2.id AS Int32)] ---------------MemoryExec: partitions=4, partition_sizes=[1, 0, 0, 0] +--------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +----------------MemoryExec: partitions=1, partition_sizes=[1] --ProjectionExec: expr=[CAST(id@0 AS Int32) as id, name@1 as name] ----ProjectionExec: expr=[id@0 as id, name@1 as name] -------CoalesceBatchesExec: target_batch_size=8192 +------CoalesceBatchesExec: target_batch_size=2 --------HashJoinExec: mode=Partitioned, join_type=LeftAnti, on=[(CAST(t2.id AS Int32)@2, id@0), (name@1, name@1)] -----------CoalesceBatchesExec: target_batch_size=8192 +----------CoalesceBatchesExec: target_batch_size=2 ------------RepartitionExec: partitioning=Hash([CAST(t2.id AS Int32)@2, name@1], 4), input_partitions=4 --------------ProjectionExec: expr=[id@0 as id, name@1 as name, CAST(id@0 AS Int32) as CAST(t2.id AS Int32)] ----------------AggregateExec: mode=FinalPartitioned, gby=[id@0 as id, name@1 as name], aggr=[] -------------------CoalesceBatchesExec: target_batch_size=8192 +------------------CoalesceBatchesExec: target_batch_size=2 --------------------RepartitionExec: partitioning=Hash([id@0, name@1], 4), input_partitions=4 ----------------------AggregateExec: mode=Partial, gby=[id@0 as id, name@1 as name], aggr=[] -------------------------MemoryExec: partitions=4, partition_sizes=[1, 0, 0, 0] -----------CoalesceBatchesExec: target_batch_size=8192 +------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +--------------------------MemoryExec: partitions=1, partition_sizes=[1] +----------CoalesceBatchesExec: target_batch_size=2 ------------RepartitionExec: partitioning=Hash([id@0, name@1], 4), input_partitions=4 ---------------MemoryExec: partitions=4, partition_sizes=[1, 0, 0, 0] +--------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +----------------MemoryExec: partitions=1, partition_sizes=[1] query IT rowsort ( @@ -339,26 +347,30 @@ Union ----TableScan: t1 projection=[name] physical_plan InterleaveExec ---CoalesceBatchesExec: target_batch_size=8192 +--CoalesceBatchesExec: target_batch_size=2 ----HashJoinExec: mode=Partitioned, join_type=LeftAnti, on=[(name@0, name@0)] ------AggregateExec: mode=FinalPartitioned, gby=[name@0 as name], aggr=[] ---------CoalesceBatchesExec: target_batch_size=8192 +--------CoalesceBatchesExec: target_batch_size=2 ----------RepartitionExec: partitioning=Hash([name@0], 4), input_partitions=4 ------------AggregateExec: mode=Partial, gby=[name@0 as name], aggr=[] ---------------MemoryExec: partitions=4, partition_sizes=[1, 0, 0, 0] -------CoalesceBatchesExec: target_batch_size=8192 +--------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +----------------MemoryExec: partitions=1, partition_sizes=[1] +------CoalesceBatchesExec: target_batch_size=2 --------RepartitionExec: partitioning=Hash([name@0], 4), input_partitions=4 -----------MemoryExec: partitions=4, partition_sizes=[1, 0, 0, 0] ---CoalesceBatchesExec: target_batch_size=8192 +----------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +------------MemoryExec: partitions=1, partition_sizes=[1] +--CoalesceBatchesExec: target_batch_size=2 ----HashJoinExec: mode=Partitioned, join_type=LeftAnti, on=[(name@0, name@0)] ------AggregateExec: mode=FinalPartitioned, gby=[name@0 as name], aggr=[] ---------CoalesceBatchesExec: target_batch_size=8192 +--------CoalesceBatchesExec: target_batch_size=2 ----------RepartitionExec: partitioning=Hash([name@0], 4), input_partitions=4 ------------AggregateExec: mode=Partial, gby=[name@0 as name], aggr=[] ---------------MemoryExec: partitions=4, partition_sizes=[1, 0, 0, 0] -------CoalesceBatchesExec: target_batch_size=8192 +--------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +----------------MemoryExec: partitions=1, partition_sizes=[1] +------CoalesceBatchesExec: target_batch_size=2 --------RepartitionExec: partitioning=Hash([name@0], 4), input_partitions=4 -----------MemoryExec: partitions=4, partition_sizes=[1, 0, 0, 0] +----------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +------------MemoryExec: partitions=1, partition_sizes=[1] # union_upcast_types query TT @@ -416,15 +428,17 @@ ProjectionExec: expr=[COUNT(*)@1 as COUNT(*)] --AggregateExec: mode=SinglePartitioned, gby=[name@0 as name], aggr=[COUNT(*)] ----InterleaveExec ------AggregateExec: mode=FinalPartitioned, gby=[name@0 as name], aggr=[] ---------CoalesceBatchesExec: target_batch_size=8192 +--------CoalesceBatchesExec: target_batch_size=2 ----------RepartitionExec: partitioning=Hash([name@0], 4), input_partitions=4 ------------AggregateExec: mode=Partial, gby=[name@0 as name], aggr=[] ---------------MemoryExec: partitions=4, partition_sizes=[1, 0, 0, 0] +--------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +----------------MemoryExec: partitions=1, partition_sizes=[1] ------AggregateExec: mode=FinalPartitioned, gby=[name@0 as name], aggr=[] ---------CoalesceBatchesExec: target_batch_size=8192 +--------CoalesceBatchesExec: target_batch_size=2 ----------RepartitionExec: partitioning=Hash([name@0], 4), input_partitions=4 ------------AggregateExec: mode=Partial, gby=[name@0 as name], aggr=[] ---------------MemoryExec: partitions=4, partition_sizes=[1, 0, 0, 0] +--------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +----------------MemoryExec: partitions=1, partition_sizes=[1] ######## @@ -530,11 +544,10 @@ physical_plan UnionExec --ProjectionExec: expr=[Int64(1)@0 as a] ----AggregateExec: mode=FinalPartitioned, gby=[Int64(1)@0 as Int64(1)], aggr=[] -------CoalesceBatchesExec: target_batch_size=8192 ---------RepartitionExec: partitioning=Hash([Int64(1)@0], 4), input_partitions=4 +------CoalesceBatchesExec: target_batch_size=2 +--------RepartitionExec: partitioning=Hash([Int64(1)@0], 4), input_partitions=1 ----------AggregateExec: mode=Partial, gby=[1 as Int64(1)], aggr=[] -------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 ---------------EmptyExec: produce_one_row=true +------------EmptyExec: produce_one_row=true --ProjectionExec: expr=[2 as a] ----EmptyExec: produce_one_row=true --ProjectionExec: expr=[3 as a] @@ -562,16 +575,12 @@ physical_plan UnionExec --ProjectionExec: expr=[COUNT(*)@1 as count, n@0 as n] ----AggregateExec: mode=FinalPartitioned, gby=[n@0 as n], aggr=[COUNT(*)] -------CoalesceBatchesExec: target_batch_size=8192 ---------RepartitionExec: partitioning=Hash([n@0], 4), input_partitions=4 +------CoalesceBatchesExec: target_batch_size=2 +--------RepartitionExec: partitioning=Hash([n@0], 4), input_partitions=1 ----------AggregateExec: mode=Partial, gby=[n@0 as n], aggr=[COUNT(*)] -------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 ---------------ProjectionExec: expr=[5 as n] -----------------EmptyExec: produce_one_row=true +------------ProjectionExec: expr=[5 as n] +--------------EmptyExec: produce_one_row=true --ProjectionExec: expr=[x@0 as count, y@1 as n] ----ProjectionExec: expr=[1 as x, MAX(Int64(10))@0 as y] -------AggregateExec: mode=Final, gby=[], aggr=[MAX(Int64(10))] ---------CoalescePartitionsExec -----------AggregateExec: mode=Partial, gby=[], aggr=[MAX(Int64(10))] -------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 ---------------EmptyExec: produce_one_row=true +------AggregateExec: mode=Single, gby=[], aggr=[MAX(Int64(10))] +--------EmptyExec: produce_one_row=true From e00932c6631172656cb442efab8858d424c5a1c5 Mon Sep 17 00:00:00 2001 From: Andrew Lamb Date: Tue, 17 Oct 2023 07:27:36 -0400 Subject: [PATCH 083/572] Minor: Add tests for binary / utf8 coercion (#7839) --- datafusion/sqllogictest/test_files/binary.slt | 89 +++++++++++++++++++ 1 file changed, 89 insertions(+) diff --git a/datafusion/sqllogictest/test_files/binary.slt b/datafusion/sqllogictest/test_files/binary.slt index d3a7e8c19334..38f8a2e14ffc 100644 --- a/datafusion/sqllogictest/test_files/binary.slt +++ b/datafusion/sqllogictest/test_files/binary.slt @@ -155,3 +155,92 @@ drop table t_source statement ok drop table t + + +############# +## Tests for binary that contains strings +############# + +statement ok +CREATE TABLE t_source +AS VALUES + ('Foo'), + (NULL), + ('Bar'), + ('FooBar') +; + +# Create a table with Binary, LargeBinary but really has strings +statement ok +CREATE TABLE t +AS SELECT + arrow_cast(column1, 'Binary') as "binary", + arrow_cast(column1, 'LargeBinary') as "largebinary" +FROM t_source; + +query ??TT +SELECT binary, largebinary, cast(binary as varchar) as binary_str, cast(largebinary as varchar) as binary_largestr from t; +---- +466f6f 466f6f Foo Foo +NULL NULL NULL NULL +426172 426172 Bar Bar +466f6f426172 466f6f426172 FooBar FooBar + +# ensure coercion works for = and <> +query ?T +SELECT binary, cast(binary as varchar) as str FROM t WHERE binary = 'Foo'; +---- +466f6f Foo + +query ?T +SELECT binary, cast(binary as varchar) as str FROM t WHERE binary <> 'Foo'; +---- +426172 Bar +466f6f426172 FooBar + +# order by +query ? +SELECT binary FROM t ORDER BY binary; +---- +426172 +466f6f +466f6f426172 +NULL + +# order by +query ? +SELECT largebinary FROM t ORDER BY largebinary; +---- +426172 +466f6f +466f6f426172 +NULL + +# LIKE +# https://github.com/apache/arrow-datafusion/issues/7342 +query error DataFusion error: type_coercion +SELECT binary FROM t where binary LIKE '%F'; + +query error DataFusion error: type_coercion +SELECT largebinary FROM t where largebinary LIKE '%F'; + + +# character_length function +# https://github.com/apache/arrow-datafusion/issues/7344 +query error DataFusion error: Error during planning: The "character_length" function can only accept strings, but got Binary\. +SELECT + cast(binary as varchar) as str, + character_length(binary) as binary_len, + cast(largebinary as varchar) as large_str, + character_length(binary) as largebinary_len +from t; + +# regexp_replace +# https://github.com/apache/arrow-datafusion/issues/7345 +query error DataFusion error: Error during planning: The "regexp_replace" function can only accept strings, but got Binary\. +SELECT + cast(binary as varchar) as str, + regexp_replace(binary, 'F', 'f') as binary_replaced, + cast(largebinary as varchar) as large_str, + regexp_replace(largebinary, 'F', 'f') as large_binary_replaced +from t; From 2cd170611547c0bb9e0c1788c40f3f006a7d1ec7 Mon Sep 17 00:00:00 2001 From: Andrew Lamb Date: Tue, 17 Oct 2023 07:47:53 -0400 Subject: [PATCH 084/572] Avoid panics on error while encoding/decoding ListValue::Array as protobuf (#7837) --- .../proto/src/logical_plan/from_proto.rs | 34 +++++++++------- datafusion/proto/src/logical_plan/to_proto.rs | 39 ++++++++----------- 2 files changed, 37 insertions(+), 36 deletions(-) diff --git a/datafusion/proto/src/logical_plan/from_proto.rs b/datafusion/proto/src/logical_plan/from_proto.rs index b3873c01dd06..c87882ca72fc 100644 --- a/datafusion/proto/src/logical_plan/from_proto.rs +++ b/datafusion/proto/src/logical_plan/from_proto.rs @@ -26,7 +26,7 @@ use crate::protobuf::{ OptimizedPhysicalPlanType, PlaceholderNode, RollupNode, }; use arrow::{ - buffer::{Buffer, MutableBuffer}, + buffer::Buffer, datatypes::{ i256, DataType, Field, IntervalMonthDayNanoType, IntervalUnit, Schema, TimeUnit, UnionFields, UnionMode, @@ -645,6 +645,7 @@ impl TryFrom<&protobuf::ScalarValue> for ScalarValue { Value::Float32Value(v) => Self::Float32(Some(*v)), Value::Float64Value(v) => Self::Float64(Some(*v)), Value::Date32Value(v) => Self::Date32(Some(*v)), + // ScalarValue::List is serialized using arrow IPC format Value::ListValue(scalar_list) => { let protobuf::ScalarListValue { ipc_message, @@ -655,29 +656,36 @@ impl TryFrom<&protobuf::ScalarValue> for ScalarValue { let schema: Schema = if let Some(schema_ref) = schema { schema_ref.try_into()? } else { - return Err(Error::General("Unexpected schema".to_string())); + return Err(Error::General( + "Invalid schema while deserializing ScalarValue::List" + .to_string(), + )); }; - let message = root_as_message(ipc_message.as_slice()).unwrap(); + let message = root_as_message(ipc_message.as_slice()).map_err(|e| { + Error::General(format!( + "Error IPC message while deserializing ScalarValue::List: {e}" + )) + })?; + let buffer = Buffer::from(arrow_data); - // TODO: Add comment to why adding 0 before arrow_data. - // This code is from https://github.com/apache/arrow-rs/blob/4320a753beaee0a1a6870c59ef46b59e88c9c323/arrow-ipc/src/reader.rs#L1670-L1674C45 - // Construct an unaligned buffer - let mut buffer = MutableBuffer::with_capacity(arrow_data.len() + 1); - buffer.push(0_u8); - buffer.extend_from_slice(arrow_data.as_slice()); - let b = Buffer::from(buffer).slice(1); + let ipc_batch = message.header_as_record_batch().ok_or_else(|| { + Error::General( + "Unexpected message type deserializing ScalarValue::List" + .to_string(), + ) + })?; - let ipc_batch = message.header_as_record_batch().unwrap(); let record_batch = read_record_batch( - &b, + &buffer, ipc_batch, Arc::new(schema), &Default::default(), None, &message.version(), ) - .unwrap(); + .map_err(DataFusionError::ArrowError) + .map_err(|e| e.context("Decoding ScalarValue::List Value"))?; let arr = record_batch.column(0); Self::List(arr.to_owned()) } diff --git a/datafusion/proto/src/logical_plan/to_proto.rs b/datafusion/proto/src/logical_plan/to_proto.rs index e80d60931cf6..125ced032e20 100644 --- a/datafusion/proto/src/logical_plan/to_proto.rs +++ b/datafusion/proto/src/logical_plan/to_proto.rs @@ -56,13 +56,6 @@ use datafusion_expr::{ pub enum Error { General(String), - InconsistentListTyping(DataType, DataType), - - InconsistentListDesignated { - value: ScalarValue, - designated: DataType, - }, - InvalidScalarValue(ScalarValue), InvalidScalarType(DataType), @@ -80,18 +73,6 @@ impl std::fmt::Display for Error { fn fmt(&self, f: &mut std::fmt::Formatter) -> std::fmt::Result { match self { Self::General(desc) => write!(f, "General error: {desc}"), - Self::InconsistentListTyping(type1, type2) => { - write!( - f, - "Lists with inconsistent typing; {type1:?} and {type2:?} found within list", - ) - } - Self::InconsistentListDesignated { value, designated } => { - write!( - f, - "Value {value:?} was inconsistent with designated type {designated:?}" - ) - } Self::InvalidScalarValue(value) => { write!(f, "{value:?} is invalid as a DataFusion scalar value") } @@ -1145,15 +1126,27 @@ impl TryFrom<&ScalarValue> for protobuf::ScalarValue { "Proto serialization error: ScalarValue::Fixedsizelist not supported" .to_string(), )), + // ScalarValue::List is serialized using Arrow IPC messages. + // as a single column RecordBatch ScalarValue::List(arr) => { - let batch = - RecordBatch::try_from_iter(vec![("field_name", arr.to_owned())]) - .unwrap(); + // Wrap in a "field_name" column + let batch = RecordBatch::try_from_iter(vec![( + "field_name", + arr.to_owned(), + )]) + .map_err(|e| { + Error::General( format!("Error creating temporary batch while encoding ScalarValue::List: {e}")) + })?; + let gen = IpcDataGenerator {}; let mut dict_tracker = DictionaryTracker::new(false); let (_, encoded_message) = gen .encoded_batch(&batch, &mut dict_tracker, &Default::default()) - .unwrap(); + .map_err(|e| { + Error::General(format!( + "Error encoding ScalarValue::List as IPC: {e}" + )) + })?; let schema: protobuf::Schema = batch.schema().try_into()?; From c9330bc09e583734aed721a208a0684118b170f1 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Berkay=20=C5=9Eahin?= <124376117+berkaysynnada@users.noreply.github.com> Date: Tue, 17 Oct 2023 16:57:36 +0300 Subject: [PATCH 085/572] Refactor Statistics, introduce precision estimates (`Exact`, `Inexact`, `Absent`) (#7793) * analysis context refactored * is_exact fix * Minor changes * minor changes * Minor changes * Minor changes * datatype check added, statistics default removed * MemExec uses the stats of projections, agg optimize excludes unbounded cases * fix after merge * proto fix * Simplifications * statistics() returns result * fix after merge * Simplifications * Remove option from column stats * exact info added * error in agg optimization * bugs are fixed * negative expr support * fix after merge * fix after merge * Minor changes, simplifications * minor changes * min max accs removed * fix after merge * minor changes * fix initialization of stats in limit * minor changes * Simplifications * more accurate row calculations * Improve comments * min-max values are init as absent, not inf * fix after merge * Review Part 1 * Cardinality calculation is fixed * Review Part 2 * get_int_range replaced by cardinality function * Fix imports * Statistics display is shortened. * fix after merge * Harmonize imports * Update datafusion/physical-expr/src/intervals/interval_aritmetic.rs Co-authored-by: Andrew Lamb * Addresses the reviews * Update tests * Remove panics * 1 bug-fix, 2 code simplifications 1) get_statistics_with_limit() is simplified 2) AnalysisContext { boundaries } does not need to be an Option, it is removed. 3) Aggregation statistics() has a special handling for 0 and 1 row cases. * conflict resolved * conflict resolved * Update datafusion/physical-plan/src/filter.rs Co-authored-by: Andrew Lamb * Simplify set_max/min helpers * fix vector copy, remove clones * resolving conflict * remove clone --------- Co-authored-by: Mustafa Akur Co-authored-by: Mehmet Ozan Kabak Co-authored-by: Andrew Lamb --- datafusion-examples/examples/csv_opener.rs | 4 +- .../examples/custom_datasource.rs | 21 +- datafusion-examples/examples/json_opener.rs | 4 +- datafusion/common/src/stats.rs | 100 ++-- .../core/src/datasource/file_format/arrow.rs | 16 +- .../core/src/datasource/file_format/avro.rs | 4 +- .../core/src/datasource/file_format/csv.rs | 39 +- .../core/src/datasource/file_format/json.rs | 62 +-- .../src/datasource/file_format/parquet.rs | 143 +++--- .../core/src/datasource/listing/table.rs | 20 +- datafusion/core/src/datasource/mod.rs | 2 +- .../datasource/physical_plan/arrow_file.rs | 12 +- .../core/src/datasource/physical_plan/avro.rs | 22 +- .../core/src/datasource/physical_plan/csv.rs | 24 +- .../physical_plan/file_scan_config.rs | 74 ++- .../datasource/physical_plan/file_stream.rs | 3 +- .../core/src/datasource/physical_plan/json.rs | 27 +- .../core/src/datasource/physical_plan/mod.rs | 60 +-- .../src/datasource/physical_plan/parquet.rs | 34 +- datafusion/core/src/datasource/statistics.rs | 258 +++++++---- .../aggregate_statistics.rs | 105 ++--- .../combine_partial_final_agg.rs | 16 +- .../enforce_distribution.rs | 31 +- .../src/physical_optimizer/join_selection.rs | 178 +++++--- .../physical_optimizer/output_requirements.rs | 2 +- .../physical_optimizer/pipeline_checker.rs | 6 +- .../replace_with_order_preserving_variants.rs | 2 +- .../core/src/physical_optimizer/test_utils.rs | 4 +- datafusion/core/src/physical_planner.rs | 55 +-- datafusion/core/src/test/mod.rs | 45 +- datafusion/core/src/test_util/mod.rs | 11 +- datafusion/core/src/test_util/parquet.rs | 5 +- datafusion/core/tests/custom_sources.rs | 70 ++- .../provider_filter_pushdown.rs | 4 +- .../tests/custom_sources_cases/statistics.rs | 79 ++-- .../core/tests/parquet/custom_reader.rs | 12 +- .../core/tests/parquet/file_statistics.rs | 28 +- datafusion/core/tests/parquet/page_pruning.rs | 15 +- .../core/tests/parquet/schema_coercion.rs | 11 +- datafusion/core/tests/path_partition.rs | 38 +- datafusion/core/tests/sql/explain_analyze.rs | 3 +- .../tests/user_defined/user_defined_plan.rs | 16 +- datafusion/execution/src/cache/cache_unit.rs | 20 +- datafusion/physical-expr/src/analysis.rs | 121 ++--- .../physical-expr/src/expressions/in_list.rs | 11 +- .../physical-expr/src/expressions/negative.rs | 3 +- .../src/intervals/interval_aritmetic.rs | 157 ++++--- .../physical-expr/src/intervals/utils.rs | 5 +- datafusion/physical-expr/src/lib.rs | 3 - .../physical-plan/src/aggregates/mod.rs | 85 ++-- datafusion/physical-plan/src/analyze.rs | 20 +- .../physical-plan/src/coalesce_batches.rs | 10 +- .../physical-plan/src/coalesce_partitions.rs | 2 +- datafusion/physical-plan/src/common.rs | 60 +-- datafusion/physical-plan/src/display.rs | 13 +- datafusion/physical-plan/src/empty.rs | 18 +- datafusion/physical-plan/src/explain.rs | 19 +- datafusion/physical-plan/src/filter.rs | 427 ++++++++++-------- datafusion/physical-plan/src/insert.rs | 27 +- .../physical-plan/src/joins/cross_join.rs | 277 ++++++------ .../physical-plan/src/joins/hash_join.rs | 3 +- .../src/joins/nested_loop_join.rs | 37 +- .../src/joins/sort_merge_join.rs | 23 +- .../src/joins/symmetric_hash_join.rs | 59 ++- datafusion/physical-plan/src/joins/utils.rs | 361 ++++++++------- datafusion/physical-plan/src/lib.rs | 128 +++--- datafusion/physical-plan/src/limit.rs | 124 +++-- datafusion/physical-plan/src/memory.rs | 27 +- datafusion/physical-plan/src/projection.rs | 172 ++++--- .../physical-plan/src/repartition/mod.rs | 2 +- datafusion/physical-plan/src/sorts/cursor.rs | 5 +- datafusion/physical-plan/src/sorts/sort.rs | 25 +- .../src/sorts/sort_preserving_merge.rs | 22 +- datafusion/physical-plan/src/streaming.rs | 21 +- datafusion/physical-plan/src/test/exec.rs | 55 +-- datafusion/physical-plan/src/union.rs | 204 ++++----- datafusion/physical-plan/src/unnest.rs | 34 +- datafusion/physical-plan/src/values.rs | 18 +- .../src/windows/bounded_window_agg_exec.rs | 24 +- .../src/windows/window_agg_exec.rs | 23 +- datafusion/proto/proto/datafusion.proto | 24 +- datafusion/proto/src/generated/pbjson.rs | 253 +++++++++-- datafusion/proto/src/generated/prost.rs | 59 ++- .../proto/src/physical_plan/from_proto.rs | 153 +++++-- .../proto/src/physical_plan/to_proto.rs | 132 +++--- .../tests/cases/roundtrip_physical_plan.rs | 32 +- .../sqllogictest/test_files/explain.slt | 8 +- datafusion/sqllogictest/test_files/window.slt | 12 +- .../substrait/src/physical_plan/consumer.rs | 14 +- .../tests/cases/roundtrip_physical_plan.rs | 10 +- 90 files changed, 2791 insertions(+), 2216 deletions(-) diff --git a/datafusion-examples/examples/csv_opener.rs b/datafusion-examples/examples/csv_opener.rs index 5126666d5e73..15fb07ded481 100644 --- a/datafusion-examples/examples/csv_opener.rs +++ b/datafusion-examples/examples/csv_opener.rs @@ -17,6 +17,7 @@ use std::{sync::Arc, vec}; +use datafusion::common::Statistics; use datafusion::{ assert_batches_eq, datasource::{ @@ -29,6 +30,7 @@ use datafusion::{ physical_plan::metrics::ExecutionPlanMetricsSet, test_util::aggr_test_schema, }; + use futures::StreamExt; use object_store::local::LocalFileSystem; @@ -60,7 +62,7 @@ async fn main() -> Result<()> { object_store_url: ObjectStoreUrl::local_filesystem(), file_schema: schema.clone(), file_groups: vec![vec![PartitionedFile::new(path.display().to_string(), 10)]], - statistics: Default::default(), + statistics: Statistics::new_unknown(&schema), projection: Some(vec![12, 0]), limit: Some(5), table_partition_cols: vec![], diff --git a/datafusion-examples/examples/custom_datasource.rs b/datafusion-examples/examples/custom_datasource.rs index a24573c860bb..dd36665a9344 100644 --- a/datafusion-examples/examples/custom_datasource.rs +++ b/datafusion-examples/examples/custom_datasource.rs @@ -15,13 +15,17 @@ // specific language governing permissions and limitations // under the License. -use async_trait::async_trait; +use std::any::Any; +use std::collections::{BTreeMap, HashMap}; +use std::fmt::{self, Debug, Formatter}; +use std::sync::{Arc, Mutex}; +use std::time::Duration; + use datafusion::arrow::array::{UInt64Builder, UInt8Builder}; use datafusion::arrow::datatypes::{DataType, Field, Schema, SchemaRef}; use datafusion::arrow::record_batch::RecordBatch; use datafusion::dataframe::DataFrame; -use datafusion::datasource::provider_as_source; -use datafusion::datasource::{TableProvider, TableType}; +use datafusion::datasource::{provider_as_source, TableProvider, TableType}; use datafusion::error::Result; use datafusion::execution::context::{SessionState, TaskContext}; use datafusion::physical_plan::expressions::PhysicalSortExpr; @@ -32,11 +36,8 @@ use datafusion::physical_plan::{ }; use datafusion::prelude::*; use datafusion_expr::{Expr, LogicalPlanBuilder}; -use std::any::Any; -use std::collections::{BTreeMap, HashMap}; -use std::fmt::{self, Debug, Formatter}; -use std::sync::{Arc, Mutex}; -use std::time::Duration; + +use async_trait::async_trait; use tokio::time::timeout; /// This example demonstrates executing a simple query against a custom datasource @@ -270,7 +271,7 @@ impl ExecutionPlan for CustomExec { )?)) } - fn statistics(&self) -> Statistics { - Statistics::default() + fn statistics(&self) -> Result { + Ok(Statistics::new_unknown(&self.schema())) } } diff --git a/datafusion-examples/examples/json_opener.rs b/datafusion-examples/examples/json_opener.rs index 74ba6f3852a8..1a3dbe57be75 100644 --- a/datafusion-examples/examples/json_opener.rs +++ b/datafusion-examples/examples/json_opener.rs @@ -29,6 +29,8 @@ use datafusion::{ error::Result, physical_plan::metrics::ExecutionPlanMetricsSet, }; +use datafusion_common::Statistics; + use futures::StreamExt; use object_store::ObjectStore; @@ -63,7 +65,7 @@ async fn main() -> Result<()> { object_store_url: ObjectStoreUrl::local_filesystem(), file_schema: schema.clone(), file_groups: vec![vec![PartitionedFile::new(path.to_string(), 10)]], - statistics: Default::default(), + statistics: Statistics::new_unknown(&schema), projection: Some(vec![1, 0]), limit: Some(5), table_partition_cols: vec![], diff --git a/datafusion/common/src/stats.rs b/datafusion/common/src/stats.rs index 279ad434a817..fbf639a32182 100644 --- a/datafusion/common/src/stats.rs +++ b/datafusion/common/src/stats.rs @@ -17,10 +17,11 @@ //! This module provides data structures to represent statistics +use std::fmt::{self, Debug, Display}; + use crate::ScalarValue; -use arrow::datatypes::DataType; -use std::fmt::{self, Debug, Display}; +use arrow_schema::Schema; /// Represents a value with a degree of certainty. `Precision` is used to /// propagate information the precision of statistical values. @@ -203,70 +204,95 @@ impl Display for Precision< /// Fields are optional and can be inexact because the sources /// sometimes provide approximate estimates for performance reasons /// and the transformations output are not always predictable. -#[derive(Debug, Clone, Default, PartialEq, Eq)] +#[derive(Debug, Clone, PartialEq, Eq)] pub struct Statistics { - /// The number of table rows - pub num_rows: Option, - /// total bytes of the table rows - pub total_byte_size: Option, - /// Statistics on a column level - pub column_statistics: Option>, - /// If true, any field that is `Some(..)` is the actual value in the data provided by the operator (it is not - /// an estimate). Any or all other fields might still be None, in which case no information is known. - /// if false, any field that is `Some(..)` may contain an inexact estimate and may not be the actual value. - pub is_exact: bool, + /// The number of table rows. + pub num_rows: Precision, + /// Total bytes of the table rows. + pub total_byte_size: Precision, + /// Statistics on a column level. It contains a [`ColumnStatistics`] for + /// each field in the schema of the the table to which the [`Statistics`] refer. + pub column_statistics: Vec, } -impl Display for Statistics { - fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { - if self.num_rows.is_none() && self.total_byte_size.is_none() && !self.is_exact { - return Ok(()); +impl Statistics { + /// Returns a [`Statistics`] instance for the given schema by assigning + /// unknown statistics to each column in the schema. + pub fn new_unknown(schema: &Schema) -> Self { + Self { + num_rows: Precision::Absent, + total_byte_size: Precision::Absent, + column_statistics: Statistics::unknown_column(schema), } + } - let rows = self - .num_rows - .map_or_else(|| "None".to_string(), |v| v.to_string()); - let bytes = self - .total_byte_size - .map_or_else(|| "None".to_string(), |v| v.to_string()); + /// Returns an unbounded `ColumnStatistics` for each field in the schema. + pub fn unknown_column(schema: &Schema) -> Vec { + schema + .fields() + .iter() + .map(|_| ColumnStatistics::new_unknown()) + .collect() + } - write!(f, "rows={}, bytes={}, exact={}", rows, bytes, self.is_exact)?; + /// If the exactness of a [`Statistics`] instance is lost, this function relaxes + /// the exactness of all information by converting them [`Precision::Inexact`]. + pub fn into_inexact(self) -> Self { + Statistics { + num_rows: self.num_rows.to_inexact(), + total_byte_size: self.total_byte_size.to_inexact(), + column_statistics: self + .column_statistics + .into_iter() + .map(|cs| ColumnStatistics { + null_count: cs.null_count.to_inexact(), + max_value: cs.max_value.to_inexact(), + min_value: cs.min_value.to_inexact(), + distinct_count: cs.distinct_count.to_inexact(), + }) + .collect::>(), + } + } +} + +impl Display for Statistics { + fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + write!(f, "Rows={}, Bytes={}", self.num_rows, self.total_byte_size)?; Ok(()) } } /// Statistics for a column within a relation -#[derive(Clone, Debug, Default, PartialEq, Eq)] +#[derive(Clone, Debug, PartialEq, Eq, Default)] pub struct ColumnStatistics { /// Number of null values on column - pub null_count: Option, + pub null_count: Precision, /// Maximum value of column - pub max_value: Option, + pub max_value: Precision, /// Minimum value of column - pub min_value: Option, + pub min_value: Precision, /// Number of distinct values - pub distinct_count: Option, + pub distinct_count: Precision, } impl ColumnStatistics { /// Column contains a single non null value (e.g constant). pub fn is_singleton(&self) -> bool { - match (&self.min_value, &self.max_value) { + match (self.min_value.get_value(), self.max_value.get_value()) { // Min and max values are the same and not infinity. (Some(min), Some(max)) => !min.is_null() && !max.is_null() && (min == max), (_, _) => false, } } - /// Returns the [`ColumnStatistics`] corresponding to the given datatype by assigning infinite bounds. - pub fn new_with_unbounded_column(dt: &DataType) -> ColumnStatistics { - let null = ScalarValue::try_from(dt.clone()).ok(); + /// Returns a [`ColumnStatistics`] instance having all [`Precision::Absent`] parameters. + pub fn new_unknown() -> ColumnStatistics { ColumnStatistics { - null_count: None, - max_value: null.clone(), - min_value: null, - distinct_count: None, + null_count: Precision::Absent, + max_value: Precision::Absent, + min_value: Precision::Absent, + distinct_count: Precision::Absent, } } } diff --git a/datafusion/core/src/datasource/file_format/arrow.rs b/datafusion/core/src/datasource/file_format/arrow.rs index 45ecdd6083e7..16ae4411d1bf 100644 --- a/datafusion/core/src/datasource/file_format/arrow.rs +++ b/datafusion/core/src/datasource/file_format/arrow.rs @@ -19,20 +19,24 @@ //! //! Works with files following the [Arrow IPC format](https://arrow.apache.org/docs/format/Columnar.html#ipc-file-format) +use std::any::Any; +use std::io::{Read, Seek}; +use std::sync::Arc; + use crate::datasource::file_format::FileFormat; use crate::datasource::physical_plan::{ArrowExec, FileScanConfig}; use crate::error::Result; use crate::execution::context::SessionState; use crate::physical_plan::ExecutionPlan; + use arrow::ipc::reader::FileReader; use arrow_schema::{Schema, SchemaRef}; -use async_trait::async_trait; + use datafusion_common::{FileType, Statistics}; use datafusion_physical_expr::PhysicalExpr; + +use async_trait::async_trait; use object_store::{GetResultPayload, ObjectMeta, ObjectStore}; -use std::any::Any; -use std::io::{Read, Seek}; -use std::sync::Arc; /// Arrow `FileFormat` implementation. #[derive(Default, Debug)] @@ -74,10 +78,10 @@ impl FileFormat for ArrowFormat { &self, _state: &SessionState, _store: &Arc, - _table_schema: SchemaRef, + table_schema: SchemaRef, _object: &ObjectMeta, ) -> Result { - Ok(Statistics::default()) + Ok(Statistics::new_unknown(&table_schema)) } async fn create_physical_plan( diff --git a/datafusion/core/src/datasource/file_format/avro.rs b/datafusion/core/src/datasource/file_format/avro.rs index cc6a03ba3f08..a24a28ad6fdd 100644 --- a/datafusion/core/src/datasource/file_format/avro.rs +++ b/datafusion/core/src/datasource/file_format/avro.rs @@ -74,10 +74,10 @@ impl FileFormat for AvroFormat { &self, _state: &SessionState, _store: &Arc, - _table_schema: SchemaRef, + table_schema: SchemaRef, _object: &ObjectMeta, ) -> Result { - Ok(Statistics::default()) + Ok(Statistics::new_unknown(&table_schema)) } async fn create_physical_plan( diff --git a/datafusion/core/src/datasource/file_format/csv.rs b/datafusion/core/src/datasource/file_format/csv.rs index e77382ad9ce8..41265ede7fc7 100644 --- a/datafusion/core/src/datasource/file_format/csv.rs +++ b/datafusion/core/src/datasource/file_format/csv.rs @@ -23,21 +23,6 @@ use std::fmt; use std::fmt::Debug; use std::sync::Arc; -use arrow::csv::WriterBuilder; -use arrow::datatypes::{DataType, Field, Fields, Schema}; -use arrow::{self, datatypes::SchemaRef}; -use arrow_array::RecordBatch; -use datafusion_common::{exec_err, not_impl_err, DataFusionError, FileType}; -use datafusion_execution::TaskContext; -use datafusion_physical_expr::{PhysicalExpr, PhysicalSortRequirement}; - -use async_trait::async_trait; -use bytes::{Buf, Bytes}; -use datafusion_physical_plan::metrics::MetricsSet; -use futures::stream::BoxStream; -use futures::{pin_mut, Stream, StreamExt, TryStreamExt}; -use object_store::{delimited::newline_delimited_stream, ObjectMeta, ObjectStore}; - use super::{FileFormat, DEFAULT_SCHEMA_INFER_MAX_RECORD}; use crate::datasource::file_format::file_compression_type::FileCompressionType; use crate::datasource::file_format::write::{ @@ -51,6 +36,21 @@ use crate::execution::context::SessionState; use crate::physical_plan::insert::{DataSink, FileSinkExec}; use crate::physical_plan::{DisplayAs, DisplayFormatType, Statistics}; use crate::physical_plan::{ExecutionPlan, SendableRecordBatchStream}; + +use arrow::csv::WriterBuilder; +use arrow::datatypes::{DataType, Field, Fields, Schema}; +use arrow::{self, datatypes::SchemaRef}; +use arrow_array::RecordBatch; +use datafusion_common::{exec_err, not_impl_err, DataFusionError, FileType}; +use datafusion_execution::TaskContext; +use datafusion_physical_expr::{PhysicalExpr, PhysicalSortRequirement}; +use datafusion_physical_plan::metrics::MetricsSet; + +use async_trait::async_trait; +use bytes::{Buf, Bytes}; +use futures::stream::BoxStream; +use futures::{pin_mut, Stream, StreamExt, TryStreamExt}; +use object_store::{delimited::newline_delimited_stream, ObjectMeta, ObjectStore}; use rand::distributions::{Alphanumeric, DistString}; /// Character Separated Value `FileFormat` implementation. @@ -236,10 +236,10 @@ impl FileFormat for CsvFormat { &self, _state: &SessionState, _store: &Arc, - _table_schema: SchemaRef, + table_schema: SchemaRef, _object: &ObjectMeta, ) -> Result { - Ok(Statistics::default()) + Ok(Statistics::new_unknown(&table_schema)) } async fn create_physical_plan( @@ -625,6 +625,7 @@ mod tests { use chrono::DateTime; use datafusion_common::cast::as_string_array; use datafusion_common::internal_err; + use datafusion_common::stats::Precision; use datafusion_common::FileType; use datafusion_common::GetExt; use datafusion_expr::{col, lit}; @@ -657,8 +658,8 @@ mod tests { assert_eq!(tt_batches, 50 /* 100/2 */); // test metadata - assert_eq!(exec.statistics().num_rows, None); - assert_eq!(exec.statistics().total_byte_size, None); + assert_eq!(exec.statistics()?.num_rows, Precision::Absent); + assert_eq!(exec.statistics()?.total_byte_size, Precision::Absent); Ok(()) } diff --git a/datafusion/core/src/datasource/file_format/json.rs b/datafusion/core/src/datasource/file_format/json.rs index fa8fb5a72331..dc5b24b2ea10 100644 --- a/datafusion/core/src/datasource/file_format/json.rs +++ b/datafusion/core/src/datasource/file_format/json.rs @@ -18,51 +18,38 @@ //! Line delimited JSON format abstractions use std::any::Any; - -use bytes::Bytes; -use datafusion_common::not_impl_err; -use datafusion_common::DataFusionError; -use datafusion_common::FileType; -use datafusion_execution::TaskContext; -use datafusion_physical_expr::PhysicalSortRequirement; -use datafusion_physical_plan::metrics::MetricsSet; -use rand::distributions::Alphanumeric; -use rand::distributions::DistString; use std::fmt; use std::fmt::Debug; use std::io::BufReader; use std::sync::Arc; -use arrow::datatypes::Schema; -use arrow::datatypes::SchemaRef; -use arrow::json; -use arrow::json::reader::infer_json_schema_from_iterator; -use arrow::json::reader::ValueIter; -use arrow_array::RecordBatch; -use async_trait::async_trait; -use bytes::Buf; - -use datafusion_physical_expr::PhysicalExpr; -use object_store::{GetResultPayload, ObjectMeta, ObjectStore}; - -use crate::datasource::physical_plan::FileGroupDisplay; -use crate::physical_plan::insert::DataSink; -use crate::physical_plan::insert::FileSinkExec; -use crate::physical_plan::SendableRecordBatchStream; -use crate::physical_plan::{DisplayAs, DisplayFormatType, Statistics}; - -use super::FileFormat; -use super::FileScanConfig; +use super::{FileFormat, FileScanConfig}; use crate::datasource::file_format::file_compression_type::FileCompressionType; use crate::datasource::file_format::write::{ create_writer, stateless_serialize_and_write_files, BatchSerializer, FileWriterMode, }; use crate::datasource::file_format::DEFAULT_SCHEMA_INFER_MAX_RECORD; -use crate::datasource::physical_plan::FileSinkConfig; -use crate::datasource::physical_plan::NdJsonExec; +use crate::datasource::physical_plan::{FileGroupDisplay, FileSinkConfig, NdJsonExec}; use crate::error::Result; use crate::execution::context::SessionState; -use crate::physical_plan::ExecutionPlan; +use crate::physical_plan::insert::{DataSink, FileSinkExec}; +use crate::physical_plan::{ + DisplayAs, DisplayFormatType, ExecutionPlan, SendableRecordBatchStream, Statistics, +}; + +use arrow::datatypes::{Schema, SchemaRef}; +use arrow::json; +use arrow::json::reader::{infer_json_schema_from_iterator, ValueIter}; +use arrow_array::RecordBatch; +use datafusion_common::{not_impl_err, DataFusionError, FileType}; +use datafusion_execution::TaskContext; +use datafusion_physical_expr::{PhysicalExpr, PhysicalSortRequirement}; +use datafusion_physical_plan::metrics::MetricsSet; + +use async_trait::async_trait; +use bytes::{Buf, Bytes}; +use object_store::{GetResultPayload, ObjectMeta, ObjectStore}; +use rand::distributions::{Alphanumeric, DistString}; /// New line delimited JSON `FileFormat` implementation. #[derive(Debug)] @@ -154,10 +141,10 @@ impl FileFormat for JsonFormat { &self, _state: &SessionState, _store: &Arc, - _table_schema: SchemaRef, + table_schema: SchemaRef, _object: &ObjectMeta, ) -> Result { - Ok(Statistics::default()) + Ok(Statistics::new_unknown(&table_schema)) } async fn create_physical_plan( @@ -394,6 +381,7 @@ impl DataSink for JsonSink { mod tests { use super::super::test_util::scan_format; use datafusion_common::cast::as_int64_array; + use datafusion_common::stats::Precision; use futures::StreamExt; use object_store::local::LocalFileSystem; @@ -424,8 +412,8 @@ mod tests { assert_eq!(tt_batches, 6 /* 12/2 */); // test metadata - assert_eq!(exec.statistics().num_rows, None); - assert_eq!(exec.statistics().total_byte_size, None); + assert_eq!(exec.statistics()?.num_rows, Precision::Absent); + assert_eq!(exec.statistics()?.total_byte_size, Precision::Absent); Ok(()) } diff --git a/datafusion/core/src/datasource/file_format/parquet.rs b/datafusion/core/src/datasource/file_format/parquet.rs index d946bfb0b90d..859bff7ae46f 100644 --- a/datafusion/core/src/datasource/file_format/parquet.rs +++ b/datafusion/core/src/datasource/file_format/parquet.rs @@ -17,51 +17,22 @@ //! Parquet format abstractions -use datafusion_physical_plan::metrics::MetricsSet; -use parquet::column::writer::ColumnCloseResult; -use parquet::file::writer::SerializedFileWriter; -use rand::distributions::DistString; use std::any::Any; use std::fmt; use std::fmt::Debug; use std::io::Write; use std::sync::Arc; -use tokio::io::{AsyncWrite, AsyncWriteExt}; -use tokio::sync::mpsc::{UnboundedReceiver, UnboundedSender}; -use tokio::task::{JoinHandle, JoinSet}; - -use crate::datasource::file_format::file_compression_type::FileCompressionType; -use arrow::datatypes::SchemaRef; -use arrow::datatypes::{Fields, Schema}; -use async_trait::async_trait; -use bytes::{BufMut, BytesMut}; -use datafusion_common::{exec_err, not_impl_err, plan_err, DataFusionError, FileType}; -use datafusion_execution::TaskContext; -use datafusion_physical_expr::{PhysicalExpr, PhysicalSortRequirement}; -use futures::{StreamExt, TryStreamExt}; -use hashbrown::HashMap; -use object_store::{ObjectMeta, ObjectStore}; -use parquet::arrow::{parquet_to_arrow_schema, AsyncArrowWriter}; -use parquet::file::footer::{decode_footer, decode_metadata}; -use parquet::file::metadata::ParquetMetaData; -use parquet::file::properties::WriterProperties; -use parquet::file::statistics::Statistics as ParquetStatistics; -use rand::distributions::Alphanumeric; use super::write::{create_writer, AbortableWrite, FileWriterMode}; -use super::FileFormat; -use super::FileScanConfig; -use crate::arrow::array::{ - BooleanArray, Float32Array, Float64Array, Int32Array, Int64Array, -}; -use crate::arrow::datatypes::DataType; -use crate::config::ConfigOptions; +use super::{FileFormat, FileScanConfig}; +use crate::config::ConfigOptions; +use crate::datasource::file_format::file_compression_type::FileCompressionType; +use crate::datasource::get_col_stats; use crate::datasource::physical_plan::{ FileGroupDisplay, FileMeta, FileSinkConfig, ParquetExec, SchemaAdapter, }; - -use crate::datasource::{create_max_min_accs, get_col_stats}; +use crate::datasource::statistics::create_max_min_accs; use crate::error::Result; use crate::execution::context::SessionState; use crate::physical_plan::expressions::{MaxAccumulator, MinAccumulator}; @@ -71,6 +42,31 @@ use crate::physical_plan::{ Statistics, }; +use arrow::array::{BooleanArray, Float32Array, Float64Array, Int32Array, Int64Array}; +use arrow::datatypes::{DataType, Fields, Schema, SchemaRef}; +use datafusion_common::stats::Precision; +use datafusion_common::{exec_err, not_impl_err, plan_err, DataFusionError, FileType}; +use datafusion_execution::TaskContext; +use datafusion_physical_expr::{PhysicalExpr, PhysicalSortRequirement}; +use datafusion_physical_plan::metrics::MetricsSet; + +use async_trait::async_trait; +use bytes::{BufMut, BytesMut}; +use futures::{StreamExt, TryStreamExt}; +use hashbrown::HashMap; +use object_store::{ObjectMeta, ObjectStore}; +use parquet::arrow::{parquet_to_arrow_schema, AsyncArrowWriter}; +use parquet::column::writer::ColumnCloseResult; +use parquet::file::footer::{decode_footer, decode_metadata}; +use parquet::file::metadata::ParquetMetaData; +use parquet::file::properties::WriterProperties; +use parquet::file::statistics::Statistics as ParquetStatistics; +use parquet::file::writer::SerializedFileWriter; +use rand::distributions::{Alphanumeric, DistString}; +use tokio::io::{AsyncWrite, AsyncWriteExt}; +use tokio::sync::mpsc::{UnboundedReceiver, UnboundedSender}; +use tokio::task::{JoinHandle, JoinSet}; + /// The Apache Parquet `FileFormat` implementation /// /// Note it is recommended these are instead configured on the [`ConfigOptions`] @@ -515,7 +511,7 @@ async fn fetch_statistics( let mut num_rows = 0; let mut total_byte_size = 0; - let mut null_counts = vec![0; num_fields]; + let mut null_counts = vec![Precision::Exact(0); num_fields]; let mut has_statistics = false; let schema_adapter = SchemaAdapter::new(table_schema.clone()); @@ -541,7 +537,7 @@ async fn fetch_statistics( schema_adapter.map_column_index(table_idx, &file_schema) { if let Some((null_count, stats)) = column_stats.get(&file_idx) { - *null_cnt += *null_count as usize; + *null_cnt = null_cnt.add(&Precision::Exact(*null_count as usize)); summarize_min_max( &mut max_values, &mut min_values, @@ -555,28 +551,22 @@ async fn fetch_statistics( min_values[table_idx] = None; } } else { - *null_cnt += num_rows as usize; + *null_cnt = null_cnt.add(&Precision::Exact(num_rows as usize)); } } } } let column_stats = if has_statistics { - Some(get_col_stats( - &table_schema, - null_counts, - &mut max_values, - &mut min_values, - )) + get_col_stats(&table_schema, null_counts, &mut max_values, &mut min_values) } else { - None + Statistics::unknown_column(&table_schema) }; let statistics = Statistics { - num_rows: Some(num_rows as usize), - total_byte_size: Some(total_byte_size as usize), + num_rows: Precision::Exact(num_rows as usize), + total_byte_size: Precision::Exact(total_byte_size as usize), column_statistics: column_stats, - is_exact: true, }; Ok(statistics) @@ -1186,20 +1176,26 @@ mod tests { let stats = fetch_statistics(store.as_ref(), schema.clone(), &meta[0], None).await?; - assert_eq!(stats.num_rows, Some(3)); - let c1_stats = &stats.column_statistics.as_ref().expect("missing c1 stats")[0]; - let c2_stats = &stats.column_statistics.as_ref().expect("missing c2 stats")[1]; - assert_eq!(c1_stats.null_count, Some(1)); - assert_eq!(c2_stats.null_count, Some(3)); + assert_eq!(stats.num_rows, Precision::Exact(3)); + let c1_stats = &stats.column_statistics[0]; + let c2_stats = &stats.column_statistics[1]; + assert_eq!(c1_stats.null_count, Precision::Exact(1)); + assert_eq!(c2_stats.null_count, Precision::Exact(3)); let stats = fetch_statistics(store.as_ref(), schema, &meta[1], None).await?; - assert_eq!(stats.num_rows, Some(3)); - let c1_stats = &stats.column_statistics.as_ref().expect("missing c1 stats")[0]; - let c2_stats = &stats.column_statistics.as_ref().expect("missing c2 stats")[1]; - assert_eq!(c1_stats.null_count, Some(3)); - assert_eq!(c2_stats.null_count, Some(1)); - assert_eq!(c2_stats.max_value, Some(ScalarValue::Int64(Some(2)))); - assert_eq!(c2_stats.min_value, Some(ScalarValue::Int64(Some(1)))); + assert_eq!(stats.num_rows, Precision::Exact(3)); + let c1_stats = &stats.column_statistics[0]; + let c2_stats = &stats.column_statistics[1]; + assert_eq!(c1_stats.null_count, Precision::Exact(3)); + assert_eq!(c2_stats.null_count, Precision::Exact(1)); + assert_eq!( + c2_stats.max_value, + Precision::Exact(ScalarValue::Int64(Some(2))) + ); + assert_eq!( + c2_stats.min_value, + Precision::Exact(ScalarValue::Int64(Some(1))) + ); Ok(()) } @@ -1335,11 +1331,11 @@ mod tests { fetch_statistics(store.upcast().as_ref(), schema.clone(), &meta[0], Some(9)) .await?; - assert_eq!(stats.num_rows, Some(3)); - let c1_stats = &stats.column_statistics.as_ref().expect("missing c1 stats")[0]; - let c2_stats = &stats.column_statistics.as_ref().expect("missing c2 stats")[1]; - assert_eq!(c1_stats.null_count, Some(1)); - assert_eq!(c2_stats.null_count, Some(3)); + assert_eq!(stats.num_rows, Precision::Exact(3)); + let c1_stats = &stats.column_statistics[0]; + let c2_stats = &stats.column_statistics[1]; + assert_eq!(c1_stats.null_count, Precision::Exact(1)); + assert_eq!(c2_stats.null_count, Precision::Exact(3)); let store = Arc::new(RequestCountingObjectStore::new(Arc::new( LocalFileSystem::new(), @@ -1368,11 +1364,11 @@ mod tests { ) .await?; - assert_eq!(stats.num_rows, Some(3)); - let c1_stats = &stats.column_statistics.as_ref().expect("missing c1 stats")[0]; - let c2_stats = &stats.column_statistics.as_ref().expect("missing c2 stats")[1]; - assert_eq!(c1_stats.null_count, Some(1)); - assert_eq!(c2_stats.null_count, Some(3)); + assert_eq!(stats.num_rows, Precision::Exact(3)); + let c1_stats = &stats.column_statistics[0]; + let c2_stats = &stats.column_statistics[1]; + assert_eq!(c1_stats.null_count, Precision::Exact(1)); + assert_eq!(c2_stats.null_count, Precision::Exact(3)); let store = Arc::new(RequestCountingObjectStore::new(Arc::new( LocalFileSystem::new(), @@ -1412,8 +1408,8 @@ mod tests { assert_eq!(tt_batches, 4 /* 8/2 */); // test metadata - assert_eq!(exec.statistics().num_rows, Some(8)); - assert_eq!(exec.statistics().total_byte_size, Some(671)); + assert_eq!(exec.statistics()?.num_rows, Precision::Exact(8)); + assert_eq!(exec.statistics()?.total_byte_size, Precision::Exact(671)); Ok(()) } @@ -1454,9 +1450,8 @@ mod tests { get_exec(&state, "alltypes_plain.parquet", projection, Some(1)).await?; // note: even if the limit is set, the executor rounds up to the batch size - assert_eq!(exec.statistics().num_rows, Some(8)); - assert_eq!(exec.statistics().total_byte_size, Some(671)); - assert!(exec.statistics().is_exact); + assert_eq!(exec.statistics()?.num_rows, Precision::Exact(8)); + assert_eq!(exec.statistics()?.total_byte_size, Precision::Exact(671)); let batches = collect(exec, task_ctx).await?; assert_eq!(1, batches.len()); assert_eq!(11, batches[0].num_columns()); diff --git a/datafusion/core/src/datasource/listing/table.rs b/datafusion/core/src/datasource/listing/table.rs index 6ef214c97fd5..ca86e3e3c7e1 100644 --- a/datafusion/core/src/datasource/listing/table.rs +++ b/datafusion/core/src/datasource/listing/table.rs @@ -962,7 +962,7 @@ impl ListingTable { let store = if let Some(url) = self.table_paths.get(0) { ctx.runtime_env().object_store(url)? } else { - return Ok((vec![], Statistics::default())); + return Ok((vec![], Statistics::new_unknown(&self.file_schema))); }; // list files (with partitions) let file_list = future::try_join_all(self.table_paths.iter().map(|table_path| { @@ -976,14 +976,12 @@ impl ListingTable { ) })) .await?; - let file_list = stream::iter(file_list).flatten(); - // collect the statistics if required by the config let files = file_list .map(|part_file| async { let part_file = part_file?; - let mut statistics_result = Statistics::default(); + let mut statistics_result = Statistics::new_unknown(&self.file_schema); if self.options.collect_stat { let statistics_cache = self.collected_statistics.clone(); match statistics_cache.get_with_extra( @@ -1051,9 +1049,9 @@ mod tests { use arrow::datatypes::{DataType, Schema}; use arrow::record_batch::RecordBatch; + use datafusion_common::stats::Precision; use datafusion_common::{assert_contains, GetExt, ScalarValue}; use datafusion_expr::{BinaryExpr, LogicalPlanBuilder, Operator}; - use rstest::*; use tempfile::TempDir; @@ -1100,8 +1098,8 @@ mod tests { assert_eq!(exec.output_partitioning().partition_count(), 1); // test metadata - assert_eq!(exec.statistics().num_rows, Some(8)); - assert_eq!(exec.statistics().total_byte_size, Some(671)); + assert_eq!(exec.statistics()?.num_rows, Precision::Exact(8)); + assert_eq!(exec.statistics()?.total_byte_size, Precision::Exact(671)); Ok(()) } @@ -1123,8 +1121,8 @@ mod tests { let table = ListingTable::try_new(config)?; let exec = table.scan(&state, None, &[], None).await?; - assert_eq!(exec.statistics().num_rows, Some(8)); - assert_eq!(exec.statistics().total_byte_size, Some(671)); + assert_eq!(exec.statistics()?.num_rows, Precision::Exact(8)); + assert_eq!(exec.statistics()?.total_byte_size, Precision::Exact(671)); Ok(()) } @@ -1147,8 +1145,8 @@ mod tests { let table = ListingTable::try_new(config)?; let exec = table.scan(&state, None, &[], None).await?; - assert_eq!(exec.statistics().num_rows, None); - assert_eq!(exec.statistics().total_byte_size, None); + assert_eq!(exec.statistics()?.num_rows, Precision::Absent); + assert_eq!(exec.statistics()?.total_byte_size, Precision::Absent); Ok(()) } diff --git a/datafusion/core/src/datasource/mod.rs b/datafusion/core/src/datasource/mod.rs index 35f56536510c..455818056f2c 100644 --- a/datafusion/core/src/datasource/mod.rs +++ b/datafusion/core/src/datasource/mod.rs @@ -42,5 +42,5 @@ pub use self::memory::MemTable; pub use self::provider::TableProvider; pub use self::view::ViewTable; pub use crate::logical_expr::TableType; +pub(crate) use statistics::get_col_stats; pub use statistics::get_statistics_with_limit; -pub(crate) use statistics::{create_max_min_accs, get_col_stats}; diff --git a/datafusion/core/src/datasource/physical_plan/arrow_file.rs b/datafusion/core/src/datasource/physical_plan/arrow_file.rs index a47376248ed3..e00e8aea0a04 100644 --- a/datafusion/core/src/datasource/physical_plan/arrow_file.rs +++ b/datafusion/core/src/datasource/physical_plan/arrow_file.rs @@ -16,6 +16,10 @@ // under the License. //! Execution plan for reading Arrow files + +use std::any::Any; +use std::sync::Arc; + use crate::datasource::physical_plan::{ FileMeta, FileOpenFuture, FileOpener, FileScanConfig, }; @@ -24,6 +28,7 @@ use crate::physical_plan::metrics::{ExecutionPlanMetricsSet, MetricsSet}; use crate::physical_plan::{ DisplayAs, DisplayFormatType, ExecutionPlan, Partitioning, SendableRecordBatchStream, }; + use arrow_schema::SchemaRef; use datafusion_common::Statistics; use datafusion_execution::TaskContext; @@ -31,10 +36,9 @@ use datafusion_physical_expr::{ ordering_equivalence_properties_helper, LexOrdering, OrderingEquivalenceProperties, PhysicalSortExpr, }; + use futures::StreamExt; use object_store::{GetResultPayload, ObjectStore}; -use std::any::Any; -use std::sync::Arc; /// Execution plan for scanning Arrow data source #[derive(Debug, Clone)] @@ -143,8 +147,8 @@ impl ExecutionPlan for ArrowExec { Some(self.metrics.clone_inner()) } - fn statistics(&self) -> Statistics { - self.projected_statistics.clone() + fn statistics(&self) -> Result { + Ok(self.projected_statistics.clone()) } } diff --git a/datafusion/core/src/datasource/physical_plan/avro.rs b/datafusion/core/src/datasource/physical_plan/avro.rs index 93655e8665f0..f08bc9b8df20 100644 --- a/datafusion/core/src/datasource/physical_plan/avro.rs +++ b/datafusion/core/src/datasource/physical_plan/avro.rs @@ -16,6 +16,11 @@ // under the License. //! Execution plan for reading line-delimited Avro files + +use std::any::Any; +use std::sync::Arc; + +use super::FileScanConfig; use crate::error::Result; use crate::physical_plan::expressions::PhysicalSortExpr; use crate::physical_plan::metrics::{ExecutionPlanMetricsSet, MetricsSet}; @@ -23,18 +28,13 @@ use crate::physical_plan::{ DisplayAs, DisplayFormatType, ExecutionPlan, Partitioning, SendableRecordBatchStream, Statistics, }; -use datafusion_execution::TaskContext; use arrow::datatypes::SchemaRef; +use datafusion_execution::TaskContext; use datafusion_physical_expr::{ ordering_equivalence_properties_helper, LexOrdering, OrderingEquivalenceProperties, }; -use std::any::Any; -use std::sync::Arc; - -use super::FileScanConfig; - /// Execution plan for scanning Avro data source #[derive(Debug, Clone)] #[allow(dead_code)] @@ -154,8 +154,8 @@ impl ExecutionPlan for AvroExec { Ok(Box::pin(stream)) } - fn statistics(&self) -> Statistics { - self.projected_statistics.clone() + fn statistics(&self) -> Result { + Ok(self.projected_statistics.clone()) } fn metrics(&self) -> Option { @@ -272,8 +272,8 @@ mod tests { let avro_exec = AvroExec::new(FileScanConfig { object_store_url: ObjectStoreUrl::local_filesystem(), file_groups: vec![vec![meta.into()]], + statistics: Statistics::new_unknown(&file_schema), file_schema, - statistics: Statistics::default(), projection: Some(vec![0, 1, 2]), limit: None, table_partition_cols: vec![], @@ -344,8 +344,8 @@ mod tests { let avro_exec = AvroExec::new(FileScanConfig { object_store_url, file_groups: vec![vec![meta.into()]], + statistics: Statistics::new_unknown(&file_schema), file_schema, - statistics: Statistics::default(), projection, limit: None, table_partition_cols: vec![], @@ -417,8 +417,8 @@ mod tests { projection: Some(vec![0, 1, file_schema.fields().len(), 2]), object_store_url, file_groups: vec![vec![partitioned_file]], + statistics: Statistics::new_unknown(&file_schema), file_schema, - statistics: Statistics::default(), limit: None, table_partition_cols: vec![("date".to_owned(), DataType::Utf8)], output_ordering: vec![], diff --git a/datafusion/core/src/datasource/physical_plan/csv.rs b/datafusion/core/src/datasource/physical_plan/csv.rs index 8277836a24ff..f3b2fa9de7a9 100644 --- a/datafusion/core/src/datasource/physical_plan/csv.rs +++ b/datafusion/core/src/datasource/physical_plan/csv.rs @@ -17,6 +17,13 @@ //! Execution plan for reading CSV files +use std::any::Any; +use std::io::{Read, Seek, SeekFrom}; +use std::ops::Range; +use std::sync::Arc; +use std::task::Poll; + +use super::FileScanConfig; use crate::datasource::file_format::file_compression_type::FileCompressionType; use crate::datasource::listing::{FileRange, ListingTableUrl}; use crate::datasource::physical_plan::file_stream::{ @@ -30,25 +37,18 @@ use crate::physical_plan::{ DisplayAs, DisplayFormatType, ExecutionPlan, Partitioning, SendableRecordBatchStream, Statistics, }; + use arrow::csv; use arrow::datatypes::SchemaRef; use datafusion_execution::TaskContext; use datafusion_physical_expr::{ ordering_equivalence_properties_helper, LexOrdering, OrderingEquivalenceProperties, }; -use tokio::io::AsyncWriteExt; - -use super::FileScanConfig; use bytes::{Buf, Bytes}; -use futures::ready; -use futures::{StreamExt, TryStreamExt}; +use futures::{ready, StreamExt, TryStreamExt}; use object_store::{GetOptions, GetResultPayload, ObjectStore}; -use std::any::Any; -use std::io::{Read, Seek, SeekFrom}; -use std::ops::Range; -use std::sync::Arc; -use std::task::Poll; +use tokio::io::AsyncWriteExt; use tokio::task::JoinSet; /// Execution plan for scanning a CSV file @@ -234,8 +234,8 @@ impl ExecutionPlan for CsvExec { Ok(Box::pin(stream) as SendableRecordBatchStream) } - fn statistics(&self) -> Statistics { - self.projected_statistics.clone() + fn statistics(&self) -> Result { + Ok(self.projected_statistics.clone()) } fn metrics(&self) -> Option { diff --git a/datafusion/core/src/datasource/physical_plan/file_scan_config.rs b/datafusion/core/src/datasource/physical_plan/file_scan_config.rs index 819bfabae290..c1a19b745b8d 100644 --- a/datafusion/core/src/datasource/physical_plan/file_scan_config.rs +++ b/datafusion/core/src/datasource/physical_plan/file_scan_config.rs @@ -18,6 +18,12 @@ //! [`FileScanConfig`] to configure scanning of possibly partitioned //! file sources. +use std::{ + borrow::Cow, cmp::min, collections::HashMap, fmt::Debug, marker::PhantomData, + sync::Arc, vec, +}; + +use super::get_projected_output_ordering; use crate::datasource::{ listing::{FileRange, PartitionedFile}, object_store::ObjectStoreUrl, @@ -32,18 +38,12 @@ use arrow::buffer::Buffer; use arrow::datatypes::{ArrowNativeType, UInt16Type}; use arrow_array::{ArrayRef, DictionaryArray, RecordBatch}; use arrow_schema::{DataType, Field, Schema, SchemaRef}; -use datafusion_common::exec_err; -use datafusion_common::{ColumnStatistics, Statistics}; +use datafusion_common::stats::Precision; +use datafusion_common::{exec_err, ColumnStatistics, Statistics}; use datafusion_physical_expr::LexOrdering; use itertools::Itertools; use log::warn; -use std::{ - borrow::Cow, cmp::min, collections::HashMap, fmt::Debug, marker::PhantomData, - sync::Arc, vec, -}; - -use super::get_projected_output_ordering; /// Convert type to a type suitable for use as a [`ListingTable`] /// partition column. Returns `Dictionary(UInt16, val_type)`, which is @@ -130,30 +130,23 @@ impl FileScanConfig { let mut table_cols_stats = vec![]; for idx in proj_iter { if idx < self.file_schema.fields().len() { - table_fields.push(self.file_schema.field(idx).clone()); - if let Some(file_cols_stats) = &self.statistics.column_statistics { - table_cols_stats.push(file_cols_stats[idx].clone()) - } else { - table_cols_stats.push(ColumnStatistics::default()) - } + let field = self.file_schema.field(idx); + table_fields.push(field.clone()); + table_cols_stats.push(self.statistics.column_statistics[idx].clone()) } else { let partition_idx = idx - self.file_schema.fields().len(); - table_fields.push(Field::new( - &self.table_partition_cols[partition_idx].0, - self.table_partition_cols[partition_idx].1.to_owned(), - false, - )); + let (name, dtype) = &self.table_partition_cols[partition_idx]; + table_fields.push(Field::new(name, dtype.to_owned(), false)); // TODO provide accurate stat for partition column (#1186) - table_cols_stats.push(ColumnStatistics::default()) + table_cols_stats.push(ColumnStatistics::new_unknown()) } } let table_stats = Statistics { - num_rows: self.statistics.num_rows, - is_exact: self.statistics.is_exact, + num_rows: self.statistics.num_rows.clone(), // TODO correct byte size? - total_byte_size: None, - column_statistics: Some(table_cols_stats), + total_byte_size: Precision::Absent, + column_statistics: table_cols_stats, }; let table_schema = Arc::new( @@ -507,7 +500,7 @@ mod tests { let conf = config_for_projection( Arc::clone(&file_schema), None, - Statistics::default(), + Statistics::new_unknown(&file_schema), vec![( "date".to_owned(), wrap_partition_type_in_dict(DataType::Utf8), @@ -522,10 +515,7 @@ mod tests { "partition columns are the last columns" ); assert_eq!( - proj_statistics - .column_statistics - .expect("projection creates column statistics") - .len(), + proj_statistics.column_statistics.len(), file_schema.fields().len() + 1 ); // TODO implement tests for partition column statistics once implemented @@ -544,18 +534,16 @@ mod tests { Arc::clone(&file_schema), Some(vec![file_schema.fields().len(), 0]), Statistics { - num_rows: Some(10), + num_rows: Precision::Inexact(10), // assign the column index to distinct_count to help assert // the source statistic after the projection - column_statistics: Some( - (0..file_schema.fields().len()) - .map(|i| ColumnStatistics { - distinct_count: Some(i), - ..Default::default() - }) - .collect(), - ), - ..Default::default() + column_statistics: (0..file_schema.fields().len()) + .map(|i| ColumnStatistics { + distinct_count: Precision::Inexact(i), + ..Default::default() + }) + .collect(), + total_byte_size: Precision::Absent, }, vec![( "date".to_owned(), @@ -568,13 +556,11 @@ mod tests { columns(&proj_schema), vec!["date".to_owned(), "c1".to_owned()] ); - let proj_stat_cols = proj_statistics - .column_statistics - .expect("projection creates column statistics"); + let proj_stat_cols = proj_statistics.column_statistics; assert_eq!(proj_stat_cols.len(), 2); // TODO implement tests for proj_stat_cols[0] once partition column // statistics are implemented - assert_eq!(proj_stat_cols[1].distinct_count, Some(0)); + assert_eq!(proj_stat_cols[1].distinct_count, Precision::Inexact(0)); let col_names = conf.projected_file_column_names(); assert_eq!(col_names, Some(vec!["c1".to_owned()])); @@ -615,7 +601,7 @@ mod tests { file_batch.schema().fields().len(), file_batch.schema().fields().len() + 2, ]), - Statistics::default(), + Statistics::new_unknown(&file_batch.schema()), partition_cols.clone(), ); let (proj_schema, ..) = conf.project(); diff --git a/datafusion/core/src/datasource/physical_plan/file_stream.rs b/datafusion/core/src/datasource/physical_plan/file_stream.rs index af304f40dd86..487d17d9c1c3 100644 --- a/datafusion/core/src/datasource/physical_plan/file_stream.rs +++ b/datafusion/core/src/datasource/physical_plan/file_stream.rs @@ -521,6 +521,7 @@ mod tests { use arrow_schema::Schema; use datafusion_common::internal_err; use datafusion_common::DataFusionError; + use datafusion_common::Statistics; use super::*; use crate::datasource::file_format::write::BatchSerializer; @@ -659,9 +660,9 @@ mod tests { let config = FileScanConfig { object_store_url: ObjectStoreUrl::parse("test:///").unwrap(), + statistics: Statistics::new_unknown(&file_schema), file_schema, file_groups: vec![file_group], - statistics: Default::default(), projection: None, limit: self.limit, table_partition_cols: vec![], diff --git a/datafusion/core/src/datasource/physical_plan/json.rs b/datafusion/core/src/datasource/physical_plan/json.rs index 493110404119..1ba8e47a523c 100644 --- a/datafusion/core/src/datasource/physical_plan/json.rs +++ b/datafusion/core/src/datasource/physical_plan/json.rs @@ -16,6 +16,13 @@ // under the License. //! Execution plan for reading line-delimited JSON files + +use std::any::Any; +use std::io::BufReader; +use std::sync::Arc; +use std::task::Poll; + +use super::FileScanConfig; use crate::datasource::file_format::file_compression_type::FileCompressionType; use crate::datasource::listing::ListingTableUrl; use crate::datasource::physical_plan::file_stream::{ @@ -29,10 +36,10 @@ use crate::physical_plan::{ DisplayAs, DisplayFormatType, ExecutionPlan, Partitioning, SendableRecordBatchStream, Statistics, }; -use datafusion_execution::TaskContext; use arrow::json::ReaderBuilder; use arrow::{datatypes::SchemaRef, json}; +use datafusion_execution::TaskContext; use datafusion_physical_expr::{ ordering_equivalence_properties_helper, LexOrdering, OrderingEquivalenceProperties, }; @@ -41,15 +48,9 @@ use bytes::{Buf, Bytes}; use futures::{ready, stream, StreamExt, TryStreamExt}; use object_store; use object_store::{GetResultPayload, ObjectStore}; -use std::any::Any; -use std::io::BufReader; -use std::sync::Arc; -use std::task::Poll; use tokio::io::AsyncWriteExt; use tokio::task::JoinSet; -use super::FileScanConfig; - /// Execution plan for scanning NdJson data source #[derive(Debug, Clone)] pub struct NdJsonExec { @@ -163,8 +164,8 @@ impl ExecutionPlan for NdJsonExec { Ok(Box::pin(stream) as SendableRecordBatchStream) } - fn statistics(&self) -> Statistics { - self.projected_statistics.clone() + fn statistics(&self) -> Result { + Ok(self.projected_statistics.clone()) } fn metrics(&self) -> Option { @@ -457,8 +458,8 @@ mod tests { FileScanConfig { object_store_url, file_groups, + statistics: Statistics::new_unknown(&file_schema), file_schema, - statistics: Statistics::default(), projection: None, limit: Some(3), table_partition_cols: vec![], @@ -536,8 +537,8 @@ mod tests { FileScanConfig { object_store_url, file_groups, + statistics: Statistics::new_unknown(&file_schema), file_schema, - statistics: Statistics::default(), projection: None, limit: Some(3), table_partition_cols: vec![], @@ -584,8 +585,8 @@ mod tests { FileScanConfig { object_store_url, file_groups, + statistics: Statistics::new_unknown(&file_schema), file_schema, - statistics: Statistics::default(), projection: Some(vec![0, 2]), limit: None, table_partition_cols: vec![], @@ -637,8 +638,8 @@ mod tests { FileScanConfig { object_store_url, file_groups, + statistics: Statistics::new_unknown(&file_schema), file_schema, - statistics: Statistics::default(), projection: Some(vec![3, 0, 2]), limit: None, table_partition_cols: vec![], diff --git a/datafusion/core/src/datasource/physical_plan/mod.rs b/datafusion/core/src/datasource/physical_plan/mod.rs index d8ae6b3c04e6..57844aac5181 100644 --- a/datafusion/core/src/datasource/physical_plan/mod.rs +++ b/datafusion/core/src/datasource/physical_plan/mod.rs @@ -20,32 +20,33 @@ mod arrow_file; mod avro; mod csv; +mod file_scan_config; mod file_stream; mod json; pub mod parquet; pub(crate) use self::csv::plan_to_csv; pub use self::csv::{CsvConfig, CsvExec, CsvOpener}; +pub(crate) use self::file_scan_config::PartitionColumnProjector; +pub(crate) use self::json::plan_to_json; pub(crate) use self::parquet::plan_to_parquet; pub use self::parquet::{ParquetExec, ParquetFileMetrics, ParquetFileReaderFactory}; -use arrow::{ - array::new_null_array, - compute::can_cast_types, - datatypes::{DataType, Schema, SchemaRef}, - record_batch::{RecordBatch, RecordBatchOptions}, -}; + pub use arrow_file::ArrowExec; pub use avro::AvroExec; -use datafusion_physical_expr::PhysicalSortExpr; -pub use file_stream::{FileOpenFuture, FileOpener, FileStream, OnError}; -pub(crate) use json::plan_to_json; -pub use json::{JsonOpener, NdJsonExec}; -mod file_scan_config; -pub(crate) use file_scan_config::PartitionColumnProjector; pub use file_scan_config::{ wrap_partition_type_in_dict, wrap_partition_value_in_dict, FileScanConfig, }; +pub use file_stream::{FileOpenFuture, FileOpener, FileStream, OnError}; +pub use json::{JsonOpener, NdJsonExec}; +use std::{ + fmt::{Debug, Formatter, Result as FmtResult}, + sync::Arc, + vec, +}; + +use super::listing::ListingTableUrl; use crate::error::{DataFusionError, Result}; use crate::{ datasource::file_format::write::FileWriterMode, @@ -59,21 +60,20 @@ use crate::{ physical_plan::display::{OutputOrderingDisplay, ProjectSchemaDisplay}, }; +use arrow::{ + array::new_null_array, + compute::{can_cast_types, cast}, + datatypes::{DataType, Schema, SchemaRef}, + record_batch::{RecordBatch, RecordBatchOptions}, +}; use datafusion_common::{file_options::FileTypeWriterOptions, plan_err}; use datafusion_physical_expr::expressions::Column; - -use arrow::compute::cast; +use datafusion_physical_expr::PhysicalSortExpr; use datafusion_physical_plan::ExecutionPlan; + use log::debug; use object_store::path::Path; use object_store::ObjectMeta; -use std::{ - fmt::{Debug, Formatter, Result as FmtResult}, - sync::Arc, - vec, -}; - -use super::listing::ListingTableUrl; /// The base configurations to provide when creating a physical plan for /// writing to any given file format. @@ -815,7 +815,7 @@ mod tests { object_store_url: ObjectStoreUrl::local_filesystem(), file_groups: file_group, file_schema: Arc::new(Schema::empty()), - statistics: Statistics::default(), + statistics: Statistics::new_unknown(&Schema::empty()), projection: None, limit: None, table_partition_cols: vec![], @@ -878,7 +878,9 @@ mod tests { object_store_url: ObjectStoreUrl::local_filesystem(), file_groups: fg.clone(), file_schema: Arc::new(Schema::empty()), - statistics: Statistics::default(), + statistics: Statistics::new_unknown(&Arc::new( + Schema::empty(), + )), projection: None, limit: None, table_partition_cols: vec![], @@ -912,7 +914,7 @@ mod tests { object_store_url: ObjectStoreUrl::local_filesystem(), file_groups: single_partition, file_schema: Arc::new(Schema::empty()), - statistics: Statistics::default(), + statistics: Statistics::new_unknown(&Schema::empty()), projection: None, limit: None, table_partition_cols: vec![], @@ -949,7 +951,7 @@ mod tests { object_store_url: ObjectStoreUrl::local_filesystem(), file_groups: single_partition, file_schema: Arc::new(Schema::empty()), - statistics: Statistics::default(), + statistics: Statistics::new_unknown(&Schema::empty()), projection: None, limit: None, table_partition_cols: vec![], @@ -992,7 +994,7 @@ mod tests { object_store_url: ObjectStoreUrl::local_filesystem(), file_groups: source_partitions, file_schema: Arc::new(Schema::empty()), - statistics: Statistics::default(), + statistics: Statistics::new_unknown(&Schema::empty()), projection: None, limit: None, table_partition_cols: vec![], @@ -1031,7 +1033,7 @@ mod tests { object_store_url: ObjectStoreUrl::local_filesystem(), file_groups: source_partitions, file_schema: Arc::new(Schema::empty()), - statistics: Statistics::default(), + statistics: Statistics::new_unknown(&Schema::empty()), projection: None, limit: None, table_partition_cols: vec![], @@ -1071,7 +1073,7 @@ mod tests { object_store_url: ObjectStoreUrl::local_filesystem(), file_groups: source_partitions, file_schema: Arc::new(Schema::empty()), - statistics: Statistics::default(), + statistics: Statistics::new_unknown(&Schema::empty()), projection: None, limit: None, table_partition_cols: vec![], @@ -1100,7 +1102,7 @@ mod tests { object_store_url: ObjectStoreUrl::local_filesystem(), file_groups: single_partition, file_schema: Arc::new(Schema::empty()), - statistics: Statistics::default(), + statistics: Statistics::new_unknown(&Schema::empty()), projection: None, limit: None, table_partition_cols: vec![], diff --git a/datafusion/core/src/datasource/physical_plan/parquet.rs b/datafusion/core/src/datasource/physical_plan/parquet.rs index 6f27acfe7f6c..c781ad81c172 100644 --- a/datafusion/core/src/datasource/physical_plan/parquet.rs +++ b/datafusion/core/src/datasource/physical_plan/parquet.rs @@ -17,6 +17,11 @@ //! Execution plan for reading Parquet files +use std::any::Any; +use std::fmt::Debug; +use std::ops::Range; +use std::sync::Arc; + use crate::datasource::physical_plan::file_stream::{ FileOpenFuture, FileOpener, FileStream, }; @@ -36,27 +41,19 @@ use crate::{ Statistics, }, }; -use datafusion_physical_expr::{ - ordering_equivalence_properties_helper, PhysicalSortExpr, -}; -use fmt::Debug; -use object_store::path::Path; -use std::any::Any; -use std::fmt; -use std::ops::Range; -use std::sync::Arc; -use tokio::task::JoinSet; use arrow::datatypes::{DataType, SchemaRef}; use arrow::error::ArrowError; use datafusion_physical_expr::{ - LexOrdering, OrderingEquivalenceProperties, PhysicalExpr, + ordering_equivalence_properties_helper, LexOrdering, OrderingEquivalenceProperties, + PhysicalExpr, PhysicalSortExpr, }; use bytes::Bytes; use futures::future::BoxFuture; use futures::{StreamExt, TryStreamExt}; use log::debug; +use object_store::path::Path; use object_store::ObjectStore; use parquet::arrow::arrow_reader::ArrowReaderOptions; use parquet::arrow::async_reader::{AsyncFileReader, ParquetObjectReader}; @@ -64,6 +61,7 @@ use parquet::arrow::{AsyncArrowWriter, ParquetRecordBatchStreamBuilder, Projecti use parquet::basic::{ConvertedType, LogicalType}; use parquet::file::{metadata::ParquetMetaData, properties::WriterProperties}; use parquet::schema::types::ColumnDescriptor; +use tokio::task::JoinSet; mod metrics; pub mod page_filter; @@ -387,8 +385,8 @@ impl ExecutionPlan for ParquetExec { Some(self.metrics.clone_inner()) } - fn statistics(&self) -> Statistics { - self.projected_statistics.clone() + fn statistics(&self) -> Result { + Ok(self.projected_statistics.clone()) } } @@ -860,8 +858,8 @@ mod tests { FileScanConfig { object_store_url: ObjectStoreUrl::local_filesystem(), file_groups: vec![file_groups], + statistics: Statistics::new_unknown(&file_schema), file_schema, - statistics: Statistics::default(), projection, limit: None, table_partition_cols: vec![], @@ -1517,8 +1515,8 @@ mod tests { FileScanConfig { object_store_url: ObjectStoreUrl::local_filesystem(), file_groups, + statistics: Statistics::new_unknown(&file_schema), file_schema, - statistics: Statistics::default(), projection: None, limit: None, table_partition_cols: vec![], @@ -1620,8 +1618,8 @@ mod tests { FileScanConfig { object_store_url, file_groups: vec![vec![partitioned_file]], - file_schema: schema, - statistics: Statistics::default(), + file_schema: schema.clone(), + statistics: Statistics::new_unknown(&schema), // file has 10 cols so index 12 should be month and 13 should be day projection: Some(vec![0, 1, 2, 12, 13]), limit: None, @@ -1695,7 +1693,7 @@ mod tests { object_store_url: ObjectStoreUrl::local_filesystem(), file_groups: vec![vec![partitioned_file]], file_schema: Arc::new(Schema::empty()), - statistics: Statistics::default(), + statistics: Statistics::new_unknown(&Schema::empty()), projection: None, limit: None, table_partition_cols: vec![], diff --git a/datafusion/core/src/datasource/statistics.rs b/datafusion/core/src/datasource/statistics.rs index 1b6a03e15c02..3d8248dfdeb2 100644 --- a/datafusion/core/src/datasource/statistics.rs +++ b/datafusion/core/src/datasource/statistics.rs @@ -15,14 +15,18 @@ // specific language governing permissions and limitations // under the License. +use super::listing::PartitionedFile; use crate::arrow::datatypes::{Schema, SchemaRef}; use crate::error::Result; use crate::physical_plan::expressions::{MaxAccumulator, MinAccumulator}; use crate::physical_plan::{Accumulator, ColumnStatistics, Statistics}; -use futures::Stream; -use futures::StreamExt; -use super::listing::PartitionedFile; +use datafusion_common::stats::Precision; +use datafusion_common::ScalarValue; + +use futures::{Stream, StreamExt}; +use itertools::izip; +use itertools::multiunzip; /// Get all files as well as the file level summary statistics (no statistic for partition columns). /// If the optional `limit` is provided, includes only sufficient files. @@ -33,101 +37,105 @@ pub async fn get_statistics_with_limit( limit: Option, ) -> Result<(Vec, Statistics)> { let mut result_files = vec![]; + // These statistics can be calculated as long as at least one file provides + // useful information. If none of the files provides any information, then + // they will end up having `Precision::Absent` values. Throughout calculations, + // missing values will be imputed as: + // - zero for summations, and + // - neutral element for extreme points. + let size = file_schema.fields().len(); + let mut null_counts: Vec> = vec![Precision::Absent; size]; + let mut max_values: Vec> = vec![Precision::Absent; size]; + let mut min_values: Vec> = vec![Precision::Absent; size]; + let mut num_rows = Precision::::Absent; + let mut total_byte_size = Precision::::Absent; - let mut null_counts = vec![0; file_schema.fields().len()]; - let mut has_statistics = false; - let (mut max_values, mut min_values) = create_max_min_accs(&file_schema); - - let mut is_exact = true; - - // The number of rows and the total byte size can be calculated as long as - // at least one file has them. If none of the files provide them, then they - // will be omitted from the statistics. The missing values will be counted - // as zero. - let mut num_rows = None; - let mut total_byte_size = None; - - // fusing the stream allows us to call next safely even once it is finished + // Fusing the stream allows us to call next safely even once it is finished. let mut all_files = Box::pin(all_files.fuse()); - while let Some(res) = all_files.next().await { - let (file, file_stats) = res?; + + if let Some(first_file) = all_files.next().await { + let (file, file_stats) = first_file?; result_files.push(file); - is_exact &= file_stats.is_exact; - num_rows = if let Some(num_rows) = num_rows { - Some(num_rows + file_stats.num_rows.unwrap_or(0)) - } else { - file_stats.num_rows - }; - total_byte_size = if let Some(total_byte_size) = total_byte_size { - Some(total_byte_size + file_stats.total_byte_size.unwrap_or(0)) - } else { - file_stats.total_byte_size - }; - if let Some(vec) = &file_stats.column_statistics { - has_statistics = true; - for (i, cs) in vec.iter().enumerate() { - null_counts[i] += cs.null_count.unwrap_or(0); - - if let Some(max_value) = &mut max_values[i] { - if let Some(file_max) = cs.max_value.clone() { - match max_value.update_batch(&[file_max.to_array()]) { - Ok(_) => {} - Err(_) => { - max_values[i] = None; - } - } - } else { - max_values[i] = None; - } - } - if let Some(min_value) = &mut min_values[i] { - if let Some(file_min) = cs.min_value.clone() { - match min_value.update_batch(&[file_min.to_array()]) { - Ok(_) => {} - Err(_) => { - min_values[i] = None; - } - } - } else { - min_values[i] = None; - } - } - } + // First file, we set them directly from the file statistics. + num_rows = file_stats.num_rows; + total_byte_size = file_stats.total_byte_size; + for (index, file_column) in file_stats.column_statistics.into_iter().enumerate() { + null_counts[index] = file_column.null_count; + max_values[index] = file_column.max_value; + min_values[index] = file_column.min_value; } // If the number of rows exceeds the limit, we can stop processing // files. This only applies when we know the number of rows. It also // currently ignores tables that have no statistics regarding the // number of rows. - if num_rows.unwrap_or(usize::MIN) > limit.unwrap_or(usize::MAX) { - break; - } - } - // if we still have files in the stream, it means that the limit kicked - // in and that the statistic could have been different if we processed - // the files in a different order. - if all_files.next().await.is_some() { - is_exact = false; - } + if num_rows.get_value().unwrap_or(&usize::MIN) <= &limit.unwrap_or(usize::MAX) { + while let Some(current) = all_files.next().await { + let (file, file_stats) = current?; + result_files.push(file); + + // We accumulate the number of rows, total byte size and null + // counts across all the files in question. If any file does not + // provide any information or provides an inexact value, we demote + // the statistic precision to inexact. + num_rows = add_row_stats(file_stats.num_rows, num_rows); - let column_stats = if has_statistics { - Some(get_col_stats( - &file_schema, - null_counts, - &mut max_values, - &mut min_values, - )) - } else { - None + total_byte_size = + add_row_stats(file_stats.total_byte_size, total_byte_size); + + (null_counts, max_values, min_values) = multiunzip( + izip!( + file_stats.column_statistics.into_iter(), + null_counts.into_iter(), + max_values.into_iter(), + min_values.into_iter() + ) + .map( + |( + ColumnStatistics { + null_count: file_nc, + max_value: file_max, + min_value: file_min, + distinct_count: _, + }, + null_count, + max_value, + min_value, + )| { + ( + add_row_stats(file_nc, null_count), + set_max_if_greater(file_max, max_value), + set_min_if_lesser(file_min, min_value), + ) + }, + ), + ); + + // If the number of rows exceeds the limit, we can stop processing + // files. This only applies when we know the number of rows. It also + // currently ignores tables that have no statistics regarding the + // number of rows. + if num_rows.get_value().unwrap_or(&usize::MIN) + > &limit.unwrap_or(usize::MAX) + { + break; + } + } + } }; - let statistics = Statistics { + let mut statistics = Statistics { num_rows, total_byte_size, - column_statistics: column_stats, - is_exact, + column_statistics: get_col_stats_vec(null_counts, max_values, min_values), }; + if all_files.next().await.is_some() { + // If we still have files in the stream, it means that the limit kicked + // in, and the statistic could have been different had we processed the + // files in a different order. + statistics = statistics.into_inexact() + } Ok((result_files, statistics)) } @@ -139,18 +147,44 @@ pub(crate) fn create_max_min_accs( .fields() .iter() .map(|field| MaxAccumulator::try_new(field.data_type()).ok()) - .collect::>(); + .collect(); let min_values: Vec> = schema .fields() .iter() .map(|field| MinAccumulator::try_new(field.data_type()).ok()) - .collect::>(); + .collect(); (max_values, min_values) } +fn add_row_stats( + file_num_rows: Precision, + num_rows: Precision, +) -> Precision { + match (file_num_rows, &num_rows) { + (Precision::Absent, _) => num_rows.to_inexact(), + (lhs, Precision::Absent) => lhs.to_inexact(), + (lhs, rhs) => lhs.add(rhs), + } +} + +pub(crate) fn get_col_stats_vec( + null_counts: Vec>, + max_values: Vec>, + min_values: Vec>, +) -> Vec { + izip!(null_counts, max_values, min_values) + .map(|(null_count, max_value, min_value)| ColumnStatistics { + null_count, + max_value, + min_value, + distinct_count: Precision::Absent, + }) + .collect() +} + pub(crate) fn get_col_stats( schema: &Schema, - null_counts: Vec, + null_counts: Vec>, max_values: &mut [Option], min_values: &mut [Option], ) -> Vec { @@ -165,11 +199,57 @@ pub(crate) fn get_col_stats( None => None, }; ColumnStatistics { - null_count: Some(null_counts[i]), - max_value, - min_value, - distinct_count: None, + null_count: null_counts[i].clone(), + max_value: max_value.map(Precision::Exact).unwrap_or(Precision::Absent), + min_value: min_value.map(Precision::Exact).unwrap_or(Precision::Absent), + distinct_count: Precision::Absent, } }) .collect() } + +/// If the given value is numerically greater than the original maximum value, +/// return the new maximum value with appropriate exactness information. +fn set_max_if_greater( + max_nominee: Precision, + max_values: Precision, +) -> Precision { + match (&max_values, &max_nominee) { + (Precision::Exact(val1), Precision::Exact(val2)) if val1 < val2 => max_nominee, + (Precision::Exact(val1), Precision::Inexact(val2)) + | (Precision::Inexact(val1), Precision::Inexact(val2)) + | (Precision::Inexact(val1), Precision::Exact(val2)) + if val1 < val2 => + { + max_nominee.to_inexact() + } + (Precision::Exact(_), Precision::Absent) => max_values.to_inexact(), + (Precision::Absent, Precision::Exact(_)) => max_nominee.to_inexact(), + (Precision::Absent, Precision::Inexact(_)) => max_nominee, + (Precision::Absent, Precision::Absent) => Precision::Absent, + _ => max_values, + } +} + +/// If the given value is numerically lesser than the original minimum value, +/// return the new minimum value with appropriate exactness information. +fn set_min_if_lesser( + min_nominee: Precision, + min_values: Precision, +) -> Precision { + match (&min_values, &min_nominee) { + (Precision::Exact(val1), Precision::Exact(val2)) if val1 > val2 => min_nominee, + (Precision::Exact(val1), Precision::Inexact(val2)) + | (Precision::Inexact(val1), Precision::Inexact(val2)) + | (Precision::Inexact(val1), Precision::Exact(val2)) + if val1 > val2 => + { + min_nominee.to_inexact() + } + (Precision::Exact(_), Precision::Absent) => min_values.to_inexact(), + (Precision::Absent, Precision::Exact(_)) => min_nominee.to_inexact(), + (Precision::Absent, Precision::Inexact(_)) => min_nominee, + (Precision::Absent, Precision::Absent) => Precision::Absent, + _ => min_values, + } +} diff --git a/datafusion/core/src/physical_optimizer/aggregate_statistics.rs b/datafusion/core/src/physical_optimizer/aggregate_statistics.rs index 52fd60aa94bd..43def5d73f73 100644 --- a/datafusion/core/src/physical_optimizer/aggregate_statistics.rs +++ b/datafusion/core/src/physical_optimizer/aggregate_statistics.rs @@ -18,20 +18,18 @@ //! Utilizing exact statistics from sources to avoid scanning data use std::sync::Arc; +use super::optimizer::PhysicalOptimizerRule; use crate::config::ConfigOptions; -use datafusion_common::tree_node::TreeNode; -use datafusion_expr::utils::COUNT_STAR_EXPANSION; - -use crate::physical_plan::aggregates::{AggregateExec, AggregateMode}; +use crate::error::Result; +use crate::physical_plan::aggregates::AggregateExec; use crate::physical_plan::empty::EmptyExec; use crate::physical_plan::projection::ProjectionExec; -use crate::physical_plan::{ - expressions, AggregateExpr, ColumnStatistics, ExecutionPlan, Statistics, -}; +use crate::physical_plan::{expressions, AggregateExpr, ExecutionPlan, Statistics}; use crate::scalar::ScalarValue; -use super::optimizer::PhysicalOptimizerRule; -use crate::error::Result; +use datafusion_common::stats::Precision; +use datafusion_common::tree_node::TreeNode; +use datafusion_expr::utils::COUNT_STAR_EXPANSION; /// Optimizer that uses available statistics for aggregate functions #[derive(Default)] @@ -58,7 +56,7 @@ impl PhysicalOptimizerRule for AggregateStatistics { .as_any() .downcast_ref::() .expect("take_optimizable() ensures that this is a AggregateExec"); - let stats = partial_agg_exec.input().statistics(); + let stats = partial_agg_exec.input().statistics()?; let mut projections = vec![]; for expr in partial_agg_exec.aggr_expr() { if let Some((non_null_rows, name)) = @@ -107,13 +105,12 @@ impl PhysicalOptimizerRule for AggregateStatistics { /// assert if the node passed as argument is a final `AggregateExec` node that can be optimized: /// - its child (with possible intermediate layers) is a partial `AggregateExec` node /// - they both have no grouping expression -/// - the statistics are exact /// If this is the case, return a ref to the partial `AggregateExec`, else `None`. /// We would have preferred to return a casted ref to AggregateExec but the recursion requires /// the `ExecutionPlan.children()` method that returns an owned reference. fn take_optimizable(node: &dyn ExecutionPlan) -> Option> { if let Some(final_agg_exec) = node.as_any().downcast_ref::() { - if final_agg_exec.mode() == &AggregateMode::Final + if !final_agg_exec.mode().is_first_stage() && final_agg_exec.group_expr().is_empty() { let mut child = Arc::clone(final_agg_exec.input()); @@ -121,14 +118,11 @@ fn take_optimizable(node: &dyn ExecutionPlan) -> Option> if let Some(partial_agg_exec) = child.as_any().downcast_ref::() { - if partial_agg_exec.mode() == &AggregateMode::Partial + if partial_agg_exec.mode().is_first_stage() && partial_agg_exec.group_expr().is_empty() && partial_agg_exec.filter_expr().iter().all(|e| e.is_none()) { - let stats = partial_agg_exec.input().statistics(); - if stats.is_exact { - return Some(child); - } + return Some(child); } } if let [ref childrens_child] = child.children().as_slice() { @@ -142,13 +136,13 @@ fn take_optimizable(node: &dyn ExecutionPlan) -> Option> None } -/// If this agg_expr is a count that is defined in the statistics, return it +/// If this agg_expr is a count that is exactly defined in the statistics, return it. fn take_optimizable_table_count( agg_expr: &dyn AggregateExpr, stats: &Statistics, ) -> Option<(ScalarValue, &'static str)> { - if let (Some(num_rows), Some(casted_expr)) = ( - stats.num_rows, + if let (&Precision::Exact(num_rows), Some(casted_expr)) = ( + &stats.num_rows, agg_expr.as_any().downcast_ref::(), ) { // TODO implementing Eq on PhysicalExpr would help a lot here @@ -169,14 +163,14 @@ fn take_optimizable_table_count( None } -/// If this agg_expr is a count that can be derived from the statistics, return it +/// If this agg_expr is a count that can be exactly derived from the statistics, return it. fn take_optimizable_column_count( agg_expr: &dyn AggregateExpr, stats: &Statistics, ) -> Option<(ScalarValue, String)> { - if let (Some(num_rows), Some(col_stats), Some(casted_expr)) = ( - stats.num_rows, - &stats.column_statistics, + let col_stats = &stats.column_statistics; + if let (&Precision::Exact(num_rows), Some(casted_expr)) = ( + &stats.num_rows, agg_expr.as_any().downcast_ref::(), ) { if casted_expr.expressions().len() == 1 { @@ -185,11 +179,8 @@ fn take_optimizable_column_count( .as_any() .downcast_ref::() { - if let ColumnStatistics { - null_count: Some(val), - .. - } = &col_stats[col_expr.index()] - { + let current_val = &col_stats[col_expr.index()].null_count; + if let &Precision::Exact(val) = current_val { return Some(( ScalarValue::Int64(Some((num_rows - val) as i64)), casted_expr.name().to_string(), @@ -201,27 +192,23 @@ fn take_optimizable_column_count( None } -/// If this agg_expr is a min that is defined in the statistics, return it +/// If this agg_expr is a min that is exactly defined in the statistics, return it. fn take_optimizable_min( agg_expr: &dyn AggregateExpr, stats: &Statistics, ) -> Option<(ScalarValue, String)> { - if let (Some(col_stats), Some(casted_expr)) = ( - &stats.column_statistics, - agg_expr.as_any().downcast_ref::(), - ) { + let col_stats = &stats.column_statistics; + if let Some(casted_expr) = agg_expr.as_any().downcast_ref::() { if casted_expr.expressions().len() == 1 { // TODO optimize with exprs other than Column if let Some(col_expr) = casted_expr.expressions()[0] .as_any() .downcast_ref::() { - if let ColumnStatistics { - min_value: Some(val), - .. - } = &col_stats[col_expr.index()] - { - return Some((val.clone(), casted_expr.name().to_string())); + if let Precision::Exact(val) = &col_stats[col_expr.index()].min_value { + if !val.is_null() { + return Some((val.clone(), casted_expr.name().to_string())); + } } } } @@ -229,27 +216,23 @@ fn take_optimizable_min( None } -/// If this agg_expr is a max that is defined in the statistics, return it +/// If this agg_expr is a max that is exactly defined in the statistics, return it. fn take_optimizable_max( agg_expr: &dyn AggregateExpr, stats: &Statistics, ) -> Option<(ScalarValue, String)> { - if let (Some(col_stats), Some(casted_expr)) = ( - &stats.column_statistics, - agg_expr.as_any().downcast_ref::(), - ) { + let col_stats = &stats.column_statistics; + if let Some(casted_expr) = agg_expr.as_any().downcast_ref::() { if casted_expr.expressions().len() == 1 { // TODO optimize with exprs other than Column if let Some(col_expr) = casted_expr.expressions()[0] .as_any() .downcast_ref::() { - if let ColumnStatistics { - max_value: Some(val), - .. - } = &col_stats[col_expr.index()] - { - return Some((val.clone(), casted_expr.name().to_string())); + if let Precision::Exact(val) = &col_stats[col_expr.index()].max_value { + if !val.is_null() { + return Some((val.clone(), casted_expr.name().to_string())); + } } } } @@ -259,16 +242,9 @@ fn take_optimizable_max( #[cfg(test)] mod tests { - use super::*; use std::sync::Arc; - use arrow::array::Int32Array; - use arrow::datatypes::{DataType, Field, Schema}; - use arrow::record_batch::RecordBatch; - use datafusion_common::cast::as_int64_array; - use datafusion_physical_expr::expressions::cast; - use datafusion_physical_expr::PhysicalExpr; - + use super::*; use crate::error::Result; use crate::logical_expr::Operator; use crate::physical_plan::aggregates::{AggregateExec, PhysicalGroupBy}; @@ -279,6 +255,14 @@ mod tests { use crate::physical_plan::memory::MemoryExec; use crate::prelude::SessionContext; + use arrow::array::Int32Array; + use arrow::datatypes::{DataType, Field, Schema}; + use arrow::record_batch::RecordBatch; + use datafusion_common::cast::as_int64_array; + use datafusion_physical_expr::expressions::cast; + use datafusion_physical_expr::PhysicalExpr; + use datafusion_physical_plan::aggregates::AggregateMode; + /// Mock data using a MemoryExec which has an exact count statistic fn mock_data() -> Result> { let schema = Arc::new(Schema::new(vec![ @@ -308,7 +292,8 @@ mod tests { ) -> Result<()> { let session_ctx = SessionContext::new(); let state = session_ctx.state(); - let plan = Arc::new(plan) as _; + let plan: Arc = Arc::new(plan); + let optimized = AggregateStatistics::new() .optimize(Arc::clone(&plan), state.config_options())?; diff --git a/datafusion/core/src/physical_optimizer/combine_partial_final_agg.rs b/datafusion/core/src/physical_optimizer/combine_partial_final_agg.rs index 40b2bcc3e140..838ae613683e 100644 --- a/datafusion/core/src/physical_optimizer/combine_partial_final_agg.rs +++ b/datafusion/core/src/physical_optimizer/combine_partial_final_agg.rs @@ -17,13 +17,15 @@ //! CombinePartialFinalAggregate optimizer rule checks the adjacent Partial and Final AggregateExecs //! and try to combine them if necessary + +use std::sync::Arc; + use crate::error::Result; use crate::physical_optimizer::PhysicalOptimizerRule; use crate::physical_plan::aggregates::{AggregateExec, AggregateMode, PhysicalGroupBy}; use crate::physical_plan::ExecutionPlan; -use datafusion_common::config::ConfigOptions; -use std::sync::Arc; +use datafusion_common::config::ConfigOptions; use datafusion_common::tree_node::{Transformed, TreeNode}; use datafusion_physical_expr::expressions::Column; use datafusion_physical_expr::{AggregateExpr, PhysicalExpr}; @@ -191,10 +193,6 @@ fn discard_column_index(group_expr: Arc) -> Arc { @@ -248,7 +250,7 @@ mod tests { object_store_url: ObjectStoreUrl::parse("test:///").unwrap(), file_schema: schema.clone(), file_groups: vec![vec![PartitionedFile::new("x".to_string(), 100)]], - statistics: Statistics::default(), + statistics: Statistics::new_unknown(schema), projection: None, limit: None, table_partition_cols: vec![], diff --git a/datafusion/core/src/physical_optimizer/enforce_distribution.rs b/datafusion/core/src/physical_optimizer/enforce_distribution.rs index 52525d1fc44d..89036e9f8ccc 100644 --- a/datafusion/core/src/physical_optimizer/enforce_distribution.rs +++ b/datafusion/core/src/physical_optimizer/enforce_distribution.rs @@ -40,13 +40,14 @@ use crate::physical_plan::joins::{ }; use crate::physical_plan::projection::ProjectionExec; use crate::physical_plan::repartition::RepartitionExec; -use crate::physical_plan::sorts::sort::SortOptions; use crate::physical_plan::sorts::sort_preserving_merge::SortPreservingMergeExec; use crate::physical_plan::union::{can_interleave, InterleaveExec, UnionExec}; use crate::physical_plan::windows::WindowAggExec; -use crate::physical_plan::Partitioning; -use crate::physical_plan::{with_new_children_if_necessary, Distribution, ExecutionPlan}; +use crate::physical_plan::{ + with_new_children_if_necessary, Distribution, ExecutionPlan, Partitioning, +}; +use arrow::compute::SortOptions; use datafusion_common::tree_node::{Transformed, TreeNode, VisitRecursion}; use datafusion_expr::logical_plan::JoinType; use datafusion_physical_expr::equivalence::EquivalenceProperties; @@ -58,8 +59,8 @@ use datafusion_physical_expr::{ expr_list_eq_strict_order, PhysicalExpr, PhysicalSortRequirement, }; use datafusion_physical_plan::unbounded_output; - use datafusion_physical_plan::windows::{get_best_fitting_window, BoundedWindowAggExec}; + use itertools::izip; /// The `EnforceDistribution` rule ensures that distribution requirements are @@ -1289,11 +1290,11 @@ fn ensure_distribution( child_idx, )| { // Don't need to apply when the returned row count is not greater than 1: - let stats = child.statistics(); - let repartition_beneficial_stats = if stats.is_exact { - stats - .num_rows - .map(|num_rows| num_rows > batch_size) + let num_rows = child.statistics()?.num_rows; + let repartition_beneficial_stats = if num_rows.is_exact().unwrap_or(false) { + num_rows + .get_value() + .map(|value| value > &batch_size) .unwrap_or(true) } else { true @@ -1808,7 +1809,7 @@ mod tests { unreachable!(); } - fn statistics(&self) -> Statistics { + fn statistics(&self) -> Result { self.input.statistics() } } @@ -1835,7 +1836,7 @@ mod tests { object_store_url: ObjectStoreUrl::parse("test:///").unwrap(), file_schema: schema(), file_groups: vec![vec![PartitionedFile::new("x".to_string(), 100)]], - statistics: Statistics::default(), + statistics: Statistics::new_unknown(&schema()), projection: None, limit: None, table_partition_cols: vec![], @@ -1863,7 +1864,7 @@ mod tests { vec![PartitionedFile::new("x".to_string(), 100)], vec![PartitionedFile::new("y".to_string(), 100)], ], - statistics: Statistics::default(), + statistics: Statistics::new_unknown(&schema()), projection: None, limit: None, table_partition_cols: vec![], @@ -1885,7 +1886,7 @@ mod tests { object_store_url: ObjectStoreUrl::parse("test:///").unwrap(), file_schema: schema(), file_groups: vec![vec![PartitionedFile::new("x".to_string(), 100)]], - statistics: Statistics::default(), + statistics: Statistics::new_unknown(&schema()), projection: None, limit: None, table_partition_cols: vec![], @@ -1916,7 +1917,7 @@ mod tests { vec![PartitionedFile::new("x".to_string(), 100)], vec![PartitionedFile::new("y".to_string(), 100)], ], - statistics: Statistics::default(), + statistics: Statistics::new_unknown(&schema()), projection: None, limit: None, table_partition_cols: vec![], @@ -3953,7 +3954,7 @@ mod tests { "x".to_string(), 100, )]], - statistics: Statistics::default(), + statistics: Statistics::new_unknown(&schema()), projection: None, limit: None, table_partition_cols: vec![], diff --git a/datafusion/core/src/physical_optimizer/join_selection.rs b/datafusion/core/src/physical_optimizer/join_selection.rs index 3d5c93d4b360..876a464257cc 100644 --- a/datafusion/core/src/physical_optimizer/join_selection.rs +++ b/datafusion/core/src/physical_optimizer/join_selection.rs @@ -60,21 +60,29 @@ impl JoinSelection { // TODO: We need some performance test for Right Semi/Right Join swap to Left Semi/Left Join in case that the right side is smaller but not much smaller. // TODO: In PrestoSQL, the optimizer flips join sides only if one side is much smaller than the other by more than SIZE_DIFFERENCE_THRESHOLD times, by default is is 8 times. /// Checks statistics for join swap. -fn should_swap_join_order(left: &dyn ExecutionPlan, right: &dyn ExecutionPlan) -> bool { +fn should_swap_join_order( + left: &dyn ExecutionPlan, + right: &dyn ExecutionPlan, +) -> Result { // Get the left and right table's total bytes // If both the left and right tables contain total_byte_size statistics, // use `total_byte_size` to determine `should_swap_join_order`, else use `num_rows` - let (left_size, right_size) = match ( - left.statistics().total_byte_size, - right.statistics().total_byte_size, + let left_stats = left.statistics()?; + let right_stats = right.statistics()?; + // First compare `total_byte_size` of left and right side, + // if information in this field is insufficient fallback to the `num_rows` + match ( + left_stats.total_byte_size.get_value(), + right_stats.total_byte_size.get_value(), ) { - (Some(l), Some(r)) => (Some(l), Some(r)), - _ => (left.statistics().num_rows, right.statistics().num_rows), - }; - - match (left_size, right_size) { - (Some(l), Some(r)) => l > r, - _ => false, + (Some(l), Some(r)) => Ok(l > r), + _ => match ( + left_stats.num_rows.get_value(), + right_stats.num_rows.get_value(), + ) { + (Some(l), Some(r)) => Ok(l > r), + _ => Ok(false), + }, } } @@ -84,10 +92,13 @@ fn supports_collect_by_size( ) -> bool { // Currently we do not trust the 0 value from stats, due to stats collection might have bug // TODO check the logic in datasource::get_statistics_with_limit() - if let Some(size) = plan.statistics().total_byte_size { - size != 0 && size < collection_size_threshold - } else if let Some(row_count) = plan.statistics().num_rows { - row_count != 0 && row_count < collection_size_threshold + let Ok(stats) = plan.statistics() else { + return false; + }; + if let Some(size) = stats.total_byte_size.get_value() { + *size != 0 && *size < collection_size_threshold + } else if let Some(row_count) = stats.num_rows.get_value() { + *row_count != 0 && *row_count < collection_size_threshold } else { false } @@ -294,7 +305,7 @@ fn try_collect_left( }; match (left_can_collect, right_can_collect) { (true, true) => { - if should_swap_join_order(&**left, &**right) + if should_swap_join_order(&**left, &**right)? && supports_swap(*hash_join.join_type()) { Ok(Some(swap_hash_join(hash_join, PartitionMode::CollectLeft)?)) @@ -333,7 +344,7 @@ fn try_collect_left( fn partitioned_hash_join(hash_join: &HashJoinExec) -> Result> { let left = hash_join.left(); let right = hash_join.right(); - if should_swap_join_order(&**left, &**right) && supports_swap(*hash_join.join_type()) + if should_swap_join_order(&**left, &**right)? && supports_swap(*hash_join.join_type()) { swap_hash_join(hash_join, PartitionMode::Partitioned) } else { @@ -373,7 +384,7 @@ fn statistical_join_selection_subrule( PartitionMode::Partitioned => { let left = hash_join.left(); let right = hash_join.right(); - if should_swap_join_order(&**left, &**right) + if should_swap_join_order(&**left, &**right)? && supports_swap(*hash_join.join_type()) { swap_hash_join(hash_join, PartitionMode::Partitioned).map(Some)? @@ -385,7 +396,7 @@ fn statistical_join_selection_subrule( } else if let Some(cross_join) = plan.as_any().downcast_ref::() { let left = cross_join.left(); let right = cross_join.right(); - if should_swap_join_order(&**left, &**right) { + if should_swap_join_order(&**left, &**right)? { let new_join = CrossJoinExec::new(Arc::clone(right), Arc::clone(left)); // TODO avoid adding ProjectionExec again and again, only adding Final Projection let proj: Arc = Arc::new(ProjectionExec::try_new( @@ -579,6 +590,8 @@ fn apply_subrules( #[cfg(test)] mod tests_statistical { + use std::sync::Arc; + use super::*; use crate::{ physical_plan::{ @@ -587,28 +600,26 @@ mod tests_statistical { test::StatisticsExec, }; - use std::sync::Arc; - use arrow::datatypes::{DataType, Field, Schema}; - use datafusion_common::{JoinType, ScalarValue}; + use datafusion_common::{stats::Precision, JoinType, ScalarValue}; use datafusion_physical_expr::expressions::Column; use datafusion_physical_expr::PhysicalExpr; fn create_big_and_small() -> (Arc, Arc) { let big = Arc::new(StatisticsExec::new( Statistics { - num_rows: Some(10), - total_byte_size: Some(100000), - ..Default::default() + num_rows: Precision::Inexact(10), + total_byte_size: Precision::Inexact(100000), + column_statistics: vec![ColumnStatistics::new_unknown()], }, Schema::new(vec![Field::new("big_col", DataType::Int32, false)]), )); let small = Arc::new(StatisticsExec::new( Statistics { - num_rows: Some(100000), - total_byte_size: Some(10), - ..Default::default() + num_rows: Precision::Inexact(100000), + total_byte_size: Precision::Inexact(10), + column_statistics: vec![ColumnStatistics::new_unknown()], }, Schema::new(vec![Field::new("small_col", DataType::Int32, false)]), )); @@ -624,13 +635,19 @@ mod tests_statistical { min: Option, max: Option, distinct_count: Option, - ) -> Option> { - Some(vec![ColumnStatistics { - distinct_count, - min_value: min.map(|size| ScalarValue::UInt64(Some(size))), - max_value: max.map(|size| ScalarValue::UInt64(Some(size))), + ) -> Vec { + vec![ColumnStatistics { + distinct_count: distinct_count + .map(Precision::Inexact) + .unwrap_or(Precision::Absent), + min_value: min + .map(|size| Precision::Inexact(ScalarValue::UInt64(Some(size)))) + .unwrap_or(Precision::Absent), + max_value: max + .map(|size| Precision::Inexact(ScalarValue::UInt64(Some(size)))) + .unwrap_or(Precision::Absent), ..Default::default() - }]) + }] } /// Returns three plans with statistics of (min, max, distinct_count) @@ -644,39 +661,39 @@ mod tests_statistical { ) { let big = Arc::new(StatisticsExec::new( Statistics { - num_rows: Some(100_000), + num_rows: Precision::Inexact(100_000), column_statistics: create_column_stats( Some(0), Some(50_000), Some(50_000), ), - ..Default::default() + total_byte_size: Precision::Absent, }, Schema::new(vec![Field::new("big_col", DataType::Int32, false)]), )); let medium = Arc::new(StatisticsExec::new( Statistics { - num_rows: Some(10_000), + num_rows: Precision::Inexact(10_000), column_statistics: create_column_stats( Some(1000), Some(5000), Some(1000), ), - ..Default::default() + total_byte_size: Precision::Absent, }, Schema::new(vec![Field::new("medium_col", DataType::Int32, false)]), )); let small = Arc::new(StatisticsExec::new( Statistics { - num_rows: Some(1000), + num_rows: Precision::Inexact(1000), column_statistics: create_column_stats( Some(0), Some(100_000), Some(1000), ), - ..Default::default() + total_byte_size: Precision::Absent, }, Schema::new(vec![Field::new("small_col", DataType::Int32, false)]), )); @@ -725,10 +742,13 @@ mod tests_statistical { .downcast_ref::() .expect("The type of the plan should not be changed"); - assert_eq!(swapped_join.left().statistics().total_byte_size, Some(10)); assert_eq!( - swapped_join.right().statistics().total_byte_size, - Some(100000) + swapped_join.left().statistics().unwrap().total_byte_size, + Precision::Inexact(10) + ); + assert_eq!( + swapped_join.right().statistics().unwrap().total_byte_size, + Precision::Inexact(100000) ); } @@ -774,10 +794,13 @@ mod tests_statistical { .expect("The type of the plan should not be changed"); assert_eq!( - swapped_join.left().statistics().total_byte_size, - Some(100000) + swapped_join.left().statistics().unwrap().total_byte_size, + Precision::Inexact(100000) + ); + assert_eq!( + swapped_join.right().statistics().unwrap().total_byte_size, + Precision::Inexact(10) ); - assert_eq!(swapped_join.right().statistics().total_byte_size, Some(10)); } #[tokio::test] @@ -815,10 +838,13 @@ mod tests_statistical { assert_eq!(swapped_join.schema().fields().len(), 1); - assert_eq!(swapped_join.left().statistics().total_byte_size, Some(10)); assert_eq!( - swapped_join.right().statistics().total_byte_size, - Some(100000) + swapped_join.left().statistics().unwrap().total_byte_size, + Precision::Inexact(10) + ); + assert_eq!( + swapped_join.right().statistics().unwrap().total_byte_size, + Precision::Inexact(100000) ); assert_eq!(original_schema, swapped_join.schema()); @@ -893,9 +919,9 @@ mod tests_statistical { " HashJoinExec: mode=CollectLeft, join_type=Right, on=[(small_col@1, medium_col@0)]", " ProjectionExec: expr=[big_col@1 as big_col, small_col@0 as small_col]", " HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(small_col@0, big_col@0)]", - " StatisticsExec: col_count=1, row_count=Some(1000)", - " StatisticsExec: col_count=1, row_count=Some(100000)", - " StatisticsExec: col_count=1, row_count=Some(10000)", + " StatisticsExec: col_count=1, row_count=Inexact(1000)", + " StatisticsExec: col_count=1, row_count=Inexact(100000)", + " StatisticsExec: col_count=1, row_count=Inexact(10000)", "", ]; assert_optimized!(expected, join); @@ -927,10 +953,13 @@ mod tests_statistical { .downcast_ref::() .expect("The type of the plan should not be changed"); - assert_eq!(swapped_join.left().statistics().total_byte_size, Some(10)); assert_eq!( - swapped_join.right().statistics().total_byte_size, - Some(100000) + swapped_join.left().statistics().unwrap().total_byte_size, + Precision::Inexact(10) + ); + assert_eq!( + swapped_join.right().statistics().unwrap().total_byte_size, + Precision::Inexact(100000) ); } @@ -973,27 +1002,27 @@ mod tests_statistical { async fn test_join_selection_collect_left() { let big = Arc::new(StatisticsExec::new( Statistics { - num_rows: Some(10000000), - total_byte_size: Some(10000000), - ..Default::default() + num_rows: Precision::Inexact(10000000), + total_byte_size: Precision::Inexact(10000000), + column_statistics: vec![ColumnStatistics::new_unknown()], }, Schema::new(vec![Field::new("big_col", DataType::Int32, false)]), )); let small = Arc::new(StatisticsExec::new( Statistics { - num_rows: Some(10), - total_byte_size: Some(10), - ..Default::default() + num_rows: Precision::Inexact(10), + total_byte_size: Precision::Inexact(10), + column_statistics: vec![ColumnStatistics::new_unknown()], }, Schema::new(vec![Field::new("small_col", DataType::Int32, false)]), )); let empty = Arc::new(StatisticsExec::new( Statistics { - num_rows: None, - total_byte_size: None, - ..Default::default() + num_rows: Precision::Absent, + total_byte_size: Precision::Absent, + column_statistics: vec![ColumnStatistics::new_unknown()], }, Schema::new(vec![Field::new("empty_col", DataType::Int32, false)]), )); @@ -1051,27 +1080,27 @@ mod tests_statistical { async fn test_join_selection_partitioned() { let big1 = Arc::new(StatisticsExec::new( Statistics { - num_rows: Some(10000000), - total_byte_size: Some(10000000), - ..Default::default() + num_rows: Precision::Inexact(10000000), + total_byte_size: Precision::Inexact(10000000), + column_statistics: vec![ColumnStatistics::new_unknown()], }, Schema::new(vec![Field::new("big_col1", DataType::Int32, false)]), )); let big2 = Arc::new(StatisticsExec::new( Statistics { - num_rows: Some(20000000), - total_byte_size: Some(20000000), - ..Default::default() + num_rows: Precision::Inexact(20000000), + total_byte_size: Precision::Inexact(20000000), + column_statistics: vec![ColumnStatistics::new_unknown()], }, Schema::new(vec![Field::new("big_col2", DataType::Int32, false)]), )); let empty = Arc::new(StatisticsExec::new( Statistics { - num_rows: None, - total_byte_size: None, - ..Default::default() + num_rows: Precision::Absent, + total_byte_size: Precision::Absent, + column_statistics: vec![ColumnStatistics::new_unknown()], }, Schema::new(vec![Field::new("empty_col", DataType::Int32, false)]), )); @@ -1173,12 +1202,13 @@ mod tests_statistical { #[cfg(test)] mod util_tests { + use std::sync::Arc; + use arrow_schema::{DataType, Field, Schema}; use datafusion_expr::Operator; use datafusion_physical_expr::expressions::{BinaryExpr, Column, NegativeExpr}; use datafusion_physical_expr::intervals::utils::check_support; use datafusion_physical_expr::PhysicalExpr; - use std::sync::Arc; #[test] fn check_expr_supported() { diff --git a/datafusion/core/src/physical_optimizer/output_requirements.rs b/datafusion/core/src/physical_optimizer/output_requirements.rs index 4b687d7f3536..f5eacd5ee60c 100644 --- a/datafusion/core/src/physical_optimizer/output_requirements.rs +++ b/datafusion/core/src/physical_optimizer/output_requirements.rs @@ -179,7 +179,7 @@ impl ExecutionPlan for OutputRequirementExec { unreachable!(); } - fn statistics(&self) -> Statistics { + fn statistics(&self) -> Result { self.input.statistics() } } diff --git a/datafusion/core/src/physical_optimizer/pipeline_checker.rs b/datafusion/core/src/physical_optimizer/pipeline_checker.rs index fbc991017911..43ae7dbfe7b6 100644 --- a/datafusion/core/src/physical_optimizer/pipeline_checker.rs +++ b/datafusion/core/src/physical_optimizer/pipeline_checker.rs @@ -18,17 +18,19 @@ //! The [PipelineChecker] rule ensures that a given plan can accommodate its //! infinite sources, if there are any. It will reject non-runnable query plans //! that use pipeline-breaking operators on infinite input(s). -//! + +use std::sync::Arc; + use crate::config::ConfigOptions; use crate::error::Result; use crate::physical_optimizer::PhysicalOptimizerRule; use crate::physical_plan::joins::SymmetricHashJoinExec; use crate::physical_plan::{with_new_children_if_necessary, ExecutionPlan}; + use datafusion_common::config::OptimizerOptions; use datafusion_common::tree_node::{Transformed, TreeNode, VisitRecursion}; use datafusion_common::{plan_err, DataFusionError}; use datafusion_physical_expr::intervals::utils::{check_support, is_datatype_supported}; -use std::sync::Arc; /// The PipelineChecker rule rejects non-runnable query plans that use /// pipeline-breaking operators on infinite input(s). diff --git a/datafusion/core/src/physical_optimizer/replace_with_order_preserving_variants.rs b/datafusion/core/src/physical_optimizer/replace_with_order_preserving_variants.rs index f4b3608d00c7..fb1a50e18d6f 100644 --- a/datafusion/core/src/physical_optimizer/replace_with_order_preserving_variants.rs +++ b/datafusion/core/src/physical_optimizer/replace_with_order_preserving_variants.rs @@ -903,7 +903,7 @@ mod tests { "file_path".to_string(), 100, )]], - statistics: Statistics::default(), + statistics: Statistics::new_unknown(schema), projection: Some(projection), limit: None, table_partition_cols: vec![], diff --git a/datafusion/core/src/physical_optimizer/test_utils.rs b/datafusion/core/src/physical_optimizer/test_utils.rs index ed7345651457..53401751b67e 100644 --- a/datafusion/core/src/physical_optimizer/test_utils.rs +++ b/datafusion/core/src/physical_optimizer/test_utils.rs @@ -267,7 +267,7 @@ pub fn parquet_exec(schema: &SchemaRef) -> Arc { object_store_url: ObjectStoreUrl::parse("test:///").unwrap(), file_schema: schema.clone(), file_groups: vec![vec![PartitionedFile::new("x".to_string(), 100)]], - statistics: Statistics::default(), + statistics: Statistics::new_unknown(schema), projection: None, limit: None, table_partition_cols: vec![], @@ -291,7 +291,7 @@ pub fn parquet_exec_sorted( object_store_url: ObjectStoreUrl::parse("test:///").unwrap(), file_schema: schema.clone(), file_groups: vec![vec![PartitionedFile::new("x".to_string(), 100)]], - statistics: Statistics::default(), + statistics: Statistics::new_unknown(schema), projection: None, limit: None, table_partition_cols: vec![], diff --git a/datafusion/core/src/physical_planner.rs b/datafusion/core/src/physical_planner.rs index 4055f9f4ebd2..5a1fdcaee509 100644 --- a/datafusion/core/src/physical_planner.rs +++ b/datafusion/core/src/physical_planner.rs @@ -17,6 +17,10 @@ //! Planner for [`LogicalPlan`] to [`ExecutionPlan`] +use std::collections::HashMap; +use std::fmt::Write; +use std::sync::Arc; + use crate::datasource::file_format::arrow::ArrowFormat; use crate::datasource::file_format::avro::AvroFormat; use crate::datasource::file_format::csv::CsvFormat; @@ -27,6 +31,7 @@ use crate::datasource::file_format::FileFormat; use crate::datasource::listing::ListingTableUrl; use crate::datasource::physical_plan::FileSinkConfig; use crate::datasource::source_as_provider; +use crate::error::{DataFusionError, Result}; use crate::execution::context::{ExecutionProps, SessionState}; use crate::logical_expr::utils::generate_sort_key; use crate::logical_expr::{ @@ -37,49 +42,45 @@ use crate::logical_expr::{ CrossJoin, Expr, LogicalPlan, Partitioning as LogicalPartitioning, PlanType, Repartition, Union, UserDefinedLogicalNode, }; -use crate::physical_plan::memory::MemoryExec; -use arrow_array::builder::StringBuilder; -use arrow_array::RecordBatch; -use datafusion_common::display::ToStringifiedPlan; -use datafusion_common::file_options::FileTypeWriterOptions; -use datafusion_common::FileType; -use datafusion_expr::dml::{CopyOptions, CopyTo}; - use crate::logical_expr::{Limit, Values}; use crate::physical_expr::create_physical_expr; use crate::physical_optimizer::optimizer::PhysicalOptimizerRule; use crate::physical_plan::aggregates::{AggregateExec, AggregateMode, PhysicalGroupBy}; use crate::physical_plan::analyze::AnalyzeExec; +use crate::physical_plan::empty::EmptyExec; use crate::physical_plan::explain::ExplainExec; use crate::physical_plan::expressions::{Column, PhysicalSortExpr}; use crate::physical_plan::filter::FilterExec; -use crate::physical_plan::joins::HashJoinExec; -use crate::physical_plan::joins::SortMergeJoinExec; -use crate::physical_plan::joins::{CrossJoinExec, NestedLoopJoinExec}; +use crate::physical_plan::joins::utils as join_utils; +use crate::physical_plan::joins::{ + CrossJoinExec, HashJoinExec, NestedLoopJoinExec, PartitionMode, SortMergeJoinExec, +}; use crate::physical_plan::limit::{GlobalLimitExec, LocalLimitExec}; +use crate::physical_plan::memory::MemoryExec; use crate::physical_plan::projection::ProjectionExec; use crate::physical_plan::repartition::RepartitionExec; use crate::physical_plan::sorts::sort::SortExec; +use crate::physical_plan::union::UnionExec; use crate::physical_plan::unnest::UnnestExec; +use crate::physical_plan::values::ValuesExec; use crate::physical_plan::windows::{ BoundedWindowAggExec, PartitionSearchMode, WindowAggExec, }; use crate::physical_plan::{ - aggregates, empty::EmptyExec, joins::PartitionMode, udaf, union::UnionExec, - values::ValuesExec, windows, -}; -use crate::physical_plan::{joins::utils as join_utils, Partitioning}; -use crate::physical_plan::{AggregateExpr, ExecutionPlan, PhysicalExpr, WindowExpr}; -use crate::{ - error::{DataFusionError, Result}, - physical_plan::displayable, + aggregates, displayable, udaf, windows, AggregateExpr, ExecutionPlan, Partitioning, + PhysicalExpr, WindowExpr, }; + use arrow::compute::SortOptions; use arrow::datatypes::{Schema, SchemaRef}; -use async_trait::async_trait; +use arrow_array::builder::StringBuilder; +use arrow_array::RecordBatch; +use datafusion_common::display::ToStringifiedPlan; +use datafusion_common::file_options::FileTypeWriterOptions; use datafusion_common::{ - exec_err, internal_err, not_impl_err, plan_err, DFSchema, ScalarValue, + exec_err, internal_err, not_impl_err, plan_err, DFSchema, FileType, ScalarValue, }; +use datafusion_expr::dml::{CopyOptions, CopyTo}; use datafusion_expr::expr::{ self, AggregateFunction, AggregateUDF, Alias, Between, BinaryExpr, Cast, GetFieldAccess, GetIndexedField, GroupingSet, InList, Like, ScalarUDF, TryCast, @@ -87,17 +88,17 @@ use datafusion_expr::expr::{ }; use datafusion_expr::expr_rewriter::{unalias, unnormalize_cols}; use datafusion_expr::logical_plan::builder::wrap_projection_for_join_if_necessary; -use datafusion_expr::{DescribeTable, DmlStatement, StringifiedPlan, WriteOp}; -use datafusion_expr::{WindowFrame, WindowFrameBound}; +use datafusion_expr::{ + DescribeTable, DmlStatement, StringifiedPlan, WindowFrame, WindowFrameBound, WriteOp, +}; use datafusion_physical_expr::expressions::Literal; use datafusion_sql::utils::window_expr_common_partition_keys; + +use async_trait::async_trait; use futures::future::BoxFuture; use futures::{FutureExt, StreamExt, TryStreamExt}; use itertools::{multiunzip, Itertools}; use log::{debug, trace}; -use std::collections::HashMap; -use std::fmt::Write; -use std::sync::Arc; fn create_function_physical_name( fun: &str, @@ -2670,7 +2671,7 @@ mod tests { unimplemented!("NoOpExecutionPlan::execute"); } - fn statistics(&self) -> Statistics { + fn statistics(&self) -> Result { unimplemented!("NoOpExecutionPlan::statistics"); } } diff --git a/datafusion/core/src/test/mod.rs b/datafusion/core/src/test/mod.rs index 903542ca3fad..aad5c19044ea 100644 --- a/datafusion/core/src/test/mod.rs +++ b/datafusion/core/src/test/mod.rs @@ -17,6 +17,13 @@ //! Common unit test utility methods +use std::any::Any; +use std::fs::File; +use std::io::prelude::*; +use std::io::{BufReader, BufWriter}; +use std::path::Path; +use std::sync::Arc; + use crate::datasource::file_format::file_compression_type::{ FileCompressionType, FileTypeExt, }; @@ -29,29 +36,23 @@ use crate::logical_expr::LogicalPlan; use crate::physical_plan::ExecutionPlan; use crate::test::object_store::local_unpartitioned_file; use crate::test_util::{aggr_test_schema, arrow_test_data}; -use array::ArrayRef; -use arrow::array::{self, Array, Decimal128Builder, Int32Array}; + +use arrow::array::{self, Array, ArrayRef, Decimal128Builder, Int32Array}; use arrow::datatypes::{DataType, Field, Schema, SchemaRef}; use arrow::record_batch::RecordBatch; +use datafusion_common::{DataFusionError, FileType, Statistics}; +use datafusion_execution::{SendableRecordBatchStream, TaskContext}; +use datafusion_physical_expr::{Partitioning, PhysicalSortExpr}; +use datafusion_physical_plan::{DisplayAs, DisplayFormatType}; + #[cfg(feature = "compression")] use bzip2::write::BzEncoder; #[cfg(feature = "compression")] use bzip2::Compression as BzCompression; -use datafusion_common::FileType; -use datafusion_common::{DataFusionError, Statistics}; -use datafusion_execution::{SendableRecordBatchStream, TaskContext}; -use datafusion_physical_expr::{Partitioning, PhysicalSortExpr}; -use datafusion_physical_plan::{DisplayAs, DisplayFormatType}; #[cfg(feature = "compression")] use flate2::write::GzEncoder; #[cfg(feature = "compression")] use flate2::Compression as GzCompression; -use std::any::Any; -use std::fs::File; -use std::io::prelude::*; -use std::io::{BufReader, BufWriter}; -use std::path::Path; -use std::sync::Arc; #[cfg(feature = "compression")] use xz2::write::XzEncoder; #[cfg(feature = "compression")] @@ -195,9 +196,9 @@ pub fn partitioned_csv_config( ) -> Result { Ok(FileScanConfig { object_store_url: ObjectStoreUrl::local_filesystem(), - file_schema: schema, + file_schema: schema.clone(), file_groups, - statistics: Default::default(), + statistics: Statistics::new_unknown(&schema), projection: None, limit: None, table_partition_cols: vec![], @@ -285,7 +286,7 @@ pub fn csv_exec_sorted( object_store_url: ObjectStoreUrl::parse("test:///").unwrap(), file_schema: schema.clone(), file_groups: vec![vec![PartitionedFile::new("x".to_string(), 100)]], - statistics: Statistics::default(), + statistics: Statistics::new_unknown(schema), projection: None, limit: None, table_partition_cols: vec![], @@ -308,12 +309,8 @@ pub struct StatisticsExec { } impl StatisticsExec { pub fn new(stats: Statistics, schema: Schema) -> Self { - assert!( - stats - .column_statistics - .as_ref() - .map(|cols| cols.len() == schema.fields().len()) - .unwrap_or(true), + assert_eq!( + stats.column_statistics.len(), schema.fields().len(), "if defined, the column statistics vector length should be the number of fields" ); Self { @@ -378,8 +375,8 @@ impl ExecutionPlan for StatisticsExec { unimplemented!("This plan only serves for testing statistics") } - fn statistics(&self) -> Statistics { - self.stats.clone() + fn statistics(&self) -> Result { + Ok(self.stats.clone()) } } diff --git a/datafusion/core/src/test_util/mod.rs b/datafusion/core/src/test_util/mod.rs index bd52c3eedaa4..d826ec8bfbb6 100644 --- a/datafusion/core/src/test_util/mod.rs +++ b/datafusion/core/src/test_util/mod.rs @@ -37,12 +37,14 @@ use crate::physical_plan::{ SendableRecordBatchStream, }; use crate::prelude::{CsvReadOptions, SessionContext}; + use arrow::datatypes::{DataType, Field, Schema, SchemaRef}; use arrow::record_batch::RecordBatch; -use async_trait::async_trait; use datafusion_common::{Statistics, TableReference}; use datafusion_expr::{CreateExternalTable, Expr, TableType}; use datafusion_physical_expr::PhysicalSortExpr; + +use async_trait::async_trait; use futures::Stream; // backwards compatibility @@ -50,8 +52,7 @@ pub use datafusion_common::test_util::{ arrow_test_data, get_data_dir, parquet_test_data, }; -pub use datafusion_common::assert_batches_eq; -pub use datafusion_common::assert_batches_sorted_eq; +pub use datafusion_common::{assert_batches_eq, assert_batches_sorted_eq}; /// Scan an empty data source, mainly used in tests pub fn scan_empty( @@ -238,8 +239,8 @@ impl ExecutionPlan for UnboundedExec { })) } - fn statistics(&self) -> Statistics { - Statistics::default() + fn statistics(&self) -> Result { + Ok(Statistics::new_unknown(&self.schema())) } } diff --git a/datafusion/core/src/test_util/parquet.rs b/datafusion/core/src/test_util/parquet.rs index d3a1f9c1ef7c..0d11526703b4 100644 --- a/datafusion/core/src/test_util/parquet.rs +++ b/datafusion/core/src/test_util/parquet.rs @@ -35,6 +35,9 @@ use crate::physical_plan::filter::FilterExec; use crate::physical_plan::metrics::MetricsSet; use crate::physical_plan::ExecutionPlan; use crate::prelude::{Expr, SessionConfig}; + +use datafusion_common::Statistics; + use object_store::path::Path; use object_store::ObjectMeta; use parquet::arrow::ArrowWriter; @@ -147,7 +150,7 @@ impl TestParquetFile { range: None, extensions: None, }]], - statistics: Default::default(), + statistics: Statistics::new_unknown(&self.schema), projection: None, limit: None, table_partition_cols: vec![], diff --git a/datafusion/core/tests/custom_sources.rs b/datafusion/core/tests/custom_sources.rs index 377c2ab2ab24..daf1ef41a297 100644 --- a/datafusion/core/tests/custom_sources.rs +++ b/datafusion/core/tests/custom_sources.rs @@ -15,10 +15,17 @@ // specific language governing permissions and limitations // under the License. +use std::any::Any; +use std::pin::Pin; +use std::sync::Arc; +use std::task::{Context, Poll}; + use arrow::array::{Int32Array, Int64Array}; use arrow::compute::kernels::aggregate; use arrow::datatypes::{DataType, Field, Int32Type, Schema, SchemaRef}; use arrow::record_batch::RecordBatch; +use datafusion::datasource::{TableProvider, TableType}; +use datafusion::error::Result; use datafusion::execution::context::{SessionContext, SessionState, TaskContext}; use datafusion::logical_expr::{ col, Expr, LogicalPlan, LogicalPlanBuilder, TableScan, UNNAMED_TABLE, @@ -26,29 +33,20 @@ use datafusion::logical_expr::{ use datafusion::physical_plan::empty::EmptyExec; use datafusion::physical_plan::expressions::PhysicalSortExpr; use datafusion::physical_plan::{ - ColumnStatistics, DisplayAs, ExecutionPlan, Partitioning, RecordBatchStream, - SendableRecordBatchStream, Statistics, + collect, ColumnStatistics, DisplayAs, DisplayFormatType, ExecutionPlan, Partitioning, + RecordBatchStream, SendableRecordBatchStream, Statistics, }; use datafusion::scalar::ScalarValue; -use datafusion::{ - datasource::{TableProvider, TableType}, - physical_plan::collect, -}; -use datafusion::{error::Result, physical_plan::DisplayFormatType}; - use datafusion_common::cast::as_primitive_array; use datafusion_common::project_schema; +use datafusion_common::stats::Precision; + +use async_trait::async_trait; use futures::stream::Stream; -use std::any::Any; -use std::pin::Pin; -use std::sync::Arc; -use std::task::{Context, Poll}; /// Also run all tests that are found in the `custom_sources_cases` directory mod custom_sources_cases; -use async_trait::async_trait; - //--- Custom source dataframe tests ---// struct CustomTableProvider; @@ -153,30 +151,28 @@ impl ExecutionPlan for CustomExecutionPlan { Ok(Box::pin(TestCustomRecordBatchStream { nb_batch: 1 })) } - fn statistics(&self) -> Statistics { + fn statistics(&self) -> Result { let batch = TEST_CUSTOM_RECORD_BATCH!().unwrap(); - Statistics { - is_exact: true, - num_rows: Some(batch.num_rows()), - total_byte_size: None, - column_statistics: Some( - self.projection - .clone() - .unwrap_or_else(|| (0..batch.columns().len()).collect()) - .iter() - .map(|i| ColumnStatistics { - null_count: Some(batch.column(*i).null_count()), - min_value: Some(ScalarValue::Int32(aggregate::min( - as_primitive_array::(batch.column(*i)).unwrap(), - ))), - max_value: Some(ScalarValue::Int32(aggregate::max( - as_primitive_array::(batch.column(*i)).unwrap(), - ))), - ..Default::default() - }) - .collect(), - ), - } + Ok(Statistics { + num_rows: Precision::Exact(batch.num_rows()), + total_byte_size: Precision::Absent, + column_statistics: self + .projection + .clone() + .unwrap_or_else(|| (0..batch.columns().len()).collect()) + .iter() + .map(|i| ColumnStatistics { + null_count: Precision::Exact(batch.column(*i).null_count()), + min_value: Precision::Exact(ScalarValue::Int32(aggregate::min( + as_primitive_array::(batch.column(*i)).unwrap(), + ))), + max_value: Precision::Exact(ScalarValue::Int32(aggregate::max( + as_primitive_array::(batch.column(*i)).unwrap(), + ))), + ..Default::default() + }) + .collect(), + }) } } diff --git a/datafusion/core/tests/custom_sources_cases/provider_filter_pushdown.rs b/datafusion/core/tests/custom_sources_cases/provider_filter_pushdown.rs index 73085937cbca..4679ca6d07df 100644 --- a/datafusion/core/tests/custom_sources_cases/provider_filter_pushdown.rs +++ b/datafusion/core/tests/custom_sources_cases/provider_filter_pushdown.rs @@ -119,10 +119,10 @@ impl ExecutionPlan for CustomPlan { ))) } - fn statistics(&self) -> Statistics { + fn statistics(&self) -> Result { // here we could provide more accurate statistics // but we want to test the filter pushdown not the CBOs - Statistics::default() + Ok(Statistics::new_unknown(&self.schema())) } } diff --git a/datafusion/core/tests/custom_sources_cases/statistics.rs b/datafusion/core/tests/custom_sources_cases/statistics.rs index 43e6c8851ec4..f0985f554654 100644 --- a/datafusion/core/tests/custom_sources_cases/statistics.rs +++ b/datafusion/core/tests/custom_sources_cases/statistics.rs @@ -34,7 +34,7 @@ use datafusion::{ use async_trait::async_trait; use datafusion::execution::context::{SessionState, TaskContext}; -use datafusion_common::project_schema; +use datafusion_common::{project_schema, stats::Precision}; /// This is a testing structure for statistics /// It will act both as a table provider and execution plan @@ -46,13 +46,10 @@ struct StatisticsValidation { impl StatisticsValidation { fn new(stats: Statistics, schema: SchemaRef) -> Self { - assert!( - stats - .column_statistics - .as_ref() - .map(|cols| cols.len() == schema.fields().len()) - .unwrap_or(true), - "if defined, the column statistics vector length should be the number of fields" + assert_eq!( + stats.column_statistics.len(), + schema.fields().len(), + "the column statistics vector length should be the number of fields" ); Self { stats, schema } } @@ -94,17 +91,16 @@ impl TableProvider for StatisticsValidation { let current_stat = self.stats.clone(); - let proj_col_stats = current_stat - .column_statistics - .map(|col_stat| projection.iter().map(|i| col_stat[*i].clone()).collect()); - + let proj_col_stats = projection + .iter() + .map(|i| current_stat.column_statistics[*i].clone()) + .collect(); Ok(Arc::new(Self::new( Statistics { - is_exact: current_stat.is_exact, num_rows: current_stat.num_rows, column_statistics: proj_col_stats, // TODO stats: knowing the type of the new columns we can guess the output size - total_byte_size: None, + total_byte_size: Precision::Absent, }, projected_schema, ))) @@ -166,8 +162,8 @@ impl ExecutionPlan for StatisticsValidation { unimplemented!("This plan only serves for testing statistics") } - fn statistics(&self) -> Statistics { - self.stats.clone() + fn statistics(&self) -> Result { + Ok(self.stats.clone()) } } @@ -182,23 +178,22 @@ fn init_ctx(stats: Statistics, schema: Schema) -> Result { fn fully_defined() -> (Statistics, Schema) { ( Statistics { - num_rows: Some(13), - is_exact: true, - total_byte_size: None, // ignore byte size for now - column_statistics: Some(vec![ + num_rows: Precision::Exact(13), + total_byte_size: Precision::Absent, // ignore byte size for now + column_statistics: vec![ ColumnStatistics { - distinct_count: Some(2), - max_value: Some(ScalarValue::Int32(Some(1023))), - min_value: Some(ScalarValue::Int32(Some(-24))), - null_count: Some(0), + distinct_count: Precision::Exact(2), + max_value: Precision::Exact(ScalarValue::Int32(Some(1023))), + min_value: Precision::Exact(ScalarValue::Int32(Some(-24))), + null_count: Precision::Exact(0), }, ColumnStatistics { - distinct_count: Some(13), - max_value: Some(ScalarValue::Int64(Some(5486))), - min_value: Some(ScalarValue::Int64(Some(-6783))), - null_count: Some(5), + distinct_count: Precision::Exact(13), + max_value: Precision::Exact(ScalarValue::Int64(Some(5486))), + min_value: Precision::Exact(ScalarValue::Int64(Some(-6783))), + null_count: Precision::Exact(5), }, - ]), + ], }, Schema::new(vec![ Field::new("c1", DataType::Int32, false), @@ -216,7 +211,7 @@ async fn sql_basic() -> Result<()> { let physical_plan = df.create_physical_plan().await.unwrap(); // the statistics should be those of the source - assert_eq!(stats, physical_plan.statistics()); + assert_eq!(stats, physical_plan.statistics()?); Ok(()) } @@ -232,10 +227,8 @@ async fn sql_filter() -> Result<()> { .unwrap(); let physical_plan = df.create_physical_plan().await.unwrap(); - - let stats = physical_plan.statistics(); - assert!(!stats.is_exact); - assert_eq!(stats.num_rows, Some(1)); + let stats = physical_plan.statistics()?; + assert_eq!(stats.num_rows, Precision::Inexact(1)); Ok(()) } @@ -243,6 +236,7 @@ async fn sql_filter() -> Result<()> { #[tokio::test] async fn sql_limit() -> Result<()> { let (stats, schema) = fully_defined(); + let col_stats = Statistics::unknown_column(&schema); let ctx = init_ctx(stats.clone(), schema)?; let df = ctx.sql("SELECT * FROM stats_table LIMIT 5").await.unwrap(); @@ -251,11 +245,11 @@ async fn sql_limit() -> Result<()> { // we loose all statistics except the for number of rows which becomes the limit assert_eq!( Statistics { - num_rows: Some(5), - is_exact: true, - ..Default::default() + num_rows: Precision::Exact(5), + column_statistics: col_stats, + total_byte_size: Precision::Absent }, - physical_plan.statistics() + physical_plan.statistics()? ); let df = ctx @@ -264,7 +258,7 @@ async fn sql_limit() -> Result<()> { .unwrap(); let physical_plan = df.create_physical_plan().await.unwrap(); // when the limit is larger than the original number of lines, statistics remain unchanged - assert_eq!(stats, physical_plan.statistics()); + assert_eq!(stats, physical_plan.statistics()?); Ok(()) } @@ -281,13 +275,12 @@ async fn sql_window() -> Result<()> { let physical_plan = df.create_physical_plan().await.unwrap(); - let result = physical_plan.statistics(); + let result = physical_plan.statistics()?; assert_eq!(stats.num_rows, result.num_rows); - assert!(result.column_statistics.is_some()); - let col_stats = result.column_statistics.unwrap(); + let col_stats = result.column_statistics; assert_eq!(2, col_stats.len()); - assert_eq!(stats.column_statistics.unwrap()[1], col_stats[0]); + assert_eq!(stats.column_statistics[1], col_stats[0]); Ok(()) } diff --git a/datafusion/core/tests/parquet/custom_reader.rs b/datafusion/core/tests/parquet/custom_reader.rs index 75ff56a26508..37481b936d24 100644 --- a/datafusion/core/tests/parquet/custom_reader.rs +++ b/datafusion/core/tests/parquet/custom_reader.rs @@ -15,6 +15,11 @@ // specific language governing permissions and limitations // under the License. +use std::io::Cursor; +use std::ops::Range; +use std::sync::Arc; +use std::time::SystemTime; + use arrow::array::{ArrayRef, Int64Array, Int8Array, StringArray}; use arrow::datatypes::{Field, Schema, SchemaBuilder}; use arrow::record_batch::RecordBatch; @@ -30,6 +35,7 @@ use datafusion::physical_plan::metrics::ExecutionPlanMetricsSet; use datafusion::physical_plan::{collect, Statistics}; use datafusion::prelude::SessionContext; use datafusion_common::Result; + use futures::future::BoxFuture; use futures::{FutureExt, TryFutureExt}; use object_store::memory::InMemory; @@ -39,10 +45,6 @@ use parquet::arrow::async_reader::AsyncFileReader; use parquet::arrow::ArrowWriter; use parquet::errors::ParquetError; use parquet::file::metadata::ParquetMetaData; -use std::io::Cursor; -use std::ops::Range; -use std::sync::Arc; -use std::time::SystemTime; const EXPECTED_USER_DEFINED_METADATA: &str = "some-user-defined-metadata"; @@ -77,8 +79,8 @@ async fn route_data_access_ops_to_parquet_file_reader_factory() { // just any url that doesn't point to in memory object store object_store_url: ObjectStoreUrl::local_filesystem(), file_groups: vec![file_groups], + statistics: Statistics::new_unknown(&file_schema), file_schema, - statistics: Statistics::default(), projection: None, limit: None, table_partition_cols: vec![], diff --git a/datafusion/core/tests/parquet/file_statistics.rs b/datafusion/core/tests/parquet/file_statistics.rs index ac4a91720ed9..1ea154303d69 100644 --- a/datafusion/core/tests/parquet/file_statistics.rs +++ b/datafusion/core/tests/parquet/file_statistics.rs @@ -15,6 +15,9 @@ // specific language governing permissions and limitations // under the License. +use std::fs; +use std::sync::Arc; + use datafusion::datasource::file_format::parquet::ParquetFormat; use datafusion::datasource::listing::{ ListingOptions, ListingTable, ListingTableConfig, ListingTableUrl, @@ -23,6 +26,7 @@ use datafusion::datasource::physical_plan::ParquetExec; use datafusion::datasource::TableProvider; use datafusion::execution::context::SessionState; use datafusion::prelude::SessionContext; +use datafusion_common::stats::Precision; use datafusion_execution::cache::cache_manager::CacheManagerConfig; use datafusion_execution::cache::cache_unit; use datafusion_execution::cache::cache_unit::{ @@ -30,8 +34,7 @@ use datafusion_execution::cache::cache_unit::{ }; use datafusion_execution::config::SessionConfig; use datafusion_execution::runtime_env::{RuntimeConfig, RuntimeEnv}; -use std::fs; -use std::sync::Arc; + use tempfile::tempdir; #[tokio::test] @@ -54,24 +57,33 @@ async fn load_table_stats_with_session_level_cache() { assert_eq!(get_static_cache_size(&state1), 0); let exec1 = table1.scan(&state1, None, &[], None).await.unwrap(); - assert_eq!(exec1.statistics().num_rows, Some(8)); - assert_eq!(exec1.statistics().total_byte_size, Some(671)); + assert_eq!(exec1.statistics().unwrap().num_rows, Precision::Exact(8)); + assert_eq!( + exec1.statistics().unwrap().total_byte_size, + Precision::Exact(671) + ); assert_eq!(get_static_cache_size(&state1), 1); //Session 2 first time list files //check session 1 cache result not show in session 2 assert_eq!(get_static_cache_size(&state2), 0); let exec2 = table2.scan(&state2, None, &[], None).await.unwrap(); - assert_eq!(exec2.statistics().num_rows, Some(8)); - assert_eq!(exec2.statistics().total_byte_size, Some(671)); + assert_eq!(exec2.statistics().unwrap().num_rows, Precision::Exact(8)); + assert_eq!( + exec2.statistics().unwrap().total_byte_size, + Precision::Exact(671) + ); assert_eq!(get_static_cache_size(&state2), 1); //Session 1 second time list files //check session 1 cache result not show in session 2 assert_eq!(get_static_cache_size(&state1), 1); let exec3 = table1.scan(&state1, None, &[], None).await.unwrap(); - assert_eq!(exec3.statistics().num_rows, Some(8)); - assert_eq!(exec3.statistics().total_byte_size, Some(671)); + assert_eq!(exec3.statistics().unwrap().num_rows, Precision::Exact(8)); + assert_eq!( + exec3.statistics().unwrap().total_byte_size, + Precision::Exact(671) + ); // List same file no increase assert_eq!(get_static_cache_size(&state1), 1); } diff --git a/datafusion/core/tests/parquet/page_pruning.rs b/datafusion/core/tests/parquet/page_pruning.rs index 4337259c1e62..b77643c35e84 100644 --- a/datafusion/core/tests/parquet/page_pruning.rs +++ b/datafusion/core/tests/parquet/page_pruning.rs @@ -17,6 +17,7 @@ use crate::parquet::Unit::Page; use crate::parquet::{ContextWithParquet, Scenario}; + use datafusion::datasource::file_format::parquet::ParquetFormat; use datafusion::datasource::file_format::FileFormat; use datafusion::datasource::listing::PartitionedFile; @@ -30,6 +31,7 @@ use datafusion_common::{ScalarValue, Statistics, ToDFSchema}; use datafusion_expr::{col, lit, Expr}; use datafusion_physical_expr::create_physical_expr; use datafusion_physical_expr::execution_props::ExecutionProps; + use futures::StreamExt; use object_store::path::Path; use object_store::ObjectMeta; @@ -71,8 +73,8 @@ async fn get_parquet_exec(state: &SessionState, filter: Expr) -> ParquetExec { FileScanConfig { object_store_url, file_groups: vec![vec![partitioned_file]], - file_schema: schema, - statistics: Statistics::default(), + file_schema: schema.clone(), + statistics: Statistics::new_unknown(&schema), // file has 10 cols so index 12 should be month projection: None, limit: None, @@ -240,10 +242,11 @@ async fn test_prune( expected_row_pages_pruned: Option, expected_results: usize, ) { - let output = ContextWithParquet::new(case_data_type, Page) - .await - .query(sql) - .await; + let output: crate::parquet::TestOutput = + ContextWithParquet::new(case_data_type, Page) + .await + .query(sql) + .await; println!("{}", output.description()); assert_eq!(output.predicate_evaluation_errors(), expected_errors); diff --git a/datafusion/core/tests/parquet/schema_coercion.rs b/datafusion/core/tests/parquet/schema_coercion.rs index f7dace993091..b3134d470b56 100644 --- a/datafusion/core/tests/parquet/schema_coercion.rs +++ b/datafusion/core/tests/parquet/schema_coercion.rs @@ -15,6 +15,8 @@ // specific language governing permissions and limitations // under the License. +use std::sync::Arc; + use arrow::datatypes::{Field, Schema}; use arrow::record_batch::RecordBatch; use arrow_array::types::Int32Type; @@ -24,14 +26,13 @@ use datafusion::assert_batches_sorted_eq; use datafusion::datasource::physical_plan::{FileScanConfig, ParquetExec}; use datafusion::physical_plan::collect; use datafusion::prelude::SessionContext; -use datafusion_common::Result; -use datafusion_common::Statistics; +use datafusion_common::{Result, Statistics}; use datafusion_execution::object_store::ObjectStoreUrl; + use object_store::path::Path; use object_store::ObjectMeta; use parquet::arrow::ArrowWriter; use parquet::file::properties::WriterProperties; -use std::sync::Arc; use tempfile::NamedTempFile; /// Test for reading data from multiple parquet files with different schemas and coercing them into a single schema. @@ -62,8 +63,8 @@ async fn multi_parquet_coercion() { FileScanConfig { object_store_url: ObjectStoreUrl::local_filesystem(), file_groups: vec![file_groups], + statistics: Statistics::new_unknown(&file_schema), file_schema, - statistics: Statistics::default(), projection: None, limit: None, table_partition_cols: vec![], @@ -126,8 +127,8 @@ async fn multi_parquet_coercion_projection() { FileScanConfig { object_store_url: ObjectStoreUrl::local_filesystem(), file_groups: vec![file_groups], + statistics: Statistics::new_unknown(&file_schema), file_schema, - statistics: Statistics::default(), projection: Some(vec![1, 0, 2]), limit: None, table_partition_cols: vec![], diff --git a/datafusion/core/tests/path_partition.rs b/datafusion/core/tests/path_partition.rs index 1c08733e3df6..27d146de798d 100644 --- a/datafusion/core/tests/path_partition.rs +++ b/datafusion/core/tests/path_partition.rs @@ -17,16 +17,13 @@ //! Test queries on partitioned datasets -use arrow::datatypes::DataType; use std::collections::BTreeSet; use std::fs::File; use std::io::{Read, Seek, SeekFrom}; use std::ops::Range; use std::sync::Arc; -use async_trait::async_trait; -use bytes::Bytes; -use chrono::{TimeZone, Utc}; +use arrow::datatypes::DataType; use datafusion::datasource::listing::ListingTableUrl; use datafusion::{ assert_batches_sorted_eq, @@ -39,7 +36,12 @@ use datafusion::{ prelude::SessionContext, test_util::{self, arrow_test_data, parquet_test_data}, }; +use datafusion_common::stats::Precision; use datafusion_common::ScalarValue; + +use async_trait::async_trait; +use bytes::Bytes; +use chrono::{TimeZone, Utc}; use futures::stream; use futures::stream::BoxStream; use object_store::{ @@ -458,34 +460,30 @@ async fn parquet_statistics() -> Result<()> { //// NO PROJECTION //// let dataframe = ctx.sql("SELECT * FROM t").await?; let physical_plan = dataframe.create_physical_plan().await?; - assert_eq!(physical_plan.schema().fields().len(), 4); + let schema = physical_plan.schema(); + assert_eq!(schema.fields().len(), 4); - let stat_cols = physical_plan - .statistics() - .column_statistics - .expect("col stats should be defined"); + let stat_cols = physical_plan.statistics()?.column_statistics; assert_eq!(stat_cols.len(), 4); // stats for the first col are read from the parquet file - assert_eq!(stat_cols[0].null_count, Some(3)); + assert_eq!(stat_cols[0].null_count, Precision::Exact(3)); // TODO assert partition column (1,2,3) stats once implemented (#1186) - assert_eq!(stat_cols[1], ColumnStatistics::default()); - assert_eq!(stat_cols[2], ColumnStatistics::default()); - assert_eq!(stat_cols[3], ColumnStatistics::default()); + assert_eq!(stat_cols[1], ColumnStatistics::new_unknown(),); + assert_eq!(stat_cols[2], ColumnStatistics::new_unknown(),); + assert_eq!(stat_cols[3], ColumnStatistics::new_unknown(),); //// WITH PROJECTION //// let dataframe = ctx.sql("SELECT mycol, day FROM t WHERE day='28'").await?; let physical_plan = dataframe.create_physical_plan().await?; - assert_eq!(physical_plan.schema().fields().len(), 2); + let schema = physical_plan.schema(); + assert_eq!(schema.fields().len(), 2); - let stat_cols = physical_plan - .statistics() - .column_statistics - .expect("col stats should be defined"); + let stat_cols = physical_plan.statistics()?.column_statistics; assert_eq!(stat_cols.len(), 2); // stats for the first col are read from the parquet file - assert_eq!(stat_cols[0].null_count, Some(1)); + assert_eq!(stat_cols[0].null_count, Precision::Exact(1)); // TODO assert partition column stats once implemented (#1186) - assert_eq!(stat_cols[1], ColumnStatistics::default()); + assert_eq!(stat_cols[1], ColumnStatistics::new_unknown(),); Ok(()) } diff --git a/datafusion/core/tests/sql/explain_analyze.rs b/datafusion/core/tests/sql/explain_analyze.rs index de7ed19d3f34..c328f46be74a 100644 --- a/datafusion/core/tests/sql/explain_analyze.rs +++ b/datafusion/core/tests/sql/explain_analyze.rs @@ -16,6 +16,7 @@ // under the License. use super::*; + use datafusion::config::ConfigOptions; use datafusion::physical_plan::display::DisplayableExecutionPlan; use datafusion::physical_plan::metrics::Timestamp; @@ -826,5 +827,5 @@ async fn csv_explain_analyze_with_statistics() { .to_string(); // should contain scan statistics - assert_contains!(&formatted, ", statistics=[]"); + assert_contains!(&formatted, ", statistics=[Rows=Absent, Bytes=Absent]"); } diff --git a/datafusion/core/tests/user_defined/user_defined_plan.rs b/datafusion/core/tests/user_defined/user_defined_plan.rs index d8c4bea1744f..d4a8842c0a7a 100644 --- a/datafusion/core/tests/user_defined/user_defined_plan.rs +++ b/datafusion/core/tests/user_defined/user_defined_plan.rs @@ -58,7 +58,9 @@ //! N elements, reducing the total amount of required buffer memory. //! -use futures::{Stream, StreamExt}; +use std::fmt::Debug; +use std::task::{Context, Poll}; +use std::{any::Any, collections::BTreeMap, fmt, sync::Arc}; use arrow::{ array::{Int64Array, StringArray}, @@ -68,8 +70,7 @@ use arrow::{ }; use datafusion::{ common::cast::{as_int64_array, as_string_array}, - common::internal_err, - common::DFSchemaRef, + common::{internal_err, DFSchemaRef}, error::{DataFusionError, Result}, execution::{ context::{QueryPlanner, SessionState, TaskContext}, @@ -89,11 +90,8 @@ use datafusion::{ prelude::{SessionConfig, SessionContext}, }; -use fmt::Debug; -use std::task::{Context, Poll}; -use std::{any::Any, collections::BTreeMap, fmt, sync::Arc}; - use async_trait::async_trait; +use futures::{Stream, StreamExt}; /// Execute the specified sql and return the resulting record batches /// pretty printed as a String. @@ -490,10 +488,10 @@ impl ExecutionPlan for TopKExec { })) } - fn statistics(&self) -> Statistics { + fn statistics(&self) -> Result { // to improve the optimizability of this plan // better statistics inference could be provided - Statistics::default() + Ok(Statistics::new_unknown(&self.schema())) } } diff --git a/datafusion/execution/src/cache/cache_unit.rs b/datafusion/execution/src/cache/cache_unit.rs index c432a67587fe..4a21dc02bd13 100644 --- a/datafusion/execution/src/cache/cache_unit.rs +++ b/datafusion/execution/src/cache/cache_unit.rs @@ -15,12 +15,15 @@ // specific language governing permissions and limitations // under the License. +use std::sync::Arc; + use crate::cache::CacheAccessor; -use dashmap::DashMap; + use datafusion_common::Statistics; + +use dashmap::DashMap; use object_store::path::Path; use object_store::ObjectMeta; -use std::sync::Arc; /// Collected statistics for files /// Cache is invalided when file size or last modification has changed @@ -158,6 +161,7 @@ impl CacheAccessor>> for DefaultListFilesCache { mod tests { use crate::cache::cache_unit::{DefaultFileStatisticsCache, DefaultListFilesCache}; use crate::cache::CacheAccessor; + use arrow::datatypes::{DataType, Field, Schema, TimeUnit}; use chrono::DateTime; use datafusion_common::Statistics; use object_store::path::Path; @@ -173,11 +177,19 @@ mod tests { size: 1024, e_tag: None, }; - let cache = DefaultFileStatisticsCache::default(); assert!(cache.get_with_extra(&meta.location, &meta).is_none()); - cache.put_with_extra(&meta.location, Statistics::default().into(), &meta); + cache.put_with_extra( + &meta.location, + Statistics::new_unknown(&Schema::new(vec![Field::new( + "test_column", + DataType::Timestamp(TimeUnit::Second, None), + false, + )])) + .into(), + &meta, + ); assert!(cache.get_with_extra(&meta.location, &meta).is_some()); // file size changed diff --git a/datafusion/physical-expr/src/analysis.rs b/datafusion/physical-expr/src/analysis.rs index 990c643c6b08..93c24014fd3e 100644 --- a/datafusion/physical-expr/src/analysis.rs +++ b/datafusion/physical-expr/src/analysis.rs @@ -17,6 +17,9 @@ //! Interval and selectivity in [`AnalysisContext`] +use std::fmt::Debug; +use std::sync::Arc; + use crate::expressions::Column; use crate::intervals::cp_solver::PropagationResult; use crate::intervals::{cardinality_ratio, ExprIntervalGraph, Interval, IntervalBound}; @@ -24,20 +27,18 @@ use crate::utils::collect_columns; use crate::PhysicalExpr; use arrow::datatypes::Schema; +use datafusion_common::stats::Precision; use datafusion_common::{ internal_err, ColumnStatistics, DataFusionError, Result, ScalarValue, }; -use std::fmt::Debug; -use std::sync::Arc; - /// The shared context used during the analysis of an expression. Includes /// the boundaries for all known columns. #[derive(Clone, Debug, PartialEq)] pub struct AnalysisContext { // A list of known column boundaries, ordered by the index // of the column in the current schema. - pub boundaries: Option>, + pub boundaries: Vec, /// The estimated percentage of rows that this expression would select, if /// it were to be used as a boolean predicate on a filter. The value will be /// between 0.0 (selects nothing) and 1.0 (selects everything). @@ -47,7 +48,7 @@ pub struct AnalysisContext { impl AnalysisContext { pub fn new(boundaries: Vec) -> Self { Self { - boundaries: Some(boundaries), + boundaries, selectivity: None, } } @@ -58,19 +59,16 @@ impl AnalysisContext { } /// Create a new analysis context from column statistics. - pub fn from_statistics( + pub fn try_from_statistics( input_schema: &Schema, statistics: &[ColumnStatistics], - ) -> Self { - let mut column_boundaries = vec![]; - for (idx, stats) in statistics.iter().enumerate() { - column_boundaries.push(ExprBoundaries::from_column( - stats, - input_schema.fields()[idx].name().clone(), - idx, - )); - } - Self::new(column_boundaries) + ) -> Result { + statistics + .iter() + .enumerate() + .map(|(idx, stats)| ExprBoundaries::try_from_column(input_schema, stats, idx)) + .collect::>>() + .map(Self::new) } } @@ -82,24 +80,40 @@ pub struct ExprBoundaries { /// Minimum and maximum values this expression can have. pub interval: Interval, /// Maximum number of distinct values this expression can produce, if known. - pub distinct_count: Option, + pub distinct_count: Precision, } impl ExprBoundaries { /// Create a new `ExprBoundaries` object from column level statistics. - pub fn from_column(stats: &ColumnStatistics, col: String, index: usize) -> Self { - Self { - column: Column::new(&col, index), - interval: Interval::new( - IntervalBound::new_closed( - stats.min_value.clone().unwrap_or(ScalarValue::Null), - ), - IntervalBound::new_closed( - stats.max_value.clone().unwrap_or(ScalarValue::Null), - ), + pub fn try_from_column( + schema: &Schema, + col_stats: &ColumnStatistics, + col_index: usize, + ) -> Result { + let field = &schema.fields()[col_index]; + let empty_field = ScalarValue::try_from(field.data_type())?; + let interval = Interval::new( + IntervalBound::new_closed( + col_stats + .min_value + .get_value() + .cloned() + .unwrap_or(empty_field.clone()), ), - distinct_count: stats.distinct_count, - } + IntervalBound::new_closed( + col_stats + .max_value + .get_value() + .cloned() + .unwrap_or(empty_field), + ), + ); + let column = Column::new(field.name(), col_index); + Ok(ExprBoundaries { + column, + interval, + distinct_count: col_stats.distinct_count.clone(), + }) } } @@ -122,9 +136,7 @@ pub fn analyze( expr: &Arc, context: AnalysisContext, ) -> Result { - let target_boundaries = context.boundaries.ok_or_else(|| { - DataFusionError::Internal("No column exists at the input to filter".to_string()) - })?; + let target_boundaries = context.boundaries; let mut graph = ExprIntervalGraph::try_new(expr.clone())?; @@ -148,18 +160,22 @@ pub fn analyze( }) }) .collect(); - - match graph.update_ranges(&mut target_indices_and_boundaries)? { - PropagationResult::Success => { - shrink_boundaries(expr, graph, target_boundaries, target_expr_and_indices) - } - PropagationResult::Infeasible => { - Ok(AnalysisContext::new(target_boundaries).with_selectivity(0.0)) - } - PropagationResult::CannotPropagate => { - Ok(AnalysisContext::new(target_boundaries).with_selectivity(1.0)) - } - } + Ok( + match graph.update_ranges(&mut target_indices_and_boundaries)? { + PropagationResult::Success => shrink_boundaries( + expr, + graph, + target_boundaries, + target_expr_and_indices, + )?, + PropagationResult::Infeasible => { + AnalysisContext::new(target_boundaries).with_selectivity(0.0) + } + PropagationResult::CannotPropagate => { + AnalysisContext::new(target_boundaries).with_selectivity(1.0) + } + }, + ) } /// If the `PropagationResult` indicates success, this function calculates the @@ -184,24 +200,19 @@ fn shrink_boundaries( } }); let graph_nodes = graph.gather_node_indices(&[expr.clone()]); - let (_, root_index) = graph_nodes.first().ok_or_else(|| { - DataFusionError::Internal("Error in constructing predicate graph".to_string()) - })?; + let Some((_, root_index)) = graph_nodes.get(0) else { + return internal_err!( + "The ExprIntervalGraph under investigation does not have any nodes." + ); + }; let final_result = graph.get_interval(*root_index); - // If during selectivity calculation we encounter an error, use 1.0 as cardinality estimate - // safest estimate(e.q largest possible value). let selectivity = calculate_selectivity( &final_result.lower.value, &final_result.upper.value, &target_boundaries, &initial_boundaries, - ) - .unwrap_or(1.0); - - if !(0.0..=1.0).contains(&selectivity) { - return internal_err!("Selectivity is out of limit: {}", selectivity); - } + )?; Ok(AnalysisContext::new(target_boundaries).with_selectivity(selectivity)) } diff --git a/datafusion/physical-expr/src/expressions/in_list.rs b/datafusion/physical-expr/src/expressions/in_list.rs index bdc476f5b3a1..643bbfd820a6 100644 --- a/datafusion/physical-expr/src/expressions/in_list.rs +++ b/datafusion/physical-expr/src/expressions/in_list.rs @@ -17,8 +17,6 @@ //! Implementation of `InList` expressions: [`InListExpr`] -use ahash::RandomState; -use datafusion_common::exec_err; use std::any::Any; use std::fmt::Debug; use std::hash::{Hash, Hasher}; @@ -27,6 +25,7 @@ use std::sync::Arc; use crate::physical_expr::down_cast_any_ref; use crate::utils::expr_list_eq_any_order; use crate::PhysicalExpr; + use arrow::array::*; use arrow::buffer::BooleanBuffer; use arrow::compute::kernels::boolean::{not, or_kleene}; @@ -36,12 +35,16 @@ use arrow::datatypes::*; use arrow::record_batch::RecordBatch; use arrow::util::bit_iterator::BitIndexIterator; use arrow::{downcast_dictionary_array, downcast_primitive_array}; +use datafusion_common::cast::{ + as_boolean_array, as_generic_binary_array, as_string_array, +}; use datafusion_common::hash_utils::HashValue; use datafusion_common::{ - cast::{as_boolean_array, as_generic_binary_array, as_string_array}, - internal_err, not_impl_err, DataFusionError, Result, ScalarValue, + exec_err, internal_err, not_impl_err, DataFusionError, Result, ScalarValue, }; use datafusion_expr::ColumnarValue; + +use ahash::RandomState; use hashbrown::hash_map::RawEntryMut; use hashbrown::HashMap; diff --git a/datafusion/physical-expr/src/expressions/negative.rs b/datafusion/physical-expr/src/expressions/negative.rs index c66c91ef3152..86b000e76a32 100644 --- a/datafusion/physical-expr/src/expressions/negative.rs +++ b/datafusion/physical-expr/src/expressions/negative.rs @@ -106,7 +106,7 @@ impl PhysicalExpr for NegativeExpr { /// Given the child interval of a NegativeExpr, it calculates the NegativeExpr's interval. /// It replaces the upper and lower bounds after multiplying them with -1. - /// Ex: `(a, b] => [-b, -a)`` + /// Ex: `(a, b]` => `[-b, -a)` fn evaluate_bounds(&self, children: &[&Interval]) -> Result { Ok(Interval::new( children[0].upper.negate()?, @@ -171,7 +171,6 @@ mod tests { expressions::{col, Column}, intervals::Interval, }; - #[allow(unused_imports)] use arrow::array::*; use arrow::datatypes::*; use arrow_schema::DataType::{Float32, Float64, Int16, Int32, Int64, Int8}; diff --git a/datafusion/physical-expr/src/intervals/interval_aritmetic.rs b/datafusion/physical-expr/src/intervals/interval_aritmetic.rs index 277124ea223c..1ea9b2d9aee6 100644 --- a/datafusion/physical-expr/src/intervals/interval_aritmetic.rs +++ b/datafusion/physical-expr/src/intervals/interval_aritmetic.rs @@ -17,20 +17,20 @@ //! Interval arithmetic library +use std::borrow::Borrow; +use std::fmt::{self, Display, Formatter}; +use std::ops::{AddAssign, SubAssign}; + use crate::aggregate::min_max::{max, min}; use crate::intervals::rounding::{alter_fp_rounding_mode, next_down, next_up}; + use arrow::compute::{cast_with_options, CastOptions}; use arrow::datatypes::DataType; use arrow_array::ArrowNativeTypeOp; -use datafusion_common::{exec_err, internal_err, DataFusionError, Result, ScalarValue}; +use datafusion_common::{internal_err, DataFusionError, Result, ScalarValue}; use datafusion_expr::type_coercion::binary::get_result_type; use datafusion_expr::Operator; -use std::borrow::Borrow; -use std::fmt; -use std::fmt::{Display, Formatter}; -use std::ops::{AddAssign, SubAssign}; - /// This type represents a single endpoint of an [`Interval`]. An /// endpoint can be open (does not include the endpoint) or closed /// (includes the endpoint). @@ -87,7 +87,7 @@ impl IntervalBound { } /// Returns a new bound with a negated value, if any, and the same open/closed. - /// For example negating `[5` would return `[-5`, or `-1)` would return `1)` + /// For example negating `[5` would return `[-5`, or `-1)` would return `1)`. pub fn negate(&self) -> Result { self.value.arithmetic_negate().map(|value| IntervalBound { value, @@ -535,60 +535,77 @@ impl Interval { }; /// Returns the cardinality of this interval, which is the number of all - /// distinct points inside it. - pub fn cardinality(&self) -> Result { - match self.get_datatype() { - Ok(data_type) if data_type.is_integer() => { - if let Some(diff) = self.upper.value.distance(&self.lower.value) { - Ok(calculate_cardinality_based_on_bounds( + /// distinct points inside it. This function returns `None` if: + /// - The interval is unbounded from either side, or + /// - Cardinality calculations for the datatype in question is not + /// implemented yet, or + /// - An overflow occurs during the calculation. + /// + /// This function returns an error if the given interval is malformed. + pub fn cardinality(&self) -> Result> { + let data_type = self.get_datatype()?; + if data_type.is_integer() { + Ok(self.upper.value.distance(&self.lower.value).map(|diff| { + calculate_cardinality_based_on_bounds( + self.lower.open, + self.upper.open, + diff as u64, + ) + })) + } + // Ordering floating-point numbers according to their binary representations + // coincide with their natural ordering. Therefore, we can consider their + // binary representations as "indices" and subtract them. For details, see: + // https://stackoverflow.com/questions/8875064/how-many-distinct-floating-point-numbers-in-a-specific-range + else if data_type.is_floating() { + match (&self.lower.value, &self.upper.value) { + ( + ScalarValue::Float32(Some(lower)), + ScalarValue::Float32(Some(upper)), + ) => { + // Negative numbers are sorted in the reverse order. To always have a positive difference after the subtraction, + // we perform following transformation: + let lower_bits = lower.to_bits() as i32; + let upper_bits = upper.to_bits() as i32; + let transformed_lower = + lower_bits ^ ((lower_bits >> 31) & 0x7fffffff); + let transformed_upper = + upper_bits ^ ((upper_bits >> 31) & 0x7fffffff); + let Ok(count) = transformed_upper.sub_checked(transformed_lower) + else { + return Ok(None); + }; + Ok(Some(calculate_cardinality_based_on_bounds( self.lower.open, self.upper.open, - diff as u64, - )) - } else { - exec_err!("Cardinality cannot be calculated for {:?}", self) + count as u64, + ))) } - } - // Ordering floating-point numbers according to their binary representations - // coincide with their natural ordering. Therefore, we can consider their - // binary representations as "indices" and subtract them. For details, see: - // https://stackoverflow.com/questions/8875064/how-many-distinct-floating-point-numbers-in-a-specific-range - Ok(data_type) if data_type.is_floating() => { - // If the minimum value is a negative number, we need to - // switch sides to ensure an unsigned result. - let (min, max) = if self.lower.value - < ScalarValue::new_zero(&self.lower.value.data_type())? - { - (self.upper.value.clone(), self.lower.value.clone()) - } else { - (self.lower.value.clone(), self.upper.value.clone()) - }; - - match (min, max) { - ( - ScalarValue::Float32(Some(lower)), - ScalarValue::Float32(Some(upper)), - ) => Ok(calculate_cardinality_based_on_bounds( - self.lower.open, - self.upper.open, - (upper.to_bits().sub_checked(lower.to_bits()))? as u64, - )), - ( - ScalarValue::Float64(Some(lower)), - ScalarValue::Float64(Some(upper)), - ) => Ok(calculate_cardinality_based_on_bounds( + ( + ScalarValue::Float64(Some(lower)), + ScalarValue::Float64(Some(upper)), + ) => { + let lower_bits = lower.to_bits() as i64; + let upper_bits = upper.to_bits() as i64; + let transformed_lower = + lower_bits ^ ((lower_bits >> 63) & 0x7fffffffffffffff); + let transformed_upper = + upper_bits ^ ((upper_bits >> 63) & 0x7fffffffffffffff); + let Ok(count) = transformed_upper.sub_checked(transformed_lower) + else { + return Ok(None); + }; + Ok(Some(calculate_cardinality_based_on_bounds( self.lower.open, self.upper.open, - upper.to_bits().sub_checked(lower.to_bits())?, - )), - _ => exec_err!( - "Cardinality cannot be calculated for the datatype {:?}", - data_type - ), + count as u64, + ))) } + _ => Ok(None), } - // If the cardinality cannot be calculated anyway, give an error. - _ => exec_err!("Cardinality cannot be calculated for {:?}", self), + } else { + // Cardinality calculations are not implemented for this data type yet: + Ok(None) } } @@ -691,12 +708,25 @@ fn next_value(value: ScalarValue) -> ScalarValue { } } -/// This function computes the cardinality ratio of the given intervals. +/// This function computes the selectivity of an operation by computing the +/// cardinality ratio of the given input/output intervals. If this can not be +/// calculated for some reason, it returns `1.0` meaning fullly selective (no +/// filtering). pub fn cardinality_ratio( initial_interval: &Interval, final_interval: &Interval, ) -> Result { - Ok(final_interval.cardinality()? as f64 / initial_interval.cardinality()? as f64) + Ok( + match ( + final_interval.cardinality()?, + initial_interval.cardinality()?, + ) { + (Some(final_interval), Some(initial_interval)) => { + final_interval as f64 / initial_interval as f64 + } + _ => 1.0, + }, + ) } pub fn apply_operator(op: &Operator, lhs: &Interval, rhs: &Interval) -> Result { @@ -1018,7 +1048,6 @@ impl NullableInterval { mod tests { use super::next_value; use crate::intervals::{Interval, IntervalBound}; - use arrow_schema::DataType; use datafusion_common::{Result, ScalarValue}; @@ -1716,7 +1745,7 @@ mod tests { ), ]; for interval in intervals { - assert_eq!(interval.cardinality()?, distinct_f64); + assert_eq!(interval.cardinality()?.unwrap(), distinct_f64); } let intervals = [ @@ -1730,20 +1759,26 @@ mod tests { ), ]; for interval in intervals { - assert_eq!(interval.cardinality()?, distinct_f32); + assert_eq!(interval.cardinality()?.unwrap(), distinct_f32); } + // The regular logarithmic distribution of floating-point numbers are + // only applicable outside of the `(-phi, phi)` interval where `phi` + // denotes the largest positive subnormal floating-point number. Since + // the following intervals include such subnormal points, we cannot use + // a simple powers-of-two type formula for our expectations. Therefore, + // we manually supply the actual expected cardinality. let interval = Interval::new( IntervalBound::new(ScalarValue::from(-0.0625), false), IntervalBound::new(ScalarValue::from(0.0625), true), ); - assert_eq!(interval.cardinality()?, distinct_f64 * 2_048); + assert_eq!(interval.cardinality()?.unwrap(), 9178336040581070849); let interval = Interval::new( IntervalBound::new(ScalarValue::from(-0.0625_f32), false), IntervalBound::new(ScalarValue::from(0.0625_f32), true), ); - assert_eq!(interval.cardinality()?, distinct_f32 * 256); + assert_eq!(interval.cardinality()?.unwrap(), 2063597569); Ok(()) } diff --git a/datafusion/physical-expr/src/intervals/utils.rs b/datafusion/physical-expr/src/intervals/utils.rs index 2ddbca40733c..7a4ccff950e6 100644 --- a/datafusion/physical-expr/src/intervals/utils.rs +++ b/datafusion/physical-expr/src/intervals/utils.rs @@ -17,17 +17,18 @@ //! Utility functions for the interval arithmetic library +use std::sync::Arc; + use super::{Interval, IntervalBound}; use crate::{ expressions::{BinaryExpr, CastExpr, Column, Literal, NegativeExpr}, PhysicalExpr, }; + use arrow_schema::{DataType, SchemaRef}; use datafusion_common::{DataFusionError, Result, ScalarValue}; use datafusion_expr::Operator; -use std::sync::Arc; - const MDN_DAY_MASK: i128 = 0xFFFF_FFFF_0000_0000_0000_0000; const MDN_NS_MASK: i128 = 0xFFFF_FFFF_FFFF_FFFF; const DT_MS_MASK: i64 = 0xFFFF_FFFF; diff --git a/datafusion/physical-expr/src/lib.rs b/datafusion/physical-expr/src/lib.rs index 48d5f4e1308b..e670380e59d2 100644 --- a/datafusion/physical-expr/src/lib.rs +++ b/datafusion/physical-expr/src/lib.rs @@ -48,9 +48,6 @@ pub mod utils; pub mod var_provider; pub mod window; -// For backwards compatibility -pub use datafusion_common::hash_utils; - pub use aggregate::groups_accumulator::{ EmitTo, GroupsAccumulator, GroupsAccumulatorAdapter, }; diff --git a/datafusion/physical-plan/src/aggregates/mod.rs b/datafusion/physical-plan/src/aggregates/mod.rs index 3c9dabf60590..7191d51fb7f0 100644 --- a/datafusion/physical-plan/src/aggregates/mod.rs +++ b/datafusion/physical-plan/src/aggregates/mod.rs @@ -17,8 +17,14 @@ //! Aggregates functionalities +use std::any::Any; +use std::collections::HashMap; +use std::sync::Arc; + +use super::DisplayAs; use crate::aggregates::{ no_grouping::AggregateStream, row_hash::GroupedHashAggregateStream, + topk_stream::GroupedTopKAggregateStream, }; use crate::metrics::{ExecutionPlanMetricsSet, MetricsSet}; use crate::{ @@ -29,23 +35,25 @@ use crate::{ use arrow::array::ArrayRef; use arrow::datatypes::{Field, Schema, SchemaRef}; use arrow::record_batch::RecordBatch; +use datafusion_common::stats::Precision; use datafusion_common::utils::longest_consecutive_prefix; use datafusion_common::{not_impl_err, plan_err, DataFusionError, Result}; use datafusion_execution::TaskContext; use datafusion_expr::Accumulator; +use datafusion_physical_expr::utils::{ + convert_to_expr, get_finer_ordering, get_indices_of_matching_exprs, + ordering_satisfy_requirement_concrete, +}; use datafusion_physical_expr::{ + aggregate::is_order_sensitive, equivalence::project_equivalence_properties, - expressions::Column, + expressions::{Column, Max, Min}, normalize_out_expr_with_columns_map, physical_exprs_contains, reverse_order_bys, - utils::{convert_to_expr, get_indices_of_matching_exprs}, AggregateExpr, LexOrdering, LexOrderingReq, OrderingEquivalenceProperties, PhysicalExpr, PhysicalSortExpr, PhysicalSortRequirement, }; use itertools::Itertools; -use std::any::Any; -use std::collections::HashMap; -use std::sync::Arc; mod group_values; mod no_grouping; @@ -54,16 +62,8 @@ mod row_hash; mod topk; mod topk_stream; -use crate::aggregates::topk_stream::GroupedTopKAggregateStream; pub use datafusion_expr::AggregateFunction; -use datafusion_physical_expr::aggregate::is_order_sensitive; pub use datafusion_physical_expr::expressions::create_aggregate_expr; -use datafusion_physical_expr::expressions::{Max, Min}; -use datafusion_physical_expr::utils::{ - get_finer_ordering, ordering_satisfy_requirement_concrete, -}; - -use super::DisplayAs; /// Hash aggregate modes #[derive(Debug, Copy, Clone, PartialEq, Eq)] @@ -1005,36 +1005,49 @@ impl ExecutionPlan for AggregateExec { Some(self.metrics.clone_inner()) } - fn statistics(&self) -> Statistics { + fn statistics(&self) -> Result { // TODO stats: group expressions: // - once expressions will be able to compute their own stats, use it here // - case where we group by on a column for which with have the `distinct` stat // TODO stats: aggr expression: // - aggregations somtimes also preserve invariants such as min, max... + let column_statistics = Statistics::unknown_column(&self.schema()); match self.mode { AggregateMode::Final | AggregateMode::FinalPartitioned if self.group_by.expr.is_empty() => { - Statistics { - num_rows: Some(1), - is_exact: true, - ..Default::default() - } + Ok(Statistics { + num_rows: Precision::Exact(1), + column_statistics, + total_byte_size: Precision::Absent, + }) } _ => { - let input_stats = self.input.statistics(); - // Input statistics is exact and number of rows not greater than 1: - let is_exact = input_stats.is_exact - && (input_stats - .num_rows - .map(|num_rows| num_rows == 1) - .unwrap_or(false)); - Statistics { - // the output row count is surely not larger than its input row count - num_rows: self.input.statistics().num_rows, - is_exact, - ..Default::default() - } + // When the input row count is 0 or 1, we can adopt that statistic keeping its reliability. + // When it is larger than 1, we degrade the precision since it may decrease after aggregation. + let num_rows = if let Some(value) = + self.input().statistics()?.num_rows.get_value() + { + if *value > 1 { + self.input().statistics()?.num_rows.to_inexact() + } else if *value == 0 { + // Aggregation on an empty table creates a null row. + self.input() + .statistics()? + .num_rows + .add(&Precision::Exact(1)) + } else { + // num_rows = 1 case + self.input().statistics()?.num_rows + } + } else { + Precision::Absent + }; + Ok(Statistics { + num_rows, + column_statistics, + total_byte_size: Precision::Absent, + }) } } } @@ -1835,9 +1848,13 @@ mod tests { Ok(Box::pin(stream)) } - fn statistics(&self) -> Statistics { + fn statistics(&self) -> Result { let (_, batches) = some_data(); - common::compute_record_batch_statistics(&[batches], &self.schema(), None) + Ok(common::compute_record_batch_statistics( + &[batches], + &self.schema(), + None, + )) } } diff --git a/datafusion/physical-plan/src/analyze.rs b/datafusion/physical-plan/src/analyze.rs index 3a2cac59cfdf..bce242513559 100644 --- a/datafusion/physical-plan/src/analyze.rs +++ b/datafusion/physical-plan/src/analyze.rs @@ -20,19 +20,19 @@ use std::sync::Arc; use std::{any::Any, time::Instant}; -use crate::{ - display::DisplayableExecutionPlan, DisplayFormatType, ExecutionPlan, Partitioning, - Statistics, -}; -use arrow::{array::StringBuilder, datatypes::SchemaRef, record_batch::RecordBatch}; -use datafusion_common::{internal_err, DataFusionError, Result}; -use futures::StreamExt; - use super::expressions::PhysicalSortExpr; use super::stream::{RecordBatchReceiverStream, RecordBatchStreamAdapter}; use super::{DisplayAs, Distribution, SendableRecordBatchStream}; + +use crate::display::DisplayableExecutionPlan; +use crate::{DisplayFormatType, ExecutionPlan, Partitioning, Statistics}; + +use arrow::{array::StringBuilder, datatypes::SchemaRef, record_batch::RecordBatch}; +use datafusion_common::{internal_err, DataFusionError, Result}; use datafusion_execution::TaskContext; +use futures::StreamExt; + /// `EXPLAIN ANALYZE` execution plan operator. This operator runs its input, /// discards the results, and then prints out an annotated plan with metrics #[derive(Debug, Clone)] @@ -196,9 +196,9 @@ impl ExecutionPlan for AnalyzeExec { ))) } - fn statistics(&self) -> Statistics { + fn statistics(&self) -> Result { // Statistics an an ANALYZE plan are not relevant - Statistics::default() + Ok(Statistics::new_unknown(&self.schema())) } } diff --git a/datafusion/physical-plan/src/coalesce_batches.rs b/datafusion/physical-plan/src/coalesce_batches.rs index f46a228064fe..df9e8a8a2b8c 100644 --- a/datafusion/physical-plan/src/coalesce_batches.rs +++ b/datafusion/physical-plan/src/coalesce_batches.rs @@ -23,6 +23,9 @@ use std::pin::Pin; use std::sync::Arc; use std::task::{Context, Poll}; +use super::expressions::PhysicalSortExpr; +use super::metrics::{BaselineMetrics, ExecutionPlanMetricsSet, MetricsSet}; +use super::{DisplayAs, Statistics}; use crate::{ DisplayFormatType, EquivalenceProperties, ExecutionPlan, Partitioning, RecordBatchStream, SendableRecordBatchStream, @@ -35,11 +38,6 @@ use datafusion_common::Result; use datafusion_execution::TaskContext; use datafusion_physical_expr::OrderingEquivalenceProperties; -use super::expressions::PhysicalSortExpr; -use super::metrics::{BaselineMetrics, MetricsSet}; -use super::DisplayAs; -use super::{metrics::ExecutionPlanMetricsSet, Statistics}; - use futures::stream::{Stream, StreamExt}; use log::trace; @@ -174,7 +172,7 @@ impl ExecutionPlan for CoalesceBatchesExec { Some(self.metrics.clone_inner()) } - fn statistics(&self) -> Statistics { + fn statistics(&self) -> Result { self.input.statistics() } } diff --git a/datafusion/physical-plan/src/coalesce_partitions.rs b/datafusion/physical-plan/src/coalesce_partitions.rs index 646d42795ba4..f09b33e3f3b5 100644 --- a/datafusion/physical-plan/src/coalesce_partitions.rs +++ b/datafusion/physical-plan/src/coalesce_partitions.rs @@ -163,7 +163,7 @@ impl ExecutionPlan for CoalescePartitionsExec { Some(self.metrics.clone_inner()) } - fn statistics(&self) -> Statistics { + fn statistics(&self) -> Result { self.input.statistics() } } diff --git a/datafusion/physical-plan/src/common.rs b/datafusion/physical-plan/src/common.rs index c6cfbbfbbac7..649f3a31aa7e 100644 --- a/datafusion/physical-plan/src/common.rs +++ b/datafusion/physical-plan/src/common.rs @@ -17,24 +17,28 @@ //! Defines common code used in execution plans +use std::fs; +use std::fs::{metadata, File}; +use std::path::{Path, PathBuf}; +use std::sync::Arc; +use std::task::{Context, Poll}; + use super::SendableRecordBatchStream; use crate::stream::RecordBatchReceiverStream; use crate::{ColumnStatistics, ExecutionPlan, Statistics}; + use arrow::datatypes::Schema; use arrow::ipc::writer::{FileWriter, IpcWriteOptions}; use arrow::record_batch::RecordBatch; +use datafusion_common::stats::Precision; use datafusion_common::{plan_err, DataFusionError, Result}; use datafusion_execution::memory_pool::MemoryReservation; use datafusion_physical_expr::expressions::{BinaryExpr, Column}; use datafusion_physical_expr::{PhysicalExpr, PhysicalSortExpr}; + use futures::{Future, StreamExt, TryStreamExt}; use parking_lot::Mutex; use pin_project_lite::pin_project; -use std::fs; -use std::fs::{metadata, File}; -use std::path::{Path, PathBuf}; -use std::sync::Arc; -use std::task::{Context, Poll}; use tokio::task::JoinHandle; /// [`MemoryReservation`] used across query execution streams @@ -146,22 +150,21 @@ pub fn compute_record_batch_statistics( None => (0..schema.fields().len()).collect(), }; - let mut column_statistics = vec![ColumnStatistics::default(); projection.len()]; + let mut column_statistics = vec![ColumnStatistics::new_unknown(); projection.len()]; for partition in batches.iter() { for batch in partition { for (stat_index, col_index) in projection.iter().enumerate() { - *column_statistics[stat_index].null_count.get_or_insert(0) += - batch.column(*col_index).null_count(); + column_statistics[stat_index].null_count = + Precision::Exact(batch.column(*col_index).null_count()); } } } Statistics { - num_rows: Some(nb_rows), - total_byte_size: Some(total_byte_size), - column_statistics: Some(column_statistics), - is_exact: true, + num_rows: Precision::Exact(nb_rows), + total_byte_size: Precision::Exact(total_byte_size), + column_statistics, } } @@ -378,6 +381,7 @@ mod tests { use crate::memory::MemoryExec; use crate::sorts::sort::SortExec; use crate::union::UnionExec; + use arrow::compute::SortOptions; use arrow::{ array::{Float32Array, Float64Array}, @@ -671,9 +675,8 @@ mod tests { ])); let stats = compute_record_batch_statistics(&[], &schema, Some(vec![0, 1])); - assert_eq!(stats.num_rows, Some(0)); - assert!(stats.is_exact); - assert_eq!(stats.total_byte_size, Some(0)); + assert_eq!(stats.num_rows, Precision::Exact(0)); + assert_eq!(stats.total_byte_size, Precision::Exact(0)); Ok(()) } @@ -694,27 +697,26 @@ mod tests { compute_record_batch_statistics(&[vec![batch]], &schema, Some(vec![0, 1])); let mut expected = Statistics { - is_exact: true, - num_rows: Some(3), - total_byte_size: Some(464), // this might change a bit if the way we compute the size changes - column_statistics: Some(vec![ + num_rows: Precision::Exact(3), + total_byte_size: Precision::Exact(464), // this might change a bit if the way we compute the size changes + column_statistics: vec![ ColumnStatistics { - distinct_count: None, - max_value: None, - min_value: None, - null_count: Some(0), + distinct_count: Precision::Absent, + max_value: Precision::Absent, + min_value: Precision::Absent, + null_count: Precision::Exact(0), }, ColumnStatistics { - distinct_count: None, - max_value: None, - min_value: None, - null_count: Some(0), + distinct_count: Precision::Absent, + max_value: Precision::Absent, + min_value: Precision::Absent, + null_count: Precision::Exact(0), }, - ]), + ], }; // Prevent test flakiness due to undefined / changing implementation details - expected.total_byte_size = actual.total_byte_size; + expected.total_byte_size = actual.total_byte_size.clone(); assert_eq!(actual, expected); Ok(()) diff --git a/datafusion/physical-plan/src/display.rs b/datafusion/physical-plan/src/display.rs index e4a4e113eb07..aa368251ebf3 100644 --- a/datafusion/physical-plan/src/display.rs +++ b/datafusion/physical-plan/src/display.rs @@ -20,13 +20,12 @@ use std::fmt; +use super::{accept, ExecutionPlan, ExecutionPlanVisitor}; + use arrow_schema::SchemaRef; -use datafusion_common::display::StringifiedPlan; +use datafusion_common::display::{GraphvizBuilder, PlanType, StringifiedPlan}; use datafusion_physical_expr::PhysicalSortExpr; -use super::{accept, ExecutionPlan, ExecutionPlanVisitor}; -use datafusion_common::display::{GraphvizBuilder, PlanType}; - /// Options for controlling how each [`ExecutionPlan`] should format itself #[derive(Debug, Clone, Copy)] pub enum DisplayFormatType { @@ -261,8 +260,9 @@ impl<'a, 'b> ExecutionPlanVisitor for IndentVisitor<'a, 'b> { } } } + let stats = plan.statistics().map_err(|_e| fmt::Error)?; if self.show_statistics { - write!(self.f, ", statistics=[{}]", plan.statistics())?; + write!(self.f, ", statistics=[{}]", stats)?; } writeln!(self.f)?; self.indent += 1; @@ -341,8 +341,9 @@ impl ExecutionPlanVisitor for GraphvizVisitor<'_, '_> { } }; + let stats = plan.statistics().map_err(|_e| fmt::Error)?; let statistics = if self.show_statistics { - format!("statistics=[{}]", plan.statistics()) + format!("statistics=[{}]", stats) } else { "".to_string() }; diff --git a/datafusion/physical-plan/src/empty.rs b/datafusion/physical-plan/src/empty.rs index 675dac9ad265..a3e1fb79edb5 100644 --- a/datafusion/physical-plan/src/empty.rs +++ b/datafusion/physical-plan/src/empty.rs @@ -20,18 +20,18 @@ use std::any::Any; use std::sync::Arc; +use super::expressions::PhysicalSortExpr; +use super::{common, DisplayAs, SendableRecordBatchStream, Statistics}; use crate::{memory::MemoryStream, DisplayFormatType, ExecutionPlan, Partitioning}; + use arrow::array::{ArrayRef, NullArray}; use arrow::datatypes::{DataType, Field, Fields, Schema, SchemaRef}; use arrow::record_batch::RecordBatch; use datafusion_common::{internal_err, DataFusionError, Result}; -use log::trace; - -use super::expressions::PhysicalSortExpr; -use super::{common, DisplayAs, SendableRecordBatchStream, Statistics}; - use datafusion_execution::TaskContext; +use log::trace; + /// Execution plan for empty relation (produces no rows) #[derive(Debug)] pub struct EmptyExec { @@ -161,11 +161,15 @@ impl ExecutionPlan for EmptyExec { )?)) } - fn statistics(&self) -> Statistics { + fn statistics(&self) -> Result { let batch = self .data() .expect("Create empty RecordBatch should not fail"); - common::compute_record_batch_statistics(&[batch], &self.schema, None) + Ok(common::compute_record_batch_statistics( + &[batch], + &self.schema, + None, + )) } } diff --git a/datafusion/physical-plan/src/explain.rs b/datafusion/physical-plan/src/explain.rs index 8d6bf4105f6a..81b8f9944110 100644 --- a/datafusion/physical-plan/src/explain.rs +++ b/datafusion/physical-plan/src/explain.rs @@ -20,19 +20,18 @@ use std::any::Any; use std::sync::Arc; -use datafusion_common::display::StringifiedPlan; +use super::expressions::PhysicalSortExpr; +use super::{DisplayAs, SendableRecordBatchStream}; +use crate::stream::RecordBatchStreamAdapter; +use crate::{DisplayFormatType, ExecutionPlan, Partitioning, Statistics}; +use arrow::{array::StringBuilder, datatypes::SchemaRef, record_batch::RecordBatch}; +use datafusion_common::display::StringifiedPlan; use datafusion_common::{internal_err, DataFusionError, Result}; +use datafusion_execution::TaskContext; -use crate::{DisplayFormatType, ExecutionPlan, Partitioning, Statistics}; -use arrow::{array::StringBuilder, datatypes::SchemaRef, record_batch::RecordBatch}; use log::trace; -use super::DisplayAs; -use super::{expressions::PhysicalSortExpr, SendableRecordBatchStream}; -use crate::stream::RecordBatchStreamAdapter; -use datafusion_execution::TaskContext; - /// Explain execution plan operator. This operator contains the string /// values of the various plans it has when it is created, and passes /// them to its output. @@ -169,9 +168,9 @@ impl ExecutionPlan for ExplainExec { ))) } - fn statistics(&self) -> Statistics { + fn statistics(&self) -> Result { // Statistics an EXPLAIN plan are not relevant - Statistics::default() + Ok(Statistics::new_unknown(&self.schema())) } } diff --git a/datafusion/physical-plan/src/filter.rs b/datafusion/physical-plan/src/filter.rs index ccd0bd525f10..59f9928d0e21 100644 --- a/datafusion/physical-plan/src/filter.rs +++ b/datafusion/physical-plan/src/filter.rs @@ -37,6 +37,7 @@ use arrow::compute::filter_record_batch; use arrow::datatypes::{DataType, SchemaRef}; use arrow::record_batch::RecordBatch; use datafusion_common::cast::as_boolean_array; +use datafusion_common::stats::Precision; use datafusion_common::{plan_err, DataFusionError, Result}; use datafusion_execution::TaskContext; use datafusion_expr::Operator; @@ -48,6 +49,7 @@ use datafusion_physical_expr::{ use datafusion_physical_expr::intervals::utils::check_support; use datafusion_physical_expr::utils::collect_columns; + use futures::stream::{Stream, StreamExt}; use log::trace; @@ -154,19 +156,17 @@ impl ExecutionPlan for FilterExec { } fn ordering_equivalence_properties(&self) -> OrderingEquivalenceProperties { - let stats = self.statistics(); + let stats = self + .statistics() + .expect("Ordering equivalences need to handle the error case of statistics"); // Add the columns that have only one value (singleton) after filtering to constants. - if let Some(col_stats) = stats.column_statistics { - let constants = collect_columns(self.predicate()) - .into_iter() - .filter(|column| col_stats[column.index()].is_singleton()) - .map(|column| Arc::new(column) as Arc) - .collect::>(); - let filter_oeq = self.input.ordering_equivalence_properties(); - filter_oeq.with_constants(constants) - } else { - self.input.ordering_equivalence_properties() - } + let constants = collect_columns(self.predicate()) + .into_iter() + .filter(|column| stats.column_statistics[column.index()].is_singleton()) + .map(|column| Arc::new(column) as Arc) + .collect::>(); + let filter_oeq = self.input.ordering_equivalence_properties(); + filter_oeq.with_constants(constants) } fn with_new_children( @@ -200,56 +200,43 @@ impl ExecutionPlan for FilterExec { /// The output statistics of a filtering operation can be estimated if the /// predicate's selectivity value can be determined for the incoming data. - fn statistics(&self) -> Statistics { + fn statistics(&self) -> Result { let predicate = self.predicate(); - if !check_support(predicate, &self.schema()) { - return Statistics::default(); + let schema = self.schema(); + if !check_support(predicate, &schema) { + return Ok(Statistics::new_unknown(&schema)); } + let input_stats = self.input.statistics()?; - let input_stats = self.input.statistics(); - let input_column_stats = match input_stats.column_statistics { - Some(stats) => stats, - None => self - .schema() - .fields - .iter() - .map(|field| { - ColumnStatistics::new_with_unbounded_column(field.data_type()) - }) - .collect::>(), - }; - - let starter_ctx = - AnalysisContext::from_statistics(&self.input.schema(), &input_column_stats); - - let analysis_ctx = match analyze(predicate, starter_ctx) { - Ok(ctx) => ctx, - Err(_) => return Statistics::default(), - }; + let num_rows = input_stats.num_rows; + let total_byte_size = input_stats.total_byte_size; + let input_analysis_ctx = AnalysisContext::try_from_statistics( + &self.input.schema(), + &input_stats.column_statistics, + )?; + let analysis_ctx = analyze(predicate, input_analysis_ctx)?; + // Estimate (inexact) selectivity of predicate let selectivity = analysis_ctx.selectivity.unwrap_or(1.0); - - let num_rows = input_stats - .num_rows - .map(|num| (num as f64 * selectivity).ceil() as usize); - let total_byte_size = input_stats - .total_byte_size - .map(|size| (size as f64 * selectivity).ceil() as usize); - - let column_statistics = if let Some(analysis_boundaries) = analysis_ctx.boundaries - { - collect_new_statistics(input_column_stats, selectivity, analysis_boundaries) - } else { - input_column_stats + let num_rows = match num_rows.get_value() { + Some(nr) => Precision::Inexact((*nr as f64 * selectivity).ceil() as usize), + None => Precision::Absent, + }; + let total_byte_size = match total_byte_size.get_value() { + Some(tbs) => Precision::Inexact((*tbs as f64 * selectivity).ceil() as usize), + None => Precision::Absent, }; - Statistics { + let column_statistics = collect_new_statistics( + &input_stats.column_statistics, + analysis_ctx.boundaries, + ); + Ok(Statistics { num_rows, total_byte_size, - column_statistics: Some(column_statistics), - is_exact: Default::default(), - } + column_statistics, + }) } } @@ -258,11 +245,9 @@ impl ExecutionPlan for FilterExec { /// is adjusted by using the next/previous value for its data type to convert /// it into a closed bound. fn collect_new_statistics( - input_column_stats: Vec, - selectivity: f64, + input_column_stats: &[ColumnStatistics], analysis_boundaries: Vec, ) -> Vec { - let nonempty_columns = selectivity > 0.0; analysis_boundaries .into_iter() .enumerate() @@ -277,10 +262,16 @@ fn collect_new_statistics( )| { let closed_interval = interval.close_bounds(); ColumnStatistics { - null_count: input_column_stats[idx].null_count, - max_value: nonempty_columns.then_some(closed_interval.upper.value), - min_value: nonempty_columns.then_some(closed_interval.lower.value), - distinct_count, + null_count: match input_column_stats[idx].null_count.get_value() { + Some(nc) => Precision::Inexact(*nc), + None => Precision::Absent, + }, + max_value: Precision::Inexact(closed_interval.upper.value), + min_value: Precision::Inexact(closed_interval.lower.value), + distinct_count: match distinct_count.get_value() { + Some(dc) => Precision::Inexact(*dc), + None => Precision::Absent, + }, } }, ) @@ -396,18 +387,18 @@ pub type EqualAndNonEqual<'a> = #[cfg(test)] mod tests { + use std::iter::Iterator; + use std::sync::Arc; use super::*; use crate::expressions::*; use crate::test; use crate::test::exec::StatisticsExec; use crate::ExecutionPlan; + use arrow::datatypes::{DataType, Field, Schema}; - use datafusion_common::ColumnStatistics; - use datafusion_common::ScalarValue; + use datafusion_common::{ColumnStatistics, ScalarValue}; use datafusion_expr::Operator; - use std::iter::Iterator; - use std::sync::Arc; #[tokio::test] async fn collect_columns_predicates() -> Result<()> { @@ -460,14 +451,13 @@ mod tests { let schema = Schema::new(vec![Field::new("a", DataType::Int32, false)]); let input = Arc::new(StatisticsExec::new( Statistics { - num_rows: Some(100), - total_byte_size: Some(100 * bytes_per_row), - column_statistics: Some(vec![ColumnStatistics { - min_value: Some(ScalarValue::Int32(Some(1))), - max_value: Some(ScalarValue::Int32(Some(100))), + num_rows: Precision::Inexact(100), + total_byte_size: Precision::Inexact(100 * bytes_per_row), + column_statistics: vec![ColumnStatistics { + min_value: Precision::Inexact(ScalarValue::Int32(Some(1))), + max_value: Precision::Inexact(ScalarValue::Int32(Some(100))), ..Default::default() - }]), - ..Default::default() + }], }, schema.clone(), )); @@ -480,16 +470,19 @@ mod tests { let filter: Arc = Arc::new(FilterExec::try_new(predicate, input)?); - let statistics = filter.statistics(); - assert_eq!(statistics.num_rows, Some(25)); - assert_eq!(statistics.total_byte_size, Some(25 * bytes_per_row)); + let statistics = filter.statistics()?; + assert_eq!(statistics.num_rows, Precision::Inexact(25)); + assert_eq!( + statistics.total_byte_size, + Precision::Inexact(25 * bytes_per_row) + ); assert_eq!( statistics.column_statistics, - Some(vec![ColumnStatistics { - min_value: Some(ScalarValue::Int32(Some(1))), - max_value: Some(ScalarValue::Int32(Some(25))), + vec![ColumnStatistics { + min_value: Precision::Inexact(ScalarValue::Int32(Some(1))), + max_value: Precision::Inexact(ScalarValue::Int32(Some(25))), ..Default::default() - }]) + }] ); Ok(()) @@ -502,13 +495,13 @@ mod tests { let schema = Schema::new(vec![Field::new("a", DataType::Int32, false)]); let input = Arc::new(StatisticsExec::new( Statistics { - num_rows: Some(100), - column_statistics: Some(vec![ColumnStatistics { - min_value: Some(ScalarValue::Int32(Some(1))), - max_value: Some(ScalarValue::Int32(Some(100))), + num_rows: Precision::Inexact(100), + column_statistics: vec![ColumnStatistics { + min_value: Precision::Inexact(ScalarValue::Int32(Some(1))), + max_value: Precision::Inexact(ScalarValue::Int32(Some(100))), ..Default::default() - }]), - ..Default::default() + }], + total_byte_size: Precision::Absent, }, schema.clone(), )); @@ -527,15 +520,15 @@ mod tests { sub_filter, )?); - let statistics = filter.statistics(); - assert_eq!(statistics.num_rows, Some(16)); + let statistics = filter.statistics()?; + assert_eq!(statistics.num_rows, Precision::Inexact(16)); assert_eq!( statistics.column_statistics, - Some(vec![ColumnStatistics { - min_value: Some(ScalarValue::Int32(Some(10))), - max_value: Some(ScalarValue::Int32(Some(25))), + vec![ColumnStatistics { + min_value: Precision::Inexact(ScalarValue::Int32(Some(10))), + max_value: Precision::Inexact(ScalarValue::Int32(Some(25))), ..Default::default() - }]) + }] ); Ok(()) @@ -552,20 +545,20 @@ mod tests { ]); let input = Arc::new(StatisticsExec::new( Statistics { - num_rows: Some(100), - column_statistics: Some(vec![ + num_rows: Precision::Inexact(100), + column_statistics: vec![ ColumnStatistics { - min_value: Some(ScalarValue::Int32(Some(1))), - max_value: Some(ScalarValue::Int32(Some(100))), + min_value: Precision::Inexact(ScalarValue::Int32(Some(1))), + max_value: Precision::Inexact(ScalarValue::Int32(Some(100))), ..Default::default() }, ColumnStatistics { - min_value: Some(ScalarValue::Int32(Some(1))), - max_value: Some(ScalarValue::Int32(Some(50))), + min_value: Precision::Inexact(ScalarValue::Int32(Some(1))), + max_value: Precision::Inexact(ScalarValue::Int32(Some(50))), ..Default::default() }, - ]), - ..Default::default() + ], + total_byte_size: Precision::Absent, }, schema.clone(), )); @@ -587,28 +580,28 @@ mod tests { binary(col("a", &schema)?, Operator::GtEq, lit(10i32), &schema)?, b_gt_5, )?); - let statistics = filter.statistics(); + let statistics = filter.statistics()?; // On a uniform distribution, only fifteen rows will satisfy the // filter that 'a' proposed (a >= 10 AND a <= 25) (15/100) and only // 5 rows will satisfy the filter that 'b' proposed (b > 45) (5/50). // // Which would result with a selectivity of '15/100 * 5/50' or 0.015 // and that means about %1.5 of the all rows (rounded up to 2 rows). - assert_eq!(statistics.num_rows, Some(2)); + assert_eq!(statistics.num_rows, Precision::Inexact(2)); assert_eq!( statistics.column_statistics, - Some(vec![ + vec![ ColumnStatistics { - min_value: Some(ScalarValue::Int32(Some(10))), - max_value: Some(ScalarValue::Int32(Some(25))), + min_value: Precision::Inexact(ScalarValue::Int32(Some(10))), + max_value: Precision::Inexact(ScalarValue::Int32(Some(25))), ..Default::default() }, ColumnStatistics { - min_value: Some(ScalarValue::Int32(Some(46))), - max_value: Some(ScalarValue::Int32(Some(50))), + min_value: Precision::Inexact(ScalarValue::Int32(Some(46))), + max_value: Precision::Inexact(ScalarValue::Int32(Some(50))), ..Default::default() } - ]) + ] ); Ok(()) @@ -620,12 +613,7 @@ mod tests { // a: min=???, max=??? (missing) let schema = Schema::new(vec![Field::new("a", DataType::Int32, false)]); let input = Arc::new(StatisticsExec::new( - Statistics { - column_statistics: Some(vec![ColumnStatistics { - ..Default::default() - }]), - ..Default::default() - }, + Statistics::new_unknown(&schema), schema.clone(), )); @@ -637,8 +625,8 @@ mod tests { let filter: Arc = Arc::new(FilterExec::try_new(predicate, input)?); - let statistics = filter.statistics(); - assert_eq!(statistics.num_rows, None); + let statistics = filter.statistics()?; + assert_eq!(statistics.num_rows, Precision::Absent); Ok(()) } @@ -656,26 +644,25 @@ mod tests { ]); let input = Arc::new(StatisticsExec::new( Statistics { - num_rows: Some(1000), - total_byte_size: Some(4000), - column_statistics: Some(vec![ + num_rows: Precision::Inexact(1000), + total_byte_size: Precision::Inexact(4000), + column_statistics: vec![ ColumnStatistics { - min_value: Some(ScalarValue::Int32(Some(1))), - max_value: Some(ScalarValue::Int32(Some(100))), + min_value: Precision::Inexact(ScalarValue::Int32(Some(1))), + max_value: Precision::Inexact(ScalarValue::Int32(Some(100))), ..Default::default() }, ColumnStatistics { - min_value: Some(ScalarValue::Int32(Some(1))), - max_value: Some(ScalarValue::Int32(Some(3))), + min_value: Precision::Inexact(ScalarValue::Int32(Some(1))), + max_value: Precision::Inexact(ScalarValue::Int32(Some(3))), ..Default::default() }, ColumnStatistics { - min_value: Some(ScalarValue::Float32(Some(1000.0))), - max_value: Some(ScalarValue::Float32(Some(1100.0))), + min_value: Precision::Inexact(ScalarValue::Float32(Some(1000.0))), + max_value: Precision::Inexact(ScalarValue::Float32(Some(1100.0))), ..Default::default() }, - ]), - ..Default::default() + ], }, schema, )); @@ -711,47 +698,51 @@ mod tests { )); let filter: Arc = Arc::new(FilterExec::try_new(predicate, input)?); - let statistics = filter.statistics(); + let statistics = filter.statistics()?; // 0.5 (from a) * 0.333333... (from b) * 0.798387... (from c) ≈ 0.1330... // num_rows after ceil => 133.0... => 134 // total_byte_size after ceil => 532.0... => 533 - assert_eq!(statistics.num_rows, Some(134)); - assert_eq!(statistics.total_byte_size, Some(533)); + assert_eq!(statistics.num_rows, Precision::Inexact(134)); + assert_eq!(statistics.total_byte_size, Precision::Inexact(533)); let exp_col_stats = vec![ ColumnStatistics { - min_value: Some(ScalarValue::Int32(Some(4))), - max_value: Some(ScalarValue::Int32(Some(53))), + min_value: Precision::Inexact(ScalarValue::Int32(Some(4))), + max_value: Precision::Inexact(ScalarValue::Int32(Some(53))), ..Default::default() }, ColumnStatistics { - min_value: Some(ScalarValue::Int32(Some(3))), - max_value: Some(ScalarValue::Int32(Some(3))), + min_value: Precision::Inexact(ScalarValue::Int32(Some(3))), + max_value: Precision::Inexact(ScalarValue::Int32(Some(3))), ..Default::default() }, ColumnStatistics { - min_value: Some(ScalarValue::Float32(Some(1000.0))), - max_value: Some(ScalarValue::Float32(Some(1075.0))), + min_value: Precision::Inexact(ScalarValue::Float32(Some(1000.0))), + max_value: Precision::Inexact(ScalarValue::Float32(Some(1075.0))), ..Default::default() }, ]; let _ = exp_col_stats .into_iter() - .zip(statistics.column_statistics.unwrap()) + .zip(statistics.column_statistics) .map(|(expected, actual)| { - if actual.min_value.clone().unwrap().data_type().is_floating() { - // Windows rounds arithmetic operation results differently for floating point numbers. - // Therefore, we check if the actual values are in an epsilon range. - let actual_min = actual.min_value.unwrap(); - let actual_max = actual.max_value.unwrap(); - let expected_min = expected.min_value.unwrap(); - let expected_max = expected.max_value.unwrap(); - let eps = ScalarValue::Float32(Some(1e-6)); - - assert!(actual_min.sub(&expected_min).unwrap() < eps); - assert!(actual_min.sub(&expected_min).unwrap() < eps); - - assert!(actual_max.sub(&expected_max).unwrap() < eps); - assert!(actual_max.sub(&expected_max).unwrap() < eps); + if let Some(val) = actual.min_value.get_value() { + if val.data_type().is_floating() { + // Windows rounds arithmetic operation results differently for floating point numbers. + // Therefore, we check if the actual values are in an epsilon range. + let actual_min = actual.min_value.get_value().unwrap(); + let actual_max = actual.max_value.get_value().unwrap(); + let expected_min = expected.min_value.get_value().unwrap(); + let expected_max = expected.max_value.get_value().unwrap(); + let eps = ScalarValue::Float32(Some(1e-6)); + + assert!(actual_min.sub(expected_min).unwrap() < eps); + assert!(actual_min.sub(expected_min).unwrap() < eps); + + assert!(actual_max.sub(expected_max).unwrap() < eps); + assert!(actual_max.sub(expected_max).unwrap() < eps); + } else { + assert_eq!(actual, expected); + } } else { assert_eq!(actual, expected); } @@ -771,21 +762,20 @@ mod tests { ]); let input = Arc::new(StatisticsExec::new( Statistics { - num_rows: Some(1000), - total_byte_size: Some(4000), - column_statistics: Some(vec![ + num_rows: Precision::Inexact(1000), + total_byte_size: Precision::Inexact(4000), + column_statistics: vec![ ColumnStatistics { - min_value: Some(ScalarValue::Int32(Some(1))), - max_value: Some(ScalarValue::Int32(Some(100))), + min_value: Precision::Inexact(ScalarValue::Int32(Some(1))), + max_value: Precision::Inexact(ScalarValue::Int32(Some(100))), ..Default::default() }, ColumnStatistics { - min_value: Some(ScalarValue::Int32(Some(1))), - max_value: Some(ScalarValue::Int32(Some(3))), + min_value: Precision::Inexact(ScalarValue::Int32(Some(1))), + max_value: Precision::Inexact(ScalarValue::Int32(Some(3))), ..Default::default() }, - ]), - ..Default::default() + ], }, schema, )); @@ -804,13 +794,13 @@ mod tests { )), )); // Since filter predicate passes all entries, statistics after filter shouldn't change. - let expected = input.statistics().column_statistics; + let expected = input.statistics()?.column_statistics; let filter: Arc = Arc::new(FilterExec::try_new(predicate, input)?); - let statistics = filter.statistics(); + let statistics = filter.statistics()?; - assert_eq!(statistics.num_rows, Some(1000)); - assert_eq!(statistics.total_byte_size, Some(4000)); + assert_eq!(statistics.num_rows, Precision::Inexact(1000)); + assert_eq!(statistics.total_byte_size, Precision::Inexact(4000)); assert_eq!(statistics.column_statistics, expected); Ok(()) @@ -827,21 +817,20 @@ mod tests { ]); let input = Arc::new(StatisticsExec::new( Statistics { - num_rows: Some(1000), - total_byte_size: Some(4000), - column_statistics: Some(vec![ + num_rows: Precision::Inexact(1000), + total_byte_size: Precision::Inexact(4000), + column_statistics: vec![ ColumnStatistics { - min_value: Some(ScalarValue::Int32(Some(1))), - max_value: Some(ScalarValue::Int32(Some(100))), + min_value: Precision::Inexact(ScalarValue::Int32(Some(1))), + max_value: Precision::Inexact(ScalarValue::Int32(Some(100))), ..Default::default() }, ColumnStatistics { - min_value: Some(ScalarValue::Int32(Some(1))), - max_value: Some(ScalarValue::Int32(Some(3))), + min_value: Precision::Inexact(ScalarValue::Int32(Some(1))), + max_value: Precision::Inexact(ScalarValue::Int32(Some(3))), ..Default::default() }, - ]), - ..Default::default() + ], }, schema, )); @@ -861,24 +850,24 @@ mod tests { )); let filter: Arc = Arc::new(FilterExec::try_new(predicate, input)?); - let statistics = filter.statistics(); + let statistics = filter.statistics()?; - assert_eq!(statistics.num_rows, Some(0)); - assert_eq!(statistics.total_byte_size, Some(0)); + assert_eq!(statistics.num_rows, Precision::Inexact(0)); + assert_eq!(statistics.total_byte_size, Precision::Inexact(0)); assert_eq!( statistics.column_statistics, - Some(vec![ + vec![ ColumnStatistics { - min_value: None, - max_value: None, + min_value: Precision::Inexact(ScalarValue::Int32(Some(1))), + max_value: Precision::Inexact(ScalarValue::Int32(Some(100))), ..Default::default() }, ColumnStatistics { - min_value: None, - max_value: None, + min_value: Precision::Inexact(ScalarValue::Int32(Some(1))), + max_value: Precision::Inexact(ScalarValue::Int32(Some(3))), ..Default::default() }, - ]) + ] ); Ok(()) @@ -892,21 +881,20 @@ mod tests { ]); let input = Arc::new(StatisticsExec::new( Statistics { - num_rows: Some(1000), - total_byte_size: Some(4000), - column_statistics: Some(vec![ + num_rows: Precision::Inexact(1000), + total_byte_size: Precision::Inexact(4000), + column_statistics: vec![ ColumnStatistics { - min_value: Some(ScalarValue::Int32(Some(1))), - max_value: Some(ScalarValue::Int32(Some(100))), + min_value: Precision::Inexact(ScalarValue::Int32(Some(1))), + max_value: Precision::Inexact(ScalarValue::Int32(Some(100))), ..Default::default() }, ColumnStatistics { - min_value: Some(ScalarValue::Int32(Some(1))), - max_value: Some(ScalarValue::Int32(Some(100))), + min_value: Precision::Inexact(ScalarValue::Int32(Some(1))), + max_value: Precision::Inexact(ScalarValue::Int32(Some(100))), ..Default::default() }, - ]), - ..Default::default() + ], }, schema, )); @@ -918,26 +906,71 @@ mod tests { )); let filter: Arc = Arc::new(FilterExec::try_new(predicate, input)?); - let statistics = filter.statistics(); + let statistics = filter.statistics()?; - assert_eq!(statistics.num_rows, Some(490)); - assert_eq!(statistics.total_byte_size, Some(1960)); + assert_eq!(statistics.num_rows, Precision::Inexact(490)); + assert_eq!(statistics.total_byte_size, Precision::Inexact(1960)); assert_eq!( statistics.column_statistics, - Some(vec![ + vec![ ColumnStatistics { - min_value: Some(ScalarValue::Int32(Some(1))), - max_value: Some(ScalarValue::Int32(Some(49))), + min_value: Precision::Inexact(ScalarValue::Int32(Some(1))), + max_value: Precision::Inexact(ScalarValue::Int32(Some(49))), ..Default::default() }, ColumnStatistics { - min_value: Some(ScalarValue::Int32(Some(1))), - max_value: Some(ScalarValue::Int32(Some(100))), + min_value: Precision::Inexact(ScalarValue::Int32(Some(1))), + max_value: Precision::Inexact(ScalarValue::Int32(Some(100))), ..Default::default() }, - ]) + ] ); Ok(()) } + + #[tokio::test] + async fn test_empty_input_statistics() -> Result<()> { + let schema = Schema::new(vec![Field::new("a", DataType::Int32, false)]); + let input = Arc::new(StatisticsExec::new( + Statistics::new_unknown(&schema), + schema, + )); + // WHERE a <= 10 AND 0 <= a - 5 + let predicate = Arc::new(BinaryExpr::new( + Arc::new(BinaryExpr::new( + Arc::new(Column::new("a", 0)), + Operator::LtEq, + Arc::new(Literal::new(ScalarValue::Int32(Some(10)))), + )), + Operator::And, + Arc::new(BinaryExpr::new( + Arc::new(Literal::new(ScalarValue::Int32(Some(0)))), + Operator::LtEq, + Arc::new(BinaryExpr::new( + Arc::new(Column::new("a", 0)), + Operator::Minus, + Arc::new(Literal::new(ScalarValue::Int32(Some(5)))), + )), + )), + )); + let filter: Arc = + Arc::new(FilterExec::try_new(predicate, input)?); + let filter_statistics = filter.statistics()?; + + let expected_filter_statistics = Statistics { + num_rows: Precision::Absent, + total_byte_size: Precision::Absent, + column_statistics: vec![ColumnStatistics { + null_count: Precision::Absent, + min_value: Precision::Inexact(ScalarValue::Int32(Some(5))), + max_value: Precision::Inexact(ScalarValue::Int32(Some(10))), + distinct_count: Precision::Absent, + }], + }; + + assert_eq!(filter_statistics, expected_filter_statistics); + + Ok(()) + } } diff --git a/datafusion/physical-plan/src/insert.rs b/datafusion/physical-plan/src/insert.rs index bff20e85b7ff..5b58a0a77134 100644 --- a/datafusion/physical-plan/src/insert.rs +++ b/datafusion/physical-plan/src/insert.rs @@ -17,28 +17,29 @@ //! Execution plan for writing data to [`DataSink`]s +use std::any::Any; +use std::fmt; +use std::fmt::Debug; +use std::sync::Arc; + use super::expressions::PhysicalSortExpr; use super::{ DisplayAs, DisplayFormatType, ExecutionPlan, Partitioning, SendableRecordBatchStream, Statistics, }; +use crate::metrics::MetricsSet; +use crate::stream::RecordBatchStreamAdapter; + use arrow::datatypes::SchemaRef; use arrow::record_batch::RecordBatch; use arrow_array::{ArrayRef, UInt64Array}; use arrow_schema::{DataType, Field, Schema}; -use async_trait::async_trait; -use core::fmt; -use datafusion_common::Result; +use datafusion_common::{exec_err, internal_err, DataFusionError, Result}; +use datafusion_execution::TaskContext; use datafusion_physical_expr::PhysicalSortRequirement; -use futures::StreamExt; -use std::any::Any; -use std::fmt::Debug; -use std::sync::Arc; -use crate::metrics::MetricsSet; -use crate::stream::RecordBatchStreamAdapter; -use datafusion_common::{exec_err, internal_err, DataFusionError}; -use datafusion_execution::TaskContext; +use async_trait::async_trait; +use futures::StreamExt; /// `DataSink` implements writing streams of [`RecordBatch`]es to /// user defined destinations. @@ -284,8 +285,8 @@ impl ExecutionPlan for FileSinkExec { ))) } - fn statistics(&self) -> Statistics { - Statistics::default() + fn statistics(&self) -> Result { + Ok(Statistics::new_unknown(&self.schema())) } } diff --git a/datafusion/physical-plan/src/joins/cross_join.rs b/datafusion/physical-plan/src/joins/cross_join.rs index 4ba29524b3e2..1ffd9ad1c18a 100644 --- a/datafusion/physical-plan/src/joins/cross_join.rs +++ b/datafusion/physical-plan/src/joins/cross_join.rs @@ -18,13 +18,12 @@ //! Defines the cross join plan for loading the left side of the cross join //! and producing batches in parallel for the right partitions -use futures::{ready, StreamExt}; -use futures::{Stream, TryStreamExt}; use std::{any::Any, sync::Arc, task::Poll}; -use arrow::datatypes::{Fields, Schema, SchemaRef}; -use arrow::record_batch::RecordBatch; - +use super::utils::{ + adjust_right_output_partitioning, cross_join_equivalence_properties, + BuildProbeJoinMetrics, OnceAsync, OnceFut, +}; use crate::metrics::{ExecutionPlanMetricsSet, MetricsSet}; use crate::DisplayAs; use crate::{ @@ -33,16 +32,17 @@ use crate::{ ExecutionPlan, Partitioning, PhysicalSortExpr, RecordBatchStream, SendableRecordBatchStream, Statistics, }; -use async_trait::async_trait; -use datafusion_common::{plan_err, DataFusionError}; -use datafusion_common::{Result, ScalarValue}; + +use arrow::datatypes::{Fields, Schema, SchemaRef}; +use arrow::record_batch::RecordBatch; +use datafusion_common::stats::Precision; +use datafusion_common::{plan_err, DataFusionError, Result, ScalarValue}; use datafusion_execution::memory_pool::{MemoryConsumer, MemoryReservation}; use datafusion_execution::TaskContext; -use super::utils::{ - adjust_right_output_partitioning, cross_join_equivalence_properties, - BuildProbeJoinMetrics, OnceAsync, OnceFut, -}; +use async_trait::async_trait; +use futures::{ready, StreamExt}; +use futures::{Stream, TryStreamExt}; /// Data of the left side type JoinLeftData = (RecordBatch, MemoryReservation); @@ -257,77 +257,55 @@ impl ExecutionPlan for CrossJoinExec { })) } - fn statistics(&self) -> Statistics { - stats_cartesian_product( - self.left.statistics(), - self.left.schema().fields().len(), - self.right.statistics(), - self.right.schema().fields().len(), - ) + fn statistics(&self) -> Result { + Ok(stats_cartesian_product( + self.left.statistics()?, + self.right.statistics()?, + )) } } /// [left/right]_col_count are required in case the column statistics are None fn stats_cartesian_product( left_stats: Statistics, - left_col_count: usize, right_stats: Statistics, - right_col_count: usize, ) -> Statistics { let left_row_count = left_stats.num_rows; let right_row_count = right_stats.num_rows; // calculate global stats - let is_exact = left_stats.is_exact && right_stats.is_exact; - let num_rows = left_stats - .num_rows - .zip(right_stats.num_rows) - .map(|(a, b)| a * b); + let num_rows = left_row_count.multiply(&right_row_count); // the result size is two times a*b because you have the columns of both left and right let total_byte_size = left_stats .total_byte_size - .zip(right_stats.total_byte_size) - .map(|(a, b)| 2 * a * b); - - // calculate column stats - let column_statistics = - // complete the column statistics if they are missing only on one side - match (left_stats.column_statistics, right_stats.column_statistics) { - (None, None) => None, - (None, Some(right_col_stat)) => Some(( - vec![ColumnStatistics::default(); left_col_count], - right_col_stat, - )), - (Some(left_col_stat), None) => Some(( - left_col_stat, - vec![ColumnStatistics::default(); right_col_count], - )), - (Some(left_col_stat), Some(right_col_stat)) => { - Some((left_col_stat, right_col_stat)) - } - } - .map(|(left_col_stats, right_col_stats)| { - // the null counts must be multiplied by the row counts of the other side (if defined) - // Min, max and distinct_count on the other hand are invariants. - left_col_stats.into_iter().map(|s| ColumnStatistics{ - null_count: s.null_count.zip(right_row_count).map(|(a, b)| a * b), - distinct_count: s.distinct_count, - min_value: s.min_value, - max_value: s.max_value, - }).chain( - right_col_stats.into_iter().map(|s| ColumnStatistics{ - null_count: s.null_count.zip(left_row_count).map(|(a, b)| a * b), - distinct_count: s.distinct_count, - min_value: s.min_value, - max_value: s.max_value, - })).collect() - }); + .multiply(&right_stats.total_byte_size) + .multiply(&Precision::Exact(2)); + + let left_col_stats = left_stats.column_statistics; + let right_col_stats = right_stats.column_statistics; + + // the null counts must be multiplied by the row counts of the other side (if defined) + // Min, max and distinct_count on the other hand are invariants. + let cross_join_stats = left_col_stats + .into_iter() + .map(|s| ColumnStatistics { + null_count: s.null_count.multiply(&right_row_count), + distinct_count: s.distinct_count, + min_value: s.min_value, + max_value: s.max_value, + }) + .chain(right_col_stats.into_iter().map(|s| ColumnStatistics { + null_count: s.null_count.multiply(&left_row_count), + distinct_count: s.distinct_count, + min_value: s.min_value, + max_value: s.max_value, + })) + .collect(); Statistics { - is_exact, num_rows, total_byte_size, - column_statistics, + column_statistics: cross_join_stats, } } @@ -459,6 +437,7 @@ mod tests { use super::*; use crate::common; use crate::test::build_table_scan_i32; + use datafusion_common::{assert_batches_sorted_eq, assert_contains}; use datafusion_execution::runtime_env::{RuntimeConfig, RuntimeEnv}; @@ -484,63 +463,68 @@ mod tests { let right_bytes = 27; let left = Statistics { - is_exact: true, - num_rows: Some(left_row_count), - total_byte_size: Some(left_bytes), - column_statistics: Some(vec![ + num_rows: Precision::Exact(left_row_count), + total_byte_size: Precision::Exact(left_bytes), + column_statistics: vec![ ColumnStatistics { - distinct_count: Some(5), - max_value: Some(ScalarValue::Int64(Some(21))), - min_value: Some(ScalarValue::Int64(Some(-4))), - null_count: Some(0), + distinct_count: Precision::Exact(5), + max_value: Precision::Exact(ScalarValue::Int64(Some(21))), + min_value: Precision::Exact(ScalarValue::Int64(Some(-4))), + null_count: Precision::Exact(0), }, ColumnStatistics { - distinct_count: Some(1), - max_value: Some(ScalarValue::Utf8(Some(String::from("x")))), - min_value: Some(ScalarValue::Utf8(Some(String::from("a")))), - null_count: Some(3), + distinct_count: Precision::Exact(1), + max_value: Precision::Exact(ScalarValue::Utf8(Some(String::from( + "x", + )))), + min_value: Precision::Exact(ScalarValue::Utf8(Some(String::from( + "a", + )))), + null_count: Precision::Exact(3), }, - ]), + ], }; let right = Statistics { - is_exact: true, - num_rows: Some(right_row_count), - total_byte_size: Some(right_bytes), - column_statistics: Some(vec![ColumnStatistics { - distinct_count: Some(3), - max_value: Some(ScalarValue::Int64(Some(12))), - min_value: Some(ScalarValue::Int64(Some(0))), - null_count: Some(2), - }]), + num_rows: Precision::Exact(right_row_count), + total_byte_size: Precision::Exact(right_bytes), + column_statistics: vec![ColumnStatistics { + distinct_count: Precision::Exact(3), + max_value: Precision::Exact(ScalarValue::Int64(Some(12))), + min_value: Precision::Exact(ScalarValue::Int64(Some(0))), + null_count: Precision::Exact(2), + }], }; - let result = stats_cartesian_product(left, 3, right, 2); + let result = stats_cartesian_product(left, right); let expected = Statistics { - is_exact: true, - num_rows: Some(left_row_count * right_row_count), - total_byte_size: Some(2 * left_bytes * right_bytes), - column_statistics: Some(vec![ + num_rows: Precision::Exact(left_row_count * right_row_count), + total_byte_size: Precision::Exact(2 * left_bytes * right_bytes), + column_statistics: vec![ ColumnStatistics { - distinct_count: Some(5), - max_value: Some(ScalarValue::Int64(Some(21))), - min_value: Some(ScalarValue::Int64(Some(-4))), - null_count: Some(0), + distinct_count: Precision::Exact(5), + max_value: Precision::Exact(ScalarValue::Int64(Some(21))), + min_value: Precision::Exact(ScalarValue::Int64(Some(-4))), + null_count: Precision::Exact(0), }, ColumnStatistics { - distinct_count: Some(1), - max_value: Some(ScalarValue::Utf8(Some(String::from("x")))), - min_value: Some(ScalarValue::Utf8(Some(String::from("a")))), - null_count: Some(3 * right_row_count), + distinct_count: Precision::Exact(1), + max_value: Precision::Exact(ScalarValue::Utf8(Some(String::from( + "x", + )))), + min_value: Precision::Exact(ScalarValue::Utf8(Some(String::from( + "a", + )))), + null_count: Precision::Exact(3 * right_row_count), }, ColumnStatistics { - distinct_count: Some(3), - max_value: Some(ScalarValue::Int64(Some(12))), - min_value: Some(ScalarValue::Int64(Some(0))), - null_count: Some(2 * left_row_count), + distinct_count: Precision::Exact(3), + max_value: Precision::Exact(ScalarValue::Int64(Some(12))), + min_value: Precision::Exact(ScalarValue::Int64(Some(0))), + null_count: Precision::Exact(2 * left_row_count), }, - ]), + ], }; assert_eq!(result, expected); @@ -551,63 +535,68 @@ mod tests { let left_row_count = 11; let left = Statistics { - is_exact: true, - num_rows: Some(left_row_count), - total_byte_size: Some(23), - column_statistics: Some(vec![ + num_rows: Precision::Exact(left_row_count), + total_byte_size: Precision::Exact(23), + column_statistics: vec![ ColumnStatistics { - distinct_count: Some(5), - max_value: Some(ScalarValue::Int64(Some(21))), - min_value: Some(ScalarValue::Int64(Some(-4))), - null_count: Some(0), + distinct_count: Precision::Exact(5), + max_value: Precision::Exact(ScalarValue::Int64(Some(21))), + min_value: Precision::Exact(ScalarValue::Int64(Some(-4))), + null_count: Precision::Exact(0), }, ColumnStatistics { - distinct_count: Some(1), - max_value: Some(ScalarValue::Utf8(Some(String::from("x")))), - min_value: Some(ScalarValue::Utf8(Some(String::from("a")))), - null_count: Some(3), + distinct_count: Precision::Exact(1), + max_value: Precision::Exact(ScalarValue::Utf8(Some(String::from( + "x", + )))), + min_value: Precision::Exact(ScalarValue::Utf8(Some(String::from( + "a", + )))), + null_count: Precision::Exact(3), }, - ]), + ], }; let right = Statistics { - is_exact: true, - num_rows: None, // not defined! - total_byte_size: None, // not defined! - column_statistics: Some(vec![ColumnStatistics { - distinct_count: Some(3), - max_value: Some(ScalarValue::Int64(Some(12))), - min_value: Some(ScalarValue::Int64(Some(0))), - null_count: Some(2), - }]), + num_rows: Precision::Absent, + total_byte_size: Precision::Absent, + column_statistics: vec![ColumnStatistics { + distinct_count: Precision::Exact(3), + max_value: Precision::Exact(ScalarValue::Int64(Some(12))), + min_value: Precision::Exact(ScalarValue::Int64(Some(0))), + null_count: Precision::Exact(2), + }], }; - let result = stats_cartesian_product(left, 3, right, 2); + let result = stats_cartesian_product(left, right); let expected = Statistics { - is_exact: true, - num_rows: None, - total_byte_size: None, - column_statistics: Some(vec![ + num_rows: Precision::Absent, + total_byte_size: Precision::Absent, + column_statistics: vec![ ColumnStatistics { - distinct_count: Some(5), - max_value: Some(ScalarValue::Int64(Some(21))), - min_value: Some(ScalarValue::Int64(Some(-4))), - null_count: None, // we don't know the row count on the right + distinct_count: Precision::Exact(5), + max_value: Precision::Exact(ScalarValue::Int64(Some(21))), + min_value: Precision::Exact(ScalarValue::Int64(Some(-4))), + null_count: Precision::Absent, // we don't know the row count on the right }, ColumnStatistics { - distinct_count: Some(1), - max_value: Some(ScalarValue::Utf8(Some(String::from("x")))), - min_value: Some(ScalarValue::Utf8(Some(String::from("a")))), - null_count: None, // we don't know the row count on the right + distinct_count: Precision::Exact(1), + max_value: Precision::Exact(ScalarValue::Utf8(Some(String::from( + "x", + )))), + min_value: Precision::Exact(ScalarValue::Utf8(Some(String::from( + "a", + )))), + null_count: Precision::Absent, // we don't know the row count on the right }, ColumnStatistics { - distinct_count: Some(3), - max_value: Some(ScalarValue::Int64(Some(12))), - min_value: Some(ScalarValue::Int64(Some(0))), - null_count: Some(2 * left_row_count), + distinct_count: Precision::Exact(3), + max_value: Precision::Exact(ScalarValue::Int64(Some(12))), + min_value: Precision::Exact(ScalarValue::Int64(Some(0))), + null_count: Precision::Exact(2 * left_row_count), }, - ]), + ], }; assert_eq!(result, expected); diff --git a/datafusion/physical-plan/src/joins/hash_join.rs b/datafusion/physical-plan/src/joins/hash_join.rs index 8e204634f3d9..4fd8882c0178 100644 --- a/datafusion/physical-plan/src/joins/hash_join.rs +++ b/datafusion/physical-plan/src/joins/hash_join.rs @@ -493,7 +493,7 @@ impl ExecutionPlan for HashJoinExec { Some(self.metrics.clone_inner()) } - fn statistics(&self) -> Statistics { + fn statistics(&self) -> Result { // TODO stats: it is not possible in general to know the output size of joins // There are some special cases though, for example: // - `A LEFT JOIN B ON A.col=B.col` with `COUNT_DISTINCT(B.col)=COUNT(B.col)` @@ -502,6 +502,7 @@ impl ExecutionPlan for HashJoinExec { self.right.clone(), self.on.clone(), &self.join_type, + &self.schema, ) } } diff --git a/datafusion/physical-plan/src/joins/nested_loop_join.rs b/datafusion/physical-plan/src/joins/nested_loop_join.rs index c49c16dba313..944efb47f429 100644 --- a/datafusion/physical-plan/src/joins/nested_loop_join.rs +++ b/datafusion/physical-plan/src/joins/nested_loop_join.rs @@ -19,6 +19,12 @@ //! The nested loop join can execute in parallel by partitions and it is //! determined by the [`JoinType`]. +use std::any::Any; +use std::fmt::Formatter; +use std::sync::Arc; +use std::task::Poll; + +use crate::coalesce_batches::concat_batches; use crate::joins::utils::{ append_right_indices, apply_join_filter_to_indices, build_batch_from_indices, build_join_schema, check_join_is_valid, combine_join_equivalence_properties, @@ -32,26 +38,20 @@ use crate::{ DisplayAs, DisplayFormatType, Distribution, ExecutionPlan, Partitioning, RecordBatchStream, SendableRecordBatchStream, }; + use arrow::array::{ BooleanBufferBuilder, UInt32Array, UInt32Builder, UInt64Array, UInt64Builder, }; use arrow::datatypes::{Schema, SchemaRef}; use arrow::record_batch::RecordBatch; use arrow::util::bit_util; -use datafusion_common::{exec_err, DataFusionError, Statistics}; -use datafusion_execution::memory_pool::MemoryReservation; +use datafusion_common::{exec_err, DataFusionError, Result, Statistics}; +use datafusion_execution::memory_pool::{MemoryConsumer, MemoryReservation}; +use datafusion_execution::TaskContext; use datafusion_expr::JoinType; use datafusion_physical_expr::{EquivalenceProperties, PhysicalSortExpr}; -use futures::{ready, Stream, StreamExt, TryStreamExt}; -use std::any::Any; -use std::fmt::Formatter; -use std::sync::Arc; -use std::task::Poll; -use crate::coalesce_batches::concat_batches; -use datafusion_common::Result; -use datafusion_execution::memory_pool::MemoryConsumer; -use datafusion_execution::TaskContext; +use futures::{ready, Stream, StreamExt, TryStreamExt}; /// Data of the inner table side type JoinLeftData = (RecordBatch, MemoryReservation); @@ -282,12 +282,13 @@ impl ExecutionPlan for NestedLoopJoinExec { Some(self.metrics.clone_inner()) } - fn statistics(&self) -> Statistics { + fn statistics(&self) -> Result { estimate_join_statistics( self.left.clone(), self.right.clone(), vec![], &self.join_type, + &self.schema, ) } } @@ -739,21 +740,21 @@ impl RecordBatchStream for NestedLoopJoinStream { #[cfg(test)] mod tests { + use std::sync::Arc; + use super::*; + use crate::joins::utils::JoinSide; use crate::{ common, expressions::Column, memory::MemoryExec, repartition::RepartitionExec, test::build_table_i32, }; + use arrow::datatypes::{DataType, Field}; + use datafusion_common::{assert_batches_sorted_eq, assert_contains, ScalarValue}; use datafusion_execution::runtime_env::{RuntimeConfig, RuntimeEnv}; use datafusion_expr::Operator; - use datafusion_physical_expr::expressions::BinaryExpr; - - use crate::joins::utils::JoinSide; - use datafusion_common::{assert_batches_sorted_eq, assert_contains, ScalarValue}; - use datafusion_physical_expr::expressions::Literal; + use datafusion_physical_expr::expressions::{BinaryExpr, Literal}; use datafusion_physical_expr::PhysicalExpr; - use std::sync::Arc; fn build_table( a: (&str, &Vec), diff --git a/datafusion/physical-plan/src/joins/sort_merge_join.rs b/datafusion/physical-plan/src/joins/sort_merge_join.rs index 4de723ab73ea..61ac864bf20d 100644 --- a/datafusion/physical-plan/src/joins/sort_merge_join.rs +++ b/datafusion/physical-plan/src/joins/sort_merge_join.rs @@ -30,8 +30,7 @@ use std::pin::Pin; use std::sync::Arc; use std::task::{Context, Poll}; -use crate::expressions::Column; -use crate::expressions::PhysicalSortExpr; +use crate::expressions::{Column, PhysicalSortExpr}; use crate::joins::utils::{ build_join_schema, calculate_join_output_ordering, check_join_is_valid, combine_join_equivalence_properties, combine_join_ordering_equivalence_properties, @@ -381,7 +380,7 @@ impl ExecutionPlan for SortMergeJoinExec { Some(self.metrics.clone_inner()) } - fn statistics(&self) -> Statistics { + fn statistics(&self) -> Result { // TODO stats: it is not possible in general to know the output size of joins // There are some special cases though, for example: // - `A LEFT JOIN B ON A.col=B.col` with `COUNT_DISTINCT(B.col)=COUNT(B.col)` @@ -390,6 +389,7 @@ impl ExecutionPlan for SortMergeJoinExec { self.right.clone(), self.on.clone(), &self.join_type, + &self.schema, ) } } @@ -1397,24 +1397,23 @@ fn is_join_arrays_equal( mod tests { use std::sync::Arc; - use arrow::array::{Date32Array, Date64Array, Int32Array}; - use arrow::compute::SortOptions; - use arrow::datatypes::{DataType, Field, Schema}; - use arrow::record_batch::RecordBatch; - use datafusion_execution::config::SessionConfig; - use datafusion_execution::TaskContext; - use crate::expressions::Column; use crate::joins::utils::JoinOn; use crate::joins::SortMergeJoinExec; use crate::memory::MemoryExec; use crate::test::build_table_i32; use crate::{common, ExecutionPlan}; - use datafusion_common::Result; + + use arrow::array::{Date32Array, Date64Array, Int32Array}; + use arrow::compute::SortOptions; + use arrow::datatypes::{DataType, Field, Schema}; + use arrow::record_batch::RecordBatch; use datafusion_common::{ - assert_batches_eq, assert_batches_sorted_eq, assert_contains, JoinType, + assert_batches_eq, assert_batches_sorted_eq, assert_contains, JoinType, Result, }; + use datafusion_execution::config::SessionConfig; use datafusion_execution::runtime_env::{RuntimeConfig, RuntimeEnv}; + use datafusion_execution::TaskContext; fn build_table( a: (&str, &Vec), diff --git a/datafusion/physical-plan/src/joins/symmetric_hash_join.rs b/datafusion/physical-plan/src/joins/symmetric_hash_join.rs index e6eb5dd69582..64128b6e3a12 100644 --- a/datafusion/physical-plan/src/joins/symmetric_hash_join.rs +++ b/datafusion/physical-plan/src/joins/symmetric_hash_join.rs @@ -18,11 +18,11 @@ //! This file implements the symmetric hash join algorithm with range-based //! data pruning to join two (potentially infinite) streams. //! -//! A [SymmetricHashJoinExec] plan takes two children plan (with appropriate +//! A [`SymmetricHashJoinExec`] plan takes two children plan (with appropriate //! output ordering) and produces the join output according to the given join //! type and other options. //! -//! This plan uses the [OneSideHashJoiner] object to facilitate join calculations +//! This plan uses the [`OneSideHashJoiner`] object to facilitate join calculations //! for both its children. use std::fmt; @@ -38,23 +38,19 @@ use crate::joins::hash_join_utils::{ calculate_filter_expr_intervals, combine_two_batches, convert_sort_expr_with_filter_schema, get_pruning_anti_indices, get_pruning_semi_indices, record_visited_indices, PruningJoinHashMap, + SortedFilterExpr, +}; +use crate::joins::utils::{ + build_batch_from_indices, build_join_schema, check_join_is_valid, + combine_join_equivalence_properties, partitioned_join_output_partitioning, + prepare_sorted_exprs, ColumnIndex, JoinFilter, JoinOn, JoinSide, }; -use crate::joins::StreamJoinPartitionMode; -use crate::DisplayAs; use crate::{ - expressions::Column, - expressions::PhysicalSortExpr, - joins::{ - hash_join_utils::SortedFilterExpr, - utils::{ - build_batch_from_indices, build_join_schema, check_join_is_valid, - combine_join_equivalence_properties, partitioned_join_output_partitioning, - ColumnIndex, JoinFilter, JoinOn, JoinSide, - }, - }, + expressions::{Column, PhysicalSortExpr}, + joins::StreamJoinPartitionMode, metrics::{self, ExecutionPlanMetricsSet, MetricBuilder, MetricsSet}, - DisplayFormatType, Distribution, EquivalenceProperties, ExecutionPlan, Partitioning, - RecordBatchStream, SendableRecordBatchStream, Statistics, + DisplayAs, DisplayFormatType, Distribution, EquivalenceProperties, ExecutionPlan, + Partitioning, RecordBatchStream, SendableRecordBatchStream, Statistics, }; use arrow::array::{ArrowPrimitiveType, NativeAdapter, PrimitiveArray, PrimitiveBuilder}; @@ -62,13 +58,11 @@ use arrow::compute::concat_batches; use arrow::datatypes::{Schema, SchemaRef}; use arrow::record_batch::RecordBatch; use datafusion_common::utils::bisect; -use datafusion_common::{internal_err, plan_err, JoinType}; -use datafusion_common::{DataFusionError, Result}; +use datafusion_common::{internal_err, plan_err, DataFusionError, JoinType, Result}; use datafusion_execution::memory_pool::MemoryConsumer; use datafusion_execution::TaskContext; use datafusion_physical_expr::intervals::ExprIntervalGraph; -use crate::joins::utils::prepare_sorted_exprs; use ahash::RandomState; use futures::stream::{select, BoxStream}; use futures::{Stream, StreamExt}; @@ -467,9 +461,9 @@ impl ExecutionPlan for SymmetricHashJoinExec { Some(self.metrics.clone_inner()) } - fn statistics(&self) -> Statistics { + fn statistics(&self) -> Result { // TODO stats: it is not possible in general to know the output size of joins - Statistics::default() + Ok(Statistics::new_unknown(&self.schema())) } fn execute( @@ -1204,17 +1198,11 @@ impl SymmetricHashJoinStream { #[cfg(test)] mod tests { - use super::*; - use arrow::compute::SortOptions; - use arrow::datatypes::{DataType, Field, IntervalUnit, Schema, TimeUnit}; - use datafusion_execution::config::SessionConfig; - use rstest::*; - - use datafusion_expr::Operator; - use datafusion_physical_expr::expressions::{binary, col, Column}; + use std::collections::HashMap; + use std::sync::Mutex; + use super::*; use crate::joins::hash_join_utils::tests::complicated_filter; - use crate::joins::test_utils::{ build_sides_record_batches, compare_batches, create_memory_table, join_expr_tests_fixture_f64, join_expr_tests_fixture_i32, @@ -1222,11 +1210,16 @@ mod tests { partitioned_sym_join_with_filter, split_record_batches, }; - const TABLE_SIZE: i32 = 30; + use arrow::compute::SortOptions; + use arrow::datatypes::{DataType, Field, IntervalUnit, Schema, TimeUnit}; + use datafusion_execution::config::SessionConfig; + use datafusion_expr::Operator; + use datafusion_physical_expr::expressions::{binary, col, Column}; use once_cell::sync::Lazy; - use std::collections::HashMap; - use std::sync::Mutex; + use rstest::*; + + const TABLE_SIZE: i32 = 30; type TableKey = (i32, i32, usize); // (cardinality.0, cardinality.1, batch_size) type TableValue = (Vec, Vec); // (left, right) diff --git a/datafusion/physical-plan/src/joins/utils.rs b/datafusion/physical-plan/src/joins/utils.rs index 1dbcf7617385..c523a8fe8135 100644 --- a/datafusion/physical-plan/src/joins/utils.rs +++ b/datafusion/physical-plan/src/joins/utils.rs @@ -17,7 +17,6 @@ //! Join related functionality used both on logical and physical plans -use std::cmp::max; use std::collections::HashSet; use std::fmt::{Display, Formatter}; use std::future::Future; @@ -25,10 +24,11 @@ use std::sync::Arc; use std::task::{Context, Poll}; use std::usize; +use crate::joins::hash_join_utils::{build_filter_input_order, SortedFilterExpr}; use crate::metrics::{self, ExecutionPlanMetricsSet, MetricBuilder}; -use crate::SchemaRef; use crate::{ - ColumnStatistics, EquivalenceProperties, ExecutionPlan, Partitioning, Statistics, + ColumnStatistics, EquivalenceProperties, ExecutionPlan, Partitioning, SchemaRef, + Statistics, }; use arrow::array::{ @@ -39,21 +39,21 @@ use arrow::compute; use arrow::datatypes::{Field, Schema, SchemaBuilder}; use arrow::record_batch::{RecordBatch, RecordBatchOptions}; use datafusion_common::cast::as_boolean_array; +use datafusion_common::stats::Precision; use datafusion_common::tree_node::{Transformed, TreeNode}; use datafusion_common::{ exec_err, plan_datafusion_err, plan_err, DataFusionError, JoinType, Result, - ScalarValue, SharedResult, + SharedResult, }; use datafusion_physical_expr::expressions::Column; +use datafusion_physical_expr::intervals::{ExprIntervalGraph, Interval, IntervalBound}; +use datafusion_physical_expr::utils::merge_vectors; use datafusion_physical_expr::{ add_offset_to_lex_ordering, EquivalentClass, LexOrdering, LexOrderingRef, OrderingEquivalenceProperties, OrderingEquivalentClass, PhysicalExpr, PhysicalSortExpr, }; -use crate::joins::hash_join_utils::{build_filter_input_order, SortedFilterExpr}; -use datafusion_physical_expr::intervals::ExprIntervalGraph; -use datafusion_physical_expr::utils::merge_vectors; use futures::future::{BoxFuture, Shared}; use futures::{ready, FutureExt}; use parking_lot::Mutex; @@ -723,21 +723,21 @@ pub(crate) fn estimate_join_statistics( right: Arc, on: JoinOn, join_type: &JoinType, -) -> Statistics { - let left_stats = left.statistics(); - let right_stats = right.statistics(); + schema: &Schema, +) -> Result { + let left_stats = left.statistics()?; + let right_stats = right.statistics()?; let join_stats = estimate_join_cardinality(join_type, left_stats, right_stats, &on); let (num_rows, column_statistics) = match join_stats { - Some(stats) => (Some(stats.num_rows), Some(stats.column_statistics)), - None => (None, None), + Some(stats) => (Precision::Inexact(stats.num_rows), stats.column_statistics), + None => (Precision::Absent, Statistics::unknown_column(schema)), }; - Statistics { + Ok(Statistics { num_rows, - total_byte_size: None, + total_byte_size: Precision::Absent, column_statistics, - is_exact: false, - } + }) } // Estimate the cardinality for the given join with input statistics. @@ -749,29 +749,27 @@ fn estimate_join_cardinality( ) -> Option { match join_type { JoinType::Inner | JoinType::Left | JoinType::Right | JoinType::Full => { - let left_num_rows = left_stats.num_rows?; - let right_num_rows = right_stats.num_rows?; - - // Take the left_col_stats and right_col_stats using the index - // obtained from index() method of the each element of 'on'. - let all_left_col_stats = left_stats.column_statistics?; - let all_right_col_stats = right_stats.column_statistics?; let (left_col_stats, right_col_stats) = on .iter() .map(|(left, right)| { ( - all_left_col_stats[left.index()].clone(), - all_right_col_stats[right.index()].clone(), + left_stats.column_statistics[left.index()].clone(), + right_stats.column_statistics[right.index()].clone(), ) }) .unzip::<_, _, Vec<_>, Vec<_>>(); let ij_cardinality = estimate_inner_join_cardinality( - left_num_rows, - right_num_rows, - left_col_stats, - right_col_stats, - left_stats.is_exact && right_stats.is_exact, + Statistics { + num_rows: left_stats.num_rows.clone(), + total_byte_size: Precision::Absent, + column_statistics: left_col_stats, + }, + Statistics { + num_rows: right_stats.num_rows.clone(), + total_byte_size: Precision::Absent, + column_statistics: right_col_stats, + }, )?; // The cardinality for inner join can also be used to estimate @@ -780,25 +778,25 @@ fn estimate_join_cardinality( // joins (so that we don't underestimate the cardinality). let cardinality = match join_type { JoinType::Inner => ij_cardinality, - JoinType::Left => max(ij_cardinality, left_num_rows), - JoinType::Right => max(ij_cardinality, right_num_rows), - JoinType::Full => { - max(ij_cardinality, left_num_rows) - + max(ij_cardinality, right_num_rows) - - ij_cardinality - } + JoinType::Left => ij_cardinality.max(&left_stats.num_rows), + JoinType::Right => ij_cardinality.max(&right_stats.num_rows), + JoinType::Full => ij_cardinality + .max(&left_stats.num_rows) + .add(&ij_cardinality.max(&right_stats.num_rows)) + .sub(&ij_cardinality), _ => unreachable!(), }; Some(PartialJoinStatistics { - num_rows: cardinality, + num_rows: *cardinality.get_value()?, // We don't do anything specific here, just combine the existing // statistics which might yield subpar results (although it is // true, esp regarding min/max). For a better estimation, we need // filter selectivity analysis first. - column_statistics: all_left_col_stats + column_statistics: left_stats + .column_statistics .into_iter() - .chain(all_right_col_stats) + .chain(right_stats.column_statistics) .collect(), }) } @@ -815,30 +813,47 @@ fn estimate_join_cardinality( /// a very conservative implementation that can quickly give up if there is not /// enough input statistics. fn estimate_inner_join_cardinality( - left_num_rows: usize, - right_num_rows: usize, - left_col_stats: Vec, - right_col_stats: Vec, - is_exact: bool, -) -> Option { + left_stats: Statistics, + right_stats: Statistics, +) -> Option> { // The algorithm here is partly based on the non-histogram selectivity estimation // from Spark's Catalyst optimizer. - - let mut join_selectivity = None; - for (left_stat, right_stat) in left_col_stats.iter().zip(right_col_stats.iter()) { - if (left_stat.min_value.clone()? > right_stat.max_value.clone()?) - || (left_stat.max_value.clone()? < right_stat.min_value.clone()?) - { - // If there is no overlap in any of the join columns, that means the join - // itself is disjoint and the cardinality is 0. Though we can only assume - // this when the statistics are exact (since it is a very strong assumption). - return if is_exact { Some(0) } else { None }; + let mut join_selectivity = Precision::Absent; + for (left_stat, right_stat) in left_stats + .column_statistics + .iter() + .zip(right_stats.column_statistics.iter()) + { + // If there is no overlap in any of the join columns, this means the join + // itself is disjoint and the cardinality is 0. Though we can only assume + // this when the statistics are exact (since it is a very strong assumption). + if left_stat.min_value.get_value()? > right_stat.max_value.get_value()? { + return Some( + if left_stat.min_value.is_exact().unwrap_or(false) + && right_stat.max_value.is_exact().unwrap_or(false) + { + Precision::Exact(0) + } else { + Precision::Inexact(0) + }, + ); + } + if left_stat.max_value.get_value()? < right_stat.min_value.get_value()? { + return Some( + if left_stat.max_value.is_exact().unwrap_or(false) + && right_stat.min_value.is_exact().unwrap_or(false) + { + Precision::Exact(0) + } else { + Precision::Inexact(0) + }, + ); } - let left_max_distinct = max_distinct_count(left_num_rows, left_stat.clone()); - let right_max_distinct = max_distinct_count(right_num_rows, right_stat.clone()); - let max_distinct = max(left_max_distinct, right_max_distinct); - if max_distinct > join_selectivity { + let left_max_distinct = max_distinct_count(&left_stats.num_rows, left_stat)?; + let right_max_distinct = max_distinct_count(&right_stats.num_rows, right_stat)?; + let max_distinct = left_max_distinct.max(&right_max_distinct); + if max_distinct.get_value().is_some() { // Seems like there are a few implementations of this algorithm that implement // exponential decay for the selectivity (like Hive's Optiq Optimizer). Needs // further exploration. @@ -849,9 +864,14 @@ fn estimate_inner_join_cardinality( // With the assumption that the smaller input's domain is generally represented in the bigger // input's domain, we can estimate the inner join's cardinality by taking the cartesian product // of the two inputs and normalizing it by the selectivity factor. + let left_num_rows = left_stats.num_rows.get_value()?; + let right_num_rows = right_stats.num_rows.get_value()?; match join_selectivity { - Some(selectivity) if selectivity > 0 => { - Some((left_num_rows * right_num_rows) / selectivity) + Precision::Exact(value) if value > 0 => { + Some(Precision::Exact((left_num_rows * right_num_rows) / value)) + } + Precision::Inexact(value) if value > 0 => { + Some(Precision::Inexact((left_num_rows * right_num_rows) / value)) } // Since we don't have any information about the selectivity (which is derived // from the number of distinct rows information) we can give up here for now. @@ -867,44 +887,46 @@ fn estimate_inner_join_cardinality( /// If distinct_count is available, uses it directly. If the column numeric, and /// has min/max values, then they might be used as a fallback option. Otherwise, /// returns None. -fn max_distinct_count(num_rows: usize, stats: ColumnStatistics) -> Option { - match (stats.distinct_count, stats.max_value, stats.min_value) { - (Some(_), _, _) => stats.distinct_count, +fn max_distinct_count( + num_rows: &Precision, + stats: &ColumnStatistics, +) -> Option> { + match ( + &stats.distinct_count, + stats.max_value.get_value(), + stats.min_value.get_value(), + ) { + (Precision::Exact(_), _, _) | (Precision::Inexact(_), _, _) => { + Some(stats.distinct_count.clone()) + } (_, Some(max), Some(min)) => { - // Note that float support is intentionally omitted here, since the computation - // of a range between two float values is not trivial and the result would be - // highly inaccurate. - let numeric_range = get_int_range(min, max)?; + let numeric_range = Interval::new( + IntervalBound::new(min.clone(), false), + IntervalBound::new(max.clone(), false), + ) + .cardinality() + .ok() + .flatten()? as usize; // The number can never be greater than the number of rows we have (minus // the nulls, since they don't count as distinct values). - let ceiling = num_rows - stats.null_count.unwrap_or(0); - Some(numeric_range.min(ceiling)) + let ceiling = + num_rows.get_value()? - stats.null_count.get_value().unwrap_or(&0); + Some( + if num_rows.is_exact().unwrap_or(false) + && stats.max_value.is_exact().unwrap_or(false) + && stats.min_value.is_exact().unwrap_or(false) + { + Precision::Exact(numeric_range.min(ceiling)) + } else { + Precision::Inexact(numeric_range.min(ceiling)) + }, + ) } _ => None, } } -/// Return the numeric range between the given min and max values. -fn get_int_range(min: ScalarValue, max: ScalarValue) -> Option { - let delta = &max.sub(&min).ok()?; - match delta { - ScalarValue::Int8(Some(delta)) if *delta >= 0 => Some(*delta as usize), - ScalarValue::Int16(Some(delta)) if *delta >= 0 => Some(*delta as usize), - ScalarValue::Int32(Some(delta)) if *delta >= 0 => Some(*delta as usize), - ScalarValue::Int64(Some(delta)) if *delta >= 0 => Some(*delta as usize), - ScalarValue::UInt8(Some(delta)) => Some(*delta as usize), - ScalarValue::UInt16(Some(delta)) => Some(*delta as usize), - ScalarValue::UInt32(Some(delta)) => Some(*delta as usize), - ScalarValue::UInt64(Some(delta)) => Some(*delta as usize), - _ => None, - } - // The delta (directly) is not the real range, since it does not include the - // first term. - // E.g. (min=2, max=4) -> (4 - 2) -> 2, but the actual result should be 3 (1, 2, 3). - .map(|open_ended_range| open_ended_range + 1) -} - enum OnceFutState { Pending(OnceFutPending), Ready(SharedResult>), @@ -1380,13 +1402,15 @@ pub fn prepare_sorted_exprs( #[cfg(test)] mod tests { + use std::pin::Pin; + use super::*; - use arrow::datatypes::Fields; - use arrow::error::Result as ArrowResult; - use arrow::{datatypes::DataType, error::ArrowError}; + + use arrow::datatypes::{DataType, Fields}; + use arrow::error::{ArrowError, Result as ArrowResult}; use arrow_schema::SortOptions; + use datafusion_common::ScalarValue; - use std::pin::Pin; fn check(left: &[Column], right: &[Column], on: &[(Column, Column)]) -> Result<()> { let left = left @@ -1539,14 +1563,18 @@ mod tests { fn create_stats( num_rows: Option, - column_stats: Option>, + column_stats: Vec, is_exact: bool, ) -> Statistics { Statistics { - num_rows, + num_rows: if is_exact { + num_rows.map(Precision::Exact) + } else { + num_rows.map(Precision::Inexact) + } + .unwrap_or(Precision::Absent), column_statistics: column_stats, - is_exact, - ..Default::default() + total_byte_size: Precision::Absent, } } @@ -1556,9 +1584,15 @@ mod tests { distinct_count: Option, ) -> ColumnStatistics { ColumnStatistics { - distinct_count, - min_value: min.map(|size| ScalarValue::Int64(Some(size))), - max_value: max.map(|size| ScalarValue::Int64(Some(size))), + distinct_count: distinct_count + .map(Precision::Inexact) + .unwrap_or(Precision::Absent), + min_value: min + .map(|size| Precision::Inexact(ScalarValue::from(size))) + .unwrap_or(Precision::Absent), + max_value: max + .map(|size| Precision::Inexact(ScalarValue::from(size))) + .unwrap_or(Precision::Absent), ..Default::default() } } @@ -1570,7 +1604,7 @@ mod tests { // over the expected output (since it depends on join type to join type). #[test] fn test_inner_join_cardinality_single_column() -> Result<()> { - let cases: Vec<(PartialStats, PartialStats, Option)> = vec![ + let cases: Vec<(PartialStats, PartialStats, Option>)> = vec![ // ----------------------------------------------------------------------------- // | left(rows, min, max, distinct), right(rows, min, max, distinct), expected | // ----------------------------------------------------------------------------- @@ -1582,70 +1616,70 @@ mod tests { ( (10, Some(1), Some(10), None), (10, Some(1), Some(10), None), - Some(10), + Some(Precision::Inexact(10)), ), // range(left) > range(right) ( (10, Some(6), Some(10), None), (10, Some(8), Some(10), None), - Some(20), + Some(Precision::Inexact(20)), ), // range(right) > range(left) ( (10, Some(8), Some(10), None), (10, Some(6), Some(10), None), - Some(20), + Some(Precision::Inexact(20)), ), // range(left) > len(left), range(right) > len(right) ( (10, Some(1), Some(15), None), (20, Some(1), Some(40), None), - Some(10), + Some(Precision::Inexact(10)), ), // When we have distinct count. ( (10, Some(1), Some(10), Some(10)), (10, Some(1), Some(10), Some(10)), - Some(10), + Some(Precision::Inexact(10)), ), // distinct(left) > distinct(right) ( (10, Some(1), Some(10), Some(5)), (10, Some(1), Some(10), Some(2)), - Some(20), + Some(Precision::Inexact(20)), ), // distinct(right) > distinct(left) ( (10, Some(1), Some(10), Some(2)), (10, Some(1), Some(10), Some(5)), - Some(20), + Some(Precision::Inexact(20)), ), // min(left) < 0 (range(left) > range(right)) ( (10, Some(-5), Some(5), None), (10, Some(1), Some(5), None), - Some(10), + Some(Precision::Inexact(10)), ), // min(right) < 0, max(right) < 0 (range(right) > range(left)) ( (10, Some(-25), Some(-20), None), (10, Some(-25), Some(-15), None), - Some(10), + Some(Precision::Inexact(10)), ), // range(left) < 0, range(right) >= 0 // (there isn't a case where both left and right ranges are negative // so one of them is always going to work, this just proves negative // ranges with bigger absolute values are not are not accidentally used). ( - (10, Some(10), Some(0), None), + (10, Some(-10), Some(0), None), (10, Some(0), Some(10), Some(5)), - Some(20), // It would have been ten if we have used abs(range(left)) + Some(Precision::Inexact(10)), ), // range(left) = 1, range(right) = 1 ( (10, Some(1), Some(1), None), (10, Some(1), Some(1), None), - Some(100), + Some(Precision::Inexact(100)), ), // // Edge cases @@ -1670,22 +1704,12 @@ mod tests { ( (10, Some(0), Some(10), None), (10, Some(11), Some(20), None), - None, + Some(Precision::Inexact(0)), ), ( (10, Some(11), Some(20), None), (10, Some(0), Some(10), None), - None, - ), - ( - (10, Some(5), Some(10), Some(10)), - (10, Some(11), Some(3), Some(10)), - None, - ), - ( - (10, Some(10), Some(5), Some(10)), - (10, Some(3), Some(7), Some(10)), - None, + Some(Precision::Inexact(0)), ), // distinct(left) = 0, distinct(right) = 0 ( @@ -1709,13 +1733,18 @@ mod tests { assert_eq!( estimate_inner_join_cardinality( - left_num_rows, - right_num_rows, - left_col_stats.clone(), - right_col_stats.clone(), - false, + Statistics { + num_rows: Precision::Inexact(left_num_rows), + total_byte_size: Precision::Absent, + column_statistics: left_col_stats.clone(), + }, + Statistics { + num_rows: Precision::Inexact(right_num_rows), + total_byte_size: Precision::Absent, + column_statistics: right_col_stats.clone(), + }, ), - expected_cardinality + expected_cardinality.clone() ); // We should also be able to use join_cardinality to get the same results @@ -1723,18 +1752,22 @@ mod tests { let join_on = vec![(Column::new("a", 0), Column::new("b", 0))]; let partial_join_stats = estimate_join_cardinality( &join_type, - create_stats(Some(left_num_rows), Some(left_col_stats.clone()), false), - create_stats(Some(right_num_rows), Some(right_col_stats.clone()), false), + create_stats(Some(left_num_rows), left_col_stats.clone(), false), + create_stats(Some(right_num_rows), right_col_stats.clone(), false), &join_on, ); assert_eq!( - partial_join_stats.clone().map(|s| s.num_rows), - expected_cardinality + partial_join_stats + .clone() + .map(|s| Precision::Inexact(s.num_rows)), + expected_cardinality.clone() ); assert_eq!( partial_join_stats.map(|s| s.column_statistics), - expected_cardinality.map(|_| [left_col_stats, right_col_stats].concat()) + expected_cardinality + .clone() + .map(|_| [left_col_stats, right_col_stats].concat()) ); } Ok(()) @@ -1756,13 +1789,18 @@ mod tests { // count is 200, so we are going to pick it. assert_eq!( estimate_inner_join_cardinality( - 400, - 400, - left_col_stats, - right_col_stats, - false + Statistics { + num_rows: Precision::Inexact(400), + total_byte_size: Precision::Absent, + column_statistics: left_col_stats, + }, + Statistics { + num_rows: Precision::Inexact(400), + total_byte_size: Precision::Absent, + column_statistics: right_col_stats, + }, ), - Some((400 * 400) / 200) + Some(Precision::Inexact((400 * 400) / 200)) ); Ok(()) } @@ -1770,26 +1808,31 @@ mod tests { #[test] fn test_inner_join_cardinality_decimal_range() -> Result<()> { let left_col_stats = vec![ColumnStatistics { - distinct_count: None, - min_value: Some(ScalarValue::Decimal128(Some(32500), 14, 4)), - max_value: Some(ScalarValue::Decimal128(Some(35000), 14, 4)), + distinct_count: Precision::Absent, + min_value: Precision::Inexact(ScalarValue::Decimal128(Some(32500), 14, 4)), + max_value: Precision::Inexact(ScalarValue::Decimal128(Some(35000), 14, 4)), ..Default::default() }]; let right_col_stats = vec![ColumnStatistics { - distinct_count: None, - min_value: Some(ScalarValue::Decimal128(Some(33500), 14, 4)), - max_value: Some(ScalarValue::Decimal128(Some(34000), 14, 4)), + distinct_count: Precision::Absent, + min_value: Precision::Inexact(ScalarValue::Decimal128(Some(33500), 14, 4)), + max_value: Precision::Inexact(ScalarValue::Decimal128(Some(34000), 14, 4)), ..Default::default() }]; assert_eq!( estimate_inner_join_cardinality( - 100, - 100, - left_col_stats, - right_col_stats, - false + Statistics { + num_rows: Precision::Inexact(100), + total_byte_size: Precision::Absent, + column_statistics: left_col_stats, + }, + Statistics { + num_rows: Precision::Inexact(100), + total_byte_size: Precision::Absent, + column_statistics: right_col_stats, + }, ), None ); @@ -1836,8 +1879,8 @@ mod tests { let partial_join_stats = estimate_join_cardinality( &join_type, - create_stats(Some(1000), Some(left_col_stats.clone()), false), - create_stats(Some(2000), Some(right_col_stats.clone()), false), + create_stats(Some(1000), left_col_stats.clone(), false), + create_stats(Some(2000), right_col_stats.clone(), false), &join_on, ) .unwrap(); @@ -1901,8 +1944,8 @@ mod tests { for (join_type, expected_num_rows) in cases { let partial_join_stats = estimate_join_cardinality( &join_type, - create_stats(Some(1000), Some(left_col_stats.clone()), true), - create_stats(Some(2000), Some(right_col_stats.clone()), true), + create_stats(Some(1000), left_col_stats.clone(), true), + create_stats(Some(2000), right_col_stats.clone(), true), &join_on, ) .unwrap(); diff --git a/datafusion/physical-plan/src/lib.rs b/datafusion/physical-plan/src/lib.rs index 30ac2cb98366..d7987ba95abf 100644 --- a/datafusion/physical-plan/src/lib.rs +++ b/datafusion/physical-plan/src/lib.rs @@ -17,39 +17,78 @@ //! Traits for physical query plan, supporting parallel execution for partitioned relations. -mod topk; -mod visitor; -pub use self::metrics::Metric; -use self::metrics::MetricsSet; -use self::{ - coalesce_partitions::CoalescePartitionsExec, display::DisplayableExecutionPlan, -}; -pub use datafusion_common::{internal_err, ColumnStatistics, Statistics}; -use datafusion_common::{plan_err, Result}; -use datafusion_physical_expr::PhysicalSortExpr; -pub use topk::TopK; -pub use visitor::{accept, visit_execution_plan, ExecutionPlanVisitor}; +use std::any::Any; +use std::fmt::Debug; +use std::sync::Arc; + +use crate::coalesce_partitions::CoalescePartitionsExec; +use crate::display::DisplayableExecutionPlan; +use crate::metrics::MetricsSet; +use crate::repartition::RepartitionExec; +use crate::sorts::sort_preserving_merge::SortPreservingMergeExec; use arrow::datatypes::SchemaRef; use arrow::record_batch::RecordBatch; - +use datafusion_common::tree_node::Transformed; use datafusion_common::utils::DataPtr; -pub use datafusion_expr::Accumulator; -pub use datafusion_expr::ColumnarValue; +use datafusion_common::{plan_err, DataFusionError, Result}; +use datafusion_execution::TaskContext; use datafusion_physical_expr::equivalence::OrderingEquivalenceProperties; -pub use display::{DefaultDisplay, DisplayAs, DisplayFormatType, VerboseDisplay}; +use datafusion_physical_expr::expressions::Column; +use datafusion_physical_expr::{ + EquivalenceProperties, PhysicalSortExpr, PhysicalSortRequirement, +}; + use futures::stream::TryStreamExt; -use std::fmt::Debug; use tokio::task::JoinSet; -use datafusion_common::tree_node::Transformed; -use datafusion_common::DataFusionError; -use std::any::Any; -use std::sync::Arc; +mod topk; +mod visitor; -// backwards compatibility +pub mod aggregates; +pub mod analyze; +pub mod coalesce_batches; +pub mod coalesce_partitions; +pub mod common; +pub mod display; +pub mod empty; +pub mod explain; +pub mod filter; +pub mod insert; +pub mod joins; +pub mod limit; +pub mod memory; +pub mod metrics; +pub mod projection; +pub mod repartition; +pub mod sorts; +pub mod stream; +pub mod streaming; +pub mod tree_node; +pub mod udaf; +pub mod union; +pub mod unnest; +pub mod values; +pub mod windows; + +pub use crate::display::{DefaultDisplay, DisplayAs, DisplayFormatType, VerboseDisplay}; +pub use crate::metrics::Metric; +pub use crate::topk::TopK; +pub use crate::visitor::{accept, visit_execution_plan, ExecutionPlanVisitor}; + +pub use datafusion_common::hash_utils; +pub use datafusion_common::utils::project_schema; +pub use datafusion_common::{internal_err, ColumnStatistics, Statistics}; +pub use datafusion_expr::{Accumulator, ColumnarValue}; +pub use datafusion_physical_expr::window::WindowExpr; +pub use datafusion_physical_expr::{ + expressions, functions, ordering_equivalence_properties_helper, udf, AggregateExpr, + Distribution, Partitioning, PhysicalExpr, +}; + +// Backwards compatibility +pub use crate::stream::EmptyRecordBatchStream; pub use datafusion_execution::{RecordBatchStream, SendableRecordBatchStream}; -pub use stream::EmptyRecordBatchStream; /// `ExecutionPlan` represent nodes in the DataFusion Physical Plan. /// @@ -193,7 +232,7 @@ pub trait ExecutionPlan: Debug + DisplayAs + Send + Sync { } /// Returns the global output statistics for this `ExecutionPlan` node. - fn statistics(&self) -> Statistics; + fn statistics(&self) -> Result; } /// Indicate whether a data exchange is needed for the input of `plan`, which will be very helpful @@ -343,46 +382,5 @@ pub fn unbounded_output(plan: &Arc) -> bool { .unwrap_or(true) } -use datafusion_physical_expr::expressions::Column; -pub use datafusion_physical_expr::window::WindowExpr; -pub use datafusion_physical_expr::{AggregateExpr, PhysicalExpr}; -pub use datafusion_physical_expr::{Distribution, Partitioning}; -use datafusion_physical_expr::{EquivalenceProperties, PhysicalSortRequirement}; - -pub mod aggregates; -pub mod analyze; -pub mod coalesce_batches; -pub mod coalesce_partitions; -pub mod common; -pub mod display; -pub mod empty; -pub mod explain; -pub mod filter; -pub mod insert; -pub mod joins; -pub mod limit; -pub mod memory; -pub mod metrics; -pub mod projection; -pub mod repartition; -pub mod sorts; -pub mod stream; -pub mod streaming; -pub mod tree_node; -pub mod udaf; -pub mod union; -pub mod unnest; -pub mod values; -pub mod windows; - -use crate::repartition::RepartitionExec; -use crate::sorts::sort_preserving_merge::SortPreservingMergeExec; -pub use datafusion_common::hash_utils; -pub use datafusion_common::utils::project_schema; -use datafusion_execution::TaskContext; -pub use datafusion_physical_expr::{ - expressions, functions, ordering_equivalence_properties_helper, udf, -}; - #[cfg(test)] pub mod test; diff --git a/datafusion/physical-plan/src/limit.rs b/datafusion/physical-plan/src/limit.rs index 31ed08399c2e..9fe1eb0763ba 100644 --- a/datafusion/physical-plan/src/limit.rs +++ b/datafusion/physical-plan/src/limit.rs @@ -22,23 +22,22 @@ use std::pin::Pin; use std::sync::Arc; use std::task::{Context, Poll}; -use crate::{ - DisplayFormatType, Distribution, EquivalenceProperties, ExecutionPlan, Partitioning, -}; - use super::expressions::PhysicalSortExpr; use super::metrics::{BaselineMetrics, ExecutionPlanMetricsSet, MetricsSet}; use super::{DisplayAs, RecordBatchStream, SendableRecordBatchStream, Statistics}; +use crate::{ + DisplayFormatType, Distribution, EquivalenceProperties, ExecutionPlan, Partitioning, +}; use arrow::array::ArrayRef; use arrow::datatypes::SchemaRef; use arrow::record_batch::{RecordBatch, RecordBatchOptions}; +use datafusion_common::stats::Precision; use datafusion_common::{internal_err, DataFusionError, Result}; use datafusion_execution::TaskContext; use datafusion_physical_expr::OrderingEquivalenceProperties; -use futures::stream::Stream; -use futures::stream::StreamExt; +use futures::stream::{Stream, StreamExt}; use log::trace; /// Limit execution plan @@ -191,8 +190,8 @@ impl ExecutionPlan for GlobalLimitExec { Some(self.metrics.clone_inner()) } - fn statistics(&self) -> Statistics { - let input_stats = self.input.statistics(); + fn statistics(&self) -> Result { + let input_stats = self.input.statistics()?; let skip = self.skip; // the maximum row number needs to be fetched let max_row_num = self @@ -205,16 +204,29 @@ impl ExecutionPlan for GlobalLimitExec { } }) .unwrap_or(usize::MAX); - match input_stats { + let col_stats = Statistics::unknown_column(&self.schema()); + + let fetched_row_number_stats = Statistics { + num_rows: Precision::Exact(max_row_num), + column_statistics: col_stats.clone(), + total_byte_size: Precision::Absent, + }; + + let stats = match input_stats { Statistics { - num_rows: Some(nr), .. + num_rows: Precision::Exact(nr), + .. + } + | Statistics { + num_rows: Precision::Inexact(nr), + .. } => { if nr <= skip { // if all input data will be skipped, return 0 Statistics { - num_rows: Some(0), - is_exact: input_stats.is_exact, - ..Default::default() + num_rows: Precision::Exact(0), + column_statistics: col_stats, + total_byte_size: Precision::Absent, } } else if nr <= max_row_num { // if the input does not reach the "fetch" globally, return input stats @@ -222,20 +234,15 @@ impl ExecutionPlan for GlobalLimitExec { } else { // if the input is greater than the "fetch", the num_row will be the "fetch", // but we won't be able to predict the other statistics - Statistics { - num_rows: Some(max_row_num), - is_exact: input_stats.is_exact, - ..Default::default() - } + fetched_row_number_stats } } - _ => Statistics { + _ => { // the result output row number will always be no greater than the limit number - num_rows: Some(max_row_num), - is_exact: false, - ..Default::default() - }, - } + fetched_row_number_stats + } + }; + Ok(stats) } } @@ -361,32 +368,53 @@ impl ExecutionPlan for LocalLimitExec { Some(self.metrics.clone_inner()) } - fn statistics(&self) -> Statistics { - let input_stats = self.input.statistics(); - match input_stats { + fn statistics(&self) -> Result { + let input_stats = self.input.statistics()?; + let col_stats = Statistics::unknown_column(&self.schema()); + let stats = match input_stats { // if the input does not reach the limit globally, return input stats Statistics { - num_rows: Some(nr), .. + num_rows: Precision::Exact(nr), + .. + } + | Statistics { + num_rows: Precision::Inexact(nr), + .. } if nr <= self.fetch => input_stats, // if the input is greater than the limit, the num_row will be greater // than the limit because the partitions will be limited separatly // the statistic Statistics { - num_rows: Some(nr), .. + num_rows: Precision::Exact(nr), + .. + } if nr > self.fetch => Statistics { + num_rows: Precision::Exact(self.fetch), + // this is not actually exact, but will be when GlobalLimit is applied + // TODO stats: find a more explicit way to vehiculate this information + column_statistics: col_stats, + total_byte_size: Precision::Absent, + }, + Statistics { + num_rows: Precision::Inexact(nr), + .. } if nr > self.fetch => Statistics { - num_rows: Some(self.fetch), + num_rows: Precision::Inexact(self.fetch), // this is not actually exact, but will be when GlobalLimit is applied // TODO stats: find a more explicit way to vehiculate this information - is_exact: input_stats.is_exact, - ..Default::default() + column_statistics: col_stats, + total_byte_size: Precision::Absent, }, _ => Statistics { // the result output row number will always be no greater than the limit number - num_rows: Some(self.fetch * self.output_partitioning().partition_count()), - is_exact: false, - ..Default::default() + num_rows: Precision::Inexact( + self.fetch * self.output_partitioning().partition_count(), + ), + + column_statistics: col_stats, + total_byte_size: Precision::Absent, }, - } + }; + Ok(stats) } } @@ -528,14 +556,12 @@ impl RecordBatchStream for LimitStream { #[cfg(test)] mod tests { - - use arrow_schema::Schema; - use common::collect; - use super::*; use crate::coalesce_partitions::CoalescePartitionsExec; - use crate::common; - use crate::test; + use crate::common::collect; + use crate::{common, test}; + + use arrow_schema::Schema; #[tokio::test] async fn limit() -> Result<()> { @@ -728,10 +754,10 @@ mod tests { #[tokio::test] async fn test_row_number_statistics_for_global_limit() -> Result<()> { let row_count = row_number_statistics_for_global_limit(0, Some(10)).await?; - assert_eq!(row_count, Some(10)); + assert_eq!(row_count, Precision::Exact(10)); let row_count = row_number_statistics_for_global_limit(5, Some(10)).await?; - assert_eq!(row_count, Some(15)); + assert_eq!(row_count, Precision::Exact(15)); Ok(()) } @@ -739,7 +765,7 @@ mod tests { #[tokio::test] async fn test_row_number_statistics_for_local_limit() -> Result<()> { let row_count = row_number_statistics_for_local_limit(4, 10).await?; - assert_eq!(row_count, Some(10)); + assert_eq!(row_count, Precision::Exact(10)); Ok(()) } @@ -747,7 +773,7 @@ mod tests { async fn row_number_statistics_for_global_limit( skip: usize, fetch: Option, - ) -> Result> { + ) -> Result> { let num_partitions = 4; let csv = test::scan_partitioned(num_partitions); @@ -756,20 +782,20 @@ mod tests { let offset = GlobalLimitExec::new(Arc::new(CoalescePartitionsExec::new(csv)), skip, fetch); - Ok(offset.statistics().num_rows) + Ok(offset.statistics()?.num_rows) } async fn row_number_statistics_for_local_limit( num_partitions: usize, fetch: usize, - ) -> Result> { + ) -> Result> { let csv = test::scan_partitioned(num_partitions); assert_eq!(csv.output_partitioning().partition_count(), num_partitions); let offset = LocalLimitExec::new(csv, fetch); - Ok(offset.statistics().num_rows) + Ok(offset.statistics()?.num_rows) } /// Return a RecordBatch with a single array with row_count sz diff --git a/datafusion/physical-plan/src/memory.rs b/datafusion/physical-plan/src/memory.rs index d919ded8d0e2..b53500579266 100644 --- a/datafusion/physical-plan/src/memory.rs +++ b/datafusion/physical-plan/src/memory.rs @@ -17,23 +17,24 @@ //! Execution plan for reading in-memory batches of data +use std::any::Any; +use std::fmt; +use std::sync::Arc; +use std::task::{Context, Poll}; + use super::expressions::PhysicalSortExpr; use super::{ common, DisplayAs, DisplayFormatType, ExecutionPlan, Partitioning, RecordBatchStream, SendableRecordBatchStream, Statistics, }; +use crate::ordering_equivalence_properties_helper; + use arrow::datatypes::SchemaRef; use arrow::record_batch::RecordBatch; -use core::fmt; -use datafusion_common::{internal_err, project_schema, Result}; -use std::any::Any; -use std::sync::Arc; -use std::task::{Context, Poll}; - -use crate::ordering_equivalence_properties_helper; -use datafusion_common::DataFusionError; +use datafusion_common::{internal_err, project_schema, DataFusionError, Result}; use datafusion_execution::TaskContext; use datafusion_physical_expr::{LexOrdering, OrderingEquivalenceProperties}; + use futures::Stream; /// Execution plan for reading in-memory batches of data @@ -150,12 +151,12 @@ impl ExecutionPlan for MemoryExec { } /// We recompute the statistics dynamically from the arrow metadata as it is pretty cheap to do so - fn statistics(&self) -> Statistics { - common::compute_record_batch_statistics( + fn statistics(&self) -> Result { + Ok(common::compute_record_batch_statistics( &self.partitions, &self.schema, self.projection.clone(), - ) + )) } } @@ -266,12 +267,14 @@ impl RecordBatchStream for MemoryStream { #[cfg(test)] mod tests { + use std::sync::Arc; + use crate::memory::MemoryExec; use crate::ExecutionPlan; + use arrow_schema::{DataType, Field, Schema, SortOptions}; use datafusion_physical_expr::expressions::col; use datafusion_physical_expr::PhysicalSortExpr; - use std::sync::Arc; #[test] fn test_memory_order_eq() -> datafusion_common::Result<()> { diff --git a/datafusion/physical-plan/src/projection.rs b/datafusion/physical-plan/src/projection.rs index f4a2cd339ee5..029dd24d7d11 100644 --- a/datafusion/physical-plan/src/projection.rs +++ b/datafusion/physical-plan/src/projection.rs @@ -36,15 +36,16 @@ use crate::{ use arrow::datatypes::{Field, Schema, SchemaRef}; use arrow::record_batch::{RecordBatch, RecordBatchOptions}; +use datafusion_common::stats::Precision; use datafusion_common::Result; use datafusion_execution::TaskContext; use datafusion_physical_expr::expressions::{Literal, UnKnownColumn}; +use datafusion_physical_expr::utils::find_orderings_of_exprs; use datafusion_physical_expr::{ normalize_out_expr_with_columns_map, project_equivalence_properties, project_ordering_equivalence_properties, OrderingEquivalenceProperties, }; -use datafusion_physical_expr::utils::find_orderings_of_exprs; use futures::stream::{Stream, StreamExt}; use log::trace; @@ -330,12 +331,12 @@ impl ExecutionPlan for ProjectionExec { Some(self.metrics.clone_inner()) } - fn statistics(&self) -> Statistics { - stats_projection( - self.input.statistics(), + fn statistics(&self) -> Result { + Ok(stats_projection( + self.input.statistics()?, self.expr.iter().map(|(e, _)| Arc::clone(e)), self.schema.clone(), - ) + )) } } @@ -392,57 +393,37 @@ fn get_field_metadata( } fn stats_projection( - stats: Statistics, + mut stats: Statistics, exprs: impl Iterator>, schema: SchemaRef, ) -> Statistics { - let inner_exprs = exprs.collect::>(); - let column_statistics = stats.column_statistics.map(|input_col_stats| { - inner_exprs - .clone() - .into_iter() - .map(|e| { - if let Some(col) = e.as_any().downcast_ref::() { - input_col_stats[col.index()].clone() - } else { - // TODO stats: estimate more statistics from expressions - // (expressions should compute their statistics themselves) - ColumnStatistics::default() - } - }) - .collect() - }); - - let primitive_row_size = inner_exprs - .into_iter() - .map(|e| match e.data_type(schema.as_ref()) { - Ok(data_type) => data_type.primitive_width(), - Err(_) => None, - }) - .try_fold(0usize, |init, v| v.map(|value| init + value)); - - match (primitive_row_size, stats.num_rows) { - (Some(row_size), Some(row_count)) => { - Statistics { - is_exact: stats.is_exact, - num_rows: stats.num_rows, - column_statistics, - // Use the row_size * row_count as the total byte size - total_byte_size: Some(row_size * row_count), - } - } - _ => { - Statistics { - is_exact: stats.is_exact, - num_rows: stats.num_rows, - column_statistics, - // TODO stats: knowing the type of the new columns we can guess the output size - // If we can't get the exact statistics for the project - // Before we get the exact result, we just use the child status - total_byte_size: stats.total_byte_size, + let mut primitive_row_size = 0; + let mut primitive_row_size_possible = true; + let mut column_statistics = vec![]; + for expr in exprs { + let col_stats = if let Some(col) = expr.as_any().downcast_ref::() { + stats.column_statistics[col.index()].clone() + } else { + // TODO stats: estimate more statistics from expressions + // (expressions should compute their statistics themselves) + ColumnStatistics::new_unknown() + }; + column_statistics.push(col_stats); + if let Ok(data_type) = expr.data_type(&schema) { + if let Some(value) = data_type.primitive_width() { + primitive_row_size += value; + continue; } } + primitive_row_size_possible = false; + } + + if primitive_row_size_possible { + stats.total_byte_size = + Precision::Exact(primitive_row_size).multiply(&stats.num_rows); } + stats.column_statistics = column_statistics; + stats } impl ProjectionStream { @@ -529,29 +510,32 @@ mod tests { fn get_stats() -> Statistics { Statistics { - is_exact: true, - num_rows: Some(5), - total_byte_size: Some(23), - column_statistics: Some(vec![ + num_rows: Precision::Exact(5), + total_byte_size: Precision::Exact(23), + column_statistics: vec![ ColumnStatistics { - distinct_count: Some(5), - max_value: Some(ScalarValue::Int64(Some(21))), - min_value: Some(ScalarValue::Int64(Some(-4))), - null_count: Some(0), + distinct_count: Precision::Exact(5), + max_value: Precision::Exact(ScalarValue::Int64(Some(21))), + min_value: Precision::Exact(ScalarValue::Int64(Some(-4))), + null_count: Precision::Exact(0), }, ColumnStatistics { - distinct_count: Some(1), - max_value: Some(ScalarValue::Utf8(Some(String::from("x")))), - min_value: Some(ScalarValue::Utf8(Some(String::from("a")))), - null_count: Some(3), + distinct_count: Precision::Exact(1), + max_value: Precision::Exact(ScalarValue::Utf8(Some(String::from( + "x", + )))), + min_value: Precision::Exact(ScalarValue::Utf8(Some(String::from( + "a", + )))), + null_count: Precision::Exact(3), }, ColumnStatistics { - distinct_count: None, - max_value: Some(ScalarValue::Float32(Some(1.1))), - min_value: Some(ScalarValue::Float32(Some(0.1))), - null_count: None, + distinct_count: Precision::Absent, + max_value: Precision::Exact(ScalarValue::Float32(Some(1.1))), + min_value: Precision::Exact(ScalarValue::Float32(Some(0.1))), + null_count: Precision::Absent, }, - ]), + ], } } @@ -574,23 +558,26 @@ mod tests { let result = stats_projection(source, exprs.into_iter(), Arc::new(schema)); let expected = Statistics { - is_exact: true, - num_rows: Some(5), - total_byte_size: Some(23), - column_statistics: Some(vec![ + num_rows: Precision::Exact(5), + total_byte_size: Precision::Exact(23), + column_statistics: vec![ ColumnStatistics { - distinct_count: Some(1), - max_value: Some(ScalarValue::Utf8(Some(String::from("x")))), - min_value: Some(ScalarValue::Utf8(Some(String::from("a")))), - null_count: Some(3), + distinct_count: Precision::Exact(1), + max_value: Precision::Exact(ScalarValue::Utf8(Some(String::from( + "x", + )))), + min_value: Precision::Exact(ScalarValue::Utf8(Some(String::from( + "a", + )))), + null_count: Precision::Exact(3), }, ColumnStatistics { - distinct_count: Some(5), - max_value: Some(ScalarValue::Int64(Some(21))), - min_value: Some(ScalarValue::Int64(Some(-4))), - null_count: Some(0), + distinct_count: Precision::Exact(5), + max_value: Precision::Exact(ScalarValue::Int64(Some(21))), + min_value: Precision::Exact(ScalarValue::Int64(Some(-4))), + null_count: Precision::Exact(0), }, - ]), + ], }; assert_eq!(result, expected); @@ -609,23 +596,22 @@ mod tests { let result = stats_projection(source, exprs.into_iter(), Arc::new(schema)); let expected = Statistics { - is_exact: true, - num_rows: Some(5), - total_byte_size: Some(60), - column_statistics: Some(vec![ + num_rows: Precision::Exact(5), + total_byte_size: Precision::Exact(60), + column_statistics: vec![ ColumnStatistics { - distinct_count: None, - max_value: Some(ScalarValue::Float32(Some(1.1))), - min_value: Some(ScalarValue::Float32(Some(0.1))), - null_count: None, + distinct_count: Precision::Absent, + max_value: Precision::Exact(ScalarValue::Float32(Some(1.1))), + min_value: Precision::Exact(ScalarValue::Float32(Some(0.1))), + null_count: Precision::Absent, }, ColumnStatistics { - distinct_count: Some(5), - max_value: Some(ScalarValue::Int64(Some(21))), - min_value: Some(ScalarValue::Int64(Some(-4))), - null_count: Some(0), + distinct_count: Precision::Exact(5), + max_value: Precision::Exact(ScalarValue::Int64(Some(21))), + min_value: Precision::Exact(ScalarValue::Int64(Some(-4))), + null_count: Precision::Exact(0), }, - ]), + ], }; assert_eq!(result, expected); diff --git a/datafusion/physical-plan/src/repartition/mod.rs b/datafusion/physical-plan/src/repartition/mod.rs index bcb9c3afeef1..d1638a02771c 100644 --- a/datafusion/physical-plan/src/repartition/mod.rs +++ b/datafusion/physical-plan/src/repartition/mod.rs @@ -620,7 +620,7 @@ impl ExecutionPlan for RepartitionExec { Some(self.metrics.clone_inner()) } - fn statistics(&self) -> Statistics { + fn statistics(&self) -> Result { self.input.statistics() } } diff --git a/datafusion/physical-plan/src/sorts/cursor.rs b/datafusion/physical-plan/src/sorts/cursor.rs index 52de880bae69..1ca41d4fe21c 100644 --- a/datafusion/physical-plan/src/sorts/cursor.rs +++ b/datafusion/physical-plan/src/sorts/cursor.rs @@ -15,14 +15,15 @@ // specific language governing permissions and limitations // under the License. -use crate::sorts::sort::SortOptions; +use std::cmp::Ordering; + use arrow::buffer::ScalarBuffer; +use arrow::compute::SortOptions; use arrow::datatypes::ArrowNativeTypeOp; use arrow::row::{Row, Rows}; use arrow_array::types::ByteArrayType; use arrow_array::{Array, ArrowPrimitiveType, GenericByteArray, PrimitiveArray}; use datafusion_execution::memory_pool::MemoryReservation; -use std::cmp::Ordering; /// A [`Cursor`] for [`Rows`] pub struct RowCursor { diff --git a/datafusion/physical-plan/src/sorts/sort.rs b/datafusion/physical-plan/src/sorts/sort.rs index fb6f460f1bcb..c7d676493f04 100644 --- a/datafusion/physical-plan/src/sorts/sort.rs +++ b/datafusion/physical-plan/src/sorts/sort.rs @@ -19,6 +19,14 @@ //! It will do in-memory sorting if it has enough memory budget //! but spills to disk if needed. +use std::any::Any; +use std::fmt; +use std::fmt::{Debug, Formatter}; +use std::fs::File; +use std::io::BufReader; +use std::path::{Path, PathBuf}; +use std::sync::Arc; + use crate::common::{spawn_buffered, IPCWriter}; use crate::expressions::PhysicalSortExpr; use crate::metrics::{ @@ -31,7 +39,7 @@ use crate::{ DisplayAs, DisplayFormatType, Distribution, EmptyRecordBatchStream, ExecutionPlan, Partitioning, SendableRecordBatchStream, Statistics, }; -pub use arrow::compute::SortOptions; + use arrow::compute::{concat_batches, lexsort_to_indices, take}; use arrow::datatypes::SchemaRef; use arrow::ipc::reader::FileReader; @@ -44,15 +52,9 @@ use datafusion_execution::memory_pool::{ use datafusion_execution::runtime_env::RuntimeEnv; use datafusion_execution::TaskContext; use datafusion_physical_expr::EquivalenceProperties; + use futures::{StreamExt, TryStreamExt}; use log::{debug, error, trace}; -use std::any::Any; -use std::fmt; -use std::fmt::{Debug, Formatter}; -use std::fs::File; -use std::io::BufReader; -use std::path::{Path, PathBuf}; -use std::sync::Arc; use tokio::sync::mpsc::Sender; use tokio::task; @@ -908,13 +910,15 @@ impl ExecutionPlan for SortExec { Some(self.metrics_set.clone_inner()) } - fn statistics(&self) -> Statistics { + fn statistics(&self) -> Result { self.input.statistics() } } #[cfg(test)] mod tests { + use std::collections::HashMap; + use super::*; use crate::coalesce_partitions::CoalescePartitionsExec; use crate::collect; @@ -923,14 +927,15 @@ mod tests { use crate::test; use crate::test::assert_is_pending; use crate::test::exec::{assert_strong_count_converges_to_zero, BlockingExec}; + use arrow::array::*; use arrow::compute::SortOptions; use arrow::datatypes::*; use datafusion_common::cast::as_primitive_array; use datafusion_execution::config::SessionConfig; use datafusion_execution::runtime_env::RuntimeConfig; + use futures::FutureExt; - use std::collections::HashMap; #[tokio::test] async fn test_in_mem_sort() -> Result<()> { diff --git a/datafusion/physical-plan/src/sorts/sort_preserving_merge.rs b/datafusion/physical-plan/src/sorts/sort_preserving_merge.rs index f7a9e193ee06..ee044d9172fb 100644 --- a/datafusion/physical-plan/src/sorts/sort_preserving_merge.rs +++ b/datafusion/physical-plan/src/sorts/sort_preserving_merge.rs @@ -28,10 +28,10 @@ use crate::{ DisplayAs, DisplayFormatType, Distribution, ExecutionPlan, Partitioning, SendableRecordBatchStream, Statistics, }; -use datafusion_execution::memory_pool::MemoryConsumer; use arrow::datatypes::SchemaRef; use datafusion_common::{internal_err, DataFusionError, Result}; +use datafusion_execution::memory_pool::MemoryConsumer; use datafusion_execution::TaskContext; use datafusion_physical_expr::{ EquivalenceProperties, OrderingEquivalenceProperties, PhysicalSortRequirement, @@ -264,22 +264,14 @@ impl ExecutionPlan for SortPreservingMergeExec { Some(self.metrics.clone_inner()) } - fn statistics(&self) -> Statistics { + fn statistics(&self) -> Result { self.input.statistics() } } #[cfg(test)] mod tests { - use std::iter::FromIterator; - - use arrow::array::ArrayRef; - use arrow::compute::SortOptions; - use arrow::datatypes::{DataType, Field, Schema}; - use arrow::record_batch::RecordBatch; - use datafusion_execution::config::SessionConfig; - use futures::{FutureExt, StreamExt}; - + use super::*; use crate::coalesce_partitions::CoalescePartitionsExec; use crate::expressions::col; use crate::memory::MemoryExec; @@ -289,10 +281,16 @@ mod tests { use crate::test::exec::{assert_strong_count_converges_to_zero, BlockingExec}; use crate::test::{self, assert_is_pending, make_partition}; use crate::{collect, common}; + use arrow::array::ArrayRef; use arrow::array::{Int32Array, StringArray, TimestampNanosecondArray}; + use arrow::compute::SortOptions; + use arrow::datatypes::{DataType, Field, Schema}; + use arrow::record_batch::RecordBatch; use datafusion_common::{assert_batches_eq, assert_contains}; + use datafusion_execution::config::SessionConfig; - use super::*; + use futures::{FutureExt, StreamExt}; + use std::iter::FromIterator; #[tokio::test] async fn test_merge_interleave() { diff --git a/datafusion/physical-plan/src/streaming.rs b/datafusion/physical-plan/src/streaming.rs index cb972fa41e3e..7bfa7e2ceefb 100644 --- a/datafusion/physical-plan/src/streaming.rs +++ b/datafusion/physical-plan/src/streaming.rs @@ -20,20 +20,19 @@ use std::any::Any; use std::sync::Arc; -use arrow::datatypes::SchemaRef; -use async_trait::async_trait; -use futures::stream::StreamExt; - -use datafusion_common::{internal_err, plan_err, DataFusionError, Result, Statistics}; -use datafusion_physical_expr::{LexOrdering, PhysicalSortExpr}; -use log::debug; - +use super::{DisplayAs, DisplayFormatType}; use crate::display::{OutputOrderingDisplay, ProjectSchemaDisplay}; use crate::stream::RecordBatchStreamAdapter; use crate::{ExecutionPlan, Partitioning, SendableRecordBatchStream}; + +use arrow::datatypes::SchemaRef; +use datafusion_common::{internal_err, plan_err, DataFusionError, Result, Statistics}; use datafusion_execution::TaskContext; +use datafusion_physical_expr::{LexOrdering, PhysicalSortExpr}; -use super::{DisplayAs, DisplayFormatType}; +use async_trait::async_trait; +use futures::stream::StreamExt; +use log::debug; /// A partition that can be converted into a [`SendableRecordBatchStream`] pub trait PartitionStream: Send + Sync { @@ -189,7 +188,7 @@ impl ExecutionPlan for StreamingTableExec { }) } - fn statistics(&self) -> Statistics { - Default::default() + fn statistics(&self) -> Result { + Ok(Statistics::new_unknown(&self.schema())) } } diff --git a/datafusion/physical-plan/src/test/exec.rs b/datafusion/physical-plan/src/test/exec.rs index a1f40c7ba909..f90f4231c620 100644 --- a/datafusion/physical-plan/src/test/exec.rs +++ b/datafusion/physical-plan/src/test/exec.rs @@ -23,23 +23,21 @@ use std::{ sync::{Arc, Weak}, task::{Context, Poll}, }; -use tokio::sync::Barrier; - -use arrow::{ - datatypes::{DataType, Field, Schema, SchemaRef}, - record_batch::RecordBatch, -}; -use futures::Stream; +use crate::stream::{RecordBatchReceiverStream, RecordBatchStreamAdapter}; use crate::{ - common, stream::RecordBatchReceiverStream, stream::RecordBatchStreamAdapter, - DisplayAs, DisplayFormatType, ExecutionPlan, Partitioning, RecordBatchStream, + common, DisplayAs, DisplayFormatType, ExecutionPlan, Partitioning, RecordBatchStream, SendableRecordBatchStream, Statistics, }; -use datafusion_physical_expr::PhysicalSortExpr; +use arrow::datatypes::{DataType, Field, Schema, SchemaRef}; +use arrow::record_batch::RecordBatch; use datafusion_common::{internal_err, DataFusionError, Result}; use datafusion_execution::TaskContext; +use datafusion_physical_expr::PhysicalSortExpr; + +use futures::Stream; +use tokio::sync::Barrier; /// Index into the data that has been returned so far #[derive(Debug, Default, Clone)] @@ -239,7 +237,7 @@ impl ExecutionPlan for MockExec { } // Panics if one of the batches is an error - fn statistics(&self) -> Statistics { + fn statistics(&self) -> Result { let data: Result> = self .data .iter() @@ -249,9 +247,13 @@ impl ExecutionPlan for MockExec { }) .collect(); - let data = data.unwrap(); + let data = data?; - common::compute_record_batch_statistics(&[data], &self.schema, None) + Ok(common::compute_record_batch_statistics( + &[data], + &self.schema, + None, + )) } } @@ -369,8 +371,12 @@ impl ExecutionPlan for BarrierExec { Ok(builder.build()) } - fn statistics(&self) -> Statistics { - common::compute_record_batch_statistics(&self.data, &self.schema, None) + fn statistics(&self) -> Result { + Ok(common::compute_record_batch_statistics( + &self.data, + &self.schema, + None, + )) } } @@ -448,8 +454,8 @@ impl ExecutionPlan for ErrorExec { internal_err!("ErrorExec, unsurprisingly, errored in partition {partition}") } - fn statistics(&self) -> Statistics { - Statistics::default() + fn statistics(&self) -> Result { + Ok(Statistics::new_unknown(&self.schema())) } } @@ -461,12 +467,9 @@ pub struct StatisticsExec { } impl StatisticsExec { pub fn new(stats: Statistics, schema: Schema) -> Self { - assert!( + assert_eq!( stats - .column_statistics - .as_ref() - .map(|cols| cols.len() == schema.fields().len()) - .unwrap_or(true), + .column_statistics.len(), schema.fields().len(), "if defined, the column statistics vector length should be the number of fields" ); Self { @@ -531,8 +534,8 @@ impl ExecutionPlan for StatisticsExec { unimplemented!("This plan only serves for testing statistics") } - fn statistics(&self) -> Statistics { - self.stats.clone() + fn statistics(&self) -> Result { + Ok(self.stats.clone()) } } @@ -625,7 +628,7 @@ impl ExecutionPlan for BlockingExec { })) } - fn statistics(&self) -> Statistics { + fn statistics(&self) -> Result { unimplemented!() } } @@ -762,7 +765,7 @@ impl ExecutionPlan for PanicExec { })) } - fn statistics(&self) -> Statistics { + fn statistics(&self) -> Result { unimplemented!() } } diff --git a/datafusion/physical-plan/src/union.rs b/datafusion/physical-plan/src/union.rs index af765e257db2..2727a6e161be 100644 --- a/datafusion/physical-plan/src/union.rs +++ b/datafusion/physical-plan/src/union.rs @@ -25,27 +25,25 @@ use std::pin::Pin; use std::task::{Context, Poll}; use std::{any::Any, sync::Arc}; -use arrow::{ - datatypes::{Field, Schema, SchemaRef}, - record_batch::RecordBatch, -}; -use datafusion_common::{exec_err, internal_err, DFSchemaRef, DataFusionError}; -use futures::Stream; -use itertools::Itertools; -use log::{debug, trace, warn}; - -use super::DisplayAs; use super::{ expressions::PhysicalSortExpr, metrics::{ExecutionPlanMetricsSet, MetricsSet}, - ColumnStatistics, DisplayFormatType, ExecutionPlan, Partitioning, RecordBatchStream, - SendableRecordBatchStream, Statistics, + ColumnStatistics, DisplayAs, DisplayFormatType, ExecutionPlan, Partitioning, + RecordBatchStream, SendableRecordBatchStream, Statistics, }; use crate::common::get_meet_of_orderings; +use crate::metrics::BaselineMetrics; use crate::stream::ObservedStream; -use crate::{expressions, metrics::BaselineMetrics}; -use datafusion_common::Result; + +use arrow::datatypes::{Field, Schema, SchemaRef}; +use arrow::record_batch::RecordBatch; +use datafusion_common::stats::Precision; +use datafusion_common::{exec_err, internal_err, DFSchemaRef, DataFusionError, Result}; use datafusion_execution::TaskContext; + +use futures::Stream; +use itertools::Itertools; +use log::{debug, trace, warn}; use tokio::macros::support::thread_rng_n; /// `UnionExec`: `UNION ALL` execution plan. @@ -264,12 +262,17 @@ impl ExecutionPlan for UnionExec { Some(self.metrics.clone_inner()) } - fn statistics(&self) -> Statistics { - self.inputs + fn statistics(&self) -> Result { + let stats = self + .inputs .iter() - .map(|ep| ep.statistics()) + .map(|stat| stat.statistics()) + .collect::>>()?; + + Ok(stats + .into_iter() .reduce(stats_union) - .unwrap_or_default() + .unwrap_or_else(|| Statistics::new_unknown(&self.schema()))) } fn benefits_from_input_partitioning(&self) -> Vec { @@ -438,12 +441,17 @@ impl ExecutionPlan for InterleaveExec { Some(self.metrics.clone_inner()) } - fn statistics(&self) -> Statistics { - self.inputs + fn statistics(&self) -> Result { + let stats = self + .inputs .iter() - .map(|ep| ep.statistics()) + .map(|stat| stat.statistics()) + .collect::>>()?; + + Ok(stats + .into_iter() .reduce(stats_union) - .unwrap_or_default() + .unwrap_or_else(|| Statistics::new_unknown(&self.schema()))) } fn benefits_from_input_partitioning(&self) -> Vec { @@ -564,47 +572,32 @@ fn col_stats_union( mut left: ColumnStatistics, right: ColumnStatistics, ) -> ColumnStatistics { - left.distinct_count = None; - left.min_value = left - .min_value - .zip(right.min_value) - .map(|(a, b)| expressions::helpers::min(&a, &b)) - .and_then(Result::ok); - left.max_value = left - .max_value - .zip(right.max_value) - .map(|(a, b)| expressions::helpers::max(&a, &b)) - .and_then(Result::ok); - left.null_count = left.null_count.zip(right.null_count).map(|(a, b)| a + b); + left.distinct_count = Precision::Absent; + left.min_value = left.min_value.min(&right.min_value); + left.max_value = left.max_value.max(&right.max_value); + left.null_count = left.null_count.add(&right.null_count); left } fn stats_union(mut left: Statistics, right: Statistics) -> Statistics { - left.is_exact = left.is_exact && right.is_exact; - left.num_rows = left.num_rows.zip(right.num_rows).map(|(a, b)| a + b); - left.total_byte_size = left - .total_byte_size - .zip(right.total_byte_size) - .map(|(a, b)| a + b); - left.column_statistics = - left.column_statistics - .zip(right.column_statistics) - .map(|(a, b)| { - a.into_iter() - .zip(b) - .map(|(ca, cb)| col_stats_union(ca, cb)) - .collect() - }); + left.num_rows = left.num_rows.add(&right.num_rows); + left.total_byte_size = left.total_byte_size.add(&right.total_byte_size); + left.column_statistics = left + .column_statistics + .into_iter() + .zip(right.column_statistics) + .map(|(a, b)| col_stats_union(a, b)) + .collect::>(); left } #[cfg(test)] mod tests { use super::*; + use crate::collect; use crate::test; - use crate::collect; use arrow::record_batch::RecordBatch; use datafusion_common::ScalarValue; @@ -630,82 +623,91 @@ mod tests { #[tokio::test] async fn test_stats_union() { let left = Statistics { - is_exact: true, - num_rows: Some(5), - total_byte_size: Some(23), - column_statistics: Some(vec![ + num_rows: Precision::Exact(5), + total_byte_size: Precision::Exact(23), + column_statistics: vec![ ColumnStatistics { - distinct_count: Some(5), - max_value: Some(ScalarValue::Int64(Some(21))), - min_value: Some(ScalarValue::Int64(Some(-4))), - null_count: Some(0), + distinct_count: Precision::Exact(5), + max_value: Precision::Exact(ScalarValue::Int64(Some(21))), + min_value: Precision::Exact(ScalarValue::Int64(Some(-4))), + null_count: Precision::Exact(0), }, ColumnStatistics { - distinct_count: Some(1), - max_value: Some(ScalarValue::Utf8(Some(String::from("x")))), - min_value: Some(ScalarValue::Utf8(Some(String::from("a")))), - null_count: Some(3), + distinct_count: Precision::Exact(1), + max_value: Precision::Exact(ScalarValue::Utf8(Some(String::from( + "x", + )))), + min_value: Precision::Exact(ScalarValue::Utf8(Some(String::from( + "a", + )))), + null_count: Precision::Exact(3), }, ColumnStatistics { - distinct_count: None, - max_value: Some(ScalarValue::Float32(Some(1.1))), - min_value: Some(ScalarValue::Float32(Some(0.1))), - null_count: None, + distinct_count: Precision::Absent, + max_value: Precision::Exact(ScalarValue::Float32(Some(1.1))), + min_value: Precision::Exact(ScalarValue::Float32(Some(0.1))), + null_count: Precision::Absent, }, - ]), + ], }; let right = Statistics { - is_exact: true, - num_rows: Some(7), - total_byte_size: Some(29), - column_statistics: Some(vec![ + num_rows: Precision::Exact(7), + total_byte_size: Precision::Exact(29), + column_statistics: vec![ ColumnStatistics { - distinct_count: Some(3), - max_value: Some(ScalarValue::Int64(Some(34))), - min_value: Some(ScalarValue::Int64(Some(1))), - null_count: Some(1), + distinct_count: Precision::Exact(3), + max_value: Precision::Exact(ScalarValue::Int64(Some(34))), + min_value: Precision::Exact(ScalarValue::Int64(Some(1))), + null_count: Precision::Exact(1), }, ColumnStatistics { - distinct_count: None, - max_value: Some(ScalarValue::Utf8(Some(String::from("c")))), - min_value: Some(ScalarValue::Utf8(Some(String::from("b")))), - null_count: None, + distinct_count: Precision::Absent, + max_value: Precision::Exact(ScalarValue::Utf8(Some(String::from( + "c", + )))), + min_value: Precision::Exact(ScalarValue::Utf8(Some(String::from( + "b", + )))), + null_count: Precision::Absent, }, ColumnStatistics { - distinct_count: None, - max_value: None, - min_value: None, - null_count: None, + distinct_count: Precision::Absent, + max_value: Precision::Absent, + min_value: Precision::Absent, + null_count: Precision::Absent, }, - ]), + ], }; let result = stats_union(left, right); let expected = Statistics { - is_exact: true, - num_rows: Some(12), - total_byte_size: Some(52), - column_statistics: Some(vec![ + num_rows: Precision::Exact(12), + total_byte_size: Precision::Exact(52), + column_statistics: vec![ ColumnStatistics { - distinct_count: None, - max_value: Some(ScalarValue::Int64(Some(34))), - min_value: Some(ScalarValue::Int64(Some(-4))), - null_count: Some(1), + distinct_count: Precision::Absent, + max_value: Precision::Exact(ScalarValue::Int64(Some(34))), + min_value: Precision::Exact(ScalarValue::Int64(Some(-4))), + null_count: Precision::Exact(1), }, ColumnStatistics { - distinct_count: None, - max_value: Some(ScalarValue::Utf8(Some(String::from("x")))), - min_value: Some(ScalarValue::Utf8(Some(String::from("a")))), - null_count: None, + distinct_count: Precision::Absent, + max_value: Precision::Exact(ScalarValue::Utf8(Some(String::from( + "x", + )))), + min_value: Precision::Exact(ScalarValue::Utf8(Some(String::from( + "a", + )))), + null_count: Precision::Absent, }, ColumnStatistics { - distinct_count: None, - max_value: None, - min_value: None, - null_count: None, + distinct_count: Precision::Absent, + max_value: Precision::Absent, + min_value: Precision::Absent, + null_count: Precision::Absent, }, - ]), + ], }; assert_eq!(result, expected); diff --git a/datafusion/physical-plan/src/unnest.rs b/datafusion/physical-plan/src/unnest.rs index 410ea97887e0..ed64735e5ad6 100644 --- a/datafusion/physical-plan/src/unnest.rs +++ b/datafusion/physical-plan/src/unnest.rs @@ -17,6 +17,17 @@ //! Defines the unnest column plan for unnesting values in a column that contains a list //! type, conceptually is like joining each row with all the values in the list column. + +use std::time::Instant; +use std::{any::Any, sync::Arc}; + +use super::DisplayAs; +use crate::{ + expressions::Column, DisplayFormatType, Distribution, EquivalenceProperties, + ExecutionPlan, Partitioning, PhysicalExpr, PhysicalSortExpr, RecordBatchStream, + SendableRecordBatchStream, Statistics, +}; + use arrow::array::{ Array, ArrayRef, ArrowPrimitiveType, FixedSizeListArray, LargeListArray, ListArray, PrimitiveArray, @@ -27,23 +38,12 @@ use arrow::datatypes::{ }; use arrow::record_batch::RecordBatch; use arrow_array::{GenericListArray, OffsetSizeTrait}; -use async_trait::async_trait; -use datafusion_common::UnnestOptions; -use datafusion_common::{exec_err, DataFusionError, Result}; +use datafusion_common::{exec_err, DataFusionError, Result, UnnestOptions}; use datafusion_execution::TaskContext; -use futures::Stream; -use futures::StreamExt; -use log::trace; -use std::time::Instant; -use std::{any::Any, sync::Arc}; - -use crate::{ - expressions::Column, DisplayFormatType, Distribution, EquivalenceProperties, - ExecutionPlan, Partitioning, PhysicalExpr, PhysicalSortExpr, RecordBatchStream, - SendableRecordBatchStream, Statistics, -}; -use super::DisplayAs; +use async_trait::async_trait; +use futures::{Stream, StreamExt}; +use log::trace; /// Unnest the given column by joining the row with each value in the /// nested type. @@ -160,8 +160,8 @@ impl ExecutionPlan for UnnestExec { })) } - fn statistics(&self) -> Statistics { - Default::default() + fn statistics(&self) -> Result { + Ok(Statistics::new_unknown(&self.schema())) } } diff --git a/datafusion/physical-plan/src/values.rs b/datafusion/physical-plan/src/values.rs index 383726401c5a..b624fb362e65 100644 --- a/datafusion/physical-plan/src/values.rs +++ b/datafusion/physical-plan/src/values.rs @@ -17,20 +17,21 @@ //! Values execution plan +use std::any::Any; +use std::sync::Arc; + use super::expressions::PhysicalSortExpr; use super::{common, DisplayAs, SendableRecordBatchStream, Statistics}; use crate::{ memory::MemoryStream, ColumnarValue, DisplayFormatType, ExecutionPlan, Partitioning, PhysicalExpr, }; + use arrow::array::new_null_array; use arrow::datatypes::SchemaRef; use arrow::record_batch::RecordBatch; -use datafusion_common::{internal_err, plan_err, ScalarValue}; -use datafusion_common::{DataFusionError, Result}; +use datafusion_common::{internal_err, plan_err, DataFusionError, Result, ScalarValue}; use datafusion_execution::TaskContext; -use std::any::Any; -use std::sync::Arc; /// Execution plan for values list based relation (produces constant rows) #[derive(Debug)] @@ -187,9 +188,13 @@ impl ExecutionPlan for ValuesExec { )?)) } - fn statistics(&self) -> Statistics { + fn statistics(&self) -> Result { let batch = self.data(); - common::compute_record_batch_statistics(&[batch], &self.schema, None) + Ok(common::compute_record_batch_statistics( + &[batch], + &self.schema, + None, + )) } } @@ -197,6 +202,7 @@ impl ExecutionPlan for ValuesExec { mod tests { use super::*; use crate::test::{self, make_partition}; + use arrow_schema::{DataType, Field, Schema}; #[tokio::test] diff --git a/datafusion/physical-plan/src/windows/bounded_window_agg_exec.rs b/datafusion/physical-plan/src/windows/bounded_window_agg_exec.rs index 800ea42b3562..eab47886c764 100644 --- a/datafusion/physical-plan/src/windows/bounded_window_agg_exec.rs +++ b/datafusion/physical-plan/src/windows/bounded_window_agg_exec.rs @@ -43,8 +43,8 @@ use arrow::{ datatypes::{Schema, SchemaBuilder, SchemaRef}, record_batch::RecordBatch, }; - use datafusion_common::hash_utils::create_hashes; +use datafusion_common::stats::Precision; use datafusion_common::utils::{ evaluate_partition_ranges, get_arrayref_at_indices, get_at_indices, get_record_batch_at_indices, get_row_at_idx, @@ -320,24 +320,22 @@ impl ExecutionPlan for BoundedWindowAggExec { Some(self.metrics.clone_inner()) } - fn statistics(&self) -> Statistics { - let input_stat = self.input.statistics(); + fn statistics(&self) -> Result { + let input_stat = self.input.statistics()?; let win_cols = self.window_expr.len(); let input_cols = self.input.schema().fields().len(); // TODO stats: some windowing function will maintain invariants such as min, max... let mut column_statistics = Vec::with_capacity(win_cols + input_cols); - if let Some(input_col_stats) = input_stat.column_statistics { - column_statistics.extend(input_col_stats); - } else { - column_statistics.extend(vec![ColumnStatistics::default(); input_cols]); + // copy stats of the input to the beginning of the schema. + column_statistics.extend(input_stat.column_statistics); + for _ in 0..win_cols { + column_statistics.push(ColumnStatistics::new_unknown()) } - column_statistics.extend(vec![ColumnStatistics::default(); win_cols]); - Statistics { - is_exact: input_stat.is_exact, + Ok(Statistics { num_rows: input_stat.num_rows, - column_statistics: Some(column_statistics), - total_byte_size: None, - } + column_statistics, + total_byte_size: Precision::Absent, + }) } } diff --git a/datafusion/physical-plan/src/windows/window_agg_exec.rs b/datafusion/physical-plan/src/windows/window_agg_exec.rs index b4dc8ec88c68..4873778425d3 100644 --- a/datafusion/physical-plan/src/windows/window_agg_exec.rs +++ b/datafusion/physical-plan/src/windows/window_agg_exec.rs @@ -42,6 +42,7 @@ use arrow::{ datatypes::{Schema, SchemaRef}, record_batch::RecordBatch, }; +use datafusion_common::stats::Precision; use datafusion_common::utils::{evaluate_partition_ranges, get_at_indices}; use datafusion_common::{internal_err, plan_err, DataFusionError, Result}; use datafusion_execution::TaskContext; @@ -246,24 +247,22 @@ impl ExecutionPlan for WindowAggExec { Some(self.metrics.clone_inner()) } - fn statistics(&self) -> Statistics { - let input_stat = self.input.statistics(); + fn statistics(&self) -> Result { + let input_stat = self.input.statistics()?; let win_cols = self.window_expr.len(); let input_cols = self.input.schema().fields().len(); // TODO stats: some windowing function will maintain invariants such as min, max... let mut column_statistics = Vec::with_capacity(win_cols + input_cols); - if let Some(input_col_stats) = input_stat.column_statistics { - column_statistics.extend(input_col_stats); - } else { - column_statistics.extend(vec![ColumnStatistics::default(); input_cols]); + // copy stats of the input to the beginning of the schema. + column_statistics.extend(input_stat.column_statistics); + for _ in 0..win_cols { + column_statistics.push(ColumnStatistics::new_unknown()) } - column_statistics.extend(vec![ColumnStatistics::default(); win_cols]); - Statistics { - is_exact: input_stat.is_exact, + Ok(Statistics { num_rows: input_stat.num_rows, - column_statistics: Some(column_statistics), - total_byte_size: None, - } + column_statistics, + total_byte_size: Precision::Absent, + }) } } diff --git a/datafusion/proto/proto/datafusion.proto b/datafusion/proto/proto/datafusion.proto index 1819d1a4392d..f4ab582593e0 100644 --- a/datafusion/proto/proto/datafusion.proto +++ b/datafusion/proto/proto/datafusion.proto @@ -1542,18 +1542,28 @@ message PartitionStats { repeated ColumnStats column_stats = 4; } +message Precision{ + PrecisionInfo precision_info = 1; + ScalarValue val = 2; +} + +enum PrecisionInfo { + EXACT = 0; + INEXACT = 1; + ABSENT = 2; +} + message Statistics { - int64 num_rows = 1; - int64 total_byte_size = 2; + Precision num_rows = 1; + Precision total_byte_size = 2; repeated ColumnStats column_stats = 3; - bool is_exact = 4; } message ColumnStats { - ScalarValue min_value = 1; - ScalarValue max_value = 2; - uint32 null_count = 3; - uint32 distinct_count = 4; + Precision min_value = 1; + Precision max_value = 2; + Precision null_count = 3; + Precision distinct_count = 4; } message NamedStructFieldExpr { diff --git a/datafusion/proto/src/generated/pbjson.rs b/datafusion/proto/src/generated/pbjson.rs index 9aed987491f3..e9e2fd0c0461 100644 --- a/datafusion/proto/src/generated/pbjson.rs +++ b/datafusion/proto/src/generated/pbjson.rs @@ -3289,10 +3289,10 @@ impl serde::Serialize for ColumnStats { if self.max_value.is_some() { len += 1; } - if self.null_count != 0 { + if self.null_count.is_some() { len += 1; } - if self.distinct_count != 0 { + if self.distinct_count.is_some() { len += 1; } let mut struct_ser = serializer.serialize_struct("datafusion.ColumnStats", len)?; @@ -3302,11 +3302,11 @@ impl serde::Serialize for ColumnStats { if let Some(v) = self.max_value.as_ref() { struct_ser.serialize_field("maxValue", v)?; } - if self.null_count != 0 { - struct_ser.serialize_field("nullCount", &self.null_count)?; + if let Some(v) = self.null_count.as_ref() { + struct_ser.serialize_field("nullCount", v)?; } - if self.distinct_count != 0 { - struct_ser.serialize_field("distinctCount", &self.distinct_count)?; + if let Some(v) = self.distinct_count.as_ref() { + struct_ser.serialize_field("distinctCount", v)?; } struct_ser.end() } @@ -3400,25 +3400,21 @@ impl<'de> serde::Deserialize<'de> for ColumnStats { if null_count__.is_some() { return Err(serde::de::Error::duplicate_field("nullCount")); } - null_count__ = - Some(map_.next_value::<::pbjson::private::NumberDeserialize<_>>()?.0) - ; + null_count__ = map_.next_value()?; } GeneratedField::DistinctCount => { if distinct_count__.is_some() { return Err(serde::de::Error::duplicate_field("distinctCount")); } - distinct_count__ = - Some(map_.next_value::<::pbjson::private::NumberDeserialize<_>>()?.0) - ; + distinct_count__ = map_.next_value()?; } } } Ok(ColumnStats { min_value: min_value__, max_value: max_value__, - null_count: null_count__.unwrap_or_default(), - distinct_count: distinct_count__.unwrap_or_default(), + null_count: null_count__, + distinct_count: distinct_count__, }) } } @@ -18320,6 +18316,191 @@ impl<'de> serde::Deserialize<'de> for PlanType { deserializer.deserialize_struct("datafusion.PlanType", FIELDS, GeneratedVisitor) } } +impl serde::Serialize for Precision { + #[allow(deprecated)] + fn serialize(&self, serializer: S) -> std::result::Result + where + S: serde::Serializer, + { + use serde::ser::SerializeStruct; + let mut len = 0; + if self.precision_info != 0 { + len += 1; + } + if self.val.is_some() { + len += 1; + } + let mut struct_ser = serializer.serialize_struct("datafusion.Precision", len)?; + if self.precision_info != 0 { + let v = PrecisionInfo::try_from(self.precision_info) + .map_err(|_| serde::ser::Error::custom(format!("Invalid variant {}", self.precision_info)))?; + struct_ser.serialize_field("precisionInfo", &v)?; + } + if let Some(v) = self.val.as_ref() { + struct_ser.serialize_field("val", v)?; + } + struct_ser.end() + } +} +impl<'de> serde::Deserialize<'de> for Precision { + #[allow(deprecated)] + fn deserialize(deserializer: D) -> std::result::Result + where + D: serde::Deserializer<'de>, + { + const FIELDS: &[&str] = &[ + "precision_info", + "precisionInfo", + "val", + ]; + + #[allow(clippy::enum_variant_names)] + enum GeneratedField { + PrecisionInfo, + Val, + } + impl<'de> serde::Deserialize<'de> for GeneratedField { + fn deserialize(deserializer: D) -> std::result::Result + where + D: serde::Deserializer<'de>, + { + struct GeneratedVisitor; + + impl<'de> serde::de::Visitor<'de> for GeneratedVisitor { + type Value = GeneratedField; + + fn expecting(&self, formatter: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + write!(formatter, "expected one of: {:?}", &FIELDS) + } + + #[allow(unused_variables)] + fn visit_str(self, value: &str) -> std::result::Result + where + E: serde::de::Error, + { + match value { + "precisionInfo" | "precision_info" => Ok(GeneratedField::PrecisionInfo), + "val" => Ok(GeneratedField::Val), + _ => Err(serde::de::Error::unknown_field(value, FIELDS)), + } + } + } + deserializer.deserialize_identifier(GeneratedVisitor) + } + } + struct GeneratedVisitor; + impl<'de> serde::de::Visitor<'de> for GeneratedVisitor { + type Value = Precision; + + fn expecting(&self, formatter: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + formatter.write_str("struct datafusion.Precision") + } + + fn visit_map(self, mut map_: V) -> std::result::Result + where + V: serde::de::MapAccess<'de>, + { + let mut precision_info__ = None; + let mut val__ = None; + while let Some(k) = map_.next_key()? { + match k { + GeneratedField::PrecisionInfo => { + if precision_info__.is_some() { + return Err(serde::de::Error::duplicate_field("precisionInfo")); + } + precision_info__ = Some(map_.next_value::()? as i32); + } + GeneratedField::Val => { + if val__.is_some() { + return Err(serde::de::Error::duplicate_field("val")); + } + val__ = map_.next_value()?; + } + } + } + Ok(Precision { + precision_info: precision_info__.unwrap_or_default(), + val: val__, + }) + } + } + deserializer.deserialize_struct("datafusion.Precision", FIELDS, GeneratedVisitor) + } +} +impl serde::Serialize for PrecisionInfo { + #[allow(deprecated)] + fn serialize(&self, serializer: S) -> std::result::Result + where + S: serde::Serializer, + { + let variant = match self { + Self::Exact => "EXACT", + Self::Inexact => "INEXACT", + Self::Absent => "ABSENT", + }; + serializer.serialize_str(variant) + } +} +impl<'de> serde::Deserialize<'de> for PrecisionInfo { + #[allow(deprecated)] + fn deserialize(deserializer: D) -> std::result::Result + where + D: serde::Deserializer<'de>, + { + const FIELDS: &[&str] = &[ + "EXACT", + "INEXACT", + "ABSENT", + ]; + + struct GeneratedVisitor; + + impl<'de> serde::de::Visitor<'de> for GeneratedVisitor { + type Value = PrecisionInfo; + + fn expecting(&self, formatter: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + write!(formatter, "expected one of: {:?}", &FIELDS) + } + + fn visit_i64(self, v: i64) -> std::result::Result + where + E: serde::de::Error, + { + i32::try_from(v) + .ok() + .and_then(|x| x.try_into().ok()) + .ok_or_else(|| { + serde::de::Error::invalid_value(serde::de::Unexpected::Signed(v), &self) + }) + } + + fn visit_u64(self, v: u64) -> std::result::Result + where + E: serde::de::Error, + { + i32::try_from(v) + .ok() + .and_then(|x| x.try_into().ok()) + .ok_or_else(|| { + serde::de::Error::invalid_value(serde::de::Unexpected::Unsigned(v), &self) + }) + } + + fn visit_str(self, value: &str) -> std::result::Result + where + E: serde::de::Error, + { + match value { + "EXACT" => Ok(PrecisionInfo::Exact), + "INEXACT" => Ok(PrecisionInfo::Inexact), + "ABSENT" => Ok(PrecisionInfo::Absent), + _ => Err(serde::de::Error::unknown_variant(value, FIELDS)), + } + } + } + deserializer.deserialize_any(GeneratedVisitor) + } +} impl serde::Serialize for PrepareNode { #[allow(deprecated)] fn serialize(&self, serializer: S) -> std::result::Result @@ -22240,33 +22421,25 @@ impl serde::Serialize for Statistics { { use serde::ser::SerializeStruct; let mut len = 0; - if self.num_rows != 0 { + if self.num_rows.is_some() { len += 1; } - if self.total_byte_size != 0 { + if self.total_byte_size.is_some() { len += 1; } if !self.column_stats.is_empty() { len += 1; } - if self.is_exact { - len += 1; - } let mut struct_ser = serializer.serialize_struct("datafusion.Statistics", len)?; - if self.num_rows != 0 { - #[allow(clippy::needless_borrow)] - struct_ser.serialize_field("numRows", ToString::to_string(&self.num_rows).as_str())?; + if let Some(v) = self.num_rows.as_ref() { + struct_ser.serialize_field("numRows", v)?; } - if self.total_byte_size != 0 { - #[allow(clippy::needless_borrow)] - struct_ser.serialize_field("totalByteSize", ToString::to_string(&self.total_byte_size).as_str())?; + if let Some(v) = self.total_byte_size.as_ref() { + struct_ser.serialize_field("totalByteSize", v)?; } if !self.column_stats.is_empty() { struct_ser.serialize_field("columnStats", &self.column_stats)?; } - if self.is_exact { - struct_ser.serialize_field("isExact", &self.is_exact)?; - } struct_ser.end() } } @@ -22283,8 +22456,6 @@ impl<'de> serde::Deserialize<'de> for Statistics { "totalByteSize", "column_stats", "columnStats", - "is_exact", - "isExact", ]; #[allow(clippy::enum_variant_names)] @@ -22292,7 +22463,6 @@ impl<'de> serde::Deserialize<'de> for Statistics { NumRows, TotalByteSize, ColumnStats, - IsExact, } impl<'de> serde::Deserialize<'de> for GeneratedField { fn deserialize(deserializer: D) -> std::result::Result @@ -22317,7 +22487,6 @@ impl<'de> serde::Deserialize<'de> for Statistics { "numRows" | "num_rows" => Ok(GeneratedField::NumRows), "totalByteSize" | "total_byte_size" => Ok(GeneratedField::TotalByteSize), "columnStats" | "column_stats" => Ok(GeneratedField::ColumnStats), - "isExact" | "is_exact" => Ok(GeneratedField::IsExact), _ => Err(serde::de::Error::unknown_field(value, FIELDS)), } } @@ -22340,24 +22509,19 @@ impl<'de> serde::Deserialize<'de> for Statistics { let mut num_rows__ = None; let mut total_byte_size__ = None; let mut column_stats__ = None; - let mut is_exact__ = None; while let Some(k) = map_.next_key()? { match k { GeneratedField::NumRows => { if num_rows__.is_some() { return Err(serde::de::Error::duplicate_field("numRows")); } - num_rows__ = - Some(map_.next_value::<::pbjson::private::NumberDeserialize<_>>()?.0) - ; + num_rows__ = map_.next_value()?; } GeneratedField::TotalByteSize => { if total_byte_size__.is_some() { return Err(serde::de::Error::duplicate_field("totalByteSize")); } - total_byte_size__ = - Some(map_.next_value::<::pbjson::private::NumberDeserialize<_>>()?.0) - ; + total_byte_size__ = map_.next_value()?; } GeneratedField::ColumnStats => { if column_stats__.is_some() { @@ -22365,19 +22529,12 @@ impl<'de> serde::Deserialize<'de> for Statistics { } column_stats__ = Some(map_.next_value()?); } - GeneratedField::IsExact => { - if is_exact__.is_some() { - return Err(serde::de::Error::duplicate_field("isExact")); - } - is_exact__ = Some(map_.next_value()?); - } } } Ok(Statistics { - num_rows: num_rows__.unwrap_or_default(), - total_byte_size: total_byte_size__.unwrap_or_default(), + num_rows: num_rows__, + total_byte_size: total_byte_size__, column_stats: column_stats__.unwrap_or_default(), - is_exact: is_exact__.unwrap_or_default(), }) } } diff --git a/datafusion/proto/src/generated/prost.rs b/datafusion/proto/src/generated/prost.rs index 883799b1590d..1c821708a971 100644 --- a/datafusion/proto/src/generated/prost.rs +++ b/datafusion/proto/src/generated/prost.rs @@ -2205,27 +2205,33 @@ pub struct PartitionStats { } #[allow(clippy::derive_partial_eq_without_eq)] #[derive(Clone, PartialEq, ::prost::Message)] +pub struct Precision { + #[prost(enumeration = "PrecisionInfo", tag = "1")] + pub precision_info: i32, + #[prost(message, optional, tag = "2")] + pub val: ::core::option::Option, +} +#[allow(clippy::derive_partial_eq_without_eq)] +#[derive(Clone, PartialEq, ::prost::Message)] pub struct Statistics { - #[prost(int64, tag = "1")] - pub num_rows: i64, - #[prost(int64, tag = "2")] - pub total_byte_size: i64, + #[prost(message, optional, tag = "1")] + pub num_rows: ::core::option::Option, + #[prost(message, optional, tag = "2")] + pub total_byte_size: ::core::option::Option, #[prost(message, repeated, tag = "3")] pub column_stats: ::prost::alloc::vec::Vec, - #[prost(bool, tag = "4")] - pub is_exact: bool, } #[allow(clippy::derive_partial_eq_without_eq)] #[derive(Clone, PartialEq, ::prost::Message)] pub struct ColumnStats { #[prost(message, optional, tag = "1")] - pub min_value: ::core::option::Option, + pub min_value: ::core::option::Option, #[prost(message, optional, tag = "2")] - pub max_value: ::core::option::Option, - #[prost(uint32, tag = "3")] - pub null_count: u32, - #[prost(uint32, tag = "4")] - pub distinct_count: u32, + pub max_value: ::core::option::Option, + #[prost(message, optional, tag = "3")] + pub null_count: ::core::option::Option, + #[prost(message, optional, tag = "4")] + pub distinct_count: ::core::option::Option, } #[allow(clippy::derive_partial_eq_without_eq)] #[derive(Clone, PartialEq, ::prost::Message)] @@ -3157,3 +3163,32 @@ impl JoinSide { } } } +#[derive(Clone, Copy, Debug, PartialEq, Eq, Hash, PartialOrd, Ord, ::prost::Enumeration)] +#[repr(i32)] +pub enum PrecisionInfo { + Exact = 0, + Inexact = 1, + Absent = 2, +} +impl PrecisionInfo { + /// String value of the enum field names used in the ProtoBuf definition. + /// + /// The values are not transformed in any way and thus are considered stable + /// (if the ProtoBuf definition does not change) and safe for programmatic use. + pub fn as_str_name(&self) -> &'static str { + match self { + PrecisionInfo::Exact => "EXACT", + PrecisionInfo::Inexact => "INEXACT", + PrecisionInfo::Absent => "ABSENT", + } + } + /// Creates an enum from field names used in the ProtoBuf definition. + pub fn from_str_name(value: &str) -> ::core::option::Option { + match value { + "EXACT" => Some(Self::Exact), + "INEXACT" => Some(Self::Inexact), + "ABSENT" => Some(Self::Absent), + _ => None, + } + } +} diff --git a/datafusion/proto/src/physical_plan/from_proto.rs b/datafusion/proto/src/physical_plan/from_proto.rs index bb38116e5dba..ff02b8052136 100644 --- a/datafusion/proto/src/physical_plan/from_proto.rs +++ b/datafusion/proto/src/physical_plan/from_proto.rs @@ -17,10 +17,12 @@ //! Serde code to convert from protocol buffers to Rust data structures. -use crate::protobuf; +use std::convert::{TryFrom, TryInto}; +use std::ops::Deref; +use std::sync::Arc; + +use arrow::compute::SortOptions; use arrow::datatypes::DataType; -use chrono::TimeZone; -use chrono::Utc; use datafusion::arrow::datatypes::Schema; use datafusion::datasource::listing::{FileRange, PartitionedFile}; use datafusion::datasource::object_store::ObjectStoreUrl; @@ -29,31 +31,28 @@ use datafusion::execution::context::ExecutionProps; use datafusion::execution::FunctionRegistry; use datafusion::logical_expr::window_function::WindowFunction; use datafusion::physical_expr::{PhysicalSortExpr, ScalarFunctionExpr}; -use datafusion::physical_plan::expressions::{in_list, LikeExpr}; +use datafusion::physical_plan::expressions::{ + in_list, BinaryExpr, CaseExpr, CastExpr, Column, IsNotNullExpr, IsNullExpr, LikeExpr, + Literal, NegativeExpr, NotExpr, TryCastExpr, +}; use datafusion::physical_plan::expressions::{GetFieldAccessExpr, GetIndexedFieldExpr}; +use datafusion::physical_plan::joins::utils::JoinSide; use datafusion::physical_plan::windows::create_window_expr; -use datafusion::physical_plan::WindowExpr; use datafusion::physical_plan::{ - expressions::{ - BinaryExpr, CaseExpr, CastExpr, Column, IsNotNullExpr, IsNullExpr, Literal, - NegativeExpr, NotExpr, TryCastExpr, - }, - functions, Partitioning, + functions, ColumnStatistics, Partitioning, PhysicalExpr, Statistics, WindowExpr, }; -use datafusion::physical_plan::{ColumnStatistics, PhysicalExpr, Statistics}; -use datafusion_common::{not_impl_err, DataFusionError, Result}; -use object_store::path::Path; -use object_store::ObjectMeta; -use std::convert::{TryFrom, TryInto}; -use std::ops::Deref; -use std::sync::Arc; +use datafusion_common::stats::Precision; +use datafusion_common::{not_impl_err, DataFusionError, Result, ScalarValue}; use crate::common::proto_error; use crate::convert_required; use crate::logical_plan; +use crate::protobuf; use crate::protobuf::physical_expr_node::ExprType; -use datafusion::physical_plan::joins::utils::JoinSide; -use datafusion::physical_plan::sorts::sort::SortOptions; + +use chrono::{TimeZone, Utc}; +use object_store::path::Path; +use object_store::ObjectMeta; impl From<&protobuf::PhysicalColumn> for Column { fn from(c: &protobuf::PhysicalColumn) -> Column { @@ -581,10 +580,96 @@ impl TryFrom<&protobuf::FileGroup> for Vec { impl From<&protobuf::ColumnStats> for ColumnStatistics { fn from(cs: &protobuf::ColumnStats) -> ColumnStatistics { ColumnStatistics { - null_count: Some(cs.null_count as usize), - max_value: cs.max_value.as_ref().map(|m| m.try_into().unwrap()), - min_value: cs.min_value.as_ref().map(|m| m.try_into().unwrap()), - distinct_count: Some(cs.distinct_count as usize), + null_count: if let Some(nc) = &cs.null_count { + nc.clone().into() + } else { + Precision::Absent + }, + max_value: if let Some(max) = &cs.max_value { + max.clone().into() + } else { + Precision::Absent + }, + min_value: if let Some(min) = &cs.min_value { + min.clone().into() + } else { + Precision::Absent + }, + distinct_count: if let Some(dc) = &cs.distinct_count { + dc.clone().into() + } else { + Precision::Absent + }, + } + } +} + +impl From for Precision { + fn from(s: protobuf::Precision) -> Self { + let Ok(precision_type) = s.precision_info.try_into() else { + return Precision::Absent; + }; + match precision_type { + protobuf::PrecisionInfo::Exact => { + if let Some(val) = s.val { + if let Ok(ScalarValue::UInt64(Some(val))) = + ScalarValue::try_from(&val) + { + Precision::Exact(val as usize) + } else { + Precision::Absent + } + } else { + Precision::Absent + } + } + protobuf::PrecisionInfo::Inexact => { + if let Some(val) = s.val { + if let Ok(ScalarValue::UInt64(Some(val))) = + ScalarValue::try_from(&val) + { + Precision::Inexact(val as usize) + } else { + Precision::Absent + } + } else { + Precision::Absent + } + } + protobuf::PrecisionInfo::Absent => Precision::Absent, + } + } +} + +impl From for Precision { + fn from(s: protobuf::Precision) -> Self { + let Ok(precision_type) = s.precision_info.try_into() else { + return Precision::Absent; + }; + match precision_type { + protobuf::PrecisionInfo::Exact => { + if let Some(val) = s.val { + if let Ok(val) = ScalarValue::try_from(&val) { + Precision::Exact(val) + } else { + Precision::Absent + } + } else { + Precision::Absent + } + } + protobuf::PrecisionInfo::Inexact => { + if let Some(val) = s.val { + if let Ok(val) = ScalarValue::try_from(&val) { + Precision::Inexact(val) + } else { + Precision::Absent + } + } else { + Precision::Absent + } + } + protobuf::PrecisionInfo::Absent => Precision::Absent, } } } @@ -603,27 +688,19 @@ impl TryFrom<&protobuf::Statistics> for Statistics { fn try_from(s: &protobuf::Statistics) -> Result { // Keep it sync with Statistics::to_proto - let none_value = -1_i64; - let column_statistics = - s.column_stats.iter().map(|s| s.into()).collect::>(); Ok(Statistics { - num_rows: if s.num_rows == none_value { - None + num_rows: if let Some(nr) = &s.num_rows { + nr.clone().into() } else { - Some(s.num_rows as usize) + Precision::Absent }, - total_byte_size: if s.total_byte_size == none_value { - None + total_byte_size: if let Some(tbs) = &s.total_byte_size { + tbs.clone().into() } else { - Some(s.total_byte_size as usize) + Precision::Absent }, // No column statistic (None) is encoded with empty array - column_statistics: if column_statistics.is_empty() { - None - } else { - Some(column_statistics) - }, - is_exact: s.is_exact, + column_statistics: s.column_stats.iter().map(|s| s.into()).collect(), }) } } diff --git a/datafusion/proto/src/physical_plan/to_proto.rs b/datafusion/proto/src/physical_plan/to_proto.rs index cf3dbe26190a..db97c393255a 100644 --- a/datafusion/proto/src/physical_plan/to_proto.rs +++ b/datafusion/proto/src/physical_plan/to_proto.rs @@ -22,58 +22,37 @@ use std::{ sync::Arc, }; -use datafusion::physical_plan::{ - expressions::{ - ApproxDistinct, ApproxMedian, ApproxPercentileCont, - ApproxPercentileContWithWeight, ArrayAgg, Correlation, Covariance, CovariancePop, - DistinctArrayAgg, DistinctBitXor, DistinctSum, FirstValue, Grouping, LastValue, - Median, OrderSensitiveArrayAgg, Regr, RegrType, Stddev, StddevPop, Variance, - VariancePop, - }, - windows::BuiltInWindowExpr, - ColumnStatistics, -}; -use datafusion::{ - physical_expr::window::NthValueKind, - physical_plan::{ - expressions::{ - CaseExpr, CumeDist, InListExpr, IsNotNullExpr, IsNullExpr, NegativeExpr, - NotExpr, NthValue, Ntile, Rank, RankType, RowNumber, WindowShift, - }, - Statistics, - }, -}; -use datafusion::{ - physical_expr::window::SlidingAggregateWindowExpr, - physical_plan::{ - expressions::{CastExpr, TryCastExpr}, - windows::PlainAggregateWindowExpr, - WindowExpr, - }, -}; - -use datafusion::datasource::listing::{FileRange, PartitionedFile}; -use datafusion::datasource::physical_plan::FileScanConfig; - -use datafusion::physical_plan::expressions::{Count, DistinctCount, Literal}; - -use datafusion::physical_plan::expressions::{ - Avg, BinaryExpr, BitAnd, BitOr, BitXor, BoolAnd, BoolOr, Column, LikeExpr, Max, Min, - Sum, -}; -use datafusion::physical_plan::{AggregateExpr, PhysicalExpr}; - -use crate::protobuf::{self, physical_window_expr_node}; +use crate::protobuf::{self, physical_window_expr_node, scalar_value::Value}; use crate::protobuf::{ physical_aggregate_expr_node, PhysicalSortExprNode, PhysicalSortExprNodeCollection, ScalarValue, }; + +use datafusion::datasource::listing::{FileRange, PartitionedFile}; +use datafusion::datasource::physical_plan::FileScanConfig; use datafusion::logical_expr::BuiltinScalarFunction; use datafusion::physical_expr::expressions::{GetFieldAccessExpr, GetIndexedFieldExpr}; +use datafusion::physical_expr::window::{NthValueKind, SlidingAggregateWindowExpr}; use datafusion::physical_expr::{PhysicalSortExpr, ScalarFunctionExpr}; +use datafusion::physical_plan::expressions::{ + ApproxDistinct, ApproxMedian, ApproxPercentileCont, ApproxPercentileContWithWeight, + ArrayAgg, Avg, BinaryExpr, BitAnd, BitOr, BitXor, BoolAnd, BoolOr, CaseExpr, + CastExpr, Column, Correlation, Count, Covariance, CovariancePop, CumeDist, + DistinctArrayAgg, DistinctBitXor, DistinctCount, DistinctSum, FirstValue, Grouping, + InListExpr, IsNotNullExpr, IsNullExpr, LastValue, LikeExpr, Literal, Max, Median, + Min, NegativeExpr, NotExpr, NthValue, Ntile, OrderSensitiveArrayAgg, Rank, RankType, + Regr, RegrType, RowNumber, Stddev, StddevPop, Sum, TryCastExpr, Variance, + VariancePop, WindowShift, +}; use datafusion::physical_plan::joins::utils::JoinSide; use datafusion::physical_plan::udaf::AggregateFunctionExpr; -use datafusion_common::{internal_err, not_impl_err, DataFusionError, Result}; +use datafusion::physical_plan::windows::{BuiltInWindowExpr, PlainAggregateWindowExpr}; +use datafusion::physical_plan::{ + AggregateExpr, ColumnStatistics, PhysicalExpr, Statistics, WindowExpr, +}; +use datafusion_common::{ + internal_err, not_impl_err, stats::Precision, DataFusionError, Result, +}; impl TryFrom> for protobuf::PhysicalExprNode { type Error = DataFusionError; @@ -644,29 +623,66 @@ impl TryFrom<&[PartitionedFile]> for protobuf::FileGroup { } } -impl From<&ColumnStatistics> for protobuf::ColumnStats { - fn from(cs: &ColumnStatistics) -> protobuf::ColumnStats { - protobuf::ColumnStats { - min_value: cs.min_value.as_ref().map(|m| m.try_into().unwrap()), - max_value: cs.max_value.as_ref().map(|m| m.try_into().unwrap()), - null_count: cs.null_count.map(|n| n as u32).unwrap_or(0), - distinct_count: cs.distinct_count.map(|n| n as u32).unwrap_or(0), +impl From<&Precision> for protobuf::Precision { + fn from(s: &Precision) -> protobuf::Precision { + match s { + Precision::Exact(val) => protobuf::Precision { + precision_info: protobuf::PrecisionInfo::Exact.into(), + val: Some(ScalarValue { + value: Some(Value::Uint64Value(*val as u64)), + }), + }, + Precision::Inexact(val) => protobuf::Precision { + precision_info: protobuf::PrecisionInfo::Inexact.into(), + val: Some(ScalarValue { + value: Some(Value::Uint64Value(*val as u64)), + }), + }, + Precision::Absent => protobuf::Precision { + precision_info: protobuf::PrecisionInfo::Absent.into(), + val: Some(ScalarValue { value: None }), + }, + } + } +} + +impl From<&Precision> for protobuf::Precision { + fn from(s: &Precision) -> protobuf::Precision { + match s { + Precision::Exact(val) => protobuf::Precision { + precision_info: protobuf::PrecisionInfo::Exact.into(), + val: val.try_into().ok(), + }, + Precision::Inexact(val) => protobuf::Precision { + precision_info: protobuf::PrecisionInfo::Inexact.into(), + val: val.try_into().ok(), + }, + Precision::Absent => protobuf::Precision { + precision_info: protobuf::PrecisionInfo::Absent.into(), + val: Some(ScalarValue { value: None }), + }, } } } impl From<&Statistics> for protobuf::Statistics { fn from(s: &Statistics) -> protobuf::Statistics { - let none_value = -1_i64; - let column_stats = match &s.column_statistics { - None => vec![], - Some(column_stats) => column_stats.iter().map(|s| s.into()).collect(), - }; + let column_stats = s.column_statistics.iter().map(|s| s.into()).collect(); protobuf::Statistics { - num_rows: s.num_rows.map(|n| n as i64).unwrap_or(none_value), - total_byte_size: s.total_byte_size.map(|n| n as i64).unwrap_or(none_value), + num_rows: Some(protobuf::Precision::from(&s.num_rows)), + total_byte_size: Some(protobuf::Precision::from(&s.total_byte_size)), column_stats, - is_exact: s.is_exact, + } + } +} + +impl From<&ColumnStatistics> for protobuf::ColumnStats { + fn from(s: &ColumnStatistics) -> protobuf::ColumnStats { + protobuf::ColumnStats { + min_value: Some(protobuf::Precision::from(&s.min_value)), + max_value: Some(protobuf::Precision::from(&s.max_value)), + null_count: Some(protobuf::Precision::from(&s.null_count)), + distinct_count: Some(protobuf::Precision::from(&s.distinct_count)), } } } diff --git a/datafusion/proto/tests/cases/roundtrip_physical_plan.rs b/datafusion/proto/tests/cases/roundtrip_physical_plan.rs index e30d416bdc95..01a0916d8cd2 100644 --- a/datafusion/proto/tests/cases/roundtrip_physical_plan.rs +++ b/datafusion/proto/tests/cases/roundtrip_physical_plan.rs @@ -25,20 +25,19 @@ use datafusion::datasource::listing::PartitionedFile; use datafusion::datasource::object_store::ObjectStoreUrl; use datafusion::datasource::physical_plan::{FileScanConfig, ParquetExec}; use datafusion::execution::context::ExecutionProps; -use datafusion::logical_expr::create_udf; -use datafusion::logical_expr::{BuiltinScalarFunction, Volatility}; -use datafusion::logical_expr::{JoinType, Operator}; -use datafusion::physical_expr::expressions::GetFieldAccessExpr; -use datafusion::physical_expr::expressions::{cast, in_list}; +use datafusion::logical_expr::{ + create_udf, BuiltinScalarFunction, JoinType, Operator, Volatility, +}; use datafusion::physical_expr::window::SlidingAggregateWindowExpr; use datafusion::physical_expr::ScalarFunctionExpr; -use datafusion::physical_plan::aggregates::PhysicalGroupBy; -use datafusion::physical_plan::aggregates::{AggregateExec, AggregateMode}; +use datafusion::physical_plan::aggregates::{ + AggregateExec, AggregateMode, PhysicalGroupBy, +}; use datafusion::physical_plan::analyze::AnalyzeExec; use datafusion::physical_plan::empty::EmptyExec; use datafusion::physical_plan::expressions::{ - binary, col, like, lit, Avg, BinaryExpr, Column, DistinctCount, GetIndexedFieldExpr, - NotExpr, NthValue, PhysicalSortExpr, Sum, + binary, cast, col, in_list, like, lit, Avg, BinaryExpr, Column, DistinctCount, + GetFieldAccessExpr, GetIndexedFieldExpr, NotExpr, NthValue, PhysicalSortExpr, Sum, }; use datafusion::physical_plan::filter::FilterExec; use datafusion::physical_plan::functions::make_scalar_function; @@ -49,10 +48,12 @@ use datafusion::physical_plan::sorts::sort::SortExec; use datafusion::physical_plan::windows::{ BuiltInWindowExpr, PlainAggregateWindowExpr, WindowAggExec, }; -use datafusion::physical_plan::{functions, udaf}; -use datafusion::physical_plan::{AggregateExpr, ExecutionPlan, PhysicalExpr, Statistics}; +use datafusion::physical_plan::{ + functions, udaf, AggregateExpr, ExecutionPlan, PhysicalExpr, Statistics, +}; use datafusion::prelude::SessionContext; use datafusion::scalar::ScalarValue; +use datafusion_common::stats::Precision; use datafusion_common::Result; use datafusion_expr::{ Accumulator, AccumulatorFactoryFunction, AggregateUDF, ReturnTypeFunction, Signature, @@ -472,10 +473,11 @@ fn roundtrip_parquet_exec_with_pruning_predicate() -> Result<()> { 1024, )]], statistics: Statistics { - num_rows: Some(100), - total_byte_size: Some(1024), - column_statistics: None, - is_exact: false, + num_rows: Precision::Inexact(100), + total_byte_size: Precision::Inexact(1024), + column_statistics: Statistics::unknown_column(&Arc::new(Schema::new(vec![ + Field::new("col", DataType::Utf8, false), + ]))), }, projection: None, limit: None, diff --git a/datafusion/sqllogictest/test_files/explain.slt b/datafusion/sqllogictest/test_files/explain.slt index 0e190a6acd62..066a31590ccd 100644 --- a/datafusion/sqllogictest/test_files/explain.slt +++ b/datafusion/sqllogictest/test_files/explain.slt @@ -273,8 +273,8 @@ query TT EXPLAIN SELECT a, b, c FROM simple_explain_test limit 10; ---- physical_plan -GlobalLimitExec: skip=0, fetch=10, statistics=[rows=10, bytes=None, exact=false] ---CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/example.csv]]}, projection=[a, b, c], limit=10, has_header=true, statistics=[] +GlobalLimitExec: skip=0, fetch=10, statistics=[Rows=Exact(10), Bytes=Absent] +--CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/example.csv]]}, projection=[a, b, c], limit=10, has_header=true, statistics=[Rows=Absent, Bytes=Absent] # Parquet scan with statistics collected statement ok @@ -287,8 +287,8 @@ query TT EXPLAIN SELECT * FROM alltypes_plain limit 10; ---- physical_plan -GlobalLimitExec: skip=0, fetch=10, statistics=[rows=8, bytes=None, exact=true] ---ParquetExec: file_groups={1 group: [[WORKSPACE_ROOT/parquet-testing/data/alltypes_plain.parquet]]}, projection=[id, bool_col, tinyint_col, smallint_col, int_col, bigint_col, float_col, double_col, date_string_col, string_col, timestamp_col], limit=10, statistics=[rows=8, bytes=None, exact=true] +GlobalLimitExec: skip=0, fetch=10, statistics=[Rows=Exact(8), Bytes=Absent] +--ParquetExec: file_groups={1 group: [[WORKSPACE_ROOT/parquet-testing/data/alltypes_plain.parquet]]}, projection=[id, bool_col, tinyint_col, smallint_col, int_col, bigint_col, float_col, double_col, date_string_col, string_col, timestamp_col], limit=10, statistics=[Rows=Exact(8), Bytes=Absent] statement ok set datafusion.execution.collect_statistics = false; diff --git a/datafusion/sqllogictest/test_files/window.slt b/datafusion/sqllogictest/test_files/window.slt index aa7b3d5bb99c..7226182cf3d0 100644 --- a/datafusion/sqllogictest/test_files/window.slt +++ b/datafusion/sqllogictest/test_files/window.slt @@ -2010,14 +2010,10 @@ Projection: ARRAY_AGG(aggregate_test_100.c13) AS array_agg1 --------TableScan: aggregate_test_100 projection=[c13] physical_plan ProjectionExec: expr=[ARRAY_AGG(aggregate_test_100.c13)@0 as array_agg1] ---AggregateExec: mode=Final, gby=[], aggr=[ARRAY_AGG(aggregate_test_100.c13)] -----CoalescePartitionsExec -------AggregateExec: mode=Partial, gby=[], aggr=[ARRAY_AGG(aggregate_test_100.c13)] ---------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -----------GlobalLimitExec: skip=0, fetch=1 -------------SortExec: TopK(fetch=1), expr=[c13@0 ASC NULLS LAST] ---------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c13], has_header=true - +--AggregateExec: mode=Single, gby=[], aggr=[ARRAY_AGG(aggregate_test_100.c13)] +----GlobalLimitExec: skip=0, fetch=1 +------SortExec: TopK(fetch=1), expr=[c13@0 ASC NULLS LAST] +--------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c13], has_header=true query ? SELECT ARRAY_AGG(c13) as array_agg1 FROM (SELECT * FROM aggregate_test_100 ORDER BY c13 LIMIT 1) diff --git a/datafusion/substrait/src/physical_plan/consumer.rs b/datafusion/substrait/src/physical_plan/consumer.rs index 8d25626a3bfe..7788ba0a69de 100644 --- a/datafusion/substrait/src/physical_plan/consumer.rs +++ b/datafusion/substrait/src/physical_plan/consumer.rs @@ -15,19 +15,21 @@ // specific language governing permissions and limitations // under the License. -use async_recursion::async_recursion; -use chrono::DateTime; +use std::collections::HashMap; +use std::sync::Arc; + use datafusion::arrow::datatypes::Schema; use datafusion::common::not_impl_err; use datafusion::datasource::listing::PartitionedFile; use datafusion::datasource::object_store::ObjectStoreUrl; use datafusion::datasource::physical_plan::{FileScanConfig, ParquetExec}; use datafusion::error::{DataFusionError, Result}; -use datafusion::physical_plan::ExecutionPlan; +use datafusion::physical_plan::{ExecutionPlan, Statistics}; use datafusion::prelude::SessionContext; + +use async_recursion::async_recursion; +use chrono::DateTime; use object_store::ObjectMeta; -use std::collections::HashMap; -use std::sync::Arc; use substrait::proto::read_rel::local_files::file_or_files::PathType; use substrait::proto::{ expression::MaskExpression, read_rel::ReadType, rel::RelType, Rel, @@ -104,7 +106,7 @@ pub async fn from_substrait_rel( object_store_url: ObjectStoreUrl::local_filesystem(), file_schema: Arc::new(Schema::empty()), file_groups, - statistics: Default::default(), + statistics: Statistics::new_unknown(&Schema::empty()), projection: None, limit: None, table_partition_cols: vec![], diff --git a/datafusion/substrait/tests/cases/roundtrip_physical_plan.rs b/datafusion/substrait/tests/cases/roundtrip_physical_plan.rs index 25d60471a9cd..3e5e757e4c39 100644 --- a/datafusion/substrait/tests/cases/roundtrip_physical_plan.rs +++ b/datafusion/substrait/tests/cases/roundtrip_physical_plan.rs @@ -15,16 +15,18 @@ // specific language governing permissions and limitations // under the License. +use std::collections::HashMap; +use std::sync::Arc; + use datafusion::arrow::datatypes::Schema; use datafusion::datasource::listing::PartitionedFile; use datafusion::datasource::object_store::ObjectStoreUrl; use datafusion::datasource::physical_plan::{FileScanConfig, ParquetExec}; use datafusion::error::Result; -use datafusion::physical_plan::{displayable, ExecutionPlan}; +use datafusion::physical_plan::{displayable, ExecutionPlan, Statistics}; use datafusion::prelude::SessionContext; use datafusion_substrait::physical_plan::{consumer, producer}; -use std::collections::HashMap; -use std::sync::Arc; + use substrait::proto::extensions; #[tokio::test] @@ -42,7 +44,7 @@ async fn parquet_exec() -> Result<()> { 123, )], ], - statistics: Default::default(), + statistics: Statistics::new_unknown(&Schema::empty()), projection: None, limit: None, table_partition_cols: vec![], From efbd1043d427ebf1783d164c90e45aee1766f12e Mon Sep 17 00:00:00 2001 From: Eugene Marushchenko Date: Wed, 18 Oct 2023 00:38:00 +1000 Subject: [PATCH 086/572] Refactor ScalarValue::new_primitive to return Result (#7830) Co-authored-by: Evgeny Maruschenko --- datafusion/common/src/scalar.rs | 8 ++++---- .../src/aggregate/bit_and_or_xor.rs | 12 ++++++------ .../physical-expr/src/aggregate/median.rs | 6 +++--- datafusion/physical-expr/src/aggregate/sum.rs | 4 ++-- .../physical-expr/src/aggregate/sum_distinct.rs | 17 +++++++++-------- 5 files changed, 24 insertions(+), 23 deletions(-) diff --git a/datafusion/common/src/scalar.rs b/datafusion/common/src/scalar.rs index 2d47b3e31472..2c3dd4c5ca55 100644 --- a/datafusion/common/src/scalar.rs +++ b/datafusion/common/src/scalar.rs @@ -744,7 +744,7 @@ macro_rules! eq_array_primitive { } impl ScalarValue { - /// Create a [`ScalarValue`] with the provided value and datatype + /// Create a [`Result`] with the provided value and datatype /// /// # Panics /// @@ -752,13 +752,13 @@ impl ScalarValue { pub fn new_primitive( a: Option, d: &DataType, - ) -> Self { + ) -> Result { match a { - None => d.try_into().unwrap(), + None => d.try_into(), Some(v) => { let array = PrimitiveArray::::new(vec![v].into(), None) .with_data_type(d.clone()); - Self::try_from_array(&array, 0).unwrap() + Self::try_from_array(&array, 0) } } } diff --git a/datafusion/physical-expr/src/aggregate/bit_and_or_xor.rs b/datafusion/physical-expr/src/aggregate/bit_and_or_xor.rs index d7934e79c366..6c97d620616a 100644 --- a/datafusion/physical-expr/src/aggregate/bit_and_or_xor.rs +++ b/datafusion/physical-expr/src/aggregate/bit_and_or_xor.rs @@ -195,7 +195,7 @@ where } fn evaluate(&self) -> Result { - Ok(ScalarValue::new_primitive::(self.value, &T::DATA_TYPE)) + ScalarValue::new_primitive::(self.value, &T::DATA_TYPE) } fn size(&self) -> usize { @@ -356,7 +356,7 @@ where } fn evaluate(&self) -> Result { - Ok(ScalarValue::new_primitive::(self.value, &T::DATA_TYPE)) + ScalarValue::new_primitive::(self.value, &T::DATA_TYPE) } fn size(&self) -> usize { @@ -517,7 +517,7 @@ where } fn evaluate(&self) -> Result { - Ok(ScalarValue::new_primitive::(self.value, &T::DATA_TYPE)) + ScalarValue::new_primitive::(self.value, &T::DATA_TYPE) } fn size(&self) -> usize { @@ -638,11 +638,11 @@ where // 1. Stores aggregate state in `ScalarValue::List` // 2. Constructs `ScalarValue::List` state from distinct numeric stored in hash set let state_out = { - let values: Vec = self + let values = self .values .iter() .map(|x| ScalarValue::new_primitive::(Some(*x), &T::DATA_TYPE)) - .collect(); + .collect::>>()?; let arr = ScalarValue::new_list(&values, &T::DATA_TYPE); vec![ScalarValue::List(arr)] @@ -685,7 +685,7 @@ where acc = acc ^ *distinct_value; } let v = (!self.values.is_empty()).then_some(acc); - Ok(ScalarValue::new_primitive::(v, &T::DATA_TYPE)) + ScalarValue::new_primitive::(v, &T::DATA_TYPE) } fn size(&self) -> usize { diff --git a/datafusion/physical-expr/src/aggregate/median.rs b/datafusion/physical-expr/src/aggregate/median.rs index 477dcadceee7..691b1c1752f4 100644 --- a/datafusion/physical-expr/src/aggregate/median.rs +++ b/datafusion/physical-expr/src/aggregate/median.rs @@ -146,11 +146,11 @@ impl std::fmt::Debug for MedianAccumulator { impl Accumulator for MedianAccumulator { fn state(&self) -> Result> { - let all_values: Vec = self + let all_values = self .all_values .iter() .map(|x| ScalarValue::new_primitive::(Some(*x), &self.data_type)) - .collect(); + .collect::>>()?; let arr = ScalarValue::new_list(&all_values, &self.data_type); Ok(vec![ScalarValue::List(arr)]) @@ -188,7 +188,7 @@ impl Accumulator for MedianAccumulator { let (_, median, _) = d.select_nth_unstable_by(len / 2, cmp); Some(*median) }; - Ok(ScalarValue::new_primitive::(median, &self.data_type)) + ScalarValue::new_primitive::(median, &self.data_type) } fn size(&self) -> usize { diff --git a/datafusion/physical-expr/src/aggregate/sum.rs b/datafusion/physical-expr/src/aggregate/sum.rs index 5cc8e933324e..d6c23d0dfafd 100644 --- a/datafusion/physical-expr/src/aggregate/sum.rs +++ b/datafusion/physical-expr/src/aggregate/sum.rs @@ -205,7 +205,7 @@ impl Accumulator for SumAccumulator { } fn evaluate(&self) -> Result { - Ok(ScalarValue::new_primitive::(self.sum, &self.data_type)) + ScalarValue::new_primitive::(self.sum, &self.data_type) } fn size(&self) -> usize { @@ -265,7 +265,7 @@ impl Accumulator for SlidingSumAccumulator { fn evaluate(&self) -> Result { let v = (self.count != 0).then_some(self.sum); - Ok(ScalarValue::new_primitive::(v, &self.data_type)) + ScalarValue::new_primitive::(v, &self.data_type) } fn size(&self) -> usize { diff --git a/datafusion/physical-expr/src/aggregate/sum_distinct.rs b/datafusion/physical-expr/src/aggregate/sum_distinct.rs index 742e24b99e71..ef1bd039a5ea 100644 --- a/datafusion/physical-expr/src/aggregate/sum_distinct.rs +++ b/datafusion/physical-expr/src/aggregate/sum_distinct.rs @@ -159,13 +159,14 @@ impl Accumulator for DistinctSumAccumulator { // 1. Stores aggregate state in `ScalarValue::List` // 2. Constructs `ScalarValue::List` state from distinct numeric stored in hash set let state_out = { - let mut distinct_values = Vec::new(); - self.values.iter().for_each(|distinct_value| { - distinct_values.push(ScalarValue::new_primitive::( - Some(distinct_value.0), - &self.data_type, - )) - }); + let distinct_values = self + .values + .iter() + .map(|value| { + ScalarValue::new_primitive::(Some(value.0), &self.data_type) + }) + .collect::>>()?; + vec![ScalarValue::List(ScalarValue::new_list( &distinct_values, &self.data_type, @@ -206,7 +207,7 @@ impl Accumulator for DistinctSumAccumulator { acc = acc.add_wrapping(distinct_value.0) } let v = (!self.values.is_empty()).then_some(acc); - Ok(ScalarValue::new_primitive::(v, &self.data_type)) + ScalarValue::new_primitive::(v, &self.data_type) } fn size(&self) -> usize { From d009d48ee927caae40dfd51c74c6893b69c0cd57 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Dani=C3=ABl=20Heres?= Date: Tue, 17 Oct 2023 20:15:28 +0200 Subject: [PATCH 087/572] Fix precision loss when coercing date_part utf8 argument (#7846) --- datafusion/core/tests/sql/expr.rs | 19 +++++++++++++++++++ datafusion/expr/src/built_in_function.rs | 20 ++++++++++---------- 2 files changed, 29 insertions(+), 10 deletions(-) diff --git a/datafusion/core/tests/sql/expr.rs b/datafusion/core/tests/sql/expr.rs index af33cfea65af..1995a040158d 100644 --- a/datafusion/core/tests/sql/expr.rs +++ b/datafusion/core/tests/sql/expr.rs @@ -717,6 +717,25 @@ async fn test_extract_date_part() -> Result<()> { "date_part('nanosecond', to_timestamp('2020-09-08T12:00:12.12345678+00:00'))", "1.212345678e10" ); + + // Keep precision when coercing Utf8 to Timestamp + test_expression!( + "date_part('second', '2020-09-08T12:00:12.12345678+00:00')", + "12.12345678" + ); + test_expression!( + "date_part('millisecond', '2020-09-08T12:00:12.12345678+00:00')", + "12123.45678" + ); + test_expression!( + "date_part('microsecond', '2020-09-08T12:00:12.12345678+00:00')", + "12123456.78" + ); + test_expression!( + "date_part('nanosecond', '2020-09-08T12:00:12.12345678+00:00')", + "1.212345678e10" + ); + Ok(()) } diff --git a/datafusion/expr/src/built_in_function.rs b/datafusion/expr/src/built_in_function.rs index d7cfd9f4209e..f1af6e829b6f 100644 --- a/datafusion/expr/src/built_in_function.rs +++ b/datafusion/expr/src/built_in_function.rs @@ -1107,28 +1107,28 @@ impl BuiltinScalarFunction { } BuiltinScalarFunction::DatePart => Signature::one_of( vec![ - Exact(vec![Utf8, Date32]), - Exact(vec![Utf8, Date64]), - Exact(vec![Utf8, Timestamp(Second, None)]), + Exact(vec![Utf8, Timestamp(Nanosecond, None)]), Exact(vec![ Utf8, - Timestamp(Second, Some(TIMEZONE_WILDCARD.into())), + Timestamp(Nanosecond, Some(TIMEZONE_WILDCARD.into())), ]), - Exact(vec![Utf8, Timestamp(Microsecond, None)]), + Exact(vec![Utf8, Timestamp(Millisecond, None)]), Exact(vec![ Utf8, - Timestamp(Microsecond, Some(TIMEZONE_WILDCARD.into())), + Timestamp(Millisecond, Some(TIMEZONE_WILDCARD.into())), ]), - Exact(vec![Utf8, Timestamp(Millisecond, None)]), + Exact(vec![Utf8, Timestamp(Microsecond, None)]), Exact(vec![ Utf8, - Timestamp(Millisecond, Some(TIMEZONE_WILDCARD.into())), + Timestamp(Microsecond, Some(TIMEZONE_WILDCARD.into())), ]), - Exact(vec![Utf8, Timestamp(Nanosecond, None)]), + Exact(vec![Utf8, Timestamp(Second, None)]), Exact(vec![ Utf8, - Timestamp(Nanosecond, Some(TIMEZONE_WILDCARD.into())), + Timestamp(Second, Some(TIMEZONE_WILDCARD.into())), ]), + Exact(vec![Utf8, Date64]), + Exact(vec![Utf8, Date32]), ], self.volatility(), ), From 360175aec4f5dfa43f6d1ab2f4665a4bd397ae83 Mon Sep 17 00:00:00 2001 From: Chih Wang Date: Wed, 18 Oct 2023 04:16:40 +0800 Subject: [PATCH 088/572] Add operator section to user guide, Add `std::ops` operations to `prelude`, and add `not()` expr_fn (#7732) * Add logical tests of Expr and logical not function * Re-export std::ops::* in prelude * Update the Expression API documentation, mainly covering boolean, bitwise, comparison, and arithmetic expressions --- datafusion/core/src/prelude.rs | 5 + datafusion/expr/src/expr.rs | 36 +++++++ datafusion/expr/src/expr_fn.rs | 6 ++ datafusion/expr/src/operator.rs | 1 + docs/source/conf.py | 2 +- docs/source/user-guide/expressions.md | 144 ++++++++++++++++---------- 6 files changed, 136 insertions(+), 58 deletions(-) diff --git a/datafusion/core/src/prelude.rs b/datafusion/core/src/prelude.rs index 3782feca191a..7689468e5d13 100644 --- a/datafusion/core/src/prelude.rs +++ b/datafusion/core/src/prelude.rs @@ -38,3 +38,8 @@ pub use datafusion_expr::{ logical_plan::{JoinType, Partitioning}, Expr, }; + +pub use std::ops::Not; +pub use std::ops::{Add, Div, Mul, Neg, Rem, Sub}; +pub use std::ops::{BitAnd, BitOr, BitXor}; +pub use std::ops::{Shl, Shr}; diff --git a/datafusion/expr/src/expr.rs b/datafusion/expr/src/expr.rs index 0b166107fb4b..239a3188502c 100644 --- a/datafusion/expr/src/expr.rs +++ b/datafusion/expr/src/expr.rs @@ -1702,4 +1702,40 @@ mod test { Ok(()) } + + #[test] + fn test_logical_ops() { + assert_eq!( + format!("{}", lit(1u32).eq(lit(2u32))), + "UInt32(1) = UInt32(2)" + ); + assert_eq!( + format!("{}", lit(1u32).not_eq(lit(2u32))), + "UInt32(1) != UInt32(2)" + ); + assert_eq!( + format!("{}", lit(1u32).gt(lit(2u32))), + "UInt32(1) > UInt32(2)" + ); + assert_eq!( + format!("{}", lit(1u32).gt_eq(lit(2u32))), + "UInt32(1) >= UInt32(2)" + ); + assert_eq!( + format!("{}", lit(1u32).lt(lit(2u32))), + "UInt32(1) < UInt32(2)" + ); + assert_eq!( + format!("{}", lit(1u32).lt_eq(lit(2u32))), + "UInt32(1) <= UInt32(2)" + ); + assert_eq!( + format!("{}", lit(1u32).and(lit(2u32))), + "UInt32(1) AND UInt32(2)" + ); + assert_eq!( + format!("{}", lit(1u32).or(lit(2u32))), + "UInt32(1) OR UInt32(2)" + ); + } } diff --git a/datafusion/expr/src/expr_fn.rs b/datafusion/expr/src/expr_fn.rs index 79a43c2353db..5368a2d8a22c 100644 --- a/datafusion/expr/src/expr_fn.rs +++ b/datafusion/expr/src/expr_fn.rs @@ -31,6 +31,7 @@ use crate::{ }; use arrow::datatypes::DataType; use datafusion_common::{Column, Result}; +use std::ops::Not; use std::sync::Arc; /// Create a column expression based on a qualified or unqualified column name. Will @@ -121,6 +122,11 @@ pub fn or(left: Expr, right: Expr) -> Expr { )) } +/// Return a new expression with a logical NOT +pub fn not(expr: Expr) -> Expr { + expr.not() +} + /// Create an expression to represent the min() aggregate function pub fn min(expr: Expr) -> Expr { Expr::AggregateFunction(AggregateFunction::new( diff --git a/datafusion/expr/src/operator.rs b/datafusion/expr/src/operator.rs index 112e29082dba..1790f1478927 100644 --- a/datafusion/expr/src/operator.rs +++ b/datafusion/expr/src/operator.rs @@ -363,6 +363,7 @@ impl ops::Neg for Expr { } } +/// Support `NOT ` fluent style impl Not for Expr { type Output = Self; diff --git a/docs/source/conf.py b/docs/source/conf.py index 9aa84d49bc0a..3fa6c6091d6f 100644 --- a/docs/source/conf.py +++ b/docs/source/conf.py @@ -118,4 +118,4 @@ myst_heading_anchors = 3 # enable nice rendering of checkboxes for the task lists -myst_enable_extensions = [ "tasklist"] +myst_enable_extensions = ["colon_fence", "deflist", "tasklist"] diff --git a/docs/source/user-guide/expressions.md b/docs/source/user-guide/expressions.md index dbd8c814b46e..28104dbfd445 100644 --- a/docs/source/user-guide/expressions.md +++ b/docs/source/user-guide/expressions.md @@ -22,60 +22,99 @@ DataFrame methods such as `select` and `filter` accept one or more logical expressions and there are many functions available for creating logical expressions. These are documented below. -Expressions can be chained together using a fluent-style API: +:::{tip} +Most functions and methods may receive and return an `Expr`, which can be chained together using a fluent-style API: ```rust // create the expression `(a > 6) AND (b < 7)` col("a").gt(lit(6)).and(col("b").lt(lit(7))) ``` +::: + ## Identifiers -| Function | Notes | -| -------- | -------------------------------------------- | -| col | Reference a column in a dataframe `col("a")` | +| Syntax | Description | +| ---------- | -------------------------------------------- | +| col(ident) | Reference a column in a dataframe `col("a")` | + +:::{note} +ident +: A type which implement `Into` trait +::: ## Literal Values -| Function | Notes | -| -------- | -------------------------------------------------- | -| lit | Literal value such as `lit(123)` or `lit("hello")` | +| Syntax | Description | +| ---------- | -------------------------------------------------- | +| lit(value) | Literal value such as `lit(123)` or `lit("hello")` | + +:::{note} +value +: A type which implement `Literal` +::: ## Boolean Expressions -| Function | Notes | -| -------- | ----------------------------------------- | -| and | `and(expr1, expr2)` or `expr1.and(expr2)` | -| or | `or(expr1, expr2)` or `expr1.or(expr2)` | -| not | `not(expr)` or `expr.not()` | +| Syntax | Description | +| ------------------- | ----------- | +| and(x, y), x.and(y) | Logical AND | +| or(x, y), x.or(y) | Logical OR | +| !x, not(x), x.not() | Logical NOT | + +:::{note} +`!` is a bitwise or logical complement operator in Rust, but it only works as a logical NOT in expression API. +::: + +:::{note} +Since `&&` and `||` are existed as logical operators in Rust, but those are not overloadable and not works with expression API. +::: -## Bitwise expressions +## Bitwise Expressions -| Function | Notes | -| ------------------- | ------------------------------------------------------------------------- | -| bitwise_and | `bitwise_and(expr1, expr2)` or `expr1.bitwise_and(expr2)` | -| bitwise_or | `bitwise_or(expr1, expr2)` or `expr1.bitwise_or(expr2)` | -| bitwise_xor | `bitwise_xor(expr1, expr2)` or `expr1.bitwise_xor(expr2)` | -| bitwise_shift_right | `bitwise_shift_right(expr1, expr2)` or `expr1.bitwise_shift_right(expr2)` | -| bitwise_shift_left | `bitwise_shift_left(expr1, expr2)` or `expr1.bitwise_shift_left(expr2)` | +| Syntax | Description | +| ------------------------------------------- | ----------- | +| x & y, bitwise_and(x, y), x.bitand(y) | AND | +| x \| y, bitwise_or(x, y), x.bitor(y) | OR | +| x ^ y, bitwise_xor(x, y), x.bitxor(y) | XOR | +| x << y, bitwise_shift_left(x, y), x.shl(y) | Left shift | +| x >> y, bitwise_shift_right(x, y), x.shr(y) | Right shift | ## Comparison Expressions -| Function | Notes | -| -------- | --------------------- | -| eq | `expr1.eq(expr2)` | -| gt | `expr1.gt(expr2)` | -| gt_eq | `expr1.gt_eq(expr2)` | -| lt | `expr1.lt(expr2)` | -| lt_eq | `expr1.lt_eq(expr2)` | -| not_eq | `expr1.not_eq(expr2)` | +| Syntax | Description | +| ----------- | --------------------- | +| x.eq(y) | Equal | +| x.not_eq(y) | Not Equal | +| x.gt(y) | Greater Than | +| x.gt_eq(y) | Greater Than or Equal | +| x.lt(y) | Less Than | +| x.lt_eq(y) | Less Than or Equal | + +:::{note} +Comparison operators (`<`, `<=`, `==`, `>=`, `>`) could be overloaded by the `PartialOrd` and `PartialEq` trait in Rust, +but these operators always return a `bool` which makes them not work with the expression API. +::: + +## Arithmetic Expressions + +| Syntax | Description | +| ---------------- | -------------- | +| x + y, x.add(y) | Addition | +| x - y, x.sub(y) | Subtraction | +| x \* y, x.mul(y) | Multiplication | +| x / y, x.div(y) | Division | +| x % y, x.rem(y) | Remainder | +| -x, x.neg() | Negation | + +:::{note} +In Rust, the keyword `mod` is reserved and cannot be used as an identifier. +To avoid any conflicts and ensure code completion works smoothly, we use `mod_` instead. +::: ## Math Functions -In addition to the math functions listed here, some Rust operators are implemented for expressions, allowing -expressions such as `col("a") + col("b")` to be used. - -| Function | Notes | +| Syntax | Description | | --------------------- | ------------------------------------------------- | | abs(x) | absolute value | | acos(x) | inverse cosine | @@ -114,11 +153,10 @@ expressions such as `col("a") + col("b")` to be used. | tanh(x) | hyperbolic tangent | | trunc(x) | truncate toward zero | -### Math functions usage notes: - +:::{note} Unlike to some databases the math functions in Datafusion works the same way as Rust math functions, avoiding failing on corner cases e.g -``` +```sql ❯ select log(-1), log(0), sqrt(-1); +----------------+---------------+-----------------+ | log(Int64(-1)) | log(Int64(0)) | sqrt(Int64(-1)) | @@ -127,27 +165,19 @@ Unlike to some databases the math functions in Datafusion works the same way as +----------------+---------------+-----------------+ ``` -## Bitwise Operators - -| Operator | Notes | -| -------- | ----------------------------------------------- | -| & | Bitwise AND => `(expr1 & expr2)` | -| | | Bitwise OR => (expr1 | expr2) | -| # | Bitwise XOR => `(expr1 # expr2)` | -| << | Bitwise left shift => `(expr1 << expr2)` | -| >> | Bitwise right shift => `(expr1 << expr2)` | +::: ## Conditional Expressions -| Function | Notes | -| -------- | --------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------- | -| coalesce | Returns the first of its arguments that is not null. Null is returned only if all arguments are null. It is often used to substitute a default value for null values when data is retrieved for display. | -| case | CASE expression. The expression may chain multiple `when` expressions and end with an `end` or `otherwise` expression. Example:
case(col("a") % lit(3))
    .when(lit(0), lit("A"))
    .when(lit(1), lit("B"))
    .when(lit(2), lit("C"))
    .end()
or, end with `otherwise` to match any other conditions:
case(col("b").gt(lit(100)))
    .when(lit(true), lit("value > 100"))
    .otherwise(lit("value <= 100"))
| -| nullif | Returns a null value if `value1` equals `value2`; otherwise it returns `value1`. This can be used to perform the inverse operation of the `coalesce` expression. | +| Syntax | Description | +| ------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------ | --------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------- | +| coalesce([value, ...]) | Returns the first of its arguments that is not null. Null is returned only if all arguments are null. It is often used to substitute a default value for null values when data is retrieved for display. | +| case(expr)
    .when(expr)
    .end(),
case(expr)
    .when(expr)
    .otherwise(expr) | CASE expression. The expression may chain multiple `when` expressions and end with an `end` or `otherwise` expression. Example:
case(col("a") % lit(3))
    .when(lit(0), lit("A"))
    .when(lit(1), lit("B"))
    .when(lit(2), lit("C"))
    .end()
or, end with `otherwise` to match any other conditions:
case(col("b").gt(lit(100)))
    .when(lit(true), lit("value > 100"))
    .otherwise(lit("value <= 100"))
| +| nullif(value1, value2) | Returns a null value if `value1` equals `value2`; otherwise it returns `value1`. This can be used to perform the inverse operation of the `coalesce` expression. | ## String Expressions -| Function | Notes | +| Syntax | Description | | ---------------------------------------------- | ---------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------- | | ascii(character) | Returns a numeric representation of the character (`character`). Example: `ascii('a') -> 97` | | bit_length(text) | Returns the length of the string (`text`) in bits. Example: `bit_length('spider') -> 48` | @@ -182,7 +212,7 @@ Unlike to some databases the math functions in Datafusion works the same way as ## Array Expressions -| Function | Notes | +| Syntax | Description | | ------------------------------------- | ------------------------------------------------------------------------------------------------------------------------------------------------------------------------ | | array_append(array, element) | Appends an element to the end of an array. `array_append([1, 2, 3], 4) -> [1, 2, 3, 4]` | | array_concat(array[, ..., array_n]) | Concatenates arrays. `array_concat([1, 2, 3], [4, 5, 6]) -> [1, 2, 3, 4, 5, 6]` | @@ -213,14 +243,14 @@ Unlike to some databases the math functions in Datafusion works the same way as ## Regular Expressions -| Function | Notes | +| Syntax | Description | | -------------- | ----------------------------------------------------------------------------- | | regexp_match | Matches a regular expression against a string and returns matched substrings. | | regexp_replace | Replaces strings that match a regular expression | ## Temporal Expressions -| Function | Notes | +| Syntax | Description | | -------------------- | ------------------------------------------------------ | | date_part | Extracts a subfield from the date. | | date_trunc | Truncates the date to a specified level of precision. | @@ -233,7 +263,7 @@ Unlike to some databases the math functions in Datafusion works the same way as ## Other Expressions -| Function | Notes | +| Syntax | Description | | ---------------------------- | ---------------------------------------------------------------------------------------------------------- | | array([value1, ...]) | Returns an array of fixed size with each argument (`[value1, ...]`) on it. | | in_list(expr, list, negated) | Returns `true` if (`expr`) belongs or not belongs (`negated`) to a list (`list`), otherwise returns false. | @@ -246,7 +276,7 @@ Unlike to some databases the math functions in Datafusion works the same way as ## Aggregate Functions -| Function | Notes | +| Syntax | Description | | ----------------------------------------------------------------- | --------------------------------------------------------------------------------------- | | avg(expr) | Сalculates the average value for `expr`. | | approx_distinct(expr) | Calculates an approximate count of the number of distinct values for `expr`. | @@ -270,7 +300,7 @@ Unlike to some databases the math functions in Datafusion works the same way as ## Subquery Expressions -| Function | Notes | +| Syntax | Description | | --------------- | --------------------------------------------------------------------------------------------- | | exists | Creates an `EXISTS` subquery expression | | in_subquery | `df1.filter(in_subquery(col("foo"), df2))?` is the equivalent of the SQL `WHERE foo IN ` | @@ -280,7 +310,7 @@ Unlike to some databases the math functions in Datafusion works the same way as ## User-Defined Function Expressions -| Function | Notes | +| Syntax | Description | | ----------- | ------------------------------------------------------------------------- | | create_udf | Creates a new UDF with a specific signature and specific return type. | | create_udaf | Creates a new UDAF with a specific signature, state type and return type. | From 511be33dc7d9164a2ff330712729d5ba3da706d6 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E5=BC=A0=E6=9E=97=E4=BC=9F?= Date: Wed, 18 Oct 2023 04:17:16 +0800 Subject: [PATCH 089/572] Expose arrow-schema `serde` crate feature flag (#7829) * Expose serde feature * Update README.md --- README.md | 1 + datafusion/core/Cargo.toml | 1 + 2 files changed, 2 insertions(+) diff --git a/README.md b/README.md index 63da8c1c1a96..2f10812f9a59 100644 --- a/README.md +++ b/README.md @@ -55,6 +55,7 @@ Optional features: - `avro`: support for reading the [Apache Avro] format - `backtrace`: include backtrace information in error messages - `pyarrow`: conversions between PyArrow and DataFusion types +- `serde`: enable arrow-schema's `serde` feature - `simd`: enable arrow-rs's manual `SIMD` kernels (requires Rust `nightly`) [apache avro]: https://avro.apache.org/ diff --git a/datafusion/core/Cargo.toml b/datafusion/core/Cargo.toml index fa580c914ce2..30e0d005e92e 100644 --- a/datafusion/core/Cargo.toml +++ b/datafusion/core/Cargo.toml @@ -45,6 +45,7 @@ encoding_expressions = ["datafusion-physical-expr/encoding_expressions"] force_hash_collisions = [] pyarrow = ["datafusion-common/pyarrow"] regex_expressions = ["datafusion-physical-expr/regex_expressions", "datafusion-optimizer/regex_expressions"] +serde = ["arrow-schema/serde"] simd = ["arrow/simd"] unicode_expressions = ["datafusion-physical-expr/unicode_expressions", "datafusion-optimizer/unicode_expressions", "datafusion-sql/unicode_expressions"] From 033b2ef2fdd869980ad7ed1c71bbfe3b604a56b9 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E5=BC=A0=E6=9E=97=E4=BC=9F?= Date: Wed, 18 Oct 2023 04:17:34 +0800 Subject: [PATCH 090/572] Improve `ContextProvider` naming: rename` get_table_provider` --> `get_table_source`, deprecate `get_table_provider` (#7831) * Improve naming * Add deprecated warning * Improve more * Improve tests --- datafusion-examples/examples/rewrite_expr.rs | 2 +- datafusion/core/src/execution/context.rs | 2 +- .../optimizer/tests/optimizer_integration.rs | 10 ++--- datafusion/sql/examples/sql.rs | 12 +++--- datafusion/sql/src/expr/function.rs | 8 ++-- datafusion/sql/src/expr/identifier.rs | 4 +- datafusion/sql/src/expr/mod.rs | 15 +++----- datafusion/sql/src/planner.rs | 18 +++++---- datafusion/sql/src/relation/mod.rs | 2 +- datafusion/sql/src/statement.rs | 37 +++++++++---------- datafusion/sql/tests/sql_integration.rs | 2 +- 11 files changed, 56 insertions(+), 56 deletions(-) diff --git a/datafusion-examples/examples/rewrite_expr.rs b/datafusion-examples/examples/rewrite_expr.rs index e657baab3df8..5e95562033e6 100644 --- a/datafusion-examples/examples/rewrite_expr.rs +++ b/datafusion-examples/examples/rewrite_expr.rs @@ -191,7 +191,7 @@ struct MyContextProvider { } impl ContextProvider for MyContextProvider { - fn get_table_provider(&self, name: TableReference) -> Result> { + fn get_table_source(&self, name: TableReference) -> Result> { if name.table() == "person" { Ok(Arc::new(MyTableSource { schema: Arc::new(Schema::new(vec![ diff --git a/datafusion/core/src/execution/context.rs b/datafusion/core/src/execution/context.rs index b393d363ca0b..8bd4de742d69 100644 --- a/datafusion/core/src/execution/context.rs +++ b/datafusion/core/src/execution/context.rs @@ -2014,7 +2014,7 @@ struct SessionContextProvider<'a> { } impl<'a> ContextProvider for SessionContextProvider<'a> { - fn get_table_provider(&self, name: TableReference) -> Result> { + fn get_table_source(&self, name: TableReference) -> Result> { let name = self.state.resolve_table_ref(name).to_string(); self.tables .get(&name) diff --git a/datafusion/optimizer/tests/optimizer_integration.rs b/datafusion/optimizer/tests/optimizer_integration.rs index 46023cfc30bc..872071e52fa7 100644 --- a/datafusion/optimizer/tests/optimizer_integration.rs +++ b/datafusion/optimizer/tests/optimizer_integration.rs @@ -339,8 +339,8 @@ fn test_sql(sql: &str) -> Result { let statement = &ast[0]; // create a logical query plan - let schema_provider = MySchemaProvider::default(); - let sql_to_rel = SqlToRel::new(&schema_provider); + let context_provider = MyContextProvider::default(); + let sql_to_rel = SqlToRel::new(&context_provider); let plan = sql_to_rel.sql_statement_to_plan(statement.clone()).unwrap(); // hard code the return value of now() @@ -357,12 +357,12 @@ fn test_sql(sql: &str) -> Result { } #[derive(Default)] -struct MySchemaProvider { +struct MyContextProvider { options: ConfigOptions, } -impl ContextProvider for MySchemaProvider { - fn get_table_provider(&self, name: TableReference) -> Result> { +impl ContextProvider for MyContextProvider { + fn get_table_source(&self, name: TableReference) -> Result> { let table_name = name.table(); if table_name.starts_with("test") { let schema = Schema::new_with_metadata( diff --git a/datafusion/sql/examples/sql.rs b/datafusion/sql/examples/sql.rs index 8a12cc32b641..9df65b99a748 100644 --- a/datafusion/sql/examples/sql.rs +++ b/datafusion/sql/examples/sql.rs @@ -49,20 +49,20 @@ fn main() { let statement = &ast[0]; // create a logical query plan - let schema_provider = MySchemaProvider::new(); - let sql_to_rel = SqlToRel::new(&schema_provider); + let context_provider = MyContextProvider::new(); + let sql_to_rel = SqlToRel::new(&context_provider); let plan = sql_to_rel.sql_statement_to_plan(statement.clone()).unwrap(); // show the plan println!("{plan:?}"); } -struct MySchemaProvider { +struct MyContextProvider { options: ConfigOptions, tables: HashMap>, } -impl MySchemaProvider { +impl MyContextProvider { fn new() -> Self { let mut tables = HashMap::new(); tables.insert( @@ -104,8 +104,8 @@ fn create_table_source(fields: Vec) -> Arc { ))) } -impl ContextProvider for MySchemaProvider { - fn get_table_provider(&self, name: TableReference) -> Result> { +impl ContextProvider for MyContextProvider { + fn get_table_source(&self, name: TableReference) -> Result> { match self.tables.get(name.table()) { Some(table) => Ok(table.clone()), _ => plan_err!("Table not found: {}", name.table()), diff --git a/datafusion/sql/src/expr/function.rs b/datafusion/sql/src/expr/function.rs index ab70853949fd..373388277351 100644 --- a/datafusion/sql/src/expr/function.rs +++ b/datafusion/sql/src/expr/function.rs @@ -49,7 +49,7 @@ impl<'a, S: ContextProvider> SqlToRel<'a, S> { }; // user-defined function (UDF) should have precedence in case it has the same name as a scalar built-in function - if let Some(fm) = self.schema_provider.get_function_meta(&name) { + if let Some(fm) = self.context_provider.get_function_meta(&name) { let args = self.function_args_to_expr(function.args, schema, planner_context)?; return Ok(Expr::ScalarUDF(ScalarUDF::new(fm, args))); @@ -127,7 +127,7 @@ impl<'a, S: ContextProvider> SqlToRel<'a, S> { } } else { // User defined aggregate functions (UDAF) have precedence in case it has the same name as a scalar built-in function - if let Some(fm) = self.schema_provider.get_aggregate_meta(&name) { + if let Some(fm) = self.context_provider.get_aggregate_meta(&name) { let args = self.function_args_to_expr(function.args, schema, planner_context)?; return Ok(Expr::AggregateUDF(expr::AggregateUDF::new( @@ -180,13 +180,13 @@ impl<'a, S: ContextProvider> SqlToRel<'a, S> { window_function::find_df_window_func(name) // next check user defined aggregates .or_else(|| { - self.schema_provider + self.context_provider .get_aggregate_meta(name) .map(WindowFunction::AggregateUDF) }) // next check user defined window functions .or_else(|| { - self.schema_provider + self.context_provider .get_window_meta(name) .map(WindowFunction::WindowUDF) }) diff --git a/datafusion/sql/src/expr/identifier.rs b/datafusion/sql/src/expr/identifier.rs index 4033c28148f8..9f53ff579e7c 100644 --- a/datafusion/sql/src/expr/identifier.rs +++ b/datafusion/sql/src/expr/identifier.rs @@ -34,7 +34,7 @@ impl<'a, S: ContextProvider> SqlToRel<'a, S> { // TODO: figure out if ScalarVariables should be insensitive. let var_names = vec![id.value]; let ty = self - .schema_provider + .context_provider .get_variable_type(&var_names) .ok_or_else(|| { plan_datafusion_err!("variable {var_names:?} has no type information") @@ -98,7 +98,7 @@ impl<'a, S: ContextProvider> SqlToRel<'a, S> { .map(|id| self.normalizer.normalize(id)) .collect(); let ty = self - .schema_provider + .context_provider .get_variable_type(&var_names) .ok_or_else(|| { DataFusionError::Execution(format!( diff --git a/datafusion/sql/src/expr/mod.rs b/datafusion/sql/src/expr/mod.rs index a90a0f121f26..26184834874a 100644 --- a/datafusion/sql/src/expr/mod.rs +++ b/datafusion/sql/src/expr/mod.rs @@ -728,12 +728,12 @@ mod tests { use crate::TableReference; - struct TestSchemaProvider { + struct TestContextProvider { options: ConfigOptions, tables: HashMap>, } - impl TestSchemaProvider { + impl TestContextProvider { pub fn new() -> Self { let mut tables = HashMap::new(); tables.insert( @@ -752,11 +752,8 @@ mod tests { } } - impl ContextProvider for TestSchemaProvider { - fn get_table_provider( - &self, - name: TableReference, - ) -> Result> { + impl ContextProvider for TestContextProvider { + fn get_table_source(&self, name: TableReference) -> Result> { match self.tables.get(name.table()) { Some(table) => Ok(table.clone()), _ => plan_err!("Table not found: {}", name.table()), @@ -809,8 +806,8 @@ mod tests { .unwrap(); let sql_expr = parser.parse_expr().unwrap(); - let schema_provider = TestSchemaProvider::new(); - let sql_to_rel = SqlToRel::new(&schema_provider); + let context_provider = TestContextProvider::new(); + let sql_to_rel = SqlToRel::new(&context_provider); // Should not stack overflow sql_to_rel.sql_expr_to_logical_expr( diff --git a/datafusion/sql/src/planner.rs b/datafusion/sql/src/planner.rs index a2d790d438cc..f7d8307d33a0 100644 --- a/datafusion/sql/src/planner.rs +++ b/datafusion/sql/src/planner.rs @@ -45,8 +45,12 @@ use crate::utils::make_decimal_type; /// The ContextProvider trait allows the query planner to obtain meta-data about tables and /// functions referenced in SQL statements pub trait ContextProvider { + #[deprecated(since = "32.0.0", note = "please use `get_table_source` instead")] + fn get_table_provider(&self, name: TableReference) -> Result> { + self.get_table_source(name) + } /// Getter for a datasource - fn get_table_provider(&self, name: TableReference) -> Result>; + fn get_table_source(&self, name: TableReference) -> Result>; /// Getter for a UDF description fn get_function_meta(&self, name: &str) -> Option>; /// Getter for a UDAF description @@ -186,22 +190,22 @@ impl PlannerContext { /// SQL query planner pub struct SqlToRel<'a, S: ContextProvider> { - pub(crate) schema_provider: &'a S, + pub(crate) context_provider: &'a S, pub(crate) options: ParserOptions, pub(crate) normalizer: IdentNormalizer, } impl<'a, S: ContextProvider> SqlToRel<'a, S> { /// Create a new query planner - pub fn new(schema_provider: &'a S) -> Self { - Self::new_with_options(schema_provider, ParserOptions::default()) + pub fn new(context_provider: &'a S) -> Self { + Self::new_with_options(context_provider, ParserOptions::default()) } /// Create a new query planner - pub fn new_with_options(schema_provider: &'a S, options: ParserOptions) -> Self { + pub fn new_with_options(context_provider: &'a S, options: ParserOptions) -> Self { let normalize = options.enable_ident_normalization; SqlToRel { - schema_provider, + context_provider, options, normalizer: IdentNormalizer::new(normalize), } @@ -334,7 +338,7 @@ impl<'a, S: ContextProvider> SqlToRel<'a, S> { // Timestamp With Time Zone // INPUT : [SQLDataType] TimestampTz + [RuntimeConfig] Time Zone // OUTPUT: [ArrowDataType] Timestamp - self.schema_provider.options().execution.time_zone.clone() + self.context_provider.options().execution.time_zone.clone() } else { // Timestamp Without Time zone None diff --git a/datafusion/sql/src/relation/mod.rs b/datafusion/sql/src/relation/mod.rs index a01a9a2fb8db..180743d19b7b 100644 --- a/datafusion/sql/src/relation/mod.rs +++ b/datafusion/sql/src/relation/mod.rs @@ -38,7 +38,7 @@ impl<'a, S: ContextProvider> SqlToRel<'a, S> { ( match ( cte, - self.schema_provider.get_table_provider(table_ref.clone()), + self.context_provider.get_table_source(table_ref.clone()), ) { (Some(cte_plan), _) => Ok(cte_plan.clone()), (_, Ok(provider)) => { diff --git a/datafusion/sql/src/statement.rs b/datafusion/sql/src/statement.rs index 5ae1e2001d83..f8504a487a66 100644 --- a/datafusion/sql/src/statement.rs +++ b/datafusion/sql/src/statement.rs @@ -611,7 +611,7 @@ impl<'a, S: ContextProvider> SqlToRel<'a, S> { let DescribeTableStmt { table_name } = statement; let table_ref = self.object_name_to_table_reference(table_name)?; - let table_source = self.schema_provider.get_table_provider(table_ref)?; + let table_source = self.context_provider.get_table_source(table_ref)?; let schema = table_source.schema(); @@ -630,7 +630,7 @@ impl<'a, S: ContextProvider> SqlToRel<'a, S> { CopyToSource::Relation(object_name) => { let table_ref = self.object_name_to_table_reference(object_name.clone())?; - let table_source = self.schema_provider.get_table_provider(table_ref)?; + let table_source = self.context_provider.get_table_source(table_ref)?; LogicalPlanBuilder::scan( object_name_to_string(&object_name), table_source, @@ -912,12 +912,15 @@ impl<'a, S: ContextProvider> SqlToRel<'a, S> { ) -> Result { // Do a table lookup to verify the table exists let table_ref = self.object_name_to_table_reference(table_name.clone())?; - let provider = self.schema_provider.get_table_provider(table_ref.clone())?; - let schema = (*provider.schema()).clone(); + let table_source = self.context_provider.get_table_source(table_ref.clone())?; + let schema = (*table_source.schema()).clone(); let schema = DFSchema::try_from(schema)?; - let scan = - LogicalPlanBuilder::scan(object_name_to_string(&table_name), provider, None)? - .build()?; + let scan = LogicalPlanBuilder::scan( + object_name_to_string(&table_name), + table_source, + None, + )? + .build()?; let mut planner_context = PlannerContext::new(); let source = match predicate_expr { @@ -960,10 +963,8 @@ impl<'a, S: ContextProvider> SqlToRel<'a, S> { // Do a table lookup to verify the table exists let table_name = self.object_name_to_table_reference(table_name)?; - let provider = self - .schema_provider - .get_table_provider(table_name.clone())?; - let arrow_schema = (*provider.schema()).clone(); + let table_source = self.context_provider.get_table_source(table_name.clone())?; + let arrow_schema = (*table_source.schema()).clone(); let table_schema = Arc::new(DFSchema::try_from_qualified_schema( table_name.clone(), &arrow_schema, @@ -1066,10 +1067,8 @@ impl<'a, S: ContextProvider> SqlToRel<'a, S> { ) -> Result { // Do a table lookup to verify the table exists let table_name = self.object_name_to_table_reference(table_name)?; - let provider = self - .schema_provider - .get_table_provider(table_name.clone())?; - let arrow_schema = (*provider.schema()).clone(); + let table_source = self.context_provider.get_table_source(table_name.clone())?; + let arrow_schema = (*table_source.schema()).clone(); let table_schema = DFSchema::try_from(arrow_schema)?; // Get insert fields and index_mapping @@ -1193,7 +1192,7 @@ impl<'a, S: ContextProvider> SqlToRel<'a, S> { // Do a table lookup to verify the table exists let table_ref = self.object_name_to_table_reference(sql_table_name)?; - let _ = self.schema_provider.get_table_provider(table_ref)?; + let _ = self.context_provider.get_table_source(table_ref)?; // treat both FULL and EXTENDED as the same let select_list = if full || extended { @@ -1228,7 +1227,7 @@ impl<'a, S: ContextProvider> SqlToRel<'a, S> { // Do a table lookup to verify the table exists let table_ref = self.object_name_to_table_reference(sql_table_name)?; - let _ = self.schema_provider.get_table_provider(table_ref)?; + let _ = self.context_provider.get_table_source(table_ref)?; let query = format!( "SELECT table_catalog, table_schema, table_name, definition FROM information_schema.views WHERE {where_clause}" @@ -1245,8 +1244,8 @@ impl<'a, S: ContextProvider> SqlToRel<'a, S> { schema: schema.into(), table: table.into(), }; - self.schema_provider - .get_table_provider(tables_reference) + self.context_provider + .get_table_source(tables_reference) .is_ok() } } diff --git a/datafusion/sql/tests/sql_integration.rs b/datafusion/sql/tests/sql_integration.rs index 702d7dbce69e..653d2ec52d92 100644 --- a/datafusion/sql/tests/sql_integration.rs +++ b/datafusion/sql/tests/sql_integration.rs @@ -2696,7 +2696,7 @@ struct MockContextProvider { } impl ContextProvider for MockContextProvider { - fn get_table_provider(&self, name: TableReference) -> Result> { + fn get_table_source(&self, name: TableReference) -> Result> { let schema = match name.table() { "test" => Ok(Schema::new(vec![ Field::new("t_date32", DataType::Date32, false), From 7c6fdcc6839f06bd0f7981bdcd45b01200a41db3 Mon Sep 17 00:00:00 2001 From: Devin D'Angelo Date: Tue, 17 Oct 2023 20:39:26 -0400 Subject: [PATCH 091/572] DataSink Dynamic Execution Time Demux (#7791) * dynamic partition * linting * update docs * fix all tests * cargo doc fix * test correct number of files * add asci art and reduce buffer size * fix config tests * review comments * cleanup post rebase * fix test --- datafusion/common/src/config.rs | 18 + .../core/src/datasource/file_format/csv.rs | 192 +++---- .../core/src/datasource/file_format/json.rs | 174 +++--- .../src/datasource/file_format/parquet.rs | 251 +++------ .../core/src/datasource/file_format/write.rs | 515 ++++++++++++++---- .../core/src/datasource/listing/table.rs | 22 +- datafusion/core/src/datasource/memory.rs | 14 +- .../src/datasource/physical_plan/parquet.rs | 19 +- datafusion/physical-plan/src/insert.rs | 30 +- .../test_files/information_schema.slt | 6 + datafusion/sqllogictest/test_files/insert.slt | 15 +- .../test_files/insert_to_external.slt | 22 +- docs/source/user-guide/configs.md | 3 + 13 files changed, 720 insertions(+), 561 deletions(-) diff --git a/datafusion/common/src/config.rs b/datafusion/common/src/config.rs index 281da1f69e69..86e6221c3bc5 100644 --- a/datafusion/common/src/config.rs +++ b/datafusion/common/src/config.rs @@ -254,6 +254,24 @@ config_namespace! { /// Number of files to read in parallel when inferring schema and statistics pub meta_fetch_concurrency: usize, default = 32 + + /// Target number of rows in output files when writing multiple. + /// This is a soft max, so it can be exceeded slightly. There also + /// will be one file smaller than the limit if the total + /// number of rows written is not roughly divisible by the soft max + pub soft_max_rows_per_output_file: usize, default = 50000000 + + /// This is the maximum number of output files being written + /// in parallel. Higher values can potentially give faster write + /// performance at the cost of higher peak memory consumption. + pub max_parallel_ouput_files: usize, default = 8 + + /// This is the maximum number of RecordBatches buffered + /// for each output file being worked. Higher values can potentially + /// give faster write performance at the cost of higher peak + /// memory consumption + pub max_buffered_batches_per_output_file: usize, default = 2 + } } diff --git a/datafusion/core/src/datasource/file_format/csv.rs b/datafusion/core/src/datasource/file_format/csv.rs index 41265ede7fc7..bc01b29ba04b 100644 --- a/datafusion/core/src/datasource/file_format/csv.rs +++ b/datafusion/core/src/datasource/file_format/csv.rs @@ -23,11 +23,10 @@ use std::fmt; use std::fmt::Debug; use std::sync::Arc; +use super::write::{stateless_append_all, stateless_multipart_put}; use super::{FileFormat, DEFAULT_SCHEMA_INFER_MAX_RECORD}; use crate::datasource::file_format::file_compression_type::FileCompressionType; -use crate::datasource::file_format::write::{ - create_writer, stateless_serialize_and_write_files, BatchSerializer, FileWriterMode, -}; +use crate::datasource::file_format::write::{BatchSerializer, FileWriterMode}; use crate::datasource::physical_plan::{ CsvExec, FileGroupDisplay, FileScanConfig, FileSinkConfig, }; @@ -51,7 +50,6 @@ use bytes::{Buf, Bytes}; use futures::stream::BoxStream; use futures::{pin_mut, Stream, StreamExt, TryStreamExt}; use object_store::{delimited::newline_delimited_stream, ObjectMeta, ObjectStore}; -use rand::distributions::{Alphanumeric, DistString}; /// Character Separated Value `FileFormat` implementation. #[derive(Debug)] @@ -481,6 +479,82 @@ impl CsvSink { fn new(config: FileSinkConfig) -> Self { Self { config } } + + async fn append_all( + &self, + data: SendableRecordBatchStream, + context: &Arc, + ) -> Result { + let writer_options = self.config.file_type_writer_options.try_into_csv()?; + let (builder, compression) = + (&writer_options.writer_options, &writer_options.compression); + let compression = FileCompressionType::from(*compression); + + let object_store = context + .runtime_env() + .object_store(&self.config.object_store_url)?; + let file_groups = &self.config.file_groups; + + let builder_clone = builder.clone(); + let options_clone = writer_options.clone(); + let get_serializer = move |file_size| { + let inner_clone = builder_clone.clone(); + // In append mode, consider has_header flag only when file is empty (at the start). + // For other modes, use has_header flag as is. + let serializer: Box = Box::new(if file_size > 0 { + CsvSerializer::new() + .with_builder(inner_clone) + .with_header(false) + } else { + CsvSerializer::new() + .with_builder(inner_clone) + .with_header(options_clone.has_header) + }); + serializer + }; + + stateless_append_all( + data, + context, + object_store, + file_groups, + self.config.unbounded_input, + compression, + Box::new(get_serializer), + ) + .await + } + + async fn multipartput_all( + &self, + data: SendableRecordBatchStream, + context: &Arc, + ) -> Result { + let writer_options = self.config.file_type_writer_options.try_into_csv()?; + let builder = &writer_options.writer_options; + + let builder_clone = builder.clone(); + let options_clone = writer_options.clone(); + let get_serializer = move || { + let inner_clone = builder_clone.clone(); + let serializer: Box = Box::new( + CsvSerializer::new() + .with_builder(inner_clone) + .with_header(options_clone.has_header), + ); + serializer + }; + + stateless_multipart_put( + data, + context, + "csv".into(), + Box::new(get_serializer), + &self.config, + writer_options.compression.into(), + ) + .await + } } #[async_trait] @@ -495,116 +569,22 @@ impl DataSink for CsvSink { async fn write_all( &self, - data: Vec, + data: SendableRecordBatchStream, context: &Arc, ) -> Result { - let num_partitions = data.len(); - let writer_options = self.config.file_type_writer_options.try_into_csv()?; - let (builder, compression) = - (&writer_options.writer_options, &writer_options.compression); - let mut has_header = writer_options.has_header; - let compression = FileCompressionType::from(*compression); - - let object_store = context - .runtime_env() - .object_store(&self.config.object_store_url)?; - // Construct serializer and writer for each file group - let mut serializers: Vec> = vec![]; - let mut writers = vec![]; match self.config.writer_mode { FileWriterMode::Append => { - for file_group in &self.config.file_groups { - let mut append_builder = builder.clone(); - // In append mode, consider has_header flag only when file is empty (at the start). - // For other modes, use has_header flag as is. - if file_group.object_meta.size != 0 { - has_header = false; - append_builder = append_builder.has_headers(false); - } - let serializer = CsvSerializer::new() - .with_builder(append_builder) - .with_header(has_header); - serializers.push(Box::new(serializer)); - - let file = file_group.clone(); - let writer = create_writer( - self.config.writer_mode, - compression, - file.object_meta.clone().into(), - object_store.clone(), - ) - .await?; - writers.push(writer); - } - } - FileWriterMode::Put => { - return not_impl_err!("Put Mode is not implemented for CSV Sink yet") + let total_count = self.append_all(data, context).await?; + Ok(total_count) } FileWriterMode::PutMultipart => { - // Currently assuming only 1 partition path (i.e. not hive-style partitioning on a column) - let base_path = &self.config.table_paths[0]; - match self.config.single_file_output { - false => { - // Uniquely identify this batch of files with a random string, to prevent collisions overwriting files - let write_id = - Alphanumeric.sample_string(&mut rand::thread_rng(), 16); - for part_idx in 0..num_partitions { - let serializer = CsvSerializer::new() - .with_builder(builder.clone()) - .with_header(has_header); - serializers.push(Box::new(serializer)); - let file_path = base_path - .prefix() - .child(format!("{}_{}.csv", write_id, part_idx)); - let object_meta = ObjectMeta { - location: file_path, - last_modified: chrono::offset::Utc::now(), - size: 0, - e_tag: None, - }; - let writer = create_writer( - self.config.writer_mode, - compression, - object_meta.into(), - object_store.clone(), - ) - .await?; - writers.push(writer); - } - } - true => { - let serializer = CsvSerializer::new() - .with_builder(builder.clone()) - .with_header(has_header); - serializers.push(Box::new(serializer)); - let file_path = base_path.prefix(); - let object_meta = ObjectMeta { - location: file_path.clone(), - last_modified: chrono::offset::Utc::now(), - size: 0, - e_tag: None, - }; - let writer = create_writer( - self.config.writer_mode, - compression, - object_meta.into(), - object_store.clone(), - ) - .await?; - writers.push(writer); - } - } + let total_count = self.multipartput_all(data, context).await?; + Ok(total_count) + } + FileWriterMode::Put => { + return not_impl_err!("FileWriterMode::Put is not supported yet!") } } - - stateless_serialize_and_write_files( - data, - serializers, - writers, - self.config.single_file_output, - self.config.unbounded_input, - ) - .await } } diff --git a/datafusion/core/src/datasource/file_format/json.rs b/datafusion/core/src/datasource/file_format/json.rs index dc5b24b2ea10..e1f8ab0d57b7 100644 --- a/datafusion/core/src/datasource/file_format/json.rs +++ b/datafusion/core/src/datasource/file_format/json.rs @@ -23,11 +23,10 @@ use std::fmt::Debug; use std::io::BufReader; use std::sync::Arc; +use super::write::{stateless_append_all, stateless_multipart_put}; use super::{FileFormat, FileScanConfig}; use crate::datasource::file_format::file_compression_type::FileCompressionType; -use crate::datasource::file_format::write::{ - create_writer, stateless_serialize_and_write_files, BatchSerializer, FileWriterMode, -}; +use crate::datasource::file_format::write::{BatchSerializer, FileWriterMode}; use crate::datasource::file_format::DEFAULT_SCHEMA_INFER_MAX_RECORD; use crate::datasource::physical_plan::{FileGroupDisplay, FileSinkConfig, NdJsonExec}; use crate::error::Result; @@ -49,7 +48,6 @@ use datafusion_physical_plan::metrics::MetricsSet; use async_trait::async_trait; use bytes::{Buf, Bytes}; use object_store::{GetResultPayload, ObjectMeta, ObjectStore}; -use rand::distributions::{Alphanumeric, DistString}; /// New line delimited JSON `FileFormat` implementation. #[derive(Debug)] @@ -172,7 +170,7 @@ impl FileFormat for JsonFormat { return not_impl_err!("Inserting compressed JSON is not implemented yet."); } let sink_schema = conf.output_schema().clone(); - let sink = Arc::new(JsonSink::new(conf, self.file_compression_type)); + let sink = Arc::new(JsonSink::new(conf)); Ok(Arc::new(FileSinkExec::new( input, @@ -226,14 +224,11 @@ impl BatchSerializer for JsonSerializer { struct JsonSink { /// Config options for writing data config: FileSinkConfig, - file_compression_type: FileCompressionType, } impl Debug for JsonSink { fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result { - f.debug_struct("JsonSink") - .field("file_compression_type", &self.file_compression_type) - .finish() + f.debug_struct("JsonSink").finish() } } @@ -254,11 +249,62 @@ impl DisplayAs for JsonSink { } impl JsonSink { - fn new(config: FileSinkConfig, file_compression_type: FileCompressionType) -> Self { - Self { - config, - file_compression_type, - } + fn new(config: FileSinkConfig) -> Self { + Self { config } + } + + async fn append_all( + &self, + data: SendableRecordBatchStream, + context: &Arc, + ) -> Result { + let writer_options = self.config.file_type_writer_options.try_into_json()?; + let compression = &writer_options.compression; + + let object_store = context + .runtime_env() + .object_store(&self.config.object_store_url)?; + let file_groups = &self.config.file_groups; + + let get_serializer = move |_| { + let serializer: Box = Box::new(JsonSerializer::new()); + serializer + }; + + stateless_append_all( + data, + context, + object_store, + file_groups, + self.config.unbounded_input, + (*compression).into(), + Box::new(get_serializer), + ) + .await + } + + async fn multipartput_all( + &self, + data: SendableRecordBatchStream, + context: &Arc, + ) -> Result { + let writer_options = self.config.file_type_writer_options.try_into_json()?; + let compression = &writer_options.compression; + + let get_serializer = move || { + let serializer: Box = Box::new(JsonSerializer::new()); + serializer + }; + + stateless_multipart_put( + data, + context, + "json".into(), + Box::new(get_serializer), + &self.config, + (*compression).into(), + ) + .await } } @@ -274,106 +320,22 @@ impl DataSink for JsonSink { async fn write_all( &self, - data: Vec, + data: SendableRecordBatchStream, context: &Arc, ) -> Result { - let num_partitions = data.len(); - - let object_store = context - .runtime_env() - .object_store(&self.config.object_store_url)?; - - let writer_options = self.config.file_type_writer_options.try_into_json()?; - - let compression = FileCompressionType::from(writer_options.compression); - - // Construct serializer and writer for each file group - let mut serializers: Vec> = vec![]; - let mut writers = vec![]; match self.config.writer_mode { FileWriterMode::Append => { - if self.config.single_file_output { - return Err(DataFusionError::NotImplemented("single_file_output=true is not implemented for JsonSink in Append mode".into())); - } - for file_group in &self.config.file_groups { - let serializer = JsonSerializer::new(); - serializers.push(Box::new(serializer)); - - let file = file_group.clone(); - let writer = create_writer( - self.config.writer_mode, - compression, - file.object_meta.clone().into(), - object_store.clone(), - ) - .await?; - writers.push(writer); - } - } - FileWriterMode::Put => { - return not_impl_err!("Put Mode is not implemented for Json Sink yet") + let total_count = self.append_all(data, context).await?; + Ok(total_count) } FileWriterMode::PutMultipart => { - // Currently assuming only 1 partition path (i.e. not hive-style partitioning on a column) - let base_path = &self.config.table_paths[0]; - match self.config.single_file_output { - false => { - // Uniquely identify this batch of files with a random string, to prevent collisions overwriting files - let write_id = - Alphanumeric.sample_string(&mut rand::thread_rng(), 16); - for part_idx in 0..num_partitions { - let serializer = JsonSerializer::new(); - serializers.push(Box::new(serializer)); - let file_path = base_path - .prefix() - .child(format!("{}_{}.json", write_id, part_idx)); - let object_meta = ObjectMeta { - location: file_path, - last_modified: chrono::offset::Utc::now(), - size: 0, - e_tag: None, - }; - let writer = create_writer( - self.config.writer_mode, - compression, - object_meta.into(), - object_store.clone(), - ) - .await?; - writers.push(writer); - } - } - true => { - let serializer = JsonSerializer::new(); - serializers.push(Box::new(serializer)); - let file_path = base_path.prefix(); - let object_meta = ObjectMeta { - location: file_path.clone(), - last_modified: chrono::offset::Utc::now(), - size: 0, - e_tag: None, - }; - let writer = create_writer( - self.config.writer_mode, - compression, - object_meta.into(), - object_store.clone(), - ) - .await?; - writers.push(writer); - } - } + let total_count = self.multipartput_all(data, context).await?; + Ok(total_count) + } + FileWriterMode::Put => { + return not_impl_err!("FileWriterMode::Put is not supported yet!") } } - - stateless_serialize_and_write_files( - data, - serializers, - writers, - self.config.single_file_output, - self.config.unbounded_input, - ) - .await } } diff --git a/datafusion/core/src/datasource/file_format/parquet.rs b/datafusion/core/src/datasource/file_format/parquet.rs index 859bff7ae46f..12d5d515bbcd 100644 --- a/datafusion/core/src/datasource/file_format/parquet.rs +++ b/datafusion/core/src/datasource/file_format/parquet.rs @@ -23,7 +23,7 @@ use std::fmt::Debug; use std::io::Write; use std::sync::Arc; -use super::write::{create_writer, AbortableWrite, FileWriterMode}; +use super::write::{create_writer, start_demuxer_task, AbortableWrite, FileWriterMode}; use super::{FileFormat, FileScanConfig}; use crate::config::ConfigOptions; @@ -62,7 +62,6 @@ use parquet::file::metadata::ParquetMetaData; use parquet::file::properties::WriterProperties; use parquet::file::statistics::Statistics as ParquetStatistics; use parquet::file::writer::SerializedFileWriter; -use rand::distributions::{Alphanumeric, DistString}; use tokio::io::{AsyncWrite, AsyncWriteExt}; use tokio::sync::mpsc::{UnboundedReceiver, UnboundedSender}; use tokio::task::{JoinHandle, JoinSet}; @@ -641,79 +640,6 @@ impl ParquetSink { } } - /// Creates an AsyncArrowWriter for each partition to be written out - /// AsyncArrowWriters are used when individual parquet file serialization is not parallelized - async fn create_all_async_arrow_writers( - &self, - num_partitions: usize, - parquet_props: &WriterProperties, - object_store: Arc, - ) -> Result< - Vec>>, - > { - // Construct writer for each file group - let mut writers = vec![]; - match self.config.writer_mode { - FileWriterMode::Append => { - return plan_err!( - "Parquet format does not support appending to existing file!" - ) - } - FileWriterMode::Put => { - return not_impl_err!("Put Mode is not implemented for ParquetSink yet") - } - FileWriterMode::PutMultipart => { - // Currently assuming only 1 partition path (i.e. not hive-style partitioning on a column) - let base_path = &self.config.table_paths[0]; - match self.config.single_file_output { - false => { - // Uniquely identify this batch of files with a random string, to prevent collisions overwriting files - let write_id = - Alphanumeric.sample_string(&mut rand::thread_rng(), 16); - for part_idx in 0..num_partitions { - let file_path = base_path - .prefix() - .child(format!("{}_{}.parquet", write_id, part_idx)); - let object_meta = ObjectMeta { - location: file_path, - last_modified: chrono::offset::Utc::now(), - size: 0, - e_tag: None, - }; - let writer = self - .create_async_arrow_writer( - object_meta.into(), - object_store.clone(), - parquet_props.clone(), - ) - .await?; - writers.push(writer); - } - } - true => { - let file_path = base_path.prefix(); - let object_meta = ObjectMeta { - location: file_path.clone(), - last_modified: chrono::offset::Utc::now(), - size: 0, - e_tag: None, - }; - let writer = self - .create_async_arrow_writer( - object_meta.into(), - object_store.clone(), - parquet_props.clone(), - ) - .await?; - writers.push(writer); - } - } - } - } - - Ok(writers) - } - /// Creates an object store writer for each output partition /// This is used when parallelizing individual parquet file writes. async fn create_object_store_writers( @@ -758,10 +684,9 @@ impl DataSink for ParquetSink { async fn write_all( &self, - mut data: Vec, + data: SendableRecordBatchStream, context: &Arc, ) -> Result { - let num_partitions = data.len(); let parquet_props = self .config .file_type_writer_options @@ -772,63 +697,93 @@ impl DataSink for ParquetSink { .runtime_env() .object_store(&self.config.object_store_url)?; - let mut row_count = 0; + let exec_options = &context.session_config().options().execution; + + let allow_single_file_parallelism = + exec_options.parquet.allow_single_file_parallelism; + + // This is a temporary special case until https://github.com/apache/arrow-datafusion/pull/7655 + // can be pulled in. + if allow_single_file_parallelism && self.config.single_file_output { + let object_store_writer = self + .create_object_store_writers(1, object_store) + .await? + .remove(0); + + let schema_clone = self.config.output_schema.clone(); + return output_single_parquet_file_parallelized( + object_store_writer, + vec![data], + schema_clone, + parquet_props, + ) + .await + .map(|r| r as u64); + } - let allow_single_file_parallelism = context - .session_config() - .options() - .execution - .parquet - .allow_single_file_parallelism; - - match self.config.single_file_output { - false => { - let writers = self - .create_all_async_arrow_writers( - num_partitions, - parquet_props, - object_store.clone(), - ) - .await?; - // TODO parallelize individual parquet serialization when already outputting multiple parquet files - // e.g. if outputting 2 parquet files on a system with 32 threads, spawn 16 tasks for each individual - // file to be serialized. - row_count = output_multiple_parquet_files(writers, data).await?; - } - true => { - if !allow_single_file_parallelism || data.len() <= 1 { - let mut writer = self - .create_all_async_arrow_writers( - num_partitions, - parquet_props, - object_store.clone(), - ) - .await? - .remove(0); - for data_stream in data.iter_mut() { - while let Some(batch) = data_stream.next().await.transpose()? { - row_count += batch.num_rows(); - writer.write(&batch).await?; - } + let (demux_task, mut file_stream_rx) = start_demuxer_task( + data, + context, + None, + self.config.table_paths[0].clone(), + "parquet".into(), + self.config.single_file_output, + ); + + let mut file_write_tasks: JoinSet> = + JoinSet::new(); + while let Some((path, mut rx)) = file_stream_rx.recv().await { + let mut writer = self + .create_async_arrow_writer( + ObjectMeta { + location: path, + last_modified: chrono::offset::Utc::now(), + size: 0, + e_tag: None, } + .into(), + object_store.clone(), + parquet_props.clone(), + ) + .await?; - writer.close().await?; - } else { - let object_store_writer = self - .create_object_store_writers(1, object_store) - .await? - .remove(0); - row_count = output_single_parquet_file_parallelized( - object_store_writer, - data, - self.config.output_schema.clone(), - parquet_props, - ) - .await?; + file_write_tasks.spawn(async move { + let mut row_count = 0; + while let Some(batch) = rx.recv().await { + row_count += batch.num_rows(); + writer.write(&batch).await?; + } + writer.close().await?; + Ok(row_count) + }); + } + + let mut row_count = 0; + while let Some(result) = file_write_tasks.join_next().await { + match result { + Ok(r) => { + row_count += r?; + } + Err(e) => { + if e.is_panic() { + std::panic::resume_unwind(e.into_panic()); + } else { + unreachable!(); + } } } } + match demux_task.await { + Ok(r) => r?, + Err(e) => { + if e.is_panic() { + std::panic::resume_unwind(e.into_panic()); + } else { + unreachable!(); + } + } + } Ok(row_count as u64) } } @@ -974,48 +929,6 @@ async fn output_single_parquet_file_parallelized( }; object_store_writer.write_all(final_buff.as_slice()).await?; object_store_writer.shutdown().await?; - println!("done!"); - - Ok(row_count) -} - -/// Serializes multiple parquet files independently in parallel from different RecordBatch streams. -/// AsyncArrowWriter is used to coordinate serialization and MultiPart puts to ObjectStore -/// Only a single CPU thread is used to serialize each individual parquet file, so write speed and overall -/// CPU utilization is dependent on the number of output files. -async fn output_multiple_parquet_files( - writers: Vec< - AsyncArrowWriter>, - >, - data: Vec, -) -> Result { - let mut row_count = 0; - let mut join_set: JoinSet> = JoinSet::new(); - for (mut data_stream, mut writer) in data.into_iter().zip(writers.into_iter()) { - join_set.spawn(async move { - let mut cnt = 0; - while let Some(batch) = data_stream.next().await.transpose()? { - cnt += batch.num_rows(); - writer.write(&batch).await?; - } - writer.close().await?; - Ok(cnt) - }); - } - while let Some(result) = join_set.join_next().await { - match result { - Ok(res) => { - row_count += res?; - } // propagate DataFusion error - Err(e) => { - if e.is_panic() { - std::panic::resume_unwind(e.into_panic()); - } else { - unreachable!(); - } - } - } - } Ok(row_count) } diff --git a/datafusion/core/src/datasource/file_format/write.rs b/datafusion/core/src/datasource/file_format/write.rs index 42d18eef634c..928d0d1ba595 100644 --- a/datafusion/core/src/datasource/file_format/write.rs +++ b/datafusion/core/src/datasource/file_format/write.rs @@ -25,24 +25,27 @@ use std::sync::Arc; use std::task::{Context, Poll}; use crate::datasource::file_format::file_compression_type::FileCompressionType; -use crate::datasource::physical_plan::FileMeta; +use crate::datasource::listing::{ListingTableUrl, PartitionedFile}; +use crate::datasource::physical_plan::{FileMeta, FileSinkConfig}; use crate::error::Result; use crate::physical_plan::SendableRecordBatchStream; use arrow_array::RecordBatch; -use datafusion_common::{exec_err, internal_err, DataFusionError}; +use datafusion_common::{exec_err, DataFusionError}; use async_trait::async_trait; use bytes::Bytes; -use datafusion_execution::RecordBatchStream; +use datafusion_execution::TaskContext; use futures::future::BoxFuture; use futures::FutureExt; use futures::{ready, StreamExt}; use object_store::path::Path; use object_store::{MultipartId, ObjectMeta, ObjectStore}; +use rand::distributions::DistString; use tokio::io::{AsyncWrite, AsyncWriteExt}; -use tokio::sync::mpsc; +use tokio::sync::mpsc::{self, Receiver, Sender}; use tokio::task::{JoinHandle, JoinSet}; +use tokio::try_join; /// `AsyncPutWriter` is an object that facilitates asynchronous writing to object stores. /// It is specifically designed for the `object_store` crate's `put` method and sends @@ -308,22 +311,20 @@ type SerializerType = Box; /// concurrently. Data order is preserved. In the event of an error, /// the ObjectStore writer is returned to the caller in addition to an error, /// so that the caller may handle aborting failed writes. -async fn serialize_rb_stream_to_object_store( - mut data_stream: Pin>, +pub(crate) async fn serialize_rb_stream_to_object_store( + mut data_rx: Receiver, mut serializer: Box, mut writer: AbortableWrite>, unbounded_input: bool, -) -> std::result::Result<(SerializerType, WriterType, u64), (WriterType, DataFusionError)> -{ +) -> std::result::Result<(WriterType, u64), (WriterType, DataFusionError)> { let (tx, mut rx) = mpsc::channel::>>(100); let serialize_task = tokio::spawn(async move { - while let Some(maybe_batch) = data_stream.next().await { + while let Some(batch) = data_rx.recv().await { match serializer.duplicate() { Ok(mut serializer_clone) => { let handle = tokio::spawn(async move { - let batch = maybe_batch?; let num_rows = batch.num_rows(); let bytes = serializer_clone.serialize(batch).await?; Ok((num_rows, bytes)) @@ -344,7 +345,7 @@ async fn serialize_rb_stream_to_object_store( } } } - Ok(serializer) + Ok(()) }); let mut row_count = 0; @@ -380,8 +381,8 @@ async fn serialize_rb_stream_to_object_store( } } - let serializer = match serialize_task.await { - Ok(Ok(serializer)) => serializer, + match serialize_task.await { + Ok(Ok(_)) => (), Ok(Err(e)) => return Err((writer, e)), Err(_) => { return Err(( @@ -390,29 +391,166 @@ async fn serialize_rb_stream_to_object_store( )) } }; - Ok((serializer, writer, row_count as u64)) + Ok((writer, row_count as u64)) } +type RecordBatchReceiver = Receiver; +type DemuxedStreamReceiver = Receiver<(Path, RecordBatchReceiver)>; + +/// Splits a single [SendableRecordBatchStream] into a dynamically determined +/// number of partitions at execution time. The partitions are determined by +/// factors known only at execution time, such as total number of rows and +/// partition column values. The demuxer task communicates to the caller +/// by sending channels over a channel. The inner channels send RecordBatches +/// which should be contained within the same output file. The outer channel +/// is used to send a dynamic number of inner channels, representing a dynamic +/// number of total output files. The caller is also responsible to monitor +/// the demux task for errors and abort accordingly. The single_file_ouput parameter +/// overrides all other settings to force only a single file to be written. +/// partition_by parameter will additionally split the input based on the unique +/// values of a specific column ``` +/// ┌───────────┐ ┌────────────┐ ┌─────────────┐ +/// ┌──────▶ │ batch 1 ├────▶...──────▶│ Batch a │ │ Output File1│ +/// │ └───────────┘ └────────────┘ └─────────────┘ +/// │ +/// ┌──────────┐ │ ┌───────────┐ ┌────────────┐ ┌─────────────┐ +/// ┌───────────┐ ┌────────────┐ │ │ ├──────▶ │ batch a+1├────▶...──────▶│ Batch b │ │ Output File2│ +/// │ batch 1 ├────▶...──────▶│ Batch N ├─────▶│ Demux ├────────┤ ... └───────────┘ └────────────┘ └─────────────┘ +/// └───────────┘ └────────────┘ │ │ │ +/// └──────────┘ │ ┌───────────┐ ┌────────────┐ ┌─────────────┐ +/// └──────▶ │ batch d ├────▶...──────▶│ Batch n │ │ Output FileN│ +/// └───────────┘ └────────────┘ └─────────────┘ +pub(crate) fn start_demuxer_task( + input: SendableRecordBatchStream, + context: &Arc, + _partition_by: Option<&str>, + base_output_path: ListingTableUrl, + file_extension: String, + single_file_output: bool, +) -> (JoinHandle>, DemuxedStreamReceiver) { + let exec_options = &context.session_config().options().execution; + + let max_rows_per_file = exec_options.soft_max_rows_per_output_file; + let max_parallel_files = exec_options.max_parallel_ouput_files; + let max_buffered_batches = exec_options.max_buffered_batches_per_output_file; + + let (tx, rx) = mpsc::channel(max_parallel_files); + + let task = tokio::spawn(async move { + row_count_demuxer( + input, + base_output_path, + file_extension, + single_file_output, + max_rows_per_file, + max_buffered_batches, + tx, + ) + .await + }); + (task, rx) +} + +fn generate_file_path( + base_output_path: &ListingTableUrl, + write_id: &str, + part_idx: usize, + file_extension: &str, + single_file_output: bool, +) -> Path { + if !single_file_output { + base_output_path + .prefix() + .child(format!("{}_{}.{}", write_id, part_idx, file_extension)) + } else { + base_output_path.prefix().to_owned() + } +} + +async fn create_new_file_stream( + base_output_path: &ListingTableUrl, + write_id: &str, + part_idx: usize, + file_extension: &str, + single_file_output: bool, + max_buffered_batches: usize, + tx: &mut Sender<(Path, Receiver)>, +) -> Result> { + let file_path = generate_file_path( + base_output_path, + write_id, + part_idx, + file_extension, + single_file_output, + ); + let (tx_file, rx_file) = mpsc::channel(max_buffered_batches / 2); + tx.send((file_path, rx_file)).await.map_err(|_| { + DataFusionError::Execution("Error sending RecordBatch to file stream!".into()) + })?; + Ok(tx_file) +} + +async fn row_count_demuxer( + mut input: SendableRecordBatchStream, + base_output_path: ListingTableUrl, + file_extension: String, + single_file_output: bool, + max_rows_per_file: usize, + max_buffered_batches: usize, + mut tx: Sender<(Path, Receiver)>, +) -> Result<()> { + let mut total_rows_current_file = 0; + let mut part_idx = 0; + let write_id = + rand::distributions::Alphanumeric.sample_string(&mut rand::thread_rng(), 16); + + let mut tx_file = create_new_file_stream( + &base_output_path, + &write_id, + part_idx, + &file_extension, + single_file_output, + max_buffered_batches, + &mut tx, + ) + .await?; + part_idx += 1; + + while let Some(rb) = input.next().await.transpose()? { + total_rows_current_file += rb.num_rows(); + tx_file.send(rb).await.map_err(|_| { + DataFusionError::Execution("Error sending RecordBatch to file stream!".into()) + })?; + + if total_rows_current_file >= max_rows_per_file && !single_file_output { + total_rows_current_file = 0; + tx_file = create_new_file_stream( + &base_output_path, + &write_id, + part_idx, + &file_extension, + single_file_output, + max_buffered_batches, + &mut tx, + ) + .await?; + part_idx += 1; + } + } + Ok(()) +} + +type FileWriteBundle = (Receiver, SerializerType, WriterType); /// Contains the common logic for serializing RecordBatches and /// writing the resulting bytes to an ObjectStore. /// Serialization is assumed to be stateless, i.e. /// each RecordBatch can be serialized without any /// dependency on the RecordBatches before or after. pub(crate) async fn stateless_serialize_and_write_files( - data: Vec, - mut serializers: Vec, - mut writers: Vec, - single_file_output: bool, + mut rx: Receiver, + tx: tokio::sync::oneshot::Sender, unbounded_input: bool, -) -> Result { - if single_file_output && (serializers.len() != 1 || writers.len() != 1) { - return internal_err!("single_file_output is true, but got more than 1 writer!"); - } - let num_partitions = data.len(); - let num_writers = writers.len(); - if !single_file_output && (num_partitions != num_writers) { - return internal_err!("single_file_ouput is false, but did not get 1 writer for each output partition!"); - } +) -> Result<()> { let mut row_count = 0; // tracks if any writers encountered an error triggering the need to abort let mut any_errors = false; @@ -421,100 +559,58 @@ pub(crate) async fn stateless_serialize_and_write_files( // tracks if any errors were encountered in the process of aborting writers. // if true, we may not have a guarentee that all written data was cleaned up. let mut any_abort_errors = false; - match single_file_output { - false => { - let mut join_set = JoinSet::new(); - for (data_stream, serializer, writer) in data - .into_iter() - .zip(serializers.into_iter()) - .zip(writers.into_iter()) - .map(|((a, b), c)| (a, b, c)) - { - join_set.spawn(async move { - serialize_rb_stream_to_object_store( - data_stream, - serializer, - writer, - unbounded_input, - ) - .await - }); - } - let mut finished_writers = Vec::with_capacity(num_writers); - while let Some(result) = join_set.join_next().await { - match result { - Ok(res) => match res { - Ok((_, writer, cnt)) => { - finished_writers.push(writer); - row_count += cnt; - } - Err((writer, e)) => { - finished_writers.push(writer); - any_errors = true; - triggering_error = Some(e); - } - }, - Err(e) => { - // Don't panic, instead try to clean up as many writers as possible. - // If we hit this code, ownership of a writer was not joined back to - // this thread, so we cannot clean it up (hence any_abort_errors is true) - any_errors = true; - any_abort_errors = true; - triggering_error = Some(DataFusionError::Internal(format!( - "Unexpected join error while serializing file {e}" - ))); - } + let mut join_set = JoinSet::new(); + while let Some((data_rx, serializer, writer)) = rx.recv().await { + join_set.spawn(async move { + serialize_rb_stream_to_object_store( + data_rx, + serializer, + writer, + unbounded_input, + ) + .await + }); + } + let mut finished_writers = Vec::new(); + while let Some(result) = join_set.join_next().await { + match result { + Ok(res) => match res { + Ok((writer, cnt)) => { + finished_writers.push(writer); + row_count += cnt; } - } - - // Finalize or abort writers as appropriate - for mut writer in finished_writers.into_iter() { - match any_errors { - true => { - let abort_result = writer.abort_writer(); - if abort_result.is_err() { - any_abort_errors = true; - } - } - false => { - writer.shutdown() - .await - .map_err(|_| DataFusionError::Internal("Error encountered while finalizing writes! Partial results may have been written to ObjectStore!".into()))?; - } + Err((writer, e)) => { + finished_writers.push(writer); + any_errors = true; + triggering_error = Some(e); } + }, + Err(e) => { + // Don't panic, instead try to clean up as many writers as possible. + // If we hit this code, ownership of a writer was not joined back to + // this thread, so we cannot clean it up (hence any_abort_errors is true) + any_errors = true; + any_abort_errors = true; + triggering_error = Some(DataFusionError::Internal(format!( + "Unexpected join error while serializing file {e}" + ))); } } - true => { - let mut writer = writers.remove(0); - let mut serializer = serializers.remove(0); - let mut cnt; - for data_stream in data.into_iter() { - (serializer, writer, cnt) = match serialize_rb_stream_to_object_store( - data_stream, - serializer, - writer, - unbounded_input, - ) - .await - { - Ok((s, w, c)) => (s, w, c), - Err((w, e)) => { - any_errors = true; - triggering_error = Some(e); - writer = w; - break; - } - }; - row_count += cnt; - } - match any_errors { - true => { - let abort_result = writer.abort_writer(); - if abort_result.is_err() { - any_abort_errors = true; - } + } + + // Finalize or abort writers as appropriate + for mut writer in finished_writers.into_iter() { + match any_errors { + true => { + let abort_result = writer.abort_writer(); + if abort_result.is_err() { + any_abort_errors = true; } - false => writer.shutdown().await?, + } + false => { + writer.shutdown() + .await + .map_err(|_| DataFusionError::Internal("Error encountered while finalizing writes! Partial results may have been written to ObjectStore!".into()))?; } } } @@ -529,5 +625,190 @@ pub(crate) async fn stateless_serialize_and_write_files( } } - Ok(row_count) + tx.send(row_count).map_err(|_| { + DataFusionError::Internal( + "Error encountered while sending row count back to file sink!".into(), + ) + })?; + Ok(()) +} + +/// Orchestrates multipart put of a dynamic number of output files from a single input stream +/// for any statelessly serialized file type. That is, any file type for which each [RecordBatch] +/// can be serialized independently of all other [RecordBatch]s. +pub(crate) async fn stateless_multipart_put( + data: SendableRecordBatchStream, + context: &Arc, + file_extension: String, + get_serializer: Box Box + Send>, + config: &FileSinkConfig, + compression: FileCompressionType, +) -> Result { + let object_store = context + .runtime_env() + .object_store(&config.object_store_url)?; + + let single_file_output = config.single_file_output; + let base_output_path = &config.table_paths[0]; + let unbounded_input = config.unbounded_input; + + let (demux_task, mut file_stream_rx) = start_demuxer_task( + data, + context, + None, + base_output_path.clone(), + file_extension, + single_file_output, + ); + + let rb_buffer_size = &context + .session_config() + .options() + .execution + .max_buffered_batches_per_output_file; + + let (tx_file_bundle, rx_file_bundle) = tokio::sync::mpsc::channel(rb_buffer_size / 2); + let (tx_row_cnt, rx_row_cnt) = tokio::sync::oneshot::channel(); + let write_coordinater_task = tokio::spawn(async move { + stateless_serialize_and_write_files(rx_file_bundle, tx_row_cnt, unbounded_input) + .await + }); + while let Some((output_location, rb_stream)) = file_stream_rx.recv().await { + let serializer = get_serializer(); + let object_meta = ObjectMeta { + location: output_location, + last_modified: chrono::offset::Utc::now(), + size: 0, + e_tag: None, + }; + let writer = create_writer( + FileWriterMode::PutMultipart, + compression, + object_meta.into(), + object_store.clone(), + ) + .await?; + + tx_file_bundle + .send((rb_stream, serializer, writer)) + .await + .map_err(|_| { + DataFusionError::Internal( + "Writer receive file bundle channel closed unexpectedly!".into(), + ) + })?; + } + + // Signal to the write coordinater that no more files are coming + drop(tx_file_bundle); + + let total_count = rx_row_cnt.await.map_err(|_| { + DataFusionError::Internal( + "Did not receieve row count from write coordinater".into(), + ) + })?; + + match try_join!(write_coordinater_task, demux_task) { + Ok((r1, r2)) => { + r1?; + r2?; + } + Err(e) => { + if e.is_panic() { + std::panic::resume_unwind(e.into_panic()); + } else { + unreachable!(); + } + } + } + + Ok(total_count) +} + +/// Orchestrates append_all for any statelessly serialized file type. Appends to all files provided +/// in a round robin fashion. +pub(crate) async fn stateless_append_all( + mut data: SendableRecordBatchStream, + context: &Arc, + object_store: Arc, + file_groups: &Vec, + unbounded_input: bool, + compression: FileCompressionType, + get_serializer: Box Box + Send>, +) -> Result { + let rb_buffer_size = &context + .session_config() + .options() + .execution + .max_buffered_batches_per_output_file; + + let (tx_file_bundle, rx_file_bundle) = tokio::sync::mpsc::channel(file_groups.len()); + let mut send_channels = vec![]; + for file_group in file_groups { + let serializer = get_serializer(file_group.object_meta.size); + + let file = file_group.clone(); + let writer = create_writer( + FileWriterMode::Append, + compression, + file.object_meta.clone().into(), + object_store.clone(), + ) + .await?; + + let (tx, rx) = tokio::sync::mpsc::channel(rb_buffer_size / 2); + send_channels.push(tx); + tx_file_bundle + .send((rx, serializer, writer)) + .await + .map_err(|_| { + DataFusionError::Internal( + "Writer receive file bundle channel closed unexpectedly!".into(), + ) + })?; + } + + let (tx_row_cnt, rx_row_cnt) = tokio::sync::oneshot::channel(); + let write_coordinater_task = tokio::spawn(async move { + stateless_serialize_and_write_files(rx_file_bundle, tx_row_cnt, unbounded_input) + .await + }); + + // Append to file groups in round robin + let mut next_file_idx = 0; + while let Some(rb) = data.next().await.transpose()? { + send_channels[next_file_idx].send(rb).await.map_err(|_| { + DataFusionError::Internal( + "Recordbatch file append stream closed unexpectedly!".into(), + ) + })?; + next_file_idx = (next_file_idx + 1) % send_channels.len(); + if unbounded_input { + tokio::task::yield_now().await; + } + } + // Signal to the write coordinater that no more files are coming + drop(tx_file_bundle); + drop(send_channels); + + let total_count = rx_row_cnt.await.map_err(|_| { + DataFusionError::Internal( + "Did not receieve row count from write coordinater".into(), + ) + })?; + + match try_join!(write_coordinater_task) { + Ok(r1) => { + r1.0?; + } + Err(e) => { + if e.is_panic() { + std::panic::resume_unwind(e.into_panic()); + } else { + unreachable!(); + } + } + } + + Ok(total_count) } diff --git a/datafusion/core/src/datasource/listing/table.rs b/datafusion/core/src/datasource/listing/table.rs index ca86e3e3c7e1..05d8ba6c451e 100644 --- a/datafusion/core/src/datasource/listing/table.rs +++ b/datafusion/core/src/datasource/listing/table.rs @@ -1615,6 +1615,10 @@ mod tests { async fn test_insert_into_append_new_json_files() -> Result<()> { let mut config_map: HashMap = HashMap::new(); config_map.insert("datafusion.execution.batch_size".into(), "1".into()); + config_map.insert( + "datafusion.execution.soft_max_rows_per_output_file".into(), + "1".into(), + ); helper_test_append_new_files_to_table( FileType::JSON, FileCompressionType::UNCOMPRESSED, @@ -1639,6 +1643,10 @@ mod tests { async fn test_insert_into_append_new_csv_files() -> Result<()> { let mut config_map: HashMap = HashMap::new(); config_map.insert("datafusion.execution.batch_size".into(), "1".into()); + config_map.insert( + "datafusion.execution.soft_max_rows_per_output_file".into(), + "1".into(), + ); helper_test_append_new_files_to_table( FileType::CSV, FileCompressionType::UNCOMPRESSED, @@ -1652,6 +1660,10 @@ mod tests { async fn test_insert_into_append_new_parquet_files_defaults() -> Result<()> { let mut config_map: HashMap = HashMap::new(); config_map.insert("datafusion.execution.batch_size".into(), "1".into()); + config_map.insert( + "datafusion.execution.soft_max_rows_per_output_file".into(), + "1".into(), + ); helper_test_append_new_files_to_table( FileType::PARQUET, FileCompressionType::UNCOMPRESSED, @@ -1782,6 +1794,11 @@ mod tests { #[tokio::test] async fn test_insert_into_append_new_parquet_files_session_overrides() -> Result<()> { let mut config_map: HashMap = HashMap::new(); + config_map.insert("datafusion.execution.batch_size".into(), "1".into()); + config_map.insert( + "datafusion.execution.soft_max_rows_per_output_file".into(), + "1".into(), + ); config_map.insert( "datafusion.execution.parquet.compression".into(), "zstd(5)".into(), @@ -2090,7 +2107,6 @@ mod tests { } None => SessionContext::new(), }; - let target_partition_number = session_ctx.state().config().target_partitions(); // Create a new schema with one field called "a" of type Int32 let schema = Arc::new(Schema::new(vec![Field::new( @@ -2230,7 +2246,7 @@ mod tests { // Assert that `target_partition_number` many files were added to the table. let num_files = tmp_dir.path().read_dir()?.count(); - assert_eq!(num_files, target_partition_number); + assert_eq!(num_files, 3); // Create a physical plan from the insert plan let plan = session_ctx @@ -2273,7 +2289,7 @@ mod tests { // Assert that another `target_partition_number` many files were added to the table. let num_files = tmp_dir.path().read_dir()?.count(); - assert_eq!(num_files, 2 * target_partition_number); + assert_eq!(num_files, 6); // Return Ok if the function Ok(()) diff --git a/datafusion/core/src/datasource/memory.rs b/datafusion/core/src/datasource/memory.rs index ba99a2b695b6..a2f8e225e121 100644 --- a/datafusion/core/src/datasource/memory.rs +++ b/datafusion/core/src/datasource/memory.rs @@ -270,7 +270,7 @@ impl DataSink for MemSink { async fn write_all( &self, - mut data: Vec, + mut data: SendableRecordBatchStream, _context: &Arc, ) -> Result { let num_partitions = self.batches.len(); @@ -280,14 +280,10 @@ impl DataSink for MemSink { let mut new_batches = vec![vec![]; num_partitions]; let mut i = 0; let mut row_count = 0; - let num_parts = data.len(); - // TODO parallelize outer and inner loops - for data_part in data.iter_mut().take(num_parts) { - while let Some(batch) = data_part.next().await.transpose()? { - row_count += batch.num_rows(); - new_batches[i].push(batch); - i = (i + 1) % num_partitions; - } + while let Some(batch) = data.next().await.transpose()? { + row_count += batch.num_rows(); + new_batches[i].push(batch); + i = (i + 1) % num_partitions; } // write the outputs into the batches diff --git a/datafusion/core/src/datasource/physical_plan/parquet.rs b/datafusion/core/src/datasource/physical_plan/parquet.rs index c781ad81c172..e59686453f0e 100644 --- a/datafusion/core/src/datasource/physical_plan/parquet.rs +++ b/datafusion/core/src/datasource/physical_plan/parquet.rs @@ -1976,24 +1976,7 @@ mod tests { ParquetReadOptions::default(), ) .await?; - ctx.register_parquet( - "part1", - &format!("{out_dir}/{write_id}_1.parquet"), - ParquetReadOptions::default(), - ) - .await?; - ctx.register_parquet( - "part2", - &format!("{out_dir}/{write_id}_2.parquet"), - ParquetReadOptions::default(), - ) - .await?; - ctx.register_parquet( - "part3", - &format!("{out_dir}/{write_id}_3.parquet"), - ParquetReadOptions::default(), - ) - .await?; + ctx.register_parquet("allparts", &out_dir, ParquetReadOptions::default()) .await?; diff --git a/datafusion/physical-plan/src/insert.rs b/datafusion/physical-plan/src/insert.rs index 5b58a0a77134..d1f2706930d2 100644 --- a/datafusion/physical-plan/src/insert.rs +++ b/datafusion/physical-plan/src/insert.rs @@ -36,7 +36,7 @@ use arrow_array::{ArrayRef, UInt64Array}; use arrow_schema::{DataType, Field, Schema}; use datafusion_common::{exec_err, internal_err, DataFusionError, Result}; use datafusion_execution::TaskContext; -use datafusion_physical_expr::PhysicalSortRequirement; +use datafusion_physical_expr::{Distribution, PhysicalSortRequirement}; use async_trait::async_trait; use futures::StreamExt; @@ -68,7 +68,7 @@ pub trait DataSink: DisplayAs + Debug + Send + Sync { /// or rollback required. async fn write_all( &self, - data: Vec, + data: SendableRecordBatchStream, context: &Arc, ) -> Result; } @@ -152,18 +152,6 @@ impl FileSinkExec { } } - fn execute_all_input_streams( - &self, - context: Arc, - ) -> Result> { - let n_input_parts = self.input.output_partitioning().partition_count(); - let mut streams = Vec::with_capacity(n_input_parts); - for part in 0..n_input_parts { - streams.push(self.execute_input_stream(part, context.clone())?); - } - Ok(streams) - } - /// Returns insert sink pub fn sink(&self) -> &dyn DataSink { self.sink.as_ref() @@ -210,13 +198,17 @@ impl ExecutionPlan for FileSinkExec { } fn benefits_from_input_partitioning(&self) -> Vec { - // Incoming number of partitions is taken to be the - // number of files the query is required to write out. - // The optimizer should not change this number. - // Parrallelism is handled within the appropriate DataSink + // DataSink is responsible for dynamically partitioning its + // own input at execution time. vec![false] } + fn required_input_distribution(&self) -> Vec { + // DataSink is responsible for dynamically partitioning its + // own input at execution time, and so requires a single input partition. + vec![Distribution::SinglePartition; self.children().len()] + } + fn required_input_ordering(&self) -> Vec>> { // The input order is either exlicitly set (such as by a ListingTable), // or require that the [FileSinkExec] gets the data in the order the @@ -269,7 +261,7 @@ impl ExecutionPlan for FileSinkExec { if partition != 0 { return internal_err!("FileSinkExec can only be called on partition 0!"); } - let data = self.execute_all_input_streams(context.clone())?; + let data = self.execute_input_stream(0, context.clone())?; let count_schema = self.count_schema.clone(); let sink = self.sink.clone(); diff --git a/datafusion/sqllogictest/test_files/information_schema.slt b/datafusion/sqllogictest/test_files/information_schema.slt index d2ec21488d3d..8e22ad833f7c 100644 --- a/datafusion/sqllogictest/test_files/information_schema.slt +++ b/datafusion/sqllogictest/test_files/information_schema.slt @@ -150,6 +150,8 @@ datafusion.execution.aggregate.scalar_update_factor 10 datafusion.execution.batch_size 8192 datafusion.execution.coalesce_batches true datafusion.execution.collect_statistics false +datafusion.execution.max_buffered_batches_per_output_file 2 +datafusion.execution.max_parallel_ouput_files 8 datafusion.execution.meta_fetch_concurrency 32 datafusion.execution.parquet.allow_single_file_parallelism false datafusion.execution.parquet.bloom_filter_enabled false @@ -175,6 +177,7 @@ datafusion.execution.parquet.statistics_enabled NULL datafusion.execution.parquet.write_batch_size 1024 datafusion.execution.parquet.writer_version 1.0 datafusion.execution.planning_concurrency 13 +datafusion.execution.soft_max_rows_per_output_file 50000000 datafusion.execution.sort_in_place_threshold_bytes 1048576 datafusion.execution.sort_spill_reservation_bytes 10485760 datafusion.execution.target_partitions 7 @@ -217,6 +220,8 @@ datafusion.execution.aggregate.scalar_update_factor 10 Specifies the threshold f datafusion.execution.batch_size 8192 Default batch size while creating new batches, it's especially useful for buffer-in-memory batches since creating tiny batches would result in too much metadata memory consumption datafusion.execution.coalesce_batches true When set to true, record batches will be examined between each operator and small batches will be coalesced into larger batches. This is helpful when there are highly selective filters or joins that could produce tiny output batches. The target batch size is determined by the configuration setting datafusion.execution.collect_statistics false Should DataFusion collect statistics after listing files +datafusion.execution.max_buffered_batches_per_output_file 2 This is the maximum number of RecordBatches buffered for each output file being worked. Higher values can potentially give faster write performance at the cost of higher peak memory consumption +datafusion.execution.max_parallel_ouput_files 8 This is the maximum number of output files being written in parallel. Higher values can potentially give faster write performance at the cost of higher peak memory consumption. datafusion.execution.meta_fetch_concurrency 32 Number of files to read in parallel when inferring schema and statistics datafusion.execution.parquet.allow_single_file_parallelism false Controls whether DataFusion will attempt to speed up writing large parquet files by first writing multiple smaller files and then stitching them together into a single large file. This will result in faster write speeds, but higher memory usage. Also currently unsupported are bloom filters and column indexes when single_file_parallelism is enabled. datafusion.execution.parquet.bloom_filter_enabled false Sets if bloom filter is enabled for any column @@ -242,6 +247,7 @@ datafusion.execution.parquet.statistics_enabled NULL Sets if statistics are enab datafusion.execution.parquet.write_batch_size 1024 Sets write_batch_size in bytes datafusion.execution.parquet.writer_version 1.0 Sets parquet writer version valid values are "1.0" and "2.0" datafusion.execution.planning_concurrency 13 Fan-out during initial physical planning. This is mostly use to plan `UNION` children in parallel. Defaults to the number of CPU cores on the system +datafusion.execution.soft_max_rows_per_output_file 50000000 Target number of rows in output files when writing multiple. This is a soft max, so it can be exceeded slightly. There also will be one file smaller than the limit if the total number of rows written is not roughly divisible by the soft max datafusion.execution.sort_in_place_threshold_bytes 1048576 When sorting, below what size should data be concatenated and sorted in a single RecordBatch rather than sorted in batches and merged. datafusion.execution.sort_spill_reservation_bytes 10485760 Specifies the reserved memory for each spillable sort operation to facilitate an in-memory merge. When a sort operation spills to disk, the in-memory data must be sorted and merged before being written to a file. This setting reserves a specific amount of memory for that in-memory sort/merge process. Note: This setting is irrelevant if the sort operation cannot spill (i.e., if there's no `DiskManager` configured). datafusion.execution.target_partitions 7 Number of partitions for query execution. Increasing partitions can increase concurrency. Defaults to the number of CPU cores on the system diff --git a/datafusion/sqllogictest/test_files/insert.slt b/datafusion/sqllogictest/test_files/insert.slt index 74968bb089d7..cc04c6227721 100644 --- a/datafusion/sqllogictest/test_files/insert.slt +++ b/datafusion/sqllogictest/test_files/insert.slt @@ -126,13 +126,14 @@ Dml: op=[Insert Into] table=[table_without_values] ------TableScan: aggregate_test_100 projection=[c1, c4, c9] physical_plan InsertExec: sink=MemoryTable (partitions=1) ---ProjectionExec: expr=[SUM(aggregate_test_100.c4) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING@3 as field1, COUNT(*) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING@4 as field2] -----BoundedWindowAggExec: wdw=[SUM(aggregate_test_100.c4) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "SUM(aggregate_test_100.c4) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(1)), end_bound: Following(UInt64(1)) }, COUNT(*) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "COUNT(*) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(1)), end_bound: Following(UInt64(1)) }], mode=[Sorted] -------SortExec: expr=[c1@0 ASC NULLS LAST,c9@2 ASC NULLS LAST] ---------CoalesceBatchesExec: target_batch_size=8192 -----------RepartitionExec: partitioning=Hash([c1@0], 8), input_partitions=8 -------------RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1 ---------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1, c4, c9], has_header=true +--CoalescePartitionsExec +----ProjectionExec: expr=[SUM(aggregate_test_100.c4) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING@3 as field1, COUNT(*) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING@4 as field2] +------BoundedWindowAggExec: wdw=[SUM(aggregate_test_100.c4) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "SUM(aggregate_test_100.c4) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(1)), end_bound: Following(UInt64(1)) }, COUNT(*) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "COUNT(*) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(1)), end_bound: Following(UInt64(1)) }], mode=[Sorted] +--------SortExec: expr=[c1@0 ASC NULLS LAST,c9@2 ASC NULLS LAST] +----------CoalesceBatchesExec: target_batch_size=8192 +------------RepartitionExec: partitioning=Hash([c1@0], 8), input_partitions=8 +--------------RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1 +----------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1, c4, c9], has_header=true diff --git a/datafusion/sqllogictest/test_files/insert_to_external.slt b/datafusion/sqllogictest/test_files/insert_to_external.slt index d1b73204e379..abbfa304be27 100644 --- a/datafusion/sqllogictest/test_files/insert_to_external.slt +++ b/datafusion/sqllogictest/test_files/insert_to_external.slt @@ -102,11 +102,18 @@ INSERT INTO single_file_test values (1, 2), (3, 4); ---- 2 +query II +INSERT INTO single_file_test values (4, 5), (6, 7); +---- +2 + query II select * from single_file_test; ---- 1 2 3 4 +4 5 +6 7 statement ok CREATE EXTERNAL TABLE @@ -215,13 +222,14 @@ Dml: op=[Insert Into] table=[table_without_values] ------TableScan: aggregate_test_100 projection=[c1, c4, c9] physical_plan InsertExec: sink=ParquetSink(writer_mode=PutMultipart, file_groups=[]) ---ProjectionExec: expr=[SUM(aggregate_test_100.c4) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING@3 as field1, COUNT(*) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING@4 as field2] -----BoundedWindowAggExec: wdw=[SUM(aggregate_test_100.c4) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "SUM(aggregate_test_100.c4) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(1)), end_bound: Following(UInt64(1)) }, COUNT(*) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "COUNT(*) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(1)), end_bound: Following(UInt64(1)) }], mode=[Sorted] -------SortExec: expr=[c1@0 ASC NULLS LAST,c9@2 ASC NULLS LAST] ---------CoalesceBatchesExec: target_batch_size=8192 -----------RepartitionExec: partitioning=Hash([c1@0], 8), input_partitions=8 -------------RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1 ---------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1, c4, c9], has_header=true +--CoalescePartitionsExec +----ProjectionExec: expr=[SUM(aggregate_test_100.c4) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING@3 as field1, COUNT(*) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING@4 as field2] +------BoundedWindowAggExec: wdw=[SUM(aggregate_test_100.c4) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "SUM(aggregate_test_100.c4) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(1)), end_bound: Following(UInt64(1)) }, COUNT(*) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "COUNT(*) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(1)), end_bound: Following(UInt64(1)) }], mode=[Sorted] +--------SortExec: expr=[c1@0 ASC NULLS LAST,c9@2 ASC NULLS LAST] +----------CoalesceBatchesExec: target_batch_size=8192 +------------RepartitionExec: partitioning=Hash([c1@0], 8), input_partitions=8 +--------------RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1 +----------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1, c4, c9], has_header=true diff --git a/docs/source/user-guide/configs.md b/docs/source/user-guide/configs.md index cab1e5c3e4a9..a0451eed088a 100644 --- a/docs/source/user-guide/configs.md +++ b/docs/source/user-guide/configs.md @@ -77,6 +77,9 @@ Environment variables are read during `SessionConfig` initialisation so they mus | datafusion.execution.sort_spill_reservation_bytes | 10485760 | Specifies the reserved memory for each spillable sort operation to facilitate an in-memory merge. When a sort operation spills to disk, the in-memory data must be sorted and merged before being written to a file. This setting reserves a specific amount of memory for that in-memory sort/merge process. Note: This setting is irrelevant if the sort operation cannot spill (i.e., if there's no `DiskManager` configured). | | datafusion.execution.sort_in_place_threshold_bytes | 1048576 | When sorting, below what size should data be concatenated and sorted in a single RecordBatch rather than sorted in batches and merged. | | datafusion.execution.meta_fetch_concurrency | 32 | Number of files to read in parallel when inferring schema and statistics | +| datafusion.execution.soft_max_rows_per_output_file | 50000000 | Target number of rows in output files when writing multiple. This is a soft max, so it can be exceeded slightly. There also will be one file smaller than the limit if the total number of rows written is not roughly divisible by the soft max | +| datafusion.execution.max_parallel_ouput_files | 8 | This is the maximum number of output files being written in parallel. Higher values can potentially give faster write performance at the cost of higher peak memory consumption. | +| datafusion.execution.max_buffered_batches_per_output_file | 2 | This is the maximum number of RecordBatches buffered for each output file being worked. Higher values can potentially give faster write performance at the cost of higher peak memory consumption | | datafusion.optimizer.enable_round_robin_repartition | true | When set to true, the physical plan optimizer will try to add round robin repartitioning to increase parallelism to leverage more CPU cores | | datafusion.optimizer.enable_topk_aggregation | true | When set to true, the optimizer will attempt to perform limit operations during aggregations, if possible | | datafusion.optimizer.filter_null_join_keys | false | When set to true, the optimizer will insert filters before a join between a nullable and non-nullable column to filter out nulls on the nullable side. This filter can add additional overhead when the file format does not fully support predicate push down. | From 7acd8833cc5d03ba7643d4ae424553c7681ccce8 Mon Sep 17 00:00:00 2001 From: Christoph Schulze Date: Wed, 18 Oct 2023 13:06:51 +0200 Subject: [PATCH 092/572] Add small column on empty projection (#7833) * Find small column when projection is empty * clippy * fix comment * fix avro.slt test * use min_by * clippy --- datafusion/core/tests/sql/explain_analyze.rs | 2 +- .../optimizer/src/push_down_projection.rs | 201 +++++++++++++++--- datafusion/sqllogictest/test_files/avro.slt | 4 +- datafusion/sqllogictest/test_files/json.slt | 4 +- 4 files changed, 179 insertions(+), 32 deletions(-) diff --git a/datafusion/core/tests/sql/explain_analyze.rs b/datafusion/core/tests/sql/explain_analyze.rs index c328f46be74a..7238369f832a 100644 --- a/datafusion/core/tests/sql/explain_analyze.rs +++ b/datafusion/core/tests/sql/explain_analyze.rs @@ -788,7 +788,7 @@ async fn explain_logical_plan_only() { "logical_plan", "Aggregate: groupBy=[[]], aggr=[[COUNT(UInt8(1)) AS COUNT(*)]]\ \n SubqueryAlias: t\ - \n Projection: column1\ + \n Projection: column2\ \n Values: (Utf8(\"a\"), Int64(1), Int64(100)), (Utf8(\"a\"), Int64(2), Int64(150))" ]]; assert_eq!(expected, actual); diff --git a/datafusion/optimizer/src/push_down_projection.rs b/datafusion/optimizer/src/push_down_projection.rs index 6db4bb9ba405..839f6b5bb8f6 100644 --- a/datafusion/optimizer/src/push_down_projection.rs +++ b/datafusion/optimizer/src/push_down_projection.rs @@ -23,6 +23,7 @@ use crate::merge_projection::merge_projection; use crate::optimizer::ApplyOrder; use crate::push_down_filter::replace_cols_by_name; use crate::{OptimizerConfig, OptimizerRule}; +use arrow::datatypes::DataType; use arrow::error::Result as ArrowResult; use datafusion_common::ScalarValue::UInt8; use datafusion_common::{ @@ -148,8 +149,10 @@ impl OptimizerRule for PushDownProjection { { let mut used_columns: HashSet = HashSet::new(); if projection_is_empty { - used_columns - .insert(scan.projected_schema.fields()[0].qualified_column()); + let field = find_small_field(scan.projected_schema.fields()).ok_or( + DataFusionError::Internal("Scan with empty schema".to_string()), + )?; + used_columns.insert(field.qualified_column()); push_down_scan(&used_columns, scan, true)? } else { for expr in projection.expr.iter() { @@ -161,10 +164,13 @@ impl OptimizerRule for PushDownProjection { } } LogicalPlan::Values(values) if projection_is_empty => { - let first_col = - Expr::Column(values.schema.fields()[0].qualified_column()); + let field = find_small_field(values.schema.fields()).ok_or( + DataFusionError::Internal("Values with empty schema".to_string()), + )?; + let column = Expr::Column(field.qualified_column()); + LogicalPlan::Projection(Projection::try_new( - vec![first_col], + vec![column], Arc::new(child_plan.clone()), )?) } @@ -423,7 +429,88 @@ pub fn collect_projection_expr(projection: &Projection) -> HashMap .collect::>() } -// Get the projection exprs from columns in the order of the schema +/// Accumulate the memory size of a data type measured in bits. +/// +/// Types with a variable size get assigned with a fixed size which is greater than most +/// primitive types. +/// +/// While traversing nested types, `nesting` is incremented on every level. +fn nested_size(data_type: &DataType, nesting: &mut usize) -> usize { + use DataType::*; + if data_type.is_primitive() { + return data_type.primitive_width().unwrap_or(1) * 8; + } + + if data_type.is_nested() { + *nesting += 1; + } + + match data_type { + Null => 0, + Boolean => 1, + Binary | Utf8 => 128, + LargeBinary | LargeUtf8 => 256, + FixedSizeBinary(bytes) => (*bytes * 8) as usize, + // primitive types + Int8 + | Int16 + | Int32 + | Int64 + | UInt8 + | UInt16 + | UInt32 + | UInt64 + | Float16 + | Float32 + | Float64 + | Timestamp(_, _) + | Date32 + | Date64 + | Time32(_) + | Time64(_) + | Duration(_) + | Interval(_) + | Dictionary(_, _) + | Decimal128(_, _) + | Decimal256(_, _) => data_type.primitive_width().unwrap_or(1) * 8, + // nested types + List(f) => nested_size(f.data_type(), nesting), + FixedSizeList(_, s) => (s * 8) as usize, + LargeList(f) => nested_size(f.data_type(), nesting), + Struct(fields) => fields + .iter() + .map(|f| nested_size(f.data_type(), nesting)) + .sum(), + Union(fields, _) => fields + .iter() + .map(|(_, f)| nested_size(f.data_type(), nesting)) + .sum(), + Map(field, _) => nested_size(field.data_type(), nesting), + RunEndEncoded(run_ends, values) => { + nested_size(run_ends.data_type(), nesting) + + nested_size(values.data_type(), nesting) + } + } +} + +/// Find a field with a presumable small memory footprint based on its data type's memory size +/// and the level of nesting. +fn find_small_field(fields: &[DFField]) -> Option { + fields + .iter() + .map(|f| { + let nesting = &mut 0; + let size = nested_size(f.data_type(), nesting); + (*nesting, size) + }) + .enumerate() + .min_by(|(_, (nesting_a, size_a)), (_, (nesting_b, size_b))| { + nesting_a.cmp(nesting_b).then(size_a.cmp(size_b)) + }) + .map(|(i, _)| fields[i].clone()) +} + +/// Get the projection exprs from columns in the order of the schema fn get_expr(columns: &HashSet, schema: &DFSchemaRef) -> Result> { let expr = schema .fields() @@ -489,23 +576,14 @@ fn push_down_scan( .filter_map(ArrowResult::ok) .collect(); - if projection.is_empty() { - if has_projection && !schema.fields().is_empty() { - // Ensure that we are reading at least one column from the table in case the query - // does not reference any columns directly such as "SELECT COUNT(1) FROM table", - // except when the table is empty (no column) - projection.insert(0); - } else { - // for table scan without projection, we default to return all columns - projection = scan - .source - .schema() - .fields() - .iter() - .enumerate() - .map(|(i, _)| i) - .collect::>(); - } + if !has_projection && projection.is_empty() { + // for table scan without projection, we default to return all columns + projection = schema + .fields() + .iter() + .enumerate() + .map(|(i, _)| i) + .collect::>(); } // Building new projection from BTreeSet @@ -562,7 +640,7 @@ mod tests { use crate::optimizer::Optimizer; use crate::test::*; use crate::OptimizerContext; - use arrow::datatypes::{DataType, Field, Schema}; + use arrow::datatypes::{DataType, Field, Schema, TimeUnit}; use datafusion_common::DFSchema; use datafusion_expr::builder::table_scan_with_filters; use datafusion_expr::expr; @@ -922,7 +1000,7 @@ mod tests { .project(vec![lit(1_i64), lit(2_i64)])? .build()?; let expected = "Projection: Int64(1), Int64(2)\ - \n TableScan: test projection=[a]"; + \n TableScan: test projection=[]"; assert_optimized_plan_eq(&plan, expected) } @@ -969,7 +1047,7 @@ mod tests { let expected = "\ Projection: Int32(1) AS a\ - \n TableScan: test projection=[a]"; + \n TableScan: test projection=[]"; assert_optimized_plan_eq(&plan, expected) } @@ -998,7 +1076,7 @@ mod tests { let expected = "\ Projection: Int32(1) AS a\ - \n TableScan: test projection=[a], full_filters=[b = Int32(1)]"; + \n TableScan: test projection=[], full_filters=[b = Int32(1)]"; assert_optimized_plan_eq(&plan, expected) } @@ -1154,4 +1232,73 @@ mod tests { .unwrap_or(optimized_plan); Ok(optimized_plan) } + + #[test] + fn test_nested_size() { + use DataType::*; + let nesting = &mut 0; + assert_eq!(nested_size(&Null, nesting), 0); + assert_eq!(*nesting, 0); + assert_eq!(nested_size(&Boolean, nesting), 1); + assert_eq!(*nesting, 0); + assert_eq!(nested_size(&UInt8, nesting), 8); + assert_eq!(*nesting, 0); + assert_eq!(nested_size(&Int64, nesting), 64); + assert_eq!(*nesting, 0); + assert_eq!(nested_size(&Decimal256(5, 2), nesting), 256); + assert_eq!(*nesting, 0); + assert_eq!( + nested_size(&List(Arc::new(Field::new("A", Int64, true))), nesting), + 64 + ); + assert_eq!(*nesting, 1); + *nesting = 0; + assert_eq!( + nested_size( + &List(Arc::new(Field::new( + "A", + List(Arc::new(Field::new("AA", Int64, true))), + true + ))), + nesting + ), + 64 + ); + assert_eq!(*nesting, 2); + } + + #[test] + fn test_find_small_field() { + use DataType::*; + let int32 = DFField::from(Field::new("a", Int32, false)); + let bin = DFField::from(Field::new("b", Binary, false)); + let list_i64 = DFField::from(Field::new( + "c", + List(Arc::new(Field::new("c_1", Int64, true))), + false, + )); + let time_s = DFField::from(Field::new("d", Time32(TimeUnit::Second), false)); + + assert_eq!( + find_small_field(&[ + int32.clone(), + bin.clone(), + list_i64.clone(), + time_s.clone() + ]), + Some(int32.clone()) + ); + assert_eq!( + find_small_field(&[bin.clone(), list_i64.clone(), time_s.clone()]), + Some(time_s.clone()) + ); + assert_eq!( + find_small_field(&[time_s.clone(), int32.clone()]), + Some(time_s.clone()) + ); + assert_eq!( + find_small_field(&[bin.clone(), list_i64.clone()]), + Some(bin.clone()) + ); + } } diff --git a/datafusion/sqllogictest/test_files/avro.slt b/datafusion/sqllogictest/test_files/avro.slt index 5cd268e8ef7f..bd2ba706663c 100644 --- a/datafusion/sqllogictest/test_files/avro.slt +++ b/datafusion/sqllogictest/test_files/avro.slt @@ -253,10 +253,10 @@ EXPLAIN SELECT count(*) from alltypes_plain ---- logical_plan Aggregate: groupBy=[[]], aggr=[[COUNT(UInt8(1)) AS COUNT(*)]] ---TableScan: alltypes_plain projection=[id] +--TableScan: alltypes_plain projection=[bool_col] physical_plan AggregateExec: mode=Final, gby=[], aggr=[COUNT(*)] --CoalescePartitionsExec ----AggregateExec: mode=Partial, gby=[], aggr=[COUNT(*)] ------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 ---------AvroExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/avro/alltypes_plain.avro]]}, projection=[id] +--------AvroExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/avro/alltypes_plain.avro]]}, projection=[bool_col] diff --git a/datafusion/sqllogictest/test_files/json.slt b/datafusion/sqllogictest/test_files/json.slt index 69902f2982dc..f903e48063f8 100644 --- a/datafusion/sqllogictest/test_files/json.slt +++ b/datafusion/sqllogictest/test_files/json.slt @@ -50,13 +50,13 @@ EXPLAIN SELECT count(*) from json_test ---- logical_plan Aggregate: groupBy=[[]], aggr=[[COUNT(UInt8(1)) AS COUNT(*)]] ---TableScan: json_test projection=[a] +--TableScan: json_test projection=[c] physical_plan AggregateExec: mode=Final, gby=[], aggr=[COUNT(*)] --CoalescePartitionsExec ----AggregateExec: mode=Partial, gby=[], aggr=[COUNT(*)] ------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 ---------JsonExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/2.json]]}, projection=[a] +--------JsonExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/2.json]]}, projection=[c] query error DataFusion error: Schema error: No field named mycol\. SELECT mycol FROM single_nan From 14a8681c0a5cde4f2b77861010b72ed337829681 Mon Sep 17 00:00:00 2001 From: Martin Hilton Date: Wed, 18 Oct 2023 22:20:34 +0100 Subject: [PATCH 093/572] feat(7849): coerce TIMESTAMP to TIMESTAMPTZ (#7850) * feat(7849): coerce TIMESTAMP to TIMESTAMPTZ Add coercion rules to support coercion from timestamp types without a timezone to a timestamp with a timezone. Like with the coercion of strings or numeric constants, when a function requires the TIMEZONE_WILDCARD placeholder timezone "+00" will be used as the timezone offset. * review comments Add additional SQL logic test suggested in review. * more review suggestions Add a positive test case for TIMESTAMPTZ and TIMESTAMP comparison. --- datafusion/expr/src/type_coercion/functions.rs | 4 ++-- .../sqllogictest/test_files/timestamps.slt | 18 ++++++++++++++++++ 2 files changed, 20 insertions(+), 2 deletions(-) diff --git a/datafusion/expr/src/type_coercion/functions.rs b/datafusion/expr/src/type_coercion/functions.rs index 17ca40236d41..b387667ad107 100644 --- a/datafusion/expr/src/type_coercion/functions.rs +++ b/datafusion/expr/src/type_coercion/functions.rs @@ -228,7 +228,7 @@ fn coerced_from<'a>( Timestamp(_, Some(from_tz)) => { Some(Timestamp(unit.clone(), Some(from_tz.clone()))) } - Null | Date32 | Utf8 | LargeUtf8 => { + Null | Date32 | Utf8 | LargeUtf8 | Timestamp(_, None) => { // In the absence of any other information assume the time zone is "+00" (UTC). Some(Timestamp(unit.clone(), Some("+00".into()))) } @@ -238,7 +238,7 @@ fn coerced_from<'a>( Timestamp(_, Some(_)) if matches!( type_from, - Null | Timestamp(_, Some(_)) | Date32 | Utf8 | LargeUtf8 + Null | Timestamp(_, _) | Date32 | Utf8 | LargeUtf8 ) => { Some(type_into.clone()) diff --git a/datafusion/sqllogictest/test_files/timestamps.slt b/datafusion/sqllogictest/test_files/timestamps.slt index edafe18caab5..fea61b076ebc 100644 --- a/datafusion/sqllogictest/test_files/timestamps.slt +++ b/datafusion/sqllogictest/test_files/timestamps.slt @@ -1389,6 +1389,12 @@ SELECT date_bin('1 day', TIMESTAMPTZ '2022-01-01 20:10:00Z', TIMESTAMPTZ '2020-0 ---- 2022-01-02T00:00:00+07:00 +# coerce TIMESTAMP to TIMESTAMPTZ +query P +SELECT date_bin('1 day', TIMESTAMPTZ '2022-01-01 20:10:00Z', TIMESTAMP '2020-01-01') +---- +2022-01-01T07:00:00+07:00 + # postgresql: 1 query R SELECT date_part('hour', TIMESTAMPTZ '2000-01-01T01:01:01') as part @@ -1758,3 +1764,15 @@ query T SELECT arrow_typeof(date_bin(INTERVAL '1 day', time, '1970-01-01T00:00:00+05:00')) FROM foo LIMIT 1 ---- Timestamp(Nanosecond, Some("+05:00")) + + +# timestamp comparison with and without timezone +query B +SELECT TIMESTAMPTZ '2022-01-01 20:10:00Z' = TIMESTAMP '2020-01-01' +---- +false + +query B +SELECT TIMESTAMPTZ '2020-01-01 00:00:00Z' = TIMESTAMP '2020-01-01' +---- +true From 7772453f205ec721b65691d664b8ec9b5cd7fd03 Mon Sep 17 00:00:00 2001 From: Andrew Lamb Date: Wed, 18 Oct 2023 17:23:33 -0400 Subject: [PATCH 094/572] Support binary argumnets (via coercion) for `like` and `ilike` and string functions (#7840) fixup --- datafusion/expr/src/built_in_function.rs | 23 +++- datafusion/expr/src/type_coercion/binary.rs | 113 +++++++++++++++--- .../expr/src/type_coercion/functions.rs | 11 +- datafusion/sqllogictest/test_files/binary.slt | 41 +++++-- 4 files changed, 159 insertions(+), 29 deletions(-) diff --git a/datafusion/expr/src/built_in_function.rs b/datafusion/expr/src/built_in_function.rs index f1af6e829b6f..fffdf74af322 100644 --- a/datafusion/expr/src/built_in_function.rs +++ b/datafusion/expr/src/built_in_function.rs @@ -1523,16 +1523,29 @@ impl FromStr for BuiltinScalarFunction { } } +/// Creates a function that returns the return type of a string function given +/// the type of its first argument. +/// +/// If the input type is `LargeUtf8` or `LargeBinary` the return type is +/// `$largeUtf8Type`, +/// +/// If the input type is `Utf8` or `Binary` the return type is `$utf8Type`, macro_rules! make_utf8_to_return_type { ($FUNC:ident, $largeUtf8Type:expr, $utf8Type:expr) => { fn $FUNC(arg_type: &DataType, name: &str) -> Result { Ok(match arg_type { - DataType::LargeUtf8 => $largeUtf8Type, - DataType::Utf8 => $utf8Type, + DataType::LargeUtf8 => $largeUtf8Type, + // LargeBinary inputs are automatically coerced to Utf8 + DataType::LargeBinary => $largeUtf8Type, + DataType::Utf8 => $utf8Type, + // Binary inputs are automatically coerced to Utf8 + DataType::Binary => $utf8Type, DataType::Null => DataType::Null, DataType::Dictionary(_, value_type) => match **value_type { - DataType::LargeUtf8 => $largeUtf8Type, + DataType::LargeUtf8 => $largeUtf8Type, + DataType::LargeBinary => $largeUtf8Type, DataType::Utf8 => $utf8Type, + DataType::Binary => $utf8Type, DataType::Null => DataType::Null, _ => { return plan_err!( @@ -1553,8 +1566,10 @@ macro_rules! make_utf8_to_return_type { } }; } - +// `utf8_to_str_type`: returns either a Utf8 or LargeUtf8 based on the input type size. make_utf8_to_return_type!(utf8_to_str_type, DataType::LargeUtf8, DataType::Utf8); + +// `utf8_to_str_type`: returns either a Int32 or Int64 based on the input type size. make_utf8_to_return_type!(utf8_to_int_type, DataType::Int64, DataType::Int32); fn utf8_or_binary_to_binary_type(arg_type: &DataType, name: &str) -> Result { diff --git a/datafusion/expr/src/type_coercion/binary.rs b/datafusion/expr/src/type_coercion/binary.rs index ba3c21a15d4f..f94ce614605c 100644 --- a/datafusion/expr/src/type_coercion/binary.rs +++ b/datafusion/expr/src/type_coercion/binary.rs @@ -29,7 +29,11 @@ use datafusion_common::{plan_err, DataFusionError}; use crate::Operator; -/// The type signature of an instantiation of binary expression +/// The type signature of an instantiation of binary operator expression such as +/// `lhs + rhs` +/// +/// Note this is different than [`crate::signature::Signature`] which +/// describes the type signature of a function. struct Signature { /// The type to coerce the left argument to lhs: DataType, @@ -648,8 +652,9 @@ fn string_concat_internal_coercion( } } -/// Coercion rules for Strings: the type that both lhs and rhs can be -/// casted to for the purpose of a string computation +/// Coercion rules for string types (Utf8/LargeUtf8): If at least on argument is +/// a string type and both arguments can be coerced into a string type, coerce +/// to string type. fn string_coercion(lhs_type: &DataType, rhs_type: &DataType) -> Option { use arrow::datatypes::DataType::*; match (lhs_type, rhs_type) { @@ -665,8 +670,30 @@ fn string_coercion(lhs_type: &DataType, rhs_type: &DataType) -> Option } } -/// Coercion rules for Binaries: the type that both lhs and rhs can be -/// casted to for the purpose of a computation +/// Coercion rules for binary (Binary/LargeBinary) to string (Utf8/LargeUtf8): +/// If one argument is binary and the other is a string then coerce to string +/// (e.g. for `like`) +fn binary_to_string_coercion( + lhs_type: &DataType, + rhs_type: &DataType, +) -> Option { + use arrow::datatypes::DataType::*; + match (lhs_type, rhs_type) { + (Binary, Utf8) => Some(Utf8), + (Binary, LargeUtf8) => Some(LargeUtf8), + (LargeBinary, Utf8) => Some(LargeUtf8), + (LargeBinary, LargeUtf8) => Some(LargeUtf8), + (Utf8, Binary) => Some(Utf8), + (Utf8, LargeBinary) => Some(LargeUtf8), + (LargeUtf8, Binary) => Some(LargeUtf8), + (LargeUtf8, LargeBinary) => Some(LargeUtf8), + _ => None, + } +} + +/// Coercion rules for binary types (Binary/LargeBinary): If at least on argument is +/// a binary type and both arguments can be coerced into a binary type, coerce +/// to binary type. fn binary_coercion(lhs_type: &DataType, rhs_type: &DataType) -> Option { use arrow::datatypes::DataType::*; match (lhs_type, rhs_type) { @@ -681,6 +708,7 @@ fn binary_coercion(lhs_type: &DataType, rhs_type: &DataType) -> Option /// This is a union of string coercion rules and dictionary coercion rules pub fn like_coercion(lhs_type: &DataType, rhs_type: &DataType) -> Option { string_coercion(lhs_type, rhs_type) + .or_else(|| binary_to_string_coercion(lhs_type, rhs_type)) .or_else(|| dictionary_coercion(lhs_type, rhs_type, false)) .or_else(|| null_coercion(lhs_type, rhs_type)) } @@ -763,7 +791,7 @@ fn temporal_coercion(lhs_type: &DataType, rhs_type: &DataType) -> Option Option { match (lhs_type, rhs_type) { @@ -917,11 +945,33 @@ mod tests { ); } + /// Test coercion rules for binary operators + /// + /// Applies coercion rules for `$LHS_TYPE $OP $RHS_TYPE` and asserts that the + /// the result type is `$RESULT_TYPE` macro_rules! test_coercion_binary_rule { - ($A_TYPE:expr, $B_TYPE:expr, $OP:expr, $C_TYPE:expr) => {{ - let (lhs, rhs) = get_input_types(&$A_TYPE, &$OP, &$B_TYPE)?; - assert_eq!(lhs, $C_TYPE); - assert_eq!(rhs, $C_TYPE); + ($LHS_TYPE:expr, $RHS_TYPE:expr, $OP:expr, $RESULT_TYPE:expr) => {{ + let (lhs, rhs) = get_input_types(&$LHS_TYPE, &$OP, &$RHS_TYPE)?; + assert_eq!(lhs, $RESULT_TYPE); + assert_eq!(rhs, $RESULT_TYPE); + }}; + } + + /// Test coercion rules for like + /// + /// Applies coercion rules for both + /// * `$LHS_TYPE LIKE $RHS_TYPE` + /// * `$RHS_TYPE LIKE $LHS_TYPE` + /// + /// And asserts the result type is `$RESULT_TYPE` + macro_rules! test_like_rule { + ($LHS_TYPE:expr, $RHS_TYPE:expr, $RESULT_TYPE:expr) => {{ + println!("Coercing {} LIKE {}", $LHS_TYPE, $RHS_TYPE); + let result = like_coercion(&$LHS_TYPE, &$RHS_TYPE); + assert_eq!(result, $RESULT_TYPE); + // reverse the order + let result = like_coercion(&$RHS_TYPE, &$LHS_TYPE); + assert_eq!(result, $RESULT_TYPE); }}; } @@ -948,11 +998,46 @@ mod tests { } #[test] - fn test_type_coercion() -> Result<()> { - // test like coercion rule - let result = like_coercion(&DataType::Utf8, &DataType::Utf8); - assert_eq!(result, Some(DataType::Utf8)); + fn test_like_coercion() { + // string coerce to strings + test_like_rule!(DataType::Utf8, DataType::Utf8, Some(DataType::Utf8)); + test_like_rule!( + DataType::LargeUtf8, + DataType::Utf8, + Some(DataType::LargeUtf8) + ); + test_like_rule!( + DataType::Utf8, + DataType::LargeUtf8, + Some(DataType::LargeUtf8) + ); + test_like_rule!( + DataType::LargeUtf8, + DataType::LargeUtf8, + Some(DataType::LargeUtf8) + ); + + // Also coerce binary to strings + test_like_rule!(DataType::Binary, DataType::Utf8, Some(DataType::Utf8)); + test_like_rule!( + DataType::LargeBinary, + DataType::Utf8, + Some(DataType::LargeUtf8) + ); + test_like_rule!( + DataType::Binary, + DataType::LargeUtf8, + Some(DataType::LargeUtf8) + ); + test_like_rule!( + DataType::LargeBinary, + DataType::LargeUtf8, + Some(DataType::LargeUtf8) + ); + } + #[test] + fn test_type_coercion() -> Result<()> { test_coercion_binary_rule!( DataType::Utf8, DataType::Date32, diff --git a/datafusion/expr/src/type_coercion/functions.rs b/datafusion/expr/src/type_coercion/functions.rs index b387667ad107..d55ed0c7216e 100644 --- a/datafusion/expr/src/type_coercion/functions.rs +++ b/datafusion/expr/src/type_coercion/functions.rs @@ -46,6 +46,8 @@ pub fn data_types( return Ok(current_types.to_vec()); } + // Try and coerce the argument types to match the signature, returning the + // coerced types from the first matching signature. for valid_types in valid_types { if let Some(types) = maybe_data_types(&valid_types, current_types) { return Ok(types); @@ -60,6 +62,7 @@ pub fn data_types( ) } +/// Returns a Vec of all possible valid argument types for the given signature. fn get_valid_types( signature: &TypeSignature, current_types: &[DataType], @@ -104,7 +107,12 @@ fn get_valid_types( Ok(valid_types) } -/// Try to coerce current_types into valid_types. +/// Try to coerce the current argument types to match the given `valid_types`. +/// +/// For example, if a function `func` accepts arguments of `(int64, int64)`, +/// but was called with `(int32, int64)`, this function could match the +/// valid_types by by coercing the first argument to `int64`, and would return +/// `Some([int64, int64])`. fn maybe_data_types( valid_types: &[DataType], current_types: &[DataType], @@ -220,6 +228,7 @@ fn coerced_from<'a>( Some(type_into.clone()) } Interval(_) if matches!(type_from, Utf8 | LargeUtf8) => Some(type_into.clone()), + // Any type can be coerced into strings Utf8 | LargeUtf8 => Some(type_into.clone()), Null if can_cast_types(type_from, type_into) => Some(type_into.clone()), diff --git a/datafusion/sqllogictest/test_files/binary.slt b/datafusion/sqllogictest/test_files/binary.slt index 38f8a2e14ffc..0568ada3ad7d 100644 --- a/datafusion/sqllogictest/test_files/binary.slt +++ b/datafusion/sqllogictest/test_files/binary.slt @@ -217,30 +217,51 @@ SELECT largebinary FROM t ORDER BY largebinary; NULL # LIKE -# https://github.com/apache/arrow-datafusion/issues/7342 -query error DataFusion error: type_coercion -SELECT binary FROM t where binary LIKE '%F'; - -query error DataFusion error: type_coercion -SELECT largebinary FROM t where largebinary LIKE '%F'; +query ? +SELECT binary FROM t where binary LIKE '%F%'; +---- +466f6f +466f6f426172 +query ? +SELECT largebinary FROM t where largebinary LIKE '%F%'; +---- +466f6f +466f6f426172 # character_length function -# https://github.com/apache/arrow-datafusion/issues/7344 -query error DataFusion error: Error during planning: The "character_length" function can only accept strings, but got Binary\. +query TITI SELECT cast(binary as varchar) as str, character_length(binary) as binary_len, cast(largebinary as varchar) as large_str, character_length(binary) as largebinary_len from t; +---- +Foo 3 Foo 3 +NULL NULL NULL NULL +Bar 3 Bar 3 +FooBar 6 FooBar 6 + +query I +SELECT character_length(X'20'); +---- +1 + +# still errors on values that can not be coerced to utf8 +query error Encountered non UTF\-8 data: invalid utf\-8 sequence of 1 bytes from index 0 +SELECT character_length(X'c328'); # regexp_replace -# https://github.com/apache/arrow-datafusion/issues/7345 -query error DataFusion error: Error during planning: The "regexp_replace" function can only accept strings, but got Binary\. +query TTTT SELECT cast(binary as varchar) as str, regexp_replace(binary, 'F', 'f') as binary_replaced, cast(largebinary as varchar) as large_str, regexp_replace(largebinary, 'F', 'f') as large_binary_replaced from t; +---- +Foo foo Foo foo +NULL NULL NULL NULL +Bar Bar Bar Bar +FooBar fooBar FooBar fooBar From ba2039ae6d9f17750bddda067a8fb8cee66f3931 Mon Sep 17 00:00:00 2001 From: Huaijin Date: Thu, 19 Oct 2023 05:35:28 +0800 Subject: [PATCH 095/572] fix typo (#7856) --- datafusion/core/src/datasource/physical_plan/file_stream.rs | 2 +- datafusion/physical-plan/src/repartition/mod.rs | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/datafusion/core/src/datasource/physical_plan/file_stream.rs b/datafusion/core/src/datasource/physical_plan/file_stream.rs index 487d17d9c1c3..22ff3f42ebda 100644 --- a/datafusion/core/src/datasource/physical_plan/file_stream.rs +++ b/datafusion/core/src/datasource/physical_plan/file_stream.rs @@ -112,7 +112,7 @@ enum FileStreamState { /// The idle state, no file is currently being read Idle, /// Currently performing asynchronous IO to obtain a stream of RecordBatch - /// for a given parquet file + /// for a given file Open { /// A [`FileOpenFuture`] returned by [`FileOpener::open`] future: FileOpenFuture, diff --git a/datafusion/physical-plan/src/repartition/mod.rs b/datafusion/physical-plan/src/repartition/mod.rs index d1638a02771c..82801bcdfdb3 100644 --- a/datafusion/physical-plan/src/repartition/mod.rs +++ b/datafusion/physical-plan/src/repartition/mod.rs @@ -238,7 +238,7 @@ impl BatchPartitioner { /// /// # Background /// -/// DataFusion, like most other commercial systems, with the the +/// DataFusion, like most other commercial systems, with the /// notable exception of DuckDB, uses the "Exchange Operator" based /// approach to parallelism which works well in practice given /// sufficient care in implementation. From 57f26756d3de0938e520bcdb8c8bd25722858b2b Mon Sep 17 00:00:00 2001 From: Andrew Lamb Date: Wed, 18 Oct 2023 17:46:51 -0400 Subject: [PATCH 096/572] Minor: improve `join` / `join_on` docs (#7813) * Minor: improve join_on docs * Update datafusion/expr/src/logical_plan/builder.rs Co-authored-by: jakevin --------- Co-authored-by: jakevin --- datafusion/core/src/dataframe.rs | 33 ++++++++++++---- datafusion/expr/src/logical_plan/builder.rs | 44 +++++++++++++++------ 2 files changed, 59 insertions(+), 18 deletions(-) diff --git a/datafusion/core/src/dataframe.rs b/datafusion/core/src/dataframe.rs index 79b8fcd519b5..2e192c2a782e 100644 --- a/datafusion/core/src/dataframe.rs +++ b/datafusion/core/src/dataframe.rs @@ -582,12 +582,21 @@ impl DataFrame { Ok(DataFrame::new(self.session_state, plan)) } - /// Join this DataFrame with another DataFrame using the specified columns as join keys. + /// Join this `DataFrame` with another `DataFrame` using explicitly specified + /// columns and an optional filter expression. /// - /// Filter expression expected to contain non-equality predicates that can not be pushed - /// down to any of join inputs. - /// In case of outer join, filter applied to only matched rows. + /// See [`join_on`](Self::join_on) for a more concise way to specify the + /// join condition. Since DataFusion will automatically identify and + /// optimize equality predicates there is no performance difference between + /// this function and `join_on` /// + /// `left_cols` and `right_cols` are used to form "equijoin" predicates (see + /// example below), which are then combined with the optional `filter` + /// expression. + /// + /// Note that in case of outer join, the `filter` is applied to only matched rows. + /// + /// # Example /// ``` /// # use datafusion::prelude::*; /// # use datafusion::error::Result; @@ -600,11 +609,14 @@ impl DataFrame { /// col("a").alias("a2"), /// col("b").alias("b2"), /// col("c").alias("c2")])?; + /// // Perform the equivalent of `left INNER JOIN right ON (a = a2 AND b = b2)` + /// // finding all pairs of rows from `left` and `right` where `a = a2` and `b = b2`. /// let join = left.join(right, JoinType::Inner, &["a", "b"], &["a2", "b2"], None)?; /// let batches = join.collect().await?; /// # Ok(()) /// # } /// ``` + /// pub fn join( self, right: DataFrame, @@ -624,10 +636,13 @@ impl DataFrame { Ok(DataFrame::new(self.session_state, plan)) } - /// Join this DataFrame with another DataFrame using the specified expressions. + /// Join this `DataFrame` with another `DataFrame` using the specified + /// expressions. /// - /// Simply a thin wrapper over [`join`](Self::join) where the join keys are not provided, - /// and the provided expressions are AND'ed together to form the filter expression. + /// Note that DataFusion automatically optimizes joins, including + /// identifying and optimizing equality predicates. + /// + /// # Example /// /// ``` /// # use datafusion::prelude::*; @@ -646,6 +661,10 @@ impl DataFrame { /// col("b").alias("b2"), /// col("c").alias("c2"), /// ])?; + /// + /// // Perform the equivalent of `left INNER JOIN right ON (a != a2 AND b != b2)` + /// // finding all pairs of rows from `left` and `right` where + /// // where `a != a2` and `b != b2`. /// let join_on = left.join_on( /// right, /// JoinType::Inner, diff --git a/datafusion/expr/src/logical_plan/builder.rs b/datafusion/expr/src/logical_plan/builder.rs index 770f39be4710..cd50dbe79cfd 100644 --- a/datafusion/expr/src/logical_plan/builder.rs +++ b/datafusion/expr/src/logical_plan/builder.rs @@ -611,11 +611,19 @@ impl LogicalPlanBuilder { }))) } - /// Apply a join with on constraint. + /// Apply a join to `right` using explicitly specified columns and an + /// optional filter expression. /// - /// Filter expression expected to contain non-equality predicates that can not be pushed - /// down to any of join inputs. - /// In case of outer join, filter applied to only matched rows. + /// See [`join_on`](Self::join_on) for a more concise way to specify the + /// join condition. Since DataFusion will automatically identify and + /// optimize equality predicates there is no performance difference between + /// this function and `join_on` + /// + /// `left_cols` and `right_cols` are used to form "equijoin" predicates (see + /// example below), which are then combined with the optional `filter` + /// expression. + /// + /// Note that in case of outer join, the `filter` is applied to only matched rows. pub fn join( self, right: LogicalPlan, @@ -626,11 +634,12 @@ impl LogicalPlanBuilder { self.join_detailed(right, join_type, join_keys, filter, false) } - /// Apply a join with on constraint. + /// Apply a join with using the specified expressions. + /// + /// Note that DataFusion automatically optimizes joins, including + /// identifying and optimizing equality predicates. /// - /// The `ExtractEquijoinPredicate` optimizer pass has the ability to split join predicates into - /// equijoin predicates and (other) filter predicates. Therefore, if you prefer not to manually split the - /// join predicates, it is recommended to use the `join_on` method instead of the `join` method. + /// # Example /// /// ``` /// # use datafusion_expr::{Expr, col, LogicalPlanBuilder, @@ -650,8 +659,15 @@ impl LogicalPlanBuilder { /// /// let right_plan = LogicalPlanBuilder::scan("right", right_table, None)?.build()?; /// - /// let exprs = vec![col("left.a").eq(col("right.a")), col("left.b").not_eq(col("right.b"))]; + /// // Form the expression `(left.a != right.a)` AND `(left.b != right.b)` + /// let exprs = vec![ + /// col("left.a").eq(col("right.a")), + /// col("left.b").not_eq(col("right.b")) + /// ]; /// + /// // Perform the equivalent of `left INNER JOIN right ON (a != a2 AND b != b2)` + /// // finding all pairs of rows from `left` and `right` where + /// // where `a = a2` and `b != b2`. /// let plan = LogicalPlanBuilder::scan("left", left_table, None)? /// .join_on(right_plan, JoinType::Inner, exprs)? /// .build()?; @@ -688,8 +704,14 @@ impl LogicalPlanBuilder { ) } - /// Apply a join with on constraint and specified null equality - /// If null_equals_null is true then null == null, else null != null + /// Apply a join with on constraint and specified null equality. + /// + /// The behavior is the same as [`join`](Self::join) except that it allows + /// specifying the null equality behavior. + /// + /// If `null_equals_null=true`, rows where both join keys are `null` will be + /// emitted. Otherwise rows where either or both join keys are `null` will be + /// omitted. pub fn join_detailed( self, right: LogicalPlan, From 634ed28fd7a7aedcf48d86d8b578b5fe4e19081a Mon Sep 17 00:00:00 2001 From: Liang-Chi Hsieh Date: Wed, 18 Oct 2023 22:31:33 -0700 Subject: [PATCH 097/572] Support Decimal256 on AVG aggregate expression (#7853) * More * More * More * More * Fix clippy --- .../physical-expr/src/aggregate/average.rs | 94 ++++++++++++++----- .../physical-expr/src/aggregate/utils.rs | 59 +++++++----- .../sqllogictest/test_files/decimal.slt | 4 +- 3 files changed, 109 insertions(+), 48 deletions(-) diff --git a/datafusion/physical-expr/src/aggregate/average.rs b/datafusion/physical-expr/src/aggregate/average.rs index 92c806f76f3c..91f2fb952dce 100644 --- a/datafusion/physical-expr/src/aggregate/average.rs +++ b/datafusion/physical-expr/src/aggregate/average.rs @@ -21,6 +21,7 @@ use arrow::array::{AsArray, PrimitiveBuilder}; use log::debug; use std::any::Any; +use std::fmt::Debug; use std::sync::Arc; use crate::aggregate::groups_accumulator::accumulate::NullState; @@ -33,15 +34,17 @@ use arrow::{ array::{ArrayRef, UInt64Array}, datatypes::Field, }; +use arrow_array::types::{Decimal256Type, DecimalType}; use arrow_array::{ Array, ArrowNativeTypeOp, ArrowNumericType, ArrowPrimitiveType, PrimitiveArray, }; +use arrow_buffer::{i256, ArrowNativeType}; use datafusion_common::{not_impl_err, DataFusionError, Result, ScalarValue}; use datafusion_expr::type_coercion::aggregates::avg_return_type; use datafusion_expr::Accumulator; use super::groups_accumulator::EmitTo; -use super::utils::Decimal128Averager; +use super::utils::DecimalAverager; /// AVG aggregate expression #[derive(Debug, Clone)] @@ -88,7 +91,19 @@ impl AggregateExpr for Avg { ( Decimal128(sum_precision, sum_scale), Decimal128(target_precision, target_scale), - ) => Ok(Box::new(DecimalAvgAccumulator { + ) => Ok(Box::new(DecimalAvgAccumulator:: { + sum: None, + count: 0, + sum_scale: *sum_scale, + sum_precision: *sum_precision, + target_precision: *target_precision, + target_scale: *target_scale, + })), + + ( + Decimal256(sum_precision, sum_scale), + Decimal256(target_precision, target_scale), + ) => Ok(Box::new(DecimalAvgAccumulator:: { sum: None, count: 0, sum_scale: *sum_scale, @@ -156,7 +171,7 @@ impl AggregateExpr for Avg { Decimal128(_sum_precision, sum_scale), Decimal128(target_precision, target_scale), ) => { - let decimal_averager = Decimal128Averager::try_new( + let decimal_averager = DecimalAverager::::try_new( *sum_scale, *target_precision, *target_scale, @@ -172,6 +187,27 @@ impl AggregateExpr for Avg { ))) } + ( + Decimal256(_sum_precision, sum_scale), + Decimal256(target_precision, target_scale), + ) => { + let decimal_averager = DecimalAverager::::try_new( + *sum_scale, + *target_precision, + *target_scale, + )?; + + let avg_fn = move |sum: i256, count: u64| { + decimal_averager.avg(sum, i256::from_usize(count as usize).unwrap()) + }; + + Ok(Box::new(AvgGroupsAccumulator::::new( + &self.input_data_type, + &self.result_data_type, + avg_fn, + ))) + } + _ => not_impl_err!( "AvgGroupsAccumulator for ({} --> {})", self.input_data_type, @@ -256,9 +292,8 @@ impl Accumulator for AvgAccumulator { } /// An accumulator to compute the average for decimals -#[derive(Debug)] -struct DecimalAvgAccumulator { - sum: Option, +struct DecimalAvgAccumulator { + sum: Option, count: u64, sum_scale: i8, sum_precision: u8, @@ -266,30 +301,46 @@ struct DecimalAvgAccumulator { target_scale: i8, } -impl Accumulator for DecimalAvgAccumulator { +impl Debug for DecimalAvgAccumulator { + fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + f.debug_struct("DecimalAvgAccumulator") + .field("sum", &self.sum) + .field("count", &self.count) + .field("sum_scale", &self.sum_scale) + .field("sum_precision", &self.sum_precision) + .field("target_precision", &self.target_precision) + .field("target_scale", &self.target_scale) + .finish() + } +} + +impl Accumulator for DecimalAvgAccumulator { fn state(&self) -> Result> { Ok(vec![ ScalarValue::from(self.count), - ScalarValue::Decimal128(self.sum, self.sum_precision, self.sum_scale), + ScalarValue::new_primitive::( + self.sum, + &T::TYPE_CONSTRUCTOR(self.sum_precision, self.sum_scale), + )?, ]) } fn update_batch(&mut self, values: &[ArrayRef]) -> Result<()> { - let values = values[0].as_primitive::(); + let values = values[0].as_primitive::(); self.count += (values.len() - values.null_count()) as u64; if let Some(x) = sum(values) { - let v = self.sum.get_or_insert(0); - *v += x; + let v = self.sum.get_or_insert(T::Native::default()); + self.sum = Some(v.add_wrapping(x)); } Ok(()) } fn retract_batch(&mut self, values: &[ArrayRef]) -> Result<()> { - let values = values[0].as_primitive::(); + let values = values[0].as_primitive::(); self.count -= (values.len() - values.null_count()) as u64; if let Some(x) = sum(values) { - self.sum = Some(self.sum.unwrap() - x); + self.sum = Some(self.sum.unwrap().sub_wrapping(x)); } Ok(()) } @@ -299,9 +350,9 @@ impl Accumulator for DecimalAvgAccumulator { self.count += sum(states[0].as_primitive::()).unwrap_or_default(); // sums are summed - if let Some(x) = sum(states[1].as_primitive::()) { - let v = self.sum.get_or_insert(0); - *v += x; + if let Some(x) = sum(states[1].as_primitive::()) { + let v = self.sum.get_or_insert(T::Native::default()); + self.sum = Some(v.add_wrapping(x)); } Ok(()) } @@ -310,20 +361,19 @@ impl Accumulator for DecimalAvgAccumulator { let v = self .sum .map(|v| { - Decimal128Averager::try_new( + DecimalAverager::::try_new( self.sum_scale, self.target_precision, self.target_scale, )? - .avg(v, self.count as _) + .avg(v, T::Native::from_usize(self.count as usize).unwrap()) }) .transpose()?; - Ok(ScalarValue::Decimal128( + ScalarValue::new_primitive::( v, - self.target_precision, - self.target_scale, - )) + &T::TYPE_CONSTRUCTOR(self.target_precision, self.target_scale), + ) } fn supports_retract_batch(&self) -> bool { true diff --git a/datafusion/physical-expr/src/aggregate/utils.rs b/datafusion/physical-expr/src/aggregate/utils.rs index 2f473f7608b0..420b26eb2d8e 100644 --- a/datafusion/physical-expr/src/aggregate/utils.rs +++ b/datafusion/physical-expr/src/aggregate/utils.rs @@ -19,12 +19,13 @@ use crate::{AggregateExpr, PhysicalSortExpr}; use arrow::array::ArrayRef; -use arrow::datatypes::{MAX_DECIMAL_FOR_EACH_PRECISION, MIN_DECIMAL_FOR_EACH_PRECISION}; use arrow_array::cast::AsArray; use arrow_array::types::{ - Decimal128Type, TimestampMicrosecondType, TimestampMillisecondType, + Decimal128Type, DecimalType, TimestampMicrosecondType, TimestampMillisecondType, TimestampNanosecondType, TimestampSecondType, }; +use arrow_array::ArrowNativeTypeOp; +use arrow_buffer::ArrowNativeType; use arrow_schema::{DataType, Field}; use datafusion_common::{exec_err, DataFusionError, Result}; use datafusion_expr::Accumulator; @@ -42,27 +43,25 @@ pub fn get_accum_scalar_values_as_arrays( .collect::>()) } -/// Computes averages for `Decimal128` values, checking for overflow +/// Computes averages for `Decimal128`/`Decimal256` values, checking for overflow /// -/// This is needed because different precisions for Decimal128 can +/// This is needed because different precisions for Decimal128/Decimal256 can /// store different ranges of values and thus sum/count may not fit in /// the target type. /// /// For example, the precision is 3, the max of value is `999` and the min /// value is `-999` -pub(crate) struct Decimal128Averager { +pub(crate) struct DecimalAverager { /// scale factor for sum values (10^sum_scale) - sum_mul: i128, + sum_mul: T::Native, /// scale factor for target (10^target_scale) - target_mul: i128, - /// The minimum output value possible to represent with the target precision - target_min: i128, - /// The maximum output value possible to represent with the target precision - target_max: i128, + target_mul: T::Native, + /// the output precision + target_precision: u8, } -impl Decimal128Averager { - /// Create a new `Decimal128Averager`: +impl DecimalAverager { + /// Create a new `DecimalAverager`: /// /// * sum_scale: the scale of `sum` values passed to [`Self::avg`] /// * target_precision: the output precision @@ -74,17 +73,23 @@ impl Decimal128Averager { target_precision: u8, target_scale: i8, ) -> Result { - let sum_mul = 10_i128.pow(sum_scale as u32); - let target_mul = 10_i128.pow(target_scale as u32); - let target_min = MIN_DECIMAL_FOR_EACH_PRECISION[target_precision as usize - 1]; - let target_max = MAX_DECIMAL_FOR_EACH_PRECISION[target_precision as usize - 1]; + let sum_mul = T::Native::from_usize(10_usize) + .map(|b| b.pow_wrapping(sum_scale as u32)) + .ok_or(DataFusionError::Internal( + "Failed to compute sum_mul in DecimalAverager".to_string(), + ))?; + + let target_mul = T::Native::from_usize(10_usize) + .map(|b| b.pow_wrapping(target_scale as u32)) + .ok_or(DataFusionError::Internal( + "Failed to compute target_mul in DecimalAverager".to_string(), + ))?; if target_mul >= sum_mul { Ok(Self { sum_mul, target_mul, - target_min, - target_max, + target_precision, }) } else { // can't convert the lit decimal to the returned data type @@ -92,17 +97,21 @@ impl Decimal128Averager { } } - /// Returns the `sum`/`count` as a i128 Decimal128 with + /// Returns the `sum`/`count` as a i128/i256 Decimal128/Decimal256 with /// target_scale and target_precision and reporting overflow. /// /// * sum: The total sum value stored as Decimal128 with sum_scale /// (passed to `Self::try_new`) - /// * count: total count, stored as a i128 (*NOT* a Decimal128 value) + /// * count: total count, stored as a i128/i256 (*NOT* a Decimal128/Decimal256 value) #[inline(always)] - pub fn avg(&self, sum: i128, count: i128) -> Result { - if let Some(value) = sum.checked_mul(self.target_mul / self.sum_mul) { - let new_value = value / count; - if new_value >= self.target_min && new_value <= self.target_max { + pub fn avg(&self, sum: T::Native, count: T::Native) -> Result { + if let Ok(value) = sum.mul_checked(self.target_mul.div_wrapping(self.sum_mul)) { + let new_value = value.div_wrapping(count); + + let validate = + T::validate_decimal_precision(new_value, self.target_precision); + + if validate.is_ok() { Ok(new_value) } else { exec_err!("Arithmetic Overflow in AvgAccumulator") diff --git a/datafusion/sqllogictest/test_files/decimal.slt b/datafusion/sqllogictest/test_files/decimal.slt index d7632138a84e..570116b7a2a2 100644 --- a/datafusion/sqllogictest/test_files/decimal.slt +++ b/datafusion/sqllogictest/test_files/decimal.slt @@ -622,8 +622,10 @@ create table t as values (arrow_cast(123, 'Decimal256(5,2)')); statement ok set datafusion.execution.target_partitions = 1; -query error DataFusion error: This feature is not implemented: AvgAccumulator for \(Decimal256\(5, 2\) --> Decimal256\(9, 6\)\) +query R select AVG(column1) from t; +---- +123 statement ok drop table t; From b6e4c8238031ae17373d9ae3be2def4b57645e42 Mon Sep 17 00:00:00 2001 From: Andrew Lamb Date: Thu, 19 Oct 2023 02:49:19 -0400 Subject: [PATCH 098/572] Minor: fix typo in comments (#7861) --- datafusion/expr/src/built_in_function.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/datafusion/expr/src/built_in_function.rs b/datafusion/expr/src/built_in_function.rs index fffdf74af322..350067a42186 100644 --- a/datafusion/expr/src/built_in_function.rs +++ b/datafusion/expr/src/built_in_function.rs @@ -1569,7 +1569,7 @@ macro_rules! make_utf8_to_return_type { // `utf8_to_str_type`: returns either a Utf8 or LargeUtf8 based on the input type size. make_utf8_to_return_type!(utf8_to_str_type, DataType::LargeUtf8, DataType::Utf8); -// `utf8_to_str_type`: returns either a Int32 or Int64 based on the input type size. +// `utf8_to_int_type`: returns either a Int32 or Int64 based on the input type size. make_utf8_to_return_type!(utf8_to_int_type, DataType::Int64, DataType::Int32); fn utf8_or_binary_to_binary_type(arg_type: &DataType, name: &str) -> Result { From 860928931365901749c012b26acd376cd019338d Mon Sep 17 00:00:00 2001 From: Aaron VonderHaar Date: Thu, 19 Oct 2023 14:25:09 +0000 Subject: [PATCH 099/572] Minor: fix typo in GreedyMemoryPool documentation (#7864) --- datafusion/execution/src/memory_pool/pool.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/datafusion/execution/src/memory_pool/pool.rs b/datafusion/execution/src/memory_pool/pool.rs index fc49c5fa94c7..4a491630fe20 100644 --- a/datafusion/execution/src/memory_pool/pool.rs +++ b/datafusion/execution/src/memory_pool/pool.rs @@ -49,7 +49,7 @@ impl MemoryPool for UnboundedMemoryPool { /// A [`MemoryPool`] that implements a greedy first-come first-serve limit. /// /// This pool works well for queries that do not need to spill or have -/// a single spillable operator. See [`GreedyMemoryPool`] if there are +/// a single spillable operator. See [`FairSpillPool`] if there are /// multiple spillable operators that all will spill. #[derive(Debug)] pub struct GreedyMemoryPool { From c97a0d2ebb8697855edd5b56904accbfa9a80290 Mon Sep 17 00:00:00 2001 From: Smoothieewastaken <86610201+Smoothieewastaken@users.noreply.github.com> Date: Thu, 19 Oct 2023 20:17:45 +0545 Subject: [PATCH 100/572] Minor: fix multiple typos (#7863) * fixed multiple typos * prettier --------- Co-authored-by: Andrew Lamb --- docs/source/_static/theme_overrides.css | 4 ++-- docs/source/user-guide/configs.md | 2 +- docs/source/user-guide/sql/write_options.md | 2 +- 3 files changed, 4 insertions(+), 4 deletions(-) diff --git a/docs/source/_static/theme_overrides.css b/docs/source/_static/theme_overrides.css index 838eab067afc..3b1b86daac6a 100644 --- a/docs/source/_static/theme_overrides.css +++ b/docs/source/_static/theme_overrides.css @@ -49,7 +49,7 @@ code { } /* This is the bootstrap CSS style for "table-striped". Since the theme does -not yet provide an easy way to configure this globaly, it easier to simply +not yet provide an easy way to configure this globally, it easier to simply include this snippet here than updating each table in all rst files to add ":class: table-striped" */ @@ -59,7 +59,7 @@ add ":class: table-striped" */ /* Limit the max height of the sidebar navigation section. Because in our -custimized template, there is more content above the navigation, i.e. +customized template, there is more content above the navigation, i.e. larger logo: if we don't decrease the max-height, it will overlap with the footer. Details: 8rem for search box etc*/ diff --git a/docs/source/user-guide/configs.md b/docs/source/user-guide/configs.md index a0451eed088a..81131fe96b36 100644 --- a/docs/source/user-guide/configs.md +++ b/docs/source/user-guide/configs.md @@ -65,7 +65,7 @@ Environment variables are read during `SessionConfig` initialisation so they mus | datafusion.execution.parquet.max_statistics_size | NULL | Sets max statistics size for any column. If NULL, uses default parquet writer setting | | datafusion.execution.parquet.max_row_group_size | 1048576 | Sets maximum number of rows in a row group | | datafusion.execution.parquet.created_by | datafusion version 32.0.0 | Sets "created by" property | -| datafusion.execution.parquet.column_index_truncate_length | NULL | Sets column index trucate length | +| datafusion.execution.parquet.column_index_truncate_length | NULL | Sets column index truncate length | | datafusion.execution.parquet.data_page_row_count_limit | 18446744073709551615 | Sets best effort maximum number of rows in data page | | datafusion.execution.parquet.encoding | NULL | Sets default encoding for any column Valid values are: plain, plain_dictionary, rle, bit_packed, delta_binary_packed, delta_length_byte_array, delta_byte_array, rle_dictionary, and byte_stream_split. These values are not case sensitive. If NULL, uses default parquet writer setting | | datafusion.execution.parquet.bloom_filter_enabled | false | Sets if bloom filter is enabled for any column | diff --git a/docs/source/user-guide/sql/write_options.md b/docs/source/user-guide/sql/write_options.md index c98a39f24b92..941484e84efd 100644 --- a/docs/source/user-guide/sql/write_options.md +++ b/docs/source/user-guide/sql/write_options.md @@ -92,7 +92,7 @@ The following options are available when writing JSON files. Note: If any unsupp | ----------- | ---------------------------------------------------------------------------------------------------------------------------------- | ------------- | | COMPRESSION | Sets the compression that should be applied to the entire JSON file. Supported values are GZIP, BZIP2, XZ, ZSTD, and UNCOMPRESSED. | UNCOMPRESSED | -### CSV Format Sepcific Options +### CSV Format Specific Options The following options are available when writing CSV files. Note: if any unsupported options is specified an error will be raised and the query will fail. From 70b1af38e7f4564734d9e4adda9bcaf43171bec6 Mon Sep 17 00:00:00 2001 From: Andrew Lamb Date: Thu, 19 Oct 2023 13:46:57 -0400 Subject: [PATCH 101/572] Minor: Fix docstring typos (#7873) * Minor: Fix docstring typos * Update configs --- datafusion/expr/src/type_coercion/binary.rs | 4 ++-- datafusion/expr/src/type_coercion/functions.rs | 2 +- docs/source/user-guide/configs.md | 2 +- 3 files changed, 4 insertions(+), 4 deletions(-) diff --git a/datafusion/expr/src/type_coercion/binary.rs b/datafusion/expr/src/type_coercion/binary.rs index f94ce614605c..a854373e880d 100644 --- a/datafusion/expr/src/type_coercion/binary.rs +++ b/datafusion/expr/src/type_coercion/binary.rs @@ -652,7 +652,7 @@ fn string_concat_internal_coercion( } } -/// Coercion rules for string types (Utf8/LargeUtf8): If at least on argument is +/// Coercion rules for string types (Utf8/LargeUtf8): If at least one argument is /// a string type and both arguments can be coerced into a string type, coerce /// to string type. fn string_coercion(lhs_type: &DataType, rhs_type: &DataType) -> Option { @@ -691,7 +691,7 @@ fn binary_to_string_coercion( } } -/// Coercion rules for binary types (Binary/LargeBinary): If at least on argument is +/// Coercion rules for binary types (Binary/LargeBinary): If at least one argument is /// a binary type and both arguments can be coerced into a binary type, coerce /// to binary type. fn binary_coercion(lhs_type: &DataType, rhs_type: &DataType) -> Option { diff --git a/datafusion/expr/src/type_coercion/functions.rs b/datafusion/expr/src/type_coercion/functions.rs index d55ed0c7216e..b49bf37d6754 100644 --- a/datafusion/expr/src/type_coercion/functions.rs +++ b/datafusion/expr/src/type_coercion/functions.rs @@ -111,7 +111,7 @@ fn get_valid_types( /// /// For example, if a function `func` accepts arguments of `(int64, int64)`, /// but was called with `(int32, int64)`, this function could match the -/// valid_types by by coercing the first argument to `int64`, and would return +/// valid_types by coercing the first argument to `int64`, and would return /// `Some([int64, int64])`. fn maybe_data_types( valid_types: &[DataType], diff --git a/docs/source/user-guide/configs.md b/docs/source/user-guide/configs.md index 81131fe96b36..a0451eed088a 100644 --- a/docs/source/user-guide/configs.md +++ b/docs/source/user-guide/configs.md @@ -65,7 +65,7 @@ Environment variables are read during `SessionConfig` initialisation so they mus | datafusion.execution.parquet.max_statistics_size | NULL | Sets max statistics size for any column. If NULL, uses default parquet writer setting | | datafusion.execution.parquet.max_row_group_size | 1048576 | Sets maximum number of rows in a row group | | datafusion.execution.parquet.created_by | datafusion version 32.0.0 | Sets "created by" property | -| datafusion.execution.parquet.column_index_truncate_length | NULL | Sets column index truncate length | +| datafusion.execution.parquet.column_index_truncate_length | NULL | Sets column index trucate length | | datafusion.execution.parquet.data_page_row_count_limit | 18446744073709551615 | Sets best effort maximum number of rows in data page | | datafusion.execution.parquet.encoding | NULL | Sets default encoding for any column Valid values are: plain, plain_dictionary, rle, bit_packed, delta_binary_packed, delta_length_byte_array, delta_byte_array, rle_dictionary, and byte_stream_split. These values are not case sensitive. If NULL, uses default parquet writer setting | | datafusion.execution.parquet.bloom_filter_enabled | false | Sets if bloom filter is enabled for any column | From 37d6bf08c948418fe6c72d072d988c2875d81e02 Mon Sep 17 00:00:00 2001 From: Raphael Taylor-Davies <1781103+tustvold@users.noreply.github.com> Date: Thu, 19 Oct 2023 20:10:48 +0100 Subject: [PATCH 102/572] Add CursorValues Decoupling Cursor Data from Cursor Position (#7855) * Decouple cursor storage * Fix doc * Format * Review feedback * Tweak ByteArrayValues::value --- datafusion/physical-plan/src/sorts/cursor.rs | 324 ++++++++++--------- datafusion/physical-plan/src/sorts/merge.rs | 16 +- datafusion/physical-plan/src/sorts/stream.rs | 24 +- 3 files changed, 191 insertions(+), 173 deletions(-) diff --git a/datafusion/physical-plan/src/sorts/cursor.rs b/datafusion/physical-plan/src/sorts/cursor.rs index 1ca41d4fe21c..df90c97faf68 100644 --- a/datafusion/physical-plan/src/sorts/cursor.rs +++ b/datafusion/physical-plan/src/sorts/cursor.rs @@ -20,125 +20,155 @@ use std::cmp::Ordering; use arrow::buffer::ScalarBuffer; use arrow::compute::SortOptions; use arrow::datatypes::ArrowNativeTypeOp; -use arrow::row::{Row, Rows}; +use arrow::row::Rows; use arrow_array::types::ByteArrayType; -use arrow_array::{Array, ArrowPrimitiveType, GenericByteArray, PrimitiveArray}; +use arrow_array::{ + Array, ArrowPrimitiveType, GenericByteArray, OffsetSizeTrait, PrimitiveArray, +}; +use arrow_buffer::{Buffer, OffsetBuffer}; use datafusion_execution::memory_pool::MemoryReservation; -/// A [`Cursor`] for [`Rows`] -pub struct RowCursor { - cur_row: usize, - num_rows: usize, +/// A comparable collection of values for use with [`Cursor`] +/// +/// This is a trait as there are several specialized implementations, such as for +/// single columns or for normalized multi column keys ([`Rows`]) +pub trait CursorValues { + fn len(&self) -> usize; - rows: Rows, + /// Returns true if `l[l_idx] == r[r_idx]` + fn eq(l: &Self, l_idx: usize, r: &Self, r_idx: usize) -> bool; - /// Tracks for the memory used by in the `Rows` of this - /// cursor. Freed on drop - #[allow(dead_code)] - reservation: MemoryReservation, + /// Returns comparison of `l[l_idx]` and `r[r_idx]` + fn compare(l: &Self, l_idx: usize, r: &Self, r_idx: usize) -> Ordering; } -impl std::fmt::Debug for RowCursor { - fn fmt(&self, f: &mut std::fmt::Formatter) -> std::fmt::Result { - f.debug_struct("SortKeyCursor") - .field("cur_row", &self.cur_row) - .field("num_rows", &self.num_rows) - .finish() - } +/// A comparable cursor, used by sort operations +/// +/// A `Cursor` is a pointer into a collection of rows, stored in +/// [`CursorValues`] +/// +/// ```text +/// +/// ┌───────────────────────┐ +/// │ │ ┌──────────────────────┐ +/// │ ┌─────────┐ ┌─────┐ │ ─ ─ ─ ─│ Cursor │ +/// │ │ 1 │ │ A │ │ │ └──────────────────────┘ +/// │ ├─────────┤ ├─────┤ │ +/// │ │ 2 │ │ A │◀─ ┼ ─ ┘ Cursor tracks an +/// │ └─────────┘ └─────┘ │ offset within a +/// │ ... ... │ CursorValues +/// │ │ +/// │ ┌─────────┐ ┌─────┐ │ +/// │ │ 3 │ │ E │ │ +/// │ └─────────┘ └─────┘ │ +/// │ │ +/// │ CursorValues │ +/// └───────────────────────┘ +/// +/// +/// Store logical rows using +/// one of several formats, +/// with specialized +/// implementations +/// depending on the column +/// types +#[derive(Debug)] +pub struct Cursor { + offset: usize, + values: T, } -impl RowCursor { - /// Create a new SortKeyCursor from `rows` and a `reservation` - /// that tracks its memory. There must be at least one row - /// - /// Panics if the reservation is not for exactly `rows.size()` - /// bytes or if `rows` is empty. - pub fn new(rows: Rows, reservation: MemoryReservation) -> Self { - assert_eq!( - rows.size(), - reservation.size(), - "memory reservation mismatch" - ); - assert!(rows.num_rows() > 0); - Self { - cur_row: 0, - num_rows: rows.num_rows(), - rows, - reservation, - } +impl Cursor { + /// Create a [`Cursor`] from the given [`CursorValues`] + pub fn new(values: T) -> Self { + Self { offset: 0, values } } - /// Returns the current row - fn current(&self) -> Row<'_> { - self.rows.row(self.cur_row) + /// Returns true if there are no more rows in this cursor + pub fn is_finished(&self) -> bool { + self.offset == self.values.len() + } + + /// Advance the cursor, returning the previous row index + pub fn advance(&mut self) -> usize { + let t = self.offset; + self.offset += 1; + t } } -impl PartialEq for RowCursor { +impl PartialEq for Cursor { fn eq(&self, other: &Self) -> bool { - self.current() == other.current() + T::eq(&self.values, self.offset, &other.values, other.offset) } } -impl Eq for RowCursor {} +impl Eq for Cursor {} -impl PartialOrd for RowCursor { +impl PartialOrd for Cursor { fn partial_cmp(&self, other: &Self) -> Option { Some(self.cmp(other)) } } -impl Ord for RowCursor { +impl Ord for Cursor { fn cmp(&self, other: &Self) -> Ordering { - self.current().cmp(&other.current()) + T::compare(&self.values, self.offset, &other.values, other.offset) } } -/// A cursor into a sorted batch of rows. +/// Implements [`CursorValues`] for [`Rows`] /// -/// Each cursor must have at least one row so `advance` can be called at least -/// once prior to calling `is_finished`. -pub trait Cursor: Ord { - /// Returns true if there are no more rows in this cursor - fn is_finished(&self) -> bool; +/// Used for sorting when there are multiple columns in the sort key +#[derive(Debug)] +pub struct RowValues { + rows: Rows, - /// Advance the cursor, returning the previous row index - fn advance(&mut self) -> usize; + /// Tracks for the memory used by in the `Rows` of this + /// cursor. Freed on drop + #[allow(dead_code)] + reservation: MemoryReservation, } -impl Cursor for RowCursor { - #[inline] - fn is_finished(&self) -> bool { - self.num_rows == self.cur_row +impl RowValues { + /// Create a new [`RowValues`] from `rows` and a `reservation` + /// that tracks its memory. There must be at least one row + /// + /// Panics if the reservation is not for exactly `rows.size()` + /// bytes or if `rows` is empty. + pub fn new(rows: Rows, reservation: MemoryReservation) -> Self { + assert_eq!( + rows.size(), + reservation.size(), + "memory reservation mismatch" + ); + assert!(rows.num_rows() > 0); + Self { rows, reservation } } +} - #[inline] - fn advance(&mut self) -> usize { - let t = self.cur_row; - self.cur_row += 1; - t +impl CursorValues for RowValues { + fn len(&self) -> usize { + self.rows.num_rows() } -} -/// An [`Array`] that can be converted into [`FieldValues`] -pub trait FieldArray: Array + 'static { - type Values: FieldValues; + fn eq(l: &Self, l_idx: usize, r: &Self, r_idx: usize) -> bool { + l.rows.row(l_idx) == r.rows.row(r_idx) + } - fn values(&self) -> Self::Values; + fn compare(l: &Self, l_idx: usize, r: &Self, r_idx: usize) -> Ordering { + l.rows.row(l_idx).cmp(&r.rows.row(r_idx)) + } } -/// A comparable set of non-nullable values -pub trait FieldValues { - type Value: ?Sized; +/// An [`Array`] that can be converted into [`CursorValues`] +pub trait CursorArray: Array + 'static { + type Values: CursorValues; - fn len(&self) -> usize; - - fn compare(a: &Self::Value, b: &Self::Value) -> Ordering; - - fn value(&self, idx: usize) -> &Self::Value; + fn values(&self) -> Self::Values; } -impl FieldArray for PrimitiveArray { +impl CursorArray for PrimitiveArray { type Values = PrimitiveValues; fn values(&self) -> Self::Values { @@ -149,74 +179,80 @@ impl FieldArray for PrimitiveArray { #[derive(Debug)] pub struct PrimitiveValues(ScalarBuffer); -impl FieldValues for PrimitiveValues { - type Value = T; - +impl CursorValues for PrimitiveValues { fn len(&self) -> usize { self.0.len() } - #[inline] - fn compare(a: &Self::Value, b: &Self::Value) -> Ordering { - T::compare(*a, *b) + fn eq(l: &Self, l_idx: usize, r: &Self, r_idx: usize) -> bool { + l.0[l_idx].is_eq(r.0[r_idx]) } - #[inline] - fn value(&self, idx: usize) -> &Self::Value { - &self.0[idx] + fn compare(l: &Self, l_idx: usize, r: &Self, r_idx: usize) -> Ordering { + l.0[l_idx].compare(r.0[r_idx]) } } -impl FieldArray for GenericByteArray { - type Values = Self; +pub struct ByteArrayValues { + offsets: OffsetBuffer, + values: Buffer, +} - fn values(&self) -> Self::Values { - // Once https://github.com/apache/arrow-rs/pull/4048 is released - // Could potentially destructure array into buffers to reduce codegen, - // in a similar vein to what is done for PrimitiveArray - self.clone() +impl ByteArrayValues { + fn value(&self, idx: usize) -> &[u8] { + assert!(idx < self.len()); + // Safety: offsets are valid and checked bounds above + unsafe { + let start = self.offsets.get_unchecked(idx).as_usize(); + let end = self.offsets.get_unchecked(idx + 1).as_usize(); + self.values.get_unchecked(start..end) + } } } -impl FieldValues for GenericByteArray { - type Value = T::Native; - +impl CursorValues for ByteArrayValues { fn len(&self) -> usize { - Array::len(self) + self.offsets.len() - 1 } - #[inline] - fn compare(a: &Self::Value, b: &Self::Value) -> Ordering { - let a: &[u8] = a.as_ref(); - let b: &[u8] = b.as_ref(); - a.cmp(b) + fn eq(l: &Self, l_idx: usize, r: &Self, r_idx: usize) -> bool { + l.value(l_idx) == r.value(r_idx) } - #[inline] - fn value(&self, idx: usize) -> &Self::Value { - self.value(idx) + fn compare(l: &Self, l_idx: usize, r: &Self, r_idx: usize) -> Ordering { + l.value(l_idx).cmp(r.value(r_idx)) + } +} + +impl CursorArray for GenericByteArray { + type Values = ByteArrayValues; + + fn values(&self) -> Self::Values { + ByteArrayValues { + offsets: self.offsets().clone(), + values: self.values().clone(), + } } } -/// A cursor over sorted, nullable [`FieldValues`] +/// A collection of sorted, nullable [`CursorValues`] /// /// Note: comparing cursors with different `SortOptions` will yield an arbitrary ordering #[derive(Debug)] -pub struct FieldCursor { +pub struct ArrayValues { values: T, - offset: usize, // If nulls first, the first non-null index // Otherwise, the first null index null_threshold: usize, options: SortOptions, } -impl FieldCursor { - /// Create a new [`FieldCursor`] from the provided `values` sorted according +impl ArrayValues { + /// Create a new [`ArrayValues`] from the provided `values` sorted according /// to `options`. /// /// Panics if the array is empty - pub fn new>(options: SortOptions, array: &A) -> Self { + pub fn new>(options: SortOptions, array: &A) -> Self { assert!(array.len() > 0, "Empty array passed to FieldCursor"); let null_threshold = match options.nulls_first { true => array.null_count(), @@ -225,67 +261,48 @@ impl FieldCursor { Self { values: array.values(), - offset: 0, null_threshold, options, } } - fn is_null(&self) -> bool { - (self.offset < self.null_threshold) == self.options.nulls_first + fn is_null(&self, idx: usize) -> bool { + (idx < self.null_threshold) == self.options.nulls_first } } -impl PartialEq for FieldCursor { - fn eq(&self, other: &Self) -> bool { - self.cmp(other).is_eq() +impl CursorValues for ArrayValues { + fn len(&self) -> usize { + self.values.len() } -} -impl Eq for FieldCursor {} -impl PartialOrd for FieldCursor { - fn partial_cmp(&self, other: &Self) -> Option { - Some(self.cmp(other)) + fn eq(l: &Self, l_idx: usize, r: &Self, r_idx: usize) -> bool { + match (l.is_null(l_idx), r.is_null(r_idx)) { + (true, true) => true, + (false, false) => T::eq(&l.values, l_idx, &r.values, r_idx), + _ => false, + } } -} -impl Ord for FieldCursor { - fn cmp(&self, other: &Self) -> Ordering { - match (self.is_null(), other.is_null()) { + fn compare(l: &Self, l_idx: usize, r: &Self, r_idx: usize) -> Ordering { + match (l.is_null(l_idx), r.is_null(r_idx)) { (true, true) => Ordering::Equal, - (true, false) => match self.options.nulls_first { + (true, false) => match l.options.nulls_first { true => Ordering::Less, false => Ordering::Greater, }, - (false, true) => match self.options.nulls_first { + (false, true) => match l.options.nulls_first { true => Ordering::Greater, false => Ordering::Less, }, - (false, false) => { - let s_v = self.values.value(self.offset); - let o_v = other.values.value(other.offset); - - match self.options.descending { - true => T::compare(o_v, s_v), - false => T::compare(s_v, o_v), - } - } + (false, false) => match l.options.descending { + true => T::compare(&r.values, r_idx, &l.values, l_idx), + false => T::compare(&l.values, l_idx, &r.values, r_idx), + }, } } } -impl Cursor for FieldCursor { - fn is_finished(&self) -> bool { - self.offset == self.values.len() - } - - fn advance(&mut self) -> usize { - let t = self.offset; - self.offset += 1; - t - } -} - #[cfg(test)] mod tests { use super::*; @@ -294,18 +311,19 @@ mod tests { options: SortOptions, values: ScalarBuffer, null_count: usize, - ) -> FieldCursor> { + ) -> Cursor>> { let null_threshold = match options.nulls_first { true => null_count, false => values.len() - null_count, }; - FieldCursor { - offset: 0, + let values = ArrayValues { values: PrimitiveValues(values), null_threshold, options, - } + }; + + Cursor::new(values) } #[test] diff --git a/datafusion/physical-plan/src/sorts/merge.rs b/datafusion/physical-plan/src/sorts/merge.rs index e60baf2cd806..422ff3aebdb3 100644 --- a/datafusion/physical-plan/src/sorts/merge.rs +++ b/datafusion/physical-plan/src/sorts/merge.rs @@ -20,7 +20,7 @@ use crate::metrics::BaselineMetrics; use crate::sorts::builder::BatchBuilder; -use crate::sorts::cursor::Cursor; +use crate::sorts::cursor::{Cursor, CursorValues}; use crate::sorts::stream::PartitionedStream; use crate::RecordBatchStream; use arrow::datatypes::SchemaRef; @@ -35,7 +35,7 @@ use std::task::{ready, Context, Poll}; type CursorStream = Box>>; #[derive(Debug)] -pub(crate) struct SortPreservingMergeStream { +pub(crate) struct SortPreservingMergeStream { in_progress: BatchBuilder, /// The sorted input streams to merge together @@ -88,8 +88,8 @@ pub(crate) struct SortPreservingMergeStream { /// target batch size batch_size: usize, - /// Vector that holds cursors for each non-exhausted input partition - cursors: Vec>, + /// Cursors for each input partition. `None` means the input is exhausted + cursors: Vec>>, /// Optional number of rows to fetch fetch: Option, @@ -98,7 +98,7 @@ pub(crate) struct SortPreservingMergeStream { produced: usize, } -impl SortPreservingMergeStream { +impl SortPreservingMergeStream { pub(crate) fn new( streams: CursorStream, schema: SchemaRef, @@ -140,7 +140,7 @@ impl SortPreservingMergeStream { None => Poll::Ready(Ok(())), Some(Err(e)) => Poll::Ready(Err(e)), Some(Ok((cursor, batch))) => { - self.cursors[idx] = Some(cursor); + self.cursors[idx] = Some(Cursor::new(cursor)); Poll::Ready(self.in_progress.push_batch(idx, batch)) } } @@ -310,7 +310,7 @@ impl SortPreservingMergeStream { } } -impl Stream for SortPreservingMergeStream { +impl Stream for SortPreservingMergeStream { type Item = Result; fn poll_next( @@ -322,7 +322,7 @@ impl Stream for SortPreservingMergeStream { } } -impl RecordBatchStream for SortPreservingMergeStream { +impl RecordBatchStream for SortPreservingMergeStream { fn schema(&self) -> SchemaRef { self.in_progress.schema().clone() } diff --git a/datafusion/physical-plan/src/sorts/stream.rs b/datafusion/physical-plan/src/sorts/stream.rs index a7f9e7380c47..4cabdc6e178c 100644 --- a/datafusion/physical-plan/src/sorts/stream.rs +++ b/datafusion/physical-plan/src/sorts/stream.rs @@ -15,7 +15,7 @@ // specific language governing permissions and limitations // under the License. -use crate::sorts::cursor::{FieldArray, FieldCursor, RowCursor}; +use crate::sorts::cursor::{ArrayValues, CursorArray, RowValues}; use crate::SendableRecordBatchStream; use crate::{PhysicalExpr, PhysicalSortExpr}; use arrow::array::Array; @@ -76,7 +76,7 @@ impl FusedStreams { } /// A [`PartitionedStream`] that wraps a set of [`SendableRecordBatchStream`] -/// and computes [`RowCursor`] based on the provided [`PhysicalSortExpr`] +/// and computes [`RowValues`] based on the provided [`PhysicalSortExpr`] #[derive(Debug)] pub struct RowCursorStream { /// Converter to convert output of physical expressions @@ -114,7 +114,7 @@ impl RowCursorStream { }) } - fn convert_batch(&mut self, batch: &RecordBatch) -> Result { + fn convert_batch(&mut self, batch: &RecordBatch) -> Result { let cols = self .column_expressions .iter() @@ -127,12 +127,12 @@ impl RowCursorStream { // track the memory in the newly created Rows. let mut rows_reservation = self.reservation.new_empty(); rows_reservation.try_grow(rows.size())?; - Ok(RowCursor::new(rows, rows_reservation)) + Ok(RowValues::new(rows, rows_reservation)) } } impl PartitionedStream for RowCursorStream { - type Output = Result<(RowCursor, RecordBatch)>; + type Output = Result<(RowValues, RecordBatch)>; fn partitions(&self) -> usize { self.streams.0.len() @@ -153,7 +153,7 @@ impl PartitionedStream for RowCursorStream { } /// Specialized stream for sorts on single primitive columns -pub struct FieldCursorStream { +pub struct FieldCursorStream { /// The physical expressions to sort by sort: PhysicalSortExpr, /// Input streams @@ -161,7 +161,7 @@ pub struct FieldCursorStream { phantom: PhantomData T>, } -impl std::fmt::Debug for FieldCursorStream { +impl std::fmt::Debug for FieldCursorStream { fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { f.debug_struct("PrimitiveCursorStream") .field("num_streams", &self.streams) @@ -169,7 +169,7 @@ impl std::fmt::Debug for FieldCursorStream { } } -impl FieldCursorStream { +impl FieldCursorStream { pub fn new(sort: PhysicalSortExpr, streams: Vec) -> Self { let streams = streams.into_iter().map(|s| s.fuse()).collect(); Self { @@ -179,16 +179,16 @@ impl FieldCursorStream { } } - fn convert_batch(&mut self, batch: &RecordBatch) -> Result> { + fn convert_batch(&mut self, batch: &RecordBatch) -> Result> { let value = self.sort.expr.evaluate(batch)?; let array = value.into_array(batch.num_rows()); let array = array.as_any().downcast_ref::().expect("field values"); - Ok(FieldCursor::new(self.sort.options, array)) + Ok(ArrayValues::new(self.sort.options, array)) } } -impl PartitionedStream for FieldCursorStream { - type Output = Result<(FieldCursor, RecordBatch)>; +impl PartitionedStream for FieldCursorStream { + type Output = Result<(ArrayValues, RecordBatch)>; fn partitions(&self) -> usize { self.streams.0.len() From 1dd887cdff518ede1d1de457f4b20c22a9c7228f Mon Sep 17 00:00:00 2001 From: Liang-Chi Hsieh Date: Fri, 20 Oct 2023 00:19:49 -0700 Subject: [PATCH 103/572] Support Decimal256 column in create external table (#7866) * Support Decimal256 column in create external table * Update test * More --- datafusion/sql/src/utils.rs | 13 ++-- datafusion/sql/tests/sql_integration.rs | 16 ++++- .../sqllogictest/test_files/decimal.slt | 64 +++++++++++++++++++ 3 files changed, 86 insertions(+), 7 deletions(-) diff --git a/datafusion/sql/src/utils.rs b/datafusion/sql/src/utils.rs index 28eaf241fa6f..616a2fc74932 100644 --- a/datafusion/sql/src/utils.rs +++ b/datafusion/sql/src/utils.rs @@ -17,7 +17,9 @@ //! SQL Utility Functions -use arrow_schema::{DataType, DECIMAL128_MAX_PRECISION, DECIMAL_DEFAULT_SCALE}; +use arrow_schema::{ + DataType, DECIMAL128_MAX_PRECISION, DECIMAL256_MAX_PRECISION, DECIMAL_DEFAULT_SCALE, +}; use datafusion_common::tree_node::{Transformed, TreeNode}; use sqlparser::ast::Ident; @@ -221,14 +223,17 @@ pub(crate) fn make_decimal_type( (None, None) => (DECIMAL128_MAX_PRECISION, DECIMAL_DEFAULT_SCALE), }; - // Arrow decimal is i128 meaning 38 maximum decimal digits if precision == 0 - || precision > DECIMAL128_MAX_PRECISION + || precision > DECIMAL256_MAX_PRECISION || scale.unsigned_abs() > precision { plan_err!( - "Decimal(precision = {precision}, scale = {scale}) should satisfy `0 < precision <= 38`, and `scale <= precision`." + "Decimal(precision = {precision}, scale = {scale}) should satisfy `0 < precision <= 76`, and `scale <= precision`." ) + } else if precision > DECIMAL128_MAX_PRECISION + && precision <= DECIMAL256_MAX_PRECISION + { + Ok(DataType::Decimal256(precision, scale)) } else { Ok(DataType::Decimal128(precision, scale)) } diff --git a/datafusion/sql/tests/sql_integration.rs b/datafusion/sql/tests/sql_integration.rs index 653d2ec52d92..2446ee0a5841 100644 --- a/datafusion/sql/tests/sql_integration.rs +++ b/datafusion/sql/tests/sql_integration.rs @@ -201,7 +201,7 @@ fn cast_to_invalid_decimal_type_precision_0() { let sql = "SELECT CAST(10 AS DECIMAL(0))"; let err = logical_plan(sql).expect_err("query should have failed"); assert_eq!( - "Error during planning: Decimal(precision = 0, scale = 0) should satisfy `0 < precision <= 38`, and `scale <= precision`.", + "Error during planning: Decimal(precision = 0, scale = 0) should satisfy `0 < precision <= 76`, and `scale <= precision`.", err.strip_backtrace() ); } @@ -212,9 +212,19 @@ fn cast_to_invalid_decimal_type_precision_gt_38() { // precision > 38 { let sql = "SELECT CAST(10 AS DECIMAL(39))"; + let plan = "Projection: CAST(Int64(10) AS Decimal256(39, 0))\n EmptyRelation"; + quick_test(sql, plan); + } +} + +#[test] +fn cast_to_invalid_decimal_type_precision_gt_76() { + // precision > 76 + { + let sql = "SELECT CAST(10 AS DECIMAL(79))"; let err = logical_plan(sql).expect_err("query should have failed"); assert_eq!( - "Error during planning: Decimal(precision = 39, scale = 0) should satisfy `0 < precision <= 38`, and `scale <= precision`.", + "Error during planning: Decimal(precision = 79, scale = 0) should satisfy `0 < precision <= 76`, and `scale <= precision`.", err.strip_backtrace() ); } @@ -227,7 +237,7 @@ fn cast_to_invalid_decimal_type_precision_lt_scale() { let sql = "SELECT CAST(10 AS DECIMAL(5, 10))"; let err = logical_plan(sql).expect_err("query should have failed"); assert_eq!( - "Error during planning: Decimal(precision = 5, scale = 10) should satisfy `0 < precision <= 38`, and `scale <= precision`.", + "Error during planning: Decimal(precision = 5, scale = 10) should satisfy `0 < precision <= 76`, and `scale <= precision`.", err.strip_backtrace() ); } diff --git a/datafusion/sqllogictest/test_files/decimal.slt b/datafusion/sqllogictest/test_files/decimal.slt index 570116b7a2a2..f968ffb90a1c 100644 --- a/datafusion/sqllogictest/test_files/decimal.slt +++ b/datafusion/sqllogictest/test_files/decimal.slt @@ -629,3 +629,67 @@ select AVG(column1) from t; statement ok drop table t; + +statement ok +CREATE EXTERNAL TABLE decimal256_simple ( +c1 DECIMAL(50,6) NOT NULL, +c2 DOUBLE NOT NULL, +c3 BIGINT NOT NULL, +c4 BOOLEAN NOT NULL, +c5 DECIMAL(52,7) NOT NULL +) +STORED AS CSV +WITH HEADER ROW +LOCATION '../core/tests/data/decimal_data.csv'; + +query TT +select arrow_typeof(c1), arrow_typeof(c5) from decimal256_simple limit 1; +---- +Decimal256(50, 6) Decimal256(52, 7) + +query R rowsort +SELECT c1 from decimal256_simple; +---- +0.00001 +0.00002 +0.00002 +0.00003 +0.00003 +0.00003 +0.00004 +0.00004 +0.00004 +0.00004 +0.00005 +0.00005 +0.00005 +0.00005 +0.00005 + +query R rowsort +select c1 from decimal256_simple where c1 > 0.000030; +---- +0.00004 +0.00004 +0.00004 +0.00004 +0.00005 +0.00005 +0.00005 +0.00005 +0.00005 + +query RRIBR rowsort +select * from decimal256_simple where c1 > c5; +---- +0.00002 0.000000000002 3 false 0.000019 +0.00003 0.000000000003 5 true 0.000011 +0.00005 0.000000000005 8 false 0.000033 + +query TR +select arrow_typeof(avg(c1)), avg(c1) from decimal256_simple; +---- +Decimal256(54, 10) 0.0000366666 + +statement ok +drop table decimal256_simple; From 113a7bdc5a093448e0d2884de9976cb27ee1d7e7 Mon Sep 17 00:00:00 2001 From: Liang-Chi Hsieh Date: Fri, 20 Oct 2023 05:13:15 -0700 Subject: [PATCH 104/572] Support Decimal256 in Min/Max aggregate expressions (#7881) --- .../physical-expr/src/aggregate/min_max.rs | 34 +++++++++++++++++++ .../sqllogictest/test_files/decimal.slt | 10 ++++++ 2 files changed, 44 insertions(+) diff --git a/datafusion/physical-expr/src/aggregate/min_max.rs b/datafusion/physical-expr/src/aggregate/min_max.rs index 5c4c48b15803..f5b708e8894e 100644 --- a/datafusion/physical-expr/src/aggregate/min_max.rs +++ b/datafusion/physical-expr/src/aggregate/min_max.rs @@ -53,6 +53,9 @@ use crate::aggregate::utils::down_cast_any_ref; use crate::expressions::format_state_name; use arrow::array::Array; use arrow::array::Decimal128Array; +use arrow::array::Decimal256Array; +use arrow::datatypes::i256; +use arrow::datatypes::Decimal256Type; use super::moving_min_max; @@ -183,6 +186,7 @@ impl AggregateExpr for Max { | Float32 | Float64 | Decimal128(_, _) + | Decimal256(_, _) | Date32 | Date64 | Time32(_) @@ -239,6 +243,9 @@ impl AggregateExpr for Max { Decimal128(_, _) => { instantiate_max_accumulator!(self, i128, Decimal128Type) } + Decimal256(_, _) => { + instantiate_max_accumulator!(self, i256, Decimal256Type) + } // It would be nice to have a fast implementation for Strings as well // https://github.com/apache/arrow-datafusion/issues/6906 @@ -318,6 +325,16 @@ macro_rules! min_max_batch { scale ) } + DataType::Decimal256(precision, scale) => { + typed_min_max_batch!( + $VALUES, + Decimal256Array, + Decimal256, + $OP, + precision, + scale + ) + } // all types that have a natural order DataType::Float64 => { typed_min_max_batch!($VALUES, Float64Array, Float64, $OP) @@ -522,6 +539,19 @@ macro_rules! min_max { ); } } + ( + lhs @ ScalarValue::Decimal256(lhsv, lhsp, lhss), + rhs @ ScalarValue::Decimal256(rhsv, rhsp, rhss) + ) => { + if lhsp.eq(rhsp) && lhss.eq(rhss) { + typed_min_max!(lhsv, rhsv, Decimal256, $OP, lhsp, lhss) + } else { + return internal_err!( + "MIN/MAX is not expected to receive scalars of incompatible types {:?}", + (lhs, rhs) + ); + } + } (ScalarValue::Boolean(lhs), ScalarValue::Boolean(rhs)) => { typed_min_max!(lhs, rhs, Boolean, $OP) } @@ -880,6 +910,7 @@ impl AggregateExpr for Min { | Float32 | Float64 | Decimal128(_, _) + | Decimal256(_, _) | Date32 | Date64 | Time32(_) @@ -935,6 +966,9 @@ impl AggregateExpr for Min { Decimal128(_, _) => { instantiate_min_accumulator!(self, i128, Decimal128Type) } + Decimal256(_, _) => { + instantiate_min_accumulator!(self, i256, Decimal256Type) + } // This is only reached if groups_accumulator_supported is out of sync _ => internal_err!( "GroupsAccumulator not supported for min({})", diff --git a/datafusion/sqllogictest/test_files/decimal.slt b/datafusion/sqllogictest/test_files/decimal.slt index f968ffb90a1c..87a846c07727 100644 --- a/datafusion/sqllogictest/test_files/decimal.slt +++ b/datafusion/sqllogictest/test_files/decimal.slt @@ -691,5 +691,15 @@ select arrow_typeof(avg(c1)), avg(c1) from decimal256_simple; ---- Decimal256(54, 10) 0.0000366666 +query TR +select arrow_typeof(min(c1)), min(c1) from decimal256_simple where c4=false; +---- +Decimal256(50, 6) 0.00002 + +query TR +select arrow_typeof(max(c1)), max(c1) from decimal256_simple where c4=false; +---- +Decimal256(50, 6) 0.00005 + statement ok drop table decimal256_simple; From e17ca27d65d537e5ec6dcdb8cd42a6ba8f520c6c Mon Sep 17 00:00:00 2001 From: Devin D'Angelo Date: Fri, 20 Oct 2023 13:44:00 -0400 Subject: [PATCH 105/572] Implement Hive-Style Partitioned Write Support (#7801) * squash * rebase * add issue link * docs and cargo fmt * Apply suggestions from code review apply review suggestions Co-authored-by: Andrew Lamb * refactor remove_partition_by_columns * make part cols unambiguous * cargo fixes * fix for nested columns --------- Co-authored-by: Andrew Lamb --- datafusion/common/src/config.rs | 5 - .../core/src/datasource/file_format/csv.rs | 25 +- .../core/src/datasource/file_format/json.rs | 43 +- .../src/datasource/file_format/parquet.rs | 47 +- .../core/src/datasource/file_format/write.rs | 814 ------------------ .../src/datasource/file_format/write/demux.rs | 380 ++++++++ .../src/datasource/file_format/write/mod.rs | 305 +++++++ .../file_format/write/orchestration.rs | 415 +++++++++ .../core/src/datasource/listing/table.rs | 6 - .../test_files/information_schema.slt | 2 - .../test_files/insert_to_external.slt | 121 +++ docs/source/user-guide/configs.md | 1 - 12 files changed, 1305 insertions(+), 859 deletions(-) delete mode 100644 datafusion/core/src/datasource/file_format/write.rs create mode 100644 datafusion/core/src/datasource/file_format/write/demux.rs create mode 100644 datafusion/core/src/datasource/file_format/write/mod.rs create mode 100644 datafusion/core/src/datasource/file_format/write/orchestration.rs diff --git a/datafusion/common/src/config.rs b/datafusion/common/src/config.rs index 86e6221c3bc5..5e1064ada6f7 100644 --- a/datafusion/common/src/config.rs +++ b/datafusion/common/src/config.rs @@ -261,11 +261,6 @@ config_namespace! { /// number of rows written is not roughly divisible by the soft max pub soft_max_rows_per_output_file: usize, default = 50000000 - /// This is the maximum number of output files being written - /// in parallel. Higher values can potentially give faster write - /// performance at the cost of higher peak memory consumption. - pub max_parallel_ouput_files: usize, default = 8 - /// This is the maximum number of RecordBatches buffered /// for each output file being worked. Higher values can potentially /// give faster write performance at the cost of higher peak diff --git a/datafusion/core/src/datasource/file_format/csv.rs b/datafusion/core/src/datasource/file_format/csv.rs index bc01b29ba04b..a45fbf11f35c 100644 --- a/datafusion/core/src/datasource/file_format/csv.rs +++ b/datafusion/core/src/datasource/file_format/csv.rs @@ -23,7 +23,18 @@ use std::fmt; use std::fmt::Debug; use std::sync::Arc; -use super::write::{stateless_append_all, stateless_multipart_put}; +use arrow_array::RecordBatch; +use datafusion_common::{exec_err, not_impl_err, DataFusionError, FileType}; +use datafusion_execution::TaskContext; +use datafusion_physical_expr::{PhysicalExpr, PhysicalSortRequirement}; + +use bytes::{Buf, Bytes}; +use datafusion_physical_plan::metrics::MetricsSet; +use futures::stream::BoxStream; +use futures::{pin_mut, Stream, StreamExt, TryStreamExt}; +use object_store::{delimited::newline_delimited_stream, ObjectMeta, ObjectStore}; + +use super::write::orchestration::{stateless_append_all, stateless_multipart_put}; use super::{FileFormat, DEFAULT_SCHEMA_INFER_MAX_RECORD}; use crate::datasource::file_format::file_compression_type::FileCompressionType; use crate::datasource::file_format::write::{BatchSerializer, FileWriterMode}; @@ -39,17 +50,8 @@ use crate::physical_plan::{ExecutionPlan, SendableRecordBatchStream}; use arrow::csv::WriterBuilder; use arrow::datatypes::{DataType, Field, Fields, Schema}; use arrow::{self, datatypes::SchemaRef}; -use arrow_array::RecordBatch; -use datafusion_common::{exec_err, not_impl_err, DataFusionError, FileType}; -use datafusion_execution::TaskContext; -use datafusion_physical_expr::{PhysicalExpr, PhysicalSortRequirement}; -use datafusion_physical_plan::metrics::MetricsSet; use async_trait::async_trait; -use bytes::{Buf, Bytes}; -use futures::stream::BoxStream; -use futures::{pin_mut, Stream, StreamExt, TryStreamExt}; -use object_store::{delimited::newline_delimited_stream, ObjectMeta, ObjectStore}; /// Character Separated Value `FileFormat` implementation. #[derive(Debug)] @@ -485,6 +487,9 @@ impl CsvSink { data: SendableRecordBatchStream, context: &Arc, ) -> Result { + if !self.config.table_partition_cols.is_empty() { + return Err(DataFusionError::NotImplemented("Inserting in append mode to hive style partitioned tables is not supported".into())); + } let writer_options = self.config.file_type_writer_options.try_into_csv()?; let (builder, compression) = (&writer_options.writer_options, &writer_options.compression); diff --git a/datafusion/core/src/datasource/file_format/json.rs b/datafusion/core/src/datasource/file_format/json.rs index e1f8ab0d57b7..70cfd1836efe 100644 --- a/datafusion/core/src/datasource/file_format/json.rs +++ b/datafusion/core/src/datasource/file_format/json.rs @@ -23,32 +23,41 @@ use std::fmt::Debug; use std::io::BufReader; use std::sync::Arc; -use super::write::{stateless_append_all, stateless_multipart_put}; use super::{FileFormat, FileScanConfig}; +use arrow::datatypes::Schema; +use arrow::datatypes::SchemaRef; +use arrow::json; +use arrow::json::reader::infer_json_schema_from_iterator; +use arrow::json::reader::ValueIter; +use arrow_array::RecordBatch; +use async_trait::async_trait; +use bytes::Buf; + +use bytes::Bytes; +use datafusion_physical_expr::PhysicalExpr; +use datafusion_physical_expr::PhysicalSortRequirement; +use datafusion_physical_plan::ExecutionPlan; +use object_store::{GetResultPayload, ObjectMeta, ObjectStore}; + +use crate::datasource::physical_plan::FileGroupDisplay; +use crate::physical_plan::insert::DataSink; +use crate::physical_plan::insert::FileSinkExec; +use crate::physical_plan::SendableRecordBatchStream; +use crate::physical_plan::{DisplayAs, DisplayFormatType, Statistics}; + +use super::write::orchestration::{stateless_append_all, stateless_multipart_put}; + use crate::datasource::file_format::file_compression_type::FileCompressionType; use crate::datasource::file_format::write::{BatchSerializer, FileWriterMode}; use crate::datasource::file_format::DEFAULT_SCHEMA_INFER_MAX_RECORD; -use crate::datasource::physical_plan::{FileGroupDisplay, FileSinkConfig, NdJsonExec}; +use crate::datasource::physical_plan::{FileSinkConfig, NdJsonExec}; use crate::error::Result; use crate::execution::context::SessionState; -use crate::physical_plan::insert::{DataSink, FileSinkExec}; -use crate::physical_plan::{ - DisplayAs, DisplayFormatType, ExecutionPlan, SendableRecordBatchStream, Statistics, -}; -use arrow::datatypes::{Schema, SchemaRef}; -use arrow::json; -use arrow::json::reader::{infer_json_schema_from_iterator, ValueIter}; -use arrow_array::RecordBatch; use datafusion_common::{not_impl_err, DataFusionError, FileType}; use datafusion_execution::TaskContext; -use datafusion_physical_expr::{PhysicalExpr, PhysicalSortRequirement}; use datafusion_physical_plan::metrics::MetricsSet; -use async_trait::async_trait; -use bytes::{Buf, Bytes}; -use object_store::{GetResultPayload, ObjectMeta, ObjectStore}; - /// New line delimited JSON `FileFormat` implementation. #[derive(Debug)] pub struct JsonFormat { @@ -258,6 +267,10 @@ impl JsonSink { data: SendableRecordBatchStream, context: &Arc, ) -> Result { + if !self.config.table_partition_cols.is_empty() { + return Err(DataFusionError::NotImplemented("Inserting in append mode to hive style partitioned tables is not supported".into())); + } + let writer_options = self.config.file_type_writer_options.try_into_json()?; let compression = &writer_options.compression; diff --git a/datafusion/core/src/datasource/file_format/parquet.rs b/datafusion/core/src/datasource/file_format/parquet.rs index 12d5d515bbcd..a16db9d43213 100644 --- a/datafusion/core/src/datasource/file_format/parquet.rs +++ b/datafusion/core/src/datasource/file_format/parquet.rs @@ -23,10 +23,15 @@ use std::fmt::Debug; use std::io::Write; use std::sync::Arc; -use super::write::{create_writer, start_demuxer_task, AbortableWrite, FileWriterMode}; +use super::write::demux::start_demuxer_task; +use super::write::{create_writer, AbortableWrite, FileWriterMode}; use super::{FileFormat, FileScanConfig}; - +use crate::arrow::array::{ + BooleanArray, Float32Array, Float64Array, Int32Array, Int64Array, +}; +use crate::arrow::datatypes::DataType; use crate::config::ConfigOptions; + use crate::datasource::file_format::file_compression_type::FileCompressionType; use crate::datasource::get_col_stats; use crate::datasource::physical_plan::{ @@ -42,8 +47,7 @@ use crate::physical_plan::{ Statistics, }; -use arrow::array::{BooleanArray, Float32Array, Float64Array, Int32Array, Int64Array}; -use arrow::datatypes::{DataType, Fields, Schema, SchemaRef}; +use arrow::datatypes::{Fields, Schema, SchemaRef}; use datafusion_common::stats::Precision; use datafusion_common::{exec_err, not_impl_err, plan_err, DataFusionError, FileType}; use datafusion_execution::TaskContext; @@ -604,6 +608,31 @@ impl ParquetSink { Self { config } } + /// Converts table schema to writer schema, which may differ in the case + /// of hive style partitioning where some columns are removed from the + /// underlying files. + fn get_writer_schema(&self) -> Arc { + if !self.config.table_partition_cols.is_empty() { + let schema = self.config.output_schema(); + let partition_names: Vec<_> = self + .config + .table_partition_cols + .iter() + .map(|(s, _)| s) + .collect(); + Arc::new(Schema::new( + schema + .fields() + .iter() + .filter(|f| !partition_names.contains(&f.name())) + .map(|f| (**f).clone()) + .collect::>(), + )) + } else { + self.config.output_schema().clone() + } + } + /// Creates an AsyncArrowWriter which serializes a parquet file to an ObjectStore /// AsyncArrowWriters are used when individual parquet file serialization is not parallelized async fn create_async_arrow_writer( @@ -631,7 +660,7 @@ impl ParquetSink { .map_err(DataFusionError::ObjectStore)?; let writer = AsyncArrowWriter::try_new( multipart_writer, - self.config.output_schema.clone(), + self.get_writer_schema(), 10485760, Some(parquet_props), )?; @@ -721,10 +750,16 @@ impl DataSink for ParquetSink { .map(|r| r as u64); } + let part_col = if !self.config.table_partition_cols.is_empty() { + Some(self.config.table_partition_cols.clone()) + } else { + None + }; + let (demux_task, mut file_stream_rx) = start_demuxer_task( data, context, - None, + part_col, self.config.table_paths[0].clone(), "parquet".into(), self.config.single_file_output, diff --git a/datafusion/core/src/datasource/file_format/write.rs b/datafusion/core/src/datasource/file_format/write.rs deleted file mode 100644 index 928d0d1ba595..000000000000 --- a/datafusion/core/src/datasource/file_format/write.rs +++ /dev/null @@ -1,814 +0,0 @@ -// Licensed to the Apache Software Foundation (ASF) under one -// or more contributor license agreements. See the NOTICE file -// distributed with this work for additional information -// regarding copyright ownership. The ASF licenses this file -// to you under the Apache License, Version 2.0 (the -// "License"); you may not use this file except in compliance -// with the License. You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, -// software distributed under the License is distributed on an -// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -// KIND, either express or implied. See the License for the -// specific language governing permissions and limitations -// under the License. - -//! Module containing helper methods/traits related to enabling -//! write support for the various file formats - -use std::io::Error; -use std::mem; -use std::pin::Pin; -use std::sync::Arc; -use std::task::{Context, Poll}; - -use crate::datasource::file_format::file_compression_type::FileCompressionType; -use crate::datasource::listing::{ListingTableUrl, PartitionedFile}; -use crate::datasource::physical_plan::{FileMeta, FileSinkConfig}; -use crate::error::Result; -use crate::physical_plan::SendableRecordBatchStream; - -use arrow_array::RecordBatch; -use datafusion_common::{exec_err, DataFusionError}; - -use async_trait::async_trait; -use bytes::Bytes; -use datafusion_execution::TaskContext; -use futures::future::BoxFuture; -use futures::FutureExt; -use futures::{ready, StreamExt}; -use object_store::path::Path; -use object_store::{MultipartId, ObjectMeta, ObjectStore}; -use rand::distributions::DistString; -use tokio::io::{AsyncWrite, AsyncWriteExt}; -use tokio::sync::mpsc::{self, Receiver, Sender}; -use tokio::task::{JoinHandle, JoinSet}; -use tokio::try_join; - -/// `AsyncPutWriter` is an object that facilitates asynchronous writing to object stores. -/// It is specifically designed for the `object_store` crate's `put` method and sends -/// whole bytes at once when the buffer is flushed. -pub struct AsyncPutWriter { - /// Object metadata - object_meta: ObjectMeta, - /// A shared reference to the object store - store: Arc, - /// A buffer that stores the bytes to be sent - current_buffer: Vec, - /// Used for async handling in flush method - inner_state: AsyncPutState, -} - -impl AsyncPutWriter { - /// Constructor for the `AsyncPutWriter` object - pub fn new(object_meta: ObjectMeta, store: Arc) -> Self { - Self { - object_meta, - store, - current_buffer: vec![], - // The writer starts out in buffering mode - inner_state: AsyncPutState::Buffer, - } - } - - /// Separate implementation function that unpins the [`AsyncPutWriter`] so - /// that partial borrows work correctly - fn poll_shutdown_inner( - &mut self, - cx: &mut Context<'_>, - ) -> Poll> { - loop { - match &mut self.inner_state { - AsyncPutState::Buffer => { - // Convert the current buffer to bytes and take ownership of it - let bytes = Bytes::from(mem::take(&mut self.current_buffer)); - // Set the inner state to Put variant with the bytes - self.inner_state = AsyncPutState::Put { bytes } - } - AsyncPutState::Put { bytes } => { - // Send the bytes to the object store's put method - return Poll::Ready( - ready!(self - .store - .put(&self.object_meta.location, bytes.clone()) - .poll_unpin(cx)) - .map_err(Error::from), - ); - } - } - } - } -} - -/// An enum that represents the inner state of AsyncPut -enum AsyncPutState { - /// Building Bytes struct in this state - Buffer, - /// Data in the buffer is being sent to the object store - Put { bytes: Bytes }, -} - -impl AsyncWrite for AsyncPutWriter { - // Define the implementation of the AsyncWrite trait for the `AsyncPutWriter` struct - fn poll_write( - mut self: Pin<&mut Self>, - _: &mut Context<'_>, - buf: &[u8], - ) -> Poll> { - // Extend the current buffer with the incoming buffer - self.current_buffer.extend_from_slice(buf); - // Return a ready poll with the length of the incoming buffer - Poll::Ready(Ok(buf.len())) - } - - fn poll_flush( - self: Pin<&mut Self>, - _: &mut Context<'_>, - ) -> Poll> { - // Return a ready poll with an empty result - Poll::Ready(Ok(())) - } - - fn poll_shutdown( - mut self: Pin<&mut Self>, - cx: &mut Context<'_>, - ) -> Poll> { - // Call the poll_shutdown_inner method to handle the actual sending of data to the object store - self.poll_shutdown_inner(cx) - } -} - -/// Stores data needed during abortion of MultiPart writers -pub(crate) struct MultiPart { - /// A shared reference to the object store - store: Arc, - multipart_id: MultipartId, - location: Path, -} - -impl MultiPart { - /// Create a new `MultiPart` - pub fn new( - store: Arc, - multipart_id: MultipartId, - location: Path, - ) -> Self { - Self { - store, - multipart_id, - location, - } - } -} - -pub(crate) enum AbortMode { - Put, - Append, - MultiPart(MultiPart), -} - -/// A wrapper struct with abort method and writer -pub(crate) struct AbortableWrite { - writer: W, - mode: AbortMode, -} - -impl AbortableWrite { - /// Create a new `AbortableWrite` instance with the given writer, and write mode. - pub(crate) fn new(writer: W, mode: AbortMode) -> Self { - Self { writer, mode } - } - - /// handling of abort for different write modes - pub(crate) fn abort_writer(&self) -> Result>> { - match &self.mode { - AbortMode::Put => Ok(async { Ok(()) }.boxed()), - AbortMode::Append => exec_err!("Cannot abort in append mode"), - AbortMode::MultiPart(MultiPart { - store, - multipart_id, - location, - }) => { - let location = location.clone(); - let multipart_id = multipart_id.clone(); - let store = store.clone(); - Ok(Box::pin(async move { - store - .abort_multipart(&location, &multipart_id) - .await - .map_err(DataFusionError::ObjectStore) - })) - } - } - } -} - -impl AsyncWrite for AbortableWrite { - fn poll_write( - self: Pin<&mut Self>, - cx: &mut Context<'_>, - buf: &[u8], - ) -> Poll> { - Pin::new(&mut self.get_mut().writer).poll_write(cx, buf) - } - - fn poll_flush( - self: Pin<&mut Self>, - cx: &mut Context<'_>, - ) -> Poll> { - Pin::new(&mut self.get_mut().writer).poll_flush(cx) - } - - fn poll_shutdown( - self: Pin<&mut Self>, - cx: &mut Context<'_>, - ) -> Poll> { - Pin::new(&mut self.get_mut().writer).poll_shutdown(cx) - } -} - -/// An enum that defines different file writer modes. -#[derive(Debug, Clone, Copy)] -pub enum FileWriterMode { - /// Data is appended to an existing file. - Append, - /// Data is written to a new file. - Put, - /// Data is written to a new file in multiple parts. - PutMultipart, -} -/// A trait that defines the methods required for a RecordBatch serializer. -#[async_trait] -pub trait BatchSerializer: Unpin + Send { - /// Asynchronously serializes a `RecordBatch` and returns the serialized bytes. - async fn serialize(&mut self, batch: RecordBatch) -> Result; - /// Duplicates self to support serializing multiple batches in parallel on multiple cores - fn duplicate(&mut self) -> Result> { - Err(DataFusionError::NotImplemented( - "Parallel serialization is not implemented for this file type".into(), - )) - } -} - -/// Returns an [`AbortableWrite`] which writes to the given object store location -/// with the specified compression -pub(crate) async fn create_writer( - writer_mode: FileWriterMode, - file_compression_type: FileCompressionType, - file_meta: FileMeta, - object_store: Arc, -) -> Result>> { - let object = &file_meta.object_meta; - match writer_mode { - // If the mode is append, call the store's append method and return wrapped in - // a boxed trait object. - FileWriterMode::Append => { - let writer = object_store - .append(&object.location) - .await - .map_err(DataFusionError::ObjectStore)?; - let writer = AbortableWrite::new( - file_compression_type.convert_async_writer(writer)?, - AbortMode::Append, - ); - Ok(writer) - } - // If the mode is put, create a new AsyncPut writer and return it wrapped in - // a boxed trait object - FileWriterMode::Put => { - let writer = Box::new(AsyncPutWriter::new(object.clone(), object_store)); - let writer = AbortableWrite::new( - file_compression_type.convert_async_writer(writer)?, - AbortMode::Put, - ); - Ok(writer) - } - // If the mode is put multipart, call the store's put_multipart method and - // return the writer wrapped in a boxed trait object. - FileWriterMode::PutMultipart => { - let (multipart_id, writer) = object_store - .put_multipart(&object.location) - .await - .map_err(DataFusionError::ObjectStore)?; - Ok(AbortableWrite::new( - file_compression_type.convert_async_writer(writer)?, - AbortMode::MultiPart(MultiPart::new( - object_store, - multipart_id, - object.location.clone(), - )), - )) - } - } -} - -type WriterType = AbortableWrite>; -type SerializerType = Box; - -/// Serializes a single data stream in parallel and writes to an ObjectStore -/// concurrently. Data order is preserved. In the event of an error, -/// the ObjectStore writer is returned to the caller in addition to an error, -/// so that the caller may handle aborting failed writes. -pub(crate) async fn serialize_rb_stream_to_object_store( - mut data_rx: Receiver, - mut serializer: Box, - mut writer: AbortableWrite>, - unbounded_input: bool, -) -> std::result::Result<(WriterType, u64), (WriterType, DataFusionError)> { - let (tx, mut rx) = - mpsc::channel::>>(100); - - let serialize_task = tokio::spawn(async move { - while let Some(batch) = data_rx.recv().await { - match serializer.duplicate() { - Ok(mut serializer_clone) => { - let handle = tokio::spawn(async move { - let num_rows = batch.num_rows(); - let bytes = serializer_clone.serialize(batch).await?; - Ok((num_rows, bytes)) - }); - tx.send(handle).await.map_err(|_| { - DataFusionError::Internal( - "Unknown error writing to object store".into(), - ) - })?; - if unbounded_input { - tokio::task::yield_now().await; - } - } - Err(_) => { - return Err(DataFusionError::Internal( - "Unknown error writing to object store".into(), - )) - } - } - } - Ok(()) - }); - - let mut row_count = 0; - while let Some(handle) = rx.recv().await { - match handle.await { - Ok(Ok((cnt, bytes))) => { - match writer.write_all(&bytes).await { - Ok(_) => (), - Err(e) => { - return Err(( - writer, - DataFusionError::Execution(format!( - "Error writing to object store: {e}" - )), - )) - } - }; - row_count += cnt; - } - Ok(Err(e)) => { - // Return the writer along with the error - return Err((writer, e)); - } - Err(e) => { - // Handle task panic or cancellation - return Err(( - writer, - DataFusionError::Execution(format!( - "Serialization task panicked or was cancelled: {e}" - )), - )); - } - } - } - - match serialize_task.await { - Ok(Ok(_)) => (), - Ok(Err(e)) => return Err((writer, e)), - Err(_) => { - return Err(( - writer, - DataFusionError::Internal("Unknown error writing to object store".into()), - )) - } - }; - Ok((writer, row_count as u64)) -} - -type RecordBatchReceiver = Receiver; -type DemuxedStreamReceiver = Receiver<(Path, RecordBatchReceiver)>; - -/// Splits a single [SendableRecordBatchStream] into a dynamically determined -/// number of partitions at execution time. The partitions are determined by -/// factors known only at execution time, such as total number of rows and -/// partition column values. The demuxer task communicates to the caller -/// by sending channels over a channel. The inner channels send RecordBatches -/// which should be contained within the same output file. The outer channel -/// is used to send a dynamic number of inner channels, representing a dynamic -/// number of total output files. The caller is also responsible to monitor -/// the demux task for errors and abort accordingly. The single_file_ouput parameter -/// overrides all other settings to force only a single file to be written. -/// partition_by parameter will additionally split the input based on the unique -/// values of a specific column ``` -/// ┌───────────┐ ┌────────────┐ ┌─────────────┐ -/// ┌──────▶ │ batch 1 ├────▶...──────▶│ Batch a │ │ Output File1│ -/// │ └───────────┘ └────────────┘ └─────────────┘ -/// │ -/// ┌──────────┐ │ ┌───────────┐ ┌────────────┐ ┌─────────────┐ -/// ┌───────────┐ ┌────────────┐ │ │ ├──────▶ │ batch a+1├────▶...──────▶│ Batch b │ │ Output File2│ -/// │ batch 1 ├────▶...──────▶│ Batch N ├─────▶│ Demux ├────────┤ ... └───────────┘ └────────────┘ └─────────────┘ -/// └───────────┘ └────────────┘ │ │ │ -/// └──────────┘ │ ┌───────────┐ ┌────────────┐ ┌─────────────┐ -/// └──────▶ │ batch d ├────▶...──────▶│ Batch n │ │ Output FileN│ -/// └───────────┘ └────────────┘ └─────────────┘ -pub(crate) fn start_demuxer_task( - input: SendableRecordBatchStream, - context: &Arc, - _partition_by: Option<&str>, - base_output_path: ListingTableUrl, - file_extension: String, - single_file_output: bool, -) -> (JoinHandle>, DemuxedStreamReceiver) { - let exec_options = &context.session_config().options().execution; - - let max_rows_per_file = exec_options.soft_max_rows_per_output_file; - let max_parallel_files = exec_options.max_parallel_ouput_files; - let max_buffered_batches = exec_options.max_buffered_batches_per_output_file; - - let (tx, rx) = mpsc::channel(max_parallel_files); - - let task = tokio::spawn(async move { - row_count_demuxer( - input, - base_output_path, - file_extension, - single_file_output, - max_rows_per_file, - max_buffered_batches, - tx, - ) - .await - }); - (task, rx) -} - -fn generate_file_path( - base_output_path: &ListingTableUrl, - write_id: &str, - part_idx: usize, - file_extension: &str, - single_file_output: bool, -) -> Path { - if !single_file_output { - base_output_path - .prefix() - .child(format!("{}_{}.{}", write_id, part_idx, file_extension)) - } else { - base_output_path.prefix().to_owned() - } -} - -async fn create_new_file_stream( - base_output_path: &ListingTableUrl, - write_id: &str, - part_idx: usize, - file_extension: &str, - single_file_output: bool, - max_buffered_batches: usize, - tx: &mut Sender<(Path, Receiver)>, -) -> Result> { - let file_path = generate_file_path( - base_output_path, - write_id, - part_idx, - file_extension, - single_file_output, - ); - let (tx_file, rx_file) = mpsc::channel(max_buffered_batches / 2); - tx.send((file_path, rx_file)).await.map_err(|_| { - DataFusionError::Execution("Error sending RecordBatch to file stream!".into()) - })?; - Ok(tx_file) -} - -async fn row_count_demuxer( - mut input: SendableRecordBatchStream, - base_output_path: ListingTableUrl, - file_extension: String, - single_file_output: bool, - max_rows_per_file: usize, - max_buffered_batches: usize, - mut tx: Sender<(Path, Receiver)>, -) -> Result<()> { - let mut total_rows_current_file = 0; - let mut part_idx = 0; - let write_id = - rand::distributions::Alphanumeric.sample_string(&mut rand::thread_rng(), 16); - - let mut tx_file = create_new_file_stream( - &base_output_path, - &write_id, - part_idx, - &file_extension, - single_file_output, - max_buffered_batches, - &mut tx, - ) - .await?; - part_idx += 1; - - while let Some(rb) = input.next().await.transpose()? { - total_rows_current_file += rb.num_rows(); - tx_file.send(rb).await.map_err(|_| { - DataFusionError::Execution("Error sending RecordBatch to file stream!".into()) - })?; - - if total_rows_current_file >= max_rows_per_file && !single_file_output { - total_rows_current_file = 0; - tx_file = create_new_file_stream( - &base_output_path, - &write_id, - part_idx, - &file_extension, - single_file_output, - max_buffered_batches, - &mut tx, - ) - .await?; - part_idx += 1; - } - } - Ok(()) -} - -type FileWriteBundle = (Receiver, SerializerType, WriterType); -/// Contains the common logic for serializing RecordBatches and -/// writing the resulting bytes to an ObjectStore. -/// Serialization is assumed to be stateless, i.e. -/// each RecordBatch can be serialized without any -/// dependency on the RecordBatches before or after. -pub(crate) async fn stateless_serialize_and_write_files( - mut rx: Receiver, - tx: tokio::sync::oneshot::Sender, - unbounded_input: bool, -) -> Result<()> { - let mut row_count = 0; - // tracks if any writers encountered an error triggering the need to abort - let mut any_errors = false; - // tracks the specific error triggering abort - let mut triggering_error = None; - // tracks if any errors were encountered in the process of aborting writers. - // if true, we may not have a guarentee that all written data was cleaned up. - let mut any_abort_errors = false; - let mut join_set = JoinSet::new(); - while let Some((data_rx, serializer, writer)) = rx.recv().await { - join_set.spawn(async move { - serialize_rb_stream_to_object_store( - data_rx, - serializer, - writer, - unbounded_input, - ) - .await - }); - } - let mut finished_writers = Vec::new(); - while let Some(result) = join_set.join_next().await { - match result { - Ok(res) => match res { - Ok((writer, cnt)) => { - finished_writers.push(writer); - row_count += cnt; - } - Err((writer, e)) => { - finished_writers.push(writer); - any_errors = true; - triggering_error = Some(e); - } - }, - Err(e) => { - // Don't panic, instead try to clean up as many writers as possible. - // If we hit this code, ownership of a writer was not joined back to - // this thread, so we cannot clean it up (hence any_abort_errors is true) - any_errors = true; - any_abort_errors = true; - triggering_error = Some(DataFusionError::Internal(format!( - "Unexpected join error while serializing file {e}" - ))); - } - } - } - - // Finalize or abort writers as appropriate - for mut writer in finished_writers.into_iter() { - match any_errors { - true => { - let abort_result = writer.abort_writer(); - if abort_result.is_err() { - any_abort_errors = true; - } - } - false => { - writer.shutdown() - .await - .map_err(|_| DataFusionError::Internal("Error encountered while finalizing writes! Partial results may have been written to ObjectStore!".into()))?; - } - } - } - - if any_errors { - match any_abort_errors{ - true => return Err(DataFusionError::Internal("Error encountered during writing to ObjectStore and failed to abort all writers. Partial result may have been written.".into())), - false => match triggering_error { - Some(e) => return Err(e), - None => return Err(DataFusionError::Internal("Unknown Error encountered during writing to ObjectStore. All writers succesfully aborted.".into())) - } - } - } - - tx.send(row_count).map_err(|_| { - DataFusionError::Internal( - "Error encountered while sending row count back to file sink!".into(), - ) - })?; - Ok(()) -} - -/// Orchestrates multipart put of a dynamic number of output files from a single input stream -/// for any statelessly serialized file type. That is, any file type for which each [RecordBatch] -/// can be serialized independently of all other [RecordBatch]s. -pub(crate) async fn stateless_multipart_put( - data: SendableRecordBatchStream, - context: &Arc, - file_extension: String, - get_serializer: Box Box + Send>, - config: &FileSinkConfig, - compression: FileCompressionType, -) -> Result { - let object_store = context - .runtime_env() - .object_store(&config.object_store_url)?; - - let single_file_output = config.single_file_output; - let base_output_path = &config.table_paths[0]; - let unbounded_input = config.unbounded_input; - - let (demux_task, mut file_stream_rx) = start_demuxer_task( - data, - context, - None, - base_output_path.clone(), - file_extension, - single_file_output, - ); - - let rb_buffer_size = &context - .session_config() - .options() - .execution - .max_buffered_batches_per_output_file; - - let (tx_file_bundle, rx_file_bundle) = tokio::sync::mpsc::channel(rb_buffer_size / 2); - let (tx_row_cnt, rx_row_cnt) = tokio::sync::oneshot::channel(); - let write_coordinater_task = tokio::spawn(async move { - stateless_serialize_and_write_files(rx_file_bundle, tx_row_cnt, unbounded_input) - .await - }); - while let Some((output_location, rb_stream)) = file_stream_rx.recv().await { - let serializer = get_serializer(); - let object_meta = ObjectMeta { - location: output_location, - last_modified: chrono::offset::Utc::now(), - size: 0, - e_tag: None, - }; - let writer = create_writer( - FileWriterMode::PutMultipart, - compression, - object_meta.into(), - object_store.clone(), - ) - .await?; - - tx_file_bundle - .send((rb_stream, serializer, writer)) - .await - .map_err(|_| { - DataFusionError::Internal( - "Writer receive file bundle channel closed unexpectedly!".into(), - ) - })?; - } - - // Signal to the write coordinater that no more files are coming - drop(tx_file_bundle); - - let total_count = rx_row_cnt.await.map_err(|_| { - DataFusionError::Internal( - "Did not receieve row count from write coordinater".into(), - ) - })?; - - match try_join!(write_coordinater_task, demux_task) { - Ok((r1, r2)) => { - r1?; - r2?; - } - Err(e) => { - if e.is_panic() { - std::panic::resume_unwind(e.into_panic()); - } else { - unreachable!(); - } - } - } - - Ok(total_count) -} - -/// Orchestrates append_all for any statelessly serialized file type. Appends to all files provided -/// in a round robin fashion. -pub(crate) async fn stateless_append_all( - mut data: SendableRecordBatchStream, - context: &Arc, - object_store: Arc, - file_groups: &Vec, - unbounded_input: bool, - compression: FileCompressionType, - get_serializer: Box Box + Send>, -) -> Result { - let rb_buffer_size = &context - .session_config() - .options() - .execution - .max_buffered_batches_per_output_file; - - let (tx_file_bundle, rx_file_bundle) = tokio::sync::mpsc::channel(file_groups.len()); - let mut send_channels = vec![]; - for file_group in file_groups { - let serializer = get_serializer(file_group.object_meta.size); - - let file = file_group.clone(); - let writer = create_writer( - FileWriterMode::Append, - compression, - file.object_meta.clone().into(), - object_store.clone(), - ) - .await?; - - let (tx, rx) = tokio::sync::mpsc::channel(rb_buffer_size / 2); - send_channels.push(tx); - tx_file_bundle - .send((rx, serializer, writer)) - .await - .map_err(|_| { - DataFusionError::Internal( - "Writer receive file bundle channel closed unexpectedly!".into(), - ) - })?; - } - - let (tx_row_cnt, rx_row_cnt) = tokio::sync::oneshot::channel(); - let write_coordinater_task = tokio::spawn(async move { - stateless_serialize_and_write_files(rx_file_bundle, tx_row_cnt, unbounded_input) - .await - }); - - // Append to file groups in round robin - let mut next_file_idx = 0; - while let Some(rb) = data.next().await.transpose()? { - send_channels[next_file_idx].send(rb).await.map_err(|_| { - DataFusionError::Internal( - "Recordbatch file append stream closed unexpectedly!".into(), - ) - })?; - next_file_idx = (next_file_idx + 1) % send_channels.len(); - if unbounded_input { - tokio::task::yield_now().await; - } - } - // Signal to the write coordinater that no more files are coming - drop(tx_file_bundle); - drop(send_channels); - - let total_count = rx_row_cnt.await.map_err(|_| { - DataFusionError::Internal( - "Did not receieve row count from write coordinater".into(), - ) - })?; - - match try_join!(write_coordinater_task) { - Ok(r1) => { - r1.0?; - } - Err(e) => { - if e.is_panic() { - std::panic::resume_unwind(e.into_panic()); - } else { - unreachable!(); - } - } - } - - Ok(total_count) -} diff --git a/datafusion/core/src/datasource/file_format/write/demux.rs b/datafusion/core/src/datasource/file_format/write/demux.rs new file mode 100644 index 000000000000..2c44c0922c76 --- /dev/null +++ b/datafusion/core/src/datasource/file_format/write/demux.rs @@ -0,0 +1,380 @@ +// 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. + +//! Module containing helper methods/traits related to enabling +//! dividing input stream into multiple output files at execution time + +use std::collections::HashMap; + +use std::sync::Arc; + +use crate::datasource::listing::ListingTableUrl; + +use crate::error::Result; +use crate::physical_plan::SendableRecordBatchStream; + +use arrow_array::builder::UInt64Builder; +use arrow_array::cast::AsArray; +use arrow_array::{RecordBatch, StructArray}; +use arrow_schema::{DataType, Schema}; +use datafusion_common::cast::as_string_array; +use datafusion_common::DataFusionError; + +use datafusion_execution::TaskContext; + +use futures::StreamExt; +use object_store::path::Path; + +use rand::distributions::DistString; + +use tokio::sync::mpsc::{self, Receiver, Sender, UnboundedReceiver, UnboundedSender}; +use tokio::task::JoinHandle; + +type RecordBatchReceiver = Receiver; +type DemuxedStreamReceiver = UnboundedReceiver<(Path, RecordBatchReceiver)>; + +/// Splits a single [SendableRecordBatchStream] into a dynamically determined +/// number of partitions at execution time. The partitions are determined by +/// factors known only at execution time, such as total number of rows and +/// partition column values. The demuxer task communicates to the caller +/// by sending channels over a channel. The inner channels send RecordBatches +/// which should be contained within the same output file. The outer channel +/// is used to send a dynamic number of inner channels, representing a dynamic +/// number of total output files. The caller is also responsible to monitor +/// the demux task for errors and abort accordingly. The single_file_ouput parameter +/// overrides all other settings to force only a single file to be written. +/// partition_by parameter will additionally split the input based on the unique +/// values of a specific column ``` +/// ┌───────────┐ ┌────────────┐ ┌─────────────┐ +/// ┌──────▶ │ batch 1 ├────▶...──────▶│ Batch a │ │ Output File1│ +/// │ └───────────┘ └────────────┘ └─────────────┘ +/// │ +/// ┌──────────┐ │ ┌───────────┐ ┌────────────┐ ┌─────────────┐ +/// ┌───────────┐ ┌────────────┐ │ │ ├──────▶ │ batch a+1├────▶...──────▶│ Batch b │ │ Output File2│ +/// │ batch 1 ├────▶...──────▶│ Batch N ├─────▶│ Demux ├────────┤ ... └───────────┘ └────────────┘ └─────────────┘ +/// └───────────┘ └────────────┘ │ │ │ +/// └──────────┘ │ ┌───────────┐ ┌────────────┐ ┌─────────────┐ +/// └──────▶ │ batch d ├────▶...──────▶│ Batch n │ │ Output FileN│ +/// └───────────┘ └────────────┘ └─────────────┘ +pub(crate) fn start_demuxer_task( + input: SendableRecordBatchStream, + context: &Arc, + partition_by: Option>, + base_output_path: ListingTableUrl, + file_extension: String, + single_file_output: bool, +) -> (JoinHandle>, DemuxedStreamReceiver) { + let (tx, rx) = tokio::sync::mpsc::unbounded_channel(); + let context = context.clone(); + let task: JoinHandle> = match partition_by { + Some(parts) => { + // There could be an arbitrarily large number of parallel hive style partitions being written to, so we cannot + // bound this channel without risking a deadlock. + tokio::spawn(async move { + hive_style_partitions_demuxer( + tx, + input, + context, + parts, + base_output_path, + file_extension, + ) + .await + }) + } + None => tokio::spawn(async move { + row_count_demuxer( + tx, + input, + context, + base_output_path, + file_extension, + single_file_output, + ) + .await + }), + }; + + (task, rx) +} + +/// Dynamically partitions input stream to acheive desired maximum rows per file +async fn row_count_demuxer( + mut tx: UnboundedSender<(Path, Receiver)>, + mut input: SendableRecordBatchStream, + context: Arc, + base_output_path: ListingTableUrl, + file_extension: String, + single_file_output: bool, +) -> Result<()> { + let exec_options = &context.session_config().options().execution; + let max_rows_per_file = exec_options.soft_max_rows_per_output_file; + let max_buffered_batches = exec_options.max_buffered_batches_per_output_file; + let mut total_rows_current_file = 0; + let mut part_idx = 0; + let write_id = + rand::distributions::Alphanumeric.sample_string(&mut rand::thread_rng(), 16); + + let mut tx_file = create_new_file_stream( + &base_output_path, + &write_id, + part_idx, + &file_extension, + single_file_output, + max_buffered_batches, + &mut tx, + )?; + part_idx += 1; + + while let Some(rb) = input.next().await.transpose()? { + total_rows_current_file += rb.num_rows(); + tx_file.send(rb).await.map_err(|_| { + DataFusionError::Execution("Error sending RecordBatch to file stream!".into()) + })?; + + if total_rows_current_file >= max_rows_per_file && !single_file_output { + total_rows_current_file = 0; + tx_file = create_new_file_stream( + &base_output_path, + &write_id, + part_idx, + &file_extension, + single_file_output, + max_buffered_batches, + &mut tx, + )?; + part_idx += 1; + } + } + Ok(()) +} + +/// Helper for row count demuxer +fn generate_file_path( + base_output_path: &ListingTableUrl, + write_id: &str, + part_idx: usize, + file_extension: &str, + single_file_output: bool, +) -> Path { + if !single_file_output { + base_output_path + .prefix() + .child(format!("{}_{}.{}", write_id, part_idx, file_extension)) + } else { + base_output_path.prefix().to_owned() + } +} + +/// Helper for row count demuxer +fn create_new_file_stream( + base_output_path: &ListingTableUrl, + write_id: &str, + part_idx: usize, + file_extension: &str, + single_file_output: bool, + max_buffered_batches: usize, + tx: &mut UnboundedSender<(Path, Receiver)>, +) -> Result> { + let file_path = generate_file_path( + base_output_path, + write_id, + part_idx, + file_extension, + single_file_output, + ); + let (tx_file, rx_file) = mpsc::channel(max_buffered_batches / 2); + tx.send((file_path, rx_file)).map_err(|_| { + DataFusionError::Execution("Error sending RecordBatch to file stream!".into()) + })?; + Ok(tx_file) +} + +/// Splits an input stream based on the distinct values of a set of columns +/// Assumes standard hive style partition paths such as +/// /col1=val1/col2=val2/outputfile.parquet +async fn hive_style_partitions_demuxer( + tx: UnboundedSender<(Path, Receiver)>, + mut input: SendableRecordBatchStream, + context: Arc, + partition_by: Vec<(String, DataType)>, + base_output_path: ListingTableUrl, + file_extension: String, +) -> Result<()> { + let write_id = + rand::distributions::Alphanumeric.sample_string(&mut rand::thread_rng(), 16); + + let exec_options = &context.session_config().options().execution; + let max_buffered_recordbatches = exec_options.max_buffered_batches_per_output_file; + + // To support non string partition col types, cast the type to &str first + let mut value_map: HashMap, Sender> = HashMap::new(); + + while let Some(rb) = input.next().await.transpose()? { + // First compute partition key for each row of batch, e.g. (col1=val1, col2=val2, ...) + let all_partition_values = compute_partition_keys_by_row(&rb, &partition_by)?; + + // Next compute how the batch should be split up to take each distinct key to its own batch + let take_map = compute_take_arrays(&rb, all_partition_values); + + // Divide up the batch into distinct partition key batches and send each batch + for (part_key, mut builder) in take_map.into_iter() { + // Take method adapted from https://github.com/lancedb/lance/pull/1337/files + // TODO: upstream RecordBatch::take to arrow-rs + let take_indices = builder.finish(); + let struct_array: StructArray = rb.clone().into(); + let parted_batch = RecordBatch::try_from( + arrow::compute::take(&struct_array, &take_indices, None)?.as_struct(), + ) + .map_err(|_| { + DataFusionError::Internal("Unexpected error partitioning batch!".into()) + })?; + + // Get or create channel for this batch + let part_tx = match value_map.get_mut(&part_key) { + Some(part_tx) => part_tx, + None => { + // Create channel for previously unseen distinct partition key and notify consumer of new file + let (part_tx, part_rx) = tokio::sync::mpsc::channel::( + max_buffered_recordbatches, + ); + let file_path = compute_hive_style_file_path( + &part_key, + &partition_by, + &write_id, + &file_extension, + &base_output_path, + ); + + tx.send((file_path, part_rx)).map_err(|_| { + DataFusionError::Execution( + "Error sending new file stream!".into(), + ) + })?; + + value_map.insert(part_key.clone(), part_tx); + value_map + .get_mut(&part_key) + .ok_or(DataFusionError::Internal( + "Key must exist since it was just inserted!".into(), + ))? + } + }; + + // remove partitions columns + let final_batch_to_send = + remove_partition_by_columns(&parted_batch, &partition_by)?; + + // Finally send the partial batch partitioned by distinct value! + part_tx.send(final_batch_to_send).await.map_err(|_| { + DataFusionError::Internal("Unexpected error sending parted batch!".into()) + })?; + } + } + + Ok(()) +} + +fn compute_partition_keys_by_row<'a>( + rb: &'a RecordBatch, + partition_by: &'a [(String, DataType)], +) -> Result>> { + let mut all_partition_values = vec![]; + + for (col, dtype) in partition_by.iter() { + let mut partition_values = vec![]; + let col_array = + rb.column_by_name(col) + .ok_or(DataFusionError::Execution(format!( + "PartitionBy Column {} does not exist in source data!", + col + )))?; + + match dtype { + DataType::Utf8 => { + let array = as_string_array(col_array)?; + for i in 0..rb.num_rows() { + partition_values.push(array.value(i)); + } + } + _ => { + return Err(DataFusionError::NotImplemented(format!( + "it is not yet supported to write to hive partitions with datatype {}", + dtype + ))) + } + } + + all_partition_values.push(partition_values); + } + + Ok(all_partition_values) +} + +fn compute_take_arrays( + rb: &RecordBatch, + all_partition_values: Vec>, +) -> HashMap, UInt64Builder> { + let mut take_map = HashMap::new(); + for i in 0..rb.num_rows() { + let mut part_key = vec![]; + for vals in all_partition_values.iter() { + part_key.push(vals[i].to_owned()); + } + let builder = take_map.entry(part_key).or_insert(UInt64Builder::new()); + builder.append_value(i as u64); + } + take_map +} + +fn remove_partition_by_columns( + parted_batch: &RecordBatch, + partition_by: &Vec<(String, DataType)>, +) -> Result { + let end_idx = parted_batch.num_columns() - partition_by.len(); + let non_part_cols = &parted_batch.columns()[..end_idx]; + + let partition_names: Vec<_> = partition_by.iter().map(|(s, _)| s).collect(); + let non_part_schema = Schema::new( + parted_batch + .schema() + .fields() + .iter() + .filter(|f| !partition_names.contains(&f.name())) + .map(|f| (**f).clone()) + .collect::>(), + ); + let final_batch_to_send = + RecordBatch::try_new(Arc::new(non_part_schema), non_part_cols.into())?; + + Ok(final_batch_to_send) +} + +fn compute_hive_style_file_path( + part_key: &Vec, + partition_by: &[(String, DataType)], + write_id: &str, + file_extension: &str, + base_output_path: &ListingTableUrl, +) -> Path { + let mut file_path = base_output_path.prefix().clone(); + for j in 0..part_key.len() { + file_path = file_path.child(format!("{}={}", partition_by[j].0, part_key[j])); + } + + file_path.child(format!("{}.{}", write_id, file_extension)) +} diff --git a/datafusion/core/src/datasource/file_format/write/mod.rs b/datafusion/core/src/datasource/file_format/write/mod.rs new file mode 100644 index 000000000000..770c7a49c326 --- /dev/null +++ b/datafusion/core/src/datasource/file_format/write/mod.rs @@ -0,0 +1,305 @@ +// 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. + +//! Module containing helper methods/traits related to enabling +//! write support for the various file formats + +use std::io::Error; +use std::mem; +use std::pin::Pin; +use std::sync::Arc; +use std::task::{Context, Poll}; + +use crate::datasource::file_format::file_compression_type::FileCompressionType; + +use crate::datasource::physical_plan::FileMeta; +use crate::error::Result; + +use arrow_array::RecordBatch; + +use datafusion_common::{exec_err, DataFusionError}; + +use async_trait::async_trait; +use bytes::Bytes; + +use futures::future::BoxFuture; +use futures::ready; +use futures::FutureExt; +use object_store::path::Path; +use object_store::{MultipartId, ObjectMeta, ObjectStore}; + +use tokio::io::AsyncWrite; + +pub(crate) mod demux; +pub(crate) mod orchestration; + +/// `AsyncPutWriter` is an object that facilitates asynchronous writing to object stores. +/// It is specifically designed for the `object_store` crate's `put` method and sends +/// whole bytes at once when the buffer is flushed. +pub struct AsyncPutWriter { + /// Object metadata + object_meta: ObjectMeta, + /// A shared reference to the object store + store: Arc, + /// A buffer that stores the bytes to be sent + current_buffer: Vec, + /// Used for async handling in flush method + inner_state: AsyncPutState, +} + +impl AsyncPutWriter { + /// Constructor for the `AsyncPutWriter` object + pub fn new(object_meta: ObjectMeta, store: Arc) -> Self { + Self { + object_meta, + store, + current_buffer: vec![], + // The writer starts out in buffering mode + inner_state: AsyncPutState::Buffer, + } + } + + /// Separate implementation function that unpins the [`AsyncPutWriter`] so + /// that partial borrows work correctly + fn poll_shutdown_inner( + &mut self, + cx: &mut Context<'_>, + ) -> Poll> { + loop { + match &mut self.inner_state { + AsyncPutState::Buffer => { + // Convert the current buffer to bytes and take ownership of it + let bytes = Bytes::from(mem::take(&mut self.current_buffer)); + // Set the inner state to Put variant with the bytes + self.inner_state = AsyncPutState::Put { bytes } + } + AsyncPutState::Put { bytes } => { + // Send the bytes to the object store's put method + return Poll::Ready( + ready!(self + .store + .put(&self.object_meta.location, bytes.clone()) + .poll_unpin(cx)) + .map_err(Error::from), + ); + } + } + } + } +} + +/// An enum that represents the inner state of AsyncPut +enum AsyncPutState { + /// Building Bytes struct in this state + Buffer, + /// Data in the buffer is being sent to the object store + Put { bytes: Bytes }, +} + +impl AsyncWrite for AsyncPutWriter { + // Define the implementation of the AsyncWrite trait for the `AsyncPutWriter` struct + fn poll_write( + mut self: Pin<&mut Self>, + _: &mut Context<'_>, + buf: &[u8], + ) -> Poll> { + // Extend the current buffer with the incoming buffer + self.current_buffer.extend_from_slice(buf); + // Return a ready poll with the length of the incoming buffer + Poll::Ready(Ok(buf.len())) + } + + fn poll_flush( + self: Pin<&mut Self>, + _: &mut Context<'_>, + ) -> Poll> { + // Return a ready poll with an empty result + Poll::Ready(Ok(())) + } + + fn poll_shutdown( + mut self: Pin<&mut Self>, + cx: &mut Context<'_>, + ) -> Poll> { + // Call the poll_shutdown_inner method to handle the actual sending of data to the object store + self.poll_shutdown_inner(cx) + } +} + +/// Stores data needed during abortion of MultiPart writers +pub(crate) struct MultiPart { + /// A shared reference to the object store + store: Arc, + multipart_id: MultipartId, + location: Path, +} + +impl MultiPart { + /// Create a new `MultiPart` + pub fn new( + store: Arc, + multipart_id: MultipartId, + location: Path, + ) -> Self { + Self { + store, + multipart_id, + location, + } + } +} + +pub(crate) enum AbortMode { + Put, + Append, + MultiPart(MultiPart), +} + +/// A wrapper struct with abort method and writer +pub(crate) struct AbortableWrite { + writer: W, + mode: AbortMode, +} + +impl AbortableWrite { + /// Create a new `AbortableWrite` instance with the given writer, and write mode. + pub(crate) fn new(writer: W, mode: AbortMode) -> Self { + Self { writer, mode } + } + + /// handling of abort for different write modes + pub(crate) fn abort_writer(&self) -> Result>> { + match &self.mode { + AbortMode::Put => Ok(async { Ok(()) }.boxed()), + AbortMode::Append => exec_err!("Cannot abort in append mode"), + AbortMode::MultiPart(MultiPart { + store, + multipart_id, + location, + }) => { + let location = location.clone(); + let multipart_id = multipart_id.clone(); + let store = store.clone(); + Ok(Box::pin(async move { + store + .abort_multipart(&location, &multipart_id) + .await + .map_err(DataFusionError::ObjectStore) + })) + } + } + } +} + +impl AsyncWrite for AbortableWrite { + fn poll_write( + self: Pin<&mut Self>, + cx: &mut Context<'_>, + buf: &[u8], + ) -> Poll> { + Pin::new(&mut self.get_mut().writer).poll_write(cx, buf) + } + + fn poll_flush( + self: Pin<&mut Self>, + cx: &mut Context<'_>, + ) -> Poll> { + Pin::new(&mut self.get_mut().writer).poll_flush(cx) + } + + fn poll_shutdown( + self: Pin<&mut Self>, + cx: &mut Context<'_>, + ) -> Poll> { + Pin::new(&mut self.get_mut().writer).poll_shutdown(cx) + } +} + +/// An enum that defines different file writer modes. +#[derive(Debug, Clone, Copy)] +pub enum FileWriterMode { + /// Data is appended to an existing file. + Append, + /// Data is written to a new file. + Put, + /// Data is written to a new file in multiple parts. + PutMultipart, +} +/// A trait that defines the methods required for a RecordBatch serializer. +#[async_trait] +pub trait BatchSerializer: Unpin + Send { + /// Asynchronously serializes a `RecordBatch` and returns the serialized bytes. + async fn serialize(&mut self, batch: RecordBatch) -> Result; + /// Duplicates self to support serializing multiple batches in parallel on multiple cores + fn duplicate(&mut self) -> Result> { + Err(DataFusionError::NotImplemented( + "Parallel serialization is not implemented for this file type".into(), + )) + } +} + +/// Returns an [`AbortableWrite`] which writes to the given object store location +/// with the specified compression +pub(crate) async fn create_writer( + writer_mode: FileWriterMode, + file_compression_type: FileCompressionType, + file_meta: FileMeta, + object_store: Arc, +) -> Result>> { + let object = &file_meta.object_meta; + match writer_mode { + // If the mode is append, call the store's append method and return wrapped in + // a boxed trait object. + FileWriterMode::Append => { + let writer = object_store + .append(&object.location) + .await + .map_err(DataFusionError::ObjectStore)?; + let writer = AbortableWrite::new( + file_compression_type.convert_async_writer(writer)?, + AbortMode::Append, + ); + Ok(writer) + } + // If the mode is put, create a new AsyncPut writer and return it wrapped in + // a boxed trait object + FileWriterMode::Put => { + let writer = Box::new(AsyncPutWriter::new(object.clone(), object_store)); + let writer = AbortableWrite::new( + file_compression_type.convert_async_writer(writer)?, + AbortMode::Put, + ); + Ok(writer) + } + // If the mode is put multipart, call the store's put_multipart method and + // return the writer wrapped in a boxed trait object. + FileWriterMode::PutMultipart => { + let (multipart_id, writer) = object_store + .put_multipart(&object.location) + .await + .map_err(DataFusionError::ObjectStore)?; + Ok(AbortableWrite::new( + file_compression_type.convert_async_writer(writer)?, + AbortMode::MultiPart(MultiPart::new( + object_store, + multipart_id, + object.location.clone(), + )), + )) + } + } +} diff --git a/datafusion/core/src/datasource/file_format/write/orchestration.rs b/datafusion/core/src/datasource/file_format/write/orchestration.rs new file mode 100644 index 000000000000..f84baa9ac225 --- /dev/null +++ b/datafusion/core/src/datasource/file_format/write/orchestration.rs @@ -0,0 +1,415 @@ +// 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. + +//! Module containing helper methods/traits related to +//! orchestrating file serialization, streaming to object store, +//! parallelization, and abort handling + +use std::sync::Arc; + +use crate::datasource::file_format::file_compression_type::FileCompressionType; +use crate::datasource::listing::PartitionedFile; +use crate::datasource::physical_plan::FileSinkConfig; +use crate::error::Result; +use crate::physical_plan::SendableRecordBatchStream; + +use arrow_array::RecordBatch; + +use datafusion_common::DataFusionError; + +use bytes::Bytes; +use datafusion_execution::TaskContext; + +use futures::StreamExt; + +use object_store::{ObjectMeta, ObjectStore}; + +use tokio::io::{AsyncWrite, AsyncWriteExt}; +use tokio::sync::mpsc::{self, Receiver}; +use tokio::task::{JoinHandle, JoinSet}; +use tokio::try_join; + +use super::demux::start_demuxer_task; +use super::{create_writer, AbortableWrite, BatchSerializer, FileWriterMode}; + +type WriterType = AbortableWrite>; +type SerializerType = Box; + +/// Serializes a single data stream in parallel and writes to an ObjectStore +/// concurrently. Data order is preserved. In the event of an error, +/// the ObjectStore writer is returned to the caller in addition to an error, +/// so that the caller may handle aborting failed writes. +pub(crate) async fn serialize_rb_stream_to_object_store( + mut data_rx: Receiver, + mut serializer: Box, + mut writer: AbortableWrite>, + unbounded_input: bool, +) -> std::result::Result<(WriterType, u64), (WriterType, DataFusionError)> { + let (tx, mut rx) = + mpsc::channel::>>(100); + + let serialize_task = tokio::spawn(async move { + while let Some(batch) = data_rx.recv().await { + match serializer.duplicate() { + Ok(mut serializer_clone) => { + let handle = tokio::spawn(async move { + let num_rows = batch.num_rows(); + let bytes = serializer_clone.serialize(batch).await?; + Ok((num_rows, bytes)) + }); + tx.send(handle).await.map_err(|_| { + DataFusionError::Internal( + "Unknown error writing to object store".into(), + ) + })?; + if unbounded_input { + tokio::task::yield_now().await; + } + } + Err(_) => { + return Err(DataFusionError::Internal( + "Unknown error writing to object store".into(), + )) + } + } + } + Ok(()) + }); + + let mut row_count = 0; + while let Some(handle) = rx.recv().await { + match handle.await { + Ok(Ok((cnt, bytes))) => { + match writer.write_all(&bytes).await { + Ok(_) => (), + Err(e) => { + return Err(( + writer, + DataFusionError::Execution(format!( + "Error writing to object store: {e}" + )), + )) + } + }; + row_count += cnt; + } + Ok(Err(e)) => { + // Return the writer along with the error + return Err((writer, e)); + } + Err(e) => { + // Handle task panic or cancellation + return Err(( + writer, + DataFusionError::Execution(format!( + "Serialization task panicked or was cancelled: {e}" + )), + )); + } + } + } + + match serialize_task.await { + Ok(Ok(_)) => (), + Ok(Err(e)) => return Err((writer, e)), + Err(_) => { + return Err(( + writer, + DataFusionError::Internal("Unknown error writing to object store".into()), + )) + } + }; + Ok((writer, row_count as u64)) +} + +type FileWriteBundle = (Receiver, SerializerType, WriterType); +/// Contains the common logic for serializing RecordBatches and +/// writing the resulting bytes to an ObjectStore. +/// Serialization is assumed to be stateless, i.e. +/// each RecordBatch can be serialized without any +/// dependency on the RecordBatches before or after. +pub(crate) async fn stateless_serialize_and_write_files( + mut rx: Receiver, + tx: tokio::sync::oneshot::Sender, + unbounded_input: bool, +) -> Result<()> { + let mut row_count = 0; + // tracks if any writers encountered an error triggering the need to abort + let mut any_errors = false; + // tracks the specific error triggering abort + let mut triggering_error = None; + // tracks if any errors were encountered in the process of aborting writers. + // if true, we may not have a guarentee that all written data was cleaned up. + let mut any_abort_errors = false; + let mut join_set = JoinSet::new(); + while let Some((data_rx, serializer, writer)) = rx.recv().await { + join_set.spawn(async move { + serialize_rb_stream_to_object_store( + data_rx, + serializer, + writer, + unbounded_input, + ) + .await + }); + } + let mut finished_writers = Vec::new(); + while let Some(result) = join_set.join_next().await { + match result { + Ok(res) => match res { + Ok((writer, cnt)) => { + finished_writers.push(writer); + row_count += cnt; + } + Err((writer, e)) => { + finished_writers.push(writer); + any_errors = true; + triggering_error = Some(e); + } + }, + Err(e) => { + // Don't panic, instead try to clean up as many writers as possible. + // If we hit this code, ownership of a writer was not joined back to + // this thread, so we cannot clean it up (hence any_abort_errors is true) + any_errors = true; + any_abort_errors = true; + triggering_error = Some(DataFusionError::Internal(format!( + "Unexpected join error while serializing file {e}" + ))); + } + } + } + + // Finalize or abort writers as appropriate + for mut writer in finished_writers.into_iter() { + match any_errors { + true => { + let abort_result = writer.abort_writer(); + if abort_result.is_err() { + any_abort_errors = true; + } + } + false => { + writer.shutdown() + .await + .map_err(|_| DataFusionError::Internal("Error encountered while finalizing writes! Partial results may have been written to ObjectStore!".into()))?; + } + } + } + + if any_errors { + match any_abort_errors{ + true => return Err(DataFusionError::Internal("Error encountered during writing to ObjectStore and failed to abort all writers. Partial result may have been written.".into())), + false => match triggering_error { + Some(e) => return Err(e), + None => return Err(DataFusionError::Internal("Unknown Error encountered during writing to ObjectStore. All writers succesfully aborted.".into())) + } + } + } + + tx.send(row_count).map_err(|_| { + DataFusionError::Internal( + "Error encountered while sending row count back to file sink!".into(), + ) + })?; + Ok(()) +} + +/// Orchestrates multipart put of a dynamic number of output files from a single input stream +/// for any statelessly serialized file type. That is, any file type for which each [RecordBatch] +/// can be serialized independently of all other [RecordBatch]s. +pub(crate) async fn stateless_multipart_put( + data: SendableRecordBatchStream, + context: &Arc, + file_extension: String, + get_serializer: Box Box + Send>, + config: &FileSinkConfig, + compression: FileCompressionType, +) -> Result { + let object_store = context + .runtime_env() + .object_store(&config.object_store_url)?; + + let single_file_output = config.single_file_output; + let base_output_path = &config.table_paths[0]; + let unbounded_input = config.unbounded_input; + let part_cols = if !config.table_partition_cols.is_empty() { + Some(config.table_partition_cols.clone()) + } else { + None + }; + + let (demux_task, mut file_stream_rx) = start_demuxer_task( + data, + context, + part_cols, + base_output_path.clone(), + file_extension, + single_file_output, + ); + + let rb_buffer_size = &context + .session_config() + .options() + .execution + .max_buffered_batches_per_output_file; + + let (tx_file_bundle, rx_file_bundle) = tokio::sync::mpsc::channel(rb_buffer_size / 2); + let (tx_row_cnt, rx_row_cnt) = tokio::sync::oneshot::channel(); + let write_coordinater_task = tokio::spawn(async move { + stateless_serialize_and_write_files(rx_file_bundle, tx_row_cnt, unbounded_input) + .await + }); + while let Some((output_location, rb_stream)) = file_stream_rx.recv().await { + let serializer = get_serializer(); + let object_meta = ObjectMeta { + location: output_location, + last_modified: chrono::offset::Utc::now(), + size: 0, + e_tag: None, + }; + let writer = create_writer( + FileWriterMode::PutMultipart, + compression, + object_meta.into(), + object_store.clone(), + ) + .await?; + + tx_file_bundle + .send((rb_stream, serializer, writer)) + .await + .map_err(|_| { + DataFusionError::Internal( + "Writer receive file bundle channel closed unexpectedly!".into(), + ) + })?; + } + + // Signal to the write coordinater that no more files are coming + drop(tx_file_bundle); + + match try_join!(write_coordinater_task, demux_task) { + Ok((r1, r2)) => { + r1?; + r2?; + } + Err(e) => { + if e.is_panic() { + std::panic::resume_unwind(e.into_panic()); + } else { + unreachable!(); + } + } + } + + let total_count = rx_row_cnt.await.map_err(|_| { + DataFusionError::Internal( + "Did not receieve row count from write coordinater".into(), + ) + })?; + + Ok(total_count) +} + +/// Orchestrates append_all for any statelessly serialized file type. Appends to all files provided +/// in a round robin fashion. +pub(crate) async fn stateless_append_all( + mut data: SendableRecordBatchStream, + context: &Arc, + object_store: Arc, + file_groups: &Vec, + unbounded_input: bool, + compression: FileCompressionType, + get_serializer: Box Box + Send>, +) -> Result { + let rb_buffer_size = &context + .session_config() + .options() + .execution + .max_buffered_batches_per_output_file; + + let (tx_file_bundle, rx_file_bundle) = tokio::sync::mpsc::channel(file_groups.len()); + let mut send_channels = vec![]; + for file_group in file_groups { + let serializer = get_serializer(file_group.object_meta.size); + + let file = file_group.clone(); + let writer = create_writer( + FileWriterMode::Append, + compression, + file.object_meta.clone().into(), + object_store.clone(), + ) + .await?; + + let (tx, rx) = tokio::sync::mpsc::channel(rb_buffer_size / 2); + send_channels.push(tx); + tx_file_bundle + .send((rx, serializer, writer)) + .await + .map_err(|_| { + DataFusionError::Internal( + "Writer receive file bundle channel closed unexpectedly!".into(), + ) + })?; + } + + let (tx_row_cnt, rx_row_cnt) = tokio::sync::oneshot::channel(); + let write_coordinater_task = tokio::spawn(async move { + stateless_serialize_and_write_files(rx_file_bundle, tx_row_cnt, unbounded_input) + .await + }); + + // Append to file groups in round robin + let mut next_file_idx = 0; + while let Some(rb) = data.next().await.transpose()? { + send_channels[next_file_idx].send(rb).await.map_err(|_| { + DataFusionError::Internal( + "Recordbatch file append stream closed unexpectedly!".into(), + ) + })?; + next_file_idx = (next_file_idx + 1) % send_channels.len(); + if unbounded_input { + tokio::task::yield_now().await; + } + } + // Signal to the write coordinater that no more files are coming + drop(tx_file_bundle); + drop(send_channels); + + let total_count = rx_row_cnt.await.map_err(|_| { + DataFusionError::Internal( + "Did not receieve row count from write coordinater".into(), + ) + })?; + + match try_join!(write_coordinater_task) { + Ok(r1) => { + r1.0?; + } + Err(e) => { + if e.is_panic() { + std::panic::resume_unwind(e.into_panic()); + } else { + unreachable!(); + } + } + } + + Ok(total_count) +} diff --git a/datafusion/core/src/datasource/listing/table.rs b/datafusion/core/src/datasource/listing/table.rs index 05d8ba6c451e..ae6aa317ce85 100644 --- a/datafusion/core/src/datasource/listing/table.rs +++ b/datafusion/core/src/datasource/listing/table.rs @@ -837,12 +837,6 @@ impl TableProvider for ListingTable { ); } - if self.table_paths().len() > 1 { - return plan_err!( - "Writing to a table backed by multiple partitions is not supported yet" - ); - } - let table_path = &self.table_paths()[0]; // Get the object store for the table path. let store = state.runtime_env().object_store(table_path)?; diff --git a/datafusion/sqllogictest/test_files/information_schema.slt b/datafusion/sqllogictest/test_files/information_schema.slt index 8e22ad833f7c..9b7b41847f8a 100644 --- a/datafusion/sqllogictest/test_files/information_schema.slt +++ b/datafusion/sqllogictest/test_files/information_schema.slt @@ -151,7 +151,6 @@ datafusion.execution.batch_size 8192 datafusion.execution.coalesce_batches true datafusion.execution.collect_statistics false datafusion.execution.max_buffered_batches_per_output_file 2 -datafusion.execution.max_parallel_ouput_files 8 datafusion.execution.meta_fetch_concurrency 32 datafusion.execution.parquet.allow_single_file_parallelism false datafusion.execution.parquet.bloom_filter_enabled false @@ -221,7 +220,6 @@ datafusion.execution.batch_size 8192 Default batch size while creating new batch datafusion.execution.coalesce_batches true When set to true, record batches will be examined between each operator and small batches will be coalesced into larger batches. This is helpful when there are highly selective filters or joins that could produce tiny output batches. The target batch size is determined by the configuration setting datafusion.execution.collect_statistics false Should DataFusion collect statistics after listing files datafusion.execution.max_buffered_batches_per_output_file 2 This is the maximum number of RecordBatches buffered for each output file being worked. Higher values can potentially give faster write performance at the cost of higher peak memory consumption -datafusion.execution.max_parallel_ouput_files 8 This is the maximum number of output files being written in parallel. Higher values can potentially give faster write performance at the cost of higher peak memory consumption. datafusion.execution.meta_fetch_concurrency 32 Number of files to read in parallel when inferring schema and statistics datafusion.execution.parquet.allow_single_file_parallelism false Controls whether DataFusion will attempt to speed up writing large parquet files by first writing multiple smaller files and then stitching them together into a single large file. This will result in faster write speeds, but higher memory usage. Also currently unsupported are bloom filters and column indexes when single_file_parallelism is enabled. datafusion.execution.parquet.bloom_filter_enabled false Sets if bloom filter is enabled for any column diff --git a/datafusion/sqllogictest/test_files/insert_to_external.slt b/datafusion/sqllogictest/test_files/insert_to_external.slt index abbfa304be27..b2206e987864 100644 --- a/datafusion/sqllogictest/test_files/insert_to_external.slt +++ b/datafusion/sqllogictest/test_files/insert_to_external.slt @@ -87,6 +87,127 @@ SELECT * from ordered_insert_test; 7 8 7 7 +# test partitioned insert + +statement ok +CREATE EXTERNAL TABLE +partitioned_insert_test(a string, b string, c bigint) +STORED AS csv +LOCATION 'test_files/scratch/insert_to_external/insert_to_partitioned/' +PARTITIONED BY (a, b) +OPTIONS( +create_local_path 'true', +insert_mode 'append_new_files', +); + +#note that partitioned cols are moved to the end so value tuples are (c, a, b) +query ITT +INSERT INTO partitioned_insert_test values (1, 10, 100), (1, 10, 200), (1, 20, 100), (1, 20, 200), (2, 20, 100), (2, 20, 200); +---- +6 + +query ITT +select * from partitioned_insert_test order by a,b,c +---- +1 10 100 +1 10 200 +1 20 100 +2 20 100 +1 20 200 +2 20 200 + +statement ok +CREATE EXTERNAL TABLE +partitioned_insert_test_verify(c bigint) +STORED AS csv +LOCATION 'test_files/scratch/insert_to_external/insert_to_partitioned/a=20/b=100/' +OPTIONS( +insert_mode 'append_new_files', +); + +query I +select * from partitioned_insert_test_verify; +---- +1 +2 + +statement ok +CREATE EXTERNAL TABLE +partitioned_insert_test_json(a string, b string) +STORED AS json +LOCATION 'test_files/scratch/insert_to_external/insert_to_partitioned_json/' +PARTITIONED BY (a) +OPTIONS( +create_local_path 'true', +insert_mode 'append_new_files', +); + +query TT +INSERT INTO partitioned_insert_test_json values (1, 2), (3, 4), (5, 6), (1, 2), (3, 4), (5, 6); +---- +6 + +# Issue open for this error: https://github.com/apache/arrow-datafusion/issues/7816 +query error DataFusion error: Arrow error: Json error: Encountered unmasked nulls in non\-nullable StructArray child: Field \{ name: "a", data_type: Utf8, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: \{\} \} +select * from partitioned_insert_test_json order by a,b + +statement ok +CREATE EXTERNAL TABLE +partitioned_insert_test_verify_json(b string) +STORED AS json +LOCATION 'test_files/scratch/insert_to_external/insert_to_partitioned_json/a=2/' +OPTIONS( +insert_mode 'append_new_files', +); + +query T +select * from partitioned_insert_test_verify_json; +---- +1 +1 + +statement ok +CREATE EXTERNAL TABLE +partitioned_insert_test_pq(a string, b bigint) +STORED AS parquet +LOCATION 'test_files/scratch/insert_to_external/insert_to_partitioned_pq/' +PARTITIONED BY (a) +OPTIONS( +create_local_path 'true', +insert_mode 'append_new_files', +); + +query IT +INSERT INTO partitioned_insert_test_pq values (1, 2), (3, 4), (5, 6), (1, 2), (3, 4), (5, 6); +---- +6 + +query IT +select * from partitioned_insert_test_pq order by a ASC, b ASC +---- +1 2 +1 2 +3 4 +3 4 +5 6 +5 6 + +statement ok +CREATE EXTERNAL TABLE +partitioned_insert_test_verify_pq(b bigint) +STORED AS parquet +LOCATION 'test_files/scratch/insert_to_external/insert_to_partitioned_pq/a=2/' +OPTIONS( +insert_mode 'append_new_files', +); + +query I +select * from partitioned_insert_test_verify_pq; +---- +1 +1 + + statement ok CREATE EXTERNAL TABLE single_file_test(a bigint, b bigint) diff --git a/docs/source/user-guide/configs.md b/docs/source/user-guide/configs.md index a0451eed088a..446931e544fc 100644 --- a/docs/source/user-guide/configs.md +++ b/docs/source/user-guide/configs.md @@ -78,7 +78,6 @@ Environment variables are read during `SessionConfig` initialisation so they mus | datafusion.execution.sort_in_place_threshold_bytes | 1048576 | When sorting, below what size should data be concatenated and sorted in a single RecordBatch rather than sorted in batches and merged. | | datafusion.execution.meta_fetch_concurrency | 32 | Number of files to read in parallel when inferring schema and statistics | | datafusion.execution.soft_max_rows_per_output_file | 50000000 | Target number of rows in output files when writing multiple. This is a soft max, so it can be exceeded slightly. There also will be one file smaller than the limit if the total number of rows written is not roughly divisible by the soft max | -| datafusion.execution.max_parallel_ouput_files | 8 | This is the maximum number of output files being written in parallel. Higher values can potentially give faster write performance at the cost of higher peak memory consumption. | | datafusion.execution.max_buffered_batches_per_output_file | 2 | This is the maximum number of RecordBatches buffered for each output file being worked. Higher values can potentially give faster write performance at the cost of higher peak memory consumption | | datafusion.optimizer.enable_round_robin_repartition | true | When set to true, the physical plan optimizer will try to add round robin repartitioning to increase parallelism to leverage more CPU cores | | datafusion.optimizer.enable_topk_aggregation | true | When set to true, the optimizer will attempt to perform limit operations during aggregations, if possible | From 2e12d14334d675a8c304ae6cab433589ec3b5c95 Mon Sep 17 00:00:00 2001 From: Andrew Lamb Date: Fri, 20 Oct 2023 15:44:22 -0400 Subject: [PATCH 106/572] Minor: fix config typo (#7874) * Minor: fix config typo * Update config docs * update test --- datafusion/common/src/config.rs | 2 +- datafusion/sqllogictest/test_files/information_schema.slt | 2 +- docs/source/user-guide/configs.md | 2 +- 3 files changed, 3 insertions(+), 3 deletions(-) diff --git a/datafusion/common/src/config.rs b/datafusion/common/src/config.rs index 5e1064ada6f7..f40324ae3a5d 100644 --- a/datafusion/common/src/config.rs +++ b/datafusion/common/src/config.rs @@ -345,7 +345,7 @@ config_namespace! { /// Sets "created by" property pub created_by: String, default = concat!("datafusion version ", env!("CARGO_PKG_VERSION")).into() - /// Sets column index trucate length + /// Sets column index truncate length pub column_index_truncate_length: Option, default = None /// Sets best effort maximum number of rows in data page diff --git a/datafusion/sqllogictest/test_files/information_schema.slt b/datafusion/sqllogictest/test_files/information_schema.slt index 9b7b41847f8a..e6a69c213b38 100644 --- a/datafusion/sqllogictest/test_files/information_schema.slt +++ b/datafusion/sqllogictest/test_files/information_schema.slt @@ -225,7 +225,7 @@ datafusion.execution.parquet.allow_single_file_parallelism false Controls whethe datafusion.execution.parquet.bloom_filter_enabled false Sets if bloom filter is enabled for any column datafusion.execution.parquet.bloom_filter_fpp NULL Sets bloom filter false positive probability. If NULL, uses default parquet writer setting datafusion.execution.parquet.bloom_filter_ndv NULL Sets bloom filter number of distinct values. If NULL, uses default parquet writer setting -datafusion.execution.parquet.column_index_truncate_length NULL Sets column index trucate length +datafusion.execution.parquet.column_index_truncate_length NULL Sets column index truncate length datafusion.execution.parquet.compression zstd(3) Sets default parquet compression codec Valid values are: uncompressed, snappy, gzip(level), lzo, brotli(level), lz4, zstd(level), and lz4_raw. These values are not case sensitive. If NULL, uses default parquet writer setting datafusion.execution.parquet.created_by datafusion Sets "created by" property datafusion.execution.parquet.data_page_row_count_limit 18446744073709551615 Sets best effort maximum number of rows in data page diff --git a/docs/source/user-guide/configs.md b/docs/source/user-guide/configs.md index 446931e544fc..f331b2de44b8 100644 --- a/docs/source/user-guide/configs.md +++ b/docs/source/user-guide/configs.md @@ -65,7 +65,7 @@ Environment variables are read during `SessionConfig` initialisation so they mus | datafusion.execution.parquet.max_statistics_size | NULL | Sets max statistics size for any column. If NULL, uses default parquet writer setting | | datafusion.execution.parquet.max_row_group_size | 1048576 | Sets maximum number of rows in a row group | | datafusion.execution.parquet.created_by | datafusion version 32.0.0 | Sets "created by" property | -| datafusion.execution.parquet.column_index_truncate_length | NULL | Sets column index trucate length | +| datafusion.execution.parquet.column_index_truncate_length | NULL | Sets column index truncate length | | datafusion.execution.parquet.data_page_row_count_limit | 18446744073709551615 | Sets best effort maximum number of rows in data page | | datafusion.execution.parquet.encoding | NULL | Sets default encoding for any column Valid values are: plain, plain_dictionary, rle, bit_packed, delta_binary_packed, delta_length_byte_array, delta_byte_array, rle_dictionary, and byte_stream_split. These values are not case sensitive. If NULL, uses default parquet writer setting | | datafusion.execution.parquet.bloom_filter_enabled | false | Sets if bloom filter is enabled for any column | From d6d324444a6695852966c2513c66027e59dbec5e Mon Sep 17 00:00:00 2001 From: Liang-Chi Hsieh Date: Sat, 21 Oct 2023 03:27:29 -0700 Subject: [PATCH 107/572] Add Decimal256 sqllogictests for SUM, MEDIAN and COUNT aggregate expressions (#7889) * Add Decimal256 sqllogictests test for SUM aggregate * Add median and avg sqllogictests tests * Remove duplicate * Add count test --- .../sqllogictest/test_files/decimal.slt | 19 +++++++++++++++++++ 1 file changed, 19 insertions(+) diff --git a/datafusion/sqllogictest/test_files/decimal.slt b/datafusion/sqllogictest/test_files/decimal.slt index 87a846c07727..c220a5fc9a52 100644 --- a/datafusion/sqllogictest/test_files/decimal.slt +++ b/datafusion/sqllogictest/test_files/decimal.slt @@ -701,5 +701,24 @@ select arrow_typeof(max(c1)), max(c1) from decimal256_simple where c4=false; ---- Decimal256(50, 6) 0.00005 +query TR +select arrow_typeof(sum(c1)), sum(c1) from decimal256_simple; +---- +Decimal256(60, 6) 0.00055 + +query TR +select arrow_typeof(median(c1)), median(c1) from decimal256_simple; +---- +Decimal256(50, 6) 0.00004 + +query IR +select count(*),c1 from decimal256_simple group by c1 order by c1; +---- +1 0.00001 +2 0.00002 +3 0.00003 +4 0.00004 +5 0.00005 + statement ok drop table decimal256_simple; From 5bdc9afd872f8523a8c8c2c58072ef482f8edcda Mon Sep 17 00:00:00 2001 From: "Reilly.tang" Date: Sat, 21 Oct 2023 20:05:51 +0800 Subject: [PATCH 108/572] [test] add fuzz test for topk (#7772) Signed-off-by: reilly Co-authored-by: Andrew Lamb --- datafusion/core/tests/fuzz_cases/sort_fuzz.rs | 308 ++++++++++++++---- 1 file changed, 244 insertions(+), 64 deletions(-) diff --git a/datafusion/core/tests/fuzz_cases/sort_fuzz.rs b/datafusion/core/tests/fuzz_cases/sort_fuzz.rs index d74144b0abce..2615abfd3c0d 100644 --- a/datafusion/core/tests/fuzz_cases/sort_fuzz.rs +++ b/datafusion/core/tests/fuzz_cases/sort_fuzz.rs @@ -22,89 +22,100 @@ use arrow::{ compute::SortOptions, record_batch::RecordBatch, }; -use datafusion::execution::runtime_env::{RuntimeConfig, RuntimeEnv}; -use datafusion::physical_plan::expressions::{col, PhysicalSortExpr}; +use arrow_array::{Float64Array, StringArray}; +use datafusion::physical_plan::expressions::PhysicalSortExpr; use datafusion::physical_plan::memory::MemoryExec; use datafusion::physical_plan::sorts::sort::SortExec; use datafusion::physical_plan::{collect, ExecutionPlan}; use datafusion::prelude::{SessionConfig, SessionContext}; +use datafusion::{ + datasource::MemTable, + execution::runtime_env::{RuntimeConfig, RuntimeEnv}, +}; +use datafusion_common::{ + cast::{as_float64_array, as_string_array}, + TableReference, +}; use datafusion_execution::memory_pool::GreedyMemoryPool; -use rand::Rng; +use datafusion_physical_expr::expressions::col; +use rand::{rngs::StdRng, Rng, SeedableRng}; use std::sync::Arc; -use test_utils::{batches_to_vec, partitions_to_sorted_vec}; +use test_utils::{batches_to_vec, partitions_to_sorted_vec, stagger_batch}; +const KB: usize = 1 << 10; #[tokio::test] #[cfg_attr(tarpaulin, ignore)] async fn test_sort_1k_mem() { - SortTest::new() - .with_int32_batches(5) - .with_pool_size(10240) - .with_should_spill(false) - .run() - .await; - - SortTest::new() - .with_int32_batches(20000) - .with_pool_size(10240) - .with_should_spill(true) - .run() - .await; - - SortTest::new() - .with_int32_batches(1000000) - .with_pool_size(10240) - .with_should_spill(true) - .run() - .await; + for (batch_size, should_spill) in [(5, false), (20000, true), (1000000, true)] { + SortTest::new() + .with_int32_batches(batch_size) + .with_pool_size(10 * KB) + .with_should_spill(should_spill) + .run() + .await; + } } #[tokio::test] #[cfg_attr(tarpaulin, ignore)] async fn test_sort_100k_mem() { - SortTest::new() - .with_int32_batches(5) - .with_pool_size(102400) - .with_should_spill(false) - .run() - .await; - - SortTest::new() - .with_int32_batches(20000) - .with_pool_size(102400) - .with_should_spill(false) - .run() - .await; - - SortTest::new() - .with_int32_batches(1000000) - .with_pool_size(102400) - .with_should_spill(true) - .run() - .await; + for (batch_size, should_spill) in [(5, false), (20000, false), (1000000, true)] { + SortTest::new() + .with_int32_batches(batch_size) + .with_pool_size(100 * KB) + .with_should_spill(should_spill) + .run() + .await; + } } #[tokio::test] async fn test_sort_unlimited_mem() { - SortTest::new() - .with_int32_batches(5) - .with_pool_size(usize::MAX) - .with_should_spill(false) - .run() - .await; - - SortTest::new() - .with_int32_batches(20000) - .with_pool_size(usize::MAX) - .with_should_spill(false) - .run() - .await; - - SortTest::new() - .with_int32_batches(1000000) - .with_pool_size(usize::MAX) - .with_should_spill(false) - .run() - .await; + for (batch_size, should_spill) in [(5, false), (20000, false), (1000000, false)] { + SortTest::new() + .with_int32_batches(batch_size) + .with_pool_size(usize::MAX) + .with_should_spill(should_spill) + .run() + .await; + } +} + +#[tokio::test] +async fn test_sort_topk() { + for size in [10, 100, 1000, 10000, 1000000] { + let mut topk_scenario = TopKScenario::new() + .with_limit(10) + .with_table_name("t") + .with_col_name("x"); + + // test topk with i32 + let collected_i32 = SortTest::new() + .with_input(topk_scenario.batches(size, ColType::I32)) + .run_with_limit(&topk_scenario) + .await; + let actual = batches_to_vec(&collected_i32); + let excepted_i32 = topk_scenario.excepted_i32(); + assert_eq!(actual, excepted_i32); + + // test topk with f64 + let collected_f64 = SortTest::new() + .with_input(topk_scenario.batches(size, ColType::F64)) + .run_with_limit(&topk_scenario) + .await; + let actual: Vec> = batches_to_f64_vec(&collected_f64); + let excepted_f64 = topk_scenario.excepted_f64(); + assert_eq!(actual, excepted_f64); + + // test topk with str + let collected_str = SortTest::new() + .with_input(topk_scenario.batches(size, ColType::Str)) + .run_with_limit(&topk_scenario) + .await; + let actual: Vec> = batches_to_str_vec(&collected_str); + let excepted_str = topk_scenario.excepted_str(); + assert_eq!(actual, excepted_str); + } } #[derive(Debug, Default)] @@ -121,6 +132,11 @@ impl SortTest { Default::default() } + fn with_input(mut self, batches: Vec>) -> Self { + self.input = batches.clone(); + self + } + /// Create batches of int32 values of rows fn with_int32_batches(mut self, rows: usize) -> Self { self.input = vec![make_staggered_i32_batches(rows)]; @@ -138,6 +154,44 @@ impl SortTest { self } + async fn run_with_limit<'a>( + &self, + topk_scenario: &TopKScenario<'a>, + ) -> Vec { + let input = self.input.clone(); + let schema = input + .iter() + .flat_map(|p| p.iter()) + .next() + .expect("at least one batch") + .schema(); + + let table = MemTable::try_new(schema, input.clone()).unwrap(); + + let ctx = SessionContext::new(); + + ctx.register_table( + TableReference::Bare { + table: topk_scenario.table_name.into(), + }, + Arc::new(table), + ) + .unwrap(); + + let df = ctx + .table(topk_scenario.table_name) + .await + .unwrap() + .sort(vec![ + datafusion_expr::col(topk_scenario.col_name).sort(true, true) + ]) + .unwrap() + .limit(0, Some(topk_scenario.limit)) + .unwrap(); + + df.collect().await.unwrap() + } + /// Sort the input using SortExec and ensure the results are /// correct according to `Vec::sort` both with and without spilling async fn run(&self) { @@ -208,6 +262,109 @@ impl SortTest { } } +enum ColType { + I32, + F64, + Str, +} + +struct TopKScenario<'a> { + limit: usize, + batches: Vec>, + table_name: &'a str, + col_name: &'a str, +} + +impl<'a> TopKScenario<'a> { + fn new() -> Self { + TopKScenario { + limit: 0, + batches: vec![], + table_name: "", + col_name: "", + } + } + + fn with_limit(mut self, limit: usize) -> Self { + self.limit = limit; + self + } + + fn with_table_name(mut self, table_name: &'a str) -> Self { + self.table_name = table_name; + self + } + + fn with_col_name(mut self, col_name: &'a str) -> Self { + self.col_name = col_name; + self + } + + fn batches(&mut self, len: usize, t: ColType) -> Vec> { + let batches = match t { + ColType::I32 => make_staggered_i32_batches(len), + ColType::F64 => make_staggered_f64_batches(len), + ColType::Str => make_staggered_str_batches(len), + }; + self.batches = vec![batches]; + self.batches.clone() + } + + fn excepted_i32(&self) -> Vec> { + let excepted = partitions_to_sorted_vec(&self.batches); + excepted[0..self.limit].into() + } + + fn excepted_f64(&self) -> Vec> { + let mut excepted: Vec> = self + .batches + .iter() + .flat_map(|batches| batches_to_f64_vec(batches).into_iter()) + .collect(); + excepted.sort_by(|a, b| a.partial_cmp(b).unwrap()); + excepted[0..self.limit].into() + } + + fn excepted_str(&self) -> Vec> { + let mut excepted: Vec> = self + .batches + .iter() + .flat_map(|batches| batches_to_str_vec(batches).into_iter()) + .collect(); + excepted.sort_unstable(); + excepted[0..self.limit].into() + } +} + +impl Default for TopKScenario<'_> { + fn default() -> Self { + Self::new() + } +} + +fn make_staggered_f64_batches(len: usize) -> Vec { + let mut rng = StdRng::seed_from_u64(100); + let remainder = RecordBatch::try_from_iter(vec![( + "x", + Arc::new(Float64Array::from_iter_values( + (0..len).map(|_| rng.gen_range(0.0..1000.7)), + )) as ArrayRef, + )]) + .unwrap(); + stagger_batch(remainder) +} + +fn make_staggered_str_batches(len: usize) -> Vec { + let remainder = RecordBatch::try_from_iter(vec![( + "x", + Arc::new(StringArray::from_iter_values( + (0..len).map(|_| get_random_string(6)), + )) as ArrayRef, + )]) + .unwrap(); + stagger_batch(remainder) +} + /// Return randomly sized record batches in a field named 'x' of type `Int32` /// with randomized i32 content fn make_staggered_i32_batches(len: usize) -> Vec { @@ -232,3 +389,26 @@ fn make_staggered_i32_batches(len: usize) -> Vec { } batches } + +/// Return random ASCII String with len +fn get_random_string(len: usize) -> String { + rand::thread_rng() + .sample_iter(rand::distributions::Alphanumeric) + .take(len) + .map(char::from) + .collect() +} + +fn batches_to_f64_vec(batches: &[RecordBatch]) -> Vec> { + batches + .iter() + .flat_map(|batch| as_float64_array(batch.column(0)).unwrap().iter()) + .collect() +} + +fn batches_to_str_vec(batches: &[RecordBatch]) -> Vec> { + batches + .iter() + .flat_map(|batch| as_string_array(batch.column(0)).unwrap().iter()) + .collect() +} From 9fde5c4282fd9f0e3332fb40998bf1562c17fcda Mon Sep 17 00:00:00 2001 From: Devin D'Angelo Date: Sat, 21 Oct 2023 16:23:56 -0400 Subject: [PATCH 109/572] Allow Setting Minimum Parallelism with RowCount Based Demuxer (#7841) * minimum_parallel_files_added * update docs * Apply suggestions from code review Co-authored-by: Andrew Lamb * generalize tests * update docs --------- Co-authored-by: Andrew Lamb --- datafusion/common/src/config.rs | 6 ++ .../src/datasource/file_format/write/demux.rs | 65 +++++++++++++------ .../core/src/datasource/listing/table.rs | 58 ++++++++++++----- .../test_files/information_schema.slt | 2 + docs/source/user-guide/configs.md | 1 + 5 files changed, 97 insertions(+), 35 deletions(-) diff --git a/datafusion/common/src/config.rs b/datafusion/common/src/config.rs index f40324ae3a5d..6aefa4e05de2 100644 --- a/datafusion/common/src/config.rs +++ b/datafusion/common/src/config.rs @@ -255,6 +255,12 @@ config_namespace! { /// Number of files to read in parallel when inferring schema and statistics pub meta_fetch_concurrency: usize, default = 32 + /// Guarantees a minimum level of output files running in parallel. + /// RecordBatches will be distributed in round robin fashion to each + /// parallel writer. Each writer is closed and a new file opened once + /// soft_max_rows_per_output_file is reached. + pub minimum_parallel_output_files: usize, default = 4 + /// Target number of rows in output files when writing multiple. /// This is a soft max, so it can be exceeded slightly. There also /// will be one file smaller than the limit if the total diff --git a/datafusion/core/src/datasource/file_format/write/demux.rs b/datafusion/core/src/datasource/file_format/write/demux.rs index 2c44c0922c76..67dd1f940676 100644 --- a/datafusion/core/src/datasource/file_format/write/demux.rs +++ b/datafusion/core/src/datasource/file_format/write/demux.rs @@ -122,33 +122,49 @@ async fn row_count_demuxer( single_file_output: bool, ) -> Result<()> { let exec_options = &context.session_config().options().execution; + let max_rows_per_file = exec_options.soft_max_rows_per_output_file; let max_buffered_batches = exec_options.max_buffered_batches_per_output_file; - let mut total_rows_current_file = 0; + let minimum_parallel_files = exec_options.minimum_parallel_output_files; let mut part_idx = 0; let write_id = rand::distributions::Alphanumeric.sample_string(&mut rand::thread_rng(), 16); - let mut tx_file = create_new_file_stream( - &base_output_path, - &write_id, - part_idx, - &file_extension, - single_file_output, - max_buffered_batches, - &mut tx, - )?; - part_idx += 1; + let mut open_file_streams = Vec::with_capacity(minimum_parallel_files); + + let mut next_send_steam = 0; + let mut row_counts = Vec::with_capacity(minimum_parallel_files); + + // Overrides if single_file_output is set + let minimum_parallel_files = if single_file_output { + 1 + } else { + minimum_parallel_files + }; + + let max_rows_per_file = if single_file_output { + usize::MAX + } else { + max_rows_per_file + }; while let Some(rb) = input.next().await.transpose()? { - total_rows_current_file += rb.num_rows(); - tx_file.send(rb).await.map_err(|_| { - DataFusionError::Execution("Error sending RecordBatch to file stream!".into()) - })?; - - if total_rows_current_file >= max_rows_per_file && !single_file_output { - total_rows_current_file = 0; - tx_file = create_new_file_stream( + // ensure we have at least minimum_parallel_files open + if open_file_streams.len() < minimum_parallel_files { + open_file_streams.push(create_new_file_stream( + &base_output_path, + &write_id, + part_idx, + &file_extension, + single_file_output, + max_buffered_batches, + &mut tx, + )?); + row_counts.push(0); + part_idx += 1; + } else if row_counts[next_send_steam] >= max_rows_per_file { + row_counts[next_send_steam] = 0; + open_file_streams[next_send_steam] = create_new_file_stream( &base_output_path, &write_id, part_idx, @@ -159,6 +175,17 @@ async fn row_count_demuxer( )?; part_idx += 1; } + row_counts[next_send_steam] += rb.num_rows(); + open_file_streams[next_send_steam] + .send(rb) + .await + .map_err(|_| { + DataFusionError::Execution( + "Error sending RecordBatch to file stream!".into(), + ) + })?; + + next_send_steam = (next_send_steam + 1) % minimum_parallel_files; } Ok(()) } diff --git a/datafusion/core/src/datasource/listing/table.rs b/datafusion/core/src/datasource/listing/table.rs index ae6aa317ce85..485ab0a902c2 100644 --- a/datafusion/core/src/datasource/listing/table.rs +++ b/datafusion/core/src/datasource/listing/table.rs @@ -1608,15 +1608,16 @@ mod tests { #[tokio::test] async fn test_insert_into_append_new_json_files() -> Result<()> { let mut config_map: HashMap = HashMap::new(); - config_map.insert("datafusion.execution.batch_size".into(), "1".into()); + config_map.insert("datafusion.execution.batch_size".into(), "10".into()); config_map.insert( "datafusion.execution.soft_max_rows_per_output_file".into(), - "1".into(), + "10".into(), ); helper_test_append_new_files_to_table( FileType::JSON, FileCompressionType::UNCOMPRESSED, Some(config_map), + 2, ) .await?; Ok(()) @@ -1636,32 +1637,52 @@ mod tests { #[tokio::test] async fn test_insert_into_append_new_csv_files() -> Result<()> { let mut config_map: HashMap = HashMap::new(); - config_map.insert("datafusion.execution.batch_size".into(), "1".into()); + config_map.insert("datafusion.execution.batch_size".into(), "10".into()); config_map.insert( "datafusion.execution.soft_max_rows_per_output_file".into(), - "1".into(), + "10".into(), ); helper_test_append_new_files_to_table( FileType::CSV, FileCompressionType::UNCOMPRESSED, Some(config_map), + 2, ) .await?; Ok(()) } #[tokio::test] - async fn test_insert_into_append_new_parquet_files_defaults() -> Result<()> { + async fn test_insert_into_append_2_new_parquet_files_defaults() -> Result<()> { let mut config_map: HashMap = HashMap::new(); - config_map.insert("datafusion.execution.batch_size".into(), "1".into()); + config_map.insert("datafusion.execution.batch_size".into(), "10".into()); + config_map.insert( + "datafusion.execution.soft_max_rows_per_output_file".into(), + "10".into(), + ); + helper_test_append_new_files_to_table( + FileType::PARQUET, + FileCompressionType::UNCOMPRESSED, + Some(config_map), + 2, + ) + .await?; + Ok(()) + } + + #[tokio::test] + async fn test_insert_into_append_1_new_parquet_files_defaults() -> Result<()> { + let mut config_map: HashMap = HashMap::new(); + config_map.insert("datafusion.execution.batch_size".into(), "20".into()); config_map.insert( "datafusion.execution.soft_max_rows_per_output_file".into(), - "1".into(), + "20".into(), ); helper_test_append_new_files_to_table( FileType::PARQUET, FileCompressionType::UNCOMPRESSED, Some(config_map), + 1, ) .await?; Ok(()) @@ -1788,10 +1809,10 @@ mod tests { #[tokio::test] async fn test_insert_into_append_new_parquet_files_session_overrides() -> Result<()> { let mut config_map: HashMap = HashMap::new(); - config_map.insert("datafusion.execution.batch_size".into(), "1".into()); + config_map.insert("datafusion.execution.batch_size".into(), "10".into()); config_map.insert( "datafusion.execution.soft_max_rows_per_output_file".into(), - "1".into(), + "10".into(), ); config_map.insert( "datafusion.execution.parquet.compression".into(), @@ -1858,6 +1879,7 @@ mod tests { FileType::PARQUET, FileCompressionType::UNCOMPRESSED, Some(config_map), + 2, ) .await?; Ok(()) @@ -1875,6 +1897,7 @@ mod tests { FileType::PARQUET, FileCompressionType::UNCOMPRESSED, Some(config_map), + 2, ) .await .expect_err("Example should fail!"); @@ -2092,6 +2115,7 @@ mod tests { file_type: FileType, file_compression_type: FileCompressionType, session_config_map: Option>, + expected_n_files_per_insert: usize, ) -> Result<()> { // Create the initial context, schema, and batch. let session_ctx = match session_config_map { @@ -2118,7 +2142,9 @@ mod tests { // Create a new batch of data to insert into the table let batch = RecordBatch::try_new( schema.clone(), - vec![Arc::new(arrow_array::Int32Array::from(vec![1, 2, 3]))], + vec![Arc::new(arrow_array::Int32Array::from(vec![ + 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, + ]))], )?; // Register appropriate table depending on file_type we want to test @@ -2214,7 +2240,7 @@ mod tests { "+-------+", "| count |", "+-------+", - "| 6 |", + "| 20 |", "+-------+", ]; @@ -2231,7 +2257,7 @@ mod tests { "+-------+", "| count |", "+-------+", - "| 6 |", + "| 20 |", "+-------+", ]; @@ -2240,7 +2266,7 @@ mod tests { // Assert that `target_partition_number` many files were added to the table. let num_files = tmp_dir.path().read_dir()?.count(); - assert_eq!(num_files, 3); + assert_eq!(num_files, expected_n_files_per_insert); // Create a physical plan from the insert plan let plan = session_ctx @@ -2255,7 +2281,7 @@ mod tests { "+-------+", "| count |", "+-------+", - "| 6 |", + "| 20 |", "+-------+", ]; @@ -2274,7 +2300,7 @@ mod tests { "+-------+", "| count |", "+-------+", - "| 12 |", + "| 40 |", "+-------+", ]; @@ -2283,7 +2309,7 @@ mod tests { // Assert that another `target_partition_number` many files were added to the table. let num_files = tmp_dir.path().read_dir()?.count(); - assert_eq!(num_files, 6); + assert_eq!(num_files, expected_n_files_per_insert * 2); // Return Ok if the function Ok(()) diff --git a/datafusion/sqllogictest/test_files/information_schema.slt b/datafusion/sqllogictest/test_files/information_schema.slt index e6a69c213b38..4a2b6220fd85 100644 --- a/datafusion/sqllogictest/test_files/information_schema.slt +++ b/datafusion/sqllogictest/test_files/information_schema.slt @@ -152,6 +152,7 @@ datafusion.execution.coalesce_batches true datafusion.execution.collect_statistics false datafusion.execution.max_buffered_batches_per_output_file 2 datafusion.execution.meta_fetch_concurrency 32 +datafusion.execution.minimum_parallel_output_files 4 datafusion.execution.parquet.allow_single_file_parallelism false datafusion.execution.parquet.bloom_filter_enabled false datafusion.execution.parquet.bloom_filter_fpp NULL @@ -221,6 +222,7 @@ datafusion.execution.coalesce_batches true When set to true, record batches will datafusion.execution.collect_statistics false Should DataFusion collect statistics after listing files datafusion.execution.max_buffered_batches_per_output_file 2 This is the maximum number of RecordBatches buffered for each output file being worked. Higher values can potentially give faster write performance at the cost of higher peak memory consumption datafusion.execution.meta_fetch_concurrency 32 Number of files to read in parallel when inferring schema and statistics +datafusion.execution.minimum_parallel_output_files 4 Guarantees a minimum level of output files running in parallel. RecordBatches will be distributed in round robin fashion to each parallel writer. Each writer is closed and a new file opened once soft_max_rows_per_output_file is reached. datafusion.execution.parquet.allow_single_file_parallelism false Controls whether DataFusion will attempt to speed up writing large parquet files by first writing multiple smaller files and then stitching them together into a single large file. This will result in faster write speeds, but higher memory usage. Also currently unsupported are bloom filters and column indexes when single_file_parallelism is enabled. datafusion.execution.parquet.bloom_filter_enabled false Sets if bloom filter is enabled for any column datafusion.execution.parquet.bloom_filter_fpp NULL Sets bloom filter false positive probability. If NULL, uses default parquet writer setting diff --git a/docs/source/user-guide/configs.md b/docs/source/user-guide/configs.md index f331b2de44b8..3476118ca645 100644 --- a/docs/source/user-guide/configs.md +++ b/docs/source/user-guide/configs.md @@ -77,6 +77,7 @@ Environment variables are read during `SessionConfig` initialisation so they mus | datafusion.execution.sort_spill_reservation_bytes | 10485760 | Specifies the reserved memory for each spillable sort operation to facilitate an in-memory merge. When a sort operation spills to disk, the in-memory data must be sorted and merged before being written to a file. This setting reserves a specific amount of memory for that in-memory sort/merge process. Note: This setting is irrelevant if the sort operation cannot spill (i.e., if there's no `DiskManager` configured). | | datafusion.execution.sort_in_place_threshold_bytes | 1048576 | When sorting, below what size should data be concatenated and sorted in a single RecordBatch rather than sorted in batches and merged. | | datafusion.execution.meta_fetch_concurrency | 32 | Number of files to read in parallel when inferring schema and statistics | +| datafusion.execution.minimum_parallel_output_files | 4 | Guarantees a minimum level of output files running in parallel. RecordBatches will be distributed in round robin fashion to each parallel writer. Each writer is closed and a new file opened once soft_max_rows_per_output_file is reached. | | datafusion.execution.soft_max_rows_per_output_file | 50000000 | Target number of rows in output files when writing multiple. This is a soft max, so it can be exceeded slightly. There also will be one file smaller than the limit if the total number of rows written is not roughly divisible by the soft max | | datafusion.execution.max_buffered_batches_per_output_file | 2 | This is the maximum number of RecordBatches buffered for each output file being worked. Higher values can potentially give faster write performance at the cost of higher peak memory consumption | | datafusion.optimizer.enable_round_robin_repartition | true | When set to true, the physical plan optimizer will try to add round robin repartitioning to increase parallelism to leverage more CPU cores | From f10036eb4e34daba72b194cc32a1b55d7b5bc120 Mon Sep 17 00:00:00 2001 From: Kirill Zaborsky Date: Sun, 22 Oct 2023 15:57:01 +0300 Subject: [PATCH 110/572] Drop single quotes to make warnings for parquet options not confusing (#7902) Fixes #7867 --- datafusion/common/src/file_options/parquet_writer.rs | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/datafusion/common/src/file_options/parquet_writer.rs b/datafusion/common/src/file_options/parquet_writer.rs index 93611114cf8e..80fa023587ee 100644 --- a/datafusion/common/src/file_options/parquet_writer.rs +++ b/datafusion/common/src/file_options/parquet_writer.rs @@ -342,7 +342,7 @@ pub(crate) fn parse_version_string(str_setting: &str) -> Result { "2.0" => Ok(WriterVersion::PARQUET_2_0), _ => Err(DataFusionError::Configuration(format!( "Unknown or unsupported parquet writer version {str_setting} \ - valid options are '1.0' and '2.0'" + valid options are 1.0 and 2.0" ))), } } @@ -355,7 +355,7 @@ pub(crate) fn parse_statistics_string(str_setting: &str) -> Result Ok(EnabledStatistics::Page), _ => Err(DataFusionError::Configuration(format!( "Unknown or unsupported parquet statistics setting {str_setting} \ - valid options are 'none', 'page', and 'chunk'" + valid options are none, page, and chunk" ))), } } From dae1efb00ee47a766261160e8087545551929801 Mon Sep 17 00:00:00 2001 From: Andrew Lamb Date: Sun, 22 Oct 2023 12:59:49 -0400 Subject: [PATCH 111/572] Add multi-column topk fuzz tests (#7898) * Add multi-column topk tests * clippy * fix validation * Update docs --- .../core/tests/fuzz_cases/limit_fuzz.rs | 349 ++++++++++++++++++ datafusion/core/tests/fuzz_cases/mod.rs | 2 + datafusion/core/tests/fuzz_cases/sort_fuzz.rs | 221 +---------- test-utils/src/lib.rs | 2 +- 4 files changed, 355 insertions(+), 219 deletions(-) create mode 100644 datafusion/core/tests/fuzz_cases/limit_fuzz.rs diff --git a/datafusion/core/tests/fuzz_cases/limit_fuzz.rs b/datafusion/core/tests/fuzz_cases/limit_fuzz.rs new file mode 100644 index 000000000000..9889ce2ae562 --- /dev/null +++ b/datafusion/core/tests/fuzz_cases/limit_fuzz.rs @@ -0,0 +1,349 @@ +// 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. + +//! Fuzz Test for Sort + Fetch/Limit (TopK!) + +use arrow::compute::concat_batches; +use arrow::util::pretty::pretty_format_batches; +use arrow::{array::Int32Array, record_batch::RecordBatch}; +use arrow_array::{Float64Array, Int64Array, StringArray}; +use arrow_schema::SchemaRef; +use datafusion::datasource::MemTable; +use datafusion::prelude::SessionContext; +use datafusion_common::assert_contains; +use rand::{thread_rng, Rng}; +use std::sync::Arc; +use test_utils::stagger_batch; + +#[tokio::test] +async fn test_sort_topk_i32() { + run_limit_fuzz_test(SortedData::new_i32).await +} + +#[tokio::test] +async fn test_sort_topk_f64() { + run_limit_fuzz_test(SortedData::new_f64).await +} + +#[tokio::test] +async fn test_sort_topk_str() { + run_limit_fuzz_test(SortedData::new_str).await +} + +#[tokio::test] +async fn test_sort_topk_i64str() { + run_limit_fuzz_test(SortedData::new_i64str).await +} + +/// Run TopK fuzz tests the specified input data with different +/// different test functions so they can run in parallel) +async fn run_limit_fuzz_test(make_data: F) +where + F: Fn(usize) -> SortedData, +{ + let mut rng = thread_rng(); + for size in [10, 1_0000, 10_000, 100_000] { + let data = make_data(size); + // test various limits including some random ones + for limit in [1, 3, 7, 17, 10000, rng.gen_range(1..size * 2)] { + // limit can be larger than the number of rows in the input + run_limit_test(limit, &data).await; + } + } +} + +/// The data column(s) to use for the TopK test +/// +/// Each variants stores the input batches and the expected sorted values +/// compute the expected output for a given fetch (limit) value. +#[derive(Debug)] +enum SortedData { + // single Int32 column + I32 { + batches: Vec, + sorted: Vec>, + }, + /// Single Float64 column + F64 { + batches: Vec, + sorted: Vec>, + }, + /// Single sorted String column + Str { + batches: Vec, + sorted: Vec>, + }, + /// (i64, string) columns + I64Str { + batches: Vec, + sorted: Vec<(Option, Option)>, + }, +} + +impl SortedData { + /// Create an i32 column of random values, with the specified number of + /// rows, sorted the default + fn new_i32(size: usize) -> Self { + let mut rng = thread_rng(); + // have some repeats (approximately 1/3 of the values are the same) + let max = size as i32 / 3; + let data: Vec> = (0..size) + .map(|_| { + // no nulls for now + Some(rng.gen_range(0..max)) + }) + .collect(); + + let batches = stagger_batch(int32_batch(data.iter().cloned())); + + let mut sorted = data; + sorted.sort_unstable(); + + Self::I32 { batches, sorted } + } + + /// Create an f64 column of random values, with the specified number of + /// rows, sorted the default + fn new_f64(size: usize) -> Self { + let mut rng = thread_rng(); + let mut data: Vec> = (0..size / 3) + .map(|_| { + // no nulls for now + Some(rng.gen_range(0.0..1.0f64)) + }) + .collect(); + + // have some repeats (approximately 1/3 of the values are the same) + while data.len() < size { + data.push(data[rng.gen_range(0..data.len())]); + } + + let batches = stagger_batch(f64_batch(data.iter().cloned())); + + let mut sorted = data; + sorted.sort_by(|a, b| a.partial_cmp(b).unwrap()); + + Self::F64 { batches, sorted } + } + + /// Create an string column of random values, with the specified number of + /// rows, sorted the default + fn new_str(size: usize) -> Self { + let mut rng = thread_rng(); + let mut data: Vec> = (0..size / 3) + .map(|_| { + // no nulls for now + Some(get_random_string(16)) + }) + .collect(); + + // have some repeats (approximately 1/3 of the values are the same) + while data.len() < size { + data.push(data[rng.gen_range(0..data.len())].clone()); + } + + let batches = stagger_batch(string_batch(data.iter())); + + let mut sorted = data; + sorted.sort_unstable(); + + Self::Str { batches, sorted } + } + + /// Create two columns of random values (int64, string), with the specified number of + /// rows, sorted the default + fn new_i64str(size: usize) -> Self { + let mut rng = thread_rng(); + + // 100 distinct values + let strings: Vec> = (0..100) + .map(|_| { + // no nulls for now + Some(get_random_string(16)) + }) + .collect(); + + // form inputs, with only 10 distinct integer values , to force collision checks + let data = (0..size) + .map(|_| { + ( + Some(rng.gen_range(0..10)), + strings[rng.gen_range(0..strings.len())].clone(), + ) + }) + .collect::>(); + + let batches = stagger_batch(i64string_batch(data.iter())); + + let mut sorted = data; + sorted.sort_unstable(); + + Self::I64Str { batches, sorted } + } + + /// Return top top `limit` values as a RecordBatch + fn topk_values(&self, limit: usize) -> RecordBatch { + match self { + Self::I32 { sorted, .. } => int32_batch(sorted.iter().take(limit).cloned()), + Self::F64 { sorted, .. } => f64_batch(sorted.iter().take(limit).cloned()), + Self::Str { sorted, .. } => string_batch(sorted.iter().take(limit)), + Self::I64Str { sorted, .. } => i64string_batch(sorted.iter().take(limit)), + } + } + + /// Return the input data to sort + fn batches(&self) -> Vec { + match self { + Self::I32 { batches, .. } => batches.clone(), + Self::F64 { batches, .. } => batches.clone(), + Self::Str { batches, .. } => batches.clone(), + Self::I64Str { batches, .. } => batches.clone(), + } + } + + /// Return the schema of the input data + fn schema(&self) -> SchemaRef { + match self { + Self::I32 { batches, .. } => batches[0].schema(), + Self::F64 { batches, .. } => batches[0].schema(), + Self::Str { batches, .. } => batches[0].schema(), + Self::I64Str { batches, .. } => batches[0].schema(), + } + } + + /// Return the sort expression to use for this data, depending on the type + fn sort_expr(&self) -> Vec { + match self { + Self::I32 { .. } | Self::F64 { .. } | Self::Str { .. } => { + vec![datafusion_expr::col("x").sort(true, true)] + } + Self::I64Str { .. } => { + vec![ + datafusion_expr::col("x").sort(true, true), + datafusion_expr::col("y").sort(true, true), + ] + } + } + } +} + +/// Create a record batch with a single column of type `Int32` named "x" +fn int32_batch(values: impl IntoIterator>) -> RecordBatch { + RecordBatch::try_from_iter(vec![( + "x", + Arc::new(Int32Array::from_iter(values.into_iter())) as _, + )]) + .unwrap() +} + +/// Create a record batch with a single column of type `Float64` named "x" +fn f64_batch(values: impl IntoIterator>) -> RecordBatch { + RecordBatch::try_from_iter(vec![( + "x", + Arc::new(Float64Array::from_iter(values.into_iter())) as _, + )]) + .unwrap() +} + +/// Create a record batch with a single column of type `StringArray` named "x" +fn string_batch<'a>(values: impl IntoIterator>) -> RecordBatch { + RecordBatch::try_from_iter(vec![( + "x", + Arc::new(StringArray::from_iter(values.into_iter())) as _, + )]) + .unwrap() +} + +/// Create a record batch with i64 column "x" and utf8 column "y" +fn i64string_batch<'a>( + values: impl IntoIterator, Option)> + Clone, +) -> RecordBatch { + let ints = values.clone().into_iter().map(|(i, _)| *i); + let strings = values.into_iter().map(|(_, s)| s); + RecordBatch::try_from_iter(vec![ + ("x", Arc::new(Int64Array::from_iter(ints)) as _), + ("y", Arc::new(StringArray::from_iter(strings)) as _), + ]) + .unwrap() +} + +/// Run the TopK test, sorting the input batches with the specified ftch +/// (limit) and compares the results to the expected values. +async fn run_limit_test(fetch: usize, data: &SortedData) { + let input = data.batches(); + let schema = data.schema(); + + let table = MemTable::try_new(schema, vec![input]).unwrap(); + + let ctx = SessionContext::new(); + let df = ctx + .read_table(Arc::new(table)) + .unwrap() + .sort(data.sort_expr()) + .unwrap() + .limit(0, Some(fetch)) + .unwrap(); + + // Verify the plan contains a TopK node + { + let explain = df + .clone() + .explain(false, false) + .unwrap() + .collect() + .await + .unwrap(); + let plan_text = pretty_format_batches(&explain).unwrap().to_string(); + let expected = format!("TopK(fetch={fetch})"); + assert_contains!(plan_text, expected); + } + + let results = df.collect().await.unwrap(); + let expected = data.topk_values(fetch); + + // Verify that all output batches conform to the specified batch size + let max_batch_size = ctx.copied_config().batch_size(); + for batch in &results { + assert!(batch.num_rows() <= max_batch_size); + } + + let results = concat_batches(&results[0].schema(), &results).unwrap(); + + let results = [results]; + let expected = [expected]; + + assert_eq!( + &expected, + &results, + "TopK mismatch fetch {fetch} \n\ + expected rows {}, actual rows {}.\ + \n\nExpected:\n{}\n\nActual:\n{}", + expected[0].num_rows(), + results[0].num_rows(), + pretty_format_batches(&expected).unwrap(), + pretty_format_batches(&results).unwrap(), + ); +} + +/// Return random ASCII String with len +fn get_random_string(len: usize) -> String { + rand::thread_rng() + .sample_iter(rand::distributions::Alphanumeric) + .take(len) + .map(char::from) + .collect() +} diff --git a/datafusion/core/tests/fuzz_cases/mod.rs b/datafusion/core/tests/fuzz_cases/mod.rs index 140cf7e5c75b..83ec928ae229 100644 --- a/datafusion/core/tests/fuzz_cases/mod.rs +++ b/datafusion/core/tests/fuzz_cases/mod.rs @@ -19,5 +19,7 @@ mod aggregate_fuzz; mod join_fuzz; mod merge_fuzz; mod sort_fuzz; + +mod limit_fuzz; mod sort_preserving_repartition_fuzz; mod window_fuzz; diff --git a/datafusion/core/tests/fuzz_cases/sort_fuzz.rs b/datafusion/core/tests/fuzz_cases/sort_fuzz.rs index 2615abfd3c0d..f4b4f16aa160 100644 --- a/datafusion/core/tests/fuzz_cases/sort_fuzz.rs +++ b/datafusion/core/tests/fuzz_cases/sort_fuzz.rs @@ -22,25 +22,17 @@ use arrow::{ compute::SortOptions, record_batch::RecordBatch, }; -use arrow_array::{Float64Array, StringArray}; +use datafusion::execution::runtime_env::{RuntimeConfig, RuntimeEnv}; use datafusion::physical_plan::expressions::PhysicalSortExpr; use datafusion::physical_plan::memory::MemoryExec; use datafusion::physical_plan::sorts::sort::SortExec; use datafusion::physical_plan::{collect, ExecutionPlan}; use datafusion::prelude::{SessionConfig, SessionContext}; -use datafusion::{ - datasource::MemTable, - execution::runtime_env::{RuntimeConfig, RuntimeEnv}, -}; -use datafusion_common::{ - cast::{as_float64_array, as_string_array}, - TableReference, -}; use datafusion_execution::memory_pool::GreedyMemoryPool; use datafusion_physical_expr::expressions::col; -use rand::{rngs::StdRng, Rng, SeedableRng}; +use rand::Rng; use std::sync::Arc; -use test_utils::{batches_to_vec, partitions_to_sorted_vec, stagger_batch}; +use test_utils::{batches_to_vec, partitions_to_sorted_vec}; const KB: usize = 1 << 10; #[tokio::test] @@ -80,44 +72,6 @@ async fn test_sort_unlimited_mem() { .await; } } - -#[tokio::test] -async fn test_sort_topk() { - for size in [10, 100, 1000, 10000, 1000000] { - let mut topk_scenario = TopKScenario::new() - .with_limit(10) - .with_table_name("t") - .with_col_name("x"); - - // test topk with i32 - let collected_i32 = SortTest::new() - .with_input(topk_scenario.batches(size, ColType::I32)) - .run_with_limit(&topk_scenario) - .await; - let actual = batches_to_vec(&collected_i32); - let excepted_i32 = topk_scenario.excepted_i32(); - assert_eq!(actual, excepted_i32); - - // test topk with f64 - let collected_f64 = SortTest::new() - .with_input(topk_scenario.batches(size, ColType::F64)) - .run_with_limit(&topk_scenario) - .await; - let actual: Vec> = batches_to_f64_vec(&collected_f64); - let excepted_f64 = topk_scenario.excepted_f64(); - assert_eq!(actual, excepted_f64); - - // test topk with str - let collected_str = SortTest::new() - .with_input(topk_scenario.batches(size, ColType::Str)) - .run_with_limit(&topk_scenario) - .await; - let actual: Vec> = batches_to_str_vec(&collected_str); - let excepted_str = topk_scenario.excepted_str(); - assert_eq!(actual, excepted_str); - } -} - #[derive(Debug, Default)] struct SortTest { input: Vec>, @@ -132,11 +86,6 @@ impl SortTest { Default::default() } - fn with_input(mut self, batches: Vec>) -> Self { - self.input = batches.clone(); - self - } - /// Create batches of int32 values of rows fn with_int32_batches(mut self, rows: usize) -> Self { self.input = vec![make_staggered_i32_batches(rows)]; @@ -154,44 +103,6 @@ impl SortTest { self } - async fn run_with_limit<'a>( - &self, - topk_scenario: &TopKScenario<'a>, - ) -> Vec { - let input = self.input.clone(); - let schema = input - .iter() - .flat_map(|p| p.iter()) - .next() - .expect("at least one batch") - .schema(); - - let table = MemTable::try_new(schema, input.clone()).unwrap(); - - let ctx = SessionContext::new(); - - ctx.register_table( - TableReference::Bare { - table: topk_scenario.table_name.into(), - }, - Arc::new(table), - ) - .unwrap(); - - let df = ctx - .table(topk_scenario.table_name) - .await - .unwrap() - .sort(vec![ - datafusion_expr::col(topk_scenario.col_name).sort(true, true) - ]) - .unwrap() - .limit(0, Some(topk_scenario.limit)) - .unwrap(); - - df.collect().await.unwrap() - } - /// Sort the input using SortExec and ensure the results are /// correct according to `Vec::sort` both with and without spilling async fn run(&self) { @@ -262,109 +173,6 @@ impl SortTest { } } -enum ColType { - I32, - F64, - Str, -} - -struct TopKScenario<'a> { - limit: usize, - batches: Vec>, - table_name: &'a str, - col_name: &'a str, -} - -impl<'a> TopKScenario<'a> { - fn new() -> Self { - TopKScenario { - limit: 0, - batches: vec![], - table_name: "", - col_name: "", - } - } - - fn with_limit(mut self, limit: usize) -> Self { - self.limit = limit; - self - } - - fn with_table_name(mut self, table_name: &'a str) -> Self { - self.table_name = table_name; - self - } - - fn with_col_name(mut self, col_name: &'a str) -> Self { - self.col_name = col_name; - self - } - - fn batches(&mut self, len: usize, t: ColType) -> Vec> { - let batches = match t { - ColType::I32 => make_staggered_i32_batches(len), - ColType::F64 => make_staggered_f64_batches(len), - ColType::Str => make_staggered_str_batches(len), - }; - self.batches = vec![batches]; - self.batches.clone() - } - - fn excepted_i32(&self) -> Vec> { - let excepted = partitions_to_sorted_vec(&self.batches); - excepted[0..self.limit].into() - } - - fn excepted_f64(&self) -> Vec> { - let mut excepted: Vec> = self - .batches - .iter() - .flat_map(|batches| batches_to_f64_vec(batches).into_iter()) - .collect(); - excepted.sort_by(|a, b| a.partial_cmp(b).unwrap()); - excepted[0..self.limit].into() - } - - fn excepted_str(&self) -> Vec> { - let mut excepted: Vec> = self - .batches - .iter() - .flat_map(|batches| batches_to_str_vec(batches).into_iter()) - .collect(); - excepted.sort_unstable(); - excepted[0..self.limit].into() - } -} - -impl Default for TopKScenario<'_> { - fn default() -> Self { - Self::new() - } -} - -fn make_staggered_f64_batches(len: usize) -> Vec { - let mut rng = StdRng::seed_from_u64(100); - let remainder = RecordBatch::try_from_iter(vec![( - "x", - Arc::new(Float64Array::from_iter_values( - (0..len).map(|_| rng.gen_range(0.0..1000.7)), - )) as ArrayRef, - )]) - .unwrap(); - stagger_batch(remainder) -} - -fn make_staggered_str_batches(len: usize) -> Vec { - let remainder = RecordBatch::try_from_iter(vec![( - "x", - Arc::new(StringArray::from_iter_values( - (0..len).map(|_| get_random_string(6)), - )) as ArrayRef, - )]) - .unwrap(); - stagger_batch(remainder) -} - /// Return randomly sized record batches in a field named 'x' of type `Int32` /// with randomized i32 content fn make_staggered_i32_batches(len: usize) -> Vec { @@ -389,26 +197,3 @@ fn make_staggered_i32_batches(len: usize) -> Vec { } batches } - -/// Return random ASCII String with len -fn get_random_string(len: usize) -> String { - rand::thread_rng() - .sample_iter(rand::distributions::Alphanumeric) - .take(len) - .map(char::from) - .collect() -} - -fn batches_to_f64_vec(batches: &[RecordBatch]) -> Vec> { - batches - .iter() - .flat_map(|batch| as_float64_array(batch.column(0)).unwrap().iter()) - .collect() -} - -fn batches_to_str_vec(batches: &[RecordBatch]) -> Vec> { - batches - .iter() - .flat_map(|batch| as_string_array(batch.column(0)).unwrap().iter()) - .collect() -} diff --git a/test-utils/src/lib.rs b/test-utils/src/lib.rs index e3c96d16eeb9..0c3668d2f8c0 100644 --- a/test-utils/src/lib.rs +++ b/test-utils/src/lib.rs @@ -38,7 +38,7 @@ pub fn batches_to_vec(batches: &[RecordBatch]) -> Vec> { .collect() } -/// extract values from batches and sort them +/// extract i32 values from batches and sort them pub fn partitions_to_sorted_vec(partitions: &[Vec]) -> Vec> { let mut values: Vec<_> = partitions .iter() From ca5dc8c066bb6da34a1f8522b6127138358a3159 Mon Sep 17 00:00:00 2001 From: Nga Tran Date: Sun, 22 Oct 2023 13:00:49 -0400 Subject: [PATCH 112/572] Change `FileScanConfig.table_partition_cols` from `(String, DataType)` to `Field`s (#7890) * feat: make data type of FileScanConfig.table_partition_cols a vector of Fields * fix: avro test * chore: Apply suggestions from code review Co-authored-by: Andrew Lamb * chore: address review comments * chore: remove uncessary to_owned --------- Co-authored-by: Andrew Lamb --- .../core/src/datasource/listing/table.rs | 10 +--- .../core/src/datasource/physical_plan/avro.rs | 2 +- .../core/src/datasource/physical_plan/csv.rs | 2 +- .../physical_plan/file_scan_config.rs | 54 +++++++++++++++---- .../datasource/physical_plan/file_stream.rs | 2 +- .../src/datasource/physical_plan/parquet.rs | 9 ++-- .../proto/src/physical_plan/from_proto.rs | 10 +--- .../proto/src/physical_plan/to_proto.rs | 2 +- 8 files changed, 57 insertions(+), 34 deletions(-) diff --git a/datafusion/core/src/datasource/listing/table.rs b/datafusion/core/src/datasource/listing/table.rs index 485ab0a902c2..bd878932d80f 100644 --- a/datafusion/core/src/datasource/listing/table.rs +++ b/datafusion/core/src/datasource/listing/table.rs @@ -746,15 +746,7 @@ impl TableProvider for ListingTable { .options .table_partition_cols .iter() - .map(|col| { - Ok(( - col.0.to_owned(), - self.table_schema - .field_with_name(&col.0)? - .data_type() - .clone(), - )) - }) + .map(|col| Ok(self.table_schema.field_with_name(&col.0)?.clone())) .collect::>>()?; let filters = if let Some(expr) = conjunction(filters.to_vec()) { diff --git a/datafusion/core/src/datasource/physical_plan/avro.rs b/datafusion/core/src/datasource/physical_plan/avro.rs index f08bc9b8df20..237772eb8360 100644 --- a/datafusion/core/src/datasource/physical_plan/avro.rs +++ b/datafusion/core/src/datasource/physical_plan/avro.rs @@ -420,7 +420,7 @@ mod tests { statistics: Statistics::new_unknown(&file_schema), file_schema, limit: None, - table_partition_cols: vec![("date".to_owned(), DataType::Utf8)], + table_partition_cols: vec![Field::new("date", DataType::Utf8, false)], output_ordering: vec![], infinite_source: false, }); diff --git a/datafusion/core/src/datasource/physical_plan/csv.rs b/datafusion/core/src/datasource/physical_plan/csv.rs index f3b2fa9de7a9..e60a249b0b4a 100644 --- a/datafusion/core/src/datasource/physical_plan/csv.rs +++ b/datafusion/core/src/datasource/physical_plan/csv.rs @@ -871,7 +871,7 @@ mod tests { let mut config = partitioned_csv_config(file_schema, file_groups)?; // Add partition columns - config.table_partition_cols = vec![("date".to_owned(), DataType::Utf8)]; + config.table_partition_cols = vec![Field::new("date", DataType::Utf8, false)]; config.file_groups[0][0].partition_values = vec![ScalarValue::Utf8(Some("2021-10-26".to_owned()))]; diff --git a/datafusion/core/src/datasource/physical_plan/file_scan_config.rs b/datafusion/core/src/datasource/physical_plan/file_scan_config.rs index c1a19b745b8d..d8a9697b2bf7 100644 --- a/datafusion/core/src/datasource/physical_plan/file_scan_config.rs +++ b/datafusion/core/src/datasource/physical_plan/file_scan_config.rs @@ -101,7 +101,7 @@ pub struct FileScanConfig { /// all records after filtering are returned. pub limit: Option, /// The partitioning columns - pub table_partition_cols: Vec<(String, DataType)>, + pub table_partition_cols: Vec, /// All equivalent lexicographical orderings that describe the schema. pub output_ordering: Vec, /// Indicates whether this plan may produce an infinite stream of records. @@ -135,8 +135,7 @@ impl FileScanConfig { table_cols_stats.push(self.statistics.column_statistics[idx].clone()) } else { let partition_idx = idx - self.file_schema.fields().len(); - let (name, dtype) = &self.table_partition_cols[partition_idx]; - table_fields.push(Field::new(name, dtype.to_owned(), false)); + table_fields.push(self.table_partition_cols[partition_idx].to_owned()); // TODO provide accurate stat for partition column (#1186) table_cols_stats.push(ColumnStatistics::new_unknown()) } @@ -501,10 +500,10 @@ mod tests { Arc::clone(&file_schema), None, Statistics::new_unknown(&file_schema), - vec![( + to_partition_cols(vec![( "date".to_owned(), wrap_partition_type_in_dict(DataType::Utf8), - )], + )]), ); let (proj_schema, proj_statistics, _) = conf.project(); @@ -527,6 +526,35 @@ mod tests { assert_eq!(col_indices, None); } + #[test] + fn physical_plan_config_no_projection_tab_cols_as_field() { + let file_schema = aggr_test_schema(); + + // make a table_partition_col as a field + let table_partition_col = + Field::new("date", wrap_partition_type_in_dict(DataType::Utf8), true) + .with_metadata(HashMap::from_iter(vec![( + "key_whatever".to_owned(), + "value_whatever".to_owned(), + )])); + + let conf = config_for_projection( + Arc::clone(&file_schema), + None, + Statistics::new_unknown(&file_schema), + vec![table_partition_col.clone()], + ); + + // verify the proj_schema inlcudes the last column and exactly the same the field it is defined + let (proj_schema, _proj_statistics, _) = conf.project(); + assert_eq!(proj_schema.fields().len(), file_schema.fields().len() + 1); + assert_eq!( + *proj_schema.field(file_schema.fields().len()), + table_partition_col, + "partition columns are the last columns and ust have all values defined in created field" + ); + } + #[test] fn physical_plan_config_with_projection() { let file_schema = aggr_test_schema(); @@ -545,10 +573,10 @@ mod tests { .collect(), total_byte_size: Precision::Absent, }, - vec![( + to_partition_cols(vec![( "date".to_owned(), wrap_partition_type_in_dict(DataType::Utf8), - )], + )]), ); let (proj_schema, proj_statistics, _) = conf.project(); @@ -602,7 +630,7 @@ mod tests { file_batch.schema().fields().len() + 2, ]), Statistics::new_unknown(&file_batch.schema()), - partition_cols.clone(), + to_partition_cols(partition_cols.clone()), ); let (proj_schema, ..) = conf.project(); // created a projector for that projected schema @@ -747,7 +775,7 @@ mod tests { file_schema: SchemaRef, projection: Option>, statistics: Statistics, - table_partition_cols: Vec<(String, DataType)>, + table_partition_cols: Vec, ) -> FileScanConfig { FileScanConfig { file_schema, @@ -762,6 +790,14 @@ mod tests { } } + /// Convert partition columns from Vec to Vec + fn to_partition_cols(table_partition_cols: Vec<(String, DataType)>) -> Vec { + table_partition_cols + .iter() + .map(|(name, dtype)| Field::new(name, dtype.clone(), false)) + .collect::>() + } + /// returns record batch with 3 columns of i32 in memory pub fn build_table_i32( a: (&str, &Vec), diff --git a/datafusion/core/src/datasource/physical_plan/file_stream.rs b/datafusion/core/src/datasource/physical_plan/file_stream.rs index 22ff3f42ebda..a715f6e8e3cd 100644 --- a/datafusion/core/src/datasource/physical_plan/file_stream.rs +++ b/datafusion/core/src/datasource/physical_plan/file_stream.rs @@ -259,7 +259,7 @@ impl FileStream { &config .table_partition_cols .iter() - .map(|x| x.0.clone()) + .map(|x| x.name().clone()) .collect::>(), ); diff --git a/datafusion/core/src/datasource/physical_plan/parquet.rs b/datafusion/core/src/datasource/physical_plan/parquet.rs index e59686453f0e..6cab27b0846c 100644 --- a/datafusion/core/src/datasource/physical_plan/parquet.rs +++ b/datafusion/core/src/datasource/physical_plan/parquet.rs @@ -1624,14 +1624,15 @@ mod tests { projection: Some(vec![0, 1, 2, 12, 13]), limit: None, table_partition_cols: vec![ - ("year".to_owned(), DataType::Utf8), - ("month".to_owned(), DataType::UInt8), - ( - "day".to_owned(), + Field::new("year", DataType::Utf8, false), + Field::new("month", DataType::UInt8, false), + Field::new( + "day", DataType::Dictionary( Box::new(DataType::UInt16), Box::new(DataType::Utf8), ), + false, ), ], output_ordering: vec![], diff --git a/datafusion/proto/src/physical_plan/from_proto.rs b/datafusion/proto/src/physical_plan/from_proto.rs index ff02b8052136..cdc772f71d80 100644 --- a/datafusion/proto/src/physical_plan/from_proto.rs +++ b/datafusion/proto/src/physical_plan/from_proto.rs @@ -22,7 +22,6 @@ use std::ops::Deref; use std::sync::Arc; use arrow::compute::SortOptions; -use arrow::datatypes::DataType; use datafusion::arrow::datatypes::Schema; use datafusion::datasource::listing::{FileRange, PartitionedFile}; use datafusion::datasource::object_store::ObjectStoreUrl; @@ -489,13 +488,8 @@ pub fn parse_protobuf_file_scan_config( let table_partition_cols = proto .table_partition_cols .iter() - .map(|col| { - Ok(( - col.to_owned(), - schema.field_with_name(col)?.data_type().clone(), - )) - }) - .collect::>>()?; + .map(|col| Ok(schema.field_with_name(col)?.clone())) + .collect::>>()?; let mut output_ordering = vec![]; for node_collection in &proto.output_ordering { diff --git a/datafusion/proto/src/physical_plan/to_proto.rs b/datafusion/proto/src/physical_plan/to_proto.rs index db97c393255a..466b99b68472 100644 --- a/datafusion/proto/src/physical_plan/to_proto.rs +++ b/datafusion/proto/src/physical_plan/to_proto.rs @@ -729,7 +729,7 @@ impl TryFrom<&FileScanConfig> for protobuf::FileScanExecConf { table_partition_cols: conf .table_partition_cols .iter() - .map(|x| x.0.clone()) + .map(|x| x.name().clone()) .collect::>(), object_store_url: conf.object_store_url.to_string(), output_ordering: output_orderings From f72ee5ee506423cccee8b5e57d79ecb84bbf1163 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Dani=C3=ABl=20Heres?= Date: Mon, 23 Oct 2023 14:07:37 +0200 Subject: [PATCH 113/572] Maintain time_zone in new_list (#7899) --- datafusion/common/src/scalar.rs | 38 +++++++++++++++++++++++++++------ 1 file changed, 31 insertions(+), 7 deletions(-) diff --git a/datafusion/common/src/scalar.rs b/datafusion/common/src/scalar.rs index 2c3dd4c5ca55..be24e2b933b5 100644 --- a/datafusion/common/src/scalar.rs +++ b/datafusion/common/src/scalar.rs @@ -620,26 +620,30 @@ macro_rules! build_timestamp_list { TimestampSecondBuilder, TimestampSecond, values, - $SIZE + $SIZE, + $TIME_ZONE ) } TimeUnit::Millisecond => build_values_list_tz!( TimestampMillisecondBuilder, TimestampMillisecond, values, - $SIZE + $SIZE, + $TIME_ZONE ), TimeUnit::Microsecond => build_values_list_tz!( TimestampMicrosecondBuilder, TimestampMicrosecond, values, - $SIZE + $SIZE, + $TIME_ZONE ), TimeUnit::Nanosecond => build_values_list_tz!( TimestampNanosecondBuilder, TimestampNanosecond, values, - $SIZE + $SIZE, + $TIME_ZONE ), }, } @@ -683,9 +687,10 @@ macro_rules! build_values_list { } macro_rules! build_values_list_tz { - ($VALUE_BUILDER_TY:ident, $SCALAR_TY:ident, $VALUES:expr, $SIZE:expr) => {{ - let mut builder = - ListBuilder::new($VALUE_BUILDER_TY::with_capacity($VALUES.len())); + ($VALUE_BUILDER_TY:ident, $SCALAR_TY:ident, $VALUES:expr, $SIZE:expr, $TIME_ZONE:expr) => {{ + let mut builder = ListBuilder::new( + $VALUE_BUILDER_TY::with_capacity($VALUES.len()).with_timezone_opt($TIME_ZONE), + ); for _ in 0..$SIZE { for scalar_value in $VALUES { @@ -5185,6 +5190,25 @@ mod tests { assert_eq!(1, arr.len()); } + #[test] + fn test_newlist_timestamp_zone() { + let s: &'static str = "UTC"; + let values = vec![ScalarValue::TimestampMillisecond(Some(1), Some(s.into()))]; + let arr = ScalarValue::new_list( + &values, + &DataType::Timestamp(TimeUnit::Millisecond, Some(s.into())), + ); + assert_eq!(1, arr.len()); + assert_eq!( + arr.data_type(), + &DataType::List(Arc::new(Field::new( + "item", + DataType::Timestamp(TimeUnit::Millisecond, Some(s.into())), + true + ))) + ); + } + fn get_random_timestamps(sample_size: u64) -> Vec { let vector_size = sample_size; let mut timestamp = vec![]; From e57dd22720249dc4e4e0c9666596df27ea8c7968 Mon Sep 17 00:00:00 2001 From: Mustafa Akur <106137913+mustafasrepo@users.noreply.github.com> Date: Mon, 23 Oct 2023 16:15:38 +0300 Subject: [PATCH 114/572] [MINOR]: Move joinside struct to common (#7908) * Move joinside struct to common * Update documentation --- datafusion/common/src/join_type.rs | 31 ++++++++++++++++++- datafusion/common/src/lib.rs | 2 +- .../src/physical_optimizer/sort_pushdown.rs | 6 ++-- .../physical-plan/src/joins/hash_join.rs | 15 +++------ .../src/joins/hash_join_utils.rs | 6 ++-- .../src/joins/nested_loop_join.rs | 5 ++- .../src/joins/sort_merge_join.rs | 4 +-- .../src/joins/symmetric_hash_join.rs | 6 ++-- datafusion/physical-plan/src/joins/utils.rs | 31 +------------------ .../proto/src/physical_plan/from_proto.rs | 3 +- .../proto/src/physical_plan/to_proto.rs | 3 +- 11 files changed, 54 insertions(+), 58 deletions(-) diff --git a/datafusion/common/src/join_type.rs b/datafusion/common/src/join_type.rs index 8d4657f1dc56..0a00a57ba45f 100644 --- a/datafusion/common/src/join_type.rs +++ b/datafusion/common/src/join_type.rs @@ -15,7 +15,7 @@ // specific language governing permissions and limitations // under the License. -//! [`JoinType`] and [`JoinConstraint`] +//! Defines the [`JoinType`], [`JoinConstraint`] and [`JoinSide`] types. use std::{ fmt::{self, Display, Formatter}, @@ -95,3 +95,32 @@ pub enum JoinConstraint { /// Join USING Using, } + +impl Display for JoinSide { + fn fmt(&self, f: &mut Formatter<'_>) -> std::fmt::Result { + match self { + JoinSide::Left => write!(f, "left"), + JoinSide::Right => write!(f, "right"), + } + } +} + +/// Join side. +/// Stores the referred table side during calculations +#[derive(Debug, Clone, Copy, PartialEq)] +pub enum JoinSide { + /// Left side of the join + Left, + /// Right side of the join + Right, +} + +impl JoinSide { + /// Inverse the join side + pub fn negate(&self) -> Self { + match self { + JoinSide::Left => JoinSide::Right, + JoinSide::Right => JoinSide::Left, + } + } +} diff --git a/datafusion/common/src/lib.rs b/datafusion/common/src/lib.rs index a939cf73dc9c..53c3cfddff8d 100644 --- a/datafusion/common/src/lib.rs +++ b/datafusion/common/src/lib.rs @@ -57,7 +57,7 @@ pub use functional_dependencies::{ aggregate_functional_dependencies, get_target_functional_dependencies, Constraint, Constraints, Dependency, FunctionalDependence, FunctionalDependencies, }; -pub use join_type::{JoinConstraint, JoinType}; +pub use join_type::{JoinConstraint, JoinSide, JoinType}; pub use scalar::{ScalarType, ScalarValue}; pub use schema_reference::{OwnedSchemaReference, SchemaReference}; pub use stats::{ColumnStatistics, Statistics}; diff --git a/datafusion/core/src/physical_optimizer/sort_pushdown.rs b/datafusion/core/src/physical_optimizer/sort_pushdown.rs index bb991115ec75..9b81ad3efb50 100644 --- a/datafusion/core/src/physical_optimizer/sort_pushdown.rs +++ b/datafusion/core/src/physical_optimizer/sort_pushdown.rs @@ -21,7 +21,7 @@ use crate::physical_optimizer::utils::{ add_sort_above, is_limit, is_sort_preserving_merge, is_union, is_window, }; use crate::physical_plan::filter::FilterExec; -use crate::physical_plan::joins::utils::{calculate_join_output_ordering, JoinSide}; +use crate::physical_plan::joins::utils::calculate_join_output_ordering; use crate::physical_plan::joins::{HashJoinExec, SortMergeJoinExec}; use crate::physical_plan::projection::ProjectionExec; use crate::physical_plan::repartition::RepartitionExec; @@ -29,7 +29,9 @@ use crate::physical_plan::sorts::sort::SortExec; use crate::physical_plan::{with_new_children_if_necessary, ExecutionPlan}; use datafusion_common::tree_node::{Transformed, TreeNode, VisitRecursion}; -use datafusion_common::{plan_datafusion_err, plan_err, DataFusionError, Result}; +use datafusion_common::{ + plan_datafusion_err, plan_err, DataFusionError, JoinSide, Result, +}; use datafusion_expr::JoinType; use datafusion_physical_expr::expressions::Column; use datafusion_physical_expr::utils::{ diff --git a/datafusion/physical-plan/src/joins/hash_join.rs b/datafusion/physical-plan/src/joins/hash_join.rs index 4fd8882c0178..2ffa1f61a23b 100644 --- a/datafusion/physical-plan/src/joins/hash_join.rs +++ b/datafusion/physical-plan/src/joins/hash_join.rs @@ -27,7 +27,7 @@ use std::{any::Any, usize, vec}; use crate::joins::utils::{ adjust_indices_by_join_type, apply_join_filter_to_indices, build_batch_from_indices, calculate_join_output_ordering, combine_join_ordering_equivalence_properties, - get_final_indices_from_bit_map, need_produce_result_in_final, JoinSide, + get_final_indices_from_bit_map, need_produce_result_in_final, }; use crate::DisplayAs; use crate::{ @@ -64,7 +64,7 @@ use arrow::util::bit_util; use arrow_array::cast::downcast_array; use arrow_schema::ArrowError; use datafusion_common::{ - exec_err, internal_err, plan_err, DataFusionError, JoinType, Result, + exec_err, internal_err, plan_err, DataFusionError, JoinSide, JoinType, Result, }; use datafusion_execution::memory_pool::{MemoryConsumer, MemoryReservation}; use datafusion_execution::TaskContext; @@ -1069,14 +1069,9 @@ mod tests { use hashbrown::raw::RawTable; use crate::{ - common, - expressions::Column, - hash_utils::create_hashes, - joins::{hash_join::build_equal_condition_join_indices, utils::JoinSide}, - memory::MemoryExec, - repartition::RepartitionExec, - test::build_table_i32, - test::exec::MockExec, + common, expressions::Column, hash_utils::create_hashes, + joins::hash_join::build_equal_condition_join_indices, memory::MemoryExec, + repartition::RepartitionExec, test::build_table_i32, test::exec::MockExec, }; use datafusion_execution::config::SessionConfig; use datafusion_execution::runtime_env::{RuntimeConfig, RuntimeEnv}; diff --git a/datafusion/physical-plan/src/joins/hash_join_utils.rs b/datafusion/physical-plan/src/joins/hash_join_utils.rs index 525c1a7145b9..3a2a85c72722 100644 --- a/datafusion/physical-plan/src/joins/hash_join_utils.rs +++ b/datafusion/physical-plan/src/joins/hash_join_utils.rs @@ -24,14 +24,14 @@ use std::ops::IndexMut; use std::sync::Arc; use std::{fmt, usize}; -use crate::joins::utils::{JoinFilter, JoinSide}; +use crate::joins::utils::JoinFilter; use arrow::compute::concat_batches; use arrow::datatypes::{ArrowNativeType, SchemaRef}; use arrow_array::builder::BooleanBufferBuilder; use arrow_array::{ArrowPrimitiveType, NativeAdapter, PrimitiveArray, RecordBatch}; use datafusion_common::tree_node::{Transformed, TreeNode}; -use datafusion_common::{DataFusionError, Result, ScalarValue}; +use datafusion_common::{DataFusionError, JoinSide, Result, ScalarValue}; use datafusion_physical_expr::expressions::Column; use datafusion_physical_expr::intervals::{Interval, IntervalBound}; use datafusion_physical_expr::utils::collect_columns; @@ -732,7 +732,7 @@ pub mod tests { use crate::{ expressions::Column, expressions::PhysicalSortExpr, - joins::utils::{ColumnIndex, JoinFilter, JoinSide}, + joins::utils::{ColumnIndex, JoinFilter}, }; use arrow::compute::SortOptions; use arrow::datatypes::{DataType, Field, Schema}; diff --git a/datafusion/physical-plan/src/joins/nested_loop_join.rs b/datafusion/physical-plan/src/joins/nested_loop_join.rs index 944efb47f429..25cb374e941f 100644 --- a/datafusion/physical-plan/src/joins/nested_loop_join.rs +++ b/datafusion/physical-plan/src/joins/nested_loop_join.rs @@ -31,7 +31,7 @@ use crate::joins::utils::{ estimate_join_statistics, get_anti_indices, get_anti_u64_indices, get_final_indices_from_bit_map, get_semi_indices, get_semi_u64_indices, partitioned_join_output_partitioning, BuildProbeJoinMetrics, ColumnIndex, JoinFilter, - JoinSide, OnceAsync, OnceFut, + OnceAsync, OnceFut, }; use crate::metrics::{ExecutionPlanMetricsSet, MetricsSet}; use crate::{ @@ -45,7 +45,7 @@ use arrow::array::{ use arrow::datatypes::{Schema, SchemaRef}; use arrow::record_batch::RecordBatch; use arrow::util::bit_util; -use datafusion_common::{exec_err, DataFusionError, Result, Statistics}; +use datafusion_common::{exec_err, DataFusionError, JoinSide, Result, Statistics}; use datafusion_execution::memory_pool::{MemoryConsumer, MemoryReservation}; use datafusion_execution::TaskContext; use datafusion_expr::JoinType; @@ -743,7 +743,6 @@ mod tests { use std::sync::Arc; use super::*; - use crate::joins::utils::JoinSide; use crate::{ common, expressions::Column, memory::MemoryExec, repartition::RepartitionExec, test::build_table_i32, diff --git a/datafusion/physical-plan/src/joins/sort_merge_join.rs b/datafusion/physical-plan/src/joins/sort_merge_join.rs index 61ac864bf20d..98fe751b22ef 100644 --- a/datafusion/physical-plan/src/joins/sort_merge_join.rs +++ b/datafusion/physical-plan/src/joins/sort_merge_join.rs @@ -34,7 +34,7 @@ use crate::expressions::{Column, PhysicalSortExpr}; use crate::joins::utils::{ build_join_schema, calculate_join_output_ordering, check_join_is_valid, combine_join_equivalence_properties, combine_join_ordering_equivalence_properties, - estimate_join_statistics, partitioned_join_output_partitioning, JoinOn, JoinSide, + estimate_join_statistics, partitioned_join_output_partitioning, JoinOn, }; use crate::metrics::{ExecutionPlanMetricsSet, MetricBuilder, MetricsSet}; use crate::{ @@ -49,7 +49,7 @@ use arrow::datatypes::{DataType, SchemaRef, TimeUnit}; use arrow::error::ArrowError; use arrow::record_batch::RecordBatch; use datafusion_common::{ - internal_err, not_impl_err, plan_err, DataFusionError, JoinType, Result, + internal_err, not_impl_err, plan_err, DataFusionError, JoinSide, JoinType, Result, }; use datafusion_execution::memory_pool::{MemoryConsumer, MemoryReservation}; use datafusion_execution::TaskContext; diff --git a/datafusion/physical-plan/src/joins/symmetric_hash_join.rs b/datafusion/physical-plan/src/joins/symmetric_hash_join.rs index 64128b6e3a12..3450331133bd 100644 --- a/datafusion/physical-plan/src/joins/symmetric_hash_join.rs +++ b/datafusion/physical-plan/src/joins/symmetric_hash_join.rs @@ -43,7 +43,7 @@ use crate::joins::hash_join_utils::{ use crate::joins::utils::{ build_batch_from_indices, build_join_schema, check_join_is_valid, combine_join_equivalence_properties, partitioned_join_output_partitioning, - prepare_sorted_exprs, ColumnIndex, JoinFilter, JoinOn, JoinSide, + prepare_sorted_exprs, ColumnIndex, JoinFilter, JoinOn, }; use crate::{ expressions::{Column, PhysicalSortExpr}, @@ -58,7 +58,9 @@ use arrow::compute::concat_batches; use arrow::datatypes::{Schema, SchemaRef}; use arrow::record_batch::RecordBatch; use datafusion_common::utils::bisect; -use datafusion_common::{internal_err, plan_err, DataFusionError, JoinType, Result}; +use datafusion_common::{ + internal_err, plan_err, DataFusionError, JoinSide, JoinType, Result, +}; use datafusion_execution::memory_pool::MemoryConsumer; use datafusion_execution::TaskContext; use datafusion_physical_expr::intervals::ExprIntervalGraph; diff --git a/datafusion/physical-plan/src/joins/utils.rs b/datafusion/physical-plan/src/joins/utils.rs index c523a8fe8135..afde986c0bb6 100644 --- a/datafusion/physical-plan/src/joins/utils.rs +++ b/datafusion/physical-plan/src/joins/utils.rs @@ -18,7 +18,6 @@ //! Join related functionality used both on logical and physical plans use std::collections::HashSet; -use std::fmt::{Display, Formatter}; use std::future::Future; use std::sync::Arc; use std::task::{Context, Poll}; @@ -42,7 +41,7 @@ use datafusion_common::cast::as_boolean_array; use datafusion_common::stats::Precision; use datafusion_common::tree_node::{Transformed, TreeNode}; use datafusion_common::{ - exec_err, plan_datafusion_err, plan_err, DataFusionError, JoinType, Result, + exec_err, plan_datafusion_err, plan_err, DataFusionError, JoinSide, JoinType, Result, SharedResult, }; use datafusion_physical_expr::expressions::Column; @@ -456,34 +455,6 @@ pub fn combine_join_ordering_equivalence_properties( Ok(new_properties) } -impl Display for JoinSide { - fn fmt(&self, f: &mut Formatter<'_>) -> std::fmt::Result { - match self { - JoinSide::Left => write!(f, "left"), - JoinSide::Right => write!(f, "right"), - } - } -} - -/// Used in ColumnIndex to distinguish which side the index is for -#[derive(Debug, Clone, Copy, PartialEq)] -pub enum JoinSide { - /// Left side of the join - Left, - /// Right side of the join - Right, -} - -impl JoinSide { - /// Inverse the join side - pub fn negate(&self) -> Self { - match self { - JoinSide::Left => JoinSide::Right, - JoinSide::Right => JoinSide::Left, - } - } -} - /// Information about the index and placement (left or right) of the columns #[derive(Debug, Clone)] pub struct ColumnIndex { diff --git a/datafusion/proto/src/physical_plan/from_proto.rs b/datafusion/proto/src/physical_plan/from_proto.rs index cdc772f71d80..a956eded9032 100644 --- a/datafusion/proto/src/physical_plan/from_proto.rs +++ b/datafusion/proto/src/physical_plan/from_proto.rs @@ -35,13 +35,12 @@ use datafusion::physical_plan::expressions::{ Literal, NegativeExpr, NotExpr, TryCastExpr, }; use datafusion::physical_plan::expressions::{GetFieldAccessExpr, GetIndexedFieldExpr}; -use datafusion::physical_plan::joins::utils::JoinSide; use datafusion::physical_plan::windows::create_window_expr; use datafusion::physical_plan::{ functions, ColumnStatistics, Partitioning, PhysicalExpr, Statistics, WindowExpr, }; use datafusion_common::stats::Precision; -use datafusion_common::{not_impl_err, DataFusionError, Result, ScalarValue}; +use datafusion_common::{not_impl_err, DataFusionError, JoinSide, Result, ScalarValue}; use crate::common::proto_error; use crate::convert_required; diff --git a/datafusion/proto/src/physical_plan/to_proto.rs b/datafusion/proto/src/physical_plan/to_proto.rs index 466b99b68472..114baab6ccc4 100644 --- a/datafusion/proto/src/physical_plan/to_proto.rs +++ b/datafusion/proto/src/physical_plan/to_proto.rs @@ -44,14 +44,13 @@ use datafusion::physical_plan::expressions::{ Regr, RegrType, RowNumber, Stddev, StddevPop, Sum, TryCastExpr, Variance, VariancePop, WindowShift, }; -use datafusion::physical_plan::joins::utils::JoinSide; use datafusion::physical_plan::udaf::AggregateFunctionExpr; use datafusion::physical_plan::windows::{BuiltInWindowExpr, PlainAggregateWindowExpr}; use datafusion::physical_plan::{ AggregateExpr, ColumnStatistics, PhysicalExpr, Statistics, WindowExpr, }; use datafusion_common::{ - internal_err, not_impl_err, stats::Precision, DataFusionError, Result, + internal_err, not_impl_err, stats::Precision, DataFusionError, JoinSide, Result, }; impl TryFrom> for protobuf::PhysicalExprNode { From e34dd7651fc3e2cc80f87e8bfaf0d98ec1312195 Mon Sep 17 00:00:00 2001 From: ZENOTME <43447882+ZENOTME@users.noreply.github.com> Date: Mon, 23 Oct 2023 21:17:02 +0800 Subject: [PATCH 115/572] fix ExecutionContext to SessionContext (#7903) Co-authored-by: ZENOTME --- docs/source/library-user-guide/custom-table-providers.md | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/docs/source/library-user-guide/custom-table-providers.md b/docs/source/library-user-guide/custom-table-providers.md index 2de4b1ba055b..ca0e9de779ef 100644 --- a/docs/source/library-user-guide/custom-table-providers.md +++ b/docs/source/library-user-guide/custom-table-providers.md @@ -139,10 +139,10 @@ For filters that can be pushed down, they'll be passed to the `scan` method as t ## Using the Custom Table Provider -In order to use the custom table provider, we need to register it with DataFusion. This is done by creating a `TableProvider` and registering it with the `ExecutionContext`. +In order to use the custom table provider, we need to register it with DataFusion. This is done by creating a `TableProvider` and registering it with the `SessionContext`. ```rust -let mut ctx = ExecutionContext::new(); +let mut ctx = SessionContext::new(); let custom_table_provider = CustomDataSource::new(); ctx.register_table("custom_table", Arc::new(custom_table_provider)); @@ -160,7 +160,7 @@ To recap, in order to implement a custom table provider, you need to: 1. Implement the `TableProvider` trait 2. Implement the `ExecutionPlan` trait -3. Register the `TableProvider` with the `ExecutionContext` +3. Register the `TableProvider` with the `SessionContext` ## Next Steps From ae85a67f44a98522204a7ed5fd125b2ef006cee0 Mon Sep 17 00:00:00 2001 From: Raphael Taylor-Davies <1781103+tustvold@users.noreply.github.com> Date: Mon, 23 Oct 2023 18:33:23 +0100 Subject: [PATCH 116/572] Update arrow 48.0.0 (#7854) * Update arrow 48.0.0 * Fix for pyo3 * Update json.slt * Update pin and fix clippy * More clippy --- Cargo.toml | 12 +-- datafusion-cli/Cargo.lock | 97 ++++++++----------- datafusion-cli/Cargo.toml | 2 +- datafusion-cli/src/print_format.rs | 2 +- datafusion/common/Cargo.toml | 2 +- .../common/src/file_options/csv_writer.rs | 24 +---- datafusion/common/src/file_options/mod.rs | 2 +- datafusion/common/src/pyarrow.rs | 9 +- .../core/src/datasource/file_format/csv.rs | 6 +- .../src/datasource/listing_table_factory.rs | 3 +- .../core/src/datasource/physical_plan/csv.rs | 4 +- datafusion/core/tests/fifo.rs | 2 +- datafusion/sqllogictest/test_files/json.slt | 4 +- datafusion/wasmtest/Cargo.toml | 2 +- 14 files changed, 69 insertions(+), 102 deletions(-) diff --git a/Cargo.toml b/Cargo.toml index d28ebd15a09e..71088e7fc7ad 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -48,12 +48,12 @@ rust-version = "1.70" version = "32.0.0" [workspace.dependencies] -arrow = { version = "47.0.0", features = ["prettyprint"] } -arrow-array = { version = "47.0.0", default-features = false, features = ["chrono-tz"] } -arrow-buffer = { version = "47.0.0", default-features = false } -arrow-flight = { version = "47.0.0", features = ["flight-sql-experimental"] } -arrow-schema = { version = "47.0.0", default-features = false } -parquet = { version = "47.0.0", features = ["arrow", "async", "object_store"] } +arrow = { version = "48.0.0", features = ["prettyprint"] } +arrow-array = { version = "48.0.0", default-features = false, features = ["chrono-tz"] } +arrow-buffer = { version = "48.0.0", default-features = false } +arrow-flight = { version = "48.0.0", features = ["flight-sql-experimental"] } +arrow-schema = { version = "48.0.0", default-features = false } +parquet = { version = "48.0.0", features = ["arrow", "async", "object_store"] } sqlparser = { version = "0.38.0", features = ["visitor"] } chrono = { version = "0.4.31", default-features = false } diff --git a/datafusion-cli/Cargo.lock b/datafusion-cli/Cargo.lock index 1c872c28485c..b83088f94c57 100644 --- a/datafusion-cli/Cargo.lock +++ b/datafusion-cli/Cargo.lock @@ -129,9 +129,9 @@ checksum = "96d30a06541fbafbc7f82ed10c06164cfbd2c401138f6addd8404629c4b16711" [[package]] name = "arrow" -version = "47.0.0" +version = "48.0.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "7fab9e93ba8ce88a37d5a30dce4b9913b75413dc1ac56cb5d72e5a840543f829" +checksum = "edb738d83750ec705808f6d44046d165e6bb8623f64e29a4d53fcb136ab22dfb" dependencies = [ "ahash", "arrow-arith", @@ -151,9 +151,9 @@ dependencies = [ [[package]] name = "arrow-arith" -version = "47.0.0" +version = "48.0.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "bc1d4e368e87ad9ee64f28b9577a3834ce10fe2703a26b28417d485bbbdff956" +checksum = "c5c3d17fc5b006e7beeaebfb1d2edfc92398b981f82d9744130437909b72a468" dependencies = [ "arrow-array", "arrow-buffer", @@ -166,9 +166,9 @@ dependencies = [ [[package]] name = "arrow-array" -version = "47.0.0" +version = "48.0.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "d02efa7253ede102d45a4e802a129e83bcc3f49884cab795b1ac223918e4318d" +checksum = "55705ada5cdde4cb0f202ffa6aa756637e33fea30e13d8d0d0fd6a24ffcee1e3" dependencies = [ "ahash", "arrow-buffer", @@ -183,9 +183,9 @@ dependencies = [ [[package]] name = "arrow-buffer" -version = "47.0.0" +version = "48.0.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "fda119225204141138cb0541c692fbfef0e875ba01bfdeaed09e9d354f9d6195" +checksum = "a722f90a09b94f295ab7102542e97199d3500128843446ef63e410ad546c5333" dependencies = [ "bytes", "half", @@ -194,9 +194,9 @@ dependencies = [ [[package]] name = "arrow-cast" -version = "47.0.0" +version = "48.0.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "1d825d51b9968868d50bc5af92388754056796dbc62a4e25307d588a1fc84dee" +checksum = "af01fc1a06f6f2baf31a04776156d47f9f31ca5939fe6d00cd7a059f95a46ff1" dependencies = [ "arrow-array", "arrow-buffer", @@ -212,9 +212,9 @@ dependencies = [ [[package]] name = "arrow-csv" -version = "47.0.0" +version = "48.0.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "43ef855dc6b126dc197f43e061d4de46b9d4c033aa51c2587657f7508242cef1" +checksum = "83cbbfde86f9ecd3f875c42a73d8aeab3d95149cd80129b18d09e039ecf5391b" dependencies = [ "arrow-array", "arrow-buffer", @@ -231,9 +231,9 @@ dependencies = [ [[package]] name = "arrow-data" -version = "47.0.0" +version = "48.0.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "475a4c3699c8b4095ca61cecf15da6f67841847a5f5aac983ccb9a377d02f73a" +checksum = "d0a547195e607e625e7fafa1a7269b8df1a4a612c919efd9b26bd86e74538f3a" dependencies = [ "arrow-buffer", "arrow-schema", @@ -243,9 +243,9 @@ dependencies = [ [[package]] name = "arrow-ipc" -version = "47.0.0" +version = "48.0.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "1248005c8ac549f869b7a840859d942bf62471479c1a2d82659d453eebcd166a" +checksum = "e36bf091502ab7e37775ff448413ef1ffff28ff93789acb669fffdd51b394d51" dependencies = [ "arrow-array", "arrow-buffer", @@ -257,9 +257,9 @@ dependencies = [ [[package]] name = "arrow-json" -version = "47.0.0" +version = "48.0.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "f03d7e3b04dd688ccec354fe449aed56b831679f03e44ee2c1cfc4045067b69c" +checksum = "7ac346bc84846ab425ab3c8c7b6721db90643bc218939677ed7e071ccbfb919d" dependencies = [ "arrow-array", "arrow-buffer", @@ -277,9 +277,9 @@ dependencies = [ [[package]] name = "arrow-ord" -version = "47.0.0" +version = "48.0.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "03b87aa408ea6a6300e49eb2eba0c032c88ed9dc19e0a9948489c55efdca71f4" +checksum = "4502123d2397319f3a13688432bc678c61cb1582f2daa01253186da650bf5841" dependencies = [ "arrow-array", "arrow-buffer", @@ -292,9 +292,9 @@ dependencies = [ [[package]] name = "arrow-row" -version = "47.0.0" +version = "48.0.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "114a348ab581e7c9b6908fcab23cb39ff9f060eb19e72b13f8fb8eaa37f65d22" +checksum = "249fc5a07906ab3f3536a6e9f118ec2883fbcde398a97a5ba70053f0276abda4" dependencies = [ "ahash", "arrow-array", @@ -307,15 +307,15 @@ dependencies = [ [[package]] name = "arrow-schema" -version = "47.0.0" +version = "48.0.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "5d1d179c117b158853e0101bfbed5615e86fe97ee356b4af901f1c5001e1ce4b" +checksum = "9d7a8c3f97f5ef6abd862155a6f39aaba36b029322462d72bbcfa69782a50614" [[package]] name = "arrow-select" -version = "47.0.0" +version = "48.0.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "d5c71e003202e67e9db139e5278c79f5520bb79922261dfe140e4637ee8b6108" +checksum = "f868f4a5001429e20f7c1994b5cd1aa68b82e3db8cf96c559cdb56dc8be21410" dependencies = [ "ahash", "arrow-array", @@ -327,9 +327,9 @@ dependencies = [ [[package]] name = "arrow-string" -version = "47.0.0" +version = "48.0.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "c4cebbb282d6b9244895f4a9a912e55e57bce112554c7fa91fcec5459cb421ab" +checksum = "a27fdf8fc70040a2dee78af2e217479cb5b263bd7ab8711c7999e74056eb688a" dependencies = [ "arrow-array", "arrow-buffer", @@ -338,7 +338,7 @@ dependencies = [ "arrow-select", "num", "regex", - "regex-syntax 0.7.5", + "regex-syntax", ] [[package]] @@ -1232,7 +1232,7 @@ dependencies = [ "hashbrown 0.14.1", "itertools", "log", - "regex-syntax 0.8.1", + "regex-syntax", ] [[package]] @@ -2072,23 +2072,12 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "b5e6163cb8c49088c2c36f57875e58ccd8c87c7427f7fbd50ea6710b2f3f2e8f" [[package]] -name = "lz4" -version = "1.24.0" +name = "lz4_flex" +version = "0.11.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "7e9e2dd86df36ce760a60f6ff6ad526f7ba1f14ba0356f8254fb6905e6494df1" +checksum = "3ea9b256699eda7b0387ffbc776dd625e28bde3918446381781245b7a50349d8" dependencies = [ - "libc", - "lz4-sys", -] - -[[package]] -name = "lz4-sys" -version = "1.9.4" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "57d27b317e207b10f69f5e75494119e391a96f48861ae870d1da6edac98ca900" -dependencies = [ - "cc", - "libc", + "twox-hash", ] [[package]] @@ -2363,9 +2352,9 @@ dependencies = [ [[package]] name = "parquet" -version = "47.0.0" +version = "48.0.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "0463cc3b256d5f50408c49a4be3a16674f4c8ceef60941709620a062b1f6bf4d" +checksum = "239229e6a668ab50c61de3dce61cf0fa1069345f7aa0f4c934491f92205a4945" dependencies = [ "ahash", "arrow-array", @@ -2382,7 +2371,7 @@ dependencies = [ "flate2", "futures", "hashbrown 0.14.1", - "lz4", + "lz4_flex", "num", "num-bigint", "object_store", @@ -2392,7 +2381,7 @@ dependencies = [ "thrift", "tokio", "twox-hash", - "zstd 0.12.4", + "zstd 0.13.0", ] [[package]] @@ -2681,7 +2670,7 @@ dependencies = [ "aho-corasick", "memchr", "regex-automata", - "regex-syntax 0.8.1", + "regex-syntax", ] [[package]] @@ -2692,7 +2681,7 @@ checksum = "465c6fc0621e4abc4187a2bda0937bfd4f722c2730b29562e19689ea796c9a4b" dependencies = [ "aho-corasick", "memchr", - "regex-syntax 0.8.1", + "regex-syntax", ] [[package]] @@ -2701,12 +2690,6 @@ version = "0.1.3" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "9a6ebcd15653947e6140f59a9811a06ed061d18a5c35dfca2e2e4c5525696878" -[[package]] -name = "regex-syntax" -version = "0.7.5" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "dbb5fb1acd8a1a18b3dd5be62d25485eb770e05afb408a9627d14d451bae12da" - [[package]] name = "regex-syntax" version = "0.8.1" diff --git a/datafusion-cli/Cargo.toml b/datafusion-cli/Cargo.toml index b2a22cec987f..64e094437c5f 100644 --- a/datafusion-cli/Cargo.toml +++ b/datafusion-cli/Cargo.toml @@ -29,7 +29,7 @@ rust-version = "1.70" readme = "README.md" [dependencies] -arrow = "47.0.0" +arrow = "48.0.0" async-trait = "0.1.41" aws-config = "0.55" aws-credential-types = "0.55" diff --git a/datafusion-cli/src/print_format.rs b/datafusion-cli/src/print_format.rs index e2994bc14034..0738bf6f9b47 100644 --- a/datafusion-cli/src/print_format.rs +++ b/datafusion-cli/src/print_format.rs @@ -59,7 +59,7 @@ fn print_batches_with_sep(batches: &[RecordBatch], delimiter: u8) -> Result for CsvWriterOptions { fn try_from(value: (&ConfigOptions, &StatementOptions)) -> Result { let _configs = value.0; let statement_options = value.1; - let mut has_header = true; let mut builder = WriterBuilder::default(); let mut compression = CompressionTypeVariant::UNCOMPRESSED; for (option, value) in &statement_options.options { builder = match option.to_lowercase().as_str(){ "header" => { - has_header = value.parse() + let has_header = value.parse() .map_err(|_| DataFusionError::Configuration(format!("Unable to parse {value} as bool as required for {option}!")))?; - builder.has_headers(has_header) + builder.with_header(has_header) }, "date_format" => builder.with_date_format(value.to_owned()), "datetime_format" => builder.with_datetime_format(value.to_owned()), "timestamp_format" => builder.with_timestamp_format(value.to_owned()), "time_format" => builder.with_time_format(value.to_owned()), - "rfc3339" => { - let value_bool = value.parse() - .map_err(|_| DataFusionError::Configuration(format!("Unable to parse {value} as bool as required for {option}!")))?; - if value_bool{ - builder.with_rfc3339() - } else{ - builder - } - }, + "rfc3339" => builder, // No-op "null_value" => builder.with_null(value.to_owned()), "compression" => { compression = CompressionTypeVariant::from_str(value.replace('\'', "").as_str())?; @@ -112,7 +95,6 @@ impl TryFrom<(&ConfigOptions, &StatementOptions)> for CsvWriterOptions { } } Ok(CsvWriterOptions { - has_header, writer_options: builder, compression, }) diff --git a/datafusion/common/src/file_options/mod.rs b/datafusion/common/src/file_options/mod.rs index 45b105dfadae..b7c1341e3046 100644 --- a/datafusion/common/src/file_options/mod.rs +++ b/datafusion/common/src/file_options/mod.rs @@ -523,9 +523,9 @@ mod tests { let csv_options = CsvWriterOptions::try_from((&config, &options))?; let builder = csv_options.writer_options; + assert!(builder.header()); let buff = Vec::new(); let _properties = builder.build(buff); - assert!(csv_options.has_header); assert_eq!(csv_options.compression, CompressionTypeVariant::GZIP); // TODO expand unit test if csv::WriterBuilder allows public read access to properties diff --git a/datafusion/common/src/pyarrow.rs b/datafusion/common/src/pyarrow.rs index d78aa8b988f7..59a8b811e3c8 100644 --- a/datafusion/common/src/pyarrow.rs +++ b/datafusion/common/src/pyarrow.rs @@ -94,10 +94,11 @@ mod tests { Some(locals), ) .expect("Couldn't get python info"); - let executable: String = - locals.get_item("executable").unwrap().extract().unwrap(); - let python_path: Vec<&str> = - locals.get_item("python_path").unwrap().extract().unwrap(); + let executable = locals.get_item("executable").unwrap().unwrap(); + let executable: String = executable.extract().unwrap(); + + let python_path = locals.get_item("python_path").unwrap().unwrap(); + let python_path: Vec<&str> = python_path.extract().unwrap(); panic!("pyarrow not found\nExecutable: {executable}\nPython path: {python_path:?}\n\ HINT: try `pip install pyarrow`\n\ diff --git a/datafusion/core/src/datasource/file_format/csv.rs b/datafusion/core/src/datasource/file_format/csv.rs index a45fbf11f35c..5f2084bc80a8 100644 --- a/datafusion/core/src/datasource/file_format/csv.rs +++ b/datafusion/core/src/datasource/file_format/csv.rs @@ -433,7 +433,7 @@ impl CsvSerializer { impl BatchSerializer for CsvSerializer { async fn serialize(&mut self, batch: RecordBatch) -> Result { let builder = self.builder.clone(); - let mut writer = builder.has_headers(self.header).build(&mut self.buffer); + let mut writer = builder.with_header(self.header).build(&mut self.buffer); writer.write(&batch)?; drop(writer); self.header = false; @@ -513,7 +513,7 @@ impl CsvSink { } else { CsvSerializer::new() .with_builder(inner_clone) - .with_header(options_clone.has_header) + .with_header(options_clone.writer_options.header()) }); serializer }; @@ -545,7 +545,7 @@ impl CsvSink { let serializer: Box = Box::new( CsvSerializer::new() .with_builder(inner_clone) - .with_header(options_clone.has_header), + .with_header(options_clone.writer_options.header()), ); serializer }; diff --git a/datafusion/core/src/datasource/listing_table_factory.rs b/datafusion/core/src/datasource/listing_table_factory.rs index ebfb589f179e..e74bf6fa6499 100644 --- a/datafusion/core/src/datasource/listing_table_factory.rs +++ b/datafusion/core/src/datasource/listing_table_factory.rs @@ -179,10 +179,9 @@ impl TableProviderFactory for ListingTableFactory { FileType::CSV => { let mut csv_writer_options = file_type_writer_options.try_into_csv()?.clone(); - csv_writer_options.has_header = cmd.has_header; csv_writer_options.writer_options = csv_writer_options .writer_options - .has_headers(cmd.has_header) + .with_header(cmd.has_header) .with_delimiter(cmd.delimiter.try_into().map_err(|_| { DataFusionError::Internal( "Unable to convert CSV delimiter into u8".into(), diff --git a/datafusion/core/src/datasource/physical_plan/csv.rs b/datafusion/core/src/datasource/physical_plan/csv.rs index e60a249b0b4a..8117e101ea99 100644 --- a/datafusion/core/src/datasource/physical_plan/csv.rs +++ b/datafusion/core/src/datasource/physical_plan/csv.rs @@ -289,7 +289,7 @@ impl CsvConfig { let mut builder = csv::ReaderBuilder::new(self.file_schema.clone()) .with_delimiter(self.delimiter) .with_batch_size(self.batch_size) - .has_header(self.has_header) + .with_header(self.has_header) .with_quote(self.quote); if let Some(proj) = &self.file_projection { @@ -538,7 +538,7 @@ pub async fn plan_to_csv( let mut write_headers = true; while let Some(batch) = stream.next().await.transpose()? { let mut writer = csv::WriterBuilder::new() - .has_headers(write_headers) + .with_header(write_headers) .build(buffer); writer.write(&batch)?; buffer = writer.into_inner(); diff --git a/datafusion/core/tests/fifo.rs b/datafusion/core/tests/fifo.rs index 2c8b0b784f26..7d9ea97f7b5b 100644 --- a/datafusion/core/tests/fifo.rs +++ b/datafusion/core/tests/fifo.rs @@ -377,7 +377,7 @@ mod unix_test { ])); let mut reader = ReaderBuilder::new(schema) - .has_header(true) + .with_header(true) .with_batch_size(TEST_BATCH_SIZE) .build(file) .map_err(|e| DataFusionError::Internal(e.to_string())) diff --git a/datafusion/sqllogictest/test_files/json.slt b/datafusion/sqllogictest/test_files/json.slt index f903e48063f8..a06565f42cdb 100644 --- a/datafusion/sqllogictest/test_files/json.slt +++ b/datafusion/sqllogictest/test_files/json.slt @@ -58,8 +58,10 @@ AggregateExec: mode=Final, gby=[], aggr=[COUNT(*)] ------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 --------JsonExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/2.json]]}, projection=[c] -query error DataFusion error: Schema error: No field named mycol\. +query ? SELECT mycol FROM single_nan +---- +NULL statement ok DROP TABLE json_test diff --git a/datafusion/wasmtest/Cargo.toml b/datafusion/wasmtest/Cargo.toml index 691031866af2..e1a9a5d41a5a 100644 --- a/datafusion/wasmtest/Cargo.toml +++ b/datafusion/wasmtest/Cargo.toml @@ -46,5 +46,5 @@ datafusion-sql = { path = "../sql" } # getrandom must be compiled with js feature getrandom = { version = "0.2.8", features = ["js"] } -parquet = { version = "47.0.0", default-features = false } +parquet = { version = "48.0.0", default-features = false } wasm-bindgen = "0.2.87" From eee790f695a58a99e880957d50a33c1f075c8edc Mon Sep 17 00:00:00 2001 From: Jonah Gao Date: Tue, 24 Oct 2023 03:54:55 +0800 Subject: [PATCH 117/572] feat: support `Decimal256` for the `abs` function (#7904) * feat: support Decimal256 for the abs function * Remove useless comment * use wrapping_abs --- .../physical-expr/src/math_expressions.rs | 28 +++++---- datafusion/sqllogictest/test_files/math.slt | 63 ++++++++++++------- 2 files changed, 58 insertions(+), 33 deletions(-) diff --git a/datafusion/physical-expr/src/math_expressions.rs b/datafusion/physical-expr/src/math_expressions.rs index a9dc0bd58f15..96f611e2b7b4 100644 --- a/datafusion/physical-expr/src/math_expressions.rs +++ b/datafusion/physical-expr/src/math_expressions.rs @@ -19,8 +19,8 @@ use arrow::array::ArrayRef; use arrow::array::{ - BooleanArray, Decimal128Array, Float32Array, Float64Array, Int16Array, Int32Array, - Int64Array, Int8Array, + BooleanArray, Decimal128Array, Decimal256Array, Float32Array, Float64Array, + Int16Array, Int32Array, Int64Array, Int8Array, }; use arrow::datatypes::DataType; use arrow::error::ArrowError; @@ -701,6 +701,18 @@ macro_rules! make_try_abs_function { }}; } +macro_rules! make_decimal_abs_function { + ($ARRAY_TYPE:ident) => {{ + |args: &[ArrayRef]| { + let array = downcast_arg!(&args[0], "abs arg", $ARRAY_TYPE); + let res: $ARRAY_TYPE = array + .unary(|x| x.wrapping_abs()) + .with_data_type(args[0].data_type().clone()); + Ok(Arc::new(res) as ArrayRef) + } + }}; +} + /// Abs SQL function /// Return different implementations based on input datatype to reduce branches during execution pub(super) fn create_abs_function( @@ -723,15 +735,9 @@ pub(super) fn create_abs_function( | DataType::UInt32 | DataType::UInt64 => Ok(|args: &[ArrayRef]| Ok(args[0].clone())), - // Decimal should keep the same precision and scale by using `with_data_type()`. - // https://github.com/apache/arrow-rs/issues/4644 - DataType::Decimal128(_, _) => Ok(|args: &[ArrayRef]| { - let array = downcast_arg!(&args[0], "abs arg", Decimal128Array); - let res: Decimal128Array = array - .unary(i128::abs) - .with_data_type(args[0].data_type().clone()); - Ok(Arc::new(res) as ArrayRef) - }), + // Decimal types + DataType::Decimal128(_, _) => Ok(make_decimal_abs_function!(Decimal128Array)), + DataType::Decimal256(_, _) => Ok(make_decimal_abs_function!(Decimal256Array)), other => not_impl_err!("Unsupported data type {other:?} for function abs"), } diff --git a/datafusion/sqllogictest/test_files/math.slt b/datafusion/sqllogictest/test_files/math.slt index a3ee307f4940..ee1e345f946a 100644 --- a/datafusion/sqllogictest/test_files/math.slt +++ b/datafusion/sqllogictest/test_files/math.slt @@ -395,7 +395,7 @@ NaN NaN # abs: return type query TT rowsort -SELECT arrow_typeof(c1), arrow_typeof(c2) FROM test_nullable_float limit 1 +SELECT arrow_typeof(abs(c1)), arrow_typeof(abs(c2)) FROM test_nullable_float limit 1 ---- Float32 Float64 @@ -466,34 +466,48 @@ drop table test_non_nullable_float statement ok CREATE TABLE test_nullable_decimal( - c1 DECIMAL(10, 2), - c2 DECIMAL(38, 10) - ) AS VALUES (0, 0), (NULL, NULL); - -query RR + c1 DECIMAL(10, 2), /* Decimal128 */ + c2 DECIMAL(38, 10), /* Decimal128 with max precision */ + c3 DECIMAL(40, 2), /* Decimal256 */ + c4 DECIMAL(76, 10) /* Decimal256 with max precision */ + ) AS VALUES + (0, 0, 0, 0), + (NULL, NULL, NULL, NULL); + +query RRRR INSERT into test_nullable_decimal values - (-99999999.99, '-9999999999999999999999999999.9999999999'), - (99999999.99, '9999999999999999999999999999.9999999999'); + ( + -99999999.99, + '-9999999999999999999999999999.9999999999', + '-99999999999999999999999999999999999999.99', + '-999999999999999999999999999999999999999999999999999999999999999999.9999999999' + ), + ( + 99999999.99, + '9999999999999999999999999999.9999999999', + '99999999999999999999999999999999999999.99', + '999999999999999999999999999999999999999999999999999999999999999999.9999999999' + ) ---- 2 -query R rowsort +query R SELECT c1*0 FROM test_nullable_decimal WHERE c1 IS NULL; ---- NULL -query R rowsort +query R SELECT c1/0 FROM test_nullable_decimal WHERE c1 IS NULL; ---- NULL -query R rowsort +query R SELECT c1%0 FROM test_nullable_decimal WHERE c1 IS NULL; ---- NULL -query R rowsort +query R SELECT c1*0 FROM test_nullable_decimal WHERE c1 IS NOT NULL; ---- 0 @@ -507,19 +521,24 @@ query error DataFusion error: Arrow error: Divide by zero error SELECT c1%0 FROM test_nullable_decimal WHERE c1 IS NOT NULL; # abs: return type -query TT rowsort -SELECT arrow_typeof(c1), arrow_typeof(c2) FROM test_nullable_decimal limit 1 +query TTTT +SELECT + arrow_typeof(abs(c1)), + arrow_typeof(abs(c2)), + arrow_typeof(abs(c3)), + arrow_typeof(abs(c4)) +FROM test_nullable_decimal limit 1 ---- -Decimal128(10, 2) Decimal128(38, 10) +Decimal128(10, 2) Decimal128(38, 10) Decimal256(40, 2) Decimal256(76, 10) -# abs: Decimal128 -query RR rowsort -SELECT abs(c1), abs(c2) FROM test_nullable_decimal +# abs: decimals +query RRRR rowsort +SELECT abs(c1), abs(c2), abs(c3), abs(c4) FROM test_nullable_decimal ---- -0 0 -99999999.99 9999999999999999999999999999.9999999999 -99999999.99 9999999999999999999999999999.9999999999 -NULL NULL +0 0 0 0 +99999999.99 9999999999999999999999999999.9999999999 99999999999999999999999999999999999999.99 999999999999999999999999999999999999999999999999999999999999999999.9999999999 +99999999.99 9999999999999999999999999999.9999999999 99999999999999999999999999999999999999.99 999999999999999999999999999999999999999999999999999999999999999999.9999999999 +NULL NULL NULL NULL statement ok drop table test_nullable_decimal From 44bbb0ec32e5957fdf0c9072d2518493c4909c6d Mon Sep 17 00:00:00 2001 From: Mustafa Akur <106137913+mustafasrepo@users.noreply.github.com> Date: Tue, 24 Oct 2023 08:44:03 +0300 Subject: [PATCH 118/572] Simplifications (#7907) --- .../physical-plan/src/aggregates/mod.rs | 81 +++++++++---------- datafusion/physical-plan/src/projection.rs | 20 ++--- 2 files changed, 45 insertions(+), 56 deletions(-) diff --git a/datafusion/physical-plan/src/aggregates/mod.rs b/datafusion/physical-plan/src/aggregates/mod.rs index 7191d51fb7f0..1fa129680cea 100644 --- a/datafusion/physical-plan/src/aggregates/mod.rs +++ b/datafusion/physical-plan/src/aggregates/mod.rs @@ -218,6 +218,23 @@ impl PhysicalGroupBy { pub fn is_single(&self) -> bool { self.null_expr.is_empty() } + + /// Calculate GROUP BY expressions according to input schema. + pub fn input_exprs(&self) -> Vec> { + self.expr + .iter() + .map(|(expr, _alias)| expr.clone()) + .collect() + } + + /// Return grouping expressions as they occur in the output schema. + fn output_exprs(&self) -> Vec> { + self.expr + .iter() + .enumerate() + .map(|(index, (_, name))| Arc::new(Column::new(name, index)) as _) + .collect() + } } impl PartialEq for PhysicalGroupBy { @@ -319,11 +336,7 @@ fn get_working_mode( // Since direction of the ordering is not important for GROUP BY columns, // we convert PhysicalSortExpr to PhysicalExpr in the existing ordering. let ordering_exprs = convert_to_expr(output_ordering); - let groupby_exprs = group_by - .expr - .iter() - .map(|(item, _)| item.clone()) - .collect::>(); + let groupby_exprs = group_by.input_exprs(); // Find where each expression of the GROUP BY clause occurs in the existing // ordering (if it occurs): let mut ordered_indices = @@ -363,7 +376,7 @@ fn calc_aggregation_ordering( ) -> Option { get_working_mode(input, group_by).map(|(mode, order_indices)| { let existing_ordering = input.output_ordering().unwrap_or(&[]); - let out_group_expr = output_group_expr_helper(group_by); + let out_group_expr = group_by.output_exprs(); // Calculate output ordering information for the operator: let out_ordering = order_indices .iter() @@ -381,18 +394,6 @@ fn calc_aggregation_ordering( }) } -/// This function returns grouping expressions as they occur in the output schema. -fn output_group_expr_helper(group_by: &PhysicalGroupBy) -> Vec> { - // Update column indices. Since the group by columns come first in the output schema, their - // indices are simply 0..self.group_expr(len). - group_by - .expr() - .iter() - .enumerate() - .map(|(index, (_, name))| Arc::new(Column::new(name, index)) as _) - .collect() -} - /// This function returns the ordering requirement of the first non-reversible /// order-sensitive aggregate function such as ARRAY_AGG. This requirement serves /// as the initial requirement while calculating the finest requirement among all @@ -591,11 +592,7 @@ fn group_by_contains_all_requirements( group_by: &PhysicalGroupBy, requirement: &LexOrdering, ) -> bool { - let physical_exprs = group_by - .expr() - .iter() - .map(|(expr, _alias)| expr.clone()) - .collect::>(); + let physical_exprs = group_by.input_exprs(); // When we have multiple groups (grouping set) // since group by may be calculated on the subset of the group_by.expr() // it is not guaranteed to have all of the requirements among group by expressions. @@ -735,7 +732,7 @@ impl AggregateExec { /// Grouping expressions as they occur in the output schema pub fn output_group_expr(&self) -> Vec> { - output_group_expr_helper(&self.group_by) + self.group_by.output_exprs() } /// Aggregate expressions @@ -894,28 +891,24 @@ impl ExecutionPlan for AggregateExec { /// Get the output partitioning of this plan fn output_partitioning(&self) -> Partitioning { - match &self.mode { - AggregateMode::Partial | AggregateMode::Single => { - // Partial and Single Aggregation will not change the output partitioning but need to respect the Alias - let input_partition = self.input.output_partitioning(); - match input_partition { - Partitioning::Hash(exprs, part) => { - let normalized_exprs = exprs - .into_iter() - .map(|expr| { - normalize_out_expr_with_columns_map( - expr, - &self.columns_map, - ) - }) - .collect::>(); - Partitioning::Hash(normalized_exprs, part) - } - _ => input_partition, - } + let input_partition = self.input.output_partitioning(); + if self.mode.is_first_stage() { + // First stage Aggregation will not change the output partitioning but need to respect the Alias + let input_partition = self.input.output_partitioning(); + if let Partitioning::Hash(exprs, part) = input_partition { + let normalized_exprs = exprs + .into_iter() + .map(|expr| { + normalize_out_expr_with_columns_map(expr, &self.columns_map) + }) + .collect::>(); + Partitioning::Hash(normalized_exprs, part) + } else { + input_partition } + } else { // Final Aggregation's output partitioning is the same as its real input - _ => self.input.output_partitioning(), + input_partition } } diff --git a/datafusion/physical-plan/src/projection.rs b/datafusion/physical-plan/src/projection.rs index 029dd24d7d11..a374154c995c 100644 --- a/datafusion/physical-plan/src/projection.rs +++ b/datafusion/physical-plan/src/projection.rs @@ -224,18 +224,14 @@ impl ExecutionPlan for ProjectionExec { fn output_partitioning(&self) -> Partitioning { // Output partition need to respect the alias let input_partition = self.input.output_partitioning(); - match input_partition { - Partitioning::Hash(exprs, part) => { - let normalized_exprs = exprs - .into_iter() - .map(|expr| { - normalize_out_expr_with_columns_map(expr, &self.columns_map) - }) - .collect::>(); - - Partitioning::Hash(normalized_exprs, part) - } - _ => input_partition, + if let Partitioning::Hash(exprs, part) = input_partition { + let normalized_exprs = exprs + .into_iter() + .map(|expr| normalize_out_expr_with_columns_map(expr, &self.columns_map)) + .collect::>(); + Partitioning::Hash(normalized_exprs, part) + } else { + input_partition } } From 0b268b28722d5f6c70a984424f9b23dfc8f981bc Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Tue, 24 Oct 2023 12:33:56 +0200 Subject: [PATCH 119/572] Bump actions/setup-node from 3 to 4 (#7915) Bumps [actions/setup-node](https://github.com/actions/setup-node) from 3 to 4. - [Release notes](https://github.com/actions/setup-node/releases) - [Commits](https://github.com/actions/setup-node/compare/v3...v4) --- updated-dependencies: - dependency-name: actions/setup-node dependency-type: direct:production update-type: version-update:semver-major ... Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- .github/workflows/dev.yml | 2 +- .github/workflows/rust.yml | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/.github/workflows/dev.yml b/.github/workflows/dev.yml index 15fbbfca0f65..1f5088a1e6ce 100644 --- a/.github/workflows/dev.yml +++ b/.github/workflows/dev.yml @@ -41,7 +41,7 @@ jobs: runs-on: ubuntu-latest steps: - uses: actions/checkout@v4 - - uses: actions/setup-node@v3 + - uses: actions/setup-node@v4 with: node-version: "14" - name: Prettier check diff --git a/.github/workflows/rust.yml b/.github/workflows/rust.yml index 49ce70a5c0bc..55f6cecf54aa 100644 --- a/.github/workflows/rust.yml +++ b/.github/workflows/rust.yml @@ -525,7 +525,7 @@ jobs: uses: ./.github/actions/setup-builder with: rust-version: stable - - uses: actions/setup-node@v3 + - uses: actions/setup-node@v4 with: node-version: "14" - name: Check if configs.md has been modified From 9e848bf0790aea2323b936e9298af8bcc7e05c85 Mon Sep 17 00:00:00 2001 From: Mustafa Akur <106137913+mustafasrepo@users.noreply.github.com> Date: Tue, 24 Oct 2023 13:58:01 +0300 Subject: [PATCH 120/572] Fix bug, first last reverse (#7914) --- .../physical-expr/src/aggregate/first_last.rs | 8 +++-- .../physical-expr/src/aggregate/utils.rs | 12 +++---- .../sqllogictest/test_files/groupby.slt | 32 +++++++++++++++++++ 3 files changed, 41 insertions(+), 11 deletions(-) diff --git a/datafusion/physical-expr/src/aggregate/first_last.rs b/datafusion/physical-expr/src/aggregate/first_last.rs index 6ae7b4895ad6..ce7a1daeec64 100644 --- a/datafusion/physical-expr/src/aggregate/first_last.rs +++ b/datafusion/physical-expr/src/aggregate/first_last.rs @@ -22,7 +22,9 @@ use std::sync::Arc; use crate::aggregate::utils::{down_cast_any_ref, ordering_fields}; use crate::expressions::format_state_name; -use crate::{AggregateExpr, LexOrdering, PhysicalExpr, PhysicalSortExpr}; +use crate::{ + reverse_order_bys, AggregateExpr, LexOrdering, PhysicalExpr, PhysicalSortExpr, +}; use arrow::array::ArrayRef; use arrow::compute; @@ -126,7 +128,7 @@ impl AggregateExpr for FirstValue { self.expr.clone(), name, self.input_data_type.clone(), - self.ordering_req.clone(), + reverse_order_bys(&self.ordering_req), self.order_by_data_types.clone(), ))) } @@ -350,7 +352,7 @@ impl AggregateExpr for LastValue { self.expr.clone(), name, self.input_data_type.clone(), - self.ordering_req.clone(), + reverse_order_bys(&self.ordering_req), self.order_by_data_types.clone(), ))) } diff --git a/datafusion/physical-expr/src/aggregate/utils.rs b/datafusion/physical-expr/src/aggregate/utils.rs index 420b26eb2d8e..da3a52713231 100644 --- a/datafusion/physical-expr/src/aggregate/utils.rs +++ b/datafusion/physical-expr/src/aggregate/utils.rs @@ -177,14 +177,10 @@ pub fn adjust_output_array( /// for [`AggregateExpr`] aggregation expressions and allows comparing the equality /// between the trait objects. pub fn down_cast_any_ref(any: &dyn Any) -> &dyn Any { - if any.is::>() { - any.downcast_ref::>() - .unwrap() - .as_any() - } else if any.is::>() { - any.downcast_ref::>() - .unwrap() - .as_any() + if let Some(obj) = any.downcast_ref::>() { + obj.as_any() + } else if let Some(obj) = any.downcast_ref::>() { + obj.as_any() } else { any } diff --git a/datafusion/sqllogictest/test_files/groupby.slt b/datafusion/sqllogictest/test_files/groupby.slt index bf93c6633bfc..5cb3ac2f8135 100644 --- a/datafusion/sqllogictest/test_files/groupby.slt +++ b/datafusion/sqllogictest/test_files/groupby.slt @@ -3613,6 +3613,38 @@ AggregateExec: mode=Final, gby=[], aggr=[FIRST_VALUE(foo.x)] ------RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1 --------MemoryExec: partitions=1, partition_sizes=[1] +# Since both ordering requirements are satisfied, there shouldn't be +# any SortExec in the final plan. +query TT +EXPLAIN SELECT FIRST_VALUE(a ORDER BY a ASC) as first_a, + LAST_VALUE(c ORDER BY c DESC) as last_c +FROM multiple_ordered_table +GROUP BY d; +---- +logical_plan +Projection: FIRST_VALUE(multiple_ordered_table.a) ORDER BY [multiple_ordered_table.a ASC NULLS LAST] AS first_a, LAST_VALUE(multiple_ordered_table.c) ORDER BY [multiple_ordered_table.c DESC NULLS FIRST] AS last_c +--Aggregate: groupBy=[[multiple_ordered_table.d]], aggr=[[FIRST_VALUE(multiple_ordered_table.a) ORDER BY [multiple_ordered_table.a ASC NULLS LAST], LAST_VALUE(multiple_ordered_table.c) ORDER BY [multiple_ordered_table.c DESC NULLS FIRST]]] +----TableScan: multiple_ordered_table projection=[a, c, d] +physical_plan +ProjectionExec: expr=[FIRST_VALUE(multiple_ordered_table.a) ORDER BY [multiple_ordered_table.a ASC NULLS LAST]@1 as first_a, LAST_VALUE(multiple_ordered_table.c) ORDER BY [multiple_ordered_table.c DESC NULLS FIRST]@2 as last_c] +--AggregateExec: mode=FinalPartitioned, gby=[d@0 as d], aggr=[FIRST_VALUE(multiple_ordered_table.a), FIRST_VALUE(multiple_ordered_table.c)] +----CoalesceBatchesExec: target_batch_size=2 +------RepartitionExec: partitioning=Hash([d@0], 8), input_partitions=8 +--------AggregateExec: mode=Partial, gby=[d@2 as d], aggr=[FIRST_VALUE(multiple_ordered_table.a), FIRST_VALUE(multiple_ordered_table.c)] +----------RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1 +------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, c, d], output_ordering=[a@0 ASC NULLS LAST], has_header=true + +query II rowsort +SELECT FIRST_VALUE(a ORDER BY a ASC) as first_a, + LAST_VALUE(c ORDER BY c DESC) as last_c +FROM multiple_ordered_table +GROUP BY d; +---- +0 0 +0 1 +0 15 +0 4 +0 9 query TT EXPLAIN SELECT c From ba50a8b178eece7e79b100d0b73bdc9d6d3ec6d5 Mon Sep 17 00:00:00 2001 From: Andrew Lamb Date: Tue, 24 Oct 2023 07:02:49 -0400 Subject: [PATCH 121/572] Minor: provide default implementation for ExecutionPlan::statistics (#7911) * Minor: provide default implementation for ExecutionPlan::statistics * fix: update statistics --- datafusion-examples/examples/custom_datasource.rs | 6 +----- datafusion/core/src/physical_planner.rs | 8 +------- datafusion/core/src/test_util/mod.rs | 6 +----- datafusion/physical-plan/src/analyze.rs | 7 +------ datafusion/physical-plan/src/explain.rs | 7 +------ datafusion/physical-plan/src/insert.rs | 5 ----- datafusion/physical-plan/src/lib.rs | 8 ++++++-- datafusion/physical-plan/src/streaming.rs | 6 +----- datafusion/physical-plan/src/test/exec.rs | 12 ------------ datafusion/physical-plan/src/unnest.rs | 6 +----- 10 files changed, 13 insertions(+), 58 deletions(-) diff --git a/datafusion-examples/examples/custom_datasource.rs b/datafusion-examples/examples/custom_datasource.rs index dd36665a9344..9f25a0b2fa47 100644 --- a/datafusion-examples/examples/custom_datasource.rs +++ b/datafusion-examples/examples/custom_datasource.rs @@ -32,7 +32,7 @@ use datafusion::physical_plan::expressions::PhysicalSortExpr; use datafusion::physical_plan::memory::MemoryStream; use datafusion::physical_plan::{ project_schema, DisplayAs, DisplayFormatType, ExecutionPlan, - SendableRecordBatchStream, Statistics, + SendableRecordBatchStream, }; use datafusion::prelude::*; use datafusion_expr::{Expr, LogicalPlanBuilder}; @@ -270,8 +270,4 @@ impl ExecutionPlan for CustomExec { None, )?)) } - - fn statistics(&self) -> Result { - Ok(Statistics::new_unknown(&self.schema())) - } } diff --git a/datafusion/core/src/physical_planner.rs b/datafusion/core/src/physical_planner.rs index 5a1fdcaee509..419f62cff664 100644 --- a/datafusion/core/src/physical_planner.rs +++ b/datafusion/core/src/physical_planner.rs @@ -2057,9 +2057,7 @@ mod tests { use super::*; use crate::datasource::file_format::options::CsvReadOptions; use crate::datasource::MemTable; - use crate::physical_plan::{ - expressions, DisplayFormatType, Partitioning, Statistics, - }; + use crate::physical_plan::{expressions, DisplayFormatType, Partitioning}; use crate::physical_plan::{DisplayAs, SendableRecordBatchStream}; use crate::physical_planner::PhysicalPlanner; use crate::prelude::{SessionConfig, SessionContext}; @@ -2670,10 +2668,6 @@ mod tests { ) -> Result { unimplemented!("NoOpExecutionPlan::execute"); } - - fn statistics(&self) -> Result { - unimplemented!("NoOpExecutionPlan::statistics"); - } } // Produces an execution plan where the schema is mismatched from diff --git a/datafusion/core/src/test_util/mod.rs b/datafusion/core/src/test_util/mod.rs index d826ec8bfbb6..4fe022f1769d 100644 --- a/datafusion/core/src/test_util/mod.rs +++ b/datafusion/core/src/test_util/mod.rs @@ -40,7 +40,7 @@ use crate::prelude::{CsvReadOptions, SessionContext}; use arrow::datatypes::{DataType, Field, Schema, SchemaRef}; use arrow::record_batch::RecordBatch; -use datafusion_common::{Statistics, TableReference}; +use datafusion_common::TableReference; use datafusion_expr::{CreateExternalTable, Expr, TableType}; use datafusion_physical_expr::PhysicalSortExpr; @@ -238,10 +238,6 @@ impl ExecutionPlan for UnboundedExec { batch: self.batch.clone(), })) } - - fn statistics(&self) -> Result { - Ok(Statistics::new_unknown(&self.schema())) - } } #[derive(Debug)] diff --git a/datafusion/physical-plan/src/analyze.rs b/datafusion/physical-plan/src/analyze.rs index bce242513559..ded37983bb21 100644 --- a/datafusion/physical-plan/src/analyze.rs +++ b/datafusion/physical-plan/src/analyze.rs @@ -25,7 +25,7 @@ use super::stream::{RecordBatchReceiverStream, RecordBatchStreamAdapter}; use super::{DisplayAs, Distribution, SendableRecordBatchStream}; use crate::display::DisplayableExecutionPlan; -use crate::{DisplayFormatType, ExecutionPlan, Partitioning, Statistics}; +use crate::{DisplayFormatType, ExecutionPlan, Partitioning}; use arrow::{array::StringBuilder, datatypes::SchemaRef, record_batch::RecordBatch}; use datafusion_common::{internal_err, DataFusionError, Result}; @@ -195,11 +195,6 @@ impl ExecutionPlan for AnalyzeExec { futures::stream::once(output), ))) } - - fn statistics(&self) -> Result { - // Statistics an an ANALYZE plan are not relevant - Ok(Statistics::new_unknown(&self.schema())) - } } /// Creates the ouput of AnalyzeExec as a RecordBatch diff --git a/datafusion/physical-plan/src/explain.rs b/datafusion/physical-plan/src/explain.rs index 81b8f9944110..e4904ddd3410 100644 --- a/datafusion/physical-plan/src/explain.rs +++ b/datafusion/physical-plan/src/explain.rs @@ -23,7 +23,7 @@ use std::sync::Arc; use super::expressions::PhysicalSortExpr; use super::{DisplayAs, SendableRecordBatchStream}; use crate::stream::RecordBatchStreamAdapter; -use crate::{DisplayFormatType, ExecutionPlan, Partitioning, Statistics}; +use crate::{DisplayFormatType, ExecutionPlan, Partitioning}; use arrow::{array::StringBuilder, datatypes::SchemaRef, record_batch::RecordBatch}; use datafusion_common::display::StringifiedPlan; @@ -167,11 +167,6 @@ impl ExecutionPlan for ExplainExec { futures::stream::iter(vec![Ok(record_batch)]), ))) } - - fn statistics(&self) -> Result { - // Statistics an EXPLAIN plan are not relevant - Ok(Statistics::new_unknown(&self.schema())) - } } /// If this plan should be shown, given the previous plan that was diff --git a/datafusion/physical-plan/src/insert.rs b/datafusion/physical-plan/src/insert.rs index d1f2706930d2..627d58e13781 100644 --- a/datafusion/physical-plan/src/insert.rs +++ b/datafusion/physical-plan/src/insert.rs @@ -25,7 +25,6 @@ use std::sync::Arc; use super::expressions::PhysicalSortExpr; use super::{ DisplayAs, DisplayFormatType, ExecutionPlan, Partitioning, SendableRecordBatchStream, - Statistics, }; use crate::metrics::MetricsSet; use crate::stream::RecordBatchStreamAdapter; @@ -276,10 +275,6 @@ impl ExecutionPlan for FileSinkExec { stream, ))) } - - fn statistics(&self) -> Result { - Ok(Statistics::new_unknown(&self.schema())) - } } /// Create a output record batch with a count diff --git a/datafusion/physical-plan/src/lib.rs b/datafusion/physical-plan/src/lib.rs index d7987ba95abf..b2f81579f8e8 100644 --- a/datafusion/physical-plan/src/lib.rs +++ b/datafusion/physical-plan/src/lib.rs @@ -231,8 +231,12 @@ pub trait ExecutionPlan: Debug + DisplayAs + Send + Sync { None } - /// Returns the global output statistics for this `ExecutionPlan` node. - fn statistics(&self) -> Result; + /// Returns statistics for this `ExecutionPlan` node. If statistics are not + /// available, should return [`Statistics::new_unknown`] (the default), not + /// an error. + fn statistics(&self) -> Result { + Ok(Statistics::new_unknown(&self.schema())) + } } /// Indicate whether a data exchange is needed for the input of `plan`, which will be very helpful diff --git a/datafusion/physical-plan/src/streaming.rs b/datafusion/physical-plan/src/streaming.rs index 7bfa7e2ceefb..27f03b727c29 100644 --- a/datafusion/physical-plan/src/streaming.rs +++ b/datafusion/physical-plan/src/streaming.rs @@ -26,7 +26,7 @@ use crate::stream::RecordBatchStreamAdapter; use crate::{ExecutionPlan, Partitioning, SendableRecordBatchStream}; use arrow::datatypes::SchemaRef; -use datafusion_common::{internal_err, plan_err, DataFusionError, Result, Statistics}; +use datafusion_common::{internal_err, plan_err, DataFusionError, Result}; use datafusion_execution::TaskContext; use datafusion_physical_expr::{LexOrdering, PhysicalSortExpr}; @@ -187,8 +187,4 @@ impl ExecutionPlan for StreamingTableExec { None => stream, }) } - - fn statistics(&self) -> Result { - Ok(Statistics::new_unknown(&self.schema())) - } } diff --git a/datafusion/physical-plan/src/test/exec.rs b/datafusion/physical-plan/src/test/exec.rs index f90f4231c620..71e6cba6741e 100644 --- a/datafusion/physical-plan/src/test/exec.rs +++ b/datafusion/physical-plan/src/test/exec.rs @@ -453,10 +453,6 @@ impl ExecutionPlan for ErrorExec { ) -> Result { internal_err!("ErrorExec, unsurprisingly, errored in partition {partition}") } - - fn statistics(&self) -> Result { - Ok(Statistics::new_unknown(&self.schema())) - } } /// A mock execution plan that simply returns the provided statistics @@ -627,10 +623,6 @@ impl ExecutionPlan for BlockingExec { _refs: Arc::clone(&self.refs), })) } - - fn statistics(&self) -> Result { - unimplemented!() - } } /// A [`RecordBatchStream`] that is pending forever. @@ -764,10 +756,6 @@ impl ExecutionPlan for PanicExec { ready: false, })) } - - fn statistics(&self) -> Result { - unimplemented!() - } } /// A [`RecordBatchStream`] that yields every other batch and panics diff --git a/datafusion/physical-plan/src/unnest.rs b/datafusion/physical-plan/src/unnest.rs index ed64735e5ad6..30f109953cbb 100644 --- a/datafusion/physical-plan/src/unnest.rs +++ b/datafusion/physical-plan/src/unnest.rs @@ -25,7 +25,7 @@ use super::DisplayAs; use crate::{ expressions::Column, DisplayFormatType, Distribution, EquivalenceProperties, ExecutionPlan, Partitioning, PhysicalExpr, PhysicalSortExpr, RecordBatchStream, - SendableRecordBatchStream, Statistics, + SendableRecordBatchStream, }; use arrow::array::{ @@ -159,10 +159,6 @@ impl ExecutionPlan for UnnestExec { unnest_time: 0, })) } - - fn statistics(&self) -> Result { - Ok(Statistics::new_unknown(&self.schema())) - } } /// A stream that issues [RecordBatch]es with unnested column data. From ea5fadea091602ce745aacba7e6197fd6cfc45d9 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Tue, 24 Oct 2023 11:35:30 -0400 Subject: [PATCH 122/572] Update substrait requirement from 0.17.0 to 0.18.0 (#7916) Updates the requirements on [substrait](https://github.com/substrait-io/substrait-rs) to permit the latest version. - [Release notes](https://github.com/substrait-io/substrait-rs/releases) - [Changelog](https://github.com/substrait-io/substrait-rs/blob/main/CHANGELOG.md) - [Commits](https://github.com/substrait-io/substrait-rs/compare/v0.17.0...v0.18.0) --- updated-dependencies: - dependency-name: substrait dependency-type: direct:production ... Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- datafusion/substrait/Cargo.toml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/datafusion/substrait/Cargo.toml b/datafusion/substrait/Cargo.toml index 44a13cdcac37..7c4ff868cfcd 100644 --- a/datafusion/substrait/Cargo.toml +++ b/datafusion/substrait/Cargo.toml @@ -35,7 +35,7 @@ itertools = "0.11" object_store = "0.7.0" prost = "0.12" prost-types = "0.12" -substrait = "0.17.0" +substrait = "0.18.0" tokio = "1.17" [features] From 08c1b69a4ec9e5ddeaefd2810c0624ce9d9a3b9b Mon Sep 17 00:00:00 2001 From: Chih Wang Date: Tue, 24 Oct 2023 23:37:08 +0800 Subject: [PATCH 123/572] Remove unnecessary clone in datafusion_proto (#7921) --- datafusion/proto/src/logical_plan/mod.rs | 8 ++++---- datafusion/proto/src/physical_plan/mod.rs | 19 +++++++------------ 2 files changed, 11 insertions(+), 16 deletions(-) diff --git a/datafusion/proto/src/logical_plan/mod.rs b/datafusion/proto/src/logical_plan/mod.rs index f0999871f568..df76fbb81396 100644 --- a/datafusion/proto/src/logical_plan/mod.rs +++ b/datafusion/proto/src/logical_plan/mod.rs @@ -363,7 +363,7 @@ impl AsLogicalPlan for LogicalPlanNode { .collect::, _>>()?; let options = ListingOptions::new(file_format) - .with_file_extension(scan.file_extension.clone()) + .with_file_extension(&scan.file_extension) .with_table_partition_cols( scan.table_partition_cols .iter() @@ -458,7 +458,7 @@ impl AsLogicalPlan for LogicalPlanNode { let input: LogicalPlan = into_logical_plan!(repartition.input, ctx, extension_codec)?; use protobuf::repartition_node::PartitionMethod; - let pb_partition_method = repartition.partition_method.clone().ok_or_else(|| { + let pb_partition_method = repartition.partition_method.as_ref().ok_or_else(|| { DataFusionError::Internal(String::from( "Protobuf deserialization error, RepartitionNode was missing required field 'partition_method'", )) @@ -473,10 +473,10 @@ impl AsLogicalPlan for LogicalPlanNode { .iter() .map(|expr| from_proto::parse_expr(expr, ctx)) .collect::, _>>()?, - partition_count as usize, + *partition_count as usize, ), PartitionMethod::RoundRobin(partition_count) => { - Partitioning::RoundRobinBatch(partition_count as usize) + Partitioning::RoundRobinBatch(*partition_count as usize) } }; diff --git a/datafusion/proto/src/physical_plan/mod.rs b/datafusion/proto/src/physical_plan/mod.rs index 08010a3151ee..ef870d8ac20b 100644 --- a/datafusion/proto/src/physical_plan/mod.rs +++ b/datafusion/proto/src/physical_plan/mod.rs @@ -394,17 +394,12 @@ impl AsExecutionPlan for PhysicalPlanNode { vec![] }; - let input_schema = hash_agg - .input_schema - .as_ref() - .ok_or_else(|| { - DataFusionError::Internal( - "input_schema in AggregateNode is missing.".to_owned(), - ) - })? - .clone(); - let physical_schema: SchemaRef = - SchemaRef::new((&input_schema).try_into()?); + let input_schema = hash_agg.input_schema.as_ref().ok_or_else(|| { + DataFusionError::Internal( + "input_schema in AggregateNode is missing.".to_owned(), + ) + })?; + let physical_schema: SchemaRef = SchemaRef::new(input_schema.try_into()?); let physical_filter_expr = hash_agg .filter_expr @@ -489,7 +484,7 @@ impl AsExecutionPlan for PhysicalPlanNode { physical_filter_expr, physical_order_by_expr, input, - Arc::new((&input_schema).try_into()?), + Arc::new(input_schema.try_into()?), )?)) } PhysicalPlanType::HashJoin(hashjoin) => { From 10eabd6822821d537e1148d63e44919607d46b87 Mon Sep 17 00:00:00 2001 From: Mustafa Akur <106137913+mustafasrepo@users.noreply.github.com> Date: Tue, 24 Oct 2023 23:14:57 +0300 Subject: [PATCH 124/572] [MINOR]: Simplify code, change requirement from PhysicalSortExpr to PhysicalSortRequirement (#7913) * simplify code, change requirement from PhysicalSortExpr to PhysicalSortRequirement * Remove unnecessary result --- .../enforce_distribution.rs | 11 ++----- .../src/physical_optimizer/enforce_sorting.rs | 16 +++++----- .../replace_with_order_preserving_variants.rs | 18 +++++------ .../src/physical_optimizer/sort_pushdown.rs | 31 +++++++++---------- .../core/src/physical_optimizer/utils.rs | 15 +++++---- 5 files changed, 41 insertions(+), 50 deletions(-) diff --git a/datafusion/core/src/physical_optimizer/enforce_distribution.rs b/datafusion/core/src/physical_optimizer/enforce_distribution.rs index 89036e9f8ccc..9cd7eff4722b 100644 --- a/datafusion/core/src/physical_optimizer/enforce_distribution.rs +++ b/datafusion/core/src/physical_optimizer/enforce_distribution.rs @@ -55,9 +55,7 @@ use datafusion_physical_expr::expressions::{Column, NoOp}; use datafusion_physical_expr::utils::{ map_columns_before_projection, ordering_satisfy_requirement_concrete, }; -use datafusion_physical_expr::{ - expr_list_eq_strict_order, PhysicalExpr, PhysicalSortRequirement, -}; +use datafusion_physical_expr::{expr_list_eq_strict_order, PhysicalExpr}; use datafusion_physical_plan::unbounded_output; use datafusion_physical_plan::windows::{get_best_fitting_window, BoundedWindowAggExec}; @@ -1374,10 +1372,7 @@ fn ensure_distribution( // make sure ordering requirements are still satisfied after. if ordering_satisfied { // Make sure to satisfy ordering requirement: - let sort_expr = PhysicalSortRequirement::to_sort_exprs( - required_input_ordering.clone(), - ); - add_sort_above(&mut child, sort_expr, None)?; + add_sort_above(&mut child, required_input_ordering, None); } } // Stop tracking distribution changing operators @@ -1715,7 +1710,7 @@ mod tests { use datafusion_physical_expr::expressions::{BinaryExpr, Literal}; use datafusion_physical_expr::{ expressions, expressions::binary, expressions::lit, expressions::Column, - LexOrdering, PhysicalExpr, PhysicalSortExpr, + LexOrdering, PhysicalExpr, PhysicalSortExpr, PhysicalSortRequirement, }; /// Models operators like BoundedWindowExec that require an input diff --git a/datafusion/core/src/physical_optimizer/enforce_sorting.rs b/datafusion/core/src/physical_optimizer/enforce_sorting.rs index 92db3bbd053e..913dae07faa1 100644 --- a/datafusion/core/src/physical_optimizer/enforce_sorting.rs +++ b/datafusion/core/src/physical_optimizer/enforce_sorting.rs @@ -399,7 +399,11 @@ fn parallelize_sorts( let mut prev_layer = plan.clone(); update_child_to_remove_coalesce(&mut prev_layer, &mut coalesce_onwards[0])?; let (sort_exprs, fetch) = get_sort_exprs(&plan)?; - add_sort_above(&mut prev_layer, sort_exprs.to_vec(), fetch)?; + add_sort_above( + &mut prev_layer, + &PhysicalSortRequirement::from_sort_exprs(sort_exprs), + fetch, + ); let spm = SortPreservingMergeExec::new(sort_exprs.to_vec(), prev_layer) .with_fetch(fetch); return Ok(Transformed::Yes(PlanWithCorrespondingCoalescePartitions { @@ -456,9 +460,7 @@ fn ensure_sorting( ) { // Make sure we preserve the ordering requirements: update_child_to_remove_unnecessary_sort(child, sort_onwards, &plan)?; - let sort_expr = - PhysicalSortRequirement::to_sort_exprs(required_ordering); - add_sort_above(child, sort_expr, None)?; + add_sort_above(child, &required_ordering, None); if is_sort(child) { *sort_onwards = Some(ExecTree::new(child.clone(), idx, vec![])); } else { @@ -468,8 +470,7 @@ fn ensure_sorting( } (Some(required), None) => { // Ordering requirement is not met, we should add a `SortExec` to the plan. - let sort_expr = PhysicalSortRequirement::to_sort_exprs(required); - add_sort_above(child, sort_expr, None)?; + add_sort_above(child, &required, None); *sort_onwards = Some(ExecTree::new(child.clone(), idx, vec![])); } (None, Some(_)) => { @@ -603,9 +604,8 @@ fn analyze_window_sort_removal( .required_input_ordering() .swap_remove(0) .unwrap_or_default(); - let sort_expr = PhysicalSortRequirement::to_sort_exprs(reqs); // Satisfy the ordering requirement so that the window can run: - add_sort_above(&mut window_child, sort_expr, None)?; + add_sort_above(&mut window_child, &reqs, None); let uses_bounded_memory = window_expr.iter().all(|e| e.uses_bounded_memory()); let new_window = if uses_bounded_memory { diff --git a/datafusion/core/src/physical_optimizer/replace_with_order_preserving_variants.rs b/datafusion/core/src/physical_optimizer/replace_with_order_preserving_variants.rs index fb1a50e18d6f..fb75c083a70a 100644 --- a/datafusion/core/src/physical_optimizer/replace_with_order_preserving_variants.rs +++ b/datafusion/core/src/physical_optimizer/replace_with_order_preserving_variants.rs @@ -71,14 +71,15 @@ impl OrderPreservationContext { // ordering, (or that can maintain ordering with the replacement of // its variant) let plan = item.plan; + let children = plan.children(); let ordering_onwards = item.ordering_onwards; - if plan.children().is_empty() { + if children.is_empty() { // Plan has no children, there is nothing to propagate. None } else if ordering_onwards[0].is_none() && ((is_repartition(&plan) && !plan.maintains_input_order()[0]) || (is_coalesce_partitions(&plan) - && plan.children()[0].output_ordering().is_some())) + && children[0].output_ordering().is_some())) { Some(ExecTree::new(plan, idx, vec![])) } else { @@ -175,19 +176,18 @@ fn get_updated_plan( // When a `RepartitionExec` doesn't preserve ordering, replace it with // a `SortPreservingRepartitionExec` if appropriate: if is_repartition(&plan) && !plan.maintains_input_order()[0] && is_spr_better { - let child = plan.children()[0].clone(); - plan = Arc::new( - RepartitionExec::try_new(child, plan.output_partitioning())? - .with_preserve_order(true), - ) as _ + let child = plan.children().swap_remove(0); + let repartition = RepartitionExec::try_new(child, plan.output_partitioning())?; + plan = Arc::new(repartition.with_preserve_order(true)) as _ } // When the input of a `CoalescePartitionsExec` has an ordering, replace it // with a `SortPreservingMergeExec` if appropriate: + let mut children = plan.children(); if is_coalesce_partitions(&plan) - && plan.children()[0].output_ordering().is_some() + && children[0].output_ordering().is_some() && is_spm_better { - let child = plan.children()[0].clone(); + let child = children.swap_remove(0); plan = Arc::new(SortPreservingMergeExec::new( child.output_ordering().unwrap_or(&[]).to_vec(), child, diff --git a/datafusion/core/src/physical_optimizer/sort_pushdown.rs b/datafusion/core/src/physical_optimizer/sort_pushdown.rs index 9b81ad3efb50..808c4a3dadae 100644 --- a/datafusion/core/src/physical_optimizer/sort_pushdown.rs +++ b/datafusion/core/src/physical_optimizer/sort_pushdown.rs @@ -139,23 +139,21 @@ pub(crate) fn pushdown_sorts( || plan.ordering_equivalence_properties(), ) { // If the current plan is a SortExec, modify it to satisfy parent requirements: - let parent_required_expr = PhysicalSortRequirement::to_sort_exprs( - parent_required.ok_or_else(err)?.iter().cloned(), - ); + let parent_required_expr = parent_required.ok_or_else(err)?; new_plan = sort_exec.input().clone(); - add_sort_above(&mut new_plan, parent_required_expr, sort_exec.fetch())?; + add_sort_above(&mut new_plan, parent_required_expr, sort_exec.fetch()); }; let required_ordering = new_plan .output_ordering() .map(PhysicalSortRequirement::from_sort_exprs); // Since new_plan is a SortExec, we can safely get the 0th index. - let child = &new_plan.children()[0]; + let child = new_plan.children().swap_remove(0); if let Some(adjusted) = - pushdown_requirement_to_children(child, required_ordering.as_deref())? + pushdown_requirement_to_children(&child, required_ordering.as_deref())? { // Can push down requirements Ok(Transformed::Yes(SortPushDown { - plan: child.clone(), + plan: child, required_ordering: None, adjusted_request_ordering: adjusted, })) @@ -180,17 +178,15 @@ pub(crate) fn pushdown_sorts( // Can not satisfy the parent requirements, check whether the requirements can be pushed down: if let Some(adjusted) = pushdown_requirement_to_children(plan, parent_required)? { Ok(Transformed::Yes(SortPushDown { - plan: plan.clone(), + plan: requirements.plan, required_ordering: None, adjusted_request_ordering: adjusted, })) } else { // Can not push down requirements, add new SortExec: - let parent_required_expr = PhysicalSortRequirement::to_sort_exprs( - parent_required.ok_or_else(err)?.iter().cloned(), - ); - let mut new_plan = plan.clone(); - add_sort_above(&mut new_plan, parent_required_expr, None)?; + let parent_required_expr = parent_required.ok_or_else(err)?; + let mut new_plan = requirements.plan; + add_sort_above(&mut new_plan, parent_required_expr, None); Ok(Transformed::Yes(SortPushDown::init(new_plan))) } } @@ -206,7 +202,7 @@ fn pushdown_requirement_to_children( if is_window(plan) { let required_input_ordering = plan.required_input_ordering(); let request_child = required_input_ordering[0].as_deref(); - let child_plan = plan.children()[0].clone(); + let child_plan = plan.children().swap_remove(0); match determine_children_requirement(parent_required, request_child, child_plan) { RequirementsCompatibility::Satisfy => { Ok(Some(vec![request_child.map(|r| r.to_vec())])) @@ -355,16 +351,17 @@ fn try_pushdown_requirements_to_join( || smj.ordering_equivalence_properties(), ) .then(|| { - let required_input_ordering = smj.required_input_ordering(); + let mut required_input_ordering = smj.required_input_ordering(); let new_req = Some(PhysicalSortRequirement::from_sort_exprs(&sort_expr)); match push_side { JoinSide::Left => { - vec![new_req, required_input_ordering[1].clone()] + required_input_ordering[0] = new_req; } JoinSide::Right => { - vec![required_input_ordering[0].clone(), new_req] + required_input_ordering[1] = new_req; } } + required_input_ordering })) } diff --git a/datafusion/core/src/physical_optimizer/utils.rs b/datafusion/core/src/physical_optimizer/utils.rs index 0d6c85f9f22b..403af4b16ec7 100644 --- a/datafusion/core/src/physical_optimizer/utils.rs +++ b/datafusion/core/src/physical_optimizer/utils.rs @@ -21,7 +21,6 @@ use std::fmt; use std::fmt::Formatter; use std::sync::Arc; -use crate::error::Result; use crate::physical_plan::coalesce_partitions::CoalescePartitionsExec; use crate::physical_plan::limit::{GlobalLimitExec, LocalLimitExec}; use crate::physical_plan::repartition::RepartitionExec; @@ -31,8 +30,8 @@ use crate::physical_plan::union::UnionExec; use crate::physical_plan::windows::{BoundedWindowAggExec, WindowAggExec}; use crate::physical_plan::{displayable, ExecutionPlan}; -use datafusion_physical_expr::utils::ordering_satisfy; -use datafusion_physical_expr::PhysicalSortExpr; +use datafusion_physical_expr::utils::ordering_satisfy_requirement; +use datafusion_physical_expr::PhysicalSortRequirement; /// This object implements a tree that we use while keeping track of paths /// leading to [`SortExec`]s. @@ -101,16 +100,17 @@ pub(crate) fn get_children_exectrees( /// given ordering requirements while preserving the original partitioning. pub fn add_sort_above( node: &mut Arc, - sort_expr: Vec, + sort_requirement: &[PhysicalSortRequirement], fetch: Option, -) -> Result<()> { +) { // If the ordering requirement is already satisfied, do not add a sort. - if !ordering_satisfy( + if !ordering_satisfy_requirement( node.output_ordering(), - Some(&sort_expr), + Some(sort_requirement), || node.equivalence_properties(), || node.ordering_equivalence_properties(), ) { + let sort_expr = PhysicalSortRequirement::to_sort_exprs(sort_requirement.to_vec()); let new_sort = SortExec::new(sort_expr, node.clone()).with_fetch(fetch); *node = Arc::new(if node.output_partitioning().partition_count() > 1 { @@ -119,7 +119,6 @@ pub fn add_sort_above( new_sort }) as _ } - Ok(()) } /// Checks whether the given operator is a limit; From 8068d7f0779bc9d5fddff35de33df8779f67cd0c Mon Sep 17 00:00:00 2001 From: Mustafa Akur <106137913+mustafasrepo@users.noreply.github.com> Date: Tue, 24 Oct 2023 23:42:33 +0300 Subject: [PATCH 125/572] [Minor] Move combine_join util to under equivalence.rs (#7917) * Move combine join to equivalences.rs file * Resolve linter errors * Simplifications, do not return Error in add_offset --- .../src/physical_optimizer/sort_pushdown.rs | 2 +- datafusion/physical-expr/src/equivalence.rs | 338 +++++++++++++++++- datafusion/physical-expr/src/lib.rs | 8 +- .../physical-plan/src/joins/cross_join.rs | 4 +- .../physical-plan/src/joins/hash_join.rs | 19 +- .../src/joins/nested_loop_join.rs | 10 +- .../src/joins/sort_merge_join.rs | 11 +- .../src/joins/symmetric_hash_join.rs | 5 +- datafusion/physical-plan/src/joins/utils.rs | 338 +----------------- 9 files changed, 368 insertions(+), 367 deletions(-) diff --git a/datafusion/core/src/physical_optimizer/sort_pushdown.rs b/datafusion/core/src/physical_optimizer/sort_pushdown.rs index 808c4a3dadae..a99399592f15 100644 --- a/datafusion/core/src/physical_optimizer/sort_pushdown.rs +++ b/datafusion/core/src/physical_optimizer/sort_pushdown.rs @@ -343,7 +343,7 @@ fn try_pushdown_requirements_to_join( smj.left().schema().fields.len(), &smj.maintains_input_order(), Some(SortMergeJoinExec::probe_side(&smj.join_type())), - )?; + ); Ok(ordering_satisfy_requirement( new_output_ordering.as_deref(), parent_required, diff --git a/datafusion/physical-expr/src/equivalence.rs b/datafusion/physical-expr/src/equivalence.rs index 369c139aa30b..4fce6854138d 100644 --- a/datafusion/physical-expr/src/equivalence.rs +++ b/datafusion/physical-expr/src/equivalence.rs @@ -26,7 +26,7 @@ use arrow::datatypes::SchemaRef; use arrow_schema::Fields; use datafusion_common::tree_node::{Transformed, TreeNode}; -use datafusion_common::Result; +use datafusion_common::{JoinSide, JoinType}; use itertools::izip; use std::collections::{HashMap, HashSet}; use std::hash::Hash; @@ -463,14 +463,14 @@ impl OrderingEquivalentClass { } /// Adds `offset` value to the index of each expression inside `self.head` and `self.others`. - pub fn add_offset(&self, offset: usize) -> Result { - let head = add_offset_to_lex_ordering(self.head(), offset)?; + pub fn add_offset(&self, offset: usize) -> OrderingEquivalentClass { + let head = add_offset_to_lex_ordering(self.head(), offset); let others = self .others() .iter() .map(|ordering| add_offset_to_lex_ordering(ordering, offset)) - .collect::>>()?; - Ok(OrderingEquivalentClass::new(head, others)) + .collect::>(); + OrderingEquivalentClass::new(head, others) } /// This function normalizes `OrderingEquivalenceProperties` according to `eq_properties`. @@ -885,6 +885,239 @@ fn req_satisfied(given: LexOrderingRef, req: &[PhysicalSortRequirement]) -> bool true } +/// Combine equivalence properties of the given join inputs. +pub fn combine_join_equivalence_properties( + join_type: JoinType, + left_properties: EquivalenceProperties, + right_properties: EquivalenceProperties, + left_columns_len: usize, + on: &[(Column, Column)], + schema: SchemaRef, +) -> EquivalenceProperties { + let mut new_properties = EquivalenceProperties::new(schema); + match join_type { + JoinType::Inner | JoinType::Left | JoinType::Full | JoinType::Right => { + new_properties.extend(left_properties.classes().to_vec()); + let new_right_properties = right_properties + .classes() + .iter() + .map(|prop| { + let new_head = Column::new( + prop.head().name(), + left_columns_len + prop.head().index(), + ); + let new_others = prop + .others() + .iter() + .map(|col| { + Column::new(col.name(), left_columns_len + col.index()) + }) + .collect::>(); + EquivalentClass::new(new_head, new_others) + }) + .collect::>(); + + new_properties.extend(new_right_properties); + } + JoinType::LeftSemi | JoinType::LeftAnti => { + new_properties.extend(left_properties.classes().to_vec()) + } + JoinType::RightSemi | JoinType::RightAnti => { + new_properties.extend(right_properties.classes().to_vec()) + } + } + + if join_type == JoinType::Inner { + on.iter().for_each(|(column1, column2)| { + let new_column2 = + Column::new(column2.name(), left_columns_len + column2.index()); + new_properties.add_equal_conditions((column1, &new_column2)) + }) + } + new_properties +} + +/// Calculate equivalence properties for the given cross join operation. +pub fn cross_join_equivalence_properties( + left_properties: EquivalenceProperties, + right_properties: EquivalenceProperties, + left_columns_len: usize, + schema: SchemaRef, +) -> EquivalenceProperties { + let mut new_properties = EquivalenceProperties::new(schema); + new_properties.extend(left_properties.classes().to_vec()); + let new_right_properties = right_properties + .classes() + .iter() + .map(|prop| { + let new_head = + Column::new(prop.head().name(), left_columns_len + prop.head().index()); + let new_others = prop + .others() + .iter() + .map(|col| Column::new(col.name(), left_columns_len + col.index())) + .collect::>(); + EquivalentClass::new(new_head, new_others) + }) + .collect::>(); + new_properties.extend(new_right_properties); + new_properties +} + +/// Update right table ordering equivalences so that: +/// - They point to valid indices at the output of the join schema, and +/// - They are normalized with respect to equivalence columns. +/// +/// To do so, we increment column indices by the size of the left table when +/// join schema consists of a combination of left and right schema (Inner, +/// Left, Full, Right joins). Then, we normalize the sort expressions of +/// ordering equivalences one by one. We make sure that each expression in the +/// ordering equivalence is either: +/// - The head of the one of the equivalent classes, or +/// - Doesn't have an equivalent column. +/// +/// This way; once we normalize an expression according to equivalence properties, +/// it can thereafter safely be used for ordering equivalence normalization. +fn get_updated_right_ordering_equivalent_class( + join_type: &JoinType, + right_oeq_class: &OrderingEquivalentClass, + left_columns_len: usize, + join_eq_properties: &EquivalenceProperties, +) -> OrderingEquivalentClass { + match join_type { + // In these modes, indices of the right schema should be offset by + // the left table size. + JoinType::Inner | JoinType::Left | JoinType::Full | JoinType::Right => { + let right_oeq_class = right_oeq_class.add_offset(left_columns_len); + return right_oeq_class + .normalize_with_equivalence_properties(join_eq_properties); + } + _ => {} + }; + right_oeq_class.normalize_with_equivalence_properties(join_eq_properties) +} + +/// Calculate ordering equivalence properties for the given join operation. +pub fn combine_join_ordering_equivalence_properties( + join_type: &JoinType, + left_oeq_properties: &OrderingEquivalenceProperties, + right_oeq_properties: &OrderingEquivalenceProperties, + schema: SchemaRef, + maintains_input_order: &[bool], + probe_side: Option, + join_eq_properties: EquivalenceProperties, +) -> OrderingEquivalenceProperties { + let mut new_properties = OrderingEquivalenceProperties::new(schema); + let left_columns_len = left_oeq_properties.schema().fields().len(); + // All joins have 2 children + assert_eq!(maintains_input_order.len(), 2); + let left_maintains = maintains_input_order[0]; + let right_maintains = maintains_input_order[1]; + match (left_maintains, right_maintains) { + (true, true) => { + unreachable!("Cannot maintain ordering of both sides"); + } + (true, false) => { + // In this special case, right side ordering can be prefixed with left side ordering. + if let ( + Some(JoinSide::Left), + JoinType::Inner, + Some(left_oeq_class), + Some(right_oeq_class), + ) = ( + probe_side, + join_type, + left_oeq_properties.oeq_class(), + right_oeq_properties.oeq_class(), + ) { + let updated_right_oeq = get_updated_right_ordering_equivalent_class( + join_type, + right_oeq_class, + left_columns_len, + &join_eq_properties, + ); + + // Right side ordering equivalence properties should be prepended with + // those of the left side while constructing output ordering equivalence + // properties since stream side is the left side. + // + // If the right table ordering equivalences contain `b ASC`, and the output + // ordering of the left table is `a ASC`, then the ordering equivalence `b ASC` + // for the right table should be converted to `a ASC, b ASC` before it is added + // to the ordering equivalences of the join. + let mut orderings = vec![]; + for left_ordering in left_oeq_class.iter() { + for right_ordering in updated_right_oeq.iter() { + let mut ordering = left_ordering.to_vec(); + ordering.extend(right_ordering.to_vec()); + let ordering_normalized = + join_eq_properties.normalize_sort_exprs(&ordering); + orderings.push(ordering_normalized); + } + } + if !orderings.is_empty() { + let head = orderings.swap_remove(0); + let new_oeq_class = OrderingEquivalentClass::new(head, orderings); + new_properties.extend(Some(new_oeq_class)); + } + } else { + new_properties.extend(left_oeq_properties.oeq_class().cloned()); + } + } + (false, true) => { + let updated_right_oeq = + right_oeq_properties.oeq_class().map(|right_oeq_class| { + get_updated_right_ordering_equivalent_class( + join_type, + right_oeq_class, + left_columns_len, + &join_eq_properties, + ) + }); + // In this special case, left side ordering can be prefixed with right side ordering. + if let ( + Some(JoinSide::Right), + JoinType::Inner, + Some(left_oeq_class), + Some(right_oeg_class), + ) = ( + probe_side, + join_type, + left_oeq_properties.oeq_class(), + &updated_right_oeq, + ) { + // Left side ordering equivalence properties should be prepended with + // those of the right side while constructing output ordering equivalence + // properties since stream side is the right side. + // + // If the right table ordering equivalences contain `b ASC`, and the output + // ordering of the left table is `a ASC`, then the ordering equivalence `b ASC` + // for the right table should be converted to `a ASC, b ASC` before it is added + // to the ordering equivalences of the join. + let mut orderings = vec![]; + for right_ordering in right_oeg_class.iter() { + for left_ordering in left_oeq_class.iter() { + let mut ordering = right_ordering.to_vec(); + ordering.extend(left_ordering.to_vec()); + let ordering_normalized = + join_eq_properties.normalize_sort_exprs(&ordering); + orderings.push(ordering_normalized); + } + } + if !orderings.is_empty() { + let head = orderings.swap_remove(0); + let new_oeq_class = OrderingEquivalentClass::new(head, orderings); + new_properties.extend(Some(new_oeq_class)); + } + } else { + new_properties.extend(updated_right_oeq); + } + } + (false, false) => {} + } + new_properties +} + /// This function searches for the slice `section` inside the slice `given`. /// It returns each range where `section` is compatible with the corresponding /// slice in `given`. @@ -935,10 +1168,10 @@ fn prune_sort_reqs_with_constants( /// Adds the `offset` value to `Column` indices inside `expr`. This function is /// generally used during the update of the right table schema in join operations. -pub(crate) fn add_offset_to_expr( +pub fn add_offset_to_expr( expr: Arc, offset: usize, -) -> Result> { +) -> Arc { expr.transform_down(&|e| match e.as_any().downcast_ref::() { Some(col) => Ok(Transformed::Yes(Arc::new(Column::new( col.name(), @@ -946,17 +1179,20 @@ pub(crate) fn add_offset_to_expr( )))), None => Ok(Transformed::No(e)), }) + .unwrap() + // Note that we can safely unwrap here since our transform always returns + // an `Ok` value. } /// Adds the `offset` value to `Column` indices inside `sort_expr.expr`. pub(crate) fn add_offset_to_sort_expr( sort_expr: &PhysicalSortExpr, offset: usize, -) -> Result { - Ok(PhysicalSortExpr { - expr: add_offset_to_expr(sort_expr.expr.clone(), offset)?, +) -> PhysicalSortExpr { + PhysicalSortExpr { + expr: add_offset_to_expr(sort_expr.expr.clone(), offset), options: sort_expr.options, - }) + } } /// Adds the `offset` value to `Column` indices for each `sort_expr.expr` @@ -964,7 +1200,7 @@ pub(crate) fn add_offset_to_sort_expr( pub fn add_offset_to_lex_ordering( sort_exprs: LexOrderingRef, offset: usize, -) -> Result { +) -> LexOrdering { sort_exprs .iter() .map(|sort_expr| add_offset_to_sort_expr(sort_expr, offset)) @@ -1131,4 +1367,82 @@ mod tests { } Ok(()) } + + #[test] + fn test_get_updated_right_ordering_equivalence_properties() -> Result<()> { + let join_type = JoinType::Inner; + + let options = SortOptions::default(); + let right_oeq_class = OrderingEquivalentClass::new( + vec![ + PhysicalSortExpr { + expr: Arc::new(Column::new("x", 0)), + options, + }, + PhysicalSortExpr { + expr: Arc::new(Column::new("y", 1)), + options, + }, + ], + vec![vec![ + PhysicalSortExpr { + expr: Arc::new(Column::new("z", 2)), + options, + }, + PhysicalSortExpr { + expr: Arc::new(Column::new("w", 3)), + options, + }, + ]], + ); + + let left_columns_len = 4; + + let fields: Fields = ["a", "b", "c", "d", "x", "y", "z", "w"] + .into_iter() + .map(|name| Field::new(name, DataType::Int32, true)) + .collect(); + + let mut join_eq_properties = + EquivalenceProperties::new(Arc::new(Schema::new(fields))); + join_eq_properties + .add_equal_conditions((&Column::new("a", 0), &Column::new("x", 4))); + join_eq_properties + .add_equal_conditions((&Column::new("d", 3), &Column::new("w", 7))); + + let result = get_updated_right_ordering_equivalent_class( + &join_type, + &right_oeq_class, + left_columns_len, + &join_eq_properties, + ); + + let expected = OrderingEquivalentClass::new( + vec![ + PhysicalSortExpr { + expr: Arc::new(Column::new("a", 0)), + options, + }, + PhysicalSortExpr { + expr: Arc::new(Column::new("y", 5)), + options, + }, + ], + vec![vec![ + PhysicalSortExpr { + expr: Arc::new(Column::new("z", 6)), + options, + }, + PhysicalSortExpr { + expr: Arc::new(Column::new("d", 3)), + options, + }, + ]], + ); + + assert_eq!(result.head(), expected.head()); + assert_eq!(result.others(), expected.others()); + + Ok(()) + } } diff --git a/datafusion/physical-expr/src/lib.rs b/datafusion/physical-expr/src/lib.rs index e670380e59d2..977542bd8e66 100644 --- a/datafusion/physical-expr/src/lib.rs +++ b/datafusion/physical-expr/src/lib.rs @@ -54,10 +54,10 @@ pub use aggregate::groups_accumulator::{ pub use aggregate::AggregateExpr; pub use analysis::{analyze, AnalysisContext, ExprBoundaries}; pub use equivalence::{ - add_offset_to_lex_ordering, ordering_equivalence_properties_helper, - project_equivalence_properties, project_ordering_equivalence_properties, - EquivalenceProperties, EquivalentClass, OrderingEquivalenceProperties, - OrderingEquivalentClass, + add_offset_to_expr, add_offset_to_lex_ordering, + ordering_equivalence_properties_helper, project_equivalence_properties, + project_ordering_equivalence_properties, EquivalenceProperties, EquivalentClass, + OrderingEquivalenceProperties, OrderingEquivalentClass, }; pub use partitioning::{Distribution, Partitioning}; diff --git a/datafusion/physical-plan/src/joins/cross_join.rs b/datafusion/physical-plan/src/joins/cross_join.rs index 1ffd9ad1c18a..c9c92e8c11ae 100644 --- a/datafusion/physical-plan/src/joins/cross_join.rs +++ b/datafusion/physical-plan/src/joins/cross_join.rs @@ -21,8 +21,7 @@ use std::{any::Any, sync::Arc, task::Poll}; use super::utils::{ - adjust_right_output_partitioning, cross_join_equivalence_properties, - BuildProbeJoinMetrics, OnceAsync, OnceFut, + adjust_right_output_partitioning, BuildProbeJoinMetrics, OnceAsync, OnceFut, }; use crate::metrics::{ExecutionPlanMetricsSet, MetricsSet}; use crate::DisplayAs; @@ -41,6 +40,7 @@ use datafusion_execution::memory_pool::{MemoryConsumer, MemoryReservation}; use datafusion_execution::TaskContext; use async_trait::async_trait; +use datafusion_physical_expr::equivalence::cross_join_equivalence_properties; use futures::{ready, StreamExt}; use futures::{Stream, TryStreamExt}; diff --git a/datafusion/physical-plan/src/joins/hash_join.rs b/datafusion/physical-plan/src/joins/hash_join.rs index 2ffa1f61a23b..9aa776fe054c 100644 --- a/datafusion/physical-plan/src/joins/hash_join.rs +++ b/datafusion/physical-plan/src/joins/hash_join.rs @@ -26,8 +26,8 @@ use std::{any::Any, usize, vec}; use crate::joins::utils::{ adjust_indices_by_join_type, apply_join_filter_to_indices, build_batch_from_indices, - calculate_join_output_ordering, combine_join_ordering_equivalence_properties, - get_final_indices_from_bit_map, need_produce_result_in_final, + calculate_join_output_ordering, get_final_indices_from_bit_map, + need_produce_result_in_final, }; use crate::DisplayAs; use crate::{ @@ -39,9 +39,8 @@ use crate::{ joins::hash_join_utils::{JoinHashMap, JoinHashMapType}, joins::utils::{ adjust_right_output_partitioning, build_join_schema, check_join_is_valid, - combine_join_equivalence_properties, estimate_join_statistics, - partitioned_join_output_partitioning, BuildProbeJoinMetrics, ColumnIndex, - JoinFilter, JoinOn, + estimate_join_statistics, partitioned_join_output_partitioning, + BuildProbeJoinMetrics, ColumnIndex, JoinFilter, JoinOn, }, metrics::{ExecutionPlanMetricsSet, MetricsSet}, DisplayFormatType, Distribution, EquivalenceProperties, ExecutionPlan, Partitioning, @@ -72,6 +71,9 @@ use datafusion_physical_expr::OrderingEquivalenceProperties; use ahash::RandomState; use arrow::compute::kernels::cmp::{eq, not_distinct}; +use datafusion_physical_expr::equivalence::{ + combine_join_equivalence_properties, combine_join_ordering_equivalence_properties, +}; use futures::{ready, Stream, StreamExt, TryStreamExt}; type JoinLeftData = (JoinHashMap, RecordBatch, MemoryReservation); @@ -146,7 +148,7 @@ impl HashJoinExec { left_schema.fields.len(), &Self::maintains_input_order(*join_type), Some(Self::probe_side()), - )?; + ); Ok(HashJoinExec { left, @@ -380,14 +382,13 @@ impl ExecutionPlan for HashJoinExec { fn ordering_equivalence_properties(&self) -> OrderingEquivalenceProperties { combine_join_ordering_equivalence_properties( &self.join_type, - &self.left, - &self.right, + &self.left.ordering_equivalence_properties(), + &self.right.ordering_equivalence_properties(), self.schema(), &self.maintains_input_order(), Some(Self::probe_side()), self.equivalence_properties(), ) - .unwrap() } fn children(&self) -> Vec> { diff --git a/datafusion/physical-plan/src/joins/nested_loop_join.rs b/datafusion/physical-plan/src/joins/nested_loop_join.rs index 25cb374e941f..a113066e39d1 100644 --- a/datafusion/physical-plan/src/joins/nested_loop_join.rs +++ b/datafusion/physical-plan/src/joins/nested_loop_join.rs @@ -27,11 +27,10 @@ use std::task::Poll; use crate::coalesce_batches::concat_batches; use crate::joins::utils::{ append_right_indices, apply_join_filter_to_indices, build_batch_from_indices, - build_join_schema, check_join_is_valid, combine_join_equivalence_properties, - estimate_join_statistics, get_anti_indices, get_anti_u64_indices, - get_final_indices_from_bit_map, get_semi_indices, get_semi_u64_indices, - partitioned_join_output_partitioning, BuildProbeJoinMetrics, ColumnIndex, JoinFilter, - OnceAsync, OnceFut, + build_join_schema, check_join_is_valid, estimate_join_statistics, get_anti_indices, + get_anti_u64_indices, get_final_indices_from_bit_map, get_semi_indices, + get_semi_u64_indices, partitioned_join_output_partitioning, BuildProbeJoinMetrics, + ColumnIndex, JoinFilter, OnceAsync, OnceFut, }; use crate::metrics::{ExecutionPlanMetricsSet, MetricsSet}; use crate::{ @@ -51,6 +50,7 @@ use datafusion_execution::TaskContext; use datafusion_expr::JoinType; use datafusion_physical_expr::{EquivalenceProperties, PhysicalSortExpr}; +use datafusion_physical_expr::equivalence::combine_join_equivalence_properties; use futures::{ready, Stream, StreamExt, TryStreamExt}; /// Data of the inner table side diff --git a/datafusion/physical-plan/src/joins/sort_merge_join.rs b/datafusion/physical-plan/src/joins/sort_merge_join.rs index 98fe751b22ef..759149a64d9f 100644 --- a/datafusion/physical-plan/src/joins/sort_merge_join.rs +++ b/datafusion/physical-plan/src/joins/sort_merge_join.rs @@ -33,7 +33,6 @@ use std::task::{Context, Poll}; use crate::expressions::{Column, PhysicalSortExpr}; use crate::joins::utils::{ build_join_schema, calculate_join_output_ordering, check_join_is_valid, - combine_join_equivalence_properties, combine_join_ordering_equivalence_properties, estimate_join_statistics, partitioned_join_output_partitioning, JoinOn, }; use crate::metrics::{ExecutionPlanMetricsSet, MetricBuilder, MetricsSet}; @@ -55,6 +54,9 @@ use datafusion_execution::memory_pool::{MemoryConsumer, MemoryReservation}; use datafusion_execution::TaskContext; use datafusion_physical_expr::{OrderingEquivalenceProperties, PhysicalSortRequirement}; +use datafusion_physical_expr::equivalence::{ + combine_join_equivalence_properties, combine_join_ordering_equivalence_properties, +}; use futures::{Stream, StreamExt}; /// join execution plan executes partitions in parallel and combines them into a set of @@ -140,7 +142,7 @@ impl SortMergeJoinExec { left_schema.fields.len(), &Self::maintains_input_order(join_type), Some(Self::probe_side(&join_type)), - )?; + ); let schema = Arc::new(build_join_schema(&left_schema, &right_schema, &join_type).0); @@ -297,14 +299,13 @@ impl ExecutionPlan for SortMergeJoinExec { fn ordering_equivalence_properties(&self) -> OrderingEquivalenceProperties { combine_join_ordering_equivalence_properties( &self.join_type, - &self.left, - &self.right, + &self.left.ordering_equivalence_properties(), + &self.right.ordering_equivalence_properties(), self.schema(), &self.maintains_input_order(), Some(Self::probe_side(&self.join_type)), self.equivalence_properties(), ) - .unwrap() } fn children(&self) -> Vec> { diff --git a/datafusion/physical-plan/src/joins/symmetric_hash_join.rs b/datafusion/physical-plan/src/joins/symmetric_hash_join.rs index 3450331133bd..00d43aead434 100644 --- a/datafusion/physical-plan/src/joins/symmetric_hash_join.rs +++ b/datafusion/physical-plan/src/joins/symmetric_hash_join.rs @@ -42,8 +42,8 @@ use crate::joins::hash_join_utils::{ }; use crate::joins::utils::{ build_batch_from_indices, build_join_schema, check_join_is_valid, - combine_join_equivalence_properties, partitioned_join_output_partitioning, - prepare_sorted_exprs, ColumnIndex, JoinFilter, JoinOn, + partitioned_join_output_partitioning, prepare_sorted_exprs, ColumnIndex, JoinFilter, + JoinOn, }; use crate::{ expressions::{Column, PhysicalSortExpr}, @@ -66,6 +66,7 @@ use datafusion_execution::TaskContext; use datafusion_physical_expr::intervals::ExprIntervalGraph; use ahash::RandomState; +use datafusion_physical_expr::equivalence::combine_join_equivalence_properties; use futures::stream::{select, BoxStream}; use futures::{Stream, StreamExt}; use hashbrown::HashSet; diff --git a/datafusion/physical-plan/src/joins/utils.rs b/datafusion/physical-plan/src/joins/utils.rs index afde986c0bb6..cf150ddf575f 100644 --- a/datafusion/physical-plan/src/joins/utils.rs +++ b/datafusion/physical-plan/src/joins/utils.rs @@ -25,10 +25,7 @@ use std::usize; use crate::joins::hash_join_utils::{build_filter_input_order, SortedFilterExpr}; use crate::metrics::{self, ExecutionPlanMetricsSet, MetricBuilder}; -use crate::{ - ColumnStatistics, EquivalenceProperties, ExecutionPlan, Partitioning, SchemaRef, - Statistics, -}; +use crate::{ColumnStatistics, ExecutionPlan, Partitioning, Statistics}; use arrow::array::{ downcast_array, new_null_array, Array, BooleanBufferBuilder, UInt32Array, @@ -39,18 +36,16 @@ use arrow::datatypes::{Field, Schema, SchemaBuilder}; use arrow::record_batch::{RecordBatch, RecordBatchOptions}; use datafusion_common::cast::as_boolean_array; use datafusion_common::stats::Precision; -use datafusion_common::tree_node::{Transformed, TreeNode}; use datafusion_common::{ - exec_err, plan_datafusion_err, plan_err, DataFusionError, JoinSide, JoinType, Result, + plan_datafusion_err, plan_err, DataFusionError, JoinSide, JoinType, Result, SharedResult, }; use datafusion_physical_expr::expressions::Column; use datafusion_physical_expr::intervals::{ExprIntervalGraph, Interval, IntervalBound}; use datafusion_physical_expr::utils::merge_vectors; use datafusion_physical_expr::{ - add_offset_to_lex_ordering, EquivalentClass, LexOrdering, LexOrderingRef, - OrderingEquivalenceProperties, OrderingEquivalentClass, PhysicalExpr, - PhysicalSortExpr, + add_offset_to_expr, add_offset_to_lex_ordering, LexOrdering, LexOrderingRef, + PhysicalExpr, PhysicalSortExpr, }; use futures::future::{BoxFuture, Shared}; @@ -137,16 +132,7 @@ pub fn adjust_right_output_partitioning( Partitioning::Hash(exprs, size) => { let new_exprs = exprs .into_iter() - .map(|expr| { - expr.transform_down(&|e| match e.as_any().downcast_ref::() { - Some(col) => Ok(Transformed::Yes(Arc::new(Column::new( - col.name(), - left_columns_len + col.index(), - )))), - None => Ok(Transformed::No(e)), - }) - .unwrap() - }) + .map(|expr| add_offset_to_expr(expr, left_columns_len)) .collect::>(); Partitioning::Hash(new_exprs, size) } @@ -182,7 +168,7 @@ pub fn calculate_join_output_ordering( left_columns_len: usize, maintains_input_order: &[bool], probe_side: Option, -) -> Result> { +) -> Option { // All joins have 2 children: assert_eq!(maintains_input_order.len(), 2); let left_maintains = maintains_input_order[0]; @@ -191,13 +177,13 @@ pub fn calculate_join_output_ordering( // In the case below, right ordering should be offseted with the left // side length, since we append the right table to the left table. JoinType::Inner | JoinType::Left | JoinType::Right | JoinType::Full => { - add_offset_to_lex_ordering(right_ordering, left_columns_len)? + add_offset_to_lex_ordering(right_ordering, left_columns_len) } _ => right_ordering.to_vec(), }; let output_ordering = match (left_maintains, right_maintains) { (true, true) => { - return exec_err!("Cannot maintain ordering of both sides"); + unreachable!("Cannot maintain ordering of both sides"); } (true, false) => { // Special case, we can prefix ordering of right side with the ordering of left side. @@ -226,233 +212,9 @@ pub fn calculate_join_output_ordering( } } // Doesn't maintain ordering, output ordering is None. - (false, false) => return Ok(None), - }; - Ok((!output_ordering.is_empty()).then_some(output_ordering)) -} - -/// Combine equivalence properties of the given join inputs. -pub fn combine_join_equivalence_properties( - join_type: JoinType, - left_properties: EquivalenceProperties, - right_properties: EquivalenceProperties, - left_columns_len: usize, - on: &[(Column, Column)], - schema: SchemaRef, -) -> EquivalenceProperties { - let mut new_properties = EquivalenceProperties::new(schema); - match join_type { - JoinType::Inner | JoinType::Left | JoinType::Full | JoinType::Right => { - new_properties.extend(left_properties.classes().to_vec()); - let new_right_properties = right_properties - .classes() - .iter() - .map(|prop| { - let new_head = Column::new( - prop.head().name(), - left_columns_len + prop.head().index(), - ); - let new_others = prop - .others() - .iter() - .map(|col| { - Column::new(col.name(), left_columns_len + col.index()) - }) - .collect::>(); - EquivalentClass::new(new_head, new_others) - }) - .collect::>(); - - new_properties.extend(new_right_properties); - } - JoinType::LeftSemi | JoinType::LeftAnti => { - new_properties.extend(left_properties.classes().to_vec()) - } - JoinType::RightSemi | JoinType::RightAnti => { - new_properties.extend(right_properties.classes().to_vec()) - } - } - - if join_type == JoinType::Inner { - on.iter().for_each(|(column1, column2)| { - let new_column2 = - Column::new(column2.name(), left_columns_len + column2.index()); - new_properties.add_equal_conditions((column1, &new_column2)) - }) - } - new_properties -} - -/// Calculate equivalence properties for the given cross join operation. -pub fn cross_join_equivalence_properties( - left_properties: EquivalenceProperties, - right_properties: EquivalenceProperties, - left_columns_len: usize, - schema: SchemaRef, -) -> EquivalenceProperties { - let mut new_properties = EquivalenceProperties::new(schema); - new_properties.extend(left_properties.classes().to_vec()); - let new_right_properties = right_properties - .classes() - .iter() - .map(|prop| { - let new_head = - Column::new(prop.head().name(), left_columns_len + prop.head().index()); - let new_others = prop - .others() - .iter() - .map(|col| Column::new(col.name(), left_columns_len + col.index())) - .collect::>(); - EquivalentClass::new(new_head, new_others) - }) - .collect::>(); - new_properties.extend(new_right_properties); - new_properties -} - -/// Update right table ordering equivalences so that: -/// - They point to valid indices at the output of the join schema, and -/// - They are normalized with respect to equivalence columns. -/// -/// To do so, we increment column indices by the size of the left table when -/// join schema consists of a combination of left and right schema (Inner, -/// Left, Full, Right joins). Then, we normalize the sort expressions of -/// ordering equivalences one by one. We make sure that each expression in the -/// ordering equivalence is either: -/// - The head of the one of the equivalent classes, or -/// - Doesn't have an equivalent column. -/// -/// This way; once we normalize an expression according to equivalence properties, -/// it can thereafter safely be used for ordering equivalence normalization. -fn get_updated_right_ordering_equivalent_class( - join_type: &JoinType, - right_oeq_class: &OrderingEquivalentClass, - left_columns_len: usize, - join_eq_properties: &EquivalenceProperties, -) -> Result { - match join_type { - // In these modes, indices of the right schema should be offset by - // the left table size. - JoinType::Inner | JoinType::Left | JoinType::Full | JoinType::Right => { - let right_oeq_class = right_oeq_class.add_offset(left_columns_len)?; - return Ok( - right_oeq_class.normalize_with_equivalence_properties(join_eq_properties) - ); - } - _ => {} + (false, false) => return None, }; - Ok(right_oeq_class.normalize_with_equivalence_properties(join_eq_properties)) -} - -/// Calculate ordering equivalence properties for the given join operation. -pub fn combine_join_ordering_equivalence_properties( - join_type: &JoinType, - left: &Arc, - right: &Arc, - schema: SchemaRef, - maintains_input_order: &[bool], - probe_side: Option, - join_eq_properties: EquivalenceProperties, -) -> Result { - let mut new_properties = OrderingEquivalenceProperties::new(schema); - let left_columns_len = left.schema().fields.len(); - let left_oeq_properties = left.ordering_equivalence_properties(); - let right_oeq_properties = right.ordering_equivalence_properties(); - // All joins have 2 children - assert_eq!(maintains_input_order.len(), 2); - let left_maintains = maintains_input_order[0]; - let right_maintains = maintains_input_order[1]; - match (left_maintains, right_maintains) { - (true, true) => return plan_err!("Cannot maintain ordering of both sides"), - (true, false) => { - new_properties.extend(left_oeq_properties.oeq_class().cloned()); - // In this special case, right side ordering can be prefixed with left side ordering. - if let ( - Some(JoinSide::Left), - // right side have an ordering - Some(_), - JoinType::Inner, - Some(oeq_class), - ) = ( - probe_side, - right.output_ordering(), - join_type, - right_oeq_properties.oeq_class(), - ) { - let left_output_ordering = left.output_ordering().unwrap_or(&[]); - - let updated_right_oeq = get_updated_right_ordering_equivalent_class( - join_type, - oeq_class, - left_columns_len, - &join_eq_properties, - )?; - - // Right side ordering equivalence properties should be prepended with - // those of the left side while constructing output ordering equivalence - // properties since stream side is the left side. - // - // If the right table ordering equivalences contain `b ASC`, and the output - // ordering of the left table is `a ASC`, then the ordering equivalence `b ASC` - // for the right table should be converted to `a ASC, b ASC` before it is added - // to the ordering equivalences of the join. - let updated_right_oeq_class = updated_right_oeq - .prefix_ordering_equivalent_class_with_existing_ordering( - left_output_ordering, - &join_eq_properties, - ); - new_properties.extend(Some(updated_right_oeq_class)); - } - } - (false, true) => { - let updated_right_oeq = right_oeq_properties - .oeq_class() - .map(|right_oeq_class| { - get_updated_right_ordering_equivalent_class( - join_type, - right_oeq_class, - left_columns_len, - &join_eq_properties, - ) - }) - .transpose()?; - new_properties.extend(updated_right_oeq); - // In this special case, left side ordering can be prefixed with right side ordering. - if let ( - Some(JoinSide::Right), - // left side have an ordering - Some(_), - JoinType::Inner, - Some(left_oeq_class), - ) = ( - probe_side, - left.output_ordering(), - join_type, - left_oeq_properties.oeq_class(), - ) { - let right_output_ordering = right.output_ordering().unwrap_or(&[]); - let right_output_ordering = - add_offset_to_lex_ordering(right_output_ordering, left_columns_len)?; - - // Left side ordering equivalence properties should be prepended with - // those of the right side while constructing output ordering equivalence - // properties since stream side is the right side. - // - // If the right table ordering equivalences contain `b ASC`, and the output - // ordering of the left table is `a ASC`, then the ordering equivalence `b ASC` - // for the right table should be converted to `a ASC, b ASC` before it is added - // to the ordering equivalences of the join. - let updated_left_oeq_class = left_oeq_class - .prefix_ordering_equivalent_class_with_existing_ordering( - &right_output_ordering, - &join_eq_properties, - ); - new_properties.extend(Some(updated_left_oeq_class)); - } - } - (false, false) => {} - } - Ok(new_properties) + (!output_ordering.is_empty()).then_some(output_ordering) } /// Information about the index and placement (left or right) of the columns @@ -1930,84 +1692,6 @@ mod tests { Ok(()) } - #[test] - fn test_get_updated_right_ordering_equivalence_properties() -> Result<()> { - let join_type = JoinType::Inner; - - let options = SortOptions::default(); - let right_oeq_class = OrderingEquivalentClass::new( - vec![ - PhysicalSortExpr { - expr: Arc::new(Column::new("x", 0)), - options, - }, - PhysicalSortExpr { - expr: Arc::new(Column::new("y", 1)), - options, - }, - ], - vec![vec![ - PhysicalSortExpr { - expr: Arc::new(Column::new("z", 2)), - options, - }, - PhysicalSortExpr { - expr: Arc::new(Column::new("w", 3)), - options, - }, - ]], - ); - - let left_columns_len = 4; - - let fields: Fields = ["a", "b", "c", "d", "x", "y", "z", "w"] - .into_iter() - .map(|name| Field::new(name, DataType::Int32, true)) - .collect(); - - let mut join_eq_properties = - EquivalenceProperties::new(Arc::new(Schema::new(fields))); - join_eq_properties - .add_equal_conditions((&Column::new("a", 0), &Column::new("x", 4))); - join_eq_properties - .add_equal_conditions((&Column::new("d", 3), &Column::new("w", 7))); - - let result = get_updated_right_ordering_equivalent_class( - &join_type, - &right_oeq_class, - left_columns_len, - &join_eq_properties, - )?; - - let expected = OrderingEquivalentClass::new( - vec![ - PhysicalSortExpr { - expr: Arc::new(Column::new("a", 0)), - options, - }, - PhysicalSortExpr { - expr: Arc::new(Column::new("y", 5)), - options, - }, - ], - vec![vec![ - PhysicalSortExpr { - expr: Arc::new(Column::new("z", 6)), - options, - }, - PhysicalSortExpr { - expr: Arc::new(Column::new("d", 3)), - options, - }, - ]], - ); - - assert_eq!(result.head(), expected.head()); - assert_eq!(result.others(), expected.others()); - - Ok(()) - } - #[test] fn test_calculate_join_output_ordering() -> Result<()> { let options = SortOptions::default(); @@ -2100,7 +1784,7 @@ mod tests { left_columns_len, maintains_input_order, probe_side - )?, + ), expected[i] ); } From af2cda928140f85d369e26931756a0cbc127e9ec Mon Sep 17 00:00:00 2001 From: Huaijin Date: Wed, 25 Oct 2023 15:56:13 +0800 Subject: [PATCH 126/572] support scan empty projection (#7920) MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit * fix * modify push_down_filter for empty projection * clean code * fix ci * fix * Update datafusion/core/tests/custom_sources_cases/provider_filter_pushdown.rs Co-authored-by: Daniël Heres * remove LogicalPlan::Values in push_down_projection --------- Co-authored-by: Daniël Heres --- .../physical_plan/file_scan_config.rs | 10 +- .../provider_filter_pushdown.rs | 14 +- datafusion/core/tests/sql/explain_analyze.rs | 2 +- .../optimizer/src/push_down_projection.rs | 168 +----------------- .../physical-plan/src/joins/cross_join.rs | 4 +- datafusion/sqllogictest/test_files/avro.slt | 4 +- datafusion/sqllogictest/test_files/json.slt | 4 +- .../sqllogictest/test_files/subquery.slt | 6 +- 8 files changed, 31 insertions(+), 181 deletions(-) diff --git a/datafusion/core/src/datasource/physical_plan/file_scan_config.rs b/datafusion/core/src/datasource/physical_plan/file_scan_config.rs index d8a9697b2bf7..3efb0df9df7c 100644 --- a/datafusion/core/src/datasource/physical_plan/file_scan_config.rs +++ b/datafusion/core/src/datasource/physical_plan/file_scan_config.rs @@ -36,7 +36,7 @@ use crate::{ use arrow::array::{ArrayData, BufferBuilder}; use arrow::buffer::Buffer; use arrow::datatypes::{ArrowNativeType, UInt16Type}; -use arrow_array::{ArrayRef, DictionaryArray, RecordBatch}; +use arrow_array::{ArrayRef, DictionaryArray, RecordBatch, RecordBatchOptions}; use arrow_schema::{DataType, Field, Schema, SchemaRef}; use datafusion_common::stats::Precision; use datafusion_common::{exec_err, ColumnStatistics, Statistics}; @@ -339,7 +339,13 @@ impl PartitionColumnProjector { ), ) } - RecordBatch::try_new(Arc::clone(&self.projected_schema), cols).map_err(Into::into) + + RecordBatch::try_new_with_options( + Arc::clone(&self.projected_schema), + cols, + &RecordBatchOptions::new().with_row_count(Some(file_batch.num_rows())), + ) + .map_err(Into::into) } } diff --git a/datafusion/core/tests/custom_sources_cases/provider_filter_pushdown.rs b/datafusion/core/tests/custom_sources_cases/provider_filter_pushdown.rs index 4679ca6d07df..e374abd6e891 100644 --- a/datafusion/core/tests/custom_sources_cases/provider_filter_pushdown.rs +++ b/datafusion/core/tests/custom_sources_cases/provider_filter_pushdown.rs @@ -149,10 +149,12 @@ impl TableProvider for CustomProvider { async fn scan( &self, _state: &SessionState, - _: Option<&Vec>, + projection: Option<&Vec>, filters: &[Expr], _: Option, ) -> Result> { + let empty = Vec::new(); + let projection = projection.unwrap_or(&empty); match &filters[0] { Expr::BinaryExpr(BinaryExpr { right, .. }) => { let int_value = match &**right { @@ -182,7 +184,10 @@ impl TableProvider for CustomProvider { }; Ok(Arc::new(CustomPlan { - schema: self.zero_batch.schema(), + schema: match projection.is_empty() { + true => Arc::new(Schema::empty()), + false => self.zero_batch.schema(), + }, batches: match int_value { 0 => vec![self.zero_batch.clone()], 1 => vec![self.one_batch.clone()], @@ -191,7 +196,10 @@ impl TableProvider for CustomProvider { })) } _ => Ok(Arc::new(CustomPlan { - schema: self.zero_batch.schema(), + schema: match projection.is_empty() { + true => Arc::new(Schema::empty()), + false => self.zero_batch.schema(), + }, batches: vec![], })), } diff --git a/datafusion/core/tests/sql/explain_analyze.rs b/datafusion/core/tests/sql/explain_analyze.rs index 7238369f832a..2436e82f3ce9 100644 --- a/datafusion/core/tests/sql/explain_analyze.rs +++ b/datafusion/core/tests/sql/explain_analyze.rs @@ -788,7 +788,7 @@ async fn explain_logical_plan_only() { "logical_plan", "Aggregate: groupBy=[[]], aggr=[[COUNT(UInt8(1)) AS COUNT(*)]]\ \n SubqueryAlias: t\ - \n Projection: column2\ + \n Projection: \ \n Values: (Utf8(\"a\"), Int64(1), Int64(100)), (Utf8(\"a\"), Int64(2), Int64(150))" ]]; assert_eq!(expected, actual); diff --git a/datafusion/optimizer/src/push_down_projection.rs b/datafusion/optimizer/src/push_down_projection.rs index 839f6b5bb8f6..e7fdaa8b0b5e 100644 --- a/datafusion/optimizer/src/push_down_projection.rs +++ b/datafusion/optimizer/src/push_down_projection.rs @@ -23,7 +23,6 @@ use crate::merge_projection::merge_projection; use crate::optimizer::ApplyOrder; use crate::push_down_filter::replace_cols_by_name; use crate::{OptimizerConfig, OptimizerRule}; -use arrow::datatypes::DataType; use arrow::error::Result as ArrowResult; use datafusion_common::ScalarValue::UInt8; use datafusion_common::{ @@ -149,10 +148,6 @@ impl OptimizerRule for PushDownProjection { { let mut used_columns: HashSet = HashSet::new(); if projection_is_empty { - let field = find_small_field(scan.projected_schema.fields()).ok_or( - DataFusionError::Internal("Scan with empty schema".to_string()), - )?; - used_columns.insert(field.qualified_column()); push_down_scan(&used_columns, scan, true)? } else { for expr in projection.expr.iter() { @@ -163,17 +158,6 @@ impl OptimizerRule for PushDownProjection { plan.with_new_inputs(&[new_scan])? } } - LogicalPlan::Values(values) if projection_is_empty => { - let field = find_small_field(values.schema.fields()).ok_or( - DataFusionError::Internal("Values with empty schema".to_string()), - )?; - let column = Expr::Column(field.qualified_column()); - - LogicalPlan::Projection(Projection::try_new( - vec![column], - Arc::new(child_plan.clone()), - )?) - } LogicalPlan::Union(union) => { let mut required_columns = HashSet::new(); exprlist_to_columns(&projection.expr, &mut required_columns)?; @@ -429,87 +413,6 @@ pub fn collect_projection_expr(projection: &Projection) -> HashMap .collect::>() } -/// Accumulate the memory size of a data type measured in bits. -/// -/// Types with a variable size get assigned with a fixed size which is greater than most -/// primitive types. -/// -/// While traversing nested types, `nesting` is incremented on every level. -fn nested_size(data_type: &DataType, nesting: &mut usize) -> usize { - use DataType::*; - if data_type.is_primitive() { - return data_type.primitive_width().unwrap_or(1) * 8; - } - - if data_type.is_nested() { - *nesting += 1; - } - - match data_type { - Null => 0, - Boolean => 1, - Binary | Utf8 => 128, - LargeBinary | LargeUtf8 => 256, - FixedSizeBinary(bytes) => (*bytes * 8) as usize, - // primitive types - Int8 - | Int16 - | Int32 - | Int64 - | UInt8 - | UInt16 - | UInt32 - | UInt64 - | Float16 - | Float32 - | Float64 - | Timestamp(_, _) - | Date32 - | Date64 - | Time32(_) - | Time64(_) - | Duration(_) - | Interval(_) - | Dictionary(_, _) - | Decimal128(_, _) - | Decimal256(_, _) => data_type.primitive_width().unwrap_or(1) * 8, - // nested types - List(f) => nested_size(f.data_type(), nesting), - FixedSizeList(_, s) => (s * 8) as usize, - LargeList(f) => nested_size(f.data_type(), nesting), - Struct(fields) => fields - .iter() - .map(|f| nested_size(f.data_type(), nesting)) - .sum(), - Union(fields, _) => fields - .iter() - .map(|(_, f)| nested_size(f.data_type(), nesting)) - .sum(), - Map(field, _) => nested_size(field.data_type(), nesting), - RunEndEncoded(run_ends, values) => { - nested_size(run_ends.data_type(), nesting) - + nested_size(values.data_type(), nesting) - } - } -} - -/// Find a field with a presumable small memory footprint based on its data type's memory size -/// and the level of nesting. -fn find_small_field(fields: &[DFField]) -> Option { - fields - .iter() - .map(|f| { - let nesting = &mut 0; - let size = nested_size(f.data_type(), nesting); - (*nesting, size) - }) - .enumerate() - .min_by(|(_, (nesting_a, size_a)), (_, (nesting_b, size_b))| { - nesting_a.cmp(nesting_b).then(size_a.cmp(size_b)) - }) - .map(|(i, _)| fields[i].clone()) -} - /// Get the projection exprs from columns in the order of the schema fn get_expr(columns: &HashSet, schema: &DFSchemaRef) -> Result> { let expr = schema @@ -640,7 +543,7 @@ mod tests { use crate::optimizer::Optimizer; use crate::test::*; use crate::OptimizerContext; - use arrow::datatypes::{DataType, Field, Schema, TimeUnit}; + use arrow::datatypes::{DataType, Field, Schema}; use datafusion_common::DFSchema; use datafusion_expr::builder::table_scan_with_filters; use datafusion_expr::expr; @@ -1232,73 +1135,4 @@ mod tests { .unwrap_or(optimized_plan); Ok(optimized_plan) } - - #[test] - fn test_nested_size() { - use DataType::*; - let nesting = &mut 0; - assert_eq!(nested_size(&Null, nesting), 0); - assert_eq!(*nesting, 0); - assert_eq!(nested_size(&Boolean, nesting), 1); - assert_eq!(*nesting, 0); - assert_eq!(nested_size(&UInt8, nesting), 8); - assert_eq!(*nesting, 0); - assert_eq!(nested_size(&Int64, nesting), 64); - assert_eq!(*nesting, 0); - assert_eq!(nested_size(&Decimal256(5, 2), nesting), 256); - assert_eq!(*nesting, 0); - assert_eq!( - nested_size(&List(Arc::new(Field::new("A", Int64, true))), nesting), - 64 - ); - assert_eq!(*nesting, 1); - *nesting = 0; - assert_eq!( - nested_size( - &List(Arc::new(Field::new( - "A", - List(Arc::new(Field::new("AA", Int64, true))), - true - ))), - nesting - ), - 64 - ); - assert_eq!(*nesting, 2); - } - - #[test] - fn test_find_small_field() { - use DataType::*; - let int32 = DFField::from(Field::new("a", Int32, false)); - let bin = DFField::from(Field::new("b", Binary, false)); - let list_i64 = DFField::from(Field::new( - "c", - List(Arc::new(Field::new("c_1", Int64, true))), - false, - )); - let time_s = DFField::from(Field::new("d", Time32(TimeUnit::Second), false)); - - assert_eq!( - find_small_field(&[ - int32.clone(), - bin.clone(), - list_i64.clone(), - time_s.clone() - ]), - Some(int32.clone()) - ); - assert_eq!( - find_small_field(&[bin.clone(), list_i64.clone(), time_s.clone()]), - Some(time_s.clone()) - ); - assert_eq!( - find_small_field(&[time_s.clone(), int32.clone()]), - Some(time_s.clone()) - ); - assert_eq!( - find_small_field(&[bin.clone(), list_i64.clone()]), - Some(bin.clone()) - ); - } } diff --git a/datafusion/physical-plan/src/joins/cross_join.rs b/datafusion/physical-plan/src/joins/cross_join.rs index c9c92e8c11ae..d8c8064e2ac1 100644 --- a/datafusion/physical-plan/src/joins/cross_join.rs +++ b/datafusion/physical-plan/src/joins/cross_join.rs @@ -34,6 +34,7 @@ use crate::{ use arrow::datatypes::{Fields, Schema, SchemaRef}; use arrow::record_batch::RecordBatch; +use arrow_array::RecordBatchOptions; use datafusion_common::stats::Precision; use datafusion_common::{plan_err, DataFusionError, Result, ScalarValue}; use datafusion_execution::memory_pool::{MemoryConsumer, MemoryReservation}; @@ -347,13 +348,14 @@ fn build_batch( }) .collect::>>()?; - RecordBatch::try_new( + RecordBatch::try_new_with_options( Arc::new(schema.clone()), arrays .iter() .chain(batch.columns().iter()) .cloned() .collect(), + &RecordBatchOptions::new().with_row_count(Some(batch.num_rows())), ) .map_err(Into::into) } diff --git a/datafusion/sqllogictest/test_files/avro.slt b/datafusion/sqllogictest/test_files/avro.slt index bd2ba706663c..3f21274c009f 100644 --- a/datafusion/sqllogictest/test_files/avro.slt +++ b/datafusion/sqllogictest/test_files/avro.slt @@ -253,10 +253,10 @@ EXPLAIN SELECT count(*) from alltypes_plain ---- logical_plan Aggregate: groupBy=[[]], aggr=[[COUNT(UInt8(1)) AS COUNT(*)]] ---TableScan: alltypes_plain projection=[bool_col] +--TableScan: alltypes_plain projection=[] physical_plan AggregateExec: mode=Final, gby=[], aggr=[COUNT(*)] --CoalescePartitionsExec ----AggregateExec: mode=Partial, gby=[], aggr=[COUNT(*)] ------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 ---------AvroExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/avro/alltypes_plain.avro]]}, projection=[bool_col] +--------AvroExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/avro/alltypes_plain.avro]]} diff --git a/datafusion/sqllogictest/test_files/json.slt b/datafusion/sqllogictest/test_files/json.slt index a06565f42cdb..c0d5e895f0f2 100644 --- a/datafusion/sqllogictest/test_files/json.slt +++ b/datafusion/sqllogictest/test_files/json.slt @@ -50,13 +50,13 @@ EXPLAIN SELECT count(*) from json_test ---- logical_plan Aggregate: groupBy=[[]], aggr=[[COUNT(UInt8(1)) AS COUNT(*)]] ---TableScan: json_test projection=[c] +--TableScan: json_test projection=[] physical_plan AggregateExec: mode=Final, gby=[], aggr=[COUNT(*)] --CoalescePartitionsExec ----AggregateExec: mode=Partial, gby=[], aggr=[COUNT(*)] ------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 ---------JsonExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/2.json]]}, projection=[c] +--------JsonExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/2.json]]} query ? SELECT mycol FROM single_nan diff --git a/datafusion/sqllogictest/test_files/subquery.slt b/datafusion/sqllogictest/test_files/subquery.slt index 7cbb848f3333..822a70bb5bad 100644 --- a/datafusion/sqllogictest/test_files/subquery.slt +++ b/datafusion/sqllogictest/test_files/subquery.slt @@ -695,7 +695,7 @@ logical_plan Projection: __scalar_sq_1.COUNT(*) AS b --SubqueryAlias: __scalar_sq_1 ----Aggregate: groupBy=[[]], aggr=[[COUNT(UInt8(1)) AS COUNT(*)]] -------TableScan: t1 projection=[t1_id] +------TableScan: t1 projection=[] #simple_uncorrelated_scalar_subquery2 query TT @@ -706,10 +706,10 @@ Projection: __scalar_sq_1.COUNT(*) AS b, __scalar_sq_2.COUNT(Int64(1)) AS COUNT( --Left Join: ----SubqueryAlias: __scalar_sq_1 ------Aggregate: groupBy=[[]], aggr=[[COUNT(UInt8(1)) AS COUNT(*)]] ---------TableScan: t1 projection=[t1_id] +--------TableScan: t1 projection=[] ----SubqueryAlias: __scalar_sq_2 ------Aggregate: groupBy=[[]], aggr=[[COUNT(Int64(1))]] ---------TableScan: t2 projection=[t2_id] +--------TableScan: t2 projection=[] query II select (select count(*) from t1) as b, (select count(1) from t2) From b16cd934698e8c1731f16873fcab17ffae04cc5b Mon Sep 17 00:00:00 2001 From: Mustafa Akur <106137913+mustafasrepo@users.noreply.github.com> Date: Wed, 25 Oct 2023 16:43:39 +0300 Subject: [PATCH 127/572] Cleanup logical optimizer rules. (#7919) * Initial commit * Address todos * Update comments * Simplifications * Minor simplifications * Address reviews * Add TableScan constructor * Minor changes * make try_new_with_schema method of Aggregate private * Use projection try_new instead of try_new_schema * Simplifications, add comment * Review changes * Improve comments * Move get_wider_type to type_coercion module * Clean up type coercion file --------- Co-authored-by: berkaysynnada Co-authored-by: Mehmet Ozan Kabak --- datafusion/common/src/dfschema.rs | 8 + .../common/src/functional_dependencies.rs | 17 ++ datafusion/core/tests/sql/group_by.rs | 14 +- datafusion/expr/src/built_in_function.rs | 45 +++-- datafusion/expr/src/logical_plan/builder.rs | 183 +++++------------- datafusion/expr/src/logical_plan/plan.rs | 134 ++++++++----- datafusion/expr/src/tree_node/expr.rs | 22 ++- datafusion/expr/src/type_coercion/binary.rs | 147 ++++++++------ .../optimizer/src/common_subexpr_eliminate.rs | 36 ++-- datafusion/optimizer/src/merge_projection.rs | 11 +- .../optimizer/src/push_down_projection.rs | 58 +++--- .../src/replace_distinct_aggregate.rs | 10 +- .../src/single_distinct_to_groupby.rs | 104 ++++++---- .../physical-expr/src/array_expressions.rs | 39 ++-- datafusion/sqllogictest/test_files/array.slt | 6 +- .../sqllogictest/test_files/tpch/q16.slt.part | 20 +- 16 files changed, 451 insertions(+), 403 deletions(-) diff --git a/datafusion/common/src/dfschema.rs b/datafusion/common/src/dfschema.rs index b1aee41978c2..e16acbfedc81 100644 --- a/datafusion/common/src/dfschema.rs +++ b/datafusion/common/src/dfschema.rs @@ -444,6 +444,14 @@ impl DFSchema { .zip(iter2) .all(|((t1, f1), (t2, f2))| t1 == t2 && Self::field_is_semantically_equal(f1, f2)) } + ( + DataType::Decimal128(_l_precision, _l_scale), + DataType::Decimal128(_r_precision, _r_scale), + ) => true, + ( + DataType::Decimal256(_l_precision, _l_scale), + DataType::Decimal256(_r_precision, _r_scale), + ) => true, _ => dt1 == dt2, } } diff --git a/datafusion/common/src/functional_dependencies.rs b/datafusion/common/src/functional_dependencies.rs index 869709bc8dfc..fbddcddab4bc 100644 --- a/datafusion/common/src/functional_dependencies.rs +++ b/datafusion/common/src/functional_dependencies.rs @@ -558,4 +558,21 @@ mod tests { assert_eq!(iter.next(), Some(&Constraint::Unique(vec![20]))); assert_eq!(iter.next(), None); } + + #[test] + fn test_get_updated_id_keys() { + let fund_dependencies = + FunctionalDependencies::new(vec![FunctionalDependence::new( + vec![1], + vec![0, 1, 2], + true, + )]); + let res = fund_dependencies.project_functional_dependencies(&[1, 2], 2); + let expected = FunctionalDependencies::new(vec![FunctionalDependence::new( + vec![0], + vec![0, 1], + true, + )]); + assert_eq!(res, expected); + } } diff --git a/datafusion/core/tests/sql/group_by.rs b/datafusion/core/tests/sql/group_by.rs index 7c7703b69683..58f0ac21d951 100644 --- a/datafusion/core/tests/sql/group_by.rs +++ b/datafusion/core/tests/sql/group_by.rs @@ -231,13 +231,13 @@ async fn group_by_dictionary() { .expect("ran plan correctly"); let expected = [ - "+-------+------------------------+", - "| t.val | COUNT(DISTINCT t.dict) |", - "+-------+------------------------+", - "| 1 | 2 |", - "| 2 | 2 |", - "| 4 | 1 |", - "+-------+------------------------+", + "+-----+------------------------+", + "| val | COUNT(DISTINCT t.dict) |", + "+-----+------------------------+", + "| 1 | 2 |", + "| 2 | 2 |", + "| 4 | 1 |", + "+-----+------------------------+", ]; assert_batches_sorted_eq!(expected, &results); } diff --git a/datafusion/expr/src/built_in_function.rs b/datafusion/expr/src/built_in_function.rs index 350067a42186..16554133d828 100644 --- a/datafusion/expr/src/built_in_function.rs +++ b/datafusion/expr/src/built_in_function.rs @@ -17,21 +17,26 @@ //! Built-in functions module contains all the built-in functions definitions. +use std::cmp::Ordering; +use std::collections::HashMap; +use std::fmt; +use std::str::FromStr; +use std::sync::{Arc, OnceLock}; + use crate::nullif::SUPPORTED_NULLIF_TYPES; use crate::signature::TIMEZONE_WILDCARD; +use crate::type_coercion::binary::get_wider_type; use crate::type_coercion::functions::data_types; use crate::{ conditional_expressions, struct_expressions, utils, FuncMonotonicity, Signature, TypeSignature, Volatility, }; + use arrow::datatypes::{DataType, Field, Fields, IntervalUnit, TimeUnit}; use datafusion_common::{ internal_err, plan_datafusion_err, plan_err, DataFusionError, Result, }; -use std::collections::HashMap; -use std::fmt; -use std::str::FromStr; -use std::sync::{Arc, OnceLock}; + use strum::IntoEnumIterator; use strum_macros::EnumIter; @@ -468,18 +473,14 @@ impl BuiltinScalarFunction { /// * `List(Int64)` has dimension 2 /// * `List(List(Int64))` has dimension 3 /// * etc. - fn return_dimension(self, input_expr_type: DataType) -> u64 { - let mut res: u64 = 1; + fn return_dimension(self, input_expr_type: &DataType) -> u64 { + let mut result: u64 = 1; let mut current_data_type = input_expr_type; - loop { - match current_data_type { - DataType::List(field) => { - current_data_type = field.data_type().clone(); - res += 1; - } - _ => return res, - } + while let DataType::List(field) = current_data_type { + current_data_type = field.data_type(); + result += 1; } + result } /// Returns the output [`DataType`] of this function @@ -538,11 +539,17 @@ impl BuiltinScalarFunction { match input_expr_type { List(field) => { if !field.data_type().equals_datatype(&Null) { - let dims = self.return_dimension(input_expr_type.clone()); - if max_dims < dims { - max_dims = dims; - expr_type = input_expr_type.clone(); - } + let dims = self.return_dimension(input_expr_type); + expr_type = match max_dims.cmp(&dims) { + Ordering::Greater => expr_type, + Ordering::Equal => { + get_wider_type(&expr_type, input_expr_type)? + } + Ordering::Less => { + max_dims = dims; + input_expr_type.clone() + } + }; } } _ => { diff --git a/datafusion/expr/src/logical_plan/builder.rs b/datafusion/expr/src/logical_plan/builder.rs index cd50dbe79cfd..9ce1d203d1c9 100644 --- a/datafusion/expr/src/logical_plan/builder.rs +++ b/datafusion/expr/src/logical_plan/builder.rs @@ -17,6 +17,13 @@ //! This module provides a builder for creating LogicalPlans +use std::any::Any; +use std::cmp::Ordering; +use std::collections::{HashMap, HashSet}; +use std::convert::TryFrom; +use std::iter::zip; +use std::sync::Arc; + use crate::dml::{CopyOptions, CopyTo}; use crate::expr::Alias; use crate::expr_rewriter::{ @@ -24,38 +31,29 @@ use crate::expr_rewriter::{ normalize_col_with_schemas_and_ambiguity_check, normalize_cols, rewrite_sort_cols_by_aggs, }; +use crate::logical_plan::{ + Aggregate, Analyze, CrossJoin, Distinct, EmptyRelation, Explain, Filter, Join, + JoinConstraint, JoinType, Limit, LogicalPlan, Partitioning, PlanType, Prepare, + Projection, Repartition, Sort, SubqueryAlias, TableScan, Union, Unnest, Values, + Window, +}; use crate::type_coercion::binary::comparison_coercion; -use crate::utils::{columnize_expr, compare_sort_expr}; -use crate::{ - and, binary_expr, DmlStatement, Operator, TableProviderFilterPushDown, WriteOp, +use crate::utils::{ + can_hash, columnize_expr, compare_sort_expr, expand_qualified_wildcard, + expand_wildcard, find_valid_equijoin_key_pair, group_window_expr_by_sort_keys, }; use crate::{ - logical_plan::{ - Aggregate, Analyze, CrossJoin, Distinct, EmptyRelation, Explain, Filter, Join, - JoinConstraint, JoinType, Limit, LogicalPlan, Partitioning, PlanType, Prepare, - Projection, Repartition, Sort, SubqueryAlias, TableScan, Union, Unnest, Values, - Window, - }, - utils::{ - can_hash, expand_qualified_wildcard, expand_wildcard, - find_valid_equijoin_key_pair, group_window_expr_by_sort_keys, - }, - Expr, ExprSchemable, TableSource, + and, binary_expr, DmlStatement, Expr, ExprSchemable, Operator, + TableProviderFilterPushDown, TableSource, WriteOp, }; + use arrow::datatypes::{DataType, Schema, SchemaRef}; -use datafusion_common::UnnestOptions; +use datafusion_common::display::ToStringifiedPlan; use datafusion_common::{ - display::ToStringifiedPlan, Column, DFField, DFSchema, DFSchemaRef, DataFusionError, - FileType, FunctionalDependencies, OwnedTableReference, Result, ScalarValue, - TableReference, ToDFSchema, + plan_datafusion_err, plan_err, Column, DFField, DFSchema, DFSchemaRef, + DataFusionError, FileType, OwnedTableReference, Result, ScalarValue, TableReference, + ToDFSchema, UnnestOptions, }; -use datafusion_common::{plan_datafusion_err, plan_err}; -use std::any::Any; -use std::cmp::Ordering; -use std::collections::{HashMap, HashSet}; -use std::convert::TryFrom; -use std::iter::zip; -use std::sync::Arc; /// Default table name for unnamed table pub const UNNAMED_TABLE: &str = "?table?"; @@ -283,53 +281,9 @@ impl LogicalPlanBuilder { projection: Option>, filters: Vec, ) -> Result { - let table_name = table_name.into(); - - if table_name.table().is_empty() { - return plan_err!("table_name cannot be empty"); - } - - let schema = table_source.schema(); - let func_dependencies = FunctionalDependencies::new_from_constraints( - table_source.constraints(), - schema.fields.len(), - ); - - let projected_schema = projection - .as_ref() - .map(|p| { - let projected_func_dependencies = - func_dependencies.project_functional_dependencies(p, p.len()); - DFSchema::new_with_metadata( - p.iter() - .map(|i| { - DFField::from_qualified( - table_name.clone(), - schema.field(*i).clone(), - ) - }) - .collect(), - schema.metadata().clone(), - ) - .map(|df_schema| { - df_schema.with_functional_dependencies(projected_func_dependencies) - }) - }) - .unwrap_or_else(|| { - DFSchema::try_from_qualified_schema(table_name.clone(), &schema).map( - |df_schema| df_schema.with_functional_dependencies(func_dependencies), - ) - })?; - - let table_scan = LogicalPlan::TableScan(TableScan { - table_name, - source: table_source, - projected_schema: Arc::new(projected_schema), - projection, - filters, - fetch: None, - }); - Ok(Self::from(table_scan)) + TableScan::try_new(table_name, table_source, projection, filters, None) + .map(LogicalPlan::TableScan) + .map(Self::from) } /// Wrap a plan in a window @@ -374,7 +328,7 @@ impl LogicalPlanBuilder { self, expr: impl IntoIterator>, ) -> Result { - Ok(Self::from(project(self.plan, expr)?)) + project(self.plan, expr).map(Self::from) } /// Select the given column indices @@ -390,10 +344,9 @@ impl LogicalPlanBuilder { /// Apply a filter pub fn filter(self, expr: impl Into) -> Result { let expr = normalize_col(expr.into(), &self.plan)?; - Ok(Self::from(LogicalPlan::Filter(Filter::try_new( - expr, - Arc::new(self.plan), - )?))) + Filter::try_new(expr, Arc::new(self.plan)) + .map(LogicalPlan::Filter) + .map(Self::from) } /// Make a builder for a prepare logical plan from the builder's plan @@ -421,7 +374,7 @@ impl LogicalPlanBuilder { /// Apply an alias pub fn alias(self, alias: impl Into) -> Result { - Ok(Self::from(subquery_alias(self.plan, alias)?)) + subquery_alias(self.plan, alias).map(Self::from) } /// Add missing sort columns to all downstream projection @@ -476,7 +429,7 @@ impl LogicalPlanBuilder { Self::ambiguous_distinct_check(&missing_exprs, missing_cols, &expr)?; } expr.extend(missing_exprs); - Ok(project((*input).clone(), expr)?) + project((*input).clone(), expr) } _ => { let is_distinct = @@ -583,15 +536,14 @@ impl LogicalPlanBuilder { fetch: None, }); - Ok(Self::from(LogicalPlan::Projection(Projection::try_new( - new_expr, - Arc::new(sort_plan), - )?))) + Projection::try_new(new_expr, Arc::new(sort_plan)) + .map(LogicalPlan::Projection) + .map(Self::from) } /// Apply a union, preserving duplicate rows pub fn union(self, plan: LogicalPlan) -> Result { - Ok(Self::from(union(self.plan, plan)?)) + union(self.plan, plan).map(Self::from) } /// Apply a union, removing duplicate rows @@ -941,11 +893,9 @@ impl LogicalPlanBuilder { ) -> Result { let group_expr = normalize_cols(group_expr, &self.plan)?; let aggr_expr = normalize_cols(aggr_expr, &self.plan)?; - Ok(Self::from(LogicalPlan::Aggregate(Aggregate::try_new( - Arc::new(self.plan), - group_expr, - aggr_expr, - )?))) + Aggregate::try_new(Arc::new(self.plan), group_expr, aggr_expr) + .map(LogicalPlan::Aggregate) + .map(Self::from) } /// Create an expression to represent the explanation of the plan @@ -1203,8 +1153,8 @@ pub fn build_join_schema( ); let mut metadata = left.metadata().clone(); metadata.extend(right.metadata().clone()); - Ok(DFSchema::new_with_metadata(fields, metadata)? - .with_functional_dependencies(func_dependencies)) + DFSchema::new_with_metadata(fields, metadata) + .map(|schema| schema.with_functional_dependencies(func_dependencies)) } /// Errors if one or more expressions have equal names. @@ -1251,9 +1201,8 @@ pub fn project_with_column_index( }) .collect::>(); - Ok(LogicalPlan::Projection(Projection::try_new_with_schema( - alias_expr, input, schema, - )?)) + Projection::try_new_with_schema(alias_expr, input, schema) + .map(LogicalPlan::Projection) } /// Union two logical plans. @@ -1349,10 +1298,7 @@ pub fn project( } validate_unique_names("Projections", projected_expr.iter())?; - Ok(LogicalPlan::Projection(Projection::try_new( - projected_expr, - Arc::new(plan.clone()), - )?)) + Projection::try_new(projected_expr, Arc::new(plan)).map(LogicalPlan::Projection) } /// Create a SubqueryAlias to wrap a LogicalPlan. @@ -1360,9 +1306,7 @@ pub fn subquery_alias( plan: LogicalPlan, alias: impl Into, ) -> Result { - Ok(LogicalPlan::SubqueryAlias(SubqueryAlias::try_new( - plan, alias, - )?)) + SubqueryAlias::try_new(plan, alias).map(LogicalPlan::SubqueryAlias) } /// Create a LogicalPlanBuilder representing a scan of a table with the provided name and schema. @@ -1525,11 +1469,11 @@ pub fn unnest_with_options( }) .collect::>(); - let schema = Arc::new( - DFSchema::new_with_metadata(fields, input_schema.metadata().clone())? - // We can use the existing functional dependencies: - .with_functional_dependencies(input_schema.functional_dependencies().clone()), - ); + let metadata = input_schema.metadata().clone(); + let df_schema = DFSchema::new_with_metadata(fields, metadata)?; + // We can use the existing functional dependencies: + let deps = input_schema.functional_dependencies().clone(); + let schema = Arc::new(df_schema.with_functional_dependencies(deps)); Ok(LogicalPlan::Unnest(Unnest { input: Arc::new(input), @@ -1541,16 +1485,12 @@ pub fn unnest_with_options( #[cfg(test)] mod tests { - use crate::logical_plan::StringifiedPlan; - use crate::{col, in_subquery, lit, scalar_subquery, sum}; - use crate::{expr, expr_fn::exists}; - use super::*; + use crate::logical_plan::StringifiedPlan; + use crate::{col, expr, expr_fn::exists, in_subquery, lit, scalar_subquery, sum}; use arrow::datatypes::{DataType, Field}; - use datafusion_common::{ - FunctionalDependence, OwnedTableReference, SchemaError, TableReference, - }; + use datafusion_common::{OwnedTableReference, SchemaError, TableReference}; #[test] fn plan_builder_simple() -> Result<()> { @@ -2051,21 +1991,4 @@ mod tests { Ok(()) } - - #[test] - fn test_get_updated_id_keys() { - let fund_dependencies = - FunctionalDependencies::new(vec![FunctionalDependence::new( - vec![1], - vec![0, 1, 2], - true, - )]); - let res = fund_dependencies.project_functional_dependencies(&[1, 2], 2); - let expected = FunctionalDependencies::new(vec![FunctionalDependence::new( - vec![0], - vec![0, 1], - true, - )]); - assert_eq!(res, expected); - } } diff --git a/datafusion/expr/src/logical_plan/plan.rs b/datafusion/expr/src/logical_plan/plan.rs index 1c526c7b4030..d62ac8926328 100644 --- a/datafusion/expr/src/logical_plan/plan.rs +++ b/datafusion/expr/src/logical_plan/plan.rs @@ -17,6 +17,13 @@ //! Logical plan types +use std::collections::{HashMap, HashSet}; +use std::fmt::{self, Debug, Display, Formatter}; +use std::hash::{Hash, Hasher}; +use std::sync::Arc; + +use super::dml::CopyTo; +use super::DdlStatement; use crate::dml::CopyOptions; use crate::expr::{Alias, Exists, InSubquery, Placeholder}; use crate::expr_rewriter::create_col_from_scalar_expr; @@ -28,15 +35,11 @@ use crate::utils::{ grouping_set_expr_count, grouping_set_to_exprlist, inspect_expr_pre, }; use crate::{ - build_join_schema, Expr, ExprSchemable, TableProviderFilterPushDown, TableSource, -}; -use crate::{ - expr_vec_fmt, BinaryExpr, CreateMemoryTable, CreateView, LogicalPlanBuilder, Operator, + build_join_schema, expr_vec_fmt, BinaryExpr, CreateMemoryTable, CreateView, Expr, + ExprSchemable, LogicalPlanBuilder, Operator, TableProviderFilterPushDown, + TableSource, }; -use super::dml::CopyTo; -use super::DdlStatement; - use arrow::datatypes::{DataType, Field, Schema, SchemaRef}; use datafusion_common::tree_node::{ RewriteRecursion, Transformed, TreeNode, TreeNodeRewriter, TreeNodeVisitor, @@ -51,11 +54,6 @@ use datafusion_common::{ pub use datafusion_common::display::{PlanType, StringifiedPlan, ToStringifiedPlan}; pub use datafusion_common::{JoinConstraint, JoinType}; -use std::collections::{HashMap, HashSet}; -use std::fmt::{self, Debug, Display, Formatter}; -use std::hash::{Hash, Hasher}; -use std::sync::Arc; - /// A LogicalPlan represents the different types of relational /// operators (such as Projection, Filter, etc) and can be created by /// the SQL query planner and the DataFrame API. @@ -531,11 +529,11 @@ impl LogicalPlan { // so we don't need to recompute Schema. match &self { LogicalPlan::Projection(projection) => { - Ok(LogicalPlan::Projection(Projection::try_new_with_schema( - projection.expr.to_vec(), - Arc::new(inputs[0].clone()), - projection.schema.clone(), - )?)) + // Schema of the projection may change + // when its input changes. Hence we should use + // `try_new` method instead of `try_new_with_schema`. + Projection::try_new(projection.expr.to_vec(), Arc::new(inputs[0].clone())) + .map(LogicalPlan::Projection) } LogicalPlan::Window(Window { window_expr, @@ -549,14 +547,16 @@ impl LogicalPlan { LogicalPlan::Aggregate(Aggregate { group_expr, aggr_expr, - schema, .. - }) => Ok(LogicalPlan::Aggregate(Aggregate::try_new_with_schema( + }) => Aggregate::try_new( + // Schema of the aggregate may change + // when its input changes. Hence we should use + // `try_new` method instead of `try_new_with_schema`. Arc::new(inputs[0].clone()), group_expr.to_vec(), aggr_expr.to_vec(), - schema.clone(), - )?)), + ) + .map(LogicalPlan::Aggregate), _ => self.with_new_exprs(self.expressions(), inputs), } } @@ -590,12 +590,11 @@ impl LogicalPlan { inputs: &[LogicalPlan], ) -> Result { match self { - LogicalPlan::Projection(Projection { schema, .. }) => { - Ok(LogicalPlan::Projection(Projection::try_new_with_schema( - expr, - Arc::new(inputs[0].clone()), - schema.clone(), - )?)) + // Since expr may be different than the previous expr, schema of the projection + // may change. We need to use try_new method instead of try_new_with_schema method. + LogicalPlan::Projection(Projection { .. }) => { + Projection::try_new(expr, Arc::new(inputs[0].clone())) + .map(LogicalPlan::Projection) } LogicalPlan::Dml(DmlStatement { table_name, @@ -672,10 +671,8 @@ impl LogicalPlan { let mut remove_aliases = RemoveAliases {}; let predicate = predicate.rewrite(&mut remove_aliases)?; - Ok(LogicalPlan::Filter(Filter::try_new( - predicate, - Arc::new(inputs[0].clone()), - )?)) + Filter::try_new(predicate, Arc::new(inputs[0].clone())) + .map(LogicalPlan::Filter) } LogicalPlan::Repartition(Repartition { partitioning_scheme, @@ -710,18 +707,12 @@ impl LogicalPlan { schema: schema.clone(), })) } - LogicalPlan::Aggregate(Aggregate { - group_expr, schema, .. - }) => { + LogicalPlan::Aggregate(Aggregate { group_expr, .. }) => { // group exprs are the first expressions let agg_expr = expr.split_off(group_expr.len()); - Ok(LogicalPlan::Aggregate(Aggregate::try_new_with_schema( - Arc::new(inputs[0].clone()), - expr, - agg_expr, - schema.clone(), - )?)) + Aggregate::try_new(Arc::new(inputs[0].clone()), expr, agg_expr) + .map(LogicalPlan::Aggregate) } LogicalPlan::Sort(Sort { fetch, .. }) => Ok(LogicalPlan::Sort(Sort { expr, @@ -790,10 +781,8 @@ impl LogicalPlan { })) } LogicalPlan::SubqueryAlias(SubqueryAlias { alias, .. }) => { - Ok(LogicalPlan::SubqueryAlias(SubqueryAlias::try_new( - inputs[0].clone(), - alias.clone(), - )?)) + SubqueryAlias::try_new(inputs[0].clone(), alias.clone()) + .map(LogicalPlan::SubqueryAlias) } LogicalPlan::Limit(Limit { skip, fetch, .. }) => { Ok(LogicalPlan::Limit(Limit { @@ -1953,6 +1942,63 @@ impl Hash for TableScan { } } +impl TableScan { + /// Initialize TableScan with appropriate schema from the given + /// arguments. + pub fn try_new( + table_name: impl Into, + table_source: Arc, + projection: Option>, + filters: Vec, + fetch: Option, + ) -> Result { + let table_name = table_name.into(); + + if table_name.table().is_empty() { + return plan_err!("table_name cannot be empty"); + } + let schema = table_source.schema(); + let func_dependencies = FunctionalDependencies::new_from_constraints( + table_source.constraints(), + schema.fields.len(), + ); + let projected_schema = projection + .as_ref() + .map(|p| { + let projected_func_dependencies = + func_dependencies.project_functional_dependencies(p, p.len()); + DFSchema::new_with_metadata( + p.iter() + .map(|i| { + DFField::from_qualified( + table_name.clone(), + schema.field(*i).clone(), + ) + }) + .collect(), + schema.metadata().clone(), + ) + .map(|df_schema| { + df_schema.with_functional_dependencies(projected_func_dependencies) + }) + }) + .unwrap_or_else(|| { + DFSchema::try_from_qualified_schema(table_name.clone(), &schema).map( + |df_schema| df_schema.with_functional_dependencies(func_dependencies), + ) + })?; + let projected_schema = Arc::new(projected_schema); + Ok(Self { + table_name, + source: table_source, + projection, + projected_schema, + filters, + fetch, + }) + } +} + /// Apply Cross Join to two logical plans #[derive(Clone, PartialEq, Eq, Hash)] pub struct CrossJoin { diff --git a/datafusion/expr/src/tree_node/expr.rs b/datafusion/expr/src/tree_node/expr.rs index f74cc164a7a5..764dcffbced9 100644 --- a/datafusion/expr/src/tree_node/expr.rs +++ b/datafusion/expr/src/tree_node/expr.rs @@ -22,9 +22,10 @@ use crate::expr::{ GetIndexedField, GroupingSet, InList, InSubquery, Like, Placeholder, ScalarFunction, ScalarUDF, Sort, TryCast, WindowFunction, }; -use crate::Expr; -use datafusion_common::tree_node::VisitRecursion; -use datafusion_common::{tree_node::TreeNode, Result}; +use crate::{Expr, GetFieldAccess}; + +use datafusion_common::tree_node::{TreeNode, VisitRecursion}; +use datafusion_common::Result; impl TreeNode for Expr { fn apply_children(&self, op: &mut F) -> Result @@ -47,8 +48,19 @@ impl TreeNode for Expr { | Expr::TryCast(TryCast { expr, .. }) | Expr::Sort(Sort { expr, .. }) | Expr::InSubquery(InSubquery{ expr, .. }) => vec![expr.as_ref().clone()], - Expr::GetIndexedField(GetIndexedField { expr, .. }) => { - vec![expr.as_ref().clone()] + Expr::GetIndexedField(GetIndexedField { expr, field }) => { + let expr = expr.as_ref().clone(); + match field { + GetFieldAccess::ListIndex {key} => { + vec![key.as_ref().clone(), expr] + }, + GetFieldAccess::ListRange {start, stop} => { + vec![start.as_ref().clone(), stop.as_ref().clone(), expr] + } + GetFieldAccess::NamedStructField {name: _name} => { + vec![expr] + } + } } Expr::GroupingSet(GroupingSet::Rollup(exprs)) | Expr::GroupingSet(GroupingSet::Cube(exprs)) => exprs.clone(), diff --git a/datafusion/expr/src/type_coercion/binary.rs b/datafusion/expr/src/type_coercion/binary.rs index a854373e880d..cf93d15e23f0 100644 --- a/datafusion/expr/src/type_coercion/binary.rs +++ b/datafusion/expr/src/type_coercion/binary.rs @@ -17,17 +17,20 @@ //! Coercion rules for matching argument types for binary operators +use std::sync::Arc; + +use crate::Operator; + use arrow::array::{new_empty_array, Array}; use arrow::compute::can_cast_types; use arrow::datatypes::{ - DataType, TimeUnit, DECIMAL128_MAX_PRECISION, DECIMAL128_MAX_SCALE, + DataType, Field, TimeUnit, DECIMAL128_MAX_PRECISION, DECIMAL128_MAX_SCALE, DECIMAL256_MAX_PRECISION, DECIMAL256_MAX_SCALE, }; -use datafusion_common::{plan_datafusion_err, Result}; -use datafusion_common::{plan_err, DataFusionError}; - -use crate::Operator; +use datafusion_common::{ + exec_datafusion_err, plan_datafusion_err, plan_err, DataFusionError, Result, +}; /// The type signature of an instantiation of binary operator expression such as /// `lhs + rhs` @@ -65,83 +68,75 @@ impl Signature { /// Returns a [`Signature`] for applying `op` to arguments of type `lhs` and `rhs` fn signature(lhs: &DataType, op: &Operator, rhs: &DataType) -> Result { + use arrow::datatypes::DataType::*; + use Operator::*; match op { - Operator::Eq | - Operator::NotEq | - Operator::Lt | - Operator::LtEq | - Operator::Gt | - Operator::GtEq | - Operator::IsDistinctFrom | - Operator::IsNotDistinctFrom => { + Eq | + NotEq | + Lt | + LtEq | + Gt | + GtEq | + IsDistinctFrom | + IsNotDistinctFrom => { comparison_coercion(lhs, rhs).map(Signature::comparison).ok_or_else(|| { plan_datafusion_err!( "Cannot infer common argument type for comparison operation {lhs} {op} {rhs}" ) }) } - Operator::And | Operator::Or => match (lhs, rhs) { - // logical binary boolean operators can only be evaluated in bools or nulls - (DataType::Boolean, DataType::Boolean) - | (DataType::Null, DataType::Null) - | (DataType::Boolean, DataType::Null) - | (DataType::Null, DataType::Boolean) => Ok(Signature::uniform(DataType::Boolean)), - _ => plan_err!( + And | Or => if matches!((lhs, rhs), (Boolean | Null, Boolean | Null)) { + // Logical binary boolean operators can only be evaluated for + // boolean or null arguments. + Ok(Signature::uniform(DataType::Boolean)) + } else { + plan_err!( "Cannot infer common argument type for logical boolean operation {lhs} {op} {rhs}" - ), - }, - Operator::RegexMatch | - Operator::RegexIMatch | - Operator::RegexNotMatch | - Operator::RegexNotIMatch => { + ) + } + RegexMatch | RegexIMatch | RegexNotMatch | RegexNotIMatch => { regex_coercion(lhs, rhs).map(Signature::comparison).ok_or_else(|| { plan_datafusion_err!( "Cannot infer common argument type for regex operation {lhs} {op} {rhs}" ) }) } - Operator::BitwiseAnd - | Operator::BitwiseOr - | Operator::BitwiseXor - | Operator::BitwiseShiftRight - | Operator::BitwiseShiftLeft => { + BitwiseAnd | BitwiseOr | BitwiseXor | BitwiseShiftRight | BitwiseShiftLeft => { bitwise_coercion(lhs, rhs).map(Signature::uniform).ok_or_else(|| { plan_datafusion_err!( "Cannot infer common type for bitwise operation {lhs} {op} {rhs}" ) }) } - Operator::StringConcat => { + StringConcat => { string_concat_coercion(lhs, rhs).map(Signature::uniform).ok_or_else(|| { plan_datafusion_err!( "Cannot infer common string type for string concat operation {lhs} {op} {rhs}" ) }) } - Operator::AtArrow - | Operator::ArrowAt => { - array_coercion(lhs, rhs).map(Signature::uniform).ok_or_else(|| { + AtArrow | ArrowAt => { + // ArrowAt and AtArrow check for whether one array ic contained in another. + // The result type is boolean. Signature::comparison defines this signature. + // Operation has nothing to do with comparison + array_coercion(lhs, rhs).map(Signature::comparison).ok_or_else(|| { plan_datafusion_err!( "Cannot infer common array type for arrow operation {lhs} {op} {rhs}" ) }) } - Operator::Plus | - Operator::Minus | - Operator::Multiply | - Operator::Divide| - Operator::Modulo => { + Plus | Minus | Multiply | Divide | Modulo => { let get_result = |lhs, rhs| { use arrow::compute::kernels::numeric::*; let l = new_empty_array(lhs); let r = new_empty_array(rhs); let result = match op { - Operator::Plus => add_wrapping(&l, &r), - Operator::Minus => sub_wrapping(&l, &r), - Operator::Multiply => mul_wrapping(&l, &r), - Operator::Divide => div(&l, &r), - Operator::Modulo => rem(&l, &r), + Plus => add_wrapping(&l, &r), + Minus => sub_wrapping(&l, &r), + Multiply => mul_wrapping(&l, &r), + Divide => div(&l, &r), + Modulo => rem(&l, &r), _ => unreachable!(), }; result.map(|x| x.data_type().clone()) @@ -228,7 +223,7 @@ fn math_decimal_coercion( (Null, dec_type @ Decimal128(_, _)) | (dec_type @ Decimal128(_, _), Null) => { Some((dec_type.clone(), dec_type.clone())) } - (Decimal128(_, _), Decimal128(_, _)) => { + (Decimal128(_, _), Decimal128(_, _)) | (Decimal256(_, _), Decimal256(_, _)) => { Some((lhs_type.clone(), rhs_type.clone())) } // Unlike with comparison we don't coerce to a decimal in the case of floating point @@ -239,9 +234,6 @@ fn math_decimal_coercion( (Int8 | Int16 | Int32 | Int64, Decimal128(_, _)) => { Some((coerce_numeric_type_to_decimal(lhs_type)?, rhs_type.clone())) } - (Decimal256(_, _), Decimal256(_, _)) => { - Some((lhs_type.clone(), rhs_type.clone())) - } (Decimal256(_, _), Int8 | Int16 | Int32 | Int64) => Some(( lhs_type.clone(), coerce_numeric_type_to_decimal256(rhs_type)?, @@ -473,6 +465,54 @@ fn get_wider_decimal_type( } } +/// Returns the wider type among arguments `lhs` and `rhs`. +/// The wider type is the type that can safely represent values from both types +/// without information loss. Returns an Error if types are incompatible. +pub fn get_wider_type(lhs: &DataType, rhs: &DataType) -> Result { + use arrow::datatypes::DataType::*; + Ok(match (lhs, rhs) { + (lhs, rhs) if lhs == rhs => lhs.clone(), + // Right UInt is larger than left UInt. + (UInt8, UInt16 | UInt32 | UInt64) | (UInt16, UInt32 | UInt64) | (UInt32, UInt64) | + // Right Int is larger than left Int. + (Int8, Int16 | Int32 | Int64) | (Int16, Int32 | Int64) | (Int32, Int64) | + // Right Float is larger than left Float. + (Float16, Float32 | Float64) | (Float32, Float64) | + // Right String is larger than left String. + (Utf8, LargeUtf8) | + // Any right type is wider than a left hand side Null. + (Null, _) => rhs.clone(), + // Left UInt is larger than right UInt. + (UInt16 | UInt32 | UInt64, UInt8) | (UInt32 | UInt64, UInt16) | (UInt64, UInt32) | + // Left Int is larger than right Int. + (Int16 | Int32 | Int64, Int8) | (Int32 | Int64, Int16) | (Int64, Int32) | + // Left Float is larger than right Float. + (Float32 | Float64, Float16) | (Float64, Float32) | + // Left String is larget than right String. + (LargeUtf8, Utf8) | + // Any left type is wider than a right hand side Null. + (_, Null) => lhs.clone(), + (List(lhs_field), List(rhs_field)) => { + let field_type = + get_wider_type(lhs_field.data_type(), rhs_field.data_type())?; + if lhs_field.name() != rhs_field.name() { + return Err(exec_datafusion_err!( + "There is no wider type that can represent both {lhs} and {rhs}." + )); + } + assert_eq!(lhs_field.name(), rhs_field.name()); + let field_name = lhs_field.name(); + let nullable = lhs_field.is_nullable() | rhs_field.is_nullable(); + List(Arc::new(Field::new(field_name, field_type, nullable))) + } + (_, _) => { + return Err(exec_datafusion_err!( + "There is no wider type that can represent both {lhs} and {rhs}." + )); + } + }) +} + /// Convert the numeric data type to the decimal data type. /// Now, we just support the signed integer type and floating-point type. fn coerce_numeric_type_to_decimal(numeric_type: &DataType) -> Option { @@ -808,14 +848,11 @@ fn null_coercion(lhs_type: &DataType, rhs_type: &DataType) -> Option { #[cfg(test)] mod tests { - use arrow::datatypes::DataType; - - use datafusion_common::assert_contains; - use datafusion_common::Result; - + use super::*; use crate::Operator; - use super::*; + use arrow::datatypes::DataType; + use datafusion_common::{assert_contains, Result}; #[test] fn test_coercion_error() -> Result<()> { diff --git a/datafusion/optimizer/src/common_subexpr_eliminate.rs b/datafusion/optimizer/src/common_subexpr_eliminate.rs index c6b138f8ca36..68a6a5607a1d 100644 --- a/datafusion/optimizer/src/common_subexpr_eliminate.rs +++ b/datafusion/optimizer/src/common_subexpr_eliminate.rs @@ -20,6 +20,8 @@ use std::collections::{BTreeSet, HashMap}; use std::sync::Arc; +use crate::{utils, OptimizerConfig, OptimizerRule}; + use arrow::datatypes::DataType; use datafusion_common::tree_node::{ RewriteRecursion, TreeNode, TreeNodeRewriter, TreeNodeVisitor, VisitRecursion, @@ -28,13 +30,10 @@ use datafusion_common::{ internal_err, Column, DFField, DFSchema, DFSchemaRef, DataFusionError, Result, }; use datafusion_expr::expr::Alias; -use datafusion_expr::{ - col, - logical_plan::{Aggregate, Filter, LogicalPlan, Projection, Sort, Window}, - Expr, ExprSchemable, +use datafusion_expr::logical_plan::{ + Aggregate, Filter, LogicalPlan, Projection, Sort, Window, }; - -use crate::{utils, OptimizerConfig, OptimizerRule}; +use datafusion_expr::{col, Expr, ExprSchemable}; /// A map from expression's identifier to tuple including /// - the expression itself (cloned) @@ -111,12 +110,7 @@ impl CommonSubexprEliminate { projection: &Projection, config: &dyn OptimizerConfig, ) -> Result { - let Projection { - expr, - input, - schema, - .. - } = projection; + let Projection { expr, input, .. } = projection; let input_schema = Arc::clone(input.schema()); let mut expr_set = ExprSet::new(); let arrays = to_arrays(expr, input_schema, &mut expr_set, ExprMask::Normal)?; @@ -124,11 +118,9 @@ impl CommonSubexprEliminate { let (mut new_expr, new_input) = self.rewrite_expr(&[expr], &[&arrays], input, &expr_set, config)?; - Ok(LogicalPlan::Projection(Projection::try_new_with_schema( - pop_expr(&mut new_expr)?, - Arc::new(new_input), - schema.clone(), - )?)) + // Since projection expr changes, schema changes also. Use try_new method. + Projection::try_new(pop_expr(&mut new_expr)?, Arc::new(new_input)) + .map(LogicalPlan::Projection) } fn try_optimize_filter( @@ -201,7 +193,6 @@ impl CommonSubexprEliminate { group_expr, aggr_expr, input, - schema, .. } = aggregate; let mut expr_set = ExprSet::new(); @@ -247,12 +238,9 @@ impl CommonSubexprEliminate { let rewritten = pop_expr(&mut rewritten)?; if affected_id.is_empty() { - Ok(LogicalPlan::Aggregate(Aggregate::try_new_with_schema( - Arc::new(new_input), - new_group_expr, - new_aggr_expr, - schema.clone(), - )?)) + // Since group_epxr changes, schema changes also. Use try_new method. + Aggregate::try_new(Arc::new(new_input), new_group_expr, new_aggr_expr) + .map(LogicalPlan::Aggregate) } else { let mut agg_exprs = vec![]; diff --git a/datafusion/optimizer/src/merge_projection.rs b/datafusion/optimizer/src/merge_projection.rs index 408055b8e7d4..ec040cba6fe4 100644 --- a/datafusion/optimizer/src/merge_projection.rs +++ b/datafusion/optimizer/src/merge_projection.rs @@ -15,14 +15,15 @@ // specific language governing permissions and limitations // under the License. -use crate::optimizer::ApplyOrder; -use datafusion_common::Result; -use datafusion_expr::{Expr, LogicalPlan, Projection}; use std::collections::HashMap; +use crate::optimizer::ApplyOrder; use crate::push_down_filter::replace_cols_by_name; use crate::{OptimizerConfig, OptimizerRule}; +use datafusion_common::Result; +use datafusion_expr::{Expr, LogicalPlan, Projection}; + /// Optimization rule that merge [LogicalPlan::Projection]. #[derive(Default)] pub struct MergeProjection; @@ -84,10 +85,10 @@ pub(super) fn merge_projection( Err(e) => Err(e), }) .collect::>>()?; - let new_plan = LogicalPlan::Projection(Projection::try_new_with_schema( + // Use try_new, since schema changes with changing expressions. + let new_plan = LogicalPlan::Projection(Projection::try_new( new_exprs, child_projection.input.clone(), - parent_projection.schema.clone(), )?); Ok(new_plan) } diff --git a/datafusion/optimizer/src/push_down_projection.rs b/datafusion/optimizer/src/push_down_projection.rs index e7fdaa8b0b5e..b05d811cb481 100644 --- a/datafusion/optimizer/src/push_down_projection.rs +++ b/datafusion/optimizer/src/push_down_projection.rs @@ -18,6 +18,9 @@ //! Projection Push Down optimizer rule ensures that only referenced columns are //! loaded into memory +use std::collections::{BTreeSet, HashMap, HashSet}; +use std::sync::Arc; + use crate::eliminate_project::can_eliminate; use crate::merge_projection::merge_projection; use crate::optimizer::ApplyOrder; @@ -26,20 +29,14 @@ use crate::{OptimizerConfig, OptimizerRule}; use arrow::error::Result as ArrowResult; use datafusion_common::ScalarValue::UInt8; use datafusion_common::{ - plan_err, Column, DFField, DFSchema, DFSchemaRef, DataFusionError, Result, ToDFSchema, + plan_err, Column, DFSchema, DFSchemaRef, DataFusionError, Result, }; use datafusion_expr::expr::{AggregateFunction, Alias}; -use datafusion_expr::utils::exprlist_to_fields; use datafusion_expr::{ logical_plan::{Aggregate, LogicalPlan, Projection, TableScan, Union}, - utils::{expr_to_columns, exprlist_to_columns}, + utils::{expr_to_columns, exprlist_to_columns, exprlist_to_fields}, Expr, LogicalPlanBuilder, SubqueryAlias, }; -use std::collections::HashMap; -use std::{ - collections::{BTreeSet, HashSet}, - sync::Arc, -}; // if projection is empty return projection-new_plan, else return new_plan. #[macro_export] @@ -501,24 +498,14 @@ fn push_down_scan( projection.into_iter().collect::>() }; - // create the projected schema - let projected_fields: Vec = projection - .iter() - .map(|i| { - DFField::from_qualified(scan.table_name.clone(), schema.fields()[*i].clone()) - }) - .collect(); - - let projected_schema = projected_fields.to_dfschema_ref()?; - - Ok(LogicalPlan::TableScan(TableScan { - table_name: scan.table_name.clone(), - source: scan.source.clone(), - projection: Some(projection), - projected_schema, - filters: scan.filters.clone(), - fetch: scan.fetch, - })) + TableScan::try_new( + scan.table_name.clone(), + scan.source.clone(), + Some(projection), + scan.filters.clone(), + scan.fetch, + ) + .map(LogicalPlan::TableScan) } fn restrict_outputs( @@ -538,25 +525,24 @@ fn restrict_outputs( #[cfg(test)] mod tests { + use std::collections::HashMap; + use std::vec; + use super::*; use crate::eliminate_project::EliminateProjection; use crate::optimizer::Optimizer; use crate::test::*; use crate::OptimizerContext; use arrow::datatypes::{DataType, Field, Schema}; - use datafusion_common::DFSchema; + use datafusion_common::{DFField, DFSchema}; use datafusion_expr::builder::table_scan_with_filters; - use datafusion_expr::expr; - use datafusion_expr::expr::Cast; - use datafusion_expr::WindowFrame; - use datafusion_expr::WindowFunction; + use datafusion_expr::expr::{self, Cast}; + use datafusion_expr::logical_plan::{ + builder::LogicalPlanBuilder, table_scan, JoinType, + }; use datafusion_expr::{ - col, count, lit, - logical_plan::{builder::LogicalPlanBuilder, table_scan, JoinType}, - max, min, AggregateFunction, Expr, + col, count, lit, max, min, AggregateFunction, Expr, WindowFrame, WindowFunction, }; - use std::collections::HashMap; - use std::vec; #[test] fn aggregate_no_group_by() -> Result<()> { diff --git a/datafusion/optimizer/src/replace_distinct_aggregate.rs b/datafusion/optimizer/src/replace_distinct_aggregate.rs index f58d4b159745..540617b77084 100644 --- a/datafusion/optimizer/src/replace_distinct_aggregate.rs +++ b/datafusion/optimizer/src/replace_distinct_aggregate.rs @@ -15,13 +15,12 @@ // specific language governing permissions and limitations // under the License. -use crate::optimizer::ApplyOrder; +use crate::optimizer::{ApplyOrder, ApplyOrder::BottomUp}; use crate::{OptimizerConfig, OptimizerRule}; + use datafusion_common::Result; use datafusion_expr::utils::expand_wildcard; -use datafusion_expr::Distinct; -use datafusion_expr::{Aggregate, LogicalPlan}; -use ApplyOrder::BottomUp; +use datafusion_expr::{Aggregate, Distinct, LogicalPlan}; /// Optimizer that replaces logical [[Distinct]] with a logical [[Aggregate]] /// @@ -54,11 +53,10 @@ impl OptimizerRule for ReplaceDistinctWithAggregate { match plan { LogicalPlan::Distinct(Distinct { input }) => { let group_expr = expand_wildcard(input.schema(), input, None)?; - let aggregate = LogicalPlan::Aggregate(Aggregate::try_new_with_schema( + let aggregate = LogicalPlan::Aggregate(Aggregate::try_new( input.clone(), group_expr, vec![], - input.schema().clone(), // input schema and aggregate schema are the same in this case )?); Ok(Some(aggregate)) } diff --git a/datafusion/optimizer/src/single_distinct_to_groupby.rs b/datafusion/optimizer/src/single_distinct_to_groupby.rs index a9e65b3e7c77..8e0f93cb5781 100644 --- a/datafusion/optimizer/src/single_distinct_to_groupby.rs +++ b/datafusion/optimizer/src/single_distinct_to_groupby.rs @@ -17,8 +17,11 @@ //! single distinct to group by optimizer rule +use std::sync::Arc; + use crate::optimizer::ApplyOrder; use crate::{OptimizerConfig, OptimizerRule}; + use datafusion_common::{DFSchema, Result}; use datafusion_expr::{ col, @@ -27,8 +30,8 @@ use datafusion_expr::{ utils::columnize_expr, Expr, ExprSchemable, }; + use hashbrown::HashSet; -use std::sync::Arc; /// single distinct to group by optimizer rule /// ```text @@ -102,24 +105,50 @@ impl OptimizerRule for SingleDistinctToGroupBy { .. }) => { if is_single_distinct_agg(plan)? && !contains_grouping_set(group_expr) { + let fields = schema.fields(); // alias all original group_by exprs - let mut group_expr_alias = Vec::with_capacity(group_expr.len()); - let mut inner_group_exprs = group_expr + let (mut inner_group_exprs, out_group_expr_with_alias): ( + Vec, + Vec<(Expr, Option)>, + ) = group_expr .iter() .enumerate() .map(|(i, group_expr)| { - let alias_str = format!("group_alias_{i}"); - let alias_expr = group_expr.clone().alias(&alias_str); - group_expr_alias - .push((alias_str, schema.fields()[i].clone())); - alias_expr + if let Expr::Column(_) = group_expr { + // For Column expressions we can use existing expression as is. + (group_expr.clone(), (group_expr.clone(), None)) + } else { + // For complex expression write is as alias, to be able to refer + // if from parent operators successfully. + // Consider plan below. + // + // Aggregate: groupBy=[[group_alias_0]], aggr=[[COUNT(alias1)]] [group_alias_0:Int32, COUNT(alias1):Int64;N]\ + // --Aggregate: groupBy=[[test.a + Int32(1) AS group_alias_0, test.c AS alias1]], aggr=[[]] [group_alias_0:Int32, alias1:UInt32]\ + // ----TableScan: test [a:UInt32, b:UInt32, c:UInt32] + // + // First aggregate(from bottom) refers to `test.a` column. + // Second aggregate refers to the `group_alias_0` column, Which is a valid field in the first aggregate. + // If we were to write plan above as below without alias + // + // Aggregate: groupBy=[[test.a + Int32(1)]], aggr=[[COUNT(alias1)]] [group_alias_0:Int32, COUNT(alias1):Int64;N]\ + // --Aggregate: groupBy=[[test.a + Int32(1), test.c AS alias1]], aggr=[[]] [group_alias_0:Int32, alias1:UInt32]\ + // ----TableScan: test [a:UInt32, b:UInt32, c:UInt32] + // + // Second aggregate refers to the `test.a + Int32(1)` expression However, its input do not have `test.a` expression in it. + let alias_str = format!("group_alias_{i}"); + let alias_expr = group_expr.clone().alias(&alias_str); + ( + alias_expr, + (col(alias_str), Some(fields[i].qualified_name())), + ) + } }) - .collect::>(); + .unzip(); // and they can be referenced by the alias in the outer aggr plan - let outer_group_exprs = group_expr_alias + let outer_group_exprs = out_group_expr_with_alias .iter() - .map(|(alias, _)| col(alias)) + .map(|(out_group_expr, _)| out_group_expr.clone()) .collect::>(); // replace the distinct arg with alias @@ -181,20 +210,22 @@ impl OptimizerRule for SingleDistinctToGroupBy { // this optimizer has two kinds of alias: // - group_by aggr // - aggr expr - let mut alias_expr: Vec = Vec::new(); - for (alias, original_field) in group_expr_alias { - alias_expr - .push(col(alias).alias(original_field.qualified_name())); - } - for (i, expr) in new_aggr_exprs.iter().enumerate() { - alias_expr.push(columnize_expr( - expr.clone().alias( - schema.clone().fields()[i + group_expr.len()] - .qualified_name(), - ), - &outer_aggr_schema, - )); - } + let group_size = group_expr.len(); + let alias_expr = out_group_expr_with_alias + .into_iter() + .map(|(group_expr, original_field)| { + if let Some(name) = original_field { + group_expr.alias(name) + } else { + group_expr + } + }) + .chain(new_aggr_exprs.iter().enumerate().map(|(idx, expr)| { + let idx = idx + group_size; + let name = fields[idx].qualified_name(); + columnize_expr(expr.clone().alias(name), &outer_aggr_schema) + })) + .collect(); let outer_aggr = LogicalPlan::Aggregate(Aggregate::try_new( Arc::new(inner_agg), @@ -202,13 +233,10 @@ impl OptimizerRule for SingleDistinctToGroupBy { new_aggr_exprs, )?); - Ok(Some(LogicalPlan::Projection( - Projection::try_new_with_schema( - alias_expr, - Arc::new(outer_aggr), - schema.clone(), - )?, - ))) + Ok(Some(LogicalPlan::Projection(Projection::try_new( + alias_expr, + Arc::new(outer_aggr), + )?))) } else { Ok(None) } @@ -362,9 +390,9 @@ mod tests { .build()?; // Should work - let expected = "Projection: group_alias_0 AS test.a, COUNT(alias1) AS COUNT(DISTINCT test.b) [a:UInt32, COUNT(DISTINCT test.b):Int64;N]\ - \n Aggregate: groupBy=[[group_alias_0]], aggr=[[COUNT(alias1)]] [group_alias_0:UInt32, COUNT(alias1):Int64;N]\ - \n Aggregate: groupBy=[[test.a AS group_alias_0, test.b AS alias1]], aggr=[[]] [group_alias_0:UInt32, alias1:UInt32]\ + let expected = "Projection: test.a, COUNT(alias1) AS COUNT(DISTINCT test.b) [a:UInt32, COUNT(DISTINCT test.b):Int64;N]\ + \n Aggregate: groupBy=[[test.a]], aggr=[[COUNT(alias1)]] [a:UInt32, COUNT(alias1):Int64;N]\ + \n Aggregate: groupBy=[[test.a, test.b AS alias1]], aggr=[[]] [a:UInt32, alias1:UInt32]\ \n TableScan: test [a:UInt32, b:UInt32, c:UInt32]"; assert_optimized_plan_equal(&plan, expected) @@ -408,9 +436,9 @@ mod tests { )? .build()?; // Should work - let expected = "Projection: group_alias_0 AS test.a, COUNT(alias1) AS COUNT(DISTINCT test.b), MAX(alias1) AS MAX(DISTINCT test.b) [a:UInt32, COUNT(DISTINCT test.b):Int64;N, MAX(DISTINCT test.b):UInt32;N]\ - \n Aggregate: groupBy=[[group_alias_0]], aggr=[[COUNT(alias1), MAX(alias1)]] [group_alias_0:UInt32, COUNT(alias1):Int64;N, MAX(alias1):UInt32;N]\ - \n Aggregate: groupBy=[[test.a AS group_alias_0, test.b AS alias1]], aggr=[[]] [group_alias_0:UInt32, alias1:UInt32]\ + let expected = "Projection: test.a, COUNT(alias1) AS COUNT(DISTINCT test.b), MAX(alias1) AS MAX(DISTINCT test.b) [a:UInt32, COUNT(DISTINCT test.b):Int64;N, MAX(DISTINCT test.b):UInt32;N]\ + \n Aggregate: groupBy=[[test.a]], aggr=[[COUNT(alias1), MAX(alias1)]] [a:UInt32, COUNT(alias1):Int64;N, MAX(alias1):UInt32;N]\ + \n Aggregate: groupBy=[[test.a, test.b AS alias1]], aggr=[[]] [a:UInt32, alias1:UInt32]\ \n TableScan: test [a:UInt32, b:UInt32, c:UInt32]"; assert_optimized_plan_equal(&plan, expected) diff --git a/datafusion/physical-expr/src/array_expressions.rs b/datafusion/physical-expr/src/array_expressions.rs index 067a4cfdffc0..af4612272676 100644 --- a/datafusion/physical-expr/src/array_expressions.rs +++ b/datafusion/physical-expr/src/array_expressions.rs @@ -17,18 +17,22 @@ //! Array expressions +use std::any::type_name; +use std::sync::Arc; + use arrow::array::*; use arrow::buffer::OffsetBuffer; use arrow::compute; use arrow::datatypes::{DataType, Field, UInt64Type}; use arrow_buffer::NullBuffer; -use core::any::type_name; use datafusion_common::cast::{as_generic_string_array, as_int64_array, as_list_array}; -use datafusion_common::{exec_err, internal_err, not_impl_err, plan_err, ScalarValue}; -use datafusion_common::{DataFusionError, Result}; +use datafusion_common::utils::wrap_into_list_array; +use datafusion_common::{ + exec_err, internal_err, not_impl_err, plan_err, DataFusionError, Result, +}; use datafusion_expr::ColumnarValue; + use itertools::Itertools; -use std::sync::Arc; macro_rules! downcast_arg { ($ARG:expr, $ARRAY_TYPE:ident) => {{ @@ -400,34 +404,26 @@ fn array(values: &[ColumnarValue]) -> Result { .iter() .map(|x| match x { ColumnarValue::Array(array) => array.clone(), - ColumnarValue::Scalar(scalar) => scalar.to_array().clone(), + ColumnarValue::Scalar(scalar) => scalar.to_array(), }) .collect(); - let mut data_type = None; + let mut data_type = DataType::Null; for arg in &arrays { let arg_data_type = arg.data_type(); if !arg_data_type.equals_datatype(&DataType::Null) { - data_type = Some(arg_data_type.clone()); + data_type = arg_data_type.clone(); break; - } else { - data_type = Some(DataType::Null); } } match data_type { - // empty array - None => { - let list_arr = ScalarValue::new_list(&[], &DataType::Null); - Ok(Arc::new(list_arr)) - } - // all nulls, set default data type as int32 - Some(DataType::Null) => { - let null_arr = vec![ScalarValue::Int32(None); arrays.len()]; - let list_arr = ScalarValue::new_list(null_arr.as_slice(), &DataType::Int32); - Ok(Arc::new(list_arr)) + // Either an empty array or all nulls: + DataType::Null => { + let array = new_null_array(&DataType::Null, arrays.len()); + Ok(Arc::new(wrap_into_list_array(array))) } - Some(data_type) => Ok(array_array(arrays.as_slice(), data_type)?), + data_type => array_array(arrays.as_slice(), data_type), } } @@ -988,7 +984,8 @@ macro_rules! general_repeat_list { /// Array_empty SQL function pub fn array_empty(args: &[ArrayRef]) -> Result { if args[0].as_any().downcast_ref::().is_some() { - return Ok(args[0].clone()); + // Make sure to return Boolean type. + return Ok(Arc::new(BooleanArray::new_null(args[0].len()))); } let array = as_list_array(&args[0])?; diff --git a/datafusion/sqllogictest/test_files/array.slt b/datafusion/sqllogictest/test_files/array.slt index f11bc5206eb4..621cb4a8f4c0 100644 --- a/datafusion/sqllogictest/test_files/array.slt +++ b/datafusion/sqllogictest/test_files/array.slt @@ -2322,7 +2322,7 @@ select 1 || make_array(2, 3, 4), 1.0 || make_array(2.0, 3.0, 4.0), 'h' || make_a ## array containment operator # array containment operator with scalars #1 (at arrow) -query ??????? +query BBBBBBB select make_array(1,2,3) @> make_array(1,3), make_array(1,2,3) @> make_array(1,4), make_array([1,2], [3,4]) @> make_array([1,2]), @@ -2334,7 +2334,7 @@ select make_array(1,2,3) @> make_array(1,3), true false true false false false true # array containment operator with scalars #2 (arrow at) -query ??????? +query BBBBBBB select make_array(1,3) <@ make_array(1,2,3), make_array(1,4) <@ make_array(1,2,3), make_array([1,2]) <@ make_array([1,2], [3,4]), @@ -2465,7 +2465,7 @@ true query B select empty(make_array(NULL)); ---- -true +false # empty scalar function #4 query B diff --git a/datafusion/sqllogictest/test_files/tpch/q16.slt.part b/datafusion/sqllogictest/test_files/tpch/q16.slt.part index fb9d98b76fe3..b93872929fe5 100644 --- a/datafusion/sqllogictest/test_files/tpch/q16.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/q16.slt.part @@ -52,9 +52,9 @@ limit 10; logical_plan Limit: skip=0, fetch=10 --Sort: supplier_cnt DESC NULLS FIRST, part.p_brand ASC NULLS LAST, part.p_type ASC NULLS LAST, part.p_size ASC NULLS LAST, fetch=10 -----Projection: group_alias_0 AS part.p_brand, group_alias_1 AS part.p_type, group_alias_2 AS part.p_size, COUNT(alias1) AS supplier_cnt -------Aggregate: groupBy=[[group_alias_0, group_alias_1, group_alias_2]], aggr=[[COUNT(alias1)]] ---------Aggregate: groupBy=[[part.p_brand AS group_alias_0, part.p_type AS group_alias_1, part.p_size AS group_alias_2, partsupp.ps_suppkey AS alias1]], aggr=[[]] +----Projection: part.p_brand, part.p_type, part.p_size, COUNT(alias1) AS supplier_cnt +------Aggregate: groupBy=[[part.p_brand, part.p_type, part.p_size]], aggr=[[COUNT(alias1)]] +--------Aggregate: groupBy=[[part.p_brand, part.p_type, part.p_size, partsupp.ps_suppkey AS alias1]], aggr=[[]] ----------LeftAnti Join: partsupp.ps_suppkey = __correlated_sq_1.s_suppkey ------------Projection: partsupp.ps_suppkey, part.p_brand, part.p_type, part.p_size --------------Inner Join: partsupp.ps_partkey = part.p_partkey @@ -69,15 +69,15 @@ physical_plan GlobalLimitExec: skip=0, fetch=10 --SortPreservingMergeExec: [supplier_cnt@3 DESC,p_brand@0 ASC NULLS LAST,p_type@1 ASC NULLS LAST,p_size@2 ASC NULLS LAST], fetch=10 ----SortExec: TopK(fetch=10), expr=[supplier_cnt@3 DESC,p_brand@0 ASC NULLS LAST,p_type@1 ASC NULLS LAST,p_size@2 ASC NULLS LAST] -------ProjectionExec: expr=[group_alias_0@0 as part.p_brand, group_alias_1@1 as part.p_type, group_alias_2@2 as part.p_size, COUNT(alias1)@3 as supplier_cnt] ---------AggregateExec: mode=FinalPartitioned, gby=[group_alias_0@0 as group_alias_0, group_alias_1@1 as group_alias_1, group_alias_2@2 as group_alias_2], aggr=[COUNT(alias1)] +------ProjectionExec: expr=[p_brand@0 as p_brand, p_type@1 as p_type, p_size@2 as p_size, COUNT(alias1)@3 as supplier_cnt] +--------AggregateExec: mode=FinalPartitioned, gby=[p_brand@0 as p_brand, p_type@1 as p_type, p_size@2 as p_size], aggr=[COUNT(alias1)] ----------CoalesceBatchesExec: target_batch_size=8192 -------------RepartitionExec: partitioning=Hash([group_alias_0@0, group_alias_1@1, group_alias_2@2], 4), input_partitions=4 ---------------AggregateExec: mode=Partial, gby=[group_alias_0@0 as group_alias_0, group_alias_1@1 as group_alias_1, group_alias_2@2 as group_alias_2], aggr=[COUNT(alias1)] -----------------AggregateExec: mode=FinalPartitioned, gby=[group_alias_0@0 as group_alias_0, group_alias_1@1 as group_alias_1, group_alias_2@2 as group_alias_2, alias1@3 as alias1], aggr=[] +------------RepartitionExec: partitioning=Hash([p_brand@0, p_type@1, p_size@2], 4), input_partitions=4 +--------------AggregateExec: mode=Partial, gby=[p_brand@0 as p_brand, p_type@1 as p_type, p_size@2 as p_size], aggr=[COUNT(alias1)] +----------------AggregateExec: mode=FinalPartitioned, gby=[p_brand@0 as p_brand, p_type@1 as p_type, p_size@2 as p_size, alias1@3 as alias1], aggr=[] ------------------CoalesceBatchesExec: target_batch_size=8192 ---------------------RepartitionExec: partitioning=Hash([group_alias_0@0, group_alias_1@1, group_alias_2@2, alias1@3], 4), input_partitions=4 -----------------------AggregateExec: mode=Partial, gby=[p_brand@1 as group_alias_0, p_type@2 as group_alias_1, p_size@3 as group_alias_2, ps_suppkey@0 as alias1], aggr=[] +--------------------RepartitionExec: partitioning=Hash([p_brand@0, p_type@1, p_size@2, alias1@3], 4), input_partitions=4 +----------------------AggregateExec: mode=Partial, gby=[p_brand@1 as p_brand, p_type@2 as p_type, p_size@3 as p_size, ps_suppkey@0 as alias1], aggr=[] ------------------------CoalesceBatchesExec: target_batch_size=8192 --------------------------HashJoinExec: mode=Partitioned, join_type=LeftAnti, on=[(ps_suppkey@0, s_suppkey@0)] ----------------------------CoalesceBatchesExec: target_batch_size=8192 From 148f890e212328399687cadac8bee022d829785d Mon Sep 17 00:00:00 2001 From: Devin D'Angelo Date: Wed, 25 Oct 2023 12:50:03 -0400 Subject: [PATCH 128/572] Parallelize Serialization of Columns within Parquet RowGroups (#7655) * merge main * fixes and cmt * review comments, tuning parameters, updating docs * cargo fmt * reduce default buffer size to 2 and update docs --- datafusion/common/src/config.rs | 32 +- .../src/datasource/file_format/parquet.rs | 541 +++++++++++------- .../test_files/information_schema.slt | 8 +- docs/source/user-guide/configs.md | 136 ++--- 4 files changed, 439 insertions(+), 278 deletions(-) diff --git a/datafusion/common/src/config.rs b/datafusion/common/src/config.rs index 6aefa4e05de2..403241fcce58 100644 --- a/datafusion/common/src/config.rs +++ b/datafusion/common/src/config.rs @@ -377,12 +377,32 @@ config_namespace! { pub bloom_filter_ndv: Option, default = None /// Controls whether DataFusion will attempt to speed up writing - /// large parquet files by first writing multiple smaller files - /// and then stitching them together into a single large file. - /// This will result in faster write speeds, but higher memory usage. - /// Also currently unsupported are bloom filters and column indexes - /// when single_file_parallelism is enabled. - pub allow_single_file_parallelism: bool, default = false + /// parquet files by serializing them in parallel. Each column + /// in each row group in each output file are serialized in parallel + /// leveraging a maximum possible core count of n_files*n_row_groups*n_columns. + pub allow_single_file_parallelism: bool, default = true + + /// By default parallel parquet writer is tuned for minimum + /// memory usage in a streaming execution plan. You may see + /// a performance benefit when writing large parquet files + /// by increasing maximum_parallel_row_group_writers and + /// maximum_buffered_record_batches_per_stream if your system + /// has idle cores and can tolerate additional memory usage. + /// Boosting these values is likely worthwhile when + /// writing out already in-memory data, such as from a cached + /// data frame. + pub maximum_parallel_row_group_writers: usize, default = 1 + + /// By default parallel parquet writer is tuned for minimum + /// memory usage in a streaming execution plan. You may see + /// a performance benefit when writing large parquet files + /// by increasing maximum_parallel_row_group_writers and + /// maximum_buffered_record_batches_per_stream if your system + /// has idle cores and can tolerate additional memory usage. + /// Boosting these values is likely worthwhile when + /// writing out already in-memory data, such as from a cached + /// data frame. + pub maximum_buffered_record_batches_per_stream: usize, default = 2 } } diff --git a/datafusion/core/src/datasource/file_format/parquet.rs b/datafusion/core/src/datasource/file_format/parquet.rs index a16db9d43213..62867c0e2b38 100644 --- a/datafusion/core/src/datasource/file_format/parquet.rs +++ b/datafusion/core/src/datasource/file_format/parquet.rs @@ -17,11 +17,42 @@ //! Parquet format abstractions +use arrow_array::RecordBatch; +use async_trait::async_trait; +use datafusion_common::stats::Precision; +use datafusion_physical_plan::metrics::MetricsSet; +use parquet::arrow::arrow_writer::{ + compute_leaves, get_column_writers, ArrowColumnChunk, ArrowColumnWriter, + ArrowLeafColumn, +}; +use parquet::file::writer::SerializedFileWriter; use std::any::Any; use std::fmt; use std::fmt::Debug; use std::io::Write; use std::sync::Arc; +use tokio::io::{AsyncWrite, AsyncWriteExt}; +use tokio::sync::mpsc::{self, Receiver, Sender}; +use tokio::task::{JoinHandle, JoinSet}; + +use crate::datasource::file_format::file_compression_type::FileCompressionType; +use crate::datasource::statistics::create_max_min_accs; +use arrow::datatypes::SchemaRef; +use arrow::datatypes::{Fields, Schema}; +use bytes::{BufMut, BytesMut}; +use datafusion_common::{exec_err, not_impl_err, plan_err, DataFusionError, FileType}; +use datafusion_execution::TaskContext; +use datafusion_physical_expr::{PhysicalExpr, PhysicalSortRequirement}; +use futures::{StreamExt, TryStreamExt}; +use hashbrown::HashMap; +use object_store::{ObjectMeta, ObjectStore}; +use parquet::arrow::{ + arrow_to_parquet_schema, parquet_to_arrow_schema, AsyncArrowWriter, +}; +use parquet::file::footer::{decode_footer, decode_metadata}; +use parquet::file::metadata::ParquetMetaData; +use parquet::file::properties::WriterProperties; +use parquet::file::statistics::Statistics as ParquetStatistics; use super::write::demux::start_demuxer_task; use super::write::{create_writer, AbortableWrite, FileWriterMode}; @@ -32,12 +63,11 @@ use crate::arrow::array::{ use crate::arrow::datatypes::DataType; use crate::config::ConfigOptions; -use crate::datasource::file_format::file_compression_type::FileCompressionType; use crate::datasource::get_col_stats; use crate::datasource::physical_plan::{ FileGroupDisplay, FileMeta, FileSinkConfig, ParquetExec, SchemaAdapter, }; -use crate::datasource::statistics::create_max_min_accs; + use crate::error::Result; use crate::execution::context::SessionState; use crate::physical_plan::expressions::{MaxAccumulator, MinAccumulator}; @@ -47,29 +77,6 @@ use crate::physical_plan::{ Statistics, }; -use arrow::datatypes::{Fields, Schema, SchemaRef}; -use datafusion_common::stats::Precision; -use datafusion_common::{exec_err, not_impl_err, plan_err, DataFusionError, FileType}; -use datafusion_execution::TaskContext; -use datafusion_physical_expr::{PhysicalExpr, PhysicalSortRequirement}; -use datafusion_physical_plan::metrics::MetricsSet; - -use async_trait::async_trait; -use bytes::{BufMut, BytesMut}; -use futures::{StreamExt, TryStreamExt}; -use hashbrown::HashMap; -use object_store::{ObjectMeta, ObjectStore}; -use parquet::arrow::{parquet_to_arrow_schema, AsyncArrowWriter}; -use parquet::column::writer::ColumnCloseResult; -use parquet::file::footer::{decode_footer, decode_metadata}; -use parquet::file::metadata::ParquetMetaData; -use parquet::file::properties::WriterProperties; -use parquet::file::statistics::Statistics as ParquetStatistics; -use parquet::file::writer::SerializedFileWriter; -use tokio::io::{AsyncWrite, AsyncWriteExt}; -use tokio::sync::mpsc::{UnboundedReceiver, UnboundedSender}; -use tokio::task::{JoinHandle, JoinSet}; - /// The Apache Parquet `FileFormat` implementation /// /// Note it is recommended these are instead configured on the [`ConfigOptions`] @@ -668,37 +675,6 @@ impl ParquetSink { } } } - - /// Creates an object store writer for each output partition - /// This is used when parallelizing individual parquet file writes. - async fn create_object_store_writers( - &self, - num_partitions: usize, - object_store: Arc, - ) -> Result>>> { - let mut writers = Vec::new(); - - for _ in 0..num_partitions { - let file_path = self.config.table_paths[0].prefix(); - let object_meta = ObjectMeta { - location: file_path.clone(), - last_modified: chrono::offset::Utc::now(), - size: 0, - e_tag: None, - }; - writers.push( - create_writer( - FileWriterMode::PutMultipart, - FileCompressionType::UNCOMPRESSED, - object_meta.into(), - object_store.clone(), - ) - .await?, - ); - } - - Ok(writers) - } } #[async_trait] @@ -726,29 +702,8 @@ impl DataSink for ParquetSink { .runtime_env() .object_store(&self.config.object_store_url)?; - let exec_options = &context.session_config().options().execution; - - let allow_single_file_parallelism = - exec_options.parquet.allow_single_file_parallelism; - - // This is a temporary special case until https://github.com/apache/arrow-datafusion/pull/7655 - // can be pulled in. - if allow_single_file_parallelism && self.config.single_file_output { - let object_store_writer = self - .create_object_store_writers(1, object_store) - .await? - .remove(0); - - let schema_clone = self.config.output_schema.clone(); - return output_single_parquet_file_parallelized( - object_store_writer, - vec![data], - schema_clone, - parquet_props, - ) - .await - .map(|r| r as u64); - } + let parquet_opts = &context.session_config().options().execution.parquet; + let allow_single_file_parallelism = parquet_opts.allow_single_file_parallelism; let part_col = if !self.config.table_partition_cols.is_empty() { Some(self.config.table_partition_cols.clone()) @@ -756,6 +711,12 @@ impl DataSink for ParquetSink { None }; + let parallel_options = ParallelParquetWriterOptions { + max_parallel_row_groups: parquet_opts.maximum_parallel_row_group_writers, + max_buffered_record_batches_per_stream: parquet_opts + .maximum_buffered_record_batches_per_stream, + }; + let (demux_task, mut file_stream_rx) = start_demuxer_task( data, context, @@ -768,8 +729,35 @@ impl DataSink for ParquetSink { let mut file_write_tasks: JoinSet> = JoinSet::new(); while let Some((path, mut rx)) = file_stream_rx.recv().await { - let mut writer = self - .create_async_arrow_writer( + if !allow_single_file_parallelism { + let mut writer = self + .create_async_arrow_writer( + ObjectMeta { + location: path, + last_modified: chrono::offset::Utc::now(), + size: 0, + e_tag: None, + } + .into(), + object_store.clone(), + parquet_props.clone(), + ) + .await?; + file_write_tasks.spawn(async move { + let mut row_count = 0; + while let Some(batch) = rx.recv().await { + row_count += batch.num_rows(); + writer.write(&batch).await?; + } + writer.close().await?; + Ok(row_count) + }); + } else { + let writer = create_writer( + FileWriterMode::PutMultipart, + // Parquet files as a whole are never compressed, since they + // manage compressed blocks themselves. + FileCompressionType::UNCOMPRESSED, ObjectMeta { location: path, last_modified: chrono::offset::Utc::now(), @@ -778,19 +766,22 @@ impl DataSink for ParquetSink { } .into(), object_store.clone(), - parquet_props.clone(), ) .await?; - - file_write_tasks.spawn(async move { - let mut row_count = 0; - while let Some(batch) = rx.recv().await { - row_count += batch.num_rows(); - writer.write(&batch).await?; - } - writer.close().await?; - Ok(row_count) - }); + let schema = self.get_writer_schema(); + let props = parquet_props.clone(); + let parallel_options_clone = parallel_options.clone(); + file_write_tasks.spawn(async move { + output_single_parquet_file_parallelized( + writer, + rx, + schema, + &props, + parallel_options_clone, + ) + .await + }); + } } let mut row_count = 0; @@ -823,119 +814,228 @@ impl DataSink for ParquetSink { } } -/// This is the return type when joining subtasks which are serializing parquet files -/// into memory buffers. The first part of the tuple is the parquet bytes and the -/// second is how many rows were written into the file. -type ParquetFileSerializedResult = Result<(Vec, usize), DataFusionError>; +/// Consumes a stream of [ArrowLeafColumn] via a channel and serializes them using an [ArrowColumnWriter] +/// Once the channel is exhausted, returns the ArrowColumnWriter. +async fn column_serializer_task( + mut rx: Receiver, + mut writer: ArrowColumnWriter, +) -> Result { + while let Some(col) = rx.recv().await { + writer.write(&col)?; + } + Ok(writer) +} -/// Parallelizes the serialization of a single parquet file, by first serializing N -/// independent RecordBatch streams in parallel to parquet files in memory. Another -/// task then stitches these independent files back together and streams this large -/// single parquet file to an ObjectStore in multiple parts. -async fn output_single_parquet_file_parallelized( - mut object_store_writer: AbortableWrite>, - mut data: Vec, - output_schema: Arc, - parquet_props: &WriterProperties, -) -> Result { - let mut row_count = 0; - // TODO decrease parallelism / buffering: - // https://github.com/apache/arrow-datafusion/issues/7591 - let parallelism = data.len(); - let mut join_handles: Vec> = - Vec::with_capacity(parallelism); - for _ in 0..parallelism { - let buffer: Vec = Vec::new(); - let mut writer = parquet::arrow::arrow_writer::ArrowWriter::try_new( - buffer, - output_schema.clone(), - Some(parquet_props.clone()), - )?; - let mut data_stream = data.remove(0); - join_handles.push(tokio::spawn(async move { - let mut inner_row_count = 0; - while let Some(batch) = data_stream.next().await.transpose()? { - inner_row_count += batch.num_rows(); - writer.write(&batch)?; - } - let out = writer.into_inner()?; - Ok((out, inner_row_count)) - })) +type ColumnJoinHandle = JoinHandle>; +type ColSender = Sender; +/// Spawns a parallel serialization task for each column +/// Returns join handles for each columns serialization task along with a send channel +/// to send arrow arrays to each serialization task. +fn spawn_column_parallel_row_group_writer( + schema: Arc, + parquet_props: Arc, + max_buffer_size: usize, +) -> Result<(Vec, Vec)> { + let schema_desc = arrow_to_parquet_schema(&schema)?; + let col_writers = get_column_writers(&schema_desc, &parquet_props, &schema)?; + let num_columns = col_writers.len(); + + let mut col_writer_handles = Vec::with_capacity(num_columns); + let mut col_array_channels = Vec::with_capacity(num_columns); + for writer in col_writers.into_iter() { + // Buffer size of this channel limits the number of arrays queued up for column level serialization + let (send_array, recieve_array) = + mpsc::channel::(max_buffer_size); + col_array_channels.push(send_array); + col_writer_handles + .push(tokio::spawn(column_serializer_task(recieve_array, writer))) } - let mut writer = None; - let endpoints: (UnboundedSender>, UnboundedReceiver>) = - tokio::sync::mpsc::unbounded_channel(); - let (tx, mut rx) = endpoints; - let writer_join_handle: JoinHandle< - Result< - AbortableWrite>, - DataFusionError, - >, - > = tokio::task::spawn(async move { - while let Some(data) = rx.recv().await { - // TODO write incrementally - // https://github.com/apache/arrow-datafusion/issues/7591 - object_store_writer.write_all(data.as_slice()).await?; + Ok((col_writer_handles, col_array_channels)) +} + +/// Settings related to writing parquet files in parallel +#[derive(Clone)] +struct ParallelParquetWriterOptions { + max_parallel_row_groups: usize, + max_buffered_record_batches_per_stream: usize, +} + +/// This is the return type of calling [ArrowColumnWriter].close() on each column +/// i.e. the Vec of encoded columns which can be appended to a row group +type RBStreamSerializeResult = Result<(Vec, usize)>; + +/// Sends the ArrowArrays in passed [RecordBatch] through the channels to their respective +/// parallel column serializers. +async fn send_arrays_to_col_writers( + col_array_channels: &[ColSender], + rb: &RecordBatch, + schema: Arc, +) -> Result<()> { + for (tx, array, field) in col_array_channels + .iter() + .zip(rb.columns()) + .zip(schema.fields()) + .map(|((a, b), c)| (a, b, c)) + { + for c in compute_leaves(field, array)? { + tx.send(c).await.map_err(|_| { + DataFusionError::Internal("Unable to send array to writer!".into()) + })?; } - Ok(object_store_writer) - }); + } + + Ok(()) +} + +/// Spawns a tokio task which joins the parallel column writer tasks, +/// and finalizes the row group. +fn spawn_rg_join_and_finalize_task( + column_writer_handles: Vec>>, + rg_rows: usize, +) -> JoinHandle { + tokio::spawn(async move { + let num_cols = column_writer_handles.len(); + let mut finalized_rg = Vec::with_capacity(num_cols); + for handle in column_writer_handles.into_iter() { + match handle.await { + Ok(r) => { + let w = r?; + finalized_rg.push(w.close()?); + } + Err(e) => { + if e.is_panic() { + std::panic::resume_unwind(e.into_panic()) + } else { + unreachable!() + } + } + } + } + + Ok((finalized_rg, rg_rows)) + }) +} + +/// This task coordinates the serialization of a parquet file in parallel. +/// As the query produces RecordBatches, these are written to a RowGroup +/// via parallel [ArrowColumnWriter] tasks. Once the desired max rows per +/// row group is reached, the parallel tasks are joined on another separate task +/// and sent to a concatenation task. This task immediately continues to work +/// on the next row group in parallel. So, parquet serialization is parallelized +/// accross both columns and row_groups, with a theoretical max number of parallel tasks +/// given by n_columns * num_row_groups. +fn spawn_parquet_parallel_serialization_task( + mut data: Receiver, + serialize_tx: Sender>, + schema: Arc, + writer_props: Arc, + parallel_options: ParallelParquetWriterOptions, +) -> JoinHandle> { + tokio::spawn(async move { + let max_buffer_rb = parallel_options.max_buffered_record_batches_per_stream; + let max_row_group_rows = writer_props.max_row_group_size(); + let (mut column_writer_handles, mut col_array_channels) = + spawn_column_parallel_row_group_writer( + schema.clone(), + writer_props.clone(), + max_buffer_rb, + )?; + let mut current_rg_rows = 0; + + while let Some(rb) = data.recv().await { + if current_rg_rows + rb.num_rows() < max_row_group_rows { + send_arrays_to_col_writers(&col_array_channels, &rb, schema.clone()) + .await?; + current_rg_rows += rb.num_rows(); + } else { + let rows_left = max_row_group_rows - current_rg_rows; + let a = rb.slice(0, rows_left); + send_arrays_to_col_writers(&col_array_channels, &a, schema.clone()) + .await?; + + // Signal the parallel column writers that the RowGroup is done, join and finalize RowGroup + // on a separate task, so that we can immediately start on the next RG before waiting + // for the current one to finish. + drop(col_array_channels); + let finalize_rg_task = spawn_rg_join_and_finalize_task( + column_writer_handles, + max_row_group_rows, + ); + + serialize_tx.send(finalize_rg_task).await.map_err(|_| { + DataFusionError::Internal( + "Unable to send closed RG to concat task!".into(), + ) + })?; + + let b = rb.slice(rows_left, rb.num_rows() - rows_left); + (column_writer_handles, col_array_channels) = + spawn_column_parallel_row_group_writer( + schema.clone(), + writer_props.clone(), + max_buffer_rb, + )?; + send_arrays_to_col_writers(&col_array_channels, &b, schema.clone()) + .await?; + current_rg_rows = b.num_rows(); + } + } + + drop(col_array_channels); + // Handle leftover rows as final rowgroup, which may be smaller than max_row_group_rows + if current_rg_rows > 0 { + let finalize_rg_task = + spawn_rg_join_and_finalize_task(column_writer_handles, current_rg_rows); + + serialize_tx.send(finalize_rg_task).await.map_err(|_| { + DataFusionError::Internal( + "Unable to send closed RG to concat task!".into(), + ) + })?; + } + + Ok(()) + }) +} + +/// Consume RowGroups serialized by other parallel tasks and concatenate them in +/// to the final parquet file, while flushing finalized bytes to an [ObjectStore] +async fn concatenate_parallel_row_groups( + mut serialize_rx: Receiver>, + schema: Arc, + writer_props: Arc, + mut object_store_writer: AbortableWrite>, +) -> Result { let merged_buff = SharedBuffer::new(1048576); - for handle in join_handles { + + let schema_desc = arrow_to_parquet_schema(schema.as_ref())?; + let mut parquet_writer = SerializedFileWriter::new( + merged_buff.clone(), + schema_desc.root_schema_ptr(), + writer_props, + )?; + + let mut row_count = 0; + + while let Some(handle) = serialize_rx.recv().await { let join_result = handle.await; match join_result { Ok(result) => { - let (out, num_rows) = result?; - let reader = bytes::Bytes::from(out); - row_count += num_rows; - //let reader = File::open(buffer)?; - let metadata = parquet::file::footer::parse_metadata(&reader)?; - let schema = metadata.file_metadata().schema(); - writer = match writer { - Some(writer) => Some(writer), - None => Some(SerializedFileWriter::new( - merged_buff.clone(), - Arc::new(schema.clone()), - Arc::new(parquet_props.clone()), - )?), - }; - - match &mut writer{ - Some(w) => { - // Note: cannot use .await within this loop as RowGroupMetaData is not Send - // Instead, use a non-blocking channel to send bytes to separate worker - // which will write to ObjectStore. - for rg in metadata.row_groups() { - let mut rg_out = w.next_row_group()?; - for column in rg.columns() { - let result = ColumnCloseResult { - bytes_written: column.compressed_size() as _, - rows_written: rg.num_rows() as _, - metadata: column.clone(), - // TODO need to populate the indexes when writing final file - // see https://github.com/apache/arrow-datafusion/issues/7589 - bloom_filter: None, - column_index: None, - offset_index: None, - }; - rg_out.append_column(&reader, result)?; - let mut buff_to_flush = merged_buff.buffer.try_lock().unwrap(); - if buff_to_flush.len() > 1024000{ - let bytes: Vec = buff_to_flush.drain(..).collect(); - tx.send(bytes).map_err(|_| DataFusionError::Execution("Failed to send bytes to ObjectStore writer".into()))?; - - } - } - rg_out.close()?; - let mut buff_to_flush = merged_buff.buffer.try_lock().unwrap(); - if buff_to_flush.len() > 1024000{ - let bytes: Vec = buff_to_flush.drain(..).collect(); - tx.send(bytes).map_err(|_| DataFusionError::Execution("Failed to send bytes to ObjectStore writer".into()))?; - } - } - }, - None => unreachable!("Parquet writer should always be initialized in first iteration of loop!") + let mut rg_out = parquet_writer.next_row_group()?; + let (serialized_columns, cnt) = result?; + row_count += cnt; + for chunk in serialized_columns { + chunk.append_to_row_group(&mut rg_out)?; + let mut buff_to_flush = merged_buff.buffer.try_lock().unwrap(); + if buff_to_flush.len() > 1024000 { + object_store_writer + .write_all(buff_to_flush.as_slice()) + .await?; + buff_to_flush.clear(); + } } + rg_out.close()?; } Err(e) => { if e.is_panic() { @@ -946,14 +1046,51 @@ async fn output_single_parquet_file_parallelized( } } } - let inner_writer = writer.unwrap().into_inner()?; + + let inner_writer = parquet_writer.into_inner()?; let final_buff = inner_writer.buffer.try_lock().unwrap(); - // Explicitly drop tx to signal to rx we are done sending data - drop(tx); + object_store_writer.write_all(final_buff.as_slice()).await?; + object_store_writer.shutdown().await?; + + Ok(row_count) +} - let mut object_store_writer = match writer_join_handle.await { - Ok(r) => r?, +/// Parallelizes the serialization of a single parquet file, by first serializing N +/// independent RecordBatch streams in parallel to RowGroups in memory. Another +/// task then stitches these independent RowGroups together and streams this large +/// single parquet file to an ObjectStore in multiple parts. +async fn output_single_parquet_file_parallelized( + object_store_writer: AbortableWrite>, + data: Receiver, + output_schema: Arc, + parquet_props: &WriterProperties, + parallel_options: ParallelParquetWriterOptions, +) -> Result { + let max_rowgroups = parallel_options.max_parallel_row_groups; + // Buffer size of this channel limits maximum number of RowGroups being worked on in parallel + let (serialize_tx, serialize_rx) = + mpsc::channel::>(max_rowgroups); + + let arc_props = Arc::new(parquet_props.clone()); + let launch_serialization_task = spawn_parquet_parallel_serialization_task( + data, + serialize_tx, + output_schema.clone(), + arc_props.clone(), + parallel_options, + ); + let row_count = concatenate_parallel_row_groups( + serialize_rx, + output_schema.clone(), + arc_props.clone(), + object_store_writer, + ) + .await?; + + match launch_serialization_task.await { + Ok(Ok(_)) => (), + Ok(Err(e)) => return Err(e), Err(e) => { if e.is_panic() { std::panic::resume_unwind(e.into_panic()) @@ -962,8 +1099,6 @@ async fn output_single_parquet_file_parallelized( } } }; - object_store_writer.write_all(final_buff.as_slice()).await?; - object_store_writer.shutdown().await?; Ok(row_count) } diff --git a/datafusion/sqllogictest/test_files/information_schema.slt b/datafusion/sqllogictest/test_files/information_schema.slt index 4a2b6220fd85..ed85f54a39aa 100644 --- a/datafusion/sqllogictest/test_files/information_schema.slt +++ b/datafusion/sqllogictest/test_files/information_schema.slt @@ -153,7 +153,7 @@ datafusion.execution.collect_statistics false datafusion.execution.max_buffered_batches_per_output_file 2 datafusion.execution.meta_fetch_concurrency 32 datafusion.execution.minimum_parallel_output_files 4 -datafusion.execution.parquet.allow_single_file_parallelism false +datafusion.execution.parquet.allow_single_file_parallelism true datafusion.execution.parquet.bloom_filter_enabled false datafusion.execution.parquet.bloom_filter_fpp NULL datafusion.execution.parquet.bloom_filter_ndv NULL @@ -168,6 +168,8 @@ datafusion.execution.parquet.enable_page_index true datafusion.execution.parquet.encoding NULL datafusion.execution.parquet.max_row_group_size 1048576 datafusion.execution.parquet.max_statistics_size NULL +datafusion.execution.parquet.maximum_buffered_record_batches_per_stream 2 +datafusion.execution.parquet.maximum_parallel_row_group_writers 1 datafusion.execution.parquet.metadata_size_hint NULL datafusion.execution.parquet.pruning true datafusion.execution.parquet.pushdown_filters false @@ -223,7 +225,7 @@ datafusion.execution.collect_statistics false Should DataFusion collect statisti datafusion.execution.max_buffered_batches_per_output_file 2 This is the maximum number of RecordBatches buffered for each output file being worked. Higher values can potentially give faster write performance at the cost of higher peak memory consumption datafusion.execution.meta_fetch_concurrency 32 Number of files to read in parallel when inferring schema and statistics datafusion.execution.minimum_parallel_output_files 4 Guarantees a minimum level of output files running in parallel. RecordBatches will be distributed in round robin fashion to each parallel writer. Each writer is closed and a new file opened once soft_max_rows_per_output_file is reached. -datafusion.execution.parquet.allow_single_file_parallelism false Controls whether DataFusion will attempt to speed up writing large parquet files by first writing multiple smaller files and then stitching them together into a single large file. This will result in faster write speeds, but higher memory usage. Also currently unsupported are bloom filters and column indexes when single_file_parallelism is enabled. +datafusion.execution.parquet.allow_single_file_parallelism true Controls whether DataFusion will attempt to speed up writing parquet files by serializing them in parallel. Each column in each row group in each output file are serialized in parallel leveraging a maximum possible core count of n_files*n_row_groups*n_columns. datafusion.execution.parquet.bloom_filter_enabled false Sets if bloom filter is enabled for any column datafusion.execution.parquet.bloom_filter_fpp NULL Sets bloom filter false positive probability. If NULL, uses default parquet writer setting datafusion.execution.parquet.bloom_filter_ndv NULL Sets bloom filter number of distinct values. If NULL, uses default parquet writer setting @@ -238,6 +240,8 @@ datafusion.execution.parquet.enable_page_index true If true, reads the Parquet d datafusion.execution.parquet.encoding NULL 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.max_row_group_size 1048576 Sets maximum number of rows in a row group datafusion.execution.parquet.max_statistics_size NULL Sets max statistics size for any column. If NULL, uses default parquet writer setting +datafusion.execution.parquet.maximum_buffered_record_batches_per_stream 2 By default parallel parquet writer is tuned for minimum memory usage in a streaming execution plan. You may see a performance benefit when writing large parquet files by increasing maximum_parallel_row_group_writers and maximum_buffered_record_batches_per_stream if your system has idle cores and can tolerate additional memory usage. Boosting these values is likely worthwhile when writing out already in-memory data, such as from a cached data frame. +datafusion.execution.parquet.maximum_parallel_row_group_writers 1 By default parallel parquet writer is tuned for minimum memory usage in a streaming execution plan. You may see a performance benefit when writing large parquet files by increasing maximum_parallel_row_group_writers and maximum_buffered_record_batches_per_stream if your system has idle cores and can tolerate additional memory usage. Boosting these values is likely worthwhile when writing out already in-memory data, such as from a cached data frame. datafusion.execution.parquet.metadata_size_hint NULL If specified, the parquet reader will try and fetch the last `size_hint` bytes of the parquet file optimistically. If not specified, two reads are required: One read to fetch the 8-byte parquet footer and another to fetch the metadata length encoded in the footer datafusion.execution.parquet.pruning true 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 If true, filter expressions are be applied during the parquet decoding operation to reduce the number of rows decoded diff --git a/docs/source/user-guide/configs.md b/docs/source/user-guide/configs.md index 3476118ca645..c8ff1b06d609 100644 --- a/docs/source/user-guide/configs.md +++ b/docs/source/user-guide/configs.md @@ -35,70 +35,72 @@ Values are parsed according to the [same rules used in casts from Utf8](https:// If the value in the environment variable cannot be cast to the type of the configuration option, the default value will be used instead and a warning emitted. Environment variables are read during `SessionConfig` initialisation so they must be set beforehand and will not affect running sessions. -| key | default | description | -| ---------------------------------------------------------- | ------------------------- | ------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------- | -| datafusion.catalog.create_default_catalog_and_schema | true | Whether the default catalog and schema should be created automatically. | -| datafusion.catalog.default_catalog | datafusion | The default catalog name - this impacts what SQL queries use if not specified | -| datafusion.catalog.default_schema | public | The default schema name - this impacts what SQL queries use if not specified | -| datafusion.catalog.information_schema | false | Should DataFusion provide access to `information_schema` virtual tables for displaying schema information | -| datafusion.catalog.location | NULL | Location scanned to load tables for `default` schema | -| datafusion.catalog.format | NULL | Type of `TableProvider` to use when loading `default` schema | -| datafusion.catalog.has_header | false | If the file has a header | -| datafusion.execution.batch_size | 8192 | Default batch size while creating new batches, it's especially useful for buffer-in-memory batches since creating tiny batches would result in too much metadata memory consumption | -| datafusion.execution.coalesce_batches | true | When set to true, record batches will be examined between each operator and small batches will be coalesced into larger batches. This is helpful when there are highly selective filters or joins that could produce tiny output batches. The target batch size is determined by the configuration setting | -| datafusion.execution.collect_statistics | false | Should DataFusion collect statistics after listing files | -| datafusion.execution.target_partitions | 0 | Number of partitions for query execution. Increasing partitions can increase concurrency. Defaults to the number of CPU cores on the system | -| datafusion.execution.time_zone | +00:00 | The default time zone Some functions, e.g. `EXTRACT(HOUR from SOME_TIME)`, shift the underlying datetime according to this time zone, and then extract the hour | -| datafusion.execution.parquet.enable_page_index | true | If true, reads the Parquet data page level metadata (the Page Index), if present, to reduce the I/O and number of rows decoded. | -| datafusion.execution.parquet.pruning | true | If true, the parquet reader attempts to skip entire row groups based on the predicate in the query and the metadata (min/max values) stored in the parquet file | -| datafusion.execution.parquet.skip_metadata | true | If true, the parquet reader skip the optional embedded metadata that may be in the file Schema. This setting can help avoid schema conflicts when querying multiple parquet files with schemas containing compatible types but different metadata | -| datafusion.execution.parquet.metadata_size_hint | NULL | If specified, the parquet reader will try and fetch the last `size_hint` bytes of the parquet file optimistically. If not specified, two reads are required: One read to fetch the 8-byte parquet footer and another to fetch the metadata length encoded in the footer | -| datafusion.execution.parquet.pushdown_filters | false | If true, filter expressions are be applied during the parquet decoding operation to reduce the number of rows decoded | -| datafusion.execution.parquet.reorder_filters | false | If true, filter expressions evaluated during the parquet decoding operation will be reordered heuristically to minimize the cost of evaluation. If false, the filters are applied in the same order as written in the query | -| datafusion.execution.parquet.data_pagesize_limit | 1048576 | Sets best effort maximum size of data page in bytes | -| datafusion.execution.parquet.write_batch_size | 1024 | Sets write_batch_size in bytes | -| datafusion.execution.parquet.writer_version | 1.0 | Sets parquet writer version valid values are "1.0" and "2.0" | -| datafusion.execution.parquet.compression | zstd(3) | Sets default parquet compression codec Valid values are: uncompressed, snappy, gzip(level), lzo, brotli(level), lz4, zstd(level), and lz4_raw. These values are not case sensitive. If NULL, uses default parquet writer setting | -| datafusion.execution.parquet.dictionary_enabled | NULL | Sets if dictionary encoding is enabled. If NULL, uses default parquet writer setting | -| datafusion.execution.parquet.dictionary_page_size_limit | 1048576 | Sets best effort maximum dictionary page size, in bytes | -| datafusion.execution.parquet.statistics_enabled | NULL | Sets if statistics are enabled for any column Valid values are: "none", "chunk", and "page" These values are not case sensitive. If NULL, uses default parquet writer setting | -| datafusion.execution.parquet.max_statistics_size | NULL | Sets max statistics size for any column. If NULL, uses default parquet writer setting | -| datafusion.execution.parquet.max_row_group_size | 1048576 | Sets maximum number of rows in a row group | -| datafusion.execution.parquet.created_by | datafusion version 32.0.0 | Sets "created by" property | -| datafusion.execution.parquet.column_index_truncate_length | NULL | Sets column index truncate length | -| datafusion.execution.parquet.data_page_row_count_limit | 18446744073709551615 | Sets best effort maximum number of rows in data page | -| datafusion.execution.parquet.encoding | NULL | Sets default encoding for any column Valid values are: plain, plain_dictionary, rle, bit_packed, delta_binary_packed, delta_length_byte_array, delta_byte_array, rle_dictionary, and byte_stream_split. These values are not case sensitive. If NULL, uses default parquet writer setting | -| datafusion.execution.parquet.bloom_filter_enabled | false | Sets if bloom filter is enabled for any column | -| datafusion.execution.parquet.bloom_filter_fpp | NULL | Sets bloom filter false positive probability. If NULL, uses default parquet writer setting | -| datafusion.execution.parquet.bloom_filter_ndv | NULL | Sets bloom filter number of distinct values. If NULL, uses default parquet writer setting | -| datafusion.execution.parquet.allow_single_file_parallelism | false | Controls whether DataFusion will attempt to speed up writing large parquet files by first writing multiple smaller files and then stitching them together into a single large file. This will result in faster write speeds, but higher memory usage. Also currently unsupported are bloom filters and column indexes when single_file_parallelism is enabled. | -| datafusion.execution.aggregate.scalar_update_factor | 10 | Specifies the threshold for using `ScalarValue`s to update accumulators during high-cardinality aggregations for each input batch. The aggregation is considered high-cardinality if the number of affected groups is greater than or equal to `batch_size / scalar_update_factor`. In such cases, `ScalarValue`s are utilized for updating accumulators, rather than the default batch-slice approach. This can lead to performance improvements. By adjusting the `scalar_update_factor`, you can balance the trade-off between more efficient accumulator updates and the number of groups affected. | -| datafusion.execution.planning_concurrency | 0 | Fan-out during initial physical planning. This is mostly use to plan `UNION` children in parallel. Defaults to the number of CPU cores on the system | -| datafusion.execution.sort_spill_reservation_bytes | 10485760 | Specifies the reserved memory for each spillable sort operation to facilitate an in-memory merge. When a sort operation spills to disk, the in-memory data must be sorted and merged before being written to a file. This setting reserves a specific amount of memory for that in-memory sort/merge process. Note: This setting is irrelevant if the sort operation cannot spill (i.e., if there's no `DiskManager` configured). | -| datafusion.execution.sort_in_place_threshold_bytes | 1048576 | When sorting, below what size should data be concatenated and sorted in a single RecordBatch rather than sorted in batches and merged. | -| datafusion.execution.meta_fetch_concurrency | 32 | Number of files to read in parallel when inferring schema and statistics | -| datafusion.execution.minimum_parallel_output_files | 4 | Guarantees a minimum level of output files running in parallel. RecordBatches will be distributed in round robin fashion to each parallel writer. Each writer is closed and a new file opened once soft_max_rows_per_output_file is reached. | -| datafusion.execution.soft_max_rows_per_output_file | 50000000 | Target number of rows in output files when writing multiple. This is a soft max, so it can be exceeded slightly. There also will be one file smaller than the limit if the total number of rows written is not roughly divisible by the soft max | -| datafusion.execution.max_buffered_batches_per_output_file | 2 | This is the maximum number of RecordBatches buffered for each output file being worked. Higher values can potentially give faster write performance at the cost of higher peak memory consumption | -| datafusion.optimizer.enable_round_robin_repartition | true | When set to true, the physical plan optimizer will try to add round robin repartitioning to increase parallelism to leverage more CPU cores | -| datafusion.optimizer.enable_topk_aggregation | true | When set to true, the optimizer will attempt to perform limit operations during aggregations, if possible | -| datafusion.optimizer.filter_null_join_keys | false | When set to true, the optimizer will insert filters before a join between a nullable and non-nullable column to filter out nulls on the nullable side. This filter can add additional overhead when the file format does not fully support predicate push down. | -| datafusion.optimizer.repartition_aggregations | true | Should DataFusion repartition data using the aggregate keys to execute aggregates in parallel using the provided `target_partitions` level | -| datafusion.optimizer.repartition_file_min_size | 10485760 | Minimum total files size in bytes to perform file scan repartitioning. | -| datafusion.optimizer.repartition_joins | true | Should DataFusion repartition data using the join keys to execute joins in parallel using the provided `target_partitions` level | -| datafusion.optimizer.allow_symmetric_joins_without_pruning | true | Should DataFusion allow symmetric hash joins for unbounded data sources even when its inputs do not have any ordering or filtering If the flag is not enabled, the SymmetricHashJoin operator will be unable to prune its internal buffers, resulting in certain join types - such as Full, Left, LeftAnti, LeftSemi, Right, RightAnti, and RightSemi - being produced only at the end of the execution. This is not typical in stream processing. Additionally, without proper design for long runner execution, all types of joins may encounter out-of-memory errors. | -| datafusion.optimizer.repartition_file_scans | true | When set to `true`, file groups will be repartitioned to achieve maximum parallelism. Currently Parquet and CSV formats are supported. If set to `true`, all files will be repartitioned evenly (i.e., a single large file might be partitioned into smaller chunks) for parallel scanning. If set to `false`, different files will be read in parallel, but repartitioning won't happen within a single file. | -| datafusion.optimizer.repartition_windows | true | Should DataFusion repartition data using the partitions keys to execute window functions in parallel using the provided `target_partitions` level | -| datafusion.optimizer.repartition_sorts | true | Should DataFusion execute sorts in a per-partition fashion and merge afterwards instead of coalescing first and sorting globally. With this flag is enabled, plans in the form below `text "SortExec: [a@0 ASC]", " CoalescePartitionsExec", " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", ` would turn into the plan below which performs better in multithreaded environments `text "SortPreservingMergeExec: [a@0 ASC]", " SortExec: [a@0 ASC]", " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", ` | -| datafusion.optimizer.prefer_existing_sort | false | When true, DataFusion will opportunistically remove sorts when the data is already sorted, (i.e. setting `preserve_order` to true on `RepartitionExec` and using `SortPreservingMergeExec`) When false, DataFusion will maximize plan parallelism using `RepartitionExec` even if this requires subsequently resorting data using a `SortExec`. | -| datafusion.optimizer.skip_failed_rules | false | When set to true, the logical plan optimizer will produce warning messages if any optimization rules produce errors and then proceed to the next rule. When set to false, any rules that produce errors will cause the query to fail | -| datafusion.optimizer.max_passes | 3 | Number of times that the optimizer will attempt to optimize the plan | -| datafusion.optimizer.top_down_join_key_reordering | true | When set to true, the physical plan optimizer will run a top down process to reorder the join keys | -| datafusion.optimizer.prefer_hash_join | true | When set to true, the physical plan optimizer will prefer HashJoin over SortMergeJoin. HashJoin can work more efficiently than SortMergeJoin but consumes more memory | -| datafusion.optimizer.hash_join_single_partition_threshold | 1048576 | The maximum estimated size in bytes for one input side of a HashJoin will be collected into a single partition | -| datafusion.explain.logical_plan_only | false | When set to true, the explain statement will only print logical plans | -| datafusion.explain.physical_plan_only | false | When set to true, the explain statement will only print physical plans | -| datafusion.explain.show_statistics | false | When set to true, the explain statement will print operator statistics for physical plans | -| datafusion.sql_parser.parse_float_as_decimal | false | When set to true, SQL parser will parse float as decimal type | -| datafusion.sql_parser.enable_ident_normalization | true | When set to true, SQL parser will normalize ident (convert ident to lowercase when not quoted) | -| datafusion.sql_parser.dialect | generic | Configure the SQL dialect used by DataFusion's parser; supported values include: Generic, MySQL, PostgreSQL, Hive, SQLite, Snowflake, Redshift, MsSQL, ClickHouse, BigQuery, and Ansi. | +| key | default | description | +| ----------------------------------------------------------------------- | ------------------------- | ------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------- | +| datafusion.catalog.create_default_catalog_and_schema | true | Whether the default catalog and schema should be created automatically. | +| datafusion.catalog.default_catalog | datafusion | The default catalog name - this impacts what SQL queries use if not specified | +| datafusion.catalog.default_schema | public | The default schema name - this impacts what SQL queries use if not specified | +| datafusion.catalog.information_schema | false | Should DataFusion provide access to `information_schema` virtual tables for displaying schema information | +| datafusion.catalog.location | NULL | Location scanned to load tables for `default` schema | +| datafusion.catalog.format | NULL | Type of `TableProvider` to use when loading `default` schema | +| datafusion.catalog.has_header | false | If the file has a header | +| datafusion.execution.batch_size | 8192 | Default batch size while creating new batches, it's especially useful for buffer-in-memory batches since creating tiny batches would result in too much metadata memory consumption | +| datafusion.execution.coalesce_batches | true | When set to true, record batches will be examined between each operator and small batches will be coalesced into larger batches. This is helpful when there are highly selective filters or joins that could produce tiny output batches. The target batch size is determined by the configuration setting | +| datafusion.execution.collect_statistics | false | Should DataFusion collect statistics after listing files | +| datafusion.execution.target_partitions | 0 | Number of partitions for query execution. Increasing partitions can increase concurrency. Defaults to the number of CPU cores on the system | +| datafusion.execution.time_zone | +00:00 | The default time zone Some functions, e.g. `EXTRACT(HOUR from SOME_TIME)`, shift the underlying datetime according to this time zone, and then extract the hour | +| datafusion.execution.parquet.enable_page_index | true | If true, reads the Parquet data page level metadata (the Page Index), if present, to reduce the I/O and number of rows decoded. | +| datafusion.execution.parquet.pruning | true | If true, the parquet reader attempts to skip entire row groups based on the predicate in the query and the metadata (min/max values) stored in the parquet file | +| datafusion.execution.parquet.skip_metadata | true | If true, the parquet reader skip the optional embedded metadata that may be in the file Schema. This setting can help avoid schema conflicts when querying multiple parquet files with schemas containing compatible types but different metadata | +| datafusion.execution.parquet.metadata_size_hint | NULL | If specified, the parquet reader will try and fetch the last `size_hint` bytes of the parquet file optimistically. If not specified, two reads are required: One read to fetch the 8-byte parquet footer and another to fetch the metadata length encoded in the footer | +| datafusion.execution.parquet.pushdown_filters | false | If true, filter expressions are be applied during the parquet decoding operation to reduce the number of rows decoded | +| datafusion.execution.parquet.reorder_filters | false | If true, filter expressions evaluated during the parquet decoding operation will be reordered heuristically to minimize the cost of evaluation. If false, the filters are applied in the same order as written in the query | +| datafusion.execution.parquet.data_pagesize_limit | 1048576 | Sets best effort maximum size of data page in bytes | +| datafusion.execution.parquet.write_batch_size | 1024 | Sets write_batch_size in bytes | +| datafusion.execution.parquet.writer_version | 1.0 | Sets parquet writer version valid values are "1.0" and "2.0" | +| datafusion.execution.parquet.compression | zstd(3) | Sets default parquet compression codec Valid values are: uncompressed, snappy, gzip(level), lzo, brotli(level), lz4, zstd(level), and lz4_raw. These values are not case sensitive. If NULL, uses default parquet writer setting | +| datafusion.execution.parquet.dictionary_enabled | NULL | Sets if dictionary encoding is enabled. If NULL, uses default parquet writer setting | +| datafusion.execution.parquet.dictionary_page_size_limit | 1048576 | Sets best effort maximum dictionary page size, in bytes | +| datafusion.execution.parquet.statistics_enabled | NULL | Sets if statistics are enabled for any column Valid values are: "none", "chunk", and "page" These values are not case sensitive. If NULL, uses default parquet writer setting | +| datafusion.execution.parquet.max_statistics_size | NULL | Sets max statistics size for any column. If NULL, uses default parquet writer setting | +| datafusion.execution.parquet.max_row_group_size | 1048576 | Sets maximum number of rows in a row group | +| datafusion.execution.parquet.created_by | datafusion version 32.0.0 | Sets "created by" property | +| datafusion.execution.parquet.column_index_truncate_length | NULL | Sets column index truncate length | +| datafusion.execution.parquet.data_page_row_count_limit | 18446744073709551615 | Sets best effort maximum number of rows in data page | +| datafusion.execution.parquet.encoding | NULL | Sets default encoding for any column Valid values are: plain, plain_dictionary, rle, bit_packed, delta_binary_packed, delta_length_byte_array, delta_byte_array, rle_dictionary, and byte_stream_split. These values are not case sensitive. If NULL, uses default parquet writer setting | +| datafusion.execution.parquet.bloom_filter_enabled | false | Sets if bloom filter is enabled for any column | +| datafusion.execution.parquet.bloom_filter_fpp | NULL | Sets bloom filter false positive probability. If NULL, uses default parquet writer setting | +| datafusion.execution.parquet.bloom_filter_ndv | NULL | Sets bloom filter number of distinct values. If NULL, uses default parquet writer setting | +| datafusion.execution.parquet.allow_single_file_parallelism | true | Controls whether DataFusion will attempt to speed up writing parquet files by serializing them in parallel. Each column in each row group in each output file are serialized in parallel leveraging a maximum possible core count of n_files*n_row_groups*n_columns. | +| datafusion.execution.parquet.maximum_parallel_row_group_writers | 1 | By default parallel parquet writer is tuned for minimum memory usage in a streaming execution plan. You may see a performance benefit when writing large parquet files by increasing maximum_parallel_row_group_writers and maximum_buffered_record_batches_per_stream if your system has idle cores and can tolerate additional memory usage. Boosting these values is likely worthwhile when writing out already in-memory data, such as from a cached data frame. | +| datafusion.execution.parquet.maximum_buffered_record_batches_per_stream | 2 | By default parallel parquet writer is tuned for minimum memory usage in a streaming execution plan. You may see a performance benefit when writing large parquet files by increasing maximum_parallel_row_group_writers and maximum_buffered_record_batches_per_stream if your system has idle cores and can tolerate additional memory usage. Boosting these values is likely worthwhile when writing out already in-memory data, such as from a cached data frame. | +| datafusion.execution.aggregate.scalar_update_factor | 10 | Specifies the threshold for using `ScalarValue`s to update accumulators during high-cardinality aggregations for each input batch. The aggregation is considered high-cardinality if the number of affected groups is greater than or equal to `batch_size / scalar_update_factor`. In such cases, `ScalarValue`s are utilized for updating accumulators, rather than the default batch-slice approach. This can lead to performance improvements. By adjusting the `scalar_update_factor`, you can balance the trade-off between more efficient accumulator updates and the number of groups affected. | +| datafusion.execution.planning_concurrency | 0 | Fan-out during initial physical planning. This is mostly use to plan `UNION` children in parallel. Defaults to the number of CPU cores on the system | +| datafusion.execution.sort_spill_reservation_bytes | 10485760 | Specifies the reserved memory for each spillable sort operation to facilitate an in-memory merge. When a sort operation spills to disk, the in-memory data must be sorted and merged before being written to a file. This setting reserves a specific amount of memory for that in-memory sort/merge process. Note: This setting is irrelevant if the sort operation cannot spill (i.e., if there's no `DiskManager` configured). | +| datafusion.execution.sort_in_place_threshold_bytes | 1048576 | When sorting, below what size should data be concatenated and sorted in a single RecordBatch rather than sorted in batches and merged. | +| datafusion.execution.meta_fetch_concurrency | 32 | Number of files to read in parallel when inferring schema and statistics | +| datafusion.execution.minimum_parallel_output_files | 4 | Guarantees a minimum level of output files running in parallel. RecordBatches will be distributed in round robin fashion to each parallel writer. Each writer is closed and a new file opened once soft_max_rows_per_output_file is reached. | +| datafusion.execution.soft_max_rows_per_output_file | 50000000 | Target number of rows in output files when writing multiple. This is a soft max, so it can be exceeded slightly. There also will be one file smaller than the limit if the total number of rows written is not roughly divisible by the soft max | +| datafusion.execution.max_buffered_batches_per_output_file | 2 | This is the maximum number of RecordBatches buffered for each output file being worked. Higher values can potentially give faster write performance at the cost of higher peak memory consumption | +| datafusion.optimizer.enable_round_robin_repartition | true | When set to true, the physical plan optimizer will try to add round robin repartitioning to increase parallelism to leverage more CPU cores | +| datafusion.optimizer.enable_topk_aggregation | true | When set to true, the optimizer will attempt to perform limit operations during aggregations, if possible | +| datafusion.optimizer.filter_null_join_keys | false | When set to true, the optimizer will insert filters before a join between a nullable and non-nullable column to filter out nulls on the nullable side. This filter can add additional overhead when the file format does not fully support predicate push down. | +| datafusion.optimizer.repartition_aggregations | true | Should DataFusion repartition data using the aggregate keys to execute aggregates in parallel using the provided `target_partitions` level | +| datafusion.optimizer.repartition_file_min_size | 10485760 | Minimum total files size in bytes to perform file scan repartitioning. | +| datafusion.optimizer.repartition_joins | true | Should DataFusion repartition data using the join keys to execute joins in parallel using the provided `target_partitions` level | +| datafusion.optimizer.allow_symmetric_joins_without_pruning | true | Should DataFusion allow symmetric hash joins for unbounded data sources even when its inputs do not have any ordering or filtering If the flag is not enabled, the SymmetricHashJoin operator will be unable to prune its internal buffers, resulting in certain join types - such as Full, Left, LeftAnti, LeftSemi, Right, RightAnti, and RightSemi - being produced only at the end of the execution. This is not typical in stream processing. Additionally, without proper design for long runner execution, all types of joins may encounter out-of-memory errors. | +| datafusion.optimizer.repartition_file_scans | true | When set to `true`, file groups will be repartitioned to achieve maximum parallelism. Currently Parquet and CSV formats are supported. If set to `true`, all files will be repartitioned evenly (i.e., a single large file might be partitioned into smaller chunks) for parallel scanning. If set to `false`, different files will be read in parallel, but repartitioning won't happen within a single file. | +| datafusion.optimizer.repartition_windows | true | Should DataFusion repartition data using the partitions keys to execute window functions in parallel using the provided `target_partitions` level | +| datafusion.optimizer.repartition_sorts | true | Should DataFusion execute sorts in a per-partition fashion and merge afterwards instead of coalescing first and sorting globally. With this flag is enabled, plans in the form below `text "SortExec: [a@0 ASC]", " CoalescePartitionsExec", " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", ` would turn into the plan below which performs better in multithreaded environments `text "SortPreservingMergeExec: [a@0 ASC]", " SortExec: [a@0 ASC]", " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", ` | +| datafusion.optimizer.prefer_existing_sort | false | When true, DataFusion will opportunistically remove sorts when the data is already sorted, (i.e. setting `preserve_order` to true on `RepartitionExec` and using `SortPreservingMergeExec`) When false, DataFusion will maximize plan parallelism using `RepartitionExec` even if this requires subsequently resorting data using a `SortExec`. | +| datafusion.optimizer.skip_failed_rules | false | When set to true, the logical plan optimizer will produce warning messages if any optimization rules produce errors and then proceed to the next rule. When set to false, any rules that produce errors will cause the query to fail | +| datafusion.optimizer.max_passes | 3 | Number of times that the optimizer will attempt to optimize the plan | +| datafusion.optimizer.top_down_join_key_reordering | true | When set to true, the physical plan optimizer will run a top down process to reorder the join keys | +| datafusion.optimizer.prefer_hash_join | true | When set to true, the physical plan optimizer will prefer HashJoin over SortMergeJoin. HashJoin can work more efficiently than SortMergeJoin but consumes more memory | +| datafusion.optimizer.hash_join_single_partition_threshold | 1048576 | The maximum estimated size in bytes for one input side of a HashJoin will be collected into a single partition | +| datafusion.explain.logical_plan_only | false | When set to true, the explain statement will only print logical plans | +| datafusion.explain.physical_plan_only | false | When set to true, the explain statement will only print physical plans | +| datafusion.explain.show_statistics | false | When set to true, the explain statement will print operator statistics for physical plans | +| datafusion.sql_parser.parse_float_as_decimal | false | When set to true, SQL parser will parse float as decimal type | +| datafusion.sql_parser.enable_ident_normalization | true | When set to true, SQL parser will normalize ident (convert ident to lowercase when not quoted) | +| datafusion.sql_parser.dialect | generic | Configure the SQL dialect used by DataFusion's parser; supported values include: Generic, MySQL, PostgreSQL, Hive, SQLite, Snowflake, Redshift, MsSQL, ClickHouse, BigQuery, and Ansi. | From d190ff16b7b4860f89052a30ca3acecfcde21060 Mon Sep 17 00:00:00 2001 From: Hengfei Yang Date: Thu, 26 Oct 2023 01:09:53 +0800 Subject: [PATCH 129/572] feat: Use bloom filter when reading parquet to skip row groups (#7821) * feat: implement read bloom filter support * test: add unit test for read bloom filter * Simplify bloom filter application * test: add unit test for bloom filter with sql `in` * fix: imrpove bloom filter match express * fix: add more test for bloom filter * ci: rollback dependences * ci: merge main branch * fix: unit tests for bloom filter * ci: cargo clippy * ci: cargo clippy --------- Co-authored-by: Andrew Lamb --- .../src/datasource/physical_plan/parquet.rs | 45 +- .../physical_plan/parquet/row_groups.rs | 572 +++++++++++++++++- .../sqllogictest/test_files/predicates.slt | 40 ++ 3 files changed, 625 insertions(+), 32 deletions(-) diff --git a/datafusion/core/src/datasource/physical_plan/parquet.rs b/datafusion/core/src/datasource/physical_plan/parquet.rs index 6cab27b0846c..3a2459bec817 100644 --- a/datafusion/core/src/datasource/physical_plan/parquet.rs +++ b/datafusion/core/src/datasource/physical_plan/parquet.rs @@ -82,6 +82,9 @@ pub struct ParquetExec { /// Override for `Self::with_enable_page_index`. If None, uses /// values from base_config enable_page_index: Option, + /// Override for `Self::with_enable_bloom_filter`. If None, uses + /// values from base_config + enable_bloom_filter: Option, /// Base configuration for this scan base_config: FileScanConfig, projected_statistics: Statistics, @@ -151,6 +154,7 @@ impl ParquetExec { pushdown_filters: None, reorder_filters: None, enable_page_index: None, + enable_bloom_filter: None, base_config, projected_schema, projected_statistics, @@ -244,6 +248,18 @@ impl ParquetExec { .unwrap_or(config_options.execution.parquet.enable_page_index) } + /// If enabled, the reader will read by the bloom filter + pub fn with_enable_bloom_filter(mut self, enable_bloom_filter: bool) -> Self { + self.enable_bloom_filter = Some(enable_bloom_filter); + self + } + + /// Return the value described in [`Self::with_enable_bloom_filter`] + fn enable_bloom_filter(&self, config_options: &ConfigOptions) -> bool { + self.enable_bloom_filter + .unwrap_or(config_options.execution.parquet.bloom_filter_enabled) + } + /// Redistribute files across partitions according to their size /// See comments on `get_file_groups_repartitioned()` for more detail. pub fn get_repartitioned( @@ -373,6 +389,7 @@ impl ExecutionPlan for ParquetExec { pushdown_filters: self.pushdown_filters(config_options), reorder_filters: self.reorder_filters(config_options), enable_page_index: self.enable_page_index(config_options), + enable_bloom_filter: self.enable_bloom_filter(config_options), }; let stream = @@ -406,6 +423,7 @@ struct ParquetOpener { pushdown_filters: bool, reorder_filters: bool, enable_page_index: bool, + enable_bloom_filter: bool, } impl FileOpener for ParquetOpener { @@ -440,6 +458,7 @@ impl FileOpener for ParquetOpener { self.enable_page_index, &self.page_pruning_predicate, ); + let enable_bloom_filter = self.enable_bloom_filter; let limit = self.limit; Ok(Box::pin(async move { @@ -482,16 +501,32 @@ impl FileOpener for ParquetOpener { }; }; - // Row group pruning: attempt to skip entire row_groups + // Row group pruning by statistics: attempt to skip entire row_groups // using metadata on the row groups - let file_metadata = builder.metadata(); - let row_groups = row_groups::prune_row_groups( + let file_metadata = builder.metadata().clone(); + let predicate = pruning_predicate.as_ref().map(|p| p.as_ref()); + let mut row_groups = row_groups::prune_row_groups_by_statistics( file_metadata.row_groups(), file_range, - pruning_predicate.as_ref().map(|p| p.as_ref()), + predicate, &file_metrics, ); + // Bloom filter pruning: if bloom filters are enabled and then attempt to skip entire row_groups + // using bloom filters on the row groups + if enable_bloom_filter && !row_groups.is_empty() { + if let Some(predicate) = predicate { + row_groups = row_groups::prune_row_groups_by_bloom_filters( + &mut builder, + &row_groups, + file_metadata.row_groups(), + predicate, + &file_metrics, + ) + .await; + } + } + // page index pruning: if all data on individual pages can // be ruled using page metadata, rows from other columns // with that range can be skipped as well @@ -567,7 +602,7 @@ impl DefaultParquetFileReaderFactory { } /// Implements [`AsyncFileReader`] for a parquet file in object storage -struct ParquetFileReader { +pub(crate) struct ParquetFileReader { file_metrics: ParquetFileMetrics, inner: ParquetObjectReader, } diff --git a/datafusion/core/src/datasource/physical_plan/parquet/row_groups.rs b/datafusion/core/src/datasource/physical_plan/parquet/row_groups.rs index c6e2c68d0211..91bceed91602 100644 --- a/datafusion/core/src/datasource/physical_plan/parquet/row_groups.rs +++ b/datafusion/core/src/datasource/physical_plan/parquet/row_groups.rs @@ -19,24 +19,31 @@ use arrow::{ array::ArrayRef, datatypes::{DataType, Schema}, }; -use datafusion_common::Column; -use datafusion_common::ScalarValue; -use log::debug; - -use parquet::file::{ - metadata::RowGroupMetaData, statistics::Statistics as ParquetStatistics, +use datafusion_common::tree_node::{TreeNode, VisitRecursion}; +use datafusion_common::{Column, DataFusionError, Result, ScalarValue}; +use parquet::{ + arrow::{async_reader::AsyncFileReader, ParquetRecordBatchStreamBuilder}, + bloom_filter::Sbbf, + file::{metadata::RowGroupMetaData, statistics::Statistics as ParquetStatistics}, }; - -use crate::datasource::physical_plan::parquet::{ - from_bytes_to_i128, parquet_to_arrow_decimal_type, +use std::{ + collections::{HashMap, HashSet}, + sync::Arc, }; -use crate::{ - datasource::listing::FileRange, - physical_optimizer::pruning::{PruningPredicate, PruningStatistics}, + +use crate::datasource::{ + listing::FileRange, + physical_plan::parquet::{from_bytes_to_i128, parquet_to_arrow_decimal_type}, }; +use crate::logical_expr::Operator; +use crate::physical_expr::expressions as phys_expr; +use crate::physical_optimizer::pruning::{PruningPredicate, PruningStatistics}; +use crate::physical_plan::PhysicalExpr; use super::ParquetFileMetrics; +/// Prune row groups based on statistics +/// /// Returns a vector of indexes into `groups` which should be scanned. /// /// If an index is NOT present in the returned Vec it means the @@ -44,7 +51,7 @@ use super::ParquetFileMetrics; /// /// If an index IS present in the returned Vec it means the predicate /// did not filter out that row group. -pub(crate) fn prune_row_groups( +pub(crate) fn prune_row_groups_by_statistics( groups: &[RowGroupMetaData], range: Option, predicate: Option<&PruningPredicate>, @@ -81,7 +88,7 @@ pub(crate) fn prune_row_groups( // stats filter array could not be built // return a closure which will not filter out any row groups Err(e) => { - debug!("Error evaluating row group predicate values {e}"); + log::debug!("Error evaluating row group predicate values {e}"); metrics.predicate_evaluation_errors.add(1); } } @@ -92,6 +99,203 @@ pub(crate) fn prune_row_groups( filtered } +/// Prune row groups by bloom filters +/// +/// Returns a vector of indexes into `groups` which should be scanned. +/// +/// If an index is NOT present in the returned Vec it means the +/// predicate filtered all the row group. +/// +/// If an index IS present in the returned Vec it means the predicate +/// did not filter out that row group. +pub(crate) async fn prune_row_groups_by_bloom_filters< + T: AsyncFileReader + Send + 'static, +>( + builder: &mut ParquetRecordBatchStreamBuilder, + row_groups: &[usize], + groups: &[RowGroupMetaData], + predicate: &PruningPredicate, + metrics: &ParquetFileMetrics, +) -> Vec { + let bf_predicates = match BloomFilterPruningPredicate::try_new(predicate.orig_expr()) + { + Ok(predicates) => predicates, + Err(_) => { + return row_groups.to_vec(); + } + }; + let mut filtered = Vec::with_capacity(groups.len()); + for idx in row_groups { + let rg_metadata = &groups[*idx]; + // get all columns bloom filter + let mut column_sbbf = + HashMap::with_capacity(bf_predicates.required_columns.len()); + for column_name in bf_predicates.required_columns.iter() { + let column_idx = match rg_metadata + .columns() + .iter() + .enumerate() + .find(|(_, column)| column.column_path().string().eq(column_name)) + { + Some((column_idx, _)) => column_idx, + None => continue, + }; + let bf = match builder + .get_row_group_column_bloom_filter(*idx, column_idx) + .await + { + Ok(bf) => match bf { + Some(bf) => bf, + None => { + continue; + } + }, + Err(e) => { + log::error!("Error evaluating row group predicate values when using BloomFilterPruningPredicate {e}"); + metrics.predicate_evaluation_errors.add(1); + continue; + } + }; + column_sbbf.insert(column_name.to_owned(), bf); + } + if bf_predicates.prune(&column_sbbf) { + metrics.row_groups_pruned.add(1); + continue; + } + filtered.push(*idx); + } + filtered +} + +struct BloomFilterPruningPredicate { + /// Actual pruning predicate + predicate_expr: Option, + /// The statistics required to evaluate this predicate + required_columns: Vec, +} + +impl BloomFilterPruningPredicate { + fn try_new(expr: &Arc) -> Result { + let binary_expr = expr.as_any().downcast_ref::(); + match binary_expr { + Some(binary_expr) => { + let columns = Self::get_predicate_columns(expr); + Ok(Self { + predicate_expr: Some(binary_expr.clone()), + required_columns: columns.into_iter().collect(), + }) + } + None => Err(DataFusionError::Execution( + "BloomFilterPruningPredicate only support binary expr".to_string(), + )), + } + } + + fn prune(&self, column_sbbf: &HashMap) -> bool { + Self::prune_expr_with_bloom_filter(self.predicate_expr.as_ref(), column_sbbf) + } + + /// Return true if the `expr` can be proved not `true` + /// based on the bloom filter. + /// + /// We only checked `BinaryExpr` but it also support `InList`, + /// Because of the `optimizer` will convert `InList` to `BinaryExpr`. + fn prune_expr_with_bloom_filter( + expr: Option<&phys_expr::BinaryExpr>, + column_sbbf: &HashMap, + ) -> bool { + let Some(expr) = expr else { + // unsupported predicate + return false; + }; + match expr.op() { + Operator::And | Operator::Or => { + let left = Self::prune_expr_with_bloom_filter( + expr.left().as_any().downcast_ref::(), + column_sbbf, + ); + let right = Self::prune_expr_with_bloom_filter( + expr.right() + .as_any() + .downcast_ref::(), + column_sbbf, + ); + match expr.op() { + Operator::And => left || right, + Operator::Or => left && right, + _ => false, + } + } + Operator::Eq => { + if let Some((col, val)) = Self::check_expr_is_col_equal_const(expr) { + if let Some(sbbf) = column_sbbf.get(col.name()) { + match val { + ScalarValue::Utf8(Some(v)) => !sbbf.check(&v.as_str()), + ScalarValue::Boolean(Some(v)) => !sbbf.check(&v), + ScalarValue::Float64(Some(v)) => !sbbf.check(&v), + ScalarValue::Float32(Some(v)) => !sbbf.check(&v), + ScalarValue::Int64(Some(v)) => !sbbf.check(&v), + ScalarValue::Int32(Some(v)) => !sbbf.check(&v), + ScalarValue::Int16(Some(v)) => !sbbf.check(&v), + ScalarValue::Int8(Some(v)) => !sbbf.check(&v), + _ => false, + } + } else { + false + } + } else { + false + } + } + _ => false, + } + } + + fn get_predicate_columns(expr: &Arc) -> HashSet { + let mut columns = HashSet::new(); + expr.apply(&mut |expr| { + if let Some(binary_expr) = + expr.as_any().downcast_ref::() + { + if let Some((column, _)) = + Self::check_expr_is_col_equal_const(binary_expr) + { + columns.insert(column.name().to_string()); + } + } + Ok(VisitRecursion::Continue) + }) + // no way to fail as only Ok(VisitRecursion::Continue) is returned + .unwrap(); + + columns + } + + fn check_expr_is_col_equal_const( + exr: &phys_expr::BinaryExpr, + ) -> Option<(phys_expr::Column, ScalarValue)> { + if Operator::Eq.ne(exr.op()) { + return None; + } + + let left_any = exr.left().as_any(); + let right_any = exr.right().as_any(); + if let (Some(col), Some(liter)) = ( + left_any.downcast_ref::(), + right_any.downcast_ref::(), + ) { + return Some((col.clone(), liter.value().clone())); + } + if let (Some(liter), Some(col)) = ( + left_any.downcast_ref::(), + right_any.downcast_ref::(), + ) { + return Some((col.clone(), liter.value().clone())); + } + None + } +} + /// Wraps parquet statistics in a way /// that implements [`PruningStatistics`] struct RowGroupPruningStatistics<'a> { @@ -246,14 +450,20 @@ impl<'a> PruningStatistics for RowGroupPruningStatistics<'a> { #[cfg(test)] mod tests { use super::*; + use crate::datasource::physical_plan::parquet::ParquetFileReader; use crate::physical_plan::metrics::ExecutionPlanMetricsSet; use arrow::datatypes::DataType::Decimal128; use arrow::datatypes::Schema; use arrow::datatypes::{DataType, Field}; - use datafusion_common::ToDFSchema; - use datafusion_expr::{cast, col, lit, Expr}; + use datafusion_common::{config::ConfigOptions, TableReference, ToDFSchema}; + use datafusion_expr::{ + builder::LogicalTableSource, cast, col, lit, AggregateUDF, Expr, ScalarUDF, + TableSource, WindowUDF, + }; use datafusion_physical_expr::execution_props::ExecutionProps; use datafusion_physical_expr::{create_physical_expr, PhysicalExpr}; + use datafusion_sql::planner::ContextProvider; + use parquet::arrow::async_reader::ParquetObjectReader; use parquet::basic::LogicalType; use parquet::data_type::{ByteArray, FixedLenByteArray}; use parquet::{ @@ -329,7 +539,12 @@ mod tests { let metrics = parquet_file_metrics(); assert_eq!( - prune_row_groups(&[rgm1, rgm2], None, Some(&pruning_predicate), &metrics), + prune_row_groups_by_statistics( + &[rgm1, rgm2], + None, + Some(&pruning_predicate), + &metrics + ), vec![1] ); } @@ -358,7 +573,12 @@ mod tests { // missing statistics for first row group mean that the result from the predicate expression // is null / undefined so the first row group can't be filtered out assert_eq!( - prune_row_groups(&[rgm1, rgm2], None, Some(&pruning_predicate), &metrics), + prune_row_groups_by_statistics( + &[rgm1, rgm2], + None, + Some(&pruning_predicate), + &metrics + ), vec![0, 1] ); } @@ -400,7 +620,12 @@ mod tests { // the first row group is still filtered out because the predicate expression can be partially evaluated // when conditions are joined using AND assert_eq!( - prune_row_groups(groups, None, Some(&pruning_predicate), &metrics), + prune_row_groups_by_statistics( + groups, + None, + Some(&pruning_predicate), + &metrics + ), vec![1] ); @@ -413,7 +638,12 @@ mod tests { // if conditions in predicate are joined with OR and an unsupported expression is used // this bypasses the entire predicate expression and no row groups are filtered out assert_eq!( - prune_row_groups(groups, None, Some(&pruning_predicate), &metrics), + prune_row_groups_by_statistics( + groups, + None, + Some(&pruning_predicate), + &metrics + ), vec![0, 1] ); } @@ -456,7 +686,12 @@ mod tests { let metrics = parquet_file_metrics(); // First row group was filtered out because it contains no null value on "c2". assert_eq!( - prune_row_groups(&groups, None, Some(&pruning_predicate), &metrics), + prune_row_groups_by_statistics( + &groups, + None, + Some(&pruning_predicate), + &metrics + ), vec![1] ); } @@ -482,7 +717,12 @@ mod tests { // bool = NULL always evaluates to NULL (and thus will not // pass predicates. Ideally these should both be false assert_eq!( - prune_row_groups(&groups, None, Some(&pruning_predicate), &metrics), + prune_row_groups_by_statistics( + &groups, + None, + Some(&pruning_predicate), + &metrics + ), vec![1] ); } @@ -535,7 +775,7 @@ mod tests { ); let metrics = parquet_file_metrics(); assert_eq!( - prune_row_groups( + prune_row_groups_by_statistics( &[rgm1, rgm2, rgm3], None, Some(&pruning_predicate), @@ -598,7 +838,7 @@ mod tests { ); let metrics = parquet_file_metrics(); assert_eq!( - prune_row_groups( + prune_row_groups_by_statistics( &[rgm1, rgm2, rgm3, rgm4], None, Some(&pruning_predicate), @@ -645,7 +885,7 @@ mod tests { ); let metrics = parquet_file_metrics(); assert_eq!( - prune_row_groups( + prune_row_groups_by_statistics( &[rgm1, rgm2, rgm3], None, Some(&pruning_predicate), @@ -715,7 +955,7 @@ mod tests { ); let metrics = parquet_file_metrics(); assert_eq!( - prune_row_groups( + prune_row_groups_by_statistics( &[rgm1, rgm2, rgm3], None, Some(&pruning_predicate), @@ -774,7 +1014,7 @@ mod tests { ); let metrics = parquet_file_metrics(); assert_eq!( - prune_row_groups( + prune_row_groups_by_statistics( &[rgm1, rgm2, rgm3], None, Some(&pruning_predicate), @@ -846,4 +1086,282 @@ mod tests { let execution_props = ExecutionProps::new(); create_physical_expr(expr, &df_schema, schema, &execution_props).unwrap() } + + #[tokio::test] + async fn test_row_group_bloom_filter_pruning_predicate_simple_expr() { + // load parquet file + let testdata = datafusion_common::test_util::parquet_test_data(); + let file_name = "data_index_bloom_encoding_stats.parquet"; + let path = format!("{testdata}/{file_name}"); + let data = bytes::Bytes::from(std::fs::read(path).unwrap()); + + // generate pruning predicate + let schema = Schema::new(vec![Field::new("String", DataType::Utf8, false)]); + let expr = col(r#""String""#).eq(lit("Hello_Not_Exists")); + let expr = logical2physical(&expr, &schema); + let pruning_predicate = + PruningPredicate::try_new(expr, Arc::new(schema)).unwrap(); + + let row_groups = vec![0]; + let pruned_row_groups = test_row_group_bloom_filter_pruning_predicate( + file_name, + data, + &pruning_predicate, + &row_groups, + ) + .await + .unwrap(); + assert!(pruned_row_groups.is_empty()); + } + + #[tokio::test] + async fn test_row_group_bloom_filter_pruning_predicate_mutiple_expr() { + // load parquet file + let testdata = datafusion_common::test_util::parquet_test_data(); + let file_name = "data_index_bloom_encoding_stats.parquet"; + let path = format!("{testdata}/{file_name}"); + let data = bytes::Bytes::from(std::fs::read(path).unwrap()); + + // generate pruning predicate + let schema = Schema::new(vec![Field::new("String", DataType::Utf8, false)]); + let expr = lit("1").eq(lit("1")).and( + col(r#""String""#) + .eq(lit("Hello_Not_Exists")) + .or(col(r#""String""#).eq(lit("Hello_Not_Exists2"))), + ); + let expr = logical2physical(&expr, &schema); + let pruning_predicate = + PruningPredicate::try_new(expr, Arc::new(schema)).unwrap(); + + let row_groups = vec![0]; + let pruned_row_groups = test_row_group_bloom_filter_pruning_predicate( + file_name, + data, + &pruning_predicate, + &row_groups, + ) + .await + .unwrap(); + assert!(pruned_row_groups.is_empty()); + } + + #[tokio::test] + async fn test_row_group_bloom_filter_pruning_predicate_sql_in() { + // load parquet file + let testdata = datafusion_common::test_util::parquet_test_data(); + let file_name = "data_index_bloom_encoding_stats.parquet"; + let path = format!("{testdata}/{file_name}"); + let data = bytes::Bytes::from(std::fs::read(path).unwrap()); + + // generate pruning predicate + let schema = Schema::new(vec![ + Field::new("String", DataType::Utf8, false), + Field::new("String3", DataType::Utf8, false), + ]); + let sql = + "SELECT * FROM tbl WHERE \"String\" IN ('Hello_Not_Exists', 'Hello_Not_Exists2')"; + let expr = sql_to_physical_plan(sql).unwrap(); + let pruning_predicate = + PruningPredicate::try_new(expr, Arc::new(schema)).unwrap(); + + let row_groups = vec![0]; + let pruned_row_groups = test_row_group_bloom_filter_pruning_predicate( + file_name, + data, + &pruning_predicate, + &row_groups, + ) + .await + .unwrap(); + assert!(pruned_row_groups.is_empty()); + } + + #[tokio::test] + async fn test_row_group_bloom_filter_pruning_predicate_with_exists_value() { + // load parquet file + let testdata = datafusion_common::test_util::parquet_test_data(); + let file_name = "data_index_bloom_encoding_stats.parquet"; + let path = format!("{testdata}/{file_name}"); + let data = bytes::Bytes::from(std::fs::read(path).unwrap()); + + // generate pruning predicate + let schema = Schema::new(vec![Field::new("String", DataType::Utf8, false)]); + let expr = col(r#""String""#).eq(lit("Hello")); + let expr = logical2physical(&expr, &schema); + let pruning_predicate = + PruningPredicate::try_new(expr, Arc::new(schema)).unwrap(); + + let row_groups = vec![0]; + let pruned_row_groups = test_row_group_bloom_filter_pruning_predicate( + file_name, + data, + &pruning_predicate, + &row_groups, + ) + .await + .unwrap(); + assert_eq!(pruned_row_groups, row_groups); + } + + #[tokio::test] + async fn test_row_group_bloom_filter_pruning_predicate_without_bloom_filter() { + // load parquet file + let testdata = datafusion_common::test_util::parquet_test_data(); + let file_name = "alltypes_plain.parquet"; + let path = format!("{testdata}/{file_name}"); + let data = bytes::Bytes::from(std::fs::read(path).unwrap()); + + // generate pruning predicate + let schema = Schema::new(vec![Field::new("string_col", DataType::Utf8, false)]); + let expr = col(r#""string_col""#).eq(lit("0")); + let expr = logical2physical(&expr, &schema); + let pruning_predicate = + PruningPredicate::try_new(expr, Arc::new(schema)).unwrap(); + + let row_groups = vec![0]; + let pruned_row_groups = test_row_group_bloom_filter_pruning_predicate( + file_name, + data, + &pruning_predicate, + &row_groups, + ) + .await + .unwrap(); + assert_eq!(pruned_row_groups, row_groups); + } + + async fn test_row_group_bloom_filter_pruning_predicate( + file_name: &str, + data: bytes::Bytes, + pruning_predicate: &PruningPredicate, + row_groups: &[usize], + ) -> Result> { + use object_store::{ObjectMeta, ObjectStore}; + + let object_meta = ObjectMeta { + location: object_store::path::Path::parse(file_name).expect("creating path"), + last_modified: chrono::DateTime::from(std::time::SystemTime::now()), + size: data.len(), + e_tag: None, + }; + let in_memory = object_store::memory::InMemory::new(); + in_memory + .put(&object_meta.location, data) + .await + .expect("put parquet file into in memory object store"); + + let metrics = ExecutionPlanMetricsSet::new(); + let file_metrics = + ParquetFileMetrics::new(0, object_meta.location.as_ref(), &metrics); + let reader = ParquetFileReader { + inner: ParquetObjectReader::new(Arc::new(in_memory), object_meta), + file_metrics: file_metrics.clone(), + }; + let mut builder = ParquetRecordBatchStreamBuilder::new(reader).await.unwrap(); + + let metadata = builder.metadata().clone(); + let pruned_row_group = prune_row_groups_by_bloom_filters( + &mut builder, + row_groups, + metadata.row_groups(), + pruning_predicate, + &file_metrics, + ) + .await; + + Ok(pruned_row_group) + } + + fn sql_to_physical_plan(sql: &str) -> Result> { + use datafusion_optimizer::{ + analyzer::Analyzer, optimizer::Optimizer, OptimizerConfig, OptimizerContext, + }; + use datafusion_sql::{ + planner::SqlToRel, + sqlparser::{ast::Statement, parser::Parser}, + }; + use sqlparser::dialect::GenericDialect; + + // parse the SQL + let dialect = GenericDialect {}; // or AnsiDialect, or your own dialect ... + let ast: Vec = Parser::parse_sql(&dialect, sql).unwrap(); + let statement = &ast[0]; + + // create a logical query plan + let schema_provider = TestSchemaProvider::new(); + let sql_to_rel = SqlToRel::new(&schema_provider); + let plan = sql_to_rel.sql_statement_to_plan(statement.clone()).unwrap(); + + // hard code the return value of now() + let config = OptimizerContext::new().with_skip_failing_rules(false); + let analyzer = Analyzer::new(); + let optimizer = Optimizer::new(); + // analyze and optimize the logical plan + let plan = analyzer.execute_and_check(&plan, config.options(), |_, _| {})?; + let plan = optimizer.optimize(&plan, &config, |_, _| {})?; + // convert the logical plan into a physical plan + let exprs = plan.expressions(); + let expr = &exprs[0]; + let df_schema = plan.schema().as_ref().to_owned(); + let tb_schema: Schema = df_schema.clone().into(); + let execution_props = ExecutionProps::new(); + create_physical_expr(expr, &df_schema, &tb_schema, &execution_props) + } + + struct TestSchemaProvider { + options: ConfigOptions, + tables: HashMap>, + } + + impl TestSchemaProvider { + pub fn new() -> Self { + let mut tables = HashMap::new(); + tables.insert( + "tbl".to_string(), + create_table_source(vec![Field::new( + "String".to_string(), + DataType::Utf8, + false, + )]), + ); + + Self { + options: Default::default(), + tables, + } + } + } + + impl ContextProvider for TestSchemaProvider { + fn get_table_source(&self, name: TableReference) -> Result> { + match self.tables.get(name.table()) { + Some(table) => Ok(table.clone()), + _ => datafusion_common::plan_err!("Table not found: {}", name.table()), + } + } + + fn get_function_meta(&self, _name: &str) -> Option> { + None + } + + fn get_aggregate_meta(&self, _name: &str) -> Option> { + None + } + + fn get_variable_type(&self, _variable_names: &[String]) -> Option { + None + } + + fn options(&self) -> &ConfigOptions { + &self.options + } + + fn get_window_meta(&self, _name: &str) -> Option> { + None + } + } + + fn create_table_source(fields: Vec) -> Arc { + Arc::new(LogicalTableSource::new(Arc::new(Schema::new(fields)))) + } } diff --git a/datafusion/sqllogictest/test_files/predicates.slt b/datafusion/sqllogictest/test_files/predicates.slt index 937b4c2eccf6..d22b2ff953b7 100644 --- a/datafusion/sqllogictest/test_files/predicates.slt +++ b/datafusion/sqllogictest/test_files/predicates.slt @@ -480,3 +480,43 @@ select * from t where (i & 3) = 1; ######## statement ok DROP TABLE t; + + +######## +# Test query with bloom filter +# Refer to https://github.com/apache/arrow-datafusion/pull/7821#pullrequestreview-1688062599 +######## + +statement ok +CREATE EXTERNAL TABLE data_index_bloom_encoding_stats STORED AS PARQUET LOCATION '../../parquet-testing/data/data_index_bloom_encoding_stats.parquet'; + +statement ok +set datafusion.execution.parquet.bloom_filter_enabled=true; + +query T +SELECT * FROM data_index_bloom_encoding_stats WHERE "String" = 'foo'; + +query T +SELECT * FROM data_index_bloom_encoding_stats WHERE "String" = 'test'; +---- +test + +query T +SELECT * FROM data_index_bloom_encoding_stats WHERE "String" like '%e%'; +---- +Hello +test +are you +the quick +over +the lazy + +statement ok +set datafusion.execution.parquet.bloom_filter_enabled=false; + + +######## +# Clean up after the test +######## +statement ok +DROP TABLE data_index_bloom_encoding_stats; From 48ea4b246b85afb58ebb6c375fd233928db71784 Mon Sep 17 00:00:00 2001 From: Huaijin Date: Thu, 26 Oct 2023 03:30:18 +0800 Subject: [PATCH 130/572] fix: don't push down volatile predicates in projection (#7909) * fix: don't push down volatile predicates in projection * Update datafusion/optimizer/src/push_down_filter.rs Co-authored-by: Andrew Lamb * Update datafusion/optimizer/src/push_down_filter.rs Co-authored-by: Andrew Lamb * Update datafusion/optimizer/src/push_down_filter.rs Co-authored-by: Andrew Lamb * add suggestions * fix * fix doc * Update datafusion/optimizer/src/push_down_filter.rs Co-authored-by: Jonah Gao * Update datafusion/optimizer/src/push_down_filter.rs Co-authored-by: Jonah Gao * Update datafusion/optimizer/src/push_down_filter.rs Co-authored-by: Jonah Gao * Update datafusion/optimizer/src/push_down_filter.rs Co-authored-by: Jonah Gao --------- Co-authored-by: Andrew Lamb Co-authored-by: Jonah Gao --- datafusion/optimizer/src/push_down_filter.rs | 194 ++++++++++++++++--- 1 file changed, 167 insertions(+), 27 deletions(-) diff --git a/datafusion/optimizer/src/push_down_filter.rs b/datafusion/optimizer/src/push_down_filter.rs index 4c5cd3ab2855..8c2eb96a48d8 100644 --- a/datafusion/optimizer/src/push_down_filter.rs +++ b/datafusion/optimizer/src/push_down_filter.rs @@ -15,13 +15,14 @@ //! Push Down Filter optimizer rule ensures that filters are applied as early as possible in the plan use crate::optimizer::ApplyOrder; -use crate::utils::{conjunction, split_conjunction}; +use crate::utils::{conjunction, split_conjunction, split_conjunction_owned}; use crate::{utils, OptimizerConfig, OptimizerRule}; use datafusion_common::tree_node::{Transformed, TreeNode, VisitRecursion}; use datafusion_common::{ internal_err, plan_datafusion_err, Column, DFSchema, DataFusionError, Result, }; use datafusion_expr::expr::Alias; +use datafusion_expr::Volatility; use datafusion_expr::{ and, expr_rewriter::replace_col, @@ -652,32 +653,60 @@ impl OptimizerRule for PushDownFilter { child_plan.with_new_inputs(&[new_filter])? } LogicalPlan::Projection(projection) => { - // A projection is filter-commutable, but re-writes all predicate expressions + // A projection is filter-commutable if it do not contain volatile predicates or contain volatile + // predicates that are not used in the filter. However, we should re-writes all predicate expressions. // collect projection. - let replace_map = projection - .schema - .fields() - .iter() - .enumerate() - .map(|(i, field)| { - // strip alias, as they should not be part of filters - let expr = match &projection.expr[i] { - Expr::Alias(Alias { expr, .. }) => expr.as_ref().clone(), - expr => expr.clone(), - }; - - (field.qualified_name(), expr) - }) - .collect::>(); + let (volatile_map, non_volatile_map): (HashMap<_, _>, HashMap<_, _>) = + projection + .schema + .fields() + .iter() + .enumerate() + .map(|(i, field)| { + // strip alias, as they should not be part of filters + let expr = match &projection.expr[i] { + Expr::Alias(Alias { expr, .. }) => expr.as_ref().clone(), + expr => expr.clone(), + }; + + (field.qualified_name(), expr) + }) + .partition(|(_, value)| is_volatile_expression(value)); - // re-write all filters based on this projection - // E.g. in `Filter: b\n Projection: a > 1 as b`, we can swap them, but the filter must be "a > 1" - let new_filter = LogicalPlan::Filter(Filter::try_new( - replace_cols_by_name(filter.predicate.clone(), &replace_map)?, - projection.input.clone(), - )?); + let mut push_predicates = vec![]; + let mut keep_predicates = vec![]; + for expr in split_conjunction_owned(filter.predicate.clone()).into_iter() + { + if contain(&expr, &volatile_map) { + keep_predicates.push(expr); + } else { + push_predicates.push(expr); + } + } - child_plan.with_new_inputs(&[new_filter])? + match conjunction(push_predicates) { + Some(expr) => { + // re-write all filters based on this projection + // E.g. in `Filter: b\n Projection: a > 1 as b`, we can swap them, but the filter must be "a > 1" + let new_filter = LogicalPlan::Filter(Filter::try_new( + replace_cols_by_name(expr, &non_volatile_map)?, + projection.input.clone(), + )?); + + match conjunction(keep_predicates) { + None => child_plan.with_new_inputs(&[new_filter])?, + Some(keep_predicate) => { + let child_plan = + child_plan.with_new_inputs(&[new_filter])?; + LogicalPlan::Filter(Filter::try_new( + keep_predicate, + Arc::new(child_plan), + )?) + } + } + } + None => return Ok(None), + } } LogicalPlan::Union(union) => { let mut inputs = Vec::with_capacity(union.inputs.len()); @@ -881,6 +910,42 @@ pub fn replace_cols_by_name( }) } +/// check whether the expression is volatile predicates +fn is_volatile_expression(e: &Expr) -> bool { + let mut is_volatile = false; + e.apply(&mut |expr| { + Ok(match expr { + Expr::ScalarFunction(f) if f.fun.volatility() == Volatility::Volatile => { + is_volatile = true; + VisitRecursion::Stop + } + _ => VisitRecursion::Continue, + }) + }) + .unwrap(); + is_volatile +} + +/// check whether the expression uses the columns in `check_map`. +fn contain(e: &Expr, check_map: &HashMap) -> bool { + let mut is_contain = false; + e.apply(&mut |expr| { + Ok(if let Expr::Column(c) = &expr { + match check_map.get(&c.flat_name()) { + Some(_) => { + is_contain = true; + VisitRecursion::Stop + } + None => VisitRecursion::Continue, + } + } else { + VisitRecursion::Continue + }) + }) + .unwrap(); + is_contain +} + #[cfg(test)] mod tests { use super::*; @@ -893,9 +958,9 @@ mod tests { use datafusion_common::{DFSchema, DFSchemaRef}; use datafusion_expr::logical_plan::table_scan; use datafusion_expr::{ - and, col, in_list, in_subquery, lit, logical_plan::JoinType, or, sum, BinaryExpr, - Expr, Extension, LogicalPlanBuilder, Operator, TableSource, TableType, - UserDefinedLogicalNodeCore, + and, col, in_list, in_subquery, lit, logical_plan::JoinType, or, random, sum, + BinaryExpr, Expr, Extension, LogicalPlanBuilder, Operator, TableSource, + TableType, UserDefinedLogicalNodeCore, }; use std::fmt::{Debug, Formatter}; use std::sync::Arc; @@ -2712,4 +2777,79 @@ Projection: a, b \n TableScan: test2"; assert_optimized_plan_eq(&plan, expected) } + + #[test] + fn test_push_down_volatile_function_in_aggregate() -> Result<()> { + // SELECT t.a, t.r FROM (SELECT a, SUM(b), random()+1 AS r FROM test1 GROUP BY a) AS t WHERE t.a > 5 AND t.r > 0.5; + let table_scan = test_table_scan_with_name("test1")?; + let plan = LogicalPlanBuilder::from(table_scan) + .aggregate(vec![col("a")], vec![sum(col("b"))])? + .project(vec![ + col("a"), + sum(col("b")), + add(random(), lit(1)).alias("r"), + ])? + .alias("t")? + .filter(col("t.a").gt(lit(5)).and(col("t.r").gt(lit(0.5))))? + .project(vec![col("t.a"), col("t.r")])? + .build()?; + + let expected_before = "Projection: t.a, t.r\ + \n Filter: t.a > Int32(5) AND t.r > Float64(0.5)\ + \n SubqueryAlias: t\ + \n Projection: test1.a, SUM(test1.b), random() + Int32(1) AS r\ + \n Aggregate: groupBy=[[test1.a]], aggr=[[SUM(test1.b)]]\ + \n TableScan: test1"; + assert_eq!(format!("{plan:?}"), expected_before); + + let expected_after = "Projection: t.a, t.r\ + \n SubqueryAlias: t\ + \n Filter: r > Float64(0.5)\ + \n Projection: test1.a, SUM(test1.b), random() + Int32(1) AS r\ + \n Aggregate: groupBy=[[test1.a]], aggr=[[SUM(test1.b)]]\ + \n TableScan: test1, full_filters=[test1.a > Int32(5)]"; + assert_optimized_plan_eq(&plan, expected_after) + } + + #[test] + fn test_push_down_volatile_function_in_join() -> Result<()> { + // SELECT t.a, t.r FROM (SELECT test1.a AS a, random() AS r FROM test1 join test2 ON test1.a = test2.a) AS t WHERE t.r > 0.5; + let table_scan = test_table_scan_with_name("test1")?; + let left = LogicalPlanBuilder::from(table_scan).build()?; + let right_table_scan = test_table_scan_with_name("test2")?; + let right = LogicalPlanBuilder::from(right_table_scan).build()?; + let plan = LogicalPlanBuilder::from(left) + .join( + right, + JoinType::Inner, + ( + vec![Column::from_qualified_name("test1.a")], + vec![Column::from_qualified_name("test2.a")], + ), + None, + )? + .project(vec![col("test1.a").alias("a"), random().alias("r")])? + .alias("t")? + .filter(col("t.r").gt(lit(0.8)))? + .project(vec![col("t.a"), col("t.r")])? + .build()?; + + let expected_before = "Projection: t.a, t.r\ + \n Filter: t.r > Float64(0.8)\ + \n SubqueryAlias: t\ + \n Projection: test1.a AS a, random() AS r\ + \n Inner Join: test1.a = test2.a\ + \n TableScan: test1\ + \n TableScan: test2"; + assert_eq!(format!("{plan:?}"), expected_before); + + let expected = "Projection: t.a, t.r\ + \n SubqueryAlias: t\ + \n Filter: r > Float64(0.8)\ + \n Projection: test1.a AS a, random() AS r\ + \n Inner Join: test1.a = test2.a\ + \n TableScan: test1\ + \n TableScan: test2"; + assert_optimized_plan_eq(&plan, expected) + } } From 12a63163a9ed872a8df82f8a23e6c4253f3eb8b9 Mon Sep 17 00:00:00 2001 From: Chih Wang Date: Thu, 26 Oct 2023 05:06:47 +0800 Subject: [PATCH 131/572] Add `parquet` feature flag, enabled by default, and make parquet conditional (#7745) * Make parquet an option by adding multiple cfg attributes without significant code changes. * Extract parquet logic into submodule from execution::context * Extract parquet logic into submodule from datafusion_core::dataframe * Extract more logic into submodule from execution::context * Move tests from execution::context * Rename submodules --- datafusion-cli/Cargo.toml | 2 +- datafusion/common/Cargo.toml | 3 +- datafusion/common/src/test_util.rs | 1 + datafusion/core/Cargo.toml | 10 +- .../src/{dataframe.rs => dataframe/mod.rs} | 151 +------- datafusion/core/src/dataframe/parquet.rs | 162 ++++++++ .../file_format/file_compression_type.rs | 18 +- .../core/src/datasource/file_format/mod.rs | 1 + .../src/datasource/file_format/options.rs | 7 +- .../core/src/datasource/listing/table.rs | 13 +- .../src/datasource/listing_table_factory.rs | 8 +- datafusion/core/src/datasource/mod.rs | 1 + .../core/src/datasource/physical_plan/mod.rs | 6 +- datafusion/core/src/execution/context/avro.rs | 83 ++++ datafusion/core/src/execution/context/csv.rs | 143 +++++++ datafusion/core/src/execution/context/json.rs | 69 ++++ .../execution/{context.rs => context/mod.rs} | 363 +----------------- .../core/src/execution/context/parquet.rs | 154 ++++++++ datafusion/core/src/lib.rs | 1 + .../enforce_distribution.rs | 67 +++- datafusion/core/src/physical_planner.rs | 2 + datafusion/core/src/test_util/mod.rs | 77 +++- datafusion/proto/Cargo.toml | 3 +- datafusion/proto/src/logical_plan/mod.rs | 66 ++-- datafusion/proto/src/physical_plan/mod.rs | 222 ++++++----- 25 files changed, 994 insertions(+), 639 deletions(-) rename datafusion/core/src/{dataframe.rs => dataframe/mod.rs} (94%) create mode 100644 datafusion/core/src/dataframe/parquet.rs create mode 100644 datafusion/core/src/execution/context/avro.rs create mode 100644 datafusion/core/src/execution/context/csv.rs create mode 100644 datafusion/core/src/execution/context/json.rs rename datafusion/core/src/execution/{context.rs => context/mod.rs} (87%) create mode 100644 datafusion/core/src/execution/context/parquet.rs diff --git a/datafusion-cli/Cargo.toml b/datafusion-cli/Cargo.toml index 64e094437c5f..7dd9cb8bcb37 100644 --- a/datafusion-cli/Cargo.toml +++ b/datafusion-cli/Cargo.toml @@ -34,7 +34,7 @@ async-trait = "0.1.41" aws-config = "0.55" aws-credential-types = "0.55" clap = { version = "3", features = ["derive", "cargo"] } -datafusion = { path = "../datafusion/core", version = "32.0.0", features = ["avro", "crypto_expressions", "encoding_expressions", "regex_expressions", "unicode_expressions", "compression"] } +datafusion = { path = "../datafusion/core", version = "32.0.0", features = ["avro", "crypto_expressions", "encoding_expressions", "parquet", "regex_expressions", "unicode_expressions", "compression"] } dirs = "4.0.0" env_logger = "0.9" mimalloc = { version = "0.1", default-features = false } diff --git a/datafusion/common/Cargo.toml b/datafusion/common/Cargo.toml index 047c502d5cc2..490fbeacad85 100644 --- a/datafusion/common/Cargo.toml +++ b/datafusion/common/Cargo.toml @@ -35,8 +35,7 @@ path = "src/lib.rs" [features] avro = ["apache-avro"] backtrace = [] -default = ["parquet"] -pyarrow = ["pyo3", "arrow/pyarrow"] +pyarrow = ["pyo3", "arrow/pyarrow", "parquet"] [dependencies] ahash = { version = "0.8", default-features = false, features = ["runtime-rng"] } diff --git a/datafusion/common/src/test_util.rs b/datafusion/common/src/test_util.rs index 60f1df7fd11a..9a4433782157 100644 --- a/datafusion/common/src/test_util.rs +++ b/datafusion/common/src/test_util.rs @@ -180,6 +180,7 @@ pub fn arrow_test_data() -> String { /// let filename = format!("{}/binary.parquet", testdata); /// assert!(std::path::PathBuf::from(filename).exists()); /// ``` +#[cfg(feature = "parquet")] pub fn parquet_test_data() -> String { match get_data_dir("PARQUET_TEST_DATA", "../../parquet-testing/data") { Ok(pb) => pb.display().to_string(), diff --git a/datafusion/core/Cargo.toml b/datafusion/core/Cargo.toml index 30e0d005e92e..5f9d28bd620b 100644 --- a/datafusion/core/Cargo.toml +++ b/datafusion/core/Cargo.toml @@ -39,11 +39,12 @@ avro = ["apache-avro", "num-traits", "datafusion-common/avro"] backtrace = ["datafusion-common/backtrace"] compression = ["xz2", "bzip2", "flate2", "zstd", "async-compression"] crypto_expressions = ["datafusion-physical-expr/crypto_expressions", "datafusion-optimizer/crypto_expressions"] -default = ["crypto_expressions", "encoding_expressions", "regex_expressions", "unicode_expressions", "compression"] +default = ["crypto_expressions", "encoding_expressions", "regex_expressions", "unicode_expressions", "compression", "parquet"] encoding_expressions = ["datafusion-physical-expr/encoding_expressions"] # Used for testing ONLY: causes all values to hash to the same value (test for collisions) force_hash_collisions = [] -pyarrow = ["datafusion-common/pyarrow"] +parquet = ["datafusion-common/parquet", "dep:parquet"] +pyarrow = ["datafusion-common/pyarrow", "parquet"] regex_expressions = ["datafusion-physical-expr/regex_expressions", "datafusion-optimizer/regex_expressions"] serde = ["arrow-schema/serde"] simd = ["arrow/simd"] @@ -61,7 +62,7 @@ bytes = "1.4" bzip2 = { version = "0.4.3", optional = true } chrono = { workspace = true } dashmap = "5.4.0" -datafusion-common = { path = "../common", version = "32.0.0", features = ["parquet", "object_store"] } +datafusion-common = { path = "../common", version = "32.0.0", features = ["object_store"], default-features = false } datafusion-execution = { path = "../execution", version = "32.0.0" } datafusion-expr = { path = "../expr", version = "32.0.0" } datafusion-optimizer = { path = "../optimizer", version = "32.0.0", default-features = false } @@ -80,7 +81,7 @@ num-traits = { version = "0.2", optional = true } num_cpus = "1.13.0" object_store = "0.7.0" parking_lot = "0.12" -parquet = { workspace = true } +parquet = { workspace = true, optional = true } percent-encoding = "2.2.0" pin-project-lite = "^0.2.7" rand = "0.8" @@ -93,7 +94,6 @@ uuid = { version = "1.0", features = ["v4"] } xz2 = { version = "0.1", optional = true } zstd = { version = "0.13", optional = true, default-features = false } - [dev-dependencies] async-trait = "0.1.53" bigdecimal = "0.4.1" diff --git a/datafusion/core/src/dataframe.rs b/datafusion/core/src/dataframe/mod.rs similarity index 94% rename from datafusion/core/src/dataframe.rs rename to datafusion/core/src/dataframe/mod.rs index 2e192c2a782e..0a99c331826c 100644 --- a/datafusion/core/src/dataframe.rs +++ b/datafusion/core/src/dataframe/mod.rs @@ -17,6 +17,9 @@ //! [`DataFrame`] API for building and executing query plans. +#[cfg(feature = "parquet")] +mod parquet; + use std::any::Any; use std::sync::Arc; @@ -27,15 +30,11 @@ use arrow::datatypes::{DataType, Field}; use async_trait::async_trait; use datafusion_common::file_options::csv_writer::CsvWriterOptions; use datafusion_common::file_options::json_writer::JsonWriterOptions; -use datafusion_common::file_options::parquet_writer::{ - default_builder, ParquetWriterOptions, -}; use datafusion_common::parsers::CompressionTypeVariant; use datafusion_common::{ DataFusionError, FileType, FileTypeWriterOptions, SchemaError, UnnestOptions, }; use datafusion_expr::dml::CopyOptions; -use parquet::file::properties::WriterProperties; use datafusion_common::{Column, DFSchema, ScalarValue}; use datafusion_expr::{ @@ -1067,40 +1066,6 @@ impl DataFrame { DataFrame::new(self.session_state, plan).collect().await } - /// Write a `DataFrame` to a Parquet file. - pub async fn write_parquet( - self, - path: &str, - options: DataFrameWriteOptions, - writer_properties: Option, - ) -> Result, DataFusionError> { - if options.overwrite { - return Err(DataFusionError::NotImplemented( - "Overwrites are not implemented for DataFrame::write_parquet.".to_owned(), - )); - } - match options.compression{ - CompressionTypeVariant::UNCOMPRESSED => (), - _ => return Err(DataFusionError::Configuration("DataFrame::write_parquet method does not support compression set via DataFrameWriteOptions. Set parquet compression via writer_properties instead.".to_owned())) - } - let props = match writer_properties { - Some(props) => props, - None => default_builder(self.session_state.config_options())?.build(), - }; - let file_type_writer_options = - FileTypeWriterOptions::Parquet(ParquetWriterOptions::new(props)); - let copy_options = CopyOptions::WriterOptions(Box::new(file_type_writer_options)); - let plan = LogicalPlanBuilder::copy_to( - self.plan, - path.into(), - FileType::PARQUET, - options.single_file_output, - copy_options, - )? - .build()?; - DataFrame::new(self.session_state, plan).collect().await - } - /// Executes a query and writes the results to a partitioned JSON file. pub async fn write_json( self, @@ -1365,19 +1330,12 @@ mod tests { WindowFunction, }; use datafusion_physical_expr::expressions::Column; - use object_store::local::LocalFileSystem; - use parquet::basic::{BrotliLevel, GzipLevel, ZstdLevel}; - use parquet::file::reader::FileReader; - use tempfile::TempDir; - use url::Url; use crate::execution::context::SessionConfig; - use crate::execution::options::{CsvReadOptions, ParquetReadOptions}; use crate::physical_plan::ColumnarValue; use crate::physical_plan::Partitioning; use crate::physical_plan::PhysicalExpr; - use crate::test_util; - use crate::test_util::parquet_test_data; + use crate::test_util::{register_aggregate_csv, test_table, test_table_with_name}; use crate::{assert_batches_sorted_eq, execution::context::SessionContext}; use super::*; @@ -1798,31 +1756,6 @@ mod tests { Ok(ctx.sql(sql).await?.into_unoptimized_plan()) } - async fn test_table_with_name(name: &str) -> Result { - let mut ctx = SessionContext::new(); - register_aggregate_csv(&mut ctx, name).await?; - ctx.table(name).await - } - - async fn test_table() -> Result { - test_table_with_name("aggregate_test_100").await - } - - async fn register_aggregate_csv( - ctx: &mut SessionContext, - table_name: &str, - ) -> Result<()> { - let schema = test_util::aggr_test_schema(); - let testdata = test_util::arrow_test_data(); - ctx.register_csv( - table_name, - &format!("{testdata}/csv/aggregate_test_100.csv"), - CsvReadOptions::new().schema(schema.as_ref()), - ) - .await?; - Ok(()) - } - #[tokio::test] async fn with_column() -> Result<()> { let df = test_table().await?.select_columns(&["c1", "c2", "c3"])?; @@ -2227,33 +2160,6 @@ mod tests { Ok(()) } - #[tokio::test] - async fn filter_pushdown_dataframe() -> Result<()> { - let ctx = SessionContext::new(); - - ctx.register_parquet( - "test", - &format!("{}/alltypes_plain.snappy.parquet", parquet_test_data()), - ParquetReadOptions::default(), - ) - .await?; - - ctx.register_table("t1", ctx.table("test").await?.into_view())?; - - let df = ctx - .table("t1") - .await? - .filter(col("id").eq(lit(1)))? - .select_columns(&["bool_col", "int_col"])?; - - 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")); - - Ok(()) - } - #[tokio::test] async fn cast_expr_test() -> Result<()> { let df = test_table() @@ -2538,53 +2444,4 @@ mod tests { Ok(()) } - - #[tokio::test] - async fn write_parquet_with_compression() -> Result<()> { - let test_df = test_table().await?; - - let output_path = "file://local/test.parquet"; - let test_compressions = vec![ - parquet::basic::Compression::SNAPPY, - parquet::basic::Compression::LZ4, - parquet::basic::Compression::LZ4_RAW, - parquet::basic::Compression::GZIP(GzipLevel::default()), - parquet::basic::Compression::BROTLI(BrotliLevel::default()), - parquet::basic::Compression::ZSTD(ZstdLevel::default()), - ]; - for compression in test_compressions.into_iter() { - let df = test_df.clone(); - let tmp_dir = TempDir::new()?; - let local = Arc::new(LocalFileSystem::new_with_prefix(&tmp_dir)?); - let local_url = Url::parse("file://local").unwrap(); - let ctx = &test_df.session_state; - ctx.runtime_env().register_object_store(&local_url, local); - df.write_parquet( - output_path, - DataFrameWriteOptions::new().with_single_file_output(true), - Some( - WriterProperties::builder() - .set_compression(compression) - .build(), - ), - ) - .await?; - - // Check that file actually used the specified compression - let file = std::fs::File::open(tmp_dir.into_path().join("test.parquet"))?; - - let reader = - parquet::file::serialized_reader::SerializedFileReader::new(file) - .unwrap(); - - let parquet_metadata = reader.metadata(); - - let written_compression = - parquet_metadata.row_group(0).column(0).compression(); - - assert_eq!(written_compression, compression); - } - - Ok(()) - } } diff --git a/datafusion/core/src/dataframe/parquet.rs b/datafusion/core/src/dataframe/parquet.rs new file mode 100644 index 000000000000..36ef90c987e3 --- /dev/null +++ b/datafusion/core/src/dataframe/parquet.rs @@ -0,0 +1,162 @@ +// 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 datafusion_common::file_options::parquet_writer::{ + default_builder, ParquetWriterOptions, +}; +use parquet::file::properties::WriterProperties; + +use super::{ + CompressionTypeVariant, CopyOptions, DataFrame, DataFrameWriteOptions, + DataFusionError, FileType, FileTypeWriterOptions, LogicalPlanBuilder, RecordBatch, +}; + +impl DataFrame { + /// Write a `DataFrame` to a Parquet file. + pub async fn write_parquet( + self, + path: &str, + options: DataFrameWriteOptions, + writer_properties: Option, + ) -> Result, DataFusionError> { + if options.overwrite { + return Err(DataFusionError::NotImplemented( + "Overwrites are not implemented for DataFrame::write_parquet.".to_owned(), + )); + } + match options.compression{ + CompressionTypeVariant::UNCOMPRESSED => (), + _ => return Err(DataFusionError::Configuration("DataFrame::write_parquet method does not support compression set via DataFrameWriteOptions. Set parquet compression via writer_properties instead.".to_owned())) + } + let props = match writer_properties { + Some(props) => props, + None => default_builder(self.session_state.config_options())?.build(), + }; + let file_type_writer_options = + FileTypeWriterOptions::Parquet(ParquetWriterOptions::new(props)); + let copy_options = CopyOptions::WriterOptions(Box::new(file_type_writer_options)); + let plan = LogicalPlanBuilder::copy_to( + self.plan, + path.into(), + FileType::PARQUET, + options.single_file_output, + copy_options, + )? + .build()?; + DataFrame::new(self.session_state, plan).collect().await + } +} + +#[cfg(test)] +mod tests { + use std::sync::Arc; + + use object_store::local::LocalFileSystem; + use parquet::basic::{BrotliLevel, GzipLevel, ZstdLevel}; + use parquet::file::reader::FileReader; + use tempfile::TempDir; + use url::Url; + + use datafusion_expr::{col, lit}; + + use crate::arrow::util::pretty; + use crate::execution::context::SessionContext; + use crate::execution::options::ParquetReadOptions; + use crate::test_util; + + use super::super::Result; + use super::*; + + #[tokio::test] + async fn filter_pushdown_dataframe() -> Result<()> { + let ctx = SessionContext::new(); + + ctx.register_parquet( + "test", + &format!( + "{}/alltypes_plain.snappy.parquet", + test_util::parquet_test_data() + ), + ParquetReadOptions::default(), + ) + .await?; + + ctx.register_table("t1", ctx.table("test").await?.into_view())?; + + let df = ctx + .table("t1") + .await? + .filter(col("id").eq(lit(1)))? + .select_columns(&["bool_col", "int_col"])?; + + 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")); + + Ok(()) + } + + #[tokio::test] + async fn write_parquet_with_compression() -> Result<()> { + let test_df = test_util::test_table().await?; + + let output_path = "file://local/test.parquet"; + let test_compressions = vec![ + parquet::basic::Compression::SNAPPY, + parquet::basic::Compression::LZ4, + parquet::basic::Compression::LZ4_RAW, + parquet::basic::Compression::GZIP(GzipLevel::default()), + parquet::basic::Compression::BROTLI(BrotliLevel::default()), + parquet::basic::Compression::ZSTD(ZstdLevel::default()), + ]; + for compression in test_compressions.into_iter() { + let df = test_df.clone(); + let tmp_dir = TempDir::new()?; + let local = Arc::new(LocalFileSystem::new_with_prefix(&tmp_dir)?); + let local_url = Url::parse("file://local").unwrap(); + let ctx = &test_df.session_state; + ctx.runtime_env().register_object_store(&local_url, local); + df.write_parquet( + output_path, + DataFrameWriteOptions::new().with_single_file_output(true), + Some( + WriterProperties::builder() + .set_compression(compression) + .build(), + ), + ) + .await?; + + // Check that file actually used the specified compression + let file = std::fs::File::open(tmp_dir.into_path().join("test.parquet"))?; + + let reader = + parquet::file::serialized_reader::SerializedFileReader::new(file) + .unwrap(); + + let parquet_metadata = reader.metadata(); + + let written_compression = + parquet_metadata.row_group(0).column(0).compression(); + + assert_eq!(written_compression, compression); + } + + Ok(()) + } +} diff --git a/datafusion/core/src/datasource/file_format/file_compression_type.rs b/datafusion/core/src/datasource/file_format/file_compression_type.rs index bd2868767090..3dac7c293050 100644 --- a/datafusion/core/src/datasource/file_format/file_compression_type.rs +++ b/datafusion/core/src/datasource/file_format/file_compression_type.rs @@ -237,7 +237,14 @@ impl FileTypeExt for FileType { match self { FileType::JSON | FileType::CSV => Ok(format!("{}{}", ext, c.get_ext())), - FileType::PARQUET | FileType::AVRO | FileType::ARROW => match c.variant { + FileType::AVRO | FileType::ARROW => match c.variant { + UNCOMPRESSED => Ok(ext), + _ => Err(DataFusionError::Internal( + "FileCompressionType can be specified for CSV/JSON FileType.".into(), + )), + }, + #[cfg(feature = "parquet")] + FileType::PARQUET => match c.variant { UNCOMPRESSED => Ok(ext), _ => Err(DataFusionError::Internal( "FileCompressionType can be specified for CSV/JSON FileType.".into(), @@ -276,10 +283,13 @@ mod tests { ); } + let mut ty_ext_tuple = vec![]; + ty_ext_tuple.push((FileType::AVRO, ".avro")); + #[cfg(feature = "parquet")] + ty_ext_tuple.push((FileType::PARQUET, ".parquet")); + // Cannot specify compression for these file types - for (file_type, extension) in - [(FileType::AVRO, ".avro"), (FileType::PARQUET, ".parquet")] - { + for (file_type, extension) in ty_ext_tuple { assert_eq!( file_type .get_ext_with_compression(FileCompressionType::UNCOMPRESSED) diff --git a/datafusion/core/src/datasource/file_format/mod.rs b/datafusion/core/src/datasource/file_format/mod.rs index 293f062d86a9..b541e2a1d44c 100644 --- a/datafusion/core/src/datasource/file_format/mod.rs +++ b/datafusion/core/src/datasource/file_format/mod.rs @@ -27,6 +27,7 @@ pub mod csv; pub mod file_compression_type; pub mod json; pub mod options; +#[cfg(feature = "parquet")] pub mod parquet; pub mod write; diff --git a/datafusion/core/src/datasource/file_format/options.rs b/datafusion/core/src/datasource/file_format/options.rs index 40d9878a0134..41a70e6d2f8f 100644 --- a/datafusion/core/src/datasource/file_format/options.rs +++ b/datafusion/core/src/datasource/file_format/options.rs @@ -25,12 +25,12 @@ use datafusion_common::{plan_err, DataFusionError}; use crate::datasource::file_format::arrow::ArrowFormat; use crate::datasource::file_format::file_compression_type::FileCompressionType; +#[cfg(feature = "parquet")] +use crate::datasource::file_format::parquet::ParquetFormat; use crate::datasource::file_format::DEFAULT_SCHEMA_INFER_MAX_RECORD; use crate::datasource::listing::{ListingTableInsertMode, ListingTableUrl}; use crate::datasource::{ - file_format::{ - avro::AvroFormat, csv::CsvFormat, json::JsonFormat, parquet::ParquetFormat, - }, + file_format::{avro::AvroFormat, csv::CsvFormat, json::JsonFormat}, listing::ListingOptions, }; use crate::error::Result; @@ -542,6 +542,7 @@ impl ReadOptions<'_> for CsvReadOptions<'_> { } } +#[cfg(feature = "parquet")] #[async_trait] impl ReadOptions<'_> for ParquetReadOptions<'_> { fn to_listing_options(&self, config: &SessionConfig) -> ListingOptions { diff --git a/datafusion/core/src/datasource/listing/table.rs b/datafusion/core/src/datasource/listing/table.rs index bd878932d80f..822a78a5522a 100644 --- a/datafusion/core/src/datasource/listing/table.rs +++ b/datafusion/core/src/datasource/listing/table.rs @@ -23,6 +23,8 @@ use std::{any::Any, sync::Arc}; use super::helpers::{expr_applicable_for_cols, pruned_partition_list, split_files}; use super::PartitionedFile; +#[cfg(feature = "parquet")] +use crate::datasource::file_format::parquet::ParquetFormat; use crate::datasource::{ file_format::{ arrow::ArrowFormat, @@ -30,7 +32,6 @@ use crate::datasource::{ csv::CsvFormat, file_compression_type::{FileCompressionType, FileTypeExt}, json::JsonFormat, - parquet::ParquetFormat, FileFormat, }, get_statistics_with_limit, @@ -150,6 +151,7 @@ impl ListingTableConfig { FileType::JSON => Arc::new( JsonFormat::default().with_file_compression_type(file_compression_type), ), + #[cfg(feature = "parquet")] FileType::PARQUET => Arc::new(ParquetFormat::default()), }; @@ -1019,15 +1021,15 @@ mod tests { use std::fs::File; use super::*; + #[cfg(feature = "parquet")] + use crate::datasource::file_format::parquet::ParquetFormat; use crate::datasource::{provider_as_source, MemTable}; use crate::execution::options::ArrowReadOptions; use crate::physical_plan::collect; use crate::prelude::*; use crate::{ assert_batches_eq, - datasource::file_format::{ - avro::AvroFormat, file_compression_type::FileTypeExt, parquet::ParquetFormat, - }, + datasource::file_format::{avro::AvroFormat, file_compression_type::FileTypeExt}, execution::options::ReadOptions, logical_expr::{col, lit}, test::{columns, object_store::register_test_store}, @@ -1090,6 +1092,7 @@ mod tests { Ok(()) } + #[cfg(feature = "parquet")] #[tokio::test] async fn load_table_stats_by_default() -> Result<()> { let testdata = crate::test_util::parquet_test_data(); @@ -1113,6 +1116,7 @@ mod tests { Ok(()) } + #[cfg(feature = "parquet")] #[tokio::test] async fn load_table_stats_when_no_stats() -> Result<()> { let testdata = crate::test_util::parquet_test_data(); @@ -1137,6 +1141,7 @@ mod tests { Ok(()) } + #[cfg(feature = "parquet")] #[tokio::test] async fn test_try_create_output_ordering() { let testdata = crate::test_util::parquet_test_data(); diff --git a/datafusion/core/src/datasource/listing_table_factory.rs b/datafusion/core/src/datasource/listing_table_factory.rs index e74bf6fa6499..26f40518979a 100644 --- a/datafusion/core/src/datasource/listing_table_factory.rs +++ b/datafusion/core/src/datasource/listing_table_factory.rs @@ -23,10 +23,11 @@ use std::sync::Arc; use super::listing::ListingTableInsertMode; +#[cfg(feature = "parquet")] +use crate::datasource::file_format::parquet::ParquetFormat; use crate::datasource::file_format::{ arrow::ArrowFormat, avro::AvroFormat, csv::CsvFormat, - file_compression_type::FileCompressionType, json::JsonFormat, parquet::ParquetFormat, - FileFormat, + file_compression_type::FileCompressionType, json::JsonFormat, FileFormat, }; use crate::datasource::listing::{ ListingOptions, ListingTable, ListingTableConfig, ListingTableUrl, @@ -79,6 +80,7 @@ impl TableProviderFactory for ListingTableFactory { .with_delimiter(cmd.delimiter as u8) .with_file_compression_type(file_compression_type), ), + #[cfg(feature = "parquet")] FileType::PARQUET => Arc::new(ParquetFormat::default()), FileType::AVRO => Arc::new(AvroFormat), FileType::JSON => Arc::new( @@ -157,6 +159,7 @@ impl TableProviderFactory for ListingTableFactory { Some(mode) => ListingTableInsertMode::from_str(mode.as_str()), None => match file_type { FileType::CSV => Ok(ListingTableInsertMode::AppendToFile), + #[cfg(feature = "parquet")] FileType::PARQUET => Ok(ListingTableInsertMode::AppendNewFiles), FileType::AVRO => Ok(ListingTableInsertMode::AppendNewFiles), FileType::JSON => Ok(ListingTableInsertMode::AppendToFile), @@ -196,6 +199,7 @@ impl TableProviderFactory for ListingTableFactory { json_writer_options.compression = cmd.file_compression_type; FileTypeWriterOptions::JSON(json_writer_options) } + #[cfg(feature = "parquet")] FileType::PARQUET => file_type_writer_options, FileType::ARROW => file_type_writer_options, FileType::AVRO => file_type_writer_options, diff --git a/datafusion/core/src/datasource/mod.rs b/datafusion/core/src/datasource/mod.rs index 455818056f2c..3ace2c239852 100644 --- a/datafusion/core/src/datasource/mod.rs +++ b/datafusion/core/src/datasource/mod.rs @@ -42,5 +42,6 @@ pub use self::memory::MemTable; pub use self::provider::TableProvider; pub use self::view::ViewTable; pub use crate::logical_expr::TableType; +#[cfg(feature = "parquet")] pub(crate) use statistics::get_col_stats; pub use statistics::get_statistics_with_limit; diff --git a/datafusion/core/src/datasource/physical_plan/mod.rs b/datafusion/core/src/datasource/physical_plan/mod.rs index 57844aac5181..3f84f87eb5d5 100644 --- a/datafusion/core/src/datasource/physical_plan/mod.rs +++ b/datafusion/core/src/datasource/physical_plan/mod.rs @@ -23,17 +23,20 @@ mod csv; mod file_scan_config; mod file_stream; mod json; +#[cfg(feature = "parquet")] pub mod parquet; pub(crate) use self::csv::plan_to_csv; pub use self::csv::{CsvConfig, CsvExec, CsvOpener}; -pub(crate) use self::file_scan_config::PartitionColumnProjector; pub(crate) use self::json::plan_to_json; +#[cfg(feature = "parquet")] pub(crate) use self::parquet::plan_to_parquet; +#[cfg(feature = "parquet")] pub use self::parquet::{ParquetExec, ParquetFileMetrics, ParquetFileReaderFactory}; pub use arrow_file::ArrowExec; pub use avro::AvroExec; +use file_scan_config::PartitionColumnProjector; pub use file_scan_config::{ wrap_partition_type_in_dict, wrap_partition_value_in_dict, FileScanConfig, }; @@ -798,6 +801,7 @@ mod tests { } /// Unit tests for `repartition_file_groups()` + #[cfg(feature = "parquet")] mod repartition_file_groups_test { use datafusion_common::Statistics; use itertools::Itertools; diff --git a/datafusion/core/src/execution/context/avro.rs b/datafusion/core/src/execution/context/avro.rs new file mode 100644 index 000000000000..d60e79862ef2 --- /dev/null +++ b/datafusion/core/src/execution/context/avro.rs @@ -0,0 +1,83 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +use std::sync::Arc; + +use super::super::options::{AvroReadOptions, ReadOptions}; +use super::{DataFilePaths, DataFrame, Result, SessionContext}; + +impl SessionContext { + /// Creates a [`DataFrame`] for reading an Avro data source. + /// + /// For more control such as reading multiple files, you can use + /// [`read_table`](Self::read_table) with a [`super::ListingTable`]. + /// + /// For an example, see [`read_csv`](Self::read_csv) + pub async fn read_avro( + &self, + table_paths: P, + options: AvroReadOptions<'_>, + ) -> Result { + self._read_type(table_paths, options).await + } + + /// Registers an Avro file as a table that can be referenced from + /// SQL statements executed against this context. + pub async fn register_avro( + &self, + name: &str, + table_path: &str, + options: AvroReadOptions<'_>, + ) -> Result<()> { + let listing_options = options.to_listing_options(&self.copied_config()); + + self.register_listing_table( + name, + table_path, + listing_options, + options.schema.map(|s| Arc::new(s.to_owned())), + None, + ) + .await?; + Ok(()) + } +} + +#[cfg(test)] +mod tests { + use super::*; + use async_trait::async_trait; + + // Test for compilation error when calling read_* functions from an #[async_trait] function. + // See https://github.com/apache/arrow-datafusion/issues/1154 + #[async_trait] + trait CallReadTrait { + async fn call_read_avro(&self) -> DataFrame; + } + + struct CallRead {} + + #[async_trait] + impl CallReadTrait for CallRead { + async fn call_read_avro(&self) -> DataFrame { + let ctx = SessionContext::new(); + ctx.read_avro("dummy", AvroReadOptions::default()) + .await + .unwrap() + } + } +} diff --git a/datafusion/core/src/execution/context/csv.rs b/datafusion/core/src/execution/context/csv.rs new file mode 100644 index 000000000000..f3675422c7d5 --- /dev/null +++ b/datafusion/core/src/execution/context/csv.rs @@ -0,0 +1,143 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +use std::sync::Arc; + +use crate::datasource::physical_plan::plan_to_csv; + +use super::super::options::{CsvReadOptions, ReadOptions}; +use super::{DataFilePaths, DataFrame, ExecutionPlan, Result, SessionContext}; + +impl SessionContext { + /// Creates a [`DataFrame`] for reading a CSV data source. + /// + /// For more control such as reading multiple files, you can use + /// [`read_table`](Self::read_table) with a [`super::ListingTable`]. + /// + /// Example usage is given below: + /// + /// ``` + /// use datafusion::prelude::*; + /// # use datafusion::error::Result; + /// # #[tokio::main] + /// # async fn main() -> Result<()> { + /// let ctx = SessionContext::new(); + /// // You can read a single file using `read_csv` + /// let df = ctx.read_csv("tests/data/example.csv", CsvReadOptions::new()).await?; + /// // you can also read multiple files: + /// let df = ctx.read_csv(vec!["tests/data/example.csv", "tests/data/example.csv"], CsvReadOptions::new()).await?; + /// # Ok(()) + /// # } + /// ``` + pub async fn read_csv( + &self, + table_paths: P, + options: CsvReadOptions<'_>, + ) -> Result { + self._read_type(table_paths, options).await + } + + /// Registers a CSV file as a table which can referenced from SQL + /// statements executed against this context. + pub async fn register_csv( + &self, + name: &str, + table_path: &str, + options: CsvReadOptions<'_>, + ) -> Result<()> { + let listing_options = options.to_listing_options(&self.copied_config()); + + self.register_listing_table( + name, + table_path, + listing_options, + options.schema.map(|s| Arc::new(s.to_owned())), + None, + ) + .await?; + + Ok(()) + } + + /// Executes a query and writes the results to a partitioned CSV file. + pub async fn write_csv( + &self, + plan: Arc, + path: impl AsRef, + ) -> Result<()> { + plan_to_csv(self.task_ctx(), plan, path).await + } +} + +#[cfg(test)] +mod tests { + use super::*; + use crate::assert_batches_eq; + use crate::test_util::{plan_and_collect, populate_csv_partitions}; + use async_trait::async_trait; + use tempfile::TempDir; + + #[tokio::test] + async fn query_csv_with_custom_partition_extension() -> Result<()> { + let tmp_dir = TempDir::new()?; + + // The main stipulation of this test: use a file extension that isn't .csv. + let file_extension = ".tst"; + + let ctx = SessionContext::new(); + let schema = populate_csv_partitions(&tmp_dir, 2, file_extension)?; + ctx.register_csv( + "test", + tmp_dir.path().to_str().unwrap(), + CsvReadOptions::new() + .schema(&schema) + .file_extension(file_extension), + ) + .await?; + let results = + plan_and_collect(&ctx, "SELECT SUM(c1), SUM(c2), COUNT(*) FROM test").await?; + + assert_eq!(results.len(), 1); + let expected = [ + "+--------------+--------------+----------+", + "| SUM(test.c1) | SUM(test.c2) | COUNT(*) |", + "+--------------+--------------+----------+", + "| 10 | 110 | 20 |", + "+--------------+--------------+----------+", + ]; + assert_batches_eq!(expected, &results); + + Ok(()) + } + + // Test for compilation error when calling read_* functions from an #[async_trait] function. + // See https://github.com/apache/arrow-datafusion/issues/1154 + #[async_trait] + trait CallReadTrait { + async fn call_read_csv(&self) -> DataFrame; + } + + struct CallRead {} + + #[async_trait] + impl CallReadTrait for CallRead { + async fn call_read_csv(&self) -> DataFrame { + let ctx = SessionContext::new(); + ctx.read_csv("dummy", CsvReadOptions::new()).await.unwrap() + } + } +} diff --git a/datafusion/core/src/execution/context/json.rs b/datafusion/core/src/execution/context/json.rs new file mode 100644 index 000000000000..f67693aa8f31 --- /dev/null +++ b/datafusion/core/src/execution/context/json.rs @@ -0,0 +1,69 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +use std::sync::Arc; + +use crate::datasource::physical_plan::plan_to_json; + +use super::super::options::{NdJsonReadOptions, ReadOptions}; +use super::{DataFilePaths, DataFrame, ExecutionPlan, Result, SessionContext}; + +impl SessionContext { + /// Creates a [`DataFrame`] for reading an JSON data source. + /// + /// For more control such as reading multiple files, you can use + /// [`read_table`](Self::read_table) with a [`super::ListingTable`]. + /// + /// For an example, see [`read_csv`](Self::read_csv) + pub async fn read_json( + &self, + table_paths: P, + options: NdJsonReadOptions<'_>, + ) -> Result { + self._read_type(table_paths, options).await + } + + /// Registers a JSON file as a table that it can be referenced + /// from SQL statements executed against this context. + pub async fn register_json( + &self, + name: &str, + table_path: &str, + options: NdJsonReadOptions<'_>, + ) -> Result<()> { + let listing_options = options.to_listing_options(&self.copied_config()); + + self.register_listing_table( + name, + table_path, + listing_options, + options.schema.map(|s| Arc::new(s.to_owned())), + None, + ) + .await?; + Ok(()) + } + + /// Executes a query and writes the results to a partitioned JSON file. + pub async fn write_json( + &self, + plan: Arc, + path: impl AsRef, + ) -> Result<()> { + plan_to_json(self.task_ctx(), plan, path).await + } +} diff --git a/datafusion/core/src/execution/context.rs b/datafusion/core/src/execution/context/mod.rs similarity index 87% rename from datafusion/core/src/execution/context.rs rename to datafusion/core/src/execution/context/mod.rs index 8bd4de742d69..d523c39ee01e 100644 --- a/datafusion/core/src/execution/context.rs +++ b/datafusion/core/src/execution/context/mod.rs @@ -16,6 +16,13 @@ // under the License. //! [`SessionContext`] contains methods for registering data sources and executing queries + +mod avro; +mod csv; +mod json; +#[cfg(feature = "parquet")] +mod parquet; + use crate::{ catalog::{CatalogList, MemoryCatalogList}, datasource::{ @@ -77,7 +84,6 @@ use datafusion_sql::{ use sqlparser::dialect::dialect_from_str; use crate::config::ConfigOptions; -use crate::datasource::physical_plan::{plan_to_csv, plan_to_json, plan_to_parquet}; use crate::execution::{runtime_env::RuntimeEnv, FunctionRegistry}; use crate::physical_plan::udaf::AggregateUDF; use crate::physical_plan::udf::ScalarUDF; @@ -92,7 +98,6 @@ use datafusion_sql::{ parser::DFParser, planner::{ContextProvider, SqlToRel}, }; -use parquet::file::properties::WriterProperties; use url::Url; use crate::catalog::information_schema::{InformationSchemaProvider, INFORMATION_SCHEMA}; @@ -110,9 +115,7 @@ use crate::execution::options::ArrowReadOptions; pub use datafusion_execution::config::SessionConfig; pub use datafusion_execution::TaskContext; -use super::options::{ - AvroReadOptions, CsvReadOptions, NdJsonReadOptions, ParquetReadOptions, ReadOptions, -}; +use super::options::ReadOptions; /// DataFilePaths adds a method to convert strings and vector of strings to vector of [`ListingTableUrl`] URLs. /// This allows methods such [`SessionContext::read_csv`] and [`SessionContext::read_avro`] @@ -856,34 +859,6 @@ impl SessionContext { self.read_table(Arc::new(provider)) } - /// Creates a [`DataFrame`] for reading an Avro data source. - /// - /// For more control such as reading multiple files, you can use - /// [`read_table`](Self::read_table) with a [`ListingTable`]. - /// - /// For an example, see [`read_csv`](Self::read_csv) - pub async fn read_avro( - &self, - table_paths: P, - options: AvroReadOptions<'_>, - ) -> Result { - self._read_type(table_paths, options).await - } - - /// Creates a [`DataFrame`] for reading an JSON data source. - /// - /// For more control such as reading multiple files, you can use - /// [`read_table`](Self::read_table) with a [`ListingTable`]. - /// - /// For an example, see [`read_csv`](Self::read_csv) - pub async fn read_json( - &self, - table_paths: P, - options: NdJsonReadOptions<'_>, - ) -> Result { - self._read_type(table_paths, options).await - } - /// Creates a [`DataFrame`] for reading an Arrow data source. /// /// For more control such as reading multiple files, you can use @@ -906,48 +881,6 @@ impl SessionContext { )) } - /// Creates a [`DataFrame`] for reading a CSV data source. - /// - /// For more control such as reading multiple files, you can use - /// [`read_table`](Self::read_table) with a [`ListingTable`]. - /// - /// Example usage is given below: - /// - /// ``` - /// use datafusion::prelude::*; - /// # use datafusion::error::Result; - /// # #[tokio::main] - /// # async fn main() -> Result<()> { - /// let ctx = SessionContext::new(); - /// // You can read a single file using `read_csv` - /// let df = ctx.read_csv("tests/data/example.csv", CsvReadOptions::new()).await?; - /// // you can also read multiple files: - /// let df = ctx.read_csv(vec!["tests/data/example.csv", "tests/data/example.csv"], CsvReadOptions::new()).await?; - /// # Ok(()) - /// # } - /// ``` - pub async fn read_csv( - &self, - table_paths: P, - options: CsvReadOptions<'_>, - ) -> Result { - self._read_type(table_paths, options).await - } - - /// Creates a [`DataFrame`] for reading a Parquet data source. - /// - /// For more control such as reading multiple files, you can use - /// [`read_table`](Self::read_table) with a [`ListingTable`]. - /// - /// For an example, see [`read_csv`](Self::read_csv) - pub async fn read_parquet( - &self, - table_paths: P, - options: ParquetReadOptions<'_>, - ) -> Result { - self._read_type(table_paths, options).await - } - /// Creates a [`DataFrame`] for a [`TableProvider`] such as a /// [`ListingTable`] or a custom user defined provider. pub fn read_table(&self, provider: Arc) -> Result { @@ -1008,91 +941,6 @@ impl SessionContext { Ok(()) } - /// Registers a CSV file as a table which can referenced from SQL - /// statements executed against this context. - pub async fn register_csv( - &self, - name: &str, - table_path: &str, - options: CsvReadOptions<'_>, - ) -> Result<()> { - let listing_options = options.to_listing_options(&self.copied_config()); - - self.register_listing_table( - name, - table_path, - listing_options, - options.schema.map(|s| Arc::new(s.to_owned())), - None, - ) - .await?; - - Ok(()) - } - - /// Registers a JSON file as a table that it can be referenced - /// from SQL statements executed against this context. - pub async fn register_json( - &self, - name: &str, - table_path: &str, - options: NdJsonReadOptions<'_>, - ) -> Result<()> { - let listing_options = options.to_listing_options(&self.copied_config()); - - self.register_listing_table( - name, - table_path, - listing_options, - options.schema.map(|s| Arc::new(s.to_owned())), - None, - ) - .await?; - Ok(()) - } - - /// Registers a Parquet file as a table that can be referenced from SQL - /// statements executed against this context. - pub async fn register_parquet( - &self, - name: &str, - table_path: &str, - options: ParquetReadOptions<'_>, - ) -> Result<()> { - let listing_options = options.to_listing_options(&self.state.read().config); - - self.register_listing_table( - name, - table_path, - listing_options, - options.schema.map(|s| Arc::new(s.to_owned())), - None, - ) - .await?; - Ok(()) - } - - /// Registers an Avro file as a table that can be referenced from - /// SQL statements executed against this context. - pub async fn register_avro( - &self, - name: &str, - table_path: &str, - options: AvroReadOptions<'_>, - ) -> Result<()> { - let listing_options = options.to_listing_options(&self.copied_config()); - - self.register_listing_table( - name, - table_path, - listing_options, - options.schema.map(|s| Arc::new(s.to_owned())), - None, - ) - .await?; - Ok(()) - } - /// Registers an Arrow file as a table that can be referenced from /// SQL statements executed against this context. pub async fn register_arrow( @@ -1268,34 +1116,6 @@ impl SessionContext { self.state().create_physical_plan(logical_plan).await } - /// Executes a query and writes the results to a partitioned CSV file. - pub async fn write_csv( - &self, - plan: Arc, - path: impl AsRef, - ) -> Result<()> { - plan_to_csv(self.task_ctx(), plan, path).await - } - - /// Executes a query and writes the results to a partitioned JSON file. - pub async fn write_json( - &self, - plan: Arc, - path: impl AsRef, - ) -> Result<()> { - plan_to_json(self.task_ctx(), plan, path).await - } - - /// Executes a query and writes the results to a partitioned Parquet file. - pub async fn write_parquet( - &self, - plan: Arc, - path: impl AsRef, - writer_properties: Option, - ) -> Result<()> { - plan_to_parquet(self.task_ctx(), plan, path, writer_properties).await - } - /// Get a new TaskContext to run in this session pub fn task_ctx(&self) -> Arc { Arc::new(TaskContext::from(self)) @@ -1447,6 +1267,7 @@ impl SessionState { // Create table_factories for all default formats let mut table_factories: HashMap> = HashMap::new(); + #[cfg(feature = "parquet")] table_factories.insert("PARQUET".into(), Arc::new(ListingTableFactory::new())); table_factories.insert("CSV".into(), Arc::new(ListingTableFactory::new())); table_factories.insert("JSON".into(), Arc::new(ListingTableFactory::new())); @@ -2238,22 +2059,21 @@ impl<'a> TreeNodeVisitor for BadPlanVisitor<'a> { #[cfg(test)] mod tests { + use super::super::options::CsvReadOptions; use super::*; use crate::assert_batches_eq; use crate::execution::context::QueryPlanner; use crate::execution::memory_pool::MemoryConsumer; use crate::execution::runtime_env::RuntimeConfig; use crate::test; - use crate::test_util::parquet_test_data; + use crate::test_util::{plan_and_collect, populate_csv_partitions}; use crate::variable::VarType; - use arrow::record_batch::RecordBatch; - use arrow_schema::{Field, Schema}; + use arrow_schema::Schema; use async_trait::async_trait; use datafusion_expr::Expr; - use std::fs::File; + use std::env; use std::path::PathBuf; use std::sync::Weak; - use std::{env, io::prelude::*}; use tempfile::TempDir; #[tokio::test] @@ -2348,39 +2168,6 @@ mod tests { Ok(()) } - #[tokio::test] - async fn query_csv_with_custom_partition_extension() -> Result<()> { - let tmp_dir = TempDir::new()?; - - // The main stipulation of this test: use a file extension that isn't .csv. - let file_extension = ".tst"; - - let ctx = SessionContext::new(); - let schema = populate_csv_partitions(&tmp_dir, 2, file_extension)?; - ctx.register_csv( - "test", - tmp_dir.path().to_str().unwrap(), - CsvReadOptions::new() - .schema(&schema) - .file_extension(file_extension), - ) - .await?; - let results = - plan_and_collect(&ctx, "SELECT SUM(c1), SUM(c2), COUNT(*) FROM test").await?; - - assert_eq!(results.len(), 1); - let expected = [ - "+--------------+--------------+----------+", - "| SUM(test.c1) | SUM(test.c2) | COUNT(*) |", - "+--------------+--------------+----------+", - "| 10 | 110 | 20 |", - "+--------------+--------------+----------+", - ]; - assert_batches_eq!(expected, &results); - - Ok(()) - } - #[tokio::test] async fn send_context_to_threads() -> Result<()> { // ensure SessionContexts can be used in a multi-threaded @@ -2645,60 +2432,6 @@ mod tests { Ok(()) } - #[tokio::test] - async fn read_with_glob_path() -> Result<()> { - let ctx = SessionContext::new(); - - let df = ctx - .read_parquet( - format!("{}/alltypes_plain*.parquet", parquet_test_data()), - ParquetReadOptions::default(), - ) - .await?; - let results = df.collect().await?; - let total_rows: usize = results.iter().map(|rb| rb.num_rows()).sum(); - // alltypes_plain.parquet = 8 rows, alltypes_plain.snappy.parquet = 2 rows, alltypes_dictionary.parquet = 2 rows - assert_eq!(total_rows, 10); - Ok(()) - } - - #[tokio::test] - async fn read_with_glob_path_issue_2465() -> Result<()> { - let ctx = SessionContext::new(); - - let df = ctx - .read_parquet( - // it was reported that when a path contains // (two consecutive separator) no files were found - // in this test, regardless of parquet_test_data() value, our path now contains a // - format!("{}/..//*/alltypes_plain*.parquet", parquet_test_data()), - ParquetReadOptions::default(), - ) - .await?; - let results = df.collect().await?; - let total_rows: usize = results.iter().map(|rb| rb.num_rows()).sum(); - // alltypes_plain.parquet = 8 rows, alltypes_plain.snappy.parquet = 2 rows, alltypes_dictionary.parquet = 2 rows - assert_eq!(total_rows, 10); - Ok(()) - } - - #[tokio::test] - async fn read_from_registered_table_with_glob_path() -> Result<()> { - let ctx = SessionContext::new(); - - ctx.register_parquet( - "test", - &format!("{}/alltypes_plain*.parquet", parquet_test_data()), - ParquetReadOptions::default(), - ) - .await?; - let df = ctx.sql("SELECT * FROM test").await?; - let results = df.collect().await?; - let total_rows: usize = results.iter().map(|rb| rb.num_rows()).sum(); - // alltypes_plain.parquet = 8 rows, alltypes_plain.snappy.parquet = 2 rows, alltypes_dictionary.parquet = 2 rows - assert_eq!(total_rows, 10); - Ok(()) - } - struct MyPhysicalPlanner {} #[async_trait] @@ -2738,43 +2471,6 @@ mod tests { } } - /// Execute SQL and return results - async fn plan_and_collect( - ctx: &SessionContext, - sql: &str, - ) -> Result> { - ctx.sql(sql).await?.collect().await - } - - /// Generate CSV partitions within the supplied directory - fn populate_csv_partitions( - tmp_dir: &TempDir, - partition_count: usize, - file_extension: &str, - ) -> Result { - // define schema for data source (csv file) - let schema = Arc::new(Schema::new(vec![ - Field::new("c1", DataType::UInt32, false), - Field::new("c2", DataType::UInt64, false), - Field::new("c3", DataType::Boolean, false), - ])); - - // generate a partitioned file - for partition in 0..partition_count { - let filename = format!("partition-{partition}.{file_extension}"); - let file_path = tmp_dir.path().join(filename); - let mut file = File::create(file_path)?; - - // generate some data - for i in 0..=10 { - let data = format!("{},{},{}\n", partition, i, i % 2 == 0); - file.write_all(data.as_bytes())?; - } - } - - Ok(schema) - } - /// Generate a partitioned CSV file and register it with an execution context async fn create_ctx( tmp_dir: &TempDir, @@ -2796,37 +2492,4 @@ mod tests { Ok(ctx) } - - // Test for compilation error when calling read_* functions from an #[async_trait] function. - // See https://github.com/apache/arrow-datafusion/issues/1154 - #[async_trait] - trait CallReadTrait { - async fn call_read_csv(&self) -> DataFrame; - async fn call_read_avro(&self) -> DataFrame; - async fn call_read_parquet(&self) -> DataFrame; - } - - struct CallRead {} - - #[async_trait] - impl CallReadTrait for CallRead { - async fn call_read_csv(&self) -> DataFrame { - let ctx = SessionContext::new(); - ctx.read_csv("dummy", CsvReadOptions::new()).await.unwrap() - } - - async fn call_read_avro(&self) -> DataFrame { - let ctx = SessionContext::new(); - ctx.read_avro("dummy", AvroReadOptions::default()) - .await - .unwrap() - } - - async fn call_read_parquet(&self) -> DataFrame { - let ctx = SessionContext::new(); - ctx.read_parquet("dummy", ParquetReadOptions::default()) - .await - .unwrap() - } - } } diff --git a/datafusion/core/src/execution/context/parquet.rs b/datafusion/core/src/execution/context/parquet.rs new file mode 100644 index 000000000000..b02576c6a868 --- /dev/null +++ b/datafusion/core/src/execution/context/parquet.rs @@ -0,0 +1,154 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +use std::sync::Arc; + +use parquet::file::properties::WriterProperties; + +use crate::datasource::physical_plan::plan_to_parquet; + +use super::super::options::{ParquetReadOptions, ReadOptions}; +use super::{DataFilePaths, DataFrame, ExecutionPlan, Result, SessionContext}; + +impl SessionContext { + /// Creates a [`DataFrame`] for reading a Parquet data source. + /// + /// For more control such as reading multiple files, you can use + /// [`read_table`](Self::read_table) with a [`super::ListingTable`]. + /// + /// For an example, see [`read_csv`](Self::read_csv) + pub async fn read_parquet( + &self, + table_paths: P, + options: ParquetReadOptions<'_>, + ) -> Result { + self._read_type(table_paths, options).await + } + + /// Registers a Parquet file as a table that can be referenced from SQL + /// statements executed against this context. + pub async fn register_parquet( + &self, + name: &str, + table_path: &str, + options: ParquetReadOptions<'_>, + ) -> Result<()> { + let listing_options = options.to_listing_options(&self.state.read().config); + + self.register_listing_table( + name, + table_path, + listing_options, + options.schema.map(|s| Arc::new(s.to_owned())), + None, + ) + .await?; + Ok(()) + } + + /// Executes a query and writes the results to a partitioned Parquet file. + pub async fn write_parquet( + &self, + plan: Arc, + path: impl AsRef, + writer_properties: Option, + ) -> Result<()> { + plan_to_parquet(self.task_ctx(), plan, path, writer_properties).await + } +} + +#[cfg(test)] +mod tests { + use async_trait::async_trait; + + use crate::test_util::parquet_test_data; + + use super::*; + + #[tokio::test] + async fn read_with_glob_path() -> Result<()> { + let ctx = SessionContext::new(); + + let df = ctx + .read_parquet( + format!("{}/alltypes_plain*.parquet", parquet_test_data()), + ParquetReadOptions::default(), + ) + .await?; + let results = df.collect().await?; + let total_rows: usize = results.iter().map(|rb| rb.num_rows()).sum(); + // alltypes_plain.parquet = 8 rows, alltypes_plain.snappy.parquet = 2 rows, alltypes_dictionary.parquet = 2 rows + assert_eq!(total_rows, 10); + Ok(()) + } + + #[tokio::test] + async fn read_with_glob_path_issue_2465() -> Result<()> { + let ctx = SessionContext::new(); + + let df = ctx + .read_parquet( + // it was reported that when a path contains // (two consecutive separator) no files were found + // in this test, regardless of parquet_test_data() value, our path now contains a // + format!("{}/..//*/alltypes_plain*.parquet", parquet_test_data()), + ParquetReadOptions::default(), + ) + .await?; + let results = df.collect().await?; + let total_rows: usize = results.iter().map(|rb| rb.num_rows()).sum(); + // alltypes_plain.parquet = 8 rows, alltypes_plain.snappy.parquet = 2 rows, alltypes_dictionary.parquet = 2 rows + assert_eq!(total_rows, 10); + Ok(()) + } + + #[tokio::test] + async fn read_from_registered_table_with_glob_path() -> Result<()> { + let ctx = SessionContext::new(); + + ctx.register_parquet( + "test", + &format!("{}/alltypes_plain*.parquet", parquet_test_data()), + ParquetReadOptions::default(), + ) + .await?; + let df = ctx.sql("SELECT * FROM test").await?; + let results = df.collect().await?; + let total_rows: usize = results.iter().map(|rb| rb.num_rows()).sum(); + // alltypes_plain.parquet = 8 rows, alltypes_plain.snappy.parquet = 2 rows, alltypes_dictionary.parquet = 2 rows + assert_eq!(total_rows, 10); + Ok(()) + } + + // Test for compilation error when calling read_* functions from an #[async_trait] function. + // See https://github.com/apache/arrow-datafusion/issues/1154 + #[async_trait] + trait CallReadTrait { + async fn call_read_parquet(&self) -> DataFrame; + } + + struct CallRead {} + + #[async_trait] + impl CallReadTrait for CallRead { + async fn call_read_parquet(&self) -> DataFrame { + let ctx = SessionContext::new(); + ctx.read_parquet("dummy", ParquetReadOptions::default()) + .await + .unwrap() + } + } +} diff --git a/datafusion/core/src/lib.rs b/datafusion/core/src/lib.rs index 5e9f130eade5..bf9a4abf4f2d 100644 --- a/datafusion/core/src/lib.rs +++ b/datafusion/core/src/lib.rs @@ -437,6 +437,7 @@ pub mod variable; // re-export dependencies from arrow-rs to minimize version maintenance for crate users pub use arrow; +#[cfg(feature = "parquet")] pub use parquet; // re-export DataFusion sub-crates at the top level. Use `pub use *` diff --git a/datafusion/core/src/physical_optimizer/enforce_distribution.rs b/datafusion/core/src/physical_optimizer/enforce_distribution.rs index 9cd7eff4722b..d3fbc46a6659 100644 --- a/datafusion/core/src/physical_optimizer/enforce_distribution.rs +++ b/datafusion/core/src/physical_optimizer/enforce_distribution.rs @@ -26,7 +26,9 @@ use std::fmt::Formatter; use std::sync::Arc; use crate::config::ConfigOptions; -use crate::datasource::physical_plan::{CsvExec, ParquetExec}; +use crate::datasource::physical_plan::CsvExec; +#[cfg(feature = "parquet")] +use crate::datasource::physical_plan::ParquetExec; use crate::error::Result; use crate::physical_optimizer::utils::{ add_sort_above, get_children_exectrees, get_plan_string, is_coalesce_partitions, @@ -1306,6 +1308,7 @@ fn ensure_distribution( // When `repartition_file_scans` is set, leverage source operators // (`ParquetExec`, `CsvExec` etc.) to increase parallelism at the source. if repartition_file_scans { + #[cfg(feature = "parquet")] if let Some(parquet_exec) = child.as_any().downcast_ref::() { @@ -1313,9 +1316,8 @@ fn ensure_distribution( target_partitions, repartition_file_min_size, )); - } else if let Some(csv_exec) = - child.as_any().downcast_ref::() - { + } + if let Some(csv_exec) = child.as_any().downcast_ref::() { if let Some(csv_exec) = csv_exec.get_repartitioned( target_partitions, repartition_file_min_size, @@ -1680,7 +1682,9 @@ mod tests { use crate::datasource::file_format::file_compression_type::FileCompressionType; use crate::datasource::listing::PartitionedFile; use crate::datasource::object_store::ObjectStoreUrl; - use crate::datasource::physical_plan::{FileScanConfig, ParquetExec}; + use crate::datasource::physical_plan::FileScanConfig; + #[cfg(feature = "parquet")] + use crate::datasource::physical_plan::ParquetExec; use crate::physical_optimizer::enforce_sorting::EnforceSorting; use crate::physical_optimizer::output_requirements::OutputRequirements; use crate::physical_plan::aggregates::{ @@ -1819,10 +1823,12 @@ mod tests { ])) } + #[cfg(feature = "parquet")] fn parquet_exec() -> Arc { parquet_exec_with_sort(vec![]) } + #[cfg(feature = "parquet")] fn parquet_exec_with_sort( output_ordering: Vec>, ) -> Arc { @@ -1843,11 +1849,13 @@ mod tests { )) } + #[cfg(feature = "parquet")] fn parquet_exec_multiple() -> Arc { parquet_exec_multiple_sorted(vec![]) } // Created a sorted parquet exec with multiple files + #[cfg(feature = "parquet")] fn parquet_exec_multiple_sorted( output_ordering: Vec>, ) -> Arc { @@ -2202,6 +2210,7 @@ mod tests { } #[test] + #[cfg(feature = "parquet")] fn multi_hash_joins() -> Result<()> { let left = parquet_exec(); let alias_pairs: Vec<(String, String)> = vec![ @@ -2364,6 +2373,7 @@ mod tests { } #[test] + #[cfg(feature = "parquet")] fn multi_joins_after_alias() -> Result<()> { let left = parquet_exec(); let right = parquet_exec(); @@ -2443,6 +2453,7 @@ mod tests { } #[test] + #[cfg(feature = "parquet")] fn multi_joins_after_multi_alias() -> Result<()> { let left = parquet_exec(); let right = parquet_exec(); @@ -2498,6 +2509,7 @@ mod tests { } #[test] + #[cfg(feature = "parquet")] fn join_after_agg_alias() -> Result<()> { // group by (a as a1) let left = aggregate_exec_with_alias( @@ -2537,6 +2549,7 @@ mod tests { } #[test] + #[cfg(feature = "parquet")] fn hash_join_key_ordering() -> Result<()> { // group by (a as a1, b as b1) let left = aggregate_exec_with_alias( @@ -2589,6 +2602,7 @@ mod tests { } #[test] + #[cfg(feature = "parquet")] fn multi_hash_join_key_ordering() -> Result<()> { let left = parquet_exec(); let alias_pairs: Vec<(String, String)> = vec![ @@ -2705,6 +2719,7 @@ mod tests { } #[test] + #[cfg(feature = "parquet")] fn reorder_join_keys_to_left_input() -> Result<()> { let left = parquet_exec(); let alias_pairs: Vec<(String, String)> = vec![ @@ -2835,6 +2850,7 @@ mod tests { } #[test] + #[cfg(feature = "parquet")] fn reorder_join_keys_to_right_input() -> Result<()> { let left = parquet_exec(); let alias_pairs: Vec<(String, String)> = vec![ @@ -2960,6 +2976,7 @@ mod tests { } #[test] + #[cfg(feature = "parquet")] fn multi_smj_joins() -> Result<()> { let left = parquet_exec(); let alias_pairs: Vec<(String, String)> = vec![ @@ -3233,6 +3250,7 @@ mod tests { } #[test] + #[cfg(feature = "parquet")] fn smj_join_key_ordering() -> Result<()> { // group by (a as a1, b as b1) let left = aggregate_exec_with_alias( @@ -3328,6 +3346,7 @@ mod tests { } #[test] + #[cfg(feature = "parquet")] fn merge_does_not_need_sort() -> Result<()> { // see https://github.com/apache/arrow-datafusion/issues/4331 let schema = schema(); @@ -3368,6 +3387,7 @@ mod tests { } #[test] + #[cfg(feature = "parquet")] fn union_to_interleave() -> Result<()> { // group by (a as a1) let left = aggregate_exec_with_alias( @@ -3409,6 +3429,7 @@ mod tests { } #[test] + #[cfg(feature = "parquet")] fn added_repartition_to_single_partition() -> Result<()> { let alias = vec![("a".to_string(), "a".to_string())]; let plan = aggregate_exec_with_alias(parquet_exec(), alias); @@ -3427,6 +3448,7 @@ mod tests { } #[test] + #[cfg(feature = "parquet")] fn repartition_deepest_node() -> Result<()> { let alias = vec![("a".to_string(), "a".to_string())]; let plan = aggregate_exec_with_alias(filter_exec(parquet_exec()), alias); @@ -3446,6 +3468,7 @@ mod tests { } #[test] + #[cfg(feature = "parquet")] fn repartition_unsorted_limit() -> Result<()> { let plan = limit_exec(filter_exec(parquet_exec())); @@ -3465,6 +3488,7 @@ mod tests { } #[test] + #[cfg(feature = "parquet")] fn repartition_sorted_limit() -> Result<()> { let schema = schema(); let sort_key = vec![PhysicalSortExpr { @@ -3487,6 +3511,7 @@ mod tests { } #[test] + #[cfg(feature = "parquet")] fn repartition_sorted_limit_with_filter() -> Result<()> { let schema = schema(); let sort_key = vec![PhysicalSortExpr { @@ -3512,6 +3537,7 @@ mod tests { } #[test] + #[cfg(feature = "parquet")] fn repartition_ignores_limit() -> Result<()> { let alias = vec![("a".to_string(), "a".to_string())]; let plan = aggregate_exec_with_alias( @@ -3542,6 +3568,7 @@ mod tests { } #[test] + #[cfg(feature = "parquet")] fn repartition_ignores_union() -> Result<()> { let plan = union_exec(vec![parquet_exec(); 5]); @@ -3561,6 +3588,7 @@ mod tests { } #[test] + #[cfg(feature = "parquet")] fn repartition_through_sort_preserving_merge() -> Result<()> { // sort preserving merge with non-sorted input let schema = schema(); @@ -3583,6 +3611,7 @@ mod tests { } #[test] + #[cfg(feature = "parquet")] fn repartition_ignores_sort_preserving_merge() -> Result<()> { // sort preserving merge already sorted input, let schema = schema(); @@ -3614,6 +3643,7 @@ mod tests { } #[test] + #[cfg(feature = "parquet")] fn repartition_ignores_sort_preserving_merge_with_union() -> Result<()> { // 2 sorted parquet files unioned (partitions are concatenated, sort is preserved) let schema = schema(); @@ -3646,6 +3676,7 @@ mod tests { } #[test] + #[cfg(feature = "parquet")] fn repartition_does_not_destroy_sort() -> Result<()> { // SortRequired // Parquet(sorted) @@ -3671,6 +3702,7 @@ mod tests { } #[test] + #[cfg(feature = "parquet")] fn repartition_does_not_destroy_sort_more_complex() -> Result<()> { // model a more complicated scenario where one child of a union can be repartitioned for performance // but the other can not be @@ -3709,6 +3741,7 @@ mod tests { } #[test] + #[cfg(feature = "parquet")] fn repartition_transitively_with_projection() -> Result<()> { let schema = schema(); let proj_exprs = vec![( @@ -3751,6 +3784,7 @@ mod tests { } #[test] + #[cfg(feature = "parquet")] fn repartition_ignores_transitively_with_projection() -> Result<()> { let schema = schema(); let sort_key = vec![PhysicalSortExpr { @@ -3781,6 +3815,7 @@ mod tests { } #[test] + #[cfg(feature = "parquet")] fn repartition_transitively_past_sort_with_projection() -> Result<()> { let schema = schema(); let sort_key = vec![PhysicalSortExpr { @@ -3810,6 +3845,7 @@ mod tests { } #[test] + #[cfg(feature = "parquet")] fn repartition_transitively_past_sort_with_filter() -> Result<()> { let schema = schema(); let sort_key = vec![PhysicalSortExpr { @@ -3842,6 +3878,7 @@ mod tests { } #[test] + #[cfg(feature = "parquet")] fn repartition_transitively_past_sort_with_projection_and_filter() -> Result<()> { let schema = schema(); let sort_key = vec![PhysicalSortExpr { @@ -3883,6 +3920,7 @@ mod tests { } #[test] + #[cfg(feature = "parquet")] fn parallelization_single_partition() -> Result<()> { let alias = vec![("a".to_string(), "a".to_string())]; let plan_parquet = aggregate_exec_with_alias(parquet_exec(), alias.clone()); @@ -3971,6 +4009,7 @@ mod tests { } #[test] + #[cfg(feature = "parquet")] fn parallelization_two_partitions() -> Result<()> { let alias = vec![("a".to_string(), "a".to_string())]; let plan_parquet = @@ -3998,6 +4037,7 @@ mod tests { } #[test] + #[cfg(feature = "parquet")] fn parallelization_two_partitions_into_four() -> Result<()> { let alias = vec![("a".to_string(), "a".to_string())]; let plan_parquet = @@ -4025,6 +4065,7 @@ mod tests { } #[test] + #[cfg(feature = "parquet")] fn parallelization_sorted_limit() -> Result<()> { let schema = schema(); let sort_key = vec![PhysicalSortExpr { @@ -4057,6 +4098,7 @@ mod tests { } #[test] + #[cfg(feature = "parquet")] fn parallelization_limit_with_filter() -> Result<()> { let schema = schema(); let sort_key = vec![PhysicalSortExpr { @@ -4102,6 +4144,7 @@ mod tests { } #[test] + #[cfg(feature = "parquet")] fn parallelization_ignores_limit() -> Result<()> { let alias = vec![("a".to_string(), "a".to_string())]; let plan_parquet = aggregate_exec_with_alias( @@ -4152,6 +4195,7 @@ mod tests { } #[test] + #[cfg(feature = "parquet")] fn parallelization_union_inputs() -> Result<()> { let plan_parquet = union_exec(vec![parquet_exec(); 5]); let plan_csv = union_exec(vec![csv_exec(); 5]); @@ -4181,6 +4225,7 @@ mod tests { } #[test] + #[cfg(feature = "parquet")] fn parallelization_prior_to_sort_preserving_merge() -> Result<()> { let schema = schema(); let sort_key = vec![PhysicalSortExpr { @@ -4211,6 +4256,7 @@ mod tests { } #[test] + #[cfg(feature = "parquet")] fn parallelization_sort_preserving_merge_with_union() -> Result<()> { let schema = schema(); let sort_key = vec![PhysicalSortExpr { @@ -4245,6 +4291,7 @@ mod tests { } #[test] + #[cfg(feature = "parquet")] fn parallelization_does_not_benefit() -> Result<()> { let schema = schema(); let sort_key = vec![PhysicalSortExpr { @@ -4273,6 +4320,7 @@ mod tests { } #[test] + #[cfg(feature = "parquet")] fn parallelization_ignores_transitively_with_projection_parquet() -> Result<()> { // sorted input let schema = schema(); @@ -4353,6 +4401,7 @@ mod tests { } #[test] + #[cfg(feature = "parquet")] fn remove_redundant_roundrobins() -> Result<()> { let input = parquet_exec(); let repartition = repartition_exec(repartition_exec(input)); @@ -4403,6 +4452,7 @@ mod tests { } #[test] + #[cfg(feature = "parquet")] fn do_not_preserve_ordering_through_repartition() -> Result<()> { let schema = schema(); let sort_key = vec![PhysicalSortExpr { @@ -4435,6 +4485,7 @@ mod tests { } #[test] + #[cfg(feature = "parquet")] fn do_not_preserve_ordering_through_repartition2() -> Result<()> { let schema = schema(); let sort_key = vec![PhysicalSortExpr { @@ -4473,6 +4524,7 @@ mod tests { } #[test] + #[cfg(feature = "parquet")] fn do_not_preserve_ordering_through_repartition3() -> Result<()> { let schema = schema(); let sort_key = vec![PhysicalSortExpr { @@ -4495,6 +4547,7 @@ mod tests { } #[test] + #[cfg(feature = "parquet")] fn do_not_put_sort_when_input_is_invalid() -> Result<()> { let schema = schema(); let sort_key = vec![PhysicalSortExpr { @@ -4533,6 +4586,7 @@ mod tests { } #[test] + #[cfg(feature = "parquet")] fn put_sort_when_input_is_valid() -> Result<()> { let schema = schema(); let sort_key = vec![PhysicalSortExpr { @@ -4575,6 +4629,7 @@ mod tests { } #[test] + #[cfg(feature = "parquet")] fn do_not_add_unnecessary_hash() -> Result<()> { let schema = schema(); let sort_key = vec![PhysicalSortExpr { @@ -4630,6 +4685,7 @@ mod tests { } #[test] + #[cfg(feature = "parquet")] fn optimize_away_unnecessary_repartition() -> Result<()> { let physical_plan = coalesce_partitions_exec(repartition_exec(parquet_exec())); let expected = &[ @@ -4649,6 +4705,7 @@ mod tests { } #[test] + #[cfg(feature = "parquet")] fn optimize_away_unnecessary_repartition2() -> Result<()> { let physical_plan = filter_exec(repartition_exec(coalesce_partitions_exec( filter_exec(repartition_exec(parquet_exec())), diff --git a/datafusion/core/src/physical_planner.rs b/datafusion/core/src/physical_planner.rs index 419f62cff664..f941e88f3a36 100644 --- a/datafusion/core/src/physical_planner.rs +++ b/datafusion/core/src/physical_planner.rs @@ -25,6 +25,7 @@ use crate::datasource::file_format::arrow::ArrowFormat; use crate::datasource::file_format::avro::AvroFormat; use crate::datasource::file_format::csv::CsvFormat; use crate::datasource::file_format::json::JsonFormat; +#[cfg(feature = "parquet")] use crate::datasource::file_format::parquet::ParquetFormat; use crate::datasource::file_format::write::FileWriterMode; use crate::datasource::file_format::FileFormat; @@ -599,6 +600,7 @@ impl DefaultPhysicalPlanner { let sink_format: Arc = match file_format { FileType::CSV => Arc::new(CsvFormat::default()), + #[cfg(feature = "parquet")] FileType::PARQUET => Arc::new(ParquetFormat::default()), FileType::JSON => Arc::new(JsonFormat::default()), FileType::AVRO => Arc::new(AvroFormat {} ), diff --git a/datafusion/core/src/test_util/mod.rs b/datafusion/core/src/test_util/mod.rs index 4fe022f1769d..c6b43de0c18d 100644 --- a/datafusion/core/src/test_util/mod.rs +++ b/datafusion/core/src/test_util/mod.rs @@ -17,15 +17,21 @@ //! Utility functions to make testing DataFusion based crates easier +#[cfg(feature = "parquet")] pub mod parquet; use std::any::Any; use std::collections::HashMap; +use std::fs::File; +use std::io::Write; use std::path::Path; use std::pin::Pin; use std::sync::Arc; use std::task::{Context, Poll}; +use tempfile::TempDir; + +use crate::dataframe::DataFrame; use crate::datasource::provider::TableProviderFactory; use crate::datasource::{empty::EmptyTable, provider_as_source, TableProvider}; use crate::error::Result; @@ -48,9 +54,9 @@ use async_trait::async_trait; use futures::Stream; // backwards compatibility -pub use datafusion_common::test_util::{ - arrow_test_data, get_data_dir, parquet_test_data, -}; +#[cfg(feature = "parquet")] +pub use datafusion_common::test_util::parquet_test_data; +pub use datafusion_common::test_util::{arrow_test_data, get_data_dir}; pub use datafusion_common::{assert_batches_eq, assert_batches_sorted_eq}; @@ -102,6 +108,71 @@ pub fn aggr_test_schema() -> SchemaRef { Arc::new(schema) } +/// Register session context for the aggregate_test_100.csv file +pub async fn register_aggregate_csv( + ctx: &mut SessionContext, + table_name: &str, +) -> Result<()> { + let schema = aggr_test_schema(); + let testdata = arrow_test_data(); + ctx.register_csv( + table_name, + &format!("{testdata}/csv/aggregate_test_100.csv"), + CsvReadOptions::new().schema(schema.as_ref()), + ) + .await?; + Ok(()) +} + +/// Create a table from the aggregate_test_100.csv file with the specified name +pub async fn test_table_with_name(name: &str) -> Result { + let mut ctx = SessionContext::new(); + register_aggregate_csv(&mut ctx, name).await?; + ctx.table(name).await +} + +/// Create a table from the aggregate_test_100.csv file with the name "aggregate_test_100" +pub async fn test_table() -> Result { + test_table_with_name("aggregate_test_100").await +} + +/// Execute SQL and return results +pub async fn plan_and_collect( + ctx: &SessionContext, + sql: &str, +) -> Result> { + ctx.sql(sql).await?.collect().await +} + +/// Generate CSV partitions within the supplied directory +pub fn populate_csv_partitions( + tmp_dir: &TempDir, + partition_count: usize, + file_extension: &str, +) -> Result { + // define schema for data source (csv file) + let schema = Arc::new(Schema::new(vec![ + Field::new("c1", DataType::UInt32, false), + Field::new("c2", DataType::UInt64, false), + Field::new("c3", DataType::Boolean, false), + ])); + + // generate a partitioned file + for partition in 0..partition_count { + let filename = format!("partition-{partition}.{file_extension}"); + let file_path = tmp_dir.path().join(filename); + let mut file = File::create(file_path)?; + + // generate some data + for i in 0..=10 { + let data = format!("{},{},{}\n", partition, i, i % 2 == 0); + file.write_all(data.as_bytes())?; + } + } + + Ok(schema) +} + /// TableFactory for tests pub struct TestTableFactory {} diff --git a/datafusion/proto/Cargo.toml b/datafusion/proto/Cargo.toml index 94e77088a7e8..32e10e58a7d7 100644 --- a/datafusion/proto/Cargo.toml +++ b/datafusion/proto/Cargo.toml @@ -36,8 +36,9 @@ name = "datafusion_proto" path = "src/lib.rs" [features] -default = [] +default = ["parquet"] json = ["pbjson", "serde", "serde_json"] +parquet = ["datafusion/parquet", "datafusion-common/parquet"] [dependencies] arrow = { workspace = true } diff --git a/datafusion/proto/src/logical_plan/mod.rs b/datafusion/proto/src/logical_plan/mod.rs index df76fbb81396..e426c598523e 100644 --- a/datafusion/proto/src/logical_plan/mod.rs +++ b/datafusion/proto/src/logical_plan/mod.rs @@ -31,11 +31,11 @@ use crate::{ }; use arrow::datatypes::{DataType, Schema, SchemaRef}; +#[cfg(feature = "parquet")] +use datafusion::datasource::file_format::parquet::ParquetFormat; use datafusion::{ datasource::{ - file_format::{ - avro::AvroFormat, csv::CsvFormat, parquet::ParquetFormat, FileFormat, - }, + file_format::{avro::AvroFormat, csv::CsvFormat, FileFormat}, listing::{ListingOptions, ListingTable, ListingTableConfig, ListingTableUrl}, view::ViewTable, TableProvider, @@ -336,6 +336,7 @@ impl AsLogicalPlan for LogicalPlanNode { "logical_plan::from_proto() Unsupported file format '{self:?}'" )) })? { + #[cfg(feature = "parquet")] &FileFormatType::Parquet(protobuf::ParquetFormat {}) => { Arc::new(ParquetFormat::default()) } @@ -849,28 +850,49 @@ impl AsLogicalPlan for LogicalPlanNode { if let Some(listing_table) = source.downcast_ref::() { let any = listing_table.options().format.as_any(); - let file_format_type = if any.is::() { - FileFormatType::Parquet(protobuf::ParquetFormat {}) - } else if let Some(csv) = any.downcast_ref::() { - FileFormatType::Csv(protobuf::CsvFormat { - delimiter: byte_to_string(csv.delimiter(), "delimiter")?, - has_header: csv.has_header(), - quote: byte_to_string(csv.quote(), "quote")?, - optional_escape: if let Some(escape) = csv.escape() { - Some(protobuf::csv_format::OptionalEscape::Escape( - byte_to_string(escape, "escape")?, - )) - } else { - None - }, - }) - } else if any.is::() { - FileFormatType::Avro(protobuf::AvroFormat {}) - } else { - return Err(proto_error(format!( + let file_format_type = { + let mut maybe_some_type = None; + + #[cfg(feature = "parquet")] + if any.is::() { + maybe_some_type = + Some(FileFormatType::Parquet(protobuf::ParquetFormat {})) + }; + + if let Some(csv) = any.downcast_ref::() { + maybe_some_type = + Some(FileFormatType::Csv(protobuf::CsvFormat { + delimiter: byte_to_string( + csv.delimiter(), + "delimiter", + )?, + has_header: csv.has_header(), + quote: byte_to_string(csv.quote(), "quote")?, + optional_escape: if let Some(escape) = csv.escape() { + Some( + protobuf::csv_format::OptionalEscape::Escape( + byte_to_string(escape, "escape")?, + ), + ) + } else { + None + }, + })) + } + + if any.is::() { + maybe_some_type = + Some(FileFormatType::Avro(protobuf::AvroFormat {})) + } + + if let Some(file_format_type) = maybe_some_type { + file_format_type + } else { + return Err(proto_error(format!( "Error converting file format, {:?} is invalid as a datafusion format.", listing_table.options().format ))); + } }; let options = listing_table.options(); diff --git a/datafusion/proto/src/physical_plan/mod.rs b/datafusion/proto/src/physical_plan/mod.rs index ef870d8ac20b..431b8e42cdaf 100644 --- a/datafusion/proto/src/physical_plan/mod.rs +++ b/datafusion/proto/src/physical_plan/mod.rs @@ -22,7 +22,9 @@ use std::sync::Arc; use datafusion::arrow::compute::SortOptions; use datafusion::arrow::datatypes::SchemaRef; use datafusion::datasource::file_format::file_compression_type::FileCompressionType; -use datafusion::datasource::physical_plan::{AvroExec, CsvExec, ParquetExec}; +#[cfg(feature = "parquet")] +use datafusion::datasource::physical_plan::ParquetExec; +use datafusion::datasource::physical_plan::{AvroExec, CsvExec}; use datafusion::execution::runtime_env::RuntimeEnv; use datafusion::execution::FunctionRegistry; use datafusion::physical_plan::aggregates::{create_aggregate_expr, AggregateMode}; @@ -171,6 +173,7 @@ impl AsExecutionPlan for PhysicalPlanNode { }, FileCompressionType::UNCOMPRESSED, ))), + #[cfg(feature = "parquet")] PhysicalPlanType::ParquetScan(scan) => { let base_config = parse_protobuf_file_scan_config( scan.base_conf.as_ref().unwrap(), @@ -796,7 +799,7 @@ impl AsExecutionPlan for PhysicalPlanNode { let plan = plan.as_any(); if let Some(exec) = plan.downcast_ref::() { - Ok(protobuf::PhysicalPlanNode { + return Ok(protobuf::PhysicalPlanNode { physical_plan_type: Some(PhysicalPlanType::Explain( protobuf::ExplainExecNode { schema: Some(exec.schema().as_ref().try_into()?), @@ -808,8 +811,10 @@ impl AsExecutionPlan for PhysicalPlanNode { verbose: exec.verbose(), }, )), - }) - } else if let Some(exec) = plan.downcast_ref::() { + }); + } + + if let Some(exec) = plan.downcast_ref::() { let input = protobuf::PhysicalPlanNode::try_from_physical_plan( exec.input().to_owned(), extension_codec, @@ -820,7 +825,7 @@ impl AsExecutionPlan for PhysicalPlanNode { .map(|expr| expr.0.clone().try_into()) .collect::>>()?; let expr_name = exec.expr().iter().map(|expr| expr.1.clone()).collect(); - Ok(protobuf::PhysicalPlanNode { + return Ok(protobuf::PhysicalPlanNode { physical_plan_type: Some(PhysicalPlanType::Projection(Box::new( protobuf::ProjectionExecNode { input: Some(Box::new(input)), @@ -828,13 +833,15 @@ impl AsExecutionPlan for PhysicalPlanNode { expr_name, }, ))), - }) - } else if let Some(exec) = plan.downcast_ref::() { + }); + } + + if let Some(exec) = plan.downcast_ref::() { let input = protobuf::PhysicalPlanNode::try_from_physical_plan( exec.input().to_owned(), extension_codec, )?; - Ok(protobuf::PhysicalPlanNode { + return Ok(protobuf::PhysicalPlanNode { physical_plan_type: Some(PhysicalPlanType::Analyze(Box::new( protobuf::AnalyzeExecNode { verbose: exec.verbose(), @@ -843,27 +850,31 @@ impl AsExecutionPlan for PhysicalPlanNode { schema: Some(exec.schema().as_ref().try_into()?), }, ))), - }) - } else if let Some(exec) = plan.downcast_ref::() { + }); + } + + if let Some(exec) = plan.downcast_ref::() { let input = protobuf::PhysicalPlanNode::try_from_physical_plan( exec.input().to_owned(), extension_codec, )?; - Ok(protobuf::PhysicalPlanNode { + return Ok(protobuf::PhysicalPlanNode { physical_plan_type: Some(PhysicalPlanType::Filter(Box::new( protobuf::FilterExecNode { input: Some(Box::new(input)), expr: Some(exec.predicate().clone().try_into()?), }, ))), - }) - } else if let Some(limit) = plan.downcast_ref::() { + }); + } + + if let Some(limit) = plan.downcast_ref::() { let input = protobuf::PhysicalPlanNode::try_from_physical_plan( limit.input().to_owned(), extension_codec, )?; - Ok(protobuf::PhysicalPlanNode { + return Ok(protobuf::PhysicalPlanNode { physical_plan_type: Some(PhysicalPlanType::GlobalLimit(Box::new( protobuf::GlobalLimitExecNode { input: Some(Box::new(input)), @@ -874,21 +885,25 @@ impl AsExecutionPlan for PhysicalPlanNode { }, }, ))), - }) - } else if let Some(limit) = plan.downcast_ref::() { + }); + } + + if let Some(limit) = plan.downcast_ref::() { let input = protobuf::PhysicalPlanNode::try_from_physical_plan( limit.input().to_owned(), extension_codec, )?; - Ok(protobuf::PhysicalPlanNode { + return Ok(protobuf::PhysicalPlanNode { physical_plan_type: Some(PhysicalPlanType::LocalLimit(Box::new( protobuf::LocalLimitExecNode { input: Some(Box::new(input)), fetch: limit.fetch() as u32, }, ))), - }) - } else if let Some(exec) = plan.downcast_ref::() { + }); + } + + if let Some(exec) = plan.downcast_ref::() { let left = protobuf::PhysicalPlanNode::try_from_physical_plan( exec.left().to_owned(), extension_codec, @@ -943,7 +958,7 @@ impl AsExecutionPlan for PhysicalPlanNode { PartitionMode::Auto => protobuf::PartitionMode::Auto, }; - Ok(protobuf::PhysicalPlanNode { + return Ok(protobuf::PhysicalPlanNode { physical_plan_type: Some(PhysicalPlanType::HashJoin(Box::new( protobuf::HashJoinExecNode { left: Some(Box::new(left)), @@ -955,8 +970,10 @@ impl AsExecutionPlan for PhysicalPlanNode { filter, }, ))), - }) - } else if let Some(exec) = plan.downcast_ref::() { + }); + } + + if let Some(exec) = plan.downcast_ref::() { let left = protobuf::PhysicalPlanNode::try_from_physical_plan( exec.left().to_owned(), extension_codec, @@ -965,15 +982,16 @@ impl AsExecutionPlan for PhysicalPlanNode { exec.right().to_owned(), extension_codec, )?; - Ok(protobuf::PhysicalPlanNode { + return Ok(protobuf::PhysicalPlanNode { physical_plan_type: Some(PhysicalPlanType::CrossJoin(Box::new( protobuf::CrossJoinExecNode { left: Some(Box::new(left)), right: Some(Box::new(right)), }, ))), - }) - } else if let Some(exec) = plan.downcast_ref::() { + }); + } + if let Some(exec) = plan.downcast_ref::() { let groups: Vec = exec .group_expr() .groups() @@ -1046,7 +1064,7 @@ impl AsExecutionPlan for PhysicalPlanNode { .map(|expr| expr.0.to_owned().try_into()) .collect::>>()?; - Ok(protobuf::PhysicalPlanNode { + return Ok(protobuf::PhysicalPlanNode { physical_plan_type: Some(PhysicalPlanType::Aggregate(Box::new( protobuf::AggregateExecNode { group_expr, @@ -1062,33 +1080,38 @@ impl AsExecutionPlan for PhysicalPlanNode { groups, }, ))), - }) - } else if let Some(empty) = plan.downcast_ref::() { + }); + } + + if let Some(empty) = plan.downcast_ref::() { let schema = empty.schema().as_ref().try_into()?; - Ok(protobuf::PhysicalPlanNode { + return Ok(protobuf::PhysicalPlanNode { physical_plan_type: Some(PhysicalPlanType::Empty( protobuf::EmptyExecNode { produce_one_row: empty.produce_one_row(), schema: Some(schema), }, )), - }) - } else if let Some(coalesce_batches) = plan.downcast_ref::() - { + }); + } + + if let Some(coalesce_batches) = plan.downcast_ref::() { let input = protobuf::PhysicalPlanNode::try_from_physical_plan( coalesce_batches.input().to_owned(), extension_codec, )?; - Ok(protobuf::PhysicalPlanNode { + return Ok(protobuf::PhysicalPlanNode { physical_plan_type: Some(PhysicalPlanType::CoalesceBatches(Box::new( protobuf::CoalesceBatchesExecNode { input: Some(Box::new(input)), target_batch_size: coalesce_batches.target_batch_size() as u32, }, ))), - }) - } else if let Some(exec) = plan.downcast_ref::() { - Ok(protobuf::PhysicalPlanNode { + }); + } + + if let Some(exec) = plan.downcast_ref::() { + return Ok(protobuf::PhysicalPlanNode { physical_plan_type: Some(PhysicalPlanType::CsvScan( protobuf::CsvScanExecNode { base_conf: Some(exec.base_config().try_into()?), @@ -1104,41 +1127,50 @@ impl AsExecutionPlan for PhysicalPlanNode { }, }, )), - }) - } else if let Some(exec) = plan.downcast_ref::() { + }); + } + + #[cfg(feature = "parquet")] + if let Some(exec) = plan.downcast_ref::() { let predicate = exec .predicate() .map(|pred| pred.clone().try_into()) .transpose()?; - Ok(protobuf::PhysicalPlanNode { + return Ok(protobuf::PhysicalPlanNode { physical_plan_type: Some(PhysicalPlanType::ParquetScan( protobuf::ParquetScanExecNode { base_conf: Some(exec.base_config().try_into()?), predicate, }, )), - }) - } else if let Some(exec) = plan.downcast_ref::() { - Ok(protobuf::PhysicalPlanNode { + }); + } + + if let Some(exec) = plan.downcast_ref::() { + return Ok(protobuf::PhysicalPlanNode { physical_plan_type: Some(PhysicalPlanType::AvroScan( protobuf::AvroScanExecNode { base_conf: Some(exec.base_config().try_into()?), }, )), - }) - } else if let Some(exec) = plan.downcast_ref::() { + }); + } + + if let Some(exec) = plan.downcast_ref::() { let input = protobuf::PhysicalPlanNode::try_from_physical_plan( exec.input().to_owned(), extension_codec, )?; - Ok(protobuf::PhysicalPlanNode { + return Ok(protobuf::PhysicalPlanNode { physical_plan_type: Some(PhysicalPlanType::Merge(Box::new( protobuf::CoalescePartitionsExecNode { input: Some(Box::new(input)), }, ))), - }) - } else if let Some(exec) = plan.downcast_ref::() { + }); + } + + if let Some(exec) = plan.downcast_ref::() { let input = protobuf::PhysicalPlanNode::try_from_physical_plan( exec.input().to_owned(), extension_codec, @@ -1162,15 +1194,17 @@ impl AsExecutionPlan for PhysicalPlanNode { } }; - Ok(protobuf::PhysicalPlanNode { + return Ok(protobuf::PhysicalPlanNode { physical_plan_type: Some(PhysicalPlanType::Repartition(Box::new( protobuf::RepartitionExecNode { input: Some(Box::new(input)), partition_method: Some(pb_partition_method), }, ))), - }) - } else if let Some(exec) = plan.downcast_ref::() { + }); + } + + if let Some(exec) = plan.downcast_ref::() { let input = protobuf::PhysicalPlanNode::try_from_physical_plan( exec.input().to_owned(), extension_codec, @@ -1191,7 +1225,7 @@ impl AsExecutionPlan for PhysicalPlanNode { }) }) .collect::>>()?; - Ok(protobuf::PhysicalPlanNode { + return Ok(protobuf::PhysicalPlanNode { physical_plan_type: Some(PhysicalPlanType::Sort(Box::new( protobuf::SortExecNode { input: Some(Box::new(input)), @@ -1203,8 +1237,10 @@ impl AsExecutionPlan for PhysicalPlanNode { preserve_partitioning: exec.preserve_partitioning(), }, ))), - }) - } else if let Some(union) = plan.downcast_ref::() { + }); + } + + if let Some(union) = plan.downcast_ref::() { let mut inputs: Vec = vec![]; for input in union.inputs() { inputs.push(protobuf::PhysicalPlanNode::try_from_physical_plan( @@ -1212,12 +1248,14 @@ impl AsExecutionPlan for PhysicalPlanNode { extension_codec, )?); } - Ok(protobuf::PhysicalPlanNode { + return Ok(protobuf::PhysicalPlanNode { physical_plan_type: Some(PhysicalPlanType::Union( protobuf::UnionExecNode { inputs }, )), - }) - } else if let Some(exec) = plan.downcast_ref::() { + }); + } + + if let Some(exec) = plan.downcast_ref::() { let input = protobuf::PhysicalPlanNode::try_from_physical_plan( exec.input().to_owned(), extension_codec, @@ -1238,7 +1276,7 @@ impl AsExecutionPlan for PhysicalPlanNode { }) }) .collect::>>()?; - Ok(protobuf::PhysicalPlanNode { + return Ok(protobuf::PhysicalPlanNode { physical_plan_type: Some(PhysicalPlanType::SortPreservingMerge( Box::new(protobuf::SortPreservingMergeExecNode { input: Some(Box::new(input)), @@ -1246,8 +1284,10 @@ impl AsExecutionPlan for PhysicalPlanNode { fetch: exec.fetch().map(|f| f as i64).unwrap_or(-1), }), )), - }) - } else if let Some(exec) = plan.downcast_ref::() { + }); + } + + if let Some(exec) = plan.downcast_ref::() { let left = protobuf::PhysicalPlanNode::try_from_physical_plan( exec.left().to_owned(), extension_codec, @@ -1283,7 +1323,7 @@ impl AsExecutionPlan for PhysicalPlanNode { }) .map_or(Ok(None), |v: Result| v.map(Some))?; - Ok(protobuf::PhysicalPlanNode { + return Ok(protobuf::PhysicalPlanNode { physical_plan_type: Some(PhysicalPlanType::NestedLoopJoin(Box::new( protobuf::NestedLoopJoinExecNode { left: Some(Box::new(left)), @@ -1292,8 +1332,10 @@ impl AsExecutionPlan for PhysicalPlanNode { filter, }, ))), - }) - } else if let Some(exec) = plan.downcast_ref::() { + }); + } + + if let Some(exec) = plan.downcast_ref::() { let input = protobuf::PhysicalPlanNode::try_from_physical_plan( exec.input().to_owned(), extension_codec, @@ -1311,7 +1353,7 @@ impl AsExecutionPlan for PhysicalPlanNode { .map(|e| e.clone().try_into()) .collect::>>()?; - Ok(protobuf::PhysicalPlanNode { + return Ok(protobuf::PhysicalPlanNode { physical_plan_type: Some(PhysicalPlanType::Window(Box::new( protobuf::WindowAggExecNode { input: Some(Box::new(input)), @@ -1320,8 +1362,10 @@ impl AsExecutionPlan for PhysicalPlanNode { partition_search_mode: None, }, ))), - }) - } else if let Some(exec) = plan.downcast_ref::() { + }); + } + + if let Some(exec) = plan.downcast_ref::() { let input = protobuf::PhysicalPlanNode::try_from_physical_plan( exec.input().to_owned(), extension_codec, @@ -1359,7 +1403,7 @@ impl AsExecutionPlan for PhysicalPlanNode { } }; - Ok(protobuf::PhysicalPlanNode { + return Ok(protobuf::PhysicalPlanNode { physical_plan_type: Some(PhysicalPlanType::Window(Box::new( protobuf::WindowAggExecNode { input: Some(Box::new(input)), @@ -1368,32 +1412,32 @@ impl AsExecutionPlan for PhysicalPlanNode { partition_search_mode: Some(partition_search_mode), }, ))), - }) - } else { - let mut buf: Vec = vec![]; - match extension_codec.try_encode(plan_clone.clone(), &mut buf) { - Ok(_) => { - let inputs: Vec = plan_clone - .children() - .into_iter() - .map(|i| { - protobuf::PhysicalPlanNode::try_from_physical_plan( - i, - extension_codec, - ) - }) - .collect::>()?; + }); + } - Ok(protobuf::PhysicalPlanNode { - physical_plan_type: Some(PhysicalPlanType::Extension( - protobuf::PhysicalExtensionNode { node: buf, inputs }, - )), + let mut buf: Vec = vec![]; + match extension_codec.try_encode(plan_clone.clone(), &mut buf) { + Ok(_) => { + let inputs: Vec = plan_clone + .children() + .into_iter() + .map(|i| { + protobuf::PhysicalPlanNode::try_from_physical_plan( + i, + extension_codec, + ) }) - } - Err(e) => internal_err!( - "Unsupported plan and extension codec failed with [{e}]. Plan: {plan_clone:?}" - ), + .collect::>()?; + + Ok(protobuf::PhysicalPlanNode { + physical_plan_type: Some(PhysicalPlanType::Extension( + protobuf::PhysicalExtensionNode { node: buf, inputs }, + )), + }) } + Err(e) => internal_err!( + "Unsupported plan and extension codec failed with [{e}]. Plan: {plan_clone:?}" + ), } } } From 128d7c6700bd6b8300cae86932b1e6d9bf74414d Mon Sep 17 00:00:00 2001 From: Mustafa Akur <106137913+mustafasrepo@users.noreply.github.com> Date: Thu, 26 Oct 2023 01:04:12 +0300 Subject: [PATCH 132/572] [MINOR]: Simplify enforce_distribution, minor changes (#7924) * Initial commit * Simplifications * Cleanup imports * Review --------- Co-authored-by: Mehmet Ozan Kabak --- .../enforce_distribution.rs | 196 +++++++----------- .../src/physical_optimizer/enforce_sorting.rs | 22 +- .../core/src/physical_optimizer/test_utils.rs | 3 +- .../core/tests/fuzz_cases/window_fuzz.rs | 17 +- .../physical-expr/src/aggregate/first_last.rs | 7 +- datafusion/physical-expr/src/physical_expr.rs | 71 +++++-- .../physical-expr/src/scalar_function.rs | 21 +- .../physical-plan/src/aggregates/mod.rs | 2 +- .../src/windows/bounded_window_agg_exec.rs | 12 +- datafusion/physical-plan/src/windows/mod.rs | 12 +- 10 files changed, 184 insertions(+), 179 deletions(-) diff --git a/datafusion/core/src/physical_optimizer/enforce_distribution.rs b/datafusion/core/src/physical_optimizer/enforce_distribution.rs index d3fbc46a6659..072c3cb6d7a6 100644 --- a/datafusion/core/src/physical_optimizer/enforce_distribution.rs +++ b/datafusion/core/src/physical_optimizer/enforce_distribution.rs @@ -486,7 +486,7 @@ fn reorder_aggregate_keys( parent_required: &[Arc], agg_exec: &AggregateExec, ) -> Result { - let out_put_columns = agg_exec + let output_columns = agg_exec .group_by() .expr() .iter() @@ -494,44 +494,32 @@ fn reorder_aggregate_keys( .map(|(index, (_col, name))| Column::new(name, index)) .collect::>(); - let out_put_exprs = out_put_columns + let output_exprs = output_columns .iter() - .map(|c| Arc::new(c.clone()) as Arc) + .map(|c| Arc::new(c.clone()) as _) .collect::>(); - if parent_required.len() != out_put_exprs.len() + if parent_required.len() != output_exprs.len() || !agg_exec.group_by().null_expr().is_empty() - || expr_list_eq_strict_order(&out_put_exprs, parent_required) + || expr_list_eq_strict_order(&output_exprs, parent_required) { Ok(PlanWithKeyRequirements::new(agg_plan)) } else { - let new_positions = expected_expr_positions(&out_put_exprs, parent_required); + let new_positions = expected_expr_positions(&output_exprs, parent_required); match new_positions { None => Ok(PlanWithKeyRequirements::new(agg_plan)), Some(positions) => { let new_partial_agg = if let Some(agg_exec) = agg_exec.input().as_any().downcast_ref::() - /*AggregateExec { - mode, - group_by, - aggr_expr, - filter_expr, - order_by_expr, - input, - input_schema, - .. - }) = - */ { if matches!(agg_exec.mode(), &AggregateMode::Partial) { - let mut new_group_exprs = vec![]; - for idx in positions.iter() { - new_group_exprs - .push(agg_exec.group_by().expr()[*idx].clone()); - } + let group_exprs = agg_exec.group_by().expr(); + let new_group_exprs = positions + .into_iter() + .map(|idx| group_exprs[idx].clone()) + .collect(); let new_partial_group_by = PhysicalGroupBy::new_single(new_group_exprs); - // new Partial AggregateExec Some(Arc::new(AggregateExec::try_new( AggregateMode::Partial, new_partial_group_by, @@ -549,18 +537,13 @@ fn reorder_aggregate_keys( }; if let Some(partial_agg) = new_partial_agg { // Build new group expressions that correspond to the output of partial_agg - let new_final_group: Vec> = - partial_agg.output_group_expr(); + let group_exprs = partial_agg.group_expr().expr(); + let new_final_group = partial_agg.output_group_expr(); let new_group_by = PhysicalGroupBy::new_single( new_final_group .iter() .enumerate() - .map(|(i, expr)| { - ( - expr.clone(), - partial_agg.group_expr().expr()[i].1.clone(), - ) - }) + .map(|(idx, expr)| (expr.clone(), group_exprs[idx].1.clone())) .collect(), ); @@ -575,29 +558,29 @@ fn reorder_aggregate_keys( )?); // Need to create a new projection to change the expr ordering back - let mut proj_exprs = out_put_columns + let agg_schema = new_final_agg.schema(); + let mut proj_exprs = output_columns .iter() .map(|col| { + let name = col.name(); ( Arc::new(Column::new( - col.name(), - new_final_agg.schema().index_of(col.name()).unwrap(), + name, + agg_schema.index_of(name).unwrap(), )) as Arc, - col.name().to_owned(), + name.to_owned(), ) }) .collect::>(); let agg_schema = new_final_agg.schema(); let agg_fields = agg_schema.fields(); for (idx, field) in - agg_fields.iter().enumerate().skip(out_put_columns.len()) + agg_fields.iter().enumerate().skip(output_columns.len()) { - proj_exprs.push(( - Arc::new(Column::new(field.name().as_str(), idx)) - as Arc, - field.name().clone(), - )) + let name = field.name(); + proj_exprs + .push((Arc::new(Column::new(name, idx)) as _, name.clone())) } // TODO merge adjacent Projections if there are Ok(PlanWithKeyRequirements::new(Arc::new( @@ -615,15 +598,14 @@ fn shift_right_required( parent_required: &[Arc], left_columns_len: usize, ) -> Option>> { - let new_right_required: Vec> = parent_required + let new_right_required = parent_required .iter() .filter_map(|r| { if let Some(col) = r.as_any().downcast_ref::() { - if col.index() >= left_columns_len { - Some( - Arc::new(Column::new(col.name(), col.index() - left_columns_len)) - as Arc, - ) + let idx = col.index(); + if idx >= left_columns_len { + let result = Column::new(col.name(), idx - left_columns_len); + Some(Arc::new(result) as _) } else { None } @@ -634,11 +616,7 @@ fn shift_right_required( .collect::>(); // if the parent required are all comming from the right side, the requirements can be pushdown - if new_right_required.len() != parent_required.len() { - None - } else { - Some(new_right_required) - } + (new_right_required.len() == parent_required.len()).then_some(new_right_required) } /// When the physical planner creates the Joins, the ordering of join keys is from the original query. @@ -662,8 +640,8 @@ fn shift_right_required( /// In that case, the datasources/tables might be pre-partitioned and we can't adjust the key ordering of the datasources /// and then can't apply the Top-Down reordering process. pub(crate) fn reorder_join_keys_to_inputs( - plan: Arc, -) -> Result> { + plan: Arc, +) -> Result> { let plan_any = plan.as_any(); if let Some(HashJoinExec { left, @@ -676,41 +654,34 @@ pub(crate) fn reorder_join_keys_to_inputs( .. }) = plan_any.downcast_ref::() { - match mode { - PartitionMode::Partitioned => { - let join_key_pairs = extract_join_keys(on); - if let Some(( - JoinKeyPairs { - left_keys, - right_keys, - }, - new_positions, - )) = reorder_current_join_keys( - join_key_pairs, - Some(left.output_partitioning()), - Some(right.output_partitioning()), - &left.equivalence_properties(), - &right.equivalence_properties(), - ) { - if !new_positions.is_empty() { - let new_join_on = new_join_conditions(&left_keys, &right_keys); - Ok(Arc::new(HashJoinExec::try_new( - left.clone(), - right.clone(), - new_join_on, - filter.clone(), - join_type, - PartitionMode::Partitioned, - *null_equals_null, - )?)) - } else { - Ok(plan) - } - } else { - Ok(plan) + if matches!(mode, PartitionMode::Partitioned) { + let join_key_pairs = extract_join_keys(on); + if let Some(( + JoinKeyPairs { + left_keys, + right_keys, + }, + new_positions, + )) = reorder_current_join_keys( + join_key_pairs, + Some(left.output_partitioning()), + Some(right.output_partitioning()), + &left.equivalence_properties(), + &right.equivalence_properties(), + ) { + if !new_positions.is_empty() { + let new_join_on = new_join_conditions(&left_keys, &right_keys); + return Ok(Arc::new(HashJoinExec::try_new( + left.clone(), + right.clone(), + new_join_on, + filter.clone(), + join_type, + PartitionMode::Partitioned, + *null_equals_null, + )?)); } } - _ => Ok(plan), } } else if let Some(SortMergeJoinExec { left, @@ -742,23 +713,18 @@ pub(crate) fn reorder_join_keys_to_inputs( for idx in 0..sort_options.len() { new_sort_options.push(sort_options[new_positions[idx]]) } - Ok(Arc::new(SortMergeJoinExec::try_new( + return Ok(Arc::new(SortMergeJoinExec::try_new( left.clone(), right.clone(), new_join_on, *join_type, new_sort_options, *null_equals_null, - )?)) - } else { - Ok(plan) + )?)); } - } else { - Ok(plan) } - } else { - Ok(plan) } + Ok(plan) } /// Reorder the current join keys ordering based on either left partition or right partition @@ -886,12 +852,7 @@ fn expected_expr_positions( fn extract_join_keys(on: &[(Column, Column)]) -> JoinKeyPairs { let (left_keys, right_keys) = on .iter() - .map(|(l, r)| { - ( - Arc::new(l.clone()) as Arc, - Arc::new(r.clone()) as Arc, - ) - }) + .map(|(l, r)| (Arc::new(l.clone()) as _, Arc::new(r.clone()) as _)) .unzip(); JoinKeyPairs { left_keys, @@ -903,7 +864,7 @@ fn new_join_conditions( new_left_keys: &[Arc], new_right_keys: &[Arc], ) -> Vec<(Column, Column)> { - let new_join_on = new_left_keys + new_left_keys .iter() .zip(new_right_keys.iter()) .map(|(l_key, r_key)| { @@ -912,8 +873,7 @@ fn new_join_conditions( r_key.as_any().downcast_ref::().unwrap().clone(), ) }) - .collect::>(); - new_join_on + .collect::>() } /// Updates `dist_onward` such that, to keep track of @@ -977,10 +937,10 @@ fn add_roundrobin_on_top( // (determined by flag `config.optimizer.bounded_order_preserving_variants`) let should_preserve_ordering = input.output_ordering().is_some(); - let new_plan = Arc::new( - RepartitionExec::try_new(input, Partitioning::RoundRobinBatch(n_target))? - .with_preserve_order(should_preserve_ordering), - ) as Arc; + let partitioning = Partitioning::RoundRobinBatch(n_target); + let repartition = RepartitionExec::try_new(input, partitioning)? + .with_preserve_order(should_preserve_ordering); + let new_plan = Arc::new(repartition) as Arc; // update distribution onward with new operator update_distribution_onward(new_plan.clone(), dist_onward, input_idx); @@ -1009,7 +969,7 @@ fn add_roundrobin_on_top( /// /// # Returns /// -/// A [Result] object that contains new execution plan, where desired distribution is +/// A [`Result`] object that contains new execution plan, where desired distribution is /// satisfied by adding Hash Repartition. fn add_hash_on_top( input: Arc, @@ -1053,10 +1013,10 @@ fn add_hash_on_top( } else { input }; - new_plan = Arc::new( - RepartitionExec::try_new(new_plan, Partitioning::Hash(hash_exprs, n_target))? - .with_preserve_order(should_preserve_ordering), - ) as _; + let partitioning = Partitioning::Hash(hash_exprs, n_target); + let repartition = RepartitionExec::try_new(new_plan, partitioning)? + .with_preserve_order(should_preserve_ordering); + new_plan = Arc::new(repartition) as _; // update distribution onward with new operator update_distribution_onward(new_plan.clone(), dist_onward, input_idx); @@ -1146,7 +1106,7 @@ fn remove_dist_changing_operators( { // All of above operators have a single child. When we remove the top // operator, we take the first child. - plan = plan.children()[0].clone(); + plan = plan.children().swap_remove(0); distribution_onwards = get_children_exectrees(plan.children().len(), &distribution_onwards[0]); } @@ -1199,14 +1159,14 @@ fn replace_order_preserving_variants_helper( } if is_sort_preserving_merge(&exec_tree.plan) { return Ok(Arc::new(CoalescePartitionsExec::new( - updated_children[0].clone(), + updated_children.swap_remove(0), ))); } if let Some(repartition) = exec_tree.plan.as_any().downcast_ref::() { if repartition.preserve_order() { return Ok(Arc::new( RepartitionExec::try_new( - updated_children[0].clone(), + updated_children.swap_remove(0), repartition.partitioning().clone(), )? .with_preserve_order(false), @@ -1427,7 +1387,7 @@ fn ensure_distribution( // Data Arc::new(InterleaveExec::try_new(new_children)?) } else { - plan.clone().with_new_children(new_children)? + plan.with_new_children(new_children)? }, distribution_onwards, }; @@ -1624,7 +1584,7 @@ impl PlanWithKeyRequirements { let length = child.children().len(); PlanWithKeyRequirements { plan: child, - required_key_ordering: from_parent.clone(), + required_key_ordering: from_parent, request_key_ordering: vec![None; length], } }) diff --git a/datafusion/core/src/physical_optimizer/enforce_sorting.rs b/datafusion/core/src/physical_optimizer/enforce_sorting.rs index 913dae07faa1..822a224d236a 100644 --- a/datafusion/core/src/physical_optimizer/enforce_sorting.rs +++ b/datafusion/core/src/physical_optimizer/enforce_sorting.rs @@ -17,8 +17,8 @@ //! EnforceSorting optimizer rule inspects the physical plan with respect //! to local sorting requirements and does the following: -//! - Adds a [SortExec] when a requirement is not met, -//! - Removes an already-existing [SortExec] if it is possible to prove +//! - Adds a [`SortExec`] when a requirement is not met, +//! - Removes an already-existing [`SortExec`] if it is possible to prove //! that this sort is unnecessary //! The rule can work on valid *and* invalid physical plans with respect to //! sorting requirements, but always produces a valid physical plan in this sense. @@ -496,9 +496,10 @@ fn ensure_sorting( { // This SortPreservingMergeExec is unnecessary, input already has a // single partition. + sort_onwards.truncate(1); return Ok(Transformed::Yes(PlanWithCorrespondingSort { - plan: children[0].clone(), - sort_onwards: vec![sort_onwards[0].clone()], + plan: children.swap_remove(0), + sort_onwards, })); } Ok(Transformed::Yes(PlanWithCorrespondingSort { @@ -649,7 +650,7 @@ fn remove_corresponding_coalesce_in_sub_plan( && is_repartition(&new_plan) && is_repartition(parent) { - new_plan = new_plan.children()[0].clone() + new_plan = new_plan.children().swap_remove(0) } new_plan } else { @@ -689,7 +690,7 @@ fn remove_corresponding_sort_from_sub_plan( ) -> Result> { // A `SortExec` is always at the bottom of the tree. let mut updated_plan = if is_sort(&sort_onwards.plan) { - sort_onwards.plan.children()[0].clone() + sort_onwards.plan.children().swap_remove(0) } else { let plan = &sort_onwards.plan; let mut children = plan.children(); @@ -703,12 +704,12 @@ fn remove_corresponding_sort_from_sub_plan( } // Replace with variants that do not preserve order. if is_sort_preserving_merge(plan) { - children[0].clone() + children.swap_remove(0) } else if let Some(repartition) = plan.as_any().downcast_ref::() { Arc::new( RepartitionExec::try_new( - children[0].clone(), + children.swap_remove(0), repartition.partitioning().clone(), )? .with_preserve_order(false), @@ -730,7 +731,7 @@ fn remove_corresponding_sort_from_sub_plan( updated_plan, )); } else { - updated_plan = Arc::new(CoalescePartitionsExec::new(updated_plan.clone())); + updated_plan = Arc::new(CoalescePartitionsExec::new(updated_plan)); } } Ok(updated_plan) @@ -777,8 +778,7 @@ mod tests { use arrow::datatypes::{DataType, Field, Schema, SchemaRef}; use datafusion_common::Result; use datafusion_expr::JoinType; - use datafusion_physical_expr::expressions::Column; - use datafusion_physical_expr::expressions::{col, NotExpr}; + use datafusion_physical_expr::expressions::{col, Column, NotExpr}; fn create_test_schema() -> Result { let nullable_column = Field::new("nullable_col", DataType::Int32, true); diff --git a/datafusion/core/src/physical_optimizer/test_utils.rs b/datafusion/core/src/physical_optimizer/test_utils.rs index 53401751b67e..159ee5089075 100644 --- a/datafusion/core/src/physical_optimizer/test_utils.rs +++ b/datafusion/core/src/physical_optimizer/test_utils.rs @@ -44,6 +44,7 @@ use datafusion_execution::object_store::ObjectStoreUrl; use datafusion_expr::{AggregateFunction, WindowFrame, WindowFunction}; use datafusion_physical_expr::expressions::col; use datafusion_physical_expr::{PhysicalExpr, PhysicalSortExpr}; +use datafusion_physical_plan::windows::PartitionSearchMode; use async_trait::async_trait; @@ -239,7 +240,7 @@ pub fn bounded_window_exec( .unwrap()], input.clone(), vec![], - crate::physical_plan::windows::PartitionSearchMode::Sorted, + PartitionSearchMode::Sorted, ) .unwrap(), ) diff --git a/datafusion/core/tests/fuzz_cases/window_fuzz.rs b/datafusion/core/tests/fuzz_cases/window_fuzz.rs index 83c8e1f57896..db940a9794a1 100644 --- a/datafusion/core/tests/fuzz_cases/window_fuzz.rs +++ b/datafusion/core/tests/fuzz_cases/window_fuzz.rs @@ -22,9 +22,6 @@ use arrow::compute::{concat_batches, SortOptions}; use arrow::datatypes::SchemaRef; use arrow::record_batch::RecordBatch; use arrow::util::pretty::pretty_format_batches; -use hashbrown::HashMap; -use rand::rngs::StdRng; -use rand::{Rng, SeedableRng}; use datafusion::physical_plan::memory::MemoryExec; use datafusion::physical_plan::sorts::sort::SortExec; @@ -32,22 +29,26 @@ use datafusion::physical_plan::windows::{ create_window_expr, BoundedWindowAggExec, PartitionSearchMode, WindowAggExec, }; use datafusion::physical_plan::{collect, ExecutionPlan}; +use datafusion::prelude::{SessionConfig, SessionContext}; +use datafusion_common::{Result, ScalarValue}; +use datafusion_expr::type_coercion::aggregates::coerce_types; use datafusion_expr::{ AggregateFunction, BuiltInWindowFunction, WindowFrame, WindowFrameBound, WindowFrameUnits, WindowFunction, }; - -use datafusion::prelude::{SessionConfig, SessionContext}; -use datafusion_common::{Result, ScalarValue}; -use datafusion_expr::type_coercion::aggregates::coerce_types; use datafusion_physical_expr::expressions::{cast, col, lit}; use datafusion_physical_expr::{PhysicalExpr, PhysicalSortExpr}; use test_utils::add_empty_batches; +use hashbrown::HashMap; +use rand::rngs::StdRng; +use rand::{Rng, SeedableRng}; + #[cfg(test)] mod tests { use super::*; - use datafusion::physical_plan::windows::PartitionSearchMode::{ + + use datafusion_physical_plan::windows::PartitionSearchMode::{ Linear, PartiallySorted, Sorted, }; diff --git a/datafusion/physical-expr/src/aggregate/first_last.rs b/datafusion/physical-expr/src/aggregate/first_last.rs index ce7a1daeec64..a4e0a6dc49a9 100644 --- a/datafusion/physical-expr/src/aggregate/first_last.rs +++ b/datafusion/physical-expr/src/aggregate/first_last.rs @@ -26,12 +26,9 @@ use crate::{ reverse_order_bys, AggregateExpr, LexOrdering, PhysicalExpr, PhysicalSortExpr, }; -use arrow::array::ArrayRef; -use arrow::compute; -use arrow::compute::{lexsort_to_indices, SortColumn}; +use arrow::array::{Array, ArrayRef, AsArray, BooleanArray}; +use arrow::compute::{self, lexsort_to_indices, SortColumn}; use arrow::datatypes::{DataType, Field}; -use arrow_array::cast::AsArray; -use arrow_array::{Array, BooleanArray}; use arrow_schema::SortOptions; use datafusion_common::utils::{compare_rows, get_arrayref_at_indices, get_row_at_idx}; use datafusion_common::{DataFusionError, Result, ScalarValue}; diff --git a/datafusion/physical-expr/src/physical_expr.rs b/datafusion/physical-expr/src/physical_expr.rs index 0eff45b6b9f7..11fa6c899621 100644 --- a/datafusion/physical-expr/src/physical_expr.rs +++ b/datafusion/physical-expr/src/physical_expr.rs @@ -15,6 +15,11 @@ // specific language governing permissions and limitations // under the License. +use std::any::Any; +use std::fmt::{Debug, Display}; +use std::hash::{Hash, Hasher}; +use std::sync::Arc; + use crate::intervals::Interval; use crate::sort_properties::SortProperties; use crate::utils::scatter; @@ -27,11 +32,6 @@ use datafusion_common::utils::DataPtr; use datafusion_common::{internal_err, not_impl_err, DataFusionError, Result}; use datafusion_expr::ColumnarValue; -use std::any::Any; -use std::fmt::{Debug, Display}; -use std::hash::{Hash, Hasher}; -use std::sync::Arc; - /// Expression that can be evaluated against a RecordBatch /// A Physical expression knows its type, nullability and how to evaluate itself. pub trait PhysicalExpr: Send + Sync + Display + Debug + PartialEq { @@ -54,13 +54,12 @@ pub trait PhysicalExpr: Send + Sync + Display + Debug + PartialEq { let tmp_batch = filter_record_batch(batch, selection)?; let tmp_result = self.evaluate(&tmp_batch)?; - // All values from the `selection` filter are true. + if batch.num_rows() == tmp_batch.num_rows() { - return Ok(tmp_result); - } - if let ColumnarValue::Array(a) = tmp_result { - let result = scatter(selection, a.as_ref())?; - Ok(ColumnarValue::Array(result)) + // All values from the `selection` filter are true. + Ok(tmp_result) + } else if let ColumnarValue::Array(a) = tmp_result { + scatter(selection, a.as_ref()).map(ColumnarValue::Array) } else { Ok(tmp_result) } @@ -216,8 +215,8 @@ pub fn down_cast_any_ref(any: &dyn Any) -> &dyn Any { } } -/// It is similar to contains method of vector. -/// Finds whether `expr` is among `physical_exprs`. +/// This function is similar to the `contains` method of `Vec`. It finds +/// whether `expr` is among `physical_exprs`. pub fn physical_exprs_contains( physical_exprs: &[Arc], expr: &Arc, @@ -226,3 +225,49 @@ pub fn physical_exprs_contains( .iter() .any(|physical_expr| physical_expr.eq(expr)) } + +#[cfg(test)] +mod tests { + use std::sync::Arc; + + use crate::expressions::{Column, Literal}; + use crate::physical_expr::{physical_exprs_contains, PhysicalExpr}; + + use datafusion_common::{Result, ScalarValue}; + + #[test] + fn test_physical_exprs_contains() -> Result<()> { + let lit_true = Arc::new(Literal::new(ScalarValue::Boolean(Some(true)))) + as Arc; + let lit_false = Arc::new(Literal::new(ScalarValue::Boolean(Some(false)))) + as Arc; + let lit4 = + Arc::new(Literal::new(ScalarValue::Int32(Some(4)))) as Arc; + let lit2 = + Arc::new(Literal::new(ScalarValue::Int32(Some(2)))) as Arc; + let lit1 = + Arc::new(Literal::new(ScalarValue::Int32(Some(1)))) as Arc; + let col_a_expr = Arc::new(Column::new("a", 0)) as Arc; + let col_b_expr = Arc::new(Column::new("b", 1)) as Arc; + let col_c_expr = Arc::new(Column::new("c", 2)) as Arc; + + // lit(true), lit(false), lit(4), lit(2), Col(a), Col(b) + let physical_exprs: Vec> = vec![ + lit_true.clone(), + lit_false.clone(), + lit4.clone(), + lit2.clone(), + col_a_expr.clone(), + col_b_expr.clone(), + ]; + // below expressions are inside physical_exprs + assert!(physical_exprs_contains(&physical_exprs, &lit_true)); + assert!(physical_exprs_contains(&physical_exprs, &lit2)); + assert!(physical_exprs_contains(&physical_exprs, &col_b_expr)); + + // below expressions are not inside physical_exprs + assert!(!physical_exprs_contains(&physical_exprs, &col_c_expr)); + assert!(!physical_exprs_contains(&physical_exprs, &lit1)); + Ok(()) + } +} diff --git a/datafusion/physical-expr/src/scalar_function.rs b/datafusion/physical-expr/src/scalar_function.rs index dc48baa23ab3..43598ce56489 100644 --- a/datafusion/physical-expr/src/scalar_function.rs +++ b/datafusion/physical-expr/src/scalar_function.rs @@ -29,24 +29,25 @@ //! This module also has a set of coercion rules to improve user experience: if an argument i32 is passed //! to a function that supports f64, it is coerced to f64. +use std::any::Any; +use std::fmt::Debug; +use std::fmt::{self, Formatter}; +use std::hash::{Hash, Hasher}; +use std::sync::Arc; + use crate::functions::out_ordering; use crate::physical_expr::down_cast_any_ref; use crate::sort_properties::SortProperties; use crate::utils::expr_list_eq_strict_order; use crate::PhysicalExpr; + use arrow::datatypes::{DataType, Schema}; use arrow::record_batch::RecordBatch; use datafusion_common::Result; -use datafusion_expr::expr_vec_fmt; -use datafusion_expr::BuiltinScalarFunction; -use datafusion_expr::ColumnarValue; -use datafusion_expr::FuncMonotonicity; -use datafusion_expr::ScalarFunctionImplementation; -use std::any::Any; -use std::fmt::Debug; -use std::fmt::{self, Formatter}; -use std::hash::{Hash, Hasher}; -use std::sync::Arc; +use datafusion_expr::{ + expr_vec_fmt, BuiltinScalarFunction, ColumnarValue, FuncMonotonicity, + ScalarFunctionImplementation, +}; /// Physical expression of a scalar function pub struct ScalarFunctionExpr { diff --git a/datafusion/physical-plan/src/aggregates/mod.rs b/datafusion/physical-plan/src/aggregates/mod.rs index 1fa129680cea..4c612223178c 100644 --- a/datafusion/physical-plan/src/aggregates/mod.rs +++ b/datafusion/physical-plan/src/aggregates/mod.rs @@ -228,7 +228,7 @@ impl PhysicalGroupBy { } /// Return grouping expressions as they occur in the output schema. - fn output_exprs(&self) -> Vec> { + pub fn output_exprs(&self) -> Vec> { self.expr .iter() .enumerate() diff --git a/datafusion/physical-plan/src/windows/bounded_window_agg_exec.rs b/datafusion/physical-plan/src/windows/bounded_window_agg_exec.rs index eab47886c764..f6ffe2e26795 100644 --- a/datafusion/physical-plan/src/windows/bounded_window_agg_exec.rs +++ b/datafusion/physical-plan/src/windows/bounded_window_agg_exec.rs @@ -31,6 +31,7 @@ use crate::expressions::PhysicalSortExpr; use crate::metrics::{BaselineMetrics, ExecutionPlanMetricsSet, MetricsSet}; use crate::windows::{ calc_requirements, get_ordered_partition_by_indices, window_ordering_equivalence, + PartitionSearchMode, }; use crate::{ ColumnStatistics, DisplayAs, DisplayFormatType, Distribution, ExecutionPlan, @@ -68,17 +69,6 @@ use hashbrown::raw::RawTable; use indexmap::IndexMap; use log::debug; -#[derive(Debug, Clone, PartialEq)] -/// Specifies partition column properties in terms of input ordering -pub enum PartitionSearchMode { - /// None of the columns among the partition columns is ordered. - Linear, - /// Some columns of the partition columns are ordered but not all - PartiallySorted(Vec), - /// All Partition columns are ordered (Also empty case) - Sorted, -} - /// Window execution plan #[derive(Debug)] pub struct BoundedWindowAggExec { diff --git a/datafusion/physical-plan/src/windows/mod.rs b/datafusion/physical-plan/src/windows/mod.rs index cc915e54af60..aff936499a5e 100644 --- a/datafusion/physical-plan/src/windows/mod.rs +++ b/datafusion/physical-plan/src/windows/mod.rs @@ -54,13 +54,23 @@ mod bounded_window_agg_exec; mod window_agg_exec; pub use bounded_window_agg_exec::BoundedWindowAggExec; -pub use bounded_window_agg_exec::PartitionSearchMode; pub use window_agg_exec::WindowAggExec; pub use datafusion_physical_expr::window::{ BuiltInWindowExpr, PlainAggregateWindowExpr, WindowExpr, }; +#[derive(Debug, Clone, PartialEq)] +/// Specifies partition column properties in terms of input ordering +pub enum PartitionSearchMode { + /// None of the columns among the partition columns is ordered. + Linear, + /// Some columns of the partition columns are ordered but not all + PartiallySorted(Vec), + /// All Partition columns are ordered (Also empty case) + Sorted, +} + /// Create a physical expression for window function pub fn create_window_expr( fun: &WindowFunction, From 4881b5db36a978ddd4523d5a03daf7c0938a23bd Mon Sep 17 00:00:00 2001 From: Jeffrey <22608443+Jefffrey@users.noreply.github.com> Date: Thu, 26 Oct 2023 09:18:52 +1100 Subject: [PATCH 133/572] Add simple window query to sqllogictest (#7928) --- datafusion/sqllogictest/test_files/window.slt | 7 +++++++ 1 file changed, 7 insertions(+) diff --git a/datafusion/sqllogictest/test_files/window.slt b/datafusion/sqllogictest/test_files/window.slt index 7226182cf3d0..213f6daaef3e 100644 --- a/datafusion/sqllogictest/test_files/window.slt +++ b/datafusion/sqllogictest/test_files/window.slt @@ -3378,3 +3378,10 @@ SELECT window1 AS (ORDER BY C3) ORDER BY C3 LIMIT 5 + + +# simple window query +query II +select sum(1) over() x, sum(1) over () y +---- +1 1 From 0911f1523ec7088bae88684ecb9bca94aa553693 Mon Sep 17 00:00:00 2001 From: Marco Neumann Date: Thu, 26 Oct 2023 13:06:37 +0200 Subject: [PATCH 134/572] ci: upgrade node to version 20 (#7918) --- .github/workflows/dev.yml | 2 +- .github/workflows/rust.yml | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/.github/workflows/dev.yml b/.github/workflows/dev.yml index 1f5088a1e6ce..cc23e99e8cba 100644 --- a/.github/workflows/dev.yml +++ b/.github/workflows/dev.yml @@ -43,7 +43,7 @@ jobs: - uses: actions/checkout@v4 - uses: actions/setup-node@v4 with: - node-version: "14" + node-version: "20" - name: Prettier check run: | # if you encounter error, rerun the command below and commit the changes diff --git a/.github/workflows/rust.yml b/.github/workflows/rust.yml index 55f6cecf54aa..6c3d60bd4399 100644 --- a/.github/workflows/rust.yml +++ b/.github/workflows/rust.yml @@ -527,7 +527,7 @@ jobs: rust-version: stable - uses: actions/setup-node@v4 with: - node-version: "14" + node-version: "20" - name: Check if configs.md has been modified run: | # If you encounter an error, run './dev/update_config_docs.sh' and commit From 12b473b08bfc7f64f67b977ffb2252ac09a502ab Mon Sep 17 00:00:00 2001 From: comphead Date: Thu, 26 Oct 2023 07:29:32 -0700 Subject: [PATCH 135/572] Change input for `to_timestamp` function to be seconds rather than nanoseconds, add `to_timestamp_nanos` (#7844) * Change input for `to_timestamp` function * docs * fix examples * output `to_timestamp` signature as ns --- datafusion/core/tests/sql/expr.rs | 18 ++++---- datafusion/core/tests/sql/timestamp.rs | 2 +- datafusion/expr/src/built_in_function.rs | 17 ++++++++ datafusion/expr/src/expr_fn.rs | 6 +++ .../simplify_expressions/expr_simplifier.rs | 2 +- .../physical-expr/src/datetime_expressions.rs | 11 ++++- datafusion/physical-expr/src/functions.rs | 42 +++++++++++++++---- datafusion/proto/proto/datafusion.proto | 1 + datafusion/proto/src/generated/pbjson.rs | 3 ++ datafusion/proto/src/generated/prost.rs | 3 ++ .../proto/src/logical_plan/from_proto.rs | 7 +++- datafusion/proto/src/logical_plan/to_proto.rs | 1 + .../sqllogictest/test_files/timestamps.slt | 29 ++++++++++--- .../source/user-guide/sql/scalar_functions.md | 20 +++++++-- 14 files changed, 130 insertions(+), 32 deletions(-) diff --git a/datafusion/core/tests/sql/expr.rs b/datafusion/core/tests/sql/expr.rs index 1995a040158d..7d41ad4a881c 100644 --- a/datafusion/core/tests/sql/expr.rs +++ b/datafusion/core/tests/sql/expr.rs @@ -639,7 +639,7 @@ async fn test_uuid_expression() -> Result<()> { async fn test_extract_date_part() -> Result<()> { test_expression!("date_part('YEAR', CAST('2000-01-01' AS DATE))", "2000.0"); test_expression!( - "EXTRACT(year FROM to_timestamp('2020-09-08T12:00:00+00:00'))", + "EXTRACT(year FROM timestamp '2020-09-08T12:00:00+00:00')", "2020.0" ); test_expression!("date_part('QUARTER', CAST('2000-01-01' AS DATE))", "1.0"); @@ -686,35 +686,35 @@ async fn test_extract_date_part() -> Result<()> { "12.0" ); test_expression!( - "EXTRACT(second FROM to_timestamp('2020-09-08T12:00:12.12345678+00:00'))", + "EXTRACT(second FROM timestamp '2020-09-08T12:00:12.12345678+00:00')", "12.12345678" ); test_expression!( - "EXTRACT(millisecond FROM to_timestamp('2020-09-08T12:00:12.12345678+00:00'))", + "EXTRACT(millisecond FROM timestamp '2020-09-08T12:00:12.12345678+00:00')", "12123.45678" ); test_expression!( - "EXTRACT(microsecond FROM to_timestamp('2020-09-08T12:00:12.12345678+00:00'))", + "EXTRACT(microsecond FROM timestamp '2020-09-08T12:00:12.12345678+00:00')", "12123456.78" ); test_expression!( - "EXTRACT(nanosecond FROM to_timestamp('2020-09-08T12:00:12.12345678+00:00'))", + "EXTRACT(nanosecond FROM timestamp '2020-09-08T12:00:12.12345678+00:00')", "1.212345678e10" ); test_expression!( - "date_part('second', to_timestamp('2020-09-08T12:00:12.12345678+00:00'))", + "date_part('second', timestamp '2020-09-08T12:00:12.12345678+00:00')", "12.12345678" ); test_expression!( - "date_part('millisecond', to_timestamp('2020-09-08T12:00:12.12345678+00:00'))", + "date_part('millisecond', timestamp '2020-09-08T12:00:12.12345678+00:00')", "12123.45678" ); test_expression!( - "date_part('microsecond', to_timestamp('2020-09-08T12:00:12.12345678+00:00'))", + "date_part('microsecond', timestamp '2020-09-08T12:00:12.12345678+00:00')", "12123456.78" ); test_expression!( - "date_part('nanosecond', to_timestamp('2020-09-08T12:00:12.12345678+00:00'))", + "date_part('nanosecond', timestamp '2020-09-08T12:00:12.12345678+00:00')", "1.212345678e10" ); diff --git a/datafusion/core/tests/sql/timestamp.rs b/datafusion/core/tests/sql/timestamp.rs index ada66503a181..a18e6831b615 100644 --- a/datafusion/core/tests/sql/timestamp.rs +++ b/datafusion/core/tests/sql/timestamp.rs @@ -742,7 +742,7 @@ async fn test_arrow_typeof() -> Result<()> { "+-----------------------------------------------------------------------+", "| arrow_typeof(date_trunc(Utf8(\"microsecond\"),to_timestamp(Int64(61)))) |", "+-----------------------------------------------------------------------+", - "| Timestamp(Nanosecond, None) |", + "| Timestamp(Second, None) |", "+-----------------------------------------------------------------------+", ]; assert_batches_eq!(expected, &actual); diff --git a/datafusion/expr/src/built_in_function.rs b/datafusion/expr/src/built_in_function.rs index 16554133d828..9edee7649f67 100644 --- a/datafusion/expr/src/built_in_function.rs +++ b/datafusion/expr/src/built_in_function.rs @@ -266,6 +266,8 @@ pub enum BuiltinScalarFunction { ToTimestampMillis, /// to_timestamp_micros ToTimestampMicros, + /// to_timestamp_nanos + ToTimestampNanos, /// to_timestamp_seconds ToTimestampSeconds, /// from_unixtime @@ -444,6 +446,7 @@ impl BuiltinScalarFunction { BuiltinScalarFunction::ToTimestamp => Volatility::Immutable, BuiltinScalarFunction::ToTimestampMillis => Volatility::Immutable, BuiltinScalarFunction::ToTimestampMicros => Volatility::Immutable, + BuiltinScalarFunction::ToTimestampNanos => Volatility::Immutable, BuiltinScalarFunction::ToTimestampSeconds => Volatility::Immutable, BuiltinScalarFunction::Translate => Volatility::Immutable, BuiltinScalarFunction::Trim => Volatility::Immutable, @@ -755,6 +758,7 @@ impl BuiltinScalarFunction { BuiltinScalarFunction::ToTimestamp => Ok(Timestamp(Nanosecond, None)), BuiltinScalarFunction::ToTimestampMillis => Ok(Timestamp(Millisecond, None)), BuiltinScalarFunction::ToTimestampMicros => Ok(Timestamp(Microsecond, None)), + BuiltinScalarFunction::ToTimestampNanos => Ok(Timestamp(Nanosecond, None)), BuiltinScalarFunction::ToTimestampSeconds => Ok(Timestamp(Second, None)), BuiltinScalarFunction::FromUnixtime => Ok(Timestamp(Second, None)), BuiltinScalarFunction::Now => { @@ -995,6 +999,18 @@ impl BuiltinScalarFunction { ], self.volatility(), ), + BuiltinScalarFunction::ToTimestampNanos => Signature::uniform( + 1, + vec![ + Int64, + Timestamp(Nanosecond, None), + Timestamp(Microsecond, None), + Timestamp(Millisecond, None), + Timestamp(Second, None), + Utf8, + ], + self.volatility(), + ), BuiltinScalarFunction::ToTimestampSeconds => Signature::uniform( 1, vec![ @@ -1431,6 +1447,7 @@ fn aliases(func: &BuiltinScalarFunction) -> &'static [&'static str] { BuiltinScalarFunction::ToTimestampMillis => &["to_timestamp_millis"], BuiltinScalarFunction::ToTimestampMicros => &["to_timestamp_micros"], BuiltinScalarFunction::ToTimestampSeconds => &["to_timestamp_seconds"], + BuiltinScalarFunction::ToTimestampNanos => &["to_timestamp_nanos"], BuiltinScalarFunction::FromUnixtime => &["from_unixtime"], // hashing functions diff --git a/datafusion/expr/src/expr_fn.rs b/datafusion/expr/src/expr_fn.rs index 5368a2d8a22c..5a60c2470c95 100644 --- a/datafusion/expr/src/expr_fn.rs +++ b/datafusion/expr/src/expr_fn.rs @@ -834,6 +834,12 @@ scalar_expr!( date, "converts a string to a `Timestamp(Microseconds, None)`" ); +scalar_expr!( + ToTimestampNanos, + to_timestamp_nanos, + date, + "converts a string to a `Timestamp(Nanoseconds, None)`" +); scalar_expr!( ToTimestampSeconds, to_timestamp_seconds, diff --git a/datafusion/optimizer/src/simplify_expressions/expr_simplifier.rs b/datafusion/optimizer/src/simplify_expressions/expr_simplifier.rs index cb3f13a51ec4..04fdcca0a994 100644 --- a/datafusion/optimizer/src/simplify_expressions/expr_simplifier.rs +++ b/datafusion/optimizer/src/simplify_expressions/expr_simplifier.rs @@ -1501,7 +1501,7 @@ mod tests { test_evaluate(expr, lit("foobarbaz")); // Check non string arguments - // to_timestamp("2020-09-08T12:00:00+00:00") --> timestamp(1599566400000000000i64) + // to_timestamp("2020-09-08T12:00:00+00:00") --> timestamp(1599566400i64) let expr = call_fn("to_timestamp", vec![lit("2020-09-08T12:00:00+00:00")]).unwrap(); test_evaluate(expr, lit_timestamp_nano(1599566400000000000i64)); diff --git a/datafusion/physical-expr/src/datetime_expressions.rs b/datafusion/physical-expr/src/datetime_expressions.rs index 5cf1c21df5c2..bb8720cb8d00 100644 --- a/datafusion/physical-expr/src/datetime_expressions.rs +++ b/datafusion/physical-expr/src/datetime_expressions.rs @@ -154,6 +154,15 @@ pub fn to_timestamp_micros(args: &[ColumnarValue]) -> Result { ) } +/// to_timestamp_nanos SQL function +pub fn to_timestamp_nanos(args: &[ColumnarValue]) -> Result { + handle::( + args, + string_to_timestamp_nanos_shim, + "to_timestamp_nanos", + ) +} + /// to_timestamp_seconds SQL function pub fn to_timestamp_seconds(args: &[ColumnarValue]) -> Result { handle::( @@ -962,7 +971,7 @@ mod tests { let mut string_builder = StringBuilder::with_capacity(2, 1024); let mut ts_builder = TimestampNanosecondArray::builder(2); - string_builder.append_value("2020-09-08T13:42:29.190855Z"); + string_builder.append_value("2020-09-08T13:42:29.190855"); ts_builder.append_value(1599572549190855000); string_builder.append_null(); diff --git a/datafusion/physical-expr/src/functions.rs b/datafusion/physical-expr/src/functions.rs index f23b45e26a03..8422862043ae 100644 --- a/datafusion/physical-expr/src/functions.rs +++ b/datafusion/physical-expr/src/functions.rs @@ -74,15 +74,20 @@ pub fn create_physical_expr( // so we don't have to pay a per-array/batch cost. BuiltinScalarFunction::ToTimestamp => { Arc::new(match input_phy_exprs[0].data_type(input_schema) { - Ok(DataType::Int64) | Ok(DataType::Timestamp(_, None)) => { - |col_values: &[ColumnarValue]| { - cast_column( - &col_values[0], - &DataType::Timestamp(TimeUnit::Nanosecond, None), - None, - ) - } - } + Ok(DataType::Int64) => |col_values: &[ColumnarValue]| { + cast_column( + &col_values[0], + &DataType::Timestamp(TimeUnit::Second, None), + None, + ) + }, + Ok(DataType::Timestamp(_, None)) => |col_values: &[ColumnarValue]| { + cast_column( + &col_values[0], + &DataType::Timestamp(TimeUnit::Nanosecond, None), + None, + ) + }, Ok(DataType::Utf8) => datetime_expressions::to_timestamp, other => { return internal_err!( @@ -129,6 +134,25 @@ pub fn create_physical_expr( } }) } + BuiltinScalarFunction::ToTimestampNanos => { + Arc::new(match input_phy_exprs[0].data_type(input_schema) { + Ok(DataType::Int64) | Ok(DataType::Timestamp(_, None)) => { + |col_values: &[ColumnarValue]| { + cast_column( + &col_values[0], + &DataType::Timestamp(TimeUnit::Nanosecond, None), + None, + ) + } + } + Ok(DataType::Utf8) => datetime_expressions::to_timestamp_nanos, + other => { + return internal_err!( + "Unsupported data type {other:?} for function to_timestamp_nanos" + ); + } + }) + } BuiltinScalarFunction::ToTimestampSeconds => Arc::new({ match input_phy_exprs[0].data_type(input_schema) { Ok(DataType::Int64) | Ok(DataType::Timestamp(_, None)) => { diff --git a/datafusion/proto/proto/datafusion.proto b/datafusion/proto/proto/datafusion.proto index f4ab582593e0..9b6a0448f810 100644 --- a/datafusion/proto/proto/datafusion.proto +++ b/datafusion/proto/proto/datafusion.proto @@ -620,6 +620,7 @@ enum ScalarFunction { ArrayEmpty = 115; ArrayPopBack = 116; StringToArray = 117; + ToTimestampNanos = 118; } message ScalarFunctionNode { diff --git a/datafusion/proto/src/generated/pbjson.rs b/datafusion/proto/src/generated/pbjson.rs index e9e2fd0c0461..3eeb060f8d01 100644 --- a/datafusion/proto/src/generated/pbjson.rs +++ b/datafusion/proto/src/generated/pbjson.rs @@ -19772,6 +19772,7 @@ impl serde::Serialize for ScalarFunction { Self::ArrayEmpty => "ArrayEmpty", Self::ArrayPopBack => "ArrayPopBack", Self::StringToArray => "StringToArray", + Self::ToTimestampNanos => "ToTimestampNanos", }; serializer.serialize_str(variant) } @@ -19901,6 +19902,7 @@ impl<'de> serde::Deserialize<'de> for ScalarFunction { "ArrayEmpty", "ArrayPopBack", "StringToArray", + "ToTimestampNanos", ]; struct GeneratedVisitor; @@ -20059,6 +20061,7 @@ impl<'de> serde::Deserialize<'de> for ScalarFunction { "ArrayEmpty" => Ok(ScalarFunction::ArrayEmpty), "ArrayPopBack" => Ok(ScalarFunction::ArrayPopBack), "StringToArray" => Ok(ScalarFunction::StringToArray), + "ToTimestampNanos" => Ok(ScalarFunction::ToTimestampNanos), _ => Err(serde::de::Error::unknown_variant(value, FIELDS)), } } diff --git a/datafusion/proto/src/generated/prost.rs b/datafusion/proto/src/generated/prost.rs index 1c821708a971..d18bacfb3bcc 100644 --- a/datafusion/proto/src/generated/prost.rs +++ b/datafusion/proto/src/generated/prost.rs @@ -2465,6 +2465,7 @@ pub enum ScalarFunction { ArrayEmpty = 115, ArrayPopBack = 116, StringToArray = 117, + ToTimestampNanos = 118, } impl ScalarFunction { /// String value of the enum field names used in the ProtoBuf definition. @@ -2591,6 +2592,7 @@ impl ScalarFunction { ScalarFunction::ArrayEmpty => "ArrayEmpty", ScalarFunction::ArrayPopBack => "ArrayPopBack", ScalarFunction::StringToArray => "StringToArray", + ScalarFunction::ToTimestampNanos => "ToTimestampNanos", } } /// Creates an enum from field names used in the ProtoBuf definition. @@ -2714,6 +2716,7 @@ impl ScalarFunction { "ArrayEmpty" => Some(Self::ArrayEmpty), "ArrayPopBack" => Some(Self::ArrayPopBack), "StringToArray" => Some(Self::StringToArray), + "ToTimestampNanos" => Some(Self::ToTimestampNanos), _ => None, } } diff --git a/datafusion/proto/src/logical_plan/from_proto.rs b/datafusion/proto/src/logical_plan/from_proto.rs index c87882ca72fc..26bd0163d0a3 100644 --- a/datafusion/proto/src/logical_plan/from_proto.rs +++ b/datafusion/proto/src/logical_plan/from_proto.rs @@ -54,7 +54,8 @@ use datafusion_expr::{ random, regexp_match, regexp_replace, repeat, replace, reverse, right, round, rpad, rtrim, sha224, sha256, sha384, sha512, signum, sin, sinh, split_part, sqrt, starts_with, strpos, substr, substring, tan, tanh, to_hex, to_timestamp_micros, - to_timestamp_millis, to_timestamp_seconds, translate, trim, trunc, upper, uuid, + to_timestamp_millis, to_timestamp_nanos, to_timestamp_seconds, translate, trim, + trunc, upper, uuid, window_frame::regularize, AggregateFunction, Between, BinaryExpr, BuiltInWindowFunction, BuiltinScalarFunction, Case, Cast, Expr, GetFieldAccess, GetIndexedField, GroupingSet, @@ -521,6 +522,7 @@ impl From<&protobuf::ScalarFunction> for BuiltinScalarFunction { ScalarFunction::Substr => Self::Substr, ScalarFunction::ToHex => Self::ToHex, ScalarFunction::ToTimestampMicros => Self::ToTimestampMicros, + ScalarFunction::ToTimestampNanos => Self::ToTimestampNanos, ScalarFunction::ToTimestampSeconds => Self::ToTimestampSeconds, ScalarFunction::Now => Self::Now, ScalarFunction::CurrentDate => Self::CurrentDate, @@ -1592,6 +1594,9 @@ pub fn parse_expr( ScalarFunction::ToTimestampMicros => { Ok(to_timestamp_micros(parse_expr(&args[0], registry)?)) } + ScalarFunction::ToTimestampNanos => { + Ok(to_timestamp_nanos(parse_expr(&args[0], registry)?)) + } ScalarFunction::ToTimestampSeconds => { Ok(to_timestamp_seconds(parse_expr(&args[0], registry)?)) } diff --git a/datafusion/proto/src/logical_plan/to_proto.rs b/datafusion/proto/src/logical_plan/to_proto.rs index 125ced032e20..687b73cfc886 100644 --- a/datafusion/proto/src/logical_plan/to_proto.rs +++ b/datafusion/proto/src/logical_plan/to_proto.rs @@ -1522,6 +1522,7 @@ impl TryFrom<&BuiltinScalarFunction> for protobuf::ScalarFunction { BuiltinScalarFunction::Substr => Self::Substr, BuiltinScalarFunction::ToHex => Self::ToHex, BuiltinScalarFunction::ToTimestampMicros => Self::ToTimestampMicros, + BuiltinScalarFunction::ToTimestampNanos => Self::ToTimestampNanos, BuiltinScalarFunction::ToTimestampSeconds => Self::ToTimestampSeconds, BuiltinScalarFunction::Now => Self::Now, BuiltinScalarFunction::CurrentDate => Self::CurrentDate, diff --git a/datafusion/sqllogictest/test_files/timestamps.slt b/datafusion/sqllogictest/test_files/timestamps.slt index fea61b076ebc..e186aa12f7a9 100644 --- a/datafusion/sqllogictest/test_files/timestamps.slt +++ b/datafusion/sqllogictest/test_files/timestamps.slt @@ -217,7 +217,7 @@ SELECT to_timestamp_micros(ts) FROM ts_data_secs LIMIT 3 # to nanos query P -SELECT to_timestamp(ts) FROM ts_data_secs LIMIT 3 +SELECT to_timestamp_nanos(ts) FROM ts_data_secs LIMIT 3 ---- 2020-09-08T13:42:29 2020-09-08T12:42:29 @@ -244,7 +244,7 @@ SELECT to_timestamp_seconds(ts) FROM ts_data_micros LIMIT 3 2020-09-08T11:42:29 -# Original column is micros, convert to nanos and check timestamp +# Original column is micros, convert to seconds and check timestamp query P SELECT to_timestamp(ts) FROM ts_data_micros LIMIT 3 @@ -266,7 +266,7 @@ SELECT from_unixtime(ts / 1000000000) FROM ts_data LIMIT 3; # to_timestamp query I -SELECT COUNT(*) FROM ts_data_nanos where ts > to_timestamp('2020-09-08T12:00:00+00:00') +SELECT COUNT(*) FROM ts_data_nanos where ts > timestamp '2020-09-08T12:00:00+00:00' ---- 2 @@ -375,7 +375,7 @@ set datafusion.optimizer.skip_failed_rules = true query P select to_timestamp(a) from (select to_timestamp(1) as a) A; ---- -1970-01-01T00:00:00.000000001 +1970-01-01T00:00:01 # cast_to_timestamp_seconds_twice query P @@ -383,7 +383,6 @@ select to_timestamp_seconds(a) from (select to_timestamp_seconds(1) as a)A ---- 1970-01-01T00:00:01 - # cast_to_timestamp_millis_twice query P select to_timestamp_millis(a) from (select to_timestamp_millis(1) as a)A; @@ -396,11 +395,17 @@ select to_timestamp_micros(a) from (select to_timestamp_micros(1) as a)A; ---- 1970-01-01T00:00:00.000001 +# cast_to_timestamp_nanos_twice +query P +select to_timestamp_nanos(a) from (select to_timestamp_nanos(1) as a)A; +---- +1970-01-01T00:00:00.000000001 + # to_timestamp_i32 query P select to_timestamp(cast (1 as int)); ---- -1970-01-01T00:00:00.000000001 +1970-01-01T00:00:01 # to_timestamp_micros_i32 query P @@ -408,6 +413,12 @@ select to_timestamp_micros(cast (1 as int)); ---- 1970-01-01T00:00:00.000001 +# to_timestamp_nanos_i32 +query P +select to_timestamp_nanos(cast (1 as int)); +---- +1970-01-01T00:00:00.000000001 + # to_timestamp_millis_i32 query P select to_timestamp_millis(cast (1 as int)); @@ -1776,3 +1787,9 @@ query B SELECT TIMESTAMPTZ '2020-01-01 00:00:00Z' = TIMESTAMP '2020-01-01' ---- true + +# verify to_timestamp edge cases to be in sync with postgresql +query PPPPP +SELECT to_timestamp(null), to_timestamp(-62125747200), to_timestamp(0), to_timestamp(1926632005177), to_timestamp(1926632005) +---- +NULL 0001-04-25T00:00:00 1970-01-01T00:00:00 +63022-07-16T12:59:37 2031-01-19T23:33:25 diff --git a/docs/source/user-guide/sql/scalar_functions.md b/docs/source/user-guide/sql/scalar_functions.md index d5717b9c2130..b7426baea3da 100644 --- a/docs/source/user-guide/sql/scalar_functions.md +++ b/docs/source/user-guide/sql/scalar_functions.md @@ -1218,6 +1218,7 @@ regexp_replace(str, regexp, replacement, flags) - [to_timestamp_millis](#to_timestamp_millis) - [to_timestamp_micros](#to_timestamp_micros) - [to_timestamp_seconds](#to_timestamp_seconds) +- [to_timestamp_nanos](#to_timestamp_nanos) - [from_unixtime](#from_unixtime) ### `now` @@ -1390,10 +1391,10 @@ extract(field FROM source) ### `to_timestamp` -Converts a value to RFC3339 nanosecond timestamp format (`YYYY-MM-DDT00:00:00.000000000Z`). +Converts a value to RFC3339 nanosecond timestamp format (`YYYY-MM-DDT00:00:00Z`). Supports timestamp, integer, and unsigned integer types as input. -Integers and unsigned integers are parsed as Unix nanosecond timestamps and -return the corresponding RFC3339 nanosecond timestamp. +Integers and unsigned integers are parsed as Unix second timestamps and +return the corresponding RFC3339 timestamp. ``` to_timestamp(expression) @@ -1428,7 +1429,18 @@ Integers and unsigned integers are parsed as Unix nanosecond timestamps and return the corresponding RFC3339 timestamp. ``` -to_timestamp_micros(expression) +to_timestamp_nanos(expression) +``` + +### `to_timestamp_nanos` + +Converts a value to RFC3339 nanosecond timestamp format (`YYYY-MM-DDT00:00:00.000000000Z`). +Supports timestamp, integer, and unsigned integer types as input. +Integers and unsigned integers are parsed as Unix nanosecond timestamps and +return the corresponding RFC3339 timestamp. + +``` +to_timestamp_nanos(expression) ``` #### Arguments From 6e87f5958fe0f1f562bac49bacee8456e747de25 Mon Sep 17 00:00:00 2001 From: Andrew Lamb Date: Thu, 26 Oct 2023 10:31:01 -0400 Subject: [PATCH 136/572] Minor: Document `parquet` crate feature (#7927) --- README.md | 2 ++ 1 file changed, 2 insertions(+) diff --git a/README.md b/README.md index 2f10812f9a59..1997a6f73dd5 100644 --- a/README.md +++ b/README.md @@ -47,6 +47,7 @@ Default features: - `compression`: reading files compressed with `xz2`, `bzip2`, `flate2`, and `zstd` - `crypto_expressions`: cryptographic functions such as `md5` and `sha256` - `encoding_expressions`: `encode` and `decode` functions +- `parquet`: support for reading the [Apache Parquet] format - `regex_expressions`: regular expression functions, such as `regexp_match` - `unicode_expressions`: Include unicode aware functions such as `character_length` @@ -59,6 +60,7 @@ Optional features: - `simd`: enable arrow-rs's manual `SIMD` kernels (requires Rust `nightly`) [apache avro]: https://avro.apache.org/ +[apache parquet]: https://parquet.apache.org/ ## Rust Version Compatibility From a892300a5a56c97b5b4ddc9aa4a421aaf412d0fe Mon Sep 17 00:00:00 2001 From: Andrew Lamb Date: Thu, 26 Oct 2023 10:32:13 -0400 Subject: [PATCH 137/572] Minor: reduce some #cfg(feature = "parquet") (#7929) --- datafusion/core/src/datasource/file_format/parquet.rs | 4 +--- datafusion/core/src/datasource/mod.rs | 2 -- datafusion/core/src/datasource/physical_plan/mod.rs | 2 -- datafusion/core/src/execution/context/parquet.rs | 3 +-- 4 files changed, 2 insertions(+), 9 deletions(-) diff --git a/datafusion/core/src/datasource/file_format/parquet.rs b/datafusion/core/src/datasource/file_format/parquet.rs index 62867c0e2b38..2cba474e559e 100644 --- a/datafusion/core/src/datasource/file_format/parquet.rs +++ b/datafusion/core/src/datasource/file_format/parquet.rs @@ -36,7 +36,7 @@ use tokio::sync::mpsc::{self, Receiver, Sender}; use tokio::task::{JoinHandle, JoinSet}; use crate::datasource::file_format::file_compression_type::FileCompressionType; -use crate::datasource::statistics::create_max_min_accs; +use crate::datasource::statistics::{create_max_min_accs, get_col_stats}; use arrow::datatypes::SchemaRef; use arrow::datatypes::{Fields, Schema}; use bytes::{BufMut, BytesMut}; @@ -63,11 +63,9 @@ use crate::arrow::array::{ use crate::arrow::datatypes::DataType; use crate::config::ConfigOptions; -use crate::datasource::get_col_stats; use crate::datasource::physical_plan::{ FileGroupDisplay, FileMeta, FileSinkConfig, ParquetExec, SchemaAdapter, }; - use crate::error::Result; use crate::execution::context::SessionState; use crate::physical_plan::expressions::{MaxAccumulator, MinAccumulator}; diff --git a/datafusion/core/src/datasource/mod.rs b/datafusion/core/src/datasource/mod.rs index 3ace2c239852..48e9d6992124 100644 --- a/datafusion/core/src/datasource/mod.rs +++ b/datafusion/core/src/datasource/mod.rs @@ -42,6 +42,4 @@ pub use self::memory::MemTable; pub use self::provider::TableProvider; pub use self::view::ViewTable; pub use crate::logical_expr::TableType; -#[cfg(feature = "parquet")] -pub(crate) use statistics::get_col_stats; pub use statistics::get_statistics_with_limit; diff --git a/datafusion/core/src/datasource/physical_plan/mod.rs b/datafusion/core/src/datasource/physical_plan/mod.rs index 3f84f87eb5d5..6643e4127dbd 100644 --- a/datafusion/core/src/datasource/physical_plan/mod.rs +++ b/datafusion/core/src/datasource/physical_plan/mod.rs @@ -30,8 +30,6 @@ pub(crate) use self::csv::plan_to_csv; pub use self::csv::{CsvConfig, CsvExec, CsvOpener}; pub(crate) use self::json::plan_to_json; #[cfg(feature = "parquet")] -pub(crate) use self::parquet::plan_to_parquet; -#[cfg(feature = "parquet")] pub use self::parquet::{ParquetExec, ParquetFileMetrics, ParquetFileReaderFactory}; pub use arrow_file::ArrowExec; diff --git a/datafusion/core/src/execution/context/parquet.rs b/datafusion/core/src/execution/context/parquet.rs index b02576c6a868..dc202b9903f5 100644 --- a/datafusion/core/src/execution/context/parquet.rs +++ b/datafusion/core/src/execution/context/parquet.rs @@ -17,10 +17,9 @@ use std::sync::Arc; +use crate::datasource::physical_plan::parquet::plan_to_parquet; use parquet::file::properties::WriterProperties; -use crate::datasource::physical_plan::plan_to_parquet; - use super::super::options::{ParquetReadOptions, ReadOptions}; use super::{DataFilePaths, DataFrame, ExecutionPlan, Result, SessionContext}; From ae9a446c41dfa515eb454561c160ad9aa26a7117 Mon Sep 17 00:00:00 2001 From: Andrew Lamb Date: Thu, 26 Oct 2023 10:33:09 -0400 Subject: [PATCH 138/572] Minor: reduce use of cfg(parquet) in tests (#7930) --- .../enforce_distribution.rs | 56 +------------------ 1 file changed, 3 insertions(+), 53 deletions(-) diff --git a/datafusion/core/src/physical_optimizer/enforce_distribution.rs b/datafusion/core/src/physical_optimizer/enforce_distribution.rs index 072c3cb6d7a6..7b91dce32aa9 100644 --- a/datafusion/core/src/physical_optimizer/enforce_distribution.rs +++ b/datafusion/core/src/physical_optimizer/enforce_distribution.rs @@ -1634,6 +1634,8 @@ impl TreeNode for PlanWithKeyRequirements { } } +/// Since almost all of these tests explicitly use `ParquetExec` they only run with the parquet feature flag on +#[cfg(feature = "parquet")] #[cfg(test)] mod tests { use std::ops::Deref; @@ -1643,7 +1645,6 @@ mod tests { use crate::datasource::listing::PartitionedFile; use crate::datasource::object_store::ObjectStoreUrl; use crate::datasource::physical_plan::FileScanConfig; - #[cfg(feature = "parquet")] use crate::datasource::physical_plan::ParquetExec; use crate::physical_optimizer::enforce_sorting::EnforceSorting; use crate::physical_optimizer::output_requirements::OutputRequirements; @@ -1783,12 +1784,10 @@ mod tests { ])) } - #[cfg(feature = "parquet")] fn parquet_exec() -> Arc { parquet_exec_with_sort(vec![]) } - #[cfg(feature = "parquet")] fn parquet_exec_with_sort( output_ordering: Vec>, ) -> Arc { @@ -1809,13 +1808,11 @@ mod tests { )) } - #[cfg(feature = "parquet")] fn parquet_exec_multiple() -> Arc { parquet_exec_multiple_sorted(vec![]) } // Created a sorted parquet exec with multiple files - #[cfg(feature = "parquet")] fn parquet_exec_multiple_sorted( output_ordering: Vec>, ) -> Arc { @@ -2170,7 +2167,6 @@ mod tests { } #[test] - #[cfg(feature = "parquet")] fn multi_hash_joins() -> Result<()> { let left = parquet_exec(); let alias_pairs: Vec<(String, String)> = vec![ @@ -2333,7 +2329,6 @@ mod tests { } #[test] - #[cfg(feature = "parquet")] fn multi_joins_after_alias() -> Result<()> { let left = parquet_exec(); let right = parquet_exec(); @@ -2413,7 +2408,6 @@ mod tests { } #[test] - #[cfg(feature = "parquet")] fn multi_joins_after_multi_alias() -> Result<()> { let left = parquet_exec(); let right = parquet_exec(); @@ -2469,7 +2463,6 @@ mod tests { } #[test] - #[cfg(feature = "parquet")] fn join_after_agg_alias() -> Result<()> { // group by (a as a1) let left = aggregate_exec_with_alias( @@ -2509,7 +2502,6 @@ mod tests { } #[test] - #[cfg(feature = "parquet")] fn hash_join_key_ordering() -> Result<()> { // group by (a as a1, b as b1) let left = aggregate_exec_with_alias( @@ -2562,7 +2554,6 @@ mod tests { } #[test] - #[cfg(feature = "parquet")] fn multi_hash_join_key_ordering() -> Result<()> { let left = parquet_exec(); let alias_pairs: Vec<(String, String)> = vec![ @@ -2679,7 +2670,6 @@ mod tests { } #[test] - #[cfg(feature = "parquet")] fn reorder_join_keys_to_left_input() -> Result<()> { let left = parquet_exec(); let alias_pairs: Vec<(String, String)> = vec![ @@ -2810,7 +2800,6 @@ mod tests { } #[test] - #[cfg(feature = "parquet")] fn reorder_join_keys_to_right_input() -> Result<()> { let left = parquet_exec(); let alias_pairs: Vec<(String, String)> = vec![ @@ -2936,7 +2925,6 @@ mod tests { } #[test] - #[cfg(feature = "parquet")] fn multi_smj_joins() -> Result<()> { let left = parquet_exec(); let alias_pairs: Vec<(String, String)> = vec![ @@ -3210,7 +3198,6 @@ mod tests { } #[test] - #[cfg(feature = "parquet")] fn smj_join_key_ordering() -> Result<()> { // group by (a as a1, b as b1) let left = aggregate_exec_with_alias( @@ -3306,7 +3293,6 @@ mod tests { } #[test] - #[cfg(feature = "parquet")] fn merge_does_not_need_sort() -> Result<()> { // see https://github.com/apache/arrow-datafusion/issues/4331 let schema = schema(); @@ -3347,7 +3333,6 @@ mod tests { } #[test] - #[cfg(feature = "parquet")] fn union_to_interleave() -> Result<()> { // group by (a as a1) let left = aggregate_exec_with_alias( @@ -3389,7 +3374,6 @@ mod tests { } #[test] - #[cfg(feature = "parquet")] fn added_repartition_to_single_partition() -> Result<()> { let alias = vec![("a".to_string(), "a".to_string())]; let plan = aggregate_exec_with_alias(parquet_exec(), alias); @@ -3408,7 +3392,6 @@ mod tests { } #[test] - #[cfg(feature = "parquet")] fn repartition_deepest_node() -> Result<()> { let alias = vec![("a".to_string(), "a".to_string())]; let plan = aggregate_exec_with_alias(filter_exec(parquet_exec()), alias); @@ -3428,7 +3411,7 @@ mod tests { } #[test] - #[cfg(feature = "parquet")] + fn repartition_unsorted_limit() -> Result<()> { let plan = limit_exec(filter_exec(parquet_exec())); @@ -3448,7 +3431,6 @@ mod tests { } #[test] - #[cfg(feature = "parquet")] fn repartition_sorted_limit() -> Result<()> { let schema = schema(); let sort_key = vec![PhysicalSortExpr { @@ -3471,7 +3453,6 @@ mod tests { } #[test] - #[cfg(feature = "parquet")] fn repartition_sorted_limit_with_filter() -> Result<()> { let schema = schema(); let sort_key = vec![PhysicalSortExpr { @@ -3497,7 +3478,6 @@ mod tests { } #[test] - #[cfg(feature = "parquet")] fn repartition_ignores_limit() -> Result<()> { let alias = vec![("a".to_string(), "a".to_string())]; let plan = aggregate_exec_with_alias( @@ -3528,7 +3508,6 @@ mod tests { } #[test] - #[cfg(feature = "parquet")] fn repartition_ignores_union() -> Result<()> { let plan = union_exec(vec![parquet_exec(); 5]); @@ -3548,7 +3527,6 @@ mod tests { } #[test] - #[cfg(feature = "parquet")] fn repartition_through_sort_preserving_merge() -> Result<()> { // sort preserving merge with non-sorted input let schema = schema(); @@ -3571,7 +3549,6 @@ mod tests { } #[test] - #[cfg(feature = "parquet")] fn repartition_ignores_sort_preserving_merge() -> Result<()> { // sort preserving merge already sorted input, let schema = schema(); @@ -3603,7 +3580,6 @@ mod tests { } #[test] - #[cfg(feature = "parquet")] fn repartition_ignores_sort_preserving_merge_with_union() -> Result<()> { // 2 sorted parquet files unioned (partitions are concatenated, sort is preserved) let schema = schema(); @@ -3636,7 +3612,6 @@ mod tests { } #[test] - #[cfg(feature = "parquet")] fn repartition_does_not_destroy_sort() -> Result<()> { // SortRequired // Parquet(sorted) @@ -3662,7 +3637,6 @@ mod tests { } #[test] - #[cfg(feature = "parquet")] fn repartition_does_not_destroy_sort_more_complex() -> Result<()> { // model a more complicated scenario where one child of a union can be repartitioned for performance // but the other can not be @@ -3701,7 +3675,6 @@ mod tests { } #[test] - #[cfg(feature = "parquet")] fn repartition_transitively_with_projection() -> Result<()> { let schema = schema(); let proj_exprs = vec![( @@ -3744,7 +3717,6 @@ mod tests { } #[test] - #[cfg(feature = "parquet")] fn repartition_ignores_transitively_with_projection() -> Result<()> { let schema = schema(); let sort_key = vec![PhysicalSortExpr { @@ -3775,7 +3747,6 @@ mod tests { } #[test] - #[cfg(feature = "parquet")] fn repartition_transitively_past_sort_with_projection() -> Result<()> { let schema = schema(); let sort_key = vec![PhysicalSortExpr { @@ -3805,7 +3776,6 @@ mod tests { } #[test] - #[cfg(feature = "parquet")] fn repartition_transitively_past_sort_with_filter() -> Result<()> { let schema = schema(); let sort_key = vec![PhysicalSortExpr { @@ -3880,7 +3850,6 @@ mod tests { } #[test] - #[cfg(feature = "parquet")] fn parallelization_single_partition() -> Result<()> { let alias = vec![("a".to_string(), "a".to_string())]; let plan_parquet = aggregate_exec_with_alias(parquet_exec(), alias.clone()); @@ -3969,7 +3938,6 @@ mod tests { } #[test] - #[cfg(feature = "parquet")] fn parallelization_two_partitions() -> Result<()> { let alias = vec![("a".to_string(), "a".to_string())]; let plan_parquet = @@ -3997,7 +3965,6 @@ mod tests { } #[test] - #[cfg(feature = "parquet")] fn parallelization_two_partitions_into_four() -> Result<()> { let alias = vec![("a".to_string(), "a".to_string())]; let plan_parquet = @@ -4025,7 +3992,6 @@ mod tests { } #[test] - #[cfg(feature = "parquet")] fn parallelization_sorted_limit() -> Result<()> { let schema = schema(); let sort_key = vec![PhysicalSortExpr { @@ -4058,7 +4024,6 @@ mod tests { } #[test] - #[cfg(feature = "parquet")] fn parallelization_limit_with_filter() -> Result<()> { let schema = schema(); let sort_key = vec![PhysicalSortExpr { @@ -4104,7 +4069,6 @@ mod tests { } #[test] - #[cfg(feature = "parquet")] fn parallelization_ignores_limit() -> Result<()> { let alias = vec![("a".to_string(), "a".to_string())]; let plan_parquet = aggregate_exec_with_alias( @@ -4155,7 +4119,6 @@ mod tests { } #[test] - #[cfg(feature = "parquet")] fn parallelization_union_inputs() -> Result<()> { let plan_parquet = union_exec(vec![parquet_exec(); 5]); let plan_csv = union_exec(vec![csv_exec(); 5]); @@ -4185,7 +4148,6 @@ mod tests { } #[test] - #[cfg(feature = "parquet")] fn parallelization_prior_to_sort_preserving_merge() -> Result<()> { let schema = schema(); let sort_key = vec![PhysicalSortExpr { @@ -4216,7 +4178,6 @@ mod tests { } #[test] - #[cfg(feature = "parquet")] fn parallelization_sort_preserving_merge_with_union() -> Result<()> { let schema = schema(); let sort_key = vec![PhysicalSortExpr { @@ -4251,7 +4212,6 @@ mod tests { } #[test] - #[cfg(feature = "parquet")] fn parallelization_does_not_benefit() -> Result<()> { let schema = schema(); let sort_key = vec![PhysicalSortExpr { @@ -4280,7 +4240,6 @@ mod tests { } #[test] - #[cfg(feature = "parquet")] fn parallelization_ignores_transitively_with_projection_parquet() -> Result<()> { // sorted input let schema = schema(); @@ -4361,7 +4320,6 @@ mod tests { } #[test] - #[cfg(feature = "parquet")] fn remove_redundant_roundrobins() -> Result<()> { let input = parquet_exec(); let repartition = repartition_exec(repartition_exec(input)); @@ -4412,7 +4370,6 @@ mod tests { } #[test] - #[cfg(feature = "parquet")] fn do_not_preserve_ordering_through_repartition() -> Result<()> { let schema = schema(); let sort_key = vec![PhysicalSortExpr { @@ -4445,7 +4402,6 @@ mod tests { } #[test] - #[cfg(feature = "parquet")] fn do_not_preserve_ordering_through_repartition2() -> Result<()> { let schema = schema(); let sort_key = vec![PhysicalSortExpr { @@ -4484,7 +4440,6 @@ mod tests { } #[test] - #[cfg(feature = "parquet")] fn do_not_preserve_ordering_through_repartition3() -> Result<()> { let schema = schema(); let sort_key = vec![PhysicalSortExpr { @@ -4507,7 +4462,6 @@ mod tests { } #[test] - #[cfg(feature = "parquet")] fn do_not_put_sort_when_input_is_invalid() -> Result<()> { let schema = schema(); let sort_key = vec![PhysicalSortExpr { @@ -4546,7 +4500,6 @@ mod tests { } #[test] - #[cfg(feature = "parquet")] fn put_sort_when_input_is_valid() -> Result<()> { let schema = schema(); let sort_key = vec![PhysicalSortExpr { @@ -4589,7 +4542,6 @@ mod tests { } #[test] - #[cfg(feature = "parquet")] fn do_not_add_unnecessary_hash() -> Result<()> { let schema = schema(); let sort_key = vec![PhysicalSortExpr { @@ -4645,7 +4597,6 @@ mod tests { } #[test] - #[cfg(feature = "parquet")] fn optimize_away_unnecessary_repartition() -> Result<()> { let physical_plan = coalesce_partitions_exec(repartition_exec(parquet_exec())); let expected = &[ @@ -4665,7 +4616,6 @@ mod tests { } #[test] - #[cfg(feature = "parquet")] fn optimize_away_unnecessary_repartition2() -> Result<()> { let physical_plan = filter_exec(repartition_exec(coalesce_partitions_exec( filter_exec(repartition_exec(parquet_exec())), From 30e5f4286dffaed3533a5dc6348dd91e98cf461a Mon Sep 17 00:00:00 2001 From: comphead Date: Thu, 26 Oct 2023 12:11:25 -0700 Subject: [PATCH 139/572] Fix CI failures on `to_timestamp()` calls (#7941) * Change input for `to_timestamp` function * docs * fix examples * output `to_timestamp` signature as ns * Fix CI `to_timestamp()` failed * Update datafusion/expr/src/built_in_function.rs Co-authored-by: Andrew Lamb * fix typo * fix --------- Co-authored-by: Andrew Lamb --- datafusion/expr/src/built_in_function.rs | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/datafusion/expr/src/built_in_function.rs b/datafusion/expr/src/built_in_function.rs index 9edee7649f67..4db565abfcf7 100644 --- a/datafusion/expr/src/built_in_function.rs +++ b/datafusion/expr/src/built_in_function.rs @@ -755,7 +755,10 @@ impl BuiltinScalarFunction { return plan_err!("The to_hex function can only accept integers."); } }), - BuiltinScalarFunction::ToTimestamp => Ok(Timestamp(Nanosecond, None)), + BuiltinScalarFunction::ToTimestamp => Ok(match &input_expr_types[0] { + Int64 => Timestamp(Second, None), + _ => Timestamp(Nanosecond, None), + }), BuiltinScalarFunction::ToTimestampMillis => Ok(Timestamp(Millisecond, None)), BuiltinScalarFunction::ToTimestampMicros => Ok(Timestamp(Microsecond, None)), BuiltinScalarFunction::ToTimestampNanos => Ok(Timestamp(Nanosecond, None)), From a9d66e2b492843c2fb335a7dfe27fed073629b09 Mon Sep 17 00:00:00 2001 From: Jonah Gao Date: Fri, 27 Oct 2023 03:14:19 +0800 Subject: [PATCH 140/572] minor: add a datatype casting for the updated value (#7922) * minor: cast the updated value to the data type of target column * Update datafusion/sqllogictest/test_files/update.slt Co-authored-by: Alex Huang * Update datafusion/sqllogictest/test_files/update.slt Co-authored-by: Alex Huang * Update datafusion/sqllogictest/test_files/update.slt Co-authored-by: Alex Huang * fix tests --------- Co-authored-by: Alex Huang --- datafusion/sql/src/statement.rs | 41 ++++++++---------- datafusion/sqllogictest/test_files/update.slt | 43 +++++++++++++++++++ 2 files changed, 61 insertions(+), 23 deletions(-) create mode 100644 datafusion/sqllogictest/test_files/update.slt diff --git a/datafusion/sql/src/statement.rs b/datafusion/sql/src/statement.rs index f8504a487a66..b5196c086638 100644 --- a/datafusion/sql/src/statement.rs +++ b/datafusion/sql/src/statement.rs @@ -31,9 +31,9 @@ use arrow_schema::DataType; use datafusion_common::file_options::StatementOptions; use datafusion_common::parsers::CompressionTypeVariant; use datafusion_common::{ - not_impl_err, plan_datafusion_err, plan_err, unqualified_field_not_found, Column, - Constraints, DFField, DFSchema, DFSchemaRef, DataFusionError, ExprSchema, - OwnedTableReference, Result, SchemaReference, TableReference, ToDFSchema, + not_impl_err, plan_datafusion_err, plan_err, unqualified_field_not_found, + Constraints, DFField, DFSchema, DFSchemaRef, DataFusionError, OwnedTableReference, + Result, SchemaReference, TableReference, ToDFSchema, }; use datafusion_expr::dml::{CopyOptions, CopyTo}; use datafusion_expr::expr::Placeholder; @@ -969,12 +969,6 @@ impl<'a, S: ContextProvider> SqlToRel<'a, S> { table_name.clone(), &arrow_schema, )?); - let values = table_schema.fields().iter().map(|f| { - ( - f.name().clone(), - ast::Expr::Identifier(ast::Ident::from(f.name().as_str())), - ) - }); // Overwrite with assignment expressions let mut planner_context = PlannerContext::new(); @@ -992,11 +986,15 @@ impl<'a, S: ContextProvider> SqlToRel<'a, S> { }) .collect::>>()?; - let values = values - .into_iter() - .map(|(k, v)| { - let val = assign_map.remove(&k).unwrap_or(v); - (k, val) + let values_and_types = table_schema + .fields() + .iter() + .map(|f| { + let col_name = f.name(); + let val = assign_map.remove(col_name).unwrap_or_else(|| { + ast::Expr::Identifier(ast::Ident::from(col_name.as_str())) + }); + (col_name, val, f.data_type()) }) .collect::>(); @@ -1026,25 +1024,22 @@ impl<'a, S: ContextProvider> SqlToRel<'a, S> { // Projection let mut exprs = vec![]; - for (col_name, expr) in values.into_iter() { + for (col_name, expr, dt) in values_and_types.into_iter() { let expr = self.sql_to_expr(expr, &table_schema, &mut planner_context)?; let expr = match expr { datafusion_expr::Expr::Placeholder(Placeholder { ref id, ref data_type, }) => match data_type { - None => { - let dt = table_schema.data_type(&Column::from_name(&col_name))?; - datafusion_expr::Expr::Placeholder(Placeholder::new( - id.clone(), - Some(dt.clone()), - )) - } + None => datafusion_expr::Expr::Placeholder(Placeholder::new( + id.clone(), + Some(dt.clone()), + )), Some(_) => expr, }, _ => expr, }; - let expr = expr.alias(col_name); + let expr = expr.cast_to(dt, source.schema())?.alias(col_name); exprs.push(expr); } let source = project(source, exprs)?; diff --git a/datafusion/sqllogictest/test_files/update.slt b/datafusion/sqllogictest/test_files/update.slt new file mode 100644 index 000000000000..4542a262390c --- /dev/null +++ b/datafusion/sqllogictest/test_files/update.slt @@ -0,0 +1,43 @@ +# 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. + +########## +## Update Tests +########## + +statement ok +create table t1(a int, b varchar, c double, d int); + +# Turn off the optimizer to make the logical plan closer to the initial one +statement ok +set datafusion.optimizer.max_passes = 0; + +query TT +explain update t1 set a=1, b=2, c=3.0, d=NULL; +---- +logical_plan +Dml: op=[Update] table=[t1] +--Projection: CAST(Int64(1) AS Int32) AS a, CAST(Int64(2) AS Utf8) AS b, Float64(3) AS c, CAST(NULL AS Int32) AS d +----TableScan: t1 + +query TT +explain update t1 set a=c+1, b=a, c=c+1.0, d=b; +---- +logical_plan +Dml: op=[Update] table=[t1] +--Projection: CAST(t1.c + CAST(Int64(1) AS Float64) AS Int32) AS a, CAST(t1.a AS Utf8) AS b, t1.c + Float64(1) AS c, CAST(t1.b AS Int32) AS d +----TableScan: t1 From 74fc6f8c8e113d4d23b11a8969c92c956629c5a6 Mon Sep 17 00:00:00 2001 From: Huaijin Date: Sat, 28 Oct 2023 05:06:38 +0800 Subject: [PATCH 141/572] fix (#7946) --- datafusion-examples/Cargo.toml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/datafusion-examples/Cargo.toml b/datafusion-examples/Cargo.toml index e5146c7fd94e..8d504f834bc5 100644 --- a/datafusion-examples/Cargo.toml +++ b/datafusion-examples/Cargo.toml @@ -36,7 +36,7 @@ arrow-schema = { workspace = true } async-trait = "0.1.41" bytes = "1.4" dashmap = "5.4" -datafusion = { path = "../datafusion/core" } +datafusion = { path = "../datafusion/core", features = ["avro"] } datafusion-common = { path = "../datafusion/common" } datafusion-expr = { path = "../datafusion/expr" } datafusion-optimizer = { path = "../datafusion/optimizer" } From 46ae9a4756aaa56440748c0c529fa312aca23e96 Mon Sep 17 00:00:00 2001 From: Jeffrey <22608443+Jefffrey@users.noreply.github.com> Date: Sat, 28 Oct 2023 20:40:13 +1100 Subject: [PATCH 142/572] Add simple exclude all columns test to sqllogictest (#7945) * Add simple exclude all columns test to sqllogictest * Add more exclude test cases --- datafusion/sqllogictest/test_files/select.slt | 20 +++++++++++++++++++ 1 file changed, 20 insertions(+) diff --git a/datafusion/sqllogictest/test_files/select.slt b/datafusion/sqllogictest/test_files/select.slt index 1d427479763a..98ea061c731b 100644 --- a/datafusion/sqllogictest/test_files/select.slt +++ b/datafusion/sqllogictest/test_files/select.slt @@ -848,6 +848,26 @@ statement error DataFusion error: Error during planning: EXCLUDE or EXCEPT conta SELECT * EXCLUDE(a, a) FROM table1 +# if EXCEPT all the columns, query should still succeed but return empty +statement ok +SELECT * EXCEPT(a, b, c, d) +FROM table1 + +# EXCLUDE order shouldn't matter +query II +SELECT * EXCLUDE(b, a) +FROM table1 +ORDER BY c +LIMIT 5 +---- +100 1000 +200 2000 + +# EXCLUDE with out of order but duplicate columns should error +statement error DataFusion error: Error during planning: EXCLUDE or EXCEPT contains duplicate column names +SELECT * EXCLUDE(d, b, c, a, a, b, c, d) +FROM table1 + # run below query in multi partitions statement ok set datafusion.execution.target_partitions = 2; From 250e71694b4f9789444e52b8cc12476dcbf35ac6 Mon Sep 17 00:00:00 2001 From: Devin D'Angelo Date: Sat, 28 Oct 2023 06:04:47 -0400 Subject: [PATCH 143/572] Support Partitioning Data by Dictionary Encoded String Array Types (#7896) * support dictionary encoded string columns for partition cols * remove debug prints * cargo fmt * generic dictionary cast and dict encoded test * updates from review * force retry checks * try checks again --- datafusion/common/src/dfschema.rs | 91 +++++++++++++++++++ .../src/datasource/file_format/write/demux.rs | 18 +++- .../core/src/datasource/listing/table.rs | 5 +- datafusion/core/src/datasource/memory.rs | 5 +- .../test_files/insert_to_external.slt | 38 +++++++- 5 files changed, 153 insertions(+), 4 deletions(-) diff --git a/datafusion/common/src/dfschema.rs b/datafusion/common/src/dfschema.rs index e16acbfedc81..d8cd103a4777 100644 --- a/datafusion/common/src/dfschema.rs +++ b/datafusion/common/src/dfschema.rs @@ -391,11 +391,33 @@ impl DFSchema { }) } + /// Returns true if the two schemas have the same qualified named + /// fields with logically equivalent data types. Returns false otherwise. + /// + /// Use [DFSchema]::equivalent_names_and_types for stricter semantic type + /// equivalence checking. + pub fn logically_equivalent_names_and_types(&self, other: &Self) -> bool { + if self.fields().len() != other.fields().len() { + return false; + } + let self_fields = self.fields().iter(); + let other_fields = other.fields().iter(); + self_fields.zip(other_fields).all(|(f1, f2)| { + f1.qualifier() == f2.qualifier() + && f1.name() == f2.name() + && Self::datatype_is_logically_equal(f1.data_type(), f2.data_type()) + }) + } + /// Returns true if the two schemas have the same qualified named /// fields with the same data types. Returns false otherwise. /// /// This is a specialized version of Eq that ignores differences /// in nullability and metadata. + /// + /// Use [DFSchema]::logically_equivalent_names_and_types for a weaker + /// logical type checking, which for example would consider a dictionary + /// encoded UTF8 array to be equivalent to a plain UTF8 array. pub fn equivalent_names_and_types(&self, other: &Self) -> bool { if self.fields().len() != other.fields().len() { return false; @@ -409,6 +431,46 @@ impl DFSchema { }) } + /// Checks if two [`DataType`]s are logically equal. This is a notably weaker constraint + /// than datatype_is_semantically_equal in that a Dictionary type is logically + /// equal to a plain V type, but not semantically equal. Dictionary is also + /// logically equal to Dictionary. + fn datatype_is_logically_equal(dt1: &DataType, dt2: &DataType) -> bool { + // check nested fields + match (dt1, dt2) { + (DataType::Dictionary(_, v1), DataType::Dictionary(_, v2)) => { + v1.as_ref() == v2.as_ref() + } + (DataType::Dictionary(_, v1), othertype) => v1.as_ref() == othertype, + (othertype, DataType::Dictionary(_, v1)) => v1.as_ref() == othertype, + (DataType::List(f1), DataType::List(f2)) + | (DataType::LargeList(f1), DataType::LargeList(f2)) + | (DataType::FixedSizeList(f1, _), DataType::FixedSizeList(f2, _)) + | (DataType::Map(f1, _), DataType::Map(f2, _)) => { + Self::field_is_logically_equal(f1, f2) + } + (DataType::Struct(fields1), DataType::Struct(fields2)) => { + let iter1 = fields1.iter(); + let iter2 = fields2.iter(); + fields1.len() == fields2.len() && + // all fields have to be the same + iter1 + .zip(iter2) + .all(|(f1, f2)| Self::field_is_logically_equal(f1, f2)) + } + (DataType::Union(fields1, _), DataType::Union(fields2, _)) => { + let iter1 = fields1.iter(); + let iter2 = fields2.iter(); + fields1.len() == fields2.len() && + // all fields have to be the same + iter1 + .zip(iter2) + .all(|((t1, f1), (t2, f2))| t1 == t2 && Self::field_is_logically_equal(f1, f2)) + } + _ => dt1 == dt2, + } + } + /// Returns true of two [`DataType`]s are semantically equal (same /// name and type), ignoring both metadata and nullability. /// @@ -456,6 +518,11 @@ impl DFSchema { } } + fn field_is_logically_equal(f1: &Field, f2: &Field) -> bool { + f1.name() == f2.name() + && Self::datatype_is_logically_equal(f1.data_type(), f2.data_type()) + } + fn field_is_semantically_equal(f1: &Field, f2: &Field) -> bool { f1.name() == f2.name() && Self::datatype_is_semantically_equal(f1.data_type(), f2.data_type()) @@ -786,6 +853,13 @@ pub trait SchemaExt { /// /// It works the same as [`DFSchema::equivalent_names_and_types`]. fn equivalent_names_and_types(&self, other: &Self) -> bool; + + /// Returns true if the two schemas have the same qualified named + /// fields with logically equivalent data types. Returns false otherwise. + /// + /// Use [DFSchema]::equivalent_names_and_types for stricter semantic type + /// equivalence checking. + fn logically_equivalent_names_and_types(&self, other: &Self) -> bool; } impl SchemaExt for Schema { @@ -805,6 +879,23 @@ impl SchemaExt for Schema { ) }) } + + fn logically_equivalent_names_and_types(&self, other: &Self) -> bool { + if self.fields().len() != other.fields().len() { + return false; + } + + self.fields() + .iter() + .zip(other.fields().iter()) + .all(|(f1, f2)| { + f1.name() == f2.name() + && DFSchema::datatype_is_logically_equal( + f1.data_type(), + f2.data_type(), + ) + }) + } } #[cfg(test)] diff --git a/datafusion/core/src/datasource/file_format/write/demux.rs b/datafusion/core/src/datasource/file_format/write/demux.rs index 67dd1f940676..27c65dd459ec 100644 --- a/datafusion/core/src/datasource/file_format/write/demux.rs +++ b/datafusion/core/src/datasource/file_format/write/demux.rs @@ -29,7 +29,7 @@ use crate::physical_plan::SendableRecordBatchStream; use arrow_array::builder::UInt64Builder; use arrow_array::cast::AsArray; -use arrow_array::{RecordBatch, StructArray}; +use arrow_array::{downcast_dictionary_array, RecordBatch, StringArray, StructArray}; use arrow_schema::{DataType, Schema}; use datafusion_common::cast::as_string_array; use datafusion_common::DataFusionError; @@ -338,6 +338,22 @@ fn compute_partition_keys_by_row<'a>( partition_values.push(array.value(i)); } } + DataType::Dictionary(_, _) => { + downcast_dictionary_array!( + col_array => { + let array = col_array.downcast_dict::() + .ok_or(DataFusionError::Execution(format!("it is not yet supported to write to hive partitions with datatype {}", + dtype)))?; + + for val in array.values() { + partition_values.push( + val.ok_or(DataFusionError::Execution(format!("Cannot partition by null value for column {}", col)))? + ); + } + }, + _ => unreachable!(), + ) + } _ => { return Err(DataFusionError::NotImplemented(format!( "it is not yet supported to write to hive partitions with datatype {}", diff --git a/datafusion/core/src/datasource/listing/table.rs b/datafusion/core/src/datasource/listing/table.rs index 822a78a5522a..d26d417bd8b2 100644 --- a/datafusion/core/src/datasource/listing/table.rs +++ b/datafusion/core/src/datasource/listing/table.rs @@ -824,7 +824,10 @@ impl TableProvider for ListingTable { overwrite: bool, ) -> Result> { // Check that the schema of the plan matches the schema of this table. - if !self.schema().equivalent_names_and_types(&input.schema()) { + if !self + .schema() + .logically_equivalent_names_and_types(&input.schema()) + { return plan_err!( // Return an error if schema of the input query does not match with the table schema. "Inserting query must have the same schema with the table." diff --git a/datafusion/core/src/datasource/memory.rs b/datafusion/core/src/datasource/memory.rs index a2f8e225e121..6bcaa97a408f 100644 --- a/datafusion/core/src/datasource/memory.rs +++ b/datafusion/core/src/datasource/memory.rs @@ -209,7 +209,10 @@ impl TableProvider for MemTable { ) -> Result> { // Create a physical plan from the logical plan. // Check that the schema of the plan matches the schema of this table. - if !self.schema().equivalent_names_and_types(&input.schema()) { + if !self + .schema() + .logically_equivalent_names_and_types(&input.schema()) + { return plan_err!( "Inserting query must have the same schema with the table." ); diff --git a/datafusion/sqllogictest/test_files/insert_to_external.slt b/datafusion/sqllogictest/test_files/insert_to_external.slt index b2206e987864..8b01a14568e7 100644 --- a/datafusion/sqllogictest/test_files/insert_to_external.slt +++ b/datafusion/sqllogictest/test_files/insert_to_external.slt @@ -40,8 +40,44 @@ STORED AS CSV WITH HEADER ROW LOCATION '../../testing/data/csv/aggregate_test_100.csv' -# test_insert_into +statement ok +create table dictionary_encoded_values as values +('a', arrow_cast('foo', 'Dictionary(Int32, Utf8)')), ('b', arrow_cast('bar', 'Dictionary(Int32, Utf8)')); + +query TTT +describe dictionary_encoded_values; +---- +column1 Utf8 YES +column2 Dictionary(Int32, Utf8) YES + +statement ok +CREATE EXTERNAL TABLE dictionary_encoded_parquet_partitioned( + a varchar, + b varchar, +) +STORED AS parquet +LOCATION 'test_files/scratch/insert_to_external/parquet_types_partitioned' +PARTITIONED BY (b) +OPTIONS( +create_local_path 'true', +insert_mode 'append_new_files', +); + +query TT +insert into dictionary_encoded_parquet_partitioned +select * from dictionary_encoded_values +---- +2 + +query TT +select * from dictionary_encoded_parquet_partitioned order by (a); +---- +a foo +b bar + + +# test_insert_into statement ok set datafusion.execution.target_partitions = 8; From d28c79d1809fd545bd4f3e15951267b7b1ec222c Mon Sep 17 00:00:00 2001 From: Jay Zhan Date: Sat, 28 Oct 2023 18:06:29 +0800 Subject: [PATCH 144/572] Minor: Remove array() in array_expression (#7961) * remove array Signed-off-by: jayzhan211 * cleanup others Signed-off-by: jayzhan211 * clippy Signed-off-by: jayzhan211 * cleanup cast Signed-off-by: jayzhan211 * fmt Signed-off-by: jayzhan211 * cleanup cast Signed-off-by: jayzhan211 --------- Signed-off-by: jayzhan211 --- .../physical-expr/src/array_expressions.rs | 391 ++++++++---------- 1 file changed, 172 insertions(+), 219 deletions(-) diff --git a/datafusion/physical-expr/src/array_expressions.rs b/datafusion/physical-expr/src/array_expressions.rs index af4612272676..7077f8b59860 100644 --- a/datafusion/physical-expr/src/array_expressions.rs +++ b/datafusion/physical-expr/src/array_expressions.rs @@ -25,12 +25,14 @@ use arrow::buffer::OffsetBuffer; use arrow::compute; use arrow::datatypes::{DataType, Field, UInt64Type}; use arrow_buffer::NullBuffer; -use datafusion_common::cast::{as_generic_string_array, as_int64_array, as_list_array}; + +use datafusion_common::cast::{ + as_generic_string_array, as_int64_array, as_list_array, as_string_array, +}; use datafusion_common::utils::wrap_into_list_array; use datafusion_common::{ exec_err, internal_err, not_impl_err, plan_err, DataFusionError, Result, }; -use datafusion_expr::ColumnarValue; use itertools::Itertools; @@ -395,21 +397,10 @@ fn array_array(args: &[ArrayRef], data_type: DataType) -> Result { Ok(res) } -/// Convert one or more [`ColumnarValue`] of the same type into a -/// `ListArray` -/// -/// See [`array_array`] for more details. -fn array(values: &[ColumnarValue]) -> Result { - let arrays: Vec = values - .iter() - .map(|x| match x { - ColumnarValue::Array(array) => array.clone(), - ColumnarValue::Scalar(scalar) => scalar.to_array(), - }) - .collect(); - +/// `make_array` SQL function +pub fn make_array(arrays: &[ArrayRef]) -> Result { let mut data_type = DataType::Null; - for arg in &arrays { + for arg in arrays { let arg_data_type = arg.data_type(); if !arg_data_type.equals_datatype(&DataType::Null) { data_type = arg_data_type.clone(); @@ -423,19 +414,10 @@ fn array(values: &[ColumnarValue]) -> Result { let array = new_null_array(&DataType::Null, arrays.len()); Ok(Arc::new(wrap_into_list_array(array))) } - data_type => array_array(arrays.as_slice(), data_type), + data_type => array_array(arrays, data_type), } } -/// `make_array` SQL function -pub fn make_array(arrays: &[ArrayRef]) -> Result { - let values: Vec = arrays - .iter() - .map(|x| ColumnarValue::Array(x.clone())) - .collect(); - array(values.as_slice()) -} - fn return_empty(return_null: bool, data_type: DataType) -> Arc { if return_null { new_null_array(&data_type, 1) @@ -654,7 +636,7 @@ pub fn array_append(args: &[ArrayRef]) -> Result { check_datatypes("array_append", &[arr.values(), element])?; let res = match arr.value_type() { DataType::List(_) => concat_internal(args)?, - DataType::Null => return array(&[ColumnarValue::Array(args[1].clone())]), + DataType::Null => return make_array(&[element.to_owned()]), data_type => { macro_rules! array_function { ($ARRAY_TYPE:ident) => { @@ -728,7 +710,7 @@ pub fn array_prepend(args: &[ArrayRef]) -> Result { check_datatypes("array_prepend", &[element, arr.values()])?; let res = match arr.value_type() { DataType::List(_) => concat_internal(args)?, - DataType::Null => return array(&[ColumnarValue::Array(args[0].clone())]), + DataType::Null => return make_array(&[element.to_owned()]), data_type => { macro_rules! array_function { ($ARRAY_TYPE:ident) => { @@ -1479,15 +1461,13 @@ macro_rules! to_string { pub fn array_to_string(args: &[ArrayRef]) -> Result { let arr = &args[0]; - let delimiters = as_generic_string_array::(&args[1])?; + let delimiters = as_string_array(&args[1])?; let delimiters: Vec> = delimiters.iter().collect(); let mut null_string = String::from(""); let mut with_null_string = false; if args.len() == 3 { - null_string = as_generic_string_array::(&args[2])? - .value(0) - .to_string(); + null_string = as_string_array(&args[2])?.value(0).to_string(); with_null_string = true; } @@ -1941,29 +1921,23 @@ pub fn string_to_array(args: &[ArrayRef]) -> Result() - .unwrap() + as_int64_array(&result.value(0)) + .expect("failed to cast to primitive array") .values() ) } @@ -1972,29 +1946,23 @@ mod tests { fn test_nested_array() { // make_array([1, 3, 5], [2, 4, 6]) = [[1, 3, 5], [2, 4, 6]] let args = [ - ColumnarValue::Array(Arc::new(Int64Array::from(vec![1, 2]))), - ColumnarValue::Array(Arc::new(Int64Array::from(vec![3, 4]))), - ColumnarValue::Array(Arc::new(Int64Array::from(vec![5, 6]))), + Arc::new(Int64Array::from(vec![1, 2])) as ArrayRef, + Arc::new(Int64Array::from(vec![3, 4])), + Arc::new(Int64Array::from(vec![5, 6])), ]; - let array = array(&args).expect("failed to initialize function array"); + let array = make_array(&args).expect("failed to initialize function array"); let result = as_list_array(&array).expect("failed to initialize function array"); assert_eq!(result.len(), 2); assert_eq!( &[1, 3, 5], - result - .value(0) - .as_any() - .downcast_ref::() - .unwrap() + as_int64_array(&result.value(0)) + .expect("failed to cast to primitive array") .values() ); assert_eq!( &[2, 4, 6], - result - .value(1) - .as_any() - .downcast_ref::() - .unwrap() + as_int64_array(&result.value(1)) + .expect("failed to cast to primitive array") .values() ); } @@ -2002,7 +1970,7 @@ mod tests { #[test] fn test_array_element() { // array_element([1, 2, 3, 4], 1) = 1 - let list_array = return_array().into_array(1); + let list_array = return_array(); let arr = array_element(&[list_array, Arc::new(Int64Array::from_value(1, 1))]) .expect("failed to initialize function array_element"); let result = @@ -2011,7 +1979,7 @@ mod tests { assert_eq!(result, &Int64Array::from_value(1, 1)); // array_element([1, 2, 3, 4], 3) = 3 - let list_array = return_array().into_array(1); + let list_array = return_array(); let arr = array_element(&[list_array, Arc::new(Int64Array::from_value(3, 1))]) .expect("failed to initialize function array_element"); let result = @@ -2020,7 +1988,7 @@ mod tests { assert_eq!(result, &Int64Array::from_value(3, 1)); // array_element([1, 2, 3, 4], 0) = NULL - let list_array = return_array().into_array(1); + let list_array = return_array(); let arr = array_element(&[list_array, Arc::new(Int64Array::from_value(0, 1))]) .expect("failed to initialize function array_element"); let result = @@ -2029,7 +1997,7 @@ mod tests { assert_eq!(result, &Int64Array::from(vec![None])); // array_element([1, 2, 3, 4], NULL) = NULL - let list_array = return_array().into_array(1); + let list_array = return_array(); let arr = array_element(&[list_array, Arc::new(Int64Array::from(vec![None]))]) .expect("failed to initialize function array_element"); let result = @@ -2038,7 +2006,7 @@ mod tests { assert_eq!(result, &Int64Array::from(vec![None])); // array_element([1, 2, 3, 4], -1) = 4 - let list_array = return_array().into_array(1); + let list_array = return_array(); let arr = array_element(&[list_array, Arc::new(Int64Array::from_value(-1, 1))]) .expect("failed to initialize function array_element"); let result = @@ -2047,7 +2015,7 @@ mod tests { assert_eq!(result, &Int64Array::from_value(4, 1)); // array_element([1, 2, 3, 4], -3) = 2 - let list_array = return_array().into_array(1); + let list_array = return_array(); let arr = array_element(&[list_array, Arc::new(Int64Array::from_value(-3, 1))]) .expect("failed to initialize function array_element"); let result = @@ -2056,7 +2024,7 @@ mod tests { assert_eq!(result, &Int64Array::from_value(2, 1)); // array_element([1, 2, 3, 4], 10) = NULL - let list_array = return_array().into_array(1); + let list_array = return_array(); let arr = array_element(&[list_array, Arc::new(Int64Array::from_value(10, 1))]) .expect("failed to initialize function array_element"); let result = @@ -2068,7 +2036,7 @@ mod tests { #[test] fn test_nested_array_element() { // array_element([[1, 2, 3, 4], [5, 6, 7, 8]], 2) = [5, 6, 7, 8] - let list_array = return_nested_array().into_array(1); + let list_array = return_nested_array(); let arr = array_element(&[list_array, Arc::new(Int64Array::from_value(2, 1))]) .expect("failed to initialize function array_element"); let result = @@ -2088,7 +2056,7 @@ mod tests { #[test] fn test_array_pop_back() { // array_pop_back([1, 2, 3, 4]) = [1, 2, 3] - let list_array = return_array().into_array(1); + let list_array = return_array(); let arr = array_pop_back(&[list_array]) .expect("failed to initialize function array_pop_back"); let result = @@ -2167,7 +2135,7 @@ mod tests { ); // array_pop_back([1, NULL, 3, NULL]) = [1, NULL, 3] - let list_array = return_array_with_nulls().into_array(1); + let list_array = return_array_with_nulls(); let arr = array_pop_back(&[list_array]) .expect("failed to initialize function array_pop_back"); let result = @@ -2185,7 +2153,7 @@ mod tests { #[test] fn test_nested_array_pop_back() { // array_pop_back([[1, 2, 3, 4], [5, 6, 7, 8]]) = [[1, 2, 3, 4]] - let list_array = return_nested_array().into_array(1); + let list_array = return_nested_array(); let arr = array_pop_back(&[list_array]) .expect("failed to initialize function array_slice"); let result = @@ -2233,7 +2201,7 @@ mod tests { #[test] fn test_array_slice() { // array_slice([1, 2, 3, 4], 1, 3) = [1, 2, 3] - let list_array = return_array().into_array(1); + let list_array = return_array(); let arr = array_slice(&[ list_array, Arc::new(Int64Array::from_value(1, 1)), @@ -2254,7 +2222,7 @@ mod tests { ); // array_slice([1, 2, 3, 4], 2, 2) = [2] - let list_array = return_array().into_array(1); + let list_array = return_array(); let arr = array_slice(&[ list_array, Arc::new(Int64Array::from_value(2, 1)), @@ -2275,7 +2243,7 @@ mod tests { ); // array_slice([1, 2, 3, 4], 0, 0) = [] - let list_array = return_array().into_array(1); + let list_array = return_array(); let arr = array_slice(&[ list_array, Arc::new(Int64Array::from_value(0, 1)), @@ -2293,7 +2261,7 @@ mod tests { .is_empty()); // array_slice([1, 2, 3, 4], 0, 6) = [1, 2, 3, 4] - let list_array = return_array().into_array(1); + let list_array = return_array(); let arr = array_slice(&[ list_array, Arc::new(Int64Array::from_value(0, 1)), @@ -2314,7 +2282,7 @@ mod tests { ); // array_slice([1, 2, 3, 4], -2, -2) = [] - let list_array = return_array().into_array(1); + let list_array = return_array(); let arr = array_slice(&[ list_array, Arc::new(Int64Array::from_value(-2, 1)), @@ -2332,7 +2300,7 @@ mod tests { .is_empty()); // array_slice([1, 2, 3, 4], -3, -1) = [2, 3] - let list_array = return_array().into_array(1); + let list_array = return_array(); let arr = array_slice(&[ list_array, Arc::new(Int64Array::from_value(-3, 1)), @@ -2353,7 +2321,7 @@ mod tests { ); // array_slice([1, 2, 3, 4], -3, 2) = [2] - let list_array = return_array().into_array(1); + let list_array = return_array(); let arr = array_slice(&[ list_array, Arc::new(Int64Array::from_value(-3, 1)), @@ -2374,7 +2342,7 @@ mod tests { ); // array_slice([1, 2, 3, 4], 2, 11) = [2, 3, 4] - let list_array = return_array().into_array(1); + let list_array = return_array(); let arr = array_slice(&[ list_array, Arc::new(Int64Array::from_value(2, 1)), @@ -2395,7 +2363,7 @@ mod tests { ); // array_slice([1, 2, 3, 4], 3, 1) = [] - let list_array = return_array().into_array(1); + let list_array = return_array(); let arr = array_slice(&[ list_array, Arc::new(Int64Array::from_value(3, 1)), @@ -2413,7 +2381,7 @@ mod tests { .is_empty()); // array_slice([1, 2, 3, 4], -7, -2) = NULL - let list_array = return_array().into_array(1); + let list_array = return_array(); let arr = array_slice(&[ list_array, Arc::new(Int64Array::from_value(-7, 1)), @@ -2434,7 +2402,7 @@ mod tests { #[test] fn test_nested_array_slice() { // array_slice([[1, 2, 3, 4], [5, 6, 7, 8]], 1, 1) = [[1, 2, 3, 4]] - let list_array = return_nested_array().into_array(1); + let list_array = return_nested_array(); let arr = array_slice(&[ list_array, Arc::new(Int64Array::from_value(1, 1)), @@ -2459,7 +2427,7 @@ mod tests { ); // array_slice([[1, 2, 3, 4], [5, 6, 7, 8]], -1, -1) = [] - let list_array = return_nested_array().into_array(1); + let list_array = return_nested_array(); let arr = array_slice(&[ list_array, Arc::new(Int64Array::from_value(-1, 1)), @@ -2477,7 +2445,7 @@ mod tests { .is_empty()); // array_slice([[1, 2, 3, 4], [5, 6, 7, 8]], -1, 2) = [[5, 6, 7, 8]] - let list_array = return_nested_array().into_array(1); + let list_array = return_nested_array(); let arr = array_slice(&[ list_array, Arc::new(Int64Array::from_value(-1, 1)), @@ -2588,7 +2556,7 @@ mod tests { #[test] fn test_nested_array_concat() { // array_concat([1, 2, 3, 4], [1, 2, 3, 4]) = [1, 2, 3, 4, 1, 2, 3, 4] - let list_array = return_array().into_array(1); + let list_array = return_array(); let arr = array_concat(&[list_array.clone(), list_array.clone()]) .expect("failed to initialize function array_concat"); let result = @@ -2605,8 +2573,8 @@ mod tests { ); // array_concat([[1, 2, 3, 4], [5, 6, 7, 8]], [1, 2, 3, 4]) = [[1, 2, 3, 4], [5, 6, 7, 8], [1, 2, 3, 4]] - let list_nested_array = return_nested_array().into_array(1); - let list_array = return_array().into_array(1); + let list_nested_array = return_nested_array(); + let list_array = return_array(); let arr = array_concat(&[list_nested_array, list_array]) .expect("failed to initialize function array_concat"); let result = @@ -2630,7 +2598,7 @@ mod tests { #[test] fn test_array_position() { // array_position([1, 2, 3, 4], 3) = 3 - let list_array = return_array().into_array(1); + let list_array = return_array(); let array = array_position(&[list_array, Arc::new(Int64Array::from_value(3, 1))]) .expect("failed to initialize function array_position"); let result = as_uint64_array(&array) @@ -2642,7 +2610,7 @@ mod tests { #[test] fn test_array_positions() { // array_positions([1, 2, 3, 4], 3) = [3] - let list_array = return_array().into_array(1); + let list_array = return_array(); let array = array_positions(&[list_array, Arc::new(Int64Array::from_value(3, 1))]) .expect("failed to initialize function array_position"); @@ -2664,7 +2632,7 @@ mod tests { #[test] fn test_array_remove() { // array_remove([3, 1, 2, 3, 2, 3], 3) = [1, 2, 3, 2, 3] - let list_array = return_array_with_repeating_elements().into_array(1); + let list_array = return_array_with_repeating_elements(); let array = array_remove(&[list_array, Arc::new(Int64Array::from_value(3, 1))]) .expect("failed to initialize function array_remove"); let result = @@ -2688,8 +2656,8 @@ mod tests { // [[1, 2, 3, 4], [5, 6, 7, 8], [1, 2, 3, 4], [9, 10, 11, 12], [5, 6, 7, 8]], // [1, 2, 3, 4], // ) = [[5, 6, 7, 8], [1, 2, 3, 4], [9, 10, 11, 12], [5, 6, 7, 8]] - let list_array = return_nested_array_with_repeating_elements().into_array(1); - let element_array = return_array().into_array(1); + let list_array = return_nested_array_with_repeating_elements(); + let element_array = return_array(); let array = array_remove(&[list_array, element_array]) .expect("failed to initialize function array_remove"); let result = @@ -2717,7 +2685,7 @@ mod tests { #[test] fn test_array_remove_n() { // array_remove_n([3, 1, 2, 3, 2, 3], 3, 2) = [1, 2, 2, 3] - let list_array = return_array_with_repeating_elements().into_array(1); + let list_array = return_array_with_repeating_elements(); let array = array_remove_n(&[ list_array, Arc::new(Int64Array::from_value(3, 1)), @@ -2746,8 +2714,8 @@ mod tests { // [1, 2, 3, 4], // 3, // ) = [[5, 6, 7, 8], [9, 10, 11, 12], [5, 6, 7, 8]] - let list_array = return_nested_array_with_repeating_elements().into_array(1); - let element_array = return_array().into_array(1); + let list_array = return_nested_array_with_repeating_elements(); + let element_array = return_array(); let array = array_remove_n(&[ list_array, element_array, @@ -2778,7 +2746,7 @@ mod tests { #[test] fn test_array_remove_all() { // array_remove_all([3, 1, 2, 3, 2, 3], 3) = [1, 2, 2] - let list_array = return_array_with_repeating_elements().into_array(1); + let list_array = return_array_with_repeating_elements(); let array = array_remove_all(&[list_array, Arc::new(Int64Array::from_value(3, 1))]) .expect("failed to initialize function array_remove_all"); @@ -2803,8 +2771,8 @@ mod tests { // [[1, 2, 3, 4], [5, 6, 7, 8], [1, 2, 3, 4], [9, 10, 11, 12], [5, 6, 7, 8]], // [1, 2, 3, 4], // ) = [[5, 6, 7, 8], [9, 10, 11, 12], [5, 6, 7, 8]] - let list_array = return_nested_array_with_repeating_elements().into_array(1); - let element_array = return_array().into_array(1); + let list_array = return_nested_array_with_repeating_elements(); + let element_array = return_array(); let array = array_remove_all(&[list_array, element_array]) .expect("failed to initialize function array_remove_all"); let result = as_list_array(&array) @@ -2831,7 +2799,7 @@ mod tests { #[test] fn test_array_replace() { // array_replace([3, 1, 2, 3, 2, 3], 3, 4) = [4, 1, 2, 3, 2, 3] - let list_array = return_array_with_repeating_elements().into_array(1); + let list_array = return_array_with_repeating_elements(); let array = array_replace(&[ list_array, Arc::new(Int64Array::from_value(3, 1)), @@ -2860,9 +2828,9 @@ mod tests { // [1, 2, 3, 4], // [11, 12, 13, 14], // ) = [[11, 12, 13, 14], [5, 6, 7, 8], [1, 2, 3, 4], [9, 10, 11, 12], [5, 6, 7, 8]] - let list_array = return_nested_array_with_repeating_elements().into_array(1); - let from_array = return_array().into_array(1); - let to_array = return_extra_array().into_array(1); + let list_array = return_nested_array_with_repeating_elements(); + let from_array = return_array(); + let to_array = return_extra_array(); let array = array_replace(&[list_array, from_array, to_array]) .expect("failed to initialize function array_replace"); let result = @@ -2891,7 +2859,7 @@ mod tests { #[test] fn test_array_replace_n() { // array_replace_n([3, 1, 2, 3, 2, 3], 3, 4, 2) = [4, 1, 2, 4, 2, 3] - let list_array = return_array_with_repeating_elements().into_array(1); + let list_array = return_array_with_repeating_elements(); let array = array_replace_n(&[ list_array, Arc::new(Int64Array::from_value(3, 1)), @@ -2922,9 +2890,9 @@ mod tests { // [11, 12, 13, 14], // 2, // ) = [[11, 12, 13, 14], [5, 6, 7, 8], [11, 12, 13, 14], [9, 10, 11, 12], [5, 6, 7, 8]] - let list_array = return_nested_array_with_repeating_elements().into_array(1); - let from_array = return_array().into_array(1); - let to_array = return_extra_array().into_array(1); + let list_array = return_nested_array_with_repeating_elements(); + let from_array = return_array(); + let to_array = return_extra_array(); let array = array_replace_n(&[ list_array, from_array, @@ -2958,7 +2926,7 @@ mod tests { #[test] fn test_array_replace_all() { // array_replace_all([3, 1, 2, 3, 2, 3], 3, 4) = [4, 1, 2, 4, 2, 4] - let list_array = return_array_with_repeating_elements().into_array(1); + let list_array = return_array_with_repeating_elements(); let array = array_replace_all(&[ list_array, Arc::new(Int64Array::from_value(3, 1)), @@ -2987,9 +2955,9 @@ mod tests { // [1, 2, 3, 4], // [11, 12, 13, 14], // ) = [[11, 12, 13, 14], [5, 6, 7, 8], [11, 12, 13, 14], [9, 10, 11, 12], [5, 6, 7, 8]] - let list_array = return_nested_array_with_repeating_elements().into_array(1); - let from_array = return_array().into_array(1); - let to_array = return_extra_array().into_array(1); + let list_array = return_nested_array_with_repeating_elements(); + let from_array = return_array(); + let to_array = return_extra_array(); let array = array_replace_all(&[list_array, from_array, to_array]) .expect("failed to initialize function array_replace_all"); let result = as_list_array(&array) @@ -3041,7 +3009,7 @@ mod tests { #[test] fn test_nested_array_repeat() { // array_repeat([1, 2, 3, 4], 3) = [[1, 2, 3, 4], [1, 2, 3, 4], [1, 2, 3, 4]] - let element = return_array().into_array(1); + let element = return_array(); let array = array_repeat(&[element, Arc::new(Int64Array::from_value(3, 1))]) .expect("failed to initialize function array_repeat"); let result = @@ -3067,25 +3035,25 @@ mod tests { #[test] fn test_array_to_string() { // array_to_string([1, 2, 3, 4], ',') = 1,2,3,4 - let list_array = return_array().into_array(1); + let list_array = return_array(); let array = array_to_string(&[list_array, Arc::new(StringArray::from(vec![Some(",")]))]) .expect("failed to initialize function array_to_string"); - let result = as_generic_string_array::(&array) + let result = as_string_array(&array) .expect("failed to initialize function array_to_string"); assert_eq!(result.len(), 1); assert_eq!("1,2,3,4", result.value(0)); // array_to_string([1, NULL, 3, NULL], ',', '*') = 1,*,3,* - let list_array = return_array_with_nulls().into_array(1); + let list_array = return_array_with_nulls(); let array = array_to_string(&[ list_array, Arc::new(StringArray::from(vec![Some(",")])), Arc::new(StringArray::from(vec![Some("*")])), ]) .expect("failed to initialize function array_to_string"); - let result = as_generic_string_array::(&array) + let result = as_string_array(&array) .expect("failed to initialize function array_to_string"); assert_eq!(result.len(), 1); @@ -3095,25 +3063,25 @@ mod tests { #[test] fn test_nested_array_to_string() { // array_to_string([[1, 2, 3, 4], [5, 6, 7, 8]], '-') = 1-2-3-4-5-6-7-8 - let list_array = return_nested_array().into_array(1); + let list_array = return_nested_array(); let array = array_to_string(&[list_array, Arc::new(StringArray::from(vec![Some("-")]))]) .expect("failed to initialize function array_to_string"); - let result = as_generic_string_array::(&array) + let result = as_string_array(&array) .expect("failed to initialize function array_to_string"); assert_eq!(result.len(), 1); assert_eq!("1-2-3-4-5-6-7-8", result.value(0)); // array_to_string([[1, NULL, 3, NULL], [NULL, 6, 7, NULL]], '-', '*') = 1-*-3-*-*-6-7-* - let list_array = return_nested_array_with_nulls().into_array(1); + let list_array = return_nested_array_with_nulls(); let array = array_to_string(&[ list_array, Arc::new(StringArray::from(vec![Some("-")])), Arc::new(StringArray::from(vec![Some("*")])), ]) .expect("failed to initialize function array_to_string"); - let result = as_generic_string_array::(&array) + let result = as_string_array(&array) .expect("failed to initialize function array_to_string"); assert_eq!(result.len(), 1); @@ -3123,7 +3091,7 @@ mod tests { #[test] fn test_cardinality() { // cardinality([1, 2, 3, 4]) = 4 - let list_array = return_array().into_array(1); + let list_array = return_array(); let arr = cardinality(&[list_array]) .expect("failed to initialize function cardinality"); let result = @@ -3135,7 +3103,7 @@ mod tests { #[test] fn test_nested_cardinality() { // cardinality([[1, 2, 3, 4], [5, 6, 7, 8]]) = 8 - let list_array = return_nested_array().into_array(1); + let list_array = return_nested_array(); let arr = cardinality(&[list_array]) .expect("failed to initialize function cardinality"); let result = @@ -3147,7 +3115,7 @@ mod tests { #[test] fn test_array_length() { // array_length([1, 2, 3, 4]) = 4 - let list_array = return_array().into_array(1); + let list_array = return_array(); let arr = array_length(&[list_array.clone()]) .expect("failed to initialize function array_ndims"); let result = @@ -3166,7 +3134,7 @@ mod tests { #[test] fn test_nested_array_length() { - let list_array = return_nested_array().into_array(1); + let list_array = return_nested_array(); // array_length([[1, 2, 3, 4], [5, 6, 7, 8]]) = 2 let arr = array_length(&[list_array.clone()]) @@ -3206,7 +3174,7 @@ mod tests { #[test] fn test_array_dims() { // array_dims([1, 2, 3, 4]) = [4] - let list_array = return_array().into_array(1); + let list_array = return_array(); let array = array_dims(&[list_array]).expect("failed to initialize function array_dims"); @@ -3227,7 +3195,7 @@ mod tests { #[test] fn test_nested_array_dims() { // array_dims([[1, 2, 3, 4], [5, 6, 7, 8]]) = [2, 4] - let list_array = return_nested_array().into_array(1); + let list_array = return_nested_array(); let array = array_dims(&[list_array]).expect("failed to initialize function array_dims"); @@ -3248,7 +3216,7 @@ mod tests { #[test] fn test_array_ndims() { // array_ndims([1, 2, 3, 4]) = 1 - let list_array = return_array().into_array(1); + let list_array = return_array(); let array = array_ndims(&[list_array]) .expect("failed to initialize function array_ndims"); @@ -3261,7 +3229,7 @@ mod tests { #[test] fn test_nested_array_ndims() { // array_ndims([[1, 2, 3, 4], [5, 6, 7, 8]]) = 2 - let list_array = return_nested_array().into_array(1); + let list_array = return_nested_array(); let array = array_ndims(&[list_array]) .expect("failed to initialize function array_ndims"); @@ -3285,152 +3253,137 @@ mod tests { assert_eq!(array.unwrap_err().strip_backtrace(), "Error during planning: array_append received incompatible types: '[Int64, Utf8]'."); } - fn return_array() -> ColumnarValue { + fn return_array() -> ArrayRef { // Returns: [1, 2, 3, 4] let args = [ - ColumnarValue::Scalar(ScalarValue::Int64(Some(1))), - ColumnarValue::Scalar(ScalarValue::Int64(Some(2))), - ColumnarValue::Scalar(ScalarValue::Int64(Some(3))), - ColumnarValue::Scalar(ScalarValue::Int64(Some(4))), + Arc::new(Int64Array::from(vec![Some(1)])) as ArrayRef, + Arc::new(Int64Array::from(vec![Some(2)])) as ArrayRef, + Arc::new(Int64Array::from(vec![Some(3)])) as ArrayRef, + Arc::new(Int64Array::from(vec![Some(4)])) as ArrayRef, ]; - let result = array(&args).expect("failed to initialize function array"); - ColumnarValue::Array(result.clone()) + make_array(&args).expect("failed to initialize function array") } - fn return_extra_array() -> ColumnarValue { + fn return_extra_array() -> ArrayRef { // Returns: [11, 12, 13, 14] let args = [ - ColumnarValue::Scalar(ScalarValue::Int64(Some(11))), - ColumnarValue::Scalar(ScalarValue::Int64(Some(12))), - ColumnarValue::Scalar(ScalarValue::Int64(Some(13))), - ColumnarValue::Scalar(ScalarValue::Int64(Some(14))), + Arc::new(Int64Array::from(vec![Some(11)])) as ArrayRef, + Arc::new(Int64Array::from(vec![Some(12)])) as ArrayRef, + Arc::new(Int64Array::from(vec![Some(13)])) as ArrayRef, + Arc::new(Int64Array::from(vec![Some(14)])) as ArrayRef, ]; - let result = array(&args).expect("failed to initialize function array"); - ColumnarValue::Array(result.clone()) + make_array(&args).expect("failed to initialize function array") } - fn return_nested_array() -> ColumnarValue { + fn return_nested_array() -> ArrayRef { // Returns: [[1, 2, 3, 4], [5, 6, 7, 8]] let args = [ - ColumnarValue::Scalar(ScalarValue::Int64(Some(1))), - ColumnarValue::Scalar(ScalarValue::Int64(Some(2))), - ColumnarValue::Scalar(ScalarValue::Int64(Some(3))), - ColumnarValue::Scalar(ScalarValue::Int64(Some(4))), + Arc::new(Int64Array::from(vec![Some(1)])) as ArrayRef, + Arc::new(Int64Array::from(vec![Some(2)])) as ArrayRef, + Arc::new(Int64Array::from(vec![Some(3)])) as ArrayRef, + Arc::new(Int64Array::from(vec![Some(4)])) as ArrayRef, ]; - let arr1 = array(&args).expect("failed to initialize function array"); + let arr1 = make_array(&args).expect("failed to initialize function array"); let args = [ - ColumnarValue::Scalar(ScalarValue::Int64(Some(5))), - ColumnarValue::Scalar(ScalarValue::Int64(Some(6))), - ColumnarValue::Scalar(ScalarValue::Int64(Some(7))), - ColumnarValue::Scalar(ScalarValue::Int64(Some(8))), + Arc::new(Int64Array::from(vec![Some(5)])) as ArrayRef, + Arc::new(Int64Array::from(vec![Some(6)])) as ArrayRef, + Arc::new(Int64Array::from(vec![Some(7)])) as ArrayRef, + Arc::new(Int64Array::from(vec![Some(8)])) as ArrayRef, ]; - let arr2 = array(&args).expect("failed to initialize function array"); + let arr2 = make_array(&args).expect("failed to initialize function array"); - let args = [ColumnarValue::Array(arr1), ColumnarValue::Array(arr2)]; - let result = array(&args).expect("failed to initialize function array"); - ColumnarValue::Array(result.clone()) + make_array(&[arr1, arr2]).expect("failed to initialize function array") } - fn return_array_with_nulls() -> ColumnarValue { + fn return_array_with_nulls() -> ArrayRef { // Returns: [1, NULL, 3, NULL] let args = [ - ColumnarValue::Scalar(ScalarValue::Int64(Some(1))), - ColumnarValue::Scalar(ScalarValue::Null), - ColumnarValue::Scalar(ScalarValue::Int64(Some(3))), - ColumnarValue::Scalar(ScalarValue::Null), + Arc::new(Int64Array::from(vec![Some(1)])) as ArrayRef, + Arc::new(Int64Array::from(vec![None])) as ArrayRef, + Arc::new(Int64Array::from(vec![Some(3)])) as ArrayRef, + Arc::new(Int64Array::from(vec![None])) as ArrayRef, ]; - let result = array(&args).expect("failed to initialize function array"); - ColumnarValue::Array(result.clone()) + make_array(&args).expect("failed to initialize function array") } - fn return_nested_array_with_nulls() -> ColumnarValue { + fn return_nested_array_with_nulls() -> ArrayRef { // Returns: [[1, NULL, 3, NULL], [NULL, 6, 7, NULL]] let args = [ - ColumnarValue::Scalar(ScalarValue::Int64(Some(1))), - ColumnarValue::Scalar(ScalarValue::Null), - ColumnarValue::Scalar(ScalarValue::Int64(Some(3))), - ColumnarValue::Scalar(ScalarValue::Null), + Arc::new(Int64Array::from(vec![Some(1)])) as ArrayRef, + Arc::new(Int64Array::from(vec![None])) as ArrayRef, + Arc::new(Int64Array::from(vec![Some(3)])) as ArrayRef, + Arc::new(Int64Array::from(vec![None])) as ArrayRef, ]; - let arr1 = array(&args).expect("failed to initialize function array"); + let arr1 = make_array(&args).expect("failed to initialize function array"); let args = [ - ColumnarValue::Scalar(ScalarValue::Null), - ColumnarValue::Scalar(ScalarValue::Int64(Some(6))), - ColumnarValue::Scalar(ScalarValue::Int64(Some(7))), - ColumnarValue::Scalar(ScalarValue::Null), + Arc::new(Int64Array::from(vec![None])) as ArrayRef, + Arc::new(Int64Array::from(vec![Some(6)])) as ArrayRef, + Arc::new(Int64Array::from(vec![Some(7)])) as ArrayRef, + Arc::new(Int64Array::from(vec![None])) as ArrayRef, ]; - let arr2 = array(&args).expect("failed to initialize function array"); + let arr2 = make_array(&args).expect("failed to initialize function array"); - let args = [ColumnarValue::Array(arr1), ColumnarValue::Array(arr2)]; - let result = array(&args).expect("failed to initialize function array"); - ColumnarValue::Array(result.clone()) + make_array(&[arr1, arr2]).expect("failed to initialize function array") } - fn return_array_with_repeating_elements() -> ColumnarValue { + fn return_array_with_repeating_elements() -> ArrayRef { // Returns: [3, 1, 2, 3, 2, 3] let args = [ - ColumnarValue::Scalar(ScalarValue::Int64(Some(3))), - ColumnarValue::Scalar(ScalarValue::Int64(Some(1))), - ColumnarValue::Scalar(ScalarValue::Int64(Some(2))), - ColumnarValue::Scalar(ScalarValue::Int64(Some(3))), - ColumnarValue::Scalar(ScalarValue::Int64(Some(2))), - ColumnarValue::Scalar(ScalarValue::Int64(Some(3))), + Arc::new(Int64Array::from(vec![Some(3)])) as ArrayRef, + Arc::new(Int64Array::from(vec![Some(1)])) as ArrayRef, + Arc::new(Int64Array::from(vec![Some(2)])) as ArrayRef, + Arc::new(Int64Array::from(vec![Some(3)])) as ArrayRef, + Arc::new(Int64Array::from(vec![Some(2)])) as ArrayRef, + Arc::new(Int64Array::from(vec![Some(3)])) as ArrayRef, ]; - let result = array(&args).expect("failed to initialize function array"); - ColumnarValue::Array(result.clone()) + make_array(&args).expect("failed to initialize function array") } - fn return_nested_array_with_repeating_elements() -> ColumnarValue { + fn return_nested_array_with_repeating_elements() -> ArrayRef { // Returns: [[1, 2, 3, 4], [5, 6, 7, 8], [1, 2, 3, 4], [9, 10, 11, 12], [5, 6, 7, 8]] let args = [ - ColumnarValue::Scalar(ScalarValue::Int64(Some(1))), - ColumnarValue::Scalar(ScalarValue::Int64(Some(2))), - ColumnarValue::Scalar(ScalarValue::Int64(Some(3))), - ColumnarValue::Scalar(ScalarValue::Int64(Some(4))), + Arc::new(Int64Array::from(vec![Some(1)])) as ArrayRef, + Arc::new(Int64Array::from(vec![Some(2)])) as ArrayRef, + Arc::new(Int64Array::from(vec![Some(3)])) as ArrayRef, + Arc::new(Int64Array::from(vec![Some(4)])) as ArrayRef, ]; - let arr1 = array(&args).expect("failed to initialize function array"); + let arr1 = make_array(&args).expect("failed to initialize function array"); let args = [ - ColumnarValue::Scalar(ScalarValue::Int64(Some(5))), - ColumnarValue::Scalar(ScalarValue::Int64(Some(6))), - ColumnarValue::Scalar(ScalarValue::Int64(Some(7))), - ColumnarValue::Scalar(ScalarValue::Int64(Some(8))), + Arc::new(Int64Array::from(vec![Some(5)])) as ArrayRef, + Arc::new(Int64Array::from(vec![Some(6)])) as ArrayRef, + Arc::new(Int64Array::from(vec![Some(7)])) as ArrayRef, + Arc::new(Int64Array::from(vec![Some(8)])) as ArrayRef, ]; - let arr2 = array(&args).expect("failed to initialize function array"); + let arr2 = make_array(&args).expect("failed to initialize function array"); let args = [ - ColumnarValue::Scalar(ScalarValue::Int64(Some(1))), - ColumnarValue::Scalar(ScalarValue::Int64(Some(2))), - ColumnarValue::Scalar(ScalarValue::Int64(Some(3))), - ColumnarValue::Scalar(ScalarValue::Int64(Some(4))), + Arc::new(Int64Array::from(vec![Some(1)])) as ArrayRef, + Arc::new(Int64Array::from(vec![Some(2)])) as ArrayRef, + Arc::new(Int64Array::from(vec![Some(3)])) as ArrayRef, + Arc::new(Int64Array::from(vec![Some(4)])) as ArrayRef, ]; - let arr3 = array(&args).expect("failed to initialize function array"); + let arr3 = make_array(&args).expect("failed to initialize function array"); let args = [ - ColumnarValue::Scalar(ScalarValue::Int64(Some(9))), - ColumnarValue::Scalar(ScalarValue::Int64(Some(10))), - ColumnarValue::Scalar(ScalarValue::Int64(Some(11))), - ColumnarValue::Scalar(ScalarValue::Int64(Some(12))), + Arc::new(Int64Array::from(vec![Some(9)])) as ArrayRef, + Arc::new(Int64Array::from(vec![Some(10)])) as ArrayRef, + Arc::new(Int64Array::from(vec![Some(11)])) as ArrayRef, + Arc::new(Int64Array::from(vec![Some(12)])) as ArrayRef, ]; - let arr4 = array(&args).expect("failed to initialize function array"); + let arr4 = make_array(&args).expect("failed to initialize function array"); let args = [ - ColumnarValue::Scalar(ScalarValue::Int64(Some(5))), - ColumnarValue::Scalar(ScalarValue::Int64(Some(6))), - ColumnarValue::Scalar(ScalarValue::Int64(Some(7))), - ColumnarValue::Scalar(ScalarValue::Int64(Some(8))), + Arc::new(Int64Array::from(vec![Some(5)])) as ArrayRef, + Arc::new(Int64Array::from(vec![Some(6)])) as ArrayRef, + Arc::new(Int64Array::from(vec![Some(7)])) as ArrayRef, + Arc::new(Int64Array::from(vec![Some(8)])) as ArrayRef, ]; - let arr5 = array(&args).expect("failed to initialize function array"); + let arr5 = make_array(&args).expect("failed to initialize function array"); - let args = [ - ColumnarValue::Array(arr1), - ColumnarValue::Array(arr2), - ColumnarValue::Array(arr3), - ColumnarValue::Array(arr4), - ColumnarValue::Array(arr5), - ]; - let result = array(&args).expect("failed to initialize function array"); - ColumnarValue::Array(result.clone()) + make_array(&[arr1, arr2, arr3, arr4, arr5]) + .expect("failed to initialize function array") } } From b02fe5bbd7854bf186c54c542b80ab75ac323c38 Mon Sep 17 00:00:00 2001 From: Andrew Lamb Date: Sat, 28 Oct 2023 11:40:34 -0400 Subject: [PATCH 145/572] Minor: simplify update code (#7943) --- datafusion/sql/src/statement.rs | 22 +++++++--------------- 1 file changed, 7 insertions(+), 15 deletions(-) diff --git a/datafusion/sql/src/statement.rs b/datafusion/sql/src/statement.rs index b5196c086638..80a27db6e63d 100644 --- a/datafusion/sql/src/statement.rs +++ b/datafusion/sql/src/statement.rs @@ -36,7 +36,6 @@ use datafusion_common::{ Result, SchemaReference, TableReference, ToDFSchema, }; use datafusion_expr::dml::{CopyOptions, CopyTo}; -use datafusion_expr::expr::Placeholder; use datafusion_expr::expr_rewriter::normalize_col_with_schemas_and_ambiguity_check; use datafusion_expr::logical_plan::builder::project; use datafusion_expr::logical_plan::DdlStatement; @@ -1025,20 +1024,13 @@ impl<'a, S: ContextProvider> SqlToRel<'a, S> { // Projection let mut exprs = vec![]; for (col_name, expr, dt) in values_and_types.into_iter() { - let expr = self.sql_to_expr(expr, &table_schema, &mut planner_context)?; - let expr = match expr { - datafusion_expr::Expr::Placeholder(Placeholder { - ref id, - ref data_type, - }) => match data_type { - None => datafusion_expr::Expr::Placeholder(Placeholder::new( - id.clone(), - Some(dt.clone()), - )), - Some(_) => expr, - }, - _ => expr, - }; + let mut expr = self.sql_to_expr(expr, &table_schema, &mut planner_context)?; + // Update placeholder's datatype to the type of the target column + if let datafusion_expr::Expr::Placeholder(placeholder) = &mut expr { + placeholder.data_type = + placeholder.data_type.take().or_else(|| Some(dt.clone())); + } + // Cast to target column type, if necessary let expr = expr.cast_to(dt, source.schema())?.alias(col_name); exprs.push(expr); } From 9ee055a3f59ec08de3432a3ec3de8ff55d29975a Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Sat, 28 Oct 2023 09:45:16 -0600 Subject: [PATCH 146/572] Add some initial content about creating logical plans (#7952) --- Cargo.toml | 1 + .../src/datasource/default_table_source.rs | 6 +- datafusion/core/src/prelude.rs | 2 +- dev/update_datafusion_versions.py | 1 + docs/Cargo.toml | 32 +++++ .../building-logical-plans.md | 129 +++++++++++++++++- docs/src/lib.rs | 19 +++ docs/src/library_logical_plan.rs | 78 +++++++++++ 8 files changed, 264 insertions(+), 4 deletions(-) create mode 100644 docs/Cargo.toml create mode 100644 docs/src/lib.rs create mode 100644 docs/src/library_logical_plan.rs diff --git a/Cargo.toml b/Cargo.toml index 71088e7fc7ad..77e3c6038ea7 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -32,6 +32,7 @@ members = [ "datafusion/substrait", "datafusion/wasmtest", "datafusion-examples", + "docs", "test-utils", "benchmarks", ] diff --git a/datafusion/core/src/datasource/default_table_source.rs b/datafusion/core/src/datasource/default_table_source.rs index f93faa50a9b9..00a9c123ceee 100644 --- a/datafusion/core/src/datasource/default_table_source.rs +++ b/datafusion/core/src/datasource/default_table_source.rs @@ -26,10 +26,12 @@ use arrow::datatypes::SchemaRef; use datafusion_common::{internal_err, Constraints, DataFusionError}; use datafusion_expr::{Expr, TableProviderFilterPushDown, TableSource}; -/// DataFusion default table source, wrapping TableProvider +/// DataFusion default table source, wrapping TableProvider. /// /// This structure adapts a `TableProvider` (physical plan trait) to the `TableSource` -/// (logical plan trait) +/// (logical plan trait) and is necessary because the logical plan is contained in +/// the `datafusion_expr` crate, and is not aware of table providers, which exist in +/// the core `datafusion` crate. pub struct DefaultTableSource { /// table provider pub table_provider: Arc, diff --git a/datafusion/core/src/prelude.rs b/datafusion/core/src/prelude.rs index 7689468e5d13..5cd8b3870f81 100644 --- a/datafusion/core/src/prelude.rs +++ b/datafusion/core/src/prelude.rs @@ -13,7 +13,7 @@ // "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY // KIND, either express or implied. See the License for the // specific language governing permissions and limitations -// under the License.pub}, +// under the License. //! DataFusion "prelude" to simplify importing common types. //! diff --git a/dev/update_datafusion_versions.py b/dev/update_datafusion_versions.py index 7cbe39fdfb66..19701b813671 100755 --- a/dev/update_datafusion_versions.py +++ b/dev/update_datafusion_versions.py @@ -43,6 +43,7 @@ 'datafusion-wasmtest': 'datafusion/wasmtest/Cargo.toml', 'datafusion-benchmarks': 'benchmarks/Cargo.toml', 'datafusion-examples': 'datafusion-examples/Cargo.toml', + 'datafusion-docs': 'docs/Cargo.toml', } def update_workspace_version(new_version: str): diff --git a/docs/Cargo.toml b/docs/Cargo.toml new file mode 100644 index 000000000000..9caa0bde3608 --- /dev/null +++ b/docs/Cargo.toml @@ -0,0 +1,32 @@ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, +# software distributed under the License is distributed on an +# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +# KIND, either express or implied. See the License for the +# specific language governing permissions and limitations +# under the License. + +[package] +name = "datafusion-docs-tests" +description = "DataFusion Documentation Tests" +publish = false +version = { workspace = true } +edition = { workspace = true } +readme = { workspace = true } +homepage = { workspace = true } +repository = { workspace = true } +license = { workspace = true } +authors = { workspace = true } +rust-version = "1.70" + +[dependencies] +datafusion = { path = "../datafusion/core", version = "32.0.0", default-features = false } diff --git a/docs/source/library-user-guide/building-logical-plans.md b/docs/source/library-user-guide/building-logical-plans.md index 406f4881129c..fe922d8eaeb1 100644 --- a/docs/source/library-user-guide/building-logical-plans.md +++ b/docs/source/library-user-guide/building-logical-plans.md @@ -19,4 +19,131 @@ # Building Logical Plans -Coming Soon +A logical plan is a structured representation of a database query that describes the high-level operations and +transformations needed to retrieve data from a database or data source. It abstracts away specific implementation +details and focuses on the logical flow of the query, including operations like filtering, sorting, and joining tables. + +This logical plan serves as an intermediate step before generating an optimized physical execution plan. This is +explained in more detail in the [Query Planning and Execution Overview] section of the [Architecture Guide]. + +## Building Logical Plans Manually + +DataFusion's [LogicalPlan] is an enum containing variants representing all the supported operators, and also +contains an `Extension` variant that allows projects building on DataFusion to add custom logical operators. + +It is possible to create logical plans by directly creating instances of the [LogicalPlan] enum as follows, but is is +much easier to use the [LogicalPlanBuilder], which is described in the next section. + +Here is an example of building a logical plan directly: + + + +```rust +// create a logical table source +let schema = Schema::new(vec![ + Field::new("id", DataType::Int32, true), + Field::new("name", DataType::Utf8, true), +]); +let table_source = LogicalTableSource::new(SchemaRef::new(schema)); + +// create a TableScan plan +let projection = None; // optional projection +let filters = vec![]; // optional filters to push down +let fetch = None; // optional LIMIT +let table_scan = LogicalPlan::TableScan(TableScan::try_new( + "person", + Arc::new(table_source), + projection, + filters, + fetch, +)?); + +// create a Filter plan that evaluates `id > 500` that wraps the TableScan +let filter_expr = col("id").gt(lit(500)); +let plan = LogicalPlan::Filter(Filter::try_new(filter_expr, Arc::new(table_scan))?); + +// print the plan +println!("{}", plan.display_indent_schema()); +``` + +This example produces the following plan: + +``` +Filter: person.id > Int32(500) [id:Int32;N, name:Utf8;N] + TableScan: person [id:Int32;N, name:Utf8;N] +``` + +## Building Logical Plans with LogicalPlanBuilder + +DataFusion logical plans can be created using the [LogicalPlanBuilder] struct. There is also a [DataFrame] API which is +a higher-level API that delegates to [LogicalPlanBuilder]. + +The following associated functions can be used to create a new builder: + +- `empty` - create an empty plan with no fields +- `values` - create a plan from a set of literal values +- `scan` - create a plan representing a table scan +- `scan_with_filters` - create a plan representing a table scan with filters + +Once the builder is created, transformation methods can be called to declare that further operations should be +performed on the plan. Note that all we are doing at this stage is building up the logical plan structure. No query +execution will be performed. + +Here are some examples of transformation methods, but for a full list, refer to the [LogicalPlanBuilder] API documentation. + +- `filter` +- `limit` +- `sort` +- `distinct` +- `join` + +The following example demonstrates building the same simple query plan as the previous example, with a table scan followed by a filter. + + + +```rust +// create a logical table source +let schema = Schema::new(vec![ + Field::new("id", DataType::Int32, true), + Field::new("name", DataType::Utf8, true), +]); +let table_source = LogicalTableSource::new(SchemaRef::new(schema)); + +// optional projection +let projection = None; + +// create a LogicalPlanBuilder for a table scan +let builder = LogicalPlanBuilder::scan("person", Arc::new(table_source), projection)?; + +// perform a filter operation and build the plan +let plan = builder + .filter(col("id").gt(lit(500)))? // WHERE id > 500 + .build()?; + +// print the plan +println!("{}", plan.display_indent_schema()); +``` + +This example produces the following plan: + +``` +Filter: person.id > Int32(500) [id:Int32;N, name:Utf8;N] + TableScan: person [id:Int32;N, name:Utf8;N] +``` + +## Table Sources + +The previous example used a [LogicalTableSource], which is used for tests and documentation in DataFusion, and is also +suitable if you are using DataFusion to build logical plans but do not use DataFusion's physical planner. However, if you +want to use a [TableSource] that can be executed in DataFusion then you will need to use [DefaultTableSource], which is a +wrapper for a [TableProvider]. + +[query planning and execution overview]: https://docs.rs/datafusion/latest/datafusion/index.html#query-planning-and-execution-overview +[architecture guide]: https://docs.rs/datafusion/latest/datafusion/index.html#architecture +[logicalplan]: https://docs.rs/datafusion-expr/latest/datafusion_expr/logical_plan/enum.LogicalPlan.html +[logicalplanbuilder]: https://docs.rs/datafusion-expr/latest/datafusion_expr/logical_plan/builder/struct.LogicalPlanBuilder.html +[dataframe]: using-the-dataframe-api.md +[logicaltablesource]: https://docs.rs/datafusion-expr/latest/datafusion_expr/logical_plan/builder/struct.LogicalTableSource.html +[defaulttablesource]: https://docs.rs/datafusion/latest/datafusion/datasource/default_table_source/struct.DefaultTableSource.html +[tableprovider]: https://docs.rs/datafusion/latest/datafusion/datasource/provider/trait.TableProvider.html +[tablesource]: https://docs.rs/datafusion-expr/latest/datafusion_expr/trait.TableSource.html diff --git a/docs/src/lib.rs b/docs/src/lib.rs new file mode 100644 index 000000000000..f73132468ec9 --- /dev/null +++ b/docs/src/lib.rs @@ -0,0 +1,19 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +#[cfg(test)] +mod library_logical_plan; diff --git a/docs/src/library_logical_plan.rs b/docs/src/library_logical_plan.rs new file mode 100644 index 000000000000..355003941570 --- /dev/null +++ b/docs/src/library_logical_plan.rs @@ -0,0 +1,78 @@ +// 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 datafusion::arrow::datatypes::{DataType, Field, Schema, SchemaRef}; +use datafusion::error::Result; +use datafusion::logical_expr::builder::LogicalTableSource; +use datafusion::logical_expr::{Filter, LogicalPlan, LogicalPlanBuilder, TableScan}; +use datafusion::prelude::*; +use std::sync::Arc; + +#[test] +fn plan_1() -> Result<()> { + // create a logical table source + let schema = Schema::new(vec![ + Field::new("id", DataType::Int32, true), + Field::new("name", DataType::Utf8, true), + ]); + let table_source = LogicalTableSource::new(SchemaRef::new(schema)); + + // create a TableScan plan + let projection = None; // optional projection + let filters = vec![]; // optional filters to push down + let fetch = None; // optional LIMIT + let table_scan = LogicalPlan::TableScan(TableScan::try_new( + "person", + Arc::new(table_source), + projection, + filters, + fetch, + )?); + + // create a Filter plan that evaluates `id > 500` and wraps the TableScan + let filter_expr = col("id").gt(lit(500)); + let plan = LogicalPlan::Filter(Filter::try_new(filter_expr, Arc::new(table_scan))?); + + // print the plan + println!("{}", plan.display_indent_schema()); + + Ok(()) +} + +#[test] +fn plan_builder_1() -> Result<()> { + // create a logical table source + let schema = Schema::new(vec![ + Field::new("id", DataType::Int32, true), + Field::new("name", DataType::Utf8, true), + ]); + let table_source = LogicalTableSource::new(SchemaRef::new(schema)); + + // optional projection + let projection = None; + + // create a LogicalPlanBuilder for a table scan + let builder = LogicalPlanBuilder::scan("person", Arc::new(table_source), projection)?; + + // perform a filter that evaluates `id > 500`, and build the plan + let plan = builder.filter(col("id").gt(lit(500)))?.build()?; + + // print the plan + println!("{}", plan.display_indent_schema()); + + Ok(()) +} From d24228a6b9552f2dce166836e60c1b928116933f Mon Sep 17 00:00:00 2001 From: yi wang <48236141+my-vegetable-has-exploded@users.noreply.github.com> Date: Sun, 29 Oct 2023 19:34:29 +0800 Subject: [PATCH 147/572] Minor: Change from `&mut SessionContext` to `&SessionContext` in substrait (#7965) * Lower &mut SessionContext in substrait * rm mut ctx in tests --- .../substrait/src/logical_plan/consumer.rs | 4 +-- .../substrait/src/physical_plan/consumer.rs | 2 +- .../tests/cases/roundtrip_logical_plan.rs | 30 +++++++++---------- .../tests/cases/roundtrip_physical_plan.rs | 4 +-- datafusion/substrait/tests/cases/serialize.rs | 4 +-- 5 files changed, 22 insertions(+), 22 deletions(-) diff --git a/datafusion/substrait/src/logical_plan/consumer.rs b/datafusion/substrait/src/logical_plan/consumer.rs index ae65a2c7d94a..a15121652452 100644 --- a/datafusion/substrait/src/logical_plan/consumer.rs +++ b/datafusion/substrait/src/logical_plan/consumer.rs @@ -177,7 +177,7 @@ fn split_eq_and_noneq_join_predicate_with_nulls_equality( /// Convert Substrait Plan to DataFusion DataFrame pub async fn from_substrait_plan( - ctx: &mut SessionContext, + ctx: &SessionContext, plan: &Plan, ) -> Result { // Register function extension @@ -219,7 +219,7 @@ pub async fn from_substrait_plan( /// Convert Substrait Rel to DataFusion DataFrame #[async_recursion] pub async fn from_substrait_rel( - ctx: &mut SessionContext, + ctx: &SessionContext, rel: &Rel, extensions: &HashMap, ) -> Result { diff --git a/datafusion/substrait/src/physical_plan/consumer.rs b/datafusion/substrait/src/physical_plan/consumer.rs index 7788ba0a69de..1dab1f9d5e39 100644 --- a/datafusion/substrait/src/physical_plan/consumer.rs +++ b/datafusion/substrait/src/physical_plan/consumer.rs @@ -38,7 +38,7 @@ use substrait::proto::{ /// Convert Substrait Rel to DataFusion ExecutionPlan #[async_recursion] pub async fn from_substrait_rel( - _ctx: &mut SessionContext, + _ctx: &SessionContext, rel: &Rel, _extensions: &HashMap, ) -> Result> { diff --git a/datafusion/substrait/tests/cases/roundtrip_logical_plan.rs b/datafusion/substrait/tests/cases/roundtrip_logical_plan.rs index 32416125de24..ca2b4d48c460 100644 --- a/datafusion/substrait/tests/cases/roundtrip_logical_plan.rs +++ b/datafusion/substrait/tests/cases/roundtrip_logical_plan.rs @@ -606,7 +606,7 @@ async fn new_test_grammar() -> Result<()> { #[tokio::test] async fn extension_logical_plan() -> Result<()> { - let mut ctx = create_context().await?; + let ctx = create_context().await?; let validation_bytes = "MockUserDefinedLogicalPlan".as_bytes().to_vec(); let ext_plan = LogicalPlan::Extension(Extension { node: Arc::new(MockUserDefinedLogicalPlan { @@ -617,7 +617,7 @@ async fn extension_logical_plan() -> Result<()> { }); let proto = to_substrait_plan(&ext_plan, &ctx)?; - let plan2 = from_substrait_plan(&mut ctx, &proto).await?; + let plan2 = from_substrait_plan(&ctx, &proto).await?; let plan1str = format!("{ext_plan:?}"); let plan2str = format!("{plan2:?}"); @@ -712,11 +712,11 @@ async fn verify_post_join_filter_value(proto: Box) -> Result<()> { } async fn assert_expected_plan(sql: &str, expected_plan_str: &str) -> Result<()> { - let mut ctx = create_context().await?; + let ctx = create_context().await?; let df = ctx.sql(sql).await?; let plan = df.into_optimized_plan()?; let proto = to_substrait_plan(&plan, &ctx)?; - let plan2 = from_substrait_plan(&mut ctx, &proto).await?; + let plan2 = from_substrait_plan(&ctx, &proto).await?; let plan2 = ctx.state().optimize(&plan2)?; let plan2str = format!("{plan2:?}"); assert_eq!(expected_plan_str, &plan2str); @@ -724,11 +724,11 @@ async fn assert_expected_plan(sql: &str, expected_plan_str: &str) -> Result<()> } async fn roundtrip_fill_na(sql: &str) -> Result<()> { - let mut ctx = create_context().await?; + let ctx = create_context().await?; let df = ctx.sql(sql).await?; let plan1 = df.into_optimized_plan()?; let proto = to_substrait_plan(&plan1, &ctx)?; - let plan2 = from_substrait_plan(&mut ctx, &proto).await?; + let plan2 = from_substrait_plan(&ctx, &proto).await?; let plan2 = ctx.state().optimize(&plan2)?; // Format plan string and replace all None's with 0 @@ -743,15 +743,15 @@ async fn test_alias(sql_with_alias: &str, sql_no_alias: &str) -> Result<()> { // Since we ignore the SubqueryAlias in the producer, the result should be // the same as producing a Substrait plan from the same query without aliases // sql_with_alias -> substrait -> logical plan = sql_no_alias -> substrait -> logical plan - let mut ctx = create_context().await?; + let ctx = create_context().await?; let df_a = ctx.sql(sql_with_alias).await?; let proto_a = to_substrait_plan(&df_a.into_optimized_plan()?, &ctx)?; - let plan_with_alias = from_substrait_plan(&mut ctx, &proto_a).await?; + let plan_with_alias = from_substrait_plan(&ctx, &proto_a).await?; let df = ctx.sql(sql_no_alias).await?; let proto = to_substrait_plan(&df.into_optimized_plan()?, &ctx)?; - let plan = from_substrait_plan(&mut ctx, &proto).await?; + let plan = from_substrait_plan(&ctx, &proto).await?; println!("{plan_with_alias:#?}"); println!("{plan:#?}"); @@ -763,11 +763,11 @@ async fn test_alias(sql_with_alias: &str, sql_no_alias: &str) -> Result<()> { } async fn roundtrip(sql: &str) -> Result<()> { - let mut ctx = create_context().await?; + let ctx = create_context().await?; let df = ctx.sql(sql).await?; let plan = df.into_optimized_plan()?; let proto = to_substrait_plan(&plan, &ctx)?; - let plan2 = from_substrait_plan(&mut ctx, &proto).await?; + let plan2 = from_substrait_plan(&ctx, &proto).await?; let plan2 = ctx.state().optimize(&plan2)?; println!("{plan:#?}"); @@ -780,11 +780,11 @@ async fn roundtrip(sql: &str) -> Result<()> { } async fn roundtrip_verify_post_join_filter(sql: &str) -> Result<()> { - let mut ctx = create_context().await?; + let ctx = create_context().await?; let df = ctx.sql(sql).await?; let plan = df.into_optimized_plan()?; let proto = to_substrait_plan(&plan, &ctx)?; - let plan2 = from_substrait_plan(&mut ctx, &proto).await?; + let plan2 = from_substrait_plan(&ctx, &proto).await?; let plan2 = ctx.state().optimize(&plan2)?; println!("{plan:#?}"); @@ -799,11 +799,11 @@ async fn roundtrip_verify_post_join_filter(sql: &str) -> Result<()> { } async fn roundtrip_all_types(sql: &str) -> Result<()> { - let mut ctx = create_all_type_context().await?; + let ctx = create_all_type_context().await?; let df = ctx.sql(sql).await?; let plan = df.into_optimized_plan()?; let proto = to_substrait_plan(&plan, &ctx)?; - let plan2 = from_substrait_plan(&mut ctx, &proto).await?; + let plan2 = from_substrait_plan(&ctx, &proto).await?; let plan2 = ctx.state().optimize(&plan2)?; println!("{plan:#?}"); diff --git a/datafusion/substrait/tests/cases/roundtrip_physical_plan.rs b/datafusion/substrait/tests/cases/roundtrip_physical_plan.rs index 3e5e757e4c39..b64dd2c138fc 100644 --- a/datafusion/substrait/tests/cases/roundtrip_physical_plan.rs +++ b/datafusion/substrait/tests/cases/roundtrip_physical_plan.rs @@ -62,10 +62,10 @@ async fn parquet_exec() -> Result<()> { let substrait_rel = producer::to_substrait_rel(parquet_exec.as_ref(), &mut extension_info)?; - let mut ctx = SessionContext::new(); + let ctx = SessionContext::new(); let parquet_exec_roundtrip = - consumer::from_substrait_rel(&mut ctx, substrait_rel.as_ref(), &HashMap::new()) + consumer::from_substrait_rel(&ctx, substrait_rel.as_ref(), &HashMap::new()) .await?; let expected = format!("{}", displayable(parquet_exec.as_ref()).indent(true)); diff --git a/datafusion/substrait/tests/cases/serialize.rs b/datafusion/substrait/tests/cases/serialize.rs index d6dc5d7e58f2..f6736ca22279 100644 --- a/datafusion/substrait/tests/cases/serialize.rs +++ b/datafusion/substrait/tests/cases/serialize.rs @@ -30,7 +30,7 @@ mod tests { #[tokio::test] async fn serialize_simple_select() -> Result<()> { - let mut ctx = create_context().await?; + let ctx = create_context().await?; let path = "tests/simple_select.bin"; let sql = "SELECT a, b FROM data"; // Test reference @@ -42,7 +42,7 @@ mod tests { // Read substrait plan from file let proto = serializer::deserialize(path).await?; // Check plan equality - let plan = from_substrait_plan(&mut ctx, &proto).await?; + let plan = from_substrait_plan(&ctx, &proto).await?; let plan_str_ref = format!("{plan_ref:?}"); let plan_str = format!("{plan:?}"); assert_eq!(plan_str_ref, plan_str); From f388a2bbce4b50fa41a81613929880af598ccd04 Mon Sep 17 00:00:00 2001 From: Jeffrey <22608443+Jefffrey@users.noreply.github.com> Date: Sun, 29 Oct 2023 23:49:36 +1100 Subject: [PATCH 148/572] Fix crate READMEs (#7964) --- datafusion-examples/Cargo.toml | 2 +- datafusion/common/Cargo.toml | 2 +- datafusion/expr/Cargo.toml | 2 +- datafusion/optimizer/Cargo.toml | 2 +- datafusion/physical-expr/Cargo.toml | 2 +- datafusion/proto/Cargo.toml | 2 +- datafusion/sql/Cargo.toml | 2 +- datafusion/sqllogictest/Cargo.toml | 2 +- datafusion/substrait/Cargo.toml | 2 +- datafusion/wasmtest/Cargo.toml | 2 +- 10 files changed, 10 insertions(+), 10 deletions(-) diff --git a/datafusion-examples/Cargo.toml b/datafusion-examples/Cargo.toml index 8d504f834bc5..7350c4ab981f 100644 --- a/datafusion-examples/Cargo.toml +++ b/datafusion-examples/Cargo.toml @@ -20,9 +20,9 @@ name = "datafusion-examples" description = "DataFusion usage examples" keywords = ["arrow", "query", "sql"] publish = false +readme = "README.md" version = { workspace = true } edition = { workspace = true } -readme = { workspace = true } homepage = { workspace = true } repository = { workspace = true } license = { workspace = true } diff --git a/datafusion/common/Cargo.toml b/datafusion/common/Cargo.toml index 490fbeacad85..87087c50a2d2 100644 --- a/datafusion/common/Cargo.toml +++ b/datafusion/common/Cargo.toml @@ -19,9 +19,9 @@ name = "datafusion-common" description = "Common functionality for DataFusion query engine" keywords = ["arrow", "query", "sql"] +readme = "README.md" version = { workspace = true } edition = { workspace = true } -readme = { workspace = true } homepage = { workspace = true } repository = { workspace = true } license = { workspace = true } diff --git a/datafusion/expr/Cargo.toml b/datafusion/expr/Cargo.toml index c5cf6a1ac11f..a7919a557ad7 100644 --- a/datafusion/expr/Cargo.toml +++ b/datafusion/expr/Cargo.toml @@ -19,9 +19,9 @@ name = "datafusion-expr" description = "Logical plan and expression representation for DataFusion query engine" keywords = ["datafusion", "logical", "plan", "expressions"] +readme = "README.md" version = { workspace = true } edition = { workspace = true } -readme = { workspace = true } homepage = { workspace = true } repository = { workspace = true } license = { workspace = true } diff --git a/datafusion/optimizer/Cargo.toml b/datafusion/optimizer/Cargo.toml index bf786686f474..dce05058b826 100644 --- a/datafusion/optimizer/Cargo.toml +++ b/datafusion/optimizer/Cargo.toml @@ -19,9 +19,9 @@ name = "datafusion-optimizer" description = "DataFusion Query Optimizer" keywords = [ "datafusion", "query", "optimizer" ] +readme = "README.md" version = { workspace = true } edition = { workspace = true } -readme = { workspace = true } homepage = { workspace = true } repository = { workspace = true } license = { workspace = true } diff --git a/datafusion/physical-expr/Cargo.toml b/datafusion/physical-expr/Cargo.toml index 6269f27310a6..f7c0221756fd 100644 --- a/datafusion/physical-expr/Cargo.toml +++ b/datafusion/physical-expr/Cargo.toml @@ -19,9 +19,9 @@ name = "datafusion-physical-expr" description = "Physical expression implementation for DataFusion query engine" keywords = ["arrow", "query", "sql"] +readme = "README.md" version = { workspace = true } edition = { workspace = true } -readme = { workspace = true } homepage = { workspace = true } repository = { workspace = true } license = { workspace = true } diff --git a/datafusion/proto/Cargo.toml b/datafusion/proto/Cargo.toml index 32e10e58a7d7..5e52dadc0b18 100644 --- a/datafusion/proto/Cargo.toml +++ b/datafusion/proto/Cargo.toml @@ -19,9 +19,9 @@ name = "datafusion-proto" description = "Protobuf serialization of DataFusion logical plan expressions" keywords = ["arrow", "query", "sql"] +readme = "README.md" version = { workspace = true } edition = { workspace = true } -readme = { workspace = true } homepage = { workspace = true } repository = { workspace = true } license = { workspace = true } diff --git a/datafusion/sql/Cargo.toml b/datafusion/sql/Cargo.toml index a00a7f021352..fe60feb6ab1f 100644 --- a/datafusion/sql/Cargo.toml +++ b/datafusion/sql/Cargo.toml @@ -19,9 +19,9 @@ name = "datafusion-sql" description = "DataFusion SQL Query Planner" keywords = ["datafusion", "sql", "parser", "planner"] +readme = "README.md" version = { workspace = true } edition = { workspace = true } -readme = { workspace = true } homepage = { workspace = true } repository = { workspace = true } license = { workspace = true } diff --git a/datafusion/sqllogictest/Cargo.toml b/datafusion/sqllogictest/Cargo.toml index 454f99942f52..f1a730351417 100644 --- a/datafusion/sqllogictest/Cargo.toml +++ b/datafusion/sqllogictest/Cargo.toml @@ -21,7 +21,7 @@ edition = { workspace = true } homepage = { workspace = true } license = { workspace = true } name = "datafusion-sqllogictest" -readme = { workspace = true } +readme = "README.md" repository = { workspace = true } rust-version = { workspace = true } version = { workspace = true } diff --git a/datafusion/substrait/Cargo.toml b/datafusion/substrait/Cargo.toml index 7c4ff868cfcd..67d31caca260 100644 --- a/datafusion/substrait/Cargo.toml +++ b/datafusion/substrait/Cargo.toml @@ -18,9 +18,9 @@ [package] name = "datafusion-substrait" description = "DataFusion Substrait Producer and Consumer" +readme = "README.md" version = { workspace = true } edition = { workspace = true } -readme = { workspace = true } homepage = { workspace = true } repository = { workspace = true } license = { workspace = true } diff --git a/datafusion/wasmtest/Cargo.toml b/datafusion/wasmtest/Cargo.toml index e1a9a5d41a5a..3195c989e223 100644 --- a/datafusion/wasmtest/Cargo.toml +++ b/datafusion/wasmtest/Cargo.toml @@ -18,9 +18,9 @@ [package] name = "datafusion-wasmtest" description = "Test library to compile datafusion crates to wasm" +readme = "README.md" version = { workspace = true } edition = { workspace = true } -readme = { workspace = true } homepage = { workspace = true } repository = { workspace = true } license = { workspace = true } From 4a91ce91e2e82ed33142405a00f619cc714a5a30 Mon Sep 17 00:00:00 2001 From: Andrew Lamb Date: Sun, 29 Oct 2023 08:50:10 -0400 Subject: [PATCH 149/572] Minor: Improve `HashJoinExec` documentation (#7953) * Minor: Improve `HashJoinExec` documentation * Apply suggestions from code review Co-authored-by: Liang-Chi Hsieh --------- Co-authored-by: Liang-Chi Hsieh --- .../physical-plan/src/joins/hash_join.rs | 138 ++++++++++++++++-- 1 file changed, 126 insertions(+), 12 deletions(-) diff --git a/datafusion/physical-plan/src/joins/hash_join.rs b/datafusion/physical-plan/src/joins/hash_join.rs index 9aa776fe054c..dc0e81a6f36e 100644 --- a/datafusion/physical-plan/src/joins/hash_join.rs +++ b/datafusion/physical-plan/src/joins/hash_join.rs @@ -15,8 +15,7 @@ // specific language governing permissions and limitations // under the License. -//! Defines the join plan for executing partitions in parallel and then joining the results -//! into a set of partitions. +//! [`HashJoinExec`] Partitioned Hash Join Operator use std::fmt; use std::mem::size_of; @@ -78,29 +77,140 @@ use futures::{ready, Stream, StreamExt, TryStreamExt}; type JoinLeftData = (JoinHashMap, RecordBatch, MemoryReservation); -/// Join execution plan executes partitions in parallel and combines them into a set of -/// partitions. +/// Join execution plan: Evaluates eqijoin predicates in parallel on multiple +/// partitions using a hash table and an optional filter list to apply post +/// join. /// -/// Filter expression expected to contain non-equality predicates that can not be pushed -/// down to any of join inputs. -/// In case of outer join, filter applied to only matched rows. +/// # Join Expressions +/// +/// This implementation is optimized for evaluating eqijoin predicates ( +/// ` = `) expressions, which are represented as a list of `Columns` +/// in [`Self::on`]. +/// +/// Non-equality predicates, which can not pushed down to a join inputs (e.g. +/// ` != `) are known as "filter expressions" and are evaluated +/// after the equijoin predicates. +/// +/// # "Build Side" vs "Probe Side" +/// +/// HashJoin takes two inputs, which are referred to as the "build" and the +/// "probe". The build side is the first child, and the probe side is the second +/// child. +/// +/// The two inputs are treated differently and it is VERY important that the +/// *smaller* input is placed on the build side to minimize the work of creating +/// the hash table. +/// +/// ```text +/// ┌───────────┐ +/// │ HashJoin │ +/// │ │ +/// └───────────┘ +/// │ │ +/// ┌─────┘ └─────┐ +/// ▼ ▼ +/// ┌────────────┐ ┌─────────────┐ +/// │ Input │ │ Input │ +/// │ [0] │ │ [1] │ +/// └────────────┘ └─────────────┘ +/// +/// "build side" "probe side" +/// ``` +/// +/// Execution proceeds in 2 stages: +/// +/// 1. the **build phase** where a hash table is created from the tuples of the +/// build side. +/// +/// 2. the **probe phase** where the tuples of the probe side are streamed +/// through, checking for matches of the join keys in the hash table. +/// +/// ```text +/// ┌────────────────┐ ┌────────────────┐ +/// │ ┌─────────┐ │ │ ┌─────────┐ │ +/// │ │ Hash │ │ │ │ Hash │ │ +/// │ │ Table │ │ │ │ Table │ │ +/// │ │(keys are│ │ │ │(keys are│ │ +/// │ │equi join│ │ │ │equi join│ │ Stage 2: batches from +/// Stage 1: the │ │columns) │ │ │ │columns) │ │ the probe side are +/// *entire* build │ │ │ │ │ │ │ │ streamed through, and +/// side is read │ └─────────┘ │ │ └─────────┘ │ checked against the +/// into the hash │ ▲ │ │ ▲ │ contents of the hash +/// table │ HashJoin │ │ HashJoin │ table +/// └──────┼─────────┘ └──────────┼─────┘ +/// ─ ─ ─ ─ ─ ─ ─ ─ ─ ─ ─ ─ ─ +/// │ │ +/// +/// │ │ +/// ┌────────────┐ ┌────────────┐ +/// │RecordBatch │ │RecordBatch │ +/// └────────────┘ └────────────┘ +/// ┌────────────┐ ┌────────────┐ +/// │RecordBatch │ │RecordBatch │ +/// └────────────┘ └────────────┘ +/// ... ... +/// ┌────────────┐ ┌────────────┐ +/// │RecordBatch │ │RecordBatch │ +/// └────────────┘ └────────────┘ +/// +/// build side probe side +/// +/// ``` +/// +/// # Example "Optimal" Plans +/// +/// The differences in the inputs means that for classic "Star Schema Query", +/// the optimal plan will be a **"Right Deep Tree"** . A Star Schema Query is +/// one where there is one large table and several smaller "dimension" tables, +/// joined on `Foreign Key = Primary Key` predicates. +/// +/// A "Right Deep Tree" looks like this large table as the probe side on the +/// lowest join: +/// +/// ```text +/// ┌───────────┐ +/// │ HashJoin │ +/// │ │ +/// └───────────┘ +/// │ │ +/// ┌───────┘ └──────────┐ +/// ▼ ▼ +/// ┌───────────────┐ ┌───────────┐ +/// │ small table 1 │ │ HashJoin │ +/// │ "dimension" │ │ │ +/// └───────────────┘ └───┬───┬───┘ +/// ┌──────────┘ └───────┐ +/// │ │ +/// ▼ ▼ +/// ┌───────────────┐ ┌───────────┐ +/// │ small table 2 │ │ HashJoin │ +/// │ "dimension" │ │ │ +/// └───────────────┘ └───┬───┬───┘ +/// ┌────────┘ └────────┐ +/// │ │ +/// ▼ ▼ +/// ┌───────────────┐ ┌───────────────┐ +/// │ small table 3 │ │ large table │ +/// │ "dimension" │ │ "fact" │ +/// └───────────────┘ └───────────────┘ +/// ``` #[derive(Debug)] pub struct HashJoinExec { /// left (build) side which gets hashed pub left: Arc, /// right (probe) side which are filtered by the hash table pub right: Arc, - /// Set of common columns used to join on + /// Set of equijoin columns from the relations: `(left_col, right_col)` pub on: Vec<(Column, Column)>, /// Filters which are applied while finding matching rows pub filter: Option, - /// How the join is performed + /// How the join is performed (`OUTER`, `INNER`, etc) pub join_type: JoinType, - /// The schema once the join is applied + /// The output schema for the join schema: SchemaRef, /// Build-side data left_fut: OnceAsync, - /// Shares the `RandomState` for the hashing algorithm + /// Shared the `RandomState` for the hashing algorithm random_state: RandomState, /// Output order output_order: Option>, @@ -110,12 +220,16 @@ pub struct HashJoinExec { metrics: ExecutionPlanMetricsSet, /// Information of index and left / right placement of columns column_indices: Vec, - /// If null_equals_null is true, null == null else null != null + /// Null matching behavior: If `null_equals_null` is true, rows that have + /// `null`s in both left and right equijoin columns will be matched. + /// Otherwise, rows that have `null`s in the join columns will not be + /// matched and thus will not appear in the output. pub null_equals_null: bool, } impl HashJoinExec { /// Tries to create a new [HashJoinExec]. + /// /// # Error /// This function errors when it is not possible to join the left and right sides on keys `on`. pub fn try_new( From 9b45967edc6dba312ea223464dad3e66604d2095 Mon Sep 17 00:00:00 2001 From: Alex Huang Date: Sun, 29 Oct 2023 22:45:34 +0100 Subject: [PATCH 150/572] chore: clean useless clone baesd on clippy (#7973) --- .../core/src/physical_optimizer/combine_partial_final_agg.rs | 2 +- datafusion/core/src/physical_optimizer/enforce_distribution.rs | 2 +- datafusion/core/src/physical_optimizer/topk_aggregation.rs | 2 +- datafusion/physical-expr/src/array_expressions.rs | 2 +- datafusion/substrait/src/logical_plan/producer.rs | 2 +- 5 files changed, 5 insertions(+), 5 deletions(-) diff --git a/datafusion/core/src/physical_optimizer/combine_partial_final_agg.rs b/datafusion/core/src/physical_optimizer/combine_partial_final_agg.rs index 838ae613683e..2c4e929788df 100644 --- a/datafusion/core/src/physical_optimizer/combine_partial_final_agg.rs +++ b/datafusion/core/src/physical_optimizer/combine_partial_final_agg.rs @@ -93,7 +93,7 @@ impl PhysicalOptimizerRule for CombinePartialFinalAggregate { input_agg_exec.filter_expr().to_vec(), input_agg_exec.order_by_expr().to_vec(), input_agg_exec.input().clone(), - input_agg_exec.input_schema().clone(), + input_agg_exec.input_schema(), ) .ok() .map(Arc::new) diff --git a/datafusion/core/src/physical_optimizer/enforce_distribution.rs b/datafusion/core/src/physical_optimizer/enforce_distribution.rs index 7b91dce32aa9..12df9efbbca6 100644 --- a/datafusion/core/src/physical_optimizer/enforce_distribution.rs +++ b/datafusion/core/src/physical_optimizer/enforce_distribution.rs @@ -554,7 +554,7 @@ fn reorder_aggregate_keys( agg_exec.filter_expr().to_vec(), agg_exec.order_by_expr().to_vec(), partial_agg, - agg_exec.input_schema().clone(), + agg_exec.input_schema(), )?); // Need to create a new projection to change the expr ordering back diff --git a/datafusion/core/src/physical_optimizer/topk_aggregation.rs b/datafusion/core/src/physical_optimizer/topk_aggregation.rs index 572e796a8ba7..e0a8da82e35f 100644 --- a/datafusion/core/src/physical_optimizer/topk_aggregation.rs +++ b/datafusion/core/src/physical_optimizer/topk_aggregation.rs @@ -75,7 +75,7 @@ impl TopKAggregation { aggr.filter_expr().to_vec(), aggr.order_by_expr().to_vec(), aggr.input().clone(), - aggr.input_schema().clone(), + aggr.input_schema(), ) .expect("Unable to copy Aggregate!") .with_limit(Some(limit)); diff --git a/datafusion/physical-expr/src/array_expressions.rs b/datafusion/physical-expr/src/array_expressions.rs index 7077f8b59860..84fd301b84de 100644 --- a/datafusion/physical-expr/src/array_expressions.rs +++ b/datafusion/physical-expr/src/array_expressions.rs @@ -811,7 +811,7 @@ fn concat_internal(args: &[ArrayRef]) -> Result { } } // Assume all arrays have the same data type - let data_type = list_arrays[0].value_type().clone(); + let data_type = list_arrays[0].value_type(); let buffer = valid.finish(); let elements = arrays diff --git a/datafusion/substrait/src/logical_plan/producer.rs b/datafusion/substrait/src/logical_plan/producer.rs index 757bddf9fe58..e3c6f94d43d5 100644 --- a/datafusion/substrait/src/logical_plan/producer.rs +++ b/datafusion/substrait/src/logical_plan/producer.rs @@ -326,7 +326,7 @@ pub fn to_substrait_rel( left: Some(left), right: Some(right), r#type: join_type as i32, - expression: join_expr.clone(), + expression: join_expr, post_join_filter: None, advanced_extension: None, }))), From 806a9631d70a1e4be5021e017304ee2ce5d4b7fc Mon Sep 17 00:00:00 2001 From: Andrew Lamb Date: Mon, 30 Oct 2023 09:17:02 -0400 Subject: [PATCH 151/572] Add README.md to `core`, `execution` and `physical-plan` crates (#7970) MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit * Add README.md to `core`, `execution` and `physical-plan` crates * prettier * Update datafusion/physical-plan/README.md * Update datafusion/wasmtest/README.md --------- Co-authored-by: Daniël Heres --- datafusion/core/README.md | 26 ++++++++++++++++++++++++++ datafusion/execution/README.md | 26 ++++++++++++++++++++++++++ datafusion/physical-plan/README.md | 27 +++++++++++++++++++++++++++ datafusion/wasmtest/README.md | 11 +++++++++-- 4 files changed, 88 insertions(+), 2 deletions(-) create mode 100644 datafusion/core/README.md create mode 100644 datafusion/execution/README.md create mode 100644 datafusion/physical-plan/README.md diff --git a/datafusion/core/README.md b/datafusion/core/README.md new file mode 100644 index 000000000000..c83e33991a07 --- /dev/null +++ b/datafusion/core/README.md @@ -0,0 +1,26 @@ + + +# DataFusion Common + +[DataFusion](df) is an extensible query execution framework, written in Rust, that uses Apache Arrow as its in-memory format. + +This crate contains the main entrypoints and high level DataFusion APIs such as SessionContext, and DataFrame and ListingTable. + +[df]: https://crates.io/crates/datafusion diff --git a/datafusion/execution/README.md b/datafusion/execution/README.md new file mode 100644 index 000000000000..40b3ffaa3ed3 --- /dev/null +++ b/datafusion/execution/README.md @@ -0,0 +1,26 @@ + + +# DataFusion Common + +[DataFusion](df) is an extensible query execution framework, written in Rust, that uses Apache Arrow as its in-memory format. + +This crate is a submodule of DataFusion that provides execution runtime such as the memory pools and disk manager. + +[df]: https://crates.io/crates/datafusion diff --git a/datafusion/physical-plan/README.md b/datafusion/physical-plan/README.md new file mode 100644 index 000000000000..3632ea7d2a26 --- /dev/null +++ b/datafusion/physical-plan/README.md @@ -0,0 +1,27 @@ + + +# DataFusion Common + +[DataFusion](df) is an extensible query execution framework, written in Rust, that uses Apache Arrow as its in-memory format. + +This crate is a submodule of DataFusion that contains the `ExecutionPlan` trait and the various implementations of that +trait for built in operators such as filters, projections, joins, aggregations, etc. + +[df]: https://crates.io/crates/datafusion diff --git a/datafusion/wasmtest/README.md b/datafusion/wasmtest/README.md index 5dc7bb2de45d..21e505f38e02 100644 --- a/datafusion/wasmtest/README.md +++ b/datafusion/wasmtest/README.md @@ -17,9 +17,16 @@ under the License. --> -## wasmtest +# DataFusion wasmtest + +[DataFusion](df) is an extensible query execution framework, written in Rust, that uses Apache Arrow as its in-memory format. + +This crate is a submodule of DataFusion used to verify that various DataFusion crates compile successfully to the +`wasm32-unknown-unknown` target with wasm-pack. -Library crate to verify that various DataFusion crates compile successfully to the `wasm32-unknown-unknown` target with wasm-pack. +[df]: https://crates.io/crates/datafusion + +## wasmtest Some of DataFusion's downstream projects compile to WASM to run in the browser. Doing so requires special care that certain library dependencies are not included in DataFusion. From fdb54549301ccbede0b48788ff0fa057dd799d63 Mon Sep 17 00:00:00 2001 From: Andrew Lamb Date: Mon, 30 Oct 2023 13:17:04 -0400 Subject: [PATCH 152/572] Move source repartitioning into `ExecutionPlan::repartition` (#7936) * Move source repartitioning into ExecutionPlan::repartition * cleanup * update test * update test * refine docs * fix merge --- .../core/src/datasource/physical_plan/csv.rs | 58 +++++------ .../core/src/datasource/physical_plan/mod.rs | 96 +++++++++---------- .../src/datasource/physical_plan/parquet.rs | 41 ++++---- .../enforce_distribution.rs | 28 ++---- datafusion/physical-plan/src/lib.rs | 30 +++++- 5 files changed, 130 insertions(+), 123 deletions(-) diff --git a/datafusion/core/src/datasource/physical_plan/csv.rs b/datafusion/core/src/datasource/physical_plan/csv.rs index 8117e101ea99..82163da64af8 100644 --- a/datafusion/core/src/datasource/physical_plan/csv.rs +++ b/datafusion/core/src/datasource/physical_plan/csv.rs @@ -46,6 +46,7 @@ use datafusion_physical_expr::{ }; use bytes::{Buf, Bytes}; +use datafusion_common::config::ConfigOptions; use futures::{ready, StreamExt, TryStreamExt}; use object_store::{GetOptions, GetResultPayload, ObjectStore}; use tokio::io::AsyncWriteExt; @@ -117,34 +118,6 @@ impl CsvExec { pub fn escape(&self) -> Option { self.escape } - - /// Redistribute files across partitions according to their size - /// See comments on `repartition_file_groups()` for more detail. - /// - /// Return `None` if can't get repartitioned(empty/compressed file). - pub fn get_repartitioned( - &self, - target_partitions: usize, - repartition_file_min_size: usize, - ) -> Option { - // Parallel execution on compressed CSV file is not supported yet. - if self.file_compression_type.is_compressed() { - return None; - } - - let repartitioned_file_groups_option = FileScanConfig::repartition_file_groups( - self.base_config.file_groups.clone(), - target_partitions, - repartition_file_min_size, - ); - - if let Some(repartitioned_file_groups) = repartitioned_file_groups_option { - let mut new_plan = self.clone(); - new_plan.base_config.file_groups = repartitioned_file_groups; - return Some(new_plan); - } - None - } } impl DisplayAs for CsvExec { @@ -205,6 +178,35 @@ impl ExecutionPlan for CsvExec { Ok(self) } + /// Redistribute files across partitions according to their size + /// See comments on `repartition_file_groups()` for more detail. + /// + /// Return `None` if can't get repartitioned(empty/compressed file). + fn repartitioned( + &self, + target_partitions: usize, + config: &ConfigOptions, + ) -> Result>> { + let repartition_file_min_size = config.optimizer.repartition_file_min_size; + // Parallel execution on compressed CSV file is not supported yet. + if self.file_compression_type.is_compressed() { + return Ok(None); + } + + let repartitioned_file_groups_option = FileScanConfig::repartition_file_groups( + self.base_config.file_groups.clone(), + target_partitions, + repartition_file_min_size, + ); + + if let Some(repartitioned_file_groups) = repartitioned_file_groups_option { + let mut new_plan = self.clone(); + new_plan.base_config.file_groups = repartitioned_file_groups; + return Ok(Some(Arc::new(new_plan))); + } + Ok(None) + } + fn execute( &self, partition: usize, diff --git a/datafusion/core/src/datasource/physical_plan/mod.rs b/datafusion/core/src/datasource/physical_plan/mod.rs index 6643e4127dbd..ea0a9698ff5c 100644 --- a/datafusion/core/src/datasource/physical_plan/mod.rs +++ b/datafusion/core/src/datasource/physical_plan/mod.rs @@ -527,6 +527,7 @@ mod tests { }; use arrow_schema::Field; use chrono::Utc; + use datafusion_common::config::ConfigOptions; use crate::physical_plan::{DefaultDisplay, VerboseDisplay}; @@ -828,11 +829,7 @@ mod tests { None, ); - let partitioned_file = parquet_exec - .get_repartitioned(4, 0) - .base_config() - .file_groups - .clone(); + let partitioned_file = repartition_with_size(&parquet_exec, 4, 0); assert!(partitioned_file[0][0].range.is_none()); } @@ -893,13 +890,8 @@ mod tests { None, ); - let actual = file_groups_to_vec( - parquet_exec - .get_repartitioned(n_partition, 10) - .base_config() - .file_groups - .clone(), - ); + let actual = + repartition_with_size_to_vec(&parquet_exec, n_partition, 10); assert_eq!(expected, &actual); } @@ -927,13 +919,7 @@ mod tests { None, ); - let actual = file_groups_to_vec( - parquet_exec - .get_repartitioned(4, 10) - .base_config() - .file_groups - .clone(), - ); + let actual = repartition_with_size_to_vec(&parquet_exec, 4, 10); let expected = vec![ (0, "a".to_string(), 0, 31), (1, "a".to_string(), 31, 62), @@ -964,13 +950,7 @@ mod tests { None, ); - let actual = file_groups_to_vec( - parquet_exec - .get_repartitioned(96, 5) - .base_config() - .file_groups - .clone(), - ); + let actual = repartition_with_size_to_vec(&parquet_exec, 96, 5); let expected = vec![ (0, "a".to_string(), 0, 1), (1, "a".to_string(), 1, 2), @@ -1007,13 +987,7 @@ mod tests { None, ); - let actual = file_groups_to_vec( - parquet_exec - .get_repartitioned(3, 10) - .base_config() - .file_groups - .clone(), - ); + let actual = repartition_with_size_to_vec(&parquet_exec, 3, 10); let expected = vec![ (0, "a".to_string(), 0, 34), (1, "a".to_string(), 34, 40), @@ -1046,13 +1020,7 @@ mod tests { None, ); - let actual = file_groups_to_vec( - parquet_exec - .get_repartitioned(2, 10) - .base_config() - .file_groups - .clone(), - ); + let actual = repartition_with_size_to_vec(&parquet_exec, 2, 10); let expected = vec![ (0, "a".to_string(), 0, 40), (0, "b".to_string(), 0, 10), @@ -1086,11 +1054,7 @@ mod tests { None, ); - let actual = parquet_exec - .get_repartitioned(65, 10) - .base_config() - .file_groups - .clone(); + let actual = repartition_with_size(&parquet_exec, 65, 10); assert_eq!(2, actual.len()); } @@ -1115,17 +1079,47 @@ mod tests { None, ); - let actual = parquet_exec - .get_repartitioned(65, 500) + let actual = repartition_with_size(&parquet_exec, 65, 500); + assert_eq!(1, actual.len()); + } + + /// Calls `ParquetExec.repartitioned` with the specified + /// `target_partitions` and `repartition_file_min_size`, returning the + /// resulting `PartitionedFile`s + fn repartition_with_size( + parquet_exec: &ParquetExec, + target_partitions: usize, + repartition_file_min_size: usize, + ) -> Vec> { + let mut config = ConfigOptions::new(); + config.optimizer.repartition_file_min_size = repartition_file_min_size; + + parquet_exec + .repartitioned(target_partitions, &config) + .unwrap() // unwrap Result + .unwrap() // unwrap Option + .as_any() + .downcast_ref::() + .unwrap() .base_config() .file_groups - .clone(); - assert_eq!(1, actual.len()); + .clone() } - fn file_groups_to_vec( - file_groups: Vec>, + /// Calls `repartition_with_size` and returns a tuple for each output `PartitionedFile`: + /// + /// `(partition index, file path, start, end)` + fn repartition_with_size_to_vec( + parquet_exec: &ParquetExec, + target_partitions: usize, + repartition_file_min_size: usize, ) -> Vec<(usize, String, i64, i64)> { + let file_groups = repartition_with_size( + parquet_exec, + target_partitions, + repartition_file_min_size, + ); + file_groups .iter() .enumerate() diff --git a/datafusion/core/src/datasource/physical_plan/parquet.rs b/datafusion/core/src/datasource/physical_plan/parquet.rs index 3a2459bec817..f6e999f60249 100644 --- a/datafusion/core/src/datasource/physical_plan/parquet.rs +++ b/datafusion/core/src/datasource/physical_plan/parquet.rs @@ -259,26 +259,6 @@ impl ParquetExec { self.enable_bloom_filter .unwrap_or(config_options.execution.parquet.bloom_filter_enabled) } - - /// Redistribute files across partitions according to their size - /// See comments on `get_file_groups_repartitioned()` for more detail. - pub fn get_repartitioned( - &self, - target_partitions: usize, - repartition_file_min_size: usize, - ) -> Self { - let repartitioned_file_groups_option = FileScanConfig::repartition_file_groups( - self.base_config.file_groups.clone(), - target_partitions, - repartition_file_min_size, - ); - - let mut new_plan = self.clone(); - if let Some(repartitioned_file_groups) = repartitioned_file_groups_option { - new_plan.base_config.file_groups = repartitioned_file_groups; - } - new_plan - } } impl DisplayAs for ParquetExec { @@ -349,6 +329,27 @@ impl ExecutionPlan for ParquetExec { Ok(self) } + /// Redistribute files across partitions according to their size + /// See comments on `get_file_groups_repartitioned()` for more detail. + fn repartitioned( + &self, + target_partitions: usize, + config: &ConfigOptions, + ) -> Result>> { + let repartition_file_min_size = config.optimizer.repartition_file_min_size; + let repartitioned_file_groups_option = FileScanConfig::repartition_file_groups( + self.base_config.file_groups.clone(), + target_partitions, + repartition_file_min_size, + ); + + let mut new_plan = self.clone(); + if let Some(repartitioned_file_groups) = repartitioned_file_groups_option { + new_plan.base_config.file_groups = repartitioned_file_groups; + } + Ok(Some(Arc::new(new_plan))) + } + fn execute( &self, partition_index: usize, diff --git a/datafusion/core/src/physical_optimizer/enforce_distribution.rs b/datafusion/core/src/physical_optimizer/enforce_distribution.rs index 12df9efbbca6..6de39db7d52a 100644 --- a/datafusion/core/src/physical_optimizer/enforce_distribution.rs +++ b/datafusion/core/src/physical_optimizer/enforce_distribution.rs @@ -26,9 +26,6 @@ use std::fmt::Formatter; use std::sync::Arc; use crate::config::ConfigOptions; -use crate::datasource::physical_plan::CsvExec; -#[cfg(feature = "parquet")] -use crate::datasource::physical_plan::ParquetExec; use crate::error::Result; use crate::physical_optimizer::utils::{ add_sort_above, get_children_exectrees, get_plan_string, is_coalesce_partitions, @@ -1188,7 +1185,6 @@ fn ensure_distribution( // When `false`, round robin repartition will not be added to increase parallelism let enable_round_robin = config.optimizer.enable_round_robin_repartition; let repartition_file_scans = config.optimizer.repartition_file_scans; - let repartition_file_min_size = config.optimizer.repartition_file_min_size; let batch_size = config.execution.batch_size; let is_unbounded = unbounded_output(&dist_context.plan); // Use order preserving variants either of the conditions true @@ -1265,25 +1261,13 @@ fn ensure_distribution( // Unless partitioning doesn't increase the partition count, it is not beneficial: && child.output_partitioning().partition_count() < target_partitions { - // When `repartition_file_scans` is set, leverage source operators - // (`ParquetExec`, `CsvExec` etc.) to increase parallelism at the source. + // When `repartition_file_scans` is set, attempt to increase + // parallelism at the source. if repartition_file_scans { - #[cfg(feature = "parquet")] - if let Some(parquet_exec) = - child.as_any().downcast_ref::() + if let Some(new_child) = + child.repartitioned(target_partitions, config)? { - child = Arc::new(parquet_exec.get_repartitioned( - target_partitions, - repartition_file_min_size, - )); - } - if let Some(csv_exec) = child.as_any().downcast_ref::() { - if let Some(csv_exec) = csv_exec.get_repartitioned( - target_partitions, - repartition_file_min_size, - ) { - child = Arc::new(csv_exec); - } + child = new_child; } } // Increase parallelism by adding round-robin repartitioning @@ -1644,8 +1628,8 @@ mod tests { use crate::datasource::file_format::file_compression_type::FileCompressionType; use crate::datasource::listing::PartitionedFile; use crate::datasource::object_store::ObjectStoreUrl; - use crate::datasource::physical_plan::FileScanConfig; use crate::datasource::physical_plan::ParquetExec; + use crate::datasource::physical_plan::{CsvExec, FileScanConfig}; use crate::physical_optimizer::enforce_sorting::EnforceSorting; use crate::physical_optimizer::output_requirements::OutputRequirements; use crate::physical_plan::aggregates::{ diff --git a/datafusion/physical-plan/src/lib.rs b/datafusion/physical-plan/src/lib.rs index b2f81579f8e8..3ada2fa163fd 100644 --- a/datafusion/physical-plan/src/lib.rs +++ b/datafusion/physical-plan/src/lib.rs @@ -76,6 +76,7 @@ pub use crate::metrics::Metric; pub use crate::topk::TopK; pub use crate::visitor::{accept, visit_execution_plan, ExecutionPlanVisitor}; +use datafusion_common::config::ConfigOptions; pub use datafusion_common::hash_utils; pub use datafusion_common::utils::project_schema; pub use datafusion_common::{internal_err, ColumnStatistics, Statistics}; @@ -209,7 +210,32 @@ pub trait ExecutionPlan: Debug + DisplayAs + Send + Sync { children: Vec>, ) -> Result>; - /// creates an iterator + /// If supported, attempt to increase the partitioning of this `ExecutionPlan` to + /// produce `target_partitions` partitions. + /// + /// If the `ExecutionPlan` does not support changing its partitioning, + /// returns `Ok(None)` (the default). + /// + /// It is the `ExecutionPlan` can increase its partitioning, but not to the + /// `target_partitions`, it may return an ExecutionPlan with fewer + /// partitions. This might happen, for example, if each new partition would + /// be too small to be efficiently processed individually. + /// + /// The DataFusion optimizer attempts to use as many threads as possible by + /// repartitioning its inputs to match the target number of threads + /// available (`target_partitions`). Some data sources, such as the built in + /// CSV and Parquet readers, implement this method as they are able to read + /// from their input files in parallel, regardless of how the source data is + /// split amongst files. + fn repartitioned( + &self, + _target_partitions: usize, + _config: &ConfigOptions, + ) -> Result>> { + Ok(None) + } + + /// Begin execution of `partition`, returning a stream of [`RecordBatch`]es. fn execute( &self, partition: usize, @@ -217,7 +243,7 @@ pub trait ExecutionPlan: Debug + DisplayAs + Send + Sync { ) -> Result; /// Return a snapshot of the set of [`Metric`]s for this - /// [`ExecutionPlan`]. + /// [`ExecutionPlan`]. If no `Metric`s are available, return None. /// /// While the values of the metrics in the returned /// [`MetricsSet`]s may change as execution progresses, the From 3fd8a2082e5910c2c1cf8e17b9596914633626db Mon Sep 17 00:00:00 2001 From: Jonah Gao Date: Tue, 31 Oct 2023 01:34:05 +0800 Subject: [PATCH 153/572] minor: fix broken links in README.md (#7986) * minor: fix broken links in README.md * fix proto link --- datafusion/common/README.md | 2 +- datafusion/core/README.md | 2 +- datafusion/execution/README.md | 2 +- datafusion/expr/README.md | 2 +- datafusion/optimizer/README.md | 2 +- datafusion/physical-expr/README.md | 2 +- datafusion/physical-plan/README.md | 2 +- datafusion/proto/README.md | 2 +- datafusion/sql/README.md | 2 +- datafusion/wasmtest/README.md | 2 +- 10 files changed, 10 insertions(+), 10 deletions(-) diff --git a/datafusion/common/README.md b/datafusion/common/README.md index 9bccf3f18b7f..524ab4420d2a 100644 --- a/datafusion/common/README.md +++ b/datafusion/common/README.md @@ -19,7 +19,7 @@ # DataFusion Common -[DataFusion](df) is an extensible query execution framework, written in Rust, that uses Apache Arrow as its in-memory format. +[DataFusion][df] is an extensible query execution framework, written in Rust, that uses Apache Arrow as its in-memory format. This crate is a submodule of DataFusion that provides common data types and utilities. diff --git a/datafusion/core/README.md b/datafusion/core/README.md index c83e33991a07..5a9493d086cd 100644 --- a/datafusion/core/README.md +++ b/datafusion/core/README.md @@ -19,7 +19,7 @@ # DataFusion Common -[DataFusion](df) is an extensible query execution framework, written in Rust, that uses Apache Arrow as its in-memory format. +[DataFusion][df] is an extensible query execution framework, written in Rust, that uses Apache Arrow as its in-memory format. This crate contains the main entrypoints and high level DataFusion APIs such as SessionContext, and DataFrame and ListingTable. diff --git a/datafusion/execution/README.md b/datafusion/execution/README.md index 40b3ffaa3ed3..67aac6be82b3 100644 --- a/datafusion/execution/README.md +++ b/datafusion/execution/README.md @@ -19,7 +19,7 @@ # DataFusion Common -[DataFusion](df) is an extensible query execution framework, written in Rust, that uses Apache Arrow as its in-memory format. +[DataFusion][df] is an extensible query execution framework, written in Rust, that uses Apache Arrow as its in-memory format. This crate is a submodule of DataFusion that provides execution runtime such as the memory pools and disk manager. diff --git a/datafusion/expr/README.md b/datafusion/expr/README.md index bcce30be39d9..b086f930e871 100644 --- a/datafusion/expr/README.md +++ b/datafusion/expr/README.md @@ -19,7 +19,7 @@ # DataFusion Logical Plan and Expressions -[DataFusion](df) is an extensible query execution framework, written in Rust, that uses Apache Arrow as its in-memory format. +[DataFusion][df] is an extensible query execution framework, written in Rust, that uses Apache Arrow as its in-memory format. This crate is a submodule of DataFusion that provides data types and utilities for logical plans and expressions. diff --git a/datafusion/optimizer/README.md b/datafusion/optimizer/README.md index c8baae03efa2..b8e5b93e6692 100644 --- a/datafusion/optimizer/README.md +++ b/datafusion/optimizer/README.md @@ -19,7 +19,7 @@ # DataFusion Query Optimizer -[DataFusion](df) is an extensible query execution framework, written in Rust, that uses Apache Arrow as its in-memory +[DataFusion][df] is an extensible query execution framework, written in Rust, that uses Apache Arrow as its in-memory format. DataFusion has modular design, allowing individual crates to be re-used in other projects. diff --git a/datafusion/physical-expr/README.md b/datafusion/physical-expr/README.md index a887d3eb29fe..424256c77e7e 100644 --- a/datafusion/physical-expr/README.md +++ b/datafusion/physical-expr/README.md @@ -19,7 +19,7 @@ # DataFusion Physical Expressions -[DataFusion](df) is an extensible query execution framework, written in Rust, that uses Apache Arrow as its in-memory format. +[DataFusion][df] is an extensible query execution framework, written in Rust, that uses Apache Arrow as its in-memory format. This crate is a submodule of DataFusion that provides data types and utilities for physical expressions. diff --git a/datafusion/physical-plan/README.md b/datafusion/physical-plan/README.md index 3632ea7d2a26..366a6b555150 100644 --- a/datafusion/physical-plan/README.md +++ b/datafusion/physical-plan/README.md @@ -19,7 +19,7 @@ # DataFusion Common -[DataFusion](df) is an extensible query execution framework, written in Rust, that uses Apache Arrow as its in-memory format. +[DataFusion][df] is an extensible query execution framework, written in Rust, that uses Apache Arrow as its in-memory format. This crate is a submodule of DataFusion that contains the `ExecutionPlan` trait and the various implementations of that trait for built in operators such as filters, projections, joins, aggregations, etc. diff --git a/datafusion/proto/README.md b/datafusion/proto/README.md index fd66d54aa2de..171aadb744d6 100644 --- a/datafusion/proto/README.md +++ b/datafusion/proto/README.md @@ -19,7 +19,7 @@ # DataFusion Proto -[DataFusion](df) is an extensible query execution framework, written in Rust, that uses Apache Arrow as its in-memory format. +[DataFusion][df] is an extensible query execution framework, written in Rust, that uses Apache Arrow as its in-memory format. This crate is a submodule of DataFusion that provides a protocol buffer format for representing query plans and expressions. diff --git a/datafusion/sql/README.md b/datafusion/sql/README.md index 2ad994e4eba5..256fa774b410 100644 --- a/datafusion/sql/README.md +++ b/datafusion/sql/README.md @@ -20,7 +20,7 @@ # DataFusion SQL Query Planner This crate provides a general purpose SQL query planner that can parse SQL and translate queries into logical -plans. Although this crate is used by the [DataFusion](df) query engine, it was designed to be easily usable from any +plans. Although this crate is used by the [DataFusion][df] query engine, it was designed to be easily usable from any project that requires a SQL query planner and does not make any assumptions about how the resulting logical plan will be translated to a physical plan. For example, there is no concept of row-based versus columnar execution in the logical plan. diff --git a/datafusion/wasmtest/README.md b/datafusion/wasmtest/README.md index 21e505f38e02..d26369a18ab9 100644 --- a/datafusion/wasmtest/README.md +++ b/datafusion/wasmtest/README.md @@ -19,7 +19,7 @@ # DataFusion wasmtest -[DataFusion](df) is an extensible query execution framework, written in Rust, that uses Apache Arrow as its in-memory format. +[DataFusion][df] is an extensible query execution framework, written in Rust, that uses Apache Arrow as its in-memory format. This crate is a submodule of DataFusion used to verify that various DataFusion crates compile successfully to the `wasm32-unknown-unknown` target with wasm-pack. From 7ee2c0bd8d33412dbc8fd3298254904e78b5156f Mon Sep 17 00:00:00 2001 From: Andrew Lamb Date: Mon, 30 Oct 2023 13:37:10 -0400 Subject: [PATCH 154/572] Minor: Upate the `sqllogictest` crate README (#7971) * Minor: Upate the sqllogictest crate README * prettier * Apply suggestions from code review Co-authored-by: Jonah Gao Co-authored-by: jakevin --------- Co-authored-by: Jonah Gao Co-authored-by: jakevin --- datafusion/sqllogictest/README.md | 41 ++++++++++++++++++------------- 1 file changed, 24 insertions(+), 17 deletions(-) diff --git a/datafusion/sqllogictest/README.md b/datafusion/sqllogictest/README.md index 3e94859d35a7..0349ed852f46 100644 --- a/datafusion/sqllogictest/README.md +++ b/datafusion/sqllogictest/README.md @@ -17,19 +17,26 @@ under the License. --> -#### Overview +# DataFusion sqllogictest -This is the Datafusion implementation of [sqllogictest](https://www.sqlite.org/sqllogictest/doc/trunk/about.wiki). We -use [sqllogictest-rs](https://github.com/risinglightdb/sqllogictest-rs) as a parser/runner of `.slt` files -in [`test_files`](test_files). +[DataFusion][df] is an extensible query execution framework, written in Rust, that uses Apache Arrow as its in-memory format. -#### Testing setup +This crate is a submodule of DataFusion that contains an implementation of [sqllogictest](https://www.sqlite.org/sqllogictest/doc/trunk/about.wiki). + +[df]: https://crates.io/crates/datafusion + +## Overview + +This crate uses [sqllogictest-rs](https://github.com/risinglightdb/sqllogictest-rs) to parse and run `.slt` files in the +[`test_files`](test_files) directory of this crate. + +## Testing setup 1. `rustup update stable` DataFusion uses the latest stable release of rust 2. `git submodule init` 3. `git submodule update` -#### Running tests: TLDR Examples +## Running tests: TLDR Examples ```shell # Run all tests @@ -56,7 +63,7 @@ cargo test --test sqllogictests -- ddl --complete RUST_LOG=debug cargo test --test sqllogictests -- ddl ``` -#### Cookbook: Adding Tests +## Cookbook: Adding Tests 1. Add queries @@ -95,11 +102,11 @@ SELECT * from foo; Assuming it looks good, check it in! -#### Reference +# Reference -#### Running tests: Validation Mode +## Running tests: Validation Mode -In this model, `sqllogictests` runs the statements and queries in a `.slt` file, comparing the expected output in the +In this mode, `sqllogictests` runs the statements and queries in a `.slt` file, comparing the expected output in the file to the output produced by that run. For example, to run all tests suites in validation mode @@ -115,10 +122,10 @@ sqllogictests also supports `cargo test` style substring matches on file names t cargo test --test sqllogictests -- information ``` -#### Running tests: Postgres compatibility +## Running tests: Postgres compatibility Test files that start with prefix `pg_compat_` verify compatibility -with Postgres by running the same script files both with DataFusion and with Posgres +with Postgres by running the same script files both with DataFusion and with Postgres In order to run the sqllogictests running against a previously running Postgres instance, do: @@ -145,7 +152,7 @@ docker run \ postgres ``` -#### Running Tests: `tpch` +## Running Tests: `tpch` Test files in `tpch` directory runs against the `TPCH` data set (SF = 0.1), which must be generated before running. You can use following @@ -165,7 +172,7 @@ Then you need to add `INCLUDE_TPCH=true` to run tpch tests: INCLUDE_TPCH=true cargo test --test sqllogictests ``` -#### Updating tests: Completion Mode +## Updating tests: Completion Mode In test script completion mode, `sqllogictests` reads a prototype script and runs the statements and queries against the database engine. The output is a full script that is a copy of the prototype script with result inserted. @@ -177,7 +184,7 @@ You can update the tests / generate expected output by passing the `--complete` cargo test --test sqllogictests -- ddl --complete ``` -#### Running tests: `scratchdir` +## Running tests: `scratchdir` The DataFusion sqllogictest runner automatically creates a directory named `test_files/scratch/`, creating it if needed and @@ -190,7 +197,7 @@ Tests that need to write temporary files should write (only) to this directory to ensure they do not interfere with others concurrently running tests. -#### `.slt` file format +## `.slt` file format [`sqllogictest`] was originally written for SQLite to verify the correctness of SQL queries against the SQLite engine. The format is designed @@ -247,7 +254,7 @@ query > :warning: It is encouraged to either apply `order by`, or use `rowsort` for queries without explicit `order by` > clauses. -##### Example +### Example ```sql # group_by_distinct From bb1d7f9343532d5fa8df871ff42000fbe836d7d7 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E5=BC=A0=E6=9E=97=E4=BC=9F?= Date: Tue, 31 Oct 2023 01:40:34 +0800 Subject: [PATCH 155/572] Improve MemoryCatalogProvider default impl block placement (#7975) --- datafusion/core/src/catalog/mod.rs | 12 ++++++------ 1 file changed, 6 insertions(+), 6 deletions(-) diff --git a/datafusion/core/src/catalog/mod.rs b/datafusion/core/src/catalog/mod.rs index fe5bdc0ec6a9..ce27d57da00d 100644 --- a/datafusion/core/src/catalog/mod.rs +++ b/datafusion/core/src/catalog/mod.rs @@ -93,12 +93,6 @@ impl CatalogList for MemoryCatalogList { } } -impl Default for MemoryCatalogProvider { - fn default() -> Self { - Self::new() - } -} - /// Represents a catalog, comprising a number of named schemas. pub trait CatalogProvider: Sync + Send { /// Returns the catalog provider as [`Any`] @@ -161,6 +155,12 @@ impl MemoryCatalogProvider { } } +impl Default for MemoryCatalogProvider { + fn default() -> Self { + Self::new() + } +} + impl CatalogProvider for MemoryCatalogProvider { fn as_any(&self) -> &dyn Any { self From 448dff519c2c46189b5909b542f15df36766544b Mon Sep 17 00:00:00 2001 From: Liang-Chi Hsieh Date: Mon, 30 Oct 2023 13:23:00 -0700 Subject: [PATCH 156/572] Fix `ScalarValue` handling of NULL values for ListArray (#7969) * Fix try_from_array data type for NULL value in ListArray * Fix * Explicitly assert the datatype * For review --- datafusion/common/src/scalar.rs | 125 +++++++++++++++---- datafusion/sqllogictest/test_files/array.slt | 11 ++ 2 files changed, 111 insertions(+), 25 deletions(-) diff --git a/datafusion/common/src/scalar.rs b/datafusion/common/src/scalar.rs index be24e2b933b5..f9b0cbdf22de 100644 --- a/datafusion/common/src/scalar.rs +++ b/datafusion/common/src/scalar.rs @@ -1312,10 +1312,11 @@ impl ScalarValue { Arc::new(ListArray::from_iter_primitive::<$ARRAY_TY, _, _>( scalars.into_iter().map(|x| match x { ScalarValue::List(arr) => { - if arr.as_any().downcast_ref::().is_some() { + // `ScalarValue::List` contains a single element `ListArray`. + let list_arr = as_list_array(&arr); + if list_arr.is_null(0) { None } else { - let list_arr = as_list_array(&arr); let primitive_arr = list_arr.values().as_primitive::<$ARRAY_TY>(); Some( @@ -1339,12 +1340,14 @@ impl ScalarValue { for scalar in scalars.into_iter() { match scalar { ScalarValue::List(arr) => { - if arr.as_any().downcast_ref::().is_some() { + // `ScalarValue::List` contains a single element `ListArray`. + let list_arr = as_list_array(&arr); + + if list_arr.is_null(0) { builder.append(false); continue; } - let list_arr = as_list_array(&arr); let string_arr = $STRING_ARRAY(list_arr.values()); for v in string_arr.iter() { @@ -1699,15 +1702,16 @@ impl ScalarValue { for scalar in scalars { if let ScalarValue::List(arr) = scalar { - // i.e. NullArray(1) - if arr.as_any().downcast_ref::().is_some() { + // `ScalarValue::List` contains a single element `ListArray`. + let list_arr = as_list_array(&arr); + + if list_arr.is_null(0) { // Repeat previous offset index offsets.push(0); // Element is null valid.append(false); } else { - let list_arr = as_list_array(&arr); let arr = list_arr.values().to_owned(); offsets.push(arr.len()); elements.push(arr); @@ -2234,28 +2238,20 @@ impl ScalarValue { } DataType::Utf8 => typed_cast!(array, index, StringArray, Utf8), DataType::LargeUtf8 => typed_cast!(array, index, LargeStringArray, LargeUtf8), - DataType::List(nested_type) => { + DataType::List(_) => { let list_array = as_list_array(array); - let arr = match list_array.is_null(index) { - true => new_null_array(nested_type.data_type(), 0), - false => { - let nested_array = list_array.value(index); - Arc::new(wrap_into_list_array(nested_array)) - } - }; + let nested_array = list_array.value(index); + // Produces a single element `ListArray` with the value at `index`. + let arr = Arc::new(wrap_into_list_array(nested_array)); ScalarValue::List(arr) } // TODO: There is no test for FixedSizeList now, add it later - DataType::FixedSizeList(nested_type, _len) => { + DataType::FixedSizeList(_, _) => { let list_array = as_fixed_size_list_array(array)?; - let arr = match list_array.is_null(index) { - true => new_null_array(nested_type.data_type(), 0), - false => { - let nested_array = list_array.value(index); - Arc::new(wrap_into_list_array(nested_array)) - } - }; + let nested_array = list_array.value(index); + // Produces a single element `ListArray` with the value at `index`. + let arr = Arc::new(wrap_into_list_array(nested_array)); ScalarValue::List(arr) } @@ -2944,8 +2940,15 @@ impl TryFrom<&DataType> for ScalarValue { index_type.clone(), Box::new(value_type.as_ref().try_into()?), ), - DataType::List(_) => ScalarValue::List(new_null_array(&DataType::Null, 0)), - + // `ScalaValue::List` contains single element `ListArray`. + DataType::List(field) => ScalarValue::List(new_null_array( + &DataType::List(Arc::new(Field::new( + "item", + field.data_type().clone(), + true, + ))), + 1, + )), DataType::Struct(fields) => ScalarValue::Struct(None, fields.clone()), DataType::Null => ScalarValue::Null, _ => { @@ -3885,6 +3888,78 @@ mod tests { ); } + #[test] + fn scalar_try_from_array_list_array_null() { + let list = ListArray::from_iter_primitive::(vec![ + Some(vec![Some(1), Some(2)]), + None, + ]); + + let non_null_list_scalar = ScalarValue::try_from_array(&list, 0).unwrap(); + let null_list_scalar = ScalarValue::try_from_array(&list, 1).unwrap(); + + let data_type = + DataType::List(Arc::new(Field::new("item", DataType::Int32, true))); + + assert_eq!(non_null_list_scalar.data_type(), data_type.clone()); + assert_eq!(null_list_scalar.data_type(), data_type); + } + + #[test] + fn scalar_try_from_list() { + let data_type = + DataType::List(Arc::new(Field::new("item", DataType::Int32, true))); + let data_type = &data_type; + let scalar: ScalarValue = data_type.try_into().unwrap(); + + let expected = ScalarValue::List(new_null_array( + &DataType::List(Arc::new(Field::new("item", DataType::Int32, true))), + 1, + )); + + assert_eq!(expected, scalar) + } + + #[test] + fn scalar_try_from_list_of_list() { + let data_type = DataType::List(Arc::new(Field::new( + "item", + DataType::List(Arc::new(Field::new("item", DataType::Int32, true))), + true, + ))); + let data_type = &data_type; + let scalar: ScalarValue = data_type.try_into().unwrap(); + + let expected = ScalarValue::List(new_null_array( + &DataType::List(Arc::new(Field::new( + "item", + DataType::List(Arc::new(Field::new("item", DataType::Int32, true))), + true, + ))), + 1, + )); + + assert_eq!(expected, scalar) + } + + #[test] + fn scalar_try_from_not_equal_list_nested_list() { + let list_data_type = + DataType::List(Arc::new(Field::new("item", DataType::Int32, true))); + let data_type = &list_data_type; + let list_scalar: ScalarValue = data_type.try_into().unwrap(); + + let nested_list_data_type = DataType::List(Arc::new(Field::new( + "item", + DataType::List(Arc::new(Field::new("item", DataType::Int32, true))), + true, + ))); + let data_type = &nested_list_data_type; + let nested_list_scalar: ScalarValue = data_type.try_into().unwrap(); + + assert_ne!(list_scalar, nested_list_scalar); + } + #[test] fn scalar_try_from_dict_datatype() { let data_type = diff --git a/datafusion/sqllogictest/test_files/array.slt b/datafusion/sqllogictest/test_files/array.slt index 621cb4a8f4c0..b5601a22226c 100644 --- a/datafusion/sqllogictest/test_files/array.slt +++ b/datafusion/sqllogictest/test_files/array.slt @@ -209,6 +209,17 @@ AS VALUES (make_array([28, 29, 30], [31, 32, 33], [34, 35, 36], [28, 29, 30], [31, 32, 33], [34, 35, 36], [28, 29, 30], [31, 32, 33], [34, 35, 36], [28, 29, 30]), [28, 29, 30], [37, 38, 39], 10) ; +query TTT +select arrow_typeof(column1), arrow_typeof(column2), arrow_typeof(column3) from arrays; +---- +List(Field { name: "item", data_type: List(Field { name: "item", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }) List(Field { name: "item", data_type: Float64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }) List(Field { name: "item", data_type: Utf8, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }) +List(Field { name: "item", data_type: List(Field { name: "item", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }) List(Field { name: "item", data_type: Float64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }) List(Field { name: "item", data_type: Utf8, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }) +List(Field { name: "item", data_type: List(Field { name: "item", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }) List(Field { name: "item", data_type: Float64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }) List(Field { name: "item", data_type: Utf8, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }) +List(Field { name: "item", data_type: List(Field { name: "item", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }) List(Field { name: "item", data_type: Float64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }) List(Field { name: "item", data_type: Utf8, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }) +List(Field { name: "item", data_type: List(Field { name: "item", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }) List(Field { name: "item", data_type: Float64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }) List(Field { name: "item", data_type: Utf8, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }) +List(Field { name: "item", data_type: List(Field { name: "item", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }) List(Field { name: "item", data_type: Float64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }) List(Field { name: "item", data_type: Utf8, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }) +List(Field { name: "item", data_type: List(Field { name: "item", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }) List(Field { name: "item", data_type: Float64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }) List(Field { name: "item", data_type: Utf8, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }) + # arrays table query ??? select column1, column2, column3 from arrays; From 0d4dc3601b390c50640e841e70c522cd733e02f4 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Berkay=20=C5=9Eahin?= <124376117+berkaysynnada@users.noreply.github.com> Date: Tue, 31 Oct 2023 00:01:09 +0300 Subject: [PATCH 157/572] Refactor of Ordering and Prunability Traversals and States (#7985) * simplify ExprOrdering * Comment improvements * Move map/transform comment up --------- Co-authored-by: Mehmet Ozan Kabak --- .../physical-expr/src/sort_properties.rs | 59 ++++++------ datafusion/physical-expr/src/utils.rs | 90 ++++++++++++++++++- 2 files changed, 120 insertions(+), 29 deletions(-) diff --git a/datafusion/physical-expr/src/sort_properties.rs b/datafusion/physical-expr/src/sort_properties.rs index 097f491cb979..8ae3379218fb 100644 --- a/datafusion/physical-expr/src/sort_properties.rs +++ b/datafusion/physical-expr/src/sort_properties.rs @@ -155,37 +155,36 @@ impl Neg for SortProperties { #[derive(Debug)] pub struct ExprOrdering { pub expr: Arc, - pub state: Option, - pub children_states: Option>, + pub state: SortProperties, + pub children_states: Vec, } impl ExprOrdering { + /// Creates a new [`ExprOrdering`] with [`SortProperties::Unordered`] states + /// for `expr` and its children. pub fn new(expr: Arc) -> Self { + let size = expr.children().len(); Self { expr, - state: None, - children_states: None, + state: SortProperties::Unordered, + children_states: vec![SortProperties::Unordered; size], } } - pub fn children(&self) -> Vec { + /// Updates this [`ExprOrdering`]'s children states with the given states. + pub fn with_new_children(mut self, children_states: Vec) -> Self { + self.children_states = children_states; + self + } + + /// Creates new [`ExprOrdering`] objects for each child of the expression. + pub fn children_expr_orderings(&self) -> Vec { self.expr .children() .into_iter() .map(ExprOrdering::new) .collect() } - - pub fn new_with_children( - children_states: Vec, - parent_expr: Arc, - ) -> Self { - Self { - expr: parent_expr, - state: None, - children_states: Some(children_states), - } - } } impl TreeNode for ExprOrdering { @@ -193,7 +192,7 @@ impl TreeNode for ExprOrdering { where F: FnMut(&Self) -> Result, { - for child in self.children() { + for child in self.children_expr_orderings() { match op(&child)? { VisitRecursion::Continue => {} VisitRecursion::Skip => return Ok(VisitRecursion::Continue), @@ -207,17 +206,20 @@ impl TreeNode for ExprOrdering { where F: FnMut(Self) -> Result, { - let children = self.children(); - if children.is_empty() { + if self.children_states.is_empty() { Ok(self) } else { - Ok(ExprOrdering::new_with_children( - children + let child_expr_orderings = self.children_expr_orderings(); + // After mapping over the children, the function `F` applies to the + // current object and updates its state. + Ok(self.with_new_children( + child_expr_orderings .into_iter() + // Update children states after this transformation: .map(transform) - .map_ok(|c| c.state.unwrap_or(SortProperties::Unordered)) + // Extract the state (i.e. sort properties) information: + .map_ok(|c| c.state) .collect::>>()?, - self.expr, )) } } @@ -248,13 +250,13 @@ pub fn update_ordering( // a BinaryExpr like a + b), and there is an ordering equivalence of // it (let's say like c + d), we actually can find it at this step. if sort_expr.expr.eq(&node.expr) { - node.state = Some(SortProperties::Ordered(sort_expr.options)); + node.state = SortProperties::Ordered(sort_expr.options); return Ok(Transformed::Yes(node)); } - if let Some(children_sort_options) = &node.children_states { + if !node.expr.children().is_empty() { // We have an intermediate (non-leaf) node, account for its children: - node.state = Some(node.expr.get_ordering(children_sort_options)); + node.state = node.expr.get_ordering(&node.children_states); } else if let Some(column) = node.expr.as_any().downcast_ref::() { // We have a Column, which is one of the two possible leaf node types: node.state = get_indices_of_matching_sort_exprs_with_order_eq( @@ -268,10 +270,11 @@ pub fn update_ordering( descending: sort_options[0].descending, nulls_first: sort_options[0].nulls_first, }) - }); + }) + .unwrap_or(SortProperties::Unordered); } else { // We have a Literal, which is the other possible leaf node type: - node.state = Some(node.expr.get_ordering(&[])); + node.state = node.expr.get_ordering(&[]); } Ok(Transformed::Yes(node)) } diff --git a/datafusion/physical-expr/src/utils.rs b/datafusion/physical-expr/src/utils.rs index b2a6bb5ca6d2..b38117d206cc 100644 --- a/datafusion/physical-expr/src/utils.rs +++ b/datafusion/physical-expr/src/utils.rs @@ -773,7 +773,7 @@ pub fn find_orderings_of_exprs( &input_ordering_equal_properties, ) })?; - if let Some(SortProperties::Ordered(sort_options)) = transformed.state { + if let SortProperties::Ordered(sort_options) = transformed.state { orderings.push(Some(PhysicalSortExpr { expr: Arc::new(Column::new(name, index)), options: sort_options, @@ -1836,4 +1836,92 @@ mod tests { Ok(()) } + + #[test] + fn test_find_orderings_of_exprs() -> Result<()> { + let schema = Schema::new(vec![ + Field::new("a", DataType::Int32, true), + Field::new("b", DataType::Int32, true), + Field::new("c", DataType::Int32, true), + Field::new("d", DataType::Int32, true), + ]); + + let mut eq = EquivalenceProperties::new(Arc::new(schema.clone())); + let col_a = &col("a", &schema)?; + let col_b = &col("b", &schema)?; + let col_c = &col("c", &schema)?; + let col_d = &col("d", &schema)?; + let option_asc = SortOptions { + descending: false, + nulls_first: false, + }; + // b=a (e.g they are aliases) + eq.add_equal_conditions((&Column::new("b", 1), &Column::new("a", 0))); + let mut oeq = OrderingEquivalenceProperties::new(Arc::new(schema.clone())); + // [b ASC], [d ASC] + oeq.add_equal_conditions(( + &vec![PhysicalSortExpr { + expr: col_b.clone(), + options: option_asc, + }], + &vec![PhysicalSortExpr { + expr: col_d.clone(), + options: option_asc, + }], + )); + + let orderings = find_orderings_of_exprs( + &[ + // d + b + ( + Arc::new(BinaryExpr::new( + col_d.clone(), + Operator::Plus, + col_b.clone(), + )), + "d+b".to_string(), + ), + // b as b_new + (col_b.clone(), "b_new".to_string()), + // a as a_new + (col_a.clone(), "a_new".to_string()), + // a + c + ( + Arc::new(BinaryExpr::new( + col_a.clone(), + Operator::Plus, + col_c.clone(), + )), + "a+c".to_string(), + ), + ], + Some(&[PhysicalSortExpr { + expr: col_b.clone(), + options: option_asc, + }]), + eq, + oeq, + )?; + + assert_eq!( + vec![ + Some(PhysicalSortExpr { + expr: Arc::new(Column::new("d+b", 0)), + options: option_asc, + }), + Some(PhysicalSortExpr { + expr: Arc::new(Column::new("b_new", 1)), + options: option_asc, + }), + Some(PhysicalSortExpr { + expr: Arc::new(Column::new("a_new", 2)), + options: option_asc, + }), + None, + ], + orderings + ); + + Ok(()) + } } From 3d78bf471f8b2a66c4298e1aca230bc87e7b4c25 Mon Sep 17 00:00:00 2001 From: Liang-Chi Hsieh Date: Tue, 31 Oct 2023 00:05:03 -0700 Subject: [PATCH 158/572] Keep output as scalar for scalar function if all inputs are scalar (#7967) * Keep output as scalar for scalar function if all inputs are scalar * Add end-to-end tests --- datafusion/physical-expr/src/functions.rs | 11 ++++- datafusion/physical-expr/src/planner.rs | 34 +++++++++++++ datafusion/sqllogictest/test_files/scalar.slt | 48 +++++++++++++++++++ 3 files changed, 92 insertions(+), 1 deletion(-) diff --git a/datafusion/physical-expr/src/functions.rs b/datafusion/physical-expr/src/functions.rs index 8422862043ae..b66bac41014d 100644 --- a/datafusion/physical-expr/src/functions.rs +++ b/datafusion/physical-expr/src/functions.rs @@ -357,6 +357,8 @@ where ColumnarValue::Array(a) => Some(a.len()), }); + let is_scalar = len.is_none(); + let inferred_length = len.unwrap_or(1); let args = args .iter() @@ -373,7 +375,14 @@ where .collect::>(); let result = (inner)(&args); - result.map(ColumnarValue::Array) + + if is_scalar { + // If all inputs are scalar, keeps output as scalar + let result = result.and_then(|arr| ScalarValue::try_from_array(&arr, 0)); + result.map(ColumnarValue::Scalar) + } else { + result.map(ColumnarValue::Array) + } }) } diff --git a/datafusion/physical-expr/src/planner.rs b/datafusion/physical-expr/src/planner.rs index 9a74c2ca64d1..64c1d0be0455 100644 --- a/datafusion/physical-expr/src/planner.rs +++ b/datafusion/physical-expr/src/planner.rs @@ -448,3 +448,37 @@ pub fn create_physical_expr( } } } + +#[cfg(test)] +mod tests { + use super::*; + use arrow_array::{ArrayRef, BooleanArray, RecordBatch, StringArray}; + use arrow_schema::{DataType, Field, Schema}; + use datafusion_common::{DFSchema, Result}; + use datafusion_expr::{col, left, Literal}; + + #[test] + fn test_create_physical_expr_scalar_input_output() -> Result<()> { + let expr = col("letter").eq(left("APACHE".lit(), 1i64.lit())); + + let schema = Schema::new(vec![Field::new("letter", DataType::Utf8, false)]); + let df_schema = DFSchema::try_from_qualified_schema("data", &schema)?; + let p = create_physical_expr(&expr, &df_schema, &schema, &ExecutionProps::new())?; + + let batch = RecordBatch::try_new( + Arc::new(schema), + vec![Arc::new(StringArray::from_iter_values(vec![ + "A", "B", "C", "D", + ]))], + )?; + let result = p.evaluate(&batch)?; + let result = result.into_array(4); + + assert_eq!( + &result, + &(Arc::new(BooleanArray::from(vec![true, false, false, false,])) as ArrayRef) + ); + + Ok(()) + } +} diff --git a/datafusion/sqllogictest/test_files/scalar.slt b/datafusion/sqllogictest/test_files/scalar.slt index e5c1a828492a..ecb7fe13fcf4 100644 --- a/datafusion/sqllogictest/test_files/scalar.slt +++ b/datafusion/sqllogictest/test_files/scalar.slt @@ -1878,3 +1878,51 @@ query T SELECT CONCAT('Hello', 'World') ---- HelloWorld + +statement ok +CREATE TABLE simple_string( + letter STRING, + letter2 STRING +) as VALUES + ('A', 'APACHE'), + ('B', 'APACHE'), + ('C', 'APACHE'), + ('D', 'APACHE') +; + +query TT +EXPLAIN SELECT letter, letter = LEFT('APACHE', 1) FROM simple_string; +---- +logical_plan +Projection: simple_string.letter, simple_string.letter = Utf8("A") AS simple_string.letter = left(Utf8("APACHE"),Int64(1)) +--TableScan: simple_string projection=[letter] +physical_plan +ProjectionExec: expr=[letter@0 as letter, letter@0 = A as simple_string.letter = left(Utf8("APACHE"),Int64(1))] +--MemoryExec: partitions=1, partition_sizes=[1] + +query TB +SELECT letter, letter = LEFT('APACHE', 1) FROM simple_string; + ---- +---- +A true +B false +C false +D false + +query TT +EXPLAIN SELECT letter, letter = LEFT(letter2, 1) FROM simple_string; +---- +logical_plan +Projection: simple_string.letter, simple_string.letter = left(simple_string.letter2, Int64(1)) +--TableScan: simple_string projection=[letter, letter2] +physical_plan +ProjectionExec: expr=[letter@0 as letter, letter@0 = left(letter2@1, 1) as simple_string.letter = left(simple_string.letter2,Int64(1))] +--MemoryExec: partitions=1, partition_sizes=[1] + +query TB +SELECT letter, letter = LEFT(letter2, 1) FROM simple_string; +---- +A true +B false +C false +D false From d8e413c0b92b86593ed9801a034bd62bdb9ddc0b Mon Sep 17 00:00:00 2001 From: Jeffrey <22608443+Jefffrey@users.noreply.github.com> Date: Tue, 31 Oct 2023 19:11:42 +1100 Subject: [PATCH 159/572] Fix crate READMEs for core, execution, physical-plan (#7990) --- datafusion/core/Cargo.toml | 2 +- datafusion/execution/Cargo.toml | 2 +- datafusion/physical-plan/Cargo.toml | 2 +- 3 files changed, 3 insertions(+), 3 deletions(-) diff --git a/datafusion/core/Cargo.toml b/datafusion/core/Cargo.toml index 5f9d28bd620b..a9614934673b 100644 --- a/datafusion/core/Cargo.toml +++ b/datafusion/core/Cargo.toml @@ -20,9 +20,9 @@ name = "datafusion" description = "DataFusion is an in-memory query engine that uses Apache Arrow as the memory model" keywords = ["arrow", "query", "sql"] include = ["benches/*.rs", "src/**/*.rs", "Cargo.toml"] +readme = "README.md" version = { workspace = true } edition = { workspace = true } -readme = { workspace = true } homepage = { workspace = true } repository = { workspace = true } license = { workspace = true } diff --git a/datafusion/execution/Cargo.toml b/datafusion/execution/Cargo.toml index 6ae8bccdae38..54a2a6d8f3df 100644 --- a/datafusion/execution/Cargo.toml +++ b/datafusion/execution/Cargo.toml @@ -19,9 +19,9 @@ name = "datafusion-execution" description = "Execution configuration support for DataFusion query engine" keywords = ["arrow", "query", "sql"] +readme = "README.md" version = { workspace = true } edition = { workspace = true } -readme = { workspace = true } homepage = { workspace = true } repository = { workspace = true } license = { workspace = true } diff --git a/datafusion/physical-plan/Cargo.toml b/datafusion/physical-plan/Cargo.toml index 2dfcf12e350a..033d2aa187dd 100644 --- a/datafusion/physical-plan/Cargo.toml +++ b/datafusion/physical-plan/Cargo.toml @@ -19,9 +19,9 @@ name = "datafusion-physical-plan" description = "Physical (ExecutionPlan) implementations for DataFusion query engine" keywords = ["arrow", "query", "sql"] +readme = "README.md" version = { workspace = true } edition = { workspace = true } -readme = { workspace = true } homepage = { workspace = true } repository = { workspace = true } license = { workspace = true } From 8fcc5e03147db93d688c2dfa112653191476f2bf Mon Sep 17 00:00:00 2001 From: jakevin Date: Tue, 31 Oct 2023 17:43:52 +0800 Subject: [PATCH 160/572] Update sqlparser requirement from 0.38.0 to 0.39.0 (#7983) * chore: Update sqlparser requirement from 0.38.0 to 0.39.0 * support FILTER Aggregates --- Cargo.toml | 2 +- datafusion-cli/Cargo.lock | 363 ++++++++++++++-------------- datafusion/sql/src/expr/function.rs | 7 +- datafusion/sql/src/expr/mod.rs | 35 ++- datafusion/sql/src/planner.rs | 16 +- datafusion/sql/src/set_expr.rs | 3 + datafusion/sql/src/statement.rs | 15 ++ 7 files changed, 252 insertions(+), 189 deletions(-) diff --git a/Cargo.toml b/Cargo.toml index 77e3c6038ea7..5aa830ddff0f 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -55,7 +55,7 @@ arrow-buffer = { version = "48.0.0", default-features = false } arrow-flight = { version = "48.0.0", features = ["flight-sql-experimental"] } arrow-schema = { version = "48.0.0", default-features = false } parquet = { version = "48.0.0", features = ["arrow", "async", "object_store"] } -sqlparser = { version = "0.38.0", features = ["visitor"] } +sqlparser = { version = "0.39.0", features = ["visitor"] } chrono = { version = "0.4.31", default-features = false } [profile.release] diff --git a/datafusion-cli/Cargo.lock b/datafusion-cli/Cargo.lock index b83088f94c57..268f9e28427a 100644 --- a/datafusion-cli/Cargo.lock +++ b/datafusion-cli/Cargo.lock @@ -25,15 +25,16 @@ checksum = "aae1277d39aeec15cb388266ecc24b11c80469deae6067e17a1a7aa9e5c1f234" [[package]] name = "ahash" -version = "0.8.3" +version = "0.8.6" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "2c99f64d1e06488f620f932677e24bc6e2897582980441ae90a671415bd7ec2f" +checksum = "91429305e9f0a25f6205c5b8e0d2db09e0708a7a6df0f42212bb56c32c8ac97a" dependencies = [ "cfg-if", "const-random", "getrandom", "once_cell", "version_check", + "zerocopy", ] [[package]] @@ -106,8 +107,8 @@ dependencies = [ "serde", "serde_json", "snap", - "strum 0.25.0", - "strum_macros 0.25.2", + "strum", + "strum_macros", "thiserror", "typed-builder", "uuid", @@ -177,7 +178,7 @@ dependencies = [ "chrono", "chrono-tz", "half", - "hashbrown 0.14.1", + "hashbrown 0.14.2", "num", ] @@ -302,7 +303,7 @@ dependencies = [ "arrow-data", "arrow-schema", "half", - "hashbrown 0.14.1", + "hashbrown 0.14.2", ] [[package]] @@ -358,9 +359,9 @@ dependencies = [ [[package]] name = "async-compression" -version = "0.4.3" +version = "0.4.4" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "bb42b2197bf15ccb092b62c74515dbd8b86d0effd934795f6687c93b6e679a2c" +checksum = "f658e2baef915ba0f26f1f7c42bfb8e12f532a01f449a090ded75ae7a07e9ba2" dependencies = [ "bzip2", "flate2", @@ -370,15 +371,15 @@ dependencies = [ "pin-project-lite", "tokio", "xz2", - "zstd 0.12.4", - "zstd-safe 6.0.6", + "zstd 0.13.0", + "zstd-safe 7.0.0", ] [[package]] name = "async-trait" -version = "0.1.73" +version = "0.1.74" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "bc00ceb34980c03614e35a3a4e218276a0a824e911d07651cd0d858a51e8c0f0" +checksum = "a66537f1bb974b254c98ed142ff995236e81b9d0fe4db0575f46612cb15eb0f9" dependencies = [ "proc-macro2", "quote", @@ -709,9 +710,9 @@ dependencies = [ [[package]] name = "base64" -version = "0.21.4" +version = "0.21.5" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "9ba43ea6f343b788c8764558649e08df62f86c6ef251fdaeb1ffd010a9ae50a2" +checksum = "35636a1494ede3b646cc98f74f8e62c773a38a659ebc777a2cf26b9b74171df9" [[package]] name = "base64-simd" @@ -731,9 +732,9 @@ checksum = "bef38d45163c2f1dde094a7dfd33ccf595c92905c8f8f4fdc18d06fb1037718a" [[package]] name = "bitflags" -version = "2.4.0" +version = "2.4.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "b4682ae6287fcf752ecaabbfcc7b6f9b72aa33933dc23a554d853aea8eea8635" +checksum = "327762f6e5a765692301e5bb513e0d9fef63be86bbc14528052b1cd3e6f03e07" [[package]] name = "blake2" @@ -779,9 +780,9 @@ dependencies = [ [[package]] name = "brotli-decompressor" -version = "2.5.0" +version = "2.5.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "da74e2b81409b1b743f8f0c62cc6254afefb8b8e50bbfe3735550f7aeefa3448" +checksum = "4e2e4afe60d7dd600fdd3de8d0f08c2b7ec039712e3b6137ff98b7004e82de4f" dependencies = [ "alloc-no-stdlib", "alloc-stdlib", @@ -878,9 +879,9 @@ dependencies = [ [[package]] name = "chrono-tz" -version = "0.8.3" +version = "0.8.4" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "f1369bc6b9e9a7dfdae2055f6ec151fe9c554a9d23d357c0237cee2e25eaabb7" +checksum = "e23185c0e21df6ed832a12e2bda87c7d1def6842881fb634a8511ced741b0d76" dependencies = [ "chrono", "chrono-tz-build", @@ -889,9 +890,9 @@ dependencies = [ [[package]] name = "chrono-tz-build" -version = "0.2.0" +version = "0.2.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "e2f5ebdc942f57ed96d560a6d1a459bae5851102a25d5bf89dc04ae453e31ecf" +checksum = "433e39f13c9a060046954e0592a8d0a4bcb1040125cbf91cb8ee58964cfb350f" dependencies = [ "parse-zoneinfo", "phf", @@ -950,34 +951,32 @@ dependencies = [ [[package]] name = "comfy-table" -version = "7.0.1" +version = "7.1.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "9ab77dbd8adecaf3f0db40581631b995f312a8a5ae3aa9993188bb8f23d83a5b" +checksum = "7c64043d6c7b7a4c58e39e7efccfdea7b93d885a795d0c054a69dbbf4dd52686" dependencies = [ - "strum 0.24.1", - "strum_macros 0.24.3", + "strum", + "strum_macros", "unicode-width", ] [[package]] name = "const-random" -version = "0.1.15" +version = "0.1.16" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "368a7a772ead6ce7e1de82bfb04c485f3db8ec744f72925af5735e29a22cc18e" +checksum = "11df32a13d7892ec42d51d3d175faba5211ffe13ed25d4fb348ac9e9ce835593" dependencies = [ "const-random-macro", - "proc-macro-hack", ] [[package]] name = "const-random-macro" -version = "0.1.15" +version = "0.1.16" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "9d7d6ab3c3a2282db210df5f02c4dab6e0a7057af0fb7ebd4070f30fe05c0ddb" +checksum = "f9d839f2a20b0aee515dc581a6172f2321f96cab76c1a38a4c584a194955390e" dependencies = [ "getrandom", "once_cell", - "proc-macro-hack", "tiny-keccak", ] @@ -1014,9 +1013,9 @@ dependencies = [ [[package]] name = "cpufeatures" -version = "0.2.9" +version = "0.2.11" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "a17b76ff3a4162b0b27f354a0c87015ddad39d35f9c0c36607a3bdd175dde1f1" +checksum = "ce420fe07aecd3e67c5f910618fe65e94158f6dcc0adf44e00d69ce2bdfe0fd0" dependencies = [ "libc", ] @@ -1090,7 +1089,7 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "978747c1d849a7d2ee5e8adc0159961c48fb7e5db2f06af6723b80123bb53856" dependencies = [ "cfg-if", - "hashbrown 0.14.1", + "hashbrown 0.14.2", "lock_api", "once_cell", "parking_lot_core", @@ -1122,7 +1121,7 @@ dependencies = [ "futures", "glob", "half", - "hashbrown 0.14.1", + "hashbrown 0.14.2", "indexmap 2.0.2", "itertools", "log", @@ -1197,7 +1196,7 @@ dependencies = [ "datafusion-common", "datafusion-expr", "futures", - "hashbrown 0.14.1", + "hashbrown 0.14.2", "log", "object_store", "parking_lot", @@ -1215,8 +1214,8 @@ dependencies = [ "arrow-array", "datafusion-common", "sqlparser", - "strum 0.25.0", - "strum_macros 0.25.2", + "strum", + "strum_macros", ] [[package]] @@ -1229,7 +1228,7 @@ dependencies = [ "datafusion-common", "datafusion-expr", "datafusion-physical-expr", - "hashbrown 0.14.1", + "hashbrown 0.14.2", "itertools", "log", "regex-syntax", @@ -1251,7 +1250,7 @@ dependencies = [ "datafusion-common", "datafusion-expr", "half", - "hashbrown 0.14.1", + "hashbrown 0.14.2", "hex", "indexmap 2.0.2", "itertools", @@ -1284,7 +1283,7 @@ dependencies = [ "datafusion-physical-expr", "futures", "half", - "hashbrown 0.14.1", + "hashbrown 0.14.2", "indexmap 2.0.2", "itertools", "log", @@ -1310,9 +1309,12 @@ dependencies = [ [[package]] name = "deranged" -version = "0.3.8" +version = "0.3.9" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "f2696e8a945f658fd14dc3b87242e6b80cd0f36ff04ea560fa39082368847946" +checksum = "0f32d04922c60427da6f9fef14d042d9edddef64cb9d4ce0d64d0685fbeb1fd3" +dependencies = [ + "powerfmt", +] [[package]] name = "difflib" @@ -1482,9 +1484,9 @@ dependencies = [ [[package]] name = "flate2" -version = "1.0.27" +version = "1.0.28" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "c6c98ee8095e9d1dcbf2fcc6d95acccb90d1c81db1e44725c6a984b1dbdfb010" +checksum = "46303f565772937ffe1d394a4fac6f411c6013172fadde9dcdb1e147a086940e" dependencies = [ "crc32fast", "miniz_oxide", @@ -1516,9 +1518,9 @@ dependencies = [ [[package]] name = "futures" -version = "0.3.28" +version = "0.3.29" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "23342abe12aba583913b2e62f22225ff9c950774065e4bfb61a19cd9770fec40" +checksum = "da0290714b38af9b4a7b094b8a37086d1b4e61f2df9122c3cad2577669145335" dependencies = [ "futures-channel", "futures-core", @@ -1531,9 +1533,9 @@ dependencies = [ [[package]] name = "futures-channel" -version = "0.3.28" +version = "0.3.29" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "955518d47e09b25bbebc7a18df10b81f0c766eaf4c4f1cccef2fca5f2a4fb5f2" +checksum = "ff4dd66668b557604244583e3e1e1eada8c5c2e96a6d0d6653ede395b78bbacb" dependencies = [ "futures-core", "futures-sink", @@ -1541,15 +1543,15 @@ dependencies = [ [[package]] name = "futures-core" -version = "0.3.28" +version = "0.3.29" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "4bca583b7e26f571124fe5b7561d49cb2868d79116cfa0eefce955557c6fee8c" +checksum = "eb1d22c66e66d9d72e1758f0bd7d4fd0bee04cad842ee34587d68c07e45d088c" [[package]] name = "futures-executor" -version = "0.3.28" +version = "0.3.29" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "ccecee823288125bd88b4d7f565c9e58e41858e47ab72e8ea2d64e93624386e0" +checksum = "0f4fb8693db0cf099eadcca0efe2a5a22e4550f98ed16aba6c48700da29597bc" dependencies = [ "futures-core", "futures-task", @@ -1558,15 +1560,15 @@ dependencies = [ [[package]] name = "futures-io" -version = "0.3.28" +version = "0.3.29" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "4fff74096e71ed47f8e023204cfd0aa1289cd54ae5430a9523be060cdb849964" +checksum = "8bf34a163b5c4c52d0478a4d757da8fb65cabef42ba90515efee0f6f9fa45aaa" [[package]] name = "futures-macro" -version = "0.3.28" +version = "0.3.29" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "89ca545a94061b6365f2c7355b4b32bd20df3ff95f02da9329b34ccc3bd6ee72" +checksum = "53b153fd91e4b0147f4aced87be237c98248656bb01050b96bf3ee89220a8ddb" dependencies = [ "proc-macro2", "quote", @@ -1575,15 +1577,15 @@ dependencies = [ [[package]] name = "futures-sink" -version = "0.3.28" +version = "0.3.29" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "f43be4fe21a13b9781a69afa4985b0f6ee0e1afab2c6f454a8cf30e2b2237b6e" +checksum = "e36d3378ee38c2a36ad710c5d30c2911d752cb941c00c72dbabfb786a7970817" [[package]] name = "futures-task" -version = "0.3.28" +version = "0.3.29" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "76d3d132be6c0e6aa1534069c705a74a5997a356c0dc2f86a47765e5617c5b65" +checksum = "efd193069b0ddadc69c46389b740bbccdd97203899b48d09c5f7969591d6bae2" [[package]] name = "futures-timer" @@ -1593,9 +1595,9 @@ checksum = "e64b03909df88034c26dc1547e8970b91f98bdb65165d6a4e9110d94263dbb2c" [[package]] name = "futures-util" -version = "0.3.28" +version = "0.3.29" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "26b01e40b772d54cf6c6d721c1d1abd0647a0106a12ecaa1c186273392a69533" +checksum = "a19526d624e703a3179b3d322efec918b6246ea0fa51d41124525f00f1cc8104" dependencies = [ "futures-channel", "futures-core", @@ -1689,9 +1691,9 @@ dependencies = [ [[package]] name = "hashbrown" -version = "0.14.1" +version = "0.14.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "7dfda62a12f55daeae5015f81b0baea145391cb4520f86c248fc615d72640d12" +checksum = "f93e7192158dbcda357bdec5fb5788eebf8bbac027f3f33e719d29135ae84156" dependencies = [ "ahash", "allocator-api2", @@ -1790,7 +1792,7 @@ dependencies = [ "httpdate", "itoa", "pin-project-lite", - "socket2 0.4.9", + "socket2 0.4.10", "tokio", "tower-service", "tracing", @@ -1814,30 +1816,30 @@ dependencies = [ [[package]] name = "hyper-rustls" -version = "0.24.1" +version = "0.24.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "8d78e1e73ec14cf7375674f74d7dde185c8206fd9dea6fb6295e8a98098aaa97" +checksum = "ec3efd23720e2049821a693cbc7e65ea87c72f1c58ff2f9522ff332b1491e590" dependencies = [ "futures-util", "http", "hyper", - "rustls 0.21.7", + "rustls 0.21.8", "tokio", "tokio-rustls 0.24.1", ] [[package]] name = "iana-time-zone" -version = "0.1.57" +version = "0.1.58" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "2fad5b825842d2b38bd206f3e81d6957625fd7f0a361e345c30e01a0ae2dd613" +checksum = "8326b86b6cff230b97d0d312a6c40a60726df3332e721f72a1b035f451663b20" dependencies = [ "android_system_properties", "core-foundation-sys", "iana-time-zone-haiku", "js-sys", "wasm-bindgen", - "windows", + "windows-core", ] [[package]] @@ -1876,7 +1878,7 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "8adf3ddd720272c6ea8bf59463c04e0f93d0bbf7c5439b691bca2987e0270897" dependencies = [ "equivalent", - "hashbrown 0.14.1", + "hashbrown 0.14.2", ] [[package]] @@ -1896,9 +1898,9 @@ checksum = "8bb03732005da905c88227371639bf1ad885cc712789c011c31c5fb3ab3ccf02" [[package]] name = "ipnet" -version = "2.8.0" +version = "2.9.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "28b29a3cd74f0f4598934efe3aeba42bae0eb4680554128851ebbecb02af14e6" +checksum = "8f518f335dce6725a761382244631d86cf0ccb2863413590b31338feb467f9c3" [[package]] name = "itertools" @@ -2057,9 +2059,9 @@ checksum = "da2479e8c062e40bf0066ffa0bc823de0a9368974af99c9f6df941d2c231e03f" [[package]] name = "lock_api" -version = "0.4.10" +version = "0.4.11" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "c1cc9717a20b1bb222f333e6a92fd32f7d8a18ddc5a3191a11af45dcbf4dcd16" +checksum = "3c168f8615b12bc01f9c17e2eb0cc07dcae1940121185446edc3744920e8ef45" dependencies = [ "autocfg", "scopeguard", @@ -2133,9 +2135,9 @@ dependencies = [ [[package]] name = "mio" -version = "0.8.8" +version = "0.8.9" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "927a765cd3fc26206e66b296465fa9d3e5ab003e651c1b3c060e7956d96b19d2" +checksum = "3dce281c5e46beae905d4de1870d8b1509a9142b62eedf18b443b011ca8343d0" dependencies = [ "libc", "wasi", @@ -2317,9 +2319,9 @@ dependencies = [ [[package]] name = "os_str_bytes" -version = "6.5.1" +version = "6.6.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "4d5d9eb14b174ee9aa2ef96dc2b94637a2d4b6e7cb873c7e171f0c20c6cf3eac" +checksum = "e2355d85b9a3786f481747ced0e0ff2ba35213a1f9bd406ed906554d7af805a1" [[package]] name = "outref" @@ -2339,13 +2341,13 @@ dependencies = [ [[package]] name = "parking_lot_core" -version = "0.9.8" +version = "0.9.9" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "93f00c865fe7cabf650081affecd3871070f26767e7b2070a3ffae14c654b447" +checksum = "4c42a9226546d68acdd9c0a280d17ce19bfe27a46bf68784e4066115788d008e" dependencies = [ "cfg-if", "libc", - "redox_syscall 0.3.5", + "redox_syscall 0.4.1", "smallvec", "windows-targets", ] @@ -2370,7 +2372,7 @@ dependencies = [ "chrono", "flate2", "futures", - "hashbrown 0.14.1", + "hashbrown 0.14.2", "lz4_flex", "num", "num-bigint", @@ -2491,6 +2493,12 @@ version = "0.3.27" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "26072860ba924cbfa98ea39c8c19b4dd6a4a25423dbdf219c1eca91aa0cf6964" +[[package]] +name = "powerfmt" +version = "0.2.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "439ee305def115ba05938db6eb1644ff94165c5ab5e9420d1c1bcedbba909391" + [[package]] name = "ppv-lite86" version = "0.2.17" @@ -2552,12 +2560,6 @@ dependencies = [ "version_check", ] -[[package]] -name = "proc-macro-hack" -version = "0.5.20+deprecated" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "dc375e1527247fe1a97d8b7156678dfe7c1af2fc075c9a4db3690ecd2a148068" - [[package]] name = "proc-macro2" version = "1.0.69" @@ -2643,9 +2645,9 @@ dependencies = [ [[package]] name = "redox_syscall" -version = "0.3.5" +version = "0.4.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "567664f262709473930a4bf9e51bf2ebf3348f2e748ccc50dea20646858f8f29" +checksum = "4722d768eff46b75989dd134e5c353f0d6296e5aaa3132e776cbdb56be7731aa" dependencies = [ "bitflags 1.3.2", ] @@ -2663,9 +2665,9 @@ dependencies = [ [[package]] name = "regex" -version = "1.10.0" +version = "1.10.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "d119d7c7ca818f8a53c300863d4f87566aac09943aef5b355bb83969dae75d87" +checksum = "380b951a9c5e80ddfd6136919eef32310721aa4aacd4889a8d39124b026ab343" dependencies = [ "aho-corasick", "memchr", @@ -2675,9 +2677,9 @@ dependencies = [ [[package]] name = "regex-automata" -version = "0.4.1" +version = "0.4.3" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "465c6fc0621e4abc4187a2bda0937bfd4f722c2730b29562e19689ea796c9a4b" +checksum = "5f804c7828047e88b2d32e2d7fe5a105da8ee3264f01902f796c8e067dc2483f" dependencies = [ "aho-corasick", "memchr", @@ -2686,15 +2688,15 @@ dependencies = [ [[package]] name = "regex-lite" -version = "0.1.3" +version = "0.1.5" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "9a6ebcd15653947e6140f59a9811a06ed061d18a5c35dfca2e2e4c5525696878" +checksum = "30b661b2f27137bdbc16f00eda72866a92bb28af1753ffbd56744fb6e2e9cd8e" [[package]] name = "regex-syntax" -version = "0.8.1" +version = "0.8.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "56d84fdd47036b038fc80dd333d10b6aab10d5d31f4a366e20014def75328d33" +checksum = "c08c74e62047bb2de4ff487b251e4a92e24f48745648451635cec7d591162d9f" [[package]] name = "reqwest" @@ -2711,7 +2713,7 @@ dependencies = [ "http", "http-body", "hyper", - "hyper-rustls 0.24.1", + "hyper-rustls 0.24.2", "ipnet", "js-sys", "log", @@ -2719,7 +2721,7 @@ dependencies = [ "once_cell", "percent-encoding", "pin-project-lite", - "rustls 0.21.7", + "rustls 0.21.8", "rustls-pemfile", "serde", "serde_json", @@ -2755,9 +2757,9 @@ dependencies = [ [[package]] name = "ring" -version = "0.17.3" +version = "0.17.5" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "9babe80d5c16becf6594aa32ad2be8fe08498e7ae60b77de8df700e67f191d7e" +checksum = "fb0205304757e5d899b9c2e448b867ffd03ae7f988002e47cd24954391394d0b" dependencies = [ "cc", "getrandom", @@ -2816,11 +2818,11 @@ dependencies = [ [[package]] name = "rustix" -version = "0.38.18" +version = "0.38.21" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "5a74ee2d7c2581cd139b42447d7d9389b889bdaad3a73f1ebb16f2a3237bb19c" +checksum = "2b426b0506e5d50a7d8dafcf2e81471400deb602392c7dd110815afb4eaf02a3" dependencies = [ - "bitflags 2.4.0", + "bitflags 2.4.1", "errno", "libc", "linux-raw-sys", @@ -2841,12 +2843,12 @@ dependencies = [ [[package]] name = "rustls" -version = "0.21.7" +version = "0.21.8" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "cd8d6c9f025a446bc4d18ad9632e69aec8f287aa84499ee335599fabd20c3fd8" +checksum = "446e14c5cda4f3f30fe71863c34ec70f5ac79d6087097ad0bb433e1be5edf04c" dependencies = [ "log", - "ring 0.16.20", + "ring 0.17.5", "rustls-webpki", "sct", ] @@ -2874,12 +2876,12 @@ dependencies = [ [[package]] name = "rustls-webpki" -version = "0.101.6" +version = "0.101.7" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "3c7d5dece342910d9ba34d259310cae3e0154b873b35408b787b59bce53d34fe" +checksum = "8b6275d1ee7a1cd780b64aca7726599a1dbc893b1e64144529e55c3c2f745765" dependencies = [ - "ring 0.16.20", - "untrusted 0.7.1", + "ring 0.17.5", + "untrusted 0.9.0", ] [[package]] @@ -2943,12 +2945,12 @@ checksum = "94143f37725109f92c262ed2cf5e59bce7498c01bcc1502d7b9afe439a4e9f49" [[package]] name = "sct" -version = "0.7.0" +version = "0.7.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "d53dcdb7c9f8158937a7981b48accfd39a43af418591a5d008c7b22b5e1b7ca4" +checksum = "da046153aa2352493d6cb7da4b6e5c0c057d8a1d0a9aa8560baffdd945acd414" dependencies = [ - "ring 0.16.20", - "untrusted 0.7.1", + "ring 0.17.5", + "untrusted 0.9.0", ] [[package]] @@ -2988,18 +2990,18 @@ checksum = "a3f0bf26fd526d2a95683cd0f87bf103b8539e2ca1ef48ce002d67aad59aa0b4" [[package]] name = "serde" -version = "1.0.188" +version = "1.0.190" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "cf9e0fcba69a370eed61bcf2b728575f726b50b55cba78064753d708ddc7549e" +checksum = "91d3c334ca1ee894a2c6f6ad698fe8c435b76d504b13d436f0685d648d6d96f7" dependencies = [ "serde_derive", ] [[package]] name = "serde_derive" -version = "1.0.188" +version = "1.0.190" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "4eca7ac642d82aa35b60049a6eccb4be6be75e599bd2e9adb5f875a737654af2" +checksum = "67c5609f394e5c2bd7fc51efda478004ea80ef42fee983d5c67a65e34f32c0e3" dependencies = [ "proc-macro2", "quote", @@ -3008,9 +3010,9 @@ dependencies = [ [[package]] name = "serde_json" -version = "1.0.107" +version = "1.0.108" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "6b420ce6e3d8bd882e9b243c6eed35dbc9a6110c9769e74b584e0d68d1f20c65" +checksum = "3d1c7e3eac408d115102c4c24ad393e0821bb3a5df4d506a80f85f7a742a526b" dependencies = [ "itoa", "ryu", @@ -3091,9 +3093,9 @@ checksum = "5e9f0ab6ef7eb7353d9119c170a436d1bf248eea575ac42d19d12f4e34130831" [[package]] name = "socket2" -version = "0.4.9" +version = "0.4.10" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "64a4a911eed85daf18834cfaa86a79b7d266ff93ff5ba14005426219480ed662" +checksum = "9f7916fc008ca5542385b89a3d3ce689953c143e9304a9bf8beec1de48994c0d" dependencies = [ "libc", "winapi", @@ -3101,9 +3103,9 @@ dependencies = [ [[package]] name = "socket2" -version = "0.5.4" +version = "0.5.5" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "4031e820eb552adee9295814c0ced9e5cf38ddf1e8b7d566d6de8e2538ea989e" +checksum = "7b5fac59a5cb5dd637972e5fca70daf0523c9067fcdc4842f053dae04a18f8e9" dependencies = [ "libc", "windows-sys", @@ -3123,9 +3125,9 @@ checksum = "6980e8d7511241f8acf4aebddbb1ff938df5eebe98691418c4468d0b72a96a67" [[package]] name = "sqlparser" -version = "0.38.0" +version = "0.39.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "0272b7bb0a225320170c99901b4b5fb3a4384e255a7f2cc228f61e2ba3893e75" +checksum = "743b4dc2cbde11890ccb254a8fc9d537fa41b36da00de2a1c5e9848c9bc42bd7" dependencies = [ "log", "sqlparser_derive", @@ -3160,39 +3162,20 @@ version = "0.10.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "73473c0e59e6d5812c5dfe2a064a6444949f089e20eec9a2e5506596494e4623" -[[package]] -name = "strum" -version = "0.24.1" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "063e6045c0e62079840579a7e47a355ae92f60eb74daaf156fb1e84ba164e63f" - [[package]] name = "strum" version = "0.25.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "290d54ea6f91c969195bdbcd7442c8c2a2ba87da8bf60a7ee86a235d4bc1e125" dependencies = [ - "strum_macros 0.25.2", + "strum_macros", ] [[package]] name = "strum_macros" -version = "0.24.3" +version = "0.25.3" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "1e385be0d24f186b4ce2f9982191e7101bb737312ad61c1f2f984f34bcf85d59" -dependencies = [ - "heck", - "proc-macro2", - "quote", - "rustversion", - "syn 1.0.109", -] - -[[package]] -name = "strum_macros" -version = "0.25.2" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "ad8d03b598d3d0fff69bf533ee3ef19b8eeb342729596df84bcc7e1f96ec4059" +checksum = "23dc1fa9ac9c169a78ba62f0b841814b7abae11bdd047b9c58f893439e309ea0" dependencies = [ "heck", "proc-macro2", @@ -3252,13 +3235,13 @@ dependencies = [ [[package]] name = "tempfile" -version = "3.8.0" +version = "3.8.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "cb94d2f3cc536af71caac6b6fcebf65860b347e7ce0cc9ebe8f70d3e521054ef" +checksum = "7ef1adac450ad7f4b3c28589471ade84f25f731a7a0fe30d71dfa9f60fd808e5" dependencies = [ "cfg-if", "fastrand 2.0.1", - "redox_syscall 0.3.5", + "redox_syscall 0.4.1", "rustix", "windows-sys", ] @@ -3286,18 +3269,18 @@ checksum = "222a222a5bfe1bba4a77b45ec488a741b3cb8872e5e499451fd7d0129c9c7c3d" [[package]] name = "thiserror" -version = "1.0.49" +version = "1.0.50" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "1177e8c6d7ede7afde3585fd2513e611227efd6481bd78d2e82ba1ce16557ed4" +checksum = "f9a7210f5c9a7156bb50aa36aed4c95afb51df0df00713949448cf9e97d382d2" dependencies = [ "thiserror-impl", ] [[package]] name = "thiserror-impl" -version = "1.0.49" +version = "1.0.50" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "10712f02019e9288794769fba95cd6847df9874d49d871d062172f9dd41bc4cc" +checksum = "266b2e40bc00e5a6c09c3584011e08b06f123c00362c92b975ba9843aaaa14b8" dependencies = [ "proc-macro2", "quote", @@ -3317,11 +3300,12 @@ dependencies = [ [[package]] name = "time" -version = "0.3.29" +version = "0.3.30" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "426f806f4089c493dcac0d24c29c01e2c38baf8e30f1b716ee37e83d200b18fe" +checksum = "c4a34ab300f2dee6e562c10a046fc05e358b29f9bf92277f30c3c8d82275f6f5" dependencies = [ "deranged", + "powerfmt", "serde", "time-core", "time-macros", @@ -3379,7 +3363,7 @@ dependencies = [ "num_cpus", "parking_lot", "pin-project-lite", - "socket2 0.5.4", + "socket2 0.5.5", "tokio-macros", "windows-sys", ] @@ -3412,7 +3396,7 @@ version = "0.24.1" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "c28327cf380ac148141087fbfb9de9d7bd4e84ab5d2c28fbc911d753de8a7081" dependencies = [ - "rustls 0.21.7", + "rustls 0.21.8", "tokio", ] @@ -3429,9 +3413,9 @@ dependencies = [ [[package]] name = "tokio-util" -version = "0.7.9" +version = "0.7.10" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "1d68074620f57a0b21594d9735eb2e98ab38b17f80d3fcb189fca266771ca60d" +checksum = "5419f34732d9eb6ee4c3578b7989078579b7f039cbbb9ca2c4da015749371e15" dependencies = [ "bytes", "futures-core", @@ -3471,11 +3455,10 @@ checksum = "b6bc1c9ce2b5135ac7f93c72918fc37feb872bdc6a5533a8b85eb4b86bfdae52" [[package]] name = "tracing" -version = "0.1.37" +version = "0.1.40" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "8ce8c33a8d48bd45d624a6e523445fd21ec13d3653cd51f681abf67418f54eb8" +checksum = "c3523ab5a71916ccf420eebdf5521fcef02141234bbc0b8a49f2fdc4544364ef" dependencies = [ - "cfg-if", "log", "pin-project-lite", "tracing-attributes", @@ -3484,9 +3467,9 @@ dependencies = [ [[package]] name = "tracing-attributes" -version = "0.1.26" +version = "0.1.27" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "5f4f31f56159e98206da9efd823404b79b6ef3143b4a7ab76e67b1751b25a4ab" +checksum = "34704c8d6ebcbc939824180af020566b01a7c01f80641264eba0999f6c2b6be7" dependencies = [ "proc-macro2", "quote", @@ -3495,9 +3478,9 @@ dependencies = [ [[package]] name = "tracing-core" -version = "0.1.31" +version = "0.1.32" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "0955b8137a1df6f1a2e9a37d8a6656291ff0297c1a97c24e0d8425fe2312f79a" +checksum = "c06d3da6113f116aaee68e4d601191614c9053067f9ab7f6edbcb161237daa54" dependencies = [ "once_cell", ] @@ -3614,9 +3597,9 @@ checksum = "711b9620af191e0cdc7468a8d14e709c3dcdb115b36f838e601583af800a370a" [[package]] name = "uuid" -version = "1.4.1" +version = "1.5.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "79daa5ed5740825c40b389c5e50312b9c86df53fccd33f281df655642b43869d" +checksum = "88ad59a7560b41a70d191093a945f0b87bc1deeda46fb237479708a1d6b6cdfc" dependencies = [ "getrandom", "serde", @@ -3763,7 +3746,7 @@ version = "0.22.4" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "ed63aea5ce73d0ff405984102c42de94fc55a6b75765d621c65262469b3c9b53" dependencies = [ - "ring 0.17.3", + "ring 0.17.5", "untrusted 0.9.0", ] @@ -3805,10 +3788,10 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "712e227841d057c1ee1cd2fb22fa7e5a5461ae8e48fa2ca79ec42cfc1931183f" [[package]] -name = "windows" -version = "0.48.0" +name = "windows-core" +version = "0.51.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "e686886bc078bc1b0b600cac0147aadb815089b6e4da64016cbd754b6342700f" +checksum = "f1f8cf84f35d2db49a46868f947758c7a1138116f7fac3bc844f43ade1292e64" dependencies = [ "windows-targets", ] @@ -3904,6 +3887,26 @@ dependencies = [ "lzma-sys", ] +[[package]] +name = "zerocopy" +version = "0.7.20" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "dd66a62464e3ffd4e37bd09950c2b9dd6c4f8767380fabba0d523f9a775bc85a" +dependencies = [ + "zerocopy-derive", +] + +[[package]] +name = "zerocopy-derive" +version = "0.7.20" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "255c4596d41e6916ced49cfafea18727b24d67878fa180ddfd69b9df34fd1726" +dependencies = [ + "proc-macro2", + "quote", + "syn 2.0.38", +] + [[package]] name = "zeroize" version = "1.6.0" diff --git a/datafusion/sql/src/expr/function.rs b/datafusion/sql/src/expr/function.rs index 373388277351..8af5fef84ac1 100644 --- a/datafusion/sql/src/expr/function.rs +++ b/datafusion/sql/src/expr/function.rs @@ -146,9 +146,14 @@ impl<'a, S: ContextProvider> SqlToRel<'a, S> { let order_by = (!order_by.is_empty()).then_some(order_by); let args = self.function_args_to_expr(function.args, schema, planner_context)?; + let filter: Option> = function + .filter + .map(|e| self.sql_expr_to_logical_expr(*e, schema, planner_context)) + .transpose()? + .map(Box::new); return Ok(Expr::AggregateFunction(expr::AggregateFunction::new( - fun, args, distinct, None, order_by, + fun, args, distinct, filter, order_by, ))); }; diff --git a/datafusion/sql/src/expr/mod.rs b/datafusion/sql/src/expr/mod.rs index 26184834874a..1cf0fc133f04 100644 --- a/datafusion/sql/src/expr/mod.rs +++ b/datafusion/sql/src/expr/mod.rs @@ -222,7 +222,9 @@ impl<'a, S: ContextProvider> SqlToRel<'a, S> { planner_context, ), - SQLExpr::Cast { expr, data_type } => Ok(Expr::Cast(Cast::new( + SQLExpr::Cast { + expr, data_type, .. + } => Ok(Expr::Cast(Cast::new( Box::new(self.sql_expr_to_logical_expr( *expr, schema, @@ -231,7 +233,9 @@ impl<'a, S: ContextProvider> SqlToRel<'a, S> { self.convert_data_type(&data_type)?, ))), - SQLExpr::TryCast { expr, data_type } => Ok(Expr::TryCast(TryCast::new( + SQLExpr::TryCast { + expr, data_type, .. + } => Ok(Expr::TryCast(TryCast::new( Box::new(self.sql_expr_to_logical_expr( *expr, schema, @@ -412,6 +416,7 @@ impl<'a, S: ContextProvider> SqlToRel<'a, S> { expr, trim_where, trim_what, + .. } => self.sql_trim_to_expr( *expr, trim_where, @@ -477,10 +482,36 @@ impl<'a, S: ContextProvider> SqlToRel<'a, S> { self.parse_array_agg(array_agg, schema, planner_context) } + SQLExpr::Struct { values, fields } => { + self.parse_struct(values, fields, schema, planner_context) + } + _ => not_impl_err!("Unsupported ast node in sqltorel: {sql:?}"), } } + fn parse_struct( + &self, + values: Vec, + fields: Vec, + input_schema: &DFSchema, + planner_context: &mut PlannerContext, + ) -> Result { + if !fields.is_empty() { + return not_impl_err!("Struct fields are not supported yet"); + } + let args = values + .into_iter() + .map(|value| { + self.sql_expr_to_logical_expr(value, input_schema, planner_context) + }) + .collect::>>()?; + Ok(Expr::ScalarFunction(ScalarFunction::new( + BuiltinScalarFunction::Struct, + args, + ))) + } + fn parse_array_agg( &self, array_agg: ArrayAgg, diff --git a/datafusion/sql/src/planner.rs b/datafusion/sql/src/planner.rs index f7d8307d33a0..ca5e260aee05 100644 --- a/datafusion/sql/src/planner.rs +++ b/datafusion/sql/src/planner.rs @@ -24,8 +24,8 @@ use arrow_schema::*; use datafusion_common::field_not_found; use datafusion_common::internal_err; use datafusion_expr::WindowUDF; -use sqlparser::ast::ExactNumberInfo; use sqlparser::ast::TimezoneInfo; +use sqlparser::ast::{ArrayElemTypeDef, ExactNumberInfo}; use sqlparser::ast::{ColumnDef as SQLColumnDef, ColumnOption}; use sqlparser::ast::{DataType as SQLDataType, Ident, ObjectName, TableAlias}; @@ -297,14 +297,15 @@ impl<'a, S: ContextProvider> SqlToRel<'a, S> { pub(crate) fn convert_data_type(&self, sql_type: &SQLDataType) -> Result { match sql_type { - SQLDataType::Array(Some(inner_sql_type)) => { + SQLDataType::Array(ArrayElemTypeDef::AngleBracket(inner_sql_type)) + | SQLDataType::Array(ArrayElemTypeDef::SquareBracket(inner_sql_type)) => { let data_type = self.convert_simple_data_type(inner_sql_type)?; Ok(DataType::List(Arc::new(Field::new( "field", data_type, true, )))) } - SQLDataType::Array(None) => { + SQLDataType::Array(ArrayElemTypeDef::None) => { not_impl_err!("Arrays with unspecified type is not supported") } other => self.convert_simple_data_type(other), @@ -330,7 +331,7 @@ impl<'a, S: ContextProvider> SqlToRel<'a, S> { SQLDataType::Char(_) | SQLDataType::Varchar(_) | SQLDataType::Text - | SQLDataType::String => Ok(DataType::Utf8), + | SQLDataType::String(_) => Ok(DataType::Utf8), SQLDataType::Timestamp(None, tz_info) => { let tz = if matches!(tz_info, TimezoneInfo::Tz) || matches!(tz_info, TimezoneInfo::WithTimeZone) @@ -400,7 +401,12 @@ impl<'a, S: ContextProvider> SqlToRel<'a, S> { | SQLDataType::Dec(_) | SQLDataType::BigNumeric(_) | SQLDataType::BigDecimal(_) - | SQLDataType::Clob(_) => not_impl_err!( + | SQLDataType::Clob(_) + | SQLDataType::Bytes(_) + | SQLDataType::Int64 + | SQLDataType::Float64 + | SQLDataType::Struct(_) + => not_impl_err!( "Unsupported SQL type {sql_type:?}" ), } diff --git a/datafusion/sql/src/set_expr.rs b/datafusion/sql/src/set_expr.rs index e771a5ba3de4..7300d49be0f5 100644 --- a/datafusion/sql/src/set_expr.rs +++ b/datafusion/sql/src/set_expr.rs @@ -44,6 +44,9 @@ impl<'a, S: ContextProvider> SqlToRel<'a, S> { SetQuantifier::AllByName => { return not_impl_err!("UNION ALL BY NAME not implemented") } + SetQuantifier::DistinctByName => { + return not_impl_err!("UNION DISTINCT BY NAME not implemented") + } }; let left_plan = self.set_expr_to_plan(*left, planner_context)?; diff --git a/datafusion/sql/src/statement.rs b/datafusion/sql/src/statement.rs index 80a27db6e63d..834bddbf20a3 100644 --- a/datafusion/sql/src/statement.rs +++ b/datafusion/sql/src/statement.rs @@ -430,6 +430,7 @@ impl<'a, S: ContextProvider> SqlToRel<'a, S> { table, on, returning, + ignore, } => { if or.is_some() { plan_err!("Inserts with or clauses not supported")?; @@ -449,6 +450,9 @@ impl<'a, S: ContextProvider> SqlToRel<'a, S> { if returning.is_some() { plan_err!("Insert-returning clause not supported")?; } + if ignore { + plan_err!("Insert-ignore clause not supported")?; + } let _ = into; // optional keyword doesn't change behavior self.insert_to_plan(table_name, columns, source, overwrite) } @@ -471,6 +475,8 @@ impl<'a, S: ContextProvider> SqlToRel<'a, S> { selection, returning, from, + order_by, + limit, } => { if !tables.is_empty() { plan_err!("DELETE
not supported")?; @@ -483,6 +489,15 @@ impl<'a, S: ContextProvider> SqlToRel<'a, S> { if returning.is_some() { plan_err!("Delete-returning clause not yet supported")?; } + + if !order_by.is_empty() { + plan_err!("Delete-order-by clause not yet supported")?; + } + + if limit.is_some() { + plan_err!("Delete-limit clause not yet supported")?; + } + let table_name = self.get_delete_target(from)?; self.delete_to_plan(table_name, selection) } From 27e64ae496b12f6e702f21b7e48f6ff8dd01c825 Mon Sep 17 00:00:00 2001 From: Andrew Lamb Date: Tue, 31 Oct 2023 10:12:42 -0400 Subject: [PATCH 161/572] Fix panic in multiple distinct aggregates by fixing `ScalarValue::new_list` (#7989) MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit * Fix panic in multiple distinct aggregates by fixing ScalarValue::new_list * Update datafusion/common/src/scalar.rs Co-authored-by: Daniël Heres --------- Co-authored-by: Daniël Heres --- Cargo.toml | 1 + datafusion/common/src/scalar.rs | 204 +----------------- .../sqllogictest/test_files/aggregate.slt | 35 ++- 3 files changed, 38 insertions(+), 202 deletions(-) diff --git a/Cargo.toml b/Cargo.toml index 5aa830ddff0f..22d5f2f64464 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -75,3 +75,4 @@ opt-level = 3 overflow-checks = false panic = 'unwind' rpath = false + diff --git a/datafusion/common/src/scalar.rs b/datafusion/common/src/scalar.rs index f9b0cbdf22de..b3c11740abf9 100644 --- a/datafusion/common/src/scalar.rs +++ b/datafusion/common/src/scalar.rs @@ -600,117 +600,6 @@ macro_rules! typed_cast { }}; } -macro_rules! build_timestamp_list { - ($TIME_UNIT:expr, $TIME_ZONE:expr, $VALUES:expr, $SIZE:expr) => {{ - match $VALUES { - // the return on the macro is necessary, to short-circuit and return ArrayRef - None => { - return new_null_array( - &DataType::List(Arc::new(Field::new( - "item", - DataType::Timestamp($TIME_UNIT, $TIME_ZONE), - true, - ))), - $SIZE, - ) - } - Some(values) => match $TIME_UNIT { - TimeUnit::Second => { - build_values_list_tz!( - TimestampSecondBuilder, - TimestampSecond, - values, - $SIZE, - $TIME_ZONE - ) - } - TimeUnit::Millisecond => build_values_list_tz!( - TimestampMillisecondBuilder, - TimestampMillisecond, - values, - $SIZE, - $TIME_ZONE - ), - TimeUnit::Microsecond => build_values_list_tz!( - TimestampMicrosecondBuilder, - TimestampMicrosecond, - values, - $SIZE, - $TIME_ZONE - ), - TimeUnit::Nanosecond => build_values_list_tz!( - TimestampNanosecondBuilder, - TimestampNanosecond, - values, - $SIZE, - $TIME_ZONE - ), - }, - } - }}; -} - -macro_rules! new_builder { - (StringBuilder, $len:expr) => { - StringBuilder::new() - }; - (LargeStringBuilder, $len:expr) => { - LargeStringBuilder::new() - }; - ($el:ident, $len:expr) => {{ - <$el>::with_capacity($len) - }}; -} - -macro_rules! build_values_list { - ($VALUE_BUILDER_TY:ident, $SCALAR_TY:ident, $VALUES:expr, $SIZE:expr) => {{ - let builder = new_builder!($VALUE_BUILDER_TY, $VALUES.len()); - let mut builder = ListBuilder::new(builder); - - for _ in 0..$SIZE { - for scalar_value in $VALUES { - match scalar_value { - ScalarValue::$SCALAR_TY(Some(v)) => { - builder.values().append_value(v.clone()); - } - ScalarValue::$SCALAR_TY(None) => { - builder.values().append_null(); - } - _ => panic!("Incompatible ScalarValue for list"), - }; - } - builder.append(true); - } - - builder.finish() - }}; -} - -macro_rules! build_values_list_tz { - ($VALUE_BUILDER_TY:ident, $SCALAR_TY:ident, $VALUES:expr, $SIZE:expr, $TIME_ZONE:expr) => {{ - let mut builder = ListBuilder::new( - $VALUE_BUILDER_TY::with_capacity($VALUES.len()).with_timezone_opt($TIME_ZONE), - ); - - for _ in 0..$SIZE { - for scalar_value in $VALUES { - match scalar_value { - ScalarValue::$SCALAR_TY(Some(v), _) => { - builder.values().append_value(v.clone()); - } - ScalarValue::$SCALAR_TY(None, _) => { - builder.values().append_null(); - } - _ => panic!("Incompatible ScalarValue for list"), - }; - } - builder.append(true); - } - - builder.finish() - }}; -} - macro_rules! build_array_from_option { ($DATA_TYPE:ident, $ARRAY_TYPE:ident, $EXPR:expr, $SIZE:expr) => {{ match $EXPR { @@ -1198,7 +1087,8 @@ impl ScalarValue { } /// Converts an iterator of references [`ScalarValue`] into an [`ArrayRef`] - /// corresponding to those values. For example, + /// corresponding to those values. For example, an iterator of + /// [`ScalarValue::Int32`] would be converted to an [`Int32Array`]. /// /// Returns an error if the iterator is empty or if the /// [`ScalarValue`]s are not all the same type @@ -1657,41 +1547,6 @@ impl ScalarValue { Ok(array) } - /// This function does not contains nulls but empty array instead. - fn iter_to_array_list_without_nulls( - values: &[ScalarValue], - data_type: &DataType, - ) -> Result> { - let mut elements: Vec = vec![]; - let mut offsets = vec![]; - - if values.is_empty() { - offsets.push(0); - } else { - let arr = ScalarValue::iter_to_array(values.to_vec())?; - offsets.push(arr.len()); - elements.push(arr); - } - - // Concatenate element arrays to create single flat array - let flat_array = if elements.is_empty() { - new_empty_array(data_type) - } else { - let element_arrays: Vec<&dyn Array> = - elements.iter().map(|a| a.as_ref()).collect(); - arrow::compute::concat(&element_arrays)? - }; - - let list_array = ListArray::new( - Arc::new(Field::new("item", flat_array.data_type().to_owned(), true)), - OffsetBuffer::::from_lengths(offsets), - flat_array, - None, - ); - - Ok(list_array) - } - /// This function build with nulls with nulls buffer. fn iter_to_array_list( scalars: impl IntoIterator, @@ -1780,7 +1635,8 @@ impl ScalarValue { .unwrap() } - /// Converts `Vec` to ListArray, simplified version of ScalarValue::to_array + /// Converts `Vec` where each element has type corresponding to + /// `data_type`, to a [`ListArray`]. /// /// Example /// ``` @@ -1806,52 +1662,12 @@ impl ScalarValue { /// assert_eq!(result, &expected); /// ``` pub fn new_list(values: &[ScalarValue], data_type: &DataType) -> ArrayRef { - Arc::new(match data_type { - DataType::Boolean => build_values_list!(BooleanBuilder, Boolean, values, 1), - DataType::Int8 => build_values_list!(Int8Builder, Int8, values, 1), - DataType::Int16 => build_values_list!(Int16Builder, Int16, values, 1), - DataType::Int32 => build_values_list!(Int32Builder, Int32, values, 1), - DataType::Int64 => build_values_list!(Int64Builder, Int64, values, 1), - DataType::UInt8 => build_values_list!(UInt8Builder, UInt8, values, 1), - DataType::UInt16 => build_values_list!(UInt16Builder, UInt16, values, 1), - DataType::UInt32 => build_values_list!(UInt32Builder, UInt32, values, 1), - DataType::UInt64 => build_values_list!(UInt64Builder, UInt64, values, 1), - DataType::Utf8 => build_values_list!(StringBuilder, Utf8, values, 1), - DataType::LargeUtf8 => { - build_values_list!(LargeStringBuilder, LargeUtf8, values, 1) - } - DataType::Float32 => build_values_list!(Float32Builder, Float32, values, 1), - DataType::Float64 => build_values_list!(Float64Builder, Float64, values, 1), - DataType::Timestamp(unit, tz) => { - let values = Some(values); - build_timestamp_list!(unit.clone(), tz.clone(), values, 1) - } - DataType::List(_) | DataType::Struct(_) => { - ScalarValue::iter_to_array_list_without_nulls(values, data_type).unwrap() - } - DataType::Decimal128(precision, scale) => { - let mut vals = vec![]; - for value in values.iter() { - if let ScalarValue::Decimal128(v, _, _) = value { - vals.push(v.to_owned()) - } - } - - let arr = Decimal128Array::from(vals) - .with_precision_and_scale(*precision, *scale) - .unwrap(); - wrap_into_list_array(Arc::new(arr)) - } - - DataType::Null => { - let arr = new_null_array(&DataType::Null, values.len()); - wrap_into_list_array(arr) - } - _ => panic!( - "Unsupported data type {:?} for ScalarValue::list_to_array", - data_type - ), - }) + let values = if values.is_empty() { + new_empty_array(data_type) + } else { + Self::iter_to_array(values.iter().cloned()).unwrap() + }; + Arc::new(wrap_into_list_array(values)) } /// Converts a scalar value into an array of `size` rows. diff --git a/datafusion/sqllogictest/test_files/aggregate.slt b/datafusion/sqllogictest/test_files/aggregate.slt index 777b634e93b1..6217f12279a9 100644 --- a/datafusion/sqllogictest/test_files/aggregate.slt +++ b/datafusion/sqllogictest/test_files/aggregate.slt @@ -2020,14 +2020,6 @@ statement ok drop table t; - - -statement error DataFusion error: Execution error: Table 't_source' doesn't exist\. -drop table t_source; - -statement error DataFusion error: Execution error: Table 't' doesn't exist\. -drop table t; - query I select median(a) from (select 1 as a where 1=0); ---- @@ -2199,6 +2191,26 @@ NULL 1 10.1 10.1 10.1 10.1 0 NULL statement ok set datafusion.sql_parser.dialect = 'Generic'; +## Multiple distinct aggregates and dictionaries +statement ok +create table dict_test as values (1, arrow_cast('foo', 'Dictionary(Int32, Utf8)')), (2, arrow_cast('bar', 'Dictionary(Int32, Utf8)')); + +query I? +select * from dict_test; +---- +1 foo +2 bar + +query II +select count(distinct column1), count(distinct column2) from dict_test group by column1; +---- +1 1 +1 1 + +statement ok +drop table dict_test; + + # Prepare the table with dictionary values for testing statement ok CREATE TABLE value(x bigint) AS VALUES (1), (2), (3), (1), (3), (4), (5), (2); @@ -2282,6 +2294,13 @@ select max(x_dict) from value_dict group by x_dict % 2 order by max(x_dict); 4 5 +statement ok +drop table value + +statement ok +drop table value_dict + + # bool aggregation statement ok CREATE TABLE value_bool(x boolean, g int) AS VALUES (NULL, 0), (false, 0), (true, 0), (false, 1), (true, 2), (NULL, 3); From 747cb502c47dec4c78f03f0d0ef2aabb447f8cfa Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Marko=20Milenkovi=C4=87?= Date: Tue, 31 Oct 2023 14:25:09 +0000 Subject: [PATCH 162/572] MemoryReservation exposes MemoryConsumer (#8000) ... as a getter method. --- datafusion/execution/src/memory_pool/mod.rs | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/datafusion/execution/src/memory_pool/mod.rs b/datafusion/execution/src/memory_pool/mod.rs index bbd8f4be4f1c..71483e200af6 100644 --- a/datafusion/execution/src/memory_pool/mod.rs +++ b/datafusion/execution/src/memory_pool/mod.rs @@ -157,6 +157,11 @@ impl MemoryReservation { self.size } + /// Returns [MemoryConsumer] for this [MemoryReservation] + pub fn consumer(&self) -> &MemoryConsumer { + &self.registration.consumer + } + /// Frees all bytes from this reservation back to the underlying /// pool, returning the number of bytes freed. pub fn free(&mut self) -> usize { From 656c6a93fadcec7bc43a8a881dfaf55388b0b5c6 Mon Sep 17 00:00:00 2001 From: Jonah Gao Date: Tue, 31 Oct 2023 22:28:19 +0800 Subject: [PATCH 163/572] fix: generate logical plan for `UPDATE SET FROM` statement (#7984) --- datafusion/sql/src/statement.rs | 68 ++++++++++--------- datafusion/sqllogictest/test_files/update.slt | 36 ++++++++++ 2 files changed, 72 insertions(+), 32 deletions(-) diff --git a/datafusion/sql/src/statement.rs b/datafusion/sql/src/statement.rs index 834bddbf20a3..9d9c55361a5e 100644 --- a/datafusion/sql/src/statement.rs +++ b/datafusion/sql/src/statement.rs @@ -978,10 +978,9 @@ impl<'a, S: ContextProvider> SqlToRel<'a, S> { // Do a table lookup to verify the table exists let table_name = self.object_name_to_table_reference(table_name)?; let table_source = self.context_provider.get_table_source(table_name.clone())?; - let arrow_schema = (*table_source.schema()).clone(); let table_schema = Arc::new(DFSchema::try_from_qualified_schema( table_name.clone(), - &arrow_schema, + &table_source.schema(), )?); // Overwrite with assignment expressions @@ -1000,21 +999,10 @@ impl<'a, S: ContextProvider> SqlToRel<'a, S> { }) .collect::>>()?; - let values_and_types = table_schema - .fields() - .iter() - .map(|f| { - let col_name = f.name(); - let val = assign_map.remove(col_name).unwrap_or_else(|| { - ast::Expr::Identifier(ast::Ident::from(col_name.as_str())) - }); - (col_name, val, f.data_type()) - }) - .collect::>(); - - // Build scan - let from = from.unwrap_or(table); - let scan = self.plan_from_tables(vec![from], &mut planner_context)?; + // Build scan, join with from table if it exists. + let mut input_tables = vec![table]; + input_tables.extend(from); + let scan = self.plan_from_tables(input_tables, &mut planner_context)?; // Filter let source = match predicate_expr { @@ -1022,33 +1010,49 @@ impl<'a, S: ContextProvider> SqlToRel<'a, S> { Some(predicate_expr) => { let filter_expr = self.sql_to_expr( predicate_expr, - &table_schema, + scan.schema(), &mut planner_context, )?; let mut using_columns = HashSet::new(); expr_to_columns(&filter_expr, &mut using_columns)?; let filter_expr = normalize_col_with_schemas_and_ambiguity_check( filter_expr, - &[&[&table_schema]], + &[&[&scan.schema()]], &[using_columns], )?; LogicalPlan::Filter(Filter::try_new(filter_expr, Arc::new(scan))?) } }; - // Projection - let mut exprs = vec![]; - for (col_name, expr, dt) in values_and_types.into_iter() { - let mut expr = self.sql_to_expr(expr, &table_schema, &mut planner_context)?; - // Update placeholder's datatype to the type of the target column - if let datafusion_expr::Expr::Placeholder(placeholder) = &mut expr { - placeholder.data_type = - placeholder.data_type.take().or_else(|| Some(dt.clone())); - } - // Cast to target column type, if necessary - let expr = expr.cast_to(dt, source.schema())?.alias(col_name); - exprs.push(expr); - } + // Build updated values for each column, using the previous value if not modified + let exprs = table_schema + .fields() + .iter() + .map(|field| { + let expr = match assign_map.remove(field.name()) { + Some(new_value) => { + let mut expr = self.sql_to_expr( + new_value, + source.schema(), + &mut planner_context, + )?; + // Update placeholder's datatype to the type of the target column + if let datafusion_expr::Expr::Placeholder(placeholder) = &mut expr + { + placeholder.data_type = placeholder + .data_type + .take() + .or_else(|| Some(field.data_type().clone())); + } + // Cast to target column type, if necessary + expr.cast_to(field.data_type(), source.schema())? + } + None => datafusion_expr::Expr::Column(field.qualified_column()), + }; + Ok(expr.alias(field.name())) + }) + .collect::>>()?; + let source = project(source, exprs)?; let plan = LogicalPlan::Dml(DmlStatement { diff --git a/datafusion/sqllogictest/test_files/update.slt b/datafusion/sqllogictest/test_files/update.slt index 4542a262390c..cb8c6a4fac28 100644 --- a/datafusion/sqllogictest/test_files/update.slt +++ b/datafusion/sqllogictest/test_files/update.slt @@ -41,3 +41,39 @@ logical_plan Dml: op=[Update] table=[t1] --Projection: CAST(t1.c + CAST(Int64(1) AS Float64) AS Int32) AS a, CAST(t1.a AS Utf8) AS b, t1.c + Float64(1) AS c, CAST(t1.b AS Int32) AS d ----TableScan: t1 + +statement ok +create table t2(a int, b varchar, c double, d int); + +## set from subquery +query TT +explain update t1 set b = (select max(b) from t2 where t1.a = t2.a) +---- +logical_plan +Dml: op=[Update] table=[t1] +--Projection: t1.a AS a, () AS b, t1.c AS c, t1.d AS d +----Subquery: +------Projection: MAX(t2.b) +--------Aggregate: groupBy=[[]], aggr=[[MAX(t2.b)]] +----------Filter: outer_ref(t1.a) = t2.a +------------TableScan: t2 +----TableScan: t1 + +# set from other table +query TT +explain update t1 set b = t2.b, c = t2.a, d = 1 from t2 where t1.a = t2.a and t1.b > 'foo' and t2.c > 1.0; +---- +logical_plan +Dml: op=[Update] table=[t1] +--Projection: t1.a AS a, t2.b AS b, CAST(t2.a AS Float64) AS c, CAST(Int64(1) AS Int32) AS d +----Filter: t1.a = t2.a AND t1.b > Utf8("foo") AND t2.c > Float64(1) +------CrossJoin: +--------TableScan: t1 +--------TableScan: t2 + +statement ok +create table t3(a int, b varchar, c double, d int); + +# set from mutiple tables, sqlparser only supports from one table +query error DataFusion error: SQL error: ParserError\("Expected end of statement, found: ,"\) +explain update t1 set b = t2.b, c = t3.a, d = 1 from t2, t3 where t1.a = t2.a and t1.a = t3.a; \ No newline at end of file From 318578314676ef09f5a6b3408c7a91414344642a Mon Sep 17 00:00:00 2001 From: jokercurry <982458633@qq.com> Date: Wed, 1 Nov 2023 17:05:37 +0800 Subject: [PATCH 164/572] Create temporary files for reading or writing (#8005) * Create temporary files for reading or writing * nit * addr comment --------- Co-authored-by: zhongjingxiong --- datafusion-examples/examples/dataframe.rs | 36 +++++++++++++---------- 1 file changed, 21 insertions(+), 15 deletions(-) diff --git a/datafusion-examples/examples/dataframe.rs b/datafusion-examples/examples/dataframe.rs index 26fddcd226a9..ea01c53b1c62 100644 --- a/datafusion-examples/examples/dataframe.rs +++ b/datafusion-examples/examples/dataframe.rs @@ -18,7 +18,9 @@ use datafusion::arrow::datatypes::{DataType, Field, Schema}; use datafusion::error::Result; use datafusion::prelude::*; -use std::fs; +use std::fs::File; +use std::io::Write; +use tempfile::tempdir; /// This example demonstrates executing a simple query against an Arrow data source (Parquet) and /// fetching results, using the DataFrame trait @@ -41,12 +43,19 @@ async fn main() -> Result<()> { // print the results df.show().await?; + // create a csv file waiting to be written + let dir = tempdir()?; + let file_path = dir.path().join("example.csv"); + let file = File::create(&file_path)?; + write_csv_file(file); + // Reading CSV file with inferred schema example - let csv_df = example_read_csv_file_with_inferred_schema().await; + let csv_df = + example_read_csv_file_with_inferred_schema(file_path.to_str().unwrap()).await; csv_df.show().await?; // Reading CSV file with defined schema - let csv_df = example_read_csv_file_with_schema().await; + let csv_df = example_read_csv_file_with_schema(file_path.to_str().unwrap()).await; csv_df.show().await?; // Reading PARQUET file and print describe @@ -59,31 +68,28 @@ async fn main() -> Result<()> { } // Function to create an test CSV file -fn create_csv_file(path: String) { +fn write_csv_file(mut file: File) { // Create the data to put into the csv file with headers let content = r#"id,time,vote,unixtime,rating a1,"10 6, 2013",3,1381017600,5.0 a2,"08 9, 2013",2,1376006400,4.5"#; // write the data - fs::write(path, content).expect("Problem with writing file!"); + file.write_all(content.as_ref()) + .expect("Problem with writing file!"); } // Example to read data from a csv file with inferred schema -async fn example_read_csv_file_with_inferred_schema() -> DataFrame { - let path = "example.csv"; - // Create a csv file using the predefined function - create_csv_file(path.to_string()); +async fn example_read_csv_file_with_inferred_schema(file_path: &str) -> DataFrame { // Create a session context let ctx = SessionContext::new(); // Register a lazy DataFrame using the context - ctx.read_csv(path, CsvReadOptions::default()).await.unwrap() + ctx.read_csv(file_path, CsvReadOptions::default()) + .await + .unwrap() } // Example to read csv file with a defined schema for the csv file -async fn example_read_csv_file_with_schema() -> DataFrame { - let path = "example.csv"; - // Create a csv file using the predefined function - create_csv_file(path.to_string()); +async fn example_read_csv_file_with_schema(file_path: &str) -> DataFrame { // Create a session context let ctx = SessionContext::new(); // Define the schema @@ -101,5 +107,5 @@ async fn example_read_csv_file_with_schema() -> DataFrame { ..Default::default() }; // Register a lazy DataFrame by using the context and option provider - ctx.read_csv(path, csv_read_option).await.unwrap() + ctx.read_csv(file_path, csv_read_option).await.unwrap() } From aef95edf9bf1324be8642146c882b8d4c89a3785 Mon Sep 17 00:00:00 2001 From: Weston Pace Date: Wed, 1 Nov 2023 02:05:50 -0700 Subject: [PATCH 165/572] doc: minor fix to SortExec::with_fetch comment (#8011) --- datafusion/physical-plan/src/sorts/sort.rs | 8 +++++++- 1 file changed, 7 insertions(+), 1 deletion(-) diff --git a/datafusion/physical-plan/src/sorts/sort.rs b/datafusion/physical-plan/src/sorts/sort.rs index c7d676493f04..08fa2c25d792 100644 --- a/datafusion/physical-plan/src/sorts/sort.rs +++ b/datafusion/physical-plan/src/sorts/sort.rs @@ -735,7 +735,13 @@ impl SortExec { self } - /// Whether this `SortExec` preserves partitioning of the children + /// Modify how many rows to include in the result + /// + /// If None, then all rows will be returned, in sorted order. + /// If Some, then only the top `fetch` rows will be returned. + /// This can reduce the memory pressure required by the sort + /// operation since rows that are not going to be included + /// can be dropped. pub fn with_fetch(mut self, fetch: Option) -> Self { self.fetch = fetch; self From 69ba82f03e3fa76a18a712fa228402f6d9324879 Mon Sep 17 00:00:00 2001 From: jokercurry <982458633@qq.com> Date: Wed, 1 Nov 2023 19:16:50 +0800 Subject: [PATCH 166/572] Fix: dataframe_subquery example Optimizer rule `common_sub_expression_eliminate` failed (#8016) * Fix: Optimizer rule 'common_sub_expression_eliminate' failed * nit * nit * nit --------- Co-authored-by: zhongjingxiong --- datafusion-examples/examples/dataframe_subquery.rs | 11 ++++++----- 1 file changed, 6 insertions(+), 5 deletions(-) diff --git a/datafusion-examples/examples/dataframe_subquery.rs b/datafusion-examples/examples/dataframe_subquery.rs index 94049e59b3ab..9fb61008b9f6 100644 --- a/datafusion-examples/examples/dataframe_subquery.rs +++ b/datafusion-examples/examples/dataframe_subquery.rs @@ -15,6 +15,7 @@ // specific language governing permissions and limitations // under the License. +use arrow_schema::DataType; use std::sync::Arc; use datafusion::error::Result; @@ -38,7 +39,7 @@ async fn main() -> Result<()> { Ok(()) } -//select c1,c2 from t1 where (select avg(t2.c2) from t2 where t1.c1 = t2.c1)>0 limit 10; +//select c1,c2 from t1 where (select avg(t2.c2) from t2 where t1.c1 = t2.c1)>0 limit 3; async fn where_scalar_subquery(ctx: &SessionContext) -> Result<()> { ctx.table("t1") .await? @@ -46,7 +47,7 @@ async fn where_scalar_subquery(ctx: &SessionContext) -> Result<()> { scalar_subquery(Arc::new( ctx.table("t2") .await? - .filter(col("t1.c1").eq(col("t2.c1")))? + .filter(out_ref_col(DataType::Utf8, "t1.c1").eq(col("t2.c1")))? .aggregate(vec![], vec![avg(col("t2.c2"))])? .select(vec![avg(col("t2.c2"))])? .into_unoptimized_plan(), @@ -60,7 +61,7 @@ async fn where_scalar_subquery(ctx: &SessionContext) -> Result<()> { Ok(()) } -//SELECT t1.c1, t1.c2 FROM t1 WHERE t1.c2 in (select max(t2.c2) from t2 where t2.c1 > 0 ) limit 10 +//SELECT t1.c1, t1.c2 FROM t1 WHERE t1.c2 in (select max(t2.c2) from t2 where t2.c1 > 0 ) limit 3; async fn where_in_subquery(ctx: &SessionContext) -> Result<()> { ctx.table("t1") .await? @@ -82,14 +83,14 @@ async fn where_in_subquery(ctx: &SessionContext) -> Result<()> { Ok(()) } -//SELECT t1.c1, t1.c2 FROM t1 WHERE EXISTS (select t2.c2 from t2 where t1.c1 = t2.c1) limit 10 +//SELECT t1.c1, t1.c2 FROM t1 WHERE EXISTS (select t2.c2 from t2 where t1.c1 = t2.c1) limit 3; async fn where_exist_subquery(ctx: &SessionContext) -> Result<()> { ctx.table("t1") .await? .filter(exists(Arc::new( ctx.table("t2") .await? - .filter(col("t1.c1").eq(col("t2.c1")))? + .filter(out_ref_col(DataType::Utf8, "t1.c1").eq(col("t2.c1")))? .select(vec![col("t2.c2")])? .into_unoptimized_plan(), )))? From 4e60cddb1caa8f3f552a6f9d4aa6fd3a3653fb58 Mon Sep 17 00:00:00 2001 From: Raphael Taylor-Davies <1781103+tustvold@users.noreply.github.com> Date: Wed, 1 Nov 2023 12:14:52 +0000 Subject: [PATCH 167/572] Percent Decode URL Paths (#8009) (#8012) * Treat ListingTableUrl as URL-encoded (#8009) * Update lockfile * Review feedback --- datafusion-cli/Cargo.lock | 1 - datafusion/core/Cargo.toml | 1 - datafusion/core/src/datasource/listing/url.rs | 48 +++++++++++++++---- 3 files changed, 38 insertions(+), 12 deletions(-) diff --git a/datafusion-cli/Cargo.lock b/datafusion-cli/Cargo.lock index 268f9e28427a..dc828f018fd5 100644 --- a/datafusion-cli/Cargo.lock +++ b/datafusion-cli/Cargo.lock @@ -1130,7 +1130,6 @@ dependencies = [ "object_store", "parking_lot", "parquet", - "percent-encoding", "pin-project-lite", "rand", "sqlparser", diff --git a/datafusion/core/Cargo.toml b/datafusion/core/Cargo.toml index a9614934673b..f1542fca0026 100644 --- a/datafusion/core/Cargo.toml +++ b/datafusion/core/Cargo.toml @@ -82,7 +82,6 @@ num_cpus = "1.13.0" object_store = "0.7.0" parking_lot = "0.12" parquet = { workspace = true, optional = true } -percent-encoding = "2.2.0" pin-project-lite = "^0.2.7" rand = "0.8" sqlparser = { workspace = true } diff --git a/datafusion/core/src/datasource/listing/url.rs b/datafusion/core/src/datasource/listing/url.rs index 4d1ca4853a73..9197e37adbd5 100644 --- a/datafusion/core/src/datasource/listing/url.rs +++ b/datafusion/core/src/datasource/listing/url.rs @@ -27,7 +27,6 @@ use itertools::Itertools; use log::debug; use object_store::path::Path; use object_store::{ObjectMeta, ObjectStore}; -use percent_encoding; use std::sync::Arc; use url::Url; @@ -46,6 +45,16 @@ pub struct ListingTableUrl { impl ListingTableUrl { /// Parse a provided string as a `ListingTableUrl` /// + /// # URL Encoding + /// + /// URL paths are expected to be URL-encoded. That is, the URL for a file named `bar%2Efoo` + /// would be `file:///bar%252Efoo`, as per the [URL] specification. + /// + /// It should be noted that some tools, such as the AWS CLI, take a different approach and + /// instead interpret the URL path verbatim. For example the object `bar%2Efoo` would be + /// addressed as `s3://BUCKET/bar%252Efoo` using [`ListingTableUrl`] but `s3://BUCKET/bar%2Efoo` + /// when using the aws-cli. + /// /// # Paths without a Scheme /// /// If no scheme is provided, or the string is an absolute filesystem path @@ -77,6 +86,7 @@ impl ListingTableUrl { /// filter when listing files from object storage /// /// [file URI]: https://en.wikipedia.org/wiki/File_URI_scheme + /// [URL]: https://url.spec.whatwg.org/ pub fn parse(s: impl AsRef) -> Result { let s = s.as_ref(); @@ -86,7 +96,7 @@ impl ListingTableUrl { } match Url::parse(s) { - Ok(url) => Ok(Self::new(url, None)), + Ok(url) => Self::try_new(url, None), Err(url::ParseError::RelativeUrlWithoutBase) => Self::parse_path(s), Err(e) => Err(DataFusionError::External(Box::new(e))), } @@ -138,15 +148,13 @@ impl ListingTableUrl { .map_err(|_| DataFusionError::Internal(format!("Can not open path: {s}")))?; // TODO: Currently we do not have an IO-related error variant that accepts () // or a string. Once we have such a variant, change the error type above. - Ok(Self::new(url, glob)) + Self::try_new(url, glob) } /// Creates a new [`ListingTableUrl`] from a url and optional glob expression - fn new(url: Url, glob: Option) -> Self { - let decoded_path = - percent_encoding::percent_decode_str(url.path()).decode_utf8_lossy(); - let prefix = Path::from(decoded_path.as_ref()); - Self { url, prefix, glob } + fn try_new(url: Url, glob: Option) -> Result { + let prefix = Path::from_url_path(url.path())?; + Ok(Self { url, prefix, glob }) } /// Returns the URL scheme @@ -286,6 +294,7 @@ fn split_glob_expression(path: &str) -> Option<(&str, &str)> { #[cfg(test)] mod tests { use super::*; + use tempfile::tempdir; #[test] fn test_prefix_path() { @@ -317,8 +326,27 @@ mod tests { let url = ListingTableUrl::parse("file:///foo/bar?").unwrap(); assert_eq!(url.prefix.as_ref(), "foo/bar"); - let url = ListingTableUrl::parse("file:///foo/😺").unwrap(); - assert_eq!(url.prefix.as_ref(), "foo/%F0%9F%98%BA"); + let err = ListingTableUrl::parse("file:///foo/😺").unwrap_err(); + assert_eq!(err.to_string(), "Object Store error: Encountered object with invalid path: Error parsing Path \"/foo/😺\": Encountered illegal character sequence \"😺\" whilst parsing path segment \"😺\""); + + let url = ListingTableUrl::parse("file:///foo/bar%2Efoo").unwrap(); + assert_eq!(url.prefix.as_ref(), "foo/bar.foo"); + + let url = ListingTableUrl::parse("file:///foo/bar%2Efoo").unwrap(); + assert_eq!(url.prefix.as_ref(), "foo/bar.foo"); + + let url = ListingTableUrl::parse("file:///foo/bar%252Ffoo").unwrap(); + assert_eq!(url.prefix.as_ref(), "foo/bar%2Ffoo"); + + let url = ListingTableUrl::parse("file:///foo/a%252Fb.txt").unwrap(); + assert_eq!(url.prefix.as_ref(), "foo/a%2Fb.txt"); + + let dir = tempdir().unwrap(); + let path = dir.path().join("bar%2Ffoo"); + std::fs::File::create(&path).unwrap(); + + let url = ListingTableUrl::parse(path.to_str().unwrap()).unwrap(); + assert!(url.prefix.as_ref().ends_with("bar%2Ffoo"), "{}", url.prefix); } #[test] From e98625cb0ecab970c9b13e8a4c2a49c37d2505b2 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E5=BC=A0=E6=9E=97=E4=BC=9F?= Date: Thu, 2 Nov 2023 00:18:23 +0800 Subject: [PATCH 168/572] Minor: Extract common deps into workspace (#7982) * Improve datafusion-* * More common crates * Extract async-trait * Extract more * Fix cli --------- Co-authored-by: Andrew Lamb --- Cargo.toml | 32 ++++++++++++++++++ benchmarks/Cargo.toml | 10 +++--- datafusion-examples/Cargo.toml | 20 +++++------ datafusion/common/Cargo.toml | 2 +- datafusion/core/Cargo.toml | 52 ++++++++++++++--------------- datafusion/execution/Cargo.toml | 20 +++++------ datafusion/expr/Cargo.toml | 6 ++-- datafusion/optimizer/Cargo.toml | 12 +++---- datafusion/physical-expr/Cargo.toml | 14 ++++---- datafusion/physical-plan/Cargo.toml | 22 ++++++------ datafusion/proto/Cargo.toml | 8 ++--- datafusion/sql/Cargo.toml | 10 +++--- datafusion/sqllogictest/Cargo.toml | 22 ++++++------ datafusion/substrait/Cargo.toml | 6 ++-- datafusion/wasmtest/Cargo.toml | 10 +++--- test-utils/Cargo.toml | 2 +- 16 files changed, 140 insertions(+), 108 deletions(-) diff --git a/Cargo.toml b/Cargo.toml index 22d5f2f64464..1a2f4a84af38 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -54,9 +54,41 @@ arrow-array = { version = "48.0.0", default-features = false, features = ["chron arrow-buffer = { version = "48.0.0", default-features = false } arrow-flight = { version = "48.0.0", features = ["flight-sql-experimental"] } arrow-schema = { version = "48.0.0", default-features = false } +async-trait = "0.1.73" +bigdecimal = "0.4.1" +bytes = "1.4" +ctor = "0.2.0" +datafusion = { path = "datafusion/core" } +datafusion-common = { path = "datafusion/common" } +datafusion-expr = { path = "datafusion/expr" } +datafusion-sql = { path = "datafusion/sql" } +datafusion-optimizer = { path = "datafusion/optimizer" } +datafusion-physical-expr = { path = "datafusion/physical-expr" } +datafusion-physical-plan = { path = "datafusion/physical-plan" } +datafusion-execution = { path = "datafusion/execution" } +datafusion-proto = { path = "datafusion/proto" } +datafusion-sqllogictest = { path = "datafusion/sqllogictest" } +datafusion-substrait = { path = "datafusion/substrait" } +dashmap = "5.4.0" +doc-comment = "0.3" +env_logger = "0.10" +futures = "0.3" +half = "2.2.1" +indexmap = "2.0.0" +itertools = "0.11" +log = "^0.4" +num_cpus = "1.13.0" +object_store = "0.7.0" +parking_lot = "0.12" parquet = { version = "48.0.0", features = ["arrow", "async", "object_store"] } +rand = "0.8" +rstest = "0.18.0" +serde_json = "1" sqlparser = { version = "0.39.0", features = ["visitor"] } +tempfile = "3" +thiserror = "1.0.44" chrono = { version = "0.4.31", default-features = false } +url = "2.2" [profile.release] codegen-units = 1 diff --git a/benchmarks/Cargo.toml b/benchmarks/Cargo.toml index 0def335521ce..ce0a4267fc28 100644 --- a/benchmarks/Cargo.toml +++ b/benchmarks/Cargo.toml @@ -36,14 +36,14 @@ snmalloc = ["snmalloc-rs"] arrow = { workspace = true } datafusion = { path = "../datafusion/core", version = "32.0.0" } datafusion-common = { path = "../datafusion/common", version = "32.0.0" } -env_logger = "0.10" -futures = "0.3" -log = "^0.4" +env_logger = { workspace = true } +futures = { workspace = true } +log = { workspace = true } mimalloc = { version = "0.1", optional = true, default-features = false } -num_cpus = "1.13.0" +num_cpus = { workspace = true } parquet = { workspace = true } serde = { version = "1.0.136", features = ["derive"] } -serde_json = "1.0.78" +serde_json = { workspace = true } snmalloc-rs = { version = "0.3", optional = true } structopt = { version = "0.3", default-features = false } test-utils = { path = "../test-utils/", version = "0.1.0" } diff --git a/datafusion-examples/Cargo.toml b/datafusion-examples/Cargo.toml index 7350c4ab981f..57691520a401 100644 --- a/datafusion-examples/Cargo.toml +++ b/datafusion-examples/Cargo.toml @@ -33,26 +33,26 @@ rust-version = { workspace = true } arrow = { workspace = true } arrow-flight = { workspace = true } arrow-schema = { workspace = true } -async-trait = "0.1.41" -bytes = "1.4" -dashmap = "5.4" +async-trait = { workspace = true } +bytes = { workspace = true } +dashmap = { workspace = true } datafusion = { path = "../datafusion/core", features = ["avro"] } datafusion-common = { path = "../datafusion/common" } datafusion-expr = { path = "../datafusion/expr" } datafusion-optimizer = { path = "../datafusion/optimizer" } datafusion-sql = { path = "../datafusion/sql" } -env_logger = "0.10" -futures = "0.3" -log = "0.4" +env_logger = { workspace = true } +futures = { workspace = true } +log = { workspace = true } mimalloc = { version = "0.1", default-features = false } -num_cpus = "1.13.0" +num_cpus = { workspace = true } object_store = { version = "0.7.0", features = ["aws", "http"] } prost = { version = "0.12", default-features = false } prost-derive = { version = "0.11", default-features = false } serde = { version = "1.0.136", features = ["derive"] } -serde_json = "1.0.82" -tempfile = "3" +serde_json = { workspace = true } +tempfile = { workspace = true } tokio = { version = "1.0", features = ["macros", "rt", "rt-multi-thread", "sync", "parking_lot"] } tonic = "0.10" -url = "2.2" +url = { workspace = true } uuid = "1.2" diff --git a/datafusion/common/Cargo.toml b/datafusion/common/Cargo.toml index 87087c50a2d2..d04db86b7830 100644 --- a/datafusion/common/Cargo.toml +++ b/datafusion/common/Cargo.toml @@ -46,7 +46,7 @@ arrow-buffer = { workspace = true } arrow-schema = { workspace = true } chrono = { workspace = true } half = { version = "2.1", default-features = false } -num_cpus = "1.13.0" +num_cpus = { workspace = true } object_store = { version = "0.7.0", default-features = false, optional = true } parquet = { workspace = true, optional = true } pyo3 = { version = "0.20.0", optional = true } diff --git a/datafusion/core/Cargo.toml b/datafusion/core/Cargo.toml index f1542fca0026..4015ba439e67 100644 --- a/datafusion/core/Cargo.toml +++ b/datafusion/core/Cargo.toml @@ -57,61 +57,61 @@ arrow = { workspace = true } arrow-array = { workspace = true } arrow-schema = { workspace = true } async-compression = { version = "0.4.0", features = ["bzip2", "gzip", "xz", "zstd", "futures-io", "tokio"], optional = true } -async-trait = "0.1.73" -bytes = "1.4" +async-trait = { workspace = true } +bytes = { workspace = true } bzip2 = { version = "0.4.3", optional = true } chrono = { workspace = true } -dashmap = "5.4.0" +dashmap = { workspace = true } datafusion-common = { path = "../common", version = "32.0.0", features = ["object_store"], default-features = false } -datafusion-execution = { path = "../execution", version = "32.0.0" } -datafusion-expr = { path = "../expr", version = "32.0.0" } +datafusion-execution = { workspace = true } +datafusion-expr = { workspace = true } datafusion-optimizer = { path = "../optimizer", version = "32.0.0", default-features = false } datafusion-physical-expr = { path = "../physical-expr", version = "32.0.0", default-features = false } -datafusion-physical-plan = { path = "../physical-plan", version = "32.0.0", default-features = false } -datafusion-sql = { path = "../sql", version = "32.0.0" } +datafusion-physical-plan = { workspace = true } +datafusion-sql = { workspace = true } flate2 = { version = "1.0.24", optional = true } -futures = "0.3" +futures = { workspace = true } glob = "0.3.0" half = { version = "2.1", default-features = false } hashbrown = { version = "0.14", features = ["raw"] } -indexmap = "2.0.0" -itertools = "0.11" -log = "^0.4" +indexmap = { workspace = true } +itertools = { workspace = true } +log = { workspace = true } num-traits = { version = "0.2", optional = true } -num_cpus = "1.13.0" -object_store = "0.7.0" -parking_lot = "0.12" +num_cpus = { workspace = true } +object_store = { workspace = true } +parking_lot = { workspace = true } parquet = { workspace = true, optional = true } pin-project-lite = "^0.2.7" -rand = "0.8" +rand = { workspace = true } sqlparser = { workspace = true } -tempfile = "3" +tempfile = { workspace = true } tokio = { version = "1.28", features = ["macros", "rt", "rt-multi-thread", "sync", "fs", "parking_lot"] } tokio-util = { version = "0.7.4", features = ["io"] } -url = "2.2" +url = { workspace = true } uuid = { version = "1.0", features = ["v4"] } xz2 = { version = "0.1", optional = true } zstd = { version = "0.13", optional = true, default-features = false } [dev-dependencies] -async-trait = "0.1.53" -bigdecimal = "0.4.1" +async-trait = { workspace = true } +bigdecimal = { workspace = true } criterion = { version = "0.5", features = ["async_tokio"] } csv = "1.1.6" -ctor = "0.2.0" -doc-comment = "0.3" -env_logger = "0.10" -half = "2.2.1" +ctor = { workspace = true } +doc-comment = { workspace = true } +env_logger = { workspace = true } +half = { workspace = true } postgres-protocol = "0.6.4" postgres-types = { version = "0.2.4", features = ["derive", "with-chrono-0_4"] } rand = { version = "0.8", features = ["small_rng"] } rand_distr = "0.4.3" regex = "1.5.4" -rstest = "0.18.0" +rstest = { workspace = true } rust_decimal = { version = "1.27.0", features = ["tokio-pg"] } -serde_json = "1" +serde_json = { workspace = true } test-utils = { path = "../../test-utils" } -thiserror = "1.0.37" +thiserror = { workspace = true } tokio-postgres = "0.7.7" [target.'cfg(not(target_os = "windows"))'.dev-dependencies] nix = { version = "0.27.1", features = ["fs"] } diff --git a/datafusion/execution/Cargo.toml b/datafusion/execution/Cargo.toml index 54a2a6d8f3df..e9bb87e9f8ac 100644 --- a/datafusion/execution/Cargo.toml +++ b/datafusion/execution/Cargo.toml @@ -35,14 +35,14 @@ path = "src/lib.rs" [dependencies] arrow = { workspace = true } chrono = { version = "0.4", default-features = false } -dashmap = "5.4.0" -datafusion-common = { path = "../common", version = "32.0.0" } -datafusion-expr = { path = "../expr", version = "32.0.0" } -futures = "0.3" +dashmap = { workspace = true } +datafusion-common = { workspace = true } +datafusion-expr = { workspace = true } +futures = { workspace = true } hashbrown = { version = "0.14", features = ["raw"] } -log = "^0.4" -object_store = "0.7.0" -parking_lot = "0.12" -rand = "0.8" -tempfile = "3" -url = "2.2" +log = { workspace = true } +object_store = { workspace = true } +parking_lot = { workspace = true } +rand = { workspace = true } +tempfile = { workspace = true } +url = { workspace = true } diff --git a/datafusion/expr/Cargo.toml b/datafusion/expr/Cargo.toml index a7919a557ad7..5b1b42153877 100644 --- a/datafusion/expr/Cargo.toml +++ b/datafusion/expr/Cargo.toml @@ -38,11 +38,11 @@ path = "src/lib.rs" ahash = { version = "0.8", default-features = false, features = ["runtime-rng"] } arrow = { workspace = true } arrow-array = { workspace = true } -datafusion-common = { path = "../common", version = "32.0.0", default-features = false } +datafusion-common = { workspace = true } sqlparser = { workspace = true } strum = { version = "0.25.0", features = ["derive"] } strum_macros = "0.25.0" [dev-dependencies] -ctor = "0.2.0" -env_logger = "0.10" +ctor = { workspace = true } +env_logger = { workspace = true } diff --git a/datafusion/optimizer/Cargo.toml b/datafusion/optimizer/Cargo.toml index dce05058b826..797dd17a26b5 100644 --- a/datafusion/optimizer/Cargo.toml +++ b/datafusion/optimizer/Cargo.toml @@ -40,17 +40,17 @@ unicode_expressions = ["datafusion-physical-expr/unicode_expressions"] [dependencies] arrow = { workspace = true } -async-trait = "0.1.41" +async-trait = { workspace = true } chrono = { workspace = true } -datafusion-common = { path = "../common", version = "32.0.0", default-features = false } -datafusion-expr = { path = "../expr", version = "32.0.0" } +datafusion-common = { workspace = true } +datafusion-expr = { workspace = true } datafusion-physical-expr = { path = "../physical-expr", version = "32.0.0", default-features = false } hashbrown = { version = "0.14", features = ["raw"] } -itertools = "0.11" -log = "^0.4" +itertools = { workspace = true } +log = { workspace = true } regex-syntax = "0.8.0" [dev-dependencies] -ctor = "0.2.0" +ctor = { workspace = true } datafusion-sql = { path = "../sql", version = "32.0.0" } env_logger = "0.10.0" diff --git a/datafusion/physical-expr/Cargo.toml b/datafusion/physical-expr/Cargo.toml index f7c0221756fd..4be625e384b9 100644 --- a/datafusion/physical-expr/Cargo.toml +++ b/datafusion/physical-expr/Cargo.toml @@ -49,19 +49,19 @@ base64 = { version = "0.21", optional = true } blake2 = { version = "^0.10.2", optional = true } blake3 = { version = "1.0", optional = true } chrono = { workspace = true } -datafusion-common = { path = "../common", version = "32.0.0", default-features = false } -datafusion-expr = { path = "../expr", version = "32.0.0" } +datafusion-common = { workspace = true } +datafusion-expr = { workspace = true } half = { version = "2.1", default-features = false } hashbrown = { version = "0.14", features = ["raw"] } hex = { version = "0.4", optional = true } -indexmap = "2.0.0" +indexmap = { workspace = true } itertools = { version = "0.11", features = ["use_std"] } libc = "0.2.140" -log = "^0.4" +log = { workspace = true } md-5 = { version = "^0.10.0", optional = true } paste = "^1.0" petgraph = "0.6.2" -rand = "0.8" +rand = { workspace = true } regex = { version = "1.8", optional = true } sha2 = { version = "^0.10.1", optional = true } unicode-segmentation = { version = "^1.7.1", optional = true } @@ -69,8 +69,8 @@ uuid = { version = "^1.2", features = ["v4"] } [dev-dependencies] criterion = "0.5" -rand = "0.8" -rstest = "0.18.0" +rand = { workspace = true } +rstest = { workspace = true } [[bench]] harness = false diff --git a/datafusion/physical-plan/Cargo.toml b/datafusion/physical-plan/Cargo.toml index 033d2aa187dd..82c8f49a764f 100644 --- a/datafusion/physical-plan/Cargo.toml +++ b/datafusion/physical-plan/Cargo.toml @@ -38,26 +38,26 @@ arrow = { workspace = true } arrow-array = { workspace = true } arrow-buffer = { workspace = true } arrow-schema = { workspace = true } -async-trait = "0.1.41" +async-trait = { workspace = true } chrono = { version = "0.4.23", default-features = false } -datafusion-common = { path = "../common", version = "32.0.0", default-features = false } -datafusion-execution = { path = "../execution", version = "32.0.0" } -datafusion-expr = { path = "../expr", version = "32.0.0" } -datafusion-physical-expr = { path = "../physical-expr", version = "32.0.0" } -futures = "0.3" +datafusion-common = { workspace = true } +datafusion-execution = { workspace = true } +datafusion-expr = { workspace = true } +datafusion-physical-expr = { workspace = true } +futures = { workspace = true } half = { version = "2.1", default-features = false } hashbrown = { version = "0.14", features = ["raw"] } -indexmap = "2.0.0" +indexmap = { workspace = true } itertools = { version = "0.11", features = ["use_std"] } -log = "^0.4" +log = { workspace = true } once_cell = "1.18.0" -parking_lot = "0.12" +parking_lot = { workspace = true } pin-project-lite = "^0.2.7" -rand = "0.8" +rand = { workspace = true } tokio = { version = "1.28", features = ["sync", "fs", "parking_lot"] } uuid = { version = "^1.2", features = ["v4"] } [dev-dependencies] -rstest = "0.18.0" +rstest = { workspace = true } termtree = "0.4.1" tokio = { version = "1.28", features = ["macros", "rt", "rt-multi-thread", "sync", "fs", "parking_lot"] } diff --git a/datafusion/proto/Cargo.toml b/datafusion/proto/Cargo.toml index 5e52dadc0b18..72a4df66ebd7 100644 --- a/datafusion/proto/Cargo.toml +++ b/datafusion/proto/Cargo.toml @@ -44,14 +44,14 @@ parquet = ["datafusion/parquet", "datafusion-common/parquet"] arrow = { workspace = true } chrono = { workspace = true } datafusion = { path = "../core", version = "32.0.0" } -datafusion-common = { path = "../common", version = "32.0.0", default-features = false } -datafusion-expr = { path = "../expr", version = "32.0.0" } +datafusion-common = { workspace = true } +datafusion-expr = { workspace = true } object_store = { version = "0.7.0" } pbjson = { version = "0.5", optional = true } prost = "0.12.0" serde = { version = "1.0", optional = true } -serde_json = { version = "1.0", optional = true } +serde_json = { workspace = true, optional = true } [dev-dependencies] -doc-comment = "0.3" +doc-comment = { workspace = true } tokio = "1.18" diff --git a/datafusion/sql/Cargo.toml b/datafusion/sql/Cargo.toml index fe60feb6ab1f..b91a2ac1fbd7 100644 --- a/datafusion/sql/Cargo.toml +++ b/datafusion/sql/Cargo.toml @@ -39,13 +39,13 @@ unicode_expressions = [] [dependencies] arrow = { workspace = true } arrow-schema = { workspace = true } -datafusion-common = { path = "../common", version = "32.0.0", default-features = false } -datafusion-expr = { path = "../expr", version = "32.0.0" } -log = "^0.4" +datafusion-common = { workspace = true } +datafusion-expr = { workspace = true } +log = { workspace = true } sqlparser = { workspace = true } [dev-dependencies] -ctor = "0.2.0" -env_logger = "0.10" +ctor = { workspace = true } +env_logger = { workspace = true } paste = "^1.0" rstest = "0.18" diff --git a/datafusion/sqllogictest/Cargo.toml b/datafusion/sqllogictest/Cargo.toml index f1a730351417..07debf179529 100644 --- a/datafusion/sqllogictest/Cargo.toml +++ b/datafusion/sqllogictest/Cargo.toml @@ -32,24 +32,24 @@ path = "src/lib.rs" [dependencies] arrow = { workspace = true } -async-trait = "0.1.41" -bigdecimal = "0.4.1" +async-trait = { workspace = true } +bigdecimal = { workspace = true } bytes = { version = "1.4.0", optional = true } chrono = { workspace = true, optional = true } datafusion = { path = "../core", version = "32.0.0" } -datafusion-common = { path = "../common", version = "32.0.0", default-features = false } +datafusion-common = { workspace = true } futures = { version = "0.3.28" } -half = "2.2.1" -itertools = "0.11" -log = "^0.4" -object_store = "0.7.0" +half = { workspace = true } +itertools = { workspace = true } +log = { workspace = true } +object_store = { workspace = true } postgres-protocol = { version = "0.6.4", optional = true } postgres-types = { version = "0.2.4", optional = true } rust_decimal = { version = "1.27.0" } sqllogictest = "0.17.0" sqlparser = { workspace = true } -tempfile = "3" -thiserror = "1.0.44" +tempfile = { workspace = true } +thiserror = { workspace = true } tokio = { version = "1.0" } tokio-postgres = { version = "0.7.7", optional = true } @@ -58,8 +58,8 @@ avro = ["datafusion/avro"] postgres = ["bytes", "chrono", "tokio-postgres", "postgres-types", "postgres-protocol"] [dev-dependencies] -env_logger = "0.10" -num_cpus = "1.13.0" +env_logger = { workspace = true } +num_cpus = { workspace = true } [[test]] harness = false diff --git a/datafusion/substrait/Cargo.toml b/datafusion/substrait/Cargo.toml index 67d31caca260..585cb6e43d15 100644 --- a/datafusion/substrait/Cargo.toml +++ b/datafusion/substrait/Cargo.toml @@ -30,9 +30,9 @@ rust-version = "1.70" [dependencies] async-recursion = "1.0" chrono = { workspace = true } -datafusion = { version = "32.0.0", path = "../core" } -itertools = "0.11" -object_store = "0.7.0" +datafusion = { workspace = true } +itertools = { workspace = true } +object_store = { workspace = true } prost = "0.12" prost-types = "0.12" substrait = "0.18.0" diff --git a/datafusion/wasmtest/Cargo.toml b/datafusion/wasmtest/Cargo.toml index 3195c989e223..882b02bcc84b 100644 --- a/datafusion/wasmtest/Cargo.toml +++ b/datafusion/wasmtest/Cargo.toml @@ -38,11 +38,11 @@ crate-type = ["cdylib", "rlib",] # code size when deploying. console_error_panic_hook = { version = "0.1.1", optional = true } -datafusion-common = { path = "../common", version = "32.0.0", default-features = false } -datafusion-expr = { path = "../expr" } -datafusion-optimizer = { path = "../optimizer" } -datafusion-physical-expr = { path = "../physical-expr" } -datafusion-sql = { path = "../sql" } +datafusion-common = { workspace = true } +datafusion-expr = { workspace = true } +datafusion-optimizer = { workspace = true } +datafusion-physical-expr = { workspace = true } +datafusion-sql = { workspace = true } # getrandom must be compiled with js feature getrandom = { version = "0.2.8", features = ["js"] } diff --git a/test-utils/Cargo.toml b/test-utils/Cargo.toml index 5ab10e42cf68..b9c4db17c098 100644 --- a/test-utils/Cargo.toml +++ b/test-utils/Cargo.toml @@ -26,4 +26,4 @@ edition = { workspace = true } arrow = { workspace = true } datafusion-common = { path = "../datafusion/common" } env_logger = "0.10.0" -rand = "0.8" +rand = { workspace = true } From 7d1cf9174c7b92af38e732cfafdd67acb243a5c3 Mon Sep 17 00:00:00 2001 From: Ruihang Xia Date: Thu, 2 Nov 2023 00:19:17 +0800 Subject: [PATCH 169/572] minor: change some plan_err to exec_err (#7996) * minor: change some plan_err to exec_err Signed-off-by: Ruihang Xia * change unreachable code to internal error Signed-off-by: Ruihang Xia --------- Signed-off-by: Ruihang Xia --- datafusion/physical-expr/src/expressions/column.rs | 3 +-- datafusion/physical-expr/src/expressions/no_op.rs | 4 ++-- datafusion/physical-expr/src/sort_expr.rs | 4 ++-- 3 files changed, 5 insertions(+), 6 deletions(-) diff --git a/datafusion/physical-expr/src/expressions/column.rs b/datafusion/physical-expr/src/expressions/column.rs index b7b5895db6d3..62da8ff9ed44 100644 --- a/datafusion/physical-expr/src/expressions/column.rs +++ b/datafusion/physical-expr/src/expressions/column.rs @@ -28,7 +28,6 @@ use arrow::{ datatypes::{DataType, Schema}, record_batch::RecordBatch, }; -use datafusion_common::plan_err; use datafusion_common::{internal_err, DataFusionError, Result}; use datafusion_expr::ColumnarValue; @@ -176,7 +175,7 @@ impl PhysicalExpr for UnKnownColumn { /// Evaluate the expression fn evaluate(&self, _batch: &RecordBatch) -> Result { - plan_err!("UnKnownColumn::evaluate() should not be called") + internal_err!("UnKnownColumn::evaluate() should not be called") } fn children(&self) -> Vec> { diff --git a/datafusion/physical-expr/src/expressions/no_op.rs b/datafusion/physical-expr/src/expressions/no_op.rs index 497fb42fe4df..95e6879a6c2d 100644 --- a/datafusion/physical-expr/src/expressions/no_op.rs +++ b/datafusion/physical-expr/src/expressions/no_op.rs @@ -28,7 +28,7 @@ use arrow::{ use crate::physical_expr::down_cast_any_ref; use crate::PhysicalExpr; -use datafusion_common::{plan_err, DataFusionError, Result}; +use datafusion_common::{internal_err, DataFusionError, Result}; use datafusion_expr::ColumnarValue; /// A place holder expression, can not be evaluated. @@ -65,7 +65,7 @@ impl PhysicalExpr for NoOp { } fn evaluate(&self, _batch: &RecordBatch) -> Result { - plan_err!("NoOp::evaluate() should not be called") + internal_err!("NoOp::evaluate() should not be called") } fn children(&self) -> Vec> { diff --git a/datafusion/physical-expr/src/sort_expr.rs b/datafusion/physical-expr/src/sort_expr.rs index 74179ba5947c..3b3221289d31 100644 --- a/datafusion/physical-expr/src/sort_expr.rs +++ b/datafusion/physical-expr/src/sort_expr.rs @@ -25,7 +25,7 @@ use crate::PhysicalExpr; use arrow::compute::kernels::sort::{SortColumn, SortOptions}; use arrow::record_batch::RecordBatch; -use datafusion_common::plan_err; +use datafusion_common::exec_err; use datafusion_common::{DataFusionError, Result}; use datafusion_expr::ColumnarValue; @@ -66,7 +66,7 @@ impl PhysicalSortExpr { let array_to_sort = match value_to_sort { ColumnarValue::Array(array) => array, ColumnarValue::Scalar(scalar) => { - return plan_err!( + return exec_err!( "Sort operation is not applicable to scalar value {scalar}" ); } From 7788b90694dc57fb88cc5a6da8cbfd024077e05b Mon Sep 17 00:00:00 2001 From: Andrew Lamb Date: Wed, 1 Nov 2023 12:19:50 -0400 Subject: [PATCH 170/572] Minor: error on unsupported RESPECT NULLs syntax (#7998) * Minor: error on unsupported RESPECT NULLs syntax * fix clippy * Update datafusion/sql/tests/sql_integration.rs Co-authored-by: Liang-Chi Hsieh --------- Co-authored-by: Liang-Chi Hsieh --- datafusion/sql/src/expr/function.rs | 68 ++++++++++++------------- datafusion/sql/tests/sql_integration.rs | 10 ++++ 2 files changed, 43 insertions(+), 35 deletions(-) diff --git a/datafusion/sql/src/expr/function.rs b/datafusion/sql/src/expr/function.rs index 8af5fef84ac1..c58b8319ceb7 100644 --- a/datafusion/sql/src/expr/function.rs +++ b/datafusion/sql/src/expr/function.rs @@ -36,44 +36,57 @@ use super::arrow_cast::ARROW_CAST_NAME; impl<'a, S: ContextProvider> SqlToRel<'a, S> { pub(super) fn sql_function_to_expr( &self, - mut function: SQLFunction, + function: SQLFunction, schema: &DFSchema, planner_context: &mut PlannerContext, ) -> Result { - let name = if function.name.0.len() > 1 { + let SQLFunction { + name, + args, + over, + distinct, + filter, + null_treatment, + special: _, // true if not called with trailing parens + order_by, + } = function; + + if let Some(null_treatment) = null_treatment { + return not_impl_err!("Null treatment in aggregate functions is not supported: {null_treatment}"); + } + + let name = if name.0.len() > 1 { // DF doesn't handle compound identifiers // (e.g. "foo.bar") for function names yet - function.name.to_string() + name.to_string() } else { - crate::utils::normalize_ident(function.name.0[0].clone()) + crate::utils::normalize_ident(name.0[0].clone()) }; // user-defined function (UDF) should have precedence in case it has the same name as a scalar built-in function if let Some(fm) = self.context_provider.get_function_meta(&name) { - let args = - self.function_args_to_expr(function.args, schema, planner_context)?; + let args = self.function_args_to_expr(args, schema, planner_context)?; return Ok(Expr::ScalarUDF(ScalarUDF::new(fm, args))); } // next, scalar built-in if let Ok(fun) = BuiltinScalarFunction::from_str(&name) { - let args = - self.function_args_to_expr(function.args, schema, planner_context)?; + let args = self.function_args_to_expr(args, schema, planner_context)?; return Ok(Expr::ScalarFunction(ScalarFunction::new(fun, args))); }; // If function is a window function (it has an OVER clause), // it shouldn't have ordering requirement as function argument // required ordering should be defined in OVER clause. - let is_function_window = function.over.is_some(); - if !function.order_by.is_empty() && is_function_window { + let is_function_window = over.is_some(); + if !order_by.is_empty() && is_function_window { return plan_err!( "Aggregate ORDER BY is not implemented for window functions" ); } // then, window function - if let Some(WindowType::WindowSpec(window)) = function.over.take() { + if let Some(WindowType::WindowSpec(window)) = over { let partition_by = window .partition_by .into_iter() @@ -97,11 +110,8 @@ impl<'a, S: ContextProvider> SqlToRel<'a, S> { if let Ok(fun) = self.find_window_func(&name) { let expr = match fun { WindowFunction::AggregateFunction(aggregate_fun) => { - let args = self.function_args_to_expr( - function.args, - schema, - planner_context, - )?; + let args = + self.function_args_to_expr(args, schema, planner_context)?; Expr::WindowFunction(expr::WindowFunction::new( WindowFunction::AggregateFunction(aggregate_fun), @@ -113,11 +123,7 @@ impl<'a, S: ContextProvider> SqlToRel<'a, S> { } _ => Expr::WindowFunction(expr::WindowFunction::new( fun, - self.function_args_to_expr( - function.args, - schema, - planner_context, - )?, + self.function_args_to_expr(args, schema, planner_context)?, partition_by, order_by, window_frame, @@ -128,8 +134,7 @@ impl<'a, S: ContextProvider> SqlToRel<'a, S> { } else { // User defined aggregate functions (UDAF) have precedence in case it has the same name as a scalar built-in function if let Some(fm) = self.context_provider.get_aggregate_meta(&name) { - let args = - self.function_args_to_expr(function.args, schema, planner_context)?; + let args = self.function_args_to_expr(args, schema, planner_context)?; return Ok(Expr::AggregateUDF(expr::AggregateUDF::new( fm, args, None, None, ))); @@ -137,17 +142,11 @@ impl<'a, S: ContextProvider> SqlToRel<'a, S> { // next, aggregate built-ins if let Ok(fun) = AggregateFunction::from_str(&name) { - let distinct = function.distinct; - let order_by = self.order_by_to_sort_expr( - &function.order_by, - schema, - planner_context, - )?; + let order_by = + self.order_by_to_sort_expr(&order_by, schema, planner_context)?; let order_by = (!order_by.is_empty()).then_some(order_by); - let args = - self.function_args_to_expr(function.args, schema, planner_context)?; - let filter: Option> = function - .filter + let args = self.function_args_to_expr(args, schema, planner_context)?; + let filter: Option> = filter .map(|e| self.sql_expr_to_logical_expr(*e, schema, planner_context)) .transpose()? .map(Box::new); @@ -159,8 +158,7 @@ impl<'a, S: ContextProvider> SqlToRel<'a, S> { // Special case arrow_cast (as its type is dependent on its argument value) if name == ARROW_CAST_NAME { - let args = - self.function_args_to_expr(function.args, schema, planner_context)?; + let args = self.function_args_to_expr(args, schema, planner_context)?; return super::arrow_cast::create_arrow_cast(args, schema); } } diff --git a/datafusion/sql/tests/sql_integration.rs b/datafusion/sql/tests/sql_integration.rs index 2446ee0a5841..ff6dca7eef2a 100644 --- a/datafusion/sql/tests/sql_integration.rs +++ b/datafusion/sql/tests/sql_integration.rs @@ -1287,6 +1287,16 @@ fn select_simple_aggregate_repeated_aggregate_with_unique_aliases() { ); } +#[test] +fn select_simple_aggregate_respect_nulls() { + let sql = "SELECT MIN(age) RESPECT NULLS FROM person"; + let err = logical_plan(sql).expect_err("query should have failed"); + + assert_contains!( + err.strip_backtrace(), + "This feature is not implemented: Null treatment in aggregate functions is not supported: RESPECT NULLS" + ); +} #[test] fn select_from_typed_string_values() { quick_test( From 94dac76e79380aad9adc781e0c89c2db56d96301 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Marko=20Milenkovi=C4=87?= Date: Wed, 1 Nov 2023 16:20:38 +0000 Subject: [PATCH 171/572] GroupedHashAggregateStream breaks spill batch (#8004) ... into smaller chunks to decrease memory required for merging. --- datafusion/physical-plan/src/aggregates/mod.rs | 2 +- datafusion/physical-plan/src/aggregates/row_hash.rs | 11 ++++++++++- 2 files changed, 11 insertions(+), 2 deletions(-) diff --git a/datafusion/physical-plan/src/aggregates/mod.rs b/datafusion/physical-plan/src/aggregates/mod.rs index 4c612223178c..da152a6264af 100644 --- a/datafusion/physical-plan/src/aggregates/mod.rs +++ b/datafusion/physical-plan/src/aggregates/mod.rs @@ -2155,7 +2155,7 @@ mod tests { spill: bool, ) -> Result<()> { let task_ctx = if spill { - new_spill_ctx(2, 2812) + new_spill_ctx(2, 2886) } else { Arc::new(TaskContext::default()) }; diff --git a/datafusion/physical-plan/src/aggregates/row_hash.rs b/datafusion/physical-plan/src/aggregates/row_hash.rs index d773533ad6a3..7b660885845b 100644 --- a/datafusion/physical-plan/src/aggregates/row_hash.rs +++ b/datafusion/physical-plan/src/aggregates/row_hash.rs @@ -673,7 +673,16 @@ impl GroupedHashAggregateStream { let spillfile = self.runtime.disk_manager.create_tmp_file("HashAggSpill")?; let mut writer = IPCWriter::new(spillfile.path(), &emit.schema())?; // TODO: slice large `sorted` and write to multiple files in parallel - writer.write(&sorted)?; + let mut offset = 0; + let total_rows = sorted.num_rows(); + + while offset < total_rows { + let length = std::cmp::min(total_rows - offset, self.batch_size); + let batch = sorted.slice(offset, length); + offset += batch.num_rows(); + writer.write(&batch)?; + } + writer.finish()?; self.spill_state.spills.push(spillfile); Ok(()) From 35e8e33930f8a60f406edcb3bbba4b3a4c0800a7 Mon Sep 17 00:00:00 2001 From: Raphael Taylor-Davies <1781103+tustvold@users.noreply.github.com> Date: Wed, 1 Nov 2023 16:47:55 +0000 Subject: [PATCH 172/572] Minor: Add implementation examples to ExecutionPlan::execute (#8013) * Add implementation examples to ExecutionPlan::execute * Review feedback --- datafusion/physical-plan/src/lib.rs | 104 ++++++++++++++++++++++++++++ 1 file changed, 104 insertions(+) diff --git a/datafusion/physical-plan/src/lib.rs b/datafusion/physical-plan/src/lib.rs index 3ada2fa163fd..8ae2a8686674 100644 --- a/datafusion/physical-plan/src/lib.rs +++ b/datafusion/physical-plan/src/lib.rs @@ -236,6 +236,110 @@ pub trait ExecutionPlan: Debug + DisplayAs + Send + Sync { } /// Begin execution of `partition`, returning a stream of [`RecordBatch`]es. + /// + /// # Implementation Examples + /// + /// ## Return Precomputed Batch + /// + /// We can return a precomputed batch as a stream + /// + /// ``` + /// # use std::sync::Arc; + /// # use arrow_array::RecordBatch; + /// # use arrow_schema::SchemaRef; + /// # use datafusion_common::Result; + /// # use datafusion_execution::{SendableRecordBatchStream, TaskContext}; + /// # use datafusion_physical_plan::memory::MemoryStream; + /// # use datafusion_physical_plan::stream::RecordBatchStreamAdapter; + /// struct MyPlan { + /// batch: RecordBatch, + /// } + /// + /// impl MyPlan { + /// fn execute( + /// &self, + /// partition: usize, + /// context: Arc + /// ) -> Result { + /// let fut = futures::future::ready(Ok(self.batch.clone())); + /// let stream = futures::stream::once(fut); + /// Ok(Box::pin(RecordBatchStreamAdapter::new(self.batch.schema(), stream))) + /// } + /// } + /// ``` + /// + /// ## Async Compute Batch + /// + /// We can also lazily compute a RecordBatch when the returned stream is polled + /// + /// ``` + /// # use std::sync::Arc; + /// # use arrow_array::RecordBatch; + /// # use arrow_schema::SchemaRef; + /// # use datafusion_common::Result; + /// # use datafusion_execution::{SendableRecordBatchStream, TaskContext}; + /// # use datafusion_physical_plan::memory::MemoryStream; + /// # use datafusion_physical_plan::stream::RecordBatchStreamAdapter; + /// struct MyPlan { + /// schema: SchemaRef, + /// } + /// + /// async fn get_batch() -> Result { + /// todo!() + /// } + /// + /// impl MyPlan { + /// fn execute( + /// &self, + /// partition: usize, + /// context: Arc + /// ) -> Result { + /// let fut = get_batch(); + /// let stream = futures::stream::once(fut); + /// Ok(Box::pin(RecordBatchStreamAdapter::new(self.schema.clone(), stream))) + /// } + /// } + /// ``` + /// + /// ## Async Compute Batch Stream + /// + /// We can lazily compute a RecordBatch stream when the returned stream is polled + /// flattening the result into a single stream + /// + /// ``` + /// # use std::sync::Arc; + /// # use arrow_array::RecordBatch; + /// # use arrow_schema::SchemaRef; + /// # use futures::TryStreamExt; + /// # use datafusion_common::Result; + /// # use datafusion_execution::{SendableRecordBatchStream, TaskContext}; + /// # use datafusion_physical_plan::memory::MemoryStream; + /// # use datafusion_physical_plan::stream::RecordBatchStreamAdapter; + /// struct MyPlan { + /// schema: SchemaRef, + /// } + /// + /// async fn get_batch_stream() -> Result { + /// todo!() + /// } + /// + /// impl MyPlan { + /// fn execute( + /// &self, + /// partition: usize, + /// context: Arc + /// ) -> Result { + /// // A future that yields a stream + /// let fut = get_batch_stream(); + /// // Use TryStreamExt::try_flatten to flatten the stream of streams + /// let stream = futures::stream::once(fut).try_flatten(); + /// Ok(Box::pin(RecordBatchStreamAdapter::new(self.schema.clone(), stream))) + /// } + /// } + /// ``` + /// + /// See [`futures::stream::StreamExt`] and [`futures::stream::TryStreamExt`] for further + /// combinators that can be used with streams fn execute( &self, partition: usize, From 06f2475b304ad173fe7dfe31656928747460588e Mon Sep 17 00:00:00 2001 From: Jay Zhan Date: Thu, 2 Nov 2023 03:04:38 +0800 Subject: [PATCH 173/572] address comment (#7993) Signed-off-by: jayzhan211 --- datafusion/common/src/scalar.rs | 20 ++++---- datafusion/common/src/utils.rs | 46 ++++++++++++++++++- .../physical-expr/src/aggregate/array_agg.rs | 4 +- .../src/aggregate/array_agg_distinct.rs | 4 +- .../physical-expr/src/array_expressions.rs | 4 +- 5 files changed, 60 insertions(+), 18 deletions(-) diff --git a/datafusion/common/src/scalar.rs b/datafusion/common/src/scalar.rs index b3c11740abf9..0d701eaad283 100644 --- a/datafusion/common/src/scalar.rs +++ b/datafusion/common/src/scalar.rs @@ -30,7 +30,7 @@ use crate::cast::{ }; use crate::error::{DataFusionError, Result, _internal_err, _not_impl_err}; use crate::hash_utils::create_hashes; -use crate::utils::wrap_into_list_array; +use crate::utils::array_into_list_array; use arrow::buffer::{NullBuffer, OffsetBuffer}; use arrow::compute::kernels::numeric::*; use arrow::datatypes::{i256, FieldRef, Fields, SchemaBuilder}; @@ -1667,7 +1667,7 @@ impl ScalarValue { } else { Self::iter_to_array(values.iter().cloned()).unwrap() }; - Arc::new(wrap_into_list_array(values)) + Arc::new(array_into_list_array(values)) } /// Converts a scalar value into an array of `size` rows. @@ -2058,7 +2058,7 @@ impl ScalarValue { let list_array = as_list_array(array); let nested_array = list_array.value(index); // Produces a single element `ListArray` with the value at `index`. - let arr = Arc::new(wrap_into_list_array(nested_array)); + let arr = Arc::new(array_into_list_array(nested_array)); ScalarValue::List(arr) } @@ -2067,7 +2067,7 @@ impl ScalarValue { let list_array = as_fixed_size_list_array(array)?; let nested_array = list_array.value(index); // Produces a single element `ListArray` with the value at `index`. - let arr = Arc::new(wrap_into_list_array(nested_array)); + let arr = Arc::new(array_into_list_array(nested_array)); ScalarValue::List(arr) } @@ -3052,7 +3052,7 @@ mod tests { let array = ScalarValue::new_list(scalars.as_slice(), &DataType::Utf8); - let expected = wrap_into_list_array(Arc::new(StringArray::from(vec![ + let expected = array_into_list_array(Arc::new(StringArray::from(vec![ "rust", "arrow", "data-fusion", @@ -3091,9 +3091,9 @@ mod tests { #[test] fn iter_to_array_string_test() { let arr1 = - wrap_into_list_array(Arc::new(StringArray::from(vec!["foo", "bar", "baz"]))); + array_into_list_array(Arc::new(StringArray::from(vec!["foo", "bar", "baz"]))); let arr2 = - wrap_into_list_array(Arc::new(StringArray::from(vec!["rust", "world"]))); + array_into_list_array(Arc::new(StringArray::from(vec!["rust", "world"]))); let scalars = vec![ ScalarValue::List(Arc::new(arr1)), @@ -4335,13 +4335,13 @@ mod tests { // Define list-of-structs scalars let nl0_array = ScalarValue::iter_to_array(vec![s0.clone(), s1.clone()]).unwrap(); - let nl0 = ScalarValue::List(Arc::new(wrap_into_list_array(nl0_array))); + let nl0 = ScalarValue::List(Arc::new(array_into_list_array(nl0_array))); let nl1_array = ScalarValue::iter_to_array(vec![s2.clone()]).unwrap(); - let nl1 = ScalarValue::List(Arc::new(wrap_into_list_array(nl1_array))); + let nl1 = ScalarValue::List(Arc::new(array_into_list_array(nl1_array))); let nl2_array = ScalarValue::iter_to_array(vec![s1.clone()]).unwrap(); - let nl2 = ScalarValue::List(Arc::new(wrap_into_list_array(nl2_array))); + let nl2 = ScalarValue::List(Arc::new(array_into_list_array(nl2_array))); // iter_to_array for list-of-struct let array = ScalarValue::iter_to_array(vec![nl0, nl1, nl2]).unwrap(); diff --git a/datafusion/common/src/utils.rs b/datafusion/common/src/utils.rs index b2f71e86f21e..f031f7880436 100644 --- a/datafusion/common/src/utils.rs +++ b/datafusion/common/src/utils.rs @@ -17,6 +17,7 @@ //! This module provides the bisect function, which implements binary search. +use crate::error::_internal_err; use crate::{DataFusionError, Result, ScalarValue}; use arrow::array::{ArrayRef, PrimitiveArray}; use arrow::buffer::OffsetBuffer; @@ -24,7 +25,7 @@ use arrow::compute; use arrow::compute::{partition, SortColumn, SortOptions}; use arrow::datatypes::{Field, SchemaRef, UInt32Type}; use arrow::record_batch::RecordBatch; -use arrow_array::ListArray; +use arrow_array::{Array, ListArray}; use sqlparser::ast::Ident; use sqlparser::dialect::GenericDialect; use sqlparser::parser::Parser; @@ -338,7 +339,7 @@ pub fn longest_consecutive_prefix>( /// Wrap an array into a single element `ListArray`. /// For example `[1, 2, 3]` would be converted into `[[1, 2, 3]]` -pub fn wrap_into_list_array(arr: ArrayRef) -> ListArray { +pub fn array_into_list_array(arr: ArrayRef) -> ListArray { let offsets = OffsetBuffer::from_lengths([arr.len()]); ListArray::new( Arc::new(Field::new("item", arr.data_type().to_owned(), true)), @@ -348,6 +349,47 @@ pub fn wrap_into_list_array(arr: ArrayRef) -> ListArray { ) } +/// Wrap arrays into a single element `ListArray`. +/// +/// Example: +/// ``` +/// use arrow::array::{Int32Array, ListArray, ArrayRef}; +/// use arrow::datatypes::{Int32Type, Field}; +/// use std::sync::Arc; +/// +/// let arr1 = Arc::new(Int32Array::from(vec![1, 2, 3])) as ArrayRef; +/// let arr2 = Arc::new(Int32Array::from(vec![4, 5, 6])) as ArrayRef; +/// +/// let list_arr = datafusion_common::utils::arrays_into_list_array([arr1, arr2]).unwrap(); +/// +/// let expected = ListArray::from_iter_primitive::( +/// vec![ +/// Some(vec![Some(1), Some(2), Some(3)]), +/// Some(vec![Some(4), Some(5), Some(6)]), +/// ] +/// ); +/// +/// assert_eq!(list_arr, expected); +pub fn arrays_into_list_array( + arr: impl IntoIterator, +) -> Result { + let arr = arr.into_iter().collect::>(); + if arr.is_empty() { + return _internal_err!("Cannot wrap empty array into list array"); + } + + let lens = arr.iter().map(|x| x.len()).collect::>(); + // Assume data type is consistent + let data_type = arr[0].data_type().to_owned(); + let values = arr.iter().map(|x| x.as_ref()).collect::>(); + Ok(ListArray::new( + Arc::new(Field::new("item", data_type, true)), + OffsetBuffer::from_lengths(lens), + arrow::compute::concat(values.as_slice())?, + None, + )) +} + /// An extension trait for smart pointers. Provides an interface to get a /// raw pointer to the data (with metadata stripped away). /// diff --git a/datafusion/physical-expr/src/aggregate/array_agg.rs b/datafusion/physical-expr/src/aggregate/array_agg.rs index 834925b8d554..4dccbfef07f8 100644 --- a/datafusion/physical-expr/src/aggregate/array_agg.rs +++ b/datafusion/physical-expr/src/aggregate/array_agg.rs @@ -24,7 +24,7 @@ use arrow::array::ArrayRef; use arrow::datatypes::{DataType, Field}; use arrow_array::Array; use datafusion_common::cast::as_list_array; -use datafusion_common::utils::wrap_into_list_array; +use datafusion_common::utils::array_into_list_array; use datafusion_common::Result; use datafusion_common::ScalarValue; use datafusion_expr::Accumulator; @@ -161,7 +161,7 @@ impl Accumulator for ArrayAggAccumulator { } let concated_array = arrow::compute::concat(&element_arrays)?; - let list_array = wrap_into_list_array(concated_array); + let list_array = array_into_list_array(concated_array); Ok(ScalarValue::List(Arc::new(list_array))) } diff --git a/datafusion/physical-expr/src/aggregate/array_agg_distinct.rs b/datafusion/physical-expr/src/aggregate/array_agg_distinct.rs index 21143ce54a20..9b391b0c42cf 100644 --- a/datafusion/physical-expr/src/aggregate/array_agg_distinct.rs +++ b/datafusion/physical-expr/src/aggregate/array_agg_distinct.rs @@ -185,7 +185,7 @@ mod tests { use arrow_array::types::Int32Type; use arrow_array::{Array, ListArray}; use arrow_buffer::OffsetBuffer; - use datafusion_common::utils::wrap_into_list_array; + use datafusion_common::utils::array_into_list_array; use datafusion_common::{internal_err, DataFusionError}; // arrow::compute::sort cann't sort ListArray directly, so we need to sort the inner primitive array and wrap it back into ListArray. @@ -201,7 +201,7 @@ mod tests { }; let arr = arrow::compute::sort(&arr, None).unwrap(); - let list_arr = wrap_into_list_array(arr); + let list_arr = array_into_list_array(arr); ScalarValue::List(Arc::new(list_arr)) } diff --git a/datafusion/physical-expr/src/array_expressions.rs b/datafusion/physical-expr/src/array_expressions.rs index 84fd301b84de..18d8c60fe7bd 100644 --- a/datafusion/physical-expr/src/array_expressions.rs +++ b/datafusion/physical-expr/src/array_expressions.rs @@ -29,7 +29,7 @@ use arrow_buffer::NullBuffer; use datafusion_common::cast::{ as_generic_string_array, as_int64_array, as_list_array, as_string_array, }; -use datafusion_common::utils::wrap_into_list_array; +use datafusion_common::utils::array_into_list_array; use datafusion_common::{ exec_err, internal_err, not_impl_err, plan_err, DataFusionError, Result, }; @@ -412,7 +412,7 @@ pub fn make_array(arrays: &[ArrayRef]) -> Result { // Either an empty array or all nulls: DataType::Null => { let array = new_null_array(&DataType::Null, arrays.len()); - Ok(Arc::new(wrap_into_list_array(array))) + Ok(Arc::new(array_into_list_array(array))) } data_type => array_array(arrays, data_type), } From 5634cce35f6ceb13cf7367b91de9e50d751e442a Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Marko=20Milenkovi=C4=87?= Date: Wed, 1 Nov 2023 20:33:28 +0000 Subject: [PATCH 174/572] GroupedHashAggregateStream should register spillable consumer (#8002) --- datafusion/physical-plan/src/aggregates/row_hash.rs | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/datafusion/physical-plan/src/aggregates/row_hash.rs b/datafusion/physical-plan/src/aggregates/row_hash.rs index 7b660885845b..f1eece4315e4 100644 --- a/datafusion/physical-plan/src/aggregates/row_hash.rs +++ b/datafusion/physical-plan/src/aggregates/row_hash.rs @@ -328,7 +328,9 @@ impl GroupedHashAggregateStream { .collect(); let name = format!("GroupedHashAggregateStream[{partition}]"); - let reservation = MemoryConsumer::new(name).register(context.memory_pool()); + let reservation = MemoryConsumer::new(name) + .with_can_spill(true) + .register(context.memory_pool()); let group_ordering = agg .aggregation_ordering From 7f3f4657185c0a7c398ced8d1cfdbaf34c19eeb5 Mon Sep 17 00:00:00 2001 From: Huaijin Date: Thu, 2 Nov 2023 19:56:03 +0800 Subject: [PATCH 175/572] fix: single_distinct_aggretation_to_group_by fail (#7997) * fix: single_distinct_aggretation_to_group_by faile * fix * move test to groupby.slt --- .../src/single_distinct_to_groupby.rs | 2 +- .../sqllogictest/test_files/groupby.slt | 58 +++++++++++++++++++ 2 files changed, 59 insertions(+), 1 deletion(-) diff --git a/datafusion/optimizer/src/single_distinct_to_groupby.rs b/datafusion/optimizer/src/single_distinct_to_groupby.rs index 8e0f93cb5781..be76c069f0b7 100644 --- a/datafusion/optimizer/src/single_distinct_to_groupby.rs +++ b/datafusion/optimizer/src/single_distinct_to_groupby.rs @@ -74,7 +74,7 @@ fn is_single_distinct_agg(plan: &LogicalPlan) -> Result { distinct_count += 1; } for e in args { - fields_set.insert(e.display_name()?); + fields_set.insert(e.canonical_name()); } } } diff --git a/datafusion/sqllogictest/test_files/groupby.slt b/datafusion/sqllogictest/test_files/groupby.slt index 5cb3ac2f8135..ef6fee69abbf 100644 --- a/datafusion/sqllogictest/test_files/groupby.slt +++ b/datafusion/sqllogictest/test_files/groupby.slt @@ -3733,3 +3733,61 @@ AggregateExec: mode=FinalPartitioned, gby=[c@0 as c, b@1 as b], aggr=[SUM(multip --------AggregateExec: mode=Partial, gby=[c@1 as c, b@0 as b], aggr=[SUM(multiple_ordered_table_with_pk.d)], ordering_mode=PartiallyOrdered ----------RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1 ------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[b, c, d], output_ordering=[c@1 ASC NULLS LAST], has_header=true + +# Tests for single distinct to group by optimization rule +statement ok +CREATE TABLE t(x int) AS VALUES (1), (2), (1); + +statement ok +create table t1(x bigint,y int) as values (9223372036854775807,2), (9223372036854775806,2); + +query II +SELECT SUM(DISTINCT x), MAX(DISTINCT x) from t GROUP BY x ORDER BY x; +---- +1 1 +2 2 + +query II +SELECT MAX(DISTINCT x), SUM(DISTINCT x) from t GROUP BY x ORDER BY x; +---- +1 1 +2 2 + +query TT +EXPLAIN SELECT SUM(DISTINCT CAST(x AS DOUBLE)), MAX(DISTINCT x) FROM t1 GROUP BY y; +---- +logical_plan +Projection: SUM(DISTINCT t1.x), MAX(DISTINCT t1.x) +--Aggregate: groupBy=[[t1.y]], aggr=[[SUM(DISTINCT CAST(t1.x AS Float64)), MAX(DISTINCT t1.x)]] +----TableScan: t1 projection=[x, y] +physical_plan +ProjectionExec: expr=[SUM(DISTINCT t1.x)@1 as SUM(DISTINCT t1.x), MAX(DISTINCT t1.x)@2 as MAX(DISTINCT t1.x)] +--AggregateExec: mode=FinalPartitioned, gby=[y@0 as y], aggr=[SUM(DISTINCT t1.x), MAX(DISTINCT t1.x)] +----CoalesceBatchesExec: target_batch_size=2 +------RepartitionExec: partitioning=Hash([y@0], 8), input_partitions=8 +--------RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1 +----------AggregateExec: mode=Partial, gby=[y@1 as y], aggr=[SUM(DISTINCT t1.x), MAX(DISTINCT t1.x)] +------------MemoryExec: partitions=1, partition_sizes=[1] + +query TT +EXPLAIN SELECT SUM(DISTINCT CAST(x AS DOUBLE)), MAX(DISTINCT CAST(x AS DOUBLE)) FROM t1 GROUP BY y; +---- +logical_plan +Projection: SUM(alias1) AS SUM(DISTINCT t1.x), MAX(alias1) AS MAX(DISTINCT t1.x) +--Aggregate: groupBy=[[t1.y]], aggr=[[SUM(alias1), MAX(alias1)]] +----Aggregate: groupBy=[[t1.y, CAST(t1.x AS Float64)t1.x AS t1.x AS alias1]], aggr=[[]] +------Projection: CAST(t1.x AS Float64) AS CAST(t1.x AS Float64)t1.x, t1.y +--------TableScan: t1 projection=[x, y] +physical_plan +ProjectionExec: expr=[SUM(alias1)@1 as SUM(DISTINCT t1.x), MAX(alias1)@2 as MAX(DISTINCT t1.x)] +--AggregateExec: mode=FinalPartitioned, gby=[y@0 as y], aggr=[SUM(alias1), MAX(alias1)] +----CoalesceBatchesExec: target_batch_size=2 +------RepartitionExec: partitioning=Hash([y@0], 8), input_partitions=8 +--------AggregateExec: mode=Partial, gby=[y@0 as y], aggr=[SUM(alias1), MAX(alias1)] +----------AggregateExec: mode=FinalPartitioned, gby=[y@0 as y, alias1@1 as alias1], aggr=[] +------------CoalesceBatchesExec: target_batch_size=2 +--------------RepartitionExec: partitioning=Hash([y@0, alias1@1], 8), input_partitions=8 +----------------RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1 +------------------AggregateExec: mode=Partial, gby=[y@1 as y, CAST(t1.x AS Float64)t1.x@0 as alias1], aggr=[] +--------------------ProjectionExec: expr=[CAST(x@0 AS Float64) as CAST(t1.x AS Float64)t1.x, y@1 as y] +----------------------MemoryExec: partitions=1, partition_sizes=[1] From 436a4fa348f1fcdf47bb2624deb24f8b5266df8c Mon Sep 17 00:00:00 2001 From: Jeffrey <22608443+Jefffrey@users.noreply.github.com> Date: Fri, 3 Nov 2023 00:48:36 +1100 Subject: [PATCH 176/572] Read only enough bytes to infer Arrow IPC file schema via stream (#7962) * Read only enough bytes to infer Arrow IPC file schema via stream * Error checking for collect bytes func * Update datafusion/core/src/datasource/file_format/arrow.rs Co-authored-by: Andrew Lamb --------- Co-authored-by: Andrew Lamb --- .../core/src/datasource/file_format/arrow.rs | 197 +++++++++++++++++- 1 file changed, 186 insertions(+), 11 deletions(-) diff --git a/datafusion/core/src/datasource/file_format/arrow.rs b/datafusion/core/src/datasource/file_format/arrow.rs index 16ae4411d1bf..2777805078c7 100644 --- a/datafusion/core/src/datasource/file_format/arrow.rs +++ b/datafusion/core/src/datasource/file_format/arrow.rs @@ -20,7 +20,7 @@ //! Works with files following the [Arrow IPC format](https://arrow.apache.org/docs/format/Columnar.html#ipc-file-format) use std::any::Any; -use std::io::{Read, Seek}; +use std::borrow::Cow; use std::sync::Arc; use crate::datasource::file_format::FileFormat; @@ -29,13 +29,18 @@ use crate::error::Result; use crate::execution::context::SessionState; use crate::physical_plan::ExecutionPlan; +use arrow::ipc::convert::fb_to_schema; use arrow::ipc::reader::FileReader; -use arrow_schema::{Schema, SchemaRef}; +use arrow::ipc::root_as_message; +use arrow_schema::{ArrowError, Schema, SchemaRef}; +use bytes::Bytes; use datafusion_common::{FileType, Statistics}; use datafusion_physical_expr::PhysicalExpr; use async_trait::async_trait; +use futures::stream::BoxStream; +use futures::StreamExt; use object_store::{GetResultPayload, ObjectMeta, ObjectStore}; /// Arrow `FileFormat` implementation. @@ -59,13 +64,11 @@ impl FileFormat for ArrowFormat { let r = store.as_ref().get(&object.location).await?; let schema = match r.payload { GetResultPayload::File(mut file, _) => { - read_arrow_schema_from_reader(&mut file)? + let reader = FileReader::try_new(&mut file, None)?; + reader.schema() } - GetResultPayload::Stream(_) => { - // TODO: Fetching entire file to get schema is potentially wasteful - let data = r.bytes().await?; - let mut cursor = std::io::Cursor::new(&data); - read_arrow_schema_from_reader(&mut cursor)? + GetResultPayload::Stream(stream) => { + infer_schema_from_file_stream(stream).await? } }; schemas.push(schema.as_ref().clone()); @@ -99,7 +102,179 @@ impl FileFormat for ArrowFormat { } } -fn read_arrow_schema_from_reader(reader: R) -> Result { - let reader = FileReader::try_new(reader, None)?; - Ok(reader.schema()) +const ARROW_MAGIC: [u8; 6] = [b'A', b'R', b'R', b'O', b'W', b'1']; +const CONTINUATION_MARKER: [u8; 4] = [0xff; 4]; + +/// Custom implementation of inferring schema. Should eventually be moved upstream to arrow-rs. +/// See https://github.com/apache/arrow-rs/issues/5021 +async fn infer_schema_from_file_stream( + mut stream: BoxStream<'static, object_store::Result>, +) -> Result { + // Expected format: + // - 6 bytes + // - 2 bytes + // - 4 bytes, not present below v0.15.0 + // - 4 bytes + // + // + + // So in first read we need at least all known sized sections, + // which is 6 + 2 + 4 + 4 = 16 bytes. + let bytes = collect_at_least_n_bytes(&mut stream, 16, None).await?; + + // Files should start with these magic bytes + if bytes[0..6] != ARROW_MAGIC { + return Err(ArrowError::ParseError( + "Arrow file does not contian correct header".to_string(), + ))?; + } + + // Since continuation marker bytes added in later versions + let (meta_len, rest_of_bytes_start_index) = if bytes[8..12] == CONTINUATION_MARKER { + (&bytes[12..16], 16) + } else { + (&bytes[8..12], 12) + }; + + let meta_len = [meta_len[0], meta_len[1], meta_len[2], meta_len[3]]; + let meta_len = i32::from_le_bytes(meta_len); + + // Read bytes for Schema message + let block_data = if bytes[rest_of_bytes_start_index..].len() < meta_len as usize { + // Need to read more bytes to decode Message + let mut block_data = Vec::with_capacity(meta_len as usize); + // In case we had some spare bytes in our initial read chunk + block_data.extend_from_slice(&bytes[rest_of_bytes_start_index..]); + let size_to_read = meta_len as usize - block_data.len(); + let block_data = + collect_at_least_n_bytes(&mut stream, size_to_read, Some(block_data)).await?; + Cow::Owned(block_data) + } else { + // Already have the bytes we need + let end_index = meta_len as usize + rest_of_bytes_start_index; + let block_data = &bytes[rest_of_bytes_start_index..end_index]; + Cow::Borrowed(block_data) + }; + + // Decode Schema message + let message = root_as_message(&block_data).map_err(|err| { + ArrowError::ParseError(format!("Unable to read IPC message as metadata: {err:?}")) + })?; + let ipc_schema = message.header_as_schema().ok_or_else(|| { + ArrowError::IpcError("Unable to read IPC message as schema".to_string()) + })?; + let schema = fb_to_schema(ipc_schema); + + Ok(Arc::new(schema)) +} + +async fn collect_at_least_n_bytes( + stream: &mut BoxStream<'static, object_store::Result>, + n: usize, + extend_from: Option>, +) -> Result> { + let mut buf = extend_from.unwrap_or_else(|| Vec::with_capacity(n)); + // If extending existing buffer then ensure we read n additional bytes + let n = n + buf.len(); + while let Some(bytes) = stream.next().await.transpose()? { + buf.extend_from_slice(&bytes); + if buf.len() >= n { + break; + } + } + if buf.len() < n { + return Err(ArrowError::ParseError( + "Unexpected end of byte stream for Arrow IPC file".to_string(), + ))?; + } + Ok(buf) +} + +#[cfg(test)] +mod tests { + use chrono::DateTime; + use object_store::{chunked::ChunkedStore, memory::InMemory, path::Path}; + + use crate::execution::context::SessionContext; + + use super::*; + + #[tokio::test] + async fn test_infer_schema_stream() -> Result<()> { + let mut bytes = std::fs::read("tests/data/example.arrow")?; + bytes.truncate(bytes.len() - 20); // mangle end to show we don't need to read whole file + let location = Path::parse("example.arrow")?; + let in_memory_store: Arc = Arc::new(InMemory::new()); + in_memory_store.put(&location, bytes.into()).await?; + + let session_ctx = SessionContext::new(); + let state = session_ctx.state(); + let object_meta = ObjectMeta { + location, + last_modified: DateTime::default(), + size: usize::MAX, + e_tag: None, + }; + + let arrow_format = ArrowFormat {}; + let expected = vec!["f0: Int64", "f1: Utf8", "f2: Boolean"]; + + // Test chunk sizes where too small so we keep having to read more bytes + // And when large enough that first read contains all we need + for chunk_size in [7, 3000] { + let store = Arc::new(ChunkedStore::new(in_memory_store.clone(), chunk_size)); + let inferred_schema = arrow_format + .infer_schema( + &state, + &(store.clone() as Arc), + &[object_meta.clone()], + ) + .await?; + let actual_fields = inferred_schema + .fields() + .iter() + .map(|f| format!("{}: {:?}", f.name(), f.data_type())) + .collect::>(); + assert_eq!(expected, actual_fields); + } + + Ok(()) + } + + #[tokio::test] + async fn test_infer_schema_short_stream() -> Result<()> { + let mut bytes = std::fs::read("tests/data/example.arrow")?; + bytes.truncate(20); // should cause error that file shorter than expected + let location = Path::parse("example.arrow")?; + let in_memory_store: Arc = Arc::new(InMemory::new()); + in_memory_store.put(&location, bytes.into()).await?; + + let session_ctx = SessionContext::new(); + let state = session_ctx.state(); + let object_meta = ObjectMeta { + location, + last_modified: DateTime::default(), + size: usize::MAX, + e_tag: None, + }; + + let arrow_format = ArrowFormat {}; + + let store = Arc::new(ChunkedStore::new(in_memory_store.clone(), 7)); + let err = arrow_format + .infer_schema( + &state, + &(store.clone() as Arc), + &[object_meta.clone()], + ) + .await; + + assert!(err.is_err()); + assert_eq!( + "Arrow error: Parser error: Unexpected end of byte stream for Arrow IPC file", + err.unwrap_err().to_string() + ); + + Ok(()) + } } From d2671cd17ed854251adc54ca1ba87776593be61b Mon Sep 17 00:00:00 2001 From: Huaijin Date: Fri, 3 Nov 2023 02:53:52 +0800 Subject: [PATCH 177/572] Minor: remove a strange char (#8030) --- datafusion/execution/src/memory_pool/mod.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/datafusion/execution/src/memory_pool/mod.rs b/datafusion/execution/src/memory_pool/mod.rs index 71483e200af6..55555014f2ef 100644 --- a/datafusion/execution/src/memory_pool/mod.rs +++ b/datafusion/execution/src/memory_pool/mod.rs @@ -235,7 +235,7 @@ impl MemoryReservation { } } - /// Returns a new empty [`MemoryReservation`] with the same [`MemoryConsumer`] + /// Returns a new empty [`MemoryReservation`] with the same [`MemoryConsumer`] pub fn new_empty(&self) -> Self { Self { size: 0, From b0891372a75d1e19fdadb37312686b53a8d03090 Mon Sep 17 00:00:00 2001 From: Andrew Lamb Date: Thu, 2 Nov 2023 14:54:41 -0400 Subject: [PATCH 178/572] Minor: Improve documentation for Filter Pushdown (#8023) * Minor: Improve documentation for Fulter Pushdown * Update datafusion/optimizer/src/push_down_filter.rs Co-authored-by: jakevin * Apply suggestions from code review * Update datafusion/optimizer/src/push_down_filter.rs Co-authored-by: Alex Huang --------- Co-authored-by: jakevin Co-authored-by: Alex Huang --- datafusion/optimizer/src/push_down_filter.rs | 101 +++++++++++++++---- 1 file changed, 82 insertions(+), 19 deletions(-) diff --git a/datafusion/optimizer/src/push_down_filter.rs b/datafusion/optimizer/src/push_down_filter.rs index 8c2eb96a48d8..ae986b3c84dd 100644 --- a/datafusion/optimizer/src/push_down_filter.rs +++ b/datafusion/optimizer/src/push_down_filter.rs @@ -12,7 +12,8 @@ // specific language governing permissions and limitations // under the License. -//! Push Down Filter optimizer rule ensures that filters are applied as early as possible in the plan +//! [`PushDownFilter`] Moves filters so they are applied as early as possible in +//! the plan. use crate::optimizer::ApplyOrder; use crate::utils::{conjunction, split_conjunction, split_conjunction_owned}; @@ -33,31 +34,93 @@ use itertools::Itertools; use std::collections::{HashMap, HashSet}; use std::sync::Arc; -/// Push Down Filter optimizer rule pushes filter clauses down the plan +/// Optimizer rule for pushing (moving) filter expressions down in a plan so +/// they are applied as early as possible. +/// /// # Introduction -/// A filter-commutative operation is an operation whose result of filter(op(data)) = op(filter(data)). -/// An example of a filter-commutative operation is a projection; a counter-example is `limit`. /// -/// The filter-commutative property is column-specific. An aggregate grouped by A on SUM(B) -/// can commute with a filter that depends on A only, but does not commute with a filter that depends -/// on SUM(B). +/// The goal of this rule is to improve query performance by eliminating +/// redundant work. +/// +/// For example, given a plan that sorts all values where `a > 10`: +/// +/// ```text +/// Filter (a > 10) +/// Sort (a, b) +/// ``` +/// +/// A better plan is to filter the data *before* the Sort, which sorts fewer +/// rows and therefore does less work overall: +/// +/// ```text +/// Sort (a, b) +/// Filter (a > 10) <-- Filter is moved before the sort +/// ``` +/// +/// However it is not always possible to push filters down. For example, given a +/// plan that finds the top 3 values and then keeps only those that are greater +/// than 10, if the filter is pushed below the limit it would produce a +/// different result. +/// +/// ```text +/// Filter (a > 10) <-- can not move this Filter before the limit +/// Limit (fetch=3) +/// Sort (a, b) +/// ``` +/// +/// +/// More formally, a filter-commutative operation is an operation `op` that +/// satisfies `filter(op(data)) = op(filter(data))`. +/// +/// The filter-commutative property is plan and column-specific. A filter on `a` +/// can be pushed through a `Aggregate(group_by = [a], agg=[SUM(b))`. However, a +/// filter on `SUM(b)` can not be pushed through the same aggregate. +/// +/// # Handling Conjunctions +/// +/// It is possible to only push down **part** of a filter expression if is +/// connected with `AND`s (more formally if it is a "conjunction"). +/// +/// For example, given the following plan: +/// +/// ```text +/// Filter(a > 10 AND SUM(b) < 5) +/// Aggregate(group_by = [a], agg = [SUM(b)) +/// ``` +/// +/// The `a > 10` is commutative with the `Aggregate` but `SUM(b) < 5` is not. +/// Therefore it is possible to only push part of the expression, resulting in: +/// +/// ```text +/// Filter(SUM(b) < 5) +/// Aggregate(group_by = [a], agg = [SUM(b)) +/// Filter(a > 10) +/// ``` +/// +/// # Handling Column Aliases /// -/// This optimizer commutes filters with filter-commutative operations to push the filters -/// the closest possible to the scans, re-writing the filter expressions by every -/// projection that changes the filter's expression. +/// This optimizer must sometimes handle re-writing filter expressions when they +/// pushed, for example if there is a projection that aliases `a+1` to `"b"`: /// -/// Filter: b Gt Int64(10) -/// Projection: a AS b +/// ```text +/// Filter (b > 10) +/// Projection: [a+1 AS "b"] <-- changes the name of `a+1` to `b` +/// ``` /// -/// is optimized to +/// To apply the filter prior to the `Projection`, all references to `b` must be +/// rewritten to `a+1`: /// -/// Projection: a AS b -/// Filter: a Gt Int64(10) <--- changed from b to a +/// ```text +/// Projection: a AS "b" +/// Filter: (a + 1 > 10) <--- changed from b to a + 1 +/// ``` +/// # Implementation Notes /// -/// This performs a single pass through the plan. When it passes through a filter, it stores that filter, -/// and when it reaches a node that does not commute with it, it adds the filter to that place. -/// When it passes through a projection, it re-writes the filter's expression taking into account that projection. -/// When multiple filters would have been written, it `AND` their expressions into a single expression. +/// This implementation performs a single pass through the plan, "pushing" down +/// filters. When it passes through a filter, it stores that filter, and when it +/// reaches a plan node that does not commute with that filter, it adds the +/// filter to that place. When it passes through a projection, it re-writes the +/// filter's expression taking into account that projection. #[derive(Default)] pub struct PushDownFilter {} From 8682be57f5208a9d1097729802e18f35c09392cd Mon Sep 17 00:00:00 2001 From: Andrew Lamb Date: Thu, 2 Nov 2023 14:55:02 -0400 Subject: [PATCH 179/572] Minor: Improve `ExecutionPlan` documentation (#8019) * Minor: Improve `ExecutionPlan` documentation * Add link to Partitioning --- datafusion/physical-plan/src/lib.rs | 113 +++++++++++++++++++--------- 1 file changed, 79 insertions(+), 34 deletions(-) diff --git a/datafusion/physical-plan/src/lib.rs b/datafusion/physical-plan/src/lib.rs index 8ae2a8686674..ed795a1cb333 100644 --- a/datafusion/physical-plan/src/lib.rs +++ b/datafusion/physical-plan/src/lib.rs @@ -91,16 +91,27 @@ pub use datafusion_physical_expr::{ pub use crate::stream::EmptyRecordBatchStream; pub use datafusion_execution::{RecordBatchStream, SendableRecordBatchStream}; -/// `ExecutionPlan` represent nodes in the DataFusion Physical Plan. +/// Represent nodes in the DataFusion Physical Plan. /// -/// Each `ExecutionPlan` is partition-aware and is responsible for -/// creating the actual `async` [`SendableRecordBatchStream`]s -/// of [`RecordBatch`] that incrementally compute the operator's -/// output from its input partition. +/// Calling [`execute`] produces an `async` [`SendableRecordBatchStream`] of +/// [`RecordBatch`] that incrementally computes a partition of the +/// `ExecutionPlan`'s output from its input. See [`Partitioning`] for more +/// details on partitioning. +/// +/// Methods such as [`schema`] and [`output_partitioning`] communicate +/// properties of this output to the DataFusion optimizer, and methods such as +/// [`required_input_distribution`] and [`required_input_ordering`] express +/// requirements of the `ExecutionPlan` from its input. /// /// [`ExecutionPlan`] can be displayed in a simplified form using the /// return value from [`displayable`] in addition to the (normally /// quite verbose) `Debug` output. +/// +/// [`execute`]: ExecutionPlan::execute +/// [`schema`]: ExecutionPlan::schema +/// [`output_partitioning`]: ExecutionPlan::output_partitioning +/// [`required_input_distribution`]: ExecutionPlan::required_input_distribution +/// [`required_input_ordering`]: ExecutionPlan::required_input_ordering pub trait ExecutionPlan: Debug + DisplayAs + Send + Sync { /// Returns the execution plan as [`Any`] so that it can be /// downcast to a specific implementation. @@ -109,7 +120,8 @@ pub trait ExecutionPlan: Debug + DisplayAs + Send + Sync { /// Get the schema for this execution plan fn schema(&self) -> SchemaRef; - /// Specifies the output partitioning scheme of this plan + /// Specifies how the output of this `ExecutionPlan` is split into + /// partitions. fn output_partitioning(&self) -> Partitioning; /// Specifies whether this plan generates an infinite stream of records. @@ -123,7 +135,7 @@ pub trait ExecutionPlan: Debug + DisplayAs + Send + Sync { } } - /// If the output of this operator within each partition is sorted, + /// If the output of this `ExecutionPlan` within each partition is sorted, /// returns `Some(keys)` with the description of how it was sorted. /// /// For example, Sort, (obviously) produces sorted output as does @@ -131,17 +143,19 @@ pub trait ExecutionPlan: Debug + DisplayAs + Send + Sync { /// produces sorted output if its input was sorted as it does not /// reorder the input rows, /// - /// It is safe to return `None` here if your operator does not + /// It is safe to return `None` here if your `ExecutionPlan` does not /// have any particular output order here fn output_ordering(&self) -> Option<&[PhysicalSortExpr]>; /// Specifies the data distribution requirements for all the - /// children for this operator, By default it's [[Distribution::UnspecifiedDistribution]] for each child, + /// children for this `ExecutionPlan`, By default it's [[Distribution::UnspecifiedDistribution]] for each child, fn required_input_distribution(&self) -> Vec { vec![Distribution::UnspecifiedDistribution; self.children().len()] } - /// Specifies the ordering requirements for all of the children + /// Specifies the ordering required for all of the children of this + /// `ExecutionPlan`. + /// /// For each child, it's the local ordering requirement within /// each partition rather than the global ordering /// @@ -152,7 +166,7 @@ pub trait ExecutionPlan: Debug + DisplayAs + Send + Sync { vec![None; self.children().len()] } - /// Returns `false` if this operator's implementation may reorder + /// Returns `false` if this `ExecutionPlan`'s implementation may reorder /// rows within or between partitions. /// /// For example, Projection, Filter, and Limit maintain the order @@ -166,19 +180,21 @@ pub trait ExecutionPlan: Debug + DisplayAs + Send + Sync { /// The default implementation returns `false` /// /// WARNING: if you override this default, you *MUST* ensure that - /// the operator's maintains the ordering invariant or else + /// the `ExecutionPlan`'s maintains the ordering invariant or else /// DataFusion may produce incorrect results. fn maintains_input_order(&self) -> Vec { vec![false; self.children().len()] } - /// Specifies whether the operator benefits from increased parallelization - /// at its input for each child. If set to `true`, this indicates that the - /// operator would benefit from partitioning its corresponding child - /// (and thus from more parallelism). For operators that do very little work - /// the overhead of extra parallelism may outweigh any benefits + /// Specifies whether the `ExecutionPlan` benefits from increased + /// parallelization at its input for each child. /// - /// The default implementation returns `true` unless this operator + /// If returns `true`, the `ExecutionPlan` would benefit from partitioning + /// its corresponding child (and thus from more parallelism). For + /// `ExecutionPlan` that do very little work the overhead of extra + /// parallelism may outweigh any benefits + /// + /// The default implementation returns `true` unless this `ExecutionPlan` /// has signalled it requires a single child input partition. fn benefits_from_input_partitioning(&self) -> Vec { // By default try to maximize parallelism with more CPUs if @@ -199,12 +215,14 @@ pub trait ExecutionPlan: Debug + DisplayAs + Send + Sync { OrderingEquivalenceProperties::new(self.schema()) } - /// Get a list of child execution plans that provide the input for this plan. The returned list - /// will be empty for leaf nodes, will contain a single value for unary nodes, or two - /// values for binary nodes (such as joins). + /// Get a list of `ExecutionPlan` that provide input for this plan. The + /// returned list will be empty for leaf nodes such as scans, will contain a + /// single value for unary nodes, or two values for binary nodes (such as + /// joins). fn children(&self) -> Vec>; - /// Returns a new plan where all children were replaced by new plans. + /// Returns a new `ExecutionPlan` where all existing children were replaced + /// by the `children`, oi order fn with_new_children( self: Arc, children: Vec>, @@ -235,13 +253,40 @@ pub trait ExecutionPlan: Debug + DisplayAs + Send + Sync { Ok(None) } - /// Begin execution of `partition`, returning a stream of [`RecordBatch`]es. + /// Begin execution of `partition`, returning a [`Stream`] of + /// [`RecordBatch`]es. + /// + /// # Notes + /// + /// The `execute` method itself is not `async` but it returns an `async` + /// [`futures::stream::Stream`]. This `Stream` should incrementally compute + /// the output, `RecordBatch` by `RecordBatch` (in a streaming fashion). + /// Most `ExecutionPlan`s should not do any work before the first + /// `RecordBatch` is requested from the stream. + /// + /// [`RecordBatchStreamAdapter`] can be used to convert an `async` + /// [`Stream`] into a [`SendableRecordBatchStream`]. + /// + /// Using `async` `Streams` allows for network I/O during execution and + /// takes advantage of Rust's built in support for `async` continuations and + /// crate ecosystem. + /// + /// [`Stream`]: futures::stream::Stream + /// [`StreamExt`]: futures::stream::StreamExt + /// [`TryStreamExt`]: futures::stream::TryStreamExt + /// [`RecordBatchStreamAdapter`]: crate::stream::RecordBatchStreamAdapter /// /// # Implementation Examples /// - /// ## Return Precomputed Batch + /// While `async` `Stream`s have a non trivial learning curve, the + /// [`futures`] crate provides [`StreamExt`] and [`TryStreamExt`] + /// which help simplify many common operations. /// - /// We can return a precomputed batch as a stream + /// Here are some common patterns: + /// + /// ## Return Precomputed `RecordBatch` + /// + /// We can return a precomputed `RecordBatch` as a `Stream`: /// /// ``` /// # use std::sync::Arc; @@ -261,6 +306,7 @@ pub trait ExecutionPlan: Debug + DisplayAs + Send + Sync { /// partition: usize, /// context: Arc /// ) -> Result { + /// // use functions from futures crate convert the batch into a stream /// let fut = futures::future::ready(Ok(self.batch.clone())); /// let stream = futures::stream::once(fut); /// Ok(Box::pin(RecordBatchStreamAdapter::new(self.batch.schema(), stream))) @@ -268,9 +314,9 @@ pub trait ExecutionPlan: Debug + DisplayAs + Send + Sync { /// } /// ``` /// - /// ## Async Compute Batch + /// ## Lazily (async) Compute `RecordBatch` /// - /// We can also lazily compute a RecordBatch when the returned stream is polled + /// We can also lazily compute a `RecordBatch` when the returned `Stream` is polled /// /// ``` /// # use std::sync::Arc; @@ -284,6 +330,7 @@ pub trait ExecutionPlan: Debug + DisplayAs + Send + Sync { /// schema: SchemaRef, /// } /// + /// /// Returns a single batch when the returned stream is polled /// async fn get_batch() -> Result { /// todo!() /// } @@ -301,10 +348,10 @@ pub trait ExecutionPlan: Debug + DisplayAs + Send + Sync { /// } /// ``` /// - /// ## Async Compute Batch Stream + /// ## Lazily (async) create a Stream /// - /// We can lazily compute a RecordBatch stream when the returned stream is polled - /// flattening the result into a single stream + /// If you need to to create the return `Stream` using an `async` function, + /// you can do so by flattening the result: /// /// ``` /// # use std::sync::Arc; @@ -319,6 +366,7 @@ pub trait ExecutionPlan: Debug + DisplayAs + Send + Sync { /// schema: SchemaRef, /// } /// + /// /// async function that returns a stream /// async fn get_batch_stream() -> Result { /// todo!() /// } @@ -337,9 +385,6 @@ pub trait ExecutionPlan: Debug + DisplayAs + Send + Sync { /// } /// } /// ``` - /// - /// See [`futures::stream::StreamExt`] and [`futures::stream::TryStreamExt`] for further - /// combinators that can be used with streams fn execute( &self, partition: usize, @@ -372,7 +417,7 @@ pub trait ExecutionPlan: Debug + DisplayAs + Send + Sync { /// Indicate whether a data exchange is needed for the input of `plan`, which will be very helpful /// especially for the distributed engine to judge whether need to deal with shuffling. /// Currently there are 3 kinds of execution plan which needs data exchange -/// 1. RepartitionExec for changing the partition number between two operators +/// 1. RepartitionExec for changing the partition number between two `ExecutionPlan`s /// 2. CoalescePartitionsExec for collapsing all of the partitions into one without ordering guarantee /// 3. SortPreservingMergeExec for collapsing all of the sorted partitions into one with ordering guarantee pub fn need_data_exchange(plan: Arc) -> bool { From 0fa4ce963af7f87afc24ab6eb474c337ddd0018d Mon Sep 17 00:00:00 2001 From: Ruihang Xia Date: Fri, 3 Nov 2023 02:59:54 +0800 Subject: [PATCH 180/572] fix: clippy warnings from nightly rust 1.75 (#8025) Signed-off-by: Ruihang Xia --- datafusion/common/src/error.rs | 2 +- datafusion/core/tests/fuzz_cases/aggregate_fuzz.rs | 1 + datafusion/core/tests/fuzz_cases/window_fuzz.rs | 1 + datafusion/optimizer/src/simplify_expressions/utils.rs | 2 +- 4 files changed, 4 insertions(+), 2 deletions(-) diff --git a/datafusion/common/src/error.rs b/datafusion/common/src/error.rs index adf58e282ed9..9114c669ab8b 100644 --- a/datafusion/common/src/error.rs +++ b/datafusion/common/src/error.rs @@ -369,7 +369,7 @@ impl From for io::Error { } impl DataFusionError { - const BACK_TRACE_SEP: &str = "\n\nbacktrace: "; + const BACK_TRACE_SEP: &'static str = "\n\nbacktrace: "; /// Get deepest underlying [`DataFusionError`] /// diff --git a/datafusion/core/tests/fuzz_cases/aggregate_fuzz.rs b/datafusion/core/tests/fuzz_cases/aggregate_fuzz.rs index 50d3610deed3..798f7ae8baf6 100644 --- a/datafusion/core/tests/fuzz_cases/aggregate_fuzz.rs +++ b/datafusion/core/tests/fuzz_cases/aggregate_fuzz.rs @@ -36,6 +36,7 @@ use datafusion_physical_expr::{AggregateExpr, PhysicalSortExpr}; use test_utils::add_empty_batches; #[cfg(test)] +#[allow(clippy::items_after_test_module)] mod tests { use super::*; diff --git a/datafusion/core/tests/fuzz_cases/window_fuzz.rs b/datafusion/core/tests/fuzz_cases/window_fuzz.rs index db940a9794a1..66f7374a51fb 100644 --- a/datafusion/core/tests/fuzz_cases/window_fuzz.rs +++ b/datafusion/core/tests/fuzz_cases/window_fuzz.rs @@ -45,6 +45,7 @@ use rand::rngs::StdRng; use rand::{Rng, SeedableRng}; #[cfg(test)] +#[allow(clippy::items_after_test_module)] mod tests { use super::*; diff --git a/datafusion/optimizer/src/simplify_expressions/utils.rs b/datafusion/optimizer/src/simplify_expressions/utils.rs index 28c61427c5ef..17e5d97c3006 100644 --- a/datafusion/optimizer/src/simplify_expressions/utils.rs +++ b/datafusion/optimizer/src/simplify_expressions/utils.rs @@ -525,8 +525,8 @@ pub fn simpl_concat_ws(delimiter: &Expr, args: &[Expr]) -> Result { d => Ok(concat_ws( d.clone(), args.iter() + .filter(|&x| !is_null(x)) .cloned() - .filter(|x| !is_null(x)) .collect::>(), )), } From 661d211a62b39a32e3aba29df55fc6ccd9fa1d22 Mon Sep 17 00:00:00 2001 From: Jay Zhan Date: Fri, 3 Nov 2023 04:03:58 +0800 Subject: [PATCH 181/572] Minor: Avoid recomputing compute_array_ndims in align_array_dimensions (#7963) * Refactor align_array_dimensions Signed-off-by: jayzhan211 * address comment Signed-off-by: jayzhan211 * remove unwrap Signed-off-by: jayzhan211 * address comment Signed-off-by: jayzhan211 * fix rebase Signed-off-by: jayzhan211 --------- Signed-off-by: jayzhan211 --- .../physical-expr/src/array_expressions.rs | 75 ++++++++++++++----- 1 file changed, 56 insertions(+), 19 deletions(-) diff --git a/datafusion/physical-expr/src/array_expressions.rs b/datafusion/physical-expr/src/array_expressions.rs index 18d8c60fe7bd..687502e79fed 100644 --- a/datafusion/physical-expr/src/array_expressions.rs +++ b/datafusion/physical-expr/src/array_expressions.rs @@ -725,35 +725,31 @@ pub fn array_prepend(args: &[ArrayRef]) -> Result { } fn align_array_dimensions(args: Vec) -> Result> { - // Find the maximum number of dimensions - let max_ndim: u64 = (*args + let args_ndim = args .iter() - .map(|arr| compute_array_ndims(Some(arr.clone()))) - .collect::>>>()? - .iter() - .max() - .unwrap()) - .unwrap(); + .map(|arg| compute_array_ndims(Some(arg.to_owned()))) + .collect::>>()? + .into_iter() + .map(|x| x.unwrap_or(0)) + .collect::>(); + let max_ndim = args_ndim.iter().max().unwrap_or(&0); // Align the dimensions of the arrays let aligned_args: Result> = args .into_iter() - .map(|array| { - let ndim = compute_array_ndims(Some(array.clone()))?.unwrap(); + .zip(args_ndim.iter()) + .map(|(array, ndim)| { if ndim < max_ndim { let mut aligned_array = array.clone(); for _ in 0..(max_ndim - ndim) { - let data_type = aligned_array.as_ref().data_type().clone(); - let offsets: Vec = - (0..downcast_arg!(aligned_array, ListArray).offsets().len()) - .map(|i| i as i32) - .collect(); - let field = Arc::new(Field::new("item", data_type, true)); + let data_type = aligned_array.data_type().to_owned(); + let array_lengths = vec![1; aligned_array.len()]; + let offsets = OffsetBuffer::::from_lengths(array_lengths); aligned_array = Arc::new(ListArray::try_new( - field, - OffsetBuffer::new(offsets.into()), - Arc::new(aligned_array.clone()), + Arc::new(Field::new("item", data_type, true)), + offsets, + aligned_array, None, )?) } @@ -1923,6 +1919,47 @@ mod tests { use arrow::datatypes::Int64Type; use datafusion_common::cast::as_uint64_array; + #[test] + fn test_align_array_dimensions() { + let array1d_1 = + Arc::new(ListArray::from_iter_primitive::(vec![ + Some(vec![Some(1), Some(2), Some(3)]), + Some(vec![Some(4), Some(5)]), + ])); + let array1d_2 = + Arc::new(ListArray::from_iter_primitive::(vec![ + Some(vec![Some(6), Some(7), Some(8)]), + ])); + + let array2d_1 = Arc::new(array_into_list_array(array1d_1.clone())) as ArrayRef; + let array2d_2 = Arc::new(array_into_list_array(array1d_2.clone())) as ArrayRef; + + let res = + align_array_dimensions(vec![array1d_1.to_owned(), array2d_2.to_owned()]) + .unwrap(); + + let expected = as_list_array(&array2d_1).unwrap(); + let expected_dim = compute_array_ndims(Some(array2d_1.to_owned())).unwrap(); + assert_ne!(as_list_array(&res[0]).unwrap(), expected); + assert_eq!( + compute_array_ndims(Some(res[0].clone())).unwrap(), + expected_dim + ); + + let array3d_1 = Arc::new(array_into_list_array(array2d_1)) as ArrayRef; + let array3d_2 = array_into_list_array(array2d_2.to_owned()); + let res = + align_array_dimensions(vec![array1d_1, Arc::new(array3d_2.clone())]).unwrap(); + + let expected = as_list_array(&array3d_1).unwrap(); + let expected_dim = compute_array_ndims(Some(array3d_1.to_owned())).unwrap(); + assert_ne!(as_list_array(&res[0]).unwrap(), expected); + assert_eq!( + compute_array_ndims(Some(res[0].clone())).unwrap(), + expected_dim + ); + } + #[test] fn test_array() { // make_array(1, 2, 3) = [1, 2, 3] From b2a16681838826ec409c93a26bd7a46262e9a5ef Mon Sep 17 00:00:00 2001 From: Andrew Lamb Date: Fri, 3 Nov 2023 04:48:44 -0400 Subject: [PATCH 182/572] Minor: fix doc check (#8037) --- datafusion/core/src/datasource/file_format/arrow.rs | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/datafusion/core/src/datasource/file_format/arrow.rs b/datafusion/core/src/datasource/file_format/arrow.rs index 2777805078c7..a9bd7d0e27bb 100644 --- a/datafusion/core/src/datasource/file_format/arrow.rs +++ b/datafusion/core/src/datasource/file_format/arrow.rs @@ -105,8 +105,8 @@ impl FileFormat for ArrowFormat { const ARROW_MAGIC: [u8; 6] = [b'A', b'R', b'R', b'O', b'W', b'1']; const CONTINUATION_MARKER: [u8; 4] = [0xff; 4]; -/// Custom implementation of inferring schema. Should eventually be moved upstream to arrow-rs. -/// See https://github.com/apache/arrow-rs/issues/5021 +/// Custom implementation of inferring schema. Should eventually be moved upstream to arrow-rs. +/// See async fn infer_schema_from_file_stream( mut stream: BoxStream<'static, object_store::Result>, ) -> Result { From 8c42d94851daa6f43053cd903222adfd1d81cb82 Mon Sep 17 00:00:00 2001 From: Andrew Lamb Date: Fri, 3 Nov 2023 05:19:24 -0400 Subject: [PATCH 183/572] Minor: remove uncessary #cfg test (#8036) MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit * Minor: remove uncessary #cfg test * fmt * Update datafusion/core/src/datasource/file_format/arrow.rs Co-authored-by: Ruihang Xia --------- Co-authored-by: Daniël Heres Co-authored-by: Ruihang Xia --- .../core/tests/fuzz_cases/aggregate_fuzz.rs | 60 +++--- .../core/tests/fuzz_cases/window_fuzz.rs | 192 +++++++++--------- 2 files changed, 120 insertions(+), 132 deletions(-) diff --git a/datafusion/core/tests/fuzz_cases/aggregate_fuzz.rs b/datafusion/core/tests/fuzz_cases/aggregate_fuzz.rs index 798f7ae8baf6..821f236af87b 100644 --- a/datafusion/core/tests/fuzz_cases/aggregate_fuzz.rs +++ b/datafusion/core/tests/fuzz_cases/aggregate_fuzz.rs @@ -35,39 +35,33 @@ use datafusion_physical_expr::expressions::{col, Sum}; use datafusion_physical_expr::{AggregateExpr, PhysicalSortExpr}; use test_utils::add_empty_batches; -#[cfg(test)] -#[allow(clippy::items_after_test_module)] -mod tests { - use super::*; - - #[tokio::test(flavor = "multi_thread", worker_threads = 8)] - async fn aggregate_test() { - let test_cases = vec![ - vec!["a"], - vec!["b", "a"], - vec!["c", "a"], - vec!["c", "b", "a"], - vec!["d", "a"], - vec!["d", "b", "a"], - vec!["d", "c", "a"], - vec!["d", "c", "b", "a"], - ]; - let n = 300; - let distincts = vec![10, 20]; - for distinct in distincts { - let mut handles = Vec::new(); - for i in 0..n { - let test_idx = i % test_cases.len(); - let group_by_columns = test_cases[test_idx].clone(); - let job = tokio::spawn(run_aggregate_test( - make_staggered_batches::(1000, distinct, i as u64), - group_by_columns, - )); - handles.push(job); - } - for job in handles { - job.await.unwrap(); - } +#[tokio::test(flavor = "multi_thread", worker_threads = 8)] +async fn aggregate_test() { + let test_cases = vec![ + vec!["a"], + vec!["b", "a"], + vec!["c", "a"], + vec!["c", "b", "a"], + vec!["d", "a"], + vec!["d", "b", "a"], + vec!["d", "c", "a"], + vec!["d", "c", "b", "a"], + ]; + let n = 300; + let distincts = vec![10, 20]; + for distinct in distincts { + let mut handles = Vec::new(); + for i in 0..n { + let test_idx = i % test_cases.len(); + let group_by_columns = test_cases[test_idx].clone(); + let job = tokio::spawn(run_aggregate_test( + make_staggered_batches::(1000, distinct, i as u64), + group_by_columns, + )); + handles.push(job); + } + for job in handles { + job.await.unwrap(); } } } diff --git a/datafusion/core/tests/fuzz_cases/window_fuzz.rs b/datafusion/core/tests/fuzz_cases/window_fuzz.rs index 66f7374a51fb..9f70321318fd 100644 --- a/datafusion/core/tests/fuzz_cases/window_fuzz.rs +++ b/datafusion/core/tests/fuzz_cases/window_fuzz.rs @@ -44,108 +44,102 @@ use hashbrown::HashMap; use rand::rngs::StdRng; use rand::{Rng, SeedableRng}; -#[cfg(test)] -#[allow(clippy::items_after_test_module)] -mod tests { - use super::*; - - use datafusion_physical_plan::windows::PartitionSearchMode::{ - Linear, PartiallySorted, Sorted, - }; +use datafusion_physical_plan::windows::PartitionSearchMode::{ + Linear, PartiallySorted, Sorted, +}; - #[tokio::test(flavor = "multi_thread", worker_threads = 16)] - async fn window_bounded_window_random_comparison() -> Result<()> { - // make_staggered_batches gives result sorted according to a, b, c - // In the test cases first entry represents partition by columns - // Second entry represents order by columns. - // Third entry represents search mode. - // In sorted mode physical plans are in the form for WindowAggExec - //``` - // WindowAggExec - // MemoryExec] - // ``` - // and in the form for BoundedWindowAggExec - // ``` - // BoundedWindowAggExec - // MemoryExec - // ``` - // In Linear and PartiallySorted mode physical plans are in the form for WindowAggExec - //``` - // WindowAggExec - // SortExec(required by window function) - // MemoryExec] - // ``` - // and in the form for BoundedWindowAggExec - // ``` - // BoundedWindowAggExec - // MemoryExec - // ``` - let test_cases = vec![ - (vec!["a"], vec!["a"], Sorted), - (vec!["a"], vec!["b"], Sorted), - (vec!["a"], vec!["a", "b"], Sorted), - (vec!["a"], vec!["b", "c"], Sorted), - (vec!["a"], vec!["a", "b", "c"], Sorted), - (vec!["b"], vec!["a"], Linear), - (vec!["b"], vec!["a", "b"], Linear), - (vec!["b"], vec!["a", "c"], Linear), - (vec!["b"], vec!["a", "b", "c"], Linear), - (vec!["c"], vec!["a"], Linear), - (vec!["c"], vec!["a", "b"], Linear), - (vec!["c"], vec!["a", "c"], Linear), - (vec!["c"], vec!["a", "b", "c"], Linear), - (vec!["b", "a"], vec!["a"], Sorted), - (vec!["b", "a"], vec!["b"], Sorted), - (vec!["b", "a"], vec!["c"], Sorted), - (vec!["b", "a"], vec!["a", "b"], Sorted), - (vec!["b", "a"], vec!["b", "c"], Sorted), - (vec!["b", "a"], vec!["a", "c"], Sorted), - (vec!["b", "a"], vec!["a", "b", "c"], Sorted), - (vec!["c", "b"], vec!["a"], Linear), - (vec!["c", "b"], vec!["a", "b"], Linear), - (vec!["c", "b"], vec!["a", "c"], Linear), - (vec!["c", "b"], vec!["a", "b", "c"], Linear), - (vec!["c", "a"], vec!["a"], PartiallySorted(vec![1])), - (vec!["c", "a"], vec!["b"], PartiallySorted(vec![1])), - (vec!["c", "a"], vec!["c"], PartiallySorted(vec![1])), - (vec!["c", "a"], vec!["a", "b"], PartiallySorted(vec![1])), - (vec!["c", "a"], vec!["b", "c"], PartiallySorted(vec![1])), - (vec!["c", "a"], vec!["a", "c"], PartiallySorted(vec![1])), - ( - vec!["c", "a"], - vec!["a", "b", "c"], - PartiallySorted(vec![1]), - ), - (vec!["c", "b", "a"], vec!["a"], Sorted), - (vec!["c", "b", "a"], vec!["b"], Sorted), - (vec!["c", "b", "a"], vec!["c"], Sorted), - (vec!["c", "b", "a"], vec!["a", "b"], Sorted), - (vec!["c", "b", "a"], vec!["b", "c"], Sorted), - (vec!["c", "b", "a"], vec!["a", "c"], Sorted), - (vec!["c", "b", "a"], vec!["a", "b", "c"], Sorted), - ]; - let n = 300; - let n_distincts = vec![10, 20]; - for n_distinct in n_distincts { - let mut handles = Vec::new(); - for i in 0..n { - let idx = i % test_cases.len(); - let (pb_cols, ob_cols, search_mode) = test_cases[idx].clone(); - let job = tokio::spawn(run_window_test( - make_staggered_batches::(1000, n_distinct, i as u64), - i as u64, - pb_cols, - ob_cols, - search_mode, - )); - handles.push(job); - } - for job in handles { - job.await.unwrap()?; - } +#[tokio::test(flavor = "multi_thread", worker_threads = 16)] +async fn window_bounded_window_random_comparison() -> Result<()> { + // make_staggered_batches gives result sorted according to a, b, c + // In the test cases first entry represents partition by columns + // Second entry represents order by columns. + // Third entry represents search mode. + // In sorted mode physical plans are in the form for WindowAggExec + //``` + // WindowAggExec + // MemoryExec] + // ``` + // and in the form for BoundedWindowAggExec + // ``` + // BoundedWindowAggExec + // MemoryExec + // ``` + // In Linear and PartiallySorted mode physical plans are in the form for WindowAggExec + //``` + // WindowAggExec + // SortExec(required by window function) + // MemoryExec] + // ``` + // and in the form for BoundedWindowAggExec + // ``` + // BoundedWindowAggExec + // MemoryExec + // ``` + let test_cases = vec![ + (vec!["a"], vec!["a"], Sorted), + (vec!["a"], vec!["b"], Sorted), + (vec!["a"], vec!["a", "b"], Sorted), + (vec!["a"], vec!["b", "c"], Sorted), + (vec!["a"], vec!["a", "b", "c"], Sorted), + (vec!["b"], vec!["a"], Linear), + (vec!["b"], vec!["a", "b"], Linear), + (vec!["b"], vec!["a", "c"], Linear), + (vec!["b"], vec!["a", "b", "c"], Linear), + (vec!["c"], vec!["a"], Linear), + (vec!["c"], vec!["a", "b"], Linear), + (vec!["c"], vec!["a", "c"], Linear), + (vec!["c"], vec!["a", "b", "c"], Linear), + (vec!["b", "a"], vec!["a"], Sorted), + (vec!["b", "a"], vec!["b"], Sorted), + (vec!["b", "a"], vec!["c"], Sorted), + (vec!["b", "a"], vec!["a", "b"], Sorted), + (vec!["b", "a"], vec!["b", "c"], Sorted), + (vec!["b", "a"], vec!["a", "c"], Sorted), + (vec!["b", "a"], vec!["a", "b", "c"], Sorted), + (vec!["c", "b"], vec!["a"], Linear), + (vec!["c", "b"], vec!["a", "b"], Linear), + (vec!["c", "b"], vec!["a", "c"], Linear), + (vec!["c", "b"], vec!["a", "b", "c"], Linear), + (vec!["c", "a"], vec!["a"], PartiallySorted(vec![1])), + (vec!["c", "a"], vec!["b"], PartiallySorted(vec![1])), + (vec!["c", "a"], vec!["c"], PartiallySorted(vec![1])), + (vec!["c", "a"], vec!["a", "b"], PartiallySorted(vec![1])), + (vec!["c", "a"], vec!["b", "c"], PartiallySorted(vec![1])), + (vec!["c", "a"], vec!["a", "c"], PartiallySorted(vec![1])), + ( + vec!["c", "a"], + vec!["a", "b", "c"], + PartiallySorted(vec![1]), + ), + (vec!["c", "b", "a"], vec!["a"], Sorted), + (vec!["c", "b", "a"], vec!["b"], Sorted), + (vec!["c", "b", "a"], vec!["c"], Sorted), + (vec!["c", "b", "a"], vec!["a", "b"], Sorted), + (vec!["c", "b", "a"], vec!["b", "c"], Sorted), + (vec!["c", "b", "a"], vec!["a", "c"], Sorted), + (vec!["c", "b", "a"], vec!["a", "b", "c"], Sorted), + ]; + let n = 300; + let n_distincts = vec![10, 20]; + for n_distinct in n_distincts { + let mut handles = Vec::new(); + for i in 0..n { + let idx = i % test_cases.len(); + let (pb_cols, ob_cols, search_mode) = test_cases[idx].clone(); + let job = tokio::spawn(run_window_test( + make_staggered_batches::(1000, n_distinct, i as u64), + i as u64, + pb_cols, + ob_cols, + search_mode, + )); + handles.push(job); + } + for job in handles { + job.await.unwrap()?; } - Ok(()) } + Ok(()) } fn get_random_function( From 2906a24e418081a995eebd2ca04cb2b4dc4a10e7 Mon Sep 17 00:00:00 2001 From: Andrew Lamb Date: Fri, 3 Nov 2023 11:49:26 -0400 Subject: [PATCH 184/572] Minor: Improve documentation for `PartitionStream` and `StreamingTableExec` (#8035) * Minor: Improve documentation for `PartitionStream` and `StreamingTableExec` * fmt * fmt --- datafusion/physical-plan/src/streaming.rs | 11 +++++++++-- 1 file changed, 9 insertions(+), 2 deletions(-) diff --git a/datafusion/physical-plan/src/streaming.rs b/datafusion/physical-plan/src/streaming.rs index 27f03b727c29..77b56e1d7540 100644 --- a/datafusion/physical-plan/src/streaming.rs +++ b/datafusion/physical-plan/src/streaming.rs @@ -15,7 +15,7 @@ // specific language governing permissions and limitations // under the License. -//! Execution plan for streaming [`PartitionStream`] +//! Generic plans for deferred execution: [`StreamingTableExec`] and [`PartitionStream`] use std::any::Any; use std::sync::Arc; @@ -35,6 +35,10 @@ use futures::stream::StreamExt; use log::debug; /// A partition that can be converted into a [`SendableRecordBatchStream`] +/// +/// Combined with [`StreamingTableExec`], you can use this trait to implement +/// [`ExecutionPlan`] for a custom source with less boiler plate than +/// implementing `ExecutionPlan` directly for many use cases. pub trait PartitionStream: Send + Sync { /// Returns the schema of this partition fn schema(&self) -> &SchemaRef; @@ -43,7 +47,10 @@ pub trait PartitionStream: Send + Sync { fn execute(&self, ctx: Arc) -> SendableRecordBatchStream; } -/// An [`ExecutionPlan`] for [`PartitionStream`] +/// An [`ExecutionPlan`] for one or more [`PartitionStream`]s. +/// +/// If your source can be represented as one or more [`PartitionStream`]s, you can +/// use this struct to implement [`ExecutionPlan`]. pub struct StreamingTableExec { partitions: Vec>, projection: Option>, From c2e768052c43e4bab6705ee76befc19de383c2cb Mon Sep 17 00:00:00 2001 From: Mustafa Akur <106137913+mustafasrepo@users.noreply.github.com> Date: Fri, 3 Nov 2023 19:03:19 +0300 Subject: [PATCH 185/572] Combine Equivalence and Ordering equivalence to simplify state (#8006) * combine equivalence and ordering equivalence * Remove EquivalenceProperties struct * Minor changes * all tests pass * Refactor oeq * Simplifications * Resolve linter errors * Minor changes * Minor changes * Add new tests * Simplifications window mode selection * Simplifications * Use set_satisfy api * Use utils for aggregate * Minor changes * Minor changes * Minor changes * All tests pass * Simplifications * Simplifications * Minor changes * Simplifications * All tests pass, fix bug * Remove unnecessary code * Simplifications * Minor changes * Simplifications * Move oeq join to methods * Simplifications * Remove redundant code * Minor changes * Minor changes * Simplifications * Simplifications * Simplifications * Move window to util from method, simplifications * Simplifications * Propagate meet in the union * Simplifications * Minor changes, rename * Address berkay reviews * Simplifications * Add new buggy test * Add data test for sort requirement * Add experimental check * Add random test * Minor changes * Random test gives error * Fix missing test case * Minor changes * Minor changes * Simplifications * Minor changes * Add new test case * Minor changes * Address reviews * Minor changes * Increase coverage of random tests * Remove redundant code * Simplifications * Simplifications * Refactor on tests * Solving clippy errors * prune_lex improvements * Fix failing tests * Update get_finer and get_meet * Fix window lex ordering implementation * Buggy state * Do not use output ordering in the aggregate * Add union test * Update comment * Fix bug, when batch_size is small * Review Part 1 * Review Part 2 * Change union meet implementation * Update comments * Remove redundant check * Simplify project out_expr function * Remove Option> API. * Do not use project_out_expr * Simplifications * Review Part 3 * Review Part 4 * Review Part 5 * Review Part 6 * Review Part 7 * Review Part 8 * Update comments * Add new unit tests, simplifications * Resolve linter errors * Simplify test codes * Review Part 9 * Add unit tests for remove_redundant entries * Simplifications * Review Part 10 * Fix test * Add new test case, fix implementation * Review Part 11 * Review Part 12 * Update comments * Review Part 13 * Review Part 14 * Review Part 15 * Review Part 16 * Review Part 17 * Review Part 18 * Review Part 19 * Review Part 20 * Review Part 21 * Review Part 22 * Review Part 23 * Review Part 24 * Do not construct idx and sort_expr unnecessarily, Update comments, Union meet single entry * Review Part 25 * Review Part 26 * Name Changes, comment updates * Review Part 27 * Add issue links * Address reviews * Fix failing test * Update comments * SortPreservingMerge, SortPreservingRepartition only preserves given expression ordering among input ordering equivalences --------- Co-authored-by: metesynnada <100111937+metesynnada@users.noreply.github.com> Co-authored-by: Mehmet Ozan Kabak --- .github/pull_request_template.md | 2 +- .../datasource/physical_plan/arrow_file.rs | 9 +- .../core/src/datasource/physical_plan/avro.rs | 8 +- .../core/src/datasource/physical_plan/csv.rs | 8 +- .../core/src/datasource/physical_plan/json.rs | 8 +- .../src/datasource/physical_plan/parquet.rs | 7 +- .../enforce_distribution.rs | 104 +- .../src/physical_optimizer/enforce_sorting.rs | 29 +- .../physical_optimizer/output_requirements.rs | 6 +- .../replace_with_order_preserving_variants.rs | 20 +- .../src/physical_optimizer/sort_pushdown.rs | 176 +- .../core/src/physical_optimizer/utils.rs | 15 +- .../core/tests/fuzz_cases/window_fuzz.rs | 1 - datafusion/physical-expr/src/equivalence.rs | 3444 +++++++++++------ .../physical-expr/src/expressions/in_list.rs | 5 +- datafusion/physical-expr/src/lib.rs | 21 +- datafusion/physical-expr/src/partitioning.rs | 39 +- datafusion/physical-expr/src/physical_expr.rs | 208 +- .../physical-expr/src/scalar_function.rs | 8 +- datafusion/physical-expr/src/sort_expr.rs | 49 +- .../physical-expr/src/sort_properties.rs | 62 +- datafusion/physical-expr/src/utils.rs | 1355 +------ .../physical-expr/src/window/built_in.rs | 41 +- .../physical-plan/src/aggregates/mod.rs | 617 +-- .../physical-plan/src/aggregates/order/mod.rs | 30 +- .../physical-plan/src/aggregates/row_hash.rs | 80 +- .../physical-plan/src/coalesce_batches.rs | 20 +- .../physical-plan/src/coalesce_partitions.rs | 8 +- datafusion/physical-plan/src/common.rs | 34 + datafusion/physical-plan/src/filter.rs | 56 +- .../physical-plan/src/joins/cross_join.rs | 34 +- .../physical-plan/src/joins/hash_join.rs | 59 +- .../src/joins/nested_loop_join.rs | 13 +- .../src/joins/sort_merge_join.rs | 27 +- .../src/joins/symmetric_hash_join.rs | 13 +- datafusion/physical-plan/src/joins/utils.rs | 35 +- datafusion/physical-plan/src/lib.rs | 11 +- datafusion/physical-plan/src/limit.rs | 9 - datafusion/physical-plan/src/memory.rs | 22 +- datafusion/physical-plan/src/projection.rs | 175 +- .../physical-plan/src/repartition/mod.rs | 27 +- datafusion/physical-plan/src/sorts/sort.rs | 5 +- .../src/sorts/sort_preserving_merge.rs | 18 +- datafusion/physical-plan/src/streaming.rs | 10 +- datafusion/physical-plan/src/union.rs | 173 + datafusion/physical-plan/src/unnest.rs | 9 +- .../src/windows/bounded_window_agg_exec.rs | 24 +- datafusion/physical-plan/src/windows/mod.rs | 311 +- .../src/windows/window_agg_exec.rs | 30 +- .../sqllogictest/test_files/groupby.slt | 93 +- datafusion/sqllogictest/test_files/joins.slt | 125 + .../sqllogictest/test_files/tpch/q17.slt.part | 26 +- datafusion/sqllogictest/test_files/window.slt | 120 +- 53 files changed, 3989 insertions(+), 3850 deletions(-) diff --git a/.github/pull_request_template.md b/.github/pull_request_template.md index 442e6e4009f6..06db092d6fc8 100644 --- a/.github/pull_request_template.md +++ b/.github/pull_request_template.md @@ -37,4 +37,4 @@ If there are user-facing changes then we may require documentation to be updated \ No newline at end of file +--> diff --git a/datafusion/core/src/datasource/physical_plan/arrow_file.rs b/datafusion/core/src/datasource/physical_plan/arrow_file.rs index e00e8aea0a04..30b55db28491 100644 --- a/datafusion/core/src/datasource/physical_plan/arrow_file.rs +++ b/datafusion/core/src/datasource/physical_plan/arrow_file.rs @@ -32,10 +32,7 @@ use crate::physical_plan::{ use arrow_schema::SchemaRef; use datafusion_common::Statistics; use datafusion_execution::TaskContext; -use datafusion_physical_expr::{ - ordering_equivalence_properties_helper, LexOrdering, OrderingEquivalenceProperties, - PhysicalSortExpr, -}; +use datafusion_physical_expr::{EquivalenceProperties, LexOrdering, PhysicalSortExpr}; use futures::StreamExt; use object_store::{GetResultPayload, ObjectStore}; @@ -106,8 +103,8 @@ impl ExecutionPlan for ArrowExec { .map(|ordering| ordering.as_slice()) } - fn ordering_equivalence_properties(&self) -> OrderingEquivalenceProperties { - ordering_equivalence_properties_helper( + fn equivalence_properties(&self) -> EquivalenceProperties { + EquivalenceProperties::new_with_orderings( self.schema(), &self.projected_output_ordering, ) diff --git a/datafusion/core/src/datasource/physical_plan/avro.rs b/datafusion/core/src/datasource/physical_plan/avro.rs index 237772eb8360..b97f162fd2f5 100644 --- a/datafusion/core/src/datasource/physical_plan/avro.rs +++ b/datafusion/core/src/datasource/physical_plan/avro.rs @@ -31,9 +31,7 @@ use crate::physical_plan::{ use arrow::datatypes::SchemaRef; use datafusion_execution::TaskContext; -use datafusion_physical_expr::{ - ordering_equivalence_properties_helper, LexOrdering, OrderingEquivalenceProperties, -}; +use datafusion_physical_expr::{EquivalenceProperties, LexOrdering}; /// Execution plan for scanning Avro data source #[derive(Debug, Clone)] @@ -101,8 +99,8 @@ impl ExecutionPlan for AvroExec { .map(|ordering| ordering.as_slice()) } - fn ordering_equivalence_properties(&self) -> OrderingEquivalenceProperties { - ordering_equivalence_properties_helper( + fn equivalence_properties(&self) -> EquivalenceProperties { + EquivalenceProperties::new_with_orderings( self.schema(), &self.projected_output_ordering, ) diff --git a/datafusion/core/src/datasource/physical_plan/csv.rs b/datafusion/core/src/datasource/physical_plan/csv.rs index 82163da64af8..75aa343ffbfc 100644 --- a/datafusion/core/src/datasource/physical_plan/csv.rs +++ b/datafusion/core/src/datasource/physical_plan/csv.rs @@ -41,9 +41,7 @@ use crate::physical_plan::{ use arrow::csv; use arrow::datatypes::SchemaRef; use datafusion_execution::TaskContext; -use datafusion_physical_expr::{ - ordering_equivalence_properties_helper, LexOrdering, OrderingEquivalenceProperties, -}; +use datafusion_physical_expr::{EquivalenceProperties, LexOrdering}; use bytes::{Buf, Bytes}; use datafusion_common::config::ConfigOptions; @@ -159,8 +157,8 @@ impl ExecutionPlan for CsvExec { .map(|ordering| ordering.as_slice()) } - fn ordering_equivalence_properties(&self) -> OrderingEquivalenceProperties { - ordering_equivalence_properties_helper( + fn equivalence_properties(&self) -> EquivalenceProperties { + EquivalenceProperties::new_with_orderings( self.schema(), &self.projected_output_ordering, ) diff --git a/datafusion/core/src/datasource/physical_plan/json.rs b/datafusion/core/src/datasource/physical_plan/json.rs index 1ba8e47a523c..73dcb32ac81f 100644 --- a/datafusion/core/src/datasource/physical_plan/json.rs +++ b/datafusion/core/src/datasource/physical_plan/json.rs @@ -40,9 +40,7 @@ use crate::physical_plan::{ use arrow::json::ReaderBuilder; use arrow::{datatypes::SchemaRef, json}; use datafusion_execution::TaskContext; -use datafusion_physical_expr::{ - ordering_equivalence_properties_helper, LexOrdering, OrderingEquivalenceProperties, -}; +use datafusion_physical_expr::{EquivalenceProperties, LexOrdering}; use bytes::{Buf, Bytes}; use futures::{ready, stream, StreamExt, TryStreamExt}; @@ -122,8 +120,8 @@ impl ExecutionPlan for NdJsonExec { .map(|ordering| ordering.as_slice()) } - fn ordering_equivalence_properties(&self) -> OrderingEquivalenceProperties { - ordering_equivalence_properties_helper( + fn equivalence_properties(&self) -> EquivalenceProperties { + EquivalenceProperties::new_with_orderings( self.schema(), &self.projected_output_ordering, ) diff --git a/datafusion/core/src/datasource/physical_plan/parquet.rs b/datafusion/core/src/datasource/physical_plan/parquet.rs index f6e999f60249..960b2ec7337d 100644 --- a/datafusion/core/src/datasource/physical_plan/parquet.rs +++ b/datafusion/core/src/datasource/physical_plan/parquet.rs @@ -45,8 +45,7 @@ use crate::{ use arrow::datatypes::{DataType, SchemaRef}; use arrow::error::ArrowError; use datafusion_physical_expr::{ - ordering_equivalence_properties_helper, LexOrdering, OrderingEquivalenceProperties, - PhysicalExpr, PhysicalSortExpr, + EquivalenceProperties, LexOrdering, PhysicalExpr, PhysicalSortExpr, }; use bytes::Bytes; @@ -315,8 +314,8 @@ impl ExecutionPlan for ParquetExec { .map(|ordering| ordering.as_slice()) } - fn ordering_equivalence_properties(&self) -> OrderingEquivalenceProperties { - ordering_equivalence_properties_helper( + fn equivalence_properties(&self) -> EquivalenceProperties { + EquivalenceProperties::new_with_orderings( self.schema(), &self.projected_output_ordering, ) diff --git a/datafusion/core/src/physical_optimizer/enforce_distribution.rs b/datafusion/core/src/physical_optimizer/enforce_distribution.rs index 6de39db7d52a..ee6e11bd271a 100644 --- a/datafusion/core/src/physical_optimizer/enforce_distribution.rs +++ b/datafusion/core/src/physical_optimizer/enforce_distribution.rs @@ -49,12 +49,11 @@ use crate::physical_plan::{ use arrow::compute::SortOptions; use datafusion_common::tree_node::{Transformed, TreeNode, VisitRecursion}; use datafusion_expr::logical_plan::JoinType; -use datafusion_physical_expr::equivalence::EquivalenceProperties; use datafusion_physical_expr::expressions::{Column, NoOp}; -use datafusion_physical_expr::utils::{ - map_columns_before_projection, ordering_satisfy_requirement_concrete, +use datafusion_physical_expr::utils::map_columns_before_projection; +use datafusion_physical_expr::{ + physical_exprs_equal, EquivalenceProperties, PhysicalExpr, }; -use datafusion_physical_expr::{expr_list_eq_strict_order, PhysicalExpr}; use datafusion_physical_plan::unbounded_output; use datafusion_physical_plan::windows::{get_best_fitting_window, BoundedWindowAggExec}; @@ -498,7 +497,7 @@ fn reorder_aggregate_keys( if parent_required.len() != output_exprs.len() || !agg_exec.group_by().null_expr().is_empty() - || expr_list_eq_strict_order(&output_exprs, parent_required) + || physical_exprs_equal(&output_exprs, parent_required) { Ok(PlanWithKeyRequirements::new(agg_plan)) } else { @@ -564,13 +563,11 @@ fn reorder_aggregate_keys( Arc::new(Column::new( name, agg_schema.index_of(name).unwrap(), - )) - as Arc, + )) as _, name.to_owned(), ) }) .collect::>(); - let agg_schema = new_final_agg.schema(); let agg_fields = agg_schema.fields(); for (idx, field) in agg_fields.iter().enumerate().skip(output_columns.len()) @@ -706,10 +703,9 @@ pub(crate) fn reorder_join_keys_to_inputs( ) { if !new_positions.is_empty() { let new_join_on = new_join_conditions(&left_keys, &right_keys); - let mut new_sort_options = vec![]; - for idx in 0..sort_options.len() { - new_sort_options.push(sort_options[new_positions[idx]]) - } + let new_sort_options = (0..sort_options.len()) + .map(|idx| sort_options[new_positions[idx]]) + .collect(); return Ok(Arc::new(SortMergeJoinExec::try_new( left.clone(), right.clone(), @@ -757,39 +753,40 @@ fn try_reorder( expected: &[Arc], equivalence_properties: &EquivalenceProperties, ) -> Option<(JoinKeyPairs, Vec)> { + let eq_groups = equivalence_properties.eq_group(); let mut normalized_expected = vec![]; let mut normalized_left_keys = vec![]; let mut normalized_right_keys = vec![]; if join_keys.left_keys.len() != expected.len() { return None; } - if expr_list_eq_strict_order(expected, &join_keys.left_keys) - || expr_list_eq_strict_order(expected, &join_keys.right_keys) + if physical_exprs_equal(expected, &join_keys.left_keys) + || physical_exprs_equal(expected, &join_keys.right_keys) { return Some((join_keys, vec![])); - } else if !equivalence_properties.classes().is_empty() { + } else if !equivalence_properties.eq_group().is_empty() { normalized_expected = expected .iter() - .map(|e| equivalence_properties.normalize_expr(e.clone())) + .map(|e| eq_groups.normalize_expr(e.clone())) .collect::>(); assert_eq!(normalized_expected.len(), expected.len()); normalized_left_keys = join_keys .left_keys .iter() - .map(|e| equivalence_properties.normalize_expr(e.clone())) + .map(|e| eq_groups.normalize_expr(e.clone())) .collect::>(); assert_eq!(join_keys.left_keys.len(), normalized_left_keys.len()); normalized_right_keys = join_keys .right_keys .iter() - .map(|e| equivalence_properties.normalize_expr(e.clone())) + .map(|e| eq_groups.normalize_expr(e.clone())) .collect::>(); assert_eq!(join_keys.right_keys.len(), normalized_right_keys.len()); - if expr_list_eq_strict_order(&normalized_expected, &normalized_left_keys) - || expr_list_eq_strict_order(&normalized_expected, &normalized_right_keys) + if physical_exprs_equal(&normalized_expected, &normalized_left_keys) + || physical_exprs_equal(&normalized_expected, &normalized_right_keys) { return Some((join_keys, vec![])); } @@ -870,7 +867,7 @@ fn new_join_conditions( r_key.as_any().downcast_ref::().unwrap().clone(), ) }) - .collect::>() + .collect() } /// Updates `dist_onward` such that, to keep track of @@ -935,9 +932,9 @@ fn add_roundrobin_on_top( let should_preserve_ordering = input.output_ordering().is_some(); let partitioning = Partitioning::RoundRobinBatch(n_target); - let repartition = RepartitionExec::try_new(input, partitioning)? - .with_preserve_order(should_preserve_ordering); - let new_plan = Arc::new(repartition) as Arc; + let repartition = RepartitionExec::try_new(input, partitioning)?; + let new_plan = Arc::new(repartition.with_preserve_order(should_preserve_ordering)) + as Arc; // update distribution onward with new operator update_distribution_onward(new_plan.clone(), dist_onward, input_idx); @@ -1011,9 +1008,9 @@ fn add_hash_on_top( input }; let partitioning = Partitioning::Hash(hash_exprs, n_target); - let repartition = RepartitionExec::try_new(new_plan, partitioning)? - .with_preserve_order(should_preserve_ordering); - new_plan = Arc::new(repartition) as _; + let repartition = RepartitionExec::try_new(new_plan, partitioning)?; + new_plan = + Arc::new(repartition.with_preserve_order(should_preserve_ordering)) as _; // update distribution onward with new operator update_distribution_onward(new_plan.clone(), dist_onward, input_idx); @@ -1302,16 +1299,12 @@ fn ensure_distribution( // There is an ordering requirement of the operator: if let Some(required_input_ordering) = required_input_ordering { - let existing_ordering = child.output_ordering().unwrap_or(&[]); // Either: // - Ordering requirement cannot be satisfied by preserving ordering through repartitions, or // - using order preserving variant is not desirable. - let ordering_satisfied = ordering_satisfy_requirement_concrete( - existing_ordering, - required_input_ordering, - || child.equivalence_properties(), - || child.ordering_equivalence_properties(), - ); + let ordering_satisfied = child + .equivalence_properties() + .ordering_satisfy_requirement(required_input_ordering); if !ordering_satisfied || !order_preserving_variants_desirable { replace_order_preserving_variants(&mut child, dist_onward)?; // If ordering requirements were satisfied before repartitioning, @@ -3763,14 +3756,14 @@ mod tests { fn repartition_transitively_past_sort_with_filter() -> Result<()> { let schema = schema(); let sort_key = vec![PhysicalSortExpr { - expr: col("c", &schema).unwrap(), + expr: col("a", &schema).unwrap(), options: SortOptions::default(), }]; let plan = sort_exec(sort_key, filter_exec(parquet_exec()), false); let expected = &[ - "SortPreservingMergeExec: [c@2 ASC]", - "SortExec: expr=[c@2 ASC]", + "SortPreservingMergeExec: [a@0 ASC]", + "SortExec: expr=[a@0 ASC]", // Expect repartition on the input to the sort (as it can benefit from additional parallelism) "FilterExec: c@2 = 0", "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", @@ -3780,7 +3773,7 @@ mod tests { assert_optimized!(expected, plan.clone(), true); let expected_first_sort_enforcement = &[ - "SortExec: expr=[c@2 ASC]", + "SortExec: expr=[a@0 ASC]", "CoalescePartitionsExec", "FilterExec: c@2 = 0", // Expect repartition on the input of the filter (as it can benefit from additional parallelism) @@ -4357,29 +4350,54 @@ mod tests { fn do_not_preserve_ordering_through_repartition() -> Result<()> { let schema = schema(); let sort_key = vec![PhysicalSortExpr { - expr: col("c", &schema).unwrap(), + expr: col("a", &schema).unwrap(), options: SortOptions::default(), }]; let input = parquet_exec_multiple_sorted(vec![sort_key.clone()]); let physical_plan = sort_preserving_merge_exec(sort_key, filter_exec(input)); let expected = &[ - "SortPreservingMergeExec: [c@2 ASC]", - "SortExec: expr=[c@2 ASC]", + "SortPreservingMergeExec: [a@0 ASC]", + "SortExec: expr=[a@0 ASC]", "FilterExec: c@2 = 0", "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=2", - "ParquetExec: file_groups={2 groups: [[x], [y]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC]", + "ParquetExec: file_groups={2 groups: [[x], [y]]}, projection=[a, b, c, d, e], output_ordering=[a@0 ASC]", ]; assert_optimized!(expected, physical_plan.clone(), true); let expected = &[ - "SortExec: expr=[c@2 ASC]", + "SortExec: expr=[a@0 ASC]", + "CoalescePartitionsExec", + "FilterExec: c@2 = 0", + "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=2", + "ParquetExec: file_groups={2 groups: [[x], [y]]}, projection=[a, b, c, d, e], output_ordering=[a@0 ASC]", + ]; + assert_optimized!(expected, physical_plan, false); + + Ok(()) + } + + #[test] + fn no_need_for_sort_after_filter() -> Result<()> { + let schema = schema(); + let sort_key = vec![PhysicalSortExpr { + expr: col("c", &schema).unwrap(), + options: SortOptions::default(), + }]; + let input = parquet_exec_multiple_sorted(vec![sort_key.clone()]); + let physical_plan = sort_preserving_merge_exec(sort_key, filter_exec(input)); + + let expected = &[ + // After CoalescePartitionsExec c is still constant. Hence c@2 ASC ordering is already satisfied. "CoalescePartitionsExec", + // Since after this stage c is constant. c@2 ASC ordering is already satisfied. "FilterExec: c@2 = 0", "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=2", "ParquetExec: file_groups={2 groups: [[x], [y]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC]", ]; + + assert_optimized!(expected, physical_plan.clone(), true); assert_optimized!(expected, physical_plan, false); Ok(()) diff --git a/datafusion/core/src/physical_optimizer/enforce_sorting.rs b/datafusion/core/src/physical_optimizer/enforce_sorting.rs index 822a224d236a..4779ced44f1a 100644 --- a/datafusion/core/src/physical_optimizer/enforce_sorting.rs +++ b/datafusion/core/src/physical_optimizer/enforce_sorting.rs @@ -51,18 +51,16 @@ use crate::physical_plan::coalesce_partitions::CoalescePartitionsExec; use crate::physical_plan::sorts::sort::SortExec; use crate::physical_plan::sorts::sort_preserving_merge::SortPreservingMergeExec; use crate::physical_plan::windows::{ - get_best_fitting_window, BoundedWindowAggExec, PartitionSearchMode, WindowAggExec, + get_best_fitting_window, BoundedWindowAggExec, WindowAggExec, }; use crate::physical_plan::{with_new_children_if_necessary, Distribution, ExecutionPlan}; use datafusion_common::tree_node::{Transformed, TreeNode, VisitRecursion}; use datafusion_common::{plan_err, DataFusionError}; -use datafusion_physical_expr::utils::{ - ordering_satisfy, ordering_satisfy_requirement_concrete, -}; use datafusion_physical_expr::{PhysicalSortExpr, PhysicalSortRequirement}; use datafusion_physical_plan::repartition::RepartitionExec; +use datafusion_physical_plan::windows::PartitionSearchMode; use itertools::izip; /// This rule inspects [`SortExec`]'s in the given physical plan and removes the @@ -451,13 +449,11 @@ fn ensure_sorting( { let physical_ordering = child.output_ordering(); match (required_ordering, physical_ordering) { - (Some(required_ordering), Some(physical_ordering)) => { - if !ordering_satisfy_requirement_concrete( - physical_ordering, - &required_ordering, - || child.equivalence_properties(), - || child.ordering_equivalence_properties(), - ) { + (Some(required_ordering), Some(_)) => { + if !child + .equivalence_properties() + .ordering_satisfy_requirement(&required_ordering) + { // Make sure we preserve the ordering requirements: update_child_to_remove_unnecessary_sort(child, sort_onwards, &plan)?; add_sort_above(child, &required_ordering, None); @@ -516,13 +512,12 @@ fn analyze_immediate_sort_removal( ) -> Option { if let Some(sort_exec) = plan.as_any().downcast_ref::() { let sort_input = sort_exec.input().clone(); + // If this sort is unnecessary, we should remove it: - if ordering_satisfy( - sort_input.output_ordering(), - sort_exec.output_ordering(), - || sort_input.equivalence_properties(), - || sort_input.ordering_equivalence_properties(), - ) { + if sort_input + .equivalence_properties() + .ordering_satisfy(sort_exec.output_ordering().unwrap_or(&[])) + { // Since we know that a `SortExec` has exactly one child, // we can use the zero index safely: return Some( diff --git a/datafusion/core/src/physical_optimizer/output_requirements.rs b/datafusion/core/src/physical_optimizer/output_requirements.rs index f5eacd5ee60c..d9cdc292dd56 100644 --- a/datafusion/core/src/physical_optimizer/output_requirements.rs +++ b/datafusion/core/src/physical_optimizer/output_requirements.rs @@ -33,7 +33,7 @@ use datafusion_common::config::ConfigOptions; use datafusion_common::tree_node::{Transformed, TreeNode}; use datafusion_common::{Result, Statistics}; use datafusion_physical_expr::{ - Distribution, LexOrderingReq, PhysicalSortExpr, PhysicalSortRequirement, + Distribution, LexRequirement, PhysicalSortExpr, PhysicalSortRequirement, }; use datafusion_physical_plan::sorts::sort_preserving_merge::SortPreservingMergeExec; @@ -90,14 +90,14 @@ enum RuleMode { #[derive(Debug)] struct OutputRequirementExec { input: Arc, - order_requirement: Option, + order_requirement: Option, dist_requirement: Distribution, } impl OutputRequirementExec { fn new( input: Arc, - requirements: Option, + requirements: Option, dist_requirement: Distribution, ) -> Self { Self { diff --git a/datafusion/core/src/physical_optimizer/replace_with_order_preserving_variants.rs b/datafusion/core/src/physical_optimizer/replace_with_order_preserving_variants.rs index fb75c083a70a..0c2f21d11acd 100644 --- a/datafusion/core/src/physical_optimizer/replace_with_order_preserving_variants.rs +++ b/datafusion/core/src/physical_optimizer/replace_with_order_preserving_variants.rs @@ -31,7 +31,6 @@ use super::utils::is_repartition; use datafusion_common::config::ConfigOptions; use datafusion_common::tree_node::{Transformed, TreeNode, VisitRecursion}; -use datafusion_physical_expr::utils::ordering_satisfy; use datafusion_physical_plan::unbounded_output; /// For a given `plan`, this object carries the information one needs from its @@ -258,12 +257,10 @@ pub(crate) fn replace_with_order_preserving_variants( is_spm_better || use_order_preserving_variant, )?; // If this sort is unnecessary, we should remove it and update the plan: - if ordering_satisfy( - updated_sort_input.output_ordering(), - plan.output_ordering(), - || updated_sort_input.equivalence_properties(), - || updated_sort_input.ordering_equivalence_properties(), - ) { + if updated_sort_input + .equivalence_properties() + .ordering_satisfy(plan.output_ordering().unwrap_or(&[])) + { return Ok(Transformed::Yes(OrderPreservationContext { plan: updated_sort_input, ordering_onwards: vec![None], @@ -278,21 +275,21 @@ pub(crate) fn replace_with_order_preserving_variants( mod tests { use super::*; - use crate::prelude::SessionConfig; - use crate::datasource::file_format::file_compression_type::FileCompressionType; use crate::datasource::listing::PartitionedFile; use crate::datasource::physical_plan::{CsvExec, FileScanConfig}; use crate::physical_plan::coalesce_batches::CoalesceBatchesExec; use crate::physical_plan::coalesce_partitions::CoalescePartitionsExec; - use crate::physical_plan::filter::FilterExec; use crate::physical_plan::joins::{HashJoinExec, PartitionMode}; use crate::physical_plan::repartition::RepartitionExec; use crate::physical_plan::sorts::sort::SortExec; use crate::physical_plan::sorts::sort_preserving_merge::SortPreservingMergeExec; use crate::physical_plan::{displayable, Partitioning}; + use crate::prelude::SessionConfig; + use arrow::compute::SortOptions; + use arrow::datatypes::{DataType, Field, Schema, SchemaRef}; use datafusion_common::tree_node::TreeNode; use datafusion_common::{Result, Statistics}; use datafusion_execution::object_store::ObjectStoreUrl; @@ -300,9 +297,6 @@ mod tests { use datafusion_physical_expr::expressions::{self, col, Column}; use datafusion_physical_expr::PhysicalSortExpr; - use arrow::compute::SortOptions; - use arrow::datatypes::{DataType, Field, Schema, SchemaRef}; - /// Runs the `replace_with_order_preserving_variants` sub-rule and asserts the plan /// against the original and expected plans. /// diff --git a/datafusion/core/src/physical_optimizer/sort_pushdown.rs b/datafusion/core/src/physical_optimizer/sort_pushdown.rs index a99399592f15..b9502d92ac12 100644 --- a/datafusion/core/src/physical_optimizer/sort_pushdown.rs +++ b/datafusion/core/src/physical_optimizer/sort_pushdown.rs @@ -29,15 +29,12 @@ use crate::physical_plan::sorts::sort::SortExec; use crate::physical_plan::{with_new_children_if_necessary, ExecutionPlan}; use datafusion_common::tree_node::{Transformed, TreeNode, VisitRecursion}; -use datafusion_common::{ - plan_datafusion_err, plan_err, DataFusionError, JoinSide, Result, -}; +use datafusion_common::{plan_err, DataFusionError, JoinSide, Result}; use datafusion_expr::JoinType; use datafusion_physical_expr::expressions::Column; -use datafusion_physical_expr::utils::{ - ordering_satisfy, ordering_satisfy_requirement, requirements_compatible, +use datafusion_physical_expr::{ + LexRequirementRef, PhysicalSortExpr, PhysicalSortRequirement, }; -use datafusion_physical_expr::{PhysicalSortExpr, PhysicalSortRequirement}; use itertools::izip; @@ -127,29 +124,27 @@ pub(crate) fn pushdown_sorts( requirements: SortPushDown, ) -> Result> { let plan = &requirements.plan; - let parent_required = requirements.required_ordering.as_deref(); - const ERR_MSG: &str = "Expects parent requirement to contain something"; - let err = || plan_datafusion_err!("{}", ERR_MSG); + let parent_required = requirements.required_ordering.as_deref().unwrap_or(&[]); if let Some(sort_exec) = plan.as_any().downcast_ref::() { - let mut new_plan = plan.clone(); - if !ordering_satisfy_requirement( - plan.output_ordering(), - parent_required, - || plan.equivalence_properties(), - || plan.ordering_equivalence_properties(), - ) { + let new_plan = if !plan + .equivalence_properties() + .ordering_satisfy_requirement(parent_required) + { // If the current plan is a SortExec, modify it to satisfy parent requirements: - let parent_required_expr = parent_required.ok_or_else(err)?; - new_plan = sort_exec.input().clone(); - add_sort_above(&mut new_plan, parent_required_expr, sort_exec.fetch()); + let mut new_plan = sort_exec.input().clone(); + add_sort_above(&mut new_plan, parent_required, sort_exec.fetch()); + new_plan + } else { + requirements.plan }; let required_ordering = new_plan .output_ordering() - .map(PhysicalSortRequirement::from_sort_exprs); + .map(PhysicalSortRequirement::from_sort_exprs) + .unwrap_or_default(); // Since new_plan is a SortExec, we can safely get the 0th index. let child = new_plan.children().swap_remove(0); if let Some(adjusted) = - pushdown_requirement_to_children(&child, required_ordering.as_deref())? + pushdown_requirement_to_children(&child, &required_ordering)? { // Can push down requirements Ok(Transformed::Yes(SortPushDown { @@ -163,12 +158,10 @@ pub(crate) fn pushdown_sorts( } } else { // Executors other than SortExec - if ordering_satisfy_requirement( - plan.output_ordering(), - parent_required, - || plan.equivalence_properties(), - || plan.ordering_equivalence_properties(), - ) { + if plan + .equivalence_properties() + .ordering_satisfy_requirement(parent_required) + { // Satisfies parent requirements, immediately return. return Ok(Transformed::Yes(SortPushDown { required_ordering: None, @@ -184,9 +177,8 @@ pub(crate) fn pushdown_sorts( })) } else { // Can not push down requirements, add new SortExec: - let parent_required_expr = parent_required.ok_or_else(err)?; let mut new_plan = requirements.plan; - add_sort_above(&mut new_plan, parent_required_expr, None); + add_sort_above(&mut new_plan, parent_required, None); Ok(Transformed::Yes(SortPushDown::init(new_plan))) } } @@ -194,18 +186,21 @@ pub(crate) fn pushdown_sorts( fn pushdown_requirement_to_children( plan: &Arc, - parent_required: Option<&[PhysicalSortRequirement]>, + parent_required: LexRequirementRef, ) -> Result>>>> { - const ERR_MSG: &str = "Expects parent requirement to contain something"; - let err = || plan_datafusion_err!("{}", ERR_MSG); let maintains_input_order = plan.maintains_input_order(); if is_window(plan) { let required_input_ordering = plan.required_input_ordering(); - let request_child = required_input_ordering[0].as_deref(); + let request_child = required_input_ordering[0].as_deref().unwrap_or(&[]); let child_plan = plan.children().swap_remove(0); match determine_children_requirement(parent_required, request_child, child_plan) { RequirementsCompatibility::Satisfy => { - Ok(Some(vec![request_child.map(|r| r.to_vec())])) + let req = if request_child.is_empty() { + None + } else { + Some(request_child.to_vec()) + }; + Ok(Some(vec![req])) } RequirementsCompatibility::Compatible(adjusted) => Ok(Some(vec![adjusted])), RequirementsCompatibility::NonCompatible => Ok(None), @@ -213,16 +208,17 @@ fn pushdown_requirement_to_children( } else if is_union(plan) { // UnionExec does not have real sort requirements for its input. Here we change the adjusted_request_ordering to UnionExec's output ordering and // propagate the sort requirements down to correct the unnecessary descendant SortExec under the UnionExec - Ok(Some(vec![ - parent_required.map(|elem| elem.to_vec()); - plan.children().len() - ])) + let req = if parent_required.is_empty() { + None + } else { + Some(parent_required.to_vec()) + }; + Ok(Some(vec![req; plan.children().len()])) } else if let Some(smj) = plan.as_any().downcast_ref::() { // If the current plan is SortMergeJoinExec let left_columns_len = smj.left().schema().fields().len(); - let parent_required_expr = PhysicalSortRequirement::to_sort_exprs( - parent_required.ok_or_else(err)?.iter().cloned(), - ); + let parent_required_expr = + PhysicalSortRequirement::to_sort_exprs(parent_required.iter().cloned()); let expr_source_side = expr_source_sides(&parent_required_expr, smj.join_type(), left_columns_len); match expr_source_side { @@ -236,10 +232,9 @@ fn pushdown_requirement_to_children( let right_offset = smj.schema().fields.len() - smj.right().schema().fields.len(); let new_right_required = - shift_right_required(parent_required.ok_or_else(err)?, right_offset)?; - let new_right_required_expr = PhysicalSortRequirement::to_sort_exprs( - new_right_required.iter().cloned(), - ); + shift_right_required(parent_required, right_offset)?; + let new_right_required_expr = + PhysicalSortRequirement::to_sort_exprs(new_right_required); try_pushdown_requirements_to_join( smj, parent_required, @@ -260,34 +255,43 @@ fn pushdown_requirement_to_children( || plan.as_any().is::() || is_limit(plan) || plan.as_any().is::() - // Do not push-down through SortPreservingMergeExec when - // ordering requirement invalidates requirement of sort preserving merge exec. - || (is_sort_preserving_merge(plan) && !ordering_satisfy( - parent_required - .map(|req| PhysicalSortRequirement::to_sort_exprs(req.to_vec())) - .as_deref(), - plan.output_ordering(), - || plan.equivalence_properties(), - || plan.ordering_equivalence_properties(), - ) - ) { // If the current plan is a leaf node or can not maintain any of the input ordering, can not pushed down requirements. // For RepartitionExec, we always choose to not push down the sort requirements even the RepartitionExec(input_partition=1) could maintain input ordering. // Pushing down is not beneficial Ok(None) + } else if is_sort_preserving_merge(plan) { + let new_ordering = + PhysicalSortRequirement::to_sort_exprs(parent_required.to_vec()); + let mut spm_eqs = plan.equivalence_properties(); + // Sort preserving merge will have new ordering, one requirement above is pushed down to its below. + spm_eqs = spm_eqs.with_reorder(new_ordering); + // Do not push-down through SortPreservingMergeExec when + // ordering requirement invalidates requirement of sort preserving merge exec. + if !spm_eqs.ordering_satisfy(plan.output_ordering().unwrap_or(&[])) { + Ok(None) + } else { + // Can push-down through SortPreservingMergeExec, because parent requirement is finer + // than SortPreservingMergeExec output ordering. + let req = if parent_required.is_empty() { + None + } else { + Some(parent_required.to_vec()) + }; + Ok(Some(vec![req])) + } } else { Ok(Some( maintains_input_order - .iter() + .into_iter() .map(|flag| { - if *flag { - parent_required.map(|elem| elem.to_vec()) + if flag && !parent_required.is_empty() { + Some(parent_required.to_vec()) } else { None } }) - .collect::>(), + .collect(), )) } // TODO: Add support for Projection push down @@ -298,26 +302,26 @@ fn pushdown_requirement_to_children( /// If the the parent requirements are more specific, push down the parent requirements /// If they are not compatible, need to add Sort. fn determine_children_requirement( - parent_required: Option<&[PhysicalSortRequirement]>, - request_child: Option<&[PhysicalSortRequirement]>, + parent_required: LexRequirementRef, + request_child: LexRequirementRef, child_plan: Arc, ) -> RequirementsCompatibility { - if requirements_compatible( - request_child, - parent_required, - || child_plan.ordering_equivalence_properties(), - || child_plan.equivalence_properties(), - ) { + if child_plan + .equivalence_properties() + .requirements_compatible(request_child, parent_required) + { // request child requirements are more specific, no need to push down the parent requirements RequirementsCompatibility::Satisfy - } else if requirements_compatible( - parent_required, - request_child, - || child_plan.ordering_equivalence_properties(), - || child_plan.equivalence_properties(), - ) { + } else if child_plan + .equivalence_properties() + .requirements_compatible(parent_required, request_child) + { // parent requirements are more specific, adjust the request child requirements and push down the new requirements - let adjusted = parent_required.map(|r| r.to_vec()); + let adjusted = if parent_required.is_empty() { + None + } else { + Some(parent_required.to_vec()) + }; RequirementsCompatibility::Compatible(adjusted) } else { RequirementsCompatibility::NonCompatible @@ -325,7 +329,7 @@ fn determine_children_requirement( } fn try_pushdown_requirements_to_join( smj: &SortMergeJoinExec, - parent_required: Option<&[PhysicalSortRequirement]>, + parent_required: LexRequirementRef, sort_expr: Vec, push_side: JoinSide, ) -> Result>>>> { @@ -335,22 +339,22 @@ fn try_pushdown_requirements_to_join( JoinSide::Left => (sort_expr.as_slice(), right_ordering), JoinSide::Right => (left_ordering, sort_expr.as_slice()), }; + let join_type = smj.join_type(); + let probe_side = SortMergeJoinExec::probe_side(&join_type); let new_output_ordering = calculate_join_output_ordering( new_left_ordering, new_right_ordering, - smj.join_type(), + join_type, smj.on(), smj.left().schema().fields.len(), &smj.maintains_input_order(), - Some(SortMergeJoinExec::probe_side(&smj.join_type())), + Some(probe_side), ); - Ok(ordering_satisfy_requirement( - new_output_ordering.as_deref(), - parent_required, - || smj.equivalence_properties(), - || smj.ordering_equivalence_properties(), - ) - .then(|| { + let mut smj_eqs = smj.equivalence_properties(); + // smj will have this ordering when its input changes. + smj_eqs = smj_eqs.with_reorder(new_output_ordering.unwrap_or_default()); + let should_pushdown = smj_eqs.ordering_satisfy_requirement(parent_required); + Ok(should_pushdown.then(|| { let mut required_input_ordering = smj.required_input_ordering(); let new_req = Some(PhysicalSortRequirement::from_sort_exprs(&sort_expr)); match push_side { @@ -414,7 +418,7 @@ fn expr_source_sides( } fn shift_right_required( - parent_required: &[PhysicalSortRequirement], + parent_required: LexRequirementRef, left_columns_len: usize, ) -> Result> { let new_right_required: Vec = parent_required diff --git a/datafusion/core/src/physical_optimizer/utils.rs b/datafusion/core/src/physical_optimizer/utils.rs index 403af4b16ec7..530df374ca7c 100644 --- a/datafusion/core/src/physical_optimizer/utils.rs +++ b/datafusion/core/src/physical_optimizer/utils.rs @@ -30,8 +30,7 @@ use crate::physical_plan::union::UnionExec; use crate::physical_plan::windows::{BoundedWindowAggExec, WindowAggExec}; use crate::physical_plan::{displayable, ExecutionPlan}; -use datafusion_physical_expr::utils::ordering_satisfy_requirement; -use datafusion_physical_expr::PhysicalSortRequirement; +use datafusion_physical_expr::{LexRequirementRef, PhysicalSortRequirement}; /// This object implements a tree that we use while keeping track of paths /// leading to [`SortExec`]s. @@ -100,16 +99,14 @@ pub(crate) fn get_children_exectrees( /// given ordering requirements while preserving the original partitioning. pub fn add_sort_above( node: &mut Arc, - sort_requirement: &[PhysicalSortRequirement], + sort_requirement: LexRequirementRef, fetch: Option, ) { // If the ordering requirement is already satisfied, do not add a sort. - if !ordering_satisfy_requirement( - node.output_ordering(), - Some(sort_requirement), - || node.equivalence_properties(), - || node.ordering_equivalence_properties(), - ) { + if !node + .equivalence_properties() + .ordering_satisfy_requirement(sort_requirement) + { let sort_expr = PhysicalSortRequirement::to_sort_exprs(sort_requirement.to_vec()); let new_sort = SortExec::new(sort_expr, node.clone()).with_fetch(fetch); diff --git a/datafusion/core/tests/fuzz_cases/window_fuzz.rs b/datafusion/core/tests/fuzz_cases/window_fuzz.rs index 9f70321318fd..af96063ffb5f 100644 --- a/datafusion/core/tests/fuzz_cases/window_fuzz.rs +++ b/datafusion/core/tests/fuzz_cases/window_fuzz.rs @@ -22,7 +22,6 @@ use arrow::compute::{concat_batches, SortOptions}; use arrow::datatypes::SchemaRef; use arrow::record_batch::RecordBatch; use arrow::util::pretty::pretty_format_batches; - use datafusion::physical_plan::memory::MemoryExec; use datafusion::physical_plan::sorts::sort::SortExec; use datafusion::physical_plan::windows::{ diff --git a/datafusion/physical-expr/src/equivalence.rs b/datafusion/physical-expr/src/equivalence.rs index 4fce6854138d..04b0f2eedcdb 100644 --- a/datafusion/physical-expr/src/equivalence.rs +++ b/datafusion/physical-expr/src/equivalence.rs @@ -15,148 +15,200 @@ // specific language governing permissions and limitations // under the License. -use crate::expressions::{CastExpr, Column}; -use crate::utils::{collect_columns, merge_vectors}; +use std::collections::HashSet; +use std::hash::Hash; +use std::sync::Arc; + +use crate::expressions::Column; +use crate::physical_expr::{deduplicate_physical_exprs, have_common_entries}; +use crate::sort_properties::{ExprOrdering, SortProperties}; use crate::{ - LexOrdering, LexOrderingRef, LexOrderingReq, PhysicalExpr, PhysicalSortExpr, - PhysicalSortRequirement, + physical_exprs_contains, LexOrdering, LexOrderingRef, LexRequirement, + LexRequirementRef, PhysicalExpr, PhysicalSortExpr, PhysicalSortRequirement, }; use arrow::datatypes::SchemaRef; -use arrow_schema::Fields; - +use arrow_schema::SortOptions; use datafusion_common::tree_node::{Transformed, TreeNode}; -use datafusion_common::{JoinSide, JoinType}; -use itertools::izip; -use std::collections::{HashMap, HashSet}; -use std::hash::Hash; -use std::ops::Range; -use std::sync::Arc; +use datafusion_common::{JoinSide, JoinType, Result}; -/// Represents a collection of [`EquivalentClass`] (equivalences -/// between columns in relations) -/// -/// This is used to represent: -/// -/// 1. Equality conditions (like `A=B`), when `T` = [`Column`] +use indexmap::map::Entry; +use indexmap::IndexMap; + +/// An `EquivalenceClass` is a set of [`Arc`]s that are known +/// to have the same value for all tuples in a relation. These are generated by +/// equality predicates, typically equi-join conditions and equality conditions +/// in filters. +pub type EquivalenceClass = Vec>; + +/// Stores the mapping between source expressions and target expressions for a +/// projection. Indices in the vector corresponds to the indices after projection. +pub type ProjectionMapping = Vec<(Arc, Arc)>; + +/// An `EquivalenceGroup` is a collection of `EquivalenceClass`es where each +/// class represents a distinct equivalence class in a relation. #[derive(Debug, Clone)] -pub struct EquivalenceProperties { - classes: Vec>, - schema: SchemaRef, +pub struct EquivalenceGroup { + classes: Vec, } -impl EquivalenceProperties { - pub fn new(schema: SchemaRef) -> Self { - EquivalenceProperties { - classes: vec![], - schema, - } +impl EquivalenceGroup { + /// Creates an empty equivalence group. + fn empty() -> Self { + Self { classes: vec![] } } - /// return the set of equivalences - pub fn classes(&self) -> &[EquivalentClass] { - &self.classes + /// Creates an equivalence group from the given equivalence classes. + fn new(classes: Vec) -> Self { + let mut result = EquivalenceGroup { classes }; + result.remove_redundant_entries(); + result } - pub fn schema(&self) -> SchemaRef { - self.schema.clone() + /// Returns how many equivalence classes there are in this group. + fn len(&self) -> usize { + self.classes.len() } - /// Add the [`EquivalentClass`] from `iter` to this list - pub fn extend>>(&mut self, iter: I) { - for ec in iter { - self.classes.push(ec) - } + /// Checks whether this equivalence group is empty. + pub fn is_empty(&self) -> bool { + self.len() == 0 } - /// Adds new equal conditions into the EquivalenceProperties. New equal - /// conditions usually come from equality predicates in a join/filter. - pub fn add_equal_conditions(&mut self, new_conditions: (&Column, &Column)) { - let mut idx1: Option = None; - let mut idx2: Option = None; - for (idx, class) in self.classes.iter_mut().enumerate() { - let contains_first = class.contains(new_conditions.0); - let contains_second = class.contains(new_conditions.1); - match (contains_first, contains_second) { - (true, false) => { - class.insert(new_conditions.1.clone()); - idx1 = Some(idx); - } - (false, true) => { - class.insert(new_conditions.0.clone()); - idx2 = Some(idx); - } - (true, true) => { - idx1 = Some(idx); - idx2 = Some(idx); - break; - } - (false, false) => {} + /// Returns an iterator over the equivalence classes in this group. + fn iter(&self) -> impl Iterator { + self.classes.iter() + } + + /// Adds the equality `left` = `right` to this equivalence group. + /// New equality conditions often arise after steps like `Filter(a = b)`, + /// `Alias(a, a as b)` etc. + fn add_equal_conditions( + &mut self, + left: &Arc, + right: &Arc, + ) { + let mut first_class = None; + let mut second_class = None; + for (idx, cls) in self.classes.iter().enumerate() { + if physical_exprs_contains(cls, left) { + first_class = Some(idx); + } + if physical_exprs_contains(cls, right) { + second_class = Some(idx); } } - - match (idx1, idx2) { - (Some(idx_1), Some(idx_2)) if idx_1 != idx_2 => { - // need to merge the two existing EquivalentClasses - let second_eq_class = self.classes.get(idx_2).unwrap().clone(); - let first_eq_class = self.classes.get_mut(idx_1).unwrap(); - for prop in second_eq_class.iter() { - if !first_eq_class.contains(prop) { - first_eq_class.insert(prop.clone()); + match (first_class, second_class) { + (Some(mut first_idx), Some(mut second_idx)) => { + // If the given left and right sides belong to different classes, + // we should unify/bridge these classes. + if first_idx != second_idx { + // By convention make sure second_idx is larger than first_idx. + if first_idx > second_idx { + (first_idx, second_idx) = (second_idx, first_idx); } + // Remove second_idx from self.classes then merge its values with class at first_idx. + // Convention above makes sure that first_idx is still valid after second_idx removal. + let other_class = self.classes.swap_remove(second_idx); + self.classes[first_idx].extend(other_class); } - self.classes.remove(idx_2); + } + (Some(group_idx), None) => { + // Right side is new, extend left side's class: + self.classes[group_idx].push(right.clone()); + } + (None, Some(group_idx)) => { + // Left side is new, extend right side's class: + self.classes[group_idx].push(left.clone()); } (None, None) => { - // adding new pairs - self.classes.push(EquivalentClass::::new( - new_conditions.0.clone(), - vec![new_conditions.1.clone()], - )); + // None of the expressions is among existing classes. + // Create a new equivalence class and extend the group. + self.classes.push(vec![left.clone(), right.clone()]); } - _ => {} } } - /// Normalizes physical expression according to `EquivalentClass`es inside `self.classes`. - /// expression is replaced with `EquivalentClass::head` expression if it is among `EquivalentClass::others`. + /// Removes redundant entries from this group. + fn remove_redundant_entries(&mut self) { + // Remove duplicate entries from each equivalence class: + self.classes.retain_mut(|cls| { + // Keep groups that have at least two entries as singleton class is + // meaningless (i.e. it contains no non-trivial information): + deduplicate_physical_exprs(cls); + cls.len() > 1 + }); + // Unify/bridge groups that have common expressions: + self.bridge_classes() + } + + /// This utility function unifies/bridges classes that have common expressions. + /// For example, assume that we have [`EquivalenceClass`]es `[a, b]` and `[b, c]`. + /// Since both classes contain `b`, columns `a`, `b` and `c` are actually all + /// equal and belong to one class. This utility converts merges such classes. + fn bridge_classes(&mut self) { + let mut idx = 0; + while idx < self.classes.len() { + let mut next_idx = idx + 1; + let start_size = self.classes[idx].len(); + while next_idx < self.classes.len() { + if have_common_entries(&self.classes[idx], &self.classes[next_idx]) { + let extension = self.classes.swap_remove(next_idx); + self.classes[idx].extend(extension); + } else { + next_idx += 1; + } + } + if self.classes[idx].len() > start_size { + deduplicate_physical_exprs(&mut self.classes[idx]); + if self.classes[idx].len() > start_size { + continue; + } + } + idx += 1; + } + } + + /// Extends this equivalence group with the `other` equivalence group. + fn extend(&mut self, other: Self) { + self.classes.extend(other.classes); + self.remove_redundant_entries(); + } + + /// Normalizes the given physical expression according to this group. + /// The expression is replaced with the first expression in the equivalence + /// class it matches with (if any). pub fn normalize_expr(&self, expr: Arc) -> Arc { expr.clone() .transform(&|expr| { - let normalized_form = - expr.as_any().downcast_ref::().and_then(|column| { - for class in &self.classes { - if class.contains(column) { - return Some(Arc::new(class.head().clone()) as _); - } - } - None - }); - Ok(if let Some(normalized_form) = normalized_form { - Transformed::Yes(normalized_form) - } else { - Transformed::No(expr) - }) + for cls in self.iter() { + if physical_exprs_contains(cls, &expr) { + return Ok(Transformed::Yes(cls[0].clone())); + } + } + Ok(Transformed::No(expr)) }) .unwrap_or(expr) } - /// This function applies the \[`normalize_expr`] - /// function for all expression in `exprs` and returns a vector of - /// normalized physical expressions. - pub fn normalize_exprs( + /// Normalizes the given sort expression according to this group. + /// The underlying physical expression is replaced with the first expression + /// in the equivalence class it matches with (if any). If the underlying + /// expression does not belong to any equivalence class in this group, returns + /// the sort expression as is. + pub fn normalize_sort_expr( &self, - exprs: &[Arc], - ) -> Vec> { - exprs - .iter() - .map(|expr| self.normalize_expr(expr.clone())) - .collect::>() + mut sort_expr: PhysicalSortExpr, + ) -> PhysicalSortExpr { + sort_expr.expr = self.normalize_expr(sort_expr.expr); + sort_expr } - /// This function normalizes `sort_requirement` according to `EquivalenceClasses` in the `self`. - /// If the given sort requirement doesn't belong to equivalence set inside - /// `self`, it returns `sort_requirement` as is. + /// Normalizes the given sort requirement according to this group. + /// The underlying physical expression is replaced with the first expression + /// in the equivalence class it matches with (if any). If the underlying + /// expression does not belong to any equivalence class in this group, returns + /// the given sort requirement as is. pub fn normalize_sort_requirement( &self, mut sort_requirement: PhysicalSortRequirement, @@ -165,1069 +217,1145 @@ impl EquivalenceProperties { sort_requirement } - /// This function applies the \[`normalize_sort_requirement`] - /// function for all sort requirements in `sort_reqs` and returns a vector of - /// normalized sort expressions. - pub fn normalize_sort_requirements( + /// This function applies the `normalize_expr` function for all expressions + /// in `exprs` and returns the corresponding normalized physical expressions. + pub fn normalize_exprs( &self, - sort_reqs: &[PhysicalSortRequirement], - ) -> Vec { - let normalized_sort_reqs = sort_reqs - .iter() - .map(|sort_req| self.normalize_sort_requirement(sort_req.clone())) - .collect::>(); - collapse_vec(normalized_sort_reqs) + exprs: impl IntoIterator>, + ) -> Vec> { + exprs + .into_iter() + .map(|expr| self.normalize_expr(expr)) + .collect() } - /// Similar to the \[`normalize_sort_requirements`] this function normalizes - /// sort expressions in `sort_exprs` and returns a vector of - /// normalized sort expressions. - pub fn normalize_sort_exprs( - &self, - sort_exprs: &[PhysicalSortExpr], - ) -> Vec { - let sort_requirements = - PhysicalSortRequirement::from_sort_exprs(sort_exprs.iter()); - let normalized_sort_requirement = - self.normalize_sort_requirements(&sort_requirements); - PhysicalSortRequirement::to_sort_exprs(normalized_sort_requirement) + /// This function applies the `normalize_sort_expr` function for all sort + /// expressions in `sort_exprs` and returns the corresponding normalized + /// sort expressions. + pub fn normalize_sort_exprs(&self, sort_exprs: LexOrderingRef) -> LexOrdering { + // Convert sort expressions to sort requirements: + let sort_reqs = PhysicalSortRequirement::from_sort_exprs(sort_exprs.iter()); + // Normalize the requirements: + let normalized_sort_reqs = self.normalize_sort_requirements(&sort_reqs); + // Convert sort requirements back to sort expressions: + PhysicalSortRequirement::to_sort_exprs(normalized_sort_reqs) } -} - -/// `OrderingEquivalenceProperties` keeps track of columns that describe the -/// global ordering of the schema. These columns are not necessarily same; e.g. -/// ```text -/// ┌-------┐ -/// | a | b | -/// |---|---| -/// | 1 | 9 | -/// | 2 | 8 | -/// | 3 | 7 | -/// | 5 | 5 | -/// └---┴---┘ -/// ``` -/// where both `a ASC` and `b DESC` can describe the table ordering. With -/// `OrderingEquivalenceProperties`, we can keep track of these equivalences -/// and treat `a ASC` and `b DESC` as the same ordering requirement. -#[derive(Debug, Clone)] -pub struct OrderingEquivalenceProperties { - oeq_class: Option, - /// Keeps track of expressions that have constant value. - constants: Vec>, - schema: SchemaRef, -} -impl OrderingEquivalenceProperties { - /// Create an empty `OrderingEquivalenceProperties` - pub fn new(schema: SchemaRef) -> Self { - Self { - oeq_class: None, - constants: vec![], - schema, - } + /// This function applies the `normalize_sort_requirement` function for all + /// requirements in `sort_reqs` and returns the corresponding normalized + /// sort requirements. + pub fn normalize_sort_requirements( + &self, + sort_reqs: LexRequirementRef, + ) -> LexRequirement { + collapse_lex_req( + sort_reqs + .iter() + .map(|sort_req| self.normalize_sort_requirement(sort_req.clone())) + .collect(), + ) } - /// Extends `OrderingEquivalenceProperties` by adding ordering inside the `other` - /// to the `self.oeq_class`. - pub fn extend(&mut self, other: Option) { - if let Some(other) = other { - if let Some(class) = &mut self.oeq_class { - class.others.insert(other.head); - class.others.extend(other.others); - } else { - self.oeq_class = Some(other); + /// Projects `expr` according to the given projection mapping. + /// If the resulting expression is invalid after projection, returns `None`. + fn project_expr( + &self, + mapping: &ProjectionMapping, + expr: &Arc, + ) -> Option> { + let children = expr.children(); + if children.is_empty() { + for (source, target) in mapping.iter() { + // If we match the source, or an equivalent expression to source, + // then we can project. For example, if we have the mapping + // (a as a1, a + c) and the equivalence class (a, b), expression + // b also projects to a1. + if source.eq(expr) + || self + .get_equivalence_class(source) + .map_or(false, |group| physical_exprs_contains(group, expr)) + { + return Some(target.clone()); + } } } + // Project a non-leaf expression by projecting its children. + else if let Some(children) = children + .into_iter() + .map(|child| self.project_expr(mapping, &child)) + .collect::>>() + { + return Some(expr.clone().with_new_children(children).unwrap()); + } + // Arriving here implies the expression was invalid after projection. + None } - pub fn oeq_class(&self) -> Option<&OrderingEquivalentClass> { - self.oeq_class.as_ref() - } - - /// Adds new equal conditions into the EquivalenceProperties. New equal - /// conditions usually come from equality predicates in a join/filter. - pub fn add_equal_conditions(&mut self, new_conditions: (&LexOrdering, &LexOrdering)) { - if let Some(class) = &mut self.oeq_class { - class.insert(new_conditions.0.clone()); - class.insert(new_conditions.1.clone()); - } else { - let head = new_conditions.0.clone(); - let others = vec![new_conditions.1.clone()]; - self.oeq_class = Some(OrderingEquivalentClass::new(head, others)) - } + /// Projects `ordering` according to the given projection mapping. + /// If the resulting ordering is invalid after projection, returns `None`. + fn project_ordering( + &self, + mapping: &ProjectionMapping, + ordering: LexOrderingRef, + ) -> Option { + // If any sort expression is invalid after projection, rest of the + // ordering shouldn't be projected either. For example, if input ordering + // is [a ASC, b ASC, c ASC], and column b is not valid after projection, + // the result should be [a ASC], not [a ASC, c ASC], even if column c is + // valid after projection. + let result = ordering + .iter() + .map_while(|sort_expr| { + self.project_expr(mapping, &sort_expr.expr) + .map(|expr| PhysicalSortExpr { + expr, + options: sort_expr.options, + }) + }) + .collect::>(); + (!result.is_empty()).then_some(result) } - /// Add physical expression that have constant value to the `self.constants` - pub fn with_constants(mut self, constants: Vec>) -> Self { - constants.into_iter().for_each(|constant| { - if !physical_exprs_contains(&self.constants, &constant) { - self.constants.push(constant); - } + /// Projects this equivalence group according to the given projection mapping. + pub fn project(&self, mapping: &ProjectionMapping) -> Self { + let projected_classes = self.iter().filter_map(|cls| { + let new_class = cls + .iter() + .filter_map(|expr| self.project_expr(mapping, expr)) + .collect::>(); + (new_class.len() > 1).then_some(new_class) }); - self + // TODO: Convert the algorithm below to a version that uses `HashMap`. + // once `Arc` can be stored in `HashMap`. + // See issue: https://github.com/apache/arrow-datafusion/issues/8027 + let mut new_classes = vec![]; + for (source, target) in mapping { + if new_classes.is_empty() { + new_classes.push((source, vec![target.clone()])); + } + if let Some((_, values)) = + new_classes.iter_mut().find(|(key, _)| key.eq(source)) + { + if !physical_exprs_contains(values, target) { + values.push(target.clone()); + } + } + } + // Only add equivalence classes with at least two members as singleton + // equivalence classes are meaningless. + let new_classes = new_classes + .into_iter() + .filter_map(|(_, values)| (values.len() > 1).then_some(values)); + let classes = projected_classes.chain(new_classes).collect(); + Self::new(classes) } - pub fn schema(&self) -> SchemaRef { - self.schema.clone() + /// Returns the equivalence class that contains `expr`. + /// If none of the equivalence classes contains `expr`, returns `None`. + fn get_equivalence_class( + &self, + expr: &Arc, + ) -> Option<&[Arc]> { + self.iter() + .map(|cls| cls.as_slice()) + .find(|cls| physical_exprs_contains(cls, expr)) } - /// This function normalizes `sort_reqs` by - /// - removing expressions that have constant value from requirement - /// - replacing sections that are in the `self.oeq_class.others` with `self.oeq_class.head` - /// - removing sections that satisfies global ordering that are in the post fix of requirement - pub fn normalize_sort_requirements( + /// Combine equivalence groups of the given join children. + pub fn join( &self, - sort_reqs: &[PhysicalSortRequirement], - ) -> Vec { - let normalized_sort_reqs = - prune_sort_reqs_with_constants(sort_reqs, &self.constants); - let mut normalized_sort_reqs = collapse_lex_req(normalized_sort_reqs); - if let Some(oeq_class) = &self.oeq_class { - for item in oeq_class.others() { - let item = PhysicalSortRequirement::from_sort_exprs(item); - let item = prune_sort_reqs_with_constants(&item, &self.constants); - let ranges = get_compatible_ranges(&normalized_sort_reqs, &item); - let mut offset: i64 = 0; - for Range { start, end } in ranges { - let head = PhysicalSortRequirement::from_sort_exprs(oeq_class.head()); - let mut head = prune_sort_reqs_with_constants(&head, &self.constants); - let updated_start = (start as i64 + offset) as usize; - let updated_end = (end as i64 + offset) as usize; - let range = end - start; - offset += head.len() as i64 - range as i64; - let all_none = normalized_sort_reqs[updated_start..updated_end] - .iter() - .all(|req| req.options.is_none()); - if all_none { - for req in head.iter_mut() { - req.options = None; - } + right_equivalences: &Self, + join_type: &JoinType, + left_size: usize, + on: &[(Column, Column)], + ) -> Self { + match join_type { + JoinType::Inner | JoinType::Left | JoinType::Full | JoinType::Right => { + let mut result = Self::new( + self.iter() + .cloned() + .chain(right_equivalences.iter().map(|item| { + item.iter() + .cloned() + .map(|expr| add_offset_to_expr(expr, left_size)) + .collect() + })) + .collect(), + ); + // In we have an inner join, expressions in the "on" condition + // are equal in the resulting table. + if join_type == &JoinType::Inner { + for (lhs, rhs) in on.iter() { + let index = rhs.index() + left_size; + let new_lhs = Arc::new(lhs.clone()) as _; + let new_rhs = Arc::new(Column::new(rhs.name(), index)) as _; + result.add_equal_conditions(&new_lhs, &new_rhs); } - normalized_sort_reqs.splice(updated_start..updated_end, head); } + result } - normalized_sort_reqs = simplify_lex_req(normalized_sort_reqs, oeq_class); + JoinType::LeftSemi | JoinType::LeftAnti => self.clone(), + JoinType::RightSemi | JoinType::RightAnti => right_equivalences.clone(), } - collapse_lex_req(normalized_sort_reqs) } +} - /// Checks whether `leading_ordering` is contained in any of the ordering - /// equivalence classes. - pub fn satisfies_leading_ordering( - &self, - leading_ordering: &PhysicalSortExpr, - ) -> bool { - if let Some(oeq_class) = &self.oeq_class { - for ordering in oeq_class - .others - .iter() - .chain(std::iter::once(&oeq_class.head)) - { - if ordering[0].eq(leading_ordering) { - return true; - } - } +/// This function constructs a duplicate-free `LexOrderingReq` by filtering out +/// duplicate entries that have same physical expression inside. For example, +/// `vec![a Some(Asc), a Some(Desc)]` collapses to `vec![a Some(Asc)]`. +pub fn collapse_lex_req(input: LexRequirement) -> LexRequirement { + let mut output = Vec::::new(); + for item in input { + if !output.iter().any(|req| req.expr.eq(&item.expr)) { + output.push(item); } - false } + output } -/// EquivalentClass is a set of [`Column`]s or [`PhysicalSortExpr`]s that are known -/// to have the same value in all tuples in a relation. `EquivalentClass` -/// is generated by equality predicates, typically equijoin conditions and equality -/// conditions in filters. `EquivalentClass` is generated by the -/// `ROW_NUMBER` window function. -#[derive(Debug, Clone)] -pub struct EquivalentClass { - /// First element in the EquivalentClass - head: T, - /// Other equal columns - others: HashSet, +/// An `OrderingEquivalenceClass` object keeps track of different alternative +/// orderings than can describe a schema. For example, consider the following table: +/// +/// ```text +/// |a|b|c|d| +/// |1|4|3|1| +/// |2|3|3|2| +/// |3|1|2|2| +/// |3|2|1|3| +/// ``` +/// +/// Here, both `vec![a ASC, b ASC]` and `vec![c DESC, d ASC]` describe the table +/// ordering. In this case, we say that these orderings are equivalent. +#[derive(Debug, Clone, Eq, PartialEq, Hash)] +pub struct OrderingEquivalenceClass { + orderings: Vec, } -impl EquivalentClass { - pub fn new(head: T, others: Vec) -> EquivalentClass { - EquivalentClass { - head, - others: HashSet::from_iter(others), - } +impl OrderingEquivalenceClass { + /// Creates new empty ordering equivalence class. + fn empty() -> Self { + Self { orderings: vec![] } } - pub fn head(&self) -> &T { - &self.head + /// Clears (empties) this ordering equivalence class. + pub fn clear(&mut self) { + self.orderings.clear(); } - pub fn others(&self) -> &HashSet { - &self.others + /// Creates new ordering equivalence class from the given orderings. + pub fn new(orderings: Vec) -> Self { + let mut result = Self { orderings }; + result.remove_redundant_entries(); + result } - pub fn contains(&self, col: &T) -> bool { - self.head == *col || self.others.contains(col) + /// Checks whether `ordering` is a member of this equivalence class. + pub fn contains(&self, ordering: &LexOrdering) -> bool { + self.orderings.contains(ordering) } - pub fn insert(&mut self, col: T) -> bool { - self.head != col && self.others.insert(col) + /// Adds `ordering` to this equivalence class. + #[allow(dead_code)] + fn push(&mut self, ordering: LexOrdering) { + self.orderings.push(ordering); + // Make sure that there are no redundant orderings: + self.remove_redundant_entries(); } - pub fn remove(&mut self, col: &T) -> bool { - let removed = self.others.remove(col); - // If we are removing the head, adjust others so that its first entry becomes the new head. - if !removed && *col == self.head { - if let Some(col) = self.others.iter().next().cloned() { - let removed = self.others.remove(&col); - self.head = col; - removed - } else { - // We don't allow empty equivalence classes, reject removal if one tries removing - // the only element in an equivalence class. - false - } - } else { - removed - } + /// Checks whether this ordering equivalence class is empty. + pub fn is_empty(&self) -> bool { + self.len() == 0 } - pub fn iter(&self) -> impl Iterator { - std::iter::once(&self.head).chain(self.others.iter()) + /// Returns an iterator over the equivalent orderings in this class. + pub fn iter(&self) -> impl Iterator { + self.orderings.iter() } + /// Returns how many equivalent orderings there are in this class. pub fn len(&self) -> usize { - self.others.len() + 1 - } - - pub fn is_empty(&self) -> bool { - self.len() == 0 + self.orderings.len() } -} -/// `LexOrdering` stores the lexicographical ordering for a schema. -/// OrderingEquivalentClass keeps track of different alternative orderings than can -/// describe the schema. -/// For instance, for the table below -/// |a|b|c|d| -/// |1|4|3|1| -/// |2|3|3|2| -/// |3|1|2|2| -/// |3|2|1|3| -/// both `vec![a ASC, b ASC]` and `vec![c DESC, d ASC]` describe the ordering of the table. -/// For this case, we say that `vec![a ASC, b ASC]`, and `vec![c DESC, d ASC]` are ordering equivalent. -pub type OrderingEquivalentClass = EquivalentClass; - -/// Update each expression in `ordering` with alias expressions. Assume -/// `ordering` is `a ASC, b ASC` and `c` is alias of `b`. Then, the result -/// will be `a ASC, c ASC`. -fn update_with_alias( - mut ordering: LexOrdering, - oeq_alias_map: &[(Column, Column)], -) -> LexOrdering { - for (source_col, target_col) in oeq_alias_map { - let source_col: Arc = Arc::new(source_col.clone()); - // Replace invalidated columns with its alias in the ordering expression. - let target_col: Arc = Arc::new(target_col.clone()); - for item in ordering.iter_mut() { - if item.expr.eq(&source_col) { - // Change the corresponding entry with alias expression - item.expr = target_col.clone(); - } - } + /// Extend this ordering equivalence class with the `other` class. + pub fn extend(&mut self, other: Self) { + self.orderings.extend(other.orderings); + // Make sure that there are no redundant orderings: + self.remove_redundant_entries(); } - ordering -} -impl OrderingEquivalentClass { - /// This function updates ordering equivalences with alias information. - /// For instance, assume columns `a` and `b` are aliases (a as b), and - /// orderings `a ASC` and `c DESC` are equivalent. Here, we replace column - /// `a` with `b` in ordering equivalence expressions. After this function, - /// `a ASC`, `c DESC` will be converted to the `b ASC`, `c DESC`. - fn update_with_aliases( + /// Adds new orderings into this ordering equivalence class. + pub fn add_new_orderings( &mut self, - oeq_alias_map: &[(Column, Column)], - fields: &Fields, + orderings: impl IntoIterator, ) { - let is_head_invalid = self.head.iter().any(|sort_expr| { - collect_columns(&sort_expr.expr) - .iter() - .any(|col| is_column_invalid_in_new_schema(col, fields)) - }); - // If head is invalidated, update head with alias expressions - if is_head_invalid { - self.head = update_with_alias(self.head.clone(), oeq_alias_map); - } else { - let new_oeq_expr = update_with_alias(self.head.clone(), oeq_alias_map); - self.insert(new_oeq_expr); - } - for ordering in self.others.clone().into_iter() { - self.insert(update_with_alias(ordering, oeq_alias_map)); + self.orderings.extend(orderings); + // Make sure that there are no redundant orderings: + self.remove_redundant_entries(); + } + + /// Removes redundant orderings from this equivalence class. + /// For instance, If we already have the ordering [a ASC, b ASC, c DESC], + /// then there is no need to keep ordering [a ASC, b ASC] in the state. + fn remove_redundant_entries(&mut self) { + let mut idx = 0; + while idx < self.orderings.len() { + let mut removal = false; + for (ordering_idx, ordering) in self.orderings[0..idx].iter().enumerate() { + if let Some(right_finer) = finer_side(ordering, &self.orderings[idx]) { + if right_finer { + self.orderings.swap(ordering_idx, idx); + } + removal = true; + break; + } + } + if removal { + self.orderings.swap_remove(idx); + } else { + idx += 1; + } } } - /// Adds `offset` value to the index of each expression inside `self.head` and `self.others`. - pub fn add_offset(&self, offset: usize) -> OrderingEquivalentClass { - let head = add_offset_to_lex_ordering(self.head(), offset); - let others = self - .others() - .iter() - .map(|ordering| add_offset_to_lex_ordering(ordering, offset)) - .collect::>(); - OrderingEquivalentClass::new(head, others) + /// Gets the first ordering entry in this ordering equivalence class. + /// This is one of the many valid orderings (if there are multiple). + pub fn output_ordering(&self) -> Option { + self.orderings.first().cloned() } - /// This function normalizes `OrderingEquivalenceProperties` according to `eq_properties`. - /// More explicitly, it makes sure that expressions in `oeq_class` are head entries - /// in `eq_properties`, replacing any non-head entries with head entries if necessary. - pub fn normalize_with_equivalence_properties( - &self, - eq_properties: &EquivalenceProperties, - ) -> OrderingEquivalentClass { - let head = eq_properties.normalize_sort_exprs(self.head()); - - let others = self - .others() - .iter() - .map(|other| eq_properties.normalize_sort_exprs(other)) - .collect(); + // Append orderings in `other` to all existing orderings in this equivalence + // class. + pub fn join_suffix(mut self, other: &Self) -> Self { + for ordering in other.iter() { + for idx in 0..self.orderings.len() { + self.orderings[idx].extend(ordering.iter().cloned()); + } + } + self + } - EquivalentClass::new(head, others) + /// Adds `offset` value to the index of each expression inside this + /// ordering equivalence class. + pub fn add_offset(&mut self, offset: usize) { + for ordering in self.orderings.iter_mut() { + for sort_expr in ordering { + sort_expr.expr = add_offset_to_expr(sort_expr.expr.clone(), offset); + } + } } - /// Prefix with existing ordering. - pub fn prefix_ordering_equivalent_class_with_existing_ordering( - &self, - existing_ordering: &[PhysicalSortExpr], - eq_properties: &EquivalenceProperties, - ) -> OrderingEquivalentClass { - let existing_ordering = eq_properties.normalize_sort_exprs(existing_ordering); - let normalized_head = eq_properties.normalize_sort_exprs(self.head()); - let updated_head = merge_vectors(&existing_ordering, &normalized_head); - let updated_others = self - .others() - .iter() - .map(|ordering| { - let normalized_ordering = eq_properties.normalize_sort_exprs(ordering); - merge_vectors(&existing_ordering, &normalized_ordering) - }) - .collect(); - OrderingEquivalentClass::new(updated_head, updated_others) + /// Gets sort options associated with this expression if it is a leading + /// ordering expression. Otherwise, returns `None`. + fn get_options(&self, expr: &Arc) -> Option { + for ordering in self.iter() { + let leading_ordering = &ordering[0]; + if leading_ordering.expr.eq(expr) { + return Some(leading_ordering.options); + } + } + None } } -/// This is a builder object facilitating incremental construction -/// for ordering equivalences. -pub struct OrderingEquivalenceBuilder { - eq_properties: EquivalenceProperties, - ordering_eq_properties: OrderingEquivalenceProperties, - existing_ordering: Vec, +/// Adds the `offset` value to `Column` indices inside `expr`. This function is +/// generally used during the update of the right table schema in join operations. +pub fn add_offset_to_expr( + expr: Arc, + offset: usize, +) -> Arc { + expr.transform_down(&|e| match e.as_any().downcast_ref::() { + Some(col) => Ok(Transformed::Yes(Arc::new(Column::new( + col.name(), + offset + col.index(), + )))), + None => Ok(Transformed::No(e)), + }) + .unwrap() + // Note that we can safely unwrap here since our transform always returns + // an `Ok` value. +} + +/// Returns `true` if the ordering `rhs` is strictly finer than the ordering `rhs`, +/// `false` if the ordering `lhs` is at least as fine as the ordering `lhs`, and +/// `None` otherwise (i.e. when given orderings are incomparable). +fn finer_side(lhs: LexOrderingRef, rhs: LexOrderingRef) -> Option { + let all_equal = lhs.iter().zip(rhs.iter()).all(|(lhs, rhs)| lhs.eq(rhs)); + all_equal.then_some(lhs.len() < rhs.len()) +} + +/// A `EquivalenceProperties` object stores useful information related to a schema. +/// Currently, it keeps track of: +/// - Equivalent expressions, e.g expressions that have same value. +/// - Valid sort expressions (orderings) for the schema. +/// - Constants expressions (e.g expressions that are known to have constant values). +/// +/// Consider table below: +/// +/// ```text +/// ┌-------┐ +/// | a | b | +/// |---|---| +/// | 1 | 9 | +/// | 2 | 8 | +/// | 3 | 7 | +/// | 5 | 5 | +/// └---┴---┘ +/// ``` +/// +/// where both `a ASC` and `b DESC` can describe the table ordering. With +/// `EquivalenceProperties`, we can keep track of these different valid sort +/// expressions and treat `a ASC` and `b DESC` on an equal footing. +/// +/// Similarly, consider the table below: +/// +/// ```text +/// ┌-------┐ +/// | a | b | +/// |---|---| +/// | 1 | 1 | +/// | 2 | 2 | +/// | 3 | 3 | +/// | 5 | 5 | +/// └---┴---┘ +/// ``` +/// +/// where columns `a` and `b` always have the same value. We keep track of such +/// equivalences inside this object. With this information, we can optimize +/// things like partitioning. For example, if the partition requirement is +/// `Hash(a)` and output partitioning is `Hash(b)`, then we can deduce that +/// the existing partitioning satisfies the requirement. +#[derive(Debug, Clone)] +pub struct EquivalenceProperties { + /// Collection of equivalence classes that store expressions with the same + /// value. + eq_group: EquivalenceGroup, + /// Equivalent sort expressions for this table. + oeq_class: OrderingEquivalenceClass, + /// Expressions whose values are constant throughout the table. + /// TODO: We do not need to track constants separately, they can be tracked + /// inside `eq_groups` as `Literal` expressions. + constants: Vec>, + /// Schema associated with this object. schema: SchemaRef, } -impl OrderingEquivalenceBuilder { +impl EquivalenceProperties { + /// Creates an empty `EquivalenceProperties` object. pub fn new(schema: SchemaRef) -> Self { - let eq_properties = EquivalenceProperties::new(schema.clone()); - let ordering_eq_properties = OrderingEquivalenceProperties::new(schema.clone()); Self { - eq_properties, - ordering_eq_properties, - existing_ordering: vec![], + eq_group: EquivalenceGroup::empty(), + oeq_class: OrderingEquivalenceClass::empty(), + constants: vec![], schema, } } - pub fn extend( - mut self, - new_ordering_eq_properties: OrderingEquivalenceProperties, - ) -> Self { - self.ordering_eq_properties - .extend(new_ordering_eq_properties.oeq_class().cloned()); - self + /// Creates a new `EquivalenceProperties` object with the given orderings. + pub fn new_with_orderings(schema: SchemaRef, orderings: &[LexOrdering]) -> Self { + Self { + eq_group: EquivalenceGroup::empty(), + oeq_class: OrderingEquivalenceClass::new(orderings.to_vec()), + constants: vec![], + schema, + } } - pub fn with_existing_ordering( - mut self, - existing_ordering: Option>, - ) -> Self { - if let Some(existing_ordering) = existing_ordering { - self.existing_ordering = existing_ordering; - } - self + /// Returns the associated schema. + pub fn schema(&self) -> &SchemaRef { + &self.schema } - pub fn with_equivalences(mut self, new_eq_properties: EquivalenceProperties) -> Self { - self.eq_properties = new_eq_properties; - self + /// Returns a reference to the ordering equivalence class within. + pub fn oeq_class(&self) -> &OrderingEquivalenceClass { + &self.oeq_class } - pub fn add_equal_conditions( - &mut self, - new_equivalent_ordering: Vec, - ) { - let mut normalized_out_ordering = vec![]; - for item in &self.existing_ordering { - // To account for ordering equivalences, first normalize the expression: - let normalized = self.eq_properties.normalize_expr(item.expr.clone()); - normalized_out_ordering.push(PhysicalSortExpr { - expr: normalized, - options: item.options, - }); - } - // If there is an existing ordering, add new ordering as an equivalence: - if !normalized_out_ordering.is_empty() { - self.ordering_eq_properties.add_equal_conditions(( - &normalized_out_ordering, - &new_equivalent_ordering, - )); - } + /// Returns a reference to the equivalence group within. + pub fn eq_group(&self) -> &EquivalenceGroup { + &self.eq_group } - /// Return a reference to the schema with which this builder was constructed with - pub fn schema(&self) -> &SchemaRef { - &self.schema + /// Returns the normalized version of the ordering equivalence class within. + /// Normalization removes constants and duplicates as well as standardizing + /// expressions according to the equivalence group within. + pub fn normalized_oeq_class(&self) -> OrderingEquivalenceClass { + OrderingEquivalenceClass::new( + self.oeq_class + .iter() + .map(|ordering| self.normalize_sort_exprs(ordering)) + .collect(), + ) } - /// Return a reference to the existing ordering - pub fn existing_ordering(&self) -> &LexOrdering { - &self.existing_ordering + /// Extends this `EquivalenceProperties` with the `other` object. + pub fn extend(mut self, other: Self) -> Self { + self.eq_group.extend(other.eq_group); + self.oeq_class.extend(other.oeq_class); + self.add_constants(other.constants) } - pub fn build(self) -> OrderingEquivalenceProperties { - self.ordering_eq_properties + /// Clears (empties) the ordering equivalence class within this object. + /// Call this method when existing orderings are invalidated. + pub fn clear_orderings(&mut self) { + self.oeq_class.clear(); } -} - -/// Checks whether column is still valid after projection. -fn is_column_invalid_in_new_schema(column: &Column, fields: &Fields) -> bool { - let idx = column.index(); - idx >= fields.len() || fields[idx].name() != column.name() -} -/// Gets first aliased version of `col` found in `alias_map`. -fn get_alias_column( - col: &Column, - alias_map: &HashMap>, -) -> Option { - alias_map - .iter() - .find_map(|(column, columns)| column.eq(col).then(|| columns[0].clone())) -} - -/// This function applies the given projection to the given equivalence -/// properties to compute the resulting (projected) equivalence properties; e.g. -/// 1) Adding an alias, which can introduce additional equivalence properties, -/// as in Projection(a, a as a1, a as a2). -/// 2) Truncate the [`EquivalentClass`]es that are not in the output schema. -pub fn project_equivalence_properties( - input_eq: EquivalenceProperties, - alias_map: &HashMap>, - output_eq: &mut EquivalenceProperties, -) { - // Get schema and fields of projection output - let schema = output_eq.schema(); - let fields = schema.fields(); - - let mut eq_classes = input_eq.classes().to_vec(); - for (column, columns) in alias_map { - let mut find_match = false; - for class in eq_classes.iter_mut() { - // If `self.head` is invalidated in the new schema, update head - // with this change `self.head` is not randomly assigned by one of the entries from `self.others` - if is_column_invalid_in_new_schema(&class.head, fields) { - if let Some(alias_col) = get_alias_column(&class.head, alias_map) { - class.head = alias_col; - } - } - if class.contains(column) { - for col in columns { - class.insert(col.clone()); - } - find_match = true; - break; - } - } - if !find_match { - eq_classes.push(EquivalentClass::new(column.clone(), columns.clone())); - } + /// Extends this `EquivalenceProperties` by adding the orderings inside the + /// ordering equivalence class `other`. + pub fn add_ordering_equivalence_class(&mut self, other: OrderingEquivalenceClass) { + self.oeq_class.extend(other); } - // Prune columns that are no longer in the schema from equivalences. - for class in eq_classes.iter_mut() { - let columns_to_remove = class - .iter() - .filter(|column| is_column_invalid_in_new_schema(column, fields)) - .cloned() - .collect::>(); - for column in columns_to_remove { - class.remove(&column); - } + /// Adds new orderings into the existing ordering equivalence class. + pub fn add_new_orderings( + &mut self, + orderings: impl IntoIterator, + ) { + self.oeq_class.add_new_orderings(orderings); } - eq_classes.retain(|props| { - props.len() > 1 - && - // A column should not give an equivalence with itself. - !(props.len() == 2 && props.head.eq(props.others().iter().next().unwrap())) - }); - - output_eq.extend(eq_classes); -} - -/// This function applies the given projection to the given ordering -/// equivalence properties to compute the resulting (projected) ordering -/// equivalence properties; e.g. -/// 1) Adding an alias, which can introduce additional ordering equivalence -/// properties, as in Projection(a, a as a1, a as a2) extends global ordering -/// of a to a1 and a2. -/// 2) Truncate the [`OrderingEquivalentClass`]es that are not in the output schema. -pub fn project_ordering_equivalence_properties( - input_eq: OrderingEquivalenceProperties, - columns_map: &HashMap>, - output_eq: &mut OrderingEquivalenceProperties, -) { - // Get schema and fields of projection output - let schema = output_eq.schema(); - let fields = schema.fields(); - - let oeq_class = input_eq.oeq_class(); - let mut oeq_class = if let Some(oeq_class) = oeq_class { - oeq_class.clone() - } else { - return; - }; - let mut oeq_alias_map = vec![]; - for (column, columns) in columns_map { - if is_column_invalid_in_new_schema(column, fields) { - oeq_alias_map.push((column.clone(), columns[0].clone())); - } - } - oeq_class.update_with_aliases(&oeq_alias_map, fields); - - // Prune columns that no longer is in the schema from from the OrderingEquivalenceProperties. - let sort_exprs_to_remove = oeq_class - .iter() - .filter(|sort_exprs| { - sort_exprs.iter().any(|sort_expr| { - let cols_in_expr = collect_columns(&sort_expr.expr); - // If any one of the columns, used in Expression is invalid, remove expression - // from ordering equivalences - cols_in_expr - .iter() - .any(|col| is_column_invalid_in_new_schema(col, fields)) - }) - }) - .cloned() - .collect::>(); - for sort_exprs in sort_exprs_to_remove { - oeq_class.remove(&sort_exprs); + /// Incorporates the given equivalence group to into the existing + /// equivalence group within. + pub fn add_equivalence_group(&mut self, other_eq_group: EquivalenceGroup) { + self.eq_group.extend(other_eq_group); } - if oeq_class.len() > 1 { - output_eq.extend(Some(oeq_class)); - } -} -/// Update `ordering` if it contains cast expression with target column -/// after projection, if there is no cast expression among `ordering` expressions, -/// returns `None`. -fn update_with_cast_exprs( - cast_exprs: &[(CastExpr, Column)], - mut ordering: LexOrdering, -) -> Option { - let mut is_changed = false; - for sort_expr in ordering.iter_mut() { - for (cast_expr, target_col) in cast_exprs.iter() { - if sort_expr.expr.eq(cast_expr.expr()) { - sort_expr.expr = Arc::new(target_col.clone()) as _; - is_changed = true; - } - } + /// Adds a new equality condition into the existing equivalence group. + /// If the given equality defines a new equivalence class, adds this new + /// equivalence class to the equivalence group. + pub fn add_equal_conditions( + &mut self, + left: &Arc, + right: &Arc, + ) { + self.eq_group.add_equal_conditions(left, right); } - is_changed.then_some(ordering) -} -/// Update cast expressions inside ordering equivalence -/// properties with its target column after projection -pub fn update_ordering_equivalence_with_cast( - cast_exprs: &[(CastExpr, Column)], - input_oeq: &mut OrderingEquivalenceProperties, -) { - if let Some(cls) = &mut input_oeq.oeq_class { - for ordering in - std::iter::once(cls.head().clone()).chain(cls.others().clone().into_iter()) - { - if let Some(updated_ordering) = update_with_cast_exprs(cast_exprs, ordering) { - cls.insert(updated_ordering); + /// Track/register physical expressions with constant values. + pub fn add_constants( + mut self, + constants: impl IntoIterator>, + ) -> Self { + for expr in self.eq_group.normalize_exprs(constants) { + if !physical_exprs_contains(&self.constants, &expr) { + self.constants.push(expr); } } + self } -} - -/// Retrieves the ordering equivalence properties for a given schema and output ordering. -pub fn ordering_equivalence_properties_helper( - schema: SchemaRef, - eq_orderings: &[LexOrdering], -) -> OrderingEquivalenceProperties { - let mut oep = OrderingEquivalenceProperties::new(schema); - let first_ordering = if let Some(first) = eq_orderings.first() { - first - } else { - // Return an empty OrderingEquivalenceProperties: - return oep; - }; - // First entry among eq_orderings is the head, skip it: - for ordering in eq_orderings.iter().skip(1) { - if !ordering.is_empty() { - oep.add_equal_conditions((first_ordering, ordering)) - } - } - oep -} - -/// This function constructs a duplicate-free vector by filtering out duplicate -/// entries inside the given vector `input`. -fn collapse_vec(input: Vec) -> Vec { - let mut output = vec![]; - for item in input { - if !output.contains(&item) { - output.push(item); - } - } - output -} -/// This function constructs a duplicate-free `LexOrderingReq` by filtering out duplicate -/// entries that have same physical expression inside the given vector `input`. -/// `vec![a Some(Asc), a Some(Desc)]` is collapsed to the `vec![a Some(Asc)]`. Since -/// when same expression is already seen before, following expressions are redundant. -fn collapse_lex_req(input: LexOrderingReq) -> LexOrderingReq { - let mut output = vec![]; - for item in input { - if !lex_req_contains(&output, &item) { - output.push(item); - } + /// Updates the ordering equivalence group within assuming that the table + /// is re-sorted according to the argument `sort_exprs`. Note that constants + /// and equivalence classes are unchanged as they are unaffected by a re-sort. + pub fn with_reorder(mut self, sort_exprs: Vec) -> Self { + // TODO: In some cases, existing ordering equivalences may still be valid add this analysis. + self.oeq_class = OrderingEquivalenceClass::new(vec![sort_exprs]); + self } - output -} -/// Check whether `sort_req.expr` is among the expressions of `lex_req`. -fn lex_req_contains( - lex_req: &[PhysicalSortRequirement], - sort_req: &PhysicalSortRequirement, -) -> bool { - for constant in lex_req { - if constant.expr.eq(&sort_req.expr) { + /// Normalizes the given sort expressions (i.e. `sort_exprs`) using the + /// equivalence group and the ordering equivalence class within. + /// + /// Assume that `self.eq_group` states column `a` and `b` are aliases. + /// Also assume that `self.oeq_class` states orderings `d ASC` and `a ASC, c ASC` + /// are equivalent (in the sense that both describe the ordering of the table). + /// If the `sort_exprs` argument were `vec![b ASC, c ASC, a ASC]`, then this + /// function would return `vec![a ASC, c ASC]`. Internally, it would first + /// normalize to `vec![a ASC, c ASC, a ASC]` and end up with the final result + /// after deduplication. + fn normalize_sort_exprs(&self, sort_exprs: LexOrderingRef) -> LexOrdering { + // Convert sort expressions to sort requirements: + let sort_reqs = PhysicalSortRequirement::from_sort_exprs(sort_exprs.iter()); + // Normalize the requirements: + let normalized_sort_reqs = self.normalize_sort_requirements(&sort_reqs); + // Convert sort requirements back to sort expressions: + PhysicalSortRequirement::to_sort_exprs(normalized_sort_reqs) + } + + /// Normalizes the given sort requirements (i.e. `sort_reqs`) using the + /// equivalence group and the ordering equivalence class within. It works by: + /// - Removing expressions that have a constant value from the given requirement. + /// - Replacing sections that belong to some equivalence class in the equivalence + /// group with the first entry in the matching equivalence class. + /// + /// Assume that `self.eq_group` states column `a` and `b` are aliases. + /// Also assume that `self.oeq_class` states orderings `d ASC` and `a ASC, c ASC` + /// are equivalent (in the sense that both describe the ordering of the table). + /// If the `sort_reqs` argument were `vec![b ASC, c ASC, a ASC]`, then this + /// function would return `vec![a ASC, c ASC]`. Internally, it would first + /// normalize to `vec![a ASC, c ASC, a ASC]` and end up with the final result + /// after deduplication. + fn normalize_sort_requirements( + &self, + sort_reqs: LexRequirementRef, + ) -> LexRequirement { + let normalized_sort_reqs = self.eq_group.normalize_sort_requirements(sort_reqs); + let constants_normalized = self.eq_group.normalize_exprs(self.constants.clone()); + // Prune redundant sections in the requirement: + collapse_lex_req( + normalized_sort_reqs + .iter() + .filter(|&order| { + !physical_exprs_contains(&constants_normalized, &order.expr) + }) + .cloned() + .collect(), + ) + } + + /// Checks whether the given ordering is satisfied by any of the existing + /// orderings. + pub fn ordering_satisfy(&self, given: LexOrderingRef) -> bool { + // Convert the given sort expressions to sort requirements: + let sort_requirements = PhysicalSortRequirement::from_sort_exprs(given.iter()); + self.ordering_satisfy_requirement(&sort_requirements) + } + + /// Checks whether the given sort requirements are satisfied by any of the + /// existing orderings. + pub fn ordering_satisfy_requirement(&self, reqs: LexRequirementRef) -> bool { + // First, standardize the given requirement: + let normalized_reqs = self.normalize_sort_requirements(reqs); + if normalized_reqs.is_empty() { + // Requirements are tautologically satisfied if empty. return true; } - } - false -} - -/// This function simplifies lexicographical ordering requirement -/// inside `input` by removing postfix lexicographical requirements -/// that satisfy global ordering (occurs inside the ordering equivalent class) -fn simplify_lex_req( - input: LexOrderingReq, - oeq_class: &OrderingEquivalentClass, -) -> LexOrderingReq { - let mut section = &input[..]; - loop { - let n_prune = prune_last_n_that_is_in_oeq(section, oeq_class); - // Cannot prune entries from the end of requirement - if n_prune == 0 { - break; + let mut indices = HashSet::new(); + for ordering in self.normalized_oeq_class().iter() { + let match_indices = ordering + .iter() + .map(|sort_expr| { + normalized_reqs + .iter() + .position(|sort_req| sort_expr.satisfy(sort_req, &self.schema)) + }) + .collect::>(); + // Find the largest contiguous increasing sequence starting from the first index: + if let Some(&Some(first)) = match_indices.first() { + indices.insert(first); + let mut iter = match_indices.windows(2); + while let Some([Some(current), Some(next)]) = iter.next() { + if next > current { + indices.insert(*next); + } else { + break; + } + } + } } - section = §ion[0..section.len() - n_prune]; + indices.len() == normalized_reqs.len() } - if section.is_empty() { - PhysicalSortRequirement::from_sort_exprs(oeq_class.head()) - } else { - section.to_vec() - } -} -/// Determines how many entries from the end can be deleted. -/// Last n entry satisfies global ordering, hence having them -/// as postfix in the lexicographical requirement is unnecessary. -/// Assume requirement is [a ASC, b ASC, c ASC], also assume that -/// existing ordering is [c ASC, d ASC]. In this case, since [c ASC] -/// is satisfied by the existing ordering (e.g corresponding section is global ordering), -/// [c ASC] can be pruned from the requirement: [a ASC, b ASC, c ASC]. In this case, -/// this function will return 1, to indicate last element can be removed from the requirement -fn prune_last_n_that_is_in_oeq( - input: &[PhysicalSortRequirement], - oeq_class: &OrderingEquivalentClass, -) -> usize { - let input_len = input.len(); - for ordering in std::iter::once(oeq_class.head()).chain(oeq_class.others().iter()) { - let mut search_range = std::cmp::min(ordering.len(), input_len); - while search_range > 0 { - let req_section = &input[input_len - search_range..]; - // let given_section = &ordering[0..search_range]; - if req_satisfied(ordering, req_section) { - return search_range; + /// Checks whether the `given`` sort requirements are equal or more specific + /// than the `reference` sort requirements. + pub fn requirements_compatible( + &self, + given: LexRequirementRef, + reference: LexRequirementRef, + ) -> bool { + let normalized_given = self.normalize_sort_requirements(given); + let normalized_reference = self.normalize_sort_requirements(reference); + + (normalized_reference.len() <= normalized_given.len()) + && normalized_reference + .into_iter() + .zip(normalized_given) + .all(|(reference, given)| given.compatible(&reference)) + } + + /// Returns the finer ordering among the orderings `lhs` and `rhs`, breaking + /// any ties by choosing `lhs`. + /// + /// The finer ordering is the ordering that satisfies both of the orderings. + /// If the orderings are incomparable, returns `None`. + /// + /// For example, the finer ordering among `[a ASC]` and `[a ASC, b ASC]` is + /// the latter. + pub fn get_finer_ordering( + &self, + lhs: LexOrderingRef, + rhs: LexOrderingRef, + ) -> Option { + // Convert the given sort expressions to sort requirements: + let lhs = PhysicalSortRequirement::from_sort_exprs(lhs); + let rhs = PhysicalSortRequirement::from_sort_exprs(rhs); + let finer = self.get_finer_requirement(&lhs, &rhs); + // Convert the chosen sort requirements back to sort expressions: + finer.map(PhysicalSortRequirement::to_sort_exprs) + } + + /// Returns the finer ordering among the requirements `lhs` and `rhs`, + /// breaking any ties by choosing `lhs`. + /// + /// The finer requirements are the ones that satisfy both of the given + /// requirements. If the requirements are incomparable, returns `None`. + /// + /// For example, the finer requirements among `[a ASC]` and `[a ASC, b ASC]` + /// is the latter. + pub fn get_finer_requirement( + &self, + req1: LexRequirementRef, + req2: LexRequirementRef, + ) -> Option { + let mut lhs = self.normalize_sort_requirements(req1); + let mut rhs = self.normalize_sort_requirements(req2); + lhs.iter_mut() + .zip(rhs.iter_mut()) + .all(|(lhs, rhs)| { + lhs.expr.eq(&rhs.expr) + && match (lhs.options, rhs.options) { + (Some(lhs_opt), Some(rhs_opt)) => lhs_opt == rhs_opt, + (Some(options), None) => { + rhs.options = Some(options); + true + } + (None, Some(options)) => { + lhs.options = Some(options); + true + } + (None, None) => true, + } + }) + .then_some(if lhs.len() >= rhs.len() { lhs } else { rhs }) + } + + /// Calculates the "meet" of the given orderings (`lhs` and `rhs`). + /// The meet of a set of orderings is the finest ordering that is satisfied + /// by all the orderings in that set. For details, see: + /// + /// + /// + /// If there is no ordering that satisfies both `lhs` and `rhs`, returns + /// `None`. As an example, the meet of orderings `[a ASC]` and `[a ASC, b ASC]` + /// is `[a ASC]`. + pub fn get_meet_ordering( + &self, + lhs: LexOrderingRef, + rhs: LexOrderingRef, + ) -> Option { + let lhs = self.normalize_sort_exprs(lhs); + let rhs = self.normalize_sort_exprs(rhs); + let mut meet = vec![]; + for (lhs, rhs) in lhs.into_iter().zip(rhs.into_iter()) { + if lhs.eq(&rhs) { + meet.push(lhs); } else { - search_range -= 1; + break; } } + (!meet.is_empty()).then_some(meet) + } + + /// Projects argument `expr` according to `projection_mapping`, taking + /// equivalences into account. + /// + /// For example, assume that columns `a` and `c` are always equal, and that + /// `projection_mapping` encodes following mapping: + /// + /// ```text + /// a -> a1 + /// b -> b1 + /// ``` + /// + /// Then, this function projects `a + b` to `Some(a1 + b1)`, `c + b` to + /// `Some(a1 + b1)` and `d` to `None`, meaning that it cannot be projected. + pub fn project_expr( + &self, + expr: &Arc, + projection_mapping: &ProjectionMapping, + ) -> Option> { + self.eq_group.project_expr(projection_mapping, expr) } - 0 -} -/// Checks whether given section satisfies req. -fn req_satisfied(given: LexOrderingRef, req: &[PhysicalSortRequirement]) -> bool { - for (given, req) in izip!(given.iter(), req.iter()) { - let PhysicalSortRequirement { expr, options } = req; - if let Some(options) = options { - if options != &given.options || !expr.eq(&given.expr) { - return false; + /// Projects the equivalences within according to `projection_mapping` + /// and `output_schema`. + pub fn project( + &self, + projection_mapping: &ProjectionMapping, + output_schema: SchemaRef, + ) -> Self { + let mut projected_orderings = self + .oeq_class + .iter() + .filter_map(|order| self.eq_group.project_ordering(projection_mapping, order)) + .collect::>(); + for (source, target) in projection_mapping { + let expr_ordering = ExprOrdering::new(source.clone()) + .transform_up(&|expr| update_ordering(expr, self)) + .unwrap(); + if let SortProperties::Ordered(options) = expr_ordering.state { + // Push new ordering to the state. + projected_orderings.push(vec![PhysicalSortExpr { + expr: target.clone(), + options, + }]); } - } else if !expr.eq(&given.expr) { - return false; - } - } - true -} - -/// Combine equivalence properties of the given join inputs. -pub fn combine_join_equivalence_properties( - join_type: JoinType, - left_properties: EquivalenceProperties, - right_properties: EquivalenceProperties, - left_columns_len: usize, - on: &[(Column, Column)], - schema: SchemaRef, -) -> EquivalenceProperties { - let mut new_properties = EquivalenceProperties::new(schema); - match join_type { - JoinType::Inner | JoinType::Left | JoinType::Full | JoinType::Right => { - new_properties.extend(left_properties.classes().to_vec()); - let new_right_properties = right_properties - .classes() - .iter() - .map(|prop| { - let new_head = Column::new( - prop.head().name(), - left_columns_len + prop.head().index(), - ); - let new_others = prop - .others() - .iter() - .map(|col| { - Column::new(col.name(), left_columns_len + col.index()) - }) - .collect::>(); - EquivalentClass::new(new_head, new_others) - }) - .collect::>(); - - new_properties.extend(new_right_properties); - } - JoinType::LeftSemi | JoinType::LeftAnti => { - new_properties.extend(left_properties.classes().to_vec()) } - JoinType::RightSemi | JoinType::RightAnti => { - new_properties.extend(right_properties.classes().to_vec()) + Self { + eq_group: self.eq_group.project(projection_mapping), + oeq_class: OrderingEquivalenceClass::new(projected_orderings), + constants: vec![], + schema: output_schema, } } - if join_type == JoinType::Inner { - on.iter().for_each(|(column1, column2)| { - let new_column2 = - Column::new(column2.name(), left_columns_len + column2.index()); - new_properties.add_equal_conditions((column1, &new_column2)) - }) - } - new_properties -} - -/// Calculate equivalence properties for the given cross join operation. -pub fn cross_join_equivalence_properties( - left_properties: EquivalenceProperties, - right_properties: EquivalenceProperties, - left_columns_len: usize, - schema: SchemaRef, -) -> EquivalenceProperties { - let mut new_properties = EquivalenceProperties::new(schema); - new_properties.extend(left_properties.classes().to_vec()); - let new_right_properties = right_properties - .classes() - .iter() - .map(|prop| { - let new_head = - Column::new(prop.head().name(), left_columns_len + prop.head().index()); - let new_others = prop - .others() - .iter() - .map(|col| Column::new(col.name(), left_columns_len + col.index())) - .collect::>(); - EquivalentClass::new(new_head, new_others) - }) - .collect::>(); - new_properties.extend(new_right_properties); - new_properties -} - -/// Update right table ordering equivalences so that: -/// - They point to valid indices at the output of the join schema, and -/// - They are normalized with respect to equivalence columns. -/// -/// To do so, we increment column indices by the size of the left table when -/// join schema consists of a combination of left and right schema (Inner, -/// Left, Full, Right joins). Then, we normalize the sort expressions of -/// ordering equivalences one by one. We make sure that each expression in the -/// ordering equivalence is either: -/// - The head of the one of the equivalent classes, or -/// - Doesn't have an equivalent column. -/// -/// This way; once we normalize an expression according to equivalence properties, -/// it can thereafter safely be used for ordering equivalence normalization. -fn get_updated_right_ordering_equivalent_class( - join_type: &JoinType, - right_oeq_class: &OrderingEquivalentClass, - left_columns_len: usize, - join_eq_properties: &EquivalenceProperties, -) -> OrderingEquivalentClass { - match join_type { - // In these modes, indices of the right schema should be offset by - // the left table size. - JoinType::Inner | JoinType::Left | JoinType::Full | JoinType::Right => { - let right_oeq_class = right_oeq_class.add_offset(left_columns_len); - return right_oeq_class - .normalize_with_equivalence_properties(join_eq_properties); + /// Returns the longest (potentially partial) permutation satisfying the + /// existing ordering. For example, if we have the equivalent orderings + /// `[a ASC, b ASC]` and `[c DESC]`, with `exprs` containing `[c, b, a, d]`, + /// then this function returns `([a ASC, b ASC, c DESC], [2, 1, 0])`. + /// This means that the specification `[a ASC, b ASC, c DESC]` is satisfied + /// by the existing ordering, and `[a, b, c]` resides at indices: `2, 1, 0` + /// inside the argument `exprs` (respectively). For the mathematical + /// definition of "partial permutation", see: + /// + /// + pub fn find_longest_permutation( + &self, + exprs: &[Arc], + ) -> (LexOrdering, Vec) { + let normalized_exprs = self.eq_group.normalize_exprs(exprs.to_vec()); + // Use a map to associate expression indices with sort options: + let mut ordered_exprs = IndexMap::::new(); + for ordering in self.normalized_oeq_class().iter() { + for sort_expr in ordering { + if let Some(idx) = normalized_exprs + .iter() + .position(|expr| sort_expr.expr.eq(expr)) + { + if let Entry::Vacant(e) = ordered_exprs.entry(idx) { + e.insert(sort_expr.options); + } + } else { + // We only consider expressions that correspond to a prefix + // of one of the equivalent orderings we have. + break; + } + } } - _ => {} - }; - right_oeq_class.normalize_with_equivalence_properties(join_eq_properties) + // Construct the lexicographical ordering according to the permutation: + ordered_exprs + .into_iter() + .map(|(idx, options)| { + ( + PhysicalSortExpr { + expr: exprs[idx].clone(), + options, + }, + idx, + ) + }) + .unzip() + } } /// Calculate ordering equivalence properties for the given join operation. -pub fn combine_join_ordering_equivalence_properties( +pub fn join_equivalence_properties( + left: EquivalenceProperties, + right: EquivalenceProperties, join_type: &JoinType, - left_oeq_properties: &OrderingEquivalenceProperties, - right_oeq_properties: &OrderingEquivalenceProperties, - schema: SchemaRef, + join_schema: SchemaRef, maintains_input_order: &[bool], probe_side: Option, - join_eq_properties: EquivalenceProperties, -) -> OrderingEquivalenceProperties { - let mut new_properties = OrderingEquivalenceProperties::new(schema); - let left_columns_len = left_oeq_properties.schema().fields().len(); - // All joins have 2 children - assert_eq!(maintains_input_order.len(), 2); - let left_maintains = maintains_input_order[0]; - let right_maintains = maintains_input_order[1]; - match (left_maintains, right_maintains) { - (true, true) => { - unreachable!("Cannot maintain ordering of both sides"); - } - (true, false) => { - // In this special case, right side ordering can be prefixed with left side ordering. - if let ( - Some(JoinSide::Left), - JoinType::Inner, - Some(left_oeq_class), - Some(right_oeq_class), - ) = ( - probe_side, - join_type, - left_oeq_properties.oeq_class(), - right_oeq_properties.oeq_class(), - ) { - let updated_right_oeq = get_updated_right_ordering_equivalent_class( + on: &[(Column, Column)], +) -> EquivalenceProperties { + let left_size = left.schema.fields.len(); + let mut result = EquivalenceProperties::new(join_schema); + result.add_equivalence_group(left.eq_group().join( + right.eq_group(), + join_type, + left_size, + on, + )); + + let left_oeq_class = left.oeq_class; + let mut right_oeq_class = right.oeq_class; + match maintains_input_order { + [true, false] => { + // In this special case, right side ordering can be prefixed with + // the left side ordering. + if let (Some(JoinSide::Left), JoinType::Inner) = (probe_side, join_type) { + updated_right_ordering_equivalence_class( + &mut right_oeq_class, join_type, - right_oeq_class, - left_columns_len, - &join_eq_properties, + left_size, ); - // Right side ordering equivalence properties should be prepended with - // those of the left side while constructing output ordering equivalence - // properties since stream side is the left side. + // Right side ordering equivalence properties should be prepended + // with those of the left side while constructing output ordering + // equivalence properties since stream side is the left side. // - // If the right table ordering equivalences contain `b ASC`, and the output - // ordering of the left table is `a ASC`, then the ordering equivalence `b ASC` - // for the right table should be converted to `a ASC, b ASC` before it is added - // to the ordering equivalences of the join. - let mut orderings = vec![]; - for left_ordering in left_oeq_class.iter() { - for right_ordering in updated_right_oeq.iter() { - let mut ordering = left_ordering.to_vec(); - ordering.extend(right_ordering.to_vec()); - let ordering_normalized = - join_eq_properties.normalize_sort_exprs(&ordering); - orderings.push(ordering_normalized); - } - } - if !orderings.is_empty() { - let head = orderings.swap_remove(0); - let new_oeq_class = OrderingEquivalentClass::new(head, orderings); - new_properties.extend(Some(new_oeq_class)); - } + // For example, if the right side ordering equivalences contain + // `b ASC`, and the left side ordering equivalences contain `a ASC`, + // then we should add `a ASC, b ASC` to the ordering equivalences + // of the join output. + let out_oeq_class = left_oeq_class.join_suffix(&right_oeq_class); + result.add_ordering_equivalence_class(out_oeq_class); } else { - new_properties.extend(left_oeq_properties.oeq_class().cloned()); + result.add_ordering_equivalence_class(left_oeq_class); } } - (false, true) => { - let updated_right_oeq = - right_oeq_properties.oeq_class().map(|right_oeq_class| { - get_updated_right_ordering_equivalent_class( - join_type, - right_oeq_class, - left_columns_len, - &join_eq_properties, - ) - }); - // In this special case, left side ordering can be prefixed with right side ordering. - if let ( - Some(JoinSide::Right), - JoinType::Inner, - Some(left_oeq_class), - Some(right_oeg_class), - ) = ( - probe_side, + [false, true] => { + updated_right_ordering_equivalence_class( + &mut right_oeq_class, join_type, - left_oeq_properties.oeq_class(), - &updated_right_oeq, - ) { - // Left side ordering equivalence properties should be prepended with - // those of the right side while constructing output ordering equivalence - // properties since stream side is the right side. + left_size, + ); + // In this special case, left side ordering can be prefixed with + // the right side ordering. + if let (Some(JoinSide::Right), JoinType::Inner) = (probe_side, join_type) { + // Left side ordering equivalence properties should be prepended + // with those of the right side while constructing output ordering + // equivalence properties since stream side is the right side. // - // If the right table ordering equivalences contain `b ASC`, and the output - // ordering of the left table is `a ASC`, then the ordering equivalence `b ASC` - // for the right table should be converted to `a ASC, b ASC` before it is added - // to the ordering equivalences of the join. - let mut orderings = vec![]; - for right_ordering in right_oeg_class.iter() { - for left_ordering in left_oeq_class.iter() { - let mut ordering = right_ordering.to_vec(); - ordering.extend(left_ordering.to_vec()); - let ordering_normalized = - join_eq_properties.normalize_sort_exprs(&ordering); - orderings.push(ordering_normalized); - } - } - if !orderings.is_empty() { - let head = orderings.swap_remove(0); - let new_oeq_class = OrderingEquivalentClass::new(head, orderings); - new_properties.extend(Some(new_oeq_class)); - } + // For example, if the left side ordering equivalences contain + // `a ASC`, and the right side ordering equivalences contain `b ASC`, + // then we should add `b ASC, a ASC` to the ordering equivalences + // of the join output. + let out_oeq_class = right_oeq_class.join_suffix(&left_oeq_class); + result.add_ordering_equivalence_class(out_oeq_class); } else { - new_properties.extend(updated_right_oeq); + result.add_ordering_equivalence_class(right_oeq_class); } } - (false, false) => {} + [false, false] => {} + [true, true] => unreachable!("Cannot maintain ordering of both sides"), + _ => unreachable!("Join operators can not have more than two children"), } - new_properties -} - -/// This function searches for the slice `section` inside the slice `given`. -/// It returns each range where `section` is compatible with the corresponding -/// slice in `given`. -fn get_compatible_ranges( - given: &[PhysicalSortRequirement], - section: &[PhysicalSortRequirement], -) -> Vec> { - let n_section = section.len(); - let n_end = if given.len() >= n_section { - given.len() - n_section + 1 - } else { - 0 - }; - (0..n_end) - .filter_map(|idx| { - let end = idx + n_section; - given[idx..end] - .iter() - .zip(section) - .all(|(req, given)| given.compatible(req)) - .then_some(Range { start: idx, end }) - }) - .collect() + result } -/// It is similar to contains method of vector. -/// Finds whether `expr` is among `physical_exprs`. -pub fn physical_exprs_contains( - physical_exprs: &[Arc], - expr: &Arc, -) -> bool { - physical_exprs - .iter() - .any(|physical_expr| physical_expr.eq(expr)) -} - -/// Remove ordering requirements that have constant value -fn prune_sort_reqs_with_constants( - ordering: &[PhysicalSortRequirement], - constants: &[Arc], -) -> Vec { - ordering - .iter() - .filter(|&order| !physical_exprs_contains(constants, &order.expr)) - .cloned() - .collect() -} - -/// Adds the `offset` value to `Column` indices inside `expr`. This function is -/// generally used during the update of the right table schema in join operations. -pub fn add_offset_to_expr( - expr: Arc, - offset: usize, -) -> Arc { - expr.transform_down(&|e| match e.as_any().downcast_ref::() { - Some(col) => Ok(Transformed::Yes(Arc::new(Column::new( - col.name(), - offset + col.index(), - )))), - None => Ok(Transformed::No(e)), - }) - .unwrap() - // Note that we can safely unwrap here since our transform always returns - // an `Ok` value. -} - -/// Adds the `offset` value to `Column` indices inside `sort_expr.expr`. -pub(crate) fn add_offset_to_sort_expr( - sort_expr: &PhysicalSortExpr, - offset: usize, -) -> PhysicalSortExpr { - PhysicalSortExpr { - expr: add_offset_to_expr(sort_expr.expr.clone(), offset), - options: sort_expr.options, +/// In the context of a join, update the right side `OrderingEquivalenceClass` +/// so that they point to valid indices in the join output schema. +/// +/// To do so, we increment column indices by the size of the left table when +/// join schema consists of a combination of the left and right schemas. This +/// is the case for `Inner`, `Left`, `Full` and `Right` joins. For other cases, +/// indices do not change. +fn updated_right_ordering_equivalence_class( + right_oeq_class: &mut OrderingEquivalenceClass, + join_type: &JoinType, + left_size: usize, +) { + if matches!( + join_type, + JoinType::Inner | JoinType::Left | JoinType::Full | JoinType::Right + ) { + right_oeq_class.add_offset(left_size); } } -/// Adds the `offset` value to `Column` indices for each `sort_expr.expr` -/// inside `sort_exprs`. -pub fn add_offset_to_lex_ordering( - sort_exprs: LexOrderingRef, - offset: usize, -) -> LexOrdering { - sort_exprs - .iter() - .map(|sort_expr| add_offset_to_sort_expr(sort_expr, offset)) - .collect() +/// Calculates the [`SortProperties`] of a given [`ExprOrdering`] node. +/// The node can either be a leaf node, or an intermediate node: +/// - If it is a leaf node, we directly find the order of the node by looking +/// at the given sort expression and equivalence properties if it is a `Column` +/// leaf, or we mark it as unordered. In the case of a `Literal` leaf, we mark +/// it as singleton so that it can cooperate with all ordered columns. +/// - If it is an intermediate node, the children states matter. Each `PhysicalExpr` +/// and operator has its own rules on how to propagate the children orderings. +/// However, before we engage in recursion, we check whether this intermediate +/// node directly matches with the sort expression. If there is a match, the +/// sort expression emerges at that node immediately, discarding the recursive +/// result coming from its children. +fn update_ordering( + mut node: ExprOrdering, + eq_properties: &EquivalenceProperties, +) -> Result> { + if !node.expr.children().is_empty() { + // We have an intermediate (non-leaf) node, account for its children: + node.state = node.expr.get_ordering(&node.children_states); + Ok(Transformed::Yes(node)) + } else if node.expr.as_any().is::() { + // We have a Column, which is one of the two possible leaf node types: + let eq_group = &eq_properties.eq_group; + let normalized_expr = eq_group.normalize_expr(node.expr.clone()); + let oeq_class = &eq_properties.oeq_class; + if let Some(options) = oeq_class.get_options(&normalized_expr) { + node.state = SortProperties::Ordered(options); + Ok(Transformed::Yes(node)) + } else { + Ok(Transformed::No(node)) + } + } else { + // We have a Literal, which is the other possible leaf node type: + node.state = node.expr.get_ordering(&[]); + Ok(Transformed::Yes(node)) + } } #[cfg(test)] mod tests { + use std::sync::Arc; + use super::*; - use crate::expressions::Column; + use crate::expressions::{col, lit, BinaryExpr, Column}; + use crate::physical_expr::{physical_exprs_bag_equal, physical_exprs_equal}; + + use arrow::compute::{lexsort_to_indices, SortColumn}; use arrow::datatypes::{DataType, Field, Schema}; + use arrow_array::{ArrayRef, RecordBatch, UInt32Array, UInt64Array}; + use arrow_schema::{Fields, SortOptions}; use datafusion_common::Result; + use datafusion_expr::Operator; + + use itertools::{izip, Itertools}; + use rand::rngs::StdRng; + use rand::seq::SliceRandom; + use rand::{Rng, SeedableRng}; + + // Generate a schema which consists of 8 columns (a, b, c, d, e, f, g, h) + fn create_test_schema() -> Result { + let a = Field::new("a", DataType::Int32, true); + let b = Field::new("b", DataType::Int32, true); + let c = Field::new("c", DataType::Int32, true); + let d = Field::new("d", DataType::Int32, true); + let e = Field::new("e", DataType::Int32, true); + let f = Field::new("f", DataType::Int32, true); + let g = Field::new("g", DataType::Int32, true); + let h = Field::new("h", DataType::Int32, true); + let schema = Arc::new(Schema::new(vec![a, b, c, d, e, f, g, h])); + + Ok(schema) + } + + /// Construct a schema with following properties + /// Schema satisfies following orderings: + /// [a ASC], [d ASC, b ASC], [e DESC, f ASC, g ASC] + /// and + /// Column [a=c] (e.g they are aliases). + fn create_test_params() -> Result<(SchemaRef, EquivalenceProperties)> { + let test_schema = create_test_schema()?; + let col_a = &col("a", &test_schema)?; + let col_b = &col("b", &test_schema)?; + let col_c = &col("c", &test_schema)?; + let col_d = &col("d", &test_schema)?; + let col_e = &col("e", &test_schema)?; + let col_f = &col("f", &test_schema)?; + let col_g = &col("g", &test_schema)?; + let mut eq_properties = EquivalenceProperties::new(test_schema.clone()); + eq_properties.add_equal_conditions(col_a, col_c); + + let option_asc = SortOptions { + descending: false, + nulls_first: false, + }; + let option_desc = SortOptions { + descending: true, + nulls_first: true, + }; + let orderings = vec![ + // [a ASC] + vec![(col_a, option_asc)], + // [d ASC, b ASC] + vec![(col_d, option_asc), (col_b, option_asc)], + // [e DESC, f ASC, g ASC] + vec![ + (col_e, option_desc), + (col_f, option_asc), + (col_g, option_asc), + ], + ]; + let orderings = convert_to_orderings(&orderings); + eq_properties.add_new_orderings(orderings); + Ok((test_schema, eq_properties)) + } + + // Generate a schema which consists of 6 columns (a, b, c, d, e, f) + fn create_test_schema_2() -> Result { + let a = Field::new("a", DataType::Int32, true); + let b = Field::new("b", DataType::Int32, true); + let c = Field::new("c", DataType::Int32, true); + let d = Field::new("d", DataType::Int32, true); + let e = Field::new("e", DataType::Int32, true); + let f = Field::new("f", DataType::Int32, true); + let schema = Arc::new(Schema::new(vec![a, b, c, d, e, f])); + + Ok(schema) + } + + /// Construct a schema with random ordering + /// among column a, b, c, d + /// where + /// Column [a=f] (e.g they are aliases). + /// Column e is constant. + fn create_random_schema(seed: u64) -> Result<(SchemaRef, EquivalenceProperties)> { + let test_schema = create_test_schema_2()?; + let col_a = &col("a", &test_schema)?; + let col_b = &col("b", &test_schema)?; + let col_c = &col("c", &test_schema)?; + let col_d = &col("d", &test_schema)?; + let col_e = &col("e", &test_schema)?; + let col_f = &col("f", &test_schema)?; + let col_exprs = [col_a, col_b, col_c, col_d, col_e, col_f]; + + let mut eq_properties = EquivalenceProperties::new(test_schema.clone()); + // Define a and f are aliases + eq_properties.add_equal_conditions(col_a, col_f); + // Column e has constant value. + eq_properties = eq_properties.add_constants([col_e.clone()]); + + // Randomly order columns for sorting + let mut rng = StdRng::seed_from_u64(seed); + let mut remaining_exprs = col_exprs[0..4].to_vec(); // only a, b, c, d are sorted + + let options_asc = SortOptions { + descending: false, + nulls_first: false, + }; - use arrow_schema::SortOptions; - use std::sync::Arc; + while !remaining_exprs.is_empty() { + let n_sort_expr = rng.gen_range(0..remaining_exprs.len() + 1); + remaining_exprs.shuffle(&mut rng); + + let ordering = remaining_exprs + .drain(0..n_sort_expr) + .map(|expr| PhysicalSortExpr { + expr: expr.clone(), + options: options_asc, + }) + .collect(); + + eq_properties.add_new_orderings([ordering]); + } + + Ok((test_schema, eq_properties)) + } - fn convert_to_requirement( - in_data: &[(&Column, Option)], + // Convert each tuple to PhysicalSortRequirement + fn convert_to_sort_reqs( + in_data: &[(&Arc, Option)], ) -> Vec { in_data .iter() - .map(|(col, options)| { - PhysicalSortRequirement::new(Arc::new((*col).clone()) as _, *options) + .map(|(expr, options)| { + PhysicalSortRequirement::new((*expr).clone(), *options) + }) + .collect::>() + } + + // Convert each tuple to PhysicalSortExpr + fn convert_to_sort_exprs( + in_data: &[(&Arc, SortOptions)], + ) -> Vec { + in_data + .iter() + .map(|(expr, options)| PhysicalSortExpr { + expr: (*expr).clone(), + options: *options, }) .collect::>() } + // Convert each inner tuple to PhysicalSortExpr + fn convert_to_orderings( + orderings: &[Vec<(&Arc, SortOptions)>], + ) -> Vec> { + orderings + .iter() + .map(|sort_exprs| convert_to_sort_exprs(sort_exprs)) + .collect() + } + #[test] fn add_equal_conditions_test() -> Result<()> { let schema = Arc::new(Schema::new(vec![ @@ -1239,38 +1367,49 @@ mod tests { ])); let mut eq_properties = EquivalenceProperties::new(schema); - let new_condition = (&Column::new("a", 0), &Column::new("b", 1)); - eq_properties.add_equal_conditions(new_condition); - assert_eq!(eq_properties.classes().len(), 1); - - let new_condition = (&Column::new("b", 1), &Column::new("a", 0)); - eq_properties.add_equal_conditions(new_condition); - assert_eq!(eq_properties.classes().len(), 1); - assert_eq!(eq_properties.classes()[0].len(), 2); - assert!(eq_properties.classes()[0].contains(&Column::new("a", 0))); - assert!(eq_properties.classes()[0].contains(&Column::new("b", 1))); - - let new_condition = (&Column::new("b", 1), &Column::new("c", 2)); - eq_properties.add_equal_conditions(new_condition); - assert_eq!(eq_properties.classes().len(), 1); - assert_eq!(eq_properties.classes()[0].len(), 3); - assert!(eq_properties.classes()[0].contains(&Column::new("a", 0))); - assert!(eq_properties.classes()[0].contains(&Column::new("b", 1))); - assert!(eq_properties.classes()[0].contains(&Column::new("c", 2))); - - let new_condition = (&Column::new("x", 3), &Column::new("y", 4)); - eq_properties.add_equal_conditions(new_condition); - assert_eq!(eq_properties.classes().len(), 2); - - let new_condition = (&Column::new("x", 3), &Column::new("a", 0)); - eq_properties.add_equal_conditions(new_condition); - assert_eq!(eq_properties.classes().len(), 1); - assert_eq!(eq_properties.classes()[0].len(), 5); - assert!(eq_properties.classes()[0].contains(&Column::new("a", 0))); - assert!(eq_properties.classes()[0].contains(&Column::new("b", 1))); - assert!(eq_properties.classes()[0].contains(&Column::new("c", 2))); - assert!(eq_properties.classes()[0].contains(&Column::new("x", 3))); - assert!(eq_properties.classes()[0].contains(&Column::new("y", 4))); + let col_a_expr = Arc::new(Column::new("a", 0)) as Arc; + let col_b_expr = Arc::new(Column::new("b", 1)) as Arc; + let col_c_expr = Arc::new(Column::new("c", 2)) as Arc; + let col_x_expr = Arc::new(Column::new("x", 3)) as Arc; + let col_y_expr = Arc::new(Column::new("y", 4)) as Arc; + + // a and b are aliases + eq_properties.add_equal_conditions(&col_a_expr, &col_b_expr); + assert_eq!(eq_properties.eq_group().len(), 1); + + // This new entry is redundant, size shouldn't increase + eq_properties.add_equal_conditions(&col_b_expr, &col_a_expr); + assert_eq!(eq_properties.eq_group().len(), 1); + let eq_groups = &eq_properties.eq_group().classes[0]; + assert_eq!(eq_groups.len(), 2); + assert!(physical_exprs_contains(eq_groups, &col_a_expr)); + assert!(physical_exprs_contains(eq_groups, &col_b_expr)); + + // b and c are aliases. Exising equivalence class should expand, + // however there shouldn't be any new equivalence class + eq_properties.add_equal_conditions(&col_b_expr, &col_c_expr); + assert_eq!(eq_properties.eq_group().len(), 1); + let eq_groups = &eq_properties.eq_group().classes[0]; + assert_eq!(eq_groups.len(), 3); + assert!(physical_exprs_contains(eq_groups, &col_a_expr)); + assert!(physical_exprs_contains(eq_groups, &col_b_expr)); + assert!(physical_exprs_contains(eq_groups, &col_c_expr)); + + // This is a new set of equality. Hence equivalent class count should be 2. + eq_properties.add_equal_conditions(&col_x_expr, &col_y_expr); + assert_eq!(eq_properties.eq_group().len(), 2); + + // This equality bridges distinct equality sets. + // Hence equivalent class count should decrease from 2 to 1. + eq_properties.add_equal_conditions(&col_x_expr, &col_a_expr); + assert_eq!(eq_properties.eq_group().len(), 1); + let eq_groups = &eq_properties.eq_group().classes[0]; + assert_eq!(eq_groups.len(), 5); + assert!(physical_exprs_contains(eq_groups, &col_a_expr)); + assert!(physical_exprs_contains(eq_groups, &col_b_expr)); + assert!(physical_exprs_contains(eq_groups, &col_c_expr)); + assert!(physical_exprs_contains(eq_groups, &col_x_expr)); + assert!(physical_exprs_contains(eq_groups, &col_y_expr)); Ok(()) } @@ -1283,11 +1422,8 @@ mod tests { Field::new("c", DataType::Int64, true), ])); - let mut input_properties = EquivalenceProperties::new(input_schema); - let new_condition = (&Column::new("a", 0), &Column::new("b", 1)); - input_properties.add_equal_conditions(new_condition); - let new_condition = (&Column::new("b", 1), &Column::new("c", 2)); - input_properties.add_equal_conditions(new_condition); + let input_properties = EquivalenceProperties::new(input_schema.clone()); + let col_a = col("a", &input_schema)?; let out_schema = Arc::new(Schema::new(vec![ Field::new("a1", DataType::Int64, true), @@ -1296,106 +1432,557 @@ mod tests { Field::new("a4", DataType::Int64, true), ])); - let mut alias_map = HashMap::new(); - alias_map.insert( - Column::new("a", 0), - vec![ - Column::new("a1", 0), - Column::new("a2", 1), - Column::new("a3", 2), - Column::new("a4", 3), - ], - ); - let mut out_properties = EquivalenceProperties::new(out_schema); + // a as a1, a as a2, a as a3, a as a3 + let col_a1 = &col("a1", &out_schema)?; + let col_a2 = &col("a2", &out_schema)?; + let col_a3 = &col("a3", &out_schema)?; + let col_a4 = &col("a4", &out_schema)?; + let projection_mapping = vec![ + (col_a.clone(), col_a1.clone()), + (col_a.clone(), col_a2.clone()), + (col_a.clone(), col_a3.clone()), + (col_a.clone(), col_a4.clone()), + ]; + let out_properties = input_properties.project(&projection_mapping, out_schema); - project_equivalence_properties(input_properties, &alias_map, &mut out_properties); - assert_eq!(out_properties.classes().len(), 1); - assert_eq!(out_properties.classes()[0].len(), 4); - assert!(out_properties.classes()[0].contains(&Column::new("a1", 0))); - assert!(out_properties.classes()[0].contains(&Column::new("a2", 1))); - assert!(out_properties.classes()[0].contains(&Column::new("a3", 2))); - assert!(out_properties.classes()[0].contains(&Column::new("a4", 3))); + // At the output a1=a2=a3=a4 + assert_eq!(out_properties.eq_group().len(), 1); + let eq_class = &out_properties.eq_group().classes[0]; + assert_eq!(eq_class.len(), 4); + assert!(physical_exprs_contains(eq_class, col_a1)); + assert!(physical_exprs_contains(eq_class, col_a2)); + assert!(physical_exprs_contains(eq_class, col_a3)); + assert!(physical_exprs_contains(eq_class, col_a4)); Ok(()) } #[test] - fn test_collapse_vec() -> Result<()> { - assert_eq!(collapse_vec(vec![1, 2, 3]), vec![1, 2, 3]); - assert_eq!(collapse_vec(vec![1, 2, 3, 2, 3]), vec![1, 2, 3]); - assert_eq!(collapse_vec(vec![3, 1, 2, 3, 2, 3]), vec![3, 1, 2]); + fn test_ordering_satisfy() -> Result<()> { + let crude = vec![PhysicalSortExpr { + expr: Arc::new(Column::new("a", 0)), + options: SortOptions::default(), + }]; + let finer = vec![ + PhysicalSortExpr { + expr: Arc::new(Column::new("a", 0)), + options: SortOptions::default(), + }, + PhysicalSortExpr { + expr: Arc::new(Column::new("b", 1)), + options: SortOptions::default(), + }, + ]; + // finer ordering satisfies, crude ordering should return true + let empty_schema = &Arc::new(Schema::empty()); + let mut eq_properties_finer = EquivalenceProperties::new(empty_schema.clone()); + eq_properties_finer.oeq_class.push(finer.clone()); + assert!(eq_properties_finer.ordering_satisfy(&crude)); + + // Crude ordering doesn't satisfy finer ordering. should return false + let mut eq_properties_crude = EquivalenceProperties::new(empty_schema.clone()); + eq_properties_crude.oeq_class.push(crude.clone()); + assert!(!eq_properties_crude.ordering_satisfy(&finer)); Ok(()) } #[test] - fn test_get_compatible_ranges() -> Result<()> { - let col_a = &Column::new("a", 0); - let col_b = &Column::new("b", 1); - let option1 = SortOptions { + fn test_ordering_satisfy_with_equivalence() -> Result<()> { + // Schema satisfies following orderings: + // [a ASC], [d ASC, b ASC], [e DESC, f ASC, g ASC] + // and + // Column [a=c] (e.g they are aliases). + let (test_schema, eq_properties) = create_test_params()?; + let col_a = &col("a", &test_schema)?; + let col_b = &col("b", &test_schema)?; + let col_c = &col("c", &test_schema)?; + let col_d = &col("d", &test_schema)?; + let col_e = &col("e", &test_schema)?; + let col_f = &col("f", &test_schema)?; + let col_g = &col("g", &test_schema)?; + let option_asc = SortOptions { descending: false, nulls_first: false, }; - let test_data = vec![ + let option_desc = SortOptions { + descending: true, + nulls_first: true, + }; + let table_data_with_properties = + generate_table_for_eq_properties(&eq_properties, 625, 5)?; + + // First element in the tuple stores vector of requirement, second element is the expected return value for ordering_satisfy function + let requirements = vec![ + // `a ASC NULLS LAST`, expects `ordering_satisfy` to be `true`, since existing ordering `a ASC NULLS LAST, b ASC NULLS LAST` satisfies it + (vec![(col_a, option_asc)], true), + (vec![(col_a, option_desc)], false), + // Test whether equivalence works as expected + (vec![(col_c, option_asc)], true), + (vec![(col_c, option_desc)], false), + // Test whether ordering equivalence works as expected + (vec![(col_d, option_asc)], true), + (vec![(col_d, option_asc), (col_b, option_asc)], true), + (vec![(col_d, option_desc), (col_b, option_asc)], false), ( - vec![(col_a, Some(option1)), (col_b, Some(option1))], - vec![(col_a, Some(option1))], - vec![(0, 1)], + vec![ + (col_e, option_desc), + (col_f, option_asc), + (col_g, option_asc), + ], + true, ), + (vec![(col_e, option_desc), (col_f, option_asc)], true), + (vec![(col_e, option_asc), (col_f, option_asc)], false), + (vec![(col_e, option_desc), (col_b, option_asc)], false), + (vec![(col_e, option_asc), (col_b, option_asc)], false), ( - vec![(col_a, None), (col_b, Some(option1))], - vec![(col_a, Some(option1))], - vec![(0, 1)], + vec![ + (col_d, option_asc), + (col_b, option_asc), + (col_d, option_asc), + (col_b, option_asc), + ], + true, ), ( vec![ - (col_a, None), - (col_b, Some(option1)), - (col_a, Some(option1)), + (col_d, option_asc), + (col_b, option_asc), + (col_e, option_desc), + (col_f, option_asc), ], - vec![(col_a, Some(option1))], - vec![(0, 1), (2, 3)], + true, ), - ]; - for (searched, to_search, expected) in test_data { - let searched = convert_to_requirement(&searched); - let to_search = convert_to_requirement(&to_search); - let expected = expected - .into_iter() - .map(|(start, end)| Range { start, end }) - .collect::>(); - assert_eq!(get_compatible_ranges(&searched, &to_search), expected); - } - Ok(()) - } - - #[test] - fn test_get_updated_right_ordering_equivalence_properties() -> Result<()> { - let join_type = JoinType::Inner; - - let options = SortOptions::default(); - let right_oeq_class = OrderingEquivalentClass::new( - vec![ - PhysicalSortExpr { - expr: Arc::new(Column::new("x", 0)), - options, - }, - PhysicalSortExpr { - expr: Arc::new(Column::new("y", 1)), - options, - }, - ], - vec![vec![ - PhysicalSortExpr { - expr: Arc::new(Column::new("z", 2)), - options, - }, - PhysicalSortExpr { - expr: Arc::new(Column::new("w", 3)), - options, - }, - ]], - ); - + ( + vec![ + (col_d, option_asc), + (col_b, option_asc), + (col_e, option_desc), + (col_b, option_asc), + ], + true, + ), + ( + vec![ + (col_d, option_asc), + (col_b, option_asc), + (col_d, option_desc), + (col_b, option_asc), + ], + true, + ), + ( + vec![ + (col_d, option_asc), + (col_b, option_asc), + (col_e, option_asc), + (col_f, option_asc), + ], + false, + ), + ( + vec![ + (col_d, option_asc), + (col_b, option_asc), + (col_e, option_asc), + (col_b, option_asc), + ], + false, + ), + (vec![(col_d, option_asc), (col_e, option_desc)], true), + ( + vec![ + (col_d, option_asc), + (col_c, option_asc), + (col_b, option_asc), + ], + true, + ), + ( + vec![ + (col_d, option_asc), + (col_e, option_desc), + (col_f, option_asc), + (col_b, option_asc), + ], + true, + ), + ( + vec![ + (col_d, option_asc), + (col_e, option_desc), + (col_c, option_asc), + (col_b, option_asc), + ], + true, + ), + ( + vec![ + (col_d, option_asc), + (col_e, option_desc), + (col_b, option_asc), + (col_f, option_asc), + ], + true, + ), + ]; + + for (cols, expected) in requirements { + let err_msg = format!("Error in test case:{cols:?}"); + let required = cols + .into_iter() + .map(|(expr, options)| PhysicalSortExpr { + expr: expr.clone(), + options, + }) + .collect::>(); + + // Check expected result with experimental result. + assert_eq!( + is_table_same_after_sort( + required.clone(), + table_data_with_properties.clone() + )?, + expected + ); + assert_eq!( + eq_properties.ordering_satisfy(&required), + expected, + "{err_msg}" + ); + } + Ok(()) + } + + #[test] + fn test_ordering_satisfy_with_equivalence_random() -> Result<()> { + const N_RANDOM_SCHEMA: usize = 5; + const N_ELEMENTS: usize = 125; + const N_DISTINCT: usize = 5; + const SORT_OPTIONS: SortOptions = SortOptions { + descending: false, + nulls_first: false, + }; + + for seed in 0..N_RANDOM_SCHEMA { + // Create a random schema with random properties + let (test_schema, eq_properties) = create_random_schema(seed as u64)?; + // Generate a data that satisfies properties given + let table_data_with_properties = + generate_table_for_eq_properties(&eq_properties, N_ELEMENTS, N_DISTINCT)?; + let col_exprs = vec![ + col("a", &test_schema)?, + col("b", &test_schema)?, + col("c", &test_schema)?, + col("d", &test_schema)?, + col("e", &test_schema)?, + col("f", &test_schema)?, + ]; + + for n_req in 0..=col_exprs.len() { + for exprs in col_exprs.iter().combinations(n_req) { + let requirement = exprs + .into_iter() + .map(|expr| PhysicalSortExpr { + expr: expr.clone(), + options: SORT_OPTIONS, + }) + .collect::>(); + let expected = is_table_same_after_sort( + requirement.clone(), + table_data_with_properties.clone(), + )?; + let err_msg = format!( + "Error in test case requirement:{:?}, expected: {:?}", + requirement, expected + ); + // Check whether ordering_satisfy API result and + // experimental result matches. + assert_eq!( + eq_properties.ordering_satisfy(&requirement), + expected, + "{}", + err_msg + ); + } + } + } + + Ok(()) + } + + #[test] + fn test_ordering_satisfy_different_lengths() -> Result<()> { + let test_schema = create_test_schema()?; + let col_a = &col("a", &test_schema)?; + let col_b = &col("b", &test_schema)?; + let col_c = &col("c", &test_schema)?; + let col_d = &col("d", &test_schema)?; + let col_e = &col("e", &test_schema)?; + let col_f = &col("f", &test_schema)?; + let options = SortOptions { + descending: false, + nulls_first: false, + }; + // a=c (e.g they are aliases). + let mut eq_properties = EquivalenceProperties::new(test_schema); + eq_properties.add_equal_conditions(col_a, col_c); + + let orderings = vec![ + vec![(col_a, options)], + vec![(col_e, options)], + vec![(col_d, options), (col_f, options)], + ]; + let orderings = convert_to_orderings(&orderings); + + // Column [a ASC], [e ASC], [d ASC, f ASC] are all valid orderings for the schema. + eq_properties.add_new_orderings(orderings); + + // First entry in the tuple is required ordering, second entry is the expected flag + // that indicates whether this required ordering is satisfied. + // ([a ASC], true) indicate a ASC requirement is already satisfied by existing orderings. + let test_cases = vec![ + // [c ASC, a ASC, e ASC], expected represents this requirement is satisfied + ( + vec![(col_c, options), (col_a, options), (col_e, options)], + true, + ), + (vec![(col_c, options), (col_b, options)], false), + (vec![(col_c, options), (col_d, options)], true), + ( + vec![(col_d, options), (col_f, options), (col_b, options)], + false, + ), + (vec![(col_d, options), (col_f, options)], true), + ]; + + for (reqs, expected) in test_cases { + let err_msg = + format!("error in test reqs: {:?}, expected: {:?}", reqs, expected,); + let reqs = convert_to_sort_exprs(&reqs); + assert_eq!( + eq_properties.ordering_satisfy(&reqs), + expected, + "{}", + err_msg + ); + } + + Ok(()) + } + + #[test] + fn test_bridge_groups() -> Result<()> { + // First entry in the tuple is argument, second entry is the bridged result + let test_cases = vec![ + // ------- TEST CASE 1 -----------// + ( + vec![vec![1, 2, 3], vec![2, 4, 5], vec![11, 12, 9], vec![7, 6, 5]], + // Expected is compared with set equality. Order of the specific results may change. + vec![vec![1, 2, 3, 4, 5, 6, 7], vec![9, 11, 12]], + ), + // ------- TEST CASE 2 -----------// + ( + vec![vec![1, 2, 3], vec![3, 4, 5], vec![9, 8, 7], vec![7, 6, 5]], + // Expected + vec![vec![1, 2, 3, 4, 5, 6, 7, 8, 9]], + ), + ]; + for (entries, expected) in test_cases { + let entries = entries + .into_iter() + .map(|entry| entry.into_iter().map(lit).collect::>()) + .collect::>(); + let expected = expected + .into_iter() + .map(|entry| entry.into_iter().map(lit).collect::>()) + .collect::>(); + let mut eq_groups = EquivalenceGroup::new(entries.clone()); + eq_groups.bridge_classes(); + let eq_groups = eq_groups.classes; + let err_msg = format!( + "error in test entries: {:?}, expected: {:?}, actual:{:?}", + entries, expected, eq_groups + ); + assert_eq!(eq_groups.len(), expected.len(), "{}", err_msg); + for idx in 0..eq_groups.len() { + assert!( + physical_exprs_bag_equal(&eq_groups[idx], &expected[idx]), + "{}", + err_msg + ); + } + } + Ok(()) + } + + #[test] + fn test_remove_redundant_entries_eq_group() -> Result<()> { + let entries = vec![ + vec![lit(1), lit(1), lit(2)], + // This group is meaningless should be removed + vec![lit(3), lit(3)], + vec![lit(4), lit(5), lit(6)], + ]; + // Given equivalences classes are not in succinct form. + // Expected form is the most plain representation that is functionally same. + let expected = vec![vec![lit(1), lit(2)], vec![lit(4), lit(5), lit(6)]]; + let mut eq_groups = EquivalenceGroup::new(entries); + eq_groups.remove_redundant_entries(); + + let eq_groups = eq_groups.classes; + assert_eq!(eq_groups.len(), expected.len()); + assert_eq!(eq_groups.len(), 2); + + assert!(physical_exprs_equal(&eq_groups[0], &expected[0])); + assert!(physical_exprs_equal(&eq_groups[1], &expected[1])); + Ok(()) + } + + #[test] + fn test_remove_redundant_entries_oeq_class() -> Result<()> { + let schema = create_test_schema()?; + let col_a = &col("a", &schema)?; + let col_b = &col("b", &schema)?; + let col_c = &col("c", &schema)?; + + let option_asc = SortOptions { + descending: false, + nulls_first: false, + }; + let option_desc = SortOptions { + descending: true, + nulls_first: true, + }; + + // First entry in the tuple is the given orderings for the table + // Second entry is the simplest version of the given orderings that is functionally equivalent. + let test_cases = vec![ + // ------- TEST CASE 1 --------- + ( + // ORDERINGS GIVEN + vec![ + // [a ASC, b ASC] + vec![(col_a, option_asc), (col_b, option_asc)], + ], + // EXPECTED orderings that is succinct. + vec![ + // [a ASC, b ASC] + vec![(col_a, option_asc), (col_b, option_asc)], + ], + ), + // ------- TEST CASE 2 --------- + ( + // ORDERINGS GIVEN + vec![ + // [a ASC, b ASC] + vec![(col_a, option_asc), (col_b, option_asc)], + // [a ASC, b ASC, c ASC] + vec![ + (col_a, option_asc), + (col_b, option_asc), + (col_c, option_asc), + ], + ], + // EXPECTED orderings that is succinct. + vec![ + // [a ASC, b ASC, c ASC] + vec![ + (col_a, option_asc), + (col_b, option_asc), + (col_c, option_asc), + ], + ], + ), + // ------- TEST CASE 3 --------- + ( + // ORDERINGS GIVEN + vec![ + // [a ASC, b DESC] + vec![(col_a, option_asc), (col_b, option_desc)], + // [a ASC] + vec![(col_a, option_asc)], + // [a ASC, c ASC] + vec![(col_a, option_asc), (col_c, option_asc)], + ], + // EXPECTED orderings that is succinct. + vec![ + // [a ASC, b DESC] + vec![(col_a, option_asc), (col_b, option_desc)], + // [a ASC, c ASC] + vec![(col_a, option_asc), (col_c, option_asc)], + ], + ), + // ------- TEST CASE 4 --------- + ( + // ORDERINGS GIVEN + vec![ + // [a ASC, b ASC] + vec![(col_a, option_asc), (col_b, option_asc)], + // [a ASC, b ASC, c ASC] + vec![ + (col_a, option_asc), + (col_b, option_asc), + (col_c, option_asc), + ], + // [a ASC] + vec![(col_a, option_asc)], + ], + // EXPECTED orderings that is succinct. + vec![ + // [a ASC, b ASC, c ASC] + vec![ + (col_a, option_asc), + (col_b, option_asc), + (col_c, option_asc), + ], + ], + ), + ]; + for (orderings, expected) in test_cases { + let orderings = convert_to_orderings(&orderings); + let expected = convert_to_orderings(&expected); + let actual = OrderingEquivalenceClass::new(orderings.clone()); + let actual = actual.orderings; + let err_msg = format!( + "orderings: {:?}, expected: {:?}, actual :{:?}", + orderings, expected, actual + ); + assert_eq!(actual.len(), expected.len(), "{}", err_msg); + for elem in actual { + assert!(expected.contains(&elem), "{}", err_msg); + } + } + + Ok(()) + } + + #[test] + fn test_get_updated_right_ordering_equivalence_properties() -> Result<()> { + let join_type = JoinType::Inner; + // Join right child schema + let child_fields: Fields = ["x", "y", "z", "w"] + .into_iter() + .map(|name| Field::new(name, DataType::Int32, true)) + .collect(); + let child_schema = Schema::new(child_fields); + let col_x = &col("x", &child_schema)?; + let col_y = &col("y", &child_schema)?; + let col_z = &col("z", &child_schema)?; + let col_w = &col("w", &child_schema)?; + let option_asc = SortOptions { + descending: false, + nulls_first: false, + }; + // [x ASC, y ASC], [z ASC, w ASC] + let orderings = vec![ + vec![(col_x, option_asc), (col_y, option_asc)], + vec![(col_z, option_asc), (col_w, option_asc)], + ]; + let orderings = convert_to_orderings(&orderings); + // Right child ordering equivalences + let mut right_oeq_class = OrderingEquivalenceClass::new(orderings); + let left_columns_len = 4; let fields: Fields = ["a", "b", "c", "d", "x", "y", "z", "w"] @@ -1403,45 +1990,578 @@ mod tests { .map(|name| Field::new(name, DataType::Int32, true)) .collect(); - let mut join_eq_properties = - EquivalenceProperties::new(Arc::new(Schema::new(fields))); - join_eq_properties - .add_equal_conditions((&Column::new("a", 0), &Column::new("x", 4))); - join_eq_properties - .add_equal_conditions((&Column::new("d", 3), &Column::new("w", 7))); - - let result = get_updated_right_ordering_equivalent_class( + // Join Schema + let schema = Schema::new(fields); + let col_a = &col("a", &schema)?; + let col_d = &col("d", &schema)?; + let col_x = &col("x", &schema)?; + let col_y = &col("y", &schema)?; + let col_z = &col("z", &schema)?; + let col_w = &col("w", &schema)?; + + let mut join_eq_properties = EquivalenceProperties::new(Arc::new(schema)); + // a=x and d=w + join_eq_properties.add_equal_conditions(col_a, col_x); + join_eq_properties.add_equal_conditions(col_d, col_w); + + updated_right_ordering_equivalence_class( + &mut right_oeq_class, &join_type, - &right_oeq_class, left_columns_len, - &join_eq_properties, ); + join_eq_properties.add_ordering_equivalence_class(right_oeq_class); + let result = join_eq_properties.oeq_class().clone(); - let expected = OrderingEquivalentClass::new( - vec![ - PhysicalSortExpr { - expr: Arc::new(Column::new("a", 0)), - options, - }, - PhysicalSortExpr { - expr: Arc::new(Column::new("y", 5)), - options, - }, - ], - vec![vec![ - PhysicalSortExpr { - expr: Arc::new(Column::new("z", 6)), - options, - }, - PhysicalSortExpr { - expr: Arc::new(Column::new("d", 3)), - options, - }, - ]], - ); + // [x ASC, y ASC], [z ASC, w ASC] + let orderings = vec![ + vec![(col_x, option_asc), (col_y, option_asc)], + vec![(col_z, option_asc), (col_w, option_asc)], + ]; + let orderings = convert_to_orderings(&orderings); + let expected = OrderingEquivalenceClass::new(orderings); + + assert_eq!(result, expected); + + Ok(()) + } + + /// Checks if the table (RecordBatch) remains unchanged when sorted according to the provided `required_ordering`. + /// + /// The function works by adding a unique column of ascending integers to the original table. This column ensures + /// that rows that are otherwise indistinguishable (e.g., if they have the same values in all other columns) can + /// still be differentiated. When sorting the extended table, the unique column acts as a tie-breaker to produce + /// deterministic sorting results. + /// + /// If the table remains the same after sorting with the added unique column, it indicates that the table was + /// already sorted according to `required_ordering` to begin with. + fn is_table_same_after_sort( + mut required_ordering: Vec, + batch: RecordBatch, + ) -> Result { + // Clone the original schema and columns + let original_schema = batch.schema(); + let mut columns = batch.columns().to_vec(); + + // Create a new unique column + let n_row = batch.num_rows() as u64; + let unique_col = Arc::new(UInt64Array::from_iter_values(0..n_row)) as ArrayRef; + columns.push(unique_col.clone()); + + // Create a new schema with the added unique column + let unique_col_name = "unique"; + let unique_field = Arc::new(Field::new(unique_col_name, DataType::UInt64, false)); + let fields: Vec<_> = original_schema + .fields() + .iter() + .cloned() + .chain(std::iter::once(unique_field)) + .collect(); + let schema = Arc::new(Schema::new(fields)); + + // Create a new batch with the added column + let new_batch = RecordBatch::try_new(schema.clone(), columns)?; + + // Add the unique column to the required ordering to ensure deterministic results + required_ordering.push(PhysicalSortExpr { + expr: Arc::new(Column::new(unique_col_name, original_schema.fields().len())), + options: Default::default(), + }); + + // Convert the required ordering to a list of SortColumn + let sort_columns: Vec<_> = required_ordering + .iter() + .filter_map(|order_expr| { + let col = order_expr.expr.as_any().downcast_ref::()?; + let col_index = schema.column_with_name(col.name())?.0; + Some(SortColumn { + values: new_batch.column(col_index).clone(), + options: Some(order_expr.options), + }) + }) + .collect(); + + // Check if the indices after sorting match the initial ordering + let sorted_indices = lexsort_to_indices(&sort_columns, None)?; + let original_indices = UInt32Array::from_iter_values(0..n_row as u32); + + Ok(sorted_indices == original_indices) + } + + // If we already generated a random result for one of the + // expressions in the equivalence classes. For other expressions in the same + // equivalence class use same result. This util gets already calculated result, when available. + fn get_representative_arr( + eq_group: &[Arc], + existing_vec: &[Option], + schema: SchemaRef, + ) -> Option { + for expr in eq_group.iter() { + let col = expr.as_any().downcast_ref::().unwrap(); + let (idx, _field) = schema.column_with_name(col.name()).unwrap(); + if let Some(res) = &existing_vec[idx] { + return Some(res.clone()); + } + } + None + } + + // Generate a table that satisfies the given equivalence properties; i.e. + // equivalences, ordering equivalences, and constants. + fn generate_table_for_eq_properties( + eq_properties: &EquivalenceProperties, + n_elem: usize, + n_distinct: usize, + ) -> Result { + let mut rng = StdRng::seed_from_u64(23); + + let schema = eq_properties.schema(); + let mut schema_vec = vec![None; schema.fields.len()]; + + // Utility closure to generate random array + let mut generate_random_array = |num_elems: usize, max_val: usize| -> ArrayRef { + let values: Vec = (0..num_elems) + .map(|_| rng.gen_range(0..max_val) as u64) + .collect(); + Arc::new(UInt64Array::from_iter_values(values)) + }; + + // Fill constant columns + for constant in &eq_properties.constants { + let col = constant.as_any().downcast_ref::().unwrap(); + let (idx, _field) = schema.column_with_name(col.name()).unwrap(); + let arr = + Arc::new(UInt64Array::from_iter_values(vec![0; n_elem])) as ArrayRef; + schema_vec[idx] = Some(arr); + } + + // Fill columns based on ordering equivalences + for ordering in eq_properties.oeq_class.iter() { + let (sort_columns, indices): (Vec<_>, Vec<_>) = ordering + .iter() + .map(|PhysicalSortExpr { expr, options }| { + let col = expr.as_any().downcast_ref::().unwrap(); + let (idx, _field) = schema.column_with_name(col.name()).unwrap(); + let arr = generate_random_array(n_elem, n_distinct); + ( + SortColumn { + values: arr, + options: Some(*options), + }, + idx, + ) + }) + .unzip(); + + let sort_arrs = arrow::compute::lexsort(&sort_columns, None)?; + for (idx, arr) in izip!(indices, sort_arrs) { + schema_vec[idx] = Some(arr); + } + } + + // Fill columns based on equivalence groups + for eq_group in eq_properties.eq_group.iter() { + let representative_array = + get_representative_arr(eq_group, &schema_vec, schema.clone()) + .unwrap_or_else(|| generate_random_array(n_elem, n_distinct)); + + for expr in eq_group { + let col = expr.as_any().downcast_ref::().unwrap(); + let (idx, _field) = schema.column_with_name(col.name()).unwrap(); + schema_vec[idx] = Some(representative_array.clone()); + } + } - assert_eq!(result.head(), expected.head()); - assert_eq!(result.others(), expected.others()); + let res: Vec<_> = schema_vec + .into_iter() + .zip(schema.fields.iter()) + .map(|(elem, field)| { + ( + field.name(), + // Generate random values for columns that do not occur in any of the groups (equivalence, ordering equivalence, constants) + elem.unwrap_or_else(|| generate_random_array(n_elem, n_distinct)), + ) + }) + .collect(); + + Ok(RecordBatch::try_from_iter(res)?) + } + + #[test] + fn test_schema_normalize_expr_with_equivalence() -> Result<()> { + let col_a = &Column::new("a", 0); + let col_b = &Column::new("b", 1); + let col_c = &Column::new("c", 2); + // Assume that column a and c are aliases. + let (_test_schema, eq_properties) = create_test_params()?; + + let col_a_expr = Arc::new(col_a.clone()) as Arc; + let col_b_expr = Arc::new(col_b.clone()) as Arc; + let col_c_expr = Arc::new(col_c.clone()) as Arc; + // Test cases for equivalence normalization, + // First entry in the tuple is argument, second entry is expected result after normalization. + let expressions = vec![ + // Normalized version of the column a and c should go to a + // (by convention all the expressions inside equivalence class are mapped to the first entry + // in this case a is the first entry in the equivalence class.) + (&col_a_expr, &col_a_expr), + (&col_c_expr, &col_a_expr), + // Cannot normalize column b + (&col_b_expr, &col_b_expr), + ]; + let eq_group = eq_properties.eq_group(); + for (expr, expected_eq) in expressions { + assert!( + expected_eq.eq(&eq_group.normalize_expr(expr.clone())), + "error in test: expr: {expr:?}" + ); + } + + Ok(()) + } + + #[test] + fn test_schema_normalize_sort_requirement_with_equivalence() -> Result<()> { + let option1 = SortOptions { + descending: false, + nulls_first: false, + }; + // Assume that column a and c are aliases. + let (test_schema, eq_properties) = create_test_params()?; + let col_a = &col("a", &test_schema)?; + let col_c = &col("c", &test_schema)?; + let col_d = &col("d", &test_schema)?; + + // Test cases for equivalence normalization + // First entry in the tuple is PhysicalSortRequirement, second entry in the tuple is + // expected PhysicalSortRequirement after normalization. + let test_cases = vec![ + (vec![(col_a, Some(option1))], vec![(col_a, Some(option1))]), + // In the normalized version column c should be replace with column a + (vec![(col_c, Some(option1))], vec![(col_a, Some(option1))]), + (vec![(col_c, None)], vec![(col_a, None)]), + (vec![(col_d, Some(option1))], vec![(col_d, Some(option1))]), + ]; + for (reqs, expected) in test_cases.into_iter() { + let reqs = convert_to_sort_reqs(&reqs); + let expected = convert_to_sort_reqs(&expected); + + let normalized = eq_properties.normalize_sort_requirements(&reqs); + assert!( + expected.eq(&normalized), + "error in test: reqs: {reqs:?}, expected: {expected:?}, normalized: {normalized:?}" + ); + } + + Ok(()) + } + + #[test] + fn test_normalize_sort_reqs() -> Result<()> { + // Schema satisfies following properties + // a=c + // and following orderings are valid + // [a ASC], [d ASC, b ASC], [e DESC, f ASC, g ASC] + let (test_schema, eq_properties) = create_test_params()?; + let col_a = &col("a", &test_schema)?; + let col_b = &col("b", &test_schema)?; + let col_c = &col("c", &test_schema)?; + let col_d = &col("d", &test_schema)?; + let col_e = &col("e", &test_schema)?; + let col_f = &col("f", &test_schema)?; + let option_asc = SortOptions { + descending: false, + nulls_first: false, + }; + let option_desc = SortOptions { + descending: true, + nulls_first: true, + }; + // First element in the tuple stores vector of requirement, second element is the expected return value for ordering_satisfy function + let requirements = vec![ + ( + vec![(col_a, Some(option_asc))], + vec![(col_a, Some(option_asc))], + ), + ( + vec![(col_a, Some(option_desc))], + vec![(col_a, Some(option_desc))], + ), + (vec![(col_a, None)], vec![(col_a, None)]), + // Test whether equivalence works as expected + ( + vec![(col_c, Some(option_asc))], + vec![(col_a, Some(option_asc))], + ), + (vec![(col_c, None)], vec![(col_a, None)]), + // Test whether ordering equivalence works as expected + ( + vec![(col_d, Some(option_asc)), (col_b, Some(option_asc))], + vec![(col_d, Some(option_asc)), (col_b, Some(option_asc))], + ), + ( + vec![(col_d, None), (col_b, None)], + vec![(col_d, None), (col_b, None)], + ), + ( + vec![(col_e, Some(option_desc)), (col_f, Some(option_asc))], + vec![(col_e, Some(option_desc)), (col_f, Some(option_asc))], + ), + // We should be able to normalize in compatible requirements also (not exactly equal) + ( + vec![(col_e, Some(option_desc)), (col_f, None)], + vec![(col_e, Some(option_desc)), (col_f, None)], + ), + ( + vec![(col_e, None), (col_f, None)], + vec![(col_e, None), (col_f, None)], + ), + ]; + + for (reqs, expected_normalized) in requirements.into_iter() { + let req = convert_to_sort_reqs(&reqs); + let expected_normalized = convert_to_sort_reqs(&expected_normalized); + + assert_eq!( + eq_properties.normalize_sort_requirements(&req), + expected_normalized + ); + } + + Ok(()) + } + + #[test] + fn test_get_finer() -> Result<()> { + let schema = create_test_schema()?; + let col_a = &col("a", &schema)?; + let col_b = &col("b", &schema)?; + let col_c = &col("c", &schema)?; + let eq_properties = EquivalenceProperties::new(schema); + let option_asc = SortOptions { + descending: false, + nulls_first: false, + }; + let option_desc = SortOptions { + descending: true, + nulls_first: true, + }; + // First entry, and second entry are the physical sort requirement that are argument for get_finer_requirement. + // Third entry is the expected result. + let tests_cases = vec![ + // Get finer requirement between [a Some(ASC)] and [a None, b Some(ASC)] + // result should be [a Some(ASC), b Some(ASC)] + ( + vec![(col_a, Some(option_asc))], + vec![(col_a, None), (col_b, Some(option_asc))], + Some(vec![(col_a, Some(option_asc)), (col_b, Some(option_asc))]), + ), + // Get finer requirement between [a Some(ASC), b Some(ASC), c Some(ASC)] and [a Some(ASC), b Some(ASC)] + // result should be [a Some(ASC), b Some(ASC), c Some(ASC)] + ( + vec![ + (col_a, Some(option_asc)), + (col_b, Some(option_asc)), + (col_c, Some(option_asc)), + ], + vec![(col_a, Some(option_asc)), (col_b, Some(option_asc))], + Some(vec![ + (col_a, Some(option_asc)), + (col_b, Some(option_asc)), + (col_c, Some(option_asc)), + ]), + ), + // Get finer requirement between [a Some(ASC), b Some(ASC)] and [a Some(ASC), b Some(DESC)] + // result should be None + ( + vec![(col_a, Some(option_asc)), (col_b, Some(option_asc))], + vec![(col_a, Some(option_asc)), (col_b, Some(option_desc))], + None, + ), + ]; + for (lhs, rhs, expected) in tests_cases { + let lhs = convert_to_sort_reqs(&lhs); + let rhs = convert_to_sort_reqs(&rhs); + let expected = expected.map(|expected| convert_to_sort_reqs(&expected)); + let finer = eq_properties.get_finer_requirement(&lhs, &rhs); + assert_eq!(finer, expected) + } + + Ok(()) + } + + #[test] + fn test_get_meet_ordering() -> Result<()> { + let schema = create_test_schema()?; + let col_a = &col("a", &schema)?; + let col_b = &col("b", &schema)?; + let eq_properties = EquivalenceProperties::new(schema); + let option_asc = SortOptions { + descending: false, + nulls_first: false, + }; + let option_desc = SortOptions { + descending: true, + nulls_first: true, + }; + let tests_cases = vec![ + // Get meet ordering between [a ASC] and [a ASC, b ASC] + // result should be [a ASC] + ( + vec![(col_a, option_asc)], + vec![(col_a, option_asc), (col_b, option_asc)], + Some(vec![(col_a, option_asc)]), + ), + // Get meet ordering between [a ASC] and [a DESC] + // result should be None. + (vec![(col_a, option_asc)], vec![(col_a, option_desc)], None), + // Get meet ordering between [a ASC, b ASC] and [a ASC, b DESC] + // result should be [a ASC]. + ( + vec![(col_a, option_asc), (col_b, option_asc)], + vec![(col_a, option_asc), (col_b, option_desc)], + Some(vec![(col_a, option_asc)]), + ), + ]; + for (lhs, rhs, expected) in tests_cases { + let lhs = convert_to_sort_exprs(&lhs); + let rhs = convert_to_sort_exprs(&rhs); + let expected = expected.map(|expected| convert_to_sort_exprs(&expected)); + let finer = eq_properties.get_meet_ordering(&lhs, &rhs); + assert_eq!(finer, expected) + } + + Ok(()) + } + + #[test] + fn test_find_longest_permutation() -> Result<()> { + // Schema satisfies following orderings: + // [a ASC], [d ASC, b ASC], [e DESC, f ASC, g ASC] + // and + // Column [a=c] (e.g they are aliases). + // At below we add [d ASC, h DESC] also, for test purposes + let (test_schema, mut eq_properties) = create_test_params()?; + let col_a = &col("a", &test_schema)?; + let col_b = &col("b", &test_schema)?; + let col_c = &col("c", &test_schema)?; + let col_d = &col("d", &test_schema)?; + let col_e = &col("e", &test_schema)?; + let col_h = &col("h", &test_schema)?; + + let option_asc = SortOptions { + descending: false, + nulls_first: false, + }; + let option_desc = SortOptions { + descending: true, + nulls_first: true, + }; + // [d ASC, h ASC] also satisfies schema. + eq_properties.add_new_orderings([vec![ + PhysicalSortExpr { + expr: col_d.clone(), + options: option_asc, + }, + PhysicalSortExpr { + expr: col_h.clone(), + options: option_desc, + }, + ]]); + let test_cases = vec![ + // TEST CASE 1 + (vec![col_a], vec![(col_a, option_asc)]), + // TEST CASE 2 + (vec![col_c], vec![(col_c, option_asc)]), + // TEST CASE 3 + ( + vec![col_d, col_e, col_b], + vec![ + (col_d, option_asc), + (col_b, option_asc), + (col_e, option_desc), + ], + ), + // TEST CASE 4 + (vec![col_b], vec![]), + // TEST CASE 5 + (vec![col_d], vec![(col_d, option_asc)]), + ]; + for (exprs, expected) in test_cases { + let exprs = exprs.into_iter().cloned().collect::>(); + let expected = convert_to_sort_exprs(&expected); + let (actual, _) = eq_properties.find_longest_permutation(&exprs); + assert_eq!(actual, expected); + } + + Ok(()) + } + + #[test] + fn test_update_ordering() -> Result<()> { + let schema = Schema::new(vec![ + Field::new("a", DataType::Int32, true), + Field::new("b", DataType::Int32, true), + Field::new("c", DataType::Int32, true), + Field::new("d", DataType::Int32, true), + ]); + + let mut eq_properties = EquivalenceProperties::new(Arc::new(schema.clone())); + let col_a = &col("a", &schema)?; + let col_b = &col("b", &schema)?; + let col_c = &col("c", &schema)?; + let col_d = &col("d", &schema)?; + let option_asc = SortOptions { + descending: false, + nulls_first: false, + }; + // b=a (e.g they are aliases) + eq_properties.add_equal_conditions(col_b, col_a); + // [b ASC], [d ASC] + eq_properties.add_new_orderings(vec![ + vec![PhysicalSortExpr { + expr: col_b.clone(), + options: option_asc, + }], + vec![PhysicalSortExpr { + expr: col_d.clone(), + options: option_asc, + }], + ]); + + let test_cases = vec![ + // d + b + ( + Arc::new(BinaryExpr::new( + col_d.clone(), + Operator::Plus, + col_b.clone(), + )) as Arc, + SortProperties::Ordered(option_asc), + ), + // b + (col_b.clone(), SortProperties::Ordered(option_asc)), + // a + (col_a.clone(), SortProperties::Ordered(option_asc)), + // a + c + ( + Arc::new(BinaryExpr::new( + col_a.clone(), + Operator::Plus, + col_c.clone(), + )), + SortProperties::Unordered, + ), + ]; + for (expr, expected) in test_cases { + let expr_ordering = ExprOrdering::new(expr.clone()); + let expr_ordering = expr_ordering + .transform_up(&|expr| update_ordering(expr, &eq_properties))?; + let err_msg = format!( + "expr:{:?}, expected: {:?}, actual: {:?}", + expr, expected, expr_ordering.state + ); + assert_eq!(expr_ordering.state, expected, "{}", err_msg); + } Ok(()) } diff --git a/datafusion/physical-expr/src/expressions/in_list.rs b/datafusion/physical-expr/src/expressions/in_list.rs index 643bbfd820a6..8d55fb70bd9e 100644 --- a/datafusion/physical-expr/src/expressions/in_list.rs +++ b/datafusion/physical-expr/src/expressions/in_list.rs @@ -22,8 +22,7 @@ use std::fmt::Debug; use std::hash::{Hash, Hasher}; use std::sync::Arc; -use crate::physical_expr::down_cast_any_ref; -use crate::utils::expr_list_eq_any_order; +use crate::physical_expr::{down_cast_any_ref, physical_exprs_bag_equal}; use crate::PhysicalExpr; use arrow::array::*; @@ -410,7 +409,7 @@ impl PartialEq for InListExpr { .downcast_ref::() .map(|x| { self.expr.eq(&x.expr) - && expr_list_eq_any_order(&self.list, &x.list) + && physical_exprs_bag_equal(&self.list, &x.list) && self.negated == x.negated }) .unwrap_or(false) diff --git a/datafusion/physical-expr/src/lib.rs b/datafusion/physical-expr/src/lib.rs index 977542bd8e66..fffa8f602d87 100644 --- a/datafusion/physical-expr/src/lib.rs +++ b/datafusion/physical-expr/src/lib.rs @@ -53,23 +53,16 @@ pub use aggregate::groups_accumulator::{ }; pub use aggregate::AggregateExpr; pub use analysis::{analyze, AnalysisContext, ExprBoundaries}; -pub use equivalence::{ - add_offset_to_expr, add_offset_to_lex_ordering, - ordering_equivalence_properties_helper, project_equivalence_properties, - project_ordering_equivalence_properties, EquivalenceProperties, EquivalentClass, - OrderingEquivalenceProperties, OrderingEquivalentClass, -}; - +pub use equivalence::EquivalenceProperties; pub use partitioning::{Distribution, Partitioning}; -pub use physical_expr::{physical_exprs_contains, PhysicalExpr, PhysicalExprRef}; +pub use physical_expr::{ + physical_exprs_bag_equal, physical_exprs_contains, physical_exprs_equal, + PhysicalExpr, PhysicalExprRef, +}; pub use planner::create_physical_expr; pub use scalar_function::ScalarFunctionExpr; pub use sort_expr::{ - LexOrdering, LexOrderingRef, LexOrderingReq, PhysicalSortExpr, + LexOrdering, LexOrderingRef, LexRequirement, LexRequirementRef, PhysicalSortExpr, PhysicalSortRequirement, }; -pub use sort_properties::update_ordering; -pub use utils::{ - expr_list_eq_any_order, expr_list_eq_strict_order, - normalize_out_expr_with_columns_map, reverse_order_bys, split_conjunction, -}; +pub use utils::{reverse_order_bys, split_conjunction}; diff --git a/datafusion/physical-expr/src/partitioning.rs b/datafusion/physical-expr/src/partitioning.rs index 773eac40dc8a..6a8fca4a1543 100644 --- a/datafusion/physical-expr/src/partitioning.rs +++ b/datafusion/physical-expr/src/partitioning.rs @@ -20,7 +20,7 @@ use std::fmt; use std::sync::Arc; -use crate::{expr_list_eq_strict_order, EquivalenceProperties, PhysicalExpr}; +use crate::{physical_exprs_equal, EquivalenceProperties, PhysicalExpr}; /// Partitioning schemes supported by operators. #[derive(Debug, Clone)] @@ -66,7 +66,7 @@ impl Partitioning { pub fn satisfy EquivalenceProperties>( &self, required: Distribution, - equal_properties: F, + eq_properties: F, ) -> bool { match required { Distribution::UnspecifiedDistribution => true, @@ -78,31 +78,28 @@ impl Partitioning { // then we need to have the partition count and hash functions validation. Partitioning::Hash(partition_exprs, _) => { let fast_match = - expr_list_eq_strict_order(&required_exprs, partition_exprs); + physical_exprs_equal(&required_exprs, partition_exprs); // If the required exprs do not match, need to leverage the eq_properties provided by the child - // and normalize both exprs based on the eq_properties + // and normalize both exprs based on the equivalent groups. if !fast_match { - let eq_properties = equal_properties(); - let eq_classes = eq_properties.classes(); - if !eq_classes.is_empty() { + let eq_properties = eq_properties(); + let eq_groups = eq_properties.eq_group(); + if !eq_groups.is_empty() { let normalized_required_exprs = required_exprs .iter() - .map(|e| eq_properties.normalize_expr(e.clone())) + .map(|e| eq_groups.normalize_expr(e.clone())) .collect::>(); let normalized_partition_exprs = partition_exprs .iter() - .map(|e| eq_properties.normalize_expr(e.clone())) + .map(|e| eq_groups.normalize_expr(e.clone())) .collect::>(); - expr_list_eq_strict_order( + return physical_exprs_equal( &normalized_required_exprs, &normalized_partition_exprs, - ) - } else { - fast_match + ); } - } else { - fast_match } + fast_match } _ => false, } @@ -120,7 +117,7 @@ impl PartialEq for Partitioning { Partitioning::RoundRobinBatch(count2), ) if count1 == count2 => true, (Partitioning::Hash(exprs1, count1), Partitioning::Hash(exprs2, count2)) - if expr_list_eq_strict_order(exprs1, exprs2) && (count1 == count2) => + if physical_exprs_equal(exprs1, exprs2) && (count1 == count2) => { true } @@ -158,15 +155,13 @@ impl Distribution { #[cfg(test)] mod tests { - use crate::expressions::Column; + use std::sync::Arc; use super::*; - use arrow::datatypes::DataType; - use arrow::datatypes::Field; - use arrow::datatypes::Schema; - use datafusion_common::Result; + use crate::expressions::Column; - use std::sync::Arc; + use arrow::datatypes::{DataType, Field, Schema}; + use datafusion_common::Result; #[test] fn partitioning_satisfy_distribution() -> Result<()> { diff --git a/datafusion/physical-expr/src/physical_expr.rs b/datafusion/physical-expr/src/physical_expr.rs index 11fa6c899621..79cbe6828b64 100644 --- a/datafusion/physical-expr/src/physical_expr.rs +++ b/datafusion/physical-expr/src/physical_expr.rs @@ -32,6 +32,8 @@ use datafusion_common::utils::DataPtr; use datafusion_common::{internal_err, not_impl_err, DataFusionError, Result}; use datafusion_expr::ColumnarValue; +use itertools::izip; + /// Expression that can be evaluated against a RecordBatch /// A Physical expression knows its type, nullability and how to evaluate itself. pub trait PhysicalExpr: Send + Sync + Display + Debug + PartialEq { @@ -226,17 +228,79 @@ pub fn physical_exprs_contains( .any(|physical_expr| physical_expr.eq(expr)) } +/// Checks whether the given slices have any common entries. +pub fn have_common_entries( + lhs: &[Arc], + rhs: &[Arc], +) -> bool { + lhs.iter().any(|expr| physical_exprs_contains(rhs, expr)) +} + +/// Checks whether the given physical expression slices are equal. +pub fn physical_exprs_equal( + lhs: &[Arc], + rhs: &[Arc], +) -> bool { + lhs.len() == rhs.len() && izip!(lhs, rhs).all(|(lhs, rhs)| lhs.eq(rhs)) +} + +/// Checks whether the given physical expression slices are equal in the sense +/// of bags (multi-sets), disregarding their orderings. +pub fn physical_exprs_bag_equal( + lhs: &[Arc], + rhs: &[Arc], +) -> bool { + // TODO: Once we can use `HashMap`s with `Arc`, this + // function should use a `HashMap` to reduce computational complexity. + if lhs.len() == rhs.len() { + let mut rhs_vec = rhs.to_vec(); + for expr in lhs { + if let Some(idx) = rhs_vec.iter().position(|e| expr.eq(e)) { + rhs_vec.swap_remove(idx); + } else { + return false; + } + } + true + } else { + false + } +} + +/// This utility function removes duplicates from the given `exprs` vector. +/// Note that this function does not necessarily preserve its input ordering. +pub fn deduplicate_physical_exprs(exprs: &mut Vec>) { + // TODO: Once we can use `HashSet`s with `Arc`, this + // function should use a `HashSet` to reduce computational complexity. + // See issue: https://github.com/apache/arrow-datafusion/issues/8027 + let mut idx = 0; + while idx < exprs.len() { + let mut rest_idx = idx + 1; + while rest_idx < exprs.len() { + if exprs[idx].eq(&exprs[rest_idx]) { + exprs.swap_remove(rest_idx); + } else { + rest_idx += 1; + } + } + idx += 1; + } +} + #[cfg(test)] mod tests { use std::sync::Arc; use crate::expressions::{Column, Literal}; - use crate::physical_expr::{physical_exprs_contains, PhysicalExpr}; + use crate::physical_expr::{ + deduplicate_physical_exprs, have_common_entries, physical_exprs_bag_equal, + physical_exprs_contains, physical_exprs_equal, PhysicalExpr, + }; - use datafusion_common::{Result, ScalarValue}; + use datafusion_common::ScalarValue; #[test] - fn test_physical_exprs_contains() -> Result<()> { + fn test_physical_exprs_contains() { let lit_true = Arc::new(Literal::new(ScalarValue::Boolean(Some(true)))) as Arc; let lit_false = Arc::new(Literal::new(ScalarValue::Boolean(Some(false)))) @@ -268,6 +332,142 @@ mod tests { // below expressions are not inside physical_exprs assert!(!physical_exprs_contains(&physical_exprs, &col_c_expr)); assert!(!physical_exprs_contains(&physical_exprs, &lit1)); - Ok(()) + } + + #[test] + fn test_have_common_entries() { + let lit_true = Arc::new(Literal::new(ScalarValue::Boolean(Some(true)))) + as Arc; + let lit_false = Arc::new(Literal::new(ScalarValue::Boolean(Some(false)))) + as Arc; + let lit2 = + Arc::new(Literal::new(ScalarValue::Int32(Some(2)))) as Arc; + let lit1 = + Arc::new(Literal::new(ScalarValue::Int32(Some(1)))) as Arc; + let col_b_expr = Arc::new(Column::new("b", 1)) as Arc; + + let vec1 = vec![lit_true.clone(), lit_false.clone()]; + let vec2 = vec![lit_true.clone(), col_b_expr.clone()]; + let vec3 = vec![lit2.clone(), lit1.clone()]; + + // lit_true is common + assert!(have_common_entries(&vec1, &vec2)); + // there is no common entry + assert!(!have_common_entries(&vec1, &vec3)); + assert!(!have_common_entries(&vec2, &vec3)); + } + + #[test] + fn test_physical_exprs_equal() { + let lit_true = Arc::new(Literal::new(ScalarValue::Boolean(Some(true)))) + as Arc; + let lit_false = Arc::new(Literal::new(ScalarValue::Boolean(Some(false)))) + as Arc; + let lit1 = + Arc::new(Literal::new(ScalarValue::Int32(Some(1)))) as Arc; + let lit2 = + Arc::new(Literal::new(ScalarValue::Int32(Some(2)))) as Arc; + let col_b_expr = Arc::new(Column::new("b", 1)) as Arc; + + let vec1 = vec![lit_true.clone(), lit_false.clone()]; + let vec2 = vec![lit_true.clone(), col_b_expr.clone()]; + let vec3 = vec![lit2.clone(), lit1.clone()]; + let vec4 = vec![lit_true.clone(), lit_false.clone()]; + + // these vectors are same + assert!(physical_exprs_equal(&vec1, &vec1)); + assert!(physical_exprs_equal(&vec1, &vec4)); + assert!(physical_exprs_bag_equal(&vec1, &vec1)); + assert!(physical_exprs_bag_equal(&vec1, &vec4)); + + // these vectors are different + assert!(!physical_exprs_equal(&vec1, &vec2)); + assert!(!physical_exprs_equal(&vec1, &vec3)); + assert!(!physical_exprs_bag_equal(&vec1, &vec2)); + assert!(!physical_exprs_bag_equal(&vec1, &vec3)); + } + + #[test] + fn test_physical_exprs_set_equal() { + let list1: Vec> = vec![ + Arc::new(Column::new("a", 0)), + Arc::new(Column::new("a", 0)), + Arc::new(Column::new("b", 1)), + ]; + let list2: Vec> = vec![ + Arc::new(Column::new("b", 1)), + Arc::new(Column::new("b", 1)), + Arc::new(Column::new("a", 0)), + ]; + assert!(!physical_exprs_bag_equal( + list1.as_slice(), + list2.as_slice() + )); + assert!(!physical_exprs_bag_equal( + list2.as_slice(), + list1.as_slice() + )); + assert!(!physical_exprs_equal(list1.as_slice(), list2.as_slice())); + assert!(!physical_exprs_equal(list2.as_slice(), list1.as_slice())); + + let list3: Vec> = vec![ + Arc::new(Column::new("a", 0)), + Arc::new(Column::new("b", 1)), + Arc::new(Column::new("c", 2)), + Arc::new(Column::new("a", 0)), + Arc::new(Column::new("b", 1)), + ]; + let list4: Vec> = vec![ + Arc::new(Column::new("b", 1)), + Arc::new(Column::new("b", 1)), + Arc::new(Column::new("a", 0)), + Arc::new(Column::new("c", 2)), + Arc::new(Column::new("a", 0)), + ]; + assert!(physical_exprs_bag_equal(list3.as_slice(), list4.as_slice())); + assert!(physical_exprs_bag_equal(list4.as_slice(), list3.as_slice())); + assert!(physical_exprs_bag_equal(list3.as_slice(), list3.as_slice())); + assert!(physical_exprs_bag_equal(list4.as_slice(), list4.as_slice())); + assert!(!physical_exprs_equal(list3.as_slice(), list4.as_slice())); + assert!(!physical_exprs_equal(list4.as_slice(), list3.as_slice())); + assert!(physical_exprs_bag_equal(list3.as_slice(), list3.as_slice())); + assert!(physical_exprs_bag_equal(list4.as_slice(), list4.as_slice())); + } + + #[test] + fn test_deduplicate_physical_exprs() { + let lit_true = &(Arc::new(Literal::new(ScalarValue::Boolean(Some(true)))) + as Arc); + let lit_false = &(Arc::new(Literal::new(ScalarValue::Boolean(Some(false)))) + as Arc); + let lit4 = &(Arc::new(Literal::new(ScalarValue::Int32(Some(4)))) + as Arc); + let lit2 = &(Arc::new(Literal::new(ScalarValue::Int32(Some(2)))) + as Arc); + let col_a_expr = &(Arc::new(Column::new("a", 0)) as Arc); + let col_b_expr = &(Arc::new(Column::new("b", 1)) as Arc); + + // First vector in the tuple is arguments, second one is the expected value. + let test_cases = vec![ + // ---------- TEST CASE 1----------// + ( + vec![ + lit_true, lit_false, lit4, lit2, col_a_expr, col_a_expr, col_b_expr, + lit_true, lit2, + ], + vec![lit_true, lit_false, lit4, lit2, col_a_expr, col_b_expr], + ), + // ---------- TEST CASE 2----------// + ( + vec![lit_true, lit_true, lit_false, lit4], + vec![lit_true, lit4, lit_false], + ), + ]; + for (exprs, expected) in test_cases { + let mut exprs = exprs.into_iter().cloned().collect::>(); + let expected = expected.into_iter().cloned().collect::>(); + deduplicate_physical_exprs(&mut exprs); + assert!(physical_exprs_equal(&exprs, &expected)); + } } } diff --git a/datafusion/physical-expr/src/scalar_function.rs b/datafusion/physical-expr/src/scalar_function.rs index 43598ce56489..5acd5dcf2336 100644 --- a/datafusion/physical-expr/src/scalar_function.rs +++ b/datafusion/physical-expr/src/scalar_function.rs @@ -30,15 +30,13 @@ //! to a function that supports f64, it is coerced to f64. use std::any::Any; -use std::fmt::Debug; -use std::fmt::{self, Formatter}; +use std::fmt::{self, Debug, Formatter}; use std::hash::{Hash, Hasher}; use std::sync::Arc; use crate::functions::out_ordering; -use crate::physical_expr::down_cast_any_ref; +use crate::physical_expr::{down_cast_any_ref, physical_exprs_equal}; use crate::sort_properties::SortProperties; -use crate::utils::expr_list_eq_strict_order; use crate::PhysicalExpr; use arrow::datatypes::{DataType, Schema}; @@ -195,7 +193,7 @@ impl PartialEq for ScalarFunctionExpr { .downcast_ref::() .map(|x| { self.name == x.name - && expr_list_eq_strict_order(&self.args, &x.args) + && physical_exprs_equal(&self.args, &x.args) && self.return_type == x.return_type }) .unwrap_or(false) diff --git a/datafusion/physical-expr/src/sort_expr.rs b/datafusion/physical-expr/src/sort_expr.rs index 3b3221289d31..664a6b65b7f7 100644 --- a/datafusion/physical-expr/src/sort_expr.rs +++ b/datafusion/physical-expr/src/sort_expr.rs @@ -25,8 +25,8 @@ use crate::PhysicalExpr; use arrow::compute::kernels::sort::{SortColumn, SortOptions}; use arrow::record_batch::RecordBatch; -use datafusion_common::exec_err; -use datafusion_common::{DataFusionError, Result}; +use arrow_schema::Schema; +use datafusion_common::{exec_err, DataFusionError, Result}; use datafusion_expr::ColumnarValue; /// Represents Sort operation for a column in a RecordBatch @@ -77,18 +77,26 @@ impl PhysicalSortExpr { }) } - /// Check whether sort expression satisfies [`PhysicalSortRequirement`]. - /// - /// If sort options is Some in `PhysicalSortRequirement`, `expr` - /// and `options` field are compared for equality. - /// - /// If sort options is None in `PhysicalSortRequirement`, only - /// `expr` is compared for equality. - pub fn satisfy(&self, requirement: &PhysicalSortRequirement) -> bool { + /// Checks whether this sort expression satisfies the given `requirement`. + /// If sort options are unspecified in `requirement`, only expressions are + /// compared for inequality. + pub fn satisfy( + &self, + requirement: &PhysicalSortRequirement, + schema: &Schema, + ) -> bool { + // If the column is not nullable, NULLS FIRST/LAST is not important. + let nullable = self.expr.nullable(schema).unwrap_or(true); self.expr.eq(&requirement.expr) - && requirement - .options - .map_or(true, |opts| self.options == opts) + && if nullable { + requirement + .options + .map_or(true, |opts| self.options == opts) + } else { + requirement + .options + .map_or(true, |opts| self.options.descending == opts.descending) + } } /// Returns a [`Display`]able list of `PhysicalSortExpr`. @@ -248,11 +256,18 @@ fn to_str(options: &SortOptions) -> &str { } } -///`LexOrdering` is a type alias for lexicographical ordering definition`Vec` +///`LexOrdering` is an alias for the type `Vec`, which represents +/// a lexicographical ordering. pub type LexOrdering = Vec; -///`LexOrderingRef` is a type alias for lexicographical ordering reference &`[PhysicalSortExpr]` +///`LexOrderingRef` is an alias for the type &`[PhysicalSortExpr]`, which represents +/// a reference to a lexicographical ordering. pub type LexOrderingRef<'a> = &'a [PhysicalSortExpr]; -///`LexOrderingReq` is a type alias for lexicographical ordering requirement definition`Vec` -pub type LexOrderingReq = Vec; +///`LexRequirement` is an alias for the type `Vec`, which +/// represents a lexicographical ordering requirement. +pub type LexRequirement = Vec; + +///`LexRequirementRef` is an alias for the type &`[PhysicalSortRequirement]`, which +/// represents a reference to a lexicographical ordering requirement. +pub type LexRequirementRef<'a> = &'a [PhysicalSortRequirement]; diff --git a/datafusion/physical-expr/src/sort_properties.rs b/datafusion/physical-expr/src/sort_properties.rs index 8ae3379218fb..a3b201f84e9d 100644 --- a/datafusion/physical-expr/src/sort_properties.rs +++ b/datafusion/physical-expr/src/sort_properties.rs @@ -17,14 +17,10 @@ use std::{ops::Neg, sync::Arc}; -use crate::expressions::Column; -use crate::utils::get_indices_of_matching_sort_exprs_with_order_eq; -use crate::{ - EquivalenceProperties, OrderingEquivalenceProperties, PhysicalExpr, PhysicalSortExpr, -}; +use crate::PhysicalExpr; use arrow_schema::SortOptions; -use datafusion_common::tree_node::{Transformed, TreeNode, VisitRecursion}; +use datafusion_common::tree_node::{TreeNode, VisitRecursion}; use datafusion_common::Result; use itertools::Itertools; @@ -224,57 +220,3 @@ impl TreeNode for ExprOrdering { } } } - -/// Calculates the [`SortProperties`] of a given [`ExprOrdering`] node. -/// The node is either a leaf node, or an intermediate node: -/// - If it is a leaf node, the children states are `None`. We directly find -/// the order of the node by looking at the given sort expression and equivalence -/// properties if it is a `Column` leaf, or we mark it as unordered. In the case -/// of a `Literal` leaf, we mark it as singleton so that it can cooperate with -/// some ordered columns at the upper steps. -/// - If it is an intermediate node, the children states matter. Each `PhysicalExpr` -/// and operator has its own rules about how to propagate the children orderings. -/// However, before the children order propagation, it is checked that whether -/// the intermediate node can be directly matched with the sort expression. If there -/// is a match, the sort expression emerges at that node immediately, discarding -/// the order coming from the children. -pub fn update_ordering( - mut node: ExprOrdering, - sort_expr: &PhysicalSortExpr, - equal_properties: &EquivalenceProperties, - ordering_equal_properties: &OrderingEquivalenceProperties, -) -> Result> { - // If we can directly match a sort expr with the current node, we can set - // its state and return early. - // TODO: If there is a PhysicalExpr other than a Column at this node (e.g. - // a BinaryExpr like a + b), and there is an ordering equivalence of - // it (let's say like c + d), we actually can find it at this step. - if sort_expr.expr.eq(&node.expr) { - node.state = SortProperties::Ordered(sort_expr.options); - return Ok(Transformed::Yes(node)); - } - - if !node.expr.children().is_empty() { - // We have an intermediate (non-leaf) node, account for its children: - node.state = node.expr.get_ordering(&node.children_states); - } else if let Some(column) = node.expr.as_any().downcast_ref::() { - // We have a Column, which is one of the two possible leaf node types: - node.state = get_indices_of_matching_sort_exprs_with_order_eq( - &[sort_expr.clone()], - &[column.clone()], - equal_properties, - ordering_equal_properties, - ) - .map(|(sort_options, _)| { - SortProperties::Ordered(SortOptions { - descending: sort_options[0].descending, - nulls_first: sort_options[0].nulls_first, - }) - }) - .unwrap_or(SortProperties::Unordered); - } else { - // We have a Literal, which is the other possible leaf node type: - node.state = node.expr.get_ordering(&[]); - } - Ok(Transformed::Yes(node)) -} diff --git a/datafusion/physical-expr/src/utils.rs b/datafusion/physical-expr/src/utils.rs index b38117d206cc..a341f5d9bc2f 100644 --- a/datafusion/physical-expr/src/utils.rs +++ b/datafusion/physical-expr/src/utils.rs @@ -15,62 +15,25 @@ // specific language governing permissions and limitations // under the License. -use crate::equivalence::{EquivalenceProperties, OrderingEquivalenceProperties}; -use crate::expressions::{BinaryExpr, Column, UnKnownColumn}; -use crate::sort_properties::{ExprOrdering, SortProperties}; -use crate::update_ordering; -use crate::{PhysicalExpr, PhysicalSortExpr, PhysicalSortRequirement}; +use std::borrow::Borrow; +use std::collections::{HashMap, HashSet}; +use std::sync::Arc; + +use crate::expressions::{BinaryExpr, Column}; +use crate::{PhysicalExpr, PhysicalSortExpr}; use arrow::array::{make_array, Array, ArrayRef, BooleanArray, MutableArrayData}; use arrow::compute::{and_kleene, is_not_null, SlicesIterator}; use arrow::datatypes::SchemaRef; -use arrow_schema::SortOptions; use datafusion_common::tree_node::{ Transformed, TreeNode, TreeNodeRewriter, VisitRecursion, }; -use datafusion_common::utils::longest_consecutive_prefix; use datafusion_common::Result; use datafusion_expr::Operator; use itertools::Itertools; use petgraph::graph::NodeIndex; use petgraph::stable_graph::StableGraph; -use std::borrow::Borrow; -use std::collections::HashMap; -use std::collections::HashSet; -use std::sync::Arc; - -/// Compare the two expr lists are equal no matter the order. -/// For example two InListExpr can be considered to be equals no matter the order: -/// -/// In('a','b','c') == In('c','b','a') -pub fn expr_list_eq_any_order( - list1: &[Arc], - list2: &[Arc], -) -> bool { - if list1.len() == list2.len() { - let mut expr_vec1 = list1.to_vec(); - let mut expr_vec2 = list2.to_vec(); - while let Some(expr1) = expr_vec1.pop() { - if let Some(idx) = expr_vec2.iter().position(|expr2| expr1.eq(expr2)) { - expr_vec2.swap_remove(idx); - } else { - break; - } - } - expr_vec1.is_empty() && expr_vec2.is_empty() - } else { - false - } -} - -/// Strictly compare the two expr lists are equal in the given order. -pub fn expr_list_eq_strict_order( - list1: &[Arc], - list2: &[Arc], -) -> bool { - list1.len() == list2.len() && list1.iter().zip(list2.iter()).all(|(e1, e2)| e1.eq(e2)) -} /// Assume the predicate is in the form of CNF, split the predicate to a Vec of PhysicalExprs. /// @@ -103,219 +66,6 @@ fn split_conjunction_impl<'a>( } } -/// Normalize the output expressions based on Columns Map. -/// -/// If there is a mapping in Columns Map, replace the Column in the output expressions with the 1st Column in the Columns Map. -/// Otherwise, replace the Column with a place holder of [UnKnownColumn] -/// -pub fn normalize_out_expr_with_columns_map( - expr: Arc, - columns_map: &HashMap>, -) -> Arc { - expr.clone() - .transform(&|expr| { - let normalized_form = match expr.as_any().downcast_ref::() { - Some(column) => columns_map - .get(column) - .map(|c| Arc::new(c[0].clone()) as _) - .or_else(|| Some(Arc::new(UnKnownColumn::new(column.name())) as _)), - None => None, - }; - Ok(if let Some(normalized_form) = normalized_form { - Transformed::Yes(normalized_form) - } else { - Transformed::No(expr) - }) - }) - .unwrap_or(expr) -} - -/// Transform `sort_exprs` vector, to standardized version using `eq_properties` and `ordering_eq_properties` -/// Assume `eq_properties` states that `Column a` and `Column b` are aliases. -/// Also assume `ordering_eq_properties` states that ordering `vec![d ASC]` and `vec![a ASC, c ASC]` are -/// ordering equivalent (in the sense that both describe the ordering of the table). -/// If the `sort_exprs` input to this function were `vec![b ASC, c ASC]`, -/// This function converts `sort_exprs` `vec![b ASC, c ASC]` to first `vec![a ASC, c ASC]` after considering `eq_properties` -/// Then converts `vec![a ASC, c ASC]` to `vec![d ASC]` after considering `ordering_eq_properties`. -/// Standardized version `vec![d ASC]` is used in subsequent operations. -fn normalize_sort_exprs( - sort_exprs: &[PhysicalSortExpr], - eq_properties: &EquivalenceProperties, - ordering_eq_properties: &OrderingEquivalenceProperties, -) -> Vec { - let sort_requirements = PhysicalSortRequirement::from_sort_exprs(sort_exprs.iter()); - let normalized_exprs = normalize_sort_requirements( - &sort_requirements, - eq_properties, - ordering_eq_properties, - ); - PhysicalSortRequirement::to_sort_exprs(normalized_exprs) -} - -/// Transform `sort_reqs` vector, to standardized version using `eq_properties` and `ordering_eq_properties` -/// Assume `eq_properties` states that `Column a` and `Column b` are aliases. -/// Also assume `ordering_eq_properties` states that ordering `vec![d ASC]` and `vec![a ASC, c ASC]` are -/// ordering equivalent (in the sense that both describe the ordering of the table). -/// If the `sort_reqs` input to this function were `vec![b Some(ASC), c None]`, -/// This function converts `sort_exprs` `vec![b Some(ASC), c None]` to first `vec![a Some(ASC), c None]` after considering `eq_properties` -/// Then converts `vec![a Some(ASC), c None]` to `vec![d Some(ASC)]` after considering `ordering_eq_properties`. -/// Standardized version `vec![d Some(ASC)]` is used in subsequent operations. -fn normalize_sort_requirements( - sort_reqs: &[PhysicalSortRequirement], - eq_properties: &EquivalenceProperties, - ordering_eq_properties: &OrderingEquivalenceProperties, -) -> Vec { - let normalized_sort_reqs = eq_properties.normalize_sort_requirements(sort_reqs); - ordering_eq_properties.normalize_sort_requirements(&normalized_sort_reqs) -} - -/// Checks whether given ordering requirements are satisfied by provided [PhysicalSortExpr]s. -pub fn ordering_satisfy< - F: FnOnce() -> EquivalenceProperties, - F2: FnOnce() -> OrderingEquivalenceProperties, ->( - provided: Option<&[PhysicalSortExpr]>, - required: Option<&[PhysicalSortExpr]>, - equal_properties: F, - ordering_equal_properties: F2, -) -> bool { - match (provided, required) { - (_, None) => true, - (None, Some(_)) => false, - (Some(provided), Some(required)) => ordering_satisfy_concrete( - provided, - required, - equal_properties, - ordering_equal_properties, - ), - } -} - -/// Checks whether the required [`PhysicalSortExpr`]s are satisfied by the -/// provided [`PhysicalSortExpr`]s. -pub fn ordering_satisfy_concrete< - F: FnOnce() -> EquivalenceProperties, - F2: FnOnce() -> OrderingEquivalenceProperties, ->( - provided: &[PhysicalSortExpr], - required: &[PhysicalSortExpr], - equal_properties: F, - ordering_equal_properties: F2, -) -> bool { - let oeq_properties = ordering_equal_properties(); - let eq_properties = equal_properties(); - let required_normalized = - normalize_sort_exprs(required, &eq_properties, &oeq_properties); - let provided_normalized = - normalize_sort_exprs(provided, &eq_properties, &oeq_properties); - if required_normalized.len() > provided_normalized.len() { - return false; - } - required_normalized - .into_iter() - .zip(provided_normalized) - .all(|(req, given)| given == req) -} - -/// Checks whether the given [`PhysicalSortRequirement`]s are satisfied by the -/// provided [`PhysicalSortExpr`]s. -pub fn ordering_satisfy_requirement< - F: FnOnce() -> EquivalenceProperties, - F2: FnOnce() -> OrderingEquivalenceProperties, ->( - provided: Option<&[PhysicalSortExpr]>, - required: Option<&[PhysicalSortRequirement]>, - equal_properties: F, - ordering_equal_properties: F2, -) -> bool { - match (provided, required) { - (_, None) => true, - (None, Some(_)) => false, - (Some(provided), Some(required)) => ordering_satisfy_requirement_concrete( - provided, - required, - equal_properties, - ordering_equal_properties, - ), - } -} - -/// Checks whether the given [`PhysicalSortRequirement`]s are satisfied by the -/// provided [`PhysicalSortExpr`]s. -pub fn ordering_satisfy_requirement_concrete< - F: FnOnce() -> EquivalenceProperties, - F2: FnOnce() -> OrderingEquivalenceProperties, ->( - provided: &[PhysicalSortExpr], - required: &[PhysicalSortRequirement], - equal_properties: F, - ordering_equal_properties: F2, -) -> bool { - let oeq_properties = ordering_equal_properties(); - let eq_properties = equal_properties(); - let required_normalized = - normalize_sort_requirements(required, &eq_properties, &oeq_properties); - let provided_normalized = - normalize_sort_exprs(provided, &eq_properties, &oeq_properties); - if required_normalized.len() > provided_normalized.len() { - return false; - } - required_normalized - .into_iter() - .zip(provided_normalized) - .all(|(req, given)| given.satisfy(&req)) -} - -/// Checks whether the given [`PhysicalSortRequirement`]s are equal or more -/// specific than the provided [`PhysicalSortRequirement`]s. -pub fn requirements_compatible< - F: FnOnce() -> OrderingEquivalenceProperties, - F2: FnOnce() -> EquivalenceProperties, ->( - provided: Option<&[PhysicalSortRequirement]>, - required: Option<&[PhysicalSortRequirement]>, - ordering_equal_properties: F, - equal_properties: F2, -) -> bool { - match (provided, required) { - (_, None) => true, - (None, Some(_)) => false, - (Some(provided), Some(required)) => requirements_compatible_concrete( - provided, - required, - ordering_equal_properties, - equal_properties, - ), - } -} - -/// Checks whether the given [`PhysicalSortRequirement`]s are equal or more -/// specific than the provided [`PhysicalSortRequirement`]s. -fn requirements_compatible_concrete< - F: FnOnce() -> OrderingEquivalenceProperties, - F2: FnOnce() -> EquivalenceProperties, ->( - provided: &[PhysicalSortRequirement], - required: &[PhysicalSortRequirement], - ordering_equal_properties: F, - equal_properties: F2, -) -> bool { - let oeq_properties = ordering_equal_properties(); - let eq_properties = equal_properties(); - - let required_normalized = - normalize_sort_requirements(required, &eq_properties, &oeq_properties); - let provided_normalized = - normalize_sort_requirements(provided, &eq_properties, &oeq_properties); - if required_normalized.len() > provided_normalized.len() { - return false; - } - required_normalized - .into_iter() - .zip(provided_normalized) - .all(|(req, given)| given.compatible(&req)) -} - /// This function maps back requirement after ProjectionExec /// to the Executor for its input. // Specifically, `ProjectionExec` changes index of `Column`s in the schema of its input executor. @@ -358,19 +108,6 @@ pub fn convert_to_expr>( .collect() } -/// This function finds the indices of `targets` within `items`, taking into -/// account equivalences according to `equal_properties`. -pub fn get_indices_of_matching_exprs EquivalenceProperties>( - targets: &[Arc], - items: &[Arc], - equal_properties: F, -) -> Vec { - let eq_properties = equal_properties(); - let normalized_items = eq_properties.normalize_exprs(items); - let normalized_targets = eq_properties.normalize_exprs(targets); - get_indices_of_exprs_strict(normalized_targets, &normalized_items) -} - /// This function finds the indices of `targets` within `items` using strict /// equality. pub fn get_indices_of_exprs_strict>>( @@ -567,31 +304,6 @@ pub fn reverse_order_bys(order_bys: &[PhysicalSortExpr]) -> Vec EquivalenceProperties, - F2: Fn() -> OrderingEquivalenceProperties, ->( - req1: &'a [PhysicalSortExpr], - req2: &'a [PhysicalSortExpr], - eq_properties: F, - ordering_eq_properties: F2, -) -> Option<&'a [PhysicalSortExpr]> { - if ordering_satisfy_concrete(req1, req2, &eq_properties, &ordering_eq_properties) { - // Finer requirement is `provided`, since it satisfies the other: - return Some(req1); - } - if ordering_satisfy_concrete(req2, req1, &eq_properties, &ordering_eq_properties) { - // Finer requirement is `req`, since it satisfies the other: - return Some(req2); - } - // Neither `provided` nor `req` satisfies one another, they are incompatible. - None -} - /// Scatter `truthy` array by boolean mask. When the mask evaluates `true`, next values of `truthy` /// are taken, when the mask evaluates `false` values null values are filled. /// @@ -635,159 +347,6 @@ pub fn scatter(mask: &BooleanArray, truthy: &dyn Array) -> Result { Ok(make_array(data)) } -/// Return indices of each item in `required_exprs` inside `provided_exprs`. -/// All the items should be found inside `provided_exprs`. Found indices will -/// be a permutation of the range 0, 1, ..., N. For example, \[2,1,0\] is valid -/// (\[0,1,2\] is consecutive), but \[3,1,0\] is not valid (\[0,1,3\] is not -/// consecutive). -fn get_lexicographical_match_indices( - required_exprs: &[Arc], - provided_exprs: &[Arc], -) -> Option> { - let indices_of_equality = get_indices_of_exprs_strict(required_exprs, provided_exprs); - let mut ordered_indices = indices_of_equality.clone(); - ordered_indices.sort(); - let n_match = indices_of_equality.len(); - let first_n = longest_consecutive_prefix(ordered_indices); - (n_match == required_exprs.len() && first_n == n_match && n_match > 0) - .then_some(indices_of_equality) -} - -/// Attempts to find a full match between the required columns to be ordered (lexicographically), and -/// the provided sort options (lexicographically), while considering equivalence properties. -/// -/// It starts by normalizing members of both the required columns and the provided sort options. -/// If a full match is found, returns the sort options and indices of the matches. If no full match is found, -/// the function proceeds to check against ordering equivalence properties. If still no full match is found, -/// the function returns `None`. -pub fn get_indices_of_matching_sort_exprs_with_order_eq( - provided_sorts: &[PhysicalSortExpr], - required_columns: &[Column], - eq_properties: &EquivalenceProperties, - order_eq_properties: &OrderingEquivalenceProperties, -) -> Option<(Vec, Vec)> { - // Create a vector of `PhysicalSortRequirement`s from the required columns: - let sort_requirement_on_requirements = required_columns - .iter() - .map(|required_column| PhysicalSortRequirement { - expr: Arc::new(required_column.clone()) as _, - options: None, - }) - .collect::>(); - - let normalized_required = normalize_sort_requirements( - &sort_requirement_on_requirements, - eq_properties, - &OrderingEquivalenceProperties::new(order_eq_properties.schema()), - ); - let normalized_provided = normalize_sort_requirements( - &PhysicalSortRequirement::from_sort_exprs(provided_sorts.iter()), - eq_properties, - &OrderingEquivalenceProperties::new(order_eq_properties.schema()), - ); - - let provided_sorts = normalized_provided - .iter() - .map(|req| req.expr.clone()) - .collect::>(); - - let normalized_required_expr = normalized_required - .iter() - .map(|req| req.expr.clone()) - .collect::>(); - - if let Some(indices_of_equality) = - get_lexicographical_match_indices(&normalized_required_expr, &provided_sorts) - { - return Some(( - indices_of_equality - .iter() - .filter_map(|index| normalized_provided[*index].options) - .collect(), - indices_of_equality, - )); - } - - // We did not find all the expressions, consult ordering equivalence properties: - if let Some(oeq_class) = order_eq_properties.oeq_class() { - let head = oeq_class.head(); - for ordering in oeq_class.others().iter().chain(std::iter::once(head)) { - let order_eq_class_exprs = convert_to_expr(ordering); - if let Some(indices_of_equality) = get_lexicographical_match_indices( - &normalized_required_expr, - &order_eq_class_exprs, - ) { - return Some(( - indices_of_equality - .iter() - .map(|index| ordering[*index].options) - .collect(), - indices_of_equality, - )); - } - } - } - // If no match found, return `None`: - None -} - -/// Calculates the output orderings for a set of expressions within the context of a given -/// execution plan. The resulting orderings are all in the type of [`Column`], since these -/// expressions become [`Column`] after the projection step. The expressions having an alias -/// are renamed with those aliases in the returned [`PhysicalSortExpr`]'s. If an expression -/// is found to be unordered, the corresponding entry in the output vector is `None`. -/// -/// # Arguments -/// -/// * `expr` - A slice of tuples containing expressions and their corresponding aliases. -/// -/// * `input_output_ordering` - Output ordering of the input plan. -/// -/// * `input_equal_properties` - Equivalence properties of the columns in the input plan. -/// -/// * `input_ordering_equal_properties` - Ordering equivalence properties of the columns in the input plan. -/// -/// # Returns -/// -/// A `Result` containing a vector of optional [`PhysicalSortExpr`]'s. Each element of the -/// vector corresponds to an expression from the input slice. If an expression can be ordered, -/// the corresponding entry is `Some(PhysicalSortExpr)`. If an expression cannot be ordered, -/// the entry is `None`. -pub fn find_orderings_of_exprs( - expr: &[(Arc, String)], - input_output_ordering: Option<&[PhysicalSortExpr]>, - input_equal_properties: EquivalenceProperties, - input_ordering_equal_properties: OrderingEquivalenceProperties, -) -> Result>> { - let mut orderings: Vec> = vec![]; - if let Some(leading_ordering) = - input_output_ordering.and_then(|output_ordering| output_ordering.first()) - { - for (index, (expression, name)) in expr.iter().enumerate() { - let initial_expr = ExprOrdering::new(expression.clone()); - let transformed = initial_expr.transform_up(&|expr| { - update_ordering( - expr, - leading_ordering, - &input_equal_properties, - &input_ordering_equal_properties, - ) - })?; - if let SortProperties::Ordered(sort_options) = transformed.state { - orderings.push(Some(PhysicalSortExpr { - expr: Arc::new(Column::new(name, index)), - options: sort_options, - })); - } else { - orderings.push(None); - } - } - } else { - orderings.extend(expr.iter().map(|_| None)); - } - Ok(orderings) -} - /// Merge left and right sort expressions, checking for duplicates. pub fn merge_vectors( left: &[PhysicalSortExpr], @@ -807,9 +366,9 @@ mod tests { use std::sync::Arc; use super::*; - use crate::equivalence::OrderingEquivalenceProperties; + use crate::equivalence::EquivalenceProperties; use crate::expressions::{binary, cast, col, in_list, lit, Column, Literal}; - use crate::{OrderingEquivalentClass, PhysicalSortExpr}; + use crate::PhysicalSortExpr; use arrow::compute::SortOptions; use arrow_array::Int32Array; @@ -858,86 +417,6 @@ mod tests { } } - // Generate a schema which consists of 5 columns (a, b, c, d, e) - fn create_test_schema() -> Result { - let a = Field::new("a", DataType::Int32, true); - let b = Field::new("b", DataType::Int32, true); - let c = Field::new("c", DataType::Int32, true); - let d = Field::new("d", DataType::Int32, true); - let e = Field::new("e", DataType::Int32, true); - let f = Field::new("f", DataType::Int32, true); - let schema = Arc::new(Schema::new(vec![a, b, c, d, e, f])); - - Ok(schema) - } - - fn create_test_params() -> Result<( - SchemaRef, - EquivalenceProperties, - OrderingEquivalenceProperties, - )> { - // Assume schema satisfies ordering a ASC NULLS LAST - // and d ASC NULLS LAST, b ASC NULLS LAST and e DESC NULLS FIRST, f ASC NULLS LAST, g ASC NULLS LAST - // Assume that column a and c are aliases. - let col_a = &Column::new("a", 0); - let col_b = &Column::new("b", 1); - let col_c = &Column::new("c", 2); - let col_d = &Column::new("d", 3); - let col_e = &Column::new("e", 4); - let col_f = &Column::new("f", 5); - let col_g = &Column::new("g", 6); - let option1 = SortOptions { - descending: false, - nulls_first: false, - }; - let option2 = SortOptions { - descending: true, - nulls_first: true, - }; - let test_schema = create_test_schema()?; - let mut eq_properties = EquivalenceProperties::new(test_schema.clone()); - eq_properties.add_equal_conditions((col_a, col_c)); - let mut ordering_eq_properties = - OrderingEquivalenceProperties::new(test_schema.clone()); - ordering_eq_properties.add_equal_conditions(( - &vec![PhysicalSortExpr { - expr: Arc::new(col_a.clone()), - options: option1, - }], - &vec![ - PhysicalSortExpr { - expr: Arc::new(col_d.clone()), - options: option1, - }, - PhysicalSortExpr { - expr: Arc::new(col_b.clone()), - options: option1, - }, - ], - )); - ordering_eq_properties.add_equal_conditions(( - &vec![PhysicalSortExpr { - expr: Arc::new(col_a.clone()), - options: option1, - }], - &vec![ - PhysicalSortExpr { - expr: Arc::new(col_e.clone()), - options: option2, - }, - PhysicalSortExpr { - expr: Arc::new(col_f.clone()), - options: option1, - }, - PhysicalSortExpr { - expr: Arc::new(col_g.clone()), - options: option1, - }, - ], - )); - Ok((test_schema, eq_properties, ordering_eq_properties)) - } - #[test] fn test_build_dag() -> Result<()> { let schema = Schema::new(vec![ @@ -1016,9 +495,7 @@ mod tests { } #[test] - fn test_get_indices_of_matching_exprs() { - let empty_schema = &Arc::new(Schema::empty()); - let equal_properties = || EquivalenceProperties::new(empty_schema.clone()); + fn test_get_indices_of_exprs_strict() { let list1: Vec> = vec![ Arc::new(Column::new("a", 0)), Arc::new(Column::new("b", 1)), @@ -1030,313 +507,8 @@ mod tests { Arc::new(Column::new("c", 2)), Arc::new(Column::new("a", 0)), ]; - assert_eq!( - get_indices_of_matching_exprs(&list1, &list2, equal_properties), - vec![2, 0, 1] - ); - assert_eq!( - get_indices_of_matching_exprs(&list2, &list1, equal_properties), - vec![1, 2, 0] - ); - } - - #[test] - fn expr_list_eq_test() -> Result<()> { - let list1: Vec> = vec![ - Arc::new(Column::new("a", 0)), - Arc::new(Column::new("a", 0)), - Arc::new(Column::new("b", 1)), - ]; - let list2: Vec> = vec![ - Arc::new(Column::new("b", 1)), - Arc::new(Column::new("b", 1)), - Arc::new(Column::new("a", 0)), - ]; - assert!(!expr_list_eq_any_order(list1.as_slice(), list2.as_slice())); - assert!(!expr_list_eq_any_order(list2.as_slice(), list1.as_slice())); - - assert!(!expr_list_eq_strict_order( - list1.as_slice(), - list2.as_slice() - )); - assert!(!expr_list_eq_strict_order( - list2.as_slice(), - list1.as_slice() - )); - - let list3: Vec> = vec![ - Arc::new(Column::new("a", 0)), - Arc::new(Column::new("b", 1)), - Arc::new(Column::new("c", 2)), - Arc::new(Column::new("a", 0)), - Arc::new(Column::new("b", 1)), - ]; - let list4: Vec> = vec![ - Arc::new(Column::new("b", 1)), - Arc::new(Column::new("b", 1)), - Arc::new(Column::new("a", 0)), - Arc::new(Column::new("c", 2)), - Arc::new(Column::new("a", 0)), - ]; - assert!(expr_list_eq_any_order(list3.as_slice(), list4.as_slice())); - assert!(expr_list_eq_any_order(list4.as_slice(), list3.as_slice())); - assert!(expr_list_eq_any_order(list3.as_slice(), list3.as_slice())); - assert!(expr_list_eq_any_order(list4.as_slice(), list4.as_slice())); - - assert!(!expr_list_eq_strict_order( - list3.as_slice(), - list4.as_slice() - )); - assert!(!expr_list_eq_strict_order( - list4.as_slice(), - list3.as_slice() - )); - assert!(expr_list_eq_any_order(list3.as_slice(), list3.as_slice())); - assert!(expr_list_eq_any_order(list4.as_slice(), list4.as_slice())); - - Ok(()) - } - - #[test] - fn test_ordering_satisfy() -> Result<()> { - let crude = vec![PhysicalSortExpr { - expr: Arc::new(Column::new("a", 0)), - options: SortOptions::default(), - }]; - let crude = Some(&crude[..]); - let finer = vec![ - PhysicalSortExpr { - expr: Arc::new(Column::new("a", 0)), - options: SortOptions::default(), - }, - PhysicalSortExpr { - expr: Arc::new(Column::new("b", 1)), - options: SortOptions::default(), - }, - ]; - let finer = Some(&finer[..]); - let empty_schema = &Arc::new(Schema::empty()); - assert!(ordering_satisfy( - finer, - crude, - || { EquivalenceProperties::new(empty_schema.clone()) }, - || { OrderingEquivalenceProperties::new(empty_schema.clone()) }, - )); - assert!(!ordering_satisfy( - crude, - finer, - || { EquivalenceProperties::new(empty_schema.clone()) }, - || { OrderingEquivalenceProperties::new(empty_schema.clone()) }, - )); - Ok(()) - } - - #[test] - fn test_ordering_satisfy_with_equivalence() -> Result<()> { - let col_a = &Column::new("a", 0); - let col_b = &Column::new("b", 1); - let col_c = &Column::new("c", 2); - let col_d = &Column::new("d", 3); - let col_e = &Column::new("e", 4); - let col_f = &Column::new("f", 5); - let col_g = &Column::new("g", 6); - let option1 = SortOptions { - descending: false, - nulls_first: false, - }; - let option2 = SortOptions { - descending: true, - nulls_first: true, - }; - // The schema is ordered by a ASC NULLS LAST, b ASC NULLS LAST - let provided = vec![ - PhysicalSortExpr { - expr: Arc::new(col_a.clone()), - options: option1, - }, - PhysicalSortExpr { - expr: Arc::new(col_b.clone()), - options: option1, - }, - ]; - let provided = Some(&provided[..]); - let (_test_schema, eq_properties, ordering_eq_properties) = create_test_params()?; - // First element in the tuple stores vector of requirement, second element is the expected return value for ordering_satisfy function - let requirements = vec![ - // `a ASC NULLS LAST`, expects `ordering_satisfy` to be `true`, since existing ordering `a ASC NULLS LAST, b ASC NULLS LAST` satisfies it - (vec![(col_a, option1)], true), - (vec![(col_a, option2)], false), - // Test whether equivalence works as expected - (vec![(col_c, option1)], true), - (vec![(col_c, option2)], false), - // Test whether ordering equivalence works as expected - (vec![(col_d, option1)], true), - (vec![(col_d, option1), (col_b, option1)], true), - (vec![(col_d, option2), (col_b, option1)], false), - ( - vec![(col_e, option2), (col_f, option1), (col_g, option1)], - true, - ), - (vec![(col_e, option2), (col_f, option1)], true), - (vec![(col_e, option1), (col_f, option1)], false), - (vec![(col_e, option2), (col_b, option1)], false), - (vec![(col_e, option1), (col_b, option1)], false), - ( - vec![ - (col_d, option1), - (col_b, option1), - (col_d, option1), - (col_b, option1), - ], - true, - ), - ( - vec![ - (col_d, option1), - (col_b, option1), - (col_e, option2), - (col_f, option1), - ], - true, - ), - ( - vec![ - (col_d, option1), - (col_b, option1), - (col_e, option2), - (col_b, option1), - ], - true, - ), - ( - vec![ - (col_d, option1), - (col_b, option1), - (col_d, option2), - (col_b, option1), - ], - true, - ), - ( - vec![ - (col_d, option1), - (col_b, option1), - (col_e, option1), - (col_f, option1), - ], - false, - ), - ( - vec![ - (col_d, option1), - (col_b, option1), - (col_e, option1), - (col_b, option1), - ], - false, - ), - (vec![(col_d, option1), (col_e, option2)], true), - ]; - - for (cols, expected) in requirements { - let err_msg = format!("Error in test case:{cols:?}"); - let required = cols - .into_iter() - .map(|(col, options)| PhysicalSortExpr { - expr: Arc::new(col.clone()), - options, - }) - .collect::>(); - - let required = Some(&required[..]); - assert_eq!( - ordering_satisfy( - provided, - required, - || eq_properties.clone(), - || ordering_eq_properties.clone(), - ), - expected, - "{err_msg}" - ); - } - Ok(()) - } - - fn convert_to_requirement( - in_data: &[(&Column, Option)], - ) -> Vec { - in_data - .iter() - .map(|(col, options)| { - PhysicalSortRequirement::new(Arc::new((*col).clone()) as _, *options) - }) - .collect::>() - } - - #[test] - fn test_normalize_sort_reqs() -> Result<()> { - let col_a = &Column::new("a", 0); - let col_b = &Column::new("b", 1); - let col_c = &Column::new("c", 2); - let col_d = &Column::new("d", 3); - let col_e = &Column::new("e", 4); - let col_f = &Column::new("f", 5); - let option1 = SortOptions { - descending: false, - nulls_first: false, - }; - let option2 = SortOptions { - descending: true, - nulls_first: true, - }; - // First element in the tuple stores vector of requirement, second element is the expected return value for ordering_satisfy function - let requirements = vec![ - (vec![(col_a, Some(option1))], vec![(col_a, Some(option1))]), - (vec![(col_a, Some(option2))], vec![(col_a, Some(option2))]), - (vec![(col_a, None)], vec![(col_a, Some(option1))]), - // Test whether equivalence works as expected - (vec![(col_c, Some(option1))], vec![(col_a, Some(option1))]), - (vec![(col_c, None)], vec![(col_a, Some(option1))]), - // Test whether ordering equivalence works as expected - ( - vec![(col_d, Some(option1)), (col_b, Some(option1))], - vec![(col_a, Some(option1))], - ), - ( - vec![(col_d, None), (col_b, None)], - vec![(col_a, Some(option1))], - ), - ( - vec![(col_e, Some(option2)), (col_f, Some(option1))], - vec![(col_a, Some(option1))], - ), - // We should be able to normalize in compatible requirements also (not exactly equal) - ( - vec![(col_e, Some(option2)), (col_f, None)], - vec![(col_a, Some(option1))], - ), - ( - vec![(col_e, None), (col_f, None)], - vec![(col_a, Some(option1))], - ), - ]; - - let (_test_schema, eq_properties, ordering_eq_properties) = create_test_params()?; - for (reqs, expected_normalized) in requirements.into_iter() { - let req = convert_to_requirement(&reqs); - let expected_normalized = convert_to_requirement(&expected_normalized); - - assert_eq!( - normalize_sort_requirements( - &req, - &eq_properties, - &ordering_eq_properties, - ), - expected_normalized - ); - } - Ok(()) + assert_eq!(get_indices_of_exprs_strict(&list1, &list2), vec![2, 0, 1]); + assert_eq!(get_indices_of_exprs_strict(&list2, &list1), vec![1, 2, 0]); } #[test] @@ -1376,174 +548,6 @@ mod tests { assert_eq!(actual.as_ref(), expected.as_any()); } - #[test] - fn test_normalize_expr_with_equivalence() -> Result<()> { - let col_a = &Column::new("a", 0); - let col_b = &Column::new("b", 1); - let col_c = &Column::new("c", 2); - let _col_d = &Column::new("d", 3); - let _col_e = &Column::new("e", 4); - // Assume that column a and c are aliases. - let (_test_schema, eq_properties, _ordering_eq_properties) = - create_test_params()?; - - let col_a_expr = Arc::new(col_a.clone()) as Arc; - let col_b_expr = Arc::new(col_b.clone()) as Arc; - let col_c_expr = Arc::new(col_c.clone()) as Arc; - // Test cases for equivalence normalization, - // First entry in the tuple is argument, second entry is expected result after normalization. - let expressions = vec![ - // Normalized version of the column a and c should go to a (since a is head) - (&col_a_expr, &col_a_expr), - (&col_c_expr, &col_a_expr), - // Cannot normalize column b - (&col_b_expr, &col_b_expr), - ]; - for (expr, expected_eq) in expressions { - assert!( - expected_eq.eq(&eq_properties.normalize_expr(expr.clone())), - "error in test: expr: {expr:?}" - ); - } - - Ok(()) - } - - #[test] - fn test_normalize_sort_requirement_with_equivalence() -> Result<()> { - let col_a = &Column::new("a", 0); - let _col_b = &Column::new("b", 1); - let col_c = &Column::new("c", 2); - let col_d = &Column::new("d", 3); - let _col_e = &Column::new("e", 4); - let option1 = SortOptions { - descending: false, - nulls_first: false, - }; - // Assume that column a and c are aliases. - let (_test_schema, eq_properties, _ordering_eq_properties) = - create_test_params()?; - - // Test cases for equivalence normalization - // First entry in the tuple is PhysicalExpr, second entry is its ordering, third entry is result after normalization. - let expressions = vec![ - (&col_a, Some(option1), &col_a, Some(option1)), - (&col_c, Some(option1), &col_a, Some(option1)), - (&col_c, None, &col_a, None), - // Cannot normalize column d, since it is not in equivalence properties. - (&col_d, Some(option1), &col_d, Some(option1)), - ]; - for (expr, sort_options, expected_col, expected_options) in - expressions.into_iter() - { - let expected = PhysicalSortRequirement::new( - Arc::new((*expected_col).clone()) as _, - expected_options, - ); - let arg = PhysicalSortRequirement::new( - Arc::new((*expr).clone()) as _, - sort_options, - ); - assert!( - expected.eq(&eq_properties.normalize_sort_requirement(arg.clone())), - "error in test: expr: {expr:?}, sort_options: {sort_options:?}" - ); - } - - Ok(()) - } - - #[test] - fn test_ordering_satisfy_different_lengths() -> Result<()> { - let col_a = &Column::new("a", 0); - let col_b = &Column::new("b", 1); - let col_c = &Column::new("c", 2); - let col_d = &Column::new("d", 3); - let col_e = &Column::new("e", 4); - let test_schema = create_test_schema()?; - let option1 = SortOptions { - descending: false, - nulls_first: false, - }; - // Column a and c are aliases. - let mut eq_properties = EquivalenceProperties::new(test_schema.clone()); - eq_properties.add_equal_conditions((col_a, col_c)); - - // Column a and e are ordering equivalent (e.g global ordering of the table can be described both as a ASC and e ASC.) - let mut ordering_eq_properties = OrderingEquivalenceProperties::new(test_schema); - ordering_eq_properties.add_equal_conditions(( - &vec![PhysicalSortExpr { - expr: Arc::new(col_a.clone()), - options: option1, - }], - &vec![PhysicalSortExpr { - expr: Arc::new(col_e.clone()), - options: option1, - }], - )); - let sort_req_a = PhysicalSortExpr { - expr: Arc::new((col_a).clone()) as _, - options: option1, - }; - let sort_req_b = PhysicalSortExpr { - expr: Arc::new((col_b).clone()) as _, - options: option1, - }; - let sort_req_c = PhysicalSortExpr { - expr: Arc::new((col_c).clone()) as _, - options: option1, - }; - let sort_req_d = PhysicalSortExpr { - expr: Arc::new((col_d).clone()) as _, - options: option1, - }; - let sort_req_e = PhysicalSortExpr { - expr: Arc::new((col_e).clone()) as _, - options: option1, - }; - - assert!(ordering_satisfy_concrete( - // After normalization would be a ASC, b ASC, d ASC - &[sort_req_a.clone(), sort_req_b.clone(), sort_req_d.clone()], - // After normalization would be a ASC, b ASC, d ASC - &[ - sort_req_c.clone(), - sort_req_b.clone(), - sort_req_a.clone(), - sort_req_d.clone(), - sort_req_e.clone(), - ], - || eq_properties.clone(), - || ordering_eq_properties.clone(), - )); - - assert!(!ordering_satisfy_concrete( - // After normalization would be a ASC, b ASC - &[sort_req_a.clone(), sort_req_b.clone()], - // After normalization would be a ASC, b ASC, d ASC - &[ - sort_req_c.clone(), - sort_req_b.clone(), - sort_req_a.clone(), - sort_req_d.clone(), - sort_req_e.clone(), - ], - || eq_properties.clone(), - || ordering_eq_properties.clone(), - )); - - assert!(!ordering_satisfy_concrete( - // After normalization would be a ASC, b ASC, d ASC - &[sort_req_a.clone(), sort_req_b.clone(), sort_req_d.clone()], - // After normalization would be a ASC, d ASC, b ASC - &[sort_req_c, sort_req_d, sort_req_a, sort_req_b, sort_req_e,], - || eq_properties.clone(), - || ordering_eq_properties.clone(), - )); - - Ok(()) - } - #[test] fn test_collect_columns() -> Result<()> { let expr1 = Arc::new(Column::new("col1", 2)) as _; @@ -1635,7 +639,15 @@ mod tests { let sort_options = SortOptions::default(); let sort_options_not = SortOptions::default().not(); - let provided_sorts = [ + let schema = Schema::new(vec![ + Field::new("a", DataType::Int32, true), + Field::new("b", DataType::Int32, true), + ]); + let col_a = &col("a", &schema)?; + let col_b = &col("b", &schema)?; + let required_columns = [col_b.clone(), col_a.clone()]; + let mut eq_properties = EquivalenceProperties::new(Arc::new(schema)); + eq_properties.add_new_orderings([vec![ PhysicalSortExpr { expr: Arc::new(Column::new("b", 1)), options: sort_options_not, @@ -1644,45 +656,38 @@ mod tests { expr: Arc::new(Column::new("a", 0)), options: sort_options, }, - ]; - let required_columns = [Column::new("b", 1), Column::new("a", 0)]; - let schema = Schema::new(vec![ - Field::new("a", DataType::Int32, true), - Field::new("b", DataType::Int32, true), - ]); - let equal_properties = EquivalenceProperties::new(Arc::new(schema.clone())); - let ordering_equal_properties = - OrderingEquivalenceProperties::new(Arc::new(schema)); + ]]); + let (result, idxs) = eq_properties.find_longest_permutation(&required_columns); + assert_eq!(idxs, vec![0, 1]); assert_eq!( - get_indices_of_matching_sort_exprs_with_order_eq( - &provided_sorts, - &required_columns, - &equal_properties, - &ordering_equal_properties, - ), - Some((vec![sort_options_not, sort_options], vec![0, 1])) + result, + vec![ + PhysicalSortExpr { + expr: col_b.clone(), + options: sort_options_not + }, + PhysicalSortExpr { + expr: col_a.clone(), + options: sort_options + } + ] ); - // required columns are provided in the equivalence classes - let provided_sorts = [PhysicalSortExpr { - expr: Arc::new(Column::new("c", 2)), - options: sort_options, - }]; - let required_columns = [Column::new("b", 1), Column::new("a", 0)]; let schema = Schema::new(vec![ Field::new("a", DataType::Int32, true), Field::new("b", DataType::Int32, true), Field::new("c", DataType::Int32, true), ]); - let equal_properties = EquivalenceProperties::new(Arc::new(schema.clone())); - let mut ordering_equal_properties = - OrderingEquivalenceProperties::new(Arc::new(schema)); - ordering_equal_properties.add_equal_conditions(( - &vec![PhysicalSortExpr { + let col_a = &col("a", &schema)?; + let col_b = &col("b", &schema)?; + let required_columns = [col_b.clone(), col_a.clone()]; + let mut eq_properties = EquivalenceProperties::new(Arc::new(schema)); + eq_properties.add_new_orderings([ + vec![PhysicalSortExpr { expr: Arc::new(Column::new("c", 2)), options: sort_options, }], - &vec![ + vec![ PhysicalSortExpr { expr: Arc::new(Column::new("b", 1)), options: sort_options_not, @@ -1692,19 +697,36 @@ mod tests { options: sort_options, }, ], - )); + ]); + let (result, idxs) = eq_properties.find_longest_permutation(&required_columns); + assert_eq!(idxs, vec![0, 1]); assert_eq!( - get_indices_of_matching_sort_exprs_with_order_eq( - &provided_sorts, - &required_columns, - &equal_properties, - &ordering_equal_properties, - ), - Some((vec![sort_options_not, sort_options], vec![0, 1])) + result, + vec![ + PhysicalSortExpr { + expr: col_b.clone(), + options: sort_options_not + }, + PhysicalSortExpr { + expr: col_a.clone(), + options: sort_options + } + ] ); + let required_columns = [ + Arc::new(Column::new("b", 1)) as _, + Arc::new(Column::new("a", 0)) as _, + ]; + let schema = Schema::new(vec![ + Field::new("a", DataType::Int32, true), + Field::new("b", DataType::Int32, true), + Field::new("c", DataType::Int32, true), + ]); + let mut eq_properties = EquivalenceProperties::new(Arc::new(schema)); + // not satisfied orders - let provided_sorts = [ + eq_properties.add_new_orderings([vec![ PhysicalSortExpr { expr: Arc::new(Column::new("b", 1)), options: sort_options_not, @@ -1717,25 +739,9 @@ mod tests { expr: Arc::new(Column::new("a", 0)), options: sort_options, }, - ]; - let required_columns = [Column::new("b", 1), Column::new("a", 0)]; - let schema = Schema::new(vec![ - Field::new("a", DataType::Int32, true), - Field::new("b", DataType::Int32, true), - Field::new("c", DataType::Int32, true), - ]); - let equal_properties = EquivalenceProperties::new(Arc::new(schema.clone())); - let ordering_equal_properties = - OrderingEquivalenceProperties::new(Arc::new(schema)); - assert_eq!( - get_indices_of_matching_sort_exprs_with_order_eq( - &provided_sorts, - &required_columns, - &equal_properties, - &ordering_equal_properties, - ), - None - ); + ]]); + let (_, idxs) = eq_properties.find_longest_permutation(&required_columns); + assert_eq!(idxs, vec![0]); Ok(()) } @@ -1749,39 +755,39 @@ mod tests { Field::new("b", DataType::Int32, true), Field::new("c", DataType::Int32, true), ]); - let mut equal_properties = EquivalenceProperties::new(Arc::new(schema.clone())); - let mut expected_oeq = OrderingEquivalenceProperties::new(Arc::new(schema)); + let col_a_expr = col("a", &schema)?; + let col_b_expr = col("b", &schema)?; + let col_c_expr = col("c", &schema)?; + let mut eq_properties = EquivalenceProperties::new(Arc::new(schema.clone())); + + eq_properties.add_equal_conditions(&col_a_expr, &col_c_expr); + let others = vec![ + vec![PhysicalSortExpr { + expr: col_b_expr.clone(), + options: sort_options, + }], + vec![PhysicalSortExpr { + expr: col_c_expr.clone(), + options: sort_options, + }], + ]; + eq_properties.add_new_orderings(others); - equal_properties - .add_equal_conditions((&Column::new("a", 0), &Column::new("c", 2))); - let head = vec![PhysicalSortExpr { - expr: Arc::new(Column::new("b", 1)), - options: sort_options, - }]; - let others = vec![vec![PhysicalSortExpr { - expr: Arc::new(Column::new("c", 2)), - options: sort_options, - }]]; - let oeq_class = OrderingEquivalentClass::new(head, others); - - expected_oeq.add_equal_conditions(( - &vec![PhysicalSortExpr { - expr: Arc::new(Column::new("b", 1)), + let mut expected_eqs = EquivalenceProperties::new(Arc::new(schema)); + expected_eqs.add_new_orderings([ + vec![PhysicalSortExpr { + expr: col_b_expr.clone(), options: sort_options, }], - &vec![PhysicalSortExpr { - expr: Arc::new(Column::new("a", 0)), + vec![PhysicalSortExpr { + expr: col_c_expr.clone(), options: sort_options, }], - )); - - let normalized_oeq_class = - oeq_class.normalize_with_equivalence_properties(&equal_properties); - let expected = expected_oeq.oeq_class().unwrap(); - assert!( - normalized_oeq_class.head().eq(expected.head()) - && normalized_oeq_class.others().eq(expected.others()) - ); + ]); + + let oeq_class = eq_properties.oeq_class().clone(); + let expected = expected_eqs.oeq_class(); + assert!(oeq_class.eq(expected)); Ok(()) } @@ -1793,27 +799,37 @@ mod tests { Field::new("b", DataType::Int32, true), Field::new("c", DataType::Int32, true), ]); - let orderings = find_orderings_of_exprs( - &[ - (Arc::new(Column::new("b", 1)), "b_new".to_string()), - (Arc::new(Column::new("a", 0)), "a_new".to_string()), - ], - Some(&[PhysicalSortExpr { - expr: Arc::new(Column::new("b", 1)), - options: SortOptions::default(), - }]), - EquivalenceProperties::new(Arc::new(schema.clone())), - OrderingEquivalenceProperties::new(Arc::new(schema.clone())), - )?; + let mut eq_properties = EquivalenceProperties::new(Arc::new(schema.clone())); + let ordering = vec![PhysicalSortExpr { + expr: Arc::new(Column::new("b", 1)), + options: SortOptions::default(), + }]; + eq_properties.add_new_orderings([ordering]); + let projection_mapping = vec![ + ( + Arc::new(Column::new("b", 1)) as _, + Arc::new(Column::new("b_new", 0)) as _, + ), + ( + Arc::new(Column::new("a", 0)) as _, + Arc::new(Column::new("a_new", 1)) as _, + ), + ]; + let projection_schema = Arc::new(Schema::new(vec![ + Field::new("b_new", DataType::Int32, true), + Field::new("a_new", DataType::Int32, true), + ])); + let orderings = eq_properties + .project(&projection_mapping, projection_schema) + .oeq_class() + .output_ordering() + .unwrap_or_default(); assert_eq!( - vec![ - Some(PhysicalSortExpr { - expr: Arc::new(Column::new("b_new", 0)), - options: SortOptions::default(), - }), - None, - ], + vec![PhysicalSortExpr { + expr: Arc::new(Column::new("b_new", 0)), + options: SortOptions::default(), + }], orderings ); @@ -1822,105 +838,24 @@ mod tests { Field::new("b", DataType::Int32, true), Field::new("c", DataType::Int32, true), ]); - let orderings = find_orderings_of_exprs( - &[ - (Arc::new(Column::new("c", 2)), "c_new".to_string()), - (Arc::new(Column::new("b", 1)), "b_new".to_string()), - ], - Some(&[]), - EquivalenceProperties::new(Arc::new(schema.clone())), - OrderingEquivalenceProperties::new(Arc::new(schema)), - )?; - - assert_eq!(vec![None, None], orderings); - - Ok(()) - } - - #[test] - fn test_find_orderings_of_exprs() -> Result<()> { - let schema = Schema::new(vec![ - Field::new("a", DataType::Int32, true), - Field::new("b", DataType::Int32, true), - Field::new("c", DataType::Int32, true), - Field::new("d", DataType::Int32, true), - ]); - - let mut eq = EquivalenceProperties::new(Arc::new(schema.clone())); - let col_a = &col("a", &schema)?; - let col_b = &col("b", &schema)?; - let col_c = &col("c", &schema)?; - let col_d = &col("d", &schema)?; - let option_asc = SortOptions { - descending: false, - nulls_first: false, - }; - // b=a (e.g they are aliases) - eq.add_equal_conditions((&Column::new("b", 1), &Column::new("a", 0))); - let mut oeq = OrderingEquivalenceProperties::new(Arc::new(schema.clone())); - // [b ASC], [d ASC] - oeq.add_equal_conditions(( - &vec![PhysicalSortExpr { - expr: col_b.clone(), - options: option_asc, - }], - &vec![PhysicalSortExpr { - expr: col_d.clone(), - options: option_asc, - }], - )); - - let orderings = find_orderings_of_exprs( - &[ - // d + b - ( - Arc::new(BinaryExpr::new( - col_d.clone(), - Operator::Plus, - col_b.clone(), - )), - "d+b".to_string(), - ), - // b as b_new - (col_b.clone(), "b_new".to_string()), - // a as a_new - (col_a.clone(), "a_new".to_string()), - // a + c - ( - Arc::new(BinaryExpr::new( - col_a.clone(), - Operator::Plus, - col_c.clone(), - )), - "a+c".to_string(), - ), - ], - Some(&[PhysicalSortExpr { - expr: col_b.clone(), - options: option_asc, - }]), - eq, - oeq, - )?; - - assert_eq!( - vec![ - Some(PhysicalSortExpr { - expr: Arc::new(Column::new("d+b", 0)), - options: option_asc, - }), - Some(PhysicalSortExpr { - expr: Arc::new(Column::new("b_new", 1)), - options: option_asc, - }), - Some(PhysicalSortExpr { - expr: Arc::new(Column::new("a_new", 2)), - options: option_asc, - }), - None, - ], - orderings - ); + let eq_properties = EquivalenceProperties::new(Arc::new(schema)); + let projection_mapping = vec![ + ( + Arc::new(Column::new("c", 2)) as _, + Arc::new(Column::new("c_new", 0)) as _, + ), + ( + Arc::new(Column::new("b", 1)) as _, + Arc::new(Column::new("b_new", 1)) as _, + ), + ]; + let projection_schema = Arc::new(Schema::new(vec![ + Field::new("c_new", DataType::Int32, true), + Field::new("b_new", DataType::Int32, true), + ])); + let projected = eq_properties.project(&projection_mapping, projection_schema); + // After projection there is no ordering. + assert!(projected.oeq_class().output_ordering().is_none()); Ok(()) } diff --git a/datafusion/physical-expr/src/window/built_in.rs b/datafusion/physical-expr/src/window/built_in.rs index a00d32e201fb..665ceb70d658 100644 --- a/datafusion/physical-expr/src/window/built_in.rs +++ b/datafusion/physical-expr/src/window/built_in.rs @@ -21,22 +21,19 @@ use std::any::Any; use std::ops::Range; use std::sync::Arc; -use super::BuiltInWindowFunctionExpr; -use super::WindowExpr; -use crate::equivalence::OrderingEquivalenceBuilder; +use super::{BuiltInWindowFunctionExpr, WindowExpr}; use crate::expressions::PhysicalSortExpr; -use crate::utils::{convert_to_expr, get_indices_of_matching_exprs}; use crate::window::window_expr::{get_orderby_values, WindowFn}; use crate::window::{PartitionBatches, PartitionWindowAggStates, WindowState}; use crate::{reverse_order_bys, EquivalenceProperties, PhysicalExpr}; + use arrow::array::{new_empty_array, ArrayRef}; use arrow::compute::SortOptions; use arrow::datatypes::Field; use arrow::record_batch::RecordBatch; use datafusion_common::utils::evaluate_partition_ranges; use datafusion_common::{Result, ScalarValue}; -use datafusion_expr::window_state::WindowAggState; -use datafusion_expr::window_state::WindowFrameContext; +use datafusion_expr::window_state::{WindowAggState, WindowFrameContext}; use datafusion_expr::WindowFrame; /// A window expr that takes the form of a [`BuiltInWindowFunctionExpr`]. @@ -75,16 +72,12 @@ impl BuiltInWindowExpr { /// If `self.expr` doesn't have an ordering, ordering equivalence properties /// are not updated. Otherwise, ordering equivalence properties are updated /// by the ordering of `self.expr`. - pub fn add_equal_orderings EquivalenceProperties>( - &self, - builder: &mut OrderingEquivalenceBuilder, - equal_properties: F, - ) { - let schema = builder.schema(); + pub fn add_equal_orderings(&self, eq_properties: &mut EquivalenceProperties) { + let schema = eq_properties.schema(); if let Some(fn_res_ordering) = self.expr.get_result_ordering(schema) { if self.partition_by.is_empty() { // In the absence of a PARTITION BY, ordering of `self.expr` is global: - builder.add_equal_conditions(vec![fn_res_ordering]); + eq_properties.add_new_orderings([vec![fn_res_ordering]]); } else { // If we have a PARTITION BY, built-in functions can not introduce // a global ordering unless the existing ordering is compatible @@ -92,23 +85,11 @@ impl BuiltInWindowExpr { // expressions and existing ordering expressions are equal (w.r.t. // set equality), we can prefix the ordering of `self.expr` with // the existing ordering. - let existing_ordering = builder.existing_ordering(); - let existing_ordering_exprs = convert_to_expr(existing_ordering); - // Get indices of the PARTITION BY expressions among input ordering expressions: - let pb_indices = get_indices_of_matching_exprs( - &self.partition_by, - &existing_ordering_exprs, - equal_properties, - ); - // Existing ordering should match exactly with PARTITION BY expressions. - // There should be no missing/extra entries in the existing ordering. - // Otherwise, prefixing wouldn't work. - if pb_indices.len() == self.partition_by.len() - && pb_indices.len() == existing_ordering.len() - { - let mut new_ordering = existing_ordering.to_vec(); - new_ordering.push(fn_res_ordering); - builder.add_equal_conditions(new_ordering); + let (mut ordering, _) = + eq_properties.find_longest_permutation(&self.partition_by); + if ordering.len() == self.partition_by.len() { + ordering.push(fn_res_ordering); + eq_properties.add_new_orderings([ordering]); } } } diff --git a/datafusion/physical-plan/src/aggregates/mod.rs b/datafusion/physical-plan/src/aggregates/mod.rs index da152a6264af..fd2dc69e1f3d 100644 --- a/datafusion/physical-plan/src/aggregates/mod.rs +++ b/datafusion/physical-plan/src/aggregates/mod.rs @@ -18,7 +18,6 @@ //! Aggregates functionalities use std::any::Any; -use std::collections::HashMap; use std::sync::Arc; use super::DisplayAs; @@ -26,9 +25,13 @@ use crate::aggregates::{ no_grouping::AggregateStream, row_hash::GroupedHashAggregateStream, topk_stream::GroupedTopKAggregateStream, }; +use crate::common::calculate_projection_mapping; use crate::metrics::{ExecutionPlanMetricsSet, MetricsSet}; +use crate::windows::{ + get_ordered_partition_by_indices, get_window_mode, PartitionSearchMode, +}; use crate::{ - DisplayFormatType, Distribution, EquivalenceProperties, ExecutionPlan, Partitioning, + DisplayFormatType, Distribution, ExecutionPlan, Partitioning, SendableRecordBatchStream, Statistics, }; @@ -36,24 +39,18 @@ use arrow::array::ArrayRef; use arrow::datatypes::{Field, Schema, SchemaRef}; use arrow::record_batch::RecordBatch; use datafusion_common::stats::Precision; -use datafusion_common::utils::longest_consecutive_prefix; use datafusion_common::{not_impl_err, plan_err, DataFusionError, Result}; use datafusion_execution::TaskContext; use datafusion_expr::Accumulator; -use datafusion_physical_expr::utils::{ - convert_to_expr, get_finer_ordering, get_indices_of_matching_exprs, - ordering_satisfy_requirement_concrete, -}; use datafusion_physical_expr::{ aggregate::is_order_sensitive, - equivalence::project_equivalence_properties, - expressions::{Column, Max, Min}, - normalize_out_expr_with_columns_map, physical_exprs_contains, reverse_order_bys, - AggregateExpr, LexOrdering, LexOrderingReq, OrderingEquivalenceProperties, - PhysicalExpr, PhysicalSortExpr, PhysicalSortRequirement, + equivalence::collapse_lex_req, + expressions::{Column, Max, Min, UnKnownColumn}, + physical_exprs_contains, reverse_order_bys, AggregateExpr, EquivalenceProperties, + LexOrdering, LexRequirement, PhysicalExpr, PhysicalSortExpr, PhysicalSortRequirement, }; -use itertools::Itertools; +use itertools::{izip, Itertools}; mod group_values; mod no_grouping; @@ -271,18 +268,6 @@ impl From for SendableRecordBatchStream { } } -/// This object encapsulates ordering-related information on GROUP BY columns. -#[derive(Debug, Clone)] -pub(crate) struct AggregationOrdering { - /// Specifies whether the GROUP BY columns are partially or fully ordered. - mode: GroupByOrderMode, - /// Stores indices such that when we iterate with these indices, GROUP BY - /// expressions match input ordering. - order_indices: Vec, - /// Actual ordering information of the GROUP BY columns. - ordering: LexOrdering, -} - /// Hash aggregate execution plan #[derive(Debug)] pub struct AggregateExec { @@ -308,90 +293,15 @@ pub struct AggregateExec { /// We need the input schema of partial aggregate to be able to deserialize aggregate /// expressions from protobuf for final aggregate. pub input_schema: SchemaRef, - /// The columns map used to normalize out expressions like Partitioning and PhysicalSortExpr - /// The key is the column from the input schema and the values are the columns from the output schema - columns_map: HashMap>, - /// Execution Metrics + /// The mapping used to normalize expressions like Partitioning and + /// PhysicalSortExpr. The key is the expression from the input schema + /// and the value is the expression from the output schema. + projection_mapping: Vec<(Arc, Arc)>, + /// Execution metrics metrics: ExecutionPlanMetricsSet, - /// Stores mode and output ordering information for the `AggregateExec`. - aggregation_ordering: Option, - required_input_ordering: Option, -} - -/// Calculates the working mode for `GROUP BY` queries. -/// - If no GROUP BY expression has an ordering, returns `None`. -/// - If some GROUP BY expressions have an ordering, returns `Some(GroupByOrderMode::PartiallyOrdered)`. -/// - If all GROUP BY expressions have orderings, returns `Some(GroupByOrderMode::Ordered)`. -fn get_working_mode( - input: &Arc, - group_by: &PhysicalGroupBy, -) -> Option<(GroupByOrderMode, Vec)> { - if !group_by.is_single() { - // We do not currently support streaming execution if we have more - // than one group (e.g. we have grouping sets). - return None; - }; - - let output_ordering = input.output_ordering().unwrap_or(&[]); - // Since direction of the ordering is not important for GROUP BY columns, - // we convert PhysicalSortExpr to PhysicalExpr in the existing ordering. - let ordering_exprs = convert_to_expr(output_ordering); - let groupby_exprs = group_by.input_exprs(); - // Find where each expression of the GROUP BY clause occurs in the existing - // ordering (if it occurs): - let mut ordered_indices = - get_indices_of_matching_exprs(&groupby_exprs, &ordering_exprs, || { - input.equivalence_properties() - }); - ordered_indices.sort(); - // Find out how many expressions of the existing ordering define ordering - // for expressions in the GROUP BY clause. For example, if the input is - // ordered by a, b, c, d and we group by b, a, d; the result below would be. - // 2, meaning 2 elements (a, b) among the GROUP BY columns define ordering. - let first_n = longest_consecutive_prefix(ordered_indices); - if first_n == 0 { - // No GROUP by columns are ordered, we can not do streaming execution. - return None; - } - let ordered_exprs = ordering_exprs[0..first_n].to_vec(); - // Find indices for the GROUP BY expressions such that when we iterate with - // these indices, we would match existing ordering. For the example above, - // this would produce 1, 0; meaning 1st and 0th entries (a, b) among the - // GROUP BY expressions b, a, d match input ordering. - let ordered_group_by_indices = - get_indices_of_matching_exprs(&ordered_exprs, &groupby_exprs, || { - input.equivalence_properties() - }); - Some(if first_n == group_by.expr.len() { - (GroupByOrderMode::FullyOrdered, ordered_group_by_indices) - } else { - (GroupByOrderMode::PartiallyOrdered, ordered_group_by_indices) - }) -} - -/// This function gathers the ordering information for the GROUP BY columns. -fn calc_aggregation_ordering( - input: &Arc, - group_by: &PhysicalGroupBy, -) -> Option { - get_working_mode(input, group_by).map(|(mode, order_indices)| { - let existing_ordering = input.output_ordering().unwrap_or(&[]); - let out_group_expr = group_by.output_exprs(); - // Calculate output ordering information for the operator: - let out_ordering = order_indices - .iter() - .zip(existing_ordering) - .map(|(idx, input_col)| PhysicalSortExpr { - expr: out_group_expr[*idx].clone(), - options: input_col.options, - }) - .collect::>(); - AggregationOrdering { - mode, - order_indices, - ordering: out_ordering, - } - }) + required_input_ordering: Option, + partition_search_mode: PartitionSearchMode, + output_ordering: Option, } /// This function returns the ordering requirement of the first non-reversible @@ -420,46 +330,57 @@ fn get_init_req( /// This function gets the finest ordering requirement among all the aggregation /// functions. If requirements are conflicting, (i.e. we can not compute the /// aggregations in a single [`AggregateExec`]), the function returns an error. -fn get_finest_requirement< - F: Fn() -> EquivalenceProperties, - F2: Fn() -> OrderingEquivalenceProperties, ->( +fn get_finest_requirement( aggr_expr: &mut [Arc], order_by_expr: &mut [Option], - eq_properties: F, - ordering_eq_properties: F2, + eq_properties: &EquivalenceProperties, ) -> Result> { + // First, we check if all the requirements are satisfied by the existing + // ordering. If so, we return `None` to indicate this. + let mut all_satisfied = true; + for (aggr_expr, fn_req) in aggr_expr.iter_mut().zip(order_by_expr.iter_mut()) { + if eq_properties.ordering_satisfy(fn_req.as_deref().unwrap_or(&[])) { + continue; + } + if let Some(reverse) = aggr_expr.reverse_expr() { + let reverse_req = fn_req.as_ref().map(|item| reverse_order_bys(item)); + if eq_properties.ordering_satisfy(reverse_req.as_deref().unwrap_or(&[])) { + // We need to update `aggr_expr` with its reverse since only its + // reverse requirement is compatible with the existing requirements: + *aggr_expr = reverse; + *fn_req = reverse_req; + continue; + } + } + // Requirement is not satisfied: + all_satisfied = false; + } + if all_satisfied { + // All of the requirements are already satisfied. + return Ok(None); + } let mut finest_req = get_init_req(aggr_expr, order_by_expr); for (aggr_expr, fn_req) in aggr_expr.iter_mut().zip(order_by_expr.iter_mut()) { - let fn_req = if let Some(fn_req) = fn_req { - fn_req - } else { + let Some(fn_req) = fn_req else { continue; }; + if let Some(finest_req) = &mut finest_req { - if let Some(finer) = get_finer_ordering( - finest_req, - fn_req, - &eq_properties, - &ordering_eq_properties, - ) { - *finest_req = finer.to_vec(); + if let Some(finer) = eq_properties.get_finer_ordering(finest_req, fn_req) { + *finest_req = finer; continue; } // If an aggregate function is reversible, analyze whether its reverse // direction is compatible with existing requirements: if let Some(reverse) = aggr_expr.reverse_expr() { let fn_req_reverse = reverse_order_bys(fn_req); - if let Some(finer) = get_finer_ordering( - finest_req, - &fn_req_reverse, - &eq_properties, - &ordering_eq_properties, - ) { + if let Some(finer) = + eq_properties.get_finer_ordering(finest_req, &fn_req_reverse) + { // We need to update `aggr_expr` with its reverse, since only its // reverse requirement is compatible with existing requirements: *aggr_expr = reverse; - *finest_req = finer.to_vec(); + *finest_req = finer; *fn_req = fn_req_reverse; continue; } @@ -477,113 +398,46 @@ fn get_finest_requirement< Ok(finest_req) } -/// Calculate the required input ordering for the [`AggregateExec`] by considering -/// ordering requirements of order-sensitive aggregation functions. -fn calc_required_input_ordering( +/// Calculates search_mode for the aggregation +fn get_aggregate_search_mode( + group_by: &PhysicalGroupBy, input: &Arc, - aggr_exprs: &mut [Arc], - order_by_exprs: &mut [Option], - aggregator_reqs: LexOrderingReq, - aggregator_reverse_reqs: Option, - aggregation_ordering: &mut Option, - mode: &AggregateMode, -) -> Result> { - let mut required_input_ordering = vec![]; - // Boolean shows that whether `required_input_ordering` stored comes from - // `aggregator_reqs` or `aggregator_reverse_reqs` - let mut reverse_req = false; - // If reverse aggregator is None, there is no way to run aggregators in reverse mode. Hence ignore it during analysis - let aggregator_requirements = - if let Some(aggregator_reverse_reqs) = aggregator_reverse_reqs { - // If existing ordering doesn't satisfy requirement, we should do calculations - // on naive requirement (by convention, otherwise the final plan will be unintuitive), - // even if reverse ordering is possible. - // Hence, while iterating consider naive requirement last, by this way - // we prioritize naive requirement over reverse requirement, when - // reverse requirement is not helpful with removing SortExec from the plan. - vec![(true, aggregator_reverse_reqs), (false, aggregator_reqs)] - } else { - vec![(false, aggregator_reqs)] - }; - for (is_reverse, aggregator_requirement) in aggregator_requirements.into_iter() { - if let Some(AggregationOrdering { - // If the mode is FullyOrdered or PartiallyOrdered (i.e. we are - // running with bounded memory, without breaking the pipeline), - // then we append the aggregator ordering requirement to the existing - // ordering. This way, we can still run with bounded memory. - mode: GroupByOrderMode::FullyOrdered | GroupByOrderMode::PartiallyOrdered, - order_indices, - .. - }) = aggregation_ordering - { - // Get the section of the input ordering that enables us to run in - // FullyOrdered or PartiallyOrdered modes: - let requirement_prefix = - if let Some(existing_ordering) = input.output_ordering() { - &existing_ordering[0..order_indices.len()] - } else { - &[] - }; - let mut requirement = - PhysicalSortRequirement::from_sort_exprs(requirement_prefix.iter()); - for req in aggregator_requirement { - // Final and FinalPartitioned modes don't enforce ordering - // requirements since order-sensitive aggregators handle such - // requirements during merging. - if mode.is_first_stage() - && requirement.iter().all(|item| req.expr.ne(&item.expr)) - { - requirement.push(req); - } - } - required_input_ordering = requirement; - } else if mode.is_first_stage() { - required_input_ordering = aggregator_requirement; - } - // Keep track of the direction from which required_input_ordering is constructed: - reverse_req = is_reverse; - // If all the order-sensitive aggregate functions are reversible (e.g. all the - // order-sensitive aggregators are either FIRST_VALUE or LAST_VALUE), then we can - // run aggregate expressions either in the given required ordering, (i.e. finest - // requirement that satisfies every aggregate function requirement) or its reverse - // (opposite) direction. We analyze these two possibilities, and use the version that - // satisfies existing ordering. This enables us to avoid an extra sort step in the final - // plan. If neither version satisfies the existing ordering, we use the given ordering - // requirement. In short, if running aggregators in reverse order help us to avoid a - // sorting step, we do so. Otherwise, we use the aggregators as is. - let existing_ordering = input.output_ordering().unwrap_or(&[]); - if ordering_satisfy_requirement_concrete( - existing_ordering, - &required_input_ordering, - || input.equivalence_properties(), - || input.ordering_equivalence_properties(), - ) { - break; - } + aggr_expr: &mut [Arc], + order_by_expr: &mut [Option], + ordering_req: &mut Vec, +) -> Result { + let groupby_exprs = group_by + .expr + .iter() + .map(|(item, _)| item.clone()) + .collect::>(); + let mut partition_search_mode = PartitionSearchMode::Linear; + if !group_by.is_single() || groupby_exprs.is_empty() { + return Ok(partition_search_mode); } - // If `required_input_ordering` is constructed using the reverse requirement, we - // should reverse each `aggr_expr` in order to correctly calculate their results - // in reverse order. - if reverse_req { - aggr_exprs - .iter_mut() - .zip(order_by_exprs.iter_mut()) - .map(|(aggr_expr, ob_expr)| { - if is_order_sensitive(aggr_expr) { - if let Some(reverse) = aggr_expr.reverse_expr() { - *aggr_expr = reverse; - *ob_expr = ob_expr.as_ref().map(|obs| reverse_order_bys(obs)); + + if let Some((should_reverse, mode)) = + get_window_mode(&groupby_exprs, ordering_req, input)? + { + let all_reversible = aggr_expr + .iter() + .all(|expr| !is_order_sensitive(expr) || expr.reverse_expr().is_some()); + if should_reverse && all_reversible { + izip!(aggr_expr.iter_mut(), order_by_expr.iter_mut()).for_each( + |(aggr, order_by)| { + if let Some(reverse) = aggr.reverse_expr() { + *aggr = reverse; } else { - return plan_err!( - "Aggregate expression should have a reverse expression" - ); + unreachable!(); } - } - Ok(()) - }) - .collect::>>()?; + *order_by = order_by.as_ref().map(|ob| reverse_order_bys(ob)); + }, + ); + *ordering_req = reverse_order_bys(ordering_req); + } + partition_search_mode = mode; } - Ok((!required_input_ordering.is_empty()).then_some(required_input_ordering)) + Ok(partition_search_mode) } /// Check whether group by expression contains all of the expression inside `requirement` @@ -647,57 +501,50 @@ impl AggregateExec { }) }) .collect::>(); - let mut aggregator_reverse_reqs = None; - // Currently we support order-sensitive aggregation only in `Single` mode. - // For `Final` and `FinalPartitioned` modes, we cannot guarantee they will receive - // data according to ordering requirements. As long as we cannot produce correct result - // in `Final` mode, it is not important to produce correct result in `Partial` mode. - // We only support `Single` mode, where we are sure that output produced is final, and it - // is produced in a single step. - let requirement = get_finest_requirement( &mut aggr_expr, &mut order_by_expr, - || input.equivalence_properties(), - || input.ordering_equivalence_properties(), + &input.equivalence_properties(), )?; - let aggregator_requirement = requirement - .as_ref() - .map(|exprs| PhysicalSortRequirement::from_sort_exprs(exprs.iter())); - let aggregator_reqs = aggregator_requirement.unwrap_or(vec![]); - // If all aggregate expressions are reversible, also consider reverse - // requirement(s). The reason is that existing ordering may satisfy the - // given requirement or its reverse. By considering both, we can generate better plans. - if aggr_expr - .iter() - .all(|expr| !is_order_sensitive(expr) || expr.reverse_expr().is_some()) - { - aggregator_reverse_reqs = requirement.map(|reqs| { - PhysicalSortRequirement::from_sort_exprs(reverse_order_bys(&reqs).iter()) - }); - } - - // construct a map from the input columns to the output columns of the Aggregation - let mut columns_map: HashMap> = HashMap::new(); - for (expression, name) in group_by.expr.iter() { - if let Some(column) = expression.as_any().downcast_ref::() { - let new_col_idx = schema.index_of(name)?; - let entry = columns_map.entry(column.clone()).or_default(); - entry.push(Column::new(name, new_col_idx)); - }; - } - - let mut aggregation_ordering = calc_aggregation_ordering(&input, &group_by); - let required_input_ordering = calc_required_input_ordering( + let mut ordering_req = requirement.unwrap_or(vec![]); + let partition_search_mode = get_aggregate_search_mode( + &group_by, &input, &mut aggr_expr, &mut order_by_expr, - aggregator_reqs, - aggregator_reverse_reqs, - &mut aggregation_ordering, - &mode, + &mut ordering_req, )?; + // Get GROUP BY expressions: + let groupby_exprs = group_by.input_exprs(); + // If existing ordering satisfies a prefix of the GROUP BY expressions, + // prefix requirements with this section. In this case, aggregation will + // work more efficiently. + let indices = get_ordered_partition_by_indices(&groupby_exprs, &input); + let mut new_requirement = indices + .into_iter() + .map(|idx| PhysicalSortRequirement { + expr: groupby_exprs[idx].clone(), + options: None, + }) + .collect::>(); + // Postfix ordering requirement of the aggregation to the requirement. + let req = PhysicalSortRequirement::from_sort_exprs(&ordering_req); + new_requirement.extend(req); + new_requirement = collapse_lex_req(new_requirement); + + // construct a map from the input expression to the output expression of the Aggregation group by + let projection_mapping = + calculate_projection_mapping(&group_by.expr, &input.schema())?; + + let required_input_ordering = + (!new_requirement.is_empty()).then_some(new_requirement); + + let aggregate_eqs = input + .equivalence_properties() + .project(&projection_mapping, schema.clone()); + let output_ordering = aggregate_eqs.oeq_class().output_ordering(); + Ok(AggregateExec { mode, group_by, @@ -707,11 +554,12 @@ impl AggregateExec { input, schema, input_schema, - columns_map, + projection_mapping, metrics: ExecutionPlanMetricsSet::new(), - aggregation_ordering, required_input_ordering, limit: None, + partition_search_mode, + output_ordering, }) } @@ -870,8 +718,8 @@ impl DisplayAs for AggregateExec { write!(f, ", lim=[{limit}]")?; } - if let Some(aggregation_ordering) = &self.aggregation_ordering { - write!(f, ", ordering_mode={:?}", aggregation_ordering.mode)?; + if self.partition_search_mode != PartitionSearchMode::Linear { + write!(f, ", ordering_mode={:?}", self.partition_search_mode)?; } } } @@ -893,23 +741,28 @@ impl ExecutionPlan for AggregateExec { fn output_partitioning(&self) -> Partitioning { let input_partition = self.input.output_partitioning(); if self.mode.is_first_stage() { + // First stage aggregation will not change the output partitioning, + // but needs to respect aliases (e.g. mapping in the GROUP BY + // expression). + let input_eq_properties = self.input.equivalence_properties(); // First stage Aggregation will not change the output partitioning but need to respect the Alias let input_partition = self.input.output_partitioning(); if let Partitioning::Hash(exprs, part) = input_partition { let normalized_exprs = exprs .into_iter() .map(|expr| { - normalize_out_expr_with_columns_map(expr, &self.columns_map) + input_eq_properties + .project_expr(&expr, &self.projection_mapping) + .unwrap_or_else(|| { + Arc::new(UnKnownColumn::new(&expr.to_string())) + }) }) - .collect::>(); - Partitioning::Hash(normalized_exprs, part) - } else { - input_partition + .collect(); + return Partitioning::Hash(normalized_exprs, part); } - } else { - // Final Aggregation's output partitioning is the same as its real input - input_partition } + // Final Aggregation's output partitioning is the same as its real input + input_partition } /// Specifies whether this plan generates an infinite stream of records. @@ -917,7 +770,7 @@ impl ExecutionPlan for AggregateExec { /// infinite, returns an error to indicate this. fn unbounded_output(&self, children: &[bool]) -> Result { if children[0] { - if self.aggregation_ordering.is_none() { + if self.partition_search_mode == PartitionSearchMode::Linear { // Cannot run without breaking pipeline. plan_err!( "Aggregate Error: `GROUP BY` clauses with columns without ordering and GROUPING SETS are not supported for unbounded inputs." @@ -931,9 +784,7 @@ impl ExecutionPlan for AggregateExec { } fn output_ordering(&self) -> Option<&[PhysicalSortExpr]> { - self.aggregation_ordering - .as_ref() - .map(|item: &AggregationOrdering| item.ordering.as_slice()) + self.output_ordering.as_deref() } fn required_input_distribution(&self) -> Vec { @@ -950,18 +801,14 @@ impl ExecutionPlan for AggregateExec { } } - fn required_input_ordering(&self) -> Vec> { + fn required_input_ordering(&self) -> Vec> { vec![self.required_input_ordering.clone()] } fn equivalence_properties(&self) -> EquivalenceProperties { - let mut new_properties = EquivalenceProperties::new(self.schema()); - project_equivalence_properties( - self.input.equivalence_properties(), - &self.columns_map, - &mut new_properties, - ); - new_properties + self.input + .equivalence_properties() + .project(&self.projection_mapping, self.schema()) } fn children(&self) -> Vec> { @@ -1287,19 +1134,21 @@ pub(crate) fn evaluate_group_by( #[cfg(test)] mod tests { + use std::any::Any; + use std::sync::Arc; + use std::task::{Context, Poll}; + use super::*; - use crate::aggregates::GroupByOrderMode::{FullyOrdered, PartiallyOrdered}; use crate::aggregates::{ - get_finest_requirement, get_working_mode, AggregateExec, AggregateMode, - PhysicalGroupBy, + get_finest_requirement, AggregateExec, AggregateMode, PhysicalGroupBy, }; use crate::coalesce_batches::CoalesceBatchesExec; use crate::coalesce_partitions::CoalescePartitionsExec; use crate::common; use crate::expressions::{col, Avg}; use crate::memory::MemoryExec; + use crate::test::assert_is_pending; use crate::test::exec::{assert_strong_count_converges_to_zero, BlockingExec}; - use crate::test::{assert_is_pending, mem_exec}; use crate::{ DisplayAs, ExecutionPlan, Partitioning, RecordBatchStream, SendableRecordBatchStream, Statistics, @@ -1313,20 +1162,15 @@ mod tests { assert_batches_eq, assert_batches_sorted_eq, internal_err, DataFusionError, Result, ScalarValue, }; + use datafusion_execution::config::SessionConfig; use datafusion_execution::runtime_env::{RuntimeConfig, RuntimeEnv}; use datafusion_physical_expr::expressions::{ - lit, ApproxDistinct, Column, Count, FirstValue, LastValue, Median, + lit, ApproxDistinct, Count, FirstValue, LastValue, Median, }; use datafusion_physical_expr::{ - AggregateExpr, EquivalenceProperties, OrderingEquivalenceProperties, - PhysicalExpr, PhysicalSortExpr, + AggregateExpr, EquivalenceProperties, PhysicalExpr, PhysicalSortExpr, }; - use std::any::Any; - use std::sync::Arc; - use std::task::{Context, Poll}; - - use datafusion_execution::config::SessionConfig; use futures::{FutureExt, Stream}; // Generate a schema which consists of 5 columns (a, b, c, d, e) @@ -1341,80 +1185,6 @@ mod tests { Ok(schema) } - /// make PhysicalSortExpr with default options - fn sort_expr(name: &str, schema: &Schema) -> PhysicalSortExpr { - sort_expr_options(name, schema, SortOptions::default()) - } - - /// PhysicalSortExpr with specified options - fn sort_expr_options( - name: &str, - schema: &Schema, - options: SortOptions, - ) -> PhysicalSortExpr { - PhysicalSortExpr { - expr: col(name, schema).unwrap(), - options, - } - } - - #[tokio::test] - async fn test_get_working_mode() -> Result<()> { - let test_schema = create_test_schema()?; - // Source is sorted by a ASC NULLS FIRST, b ASC NULLS FIRST, c ASC NULLS FIRST - // Column d, e is not ordered. - let sort_exprs = vec![ - sort_expr("a", &test_schema), - sort_expr("b", &test_schema), - sort_expr("c", &test_schema), - ]; - let input = mem_exec(1).with_sort_information(vec![sort_exprs]); - let input = Arc::new(input) as _; - - // test cases consists of vector of tuples. Where each tuple represents a single test case. - // First field in the tuple is Vec where each element in the vector represents GROUP BY columns - // For instance `vec!["a", "b"]` corresponds to GROUP BY a, b - // Second field in the tuple is Option, which corresponds to expected algorithm mode. - // None represents that existing ordering is not sufficient to run executor with any one of the algorithms - // (We need to add SortExec to be able to run it). - // Some(GroupByOrderMode) represents, we can run algorithm with existing ordering; and algorithm should work in - // GroupByOrderMode. - let test_cases = vec![ - (vec!["a"], Some((FullyOrdered, vec![0]))), - (vec!["b"], None), - (vec!["c"], None), - (vec!["b", "a"], Some((FullyOrdered, vec![1, 0]))), - (vec!["c", "b"], None), - (vec!["c", "a"], Some((PartiallyOrdered, vec![1]))), - (vec!["c", "b", "a"], Some((FullyOrdered, vec![2, 1, 0]))), - (vec!["d", "a"], Some((PartiallyOrdered, vec![1]))), - (vec!["d", "b"], None), - (vec!["d", "c"], None), - (vec!["d", "b", "a"], Some((PartiallyOrdered, vec![2, 1]))), - (vec!["d", "c", "b"], None), - (vec!["d", "c", "a"], Some((PartiallyOrdered, vec![2]))), - ( - vec!["d", "c", "b", "a"], - Some((PartiallyOrdered, vec![3, 2, 1])), - ), - ]; - for (case_idx, test_case) in test_cases.iter().enumerate() { - let (group_by_columns, expected) = &test_case; - let mut group_by_exprs = vec![]; - for col_name in group_by_columns { - group_by_exprs.push((col(col_name, &test_schema)?, col_name.to_string())); - } - let group_bys = PhysicalGroupBy::new_single(group_by_exprs); - let res = get_working_mode(&input, &group_bys); - assert_eq!( - res, *expected, - "Unexpected result for in unbounded test case#: {case_idx:?}, case: {test_case:?}" - ); - } - - Ok(()) - } - /// some mock data to aggregates fn some_data() -> (Arc, Vec) { // define a schema. @@ -2268,69 +2038,72 @@ mod tests { descending: true, nulls_first: true, }; - let mut eq_properties = EquivalenceProperties::new(test_schema.clone()); - let col_a = Column::new("a", 0); - let col_b = Column::new("b", 1); - let col_c = Column::new("c", 2); - let col_d = Column::new("d", 3); - eq_properties.add_equal_conditions((&col_a, &col_b)); - let mut ordering_eq_properties = OrderingEquivalenceProperties::new(test_schema); - ordering_eq_properties.add_equal_conditions(( - &vec![PhysicalSortExpr { - expr: Arc::new(col_a.clone()) as _, - options: options1, - }], - &vec![PhysicalSortExpr { - expr: Arc::new(col_c.clone()) as _, - options: options2, - }], - )); + let col_a = &col("a", &test_schema)?; + let col_b = &col("b", &test_schema)?; + let col_c = &col("c", &test_schema)?; + let mut eq_properties = EquivalenceProperties::new(test_schema); + // Columns a and b are equal. + eq_properties.add_equal_conditions(col_a, col_b); + // Aggregate requirements are + // [None], [a ASC], [a ASC, b ASC, c ASC], [a ASC, b ASC] respectively let mut order_by_exprs = vec![ None, Some(vec![PhysicalSortExpr { - expr: Arc::new(col_a.clone()), - options: options1, - }]), - Some(vec![PhysicalSortExpr { - expr: Arc::new(col_b.clone()), + expr: col_a.clone(), options: options1, }]), - Some(vec![PhysicalSortExpr { - expr: Arc::new(col_c), - options: options2, - }]), Some(vec![ PhysicalSortExpr { - expr: Arc::new(col_a.clone()), + expr: col_a.clone(), options: options1, }, PhysicalSortExpr { - expr: Arc::new(col_d), + expr: col_b.clone(), + options: options1, + }, + PhysicalSortExpr { + expr: col_c.clone(), + options: options1, + }, + ]), + Some(vec![ + PhysicalSortExpr { + expr: col_a.clone(), + options: options1, + }, + PhysicalSortExpr { + expr: col_b.clone(), options: options1, }, ]), // Since aggregate expression is reversible (FirstValue), we should be able to resolve below // contradictory requirement by reversing it. Some(vec![PhysicalSortExpr { - expr: Arc::new(col_b.clone()), + expr: col_b.clone(), options: options2, }]), ]; + let common_requirement = Some(vec![ + PhysicalSortExpr { + expr: col_a.clone(), + options: options1, + }, + PhysicalSortExpr { + expr: col_c.clone(), + options: options1, + }, + ]); let aggr_expr = Arc::new(FirstValue::new( - Arc::new(col_a.clone()), + col_a.clone(), "first1", DataType::Int32, vec![], vec![], )) as _; let mut aggr_exprs = vec![aggr_expr; order_by_exprs.len()]; - let res = get_finest_requirement( - &mut aggr_exprs, - &mut order_by_exprs, - || eq_properties.clone(), - || ordering_eq_properties.clone(), - )?; - assert_eq!(res, order_by_exprs[4]); + let res = + get_finest_requirement(&mut aggr_exprs, &mut order_by_exprs, &eq_properties)?; + assert_eq!(res, common_requirement); Ok(()) } } diff --git a/datafusion/physical-plan/src/aggregates/order/mod.rs b/datafusion/physical-plan/src/aggregates/order/mod.rs index f0b49872b1c5..f72d2f06e459 100644 --- a/datafusion/physical-plan/src/aggregates/order/mod.rs +++ b/datafusion/physical-plan/src/aggregates/order/mod.rs @@ -18,13 +18,12 @@ use arrow_array::ArrayRef; use arrow_schema::Schema; use datafusion_common::Result; -use datafusion_physical_expr::EmitTo; - -use super::{AggregationOrdering, GroupByOrderMode}; +use datafusion_physical_expr::{EmitTo, PhysicalSortExpr}; mod full; mod partial; +use crate::windows::PartitionSearchMode; pub(crate) use full::GroupOrderingFull; pub(crate) use partial::GroupOrderingPartial; @@ -43,24 +42,19 @@ impl GroupOrdering { /// Create a `GroupOrdering` for the the specified ordering pub fn try_new( input_schema: &Schema, - ordering: &AggregationOrdering, + mode: &PartitionSearchMode, + ordering: &[PhysicalSortExpr], ) -> Result { - let AggregationOrdering { - mode, - order_indices, - ordering, - } = ordering; - - Ok(match mode { - GroupByOrderMode::PartiallyOrdered => { - let partial = - GroupOrderingPartial::try_new(input_schema, order_indices, ordering)?; - GroupOrdering::Partial(partial) + match mode { + PartitionSearchMode::Linear => Ok(GroupOrdering::None), + PartitionSearchMode::PartiallySorted(order_indices) => { + GroupOrderingPartial::try_new(input_schema, order_indices, ordering) + .map(GroupOrdering::Partial) } - GroupByOrderMode::FullyOrdered => { - GroupOrdering::Full(GroupOrderingFull::new()) + PartitionSearchMode::Sorted => { + Ok(GroupOrdering::Full(GroupOrderingFull::new())) } - }) + } } // How many groups be emitted, or None if no data can be emitted diff --git a/datafusion/physical-plan/src/aggregates/row_hash.rs b/datafusion/physical-plan/src/aggregates/row_hash.rs index f1eece4315e4..7cee4a3e7cfc 100644 --- a/datafusion/physical-plan/src/aggregates/row_hash.rs +++ b/datafusion/physical-plan/src/aggregates/row_hash.rs @@ -17,17 +17,10 @@ //! Hash aggregation -use datafusion_physical_expr::{ - AggregateExpr, EmitTo, GroupsAccumulator, GroupsAccumulatorAdapter, PhysicalSortExpr, -}; -use log::debug; use std::sync::Arc; use std::task::{Context, Poll}; use std::vec; -use futures::ready; -use futures::stream::{Stream, StreamExt}; - use crate::aggregates::group_values::{new_group_values, GroupValues}; use crate::aggregates::order::GroupOrderingFull; use crate::aggregates::{ @@ -39,8 +32,9 @@ use crate::metrics::{BaselineMetrics, RecordOutput}; use crate::sorts::sort::{read_spill_as_stream, sort_batch}; use crate::sorts::streaming_merge; use crate::stream::RecordBatchStreamAdapter; -use crate::{aggregates, PhysicalExpr}; +use crate::{aggregates, ExecutionPlan, PhysicalExpr}; use crate::{RecordBatchStream, SendableRecordBatchStream}; + use arrow::array::*; use arrow::{datatypes::SchemaRef, record_batch::RecordBatch}; use arrow_schema::SortOptions; @@ -50,7 +44,14 @@ use datafusion_execution::memory_pool::proxy::VecAllocExt; use datafusion_execution::memory_pool::{MemoryConsumer, MemoryReservation}; use datafusion_execution::runtime_env::RuntimeEnv; use datafusion_execution::TaskContext; -use datafusion_physical_expr::expressions::col; +use datafusion_physical_expr::expressions::Column; +use datafusion_physical_expr::{ + AggregateExpr, EmitTo, GroupsAccumulator, GroupsAccumulatorAdapter, PhysicalSortExpr, +}; + +use futures::ready; +use futures::stream::{Stream, StreamExt}; +use log::debug; #[derive(Debug, Clone)] /// This object tracks the aggregation phase (input/output) @@ -321,8 +322,9 @@ impl GroupedHashAggregateStream { let spill_expr = group_schema .fields .into_iter() - .map(|field| PhysicalSortExpr { - expr: col(field.name(), &group_schema).unwrap(), + .enumerate() + .map(|(idx, field)| PhysicalSortExpr { + expr: Arc::new(Column::new(field.name().as_str(), idx)) as _, options: SortOptions::default(), }) .collect(); @@ -331,16 +333,14 @@ impl GroupedHashAggregateStream { let reservation = MemoryConsumer::new(name) .with_can_spill(true) .register(context.memory_pool()); - - let group_ordering = agg - .aggregation_ordering - .as_ref() - .map(|aggregation_ordering| { - GroupOrdering::try_new(&group_schema, aggregation_ordering) - }) - // return error if any - .transpose()? - .unwrap_or(GroupOrdering::None); + let (ordering, _) = agg + .equivalence_properties() + .find_longest_permutation(&agg_group_by.output_exprs()); + let group_ordering = GroupOrdering::try_new( + &group_schema, + &agg.partition_search_mode, + ordering.as_slice(), + )?; let group_values = new_group_values(group_schema)?; timer.done(); @@ -418,8 +418,7 @@ impl Stream for GroupedHashAggregateStream { let elapsed_compute = self.baseline_metrics.elapsed_compute().clone(); loop { - let exec_state = self.exec_state.clone(); - match exec_state { + match &self.exec_state { ExecutionState::ReadingInput => { match ready!(self.input.poll_next_unpin(cx)) { // new batch to aggregate @@ -453,14 +452,14 @@ impl Stream for GroupedHashAggregateStream { self.input_done = true; self.group_ordering.input_done(); let timer = elapsed_compute.timer(); - if self.spill_state.spills.is_empty() { + self.exec_state = if self.spill_state.spills.is_empty() { let batch = extract_ok!(self.emit(EmitTo::All, false)); - self.exec_state = ExecutionState::ProducingOutput(batch); + ExecutionState::ProducingOutput(batch) } else { // If spill files exist, stream-merge them. extract_ok!(self.update_merged_stream()); - self.exec_state = ExecutionState::ReadingInput; - } + ExecutionState::ReadingInput + }; timer.done(); } } @@ -468,19 +467,24 @@ impl Stream for GroupedHashAggregateStream { ExecutionState::ProducingOutput(batch) => { // slice off a part of the batch, if needed - let output_batch = if batch.num_rows() <= self.batch_size { - if self.input_done { - self.exec_state = ExecutionState::Done; - } else { - self.exec_state = ExecutionState::ReadingInput - } - batch + let output_batch; + let size = self.batch_size; + (self.exec_state, output_batch) = if batch.num_rows() <= size { + ( + if self.input_done { + ExecutionState::Done + } else { + ExecutionState::ReadingInput + }, + batch.clone(), + ) } else { // output first batch_size rows - let num_remaining = batch.num_rows() - self.batch_size; - let remaining = batch.slice(self.batch_size, num_remaining); - self.exec_state = ExecutionState::ProducingOutput(remaining); - batch.slice(0, self.batch_size) + let size = self.batch_size; + let num_remaining = batch.num_rows() - size; + let remaining = batch.slice(size, num_remaining); + let output = batch.slice(0, size); + (ExecutionState::ProducingOutput(remaining), output) }; return Poll::Ready(Some(Ok( output_batch.record_output(&self.baseline_metrics) diff --git a/datafusion/physical-plan/src/coalesce_batches.rs b/datafusion/physical-plan/src/coalesce_batches.rs index df9e8a8a2b8c..09d1ea87ca37 100644 --- a/datafusion/physical-plan/src/coalesce_batches.rs +++ b/datafusion/physical-plan/src/coalesce_batches.rs @@ -27,8 +27,8 @@ use super::expressions::PhysicalSortExpr; use super::metrics::{BaselineMetrics, ExecutionPlanMetricsSet, MetricsSet}; use super::{DisplayAs, Statistics}; use crate::{ - DisplayFormatType, EquivalenceProperties, ExecutionPlan, Partitioning, - RecordBatchStream, SendableRecordBatchStream, + DisplayFormatType, ExecutionPlan, Partitioning, RecordBatchStream, + SendableRecordBatchStream, }; use arrow::datatypes::SchemaRef; @@ -36,7 +36,7 @@ use arrow::error::Result as ArrowResult; use arrow::record_batch::RecordBatch; use datafusion_common::Result; use datafusion_execution::TaskContext; -use datafusion_physical_expr::OrderingEquivalenceProperties; +use datafusion_physical_expr::EquivalenceProperties; use futures::stream::{Stream, StreamExt}; use log::trace; @@ -138,10 +138,6 @@ impl ExecutionPlan for CoalesceBatchesExec { self.input.equivalence_properties() } - fn ordering_equivalence_properties(&self) -> OrderingEquivalenceProperties { - self.input.ordering_equivalence_properties() - } - fn with_new_children( self: Arc, children: Vec>, @@ -228,17 +224,17 @@ impl CoalesceBatchesStream { let _timer = cloned_time.timer(); match input_batch { Poll::Ready(x) => match x { - Some(Ok(ref batch)) => { + Some(Ok(batch)) => { if batch.num_rows() >= self.target_batch_size && self.buffer.is_empty() { - return Poll::Ready(Some(Ok(batch.clone()))); + return Poll::Ready(Some(Ok(batch))); } else if batch.num_rows() == 0 { // discard empty batches } else { // add to the buffered batches - self.buffer.push(batch.clone()); self.buffered_rows += batch.num_rows(); + self.buffer.push(batch); // check to see if we have enough batches yet if self.buffered_rows >= self.target_batch_size { // combine the batches and return @@ -300,14 +296,14 @@ pub fn concat_batches( batches.len(), row_count ); - let b = arrow::compute::concat_batches(schema, batches)?; - Ok(b) + arrow::compute::concat_batches(schema, batches) } #[cfg(test)] mod tests { use super::*; use crate::{memory::MemoryExec, repartition::RepartitionExec}; + use arrow::datatypes::{DataType, Field, Schema}; use arrow_array::UInt32Array; diff --git a/datafusion/physical-plan/src/coalesce_partitions.rs b/datafusion/physical-plan/src/coalesce_partitions.rs index f09b33e3f3b5..bfcff2853538 100644 --- a/datafusion/physical-plan/src/coalesce_partitions.rs +++ b/datafusion/physical-plan/src/coalesce_partitions.rs @@ -26,11 +26,12 @@ use super::metrics::{BaselineMetrics, ExecutionPlanMetricsSet, MetricsSet}; use super::stream::{ObservedStream, RecordBatchReceiverStream}; use super::{DisplayAs, SendableRecordBatchStream, Statistics}; -use crate::{DisplayFormatType, EquivalenceProperties, ExecutionPlan, Partitioning}; +use crate::{DisplayFormatType, ExecutionPlan, Partitioning}; use arrow::datatypes::SchemaRef; use datafusion_common::{internal_err, DataFusionError, Result}; use datafusion_execution::TaskContext; +use datafusion_physical_expr::EquivalenceProperties; /// Merge execution plan executes partitions in parallel and combines them into a single /// partition. No guarantees are made about the order of the resulting partition. @@ -101,7 +102,10 @@ impl ExecutionPlan for CoalescePartitionsExec { } fn equivalence_properties(&self) -> EquivalenceProperties { - self.input.equivalence_properties() + let mut output_eq = self.input.equivalence_properties(); + // Coalesce partitions loses existing orderings. + output_eq.clear_orderings(); + output_eq } fn benefits_from_input_partitioning(&self) -> Vec { diff --git a/datafusion/physical-plan/src/common.rs b/datafusion/physical-plan/src/common.rs index 649f3a31aa7e..81a59ad7ab3c 100644 --- a/datafusion/physical-plan/src/common.rs +++ b/datafusion/physical-plan/src/common.rs @@ -31,8 +31,10 @@ use arrow::datatypes::Schema; use arrow::ipc::writer::{FileWriter, IpcWriteOptions}; use arrow::record_batch::RecordBatch; use datafusion_common::stats::Precision; +use datafusion_common::tree_node::{Transformed, TreeNode}; use datafusion_common::{plan_err, DataFusionError, Result}; use datafusion_execution::memory_pool::MemoryReservation; +use datafusion_physical_expr::equivalence::ProjectionMapping; use datafusion_physical_expr::expressions::{BinaryExpr, Column}; use datafusion_physical_expr::{PhysicalExpr, PhysicalSortExpr}; @@ -373,6 +375,38 @@ pub fn batch_byte_size(batch: &RecordBatch) -> usize { batch.get_array_memory_size() } +/// Constructs the mapping between a projection's input and output +pub fn calculate_projection_mapping( + expr: &[(Arc, String)], + input_schema: &Arc, +) -> Result { + // Construct a map from the input expressions to the output expression of the projection: + let mut projection_mapping = vec![]; + for (expr_idx, (expression, name)) in expr.iter().enumerate() { + let target_expr = Arc::new(Column::new(name, expr_idx)) as _; + + let source_expr = expression.clone().transform_down(&|e| match e + .as_any() + .downcast_ref::() + { + Some(col) => { + // Sometimes, expression and its name in the input_schema doesn't match. + // This can cause problems. Hence in here we make sure that expression name + // matches with the name in the inout_schema. + // Conceptually, source_expr and expression should be same. + let idx = col.index(); + let matching_input_field = input_schema.field(idx); + let matching_input_column = Column::new(matching_input_field.name(), idx); + Ok(Transformed::Yes(Arc::new(matching_input_column))) + } + None => Ok(Transformed::No(e)), + })?; + + projection_mapping.push((source_expr, target_expr)); + } + Ok(projection_mapping) +} + #[cfg(test)] mod tests { use std::ops::Not; diff --git a/datafusion/physical-plan/src/filter.rs b/datafusion/physical-plan/src/filter.rs index 59f9928d0e21..ce66d614721c 100644 --- a/datafusion/physical-plan/src/filter.rs +++ b/datafusion/physical-plan/src/filter.rs @@ -30,7 +30,7 @@ use super::{ use crate::{ metrics::{BaselineMetrics, ExecutionPlanMetricsSet, MetricsSet}, - Column, DisplayFormatType, EquivalenceProperties, ExecutionPlan, Partitioning, + Column, DisplayFormatType, ExecutionPlan, Partitioning, }; use arrow::compute::filter_record_batch; @@ -42,13 +42,12 @@ use datafusion_common::{plan_err, DataFusionError, Result}; use datafusion_execution::TaskContext; use datafusion_expr::Operator; use datafusion_physical_expr::expressions::BinaryExpr; -use datafusion_physical_expr::{ - analyze, split_conjunction, AnalysisContext, ExprBoundaries, - OrderingEquivalenceProperties, PhysicalExpr, -}; - use datafusion_physical_expr::intervals::utils::check_support; use datafusion_physical_expr::utils::collect_columns; +use datafusion_physical_expr::{ + analyze, split_conjunction, AnalysisContext, EquivalenceProperties, ExprBoundaries, + PhysicalExpr, +}; use futures::stream::{Stream, StreamExt}; use log::trace; @@ -146,37 +145,29 @@ impl ExecutionPlan for FilterExec { } fn equivalence_properties(&self) -> EquivalenceProperties { + let stats = self.statistics().unwrap(); // Combine the equal predicates with the input equivalence properties - let mut input_properties = self.input.equivalence_properties(); - let (equal_pairs, _ne_pairs) = collect_columns_from_predicate(&self.predicate); - for new_condition in equal_pairs { - input_properties.add_equal_conditions(new_condition) + let mut result = self.input.equivalence_properties(); + let (equal_pairs, _) = collect_columns_from_predicate(&self.predicate); + for (lhs, rhs) in equal_pairs { + let lhs_expr = Arc::new(lhs.clone()) as _; + let rhs_expr = Arc::new(rhs.clone()) as _; + result.add_equal_conditions(&lhs_expr, &rhs_expr) } - input_properties - } - - fn ordering_equivalence_properties(&self) -> OrderingEquivalenceProperties { - let stats = self - .statistics() - .expect("Ordering equivalences need to handle the error case of statistics"); // Add the columns that have only one value (singleton) after filtering to constants. let constants = collect_columns(self.predicate()) .into_iter() .filter(|column| stats.column_statistics[column.index()].is_singleton()) - .map(|column| Arc::new(column) as Arc) - .collect::>(); - let filter_oeq = self.input.ordering_equivalence_properties(); - filter_oeq.with_constants(constants) + .map(|column| Arc::new(column) as _); + result.add_constants(constants) } fn with_new_children( self: Arc, - children: Vec>, + mut children: Vec>, ) -> Result> { - Ok(Arc::new(FilterExec::try_new( - self.predicate.clone(), - children[0].clone(), - )?)) + FilterExec::try_new(self.predicate.clone(), children.swap_remove(0)) + .map(|e| Arc::new(e) as _) } fn execute( @@ -355,17 +346,16 @@ impl RecordBatchStream for FilterExecStream { /// Return the equals Column-Pairs and Non-equals Column-Pairs fn collect_columns_from_predicate(predicate: &Arc) -> EqualAndNonEqual { - let mut eq_predicate_columns: Vec<(&Column, &Column)> = Vec::new(); - let mut ne_predicate_columns: Vec<(&Column, &Column)> = Vec::new(); + let mut eq_predicate_columns = Vec::<(&Column, &Column)>::new(); + let mut ne_predicate_columns = Vec::<(&Column, &Column)>::new(); let predicates = split_conjunction(predicate); predicates.into_iter().for_each(|p| { if let Some(binary) = p.as_any().downcast_ref::() { - let left = binary.left(); - let right = binary.right(); - if left.as_any().is::() && right.as_any().is::() { - let left_column = left.as_any().downcast_ref::().unwrap(); - let right_column = right.as_any().downcast_ref::().unwrap(); + if let (Some(left_column), Some(right_column)) = ( + binary.left().as_any().downcast_ref::(), + binary.right().as_any().downcast_ref::(), + ) { match binary.op() { Operator::Eq => { eq_predicate_columns.push((left_column, right_column)) diff --git a/datafusion/physical-plan/src/joins/cross_join.rs b/datafusion/physical-plan/src/joins/cross_join.rs index d8c8064e2ac1..102f0c42e90c 100644 --- a/datafusion/physical-plan/src/joins/cross_join.rs +++ b/datafusion/physical-plan/src/joins/cross_join.rs @@ -27,23 +27,22 @@ use crate::metrics::{ExecutionPlanMetricsSet, MetricsSet}; use crate::DisplayAs; use crate::{ coalesce_batches::concat_batches, coalesce_partitions::CoalescePartitionsExec, - ColumnStatistics, DisplayFormatType, Distribution, EquivalenceProperties, - ExecutionPlan, Partitioning, PhysicalSortExpr, RecordBatchStream, - SendableRecordBatchStream, Statistics, + ColumnStatistics, DisplayFormatType, Distribution, ExecutionPlan, Partitioning, + PhysicalSortExpr, RecordBatchStream, SendableRecordBatchStream, Statistics, }; use arrow::datatypes::{Fields, Schema, SchemaRef}; use arrow::record_batch::RecordBatch; use arrow_array::RecordBatchOptions; use datafusion_common::stats::Precision; -use datafusion_common::{plan_err, DataFusionError, Result, ScalarValue}; +use datafusion_common::{plan_err, DataFusionError, JoinType, Result, ScalarValue}; use datafusion_execution::memory_pool::{MemoryConsumer, MemoryReservation}; use datafusion_execution::TaskContext; +use datafusion_physical_expr::equivalence::join_equivalence_properties; +use datafusion_physical_expr::EquivalenceProperties; use async_trait::async_trait; -use datafusion_physical_expr::equivalence::cross_join_equivalence_properties; -use futures::{ready, StreamExt}; -use futures::{Stream, TryStreamExt}; +use futures::{ready, Stream, StreamExt, TryStreamExt}; /// Data of the left side type JoinLeftData = (RecordBatch, MemoryReservation); @@ -106,12 +105,11 @@ async fn load_left_input( reservation: MemoryReservation, ) -> Result { // merge all left parts into a single stream - let merge = { - if left.output_partitioning().partition_count() != 1 { - Arc::new(CoalescePartitionsExec::new(left.clone())) - } else { - left.clone() - } + let left_schema = left.schema(); + let merge = if left.output_partitioning().partition_count() != 1 { + Arc::new(CoalescePartitionsExec::new(left)) + } else { + left }; let stream = merge.execute(0, context)?; @@ -136,7 +134,7 @@ async fn load_left_input( ) .await?; - let merged_batch = concat_batches(&left.schema(), &batches, num_rows)?; + let merged_batch = concat_batches(&left_schema, &batches, num_rows)?; Ok((merged_batch, reservation)) } @@ -217,12 +215,14 @@ impl ExecutionPlan for CrossJoinExec { } fn equivalence_properties(&self) -> EquivalenceProperties { - let left_columns_len = self.left.schema().fields.len(); - cross_join_equivalence_properties( + join_equivalence_properties( self.left.equivalence_properties(), self.right.equivalence_properties(), - left_columns_len, + &JoinType::Full, self.schema(), + &[false, false], + None, + &[], ) } diff --git a/datafusion/physical-plan/src/joins/hash_join.rs b/datafusion/physical-plan/src/joins/hash_join.rs index dc0e81a6f36e..0e2d552b543b 100644 --- a/datafusion/physical-plan/src/joins/hash_join.rs +++ b/datafusion/physical-plan/src/joins/hash_join.rs @@ -42,8 +42,8 @@ use crate::{ BuildProbeJoinMetrics, ColumnIndex, JoinFilter, JoinOn, }, metrics::{ExecutionPlanMetricsSet, MetricsSet}, - DisplayFormatType, Distribution, EquivalenceProperties, ExecutionPlan, Partitioning, - PhysicalExpr, RecordBatchStream, SendableRecordBatchStream, Statistics, + DisplayFormatType, Distribution, ExecutionPlan, Partitioning, PhysicalExpr, + RecordBatchStream, SendableRecordBatchStream, Statistics, }; use super::{ @@ -55,6 +55,7 @@ use arrow::array::{ Array, ArrayRef, BooleanArray, BooleanBufferBuilder, PrimitiveArray, UInt32Array, UInt32BufferBuilder, UInt64Array, UInt64BufferBuilder, }; +use arrow::compute::kernels::cmp::{eq, not_distinct}; use arrow::compute::{and, take, FilterBuilder}; use arrow::datatypes::{Schema, SchemaRef}; use arrow::record_batch::RecordBatch; @@ -66,13 +67,10 @@ use datafusion_common::{ }; use datafusion_execution::memory_pool::{MemoryConsumer, MemoryReservation}; use datafusion_execution::TaskContext; -use datafusion_physical_expr::OrderingEquivalenceProperties; +use datafusion_physical_expr::equivalence::join_equivalence_properties; +use datafusion_physical_expr::EquivalenceProperties; use ahash::RandomState; -use arrow::compute::kernels::cmp::{eq, not_distinct}; -use datafusion_physical_expr::equivalence::{ - combine_join_equivalence_properties, combine_join_ordering_equivalence_properties, -}; use futures::{ready, Stream, StreamExt, TryStreamExt}; type JoinLeftData = (JoinHashMap, RecordBatch, MemoryReservation); @@ -482,26 +480,14 @@ impl ExecutionPlan for HashJoinExec { } fn equivalence_properties(&self) -> EquivalenceProperties { - let left_columns_len = self.left.schema().fields.len(); - combine_join_equivalence_properties( - self.join_type, + join_equivalence_properties( self.left.equivalence_properties(), self.right.equivalence_properties(), - left_columns_len, - self.on(), - self.schema(), - ) - } - - fn ordering_equivalence_properties(&self) -> OrderingEquivalenceProperties { - combine_join_ordering_equivalence_properties( &self.join_type, - &self.left.ordering_equivalence_properties(), - &self.right.ordering_equivalence_properties(), self.schema(), &self.maintains_input_order(), Some(Self::probe_side()), - self.equivalence_properties(), + self.on(), ) } @@ -533,6 +519,7 @@ impl ExecutionPlan for HashJoinExec { let on_right = self.on.iter().map(|on| on.1.clone()).collect::>(); let left_partitions = self.left.output_partitioning().partition_count(); let right_partitions = self.right.output_partitioning().partition_count(); + if self.mode == PartitionMode::Partitioned && left_partitions != right_partitions { return internal_err!( @@ -635,16 +622,10 @@ async fn collect_left_input( let (left_input, left_input_partition) = if let Some(partition) = partition { (left, partition) + } else if left.output_partitioning().partition_count() != 1 { + (Arc::new(CoalescePartitionsExec::new(left)) as _, 0) } else { - let merge = { - if left.output_partitioning().partition_count() != 1 { - Arc::new(CoalescePartitionsExec::new(left)) - } else { - left - } - }; - - (merge, 0) + (left, 0) }; // Depending on partition argument load single partition or whole left side in memory @@ -1175,24 +1156,22 @@ impl Stream for HashJoinStream { mod tests { use std::sync::Arc; - use arrow::array::{ArrayRef, Date32Array, Int32Array, UInt32Builder, UInt64Builder}; - use arrow::datatypes::{DataType, Field, Schema}; - - use datafusion_common::{assert_batches_sorted_eq, assert_contains, ScalarValue}; - use datafusion_expr::Operator; - use datafusion_physical_expr::expressions::Literal; - use hashbrown::raw::RawTable; - + use super::*; use crate::{ common, expressions::Column, hash_utils::create_hashes, joins::hash_join::build_equal_condition_join_indices, memory::MemoryExec, repartition::RepartitionExec, test::build_table_i32, test::exec::MockExec, }; + + use arrow::array::{ArrayRef, Date32Array, Int32Array, UInt32Builder, UInt64Builder}; + use arrow::datatypes::{DataType, Field, Schema}; + use datafusion_common::{assert_batches_sorted_eq, assert_contains, ScalarValue}; use datafusion_execution::config::SessionConfig; use datafusion_execution::runtime_env::{RuntimeConfig, RuntimeEnv}; - use datafusion_physical_expr::expressions::BinaryExpr; + use datafusion_expr::Operator; + use datafusion_physical_expr::expressions::{BinaryExpr, Literal}; - use super::*; + use hashbrown::raw::RawTable; fn build_table( a: (&str, &Vec), diff --git a/datafusion/physical-plan/src/joins/nested_loop_join.rs b/datafusion/physical-plan/src/joins/nested_loop_join.rs index a113066e39d1..5a77ed6e2907 100644 --- a/datafusion/physical-plan/src/joins/nested_loop_join.rs +++ b/datafusion/physical-plan/src/joins/nested_loop_join.rs @@ -50,7 +50,7 @@ use datafusion_execution::TaskContext; use datafusion_expr::JoinType; use datafusion_physical_expr::{EquivalenceProperties, PhysicalSortExpr}; -use datafusion_physical_expr::equivalence::combine_join_equivalence_properties; +use datafusion_physical_expr::equivalence::join_equivalence_properties; use futures::{ready, Stream, StreamExt, TryStreamExt}; /// Data of the inner table side @@ -192,14 +192,15 @@ impl ExecutionPlan for NestedLoopJoinExec { } fn equivalence_properties(&self) -> EquivalenceProperties { - let left_columns_len = self.left.schema().fields.len(); - combine_join_equivalence_properties( - self.join_type, + join_equivalence_properties( self.left.equivalence_properties(), self.right.equivalence_properties(), - left_columns_len, - &[], // empty join keys + &self.join_type, self.schema(), + &self.maintains_input_order(), + None, + // No on columns in nested loop join + &[], ) } diff --git a/datafusion/physical-plan/src/joins/sort_merge_join.rs b/datafusion/physical-plan/src/joins/sort_merge_join.rs index 759149a64d9f..f6fdc6d77c0c 100644 --- a/datafusion/physical-plan/src/joins/sort_merge_join.rs +++ b/datafusion/physical-plan/src/joins/sort_merge_join.rs @@ -37,9 +37,8 @@ use crate::joins::utils::{ }; use crate::metrics::{ExecutionPlanMetricsSet, MetricBuilder, MetricsSet}; use crate::{ - metrics, DisplayAs, DisplayFormatType, Distribution, EquivalenceProperties, - ExecutionPlan, Partitioning, PhysicalExpr, RecordBatchStream, - SendableRecordBatchStream, Statistics, + metrics, DisplayAs, DisplayFormatType, Distribution, ExecutionPlan, Partitioning, + PhysicalExpr, RecordBatchStream, SendableRecordBatchStream, Statistics, }; use arrow::array::*; @@ -52,11 +51,9 @@ use datafusion_common::{ }; use datafusion_execution::memory_pool::{MemoryConsumer, MemoryReservation}; use datafusion_execution::TaskContext; -use datafusion_physical_expr::{OrderingEquivalenceProperties, PhysicalSortRequirement}; +use datafusion_physical_expr::equivalence::join_equivalence_properties; +use datafusion_physical_expr::{EquivalenceProperties, PhysicalSortRequirement}; -use datafusion_physical_expr::equivalence::{ - combine_join_equivalence_properties, combine_join_ordering_equivalence_properties, -}; use futures::{Stream, StreamExt}; /// join execution plan executes partitions in parallel and combines them into a set of @@ -285,26 +282,14 @@ impl ExecutionPlan for SortMergeJoinExec { } fn equivalence_properties(&self) -> EquivalenceProperties { - let left_columns_len = self.left.schema().fields.len(); - combine_join_equivalence_properties( - self.join_type, + join_equivalence_properties( self.left.equivalence_properties(), self.right.equivalence_properties(), - left_columns_len, - self.on(), - self.schema(), - ) - } - - fn ordering_equivalence_properties(&self) -> OrderingEquivalenceProperties { - combine_join_ordering_equivalence_properties( &self.join_type, - &self.left.ordering_equivalence_properties(), - &self.right.ordering_equivalence_properties(), self.schema(), &self.maintains_input_order(), Some(Self::probe_side(&self.join_type)), - self.equivalence_properties(), + self.on(), ) } diff --git a/datafusion/physical-plan/src/joins/symmetric_hash_join.rs b/datafusion/physical-plan/src/joins/symmetric_hash_join.rs index 00d43aead434..3617893a1c61 100644 --- a/datafusion/physical-plan/src/joins/symmetric_hash_join.rs +++ b/datafusion/physical-plan/src/joins/symmetric_hash_join.rs @@ -63,10 +63,10 @@ use datafusion_common::{ }; use datafusion_execution::memory_pool::MemoryConsumer; use datafusion_execution::TaskContext; +use datafusion_physical_expr::equivalence::join_equivalence_properties; use datafusion_physical_expr::intervals::ExprIntervalGraph; use ahash::RandomState; -use datafusion_physical_expr::equivalence::combine_join_equivalence_properties; use futures::stream::{select, BoxStream}; use futures::{Stream, StreamExt}; use hashbrown::HashSet; @@ -430,14 +430,15 @@ impl ExecutionPlan for SymmetricHashJoinExec { } fn equivalence_properties(&self) -> EquivalenceProperties { - let left_columns_len = self.left.schema().fields.len(); - combine_join_equivalence_properties( - self.join_type, + join_equivalence_properties( self.left.equivalence_properties(), self.right.equivalence_properties(), - left_columns_len, - self.on(), + &self.join_type, self.schema(), + &self.maintains_input_order(), + // Has alternating probe side + None, + self.on(), ) } diff --git a/datafusion/physical-plan/src/joins/utils.rs b/datafusion/physical-plan/src/joins/utils.rs index cf150ddf575f..c91dc92fbc7a 100644 --- a/datafusion/physical-plan/src/joins/utils.rs +++ b/datafusion/physical-plan/src/joins/utils.rs @@ -40,12 +40,12 @@ use datafusion_common::{ plan_datafusion_err, plan_err, DataFusionError, JoinSide, JoinType, Result, SharedResult, }; +use datafusion_physical_expr::equivalence::add_offset_to_expr; use datafusion_physical_expr::expressions::Column; use datafusion_physical_expr::intervals::{ExprIntervalGraph, Interval, IntervalBound}; use datafusion_physical_expr::utils::merge_vectors; use datafusion_physical_expr::{ - add_offset_to_expr, add_offset_to_lex_ordering, LexOrdering, LexOrderingRef, - PhysicalExpr, PhysicalSortExpr, + LexOrdering, LexOrderingRef, PhysicalExpr, PhysicalSortExpr, }; use futures::future::{BoxFuture, Shared}; @@ -91,8 +91,8 @@ fn check_join_set_is_valid( if !left_missing.is_empty() | !right_missing.is_empty() { return plan_err!( - "The left or right side of the join does not have all columns on \"on\": \nMissing on the left: {left_missing:?}\nMissing on the right: {right_missing:?}" - ); + "The left or right side of the join does not have all columns on \"on\": \nMissing on the left: {left_missing:?}\nMissing on the right: {right_missing:?}" + ); }; Ok(()) @@ -133,7 +133,7 @@ pub fn adjust_right_output_partitioning( let new_exprs = exprs .into_iter() .map(|expr| add_offset_to_expr(expr, left_columns_len)) - .collect::>(); + .collect(); Partitioning::Hash(new_exprs, size) } } @@ -169,23 +169,22 @@ pub fn calculate_join_output_ordering( maintains_input_order: &[bool], probe_side: Option, ) -> Option { - // All joins have 2 children: - assert_eq!(maintains_input_order.len(), 2); - let left_maintains = maintains_input_order[0]; - let right_maintains = maintains_input_order[1]; let mut right_ordering = match join_type { // In the case below, right ordering should be offseted with the left // side length, since we append the right table to the left table. JoinType::Inner | JoinType::Left | JoinType::Right | JoinType::Full => { - add_offset_to_lex_ordering(right_ordering, left_columns_len) + right_ordering + .iter() + .map(|sort_expr| PhysicalSortExpr { + expr: add_offset_to_expr(sort_expr.expr.clone(), left_columns_len), + options: sort_expr.options, + }) + .collect() } _ => right_ordering.to_vec(), }; - let output_ordering = match (left_maintains, right_maintains) { - (true, true) => { - unreachable!("Cannot maintain ordering of both sides"); - } - (true, false) => { + let output_ordering = match maintains_input_order { + [true, false] => { // Special case, we can prefix ordering of right side with the ordering of left side. if join_type == JoinType::Inner && probe_side == Some(JoinSide::Left) { replace_on_columns_of_right_ordering( @@ -198,7 +197,7 @@ pub fn calculate_join_output_ordering( left_ordering.to_vec() } } - (false, true) => { + [false, true] => { // Special case, we can prefix ordering of left side with the ordering of right side. if join_type == JoinType::Inner && probe_side == Some(JoinSide::Right) { replace_on_columns_of_right_ordering( @@ -212,7 +211,9 @@ pub fn calculate_join_output_ordering( } } // Doesn't maintain ordering, output ordering is None. - (false, false) => return None, + [false, false] => return None, + [true, true] => unreachable!("Cannot maintain ordering of both sides"), + _ => unreachable!("Join operators can not have more than two children"), }; (!output_ordering.is_empty()).then_some(output_ordering) } diff --git a/datafusion/physical-plan/src/lib.rs b/datafusion/physical-plan/src/lib.rs index ed795a1cb333..694a6d3e5c3e 100644 --- a/datafusion/physical-plan/src/lib.rs +++ b/datafusion/physical-plan/src/lib.rs @@ -33,7 +33,6 @@ use datafusion_common::tree_node::Transformed; use datafusion_common::utils::DataPtr; use datafusion_common::{plan_err, DataFusionError, Result}; use datafusion_execution::TaskContext; -use datafusion_physical_expr::equivalence::OrderingEquivalenceProperties; use datafusion_physical_expr::expressions::Column; use datafusion_physical_expr::{ EquivalenceProperties, PhysicalSortExpr, PhysicalSortRequirement, @@ -83,8 +82,7 @@ pub use datafusion_common::{internal_err, ColumnStatistics, Statistics}; pub use datafusion_expr::{Accumulator, ColumnarValue}; pub use datafusion_physical_expr::window::WindowExpr; pub use datafusion_physical_expr::{ - expressions, functions, ordering_equivalence_properties_helper, udf, AggregateExpr, - Distribution, Partitioning, PhysicalExpr, + expressions, functions, udf, AggregateExpr, Distribution, Partitioning, PhysicalExpr, }; // Backwards compatibility @@ -205,16 +203,11 @@ pub trait ExecutionPlan: Debug + DisplayAs + Send + Sync { .collect() } - /// Get the EquivalenceProperties within the plan + /// Get the [`EquivalenceProperties`] within the plan fn equivalence_properties(&self) -> EquivalenceProperties { EquivalenceProperties::new(self.schema()) } - /// Get the OrderingEquivalenceProperties within the plan - fn ordering_equivalence_properties(&self) -> OrderingEquivalenceProperties { - OrderingEquivalenceProperties::new(self.schema()) - } - /// Get a list of `ExecutionPlan` that provide input for this plan. The /// returned list will be empty for leaf nodes such as scans, will contain a /// single value for unary nodes, or two values for binary nodes (such as diff --git a/datafusion/physical-plan/src/limit.rs b/datafusion/physical-plan/src/limit.rs index 9fe1eb0763ba..945dad16b794 100644 --- a/datafusion/physical-plan/src/limit.rs +++ b/datafusion/physical-plan/src/limit.rs @@ -35,7 +35,6 @@ use arrow::record_batch::{RecordBatch, RecordBatchOptions}; use datafusion_common::stats::Precision; use datafusion_common::{internal_err, DataFusionError, Result}; use datafusion_execution::TaskContext; -use datafusion_physical_expr::OrderingEquivalenceProperties; use futures::stream::{Stream, StreamExt}; use log::trace; @@ -138,10 +137,6 @@ impl ExecutionPlan for GlobalLimitExec { self.input.equivalence_properties() } - fn ordering_equivalence_properties(&self) -> OrderingEquivalenceProperties { - self.input.ordering_equivalence_properties() - } - fn with_new_children( self: Arc, children: Vec>, @@ -327,10 +322,6 @@ impl ExecutionPlan for LocalLimitExec { self.input.equivalence_properties() } - fn ordering_equivalence_properties(&self) -> OrderingEquivalenceProperties { - self.input.ordering_equivalence_properties() - } - fn unbounded_output(&self, _children: &[bool]) -> Result { Ok(false) } diff --git a/datafusion/physical-plan/src/memory.rs b/datafusion/physical-plan/src/memory.rs index b53500579266..5f1660a225b9 100644 --- a/datafusion/physical-plan/src/memory.rs +++ b/datafusion/physical-plan/src/memory.rs @@ -27,13 +27,12 @@ use super::{ common, DisplayAs, DisplayFormatType, ExecutionPlan, Partitioning, RecordBatchStream, SendableRecordBatchStream, Statistics, }; -use crate::ordering_equivalence_properties_helper; use arrow::datatypes::SchemaRef; use arrow::record_batch::RecordBatch; use datafusion_common::{internal_err, project_schema, DataFusionError, Result}; use datafusion_execution::TaskContext; -use datafusion_physical_expr::{LexOrdering, OrderingEquivalenceProperties}; +use datafusion_physical_expr::{EquivalenceProperties, LexOrdering}; use futures::Stream; @@ -122,8 +121,8 @@ impl ExecutionPlan for MemoryExec { .map(|ordering| ordering.as_slice()) } - fn ordering_equivalence_properties(&self) -> OrderingEquivalenceProperties { - ordering_equivalence_properties_helper(self.schema(), &self.sort_information) + fn equivalence_properties(&self) -> EquivalenceProperties { + EquivalenceProperties::new_with_orderings(self.schema(), &self.sort_information) } fn with_new_children( @@ -179,7 +178,7 @@ impl MemoryExec { } /// A memory table can be ordered by multiple expressions simultaneously. - /// `OrderingEquivalenceProperties` keeps track of expressions that describe the + /// [`EquivalenceProperties`] keeps track of expressions that describe the /// global ordering of the schema. These columns are not necessarily same; e.g. /// ```text /// ┌-------┐ @@ -192,10 +191,8 @@ impl MemoryExec { /// └---┴---┘ /// ``` /// where both `a ASC` and `b DESC` can describe the table ordering. With - /// `OrderingEquivalenceProperties`, we can keep track of these equivalences - /// and treat `a ASC` and `b DESC` as the same ordering requirement - /// by outputting the `a ASC` from output_ordering API - /// and add `b DESC` into `OrderingEquivalenceProperties` + /// [`EquivalenceProperties`], we can keep track of these equivalences + /// and treat `a ASC` and `b DESC` as the same ordering requirement. pub fn with_sort_information(mut self, sort_information: Vec) -> Self { self.sort_information = sort_information; self @@ -303,11 +300,8 @@ mod tests { .with_sort_information(sort_information); assert_eq!(mem_exec.output_ordering().unwrap(), expected_output_order); - let order_eq = mem_exec.ordering_equivalence_properties(); - assert!(order_eq - .oeq_class() - .map(|class| class.contains(&expected_order_eq)) - .unwrap_or(false)); + let eq_properties = mem_exec.equivalence_properties(); + assert!(eq_properties.oeq_class().contains(&expected_order_eq)); Ok(()) } } diff --git a/datafusion/physical-plan/src/projection.rs b/datafusion/physical-plan/src/projection.rs index a374154c995c..d4242f20a8a3 100644 --- a/datafusion/physical-plan/src/projection.rs +++ b/datafusion/physical-plan/src/projection.rs @@ -29,9 +29,9 @@ use std::task::{Context, Poll}; use super::expressions::{Column, PhysicalSortExpr}; use super::metrics::{BaselineMetrics, ExecutionPlanMetricsSet, MetricsSet}; use super::{DisplayAs, RecordBatchStream, SendableRecordBatchStream, Statistics}; +use crate::common::calculate_projection_mapping; use crate::{ - ColumnStatistics, DisplayFormatType, EquivalenceProperties, ExecutionPlan, - Partitioning, PhysicalExpr, + ColumnStatistics, DisplayFormatType, ExecutionPlan, Partitioning, PhysicalExpr, }; use arrow::datatypes::{Field, Schema, SchemaRef}; @@ -40,11 +40,7 @@ use datafusion_common::stats::Precision; use datafusion_common::Result; use datafusion_execution::TaskContext; use datafusion_physical_expr::expressions::{Literal, UnKnownColumn}; -use datafusion_physical_expr::utils::find_orderings_of_exprs; -use datafusion_physical_expr::{ - normalize_out_expr_with_columns_map, project_equivalence_properties, - project_ordering_equivalence_properties, OrderingEquivalenceProperties, -}; +use datafusion_physical_expr::EquivalenceProperties; use futures::stream::{Stream, StreamExt}; use log::trace; @@ -60,15 +56,12 @@ pub struct ProjectionExec { input: Arc, /// The output ordering output_ordering: Option>, - /// The columns map used to normalize out expressions like Partitioning and PhysicalSortExpr - /// The key is the column from the input schema and the values are the columns from the output schema - columns_map: HashMap>, + /// The mapping used to normalize expressions like Partitioning and + /// PhysicalSortExpr. The key is the expression from the input schema + /// and the value is the expression from the output schema. + projection_mapping: Vec<(Arc, Arc)>, /// Execution metrics metrics: ExecutionPlanMetricsSet, - /// Expressions' normalized orderings (as given by the output ordering API - /// and normalized with respect to equivalence classes of input plan). The - /// projected expressions are mapped by their indices to this vector. - orderings: Vec>, } impl ProjectionExec { @@ -100,63 +93,20 @@ impl ProjectionExec { input_schema.metadata().clone(), )); - // construct a map from the input columns to the output columns of the Projection - let mut columns_map: HashMap> = HashMap::new(); - for (expr_idx, (expression, name)) in expr.iter().enumerate() { - if let Some(column) = expression.as_any().downcast_ref::() { - // For some executors, logical and physical plan schema fields - // are not the same. The information in a `Column` comes from - // the logical plan schema. Therefore, to produce correct results - // we use the field in the input schema with the same index. This - // corresponds to the physical plan `Column`. - let idx = column.index(); - let matching_input_field = input_schema.field(idx); - let matching_input_column = Column::new(matching_input_field.name(), idx); - let entry = columns_map.entry(matching_input_column).or_default(); - entry.push(Column::new(name, expr_idx)); - }; - } - - // Output Ordering need to respect the alias - let child_output_ordering = input.output_ordering(); - let output_ordering = match child_output_ordering { - Some(sort_exprs) => { - let normalized_exprs = sort_exprs - .iter() - .map(|sort_expr| { - let expr = normalize_out_expr_with_columns_map( - sort_expr.expr.clone(), - &columns_map, - ); - PhysicalSortExpr { - expr, - options: sort_expr.options, - } - }) - .collect::>(); - Some(normalized_exprs) - } - None => None, - }; - - let orderings = find_orderings_of_exprs( - &expr, - input.output_ordering(), - input.equivalence_properties(), - input.ordering_equivalence_properties(), - )?; + // construct a map from the input expressions to the output expression of the Projection + let projection_mapping = calculate_projection_mapping(&expr, &input_schema)?; - let output_ordering = - validate_output_ordering(output_ordering, &orderings, &expr); + let input_eqs = input.equivalence_properties(); + let project_eqs = input_eqs.project(&projection_mapping, schema.clone()); + let output_ordering = project_eqs.oeq_class().output_ordering(); Ok(Self { expr, schema, input, output_ordering, - columns_map, + projection_mapping, metrics: ExecutionPlanMetricsSet::new(), - orderings, }) } @@ -224,11 +174,18 @@ impl ExecutionPlan for ProjectionExec { fn output_partitioning(&self) -> Partitioning { // Output partition need to respect the alias let input_partition = self.input.output_partitioning(); + let input_eq_properties = self.input.equivalence_properties(); if let Partitioning::Hash(exprs, part) = input_partition { let normalized_exprs = exprs .into_iter() - .map(|expr| normalize_out_expr_with_columns_map(expr, &self.columns_map)) - .collect::>(); + .map(|expr| { + input_eq_properties + .project_expr(&expr, &self.projection_mapping) + .unwrap_or_else(|| { + Arc::new(UnKnownColumn::new(&expr.to_string())) + }) + }) + .collect(); Partitioning::Hash(normalized_exprs, part) } else { input_partition @@ -245,58 +202,17 @@ impl ExecutionPlan for ProjectionExec { } fn equivalence_properties(&self) -> EquivalenceProperties { - let mut new_properties = EquivalenceProperties::new(self.schema()); - project_equivalence_properties( - self.input.equivalence_properties(), - &self.columns_map, - &mut new_properties, - ); - new_properties - } - - fn ordering_equivalence_properties(&self) -> OrderingEquivalenceProperties { - let mut new_properties = OrderingEquivalenceProperties::new(self.schema()); - if self.output_ordering.is_none() { - // If there is no output ordering, return an "empty" equivalence set: - return new_properties; - } - - let input_oeq = self.input().ordering_equivalence_properties(); - - project_ordering_equivalence_properties( - input_oeq, - &self.columns_map, - &mut new_properties, - ); - - if let Some(leading_ordering) = self - .output_ordering - .as_ref() - .map(|output_ordering| &output_ordering[0]) - { - for order in self.orderings.iter().flatten() { - if !order.eq(leading_ordering) - && !new_properties.satisfies_leading_ordering(order) - { - new_properties.add_equal_conditions(( - &vec![leading_ordering.clone()], - &vec![order.clone()], - )); - } - } - } - - new_properties + self.input + .equivalence_properties() + .project(&self.projection_mapping, self.schema()) } fn with_new_children( self: Arc, - children: Vec>, + mut children: Vec>, ) -> Result> { - Ok(Arc::new(ProjectionExec::try_new( - self.expr.clone(), - children[0].clone(), - )?)) + ProjectionExec::try_new(self.expr.clone(), children.swap_remove(0)) + .map(|p| Arc::new(p) as _) } fn benefits_from_input_partitioning(&self) -> Vec { @@ -336,40 +252,6 @@ impl ExecutionPlan for ProjectionExec { } } -/// This function takes the current `output_ordering`, the `orderings` based on projected expressions, -/// and the `expr` representing the projected expressions themselves. It aims to ensure that the output -/// ordering is valid and correctly corresponds to the projected columns. -/// -/// If the leading expression in the `output_ordering` is an [`UnKnownColumn`], it indicates that the column -/// referenced in the ordering is not found among the projected expressions. In such cases, this function -/// attempts to create a new output ordering by referring to valid columns from the leftmost side of the -/// expressions that have an ordering specified. -fn validate_output_ordering( - output_ordering: Option>, - orderings: &[Option], - expr: &[(Arc, String)], -) -> Option> { - output_ordering.and_then(|ordering| { - // If the leading expression is invalid column, change output - // ordering of the projection so that it refers to valid columns if - // possible. - if ordering[0].expr.as_any().is::() { - for (idx, order) in orderings.iter().enumerate() { - if let Some(sort_expr) = order { - let (_, col_name) = &expr[idx]; - return Some(vec![PhysicalSortExpr { - expr: Arc::new(Column::new(col_name, idx)), - options: sort_expr.options, - }]); - } - } - None - } else { - Some(ordering) - } - }) -} - /// If e is a direct column reference, returns the field level /// metadata for that field, if any. Otherwise returns None fn get_field_metadata( @@ -486,6 +368,7 @@ mod tests { use crate::common::collect; use crate::expressions; use crate::test; + use arrow_schema::DataType; use datafusion_common::ScalarValue; diff --git a/datafusion/physical-plan/src/repartition/mod.rs b/datafusion/physical-plan/src/repartition/mod.rs index 82801bcdfdb3..66f7037e5c2d 100644 --- a/datafusion/physical-plan/src/repartition/mod.rs +++ b/datafusion/physical-plan/src/repartition/mod.rs @@ -29,9 +29,7 @@ use crate::hash_utils::create_hashes; use crate::metrics::BaselineMetrics; use crate::repartition::distributor_channels::{channels, partition_aware_channels}; use crate::sorts::streaming_merge; -use crate::{ - DisplayFormatType, EquivalenceProperties, ExecutionPlan, Partitioning, Statistics, -}; +use crate::{DisplayFormatType, ExecutionPlan, Partitioning, Statistics}; use self::distributor_channels::{DistributionReceiver, DistributionSender}; @@ -46,7 +44,7 @@ use arrow::record_batch::RecordBatch; use datafusion_common::{not_impl_err, DataFusionError, Result}; use datafusion_execution::memory_pool::MemoryConsumer; use datafusion_execution::TaskContext; -use datafusion_physical_expr::{OrderingEquivalenceProperties, PhysicalExpr}; +use datafusion_physical_expr::{EquivalenceProperties, PhysicalExpr}; use futures::stream::Stream; use futures::{FutureExt, StreamExt}; @@ -427,12 +425,11 @@ impl ExecutionPlan for RepartitionExec { fn with_new_children( self: Arc, - children: Vec>, + mut children: Vec>, ) -> Result> { let repartition = - RepartitionExec::try_new(children[0].clone(), self.partitioning.clone())? - .with_preserve_order(self.preserve_order); - Ok(Arc::new(repartition)) + RepartitionExec::try_new(children.swap_remove(0), self.partitioning.clone()); + repartition.map(|r| Arc::new(r.with_preserve_order(self.preserve_order)) as _) } /// Specifies whether this plan generates an infinite stream of records. @@ -468,11 +465,15 @@ impl ExecutionPlan for RepartitionExec { } fn equivalence_properties(&self) -> EquivalenceProperties { - self.input.equivalence_properties() - } - - fn ordering_equivalence_properties(&self) -> OrderingEquivalenceProperties { - self.input.ordering_equivalence_properties() + let mut result = self.input.equivalence_properties(); + // If the ordering is lost, reset the ordering equivalence class. + if !self.maintains_input_order()[0] { + result.clear_orderings(); + } + if self.preserve_order { + result = result.with_reorder(self.sort_exprs().unwrap_or_default().to_vec()) + } + result } fn execute( diff --git a/datafusion/physical-plan/src/sorts/sort.rs b/datafusion/physical-plan/src/sorts/sort.rs index 08fa2c25d792..2d8237011fff 100644 --- a/datafusion/physical-plan/src/sorts/sort.rs +++ b/datafusion/physical-plan/src/sorts/sort.rs @@ -835,7 +835,10 @@ impl ExecutionPlan for SortExec { } fn equivalence_properties(&self) -> EquivalenceProperties { - self.input.equivalence_properties() + // Reset the ordering equivalence class with the new ordering: + self.input + .equivalence_properties() + .with_reorder(self.expr.to_vec()) } fn with_new_children( diff --git a/datafusion/physical-plan/src/sorts/sort_preserving_merge.rs b/datafusion/physical-plan/src/sorts/sort_preserving_merge.rs index ee044d9172fb..65cd8e41480e 100644 --- a/datafusion/physical-plan/src/sorts/sort_preserving_merge.rs +++ b/datafusion/physical-plan/src/sorts/sort_preserving_merge.rs @@ -33,9 +33,7 @@ use arrow::datatypes::SchemaRef; use datafusion_common::{internal_err, DataFusionError, Result}; use datafusion_execution::memory_pool::MemoryConsumer; use datafusion_execution::TaskContext; -use datafusion_physical_expr::{ - EquivalenceProperties, OrderingEquivalenceProperties, PhysicalSortRequirement, -}; +use datafusion_physical_expr::{EquivalenceProperties, PhysicalSortRequirement}; use log::{debug, trace}; @@ -176,11 +174,8 @@ impl ExecutionPlan for SortPreservingMergeExec { } fn equivalence_properties(&self) -> EquivalenceProperties { - self.input.equivalence_properties() - } - - fn ordering_equivalence_properties(&self) -> OrderingEquivalenceProperties { - self.input.ordering_equivalence_properties() + let output_oeq = self.input.equivalence_properties(); + output_oeq.with_reorder(self.expr.to_vec()) } fn children(&self) -> Vec> { @@ -271,6 +266,8 @@ impl ExecutionPlan for SortPreservingMergeExec { #[cfg(test)] mod tests { + use std::iter::FromIterator; + use super::*; use crate::coalesce_partitions::CoalescePartitionsExec; use crate::expressions::col; @@ -281,8 +278,8 @@ mod tests { use crate::test::exec::{assert_strong_count_converges_to_zero, BlockingExec}; use crate::test::{self, assert_is_pending, make_partition}; use crate::{collect, common}; - use arrow::array::ArrayRef; - use arrow::array::{Int32Array, StringArray, TimestampNanosecondArray}; + + use arrow::array::{ArrayRef, Int32Array, StringArray, TimestampNanosecondArray}; use arrow::compute::SortOptions; use arrow::datatypes::{DataType, Field, Schema}; use arrow::record_batch::RecordBatch; @@ -290,7 +287,6 @@ mod tests { use datafusion_execution::config::SessionConfig; use futures::{FutureExt, StreamExt}; - use std::iter::FromIterator; #[tokio::test] async fn test_merge_interleave() { diff --git a/datafusion/physical-plan/src/streaming.rs b/datafusion/physical-plan/src/streaming.rs index 77b56e1d7540..1923a5f3abad 100644 --- a/datafusion/physical-plan/src/streaming.rs +++ b/datafusion/physical-plan/src/streaming.rs @@ -28,7 +28,7 @@ use crate::{ExecutionPlan, Partitioning, SendableRecordBatchStream}; use arrow::datatypes::SchemaRef; use datafusion_common::{internal_err, plan_err, DataFusionError, Result}; use datafusion_execution::TaskContext; -use datafusion_physical_expr::{LexOrdering, PhysicalSortExpr}; +use datafusion_physical_expr::{EquivalenceProperties, LexOrdering, PhysicalSortExpr}; use async_trait::async_trait; use futures::stream::StreamExt; @@ -163,6 +163,14 @@ impl ExecutionPlan for StreamingTableExec { self.projected_output_ordering.as_deref() } + fn equivalence_properties(&self) -> EquivalenceProperties { + let mut result = EquivalenceProperties::new(self.schema()); + if let Some(ordering) = &self.projected_output_ordering { + result.add_new_orderings([ordering.clone()]) + } + result + } + fn children(&self) -> Vec> { vec![] } diff --git a/datafusion/physical-plan/src/union.rs b/datafusion/physical-plan/src/union.rs index 2727a6e161be..9700605ce406 100644 --- a/datafusion/physical-plan/src/union.rs +++ b/datafusion/physical-plan/src/union.rs @@ -40,6 +40,7 @@ use arrow::record_batch::RecordBatch; use datafusion_common::stats::Precision; use datafusion_common::{exec_err, internal_err, DFSchemaRef, DataFusionError, Result}; use datafusion_execution::TaskContext; +use datafusion_physical_expr::EquivalenceProperties; use futures::Stream; use itertools::Itertools; @@ -222,6 +223,46 @@ impl ExecutionPlan for UnionExec { } } + fn equivalence_properties(&self) -> EquivalenceProperties { + // TODO: In some cases, we should be able to preserve some equivalence + // classes and constants. Add support for such cases. + let children_eqs = self + .inputs + .iter() + .map(|child| child.equivalence_properties()) + .collect::>(); + let mut result = EquivalenceProperties::new(self.schema()); + // Use the ordering equivalence class of the first child as the seed: + let mut meets = children_eqs[0] + .oeq_class() + .iter() + .map(|item| item.to_vec()) + .collect::>(); + // Iterate over all the children: + for child_eqs in &children_eqs[1..] { + // Compute meet orderings of the current meets and the new ordering + // equivalence class. + let mut idx = 0; + while idx < meets.len() { + // Find all the meets of `current_meet` with this child's orderings: + let valid_meets = child_eqs.oeq_class().iter().filter_map(|ordering| { + child_eqs.get_meet_ordering(ordering, &meets[idx]) + }); + // Use the longest of these meets as others are redundant: + if let Some(next_meet) = valid_meets.max_by_key(|m| m.len()) { + meets[idx] = next_meet; + idx += 1; + } else { + meets.swap_remove(idx); + } + } + } + // We know have all the valid orderings after union, remove redundant + // entries (implicitly) and return: + result.add_new_orderings(meets); + result + } + fn with_new_children( self: Arc, children: Vec>, @@ -596,10 +637,41 @@ fn stats_union(mut left: Statistics, right: Statistics) -> Statistics { mod tests { use super::*; use crate::collect; + use crate::memory::MemoryExec; use crate::test; use arrow::record_batch::RecordBatch; + use arrow_schema::{DataType, SortOptions}; use datafusion_common::ScalarValue; + use datafusion_physical_expr::expressions::col; + use datafusion_physical_expr::PhysicalExpr; + + // Generate a schema which consists of 7 columns (a, b, c, d, e, f, g) + fn create_test_schema() -> Result { + let a = Field::new("a", DataType::Int32, true); + let b = Field::new("b", DataType::Int32, true); + let c = Field::new("c", DataType::Int32, true); + let d = Field::new("d", DataType::Int32, true); + let e = Field::new("e", DataType::Int32, true); + let f = Field::new("f", DataType::Int32, true); + let g = Field::new("g", DataType::Int32, true); + let schema = Arc::new(Schema::new(vec![a, b, c, d, e, f, g])); + + Ok(schema) + } + + // Convert each tuple to PhysicalSortExpr + fn convert_to_sort_exprs( + in_data: &[(&Arc, SortOptions)], + ) -> Vec { + in_data + .iter() + .map(|(expr, options)| PhysicalSortExpr { + expr: (*expr).clone(), + options: *options, + }) + .collect::>() + } #[tokio::test] async fn test_union_partitions() -> Result<()> { @@ -712,4 +784,105 @@ mod tests { assert_eq!(result, expected); } + + #[tokio::test] + async fn test_union_equivalence_properties() -> Result<()> { + let schema = create_test_schema()?; + let col_a = &col("a", &schema)?; + let col_b = &col("b", &schema)?; + let col_c = &col("c", &schema)?; + let col_d = &col("d", &schema)?; + let col_e = &col("e", &schema)?; + let col_f = &col("f", &schema)?; + let options = SortOptions::default(); + let test_cases = vec![ + //-----------TEST CASE 1----------// + ( + // First child orderings + vec![ + // [a ASC, b ASC, f ASC] + vec![(col_a, options), (col_b, options), (col_f, options)], + ], + // Second child orderings + vec![ + // [a ASC, b ASC, c ASC] + vec![(col_a, options), (col_b, options), (col_c, options)], + // [a ASC, b ASC, f ASC] + vec![(col_a, options), (col_b, options), (col_f, options)], + ], + // Union output orderings + vec![ + // [a ASC, b ASC, f ASC] + vec![(col_a, options), (col_b, options), (col_f, options)], + ], + ), + //-----------TEST CASE 2----------// + ( + // First child orderings + vec![ + // [a ASC, b ASC, f ASC] + vec![(col_a, options), (col_b, options), (col_f, options)], + // d ASC + vec![(col_d, options)], + ], + // Second child orderings + vec![ + // [a ASC, b ASC, c ASC] + vec![(col_a, options), (col_b, options), (col_c, options)], + // [e ASC] + vec![(col_e, options)], + ], + // Union output orderings + vec![ + // [a ASC, b ASC] + vec![(col_a, options), (col_b, options)], + ], + ), + ]; + + for ( + test_idx, + (first_child_orderings, second_child_orderings, union_orderings), + ) in test_cases.iter().enumerate() + { + let first_orderings = first_child_orderings + .iter() + .map(|ordering| convert_to_sort_exprs(ordering)) + .collect::>(); + let second_orderings = second_child_orderings + .iter() + .map(|ordering| convert_to_sort_exprs(ordering)) + .collect::>(); + let union_expected_orderings = union_orderings + .iter() + .map(|ordering| convert_to_sort_exprs(ordering)) + .collect::>(); + let child1 = Arc::new( + MemoryExec::try_new(&[], schema.clone(), None)? + .with_sort_information(first_orderings), + ); + let child2 = Arc::new( + MemoryExec::try_new(&[], schema.clone(), None)? + .with_sort_information(second_orderings), + ); + + let union = UnionExec::new(vec![child1, child2]); + let union_eq_properties = union.equivalence_properties(); + let union_actual_orderings = union_eq_properties.oeq_class(); + let err_msg = format!( + "Error in test id: {:?}, test case: {:?}", + test_idx, test_cases[test_idx] + ); + assert_eq!( + union_actual_orderings.len(), + union_expected_orderings.len(), + "{}", + err_msg + ); + for expected in &union_expected_orderings { + assert!(union_actual_orderings.contains(expected), "{}", err_msg); + } + } + Ok(()) + } } diff --git a/datafusion/physical-plan/src/unnest.rs b/datafusion/physical-plan/src/unnest.rs index 30f109953cbb..c9f3fb76c2e5 100644 --- a/datafusion/physical-plan/src/unnest.rs +++ b/datafusion/physical-plan/src/unnest.rs @@ -23,9 +23,8 @@ use std::{any::Any, sync::Arc}; use super::DisplayAs; use crate::{ - expressions::Column, DisplayFormatType, Distribution, EquivalenceProperties, - ExecutionPlan, Partitioning, PhysicalExpr, PhysicalSortExpr, RecordBatchStream, - SendableRecordBatchStream, + expressions::Column, DisplayFormatType, Distribution, ExecutionPlan, Partitioning, + PhysicalExpr, PhysicalSortExpr, RecordBatchStream, SendableRecordBatchStream, }; use arrow::array::{ @@ -136,10 +135,6 @@ impl ExecutionPlan for UnnestExec { None } - fn equivalence_properties(&self) -> EquivalenceProperties { - self.input.equivalence_properties() - } - fn execute( &self, partition: usize, diff --git a/datafusion/physical-plan/src/windows/bounded_window_agg_exec.rs b/datafusion/physical-plan/src/windows/bounded_window_agg_exec.rs index f6ffe2e26795..fb679b013863 100644 --- a/datafusion/physical-plan/src/windows/bounded_window_agg_exec.rs +++ b/datafusion/physical-plan/src/windows/bounded_window_agg_exec.rs @@ -30,8 +30,8 @@ use std::task::{Context, Poll}; use crate::expressions::PhysicalSortExpr; use crate::metrics::{BaselineMetrics, ExecutionPlanMetricsSet, MetricsSet}; use crate::windows::{ - calc_requirements, get_ordered_partition_by_indices, window_ordering_equivalence, - PartitionSearchMode, + calc_requirements, get_ordered_partition_by_indices, get_partition_by_sort_exprs, + window_equivalence_properties, PartitionSearchMode, }; use crate::{ ColumnStatistics, DisplayAs, DisplayFormatType, Distribution, ExecutionPlan, @@ -58,8 +58,7 @@ use datafusion_physical_expr::window::{ PartitionBatches, PartitionKey, PartitionWindowAggStates, WindowState, }; use datafusion_physical_expr::{ - EquivalenceProperties, OrderingEquivalenceProperties, PhysicalExpr, - PhysicalSortRequirement, + EquivalenceProperties, PhysicalExpr, PhysicalSortRequirement, }; use ahash::RandomState; @@ -150,9 +149,12 @@ impl BoundedWindowAggExec { // Hence returned `PhysicalSortExpr` corresponding to `PARTITION BY` columns can be used safely // to calculate partition separation points pub fn partition_by_sort_keys(&self) -> Result> { - // Partition by sort keys indices are stored in self.ordered_partition_by_indices. - let sort_keys = self.input.output_ordering().unwrap_or(&[]); - get_at_indices(sort_keys, &self.ordered_partition_by_indices) + let partition_by = self.window_expr()[0].partition_by(); + get_partition_by_sort_exprs( + &self.input, + partition_by, + &self.ordered_partition_by_indices, + ) } /// Initializes the appropriate [`PartitionSearcher`] implementation from @@ -264,13 +266,9 @@ impl ExecutionPlan for BoundedWindowAggExec { } } + /// Get the [`EquivalenceProperties`] within the plan fn equivalence_properties(&self) -> EquivalenceProperties { - self.input().equivalence_properties() - } - - /// Get the OrderingEquivalenceProperties within the plan - fn ordering_equivalence_properties(&self) -> OrderingEquivalenceProperties { - window_ordering_equivalence(&self.schema, &self.input, &self.window_expr) + window_equivalence_properties(&self.schema, &self.input, &self.window_expr) } fn maintains_input_order(&self) -> Vec { diff --git a/datafusion/physical-plan/src/windows/mod.rs b/datafusion/physical-plan/src/windows/mod.rs index aff936499a5e..26dddc4ddde4 100644 --- a/datafusion/physical-plan/src/windows/mod.rs +++ b/datafusion/physical-plan/src/windows/mod.rs @@ -32,24 +32,18 @@ use crate::{ use arrow::datatypes::Schema; use arrow_schema::{DataType, Field, SchemaRef}; -use datafusion_common::utils::{ - find_indices, get_at_indices, is_sorted, longest_consecutive_prefix, - merge_and_order_indices, set_difference, -}; -use datafusion_common::{DataFusionError, Result, ScalarValue}; +use datafusion_common::{exec_err, DataFusionError, Result, ScalarValue}; use datafusion_expr::{ window_function::{BuiltInWindowFunction, WindowFunction}, PartitionEvaluator, WindowFrame, WindowUDF, }; +use datafusion_physical_expr::equivalence::collapse_lex_req; use datafusion_physical_expr::{ - equivalence::OrderingEquivalenceBuilder, - utils::{convert_to_expr, get_indices_of_matching_exprs}, + reverse_order_bys, window::{BuiltInWindowFunctionExpr, SlidingAggregateWindowExpr}, - AggregateExpr, OrderingEquivalenceProperties, PhysicalSortRequirement, + AggregateExpr, EquivalenceProperties, LexOrdering, PhysicalSortRequirement, }; -use itertools::{izip, Itertools}; - mod bounded_window_agg_exec; mod window_agg_exec; @@ -61,13 +55,19 @@ pub use datafusion_physical_expr::window::{ }; #[derive(Debug, Clone, PartialEq)] -/// Specifies partition column properties in terms of input ordering +/// Specifies partition expression properties in terms of existing ordering(s). +/// As an example if existing ordering is [a ASC, b ASC, c ASC], +/// `PARTITION BY b` will have `PartitionSearchMode::Linear`. +/// `PARTITION BY a, c` and `PARTITION BY c, a` will have `PartitionSearchMode::PartiallySorted(0)`, `PartitionSearchMode::PartiallySorted(1)` +/// respectively (subset `a` defines an ordered section. Indices points to index of `a` among partition by expressions). +/// `PARTITION BY a, b` and `PARTITION BY b, a` will have `PartitionSearchMode::Sorted` mode. pub enum PartitionSearchMode { - /// None of the columns among the partition columns is ordered. + /// None of the partition expressions is ordered. Linear, - /// Some columns of the partition columns are ordered but not all + /// A non-empty subset of the the partition expressions are ordered. + /// Indices stored constructs ordered subset, that is satisfied by existing ordering(s). PartiallySorted(Vec), - /// All Partition columns are ordered (Also empty case) + /// All Partition expressions are ordered (Also empty case) Sorted, } @@ -331,45 +331,51 @@ pub(crate) fn get_ordered_partition_by_indices( partition_by_exprs: &[Arc], input: &Arc, ) -> Vec { - let input_ordering = input.output_ordering().unwrap_or(&[]); - let input_ordering_exprs = convert_to_expr(input_ordering); - let equal_properties = || input.equivalence_properties(); - let input_places = get_indices_of_matching_exprs( - &input_ordering_exprs, - partition_by_exprs, - equal_properties, - ); - let mut partition_places = get_indices_of_matching_exprs( - partition_by_exprs, - &input_ordering_exprs, - equal_properties, - ); - partition_places.sort(); - let first_n = longest_consecutive_prefix(partition_places); - input_places[0..first_n].to_vec() + let (_, indices) = input + .equivalence_properties() + .find_longest_permutation(partition_by_exprs); + indices } -pub(crate) fn window_ordering_equivalence( +pub(crate) fn get_partition_by_sort_exprs( + input: &Arc, + partition_by_exprs: &[Arc], + ordered_partition_by_indices: &[usize], +) -> Result { + let ordered_partition_exprs = ordered_partition_by_indices + .iter() + .map(|idx| partition_by_exprs[*idx].clone()) + .collect::>(); + // Make sure ordered section doesn't move over the partition by expression + assert!(ordered_partition_by_indices.len() <= partition_by_exprs.len()); + let (ordering, _) = input + .equivalence_properties() + .find_longest_permutation(&ordered_partition_exprs); + if ordering.len() == ordered_partition_exprs.len() { + Ok(ordering) + } else { + exec_err!("Expects PARTITION BY expression to be ordered") + } +} + +pub(crate) fn window_equivalence_properties( schema: &SchemaRef, input: &Arc, window_expr: &[Arc], -) -> OrderingEquivalenceProperties { +) -> EquivalenceProperties { // We need to update the schema, so we can not directly use - // `input.ordering_equivalence_properties()`. - let mut builder = OrderingEquivalenceBuilder::new(schema.clone()) - .with_equivalences(input.equivalence_properties()) - .with_existing_ordering(input.output_ordering().map(|elem| elem.to_vec())) - .extend(input.ordering_equivalence_properties()); + // `input.equivalence_properties()`. + let mut window_eq_properties = + EquivalenceProperties::new(schema.clone()).extend(input.equivalence_properties()); for expr in window_expr { if let Some(builtin_window_expr) = expr.as_any().downcast_ref::() { - builtin_window_expr - .add_equal_orderings(&mut builder, || input.equivalence_properties()); + builtin_window_expr.add_equal_orderings(&mut window_eq_properties); } } - builder.build() + window_eq_properties } /// Constructs the best-fitting windowing operator (a `WindowAggExec` or a @@ -396,7 +402,7 @@ pub fn get_best_fitting_window( let orderby_keys = window_exprs[0].order_by(); let (should_reverse, partition_search_mode) = if let Some((should_reverse, partition_search_mode)) = - can_skip_sort(partitionby_exprs, orderby_keys, input)? + get_window_mode(partitionby_exprs, orderby_keys, input)? { (should_reverse, partition_search_mode) } else { @@ -459,149 +465,41 @@ pub fn get_best_fitting_window( /// The `bool` field in the return value represents whether we should reverse window /// operator to remove `SortExec` before it. The `PartitionSearchMode` field represents /// the mode this window operator should work in to accomodate the existing ordering. -fn can_skip_sort( +pub fn get_window_mode( partitionby_exprs: &[Arc], orderby_keys: &[PhysicalSortExpr], input: &Arc, ) -> Result> { - let physical_ordering = if let Some(physical_ordering) = input.output_ordering() { - physical_ordering - } else { - // If there is no physical ordering, there is no way to remove a - // sort, so immediately return. - return Ok(None); - }; - let orderby_exprs = convert_to_expr(orderby_keys); - let physical_ordering_exprs = convert_to_expr(physical_ordering); - let equal_properties = || input.equivalence_properties(); - // Get the indices of the ORDER BY expressions among input ordering expressions: - let ob_indices = get_indices_of_matching_exprs( - &orderby_exprs, - &physical_ordering_exprs, - equal_properties, - ); - if ob_indices.len() != orderby_exprs.len() { - // If all order by expressions are not in the input ordering, - // there is no way to remove a sort -- immediately return: - return Ok(None); - } - // Get the indices of the PARTITION BY expressions among input ordering expressions: - let pb_indices = get_indices_of_matching_exprs( - partitionby_exprs, - &physical_ordering_exprs, - equal_properties, - ); - let ordered_merged_indices = merge_and_order_indices(&pb_indices, &ob_indices); - // Get the indices of the ORDER BY columns that don't appear in the - // PARTITION BY clause; i.e. calculate (ORDER BY columns) ∖ (PARTITION - // BY columns) where `∖` represents set difference. - let unique_ob_indices = set_difference(&ob_indices, &pb_indices); - if !is_sorted(&unique_ob_indices) { - // ORDER BY indices should be ascending ordered - return Ok(None); - } - let first_n = longest_consecutive_prefix(ordered_merged_indices); - let furthest_ob_index = *unique_ob_indices.last().unwrap_or(&0); - // Cannot skip sort if last order by index is not within consecutive prefix. - // For instance, if input is ordered by a, b, c, d for the expression - // `PARTITION BY a, ORDER BY b, d`, then `first_n` would be 2 (meaning a, b defines a - // prefix for input ordering). However, `furthest_ob_index` would be 3 as column d - // occurs at the 3rd index of the existing ordering. Hence, existing ordering would - // not be sufficient to run the current operator. - // However, for expression `PARTITION BY a, ORDER BY b, c, d`, `first_n` would be 4 (meaning - // a, b, c, d defines a prefix for input ordering). Similarly, `furthest_ob_index` would be - // 3 as column d occurs at the 3rd index of the existing ordering. Therefore, the existing - // ordering would be sufficient to run the current operator. - if first_n <= furthest_ob_index { - return Ok(None); - } - let input_orderby_columns = get_at_indices(physical_ordering, &unique_ob_indices)?; - let expected_orderby_columns = - get_at_indices(orderby_keys, find_indices(&ob_indices, &unique_ob_indices)?)?; - let should_reverse = if let Some(should_reverse) = check_alignments( - &input.schema(), - &input_orderby_columns, - &expected_orderby_columns, - )? { - should_reverse - } else { - // If ordering directions are not aligned, we cannot calculate the - // result without changing existing ordering. - return Ok(None); - }; - - let ordered_pb_indices = pb_indices.iter().copied().sorted().collect::>(); - // Determine how many elements in the PARTITION BY columns defines a consecutive range from zero. - let first_n = longest_consecutive_prefix(&ordered_pb_indices); - let mode = if first_n == partitionby_exprs.len() { - // All of the PARTITION BY columns defines a consecutive range from zero. - PartitionSearchMode::Sorted - } else if first_n > 0 { - // All of the PARTITION BY columns defines a consecutive range from zero. - let ordered_range = &ordered_pb_indices[0..first_n]; - let input_pb_exprs = get_at_indices(&physical_ordering_exprs, ordered_range)?; - let partially_ordered_indices = get_indices_of_matching_exprs( - &input_pb_exprs, - partitionby_exprs, - equal_properties, - ); - PartitionSearchMode::PartiallySorted(partially_ordered_indices) - } else { - // None of the PARTITION BY columns defines a consecutive range from zero. - PartitionSearchMode::Linear - }; - - Ok(Some((should_reverse, mode))) -} - -/// Compares all the orderings in `physical_ordering` and `required`, decides -/// whether alignments match. A `None` return value indicates that current -/// column is not aligned. A `Some(bool)` value indicates otherwise, and signals -/// whether we should reverse the window expression in order to avoid sorting. -fn check_alignments( - schema: &SchemaRef, - physical_ordering: &[PhysicalSortExpr], - required: &[PhysicalSortExpr], -) -> Result> { - let result = izip!(physical_ordering, required) - .map(|(lhs, rhs)| check_alignment(schema, lhs, rhs)) - .collect::>>>()?; - Ok(if let Some(res) = result { - if !res.is_empty() { - let first = res[0]; - let all_same = res.into_iter().all(|elem| elem == first); - all_same.then_some(first) - } else { - Some(false) - } - } else { - // Cannot skip some of the requirements in the input. - None - }) -} - -/// Compares `physical_ordering` and `required` ordering, decides whether -/// alignments match. A `None` return value indicates that current column is -/// not aligned. A `Some(bool)` value indicates otherwise, and signals whether -/// we should reverse the window expression in order to avoid sorting. -fn check_alignment( - input_schema: &SchemaRef, - physical_ordering: &PhysicalSortExpr, - required: &PhysicalSortExpr, -) -> Result> { - Ok(if required.expr.eq(&physical_ordering.expr) { - let physical_opts = physical_ordering.options; - let required_opts = required.options; - if required.expr.nullable(input_schema)? { - let reverse = physical_opts == !required_opts; - (reverse || physical_opts == required_opts).then_some(reverse) - } else { - // If the column is not nullable, NULLS FIRST/LAST is not important. - Some(physical_opts.descending != required_opts.descending) + let input_eqs = input.equivalence_properties(); + let mut partition_by_reqs: Vec = vec![]; + let (_, indices) = input_eqs.find_longest_permutation(partitionby_exprs); + partition_by_reqs.extend(indices.iter().map(|&idx| PhysicalSortRequirement { + expr: partitionby_exprs[idx].clone(), + options: None, + })); + // Treat partition by exprs as constant. During analysis of requirements are satisfied. + let partition_by_eqs = input_eqs.add_constants(partitionby_exprs.iter().cloned()); + let order_by_reqs = PhysicalSortRequirement::from_sort_exprs(orderby_keys); + let reverse_order_by_reqs = + PhysicalSortRequirement::from_sort_exprs(&reverse_order_bys(orderby_keys)); + for (should_swap, order_by_reqs) in + [(false, order_by_reqs), (true, reverse_order_by_reqs)] + { + let req = [partition_by_reqs.clone(), order_by_reqs].concat(); + let req = collapse_lex_req(req); + if partition_by_eqs.ordering_satisfy_requirement(&req) { + // Window can be run with existing ordering + let mode = if indices.len() == partitionby_exprs.len() { + PartitionSearchMode::Sorted + } else if indices.is_empty() { + PartitionSearchMode::Linear + } else { + PartitionSearchMode::PartiallySorted(indices) + }; + return Ok(Some((should_swap, mode))); } - } else { - None - }) + } + Ok(None) } #[cfg(test)] @@ -613,7 +511,6 @@ mod tests { use crate::streaming::StreamingTableExec; use crate::test::assert_is_pending; use crate::test::exec::{assert_strong_count_converges_to_zero, BlockingExec}; - use crate::windows::PartitionSearchMode::{Linear, PartiallySorted, Sorted}; use arrow::compute::SortOptions; use arrow::datatypes::{DataType, Field, SchemaRef}; @@ -621,6 +518,8 @@ mod tests { use futures::FutureExt; + use PartitionSearchMode::{Linear, PartiallySorted, Sorted}; + fn create_test_schema() -> Result { let nullable_column = Field::new("nullable_col", DataType::Int32, true); let non_nullable_column = Field::new("non_nullable_col", DataType::Int32, false); @@ -781,15 +680,16 @@ mod tests { } #[tokio::test] - async fn test_is_column_aligned_nullable() -> Result<()> { + async fn test_satisfiy_nullable() -> Result<()> { let schema = create_test_schema()?; let params = vec![ - ((true, true), (false, false), Some(true)), - ((true, true), (false, true), None), - ((true, true), (true, false), None), - ((true, false), (false, true), Some(true)), - ((true, false), (false, false), None), - ((true, false), (true, true), None), + ((true, true), (false, false), false), + ((true, true), (false, true), false), + ((true, true), (true, false), false), + ((true, false), (false, true), false), + ((true, false), (false, false), false), + ((true, false), (true, true), false), + ((true, false), (true, false), true), ]; for ( (physical_desc, physical_nulls_first), @@ -811,7 +711,7 @@ mod tests { nulls_first: req_nulls_first, }, }; - let res = check_alignment(&schema, &physical_ordering, &required_ordering)?; + let res = physical_ordering.satisfy(&required_ordering.into(), &schema); assert_eq!(res, expected); } @@ -819,16 +719,17 @@ mod tests { } #[tokio::test] - async fn test_is_column_aligned_non_nullable() -> Result<()> { + async fn test_satisfy_non_nullable() -> Result<()> { let schema = create_test_schema()?; let params = vec![ - ((true, true), (false, false), Some(true)), - ((true, true), (false, true), Some(true)), - ((true, true), (true, false), Some(false)), - ((true, false), (false, true), Some(true)), - ((true, false), (false, false), Some(true)), - ((true, false), (true, true), Some(false)), + ((true, true), (false, false), false), + ((true, true), (false, true), false), + ((true, true), (true, false), true), + ((true, false), (false, true), false), + ((true, false), (false, false), false), + ((true, false), (true, true), true), + ((true, false), (true, false), true), ]; for ( (physical_desc, physical_nulls_first), @@ -850,7 +751,7 @@ mod tests { nulls_first: req_nulls_first, }, }; - let res = check_alignment(&schema, &physical_ordering, &required_ordering)?; + let res = physical_ordering.satisfy(&required_ordering.into(), &schema); assert_eq!(res, expected); } @@ -858,7 +759,7 @@ mod tests { } #[tokio::test] - async fn test_can_skip_ordering_exhaustive() -> Result<()> { + async fn test_get_window_mode_exhaustive() -> Result<()> { let test_schema = create_test_schema3()?; // Columns a,c are nullable whereas b,d are not nullable. // Source is sorted by a ASC NULLS FIRST, b ASC NULLS FIRST, c ASC NULLS FIRST, d ASC NULLS FIRST @@ -891,7 +792,7 @@ mod tests { (vec!["a"], vec!["a", "c"], None), (vec!["a"], vec!["a", "b", "c"], Some(Sorted)), (vec!["b"], vec!["a"], Some(Linear)), - (vec!["b"], vec!["b"], None), + (vec!["b"], vec!["b"], Some(Linear)), (vec!["b"], vec!["c"], None), (vec!["b"], vec!["a", "b"], Some(Linear)), (vec!["b"], vec!["b", "c"], None), @@ -899,7 +800,7 @@ mod tests { (vec!["b"], vec!["a", "b", "c"], Some(Linear)), (vec!["c"], vec!["a"], Some(Linear)), (vec!["c"], vec!["b"], None), - (vec!["c"], vec!["c"], None), + (vec!["c"], vec!["c"], Some(Linear)), (vec!["c"], vec!["a", "b"], Some(Linear)), (vec!["c"], vec!["b", "c"], None), (vec!["c"], vec!["a", "c"], Some(Linear)), @@ -912,10 +813,10 @@ mod tests { (vec!["b", "a"], vec!["a", "c"], Some(Sorted)), (vec!["b", "a"], vec!["a", "b", "c"], Some(Sorted)), (vec!["c", "b"], vec!["a"], Some(Linear)), - (vec!["c", "b"], vec!["b"], None), - (vec!["c", "b"], vec!["c"], None), + (vec!["c", "b"], vec!["b"], Some(Linear)), + (vec!["c", "b"], vec!["c"], Some(Linear)), (vec!["c", "b"], vec!["a", "b"], Some(Linear)), - (vec!["c", "b"], vec!["b", "c"], None), + (vec!["c", "b"], vec!["b", "c"], Some(Linear)), (vec!["c", "b"], vec!["a", "c"], Some(Linear)), (vec!["c", "b"], vec!["a", "b", "c"], Some(Linear)), (vec!["c", "a"], vec!["a"], Some(PartiallySorted(vec![1]))), @@ -965,7 +866,7 @@ mod tests { order_by_exprs.push(PhysicalSortExpr { expr, options }); } let res = - can_skip_sort(&partition_by_exprs, &order_by_exprs, &exec_unbounded)?; + get_window_mode(&partition_by_exprs, &order_by_exprs, &exec_unbounded)?; // Since reversibility is not important in this test. Convert Option<(bool, PartitionSearchMode)> to Option let res = res.map(|(_, mode)| mode); assert_eq!( @@ -978,7 +879,7 @@ mod tests { } #[tokio::test] - async fn test_can_skip_ordering() -> Result<()> { + async fn test_get_window_mode() -> Result<()> { let test_schema = create_test_schema3()?; // Columns a,c are nullable whereas b,d are not nullable. // Source is sorted by a ASC NULLS FIRST, b ASC NULLS FIRST, c ASC NULLS FIRST, d ASC NULLS FIRST @@ -1129,7 +1030,7 @@ mod tests { } assert_eq!( - can_skip_sort(&partition_by_exprs, &order_by_exprs, &exec_unbounded)?, + get_window_mode(&partition_by_exprs, &order_by_exprs, &exec_unbounded)?, *expected, "Unexpected result for in unbounded test case#: {case_idx:?}, case: {test_case:?}" ); diff --git a/datafusion/physical-plan/src/windows/window_agg_exec.rs b/datafusion/physical-plan/src/windows/window_agg_exec.rs index 4873778425d3..6c245f65ba4f 100644 --- a/datafusion/physical-plan/src/windows/window_agg_exec.rs +++ b/datafusion/physical-plan/src/windows/window_agg_exec.rs @@ -26,12 +26,13 @@ use crate::common::transpose; use crate::expressions::PhysicalSortExpr; use crate::metrics::{BaselineMetrics, ExecutionPlanMetricsSet, MetricsSet}; use crate::windows::{ - calc_requirements, get_ordered_partition_by_indices, window_ordering_equivalence, + calc_requirements, get_ordered_partition_by_indices, get_partition_by_sort_exprs, + window_equivalence_properties, }; use crate::{ - ColumnStatistics, DisplayAs, DisplayFormatType, Distribution, EquivalenceProperties, - ExecutionPlan, Partitioning, PhysicalExpr, RecordBatchStream, - SendableRecordBatchStream, Statistics, WindowExpr, + ColumnStatistics, DisplayAs, DisplayFormatType, Distribution, ExecutionPlan, + Partitioning, PhysicalExpr, RecordBatchStream, SendableRecordBatchStream, Statistics, + WindowExpr, }; use arrow::compute::{concat, concat_batches}; @@ -43,10 +44,10 @@ use arrow::{ record_batch::RecordBatch, }; use datafusion_common::stats::Precision; -use datafusion_common::utils::{evaluate_partition_ranges, get_at_indices}; +use datafusion_common::utils::evaluate_partition_ranges; use datafusion_common::{internal_err, plan_err, DataFusionError, Result}; use datafusion_execution::TaskContext; -use datafusion_physical_expr::{OrderingEquivalenceProperties, PhysicalSortRequirement}; +use datafusion_physical_expr::{EquivalenceProperties, PhysicalSortRequirement}; use futures::stream::Stream; use futures::{ready, StreamExt}; @@ -107,9 +108,12 @@ impl WindowAggExec { // Hence returned `PhysicalSortExpr` corresponding to `PARTITION BY` columns can be used safely // to calculate partition separation points pub fn partition_by_sort_keys(&self) -> Result> { - // Partition by sort keys indices are stored in self.ordered_partition_by_indices. - let sort_keys = self.input.output_ordering().unwrap_or(&[]); - get_at_indices(sort_keys, &self.ordered_partition_by_indices) + let partition_by = self.window_expr()[0].partition_by(); + get_partition_by_sort_exprs( + &self.input, + partition_by, + &self.ordered_partition_by_indices, + ) } } @@ -206,13 +210,9 @@ impl ExecutionPlan for WindowAggExec { } } + /// Get the [`EquivalenceProperties`] within the plan fn equivalence_properties(&self) -> EquivalenceProperties { - self.input().equivalence_properties() - } - - /// Get the OrderingEquivalenceProperties within the plan - fn ordering_equivalence_properties(&self) -> OrderingEquivalenceProperties { - window_ordering_equivalence(&self.schema, &self.input, &self.window_expr) + window_equivalence_properties(&self.schema, &self.input, &self.window_expr) } fn with_new_children( diff --git a/datafusion/sqllogictest/test_files/groupby.slt b/datafusion/sqllogictest/test_files/groupby.slt index ef6fee69abbf..cb0b0b7c76a5 100644 --- a/datafusion/sqllogictest/test_files/groupby.slt +++ b/datafusion/sqllogictest/test_files/groupby.slt @@ -2019,7 +2019,7 @@ SortPreservingMergeExec: [col0@0 ASC NULLS LAST] ------AggregateExec: mode=FinalPartitioned, gby=[col0@0 as col0, col1@1 as col1, col2@2 as col2], aggr=[LAST_VALUE(r.col1)] --------CoalesceBatchesExec: target_batch_size=8192 ----------RepartitionExec: partitioning=Hash([col0@0, col1@1, col2@2], 4), input_partitions=4 -------------AggregateExec: mode=Partial, gby=[col0@0 as col0, col1@1 as col1, col2@2 as col2], aggr=[LAST_VALUE(r.col1)], ordering_mode=PartiallyOrdered +------------AggregateExec: mode=Partial, gby=[col0@0 as col0, col1@1 as col1, col2@2 as col2], aggr=[LAST_VALUE(r.col1)], ordering_mode=PartiallySorted([0]) --------------SortExec: expr=[col0@3 ASC NULLS LAST] ----------------CoalesceBatchesExec: target_batch_size=8192 ------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(col0@0, col0@0)] @@ -2118,7 +2118,7 @@ Projection: annotated_data_infinite2.a, annotated_data_infinite2.b, SUM(annotate ----TableScan: annotated_data_infinite2 projection=[a, b, c] physical_plan ProjectionExec: expr=[a@1 as a, b@0 as b, SUM(annotated_data_infinite2.c)@2 as summation1] ---AggregateExec: mode=Single, gby=[b@1 as b, a@0 as a], aggr=[SUM(annotated_data_infinite2.c)], ordering_mode=FullyOrdered +--AggregateExec: mode=Single, gby=[b@1 as b, a@0 as a], aggr=[SUM(annotated_data_infinite2.c)], ordering_mode=Sorted ----CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, b, c], infinite_source=true, output_ordering=[a@0 ASC NULLS LAST, b@1 ASC NULLS LAST, c@2 ASC NULLS LAST], has_header=true @@ -2149,7 +2149,7 @@ Projection: annotated_data_infinite2.a, annotated_data_infinite2.d, SUM(annotate ----TableScan: annotated_data_infinite2 projection=[a, c, d] physical_plan ProjectionExec: expr=[a@1 as a, d@0 as d, SUM(annotated_data_infinite2.c) ORDER BY [annotated_data_infinite2.a DESC NULLS FIRST]@2 as summation1] ---AggregateExec: mode=Single, gby=[d@2 as d, a@0 as a], aggr=[SUM(annotated_data_infinite2.c)], ordering_mode=PartiallyOrdered +--AggregateExec: mode=Single, gby=[d@2 as d, a@0 as a], aggr=[SUM(annotated_data_infinite2.c)], ordering_mode=PartiallySorted([1]) ----CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, c, d], infinite_source=true, output_ordering=[a@0 ASC NULLS LAST], has_header=true query III @@ -2182,7 +2182,7 @@ Projection: annotated_data_infinite2.a, annotated_data_infinite2.b, FIRST_VALUE( ----TableScan: annotated_data_infinite2 projection=[a, b, c] physical_plan ProjectionExec: expr=[a@0 as a, b@1 as b, FIRST_VALUE(annotated_data_infinite2.c) ORDER BY [annotated_data_infinite2.a DESC NULLS FIRST]@2 as first_c] ---AggregateExec: mode=Single, gby=[a@0 as a, b@1 as b], aggr=[FIRST_VALUE(annotated_data_infinite2.c)], ordering_mode=FullyOrdered +--AggregateExec: mode=Single, gby=[a@0 as a, b@1 as b], aggr=[FIRST_VALUE(annotated_data_infinite2.c)], ordering_mode=Sorted ----CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, b, c], infinite_source=true, output_ordering=[a@0 ASC NULLS LAST, b@1 ASC NULLS LAST, c@2 ASC NULLS LAST], has_header=true query III @@ -2208,7 +2208,7 @@ Projection: annotated_data_infinite2.a, annotated_data_infinite2.b, LAST_VALUE(a ----TableScan: annotated_data_infinite2 projection=[a, b, c] physical_plan ProjectionExec: expr=[a@0 as a, b@1 as b, LAST_VALUE(annotated_data_infinite2.c) ORDER BY [annotated_data_infinite2.a DESC NULLS FIRST]@2 as last_c] ---AggregateExec: mode=Single, gby=[a@0 as a, b@1 as b], aggr=[LAST_VALUE(annotated_data_infinite2.c)], ordering_mode=FullyOrdered +--AggregateExec: mode=Single, gby=[a@0 as a, b@1 as b], aggr=[LAST_VALUE(annotated_data_infinite2.c)], ordering_mode=Sorted ----CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, b, c], infinite_source=true, output_ordering=[a@0 ASC NULLS LAST, b@1 ASC NULLS LAST, c@2 ASC NULLS LAST], has_header=true query III @@ -2235,7 +2235,7 @@ Projection: annotated_data_infinite2.a, annotated_data_infinite2.b, LAST_VALUE(a ----TableScan: annotated_data_infinite2 projection=[a, b, c] physical_plan ProjectionExec: expr=[a@0 as a, b@1 as b, LAST_VALUE(annotated_data_infinite2.c)@2 as last_c] ---AggregateExec: mode=Single, gby=[a@0 as a, b@1 as b], aggr=[LAST_VALUE(annotated_data_infinite2.c)], ordering_mode=FullyOrdered +--AggregateExec: mode=Single, gby=[a@0 as a, b@1 as b], aggr=[LAST_VALUE(annotated_data_infinite2.c)], ordering_mode=Sorted ----CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, b, c], infinite_source=true, output_ordering=[a@0 ASC NULLS LAST, b@1 ASC NULLS LAST, c@2 ASC NULLS LAST], has_header=true query III @@ -2354,9 +2354,9 @@ SELECT ARRAY_AGG(s.amount ORDER BY s.amount DESC) AS amounts, GROUP BY s.country # test_ordering_sensitive_aggregation4 -# If aggregators can work with bounded memory (FullyOrdered or PartiallyOrdered mode), we should append requirement to +# If aggregators can work with bounded memory (Sorted or PartiallySorted mode), we should append requirement to # the existing ordering. This enables us to still work with bounded memory, and also satisfy aggregation requirement. -# This test checks for whether we can satisfy aggregation requirement in FullyOrdered mode. +# This test checks for whether we can satisfy aggregation requirement in Sorted mode. query TT EXPLAIN SELECT s.country, ARRAY_AGG(s.amount ORDER BY s.amount DESC) AS amounts, SUM(s.amount) AS sum1 @@ -2373,7 +2373,7 @@ Projection: s.country, ARRAY_AGG(s.amount) ORDER BY [s.amount DESC NULLS FIRST] --------TableScan: sales_global projection=[country, amount] physical_plan ProjectionExec: expr=[country@0 as country, ARRAY_AGG(s.amount) ORDER BY [s.amount DESC NULLS FIRST]@1 as amounts, SUM(s.amount)@2 as sum1] ---AggregateExec: mode=Single, gby=[country@0 as country], aggr=[ARRAY_AGG(s.amount), SUM(s.amount)], ordering_mode=FullyOrdered +--AggregateExec: mode=Single, gby=[country@0 as country], aggr=[ARRAY_AGG(s.amount), SUM(s.amount)], ordering_mode=Sorted ----SortExec: expr=[country@0 ASC NULLS LAST,amount@1 DESC] ------MemoryExec: partitions=1, partition_sizes=[1] @@ -2390,9 +2390,9 @@ GRC [80.0, 30.0] 110 TUR [100.0, 75.0] 175 # test_ordering_sensitive_aggregation5 -# If aggregators can work with bounded memory (FullyOrdered or PartiallyOrdered mode), we should be append requirement to +# If aggregators can work with bounded memory (Sorted or PartiallySorted mode), we should be append requirement to # the existing ordering. This enables us to still work with bounded memory, and also satisfy aggregation requirement. -# This test checks for whether we can satisfy aggregation requirement in PartiallyOrdered mode. +# This test checks for whether we can satisfy aggregation requirement in PartiallySorted mode. query TT EXPLAIN SELECT s.country, s.zip_code, ARRAY_AGG(s.amount ORDER BY s.amount DESC) AS amounts, SUM(s.amount) AS sum1 @@ -2409,7 +2409,7 @@ Projection: s.country, s.zip_code, ARRAY_AGG(s.amount) ORDER BY [s.amount DESC N --------TableScan: sales_global projection=[zip_code, country, amount] physical_plan ProjectionExec: expr=[country@0 as country, zip_code@1 as zip_code, ARRAY_AGG(s.amount) ORDER BY [s.amount DESC NULLS FIRST]@2 as amounts, SUM(s.amount)@3 as sum1] ---AggregateExec: mode=Single, gby=[country@1 as country, zip_code@0 as zip_code], aggr=[ARRAY_AGG(s.amount), SUM(s.amount)], ordering_mode=PartiallyOrdered +--AggregateExec: mode=Single, gby=[country@1 as country, zip_code@0 as zip_code], aggr=[ARRAY_AGG(s.amount), SUM(s.amount)], ordering_mode=PartiallySorted([0]) ----SortExec: expr=[country@1 ASC NULLS LAST,amount@2 DESC] ------MemoryExec: partitions=1, partition_sizes=[1] @@ -2426,7 +2426,7 @@ GRC 0 [80.0, 30.0] 110 TUR 1 [100.0, 75.0] 175 # test_ordering_sensitive_aggregation6 -# If aggregators can work with bounded memory (FullyOrdered or PartiallyOrdered mode), we should be append requirement to +# If aggregators can work with bounded memory (FullySorted or PartiallySorted mode), we should be append requirement to # the existing ordering. When group by expressions contain aggregation requirement, we shouldn't append redundant expression. # Hence in the final plan SortExec should be SortExec: expr=[country@0 DESC] not SortExec: expr=[country@0 ASC NULLS LAST,country@0 DESC] query TT @@ -2445,7 +2445,7 @@ Projection: s.country, ARRAY_AGG(s.amount) ORDER BY [s.country DESC NULLS FIRST] --------TableScan: sales_global projection=[country, amount] physical_plan ProjectionExec: expr=[country@0 as country, ARRAY_AGG(s.amount) ORDER BY [s.country DESC NULLS FIRST]@1 as amounts, SUM(s.amount)@2 as sum1] ---AggregateExec: mode=Single, gby=[country@0 as country], aggr=[ARRAY_AGG(s.amount), SUM(s.amount)], ordering_mode=FullyOrdered +--AggregateExec: mode=Single, gby=[country@0 as country], aggr=[ARRAY_AGG(s.amount), SUM(s.amount)], ordering_mode=Sorted ----SortExec: expr=[country@0 ASC NULLS LAST] ------MemoryExec: partitions=1, partition_sizes=[1] @@ -2480,7 +2480,7 @@ Projection: s.country, ARRAY_AGG(s.amount) ORDER BY [s.country DESC NULLS FIRST, --------TableScan: sales_global projection=[country, amount] physical_plan ProjectionExec: expr=[country@0 as country, ARRAY_AGG(s.amount) ORDER BY [s.country DESC NULLS FIRST, s.amount DESC NULLS FIRST]@1 as amounts, SUM(s.amount)@2 as sum1] ---AggregateExec: mode=Single, gby=[country@0 as country], aggr=[ARRAY_AGG(s.amount), SUM(s.amount)], ordering_mode=FullyOrdered +--AggregateExec: mode=Single, gby=[country@0 as country], aggr=[ARRAY_AGG(s.amount), SUM(s.amount)], ordering_mode=Sorted ----SortExec: expr=[country@0 ASC NULLS LAST,amount@1 DESC] ------MemoryExec: partitions=1, partition_sizes=[1] @@ -3646,6 +3646,18 @@ GROUP BY d; 0 4 0 9 +query III rowsort +SELECT d, FIRST_VALUE(c ORDER BY a DESC, c DESC) as first_a, + LAST_VALUE(c ORDER BY c DESC) as last_c +FROM multiple_ordered_table +GROUP BY d; +---- +0 95 0 +1 90 4 +2 97 1 +3 99 15 +4 98 9 + query TT EXPLAIN SELECT c FROM multiple_ordered_table @@ -3656,6 +3668,49 @@ Sort: multiple_ordered_table.c ASC NULLS LAST --TableScan: multiple_ordered_table projection=[c] physical_plan CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[c], output_ordering=[c@0 ASC NULLS LAST], has_header=true +statement ok +set datafusion.execution.target_partitions = 1; + +query TT +EXPLAIN SELECT LAST_VALUE(l.d ORDER BY l.a) AS amount_usd +FROM multiple_ordered_table AS l +INNER JOIN ( + SELECT *, ROW_NUMBER() OVER (ORDER BY r.a) as row_n FROM multiple_ordered_table AS r +) +ON l.d = r.d AND + l.a >= r.a - 10 +GROUP BY row_n +ORDER BY row_n +---- +logical_plan +Projection: amount_usd +--Sort: row_n ASC NULLS LAST +----Projection: LAST_VALUE(l.d) ORDER BY [l.a ASC NULLS LAST] AS amount_usd, row_n +------Aggregate: groupBy=[[row_n]], aggr=[[LAST_VALUE(l.d) ORDER BY [l.a ASC NULLS LAST]]] +--------Projection: l.a, l.d, row_n +----------Inner Join: l.d = r.d Filter: CAST(l.a AS Int64) >= CAST(r.a AS Int64) - Int64(10) +------------SubqueryAlias: l +--------------TableScan: multiple_ordered_table projection=[a, d] +------------Projection: r.a, r.d, ROW_NUMBER() ORDER BY [r.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW AS row_n +--------------WindowAggr: windowExpr=[[ROW_NUMBER() ORDER BY [r.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW]] +----------------SubqueryAlias: r +------------------TableScan: multiple_ordered_table projection=[a, d] +physical_plan +ProjectionExec: expr=[amount_usd@0 as amount_usd] +--ProjectionExec: expr=[LAST_VALUE(l.d) ORDER BY [l.a ASC NULLS LAST]@1 as amount_usd, row_n@0 as row_n] +----AggregateExec: mode=Single, gby=[row_n@2 as row_n], aggr=[LAST_VALUE(l.d)], ordering_mode=Sorted +------ProjectionExec: expr=[a@0 as a, d@1 as d, row_n@4 as row_n] +--------CoalesceBatchesExec: target_batch_size=2 +----------HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(d@1, d@1)], filter=CAST(a@0 AS Int64) >= CAST(a@1 AS Int64) - 10 +------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, d], output_ordering=[a@0 ASC NULLS LAST], has_header=true +------------ProjectionExec: expr=[a@0 as a, d@1 as d, ROW_NUMBER() ORDER BY [r.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@2 as row_n] +--------------BoundedWindowAggExec: wdw=[ROW_NUMBER() ORDER BY [r.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "ROW_NUMBER() ORDER BY [r.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW", data_type: UInt64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(NULL)), end_bound: CurrentRow }], mode=[Sorted] +----------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, d], output_ordering=[a@0 ASC NULLS LAST], has_header=true + +# reset partition number to 8. +statement ok +set datafusion.execution.target_partitions = 8; + # Create an external table with primary key # column c statement ok @@ -3685,11 +3740,11 @@ logical_plan Aggregate: groupBy=[[multiple_ordered_table_with_pk.c, multiple_ordered_table_with_pk.b]], aggr=[[SUM(CAST(multiple_ordered_table_with_pk.d AS Int64))]] --TableScan: multiple_ordered_table_with_pk projection=[b, c, d] physical_plan -AggregateExec: mode=FinalPartitioned, gby=[c@0 as c, b@1 as b], aggr=[SUM(multiple_ordered_table_with_pk.d)], ordering_mode=PartiallyOrdered +AggregateExec: mode=FinalPartitioned, gby=[c@0 as c, b@1 as b], aggr=[SUM(multiple_ordered_table_with_pk.d)], ordering_mode=PartiallySorted([0]) --SortExec: expr=[c@0 ASC NULLS LAST] ----CoalesceBatchesExec: target_batch_size=2 ------RepartitionExec: partitioning=Hash([c@0, b@1], 8), input_partitions=8 ---------AggregateExec: mode=Partial, gby=[c@1 as c, b@0 as b], aggr=[SUM(multiple_ordered_table_with_pk.d)], ordering_mode=PartiallyOrdered +--------AggregateExec: mode=Partial, gby=[c@1 as c, b@0 as b], aggr=[SUM(multiple_ordered_table_with_pk.d)], ordering_mode=PartiallySorted([0]) ----------RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1 ------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[b, c, d], output_ordering=[c@1 ASC NULLS LAST], has_header=true @@ -3726,11 +3781,11 @@ logical_plan Aggregate: groupBy=[[multiple_ordered_table_with_pk.c, multiple_ordered_table_with_pk.b]], aggr=[[SUM(CAST(multiple_ordered_table_with_pk.d AS Int64))]] --TableScan: multiple_ordered_table_with_pk projection=[b, c, d] physical_plan -AggregateExec: mode=FinalPartitioned, gby=[c@0 as c, b@1 as b], aggr=[SUM(multiple_ordered_table_with_pk.d)], ordering_mode=PartiallyOrdered +AggregateExec: mode=FinalPartitioned, gby=[c@0 as c, b@1 as b], aggr=[SUM(multiple_ordered_table_with_pk.d)], ordering_mode=PartiallySorted([0]) --SortExec: expr=[c@0 ASC NULLS LAST] ----CoalesceBatchesExec: target_batch_size=2 ------RepartitionExec: partitioning=Hash([c@0, b@1], 8), input_partitions=8 ---------AggregateExec: mode=Partial, gby=[c@1 as c, b@0 as b], aggr=[SUM(multiple_ordered_table_with_pk.d)], ordering_mode=PartiallyOrdered +--------AggregateExec: mode=Partial, gby=[c@1 as c, b@0 as b], aggr=[SUM(multiple_ordered_table_with_pk.d)], ordering_mode=PartiallySorted([0]) ----------RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1 ------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[b, c, d], output_ordering=[c@1 ASC NULLS LAST], has_header=true diff --git a/datafusion/sqllogictest/test_files/joins.slt b/datafusion/sqllogictest/test_files/joins.slt index cc90e6431389..c794c4da4310 100644 --- a/datafusion/sqllogictest/test_files/joins.slt +++ b/datafusion/sqllogictest/test_files/joins.slt @@ -3283,6 +3283,127 @@ CoalesceBatchesExec: target_batch_size=2 ------BoundedWindowAggExec: wdw=[ROW_NUMBER() ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING: Ok(Field { name: "ROW_NUMBER() ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING", data_type: UInt64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(NULL)), end_bound: Following(UInt64(NULL)) }], mode=[Sorted] --------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a0, a, b, c, d], output_ordering=[a@1 ASC, b@2 ASC NULLS LAST, c@3 ASC NULLS LAST], has_header=true +query TT +EXPLAIN SELECT l.a, LAST_VALUE(r.b ORDER BY r.a ASC NULLS FIRST) as last_col1 +FROM annotated_data as l +JOIN annotated_data as r +ON l.a = r.a +GROUP BY l.a, l.b, l.c +ORDER BY l.a ASC NULLS FIRST; +---- +logical_plan +Sort: l.a ASC NULLS FIRST +--Projection: l.a, LAST_VALUE(r.b) ORDER BY [r.a ASC NULLS FIRST] AS last_col1 +----Aggregate: groupBy=[[l.a, l.b, l.c]], aggr=[[LAST_VALUE(r.b) ORDER BY [r.a ASC NULLS FIRST]]] +------Inner Join: l.a = r.a +--------SubqueryAlias: l +----------TableScan: annotated_data projection=[a, b, c] +--------SubqueryAlias: r +----------TableScan: annotated_data projection=[a, b] +physical_plan +ProjectionExec: expr=[a@0 as a, LAST_VALUE(r.b) ORDER BY [r.a ASC NULLS FIRST]@3 as last_col1] +--AggregateExec: mode=Single, gby=[a@0 as a, b@1 as b, c@2 as c], aggr=[LAST_VALUE(r.b)], ordering_mode=PartiallySorted([0]) +----CoalesceBatchesExec: target_batch_size=2 +------HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(a@0, a@0)] +--------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, b, c], output_ordering=[a@0 ASC, b@1 ASC NULLS LAST, c@2 ASC NULLS LAST], has_header=true +--------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, b], output_ordering=[a@0 ASC, b@1 ASC NULLS LAST], has_header=true + +# create a table where there more than one valid ordering +# that describes table. +statement ok +CREATE EXTERNAL TABLE multiple_ordered_table ( + a0 INTEGER, + a INTEGER, + b INTEGER, + c INTEGER, + d INTEGER +) +STORED AS CSV +WITH HEADER ROW +WITH ORDER (a ASC, b ASC) +WITH ORDER (c ASC) +LOCATION '../core/tests/data/window_2.csv'; + +query TT +EXPLAIN SELECT LAST_VALUE(l.d ORDER BY l.a) AS amount_usd +FROM multiple_ordered_table AS l +INNER JOIN ( + SELECT *, ROW_NUMBER() OVER (ORDER BY r.a) as row_n FROM multiple_ordered_table AS r +) +ON l.d = r.d AND + l.a >= r.a - 10 +GROUP BY row_n +ORDER BY row_n +---- +logical_plan +Projection: amount_usd +--Sort: row_n ASC NULLS LAST +----Projection: LAST_VALUE(l.d) ORDER BY [l.a ASC NULLS LAST] AS amount_usd, row_n +------Aggregate: groupBy=[[row_n]], aggr=[[LAST_VALUE(l.d) ORDER BY [l.a ASC NULLS LAST]]] +--------Projection: l.a, l.d, row_n +----------Inner Join: l.d = r.d Filter: CAST(l.a AS Int64) >= CAST(r.a AS Int64) - Int64(10) +------------SubqueryAlias: l +--------------TableScan: multiple_ordered_table projection=[a, d] +------------Projection: r.a, r.d, ROW_NUMBER() ORDER BY [r.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW AS row_n +--------------WindowAggr: windowExpr=[[ROW_NUMBER() ORDER BY [r.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW]] +----------------SubqueryAlias: r +------------------TableScan: multiple_ordered_table projection=[a, d] +physical_plan +ProjectionExec: expr=[amount_usd@0 as amount_usd] +--ProjectionExec: expr=[LAST_VALUE(l.d) ORDER BY [l.a ASC NULLS LAST]@1 as amount_usd, row_n@0 as row_n] +----AggregateExec: mode=Single, gby=[row_n@2 as row_n], aggr=[LAST_VALUE(l.d)], ordering_mode=Sorted +------ProjectionExec: expr=[a@0 as a, d@1 as d, row_n@4 as row_n] +--------CoalesceBatchesExec: target_batch_size=2 +----------HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(d@1, d@1)], filter=CAST(a@0 AS Int64) >= CAST(a@1 AS Int64) - 10 +------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, d], output_ordering=[a@0 ASC NULLS LAST], has_header=true +------------ProjectionExec: expr=[a@0 as a, d@1 as d, ROW_NUMBER() ORDER BY [r.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@2 as row_n] +--------------BoundedWindowAggExec: wdw=[ROW_NUMBER() ORDER BY [r.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "ROW_NUMBER() ORDER BY [r.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW", data_type: UInt64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(NULL)), end_bound: CurrentRow }], mode=[Sorted] +----------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, d], output_ordering=[a@0 ASC NULLS LAST], has_header=true + +# run query above in multiple partitions +statement ok +set datafusion.execution.target_partitions = 2; + +# use bounded variants +statement ok +set datafusion.optimizer.prefer_existing_sort = true; + +query TT +EXPLAIN SELECT l.a, LAST_VALUE(r.b ORDER BY r.a ASC NULLS FIRST) as last_col1 +FROM annotated_data as l +JOIN annotated_data as r +ON l.a = r.a +GROUP BY l.a, l.b, l.c +ORDER BY l.a ASC NULLS FIRST; +---- +logical_plan +Sort: l.a ASC NULLS FIRST +--Projection: l.a, LAST_VALUE(r.b) ORDER BY [r.a ASC NULLS FIRST] AS last_col1 +----Aggregate: groupBy=[[l.a, l.b, l.c]], aggr=[[LAST_VALUE(r.b) ORDER BY [r.a ASC NULLS FIRST]]] +------Inner Join: l.a = r.a +--------SubqueryAlias: l +----------TableScan: annotated_data projection=[a, b, c] +--------SubqueryAlias: r +----------TableScan: annotated_data projection=[a, b] +physical_plan +SortPreservingMergeExec: [a@0 ASC] +--SortExec: expr=[a@0 ASC] +----ProjectionExec: expr=[a@0 as a, LAST_VALUE(r.b) ORDER BY [r.a ASC NULLS FIRST]@3 as last_col1] +------AggregateExec: mode=FinalPartitioned, gby=[a@0 as a, b@1 as b, c@2 as c], aggr=[LAST_VALUE(r.b)] +--------CoalesceBatchesExec: target_batch_size=2 +----------RepartitionExec: partitioning=Hash([a@0, b@1, c@2], 2), input_partitions=2 +------------AggregateExec: mode=Partial, gby=[a@0 as a, b@1 as b, c@2 as c], aggr=[LAST_VALUE(r.b)], ordering_mode=PartiallySorted([0]) +--------------CoalesceBatchesExec: target_batch_size=2 +----------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(a@0, a@0)] +------------------CoalesceBatchesExec: target_batch_size=2 +--------------------RepartitionExec: partitioning=Hash([a@0], 2), input_partitions=2 +----------------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, b, c], output_ordering=[a@0 ASC, b@1 ASC NULLS LAST, c@2 ASC NULLS LAST], has_header=true +------------------CoalesceBatchesExec: target_batch_size=2 +--------------------SortPreservingRepartitionExec: partitioning=Hash([a@0], 2), input_partitions=2, sort_exprs=a@0 ASC,b@1 ASC NULLS LAST +----------------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, b], output_ordering=[a@0 ASC, b@1 ASC NULLS LAST], has_header=true + #### # Config teardown #### @@ -3296,5 +3417,9 @@ set datafusion.optimizer.prefer_hash_join = true; statement ok set datafusion.execution.target_partitions = 2; +statement ok +set datafusion.optimizer.prefer_existing_sort = false; + statement ok drop table annotated_data; + diff --git a/datafusion/sqllogictest/test_files/tpch/q17.slt.part b/datafusion/sqllogictest/test_files/tpch/q17.slt.part index 50661b9b10a8..4d4aa4b1395f 100644 --- a/datafusion/sqllogictest/test_files/tpch/q17.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/q17.slt.part @@ -58,21 +58,19 @@ ProjectionExec: expr=[CAST(SUM(lineitem.l_extendedprice)@0 AS Float64) / 7 as av --------ProjectionExec: expr=[l_extendedprice@1 as l_extendedprice] ----------CoalesceBatchesExec: target_batch_size=8192 ------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(p_partkey@2, l_partkey@1)], filter=CAST(l_quantity@0 AS Decimal128(30, 15)) < Float64(0.2) * AVG(lineitem.l_quantity)@1 ---------------CoalesceBatchesExec: target_batch_size=8192 -----------------RepartitionExec: partitioning=Hash([p_partkey@2], 4), input_partitions=4 -------------------ProjectionExec: expr=[l_quantity@1 as l_quantity, l_extendedprice@2 as l_extendedprice, p_partkey@3 as p_partkey] +--------------ProjectionExec: expr=[l_quantity@1 as l_quantity, l_extendedprice@2 as l_extendedprice, p_partkey@3 as p_partkey] +----------------CoalesceBatchesExec: target_batch_size=8192 +------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(l_partkey@0, p_partkey@0)] --------------------CoalesceBatchesExec: target_batch_size=8192 -----------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(l_partkey@0, p_partkey@0)] -------------------------CoalesceBatchesExec: target_batch_size=8192 ---------------------------RepartitionExec: partitioning=Hash([l_partkey@0], 4), input_partitions=4 -----------------------------CsvExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:0..18561749], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:18561749..37123498], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:37123498..55685247], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:55685247..74246996]]}, projection=[l_partkey, l_quantity, l_extendedprice], has_header=false -------------------------CoalesceBatchesExec: target_batch_size=8192 ---------------------------RepartitionExec: partitioning=Hash([p_partkey@0], 4), input_partitions=4 -----------------------------ProjectionExec: expr=[p_partkey@0 as p_partkey] -------------------------------CoalesceBatchesExec: target_batch_size=8192 ---------------------------------FilterExec: p_brand@1 = Brand#23 AND p_container@2 = MED BOX -----------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/part.tbl]]}, projection=[p_partkey, p_brand, p_container], has_header=false +----------------------RepartitionExec: partitioning=Hash([l_partkey@0], 4), input_partitions=4 +------------------------CsvExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:0..18561749], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:18561749..37123498], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:37123498..55685247], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:55685247..74246996]]}, projection=[l_partkey, l_quantity, l_extendedprice], has_header=false +--------------------CoalesceBatchesExec: target_batch_size=8192 +----------------------RepartitionExec: partitioning=Hash([p_partkey@0], 4), input_partitions=4 +------------------------ProjectionExec: expr=[p_partkey@0 as p_partkey] +--------------------------CoalesceBatchesExec: target_batch_size=8192 +----------------------------FilterExec: p_brand@1 = Brand#23 AND p_container@2 = MED BOX +------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +--------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/part.tbl]]}, projection=[p_partkey, p_brand, p_container], has_header=false --------------ProjectionExec: expr=[CAST(0.2 * CAST(AVG(lineitem.l_quantity)@1 AS Float64) AS Decimal128(30, 15)) as Float64(0.2) * AVG(lineitem.l_quantity), l_partkey@0 as l_partkey] ----------------AggregateExec: mode=FinalPartitioned, gby=[l_partkey@0 as l_partkey], aggr=[AVG(lineitem.l_quantity)] ------------------CoalesceBatchesExec: target_batch_size=8192 diff --git a/datafusion/sqllogictest/test_files/window.slt b/datafusion/sqllogictest/test_files/window.slt index 213f6daaef3e..c7060433d999 100644 --- a/datafusion/sqllogictest/test_files/window.slt +++ b/datafusion/sqllogictest/test_files/window.slt @@ -357,7 +357,7 @@ Sort: d.b ASC NULLS LAST physical_plan SortPreservingMergeExec: [b@0 ASC NULLS LAST] --ProjectionExec: expr=[b@0 as b, MAX(d.a)@1 as max_a, MAX(d.seq)@2 as MAX(d.seq)] -----AggregateExec: mode=SinglePartitioned, gby=[b@2 as b], aggr=[MAX(d.a), MAX(d.seq)], ordering_mode=FullyOrdered +----AggregateExec: mode=SinglePartitioned, gby=[b@2 as b], aggr=[MAX(d.a), MAX(d.seq)], ordering_mode=Sorted ------ProjectionExec: expr=[ROW_NUMBER() PARTITION BY [s.b] ORDER BY [s.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@2 as seq, a@0 as a, b@1 as b] --------BoundedWindowAggExec: wdw=[ROW_NUMBER() PARTITION BY [s.b] ORDER BY [s.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "ROW_NUMBER() PARTITION BY [s.b] ORDER BY [s.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW", data_type: UInt64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int64(NULL)), end_bound: CurrentRow }], mode=[Sorted] ----------SortExec: expr=[b@1 ASC NULLS LAST,a@0 ASC NULLS LAST] @@ -1957,7 +1957,7 @@ Sort: aggregate_test_100.c1 ASC NULLS LAST ----WindowAggr: windowExpr=[[ROW_NUMBER() PARTITION BY [aggregate_test_100.c1] ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING]] ------TableScan: aggregate_test_100 projection=[c1] physical_plan -SortPreservingMergeExec: [c1@0 ASC NULLS LAST] +SortPreservingMergeExec: [c1@0 ASC NULLS LAST,rn1@1 ASC NULLS LAST] --ProjectionExec: expr=[c1@0 as c1, ROW_NUMBER() PARTITION BY [aggregate_test_100.c1] ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING@1 as rn1] ----BoundedWindowAggExec: wdw=[ROW_NUMBER() PARTITION BY [aggregate_test_100.c1] ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING: Ok(Field { name: "ROW_NUMBER() PARTITION BY [aggregate_test_100.c1] ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING", data_type: UInt64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(NULL)), end_bound: Following(UInt64(NULL)) }], mode=[Sorted] ------SortExec: expr=[c1@0 ASC NULLS LAST] @@ -2132,15 +2132,12 @@ ProjectionExec: expr=[c9@1 as c9, SUM(t1.c9) PARTITION BY [t1.c1, t1.c2] ORDER B ----BoundedWindowAggExec: wdw=[SUM(t1.c9) PARTITION BY [t1.c2, t1.c1_alias] ORDER BY [t1.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING: Ok(Field { name: "SUM(t1.c9) PARTITION BY [t1.c2, t1.c1_alias] ORDER BY [t1.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING", data_type: UInt64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(1)), end_bound: Following(UInt64(5)) }], mode=[Sorted] ------ProjectionExec: expr=[c2@0 as c2, c9@2 as c9, c1_alias@3 as c1_alias, SUM(t1.c9) PARTITION BY [t1.c1, t1.c2] ORDER BY [t1.c9 ASC NULLS LAST, t1.c8 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND UNBOUNDED FOLLOWING@4 as SUM(t1.c9) PARTITION BY [t1.c1, t1.c2] ORDER BY [t1.c9 ASC NULLS LAST, t1.c8 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND UNBOUNDED FOLLOWING, SUM(t1.c9) PARTITION BY [t1.c1, t1.c2] ORDER BY [t1.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING@5 as SUM(t1.c9) PARTITION BY [t1.c1, t1.c2] ORDER BY [t1.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING, SUM(t1.c9) PARTITION BY [t1.c2, t1.c1_alias] ORDER BY [t1.c9 ASC NULLS LAST, t1.c8 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND UNBOUNDED FOLLOWING@6 as SUM(t1.c9) PARTITION BY [t1.c2, t1.c1_alias] ORDER BY [t1.c9 ASC NULLS LAST, t1.c8 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND UNBOUNDED FOLLOWING] --------WindowAggExec: wdw=[SUM(t1.c9) PARTITION BY [t1.c2, t1.c1_alias] ORDER BY [t1.c9 ASC NULLS LAST, t1.c8 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND UNBOUNDED FOLLOWING: Ok(Field { name: "SUM(t1.c9) PARTITION BY [t1.c2, t1.c1_alias] ORDER BY [t1.c9 ASC NULLS LAST, t1.c8 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND UNBOUNDED FOLLOWING", data_type: UInt64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(1)), end_bound: Following(UInt64(NULL)) }] -----------SortExec: expr=[c2@0 ASC NULLS LAST,c1_alias@3 ASC NULLS LAST,c9@2 ASC NULLS LAST,c8@1 ASC NULLS LAST] -------------ProjectionExec: expr=[c2@1 as c2, c8@2 as c8, c9@3 as c9, c1_alias@4 as c1_alias, SUM(t1.c9) PARTITION BY [t1.c1, t1.c2] ORDER BY [t1.c9 ASC NULLS LAST, t1.c8 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND UNBOUNDED FOLLOWING@5 as SUM(t1.c9) PARTITION BY [t1.c1, t1.c2] ORDER BY [t1.c9 ASC NULLS LAST, t1.c8 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND UNBOUNDED FOLLOWING, SUM(t1.c9) PARTITION BY [t1.c1, t1.c2] ORDER BY [t1.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING@6 as SUM(t1.c9) PARTITION BY [t1.c1, t1.c2] ORDER BY [t1.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING] ---------------BoundedWindowAggExec: wdw=[SUM(t1.c9) PARTITION BY [t1.c1, t1.c2] ORDER BY [t1.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING: Ok(Field { name: "SUM(t1.c9) PARTITION BY [t1.c1, t1.c2] ORDER BY [t1.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING", data_type: UInt64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(1)), end_bound: Following(UInt64(5)) }], mode=[Sorted] -----------------WindowAggExec: wdw=[SUM(t1.c9) PARTITION BY [t1.c1, t1.c2] ORDER BY [t1.c9 ASC NULLS LAST, t1.c8 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND UNBOUNDED FOLLOWING: Ok(Field { name: "SUM(t1.c9) PARTITION BY [t1.c1, t1.c2] ORDER BY [t1.c9 ASC NULLS LAST, t1.c8 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND UNBOUNDED FOLLOWING", data_type: UInt64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(1)), end_bound: Following(UInt64(NULL)) }] -------------------SortExec: expr=[c1@0 ASC NULLS LAST,c2@1 ASC NULLS LAST,c9@3 ASC NULLS LAST,c8@2 ASC NULLS LAST] ---------------------ProjectionExec: expr=[c1@0 as c1, c2@1 as c2, c8@2 as c8, c9@3 as c9, c1@0 as c1_alias] -----------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1, c2, c8, c9], has_header=true - - +----------ProjectionExec: expr=[c2@1 as c2, c8@2 as c8, c9@3 as c9, c1_alias@4 as c1_alias, SUM(t1.c9) PARTITION BY [t1.c1, t1.c2] ORDER BY [t1.c9 ASC NULLS LAST, t1.c8 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND UNBOUNDED FOLLOWING@5 as SUM(t1.c9) PARTITION BY [t1.c1, t1.c2] ORDER BY [t1.c9 ASC NULLS LAST, t1.c8 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND UNBOUNDED FOLLOWING, SUM(t1.c9) PARTITION BY [t1.c1, t1.c2] ORDER BY [t1.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING@6 as SUM(t1.c9) PARTITION BY [t1.c1, t1.c2] ORDER BY [t1.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING] +------------BoundedWindowAggExec: wdw=[SUM(t1.c9) PARTITION BY [t1.c1, t1.c2] ORDER BY [t1.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING: Ok(Field { name: "SUM(t1.c9) PARTITION BY [t1.c1, t1.c2] ORDER BY [t1.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING", data_type: UInt64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(1)), end_bound: Following(UInt64(5)) }], mode=[Sorted] +--------------WindowAggExec: wdw=[SUM(t1.c9) PARTITION BY [t1.c1, t1.c2] ORDER BY [t1.c9 ASC NULLS LAST, t1.c8 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND UNBOUNDED FOLLOWING: Ok(Field { name: "SUM(t1.c9) PARTITION BY [t1.c1, t1.c2] ORDER BY [t1.c9 ASC NULLS LAST, t1.c8 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND UNBOUNDED FOLLOWING", data_type: UInt64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(1)), end_bound: Following(UInt64(NULL)) }] +----------------SortExec: expr=[c1@0 ASC NULLS LAST,c2@1 ASC NULLS LAST,c9@3 ASC NULLS LAST,c8@2 ASC NULLS LAST] +------------------ProjectionExec: expr=[c1@0 as c1, c2@1 as c2, c8@2 as c8, c9@3 as c9, c1@0 as c1_alias] +--------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1, c2, c8, c9], has_header=true query IIIII SELECT c9, @@ -2338,10 +2335,11 @@ Limit: skip=0, fetch=5 ----------TableScan: aggregate_test_100 projection=[c9] physical_plan GlobalLimitExec: skip=0, fetch=5 ---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] -----BoundedWindowAggExec: wdw=[ROW_NUMBER() ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "ROW_NUMBER() ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW", data_type: UInt64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(UInt64(NULL)), end_bound: CurrentRow }], mode=[Sorted] -------SortExec: expr=[c9@0 DESC] ---------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c9], has_header=true +--SortExec: TopK(fetch=5), expr=[rn1@1 ASC NULLS LAST,c9@0 ASC NULLS LAST] +----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] +------BoundedWindowAggExec: wdw=[ROW_NUMBER() ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "ROW_NUMBER() ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW", data_type: UInt64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(UInt64(NULL)), end_bound: CurrentRow }], mode=[Sorted] +--------SortExec: expr=[c9@0 DESC] +----------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c9], has_header=true query II SELECT c9, rn1 FROM (SELECT c9, @@ -3241,7 +3239,7 @@ physical_plan ProjectionExec: expr=[SUM(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.a, annotated_data_infinite2.b] ORDER BY [annotated_data_infinite2.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@2 as sum1, SUM(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.b, annotated_data_infinite2.a] ORDER BY [annotated_data_infinite2.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@4 as sum2, SUM(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.a, annotated_data_infinite2.d] ORDER BY [annotated_data_infinite2.b ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@3 as sum3, SUM(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.d] ORDER BY [annotated_data_infinite2.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@5 as sum4] --BoundedWindowAggExec: wdw=[SUM(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.d] ORDER BY [annotated_data_infinite2.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "SUM(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.d] ORDER BY [annotated_data_infinite2.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(NULL)), end_bound: CurrentRow }], mode=[Linear] ----CoalesceBatchesExec: target_batch_size=4096 -------SortPreservingRepartitionExec: partitioning=Hash([d@1], 2), input_partitions=2, sort_exprs=a@0 ASC NULLS LAST,b ASC NULLS LAST,c ASC NULLS LAST +------SortPreservingRepartitionExec: partitioning=Hash([d@1], 2), input_partitions=2, sort_exprs=a@0 ASC NULLS LAST --------ProjectionExec: expr=[a@0 as a, d@3 as d, SUM(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.a, annotated_data_infinite2.b] ORDER BY [annotated_data_infinite2.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@4 as SUM(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.a, annotated_data_infinite2.b] ORDER BY [annotated_data_infinite2.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW, SUM(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.a, annotated_data_infinite2.d] ORDER BY [annotated_data_infinite2.b ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@5 as SUM(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.a, annotated_data_infinite2.d] ORDER BY [annotated_data_infinite2.b ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW, SUM(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.b, annotated_data_infinite2.a] ORDER BY [annotated_data_infinite2.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@6 as SUM(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.b, annotated_data_infinite2.a] ORDER BY [annotated_data_infinite2.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW] ----------BoundedWindowAggExec: wdw=[SUM(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.b, annotated_data_infinite2.a] ORDER BY [annotated_data_infinite2.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "SUM(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.b, annotated_data_infinite2.a] ORDER BY [annotated_data_infinite2.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(NULL)), end_bound: CurrentRow }], mode=[Sorted] ------------CoalesceBatchesExec: target_batch_size=4096 @@ -3379,6 +3377,96 @@ SELECT ORDER BY C3 LIMIT 5 +# Create a source where there is multiple orderings. +statement ok +CREATE EXTERNAL TABLE multiple_ordered_table ( + a0 INTEGER, + a INTEGER, + b INTEGER, + c INTEGER, + d INTEGER +) +STORED AS CSV +WITH HEADER ROW +WITH ORDER (a ASC, b ASC) +WITH ORDER (c ASC) +LOCATION '../core/tests/data/window_2.csv'; + +# All of the window execs in the physical plan should work in the +# sorted mode. +query TT +EXPLAIN SELECT MIN(d) OVER(ORDER BY c ASC) as min1, + MAX(d) OVER(PARTITION BY b, a ORDER BY c ASC) as max1 +FROM multiple_ordered_table +---- +logical_plan +Projection: MIN(multiple_ordered_table.d) ORDER BY [multiple_ordered_table.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW AS min1, MAX(multiple_ordered_table.d) PARTITION BY [multiple_ordered_table.b, multiple_ordered_table.a] ORDER BY [multiple_ordered_table.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW AS max1 +--WindowAggr: windowExpr=[[MIN(multiple_ordered_table.d) ORDER BY [multiple_ordered_table.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW]] +----Projection: multiple_ordered_table.c, multiple_ordered_table.d, MAX(multiple_ordered_table.d) PARTITION BY [multiple_ordered_table.b, multiple_ordered_table.a] ORDER BY [multiple_ordered_table.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW +------WindowAggr: windowExpr=[[MAX(multiple_ordered_table.d) PARTITION BY [multiple_ordered_table.b, multiple_ordered_table.a] ORDER BY [multiple_ordered_table.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW]] +--------TableScan: multiple_ordered_table projection=[a, b, c, d] +physical_plan +ProjectionExec: expr=[MIN(multiple_ordered_table.d) ORDER BY [multiple_ordered_table.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@3 as min1, MAX(multiple_ordered_table.d) PARTITION BY [multiple_ordered_table.b, multiple_ordered_table.a] ORDER BY [multiple_ordered_table.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@2 as max1] +--BoundedWindowAggExec: wdw=[MIN(multiple_ordered_table.d) ORDER BY [multiple_ordered_table.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "MIN(multiple_ordered_table.d) ORDER BY [multiple_ordered_table.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(NULL)), end_bound: CurrentRow }], mode=[Sorted] +----ProjectionExec: expr=[c@2 as c, d@3 as d, MAX(multiple_ordered_table.d) PARTITION BY [multiple_ordered_table.b, multiple_ordered_table.a] ORDER BY [multiple_ordered_table.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@4 as MAX(multiple_ordered_table.d) PARTITION BY [multiple_ordered_table.b, multiple_ordered_table.a] ORDER BY [multiple_ordered_table.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW] +------BoundedWindowAggExec: wdw=[MAX(multiple_ordered_table.d) PARTITION BY [multiple_ordered_table.b, multiple_ordered_table.a] ORDER BY [multiple_ordered_table.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "MAX(multiple_ordered_table.d) PARTITION BY [multiple_ordered_table.b, multiple_ordered_table.a] ORDER BY [multiple_ordered_table.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(NULL)), end_bound: CurrentRow }], mode=[Sorted] +--------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, b, c, d], output_ordering=[a@0 ASC NULLS LAST, b@1 ASC NULLS LAST], has_header=true + +query TT +EXPLAIN SELECT MAX(c) OVER(PARTITION BY d ORDER BY c ASC) as max_c +FROM( + SELECT * + FROM multiple_ordered_table + WHERE d=0) +---- +logical_plan +Projection: MAX(multiple_ordered_table.c) PARTITION BY [multiple_ordered_table.d] ORDER BY [multiple_ordered_table.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW AS max_c +--WindowAggr: windowExpr=[[MAX(multiple_ordered_table.c) PARTITION BY [multiple_ordered_table.d] ORDER BY [multiple_ordered_table.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW]] +----Filter: multiple_ordered_table.d = Int32(0) +------TableScan: multiple_ordered_table projection=[c, d], partial_filters=[multiple_ordered_table.d = Int32(0)] +physical_plan +ProjectionExec: expr=[MAX(multiple_ordered_table.c) PARTITION BY [multiple_ordered_table.d] ORDER BY [multiple_ordered_table.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@2 as max_c] +--BoundedWindowAggExec: wdw=[MAX(multiple_ordered_table.c) PARTITION BY [multiple_ordered_table.d] ORDER BY [multiple_ordered_table.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "MAX(multiple_ordered_table.c) PARTITION BY [multiple_ordered_table.d] ORDER BY [multiple_ordered_table.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(NULL)), end_bound: CurrentRow }], mode=[Sorted] +----CoalesceBatchesExec: target_batch_size=4096 +------FilterExec: d@1 = 0 +--------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[c, d], output_ordering=[c@0 ASC NULLS LAST], has_header=true + +query TT +explain SELECT SUM(d) OVER(PARTITION BY c ORDER BY a ASC) +FROM multiple_ordered_table; +---- +logical_plan +Projection: SUM(multiple_ordered_table.d) PARTITION BY [multiple_ordered_table.c] ORDER BY [multiple_ordered_table.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW +--WindowAggr: windowExpr=[[SUM(CAST(multiple_ordered_table.d AS Int64)) PARTITION BY [multiple_ordered_table.c] ORDER BY [multiple_ordered_table.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW]] +----TableScan: multiple_ordered_table projection=[a, c, d] +physical_plan +ProjectionExec: expr=[SUM(multiple_ordered_table.d) PARTITION BY [multiple_ordered_table.c] ORDER BY [multiple_ordered_table.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@3 as SUM(multiple_ordered_table.d) PARTITION BY [multiple_ordered_table.c] ORDER BY [multiple_ordered_table.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW] +--BoundedWindowAggExec: wdw=[SUM(multiple_ordered_table.d) PARTITION BY [multiple_ordered_table.c] ORDER BY [multiple_ordered_table.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "SUM(multiple_ordered_table.d) PARTITION BY [multiple_ordered_table.c] ORDER BY [multiple_ordered_table.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(NULL)), end_bound: CurrentRow }], mode=[Sorted] +----CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, c, d], output_ordering=[a@0 ASC NULLS LAST], has_header=true + +query TT +explain SELECT SUM(d) OVER(PARTITION BY c, a ORDER BY b ASC) +FROM multiple_ordered_table; +---- +logical_plan +Projection: SUM(multiple_ordered_table.d) PARTITION BY [multiple_ordered_table.c, multiple_ordered_table.a] ORDER BY [multiple_ordered_table.b ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW +--WindowAggr: windowExpr=[[SUM(CAST(multiple_ordered_table.d AS Int64)) PARTITION BY [multiple_ordered_table.c, multiple_ordered_table.a] ORDER BY [multiple_ordered_table.b ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW]] +----TableScan: multiple_ordered_table projection=[a, b, c, d] +physical_plan +ProjectionExec: expr=[SUM(multiple_ordered_table.d) PARTITION BY [multiple_ordered_table.c, multiple_ordered_table.a] ORDER BY [multiple_ordered_table.b ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@4 as SUM(multiple_ordered_table.d) PARTITION BY [multiple_ordered_table.c, multiple_ordered_table.a] ORDER BY [multiple_ordered_table.b ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW] +--BoundedWindowAggExec: wdw=[SUM(multiple_ordered_table.d) PARTITION BY [multiple_ordered_table.c, multiple_ordered_table.a] ORDER BY [multiple_ordered_table.b ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "SUM(multiple_ordered_table.d) PARTITION BY [multiple_ordered_table.c, multiple_ordered_table.a] ORDER BY [multiple_ordered_table.b ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(NULL)), end_bound: CurrentRow }], mode=[Sorted] +----CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, b, c, d], output_ordering=[a@0 ASC NULLS LAST, b@1 ASC NULLS LAST], has_header=true + +query I +SELECT SUM(d) OVER(PARTITION BY c, a ORDER BY b ASC) +FROM multiple_ordered_table +LIMIT 5; +---- +0 +2 +0 +0 +1 # simple window query query II From 41effc4a776d9d10118b3fe73d0fc554cd60d399 Mon Sep 17 00:00:00 2001 From: Andrew Lamb Date: Sat, 4 Nov 2023 05:42:14 -0400 Subject: [PATCH 186/572] Encapsulate `ProjectionMapping` as a struct (#8033) --- datafusion/physical-expr/src/equivalence.rs | 310 +++++++++++++++++- datafusion/physical-expr/src/utils.rs | 229 ------------- .../physical-plan/src/aggregates/mod.rs | 10 +- datafusion/physical-plan/src/common.rs | 34 -- datafusion/physical-plan/src/projection.rs | 9 +- 5 files changed, 309 insertions(+), 283 deletions(-) diff --git a/datafusion/physical-expr/src/equivalence.rs b/datafusion/physical-expr/src/equivalence.rs index 04b0f2eedcdb..d8aa09b90460 100644 --- a/datafusion/physical-expr/src/equivalence.rs +++ b/datafusion/physical-expr/src/equivalence.rs @@ -42,8 +42,65 @@ use indexmap::IndexMap; pub type EquivalenceClass = Vec>; /// Stores the mapping between source expressions and target expressions for a -/// projection. Indices in the vector corresponds to the indices after projection. -pub type ProjectionMapping = Vec<(Arc, Arc)>; +/// projection. +#[derive(Debug, Clone)] +pub struct ProjectionMapping { + /// `(source expression)` --> `(target expression)` + /// Indices in the vector corresponds to the indices after projection. + inner: Vec<(Arc, Arc)>, +} + +impl ProjectionMapping { + /// Constructs the mapping between a projection's input and output + /// expressions. + /// + /// For example, given the input projection expressions (`a+b`, `c+d`) + /// and an output schema with two columns `"c+d"` and `"a+b"` + /// the projection mapping would be + /// ```text + /// [0]: (c+d, col("c+d")) + /// [1]: (a+b, col("a+b")) + /// ``` + /// where `col("c+d")` means the column named "c+d". + pub fn try_new( + expr: &[(Arc, String)], + input_schema: &SchemaRef, + ) -> Result { + // Construct a map from the input expressions to the output expression of the projection: + let mut inner = vec![]; + for (expr_idx, (expression, name)) in expr.iter().enumerate() { + let target_expr = Arc::new(Column::new(name, expr_idx)) as _; + + let source_expr = expression.clone().transform_down(&|e| match e + .as_any() + .downcast_ref::( + ) { + Some(col) => { + // Sometimes, expression and its name in the input_schema doesn't match. + // This can cause problems. Hence in here we make sure that expression name + // matches with the name in the inout_schema. + // Conceptually, source_expr and expression should be same. + let idx = col.index(); + let matching_input_field = input_schema.field(idx); + let matching_input_column = + Column::new(matching_input_field.name(), idx); + Ok(Transformed::Yes(Arc::new(matching_input_column))) + } + None => Ok(Transformed::No(e)), + })?; + + inner.push((source_expr, target_expr)); + } + Ok(Self { inner }) + } + + /// Iterate over pairs of (source, target) expressions + pub fn iter( + &self, + ) -> impl Iterator, Arc)> + '_ { + self.inner.iter() + } +} /// An `EquivalenceGroup` is a collection of `EquivalenceClass`es where each /// class represents a distinct equivalence class in a relation. @@ -329,7 +386,7 @@ impl EquivalenceGroup { // once `Arc` can be stored in `HashMap`. // See issue: https://github.com/apache/arrow-datafusion/issues/8027 let mut new_classes = vec![]; - for (source, target) in mapping { + for (source, target) in mapping.iter() { if new_classes.is_empty() { new_classes.push((source, vec![target.clone()])); } @@ -980,7 +1037,7 @@ impl EquivalenceProperties { .iter() .filter_map(|order| self.eq_group.project_ordering(projection_mapping, order)) .collect::>(); - for (source, target) in projection_mapping { + for (source, target) in projection_mapping.iter() { let expr_ordering = ExprOrdering::new(source.clone()) .transform_up(&|expr| update_ordering(expr, self)) .unwrap(); @@ -1185,6 +1242,7 @@ fn update_ordering( #[cfg(test)] mod tests { + use std::ops::Not; use std::sync::Arc; use super::*; @@ -1437,12 +1495,14 @@ mod tests { let col_a2 = &col("a2", &out_schema)?; let col_a3 = &col("a3", &out_schema)?; let col_a4 = &col("a4", &out_schema)?; - let projection_mapping = vec![ - (col_a.clone(), col_a1.clone()), - (col_a.clone(), col_a2.clone()), - (col_a.clone(), col_a3.clone()), - (col_a.clone(), col_a4.clone()), - ]; + let projection_mapping = ProjectionMapping { + inner: vec![ + (col_a.clone(), col_a1.clone()), + (col_a.clone(), col_a2.clone()), + (col_a.clone(), col_a3.clone()), + (col_a.clone(), col_a4.clone()), + ], + }; let out_properties = input_properties.project(&projection_mapping, out_schema); // At the output a1=a2=a3=a4 @@ -2565,4 +2625,234 @@ mod tests { Ok(()) } + + #[test] + fn test_get_indices_of_matching_sort_exprs_with_order_eq() -> Result<()> { + let sort_options = SortOptions::default(); + let sort_options_not = SortOptions::default().not(); + + let schema = Schema::new(vec![ + Field::new("a", DataType::Int32, true), + Field::new("b", DataType::Int32, true), + ]); + let col_a = &col("a", &schema)?; + let col_b = &col("b", &schema)?; + let required_columns = [col_b.clone(), col_a.clone()]; + let mut eq_properties = EquivalenceProperties::new(Arc::new(schema)); + eq_properties.add_new_orderings([vec![ + PhysicalSortExpr { + expr: Arc::new(Column::new("b", 1)), + options: sort_options_not, + }, + PhysicalSortExpr { + expr: Arc::new(Column::new("a", 0)), + options: sort_options, + }, + ]]); + let (result, idxs) = eq_properties.find_longest_permutation(&required_columns); + assert_eq!(idxs, vec![0, 1]); + assert_eq!( + result, + vec![ + PhysicalSortExpr { + expr: col_b.clone(), + options: sort_options_not + }, + PhysicalSortExpr { + expr: col_a.clone(), + options: sort_options + } + ] + ); + + let schema = Schema::new(vec![ + Field::new("a", DataType::Int32, true), + Field::new("b", DataType::Int32, true), + Field::new("c", DataType::Int32, true), + ]); + let col_a = &col("a", &schema)?; + let col_b = &col("b", &schema)?; + let required_columns = [col_b.clone(), col_a.clone()]; + let mut eq_properties = EquivalenceProperties::new(Arc::new(schema)); + eq_properties.add_new_orderings([ + vec![PhysicalSortExpr { + expr: Arc::new(Column::new("c", 2)), + options: sort_options, + }], + vec![ + PhysicalSortExpr { + expr: Arc::new(Column::new("b", 1)), + options: sort_options_not, + }, + PhysicalSortExpr { + expr: Arc::new(Column::new("a", 0)), + options: sort_options, + }, + ], + ]); + let (result, idxs) = eq_properties.find_longest_permutation(&required_columns); + assert_eq!(idxs, vec![0, 1]); + assert_eq!( + result, + vec![ + PhysicalSortExpr { + expr: col_b.clone(), + options: sort_options_not + }, + PhysicalSortExpr { + expr: col_a.clone(), + options: sort_options + } + ] + ); + + let required_columns = [ + Arc::new(Column::new("b", 1)) as _, + Arc::new(Column::new("a", 0)) as _, + ]; + let schema = Schema::new(vec![ + Field::new("a", DataType::Int32, true), + Field::new("b", DataType::Int32, true), + Field::new("c", DataType::Int32, true), + ]); + let mut eq_properties = EquivalenceProperties::new(Arc::new(schema)); + + // not satisfied orders + eq_properties.add_new_orderings([vec![ + PhysicalSortExpr { + expr: Arc::new(Column::new("b", 1)), + options: sort_options_not, + }, + PhysicalSortExpr { + expr: Arc::new(Column::new("c", 2)), + options: sort_options, + }, + PhysicalSortExpr { + expr: Arc::new(Column::new("a", 0)), + options: sort_options, + }, + ]]); + let (_, idxs) = eq_properties.find_longest_permutation(&required_columns); + assert_eq!(idxs, vec![0]); + + Ok(()) + } + + #[test] + fn test_normalize_ordering_equivalence_classes() -> Result<()> { + let sort_options = SortOptions::default(); + + let schema = Schema::new(vec![ + Field::new("a", DataType::Int32, true), + Field::new("b", DataType::Int32, true), + Field::new("c", DataType::Int32, true), + ]); + let col_a_expr = col("a", &schema)?; + let col_b_expr = col("b", &schema)?; + let col_c_expr = col("c", &schema)?; + let mut eq_properties = EquivalenceProperties::new(Arc::new(schema.clone())); + + eq_properties.add_equal_conditions(&col_a_expr, &col_c_expr); + let others = vec![ + vec![PhysicalSortExpr { + expr: col_b_expr.clone(), + options: sort_options, + }], + vec![PhysicalSortExpr { + expr: col_c_expr.clone(), + options: sort_options, + }], + ]; + eq_properties.add_new_orderings(others); + + let mut expected_eqs = EquivalenceProperties::new(Arc::new(schema)); + expected_eqs.add_new_orderings([ + vec![PhysicalSortExpr { + expr: col_b_expr.clone(), + options: sort_options, + }], + vec![PhysicalSortExpr { + expr: col_c_expr.clone(), + options: sort_options, + }], + ]); + + let oeq_class = eq_properties.oeq_class().clone(); + let expected = expected_eqs.oeq_class(); + assert!(oeq_class.eq(expected)); + + Ok(()) + } + + #[test] + fn project_empty_output_ordering() -> Result<()> { + let schema = Schema::new(vec![ + Field::new("a", DataType::Int32, true), + Field::new("b", DataType::Int32, true), + Field::new("c", DataType::Int32, true), + ]); + let mut eq_properties = EquivalenceProperties::new(Arc::new(schema.clone())); + let ordering = vec![PhysicalSortExpr { + expr: Arc::new(Column::new("b", 1)), + options: SortOptions::default(), + }]; + eq_properties.add_new_orderings([ordering]); + let projection_mapping = ProjectionMapping { + inner: vec![ + ( + Arc::new(Column::new("b", 1)) as _, + Arc::new(Column::new("b_new", 0)) as _, + ), + ( + Arc::new(Column::new("a", 0)) as _, + Arc::new(Column::new("a_new", 1)) as _, + ), + ], + }; + let projection_schema = Arc::new(Schema::new(vec![ + Field::new("b_new", DataType::Int32, true), + Field::new("a_new", DataType::Int32, true), + ])); + let orderings = eq_properties + .project(&projection_mapping, projection_schema) + .oeq_class() + .output_ordering() + .unwrap_or_default(); + + assert_eq!( + vec![PhysicalSortExpr { + expr: Arc::new(Column::new("b_new", 0)), + options: SortOptions::default(), + }], + orderings + ); + + let schema = Schema::new(vec![ + Field::new("a", DataType::Int32, true), + Field::new("b", DataType::Int32, true), + Field::new("c", DataType::Int32, true), + ]); + let eq_properties = EquivalenceProperties::new(Arc::new(schema)); + let projection_mapping = ProjectionMapping { + inner: vec![ + ( + Arc::new(Column::new("c", 2)) as _, + Arc::new(Column::new("c_new", 0)) as _, + ), + ( + Arc::new(Column::new("b", 1)) as _, + Arc::new(Column::new("b_new", 1)) as _, + ), + ], + }; + let projection_schema = Arc::new(Schema::new(vec![ + Field::new("c_new", DataType::Int32, true), + Field::new("b_new", DataType::Int32, true), + ])); + let projected = eq_properties.project(&projection_mapping, projection_schema); + // After projection there is no ordering. + assert!(projected.oeq_class().output_ordering().is_none()); + + Ok(()) + } } diff --git a/datafusion/physical-expr/src/utils.rs b/datafusion/physical-expr/src/utils.rs index a341f5d9bc2f..2f4ee89463a8 100644 --- a/datafusion/physical-expr/src/utils.rs +++ b/datafusion/physical-expr/src/utils.rs @@ -362,15 +362,12 @@ pub fn merge_vectors( #[cfg(test)] mod tests { use std::fmt::{Display, Formatter}; - use std::ops::Not; use std::sync::Arc; use super::*; - use crate::equivalence::EquivalenceProperties; use crate::expressions::{binary, cast, col, in_list, lit, Column, Literal}; use crate::PhysicalSortExpr; - use arrow::compute::SortOptions; use arrow_array::Int32Array; use arrow_schema::{DataType, Field, Schema}; use datafusion_common::cast::{as_boolean_array, as_int32_array}; @@ -633,230 +630,4 @@ mod tests { assert_eq!(&expected, result); Ok(()) } - - #[test] - fn test_get_indices_of_matching_sort_exprs_with_order_eq() -> Result<()> { - let sort_options = SortOptions::default(); - let sort_options_not = SortOptions::default().not(); - - let schema = Schema::new(vec![ - Field::new("a", DataType::Int32, true), - Field::new("b", DataType::Int32, true), - ]); - let col_a = &col("a", &schema)?; - let col_b = &col("b", &schema)?; - let required_columns = [col_b.clone(), col_a.clone()]; - let mut eq_properties = EquivalenceProperties::new(Arc::new(schema)); - eq_properties.add_new_orderings([vec![ - PhysicalSortExpr { - expr: Arc::new(Column::new("b", 1)), - options: sort_options_not, - }, - PhysicalSortExpr { - expr: Arc::new(Column::new("a", 0)), - options: sort_options, - }, - ]]); - let (result, idxs) = eq_properties.find_longest_permutation(&required_columns); - assert_eq!(idxs, vec![0, 1]); - assert_eq!( - result, - vec![ - PhysicalSortExpr { - expr: col_b.clone(), - options: sort_options_not - }, - PhysicalSortExpr { - expr: col_a.clone(), - options: sort_options - } - ] - ); - - let schema = Schema::new(vec![ - Field::new("a", DataType::Int32, true), - Field::new("b", DataType::Int32, true), - Field::new("c", DataType::Int32, true), - ]); - let col_a = &col("a", &schema)?; - let col_b = &col("b", &schema)?; - let required_columns = [col_b.clone(), col_a.clone()]; - let mut eq_properties = EquivalenceProperties::new(Arc::new(schema)); - eq_properties.add_new_orderings([ - vec![PhysicalSortExpr { - expr: Arc::new(Column::new("c", 2)), - options: sort_options, - }], - vec![ - PhysicalSortExpr { - expr: Arc::new(Column::new("b", 1)), - options: sort_options_not, - }, - PhysicalSortExpr { - expr: Arc::new(Column::new("a", 0)), - options: sort_options, - }, - ], - ]); - let (result, idxs) = eq_properties.find_longest_permutation(&required_columns); - assert_eq!(idxs, vec![0, 1]); - assert_eq!( - result, - vec![ - PhysicalSortExpr { - expr: col_b.clone(), - options: sort_options_not - }, - PhysicalSortExpr { - expr: col_a.clone(), - options: sort_options - } - ] - ); - - let required_columns = [ - Arc::new(Column::new("b", 1)) as _, - Arc::new(Column::new("a", 0)) as _, - ]; - let schema = Schema::new(vec![ - Field::new("a", DataType::Int32, true), - Field::new("b", DataType::Int32, true), - Field::new("c", DataType::Int32, true), - ]); - let mut eq_properties = EquivalenceProperties::new(Arc::new(schema)); - - // not satisfied orders - eq_properties.add_new_orderings([vec![ - PhysicalSortExpr { - expr: Arc::new(Column::new("b", 1)), - options: sort_options_not, - }, - PhysicalSortExpr { - expr: Arc::new(Column::new("c", 2)), - options: sort_options, - }, - PhysicalSortExpr { - expr: Arc::new(Column::new("a", 0)), - options: sort_options, - }, - ]]); - let (_, idxs) = eq_properties.find_longest_permutation(&required_columns); - assert_eq!(idxs, vec![0]); - - Ok(()) - } - - #[test] - fn test_normalize_ordering_equivalence_classes() -> Result<()> { - let sort_options = SortOptions::default(); - - let schema = Schema::new(vec![ - Field::new("a", DataType::Int32, true), - Field::new("b", DataType::Int32, true), - Field::new("c", DataType::Int32, true), - ]); - let col_a_expr = col("a", &schema)?; - let col_b_expr = col("b", &schema)?; - let col_c_expr = col("c", &schema)?; - let mut eq_properties = EquivalenceProperties::new(Arc::new(schema.clone())); - - eq_properties.add_equal_conditions(&col_a_expr, &col_c_expr); - let others = vec![ - vec![PhysicalSortExpr { - expr: col_b_expr.clone(), - options: sort_options, - }], - vec![PhysicalSortExpr { - expr: col_c_expr.clone(), - options: sort_options, - }], - ]; - eq_properties.add_new_orderings(others); - - let mut expected_eqs = EquivalenceProperties::new(Arc::new(schema)); - expected_eqs.add_new_orderings([ - vec![PhysicalSortExpr { - expr: col_b_expr.clone(), - options: sort_options, - }], - vec![PhysicalSortExpr { - expr: col_c_expr.clone(), - options: sort_options, - }], - ]); - - let oeq_class = eq_properties.oeq_class().clone(); - let expected = expected_eqs.oeq_class(); - assert!(oeq_class.eq(expected)); - - Ok(()) - } - - #[test] - fn project_empty_output_ordering() -> Result<()> { - let schema = Schema::new(vec![ - Field::new("a", DataType::Int32, true), - Field::new("b", DataType::Int32, true), - Field::new("c", DataType::Int32, true), - ]); - let mut eq_properties = EquivalenceProperties::new(Arc::new(schema.clone())); - let ordering = vec![PhysicalSortExpr { - expr: Arc::new(Column::new("b", 1)), - options: SortOptions::default(), - }]; - eq_properties.add_new_orderings([ordering]); - let projection_mapping = vec![ - ( - Arc::new(Column::new("b", 1)) as _, - Arc::new(Column::new("b_new", 0)) as _, - ), - ( - Arc::new(Column::new("a", 0)) as _, - Arc::new(Column::new("a_new", 1)) as _, - ), - ]; - let projection_schema = Arc::new(Schema::new(vec![ - Field::new("b_new", DataType::Int32, true), - Field::new("a_new", DataType::Int32, true), - ])); - let orderings = eq_properties - .project(&projection_mapping, projection_schema) - .oeq_class() - .output_ordering() - .unwrap_or_default(); - - assert_eq!( - vec![PhysicalSortExpr { - expr: Arc::new(Column::new("b_new", 0)), - options: SortOptions::default(), - }], - orderings - ); - - let schema = Schema::new(vec![ - Field::new("a", DataType::Int32, true), - Field::new("b", DataType::Int32, true), - Field::new("c", DataType::Int32, true), - ]); - let eq_properties = EquivalenceProperties::new(Arc::new(schema)); - let projection_mapping = vec![ - ( - Arc::new(Column::new("c", 2)) as _, - Arc::new(Column::new("c_new", 0)) as _, - ), - ( - Arc::new(Column::new("b", 1)) as _, - Arc::new(Column::new("b_new", 1)) as _, - ), - ]; - let projection_schema = Arc::new(Schema::new(vec![ - Field::new("c_new", DataType::Int32, true), - Field::new("b_new", DataType::Int32, true), - ])); - let projected = eq_properties.project(&projection_mapping, projection_schema); - // After projection there is no ordering. - assert!(projected.oeq_class().output_ordering().is_none()); - - Ok(()) - } } diff --git a/datafusion/physical-plan/src/aggregates/mod.rs b/datafusion/physical-plan/src/aggregates/mod.rs index fd2dc69e1f3d..8dab38bc5f0e 100644 --- a/datafusion/physical-plan/src/aggregates/mod.rs +++ b/datafusion/physical-plan/src/aggregates/mod.rs @@ -25,7 +25,7 @@ use crate::aggregates::{ no_grouping::AggregateStream, row_hash::GroupedHashAggregateStream, topk_stream::GroupedTopKAggregateStream, }; -use crate::common::calculate_projection_mapping; + use crate::metrics::{ExecutionPlanMetricsSet, MetricsSet}; use crate::windows::{ get_ordered_partition_by_indices, get_window_mode, PartitionSearchMode, @@ -60,6 +60,7 @@ mod topk; mod topk_stream; pub use datafusion_expr::AggregateFunction; +use datafusion_physical_expr::equivalence::ProjectionMapping; pub use datafusion_physical_expr::expressions::create_aggregate_expr; /// Hash aggregate modes @@ -294,9 +295,8 @@ pub struct AggregateExec { /// expressions from protobuf for final aggregate. pub input_schema: SchemaRef, /// The mapping used to normalize expressions like Partitioning and - /// PhysicalSortExpr. The key is the expression from the input schema - /// and the value is the expression from the output schema. - projection_mapping: Vec<(Arc, Arc)>, + /// PhysicalSortExpr that maps input to output + projection_mapping: ProjectionMapping, /// Execution metrics metrics: ExecutionPlanMetricsSet, required_input_ordering: Option, @@ -535,7 +535,7 @@ impl AggregateExec { // construct a map from the input expression to the output expression of the Aggregation group by let projection_mapping = - calculate_projection_mapping(&group_by.expr, &input.schema())?; + ProjectionMapping::try_new(&group_by.expr, &input.schema())?; let required_input_ordering = (!new_requirement.is_empty()).then_some(new_requirement); diff --git a/datafusion/physical-plan/src/common.rs b/datafusion/physical-plan/src/common.rs index 81a59ad7ab3c..649f3a31aa7e 100644 --- a/datafusion/physical-plan/src/common.rs +++ b/datafusion/physical-plan/src/common.rs @@ -31,10 +31,8 @@ use arrow::datatypes::Schema; use arrow::ipc::writer::{FileWriter, IpcWriteOptions}; use arrow::record_batch::RecordBatch; use datafusion_common::stats::Precision; -use datafusion_common::tree_node::{Transformed, TreeNode}; use datafusion_common::{plan_err, DataFusionError, Result}; use datafusion_execution::memory_pool::MemoryReservation; -use datafusion_physical_expr::equivalence::ProjectionMapping; use datafusion_physical_expr::expressions::{BinaryExpr, Column}; use datafusion_physical_expr::{PhysicalExpr, PhysicalSortExpr}; @@ -375,38 +373,6 @@ pub fn batch_byte_size(batch: &RecordBatch) -> usize { batch.get_array_memory_size() } -/// Constructs the mapping between a projection's input and output -pub fn calculate_projection_mapping( - expr: &[(Arc, String)], - input_schema: &Arc, -) -> Result { - // Construct a map from the input expressions to the output expression of the projection: - let mut projection_mapping = vec![]; - for (expr_idx, (expression, name)) in expr.iter().enumerate() { - let target_expr = Arc::new(Column::new(name, expr_idx)) as _; - - let source_expr = expression.clone().transform_down(&|e| match e - .as_any() - .downcast_ref::() - { - Some(col) => { - // Sometimes, expression and its name in the input_schema doesn't match. - // This can cause problems. Hence in here we make sure that expression name - // matches with the name in the inout_schema. - // Conceptually, source_expr and expression should be same. - let idx = col.index(); - let matching_input_field = input_schema.field(idx); - let matching_input_column = Column::new(matching_input_field.name(), idx); - Ok(Transformed::Yes(Arc::new(matching_input_column))) - } - None => Ok(Transformed::No(e)), - })?; - - projection_mapping.push((source_expr, target_expr)); - } - Ok(projection_mapping) -} - #[cfg(test)] mod tests { use std::ops::Not; diff --git a/datafusion/physical-plan/src/projection.rs b/datafusion/physical-plan/src/projection.rs index d4242f20a8a3..c5d94b08e0e1 100644 --- a/datafusion/physical-plan/src/projection.rs +++ b/datafusion/physical-plan/src/projection.rs @@ -29,7 +29,6 @@ use std::task::{Context, Poll}; use super::expressions::{Column, PhysicalSortExpr}; use super::metrics::{BaselineMetrics, ExecutionPlanMetricsSet, MetricsSet}; use super::{DisplayAs, RecordBatchStream, SendableRecordBatchStream, Statistics}; -use crate::common::calculate_projection_mapping; use crate::{ ColumnStatistics, DisplayFormatType, ExecutionPlan, Partitioning, PhysicalExpr, }; @@ -42,6 +41,7 @@ use datafusion_execution::TaskContext; use datafusion_physical_expr::expressions::{Literal, UnKnownColumn}; use datafusion_physical_expr::EquivalenceProperties; +use datafusion_physical_expr::equivalence::ProjectionMapping; use futures::stream::{Stream, StreamExt}; use log::trace; @@ -57,9 +57,8 @@ pub struct ProjectionExec { /// The output ordering output_ordering: Option>, /// The mapping used to normalize expressions like Partitioning and - /// PhysicalSortExpr. The key is the expression from the input schema - /// and the value is the expression from the output schema. - projection_mapping: Vec<(Arc, Arc)>, + /// PhysicalSortExpr that maps input to output + projection_mapping: ProjectionMapping, /// Execution metrics metrics: ExecutionPlanMetricsSet, } @@ -94,7 +93,7 @@ impl ProjectionExec { )); // construct a map from the input expressions to the output expression of the Projection - let projection_mapping = calculate_projection_mapping(&expr, &input_schema)?; + let projection_mapping = ProjectionMapping::try_new(&expr, &input_schema)?; let input_eqs = input.equivalence_properties(); let project_eqs = input_eqs.project(&projection_mapping, schema.clone()); From 8acdb07a59103d8a22fe64ac51bfe6bb5b07a6f5 Mon Sep 17 00:00:00 2001 From: Andrew Lamb Date: Sat, 4 Nov 2023 05:42:24 -0400 Subject: [PATCH 187/572] Minor: Fix bugs in docs for `to_timestamp`, `to_timestamp_seconds`, ... (#8040) * Minor: Fix bugs in docs for `to_timestamp`, `to_timestamp_seconds`, etc * prettier * Update docs/source/user-guide/sql/scalar_functions.md Co-authored-by: comphead * Update docs/source/user-guide/sql/scalar_functions.md Co-authored-by: comphead --------- Co-authored-by: comphead --- .../source/user-guide/sql/scalar_functions.md | 49 ++++++++++--------- 1 file changed, 27 insertions(+), 22 deletions(-) diff --git a/docs/source/user-guide/sql/scalar_functions.md b/docs/source/user-guide/sql/scalar_functions.md index b7426baea3da..be05084fb249 100644 --- a/docs/source/user-guide/sql/scalar_functions.md +++ b/docs/source/user-guide/sql/scalar_functions.md @@ -1391,10 +1391,11 @@ extract(field FROM source) ### `to_timestamp` -Converts a value to RFC3339 nanosecond timestamp format (`YYYY-MM-DDT00:00:00Z`). -Supports timestamp, integer, and unsigned integer types as input. -Integers and unsigned integers are parsed as Unix second timestamps and -return the corresponding RFC3339 timestamp. +Converts a value to a timestamp (`YYYY-MM-DDT00:00:00Z`). +Supports strings, integer, and unsigned integer types as input. +Strings are parsed as RFC3339 (e.g. '2023-07-20T05:44:00') +Integers and unsigned integers are interpreted as seconds since the unix epoch (`1970-01-01T00:00:00Z`) +return the corresponding timestamp. ``` to_timestamp(expression) @@ -1407,10 +1408,11 @@ to_timestamp(expression) ### `to_timestamp_millis` -Converts a value to RFC3339 millisecond timestamp format (`YYYY-MM-DDT00:00:00.000Z`). -Supports timestamp, integer, and unsigned integer types as input. -Integers and unsigned integers are parsed as Unix nanosecond timestamps and -return the corresponding RFC3339 timestamp. +Converts a value to a timestamp (`YYYY-MM-DDT00:00:00.000Z`). +Supports strings, integer, and unsigned integer types as input. +Strings are parsed as RFC3339 (e.g. '2023-07-20T05:44:00') +Integers and unsigned integers are interpreted as milliseconds since the unix epoch (`1970-01-01T00:00:00Z`) +return the corresponding timestamp. ``` to_timestamp_millis(expression) @@ -1423,10 +1425,11 @@ to_timestamp_millis(expression) ### `to_timestamp_micros` -Converts a value to RFC3339 microsecond timestamp format (`YYYY-MM-DDT00:00:00.000000Z`). -Supports timestamp, integer, and unsigned integer types as input. -Integers and unsigned integers are parsed as Unix nanosecond timestamps and -return the corresponding RFC3339 timestamp. +Converts a value to a timestamp (`YYYY-MM-DDT00:00:00.000000Z`). +Supports strings, integer, and unsigned integer types as input. +Strings are parsed as RFC3339 (e.g. '2023-07-20T05:44:00') +Integers and unsigned integers are interpreted as microseconds since the unix epoch (`1970-01-01T00:00:00Z`) +return the corresponding timestamp. ``` to_timestamp_nanos(expression) @@ -1434,10 +1437,11 @@ to_timestamp_nanos(expression) ### `to_timestamp_nanos` -Converts a value to RFC3339 nanosecond timestamp format (`YYYY-MM-DDT00:00:00.000000000Z`). -Supports timestamp, integer, and unsigned integer types as input. -Integers and unsigned integers are parsed as Unix nanosecond timestamps and -return the corresponding RFC3339 timestamp. +Converts a value to a timestamp (`YYYY-MM-DDT00:00:00.000000000Z`). +Supports strings, integer, and unsigned integer types as input. +Strings are parsed as RFC3339 (e.g. '2023-07-20T05:44:00') +Integers and unsigned integers are interpreted as nanoseconds since the unix epoch (`1970-01-01T00:00:00Z`) +return the corresponding timestamp. ``` to_timestamp_nanos(expression) @@ -1450,10 +1454,11 @@ to_timestamp_nanos(expression) ### `to_timestamp_seconds` -Converts a value to RFC3339 second timestamp format (`YYYY-MM-DDT00:00:00Z`). -Supports timestamp, integer, and unsigned integer types as input. -Integers and unsigned integers are parsed as Unix nanosecond timestamps and -return the corresponding RFC3339 timestamp. +Converts a value to a timestamp (`YYYY-MM-DDT00:00:00.000Z`). +Supports strings, integer, and unsigned integer types as input. +Strings are parsed as RFC3339 (e.g. '2023-07-20T05:44:00') +Integers and unsigned integers are interpreted as seconds since the unix epoch (`1970-01-01T00:00:00Z`) +return the corresponding timestamp. ``` to_timestamp_seconds(expression) @@ -1467,8 +1472,8 @@ to_timestamp_seconds(expression) ### `from_unixtime` Converts an integer to RFC3339 timestamp format (`YYYY-MM-DDT00:00:00.000000000Z`). -Input is parsed as a Unix nanosecond timestamp and returns the corresponding -RFC3339 timestamp. +Integers and unsigned integers are interpreted as nanoseconds since the unix epoch (`1970-01-01T00:00:00Z`) +return the corresponding timestamp. ``` from_unixtime(expression) From 2af326a0bdbb60e7d7420089c7aa9ff22c7319ee Mon Sep 17 00:00:00 2001 From: Mehmet Ozan Kabak Date: Sat, 4 Nov 2023 13:27:10 +0300 Subject: [PATCH 188/572] Improve comments for `PartitionSearchMode` struct (#8047) * Improve comments * Make comments partition/group agnostic --- .../src/joins/nested_loop_join.rs | 2 +- datafusion/physical-plan/src/lib.rs | 6 ++--- datafusion/physical-plan/src/windows/mod.rs | 27 ++++++++++++------- 3 files changed, 21 insertions(+), 14 deletions(-) diff --git a/datafusion/physical-plan/src/joins/nested_loop_join.rs b/datafusion/physical-plan/src/joins/nested_loop_join.rs index 5a77ed6e2907..6951642ff801 100644 --- a/datafusion/physical-plan/src/joins/nested_loop_join.rs +++ b/datafusion/physical-plan/src/joins/nested_loop_join.rs @@ -48,9 +48,9 @@ use datafusion_common::{exec_err, DataFusionError, JoinSide, Result, Statistics} use datafusion_execution::memory_pool::{MemoryConsumer, MemoryReservation}; use datafusion_execution::TaskContext; use datafusion_expr::JoinType; +use datafusion_physical_expr::equivalence::join_equivalence_properties; use datafusion_physical_expr::{EquivalenceProperties, PhysicalSortExpr}; -use datafusion_physical_expr::equivalence::join_equivalence_properties; use futures::{ready, Stream, StreamExt, TryStreamExt}; /// Data of the inner table side diff --git a/datafusion/physical-plan/src/lib.rs b/datafusion/physical-plan/src/lib.rs index 694a6d3e5c3e..9519f6a5a1dd 100644 --- a/datafusion/physical-plan/src/lib.rs +++ b/datafusion/physical-plan/src/lib.rs @@ -208,9 +208,9 @@ pub trait ExecutionPlan: Debug + DisplayAs + Send + Sync { EquivalenceProperties::new(self.schema()) } - /// Get a list of `ExecutionPlan` that provide input for this plan. The - /// returned list will be empty for leaf nodes such as scans, will contain a - /// single value for unary nodes, or two values for binary nodes (such as + /// Get a list of children `ExecutionPlan`s that act as inputs to this plan. + /// The returned list will be empty for leaf nodes such as scans, will contain + /// a single value for unary nodes, or two values for binary nodes (such as /// joins). fn children(&self) -> Vec>; diff --git a/datafusion/physical-plan/src/windows/mod.rs b/datafusion/physical-plan/src/windows/mod.rs index 26dddc4ddde4..b6ed6e482ff5 100644 --- a/datafusion/physical-plan/src/windows/mod.rs +++ b/datafusion/physical-plan/src/windows/mod.rs @@ -55,19 +55,26 @@ pub use datafusion_physical_expr::window::{ }; #[derive(Debug, Clone, PartialEq)] -/// Specifies partition expression properties in terms of existing ordering(s). -/// As an example if existing ordering is [a ASC, b ASC, c ASC], -/// `PARTITION BY b` will have `PartitionSearchMode::Linear`. -/// `PARTITION BY a, c` and `PARTITION BY c, a` will have `PartitionSearchMode::PartiallySorted(0)`, `PartitionSearchMode::PartiallySorted(1)` -/// respectively (subset `a` defines an ordered section. Indices points to index of `a` among partition by expressions). -/// `PARTITION BY a, b` and `PARTITION BY b, a` will have `PartitionSearchMode::Sorted` mode. +/// Specifies aggregation grouping and/or window partitioning properties of a +/// set of expressions in terms of the existing ordering. +/// For example, if the existing ordering is `[a ASC, b ASC, c ASC]`: +/// - A `PARTITION BY b` clause will result in `Linear` mode. +/// - A `PARTITION BY a, c` or a `PARTITION BY c, a` clause will result in +/// `PartiallySorted([0])` or `PartiallySorted([1])` modes, respectively. +/// The vector stores the index of `a` in the respective PARTITION BY expression. +/// - A `PARTITION BY a, b` or a `PARTITION BY b, a` clause will result in +/// `Sorted` mode. +/// Note that the examples above are applicable for `GROUP BY` clauses too. pub enum PartitionSearchMode { - /// None of the partition expressions is ordered. + /// There is no partial permutation of the expressions satisfying the + /// existing ordering. Linear, - /// A non-empty subset of the the partition expressions are ordered. - /// Indices stored constructs ordered subset, that is satisfied by existing ordering(s). + /// There is a partial permutation of the expressions satisfying the + /// existing ordering. Indices describing the longest partial permutation + /// are stored in the vector. PartiallySorted(Vec), - /// All Partition expressions are ordered (Also empty case) + /// There is a (full) permutation of the expressions satisfying the + /// existing ordering. Sorted, } From 3469c4e09a3d32381949dd0c0f626f406c00c6ad Mon Sep 17 00:00:00 2001 From: Jay Zhan Date: Sun, 5 Nov 2023 03:24:13 +0800 Subject: [PATCH 189/572] General approach for Array replace (#8050) * checkpoint Signed-off-by: jayzhan211 * optimize non-list Signed-off-by: jayzhan211 * replace list ver Signed-off-by: jayzhan211 * cleanup Signed-off-by: jayzhan211 * rename Signed-off-by: jayzhan211 * cleanup Signed-off-by: jayzhan211 --------- Signed-off-by: jayzhan211 --- Cargo.toml | 1 + datafusion-cli/Cargo.lock | 1 + datafusion/physical-expr/Cargo.toml | 1 + .../physical-expr/src/array_expressions.rs | 293 ++++++------------ 4 files changed, 102 insertions(+), 194 deletions(-) diff --git a/Cargo.toml b/Cargo.toml index 1a2f4a84af38..6558642d4a5e 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -53,6 +53,7 @@ arrow = { version = "48.0.0", features = ["prettyprint"] } arrow-array = { version = "48.0.0", default-features = false, features = ["chrono-tz"] } arrow-buffer = { version = "48.0.0", default-features = false } arrow-flight = { version = "48.0.0", features = ["flight-sql-experimental"] } +arrow-ord = { version = "48.0.0", default-features = false } arrow-schema = { version = "48.0.0", default-features = false } async-trait = "0.1.73" bigdecimal = "0.4.1" diff --git a/datafusion-cli/Cargo.lock b/datafusion-cli/Cargo.lock index dc828f018fd5..a5eafa68cf44 100644 --- a/datafusion-cli/Cargo.lock +++ b/datafusion-cli/Cargo.lock @@ -1241,6 +1241,7 @@ dependencies = [ "arrow", "arrow-array", "arrow-buffer", + "arrow-ord", "arrow-schema", "base64", "blake2", diff --git a/datafusion/physical-expr/Cargo.toml b/datafusion/physical-expr/Cargo.toml index 4be625e384b9..4496e7215204 100644 --- a/datafusion/physical-expr/Cargo.toml +++ b/datafusion/physical-expr/Cargo.toml @@ -44,6 +44,7 @@ ahash = { version = "0.8", default-features = false, features = ["runtime-rng"] arrow = { workspace = true } arrow-array = { workspace = true } arrow-buffer = { workspace = true } +arrow-ord = { workspace = true } arrow-schema = { workspace = true } base64 = { version = "0.21", optional = true } blake2 = { version = "^0.10.2", optional = true } diff --git a/datafusion/physical-expr/src/array_expressions.rs b/datafusion/physical-expr/src/array_expressions.rs index 687502e79fed..e296e9c96fad 100644 --- a/datafusion/physical-expr/src/array_expressions.rs +++ b/datafusion/physical-expr/src/array_expressions.rs @@ -31,7 +31,8 @@ use datafusion_common::cast::{ }; use datafusion_common::utils::array_into_list_array; use datafusion_common::{ - exec_err, internal_err, not_impl_err, plan_err, DataFusionError, Result, + exec_err, internal_datafusion_err, internal_err, not_impl_err, plan_err, + DataFusionError, Result, }; use itertools::Itertools; @@ -1221,217 +1222,121 @@ array_removement_function!( "Array_remove_all SQL function" ); -macro_rules! general_replace { - ($ARRAY:expr, $FROM:expr, $TO:expr, $MAX:expr, $ARRAY_TYPE:ident) => {{ - let mut offsets: Vec = vec![0]; - let mut values = - downcast_arg!(new_empty_array($FROM.data_type()), $ARRAY_TYPE).clone(); - - let from_array = downcast_arg!($FROM, $ARRAY_TYPE); - let to_array = downcast_arg!($TO, $ARRAY_TYPE); - for (((arr, from), to), max) in $ARRAY - .iter() - .zip(from_array.iter()) - .zip(to_array.iter()) - .zip($MAX.iter()) - { - let last_offset: i32 = offsets.last().copied().ok_or_else(|| { - DataFusionError::Internal(format!("offsets should not be empty")) - })?; - match arr { - Some(arr) => { - let child_array = downcast_arg!(arr, $ARRAY_TYPE); - let mut counter = 0; - let max = if max < Some(1) { 1 } else { max.unwrap() }; - - let replaced_array = child_array - .iter() - .map(|el| { - if counter != max && el == from { - counter += 1; - to +fn general_replace(args: &[ArrayRef], arr_n: Vec) -> Result { + let list_array = as_list_array(&args[0])?; + let from_array = &args[1]; + let to_array = &args[2]; + + let mut offsets: Vec = vec![0]; + let data_type = list_array.value_type(); + let mut values = new_empty_array(&data_type); + + for (row_index, (arr, n)) in list_array.iter().zip(arr_n.iter()).enumerate() { + let last_offset: i32 = offsets + .last() + .copied() + .ok_or_else(|| internal_datafusion_err!("offsets should not be empty"))?; + match arr { + Some(arr) => { + let indices = UInt32Array::from(vec![row_index as u32]); + let from_arr = arrow::compute::take(from_array, &indices, None)?; + + let eq_array = match from_arr.data_type() { + // arrow_ord::cmp_eq does not support ListArray, so we need to compare it by loop + DataType::List(_) => { + let from_a = as_list_array(&from_arr)?.value(0); + let list_arr = as_list_array(&arr)?; + + let mut bool_values = vec![]; + for arr in list_arr.iter() { + if let Some(a) = arr { + bool_values.push(Some(a.eq(&from_a))); } else { - el + return internal_err!( + "Null value is not supported in array_replace" + ); } - }) - .collect::<$ARRAY_TYPE>(); - - values = downcast_arg!( - compute::concat(&[&values, &replaced_array])?.clone(), - $ARRAY_TYPE - ) - .clone(); - offsets.push(last_offset + replaced_array.len() as i32); - } - None => { - offsets.push(last_offset); - } - } - } - - let field = Arc::new(Field::new("item", $FROM.data_type().clone(), true)); - - Arc::new(ListArray::try_new( - field, - OffsetBuffer::new(offsets.into()), - Arc::new(values), - None, - )?) - }}; -} - -macro_rules! general_replace_list { - ($ARRAY:expr, $FROM:expr, $TO:expr, $MAX:expr, $ARRAY_TYPE:ident) => {{ - let mut offsets: Vec = vec![0]; - let mut values = - downcast_arg!(new_empty_array($FROM.data_type()), ListArray).clone(); - - let from_array = downcast_arg!($FROM, ListArray); - let to_array = downcast_arg!($TO, ListArray); - for (((arr, from), to), max) in $ARRAY - .iter() - .zip(from_array.iter()) - .zip(to_array.iter()) - .zip($MAX.iter()) - { - let last_offset: i32 = offsets.last().copied().ok_or_else(|| { - DataFusionError::Internal(format!("offsets should not be empty")) - })?; - match arr { - Some(arr) => { - let child_array = downcast_arg!(arr, ListArray); - let mut counter = 0; - let max = if max < Some(1) { 1 } else { max.unwrap() }; + } + BooleanArray::from(bool_values) + } + _ => { + let from_arr = Scalar::new(from_arr); + arrow_ord::cmp::eq(&arr, &from_arr)? + } + }; - let replaced_vec = child_array - .iter() - .map(|el| { - if counter != max && el == from { - counter += 1; - to.clone().unwrap() - } else { - el.clone().unwrap() + // Use MutableArrayData to build the replaced array + // First array is the original array, second array is the element to replace with. + let arrays = vec![arr, to_array.clone()]; + let arrays_data = arrays + .iter() + .map(|a| a.to_data()) + .collect::>(); + let arrays_data = arrays_data.iter().collect::>(); + + let arrays = arrays + .iter() + .map(|arr| arr.as_ref()) + .collect::>(); + let capacity = Capacities::Array(arrays.iter().map(|a| a.len()).sum()); + + let mut mutable = + MutableArrayData::with_capacities(arrays_data, false, capacity); + + let mut counter = 0; + for (i, to_replace) in eq_array.iter().enumerate() { + if let Some(to_replace) = to_replace { + if to_replace { + mutable.extend(1, row_index, row_index + 1); + counter += 1; + if counter == *n { + // extend the rest of the array + mutable.extend(0, i + 1, eq_array.len()); + break; } - }) - .collect::>(); - - let mut i: i32 = 0; - let mut replaced_offsets = vec![i]; - replaced_offsets.extend( - replaced_vec - .clone() - .into_iter() - .map(|a| { - i += a.len() as i32; - i - }) - .collect::>(), - ); - - let mut replaced_values = downcast_arg!( - new_empty_array(&from_array.value_type()), - $ARRAY_TYPE - ) - .clone(); - for replaced_list in replaced_vec { - replaced_values = downcast_arg!( - compute::concat(&[&replaced_values, &replaced_list])?, - $ARRAY_TYPE - ) - .clone(); + } else { + mutable.extend(0, i, i + 1); + } + } else { + return internal_err!("eq_array should not contain None"); } + } - let field = Arc::new(Field::new( - "item", - from_array.value_type().clone(), - true, - )); - let replaced_array = ListArray::try_new( - field, - OffsetBuffer::new(replaced_offsets.clone().into()), - Arc::new(replaced_values), - None, - )?; + let data = mutable.freeze(); + let replaced_array = arrow_array::make_array(data); - values = downcast_arg!( - compute::concat(&[&values, &replaced_array,])?.clone(), - ListArray - ) - .clone(); - offsets.push(last_offset + replaced_array.len() as i32); - } - None => { - offsets.push(last_offset); - } + let v = arrow::compute::concat(&[&values, &replaced_array])?; + values = v; + offsets.push(last_offset + replaced_array.len() as i32); + } + None => { + offsets.push(last_offset); } } + } - let field = Arc::new(Field::new("item", $FROM.data_type().clone(), true)); - - Arc::new(ListArray::try_new( - field, - OffsetBuffer::new(offsets.into()), - Arc::new(values), - None, - )?) - }}; -} - -macro_rules! array_replacement_function { - ($FUNC:ident, $MAX_FUNC:expr, $DOC:expr) => { - #[doc = $DOC] - pub fn $FUNC(args: &[ArrayRef]) -> Result { - let arr = as_list_array(&args[0])?; - let from = &args[1]; - let to = &args[2]; - let max = $MAX_FUNC(args)?; - - check_datatypes(stringify!($FUNC), &[arr.values(), from, to])?; - let res = match arr.value_type() { - DataType::List(field) => { - macro_rules! array_function { - ($ARRAY_TYPE:ident) => { - general_replace_list!(arr, from, to, max, $ARRAY_TYPE) - }; - } - call_array_function!(field.data_type(), true) - } - data_type => { - macro_rules! array_function { - ($ARRAY_TYPE:ident) => { - general_replace!(arr, from, to, max, $ARRAY_TYPE) - }; - } - call_array_function!(data_type, false) - } - }; - - Ok(res) - } - }; + Ok(Arc::new(ListArray::try_new( + Arc::new(Field::new("item", data_type, true)), + OffsetBuffer::new(offsets.into()), + values, + None, + )?)) } -fn replace_one(args: &[ArrayRef]) -> Result { - Ok(Int64Array::from_value(1, args[0].len())) +pub fn array_replace(args: &[ArrayRef]) -> Result { + general_replace(args, vec![1; args[0].len()]) } -fn replace_n(args: &[ArrayRef]) -> Result { - as_int64_array(&args[3]).cloned() +pub fn array_replace_n(args: &[ArrayRef]) -> Result { + let arr = as_int64_array(&args[3])?; + let arr_n = arr.values().to_vec(); + general_replace(args, arr_n) } -fn replace_all(args: &[ArrayRef]) -> Result { - Ok(Int64Array::from_value(i64::MAX, args[0].len())) +pub fn array_replace_all(args: &[ArrayRef]) -> Result { + general_replace(args, vec![i64::MAX; args[0].len()]) } -// array replacement functions -array_replacement_function!(array_replace, replace_one, "Array_replace SQL function"); -array_replacement_function!(array_replace_n, replace_n, "Array_replace_n SQL function"); -array_replacement_function!( - array_replace_all, - replace_all, - "Array_replace_all SQL function" -); - macro_rules! to_string { ($ARG:expr, $ARRAY:expr, $DELIMITER:expr, $NULL_STRING:expr, $WITH_NULL_STRING:expr, $ARRAY_TYPE:ident) => {{ let arr = downcast_arg!($ARRAY, $ARRAY_TYPE); From e505cdd5558e7a20e89ec7caa73577472a95e1a7 Mon Sep 17 00:00:00 2001 From: Chih Wang Date: Sun, 5 Nov 2023 18:44:26 +0800 Subject: [PATCH 190/572] Minor: Remove the irrelevant note from the Expression API doc (#8053) --- docs/source/user-guide/expressions.md | 5 ----- 1 file changed, 5 deletions(-) diff --git a/docs/source/user-guide/expressions.md b/docs/source/user-guide/expressions.md index 28104dbfd445..dbe12df33564 100644 --- a/docs/source/user-guide/expressions.md +++ b/docs/source/user-guide/expressions.md @@ -107,11 +107,6 @@ but these operators always return a `bool` which makes them not work with the ex | x % y, x.rem(y) | Remainder | | -x, x.neg() | Negation | -:::{note} -In Rust, the keyword `mod` is reserved and cannot be used as an identifier. -To avoid any conflicts and ensure code completion works smoothly, we use `mod_` instead. -::: - ## Math Functions | Syntax | Description | From b54990d7aaab38163d9666ffa4304f9e3060fd2d Mon Sep 17 00:00:00 2001 From: Andrew Lamb Date: Sun, 5 Nov 2023 07:01:31 -0500 Subject: [PATCH 191/572] Minor: Add more documentation about Partitioning (#8022) * Minor: Add more documentation about Partitioning * fix typo * Apply suggestions from code review Co-authored-by: comphead * Add more diagrams, improve text * undo unintended changes * undo unintended changes * fix links * Try and clarify --------- Co-authored-by: comphead --- datafusion/physical-expr/src/partitioning.rs | 90 +++++++++++++++++++- 1 file changed, 86 insertions(+), 4 deletions(-) diff --git a/datafusion/physical-expr/src/partitioning.rs b/datafusion/physical-expr/src/partitioning.rs index 6a8fca4a1543..cbacb7a8a906 100644 --- a/datafusion/physical-expr/src/partitioning.rs +++ b/datafusion/physical-expr/src/partitioning.rs @@ -15,14 +15,95 @@ // specific language governing permissions and limitations // under the License. -//! [`Partitioning`] and [`Distribution`] for physical expressions +//! [`Partitioning`] and [`Distribution`] for `ExecutionPlans` use std::fmt; use std::sync::Arc; use crate::{physical_exprs_equal, EquivalenceProperties, PhysicalExpr}; -/// Partitioning schemes supported by operators. +/// Output partitioning supported by [`ExecutionPlan`]s. +/// +/// When `executed`, `ExecutionPlan`s produce one or more independent stream of +/// data batches in parallel, referred to as partitions. The streams are Rust +/// `aync` [`Stream`]s (a special kind of future). The number of output +/// partitions varies based on the input and the operation performed. +/// +/// For example, an `ExecutionPlan` that has output partitioning of 3 will +/// produce 3 distinct output streams as the result of calling +/// `ExecutionPlan::execute(0)`, `ExecutionPlan::execute(1)`, and +/// `ExecutionPlan::execute(2)`, as shown below: +/// +/// ```text +/// ... ... ... +/// ... ▲ ▲ ▲ +/// │ │ │ +/// ▲ │ │ │ +/// │ │ │ │ +/// │ ┌───┴────┐ ┌───┴────┐ ┌───┴────┐ +/// ┌────────────────────┐ │ Stream │ │ Stream │ │ Stream │ +/// │ ExecutionPlan │ │ (0) │ │ (1) │ │ (2) │ +/// └────────────────────┘ └────────┘ └────────┘ └────────┘ +/// ▲ ▲ ▲ ▲ +/// │ │ │ │ +/// ┌ ─ ─ ─ ─ ─ ─ ─ ─ ─ ─ │ │ │ +/// Input │ │ │ │ +/// └ ─ ─ ─ ─ ─ ─ ─ ─ ─ ─ │ │ │ +/// ▲ ┌ ─ ─ ─ ─ ┌ ─ ─ ─ ─ ┌ ─ ─ ─ ─ +/// │ Input │ Input │ Input │ +/// │ │ Stream │ Stream │ Stream +/// (0) │ (1) │ (2) │ +/// ... └ ─ ▲ ─ ─ └ ─ ▲ ─ ─ └ ─ ▲ ─ ─ +/// │ │ │ +/// │ │ │ +/// │ │ │ +/// +/// ExecutionPlan with 1 input 3 (async) streams, one for each +/// that has 3 partitions, which itself output partition +/// has 3 output partitions +/// ``` +/// +/// It is common (but not required) that an `ExecutionPlan` has the same number +/// of input partitions as output partitons. However, some plans have different +/// numbers such as the `RepartitionExec` that redistributes batches from some +/// number of inputs to some number of outputs +/// +/// ```text +/// ... ... ... ... +/// +/// ▲ ▲ ▲ +/// ▲ │ │ │ +/// │ │ │ │ +/// ┌────────┴───────────┐ │ │ │ +/// │ RepartitionExec │ ┌────┴───┐ ┌────┴───┐ ┌────┴───┐ +/// └────────────────────┘ │ Stream │ │ Stream │ │ Stream │ +/// ▲ │ (0) │ │ (1) │ │ (2) │ +/// │ └────────┘ └────────┘ └────────┘ +/// │ ▲ ▲ ▲ +/// ... │ │ │ +/// └──────────┐│┌──────────┘ +/// │││ +/// │││ +/// RepartitionExec with one input +/// that has 3 partitions, but 3 (async) streams, that internally +/// itself has only 1 output partition pull from the same input stream +/// ... +/// ``` +/// +/// # Additional Examples +/// +/// A simple `FileScanExec` might produce one output stream (partition) for each +/// file (note the actual DataFusion file scaners can read individual files in +/// parallel, potentially producing multiple partitions per file) +/// +/// Plans such as `SortPreservingMerge` produce a single output stream +/// (1 output partition) by combining some number of input streams (input partitions) +/// +/// Plans such as `FilterExec` produce the same number of output streams +/// (partitions) as input streams (partitions). +/// +/// [`ExecutionPlan`]: https://docs.rs/datafusion/latest/datafusion/physical_plan/trait.ExecutionPlan.html +/// [`Stream`]: https://docs.rs/futures/latest/futures/stream/trait.Stream.html #[derive(Debug, Clone)] pub enum Partitioning { /// Allocate batches using a round-robin algorithm and the specified number of partitions @@ -126,7 +207,8 @@ impl PartialEq for Partitioning { } } -/// Distribution schemes +/// How data is distributed amongst partitions. See [`Partitioning`] for more +/// details. #[derive(Debug, Clone)] pub enum Distribution { /// Unspecified distribution @@ -139,7 +221,7 @@ pub enum Distribution { } impl Distribution { - /// Creates a Partitioning for this Distribution to satisfy itself + /// Creates a `Partitioning` that satisfies this `Distribution` pub fn create_partitioning(&self, partition_count: usize) -> Partitioning { match self { Distribution::UnspecifiedDistribution => { From 40a3cd05ff0ff289adfd301d2401eb420fa45490 Mon Sep 17 00:00:00 2001 From: Andrew Lamb Date: Mon, 6 Nov 2023 07:16:03 -0500 Subject: [PATCH 192/572] Minor: improve documentation for IsNotNull, DISTINCT, etc (#8052) * Minor: improve documentation for IsNotNull, DISTINCT, etc * fix --- datafusion/expr/src/expr.rs | 16 ++++++++-------- datafusion/expr/src/operator.rs | 8 ++++++-- 2 files changed, 14 insertions(+), 10 deletions(-) diff --git a/datafusion/expr/src/expr.rs b/datafusion/expr/src/expr.rs index 239a3188502c..8929b21f4412 100644 --- a/datafusion/expr/src/expr.rs +++ b/datafusion/expr/src/expr.rs @@ -99,21 +99,21 @@ pub enum Expr { SimilarTo(Like), /// Negation of an expression. The expression's type must be a boolean to make sense. Not(Box), - /// Whether an expression is not Null. This expression is never null. + /// True if argument is not NULL, false otherwise. This expression itself is never NULL. IsNotNull(Box), - /// Whether an expression is Null. This expression is never null. + /// True if argument is NULL, false otherwise. This expression itself is never NULL. IsNull(Box), - /// Whether an expression is True. Boolean operation + /// True if argument is true, false otherwise. This expression itself is never NULL. IsTrue(Box), - /// Whether an expression is False. Boolean operation + /// True if argument is false, false otherwise. This expression itself is never NULL. IsFalse(Box), - /// Whether an expression is Unknown. Boolean operation + /// True if argument is NULL, false otherwise. This expression itself is never NULL. IsUnknown(Box), - /// Whether an expression is not True. Boolean operation + /// True if argument is FALSE or NULL, false otherwise. This expression itself is never NULL. IsNotTrue(Box), - /// Whether an expression is not False. Boolean operation + /// True if argument is TRUE OR NULL, false otherwise. This expression itself is never NULL. IsNotFalse(Box), - /// Whether an expression is not Unknown. Boolean operation + /// True if argument is TRUE or FALSE, false otherwise. This expression itself is never NULL. IsNotUnknown(Box), /// arithmetic negation of an expression, the operand must be of a signed numeric data type Negative(Box), diff --git a/datafusion/expr/src/operator.rs b/datafusion/expr/src/operator.rs index 1790f1478927..57888a11d426 100644 --- a/datafusion/expr/src/operator.rs +++ b/datafusion/expr/src/operator.rs @@ -53,9 +53,13 @@ pub enum Operator { And, /// Logical OR, like `||` Or, - /// IS DISTINCT FROM + /// `IS DISTINCT FROM` (see [`distinct`]) + /// + /// [`distinct`]: arrow::compute::kernels::cmp::distinct IsDistinctFrom, - /// IS NOT DISTINCT FROM + /// `IS NOT DISTINCT FROM` (see [`not_distinct`]) + /// + /// [`not_distinct`]: arrow::compute::kernels::cmp::not_distinct IsNotDistinctFrom, /// Case sensitive regex match RegexMatch, From e95e3f89c97ae27149c1dd8093f91a5574210fe6 Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Mon, 6 Nov 2023 07:44:06 -0700 Subject: [PATCH 193/572] Prepare 33.0.0 Release (#8057) * changelog * update version * update changelog --- Cargo.toml | 2 +- benchmarks/Cargo.toml | 8 +- datafusion-cli/Cargo.lock | 110 +++++++------- datafusion-cli/Cargo.toml | 4 +- datafusion/CHANGELOG.md | 1 + datafusion/core/Cargo.toml | 6 +- datafusion/optimizer/Cargo.toml | 4 +- datafusion/proto/Cargo.toml | 2 +- datafusion/sqllogictest/Cargo.toml | 2 +- dev/changelog/33.0.0.md | 228 +++++++++++++++++++++++++++++ docs/Cargo.toml | 2 +- docs/source/user-guide/configs.md | 2 +- 12 files changed, 301 insertions(+), 70 deletions(-) create mode 100644 dev/changelog/33.0.0.md diff --git a/Cargo.toml b/Cargo.toml index 6558642d4a5e..39ebd1fa59b5 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -46,7 +46,7 @@ license = "Apache-2.0" readme = "README.md" repository = "https://github.com/apache/arrow-datafusion" rust-version = "1.70" -version = "32.0.0" +version = "33.0.0" [workspace.dependencies] arrow = { version = "48.0.0", features = ["prettyprint"] } diff --git a/benchmarks/Cargo.toml b/benchmarks/Cargo.toml index ce0a4267fc28..35f94f677d86 100644 --- a/benchmarks/Cargo.toml +++ b/benchmarks/Cargo.toml @@ -18,7 +18,7 @@ [package] name = "datafusion-benchmarks" description = "DataFusion Benchmarks" -version = "32.0.0" +version = "33.0.0" edition = { workspace = true } authors = ["Apache Arrow "] homepage = "https://github.com/apache/arrow-datafusion" @@ -34,8 +34,8 @@ snmalloc = ["snmalloc-rs"] [dependencies] arrow = { workspace = true } -datafusion = { path = "../datafusion/core", version = "32.0.0" } -datafusion-common = { path = "../datafusion/common", version = "32.0.0" } +datafusion = { path = "../datafusion/core", version = "33.0.0" } +datafusion-common = { path = "../datafusion/common", version = "33.0.0" } env_logger = { workspace = true } futures = { workspace = true } log = { workspace = true } @@ -50,4 +50,4 @@ test-utils = { path = "../test-utils/", version = "0.1.0" } tokio = { version = "^1.0", features = ["macros", "rt", "rt-multi-thread", "parking_lot"] } [dev-dependencies] -datafusion-proto = { path = "../datafusion/proto", version = "32.0.0" } +datafusion-proto = { path = "../datafusion/proto", version = "33.0.0" } diff --git a/datafusion-cli/Cargo.lock b/datafusion-cli/Cargo.lock index a5eafa68cf44..74df8aab0175 100644 --- a/datafusion-cli/Cargo.lock +++ b/datafusion-cli/Cargo.lock @@ -269,7 +269,7 @@ dependencies = [ "arrow-schema", "chrono", "half", - "indexmap 2.0.2", + "indexmap 2.1.0", "lexical-core", "num", "serde", @@ -962,9 +962,9 @@ dependencies = [ [[package]] name = "const-random" -version = "0.1.16" +version = "0.1.17" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "11df32a13d7892ec42d51d3d175faba5211ffe13ed25d4fb348ac9e9ce835593" +checksum = "5aaf16c9c2c612020bcfd042e170f6e32de9b9d75adb5277cdbbd2e2c8c8299a" dependencies = [ "const-random-macro", ] @@ -1097,7 +1097,7 @@ dependencies = [ [[package]] name = "datafusion" -version = "32.0.0" +version = "33.0.0" dependencies = [ "ahash", "apache-avro", @@ -1122,7 +1122,7 @@ dependencies = [ "glob", "half", "hashbrown 0.14.2", - "indexmap 2.0.2", + "indexmap 2.1.0", "itertools", "log", "num-traits", @@ -1144,7 +1144,7 @@ dependencies = [ [[package]] name = "datafusion-cli" -version = "32.0.0" +version = "33.0.0" dependencies = [ "arrow", "assert_cmd", @@ -1169,7 +1169,7 @@ dependencies = [ [[package]] name = "datafusion-common" -version = "32.0.0" +version = "33.0.0" dependencies = [ "ahash", "apache-avro", @@ -1187,7 +1187,7 @@ dependencies = [ [[package]] name = "datafusion-execution" -version = "32.0.0" +version = "33.0.0" dependencies = [ "arrow", "chrono", @@ -1206,7 +1206,7 @@ dependencies = [ [[package]] name = "datafusion-expr" -version = "32.0.0" +version = "33.0.0" dependencies = [ "ahash", "arrow", @@ -1219,7 +1219,7 @@ dependencies = [ [[package]] name = "datafusion-optimizer" -version = "32.0.0" +version = "33.0.0" dependencies = [ "arrow", "async-trait", @@ -1235,7 +1235,7 @@ dependencies = [ [[package]] name = "datafusion-physical-expr" -version = "32.0.0" +version = "33.0.0" dependencies = [ "ahash", "arrow", @@ -1252,7 +1252,7 @@ dependencies = [ "half", "hashbrown 0.14.2", "hex", - "indexmap 2.0.2", + "indexmap 2.1.0", "itertools", "libc", "log", @@ -1268,7 +1268,7 @@ dependencies = [ [[package]] name = "datafusion-physical-plan" -version = "32.0.0" +version = "33.0.0" dependencies = [ "ahash", "arrow", @@ -1284,7 +1284,7 @@ dependencies = [ "futures", "half", "hashbrown 0.14.2", - "indexmap 2.0.2", + "indexmap 2.1.0", "itertools", "log", "once_cell", @@ -1297,7 +1297,7 @@ dependencies = [ [[package]] name = "datafusion-sql" -version = "32.0.0" +version = "33.0.0" dependencies = [ "arrow", "arrow-schema", @@ -1873,9 +1873,9 @@ dependencies = [ [[package]] name = "indexmap" -version = "2.0.2" +version = "2.1.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "8adf3ddd720272c6ea8bf59463c04e0f93d0bbf7c5439b691bca2987e0270897" +checksum = "d530e1a18b1cb4c484e6e34556a0d948706958449fca0cab753d649f2bce3d1f" dependencies = [ "equivalent", "hashbrown 0.14.2", @@ -1928,9 +1928,9 @@ dependencies = [ [[package]] name = "js-sys" -version = "0.3.64" +version = "0.3.65" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "c5f195fe497f702db0f318b07fdd68edb16955aed830df8363d837542f8f935a" +checksum = "54c0c35952f67de54bb584e9fd912b3023117cbafc0a77d8f3dee1fb5f572fe8" dependencies = [ "wasm-bindgen", ] @@ -2007,9 +2007,9 @@ dependencies = [ [[package]] name = "libc" -version = "0.2.149" +version = "0.2.150" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "a08173bc88b7955d1b3145aa561539096c421ac8debde8cbc3612ec635fee29b" +checksum = "89d92a4743f9a61002fae18374ed11e7973f530cb3a3255fb354818118b2203c" [[package]] name = "libflate" @@ -2051,6 +2051,17 @@ dependencies = [ "libc", ] +[[package]] +name = "libredox" +version = "0.0.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "85c833ca1e66078851dba29046874e38f08b2c883700aa29a03ddd3b23814ee8" +dependencies = [ + "bitflags 2.4.1", + "libc", + "redox_syscall", +] + [[package]] name = "linux-raw-sys" version = "0.4.10" @@ -2347,7 +2358,7 @@ checksum = "4c42a9226546d68acdd9c0a280d17ce19bfe27a46bf68784e4066115788d008e" dependencies = [ "cfg-if", "libc", - "redox_syscall 0.4.1", + "redox_syscall", "smallvec", "windows-targets", ] @@ -2414,7 +2425,7 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "e1d3afd2628e69da2be385eb6f2fd57c8ac7977ceeff6dc166ff1657b0e386a9" dependencies = [ "fixedbitset", - "indexmap 2.0.2", + "indexmap 2.1.0", ] [[package]] @@ -2634,15 +2645,6 @@ dependencies = [ "getrandom", ] -[[package]] -name = "redox_syscall" -version = "0.2.16" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "fb5a58c1855b4b6819d59012155603f0b22ad30cad752600aadfcb695265519a" -dependencies = [ - "bitflags 1.3.2", -] - [[package]] name = "redox_syscall" version = "0.4.1" @@ -2654,12 +2656,12 @@ dependencies = [ [[package]] name = "redox_users" -version = "0.4.3" +version = "0.4.4" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "b033d837a7cf162d7993aded9304e30a83213c648b6e389db233191f891e5c2b" +checksum = "a18479200779601e498ada4e8c1e1f50e3ee19deb0259c25825a98b5603b2cb4" dependencies = [ "getrandom", - "redox_syscall 0.2.16", + "libredox", "thiserror", ] @@ -3241,7 +3243,7 @@ checksum = "7ef1adac450ad7f4b3c28589471ade84f25f731a7a0fe30d71dfa9f60fd808e5" dependencies = [ "cfg-if", "fastrand 2.0.1", - "redox_syscall 0.4.1", + "redox_syscall", "rustix", "windows-sys", ] @@ -3653,9 +3655,9 @@ checksum = "9c8d87e72b64a3b4db28d11ce29237c246188f4f51057d65a7eab63b7987e423" [[package]] name = "wasm-bindgen" -version = "0.2.87" +version = "0.2.88" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "7706a72ab36d8cb1f80ffbf0e071533974a60d0a308d01a5d0375bf60499a342" +checksum = "7daec296f25a1bae309c0cd5c29c4b260e510e6d813c286b19eaadf409d40fce" dependencies = [ "cfg-if", "wasm-bindgen-macro", @@ -3663,9 +3665,9 @@ dependencies = [ [[package]] name = "wasm-bindgen-backend" -version = "0.2.87" +version = "0.2.88" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "5ef2b6d3c510e9625e5fe6f509ab07d66a760f0885d858736483c32ed7809abd" +checksum = "e397f4664c0e4e428e8313a469aaa58310d302159845980fd23b0f22a847f217" dependencies = [ "bumpalo", "log", @@ -3678,9 +3680,9 @@ dependencies = [ [[package]] name = "wasm-bindgen-futures" -version = "0.4.37" +version = "0.4.38" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "c02dbc21516f9f1f04f187958890d7e6026df8d16540b7ad9492bc34a67cea03" +checksum = "9afec9963e3d0994cac82455b2b3502b81a7f40f9a0d32181f7528d9f4b43e02" dependencies = [ "cfg-if", "js-sys", @@ -3690,9 +3692,9 @@ dependencies = [ [[package]] name = "wasm-bindgen-macro" -version = "0.2.87" +version = "0.2.88" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "dee495e55982a3bd48105a7b947fd2a9b4a8ae3010041b9e0faab3f9cd028f1d" +checksum = "5961017b3b08ad5f3fe39f1e79877f8ee7c23c5e5fd5eb80de95abc41f1f16b2" dependencies = [ "quote", "wasm-bindgen-macro-support", @@ -3700,9 +3702,9 @@ dependencies = [ [[package]] name = "wasm-bindgen-macro-support" -version = "0.2.87" +version = "0.2.88" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "54681b18a46765f095758388f2d0cf16eb8d4169b639ab575a8f5693af210c7b" +checksum = "c5353b8dab669f5e10f5bd76df26a9360c748f054f862ff5f3f8aae0c7fb3907" dependencies = [ "proc-macro2", "quote", @@ -3713,9 +3715,9 @@ dependencies = [ [[package]] name = "wasm-bindgen-shared" -version = "0.2.87" +version = "0.2.88" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "ca6ad05a4870b2bf5fe995117d3728437bd27d7cd5f06f13c17443ef369775a1" +checksum = "0d046c5d029ba91a1ed14da14dca44b68bf2f124cfbaf741c54151fdb3e0750b" [[package]] name = "wasm-streams" @@ -3732,9 +3734,9 @@ dependencies = [ [[package]] name = "web-sys" -version = "0.3.64" +version = "0.3.65" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "9b85cbef8c220a6abc02aefd892dfc0fc23afb1c6a426316ec33253a3877249b" +checksum = "5db499c5f66323272151db0e666cd34f78617522fb0c1604d31a27c50c206a85" dependencies = [ "js-sys", "wasm-bindgen", @@ -3889,18 +3891,18 @@ dependencies = [ [[package]] name = "zerocopy" -version = "0.7.20" +version = "0.7.25" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "dd66a62464e3ffd4e37bd09950c2b9dd6c4f8767380fabba0d523f9a775bc85a" +checksum = "8cd369a67c0edfef15010f980c3cbe45d7f651deac2cd67ce097cd801de16557" dependencies = [ "zerocopy-derive", ] [[package]] name = "zerocopy-derive" -version = "0.7.20" +version = "0.7.25" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "255c4596d41e6916ced49cfafea18727b24d67878fa180ddfd69b9df34fd1726" +checksum = "c2f140bda219a26ccc0cdb03dba58af72590c53b22642577d88a927bc5c87d6b" dependencies = [ "proc-macro2", "quote", diff --git a/datafusion-cli/Cargo.toml b/datafusion-cli/Cargo.toml index 7dd9cb8bcb37..73c4431f4352 100644 --- a/datafusion-cli/Cargo.toml +++ b/datafusion-cli/Cargo.toml @@ -18,7 +18,7 @@ [package] name = "datafusion-cli" description = "Command Line Client for DataFusion query engine." -version = "32.0.0" +version = "33.0.0" authors = ["Apache Arrow "] edition = "2021" keywords = ["arrow", "datafusion", "query", "sql"] @@ -34,7 +34,7 @@ async-trait = "0.1.41" aws-config = "0.55" aws-credential-types = "0.55" clap = { version = "3", features = ["derive", "cargo"] } -datafusion = { path = "../datafusion/core", version = "32.0.0", features = ["avro", "crypto_expressions", "encoding_expressions", "parquet", "regex_expressions", "unicode_expressions", "compression"] } +datafusion = { path = "../datafusion/core", version = "33.0.0", features = ["avro", "crypto_expressions", "encoding_expressions", "parquet", "regex_expressions", "unicode_expressions", "compression"] } dirs = "4.0.0" env_logger = "0.9" mimalloc = { version = "0.1", default-features = false } diff --git a/datafusion/CHANGELOG.md b/datafusion/CHANGELOG.md index d26081dcb6df..e224b9387655 100644 --- a/datafusion/CHANGELOG.md +++ b/datafusion/CHANGELOG.md @@ -19,6 +19,7 @@ # Changelog +- [33.0.0](../dev/changelog/33.0.0.md) - [32.0.0](../dev/changelog/32.0.0.md) - [31.0.0](../dev/changelog/31.0.0.md) - [30.0.0](../dev/changelog/30.0.0.md) diff --git a/datafusion/core/Cargo.toml b/datafusion/core/Cargo.toml index 4015ba439e67..b44914ec719f 100644 --- a/datafusion/core/Cargo.toml +++ b/datafusion/core/Cargo.toml @@ -62,11 +62,11 @@ bytes = { workspace = true } bzip2 = { version = "0.4.3", optional = true } chrono = { workspace = true } dashmap = { workspace = true } -datafusion-common = { path = "../common", version = "32.0.0", features = ["object_store"], default-features = false } +datafusion-common = { path = "../common", version = "33.0.0", features = ["object_store"], default-features = false } datafusion-execution = { workspace = true } datafusion-expr = { workspace = true } -datafusion-optimizer = { path = "../optimizer", version = "32.0.0", default-features = false } -datafusion-physical-expr = { path = "../physical-expr", version = "32.0.0", default-features = false } +datafusion-optimizer = { path = "../optimizer", version = "33.0.0", default-features = false } +datafusion-physical-expr = { path = "../physical-expr", version = "33.0.0", default-features = false } datafusion-physical-plan = { workspace = true } datafusion-sql = { workspace = true } flate2 = { version = "1.0.24", optional = true } diff --git a/datafusion/optimizer/Cargo.toml b/datafusion/optimizer/Cargo.toml index 797dd17a26b5..fac880867fef 100644 --- a/datafusion/optimizer/Cargo.toml +++ b/datafusion/optimizer/Cargo.toml @@ -44,7 +44,7 @@ async-trait = { workspace = true } chrono = { workspace = true } datafusion-common = { workspace = true } datafusion-expr = { workspace = true } -datafusion-physical-expr = { path = "../physical-expr", version = "32.0.0", default-features = false } +datafusion-physical-expr = { path = "../physical-expr", version = "33.0.0", default-features = false } hashbrown = { version = "0.14", features = ["raw"] } itertools = { workspace = true } log = { workspace = true } @@ -52,5 +52,5 @@ regex-syntax = "0.8.0" [dev-dependencies] ctor = { workspace = true } -datafusion-sql = { path = "../sql", version = "32.0.0" } +datafusion-sql = { path = "../sql", version = "33.0.0" } env_logger = "0.10.0" diff --git a/datafusion/proto/Cargo.toml b/datafusion/proto/Cargo.toml index 72a4df66ebd7..ac3439a64ca8 100644 --- a/datafusion/proto/Cargo.toml +++ b/datafusion/proto/Cargo.toml @@ -43,7 +43,7 @@ parquet = ["datafusion/parquet", "datafusion-common/parquet"] [dependencies] arrow = { workspace = true } chrono = { workspace = true } -datafusion = { path = "../core", version = "32.0.0" } +datafusion = { path = "../core", version = "33.0.0" } datafusion-common = { workspace = true } datafusion-expr = { workspace = true } object_store = { version = "0.7.0" } diff --git a/datafusion/sqllogictest/Cargo.toml b/datafusion/sqllogictest/Cargo.toml index 07debf179529..d27e88274f8f 100644 --- a/datafusion/sqllogictest/Cargo.toml +++ b/datafusion/sqllogictest/Cargo.toml @@ -36,7 +36,7 @@ async-trait = { workspace = true } bigdecimal = { workspace = true } bytes = { version = "1.4.0", optional = true } chrono = { workspace = true, optional = true } -datafusion = { path = "../core", version = "32.0.0" } +datafusion = { path = "../core", version = "33.0.0" } datafusion-common = { workspace = true } futures = { version = "0.3.28" } half = { workspace = true } diff --git a/dev/changelog/33.0.0.md b/dev/changelog/33.0.0.md new file mode 100644 index 000000000000..9acf40705264 --- /dev/null +++ b/dev/changelog/33.0.0.md @@ -0,0 +1,228 @@ + + +## [33.0.0](https://github.com/apache/arrow-datafusion/tree/33.0.0) (2023-11-05) + +[Full Changelog](https://github.com/apache/arrow-datafusion/compare/31.0.0...32.0.0) + +**Breaking changes:** + +- Refactor Statistics, introduce precision estimates (`Exact`, `Inexact`, `Absent`) [#7793](https://github.com/apache/arrow-datafusion/pull/7793) (berkaysynnada) +- Remove redundant unwrap in `ScalarValue::new_primitive`, return a `Result` [#7830](https://github.com/apache/arrow-datafusion/pull/7830) (maruschin) +- Add `parquet` feature flag, enabled by default, and make parquet conditional [#7745](https://github.com/apache/arrow-datafusion/pull/7745) (ongchi) +- Change input for `to_timestamp` function to be seconds rather than nanoseconds, add `to_timestamp_nanos` [#7844](https://github.com/apache/arrow-datafusion/pull/7844) (comphead) +- Percent Decode URL Paths (#8009) [#8012](https://github.com/apache/arrow-datafusion/pull/8012) (tustvold) + +**Implemented enhancements:** + +- Support InsertInto Sorted ListingTable [#7743](https://github.com/apache/arrow-datafusion/pull/7743) (devinjdangelo) +- External Table Primary key support [#7755](https://github.com/apache/arrow-datafusion/pull/7755) (mustafasrepo) +- add interval arithmetic for timestamp types [#7758](https://github.com/apache/arrow-datafusion/pull/7758) (mhilton) +- Interval Arithmetic NegativeExpr Support [#7804](https://github.com/apache/arrow-datafusion/pull/7804) (berkaysynnada) +- Exactness Indicator of Parameters: Precision [#7809](https://github.com/apache/arrow-datafusion/pull/7809) (berkaysynnada) +- Add distinct union optimization [#7788](https://github.com/apache/arrow-datafusion/pull/7788) (maruschin) +- Implement GetIndexedField for map-typed columns [#7825](https://github.com/apache/arrow-datafusion/pull/7825) (swgillespie) +- Fix precision loss when coercing date_part utf8 argument [#7846](https://github.com/apache/arrow-datafusion/pull/7846) (Dandandan) +- Support `Binary`/`LargeBinary` --> `Utf8`/`LargeUtf8` in ilike and string functions [#7840](https://github.com/apache/arrow-datafusion/pull/7840) (alamb) +- Support Decimal256 on AVG aggregate expression [#7853](https://github.com/apache/arrow-datafusion/pull/7853) (viirya) +- Support Decimal256 column in create external table [#7866](https://github.com/apache/arrow-datafusion/pull/7866) (viirya) +- Support Decimal256 in Min/Max aggregate expressions [#7881](https://github.com/apache/arrow-datafusion/pull/7881) (viirya) +- Implement Hive-Style Partitioned Write Support [#7801](https://github.com/apache/arrow-datafusion/pull/7801) (devinjdangelo) +- feat: support `Decimal256` for the `abs` function [#7904](https://github.com/apache/arrow-datafusion/pull/7904) (jonahgao) +- Parallelize Serialization of Columns within Parquet RowGroups [#7655](https://github.com/apache/arrow-datafusion/pull/7655) (devinjdangelo) +- feat: Use bloom filter when reading parquet to skip row groups [#7821](https://github.com/apache/arrow-datafusion/pull/7821) (hengfeiyang) +- Support Partitioning Data by Dictionary Encoded String Array Types [#7896](https://github.com/apache/arrow-datafusion/pull/7896) (devinjdangelo) +- Read only enough bytes to infer Arrow IPC file schema via stream [#7962](https://github.com/apache/arrow-datafusion/pull/7962) (Jefffrey) + +**Fixed bugs:** + +- fix: preserve column qualifier for `DataFrame::with_column` [#7792](https://github.com/apache/arrow-datafusion/pull/7792) (jonahgao) +- fix: don't push down volatile predicates in projection [#7909](https://github.com/apache/arrow-datafusion/pull/7909) (haohuaijin) +- fix: generate logical plan for `UPDATE SET FROM` statement [#7984](https://github.com/apache/arrow-datafusion/pull/7984) (jonahgao) +- fix: single_distinct_aggretation_to_group_by fail [#7997](https://github.com/apache/arrow-datafusion/pull/7997) (haohuaijin) +- fix: clippy warnings from nightly rust 1.75 [#8025](https://github.com/apache/arrow-datafusion/pull/8025) (waynexia) + +**Documentation updates:** + +- Minor: Improve TableProvider document, and add ascii art [#7759](https://github.com/apache/arrow-datafusion/pull/7759) (alamb) +- Expose arrow-schema `serde` crate feature flag [#7829](https://github.com/apache/arrow-datafusion/pull/7829) (lewiszlw) +- doc: fix ExecutionContext to SessionContext in custom-table-providers.md [#7903](https://github.com/apache/arrow-datafusion/pull/7903) (ZENOTME) +- Minor: Document `parquet` crate feature [#7927](https://github.com/apache/arrow-datafusion/pull/7927) (alamb) +- Add some initial content about creating logical plans [#7952](https://github.com/apache/arrow-datafusion/pull/7952) (andygrove) +- Minor: Add implementation examples to ExecutionPlan::execute [#8013](https://github.com/apache/arrow-datafusion/pull/8013) (tustvold) +- Minor: Improve documentation for Filter Pushdown [#8023](https://github.com/apache/arrow-datafusion/pull/8023) (alamb) +- Minor: Improve `ExecutionPlan` documentation [#8019](https://github.com/apache/arrow-datafusion/pull/8019) (alamb) +- Improve comments for `PartitionSearchMode` struct [#8047](https://github.com/apache/arrow-datafusion/pull/8047) (ozankabak) + +**Merged pull requests:** + +- Minor: Improve TableProvider document, and add ascii art [#7759](https://github.com/apache/arrow-datafusion/pull/7759) (alamb) +- Prepare 32.0.0 Release [#7769](https://github.com/apache/arrow-datafusion/pull/7769) (andygrove) +- Minor: Change all file links to GitHub in document [#7768](https://github.com/apache/arrow-datafusion/pull/7768) (ongchi) +- Minor: Improve `PruningPredicate` documentation [#7738](https://github.com/apache/arrow-datafusion/pull/7738) (alamb) +- Support InsertInto Sorted ListingTable [#7743](https://github.com/apache/arrow-datafusion/pull/7743) (devinjdangelo) +- Minor: improve documentation to `stagger_batch` [#7754](https://github.com/apache/arrow-datafusion/pull/7754) (alamb) +- External Table Primary key support [#7755](https://github.com/apache/arrow-datafusion/pull/7755) (mustafasrepo) +- Minor: Build array_array() with ListArray construction instead of ArrayData [#7780](https://github.com/apache/arrow-datafusion/pull/7780) (jayzhan211) +- Minor: Remove unnecessary `#[cfg(feature = "avro")]` [#7773](https://github.com/apache/arrow-datafusion/pull/7773) (sarutak) +- add interval arithmetic for timestamp types [#7758](https://github.com/apache/arrow-datafusion/pull/7758) (mhilton) +- Minor: make tests deterministic [#7771](https://github.com/apache/arrow-datafusion/pull/7771) (Weijun-H) +- Minor: Improve `Interval` Docs [#7782](https://github.com/apache/arrow-datafusion/pull/7782) (alamb) +- `DataSink` additions [#7778](https://github.com/apache/arrow-datafusion/pull/7778) (Dandandan) +- Update substrait requirement from 0.15.0 to 0.16.0 [#7783](https://github.com/apache/arrow-datafusion/pull/7783) (dependabot[bot]) +- Move nested union optimization from plan builder to logical optimizer [#7695](https://github.com/apache/arrow-datafusion/pull/7695) (maruschin) +- Minor: comments that explain the schema used in simply_expressions [#7747](https://github.com/apache/arrow-datafusion/pull/7747) (alamb) +- Update regex-syntax requirement from 0.7.1 to 0.8.0 [#7784](https://github.com/apache/arrow-datafusion/pull/7784) (dependabot[bot]) +- Minor: Add sql test for `UNION` / `UNION ALL` + plans [#7787](https://github.com/apache/arrow-datafusion/pull/7787) (alamb) +- fix: preserve column qualifier for `DataFrame::with_column` [#7792](https://github.com/apache/arrow-datafusion/pull/7792) (jonahgao) +- Interval Arithmetic NegativeExpr Support [#7804](https://github.com/apache/arrow-datafusion/pull/7804) (berkaysynnada) +- Exactness Indicator of Parameters: Precision [#7809](https://github.com/apache/arrow-datafusion/pull/7809) (berkaysynnada) +- add `LogicalPlanBuilder::join_on` [#7805](https://github.com/apache/arrow-datafusion/pull/7805) (haohuaijin) +- Fix SortPreservingRepartition with no existing ordering. [#7811](https://github.com/apache/arrow-datafusion/pull/7811) (mustafasrepo) +- Update zstd requirement from 0.12 to 0.13 [#7806](https://github.com/apache/arrow-datafusion/pull/7806) (dependabot[bot]) +- [Minor]: Remove input_schema field from window executor [#7810](https://github.com/apache/arrow-datafusion/pull/7810) (mustafasrepo) +- refactor(7181): move streaming_merge() into separate mod from the merge node [#7799](https://github.com/apache/arrow-datafusion/pull/7799) (wiedld) +- Improve update error [#7777](https://github.com/apache/arrow-datafusion/pull/7777) (lewiszlw) +- Minor: Update LogicalPlan::join_on API, use it more [#7814](https://github.com/apache/arrow-datafusion/pull/7814) (alamb) +- Add distinct union optimization [#7788](https://github.com/apache/arrow-datafusion/pull/7788) (maruschin) +- Make CI fail on any occurrence of rust-tomlfmt failed [#7774](https://github.com/apache/arrow-datafusion/pull/7774) (ongchi) +- Encode all join conditions in a single expression field [#7612](https://github.com/apache/arrow-datafusion/pull/7612) (nseekhao) +- Update substrait requirement from 0.16.0 to 0.17.0 [#7808](https://github.com/apache/arrow-datafusion/pull/7808) (dependabot[bot]) +- Minor: include `sort` expressions in `SortPreservingRepartitionExec` explain plan [#7796](https://github.com/apache/arrow-datafusion/pull/7796) (alamb) +- minor: add more document to Wildcard expr [#7822](https://github.com/apache/arrow-datafusion/pull/7822) (waynexia) +- Minor: Move `Monotonicity` to `expr` crate [#7820](https://github.com/apache/arrow-datafusion/pull/7820) (2010YOUY01) +- Use code block for better formatting of rustdoc for PhysicalGroupBy [#7823](https://github.com/apache/arrow-datafusion/pull/7823) (qrilka) +- Update explain plan to show `TopK` operator [#7826](https://github.com/apache/arrow-datafusion/pull/7826) (haohuaijin) +- Extract ReceiverStreamBuilder [#7817](https://github.com/apache/arrow-datafusion/pull/7817) (tustvold) +- Extend backtrace coverage for `DatafusionError::Plan` errors errors [#7803](https://github.com/apache/arrow-datafusion/pull/7803) (comphead) +- Add documentation and usability for prepared parameters [#7785](https://github.com/apache/arrow-datafusion/pull/7785) (alamb) +- Implement GetIndexedField for map-typed columns [#7825](https://github.com/apache/arrow-datafusion/pull/7825) (swgillespie) +- Minor: Assert `streaming_merge` has non empty sort exprs [#7795](https://github.com/apache/arrow-datafusion/pull/7795) (alamb) +- Minor: Upgrade docs for `PhysicalExpr::{propagate_constraints, evaluate_bounds}` [#7812](https://github.com/apache/arrow-datafusion/pull/7812) (alamb) +- Change ScalarValue::List to store ArrayRef [#7629](https://github.com/apache/arrow-datafusion/pull/7629) (jayzhan211) +- [MINOR]:Do not introduce unnecessary repartition when row count is 1. [#7832](https://github.com/apache/arrow-datafusion/pull/7832) (mustafasrepo) +- Minor: Add tests for binary / utf8 coercion [#7839](https://github.com/apache/arrow-datafusion/pull/7839) (alamb) +- Avoid panics on error while encoding/decoding ListValue::Array as protobuf [#7837](https://github.com/apache/arrow-datafusion/pull/7837) (alamb) +- Refactor Statistics, introduce precision estimates (`Exact`, `Inexact`, `Absent`) [#7793](https://github.com/apache/arrow-datafusion/pull/7793) (berkaysynnada) +- Remove redundant unwrap in `ScalarValue::new_primitive`, return a `Result` [#7830](https://github.com/apache/arrow-datafusion/pull/7830) (maruschin) +- Fix precision loss when coercing date_part utf8 argument [#7846](https://github.com/apache/arrow-datafusion/pull/7846) (Dandandan) +- Add operator section to user guide, Add `std::ops` operations to `prelude`, and add `not()` expr_fn [#7732](https://github.com/apache/arrow-datafusion/pull/7732) (ongchi) +- Expose arrow-schema `serde` crate feature flag [#7829](https://github.com/apache/arrow-datafusion/pull/7829) (lewiszlw) +- Improve `ContextProvider` naming: rename` get_table_provider` --> `get_table_source`, deprecate `get_table_provider` [#7831](https://github.com/apache/arrow-datafusion/pull/7831) (lewiszlw) +- DataSink Dynamic Execution Time Demux [#7791](https://github.com/apache/arrow-datafusion/pull/7791) (devinjdangelo) +- Add small column on empty projection [#7833](https://github.com/apache/arrow-datafusion/pull/7833) (ch-sc) +- feat(7849): coerce TIMESTAMP to TIMESTAMPTZ [#7850](https://github.com/apache/arrow-datafusion/pull/7850) (mhilton) +- Support `Binary`/`LargeBinary` --> `Utf8`/`LargeUtf8` in ilike and string functions [#7840](https://github.com/apache/arrow-datafusion/pull/7840) (alamb) +- Minor: fix typo in comments [#7856](https://github.com/apache/arrow-datafusion/pull/7856) (haohuaijin) +- Minor: improve `join` / `join_on` docs [#7813](https://github.com/apache/arrow-datafusion/pull/7813) (alamb) +- Support Decimal256 on AVG aggregate expression [#7853](https://github.com/apache/arrow-datafusion/pull/7853) (viirya) +- Minor: fix typo in comments [#7861](https://github.com/apache/arrow-datafusion/pull/7861) (alamb) +- Minor: fix typo in GreedyMemoryPool documentation [#7864](https://github.com/apache/arrow-datafusion/pull/7864) (avh4) +- Minor: fix multiple typos [#7863](https://github.com/apache/arrow-datafusion/pull/7863) (Smoothieewastaken) +- Minor: Fix docstring typos [#7873](https://github.com/apache/arrow-datafusion/pull/7873) (alamb) +- Add CursorValues Decoupling Cursor Data from Cursor Position [#7855](https://github.com/apache/arrow-datafusion/pull/7855) (tustvold) +- Support Decimal256 column in create external table [#7866](https://github.com/apache/arrow-datafusion/pull/7866) (viirya) +- Support Decimal256 in Min/Max aggregate expressions [#7881](https://github.com/apache/arrow-datafusion/pull/7881) (viirya) +- Implement Hive-Style Partitioned Write Support [#7801](https://github.com/apache/arrow-datafusion/pull/7801) (devinjdangelo) +- Minor: fix config typo [#7874](https://github.com/apache/arrow-datafusion/pull/7874) (alamb) +- Add Decimal256 sqllogictests for SUM, MEDIAN and COUNT aggregate expressions [#7889](https://github.com/apache/arrow-datafusion/pull/7889) (viirya) +- [test] add fuzz test for topk [#7772](https://github.com/apache/arrow-datafusion/pull/7772) (Tangruilin) +- Allow Setting Minimum Parallelism with RowCount Based Demuxer [#7841](https://github.com/apache/arrow-datafusion/pull/7841) (devinjdangelo) +- Drop single quotes to make warnings for parquet options not confusing [#7902](https://github.com/apache/arrow-datafusion/pull/7902) (qrilka) +- Add multi-column topk fuzz tests [#7898](https://github.com/apache/arrow-datafusion/pull/7898) (alamb) +- Change `FileScanConfig.table_partition_cols` from `(String, DataType)` to `Field`s [#7890](https://github.com/apache/arrow-datafusion/pull/7890) (NGA-TRAN) +- Maintain time zone in `ScalarValue::new_list` [#7899](https://github.com/apache/arrow-datafusion/pull/7899) (Dandandan) +- [MINOR]: Move joinside struct to common [#7908](https://github.com/apache/arrow-datafusion/pull/7908) (mustafasrepo) +- doc: fix ExecutionContext to SessionContext in custom-table-providers.md [#7903](https://github.com/apache/arrow-datafusion/pull/7903) (ZENOTME) +- Update arrow 48.0.0 [#7854](https://github.com/apache/arrow-datafusion/pull/7854) (tustvold) +- feat: support `Decimal256` for the `abs` function [#7904](https://github.com/apache/arrow-datafusion/pull/7904) (jonahgao) +- [MINOR] Simplify Aggregate, and Projection output_partitioning implementation [#7907](https://github.com/apache/arrow-datafusion/pull/7907) (mustafasrepo) +- Bump actions/setup-node from 3 to 4 [#7915](https://github.com/apache/arrow-datafusion/pull/7915) (dependabot[bot]) +- [Bug Fix]: Fix bug, first last reverse [#7914](https://github.com/apache/arrow-datafusion/pull/7914) (mustafasrepo) +- Minor: provide default implementation for ExecutionPlan::statistics [#7911](https://github.com/apache/arrow-datafusion/pull/7911) (alamb) +- Update substrait requirement from 0.17.0 to 0.18.0 [#7916](https://github.com/apache/arrow-datafusion/pull/7916) (dependabot[bot]) +- Minor: Remove unnecessary clone in datafusion_proto [#7921](https://github.com/apache/arrow-datafusion/pull/7921) (ongchi) +- [MINOR]: Simplify code, change requirement from PhysicalSortExpr to PhysicalSortRequirement [#7913](https://github.com/apache/arrow-datafusion/pull/7913) (mustafasrepo) +- [Minor] Move combine_join util to under equivalence.rs [#7917](https://github.com/apache/arrow-datafusion/pull/7917) (mustafasrepo) +- support scan empty projection [#7920](https://github.com/apache/arrow-datafusion/pull/7920) (haohuaijin) +- Cleanup logical optimizer rules. [#7919](https://github.com/apache/arrow-datafusion/pull/7919) (mustafasrepo) +- Parallelize Serialization of Columns within Parquet RowGroups [#7655](https://github.com/apache/arrow-datafusion/pull/7655) (devinjdangelo) +- feat: Use bloom filter when reading parquet to skip row groups [#7821](https://github.com/apache/arrow-datafusion/pull/7821) (hengfeiyang) +- fix: don't push down volatile predicates in projection [#7909](https://github.com/apache/arrow-datafusion/pull/7909) (haohuaijin) +- Add `parquet` feature flag, enabled by default, and make parquet conditional [#7745](https://github.com/apache/arrow-datafusion/pull/7745) (ongchi) +- [MINOR]: Simplify enforce_distribution, minor changes [#7924](https://github.com/apache/arrow-datafusion/pull/7924) (mustafasrepo) +- Add simple window query to sqllogictest [#7928](https://github.com/apache/arrow-datafusion/pull/7928) (Jefffrey) +- ci: upgrade node to version 20 [#7918](https://github.com/apache/arrow-datafusion/pull/7918) (crepererum) +- Change input for `to_timestamp` function to be seconds rather than nanoseconds, add `to_timestamp_nanos` [#7844](https://github.com/apache/arrow-datafusion/pull/7844) (comphead) +- Minor: Document `parquet` crate feature [#7927](https://github.com/apache/arrow-datafusion/pull/7927) (alamb) +- Minor: reduce some `#cfg(feature = "parquet")` [#7929](https://github.com/apache/arrow-datafusion/pull/7929) (alamb) +- Minor: reduce use of `#cfg(feature = "parquet")` in tests [#7930](https://github.com/apache/arrow-datafusion/pull/7930) (alamb) +- Fix CI failures on `to_timestamp()` calls [#7941](https://github.com/apache/arrow-datafusion/pull/7941) (comphead) +- minor: add a datatype casting for the updated value [#7922](https://github.com/apache/arrow-datafusion/pull/7922) (jonahgao) +- Minor:add `avro` feature in datafusion-examples to make `avro_sql` run [#7946](https://github.com/apache/arrow-datafusion/pull/7946) (haohuaijin) +- Add simple exclude all columns test to sqllogictest [#7945](https://github.com/apache/arrow-datafusion/pull/7945) (Jefffrey) +- Support Partitioning Data by Dictionary Encoded String Array Types [#7896](https://github.com/apache/arrow-datafusion/pull/7896) (devinjdangelo) +- Minor: Remove array() in array_expression [#7961](https://github.com/apache/arrow-datafusion/pull/7961) (jayzhan211) +- Minor: simplify update code [#7943](https://github.com/apache/arrow-datafusion/pull/7943) (alamb) +- Add some initial content about creating logical plans [#7952](https://github.com/apache/arrow-datafusion/pull/7952) (andygrove) +- Minor: Change from `&mut SessionContext` to `&SessionContext` in substrait [#7965](https://github.com/apache/arrow-datafusion/pull/7965) (my-vegetable-has-exploded) +- Fix crate READMEs [#7964](https://github.com/apache/arrow-datafusion/pull/7964) (Jefffrey) +- Minor: Improve `HashJoinExec` documentation [#7953](https://github.com/apache/arrow-datafusion/pull/7953) (alamb) +- chore: clean useless clone baesd on clippy [#7973](https://github.com/apache/arrow-datafusion/pull/7973) (Weijun-H) +- Add README.md to `core`, `execution` and `physical-plan` crates [#7970](https://github.com/apache/arrow-datafusion/pull/7970) (alamb) +- Move source repartitioning into `ExecutionPlan::repartition` [#7936](https://github.com/apache/arrow-datafusion/pull/7936) (alamb) +- minor: fix broken links in README.md [#7986](https://github.com/apache/arrow-datafusion/pull/7986) (jonahgao) +- Minor: Upate the `sqllogictest` crate README [#7971](https://github.com/apache/arrow-datafusion/pull/7971) (alamb) +- Improve MemoryCatalogProvider default impl block placement [#7975](https://github.com/apache/arrow-datafusion/pull/7975) (lewiszlw) +- Fix `ScalarValue` handling of NULL values for ListArray [#7969](https://github.com/apache/arrow-datafusion/pull/7969) (viirya) +- Refactor of Ordering and Prunability Traversals and States [#7985](https://github.com/apache/arrow-datafusion/pull/7985) (berkaysynnada) +- Keep output as scalar for scalar function if all inputs are scalar [#7967](https://github.com/apache/arrow-datafusion/pull/7967) (viirya) +- Fix crate READMEs for core, execution, physical-plan [#7990](https://github.com/apache/arrow-datafusion/pull/7990) (Jefffrey) +- Update sqlparser requirement from 0.38.0 to 0.39.0 [#7983](https://github.com/apache/arrow-datafusion/pull/7983) (jackwener) +- Fix panic in multiple distinct aggregates by fixing `ScalarValue::new_list` [#7989](https://github.com/apache/arrow-datafusion/pull/7989) (alamb) +- Minor: Add `MemoryReservation::consumer` getter [#8000](https://github.com/apache/arrow-datafusion/pull/8000) (milenkovicm) +- fix: generate logical plan for `UPDATE SET FROM` statement [#7984](https://github.com/apache/arrow-datafusion/pull/7984) (jonahgao) +- Create temporary files for reading or writing [#8005](https://github.com/apache/arrow-datafusion/pull/8005) (smallzhongfeng) +- Minor: fix comment on SortExec::with_fetch method [#8011](https://github.com/apache/arrow-datafusion/pull/8011) (westonpace) +- Fix: dataframe_subquery example Optimizer rule `common_sub_expression_eliminate` failed [#8016](https://github.com/apache/arrow-datafusion/pull/8016) (smallzhongfeng) +- Percent Decode URL Paths (#8009) [#8012](https://github.com/apache/arrow-datafusion/pull/8012) (tustvold) +- Minor: Extract common deps into workspace [#7982](https://github.com/apache/arrow-datafusion/pull/7982) (lewiszlw) +- minor: change some plan_err to exec_err [#7996](https://github.com/apache/arrow-datafusion/pull/7996) (waynexia) +- Minor: error on unsupported RESPECT NULLs syntax [#7998](https://github.com/apache/arrow-datafusion/pull/7998) (alamb) +- Break GroupedHashAggregateStream spill batch into smaller chunks [#8004](https://github.com/apache/arrow-datafusion/pull/8004) (milenkovicm) +- Minor: Add implementation examples to ExecutionPlan::execute [#8013](https://github.com/apache/arrow-datafusion/pull/8013) (tustvold) +- Minor: Extend wrap_into_list_array to accept multiple args [#7993](https://github.com/apache/arrow-datafusion/pull/7993) (jayzhan211) +- GroupedHashAggregateStream should register spillable consumer [#8002](https://github.com/apache/arrow-datafusion/pull/8002) (milenkovicm) +- fix: single_distinct_aggretation_to_group_by fail [#7997](https://github.com/apache/arrow-datafusion/pull/7997) (haohuaijin) +- Read only enough bytes to infer Arrow IPC file schema via stream [#7962](https://github.com/apache/arrow-datafusion/pull/7962) (Jefffrey) +- Minor: remove a strange char [#8030](https://github.com/apache/arrow-datafusion/pull/8030) (haohuaijin) +- Minor: Improve documentation for Filter Pushdown [#8023](https://github.com/apache/arrow-datafusion/pull/8023) (alamb) +- Minor: Improve `ExecutionPlan` documentation [#8019](https://github.com/apache/arrow-datafusion/pull/8019) (alamb) +- fix: clippy warnings from nightly rust 1.75 [#8025](https://github.com/apache/arrow-datafusion/pull/8025) (waynexia) +- Minor: Avoid recomputing compute_array_ndims in align_array_dimensions [#7963](https://github.com/apache/arrow-datafusion/pull/7963) (jayzhan211) +- Minor: fix doc and fmt CI check [#8037](https://github.com/apache/arrow-datafusion/pull/8037) (alamb) +- Minor: remove uncessary #cfg test [#8036](https://github.com/apache/arrow-datafusion/pull/8036) (alamb) +- Minor: Improve documentation for `PartitionStream` and `StreamingTableExec` [#8035](https://github.com/apache/arrow-datafusion/pull/8035) (alamb) +- Combine Equivalence and Ordering equivalence to simplify state [#8006](https://github.com/apache/arrow-datafusion/pull/8006) (mustafasrepo) +- Encapsulate `ProjectionMapping` as a struct [#8033](https://github.com/apache/arrow-datafusion/pull/8033) (alamb) +- Minor: Fix bugs in docs for `to_timestamp`, `to_timestamp_seconds`, ... [#8040](https://github.com/apache/arrow-datafusion/pull/8040) (alamb) +- Improve comments for `PartitionSearchMode` struct [#8047](https://github.com/apache/arrow-datafusion/pull/8047) (ozankabak) +- General approach for Array replace [#8050](https://github.com/apache/arrow-datafusion/pull/8050) (jayzhan211) +- Minor: Remove the irrelevant note from the Expression API doc [#8053](https://github.com/apache/arrow-datafusion/pull/8053) (ongchi) +- Minor: Add more documentation about Partitioning [#8022](https://github.com/apache/arrow-datafusion/pull/8022) (alamb) diff --git a/docs/Cargo.toml b/docs/Cargo.toml index 9caa0bde3608..4d01466924f9 100644 --- a/docs/Cargo.toml +++ b/docs/Cargo.toml @@ -29,4 +29,4 @@ authors = { workspace = true } rust-version = "1.70" [dependencies] -datafusion = { path = "../datafusion/core", version = "32.0.0", default-features = false } +datafusion = { path = "../datafusion/core", version = "33.0.0", default-features = false } diff --git a/docs/source/user-guide/configs.md b/docs/source/user-guide/configs.md index c8ff1b06d609..4cc4fd1c3a25 100644 --- a/docs/source/user-guide/configs.md +++ b/docs/source/user-guide/configs.md @@ -64,7 +64,7 @@ Environment variables are read during `SessionConfig` initialisation so they mus | datafusion.execution.parquet.statistics_enabled | NULL | Sets if statistics are enabled for any column Valid values are: "none", "chunk", and "page" These values are not case sensitive. If NULL, uses default parquet writer setting | | datafusion.execution.parquet.max_statistics_size | NULL | Sets max statistics size for any column. If NULL, uses default parquet writer setting | | datafusion.execution.parquet.max_row_group_size | 1048576 | Sets maximum number of rows in a row group | -| datafusion.execution.parquet.created_by | datafusion version 32.0.0 | Sets "created by" property | +| datafusion.execution.parquet.created_by | datafusion version 33.0.0 | Sets "created by" property | | datafusion.execution.parquet.column_index_truncate_length | NULL | Sets column index truncate length | | datafusion.execution.parquet.data_page_row_count_limit | 18446744073709551615 | Sets best effort maximum number of rows in data page | | datafusion.execution.parquet.encoding | NULL | 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 | From af3ce6ba5131d1690d8b1af51ab63b969d715b99 Mon Sep 17 00:00:00 2001 From: Andrew Lamb Date: Mon, 6 Nov 2023 13:47:32 -0500 Subject: [PATCH 194/572] Minor: improve error message by adding types to message (#8065) * Minor: improve error message * add test --- datafusion/expr/src/logical_plan/builder.rs | 2 +- datafusion/sqllogictest/test_files/errors.slt | 4 ++++ 2 files changed, 5 insertions(+), 1 deletion(-) diff --git a/datafusion/expr/src/logical_plan/builder.rs b/datafusion/expr/src/logical_plan/builder.rs index 9ce1d203d1c9..162a6a959e59 100644 --- a/datafusion/expr/src/logical_plan/builder.rs +++ b/datafusion/expr/src/logical_plan/builder.rs @@ -165,7 +165,7 @@ impl LogicalPlanBuilder { let data_type = expr.get_type(&empty_schema)?; if let Some(prev_data_type) = &field_types[j] { if prev_data_type != &data_type { - return plan_err!("Inconsistent data type across values list at row {i} column {j}"); + return plan_err!("Inconsistent data type across values list at row {i} column {j}. Was {prev_data_type} but found {data_type}") } } Ok(Some(data_type)) diff --git a/datafusion/sqllogictest/test_files/errors.slt b/datafusion/sqllogictest/test_files/errors.slt index 1380ac2f2bfd..4aded8a576fb 100644 --- a/datafusion/sqllogictest/test_files/errors.slt +++ b/datafusion/sqllogictest/test_files/errors.slt @@ -130,3 +130,7 @@ c9, nth_value(c5, 2, 3) over (order by c9) as nv1 from aggregate_test_100 order by c9 + + +statement error Inconsistent data type across values list at row 1 column 0. Was Int64 but found Utf8 +create table foo as values (1), ('foo'); \ No newline at end of file From 308c35404939ed39e4e398054d6fc78c89817109 Mon Sep 17 00:00:00 2001 From: Yongting You <2010youy01@gmail.com> Date: Mon, 6 Nov 2023 10:56:26 -0800 Subject: [PATCH 195/572] Minor: Remove redundant BuiltinScalarFunction::supports_zero_argument() (#8059) * deprecate BuiltinScalarFunction::supports_zero_argument() * unify old supports_zero_argument() impl --- datafusion/expr/src/built_in_function.rs | 23 +++--- datafusion/expr/src/signature.rs | 75 +++++++++++++++++-- .../physical-expr/src/scalar_function.rs | 5 +- 3 files changed, 85 insertions(+), 18 deletions(-) diff --git a/datafusion/expr/src/built_in_function.rs b/datafusion/expr/src/built_in_function.rs index 4db565abfcf7..16187572c521 100644 --- a/datafusion/expr/src/built_in_function.rs +++ b/datafusion/expr/src/built_in_function.rs @@ -325,18 +325,14 @@ fn function_to_name() -> &'static HashMap { impl BuiltinScalarFunction { /// an allowlist of functions to take zero arguments, so that they will get special treatment /// while executing. + #[deprecated( + since = "32.0.0", + note = "please use TypeSignature::supports_zero_argument instead" + )] pub fn supports_zero_argument(&self) -> bool { - matches!( - self, - BuiltinScalarFunction::Pi - | BuiltinScalarFunction::Random - | BuiltinScalarFunction::Now - | BuiltinScalarFunction::CurrentDate - | BuiltinScalarFunction::CurrentTime - | BuiltinScalarFunction::Uuid - | BuiltinScalarFunction::MakeArray - ) + self.signature().type_signature.supports_zero_argument() } + /// Returns the [Volatility] of the builtin function. pub fn volatility(&self) -> Volatility { match self { @@ -494,7 +490,9 @@ impl BuiltinScalarFunction { // Note that this function *must* return the same type that the respective physical expression returns // or the execution panics. - if input_expr_types.is_empty() && !self.supports_zero_argument() { + if input_expr_types.is_empty() + && !self.signature().type_signature.supports_zero_argument() + { return plan_err!( "{}", utils::generate_signature_error_msg( @@ -904,7 +902,8 @@ impl BuiltinScalarFunction { } BuiltinScalarFunction::Cardinality => Signature::any(1, self.volatility()), BuiltinScalarFunction::MakeArray => { - Signature::variadic_any(self.volatility()) + // 0 or more arguments of arbitrary type + Signature::one_of(vec![VariadicAny, Any(0)], self.volatility()) } BuiltinScalarFunction::Struct => Signature::variadic( struct_expressions::SUPPORTED_STRUCT_TYPES.to_vec(), diff --git a/datafusion/expr/src/signature.rs b/datafusion/expr/src/signature.rs index 399aefc4b66e..685601523f9b 100644 --- a/datafusion/expr/src/signature.rs +++ b/datafusion/expr/src/signature.rs @@ -82,18 +82,18 @@ pub enum Volatility { /// ``` #[derive(Debug, Clone, PartialEq, Eq, Hash)] pub enum TypeSignature { - /// arbitrary number of arguments of an common type out of a list of valid types. + /// One or more arguments of an common type out of a list of valid types. /// /// # Examples /// A function such as `concat` is `Variadic(vec![DataType::Utf8, DataType::LargeUtf8])` Variadic(Vec), - /// arbitrary number of arguments of an arbitrary but equal type. + /// One or more arguments of an arbitrary but equal type. /// DataFusion attempts to coerce all argument types to match the first argument's type /// /// # Examples /// A function such as `array` is `VariadicEqual` VariadicEqual, - /// arbitrary number of arguments with arbitrary types + /// One or more arguments with arbitrary types VariadicAny, /// fixed number of arguments of an arbitrary but equal type out of a list of valid types. /// @@ -101,12 +101,17 @@ pub enum TypeSignature { /// 1. A function of one argument of f64 is `Uniform(1, vec![DataType::Float64])` /// 2. A function of one argument of f64 or f32 is `Uniform(1, vec![DataType::Float32, DataType::Float64])` Uniform(usize, Vec), - /// exact number of arguments of an exact type + /// Exact number of arguments of an exact type Exact(Vec), - /// fixed number of arguments of arbitrary types + /// Fixed number of arguments of arbitrary types + /// If a function takes 0 argument, its `TypeSignature` should be `Any(0)` Any(usize), /// Matches exactly one of a list of [`TypeSignature`]s. Coercion is attempted to match /// the signatures in order, and stops after the first success, if any. + /// + /// # Examples + /// Function `make_array` takes 0 or more arguments with arbitrary types, its `TypeSignature` + /// is `OneOf(vec![Any(0), VariadicAny])`. OneOf(Vec), } @@ -150,6 +155,18 @@ impl TypeSignature { .collect::>() .join(delimiter) } + + /// Check whether 0 input argument is valid for given `TypeSignature` + pub fn supports_zero_argument(&self) -> bool { + match &self { + TypeSignature::Exact(vec) => vec.is_empty(), + TypeSignature::Uniform(0, _) | TypeSignature::Any(0) => true, + TypeSignature::OneOf(types) => types + .iter() + .any(|type_sig| type_sig.supports_zero_argument()), + _ => false, + } + } } /// Defines the supported argument types ([`TypeSignature`]) and [`Volatility`] for a function. @@ -234,3 +251,51 @@ impl Signature { /// - `Some(true)` indicates that the function is monotonically increasing w.r.t. the argument in question. /// - Some(false) indicates that the function is monotonically decreasing w.r.t. the argument in question. pub type FuncMonotonicity = Vec>; + +#[cfg(test)] +mod tests { + use super::*; + + #[test] + fn supports_zero_argument_tests() { + // Testing `TypeSignature`s which supports 0 arg + let positive_cases = vec![ + TypeSignature::Exact(vec![]), + TypeSignature::Uniform(0, vec![DataType::Float64]), + TypeSignature::Any(0), + TypeSignature::OneOf(vec![ + TypeSignature::Exact(vec![DataType::Int8]), + TypeSignature::Any(0), + TypeSignature::Uniform(1, vec![DataType::Int8]), + ]), + ]; + + for case in positive_cases { + assert!( + case.supports_zero_argument(), + "Expected {:?} to support zero arguments", + case + ); + } + + // Testing `TypeSignature`s which doesn't support 0 arg + let negative_cases = vec![ + TypeSignature::Exact(vec![DataType::Utf8]), + TypeSignature::Uniform(1, vec![DataType::Float64]), + TypeSignature::Any(1), + TypeSignature::VariadicAny, + TypeSignature::OneOf(vec![ + TypeSignature::Exact(vec![DataType::Int8]), + TypeSignature::Uniform(1, vec![DataType::Int8]), + ]), + ]; + + for case in negative_cases { + assert!( + !case.supports_zero_argument(), + "Expected {:?} not to support zero arguments", + case + ); + } + } +} diff --git a/datafusion/physical-expr/src/scalar_function.rs b/datafusion/physical-expr/src/scalar_function.rs index 5acd5dcf2336..768aa04dd9c1 100644 --- a/datafusion/physical-expr/src/scalar_function.rs +++ b/datafusion/physical-expr/src/scalar_function.rs @@ -136,7 +136,10 @@ impl PhysicalExpr for ScalarFunctionExpr { let inputs = match (self.args.len(), self.name.parse::()) { // MakeArray support zero argument but has the different behavior from the array with one null. (0, Ok(scalar_fun)) - if scalar_fun.supports_zero_argument() + if scalar_fun + .signature() + .type_signature + .supports_zero_argument() && scalar_fun != BuiltinScalarFunction::MakeArray => { vec![ColumnarValue::create_null_array(batch.num_rows())] From 223a7fb4694dafa9fd72e49650cd75f45540d99b Mon Sep 17 00:00:00 2001 From: jokercurry <982458633@qq.com> Date: Tue, 7 Nov 2023 20:52:10 +0800 Subject: [PATCH 196/572] Add example to ci (#8060) * feat: add example to ci * nit * addr comments --------- Co-authored-by: zhongjingxiong --- .github/workflows/rust.yml | 14 +------- ci/scripts/rust_example.sh | 35 +++++++++++++++++++ datafusion-examples/README.md | 8 +++-- .../{ => external_dependency}/catalog.rs | 0 .../dataframe-to-s3.rs | 0 .../{ => external_dependency}/query-aws-s3.rs | 0 .../examples/{ => flight}/flight_client.rs | 0 .../examples/{ => flight}/flight_server.rs | 0 .../{ => flight}/flight_sql_server.rs | 0 datafusion-examples/examples/simple_udwf.rs | 2 +- 10 files changed, 42 insertions(+), 17 deletions(-) create mode 100755 ci/scripts/rust_example.sh rename datafusion-examples/examples/{ => external_dependency}/catalog.rs (100%) rename datafusion-examples/examples/{ => external_dependency}/dataframe-to-s3.rs (100%) rename datafusion-examples/examples/{ => external_dependency}/query-aws-s3.rs (100%) rename datafusion-examples/examples/{ => flight}/flight_client.rs (100%) rename datafusion-examples/examples/{ => flight}/flight_server.rs (100%) rename datafusion-examples/examples/{ => flight}/flight_sql_server.rs (100%) diff --git a/.github/workflows/rust.yml b/.github/workflows/rust.yml index 6c3d60bd4399..485d179571e3 100644 --- a/.github/workflows/rust.yml +++ b/.github/workflows/rust.yml @@ -139,19 +139,7 @@ jobs: # test datafusion-sql examples cargo run --example sql # test datafusion-examples - cargo run --example avro_sql --features=datafusion/avro - cargo run --example csv_sql - cargo run --example custom_datasource - cargo run --example dataframe - cargo run --example dataframe_in_memory - cargo run --example deserialize_to_struct - cargo run --example expr_api - cargo run --example parquet_sql - cargo run --example parquet_sql_multiple_files - cargo run --example memtable - cargo run --example rewrite_expr - cargo run --example simple_udf - cargo run --example simple_udaf + ci/scripts/rust_example.sh - name: Verify Working Directory Clean run: git diff --exit-code diff --git a/ci/scripts/rust_example.sh b/ci/scripts/rust_example.sh new file mode 100755 index 000000000000..fe3696f20865 --- /dev/null +++ b/ci/scripts/rust_example.sh @@ -0,0 +1,35 @@ +#!/usr/bin/env bash +# +# 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. + +set -ex +cd datafusion-examples/examples/ +cargo fmt --all -- --check + +files=$(ls .) +for filename in $files +do + example_name=`basename $filename ".rs"` + # Skip tests that rely on external storage and flight + # todo: Currently, catalog.rs is placed in the external-dependence directory because there is a problem parsing + # the parquet file of the external parquet-test that it currently relies on. + # We will wait for this issue[https://github.com/apache/arrow-datafusion/issues/8041] to be resolved. + if [ ! -d $filename ]; then + cargo run --example $example_name + fi +done diff --git a/datafusion-examples/README.md b/datafusion-examples/README.md index bfed3976c946..9f7c9f99d14e 100644 --- a/datafusion-examples/README.md +++ b/datafusion-examples/README.md @@ -44,16 +44,18 @@ cargo run --example csv_sql - [`avro_sql.rs`](examples/avro_sql.rs): Build and run a query plan from a SQL statement against a local AVRO file - [`csv_sql.rs`](examples/csv_sql.rs): Build and run a query plan from a SQL statement against a local CSV file +- [`catalog.rs`](examples/external_dependency/catalog.rs): Register the table into a custom catalog - [`custom_datasource.rs`](examples/custom_datasource.rs): Run queries against a custom datasource (TableProvider) - [`dataframe.rs`](examples/dataframe.rs): Run a query using a DataFrame against a local parquet file +- [`dataframe-to-s3.rs`](examples/external_dependency/dataframe-to-s3.rs): Run a query using a DataFrame against a parquet file from s3 - [`dataframe_in_memory.rs`](examples/dataframe_in_memory.rs): Run a query using a DataFrame against data in memory - [`deserialize_to_struct.rs`](examples/deserialize_to_struct.rs): Convert query results into rust structs using serde - [`expr_api.rs`](examples/expr_api.rs): Use the `Expr` construction and simplification API -- [`flight_sql_server.rs`](examples/flight_sql_server.rs): Run DataFusion as a standalone process and execute SQL queries from JDBC clients +- [`flight_sql_server.rs`](examples/flight/flight_sql_server.rs): Run DataFusion as a standalone process and execute SQL queries from JDBC clients - [`memtable.rs`](examples/memtable.rs): Create an query data in memory using SQL and `RecordBatch`es - [`parquet_sql.rs`](examples/parquet_sql.rs): Build and run a query plan from a SQL statement against a local Parquet file - [`parquet_sql_multiple_files.rs`](examples/parquet_sql_multiple_files.rs): Build and run a query plan from a SQL statement against multiple local Parquet files -- [`query-aws-s3.rs`](examples/query-aws-s3.rs): Configure `object_store` and run a query against files stored in AWS S3 +- [`query-aws-s3.rs`](examples/external_dependency/query-aws-s3.rs): Configure `object_store` and run a query against files stored in AWS S3 - [`query-http-csv.rs`](examples/query-http-csv.rs): Configure `object_store` and run a query against files vi HTTP - [`rewrite_expr.rs`](examples/rewrite_expr.rs): Define and invoke a custom Query Optimizer pass - [`simple_udaf.rs`](examples/simple_udaf.rs): Define and invoke a User Defined Aggregate Function (UDAF) @@ -62,4 +64,4 @@ cargo run --example csv_sql ## Distributed -- [`flight_client.rs`](examples/flight_client.rs) and [`flight_server.rs`](examples/flight_server.rs): Run DataFusion as a standalone process and execute SQL queries from a client using the Flight protocol. +- [`flight_client.rs`](examples/flight/flight_client.rs) and [`flight_server.rs`](examples/flight/flight_server.rs): Run DataFusion as a standalone process and execute SQL queries from a client using the Flight protocol. diff --git a/datafusion-examples/examples/catalog.rs b/datafusion-examples/examples/external_dependency/catalog.rs similarity index 100% rename from datafusion-examples/examples/catalog.rs rename to datafusion-examples/examples/external_dependency/catalog.rs diff --git a/datafusion-examples/examples/dataframe-to-s3.rs b/datafusion-examples/examples/external_dependency/dataframe-to-s3.rs similarity index 100% rename from datafusion-examples/examples/dataframe-to-s3.rs rename to datafusion-examples/examples/external_dependency/dataframe-to-s3.rs diff --git a/datafusion-examples/examples/query-aws-s3.rs b/datafusion-examples/examples/external_dependency/query-aws-s3.rs similarity index 100% rename from datafusion-examples/examples/query-aws-s3.rs rename to datafusion-examples/examples/external_dependency/query-aws-s3.rs diff --git a/datafusion-examples/examples/flight_client.rs b/datafusion-examples/examples/flight/flight_client.rs similarity index 100% rename from datafusion-examples/examples/flight_client.rs rename to datafusion-examples/examples/flight/flight_client.rs diff --git a/datafusion-examples/examples/flight_server.rs b/datafusion-examples/examples/flight/flight_server.rs similarity index 100% rename from datafusion-examples/examples/flight_server.rs rename to datafusion-examples/examples/flight/flight_server.rs diff --git a/datafusion-examples/examples/flight_sql_server.rs b/datafusion-examples/examples/flight/flight_sql_server.rs similarity index 100% rename from datafusion-examples/examples/flight_sql_server.rs rename to datafusion-examples/examples/flight/flight_sql_server.rs diff --git a/datafusion-examples/examples/simple_udwf.rs b/datafusion-examples/examples/simple_udwf.rs index 39042a35629b..d1cbcc7c4389 100644 --- a/datafusion-examples/examples/simple_udwf.rs +++ b/datafusion-examples/examples/simple_udwf.rs @@ -36,7 +36,7 @@ async fn create_context() -> Result { // declare a table in memory. In spark API, this corresponds to createDataFrame(...). println!("pwd: {}", std::env::current_dir().unwrap().display()); - let csv_path = "datafusion/core/tests/data/cars.csv".to_string(); + let csv_path = "../../datafusion/core/tests/data/cars.csv".to_string(); let read_options = CsvReadOptions::default().has_header(true); ctx.register_csv("cars", &csv_path, read_options).await?; From 07c08a3040eee3477b85b6ca87b95c4836547d47 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Tue, 7 Nov 2023 08:47:10 -0500 Subject: [PATCH 197/572] Update substrait requirement from 0.18.0 to 0.19.0 (#8076) Updates the requirements on [substrait](https://github.com/substrait-io/substrait-rs) to permit the latest version. - [Release notes](https://github.com/substrait-io/substrait-rs/releases) - [Changelog](https://github.com/substrait-io/substrait-rs/blob/main/CHANGELOG.md) - [Commits](https://github.com/substrait-io/substrait-rs/compare/v0.18.0...v0.19.0) --- updated-dependencies: - dependency-name: substrait dependency-type: direct:production ... Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- datafusion/substrait/Cargo.toml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/datafusion/substrait/Cargo.toml b/datafusion/substrait/Cargo.toml index 585cb6e43d15..102b0a7c58f1 100644 --- a/datafusion/substrait/Cargo.toml +++ b/datafusion/substrait/Cargo.toml @@ -35,7 +35,7 @@ itertools = { workspace = true } object_store = { workspace = true } prost = "0.12" prost-types = "0.12" -substrait = "0.18.0" +substrait = "0.19.0" tokio = "1.17" [features] From 06fd26b86dd8e5269966be8658862e5a5a12f948 Mon Sep 17 00:00:00 2001 From: Mark Sirek Date: Tue, 7 Nov 2023 05:57:10 -0800 Subject: [PATCH 198/572] Fix incorrect results in COUNT(*) queries with LIMIT (#8049) Co-authored-by: Mark Sirek --- datafusion/physical-plan/src/limit.rs | 159 +++++++++++++++--- .../sqllogictest/test_files/explain.slt | 2 +- datafusion/sqllogictest/test_files/limit.slt | 85 ++++++++++ datafusion/sqllogictest/test_files/window.slt | 12 +- 4 files changed, 232 insertions(+), 26 deletions(-) diff --git a/datafusion/physical-plan/src/limit.rs b/datafusion/physical-plan/src/limit.rs index 945dad16b794..c8427f9bc2c6 100644 --- a/datafusion/physical-plan/src/limit.rs +++ b/datafusion/physical-plan/src/limit.rs @@ -188,21 +188,11 @@ impl ExecutionPlan for GlobalLimitExec { fn statistics(&self) -> Result { let input_stats = self.input.statistics()?; let skip = self.skip; - // the maximum row number needs to be fetched - let max_row_num = self - .fetch - .map(|fetch| { - if fetch >= usize::MAX - skip { - usize::MAX - } else { - fetch + skip - } - }) - .unwrap_or(usize::MAX); let col_stats = Statistics::unknown_column(&self.schema()); + let fetch = self.fetch.unwrap_or(usize::MAX); - let fetched_row_number_stats = Statistics { - num_rows: Precision::Exact(max_row_num), + let mut fetched_row_number_stats = Statistics { + num_rows: Precision::Exact(fetch), column_statistics: col_stats.clone(), total_byte_size: Precision::Absent, }; @@ -218,23 +208,55 @@ impl ExecutionPlan for GlobalLimitExec { } => { if nr <= skip { // if all input data will be skipped, return 0 - Statistics { + let mut skip_all_rows_stats = Statistics { num_rows: Precision::Exact(0), column_statistics: col_stats, total_byte_size: Precision::Absent, + }; + if !input_stats.num_rows.is_exact().unwrap_or(false) { + // The input stats are inexact, so the output stats must be too. + skip_all_rows_stats = skip_all_rows_stats.into_inexact(); } - } else if nr <= max_row_num { - // if the input does not reach the "fetch" globally, return input stats + skip_all_rows_stats + } else if nr <= fetch && self.skip == 0 { + // if the input does not reach the "fetch" globally, and "skip" is zero + // (meaning the input and output are identical), return input stats. + // Can input_stats still be used, but adjusted, in the "skip != 0" case? input_stats + } else if nr - skip <= fetch { + // after "skip" input rows are skipped, the remaining rows are less than or equal to the + // "fetch" values, so `num_rows` must equal the remaining rows + let remaining_rows: usize = nr - skip; + let mut skip_some_rows_stats = Statistics { + num_rows: Precision::Exact(remaining_rows), + column_statistics: col_stats.clone(), + total_byte_size: Precision::Absent, + }; + if !input_stats.num_rows.is_exact().unwrap_or(false) { + // The input stats are inexact, so the output stats must be too. + skip_some_rows_stats = skip_some_rows_stats.into_inexact(); + } + skip_some_rows_stats } else { - // if the input is greater than the "fetch", the num_row will be the "fetch", + // if the input is greater than "fetch+skip", the num_rows will be the "fetch", // but we won't be able to predict the other statistics + if !input_stats.num_rows.is_exact().unwrap_or(false) + || self.fetch.is_none() + { + // If the input stats are inexact, the output stats must be too. + // If the fetch value is `usize::MAX` because no LIMIT was specified, + // we also can't represent it as an exact value. + fetched_row_number_stats = + fetched_row_number_stats.into_inexact(); + } fetched_row_number_stats } } _ => { - // the result output row number will always be no greater than the limit number - fetched_row_number_stats + // The result output `num_rows` will always be no greater than the limit number. + // Should `num_rows` be marked as `Absent` here when the `fetch` value is large, + // as the actual `num_rows` may be far away from the `fetch` value? + fetched_row_number_stats.into_inexact() } }; Ok(stats) @@ -552,7 +574,10 @@ mod tests { use crate::common::collect; use crate::{common, test}; + use crate::aggregates::{AggregateExec, AggregateMode, PhysicalGroupBy}; use arrow_schema::Schema; + use datafusion_physical_expr::expressions::col; + use datafusion_physical_expr::PhysicalExpr; #[tokio::test] async fn limit() -> Result<()> { @@ -712,7 +737,7 @@ mod tests { } #[tokio::test] - async fn skip_3_fetch_10() -> Result<()> { + async fn skip_3_fetch_10_stats() -> Result<()> { // there are total of 100 rows, we skipped 3 rows (offset = 3) let row_count = skip_and_fetch(3, Some(10)).await?; assert_eq!(row_count, 10); @@ -748,7 +773,58 @@ mod tests { assert_eq!(row_count, Precision::Exact(10)); let row_count = row_number_statistics_for_global_limit(5, Some(10)).await?; - assert_eq!(row_count, Precision::Exact(15)); + assert_eq!(row_count, Precision::Exact(10)); + + let row_count = row_number_statistics_for_global_limit(400, Some(10)).await?; + assert_eq!(row_count, Precision::Exact(0)); + + let row_count = row_number_statistics_for_global_limit(398, Some(10)).await?; + assert_eq!(row_count, Precision::Exact(2)); + + let row_count = row_number_statistics_for_global_limit(398, Some(1)).await?; + assert_eq!(row_count, Precision::Exact(1)); + + let row_count = row_number_statistics_for_global_limit(398, None).await?; + assert_eq!(row_count, Precision::Exact(2)); + + let row_count = + row_number_statistics_for_global_limit(0, Some(usize::MAX)).await?; + assert_eq!(row_count, Precision::Exact(400)); + + let row_count = + row_number_statistics_for_global_limit(398, Some(usize::MAX)).await?; + assert_eq!(row_count, Precision::Exact(2)); + + let row_count = + row_number_inexact_statistics_for_global_limit(0, Some(10)).await?; + assert_eq!(row_count, Precision::Inexact(10)); + + let row_count = + row_number_inexact_statistics_for_global_limit(5, Some(10)).await?; + assert_eq!(row_count, Precision::Inexact(10)); + + let row_count = + row_number_inexact_statistics_for_global_limit(400, Some(10)).await?; + assert_eq!(row_count, Precision::Inexact(0)); + + let row_count = + row_number_inexact_statistics_for_global_limit(398, Some(10)).await?; + assert_eq!(row_count, Precision::Inexact(2)); + + let row_count = + row_number_inexact_statistics_for_global_limit(398, Some(1)).await?; + assert_eq!(row_count, Precision::Inexact(1)); + + let row_count = row_number_inexact_statistics_for_global_limit(398, None).await?; + assert_eq!(row_count, Precision::Inexact(2)); + + let row_count = + row_number_inexact_statistics_for_global_limit(0, Some(usize::MAX)).await?; + assert_eq!(row_count, Precision::Inexact(400)); + + let row_count = + row_number_inexact_statistics_for_global_limit(398, Some(usize::MAX)).await?; + assert_eq!(row_count, Precision::Inexact(2)); Ok(()) } @@ -776,6 +852,47 @@ mod tests { Ok(offset.statistics()?.num_rows) } + pub fn build_group_by( + input_schema: &SchemaRef, + columns: Vec, + ) -> PhysicalGroupBy { + let mut group_by_expr: Vec<(Arc, String)> = vec![]; + for column in columns.iter() { + group_by_expr.push((col(column, input_schema).unwrap(), column.to_string())); + } + PhysicalGroupBy::new_single(group_by_expr.clone()) + } + + async fn row_number_inexact_statistics_for_global_limit( + skip: usize, + fetch: Option, + ) -> Result> { + let num_partitions = 4; + let csv = test::scan_partitioned(num_partitions); + + assert_eq!(csv.output_partitioning().partition_count(), num_partitions); + + // Adding a "GROUP BY i" changes the input stats from Exact to Inexact. + let agg = AggregateExec::try_new( + AggregateMode::Final, + build_group_by(&csv.schema().clone(), vec!["i".to_string()]), + vec![], + vec![None], + vec![None], + csv.clone(), + csv.schema().clone(), + )?; + let agg_exec: Arc = Arc::new(agg); + + let offset = GlobalLimitExec::new( + Arc::new(CoalescePartitionsExec::new(agg_exec)), + skip, + fetch, + ); + + Ok(offset.statistics()?.num_rows) + } + async fn row_number_statistics_for_local_limit( num_partitions: usize, fetch: usize, diff --git a/datafusion/sqllogictest/test_files/explain.slt b/datafusion/sqllogictest/test_files/explain.slt index 066a31590ccd..40a6d4357488 100644 --- a/datafusion/sqllogictest/test_files/explain.slt +++ b/datafusion/sqllogictest/test_files/explain.slt @@ -273,7 +273,7 @@ query TT EXPLAIN SELECT a, b, c FROM simple_explain_test limit 10; ---- physical_plan -GlobalLimitExec: skip=0, fetch=10, statistics=[Rows=Exact(10), Bytes=Absent] +GlobalLimitExec: skip=0, fetch=10, statistics=[Rows=Inexact(10), Bytes=Absent] --CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/example.csv]]}, projection=[a, b, c], limit=10, has_header=true, statistics=[Rows=Absent, Bytes=Absent] # Parquet scan with statistics collected diff --git a/datafusion/sqllogictest/test_files/limit.slt b/datafusion/sqllogictest/test_files/limit.slt index 253ca8f335af..21248ddbd8d7 100644 --- a/datafusion/sqllogictest/test_files/limit.slt +++ b/datafusion/sqllogictest/test_files/limit.slt @@ -294,6 +294,91 @@ query T SELECT c1 FROM aggregate_test_100 LIMIT 1 OFFSET 101 ---- +# +# global limit statistics test +# + +statement ok +CREATE TABLE IF NOT EXISTS t1 (a INT) AS VALUES(1),(2),(3),(4),(5),(6),(7),(8),(9),(10); + +# The aggregate does not need to be computed because the input statistics are exact and +# the number of rows is less than the skip value (OFFSET). +query TT +EXPLAIN SELECT COUNT(*) FROM (SELECT a FROM t1 LIMIT 3 OFFSET 11); +---- +logical_plan +Aggregate: groupBy=[[]], aggr=[[COUNT(UInt8(1)) AS COUNT(*)]] +--Limit: skip=11, fetch=3 +----TableScan: t1 projection=[], fetch=14 +physical_plan +ProjectionExec: expr=[0 as COUNT(*)] +--EmptyExec: produce_one_row=true + +query I +SELECT COUNT(*) FROM (SELECT a FROM t1 LIMIT 3 OFFSET 11); +---- +0 + +# The aggregate does not need to be computed because the input statistics are exact and +# the number of rows is less than or equal to the the "fetch+skip" value (LIMIT+OFFSET). +query TT +EXPLAIN SELECT COUNT(*) FROM (SELECT a FROM t1 LIMIT 3 OFFSET 8); +---- +logical_plan +Aggregate: groupBy=[[]], aggr=[[COUNT(UInt8(1)) AS COUNT(*)]] +--Limit: skip=8, fetch=3 +----TableScan: t1 projection=[], fetch=11 +physical_plan +ProjectionExec: expr=[2 as COUNT(*)] +--EmptyExec: produce_one_row=true + +query I +SELECT COUNT(*) FROM (SELECT a FROM t1 LIMIT 3 OFFSET 8); +---- +2 + +# The aggregate does not need to be computed because the input statistics are exact and +# an OFFSET, but no LIMIT, is specified. +query TT +EXPLAIN SELECT COUNT(*) FROM (SELECT a FROM t1 OFFSET 8); +---- +logical_plan +Aggregate: groupBy=[[]], aggr=[[COUNT(UInt8(1)) AS COUNT(*)]] +--Limit: skip=8, fetch=None +----TableScan: t1 projection=[] +physical_plan +ProjectionExec: expr=[2 as COUNT(*)] +--EmptyExec: produce_one_row=true + +query I +SELECT COUNT(*) FROM (SELECT a FROM t1 LIMIT 3 OFFSET 8); +---- +2 + +# The aggregate needs to be computed because the input statistics are inexact. +query TT +EXPLAIN SELECT COUNT(*) FROM (SELECT a FROM t1 WHERE a > 3 LIMIT 3 OFFSET 6); +---- +logical_plan +Aggregate: groupBy=[[]], aggr=[[COUNT(UInt8(1)) AS COUNT(*)]] +--Limit: skip=6, fetch=3 +----Filter: t1.a > Int32(3) +------TableScan: t1 projection=[a] +physical_plan +AggregateExec: mode=Final, gby=[], aggr=[COUNT(*)] +--CoalescePartitionsExec +----AggregateExec: mode=Partial, gby=[], aggr=[COUNT(*)] +------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +--------GlobalLimitExec: skip=6, fetch=3 +----------CoalesceBatchesExec: target_batch_size=8192 +------------FilterExec: a@0 > 3 +--------------MemoryExec: partitions=1, partition_sizes=[1] + +query I +SELECT COUNT(*) FROM (SELECT a FROM t1 WHERE a > 3 LIMIT 3 OFFSET 6); +---- +1 + ######## # Clean up after the test ######## diff --git a/datafusion/sqllogictest/test_files/window.slt b/datafusion/sqllogictest/test_files/window.slt index c7060433d999..2eb0576d559b 100644 --- a/datafusion/sqllogictest/test_files/window.slt +++ b/datafusion/sqllogictest/test_files/window.slt @@ -2010,10 +2010,14 @@ Projection: ARRAY_AGG(aggregate_test_100.c13) AS array_agg1 --------TableScan: aggregate_test_100 projection=[c13] physical_plan ProjectionExec: expr=[ARRAY_AGG(aggregate_test_100.c13)@0 as array_agg1] ---AggregateExec: mode=Single, gby=[], aggr=[ARRAY_AGG(aggregate_test_100.c13)] -----GlobalLimitExec: skip=0, fetch=1 -------SortExec: TopK(fetch=1), expr=[c13@0 ASC NULLS LAST] ---------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c13], has_header=true +--AggregateExec: mode=Final, gby=[], aggr=[ARRAY_AGG(aggregate_test_100.c13)] +----CoalescePartitionsExec +------AggregateExec: mode=Partial, gby=[], aggr=[ARRAY_AGG(aggregate_test_100.c13)] +--------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +----------GlobalLimitExec: skip=0, fetch=1 +------------SortExec: TopK(fetch=1), expr=[c13@0 ASC NULLS LAST] +--------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c13], has_header=true + query ? SELECT ARRAY_AGG(c13) as array_agg1 FROM (SELECT * FROM aggregate_test_100 ORDER BY c13 LIMIT 1) From 56f6437317e02cd6f8e6ffd52abe5fed202bd090 Mon Sep 17 00:00:00 2001 From: Alex Huang Date: Tue, 7 Nov 2023 15:09:16 +0100 Subject: [PATCH 199/572] feat: Support determining extensions from names like `foo.parquet.snappy` as well as `foo.parquet` (#7972) * feat: read files based on the file extention * fix: some the file extension might be started with . and some not * fix: rename extention to extension * chore: use exec_err * chore: rename extention to extension * chore: rename extention to extension * chore: simplify the code * fix: check table is empty * ci: fix test * fix: add err info * refactor: extract the logic to infer_types * fix: add tests for different extensions * fix: ci clippy * fix: add more tests * fix: simplify the logic * fix: ci --- datafusion/core/src/execution/context/mod.rs | 17 +++ .../core/src/execution/context/parquet.rs | 123 ++++++++++++++++++ 2 files changed, 140 insertions(+) diff --git a/datafusion/core/src/execution/context/mod.rs b/datafusion/core/src/execution/context/mod.rs index d523c39ee01e..9c500ec07293 100644 --- a/datafusion/core/src/execution/context/mod.rs +++ b/datafusion/core/src/execution/context/mod.rs @@ -849,6 +849,23 @@ impl SessionContext { let table_paths = table_paths.to_urls()?; let session_config = self.copied_config(); let listing_options = options.to_listing_options(&session_config); + + let option_extension = listing_options.file_extension.clone(); + + if table_paths.is_empty() { + return exec_err!("No table paths were provided"); + } + + // check if the file extension matches the expected extension + for path in &table_paths { + let file_name = path.prefix().filename().unwrap_or_default(); + if !path.as_str().ends_with(&option_extension) && file_name.contains('.') { + return exec_err!( + "File '{file_name}' does not match the expected extension '{option_extension}'" + ); + } + } + let resolved_schema = options .get_resolved_schema(&session_config, self.state(), table_paths[0].clone()) .await?; diff --git a/datafusion/core/src/execution/context/parquet.rs b/datafusion/core/src/execution/context/parquet.rs index dc202b9903f5..ef1f0143543d 100644 --- a/datafusion/core/src/execution/context/parquet.rs +++ b/datafusion/core/src/execution/context/parquet.rs @@ -74,6 +74,11 @@ impl SessionContext { mod tests { use async_trait::async_trait; + use crate::arrow::array::{Float32Array, Int32Array}; + use crate::arrow::datatypes::{DataType, Field, Schema}; + use crate::arrow::record_batch::RecordBatch; + use crate::dataframe::DataFrameWriteOptions; + use crate::parquet::basic::Compression; use crate::test_util::parquet_test_data; use super::*; @@ -132,6 +137,124 @@ mod tests { Ok(()) } + #[tokio::test] + async fn read_from_different_file_extension() -> Result<()> { + let ctx = SessionContext::new(); + + // Make up a new dataframe. + let write_df = ctx.read_batch(RecordBatch::try_new( + Arc::new(Schema::new(vec![ + Field::new("purchase_id", DataType::Int32, false), + Field::new("price", DataType::Float32, false), + Field::new("quantity", DataType::Int32, false), + ])), + vec![ + Arc::new(Int32Array::from(vec![1, 2, 3, 4, 5])), + Arc::new(Float32Array::from(vec![1.12, 3.40, 2.33, 9.10, 6.66])), + Arc::new(Int32Array::from(vec![1, 3, 2, 4, 3])), + ], + )?)?; + + // Write the dataframe to a parquet file named 'output1.parquet' + write_df + .clone() + .write_parquet( + "output1.parquet", + DataFrameWriteOptions::new().with_single_file_output(true), + Some( + WriterProperties::builder() + .set_compression(Compression::SNAPPY) + .build(), + ), + ) + .await?; + + // Write the dataframe to a parquet file named 'output2.parquet.snappy' + write_df + .clone() + .write_parquet( + "output2.parquet.snappy", + DataFrameWriteOptions::new().with_single_file_output(true), + Some( + WriterProperties::builder() + .set_compression(Compression::SNAPPY) + .build(), + ), + ) + .await?; + + // Write the dataframe to a parquet file named 'output3.parquet.snappy.parquet' + write_df + .write_parquet( + "output3.parquet.snappy.parquet", + DataFrameWriteOptions::new().with_single_file_output(true), + Some( + WriterProperties::builder() + .set_compression(Compression::SNAPPY) + .build(), + ), + ) + .await?; + + // Read the dataframe from 'output1.parquet' with the default file extension. + let read_df = ctx + .read_parquet( + "output1.parquet", + ParquetReadOptions { + ..Default::default() + }, + ) + .await?; + + let results = read_df.collect().await?; + let total_rows: usize = results.iter().map(|rb| rb.num_rows()).sum(); + assert_eq!(total_rows, 5); + + // Read the dataframe from 'output2.parquet.snappy' with the correct file extension. + let read_df = ctx + .read_parquet( + "output2.parquet.snappy", + ParquetReadOptions { + file_extension: "snappy", + ..Default::default() + }, + ) + .await?; + let results = read_df.collect().await?; + let total_rows: usize = results.iter().map(|rb| rb.num_rows()).sum(); + assert_eq!(total_rows, 5); + + // Read the dataframe from 'output3.parquet.snappy.parquet' with the wrong file extension. + let read_df = ctx + .read_parquet( + "output2.parquet.snappy", + ParquetReadOptions { + ..Default::default() + }, + ) + .await; + + assert_eq!( + read_df.unwrap_err().strip_backtrace(), + "Execution error: File 'output2.parquet.snappy' does not match the expected extension '.parquet'" + ); + + // Read the dataframe from 'output3.parquet.snappy.parquet' with the correct file extension. + let read_df = ctx + .read_parquet( + "output3.parquet.snappy.parquet", + ParquetReadOptions { + ..Default::default() + }, + ) + .await?; + + let results = read_df.collect().await?; + let total_rows: usize = results.iter().map(|rb| rb.num_rows()).sum(); + assert_eq!(total_rows, 5); + Ok(()) + } + // Test for compilation error when calling read_* functions from an #[async_trait] function. // See https://github.com/apache/arrow-datafusion/issues/1154 #[async_trait] From f3c9009e50fdb5811c522bfb07ba29ac04cd1d22 Mon Sep 17 00:00:00 2001 From: Liang-Chi Hsieh Date: Tue, 7 Nov 2023 06:54:16 -0800 Subject: [PATCH 200/572] Use FairSpillPool for TaskContext with spillable config (#8072) --- datafusion/physical-plan/src/aggregates/mod.rs | 8 ++++++-- 1 file changed, 6 insertions(+), 2 deletions(-) diff --git a/datafusion/physical-plan/src/aggregates/mod.rs b/datafusion/physical-plan/src/aggregates/mod.rs index 8dab38bc5f0e..9cbf12aeeb88 100644 --- a/datafusion/physical-plan/src/aggregates/mod.rs +++ b/datafusion/physical-plan/src/aggregates/mod.rs @@ -1171,6 +1171,7 @@ mod tests { AggregateExpr, EquivalenceProperties, PhysicalExpr, PhysicalSortExpr, }; + use datafusion_execution::memory_pool::FairSpillPool; use futures::{FutureExt, Stream}; // Generate a schema which consists of 5 columns (a, b, c, d, e) @@ -1271,8 +1272,11 @@ mod tests { fn new_spill_ctx(batch_size: usize, max_memory: usize) -> Arc { let session_config = SessionConfig::new().with_batch_size(batch_size); let runtime = Arc::new( - RuntimeEnv::new(RuntimeConfig::default().with_memory_limit(max_memory, 1.0)) - .unwrap(), + RuntimeEnv::new( + RuntimeConfig::default() + .with_memory_pool(Arc::new(FairSpillPool::new(max_memory))), + ) + .unwrap(), ); let task_ctx = TaskContext::default() .with_session_config(session_config) From 0506a5cff2c61f289e805d9ecd222ce2f79e4097 Mon Sep 17 00:00:00 2001 From: Andrew Lamb Date: Tue, 7 Nov 2023 13:56:13 -0500 Subject: [PATCH 201/572] Minor: Improve HashJoinStream docstrings (#8070) MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit * Minor: Improve HashJoinStream docstrings * fix comments * Update datafusion/physical-plan/src/joins/hash_join.rs Co-authored-by: comphead * Update datafusion/physical-plan/src/joins/hash_join.rs Co-authored-by: comphead --------- Co-authored-by: Daniël Heres Co-authored-by: comphead --- .../physical-plan/src/joins/hash_join.rs | 121 +++++++++++------- 1 file changed, 74 insertions(+), 47 deletions(-) diff --git a/datafusion/physical-plan/src/joins/hash_join.rs b/datafusion/physical-plan/src/joins/hash_join.rs index 0e2d552b543b..1a2db87d98a2 100644 --- a/datafusion/physical-plan/src/joins/hash_join.rs +++ b/datafusion/physical-plan/src/joins/hash_join.rs @@ -206,7 +206,7 @@ pub struct HashJoinExec { pub join_type: JoinType, /// The output schema for the join schema: SchemaRef, - /// Build-side data + /// Future that consumes left input and builds the hash table left_fut: OnceAsync, /// Shared the `RandomState` for the hashing algorithm random_state: RandomState, @@ -747,27 +747,38 @@ where Ok(()) } -/// A stream that issues [RecordBatch]es as they arrive from the right of the join. +/// [`Stream`] for [`HashJoinExec`] that does the actual join. +/// +/// This stream: +/// +/// 1. Reads the entire left input (build) and constructs a hash table +/// +/// 2. Streams [RecordBatch]es as they arrive from the right input (probe) and joins +/// them with the contents of the hash table struct HashJoinStream { /// Input schema schema: Arc, - /// columns from the left + /// equijoin columns from the left (build side) on_left: Vec, - /// columns from the right used to compute the hash + /// equijoin columns from the right (probe side) on_right: Vec, - /// join filter + /// optional join filter filter: Option, - /// type of the join + /// type of the join (left, right, semi, etc) join_type: JoinType, - /// future for data from left side + /// future which builds hash table from left side left_fut: OnceFut, - /// Keeps track of the left side rows whether they are visited + /// Which left (probe) side rows have been matches while creating output. + /// For some OUTER joins, we need to know which rows have not been matched + /// to produce the correct. visited_left_side: Option, - /// right + /// right (probe) input right: SendableRecordBatchStream, /// Random state used for hashing initialization random_state: RandomState, - /// There is nothing to process anymore and left side is processed in case of left join + /// The join output is complete. For outer joins, this is used to + /// distinguish when the input stream is exhausted and when any unmatched + /// rows are output. is_exhausted: bool, /// Metrics join_metrics: BuildProbeJoinMetrics, @@ -785,37 +796,51 @@ impl RecordBatchStream for HashJoinStream { } } -// Returns build/probe indices satisfying the equality condition. -// On LEFT.b1 = RIGHT.b2 -// LEFT Table: -// a1 b1 c1 -// 1 1 10 -// 3 3 30 -// 5 5 50 -// 7 7 70 -// 9 8 90 -// 11 8 110 -// 13 10 130 -// RIGHT Table: -// a2 b2 c2 -// 2 2 20 -// 4 4 40 -// 6 6 60 -// 8 8 80 -// 10 10 100 -// 12 10 120 -// The result is -// "+----+----+-----+----+----+-----+", -// "| a1 | b1 | c1 | a2 | b2 | c2 |", -// "+----+----+-----+----+----+-----+", -// "| 9 | 8 | 90 | 8 | 8 | 80 |", -// "| 11 | 8 | 110 | 8 | 8 | 80 |", -// "| 13 | 10 | 130 | 10 | 10 | 100 |", -// "| 13 | 10 | 130 | 12 | 10 | 120 |", -// "+----+----+-----+----+----+-----+" -// And the result of build and probe indices are: -// Build indices: 4, 5, 6, 6 -// Probe indices: 3, 3, 4, 5 +/// Returns build/probe indices satisfying the equality condition. +/// +/// # Example +/// +/// For `LEFT.b1 = RIGHT.b2`: +/// LEFT Table: +/// ```text +/// a1 b1 c1 +/// 1 1 10 +/// 3 3 30 +/// 5 5 50 +/// 7 7 70 +/// 9 8 90 +/// 11 8 110 +/// 13 10 130 +/// ``` +/// +/// RIGHT Table: +/// ```text +/// a2 b2 c2 +/// 2 2 20 +/// 4 4 40 +/// 6 6 60 +/// 8 8 80 +/// 10 10 100 +/// 12 10 120 +/// ``` +/// +/// The result is +/// ```text +/// "+----+----+-----+----+----+-----+", +/// "| a1 | b1 | c1 | a2 | b2 | c2 |", +/// "+----+----+-----+----+----+-----+", +/// "| 9 | 8 | 90 | 8 | 8 | 80 |", +/// "| 11 | 8 | 110 | 8 | 8 | 80 |", +/// "| 13 | 10 | 130 | 10 | 10 | 100 |", +/// "| 13 | 10 | 130 | 12 | 10 | 120 |", +/// "+----+----+-----+----+----+-----+" +/// ``` +/// +/// And the result of build and probe indices are: +/// ```text +/// Build indices: 4, 5, 6, 6 +/// Probe indices: 3, 3, 4, 5 +/// ``` #[allow(clippy::too_many_arguments)] pub fn build_equal_condition_join_indices( build_hashmap: &T, @@ -1003,13 +1028,14 @@ impl HashJoinStream { cx: &mut std::task::Context<'_>, ) -> Poll>> { let build_timer = self.join_metrics.build_time.timer(); + // build hash table from left (build) side, if not yet done let left_data = match ready!(self.left_fut.get(cx)) { Ok(left_data) => left_data, Err(e) => return Poll::Ready(Some(Err(e))), }; build_timer.done(); - // Reserving memory for visited_left_side bitmap in case it hasn't been initialied yet + // Reserving memory for visited_left_side bitmap in case it hasn't been initialized yet // and join_type requires to store it if self.visited_left_side.is_none() && need_produce_result_in_final(self.join_type) @@ -1024,11 +1050,11 @@ impl HashJoinStream { let visited_left_side = self.visited_left_side.get_or_insert_with(|| { let num_rows = left_data.1.num_rows(); if need_produce_result_in_final(self.join_type) { - // these join type need the bitmap to identify which row has be matched or unmatched. - // For the `left semi` join, need to use the bitmap to produce the matched row in the left side - // For the `left` join, need to use the bitmap to produce the unmatched row in the left side with null - // For the `left anti` join, need to use the bitmap to produce the unmatched row in the left side - // For the `full` join, need to use the bitmap to produce the unmatched row in the left side with null + // Some join types need to track which row has be matched or unmatched: + // `left semi` join: need to use the bitmap to produce the matched row in the left side + // `left` join: need to use the bitmap to produce the unmatched row in the left side with null + // `left anti` join: need to use the bitmap to produce the unmatched row in the left side + // `full` join: need to use the bitmap to produce the unmatched row in the left side with null let mut buffer = BooleanBufferBuilder::new(num_rows); buffer.append_n(num_rows, false); buffer @@ -1037,6 +1063,7 @@ impl HashJoinStream { } }); let mut hashes_buffer = vec![]; + // get next right (probe) input batch self.right .poll_next_unpin(cx) .map(|maybe_batch| match maybe_batch { From 724bafd4de98eff8d6ffd67942d29d0f9faf2aa3 Mon Sep 17 00:00:00 2001 From: Edmondo Porcu Date: Tue, 7 Nov 2023 23:01:44 -0800 Subject: [PATCH 202/572] Fixing broken link (#8085) * Fixing broken link * Update docs/source/contributor-guide/index.md Thanks for spotting this as well Co-authored-by: Liang-Chi Hsieh --------- Co-authored-by: Liang-Chi Hsieh --- docs/source/contributor-guide/index.md | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/source/contributor-guide/index.md b/docs/source/contributor-guide/index.md index e42ab0dee07a..1a8b5e427087 100644 --- a/docs/source/contributor-guide/index.md +++ b/docs/source/contributor-guide/index.md @@ -221,8 +221,8 @@ Below is a checklist of what you need to do to add a new scalar function to Data - a new line in `signature` with the signature of the function (number and types of its arguments) - a new line in `create_physical_expr`/`create_physical_fun` mapping the built-in to the implementation - tests to the function. -- In [core/tests/sqllogictests/test_files](../../../datafusion/core/tests/sqllogictests/test_files), add new `sqllogictest` integration tests where the function is called through SQL against well known data and returns the expected result. - - Documentation for `sqllogictest` [here](../../../datafusion/core/tests/sqllogictests/README.md) +- In [sqllogictest/test_files](../../../datafusion/sqllogictest/test_files), add new `sqllogictest` integration tests where the function is called through SQL against well known data and returns the expected result. + - Documentation for `sqllogictest` [here](../../../datafusion/sqllogictest/README.md) - In [expr/src/expr_fn.rs](../../../datafusion/expr/src/expr_fn.rs), add: - a new entry of the `unary_scalar_expr!` macro for the new function. - Add SQL reference documentation [here](../../../docs/source/user-guide/sql/scalar_functions.md) From 34463822b85bf0e5ddceb23970c89339f103c494 Mon Sep 17 00:00:00 2001 From: Jonah Gao Date: Thu, 9 Nov 2023 00:24:57 +0800 Subject: [PATCH 203/572] fix: DataFusion suggests invalid functions (#8083) * fix: DataFusion suggests invalid functions * update test * Add test for BuiltInWindowFunction --- datafusion/expr/src/aggregate_function.rs | 31 ++++++++++++++++--- datafusion/expr/src/built_in_function.rs | 3 +- datafusion/expr/src/window_function.rs | 15 +++++++++ .../sqllogictest/test_files/functions.slt | 4 +++ 4 files changed, 47 insertions(+), 6 deletions(-) diff --git a/datafusion/expr/src/aggregate_function.rs b/datafusion/expr/src/aggregate_function.rs index eaf4ff5ad806..ea0b01825170 100644 --- a/datafusion/expr/src/aggregate_function.rs +++ b/datafusion/expr/src/aggregate_function.rs @@ -116,13 +116,13 @@ impl AggregateFunction { ArrayAgg => "ARRAY_AGG", FirstValue => "FIRST_VALUE", LastValue => "LAST_VALUE", - Variance => "VARIANCE", - VariancePop => "VARIANCE_POP", + Variance => "VAR", + VariancePop => "VAR_POP", Stddev => "STDDEV", StddevPop => "STDDEV_POP", - Covariance => "COVARIANCE", - CovariancePop => "COVARIANCE_POP", - Correlation => "CORRELATION", + Covariance => "COVAR", + CovariancePop => "COVAR_POP", + Correlation => "CORR", RegrSlope => "REGR_SLOPE", RegrIntercept => "REGR_INTERCEPT", RegrCount => "REGR_COUNT", @@ -411,3 +411,24 @@ impl AggregateFunction { } } } + +#[cfg(test)] +mod tests { + use super::*; + use strum::IntoEnumIterator; + + #[test] + // Test for AggregateFuncion's Display and from_str() implementations. + // For each variant in AggregateFuncion, it converts the variant to a string + // and then back to a variant. The test asserts that the original variant and + // the reconstructed variant are the same. This assertion is also necessary for + // function suggestion. See https://github.com/apache/arrow-datafusion/issues/8082 + fn test_display_and_from_str() { + for func_original in AggregateFunction::iter() { + let func_name = func_original.to_string(); + let func_from_str = + AggregateFunction::from_str(func_name.to_lowercase().as_str()).unwrap(); + assert_eq!(func_from_str, func_original); + } + } +} diff --git a/datafusion/expr/src/built_in_function.rs b/datafusion/expr/src/built_in_function.rs index 16187572c521..1ebd9cc0187a 100644 --- a/datafusion/expr/src/built_in_function.rs +++ b/datafusion/expr/src/built_in_function.rs @@ -1621,7 +1621,8 @@ mod tests { // Test for BuiltinScalarFunction's Display and from_str() implementations. // For each variant in BuiltinScalarFunction, it converts the variant to a string // and then back to a variant. The test asserts that the original variant and - // the reconstructed variant are the same. + // the reconstructed variant are the same. This assertion is also necessary for + // function suggestion. See https://github.com/apache/arrow-datafusion/issues/8082 fn test_display_and_from_str() { for (_, func_original) in name_to_function().iter() { let func_name = func_original.to_string(); diff --git a/datafusion/expr/src/window_function.rs b/datafusion/expr/src/window_function.rs index e5b00c8f298b..463cceafeb6e 100644 --- a/datafusion/expr/src/window_function.rs +++ b/datafusion/expr/src/window_function.rs @@ -281,6 +281,7 @@ impl BuiltInWindowFunction { #[cfg(test)] mod tests { use super::*; + use strum::IntoEnumIterator; #[test] fn test_count_return_type() -> Result<()> { @@ -447,4 +448,18 @@ mod tests { ); assert_eq!(find_df_window_func("not_exist"), None) } + + #[test] + // Test for BuiltInWindowFunction's Display and from_str() implementations. + // For each variant in BuiltInWindowFunction, it converts the variant to a string + // and then back to a variant. The test asserts that the original variant and + // the reconstructed variant are the same. This assertion is also necessary for + // function suggestion. See https://github.com/apache/arrow-datafusion/issues/8082 + fn test_display_and_from_str() { + for func_original in BuiltInWindowFunction::iter() { + let func_name = func_original.to_string(); + let func_from_str = BuiltInWindowFunction::from_str(&func_name).unwrap(); + assert_eq!(func_from_str, func_original); + } + } } diff --git a/datafusion/sqllogictest/test_files/functions.slt b/datafusion/sqllogictest/test_files/functions.slt index e3e39ef6cc4c..2054752cc59c 100644 --- a/datafusion/sqllogictest/test_files/functions.slt +++ b/datafusion/sqllogictest/test_files/functions.slt @@ -494,6 +494,10 @@ SELECT counter(*) from test; statement error Did you mean 'STDDEV'? SELECT STDEV(v1) from test; +# Aggregate function +statement error Did you mean 'COVAR'? +SELECT COVARIA(1,1); + # Window function statement error Did you mean 'SUM'? SELECT v1, v2, SUMM(v2) OVER(ORDER BY v1) from test; From aefee03e114f56d806b7e97682e50adc87a50a7e Mon Sep 17 00:00:00 2001 From: Jay Zhan Date: Thu, 9 Nov 2023 00:29:33 +0800 Subject: [PATCH 204/572] Replace macro with function for `array_repeat` (#8071) * General array repeat Signed-off-by: jayzhan211 * cleanup Signed-off-by: jayzhan211 * cleanup Signed-off-by: jayzhan211 * cleanup Signed-off-by: jayzhan211 * add test Signed-off-by: jayzhan211 * add test Signed-off-by: jayzhan211 * done Signed-off-by: jayzhan211 * remove test Signed-off-by: jayzhan211 * add comment Signed-off-by: jayzhan211 * fm Signed-off-by: jayzhan211 --------- Signed-off-by: jayzhan211 --- .../physical-expr/src/array_expressions.rs | 312 +++++++----------- datafusion/sqllogictest/test_files/array.slt | 98 +++--- 2 files changed, 169 insertions(+), 241 deletions(-) diff --git a/datafusion/physical-expr/src/array_expressions.rs b/datafusion/physical-expr/src/array_expressions.rs index e296e9c96fad..64550aabf424 100644 --- a/datafusion/physical-expr/src/array_expressions.rs +++ b/datafusion/physical-expr/src/array_expressions.rs @@ -841,125 +841,6 @@ pub fn array_concat(args: &[ArrayRef]) -> Result { concat_internal(new_args.as_slice()) } -macro_rules! general_repeat { - ($ELEMENT:expr, $COUNT:expr, $ARRAY_TYPE:ident) => {{ - let mut offsets: Vec = vec![0]; - let mut values = - downcast_arg!(new_empty_array($ELEMENT.data_type()), $ARRAY_TYPE).clone(); - - let element_array = downcast_arg!($ELEMENT, $ARRAY_TYPE); - for (el, c) in element_array.iter().zip($COUNT.iter()) { - let last_offset: i32 = offsets.last().copied().ok_or_else(|| { - DataFusionError::Internal(format!("offsets should not be empty")) - })?; - match el { - Some(el) => { - let c = if c < Some(0) { 0 } else { c.unwrap() } as usize; - let repeated_array = - [Some(el.clone())].repeat(c).iter().collect::<$ARRAY_TYPE>(); - - values = downcast_arg!( - compute::concat(&[&values, &repeated_array])?.clone(), - $ARRAY_TYPE - ) - .clone(); - offsets.push(last_offset + repeated_array.len() as i32); - } - None => { - offsets.push(last_offset); - } - } - } - - let field = Arc::new(Field::new("item", $ELEMENT.data_type().clone(), true)); - - Arc::new(ListArray::try_new( - field, - OffsetBuffer::new(offsets.into()), - Arc::new(values), - None, - )?) - }}; -} - -macro_rules! general_repeat_list { - ($ELEMENT:expr, $COUNT:expr, $ARRAY_TYPE:ident) => {{ - let mut offsets: Vec = vec![0]; - let mut values = - downcast_arg!(new_empty_array($ELEMENT.data_type()), ListArray).clone(); - - let element_array = downcast_arg!($ELEMENT, ListArray); - for (el, c) in element_array.iter().zip($COUNT.iter()) { - let last_offset: i32 = offsets.last().copied().ok_or_else(|| { - DataFusionError::Internal(format!("offsets should not be empty")) - })?; - match el { - Some(el) => { - let c = if c < Some(0) { 0 } else { c.unwrap() } as usize; - let repeated_vec = vec![el; c]; - - let mut i: i32 = 0; - let mut repeated_offsets = vec![i]; - repeated_offsets.extend( - repeated_vec - .clone() - .into_iter() - .map(|a| { - i += a.len() as i32; - i - }) - .collect::>(), - ); - - let mut repeated_values = downcast_arg!( - new_empty_array(&element_array.value_type()), - $ARRAY_TYPE - ) - .clone(); - for repeated_list in repeated_vec { - repeated_values = downcast_arg!( - compute::concat(&[&repeated_values, &repeated_list])?, - $ARRAY_TYPE - ) - .clone(); - } - - let field = Arc::new(Field::new( - "item", - element_array.value_type().clone(), - true, - )); - let repeated_array = ListArray::try_new( - field, - OffsetBuffer::new(repeated_offsets.clone().into()), - Arc::new(repeated_values), - None, - )?; - - values = downcast_arg!( - compute::concat(&[&values, &repeated_array,])?.clone(), - ListArray - ) - .clone(); - offsets.push(last_offset + repeated_array.len() as i32); - } - None => { - offsets.push(last_offset); - } - } - } - - let field = Arc::new(Field::new("item", $ELEMENT.data_type().clone(), true)); - - Arc::new(ListArray::try_new( - field, - OffsetBuffer::new(offsets.into()), - Arc::new(values), - None, - )?) - }}; -} - /// Array_empty SQL function pub fn array_empty(args: &[ArrayRef]) -> Result { if args[0].as_any().downcast_ref::().is_some() { @@ -978,28 +859,136 @@ pub fn array_empty(args: &[ArrayRef]) -> Result { /// Array_repeat SQL function pub fn array_repeat(args: &[ArrayRef]) -> Result { let element = &args[0]; - let count = as_int64_array(&args[1])?; + let count_array = as_int64_array(&args[1])?; - let res = match element.data_type() { - DataType::List(field) => { - macro_rules! array_function { - ($ARRAY_TYPE:ident) => { - general_repeat_list!(element, count, $ARRAY_TYPE) - }; - } - call_array_function!(field.data_type(), true) + match element.data_type() { + DataType::List(_) => { + let list_array = as_list_array(element)?; + general_list_repeat(list_array, count_array) } - data_type => { - macro_rules! array_function { - ($ARRAY_TYPE:ident) => { - general_repeat!(element, count, $ARRAY_TYPE) - }; + _ => general_repeat(element, count_array), + } +} + +/// For each element of `array[i]` repeat `count_array[i]` times. +/// +/// Assumption for the input: +/// 1. `count[i] >= 0` +/// 2. `array.len() == count_array.len()` +/// +/// For example, +/// ```text +/// array_repeat( +/// [1, 2, 3], [2, 0, 1] => [[1, 1], [], [3]] +/// ) +/// ``` +fn general_repeat(array: &ArrayRef, count_array: &Int64Array) -> Result { + let data_type = array.data_type(); + let mut new_values = vec![]; + + let count_vec = count_array + .values() + .to_vec() + .iter() + .map(|x| *x as usize) + .collect::>(); + + for (row_index, &count) in count_vec.iter().enumerate() { + let repeated_array = if array.is_null(row_index) { + new_null_array(data_type, count) + } else { + let original_data = array.to_data(); + let capacity = Capacities::Array(count); + let mut mutable = + MutableArrayData::with_capacities(vec![&original_data], false, capacity); + + for _ in 0..count { + mutable.extend(0, row_index, row_index + 1); } - call_array_function!(data_type, false) - } - }; - Ok(res) + let data = mutable.freeze(); + arrow_array::make_array(data) + }; + new_values.push(repeated_array); + } + + let new_values: Vec<_> = new_values.iter().map(|a| a.as_ref()).collect(); + let values = arrow::compute::concat(&new_values)?; + + Ok(Arc::new(ListArray::try_new( + Arc::new(Field::new("item", data_type.to_owned(), true)), + OffsetBuffer::from_lengths(count_vec), + values, + None, + )?)) +} + +/// Handle List version of `general_repeat` +/// +/// For each element of `list_array[i]` repeat `count_array[i]` times. +/// +/// For example, +/// ```text +/// array_repeat( +/// [[1, 2, 3], [4, 5], [6]], [2, 0, 1] => [[[1, 2, 3], [1, 2, 3]], [], [[6]]] +/// ) +/// ``` +fn general_list_repeat( + list_array: &ListArray, + count_array: &Int64Array, +) -> Result { + let data_type = list_array.data_type(); + let value_type = list_array.value_type(); + let mut new_values = vec![]; + + let count_vec = count_array + .values() + .to_vec() + .iter() + .map(|x| *x as usize) + .collect::>(); + + for (list_array_row, &count) in list_array.iter().zip(count_vec.iter()) { + let list_arr = match list_array_row { + Some(list_array_row) => { + let original_data = list_array_row.to_data(); + let capacity = Capacities::Array(original_data.len() * count); + let mut mutable = MutableArrayData::with_capacities( + vec![&original_data], + false, + capacity, + ); + + for _ in 0..count { + mutable.extend(0, 0, original_data.len()); + } + + let data = mutable.freeze(); + let repeated_array = arrow_array::make_array(data); + + let list_arr = ListArray::try_new( + Arc::new(Field::new("item", value_type.clone(), true)), + OffsetBuffer::from_lengths(vec![original_data.len(); count]), + repeated_array, + None, + )?; + Arc::new(list_arr) as ArrayRef + } + None => new_null_array(data_type, count), + }; + new_values.push(list_arr); + } + + let lengths = new_values.iter().map(|a| a.len()).collect::>(); + let new_values: Vec<_> = new_values.iter().map(|a| a.as_ref()).collect(); + let values = arrow::compute::concat(&new_values)?; + + Ok(Arc::new(ListArray::try_new( + Arc::new(Field::new("item", data_type.to_owned(), true)), + OffsetBuffer::from_lengths(lengths), + values, + None, + )?)) } macro_rules! position { @@ -2925,55 +2914,6 @@ mod tests { ); } - #[test] - fn test_array_repeat() { - // array_repeat(3, 5) = [3, 3, 3, 3, 3] - let array = array_repeat(&[ - Arc::new(Int64Array::from_value(3, 1)), - Arc::new(Int64Array::from_value(5, 1)), - ]) - .expect("failed to initialize function array_repeat"); - let result = - as_list_array(&array).expect("failed to initialize function array_repeat"); - - assert_eq!(result.len(), 1); - assert_eq!( - &[3, 3, 3, 3, 3], - result - .value(0) - .as_any() - .downcast_ref::() - .unwrap() - .values() - ); - } - - #[test] - fn test_nested_array_repeat() { - // array_repeat([1, 2, 3, 4], 3) = [[1, 2, 3, 4], [1, 2, 3, 4], [1, 2, 3, 4]] - let element = return_array(); - let array = array_repeat(&[element, Arc::new(Int64Array::from_value(3, 1))]) - .expect("failed to initialize function array_repeat"); - let result = - as_list_array(&array).expect("failed to initialize function array_repeat"); - - assert_eq!(result.len(), 1); - let data = vec![ - Some(vec![Some(1), Some(2), Some(3), Some(4)]), - Some(vec![Some(1), Some(2), Some(3), Some(4)]), - Some(vec![Some(1), Some(2), Some(3), Some(4)]), - ]; - let expected = ListArray::from_iter_primitive::(data); - assert_eq!( - expected, - result - .value(0) - .as_any() - .downcast_ref::() - .unwrap() - .clone() - ); - } #[test] fn test_array_to_string() { // array_to_string([1, 2, 3, 4], ',') = 1,2,3,4 diff --git a/datafusion/sqllogictest/test_files/array.slt b/datafusion/sqllogictest/test_files/array.slt index b5601a22226c..85218efb5e14 100644 --- a/datafusion/sqllogictest/test_files/array.slt +++ b/datafusion/sqllogictest/test_files/array.slt @@ -1121,68 +1121,56 @@ select array_prepend(make_array(1, 11, 111), column1), array_prepend(column2, ma ## array_repeat (aliases: `list_repeat`) # array_repeat scalar function #1 -query ??? -select array_repeat(1, 5), array_repeat(3.14, 3), array_repeat('l', 4); ----- -[1, 1, 1, 1, 1] [3.14, 3.14, 3.14] [l, l, l, l] +query ???????? +select + array_repeat(1, 5), + array_repeat(3.14, 3), + array_repeat('l', 4), + array_repeat(null, 2), + list_repeat(-1, 5), + list_repeat(-3.14, 0), + list_repeat('rust', 4), + list_repeat(null, 0); +---- +[1, 1, 1, 1, 1] [3.14, 3.14, 3.14] [l, l, l, l] [, ] [-1, -1, -1, -1, -1] [] [rust, rust, rust, rust] [] # array_repeat scalar function #2 (element as list) -query ??? -select array_repeat([1], 5), array_repeat([1.1, 2.2, 3.3], 3), array_repeat([[1, 2], [3, 4]], 2); ----- -[[1], [1], [1], [1], [1]] [[1.1, 2.2, 3.3], [1.1, 2.2, 3.3], [1.1, 2.2, 3.3]] [[[1, 2], [3, 4]], [[1, 2], [3, 4]]] - -# list_repeat scalar function #3 (function alias: `array_repeat`) -query ??? -select list_repeat(1, 5), list_repeat(3.14, 3), list_repeat('l', 4); +query ???? +select + array_repeat([1], 5), + array_repeat([1.1, 2.2, 3.3], 3), + array_repeat([null, null], 3), + array_repeat([[1, 2], [3, 4]], 2); ---- -[1, 1, 1, 1, 1] [3.14, 3.14, 3.14] [l, l, l, l] +[[1], [1], [1], [1], [1]] [[1.1, 2.2, 3.3], [1.1, 2.2, 3.3], [1.1, 2.2, 3.3]] [[, ], [, ], [, ]] [[[1, 2], [3, 4]], [[1, 2], [3, 4]]] # array_repeat with columns #1 -query ? -select array_repeat(column4, column1) from values_without_nulls; ----- -[1.1] -[2.2, 2.2] -[3.3, 3.3, 3.3] -[4.4, 4.4, 4.4, 4.4] -[5.5, 5.5, 5.5, 5.5, 5.5] -[6.6, 6.6, 6.6, 6.6, 6.6, 6.6] -[7.7, 7.7, 7.7, 7.7, 7.7, 7.7, 7.7] -[8.8, 8.8, 8.8, 8.8, 8.8, 8.8, 8.8, 8.8] -[9.9, 9.9, 9.9, 9.9, 9.9, 9.9, 9.9, 9.9, 9.9] -# array_repeat with columns #2 (element as list) -query ? -select array_repeat(column1, column3) from arrays_values_without_nulls; ----- -[[1, 2, 3, 4, 5, 6, 7, 8, 9, 10]] -[[11, 12, 13, 14, 15, 16, 17, 18, 19, 20], [11, 12, 13, 14, 15, 16, 17, 18, 19, 20]] -[[21, 22, 23, 24, 25, 26, 27, 28, 29, 30], [21, 22, 23, 24, 25, 26, 27, 28, 29, 30], [21, 22, 23, 24, 25, 26, 27, 28, 29, 30]] -[[31, 32, 33, 34, 35, 26, 37, 38, 39, 40], [31, 32, 33, 34, 35, 26, 37, 38, 39, 40], [31, 32, 33, 34, 35, 26, 37, 38, 39, 40], [31, 32, 33, 34, 35, 26, 37, 38, 39, 40]] +statement ok +CREATE TABLE array_repeat_table +AS VALUES + (1, 1, 1.1, 'a', make_array(4, 5, 6)), + (2, null, null, null, null), + (3, 2, 2.2, 'rust', make_array(7)), + (0, 3, 3.3, 'datafusion', make_array(8, 9)); + +query ?????? +select + array_repeat(column2, column1), + array_repeat(column3, column1), + array_repeat(column4, column1), + array_repeat(column5, column1), + array_repeat(column2, 3), + array_repeat(make_array(1), column1) +from array_repeat_table; +---- +[1] [1.1] [a] [[4, 5, 6]] [1, 1, 1] [[1]] +[, ] [, ] [, ] [, ] [, , ] [[1], [1]] +[2, 2, 2] [2.2, 2.2, 2.2] [rust, rust, rust] [[7], [7], [7]] [2, 2, 2] [[1], [1], [1]] +[] [] [] [] [3, 3, 3] [] -# array_repeat with columns and scalars #1 -query ?? -select array_repeat(1, column1), array_repeat(column4, 3) from values_without_nulls; ----- -[1] [1.1, 1.1, 1.1] -[1, 1] [2.2, 2.2, 2.2] -[1, 1, 1] [3.3, 3.3, 3.3] -[1, 1, 1, 1] [4.4, 4.4, 4.4] -[1, 1, 1, 1, 1] [5.5, 5.5, 5.5] -[1, 1, 1, 1, 1, 1] [6.6, 6.6, 6.6] -[1, 1, 1, 1, 1, 1, 1] [7.7, 7.7, 7.7] -[1, 1, 1, 1, 1, 1, 1, 1] [8.8, 8.8, 8.8] -[1, 1, 1, 1, 1, 1, 1, 1, 1] [9.9, 9.9, 9.9] - -# array_repeat with columns and scalars #2 (element as list) -query ?? -select array_repeat([1], column3), array_repeat(column1, 3) from arrays_values_without_nulls; ----- -[[1]] [[1, 2, 3, 4, 5, 6, 7, 8, 9, 10], [1, 2, 3, 4, 5, 6, 7, 8, 9, 10], [1, 2, 3, 4, 5, 6, 7, 8, 9, 10]] -[[1], [1]] [[11, 12, 13, 14, 15, 16, 17, 18, 19, 20], [11, 12, 13, 14, 15, 16, 17, 18, 19, 20], [11, 12, 13, 14, 15, 16, 17, 18, 19, 20]] -[[1], [1], [1]] [[21, 22, 23, 24, 25, 26, 27, 28, 29, 30], [21, 22, 23, 24, 25, 26, 27, 28, 29, 30], [21, 22, 23, 24, 25, 26, 27, 28, 29, 30]] -[[1], [1], [1], [1]] [[31, 32, 33, 34, 35, 26, 37, 38, 39, 40], [31, 32, 33, 34, 35, 26, 37, 38, 39, 40], [31, 32, 33, 34, 35, 26, 37, 38, 39, 40]] +statement ok +drop table array_repeat_table; ## array_concat (aliases: `array_cat`, `list_concat`, `list_cat`) From 15d8c9bf48a56ae9de34d18becab13fd1942dc4a Mon Sep 17 00:00:00 2001 From: Huaijin Date: Thu, 9 Nov 2023 01:17:11 +0800 Subject: [PATCH 205/572] Minor: remove unnecessary projection in `single_distinct_to_group_by` rule (#8061) * Minor: remove unnecessary projection * fix ci --- .../src/single_distinct_to_groupby.rs | 109 +++++------------- .../sqllogictest/test_files/groupby.slt | 10 +- datafusion/sqllogictest/test_files/joins.slt | 47 ++++---- .../sqllogictest/test_files/tpch/q16.slt.part | 10 +- 4 files changed, 63 insertions(+), 113 deletions(-) diff --git a/datafusion/optimizer/src/single_distinct_to_groupby.rs b/datafusion/optimizer/src/single_distinct_to_groupby.rs index be76c069f0b7..548f00b4138a 100644 --- a/datafusion/optimizer/src/single_distinct_to_groupby.rs +++ b/datafusion/optimizer/src/single_distinct_to_groupby.rs @@ -22,13 +22,12 @@ use std::sync::Arc; use crate::optimizer::ApplyOrder; use crate::{OptimizerConfig, OptimizerRule}; -use datafusion_common::{DFSchema, Result}; +use datafusion_common::Result; use datafusion_expr::{ col, - expr::AggregateFunction, - logical_plan::{Aggregate, LogicalPlan, Projection}, - utils::columnize_expr, - Expr, ExprSchemable, + expr::{AggregateFunction, Alias}, + logical_plan::{Aggregate, LogicalPlan}, + Expr, }; use hashbrown::HashSet; @@ -153,7 +152,7 @@ impl OptimizerRule for SingleDistinctToGroupBy { // replace the distinct arg with alias let mut group_fields_set = HashSet::new(); - let new_aggr_exprs = aggr_expr + let outer_aggr_exprs = aggr_expr .iter() .map(|aggr_expr| match aggr_expr { Expr::AggregateFunction(AggregateFunction { @@ -169,12 +168,15 @@ impl OptimizerRule for SingleDistinctToGroupBy { args[0].clone().alias(SINGLE_DISTINCT_ALIAS), ); } - Ok(Expr::AggregateFunction(AggregateFunction::new( - fun.clone(), - vec![col(SINGLE_DISTINCT_ALIAS)], - false, // intentional to remove distinct here - filter.clone(), - order_by.clone(), + Ok(Expr::Alias(Alias::new( + Expr::AggregateFunction(AggregateFunction::new( + fun.clone(), + vec![col(SINGLE_DISTINCT_ALIAS)], + false, // intentional to remove distinct here + filter.clone(), + order_by.clone(), + )), + aggr_expr.display_name()?, ))) } _ => Ok(aggr_expr.clone()), @@ -182,60 +184,16 @@ impl OptimizerRule for SingleDistinctToGroupBy { .collect::>>()?; // construct the inner AggrPlan - let inner_fields = inner_group_exprs - .iter() - .map(|expr| expr.to_field(input.schema())) - .collect::>>()?; - let inner_schema = DFSchema::new_with_metadata( - inner_fields, - input.schema().metadata().clone(), - )?; let inner_agg = LogicalPlan::Aggregate(Aggregate::try_new( input.clone(), inner_group_exprs, Vec::new(), )?); - let outer_fields = outer_group_exprs - .iter() - .chain(new_aggr_exprs.iter()) - .map(|expr| expr.to_field(&inner_schema)) - .collect::>>()?; - let outer_aggr_schema = Arc::new(DFSchema::new_with_metadata( - outer_fields, - input.schema().metadata().clone(), - )?); - - // so the aggregates are displayed in the same way even after the rewrite - // this optimizer has two kinds of alias: - // - group_by aggr - // - aggr expr - let group_size = group_expr.len(); - let alias_expr = out_group_expr_with_alias - .into_iter() - .map(|(group_expr, original_field)| { - if let Some(name) = original_field { - group_expr.alias(name) - } else { - group_expr - } - }) - .chain(new_aggr_exprs.iter().enumerate().map(|(idx, expr)| { - let idx = idx + group_size; - let name = fields[idx].qualified_name(); - columnize_expr(expr.clone().alias(name), &outer_aggr_schema) - })) - .collect(); - - let outer_aggr = LogicalPlan::Aggregate(Aggregate::try_new( + Ok(Some(LogicalPlan::Aggregate(Aggregate::try_new( Arc::new(inner_agg), outer_group_exprs, - new_aggr_exprs, - )?); - - Ok(Some(LogicalPlan::Projection(Projection::try_new( - alias_expr, - Arc::new(outer_aggr), + outer_aggr_exprs, )?))) } else { Ok(None) @@ -299,10 +257,9 @@ mod tests { .build()?; // Should work - let expected = "Projection: COUNT(alias1) AS COUNT(DISTINCT test.b) [COUNT(DISTINCT test.b):Int64;N]\ - \n Aggregate: groupBy=[[]], aggr=[[COUNT(alias1)]] [COUNT(alias1):Int64;N]\ - \n Aggregate: groupBy=[[test.b AS alias1]], aggr=[[]] [alias1:UInt32]\ - \n TableScan: test [a:UInt32, b:UInt32, c:UInt32]"; + let expected = "Aggregate: groupBy=[[]], aggr=[[COUNT(alias1) AS COUNT(DISTINCT test.b)]] [COUNT(DISTINCT test.b):Int64;N]\ + \n Aggregate: groupBy=[[test.b AS alias1]], aggr=[[]] [alias1:UInt32]\ + \n TableScan: test [a:UInt32, b:UInt32, c:UInt32]"; assert_optimized_plan_equal(&plan, expected) } @@ -373,10 +330,9 @@ mod tests { .aggregate(Vec::::new(), vec![count_distinct(lit(2) * col("b"))])? .build()?; - let expected = "Projection: COUNT(alias1) AS COUNT(DISTINCT Int32(2) * test.b) [COUNT(DISTINCT Int32(2) * test.b):Int64;N]\ - \n Aggregate: groupBy=[[]], aggr=[[COUNT(alias1)]] [COUNT(alias1):Int64;N]\ - \n Aggregate: groupBy=[[Int32(2) * test.b AS alias1]], aggr=[[]] [alias1:Int32]\ - \n TableScan: test [a:UInt32, b:UInt32, c:UInt32]"; + let expected = "Aggregate: groupBy=[[]], aggr=[[COUNT(alias1) AS COUNT(DISTINCT Int32(2) * test.b)]] [COUNT(DISTINCT Int32(2) * test.b):Int64;N]\ + \n Aggregate: groupBy=[[Int32(2) * test.b AS alias1]], aggr=[[]] [alias1:Int32]\ + \n TableScan: test [a:UInt32, b:UInt32, c:UInt32]"; assert_optimized_plan_equal(&plan, expected) } @@ -390,10 +346,9 @@ mod tests { .build()?; // Should work - let expected = "Projection: test.a, COUNT(alias1) AS COUNT(DISTINCT test.b) [a:UInt32, COUNT(DISTINCT test.b):Int64;N]\ - \n Aggregate: groupBy=[[test.a]], aggr=[[COUNT(alias1)]] [a:UInt32, COUNT(alias1):Int64;N]\ - \n Aggregate: groupBy=[[test.a, test.b AS alias1]], aggr=[[]] [a:UInt32, alias1:UInt32]\ - \n TableScan: test [a:UInt32, b:UInt32, c:UInt32]"; + let expected = "Aggregate: groupBy=[[test.a]], aggr=[[COUNT(alias1) AS COUNT(DISTINCT test.b)]] [a:UInt32, COUNT(DISTINCT test.b):Int64;N]\ + \n Aggregate: groupBy=[[test.a, test.b AS alias1]], aggr=[[]] [a:UInt32, alias1:UInt32]\ + \n TableScan: test [a:UInt32, b:UInt32, c:UInt32]"; assert_optimized_plan_equal(&plan, expected) } @@ -436,10 +391,9 @@ mod tests { )? .build()?; // Should work - let expected = "Projection: test.a, COUNT(alias1) AS COUNT(DISTINCT test.b), MAX(alias1) AS MAX(DISTINCT test.b) [a:UInt32, COUNT(DISTINCT test.b):Int64;N, MAX(DISTINCT test.b):UInt32;N]\ - \n Aggregate: groupBy=[[test.a]], aggr=[[COUNT(alias1), MAX(alias1)]] [a:UInt32, COUNT(alias1):Int64;N, MAX(alias1):UInt32;N]\ - \n Aggregate: groupBy=[[test.a, test.b AS alias1]], aggr=[[]] [a:UInt32, alias1:UInt32]\ - \n TableScan: test [a:UInt32, b:UInt32, c:UInt32]"; + let expected = "Aggregate: groupBy=[[test.a]], aggr=[[COUNT(alias1) AS COUNT(DISTINCT test.b), MAX(alias1) AS MAX(DISTINCT test.b)]] [a:UInt32, COUNT(DISTINCT test.b):Int64;N, MAX(DISTINCT test.b):UInt32;N]\ + \n Aggregate: groupBy=[[test.a, test.b AS alias1]], aggr=[[]] [a:UInt32, alias1:UInt32]\ + \n TableScan: test [a:UInt32, b:UInt32, c:UInt32]"; assert_optimized_plan_equal(&plan, expected) } @@ -471,10 +425,9 @@ mod tests { .build()?; // Should work - let expected = "Projection: group_alias_0 AS test.a + Int32(1), COUNT(alias1) AS COUNT(DISTINCT test.c) [test.a + Int32(1):Int32, COUNT(DISTINCT test.c):Int64;N]\ - \n Aggregate: groupBy=[[group_alias_0]], aggr=[[COUNT(alias1)]] [group_alias_0:Int32, COUNT(alias1):Int64;N]\ - \n Aggregate: groupBy=[[test.a + Int32(1) AS group_alias_0, test.c AS alias1]], aggr=[[]] [group_alias_0:Int32, alias1:UInt32]\ - \n TableScan: test [a:UInt32, b:UInt32, c:UInt32]"; + let expected = "Aggregate: groupBy=[[group_alias_0]], aggr=[[COUNT(alias1) AS COUNT(DISTINCT test.c)]] [group_alias_0:Int32, COUNT(DISTINCT test.c):Int64;N]\ + \n Aggregate: groupBy=[[test.a + Int32(1) AS group_alias_0, test.c AS alias1]], aggr=[[]] [group_alias_0:Int32, alias1:UInt32]\ + \n TableScan: test [a:UInt32, b:UInt32, c:UInt32]"; assert_optimized_plan_equal(&plan, expected) } diff --git a/datafusion/sqllogictest/test_files/groupby.slt b/datafusion/sqllogictest/test_files/groupby.slt index cb0b0b7c76a5..000c3dc3b503 100644 --- a/datafusion/sqllogictest/test_files/groupby.slt +++ b/datafusion/sqllogictest/test_files/groupby.slt @@ -3828,17 +3828,17 @@ query TT EXPLAIN SELECT SUM(DISTINCT CAST(x AS DOUBLE)), MAX(DISTINCT CAST(x AS DOUBLE)) FROM t1 GROUP BY y; ---- logical_plan -Projection: SUM(alias1) AS SUM(DISTINCT t1.x), MAX(alias1) AS MAX(DISTINCT t1.x) ---Aggregate: groupBy=[[t1.y]], aggr=[[SUM(alias1), MAX(alias1)]] +Projection: SUM(DISTINCT t1.x), MAX(DISTINCT t1.x) +--Aggregate: groupBy=[[t1.y]], aggr=[[SUM(alias1) AS SUM(DISTINCT t1.x), MAX(alias1) AS MAX(DISTINCT t1.x)]] ----Aggregate: groupBy=[[t1.y, CAST(t1.x AS Float64)t1.x AS t1.x AS alias1]], aggr=[[]] ------Projection: CAST(t1.x AS Float64) AS CAST(t1.x AS Float64)t1.x, t1.y --------TableScan: t1 projection=[x, y] physical_plan -ProjectionExec: expr=[SUM(alias1)@1 as SUM(DISTINCT t1.x), MAX(alias1)@2 as MAX(DISTINCT t1.x)] ---AggregateExec: mode=FinalPartitioned, gby=[y@0 as y], aggr=[SUM(alias1), MAX(alias1)] +ProjectionExec: expr=[SUM(DISTINCT t1.x)@1 as SUM(DISTINCT t1.x), MAX(DISTINCT t1.x)@2 as MAX(DISTINCT t1.x)] +--AggregateExec: mode=FinalPartitioned, gby=[y@0 as y], aggr=[SUM(DISTINCT t1.x), MAX(DISTINCT t1.x)] ----CoalesceBatchesExec: target_batch_size=2 ------RepartitionExec: partitioning=Hash([y@0], 8), input_partitions=8 ---------AggregateExec: mode=Partial, gby=[y@0 as y], aggr=[SUM(alias1), MAX(alias1)] +--------AggregateExec: mode=Partial, gby=[y@0 as y], aggr=[SUM(DISTINCT t1.x), MAX(DISTINCT t1.x)] ----------AggregateExec: mode=FinalPartitioned, gby=[y@0 as y, alias1@1 as alias1], aggr=[] ------------CoalesceBatchesExec: target_batch_size=2 --------------RepartitionExec: partitioning=Hash([y@0, alias1@1], 8), input_partitions=8 diff --git a/datafusion/sqllogictest/test_files/joins.slt b/datafusion/sqllogictest/test_files/joins.slt index c794c4da4310..25ab2032f0b0 100644 --- a/datafusion/sqllogictest/test_files/joins.slt +++ b/datafusion/sqllogictest/test_files/joins.slt @@ -1361,31 +1361,29 @@ from join_t1 inner join join_t2 on join_t1.t1_id = join_t2.t2_id ---- logical_plan -Projection: COUNT(alias1) AS COUNT(DISTINCT join_t1.t1_id) ---Aggregate: groupBy=[[]], aggr=[[COUNT(alias1)]] -----Aggregate: groupBy=[[join_t1.t1_id AS alias1]], aggr=[[]] -------Projection: join_t1.t1_id ---------Inner Join: join_t1.t1_id = join_t2.t2_id -----------TableScan: join_t1 projection=[t1_id] -----------TableScan: join_t2 projection=[t2_id] +Aggregate: groupBy=[[]], aggr=[[COUNT(alias1) AS COUNT(DISTINCT join_t1.t1_id)]] +--Aggregate: groupBy=[[join_t1.t1_id AS alias1]], aggr=[[]] +----Projection: join_t1.t1_id +------Inner Join: join_t1.t1_id = join_t2.t2_id +--------TableScan: join_t1 projection=[t1_id] +--------TableScan: join_t2 projection=[t2_id] physical_plan -ProjectionExec: expr=[COUNT(alias1)@0 as COUNT(DISTINCT join_t1.t1_id)] ---AggregateExec: mode=Final, gby=[], aggr=[COUNT(alias1)] -----CoalescePartitionsExec -------AggregateExec: mode=Partial, gby=[], aggr=[COUNT(alias1)] ---------AggregateExec: mode=FinalPartitioned, gby=[alias1@0 as alias1], aggr=[] -----------AggregateExec: mode=Partial, gby=[t1_id@0 as alias1], aggr=[] -------------ProjectionExec: expr=[t1_id@0 as t1_id] ---------------CoalesceBatchesExec: target_batch_size=2 -----------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(t1_id@0, t2_id@0)] -------------------CoalesceBatchesExec: target_batch_size=2 ---------------------RepartitionExec: partitioning=Hash([t1_id@0], 2), input_partitions=2 -----------------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -------------------------MemoryExec: partitions=1, partition_sizes=[1] -------------------CoalesceBatchesExec: target_batch_size=2 ---------------------RepartitionExec: partitioning=Hash([t2_id@0], 2), input_partitions=2 -----------------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -------------------------MemoryExec: partitions=1, partition_sizes=[1] +AggregateExec: mode=Final, gby=[], aggr=[COUNT(DISTINCT join_t1.t1_id)] +--CoalescePartitionsExec +----AggregateExec: mode=Partial, gby=[], aggr=[COUNT(DISTINCT join_t1.t1_id)] +------AggregateExec: mode=FinalPartitioned, gby=[alias1@0 as alias1], aggr=[] +--------AggregateExec: mode=Partial, gby=[t1_id@0 as alias1], aggr=[] +----------ProjectionExec: expr=[t1_id@0 as t1_id] +------------CoalesceBatchesExec: target_batch_size=2 +--------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(t1_id@0, t2_id@0)] +----------------CoalesceBatchesExec: target_batch_size=2 +------------------RepartitionExec: partitioning=Hash([t1_id@0], 2), input_partitions=2 +--------------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +----------------------MemoryExec: partitions=1, partition_sizes=[1] +----------------CoalesceBatchesExec: target_batch_size=2 +------------------RepartitionExec: partitioning=Hash([t2_id@0], 2), input_partitions=2 +--------------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +----------------------MemoryExec: partitions=1, partition_sizes=[1] statement ok set datafusion.explain.logical_plan_only = true; @@ -3422,4 +3420,3 @@ set datafusion.optimizer.prefer_existing_sort = false; statement ok drop table annotated_data; - diff --git a/datafusion/sqllogictest/test_files/tpch/q16.slt.part b/datafusion/sqllogictest/test_files/tpch/q16.slt.part index b93872929fe5..c04782958917 100644 --- a/datafusion/sqllogictest/test_files/tpch/q16.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/q16.slt.part @@ -52,8 +52,8 @@ limit 10; logical_plan Limit: skip=0, fetch=10 --Sort: supplier_cnt DESC NULLS FIRST, part.p_brand ASC NULLS LAST, part.p_type ASC NULLS LAST, part.p_size ASC NULLS LAST, fetch=10 -----Projection: part.p_brand, part.p_type, part.p_size, COUNT(alias1) AS supplier_cnt -------Aggregate: groupBy=[[part.p_brand, part.p_type, part.p_size]], aggr=[[COUNT(alias1)]] +----Projection: part.p_brand, part.p_type, part.p_size, COUNT(DISTINCT partsupp.ps_suppkey) AS supplier_cnt +------Aggregate: groupBy=[[part.p_brand, part.p_type, part.p_size]], aggr=[[COUNT(alias1) AS COUNT(DISTINCT partsupp.ps_suppkey)]] --------Aggregate: groupBy=[[part.p_brand, part.p_type, part.p_size, partsupp.ps_suppkey AS alias1]], aggr=[[]] ----------LeftAnti Join: partsupp.ps_suppkey = __correlated_sq_1.s_suppkey ------------Projection: partsupp.ps_suppkey, part.p_brand, part.p_type, part.p_size @@ -69,11 +69,11 @@ physical_plan GlobalLimitExec: skip=0, fetch=10 --SortPreservingMergeExec: [supplier_cnt@3 DESC,p_brand@0 ASC NULLS LAST,p_type@1 ASC NULLS LAST,p_size@2 ASC NULLS LAST], fetch=10 ----SortExec: TopK(fetch=10), expr=[supplier_cnt@3 DESC,p_brand@0 ASC NULLS LAST,p_type@1 ASC NULLS LAST,p_size@2 ASC NULLS LAST] -------ProjectionExec: expr=[p_brand@0 as p_brand, p_type@1 as p_type, p_size@2 as p_size, COUNT(alias1)@3 as supplier_cnt] ---------AggregateExec: mode=FinalPartitioned, gby=[p_brand@0 as p_brand, p_type@1 as p_type, p_size@2 as p_size], aggr=[COUNT(alias1)] +------ProjectionExec: expr=[p_brand@0 as p_brand, p_type@1 as p_type, p_size@2 as p_size, COUNT(DISTINCT partsupp.ps_suppkey)@3 as supplier_cnt] +--------AggregateExec: mode=FinalPartitioned, gby=[p_brand@0 as p_brand, p_type@1 as p_type, p_size@2 as p_size], aggr=[COUNT(DISTINCT partsupp.ps_suppkey)] ----------CoalesceBatchesExec: target_batch_size=8192 ------------RepartitionExec: partitioning=Hash([p_brand@0, p_type@1, p_size@2], 4), input_partitions=4 ---------------AggregateExec: mode=Partial, gby=[p_brand@0 as p_brand, p_type@1 as p_type, p_size@2 as p_size], aggr=[COUNT(alias1)] +--------------AggregateExec: mode=Partial, gby=[p_brand@0 as p_brand, p_type@1 as p_type, p_size@2 as p_size], aggr=[COUNT(DISTINCT partsupp.ps_suppkey)] ----------------AggregateExec: mode=FinalPartitioned, gby=[p_brand@0 as p_brand, p_type@1 as p_type, p_size@2 as p_size, alias1@3 as alias1], aggr=[] ------------------CoalesceBatchesExec: target_batch_size=8192 --------------------RepartitionExec: partitioning=Hash([p_brand@0, p_type@1, p_size@2, alias1@3], 4), input_partitions=4 From b7251e414bd293ec26a16f7358e6391eaacb62b2 Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Wed, 8 Nov 2023 11:58:07 -0700 Subject: [PATCH 206/572] minor: Remove duplicate version numbers for arrow, object_store, and parquet dependencies (#8095) * remove duplicate version numbers for arrow, object_store, and parquet dependencies * cargo update * use default features in parquet crate * disable default parquet features in wasmtest --- Cargo.toml | 4 +-- benchmarks/Cargo.toml | 2 +- datafusion-cli/Cargo.lock | 50 +++++++++++++++++----------------- datafusion-examples/Cargo.toml | 2 +- datafusion/common/Cargo.toml | 4 +-- datafusion/core/Cargo.toml | 2 +- datafusion/proto/Cargo.toml | 2 +- datafusion/wasmtest/Cargo.toml | 2 +- 8 files changed, 34 insertions(+), 34 deletions(-) diff --git a/Cargo.toml b/Cargo.toml index 39ebd1fa59b5..e7a4126743f2 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -79,9 +79,9 @@ indexmap = "2.0.0" itertools = "0.11" log = "^0.4" num_cpus = "1.13.0" -object_store = "0.7.0" +object_store = { version = "0.7.0", default-features = false } parking_lot = "0.12" -parquet = { version = "48.0.0", features = ["arrow", "async", "object_store"] } +parquet = { version = "48.0.0", default-features = false, features = ["arrow", "async", "object_store"] } rand = "0.8" rstest = "0.18.0" serde_json = "1" diff --git a/benchmarks/Cargo.toml b/benchmarks/Cargo.toml index 35f94f677d86..c5a24a0a5cf9 100644 --- a/benchmarks/Cargo.toml +++ b/benchmarks/Cargo.toml @@ -41,7 +41,7 @@ futures = { workspace = true } log = { workspace = true } mimalloc = { version = "0.1", optional = true, default-features = false } num_cpus = { workspace = true } -parquet = { workspace = true } +parquet = { workspace = true, default-features = true } serde = { version = "1.0.136", features = ["derive"] } serde_json = { workspace = true } snmalloc-rs = { version = "0.3", optional = true } diff --git a/datafusion-cli/Cargo.lock b/datafusion-cli/Cargo.lock index 74df8aab0175..629293e4839b 100644 --- a/datafusion-cli/Cargo.lock +++ b/datafusion-cli/Cargo.lock @@ -383,7 +383,7 @@ checksum = "a66537f1bb974b254c98ed142ff995236e81b9d0fe4db0575f46612cb15eb0f9" dependencies = [ "proc-macro2", "quote", - "syn 2.0.38", + "syn 2.0.39", ] [[package]] @@ -1073,7 +1073,7 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "37e366bff8cd32dd8754b0991fb66b279dc48f598c3a18914852a6673deef583" dependencies = [ "quote", - "syn 2.0.38", + "syn 2.0.39", ] [[package]] @@ -1422,9 +1422,9 @@ checksum = "5443807d6dff69373d433ab9ef5378ad8df50ca6298caf15de6e52e24aaf54d5" [[package]] name = "errno" -version = "0.3.5" +version = "0.3.6" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "ac3e13f66a2f95e32a39eaa81f6b95d42878ca0e1db0c7543723dfe12557e860" +checksum = "7c18ee0ed65a5f1f81cac6b1d213b69c35fa47d4252ad41f1486dbd8226fe36e" dependencies = [ "libc", "windows-sys", @@ -1572,7 +1572,7 @@ checksum = "53b153fd91e4b0147f4aced87be237c98248656bb01050b96bf3ee89220a8ddb" dependencies = [ "proc-macro2", "quote", - "syn 2.0.38", + "syn 2.0.39", ] [[package]] @@ -1623,9 +1623,9 @@ dependencies = [ [[package]] name = "getrandom" -version = "0.2.10" +version = "0.2.11" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "be4136b2a15dd319360be1c07d9933517ccf0be8f16bf62a3bee4f0d618df427" +checksum = "fe9006bed769170c11f845cf00c7c1e9092aeb3f268e007c3e760ac68008070f" dependencies = [ "cfg-if", "libc", @@ -2064,9 +2064,9 @@ dependencies = [ [[package]] name = "linux-raw-sys" -version = "0.4.10" +version = "0.4.11" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "da2479e8c062e40bf0066ffa0bc823de0a9368974af99c9f6df941d2c231e03f" +checksum = "969488b55f8ac402214f3f5fd243ebb7206cf82de60d3172994707a4bcc2b829" [[package]] name = "lock_api" @@ -2483,7 +2483,7 @@ checksum = "4359fd9c9171ec6e8c62926d6faaf553a8dc3f64e1507e76da7911b4f6a04405" dependencies = [ "proc-macro2", "quote", - "syn 2.0.38", + "syn 2.0.39", ] [[package]] @@ -2992,22 +2992,22 @@ checksum = "a3f0bf26fd526d2a95683cd0f87bf103b8539e2ca1ef48ce002d67aad59aa0b4" [[package]] name = "serde" -version = "1.0.190" +version = "1.0.192" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "91d3c334ca1ee894a2c6f6ad698fe8c435b76d504b13d436f0685d648d6d96f7" +checksum = "bca2a08484b285dcb282d0f67b26cadc0df8b19f8c12502c13d966bf9482f001" dependencies = [ "serde_derive", ] [[package]] name = "serde_derive" -version = "1.0.190" +version = "1.0.192" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "67c5609f394e5c2bd7fc51efda478004ea80ef42fee983d5c67a65e34f32c0e3" +checksum = "d6c7207fbec9faa48073f3e3074cbe553af6ea512d7c21ba46e434e70ea9fbc1" dependencies = [ "proc-macro2", "quote", - "syn 2.0.38", + "syn 2.0.39", ] [[package]] @@ -3183,7 +3183,7 @@ dependencies = [ "proc-macro2", "quote", "rustversion", - "syn 2.0.38", + "syn 2.0.39", ] [[package]] @@ -3205,9 +3205,9 @@ dependencies = [ [[package]] name = "syn" -version = "2.0.38" +version = "2.0.39" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "e96b79aaa137db8f61e26363a0c9b47d8b4ec75da28b7d1d614c2303e232408b" +checksum = "23e78b90f2fcf45d3e842032ce32e3f2d1545ba6636271dcbf24fa306d87be7a" dependencies = [ "proc-macro2", "quote", @@ -3286,7 +3286,7 @@ checksum = "266b2e40bc00e5a6c09c3584011e08b06f123c00362c92b975ba9843aaaa14b8" dependencies = [ "proc-macro2", "quote", - "syn 2.0.38", + "syn 2.0.39", ] [[package]] @@ -3378,7 +3378,7 @@ checksum = "630bdcf245f78637c13ec01ffae6187cca34625e8c63150d424b59e55af2675e" dependencies = [ "proc-macro2", "quote", - "syn 2.0.38", + "syn 2.0.39", ] [[package]] @@ -3475,7 +3475,7 @@ checksum = "34704c8d6ebcbc939824180af020566b01a7c01f80641264eba0999f6c2b6be7" dependencies = [ "proc-macro2", "quote", - "syn 2.0.38", + "syn 2.0.39", ] [[package]] @@ -3520,7 +3520,7 @@ checksum = "f03ca4cb38206e2bef0700092660bb74d696f808514dae47fa1467cbfe26e96e" dependencies = [ "proc-macro2", "quote", - "syn 2.0.38", + "syn 2.0.39", ] [[package]] @@ -3674,7 +3674,7 @@ dependencies = [ "once_cell", "proc-macro2", "quote", - "syn 2.0.38", + "syn 2.0.39", "wasm-bindgen-shared", ] @@ -3708,7 +3708,7 @@ checksum = "c5353b8dab669f5e10f5bd76df26a9360c748f054f862ff5f3f8aae0c7fb3907" dependencies = [ "proc-macro2", "quote", - "syn 2.0.38", + "syn 2.0.39", "wasm-bindgen-backend", "wasm-bindgen-shared", ] @@ -3906,7 +3906,7 @@ checksum = "c2f140bda219a26ccc0cdb03dba58af72590c53b22642577d88a927bc5c87d6b" dependencies = [ "proc-macro2", "quote", - "syn 2.0.38", + "syn 2.0.39", ] [[package]] diff --git a/datafusion-examples/Cargo.toml b/datafusion-examples/Cargo.toml index 57691520a401..676b4aaa78c0 100644 --- a/datafusion-examples/Cargo.toml +++ b/datafusion-examples/Cargo.toml @@ -46,7 +46,7 @@ futures = { workspace = true } log = { workspace = true } mimalloc = { version = "0.1", default-features = false } num_cpus = { workspace = true } -object_store = { version = "0.7.0", features = ["aws", "http"] } +object_store = { workspace = true, features = ["aws", "http"] } prost = { version = "0.12", default-features = false } prost-derive = { version = "0.11", default-features = false } serde = { version = "1.0.136", features = ["derive"] } diff --git a/datafusion/common/Cargo.toml b/datafusion/common/Cargo.toml index d04db86b7830..b3a810153923 100644 --- a/datafusion/common/Cargo.toml +++ b/datafusion/common/Cargo.toml @@ -47,8 +47,8 @@ arrow-schema = { workspace = true } chrono = { workspace = true } half = { version = "2.1", default-features = false } num_cpus = { workspace = true } -object_store = { version = "0.7.0", default-features = false, optional = true } -parquet = { workspace = true, optional = true } +object_store = { workspace = true, optional = true } +parquet = { workspace = true, optional = true, default-features = true } pyo3 = { version = "0.20.0", optional = true } sqlparser = { workspace = true } diff --git a/datafusion/core/Cargo.toml b/datafusion/core/Cargo.toml index b44914ec719f..80aec800d697 100644 --- a/datafusion/core/Cargo.toml +++ b/datafusion/core/Cargo.toml @@ -81,7 +81,7 @@ num-traits = { version = "0.2", optional = true } num_cpus = { workspace = true } object_store = { workspace = true } parking_lot = { workspace = true } -parquet = { workspace = true, optional = true } +parquet = { workspace = true, optional = true, default-features = true } pin-project-lite = "^0.2.7" rand = { workspace = true } sqlparser = { workspace = true } diff --git a/datafusion/proto/Cargo.toml b/datafusion/proto/Cargo.toml index ac3439a64ca8..4dda689fff4c 100644 --- a/datafusion/proto/Cargo.toml +++ b/datafusion/proto/Cargo.toml @@ -46,7 +46,7 @@ chrono = { workspace = true } datafusion = { path = "../core", version = "33.0.0" } datafusion-common = { workspace = true } datafusion-expr = { workspace = true } -object_store = { version = "0.7.0" } +object_store = { workspace = true } pbjson = { version = "0.5", optional = true } prost = "0.12.0" serde = { version = "1.0", optional = true } diff --git a/datafusion/wasmtest/Cargo.toml b/datafusion/wasmtest/Cargo.toml index 882b02bcc84b..c5f795d0653a 100644 --- a/datafusion/wasmtest/Cargo.toml +++ b/datafusion/wasmtest/Cargo.toml @@ -46,5 +46,5 @@ datafusion-sql = { workspace = true } # getrandom must be compiled with js feature getrandom = { version = "0.2.8", features = ["js"] } -parquet = { version = "48.0.0", default-features = false } +parquet = { workspace = true } wasm-bindgen = "0.2.87" From 21b2af126d2a711fc90e7fd938cfdbd8d8c90d5c Mon Sep 17 00:00:00 2001 From: Syleechan <38198463+Syleechan@users.noreply.github.com> Date: Thu, 9 Nov 2023 03:00:00 +0800 Subject: [PATCH 207/572] fix: add match encode/decode scalar function type (#8089) --- datafusion/proto/src/logical_plan/from_proto.rs | 10 +++++++++- 1 file changed, 9 insertions(+), 1 deletion(-) diff --git a/datafusion/proto/src/logical_plan/from_proto.rs b/datafusion/proto/src/logical_plan/from_proto.rs index 26bd0163d0a3..cdb0fe9bda7f 100644 --- a/datafusion/proto/src/logical_plan/from_proto.rs +++ b/datafusion/proto/src/logical_plan/from_proto.rs @@ -46,7 +46,7 @@ use datafusion_expr::{ array_to_string, ascii, asin, asinh, atan, atan2, atanh, bit_length, btrim, cardinality, cbrt, ceil, character_length, chr, coalesce, concat_expr, concat_ws_expr, cos, cosh, cot, current_date, current_time, date_bin, date_part, - date_trunc, degrees, digest, exp, + date_trunc, decode, degrees, digest, encode, exp, expr::{self, InList, Sort, WindowFunction}, factorial, floor, from_unixtime, gcd, isnan, iszero, lcm, left, ln, log, log10, log2, logical_plan::{PlanType, StringifiedPlan}, @@ -1472,6 +1472,14 @@ pub fn parse_expr( ScalarFunction::Sha384 => Ok(sha384(parse_expr(&args[0], registry)?)), ScalarFunction::Sha512 => Ok(sha512(parse_expr(&args[0], registry)?)), ScalarFunction::Md5 => Ok(md5(parse_expr(&args[0], registry)?)), + ScalarFunction::Encode => Ok(encode( + parse_expr(&args[0], registry)?, + parse_expr(&args[1], registry)?, + )), + ScalarFunction::Decode => Ok(decode( + parse_expr(&args[0], registry)?, + parse_expr(&args[1], registry)?, + )), ScalarFunction::NullIf => Ok(nullif( parse_expr(&args[0], registry)?, parse_expr(&args[1], registry)?, From 965b3182fb819a6880cc1b0ca13a76c2358d8bf0 Mon Sep 17 00:00:00 2001 From: Jeffrey <22608443+Jefffrey@users.noreply.github.com> Date: Thu, 9 Nov 2023 08:29:03 +1100 Subject: [PATCH 208/572] feat: Protobuf serde for Json file sink (#8062) * Protobuf serde for Json file sink * Fix tests * Fix test --- .../core/src/datasource/file_format/json.rs | 10 +- datafusion/physical-plan/src/insert.rs | 12 +- datafusion/proto/proto/datafusion.proto | 54 + datafusion/proto/src/generated/pbjson.rs | 1317 ++++++++++++++--- datafusion/proto/src/generated/prost.rs | 139 +- .../proto/src/physical_plan/from_proto.rs | 95 +- datafusion/proto/src/physical_plan/mod.rs | 81 +- .../proto/src/physical_plan/to_proto.rs | 128 +- .../tests/cases/roundtrip_physical_plan.rs | 55 +- datafusion/sqllogictest/test_files/copy.slt | 2 +- .../sqllogictest/test_files/explain.slt | 2 +- datafusion/sqllogictest/test_files/insert.slt | 8 +- .../test_files/insert_to_external.slt | 8 +- 13 files changed, 1703 insertions(+), 208 deletions(-) diff --git a/datafusion/core/src/datasource/file_format/json.rs b/datafusion/core/src/datasource/file_format/json.rs index 70cfd1836efe..8d62d0a858ac 100644 --- a/datafusion/core/src/datasource/file_format/json.rs +++ b/datafusion/core/src/datasource/file_format/json.rs @@ -230,7 +230,7 @@ impl BatchSerializer for JsonSerializer { } /// Implements [`DataSink`] for writing to a Json file. -struct JsonSink { +pub struct JsonSink { /// Config options for writing data config: FileSinkConfig, } @@ -258,10 +258,16 @@ impl DisplayAs for JsonSink { } impl JsonSink { - fn new(config: FileSinkConfig) -> Self { + /// Create from config. + pub fn new(config: FileSinkConfig) -> Self { Self { config } } + /// Retrieve the inner [`FileSinkConfig`]. + pub fn config(&self) -> &FileSinkConfig { + &self.config + } + async fn append_all( &self, data: SendableRecordBatchStream, diff --git a/datafusion/physical-plan/src/insert.rs b/datafusion/physical-plan/src/insert.rs index 627d58e13781..4eeb58974aba 100644 --- a/datafusion/physical-plan/src/insert.rs +++ b/datafusion/physical-plan/src/insert.rs @@ -151,11 +151,21 @@ impl FileSinkExec { } } + /// Input execution plan + pub fn input(&self) -> &Arc { + &self.input + } + /// Returns insert sink pub fn sink(&self) -> &dyn DataSink { self.sink.as_ref() } + /// Optional sort order for output data + pub fn sort_order(&self) -> &Option> { + &self.sort_order + } + /// Returns the metrics of the underlying [DataSink] pub fn metrics(&self) -> Option { self.sink.metrics() @@ -170,7 +180,7 @@ impl DisplayAs for FileSinkExec { ) -> std::fmt::Result { match t { DisplayFormatType::Default | DisplayFormatType::Verbose => { - write!(f, "InsertExec: sink=")?; + write!(f, "FileSinkExec: sink=")?; self.sink.fmt_as(t, f) } } diff --git a/datafusion/proto/proto/datafusion.proto b/datafusion/proto/proto/datafusion.proto index 9b6a0448f810..bc6de2348e8d 100644 --- a/datafusion/proto/proto/datafusion.proto +++ b/datafusion/proto/proto/datafusion.proto @@ -1130,9 +1130,63 @@ message PhysicalPlanNode { SortPreservingMergeExecNode sort_preserving_merge = 21; NestedLoopJoinExecNode nested_loop_join = 22; AnalyzeExecNode analyze = 23; + JsonSinkExecNode json_sink = 24; } } +enum FileWriterMode { + APPEND = 0; + PUT = 1; + PUT_MULTIPART = 2; +} + +enum CompressionTypeVariant { + GZIP = 0; + BZIP2 = 1; + XZ = 2; + ZSTD = 3; + UNCOMPRESSED = 4; +} + +message PartitionColumn { + string name = 1; + ArrowType arrow_type = 2; +} + +message FileTypeWriterOptions { + oneof FileType { + JsonWriterOptions json_options = 1; + } +} + +message JsonWriterOptions { + CompressionTypeVariant compression = 1; +} + +message FileSinkConfig { + string object_store_url = 1; + repeated PartitionedFile file_groups = 2; + repeated string table_paths = 3; + Schema output_schema = 4; + repeated PartitionColumn table_partition_cols = 5; + FileWriterMode writer_mode = 6; + bool single_file_output = 7; + bool unbounded_input = 8; + bool overwrite = 9; + FileTypeWriterOptions file_type_writer_options = 10; +} + +message JsonSink { + FileSinkConfig config = 1; +} + +message JsonSinkExecNode { + PhysicalPlanNode input = 1; + JsonSink sink = 2; + Schema sink_schema = 3; + PhysicalSortExprNodeCollection sort_order = 4; +} + message PhysicalExtensionNode { bytes node = 1; repeated PhysicalPlanNode inputs = 2; diff --git a/datafusion/proto/src/generated/pbjson.rs b/datafusion/proto/src/generated/pbjson.rs index 3eeb060f8d01..659a25f9fa35 100644 --- a/datafusion/proto/src/generated/pbjson.rs +++ b/datafusion/proto/src/generated/pbjson.rs @@ -3421,6 +3421,86 @@ impl<'de> serde::Deserialize<'de> for ColumnStats { deserializer.deserialize_struct("datafusion.ColumnStats", FIELDS, GeneratedVisitor) } } +impl serde::Serialize for CompressionTypeVariant { + #[allow(deprecated)] + fn serialize(&self, serializer: S) -> std::result::Result + where + S: serde::Serializer, + { + let variant = match self { + Self::Gzip => "GZIP", + Self::Bzip2 => "BZIP2", + Self::Xz => "XZ", + Self::Zstd => "ZSTD", + Self::Uncompressed => "UNCOMPRESSED", + }; + serializer.serialize_str(variant) + } +} +impl<'de> serde::Deserialize<'de> for CompressionTypeVariant { + #[allow(deprecated)] + fn deserialize(deserializer: D) -> std::result::Result + where + D: serde::Deserializer<'de>, + { + const FIELDS: &[&str] = &[ + "GZIP", + "BZIP2", + "XZ", + "ZSTD", + "UNCOMPRESSED", + ]; + + struct GeneratedVisitor; + + impl<'de> serde::de::Visitor<'de> for GeneratedVisitor { + type Value = CompressionTypeVariant; + + fn expecting(&self, formatter: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + write!(formatter, "expected one of: {:?}", &FIELDS) + } + + fn visit_i64(self, v: i64) -> std::result::Result + where + E: serde::de::Error, + { + i32::try_from(v) + .ok() + .and_then(|x| x.try_into().ok()) + .ok_or_else(|| { + serde::de::Error::invalid_value(serde::de::Unexpected::Signed(v), &self) + }) + } + + fn visit_u64(self, v: u64) -> std::result::Result + where + E: serde::de::Error, + { + i32::try_from(v) + .ok() + .and_then(|x| x.try_into().ok()) + .ok_or_else(|| { + serde::de::Error::invalid_value(serde::de::Unexpected::Unsigned(v), &self) + }) + } + + fn visit_str(self, value: &str) -> std::result::Result + where + E: serde::de::Error, + { + match value { + "GZIP" => Ok(CompressionTypeVariant::Gzip), + "BZIP2" => Ok(CompressionTypeVariant::Bzip2), + "XZ" => Ok(CompressionTypeVariant::Xz), + "ZSTD" => Ok(CompressionTypeVariant::Zstd), + "UNCOMPRESSED" => Ok(CompressionTypeVariant::Uncompressed), + _ => Err(serde::de::Error::unknown_variant(value, FIELDS)), + } + } + } + deserializer.deserialize_any(GeneratedVisitor) + } +} impl serde::Serialize for Constraint { #[allow(deprecated)] fn serialize(&self, serializer: S) -> std::result::Result @@ -7206,7 +7286,7 @@ impl<'de> serde::Deserialize<'de> for FileScanExecConf { deserializer.deserialize_struct("datafusion.FileScanExecConf", FIELDS, GeneratedVisitor) } } -impl serde::Serialize for FilterExecNode { +impl serde::Serialize for FileSinkConfig { #[allow(deprecated)] fn serialize(&self, serializer: S) -> std::result::Result where @@ -7214,37 +7294,112 @@ impl serde::Serialize for FilterExecNode { { use serde::ser::SerializeStruct; let mut len = 0; - if self.input.is_some() { + if !self.object_store_url.is_empty() { len += 1; } - if self.expr.is_some() { + if !self.file_groups.is_empty() { len += 1; } - let mut struct_ser = serializer.serialize_struct("datafusion.FilterExecNode", len)?; - if let Some(v) = self.input.as_ref() { - struct_ser.serialize_field("input", v)?; + if !self.table_paths.is_empty() { + len += 1; } - if let Some(v) = self.expr.as_ref() { - struct_ser.serialize_field("expr", v)?; + if self.output_schema.is_some() { + len += 1; + } + if !self.table_partition_cols.is_empty() { + len += 1; + } + if self.writer_mode != 0 { + len += 1; + } + if self.single_file_output { + len += 1; + } + if self.unbounded_input { + len += 1; + } + if self.overwrite { + len += 1; + } + if self.file_type_writer_options.is_some() { + len += 1; + } + let mut struct_ser = serializer.serialize_struct("datafusion.FileSinkConfig", len)?; + if !self.object_store_url.is_empty() { + struct_ser.serialize_field("objectStoreUrl", &self.object_store_url)?; + } + if !self.file_groups.is_empty() { + struct_ser.serialize_field("fileGroups", &self.file_groups)?; + } + if !self.table_paths.is_empty() { + struct_ser.serialize_field("tablePaths", &self.table_paths)?; + } + if let Some(v) = self.output_schema.as_ref() { + struct_ser.serialize_field("outputSchema", v)?; + } + if !self.table_partition_cols.is_empty() { + struct_ser.serialize_field("tablePartitionCols", &self.table_partition_cols)?; + } + if self.writer_mode != 0 { + let v = FileWriterMode::try_from(self.writer_mode) + .map_err(|_| serde::ser::Error::custom(format!("Invalid variant {}", self.writer_mode)))?; + struct_ser.serialize_field("writerMode", &v)?; + } + if self.single_file_output { + struct_ser.serialize_field("singleFileOutput", &self.single_file_output)?; + } + if self.unbounded_input { + struct_ser.serialize_field("unboundedInput", &self.unbounded_input)?; + } + if self.overwrite { + struct_ser.serialize_field("overwrite", &self.overwrite)?; + } + if let Some(v) = self.file_type_writer_options.as_ref() { + struct_ser.serialize_field("fileTypeWriterOptions", v)?; } struct_ser.end() } } -impl<'de> serde::Deserialize<'de> for FilterExecNode { +impl<'de> serde::Deserialize<'de> for FileSinkConfig { #[allow(deprecated)] fn deserialize(deserializer: D) -> std::result::Result where D: serde::Deserializer<'de>, { const FIELDS: &[&str] = &[ - "input", - "expr", + "object_store_url", + "objectStoreUrl", + "file_groups", + "fileGroups", + "table_paths", + "tablePaths", + "output_schema", + "outputSchema", + "table_partition_cols", + "tablePartitionCols", + "writer_mode", + "writerMode", + "single_file_output", + "singleFileOutput", + "unbounded_input", + "unboundedInput", + "overwrite", + "file_type_writer_options", + "fileTypeWriterOptions", ]; #[allow(clippy::enum_variant_names)] enum GeneratedField { - Input, - Expr, + ObjectStoreUrl, + FileGroups, + TablePaths, + OutputSchema, + TablePartitionCols, + WriterMode, + SingleFileOutput, + UnboundedInput, + Overwrite, + FileTypeWriterOptions, } impl<'de> serde::Deserialize<'de> for GeneratedField { fn deserialize(deserializer: D) -> std::result::Result @@ -7266,8 +7421,16 @@ impl<'de> serde::Deserialize<'de> for FilterExecNode { E: serde::de::Error, { match value { - "input" => Ok(GeneratedField::Input), - "expr" => Ok(GeneratedField::Expr), + "objectStoreUrl" | "object_store_url" => Ok(GeneratedField::ObjectStoreUrl), + "fileGroups" | "file_groups" => Ok(GeneratedField::FileGroups), + "tablePaths" | "table_paths" => Ok(GeneratedField::TablePaths), + "outputSchema" | "output_schema" => Ok(GeneratedField::OutputSchema), + "tablePartitionCols" | "table_partition_cols" => Ok(GeneratedField::TablePartitionCols), + "writerMode" | "writer_mode" => Ok(GeneratedField::WriterMode), + "singleFileOutput" | "single_file_output" => Ok(GeneratedField::SingleFileOutput), + "unboundedInput" | "unbounded_input" => Ok(GeneratedField::UnboundedInput), + "overwrite" => Ok(GeneratedField::Overwrite), + "fileTypeWriterOptions" | "file_type_writer_options" => Ok(GeneratedField::FileTypeWriterOptions), _ => Err(serde::de::Error::unknown_field(value, FIELDS)), } } @@ -7277,44 +7440,108 @@ impl<'de> serde::Deserialize<'de> for FilterExecNode { } struct GeneratedVisitor; impl<'de> serde::de::Visitor<'de> for GeneratedVisitor { - type Value = FilterExecNode; + type Value = FileSinkConfig; fn expecting(&self, formatter: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { - formatter.write_str("struct datafusion.FilterExecNode") + formatter.write_str("struct datafusion.FileSinkConfig") } - fn visit_map(self, mut map_: V) -> std::result::Result + fn visit_map(self, mut map_: V) -> std::result::Result where V: serde::de::MapAccess<'de>, { - let mut input__ = None; - let mut expr__ = None; + let mut object_store_url__ = None; + let mut file_groups__ = None; + let mut table_paths__ = None; + let mut output_schema__ = None; + let mut table_partition_cols__ = None; + let mut writer_mode__ = None; + let mut single_file_output__ = None; + let mut unbounded_input__ = None; + let mut overwrite__ = None; + let mut file_type_writer_options__ = None; while let Some(k) = map_.next_key()? { match k { - GeneratedField::Input => { - if input__.is_some() { - return Err(serde::de::Error::duplicate_field("input")); + GeneratedField::ObjectStoreUrl => { + if object_store_url__.is_some() { + return Err(serde::de::Error::duplicate_field("objectStoreUrl")); } - input__ = map_.next_value()?; + object_store_url__ = Some(map_.next_value()?); } - GeneratedField::Expr => { - if expr__.is_some() { - return Err(serde::de::Error::duplicate_field("expr")); + GeneratedField::FileGroups => { + if file_groups__.is_some() { + return Err(serde::de::Error::duplicate_field("fileGroups")); } - expr__ = map_.next_value()?; + file_groups__ = Some(map_.next_value()?); + } + GeneratedField::TablePaths => { + if table_paths__.is_some() { + return Err(serde::de::Error::duplicate_field("tablePaths")); + } + table_paths__ = Some(map_.next_value()?); + } + GeneratedField::OutputSchema => { + if output_schema__.is_some() { + return Err(serde::de::Error::duplicate_field("outputSchema")); + } + output_schema__ = map_.next_value()?; + } + GeneratedField::TablePartitionCols => { + if table_partition_cols__.is_some() { + return Err(serde::de::Error::duplicate_field("tablePartitionCols")); + } + table_partition_cols__ = Some(map_.next_value()?); + } + GeneratedField::WriterMode => { + if writer_mode__.is_some() { + return Err(serde::de::Error::duplicate_field("writerMode")); + } + writer_mode__ = Some(map_.next_value::()? as i32); + } + GeneratedField::SingleFileOutput => { + if single_file_output__.is_some() { + return Err(serde::de::Error::duplicate_field("singleFileOutput")); + } + single_file_output__ = Some(map_.next_value()?); + } + GeneratedField::UnboundedInput => { + if unbounded_input__.is_some() { + return Err(serde::de::Error::duplicate_field("unboundedInput")); + } + unbounded_input__ = Some(map_.next_value()?); + } + GeneratedField::Overwrite => { + if overwrite__.is_some() { + return Err(serde::de::Error::duplicate_field("overwrite")); + } + overwrite__ = Some(map_.next_value()?); + } + GeneratedField::FileTypeWriterOptions => { + if file_type_writer_options__.is_some() { + return Err(serde::de::Error::duplicate_field("fileTypeWriterOptions")); + } + file_type_writer_options__ = map_.next_value()?; } } } - Ok(FilterExecNode { - input: input__, - expr: expr__, + Ok(FileSinkConfig { + object_store_url: object_store_url__.unwrap_or_default(), + file_groups: file_groups__.unwrap_or_default(), + table_paths: table_paths__.unwrap_or_default(), + output_schema: output_schema__, + table_partition_cols: table_partition_cols__.unwrap_or_default(), + writer_mode: writer_mode__.unwrap_or_default(), + single_file_output: single_file_output__.unwrap_or_default(), + unbounded_input: unbounded_input__.unwrap_or_default(), + overwrite: overwrite__.unwrap_or_default(), + file_type_writer_options: file_type_writer_options__, }) } } - deserializer.deserialize_struct("datafusion.FilterExecNode", FIELDS, GeneratedVisitor) + deserializer.deserialize_struct("datafusion.FileSinkConfig", FIELDS, GeneratedVisitor) } } -impl serde::Serialize for FixedSizeBinary { +impl serde::Serialize for FileTypeWriterOptions { #[allow(deprecated)] fn serialize(&self, serializer: S) -> std::result::Result where @@ -7322,29 +7549,34 @@ impl serde::Serialize for FixedSizeBinary { { use serde::ser::SerializeStruct; let mut len = 0; - if self.length != 0 { + if self.file_type.is_some() { len += 1; } - let mut struct_ser = serializer.serialize_struct("datafusion.FixedSizeBinary", len)?; - if self.length != 0 { - struct_ser.serialize_field("length", &self.length)?; + let mut struct_ser = serializer.serialize_struct("datafusion.FileTypeWriterOptions", len)?; + if let Some(v) = self.file_type.as_ref() { + match v { + file_type_writer_options::FileType::JsonOptions(v) => { + struct_ser.serialize_field("jsonOptions", v)?; + } + } } struct_ser.end() } } -impl<'de> serde::Deserialize<'de> for FixedSizeBinary { +impl<'de> serde::Deserialize<'de> for FileTypeWriterOptions { #[allow(deprecated)] fn deserialize(deserializer: D) -> std::result::Result where D: serde::Deserializer<'de>, { const FIELDS: &[&str] = &[ - "length", + "json_options", + "jsonOptions", ]; #[allow(clippy::enum_variant_names)] enum GeneratedField { - Length, + JsonOptions, } impl<'de> serde::Deserialize<'de> for GeneratedField { fn deserialize(deserializer: D) -> std::result::Result @@ -7366,7 +7598,7 @@ impl<'de> serde::Deserialize<'de> for FixedSizeBinary { E: serde::de::Error, { match value { - "length" => Ok(GeneratedField::Length), + "jsonOptions" | "json_options" => Ok(GeneratedField::JsonOptions), _ => Err(serde::de::Error::unknown_field(value, FIELDS)), } } @@ -7376,102 +7608,376 @@ impl<'de> serde::Deserialize<'de> for FixedSizeBinary { } struct GeneratedVisitor; impl<'de> serde::de::Visitor<'de> for GeneratedVisitor { - type Value = FixedSizeBinary; + type Value = FileTypeWriterOptions; fn expecting(&self, formatter: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { - formatter.write_str("struct datafusion.FixedSizeBinary") + formatter.write_str("struct datafusion.FileTypeWriterOptions") } - fn visit_map(self, mut map_: V) -> std::result::Result + fn visit_map(self, mut map_: V) -> std::result::Result where V: serde::de::MapAccess<'de>, { - let mut length__ = None; + let mut file_type__ = None; while let Some(k) = map_.next_key()? { match k { - GeneratedField::Length => { - if length__.is_some() { - return Err(serde::de::Error::duplicate_field("length")); + GeneratedField::JsonOptions => { + if file_type__.is_some() { + return Err(serde::de::Error::duplicate_field("jsonOptions")); } - length__ = - Some(map_.next_value::<::pbjson::private::NumberDeserialize<_>>()?.0) - ; + file_type__ = map_.next_value::<::std::option::Option<_>>()?.map(file_type_writer_options::FileType::JsonOptions) +; } } } - Ok(FixedSizeBinary { - length: length__.unwrap_or_default(), + Ok(FileTypeWriterOptions { + file_type: file_type__, }) } } - deserializer.deserialize_struct("datafusion.FixedSizeBinary", FIELDS, GeneratedVisitor) + deserializer.deserialize_struct("datafusion.FileTypeWriterOptions", FIELDS, GeneratedVisitor) } } -impl serde::Serialize for FixedSizeList { +impl serde::Serialize for FileWriterMode { #[allow(deprecated)] fn serialize(&self, serializer: S) -> std::result::Result where S: serde::Serializer, { - use serde::ser::SerializeStruct; - let mut len = 0; - if self.field_type.is_some() { - len += 1; - } - if self.list_size != 0 { - len += 1; - } - let mut struct_ser = serializer.serialize_struct("datafusion.FixedSizeList", len)?; - if let Some(v) = self.field_type.as_ref() { - struct_ser.serialize_field("fieldType", v)?; - } - if self.list_size != 0 { - struct_ser.serialize_field("listSize", &self.list_size)?; - } - struct_ser.end() + let variant = match self { + Self::Append => "APPEND", + Self::Put => "PUT", + Self::PutMultipart => "PUT_MULTIPART", + }; + serializer.serialize_str(variant) } } -impl<'de> serde::Deserialize<'de> for FixedSizeList { +impl<'de> serde::Deserialize<'de> for FileWriterMode { #[allow(deprecated)] fn deserialize(deserializer: D) -> std::result::Result where D: serde::Deserializer<'de>, { const FIELDS: &[&str] = &[ - "field_type", - "fieldType", - "list_size", - "listSize", + "APPEND", + "PUT", + "PUT_MULTIPART", ]; - #[allow(clippy::enum_variant_names)] - enum GeneratedField { - FieldType, - ListSize, - } - impl<'de> serde::Deserialize<'de> for GeneratedField { - fn deserialize(deserializer: D) -> std::result::Result - where - D: serde::Deserializer<'de>, - { - struct GeneratedVisitor; + struct GeneratedVisitor; - impl<'de> serde::de::Visitor<'de> for GeneratedVisitor { - type Value = GeneratedField; + impl<'de> serde::de::Visitor<'de> for GeneratedVisitor { + type Value = FileWriterMode; - fn expecting(&self, formatter: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { - write!(formatter, "expected one of: {:?}", &FIELDS) - } + fn expecting(&self, formatter: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + write!(formatter, "expected one of: {:?}", &FIELDS) + } - #[allow(unused_variables)] - fn visit_str(self, value: &str) -> std::result::Result - where - E: serde::de::Error, - { - match value { - "fieldType" | "field_type" => Ok(GeneratedField::FieldType), - "listSize" | "list_size" => Ok(GeneratedField::ListSize), - _ => Err(serde::de::Error::unknown_field(value, FIELDS)), + fn visit_i64(self, v: i64) -> std::result::Result + where + E: serde::de::Error, + { + i32::try_from(v) + .ok() + .and_then(|x| x.try_into().ok()) + .ok_or_else(|| { + serde::de::Error::invalid_value(serde::de::Unexpected::Signed(v), &self) + }) + } + + fn visit_u64(self, v: u64) -> std::result::Result + where + E: serde::de::Error, + { + i32::try_from(v) + .ok() + .and_then(|x| x.try_into().ok()) + .ok_or_else(|| { + serde::de::Error::invalid_value(serde::de::Unexpected::Unsigned(v), &self) + }) + } + + fn visit_str(self, value: &str) -> std::result::Result + where + E: serde::de::Error, + { + match value { + "APPEND" => Ok(FileWriterMode::Append), + "PUT" => Ok(FileWriterMode::Put), + "PUT_MULTIPART" => Ok(FileWriterMode::PutMultipart), + _ => Err(serde::de::Error::unknown_variant(value, FIELDS)), + } + } + } + deserializer.deserialize_any(GeneratedVisitor) + } +} +impl serde::Serialize for FilterExecNode { + #[allow(deprecated)] + fn serialize(&self, serializer: S) -> std::result::Result + where + S: serde::Serializer, + { + use serde::ser::SerializeStruct; + let mut len = 0; + if self.input.is_some() { + len += 1; + } + if self.expr.is_some() { + len += 1; + } + let mut struct_ser = serializer.serialize_struct("datafusion.FilterExecNode", len)?; + if let Some(v) = self.input.as_ref() { + struct_ser.serialize_field("input", v)?; + } + if let Some(v) = self.expr.as_ref() { + struct_ser.serialize_field("expr", v)?; + } + struct_ser.end() + } +} +impl<'de> serde::Deserialize<'de> for FilterExecNode { + #[allow(deprecated)] + fn deserialize(deserializer: D) -> std::result::Result + where + D: serde::Deserializer<'de>, + { + const FIELDS: &[&str] = &[ + "input", + "expr", + ]; + + #[allow(clippy::enum_variant_names)] + enum GeneratedField { + Input, + Expr, + } + impl<'de> serde::Deserialize<'de> for GeneratedField { + fn deserialize(deserializer: D) -> std::result::Result + where + D: serde::Deserializer<'de>, + { + struct GeneratedVisitor; + + impl<'de> serde::de::Visitor<'de> for GeneratedVisitor { + type Value = GeneratedField; + + fn expecting(&self, formatter: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + write!(formatter, "expected one of: {:?}", &FIELDS) + } + + #[allow(unused_variables)] + fn visit_str(self, value: &str) -> std::result::Result + where + E: serde::de::Error, + { + match value { + "input" => Ok(GeneratedField::Input), + "expr" => Ok(GeneratedField::Expr), + _ => Err(serde::de::Error::unknown_field(value, FIELDS)), + } + } + } + deserializer.deserialize_identifier(GeneratedVisitor) + } + } + struct GeneratedVisitor; + impl<'de> serde::de::Visitor<'de> for GeneratedVisitor { + type Value = FilterExecNode; + + fn expecting(&self, formatter: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + formatter.write_str("struct datafusion.FilterExecNode") + } + + fn visit_map(self, mut map_: V) -> std::result::Result + where + V: serde::de::MapAccess<'de>, + { + let mut input__ = None; + let mut expr__ = None; + while let Some(k) = map_.next_key()? { + match k { + GeneratedField::Input => { + if input__.is_some() { + return Err(serde::de::Error::duplicate_field("input")); + } + input__ = map_.next_value()?; + } + GeneratedField::Expr => { + if expr__.is_some() { + return Err(serde::de::Error::duplicate_field("expr")); + } + expr__ = map_.next_value()?; + } + } + } + Ok(FilterExecNode { + input: input__, + expr: expr__, + }) + } + } + deserializer.deserialize_struct("datafusion.FilterExecNode", FIELDS, GeneratedVisitor) + } +} +impl serde::Serialize for FixedSizeBinary { + #[allow(deprecated)] + fn serialize(&self, serializer: S) -> std::result::Result + where + S: serde::Serializer, + { + use serde::ser::SerializeStruct; + let mut len = 0; + if self.length != 0 { + len += 1; + } + let mut struct_ser = serializer.serialize_struct("datafusion.FixedSizeBinary", len)?; + if self.length != 0 { + struct_ser.serialize_field("length", &self.length)?; + } + struct_ser.end() + } +} +impl<'de> serde::Deserialize<'de> for FixedSizeBinary { + #[allow(deprecated)] + fn deserialize(deserializer: D) -> std::result::Result + where + D: serde::Deserializer<'de>, + { + const FIELDS: &[&str] = &[ + "length", + ]; + + #[allow(clippy::enum_variant_names)] + enum GeneratedField { + Length, + } + impl<'de> serde::Deserialize<'de> for GeneratedField { + fn deserialize(deserializer: D) -> std::result::Result + where + D: serde::Deserializer<'de>, + { + struct GeneratedVisitor; + + impl<'de> serde::de::Visitor<'de> for GeneratedVisitor { + type Value = GeneratedField; + + fn expecting(&self, formatter: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + write!(formatter, "expected one of: {:?}", &FIELDS) + } + + #[allow(unused_variables)] + fn visit_str(self, value: &str) -> std::result::Result + where + E: serde::de::Error, + { + match value { + "length" => Ok(GeneratedField::Length), + _ => Err(serde::de::Error::unknown_field(value, FIELDS)), + } + } + } + deserializer.deserialize_identifier(GeneratedVisitor) + } + } + struct GeneratedVisitor; + impl<'de> serde::de::Visitor<'de> for GeneratedVisitor { + type Value = FixedSizeBinary; + + fn expecting(&self, formatter: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + formatter.write_str("struct datafusion.FixedSizeBinary") + } + + fn visit_map(self, mut map_: V) -> std::result::Result + where + V: serde::de::MapAccess<'de>, + { + let mut length__ = None; + while let Some(k) = map_.next_key()? { + match k { + GeneratedField::Length => { + if length__.is_some() { + return Err(serde::de::Error::duplicate_field("length")); + } + length__ = + Some(map_.next_value::<::pbjson::private::NumberDeserialize<_>>()?.0) + ; + } + } + } + Ok(FixedSizeBinary { + length: length__.unwrap_or_default(), + }) + } + } + deserializer.deserialize_struct("datafusion.FixedSizeBinary", FIELDS, GeneratedVisitor) + } +} +impl serde::Serialize for FixedSizeList { + #[allow(deprecated)] + fn serialize(&self, serializer: S) -> std::result::Result + where + S: serde::Serializer, + { + use serde::ser::SerializeStruct; + let mut len = 0; + if self.field_type.is_some() { + len += 1; + } + if self.list_size != 0 { + len += 1; + } + let mut struct_ser = serializer.serialize_struct("datafusion.FixedSizeList", len)?; + if let Some(v) = self.field_type.as_ref() { + struct_ser.serialize_field("fieldType", v)?; + } + if self.list_size != 0 { + struct_ser.serialize_field("listSize", &self.list_size)?; + } + struct_ser.end() + } +} +impl<'de> serde::Deserialize<'de> for FixedSizeList { + #[allow(deprecated)] + fn deserialize(deserializer: D) -> std::result::Result + where + D: serde::Deserializer<'de>, + { + const FIELDS: &[&str] = &[ + "field_type", + "fieldType", + "list_size", + "listSize", + ]; + + #[allow(clippy::enum_variant_names)] + enum GeneratedField { + FieldType, + ListSize, + } + impl<'de> serde::Deserialize<'de> for GeneratedField { + fn deserialize(deserializer: D) -> std::result::Result + where + D: serde::Deserializer<'de>, + { + struct GeneratedVisitor; + + impl<'de> serde::de::Visitor<'de> for GeneratedVisitor { + type Value = GeneratedField; + + fn expecting(&self, formatter: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + write!(formatter, "expected one of: {:?}", &FIELDS) + } + + #[allow(unused_variables)] + fn visit_str(self, value: &str) -> std::result::Result + where + E: serde::de::Error, + { + match value { + "fieldType" | "field_type" => Ok(GeneratedField::FieldType), + "listSize" | "list_size" => Ok(GeneratedField::ListSize), + _ => Err(serde::de::Error::unknown_field(value, FIELDS)), } } } @@ -10091,119 +10597,447 @@ impl<'de> serde::Deserialize<'de> for JoinSide { }) } - fn visit_u64(self, v: u64) -> std::result::Result - where - E: serde::de::Error, - { - i32::try_from(v) - .ok() - .and_then(|x| x.try_into().ok()) - .ok_or_else(|| { - serde::de::Error::invalid_value(serde::de::Unexpected::Unsigned(v), &self) - }) + fn visit_u64(self, v: u64) -> std::result::Result + where + E: serde::de::Error, + { + i32::try_from(v) + .ok() + .and_then(|x| x.try_into().ok()) + .ok_or_else(|| { + serde::de::Error::invalid_value(serde::de::Unexpected::Unsigned(v), &self) + }) + } + + fn visit_str(self, value: &str) -> std::result::Result + where + E: serde::de::Error, + { + match value { + "LEFT_SIDE" => Ok(JoinSide::LeftSide), + "RIGHT_SIDE" => Ok(JoinSide::RightSide), + _ => Err(serde::de::Error::unknown_variant(value, FIELDS)), + } + } + } + deserializer.deserialize_any(GeneratedVisitor) + } +} +impl serde::Serialize for JoinType { + #[allow(deprecated)] + fn serialize(&self, serializer: S) -> std::result::Result + where + S: serde::Serializer, + { + let variant = match self { + Self::Inner => "INNER", + Self::Left => "LEFT", + Self::Right => "RIGHT", + Self::Full => "FULL", + Self::Leftsemi => "LEFTSEMI", + Self::Leftanti => "LEFTANTI", + Self::Rightsemi => "RIGHTSEMI", + Self::Rightanti => "RIGHTANTI", + }; + serializer.serialize_str(variant) + } +} +impl<'de> serde::Deserialize<'de> for JoinType { + #[allow(deprecated)] + fn deserialize(deserializer: D) -> std::result::Result + where + D: serde::Deserializer<'de>, + { + const FIELDS: &[&str] = &[ + "INNER", + "LEFT", + "RIGHT", + "FULL", + "LEFTSEMI", + "LEFTANTI", + "RIGHTSEMI", + "RIGHTANTI", + ]; + + struct GeneratedVisitor; + + impl<'de> serde::de::Visitor<'de> for GeneratedVisitor { + type Value = JoinType; + + fn expecting(&self, formatter: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + write!(formatter, "expected one of: {:?}", &FIELDS) + } + + fn visit_i64(self, v: i64) -> std::result::Result + where + E: serde::de::Error, + { + i32::try_from(v) + .ok() + .and_then(|x| x.try_into().ok()) + .ok_or_else(|| { + serde::de::Error::invalid_value(serde::de::Unexpected::Signed(v), &self) + }) + } + + fn visit_u64(self, v: u64) -> std::result::Result + where + E: serde::de::Error, + { + i32::try_from(v) + .ok() + .and_then(|x| x.try_into().ok()) + .ok_or_else(|| { + serde::de::Error::invalid_value(serde::de::Unexpected::Unsigned(v), &self) + }) + } + + fn visit_str(self, value: &str) -> std::result::Result + where + E: serde::de::Error, + { + match value { + "INNER" => Ok(JoinType::Inner), + "LEFT" => Ok(JoinType::Left), + "RIGHT" => Ok(JoinType::Right), + "FULL" => Ok(JoinType::Full), + "LEFTSEMI" => Ok(JoinType::Leftsemi), + "LEFTANTI" => Ok(JoinType::Leftanti), + "RIGHTSEMI" => Ok(JoinType::Rightsemi), + "RIGHTANTI" => Ok(JoinType::Rightanti), + _ => Err(serde::de::Error::unknown_variant(value, FIELDS)), + } + } + } + deserializer.deserialize_any(GeneratedVisitor) + } +} +impl serde::Serialize for JsonSink { + #[allow(deprecated)] + fn serialize(&self, serializer: S) -> std::result::Result + where + S: serde::Serializer, + { + use serde::ser::SerializeStruct; + let mut len = 0; + if self.config.is_some() { + len += 1; + } + let mut struct_ser = serializer.serialize_struct("datafusion.JsonSink", len)?; + if let Some(v) = self.config.as_ref() { + struct_ser.serialize_field("config", v)?; + } + struct_ser.end() + } +} +impl<'de> serde::Deserialize<'de> for JsonSink { + #[allow(deprecated)] + fn deserialize(deserializer: D) -> std::result::Result + where + D: serde::Deserializer<'de>, + { + const FIELDS: &[&str] = &[ + "config", + ]; + + #[allow(clippy::enum_variant_names)] + enum GeneratedField { + Config, + } + impl<'de> serde::Deserialize<'de> for GeneratedField { + fn deserialize(deserializer: D) -> std::result::Result + where + D: serde::Deserializer<'de>, + { + struct GeneratedVisitor; + + impl<'de> serde::de::Visitor<'de> for GeneratedVisitor { + type Value = GeneratedField; + + fn expecting(&self, formatter: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + write!(formatter, "expected one of: {:?}", &FIELDS) + } + + #[allow(unused_variables)] + fn visit_str(self, value: &str) -> std::result::Result + where + E: serde::de::Error, + { + match value { + "config" => Ok(GeneratedField::Config), + _ => Err(serde::de::Error::unknown_field(value, FIELDS)), + } + } + } + deserializer.deserialize_identifier(GeneratedVisitor) + } + } + struct GeneratedVisitor; + impl<'de> serde::de::Visitor<'de> for GeneratedVisitor { + type Value = JsonSink; + + fn expecting(&self, formatter: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + formatter.write_str("struct datafusion.JsonSink") + } + + fn visit_map(self, mut map_: V) -> std::result::Result + where + V: serde::de::MapAccess<'de>, + { + let mut config__ = None; + while let Some(k) = map_.next_key()? { + match k { + GeneratedField::Config => { + if config__.is_some() { + return Err(serde::de::Error::duplicate_field("config")); + } + config__ = map_.next_value()?; + } + } + } + Ok(JsonSink { + config: config__, + }) + } + } + deserializer.deserialize_struct("datafusion.JsonSink", FIELDS, GeneratedVisitor) + } +} +impl serde::Serialize for JsonSinkExecNode { + #[allow(deprecated)] + fn serialize(&self, serializer: S) -> std::result::Result + where + S: serde::Serializer, + { + use serde::ser::SerializeStruct; + let mut len = 0; + if self.input.is_some() { + len += 1; + } + if self.sink.is_some() { + len += 1; + } + if self.sink_schema.is_some() { + len += 1; + } + if self.sort_order.is_some() { + len += 1; + } + let mut struct_ser = serializer.serialize_struct("datafusion.JsonSinkExecNode", len)?; + if let Some(v) = self.input.as_ref() { + struct_ser.serialize_field("input", v)?; + } + if let Some(v) = self.sink.as_ref() { + struct_ser.serialize_field("sink", v)?; + } + if let Some(v) = self.sink_schema.as_ref() { + struct_ser.serialize_field("sinkSchema", v)?; + } + if let Some(v) = self.sort_order.as_ref() { + struct_ser.serialize_field("sortOrder", v)?; + } + struct_ser.end() + } +} +impl<'de> serde::Deserialize<'de> for JsonSinkExecNode { + #[allow(deprecated)] + fn deserialize(deserializer: D) -> std::result::Result + where + D: serde::Deserializer<'de>, + { + const FIELDS: &[&str] = &[ + "input", + "sink", + "sink_schema", + "sinkSchema", + "sort_order", + "sortOrder", + ]; + + #[allow(clippy::enum_variant_names)] + enum GeneratedField { + Input, + Sink, + SinkSchema, + SortOrder, + } + impl<'de> serde::Deserialize<'de> for GeneratedField { + fn deserialize(deserializer: D) -> std::result::Result + where + D: serde::Deserializer<'de>, + { + struct GeneratedVisitor; + + impl<'de> serde::de::Visitor<'de> for GeneratedVisitor { + type Value = GeneratedField; + + fn expecting(&self, formatter: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + write!(formatter, "expected one of: {:?}", &FIELDS) + } + + #[allow(unused_variables)] + fn visit_str(self, value: &str) -> std::result::Result + where + E: serde::de::Error, + { + match value { + "input" => Ok(GeneratedField::Input), + "sink" => Ok(GeneratedField::Sink), + "sinkSchema" | "sink_schema" => Ok(GeneratedField::SinkSchema), + "sortOrder" | "sort_order" => Ok(GeneratedField::SortOrder), + _ => Err(serde::de::Error::unknown_field(value, FIELDS)), + } + } + } + deserializer.deserialize_identifier(GeneratedVisitor) + } + } + struct GeneratedVisitor; + impl<'de> serde::de::Visitor<'de> for GeneratedVisitor { + type Value = JsonSinkExecNode; + + fn expecting(&self, formatter: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + formatter.write_str("struct datafusion.JsonSinkExecNode") } - fn visit_str(self, value: &str) -> std::result::Result - where - E: serde::de::Error, + fn visit_map(self, mut map_: V) -> std::result::Result + where + V: serde::de::MapAccess<'de>, { - match value { - "LEFT_SIDE" => Ok(JoinSide::LeftSide), - "RIGHT_SIDE" => Ok(JoinSide::RightSide), - _ => Err(serde::de::Error::unknown_variant(value, FIELDS)), + let mut input__ = None; + let mut sink__ = None; + let mut sink_schema__ = None; + let mut sort_order__ = None; + while let Some(k) = map_.next_key()? { + match k { + GeneratedField::Input => { + if input__.is_some() { + return Err(serde::de::Error::duplicate_field("input")); + } + input__ = map_.next_value()?; + } + GeneratedField::Sink => { + if sink__.is_some() { + return Err(serde::de::Error::duplicate_field("sink")); + } + sink__ = map_.next_value()?; + } + GeneratedField::SinkSchema => { + if sink_schema__.is_some() { + return Err(serde::de::Error::duplicate_field("sinkSchema")); + } + sink_schema__ = map_.next_value()?; + } + GeneratedField::SortOrder => { + if sort_order__.is_some() { + return Err(serde::de::Error::duplicate_field("sortOrder")); + } + sort_order__ = map_.next_value()?; + } + } } + Ok(JsonSinkExecNode { + input: input__, + sink: sink__, + sink_schema: sink_schema__, + sort_order: sort_order__, + }) } } - deserializer.deserialize_any(GeneratedVisitor) + deserializer.deserialize_struct("datafusion.JsonSinkExecNode", FIELDS, GeneratedVisitor) } } -impl serde::Serialize for JoinType { +impl serde::Serialize for JsonWriterOptions { #[allow(deprecated)] fn serialize(&self, serializer: S) -> std::result::Result where S: serde::Serializer, { - let variant = match self { - Self::Inner => "INNER", - Self::Left => "LEFT", - Self::Right => "RIGHT", - Self::Full => "FULL", - Self::Leftsemi => "LEFTSEMI", - Self::Leftanti => "LEFTANTI", - Self::Rightsemi => "RIGHTSEMI", - Self::Rightanti => "RIGHTANTI", - }; - serializer.serialize_str(variant) + use serde::ser::SerializeStruct; + let mut len = 0; + if self.compression != 0 { + len += 1; + } + let mut struct_ser = serializer.serialize_struct("datafusion.JsonWriterOptions", len)?; + if self.compression != 0 { + let v = CompressionTypeVariant::try_from(self.compression) + .map_err(|_| serde::ser::Error::custom(format!("Invalid variant {}", self.compression)))?; + struct_ser.serialize_field("compression", &v)?; + } + struct_ser.end() } } -impl<'de> serde::Deserialize<'de> for JoinType { +impl<'de> serde::Deserialize<'de> for JsonWriterOptions { #[allow(deprecated)] fn deserialize(deserializer: D) -> std::result::Result where D: serde::Deserializer<'de>, { const FIELDS: &[&str] = &[ - "INNER", - "LEFT", - "RIGHT", - "FULL", - "LEFTSEMI", - "LEFTANTI", - "RIGHTSEMI", - "RIGHTANTI", + "compression", ]; - struct GeneratedVisitor; + #[allow(clippy::enum_variant_names)] + enum GeneratedField { + Compression, + } + impl<'de> serde::Deserialize<'de> for GeneratedField { + fn deserialize(deserializer: D) -> std::result::Result + where + D: serde::Deserializer<'de>, + { + struct GeneratedVisitor; - impl<'de> serde::de::Visitor<'de> for GeneratedVisitor { - type Value = JoinType; + impl<'de> serde::de::Visitor<'de> for GeneratedVisitor { + type Value = GeneratedField; - fn expecting(&self, formatter: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { - write!(formatter, "expected one of: {:?}", &FIELDS) - } + fn expecting(&self, formatter: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + write!(formatter, "expected one of: {:?}", &FIELDS) + } - fn visit_i64(self, v: i64) -> std::result::Result - where - E: serde::de::Error, - { - i32::try_from(v) - .ok() - .and_then(|x| x.try_into().ok()) - .ok_or_else(|| { - serde::de::Error::invalid_value(serde::de::Unexpected::Signed(v), &self) - }) + #[allow(unused_variables)] + fn visit_str(self, value: &str) -> std::result::Result + where + E: serde::de::Error, + { + match value { + "compression" => Ok(GeneratedField::Compression), + _ => Err(serde::de::Error::unknown_field(value, FIELDS)), + } + } + } + deserializer.deserialize_identifier(GeneratedVisitor) } + } + struct GeneratedVisitor; + impl<'de> serde::de::Visitor<'de> for GeneratedVisitor { + type Value = JsonWriterOptions; - fn visit_u64(self, v: u64) -> std::result::Result - where - E: serde::de::Error, - { - i32::try_from(v) - .ok() - .and_then(|x| x.try_into().ok()) - .ok_or_else(|| { - serde::de::Error::invalid_value(serde::de::Unexpected::Unsigned(v), &self) - }) + fn expecting(&self, formatter: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + formatter.write_str("struct datafusion.JsonWriterOptions") } - fn visit_str(self, value: &str) -> std::result::Result - where - E: serde::de::Error, + fn visit_map(self, mut map_: V) -> std::result::Result + where + V: serde::de::MapAccess<'de>, { - match value { - "INNER" => Ok(JoinType::Inner), - "LEFT" => Ok(JoinType::Left), - "RIGHT" => Ok(JoinType::Right), - "FULL" => Ok(JoinType::Full), - "LEFTSEMI" => Ok(JoinType::Leftsemi), - "LEFTANTI" => Ok(JoinType::Leftanti), - "RIGHTSEMI" => Ok(JoinType::Rightsemi), - "RIGHTANTI" => Ok(JoinType::Rightanti), - _ => Err(serde::de::Error::unknown_variant(value, FIELDS)), + let mut compression__ = None; + while let Some(k) = map_.next_key()? { + match k { + GeneratedField::Compression => { + if compression__.is_some() { + return Err(serde::de::Error::duplicate_field("compression")); + } + compression__ = Some(map_.next_value::()? as i32); + } + } } + Ok(JsonWriterOptions { + compression: compression__.unwrap_or_default(), + }) } } - deserializer.deserialize_any(GeneratedVisitor) + deserializer.deserialize_struct("datafusion.JsonWriterOptions", FIELDS, GeneratedVisitor) } } impl serde::Serialize for LikeNode { @@ -14141,6 +14975,115 @@ impl<'de> serde::Deserialize<'de> for PartiallySortedPartitionSearchMode { deserializer.deserialize_struct("datafusion.PartiallySortedPartitionSearchMode", FIELDS, GeneratedVisitor) } } +impl serde::Serialize for PartitionColumn { + #[allow(deprecated)] + fn serialize(&self, serializer: S) -> std::result::Result + where + S: serde::Serializer, + { + use serde::ser::SerializeStruct; + let mut len = 0; + if !self.name.is_empty() { + len += 1; + } + if self.arrow_type.is_some() { + len += 1; + } + let mut struct_ser = serializer.serialize_struct("datafusion.PartitionColumn", len)?; + if !self.name.is_empty() { + struct_ser.serialize_field("name", &self.name)?; + } + if let Some(v) = self.arrow_type.as_ref() { + struct_ser.serialize_field("arrowType", v)?; + } + struct_ser.end() + } +} +impl<'de> serde::Deserialize<'de> for PartitionColumn { + #[allow(deprecated)] + fn deserialize(deserializer: D) -> std::result::Result + where + D: serde::Deserializer<'de>, + { + const FIELDS: &[&str] = &[ + "name", + "arrow_type", + "arrowType", + ]; + + #[allow(clippy::enum_variant_names)] + enum GeneratedField { + Name, + ArrowType, + } + impl<'de> serde::Deserialize<'de> for GeneratedField { + fn deserialize(deserializer: D) -> std::result::Result + where + D: serde::Deserializer<'de>, + { + struct GeneratedVisitor; + + impl<'de> serde::de::Visitor<'de> for GeneratedVisitor { + type Value = GeneratedField; + + fn expecting(&self, formatter: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + write!(formatter, "expected one of: {:?}", &FIELDS) + } + + #[allow(unused_variables)] + fn visit_str(self, value: &str) -> std::result::Result + where + E: serde::de::Error, + { + match value { + "name" => Ok(GeneratedField::Name), + "arrowType" | "arrow_type" => Ok(GeneratedField::ArrowType), + _ => Err(serde::de::Error::unknown_field(value, FIELDS)), + } + } + } + deserializer.deserialize_identifier(GeneratedVisitor) + } + } + struct GeneratedVisitor; + impl<'de> serde::de::Visitor<'de> for GeneratedVisitor { + type Value = PartitionColumn; + + fn expecting(&self, formatter: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + formatter.write_str("struct datafusion.PartitionColumn") + } + + fn visit_map(self, mut map_: V) -> std::result::Result + where + V: serde::de::MapAccess<'de>, + { + let mut name__ = None; + let mut arrow_type__ = None; + while let Some(k) = map_.next_key()? { + match k { + GeneratedField::Name => { + if name__.is_some() { + return Err(serde::de::Error::duplicate_field("name")); + } + name__ = Some(map_.next_value()?); + } + GeneratedField::ArrowType => { + if arrow_type__.is_some() { + return Err(serde::de::Error::duplicate_field("arrowType")); + } + arrow_type__ = map_.next_value()?; + } + } + } + Ok(PartitionColumn { + name: name__.unwrap_or_default(), + arrow_type: arrow_type__, + }) + } + } + deserializer.deserialize_struct("datafusion.PartitionColumn", FIELDS, GeneratedVisitor) + } +} impl serde::Serialize for PartitionMode { #[allow(deprecated)] fn serialize(&self, serializer: S) -> std::result::Result @@ -16812,6 +17755,9 @@ impl serde::Serialize for PhysicalPlanNode { physical_plan_node::PhysicalPlanType::Analyze(v) => { struct_ser.serialize_field("analyze", v)?; } + physical_plan_node::PhysicalPlanType::JsonSink(v) => { + struct_ser.serialize_field("jsonSink", v)?; + } } } struct_ser.end() @@ -16856,6 +17802,8 @@ impl<'de> serde::Deserialize<'de> for PhysicalPlanNode { "nested_loop_join", "nestedLoopJoin", "analyze", + "json_sink", + "jsonSink", ]; #[allow(clippy::enum_variant_names)] @@ -16882,6 +17830,7 @@ impl<'de> serde::Deserialize<'de> for PhysicalPlanNode { SortPreservingMerge, NestedLoopJoin, Analyze, + JsonSink, } impl<'de> serde::Deserialize<'de> for GeneratedField { fn deserialize(deserializer: D) -> std::result::Result @@ -16925,6 +17874,7 @@ impl<'de> serde::Deserialize<'de> for PhysicalPlanNode { "sortPreservingMerge" | "sort_preserving_merge" => Ok(GeneratedField::SortPreservingMerge), "nestedLoopJoin" | "nested_loop_join" => Ok(GeneratedField::NestedLoopJoin), "analyze" => Ok(GeneratedField::Analyze), + "jsonSink" | "json_sink" => Ok(GeneratedField::JsonSink), _ => Err(serde::de::Error::unknown_field(value, FIELDS)), } } @@ -17099,6 +18049,13 @@ impl<'de> serde::Deserialize<'de> for PhysicalPlanNode { return Err(serde::de::Error::duplicate_field("analyze")); } physical_plan_type__ = map_.next_value::<::std::option::Option<_>>()?.map(physical_plan_node::PhysicalPlanType::Analyze) +; + } + GeneratedField::JsonSink => { + if physical_plan_type__.is_some() { + return Err(serde::de::Error::duplicate_field("jsonSink")); + } + physical_plan_type__ = map_.next_value::<::std::option::Option<_>>()?.map(physical_plan_node::PhysicalPlanType::JsonSink) ; } } diff --git a/datafusion/proto/src/generated/prost.rs b/datafusion/proto/src/generated/prost.rs index d18bacfb3bcc..75050e9d3dfa 100644 --- a/datafusion/proto/src/generated/prost.rs +++ b/datafusion/proto/src/generated/prost.rs @@ -1486,7 +1486,7 @@ pub mod owned_table_reference { pub struct PhysicalPlanNode { #[prost( oneof = "physical_plan_node::PhysicalPlanType", - tags = "1, 2, 3, 4, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23" + tags = "1, 2, 3, 4, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24" )] pub physical_plan_type: ::core::option::Option, } @@ -1541,10 +1541,83 @@ pub mod physical_plan_node { NestedLoopJoin(::prost::alloc::boxed::Box), #[prost(message, tag = "23")] Analyze(::prost::alloc::boxed::Box), + #[prost(message, tag = "24")] + JsonSink(::prost::alloc::boxed::Box), + } +} +#[allow(clippy::derive_partial_eq_without_eq)] +#[derive(Clone, PartialEq, ::prost::Message)] +pub struct PartitionColumn { + #[prost(string, tag = "1")] + pub name: ::prost::alloc::string::String, + #[prost(message, optional, tag = "2")] + pub arrow_type: ::core::option::Option, +} +#[allow(clippy::derive_partial_eq_without_eq)] +#[derive(Clone, PartialEq, ::prost::Message)] +pub struct FileTypeWriterOptions { + #[prost(oneof = "file_type_writer_options::FileType", tags = "1")] + pub file_type: ::core::option::Option, +} +/// Nested message and enum types in `FileTypeWriterOptions`. +pub mod file_type_writer_options { + #[allow(clippy::derive_partial_eq_without_eq)] + #[derive(Clone, PartialEq, ::prost::Oneof)] + pub enum FileType { + #[prost(message, tag = "1")] + JsonOptions(super::JsonWriterOptions), } } #[allow(clippy::derive_partial_eq_without_eq)] #[derive(Clone, PartialEq, ::prost::Message)] +pub struct JsonWriterOptions { + #[prost(enumeration = "CompressionTypeVariant", tag = "1")] + pub compression: i32, +} +#[allow(clippy::derive_partial_eq_without_eq)] +#[derive(Clone, PartialEq, ::prost::Message)] +pub struct FileSinkConfig { + #[prost(string, tag = "1")] + pub object_store_url: ::prost::alloc::string::String, + #[prost(message, repeated, tag = "2")] + pub file_groups: ::prost::alloc::vec::Vec, + #[prost(string, repeated, tag = "3")] + pub table_paths: ::prost::alloc::vec::Vec<::prost::alloc::string::String>, + #[prost(message, optional, tag = "4")] + pub output_schema: ::core::option::Option, + #[prost(message, repeated, tag = "5")] + pub table_partition_cols: ::prost::alloc::vec::Vec, + #[prost(enumeration = "FileWriterMode", tag = "6")] + pub writer_mode: i32, + #[prost(bool, tag = "7")] + pub single_file_output: bool, + #[prost(bool, tag = "8")] + pub unbounded_input: bool, + #[prost(bool, tag = "9")] + pub overwrite: bool, + #[prost(message, optional, tag = "10")] + pub file_type_writer_options: ::core::option::Option, +} +#[allow(clippy::derive_partial_eq_without_eq)] +#[derive(Clone, PartialEq, ::prost::Message)] +pub struct JsonSink { + #[prost(message, optional, tag = "1")] + pub config: ::core::option::Option, +} +#[allow(clippy::derive_partial_eq_without_eq)] +#[derive(Clone, PartialEq, ::prost::Message)] +pub struct JsonSinkExecNode { + #[prost(message, optional, boxed, tag = "1")] + pub input: ::core::option::Option<::prost::alloc::boxed::Box>, + #[prost(message, optional, tag = "2")] + pub sink: ::core::option::Option, + #[prost(message, optional, tag = "3")] + pub sink_schema: ::core::option::Option, + #[prost(message, optional, tag = "4")] + pub sort_order: ::core::option::Option, +} +#[allow(clippy::derive_partial_eq_without_eq)] +#[derive(Clone, PartialEq, ::prost::Message)] pub struct PhysicalExtensionNode { #[prost(bytes = "vec", tag = "1")] pub node: ::prost::alloc::vec::Vec, @@ -3078,6 +3151,70 @@ impl UnionMode { } #[derive(Clone, Copy, Debug, PartialEq, Eq, Hash, PartialOrd, Ord, ::prost::Enumeration)] #[repr(i32)] +pub enum FileWriterMode { + Append = 0, + Put = 1, + PutMultipart = 2, +} +impl FileWriterMode { + /// String value of the enum field names used in the ProtoBuf definition. + /// + /// The values are not transformed in any way and thus are considered stable + /// (if the ProtoBuf definition does not change) and safe for programmatic use. + pub fn as_str_name(&self) -> &'static str { + match self { + FileWriterMode::Append => "APPEND", + FileWriterMode::Put => "PUT", + FileWriterMode::PutMultipart => "PUT_MULTIPART", + } + } + /// Creates an enum from field names used in the ProtoBuf definition. + pub fn from_str_name(value: &str) -> ::core::option::Option { + match value { + "APPEND" => Some(Self::Append), + "PUT" => Some(Self::Put), + "PUT_MULTIPART" => Some(Self::PutMultipart), + _ => None, + } + } +} +#[derive(Clone, Copy, Debug, PartialEq, Eq, Hash, PartialOrd, Ord, ::prost::Enumeration)] +#[repr(i32)] +pub enum CompressionTypeVariant { + Gzip = 0, + Bzip2 = 1, + Xz = 2, + Zstd = 3, + Uncompressed = 4, +} +impl CompressionTypeVariant { + /// String value of the enum field names used in the ProtoBuf definition. + /// + /// The values are not transformed in any way and thus are considered stable + /// (if the ProtoBuf definition does not change) and safe for programmatic use. + pub fn as_str_name(&self) -> &'static str { + match self { + CompressionTypeVariant::Gzip => "GZIP", + CompressionTypeVariant::Bzip2 => "BZIP2", + CompressionTypeVariant::Xz => "XZ", + CompressionTypeVariant::Zstd => "ZSTD", + CompressionTypeVariant::Uncompressed => "UNCOMPRESSED", + } + } + /// Creates an enum from field names used in the ProtoBuf definition. + pub fn from_str_name(value: &str) -> ::core::option::Option { + match value { + "GZIP" => Some(Self::Gzip), + "BZIP2" => Some(Self::Bzip2), + "XZ" => Some(Self::Xz), + "ZSTD" => Some(Self::Zstd), + "UNCOMPRESSED" => Some(Self::Uncompressed), + _ => None, + } + } +} +#[derive(Clone, Copy, Debug, PartialEq, Eq, Hash, PartialOrd, Ord, ::prost::Enumeration)] +#[repr(i32)] pub enum PartitionMode { CollectLeft = 0, Partitioned = 1, diff --git a/datafusion/proto/src/physical_plan/from_proto.rs b/datafusion/proto/src/physical_plan/from_proto.rs index a956eded9032..a628523f0e74 100644 --- a/datafusion/proto/src/physical_plan/from_proto.rs +++ b/datafusion/proto/src/physical_plan/from_proto.rs @@ -23,9 +23,11 @@ use std::sync::Arc; use arrow::compute::SortOptions; use datafusion::arrow::datatypes::Schema; -use datafusion::datasource::listing::{FileRange, PartitionedFile}; +use datafusion::datasource::file_format::json::JsonSink; +use datafusion::datasource::file_format::write::FileWriterMode; +use datafusion::datasource::listing::{FileRange, ListingTableUrl, PartitionedFile}; use datafusion::datasource::object_store::ObjectStoreUrl; -use datafusion::datasource::physical_plan::FileScanConfig; +use datafusion::datasource::physical_plan::{FileScanConfig, FileSinkConfig}; use datafusion::execution::context::ExecutionProps; use datafusion::execution::FunctionRegistry; use datafusion::logical_expr::window_function::WindowFunction; @@ -39,8 +41,12 @@ use datafusion::physical_plan::windows::create_window_expr; use datafusion::physical_plan::{ functions, ColumnStatistics, Partitioning, PhysicalExpr, Statistics, WindowExpr, }; +use datafusion_common::file_options::json_writer::JsonWriterOptions; +use datafusion_common::parsers::CompressionTypeVariant; use datafusion_common::stats::Precision; -use datafusion_common::{not_impl_err, DataFusionError, JoinSide, Result, ScalarValue}; +use datafusion_common::{ + not_impl_err, DataFusionError, FileTypeWriterOptions, JoinSide, Result, ScalarValue, +}; use crate::common::proto_error; use crate::convert_required; @@ -697,3 +703,86 @@ impl TryFrom<&protobuf::Statistics> for Statistics { }) } } + +impl TryFrom<&protobuf::JsonSink> for JsonSink { + type Error = DataFusionError; + + fn try_from(value: &protobuf::JsonSink) -> Result { + Ok(Self::new(convert_required!(value.config)?)) + } +} + +impl TryFrom<&protobuf::FileSinkConfig> for FileSinkConfig { + type Error = DataFusionError; + + fn try_from(conf: &protobuf::FileSinkConfig) -> Result { + let file_groups = conf + .file_groups + .iter() + .map(TryInto::try_into) + .collect::>>()?; + let table_paths = conf + .table_paths + .iter() + .map(ListingTableUrl::parse) + .collect::>>()?; + let table_partition_cols = conf + .table_partition_cols + .iter() + .map(|protobuf::PartitionColumn { name, arrow_type }| { + let data_type = convert_required!(arrow_type)?; + Ok((name.clone(), data_type)) + }) + .collect::>>()?; + Ok(Self { + object_store_url: ObjectStoreUrl::parse(&conf.object_store_url)?, + file_groups, + table_paths, + output_schema: Arc::new(convert_required!(conf.output_schema)?), + table_partition_cols, + writer_mode: conf.writer_mode().into(), + single_file_output: conf.single_file_output, + unbounded_input: conf.unbounded_input, + overwrite: conf.overwrite, + file_type_writer_options: convert_required!(conf.file_type_writer_options)?, + }) + } +} + +impl From for FileWriterMode { + fn from(value: protobuf::FileWriterMode) -> Self { + match value { + protobuf::FileWriterMode::Append => Self::Append, + protobuf::FileWriterMode::Put => Self::Put, + protobuf::FileWriterMode::PutMultipart => Self::PutMultipart, + } + } +} + +impl From for CompressionTypeVariant { + fn from(value: protobuf::CompressionTypeVariant) -> Self { + match value { + protobuf::CompressionTypeVariant::Gzip => Self::GZIP, + protobuf::CompressionTypeVariant::Bzip2 => Self::BZIP2, + protobuf::CompressionTypeVariant::Xz => Self::XZ, + protobuf::CompressionTypeVariant::Zstd => Self::ZSTD, + protobuf::CompressionTypeVariant::Uncompressed => Self::UNCOMPRESSED, + } + } +} + +impl TryFrom<&protobuf::FileTypeWriterOptions> for FileTypeWriterOptions { + type Error = DataFusionError; + + fn try_from(value: &protobuf::FileTypeWriterOptions) -> Result { + let file_type = value + .file_type + .as_ref() + .ok_or_else(|| proto_error("Missing required field in protobuf"))?; + match file_type { + protobuf::file_type_writer_options::FileType::JsonOptions(opts) => Ok( + Self::JSON(JsonWriterOptions::new(opts.compression().into())), + ), + } + } +} diff --git a/datafusion/proto/src/physical_plan/mod.rs b/datafusion/proto/src/physical_plan/mod.rs index 431b8e42cdaf..1eedbe987ec1 100644 --- a/datafusion/proto/src/physical_plan/mod.rs +++ b/datafusion/proto/src/physical_plan/mod.rs @@ -22,6 +22,7 @@ use std::sync::Arc; use datafusion::arrow::compute::SortOptions; use datafusion::arrow::datatypes::SchemaRef; use datafusion::datasource::file_format::file_compression_type::FileCompressionType; +use datafusion::datasource::file_format::json::JsonSink; #[cfg(feature = "parquet")] use datafusion::datasource::physical_plan::ParquetExec; use datafusion::datasource::physical_plan::{AvroExec, CsvExec}; @@ -36,6 +37,7 @@ use datafusion::physical_plan::empty::EmptyExec; use datafusion::physical_plan::explain::ExplainExec; use datafusion::physical_plan::expressions::{Column, PhysicalSortExpr}; use datafusion::physical_plan::filter::FilterExec; +use datafusion::physical_plan::insert::FileSinkExec; use datafusion::physical_plan::joins::utils::{ColumnIndex, JoinFilter}; use datafusion::physical_plan::joins::{CrossJoinExec, NestedLoopJoinExec}; use datafusion::physical_plan::joins::{HashJoinExec, PartitionMode}; @@ -64,7 +66,9 @@ use crate::protobuf::physical_aggregate_expr_node::AggregateFunction; use crate::protobuf::physical_expr_node::ExprType; use crate::protobuf::physical_plan_node::PhysicalPlanType; use crate::protobuf::repartition_exec_node::PartitionMethod; -use crate::protobuf::{self, window_agg_exec_node, PhysicalPlanNode}; +use crate::protobuf::{ + self, window_agg_exec_node, PhysicalPlanNode, PhysicalSortExprNodeCollection, +}; use crate::{convert_required, into_required}; use self::from_proto::parse_physical_window_expr; @@ -782,7 +786,38 @@ impl AsExecutionPlan for PhysicalPlanNode { analyze.verbose, analyze.show_statistics, input, - Arc::new(analyze.schema.as_ref().unwrap().try_into()?), + Arc::new(convert_required!(analyze.schema)?), + ))) + } + PhysicalPlanType::JsonSink(sink) => { + let input = + into_physical_plan(&sink.input, registry, runtime, extension_codec)?; + + let data_sink: JsonSink = sink + .sink + .as_ref() + .ok_or_else(|| proto_error("Missing required field in protobuf"))? + .try_into()?; + let sink_schema = convert_required!(sink.sink_schema)?; + let sort_order = sink + .sort_order + .as_ref() + .map(|collection| { + collection + .physical_sort_expr_nodes + .iter() + .map(|proto| { + parse_physical_sort_expr(proto, registry, &sink_schema) + .map(Into::into) + }) + .collect::>>() + }) + .transpose()?; + Ok(Arc::new(FileSinkExec::new( + input, + Arc::new(data_sink), + Arc::new(sink_schema), + sort_order, ))) } } @@ -1415,6 +1450,48 @@ impl AsExecutionPlan for PhysicalPlanNode { }); } + if let Some(exec) = plan.downcast_ref::() { + let input = protobuf::PhysicalPlanNode::try_from_physical_plan( + exec.input().to_owned(), + extension_codec, + )?; + let sort_order = match exec.sort_order() { + Some(requirements) => { + let expr = requirements + .iter() + .map(|requirement| { + let expr: PhysicalSortExpr = requirement.to_owned().into(); + let sort_expr = protobuf::PhysicalSortExprNode { + expr: Some(Box::new(expr.expr.to_owned().try_into()?)), + asc: !expr.options.descending, + nulls_first: expr.options.nulls_first, + }; + Ok(sort_expr) + }) + .collect::>>()?; + Some(PhysicalSortExprNodeCollection { + physical_sort_expr_nodes: expr, + }) + } + None => None, + }; + + if let Some(sink) = exec.sink().as_any().downcast_ref::() { + return Ok(protobuf::PhysicalPlanNode { + physical_plan_type: Some(PhysicalPlanType::JsonSink(Box::new( + protobuf::JsonSinkExecNode { + input: Some(Box::new(input)), + sink: Some(sink.try_into()?), + sink_schema: Some(exec.schema().as_ref().try_into()?), + sort_order, + }, + ))), + }); + } + + // If unknown DataSink then let extension handle it + } + let mut buf: Vec = vec![]; match extension_codec.try_encode(plan_clone.clone(), &mut buf) { Ok(_) => { diff --git a/datafusion/proto/src/physical_plan/to_proto.rs b/datafusion/proto/src/physical_plan/to_proto.rs index 114baab6ccc4..8201ef86b528 100644 --- a/datafusion/proto/src/physical_plan/to_proto.rs +++ b/datafusion/proto/src/physical_plan/to_proto.rs @@ -27,9 +27,14 @@ use crate::protobuf::{ physical_aggregate_expr_node, PhysicalSortExprNode, PhysicalSortExprNodeCollection, ScalarValue, }; - -use datafusion::datasource::listing::{FileRange, PartitionedFile}; -use datafusion::datasource::physical_plan::FileScanConfig; +use datafusion::datasource::{ + file_format::json::JsonSink, physical_plan::FileScanConfig, +}; +use datafusion::datasource::{ + file_format::write::FileWriterMode, + listing::{FileRange, PartitionedFile}, + physical_plan::FileSinkConfig, +}; use datafusion::logical_expr::BuiltinScalarFunction; use datafusion::physical_expr::expressions::{GetFieldAccessExpr, GetIndexedFieldExpr}; use datafusion::physical_expr::window::{NthValueKind, SlidingAggregateWindowExpr}; @@ -50,7 +55,15 @@ use datafusion::physical_plan::{ AggregateExpr, ColumnStatistics, PhysicalExpr, Statistics, WindowExpr, }; use datafusion_common::{ - internal_err, not_impl_err, stats::Precision, DataFusionError, JoinSide, Result, + file_options::{ + arrow_writer::ArrowWriterOptions, avro_writer::AvroWriterOptions, + csv_writer::CsvWriterOptions, json_writer::JsonWriterOptions, + parquet_writer::ParquetWriterOptions, + }, + internal_err, not_impl_err, + parsers::CompressionTypeVariant, + stats::Precision, + DataFusionError, FileTypeWriterOptions, JoinSide, Result, }; impl TryFrom> for protobuf::PhysicalExprNode { @@ -790,3 +803,110 @@ impl TryFrom for protobuf::PhysicalSortExprNode { }) } } + +impl TryFrom<&JsonSink> for protobuf::JsonSink { + type Error = DataFusionError; + + fn try_from(value: &JsonSink) -> Result { + Ok(Self { + config: Some(value.config().try_into()?), + }) + } +} + +impl TryFrom<&FileSinkConfig> for protobuf::FileSinkConfig { + type Error = DataFusionError; + + fn try_from(conf: &FileSinkConfig) -> Result { + let writer_mode: protobuf::FileWriterMode = conf.writer_mode.into(); + let file_groups = conf + .file_groups + .iter() + .map(TryInto::try_into) + .collect::>>()?; + let table_paths = conf + .table_paths + .iter() + .map(ToString::to_string) + .collect::>(); + let table_partition_cols = conf + .table_partition_cols + .iter() + .map(|(name, data_type)| { + Ok(protobuf::PartitionColumn { + name: name.to_owned(), + arrow_type: Some(data_type.try_into()?), + }) + }) + .collect::>>()?; + let file_type_writer_options = &conf.file_type_writer_options; + Ok(Self { + object_store_url: conf.object_store_url.to_string(), + file_groups, + table_paths, + output_schema: Some(conf.output_schema.as_ref().try_into()?), + table_partition_cols, + writer_mode: writer_mode.into(), + single_file_output: conf.single_file_output, + unbounded_input: conf.unbounded_input, + overwrite: conf.overwrite, + file_type_writer_options: Some(file_type_writer_options.try_into()?), + }) + } +} + +impl From for protobuf::FileWriterMode { + fn from(value: FileWriterMode) -> Self { + match value { + FileWriterMode::Append => Self::Append, + FileWriterMode::Put => Self::Put, + FileWriterMode::PutMultipart => Self::PutMultipart, + } + } +} + +impl From<&CompressionTypeVariant> for protobuf::CompressionTypeVariant { + fn from(value: &CompressionTypeVariant) -> Self { + match value { + CompressionTypeVariant::GZIP => Self::Gzip, + CompressionTypeVariant::BZIP2 => Self::Bzip2, + CompressionTypeVariant::XZ => Self::Xz, + CompressionTypeVariant::ZSTD => Self::Zstd, + CompressionTypeVariant::UNCOMPRESSED => Self::Uncompressed, + } + } +} + +impl TryFrom<&FileTypeWriterOptions> for protobuf::FileTypeWriterOptions { + type Error = DataFusionError; + + fn try_from(opts: &FileTypeWriterOptions) -> Result { + let file_type = match opts { + #[cfg(feature = "parquet")] + FileTypeWriterOptions::Parquet(ParquetWriterOptions { + writer_options: _, + }) => return not_impl_err!("Parquet file sink protobuf serialization"), + FileTypeWriterOptions::CSV(CsvWriterOptions { + writer_options: _, + compression: _, + }) => return not_impl_err!("CSV file sink protobuf serialization"), + FileTypeWriterOptions::JSON(JsonWriterOptions { compression }) => { + let compression: protobuf::CompressionTypeVariant = compression.into(); + protobuf::file_type_writer_options::FileType::JsonOptions( + protobuf::JsonWriterOptions { + compression: compression.into(), + }, + ) + } + FileTypeWriterOptions::Avro(AvroWriterOptions {}) => { + return not_impl_err!("Avro file sink protobuf serialization") + } + FileTypeWriterOptions::Arrow(ArrowWriterOptions {}) => { + return not_impl_err!("Arrow file sink protobuf serialization") + } + }; + Ok(Self { + file_type: Some(file_type), + }) + } +} diff --git a/datafusion/proto/tests/cases/roundtrip_physical_plan.rs b/datafusion/proto/tests/cases/roundtrip_physical_plan.rs index 01a0916d8cd2..81e66d5ead36 100644 --- a/datafusion/proto/tests/cases/roundtrip_physical_plan.rs +++ b/datafusion/proto/tests/cases/roundtrip_physical_plan.rs @@ -21,15 +21,19 @@ use std::sync::Arc; use datafusion::arrow::array::ArrayRef; use datafusion::arrow::compute::kernels::sort::SortOptions; use datafusion::arrow::datatypes::{DataType, Field, Fields, IntervalUnit, Schema}; -use datafusion::datasource::listing::PartitionedFile; +use datafusion::datasource::file_format::json::JsonSink; +use datafusion::datasource::file_format::write::FileWriterMode; +use datafusion::datasource::listing::{ListingTableUrl, PartitionedFile}; use datafusion::datasource::object_store::ObjectStoreUrl; -use datafusion::datasource::physical_plan::{FileScanConfig, ParquetExec}; +use datafusion::datasource::physical_plan::{ + FileScanConfig, FileSinkConfig, ParquetExec, +}; use datafusion::execution::context::ExecutionProps; use datafusion::logical_expr::{ create_udf, BuiltinScalarFunction, JoinType, Operator, Volatility, }; use datafusion::physical_expr::window::SlidingAggregateWindowExpr; -use datafusion::physical_expr::ScalarFunctionExpr; +use datafusion::physical_expr::{PhysicalSortRequirement, ScalarFunctionExpr}; use datafusion::physical_plan::aggregates::{ AggregateExec, AggregateMode, PhysicalGroupBy, }; @@ -41,6 +45,7 @@ use datafusion::physical_plan::expressions::{ }; use datafusion::physical_plan::filter::FilterExec; use datafusion::physical_plan::functions::make_scalar_function; +use datafusion::physical_plan::insert::FileSinkExec; use datafusion::physical_plan::joins::{HashJoinExec, NestedLoopJoinExec, PartitionMode}; use datafusion::physical_plan::limit::{GlobalLimitExec, LocalLimitExec}; use datafusion::physical_plan::projection::ProjectionExec; @@ -53,8 +58,10 @@ use datafusion::physical_plan::{ }; use datafusion::prelude::SessionContext; use datafusion::scalar::ScalarValue; +use datafusion_common::file_options::json_writer::JsonWriterOptions; +use datafusion_common::parsers::CompressionTypeVariant; use datafusion_common::stats::Precision; -use datafusion_common::Result; +use datafusion_common::{FileTypeWriterOptions, Result}; use datafusion_expr::{ Accumulator, AccumulatorFactoryFunction, AggregateUDF, ReturnTypeFunction, Signature, StateTypeFunction, WindowFrame, WindowFrameBound, @@ -698,7 +705,7 @@ fn roundtrip_get_indexed_field_list_range() -> Result<()> { } #[test] -fn rountrip_analyze() -> Result<()> { +fn roundtrip_analyze() -> Result<()> { let field_a = Field::new("plan_type", DataType::Utf8, false); let field_b = Field::new("plan", DataType::Utf8, false); let schema = Schema::new(vec![field_a, field_b]); @@ -711,3 +718,41 @@ fn rountrip_analyze() -> Result<()> { Arc::new(schema), ))) } + +#[test] +fn roundtrip_json_sink() -> Result<()> { + let field_a = Field::new("plan_type", DataType::Utf8, false); + let field_b = Field::new("plan", DataType::Utf8, false); + let schema = Arc::new(Schema::new(vec![field_a, field_b])); + let input = Arc::new(EmptyExec::new(true, schema.clone())); + + let file_sink_config = FileSinkConfig { + object_store_url: ObjectStoreUrl::local_filesystem(), + file_groups: vec![PartitionedFile::new("/tmp".to_string(), 1)], + table_paths: vec![ListingTableUrl::parse("file:///")?], + output_schema: schema.clone(), + table_partition_cols: vec![("plan_type".to_string(), DataType::Utf8)], + writer_mode: FileWriterMode::Put, + single_file_output: true, + unbounded_input: false, + overwrite: true, + file_type_writer_options: FileTypeWriterOptions::JSON(JsonWriterOptions::new( + CompressionTypeVariant::UNCOMPRESSED, + )), + }; + let data_sink = Arc::new(JsonSink::new(file_sink_config)); + let sort_order = vec![PhysicalSortRequirement::new( + Arc::new(Column::new("plan_type", 0)), + Some(SortOptions { + descending: true, + nulls_first: false, + }), + )]; + + roundtrip_test(Arc::new(FileSinkExec::new( + input, + data_sink, + schema.clone(), + Some(sort_order), + ))) +} diff --git a/datafusion/sqllogictest/test_files/copy.slt b/datafusion/sqllogictest/test_files/copy.slt index f2fe216ee864..6e4a711a0115 100644 --- a/datafusion/sqllogictest/test_files/copy.slt +++ b/datafusion/sqllogictest/test_files/copy.slt @@ -32,7 +32,7 @@ logical_plan CopyTo: format=parquet output_url=test_files/scratch/copy/table single_file_output=false options: (compression 'zstd(10)') --TableScan: source_table projection=[col1, col2] physical_plan -InsertExec: sink=ParquetSink(writer_mode=PutMultipart, file_groups=[]) +FileSinkExec: sink=ParquetSink(writer_mode=PutMultipart, file_groups=[]) --MemoryExec: partitions=1, partition_sizes=[1] # Error case diff --git a/datafusion/sqllogictest/test_files/explain.slt b/datafusion/sqllogictest/test_files/explain.slt index 40a6d4357488..d28f9fc6e372 100644 --- a/datafusion/sqllogictest/test_files/explain.slt +++ b/datafusion/sqllogictest/test_files/explain.slt @@ -168,7 +168,7 @@ Dml: op=[Insert Into] table=[sink_table] ----Sort: aggregate_test_100.c1 ASC NULLS LAST ------TableScan: aggregate_test_100 projection=[c1, c2, c3, c4, c5, c6, c7, c8, c9, c10, c11, c12, c13] physical_plan -InsertExec: sink=CsvSink(writer_mode=Append, file_groups=[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]) +FileSinkExec: sink=CsvSink(writer_mode=Append, file_groups=[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]) --ProjectionExec: expr=[c1@0 as c1, c2@1 as c2, c3@2 as c3, c4@3 as c4, c5@4 as c5, c6@5 as c6, c7@6 as c7, c8@7 as c8, c9@8 as c9, c10@9 as c10, c11@10 as c11, c12@11 as c12, c13@12 as c13] ----SortExec: expr=[c1@0 ASC NULLS LAST] ------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1, c2, c3, c4, c5, c6, c7, c8, c9, c10, c11, c12, c13], has_header=true diff --git a/datafusion/sqllogictest/test_files/insert.slt b/datafusion/sqllogictest/test_files/insert.slt index cc04c6227721..0c63a3481996 100644 --- a/datafusion/sqllogictest/test_files/insert.slt +++ b/datafusion/sqllogictest/test_files/insert.slt @@ -64,7 +64,7 @@ Dml: op=[Insert Into] table=[table_without_values] --------WindowAggr: windowExpr=[[SUM(CAST(aggregate_test_100.c4 AS Int64)) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING, COUNT(UInt8(1)) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING AS COUNT(*) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING]] ----------TableScan: aggregate_test_100 projection=[c1, c4, c9] physical_plan -InsertExec: sink=MemoryTable (partitions=1) +FileSinkExec: sink=MemoryTable (partitions=1) --ProjectionExec: expr=[SUM(aggregate_test_100.c4) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING@0 as field1, COUNT(*) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING@1 as field2] ----SortPreservingMergeExec: [c1@2 ASC NULLS LAST] ------ProjectionExec: expr=[SUM(aggregate_test_100.c4) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING@3 as SUM(aggregate_test_100.c4) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING, COUNT(*) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING@4 as COUNT(*) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING, c1@0 as c1] @@ -125,7 +125,7 @@ Dml: op=[Insert Into] table=[table_without_values] ----WindowAggr: windowExpr=[[SUM(CAST(aggregate_test_100.c4 AS Int64)) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING, COUNT(UInt8(1)) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING AS COUNT(*) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING]] ------TableScan: aggregate_test_100 projection=[c1, c4, c9] physical_plan -InsertExec: sink=MemoryTable (partitions=1) +FileSinkExec: sink=MemoryTable (partitions=1) --CoalescePartitionsExec ----ProjectionExec: expr=[SUM(aggregate_test_100.c4) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING@3 as field1, COUNT(*) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING@4 as field2] ------BoundedWindowAggExec: wdw=[SUM(aggregate_test_100.c4) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "SUM(aggregate_test_100.c4) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(1)), end_bound: Following(UInt64(1)) }, COUNT(*) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "COUNT(*) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(1)), end_bound: Following(UInt64(1)) }], mode=[Sorted] @@ -175,7 +175,7 @@ Dml: op=[Insert Into] table=[table_without_values] --------WindowAggr: windowExpr=[[SUM(CAST(aggregate_test_100.c4 AS Int64)) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING, COUNT(UInt8(1)) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING AS COUNT(*) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING]] ----------TableScan: aggregate_test_100 projection=[c1, c4, c9] physical_plan -InsertExec: sink=MemoryTable (partitions=8) +FileSinkExec: sink=MemoryTable (partitions=8) --ProjectionExec: expr=[a1@0 as a1, a2@1 as a2] ----SortPreservingMergeExec: [c1@2 ASC NULLS LAST] ------ProjectionExec: expr=[SUM(aggregate_test_100.c4) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING@3 as a1, COUNT(*) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING@4 as a2, c1@0 as c1] @@ -217,7 +217,7 @@ Dml: op=[Insert Into] table=[table_without_values] ----Sort: aggregate_test_100.c1 ASC NULLS LAST ------TableScan: aggregate_test_100 projection=[c1] physical_plan -InsertExec: sink=MemoryTable (partitions=1) +FileSinkExec: sink=MemoryTable (partitions=1) --ProjectionExec: expr=[c1@0 as c1] ----SortExec: expr=[c1@0 ASC NULLS LAST] ------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1], has_header=true diff --git a/datafusion/sqllogictest/test_files/insert_to_external.slt b/datafusion/sqllogictest/test_files/insert_to_external.slt index 8b01a14568e7..fa1d646d1413 100644 --- a/datafusion/sqllogictest/test_files/insert_to_external.slt +++ b/datafusion/sqllogictest/test_files/insert_to_external.slt @@ -100,7 +100,7 @@ Dml: op=[Insert Into] table=[ordered_insert_test] --Projection: column1 AS a, column2 AS b ----Values: (Int64(5), Int64(1)), (Int64(4), Int64(2)), (Int64(7), Int64(7)), (Int64(7), Int64(8)), (Int64(7), Int64(9))... physical_plan -InsertExec: sink=CsvSink(writer_mode=PutMultipart, file_groups=[]) +FileSinkExec: sink=CsvSink(writer_mode=PutMultipart, file_groups=[]) --SortExec: expr=[a@0 ASC NULLS LAST,b@1 DESC] ----ProjectionExec: expr=[column1@0 as a, column2@1 as b] ------ValuesExec @@ -315,7 +315,7 @@ Dml: op=[Insert Into] table=[table_without_values] --------WindowAggr: windowExpr=[[SUM(CAST(aggregate_test_100.c4 AS Int64)) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING, COUNT(UInt8(1)) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING AS COUNT(*) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING]] ----------TableScan: aggregate_test_100 projection=[c1, c4, c9] physical_plan -InsertExec: sink=ParquetSink(writer_mode=PutMultipart, file_groups=[]) +FileSinkExec: sink=ParquetSink(writer_mode=PutMultipart, file_groups=[]) --ProjectionExec: expr=[SUM(aggregate_test_100.c4) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING@0 as field1, COUNT(*) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING@1 as field2] ----SortPreservingMergeExec: [c1@2 ASC NULLS LAST] ------ProjectionExec: expr=[SUM(aggregate_test_100.c4) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING@3 as SUM(aggregate_test_100.c4) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING, COUNT(*) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING@4 as COUNT(*) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING, c1@0 as c1] @@ -378,7 +378,7 @@ Dml: op=[Insert Into] table=[table_without_values] ----WindowAggr: windowExpr=[[SUM(CAST(aggregate_test_100.c4 AS Int64)) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING, COUNT(UInt8(1)) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING AS COUNT(*) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING]] ------TableScan: aggregate_test_100 projection=[c1, c4, c9] physical_plan -InsertExec: sink=ParquetSink(writer_mode=PutMultipart, file_groups=[]) +FileSinkExec: sink=ParquetSink(writer_mode=PutMultipart, file_groups=[]) --CoalescePartitionsExec ----ProjectionExec: expr=[SUM(aggregate_test_100.c4) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING@3 as field1, COUNT(*) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING@4 as field2] ------BoundedWindowAggExec: wdw=[SUM(aggregate_test_100.c4) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "SUM(aggregate_test_100.c4) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(1)), end_bound: Following(UInt64(1)) }, COUNT(*) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "COUNT(*) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(1)), end_bound: Following(UInt64(1)) }], mode=[Sorted] @@ -422,7 +422,7 @@ Dml: op=[Insert Into] table=[table_without_values] ----Sort: aggregate_test_100.c1 ASC NULLS LAST ------TableScan: aggregate_test_100 projection=[c1] physical_plan -InsertExec: sink=ParquetSink(writer_mode=PutMultipart, file_groups=[]) +FileSinkExec: sink=ParquetSink(writer_mode=PutMultipart, file_groups=[]) --ProjectionExec: expr=[c1@0 as c1] ----SortExec: expr=[c1@0 ASC NULLS LAST] ------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1], has_header=true From a70369ca9a5f07168f3fa3ec93bef0b1b0141179 Mon Sep 17 00:00:00 2001 From: Andrew Lamb Date: Thu, 9 Nov 2023 00:45:53 -0500 Subject: [PATCH 209/572] Minor: use `Expr::alias` in a few places to make the code more concise (#8097) --- datafusion/optimizer/src/decorrelate.rs | 7 +++---- .../src/single_distinct_to_groupby.rs | 20 +++++++++---------- datafusion/sql/src/select.rs | 2 +- 3 files changed, 13 insertions(+), 16 deletions(-) diff --git a/datafusion/optimizer/src/decorrelate.rs b/datafusion/optimizer/src/decorrelate.rs index b5cf73733896..c8162683f39e 100644 --- a/datafusion/optimizer/src/decorrelate.rs +++ b/datafusion/optimizer/src/decorrelate.rs @@ -227,10 +227,9 @@ impl TreeNodeRewriter for PullUpCorrelatedExpr { )?; if !expr_result_map_for_count_bug.is_empty() { // has count bug - let un_matched_row = Expr::Alias(Alias::new( - Expr::Literal(ScalarValue::Boolean(Some(true))), - UN_MATCHED_ROW_INDICATOR.to_string(), - )); + let un_matched_row = + Expr::Literal(ScalarValue::Boolean(Some(true))) + .alias(UN_MATCHED_ROW_INDICATOR); // add the unmatched rows indicator to the Aggregation's group expressions missing_exprs.push(un_matched_row); } diff --git a/datafusion/optimizer/src/single_distinct_to_groupby.rs b/datafusion/optimizer/src/single_distinct_to_groupby.rs index 548f00b4138a..414217612d1e 100644 --- a/datafusion/optimizer/src/single_distinct_to_groupby.rs +++ b/datafusion/optimizer/src/single_distinct_to_groupby.rs @@ -25,7 +25,7 @@ use crate::{OptimizerConfig, OptimizerRule}; use datafusion_common::Result; use datafusion_expr::{ col, - expr::{AggregateFunction, Alias}, + expr::AggregateFunction, logical_plan::{Aggregate, LogicalPlan}, Expr, }; @@ -168,16 +168,14 @@ impl OptimizerRule for SingleDistinctToGroupBy { args[0].clone().alias(SINGLE_DISTINCT_ALIAS), ); } - Ok(Expr::Alias(Alias::new( - Expr::AggregateFunction(AggregateFunction::new( - fun.clone(), - vec![col(SINGLE_DISTINCT_ALIAS)], - false, // intentional to remove distinct here - filter.clone(), - order_by.clone(), - )), - aggr_expr.display_name()?, - ))) + Ok(Expr::AggregateFunction(AggregateFunction::new( + fun.clone(), + vec![col(SINGLE_DISTINCT_ALIAS)], + false, // intentional to remove distinct here + filter.clone(), + order_by.clone(), + )) + .alias(aggr_expr.display_name()?)) } _ => Ok(aggr_expr.clone()), }) diff --git a/datafusion/sql/src/select.rs b/datafusion/sql/src/select.rs index 2062afabfc1a..e9a7941ab064 100644 --- a/datafusion/sql/src/select.rs +++ b/datafusion/sql/src/select.rs @@ -373,7 +373,7 @@ impl<'a, S: ContextProvider> SqlToRel<'a, S> { &[&[plan.schema()]], &plan.using_columns()?, )?; - let expr = Expr::Alias(Alias::new(col, self.normalizer.normalize(alias))); + let expr = col.alias(self.normalizer.normalize(alias)); Ok(vec![expr]) } SelectItem::Wildcard(options) => { From 2e384898e8bacf67033276db33b62a7d622b50e0 Mon Sep 17 00:00:00 2001 From: Yongting You <2010youy01@gmail.com> Date: Wed, 8 Nov 2023 21:50:09 -0800 Subject: [PATCH 210/572] Minor: Cleanup BuiltinScalarFunction::return_type() (#8088) --- datafusion/expr/src/built_in_function.rs | 38 +++++-------------- datafusion/expr/src/expr_schema.rs | 19 ++++++++-- .../expr/src/type_coercion/functions.rs | 11 +++++- datafusion/physical-expr/src/functions.rs | 15 ++++---- 4 files changed, 42 insertions(+), 41 deletions(-) diff --git a/datafusion/expr/src/built_in_function.rs b/datafusion/expr/src/built_in_function.rs index 1ebd9cc0187a..f3f52e9dafb6 100644 --- a/datafusion/expr/src/built_in_function.rs +++ b/datafusion/expr/src/built_in_function.rs @@ -28,14 +28,12 @@ use crate::signature::TIMEZONE_WILDCARD; use crate::type_coercion::binary::get_wider_type; use crate::type_coercion::functions::data_types; use crate::{ - conditional_expressions, struct_expressions, utils, FuncMonotonicity, Signature, + conditional_expressions, struct_expressions, FuncMonotonicity, Signature, TypeSignature, Volatility, }; use arrow::datatypes::{DataType, Field, Fields, IntervalUnit, TimeUnit}; -use datafusion_common::{ - internal_err, plan_datafusion_err, plan_err, DataFusionError, Result, -}; +use datafusion_common::{internal_err, plan_err, DataFusionError, Result}; use strum::IntoEnumIterator; use strum_macros::EnumIter; @@ -483,6 +481,13 @@ impl BuiltinScalarFunction { } /// Returns the output [`DataType`] of this function + /// + /// This method should be invoked only after `input_expr_types` have been validated + /// against the function's `TypeSignature` using `type_coercion::functions::data_types()`. + /// + /// This method will: + /// 1. Perform additional checks on `input_expr_types` that are beyond the scope of `TypeSignature` validation. + /// 2. Deduce the output `DataType` based on the provided `input_expr_types`. pub fn return_type(self, input_expr_types: &[DataType]) -> Result { use DataType::*; use TimeUnit::*; @@ -490,31 +495,6 @@ impl BuiltinScalarFunction { // Note that this function *must* return the same type that the respective physical expression returns // or the execution panics. - if input_expr_types.is_empty() - && !self.signature().type_signature.supports_zero_argument() - { - return plan_err!( - "{}", - utils::generate_signature_error_msg( - &format!("{self}"), - self.signature(), - input_expr_types - ) - ); - } - - // verify that this is a valid set of data types for this function - data_types(input_expr_types, &self.signature()).map_err(|_| { - plan_datafusion_err!( - "{}", - utils::generate_signature_error_msg( - &format!("{self}"), - self.signature(), - input_expr_types, - ) - ) - })?; - // the return type of the built in function. // Some built-in functions' return type depends on the incoming type. match self { diff --git a/datafusion/expr/src/expr_schema.rs b/datafusion/expr/src/expr_schema.rs index 025b74eb5009..2889fac8c1ee 100644 --- a/datafusion/expr/src/expr_schema.rs +++ b/datafusion/expr/src/expr_schema.rs @@ -23,7 +23,8 @@ use crate::expr::{ }; use crate::field_util::GetFieldAccessSchema; use crate::type_coercion::binary::get_result_type; -use crate::{LogicalPlan, Projection, Subquery}; +use crate::type_coercion::functions::data_types; +use crate::{utils, LogicalPlan, Projection, Subquery}; use arrow::compute::can_cast_types; use arrow::datatypes::{DataType, Field}; use datafusion_common::{ @@ -89,12 +90,24 @@ impl ExprSchemable for Expr { Ok((fun.return_type)(&data_types)?.as_ref().clone()) } Expr::ScalarFunction(ScalarFunction { fun, args }) => { - let data_types = args + let arg_data_types = args .iter() .map(|e| e.get_type(schema)) .collect::>>()?; - fun.return_type(&data_types) + // verify that input data types is consistent with function's `TypeSignature` + data_types(&arg_data_types, &fun.signature()).map_err(|_| { + plan_datafusion_err!( + "{}", + utils::generate_signature_error_msg( + &format!("{fun}"), + fun.signature(), + &arg_data_types, + ) + ) + })?; + + fun.return_type(&arg_data_types) } Expr::WindowFunction(WindowFunction { fun, args, .. }) => { let data_types = args diff --git a/datafusion/expr/src/type_coercion/functions.rs b/datafusion/expr/src/type_coercion/functions.rs index b49bf37d6754..79b574238495 100644 --- a/datafusion/expr/src/type_coercion/functions.rs +++ b/datafusion/expr/src/type_coercion/functions.rs @@ -35,8 +35,17 @@ pub fn data_types( signature: &Signature, ) -> Result> { if current_types.is_empty() { - return Ok(vec![]); + if signature.type_signature.supports_zero_argument() { + return Ok(vec![]); + } else { + return plan_err!( + "Coercion from {:?} to the signature {:?} failed.", + current_types, + &signature.type_signature + ); + } } + let valid_types = get_valid_types(&signature.type_signature, current_types)?; if valid_types diff --git a/datafusion/physical-expr/src/functions.rs b/datafusion/physical-expr/src/functions.rs index b66bac41014d..f14bad093ac7 100644 --- a/datafusion/physical-expr/src/functions.rs +++ b/datafusion/physical-expr/src/functions.rs @@ -47,7 +47,8 @@ use arrow::{ use datafusion_common::{internal_err, DataFusionError, Result, ScalarValue}; pub use datafusion_expr::FuncMonotonicity; use datafusion_expr::{ - BuiltinScalarFunction, ColumnarValue, ScalarFunctionImplementation, + type_coercion::functions::data_types, BuiltinScalarFunction, ColumnarValue, + ScalarFunctionImplementation, }; use std::ops::Neg; use std::sync::Arc; @@ -65,6 +66,9 @@ pub fn create_physical_expr( .map(|e| e.data_type(input_schema)) .collect::>>()?; + // verify that input data types is consistent with function's `TypeSignature` + data_types(&input_expr_types, &fun.signature())?; + let data_type = fun.return_type(&input_expr_types)?; let fun_expr: ScalarFunctionImplementation = match fun { @@ -2952,13 +2956,8 @@ mod tests { "Builtin scalar function {fun} does not support empty arguments" ); } - Err(DataFusionError::Plan(err)) => { - if !err - .contains("No function matches the given name and argument types") - { - return plan_err!( - "Builtin scalar function {fun} didn't got the right error message with empty arguments"); - } + Err(DataFusionError::Plan(_)) => { + // Continue the loop } Err(..) => { return internal_err!( From 4512805c2087d1a5538afdaba9d2e2ca5347c90c Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Thu, 9 Nov 2023 11:01:43 +0100 Subject: [PATCH 211/572] Update sqllogictest requirement from 0.17.0 to 0.18.0 (#8102) Updates the requirements on [sqllogictest](https://github.com/risinglightdb/sqllogictest-rs) to permit the latest version. - [Release notes](https://github.com/risinglightdb/sqllogictest-rs/releases) - [Changelog](https://github.com/risinglightdb/sqllogictest-rs/blob/main/CHANGELOG.md) - [Commits](https://github.com/risinglightdb/sqllogictest-rs/compare/v0.17.0...v0.18.0) --- updated-dependencies: - dependency-name: sqllogictest dependency-type: direct:production ... Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- datafusion/sqllogictest/Cargo.toml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/datafusion/sqllogictest/Cargo.toml b/datafusion/sqllogictest/Cargo.toml index d27e88274f8f..4caec0e84b7f 100644 --- a/datafusion/sqllogictest/Cargo.toml +++ b/datafusion/sqllogictest/Cargo.toml @@ -46,7 +46,7 @@ object_store = { workspace = true } postgres-protocol = { version = "0.6.4", optional = true } postgres-types = { version = "0.2.4", optional = true } rust_decimal = { version = "1.27.0" } -sqllogictest = "0.17.0" +sqllogictest = "0.18.0" sqlparser = { workspace = true } tempfile = { workspace = true } thiserror = { workspace = true } From 1c17c47b4e08754c8c19b4cf578a34d2c9249e30 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Berkay=20=C5=9Eahin?= <124376117+berkaysynnada@users.noreply.github.com> Date: Thu, 9 Nov 2023 17:18:05 +0300 Subject: [PATCH 212/572] Projection Pushdown rule and test changes (#8073) --- datafusion/core/src/physical_optimizer/mod.rs | 1 + .../core/src/physical_optimizer/optimizer.rs | 8 + .../physical_optimizer/output_requirements.rs | 6 +- .../physical_optimizer/projection_pushdown.rs | 2181 +++++++++++++++++ .../physical-expr/src/expressions/cast.rs | 5 + .../physical-expr/src/scalar_function.rs | 5 + .../src/joins/symmetric_hash_join.rs | 5 + datafusion/physical-plan/src/joins/utils.rs | 2 +- datafusion/physical-plan/src/memory.rs | 12 + datafusion/physical-plan/src/projection.rs | 2 +- .../sqllogictest/test_files/explain.slt | 6 +- .../sqllogictest/test_files/groupby.slt | 27 +- datafusion/sqllogictest/test_files/insert.slt | 5 +- .../test_files/insert_to_external.slt | 5 +- datafusion/sqllogictest/test_files/joins.slt | 191 +- .../sqllogictest/test_files/subquery.slt | 77 +- datafusion/sqllogictest/test_files/union.slt | 60 +- 17 files changed, 2395 insertions(+), 203 deletions(-) create mode 100644 datafusion/core/src/physical_optimizer/projection_pushdown.rs diff --git a/datafusion/core/src/physical_optimizer/mod.rs b/datafusion/core/src/physical_optimizer/mod.rs index 9e22bff340c9..d2a0c6fefd8f 100644 --- a/datafusion/core/src/physical_optimizer/mod.rs +++ b/datafusion/core/src/physical_optimizer/mod.rs @@ -30,6 +30,7 @@ pub mod join_selection; pub mod optimizer; pub mod output_requirements; pub mod pipeline_checker; +mod projection_pushdown; pub mod pruning; pub mod replace_with_order_preserving_variants; mod sort_pushdown; diff --git a/datafusion/core/src/physical_optimizer/optimizer.rs b/datafusion/core/src/physical_optimizer/optimizer.rs index 95035e5f81a0..20a59b58ea50 100644 --- a/datafusion/core/src/physical_optimizer/optimizer.rs +++ b/datafusion/core/src/physical_optimizer/optimizer.rs @@ -19,6 +19,7 @@ use std::sync::Arc; +use super::projection_pushdown::ProjectionPushdown; use crate::config::ConfigOptions; use crate::physical_optimizer::aggregate_statistics::AggregateStatistics; use crate::physical_optimizer::coalesce_batches::CoalesceBatches; @@ -107,6 +108,13 @@ impl PhysicalOptimizer { // into an `order by max(x) limit y`. In this case it will copy the limit value down // to the aggregation, allowing it to use only y number of accumulators. Arc::new(TopKAggregation::new()), + // The ProjectionPushdown rule tries to push projections towards + // the sources in the execution plan. As a result of this process, + // a projection can disappear if it reaches the source providers, and + // sequential projections can merge into one. Even if these two cases + // are not present, the load of executors such as join or union will be + // reduced by narrowing their input tables. + Arc::new(ProjectionPushdown::new()), ]; Self::with_rules(rules) diff --git a/datafusion/core/src/physical_optimizer/output_requirements.rs b/datafusion/core/src/physical_optimizer/output_requirements.rs index d9cdc292dd56..f8bf3bb965e8 100644 --- a/datafusion/core/src/physical_optimizer/output_requirements.rs +++ b/datafusion/core/src/physical_optimizer/output_requirements.rs @@ -88,14 +88,14 @@ enum RuleMode { /// /// See [`OutputRequirements`] for more details #[derive(Debug)] -struct OutputRequirementExec { +pub(crate) struct OutputRequirementExec { input: Arc, order_requirement: Option, dist_requirement: Distribution, } impl OutputRequirementExec { - fn new( + pub(crate) fn new( input: Arc, requirements: Option, dist_requirement: Distribution, @@ -107,7 +107,7 @@ impl OutputRequirementExec { } } - fn input(&self) -> Arc { + pub(crate) fn input(&self) -> Arc { self.input.clone() } } diff --git a/datafusion/core/src/physical_optimizer/projection_pushdown.rs b/datafusion/core/src/physical_optimizer/projection_pushdown.rs new file mode 100644 index 000000000000..18495955612f --- /dev/null +++ b/datafusion/core/src/physical_optimizer/projection_pushdown.rs @@ -0,0 +1,2181 @@ +// 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. + +//! This file implements the `ProjectionPushdown` physical optimization rule. +//! The function [`remove_unnecessary_projections`] tries to push down all +//! projections one by one if the operator below is amenable to this. If a +//! projection reaches a source, it can even dissappear from the plan entirely. + +use super::output_requirements::OutputRequirementExec; +use super::PhysicalOptimizerRule; +use crate::datasource::physical_plan::CsvExec; +use crate::error::Result; +use crate::physical_plan::coalesce_partitions::CoalescePartitionsExec; +use crate::physical_plan::filter::FilterExec; +use crate::physical_plan::joins::utils::{ColumnIndex, JoinFilter}; +use crate::physical_plan::joins::{ + CrossJoinExec, HashJoinExec, NestedLoopJoinExec, SortMergeJoinExec, + SymmetricHashJoinExec, +}; +use crate::physical_plan::memory::MemoryExec; +use crate::physical_plan::projection::ProjectionExec; +use crate::physical_plan::repartition::RepartitionExec; +use crate::physical_plan::sorts::sort::SortExec; +use crate::physical_plan::sorts::sort_preserving_merge::SortPreservingMergeExec; +use crate::physical_plan::{Distribution, ExecutionPlan}; + +use arrow_schema::SchemaRef; + +use datafusion_common::config::ConfigOptions; +use datafusion_common::tree_node::{Transformed, TreeNode}; +use datafusion_common::JoinSide; +use datafusion_physical_expr::expressions::{ + BinaryExpr, CaseExpr, CastExpr, Column, Literal, NegativeExpr, +}; +use datafusion_physical_expr::{ + Partitioning, PhysicalExpr, PhysicalSortExpr, PhysicalSortRequirement, + ScalarFunctionExpr, +}; +use datafusion_physical_plan::union::UnionExec; + +use itertools::Itertools; +use std::sync::Arc; + +/// This rule inspects [`ProjectionExec`]'s in the given physical plan and tries to +/// remove or swap with its child. +#[derive(Default)] +pub struct ProjectionPushdown {} + +impl ProjectionPushdown { + #[allow(missing_docs)] + pub fn new() -> Self { + Self {} + } +} + +impl PhysicalOptimizerRule for ProjectionPushdown { + fn optimize( + &self, + plan: Arc, + _config: &ConfigOptions, + ) -> Result> { + plan.transform_down(&remove_unnecessary_projections) + } + + fn name(&self) -> &str { + "ProjectionPushdown" + } + + fn schema_check(&self) -> bool { + true + } +} + +/// This function checks if `plan` is a [`ProjectionExec`], and inspects its +/// input(s) to test whether it can push `plan` under its input(s). This function +/// will operate on the entire tree and may ultimately remove `plan` entirely +/// by leveraging source providers with built-in projection capabilities. +pub fn remove_unnecessary_projections( + plan: Arc, +) -> Result>> { + let maybe_modified = if let Some(projection) = + plan.as_any().downcast_ref::() + { + // If the projection does not cause any change on the input, we can + // safely remove it: + if is_projection_removable(projection) { + return Ok(Transformed::Yes(projection.input().clone())); + } + // If it does, check if we can push it under its child(ren): + let input = projection.input().as_any(); + if let Some(csv) = input.downcast_ref::() { + try_swapping_with_csv(projection, csv) + } else if let Some(memory) = input.downcast_ref::() { + try_swapping_with_memory(projection, memory)? + } else if let Some(child_projection) = input.downcast_ref::() { + let maybe_unified = try_unifying_projections(projection, child_projection)?; + return if let Some(new_plan) = maybe_unified { + // To unify 3 or more sequential projections: + remove_unnecessary_projections(new_plan) + } else { + Ok(Transformed::No(plan)) + }; + } else if let Some(output_req) = input.downcast_ref::() { + try_swapping_with_output_req(projection, output_req)? + } else if input.is::() { + try_swapping_with_coalesce_partitions(projection)? + } else if let Some(filter) = input.downcast_ref::() { + try_swapping_with_filter(projection, filter)? + } else if let Some(repartition) = input.downcast_ref::() { + try_swapping_with_repartition(projection, repartition)? + } else if let Some(sort) = input.downcast_ref::() { + try_swapping_with_sort(projection, sort)? + } else if let Some(spm) = input.downcast_ref::() { + try_swapping_with_sort_preserving_merge(projection, spm)? + } else if let Some(union) = input.downcast_ref::() { + try_pushdown_through_union(projection, union)? + } else if let Some(hash_join) = input.downcast_ref::() { + try_pushdown_through_hash_join(projection, hash_join)? + } else if let Some(cross_join) = input.downcast_ref::() { + try_swapping_with_cross_join(projection, cross_join)? + } else if let Some(nl_join) = input.downcast_ref::() { + try_swapping_with_nested_loop_join(projection, nl_join)? + } else if let Some(sm_join) = input.downcast_ref::() { + try_swapping_with_sort_merge_join(projection, sm_join)? + } else if let Some(sym_join) = input.downcast_ref::() { + try_swapping_with_sym_hash_join(projection, sym_join)? + } else { + // If the input plan of the projection is not one of the above, we + // conservatively assume that pushing the projection down may hurt. + // When adding new operators, consider adding them here if you + // think pushing projections under them is beneficial. + None + } + } else { + return Ok(Transformed::No(plan)); + }; + + Ok(maybe_modified.map_or(Transformed::No(plan), Transformed::Yes)) +} + +/// Tries to swap `projection` with its input (`csv`). If possible, performs +/// the swap and returns [`CsvExec`] as the top plan. Otherwise, returns `None`. +fn try_swapping_with_csv( + projection: &ProjectionExec, + csv: &CsvExec, +) -> Option> { + // If there is any non-column or alias-carrier expression, Projection should not be removed. + // This process can be moved into CsvExec, but it would be an overlap of their responsibility. + all_alias_free_columns(projection.expr()).then(|| { + let mut file_scan = csv.base_config().clone(); + let new_projections = + new_projections_for_columns(projection, &file_scan.projection); + file_scan.projection = Some(new_projections); + + Arc::new(CsvExec::new( + file_scan, + csv.has_header(), + csv.delimiter(), + csv.quote(), + csv.escape(), + csv.file_compression_type, + )) as _ + }) +} + +/// Tries to swap `projection` with its input (`memory`). If possible, performs +/// the swap and returns [`MemoryExec`] as the top plan. Otherwise, returns `None`. +fn try_swapping_with_memory( + projection: &ProjectionExec, + memory: &MemoryExec, +) -> Result>> { + // If there is any non-column or alias-carrier expression, Projection should not be removed. + // This process can be moved into MemoryExec, but it would be an overlap of their responsibility. + all_alias_free_columns(projection.expr()) + .then(|| { + let new_projections = + new_projections_for_columns(projection, memory.projection()); + + MemoryExec::try_new( + memory.partitions(), + memory.original_schema(), + Some(new_projections), + ) + .map(|e| Arc::new(e) as _) + }) + .transpose() +} + +/// Unifies `projection` with its input (which is also a [`ProjectionExec`]). +/// Two consecutive projections can always merge into a single projection unless +/// the [`update_expr`] function does not support one of the expression +/// types involved in the projection. +fn try_unifying_projections( + projection: &ProjectionExec, + child: &ProjectionExec, +) -> Result>> { + let mut projected_exprs = vec![]; + for (expr, alias) in projection.expr() { + // If there is no match in the input projection, we cannot unify these + // projections. This case will arise if the projection expression contains + // a `PhysicalExpr` variant `update_expr` doesn't support. + let Some(expr) = update_expr(expr, child.expr(), true)? else { + return Ok(None); + }; + projected_exprs.push((expr, alias.clone())); + } + + ProjectionExec::try_new(projected_exprs, child.input().clone()) + .map(|e| Some(Arc::new(e) as _)) +} + +/// Tries to swap `projection` with its input (`output_req`). If possible, +/// performs the swap and returns [`OutputRequirementExec`] as the top plan. +/// Otherwise, returns `None`. +fn try_swapping_with_output_req( + projection: &ProjectionExec, + output_req: &OutputRequirementExec, +) -> Result>> { + // If the projection does not narrow the the schema, we should not try to push it down: + if projection.expr().len() >= projection.input().schema().fields().len() { + return Ok(None); + } + + let mut updated_sort_reqs = vec![]; + // None or empty_vec can be treated in the same way. + if let Some(reqs) = &output_req.required_input_ordering()[0] { + for req in reqs { + let Some(new_expr) = update_expr(&req.expr, projection.expr(), false)? else { + return Ok(None); + }; + updated_sort_reqs.push(PhysicalSortRequirement { + expr: new_expr, + options: req.options, + }); + } + } + + let dist_req = match &output_req.required_input_distribution()[0] { + Distribution::HashPartitioned(exprs) => { + let mut updated_exprs = vec![]; + for expr in exprs { + let Some(new_expr) = update_expr(expr, projection.expr(), false)? else { + return Ok(None); + }; + updated_exprs.push(new_expr); + } + Distribution::HashPartitioned(updated_exprs) + } + dist => dist.clone(), + }; + + make_with_child(projection, &output_req.input()) + .map(|input| { + OutputRequirementExec::new( + input, + (!updated_sort_reqs.is_empty()).then_some(updated_sort_reqs), + dist_req, + ) + }) + .map(|e| Some(Arc::new(e) as _)) +} + +/// Tries to swap `projection` with its input, which is known to be a +/// [`CoalescePartitionsExec`]. If possible, performs the swap and returns +/// [`CoalescePartitionsExec`] as the top plan. Otherwise, returns `None`. +fn try_swapping_with_coalesce_partitions( + projection: &ProjectionExec, +) -> Result>> { + // If the projection does not narrow the the schema, we should not try to push it down: + if projection.expr().len() >= projection.input().schema().fields().len() { + return Ok(None); + } + // CoalescePartitionsExec always has a single child, so zero indexing is safe. + make_with_child(projection, &projection.input().children()[0]) + .map(|e| Some(Arc::new(CoalescePartitionsExec::new(e)) as _)) +} + +/// Tries to swap `projection` with its input (`filter`). If possible, performs +/// the swap and returns [`FilterExec`] as the top plan. Otherwise, returns `None`. +fn try_swapping_with_filter( + projection: &ProjectionExec, + filter: &FilterExec, +) -> Result>> { + // If the projection does not narrow the the schema, we should not try to push it down: + if projection.expr().len() >= projection.input().schema().fields().len() { + return Ok(None); + } + // Each column in the predicate expression must exist after the projection. + let Some(new_predicate) = update_expr(filter.predicate(), projection.expr(), false)? + else { + return Ok(None); + }; + + FilterExec::try_new(new_predicate, make_with_child(projection, filter.input())?) + .map(|e| Some(Arc::new(e) as _)) +} + +/// Tries to swap the projection with its input [`RepartitionExec`]. If it can be done, +/// it returns the new swapped version having the [`RepartitionExec`] as the top plan. +/// Otherwise, it returns None. +fn try_swapping_with_repartition( + projection: &ProjectionExec, + repartition: &RepartitionExec, +) -> Result>> { + // If the projection does not narrow the the schema, we should not try to push it down. + if projection.expr().len() >= projection.input().schema().fields().len() { + return Ok(None); + } + + // If pushdown is not beneficial or applicable, break it. + if projection.benefits_from_input_partitioning()[0] || !all_columns(projection.expr()) + { + return Ok(None); + } + + let new_projection = make_with_child(projection, repartition.input())?; + + let new_partitioning = match repartition.partitioning() { + Partitioning::Hash(partitions, size) => { + let mut new_partitions = vec![]; + for partition in partitions { + let Some(new_partition) = + update_expr(partition, projection.expr(), false)? + else { + return Ok(None); + }; + new_partitions.push(new_partition); + } + Partitioning::Hash(new_partitions, *size) + } + others => others.clone(), + }; + + Ok(Some(Arc::new(RepartitionExec::try_new( + new_projection, + new_partitioning, + )?))) +} + +/// Tries to swap the projection with its input [`SortExec`]. If it can be done, +/// it returns the new swapped version having the [`SortExec`] as the top plan. +/// Otherwise, it returns None. +fn try_swapping_with_sort( + projection: &ProjectionExec, + sort: &SortExec, +) -> Result>> { + // If the projection does not narrow the the schema, we should not try to push it down. + if projection.expr().len() >= projection.input().schema().fields().len() { + return Ok(None); + } + + let mut updated_exprs = vec![]; + for sort in sort.expr() { + let Some(new_expr) = update_expr(&sort.expr, projection.expr(), false)? else { + return Ok(None); + }; + updated_exprs.push(PhysicalSortExpr { + expr: new_expr, + options: sort.options, + }); + } + + Ok(Some(Arc::new( + SortExec::new(updated_exprs, make_with_child(projection, sort.input())?) + .with_fetch(sort.fetch()), + ))) +} + +/// Tries to swap the projection with its input [`SortPreservingMergeExec`]. +/// If this is possible, it returns the new [`SortPreservingMergeExec`] whose +/// child is a projection. Otherwise, it returns None. +fn try_swapping_with_sort_preserving_merge( + projection: &ProjectionExec, + spm: &SortPreservingMergeExec, +) -> Result>> { + // If the projection does not narrow the the schema, we should not try to push it down. + if projection.expr().len() >= projection.input().schema().fields().len() { + return Ok(None); + } + + let mut updated_exprs = vec![]; + for sort in spm.expr() { + let Some(updated_expr) = update_expr(&sort.expr, projection.expr(), false)? + else { + return Ok(None); + }; + updated_exprs.push(PhysicalSortExpr { + expr: updated_expr, + options: sort.options, + }); + } + + Ok(Some(Arc::new( + SortPreservingMergeExec::new( + updated_exprs, + make_with_child(projection, spm.input())?, + ) + .with_fetch(spm.fetch()), + ))) +} + +/// Tries to push `projection` down through `union`. If possible, performs the +/// pushdown and returns a new [`UnionExec`] as the top plan which has projections +/// as its children. Otherwise, returns `None`. +fn try_pushdown_through_union( + projection: &ProjectionExec, + union: &UnionExec, +) -> Result>> { + // If the projection doesn't narrow the schema, we shouldn't try to push it down. + if projection.expr().len() >= projection.input().schema().fields().len() { + return Ok(None); + } + + let new_children = union + .children() + .into_iter() + .map(|child| make_with_child(projection, &child)) + .collect::>>()?; + + Ok(Some(Arc::new(UnionExec::new(new_children)))) +} + +/// Tries to push `projection` down through `hash_join`. If possible, performs the +/// pushdown and returns a new [`HashJoinExec`] as the top plan which has projections +/// as its children. Otherwise, returns `None`. +fn try_pushdown_through_hash_join( + projection: &ProjectionExec, + hash_join: &HashJoinExec, +) -> Result>> { + // Convert projected expressions to columns. We can not proceed if this is + // not possible. + let Some(projection_as_columns) = physical_to_column_exprs(projection.expr()) else { + return Ok(None); + }; + + let (far_right_left_col_ind, far_left_right_col_ind) = join_table_borders( + hash_join.left().schema().fields().len(), + &projection_as_columns, + ); + + if !join_allows_pushdown( + &projection_as_columns, + hash_join.schema(), + far_right_left_col_ind, + far_left_right_col_ind, + ) { + return Ok(None); + } + + let Some(new_on) = update_join_on( + &projection_as_columns[0..=far_right_left_col_ind as _], + &projection_as_columns[far_left_right_col_ind as _..], + hash_join.on(), + ) else { + return Ok(None); + }; + + let new_filter = if let Some(filter) = hash_join.filter() { + match update_join_filter( + &projection_as_columns[0..=far_right_left_col_ind as _], + &projection_as_columns[far_left_right_col_ind as _..], + filter, + hash_join.left(), + hash_join.right(), + ) { + Some(updated_filter) => Some(updated_filter), + None => return Ok(None), + } + } else { + None + }; + + let (new_left, new_right) = new_join_children( + projection_as_columns, + far_right_left_col_ind, + far_left_right_col_ind, + hash_join.left(), + hash_join.right(), + )?; + + Ok(Some(Arc::new(HashJoinExec::try_new( + Arc::new(new_left), + Arc::new(new_right), + new_on, + new_filter, + hash_join.join_type(), + *hash_join.partition_mode(), + hash_join.null_equals_null, + )?))) +} + +/// Tries to swap the projection with its input [`CrossJoinExec`]. If it can be done, +/// it returns the new swapped version having the [`CrossJoinExec`] as the top plan. +/// Otherwise, it returns None. +fn try_swapping_with_cross_join( + projection: &ProjectionExec, + cross_join: &CrossJoinExec, +) -> Result>> { + // Convert projected PhysicalExpr's to columns. If not possible, we cannot proceed. + let Some(projection_as_columns) = physical_to_column_exprs(projection.expr()) else { + return Ok(None); + }; + + let (far_right_left_col_ind, far_left_right_col_ind) = join_table_borders( + cross_join.left().schema().fields().len(), + &projection_as_columns, + ); + + if !join_allows_pushdown( + &projection_as_columns, + cross_join.schema(), + far_right_left_col_ind, + far_left_right_col_ind, + ) { + return Ok(None); + } + + let (new_left, new_right) = new_join_children( + projection_as_columns, + far_right_left_col_ind, + far_left_right_col_ind, + cross_join.left(), + cross_join.right(), + )?; + + Ok(Some(Arc::new(CrossJoinExec::new( + Arc::new(new_left), + Arc::new(new_right), + )))) +} + +/// Tries to swap the projection with its input [`NestedLoopJoinExec`]. If it can be done, +/// it returns the new swapped version having the [`NestedLoopJoinExec`] as the top plan. +/// Otherwise, it returns None. +fn try_swapping_with_nested_loop_join( + projection: &ProjectionExec, + nl_join: &NestedLoopJoinExec, +) -> Result>> { + // Convert projected PhysicalExpr's to columns. If not possible, we cannot proceed. + let Some(projection_as_columns) = physical_to_column_exprs(projection.expr()) else { + return Ok(None); + }; + + let (far_right_left_col_ind, far_left_right_col_ind) = join_table_borders( + nl_join.left().schema().fields().len(), + &projection_as_columns, + ); + + if !join_allows_pushdown( + &projection_as_columns, + nl_join.schema(), + far_right_left_col_ind, + far_left_right_col_ind, + ) { + return Ok(None); + } + + let new_filter = if let Some(filter) = nl_join.filter() { + match update_join_filter( + &projection_as_columns[0..=far_right_left_col_ind as _], + &projection_as_columns[far_left_right_col_ind as _..], + filter, + nl_join.left(), + nl_join.right(), + ) { + Some(updated_filter) => Some(updated_filter), + None => return Ok(None), + } + } else { + None + }; + + let (new_left, new_right) = new_join_children( + projection_as_columns, + far_right_left_col_ind, + far_left_right_col_ind, + nl_join.left(), + nl_join.right(), + )?; + + Ok(Some(Arc::new(NestedLoopJoinExec::try_new( + Arc::new(new_left), + Arc::new(new_right), + new_filter, + nl_join.join_type(), + )?))) +} + +/// Tries to swap the projection with its input [`SortMergeJoinExec`]. If it can be done, +/// it returns the new swapped version having the [`SortMergeJoinExec`] as the top plan. +/// Otherwise, it returns None. +fn try_swapping_with_sort_merge_join( + projection: &ProjectionExec, + sm_join: &SortMergeJoinExec, +) -> Result>> { + // Convert projected PhysicalExpr's to columns. If not possible, we cannot proceed. + let Some(projection_as_columns) = physical_to_column_exprs(projection.expr()) else { + return Ok(None); + }; + + let (far_right_left_col_ind, far_left_right_col_ind) = join_table_borders( + sm_join.left().schema().fields().len(), + &projection_as_columns, + ); + + if !join_allows_pushdown( + &projection_as_columns, + sm_join.schema(), + far_right_left_col_ind, + far_left_right_col_ind, + ) { + return Ok(None); + } + + let Some(new_on) = update_join_on( + &projection_as_columns[0..=far_right_left_col_ind as _], + &projection_as_columns[far_left_right_col_ind as _..], + sm_join.on(), + ) else { + return Ok(None); + }; + + let (new_left, new_right) = new_join_children( + projection_as_columns, + far_right_left_col_ind, + far_left_right_col_ind, + &sm_join.children()[0], + &sm_join.children()[1], + )?; + + Ok(Some(Arc::new(SortMergeJoinExec::try_new( + Arc::new(new_left), + Arc::new(new_right), + new_on, + sm_join.join_type, + sm_join.sort_options.clone(), + sm_join.null_equals_null, + )?))) +} + +/// Tries to swap the projection with its input [`SymmetricHashJoinExec`]. If it can be done, +/// it returns the new swapped version having the [`SymmetricHashJoinExec`] as the top plan. +/// Otherwise, it returns None. +fn try_swapping_with_sym_hash_join( + projection: &ProjectionExec, + sym_join: &SymmetricHashJoinExec, +) -> Result>> { + // Convert projected PhysicalExpr's to columns. If not possible, we cannot proceed. + let Some(projection_as_columns) = physical_to_column_exprs(projection.expr()) else { + return Ok(None); + }; + + let (far_right_left_col_ind, far_left_right_col_ind) = join_table_borders( + sym_join.left().schema().fields().len(), + &projection_as_columns, + ); + + if !join_allows_pushdown( + &projection_as_columns, + sym_join.schema(), + far_right_left_col_ind, + far_left_right_col_ind, + ) { + return Ok(None); + } + + let Some(new_on) = update_join_on( + &projection_as_columns[0..=far_right_left_col_ind as _], + &projection_as_columns[far_left_right_col_ind as _..], + sym_join.on(), + ) else { + return Ok(None); + }; + + let new_filter = if let Some(filter) = sym_join.filter() { + match update_join_filter( + &projection_as_columns[0..=far_right_left_col_ind as _], + &projection_as_columns[far_left_right_col_ind as _..], + filter, + sym_join.left(), + sym_join.right(), + ) { + Some(updated_filter) => Some(updated_filter), + None => return Ok(None), + } + } else { + None + }; + + let (new_left, new_right) = new_join_children( + projection_as_columns, + far_right_left_col_ind, + far_left_right_col_ind, + sym_join.left(), + sym_join.right(), + )?; + + Ok(Some(Arc::new(SymmetricHashJoinExec::try_new( + Arc::new(new_left), + Arc::new(new_right), + new_on, + new_filter, + sym_join.join_type(), + sym_join.null_equals_null(), + sym_join.partition_mode(), + )?))) +} + +/// Compare the inputs and outputs of the projection. If the projection causes +/// any change in the fields, it returns `false`. +fn is_projection_removable(projection: &ProjectionExec) -> bool { + all_alias_free_columns(projection.expr()) && { + let schema = projection.schema(); + let input_schema = projection.input().schema(); + let fields = schema.fields(); + let input_fields = input_schema.fields(); + fields.len() == input_fields.len() + && fields + .iter() + .zip(input_fields.iter()) + .all(|(out, input)| out.eq(input)) + } +} + +/// Given the expression set of a projection, checks if the projection causes +/// any renaming or constructs a non-`Column` physical expression. +fn all_alias_free_columns(exprs: &[(Arc, String)]) -> bool { + exprs.iter().all(|(expr, alias)| { + expr.as_any() + .downcast_ref::() + .map(|column| column.name() == alias) + .unwrap_or(false) + }) +} + +/// Updates a source provider's projected columns according to the given +/// projection operator's expressions. To use this function safely, one must +/// ensure that all expressions are `Column` expressions without aliases. +fn new_projections_for_columns( + projection: &ProjectionExec, + source: &Option>, +) -> Vec { + projection + .expr() + .iter() + .filter_map(|(expr, _)| { + expr.as_any() + .downcast_ref::() + .and_then(|expr| source.as_ref().map(|proj| proj[expr.index()])) + }) + .collect() +} + +/// The function operates in two modes: +/// +/// 1) When `sync_with_child` is `true`: +/// +/// The function updates the indices of `expr` if the expression resides +/// in the input plan. For instance, given the expressions `a@1 + b@2` +/// and `c@0` with the input schema `c@2, a@0, b@1`, the expressions are +/// updated to `a@0 + b@1` and `c@2`. +/// +/// 2) When `sync_with_child` is `false`: +/// +/// The function determines how the expression would be updated if a projection +/// was placed before the plan associated with the expression. If the expression +/// cannot be rewritten after the projection, it returns `None`. For example, +/// given the expressions `c@0`, `a@1` and `b@2`, and the [`ProjectionExec`] with +/// an output schema of `a, c_new`, then `c@0` becomes `c_new@1`, `a@1` becomes +/// `a@0`, but `b@2` results in `None` since the projection does not include `b`. +/// +/// If the expression contains a `PhysicalExpr` variant that this function does +/// not support, it will return `None`. An error can only be introduced if +/// `CaseExpr::try_new` returns an error. +fn update_expr( + expr: &Arc, + projected_exprs: &[(Arc, String)], + sync_with_child: bool, +) -> Result>> { + let expr_any = expr.as_any(); + if let Some(column) = expr_any.downcast_ref::() { + if sync_with_child { + // Update the index of `column`: + Ok(Some(projected_exprs[column.index()].0.clone())) + } else { + // Determine how to update `column` to accommodate `projected_exprs`: + Ok(projected_exprs.iter().enumerate().find_map( + |(index, (projected_expr, alias))| { + projected_expr.as_any().downcast_ref::().and_then( + |projected_column| { + column + .name() + .eq(projected_column.name()) + .then(|| Arc::new(Column::new(alias, index)) as _) + }, + ) + }, + )) + } + } else if let Some(binary) = expr_any.downcast_ref::() { + match ( + update_expr(binary.left(), projected_exprs, sync_with_child)?, + update_expr(binary.right(), projected_exprs, sync_with_child)?, + ) { + (Some(left), Some(right)) => { + Ok(Some(Arc::new(BinaryExpr::new(left, *binary.op(), right)))) + } + _ => Ok(None), + } + } else if let Some(cast) = expr_any.downcast_ref::() { + update_expr(cast.expr(), projected_exprs, sync_with_child).map(|maybe_expr| { + maybe_expr.map(|expr| { + Arc::new(CastExpr::new( + expr, + cast.cast_type().clone(), + Some(cast.cast_options().clone()), + )) as _ + }) + }) + } else if expr_any.is::() { + Ok(Some(expr.clone())) + } else if let Some(negative) = expr_any.downcast_ref::() { + update_expr(negative.arg(), projected_exprs, sync_with_child).map(|maybe_expr| { + maybe_expr.map(|expr| Arc::new(NegativeExpr::new(expr)) as _) + }) + } else if let Some(scalar_func) = expr_any.downcast_ref::() { + scalar_func + .args() + .iter() + .map(|expr| update_expr(expr, projected_exprs, sync_with_child)) + .collect::>>>() + .map(|maybe_args| { + maybe_args.map(|new_args| { + Arc::new(ScalarFunctionExpr::new( + scalar_func.name(), + scalar_func.fun().clone(), + new_args, + scalar_func.return_type(), + scalar_func.monotonicity().clone(), + )) as _ + }) + }) + } else if let Some(case) = expr_any.downcast_ref::() { + update_case_expr(case, projected_exprs, sync_with_child) + } else { + Ok(None) + } +} + +/// Updates the indices `case` refers to according to `projected_exprs`. +fn update_case_expr( + case: &CaseExpr, + projected_exprs: &[(Arc, String)], + sync_with_child: bool, +) -> Result>> { + let new_case = case + .expr() + .map(|expr| update_expr(expr, projected_exprs, sync_with_child)) + .transpose()? + .flatten(); + + let new_else = case + .else_expr() + .map(|expr| update_expr(expr, projected_exprs, sync_with_child)) + .transpose()? + .flatten(); + + let new_when_then = case + .when_then_expr() + .iter() + .map(|(when, then)| { + Ok(( + update_expr(when, projected_exprs, sync_with_child)?, + update_expr(then, projected_exprs, sync_with_child)?, + )) + }) + .collect::>>()? + .into_iter() + .filter_map(|(maybe_when, maybe_then)| match (maybe_when, maybe_then) { + (Some(when), Some(then)) => Some((when, then)), + _ => None, + }) + .collect::>(); + + if new_when_then.len() != case.when_then_expr().len() + || case.expr().is_some() && new_case.is_none() + || case.else_expr().is_some() && new_else.is_none() + { + return Ok(None); + } + + CaseExpr::try_new(new_case, new_when_then, new_else).map(|e| Some(Arc::new(e) as _)) +} + +/// Creates a new [`ProjectionExec`] instance with the given child plan and +/// projected expressions. +fn make_with_child( + projection: &ProjectionExec, + child: &Arc, +) -> Result> { + ProjectionExec::try_new(projection.expr().to_vec(), child.clone()) + .map(|e| Arc::new(e) as _) +} + +/// Returns `true` if all the expressions in the argument are `Column`s. +fn all_columns(exprs: &[(Arc, String)]) -> bool { + exprs.iter().all(|(expr, _)| expr.as_any().is::()) +} + +/// Downcasts all the expressions in `exprs` to `Column`s. If any of the given +/// expressions is not a `Column`, returns `None`. +fn physical_to_column_exprs( + exprs: &[(Arc, String)], +) -> Option> { + exprs + .iter() + .map(|(expr, alias)| { + expr.as_any() + .downcast_ref::() + .map(|col| (col.clone(), alias.clone())) + }) + .collect() +} + +/// Returns the last index before encountering a column coming from the right table when traveling +/// through the projection from left to right, and the last index before encountering a column +/// coming from the left table when traveling through the projection from right to left. +/// If there is no column in the projection coming from the left side, it returns (-1, ...), +/// if there is no column in the projection coming from the right side, it returns (..., projection length). +fn join_table_borders( + left_table_column_count: usize, + projection_as_columns: &[(Column, String)], +) -> (i32, i32) { + let far_right_left_col_ind = projection_as_columns + .iter() + .enumerate() + .take_while(|(_, (projection_column, _))| { + projection_column.index() < left_table_column_count + }) + .last() + .map(|(index, _)| index as i32) + .unwrap_or(-1); + + let far_left_right_col_ind = projection_as_columns + .iter() + .enumerate() + .rev() + .take_while(|(_, (projection_column, _))| { + projection_column.index() >= left_table_column_count + }) + .last() + .map(|(index, _)| index as i32) + .unwrap_or(projection_as_columns.len() as i32); + + (far_right_left_col_ind, far_left_right_col_ind) +} + +/// Tries to update the equi-join `Column`'s of a join as if the the input of +/// the join was replaced by a projection. +fn update_join_on( + proj_left_exprs: &[(Column, String)], + proj_right_exprs: &[(Column, String)], + hash_join_on: &[(Column, Column)], +) -> Option> { + let (left_idx, right_idx): (Vec<_>, Vec<_>) = hash_join_on + .iter() + .map(|(left, right)| (left, right)) + .unzip(); + + let new_left_columns = new_columns_for_join_on(&left_idx, proj_left_exprs); + let new_right_columns = new_columns_for_join_on(&right_idx, proj_right_exprs); + + match (new_left_columns, new_right_columns) { + (Some(left), Some(right)) => Some(left.into_iter().zip(right).collect()), + _ => None, + } +} + +/// This function generates a new set of columns to be used in a hash join +/// operation based on a set of equi-join conditions (`hash_join_on`) and a +/// list of projection expressions (`projection_exprs`). +fn new_columns_for_join_on( + hash_join_on: &[&Column], + projection_exprs: &[(Column, String)], +) -> Option> { + let new_columns = hash_join_on + .iter() + .filter_map(|on| { + projection_exprs + .iter() + .enumerate() + .find(|(_, (proj_column, _))| on.name() == proj_column.name()) + .map(|(index, (_, alias))| Column::new(alias, index)) + }) + .collect::>(); + (new_columns.len() == hash_join_on.len()).then_some(new_columns) +} + +/// Tries to update the column indices of a [`JoinFilter`] as if the the input of +/// the join was replaced by a projection. +fn update_join_filter( + projection_left_exprs: &[(Column, String)], + projection_right_exprs: &[(Column, String)], + join_filter: &JoinFilter, + join_left: &Arc, + join_right: &Arc, +) -> Option { + let mut new_left_indices = new_indices_for_join_filter( + join_filter, + JoinSide::Left, + projection_left_exprs, + join_left.schema(), + ) + .into_iter(); + let mut new_right_indices = new_indices_for_join_filter( + join_filter, + JoinSide::Right, + projection_right_exprs, + join_right.schema(), + ) + .into_iter(); + + // Check if all columns match: + (new_right_indices.len() + new_left_indices.len() + == join_filter.column_indices().len()) + .then(|| { + JoinFilter::new( + join_filter.expression().clone(), + join_filter + .column_indices() + .iter() + .map(|col_idx| ColumnIndex { + index: if col_idx.side == JoinSide::Left { + new_left_indices.next().unwrap() + } else { + new_right_indices.next().unwrap() + }, + side: col_idx.side, + }) + .collect(), + join_filter.schema().clone(), + ) + }) +} + +/// This function determines and returns a vector of indices representing the +/// positions of columns in `projection_exprs` that are involved in `join_filter`, +/// and correspond to a particular side (`join_side`) of the join operation. +fn new_indices_for_join_filter( + join_filter: &JoinFilter, + join_side: JoinSide, + projection_exprs: &[(Column, String)], + join_child_schema: SchemaRef, +) -> Vec { + join_filter + .column_indices() + .iter() + .filter(|col_idx| col_idx.side == join_side) + .filter_map(|col_idx| { + projection_exprs.iter().position(|(col, _)| { + col.name() == join_child_schema.fields()[col_idx.index].name() + }) + }) + .collect() +} + +/// Checks three conditions for pushing a projection down through a join: +/// - Projection must narrow the join output schema. +/// - Columns coming from left/right tables must be collected at the left/right +/// sides of the output table. +/// - Left or right table is not lost after the projection. +fn join_allows_pushdown( + projection_as_columns: &[(Column, String)], + join_schema: SchemaRef, + far_right_left_col_ind: i32, + far_left_right_col_ind: i32, +) -> bool { + // Projection must narrow the join output: + projection_as_columns.len() < join_schema.fields().len() + // Are the columns from different tables mixed? + && (far_right_left_col_ind + 1 == far_left_right_col_ind) + // Left or right table is not lost after the projection. + && far_right_left_col_ind >= 0 + && far_left_right_col_ind < projection_as_columns.len() as i32 +} + +/// If pushing down the projection over this join's children seems possible, +/// this function constructs the new [`ProjectionExec`]s that will come on top +/// of the original children of the join. +fn new_join_children( + projection_as_columns: Vec<(Column, String)>, + far_right_left_col_ind: i32, + far_left_right_col_ind: i32, + left_child: &Arc, + right_child: &Arc, +) -> Result<(ProjectionExec, ProjectionExec)> { + let new_left = ProjectionExec::try_new( + projection_as_columns[0..=far_right_left_col_ind as _] + .iter() + .map(|(col, alias)| { + ( + Arc::new(Column::new(col.name(), col.index())) as _, + alias.clone(), + ) + }) + .collect_vec(), + left_child.clone(), + )?; + let left_size = left_child.schema().fields().len() as i32; + let new_right = ProjectionExec::try_new( + projection_as_columns[far_left_right_col_ind as _..] + .iter() + .map(|(col, alias)| { + ( + Arc::new(Column::new( + col.name(), + // Align projected expressions coming from the right + // table with the new right child projection: + (col.index() as i32 - left_size) as _, + )) as _, + alias.clone(), + ) + }) + .collect_vec(), + right_child.clone(), + )?; + + Ok((new_left, new_right)) +} + +#[cfg(test)] +mod tests { + use std::sync::Arc; + + use crate::datasource::file_format::file_compression_type::FileCompressionType; + use crate::datasource::listing::PartitionedFile; + use crate::datasource::physical_plan::{CsvExec, FileScanConfig}; + use crate::physical_optimizer::output_requirements::OutputRequirementExec; + use crate::physical_optimizer::projection_pushdown::{ + join_table_borders, update_expr, ProjectionPushdown, + }; + use crate::physical_optimizer::utils::get_plan_string; + use crate::physical_optimizer::PhysicalOptimizerRule; + use crate::physical_plan::coalesce_partitions::CoalescePartitionsExec; + use crate::physical_plan::filter::FilterExec; + use crate::physical_plan::joins::utils::{ColumnIndex, JoinFilter}; + use crate::physical_plan::joins::StreamJoinPartitionMode; + use crate::physical_plan::memory::MemoryExec; + use crate::physical_plan::projection::ProjectionExec; + use crate::physical_plan::repartition::RepartitionExec; + use crate::physical_plan::sorts::sort::SortExec; + use crate::physical_plan::sorts::sort_preserving_merge::SortPreservingMergeExec; + use crate::physical_plan::ExecutionPlan; + + use arrow_schema::{DataType, Field, Schema, SortOptions}; + use datafusion_common::config::ConfigOptions; + use datafusion_common::{JoinSide, JoinType, Result, ScalarValue, Statistics}; + use datafusion_execution::object_store::ObjectStoreUrl; + use datafusion_expr::{ColumnarValue, Operator}; + use datafusion_physical_expr::expressions::{ + BinaryExpr, CaseExpr, CastExpr, Column, Literal, NegativeExpr, + }; + use datafusion_physical_expr::{ + Distribution, Partitioning, PhysicalExpr, PhysicalSortExpr, + PhysicalSortRequirement, ScalarFunctionExpr, + }; + use datafusion_physical_plan::joins::SymmetricHashJoinExec; + use datafusion_physical_plan::union::UnionExec; + + #[test] + fn test_update_matching_exprs() -> Result<()> { + let exprs: Vec> = vec![ + Arc::new(BinaryExpr::new( + Arc::new(Column::new("a", 3)), + Operator::Divide, + Arc::new(Column::new("e", 5)), + )), + Arc::new(CastExpr::new( + Arc::new(Column::new("a", 3)), + DataType::Float32, + None, + )), + Arc::new(NegativeExpr::new(Arc::new(Column::new("f", 4)))), + Arc::new(ScalarFunctionExpr::new( + "scalar_expr", + Arc::new(|_: &[ColumnarValue]| unimplemented!("not implemented")), + vec![ + Arc::new(BinaryExpr::new( + Arc::new(Column::new("b", 1)), + Operator::Divide, + Arc::new(Column::new("c", 0)), + )), + Arc::new(BinaryExpr::new( + Arc::new(Column::new("c", 0)), + Operator::Divide, + Arc::new(Column::new("b", 1)), + )), + ], + &DataType::Int32, + None, + )), + Arc::new(CaseExpr::try_new( + Some(Arc::new(Column::new("d", 2))), + vec![ + ( + Arc::new(Column::new("a", 3)) as Arc, + Arc::new(BinaryExpr::new( + Arc::new(Column::new("d", 2)), + Operator::Plus, + Arc::new(Column::new("e", 5)), + )) as Arc, + ), + ( + Arc::new(Column::new("a", 3)) as Arc, + Arc::new(BinaryExpr::new( + Arc::new(Column::new("e", 5)), + Operator::Plus, + Arc::new(Column::new("d", 2)), + )) as Arc, + ), + ], + Some(Arc::new(BinaryExpr::new( + Arc::new(Column::new("a", 3)), + Operator::Modulo, + Arc::new(Column::new("e", 5)), + ))), + )?), + ]; + let child: Vec<(Arc, String)> = vec![ + (Arc::new(Column::new("c", 2)), "c".to_owned()), + (Arc::new(Column::new("b", 1)), "b".to_owned()), + (Arc::new(Column::new("d", 3)), "d".to_owned()), + (Arc::new(Column::new("a", 0)), "a".to_owned()), + (Arc::new(Column::new("f", 5)), "f".to_owned()), + (Arc::new(Column::new("e", 4)), "e".to_owned()), + ]; + + let expected_exprs: Vec> = vec![ + Arc::new(BinaryExpr::new( + Arc::new(Column::new("a", 0)), + Operator::Divide, + Arc::new(Column::new("e", 4)), + )), + Arc::new(CastExpr::new( + Arc::new(Column::new("a", 0)), + DataType::Float32, + None, + )), + Arc::new(NegativeExpr::new(Arc::new(Column::new("f", 5)))), + Arc::new(ScalarFunctionExpr::new( + "scalar_expr", + Arc::new(|_: &[ColumnarValue]| unimplemented!("not implemented")), + vec![ + Arc::new(BinaryExpr::new( + Arc::new(Column::new("b", 1)), + Operator::Divide, + Arc::new(Column::new("c", 2)), + )), + Arc::new(BinaryExpr::new( + Arc::new(Column::new("c", 2)), + Operator::Divide, + Arc::new(Column::new("b", 1)), + )), + ], + &DataType::Int32, + None, + )), + Arc::new(CaseExpr::try_new( + Some(Arc::new(Column::new("d", 3))), + vec![ + ( + Arc::new(Column::new("a", 0)) as Arc, + Arc::new(BinaryExpr::new( + Arc::new(Column::new("d", 3)), + Operator::Plus, + Arc::new(Column::new("e", 4)), + )) as Arc, + ), + ( + Arc::new(Column::new("a", 0)) as Arc, + Arc::new(BinaryExpr::new( + Arc::new(Column::new("e", 4)), + Operator::Plus, + Arc::new(Column::new("d", 3)), + )) as Arc, + ), + ], + Some(Arc::new(BinaryExpr::new( + Arc::new(Column::new("a", 0)), + Operator::Modulo, + Arc::new(Column::new("e", 4)), + ))), + )?), + ]; + + for (expr, expected_expr) in exprs.into_iter().zip(expected_exprs.into_iter()) { + assert!(update_expr(&expr, &child, true)? + .unwrap() + .eq(&expected_expr)); + } + + Ok(()) + } + + #[test] + fn test_update_projected_exprs() -> Result<()> { + let exprs: Vec> = vec![ + Arc::new(BinaryExpr::new( + Arc::new(Column::new("a", 3)), + Operator::Divide, + Arc::new(Column::new("e", 5)), + )), + Arc::new(CastExpr::new( + Arc::new(Column::new("a", 3)), + DataType::Float32, + None, + )), + Arc::new(NegativeExpr::new(Arc::new(Column::new("f", 4)))), + Arc::new(ScalarFunctionExpr::new( + "scalar_expr", + Arc::new(|_: &[ColumnarValue]| unimplemented!("not implemented")), + vec![ + Arc::new(BinaryExpr::new( + Arc::new(Column::new("b", 1)), + Operator::Divide, + Arc::new(Column::new("c", 0)), + )), + Arc::new(BinaryExpr::new( + Arc::new(Column::new("c", 0)), + Operator::Divide, + Arc::new(Column::new("b", 1)), + )), + ], + &DataType::Int32, + None, + )), + Arc::new(CaseExpr::try_new( + Some(Arc::new(Column::new("d", 2))), + vec![ + ( + Arc::new(Column::new("a", 3)) as Arc, + Arc::new(BinaryExpr::new( + Arc::new(Column::new("d", 2)), + Operator::Plus, + Arc::new(Column::new("e", 5)), + )) as Arc, + ), + ( + Arc::new(Column::new("a", 3)) as Arc, + Arc::new(BinaryExpr::new( + Arc::new(Column::new("e", 5)), + Operator::Plus, + Arc::new(Column::new("d", 2)), + )) as Arc, + ), + ], + Some(Arc::new(BinaryExpr::new( + Arc::new(Column::new("a", 3)), + Operator::Modulo, + Arc::new(Column::new("e", 5)), + ))), + )?), + ]; + let projected_exprs: Vec<(Arc, String)> = vec![ + (Arc::new(Column::new("a", 0)), "a".to_owned()), + (Arc::new(Column::new("b", 1)), "b_new".to_owned()), + (Arc::new(Column::new("c", 2)), "c".to_owned()), + (Arc::new(Column::new("d", 3)), "d_new".to_owned()), + (Arc::new(Column::new("e", 4)), "e".to_owned()), + (Arc::new(Column::new("f", 5)), "f_new".to_owned()), + ]; + + let expected_exprs: Vec> = vec![ + Arc::new(BinaryExpr::new( + Arc::new(Column::new("a", 0)), + Operator::Divide, + Arc::new(Column::new("e", 4)), + )), + Arc::new(CastExpr::new( + Arc::new(Column::new("a", 0)), + DataType::Float32, + None, + )), + Arc::new(NegativeExpr::new(Arc::new(Column::new("f_new", 5)))), + Arc::new(ScalarFunctionExpr::new( + "scalar_expr", + Arc::new(|_: &[ColumnarValue]| unimplemented!("not implemented")), + vec![ + Arc::new(BinaryExpr::new( + Arc::new(Column::new("b_new", 1)), + Operator::Divide, + Arc::new(Column::new("c", 2)), + )), + Arc::new(BinaryExpr::new( + Arc::new(Column::new("c", 2)), + Operator::Divide, + Arc::new(Column::new("b_new", 1)), + )), + ], + &DataType::Int32, + None, + )), + Arc::new(CaseExpr::try_new( + Some(Arc::new(Column::new("d_new", 3))), + vec![ + ( + Arc::new(Column::new("a", 0)) as Arc, + Arc::new(BinaryExpr::new( + Arc::new(Column::new("d_new", 3)), + Operator::Plus, + Arc::new(Column::new("e", 4)), + )) as Arc, + ), + ( + Arc::new(Column::new("a", 0)) as Arc, + Arc::new(BinaryExpr::new( + Arc::new(Column::new("e", 4)), + Operator::Plus, + Arc::new(Column::new("d_new", 3)), + )) as Arc, + ), + ], + Some(Arc::new(BinaryExpr::new( + Arc::new(Column::new("a", 0)), + Operator::Modulo, + Arc::new(Column::new("e", 4)), + ))), + )?), + ]; + + for (expr, expected_expr) in exprs.into_iter().zip(expected_exprs.into_iter()) { + assert!(update_expr(&expr, &projected_exprs, false)? + .unwrap() + .eq(&expected_expr)); + } + + Ok(()) + } + + #[test] + fn test_join_table_borders() -> Result<()> { + let projections = vec![ + (Column::new("b", 1), "b".to_owned()), + (Column::new("c", 2), "c".to_owned()), + (Column::new("e", 4), "e".to_owned()), + (Column::new("d", 3), "d".to_owned()), + (Column::new("c", 2), "c".to_owned()), + (Column::new("f", 5), "f".to_owned()), + (Column::new("h", 7), "h".to_owned()), + (Column::new("g", 6), "g".to_owned()), + ]; + let left_table_column_count = 5; + assert_eq!( + join_table_borders(left_table_column_count, &projections), + (4, 5) + ); + + let left_table_column_count = 8; + assert_eq!( + join_table_borders(left_table_column_count, &projections), + (7, 8) + ); + + let left_table_column_count = 1; + assert_eq!( + join_table_borders(left_table_column_count, &projections), + (-1, 0) + ); + + let projections = vec![ + (Column::new("a", 0), "a".to_owned()), + (Column::new("b", 1), "b".to_owned()), + (Column::new("d", 3), "d".to_owned()), + (Column::new("g", 6), "g".to_owned()), + (Column::new("e", 4), "e".to_owned()), + (Column::new("f", 5), "f".to_owned()), + (Column::new("e", 4), "e".to_owned()), + (Column::new("h", 7), "h".to_owned()), + ]; + let left_table_column_count = 5; + assert_eq!( + join_table_borders(left_table_column_count, &projections), + (2, 7) + ); + + let left_table_column_count = 7; + assert_eq!( + join_table_borders(left_table_column_count, &projections), + (6, 7) + ); + + Ok(()) + } + + fn create_simple_csv_exec() -> Arc { + let schema = Arc::new(Schema::new(vec![ + Field::new("a", DataType::Int32, true), + Field::new("b", DataType::Int32, true), + Field::new("c", DataType::Int32, true), + Field::new("d", DataType::Int32, true), + Field::new("e", DataType::Int32, true), + ])); + Arc::new(CsvExec::new( + FileScanConfig { + object_store_url: ObjectStoreUrl::parse("test:///").unwrap(), + file_schema: schema.clone(), + file_groups: vec![vec![PartitionedFile::new("x".to_string(), 100)]], + statistics: Statistics::new_unknown(&schema), + projection: Some(vec![0, 1, 2, 3, 4]), + limit: None, + table_partition_cols: vec![], + output_ordering: vec![vec![]], + infinite_source: false, + }, + false, + 0, + 0, + None, + FileCompressionType::UNCOMPRESSED, + )) + } + + fn create_projecting_csv_exec() -> Arc { + let schema = Arc::new(Schema::new(vec![ + Field::new("a", DataType::Int32, true), + Field::new("b", DataType::Int32, true), + Field::new("c", DataType::Int32, true), + Field::new("d", DataType::Int32, true), + ])); + Arc::new(CsvExec::new( + FileScanConfig { + object_store_url: ObjectStoreUrl::parse("test:///").unwrap(), + file_schema: schema.clone(), + file_groups: vec![vec![PartitionedFile::new("x".to_string(), 100)]], + statistics: Statistics::new_unknown(&schema), + projection: Some(vec![3, 2, 1]), + limit: None, + table_partition_cols: vec![], + output_ordering: vec![vec![]], + infinite_source: false, + }, + false, + 0, + 0, + None, + FileCompressionType::UNCOMPRESSED, + )) + } + + fn create_projecting_memory_exec() -> Arc { + let schema = Arc::new(Schema::new(vec![ + Field::new("a", DataType::Int32, true), + Field::new("b", DataType::Int32, true), + Field::new("c", DataType::Int32, true), + Field::new("d", DataType::Int32, true), + Field::new("e", DataType::Int32, true), + ])); + + Arc::new(MemoryExec::try_new(&[], schema, Some(vec![2, 0, 3, 4])).unwrap()) + } + + #[test] + fn test_csv_after_projection() -> Result<()> { + let csv = create_projecting_csv_exec(); + let projection: Arc = Arc::new(ProjectionExec::try_new( + vec![ + (Arc::new(Column::new("b", 2)), "b".to_string()), + (Arc::new(Column::new("d", 0)), "d".to_string()), + ], + csv.clone(), + )?); + let initial = get_plan_string(&projection); + let expected_initial = [ + "ProjectionExec: expr=[b@2 as b, d@0 as d]", + " CsvExec: file_groups={1 group: [[x]]}, projection=[d, c, b], has_header=false", + ]; + assert_eq!(initial, expected_initial); + + let after_optimize = + ProjectionPushdown::new().optimize(projection, &ConfigOptions::new())?; + + let expected = [ + "CsvExec: file_groups={1 group: [[x]]}, projection=[b, d], has_header=false", + ]; + assert_eq!(get_plan_string(&after_optimize), expected); + + Ok(()) + } + + #[test] + fn test_memory_after_projection() -> Result<()> { + let memory = create_projecting_memory_exec(); + let projection: Arc = Arc::new(ProjectionExec::try_new( + vec![ + (Arc::new(Column::new("d", 2)), "d".to_string()), + (Arc::new(Column::new("e", 3)), "e".to_string()), + (Arc::new(Column::new("a", 1)), "a".to_string()), + ], + memory.clone(), + )?); + let initial = get_plan_string(&projection); + let expected_initial = [ + "ProjectionExec: expr=[d@2 as d, e@3 as e, a@1 as a]", + " MemoryExec: partitions=0, partition_sizes=[]", + ]; + assert_eq!(initial, expected_initial); + + let after_optimize = + ProjectionPushdown::new().optimize(projection, &ConfigOptions::new())?; + + let expected = ["MemoryExec: partitions=0, partition_sizes=[]"]; + assert_eq!(get_plan_string(&after_optimize), expected); + assert_eq!( + after_optimize + .clone() + .as_any() + .downcast_ref::() + .unwrap() + .projection() + .clone() + .unwrap(), + vec![3, 4, 0] + ); + + Ok(()) + } + + #[test] + fn test_projection_after_projection() -> Result<()> { + let csv = create_simple_csv_exec(); + let child_projection: Arc = Arc::new(ProjectionExec::try_new( + vec![ + (Arc::new(Column::new("c", 2)), "c".to_string()), + (Arc::new(Column::new("e", 4)), "new_e".to_string()), + (Arc::new(Column::new("a", 0)), "a".to_string()), + (Arc::new(Column::new("b", 1)), "new_b".to_string()), + ], + csv.clone(), + )?); + let top_projection: Arc = Arc::new(ProjectionExec::try_new( + vec![ + (Arc::new(Column::new("new_b", 3)), "new_b".to_string()), + ( + Arc::new(BinaryExpr::new( + Arc::new(Column::new("c", 0)), + Operator::Plus, + Arc::new(Column::new("new_e", 1)), + )), + "binary".to_string(), + ), + (Arc::new(Column::new("new_b", 3)), "newest_b".to_string()), + ], + child_projection.clone(), + )?); + + let initial = get_plan_string(&top_projection); + let expected_initial = [ + "ProjectionExec: expr=[new_b@3 as new_b, c@0 + new_e@1 as binary, new_b@3 as newest_b]", + " ProjectionExec: expr=[c@2 as c, e@4 as new_e, a@0 as a, b@1 as new_b]", + " CsvExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], has_header=false" + ]; + assert_eq!(initial, expected_initial); + + let after_optimize = + ProjectionPushdown::new().optimize(top_projection, &ConfigOptions::new())?; + + let expected = [ + "ProjectionExec: expr=[b@1 as new_b, c@2 + e@4 as binary, b@1 as newest_b]", + " CsvExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], has_header=false" + ]; + assert_eq!(get_plan_string(&after_optimize), expected); + + Ok(()) + } + + #[test] + fn test_output_req_after_projection() -> Result<()> { + let csv = create_simple_csv_exec(); + let sort_req: Arc = Arc::new(OutputRequirementExec::new( + csv.clone(), + Some(vec![ + PhysicalSortRequirement { + expr: Arc::new(Column::new("b", 1)), + options: Some(SortOptions::default()), + }, + PhysicalSortRequirement { + expr: Arc::new(BinaryExpr::new( + Arc::new(Column::new("c", 2)), + Operator::Plus, + Arc::new(Column::new("a", 0)), + )), + options: Some(SortOptions::default()), + }, + ]), + Distribution::HashPartitioned(vec![ + Arc::new(Column::new("a", 0)), + Arc::new(Column::new("b", 1)), + ]), + )); + let projection: Arc = Arc::new(ProjectionExec::try_new( + vec![ + (Arc::new(Column::new("c", 2)), "c".to_string()), + (Arc::new(Column::new("a", 0)), "new_a".to_string()), + (Arc::new(Column::new("b", 1)), "b".to_string()), + ], + sort_req.clone(), + )?); + + let initial = get_plan_string(&projection); + let expected_initial = [ + "ProjectionExec: expr=[c@2 as c, a@0 as new_a, b@1 as b]", + " OutputRequirementExec", + " CsvExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], has_header=false" + ]; + assert_eq!(initial, expected_initial); + + let after_optimize = + ProjectionPushdown::new().optimize(projection, &ConfigOptions::new())?; + + let expected: [&str; 3] = [ + "OutputRequirementExec", + " ProjectionExec: expr=[c@2 as c, a@0 as new_a, b@1 as b]", + " CsvExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], has_header=false" + ]; + + assert_eq!(get_plan_string(&after_optimize), expected); + let expected_reqs = vec![ + PhysicalSortRequirement { + expr: Arc::new(Column::new("b", 2)), + options: Some(SortOptions::default()), + }, + PhysicalSortRequirement { + expr: Arc::new(BinaryExpr::new( + Arc::new(Column::new("c", 0)), + Operator::Plus, + Arc::new(Column::new("new_a", 1)), + )), + options: Some(SortOptions::default()), + }, + ]; + assert_eq!( + after_optimize + .as_any() + .downcast_ref::() + .unwrap() + .required_input_ordering()[0] + .clone() + .unwrap(), + expected_reqs + ); + let expected_distribution: Vec> = vec![ + Arc::new(Column::new("new_a", 1)), + Arc::new(Column::new("b", 2)), + ]; + if let Distribution::HashPartitioned(vec) = after_optimize + .as_any() + .downcast_ref::() + .unwrap() + .required_input_distribution()[0] + .clone() + { + assert!(vec + .iter() + .zip(expected_distribution) + .all(|(actual, expected)| actual.eq(&expected))); + } else { + panic!("Expected HashPartitioned distribution!"); + }; + + Ok(()) + } + + #[test] + fn test_coalesce_partitions_after_projection() -> Result<()> { + let csv = create_simple_csv_exec(); + let coalesce_partitions: Arc = + Arc::new(CoalescePartitionsExec::new(csv)); + let projection: Arc = Arc::new(ProjectionExec::try_new( + vec![ + (Arc::new(Column::new("b", 1)), "b".to_string()), + (Arc::new(Column::new("a", 0)), "a_new".to_string()), + (Arc::new(Column::new("d", 3)), "d".to_string()), + ], + coalesce_partitions, + )?); + let initial = get_plan_string(&projection); + let expected_initial = [ + "ProjectionExec: expr=[b@1 as b, a@0 as a_new, d@3 as d]", + " CoalescePartitionsExec", + " CsvExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], has_header=false", + ]; + assert_eq!(initial, expected_initial); + + let after_optimize = + ProjectionPushdown::new().optimize(projection, &ConfigOptions::new())?; + + let expected = [ + "CoalescePartitionsExec", + " ProjectionExec: expr=[b@1 as b, a@0 as a_new, d@3 as d]", + " CsvExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], has_header=false", + ]; + assert_eq!(get_plan_string(&after_optimize), expected); + + Ok(()) + } + + #[test] + fn test_filter_after_projection() -> Result<()> { + let csv = create_simple_csv_exec(); + let predicate = Arc::new(BinaryExpr::new( + Arc::new(BinaryExpr::new( + Arc::new(Column::new("b", 1)), + Operator::Minus, + Arc::new(Column::new("a", 0)), + )), + Operator::Gt, + Arc::new(BinaryExpr::new( + Arc::new(Column::new("d", 3)), + Operator::Minus, + Arc::new(Column::new("a", 0)), + )), + )); + let filter: Arc = + Arc::new(FilterExec::try_new(predicate, csv)?); + let projection: Arc = Arc::new(ProjectionExec::try_new( + vec![ + (Arc::new(Column::new("a", 0)), "a_new".to_string()), + (Arc::new(Column::new("b", 1)), "b".to_string()), + (Arc::new(Column::new("d", 3)), "d".to_string()), + ], + filter.clone(), + )?); + + let initial = get_plan_string(&projection); + let expected_initial = [ + "ProjectionExec: expr=[a@0 as a_new, b@1 as b, d@3 as d]", + " FilterExec: b@1 - a@0 > d@3 - a@0", + " CsvExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], has_header=false", + ]; + assert_eq!(initial, expected_initial); + + let after_optimize = + ProjectionPushdown::new().optimize(projection, &ConfigOptions::new())?; + + let expected = [ + "FilterExec: b@1 - a_new@0 > d@2 - a_new@0", + " ProjectionExec: expr=[a@0 as a_new, b@1 as b, d@3 as d]", + " CsvExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], has_header=false", + ]; + assert_eq!(get_plan_string(&after_optimize), expected); + + Ok(()) + } + + #[test] + fn test_join_after_projection() -> Result<()> { + let left_csv = create_simple_csv_exec(); + let right_csv = create_simple_csv_exec(); + + let join: Arc = Arc::new(SymmetricHashJoinExec::try_new( + left_csv, + right_csv, + vec![(Column::new("b", 1), Column::new("c", 2))], + // b_left-(1+a_right)<=a_right+c_left + Some(JoinFilter::new( + Arc::new(BinaryExpr::new( + Arc::new(BinaryExpr::new( + Arc::new(Column::new("b_left_inter", 0)), + Operator::Minus, + Arc::new(BinaryExpr::new( + Arc::new(Literal::new(ScalarValue::Int32(Some(1)))), + Operator::Plus, + Arc::new(Column::new("a_right_inter", 1)), + )), + )), + Operator::LtEq, + Arc::new(BinaryExpr::new( + Arc::new(Column::new("a_right_inter", 1)), + Operator::Plus, + Arc::new(Column::new("c_left_inter", 2)), + )), + )), + vec![ + ColumnIndex { + index: 1, + side: JoinSide::Left, + }, + ColumnIndex { + index: 0, + side: JoinSide::Right, + }, + ColumnIndex { + index: 2, + side: JoinSide::Left, + }, + ], + Schema::new(vec![ + Field::new("b_left_inter", DataType::Int32, true), + Field::new("a_right_inter", DataType::Int32, true), + Field::new("c_left_inter", DataType::Int32, true), + ]), + )), + &JoinType::Inner, + true, + StreamJoinPartitionMode::SinglePartition, + )?); + let projection: Arc = Arc::new(ProjectionExec::try_new( + vec![ + (Arc::new(Column::new("c", 2)), "c_from_left".to_string()), + (Arc::new(Column::new("b", 1)), "b_from_left".to_string()), + (Arc::new(Column::new("a", 0)), "a_from_left".to_string()), + (Arc::new(Column::new("a", 5)), "a_from_right".to_string()), + (Arc::new(Column::new("c", 7)), "c_from_right".to_string()), + ], + join, + )?); + let initial = get_plan_string(&projection); + let expected_initial = [ + "ProjectionExec: expr=[c@2 as c_from_left, b@1 as b_from_left, a@0 as a_from_left, a@5 as a_from_right, c@7 as c_from_right]", + " SymmetricHashJoinExec: mode=SinglePartition, join_type=Inner, on=[(b@1, c@2)], filter=b_left_inter@0 - 1 + a_right_inter@1 <= a_right_inter@1 + c_left_inter@2", + " CsvExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], has_header=false", + " CsvExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], has_header=false" + ]; + assert_eq!(initial, expected_initial); + + let after_optimize = + ProjectionPushdown::new().optimize(projection, &ConfigOptions::new())?; + + let expected = [ + "SymmetricHashJoinExec: mode=SinglePartition, join_type=Inner, on=[(b_from_left@1, c_from_right@1)], filter=b_left_inter@0 - 1 + a_right_inter@1 <= a_right_inter@1 + c_left_inter@2", + " ProjectionExec: expr=[c@2 as c_from_left, b@1 as b_from_left, a@0 as a_from_left]", + " CsvExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], has_header=false", + " ProjectionExec: expr=[a@0 as a_from_right, c@2 as c_from_right]", + " CsvExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], has_header=false" + ]; + assert_eq!(get_plan_string(&after_optimize), expected); + + let expected_filter_col_ind = vec![ + ColumnIndex { + index: 1, + side: JoinSide::Left, + }, + ColumnIndex { + index: 0, + side: JoinSide::Right, + }, + ColumnIndex { + index: 0, + side: JoinSide::Left, + }, + ]; + + assert_eq!( + expected_filter_col_ind, + after_optimize + .as_any() + .downcast_ref::() + .unwrap() + .filter() + .unwrap() + .column_indices() + ); + + Ok(()) + } + + #[test] + fn test_repartition_after_projection() -> Result<()> { + let csv = create_simple_csv_exec(); + let repartition: Arc = Arc::new(RepartitionExec::try_new( + csv, + Partitioning::Hash( + vec![ + Arc::new(Column::new("a", 0)), + Arc::new(Column::new("b", 1)), + Arc::new(Column::new("d", 3)), + ], + 6, + ), + )?); + let projection: Arc = Arc::new(ProjectionExec::try_new( + vec![ + (Arc::new(Column::new("b", 1)), "b_new".to_string()), + (Arc::new(Column::new("a", 0)), "a".to_string()), + (Arc::new(Column::new("d", 3)), "d_new".to_string()), + ], + repartition, + )?); + let initial = get_plan_string(&projection); + let expected_initial = [ + "ProjectionExec: expr=[b@1 as b_new, a@0 as a, d@3 as d_new]", + " RepartitionExec: partitioning=Hash([a@0, b@1, d@3], 6), input_partitions=1", + " CsvExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], has_header=false", + ]; + assert_eq!(initial, expected_initial); + + let after_optimize = + ProjectionPushdown::new().optimize(projection, &ConfigOptions::new())?; + + let expected = [ + "RepartitionExec: partitioning=Hash([a@1, b_new@0, d_new@2], 6), input_partitions=1", + " ProjectionExec: expr=[b@1 as b_new, a@0 as a, d@3 as d_new]", + " CsvExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], has_header=false", + ]; + assert_eq!(get_plan_string(&after_optimize), expected); + + assert_eq!( + after_optimize + .as_any() + .downcast_ref::() + .unwrap() + .partitioning() + .clone(), + Partitioning::Hash( + vec![ + Arc::new(Column::new("a", 1)), + Arc::new(Column::new("b_new", 0)), + Arc::new(Column::new("d_new", 2)), + ], + 6, + ), + ); + + Ok(()) + } + + #[test] + fn test_sort_after_projection() -> Result<()> { + let csv = create_simple_csv_exec(); + let sort_req: Arc = Arc::new(SortExec::new( + vec![ + PhysicalSortExpr { + expr: Arc::new(Column::new("b", 1)), + options: SortOptions::default(), + }, + PhysicalSortExpr { + expr: Arc::new(BinaryExpr::new( + Arc::new(Column::new("c", 2)), + Operator::Plus, + Arc::new(Column::new("a", 0)), + )), + options: SortOptions::default(), + }, + ], + csv.clone(), + )); + let projection: Arc = Arc::new(ProjectionExec::try_new( + vec![ + (Arc::new(Column::new("c", 2)), "c".to_string()), + (Arc::new(Column::new("a", 0)), "new_a".to_string()), + (Arc::new(Column::new("b", 1)), "b".to_string()), + ], + sort_req.clone(), + )?); + + let initial = get_plan_string(&projection); + let expected_initial = [ + "ProjectionExec: expr=[c@2 as c, a@0 as new_a, b@1 as b]", + " SortExec: expr=[b@1 ASC,c@2 + a@0 ASC]", + " CsvExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], has_header=false" + ]; + assert_eq!(initial, expected_initial); + + let after_optimize = + ProjectionPushdown::new().optimize(projection, &ConfigOptions::new())?; + + let expected = [ + "SortExec: expr=[b@2 ASC,c@0 + new_a@1 ASC]", + " ProjectionExec: expr=[c@2 as c, a@0 as new_a, b@1 as b]", + " CsvExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], has_header=false" + ]; + assert_eq!(get_plan_string(&after_optimize), expected); + + Ok(()) + } + + #[test] + fn test_sort_preserving_after_projection() -> Result<()> { + let csv = create_simple_csv_exec(); + let sort_req: Arc = Arc::new(SortPreservingMergeExec::new( + vec![ + PhysicalSortExpr { + expr: Arc::new(Column::new("b", 1)), + options: SortOptions::default(), + }, + PhysicalSortExpr { + expr: Arc::new(BinaryExpr::new( + Arc::new(Column::new("c", 2)), + Operator::Plus, + Arc::new(Column::new("a", 0)), + )), + options: SortOptions::default(), + }, + ], + csv.clone(), + )); + let projection: Arc = Arc::new(ProjectionExec::try_new( + vec![ + (Arc::new(Column::new("c", 2)), "c".to_string()), + (Arc::new(Column::new("a", 0)), "new_a".to_string()), + (Arc::new(Column::new("b", 1)), "b".to_string()), + ], + sort_req.clone(), + )?); + + let initial = get_plan_string(&projection); + let expected_initial = [ + "ProjectionExec: expr=[c@2 as c, a@0 as new_a, b@1 as b]", + " SortPreservingMergeExec: [b@1 ASC,c@2 + a@0 ASC]", + " CsvExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], has_header=false" + ]; + assert_eq!(initial, expected_initial); + + let after_optimize = + ProjectionPushdown::new().optimize(projection, &ConfigOptions::new())?; + + let expected = [ + "SortPreservingMergeExec: [b@2 ASC,c@0 + new_a@1 ASC]", + " ProjectionExec: expr=[c@2 as c, a@0 as new_a, b@1 as b]", + " CsvExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], has_header=false" + ]; + assert_eq!(get_plan_string(&after_optimize), expected); + + Ok(()) + } + + #[test] + fn test_union_after_projection() -> Result<()> { + let csv = create_simple_csv_exec(); + let union: Arc = + Arc::new(UnionExec::new(vec![csv.clone(), csv.clone(), csv])); + let projection: Arc = Arc::new(ProjectionExec::try_new( + vec![ + (Arc::new(Column::new("c", 2)), "c".to_string()), + (Arc::new(Column::new("a", 0)), "new_a".to_string()), + (Arc::new(Column::new("b", 1)), "b".to_string()), + ], + union.clone(), + )?); + + let initial = get_plan_string(&projection); + let expected_initial = [ + "ProjectionExec: expr=[c@2 as c, a@0 as new_a, b@1 as b]", + " UnionExec", + " CsvExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], has_header=false", + " CsvExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], has_header=false", + " CsvExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], has_header=false" + ]; + assert_eq!(initial, expected_initial); + + let after_optimize = + ProjectionPushdown::new().optimize(projection, &ConfigOptions::new())?; + + let expected = [ + "UnionExec", + " ProjectionExec: expr=[c@2 as c, a@0 as new_a, b@1 as b]", + " CsvExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], has_header=false", + " ProjectionExec: expr=[c@2 as c, a@0 as new_a, b@1 as b]", + " CsvExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], has_header=false", + " ProjectionExec: expr=[c@2 as c, a@0 as new_a, b@1 as b]", + " CsvExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], has_header=false" + ]; + assert_eq!(get_plan_string(&after_optimize), expected); + + Ok(()) + } +} diff --git a/datafusion/physical-expr/src/expressions/cast.rs b/datafusion/physical-expr/src/expressions/cast.rs index 9390089063a0..5d56af364636 100644 --- a/datafusion/physical-expr/src/expressions/cast.rs +++ b/datafusion/physical-expr/src/expressions/cast.rs @@ -73,6 +73,11 @@ impl CastExpr { pub fn cast_type(&self) -> &DataType { &self.cast_type } + + /// The cast options + pub fn cast_options(&self) -> &CastOptions<'static> { + &self.cast_options + } } impl fmt::Display for CastExpr { diff --git a/datafusion/physical-expr/src/scalar_function.rs b/datafusion/physical-expr/src/scalar_function.rs index 768aa04dd9c1..63101c03bc4a 100644 --- a/datafusion/physical-expr/src/scalar_function.rs +++ b/datafusion/physical-expr/src/scalar_function.rs @@ -108,6 +108,11 @@ impl ScalarFunctionExpr { pub fn return_type(&self) -> &DataType { &self.return_type } + + /// Monotonicity information of the function + pub fn monotonicity(&self) -> &Option { + &self.monotonicity + } } impl fmt::Display for ScalarFunctionExpr { diff --git a/datafusion/physical-plan/src/joins/symmetric_hash_join.rs b/datafusion/physical-plan/src/joins/symmetric_hash_join.rs index 3617893a1c61..39ac25ecb561 100644 --- a/datafusion/physical-plan/src/joins/symmetric_hash_join.rs +++ b/datafusion/physical-plan/src/joins/symmetric_hash_join.rs @@ -325,6 +325,11 @@ impl SymmetricHashJoinExec { self.null_equals_null } + /// Get partition mode + pub fn partition_mode(&self) -> StreamJoinPartitionMode { + self.mode + } + /// Check if order information covers every column in the filter expression. pub fn check_if_order_information_available(&self) -> Result { if let Some(filter) = self.filter() { diff --git a/datafusion/physical-plan/src/joins/utils.rs b/datafusion/physical-plan/src/joins/utils.rs index c91dc92fbc7a..5efeedfe6536 100644 --- a/datafusion/physical-plan/src/joins/utils.rs +++ b/datafusion/physical-plan/src/joins/utils.rs @@ -219,7 +219,7 @@ pub fn calculate_join_output_ordering( } /// Information about the index and placement (left or right) of the columns -#[derive(Debug, Clone)] +#[derive(Debug, Clone, PartialEq)] pub struct ColumnIndex { /// Index of the column pub index: usize, diff --git a/datafusion/physical-plan/src/memory.rs b/datafusion/physical-plan/src/memory.rs index 5f1660a225b9..39cd47452eff 100644 --- a/datafusion/physical-plan/src/memory.rs +++ b/datafusion/physical-plan/src/memory.rs @@ -177,6 +177,14 @@ impl MemoryExec { }) } + pub fn partitions(&self) -> &[Vec] { + &self.partitions + } + + pub fn projection(&self) -> &Option> { + &self.projection + } + /// A memory table can be ordered by multiple expressions simultaneously. /// [`EquivalenceProperties`] keeps track of expressions that describe the /// global ordering of the schema. These columns are not necessarily same; e.g. @@ -197,6 +205,10 @@ impl MemoryExec { self.sort_information = sort_information; self } + + pub fn original_schema(&self) -> SchemaRef { + self.schema.clone() + } } /// Iterator over batches diff --git a/datafusion/physical-plan/src/projection.rs b/datafusion/physical-plan/src/projection.rs index c5d94b08e0e1..bbf0d6d4b31c 100644 --- a/datafusion/physical-plan/src/projection.rs +++ b/datafusion/physical-plan/src/projection.rs @@ -46,7 +46,7 @@ use futures::stream::{Stream, StreamExt}; use log::trace; /// Execution plan for a projection -#[derive(Debug)] +#[derive(Debug, Clone)] pub struct ProjectionExec { /// The projection expressions stored as tuples of (expression, output column name) pub(crate) expr: Vec<(Arc, String)>, diff --git a/datafusion/sqllogictest/test_files/explain.slt b/datafusion/sqllogictest/test_files/explain.slt index d28f9fc6e372..49bb63d75d8b 100644 --- a/datafusion/sqllogictest/test_files/explain.slt +++ b/datafusion/sqllogictest/test_files/explain.slt @@ -169,9 +169,8 @@ Dml: op=[Insert Into] table=[sink_table] ------TableScan: aggregate_test_100 projection=[c1, c2, c3, c4, c5, c6, c7, c8, c9, c10, c11, c12, c13] physical_plan FileSinkExec: sink=CsvSink(writer_mode=Append, file_groups=[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]) ---ProjectionExec: expr=[c1@0 as c1, c2@1 as c2, c3@2 as c3, c4@3 as c4, c5@4 as c5, c6@5 as c6, c7@6 as c7, c8@7 as c8, c9@8 as c9, c10@9 as c10, c11@10 as c11, c12@11 as c12, c13@12 as c13] -----SortExec: expr=[c1@0 ASC NULLS LAST] -------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1, c2, c3, c4, c5, c6, c7, c8, c9, c10, c11, c12, c13], has_header=true +--SortExec: expr=[c1@0 ASC NULLS LAST] +----CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1, c2, c3, c4, c5, c6, c7, c8, c9, c10, c11, c12, c13], has_header=true # test EXPLAIN VERBOSE query TT @@ -258,6 +257,7 @@ physical_plan after coalesce_batches SAME TEXT AS ABOVE physical_plan after OutputRequirements CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/example.csv]]}, projection=[a, b, c], has_header=true physical_plan after PipelineChecker SAME TEXT AS ABOVE physical_plan after LimitAggregation SAME TEXT AS ABOVE +physical_plan after ProjectionPushdown SAME TEXT AS ABOVE physical_plan CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/example.csv]]}, projection=[a, b, c], has_header=true diff --git a/datafusion/sqllogictest/test_files/groupby.slt b/datafusion/sqllogictest/test_files/groupby.slt index 000c3dc3b503..105f11f21628 100644 --- a/datafusion/sqllogictest/test_files/groupby.slt +++ b/datafusion/sqllogictest/test_files/groupby.slt @@ -2084,9 +2084,7 @@ logical_plan Projection: multiple_ordered_table.a --Sort: multiple_ordered_table.c ASC NULLS LAST ----TableScan: multiple_ordered_table projection=[a, c] -physical_plan -ProjectionExec: expr=[a@0 as a] ---CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, c], output_ordering=[a@0 ASC NULLS LAST], has_header=true +physical_plan CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a], output_ordering=[a@0 ASC NULLS LAST], has_header=true # Final plan shouldn't have SortExec a ASC, b ASC, # because table already satisfies this ordering. @@ -2097,9 +2095,7 @@ logical_plan Projection: multiple_ordered_table.a --Sort: multiple_ordered_table.a ASC NULLS LAST, multiple_ordered_table.b ASC NULLS LAST ----TableScan: multiple_ordered_table projection=[a, b] -physical_plan -ProjectionExec: expr=[a@0 as a] ---CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, b], output_ordering=[a@0 ASC NULLS LAST, b@1 ASC NULLS LAST], has_header=true +physical_plan CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a], output_ordering=[a@0 ASC NULLS LAST], has_header=true # test_window_agg_sort statement ok @@ -3696,16 +3692,15 @@ Projection: amount_usd ----------------SubqueryAlias: r ------------------TableScan: multiple_ordered_table projection=[a, d] physical_plan -ProjectionExec: expr=[amount_usd@0 as amount_usd] ---ProjectionExec: expr=[LAST_VALUE(l.d) ORDER BY [l.a ASC NULLS LAST]@1 as amount_usd, row_n@0 as row_n] -----AggregateExec: mode=Single, gby=[row_n@2 as row_n], aggr=[LAST_VALUE(l.d)], ordering_mode=Sorted -------ProjectionExec: expr=[a@0 as a, d@1 as d, row_n@4 as row_n] ---------CoalesceBatchesExec: target_batch_size=2 -----------HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(d@1, d@1)], filter=CAST(a@0 AS Int64) >= CAST(a@1 AS Int64) - 10 -------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, d], output_ordering=[a@0 ASC NULLS LAST], has_header=true -------------ProjectionExec: expr=[a@0 as a, d@1 as d, ROW_NUMBER() ORDER BY [r.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@2 as row_n] ---------------BoundedWindowAggExec: wdw=[ROW_NUMBER() ORDER BY [r.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "ROW_NUMBER() ORDER BY [r.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW", data_type: UInt64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(NULL)), end_bound: CurrentRow }], mode=[Sorted] -----------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, d], output_ordering=[a@0 ASC NULLS LAST], has_header=true +ProjectionExec: expr=[LAST_VALUE(l.d) ORDER BY [l.a ASC NULLS LAST]@1 as amount_usd] +--AggregateExec: mode=Single, gby=[row_n@2 as row_n], aggr=[LAST_VALUE(l.d)], ordering_mode=Sorted +----ProjectionExec: expr=[a@0 as a, d@1 as d, row_n@4 as row_n] +------CoalesceBatchesExec: target_batch_size=2 +--------HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(d@1, d@1)], filter=CAST(a@0 AS Int64) >= CAST(a@1 AS Int64) - 10 +----------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, d], output_ordering=[a@0 ASC NULLS LAST], has_header=true +----------ProjectionExec: expr=[a@0 as a, d@1 as d, ROW_NUMBER() ORDER BY [r.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@2 as row_n] +------------BoundedWindowAggExec: wdw=[ROW_NUMBER() ORDER BY [r.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "ROW_NUMBER() ORDER BY [r.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW", data_type: UInt64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(NULL)), end_bound: CurrentRow }], mode=[Sorted] +--------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, d], output_ordering=[a@0 ASC NULLS LAST], has_header=true # reset partition number to 8. statement ok diff --git a/datafusion/sqllogictest/test_files/insert.slt b/datafusion/sqllogictest/test_files/insert.slt index 0c63a3481996..8b9fd52e0d94 100644 --- a/datafusion/sqllogictest/test_files/insert.slt +++ b/datafusion/sqllogictest/test_files/insert.slt @@ -218,9 +218,8 @@ Dml: op=[Insert Into] table=[table_without_values] ------TableScan: aggregate_test_100 projection=[c1] physical_plan FileSinkExec: sink=MemoryTable (partitions=1) ---ProjectionExec: expr=[c1@0 as c1] -----SortExec: expr=[c1@0 ASC NULLS LAST] -------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1], has_header=true +--SortExec: expr=[c1@0 ASC NULLS LAST] +----CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1], has_header=true query T insert into table_without_values select c1 from aggregate_test_100 order by c1; diff --git a/datafusion/sqllogictest/test_files/insert_to_external.slt b/datafusion/sqllogictest/test_files/insert_to_external.slt index fa1d646d1413..d6449bc2726e 100644 --- a/datafusion/sqllogictest/test_files/insert_to_external.slt +++ b/datafusion/sqllogictest/test_files/insert_to_external.slt @@ -423,9 +423,8 @@ Dml: op=[Insert Into] table=[table_without_values] ------TableScan: aggregate_test_100 projection=[c1] physical_plan FileSinkExec: sink=ParquetSink(writer_mode=PutMultipart, file_groups=[]) ---ProjectionExec: expr=[c1@0 as c1] -----SortExec: expr=[c1@0 ASC NULLS LAST] -------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1], has_header=true +--SortExec: expr=[c1@0 ASC NULLS LAST] +----CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1], has_header=true query T insert into table_without_values select c1 from aggregate_test_100 order by c1; diff --git a/datafusion/sqllogictest/test_files/joins.slt b/datafusion/sqllogictest/test_files/joins.slt index 25ab2032f0b0..24893297f163 100644 --- a/datafusion/sqllogictest/test_files/joins.slt +++ b/datafusion/sqllogictest/test_files/joins.slt @@ -1441,17 +1441,16 @@ Projection: join_t1.t1_id, join_t1.t1_name, join_t1.t1_int, join_t2.t2_id, join_ ----TableScan: join_t1 projection=[t1_id, t1_name, t1_int] ----TableScan: join_t2 projection=[t2_id, t2_name, t2_int] physical_plan -ProjectionExec: expr=[t1_id@0 as t1_id, t1_name@1 as t1_name, t1_int@2 as t1_int, t2_id@3 as t2_id, t2_name@4 as t2_name, t2_int@5 as t2_int, CAST(t1_id@0 AS Int64) + 11 as join_t1.t1_id + Int64(11)] ---ProjectionExec: expr=[t1_id@0 as t1_id, t1_name@1 as t1_name, t1_int@2 as t1_int, t2_id@4 as t2_id, t2_name@5 as t2_name, t2_int@6 as t2_int] -----CoalesceBatchesExec: target_batch_size=2 -------HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(join_t1.t1_id + Int64(11)@3, CAST(join_t2.t2_id AS Int64)@3)] ---------CoalescePartitionsExec -----------ProjectionExec: expr=[t1_id@0 as t1_id, t1_name@1 as t1_name, t1_int@2 as t1_int, CAST(t1_id@0 AS Int64) + 11 as join_t1.t1_id + Int64(11)] -------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 ---------------MemoryExec: partitions=1, partition_sizes=[1] ---------ProjectionExec: expr=[t2_id@0 as t2_id, t2_name@1 as t2_name, t2_int@2 as t2_int, CAST(t2_id@0 AS Int64) as CAST(join_t2.t2_id AS Int64)] +ProjectionExec: expr=[t1_id@0 as t1_id, t1_name@1 as t1_name, t1_int@2 as t1_int, t2_id@4 as t2_id, t2_name@5 as t2_name, t2_int@6 as t2_int, CAST(t1_id@0 AS Int64) + 11 as join_t1.t1_id + Int64(11)] +--CoalesceBatchesExec: target_batch_size=2 +----HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(join_t1.t1_id + Int64(11)@3, CAST(join_t2.t2_id AS Int64)@3)] +------CoalescePartitionsExec +--------ProjectionExec: expr=[t1_id@0 as t1_id, t1_name@1 as t1_name, t1_int@2 as t1_int, CAST(t1_id@0 AS Int64) + 11 as join_t1.t1_id + Int64(11)] ----------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 ------------MemoryExec: partitions=1, partition_sizes=[1] +------ProjectionExec: expr=[t2_id@0 as t2_id, t2_name@1 as t2_name, t2_int@2 as t2_int, CAST(t2_id@0 AS Int64) as CAST(join_t2.t2_id AS Int64)] +--------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +----------MemoryExec: partitions=1, partition_sizes=[1] statement ok set datafusion.optimizer.repartition_joins = true; @@ -1468,20 +1467,19 @@ Projection: join_t1.t1_id, join_t1.t1_name, join_t1.t1_int, join_t2.t2_id, join_ ----TableScan: join_t1 projection=[t1_id, t1_name, t1_int] ----TableScan: join_t2 projection=[t2_id, t2_name, t2_int] physical_plan -ProjectionExec: expr=[t1_id@0 as t1_id, t1_name@1 as t1_name, t1_int@2 as t1_int, t2_id@3 as t2_id, t2_name@4 as t2_name, t2_int@5 as t2_int, CAST(t1_id@0 AS Int64) + 11 as join_t1.t1_id + Int64(11)] ---ProjectionExec: expr=[t1_id@0 as t1_id, t1_name@1 as t1_name, t1_int@2 as t1_int, t2_id@4 as t2_id, t2_name@5 as t2_name, t2_int@6 as t2_int] -----CoalesceBatchesExec: target_batch_size=2 -------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(join_t1.t1_id + Int64(11)@3, CAST(join_t2.t2_id AS Int64)@3)] ---------CoalesceBatchesExec: target_batch_size=2 -----------RepartitionExec: partitioning=Hash([join_t1.t1_id + Int64(11)@3], 2), input_partitions=2 -------------ProjectionExec: expr=[t1_id@0 as t1_id, t1_name@1 as t1_name, t1_int@2 as t1_int, CAST(t1_id@0 AS Int64) + 11 as join_t1.t1_id + Int64(11)] ---------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -----------------MemoryExec: partitions=1, partition_sizes=[1] ---------CoalesceBatchesExec: target_batch_size=2 -----------RepartitionExec: partitioning=Hash([CAST(join_t2.t2_id AS Int64)@3], 2), input_partitions=2 -------------ProjectionExec: expr=[t2_id@0 as t2_id, t2_name@1 as t2_name, t2_int@2 as t2_int, CAST(t2_id@0 AS Int64) as CAST(join_t2.t2_id AS Int64)] ---------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -----------------MemoryExec: partitions=1, partition_sizes=[1] +ProjectionExec: expr=[t1_id@0 as t1_id, t1_name@1 as t1_name, t1_int@2 as t1_int, t2_id@4 as t2_id, t2_name@5 as t2_name, t2_int@6 as t2_int, CAST(t1_id@0 AS Int64) + 11 as join_t1.t1_id + Int64(11)] +--CoalesceBatchesExec: target_batch_size=2 +----HashJoinExec: mode=Partitioned, join_type=Inner, on=[(join_t1.t1_id + Int64(11)@3, CAST(join_t2.t2_id AS Int64)@3)] +------CoalesceBatchesExec: target_batch_size=2 +--------RepartitionExec: partitioning=Hash([join_t1.t1_id + Int64(11)@3], 2), input_partitions=2 +----------ProjectionExec: expr=[t1_id@0 as t1_id, t1_name@1 as t1_name, t1_int@2 as t1_int, CAST(t1_id@0 AS Int64) + 11 as join_t1.t1_id + Int64(11)] +------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +--------------MemoryExec: partitions=1, partition_sizes=[1] +------CoalesceBatchesExec: target_batch_size=2 +--------RepartitionExec: partitioning=Hash([CAST(join_t2.t2_id AS Int64)@3], 2), input_partitions=2 +----------ProjectionExec: expr=[t2_id@0 as t2_id, t2_name@1 as t2_name, t2_int@2 as t2_int, CAST(t2_id@0 AS Int64) as CAST(join_t2.t2_id AS Int64)] +------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +--------------MemoryExec: partitions=1, partition_sizes=[1] # Both side expr key inner join @@ -1500,18 +1498,16 @@ Projection: join_t1.t1_id, join_t2.t2_id, join_t1.t1_name ----TableScan: join_t1 projection=[t1_id, t1_name] ----TableScan: join_t2 projection=[t2_id] physical_plan -ProjectionExec: expr=[t1_id@0 as t1_id, t2_id@2 as t2_id, t1_name@1 as t1_name] ---ProjectionExec: expr=[t1_id@0 as t1_id, t1_name@1 as t1_name, t2_id@3 as t2_id] -----ProjectionExec: expr=[t1_id@2 as t1_id, t1_name@3 as t1_name, join_t1.t1_id + UInt32(12)@4 as join_t1.t1_id + UInt32(12), t2_id@0 as t2_id, join_t2.t2_id + UInt32(1)@1 as join_t2.t2_id + UInt32(1)] -------CoalesceBatchesExec: target_batch_size=2 ---------HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(join_t2.t2_id + UInt32(1)@1, join_t1.t1_id + UInt32(12)@2)] -----------CoalescePartitionsExec -------------ProjectionExec: expr=[t2_id@0 as t2_id, t2_id@0 + 1 as join_t2.t2_id + UInt32(1)] ---------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -----------------MemoryExec: partitions=1, partition_sizes=[1] -----------ProjectionExec: expr=[t1_id@0 as t1_id, t1_name@1 as t1_name, t1_id@0 + 12 as join_t1.t1_id + UInt32(12)] -------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 ---------------MemoryExec: partitions=1, partition_sizes=[1] +ProjectionExec: expr=[t1_id@2 as t1_id, t2_id@0 as t2_id, t1_name@3 as t1_name] +--CoalesceBatchesExec: target_batch_size=2 +----HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(join_t2.t2_id + UInt32(1)@1, join_t1.t1_id + UInt32(12)@2)] +------CoalescePartitionsExec +--------ProjectionExec: expr=[t2_id@0 as t2_id, t2_id@0 + 1 as join_t2.t2_id + UInt32(1)] +----------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +------------MemoryExec: partitions=1, partition_sizes=[1] +------ProjectionExec: expr=[t1_id@0 as t1_id, t1_name@1 as t1_name, t1_id@0 + 12 as join_t1.t1_id + UInt32(12)] +--------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +----------MemoryExec: partitions=1, partition_sizes=[1] statement ok set datafusion.optimizer.repartition_joins = true; @@ -1528,21 +1524,19 @@ Projection: join_t1.t1_id, join_t2.t2_id, join_t1.t1_name ----TableScan: join_t1 projection=[t1_id, t1_name] ----TableScan: join_t2 projection=[t2_id] physical_plan -ProjectionExec: expr=[t1_id@0 as t1_id, t2_id@2 as t2_id, t1_name@1 as t1_name] ---ProjectionExec: expr=[t1_id@0 as t1_id, t1_name@1 as t1_name, t2_id@3 as t2_id] -----ProjectionExec: expr=[t1_id@2 as t1_id, t1_name@3 as t1_name, join_t1.t1_id + UInt32(12)@4 as join_t1.t1_id + UInt32(12), t2_id@0 as t2_id, join_t2.t2_id + UInt32(1)@1 as join_t2.t2_id + UInt32(1)] +ProjectionExec: expr=[t1_id@2 as t1_id, t2_id@0 as t2_id, t1_name@3 as t1_name] +--CoalesceBatchesExec: target_batch_size=2 +----HashJoinExec: mode=Partitioned, join_type=Inner, on=[(join_t2.t2_id + UInt32(1)@1, join_t1.t1_id + UInt32(12)@2)] ------CoalesceBatchesExec: target_batch_size=2 ---------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(join_t2.t2_id + UInt32(1)@1, join_t1.t1_id + UInt32(12)@2)] -----------CoalesceBatchesExec: target_batch_size=2 -------------RepartitionExec: partitioning=Hash([join_t2.t2_id + UInt32(1)@1], 2), input_partitions=2 ---------------ProjectionExec: expr=[t2_id@0 as t2_id, t2_id@0 + 1 as join_t2.t2_id + UInt32(1)] -----------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -------------------MemoryExec: partitions=1, partition_sizes=[1] -----------CoalesceBatchesExec: target_batch_size=2 -------------RepartitionExec: partitioning=Hash([join_t1.t1_id + UInt32(12)@2], 2), input_partitions=2 ---------------ProjectionExec: expr=[t1_id@0 as t1_id, t1_name@1 as t1_name, t1_id@0 + 12 as join_t1.t1_id + UInt32(12)] -----------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -------------------MemoryExec: partitions=1, partition_sizes=[1] +--------RepartitionExec: partitioning=Hash([join_t2.t2_id + UInt32(1)@1], 2), input_partitions=2 +----------ProjectionExec: expr=[t2_id@0 as t2_id, t2_id@0 + 1 as join_t2.t2_id + UInt32(1)] +------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +--------------MemoryExec: partitions=1, partition_sizes=[1] +------CoalesceBatchesExec: target_batch_size=2 +--------RepartitionExec: partitioning=Hash([join_t1.t1_id + UInt32(12)@2], 2), input_partitions=2 +----------ProjectionExec: expr=[t1_id@0 as t1_id, t1_name@1 as t1_name, t1_id@0 + 12 as join_t1.t1_id + UInt32(12)] +------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +--------------MemoryExec: partitions=1, partition_sizes=[1] # Left side expr key inner join @@ -1562,16 +1556,15 @@ Projection: join_t1.t1_id, join_t2.t2_id, join_t1.t1_name ----TableScan: join_t1 projection=[t1_id, t1_name] ----TableScan: join_t2 projection=[t2_id] physical_plan -ProjectionExec: expr=[t1_id@0 as t1_id, t2_id@2 as t2_id, t1_name@1 as t1_name] ---ProjectionExec: expr=[t1_id@0 as t1_id, t1_name@1 as t1_name, t2_id@3 as t2_id] -----CoalesceBatchesExec: target_batch_size=2 -------HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(join_t1.t1_id + UInt32(11)@2, t2_id@0)] ---------CoalescePartitionsExec -----------ProjectionExec: expr=[t1_id@0 as t1_id, t1_name@1 as t1_name, t1_id@0 + 11 as join_t1.t1_id + UInt32(11)] -------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 ---------------MemoryExec: partitions=1, partition_sizes=[1] ---------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -----------MemoryExec: partitions=1, partition_sizes=[1] +ProjectionExec: expr=[t1_id@0 as t1_id, t2_id@3 as t2_id, t1_name@1 as t1_name] +--CoalesceBatchesExec: target_batch_size=2 +----HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(join_t1.t1_id + UInt32(11)@2, t2_id@0)] +------CoalescePartitionsExec +--------ProjectionExec: expr=[t1_id@0 as t1_id, t1_name@1 as t1_name, t1_id@0 + 11 as join_t1.t1_id + UInt32(11)] +----------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +------------MemoryExec: partitions=1, partition_sizes=[1] +------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +--------MemoryExec: partitions=1, partition_sizes=[1] statement ok set datafusion.optimizer.repartition_joins = true; @@ -1589,19 +1582,18 @@ Projection: join_t1.t1_id, join_t2.t2_id, join_t1.t1_name ----TableScan: join_t1 projection=[t1_id, t1_name] ----TableScan: join_t2 projection=[t2_id] physical_plan -ProjectionExec: expr=[t1_id@0 as t1_id, t2_id@2 as t2_id, t1_name@1 as t1_name] ---ProjectionExec: expr=[t1_id@0 as t1_id, t1_name@1 as t1_name, t2_id@3 as t2_id] -----CoalesceBatchesExec: target_batch_size=2 -------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(join_t1.t1_id + UInt32(11)@2, t2_id@0)] ---------CoalesceBatchesExec: target_batch_size=2 -----------RepartitionExec: partitioning=Hash([join_t1.t1_id + UInt32(11)@2], 2), input_partitions=2 -------------ProjectionExec: expr=[t1_id@0 as t1_id, t1_name@1 as t1_name, t1_id@0 + 11 as join_t1.t1_id + UInt32(11)] ---------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -----------------MemoryExec: partitions=1, partition_sizes=[1] ---------CoalesceBatchesExec: target_batch_size=2 -----------RepartitionExec: partitioning=Hash([t2_id@0], 2), input_partitions=2 +ProjectionExec: expr=[t1_id@0 as t1_id, t2_id@3 as t2_id, t1_name@1 as t1_name] +--CoalesceBatchesExec: target_batch_size=2 +----HashJoinExec: mode=Partitioned, join_type=Inner, on=[(join_t1.t1_id + UInt32(11)@2, t2_id@0)] +------CoalesceBatchesExec: target_batch_size=2 +--------RepartitionExec: partitioning=Hash([join_t1.t1_id + UInt32(11)@2], 2), input_partitions=2 +----------ProjectionExec: expr=[t1_id@0 as t1_id, t1_name@1 as t1_name, t1_id@0 + 11 as join_t1.t1_id + UInt32(11)] ------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 --------------MemoryExec: partitions=1, partition_sizes=[1] +------CoalesceBatchesExec: target_batch_size=2 +--------RepartitionExec: partitioning=Hash([t2_id@0], 2), input_partitions=2 +----------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +------------MemoryExec: partitions=1, partition_sizes=[1] # Right side expr key inner join @@ -1621,17 +1613,15 @@ Projection: join_t1.t1_id, join_t2.t2_id, join_t1.t1_name ----TableScan: join_t1 projection=[t1_id, t1_name] ----TableScan: join_t2 projection=[t2_id] physical_plan -ProjectionExec: expr=[t1_id@0 as t1_id, t2_id@2 as t2_id, t1_name@1 as t1_name] ---ProjectionExec: expr=[t1_id@0 as t1_id, t1_name@1 as t1_name, t2_id@2 as t2_id] -----ProjectionExec: expr=[t1_id@2 as t1_id, t1_name@3 as t1_name, t2_id@0 as t2_id, join_t2.t2_id - UInt32(11)@1 as join_t2.t2_id - UInt32(11)] -------CoalesceBatchesExec: target_batch_size=2 ---------HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(join_t2.t2_id - UInt32(11)@1, t1_id@0)] -----------CoalescePartitionsExec -------------ProjectionExec: expr=[t2_id@0 as t2_id, t2_id@0 - 11 as join_t2.t2_id - UInt32(11)] ---------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -----------------MemoryExec: partitions=1, partition_sizes=[1] +ProjectionExec: expr=[t1_id@2 as t1_id, t2_id@0 as t2_id, t1_name@3 as t1_name] +--CoalesceBatchesExec: target_batch_size=2 +----HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(join_t2.t2_id - UInt32(11)@1, t1_id@0)] +------CoalescePartitionsExec +--------ProjectionExec: expr=[t2_id@0 as t2_id, t2_id@0 - 11 as join_t2.t2_id - UInt32(11)] ----------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 ------------MemoryExec: partitions=1, partition_sizes=[1] +------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +--------MemoryExec: partitions=1, partition_sizes=[1] statement ok set datafusion.optimizer.repartition_joins = true; @@ -1649,20 +1639,18 @@ Projection: join_t1.t1_id, join_t2.t2_id, join_t1.t1_name ----TableScan: join_t1 projection=[t1_id, t1_name] ----TableScan: join_t2 projection=[t2_id] physical_plan -ProjectionExec: expr=[t1_id@0 as t1_id, t2_id@2 as t2_id, t1_name@1 as t1_name] ---ProjectionExec: expr=[t1_id@0 as t1_id, t1_name@1 as t1_name, t2_id@2 as t2_id] -----ProjectionExec: expr=[t1_id@2 as t1_id, t1_name@3 as t1_name, t2_id@0 as t2_id, join_t2.t2_id - UInt32(11)@1 as join_t2.t2_id - UInt32(11)] +ProjectionExec: expr=[t1_id@2 as t1_id, t2_id@0 as t2_id, t1_name@3 as t1_name] +--CoalesceBatchesExec: target_batch_size=2 +----HashJoinExec: mode=Partitioned, join_type=Inner, on=[(join_t2.t2_id - UInt32(11)@1, t1_id@0)] ------CoalesceBatchesExec: target_batch_size=2 ---------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(join_t2.t2_id - UInt32(11)@1, t1_id@0)] -----------CoalesceBatchesExec: target_batch_size=2 -------------RepartitionExec: partitioning=Hash([join_t2.t2_id - UInt32(11)@1], 2), input_partitions=2 ---------------ProjectionExec: expr=[t2_id@0 as t2_id, t2_id@0 - 11 as join_t2.t2_id - UInt32(11)] -----------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -------------------MemoryExec: partitions=1, partition_sizes=[1] -----------CoalesceBatchesExec: target_batch_size=2 -------------RepartitionExec: partitioning=Hash([t1_id@0], 2), input_partitions=2 ---------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -----------------MemoryExec: partitions=1, partition_sizes=[1] +--------RepartitionExec: partitioning=Hash([join_t2.t2_id - UInt32(11)@1], 2), input_partitions=2 +----------ProjectionExec: expr=[t2_id@0 as t2_id, t2_id@0 - 11 as join_t2.t2_id - UInt32(11)] +------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +--------------MemoryExec: partitions=1, partition_sizes=[1] +------CoalesceBatchesExec: target_batch_size=2 +--------RepartitionExec: partitioning=Hash([t1_id@0], 2), input_partitions=2 +----------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +------------MemoryExec: partitions=1, partition_sizes=[1] # Select wildcard with expr key inner join @@ -3347,16 +3335,15 @@ Projection: amount_usd ----------------SubqueryAlias: r ------------------TableScan: multiple_ordered_table projection=[a, d] physical_plan -ProjectionExec: expr=[amount_usd@0 as amount_usd] ---ProjectionExec: expr=[LAST_VALUE(l.d) ORDER BY [l.a ASC NULLS LAST]@1 as amount_usd, row_n@0 as row_n] -----AggregateExec: mode=Single, gby=[row_n@2 as row_n], aggr=[LAST_VALUE(l.d)], ordering_mode=Sorted -------ProjectionExec: expr=[a@0 as a, d@1 as d, row_n@4 as row_n] ---------CoalesceBatchesExec: target_batch_size=2 -----------HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(d@1, d@1)], filter=CAST(a@0 AS Int64) >= CAST(a@1 AS Int64) - 10 -------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, d], output_ordering=[a@0 ASC NULLS LAST], has_header=true -------------ProjectionExec: expr=[a@0 as a, d@1 as d, ROW_NUMBER() ORDER BY [r.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@2 as row_n] ---------------BoundedWindowAggExec: wdw=[ROW_NUMBER() ORDER BY [r.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "ROW_NUMBER() ORDER BY [r.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW", data_type: UInt64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(NULL)), end_bound: CurrentRow }], mode=[Sorted] -----------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, d], output_ordering=[a@0 ASC NULLS LAST], has_header=true +ProjectionExec: expr=[LAST_VALUE(l.d) ORDER BY [l.a ASC NULLS LAST]@1 as amount_usd] +--AggregateExec: mode=Single, gby=[row_n@2 as row_n], aggr=[LAST_VALUE(l.d)], ordering_mode=Sorted +----ProjectionExec: expr=[a@0 as a, d@1 as d, row_n@4 as row_n] +------CoalesceBatchesExec: target_batch_size=2 +--------HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(d@1, d@1)], filter=CAST(a@0 AS Int64) >= CAST(a@1 AS Int64) - 10 +----------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, d], output_ordering=[a@0 ASC NULLS LAST], has_header=true +----------ProjectionExec: expr=[a@0 as a, d@1 as d, ROW_NUMBER() ORDER BY [r.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@2 as row_n] +------------BoundedWindowAggExec: wdw=[ROW_NUMBER() ORDER BY [r.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "ROW_NUMBER() ORDER BY [r.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW", data_type: UInt64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(NULL)), end_bound: CurrentRow }], mode=[Sorted] +--------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, d], output_ordering=[a@0 ASC NULLS LAST], has_header=true # run query above in multiple partitions statement ok diff --git a/datafusion/sqllogictest/test_files/subquery.slt b/datafusion/sqllogictest/test_files/subquery.slt index 822a70bb5bad..ef08c88a9d20 100644 --- a/datafusion/sqllogictest/test_files/subquery.slt +++ b/datafusion/sqllogictest/test_files/subquery.slt @@ -180,19 +180,18 @@ Projection: t1.t1_id, __scalar_sq_1.SUM(t2.t2_int) AS t2_sum --------Aggregate: groupBy=[[t2.t2_id]], aggr=[[SUM(CAST(t2.t2_int AS Int64))]] ----------TableScan: t2 projection=[t2_id, t2_int] physical_plan -ProjectionExec: expr=[t1_id@0 as t1_id, SUM(t2.t2_int)@1 as t2_sum] ---ProjectionExec: expr=[t1_id@2 as t1_id, SUM(t2.t2_int)@0 as SUM(t2.t2_int), t2_id@1 as t2_id] -----CoalesceBatchesExec: target_batch_size=2 -------HashJoinExec: mode=Partitioned, join_type=Right, on=[(t2_id@1, t1_id@0)] ---------ProjectionExec: expr=[SUM(t2.t2_int)@1 as SUM(t2.t2_int), t2_id@0 as t2_id] -----------AggregateExec: mode=FinalPartitioned, gby=[t2_id@0 as t2_id], aggr=[SUM(t2.t2_int)] -------------CoalesceBatchesExec: target_batch_size=2 ---------------RepartitionExec: partitioning=Hash([t2_id@0], 4), input_partitions=4 -----------------AggregateExec: mode=Partial, gby=[t2_id@0 as t2_id], aggr=[SUM(t2.t2_int)] -------------------MemoryExec: partitions=4, partition_sizes=[1, 0, 0, 0] ---------CoalesceBatchesExec: target_batch_size=2 -----------RepartitionExec: partitioning=Hash([t1_id@0], 4), input_partitions=4 -------------MemoryExec: partitions=4, partition_sizes=[1, 0, 0, 0] +ProjectionExec: expr=[t1_id@2 as t1_id, SUM(t2.t2_int)@0 as t2_sum] +--CoalesceBatchesExec: target_batch_size=2 +----HashJoinExec: mode=Partitioned, join_type=Right, on=[(t2_id@1, t1_id@0)] +------ProjectionExec: expr=[SUM(t2.t2_int)@1 as SUM(t2.t2_int), t2_id@0 as t2_id] +--------AggregateExec: mode=FinalPartitioned, gby=[t2_id@0 as t2_id], aggr=[SUM(t2.t2_int)] +----------CoalesceBatchesExec: target_batch_size=2 +------------RepartitionExec: partitioning=Hash([t2_id@0], 4), input_partitions=4 +--------------AggregateExec: mode=Partial, gby=[t2_id@0 as t2_id], aggr=[SUM(t2.t2_int)] +----------------MemoryExec: partitions=4, partition_sizes=[1, 0, 0, 0] +------CoalesceBatchesExec: target_batch_size=2 +--------RepartitionExec: partitioning=Hash([t1_id@0], 4), input_partitions=4 +----------MemoryExec: partitions=4, partition_sizes=[1, 0, 0, 0] query II rowsort SELECT t1_id, (SELECT sum(t2_int) FROM t2 WHERE t2.t2_id = t1.t1_id) as t2_sum from t1 @@ -215,19 +214,18 @@ Projection: t1.t1_id, __scalar_sq_1.SUM(t2.t2_int * Float64(1)) + Int64(1) AS t2 --------Aggregate: groupBy=[[t2.t2_id]], aggr=[[SUM(CAST(t2.t2_int AS Float64)) AS SUM(t2.t2_int * Float64(1))]] ----------TableScan: t2 projection=[t2_id, t2_int] physical_plan -ProjectionExec: expr=[t1_id@0 as t1_id, SUM(t2.t2_int * Float64(1)) + Int64(1)@1 as t2_sum] ---ProjectionExec: expr=[t1_id@2 as t1_id, SUM(t2.t2_int * Float64(1)) + Int64(1)@0 as SUM(t2.t2_int * Float64(1)) + Int64(1), t2_id@1 as t2_id] -----CoalesceBatchesExec: target_batch_size=2 -------HashJoinExec: mode=Partitioned, join_type=Right, on=[(t2_id@1, t1_id@0)] ---------ProjectionExec: expr=[SUM(t2.t2_int * Float64(1))@1 + 1 as SUM(t2.t2_int * Float64(1)) + Int64(1), t2_id@0 as t2_id] -----------AggregateExec: mode=FinalPartitioned, gby=[t2_id@0 as t2_id], aggr=[SUM(t2.t2_int * Float64(1))] -------------CoalesceBatchesExec: target_batch_size=2 ---------------RepartitionExec: partitioning=Hash([t2_id@0], 4), input_partitions=4 -----------------AggregateExec: mode=Partial, gby=[t2_id@0 as t2_id], aggr=[SUM(t2.t2_int * Float64(1))] -------------------MemoryExec: partitions=4, partition_sizes=[1, 0, 0, 0] ---------CoalesceBatchesExec: target_batch_size=2 -----------RepartitionExec: partitioning=Hash([t1_id@0], 4), input_partitions=4 -------------MemoryExec: partitions=4, partition_sizes=[1, 0, 0, 0] +ProjectionExec: expr=[t1_id@2 as t1_id, SUM(t2.t2_int * Float64(1)) + Int64(1)@0 as t2_sum] +--CoalesceBatchesExec: target_batch_size=2 +----HashJoinExec: mode=Partitioned, join_type=Right, on=[(t2_id@1, t1_id@0)] +------ProjectionExec: expr=[SUM(t2.t2_int * Float64(1))@1 + 1 as SUM(t2.t2_int * Float64(1)) + Int64(1), t2_id@0 as t2_id] +--------AggregateExec: mode=FinalPartitioned, gby=[t2_id@0 as t2_id], aggr=[SUM(t2.t2_int * Float64(1))] +----------CoalesceBatchesExec: target_batch_size=2 +------------RepartitionExec: partitioning=Hash([t2_id@0], 4), input_partitions=4 +--------------AggregateExec: mode=Partial, gby=[t2_id@0 as t2_id], aggr=[SUM(t2.t2_int * Float64(1))] +----------------MemoryExec: partitions=4, partition_sizes=[1, 0, 0, 0] +------CoalesceBatchesExec: target_batch_size=2 +--------RepartitionExec: partitioning=Hash([t1_id@0], 4), input_partitions=4 +----------MemoryExec: partitions=4, partition_sizes=[1, 0, 0, 0] query IR rowsort SELECT t1_id, (SELECT sum(t2_int * 1.0) + 1 FROM t2 WHERE t2.t2_id = t1.t1_id) as t2_sum from t1 @@ -287,21 +285,20 @@ Projection: t1.t1_id, __scalar_sq_1.SUM(t2.t2_int) AS t2_sum ----------Aggregate: groupBy=[[t2.t2_id]], aggr=[[SUM(CAST(t2.t2_int AS Int64))]] ------------TableScan: t2 projection=[t2_id, t2_int] physical_plan -ProjectionExec: expr=[t1_id@0 as t1_id, SUM(t2.t2_int)@1 as t2_sum] ---ProjectionExec: expr=[t1_id@2 as t1_id, SUM(t2.t2_int)@0 as SUM(t2.t2_int), t2_id@1 as t2_id] -----CoalesceBatchesExec: target_batch_size=2 -------HashJoinExec: mode=Partitioned, join_type=Right, on=[(t2_id@1, t1_id@0)] ---------ProjectionExec: expr=[SUM(t2.t2_int)@1 as SUM(t2.t2_int), t2_id@0 as t2_id] -----------CoalesceBatchesExec: target_batch_size=2 -------------FilterExec: SUM(t2.t2_int)@1 < 3 ---------------AggregateExec: mode=FinalPartitioned, gby=[t2_id@0 as t2_id], aggr=[SUM(t2.t2_int)] -----------------CoalesceBatchesExec: target_batch_size=2 -------------------RepartitionExec: partitioning=Hash([t2_id@0], 4), input_partitions=4 ---------------------AggregateExec: mode=Partial, gby=[t2_id@0 as t2_id], aggr=[SUM(t2.t2_int)] -----------------------MemoryExec: partitions=4, partition_sizes=[1, 0, 0, 0] +ProjectionExec: expr=[t1_id@2 as t1_id, SUM(t2.t2_int)@0 as t2_sum] +--CoalesceBatchesExec: target_batch_size=2 +----HashJoinExec: mode=Partitioned, join_type=Right, on=[(t2_id@1, t1_id@0)] +------ProjectionExec: expr=[SUM(t2.t2_int)@1 as SUM(t2.t2_int), t2_id@0 as t2_id] --------CoalesceBatchesExec: target_batch_size=2 -----------RepartitionExec: partitioning=Hash([t1_id@0], 4), input_partitions=4 -------------MemoryExec: partitions=4, partition_sizes=[1, 0, 0, 0] +----------FilterExec: SUM(t2.t2_int)@1 < 3 +------------AggregateExec: mode=FinalPartitioned, gby=[t2_id@0 as t2_id], aggr=[SUM(t2.t2_int)] +--------------CoalesceBatchesExec: target_batch_size=2 +----------------RepartitionExec: partitioning=Hash([t2_id@0], 4), input_partitions=4 +------------------AggregateExec: mode=Partial, gby=[t2_id@0 as t2_id], aggr=[SUM(t2.t2_int)] +--------------------MemoryExec: partitions=4, partition_sizes=[1, 0, 0, 0] +------CoalesceBatchesExec: target_batch_size=2 +--------RepartitionExec: partitioning=Hash([t1_id@0], 4), input_partitions=4 +----------MemoryExec: partitions=4, partition_sizes=[1, 0, 0, 0] query II rowsort SELECT t1_id, (SELECT sum(t2_int) FROM t2 WHERE t2.t2_id = t1.t1_id having sum(t2_int) < 3) as t2_sum from t1 diff --git a/datafusion/sqllogictest/test_files/union.slt b/datafusion/sqllogictest/test_files/union.slt index 688774c906fe..0f255cdb9fb9 100644 --- a/datafusion/sqllogictest/test_files/union.slt +++ b/datafusion/sqllogictest/test_files/union.slt @@ -272,37 +272,36 @@ Union ------TableScan: t1 projection=[id, name] physical_plan UnionExec ---ProjectionExec: expr=[id@0 as id, name@1 as name] -----CoalesceBatchesExec: target_batch_size=2 -------HashJoinExec: mode=Partitioned, join_type=LeftAnti, on=[(id@0, CAST(t2.id AS Int32)@2), (name@1, name@1)] ---------AggregateExec: mode=FinalPartitioned, gby=[id@0 as id, name@1 as name], aggr=[] -----------CoalesceBatchesExec: target_batch_size=2 -------------RepartitionExec: partitioning=Hash([id@0, name@1], 4), input_partitions=4 ---------------AggregateExec: mode=Partial, gby=[id@0 as id, name@1 as name], aggr=[] -----------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -------------------MemoryExec: partitions=1, partition_sizes=[1] +--CoalesceBatchesExec: target_batch_size=2 +----HashJoinExec: mode=Partitioned, join_type=LeftAnti, on=[(id@0, CAST(t2.id AS Int32)@2), (name@1, name@1)] +------AggregateExec: mode=FinalPartitioned, gby=[id@0 as id, name@1 as name], aggr=[] --------CoalesceBatchesExec: target_batch_size=2 -----------RepartitionExec: partitioning=Hash([CAST(t2.id AS Int32)@2, name@1], 4), input_partitions=4 -------------ProjectionExec: expr=[id@0 as id, name@1 as name, CAST(id@0 AS Int32) as CAST(t2.id AS Int32)] +----------RepartitionExec: partitioning=Hash([id@0, name@1], 4), input_partitions=4 +------------AggregateExec: mode=Partial, gby=[id@0 as id, name@1 as name], aggr=[] --------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 ----------------MemoryExec: partitions=1, partition_sizes=[1] ---ProjectionExec: expr=[CAST(id@0 AS Int32) as id, name@1 as name] -----ProjectionExec: expr=[id@0 as id, name@1 as name] ------CoalesceBatchesExec: target_batch_size=2 ---------HashJoinExec: mode=Partitioned, join_type=LeftAnti, on=[(CAST(t2.id AS Int32)@2, id@0), (name@1, name@1)] -----------CoalesceBatchesExec: target_batch_size=2 -------------RepartitionExec: partitioning=Hash([CAST(t2.id AS Int32)@2, name@1], 4), input_partitions=4 ---------------ProjectionExec: expr=[id@0 as id, name@1 as name, CAST(id@0 AS Int32) as CAST(t2.id AS Int32)] -----------------AggregateExec: mode=FinalPartitioned, gby=[id@0 as id, name@1 as name], aggr=[] -------------------CoalesceBatchesExec: target_batch_size=2 ---------------------RepartitionExec: partitioning=Hash([id@0, name@1], 4), input_partitions=4 -----------------------AggregateExec: mode=Partial, gby=[id@0 as id, name@1 as name], aggr=[] -------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 ---------------------------MemoryExec: partitions=1, partition_sizes=[1] -----------CoalesceBatchesExec: target_batch_size=2 -------------RepartitionExec: partitioning=Hash([id@0, name@1], 4), input_partitions=4 ---------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -----------------MemoryExec: partitions=1, partition_sizes=[1] +--------RepartitionExec: partitioning=Hash([CAST(t2.id AS Int32)@2, name@1], 4), input_partitions=4 +----------ProjectionExec: expr=[id@0 as id, name@1 as name, CAST(id@0 AS Int32) as CAST(t2.id AS Int32)] +------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +--------------MemoryExec: partitions=1, partition_sizes=[1] +--ProjectionExec: expr=[CAST(id@0 AS Int32) as id, name@1 as name] +----CoalesceBatchesExec: target_batch_size=2 +------HashJoinExec: mode=Partitioned, join_type=LeftAnti, on=[(CAST(t2.id AS Int32)@2, id@0), (name@1, name@1)] +--------CoalesceBatchesExec: target_batch_size=2 +----------RepartitionExec: partitioning=Hash([CAST(t2.id AS Int32)@2, name@1], 4), input_partitions=4 +------------ProjectionExec: expr=[id@0 as id, name@1 as name, CAST(id@0 AS Int32) as CAST(t2.id AS Int32)] +--------------AggregateExec: mode=FinalPartitioned, gby=[id@0 as id, name@1 as name], aggr=[] +----------------CoalesceBatchesExec: target_batch_size=2 +------------------RepartitionExec: partitioning=Hash([id@0, name@1], 4), input_partitions=4 +--------------------AggregateExec: mode=Partial, gby=[id@0 as id, name@1 as name], aggr=[] +----------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +------------------------MemoryExec: partitions=1, partition_sizes=[1] +--------CoalesceBatchesExec: target_batch_size=2 +----------RepartitionExec: partitioning=Hash([id@0, name@1], 4), input_partitions=4 +------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +--------------MemoryExec: partitions=1, partition_sizes=[1] + query IT rowsort ( @@ -580,7 +579,6 @@ UnionExec ----------AggregateExec: mode=Partial, gby=[n@0 as n], aggr=[COUNT(*)] ------------ProjectionExec: expr=[5 as n] --------------EmptyExec: produce_one_row=true ---ProjectionExec: expr=[x@0 as count, y@1 as n] -----ProjectionExec: expr=[1 as x, MAX(Int64(10))@0 as y] -------AggregateExec: mode=Single, gby=[], aggr=[MAX(Int64(10))] ---------EmptyExec: produce_one_row=true +--ProjectionExec: expr=[1 as count, MAX(Int64(10))@0 as n] +----AggregateExec: mode=Single, gby=[], aggr=[MAX(Int64(10))] +------EmptyExec: produce_one_row=true From 43cc870a951611e9081a462d5a8a1686e87fce9a Mon Sep 17 00:00:00 2001 From: Mark Sirek Date: Thu, 9 Nov 2023 06:18:57 -0800 Subject: [PATCH 213/572] Push limit into aggregation for DISTINCT ... LIMIT queries (#8038) * Push limit into AggregateExec for DISTINCT with GROUP BY * Soft limit for GroupedHashAggregateStream with no aggregate expressions * Add datafusion.optimizer.enable_distinct_aggregation_soft_limit setting * Fix result checking in topk_aggregate benchmark * Make the topk_aggregate benchmark's make_data function public * Add benchmark for DISTINCT queries * Fix doc formatting with prettier * Minor: Simply early emit logic in GroupByHash * remove level of indentation * Use '///' for function comments * Address review comments * rename transform_local_limit to transform_limit * Resolve conflicts * Update test after merge with main --------- Co-authored-by: Mark Sirek Co-authored-by: Andrew Lamb --- datafusion/common/src/config.rs | 5 + datafusion/common/src/tree_node.rs | 11 + datafusion/core/Cargo.toml | 4 + datafusion/core/benches/data_utils/mod.rs | 85 +++ datafusion/core/benches/distinct_query_sql.rs | 208 ++++++ datafusion/core/benches/topk_aggregate.rs | 92 +-- .../aggregate_statistics.rs | 8 +- .../combine_partial_final_agg.rs | 48 ++ .../enforce_distribution.rs | 8 +- .../limited_distinct_aggregation.rs | 626 ++++++++++++++++++ datafusion/core/src/physical_optimizer/mod.rs | 1 + .../core/src/physical_optimizer/optimizer.rs | 5 + .../physical_optimizer/topk_aggregation.rs | 13 +- .../physical-plan/src/aggregates/mod.rs | 46 +- .../physical-plan/src/aggregates/row_hash.rs | 61 +- .../sqllogictest/test_files/aggregate.slt | 198 ++++++ .../sqllogictest/test_files/explain.slt | 1 + .../test_files/information_schema.slt | 2 + docs/source/user-guide/configs.md | 1 + 19 files changed, 1299 insertions(+), 124 deletions(-) create mode 100644 datafusion/core/benches/distinct_query_sql.rs create mode 100644 datafusion/core/src/physical_optimizer/limited_distinct_aggregation.rs diff --git a/datafusion/common/src/config.rs b/datafusion/common/src/config.rs index 403241fcce58..ba2072ecc151 100644 --- a/datafusion/common/src/config.rs +++ b/datafusion/common/src/config.rs @@ -427,6 +427,11 @@ config_namespace! { config_namespace! { /// Options related to query optimization pub struct OptimizerOptions { + /// When set to true, the optimizer will push a limit operation into + /// grouped aggregations which have no aggregate expressions, as a soft limit, + /// emitting groups once the limit is reached, before all rows in the group are read. + pub enable_distinct_aggregation_soft_limit: bool, default = true + /// When set to true, the physical plan optimizer will try to add round robin /// repartitioning to increase parallelism to leverage more CPU cores pub enable_round_robin_repartition: bool, default = true diff --git a/datafusion/common/src/tree_node.rs b/datafusion/common/src/tree_node.rs index 2919d9a39c9c..d0ef507294cc 100644 --- a/datafusion/common/src/tree_node.rs +++ b/datafusion/common/src/tree_node.rs @@ -125,6 +125,17 @@ pub trait TreeNode: Sized { after_op.map_children(|node| node.transform_down(op)) } + /// Convenience utils for writing optimizers rule: recursively apply the given 'op' to the node and all of its + /// children(Preorder Traversal) using a mutable function, `F`. + /// When the `op` does not apply to a given node, it is left unchanged. + fn transform_down_mut(self, op: &mut F) -> Result + where + F: FnMut(Self) -> Result>, + { + let after_op = op(self)?.into(); + after_op.map_children(|node| node.transform_down_mut(op)) + } + /// Convenience utils for writing optimizers rule: recursively apply the given 'op' first to all of its /// children and then itself(Postorder Traversal). /// When the `op` does not apply to a given node, it is left unchanged. diff --git a/datafusion/core/Cargo.toml b/datafusion/core/Cargo.toml index 80aec800d697..0b7aa1509820 100644 --- a/datafusion/core/Cargo.toml +++ b/datafusion/core/Cargo.toml @@ -120,6 +120,10 @@ nix = { version = "0.27.1", features = ["fs"] } harness = false name = "aggregate_query_sql" +[[bench]] +harness = false +name = "distinct_query_sql" + [[bench]] harness = false name = "sort_limit_query_sql" diff --git a/datafusion/core/benches/data_utils/mod.rs b/datafusion/core/benches/data_utils/mod.rs index 64c0e4b100a1..9d2864919225 100644 --- a/datafusion/core/benches/data_utils/mod.rs +++ b/datafusion/core/benches/data_utils/mod.rs @@ -25,11 +25,16 @@ use arrow::{ datatypes::{DataType, Field, Schema, SchemaRef}, record_batch::RecordBatch, }; +use arrow_array::builder::{Int64Builder, StringBuilder}; use datafusion::datasource::MemTable; use datafusion::error::Result; +use datafusion_common::DataFusionError; use rand::rngs::StdRng; use rand::seq::SliceRandom; use rand::{Rng, SeedableRng}; +use rand_distr::Distribution; +use rand_distr::{Normal, Pareto}; +use std::fmt::Write; use std::sync::Arc; /// create an in-memory table given the partition len, array len, and batch size, @@ -156,3 +161,83 @@ pub fn create_record_batches( }) .collect::>() } + +/// Create time series data with `partition_cnt` partitions and `sample_cnt` rows per partition +/// in ascending order, if `asc` is true, otherwise randomly sampled using a Pareto distribution +#[allow(dead_code)] +pub(crate) fn make_data( + partition_cnt: i32, + sample_cnt: i32, + asc: bool, +) -> Result<(Arc, Vec>), DataFusionError> { + // constants observed from trace data + let simultaneous_group_cnt = 2000; + let fitted_shape = 12f64; + let fitted_scale = 5f64; + let mean = 0.1; + let stddev = 1.1; + let pareto = Pareto::new(fitted_scale, fitted_shape).unwrap(); + let normal = Normal::new(mean, stddev).unwrap(); + let mut rng = rand::rngs::SmallRng::from_seed([0; 32]); + + // populate data + let schema = test_schema(); + let mut partitions = vec![]; + let mut cur_time = 16909000000000i64; + for _ in 0..partition_cnt { + let mut id_builder = StringBuilder::new(); + let mut ts_builder = Int64Builder::new(); + let gen_id = |rng: &mut rand::rngs::SmallRng| { + rng.gen::<[u8; 16]>() + .iter() + .fold(String::new(), |mut output, b| { + let _ = write!(output, "{b:02X}"); + output + }) + }; + let gen_sample_cnt = + |mut rng: &mut rand::rngs::SmallRng| pareto.sample(&mut rng).ceil() as u32; + let mut group_ids = (0..simultaneous_group_cnt) + .map(|_| gen_id(&mut rng)) + .collect::>(); + let mut group_sample_cnts = (0..simultaneous_group_cnt) + .map(|_| gen_sample_cnt(&mut rng)) + .collect::>(); + for _ in 0..sample_cnt { + let random_index = rng.gen_range(0..simultaneous_group_cnt); + let trace_id = &mut group_ids[random_index]; + let sample_cnt = &mut group_sample_cnts[random_index]; + *sample_cnt -= 1; + if *sample_cnt == 0 { + *trace_id = gen_id(&mut rng); + *sample_cnt = gen_sample_cnt(&mut rng); + } + + id_builder.append_value(trace_id); + ts_builder.append_value(cur_time); + + if asc { + cur_time += 1; + } else { + let samp: f64 = normal.sample(&mut rng); + let samp = samp.round(); + cur_time += samp as i64; + } + } + + // convert to MemTable + let id_col = Arc::new(id_builder.finish()); + let ts_col = Arc::new(ts_builder.finish()); + let batch = RecordBatch::try_new(schema.clone(), vec![id_col, ts_col])?; + partitions.push(vec![batch]); + } + Ok((schema, partitions)) +} + +/// The Schema used by make_data +fn test_schema() -> SchemaRef { + Arc::new(Schema::new(vec![ + Field::new("trace_id", DataType::Utf8, false), + Field::new("timestamp_ms", DataType::Int64, false), + ])) +} diff --git a/datafusion/core/benches/distinct_query_sql.rs b/datafusion/core/benches/distinct_query_sql.rs new file mode 100644 index 000000000000..c242798a56f0 --- /dev/null +++ b/datafusion/core/benches/distinct_query_sql.rs @@ -0,0 +1,208 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +#[macro_use] +extern crate criterion; +extern crate arrow; +extern crate datafusion; + +mod data_utils; +use crate::criterion::Criterion; +use data_utils::{create_table_provider, make_data}; +use datafusion::execution::context::SessionContext; +use datafusion::physical_plan::{collect, ExecutionPlan}; +use datafusion::{datasource::MemTable, error::Result}; +use datafusion_execution::config::SessionConfig; +use datafusion_execution::TaskContext; + +use parking_lot::Mutex; +use std::{sync::Arc, time::Duration}; +use tokio::runtime::Runtime; + +fn query(ctx: Arc>, sql: &str) { + let rt = Runtime::new().unwrap(); + let df = rt.block_on(ctx.lock().sql(sql)).unwrap(); + criterion::black_box(rt.block_on(df.collect()).unwrap()); +} + +fn create_context( + partitions_len: usize, + array_len: usize, + batch_size: usize, +) -> Result>> { + let ctx = SessionContext::new(); + let provider = create_table_provider(partitions_len, array_len, batch_size)?; + ctx.register_table("t", provider)?; + Ok(Arc::new(Mutex::new(ctx))) +} + +fn criterion_benchmark_limited_distinct(c: &mut Criterion) { + let partitions_len = 10; + let array_len = 1 << 26; // 64 M + let batch_size = 8192; + let ctx = create_context(partitions_len, array_len, batch_size).unwrap(); + + let mut group = c.benchmark_group("custom-measurement-time"); + group.measurement_time(Duration::from_secs(40)); + + group.bench_function("distinct_group_by_u64_narrow_limit_10", |b| { + b.iter(|| { + query( + ctx.clone(), + "SELECT DISTINCT u64_narrow FROM t GROUP BY u64_narrow LIMIT 10", + ) + }) + }); + + group.bench_function("distinct_group_by_u64_narrow_limit_100", |b| { + b.iter(|| { + query( + ctx.clone(), + "SELECT DISTINCT u64_narrow FROM t GROUP BY u64_narrow LIMIT 100", + ) + }) + }); + + group.bench_function("distinct_group_by_u64_narrow_limit_1000", |b| { + b.iter(|| { + query( + ctx.clone(), + "SELECT DISTINCT u64_narrow FROM t GROUP BY u64_narrow LIMIT 1000", + ) + }) + }); + + group.bench_function("distinct_group_by_u64_narrow_limit_10000", |b| { + b.iter(|| { + query( + ctx.clone(), + "SELECT DISTINCT u64_narrow FROM t GROUP BY u64_narrow LIMIT 10000", + ) + }) + }); + + group.bench_function("group_by_multiple_columns_limit_10", |b| { + b.iter(|| { + query( + ctx.clone(), + "SELECT u64_narrow, u64_wide, utf8, f64 FROM t GROUP BY 1, 2, 3, 4 LIMIT 10", + ) + }) + }); + group.finish(); +} + +async fn distinct_with_limit( + plan: Arc, + ctx: Arc, +) -> Result<()> { + let batches = collect(plan, ctx).await?; + assert_eq!(batches.len(), 1); + let batch = batches.first().unwrap(); + assert_eq!(batch.num_rows(), 10); + + Ok(()) +} + +fn run(plan: Arc, ctx: Arc) { + let rt = Runtime::new().unwrap(); + criterion::black_box( + rt.block_on(async { distinct_with_limit(plan.clone(), ctx.clone()).await }), + ) + .unwrap(); +} + +pub async fn create_context_sampled_data( + sql: &str, + partition_cnt: i32, + sample_cnt: i32, +) -> Result<(Arc, Arc)> { + let (schema, parts) = make_data(partition_cnt, sample_cnt, false /* asc */).unwrap(); + let mem_table = Arc::new(MemTable::try_new(schema, parts).unwrap()); + + // Create the DataFrame + let cfg = SessionConfig::new(); + let ctx = SessionContext::new_with_config(cfg); + let _ = ctx.register_table("traces", mem_table)?; + let df = ctx.sql(sql).await?; + let physical_plan = df.create_physical_plan().await?; + Ok((physical_plan, ctx.task_ctx())) +} + +fn criterion_benchmark_limited_distinct_sampled(c: &mut Criterion) { + let rt = Runtime::new().unwrap(); + + let limit = 10; + let partitions = 100; + let samples = 100_000; + let sql = + format!("select DISTINCT trace_id from traces group by trace_id limit {limit};"); + + let distinct_trace_id_100_partitions_100_000_samples_limit_100 = rt.block_on(async { + create_context_sampled_data(sql.as_str(), partitions, samples) + .await + .unwrap() + }); + + c.bench_function( + format!("distinct query with {} partitions and {} samples per partition with limit {}", partitions, samples, limit).as_str(), + |b| b.iter(|| run(distinct_trace_id_100_partitions_100_000_samples_limit_100.0.clone(), + distinct_trace_id_100_partitions_100_000_samples_limit_100.1.clone())), + ); + + let partitions = 10; + let samples = 1_000_000; + let sql = + format!("select DISTINCT trace_id from traces group by trace_id limit {limit};"); + + let distinct_trace_id_10_partitions_1_000_000_samples_limit_10 = rt.block_on(async { + create_context_sampled_data(sql.as_str(), partitions, samples) + .await + .unwrap() + }); + + c.bench_function( + format!("distinct query with {} partitions and {} samples per partition with limit {}", partitions, samples, limit).as_str(), + |b| b.iter(|| run(distinct_trace_id_10_partitions_1_000_000_samples_limit_10.0.clone(), + distinct_trace_id_10_partitions_1_000_000_samples_limit_10.1.clone())), + ); + + let partitions = 1; + let samples = 10_000_000; + let sql = + format!("select DISTINCT trace_id from traces group by trace_id limit {limit};"); + + let rt = Runtime::new().unwrap(); + let distinct_trace_id_1_partition_10_000_000_samples_limit_10 = rt.block_on(async { + create_context_sampled_data(sql.as_str(), partitions, samples) + .await + .unwrap() + }); + + c.bench_function( + format!("distinct query with {} partitions and {} samples per partition with limit {}", partitions, samples, limit).as_str(), + |b| b.iter(|| run(distinct_trace_id_1_partition_10_000_000_samples_limit_10.0.clone(), + distinct_trace_id_1_partition_10_000_000_samples_limit_10.1.clone())), + ); +} + +criterion_group!( + benches, + criterion_benchmark_limited_distinct, + criterion_benchmark_limited_distinct_sampled +); +criterion_main!(benches); diff --git a/datafusion/core/benches/topk_aggregate.rs b/datafusion/core/benches/topk_aggregate.rs index ef84d6e3cac8..922cbd2b4229 100644 --- a/datafusion/core/benches/topk_aggregate.rs +++ b/datafusion/core/benches/topk_aggregate.rs @@ -15,20 +15,15 @@ // specific language governing permissions and limitations // under the License. +mod data_utils; use arrow::util::pretty::pretty_format_batches; -use arrow::{datatypes::Schema, record_batch::RecordBatch}; -use arrow_array::builder::{Int64Builder, StringBuilder}; -use arrow_schema::{DataType, Field, SchemaRef}; use criterion::{criterion_group, criterion_main, Criterion}; +use data_utils::make_data; use datafusion::physical_plan::{collect, displayable, ExecutionPlan}; use datafusion::prelude::SessionContext; use datafusion::{datasource::MemTable, error::Result}; -use datafusion_common::DataFusionError; use datafusion_execution::config::SessionConfig; use datafusion_execution::TaskContext; -use rand_distr::Distribution; -use rand_distr::{Normal, Pareto}; -use std::fmt::Write; use std::sync::Arc; use tokio::runtime::Runtime; @@ -78,10 +73,10 @@ async fn aggregate( let batch = batches.first().unwrap(); assert_eq!(batch.num_rows(), 10); - let actual = format!("{}", pretty_format_batches(&batches)?); + let actual = format!("{}", pretty_format_batches(&batches)?).to_lowercase(); let expected_asc = r#" +----------------------------------+--------------------------+ -| trace_id | MAX(traces.timestamp_ms) | +| trace_id | max(traces.timestamp_ms) | +----------------------------------+--------------------------+ | 5868861a23ed31355efc5200eb80fe74 | 16909009999999 | | 4040e64656804c3d77320d7a0e7eb1f0 | 16909009999998 | @@ -103,85 +98,6 @@ async fn aggregate( Ok(()) } -fn make_data( - partition_cnt: i32, - sample_cnt: i32, - asc: bool, -) -> Result<(Arc, Vec>), DataFusionError> { - use rand::Rng; - use rand::SeedableRng; - - // constants observed from trace data - let simultaneous_group_cnt = 2000; - let fitted_shape = 12f64; - let fitted_scale = 5f64; - let mean = 0.1; - let stddev = 1.1; - let pareto = Pareto::new(fitted_scale, fitted_shape).unwrap(); - let normal = Normal::new(mean, stddev).unwrap(); - let mut rng = rand::rngs::SmallRng::from_seed([0; 32]); - - // populate data - let schema = test_schema(); - let mut partitions = vec![]; - let mut cur_time = 16909000000000i64; - for _ in 0..partition_cnt { - let mut id_builder = StringBuilder::new(); - let mut ts_builder = Int64Builder::new(); - let gen_id = |rng: &mut rand::rngs::SmallRng| { - rng.gen::<[u8; 16]>() - .iter() - .fold(String::new(), |mut output, b| { - let _ = write!(output, "{b:02X}"); - output - }) - }; - let gen_sample_cnt = - |mut rng: &mut rand::rngs::SmallRng| pareto.sample(&mut rng).ceil() as u32; - let mut group_ids = (0..simultaneous_group_cnt) - .map(|_| gen_id(&mut rng)) - .collect::>(); - let mut group_sample_cnts = (0..simultaneous_group_cnt) - .map(|_| gen_sample_cnt(&mut rng)) - .collect::>(); - for _ in 0..sample_cnt { - let random_index = rng.gen_range(0..simultaneous_group_cnt); - let trace_id = &mut group_ids[random_index]; - let sample_cnt = &mut group_sample_cnts[random_index]; - *sample_cnt -= 1; - if *sample_cnt == 0 { - *trace_id = gen_id(&mut rng); - *sample_cnt = gen_sample_cnt(&mut rng); - } - - id_builder.append_value(trace_id); - ts_builder.append_value(cur_time); - - if asc { - cur_time += 1; - } else { - let samp: f64 = normal.sample(&mut rng); - let samp = samp.round(); - cur_time += samp as i64; - } - } - - // convert to MemTable - let id_col = Arc::new(id_builder.finish()); - let ts_col = Arc::new(ts_builder.finish()); - let batch = RecordBatch::try_new(schema.clone(), vec![id_col, ts_col])?; - partitions.push(vec![batch]); - } - Ok((schema, partitions)) -} - -fn test_schema() -> SchemaRef { - Arc::new(Schema::new(vec![ - Field::new("trace_id", DataType::Utf8, false), - Field::new("timestamp_ms", DataType::Int64, false), - ])) -} - fn criterion_benchmark(c: &mut Criterion) { let limit = 10; let partitions = 10; diff --git a/datafusion/core/src/physical_optimizer/aggregate_statistics.rs b/datafusion/core/src/physical_optimizer/aggregate_statistics.rs index 43def5d73f73..4265e3ff80d0 100644 --- a/datafusion/core/src/physical_optimizer/aggregate_statistics.rs +++ b/datafusion/core/src/physical_optimizer/aggregate_statistics.rs @@ -241,7 +241,7 @@ fn take_optimizable_max( } #[cfg(test)] -mod tests { +pub(crate) mod tests { use std::sync::Arc; use super::*; @@ -334,7 +334,7 @@ mod tests { } /// Describe the type of aggregate being tested - enum TestAggregate { + pub(crate) enum TestAggregate { /// Testing COUNT(*) type aggregates CountStar, @@ -343,7 +343,7 @@ mod tests { } impl TestAggregate { - fn new_count_star() -> Self { + pub(crate) fn new_count_star() -> Self { Self::CountStar } @@ -352,7 +352,7 @@ mod tests { } /// Return appropriate expr depending if COUNT is for col or table (*) - fn count_expr(&self) -> Arc { + pub(crate) fn count_expr(&self) -> Arc { Arc::new(Count::new( self.column(), self.column_name(), diff --git a/datafusion/core/src/physical_optimizer/combine_partial_final_agg.rs b/datafusion/core/src/physical_optimizer/combine_partial_final_agg.rs index 2c4e929788df..0948445de20d 100644 --- a/datafusion/core/src/physical_optimizer/combine_partial_final_agg.rs +++ b/datafusion/core/src/physical_optimizer/combine_partial_final_agg.rs @@ -95,6 +95,9 @@ impl PhysicalOptimizerRule for CombinePartialFinalAggregate { input_agg_exec.input().clone(), input_agg_exec.input_schema(), ) + .map(|combined_agg| { + combined_agg.with_limit(agg_exec.limit()) + }) .ok() .map(Arc::new) } else { @@ -428,4 +431,49 @@ mod tests { assert_optimized!(expected, plan); Ok(()) } + + #[test] + fn aggregations_with_limit_combined() -> Result<()> { + let schema = schema(); + let aggr_expr = vec![]; + + let groups: Vec<(Arc, String)> = + vec![(col("c", &schema)?, "c".to_string())]; + + let partial_group_by = PhysicalGroupBy::new_single(groups); + let partial_agg = partial_aggregate_exec( + parquet_exec(&schema), + partial_group_by, + aggr_expr.clone(), + ); + + let groups: Vec<(Arc, String)> = + vec![(col("c", &partial_agg.schema())?, "c".to_string())]; + let final_group_by = PhysicalGroupBy::new_single(groups); + + let schema = partial_agg.schema(); + let final_agg = Arc::new( + AggregateExec::try_new( + AggregateMode::Final, + final_group_by, + aggr_expr, + vec![], + vec![], + partial_agg, + schema, + ) + .unwrap() + .with_limit(Some(5)), + ); + let plan: Arc = final_agg; + // should combine the Partial/Final AggregateExecs to a Single AggregateExec + // with the final limit preserved + let expected = &[ + "AggregateExec: mode=Single, gby=[c@2 as c], aggr=[], lim=[5]", + "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c]", + ]; + + assert_optimized!(expected, plan); + Ok(()) + } } diff --git a/datafusion/core/src/physical_optimizer/enforce_distribution.rs b/datafusion/core/src/physical_optimizer/enforce_distribution.rs index ee6e11bd271a..c562d7853f1c 100644 --- a/datafusion/core/src/physical_optimizer/enforce_distribution.rs +++ b/datafusion/core/src/physical_optimizer/enforce_distribution.rs @@ -1614,7 +1614,7 @@ impl TreeNode for PlanWithKeyRequirements { /// Since almost all of these tests explicitly use `ParquetExec` they only run with the parquet feature flag on #[cfg(feature = "parquet")] #[cfg(test)] -mod tests { +pub(crate) mod tests { use std::ops::Deref; use super::*; @@ -1751,7 +1751,7 @@ mod tests { } } - fn schema() -> SchemaRef { + pub(crate) fn schema() -> SchemaRef { Arc::new(Schema::new(vec![ Field::new("a", DataType::Int64, true), Field::new("b", DataType::Int64, true), @@ -1765,7 +1765,7 @@ mod tests { parquet_exec_with_sort(vec![]) } - fn parquet_exec_with_sort( + pub(crate) fn parquet_exec_with_sort( output_ordering: Vec>, ) -> Arc { Arc::new(ParquetExec::new( @@ -2018,7 +2018,7 @@ mod tests { Arc::new(SortRequiredExec::new_with_requirement(input, sort_exprs)) } - fn trim_plan_display(plan: &str) -> Vec<&str> { + pub(crate) fn trim_plan_display(plan: &str) -> Vec<&str> { plan.split('\n') .map(|s| s.trim()) .filter(|s| !s.is_empty()) diff --git a/datafusion/core/src/physical_optimizer/limited_distinct_aggregation.rs b/datafusion/core/src/physical_optimizer/limited_distinct_aggregation.rs new file mode 100644 index 000000000000..832a92bb69c6 --- /dev/null +++ b/datafusion/core/src/physical_optimizer/limited_distinct_aggregation.rs @@ -0,0 +1,626 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +//! A special-case optimizer rule that pushes limit into a grouped aggregation +//! which has no aggregate expressions or sorting requirements + +use crate::physical_optimizer::PhysicalOptimizerRule; +use crate::physical_plan::aggregates::AggregateExec; +use crate::physical_plan::limit::{GlobalLimitExec, LocalLimitExec}; +use crate::physical_plan::ExecutionPlan; +use datafusion_common::config::ConfigOptions; +use datafusion_common::tree_node::{Transformed, TreeNode}; +use datafusion_common::Result; +use itertools::Itertools; +use std::sync::Arc; + +/// An optimizer rule that passes a `limit` hint into grouped aggregations which don't require all +/// rows in the group to be processed for correctness. Example queries fitting this description are: +/// `SELECT distinct l_orderkey FROM lineitem LIMIT 10;` +/// `SELECT l_orderkey FROM lineitem GROUP BY l_orderkey LIMIT 10;` +pub struct LimitedDistinctAggregation {} + +impl LimitedDistinctAggregation { + /// Create a new `LimitedDistinctAggregation` + pub fn new() -> Self { + Self {} + } + + fn transform_agg( + aggr: &AggregateExec, + limit: usize, + ) -> Option> { + // rules for transforming this Aggregate are held in this method + if !aggr.is_unordered_unfiltered_group_by_distinct() { + return None; + } + + // We found what we want: clone, copy the limit down, and return modified node + let new_aggr = AggregateExec::try_new( + *aggr.mode(), + aggr.group_by().clone(), + aggr.aggr_expr().to_vec(), + aggr.filter_expr().to_vec(), + aggr.order_by_expr().to_vec(), + aggr.input().clone(), + aggr.input_schema().clone(), + ) + .expect("Unable to copy Aggregate!") + .with_limit(Some(limit)); + Some(Arc::new(new_aggr)) + } + + /// transform_limit matches an `AggregateExec` as the child of a `LocalLimitExec` + /// or `GlobalLimitExec` and pushes the limit into the aggregation as a soft limit when + /// there is a group by, but no sorting, no aggregate expressions, and no filters in the + /// aggregation + fn transform_limit(plan: Arc) -> Option> { + let limit: usize; + let mut global_fetch: Option = None; + let mut global_skip: usize = 0; + let children: Vec>; + let mut is_global_limit = false; + if let Some(local_limit) = plan.as_any().downcast_ref::() { + limit = local_limit.fetch(); + children = local_limit.children(); + } else if let Some(global_limit) = plan.as_any().downcast_ref::() + { + global_fetch = global_limit.fetch(); + global_fetch?; + global_skip = global_limit.skip(); + // the aggregate must read at least fetch+skip number of rows + limit = global_fetch.unwrap() + global_skip; + children = global_limit.children(); + is_global_limit = true + } else { + return None; + } + let child = children.iter().exactly_one().ok()?; + // ensure there is no output ordering; can this rule be relaxed? + if plan.output_ordering().is_some() { + return None; + } + // ensure no ordering is required on the input + if plan.required_input_ordering()[0].is_some() { + return None; + } + + // if found_match_aggr is true, match_aggr holds a parent aggregation whose group_by + // must match that of a child aggregation in order to rewrite the child aggregation + let mut match_aggr: Arc = plan; + let mut found_match_aggr = false; + + let mut rewrite_applicable = true; + let mut closure = |plan: Arc| { + if !rewrite_applicable { + return Ok(Transformed::No(plan)); + } + if let Some(aggr) = plan.as_any().downcast_ref::() { + if found_match_aggr { + if let Some(parent_aggr) = + match_aggr.as_any().downcast_ref::() + { + if !parent_aggr.group_by().eq(aggr.group_by()) { + // a partial and final aggregation with different groupings disqualifies + // rewriting the child aggregation + rewrite_applicable = false; + return Ok(Transformed::No(plan)); + } + } + } + // either we run into an Aggregate and transform it, or disable the rewrite + // for subsequent children + match Self::transform_agg(aggr, limit) { + None => {} + Some(new_aggr) => { + match_aggr = plan; + found_match_aggr = true; + return Ok(Transformed::Yes(new_aggr)); + } + } + } + rewrite_applicable = false; + Ok(Transformed::No(plan)) + }; + let child = child.clone().transform_down_mut(&mut closure).ok()?; + if is_global_limit { + return Some(Arc::new(GlobalLimitExec::new( + child, + global_skip, + global_fetch, + ))); + } + Some(Arc::new(LocalLimitExec::new(child, limit))) + } +} + +impl Default for LimitedDistinctAggregation { + fn default() -> Self { + Self::new() + } +} + +impl PhysicalOptimizerRule for LimitedDistinctAggregation { + fn optimize( + &self, + plan: Arc, + config: &ConfigOptions, + ) -> Result> { + let plan = if config.optimizer.enable_distinct_aggregation_soft_limit { + plan.transform_down(&|plan| { + Ok( + if let Some(plan) = + LimitedDistinctAggregation::transform_limit(plan.clone()) + { + Transformed::Yes(plan) + } else { + Transformed::No(plan) + }, + ) + })? + } else { + plan + }; + Ok(plan) + } + + fn name(&self) -> &str { + "LimitedDistinctAggregation" + } + + fn schema_check(&self) -> bool { + true + } +} + +#[cfg(test)] +mod tests { + use super::*; + use crate::error::Result; + use crate::physical_optimizer::aggregate_statistics::tests::TestAggregate; + use crate::physical_optimizer::enforce_distribution::tests::{ + parquet_exec_with_sort, schema, trim_plan_display, + }; + use crate::physical_plan::aggregates::{AggregateExec, PhysicalGroupBy}; + use crate::physical_plan::collect; + use crate::physical_plan::memory::MemoryExec; + use crate::prelude::SessionContext; + use arrow::array::Int32Array; + use arrow::compute::SortOptions; + use arrow::datatypes::{DataType, Field, Schema}; + use arrow::record_batch::RecordBatch; + use arrow::util::pretty::pretty_format_batches; + use arrow_schema::SchemaRef; + use datafusion_execution::config::SessionConfig; + use datafusion_expr::Operator; + use datafusion_physical_expr::expressions::cast; + use datafusion_physical_expr::expressions::col; + use datafusion_physical_expr::PhysicalSortExpr; + use datafusion_physical_expr::{expressions, PhysicalExpr}; + use datafusion_physical_plan::aggregates::AggregateMode; + use datafusion_physical_plan::displayable; + use std::sync::Arc; + + fn mock_data() -> Result> { + let schema = Arc::new(Schema::new(vec![ + Field::new("a", DataType::Int32, true), + Field::new("b", DataType::Int32, true), + ])); + + let batch = RecordBatch::try_new( + Arc::clone(&schema), + vec![ + Arc::new(Int32Array::from(vec![ + Some(1), + Some(2), + None, + Some(1), + Some(4), + Some(5), + ])), + Arc::new(Int32Array::from(vec![ + Some(1), + None, + Some(6), + Some(2), + Some(8), + Some(9), + ])), + ], + )?; + + Ok(Arc::new(MemoryExec::try_new( + &[vec![batch]], + Arc::clone(&schema), + None, + )?)) + } + + fn assert_plan_matches_expected( + plan: &Arc, + expected: &[&str], + ) -> Result<()> { + let expected_lines: Vec<&str> = expected.to_vec(); + let session_ctx = SessionContext::new(); + let state = session_ctx.state(); + + let optimized = LimitedDistinctAggregation::new() + .optimize(Arc::clone(plan), state.config_options())?; + + let optimized_result = displayable(optimized.as_ref()).indent(true).to_string(); + let actual_lines = trim_plan_display(&optimized_result); + + assert_eq!( + &expected_lines, &actual_lines, + "\n\nexpected:\n\n{:#?}\nactual:\n\n{:#?}\n\n", + expected_lines, actual_lines + ); + + Ok(()) + } + + async fn assert_results_match_expected( + plan: Arc, + expected: &str, + ) -> Result<()> { + let cfg = SessionConfig::new().with_target_partitions(1); + let ctx = SessionContext::new_with_config(cfg); + let batches = collect(plan, ctx.task_ctx()).await?; + let actual = format!("{}", pretty_format_batches(&batches)?); + assert_eq!(actual, expected); + Ok(()) + } + + pub fn build_group_by( + input_schema: &SchemaRef, + columns: Vec, + ) -> PhysicalGroupBy { + let mut group_by_expr: Vec<(Arc, String)> = vec![]; + for column in columns.iter() { + group_by_expr.push((col(column, input_schema).unwrap(), column.to_string())); + } + PhysicalGroupBy::new_single(group_by_expr.clone()) + } + + #[tokio::test] + async fn test_partial_final() -> Result<()> { + let source = mock_data()?; + let schema = source.schema(); + + // `SELECT a FROM MemoryExec GROUP BY a LIMIT 4;`, Partial/Final AggregateExec + let partial_agg = AggregateExec::try_new( + AggregateMode::Partial, + build_group_by(&schema.clone(), vec!["a".to_string()]), + vec![], /* aggr_expr */ + vec![None], /* filter_expr */ + vec![None], /* order_by_expr */ + source, /* input */ + schema.clone(), /* input_schema */ + )?; + let final_agg = AggregateExec::try_new( + AggregateMode::Final, + build_group_by(&schema.clone(), vec!["a".to_string()]), + vec![], /* aggr_expr */ + vec![None], /* filter_expr */ + vec![None], /* order_by_expr */ + Arc::new(partial_agg), /* input */ + schema.clone(), /* input_schema */ + )?; + let limit_exec = LocalLimitExec::new( + Arc::new(final_agg), + 4, // fetch + ); + // expected to push the limit to the Partial and Final AggregateExecs + let expected = [ + "LocalLimitExec: fetch=4", + "AggregateExec: mode=Final, gby=[a@0 as a], aggr=[], lim=[4]", + "AggregateExec: mode=Partial, gby=[a@0 as a], aggr=[], lim=[4]", + "MemoryExec: partitions=1, partition_sizes=[1]", + ]; + let plan: Arc = Arc::new(limit_exec); + assert_plan_matches_expected(&plan, &expected)?; + let expected = r#" ++---+ +| a | ++---+ +| 1 | +| 2 | +| | +| 4 | ++---+ +"# + .trim(); + assert_results_match_expected(plan, expected).await?; + Ok(()) + } + + #[tokio::test] + async fn test_single_local() -> Result<()> { + let source = mock_data()?; + let schema = source.schema(); + + // `SELECT a FROM MemoryExec GROUP BY a LIMIT 4;`, Single AggregateExec + let single_agg = AggregateExec::try_new( + AggregateMode::Single, + build_group_by(&schema.clone(), vec!["a".to_string()]), + vec![], /* aggr_expr */ + vec![None], /* filter_expr */ + vec![None], /* order_by_expr */ + source, /* input */ + schema.clone(), /* input_schema */ + )?; + let limit_exec = LocalLimitExec::new( + Arc::new(single_agg), + 4, // fetch + ); + // expected to push the limit to the AggregateExec + let expected = [ + "LocalLimitExec: fetch=4", + "AggregateExec: mode=Single, gby=[a@0 as a], aggr=[], lim=[4]", + "MemoryExec: partitions=1, partition_sizes=[1]", + ]; + let plan: Arc = Arc::new(limit_exec); + assert_plan_matches_expected(&plan, &expected)?; + let expected = r#" ++---+ +| a | ++---+ +| 1 | +| 2 | +| | +| 4 | ++---+ +"# + .trim(); + assert_results_match_expected(plan, expected).await?; + Ok(()) + } + + #[tokio::test] + async fn test_single_global() -> Result<()> { + let source = mock_data()?; + let schema = source.schema(); + + // `SELECT a FROM MemoryExec GROUP BY a LIMIT 4;`, Single AggregateExec + let single_agg = AggregateExec::try_new( + AggregateMode::Single, + build_group_by(&schema.clone(), vec!["a".to_string()]), + vec![], /* aggr_expr */ + vec![None], /* filter_expr */ + vec![None], /* order_by_expr */ + source, /* input */ + schema.clone(), /* input_schema */ + )?; + let limit_exec = GlobalLimitExec::new( + Arc::new(single_agg), + 1, // skip + Some(3), // fetch + ); + // expected to push the skip+fetch limit to the AggregateExec + let expected = [ + "GlobalLimitExec: skip=1, fetch=3", + "AggregateExec: mode=Single, gby=[a@0 as a], aggr=[], lim=[4]", + "MemoryExec: partitions=1, partition_sizes=[1]", + ]; + let plan: Arc = Arc::new(limit_exec); + assert_plan_matches_expected(&plan, &expected)?; + let expected = r#" ++---+ +| a | ++---+ +| 2 | +| | +| 4 | ++---+ +"# + .trim(); + assert_results_match_expected(plan, expected).await?; + Ok(()) + } + + #[tokio::test] + async fn test_distinct_cols_different_than_group_by_cols() -> Result<()> { + let source = mock_data()?; + let schema = source.schema(); + + // `SELECT distinct a FROM MemoryExec GROUP BY a, b LIMIT 4;`, Single/Single AggregateExec + let group_by_agg = AggregateExec::try_new( + AggregateMode::Single, + build_group_by(&schema.clone(), vec!["a".to_string(), "b".to_string()]), + vec![], /* aggr_expr */ + vec![None], /* filter_expr */ + vec![None], /* order_by_expr */ + source, /* input */ + schema.clone(), /* input_schema */ + )?; + let distinct_agg = AggregateExec::try_new( + AggregateMode::Single, + build_group_by(&schema.clone(), vec!["a".to_string()]), + vec![], /* aggr_expr */ + vec![None], /* filter_expr */ + vec![None], /* order_by_expr */ + Arc::new(group_by_agg), /* input */ + schema.clone(), /* input_schema */ + )?; + let limit_exec = LocalLimitExec::new( + Arc::new(distinct_agg), + 4, // fetch + ); + // expected to push the limit to the outer AggregateExec only + let expected = [ + "LocalLimitExec: fetch=4", + "AggregateExec: mode=Single, gby=[a@0 as a], aggr=[], lim=[4]", + "AggregateExec: mode=Single, gby=[a@0 as a, b@1 as b], aggr=[]", + "MemoryExec: partitions=1, partition_sizes=[1]", + ]; + let plan: Arc = Arc::new(limit_exec); + assert_plan_matches_expected(&plan, &expected)?; + let expected = r#" ++---+ +| a | ++---+ +| 1 | +| 2 | +| | +| 4 | ++---+ +"# + .trim(); + assert_results_match_expected(plan, expected).await?; + Ok(()) + } + + #[test] + fn test_no_group_by() -> Result<()> { + let source = mock_data()?; + let schema = source.schema(); + + // `SELECT FROM MemoryExec LIMIT 10;`, Single AggregateExec + let single_agg = AggregateExec::try_new( + AggregateMode::Single, + build_group_by(&schema.clone(), vec![]), + vec![], /* aggr_expr */ + vec![None], /* filter_expr */ + vec![None], /* order_by_expr */ + source, /* input */ + schema.clone(), /* input_schema */ + )?; + let limit_exec = LocalLimitExec::new( + Arc::new(single_agg), + 10, // fetch + ); + // expected not to push the limit to the AggregateExec + let expected = [ + "LocalLimitExec: fetch=10", + "AggregateExec: mode=Single, gby=[], aggr=[]", + "MemoryExec: partitions=1, partition_sizes=[1]", + ]; + let plan: Arc = Arc::new(limit_exec); + assert_plan_matches_expected(&plan, &expected)?; + Ok(()) + } + + #[test] + fn test_has_aggregate_expression() -> Result<()> { + let source = mock_data()?; + let schema = source.schema(); + let agg = TestAggregate::new_count_star(); + + // `SELECT FROM MemoryExec LIMIT 10;`, Single AggregateExec + let single_agg = AggregateExec::try_new( + AggregateMode::Single, + build_group_by(&schema.clone(), vec!["a".to_string()]), + vec![agg.count_expr()], /* aggr_expr */ + vec![None], /* filter_expr */ + vec![None], /* order_by_expr */ + source, /* input */ + schema.clone(), /* input_schema */ + )?; + let limit_exec = LocalLimitExec::new( + Arc::new(single_agg), + 10, // fetch + ); + // expected not to push the limit to the AggregateExec + let expected = [ + "LocalLimitExec: fetch=10", + "AggregateExec: mode=Single, gby=[a@0 as a], aggr=[COUNT(*)]", + "MemoryExec: partitions=1, partition_sizes=[1]", + ]; + let plan: Arc = Arc::new(limit_exec); + assert_plan_matches_expected(&plan, &expected)?; + Ok(()) + } + + #[test] + fn test_has_filter() -> Result<()> { + let source = mock_data()?; + let schema = source.schema(); + + // `SELECT a FROM MemoryExec WHERE a > 1 GROUP BY a LIMIT 10;`, Single AggregateExec + // the `a > 1` filter is applied in the AggregateExec + let filter_expr = Some(expressions::binary( + expressions::col("a", &schema)?, + Operator::Gt, + cast(expressions::lit(1u32), &schema, DataType::Int32)?, + &schema, + )?); + let single_agg = AggregateExec::try_new( + AggregateMode::Single, + build_group_by(&schema.clone(), vec!["a".to_string()]), + vec![], /* aggr_expr */ + vec![filter_expr], /* filter_expr */ + vec![None], /* order_by_expr */ + source, /* input */ + schema.clone(), /* input_schema */ + )?; + let limit_exec = LocalLimitExec::new( + Arc::new(single_agg), + 10, // fetch + ); + // expected not to push the limit to the AggregateExec + // TODO(msirek): open an issue for `filter_expr` of `AggregateExec` not printing out + let expected = [ + "LocalLimitExec: fetch=10", + "AggregateExec: mode=Single, gby=[a@0 as a], aggr=[]", + "MemoryExec: partitions=1, partition_sizes=[1]", + ]; + let plan: Arc = Arc::new(limit_exec); + assert_plan_matches_expected(&plan, &expected)?; + Ok(()) + } + + #[test] + fn test_has_order_by() -> Result<()> { + let sort_key = vec![PhysicalSortExpr { + expr: expressions::col("a", &schema()).unwrap(), + options: SortOptions::default(), + }]; + let source = parquet_exec_with_sort(vec![sort_key]); + let schema = source.schema(); + + // `SELECT a FROM MemoryExec GROUP BY a ORDER BY a LIMIT 10;`, Single AggregateExec + let order_by_expr = Some(vec![PhysicalSortExpr { + expr: expressions::col("a", &schema.clone()).unwrap(), + options: SortOptions::default(), + }]); + + // `SELECT a FROM MemoryExec WHERE a > 1 GROUP BY a LIMIT 10;`, Single AggregateExec + // the `a > 1` filter is applied in the AggregateExec + let single_agg = AggregateExec::try_new( + AggregateMode::Single, + build_group_by(&schema.clone(), vec!["a".to_string()]), + vec![], /* aggr_expr */ + vec![None], /* filter_expr */ + vec![order_by_expr], /* order_by_expr */ + source, /* input */ + schema.clone(), /* input_schema */ + )?; + let limit_exec = LocalLimitExec::new( + Arc::new(single_agg), + 10, // fetch + ); + // expected not to push the limit to the AggregateExec + let expected = [ + "LocalLimitExec: fetch=10", + "AggregateExec: mode=Single, gby=[a@0 as a], aggr=[], ordering_mode=Sorted", + "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[a@0 ASC]", + ]; + let plan: Arc = Arc::new(limit_exec); + assert_plan_matches_expected(&plan, &expected)?; + Ok(()) + } +} diff --git a/datafusion/core/src/physical_optimizer/mod.rs b/datafusion/core/src/physical_optimizer/mod.rs index d2a0c6fefd8f..e990fead610d 100644 --- a/datafusion/core/src/physical_optimizer/mod.rs +++ b/datafusion/core/src/physical_optimizer/mod.rs @@ -27,6 +27,7 @@ pub mod combine_partial_final_agg; pub mod enforce_distribution; pub mod enforce_sorting; pub mod join_selection; +pub mod limited_distinct_aggregation; pub mod optimizer; pub mod output_requirements; pub mod pipeline_checker; diff --git a/datafusion/core/src/physical_optimizer/optimizer.rs b/datafusion/core/src/physical_optimizer/optimizer.rs index 20a59b58ea50..f8c82576e254 100644 --- a/datafusion/core/src/physical_optimizer/optimizer.rs +++ b/datafusion/core/src/physical_optimizer/optimizer.rs @@ -27,6 +27,7 @@ use crate::physical_optimizer::combine_partial_final_agg::CombinePartialFinalAgg use crate::physical_optimizer::enforce_distribution::EnforceDistribution; use crate::physical_optimizer::enforce_sorting::EnforceSorting; use crate::physical_optimizer::join_selection::JoinSelection; +use crate::physical_optimizer::limited_distinct_aggregation::LimitedDistinctAggregation; use crate::physical_optimizer::output_requirements::OutputRequirements; use crate::physical_optimizer::pipeline_checker::PipelineChecker; use crate::physical_optimizer::topk_aggregation::TopKAggregation; @@ -80,6 +81,10 @@ impl PhysicalOptimizer { // repartitioning and local sorting steps to meet distribution and ordering requirements. // Therefore, it should run before EnforceDistribution and EnforceSorting. Arc::new(JoinSelection::new()), + // The LimitedDistinctAggregation rule should be applied before the EnforceDistribution rule, + // as that rule may inject other operations in between the different AggregateExecs. + // Applying the rule early means only directly-connected AggregateExecs must be examined. + Arc::new(LimitedDistinctAggregation::new()), // The EnforceDistribution rule is for adding essential repartitioning to satisfy distribution // requirements. Please make sure that the whole plan tree is determined before this rule. // This rule increases parallelism if doing so is beneficial to the physical plan; i.e. at diff --git a/datafusion/core/src/physical_optimizer/topk_aggregation.rs b/datafusion/core/src/physical_optimizer/topk_aggregation.rs index e0a8da82e35f..52d34d4f8198 100644 --- a/datafusion/core/src/physical_optimizer/topk_aggregation.rs +++ b/datafusion/core/src/physical_optimizer/topk_aggregation.rs @@ -118,7 +118,7 @@ impl TopKAggregation { } Ok(Transformed::No(plan)) }; - let child = transform_down_mut(child.clone(), &mut closure).ok()?; + let child = child.clone().transform_down_mut(&mut closure).ok()?; let sort = SortExec::new(sort.expr().to_vec(), child) .with_fetch(sort.fetch()) .with_preserve_partitioning(sort.preserve_partitioning()); @@ -126,17 +126,6 @@ impl TopKAggregation { } } -fn transform_down_mut( - me: Arc, - op: &mut F, -) -> Result> -where - F: FnMut(Arc) -> Result>>, -{ - let after_op = op(me)?.into(); - after_op.map_children(|node| transform_down_mut(node, op)) -} - impl Default for TopKAggregation { fn default() -> Self { Self::new() diff --git a/datafusion/physical-plan/src/aggregates/mod.rs b/datafusion/physical-plan/src/aggregates/mod.rs index 9cbf12aeeb88..4052d6aef0ae 100644 --- a/datafusion/physical-plan/src/aggregates/mod.rs +++ b/datafusion/physical-plan/src/aggregates/mod.rs @@ -608,6 +608,11 @@ impl AggregateExec { self.input_schema.clone() } + /// number of rows soft limit of the AggregateExec + pub fn limit(&self) -> Option { + self.limit + } + fn execute_typed( &self, partition: usize, @@ -622,9 +627,11 @@ impl AggregateExec { // grouping by an expression that has a sort/limit upstream if let Some(limit) = self.limit { - return Ok(StreamType::GroupedPriorityQueue( - GroupedTopKAggregateStream::new(self, context, partition, limit)?, - )); + if !self.is_unordered_unfiltered_group_by_distinct() { + return Ok(StreamType::GroupedPriorityQueue( + GroupedTopKAggregateStream::new(self, context, partition, limit)?, + )); + } } // grouping by something else and we need to just materialize all results @@ -648,6 +655,39 @@ impl AggregateExec { pub fn group_by(&self) -> &PhysicalGroupBy { &self.group_by } + + /// true, if this Aggregate has a group-by with no required or explicit ordering, + /// no filtering and no aggregate expressions + /// This method qualifies the use of the LimitedDistinctAggregation rewrite rule + /// on an AggregateExec. + pub fn is_unordered_unfiltered_group_by_distinct(&self) -> bool { + // ensure there is a group by + if self.group_by().is_empty() { + return false; + } + // ensure there are no aggregate expressions + if !self.aggr_expr().is_empty() { + return false; + } + // ensure there are no filters on aggregate expressions; the above check + // may preclude this case + if self.filter_expr().iter().any(|e| e.is_some()) { + return false; + } + // ensure there are no order by expressions + if self.order_by_expr().iter().any(|e| e.is_some()) { + return false; + } + // ensure there is no output ordering; can this rule be relaxed? + if self.output_ordering().is_some() { + return false; + } + // ensure no ordering is required on the input + if self.required_input_ordering()[0].is_some() { + return false; + } + true + } } impl DisplayAs for AggregateExec { diff --git a/datafusion/physical-plan/src/aggregates/row_hash.rs b/datafusion/physical-plan/src/aggregates/row_hash.rs index 7cee4a3e7cfc..f96417fc323b 100644 --- a/datafusion/physical-plan/src/aggregates/row_hash.rs +++ b/datafusion/physical-plan/src/aggregates/row_hash.rs @@ -267,6 +267,12 @@ pub(crate) struct GroupedHashAggregateStream { /// The spill state object spill_state: SpillState, + + /// Optional soft limit on the number of `group_values` in a batch + /// If the number of `group_values` in a single batch exceeds this value, + /// the `GroupedHashAggregateStream` operation immediately switches to + /// output mode and emits all groups. + group_values_soft_limit: Option, } impl GroupedHashAggregateStream { @@ -374,6 +380,7 @@ impl GroupedHashAggregateStream { input_done: false, runtime: context.runtime_env(), spill_state, + group_values_soft_limit: agg.limit, }) } } @@ -419,7 +426,7 @@ impl Stream for GroupedHashAggregateStream { loop { match &self.exec_state { - ExecutionState::ReadingInput => { + ExecutionState::ReadingInput => 'reading_input: { match ready!(self.input.poll_next_unpin(cx)) { // new batch to aggregate Some(Ok(batch)) => { @@ -434,9 +441,21 @@ impl Stream for GroupedHashAggregateStream { // otherwise keep consuming input assert!(!self.input_done); + // If the number of group values equals or exceeds the soft limit, + // emit all groups and switch to producing output + if self.hit_soft_group_limit() { + timer.done(); + extract_ok!(self.set_input_done_and_produce_output()); + // make sure the exec_state just set is not overwritten below + break 'reading_input; + } + if let Some(to_emit) = self.group_ordering.emit_to() { let batch = extract_ok!(self.emit(to_emit, false)); self.exec_state = ExecutionState::ProducingOutput(batch); + timer.done(); + // make sure the exec_state just set is not overwritten below + break 'reading_input; } extract_ok!(self.emit_early_if_necessary()); @@ -449,18 +468,7 @@ impl Stream for GroupedHashAggregateStream { } None => { // inner is done, emit all rows and switch to producing output - self.input_done = true; - self.group_ordering.input_done(); - let timer = elapsed_compute.timer(); - self.exec_state = if self.spill_state.spills.is_empty() { - let batch = extract_ok!(self.emit(EmitTo::All, false)); - ExecutionState::ProducingOutput(batch) - } else { - // If spill files exist, stream-merge them. - extract_ok!(self.update_merged_stream()); - ExecutionState::ReadingInput - }; - timer.done(); + extract_ok!(self.set_input_done_and_produce_output()); } } } @@ -759,4 +767,31 @@ impl GroupedHashAggregateStream { self.group_ordering = GroupOrdering::Full(GroupOrderingFull::new()); Ok(()) } + + /// returns true if there is a soft groups limit and the number of distinct + /// groups we have seen is over that limit + fn hit_soft_group_limit(&self) -> bool { + let Some(group_values_soft_limit) = self.group_values_soft_limit else { + return false; + }; + group_values_soft_limit <= self.group_values.len() + } + + /// common function for signalling end of processing of the input stream + fn set_input_done_and_produce_output(&mut self) -> Result<()> { + self.input_done = true; + self.group_ordering.input_done(); + let elapsed_compute = self.baseline_metrics.elapsed_compute().clone(); + let timer = elapsed_compute.timer(); + self.exec_state = if self.spill_state.spills.is_empty() { + let batch = self.emit(EmitTo::All, false)?; + ExecutionState::ProducingOutput(batch) + } else { + // If spill files exist, stream-merge them. + self.update_merged_stream()?; + ExecutionState::ReadingInput + }; + timer.done(); + Ok(()) + } } diff --git a/datafusion/sqllogictest/test_files/aggregate.slt b/datafusion/sqllogictest/test_files/aggregate.slt index 6217f12279a9..a1bb93ed53c4 100644 --- a/datafusion/sqllogictest/test_files/aggregate.slt +++ b/datafusion/sqllogictest/test_files/aggregate.slt @@ -2523,6 +2523,204 @@ NULL 0 0 b 0 0 c 1 1 +# +# Push limit into distinct group-by aggregation tests +# + +# Make results deterministic +statement ok +set datafusion.optimizer.repartition_aggregations = false; + +# +query TT +EXPLAIN SELECT DISTINCT c3 FROM aggregate_test_100 group by c3 limit 5; +---- +logical_plan +Limit: skip=0, fetch=5 +--Aggregate: groupBy=[[aggregate_test_100.c3]], aggr=[[]] +----Aggregate: groupBy=[[aggregate_test_100.c3]], aggr=[[]] +------TableScan: aggregate_test_100 projection=[c3] +physical_plan +GlobalLimitExec: skip=0, fetch=5 +--AggregateExec: mode=Final, gby=[c3@0 as c3], aggr=[], lim=[5] +----CoalescePartitionsExec +------AggregateExec: mode=Partial, gby=[c3@0 as c3], aggr=[], lim=[5] +--------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +----------AggregateExec: mode=Final, gby=[c3@0 as c3], aggr=[], lim=[5] +------------CoalescePartitionsExec +--------------AggregateExec: mode=Partial, gby=[c3@0 as c3], aggr=[], lim=[5] +----------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c3], has_header=true + +query I +SELECT DISTINCT c3 FROM aggregate_test_100 group by c3 limit 5; +---- +1 +-40 +29 +-85 +-82 + +query TT +EXPLAIN SELECT c2, c3 FROM aggregate_test_100 group by c2, c3 limit 5 offset 4; +---- +logical_plan +Limit: skip=4, fetch=5 +--Aggregate: groupBy=[[aggregate_test_100.c2, aggregate_test_100.c3]], aggr=[[]] +----TableScan: aggregate_test_100 projection=[c2, c3] +physical_plan +GlobalLimitExec: skip=4, fetch=5 +--AggregateExec: mode=Final, gby=[c2@0 as c2, c3@1 as c3], aggr=[], lim=[9] +----CoalescePartitionsExec +------AggregateExec: mode=Partial, gby=[c2@0 as c2, c3@1 as c3], aggr=[], lim=[9] +--------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +----------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c2, c3], has_header=true + +query II +SELECT c2, c3 FROM aggregate_test_100 group by c2, c3 limit 5 offset 4; +---- +5 -82 +4 -111 +3 104 +3 13 +1 38 + +# The limit should only apply to the aggregations which group by c3 +query TT +EXPLAIN SELECT DISTINCT c3 FROM aggregate_test_100 WHERE c3 between 10 and 20 group by c2, c3 limit 4; +---- +logical_plan +Limit: skip=0, fetch=4 +--Aggregate: groupBy=[[aggregate_test_100.c3]], aggr=[[]] +----Projection: aggregate_test_100.c3 +------Aggregate: groupBy=[[aggregate_test_100.c2, aggregate_test_100.c3]], aggr=[[]] +--------Filter: aggregate_test_100.c3 >= Int16(10) AND aggregate_test_100.c3 <= Int16(20) +----------TableScan: aggregate_test_100 projection=[c2, c3], partial_filters=[aggregate_test_100.c3 >= Int16(10), aggregate_test_100.c3 <= Int16(20)] +physical_plan +GlobalLimitExec: skip=0, fetch=4 +--AggregateExec: mode=Final, gby=[c3@0 as c3], aggr=[], lim=[4] +----CoalescePartitionsExec +------AggregateExec: mode=Partial, gby=[c3@0 as c3], aggr=[], lim=[4] +--------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +----------ProjectionExec: expr=[c3@1 as c3] +------------AggregateExec: mode=Final, gby=[c2@0 as c2, c3@1 as c3], aggr=[] +--------------CoalescePartitionsExec +----------------AggregateExec: mode=Partial, gby=[c2@0 as c2, c3@1 as c3], aggr=[] +------------------CoalesceBatchesExec: target_batch_size=8192 +--------------------FilterExec: c3@1 >= 10 AND c3@1 <= 20 +----------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c2, c3], has_header=true + +query I +SELECT DISTINCT c3 FROM aggregate_test_100 WHERE c3 between 10 and 20 group by c2, c3 limit 4; +---- +13 +17 +12 +14 + +# An aggregate expression causes the limit to not be pushed to the aggregation +query TT +EXPLAIN SELECT max(c1), c2, c3 FROM aggregate_test_100 group by c2, c3 limit 5; +---- +logical_plan +Projection: MAX(aggregate_test_100.c1), aggregate_test_100.c2, aggregate_test_100.c3 +--Limit: skip=0, fetch=5 +----Aggregate: groupBy=[[aggregate_test_100.c2, aggregate_test_100.c3]], aggr=[[MAX(aggregate_test_100.c1)]] +------TableScan: aggregate_test_100 projection=[c1, c2, c3] +physical_plan +ProjectionExec: expr=[MAX(aggregate_test_100.c1)@2 as MAX(aggregate_test_100.c1), c2@0 as c2, c3@1 as c3] +--GlobalLimitExec: skip=0, fetch=5 +----AggregateExec: mode=Final, gby=[c2@0 as c2, c3@1 as c3], aggr=[MAX(aggregate_test_100.c1)] +------CoalescePartitionsExec +--------AggregateExec: mode=Partial, gby=[c2@1 as c2, c3@2 as c3], aggr=[MAX(aggregate_test_100.c1)] +----------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1, c2, c3], has_header=true + +# TODO(msirek): Extend checking in LimitedDistinctAggregation equal groupings to ignore the order of columns +# in the group-by column lists, so the limit could be pushed to the lowest AggregateExec in this case +query TT +EXPLAIN SELECT DISTINCT c3, c2 FROM aggregate_test_100 group by c2, c3 limit 3 offset 10; +---- +logical_plan +Limit: skip=10, fetch=3 +--Aggregate: groupBy=[[aggregate_test_100.c3, aggregate_test_100.c2]], aggr=[[]] +----Projection: aggregate_test_100.c3, aggregate_test_100.c2 +------Aggregate: groupBy=[[aggregate_test_100.c2, aggregate_test_100.c3]], aggr=[[]] +--------TableScan: aggregate_test_100 projection=[c2, c3] +physical_plan +GlobalLimitExec: skip=10, fetch=3 +--AggregateExec: mode=Final, gby=[c3@0 as c3, c2@1 as c2], aggr=[], lim=[13] +----CoalescePartitionsExec +------AggregateExec: mode=Partial, gby=[c3@0 as c3, c2@1 as c2], aggr=[], lim=[13] +--------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +----------ProjectionExec: expr=[c3@1 as c3, c2@0 as c2] +------------AggregateExec: mode=Final, gby=[c2@0 as c2, c3@1 as c3], aggr=[] +--------------CoalescePartitionsExec +----------------AggregateExec: mode=Partial, gby=[c2@0 as c2, c3@1 as c3], aggr=[] +------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +--------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c2, c3], has_header=true + +query II +SELECT DISTINCT c3, c2 FROM aggregate_test_100 group by c2, c3 limit 3 offset 10; +---- +57 1 +-54 4 +112 3 + +query TT +EXPLAIN SELECT c2, c3 FROM aggregate_test_100 group by rollup(c2, c3) limit 3; +---- +logical_plan +Limit: skip=0, fetch=3 +--Aggregate: groupBy=[[ROLLUP (aggregate_test_100.c2, aggregate_test_100.c3)]], aggr=[[]] +----TableScan: aggregate_test_100 projection=[c2, c3] +physical_plan +GlobalLimitExec: skip=0, fetch=3 +--AggregateExec: mode=Final, gby=[c2@0 as c2, c3@1 as c3], aggr=[], lim=[3] +----CoalescePartitionsExec +------AggregateExec: mode=Partial, gby=[(NULL as c2, NULL as c3), (c2@0 as c2, NULL as c3), (c2@0 as c2, c3@1 as c3)], aggr=[] +--------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +----------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c2, c3], has_header=true + +query II +SELECT c2, c3 FROM aggregate_test_100 group by rollup(c2, c3) limit 3; +---- +NULL NULL +2 NULL +5 NULL + + +statement ok +set datafusion.optimizer.enable_distinct_aggregation_soft_limit = false; + +# The limit should not be pushed into the aggregations +query TT +EXPLAIN SELECT DISTINCT c3 FROM aggregate_test_100 group by c3 limit 5; +---- +logical_plan +Limit: skip=0, fetch=5 +--Aggregate: groupBy=[[aggregate_test_100.c3]], aggr=[[]] +----Aggregate: groupBy=[[aggregate_test_100.c3]], aggr=[[]] +------TableScan: aggregate_test_100 projection=[c3] +physical_plan +GlobalLimitExec: skip=0, fetch=5 +--AggregateExec: mode=Final, gby=[c3@0 as c3], aggr=[] +----CoalescePartitionsExec +------AggregateExec: mode=Partial, gby=[c3@0 as c3], aggr=[] +--------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +----------AggregateExec: mode=Final, gby=[c3@0 as c3], aggr=[] +------------CoalescePartitionsExec +--------------AggregateExec: mode=Partial, gby=[c3@0 as c3], aggr=[] +----------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c3], has_header=true + +statement ok +set datafusion.optimizer.enable_distinct_aggregation_soft_limit = true; + +statement ok +set datafusion.optimizer.repartition_aggregations = true; + # # regr_*() tests # diff --git a/datafusion/sqllogictest/test_files/explain.slt b/datafusion/sqllogictest/test_files/explain.slt index 49bb63d75d8b..911ede678bde 100644 --- a/datafusion/sqllogictest/test_files/explain.slt +++ b/datafusion/sqllogictest/test_files/explain.slt @@ -250,6 +250,7 @@ OutputRequirementExec --CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/example.csv]]}, projection=[a, b, c], has_header=true physical_plan after aggregate_statistics SAME TEXT AS ABOVE physical_plan after join_selection SAME TEXT AS ABOVE +physical_plan after LimitedDistinctAggregation SAME TEXT AS ABOVE physical_plan after EnforceDistribution SAME TEXT AS ABOVE physical_plan after CombinePartialFinalAggregate SAME TEXT AS ABOVE physical_plan after EnforceSorting SAME TEXT AS ABOVE diff --git a/datafusion/sqllogictest/test_files/information_schema.slt b/datafusion/sqllogictest/test_files/information_schema.slt index ed85f54a39aa..741ff724781f 100644 --- a/datafusion/sqllogictest/test_files/information_schema.slt +++ b/datafusion/sqllogictest/test_files/information_schema.slt @@ -188,6 +188,7 @@ datafusion.explain.logical_plan_only false datafusion.explain.physical_plan_only false datafusion.explain.show_statistics false datafusion.optimizer.allow_symmetric_joins_without_pruning true +datafusion.optimizer.enable_distinct_aggregation_soft_limit true datafusion.optimizer.enable_round_robin_repartition true datafusion.optimizer.enable_topk_aggregation true datafusion.optimizer.filter_null_join_keys false @@ -260,6 +261,7 @@ datafusion.explain.logical_plan_only false When set to true, the explain stateme datafusion.explain.physical_plan_only false When set to true, the explain statement will only print physical plans datafusion.explain.show_statistics false When set to true, the explain statement will print operator statistics for physical plans datafusion.optimizer.allow_symmetric_joins_without_pruning true Should DataFusion allow symmetric hash joins for unbounded data sources even when its inputs do not have any ordering or filtering If the flag is not enabled, the SymmetricHashJoin operator will be unable to prune its internal buffers, resulting in certain join types - such as Full, Left, LeftAnti, LeftSemi, Right, RightAnti, and RightSemi - being produced only at the end of the execution. This is not typical in stream processing. Additionally, without proper design for long runner execution, all types of joins may encounter out-of-memory errors. +datafusion.optimizer.enable_distinct_aggregation_soft_limit true When set to true, the optimizer will push a limit operation into grouped aggregations which have no aggregate expressions, as a soft limit, emitting groups once the limit is reached, before all rows in the group are read. datafusion.optimizer.enable_round_robin_repartition true When set to true, the physical plan optimizer will try to add round robin repartitioning to increase parallelism to leverage more CPU cores datafusion.optimizer.enable_topk_aggregation true When set to true, the optimizer will attempt to perform limit operations during aggregations, if possible datafusion.optimizer.filter_null_join_keys false When set to true, the optimizer will insert filters before a join between a nullable and non-nullable column to filter out nulls on the nullable side. This filter can add additional overhead when the file format does not fully support predicate push down. diff --git a/docs/source/user-guide/configs.md b/docs/source/user-guide/configs.md index 4cc4fd1c3a25..11363f0657f6 100644 --- a/docs/source/user-guide/configs.md +++ b/docs/source/user-guide/configs.md @@ -82,6 +82,7 @@ Environment variables are read during `SessionConfig` initialisation so they mus | datafusion.execution.minimum_parallel_output_files | 4 | Guarantees a minimum level of output files running in parallel. RecordBatches will be distributed in round robin fashion to each parallel writer. Each writer is closed and a new file opened once soft_max_rows_per_output_file is reached. | | datafusion.execution.soft_max_rows_per_output_file | 50000000 | Target number of rows in output files when writing multiple. This is a soft max, so it can be exceeded slightly. There also will be one file smaller than the limit if the total number of rows written is not roughly divisible by the soft max | | datafusion.execution.max_buffered_batches_per_output_file | 2 | This is the maximum number of RecordBatches buffered for each output file being worked. Higher values can potentially give faster write performance at the cost of higher peak memory consumption | +| datafusion.optimizer.enable_distinct_aggregation_soft_limit | true | When set to true, the optimizer will push a limit operation into grouped aggregations which have no aggregate expressions, as a soft limit, emitting groups once the limit is reached, before all rows in the group are read. | | datafusion.optimizer.enable_round_robin_repartition | true | When set to true, the physical plan optimizer will try to add round robin repartitioning to increase parallelism to leverage more CPU cores | | datafusion.optimizer.enable_topk_aggregation | true | When set to true, the optimizer will attempt to perform limit operations during aggregations, if possible | | datafusion.optimizer.filter_null_join_keys | false | When set to true, the optimizer will insert filters before a join between a nullable and non-nullable column to filter out nulls on the nullable side. This filter can add additional overhead when the file format does not fully support predicate push down. | From e54894c39202815b14d9e7eae58f64d3a269c165 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Berkay=20=C5=9Eahin?= <124376117+berkaysynnada@users.noreply.github.com> Date: Thu, 9 Nov 2023 19:49:42 +0300 Subject: [PATCH 214/572] Bug-fix in Filter and Limit statistics (#8094) * Bug fix and code simplification * Remove limit stats changes * Test added * Reduce code diff --- datafusion/common/src/stats.rs | 6 ++-- datafusion/core/src/datasource/statistics.rs | 6 +++- datafusion/physical-plan/src/filter.rs | 38 ++++++++++++++++++-- 3 files changed, 45 insertions(+), 5 deletions(-) diff --git a/datafusion/common/src/stats.rs b/datafusion/common/src/stats.rs index fbf639a32182..2e799c92bea7 100644 --- a/datafusion/common/src/stats.rs +++ b/datafusion/common/src/stats.rs @@ -279,9 +279,11 @@ pub struct ColumnStatistics { impl ColumnStatistics { /// Column contains a single non null value (e.g constant). pub fn is_singleton(&self) -> bool { - match (self.min_value.get_value(), self.max_value.get_value()) { + match (&self.min_value, &self.max_value) { // Min and max values are the same and not infinity. - (Some(min), Some(max)) => !min.is_null() && !max.is_null() && (min == max), + (Precision::Exact(min), Precision::Exact(max)) => { + !min.is_null() && !max.is_null() && (min == max) + } (_, _) => false, } } diff --git a/datafusion/core/src/datasource/statistics.rs b/datafusion/core/src/datasource/statistics.rs index 3d8248dfdeb2..695e139517cf 100644 --- a/datafusion/core/src/datasource/statistics.rs +++ b/datafusion/core/src/datasource/statistics.rs @@ -70,7 +70,11 @@ pub async fn get_statistics_with_limit( // files. This only applies when we know the number of rows. It also // currently ignores tables that have no statistics regarding the // number of rows. - if num_rows.get_value().unwrap_or(&usize::MIN) <= &limit.unwrap_or(usize::MAX) { + let conservative_num_rows = match num_rows { + Precision::Exact(nr) => nr, + _ => usize::MIN, + }; + if conservative_num_rows <= limit.unwrap_or(usize::MAX) { while let Some(current) = all_files.next().await { let (file, file_stats) = current?; result_files.push(file); diff --git a/datafusion/physical-plan/src/filter.rs b/datafusion/physical-plan/src/filter.rs index ce66d614721c..0c44b367e514 100644 --- a/datafusion/physical-plan/src/filter.rs +++ b/datafusion/physical-plan/src/filter.rs @@ -252,13 +252,25 @@ fn collect_new_statistics( }, )| { let closed_interval = interval.close_bounds(); + let (min_value, max_value) = + if closed_interval.lower.value.eq(&closed_interval.upper.value) { + ( + Precision::Exact(closed_interval.lower.value), + Precision::Exact(closed_interval.upper.value), + ) + } else { + ( + Precision::Inexact(closed_interval.lower.value), + Precision::Inexact(closed_interval.upper.value), + ) + }; ColumnStatistics { null_count: match input_column_stats[idx].null_count.get_value() { Some(nc) => Precision::Inexact(*nc), None => Precision::Absent, }, - max_value: Precision::Inexact(closed_interval.upper.value), - min_value: Precision::Inexact(closed_interval.lower.value), + max_value, + min_value, distinct_count: match distinct_count.get_value() { Some(dc) => Precision::Inexact(*dc), None => Precision::Absent, @@ -963,4 +975,26 @@ mod tests { Ok(()) } + + #[tokio::test] + async fn test_statistics_with_constant_column() -> Result<()> { + let schema = Schema::new(vec![Field::new("a", DataType::Int32, false)]); + let input = Arc::new(StatisticsExec::new( + Statistics::new_unknown(&schema), + schema, + )); + // WHERE a = 10 + let predicate = Arc::new(BinaryExpr::new( + Arc::new(Column::new("a", 0)), + Operator::Eq, + Arc::new(Literal::new(ScalarValue::Int32(Some(10)))), + )); + let filter: Arc = + Arc::new(FilterExec::try_new(predicate, input)?); + let filter_statistics = filter.statistics()?; + // First column is "a", and it is a column with only one value after the filter. + assert!(filter_statistics.column_statistics[0].is_singleton()); + + Ok(()) + } } From a564af59ed7dd039d5fba17f88e8cb70dda37292 Mon Sep 17 00:00:00 2001 From: Jonah Gao Date: Fri, 10 Nov 2023 00:52:28 +0800 Subject: [PATCH 215/572] feat: support target table alias in update statement (#8080) --- datafusion/sql/src/statement.rs | 18 ++++++++++++++---- datafusion/sqllogictest/test_files/update.slt | 15 ++++++++++++++- 2 files changed, 28 insertions(+), 5 deletions(-) diff --git a/datafusion/sql/src/statement.rs b/datafusion/sql/src/statement.rs index 9d9c55361a5e..116624c6f7b9 100644 --- a/datafusion/sql/src/statement.rs +++ b/datafusion/sql/src/statement.rs @@ -31,7 +31,7 @@ use arrow_schema::DataType; use datafusion_common::file_options::StatementOptions; use datafusion_common::parsers::CompressionTypeVariant; use datafusion_common::{ - not_impl_err, plan_datafusion_err, plan_err, unqualified_field_not_found, + not_impl_err, plan_datafusion_err, plan_err, unqualified_field_not_found, Column, Constraints, DFField, DFSchema, DFSchemaRef, DataFusionError, OwnedTableReference, Result, SchemaReference, TableReference, ToDFSchema, }; @@ -970,8 +970,8 @@ impl<'a, S: ContextProvider> SqlToRel<'a, S> { from: Option, predicate_expr: Option, ) -> Result { - let table_name = match &table.relation { - TableFactor::Table { name, .. } => name.clone(), + let (table_name, table_alias) = match &table.relation { + TableFactor::Table { name, alias, .. } => (name.clone(), alias.clone()), _ => plan_err!("Cannot update non-table relation!")?, }; @@ -1047,7 +1047,17 @@ impl<'a, S: ContextProvider> SqlToRel<'a, S> { // Cast to target column type, if necessary expr.cast_to(field.data_type(), source.schema())? } - None => datafusion_expr::Expr::Column(field.qualified_column()), + None => { + // If the target table has an alias, use it to qualify the column name + if let Some(alias) = &table_alias { + datafusion_expr::Expr::Column(Column::new( + Some(self.normalizer.normalize(alias.name.clone())), + field.name(), + )) + } else { + datafusion_expr::Expr::Column(field.qualified_column()) + } + } }; Ok(expr.alias(field.name())) }) diff --git a/datafusion/sqllogictest/test_files/update.slt b/datafusion/sqllogictest/test_files/update.slt index cb8c6a4fac28..c88082fc7272 100644 --- a/datafusion/sqllogictest/test_files/update.slt +++ b/datafusion/sqllogictest/test_files/update.slt @@ -76,4 +76,17 @@ create table t3(a int, b varchar, c double, d int); # set from mutiple tables, sqlparser only supports from one table query error DataFusion error: SQL error: ParserError\("Expected end of statement, found: ,"\) -explain update t1 set b = t2.b, c = t3.a, d = 1 from t2, t3 where t1.a = t2.a and t1.a = t3.a; \ No newline at end of file +explain update t1 set b = t2.b, c = t3.a, d = 1 from t2, t3 where t1.a = t2.a and t1.a = t3.a; + +# test table alias +query TT +explain update t1 as T set b = t2.b, c = t.a, d = 1 from t2 where t.a = t2.a and t.b > 'foo' and t2.c > 1.0; +---- +logical_plan +Dml: op=[Update] table=[t1] +--Projection: t.a AS a, t2.b AS b, CAST(t.a AS Float64) AS c, CAST(Int64(1) AS Int32) AS d +----Filter: t.a = t2.a AND t.b > Utf8("foo") AND t2.c > Float64(1) +------CrossJoin: +--------SubqueryAlias: t +----------TableScan: t1 +--------TableScan: t2 \ No newline at end of file From 1803b25c3953ce0422a3a2d3f768362715635c5b Mon Sep 17 00:00:00 2001 From: Alex Huang Date: Thu, 9 Nov 2023 18:42:07 +0100 Subject: [PATCH 216/572] Simlify downcast functions in cast.rs. (#8103) --- datafusion/common/src/cast.rs | 12 +++--------- 1 file changed, 3 insertions(+), 9 deletions(-) diff --git a/datafusion/common/src/cast.rs b/datafusion/common/src/cast.rs index 4356f36b18d8..088f03e002ed 100644 --- a/datafusion/common/src/cast.rs +++ b/datafusion/common/src/cast.rs @@ -181,23 +181,17 @@ pub fn as_timestamp_second_array(array: &dyn Array) -> Result<&TimestampSecondAr } // Downcast ArrayRef to IntervalYearMonthArray -pub fn as_interval_ym_array( - array: &dyn Array, -) -> Result<&IntervalYearMonthArray, DataFusionError> { +pub fn as_interval_ym_array(array: &dyn Array) -> Result<&IntervalYearMonthArray> { Ok(downcast_value!(array, IntervalYearMonthArray)) } // Downcast ArrayRef to IntervalDayTimeArray -pub fn as_interval_dt_array( - array: &dyn Array, -) -> Result<&IntervalDayTimeArray, DataFusionError> { +pub fn as_interval_dt_array(array: &dyn Array) -> Result<&IntervalDayTimeArray> { Ok(downcast_value!(array, IntervalDayTimeArray)) } // Downcast ArrayRef to IntervalMonthDayNanoArray -pub fn as_interval_mdn_array( - array: &dyn Array, -) -> Result<&IntervalMonthDayNanoArray, DataFusionError> { +pub fn as_interval_mdn_array(array: &dyn Array) -> Result<&IntervalMonthDayNanoArray> { Ok(downcast_value!(array, IntervalMonthDayNanoArray)) } From 91a44c1e5aaed6b3037eb620c7a753b51755b187 Mon Sep 17 00:00:00 2001 From: Jay Zhan Date: Fri, 10 Nov 2023 01:44:54 +0800 Subject: [PATCH 217/572] Fix ArrayAgg schema mismatch issue (#8055) * fix schema Signed-off-by: jayzhan211 * cleanup Signed-off-by: jayzhan211 * upd parquet-testing Signed-off-by: jayzhan211 * avoid parquet file Signed-off-by: jayzhan211 * reset parquet-testing Signed-off-by: jayzhan211 * remove file Signed-off-by: jayzhan211 * fix Signed-off-by: jayzhan211 * fix test Signed-off-by: jayzhan211 * cleanup Signed-off-by: jayzhan211 * rename and upd docstring Signed-off-by: jayzhan211 --------- Signed-off-by: jayzhan211 --- datafusion/core/src/dataframe/mod.rs | 86 +++++++++++++++++++ .../physical-expr/src/aggregate/array_agg.rs | 43 ++++++++-- .../src/aggregate/array_agg_distinct.rs | 13 ++- .../src/aggregate/array_agg_ordered.rs | 20 +++-- .../physical-expr/src/aggregate/build_in.rs | 18 ++-- 5 files changed, 160 insertions(+), 20 deletions(-) diff --git a/datafusion/core/src/dataframe/mod.rs b/datafusion/core/src/dataframe/mod.rs index 0a99c331826c..89e82fa952bb 100644 --- a/datafusion/core/src/dataframe/mod.rs +++ b/datafusion/core/src/dataframe/mod.rs @@ -1340,6 +1340,92 @@ mod tests { use super::*; + async fn assert_logical_expr_schema_eq_physical_expr_schema( + df: DataFrame, + ) -> Result<()> { + let logical_expr_dfschema = df.schema(); + let logical_expr_schema = SchemaRef::from(logical_expr_dfschema.to_owned()); + let batches = df.collect().await?; + let physical_expr_schema = batches[0].schema(); + assert_eq!(logical_expr_schema, physical_expr_schema); + Ok(()) + } + + #[tokio::test] + async fn test_array_agg_ord_schema() -> Result<()> { + let ctx = SessionContext::new(); + + let create_table_query = r#" + CREATE TABLE test_table ( + "double_field" DOUBLE, + "string_field" VARCHAR + ) AS VALUES + (1.0, 'a'), + (2.0, 'b'), + (3.0, 'c') + "#; + ctx.sql(create_table_query).await?; + + let query = r#"SELECT + array_agg("double_field" ORDER BY "string_field") as "double_field", + array_agg("string_field" ORDER BY "string_field") as "string_field" + FROM test_table"#; + + let result = ctx.sql(query).await?; + assert_logical_expr_schema_eq_physical_expr_schema(result).await?; + Ok(()) + } + + #[tokio::test] + async fn test_array_agg_schema() -> Result<()> { + let ctx = SessionContext::new(); + + let create_table_query = r#" + CREATE TABLE test_table ( + "double_field" DOUBLE, + "string_field" VARCHAR + ) AS VALUES + (1.0, 'a'), + (2.0, 'b'), + (3.0, 'c') + "#; + ctx.sql(create_table_query).await?; + + let query = r#"SELECT + array_agg("double_field") as "double_field", + array_agg("string_field") as "string_field" + FROM test_table"#; + + let result = ctx.sql(query).await?; + assert_logical_expr_schema_eq_physical_expr_schema(result).await?; + Ok(()) + } + + #[tokio::test] + async fn test_array_agg_distinct_schema() -> Result<()> { + let ctx = SessionContext::new(); + + let create_table_query = r#" + CREATE TABLE test_table ( + "double_field" DOUBLE, + "string_field" VARCHAR + ) AS VALUES + (1.0, 'a'), + (2.0, 'b'), + (2.0, 'a') + "#; + ctx.sql(create_table_query).await?; + + let query = r#"SELECT + array_agg(distinct "double_field") as "double_field", + array_agg(distinct "string_field") as "string_field" + FROM test_table"#; + + let result = ctx.sql(query).await?; + assert_logical_expr_schema_eq_physical_expr_schema(result).await?; + Ok(()) + } + #[tokio::test] async fn select_columns() -> Result<()> { // build plan using Table API diff --git a/datafusion/physical-expr/src/aggregate/array_agg.rs b/datafusion/physical-expr/src/aggregate/array_agg.rs index 4dccbfef07f8..91d5c867d312 100644 --- a/datafusion/physical-expr/src/aggregate/array_agg.rs +++ b/datafusion/physical-expr/src/aggregate/array_agg.rs @@ -34,9 +34,14 @@ use std::sync::Arc; /// ARRAY_AGG aggregate expression #[derive(Debug)] pub struct ArrayAgg { + /// Column name name: String, + /// The DataType for the input expression input_data_type: DataType, + /// The input expression expr: Arc, + /// If the input expression can have NULLs + nullable: bool, } impl ArrayAgg { @@ -45,11 +50,13 @@ impl ArrayAgg { expr: Arc, name: impl Into, data_type: DataType, + nullable: bool, ) -> Self { Self { name: name.into(), - expr, input_data_type: data_type, + expr, + nullable, } } } @@ -62,8 +69,9 @@ impl AggregateExpr for ArrayAgg { fn field(&self) -> Result { Ok(Field::new_list( &self.name, + // This should be the same as return type of AggregateFunction::ArrayAgg Field::new("item", self.input_data_type.clone(), true), - false, + self.nullable, )) } @@ -77,7 +85,7 @@ impl AggregateExpr for ArrayAgg { Ok(vec![Field::new_list( format_state_name(&self.name, "array_agg"), Field::new("item", self.input_data_type.clone(), true), - false, + self.nullable, )]) } @@ -184,7 +192,6 @@ mod tests { use super::*; use crate::expressions::col; use crate::expressions::tests::aggregate; - use crate::generic_test_op; use arrow::array::ArrayRef; use arrow::array::Int32Array; use arrow::datatypes::*; @@ -195,6 +202,30 @@ mod tests { use datafusion_common::DataFusionError; use datafusion_common::Result; + macro_rules! test_op { + ($ARRAY:expr, $DATATYPE:expr, $OP:ident, $EXPECTED:expr) => { + test_op!($ARRAY, $DATATYPE, $OP, $EXPECTED, $EXPECTED.data_type()) + }; + ($ARRAY:expr, $DATATYPE:expr, $OP:ident, $EXPECTED:expr, $EXPECTED_DATATYPE:expr) => {{ + let schema = Schema::new(vec![Field::new("a", $DATATYPE, true)]); + + let batch = RecordBatch::try_new(Arc::new(schema.clone()), vec![$ARRAY])?; + + let agg = Arc::new(<$OP>::new( + col("a", &schema)?, + "bla".to_string(), + $EXPECTED_DATATYPE, + true, + )); + let actual = aggregate(&batch, agg)?; + let expected = ScalarValue::from($EXPECTED); + + assert_eq!(expected, actual); + + Ok(()) as Result<(), DataFusionError> + }}; + } + #[test] fn array_agg_i32() -> Result<()> { let a: ArrayRef = Arc::new(Int32Array::from(vec![1, 2, 3, 4, 5])); @@ -208,7 +239,7 @@ mod tests { ])]); let list = ScalarValue::List(Arc::new(list)); - generic_test_op!(a, DataType::Int32, ArrayAgg, list, DataType::Int32) + test_op!(a, DataType::Int32, ArrayAgg, list, DataType::Int32) } #[test] @@ -264,7 +295,7 @@ mod tests { let array = ScalarValue::iter_to_array(vec![l1, l2, l3]).unwrap(); - generic_test_op!( + test_op!( array, DataType::List(Arc::new(Field::new_list( "item", diff --git a/datafusion/physical-expr/src/aggregate/array_agg_distinct.rs b/datafusion/physical-expr/src/aggregate/array_agg_distinct.rs index 9b391b0c42cf..1efae424cc69 100644 --- a/datafusion/physical-expr/src/aggregate/array_agg_distinct.rs +++ b/datafusion/physical-expr/src/aggregate/array_agg_distinct.rs @@ -40,6 +40,8 @@ pub struct DistinctArrayAgg { input_data_type: DataType, /// The input expression expr: Arc, + /// If the input expression can have NULLs + nullable: bool, } impl DistinctArrayAgg { @@ -48,12 +50,14 @@ impl DistinctArrayAgg { expr: Arc, name: impl Into, input_data_type: DataType, + nullable: bool, ) -> Self { let name = name.into(); Self { name, - expr, input_data_type, + expr, + nullable, } } } @@ -67,8 +71,9 @@ impl AggregateExpr for DistinctArrayAgg { fn field(&self) -> Result { Ok(Field::new_list( &self.name, + // This should be the same as return type of AggregateFunction::ArrayAgg Field::new("item", self.input_data_type.clone(), true), - false, + self.nullable, )) } @@ -82,7 +87,7 @@ impl AggregateExpr for DistinctArrayAgg { Ok(vec![Field::new_list( format_state_name(&self.name, "distinct_array_agg"), Field::new("item", self.input_data_type.clone(), true), - false, + self.nullable, )]) } @@ -238,6 +243,7 @@ mod tests { col("a", &schema)?, "bla".to_string(), datatype, + true, )); let actual = aggregate(&batch, agg)?; @@ -255,6 +261,7 @@ mod tests { col("a", &schema)?, "bla".to_string(), datatype, + true, )); let mut accum1 = agg.create_accumulator()?; diff --git a/datafusion/physical-expr/src/aggregate/array_agg_ordered.rs b/datafusion/physical-expr/src/aggregate/array_agg_ordered.rs index a53d53107add..9ca83a781a01 100644 --- a/datafusion/physical-expr/src/aggregate/array_agg_ordered.rs +++ b/datafusion/physical-expr/src/aggregate/array_agg_ordered.rs @@ -48,10 +48,17 @@ use itertools::izip; /// and that can merge aggregations from multiple partitions. #[derive(Debug)] pub struct OrderSensitiveArrayAgg { + /// Column name name: String, + /// The DataType for the input expression input_data_type: DataType, - order_by_data_types: Vec, + /// The input expression expr: Arc, + /// If the input expression can have NULLs + nullable: bool, + /// Ordering data types + order_by_data_types: Vec, + /// Ordering requirement ordering_req: LexOrdering, } @@ -61,13 +68,15 @@ impl OrderSensitiveArrayAgg { expr: Arc, name: impl Into, input_data_type: DataType, + nullable: bool, order_by_data_types: Vec, ordering_req: LexOrdering, ) -> Self { Self { name: name.into(), - expr, input_data_type, + expr, + nullable, order_by_data_types, ordering_req, } @@ -82,8 +91,9 @@ impl AggregateExpr for OrderSensitiveArrayAgg { fn field(&self) -> Result { Ok(Field::new_list( &self.name, + // This should be the same as return type of AggregateFunction::ArrayAgg Field::new("item", self.input_data_type.clone(), true), - false, + self.nullable, )) } @@ -99,13 +109,13 @@ impl AggregateExpr for OrderSensitiveArrayAgg { let mut fields = vec![Field::new_list( format_state_name(&self.name, "array_agg"), Field::new("item", self.input_data_type.clone(), true), - false, + self.nullable, // This should be the same as field() )]; let orderings = ordering_fields(&self.ordering_req, &self.order_by_data_types); fields.push(Field::new_list( format_state_name(&self.name, "array_agg_orderings"), Field::new("item", DataType::Struct(Fields::from(orderings)), true), - false, + self.nullable, )); Ok(fields) } diff --git a/datafusion/physical-expr/src/aggregate/build_in.rs b/datafusion/physical-expr/src/aggregate/build_in.rs index 6568457bc234..596197b4eebe 100644 --- a/datafusion/physical-expr/src/aggregate/build_in.rs +++ b/datafusion/physical-expr/src/aggregate/build_in.rs @@ -114,13 +114,16 @@ pub fn create_aggregate_expr( ), (AggregateFunction::ArrayAgg, false) => { let expr = input_phy_exprs[0].clone(); + let nullable = expr.nullable(input_schema)?; + if ordering_req.is_empty() { - Arc::new(expressions::ArrayAgg::new(expr, name, data_type)) + Arc::new(expressions::ArrayAgg::new(expr, name, data_type, nullable)) } else { Arc::new(expressions::OrderSensitiveArrayAgg::new( expr, name, data_type, + nullable, ordering_types, ordering_req.to_vec(), )) @@ -132,10 +135,13 @@ pub fn create_aggregate_expr( "ARRAY_AGG(DISTINCT ORDER BY a ASC) order-sensitive aggregations are not available" ); } + let expr = input_phy_exprs[0].clone(); + let is_expr_nullable = expr.nullable(input_schema)?; Arc::new(expressions::DistinctArrayAgg::new( - input_phy_exprs[0].clone(), + expr, name, data_type, + is_expr_nullable, )) } (AggregateFunction::Min, _) => Arc::new(expressions::Min::new( @@ -432,8 +438,8 @@ mod tests { assert_eq!( Field::new_list( "c1", - Field::new("item", data_type.clone(), true,), - false, + Field::new("item", data_type.clone(), true), + true, ), result_agg_phy_exprs.field().unwrap() ); @@ -471,8 +477,8 @@ mod tests { assert_eq!( Field::new_list( "c1", - Field::new("item", data_type.clone(), true,), - false, + Field::new("item", data_type.clone(), true), + true, ), result_agg_phy_exprs.field().unwrap() ); From 93af4401d77e9761ca3d187cdc56aa245f7aa7aa Mon Sep 17 00:00:00 2001 From: Andrew Lamb Date: Thu, 9 Nov 2023 12:45:16 -0500 Subject: [PATCH 218/572] Minor: Support `nulls` in `array_replace`, avoid a copy (#8054) * Minor: clean up array_replace * null test * remove println * Fix doc test * port test to sqllogictest * Use not_distinct * Apply suggestions from code review Co-authored-by: jakevin --------- Co-authored-by: jakevin --- .../physical-expr/src/array_expressions.rs | 151 ++++++++++-------- datafusion/sqllogictest/test_files/array.slt | 31 ++++ 2 files changed, 119 insertions(+), 63 deletions(-) diff --git a/datafusion/physical-expr/src/array_expressions.rs b/datafusion/physical-expr/src/array_expressions.rs index 64550aabf424..deb4372baa32 100644 --- a/datafusion/physical-expr/src/array_expressions.rs +++ b/datafusion/physical-expr/src/array_expressions.rs @@ -1211,119 +1211,144 @@ array_removement_function!( "Array_remove_all SQL function" ); -fn general_replace(args: &[ArrayRef], arr_n: Vec) -> Result { - let list_array = as_list_array(&args[0])?; - let from_array = &args[1]; - let to_array = &args[2]; - +/// For each element of `list_array[i]`, replaces up to `arr_n[i]` occurences +/// of `from_array[i]`, `to_array[i]`. +/// +/// The type of each **element** in `list_array` must be the same as the type of +/// `from_array` and `to_array`. This function also handles nested arrays +/// ([`ListArray`] of [`ListArray`]s) +/// +/// For example, when called to replace a list array (where each element is a +/// list of int32s, the second and third argument are int32 arrays, and the +/// fourth argument is the number of occurrences to replace +/// +/// ```text +/// general_replace( +/// [1, 2, 3, 2], 2, 10, 1 ==> [1, 10, 3, 2] (only the first 2 is replaced) +/// [4, 5, 6, 5], 5, 20, 2 ==> [4, 20, 6, 20] (both 5s are replaced) +/// ) +/// ``` +fn general_replace( + list_array: &ListArray, + from_array: &ArrayRef, + to_array: &ArrayRef, + arr_n: Vec, +) -> Result { + // Build up the offsets for the final output array let mut offsets: Vec = vec![0]; let data_type = list_array.value_type(); - let mut values = new_empty_array(&data_type); + let mut new_values = vec![]; - for (row_index, (arr, n)) in list_array.iter().zip(arr_n.iter()).enumerate() { + // n is the number of elements to replace in this row + for (row_index, (list_array_row, n)) in + list_array.iter().zip(arr_n.iter()).enumerate() + { let last_offset: i32 = offsets .last() .copied() .ok_or_else(|| internal_datafusion_err!("offsets should not be empty"))?; - match arr { - Some(arr) => { - let indices = UInt32Array::from(vec![row_index as u32]); - let from_arr = arrow::compute::take(from_array, &indices, None)?; - let eq_array = match from_arr.data_type() { - // arrow_ord::cmp_eq does not support ListArray, so we need to compare it by loop + match list_array_row { + Some(list_array_row) => { + let indices = UInt32Array::from(vec![row_index as u32]); + let from_array_row = arrow::compute::take(from_array, &indices, None)?; + // Compute all positions in list_row_array (that is itself an + // array) that are equal to `from_array_row` + let eq_array = match from_array_row.data_type() { + // arrow_ord::cmp::eq does not support ListArray, so we need to compare it by loop DataType::List(_) => { - let from_a = as_list_array(&from_arr)?.value(0); - let list_arr = as_list_array(&arr)?; + // compare each element of the from array + let from_array_row_inner = + as_list_array(&from_array_row)?.value(0); + let list_array_row_inner = as_list_array(&list_array_row)?; - let mut bool_values = vec![]; - for arr in list_arr.iter() { - if let Some(a) = arr { - bool_values.push(Some(a.eq(&from_a))); - } else { - return internal_err!( - "Null value is not supported in array_replace" - ); - } - } - BooleanArray::from(bool_values) + list_array_row_inner + .iter() + // compare element by element the current row of list_array + .map(|row| row.map(|row| row.eq(&from_array_row_inner))) + .collect::() } _ => { - let from_arr = Scalar::new(from_arr); - arrow_ord::cmp::eq(&arr, &from_arr)? + let from_arr = Scalar::new(from_array_row); + // use not_distinct so NULL = NULL + arrow_ord::cmp::not_distinct(&list_array_row, &from_arr)? } }; // Use MutableArrayData to build the replaced array + let original_data = list_array_row.to_data(); + let to_data = to_array.to_data(); + let capacity = Capacities::Array(original_data.len() + to_data.len()); + // First array is the original array, second array is the element to replace with. - let arrays = vec![arr, to_array.clone()]; - let arrays_data = arrays - .iter() - .map(|a| a.to_data()) - .collect::>(); - let arrays_data = arrays_data.iter().collect::>(); - - let arrays = arrays - .iter() - .map(|arr| arr.as_ref()) - .collect::>(); - let capacity = Capacities::Array(arrays.iter().map(|a| a.len()).sum()); - - let mut mutable = - MutableArrayData::with_capacities(arrays_data, false, capacity); + let mut mutable = MutableArrayData::with_capacities( + vec![&original_data, &to_data], + false, + capacity, + ); + let original_idx = 0; + let replace_idx = 1; let mut counter = 0; for (i, to_replace) in eq_array.iter().enumerate() { - if let Some(to_replace) = to_replace { - if to_replace { - mutable.extend(1, row_index, row_index + 1); - counter += 1; - if counter == *n { - // extend the rest of the array - mutable.extend(0, i + 1, eq_array.len()); - break; - } - } else { - mutable.extend(0, i, i + 1); + if let Some(true) = to_replace { + mutable.extend(replace_idx, row_index, row_index + 1); + counter += 1; + if counter == *n { + // copy original data for any matches past n + mutable.extend(original_idx, i + 1, eq_array.len()); + break; } } else { - return internal_err!("eq_array should not contain None"); + // copy original data for false / null matches + mutable.extend(original_idx, i, i + 1); } } let data = mutable.freeze(); let replaced_array = arrow_array::make_array(data); - let v = arrow::compute::concat(&[&values, &replaced_array])?; - values = v; offsets.push(last_offset + replaced_array.len() as i32); + new_values.push(replaced_array); } None => { + // Null element results in a null row (no new offsets) offsets.push(last_offset); } } } + let values = if new_values.is_empty() { + new_empty_array(&data_type) + } else { + let new_values: Vec<_> = new_values.iter().map(|a| a.as_ref()).collect(); + arrow::compute::concat(&new_values)? + }; + Ok(Arc::new(ListArray::try_new( Arc::new(Field::new("item", data_type, true)), OffsetBuffer::new(offsets.into()), values, - None, + list_array.nulls().cloned(), )?)) } pub fn array_replace(args: &[ArrayRef]) -> Result { - general_replace(args, vec![1; args[0].len()]) + // replace at most one occurence for each element + let arr_n = vec![1; args[0].len()]; + general_replace(as_list_array(&args[0])?, &args[1], &args[2], arr_n) } pub fn array_replace_n(args: &[ArrayRef]) -> Result { - let arr = as_int64_array(&args[3])?; - let arr_n = arr.values().to_vec(); - general_replace(args, arr_n) + // replace the specified number of occurences + let arr_n = as_int64_array(&args[3])?.values().to_vec(); + general_replace(as_list_array(&args[0])?, &args[1], &args[2], arr_n) } pub fn array_replace_all(args: &[ArrayRef]) -> Result { - general_replace(args, vec![i64::MAX; args[0].len()]) + // replace all occurences (up to "i64::MAX") + let arr_n = vec![i64::MAX; args[0].len()]; + general_replace(as_list_array(&args[0])?, &args[1], &args[2], arr_n) } macro_rules! to_string { diff --git a/datafusion/sqllogictest/test_files/array.slt b/datafusion/sqllogictest/test_files/array.slt index 85218efb5e14..c57369c167f4 100644 --- a/datafusion/sqllogictest/test_files/array.slt +++ b/datafusion/sqllogictest/test_files/array.slt @@ -1720,6 +1720,37 @@ select array_replace_all(make_array([1, 2, 3], [4, 5, 6], [4, 5, 6], [10, 11, 12 [[1, 2, 3], [4, 5, 6], [4, 5, 6], [10, 11, 12], [13, 14, 15], [10, 11, 12], [10, 11, 12], [28, 29, 30], [28, 29, 30], [28, 29, 30], [28, 29, 30], [22, 23, 24]] [[19, 20, 21], [19, 20, 21], [19, 20, 21], [22, 23, 24], [19, 20, 21], [25, 26, 27], [19, 20, 21], [22, 23, 24], [19, 20, 21], [19, 20, 21]] [[11, 12, 13], [11, 12, 13], [11, 12, 13], [22, 23, 24], [11, 12, 13], [25, 26, 27], [11, 12, 13], [22, 23, 24], [11, 12, 13], [11, 12, 13]] [[1, 2, 3], [4, 5, 6], [4, 5, 6], [10, 11, 12], [13, 14, 15], [10, 11, 12], [10, 11, 12], [19, 20, 21], [19, 20, 21], [37, 38, 39], [19, 20, 21], [22, 23, 24]] [[28, 29, 30], [31, 32, 33], [34, 35, 36], [28, 29, 30], [31, 32, 33], [34, 35, 36], [28, 29, 30], [31, 32, 33], [34, 35, 36], [28, 29, 30]] [[11, 12, 13], [31, 32, 33], [34, 35, 36], [11, 12, 13], [31, 32, 33], [34, 35, 36], [11, 12, 13], [31, 32, 33], [34, 35, 36], [11, 12, 13]] +# array_replace with null handling + +statement ok +create table t as values + (make_array(3, 1, NULL, 3), 3, 4, 2), + (make_array(3, 1, NULL, 3), NULL, 5, 2), + (NULL, 3, 2, 1), + (make_array(3, 1, 3), 3, NULL, 1) +; + + +# ([3, 1, NULL, 3], 3, 4, 2) => [4, 1, NULL, 4] NULL not matched +# ([3, 1, NULL, 3], NULL, 5, 2) => [3, 1, NULL, 3] NULL is replaced with 5 +# ([NULL], 3, 2, 1) => NULL +# ([3, 1, 3], 3, NULL, 1) => [NULL, 1 3] + +query ?III? +select column1, column2, column3, column4, array_replace_n(column1, column2, column3, column4) from t; +---- +[3, 1, , 3] 3 4 2 [4, 1, , 4] +[3, 1, , 3] NULL 5 2 [3, 1, 5, 3] +NULL 3 2 1 NULL +[3, 1, 3] 3 NULL 1 [, 1, 3] + + + +statement ok +drop table t; + + + ## array_to_string (aliases: `list_to_string`, `array_join`, `list_join`) # array_to_string scalar function #1 From 91c9d6f847eda0b5b1d01257b5c24459651d3926 Mon Sep 17 00:00:00 2001 From: Asura7969 <1402357969@qq.com> Date: Fri, 10 Nov 2023 04:40:53 +0800 Subject: [PATCH 219/572] Minor: Improve the document format of JoinHashMap (#8090) * Minor: Improve the document format of JoinHashMap * fix:delete todo from document * fix:equal_rows to equal_rows_arr * fix:equal_rows_arr link * fix:cargo doc error --- .../src/joins/hash_join_utils.rs | 114 ++++++++++-------- 1 file changed, 61 insertions(+), 53 deletions(-) diff --git a/datafusion/physical-plan/src/joins/hash_join_utils.rs b/datafusion/physical-plan/src/joins/hash_join_utils.rs index 3a2a85c72722..c134b23d78cf 100644 --- a/datafusion/physical-plan/src/joins/hash_join_utils.rs +++ b/datafusion/physical-plan/src/joins/hash_join_utils.rs @@ -40,59 +40,67 @@ use datafusion_physical_expr::{PhysicalExpr, PhysicalSortExpr}; use hashbrown::raw::RawTable; use hashbrown::HashSet; -// Maps a `u64` hash value based on the build side ["on" values] to a list of indices with this key's value. -// By allocating a `HashMap` with capacity for *at least* the number of rows for entries at the build side, -// we make sure that we don't have to re-hash the hashmap, which needs access to the key (the hash in this case) value. -// E.g. 1 -> [3, 6, 8] indicates that the column values map to rows 3, 6 and 8 for hash value 1 -// As the key is a hash value, we need to check possible hash collisions in the probe stage -// During this stage it might be the case that a row is contained the same hashmap value, -// but the values don't match. Those are checked in the [equal_rows] macro -// The indices (values) are stored in a separate chained list stored in the `Vec`. -// The first value (+1) is stored in the hashmap, whereas the next value is stored in array at the position value. -// The chain can be followed until the value "0" has been reached, meaning the end of the list. -// Also see chapter 5.3 of [Balancing vectorized query execution with bandwidth-optimized storage](https://dare.uva.nl/search?identifier=5ccbb60a-38b8-4eeb-858a-e7735dd37487) -// See the example below: -// Insert (1,1) -// map: -// --------- -// | 1 | 2 | -// --------- -// next: -// --------------------- -// | 0 | 0 | 0 | 0 | 0 | -// --------------------- -// Insert (2,2) -// map: -// --------- -// | 1 | 2 | -// | 2 | 3 | -// --------- -// next: -// --------------------- -// | 0 | 0 | 0 | 0 | 0 | -// --------------------- -// Insert (1,3) -// map: -// --------- -// | 1 | 4 | -// | 2 | 3 | -// --------- -// next: -// --------------------- -// | 0 | 0 | 0 | 2 | 0 | <--- hash value 1 maps to 4,2 (which means indices values 3,1) -// --------------------- -// Insert (1,4) -// map: -// --------- -// | 1 | 5 | -// | 2 | 3 | -// --------- -// next: -// --------------------- -// | 0 | 0 | 0 | 2 | 4 | <--- hash value 1 maps to 5,4,2 (which means indices values 4,3,1) -// --------------------- -// TODO: speed up collision checks -// https://github.com/apache/arrow-datafusion/issues/50 +/// Maps a `u64` hash value based on the build side ["on" values] to a list of indices with this key's value. +/// +/// By allocating a `HashMap` with capacity for *at least* the number of rows for entries at the build side, +/// we make sure that we don't have to re-hash the hashmap, which needs access to the key (the hash in this case) value. +/// +/// E.g. 1 -> [3, 6, 8] indicates that the column values map to rows 3, 6 and 8 for hash value 1 +/// As the key is a hash value, we need to check possible hash collisions in the probe stage +/// During this stage it might be the case that a row is contained the same hashmap value, +/// but the values don't match. Those are checked in the [`equal_rows_arr`](crate::joins::hash_join::equal_rows_arr) method. +/// +/// The indices (values) are stored in a separate chained list stored in the `Vec`. +/// +/// The first value (+1) is stored in the hashmap, whereas the next value is stored in array at the position value. +/// +/// The chain can be followed until the value "0" has been reached, meaning the end of the list. +/// Also see chapter 5.3 of [Balancing vectorized query execution with bandwidth-optimized storage](https://dare.uva.nl/search?identifier=5ccbb60a-38b8-4eeb-858a-e7735dd37487) +/// +/// # Example +/// +/// ``` text +/// See the example below: +/// Insert (1,1) +/// map: +/// --------- +/// | 1 | 2 | +/// --------- +/// next: +/// --------------------- +/// | 0 | 0 | 0 | 0 | 0 | +/// --------------------- +/// Insert (2,2) +/// map: +/// --------- +/// | 1 | 2 | +/// | 2 | 3 | +/// --------- +/// next: +/// --------------------- +/// | 0 | 0 | 0 | 0 | 0 | +/// --------------------- +/// Insert (1,3) +/// map: +/// --------- +/// | 1 | 4 | +/// | 2 | 3 | +/// --------- +/// next: +/// --------------------- +/// | 0 | 0 | 0 | 2 | 0 | <--- hash value 1 maps to 4,2 (which means indices values 3,1) +/// --------------------- +/// Insert (1,4) +/// map: +/// --------- +/// | 1 | 5 | +/// | 2 | 3 | +/// --------- +/// next: +/// --------------------- +/// | 0 | 0 | 0 | 2 | 4 | <--- hash value 1 maps to 5,4,2 (which means indices values 4,3,1) +/// --------------------- +/// ``` pub struct JoinHashMap { // Stores hash value to last row index pub map: RawTable<(u64, u64)>, From e305bcf197509dfb5c40d392cafad28d79effe08 Mon Sep 17 00:00:00 2001 From: Andrew Lamb Date: Fri, 10 Nov 2023 16:04:00 -0500 Subject: [PATCH 220/572] Simplify ProjectionPushdown and make it more general (#8109) MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit * Simply expression rewrite in ProjectionPushdown, make more general * Do not use partial rewrites * Apply suggestions from code review Co-authored-by: Berkay Şahin <124376117+berkaysynnada@users.noreply.github.com> Co-authored-by: Mehmet Ozan Kabak * cargo fmt, update comments --------- Co-authored-by: Berkay Şahin <124376117+berkaysynnada@users.noreply.github.com> Co-authored-by: Mehmet Ozan Kabak --- datafusion/common/src/tree_node.rs | 13 ++ .../physical_optimizer/projection_pushdown.rs | 162 ++++++------------ 2 files changed, 62 insertions(+), 113 deletions(-) diff --git a/datafusion/common/src/tree_node.rs b/datafusion/common/src/tree_node.rs index d0ef507294cc..5da9636ffe18 100644 --- a/datafusion/common/src/tree_node.rs +++ b/datafusion/common/src/tree_node.rs @@ -149,6 +149,19 @@ pub trait TreeNode: Sized { Ok(new_node) } + /// Convenience utils for writing optimizers rule: recursively apply the given 'op' first to all of its + /// children and then itself(Postorder Traversal) using a mutable function, `F`. + /// When the `op` does not apply to a given node, it is left unchanged. + fn transform_up_mut(self, op: &mut F) -> Result + where + F: FnMut(Self) -> Result>, + { + let after_op_children = self.map_children(|node| node.transform_up_mut(op))?; + + let new_node = op(after_op_children)?.into(); + Ok(new_node) + } + /// Transform the tree node using the given [TreeNodeRewriter] /// It performs a depth first walk of an node and its children. /// diff --git a/datafusion/core/src/physical_optimizer/projection_pushdown.rs b/datafusion/core/src/physical_optimizer/projection_pushdown.rs index 18495955612f..8e50492ae5e5 100644 --- a/datafusion/core/src/physical_optimizer/projection_pushdown.rs +++ b/datafusion/core/src/physical_optimizer/projection_pushdown.rs @@ -43,12 +43,9 @@ use arrow_schema::SchemaRef; use datafusion_common::config::ConfigOptions; use datafusion_common::tree_node::{Transformed, TreeNode}; use datafusion_common::JoinSide; -use datafusion_physical_expr::expressions::{ - BinaryExpr, CaseExpr, CastExpr, Column, Literal, NegativeExpr, -}; +use datafusion_physical_expr::expressions::Column; use datafusion_physical_expr::{ Partitioning, PhysicalExpr, PhysicalSortExpr, PhysicalSortRequirement, - ScalarFunctionExpr, }; use datafusion_physical_plan::union::UnionExec; @@ -791,119 +788,58 @@ fn update_expr( projected_exprs: &[(Arc, String)], sync_with_child: bool, ) -> Result>> { - let expr_any = expr.as_any(); - if let Some(column) = expr_any.downcast_ref::() { - if sync_with_child { - // Update the index of `column`: - Ok(Some(projected_exprs[column.index()].0.clone())) - } else { - // Determine how to update `column` to accommodate `projected_exprs`: - Ok(projected_exprs.iter().enumerate().find_map( - |(index, (projected_expr, alias))| { - projected_expr.as_any().downcast_ref::().and_then( - |projected_column| { - column - .name() - .eq(projected_column.name()) - .then(|| Arc::new(Column::new(alias, index)) as _) - }, - ) - }, - )) - } - } else if let Some(binary) = expr_any.downcast_ref::() { - match ( - update_expr(binary.left(), projected_exprs, sync_with_child)?, - update_expr(binary.right(), projected_exprs, sync_with_child)?, - ) { - (Some(left), Some(right)) => { - Ok(Some(Arc::new(BinaryExpr::new(left, *binary.op(), right)))) - } - _ => Ok(None), - } - } else if let Some(cast) = expr_any.downcast_ref::() { - update_expr(cast.expr(), projected_exprs, sync_with_child).map(|maybe_expr| { - maybe_expr.map(|expr| { - Arc::new(CastExpr::new( - expr, - cast.cast_type().clone(), - Some(cast.cast_options().clone()), - )) as _ - }) - }) - } else if expr_any.is::() { - Ok(Some(expr.clone())) - } else if let Some(negative) = expr_any.downcast_ref::() { - update_expr(negative.arg(), projected_exprs, sync_with_child).map(|maybe_expr| { - maybe_expr.map(|expr| Arc::new(NegativeExpr::new(expr)) as _) - }) - } else if let Some(scalar_func) = expr_any.downcast_ref::() { - scalar_func - .args() - .iter() - .map(|expr| update_expr(expr, projected_exprs, sync_with_child)) - .collect::>>>() - .map(|maybe_args| { - maybe_args.map(|new_args| { - Arc::new(ScalarFunctionExpr::new( - scalar_func.name(), - scalar_func.fun().clone(), - new_args, - scalar_func.return_type(), - scalar_func.monotonicity().clone(), - )) as _ - }) - }) - } else if let Some(case) = expr_any.downcast_ref::() { - update_case_expr(case, projected_exprs, sync_with_child) - } else { - Ok(None) + #[derive(Debug, PartialEq)] + enum RewriteState { + /// The expression is unchanged. + Unchanged, + /// Some part of the expression has been rewritten + RewrittenValid, + /// Some part of the expression has been rewritten, but some column + /// references could not be. + RewrittenInvalid, } -} -/// Updates the indices `case` refers to according to `projected_exprs`. -fn update_case_expr( - case: &CaseExpr, - projected_exprs: &[(Arc, String)], - sync_with_child: bool, -) -> Result>> { - let new_case = case - .expr() - .map(|expr| update_expr(expr, projected_exprs, sync_with_child)) - .transpose()? - .flatten(); - - let new_else = case - .else_expr() - .map(|expr| update_expr(expr, projected_exprs, sync_with_child)) - .transpose()? - .flatten(); - - let new_when_then = case - .when_then_expr() - .iter() - .map(|(when, then)| { - Ok(( - update_expr(when, projected_exprs, sync_with_child)?, - update_expr(then, projected_exprs, sync_with_child)?, - )) - }) - .collect::>>()? - .into_iter() - .filter_map(|(maybe_when, maybe_then)| match (maybe_when, maybe_then) { - (Some(when), Some(then)) => Some((when, then)), - _ => None, - }) - .collect::>(); + let mut state = RewriteState::Unchanged; - if new_when_then.len() != case.when_then_expr().len() - || case.expr().is_some() && new_case.is_none() - || case.else_expr().is_some() && new_else.is_none() - { - return Ok(None); - } + let new_expr = expr + .clone() + .transform_up_mut(&mut |expr: Arc| { + if state == RewriteState::RewrittenInvalid { + return Ok(Transformed::No(expr)); + } + + let Some(column) = expr.as_any().downcast_ref::() else { + return Ok(Transformed::No(expr)); + }; + if sync_with_child { + state = RewriteState::RewrittenValid; + // Update the index of `column`: + Ok(Transformed::Yes(projected_exprs[column.index()].0.clone())) + } else { + // default to invalid, in case we can't find the relevant column + state = RewriteState::RewrittenInvalid; + // Determine how to update `column` to accommodate `projected_exprs` + projected_exprs + .iter() + .enumerate() + .find_map(|(index, (projected_expr, alias))| { + projected_expr.as_any().downcast_ref::().and_then( + |projected_column| { + column.name().eq(projected_column.name()).then(|| { + state = RewriteState::RewrittenValid; + Arc::new(Column::new(alias, index)) as _ + }) + }, + ) + }) + .map_or_else( + || Ok(Transformed::No(expr)), + |c| Ok(Transformed::Yes(c)), + ) + } + }); - CaseExpr::try_new(new_case, new_when_then, new_else).map(|e| Some(Arc::new(e) as _)) + new_expr.map(|e| (state == RewriteState::RewrittenValid).then_some(e)) } /// Creates a new [`ProjectionExec`] instance with the given child plan and From fdf3f6c3304956cd56131d8783d7cb38a2242a9f Mon Sep 17 00:00:00 2001 From: Alex Huang Date: Fri, 10 Nov 2023 22:04:41 +0100 Subject: [PATCH 221/572] Minor: clean up the code regarding clippy (#8122) --- .../core/src/physical_optimizer/limited_distinct_aggregation.rs | 2 +- datafusion/physical-plan/src/limit.rs | 2 +- datafusion/sql/src/statement.rs | 2 +- 3 files changed, 3 insertions(+), 3 deletions(-) diff --git a/datafusion/core/src/physical_optimizer/limited_distinct_aggregation.rs b/datafusion/core/src/physical_optimizer/limited_distinct_aggregation.rs index 832a92bb69c6..8f5dbc2e9214 100644 --- a/datafusion/core/src/physical_optimizer/limited_distinct_aggregation.rs +++ b/datafusion/core/src/physical_optimizer/limited_distinct_aggregation.rs @@ -57,7 +57,7 @@ impl LimitedDistinctAggregation { aggr.filter_expr().to_vec(), aggr.order_by_expr().to_vec(), aggr.input().clone(), - aggr.input_schema().clone(), + aggr.input_schema(), ) .expect("Unable to copy Aggregate!") .with_limit(Some(limit)); diff --git a/datafusion/physical-plan/src/limit.rs b/datafusion/physical-plan/src/limit.rs index c8427f9bc2c6..355561c36f35 100644 --- a/datafusion/physical-plan/src/limit.rs +++ b/datafusion/physical-plan/src/limit.rs @@ -229,7 +229,7 @@ impl ExecutionPlan for GlobalLimitExec { let remaining_rows: usize = nr - skip; let mut skip_some_rows_stats = Statistics { num_rows: Precision::Exact(remaining_rows), - column_statistics: col_stats.clone(), + column_statistics: col_stats, total_byte_size: Precision::Absent, }; if !input_stats.num_rows.is_exact().unwrap_or(false) { diff --git a/datafusion/sql/src/statement.rs b/datafusion/sql/src/statement.rs index 116624c6f7b9..ecc77b044223 100644 --- a/datafusion/sql/src/statement.rs +++ b/datafusion/sql/src/statement.rs @@ -1017,7 +1017,7 @@ impl<'a, S: ContextProvider> SqlToRel<'a, S> { expr_to_columns(&filter_expr, &mut using_columns)?; let filter_expr = normalize_col_with_schemas_and_ambiguity_check( filter_expr, - &[&[&scan.schema()]], + &[&[scan.schema()]], &[using_columns], )?; LogicalPlan::Filter(Filter::try_new(filter_expr, Arc::new(scan))?) From 7fde76e33dcc26b0816fc8513c396becd431c1ad Mon Sep 17 00:00:00 2001 From: Jacob Ogle <123908271+JacobOgle@users.noreply.github.com> Date: Fri, 10 Nov 2023 16:06:32 -0500 Subject: [PATCH 222/572] Support remaining functions in protobuf serialization, add `expr_fn` for `StructFunction` (#8100) * working fix for #8098 * Added enum match for StringToArray * Added enum match for StructFun as well as mapping to a supporting scalar function * cargo fmt --------- Co-authored-by: Jacob Ogle --- datafusion/expr/src/expr_fn.rs | 7 +++++ .../proto/src/logical_plan/from_proto.rs | 29 ++++++++++++++----- 2 files changed, 28 insertions(+), 8 deletions(-) diff --git a/datafusion/expr/src/expr_fn.rs b/datafusion/expr/src/expr_fn.rs index 5a60c2470c95..5b1050020755 100644 --- a/datafusion/expr/src/expr_fn.rs +++ b/datafusion/expr/src/expr_fn.rs @@ -871,6 +871,13 @@ scalar_expr!( scalar_expr!(ArrowTypeof, arrow_typeof, val, "data type"); +scalar_expr!( + Struct, + struct_fun, + val, + "returns a vector of fields from the struct" +); + /// Create a CASE WHEN statement with literal WHEN expressions for comparison to the base expression. pub fn case(expr: Expr) -> CaseBuilder { CaseBuilder::new(Some(Box::new(expr)), vec![], vec![], None) diff --git a/datafusion/proto/src/logical_plan/from_proto.rs b/datafusion/proto/src/logical_plan/from_proto.rs index cdb0fe9bda7f..a3dcbc3fc80a 100644 --- a/datafusion/proto/src/logical_plan/from_proto.rs +++ b/datafusion/proto/src/logical_plan/from_proto.rs @@ -43,19 +43,20 @@ use datafusion_expr::{ array_has, array_has_all, array_has_any, array_length, array_ndims, array_position, array_positions, array_prepend, array_remove, array_remove_all, array_remove_n, array_repeat, array_replace, array_replace_all, array_replace_n, array_slice, - array_to_string, ascii, asin, asinh, atan, atan2, atanh, bit_length, btrim, - cardinality, cbrt, ceil, character_length, chr, coalesce, concat_expr, + array_to_string, arrow_typeof, ascii, asin, asinh, atan, atan2, atanh, bit_length, + btrim, cardinality, cbrt, ceil, character_length, chr, coalesce, concat_expr, concat_ws_expr, cos, cosh, cot, current_date, current_time, date_bin, date_part, date_trunc, decode, degrees, digest, encode, exp, expr::{self, InList, Sort, WindowFunction}, - factorial, floor, from_unixtime, gcd, isnan, iszero, lcm, left, ln, log, log10, log2, + factorial, flatten, floor, from_unixtime, gcd, isnan, iszero, lcm, left, ln, log, + log10, log2, logical_plan::{PlanType, StringifiedPlan}, lower, lpad, ltrim, md5, nanvl, now, nullif, octet_length, pi, power, radians, random, regexp_match, regexp_replace, repeat, replace, reverse, right, round, rpad, rtrim, sha224, sha256, sha384, sha512, signum, sin, sinh, split_part, sqrt, - starts_with, strpos, substr, substring, tan, tanh, to_hex, to_timestamp_micros, - to_timestamp_millis, to_timestamp_nanos, to_timestamp_seconds, translate, trim, - trunc, upper, uuid, + starts_with, string_to_array, strpos, struct_fun, substr, substring, tan, tanh, + to_hex, to_timestamp_micros, to_timestamp_millis, to_timestamp_nanos, + to_timestamp_seconds, translate, trim, trunc, upper, uuid, window_frame::regularize, AggregateFunction, Between, BinaryExpr, BuiltInWindowFunction, BuiltinScalarFunction, Case, Cast, Expr, GetFieldAccess, GetIndexedField, GroupingSet, @@ -1645,9 +1646,21 @@ pub fn parse_expr( )), ScalarFunction::Isnan => Ok(isnan(parse_expr(&args[0], registry)?)), ScalarFunction::Iszero => Ok(iszero(parse_expr(&args[0], registry)?)), - _ => Err(proto_error( - "Protobuf deserialization error: Unsupported scalar function", + ScalarFunction::ArrowTypeof => { + Ok(arrow_typeof(parse_expr(&args[0], registry)?)) + } + ScalarFunction::ToTimestamp => { + Ok(to_timestamp_seconds(parse_expr(&args[0], registry)?)) + } + ScalarFunction::Flatten => Ok(flatten(parse_expr(&args[0], registry)?)), + ScalarFunction::StringToArray => Ok(string_to_array( + parse_expr(&args[0], registry)?, + parse_expr(&args[1], registry)?, + parse_expr(&args[2], registry)?, )), + ScalarFunction::StructFun => { + Ok(struct_fun(parse_expr(&args[0], registry)?)) + } } } ExprType::ScalarUdfExpr(protobuf::ScalarUdfExprNode { fun_name, args }) => { From e727bbf370802b8066d3b07e3001f936cfad0383 Mon Sep 17 00:00:00 2001 From: Yongting You <2010youy01@gmail.com> Date: Fri, 10 Nov 2023 13:31:27 -0800 Subject: [PATCH 223/572] cleanup scalar function impl (#8114) --- .../physical-expr/src/datetime_expressions.rs | 150 ++++++++++++++ datafusion/physical-expr/src/functions.rs | 184 ++++-------------- .../physical-expr/src/math_expressions.rs | 12 ++ 3 files changed, 200 insertions(+), 146 deletions(-) diff --git a/datafusion/physical-expr/src/datetime_expressions.rs b/datafusion/physical-expr/src/datetime_expressions.rs index bb8720cb8d00..3b61e7f48d59 100644 --- a/datafusion/physical-expr/src/datetime_expressions.rs +++ b/datafusion/physical-expr/src/datetime_expressions.rs @@ -17,6 +17,8 @@ //! DateTime expressions +use crate::datetime_expressions; +use crate::expressions::cast_column; use arrow::array::Float64Builder; use arrow::compute::cast; use arrow::{ @@ -954,6 +956,154 @@ where Ok(b.finish()) } +/// to_timestammp() SQL function implementation +pub fn to_timestamp_invoke(args: &[ColumnarValue]) -> Result { + if args.len() != 1 { + return internal_err!( + "to_timestamp function requires 1 arguments, got {}", + args.len() + ); + } + + match args[0].data_type() { + DataType::Int64 => { + cast_column(&args[0], &DataType::Timestamp(TimeUnit::Second, None), None) + } + DataType::Timestamp(_, None) => cast_column( + &args[0], + &DataType::Timestamp(TimeUnit::Nanosecond, None), + None, + ), + DataType::Utf8 => datetime_expressions::to_timestamp(args), + other => { + internal_err!( + "Unsupported data type {:?} for function to_timestamp", + other + ) + } + } +} + +/// to_timestamp_millis() SQL function implementation +pub fn to_timestamp_millis_invoke(args: &[ColumnarValue]) -> Result { + if args.len() != 1 { + return internal_err!( + "to_timestamp_millis function requires 1 argument, got {}", + args.len() + ); + } + + match args[0].data_type() { + DataType::Int64 | DataType::Timestamp(_, None) => cast_column( + &args[0], + &DataType::Timestamp(TimeUnit::Millisecond, None), + None, + ), + DataType::Utf8 => datetime_expressions::to_timestamp_millis(args), + other => { + internal_err!( + "Unsupported data type {:?} for function to_timestamp_millis", + other + ) + } + } +} + +/// to_timestamp_micros() SQL function implementation +pub fn to_timestamp_micros_invoke(args: &[ColumnarValue]) -> Result { + if args.len() != 1 { + return internal_err!( + "to_timestamp_micros function requires 1 argument, got {}", + args.len() + ); + } + + match args[0].data_type() { + DataType::Int64 | DataType::Timestamp(_, None) => cast_column( + &args[0], + &DataType::Timestamp(TimeUnit::Microsecond, None), + None, + ), + DataType::Utf8 => datetime_expressions::to_timestamp_micros(args), + other => { + internal_err!( + "Unsupported data type {:?} for function to_timestamp_micros", + other + ) + } + } +} + +/// to_timestamp_nanos() SQL function implementation +pub fn to_timestamp_nanos_invoke(args: &[ColumnarValue]) -> Result { + if args.len() != 1 { + return internal_err!( + "to_timestamp_nanos function requires 1 argument, got {}", + args.len() + ); + } + + match args[0].data_type() { + DataType::Int64 | DataType::Timestamp(_, None) => cast_column( + &args[0], + &DataType::Timestamp(TimeUnit::Nanosecond, None), + None, + ), + DataType::Utf8 => datetime_expressions::to_timestamp_nanos(args), + other => { + internal_err!( + "Unsupported data type {:?} for function to_timestamp_nanos", + other + ) + } + } +} + +/// to_timestamp_seconds() SQL function implementation +pub fn to_timestamp_seconds_invoke(args: &[ColumnarValue]) -> Result { + if args.len() != 1 { + return internal_err!( + "to_timestamp_seconds function requires 1 argument, got {}", + args.len() + ); + } + + match args[0].data_type() { + DataType::Int64 | DataType::Timestamp(_, None) => { + cast_column(&args[0], &DataType::Timestamp(TimeUnit::Second, None), None) + } + DataType::Utf8 => datetime_expressions::to_timestamp_seconds(args), + other => { + internal_err!( + "Unsupported data type {:?} for function to_timestamp_seconds", + other + ) + } + } +} + +/// from_unixtime() SQL function implementation +pub fn from_unixtime_invoke(args: &[ColumnarValue]) -> Result { + if args.len() != 1 { + return internal_err!( + "from_unixtime function requires 1 argument, got {}", + args.len() + ); + } + + match args[0].data_type() { + DataType::Int64 => { + cast_column(&args[0], &DataType::Timestamp(TimeUnit::Second, None), None) + } + other => { + internal_err!( + "Unsupported data type {:?} for function from_unixtime", + other + ) + } + } +} + #[cfg(test)] mod tests { use std::sync::Arc; diff --git a/datafusion/physical-expr/src/functions.rs b/datafusion/physical-expr/src/functions.rs index f14bad093ac7..088bac100978 100644 --- a/datafusion/physical-expr/src/functions.rs +++ b/datafusion/physical-expr/src/functions.rs @@ -34,14 +34,12 @@ use crate::execution_props::ExecutionProps; use crate::sort_properties::SortProperties; use crate::{ array_expressions, conditional_expressions, datetime_expressions, - expressions::{cast_column, nullif_func}, - math_expressions, string_expressions, struct_expressions, PhysicalExpr, - ScalarFunctionExpr, + expressions::nullif_func, math_expressions, string_expressions, struct_expressions, + PhysicalExpr, ScalarFunctionExpr, }; use arrow::{ array::ArrayRef, compute::kernels::length::{bit_length, length}, - datatypes::TimeUnit, datatypes::{DataType, Int32Type, Int64Type, Schema}, }; use datafusion_common::{internal_err, DataFusionError, Result, ScalarValue}; @@ -71,143 +69,8 @@ pub fn create_physical_expr( let data_type = fun.return_type(&input_expr_types)?; - let fun_expr: ScalarFunctionImplementation = match fun { - // These functions need args and input schema to pick an implementation - // Unlike the string functions, which actually figure out the function to use with each array, - // here we return either a cast fn or string timestamp translation based on the expression data type - // so we don't have to pay a per-array/batch cost. - BuiltinScalarFunction::ToTimestamp => { - Arc::new(match input_phy_exprs[0].data_type(input_schema) { - Ok(DataType::Int64) => |col_values: &[ColumnarValue]| { - cast_column( - &col_values[0], - &DataType::Timestamp(TimeUnit::Second, None), - None, - ) - }, - Ok(DataType::Timestamp(_, None)) => |col_values: &[ColumnarValue]| { - cast_column( - &col_values[0], - &DataType::Timestamp(TimeUnit::Nanosecond, None), - None, - ) - }, - Ok(DataType::Utf8) => datetime_expressions::to_timestamp, - other => { - return internal_err!( - "Unsupported data type {other:?} for function to_timestamp" - ); - } - }) - } - BuiltinScalarFunction::ToTimestampMillis => { - Arc::new(match input_phy_exprs[0].data_type(input_schema) { - Ok(DataType::Int64) | Ok(DataType::Timestamp(_, None)) => { - |col_values: &[ColumnarValue]| { - cast_column( - &col_values[0], - &DataType::Timestamp(TimeUnit::Millisecond, None), - None, - ) - } - } - Ok(DataType::Utf8) => datetime_expressions::to_timestamp_millis, - other => { - return internal_err!( - "Unsupported data type {other:?} for function to_timestamp_millis" - ); - } - }) - } - BuiltinScalarFunction::ToTimestampMicros => { - Arc::new(match input_phy_exprs[0].data_type(input_schema) { - Ok(DataType::Int64) | Ok(DataType::Timestamp(_, None)) => { - |col_values: &[ColumnarValue]| { - cast_column( - &col_values[0], - &DataType::Timestamp(TimeUnit::Microsecond, None), - None, - ) - } - } - Ok(DataType::Utf8) => datetime_expressions::to_timestamp_micros, - other => { - return internal_err!( - "Unsupported data type {other:?} for function to_timestamp_micros" - ); - } - }) - } - BuiltinScalarFunction::ToTimestampNanos => { - Arc::new(match input_phy_exprs[0].data_type(input_schema) { - Ok(DataType::Int64) | Ok(DataType::Timestamp(_, None)) => { - |col_values: &[ColumnarValue]| { - cast_column( - &col_values[0], - &DataType::Timestamp(TimeUnit::Nanosecond, None), - None, - ) - } - } - Ok(DataType::Utf8) => datetime_expressions::to_timestamp_nanos, - other => { - return internal_err!( - "Unsupported data type {other:?} for function to_timestamp_nanos" - ); - } - }) - } - BuiltinScalarFunction::ToTimestampSeconds => Arc::new({ - match input_phy_exprs[0].data_type(input_schema) { - Ok(DataType::Int64) | Ok(DataType::Timestamp(_, None)) => { - |col_values: &[ColumnarValue]| { - cast_column( - &col_values[0], - &DataType::Timestamp(TimeUnit::Second, None), - None, - ) - } - } - Ok(DataType::Utf8) => datetime_expressions::to_timestamp_seconds, - other => { - return internal_err!( - "Unsupported data type {other:?} for function to_timestamp_seconds" - ); - } - } - }), - BuiltinScalarFunction::FromUnixtime => Arc::new({ - match input_phy_exprs[0].data_type(input_schema) { - Ok(DataType::Int64) => |col_values: &[ColumnarValue]| { - cast_column( - &col_values[0], - &DataType::Timestamp(TimeUnit::Second, None), - None, - ) - }, - other => { - return internal_err!( - "Unsupported data type {other:?} for function from_unixtime" - ); - } - } - }), - BuiltinScalarFunction::ArrowTypeof => { - let input_data_type = input_phy_exprs[0].data_type(input_schema)?; - Arc::new(move |_| { - Ok(ColumnarValue::Scalar(ScalarValue::Utf8(Some(format!( - "{input_data_type}" - ))))) - }) - } - BuiltinScalarFunction::Abs => { - let input_data_type = input_phy_exprs[0].data_type(input_schema)?; - let abs_fun = math_expressions::create_abs_function(&input_data_type)?; - Arc::new(move |args| make_scalar_function(abs_fun)(args)) - } - // These don't need args and input schema - _ => create_physical_fun(fun, execution_props)?, - }; + let fun_expr: ScalarFunctionImplementation = + create_physical_fun(fun, execution_props)?; let monotonicity = fun.monotonicity(); @@ -397,6 +260,9 @@ pub fn create_physical_fun( ) -> Result { Ok(match fun { // math functions + BuiltinScalarFunction::Abs => { + Arc::new(|args| make_scalar_function(math_expressions::abs_invoke)(args)) + } BuiltinScalarFunction::Acos => Arc::new(math_expressions::acos), BuiltinScalarFunction::Asin => Arc::new(math_expressions::asin), BuiltinScalarFunction::Atan => Arc::new(math_expressions::atan), @@ -625,6 +491,24 @@ pub fn create_physical_fun( execution_props.query_execution_start_time, )) } + BuiltinScalarFunction::ToTimestamp => { + Arc::new(datetime_expressions::to_timestamp_invoke) + } + BuiltinScalarFunction::ToTimestampMillis => { + Arc::new(datetime_expressions::to_timestamp_millis_invoke) + } + BuiltinScalarFunction::ToTimestampMicros => { + Arc::new(datetime_expressions::to_timestamp_micros_invoke) + } + BuiltinScalarFunction::ToTimestampNanos => { + Arc::new(datetime_expressions::to_timestamp_nanos_invoke) + } + BuiltinScalarFunction::ToTimestampSeconds => { + Arc::new(datetime_expressions::to_timestamp_seconds_invoke) + } + BuiltinScalarFunction::FromUnixtime => { + Arc::new(datetime_expressions::from_unixtime_invoke) + } BuiltinScalarFunction::InitCap => Arc::new(|args| match args[0].data_type() { DataType::Utf8 => { make_scalar_function(string_expressions::initcap::)(args) @@ -927,11 +811,19 @@ pub fn create_physical_fun( }), BuiltinScalarFunction::Upper => Arc::new(string_expressions::upper), BuiltinScalarFunction::Uuid => Arc::new(string_expressions::uuid), - _ => { - return internal_err!( - "create_physical_fun: Unsupported scalar function {fun:?}" - ); - } + BuiltinScalarFunction::ArrowTypeof => Arc::new(move |args| { + if args.len() != 1 { + return internal_err!( + "arrow_typeof function requires 1 arguments, got {}", + args.len() + ); + } + + let input_data_type = args[0].data_type(); + Ok(ColumnarValue::Scalar(ScalarValue::Utf8(Some(format!( + "{input_data_type}" + ))))) + }), }) } diff --git a/datafusion/physical-expr/src/math_expressions.rs b/datafusion/physical-expr/src/math_expressions.rs index 96f611e2b7b4..0b7bc34014f9 100644 --- a/datafusion/physical-expr/src/math_expressions.rs +++ b/datafusion/physical-expr/src/math_expressions.rs @@ -743,6 +743,18 @@ pub(super) fn create_abs_function( } } +/// abs() SQL function implementation +pub fn abs_invoke(args: &[ArrayRef]) -> Result { + if args.len() != 1 { + return internal_err!("abs function requires 1 argument, got {}", args.len()); + } + + let input_data_type = args[0].data_type(); + let abs_fun = create_abs_function(input_data_type)?; + + abs_fun(args) +} + #[cfg(test)] mod tests { From 4e8777d43847aef3f602320d9b7c27ee872530e3 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E8=B0=AD=E5=B7=8D?= Date: Sat, 11 Nov 2023 06:48:30 +0800 Subject: [PATCH 224/572] rewrite `array_append/array_prepend` to remove deplicate codes (#8108) * rewrite `array_append/array_prepend` to remove deplicate codes Signed-off-by: veeupup * reimplemented array_append with MutableArrayData Signed-off-by: veeupup * reimplemented array_prepend with MutableArrayData Signed-off-by: veeupup --------- Signed-off-by: veeupup --- .../avro_to_arrow/arrow_array_reader.rs | 20 +- .../physical-expr/src/array_expressions.rs | 190 +++++++----------- 2 files changed, 86 insertions(+), 124 deletions(-) diff --git a/datafusion/core/src/datasource/avro_to_arrow/arrow_array_reader.rs b/datafusion/core/src/datasource/avro_to_arrow/arrow_array_reader.rs index fd91ea1cc538..855a8d0dbf40 100644 --- a/datafusion/core/src/datasource/avro_to_arrow/arrow_array_reader.rs +++ b/datafusion/core/src/datasource/avro_to_arrow/arrow_array_reader.rs @@ -1536,12 +1536,10 @@ mod test { .unwrap() .resolve(&schema) .unwrap(); - let r4 = apache_avro::to_value(serde_json::json!({ - "col1": null - })) - .unwrap() - .resolve(&schema) - .unwrap(); + let r4 = apache_avro::to_value(serde_json::json!({ "col1": null })) + .unwrap() + .resolve(&schema) + .unwrap(); let mut w = apache_avro::Writer::new(&schema, vec![]); w.append(r1).unwrap(); @@ -1600,12 +1598,10 @@ mod test { }"#, ) .unwrap(); - let r1 = apache_avro::to_value(serde_json::json!({ - "col1": null - })) - .unwrap() - .resolve(&schema) - .unwrap(); + let r1 = apache_avro::to_value(serde_json::json!({ "col1": null })) + .unwrap() + .resolve(&schema) + .unwrap(); let r2 = apache_avro::to_value(serde_json::json!({ "col1": { "col2": "hello" diff --git a/datafusion/physical-expr/src/array_expressions.rs b/datafusion/physical-expr/src/array_expressions.rs index deb4372baa32..1bc25f56104b 100644 --- a/datafusion/physical-expr/src/array_expressions.rs +++ b/datafusion/physical-expr/src/array_expressions.rs @@ -577,58 +577,6 @@ pub fn array_pop_back(args: &[ArrayRef]) -> Result { ) } -macro_rules! append { - ($ARRAY:expr, $ELEMENT:expr, $ARRAY_TYPE:ident) => {{ - let mut offsets: Vec = vec![0]; - let mut values = - downcast_arg!(new_empty_array($ELEMENT.data_type()), $ARRAY_TYPE).clone(); - - let element = downcast_arg!($ELEMENT, $ARRAY_TYPE); - for (arr, el) in $ARRAY.iter().zip(element.iter()) { - let last_offset: i32 = offsets.last().copied().ok_or_else(|| { - DataFusionError::Internal(format!("offsets should not be empty")) - })?; - match arr { - Some(arr) => { - let child_array = downcast_arg!(arr, $ARRAY_TYPE); - values = downcast_arg!( - compute::concat(&[ - &values, - child_array, - &$ARRAY_TYPE::from(vec![el]) - ])? - .clone(), - $ARRAY_TYPE - ) - .clone(); - offsets.push(last_offset + child_array.len() as i32 + 1i32); - } - None => { - values = downcast_arg!( - compute::concat(&[ - &values, - &$ARRAY_TYPE::from(vec![el.clone()]) - ])? - .clone(), - $ARRAY_TYPE - ) - .clone(); - offsets.push(last_offset + 1i32); - } - } - } - - let field = Arc::new(Field::new("item", $ELEMENT.data_type().clone(), true)); - - Arc::new(ListArray::try_new( - field, - OffsetBuffer::new(offsets.into()), - Arc::new(values), - None, - )?) - }}; -} - /// Array_append SQL function pub fn array_append(args: &[ArrayRef]) -> Result { let arr = as_list_array(&args[0])?; @@ -639,68 +587,51 @@ pub fn array_append(args: &[ArrayRef]) -> Result { DataType::List(_) => concat_internal(args)?, DataType::Null => return make_array(&[element.to_owned()]), data_type => { - macro_rules! array_function { - ($ARRAY_TYPE:ident) => { - append!(arr, element, $ARRAY_TYPE) + let mut new_values = vec![]; + let mut offsets = vec![0]; + + let elem_data = element.to_data(); + for (row_index, arr) in arr.iter().enumerate() { + let new_array = if let Some(arr) = arr { + let original_data = arr.to_data(); + let capacity = Capacities::Array(original_data.len() + 1); + let mut mutable = MutableArrayData::with_capacities( + vec![&original_data, &elem_data], + false, + capacity, + ); + mutable.extend(0, 0, original_data.len()); + mutable.extend(1, row_index, row_index + 1); + let data = mutable.freeze(); + arrow_array::make_array(data) + } else { + let capacity = Capacities::Array(1); + let mut mutable = MutableArrayData::with_capacities( + vec![&elem_data], + false, + capacity, + ); + mutable.extend(0, row_index, row_index + 1); + let data = mutable.freeze(); + arrow_array::make_array(data) }; + offsets.push(offsets[row_index] + new_array.len() as i32); + new_values.push(new_array); } - call_array_function!(data_type, false) - } - }; - - Ok(res) -} -macro_rules! prepend { - ($ARRAY:expr, $ELEMENT:expr, $ARRAY_TYPE:ident) => {{ - let mut offsets: Vec = vec![0]; - let mut values = - downcast_arg!(new_empty_array($ELEMENT.data_type()), $ARRAY_TYPE).clone(); + let new_values: Vec<_> = new_values.iter().map(|a| a.as_ref()).collect(); + let values = arrow::compute::concat(&new_values)?; - let element = downcast_arg!($ELEMENT, $ARRAY_TYPE); - for (arr, el) in $ARRAY.iter().zip(element.iter()) { - let last_offset: i32 = offsets.last().copied().ok_or_else(|| { - DataFusionError::Internal(format!("offsets should not be empty")) - })?; - match arr { - Some(arr) => { - let child_array = downcast_arg!(arr, $ARRAY_TYPE); - values = downcast_arg!( - compute::concat(&[ - &values, - &$ARRAY_TYPE::from(vec![el]), - child_array - ])? - .clone(), - $ARRAY_TYPE - ) - .clone(); - offsets.push(last_offset + child_array.len() as i32 + 1i32); - } - None => { - values = downcast_arg!( - compute::concat(&[ - &values, - &$ARRAY_TYPE::from(vec![el.clone()]) - ])? - .clone(), - $ARRAY_TYPE - ) - .clone(); - offsets.push(last_offset + 1i32); - } - } + Arc::new(ListArray::try_new( + Arc::new(Field::new("item", data_type.to_owned(), true)), + OffsetBuffer::new(offsets.into()), + values, + None, + )?) } + }; - let field = Arc::new(Field::new("item", $ELEMENT.data_type().clone(), true)); - - Arc::new(ListArray::try_new( - field, - OffsetBuffer::new(offsets.into()), - Arc::new(values), - None, - )?) - }}; + Ok(res) } /// Array_prepend SQL function @@ -713,12 +644,47 @@ pub fn array_prepend(args: &[ArrayRef]) -> Result { DataType::List(_) => concat_internal(args)?, DataType::Null => return make_array(&[element.to_owned()]), data_type => { - macro_rules! array_function { - ($ARRAY_TYPE:ident) => { - prepend!(arr, element, $ARRAY_TYPE) + let mut new_values = vec![]; + let mut offsets = vec![0]; + + let elem_data = element.to_data(); + for (row_index, arr) in arr.iter().enumerate() { + let new_array = if let Some(arr) = arr { + let original_data = arr.to_data(); + let capacity = Capacities::Array(original_data.len() + 1); + let mut mutable = MutableArrayData::with_capacities( + vec![&original_data, &elem_data], + false, + capacity, + ); + mutable.extend(1, row_index, row_index + 1); + mutable.extend(0, 0, original_data.len()); + let data = mutable.freeze(); + arrow_array::make_array(data) + } else { + let capacity = Capacities::Array(1); + let mut mutable = MutableArrayData::with_capacities( + vec![&elem_data], + false, + capacity, + ); + mutable.extend(0, row_index, row_index + 1); + let data = mutable.freeze(); + arrow_array::make_array(data) }; + offsets.push(offsets[row_index] + new_array.len() as i32); + new_values.push(new_array); } - call_array_function!(data_type, false) + + let new_values: Vec<_> = new_values.iter().map(|a| a.as_ref()).collect(); + let values = arrow::compute::concat(&new_values)?; + + Arc::new(ListArray::try_new( + Arc::new(Field::new("item", data_type.to_owned(), true)), + OffsetBuffer::new(offsets.into()), + values, + None, + )?) } }; From 8966dc005656e329dcf0cf6a47768240a4257c5e Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E8=B0=AD=E5=B7=8D?= Date: Sat, 11 Nov 2023 10:16:20 +0800 Subject: [PATCH 225/572] Implementation of `array_intersect` (#8081) * Initial Implementation of array_intersect Signed-off-by: veeupup * fix comments Signed-off-by: veeupup x --------- Signed-off-by: veeupup --- datafusion/expr/src/built_in_function.rs | 6 + datafusion/expr/src/expr_fn.rs | 6 + .../physical-expr/src/array_expressions.rs | 69 +++++++- datafusion/physical-expr/src/functions.rs | 3 + datafusion/proto/proto/datafusion.proto | 1 + datafusion/proto/src/generated/pbjson.rs | 3 + datafusion/proto/src/generated/prost.rs | 3 + .../proto/src/logical_plan/from_proto.rs | 1 + datafusion/proto/src/logical_plan/to_proto.rs | 1 + datafusion/sqllogictest/test_files/array.slt | 150 ++++++++++++++++++ docs/source/user-guide/expressions.md | 1 + 11 files changed, 238 insertions(+), 6 deletions(-) diff --git a/datafusion/expr/src/built_in_function.rs b/datafusion/expr/src/built_in_function.rs index f3f52e9dafb6..ca3ca18e4d77 100644 --- a/datafusion/expr/src/built_in_function.rs +++ b/datafusion/expr/src/built_in_function.rs @@ -174,6 +174,8 @@ pub enum BuiltinScalarFunction { ArraySlice, /// array_to_string ArrayToString, + /// array_intersect + ArrayIntersect, /// cardinality Cardinality, /// construct an array from columns @@ -398,6 +400,7 @@ impl BuiltinScalarFunction { BuiltinScalarFunction::Flatten => Volatility::Immutable, BuiltinScalarFunction::ArraySlice => Volatility::Immutable, BuiltinScalarFunction::ArrayToString => Volatility::Immutable, + BuiltinScalarFunction::ArrayIntersect => Volatility::Immutable, BuiltinScalarFunction::Cardinality => Volatility::Immutable, BuiltinScalarFunction::MakeArray => Volatility::Immutable, BuiltinScalarFunction::Ascii => Volatility::Immutable, @@ -577,6 +580,7 @@ impl BuiltinScalarFunction { BuiltinScalarFunction::ArrayReplaceAll => Ok(input_expr_types[0].clone()), BuiltinScalarFunction::ArraySlice => Ok(input_expr_types[0].clone()), BuiltinScalarFunction::ArrayToString => Ok(Utf8), + BuiltinScalarFunction::ArrayIntersect => Ok(input_expr_types[0].clone()), BuiltinScalarFunction::Cardinality => Ok(UInt64), BuiltinScalarFunction::MakeArray => match input_expr_types.len() { 0 => Ok(List(Arc::new(Field::new("item", Null, true)))), @@ -880,6 +884,7 @@ impl BuiltinScalarFunction { BuiltinScalarFunction::ArrayToString => { Signature::variadic_any(self.volatility()) } + BuiltinScalarFunction::ArrayIntersect => Signature::any(2, self.volatility()), BuiltinScalarFunction::Cardinality => Signature::any(1, self.volatility()), BuiltinScalarFunction::MakeArray => { // 0 or more arguments of arbitrary type @@ -1505,6 +1510,7 @@ fn aliases(func: &BuiltinScalarFunction) -> &'static [&'static str] { ], BuiltinScalarFunction::Cardinality => &["cardinality"], BuiltinScalarFunction::MakeArray => &["make_array", "make_list"], + BuiltinScalarFunction::ArrayIntersect => &["array_intersect", "list_intersect"], // struct functions BuiltinScalarFunction::Struct => &["struct"], diff --git a/datafusion/expr/src/expr_fn.rs b/datafusion/expr/src/expr_fn.rs index 5b1050020755..98cacc039228 100644 --- a/datafusion/expr/src/expr_fn.rs +++ b/datafusion/expr/src/expr_fn.rs @@ -715,6 +715,12 @@ nary_scalar_expr!( array, "returns an Arrow array using the specified input expressions." ); +scalar_expr!( + ArrayIntersect, + array_intersect, + first_array second_array, + "Returns an array of the elements in the intersection of array1 and array2." +); // string functions scalar_expr!(Ascii, ascii, chr, "ASCII code value of the character"); diff --git a/datafusion/physical-expr/src/array_expressions.rs b/datafusion/physical-expr/src/array_expressions.rs index 1bc25f56104b..87ba77b497b2 100644 --- a/datafusion/physical-expr/src/array_expressions.rs +++ b/datafusion/physical-expr/src/array_expressions.rs @@ -24,6 +24,7 @@ use arrow::array::*; use arrow::buffer::OffsetBuffer; use arrow::compute; use arrow::datatypes::{DataType, Field, UInt64Type}; +use arrow::row::{RowConverter, SortField}; use arrow_buffer::NullBuffer; use datafusion_common::cast::{ @@ -35,6 +36,7 @@ use datafusion_common::{ DataFusionError, Result, }; +use hashbrown::HashSet; use itertools::Itertools; macro_rules! downcast_arg { @@ -347,7 +349,7 @@ fn array_array(args: &[ArrayRef], data_type: DataType) -> Result { let data_type = arrays[0].data_type(); let field = Arc::new(Field::new("item", data_type.to_owned(), true)); let elements = arrays.iter().map(|x| x.as_ref()).collect::>(); - let values = arrow::compute::concat(elements.as_slice())?; + let values = compute::concat(elements.as_slice())?; let list_arr = ListArray::new( field, OffsetBuffer::from_lengths(array_lengths), @@ -368,7 +370,7 @@ fn array_array(args: &[ArrayRef], data_type: DataType) -> Result { .iter() .map(|x| x as &dyn Array) .collect::>(); - let values = arrow::compute::concat(elements.as_slice())?; + let values = compute::concat(elements.as_slice())?; let list_arr = ListArray::new( field, OffsetBuffer::from_lengths(list_array_lengths), @@ -767,7 +769,7 @@ fn concat_internal(args: &[ArrayRef]) -> Result { .collect::>(); // Concatenated array on i-th row - let concated_array = arrow::compute::concat(elements.as_slice())?; + let concated_array = compute::concat(elements.as_slice())?; array_lengths.push(concated_array.len()); arrays.push(concated_array); valid.append(true); @@ -785,7 +787,7 @@ fn concat_internal(args: &[ArrayRef]) -> Result { let list_arr = ListArray::new( Arc::new(Field::new("item", data_type, true)), OffsetBuffer::from_lengths(array_lengths), - Arc::new(arrow::compute::concat(elements.as_slice())?), + Arc::new(compute::concat(elements.as_slice())?), Some(NullBuffer::new(buffer)), ); Ok(Arc::new(list_arr)) @@ -879,7 +881,7 @@ fn general_repeat(array: &ArrayRef, count_array: &Int64Array) -> Result = new_values.iter().map(|a| a.as_ref()).collect(); - let values = arrow::compute::concat(&new_values)?; + let values = compute::concat(&new_values)?; Ok(Arc::new(ListArray::try_new( Arc::new(Field::new("item", data_type.to_owned(), true)), @@ -947,7 +949,7 @@ fn general_list_repeat( let lengths = new_values.iter().map(|a| a.len()).collect::>(); let new_values: Vec<_> = new_values.iter().map(|a| a.as_ref()).collect(); - let values = arrow::compute::concat(&new_values)?; + let values = compute::concat(&new_values)?; Ok(Arc::new(ListArray::try_new( Arc::new(Field::new("item", data_type.to_owned(), true)), @@ -1798,6 +1800,61 @@ pub fn string_to_array(args: &[ArrayRef]) -> Result Result { + assert_eq!(args.len(), 2); + + let first_array = as_list_array(&args[0])?; + let second_array = as_list_array(&args[1])?; + + if first_array.value_type() != second_array.value_type() { + return internal_err!("array_intersect is not implemented for '{first_array:?}' and '{second_array:?}'"); + } + let dt = first_array.value_type().clone(); + + let mut offsets = vec![0]; + let mut new_arrays = vec![]; + + let converter = RowConverter::new(vec![SortField::new(dt.clone())])?; + for (first_arr, second_arr) in first_array.iter().zip(second_array.iter()) { + if let (Some(first_arr), Some(second_arr)) = (first_arr, second_arr) { + let l_values = converter.convert_columns(&[first_arr])?; + let r_values = converter.convert_columns(&[second_arr])?; + + let values_set: HashSet<_> = l_values.iter().collect(); + let mut rows = Vec::with_capacity(r_values.num_rows()); + for r_val in r_values.iter().sorted().dedup() { + if values_set.contains(&r_val) { + rows.push(r_val); + } + } + + let last_offset: i32 = match offsets.last().copied() { + Some(offset) => offset, + None => return internal_err!("offsets should not be empty"), + }; + offsets.push(last_offset + rows.len() as i32); + let arrays = converter.convert_rows(rows)?; + let array = match arrays.get(0) { + Some(array) => array.clone(), + None => { + return internal_err!( + "array_intersect: failed to get array from rows" + ) + } + }; + new_arrays.push(array); + } + } + + let field = Arc::new(Field::new("item", dt, true)); + let offsets = OffsetBuffer::new(offsets.into()); + let new_arrays_ref = new_arrays.iter().map(|v| v.as_ref()).collect::>(); + let values = compute::concat(&new_arrays_ref)?; + let arr = Arc::new(ListArray::try_new(field, offsets, values, None)?); + Ok(arr) +} + #[cfg(test)] mod tests { use super::*; diff --git a/datafusion/physical-expr/src/functions.rs b/datafusion/physical-expr/src/functions.rs index 088bac100978..c973232c75a6 100644 --- a/datafusion/physical-expr/src/functions.rs +++ b/datafusion/physical-expr/src/functions.rs @@ -398,6 +398,9 @@ pub fn create_physical_fun( BuiltinScalarFunction::ArrayToString => Arc::new(|args| { make_scalar_function(array_expressions::array_to_string)(args) }), + BuiltinScalarFunction::ArrayIntersect => Arc::new(|args| { + make_scalar_function(array_expressions::array_intersect)(args) + }), BuiltinScalarFunction::Cardinality => { Arc::new(|args| make_scalar_function(array_expressions::cardinality)(args)) } diff --git a/datafusion/proto/proto/datafusion.proto b/datafusion/proto/proto/datafusion.proto index bc6de2348e8d..f9deca2f1e52 100644 --- a/datafusion/proto/proto/datafusion.proto +++ b/datafusion/proto/proto/datafusion.proto @@ -621,6 +621,7 @@ enum ScalarFunction { ArrayPopBack = 116; StringToArray = 117; ToTimestampNanos = 118; + ArrayIntersect = 119; } message ScalarFunctionNode { diff --git a/datafusion/proto/src/generated/pbjson.rs b/datafusion/proto/src/generated/pbjson.rs index 659a25f9fa35..81f260c28bed 100644 --- a/datafusion/proto/src/generated/pbjson.rs +++ b/datafusion/proto/src/generated/pbjson.rs @@ -20730,6 +20730,7 @@ impl serde::Serialize for ScalarFunction { Self::ArrayPopBack => "ArrayPopBack", Self::StringToArray => "StringToArray", Self::ToTimestampNanos => "ToTimestampNanos", + Self::ArrayIntersect => "ArrayIntersect", }; serializer.serialize_str(variant) } @@ -20860,6 +20861,7 @@ impl<'de> serde::Deserialize<'de> for ScalarFunction { "ArrayPopBack", "StringToArray", "ToTimestampNanos", + "ArrayIntersect", ]; struct GeneratedVisitor; @@ -21019,6 +21021,7 @@ impl<'de> serde::Deserialize<'de> for ScalarFunction { "ArrayPopBack" => Ok(ScalarFunction::ArrayPopBack), "StringToArray" => Ok(ScalarFunction::StringToArray), "ToTimestampNanos" => Ok(ScalarFunction::ToTimestampNanos), + "ArrayIntersect" => Ok(ScalarFunction::ArrayIntersect), _ => Err(serde::de::Error::unknown_variant(value, FIELDS)), } } diff --git a/datafusion/proto/src/generated/prost.rs b/datafusion/proto/src/generated/prost.rs index 75050e9d3dfa..ae64c11b3b74 100644 --- a/datafusion/proto/src/generated/prost.rs +++ b/datafusion/proto/src/generated/prost.rs @@ -2539,6 +2539,7 @@ pub enum ScalarFunction { ArrayPopBack = 116, StringToArray = 117, ToTimestampNanos = 118, + ArrayIntersect = 119, } impl ScalarFunction { /// String value of the enum field names used in the ProtoBuf definition. @@ -2666,6 +2667,7 @@ impl ScalarFunction { ScalarFunction::ArrayPopBack => "ArrayPopBack", ScalarFunction::StringToArray => "StringToArray", ScalarFunction::ToTimestampNanos => "ToTimestampNanos", + ScalarFunction::ArrayIntersect => "ArrayIntersect", } } /// Creates an enum from field names used in the ProtoBuf definition. @@ -2790,6 +2792,7 @@ impl ScalarFunction { "ArrayPopBack" => Some(Self::ArrayPopBack), "StringToArray" => Some(Self::StringToArray), "ToTimestampNanos" => Some(Self::ToTimestampNanos), + "ArrayIntersect" => Some(Self::ArrayIntersect), _ => None, } } diff --git a/datafusion/proto/src/logical_plan/from_proto.rs b/datafusion/proto/src/logical_plan/from_proto.rs index a3dcbc3fc80a..e5bcc934036d 100644 --- a/datafusion/proto/src/logical_plan/from_proto.rs +++ b/datafusion/proto/src/logical_plan/from_proto.rs @@ -483,6 +483,7 @@ impl From<&protobuf::ScalarFunction> for BuiltinScalarFunction { ScalarFunction::ArrayReplaceAll => Self::ArrayReplaceAll, ScalarFunction::ArraySlice => Self::ArraySlice, ScalarFunction::ArrayToString => Self::ArrayToString, + ScalarFunction::ArrayIntersect => Self::ArrayIntersect, ScalarFunction::Cardinality => Self::Cardinality, ScalarFunction::Array => Self::MakeArray, ScalarFunction::NullIf => Self::NullIf, diff --git a/datafusion/proto/src/logical_plan/to_proto.rs b/datafusion/proto/src/logical_plan/to_proto.rs index 687b73cfc886..803becbcaece 100644 --- a/datafusion/proto/src/logical_plan/to_proto.rs +++ b/datafusion/proto/src/logical_plan/to_proto.rs @@ -1481,6 +1481,7 @@ impl TryFrom<&BuiltinScalarFunction> for protobuf::ScalarFunction { BuiltinScalarFunction::ArrayReplaceAll => Self::ArrayReplaceAll, BuiltinScalarFunction::ArraySlice => Self::ArraySlice, BuiltinScalarFunction::ArrayToString => Self::ArrayToString, + BuiltinScalarFunction::ArrayIntersect => Self::ArrayIntersect, BuiltinScalarFunction::Cardinality => Self::Cardinality, BuiltinScalarFunction::MakeArray => Self::Array, BuiltinScalarFunction::NullIf => Self::NullIf, diff --git a/datafusion/sqllogictest/test_files/array.slt b/datafusion/sqllogictest/test_files/array.slt index c57369c167f4..f83ed5a95ff3 100644 --- a/datafusion/sqllogictest/test_files/array.slt +++ b/datafusion/sqllogictest/test_files/array.slt @@ -182,6 +182,55 @@ AS VALUES (make_array([[1], [2]], [[2], [3]]), make_array([1], [2])) ; +statement ok +CREATE TABLE array_intersect_table_1D +AS VALUES + (make_array(1, 2), make_array(1), make_array(1,2,3), make_array(1,3), make_array(1,3,5), make_array(2,4,6,8,1,3)), + (make_array(11, 22), make_array(11), make_array(11,22,33), make_array(11,33), make_array(11,33,55), make_array(22,44,66,88,11,33)) +; + +statement ok +CREATE TABLE array_intersect_table_1D_Float +AS VALUES + (make_array(1.0, 2.0), make_array(1.0), make_array(1.0,2.0,3.0), make_array(1.0,3.0), make_array(1.11), make_array(2.22, 3.33)), + (make_array(3.0, 4.0, 5.0), make_array(2.0), make_array(1.0,2.0,3.0,4.0), make_array(2.0,5.0), make_array(2.22, 1.11), make_array(1.11, 3.33)) +; + +statement ok +CREATE TABLE array_intersect_table_1D_Boolean +AS VALUES + (make_array(true, true, true), make_array(false), make_array(true, true, false, true, false), make_array(true, false, true), make_array(false), make_array(true, false)), + (make_array(false, false, false), make_array(false), make_array(true, false, true), make_array(true, true), make_array(true, true), make_array(false,false,true)) +; + +statement ok +CREATE TABLE array_intersect_table_1D_UTF8 +AS VALUES + (make_array('a', 'bc', 'def'), make_array('bc'), make_array('datafusion', 'rust', 'arrow'), make_array('rust', 'arrow'), make_array('rust', 'arrow', 'python'), make_array('data')), + (make_array('a', 'bc', 'def'), make_array('defg'), make_array('datafusion', 'rust', 'arrow'), make_array('datafusion', 'rust', 'arrow', 'python'), make_array('rust', 'arrow'), make_array('datafusion', 'rust', 'arrow')) +; + +statement ok +CREATE TABLE array_intersect_table_2D +AS VALUES + (make_array([1,2]), make_array([1,3]), make_array([1,2,3], [4,5], [6,7]), make_array([4,5], [6,7])), + (make_array([3,4], [5]), make_array([3,4]), make_array([1,2,3,4], [5,6,7], [8,9,10]), make_array([1,2,3], [5,6,7], [8,9,10])) +; + +statement ok +CREATE TABLE array_intersect_table_2D_float +AS VALUES + (make_array([1.0, 2.0, 3.0], [1.1, 2.2], [3.3]), make_array([1.1, 2.2], [3.3])), + (make_array([1.0, 2.0, 3.0], [1.1, 2.2], [3.3]), make_array([1.0], [1.1, 2.2], [3.3])) +; + +statement ok +CREATE TABLE array_intersect_table_3D +AS VALUES + (make_array([[1,2]]), make_array([[1]])), + (make_array([[1,2]]), make_array([[1,2]])) +; + statement ok CREATE TABLE arrays_values_without_nulls AS VALUES @@ -2316,6 +2365,86 @@ select array_has_all(make_array(1,2,3), make_array(1,3)), ---- true false true false false false true true false false true false true +query ??? +select array_intersect(column1, column2), + array_intersect(column3, column4), + array_intersect(column5, column6) +from array_intersect_table_1D; +---- +[1] [1, 3] [1, 3] +[11] [11, 33] [11, 33] + +query ??? +select array_intersect(column1, column2), + array_intersect(column3, column4), + array_intersect(column5, column6) +from array_intersect_table_1D_Float; +---- +[1.0] [1.0, 3.0] [] +[] [2.0] [1.11] + +query ??? +select array_intersect(column1, column2), + array_intersect(column3, column4), + array_intersect(column5, column6) +from array_intersect_table_1D_Boolean; +---- +[] [false, true] [false] +[false] [true] [true] + +query ??? +select array_intersect(column1, column2), + array_intersect(column3, column4), + array_intersect(column5, column6) +from array_intersect_table_1D_UTF8; +---- +[bc] [arrow, rust] [] +[] [arrow, datafusion, rust] [arrow, rust] + +query ?? +select array_intersect(column1, column2), + array_intersect(column3, column4) +from array_intersect_table_2D; +---- +[] [[4, 5], [6, 7]] +[[3, 4]] [[5, 6, 7], [8, 9, 10]] + +query ? +select array_intersect(column1, column2) +from array_intersect_table_2D_float; +---- +[[1.1, 2.2], [3.3]] +[[1.1, 2.2], [3.3]] + +query ? +select array_intersect(column1, column2) +from array_intersect_table_3D; +---- +[] +[[[1, 2]]] + +query ?????? +SELECT array_intersect(make_array(1,2,3), make_array(2,3,4)), + array_intersect(make_array(1,3,5), make_array(2,4,6)), + array_intersect(make_array('aa','bb','cc'), make_array('cc','aa','dd')), + array_intersect(make_array(true, false), make_array(true)), + array_intersect(make_array(1.1, 2.2, 3.3), make_array(2.2, 3.3, 4.4)), + array_intersect(make_array([1, 1], [2, 2], [3, 3]), make_array([2, 2], [3, 3], [4, 4])) +; +---- +[2, 3] [] [aa, cc] [true] [2.2, 3.3] [[2, 2], [3, 3]] + +query ?????? +SELECT list_intersect(make_array(1,2,3), make_array(2,3,4)), + list_intersect(make_array(1,3,5), make_array(2,4,6)), + list_intersect(make_array('aa','bb','cc'), make_array('cc','aa','dd')), + list_intersect(make_array(true, false), make_array(true)), + list_intersect(make_array(1.1, 2.2, 3.3), make_array(2.2, 3.3, 4.4)), + list_intersect(make_array([1, 1], [2, 2], [3, 3]), make_array([2, 2], [3, 3], [4, 4])) +; +---- +[2, 3] [] [aa, cc] [true] [2.2, 3.3] [[2, 2], [3, 3]] + query BBBB select list_has_all(make_array(1,2,3), make_array(4,5,6)), list_has_all(make_array(1,2,3), make_array(1,2)), @@ -2608,6 +2737,27 @@ drop table array_has_table_2D_float; statement ok drop table array_has_table_3D; +statement ok +drop table array_intersect_table_1D; + +statement ok +drop table array_intersect_table_1D_Float; + +statement ok +drop table array_intersect_table_1D_Boolean; + +statement ok +drop table array_intersect_table_1D_UTF8; + +statement ok +drop table array_intersect_table_2D; + +statement ok +drop table array_intersect_table_2D_float; + +statement ok +drop table array_intersect_table_3D; + statement ok drop table arrays_values_without_nulls; diff --git a/docs/source/user-guide/expressions.md b/docs/source/user-guide/expressions.md index dbe12df33564..27384dccffe0 100644 --- a/docs/source/user-guide/expressions.md +++ b/docs/source/user-guide/expressions.md @@ -232,6 +232,7 @@ Unlike to some databases the math functions in Datafusion works the same way as | array_replace_all(array, from, to) | Replaces all occurrences of the specified element with another specified element. `array_replace_all([1, 2, 2, 3, 2, 1, 4], 2, 5) -> [1, 5, 5, 3, 5, 1, 4]` | | array_slice(array, index) | Returns a slice of the array. `array_slice([1, 2, 3, 4, 5, 6, 7, 8], 3, 6) -> [3, 4, 5, 6]` | | array_to_string(array, delimiter) | Converts each element to its text representation. `array_to_string([1, 2, 3, 4], ',') -> 1,2,3,4` | +| array_intersect(array1, array2) | Returns an array of the elements in the intersection of array1 and array2. `array_intersect([1, 2, 3, 4], [5, 6, 3, 4]) -> [3, 4]` | | cardinality(array) | Returns the total number of elements in the array. `cardinality([[1, 2, 3], [4, 5, 6]]) -> 6` | | make_array(value1, [value2 [, ...]]) | Returns an Arrow array using the specified input expressions. `make_array(1, 2, 3) -> [1, 2, 3]` | | trim_array(array, n) | Deprecated | From bd4b3f42e0cc685dcf599bd34596a7f37f0c0510 Mon Sep 17 00:00:00 2001 From: Huaijin Date: Sat, 11 Nov 2023 16:00:58 +0800 Subject: [PATCH 226/572] fix build by adding ScalarFunction::ArrayIntersect match (#8136) --- .../proto/src/logical_plan/from_proto.rs | 18 +++++++++++------- 1 file changed, 11 insertions(+), 7 deletions(-) diff --git a/datafusion/proto/src/logical_plan/from_proto.rs b/datafusion/proto/src/logical_plan/from_proto.rs index e5bcc934036d..31fffca3bbed 100644 --- a/datafusion/proto/src/logical_plan/from_proto.rs +++ b/datafusion/proto/src/logical_plan/from_proto.rs @@ -40,13 +40,13 @@ use datafusion_common::{ }; use datafusion_expr::{ abs, acos, acosh, array, array_append, array_concat, array_dims, array_element, - array_has, array_has_all, array_has_any, array_length, array_ndims, array_position, - array_positions, array_prepend, array_remove, array_remove_all, array_remove_n, - array_repeat, array_replace, array_replace_all, array_replace_n, array_slice, - array_to_string, arrow_typeof, ascii, asin, asinh, atan, atan2, atanh, bit_length, - btrim, cardinality, cbrt, ceil, character_length, chr, coalesce, concat_expr, - concat_ws_expr, cos, cosh, cot, current_date, current_time, date_bin, date_part, - date_trunc, decode, degrees, digest, encode, exp, + array_has, array_has_all, array_has_any, array_intersect, array_length, array_ndims, + array_position, array_positions, array_prepend, array_remove, array_remove_all, + array_remove_n, array_repeat, array_replace, array_replace_all, array_replace_n, + array_slice, array_to_string, arrow_typeof, ascii, asin, asinh, atan, atan2, atanh, + bit_length, btrim, cardinality, cbrt, ceil, character_length, chr, coalesce, + concat_expr, concat_ws_expr, cos, cosh, cot, current_date, current_time, date_bin, + date_part, date_trunc, decode, degrees, digest, encode, exp, expr::{self, InList, Sort, WindowFunction}, factorial, flatten, floor, from_unixtime, gcd, isnan, iszero, lcm, left, ln, log, log10, log2, @@ -1393,6 +1393,10 @@ pub fn parse_expr( parse_expr(&args[0], registry)?, parse_expr(&args[1], registry)?, )), + ScalarFunction::ArrayIntersect => Ok(array_intersect( + parse_expr(&args[0], registry)?, + parse_expr(&args[1], registry)?, + )), ScalarFunction::Cardinality => { Ok(cardinality(parse_expr(&args[0], registry)?)) } From 21b96261b48b86381edbdda8c7405f764d9aea33 Mon Sep 17 00:00:00 2001 From: Asura7969 <1402357969@qq.com> Date: Sat, 11 Nov 2023 19:10:28 +0800 Subject: [PATCH 227/572] Improve documentation for calculate_prune_length method in `SymmetricHashJoin` (#8125) * Minor: Improve the document format of JoinHashMap * Minor: Improve documentation for calculate_prune_length method * fix: method describe --- .../src/joins/symmetric_hash_join.rs | 16 +++------------- 1 file changed, 3 insertions(+), 13 deletions(-) diff --git a/datafusion/physical-plan/src/joins/symmetric_hash_join.rs b/datafusion/physical-plan/src/joins/symmetric_hash_join.rs index 39ac25ecb561..1306a4874436 100644 --- a/datafusion/physical-plan/src/joins/symmetric_hash_join.rs +++ b/datafusion/physical-plan/src/joins/symmetric_hash_join.rs @@ -952,27 +952,17 @@ impl OneSideHashJoiner { Ok(()) } - /// Prunes the internal buffer. - /// - /// Argument `probe_batch` is used to update the intervals of the sorted - /// filter expressions. The updated build interval determines the new length - /// of the build side. If there are rows to prune, they are removed from the - /// internal buffer. + /// Calculate prune length. /// /// # Arguments /// - /// * `schema` - The schema of the final output record batch - /// * `probe_batch` - Incoming RecordBatch of the probe side. + /// * `build_side_sorted_filter_expr` - Build side mutable sorted filter expression.. /// * `probe_side_sorted_filter_expr` - Probe side mutable sorted filter expression. - /// * `join_type` - The type of join (e.g. inner, left, right, etc.). - /// * `column_indices` - A vector of column indices that specifies which columns from the - /// build side should be included in the output. /// * `graph` - A mutable reference to the physical expression graph. /// /// # Returns /// - /// If there are rows to prune, returns the pruned build side record batch wrapped in an `Ok` variant. - /// Otherwise, returns `Ok(None)`. + /// A Result object that contains the pruning length. pub(crate) fn calculate_prune_length_with_probe_batch( &mut self, build_side_sorted_filter_expr: &mut SortedFilterExpr, From 4068b0614877b5650b4e702e26d9f263802198fa Mon Sep 17 00:00:00 2001 From: Andrew Lamb Date: Sat, 11 Nov 2023 06:14:13 -0500 Subject: [PATCH 228/572] Minor: remove duplicated `array_replace` tests (#8066) * Add whitespace * remove redundant test * remove unused code --- .../physical-expr/src/array_expressions.rs | 198 ------------------ datafusion/sqllogictest/test_files/array.slt | 137 ++++++++++-- 2 files changed, 118 insertions(+), 217 deletions(-) diff --git a/datafusion/physical-expr/src/array_expressions.rs b/datafusion/physical-expr/src/array_expressions.rs index 87ba77b497b2..54452e3653a8 100644 --- a/datafusion/physical-expr/src/array_expressions.rs +++ b/datafusion/physical-expr/src/array_expressions.rs @@ -2775,193 +2775,6 @@ mod tests { ); } - #[test] - fn test_array_replace() { - // array_replace([3, 1, 2, 3, 2, 3], 3, 4) = [4, 1, 2, 3, 2, 3] - let list_array = return_array_with_repeating_elements(); - let array = array_replace(&[ - list_array, - Arc::new(Int64Array::from_value(3, 1)), - Arc::new(Int64Array::from_value(4, 1)), - ]) - .expect("failed to initialize function array_replace"); - let result = - as_list_array(&array).expect("failed to initialize function array_replace"); - - assert_eq!(result.len(), 1); - assert_eq!( - &[4, 1, 2, 3, 2, 3], - result - .value(0) - .as_any() - .downcast_ref::() - .unwrap() - .values() - ); - } - - #[test] - fn test_nested_array_replace() { - // array_replace( - // [[1, 2, 3, 4], [5, 6, 7, 8], [1, 2, 3, 4], [9, 10, 11, 12], [5, 6, 7, 8]], - // [1, 2, 3, 4], - // [11, 12, 13, 14], - // ) = [[11, 12, 13, 14], [5, 6, 7, 8], [1, 2, 3, 4], [9, 10, 11, 12], [5, 6, 7, 8]] - let list_array = return_nested_array_with_repeating_elements(); - let from_array = return_array(); - let to_array = return_extra_array(); - let array = array_replace(&[list_array, from_array, to_array]) - .expect("failed to initialize function array_replace"); - let result = - as_list_array(&array).expect("failed to initialize function array_replace"); - - assert_eq!(result.len(), 1); - let data = vec![ - Some(vec![Some(11), Some(12), Some(13), Some(14)]), - Some(vec![Some(5), Some(6), Some(7), Some(8)]), - Some(vec![Some(1), Some(2), Some(3), Some(4)]), - Some(vec![Some(9), Some(10), Some(11), Some(12)]), - Some(vec![Some(5), Some(6), Some(7), Some(8)]), - ]; - let expected = ListArray::from_iter_primitive::(data); - assert_eq!( - expected, - result - .value(0) - .as_any() - .downcast_ref::() - .unwrap() - .clone() - ); - } - - #[test] - fn test_array_replace_n() { - // array_replace_n([3, 1, 2, 3, 2, 3], 3, 4, 2) = [4, 1, 2, 4, 2, 3] - let list_array = return_array_with_repeating_elements(); - let array = array_replace_n(&[ - list_array, - Arc::new(Int64Array::from_value(3, 1)), - Arc::new(Int64Array::from_value(4, 1)), - Arc::new(Int64Array::from_value(2, 1)), - ]) - .expect("failed to initialize function array_replace_n"); - let result = - as_list_array(&array).expect("failed to initialize function array_replace_n"); - - assert_eq!(result.len(), 1); - assert_eq!( - &[4, 1, 2, 4, 2, 3], - result - .value(0) - .as_any() - .downcast_ref::() - .unwrap() - .values() - ); - } - - #[test] - fn test_nested_array_replace_n() { - // array_replace_n( - // [[1, 2, 3, 4], [5, 6, 7, 8], [1, 2, 3, 4], [9, 10, 11, 12], [5, 6, 7, 8]], - // [1, 2, 3, 4], - // [11, 12, 13, 14], - // 2, - // ) = [[11, 12, 13, 14], [5, 6, 7, 8], [11, 12, 13, 14], [9, 10, 11, 12], [5, 6, 7, 8]] - let list_array = return_nested_array_with_repeating_elements(); - let from_array = return_array(); - let to_array = return_extra_array(); - let array = array_replace_n(&[ - list_array, - from_array, - to_array, - Arc::new(Int64Array::from_value(2, 1)), - ]) - .expect("failed to initialize function array_replace_n"); - let result = - as_list_array(&array).expect("failed to initialize function array_replace_n"); - - assert_eq!(result.len(), 1); - let data = vec![ - Some(vec![Some(11), Some(12), Some(13), Some(14)]), - Some(vec![Some(5), Some(6), Some(7), Some(8)]), - Some(vec![Some(11), Some(12), Some(13), Some(14)]), - Some(vec![Some(9), Some(10), Some(11), Some(12)]), - Some(vec![Some(5), Some(6), Some(7), Some(8)]), - ]; - let expected = ListArray::from_iter_primitive::(data); - assert_eq!( - expected, - result - .value(0) - .as_any() - .downcast_ref::() - .unwrap() - .clone() - ); - } - - #[test] - fn test_array_replace_all() { - // array_replace_all([3, 1, 2, 3, 2, 3], 3, 4) = [4, 1, 2, 4, 2, 4] - let list_array = return_array_with_repeating_elements(); - let array = array_replace_all(&[ - list_array, - Arc::new(Int64Array::from_value(3, 1)), - Arc::new(Int64Array::from_value(4, 1)), - ]) - .expect("failed to initialize function array_replace_all"); - let result = as_list_array(&array) - .expect("failed to initialize function array_replace_all"); - - assert_eq!(result.len(), 1); - assert_eq!( - &[4, 1, 2, 4, 2, 4], - result - .value(0) - .as_any() - .downcast_ref::() - .unwrap() - .values() - ); - } - - #[test] - fn test_nested_array_replace_all() { - // array_replace_all( - // [[1, 2, 3, 4], [5, 6, 7, 8], [1, 2, 3, 4], [9, 10, 11, 12], [5, 6, 7, 8]], - // [1, 2, 3, 4], - // [11, 12, 13, 14], - // ) = [[11, 12, 13, 14], [5, 6, 7, 8], [11, 12, 13, 14], [9, 10, 11, 12], [5, 6, 7, 8]] - let list_array = return_nested_array_with_repeating_elements(); - let from_array = return_array(); - let to_array = return_extra_array(); - let array = array_replace_all(&[list_array, from_array, to_array]) - .expect("failed to initialize function array_replace_all"); - let result = as_list_array(&array) - .expect("failed to initialize function array_replace_all"); - - assert_eq!(result.len(), 1); - let data = vec![ - Some(vec![Some(11), Some(12), Some(13), Some(14)]), - Some(vec![Some(5), Some(6), Some(7), Some(8)]), - Some(vec![Some(11), Some(12), Some(13), Some(14)]), - Some(vec![Some(9), Some(10), Some(11), Some(12)]), - Some(vec![Some(5), Some(6), Some(7), Some(8)]), - ]; - let expected = ListArray::from_iter_primitive::(data); - assert_eq!( - expected, - result - .value(0) - .as_any() - .downcast_ref::() - .unwrap() - .clone() - ); - } - #[test] fn test_array_to_string() { // array_to_string([1, 2, 3, 4], ',') = 1,2,3,4 @@ -3194,17 +3007,6 @@ mod tests { make_array(&args).expect("failed to initialize function array") } - fn return_extra_array() -> ArrayRef { - // Returns: [11, 12, 13, 14] - let args = [ - Arc::new(Int64Array::from(vec![Some(11)])) as ArrayRef, - Arc::new(Int64Array::from(vec![Some(12)])) as ArrayRef, - Arc::new(Int64Array::from(vec![Some(13)])) as ArrayRef, - Arc::new(Int64Array::from(vec![Some(14)])) as ArrayRef, - ]; - make_array(&args).expect("failed to initialize function array") - } - fn return_nested_array() -> ArrayRef { // Returns: [[1, 2, 3, 4], [5, 6, 7, 8]] let args = [ diff --git a/datafusion/sqllogictest/test_files/array.slt b/datafusion/sqllogictest/test_files/array.slt index f83ed5a95ff3..ad81f37e0764 100644 --- a/datafusion/sqllogictest/test_files/array.slt +++ b/datafusion/sqllogictest/test_files/array.slt @@ -1605,19 +1605,35 @@ select array_positions(column1, make_array(4, 5, 6)), array_positions(make_array # array_replace scalar function #1 query ??? -select array_replace(make_array(1, 2, 3, 4), 2, 3), array_replace(make_array(1, 4, 4, 5, 4, 6, 7), 4, 0), array_replace(make_array(1, 2, 3), 4, 0); +select + array_replace(make_array(1, 2, 3, 4), 2, 3), + array_replace(make_array(1, 4, 4, 5, 4, 6, 7), 4, 0), + array_replace(make_array(1, 2, 3), 4, 0); ---- [1, 3, 3, 4] [1, 0, 4, 5, 4, 6, 7] [1, 2, 3] # array_replace scalar function #2 (element is list) query ?? -select array_replace(make_array([1, 2, 3], [4, 5, 6], [5, 5, 5], [4, 5, 6], [7, 8, 9]), [4, 5, 6], [1, 1, 1]), array_replace(make_array([1, 3, 2], [2, 3, 4], [2, 3, 4], [5, 3, 1], [1, 3, 2]), [2, 3, 4], [3, 1, 4]); +select + array_replace( + make_array([1, 2, 3], [4, 5, 6], [5, 5, 5], [4, 5, 6], [7, 8, 9]), + [4, 5, 6], + [1, 1, 1] + ), + array_replace( + make_array([1, 3, 2], [2, 3, 4], [2, 3, 4], [5, 3, 1], [1, 3, 2]), + [2, 3, 4], + [3, 1, 4] + ); ---- [[1, 2, 3], [1, 1, 1], [5, 5, 5], [4, 5, 6], [7, 8, 9]] [[1, 3, 2], [3, 1, 4], [2, 3, 4], [5, 3, 1], [1, 3, 2]] # list_replace scalar function #3 (function alias `list_replace`) query ??? -select list_replace(make_array(1, 2, 3, 4), 2, 3), list_replace(make_array(1, 4, 4, 5, 4, 6, 7), 4, 0), list_replace(make_array(1, 2, 3), 4, 0); +select list_replace( + make_array(1, 2, 3, 4), 2, 3), + list_replace(make_array(1, 4, 4, 5, 4, 6, 7), 4, 0), + list_replace(make_array(1, 2, 3), 4, 0); ---- [1, 3, 3, 4] [1, 0, 4, 5, 4, 6, 7] [1, 2, 3] @@ -1641,7 +1657,11 @@ select array_replace(column1, column2, column3) from nested_arrays_with_repeatin # array_replace scalar function with columns and scalars #1 query ??? -select array_replace(make_array(1, 2, 2, 4, 5, 4, 4, 7, 7, 10, 7, 8), column2, column3), array_replace(column1, 1, column3), array_replace(column1, column2, 4) from arrays_with_repeating_elements; +select + array_replace(make_array(1, 2, 2, 4, 5, 4, 4, 7, 7, 10, 7, 8), column2, column3), + array_replace(column1, 1, column3), + array_replace(column1, column2, 4) +from arrays_with_repeating_elements; ---- [1, 4, 2, 4, 5, 4, 4, 7, 7, 10, 7, 8] [4, 2, 1, 3, 2, 2, 1, 3, 2, 3] [1, 4, 1, 3, 2, 2, 1, 3, 2, 3] [1, 2, 2, 7, 5, 4, 4, 7, 7, 10, 7, 8] [4, 4, 5, 5, 6, 5, 5, 5, 4, 4] [4, 4, 5, 5, 6, 5, 5, 5, 4, 4] @@ -1650,7 +1670,16 @@ select array_replace(make_array(1, 2, 2, 4, 5, 4, 4, 7, 7, 10, 7, 8), column2, c # array_replace scalar function with columns and scalars #2 (element is list) query ??? -select array_replace(make_array([1, 2, 3], [4, 5, 6], [4, 5, 6], [10, 11, 12], [13, 14, 15], [10, 11, 12], [10, 11, 12], [19, 20, 21], [19, 20, 21], [28, 29, 30], [19, 20, 21], [22, 23, 24]), column2, column3), array_replace(column1, make_array(1, 2, 3), column3), array_replace(column1, column2, make_array(11, 12, 13)) from nested_arrays_with_repeating_elements; +select + array_replace( + make_array( + [1, 2, 3], [4, 5, 6], [4, 5, 6], [10, 11, 12], [13, 14, 15], [10, 11, 12], [10, 11, 12], [19, 20, 21], [19, 20, 21], [28, 29, 30], [19, 20, 21], [22, 23, 24]), + column2, + column3 + ), + array_replace(column1, make_array(1, 2, 3), column3), + array_replace(column1, column2, make_array(11, 12, 13)) +from nested_arrays_with_repeating_elements; ---- [[1, 2, 3], [10, 11, 12], [4, 5, 6], [10, 11, 12], [13, 14, 15], [10, 11, 12], [10, 11, 12], [19, 20, 21], [19, 20, 21], [28, 29, 30], [19, 20, 21], [22, 23, 24]] [[10, 11, 12], [4, 5, 6], [1, 2, 3], [7, 8, 9], [4, 5, 6], [4, 5, 6], [1, 2, 3], [7, 8, 9], [4, 5, 6], [7, 8, 9]] [[1, 2, 3], [11, 12, 13], [1, 2, 3], [7, 8, 9], [4, 5, 6], [4, 5, 6], [1, 2, 3], [7, 8, 9], [4, 5, 6], [7, 8, 9]] [[1, 2, 3], [4, 5, 6], [4, 5, 6], [19, 20, 21], [13, 14, 15], [10, 11, 12], [10, 11, 12], [19, 20, 21], [19, 20, 21], [28, 29, 30], [19, 20, 21], [22, 23, 24]] [[10, 11, 12], [10, 11, 12], [13, 14, 15], [13, 14, 15], [16, 17, 18], [13, 14, 15], [13, 14, 15], [13, 14, 15], [10, 11, 12], [10, 11, 12]] [[11, 12, 13], [10, 11, 12], [13, 14, 15], [13, 14, 15], [16, 17, 18], [13, 14, 15], [13, 14, 15], [13, 14, 15], [10, 11, 12], [10, 11, 12]] @@ -1661,25 +1690,45 @@ select array_replace(make_array([1, 2, 3], [4, 5, 6], [4, 5, 6], [10, 11, 12], [ # array_replace_n scalar function #1 query ??? -select array_replace_n(make_array(1, 2, 3, 4), 2, 3, 2), array_replace_n(make_array(1, 4, 4, 5, 4, 6, 7), 4, 0, 2), array_replace_n(make_array(1, 2, 3), 4, 0, 3); +select + array_replace_n(make_array(1, 2, 3, 4), 2, 3, 2), + array_replace_n(make_array(1, 4, 4, 5, 4, 6, 7), 4, 0, 2), + array_replace_n(make_array(1, 2, 3), 4, 0, 3); ---- [1, 3, 3, 4] [1, 0, 0, 5, 4, 6, 7] [1, 2, 3] # array_replace_n scalar function #2 (element is list) query ?? -select array_replace_n(make_array([1, 2, 3], [4, 5, 6], [5, 5, 5], [4, 5, 6], [7, 8, 9]), [4, 5, 6], [1, 1, 1], 2), array_replace_n(make_array([1, 3, 2], [2, 3, 4], [2, 3, 4], [5, 3, 1], [1, 3, 2]), [2, 3, 4], [3, 1, 4], 2); +select + array_replace_n( + make_array([1, 2, 3], [4, 5, 6], [5, 5, 5], [4, 5, 6], [7, 8, 9]), + [4, 5, 6], + [1, 1, 1], + 2 + ), + array_replace_n( + make_array([1, 3, 2], [2, 3, 4], [2, 3, 4], [5, 3, 1], [1, 3, 2]), + [2, 3, 4], + [3, 1, 4], + 2 + ); ---- [[1, 2, 3], [1, 1, 1], [5, 5, 5], [1, 1, 1], [7, 8, 9]] [[1, 3, 2], [3, 1, 4], [3, 1, 4], [5, 3, 1], [1, 3, 2]] # list_replace_n scalar function #3 (function alias `array_replace_n`) query ??? -select list_replace_n(make_array(1, 2, 3, 4), 2, 3, 2), list_replace_n(make_array(1, 4, 4, 5, 4, 6, 7), 4, 0, 2), list_replace_n(make_array(1, 2, 3), 4, 0, 3); +select + list_replace_n(make_array(1, 2, 3, 4), 2, 3, 2), + list_replace_n(make_array(1, 4, 4, 5, 4, 6, 7), 4, 0, 2), + list_replace_n(make_array(1, 2, 3), 4, 0, 3); ---- [1, 3, 3, 4] [1, 0, 0, 5, 4, 6, 7] [1, 2, 3] # array_replace_n scalar function with columns #1 query ? -select array_replace_n(column1, column2, column3, column4) from arrays_with_repeating_elements; +select + array_replace_n(column1, column2, column3, column4) +from arrays_with_repeating_elements; ---- [1, 4, 1, 3, 4, 4, 1, 3, 2, 3] [7, 7, 5, 5, 6, 5, 5, 5, 4, 4] @@ -1688,7 +1737,9 @@ select array_replace_n(column1, column2, column3, column4) from arrays_with_repe # array_replace_n scalar function with columns #2 (element is list) query ? -select array_replace_n(column1, column2, column3, column4) from nested_arrays_with_repeating_elements; +select + array_replace_n(column1, column2, column3, column4) +from nested_arrays_with_repeating_elements; ---- [[1, 2, 3], [10, 11, 12], [1, 2, 3], [7, 8, 9], [10, 11, 12], [10, 11, 12], [1, 2, 3], [7, 8, 9], [4, 5, 6], [7, 8, 9]] [[19, 20, 21], [19, 20, 21], [13, 14, 15], [13, 14, 15], [16, 17, 18], [13, 14, 15], [13, 14, 15], [13, 14, 15], [10, 11, 12], [10, 11, 12]] @@ -1697,7 +1748,12 @@ select array_replace_n(column1, column2, column3, column4) from nested_arrays_wi # array_replace_n scalar function with columns and scalars #1 query ???? -select array_replace_n(make_array(1, 2, 2, 4, 5, 4, 4, 7, 7, 10, 7, 8), column2, column3, column4), array_replace_n(column1, 1, column3, column4), array_replace_n(column1, column2, 4, column4), array_replace_n(column1, column2, column3, 2) from arrays_with_repeating_elements; +select + array_replace_n(make_array(1, 2, 2, 4, 5, 4, 4, 7, 7, 10, 7, 8), column2, column3, column4), + array_replace_n(column1, 1, column3, column4), + array_replace_n(column1, column2, 4, column4), + array_replace_n(column1, column2, column3, 2) +from arrays_with_repeating_elements; ---- [1, 4, 4, 4, 5, 4, 4, 7, 7, 10, 7, 8] [4, 2, 4, 3, 2, 2, 4, 3, 2, 3] [1, 4, 1, 3, 4, 4, 1, 3, 2, 3] [1, 4, 1, 3, 4, 2, 1, 3, 2, 3] [1, 2, 2, 7, 5, 7, 4, 7, 7, 10, 7, 8] [4, 4, 5, 5, 6, 5, 5, 5, 4, 4] [4, 4, 5, 5, 6, 5, 5, 5, 4, 4] [7, 7, 5, 5, 6, 5, 5, 5, 4, 4] @@ -1706,7 +1762,18 @@ select array_replace_n(make_array(1, 2, 2, 4, 5, 4, 4, 7, 7, 10, 7, 8), column2, # array_replace_n scalar function with columns and scalars #2 (element is list) query ???? -select array_replace_n(make_array([7, 8, 9], [2, 1, 3], [1, 5, 6], [10, 11, 12], [2, 1, 3], [7, 8, 9], [4, 5, 6]), column2, column3, column4), array_replace_n(column1, make_array(1, 2, 3), column3, column4), array_replace_n(column1, column2, make_array(11, 12, 13), column4), array_replace_n(column1, column2, column3, 2) from nested_arrays_with_repeating_elements; +select + array_replace_n( + make_array( + [7, 8, 9], [2, 1, 3], [1, 5, 6], [10, 11, 12], [2, 1, 3], [7, 8, 9], [4, 5, 6]), + column2, + column3, + column4 + ), + array_replace_n(column1, make_array(1, 2, 3), column3, column4), + array_replace_n(column1, column2, make_array(11, 12, 13), column4), + array_replace_n(column1, column2, column3, 2) +from nested_arrays_with_repeating_elements; ---- [[7, 8, 9], [2, 1, 3], [1, 5, 6], [10, 11, 12], [2, 1, 3], [7, 8, 9], [10, 11, 12]] [[10, 11, 12], [4, 5, 6], [10, 11, 12], [7, 8, 9], [4, 5, 6], [4, 5, 6], [10, 11, 12], [7, 8, 9], [4, 5, 6], [7, 8, 9]] [[1, 2, 3], [11, 12, 13], [1, 2, 3], [7, 8, 9], [11, 12, 13], [11, 12, 13], [1, 2, 3], [7, 8, 9], [4, 5, 6], [7, 8, 9]] [[1, 2, 3], [10, 11, 12], [1, 2, 3], [7, 8, 9], [10, 11, 12], [4, 5, 6], [1, 2, 3], [7, 8, 9], [4, 5, 6], [7, 8, 9]] [[7, 8, 9], [2, 1, 3], [1, 5, 6], [19, 20, 21], [2, 1, 3], [7, 8, 9], [4, 5, 6]] [[10, 11, 12], [10, 11, 12], [13, 14, 15], [13, 14, 15], [16, 17, 18], [13, 14, 15], [13, 14, 15], [13, 14, 15], [10, 11, 12], [10, 11, 12]] [[11, 12, 13], [11, 12, 13], [13, 14, 15], [13, 14, 15], [16, 17, 18], [13, 14, 15], [13, 14, 15], [13, 14, 15], [10, 11, 12], [10, 11, 12]] [[19, 20, 21], [19, 20, 21], [13, 14, 15], [13, 14, 15], [16, 17, 18], [13, 14, 15], [13, 14, 15], [13, 14, 15], [10, 11, 12], [10, 11, 12]] @@ -1717,25 +1784,43 @@ select array_replace_n(make_array([7, 8, 9], [2, 1, 3], [1, 5, 6], [10, 11, 12], # array_replace_all scalar function #1 query ??? -select array_replace_all(make_array(1, 2, 3, 4), 2, 3), array_replace_all(make_array(1, 4, 4, 5, 4, 6, 7), 4, 0), array_replace_all(make_array(1, 2, 3), 4, 0); +select + array_replace_all(make_array(1, 2, 3, 4), 2, 3), + array_replace_all(make_array(1, 4, 4, 5, 4, 6, 7), 4, 0), + array_replace_all(make_array(1, 2, 3), 4, 0); ---- [1, 3, 3, 4] [1, 0, 0, 5, 0, 6, 7] [1, 2, 3] # array_replace_all scalar function #2 (element is list) query ?? -select array_replace_all(make_array([1, 2, 3], [4, 5, 6], [5, 5, 5], [4, 5, 6], [7, 8, 9]), [4, 5, 6], [1, 1, 1]), array_replace_all(make_array([1, 3, 2], [2, 3, 4], [2, 3, 4], [5, 3, 1], [1, 3, 2]), [2, 3, 4], [3, 1, 4]); +select + array_replace_all( + make_array([1, 2, 3], [4, 5, 6], [5, 5, 5], [4, 5, 6], [7, 8, 9]), + [4, 5, 6], + [1, 1, 1] + ), + array_replace_all( + make_array([1, 3, 2], [2, 3, 4], [2, 3, 4], [5, 3, 1], [1, 3, 2]), + [2, 3, 4], + [3, 1, 4] + ); ---- [[1, 2, 3], [1, 1, 1], [5, 5, 5], [1, 1, 1], [7, 8, 9]] [[1, 3, 2], [3, 1, 4], [3, 1, 4], [5, 3, 1], [1, 3, 2]] # list_replace_all scalar function #3 (function alias `array_replace_all`) query ??? -select list_replace_all(make_array(1, 2, 3, 4), 2, 3), list_replace_all(make_array(1, 4, 4, 5, 4, 6, 7), 4, 0), list_replace_all(make_array(1, 2, 3), 4, 0); +select + list_replace_all(make_array(1, 2, 3, 4), 2, 3), + list_replace_all(make_array(1, 4, 4, 5, 4, 6, 7), 4, 0), + list_replace_all(make_array(1, 2, 3), 4, 0); ---- [1, 3, 3, 4] [1, 0, 0, 5, 0, 6, 7] [1, 2, 3] # array_replace_all scalar function with columns #1 query ? -select array_replace_all(column1, column2, column3) from arrays_with_repeating_elements; +select + array_replace_all(column1, column2, column3) +from arrays_with_repeating_elements; ---- [1, 4, 1, 3, 4, 4, 1, 3, 4, 3] [7, 7, 5, 5, 6, 5, 5, 5, 7, 7] @@ -1744,7 +1829,9 @@ select array_replace_all(column1, column2, column3) from arrays_with_repeating_e # array_replace_all scalar function with columns #2 (element is list) query ? -select array_replace_all(column1, column2, column3) from nested_arrays_with_repeating_elements; +select + array_replace_all(column1, column2, column3) +from nested_arrays_with_repeating_elements; ---- [[1, 2, 3], [10, 11, 12], [1, 2, 3], [7, 8, 9], [10, 11, 12], [10, 11, 12], [1, 2, 3], [7, 8, 9], [10, 11, 12], [7, 8, 9]] [[19, 20, 21], [19, 20, 21], [13, 14, 15], [13, 14, 15], [16, 17, 18], [13, 14, 15], [13, 14, 15], [13, 14, 15], [19, 20, 21], [19, 20, 21]] @@ -1753,7 +1840,11 @@ select array_replace_all(column1, column2, column3) from nested_arrays_with_repe # array_replace_all scalar function with columns and scalars #1 query ??? -select array_replace_all(make_array(1, 2, 2, 4, 5, 4, 4, 7, 7, 10, 7, 8), column2, column3), array_replace_all(column1, 1, column3), array_replace_all(column1, column2, 4) from arrays_with_repeating_elements; +select + array_replace_all(make_array(1, 2, 2, 4, 5, 4, 4, 7, 7, 10, 7, 8), column2, column3), + array_replace_all(column1, 1, column3), + array_replace_all(column1, column2, 4) +from arrays_with_repeating_elements; ---- [1, 4, 4, 4, 5, 4, 4, 7, 7, 10, 7, 8] [4, 2, 4, 3, 2, 2, 4, 3, 2, 3] [1, 4, 1, 3, 4, 4, 1, 3, 4, 3] [1, 2, 2, 7, 5, 7, 7, 7, 7, 10, 7, 8] [4, 4, 5, 5, 6, 5, 5, 5, 4, 4] [4, 4, 5, 5, 6, 5, 5, 5, 4, 4] @@ -1762,7 +1853,15 @@ select array_replace_all(make_array(1, 2, 2, 4, 5, 4, 4, 7, 7, 10, 7, 8), column # array_replace_all scalar function with columns and scalars #2 (element is list) query ??? -select array_replace_all(make_array([1, 2, 3], [4, 5, 6], [4, 5, 6], [10, 11, 12], [13, 14, 15], [10, 11, 12], [10, 11, 12], [19, 20, 21], [19, 20, 21], [28, 29, 30], [19, 20, 21], [22, 23, 24]), column2, column3), array_replace_all(column1, make_array(1, 2, 3), column3), array_replace_all(column1, column2, make_array(11, 12, 13)) from nested_arrays_with_repeating_elements; +select + array_replace_all( + make_array([1, 2, 3], [4, 5, 6], [4, 5, 6], [10, 11, 12], [13, 14, 15], [10, 11, 12], [10, 11, 12], [19, 20, 21], [19, 20, 21], [28, 29, 30], [19, 20, 21], [22, 23, 24]), + column2, + column3 + ), + array_replace_all(column1, make_array(1, 2, 3), column3), + array_replace_all(column1, column2, make_array(11, 12, 13)) +from nested_arrays_with_repeating_elements; ---- [[1, 2, 3], [10, 11, 12], [10, 11, 12], [10, 11, 12], [13, 14, 15], [10, 11, 12], [10, 11, 12], [19, 20, 21], [19, 20, 21], [28, 29, 30], [19, 20, 21], [22, 23, 24]] [[10, 11, 12], [4, 5, 6], [10, 11, 12], [7, 8, 9], [4, 5, 6], [4, 5, 6], [10, 11, 12], [7, 8, 9], [4, 5, 6], [7, 8, 9]] [[1, 2, 3], [11, 12, 13], [1, 2, 3], [7, 8, 9], [11, 12, 13], [11, 12, 13], [1, 2, 3], [7, 8, 9], [11, 12, 13], [7, 8, 9]] [[1, 2, 3], [4, 5, 6], [4, 5, 6], [19, 20, 21], [13, 14, 15], [19, 20, 21], [19, 20, 21], [19, 20, 21], [19, 20, 21], [28, 29, 30], [19, 20, 21], [22, 23, 24]] [[10, 11, 12], [10, 11, 12], [13, 14, 15], [13, 14, 15], [16, 17, 18], [13, 14, 15], [13, 14, 15], [13, 14, 15], [10, 11, 12], [10, 11, 12]] [[11, 12, 13], [11, 12, 13], [13, 14, 15], [13, 14, 15], [16, 17, 18], [13, 14, 15], [13, 14, 15], [13, 14, 15], [11, 12, 13], [11, 12, 13]] From ceb09b2576ba1727d2f67c88b4f10cf08f20be41 Mon Sep 17 00:00:00 2001 From: Yongting You <2010youy01@gmail.com> Date: Sat, 11 Nov 2023 03:16:08 -0800 Subject: [PATCH 229/572] use tmp directory in test (#8115) --- .../core/src/execution/context/parquet.rs | 34 +++++++++++++++---- 1 file changed, 27 insertions(+), 7 deletions(-) diff --git a/datafusion/core/src/execution/context/parquet.rs b/datafusion/core/src/execution/context/parquet.rs index ef1f0143543d..821b1ccf1823 100644 --- a/datafusion/core/src/execution/context/parquet.rs +++ b/datafusion/core/src/execution/context/parquet.rs @@ -80,6 +80,7 @@ mod tests { use crate::dataframe::DataFrameWriteOptions; use crate::parquet::basic::Compression; use crate::test_util::parquet_test_data; + use tempfile::tempdir; use super::*; @@ -137,6 +138,7 @@ mod tests { Ok(()) } + #[cfg(not(target_family = "windows"))] #[tokio::test] async fn read_from_different_file_extension() -> Result<()> { let ctx = SessionContext::new(); @@ -155,11 +157,29 @@ mod tests { ], )?)?; + let temp_dir = tempdir()?; + let temp_dir_path = temp_dir.path(); + let path1 = temp_dir_path + .join("output1.parquet") + .to_str() + .unwrap() + .to_string(); + let path2 = temp_dir_path + .join("output2.parquet.snappy") + .to_str() + .unwrap() + .to_string(); + let path3 = temp_dir_path + .join("output3.parquet.snappy.parquet") + .to_str() + .unwrap() + .to_string(); + // Write the dataframe to a parquet file named 'output1.parquet' write_df .clone() .write_parquet( - "output1.parquet", + &path1, DataFrameWriteOptions::new().with_single_file_output(true), Some( WriterProperties::builder() @@ -173,7 +193,7 @@ mod tests { write_df .clone() .write_parquet( - "output2.parquet.snappy", + &path2, DataFrameWriteOptions::new().with_single_file_output(true), Some( WriterProperties::builder() @@ -186,7 +206,7 @@ mod tests { // Write the dataframe to a parquet file named 'output3.parquet.snappy.parquet' write_df .write_parquet( - "output3.parquet.snappy.parquet", + &path3, DataFrameWriteOptions::new().with_single_file_output(true), Some( WriterProperties::builder() @@ -199,7 +219,7 @@ mod tests { // Read the dataframe from 'output1.parquet' with the default file extension. let read_df = ctx .read_parquet( - "output1.parquet", + &path1, ParquetReadOptions { ..Default::default() }, @@ -213,7 +233,7 @@ mod tests { // Read the dataframe from 'output2.parquet.snappy' with the correct file extension. let read_df = ctx .read_parquet( - "output2.parquet.snappy", + &path2, ParquetReadOptions { file_extension: "snappy", ..Default::default() @@ -227,7 +247,7 @@ mod tests { // Read the dataframe from 'output3.parquet.snappy.parquet' with the wrong file extension. let read_df = ctx .read_parquet( - "output2.parquet.snappy", + &path2, ParquetReadOptions { ..Default::default() }, @@ -242,7 +262,7 @@ mod tests { // Read the dataframe from 'output3.parquet.snappy.parquet' with the correct file extension. let read_df = ctx .read_parquet( - "output3.parquet.snappy.parquet", + &path3, ParquetReadOptions { ..Default::default() }, From e642cc2a94f38518d765d25c8113523aedc29198 Mon Sep 17 00:00:00 2001 From: Junjun Dong Date: Sat, 11 Nov 2023 06:40:20 -0800 Subject: [PATCH 230/572] chore: remove panics in datafusion-common::scalar (#7901) --- datafusion/common/src/pyarrow.rs | 2 +- datafusion/common/src/scalar.rs | 598 ++++++++++-------- datafusion/core/benches/scalar.rs | 10 +- .../core/src/datasource/listing/helpers.rs | 5 +- .../physical_plan/file_scan_config.rs | 12 +- .../physical_plan/parquet/row_filter.rs | 2 +- .../physical_plan/parquet/row_groups.rs | 4 +- datafusion/expr/src/columnar_value.rs | 14 +- datafusion/expr/src/window_state.rs | 2 +- .../src/unwrap_cast_in_comparison.rs | 8 +- .../src/aggregate/correlation.rs | 12 +- .../physical-expr/src/aggregate/covariance.rs | 12 +- .../physical-expr/src/aggregate/first_last.rs | 10 +- .../physical-expr/src/aggregate/stddev.rs | 12 +- .../physical-expr/src/aggregate/utils.rs | 4 +- .../physical-expr/src/aggregate/variance.rs | 12 +- .../src/conditional_expressions.rs | 2 +- .../physical-expr/src/datetime_expressions.rs | 2 +- .../physical-expr/src/expressions/binary.rs | 49 +- .../physical-expr/src/expressions/case.rs | 64 +- .../physical-expr/src/expressions/cast.rs | 12 +- .../physical-expr/src/expressions/datum.rs | 14 +- .../src/expressions/get_indexed_field.rs | 33 +- .../physical-expr/src/expressions/in_list.rs | 11 +- .../src/expressions/is_not_null.rs | 5 +- .../physical-expr/src/expressions/is_null.rs | 5 +- .../physical-expr/src/expressions/like.rs | 5 +- .../physical-expr/src/expressions/literal.rs | 5 +- .../physical-expr/src/expressions/mod.rs | 12 +- .../physical-expr/src/expressions/negative.rs | 2 +- .../physical-expr/src/expressions/not.rs | 5 +- .../physical-expr/src/expressions/nullif.rs | 20 +- .../physical-expr/src/expressions/try_cast.rs | 12 +- datafusion/physical-expr/src/functions.rs | 63 +- .../src/intervals/interval_aritmetic.rs | 2 +- .../physical-expr/src/math_expressions.rs | 3 +- datafusion/physical-expr/src/planner.rs | 2 +- .../physical-expr/src/struct_expressions.rs | 15 +- .../window/built_in_window_function_expr.rs | 6 +- .../physical-expr/src/window/lead_lag.rs | 1 + .../physical-expr/src/window/window_expr.rs | 6 +- .../physical-plan/src/aggregates/mod.rs | 29 +- .../src/aggregates/no_grouping.rs | 6 +- datafusion/physical-plan/src/filter.rs | 2 +- .../physical-plan/src/joins/cross_join.rs | 2 +- .../physical-plan/src/joins/hash_join.rs | 8 +- .../src/joins/hash_join_utils.rs | 2 +- .../src/joins/symmetric_hash_join.rs | 6 +- datafusion/physical-plan/src/joins/utils.rs | 2 +- datafusion/physical-plan/src/projection.rs | 6 +- .../physical-plan/src/repartition/mod.rs | 4 +- datafusion/physical-plan/src/sorts/stream.rs | 4 +- datafusion/physical-plan/src/topk/mod.rs | 2 +- datafusion/physical-plan/src/unnest.rs | 2 +- 54 files changed, 723 insertions(+), 427 deletions(-) diff --git a/datafusion/common/src/pyarrow.rs b/datafusion/common/src/pyarrow.rs index 59a8b811e3c8..aa0153919360 100644 --- a/datafusion/common/src/pyarrow.rs +++ b/datafusion/common/src/pyarrow.rs @@ -54,7 +54,7 @@ impl FromPyArrow for ScalarValue { impl ToPyArrow for ScalarValue { fn to_pyarrow(&self, py: Python) -> PyResult { - let array = self.to_array(); + let array = self.to_array()?; // convert to pyarrow array using C data interface let pyarray = array.to_data().to_pyarrow(py)?; let pyscalar = pyarray.call_method1(py, "__getitem__", (0,))?; diff --git a/datafusion/common/src/scalar.rs b/datafusion/common/src/scalar.rs index 0d701eaad283..cdcc9aa4fbc5 100644 --- a/datafusion/common/src/scalar.rs +++ b/datafusion/common/src/scalar.rs @@ -330,9 +330,9 @@ impl PartialOrd for ScalarValue { let arr2 = list_arr2.value(i); let lt_res = - arrow::compute::kernels::cmp::lt(&arr1, &arr2).unwrap(); + arrow::compute::kernels::cmp::lt(&arr1, &arr2).ok()?; let eq_res = - arrow::compute::kernels::cmp::eq(&arr1, &arr2).unwrap(); + arrow::compute::kernels::cmp::eq(&arr1, &arr2).ok()?; for j in 0..lt_res.len() { if lt_res.is_valid(j) && lt_res.value(j) { @@ -431,6 +431,10 @@ macro_rules! hash_float_value { hash_float_value!((f64, u64), (f32, u32)); // manual implementation of `Hash` +// +// # Panics +// +// Panics if there is an error when creating hash values for rows impl std::hash::Hash for ScalarValue { fn hash(&self, state: &mut H) { use ScalarValue::*; @@ -506,15 +510,19 @@ impl std::hash::Hash for ScalarValue { } } -/// return a reference to the values array and the index into it for a +/// Return a reference to the values array and the index into it for a /// dictionary array +/// +/// # Errors +/// +/// Errors if the array cannot be downcasted to DictionaryArray #[inline] pub fn get_dict_value( array: &dyn Array, index: usize, -) -> (&ArrayRef, Option) { - let dict_array = as_dictionary_array::(array).unwrap(); - (dict_array.values(), dict_array.key(index)) +) -> Result<(&ArrayRef, Option)> { + let dict_array = as_dictionary_array::(array)?; + Ok((dict_array.values(), dict_array.key(index))) } /// Create a dictionary array representing `value` repeated `size` @@ -522,9 +530,9 @@ pub fn get_dict_value( fn dict_from_scalar( value: &ScalarValue, size: usize, -) -> ArrayRef { +) -> Result { // values array is one element long (the value) - let values_array = value.to_array_of_size(1); + let values_array = value.to_array_of_size(1)?; // Create a key array with `size` elements, each of 0 let key_array: PrimitiveArray = std::iter::repeat(Some(K::default_value())) @@ -536,11 +544,9 @@ fn dict_from_scalar( // Note: this path could be made faster by using the ArrayData // APIs and skipping validation, if it every comes up in // performance traces. - Arc::new( - DictionaryArray::::try_new(key_array, values_array) - // should always be valid by construction above - .expect("Can not construct dictionary array"), - ) + Ok(Arc::new( + DictionaryArray::::try_new(key_array, values_array)?, // should always be valid by construction above + )) } /// Create a dictionary array representing all the values in values @@ -579,24 +585,44 @@ fn dict_from_values( macro_rules! typed_cast_tz { ($array:expr, $index:expr, $ARRAYTYPE:ident, $SCALAR:ident, $TZ:expr) => {{ - let array = $array.as_any().downcast_ref::<$ARRAYTYPE>().unwrap(); - ScalarValue::$SCALAR( + use std::any::type_name; + let array = $array + .as_any() + .downcast_ref::<$ARRAYTYPE>() + .ok_or_else(|| { + DataFusionError::Internal(format!( + "could not cast value to {}", + type_name::<$ARRAYTYPE>() + )) + })?; + Ok::(ScalarValue::$SCALAR( match array.is_null($index) { true => None, false => Some(array.value($index).into()), }, $TZ.clone(), - ) + )) }}; } macro_rules! typed_cast { ($array:expr, $index:expr, $ARRAYTYPE:ident, $SCALAR:ident) => {{ - let array = $array.as_any().downcast_ref::<$ARRAYTYPE>().unwrap(); - ScalarValue::$SCALAR(match array.is_null($index) { - true => None, - false => Some(array.value($index).into()), - }) + use std::any::type_name; + let array = $array + .as_any() + .downcast_ref::<$ARRAYTYPE>() + .ok_or_else(|| { + DataFusionError::Internal(format!( + "could not cast value to {}", + type_name::<$ARRAYTYPE>() + )) + })?; + Ok::(ScalarValue::$SCALAR( + match array.is_null($index) { + true => None, + false => Some(array.value($index).into()), + }, + )) }}; } @@ -628,12 +654,21 @@ macro_rules! build_timestamp_array_from_option { macro_rules! eq_array_primitive { ($array:expr, $index:expr, $ARRAYTYPE:ident, $VALUE:expr) => {{ - let array = $array.as_any().downcast_ref::<$ARRAYTYPE>().unwrap(); + use std::any::type_name; + let array = $array + .as_any() + .downcast_ref::<$ARRAYTYPE>() + .ok_or_else(|| { + DataFusionError::Internal(format!( + "could not cast value to {}", + type_name::<$ARRAYTYPE>() + )) + })?; let is_valid = array.is_valid($index); - match $VALUE { + Ok::(match $VALUE { Some(val) => is_valid && &array.value($index) == val, None => !is_valid, - } + }) }}; } @@ -935,7 +970,7 @@ impl ScalarValue { /// NB: operating on `ScalarValue` directly is not efficient, performance sensitive code /// should operate on Arrays directly, using vectorized array kernels pub fn add>(&self, other: T) -> Result { - let r = add_wrapping(&self.to_scalar(), &other.borrow().to_scalar())?; + let r = add_wrapping(&self.to_scalar()?, &other.borrow().to_scalar()?)?; Self::try_from_array(r.as_ref(), 0) } /// Checked addition of `ScalarValue` @@ -943,7 +978,7 @@ impl ScalarValue { /// NB: operating on `ScalarValue` directly is not efficient, performance sensitive code /// should operate on Arrays directly, using vectorized array kernels pub fn add_checked>(&self, other: T) -> Result { - let r = add(&self.to_scalar(), &other.borrow().to_scalar())?; + let r = add(&self.to_scalar()?, &other.borrow().to_scalar()?)?; Self::try_from_array(r.as_ref(), 0) } @@ -952,7 +987,7 @@ impl ScalarValue { /// NB: operating on `ScalarValue` directly is not efficient, performance sensitive code /// should operate on Arrays directly, using vectorized array kernels pub fn sub>(&self, other: T) -> Result { - let r = sub_wrapping(&self.to_scalar(), &other.borrow().to_scalar())?; + let r = sub_wrapping(&self.to_scalar()?, &other.borrow().to_scalar()?)?; Self::try_from_array(r.as_ref(), 0) } @@ -961,7 +996,7 @@ impl ScalarValue { /// NB: operating on `ScalarValue` directly is not efficient, performance sensitive code /// should operate on Arrays directly, using vectorized array kernels pub fn sub_checked>(&self, other: T) -> Result { - let r = sub(&self.to_scalar(), &other.borrow().to_scalar())?; + let r = sub(&self.to_scalar()?, &other.borrow().to_scalar()?)?; Self::try_from_array(r.as_ref(), 0) } @@ -1050,7 +1085,11 @@ impl ScalarValue { } /// Converts a scalar value into an 1-row array. - pub fn to_array(&self) -> ArrayRef { + /// + /// # Errors + /// + /// Errors if the ScalarValue cannot be converted into a 1-row array + pub fn to_array(&self) -> Result { self.to_array_of_size(1) } @@ -1059,6 +1098,10 @@ impl ScalarValue { /// /// This can be used to call arrow compute kernels such as `lt` /// + /// # Errors + /// + /// Errors if the ScalarValue cannot be converted into a 1-row array + /// /// # Example /// ``` /// use datafusion_common::ScalarValue; @@ -1069,7 +1112,7 @@ impl ScalarValue { /// /// let result = arrow::compute::kernels::cmp::lt( /// &arr, - /// &five.to_scalar(), + /// &five.to_scalar().unwrap(), /// ).unwrap(); /// /// let expected = BooleanArray::from(vec![ @@ -1082,8 +1125,8 @@ impl ScalarValue { /// assert_eq!(&result, &expected); /// ``` /// [`Datum`]: arrow_array::Datum - pub fn to_scalar(&self) -> Scalar { - Scalar::new(self.to_array_of_size(1)) + pub fn to_scalar(&self) -> Result> { + Ok(Scalar::new(self.to_array_of_size(1)?)) } /// Converts an iterator of references [`ScalarValue`] into an [`ArrayRef`] @@ -1093,6 +1136,10 @@ impl ScalarValue { /// Returns an error if the iterator is empty or if the /// [`ScalarValue`]s are not all the same type /// + /// # Panics + /// + /// Panics if `self` is a dictionary with invalid key type + /// /// # Example /// ``` /// use datafusion_common::ScalarValue; @@ -1199,28 +1246,29 @@ impl ScalarValue { macro_rules! build_array_list_primitive { ($ARRAY_TY:ident, $SCALAR_TY:ident, $NATIVE_TYPE:ident) => {{ - Arc::new(ListArray::from_iter_primitive::<$ARRAY_TY, _, _>( + Ok::(Arc::new(ListArray::from_iter_primitive::<$ARRAY_TY, _, _>( scalars.into_iter().map(|x| match x { ScalarValue::List(arr) => { // `ScalarValue::List` contains a single element `ListArray`. let list_arr = as_list_array(&arr); if list_arr.is_null(0) { - None + Ok(None) } else { let primitive_arr = list_arr.values().as_primitive::<$ARRAY_TY>(); - Some( + Ok(Some( primitive_arr.into_iter().collect::>>(), - ) + )) } } - sv => panic!( + sv => _internal_err!( "Inconsistent types in ScalarValue::iter_to_array. \ Expected {:?}, got {:?}", data_type, sv ), - }), - )) + }) + .collect::>>()?, + ))) }}; } @@ -1273,7 +1321,7 @@ impl ScalarValue { ScalarValue::iter_to_decimal256_array(scalars, *precision, *scale)?; Arc::new(decimal_array) } - DataType::Null => ScalarValue::iter_to_null_array(scalars), + DataType::Null => ScalarValue::iter_to_null_array(scalars)?, DataType::Boolean => build_array_primitive!(BooleanArray, Boolean), DataType::Float32 => build_array_primitive!(Float32Array, Float32), DataType::Float64 => build_array_primitive!(Float64Array, Float64), @@ -1337,34 +1385,34 @@ impl ScalarValue { build_array_primitive!(IntervalMonthDayNanoArray, IntervalMonthDayNano) } DataType::List(fields) if fields.data_type() == &DataType::Int8 => { - build_array_list_primitive!(Int8Type, Int8, i8) + build_array_list_primitive!(Int8Type, Int8, i8)? } DataType::List(fields) if fields.data_type() == &DataType::Int16 => { - build_array_list_primitive!(Int16Type, Int16, i16) + build_array_list_primitive!(Int16Type, Int16, i16)? } DataType::List(fields) if fields.data_type() == &DataType::Int32 => { - build_array_list_primitive!(Int32Type, Int32, i32) + build_array_list_primitive!(Int32Type, Int32, i32)? } DataType::List(fields) if fields.data_type() == &DataType::Int64 => { - build_array_list_primitive!(Int64Type, Int64, i64) + build_array_list_primitive!(Int64Type, Int64, i64)? } DataType::List(fields) if fields.data_type() == &DataType::UInt8 => { - build_array_list_primitive!(UInt8Type, UInt8, u8) + build_array_list_primitive!(UInt8Type, UInt8, u8)? } DataType::List(fields) if fields.data_type() == &DataType::UInt16 => { - build_array_list_primitive!(UInt16Type, UInt16, u16) + build_array_list_primitive!(UInt16Type, UInt16, u16)? } DataType::List(fields) if fields.data_type() == &DataType::UInt32 => { - build_array_list_primitive!(UInt32Type, UInt32, u32) + build_array_list_primitive!(UInt32Type, UInt32, u32)? } DataType::List(fields) if fields.data_type() == &DataType::UInt64 => { - build_array_list_primitive!(UInt64Type, UInt64, u64) + build_array_list_primitive!(UInt64Type, UInt64, u64)? } DataType::List(fields) if fields.data_type() == &DataType::Float32 => { - build_array_list_primitive!(Float32Type, Float32, f32) + build_array_list_primitive!(Float32Type, Float32, f32)? } DataType::List(fields) if fields.data_type() == &DataType::Float64 => { - build_array_list_primitive!(Float64Type, Float64, f64) + build_array_list_primitive!(Float64Type, Float64, f64)? } DataType::List(fields) if fields.data_type() == &DataType::Utf8 => { build_array_list_string!(StringBuilder, as_string_array) @@ -1432,7 +1480,7 @@ impl ScalarValue { if &inner_key_type == key_type { Ok(*scalar) } else { - panic!("Expected inner key type of {key_type} but found: {inner_key_type}, value was ({scalar:?})"); + _internal_err!("Expected inner key type of {key_type} but found: {inner_key_type}, value was ({scalar:?})") } } _ => { @@ -1504,15 +1552,19 @@ impl ScalarValue { Ok(array) } - fn iter_to_null_array(scalars: impl IntoIterator) -> ArrayRef { - let length = - scalars - .into_iter() - .fold(0usize, |r, element: ScalarValue| match element { - ScalarValue::Null => r + 1, - _ => unreachable!(), - }); - new_null_array(&DataType::Null, length) + fn iter_to_null_array( + scalars: impl IntoIterator, + ) -> Result { + let length = scalars.into_iter().try_fold( + 0usize, + |r, element: ScalarValue| match element { + ScalarValue::Null => Ok::(r + 1), + s => { + _internal_err!("Expected ScalarValue::Null element. Received {s:?}") + } + }, + )?; + Ok(new_null_array(&DataType::Null, length)) } fn iter_to_decimal_array( @@ -1523,10 +1575,12 @@ impl ScalarValue { let array = scalars .into_iter() .map(|element: ScalarValue| match element { - ScalarValue::Decimal128(v1, _, _) => v1, - _ => unreachable!(), + ScalarValue::Decimal128(v1, _, _) => Ok(v1), + s => { + _internal_err!("Expected ScalarValue::Null element. Received {s:?}") + } }) - .collect::() + .collect::>()? .with_precision_and_scale(precision, scale)?; Ok(array) } @@ -1539,10 +1593,14 @@ impl ScalarValue { let array = scalars .into_iter() .map(|element: ScalarValue| match element { - ScalarValue::Decimal256(v1, _, _) => v1, - _ => unreachable!(), + ScalarValue::Decimal256(v1, _, _) => Ok(v1), + s => { + _internal_err!( + "Expected ScalarValue::Decimal256 element. Received {s:?}" + ) + } }) - .collect::() + .collect::>()? .with_precision_and_scale(precision, scale)?; Ok(array) } @@ -1607,17 +1665,17 @@ impl ScalarValue { precision: u8, scale: i8, size: usize, - ) -> Decimal128Array { + ) -> Result { match value { Some(val) => Decimal128Array::from(vec![val; size]) .with_precision_and_scale(precision, scale) - .unwrap(), + .map_err(DataFusionError::ArrowError), None => { let mut builder = Decimal128Array::builder(size) .with_precision_and_scale(precision, scale) - .unwrap(); + .map_err(DataFusionError::ArrowError)?; builder.append_nulls(size); - builder.finish() + Ok(builder.finish()) } } } @@ -1627,12 +1685,12 @@ impl ScalarValue { precision: u8, scale: i8, size: usize, - ) -> Decimal256Array { + ) -> Result { std::iter::repeat(value) .take(size) .collect::() .with_precision_and_scale(precision, scale) - .unwrap() + .map_err(DataFusionError::ArrowError) } /// Converts `Vec` where each element has type corresponding to @@ -1671,13 +1729,21 @@ impl ScalarValue { } /// Converts a scalar value into an array of `size` rows. - pub fn to_array_of_size(&self, size: usize) -> ArrayRef { - match self { + /// + /// # Errors + /// + /// Errors if `self` is + /// - a decimal that fails be converted to a decimal array of size + /// - a `Fixedsizelist` that is not supported yet + /// - a `List` that fails to be concatenated into an array of size + /// - a `Dictionary` that fails be converted to a dictionary array of size + pub fn to_array_of_size(&self, size: usize) -> Result { + Ok(match self { ScalarValue::Decimal128(e, precision, scale) => Arc::new( - ScalarValue::build_decimal_array(*e, *precision, *scale, size), + ScalarValue::build_decimal_array(*e, *precision, *scale, size)?, ), ScalarValue::Decimal256(e, precision, scale) => Arc::new( - ScalarValue::build_decimal256_array(*e, *precision, *scale, size), + ScalarValue::build_decimal256_array(*e, *precision, *scale, size)?, ), ScalarValue::Boolean(e) => { Arc::new(BooleanArray::from(vec![*e; size])) as ArrayRef @@ -1790,13 +1856,14 @@ impl ScalarValue { ), }, ScalarValue::Fixedsizelist(..) => { - unimplemented!("FixedSizeList is not supported yet") + return _not_impl_err!("FixedSizeList is not supported yet") } ScalarValue::List(arr) => { let arrays = std::iter::repeat(arr.as_ref()) .take(size) .collect::>(); - arrow::compute::concat(arrays.as_slice()).unwrap() + arrow::compute::concat(arrays.as_slice()) + .map_err(DataFusionError::ArrowError)? } ScalarValue::Date32(e) => { build_array_from_option!(Date32, Date32Array, e, size) @@ -1891,13 +1958,13 @@ impl ScalarValue { ), ScalarValue::Struct(values, fields) => match values { Some(values) => { - let field_values: Vec<_> = fields + let field_values = fields .iter() .zip(values.iter()) .map(|(field, value)| { - (field.clone(), value.to_array_of_size(size)) + Ok((field.clone(), value.to_array_of_size(size)?)) }) - .collect(); + .collect::>>()?; Arc::new(StructArray::from(field_values)) } @@ -1909,19 +1976,19 @@ impl ScalarValue { ScalarValue::Dictionary(key_type, v) => { // values array is one element long (the value) match key_type.as_ref() { - DataType::Int8 => dict_from_scalar::(v, size), - DataType::Int16 => dict_from_scalar::(v, size), - DataType::Int32 => dict_from_scalar::(v, size), - DataType::Int64 => dict_from_scalar::(v, size), - DataType::UInt8 => dict_from_scalar::(v, size), - DataType::UInt16 => dict_from_scalar::(v, size), - DataType::UInt32 => dict_from_scalar::(v, size), - DataType::UInt64 => dict_from_scalar::(v, size), + DataType::Int8 => dict_from_scalar::(v, size)?, + DataType::Int16 => dict_from_scalar::(v, size)?, + DataType::Int32 => dict_from_scalar::(v, size)?, + DataType::Int64 => dict_from_scalar::(v, size)?, + DataType::UInt8 => dict_from_scalar::(v, size)?, + DataType::UInt16 => dict_from_scalar::(v, size)?, + DataType::UInt32 => dict_from_scalar::(v, size)?, + DataType::UInt64 => dict_from_scalar::(v, size)?, _ => unreachable!("Invalid dictionary keys type: {:?}", key_type), } } ScalarValue::Null => new_null_array(&DataType::Null, size), - } + }) } fn get_decimal_value_from_array( @@ -2037,23 +2104,25 @@ impl ScalarValue { array, index, *precision, *scale, )? } - DataType::Boolean => typed_cast!(array, index, BooleanArray, Boolean), - DataType::Float64 => typed_cast!(array, index, Float64Array, Float64), - DataType::Float32 => typed_cast!(array, index, Float32Array, Float32), - DataType::UInt64 => typed_cast!(array, index, UInt64Array, UInt64), - DataType::UInt32 => typed_cast!(array, index, UInt32Array, UInt32), - DataType::UInt16 => typed_cast!(array, index, UInt16Array, UInt16), - DataType::UInt8 => typed_cast!(array, index, UInt8Array, UInt8), - DataType::Int64 => typed_cast!(array, index, Int64Array, Int64), - DataType::Int32 => typed_cast!(array, index, Int32Array, Int32), - DataType::Int16 => typed_cast!(array, index, Int16Array, Int16), - DataType::Int8 => typed_cast!(array, index, Int8Array, Int8), - DataType::Binary => typed_cast!(array, index, BinaryArray, Binary), + DataType::Boolean => typed_cast!(array, index, BooleanArray, Boolean)?, + DataType::Float64 => typed_cast!(array, index, Float64Array, Float64)?, + DataType::Float32 => typed_cast!(array, index, Float32Array, Float32)?, + DataType::UInt64 => typed_cast!(array, index, UInt64Array, UInt64)?, + DataType::UInt32 => typed_cast!(array, index, UInt32Array, UInt32)?, + DataType::UInt16 => typed_cast!(array, index, UInt16Array, UInt16)?, + DataType::UInt8 => typed_cast!(array, index, UInt8Array, UInt8)?, + DataType::Int64 => typed_cast!(array, index, Int64Array, Int64)?, + DataType::Int32 => typed_cast!(array, index, Int32Array, Int32)?, + DataType::Int16 => typed_cast!(array, index, Int16Array, Int16)?, + DataType::Int8 => typed_cast!(array, index, Int8Array, Int8)?, + DataType::Binary => typed_cast!(array, index, BinaryArray, Binary)?, DataType::LargeBinary => { - typed_cast!(array, index, LargeBinaryArray, LargeBinary) + typed_cast!(array, index, LargeBinaryArray, LargeBinary)? + } + DataType::Utf8 => typed_cast!(array, index, StringArray, Utf8)?, + DataType::LargeUtf8 => { + typed_cast!(array, index, LargeStringArray, LargeUtf8)? } - DataType::Utf8 => typed_cast!(array, index, StringArray, Utf8), - DataType::LargeUtf8 => typed_cast!(array, index, LargeStringArray, LargeUtf8), DataType::List(_) => { let list_array = as_list_array(array); let nested_array = list_array.value(index); @@ -2071,70 +2140,58 @@ impl ScalarValue { ScalarValue::List(arr) } - DataType::Date32 => { - typed_cast!(array, index, Date32Array, Date32) - } - DataType::Date64 => { - typed_cast!(array, index, Date64Array, Date64) - } + DataType::Date32 => typed_cast!(array, index, Date32Array, Date32)?, + DataType::Date64 => typed_cast!(array, index, Date64Array, Date64)?, DataType::Time32(TimeUnit::Second) => { - typed_cast!(array, index, Time32SecondArray, Time32Second) + typed_cast!(array, index, Time32SecondArray, Time32Second)? } DataType::Time32(TimeUnit::Millisecond) => { - typed_cast!(array, index, Time32MillisecondArray, Time32Millisecond) + typed_cast!(array, index, Time32MillisecondArray, Time32Millisecond)? } DataType::Time64(TimeUnit::Microsecond) => { - typed_cast!(array, index, Time64MicrosecondArray, Time64Microsecond) + typed_cast!(array, index, Time64MicrosecondArray, Time64Microsecond)? } DataType::Time64(TimeUnit::Nanosecond) => { - typed_cast!(array, index, Time64NanosecondArray, Time64Nanosecond) - } - DataType::Timestamp(TimeUnit::Second, tz_opt) => { - typed_cast_tz!( - array, - index, - TimestampSecondArray, - TimestampSecond, - tz_opt - ) - } - DataType::Timestamp(TimeUnit::Millisecond, tz_opt) => { - typed_cast_tz!( - array, - index, - TimestampMillisecondArray, - TimestampMillisecond, - tz_opt - ) - } - DataType::Timestamp(TimeUnit::Microsecond, tz_opt) => { - typed_cast_tz!( - array, - index, - TimestampMicrosecondArray, - TimestampMicrosecond, - tz_opt - ) - } - DataType::Timestamp(TimeUnit::Nanosecond, tz_opt) => { - typed_cast_tz!( - array, - index, - TimestampNanosecondArray, - TimestampNanosecond, - tz_opt - ) + typed_cast!(array, index, Time64NanosecondArray, Time64Nanosecond)? } + DataType::Timestamp(TimeUnit::Second, tz_opt) => typed_cast_tz!( + array, + index, + TimestampSecondArray, + TimestampSecond, + tz_opt + )?, + DataType::Timestamp(TimeUnit::Millisecond, tz_opt) => typed_cast_tz!( + array, + index, + TimestampMillisecondArray, + TimestampMillisecond, + tz_opt + )?, + DataType::Timestamp(TimeUnit::Microsecond, tz_opt) => typed_cast_tz!( + array, + index, + TimestampMicrosecondArray, + TimestampMicrosecond, + tz_opt + )?, + DataType::Timestamp(TimeUnit::Nanosecond, tz_opt) => typed_cast_tz!( + array, + index, + TimestampNanosecondArray, + TimestampNanosecond, + tz_opt + )?, DataType::Dictionary(key_type, _) => { let (values_array, values_index) = match key_type.as_ref() { - DataType::Int8 => get_dict_value::(array, index), - DataType::Int16 => get_dict_value::(array, index), - DataType::Int32 => get_dict_value::(array, index), - DataType::Int64 => get_dict_value::(array, index), - DataType::UInt8 => get_dict_value::(array, index), - DataType::UInt16 => get_dict_value::(array, index), - DataType::UInt32 => get_dict_value::(array, index), - DataType::UInt64 => get_dict_value::(array, index), + DataType::Int8 => get_dict_value::(array, index)?, + DataType::Int16 => get_dict_value::(array, index)?, + DataType::Int32 => get_dict_value::(array, index)?, + DataType::Int64 => get_dict_value::(array, index)?, + DataType::UInt8 => get_dict_value::(array, index)?, + DataType::UInt16 => get_dict_value::(array, index)?, + DataType::UInt32 => get_dict_value::(array, index)?, + DataType::UInt64 => get_dict_value::(array, index)?, _ => unreachable!("Invalid dictionary keys type: {:?}", key_type), }; // look up the index in the values dictionary @@ -2173,31 +2230,29 @@ impl ScalarValue { ) } DataType::Interval(IntervalUnit::DayTime) => { - typed_cast!(array, index, IntervalDayTimeArray, IntervalDayTime) + typed_cast!(array, index, IntervalDayTimeArray, IntervalDayTime)? } DataType::Interval(IntervalUnit::YearMonth) => { - typed_cast!(array, index, IntervalYearMonthArray, IntervalYearMonth) - } - DataType::Interval(IntervalUnit::MonthDayNano) => { - typed_cast!( - array, - index, - IntervalMonthDayNanoArray, - IntervalMonthDayNano - ) + typed_cast!(array, index, IntervalYearMonthArray, IntervalYearMonth)? } + DataType::Interval(IntervalUnit::MonthDayNano) => typed_cast!( + array, + index, + IntervalMonthDayNanoArray, + IntervalMonthDayNano + )?, DataType::Duration(TimeUnit::Second) => { - typed_cast!(array, index, DurationSecondArray, DurationSecond) + typed_cast!(array, index, DurationSecondArray, DurationSecond)? } DataType::Duration(TimeUnit::Millisecond) => { - typed_cast!(array, index, DurationMillisecondArray, DurationMillisecond) + typed_cast!(array, index, DurationMillisecondArray, DurationMillisecond)? } DataType::Duration(TimeUnit::Microsecond) => { - typed_cast!(array, index, DurationMicrosecondArray, DurationMicrosecond) + typed_cast!(array, index, DurationMicrosecondArray, DurationMicrosecond)? } DataType::Duration(TimeUnit::Nanosecond) => { - typed_cast!(array, index, DurationNanosecondArray, DurationNanosecond) + typed_cast!(array, index, DurationNanosecondArray, DurationNanosecond)? } other => { @@ -2215,7 +2270,7 @@ impl ScalarValue { safe: false, format_options: Default::default(), }; - let cast_arr = cast_with_options(&value.to_array(), target_type, &cast_options)?; + let cast_arr = cast_with_options(&value.to_array()?, target_type, &cast_options)?; ScalarValue::try_from_array(&cast_arr, 0) } @@ -2273,9 +2328,21 @@ impl ScalarValue { /// /// This function has a few narrow usescases such as hash table key /// comparisons where comparing a single row at a time is necessary. + /// + /// # Errors + /// + /// Errors if + /// - it fails to downcast `array` to the data type of `self` + /// - `self` is a `Fixedsizelist` + /// - `self` is a `List` + /// - `self` is a `Struct` + /// + /// # Panics + /// + /// Panics if `self` is a dictionary with invalid key type #[inline] - pub fn eq_array(&self, array: &ArrayRef, index: usize) -> bool { - match self { + pub fn eq_array(&self, array: &ArrayRef, index: usize) -> Result { + Ok(match self { ScalarValue::Decimal128(v, precision, scale) => { ScalarValue::eq_array_decimal( array, @@ -2283,8 +2350,7 @@ impl ScalarValue { v.as_ref(), *precision, *scale, - ) - .unwrap() + )? } ScalarValue::Decimal256(v, precision, scale) => { ScalarValue::eq_array_decimal256( @@ -2293,119 +2359,132 @@ impl ScalarValue { v.as_ref(), *precision, *scale, - ) - .unwrap() + )? } ScalarValue::Boolean(val) => { - eq_array_primitive!(array, index, BooleanArray, val) + eq_array_primitive!(array, index, BooleanArray, val)? } ScalarValue::Float32(val) => { - eq_array_primitive!(array, index, Float32Array, val) + eq_array_primitive!(array, index, Float32Array, val)? } ScalarValue::Float64(val) => { - eq_array_primitive!(array, index, Float64Array, val) + eq_array_primitive!(array, index, Float64Array, val)? + } + ScalarValue::Int8(val) => eq_array_primitive!(array, index, Int8Array, val)?, + ScalarValue::Int16(val) => { + eq_array_primitive!(array, index, Int16Array, val)? + } + ScalarValue::Int32(val) => { + eq_array_primitive!(array, index, Int32Array, val)? + } + ScalarValue::Int64(val) => { + eq_array_primitive!(array, index, Int64Array, val)? + } + ScalarValue::UInt8(val) => { + eq_array_primitive!(array, index, UInt8Array, val)? } - ScalarValue::Int8(val) => eq_array_primitive!(array, index, Int8Array, val), - ScalarValue::Int16(val) => eq_array_primitive!(array, index, Int16Array, val), - ScalarValue::Int32(val) => eq_array_primitive!(array, index, Int32Array, val), - ScalarValue::Int64(val) => eq_array_primitive!(array, index, Int64Array, val), - ScalarValue::UInt8(val) => eq_array_primitive!(array, index, UInt8Array, val), ScalarValue::UInt16(val) => { - eq_array_primitive!(array, index, UInt16Array, val) + eq_array_primitive!(array, index, UInt16Array, val)? } ScalarValue::UInt32(val) => { - eq_array_primitive!(array, index, UInt32Array, val) + eq_array_primitive!(array, index, UInt32Array, val)? } ScalarValue::UInt64(val) => { - eq_array_primitive!(array, index, UInt64Array, val) + eq_array_primitive!(array, index, UInt64Array, val)? + } + ScalarValue::Utf8(val) => { + eq_array_primitive!(array, index, StringArray, val)? } - ScalarValue::Utf8(val) => eq_array_primitive!(array, index, StringArray, val), ScalarValue::LargeUtf8(val) => { - eq_array_primitive!(array, index, LargeStringArray, val) + eq_array_primitive!(array, index, LargeStringArray, val)? } ScalarValue::Binary(val) => { - eq_array_primitive!(array, index, BinaryArray, val) + eq_array_primitive!(array, index, BinaryArray, val)? } ScalarValue::FixedSizeBinary(_, val) => { - eq_array_primitive!(array, index, FixedSizeBinaryArray, val) + eq_array_primitive!(array, index, FixedSizeBinaryArray, val)? } ScalarValue::LargeBinary(val) => { - eq_array_primitive!(array, index, LargeBinaryArray, val) + eq_array_primitive!(array, index, LargeBinaryArray, val)? + } + ScalarValue::Fixedsizelist(..) => { + return _not_impl_err!("FixedSizeList is not supported yet") } - ScalarValue::Fixedsizelist(..) => unimplemented!(), - ScalarValue::List(_) => unimplemented!("ListArr"), + ScalarValue::List(_) => return _not_impl_err!("List is not supported yet"), ScalarValue::Date32(val) => { - eq_array_primitive!(array, index, Date32Array, val) + eq_array_primitive!(array, index, Date32Array, val)? } ScalarValue::Date64(val) => { - eq_array_primitive!(array, index, Date64Array, val) + eq_array_primitive!(array, index, Date64Array, val)? } ScalarValue::Time32Second(val) => { - eq_array_primitive!(array, index, Time32SecondArray, val) + eq_array_primitive!(array, index, Time32SecondArray, val)? } ScalarValue::Time32Millisecond(val) => { - eq_array_primitive!(array, index, Time32MillisecondArray, val) + eq_array_primitive!(array, index, Time32MillisecondArray, val)? } ScalarValue::Time64Microsecond(val) => { - eq_array_primitive!(array, index, Time64MicrosecondArray, val) + eq_array_primitive!(array, index, Time64MicrosecondArray, val)? } ScalarValue::Time64Nanosecond(val) => { - eq_array_primitive!(array, index, Time64NanosecondArray, val) + eq_array_primitive!(array, index, Time64NanosecondArray, val)? } ScalarValue::TimestampSecond(val, _) => { - eq_array_primitive!(array, index, TimestampSecondArray, val) + eq_array_primitive!(array, index, TimestampSecondArray, val)? } ScalarValue::TimestampMillisecond(val, _) => { - eq_array_primitive!(array, index, TimestampMillisecondArray, val) + eq_array_primitive!(array, index, TimestampMillisecondArray, val)? } ScalarValue::TimestampMicrosecond(val, _) => { - eq_array_primitive!(array, index, TimestampMicrosecondArray, val) + eq_array_primitive!(array, index, TimestampMicrosecondArray, val)? } ScalarValue::TimestampNanosecond(val, _) => { - eq_array_primitive!(array, index, TimestampNanosecondArray, val) + eq_array_primitive!(array, index, TimestampNanosecondArray, val)? } ScalarValue::IntervalYearMonth(val) => { - eq_array_primitive!(array, index, IntervalYearMonthArray, val) + eq_array_primitive!(array, index, IntervalYearMonthArray, val)? } ScalarValue::IntervalDayTime(val) => { - eq_array_primitive!(array, index, IntervalDayTimeArray, val) + eq_array_primitive!(array, index, IntervalDayTimeArray, val)? } ScalarValue::IntervalMonthDayNano(val) => { - eq_array_primitive!(array, index, IntervalMonthDayNanoArray, val) + eq_array_primitive!(array, index, IntervalMonthDayNanoArray, val)? } ScalarValue::DurationSecond(val) => { - eq_array_primitive!(array, index, DurationSecondArray, val) + eq_array_primitive!(array, index, DurationSecondArray, val)? } ScalarValue::DurationMillisecond(val) => { - eq_array_primitive!(array, index, DurationMillisecondArray, val) + eq_array_primitive!(array, index, DurationMillisecondArray, val)? } ScalarValue::DurationMicrosecond(val) => { - eq_array_primitive!(array, index, DurationMicrosecondArray, val) + eq_array_primitive!(array, index, DurationMicrosecondArray, val)? } ScalarValue::DurationNanosecond(val) => { - eq_array_primitive!(array, index, DurationNanosecondArray, val) + eq_array_primitive!(array, index, DurationNanosecondArray, val)? + } + ScalarValue::Struct(_, _) => { + return _not_impl_err!("Struct is not supported yet") } - ScalarValue::Struct(_, _) => unimplemented!(), ScalarValue::Dictionary(key_type, v) => { let (values_array, values_index) = match key_type.as_ref() { - DataType::Int8 => get_dict_value::(array, index), - DataType::Int16 => get_dict_value::(array, index), - DataType::Int32 => get_dict_value::(array, index), - DataType::Int64 => get_dict_value::(array, index), - DataType::UInt8 => get_dict_value::(array, index), - DataType::UInt16 => get_dict_value::(array, index), - DataType::UInt32 => get_dict_value::(array, index), - DataType::UInt64 => get_dict_value::(array, index), + DataType::Int8 => get_dict_value::(array, index)?, + DataType::Int16 => get_dict_value::(array, index)?, + DataType::Int32 => get_dict_value::(array, index)?, + DataType::Int64 => get_dict_value::(array, index)?, + DataType::UInt8 => get_dict_value::(array, index)?, + DataType::UInt16 => get_dict_value::(array, index)?, + DataType::UInt32 => get_dict_value::(array, index)?, + DataType::UInt64 => get_dict_value::(array, index)?, _ => unreachable!("Invalid dictionary keys type: {:?}", key_type), }; // was the value in the array non null? match values_index { - Some(values_index) => v.eq_array(values_array, values_index), + Some(values_index) => v.eq_array(values_array, values_index)?, None => v.is_null(), } } ScalarValue::Null => array.is_null(index), - } + }) } /// Estimate size if bytes including `Self`. For values with internal containers such as `String` @@ -2785,6 +2864,11 @@ macro_rules! format_option { }}; } +// Implement Display trait for ScalarValue +// +// # Panics +// +// Panics if there is an error when creating a visual representation of columns via `arrow::util::pretty` impl fmt::Display for ScalarValue { fn fmt(&self, f: &mut fmt::Formatter) -> fmt::Result { match self { @@ -3031,7 +3115,9 @@ mod tests { ])]); let sv = ScalarValue::List(Arc::new(arr)); - let actual_arr = sv.to_array_of_size(2); + let actual_arr = sv + .to_array_of_size(2) + .expect("Failed to convert to array of size"); let actual_list_arr = as_list_array(&actual_arr); let arr = ListArray::from_iter_primitive::(vec![ @@ -3238,8 +3324,8 @@ mod tests { { let scalar_result = left.add_checked(&right); - let left_array = left.to_array(); - let right_array = right.to_array(); + let left_array = left.to_array().expect("Failed to convert to array"); + let right_array = right.to_array().expect("Failed to convert to array"); let arrow_left_array = left_array.as_primitive::(); let arrow_right_array = right_array.as_primitive::(); let arrow_result = kernels::numeric::add(arrow_left_array, arrow_right_array); @@ -3287,22 +3373,30 @@ mod tests { } // decimal scalar to array - let array = decimal_value.to_array(); + let array = decimal_value + .to_array() + .expect("Failed to convert to array"); let array = as_decimal128_array(&array)?; assert_eq!(1, array.len()); assert_eq!(DataType::Decimal128(10, 1), array.data_type().clone()); assert_eq!(123i128, array.value(0)); // decimal scalar to array with size - let array = decimal_value.to_array_of_size(10); + let array = decimal_value + .to_array_of_size(10) + .expect("Failed to convert to array of size"); let array_decimal = as_decimal128_array(&array)?; assert_eq!(10, array.len()); assert_eq!(DataType::Decimal128(10, 1), array.data_type().clone()); assert_eq!(123i128, array_decimal.value(0)); assert_eq!(123i128, array_decimal.value(9)); // test eq array - assert!(decimal_value.eq_array(&array, 1)); - assert!(decimal_value.eq_array(&array, 5)); + assert!(decimal_value + .eq_array(&array, 1) + .expect("Failed to compare arrays")); + assert!(decimal_value + .eq_array(&array, 5) + .expect("Failed to compare arrays")); // test try from array assert_eq!( decimal_value, @@ -3349,13 +3443,16 @@ mod tests { assert!(ScalarValue::try_new_decimal128(1, 10, 2) .unwrap() - .eq_array(&array, 0)); + .eq_array(&array, 0) + .expect("Failed to compare arrays")); assert!(ScalarValue::try_new_decimal128(2, 10, 2) .unwrap() - .eq_array(&array, 1)); + .eq_array(&array, 1) + .expect("Failed to compare arrays")); assert!(ScalarValue::try_new_decimal128(3, 10, 2) .unwrap() - .eq_array(&array, 2)); + .eq_array(&array, 2) + .expect("Failed to compare arrays")); assert_eq!( ScalarValue::Decimal128(None, 10, 2), ScalarValue::try_from_array(&array, 3).unwrap() @@ -3442,14 +3539,14 @@ mod tests { #[test] fn scalar_value_to_array_u64() -> Result<()> { let value = ScalarValue::UInt64(Some(13u64)); - let array = value.to_array(); + let array = value.to_array().expect("Failed to convert to array"); let array = as_uint64_array(&array)?; assert_eq!(array.len(), 1); assert!(!array.is_null(0)); assert_eq!(array.value(0), 13); let value = ScalarValue::UInt64(None); - let array = value.to_array(); + let array = value.to_array().expect("Failed to convert to array"); let array = as_uint64_array(&array)?; assert_eq!(array.len(), 1); assert!(array.is_null(0)); @@ -3459,14 +3556,14 @@ mod tests { #[test] fn scalar_value_to_array_u32() -> Result<()> { let value = ScalarValue::UInt32(Some(13u32)); - let array = value.to_array(); + let array = value.to_array().expect("Failed to convert to array"); let array = as_uint32_array(&array)?; assert_eq!(array.len(), 1); assert!(!array.is_null(0)); assert_eq!(array.value(0), 13); let value = ScalarValue::UInt32(None); - let array = value.to_array(); + let array = value.to_array().expect("Failed to convert to array"); let array = as_uint32_array(&array)?; assert_eq!(array.len(), 1); assert!(array.is_null(0)); @@ -4025,7 +4122,9 @@ mod tests { for (index, scalar) in scalars.into_iter().enumerate() { assert!( - scalar.eq_array(&array, index), + scalar + .eq_array(&array, index) + .expect("Failed to compare arrays"), "Expected {scalar:?} to be equal to {array:?} at index {index}" ); @@ -4033,7 +4132,7 @@ mod tests { for other_index in 0..array.len() { if index != other_index { assert!( - !scalar.eq_array(&array, other_index), + !scalar.eq_array(&array, other_index).expect("Failed to compare arrays"), "Expected {scalar:?} to be NOT equal to {array:?} at index {other_index}" ); } @@ -4136,7 +4235,9 @@ mod tests { ); // Convert to length-2 array - let array = scalar.to_array_of_size(2); + let array = scalar + .to_array_of_size(2) + .expect("Failed to convert to array of size"); let expected = Arc::new(StructArray::from(vec![ ( @@ -4570,7 +4671,7 @@ mod tests { DataType::Timestamp(TimeUnit::Nanosecond, Some("UTC".into())) ); - let array = scalar.to_array(); + let array = scalar.to_array().expect("Failed to convert to array"); assert_eq!(array.len(), 1); assert_eq!( array.data_type(), @@ -4607,7 +4708,7 @@ mod tests { // mimics how casting work on scalar values by `casting` `scalar` to `desired_type` fn check_scalar_cast(scalar: ScalarValue, desired_type: DataType) { // convert from scalar --> Array to call cast - let scalar_array = scalar.to_array(); + let scalar_array = scalar.to_array().expect("Failed to convert to array"); // cast the actual value let cast_array = kernels::cast::cast(&scalar_array, &desired_type).unwrap(); @@ -4616,7 +4717,9 @@ mod tests { assert_eq!(cast_scalar.data_type(), desired_type); // Some time later the "cast" scalar is turned back into an array: - let array = cast_scalar.to_array_of_size(10); + let array = cast_scalar + .to_array_of_size(10) + .expect("Failed to convert to array of size"); // The datatype should be "Dictionary" but is actually Utf8!!! assert_eq!(array.data_type(), &desired_type) @@ -5065,7 +5168,8 @@ mod tests { let arrays = scalars .iter() .map(ScalarValue::to_array) - .collect::>(); + .collect::>>() + .expect("Failed to convert to array"); let arrays = arrays.iter().map(|a| a.as_ref()).collect::>(); let array = concat(&arrays).unwrap(); check_array(array); diff --git a/datafusion/core/benches/scalar.rs b/datafusion/core/benches/scalar.rs index 30f21a964d5f..540f7212e96e 100644 --- a/datafusion/core/benches/scalar.rs +++ b/datafusion/core/benches/scalar.rs @@ -22,7 +22,15 @@ fn criterion_benchmark(c: &mut Criterion) { c.bench_function("to_array_of_size 100000", |b| { let scalar = ScalarValue::Int32(Some(100)); - b.iter(|| assert_eq!(scalar.to_array_of_size(100000).null_count(), 0)) + b.iter(|| { + assert_eq!( + scalar + .to_array_of_size(100000) + .expect("Failed to convert to array of size") + .null_count(), + 0 + ) + }) }); } diff --git a/datafusion/core/src/datasource/listing/helpers.rs b/datafusion/core/src/datasource/listing/helpers.rs index d6a0add9b253..986e54ebbe85 100644 --- a/datafusion/core/src/datasource/listing/helpers.rs +++ b/datafusion/core/src/datasource/listing/helpers.rs @@ -276,7 +276,10 @@ async fn prune_partitions( // Applies `filter` to `batch` returning `None` on error let do_filter = |filter| -> Option { let expr = create_physical_expr(filter, &df_schema, &schema, &props).ok()?; - Some(expr.evaluate(&batch).ok()?.into_array(partitions.len())) + expr.evaluate(&batch) + .ok()? + .into_array(partitions.len()) + .ok() }; //.Compute the conjunction of the filters, ignoring errors diff --git a/datafusion/core/src/datasource/physical_plan/file_scan_config.rs b/datafusion/core/src/datasource/physical_plan/file_scan_config.rs index 3efb0df9df7c..68e996391cc3 100644 --- a/datafusion/core/src/datasource/physical_plan/file_scan_config.rs +++ b/datafusion/core/src/datasource/physical_plan/file_scan_config.rs @@ -336,7 +336,7 @@ impl PartitionColumnProjector { &mut self.key_buffer_cache, partition_value.as_ref(), file_batch.num_rows(), - ), + )?, ) } @@ -396,11 +396,11 @@ fn create_dict_array( dict_val: &ScalarValue, len: usize, data_type: DataType, -) -> ArrayRef +) -> Result where T: ArrowNativeType, { - let dict_vals = dict_val.to_array(); + let dict_vals = dict_val.to_array()?; let sliced_key_buffer = buffer_gen.get_buffer(len); @@ -409,16 +409,16 @@ where .len(len) .add_buffer(sliced_key_buffer); builder = builder.add_child_data(dict_vals.to_data()); - Arc::new(DictionaryArray::::from( + Ok(Arc::new(DictionaryArray::::from( builder.build().unwrap(), - )) + ))) } fn create_output_array( key_buffer_cache: &mut ZeroBufferGenerators, val: &ScalarValue, len: usize, -) -> ArrayRef { +) -> Result { if let ScalarValue::Dictionary(key_type, dict_val) = &val { match key_type.as_ref() { DataType::Int8 => { diff --git a/datafusion/core/src/datasource/physical_plan/parquet/row_filter.rs b/datafusion/core/src/datasource/physical_plan/parquet/row_filter.rs index 0f4b09caeded..5fe0a0a13a73 100644 --- a/datafusion/core/src/datasource/physical_plan/parquet/row_filter.rs +++ b/datafusion/core/src/datasource/physical_plan/parquet/row_filter.rs @@ -126,7 +126,7 @@ impl ArrowPredicate for DatafusionArrowPredicate { match self .physical_expr .evaluate(&batch) - .map(|v| v.into_array(batch.num_rows())) + .and_then(|v| v.into_array(batch.num_rows())) { Ok(array) => { let bool_arr = as_boolean_array(&array)?.clone(); diff --git a/datafusion/core/src/datasource/physical_plan/parquet/row_groups.rs b/datafusion/core/src/datasource/physical_plan/parquet/row_groups.rs index 91bceed91602..dc6ef50bc101 100644 --- a/datafusion/core/src/datasource/physical_plan/parquet/row_groups.rs +++ b/datafusion/core/src/datasource/physical_plan/parquet/row_groups.rs @@ -405,7 +405,7 @@ macro_rules! get_min_max_values { .flatten() // column either didn't have statistics at all or didn't have min/max values .or_else(|| Some(null_scalar.clone())) - .map(|s| s.to_array()) + .and_then(|s| s.to_array().ok()) }} } @@ -425,7 +425,7 @@ macro_rules! get_null_count_values { }, ); - Some(value.to_array()) + value.to_array().ok() }}; } diff --git a/datafusion/expr/src/columnar_value.rs b/datafusion/expr/src/columnar_value.rs index c72aae69c831..7a2883928169 100644 --- a/datafusion/expr/src/columnar_value.rs +++ b/datafusion/expr/src/columnar_value.rs @@ -20,7 +20,7 @@ use arrow::array::ArrayRef; use arrow::array::NullArray; use arrow::datatypes::DataType; -use datafusion_common::ScalarValue; +use datafusion_common::{Result, ScalarValue}; use std::sync::Arc; /// Represents the result of evaluating an expression: either a single @@ -47,11 +47,15 @@ impl ColumnarValue { /// Convert a columnar value into an ArrayRef. [`Self::Scalar`] is /// converted by repeating the same scalar multiple times. - pub fn into_array(self, num_rows: usize) -> ArrayRef { - match self { + /// + /// # Errors + /// + /// Errors if `self` is a Scalar that fails to be converted into an array of size + pub fn into_array(self, num_rows: usize) -> Result { + Ok(match self { ColumnarValue::Array(array) => array, - ColumnarValue::Scalar(scalar) => scalar.to_array_of_size(num_rows), - } + ColumnarValue::Scalar(scalar) => scalar.to_array_of_size(num_rows)?, + }) } /// null columnar values are implemented as a null array in order to pass batch diff --git a/datafusion/expr/src/window_state.rs b/datafusion/expr/src/window_state.rs index 4ea9ecea5fc6..de88396d9b0e 100644 --- a/datafusion/expr/src/window_state.rs +++ b/datafusion/expr/src/window_state.rs @@ -98,7 +98,7 @@ impl WindowAggState { } pub fn new(out_type: &DataType) -> Result { - let empty_out_col = ScalarValue::try_from(out_type)?.to_array_of_size(0); + let empty_out_col = ScalarValue::try_from(out_type)?.to_array_of_size(0)?; Ok(Self { window_frame_range: Range { start: 0, end: 0 }, window_frame_ctx: None, diff --git a/datafusion/optimizer/src/unwrap_cast_in_comparison.rs b/datafusion/optimizer/src/unwrap_cast_in_comparison.rs index 468981a5fb0c..907c12b7afb1 100644 --- a/datafusion/optimizer/src/unwrap_cast_in_comparison.rs +++ b/datafusion/optimizer/src/unwrap_cast_in_comparison.rs @@ -1089,8 +1089,12 @@ mod tests { // Verify that calling the arrow // cast kernel yields the same results // input array - let literal_array = literal.to_array_of_size(1); - let expected_array = expected_value.to_array_of_size(1); + let literal_array = literal + .to_array_of_size(1) + .expect("Failed to convert to array of size"); + let expected_array = expected_value + .to_array_of_size(1) + .expect("Failed to convert to array of size"); let cast_array = cast_with_options( &literal_array, &target_type, diff --git a/datafusion/physical-expr/src/aggregate/correlation.rs b/datafusion/physical-expr/src/aggregate/correlation.rs index 475bfa4ce0da..61f2db5c8ef9 100644 --- a/datafusion/physical-expr/src/aggregate/correlation.rs +++ b/datafusion/physical-expr/src/aggregate/correlation.rs @@ -505,13 +505,17 @@ mod tests { let values1 = expr1 .iter() - .map(|e| e.evaluate(batch1)) - .map(|r| r.map(|v| v.into_array(batch1.num_rows()))) + .map(|e| { + e.evaluate(batch1) + .and_then(|v| v.into_array(batch1.num_rows())) + }) .collect::>>()?; let values2 = expr2 .iter() - .map(|e| e.evaluate(batch2)) - .map(|r| r.map(|v| v.into_array(batch2.num_rows()))) + .map(|e| { + e.evaluate(batch2) + .and_then(|v| v.into_array(batch2.num_rows())) + }) .collect::>>()?; accum1.update_batch(&values1)?; accum2.update_batch(&values2)?; diff --git a/datafusion/physical-expr/src/aggregate/covariance.rs b/datafusion/physical-expr/src/aggregate/covariance.rs index 5e589d4e39fd..0f838eb6fa1c 100644 --- a/datafusion/physical-expr/src/aggregate/covariance.rs +++ b/datafusion/physical-expr/src/aggregate/covariance.rs @@ -754,13 +754,17 @@ mod tests { let values1 = expr1 .iter() - .map(|e| e.evaluate(batch1)) - .map(|r| r.map(|v| v.into_array(batch1.num_rows()))) + .map(|e| { + e.evaluate(batch1) + .and_then(|v| v.into_array(batch1.num_rows())) + }) .collect::>>()?; let values2 = expr2 .iter() - .map(|e| e.evaluate(batch2)) - .map(|r| r.map(|v| v.into_array(batch2.num_rows()))) + .map(|e| { + e.evaluate(batch2) + .and_then(|v| v.into_array(batch2.num_rows())) + }) .collect::>>()?; accum1.update_batch(&values1)?; accum2.update_batch(&values2)?; diff --git a/datafusion/physical-expr/src/aggregate/first_last.rs b/datafusion/physical-expr/src/aggregate/first_last.rs index a4e0a6dc49a9..0dc27dede8b6 100644 --- a/datafusion/physical-expr/src/aggregate/first_last.rs +++ b/datafusion/physical-expr/src/aggregate/first_last.rs @@ -587,7 +587,10 @@ mod tests { let mut states = vec![]; for idx in 0..state1.len() { - states.push(concat(&[&state1[idx].to_array(), &state2[idx].to_array()])?); + states.push(concat(&[ + &state1[idx].to_array()?, + &state2[idx].to_array()?, + ])?); } let mut first_accumulator = @@ -614,7 +617,10 @@ mod tests { let mut states = vec![]; for idx in 0..state1.len() { - states.push(concat(&[&state1[idx].to_array(), &state2[idx].to_array()])?); + states.push(concat(&[ + &state1[idx].to_array()?, + &state2[idx].to_array()?, + ])?); } let mut last_accumulator = diff --git a/datafusion/physical-expr/src/aggregate/stddev.rs b/datafusion/physical-expr/src/aggregate/stddev.rs index 330507d6ffa6..64e19ef502c7 100644 --- a/datafusion/physical-expr/src/aggregate/stddev.rs +++ b/datafusion/physical-expr/src/aggregate/stddev.rs @@ -445,13 +445,17 @@ mod tests { let values1 = expr1 .iter() - .map(|e| e.evaluate(batch1)) - .map(|r| r.map(|v| v.into_array(batch1.num_rows()))) + .map(|e| { + e.evaluate(batch1) + .and_then(|v| v.into_array(batch1.num_rows())) + }) .collect::>>()?; let values2 = expr2 .iter() - .map(|e| e.evaluate(batch2)) - .map(|r| r.map(|v| v.into_array(batch2.num_rows()))) + .map(|e| { + e.evaluate(batch2) + .and_then(|v| v.into_array(batch2.num_rows())) + }) .collect::>>()?; accum1.update_batch(&values1)?; accum2.update_batch(&values2)?; diff --git a/datafusion/physical-expr/src/aggregate/utils.rs b/datafusion/physical-expr/src/aggregate/utils.rs index da3a52713231..e5421ef5ab7e 100644 --- a/datafusion/physical-expr/src/aggregate/utils.rs +++ b/datafusion/physical-expr/src/aggregate/utils.rs @@ -36,11 +36,11 @@ use std::sync::Arc; pub fn get_accum_scalar_values_as_arrays( accum: &dyn Accumulator, ) -> Result> { - Ok(accum + accum .state()? .iter() .map(|s| s.to_array_of_size(1)) - .collect::>()) + .collect::>>() } /// Computes averages for `Decimal128`/`Decimal256` values, checking for overflow diff --git a/datafusion/physical-expr/src/aggregate/variance.rs b/datafusion/physical-expr/src/aggregate/variance.rs index a720dd833a87..d82c5ad5626f 100644 --- a/datafusion/physical-expr/src/aggregate/variance.rs +++ b/datafusion/physical-expr/src/aggregate/variance.rs @@ -519,13 +519,17 @@ mod tests { let values1 = expr1 .iter() - .map(|e| e.evaluate(batch1)) - .map(|r| r.map(|v| v.into_array(batch1.num_rows()))) + .map(|e| { + e.evaluate(batch1) + .and_then(|v| v.into_array(batch1.num_rows())) + }) .collect::>>()?; let values2 = expr2 .iter() - .map(|e| e.evaluate(batch2)) - .map(|r| r.map(|v| v.into_array(batch2.num_rows()))) + .map(|e| { + e.evaluate(batch2) + .and_then(|v| v.into_array(batch2.num_rows())) + }) .collect::>>()?; accum1.update_batch(&values1)?; accum2.update_batch(&values2)?; diff --git a/datafusion/physical-expr/src/conditional_expressions.rs b/datafusion/physical-expr/src/conditional_expressions.rs index 37adb2d71ce8..a9a25ffe2ec1 100644 --- a/datafusion/physical-expr/src/conditional_expressions.rs +++ b/datafusion/physical-expr/src/conditional_expressions.rs @@ -54,7 +54,7 @@ pub fn coalesce(args: &[ColumnarValue]) -> Result { if value.is_null() { continue; } else { - let last_value = value.to_array_of_size(size); + let last_value = value.to_array_of_size(size)?; current_value = zip(&remainder, &last_value, current_value.as_ref())?; break; diff --git a/datafusion/physical-expr/src/datetime_expressions.rs b/datafusion/physical-expr/src/datetime_expressions.rs index 3b61e7f48d59..5b597de78ac9 100644 --- a/datafusion/physical-expr/src/datetime_expressions.rs +++ b/datafusion/physical-expr/src/datetime_expressions.rs @@ -852,7 +852,7 @@ pub fn date_part(args: &[ColumnarValue]) -> Result { let array = match array { ColumnarValue::Array(array) => array.clone(), - ColumnarValue::Scalar(scalar) => scalar.to_array(), + ColumnarValue::Scalar(scalar) => scalar.to_array()?, }; let arr = match date_part.to_lowercase().as_str() { diff --git a/datafusion/physical-expr/src/expressions/binary.rs b/datafusion/physical-expr/src/expressions/binary.rs index 63fa98011fdd..0a05a479e5a7 100644 --- a/datafusion/physical-expr/src/expressions/binary.rs +++ b/datafusion/physical-expr/src/expressions/binary.rs @@ -304,8 +304,8 @@ impl PhysicalExpr for BinaryExpr { // if both arrays or both literals - extract arrays and continue execution let (left, right) = ( - lhs.into_array(batch.num_rows()), - rhs.into_array(batch.num_rows()), + lhs.into_array(batch.num_rows())?, + rhs.into_array(batch.num_rows())?, ); self.evaluate_with_resolved_args(left, &left_data_type, right, &right_data_type) .map(ColumnarValue::Array) @@ -597,7 +597,10 @@ mod tests { let batch = RecordBatch::try_new(Arc::new(schema), vec![Arc::new(a), Arc::new(b)])?; - let result = lt.evaluate(&batch)?.into_array(batch.num_rows()); + let result = lt + .evaluate(&batch)? + .into_array(batch.num_rows()) + .expect("Failed to convert to array"); assert_eq!(result.len(), 5); let expected = [false, false, true, true, true]; @@ -641,7 +644,10 @@ mod tests { assert_eq!("a@0 < b@1 OR a@0 = b@1", format!("{expr}")); - let result = expr.evaluate(&batch)?.into_array(batch.num_rows()); + let result = expr + .evaluate(&batch)? + .into_array(batch.num_rows()) + .expect("Failed to convert to array"); assert_eq!(result.len(), 5); let expected = [true, true, false, true, false]; @@ -685,7 +691,7 @@ mod tests { assert_eq!(expression.data_type(&schema)?, $C_TYPE); // compute - let result = expression.evaluate(&batch)?.into_array(batch.num_rows()); + let result = expression.evaluate(&batch)?.into_array(batch.num_rows()).expect("Failed to convert to array"); // verify that the array's data_type is correct assert_eq!(*result.data_type(), $C_TYPE); @@ -2138,7 +2144,10 @@ mod tests { let arithmetic_op = binary_op(col("a", &schema)?, op, col("b", &schema)?, &schema)?; let batch = RecordBatch::try_new(schema, data)?; - let result = arithmetic_op.evaluate(&batch)?.into_array(batch.num_rows()); + let result = arithmetic_op + .evaluate(&batch)? + .into_array(batch.num_rows()) + .expect("Failed to convert to array"); assert_eq!(result.as_ref(), &expected); Ok(()) @@ -2154,7 +2163,10 @@ mod tests { let lit = Arc::new(Literal::new(literal)); let arithmetic_op = binary_op(col("a", &schema)?, op, lit, &schema)?; let batch = RecordBatch::try_new(schema, data)?; - let result = arithmetic_op.evaluate(&batch)?.into_array(batch.num_rows()); + let result = arithmetic_op + .evaluate(&batch)? + .into_array(batch.num_rows()) + .expect("Failed to convert to array"); assert_eq!(&result, &expected); Ok(()) @@ -2170,7 +2182,10 @@ mod tests { let op = binary_op(col("a", schema)?, op, col("b", schema)?, schema)?; let data: Vec = vec![left.clone(), right.clone()]; let batch = RecordBatch::try_new(schema.clone(), data)?; - let result = op.evaluate(&batch)?.into_array(batch.num_rows()); + let result = op + .evaluate(&batch)? + .into_array(batch.num_rows()) + .expect("Failed to convert to array"); assert_eq!(result.as_ref(), &expected); Ok(()) @@ -2187,7 +2202,10 @@ mod tests { let scalar = lit(scalar.clone()); let op = binary_op(scalar, op, col("a", schema)?, schema)?; let batch = RecordBatch::try_new(Arc::clone(schema), vec![Arc::clone(arr)])?; - let result = op.evaluate(&batch)?.into_array(batch.num_rows()); + let result = op + .evaluate(&batch)? + .into_array(batch.num_rows()) + .expect("Failed to convert to array"); assert_eq!(result.as_ref(), expected); Ok(()) @@ -2204,7 +2222,10 @@ mod tests { let scalar = lit(scalar.clone()); let op = binary_op(col("a", schema)?, op, scalar, schema)?; let batch = RecordBatch::try_new(Arc::clone(schema), vec![Arc::clone(arr)])?; - let result = op.evaluate(&batch)?.into_array(batch.num_rows()); + let result = op + .evaluate(&batch)? + .into_array(batch.num_rows()) + .expect("Failed to convert to array"); assert_eq!(result.as_ref(), expected); Ok(()) @@ -2776,7 +2797,8 @@ mod tests { let result = expr .evaluate(&batch) .expect("evaluation") - .into_array(batch.num_rows()); + .into_array(batch.num_rows()) + .expect("Failed to convert to array"); let expected: Int32Array = input .into_iter() @@ -3255,7 +3277,10 @@ mod tests { let arithmetic_op = binary_op(col("a", schema)?, op, col("b", schema)?, schema)?; let data: Vec = vec![left.clone(), right.clone()]; let batch = RecordBatch::try_new(schema.clone(), data)?; - let result = arithmetic_op.evaluate(&batch)?.into_array(batch.num_rows()); + let result = arithmetic_op + .evaluate(&batch)? + .into_array(batch.num_rows()) + .expect("Failed to convert to array"); assert_eq!(result.as_ref(), expected.as_ref()); Ok(()) diff --git a/datafusion/physical-expr/src/expressions/case.rs b/datafusion/physical-expr/src/expressions/case.rs index a2395c4a0ca2..5fcfd61d90e4 100644 --- a/datafusion/physical-expr/src/expressions/case.rs +++ b/datafusion/physical-expr/src/expressions/case.rs @@ -126,7 +126,7 @@ impl CaseExpr { let return_type = self.data_type(&batch.schema())?; let expr = self.expr.as_ref().unwrap(); let base_value = expr.evaluate(batch)?; - let base_value = base_value.into_array(batch.num_rows()); + let base_value = base_value.into_array(batch.num_rows())?; let base_nulls = is_null(base_value.as_ref())?; // start with nulls as default output @@ -137,7 +137,7 @@ impl CaseExpr { let when_value = self.when_then_expr[i] .0 .evaluate_selection(batch, &remainder)?; - let when_value = when_value.into_array(batch.num_rows()); + let when_value = when_value.into_array(batch.num_rows())?; // build boolean array representing which rows match the "when" value let when_match = eq(&when_value, &base_value)?; // Treat nulls as false @@ -153,7 +153,7 @@ impl CaseExpr { ColumnarValue::Scalar(value) if value.is_null() => { new_null_array(&return_type, batch.num_rows()) } - _ => then_value.into_array(batch.num_rows()), + _ => then_value.into_array(batch.num_rows())?, }; current_value = @@ -170,7 +170,7 @@ impl CaseExpr { remainder = or(&base_nulls, &remainder)?; let else_ = expr .evaluate_selection(batch, &remainder)? - .into_array(batch.num_rows()); + .into_array(batch.num_rows())?; current_value = zip(&remainder, else_.as_ref(), current_value.as_ref())?; } @@ -194,7 +194,7 @@ impl CaseExpr { let when_value = self.when_then_expr[i] .0 .evaluate_selection(batch, &remainder)?; - let when_value = when_value.into_array(batch.num_rows()); + let when_value = when_value.into_array(batch.num_rows())?; let when_value = as_boolean_array(&when_value).map_err(|e| { DataFusionError::Context( "WHEN expression did not return a BooleanArray".to_string(), @@ -214,7 +214,7 @@ impl CaseExpr { ColumnarValue::Scalar(value) if value.is_null() => { new_null_array(&return_type, batch.num_rows()) } - _ => then_value.into_array(batch.num_rows()), + _ => then_value.into_array(batch.num_rows())?, }; current_value = @@ -231,7 +231,7 @@ impl CaseExpr { .unwrap_or_else(|_| e.clone()); let else_ = expr .evaluate_selection(batch, &remainder)? - .into_array(batch.num_rows()); + .into_array(batch.num_rows())?; current_value = zip(&remainder, else_.as_ref(), current_value.as_ref())?; } @@ -425,7 +425,10 @@ mod tests { None, schema.as_ref(), )?; - let result = expr.evaluate(&batch)?.into_array(batch.num_rows()); + let result = expr + .evaluate(&batch)? + .into_array(batch.num_rows()) + .expect("Failed to convert to array"); let result = as_int32_array(&result)?; let expected = &Int32Array::from(vec![Some(123), None, None, Some(456)]); @@ -453,7 +456,10 @@ mod tests { Some(else_value), schema.as_ref(), )?; - let result = expr.evaluate(&batch)?.into_array(batch.num_rows()); + let result = expr + .evaluate(&batch)? + .into_array(batch.num_rows()) + .expect("Failed to convert to array"); let result = as_int32_array(&result)?; let expected = @@ -485,7 +491,10 @@ mod tests { Some(else_value), schema.as_ref(), )?; - let result = expr.evaluate(&batch)?.into_array(batch.num_rows()); + let result = expr + .evaluate(&batch)? + .into_array(batch.num_rows()) + .expect("Failed to convert to array"); let result = as_float64_array(&result).expect("failed to downcast to Float64Array"); @@ -523,7 +532,10 @@ mod tests { None, schema.as_ref(), )?; - let result = expr.evaluate(&batch)?.into_array(batch.num_rows()); + let result = expr + .evaluate(&batch)? + .into_array(batch.num_rows()) + .expect("Failed to convert to array"); let result = as_int32_array(&result)?; let expected = &Int32Array::from(vec![Some(123), None, None, Some(456)]); @@ -551,7 +563,10 @@ mod tests { Some(else_value), schema.as_ref(), )?; - let result = expr.evaluate(&batch)?.into_array(batch.num_rows()); + let result = expr + .evaluate(&batch)? + .into_array(batch.num_rows()) + .expect("Failed to convert to array"); let result = as_int32_array(&result)?; let expected = @@ -583,7 +598,10 @@ mod tests { Some(x), schema.as_ref(), )?; - let result = expr.evaluate(&batch)?.into_array(batch.num_rows()); + let result = expr + .evaluate(&batch)? + .into_array(batch.num_rows()) + .expect("Failed to convert to array"); let result = as_float64_array(&result).expect("failed to downcast to Float64Array"); @@ -629,7 +647,10 @@ mod tests { Some(else_value), schema.as_ref(), )?; - let result = expr.evaluate(&batch)?.into_array(batch.num_rows()); + let result = expr + .evaluate(&batch)? + .into_array(batch.num_rows()) + .expect("Failed to convert to array"); let result = as_int32_array(&result)?; let expected = @@ -661,7 +682,10 @@ mod tests { Some(else_value), schema.as_ref(), )?; - let result = expr.evaluate(&batch)?.into_array(batch.num_rows()); + let result = expr + .evaluate(&batch)? + .into_array(batch.num_rows()) + .expect("Failed to convert to array"); let result = as_float64_array(&result).expect("failed to downcast to Float64Array"); @@ -693,7 +717,10 @@ mod tests { None, schema.as_ref(), )?; - let result = expr.evaluate(&batch)?.into_array(batch.num_rows()); + let result = expr + .evaluate(&batch)? + .into_array(batch.num_rows()) + .expect("Failed to convert to array"); let result = as_float64_array(&result).expect("failed to downcast to Float64Array"); @@ -721,7 +748,10 @@ mod tests { None, schema.as_ref(), )?; - let result = expr.evaluate(&batch)?.into_array(batch.num_rows()); + let result = expr + .evaluate(&batch)? + .into_array(batch.num_rows()) + .expect("Failed to convert to array"); let result = as_float64_array(&result).expect("failed to downcast to Float64Array"); diff --git a/datafusion/physical-expr/src/expressions/cast.rs b/datafusion/physical-expr/src/expressions/cast.rs index 5d56af364636..780e042156b8 100644 --- a/datafusion/physical-expr/src/expressions/cast.rs +++ b/datafusion/physical-expr/src/expressions/cast.rs @@ -178,7 +178,7 @@ pub fn cast_column( kernels::cast::cast_with_options(array, cast_type, &cast_options)?, )), ColumnarValue::Scalar(scalar) => { - let scalar_array = scalar.to_array(); + let scalar_array = scalar.to_array()?; let cast_array = kernels::cast::cast_with_options( &scalar_array, cast_type, @@ -263,7 +263,10 @@ mod tests { assert_eq!(expression.data_type(&schema)?, $TYPE); // compute - let result = expression.evaluate(&batch)?.into_array(batch.num_rows()); + let result = expression + .evaluate(&batch)? + .into_array(batch.num_rows()) + .expect("Failed to convert to array"); // verify that the array's data_type is correct assert_eq!(*result.data_type(), $TYPE); @@ -312,7 +315,10 @@ mod tests { assert_eq!(expression.data_type(&schema)?, $TYPE); // compute - let result = expression.evaluate(&batch)?.into_array(batch.num_rows()); + let result = expression + .evaluate(&batch)? + .into_array(batch.num_rows()) + .expect("Failed to convert to array"); // verify that the array's data_type is correct assert_eq!(*result.data_type(), $TYPE); diff --git a/datafusion/physical-expr/src/expressions/datum.rs b/datafusion/physical-expr/src/expressions/datum.rs index f57cbbd4ffa3..2bb79922cfec 100644 --- a/datafusion/physical-expr/src/expressions/datum.rs +++ b/datafusion/physical-expr/src/expressions/datum.rs @@ -34,14 +34,14 @@ pub(crate) fn apply( (ColumnarValue::Array(left), ColumnarValue::Array(right)) => { Ok(ColumnarValue::Array(f(&left.as_ref(), &right.as_ref())?)) } - (ColumnarValue::Scalar(left), ColumnarValue::Array(right)) => { - Ok(ColumnarValue::Array(f(&left.to_scalar(), &right.as_ref())?)) - } - (ColumnarValue::Array(left), ColumnarValue::Scalar(right)) => { - Ok(ColumnarValue::Array(f(&left.as_ref(), &right.to_scalar())?)) - } + (ColumnarValue::Scalar(left), ColumnarValue::Array(right)) => Ok( + ColumnarValue::Array(f(&left.to_scalar()?, &right.as_ref())?), + ), + (ColumnarValue::Array(left), ColumnarValue::Scalar(right)) => Ok( + ColumnarValue::Array(f(&left.as_ref(), &right.to_scalar()?)?), + ), (ColumnarValue::Scalar(left), ColumnarValue::Scalar(right)) => { - let array = f(&left.to_scalar(), &right.to_scalar())?; + let array = f(&left.to_scalar()?, &right.to_scalar()?)?; let scalar = ScalarValue::try_from_array(array.as_ref(), 0)?; Ok(ColumnarValue::Scalar(scalar)) } diff --git a/datafusion/physical-expr/src/expressions/get_indexed_field.rs b/datafusion/physical-expr/src/expressions/get_indexed_field.rs index df79e2835820..7d5f16c454d6 100644 --- a/datafusion/physical-expr/src/expressions/get_indexed_field.rs +++ b/datafusion/physical-expr/src/expressions/get_indexed_field.rs @@ -183,7 +183,7 @@ impl PhysicalExpr for GetIndexedFieldExpr { } fn evaluate(&self, batch: &RecordBatch) -> Result { - let array = self.arg.evaluate(batch)?.into_array(batch.num_rows()); + let array = self.arg.evaluate(batch)?.into_array(batch.num_rows())?; match &self.field { GetFieldAccessExpr::NamedStructField{name} => match (array.data_type(), name) { (DataType::Map(_, _), ScalarValue::Utf8(Some(k))) => { @@ -210,7 +210,7 @@ impl PhysicalExpr for GetIndexedFieldExpr { with utf8 indexes. Tried {dt:?} with {name:?} index"), }, GetFieldAccessExpr::ListIndex{key} => { - let key = key.evaluate(batch)?.into_array(batch.num_rows()); + let key = key.evaluate(batch)?.into_array(batch.num_rows())?; match (array.data_type(), key.data_type()) { (DataType::List(_), DataType::Int64) => Ok(ColumnarValue::Array(array_element(&[ array, key @@ -224,8 +224,8 @@ impl PhysicalExpr for GetIndexedFieldExpr { } }, GetFieldAccessExpr::ListRange{start, stop} => { - let start = start.evaluate(batch)?.into_array(batch.num_rows()); - let stop = stop.evaluate(batch)?.into_array(batch.num_rows()); + let start = start.evaluate(batch)?.into_array(batch.num_rows())?; + let stop = stop.evaluate(batch)?.into_array(batch.num_rows())?; match (array.data_type(), start.data_type(), stop.data_type()) { (DataType::List(_), DataType::Int64, DataType::Int64) => Ok(ColumnarValue::Array(array_slice(&[ array, start, stop @@ -326,7 +326,10 @@ mod tests { // only one row should be processed let batch = RecordBatch::try_new(Arc::new(schema), vec![Arc::new(struct_array)])?; let expr = Arc::new(GetIndexedFieldExpr::new_field(expr, "a")); - let result = expr.evaluate(&batch)?.into_array(1); + let result = expr + .evaluate(&batch)? + .into_array(1) + .expect("Failed to convert to array"); let result = as_boolean_array(&result).expect("failed to downcast to BooleanArray"); assert_eq!(boolean, result.clone()); @@ -383,7 +386,10 @@ mod tests { vec![Arc::new(list_col), Arc::new(key_col)], )?; let expr = Arc::new(GetIndexedFieldExpr::new_index(expr, key)); - let result = expr.evaluate(&batch)?.into_array(1); + let result = expr + .evaluate(&batch)? + .into_array(1) + .expect("Failed to convert to array"); let result = as_string_array(&result).expect("failed to downcast to ListArray"); let expected = StringArray::from(expected_list); assert_eq!(expected, result.clone()); @@ -419,7 +425,10 @@ mod tests { vec![Arc::new(list_col), Arc::new(start_col), Arc::new(stop_col)], )?; let expr = Arc::new(GetIndexedFieldExpr::new_range(expr, start, stop)); - let result = expr.evaluate(&batch)?.into_array(1); + let result = expr + .evaluate(&batch)? + .into_array(1) + .expect("Failed to convert to array"); let result = as_list_array(&result).expect("failed to downcast to ListArray"); let (expected, _, _) = build_list_arguments(expected_list, vec![None], vec![None]); @@ -440,7 +449,10 @@ mod tests { vec![Arc::new(list_builder.finish()), key_array], )?; let expr = Arc::new(GetIndexedFieldExpr::new_index(expr, key)); - let result = expr.evaluate(&batch)?.into_array(batch.num_rows()); + let result = expr + .evaluate(&batch)? + .into_array(batch.num_rows()) + .expect("Failed to convert to array"); assert!(result.is_null(0)); Ok(()) } @@ -461,7 +473,10 @@ mod tests { vec![Arc::new(list_builder.finish()), Arc::new(key_array)], )?; let expr = Arc::new(GetIndexedFieldExpr::new_index(expr, key)); - let result = expr.evaluate(&batch)?.into_array(1); + let result = expr + .evaluate(&batch)? + .into_array(1) + .expect("Failed to convert to array"); assert!(result.is_null(0)); Ok(()) } diff --git a/datafusion/physical-expr/src/expressions/in_list.rs b/datafusion/physical-expr/src/expressions/in_list.rs index 8d55fb70bd9e..625b01ec9a7e 100644 --- a/datafusion/physical-expr/src/expressions/in_list.rs +++ b/datafusion/physical-expr/src/expressions/in_list.rs @@ -351,15 +351,15 @@ impl PhysicalExpr for InListExpr { fn evaluate(&self, batch: &RecordBatch) -> Result { let value = self.expr.evaluate(batch)?; let r = match &self.static_filter { - Some(f) => f.contains(value.into_array(1).as_ref(), self.negated)?, + Some(f) => f.contains(value.into_array(1)?.as_ref(), self.negated)?, None => { - let value = value.into_array(batch.num_rows()); + let value = value.into_array(batch.num_rows())?; let found = self.list.iter().map(|expr| expr.evaluate(batch)).try_fold( BooleanArray::new(BooleanBuffer::new_unset(batch.num_rows()), None), |result, expr| -> Result { Ok(or_kleene( &result, - &eq(&value, &expr?.into_array(batch.num_rows()))?, + &eq(&value, &expr?.into_array(batch.num_rows())?)?, )?) }, )?; @@ -501,7 +501,10 @@ mod tests { ($BATCH:expr, $LIST:expr, $NEGATED:expr, $EXPECTED:expr, $COL:expr, $SCHEMA:expr) => {{ let (cast_expr, cast_list_exprs) = in_list_cast($COL, $LIST, $SCHEMA)?; let expr = in_list(cast_expr, cast_list_exprs, $NEGATED, $SCHEMA).unwrap(); - let result = expr.evaluate(&$BATCH)?.into_array($BATCH.num_rows()); + let result = expr + .evaluate(&$BATCH)? + .into_array($BATCH.num_rows()) + .expect("Failed to convert to array"); let result = as_boolean_array(&result).expect("failed to downcast to BooleanArray"); let expected = &BooleanArray::from($EXPECTED); diff --git a/datafusion/physical-expr/src/expressions/is_not_null.rs b/datafusion/physical-expr/src/expressions/is_not_null.rs index da717a517fb3..2e6a2bec9cab 100644 --- a/datafusion/physical-expr/src/expressions/is_not_null.rs +++ b/datafusion/physical-expr/src/expressions/is_not_null.rs @@ -132,7 +132,10 @@ mod tests { let batch = RecordBatch::try_new(Arc::new(schema), vec![Arc::new(a)])?; // expression: "a is not null" - let result = expr.evaluate(&batch)?.into_array(batch.num_rows()); + let result = expr + .evaluate(&batch)? + .into_array(batch.num_rows()) + .expect("Failed to convert to array"); let result = as_boolean_array(&result).expect("failed to downcast to BooleanArray"); diff --git a/datafusion/physical-expr/src/expressions/is_null.rs b/datafusion/physical-expr/src/expressions/is_null.rs index ee7897edd4de..3ad4058dd649 100644 --- a/datafusion/physical-expr/src/expressions/is_null.rs +++ b/datafusion/physical-expr/src/expressions/is_null.rs @@ -134,7 +134,10 @@ mod tests { let expr = is_null(col("a", &schema)?).unwrap(); let batch = RecordBatch::try_new(Arc::new(schema), vec![Arc::new(a)])?; - let result = expr.evaluate(&batch)?.into_array(batch.num_rows()); + let result = expr + .evaluate(&batch)? + .into_array(batch.num_rows()) + .expect("Failed to convert to array"); let result = as_boolean_array(&result).expect("failed to downcast to BooleanArray"); diff --git a/datafusion/physical-expr/src/expressions/like.rs b/datafusion/physical-expr/src/expressions/like.rs index e833eabbfff2..37452e278484 100644 --- a/datafusion/physical-expr/src/expressions/like.rs +++ b/datafusion/physical-expr/src/expressions/like.rs @@ -201,7 +201,10 @@ mod test { )?; // compute - let result = expression.evaluate(&batch)?.into_array(batch.num_rows()); + let result = expression + .evaluate(&batch)? + .into_array(batch.num_rows()) + .expect("Failed to convert to array"); let result = as_boolean_array(&result).expect("failed to downcast to BooleanArray"); let expected = &BooleanArray::from($VEC); diff --git a/datafusion/physical-expr/src/expressions/literal.rs b/datafusion/physical-expr/src/expressions/literal.rs index 91cb23d5864e..cd3b51f09105 100644 --- a/datafusion/physical-expr/src/expressions/literal.rs +++ b/datafusion/physical-expr/src/expressions/literal.rs @@ -131,7 +131,10 @@ mod tests { let literal_expr = lit(42i32); assert_eq!("42", format!("{literal_expr}")); - let literal_array = literal_expr.evaluate(&batch)?.into_array(batch.num_rows()); + let literal_array = literal_expr + .evaluate(&batch)? + .into_array(batch.num_rows()) + .expect("Failed to convert to array"); let literal_array = as_int32_array(&literal_array)?; // note that the contents of the literal array are unrelated to the batch contents except for the length of the array diff --git a/datafusion/physical-expr/src/expressions/mod.rs b/datafusion/physical-expr/src/expressions/mod.rs index c44b3cf01d36..1919cac97986 100644 --- a/datafusion/physical-expr/src/expressions/mod.rs +++ b/datafusion/physical-expr/src/expressions/mod.rs @@ -247,8 +247,10 @@ pub(crate) mod tests { let expr = agg.expressions(); let values = expr .iter() - .map(|e| e.evaluate(batch)) - .map(|r| r.map(|v| v.into_array(batch.num_rows()))) + .map(|e| { + e.evaluate(batch) + .and_then(|v| v.into_array(batch.num_rows())) + }) .collect::>>()?; accum.update_batch(&values)?; accum.evaluate() @@ -262,8 +264,10 @@ pub(crate) mod tests { let expr = agg.expressions(); let values = expr .iter() - .map(|e| e.evaluate(batch)) - .map(|r| r.map(|v| v.into_array(batch.num_rows()))) + .map(|e| { + e.evaluate(batch) + .and_then(|v| v.into_array(batch.num_rows())) + }) .collect::>>()?; let indices = vec![0; batch.num_rows()]; accum.update_batch(&values, &indices, None, 1)?; diff --git a/datafusion/physical-expr/src/expressions/negative.rs b/datafusion/physical-expr/src/expressions/negative.rs index 86b000e76a32..65b347941163 100644 --- a/datafusion/physical-expr/src/expressions/negative.rs +++ b/datafusion/physical-expr/src/expressions/negative.rs @@ -195,7 +195,7 @@ mod tests { let expected = &paste!{[<$DATA_TY Array>]::from(arr_expected)}; let batch = RecordBatch::try_new(Arc::new(schema.clone()), vec![Arc::new(input)])?; - let result = expr.evaluate(&batch)?.into_array(batch.num_rows()); + let result = expr.evaluate(&batch)?.into_array(batch.num_rows()).expect("Failed to convert to array"); let result = as_primitive_array(&result).expect(format!("failed to downcast to {:?}Array", $DATA_TY).as_str()); assert_eq!(result, expected); diff --git a/datafusion/physical-expr/src/expressions/not.rs b/datafusion/physical-expr/src/expressions/not.rs index c154fad10037..4ceccc6932fe 100644 --- a/datafusion/physical-expr/src/expressions/not.rs +++ b/datafusion/physical-expr/src/expressions/not.rs @@ -150,7 +150,10 @@ mod tests { let batch = RecordBatch::try_new(Arc::new(schema.clone()), vec![Arc::new(input)])?; - let result = expr.evaluate(&batch)?.into_array(batch.num_rows()); + let result = expr + .evaluate(&batch)? + .into_array(batch.num_rows()) + .expect("Failed to convert to array"); let result = as_boolean_array(&result).expect("failed to downcast to BooleanArray"); assert_eq!(result, expected); diff --git a/datafusion/physical-expr/src/expressions/nullif.rs b/datafusion/physical-expr/src/expressions/nullif.rs index 7bbe9d73d435..252bd10c3e73 100644 --- a/datafusion/physical-expr/src/expressions/nullif.rs +++ b/datafusion/physical-expr/src/expressions/nullif.rs @@ -37,7 +37,7 @@ pub fn nullif_func(args: &[ColumnarValue]) -> Result { match (lhs, rhs) { (ColumnarValue::Array(lhs), ColumnarValue::Scalar(rhs)) => { - let rhs = rhs.to_scalar(); + let rhs = rhs.to_scalar()?; let array = nullif(lhs, &eq(&lhs, &rhs)?)?; Ok(ColumnarValue::Array(array)) @@ -47,7 +47,7 @@ pub fn nullif_func(args: &[ColumnarValue]) -> Result { Ok(ColumnarValue::Array(array)) } (ColumnarValue::Scalar(lhs), ColumnarValue::Array(rhs)) => { - let lhs = lhs.to_array_of_size(rhs.len()); + let lhs = lhs.to_array_of_size(rhs.len())?; let array = nullif(&lhs, &eq(&lhs, &rhs)?)?; Ok(ColumnarValue::Array(array)) } @@ -89,7 +89,7 @@ mod tests { let lit_array = ColumnarValue::Scalar(ScalarValue::Int32(Some(2i32))); let result = nullif_func(&[a, lit_array])?; - let result = result.into_array(0); + let result = result.into_array(0).expect("Failed to convert to array"); let expected = Arc::new(Int32Array::from(vec![ Some(1), @@ -115,7 +115,7 @@ mod tests { let lit_array = ColumnarValue::Scalar(ScalarValue::Int32(Some(1i32))); let result = nullif_func(&[a, lit_array])?; - let result = result.into_array(0); + let result = result.into_array(0).expect("Failed to convert to array"); let expected = Arc::new(Int32Array::from(vec![ None, @@ -140,7 +140,7 @@ mod tests { let lit_array = ColumnarValue::Scalar(ScalarValue::Boolean(Some(false))); let result = nullif_func(&[a, lit_array])?; - let result = result.into_array(0); + let result = result.into_array(0).expect("Failed to convert to array"); let expected = Arc::new(BooleanArray::from(vec![Some(true), None, None])) as ArrayRef; @@ -157,7 +157,7 @@ mod tests { let lit_array = ColumnarValue::Scalar(ScalarValue::Utf8(Some("bar".to_string()))); let result = nullif_func(&[a, lit_array])?; - let result = result.into_array(0); + let result = result.into_array(0).expect("Failed to convert to array"); let expected = Arc::new(StringArray::from(vec![ Some("foo"), @@ -178,7 +178,7 @@ mod tests { let lit_array = ColumnarValue::Scalar(ScalarValue::Int32(Some(2i32))); let result = nullif_func(&[lit_array, a])?; - let result = result.into_array(0); + let result = result.into_array(0).expect("Failed to convert to array"); let expected = Arc::new(Int32Array::from(vec![ Some(2), @@ -198,7 +198,7 @@ mod tests { let b_eq = ColumnarValue::Scalar(ScalarValue::Int32(Some(2i32))); let result_eq = nullif_func(&[a_eq, b_eq])?; - let result_eq = result_eq.into_array(1); + let result_eq = result_eq.into_array(1).expect("Failed to convert to array"); let expected_eq = Arc::new(Int32Array::from(vec![None])) as ArrayRef; @@ -208,7 +208,9 @@ mod tests { let b_neq = ColumnarValue::Scalar(ScalarValue::Int32(Some(1i32))); let result_neq = nullif_func(&[a_neq, b_neq])?; - let result_neq = result_neq.into_array(1); + let result_neq = result_neq + .into_array(1) + .expect("Failed to convert to array"); let expected_neq = Arc::new(Int32Array::from(vec![Some(2i32)])) as ArrayRef; assert_eq!(expected_neq.as_ref(), result_neq.as_ref()); diff --git a/datafusion/physical-expr/src/expressions/try_cast.rs b/datafusion/physical-expr/src/expressions/try_cast.rs index cba026c56513..dea7f9f86a62 100644 --- a/datafusion/physical-expr/src/expressions/try_cast.rs +++ b/datafusion/physical-expr/src/expressions/try_cast.rs @@ -89,7 +89,7 @@ impl PhysicalExpr for TryCastExpr { Ok(ColumnarValue::Array(cast)) } ColumnarValue::Scalar(scalar) => { - let array = scalar.to_array(); + let array = scalar.to_array()?; let cast_array = cast_with_options(&array, &self.cast_type, &options)?; let cast_scalar = ScalarValue::try_from_array(&cast_array, 0)?; Ok(ColumnarValue::Scalar(cast_scalar)) @@ -187,7 +187,10 @@ mod tests { assert_eq!(expression.data_type(&schema)?, $TYPE); // compute - let result = expression.evaluate(&batch)?.into_array(batch.num_rows()); + let result = expression + .evaluate(&batch)? + .into_array(batch.num_rows()) + .expect("Failed to convert to array"); // verify that the array's data_type is correct assert_eq!(*result.data_type(), $TYPE); @@ -235,7 +238,10 @@ mod tests { assert_eq!(expression.data_type(&schema)?, $TYPE); // compute - let result = expression.evaluate(&batch)?.into_array(batch.num_rows()); + let result = expression + .evaluate(&batch)? + .into_array(batch.num_rows()) + .expect("Failed to convert to array"); // verify that the array's data_type is correct assert_eq!(*result.data_type(), $TYPE); diff --git a/datafusion/physical-expr/src/functions.rs b/datafusion/physical-expr/src/functions.rs index c973232c75a6..9185ade313eb 100644 --- a/datafusion/physical-expr/src/functions.rs +++ b/datafusion/physical-expr/src/functions.rs @@ -239,7 +239,7 @@ where }; arg.clone().into_array(expansion_len) }) - .collect::>(); + .collect::>>()?; let result = (inner)(&args); @@ -937,7 +937,7 @@ mod tests { match expected { Ok(expected) => { let result = expr.evaluate(&batch)?; - let result = result.into_array(batch.num_rows()); + let result = result.into_array(batch.num_rows()).expect("Failed to convert to array"); let result = result.as_any().downcast_ref::<$ARRAY_TYPE>().unwrap(); // value is correct @@ -2906,7 +2906,10 @@ mod tests { // evaluate works let batch = RecordBatch::try_new(Arc::new(schema.clone()), columns)?; - let result = expr.evaluate(&batch)?.into_array(batch.num_rows()); + let result = expr + .evaluate(&batch)? + .into_array(batch.num_rows()) + .expect("Failed to convert to array"); // downcast works let result = as_list_array(&result)?; @@ -2945,7 +2948,10 @@ mod tests { // evaluate works let batch = RecordBatch::try_new(Arc::new(schema.clone()), columns)?; - let result = expr.evaluate(&batch)?.into_array(batch.num_rows()); + let result = expr + .evaluate(&batch)? + .into_array(batch.num_rows()) + .expect("Failed to convert to array"); // downcast works let result = as_list_array(&result)?; @@ -3017,8 +3023,11 @@ mod tests { let adapter_func = make_scalar_function(dummy_function); let scalar_arg = ColumnarValue::Scalar(ScalarValue::Int64(Some(1))); - let array_arg = - ColumnarValue::Array(ScalarValue::Int64(Some(1)).to_array_of_size(5)); + let array_arg = ColumnarValue::Array( + ScalarValue::Int64(Some(1)) + .to_array_of_size(5) + .expect("Failed to convert to array of size"), + ); let result = unpack_uint64_array(adapter_func(&[array_arg, scalar_arg]))?; assert_eq!(result, vec![5, 5]); @@ -3030,8 +3039,11 @@ mod tests { let adapter_func = make_scalar_function_with_hints(dummy_function, vec![]); let scalar_arg = ColumnarValue::Scalar(ScalarValue::Int64(Some(1))); - let array_arg = - ColumnarValue::Array(ScalarValue::Int64(Some(1)).to_array_of_size(5)); + let array_arg = ColumnarValue::Array( + ScalarValue::Int64(Some(1)) + .to_array_of_size(5) + .expect("Failed to convert to array of size"), + ); let result = unpack_uint64_array(adapter_func(&[array_arg, scalar_arg]))?; assert_eq!(result, vec![5, 5]); @@ -3046,8 +3058,11 @@ mod tests { ); let scalar_arg = ColumnarValue::Scalar(ScalarValue::Int64(Some(1))); - let array_arg = - ColumnarValue::Array(ScalarValue::Int64(Some(1)).to_array_of_size(5)); + let array_arg = ColumnarValue::Array( + ScalarValue::Int64(Some(1)) + .to_array_of_size(5) + .expect("Failed to convert to array of size"), + ); let result = unpack_uint64_array(adapter_func(&[array_arg, scalar_arg]))?; assert_eq!(result, vec![5, 1]); @@ -3056,8 +3071,11 @@ mod tests { #[test] fn test_make_scalar_function_with_hints_on_arrays() -> Result<()> { - let array_arg = - ColumnarValue::Array(ScalarValue::Int64(Some(1)).to_array_of_size(5)); + let array_arg = ColumnarValue::Array( + ScalarValue::Int64(Some(1)) + .to_array_of_size(5) + .expect("Failed to convert to array of size"), + ); let adapter_func = make_scalar_function_with_hints( dummy_function, vec![Hint::Pad, Hint::AcceptsSingular], @@ -3077,8 +3095,11 @@ mod tests { ); let scalar_arg = ColumnarValue::Scalar(ScalarValue::Int64(Some(1))); - let array_arg = - ColumnarValue::Array(ScalarValue::Int64(Some(1)).to_array_of_size(5)); + let array_arg = ColumnarValue::Array( + ScalarValue::Int64(Some(1)) + .to_array_of_size(5) + .expect("Failed to convert to array of size"), + ); let result = unpack_uint64_array(adapter_func(&[ array_arg, scalar_arg.clone(), @@ -3097,8 +3118,11 @@ mod tests { ); let scalar_arg = ColumnarValue::Scalar(ScalarValue::Int64(Some(1))); - let array_arg = - ColumnarValue::Array(ScalarValue::Int64(Some(1)).to_array_of_size(5)); + let array_arg = ColumnarValue::Array( + ScalarValue::Int64(Some(1)) + .to_array_of_size(5) + .expect("Failed to convert to array of size"), + ); let result = unpack_uint64_array(adapter_func(&[ array_arg.clone(), scalar_arg.clone(), @@ -3125,8 +3149,11 @@ mod tests { ); let scalar_arg = ColumnarValue::Scalar(ScalarValue::Int64(Some(1))); - let array_arg = - ColumnarValue::Array(ScalarValue::Int64(Some(1)).to_array_of_size(5)); + let array_arg = ColumnarValue::Array( + ScalarValue::Int64(Some(1)) + .to_array_of_size(5) + .expect("Failed to convert to array of size"), + ); let result = unpack_uint64_array(adapter_func(&[array_arg, scalar_arg]))?; assert_eq!(result, vec![5, 1]); diff --git a/datafusion/physical-expr/src/intervals/interval_aritmetic.rs b/datafusion/physical-expr/src/intervals/interval_aritmetic.rs index 1ea9b2d9aee6..4b81adfbb1f8 100644 --- a/datafusion/physical-expr/src/intervals/interval_aritmetic.rs +++ b/datafusion/physical-expr/src/intervals/interval_aritmetic.rs @@ -750,7 +750,7 @@ fn cast_scalar_value( data_type: &DataType, cast_options: &CastOptions, ) -> Result { - let cast_array = cast_with_options(&value.to_array(), data_type, cast_options)?; + let cast_array = cast_with_options(&value.to_array()?, data_type, cast_options)?; ScalarValue::try_from_array(&cast_array, 0) } diff --git a/datafusion/physical-expr/src/math_expressions.rs b/datafusion/physical-expr/src/math_expressions.rs index 0b7bc34014f9..af66862aecc5 100644 --- a/datafusion/physical-expr/src/math_expressions.rs +++ b/datafusion/physical-expr/src/math_expressions.rs @@ -769,7 +769,8 @@ mod tests { let args = vec![ColumnarValue::Array(Arc::new(NullArray::new(1)))]; let array = random(&args) .expect("failed to initialize function random") - .into_array(1); + .into_array(1) + .expect("Failed to convert to array"); let floats = as_float64_array(&array).expect("failed to initialize function random"); diff --git a/datafusion/physical-expr/src/planner.rs b/datafusion/physical-expr/src/planner.rs index 64c1d0be0455..f318cd3b0f4d 100644 --- a/datafusion/physical-expr/src/planner.rs +++ b/datafusion/physical-expr/src/planner.rs @@ -472,7 +472,7 @@ mod tests { ]))], )?; let result = p.evaluate(&batch)?; - let result = result.into_array(4); + let result = result.into_array(4).expect("Failed to convert to array"); assert_eq!( &result, diff --git a/datafusion/physical-expr/src/struct_expressions.rs b/datafusion/physical-expr/src/struct_expressions.rs index baa29d668e90..0eed1d16fba8 100644 --- a/datafusion/physical-expr/src/struct_expressions.rs +++ b/datafusion/physical-expr/src/struct_expressions.rs @@ -67,13 +67,15 @@ fn array_struct(args: &[ArrayRef]) -> Result { /// put values in a struct array. pub fn struct_expr(values: &[ColumnarValue]) -> Result { - let arrays: Vec = values + let arrays = values .iter() - .map(|x| match x { - ColumnarValue::Array(array) => array.clone(), - ColumnarValue::Scalar(scalar) => scalar.to_array().clone(), + .map(|x| { + Ok(match x { + ColumnarValue::Array(array) => array.clone(), + ColumnarValue::Scalar(scalar) => scalar.to_array()?.clone(), + }) }) - .collect(); + .collect::>>()?; Ok(ColumnarValue::Array(array_struct(arrays.as_slice())?)) } @@ -93,7 +95,8 @@ mod tests { ]; let struc = struct_expr(&args) .expect("failed to initialize function struct") - .into_array(1); + .into_array(1) + .expect("Failed to convert to array"); let result = as_struct_array(&struc).expect("failed to initialize function struct"); assert_eq!( diff --git a/datafusion/physical-expr/src/window/built_in_window_function_expr.rs b/datafusion/physical-expr/src/window/built_in_window_function_expr.rs index 66ffa990b78b..7aa4f6536a6e 100644 --- a/datafusion/physical-expr/src/window/built_in_window_function_expr.rs +++ b/datafusion/physical-expr/src/window/built_in_window_function_expr.rs @@ -60,8 +60,10 @@ pub trait BuiltInWindowFunctionExpr: Send + Sync + std::fmt::Debug { fn evaluate_args(&self, batch: &RecordBatch) -> Result> { self.expressions() .iter() - .map(|e| e.evaluate(batch)) - .map(|r| r.map(|v| v.into_array(batch.num_rows()))) + .map(|e| { + e.evaluate(batch) + .and_then(|v| v.into_array(batch.num_rows())) + }) .collect() } diff --git a/datafusion/physical-expr/src/window/lead_lag.rs b/datafusion/physical-expr/src/window/lead_lag.rs index f55f1600b9ca..d22660d41ebd 100644 --- a/datafusion/physical-expr/src/window/lead_lag.rs +++ b/datafusion/physical-expr/src/window/lead_lag.rs @@ -139,6 +139,7 @@ fn create_empty_array( let array = value .as_ref() .map(|scalar| scalar.to_array_of_size(size)) + .transpose()? .unwrap_or_else(|| new_null_array(data_type, size)); if array.data_type() != data_type { cast(&array, data_type).map_err(DataFusionError::ArrowError) diff --git a/datafusion/physical-expr/src/window/window_expr.rs b/datafusion/physical-expr/src/window/window_expr.rs index 9b0a02d329c4..b282e3579754 100644 --- a/datafusion/physical-expr/src/window/window_expr.rs +++ b/datafusion/physical-expr/src/window/window_expr.rs @@ -82,8 +82,10 @@ pub trait WindowExpr: Send + Sync + Debug { fn evaluate_args(&self, batch: &RecordBatch) -> Result> { self.expressions() .iter() - .map(|e| e.evaluate(batch)) - .map(|r| r.map(|v| v.into_array(batch.num_rows()))) + .map(|e| { + e.evaluate(batch) + .and_then(|v| v.into_array(batch.num_rows())) + }) .collect() } diff --git a/datafusion/physical-plan/src/aggregates/mod.rs b/datafusion/physical-plan/src/aggregates/mod.rs index 4052d6aef0ae..3ac812929772 100644 --- a/datafusion/physical-plan/src/aggregates/mod.rs +++ b/datafusion/physical-plan/src/aggregates/mod.rs @@ -1064,10 +1064,11 @@ fn finalize_aggregation( // build the vector of states let a = accumulators .iter() - .map(|accumulator| accumulator.state()) - .map(|value| { - value.map(|e| { - e.iter().map(|v| v.to_array()).collect::>() + .map(|accumulator| { + accumulator.state().and_then(|e| { + e.iter() + .map(|v| v.to_array()) + .collect::>>() }) }) .collect::>>()?; @@ -1080,7 +1081,7 @@ fn finalize_aggregation( // merge the state to the final value accumulators .iter() - .map(|accumulator| accumulator.evaluate().map(|v| v.to_array())) + .map(|accumulator| accumulator.evaluate().and_then(|v| v.to_array())) .collect::>>() } } @@ -1092,9 +1093,11 @@ fn evaluate( batch: &RecordBatch, ) -> Result> { expr.iter() - .map(|expr| expr.evaluate(batch)) - .map(|r| r.map(|v| v.into_array(batch.num_rows()))) - .collect::>>() + .map(|expr| { + expr.evaluate(batch) + .and_then(|v| v.into_array(batch.num_rows())) + }) + .collect() } /// Evaluates expressions against a record batch. @@ -1114,9 +1117,11 @@ fn evaluate_optional( expr.iter() .map(|expr| { expr.as_ref() - .map(|expr| expr.evaluate(batch)) + .map(|expr| { + expr.evaluate(batch) + .and_then(|v| v.into_array(batch.num_rows())) + }) .transpose() - .map(|r| r.map(|v| v.into_array(batch.num_rows()))) }) .collect::>>() } @@ -1140,7 +1145,7 @@ pub(crate) fn evaluate_group_by( .iter() .map(|(expr, _)| { let value = expr.evaluate(batch)?; - Ok(value.into_array(batch.num_rows())) + value.into_array(batch.num_rows()) }) .collect::>>()?; @@ -1149,7 +1154,7 @@ pub(crate) fn evaluate_group_by( .iter() .map(|(expr, _)| { let value = expr.evaluate(batch)?; - Ok(value.into_array(batch.num_rows())) + value.into_array(batch.num_rows()) }) .collect::>>()?; diff --git a/datafusion/physical-plan/src/aggregates/no_grouping.rs b/datafusion/physical-plan/src/aggregates/no_grouping.rs index 32c0bbc78a5d..90eb488a2ead 100644 --- a/datafusion/physical-plan/src/aggregates/no_grouping.rs +++ b/datafusion/physical-plan/src/aggregates/no_grouping.rs @@ -217,8 +217,10 @@ fn aggregate_batch( // 1.3 let values = &expr .iter() - .map(|e| e.evaluate(&batch)) - .map(|r| r.map(|v| v.into_array(batch.num_rows()))) + .map(|e| { + e.evaluate(&batch) + .and_then(|v| v.into_array(batch.num_rows())) + }) .collect::>>()?; // 1.4 diff --git a/datafusion/physical-plan/src/filter.rs b/datafusion/physical-plan/src/filter.rs index 0c44b367e514..d560a219f230 100644 --- a/datafusion/physical-plan/src/filter.rs +++ b/datafusion/physical-plan/src/filter.rs @@ -300,7 +300,7 @@ pub(crate) fn batch_filter( ) -> Result { predicate .evaluate(batch) - .map(|v| v.into_array(batch.num_rows())) + .and_then(|v| v.into_array(batch.num_rows())) .and_then(|array| { Ok(as_boolean_array(&array)?) // apply filter array to record batch diff --git a/datafusion/physical-plan/src/joins/cross_join.rs b/datafusion/physical-plan/src/joins/cross_join.rs index 102f0c42e90c..4c928d44caf4 100644 --- a/datafusion/physical-plan/src/joins/cross_join.rs +++ b/datafusion/physical-plan/src/joins/cross_join.rs @@ -344,7 +344,7 @@ fn build_batch( .iter() .map(|arr| { let scalar = ScalarValue::try_from_array(arr, left_index)?; - Ok(scalar.to_array_of_size(batch.num_rows())) + scalar.to_array_of_size(batch.num_rows()) }) .collect::>>()?; diff --git a/datafusion/physical-plan/src/joins/hash_join.rs b/datafusion/physical-plan/src/joins/hash_join.rs index 1a2db87d98a2..546a929bf939 100644 --- a/datafusion/physical-plan/src/joins/hash_join.rs +++ b/datafusion/physical-plan/src/joins/hash_join.rs @@ -713,7 +713,7 @@ where // evaluate the keys let keys_values = on .iter() - .map(|c| Ok(c.evaluate(batch)?.into_array(batch.num_rows()))) + .map(|c| c.evaluate(batch)?.into_array(batch.num_rows())) .collect::>>()?; // calculate the hash values @@ -857,13 +857,13 @@ pub fn build_equal_condition_join_indices( ) -> Result<(UInt64Array, UInt32Array)> { let keys_values = probe_on .iter() - .map(|c| Ok(c.evaluate(probe_batch)?.into_array(probe_batch.num_rows()))) + .map(|c| c.evaluate(probe_batch)?.into_array(probe_batch.num_rows())) .collect::>>()?; let build_join_values = build_on .iter() .map(|c| { - Ok(c.evaluate(build_input_buffer)? - .into_array(build_input_buffer.num_rows())) + c.evaluate(build_input_buffer)? + .into_array(build_input_buffer.num_rows()) }) .collect::>>()?; hashes_buffer.clear(); diff --git a/datafusion/physical-plan/src/joins/hash_join_utils.rs b/datafusion/physical-plan/src/joins/hash_join_utils.rs index c134b23d78cf..5ebf370b6d71 100644 --- a/datafusion/physical-plan/src/joins/hash_join_utils.rs +++ b/datafusion/physical-plan/src/joins/hash_join_utils.rs @@ -607,7 +607,7 @@ pub fn update_filter_expr_interval( .origin_sorted_expr() .expr .evaluate(batch)? - .into_array(1); + .into_array(1)?; // Convert the array to a ScalarValue: let value = ScalarValue::try_from_array(&array, 0)?; // Create a ScalarValue representing positive or negative infinity for the same data type: diff --git a/datafusion/physical-plan/src/joins/symmetric_hash_join.rs b/datafusion/physical-plan/src/joins/symmetric_hash_join.rs index 1306a4874436..51561f5dab24 100644 --- a/datafusion/physical-plan/src/joins/symmetric_hash_join.rs +++ b/datafusion/physical-plan/src/joins/symmetric_hash_join.rs @@ -626,7 +626,9 @@ impl Stream for SymmetricHashJoinStream { /// # Returns /// /// A [Result] object that contains the pruning length. The function will return -/// an error if there is an issue evaluating the build side filter expression. +/// an error if +/// - there is an issue evaluating the build side filter expression; +/// - there is an issue converting the build side filter expression into an array fn determine_prune_length( buffer: &RecordBatch, build_side_filter_expr: &SortedFilterExpr, @@ -637,7 +639,7 @@ fn determine_prune_length( let batch_arr = origin_sorted_expr .expr .evaluate(buffer)? - .into_array(buffer.num_rows()); + .into_array(buffer.num_rows())?; // Get the lower or upper interval based on the sort direction let target = if origin_sorted_expr.options.descending { diff --git a/datafusion/physical-plan/src/joins/utils.rs b/datafusion/physical-plan/src/joins/utils.rs index 5efeedfe6536..f93f08255e0c 100644 --- a/datafusion/physical-plan/src/joins/utils.rs +++ b/datafusion/physical-plan/src/joins/utils.rs @@ -778,7 +778,7 @@ pub(crate) fn apply_join_filter_to_indices( let filter_result = filter .expression() .evaluate(&intermediate_batch)? - .into_array(intermediate_batch.num_rows()); + .into_array(intermediate_batch.num_rows())?; let mask = as_boolean_array(&filter_result)?; let left_filtered = compute::filter(&build_indices, mask)?; diff --git a/datafusion/physical-plan/src/projection.rs b/datafusion/physical-plan/src/projection.rs index bbf0d6d4b31c..b8e2d0e425d4 100644 --- a/datafusion/physical-plan/src/projection.rs +++ b/datafusion/physical-plan/src/projection.rs @@ -310,8 +310,10 @@ impl ProjectionStream { let arrays = self .expr .iter() - .map(|expr| expr.evaluate(batch)) - .map(|r| r.map(|v| v.into_array(batch.num_rows()))) + .map(|expr| { + expr.evaluate(batch) + .and_then(|v| v.into_array(batch.num_rows())) + }) .collect::>>()?; if arrays.is_empty() { diff --git a/datafusion/physical-plan/src/repartition/mod.rs b/datafusion/physical-plan/src/repartition/mod.rs index 66f7037e5c2d..9836e057ff87 100644 --- a/datafusion/physical-plan/src/repartition/mod.rs +++ b/datafusion/physical-plan/src/repartition/mod.rs @@ -169,9 +169,7 @@ impl BatchPartitioner { let arrays = exprs .iter() - .map(|expr| { - Ok(expr.evaluate(&batch)?.into_array(batch.num_rows())) - }) + .map(|expr| expr.evaluate(&batch)?.into_array(batch.num_rows())) .collect::>>()?; hash_buffer.clear(); diff --git a/datafusion/physical-plan/src/sorts/stream.rs b/datafusion/physical-plan/src/sorts/stream.rs index 4cabdc6e178c..135b4fbdece4 100644 --- a/datafusion/physical-plan/src/sorts/stream.rs +++ b/datafusion/physical-plan/src/sorts/stream.rs @@ -118,7 +118,7 @@ impl RowCursorStream { let cols = self .column_expressions .iter() - .map(|expr| Ok(expr.evaluate(batch)?.into_array(batch.num_rows()))) + .map(|expr| expr.evaluate(batch)?.into_array(batch.num_rows())) .collect::>>()?; let rows = self.converter.convert_columns(&cols)?; @@ -181,7 +181,7 @@ impl FieldCursorStream { fn convert_batch(&mut self, batch: &RecordBatch) -> Result> { let value = self.sort.expr.evaluate(batch)?; - let array = value.into_array(batch.num_rows()); + let array = value.into_array(batch.num_rows())?; let array = array.as_any().downcast_ref::().expect("field values"); Ok(ArrayValues::new(self.sort.options, array)) } diff --git a/datafusion/physical-plan/src/topk/mod.rs b/datafusion/physical-plan/src/topk/mod.rs index 4638c0dcf264..9120566273d3 100644 --- a/datafusion/physical-plan/src/topk/mod.rs +++ b/datafusion/physical-plan/src/topk/mod.rs @@ -153,7 +153,7 @@ impl TopK { .iter() .map(|expr| { let value = expr.expr.evaluate(&batch)?; - Ok(value.into_array(batch.num_rows())) + value.into_array(batch.num_rows()) }) .collect::>>()?; diff --git a/datafusion/physical-plan/src/unnest.rs b/datafusion/physical-plan/src/unnest.rs index c9f3fb76c2e5..af4a81626cd7 100644 --- a/datafusion/physical-plan/src/unnest.rs +++ b/datafusion/physical-plan/src/unnest.rs @@ -242,7 +242,7 @@ fn build_batch( column: &Column, options: &UnnestOptions, ) -> Result { - let list_array = column.evaluate(batch)?.into_array(batch.num_rows()); + let list_array = column.evaluate(batch)?.into_array(batch.num_rows())?; match list_array.data_type() { DataType::List(_) => { let list_array = list_array.as_any().downcast_ref::().unwrap(); From 6fe00ce2e30d2af2b64d3a97b877a96109c215f9 Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Sun, 12 Nov 2023 01:41:15 -0700 Subject: [PATCH 231/572] Fix join order for TPCH Q17 & Q18 by improving FilterExec statistics (#8126) MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit * Assume filters are highly selective if we cannot truly estimate cardinality * fix regression * cargo fmt * simplify code * Update datafusion/physical-plan/src/filter.rs Co-authored-by: Daniël Heres * add comment with link to follow on issue * Use default of 20% selectivity * trigger CI * remove files * trigger CI * address feedback --------- Co-authored-by: Daniël Heres --- datafusion/physical-plan/src/filter.rs | 16 ++++++++++++++-- 1 file changed, 14 insertions(+), 2 deletions(-) diff --git a/datafusion/physical-plan/src/filter.rs b/datafusion/physical-plan/src/filter.rs index d560a219f230..822ddfdf3eb0 100644 --- a/datafusion/physical-plan/src/filter.rs +++ b/datafusion/physical-plan/src/filter.rs @@ -194,11 +194,23 @@ impl ExecutionPlan for FilterExec { fn statistics(&self) -> Result { let predicate = self.predicate(); + let input_stats = self.input.statistics()?; let schema = self.schema(); if !check_support(predicate, &schema) { - return Ok(Statistics::new_unknown(&schema)); + // assume filter selects 20% of rows if we cannot do anything smarter + // tracking issue for making this configurable: + // https://github.com/apache/arrow-datafusion/issues/8133 + let selectivity = 0.2_f32; + let mut stats = input_stats.clone().into_inexact(); + if let Precision::Inexact(n) = stats.num_rows { + stats.num_rows = Precision::Inexact((selectivity * n as f32) as usize); + } + if let Precision::Inexact(n) = stats.total_byte_size { + stats.total_byte_size = + Precision::Inexact((selectivity * n as f32) as usize); + } + return Ok(stats); } - let input_stats = self.input.statistics()?; let num_rows = input_stats.num_rows; let total_byte_size = input_stats.total_byte_size; From 5a2e0ba646152bb690275f9583c76d227a6c6fb6 Mon Sep 17 00:00:00 2001 From: Andrew Lamb Date: Sun, 12 Nov 2023 06:19:16 -0500 Subject: [PATCH 232/572] Fix: Do not try and preserve order when there is no order to preserve in RepartitionExec (#8127) * Do not try and maintain input order when there is none to maintain * Improve documentation * Test to ensure sort preserving repartition is not used incorrectly * fix test * Undo import reorg * Improve documentation * Update datafusion/physical-plan/src/repartition/mod.rs Co-authored-by: Mehmet Ozan Kabak * Move tests to RepartitonExec module * Rework with_preserve_order usage --------- Co-authored-by: Mehmet Ozan Kabak --- .../enforce_distribution.rs | 20 +- .../src/physical_optimizer/enforce_sorting.rs | 4 +- .../replace_with_order_preserving_variants.rs | 5 +- .../core/src/physical_optimizer/test_utils.rs | 2 +- .../sort_preserving_repartition_fuzz.rs | 4 +- .../physical-plan/src/repartition/mod.rs | 232 ++++++++++++++---- 6 files changed, 203 insertions(+), 64 deletions(-) diff --git a/datafusion/core/src/physical_optimizer/enforce_distribution.rs b/datafusion/core/src/physical_optimizer/enforce_distribution.rs index c562d7853f1c..12f27ab18fbd 100644 --- a/datafusion/core/src/physical_optimizer/enforce_distribution.rs +++ b/datafusion/core/src/physical_optimizer/enforce_distribution.rs @@ -929,14 +929,12 @@ fn add_roundrobin_on_top( // - Preserving ordering is not helpful in terms of satisfying ordering requirements // - Usage of order preserving variants is not desirable // (determined by flag `config.optimizer.bounded_order_preserving_variants`) - let should_preserve_ordering = input.output_ordering().is_some(); - let partitioning = Partitioning::RoundRobinBatch(n_target); - let repartition = RepartitionExec::try_new(input, partitioning)?; - let new_plan = Arc::new(repartition.with_preserve_order(should_preserve_ordering)) - as Arc; + let repartition = + RepartitionExec::try_new(input, partitioning)?.with_preserve_order(); // update distribution onward with new operator + let new_plan = Arc::new(repartition) as Arc; update_distribution_onward(new_plan.clone(), dist_onward, input_idx); Ok(new_plan) } else { @@ -999,7 +997,6 @@ fn add_hash_on_top( // requirements. // - Usage of order preserving variants is not desirable (per the flag // `config.optimizer.bounded_order_preserving_variants`). - let should_preserve_ordering = input.output_ordering().is_some(); let mut new_plan = if repartition_beneficial_stats { // Since hashing benefits from partitioning, add a round-robin repartition // before it: @@ -1008,9 +1005,10 @@ fn add_hash_on_top( input }; let partitioning = Partitioning::Hash(hash_exprs, n_target); - let repartition = RepartitionExec::try_new(new_plan, partitioning)?; - new_plan = - Arc::new(repartition.with_preserve_order(should_preserve_ordering)) as _; + let repartition = RepartitionExec::try_new(new_plan, partitioning)? + // preserve any ordering if possible + .with_preserve_order(); + new_plan = Arc::new(repartition) as _; // update distribution onward with new operator update_distribution_onward(new_plan.clone(), dist_onward, input_idx); @@ -1159,11 +1157,11 @@ fn replace_order_preserving_variants_helper( if let Some(repartition) = exec_tree.plan.as_any().downcast_ref::() { if repartition.preserve_order() { return Ok(Arc::new( + // new RepartitionExec don't preserve order RepartitionExec::try_new( updated_children.swap_remove(0), repartition.partitioning().clone(), - )? - .with_preserve_order(false), + )?, )); } } diff --git a/datafusion/core/src/physical_optimizer/enforce_sorting.rs b/datafusion/core/src/physical_optimizer/enforce_sorting.rs index 4779ced44f1a..2590948d3b3e 100644 --- a/datafusion/core/src/physical_optimizer/enforce_sorting.rs +++ b/datafusion/core/src/physical_optimizer/enforce_sorting.rs @@ -703,11 +703,11 @@ fn remove_corresponding_sort_from_sub_plan( } else if let Some(repartition) = plan.as_any().downcast_ref::() { Arc::new( + // By default, RepartitionExec does not preserve order RepartitionExec::try_new( children.swap_remove(0), repartition.partitioning().clone(), - )? - .with_preserve_order(false), + )?, ) } else { plan.clone().with_new_children(children)? diff --git a/datafusion/core/src/physical_optimizer/replace_with_order_preserving_variants.rs b/datafusion/core/src/physical_optimizer/replace_with_order_preserving_variants.rs index 0c2f21d11acd..58806be6d411 100644 --- a/datafusion/core/src/physical_optimizer/replace_with_order_preserving_variants.rs +++ b/datafusion/core/src/physical_optimizer/replace_with_order_preserving_variants.rs @@ -176,8 +176,9 @@ fn get_updated_plan( // a `SortPreservingRepartitionExec` if appropriate: if is_repartition(&plan) && !plan.maintains_input_order()[0] && is_spr_better { let child = plan.children().swap_remove(0); - let repartition = RepartitionExec::try_new(child, plan.output_partitioning())?; - plan = Arc::new(repartition.with_preserve_order(true)) as _ + let repartition = RepartitionExec::try_new(child, plan.output_partitioning())? + .with_preserve_order(); + plan = Arc::new(repartition) as _ } // When the input of a `CoalescePartitionsExec` has an ordering, replace it // with a `SortPreservingMergeExec` if appropriate: diff --git a/datafusion/core/src/physical_optimizer/test_utils.rs b/datafusion/core/src/physical_optimizer/test_utils.rs index 159ee5089075..cc62cda41266 100644 --- a/datafusion/core/src/physical_optimizer/test_utils.rs +++ b/datafusion/core/src/physical_optimizer/test_utils.rs @@ -328,7 +328,7 @@ pub fn spr_repartition_exec(input: Arc) -> Arc, mut children: Vec>, ) -> Result> { - let repartition = - RepartitionExec::try_new(children.swap_remove(0), self.partitioning.clone()); - repartition.map(|r| Arc::new(r.with_preserve_order(self.preserve_order)) as _) + let mut repartition = + RepartitionExec::try_new(children.swap_remove(0), self.partitioning.clone())?; + if self.preserve_order { + repartition = repartition.with_preserve_order(); + } + Ok(Arc::new(repartition)) } /// Specifies whether this plan generates an infinite stream of records. @@ -625,7 +629,9 @@ impl ExecutionPlan for RepartitionExec { } impl RepartitionExec { - /// Create a new RepartitionExec + /// Create a new RepartitionExec, that produces output `partitioning`, and + /// does not preserve the order of the input (see [`Self::with_preserve_order`] + /// for more details) pub fn try_new( input: Arc, partitioning: Partitioning, @@ -642,16 +648,20 @@ impl RepartitionExec { }) } - /// Set Order preserving flag - pub fn with_preserve_order(mut self, preserve_order: bool) -> Self { - // Set "preserve order" mode only if the input partition count is larger than 1 - // Because in these cases naive `RepartitionExec` cannot maintain ordering. Using - // `SortPreservingRepartitionExec` is necessity. However, when input partition number - // is 1, `RepartitionExec` can maintain ordering. In this case, we don't need to use - // `SortPreservingRepartitionExec` variant to maintain ordering. - if self.input.output_partitioning().partition_count() > 1 { - self.preserve_order = preserve_order - } + /// Specify if this reparititoning operation should preserve the order of + /// rows from its input when producing output. Preserving order is more + /// expensive at runtime, so should only be set if the output of this + /// operator can take advantage of it. + /// + /// If the input is not ordered, or has only one partition, this is a no op, + /// and the node remains a `RepartitionExec`. + pub fn with_preserve_order(mut self) -> Self { + self.preserve_order = + // If the input isn't ordered, there is no ordering to preserve + self.input.output_ordering().is_some() && + // if there is only one input partition, merging is not required + // to maintain order + self.input.output_partitioning().partition_count() > 1; self } @@ -911,7 +921,19 @@ impl RecordBatchStream for PerPartitionStream { #[cfg(test)] mod tests { - use super::*; + use std::collections::HashSet; + + use arrow::array::{ArrayRef, StringArray}; + use arrow::datatypes::{DataType, Field, Schema}; + use arrow::record_batch::RecordBatch; + use arrow_array::UInt32Array; + use futures::FutureExt; + use tokio::task::JoinHandle; + + use datafusion_common::cast::as_string_array; + use datafusion_common::{assert_batches_sorted_eq, exec_err}; + use datafusion_execution::runtime_env::{RuntimeConfig, RuntimeEnv}; + use crate::{ test::{ assert_is_pending, @@ -922,16 +944,8 @@ mod tests { }, {collect, expressions::col, memory::MemoryExec}, }; - use arrow::array::{ArrayRef, StringArray}; - use arrow::datatypes::{DataType, Field, Schema}; - use arrow::record_batch::RecordBatch; - use arrow_array::UInt32Array; - use datafusion_common::cast::as_string_array; - use datafusion_common::{assert_batches_sorted_eq, exec_err}; - use datafusion_execution::runtime_env::{RuntimeConfig, RuntimeEnv}; - use futures::FutureExt; - use std::collections::HashSet; - use tokio::task::JoinHandle; + + use super::*; #[tokio::test] async fn one_to_many_round_robin() -> Result<()> { @@ -1432,3 +1446,129 @@ mod tests { .unwrap() } } + +#[cfg(test)] +mod test { + use arrow_schema::{DataType, Field, Schema, SortOptions}; + + use datafusion_physical_expr::expressions::col; + + use crate::memory::MemoryExec; + use crate::union::UnionExec; + + use super::*; + + /// Asserts that the plan is as expected + /// + /// `$EXPECTED_PLAN_LINES`: input plan + /// `$PLAN`: the plan to optimized + /// + macro_rules! assert_plan { + ($EXPECTED_PLAN_LINES: expr, $PLAN: expr) => { + let physical_plan = $PLAN; + let formatted = crate::displayable(&physical_plan).indent(true).to_string(); + let actual: Vec<&str> = formatted.trim().lines().collect(); + + let expected_plan_lines: Vec<&str> = $EXPECTED_PLAN_LINES + .iter().map(|s| *s).collect(); + + assert_eq!( + expected_plan_lines, actual, + "\n**Original Plan Mismatch\n\nexpected:\n\n{expected_plan_lines:#?}\nactual:\n\n{actual:#?}\n\n" + ); + }; + } + + #[tokio::test] + async fn test_preserve_order() -> Result<()> { + let schema = test_schema(); + let sort_exprs = sort_exprs(&schema); + let source1 = sorted_memory_exec(&schema, sort_exprs.clone()); + let source2 = sorted_memory_exec(&schema, sort_exprs); + // output has multiple partitions, and is sorted + let union = UnionExec::new(vec![source1, source2]); + let exec = + RepartitionExec::try_new(Arc::new(union), Partitioning::RoundRobinBatch(10)) + .unwrap() + .with_preserve_order(); + + // Repartition should preserve order + let expected_plan = [ + "SortPreservingRepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=2, sort_exprs=c0@0 ASC", + " UnionExec", + " MemoryExec: partitions=1, partition_sizes=[0], output_ordering=c0@0 ASC", + " MemoryExec: partitions=1, partition_sizes=[0], output_ordering=c0@0 ASC", + ]; + assert_plan!(expected_plan, exec); + Ok(()) + } + + #[tokio::test] + async fn test_preserve_order_one_partition() -> Result<()> { + let schema = test_schema(); + let sort_exprs = sort_exprs(&schema); + let source = sorted_memory_exec(&schema, sort_exprs); + // output is sorted, but has only a single partition, so no need to sort + let exec = RepartitionExec::try_new(source, Partitioning::RoundRobinBatch(10)) + .unwrap() + .with_preserve_order(); + + // Repartition should not preserve order + let expected_plan = [ + "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", + " MemoryExec: partitions=1, partition_sizes=[0], output_ordering=c0@0 ASC", + ]; + assert_plan!(expected_plan, exec); + Ok(()) + } + + #[tokio::test] + async fn test_preserve_order_input_not_sorted() -> Result<()> { + let schema = test_schema(); + let source1 = memory_exec(&schema); + let source2 = memory_exec(&schema); + // output has multiple partitions, but is not sorted + let union = UnionExec::new(vec![source1, source2]); + let exec = + RepartitionExec::try_new(Arc::new(union), Partitioning::RoundRobinBatch(10)) + .unwrap() + .with_preserve_order(); + + // Repartition should not preserve order, as there is no order to preserve + let expected_plan = [ + "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=2", + " UnionExec", + " MemoryExec: partitions=1, partition_sizes=[0]", + " MemoryExec: partitions=1, partition_sizes=[0]", + ]; + assert_plan!(expected_plan, exec); + Ok(()) + } + + fn test_schema() -> Arc { + Arc::new(Schema::new(vec![Field::new("c0", DataType::UInt32, false)])) + } + + fn sort_exprs(schema: &Schema) -> Vec { + let options = SortOptions::default(); + vec![PhysicalSortExpr { + expr: col("c0", schema).unwrap(), + options, + }] + } + + fn memory_exec(schema: &SchemaRef) -> Arc { + Arc::new(MemoryExec::try_new(&[vec![]], schema.clone(), None).unwrap()) + } + + fn sorted_memory_exec( + schema: &SchemaRef, + sort_exprs: Vec, + ) -> Arc { + Arc::new( + MemoryExec::try_new(&[vec![]], schema.clone(), None) + .unwrap() + .with_sort_information(vec![sort_exprs]), + ) + } +} From 9e012a6c1c495ebfba8c863755cdcb069e31d410 Mon Sep 17 00:00:00 2001 From: Nga Tran Date: Sun, 12 Nov 2023 06:20:09 -0500 Subject: [PATCH 233/572] feat: add column statistics into explain (#8112) * feat: add column statistics into explain * feat: only show non-absent statistics * fix: update test output --- datafusion/common/src/stats.rs | 39 ++++++++++++++++++- datafusion/core/tests/sql/explain_analyze.rs | 5 ++- .../sqllogictest/test_files/explain.slt | 8 ++-- 3 files changed, 46 insertions(+), 6 deletions(-) diff --git a/datafusion/common/src/stats.rs b/datafusion/common/src/stats.rs index 2e799c92bea7..1c7a4fd4d553 100644 --- a/datafusion/common/src/stats.rs +++ b/datafusion/common/src/stats.rs @@ -257,7 +257,44 @@ impl Statistics { impl Display for Statistics { fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { - write!(f, "Rows={}, Bytes={}", self.num_rows, self.total_byte_size)?; + // string of column statistics + let column_stats = self + .column_statistics + .iter() + .enumerate() + .map(|(i, cs)| { + let s = format!("(Col[{}]:", i); + let s = if cs.min_value != Precision::Absent { + format!("{} Min={}", s, cs.min_value) + } else { + s + }; + let s = if cs.max_value != Precision::Absent { + format!("{} Max={}", s, cs.max_value) + } else { + s + }; + let s = if cs.null_count != Precision::Absent { + format!("{} Null={}", s, cs.null_count) + } else { + s + }; + let s = if cs.distinct_count != Precision::Absent { + format!("{} Distinct={}", s, cs.distinct_count) + } else { + s + }; + + s + ")" + }) + .collect::>() + .join(","); + + write!( + f, + "Rows={}, Bytes={}, [{}]", + self.num_rows, self.total_byte_size, column_stats + )?; Ok(()) } diff --git a/datafusion/core/tests/sql/explain_analyze.rs b/datafusion/core/tests/sql/explain_analyze.rs index 2436e82f3ce9..0ebd3a0c69d1 100644 --- a/datafusion/core/tests/sql/explain_analyze.rs +++ b/datafusion/core/tests/sql/explain_analyze.rs @@ -827,5 +827,8 @@ async fn csv_explain_analyze_with_statistics() { .to_string(); // should contain scan statistics - assert_contains!(&formatted, ", statistics=[Rows=Absent, Bytes=Absent]"); + assert_contains!( + &formatted, + ", statistics=[Rows=Absent, Bytes=Absent, [(Col[0]:)]]" + ); } diff --git a/datafusion/sqllogictest/test_files/explain.slt b/datafusion/sqllogictest/test_files/explain.slt index 911ede678bde..1db24efd9b4a 100644 --- a/datafusion/sqllogictest/test_files/explain.slt +++ b/datafusion/sqllogictest/test_files/explain.slt @@ -274,8 +274,8 @@ query TT EXPLAIN SELECT a, b, c FROM simple_explain_test limit 10; ---- physical_plan -GlobalLimitExec: skip=0, fetch=10, statistics=[Rows=Inexact(10), Bytes=Absent] ---CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/example.csv]]}, projection=[a, b, c], limit=10, has_header=true, statistics=[Rows=Absent, Bytes=Absent] +GlobalLimitExec: skip=0, fetch=10, statistics=[Rows=Inexact(10), Bytes=Absent, [(Col[0]:),(Col[1]:),(Col[2]:)]] +--CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/example.csv]]}, projection=[a, b, c], limit=10, has_header=true, statistics=[Rows=Absent, Bytes=Absent, [(Col[0]:),(Col[1]:),(Col[2]:)]] # Parquet scan with statistics collected statement ok @@ -288,8 +288,8 @@ query TT EXPLAIN SELECT * FROM alltypes_plain limit 10; ---- physical_plan -GlobalLimitExec: skip=0, fetch=10, statistics=[Rows=Exact(8), Bytes=Absent] ---ParquetExec: file_groups={1 group: [[WORKSPACE_ROOT/parquet-testing/data/alltypes_plain.parquet]]}, projection=[id, bool_col, tinyint_col, smallint_col, int_col, bigint_col, float_col, double_col, date_string_col, string_col, timestamp_col], limit=10, statistics=[Rows=Exact(8), Bytes=Absent] +GlobalLimitExec: skip=0, fetch=10, statistics=[Rows=Exact(8), Bytes=Absent, [(Col[0]:),(Col[1]:),(Col[2]:),(Col[3]:),(Col[4]:),(Col[5]:),(Col[6]:),(Col[7]:),(Col[8]:),(Col[9]:),(Col[10]:)]] +--ParquetExec: file_groups={1 group: [[WORKSPACE_ROOT/parquet-testing/data/alltypes_plain.parquet]]}, projection=[id, bool_col, tinyint_col, smallint_col, int_col, bigint_col, float_col, double_col, date_string_col, string_col, timestamp_col], limit=10, statistics=[Rows=Exact(8), Bytes=Absent, [(Col[0]:),(Col[1]:),(Col[2]:),(Col[3]:),(Col[4]:),(Col[5]:),(Col[6]:),(Col[7]:),(Col[8]:),(Col[9]:),(Col[10]:)]] statement ok set datafusion.execution.collect_statistics = false; From e18c709914e40f77f44f938bbeb449a16520f193 Mon Sep 17 00:00:00 2001 From: Tanmay Gujar Date: Sun, 12 Nov 2023 06:22:26 -0500 Subject: [PATCH 234/572] Add subtrait support for `IS NULL` and `IS NOT NULL` (#8093) * added match arms and tests for is null * fixed formatting --------- Co-authored-by: Tanmay Gujar --- .../substrait/src/logical_plan/consumer.rs | 42 ++++++++++++++++ .../substrait/src/logical_plan/producer.rs | 48 ++++++++++++++++++- .../tests/cases/roundtrip_logical_plan.rs | 10 ++++ 3 files changed, 99 insertions(+), 1 deletion(-) diff --git a/datafusion/substrait/src/logical_plan/consumer.rs b/datafusion/substrait/src/logical_plan/consumer.rs index a15121652452..c6bcbb479e80 100644 --- a/datafusion/substrait/src/logical_plan/consumer.rs +++ b/datafusion/substrait/src/logical_plan/consumer.rs @@ -78,6 +78,10 @@ enum ScalarFunctionType { Like, /// [Expr::Like] Case insensitive operator counterpart of `Like` ILike, + /// [Expr::IsNull] + IsNull, + /// [Expr::IsNotNull] + IsNotNull, } pub fn name_to_op(name: &str) -> Result { @@ -126,6 +130,8 @@ fn scalar_function_type_from_str(name: &str) -> Result { "not" => Ok(ScalarFunctionType::Not), "like" => Ok(ScalarFunctionType::Like), "ilike" => Ok(ScalarFunctionType::ILike), + "is_null" => Ok(ScalarFunctionType::IsNull), + "is_not_null" => Ok(ScalarFunctionType::IsNotNull), others => not_impl_err!("Unsupported function name: {others:?}"), } } @@ -880,6 +886,42 @@ pub async fn from_substrait_rex( ScalarFunctionType::ILike => { make_datafusion_like(true, f, input_schema, extensions).await } + ScalarFunctionType::IsNull => { + let arg = f.arguments.first().ok_or_else(|| { + DataFusionError::Substrait( + "expect one argument for `IS NULL` expr".to_string(), + ) + })?; + match &arg.arg_type { + Some(ArgType::Value(e)) => { + let expr = from_substrait_rex(e, input_schema, extensions) + .await? + .as_ref() + .clone(); + Ok(Arc::new(Expr::IsNull(Box::new(expr)))) + } + _ => not_impl_err!("Invalid arguments for IS NULL expression"), + } + } + ScalarFunctionType::IsNotNull => { + let arg = f.arguments.first().ok_or_else(|| { + DataFusionError::Substrait( + "expect one argument for `IS NOT NULL` expr".to_string(), + ) + })?; + match &arg.arg_type { + Some(ArgType::Value(e)) => { + let expr = from_substrait_rex(e, input_schema, extensions) + .await? + .as_ref() + .clone(); + Ok(Arc::new(Expr::IsNotNull(Box::new(expr)))) + } + _ => { + not_impl_err!("Invalid arguments for IS NOT NULL expression") + } + } + } } } Some(RexType::Literal(lit)) => { diff --git a/datafusion/substrait/src/logical_plan/producer.rs b/datafusion/substrait/src/logical_plan/producer.rs index e3c6f94d43d5..142b6c3628bb 100644 --- a/datafusion/substrait/src/logical_plan/producer.rs +++ b/datafusion/substrait/src/logical_plan/producer.rs @@ -1025,7 +1025,53 @@ pub fn to_substrait_rex( col_ref_offset, extension_info, ), - _ => not_impl_err!("Unsupported expression: {expr:?}"), + Expr::IsNull(arg) => { + let arguments: Vec = vec![FunctionArgument { + arg_type: Some(ArgType::Value(to_substrait_rex( + arg, + schema, + col_ref_offset, + extension_info, + )?)), + }]; + + let function_name = "is_null".to_string(); + let function_anchor = _register_function(function_name, extension_info); + Ok(Expression { + rex_type: Some(RexType::ScalarFunction(ScalarFunction { + function_reference: function_anchor, + arguments, + output_type: None, + args: vec![], + options: vec![], + })), + }) + } + Expr::IsNotNull(arg) => { + let arguments: Vec = vec![FunctionArgument { + arg_type: Some(ArgType::Value(to_substrait_rex( + arg, + schema, + col_ref_offset, + extension_info, + )?)), + }]; + + let function_name = "is_not_null".to_string(); + let function_anchor = _register_function(function_name, extension_info); + Ok(Expression { + rex_type: Some(RexType::ScalarFunction(ScalarFunction { + function_reference: function_anchor, + arguments, + output_type: None, + args: vec![], + options: vec![], + })), + }) + } + _ => { + not_impl_err!("Unsupported expression: {expr:?}") + } } } diff --git a/datafusion/substrait/tests/cases/roundtrip_logical_plan.rs b/datafusion/substrait/tests/cases/roundtrip_logical_plan.rs index ca2b4d48c460..582e5a5d7c8e 100644 --- a/datafusion/substrait/tests/cases/roundtrip_logical_plan.rs +++ b/datafusion/substrait/tests/cases/roundtrip_logical_plan.rs @@ -314,6 +314,16 @@ async fn simple_scalar_function_substr() -> Result<()> { roundtrip("SELECT * FROM data WHERE a = SUBSTR('datafusion', 0, 3)").await } +#[tokio::test] +async fn simple_scalar_function_is_null() -> Result<()> { + roundtrip("SELECT * FROM data WHERE a IS NULL").await +} + +#[tokio::test] +async fn simple_scalar_function_is_not_null() -> Result<()> { + roundtrip("SELECT * FROM data WHERE a IS NOT NULL").await +} + #[tokio::test] async fn case_without_base_expression() -> Result<()> { roundtrip("SELECT (CASE WHEN a >= 0 THEN 'positive' ELSE 'negative' END) FROM data") From 96ef1af62fccc403ff7532545618b50da6ad6c9d Mon Sep 17 00:00:00 2001 From: Andrew Lamb Date: Sun, 12 Nov 2023 06:29:32 -0500 Subject: [PATCH 235/572] Combine `Wildcard` and `QualifiedWildcard`, add `wildcard()` expr fn (#8105) --- .../core/src/datasource/listing/helpers.rs | 3 +- datafusion/core/src/physical_planner.rs | 5 +- datafusion/core/tests/dataframe/mod.rs | 29 +++--- datafusion/expr/src/expr.rs | 29 +++--- datafusion/expr/src/expr_fn.rs | 13 +++ datafusion/expr/src/expr_schema.rs | 15 ++- datafusion/expr/src/logical_plan/builder.rs | 13 ++- datafusion/expr/src/tree_node/expr.rs | 8 +- datafusion/expr/src/utils.rs | 3 +- .../src/analyzer/count_wildcard_rule.rs | 66 +++++++------ .../src/analyzer/inline_table_scan.rs | 2 +- .../optimizer/src/common_subexpr_eliminate.rs | 2 +- datafusion/optimizer/src/push_down_filter.rs | 3 +- .../simplify_expressions/expr_simplifier.rs | 3 +- datafusion/proto/proto/datafusion.proto | 6 +- datafusion/proto/src/generated/pbjson.rs | 94 ++++++++++++++++++- datafusion/proto/src/generated/prost.rs | 10 +- .../proto/src/logical_plan/from_proto.rs | 4 +- datafusion/proto/src/logical_plan/to_proto.rs | 11 +-- .../tests/cases/roundtrip_logical_plan.rs | 12 ++- datafusion/sql/src/expr/function.rs | 6 +- 21 files changed, 229 insertions(+), 108 deletions(-) diff --git a/datafusion/core/src/datasource/listing/helpers.rs b/datafusion/core/src/datasource/listing/helpers.rs index 986e54ebbe85..1d929f4bd4b1 100644 --- a/datafusion/core/src/datasource/listing/helpers.rs +++ b/datafusion/core/src/datasource/listing/helpers.rs @@ -120,8 +120,7 @@ pub fn expr_applicable_for_cols(col_names: &[String], expr: &Expr) -> bool { | Expr::AggregateFunction { .. } | Expr::Sort { .. } | Expr::WindowFunction { .. } - | Expr::Wildcard - | Expr::QualifiedWildcard { .. } + | Expr::Wildcard { .. } | Expr::Placeholder(_) => { is_applicable = false; VisitRecursion::Stop diff --git a/datafusion/core/src/physical_planner.rs b/datafusion/core/src/physical_planner.rs index f941e88f3a36..9f9b529ace03 100644 --- a/datafusion/core/src/physical_planner.rs +++ b/datafusion/core/src/physical_planner.rs @@ -364,9 +364,8 @@ fn create_physical_name(e: &Expr, is_first_expr: bool) -> Result { Expr::Sort { .. } => { internal_err!("Create physical name does not support sort expression") } - Expr::Wildcard => internal_err!("Create physical name does not support wildcard"), - Expr::QualifiedWildcard { .. } => { - internal_err!("Create physical name does not support qualified wildcard") + Expr::Wildcard { .. } => { + internal_err!("Create physical name does not support wildcard") } Expr::Placeholder(_) => { internal_err!("Create physical name does not support placeholder") diff --git a/datafusion/core/tests/dataframe/mod.rs b/datafusion/core/tests/dataframe/mod.rs index 845d77581b59..10f4574020bf 100644 --- a/datafusion/core/tests/dataframe/mod.rs +++ b/datafusion/core/tests/dataframe/mod.rs @@ -42,10 +42,9 @@ use datafusion::{assert_batches_eq, assert_batches_sorted_eq}; use datafusion_common::{DataFusionError, ScalarValue, UnnestOptions}; use datafusion_execution::config::SessionConfig; use datafusion_expr::expr::{GroupingSet, Sort}; -use datafusion_expr::Expr::Wildcard; use datafusion_expr::{ array_agg, avg, col, count, exists, expr, in_subquery, lit, max, out_ref_col, - scalar_subquery, sum, AggregateFunction, Expr, ExprSchemable, WindowFrame, + scalar_subquery, sum, wildcard, AggregateFunction, Expr, ExprSchemable, WindowFrame, WindowFrameBound, WindowFrameUnits, WindowFunction, }; use datafusion_physical_expr::var_provider::{VarProvider, VarType}; @@ -64,8 +63,8 @@ async fn test_count_wildcard_on_sort() -> Result<()> { let df_results = ctx .table("t1") .await? - .aggregate(vec![col("b")], vec![count(Wildcard)])? - .sort(vec![count(Wildcard).sort(true, false)])? + .aggregate(vec![col("b")], vec![count(wildcard())])? + .sort(vec![count(wildcard()).sort(true, false)])? .explain(false, false)? .collect() .await?; @@ -99,8 +98,8 @@ async fn test_count_wildcard_on_where_in() -> Result<()> { Arc::new( ctx.table("t2") .await? - .aggregate(vec![], vec![count(Expr::Wildcard)])? - .select(vec![count(Expr::Wildcard)])? + .aggregate(vec![], vec![count(wildcard())])? + .select(vec![count(wildcard())])? .into_unoptimized_plan(), // Usually, into_optimized_plan() should be used here, but due to // https://github.com/apache/arrow-datafusion/issues/5771, @@ -136,8 +135,8 @@ async fn test_count_wildcard_on_where_exist() -> Result<()> { .filter(exists(Arc::new( ctx.table("t2") .await? - .aggregate(vec![], vec![count(Expr::Wildcard)])? - .select(vec![count(Expr::Wildcard)])? + .aggregate(vec![], vec![count(wildcard())])? + .select(vec![count(wildcard())])? .into_unoptimized_plan(), // Usually, into_optimized_plan() should be used here, but due to // https://github.com/apache/arrow-datafusion/issues/5771, @@ -172,7 +171,7 @@ async fn test_count_wildcard_on_window() -> Result<()> { .await? .select(vec![Expr::WindowFunction(expr::WindowFunction::new( WindowFunction::AggregateFunction(AggregateFunction::Count), - vec![Expr::Wildcard], + vec![wildcard()], vec![], vec![Expr::Sort(Sort::new(Box::new(col("a")), false, true))], WindowFrame { @@ -202,17 +201,17 @@ async fn test_count_wildcard_on_aggregate() -> Result<()> { let sql_results = ctx .sql("select count(*) from t1") .await? - .select(vec![count(Expr::Wildcard)])? + .select(vec![count(wildcard())])? .explain(false, false)? .collect() .await?; - // add `.select(vec![count(Expr::Wildcard)])?` to make sure we can analyze all node instead of just top node. + // add `.select(vec![count(wildcard())])?` to make sure we can analyze all node instead of just top node. let df_results = ctx .table("t1") .await? - .aggregate(vec![], vec![count(Expr::Wildcard)])? - .select(vec![count(Expr::Wildcard)])? + .aggregate(vec![], vec![count(wildcard())])? + .select(vec![count(wildcard())])? .explain(false, false)? .collect() .await?; @@ -248,8 +247,8 @@ async fn test_count_wildcard_on_where_scalar_subquery() -> Result<()> { ctx.table("t2") .await? .filter(out_ref_col(DataType::UInt32, "t1.a").eq(col("t2.a")))? - .aggregate(vec![], vec![count(Wildcard)])? - .select(vec![col(count(Wildcard).to_string())])? + .aggregate(vec![], vec![count(wildcard())])? + .select(vec![col(count(wildcard()).to_string())])? .into_unoptimized_plan(), )) .gt(lit(ScalarValue::UInt8(Some(0)))), diff --git a/datafusion/expr/src/expr.rs b/datafusion/expr/src/expr.rs index 8929b21f4412..4267f182bda8 100644 --- a/datafusion/expr/src/expr.rs +++ b/datafusion/expr/src/expr.rs @@ -166,16 +166,12 @@ pub enum Expr { InSubquery(InSubquery), /// Scalar subquery ScalarSubquery(Subquery), - /// Represents a reference to all available fields. + /// Represents a reference to all available fields in a specific schema, + /// with an optional (schema) qualifier. /// /// This expr has to be resolved to a list of columns before translating logical /// plan into physical plan. - Wildcard, - /// Represents a reference to all available fields in a specific schema. - /// - /// This expr has to be resolved to a list of columns before translating logical - /// plan into physical plan. - QualifiedWildcard { qualifier: String }, + Wildcard { qualifier: Option }, /// List of grouping set expressions. Only valid in the context of an aggregate /// GROUP BY expression list GroupingSet(GroupingSet), @@ -729,7 +725,6 @@ impl Expr { Expr::Negative(..) => "Negative", Expr::Not(..) => "Not", Expr::Placeholder(_) => "Placeholder", - Expr::QualifiedWildcard { .. } => "QualifiedWildcard", Expr::ScalarFunction(..) => "ScalarFunction", Expr::ScalarSubquery { .. } => "ScalarSubquery", Expr::ScalarUDF(..) => "ScalarUDF", @@ -737,7 +732,7 @@ impl Expr { Expr::Sort { .. } => "Sort", Expr::TryCast { .. } => "TryCast", Expr::WindowFunction { .. } => "WindowFunction", - Expr::Wildcard => "Wildcard", + Expr::Wildcard { .. } => "Wildcard", } } @@ -1292,8 +1287,10 @@ impl fmt::Display for Expr { write!(f, "{expr} IN ([{}])", expr_vec_fmt!(list)) } } - Expr::Wildcard => write!(f, "*"), - Expr::QualifiedWildcard { qualifier } => write!(f, "{qualifier}.*"), + Expr::Wildcard { qualifier } => match qualifier { + Some(qualifier) => write!(f, "{qualifier}.*"), + None => write!(f, "*"), + }, Expr::GetIndexedField(GetIndexedField { field, expr }) => match field { GetFieldAccess::NamedStructField { name } => { write!(f, "({expr})[{name}]") @@ -1613,10 +1610,12 @@ fn create_name(e: &Expr) -> Result { Expr::Sort { .. } => { internal_err!("Create name does not support sort expression") } - Expr::Wildcard => Ok("*".to_string()), - Expr::QualifiedWildcard { .. } => { - internal_err!("Create name does not support qualified wildcard") - } + Expr::Wildcard { qualifier } => match qualifier { + Some(qualifier) => internal_err!( + "Create name does not support qualified wildcard, got {qualifier}" + ), + None => Ok("*".to_string()), + }, Expr::Placeholder(Placeholder { id, .. }) => Ok((*id).to_string()), } } diff --git a/datafusion/expr/src/expr_fn.rs b/datafusion/expr/src/expr_fn.rs index 98cacc039228..0e0ad46da101 100644 --- a/datafusion/expr/src/expr_fn.rs +++ b/datafusion/expr/src/expr_fn.rs @@ -99,6 +99,19 @@ pub fn placeholder(id: impl Into) -> Expr { }) } +/// Create an '*' [`Expr::Wildcard`] expression that matches all columns +/// +/// # Example +/// +/// ```rust +/// # use datafusion_expr::{wildcard}; +/// let p = wildcard(); +/// assert_eq!(p.to_string(), "*") +/// ``` +pub fn wildcard() -> Expr { + Expr::Wildcard { qualifier: None } +} + /// Return a new expression `left right` pub fn binary_expr(left: Expr, op: Operator, right: Expr) -> Expr { Expr::BinaryExpr(BinaryExpr::new(Box::new(left), op, Box::new(right))) diff --git a/datafusion/expr/src/expr_schema.rs b/datafusion/expr/src/expr_schema.rs index 2889fac8c1ee..2631708fb780 100644 --- a/datafusion/expr/src/expr_schema.rs +++ b/datafusion/expr/src/expr_schema.rs @@ -157,13 +157,13 @@ impl ExprSchemable for Expr { plan_datafusion_err!("Placeholder type could not be resolved") }) } - Expr::Wildcard => { + Expr::Wildcard { qualifier } => { // Wildcard do not really have a type and do not appear in projections - Ok(DataType::Null) + match qualifier { + Some(_) => internal_err!("QualifiedWildcard expressions are not valid in a logical query plan"), + None => Ok(DataType::Null) + } } - Expr::QualifiedWildcard { .. } => internal_err!( - "QualifiedWildcard expressions are not valid in a logical query plan" - ), Expr::GroupingSet(_) => { // grouping sets do not really have a type and do not appear in projections Ok(DataType::Null) @@ -270,12 +270,9 @@ impl ExprSchemable for Expr { | Expr::SimilarTo(Like { expr, pattern, .. }) => { Ok(expr.nullable(input_schema)? || pattern.nullable(input_schema)?) } - Expr::Wildcard => internal_err!( + Expr::Wildcard { .. } => internal_err!( "Wildcard expressions are not valid in a logical query plan" ), - Expr::QualifiedWildcard { .. } => internal_err!( - "QualifiedWildcard expressions are not valid in a logical query plan" - ), Expr::GetIndexedField(GetIndexedField { expr, field }) => { field_for_index(expr, field, input_schema).map(|x| x.is_nullable()) } diff --git a/datafusion/expr/src/logical_plan/builder.rs b/datafusion/expr/src/logical_plan/builder.rs index 162a6a959e59..4a30f4e223bf 100644 --- a/datafusion/expr/src/logical_plan/builder.rs +++ b/datafusion/expr/src/logical_plan/builder.rs @@ -1287,11 +1287,16 @@ pub fn project( for e in expr { let e = e.into(); match e { - Expr::Wildcard => { + Expr::Wildcard { qualifier: None } => { projected_expr.extend(expand_wildcard(input_schema, &plan, None)?) } - Expr::QualifiedWildcard { ref qualifier } => projected_expr - .extend(expand_qualified_wildcard(qualifier, input_schema, None)?), + Expr::Wildcard { + qualifier: Some(qualifier), + } => projected_expr.extend(expand_qualified_wildcard( + &qualifier, + input_schema, + None, + )?), _ => projected_expr .push(columnize_expr(normalize_col(e, &plan)?, input_schema)), } @@ -1590,7 +1595,7 @@ mod tests { let plan = table_scan(Some("t1"), &employee_schema(), None)? .join_using(t2, JoinType::Inner, vec!["id"])? - .project(vec![Expr::Wildcard])? + .project(vec![Expr::Wildcard { qualifier: None }])? .build()?; // id column should only show up once in projection diff --git a/datafusion/expr/src/tree_node/expr.rs b/datafusion/expr/src/tree_node/expr.rs index 764dcffbced9..d6c14b86227a 100644 --- a/datafusion/expr/src/tree_node/expr.rs +++ b/datafusion/expr/src/tree_node/expr.rs @@ -77,8 +77,7 @@ impl TreeNode for Expr { | Expr::Literal(_) | Expr::Exists { .. } | Expr::ScalarSubquery(_) - | Expr::Wildcard - | Expr::QualifiedWildcard { .. } + | Expr::Wildcard {..} | Expr::Placeholder (_) => vec![], Expr::BinaryExpr(BinaryExpr { left, right, .. }) => { vec![left.as_ref().clone(), right.as_ref().clone()] @@ -350,10 +349,7 @@ impl TreeNode for Expr { transform_vec(list, &mut transform)?, negated, )), - Expr::Wildcard => Expr::Wildcard, - Expr::QualifiedWildcard { qualifier } => { - Expr::QualifiedWildcard { qualifier } - } + Expr::Wildcard { qualifier } => Expr::Wildcard { qualifier }, Expr::GetIndexedField(GetIndexedField { expr, field }) => { Expr::GetIndexedField(GetIndexedField::new( transform_boxed(expr, &mut transform)?, diff --git a/datafusion/expr/src/utils.rs b/datafusion/expr/src/utils.rs index 5fc5b5b3f9c7..a462cdb34631 100644 --- a/datafusion/expr/src/utils.rs +++ b/datafusion/expr/src/utils.rs @@ -292,8 +292,7 @@ pub fn expr_to_columns(expr: &Expr, accum: &mut HashSet) -> Result<()> { | Expr::Exists { .. } | Expr::InSubquery(_) | Expr::ScalarSubquery(_) - | Expr::Wildcard - | Expr::QualifiedWildcard { .. } + | Expr::Wildcard { .. } | Expr::GetIndexedField { .. } | Expr::Placeholder(_) | Expr::OuterReferenceColumn { .. } => {} diff --git a/datafusion/optimizer/src/analyzer/count_wildcard_rule.rs b/datafusion/optimizer/src/analyzer/count_wildcard_rule.rs index 912ac069e0b6..b4de322f76f6 100644 --- a/datafusion/optimizer/src/analyzer/count_wildcard_rule.rs +++ b/datafusion/optimizer/src/analyzer/count_wildcard_rule.rs @@ -129,15 +129,17 @@ impl TreeNodeRewriter for CountWildcardRewriter { order_by, window_frame, }) if args.len() == 1 => match args[0] { - Expr::Wildcard => Expr::WindowFunction(expr::WindowFunction { - fun: window_function::WindowFunction::AggregateFunction( - aggregate_function::AggregateFunction::Count, - ), - args: vec![lit(COUNT_STAR_EXPANSION)], - partition_by, - order_by, - window_frame, - }), + Expr::Wildcard { qualifier: None } => { + Expr::WindowFunction(expr::WindowFunction { + fun: window_function::WindowFunction::AggregateFunction( + aggregate_function::AggregateFunction::Count, + ), + args: vec![lit(COUNT_STAR_EXPANSION)], + partition_by, + order_by, + window_frame, + }) + } _ => old_expr, }, @@ -148,13 +150,15 @@ impl TreeNodeRewriter for CountWildcardRewriter { filter, order_by, }) if args.len() == 1 => match args[0] { - Expr::Wildcard => Expr::AggregateFunction(AggregateFunction { - fun: aggregate_function::AggregateFunction::Count, - args: vec![lit(COUNT_STAR_EXPANSION)], - distinct, - filter, - order_by, - }), + Expr::Wildcard { qualifier: None } => { + Expr::AggregateFunction(AggregateFunction { + fun: aggregate_function::AggregateFunction::Count, + args: vec![lit(COUNT_STAR_EXPANSION)], + distinct, + filter, + order_by, + }) + } _ => old_expr, }, @@ -221,8 +225,8 @@ mod tests { use datafusion_expr::expr::Sort; use datafusion_expr::{ col, count, exists, expr, in_subquery, lit, logical_plan::LogicalPlanBuilder, - max, out_ref_col, scalar_subquery, AggregateFunction, Expr, WindowFrame, - WindowFrameBound, WindowFrameUnits, WindowFunction, + max, out_ref_col, scalar_subquery, wildcard, AggregateFunction, Expr, + WindowFrame, WindowFrameBound, WindowFrameUnits, WindowFunction, }; fn assert_plan_eq(plan: &LogicalPlan, expected: &str) -> Result<()> { @@ -237,9 +241,9 @@ mod tests { fn test_count_wildcard_on_sort() -> Result<()> { let table_scan = test_table_scan()?; let plan = LogicalPlanBuilder::from(table_scan) - .aggregate(vec![col("b")], vec![count(Expr::Wildcard)])? - .project(vec![count(Expr::Wildcard)])? - .sort(vec![count(Expr::Wildcard).sort(true, false)])? + .aggregate(vec![col("b")], vec![count(wildcard())])? + .project(vec![count(wildcard())])? + .sort(vec![count(wildcard()).sort(true, false)])? .build()?; let expected = "Sort: COUNT(*) ASC NULLS LAST [COUNT(*):Int64;N]\ \n Projection: COUNT(*) [COUNT(*):Int64;N]\ @@ -258,8 +262,8 @@ mod tests { col("a"), Arc::new( LogicalPlanBuilder::from(table_scan_t2) - .aggregate(Vec::::new(), vec![count(Expr::Wildcard)])? - .project(vec![count(Expr::Wildcard)])? + .aggregate(Vec::::new(), vec![count(wildcard())])? + .project(vec![count(wildcard())])? .build()?, ), ))? @@ -282,8 +286,8 @@ mod tests { let plan = LogicalPlanBuilder::from(table_scan_t1) .filter(exists(Arc::new( LogicalPlanBuilder::from(table_scan_t2) - .aggregate(Vec::::new(), vec![count(Expr::Wildcard)])? - .project(vec![count(Expr::Wildcard)])? + .aggregate(Vec::::new(), vec![count(wildcard())])? + .project(vec![count(wildcard())])? .build()?, )))? .build()?; @@ -336,7 +340,7 @@ mod tests { let plan = LogicalPlanBuilder::from(table_scan) .window(vec![Expr::WindowFunction(expr::WindowFunction::new( WindowFunction::AggregateFunction(AggregateFunction::Count), - vec![Expr::Wildcard], + vec![wildcard()], vec![], vec![Expr::Sort(Sort::new(Box::new(col("a")), false, true))], WindowFrame { @@ -347,7 +351,7 @@ mod tests { end_bound: WindowFrameBound::Following(ScalarValue::UInt32(Some(2))), }, ))])? - .project(vec![count(Expr::Wildcard)])? + .project(vec![count(wildcard())])? .build()?; let expected = "Projection: COUNT(UInt8(1)) AS COUNT(*) [COUNT(*):Int64;N]\ @@ -360,8 +364,8 @@ mod tests { fn test_count_wildcard_on_aggregate() -> Result<()> { let table_scan = test_table_scan()?; let plan = LogicalPlanBuilder::from(table_scan) - .aggregate(Vec::::new(), vec![count(Expr::Wildcard)])? - .project(vec![count(Expr::Wildcard)])? + .aggregate(Vec::::new(), vec![count(wildcard())])? + .project(vec![count(wildcard())])? .build()?; let expected = "Projection: COUNT(*) [COUNT(*):Int64;N]\ @@ -374,8 +378,8 @@ mod tests { fn test_count_wildcard_on_nesting() -> Result<()> { let table_scan = test_table_scan()?; let plan = LogicalPlanBuilder::from(table_scan) - .aggregate(Vec::::new(), vec![max(count(Expr::Wildcard))])? - .project(vec![count(Expr::Wildcard)])? + .aggregate(Vec::::new(), vec![max(count(wildcard()))])? + .project(vec![count(wildcard())])? .build()?; let expected = "Projection: COUNT(UInt8(1)) AS COUNT(*) [COUNT(*):Int64;N]\ diff --git a/datafusion/optimizer/src/analyzer/inline_table_scan.rs b/datafusion/optimizer/src/analyzer/inline_table_scan.rs index 3d0dabdd377c..90af7aec8293 100644 --- a/datafusion/optimizer/src/analyzer/inline_table_scan.rs +++ b/datafusion/optimizer/src/analyzer/inline_table_scan.rs @@ -126,7 +126,7 @@ fn generate_projection_expr( )); } } else { - exprs.push(Expr::Wildcard); + exprs.push(Expr::Wildcard { qualifier: None }); } Ok(exprs) } diff --git a/datafusion/optimizer/src/common_subexpr_eliminate.rs b/datafusion/optimizer/src/common_subexpr_eliminate.rs index 68a6a5607a1d..8025402ccef5 100644 --- a/datafusion/optimizer/src/common_subexpr_eliminate.rs +++ b/datafusion/optimizer/src/common_subexpr_eliminate.rs @@ -514,7 +514,7 @@ impl ExprMask { | Expr::ScalarVariable(..) | Expr::Alias(..) | Expr::Sort { .. } - | Expr::Wildcard + | Expr::Wildcard { .. } ); let is_aggr = matches!( diff --git a/datafusion/optimizer/src/push_down_filter.rs b/datafusion/optimizer/src/push_down_filter.rs index ae986b3c84dd..05f4072e3857 100644 --- a/datafusion/optimizer/src/push_down_filter.rs +++ b/datafusion/optimizer/src/push_down_filter.rs @@ -250,8 +250,7 @@ fn can_evaluate_as_join_condition(predicate: &Expr) -> Result { | Expr::AggregateFunction(_) | Expr::WindowFunction(_) | Expr::AggregateUDF { .. } - | Expr::Wildcard - | Expr::QualifiedWildcard { .. } + | Expr::Wildcard { .. } | Expr::GroupingSet(_) => internal_err!("Unsupported predicate type"), })?; Ok(is_evaluate) diff --git a/datafusion/optimizer/src/simplify_expressions/expr_simplifier.rs b/datafusion/optimizer/src/simplify_expressions/expr_simplifier.rs index 04fdcca0a994..c5a1aacce745 100644 --- a/datafusion/optimizer/src/simplify_expressions/expr_simplifier.rs +++ b/datafusion/optimizer/src/simplify_expressions/expr_simplifier.rs @@ -343,8 +343,7 @@ impl<'a> ConstEvaluator<'a> { | Expr::WindowFunction { .. } | Expr::Sort { .. } | Expr::GroupingSet(_) - | Expr::Wildcard - | Expr::QualifiedWildcard { .. } + | Expr::Wildcard { .. } | Expr::Placeholder(_) => false, Expr::ScalarFunction(ScalarFunction { fun, .. }) => { Self::volatility_ok(fun.volatility()) diff --git a/datafusion/proto/proto/datafusion.proto b/datafusion/proto/proto/datafusion.proto index f9deca2f1e52..9dcd55e731bb 100644 --- a/datafusion/proto/proto/datafusion.proto +++ b/datafusion/proto/proto/datafusion.proto @@ -363,7 +363,7 @@ message LogicalExprNode { SortExprNode sort = 12; NegativeNode negative = 13; InListNode in_list = 14; - bool wildcard = 15; + Wildcard wildcard = 15; ScalarFunctionNode scalar_function = 16; TryCastNode try_cast = 17; @@ -399,6 +399,10 @@ message LogicalExprNode { } } +message Wildcard { + optional string qualifier = 1; +} + message PlaceholderNode { string id = 1; ArrowType data_type = 2; diff --git a/datafusion/proto/src/generated/pbjson.rs b/datafusion/proto/src/generated/pbjson.rs index 81f260c28bed..948ad0c4cedb 100644 --- a/datafusion/proto/src/generated/pbjson.rs +++ b/datafusion/proto/src/generated/pbjson.rs @@ -12705,7 +12705,8 @@ impl<'de> serde::Deserialize<'de> for LogicalExprNode { if expr_type__.is_some() { return Err(serde::de::Error::duplicate_field("wildcard")); } - expr_type__ = map_.next_value::<::std::option::Option<_>>()?.map(logical_expr_node::ExprType::Wildcard); + expr_type__ = map_.next_value::<::std::option::Option<_>>()?.map(logical_expr_node::ExprType::Wildcard) +; } GeneratedField::ScalarFunction => { if expr_type__.is_some() { @@ -25082,6 +25083,97 @@ impl<'de> serde::Deserialize<'de> for WhenThen { deserializer.deserialize_struct("datafusion.WhenThen", FIELDS, GeneratedVisitor) } } +impl serde::Serialize for Wildcard { + #[allow(deprecated)] + fn serialize(&self, serializer: S) -> std::result::Result + where + S: serde::Serializer, + { + use serde::ser::SerializeStruct; + let mut len = 0; + if self.qualifier.is_some() { + len += 1; + } + let mut struct_ser = serializer.serialize_struct("datafusion.Wildcard", len)?; + if let Some(v) = self.qualifier.as_ref() { + struct_ser.serialize_field("qualifier", v)?; + } + struct_ser.end() + } +} +impl<'de> serde::Deserialize<'de> for Wildcard { + #[allow(deprecated)] + fn deserialize(deserializer: D) -> std::result::Result + where + D: serde::Deserializer<'de>, + { + const FIELDS: &[&str] = &[ + "qualifier", + ]; + + #[allow(clippy::enum_variant_names)] + enum GeneratedField { + Qualifier, + } + impl<'de> serde::Deserialize<'de> for GeneratedField { + fn deserialize(deserializer: D) -> std::result::Result + where + D: serde::Deserializer<'de>, + { + struct GeneratedVisitor; + + impl<'de> serde::de::Visitor<'de> for GeneratedVisitor { + type Value = GeneratedField; + + fn expecting(&self, formatter: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + write!(formatter, "expected one of: {:?}", &FIELDS) + } + + #[allow(unused_variables)] + fn visit_str(self, value: &str) -> std::result::Result + where + E: serde::de::Error, + { + match value { + "qualifier" => Ok(GeneratedField::Qualifier), + _ => Err(serde::de::Error::unknown_field(value, FIELDS)), + } + } + } + deserializer.deserialize_identifier(GeneratedVisitor) + } + } + struct GeneratedVisitor; + impl<'de> serde::de::Visitor<'de> for GeneratedVisitor { + type Value = Wildcard; + + fn expecting(&self, formatter: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + formatter.write_str("struct datafusion.Wildcard") + } + + fn visit_map(self, mut map_: V) -> std::result::Result + where + V: serde::de::MapAccess<'de>, + { + let mut qualifier__ = None; + while let Some(k) = map_.next_key()? { + match k { + GeneratedField::Qualifier => { + if qualifier__.is_some() { + return Err(serde::de::Error::duplicate_field("qualifier")); + } + qualifier__ = map_.next_value()?; + } + } + } + Ok(Wildcard { + qualifier: qualifier__, + }) + } + } + deserializer.deserialize_struct("datafusion.Wildcard", FIELDS, GeneratedVisitor) + } +} impl serde::Serialize for WindowAggExecNode { #[allow(deprecated)] fn serialize(&self, serializer: S) -> std::result::Result diff --git a/datafusion/proto/src/generated/prost.rs b/datafusion/proto/src/generated/prost.rs index ae64c11b3b74..93b0a05c314d 100644 --- a/datafusion/proto/src/generated/prost.rs +++ b/datafusion/proto/src/generated/prost.rs @@ -569,8 +569,8 @@ pub mod logical_expr_node { Negative(::prost::alloc::boxed::Box), #[prost(message, tag = "14")] InList(::prost::alloc::boxed::Box), - #[prost(bool, tag = "15")] - Wildcard(bool), + #[prost(message, tag = "15")] + Wildcard(super::Wildcard), #[prost(message, tag = "16")] ScalarFunction(super::ScalarFunctionNode), #[prost(message, tag = "17")] @@ -616,6 +616,12 @@ pub mod logical_expr_node { } #[allow(clippy::derive_partial_eq_without_eq)] #[derive(Clone, PartialEq, ::prost::Message)] +pub struct Wildcard { + #[prost(string, optional, tag = "1")] + pub qualifier: ::core::option::Option<::prost::alloc::string::String>, +} +#[allow(clippy::derive_partial_eq_without_eq)] +#[derive(Clone, PartialEq, ::prost::Message)] pub struct PlaceholderNode { #[prost(string, tag = "1")] pub id: ::prost::alloc::string::String, diff --git a/datafusion/proto/src/logical_plan/from_proto.rs b/datafusion/proto/src/logical_plan/from_proto.rs index 31fffca3bbed..b2b66693f78d 100644 --- a/datafusion/proto/src/logical_plan/from_proto.rs +++ b/datafusion/proto/src/logical_plan/from_proto.rs @@ -1296,7 +1296,9 @@ pub fn parse_expr( .collect::, _>>()?, in_list.negated, ))), - ExprType::Wildcard(_) => Ok(Expr::Wildcard), + ExprType::Wildcard(protobuf::Wildcard { qualifier }) => Ok(Expr::Wildcard { + qualifier: qualifier.clone(), + }), ExprType::ScalarFunction(expr) => { let scalar_function = protobuf::ScalarFunction::try_from(expr.fun) .map_err(|_| Error::unknown("ScalarFunction", expr.fun))?; diff --git a/datafusion/proto/src/logical_plan/to_proto.rs b/datafusion/proto/src/logical_plan/to_proto.rs index 803becbcaece..e590731f5810 100644 --- a/datafusion/proto/src/logical_plan/to_proto.rs +++ b/datafusion/proto/src/logical_plan/to_proto.rs @@ -960,8 +960,10 @@ impl TryFrom<&Expr> for protobuf::LogicalExprNode { expr_type: Some(ExprType::InList(expr)), } } - Expr::Wildcard => Self { - expr_type: Some(ExprType::Wildcard(true)), + Expr::Wildcard { qualifier } => Self { + expr_type: Some(ExprType::Wildcard(protobuf::Wildcard { + qualifier: qualifier.clone(), + })), }, Expr::ScalarSubquery(_) | Expr::InSubquery(_) @@ -1052,11 +1054,6 @@ impl TryFrom<&Expr> for protobuf::LogicalExprNode { })), } } - - Expr::QualifiedWildcard { .. } => return Err(Error::General( - "Proto serialization error: Expr::QualifiedWildcard { .. } not supported" - .to_string(), - )), }; Ok(expr_node) diff --git a/datafusion/proto/tests/cases/roundtrip_logical_plan.rs b/datafusion/proto/tests/cases/roundtrip_logical_plan.rs index ca801df337f1..97c553dc04e6 100644 --- a/datafusion/proto/tests/cases/roundtrip_logical_plan.rs +++ b/datafusion/proto/tests/cases/roundtrip_logical_plan.rs @@ -1147,7 +1147,17 @@ fn roundtrip_inlist() { #[test] fn roundtrip_wildcard() { - let test_expr = Expr::Wildcard; + let test_expr = Expr::Wildcard { qualifier: None }; + + let ctx = SessionContext::new(); + roundtrip_expr_test(test_expr, ctx); +} + +#[test] +fn roundtrip_qualified_wildcard() { + let test_expr = Expr::Wildcard { + qualifier: Some("foo".into()), + }; let ctx = SessionContext::new(); roundtrip_expr_test(test_expr, ctx); diff --git a/datafusion/sql/src/expr/function.rs b/datafusion/sql/src/expr/function.rs index c58b8319ceb7..c77ef64718bb 100644 --- a/datafusion/sql/src/expr/function.rs +++ b/datafusion/sql/src/expr/function.rs @@ -212,11 +212,13 @@ impl<'a, S: ContextProvider> SqlToRel<'a, S> { FunctionArg::Named { name: _, arg: FunctionArgExpr::Wildcard, - } => Ok(Expr::Wildcard), + } => Ok(Expr::Wildcard { qualifier: None }), FunctionArg::Unnamed(FunctionArgExpr::Expr(arg)) => { self.sql_expr_to_logical_expr(arg, schema, planner_context) } - FunctionArg::Unnamed(FunctionArgExpr::Wildcard) => Ok(Expr::Wildcard), + FunctionArg::Unnamed(FunctionArgExpr::Wildcard) => { + Ok(Expr::Wildcard { qualifier: None }) + } _ => not_impl_err!("Unsupported qualified wildcard argument: {sql:?}"), } } From f67c20f9d21e123840374d2198b81cfd2757c651 Mon Sep 17 00:00:00 2001 From: Will Jones Date: Sun, 12 Nov 2023 03:31:48 -0800 Subject: [PATCH 236/572] docs: show creation of DFSchema (#8132) * docs: show creation of DFSchema * Apply suggestions from code review Co-authored-by: Andy Grove --------- Co-authored-by: Andy Grove --- datafusion/common/src/dfschema.rs | 72 ++++++++++++++++++++++++++++++- 1 file changed, 70 insertions(+), 2 deletions(-) diff --git a/datafusion/common/src/dfschema.rs b/datafusion/common/src/dfschema.rs index d8cd103a4777..52cd85675824 100644 --- a/datafusion/common/src/dfschema.rs +++ b/datafusion/common/src/dfschema.rs @@ -34,10 +34,75 @@ use crate::{ use arrow::compute::can_cast_types; use arrow::datatypes::{DataType, Field, FieldRef, Fields, Schema, SchemaRef}; -/// A reference-counted reference to a `DFSchema`. +/// A reference-counted reference to a [DFSchema]. pub type DFSchemaRef = Arc; -/// DFSchema wraps an Arrow schema and adds relation names +/// DFSchema wraps an Arrow schema and adds relation names. +/// +/// The schema may hold the fields across multiple tables. Some fields may be +/// qualified and some unqualified. A qualified field is a field that has a +/// relation name associated with it. +/// +/// Unqualified fields must be unique not only amongst themselves, but also must +/// have a distinct name from any qualified field names. This allows finding a +/// qualified field by name to be possible, so long as there aren't multiple +/// qualified fields with the same name. +/// +/// There is an alias to `Arc` named [DFSchemaRef]. +/// +/// # Creating qualified schemas +/// +/// Use [DFSchema::try_from_qualified_schema] to create a qualified schema from +/// an Arrow schema. +/// +/// ```rust +/// use datafusion_common::{DFSchema, Column}; +/// use arrow_schema::{DataType, Field, Schema}; +/// +/// let arrow_schema = Schema::new(vec![ +/// Field::new("c1", DataType::Int32, false), +/// ]); +/// +/// let df_schema = DFSchema::try_from_qualified_schema("t1", &arrow_schema).unwrap(); +/// let column = Column::from_qualified_name("t1.c1"); +/// assert!(df_schema.has_column(&column)); +/// +/// // Can also access qualified fields with unqualified name, if it's unambiguous +/// let column = Column::from_qualified_name("c1"); +/// assert!(df_schema.has_column(&column)); +/// ``` +/// +/// # Creating unqualified schemas +/// +/// Create an unqualified schema using TryFrom: +/// +/// ```rust +/// use datafusion_common::{DFSchema, Column}; +/// use arrow_schema::{DataType, Field, Schema}; +/// +/// let arrow_schema = Schema::new(vec![ +/// Field::new("c1", DataType::Int32, false), +/// ]); +/// +/// let df_schema = DFSchema::try_from(arrow_schema).unwrap(); +/// let column = Column::new_unqualified("c1"); +/// assert!(df_schema.has_column(&column)); +/// ``` +/// +/// # Converting back to Arrow schema +/// +/// Use the `Into` trait to convert `DFSchema` into an Arrow schema: +/// +/// ```rust +/// use datafusion_common::{DFSchema, DFField}; +/// use arrow_schema::Schema; +/// +/// let df_schema = DFSchema::new(vec![ +/// DFField::new_unqualified("c1", arrow::datatypes::DataType::Int32, false), +/// ]).unwrap(); +/// let schema = Schema::from(df_schema); +/// assert_eq!(schema.fields().len(), 1); +/// ``` #[derive(Debug, Clone, PartialEq, Eq)] pub struct DFSchema { /// Fields @@ -112,6 +177,9 @@ impl DFSchema { } /// Create a `DFSchema` from an Arrow schema and a given qualifier + /// + /// To create a schema from an Arrow schema without a qualifier, use + /// `DFSchema::try_from`. pub fn try_from_qualified_schema<'a>( qualifier: impl Into>, schema: &Schema, From 824bb66370eba3cd93a21b4a594315322d4c1718 Mon Sep 17 00:00:00 2001 From: Ruihang Xia Date: Sun, 12 Nov 2023 19:57:41 +0800 Subject: [PATCH 237/572] feat: support UDAF in substrait producer/consumer (#8119) * feat: support UDAF in substrait producer/consumer Signed-off-by: Ruihang Xia * Update datafusion/substrait/src/logical_plan/consumer.rs Co-authored-by: Andrew Lamb * remove redundent to_lowercase Signed-off-by: Ruihang Xia --------- Signed-off-by: Ruihang Xia Co-authored-by: Andrew Lamb --- .../substrait/src/logical_plan/consumer.rs | 45 +++++++++---- .../substrait/src/logical_plan/producer.rs | 41 +++++++++--- .../tests/cases/roundtrip_logical_plan.rs | 64 ++++++++++++++++++- 3 files changed, 125 insertions(+), 25 deletions(-) diff --git a/datafusion/substrait/src/logical_plan/consumer.rs b/datafusion/substrait/src/logical_plan/consumer.rs index c6bcbb479e80..f4c36557dac8 100644 --- a/datafusion/substrait/src/logical_plan/consumer.rs +++ b/datafusion/substrait/src/logical_plan/consumer.rs @@ -19,6 +19,7 @@ use async_recursion::async_recursion; use datafusion::arrow::datatypes::{DataType, Field, TimeUnit}; use datafusion::common::{not_impl_err, DFField, DFSchema, DFSchemaRef}; +use datafusion::execution::FunctionRegistry; use datafusion::logical_expr::{ aggregate_function, window_function::find_df_window_func, BinaryExpr, BuiltinScalarFunction, Case, Expr, LogicalPlan, Operator, @@ -365,6 +366,7 @@ pub async fn from_substrait_rel( _ => false, }; from_substrait_agg_func( + ctx, f, input.schema(), extensions, @@ -660,6 +662,7 @@ pub async fn from_substriat_func_args( /// Convert Substrait AggregateFunction to DataFusion Expr pub async fn from_substrait_agg_func( + ctx: &SessionContext, f: &AggregateFunction, input_schema: &DFSchema, extensions: &HashMap, @@ -680,23 +683,37 @@ pub async fn from_substrait_agg_func( args.push(arg_expr?.as_ref().clone()); } - let fun = match extensions.get(&f.function_reference) { - Some(function_name) => { - aggregate_function::AggregateFunction::from_str(function_name) - } - None => not_impl_err!( - "Aggregated function not found: function anchor = {:?}", + let Some(function_name) = extensions.get(&f.function_reference) else { + return plan_err!( + "Aggregate function not registered: function anchor = {:?}", f.function_reference - ), + ); }; - Ok(Arc::new(Expr::AggregateFunction(expr::AggregateFunction { - fun: fun.unwrap(), - args, - distinct, - filter, - order_by, - }))) + // try udaf first, then built-in aggr fn. + if let Ok(fun) = ctx.udaf(function_name) { + Ok(Arc::new(Expr::AggregateUDF(expr::AggregateUDF { + fun, + args, + filter, + order_by, + }))) + } else if let Ok(fun) = aggregate_function::AggregateFunction::from_str(function_name) + { + Ok(Arc::new(Expr::AggregateFunction(expr::AggregateFunction { + fun, + args, + distinct, + filter, + order_by, + }))) + } else { + not_impl_err!( + "Aggregated function {} is not supported: function anchor = {:?}", + function_name, + f.function_reference + ) + } } /// Convert Substrait Rex to DataFusion Expr diff --git a/datafusion/substrait/src/logical_plan/producer.rs b/datafusion/substrait/src/logical_plan/producer.rs index 142b6c3628bb..6fe8eca33705 100644 --- a/datafusion/substrait/src/logical_plan/producer.rs +++ b/datafusion/substrait/src/logical_plan/producer.rs @@ -588,8 +588,7 @@ pub fn to_substrait_agg_measure( for arg in args { arguments.push(FunctionArgument { arg_type: Some(ArgType::Value(to_substrait_rex(arg, schema, 0, extension_info)?)) }); } - let function_name = fun.to_string().to_lowercase(); - let function_anchor = _register_function(function_name, extension_info); + let function_anchor = _register_function(fun.to_string(), extension_info); Ok(Measure { measure: Some(AggregateFunction { function_reference: function_anchor, @@ -610,6 +609,34 @@ pub fn to_substrait_agg_measure( } }) } + Expr::AggregateUDF(expr::AggregateUDF{ fun, args, filter, order_by }) =>{ + let sorts = if let Some(order_by) = order_by { + order_by.iter().map(|expr| to_substrait_sort_field(expr, schema, extension_info)).collect::>>()? + } else { + vec![] + }; + let mut arguments: Vec = vec![]; + for arg in args { + arguments.push(FunctionArgument { arg_type: Some(ArgType::Value(to_substrait_rex(arg, schema, 0, extension_info)?)) }); + } + let function_anchor = _register_function(fun.name.clone(), extension_info); + Ok(Measure { + measure: Some(AggregateFunction { + function_reference: function_anchor, + arguments, + sorts, + output_type: None, + invocation: AggregationInvocation::All as i32, + phase: AggregationPhase::Unspecified as i32, + args: vec![], + options: vec![], + }), + filter: match filter { + Some(f) => Some(to_substrait_rex(f, schema, 0, extension_info)?), + None => None + } + }) + }, Expr::Alias(Alias{expr,..})=> { to_substrait_agg_measure(expr, schema, extension_info) } @@ -703,8 +730,8 @@ pub fn make_binary_op_scalar_func( HashMap, ), ) -> Expression { - let function_name = operator_to_name(op).to_string().to_lowercase(); - let function_anchor = _register_function(function_name, extension_info); + let function_anchor = + _register_function(operator_to_name(op).to_string(), extension_info); Expression { rex_type: Some(RexType::ScalarFunction(ScalarFunction { function_reference: function_anchor, @@ -807,8 +834,7 @@ pub fn to_substrait_rex( )?)), }); } - let function_name = fun.to_string().to_lowercase(); - let function_anchor = _register_function(function_name, extension_info); + let function_anchor = _register_function(fun.to_string(), extension_info); Ok(Expression { rex_type: Some(RexType::ScalarFunction(ScalarFunction { function_reference: function_anchor, @@ -973,8 +999,7 @@ pub fn to_substrait_rex( window_frame, }) => { // function reference - let function_name = fun.to_string().to_lowercase(); - let function_anchor = _register_function(function_name, extension_info); + let function_anchor = _register_function(fun.to_string(), extension_info); // arguments let mut arguments: Vec = vec![]; for arg in args { diff --git a/datafusion/substrait/tests/cases/roundtrip_logical_plan.rs b/datafusion/substrait/tests/cases/roundtrip_logical_plan.rs index 582e5a5d7c8e..cee3a346495b 100644 --- a/datafusion/substrait/tests/cases/roundtrip_logical_plan.rs +++ b/datafusion/substrait/tests/cases/roundtrip_logical_plan.rs @@ -15,6 +15,9 @@ // specific language governing permissions and limitations // under the License. +use datafusion::arrow::array::ArrayRef; +use datafusion::physical_plan::Accumulator; +use datafusion::scalar::ScalarValue; use datafusion_substrait::logical_plan::{ consumer::from_substrait_plan, producer::to_substrait_plan, }; @@ -28,7 +31,9 @@ use datafusion::error::{DataFusionError, Result}; use datafusion::execution::context::SessionState; use datafusion::execution::registry::SerializerRegistry; use datafusion::execution::runtime_env::RuntimeEnv; -use datafusion::logical_expr::{Extension, LogicalPlan, UserDefinedLogicalNode}; +use datafusion::logical_expr::{ + Extension, LogicalPlan, UserDefinedLogicalNode, Volatility, +}; use datafusion::optimizer::simplify_expressions::expr_simplifier::THRESHOLD_INLINE_INLIST; use datafusion::prelude::*; @@ -636,6 +641,56 @@ async fn extension_logical_plan() -> Result<()> { Ok(()) } +#[tokio::test] +async fn roundtrip_aggregate_udf() -> Result<()> { + #[derive(Debug)] + struct Dummy {} + + impl Accumulator for Dummy { + fn state(&self) -> datafusion::error::Result> { + Ok(vec![]) + } + + fn update_batch( + &mut self, + _values: &[ArrayRef], + ) -> datafusion::error::Result<()> { + Ok(()) + } + + fn merge_batch(&mut self, _states: &[ArrayRef]) -> datafusion::error::Result<()> { + Ok(()) + } + + fn evaluate(&self) -> datafusion::error::Result { + Ok(ScalarValue::Float64(None)) + } + + fn size(&self) -> usize { + std::mem::size_of_val(self) + } + } + + let dummy_agg = create_udaf( + // the name; used to represent it in plan descriptions and in the registry, to use in SQL. + "dummy_agg", + // the input type; DataFusion guarantees that the first entry of `values` in `update` has this type. + vec![DataType::Int64], + // the return type; DataFusion expects this to match the type returned by `evaluate`. + Arc::new(DataType::Int64), + Volatility::Immutable, + // This is the accumulator factory; DataFusion uses it to create new accumulators. + Arc::new(|_| Ok(Box::new(Dummy {}))), + // This is the description of the state. `state()` must match the types here. + Arc::new(vec![DataType::Float64, DataType::UInt32]), + ); + + let ctx = create_context().await?; + ctx.register_udaf(dummy_agg); + + roundtrip_with_ctx("select dummy_agg(a) from data", ctx).await +} + fn check_post_join_filters(rel: &Rel) -> Result<()> { // search for target_rel and field value in proto match &rel.rel_type { @@ -772,8 +827,7 @@ async fn test_alias(sql_with_alias: &str, sql_no_alias: &str) -> Result<()> { Ok(()) } -async fn roundtrip(sql: &str) -> Result<()> { - let ctx = create_context().await?; +async fn roundtrip_with_ctx(sql: &str, ctx: SessionContext) -> Result<()> { let df = ctx.sql(sql).await?; let plan = df.into_optimized_plan()?; let proto = to_substrait_plan(&plan, &ctx)?; @@ -789,6 +843,10 @@ async fn roundtrip(sql: &str) -> Result<()> { Ok(()) } +async fn roundtrip(sql: &str) -> Result<()> { + roundtrip_with_ctx(sql, create_context().await?).await +} + async fn roundtrip_verify_post_join_filter(sql: &str) -> Result<()> { let ctx = create_context().await?; let df = ctx.sql(sql).await?; From f33244a646f97f0355eab3195d4b89eff3b602c2 Mon Sep 17 00:00:00 2001 From: Andrew Lamb Date: Sun, 12 Nov 2023 11:12:41 -0500 Subject: [PATCH 238/572] Improve documentation site to make it easier to find communication on Slack/Discord (#8138) --- README.md | 17 ++++--- .../source/contributor-guide/communication.md | 50 ++++++------------- docs/source/index.rst | 3 +- 3 files changed, 28 insertions(+), 42 deletions(-) diff --git a/README.md b/README.md index 1997a6f73dd5..f5ee1d6d806f 100644 --- a/README.md +++ b/README.md @@ -40,8 +40,19 @@ Here are links to some important information DataFusion is great for building projects such as domain specific query engines, new database platforms and data pipelines, query languages and more. It lets you start quickly from a fully working engine, and then customize those features specific to your use. [Click Here](https://arrow.apache.org/datafusion/user-guide/introduction.html#known-users) to see a list known users. +## Contributing to DataFusion + +Please see the [developer’s guide] for contributing and [communication] for getting in touch with us. + +[developer’s guide]: https://arrow.apache.org/datafusion/contributor-guide/index.html#developer-s-guide +[communication]: https://arrow.apache.org/datafusion/contributor-guide/communication.html + ## Crate features +This crate has several [features] which can be specified in your `Cargo.toml`. + +[features]: https://doc.rust-lang.org/cargo/reference/features.html + Default features: - `compression`: reading files compressed with `xz2`, `bzip2`, `flate2`, and `zstd` @@ -65,9 +76,3 @@ Optional features: ## Rust Version Compatibility This crate is tested with the latest stable version of Rust. We do not currently test against other, older versions of the Rust compiler. - -## Contributing to DataFusion - -The [developer’s guide] contains information on how to contribute. - -[developer’s guide]: https://arrow.apache.org/datafusion/contributor-guide/index.html#developer-s-guide diff --git a/docs/source/contributor-guide/communication.md b/docs/source/contributor-guide/communication.md index 11e0e4e0f0ea..8678aa534baf 100644 --- a/docs/source/contributor-guide/communication.md +++ b/docs/source/contributor-guide/communication.md @@ -26,15 +26,25 @@ All participation in the Apache Arrow DataFusion project is governed by the Apache Software Foundation's [code of conduct](https://www.apache.org/foundation/policies/conduct.html). +## GitHub + The vast majority of communication occurs in the open on our -[github repository](https://github.com/apache/arrow-datafusion). +[github repository](https://github.com/apache/arrow-datafusion) in the form of tickets, issues, discussions, and Pull Requests. + +## Slack and Discord -## Questions? +We use the Slack and Discord platforms for informal discussions and coordination. These are great places to +meet other contributors and get guidance on where to contribute. It is important to note that any technical designs and +decisions are made fully in the open, on GitHub. -### Mailing list +Most of us use the `#arrow-datafusion` and `#arrow-rust` channels in the [ASF Slack workspace](https://s.apache.org/slack-invite) . +Unfortunately, due to spammers, the ASF Slack workspace requires an invitation to join. To get an invitation, +request one in the `Arrow Rust` channel of the [Arrow Rust Discord server](https://discord.gg/Qw5gKqHxUM). -We use arrow.apache.org's `dev@` mailing list for project management, release -coordination and design discussions +## Mailing list + +We also use arrow.apache.org's `dev@` mailing list for release coordination and occasional design discussions. Other +than the the release process, most DataFusion mailing list traffic will link to a GitHub issue or PR for discussion. ([subscribe](mailto:dev-subscribe@arrow.apache.org), [unsubscribe](mailto:dev-unsubscribe@arrow.apache.org), [archives](https://lists.apache.org/list.html?dev@arrow.apache.org)). @@ -42,33 +52,3 @@ coordination and design discussions When emailing the dev list, please make sure to prefix the subject line with a `[DataFusion]` tag, e.g. `"[DataFusion] New API for remote data sources"`, so that the appropriate people in the Apache Arrow community notice the message. - -### Slack and Discord - -We use the official [ASF](https://s.apache.org/slack-invite) Slack workspace -for informal discussions and coordination. This is a great place to meet other -contributors and get guidance on where to contribute. Join us in the -`#arrow-rust` channel. - -We also have a backup Arrow Rust Discord -server ([invite link](https://discord.gg/Qw5gKqHxUM)) in case you are not able -to join the Slack workspace. If you need an invite to the Slack workspace, you -can also ask for one in our Discord server. - -### Sync up video calls - -We have biweekly sync calls every other Thursdays at both 04:00 UTC -and 16:00 UTC (starting September 30, 2021) depending on if there are -items on the agenda to discuss and someone being willing to host. - -Please see the [agenda](https://docs.google.com/document/d/1atCVnoff5SR4eM4Lwf2M1BBJTY6g3_HUNR6qswYJW_U/edit) -for the video call link, add topics and to see what others plan to discuss. - -The goals of these calls are: - -1. Help "put a face to the name" of some of other contributors we are working with -2. Discuss / synchronize on the goals and major initiatives from different stakeholders to identify areas where more alignment is needed - -No decisions are made on the call and anything of substance will be discussed on the mailing list or in github issues / google docs. - -We will send a summary of all sync ups to the dev@arrow.apache.org mailing list. diff --git a/docs/source/index.rst b/docs/source/index.rst index bb8e2127f1e7..385371661716 100644 --- a/docs/source/index.rst +++ b/docs/source/index.rst @@ -43,11 +43,12 @@ community. The `example usage`_ section in the user guide and the `datafusion-examples`_ code in the crate contain information on using DataFusion. -The `developer’s guide`_ contains information on how to contribute. +Please see the `developer’s guide`_ for contributing and `communication`_ for getting in touch with us. .. _example usage: user-guide/example-usage.html .. _datafusion-examples: https://github.com/apache/arrow-datafusion/tree/master/datafusion-examples .. _developer’s guide: contributor-guide/index.html#developer-s-guide +.. _communication: contributor-guide/communication.html .. _toc.links: .. toctree:: From 7889bf9c4f171b6319ef57b5d17ca8aeea64fa68 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E5=BC=A0=E6=9E=97=E4=BC=9F?= Date: Mon, 13 Nov 2023 16:54:24 +0800 Subject: [PATCH 239/572] Fix typo in partitioning.rs (#8134) * Fix typo in partitioning.rs * Update partitioning.rs --- datafusion/physical-expr/src/partitioning.rs | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/datafusion/physical-expr/src/partitioning.rs b/datafusion/physical-expr/src/partitioning.rs index cbacb7a8a906..301f12e9aa2e 100644 --- a/datafusion/physical-expr/src/partitioning.rs +++ b/datafusion/physical-expr/src/partitioning.rs @@ -26,7 +26,7 @@ use crate::{physical_exprs_equal, EquivalenceProperties, PhysicalExpr}; /// /// When `executed`, `ExecutionPlan`s produce one or more independent stream of /// data batches in parallel, referred to as partitions. The streams are Rust -/// `aync` [`Stream`]s (a special kind of future). The number of output +/// `async` [`Stream`]s (a special kind of future). The number of output /// partitions varies based on the input and the operation performed. /// /// For example, an `ExecutionPlan` that has output partitioning of 3 will @@ -64,7 +64,7 @@ use crate::{physical_exprs_equal, EquivalenceProperties, PhysicalExpr}; /// ``` /// /// It is common (but not required) that an `ExecutionPlan` has the same number -/// of input partitions as output partitons. However, some plans have different +/// of input partitions as output partitions. However, some plans have different /// numbers such as the `RepartitionExec` that redistributes batches from some /// number of inputs to some number of outputs /// From 2185842be22b695cf00e615db68b373f86fd162b Mon Sep 17 00:00:00 2001 From: Marko Grujic Date: Mon, 13 Nov 2023 15:45:54 +0100 Subject: [PATCH 240/572] Implement `DISTINCT ON` from Postgres (#7981) * Initial DISTINT ON implementation * Add a couple more tests * Add comments in the replace_distinct_aggregate optimizer * Run cargo fmt to fix CI * Make DISTINCT ON planning more robust to support arbitrary selection expressions * Add DISTINCT ON + join SLT * Handle no DISTINCT ON expressions and extend the docs for the replace_distinct_aggregate optimizer * Remove misleading DISTINCT ON SLT comment * Add an EXPLAIN SLT for a basic DISTINCT ON query * Revise comment in CommonSubexprEliminate::try_optimize_aggregate * Implement qualified expression alias and extend test coverage * Update datafusion/proto/proto/datafusion.proto Co-authored-by: Jonah Gao * Accompanying generated changes to alias proto tag revision * Remove obsolete comment --------- Co-authored-by: Jonah Gao --- datafusion/expr/src/expr.rs | 33 +++- datafusion/expr/src/expr_schema.rs | 7 + datafusion/expr/src/logical_plan/builder.rs | 29 ++- datafusion/expr/src/logical_plan/mod.rs | 8 +- datafusion/expr/src/logical_plan/plan.rs | 163 ++++++++++++++-- datafusion/expr/src/tree_node/expr.rs | 8 +- datafusion/expr/src/utils.rs | 8 +- .../optimizer/src/common_subexpr_eliminate.rs | 23 ++- .../optimizer/src/eliminate_nested_union.rs | 12 +- datafusion/optimizer/src/optimizer.rs | 6 +- .../optimizer/src/push_down_projection.rs | 2 +- .../src/replace_distinct_aggregate.rs | 106 ++++++++++- datafusion/optimizer/src/test/mod.rs | 7 +- datafusion/proto/proto/datafusion.proto | 9 + datafusion/proto/src/generated/pbjson.rs | 176 ++++++++++++++++++ datafusion/proto/src/generated/prost.rs | 18 +- .../proto/src/logical_plan/from_proto.rs | 5 + datafusion/proto/src/logical_plan/mod.rs | 67 ++++++- datafusion/proto/src/logical_plan/to_proto.rs | 10 +- .../tests/cases/roundtrip_logical_plan.rs | 26 +++ datafusion/proto/tests/cases/serialize.rs | 6 + datafusion/sql/src/query.rs | 12 +- datafusion/sql/src/select.rs | 83 +++++---- .../sqllogictest/test_files/distinct_on.slt | 146 +++++++++++++++ .../substrait/src/logical_plan/producer.rs | 8 +- 25 files changed, 879 insertions(+), 99 deletions(-) create mode 100644 datafusion/sqllogictest/test_files/distinct_on.slt diff --git a/datafusion/expr/src/expr.rs b/datafusion/expr/src/expr.rs index 4267f182bda8..97e4fcc327c3 100644 --- a/datafusion/expr/src/expr.rs +++ b/datafusion/expr/src/expr.rs @@ -28,7 +28,7 @@ use crate::Operator; use crate::{aggregate_function, ExprSchemable}; use arrow::datatypes::DataType; use datafusion_common::tree_node::{Transformed, TreeNode}; -use datafusion_common::{internal_err, DFSchema}; +use datafusion_common::{internal_err, DFSchema, OwnedTableReference}; use datafusion_common::{plan_err, Column, DataFusionError, Result, ScalarValue}; use std::collections::HashSet; use std::fmt; @@ -187,13 +187,20 @@ pub enum Expr { #[derive(Clone, PartialEq, Eq, Hash, Debug)] pub struct Alias { pub expr: Box, + pub relation: Option, pub name: String, } impl Alias { - pub fn new(expr: Expr, name: impl Into) -> Self { + /// Create an alias with an optional schema/field qualifier. + pub fn new( + expr: Expr, + relation: Option>, + name: impl Into, + ) -> Self { Self { expr: Box::new(expr), + relation: relation.map(|r| r.into()), name: name.into(), } } @@ -844,7 +851,27 @@ impl Expr { asc, nulls_first, }) => Expr::Sort(Sort::new(Box::new(expr.alias(name)), asc, nulls_first)), - _ => Expr::Alias(Alias::new(self, name.into())), + _ => Expr::Alias(Alias::new(self, None::<&str>, name.into())), + } + } + + /// Return `self AS name` alias expression with a specific qualifier + pub fn alias_qualified( + self, + relation: Option>, + name: impl Into, + ) -> Expr { + match self { + Expr::Sort(Sort { + expr, + asc, + nulls_first, + }) => Expr::Sort(Sort::new( + Box::new(expr.alias_qualified(relation, name)), + asc, + nulls_first, + )), + _ => Expr::Alias(Alias::new(self, relation, name.into())), } } diff --git a/datafusion/expr/src/expr_schema.rs b/datafusion/expr/src/expr_schema.rs index 2631708fb780..5881feece1fc 100644 --- a/datafusion/expr/src/expr_schema.rs +++ b/datafusion/expr/src/expr_schema.rs @@ -305,6 +305,13 @@ impl ExprSchemable for Expr { self.nullable(input_schema)?, ) .with_metadata(self.metadata(input_schema)?)), + Expr::Alias(Alias { relation, name, .. }) => Ok(DFField::new( + relation.clone(), + name, + self.get_type(input_schema)?, + self.nullable(input_schema)?, + ) + .with_metadata(self.metadata(input_schema)?)), _ => Ok(DFField::new_unqualified( &self.display_name()?, self.get_type(input_schema)?, diff --git a/datafusion/expr/src/logical_plan/builder.rs b/datafusion/expr/src/logical_plan/builder.rs index 4a30f4e223bf..c4ff9fe95435 100644 --- a/datafusion/expr/src/logical_plan/builder.rs +++ b/datafusion/expr/src/logical_plan/builder.rs @@ -32,8 +32,8 @@ use crate::expr_rewriter::{ rewrite_sort_cols_by_aggs, }; use crate::logical_plan::{ - Aggregate, Analyze, CrossJoin, Distinct, EmptyRelation, Explain, Filter, Join, - JoinConstraint, JoinType, Limit, LogicalPlan, Partitioning, PlanType, Prepare, + Aggregate, Analyze, CrossJoin, Distinct, DistinctOn, EmptyRelation, Explain, Filter, + Join, JoinConstraint, JoinType, Limit, LogicalPlan, Partitioning, PlanType, Prepare, Projection, Repartition, Sort, SubqueryAlias, TableScan, Union, Unnest, Values, Window, }; @@ -551,16 +551,29 @@ impl LogicalPlanBuilder { let left_plan: LogicalPlan = self.plan; let right_plan: LogicalPlan = plan; - Ok(Self::from(LogicalPlan::Distinct(Distinct { - input: Arc::new(union(left_plan, right_plan)?), - }))) + Ok(Self::from(LogicalPlan::Distinct(Distinct::All(Arc::new( + union(left_plan, right_plan)?, + ))))) } /// Apply deduplication: Only distinct (different) values are returned) pub fn distinct(self) -> Result { - Ok(Self::from(LogicalPlan::Distinct(Distinct { - input: Arc::new(self.plan), - }))) + Ok(Self::from(LogicalPlan::Distinct(Distinct::All(Arc::new( + self.plan, + ))))) + } + + /// Project first values of the specified expression list according to the provided + /// sorting expressions grouped by the `DISTINCT ON` clause expressions. + pub fn distinct_on( + self, + on_expr: Vec, + select_expr: Vec, + sort_expr: Option>, + ) -> Result { + Ok(Self::from(LogicalPlan::Distinct(Distinct::On( + DistinctOn::try_new(on_expr, select_expr, sort_expr, Arc::new(self.plan))?, + )))) } /// Apply a join to `right` using explicitly specified columns and an diff --git a/datafusion/expr/src/logical_plan/mod.rs b/datafusion/expr/src/logical_plan/mod.rs index 8316417138bd..51d78cd721b6 100644 --- a/datafusion/expr/src/logical_plan/mod.rs +++ b/datafusion/expr/src/logical_plan/mod.rs @@ -33,10 +33,10 @@ pub use ddl::{ }; pub use dml::{DmlStatement, WriteOp}; pub use plan::{ - Aggregate, Analyze, CrossJoin, DescribeTable, Distinct, EmptyRelation, Explain, - Extension, Filter, Join, JoinConstraint, JoinType, Limit, LogicalPlan, Partitioning, - PlanType, Prepare, Projection, Repartition, Sort, StringifiedPlan, Subquery, - SubqueryAlias, TableScan, ToStringifiedPlan, Union, Unnest, Values, Window, + Aggregate, Analyze, CrossJoin, DescribeTable, Distinct, DistinctOn, EmptyRelation, + Explain, Extension, Filter, Join, JoinConstraint, JoinType, Limit, LogicalPlan, + Partitioning, PlanType, Prepare, Projection, Repartition, Sort, StringifiedPlan, + Subquery, SubqueryAlias, TableScan, ToStringifiedPlan, Union, Unnest, Values, Window, }; pub use statement::{ SetVariable, Statement, TransactionAccessMode, TransactionConclusion, TransactionEnd, diff --git a/datafusion/expr/src/logical_plan/plan.rs b/datafusion/expr/src/logical_plan/plan.rs index d62ac8926328..b7537dc02e9d 100644 --- a/datafusion/expr/src/logical_plan/plan.rs +++ b/datafusion/expr/src/logical_plan/plan.rs @@ -25,8 +25,8 @@ use std::sync::Arc; use super::dml::CopyTo; use super::DdlStatement; use crate::dml::CopyOptions; -use crate::expr::{Alias, Exists, InSubquery, Placeholder}; -use crate::expr_rewriter::create_col_from_scalar_expr; +use crate::expr::{Alias, Exists, InSubquery, Placeholder, Sort as SortExpr}; +use crate::expr_rewriter::{create_col_from_scalar_expr, normalize_cols}; use crate::logical_plan::display::{GraphvizVisitor, IndentVisitor}; use crate::logical_plan::extension::UserDefinedLogicalNode; use crate::logical_plan::{DmlStatement, Statement}; @@ -163,7 +163,8 @@ impl LogicalPlan { }) => projected_schema, LogicalPlan::Projection(Projection { schema, .. }) => schema, LogicalPlan::Filter(Filter { input, .. }) => input.schema(), - LogicalPlan::Distinct(Distinct { input }) => input.schema(), + LogicalPlan::Distinct(Distinct::All(input)) => input.schema(), + LogicalPlan::Distinct(Distinct::On(DistinctOn { schema, .. })) => schema, LogicalPlan::Window(Window { schema, .. }) => schema, LogicalPlan::Aggregate(Aggregate { schema, .. }) => schema, LogicalPlan::Sort(Sort { input, .. }) => input.schema(), @@ -367,6 +368,16 @@ impl LogicalPlan { LogicalPlan::Unnest(Unnest { column, .. }) => { f(&Expr::Column(column.clone())) } + LogicalPlan::Distinct(Distinct::On(DistinctOn { + on_expr, + select_expr, + sort_expr, + .. + })) => on_expr + .iter() + .chain(select_expr.iter()) + .chain(sort_expr.clone().unwrap_or(vec![]).iter()) + .try_for_each(f), // plans without expressions LogicalPlan::EmptyRelation(_) | LogicalPlan::Subquery(_) @@ -377,7 +388,7 @@ impl LogicalPlan { | LogicalPlan::Analyze(_) | LogicalPlan::Explain(_) | LogicalPlan::Union(_) - | LogicalPlan::Distinct(_) + | LogicalPlan::Distinct(Distinct::All(_)) | LogicalPlan::Dml(_) | LogicalPlan::Ddl(_) | LogicalPlan::Copy(_) @@ -405,7 +416,9 @@ impl LogicalPlan { LogicalPlan::Union(Union { inputs, .. }) => { inputs.iter().map(|arc| arc.as_ref()).collect() } - LogicalPlan::Distinct(Distinct { input }) => vec![input], + LogicalPlan::Distinct( + Distinct::All(input) | Distinct::On(DistinctOn { input, .. }), + ) => vec![input], LogicalPlan::Explain(explain) => vec![&explain.plan], LogicalPlan::Analyze(analyze) => vec![&analyze.input], LogicalPlan::Dml(write) => vec![&write.input], @@ -461,8 +474,11 @@ impl LogicalPlan { Ok(Some(agg.group_expr.as_slice()[0].clone())) } } + LogicalPlan::Distinct(Distinct::On(DistinctOn { select_expr, .. })) => { + Ok(Some(select_expr[0].clone())) + } LogicalPlan::Filter(Filter { input, .. }) - | LogicalPlan::Distinct(Distinct { input, .. }) + | LogicalPlan::Distinct(Distinct::All(input)) | LogicalPlan::Sort(Sort { input, .. }) | LogicalPlan::Limit(Limit { input, .. }) | LogicalPlan::Repartition(Repartition { input, .. }) @@ -823,10 +839,29 @@ impl LogicalPlan { inputs: inputs.iter().cloned().map(Arc::new).collect(), schema: schema.clone(), })), - LogicalPlan::Distinct(Distinct { .. }) => { - Ok(LogicalPlan::Distinct(Distinct { - input: Arc::new(inputs[0].clone()), - })) + LogicalPlan::Distinct(distinct) => { + let distinct = match distinct { + Distinct::All(_) => Distinct::All(Arc::new(inputs[0].clone())), + Distinct::On(DistinctOn { + on_expr, + select_expr, + .. + }) => { + let sort_expr = expr.split_off(on_expr.len() + select_expr.len()); + let select_expr = expr.split_off(on_expr.len()); + Distinct::On(DistinctOn::try_new( + expr, + select_expr, + if !sort_expr.is_empty() { + Some(sort_expr) + } else { + None + }, + Arc::new(inputs[0].clone()), + )?) + } + }; + Ok(LogicalPlan::Distinct(distinct)) } LogicalPlan::Analyze(a) => { assert!(expr.is_empty()); @@ -1064,7 +1099,9 @@ impl LogicalPlan { LogicalPlan::Subquery(_) => None, LogicalPlan::SubqueryAlias(SubqueryAlias { input, .. }) => input.max_rows(), LogicalPlan::Limit(Limit { fetch, .. }) => *fetch, - LogicalPlan::Distinct(Distinct { input }) => input.max_rows(), + LogicalPlan::Distinct( + Distinct::All(input) | Distinct::On(DistinctOn { input, .. }), + ) => input.max_rows(), LogicalPlan::Values(v) => Some(v.values.len()), LogicalPlan::Unnest(_) => None, LogicalPlan::Ddl(_) @@ -1667,9 +1704,21 @@ impl LogicalPlan { LogicalPlan::Statement(statement) => { write!(f, "{}", statement.display()) } - LogicalPlan::Distinct(Distinct { .. }) => { - write!(f, "Distinct:") - } + LogicalPlan::Distinct(distinct) => match distinct { + Distinct::All(_) => write!(f, "Distinct:"), + Distinct::On(DistinctOn { + on_expr, + select_expr, + sort_expr, + .. + }) => write!( + f, + "DistinctOn: on_expr=[[{}]], select_expr=[[{}]], sort_expr=[[{}]]", + expr_vec_fmt!(on_expr), + expr_vec_fmt!(select_expr), + if let Some(sort_expr) = sort_expr { expr_vec_fmt!(sort_expr) } else { "".to_string() }, + ), + }, LogicalPlan::Explain { .. } => write!(f, "Explain"), LogicalPlan::Analyze { .. } => write!(f, "Analyze"), LogicalPlan::Union(_) => write!(f, "Union"), @@ -2132,9 +2181,93 @@ pub struct Limit { /// Removes duplicate rows from the input #[derive(Clone, PartialEq, Eq, Hash)] -pub struct Distinct { +pub enum Distinct { + /// Plain `DISTINCT` referencing all selection expressions + All(Arc), + /// The `Postgres` addition, allowing separate control over DISTINCT'd and selected columns + On(DistinctOn), +} + +/// Removes duplicate rows from the input +#[derive(Clone, PartialEq, Eq, Hash)] +pub struct DistinctOn { + /// The `DISTINCT ON` clause expression list + pub on_expr: Vec, + /// The selected projection expression list + pub select_expr: Vec, + /// The `ORDER BY` clause, whose initial expressions must match those of the `ON` clause when + /// present. Note that those matching expressions actually wrap the `ON` expressions with + /// additional info pertaining to the sorting procedure (i.e. ASC/DESC, and NULLS FIRST/LAST). + pub sort_expr: Option>, /// The logical plan that is being DISTINCT'd pub input: Arc, + /// The schema description of the DISTINCT ON output + pub schema: DFSchemaRef, +} + +impl DistinctOn { + /// Create a new `DistinctOn` struct. + pub fn try_new( + on_expr: Vec, + select_expr: Vec, + sort_expr: Option>, + input: Arc, + ) -> Result { + if on_expr.is_empty() { + return plan_err!("No `ON` expressions provided"); + } + + let on_expr = normalize_cols(on_expr, input.as_ref())?; + + let schema = DFSchema::new_with_metadata( + exprlist_to_fields(&select_expr, &input)?, + input.schema().metadata().clone(), + )?; + + let mut distinct_on = DistinctOn { + on_expr, + select_expr, + sort_expr: None, + input, + schema: Arc::new(schema), + }; + + if let Some(sort_expr) = sort_expr { + distinct_on = distinct_on.with_sort_expr(sort_expr)?; + } + + Ok(distinct_on) + } + + /// Try to update `self` with a new sort expressions. + /// + /// Validates that the sort expressions are a super-set of the `ON` expressions. + pub fn with_sort_expr(mut self, sort_expr: Vec) -> Result { + let sort_expr = normalize_cols(sort_expr, self.input.as_ref())?; + + // Check that the left-most sort expressions are the same as the `ON` expressions. + let mut matched = true; + for (on, sort) in self.on_expr.iter().zip(sort_expr.iter()) { + match sort { + Expr::Sort(SortExpr { expr, .. }) => { + if on != &**expr { + matched = false; + break; + } + } + _ => return plan_err!("Not a sort expression: {sort}"), + } + } + + if self.on_expr.len() > sort_expr.len() || !matched { + return plan_err!( + "SELECT DISTINCT ON expressions must match initial ORDER BY expressions" + ); + } + + self.sort_expr = Some(sort_expr); + Ok(self) + } } /// Aggregates its input based on a set of grouping and aggregate diff --git a/datafusion/expr/src/tree_node/expr.rs b/datafusion/expr/src/tree_node/expr.rs index d6c14b86227a..6b86de37ba44 100644 --- a/datafusion/expr/src/tree_node/expr.rs +++ b/datafusion/expr/src/tree_node/expr.rs @@ -157,9 +157,11 @@ impl TreeNode for Expr { let mut transform = transform; Ok(match self { - Expr::Alias(Alias { expr, name, .. }) => { - Expr::Alias(Alias::new(transform(*expr)?, name)) - } + Expr::Alias(Alias { + expr, + relation, + name, + }) => Expr::Alias(Alias::new(transform(*expr)?, relation, name)), Expr::Column(_) => self, Expr::OuterReferenceColumn(_, _) => self, Expr::Exists { .. } => self, diff --git a/datafusion/expr/src/utils.rs b/datafusion/expr/src/utils.rs index a462cdb34631..8f13bf5f61be 100644 --- a/datafusion/expr/src/utils.rs +++ b/datafusion/expr/src/utils.rs @@ -800,9 +800,11 @@ pub fn columnize_expr(e: Expr, input_schema: &DFSchema) -> Expr { match e { Expr::Column(_) => e, Expr::OuterReferenceColumn(_, _) => e, - Expr::Alias(Alias { expr, name, .. }) => { - columnize_expr(*expr, input_schema).alias(name) - } + Expr::Alias(Alias { + expr, + relation, + name, + }) => columnize_expr(*expr, input_schema).alias_qualified(relation, name), Expr::Cast(Cast { expr, data_type }) => Expr::Cast(Cast { expr: Box::new(columnize_expr(*expr, input_schema)), data_type, diff --git a/datafusion/optimizer/src/common_subexpr_eliminate.rs b/datafusion/optimizer/src/common_subexpr_eliminate.rs index 8025402ccef5..f5ad767c5016 100644 --- a/datafusion/optimizer/src/common_subexpr_eliminate.rs +++ b/datafusion/optimizer/src/common_subexpr_eliminate.rs @@ -238,6 +238,14 @@ impl CommonSubexprEliminate { let rewritten = pop_expr(&mut rewritten)?; if affected_id.is_empty() { + // Alias aggregation expressions if they have changed + let new_aggr_expr = new_aggr_expr + .iter() + .zip(aggr_expr.iter()) + .map(|(new_expr, old_expr)| { + new_expr.clone().alias_if_changed(old_expr.display_name()?) + }) + .collect::>>()?; // Since group_epxr changes, schema changes also. Use try_new method. Aggregate::try_new(Arc::new(new_input), new_group_expr, new_aggr_expr) .map(LogicalPlan::Aggregate) @@ -367,7 +375,7 @@ impl OptimizerRule for CommonSubexprEliminate { Ok(Some(build_recover_project_plan( &original_schema, optimized_plan, - ))) + )?)) } plan => Ok(plan), } @@ -458,16 +466,19 @@ fn build_common_expr_project_plan( /// the "intermediate" projection plan built in [build_common_expr_project_plan]. /// /// This is for those plans who don't keep its own output schema like `Filter` or `Sort`. -fn build_recover_project_plan(schema: &DFSchema, input: LogicalPlan) -> LogicalPlan { +fn build_recover_project_plan( + schema: &DFSchema, + input: LogicalPlan, +) -> Result { let col_exprs = schema .fields() .iter() .map(|field| Expr::Column(field.qualified_column())) .collect(); - LogicalPlan::Projection( - Projection::try_new(col_exprs, Arc::new(input)) - .expect("Cannot build projection plan from an invalid schema"), - ) + Ok(LogicalPlan::Projection(Projection::try_new( + col_exprs, + Arc::new(input), + )?)) } fn extract_expressions( diff --git a/datafusion/optimizer/src/eliminate_nested_union.rs b/datafusion/optimizer/src/eliminate_nested_union.rs index 89bcc90bc075..5771ea2e19a2 100644 --- a/datafusion/optimizer/src/eliminate_nested_union.rs +++ b/datafusion/optimizer/src/eliminate_nested_union.rs @@ -52,7 +52,7 @@ impl OptimizerRule for EliminateNestedUnion { schema: schema.clone(), }))) } - LogicalPlan::Distinct(Distinct { input: plan }) => match plan.as_ref() { + LogicalPlan::Distinct(Distinct::All(plan)) => match plan.as_ref() { LogicalPlan::Union(Union { inputs, schema }) => { let inputs = inputs .iter() @@ -60,12 +60,12 @@ impl OptimizerRule for EliminateNestedUnion { .flat_map(extract_plans_from_union) .collect::>(); - Ok(Some(LogicalPlan::Distinct(Distinct { - input: Arc::new(LogicalPlan::Union(Union { + Ok(Some(LogicalPlan::Distinct(Distinct::All(Arc::new( + LogicalPlan::Union(Union { inputs, schema: schema.clone(), - })), - }))) + }), + ))))) } _ => Ok(None), }, @@ -94,7 +94,7 @@ fn extract_plans_from_union(plan: &Arc) -> Vec> { fn extract_plan_from_distinct(plan: &Arc) -> &Arc { match plan.as_ref() { - LogicalPlan::Distinct(Distinct { input: plan }) => plan, + LogicalPlan::Distinct(Distinct::All(plan)) => plan, _ => plan, } } diff --git a/datafusion/optimizer/src/optimizer.rs b/datafusion/optimizer/src/optimizer.rs index 5231dc869875..e93565fef0a0 100644 --- a/datafusion/optimizer/src/optimizer.rs +++ b/datafusion/optimizer/src/optimizer.rs @@ -427,7 +427,7 @@ impl Optimizer { /// Returns an error if plans have different schemas. /// /// It ignores metadata and nullability. -fn assert_schema_is_the_same( +pub(crate) fn assert_schema_is_the_same( rule_name: &str, prev_plan: &LogicalPlan, new_plan: &LogicalPlan, @@ -438,7 +438,7 @@ fn assert_schema_is_the_same( if !equivalent { let e = DataFusionError::Internal(format!( - "Failed due to generate a different schema, original schema: {:?}, new schema: {:?}", + "Failed due to a difference in schemas, original schema: {:?}, new schema: {:?}", prev_plan.schema(), new_plan.schema() )); @@ -503,7 +503,7 @@ mod tests { let err = opt.optimize(&plan, &config, &observe).unwrap_err(); assert_eq!( "Optimizer rule 'get table_scan rule' failed\ncaused by\nget table_scan rule\ncaused by\n\ - Internal error: Failed due to generate a different schema, \ + Internal error: Failed due to a difference in schemas, \ original schema: DFSchema { fields: [], metadata: {}, functional_dependencies: FunctionalDependencies { deps: [] } }, \ new schema: DFSchema { fields: [\ DFField { qualifier: Some(Bare { table: \"test\" }), field: Field { name: \"a\", data_type: UInt32, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }, \ diff --git a/datafusion/optimizer/src/push_down_projection.rs b/datafusion/optimizer/src/push_down_projection.rs index b05d811cb481..2c314bf7651c 100644 --- a/datafusion/optimizer/src/push_down_projection.rs +++ b/datafusion/optimizer/src/push_down_projection.rs @@ -228,7 +228,7 @@ impl OptimizerRule for PushDownProjection { // Gather all columns needed for expressions in this Aggregate let mut new_aggr_expr = vec![]; for e in agg.aggr_expr.iter() { - let column = Column::from_name(e.display_name()?); + let column = Column::from(e.display_name()?); if required_columns.contains(&column) { new_aggr_expr.push(e.clone()); } diff --git a/datafusion/optimizer/src/replace_distinct_aggregate.rs b/datafusion/optimizer/src/replace_distinct_aggregate.rs index 540617b77084..187e510e557d 100644 --- a/datafusion/optimizer/src/replace_distinct_aggregate.rs +++ b/datafusion/optimizer/src/replace_distinct_aggregate.rs @@ -20,7 +20,11 @@ use crate::{OptimizerConfig, OptimizerRule}; use datafusion_common::Result; use datafusion_expr::utils::expand_wildcard; -use datafusion_expr::{Aggregate, Distinct, LogicalPlan}; +use datafusion_expr::{ + aggregate_function::AggregateFunction as AggregateFunctionFunc, col, + expr::AggregateFunction, LogicalPlanBuilder, +}; +use datafusion_expr::{Aggregate, Distinct, DistinctOn, Expr, LogicalPlan}; /// Optimizer that replaces logical [[Distinct]] with a logical [[Aggregate]] /// @@ -32,6 +36,22 @@ use datafusion_expr::{Aggregate, Distinct, LogicalPlan}; /// ```text /// SELECT a, b FROM tab GROUP BY a, b /// ``` +/// +/// On the other hand, for a `DISTINCT ON` query the replacement is +/// a bit more involved and effectively converts +/// ```text +/// SELECT DISTINCT ON (a) b FROM tab ORDER BY a DESC, c +/// ``` +/// +/// into +/// ```text +/// SELECT b FROM ( +/// SELECT a, FIRST_VALUE(b ORDER BY a DESC, c) AS b +/// FROM tab +/// GROUP BY a +/// ) +/// ORDER BY a DESC +/// ``` /// Optimizer that replaces logical [[Distinct]] with a logical [[Aggregate]] #[derive(Default)] @@ -51,7 +71,7 @@ impl OptimizerRule for ReplaceDistinctWithAggregate { _config: &dyn OptimizerConfig, ) -> Result> { match plan { - LogicalPlan::Distinct(Distinct { input }) => { + LogicalPlan::Distinct(Distinct::All(input)) => { let group_expr = expand_wildcard(input.schema(), input, None)?; let aggregate = LogicalPlan::Aggregate(Aggregate::try_new( input.clone(), @@ -60,6 +80,65 @@ impl OptimizerRule for ReplaceDistinctWithAggregate { )?); Ok(Some(aggregate)) } + LogicalPlan::Distinct(Distinct::On(DistinctOn { + select_expr, + on_expr, + sort_expr, + input, + schema, + })) => { + // Construct the aggregation expression to be used to fetch the selected expressions. + let aggr_expr = select_expr + .iter() + .map(|e| { + Expr::AggregateFunction(AggregateFunction::new( + AggregateFunctionFunc::FirstValue, + vec![e.clone()], + false, + None, + sort_expr.clone(), + )) + }) + .collect::>(); + + // Build the aggregation plan + let plan = LogicalPlanBuilder::from(input.as_ref().clone()) + .aggregate(on_expr.clone(), aggr_expr.to_vec())? + .build()?; + + let plan = if let Some(sort_expr) = sort_expr { + // While sort expressions were used in the `FIRST_VALUE` aggregation itself above, + // this on it's own isn't enough to guarantee the proper output order of the grouping + // (`ON`) expression, so we need to sort those as well. + LogicalPlanBuilder::from(plan) + .sort(sort_expr[..on_expr.len()].to_vec())? + .build()? + } else { + plan + }; + + // Whereas the aggregation plan by default outputs both the grouping and the aggregation + // expressions, for `DISTINCT ON` we only need to emit the original selection expressions. + let project_exprs = plan + .schema() + .fields() + .iter() + .skip(on_expr.len()) + .zip(schema.fields().iter()) + .map(|(new_field, old_field)| { + Ok(col(new_field.qualified_column()).alias_qualified( + old_field.qualifier().cloned(), + old_field.name(), + )) + }) + .collect::>>()?; + + let plan = LogicalPlanBuilder::from(plan) + .project(project_exprs)? + .build()?; + + Ok(Some(plan)) + } _ => Ok(None), } } @@ -98,4 +177,27 @@ mod tests { expected, ) } + + #[test] + fn replace_distinct_on() -> datafusion_common::Result<()> { + let table_scan = test_table_scan().unwrap(); + let plan = LogicalPlanBuilder::from(table_scan) + .distinct_on( + vec![col("a")], + vec![col("b")], + Some(vec![col("a").sort(false, true), col("c").sort(true, false)]), + )? + .build()?; + + let expected = "Projection: FIRST_VALUE(test.b) ORDER BY [test.a DESC NULLS FIRST, test.c ASC NULLS LAST] AS b\ + \n Sort: test.a DESC NULLS FIRST\ + \n Aggregate: groupBy=[[test.a]], aggr=[[FIRST_VALUE(test.b) ORDER BY [test.a DESC NULLS FIRST, test.c ASC NULLS LAST]]]\ + \n TableScan: test"; + + assert_optimized_plan_eq( + Arc::new(ReplaceDistinctWithAggregate::new()), + &plan, + expected, + ) + } } diff --git a/datafusion/optimizer/src/test/mod.rs b/datafusion/optimizer/src/test/mod.rs index 3eac2317b849..917ddc565c9e 100644 --- a/datafusion/optimizer/src/test/mod.rs +++ b/datafusion/optimizer/src/test/mod.rs @@ -16,7 +16,7 @@ // under the License. use crate::analyzer::{Analyzer, AnalyzerRule}; -use crate::optimizer::Optimizer; +use crate::optimizer::{assert_schema_is_the_same, Optimizer}; use crate::{OptimizerContext, OptimizerRule}; use arrow::datatypes::{DataType, Field, Schema}; use datafusion_common::config::ConfigOptions; @@ -155,7 +155,7 @@ pub fn assert_optimized_plan_eq( plan: &LogicalPlan, expected: &str, ) -> Result<()> { - let optimizer = Optimizer::with_rules(vec![rule]); + let optimizer = Optimizer::with_rules(vec![rule.clone()]); let optimized_plan = optimizer .optimize_recursively( optimizer.rules.get(0).unwrap(), @@ -163,6 +163,9 @@ pub fn assert_optimized_plan_eq( &OptimizerContext::new(), )? .unwrap_or_else(|| plan.clone()); + + // Ensure schemas always match after an optimization + assert_schema_is_the_same(rule.name(), plan, &optimized_plan)?; let formatted_plan = format!("{optimized_plan:?}"); assert_eq!(formatted_plan, expected); diff --git a/datafusion/proto/proto/datafusion.proto b/datafusion/proto/proto/datafusion.proto index 9dcd55e731bb..62b226e33339 100644 --- a/datafusion/proto/proto/datafusion.proto +++ b/datafusion/proto/proto/datafusion.proto @@ -73,6 +73,7 @@ message LogicalPlanNode { CustomTableScanNode custom_scan = 25; PrepareNode prepare = 26; DropViewNode drop_view = 27; + DistinctOnNode distinct_on = 28; } } @@ -308,6 +309,13 @@ message DistinctNode { LogicalPlanNode input = 1; } +message DistinctOnNode { + repeated LogicalExprNode on_expr = 1; + repeated LogicalExprNode select_expr = 2; + repeated LogicalExprNode sort_expr = 3; + LogicalPlanNode input = 4; +} + message UnionNode { repeated LogicalPlanNode inputs = 1; } @@ -485,6 +493,7 @@ message Not { message AliasNode { LogicalExprNode expr = 1; string alias = 2; + repeated OwnedTableReference relation = 3; } message BinaryExprNode { diff --git a/datafusion/proto/src/generated/pbjson.rs b/datafusion/proto/src/generated/pbjson.rs index 948ad0c4cedb..7602e1a36657 100644 --- a/datafusion/proto/src/generated/pbjson.rs +++ b/datafusion/proto/src/generated/pbjson.rs @@ -967,6 +967,9 @@ impl serde::Serialize for AliasNode { if !self.alias.is_empty() { len += 1; } + if !self.relation.is_empty() { + len += 1; + } let mut struct_ser = serializer.serialize_struct("datafusion.AliasNode", len)?; if let Some(v) = self.expr.as_ref() { struct_ser.serialize_field("expr", v)?; @@ -974,6 +977,9 @@ impl serde::Serialize for AliasNode { if !self.alias.is_empty() { struct_ser.serialize_field("alias", &self.alias)?; } + if !self.relation.is_empty() { + struct_ser.serialize_field("relation", &self.relation)?; + } struct_ser.end() } } @@ -986,12 +992,14 @@ impl<'de> serde::Deserialize<'de> for AliasNode { const FIELDS: &[&str] = &[ "expr", "alias", + "relation", ]; #[allow(clippy::enum_variant_names)] enum GeneratedField { Expr, Alias, + Relation, } impl<'de> serde::Deserialize<'de> for GeneratedField { fn deserialize(deserializer: D) -> std::result::Result @@ -1015,6 +1023,7 @@ impl<'de> serde::Deserialize<'de> for AliasNode { match value { "expr" => Ok(GeneratedField::Expr), "alias" => Ok(GeneratedField::Alias), + "relation" => Ok(GeneratedField::Relation), _ => Err(serde::de::Error::unknown_field(value, FIELDS)), } } @@ -1036,6 +1045,7 @@ impl<'de> serde::Deserialize<'de> for AliasNode { { let mut expr__ = None; let mut alias__ = None; + let mut relation__ = None; while let Some(k) = map_.next_key()? { match k { GeneratedField::Expr => { @@ -1050,11 +1060,18 @@ impl<'de> serde::Deserialize<'de> for AliasNode { } alias__ = Some(map_.next_value()?); } + GeneratedField::Relation => { + if relation__.is_some() { + return Err(serde::de::Error::duplicate_field("relation")); + } + relation__ = Some(map_.next_value()?); + } } } Ok(AliasNode { expr: expr__, alias: alias__.unwrap_or_default(), + relation: relation__.unwrap_or_default(), }) } } @@ -6070,6 +6087,151 @@ impl<'de> serde::Deserialize<'de> for DistinctNode { deserializer.deserialize_struct("datafusion.DistinctNode", FIELDS, GeneratedVisitor) } } +impl serde::Serialize for DistinctOnNode { + #[allow(deprecated)] + fn serialize(&self, serializer: S) -> std::result::Result + where + S: serde::Serializer, + { + use serde::ser::SerializeStruct; + let mut len = 0; + if !self.on_expr.is_empty() { + len += 1; + } + if !self.select_expr.is_empty() { + len += 1; + } + if !self.sort_expr.is_empty() { + len += 1; + } + if self.input.is_some() { + len += 1; + } + let mut struct_ser = serializer.serialize_struct("datafusion.DistinctOnNode", len)?; + if !self.on_expr.is_empty() { + struct_ser.serialize_field("onExpr", &self.on_expr)?; + } + if !self.select_expr.is_empty() { + struct_ser.serialize_field("selectExpr", &self.select_expr)?; + } + if !self.sort_expr.is_empty() { + struct_ser.serialize_field("sortExpr", &self.sort_expr)?; + } + if let Some(v) = self.input.as_ref() { + struct_ser.serialize_field("input", v)?; + } + struct_ser.end() + } +} +impl<'de> serde::Deserialize<'de> for DistinctOnNode { + #[allow(deprecated)] + fn deserialize(deserializer: D) -> std::result::Result + where + D: serde::Deserializer<'de>, + { + const FIELDS: &[&str] = &[ + "on_expr", + "onExpr", + "select_expr", + "selectExpr", + "sort_expr", + "sortExpr", + "input", + ]; + + #[allow(clippy::enum_variant_names)] + enum GeneratedField { + OnExpr, + SelectExpr, + SortExpr, + Input, + } + impl<'de> serde::Deserialize<'de> for GeneratedField { + fn deserialize(deserializer: D) -> std::result::Result + where + D: serde::Deserializer<'de>, + { + struct GeneratedVisitor; + + impl<'de> serde::de::Visitor<'de> for GeneratedVisitor { + type Value = GeneratedField; + + fn expecting(&self, formatter: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + write!(formatter, "expected one of: {:?}", &FIELDS) + } + + #[allow(unused_variables)] + fn visit_str(self, value: &str) -> std::result::Result + where + E: serde::de::Error, + { + match value { + "onExpr" | "on_expr" => Ok(GeneratedField::OnExpr), + "selectExpr" | "select_expr" => Ok(GeneratedField::SelectExpr), + "sortExpr" | "sort_expr" => Ok(GeneratedField::SortExpr), + "input" => Ok(GeneratedField::Input), + _ => Err(serde::de::Error::unknown_field(value, FIELDS)), + } + } + } + deserializer.deserialize_identifier(GeneratedVisitor) + } + } + struct GeneratedVisitor; + impl<'de> serde::de::Visitor<'de> for GeneratedVisitor { + type Value = DistinctOnNode; + + fn expecting(&self, formatter: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + formatter.write_str("struct datafusion.DistinctOnNode") + } + + fn visit_map(self, mut map_: V) -> std::result::Result + where + V: serde::de::MapAccess<'de>, + { + let mut on_expr__ = None; + let mut select_expr__ = None; + let mut sort_expr__ = None; + let mut input__ = None; + while let Some(k) = map_.next_key()? { + match k { + GeneratedField::OnExpr => { + if on_expr__.is_some() { + return Err(serde::de::Error::duplicate_field("onExpr")); + } + on_expr__ = Some(map_.next_value()?); + } + GeneratedField::SelectExpr => { + if select_expr__.is_some() { + return Err(serde::de::Error::duplicate_field("selectExpr")); + } + select_expr__ = Some(map_.next_value()?); + } + GeneratedField::SortExpr => { + if sort_expr__.is_some() { + return Err(serde::de::Error::duplicate_field("sortExpr")); + } + sort_expr__ = Some(map_.next_value()?); + } + GeneratedField::Input => { + if input__.is_some() { + return Err(serde::de::Error::duplicate_field("input")); + } + input__ = map_.next_value()?; + } + } + } + Ok(DistinctOnNode { + on_expr: on_expr__.unwrap_or_default(), + select_expr: select_expr__.unwrap_or_default(), + sort_expr: sort_expr__.unwrap_or_default(), + input: input__, + }) + } + } + deserializer.deserialize_struct("datafusion.DistinctOnNode", FIELDS, GeneratedVisitor) + } +} impl serde::Serialize for DropViewNode { #[allow(deprecated)] fn serialize(&self, serializer: S) -> std::result::Result @@ -13146,6 +13308,9 @@ impl serde::Serialize for LogicalPlanNode { logical_plan_node::LogicalPlanType::DropView(v) => { struct_ser.serialize_field("dropView", v)?; } + logical_plan_node::LogicalPlanType::DistinctOn(v) => { + struct_ser.serialize_field("distinctOn", v)?; + } } } struct_ser.end() @@ -13195,6 +13360,8 @@ impl<'de> serde::Deserialize<'de> for LogicalPlanNode { "prepare", "drop_view", "dropView", + "distinct_on", + "distinctOn", ]; #[allow(clippy::enum_variant_names)] @@ -13225,6 +13392,7 @@ impl<'de> serde::Deserialize<'de> for LogicalPlanNode { CustomScan, Prepare, DropView, + DistinctOn, } impl<'de> serde::Deserialize<'de> for GeneratedField { fn deserialize(deserializer: D) -> std::result::Result @@ -13272,6 +13440,7 @@ impl<'de> serde::Deserialize<'de> for LogicalPlanNode { "customScan" | "custom_scan" => Ok(GeneratedField::CustomScan), "prepare" => Ok(GeneratedField::Prepare), "dropView" | "drop_view" => Ok(GeneratedField::DropView), + "distinctOn" | "distinct_on" => Ok(GeneratedField::DistinctOn), _ => Err(serde::de::Error::unknown_field(value, FIELDS)), } } @@ -13474,6 +13643,13 @@ impl<'de> serde::Deserialize<'de> for LogicalPlanNode { return Err(serde::de::Error::duplicate_field("dropView")); } logical_plan_type__ = map_.next_value::<::std::option::Option<_>>()?.map(logical_plan_node::LogicalPlanType::DropView) +; + } + GeneratedField::DistinctOn => { + if logical_plan_type__.is_some() { + return Err(serde::de::Error::duplicate_field("distinctOn")); + } + logical_plan_type__ = map_.next_value::<::std::option::Option<_>>()?.map(logical_plan_node::LogicalPlanType::DistinctOn) ; } } diff --git a/datafusion/proto/src/generated/prost.rs b/datafusion/proto/src/generated/prost.rs index 93b0a05c314d..825481a18822 100644 --- a/datafusion/proto/src/generated/prost.rs +++ b/datafusion/proto/src/generated/prost.rs @@ -38,7 +38,7 @@ pub struct DfSchema { pub struct LogicalPlanNode { #[prost( oneof = "logical_plan_node::LogicalPlanType", - tags = "1, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27" + tags = "1, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28" )] pub logical_plan_type: ::core::option::Option, } @@ -99,6 +99,8 @@ pub mod logical_plan_node { Prepare(::prost::alloc::boxed::Box), #[prost(message, tag = "27")] DropView(super::DropViewNode), + #[prost(message, tag = "28")] + DistinctOn(::prost::alloc::boxed::Box), } } #[allow(clippy::derive_partial_eq_without_eq)] @@ -483,6 +485,18 @@ pub struct DistinctNode { } #[allow(clippy::derive_partial_eq_without_eq)] #[derive(Clone, PartialEq, ::prost::Message)] +pub struct DistinctOnNode { + #[prost(message, repeated, tag = "1")] + pub on_expr: ::prost::alloc::vec::Vec, + #[prost(message, repeated, tag = "2")] + pub select_expr: ::prost::alloc::vec::Vec, + #[prost(message, repeated, tag = "3")] + pub sort_expr: ::prost::alloc::vec::Vec, + #[prost(message, optional, boxed, tag = "4")] + pub input: ::core::option::Option<::prost::alloc::boxed::Box>, +} +#[allow(clippy::derive_partial_eq_without_eq)] +#[derive(Clone, PartialEq, ::prost::Message)] pub struct UnionNode { #[prost(message, repeated, tag = "1")] pub inputs: ::prost::alloc::vec::Vec, @@ -754,6 +768,8 @@ pub struct AliasNode { pub expr: ::core::option::Option<::prost::alloc::boxed::Box>, #[prost(string, tag = "2")] pub alias: ::prost::alloc::string::String, + #[prost(message, repeated, tag = "3")] + pub relation: ::prost::alloc::vec::Vec, } #[allow(clippy::derive_partial_eq_without_eq)] #[derive(Clone, PartialEq, ::prost::Message)] diff --git a/datafusion/proto/src/logical_plan/from_proto.rs b/datafusion/proto/src/logical_plan/from_proto.rs index b2b66693f78d..674492edef43 100644 --- a/datafusion/proto/src/logical_plan/from_proto.rs +++ b/datafusion/proto/src/logical_plan/from_proto.rs @@ -1151,6 +1151,11 @@ pub fn parse_expr( } ExprType::Alias(alias) => Ok(Expr::Alias(Alias::new( parse_required_expr(alias.expr.as_deref(), registry, "expr")?, + alias + .relation + .first() + .map(|r| OwnedTableReference::try_from(r.clone())) + .transpose()?, alias.alias.clone(), ))), ExprType::IsNullExpr(is_null) => Ok(Expr::IsNull(Box::new(parse_required_expr( diff --git a/datafusion/proto/src/logical_plan/mod.rs b/datafusion/proto/src/logical_plan/mod.rs index e426c598523e..851f062bd51f 100644 --- a/datafusion/proto/src/logical_plan/mod.rs +++ b/datafusion/proto/src/logical_plan/mod.rs @@ -55,7 +55,7 @@ use datafusion_expr::{ EmptyRelation, Extension, Join, JoinConstraint, Limit, Prepare, Projection, Repartition, Sort, SubqueryAlias, TableScan, Values, Window, }, - DropView, Expr, LogicalPlan, LogicalPlanBuilder, + DistinctOn, DropView, Expr, LogicalPlan, LogicalPlanBuilder, }; use prost::bytes::BufMut; @@ -734,6 +734,33 @@ impl AsLogicalPlan for LogicalPlanNode { into_logical_plan!(distinct.input, ctx, extension_codec)?; LogicalPlanBuilder::from(input).distinct()?.build() } + LogicalPlanType::DistinctOn(distinct_on) => { + let input: LogicalPlan = + into_logical_plan!(distinct_on.input, ctx, extension_codec)?; + let on_expr = distinct_on + .on_expr + .iter() + .map(|expr| from_proto::parse_expr(expr, ctx)) + .collect::, _>>()?; + let select_expr = distinct_on + .select_expr + .iter() + .map(|expr| from_proto::parse_expr(expr, ctx)) + .collect::, _>>()?; + let sort_expr = match distinct_on.sort_expr.len() { + 0 => None, + _ => Some( + distinct_on + .sort_expr + .iter() + .map(|expr| from_proto::parse_expr(expr, ctx)) + .collect::, _>>()?, + ), + }; + LogicalPlanBuilder::from(input) + .distinct_on(on_expr, select_expr, sort_expr)? + .build() + } LogicalPlanType::ViewScan(scan) => { let schema: Schema = convert_required!(scan.schema)?; @@ -1005,7 +1032,7 @@ impl AsLogicalPlan for LogicalPlanNode { ))), }) } - LogicalPlan::Distinct(Distinct { input }) => { + LogicalPlan::Distinct(Distinct::All(input)) => { let input: protobuf::LogicalPlanNode = protobuf::LogicalPlanNode::try_from_logical_plan( input.as_ref(), @@ -1019,6 +1046,42 @@ impl AsLogicalPlan for LogicalPlanNode { ))), }) } + LogicalPlan::Distinct(Distinct::On(DistinctOn { + on_expr, + select_expr, + sort_expr, + input, + .. + })) => { + let input: protobuf::LogicalPlanNode = + protobuf::LogicalPlanNode::try_from_logical_plan( + input.as_ref(), + extension_codec, + )?; + let sort_expr = match sort_expr { + None => vec![], + Some(sort_expr) => sort_expr + .iter() + .map(|expr| expr.try_into()) + .collect::, _>>()?, + }; + Ok(protobuf::LogicalPlanNode { + logical_plan_type: Some(LogicalPlanType::DistinctOn(Box::new( + protobuf::DistinctOnNode { + on_expr: on_expr + .iter() + .map(|expr| expr.try_into()) + .collect::, _>>()?, + select_expr: select_expr + .iter() + .map(|expr| expr.try_into()) + .collect::, _>>()?, + sort_expr, + input: Some(Box::new(input)), + }, + ))), + }) + } LogicalPlan::Window(Window { input, window_expr, .. }) => { diff --git a/datafusion/proto/src/logical_plan/to_proto.rs b/datafusion/proto/src/logical_plan/to_proto.rs index e590731f5810..946f2c6964a5 100644 --- a/datafusion/proto/src/logical_plan/to_proto.rs +++ b/datafusion/proto/src/logical_plan/to_proto.rs @@ -476,9 +476,17 @@ impl TryFrom<&Expr> for protobuf::LogicalExprNode { Expr::Column(c) => Self { expr_type: Some(ExprType::Column(c.into())), }, - Expr::Alias(Alias { expr, name, .. }) => { + Expr::Alias(Alias { + expr, + relation, + name, + }) => { let alias = Box::new(protobuf::AliasNode { expr: Some(Box::new(expr.as_ref().try_into()?)), + relation: relation + .to_owned() + .map(|r| vec![r.into()]) + .unwrap_or(vec![]), alias: name.to_owned(), }); Self { diff --git a/datafusion/proto/tests/cases/roundtrip_logical_plan.rs b/datafusion/proto/tests/cases/roundtrip_logical_plan.rs index 97c553dc04e6..cc76e8a19e98 100644 --- a/datafusion/proto/tests/cases/roundtrip_logical_plan.rs +++ b/datafusion/proto/tests/cases/roundtrip_logical_plan.rs @@ -300,6 +300,32 @@ async fn roundtrip_logical_plan_aggregation() -> Result<()> { Ok(()) } +#[tokio::test] +async fn roundtrip_logical_plan_distinct_on() -> Result<()> { + let ctx = SessionContext::new(); + + let schema = Schema::new(vec![ + Field::new("a", DataType::Int64, true), + Field::new("b", DataType::Decimal128(15, 2), true), + ]); + + ctx.register_csv( + "t1", + "tests/testdata/test.csv", + CsvReadOptions::default().schema(&schema), + ) + .await?; + + let query = "SELECT DISTINCT ON (a % 2) a, b * 2 FROM t1 ORDER BY a % 2 DESC, b"; + let plan = ctx.sql(query).await?.into_optimized_plan()?; + + let bytes = logical_plan_to_bytes(&plan)?; + let logical_round_trip = logical_plan_from_bytes(&bytes, &ctx)?; + assert_eq!(format!("{plan:?}"), format!("{logical_round_trip:?}")); + + Ok(()) +} + #[tokio::test] async fn roundtrip_single_count_distinct() -> Result<()> { let ctx = SessionContext::new(); diff --git a/datafusion/proto/tests/cases/serialize.rs b/datafusion/proto/tests/cases/serialize.rs index f32c81527925..5b890accd81f 100644 --- a/datafusion/proto/tests/cases/serialize.rs +++ b/datafusion/proto/tests/cases/serialize.rs @@ -128,6 +128,12 @@ fn exact_roundtrip_linearized_binary_expr() { } } +#[test] +fn roundtrip_qualified_alias() { + let qual_alias = col("c1").alias_qualified(Some("my_table"), "my_column"); + assert_eq!(qual_alias, roundtrip_expr(&qual_alias)); +} + #[test] fn roundtrip_deeply_nested_binary_expr() { // We need more stack space so this doesn't overflow in dev builds diff --git a/datafusion/sql/src/query.rs b/datafusion/sql/src/query.rs index fc2a3fb9a57b..832e2da9c6ec 100644 --- a/datafusion/sql/src/query.rs +++ b/datafusion/sql/src/query.rs @@ -23,7 +23,7 @@ use datafusion_common::{ not_impl_err, plan_err, sql_err, Constraints, DataFusionError, Result, ScalarValue, }; use datafusion_expr::{ - CreateMemoryTable, DdlStatement, Expr, LogicalPlan, LogicalPlanBuilder, + CreateMemoryTable, DdlStatement, Distinct, Expr, LogicalPlan, LogicalPlanBuilder, }; use sqlparser::ast::{ Expr as SQLExpr, Offset as SQLOffset, OrderByExpr, Query, SetExpr, Value, @@ -161,6 +161,14 @@ impl<'a, S: ContextProvider> SqlToRel<'a, S> { let order_by_rex = self.order_by_to_sort_expr(&order_by, plan.schema(), planner_context)?; - LogicalPlanBuilder::from(plan).sort(order_by_rex)?.build() + + if let LogicalPlan::Distinct(Distinct::On(ref distinct_on)) = plan { + // In case of `DISTINCT ON` we must capture the sort expressions since during the plan + // optimization we're effectively doing a `first_value` aggregation according to them. + let distinct_on = distinct_on.clone().with_sort_expr(order_by_rex)?; + Ok(LogicalPlan::Distinct(Distinct::On(distinct_on))) + } else { + LogicalPlanBuilder::from(plan).sort(order_by_rex)?.build() + } } } diff --git a/datafusion/sql/src/select.rs b/datafusion/sql/src/select.rs index e9a7941ab064..31333affe0af 100644 --- a/datafusion/sql/src/select.rs +++ b/datafusion/sql/src/select.rs @@ -76,7 +76,7 @@ impl<'a, S: ContextProvider> SqlToRel<'a, S> { let empty_from = matches!(plan, LogicalPlan::EmptyRelation(_)); // process `where` clause - let plan = self.plan_selection(select.selection, plan, planner_context)?; + let base_plan = self.plan_selection(select.selection, plan, planner_context)?; // handle named windows before processing the projection expression check_conflicting_windows(&select.named_window)?; @@ -84,16 +84,16 @@ impl<'a, S: ContextProvider> SqlToRel<'a, S> { // process the SELECT expressions, with wildcards expanded. let select_exprs = self.prepare_select_exprs( - &plan, + &base_plan, select.projection, empty_from, planner_context, )?; // having and group by clause may reference aliases defined in select projection - let projected_plan = self.project(plan.clone(), select_exprs.clone())?; + let projected_plan = self.project(base_plan.clone(), select_exprs.clone())?; let mut combined_schema = (**projected_plan.schema()).clone(); - combined_schema.merge(plan.schema()); + combined_schema.merge(base_plan.schema()); // this alias map is resolved and looked up in both having exprs and group by exprs let alias_map = extract_aliases(&select_exprs); @@ -148,7 +148,7 @@ impl<'a, S: ContextProvider> SqlToRel<'a, S> { )?; // aliases from the projection can conflict with same-named expressions in the input let mut alias_map = alias_map.clone(); - for f in plan.schema().fields() { + for f in base_plan.schema().fields() { alias_map.remove(f.name()); } let group_by_expr = @@ -158,7 +158,7 @@ impl<'a, S: ContextProvider> SqlToRel<'a, S> { .unwrap_or(group_by_expr); let group_by_expr = normalize_col(group_by_expr, &projected_plan)?; self.validate_schema_satisfies_exprs( - plan.schema(), + base_plan.schema(), &[group_by_expr.clone()], )?; Ok(group_by_expr) @@ -171,7 +171,7 @@ impl<'a, S: ContextProvider> SqlToRel<'a, S> { .iter() .filter(|select_expr| match select_expr { Expr::AggregateFunction(_) | Expr::AggregateUDF(_) => false, - Expr::Alias(Alias { expr, name: _ }) => !matches!( + Expr::Alias(Alias { expr, name: _, .. }) => !matches!( **expr, Expr::AggregateFunction(_) | Expr::AggregateUDF(_) ), @@ -187,16 +187,16 @@ impl<'a, S: ContextProvider> SqlToRel<'a, S> { || !aggr_exprs.is_empty() { self.aggregate( - plan, + &base_plan, &select_exprs, having_expr_opt.as_ref(), - group_by_exprs, - aggr_exprs, + &group_by_exprs, + &aggr_exprs, )? } else { match having_expr_opt { Some(having_expr) => return plan_err!("HAVING clause references: {having_expr} must appear in the GROUP BY clause or be used in an aggregate function"), - None => (plan, select_exprs, having_expr_opt) + None => (base_plan.clone(), select_exprs.clone(), having_expr_opt) } }; @@ -229,19 +229,35 @@ impl<'a, S: ContextProvider> SqlToRel<'a, S> { let plan = project(plan, select_exprs_post_aggr)?; // process distinct clause - let distinct = select - .distinct - .map(|distinct| match distinct { - Distinct::Distinct => Ok(true), - Distinct::On(_) => not_impl_err!("DISTINCT ON Exprs not supported"), - }) - .transpose()? - .unwrap_or(false); + let plan = match select.distinct { + None => Ok(plan), + Some(Distinct::Distinct) => { + LogicalPlanBuilder::from(plan).distinct()?.build() + } + Some(Distinct::On(on_expr)) => { + if !aggr_exprs.is_empty() + || !group_by_exprs.is_empty() + || !window_func_exprs.is_empty() + { + return not_impl_err!("DISTINCT ON expressions with GROUP BY, aggregation or window functions are not supported "); + } - let plan = if distinct { - LogicalPlanBuilder::from(plan).distinct()?.build() - } else { - Ok(plan) + let on_expr = on_expr + .into_iter() + .map(|e| { + self.sql_expr_to_logical_expr( + e.clone(), + plan.schema(), + planner_context, + ) + }) + .collect::>>()?; + + // Build the final plan + return LogicalPlanBuilder::from(base_plan) + .distinct_on(on_expr, select_exprs, None)? + .build(); + } }?; // DISTRIBUTE BY @@ -471,6 +487,7 @@ impl<'a, S: ContextProvider> SqlToRel<'a, S> { .clone(); *expr = Expr::Alias(Alias { expr: Box::new(new_expr), + relation: None, name: name.clone(), }); } @@ -511,18 +528,18 @@ impl<'a, S: ContextProvider> SqlToRel<'a, S> { /// the aggregate fn aggregate( &self, - input: LogicalPlan, + input: &LogicalPlan, select_exprs: &[Expr], having_expr_opt: Option<&Expr>, - group_by_exprs: Vec, - aggr_exprs: Vec, + group_by_exprs: &[Expr], + aggr_exprs: &[Expr], ) -> Result<(LogicalPlan, Vec, Option)> { let group_by_exprs = - get_updated_group_by_exprs(&group_by_exprs, select_exprs, input.schema())?; + get_updated_group_by_exprs(group_by_exprs, select_exprs, input.schema())?; // create the aggregate plan let plan = LogicalPlanBuilder::from(input.clone()) - .aggregate(group_by_exprs.clone(), aggr_exprs.clone())? + .aggregate(group_by_exprs.clone(), aggr_exprs.to_vec())? .build()?; // in this next section of code we are re-writing the projection to refer to columns @@ -549,25 +566,25 @@ impl<'a, S: ContextProvider> SqlToRel<'a, S> { _ => aggr_projection_exprs.push(expr.clone()), } } - aggr_projection_exprs.extend_from_slice(&aggr_exprs); + aggr_projection_exprs.extend_from_slice(aggr_exprs); // now attempt to resolve columns and replace with fully-qualified columns let aggr_projection_exprs = aggr_projection_exprs .iter() - .map(|expr| resolve_columns(expr, &input)) + .map(|expr| resolve_columns(expr, input)) .collect::>>()?; // next we replace any expressions that are not a column with a column referencing // an output column from the aggregate schema let column_exprs_post_aggr = aggr_projection_exprs .iter() - .map(|expr| expr_as_column_expr(expr, &input)) + .map(|expr| expr_as_column_expr(expr, input)) .collect::>>()?; // next we re-write the projection let select_exprs_post_aggr = select_exprs .iter() - .map(|expr| rebase_expr(expr, &aggr_projection_exprs, &input)) + .map(|expr| rebase_expr(expr, &aggr_projection_exprs, input)) .collect::>>()?; // finally, we have some validation that the re-written projection can be resolved @@ -582,7 +599,7 @@ impl<'a, S: ContextProvider> SqlToRel<'a, S> { // aggregation. let having_expr_post_aggr = if let Some(having_expr) = having_expr_opt { let having_expr_post_aggr = - rebase_expr(having_expr, &aggr_projection_exprs, &input)?; + rebase_expr(having_expr, &aggr_projection_exprs, input)?; check_columns_satisfy_exprs( &column_exprs_post_aggr, diff --git a/datafusion/sqllogictest/test_files/distinct_on.slt b/datafusion/sqllogictest/test_files/distinct_on.slt new file mode 100644 index 000000000000..8a36b49b98c6 --- /dev/null +++ b/datafusion/sqllogictest/test_files/distinct_on.slt @@ -0,0 +1,146 @@ +# 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 EXTERNAL TABLE aggregate_test_100 ( + c1 VARCHAR NOT NULL, + c2 TINYINT NOT NULL, + c3 SMALLINT NOT NULL, + c4 SMALLINT, + c5 INT, + c6 BIGINT NOT NULL, + c7 SMALLINT NOT NULL, + c8 INT NOT NULL, + c9 BIGINT UNSIGNED NOT NULL, + c10 VARCHAR NOT NULL, + c11 FLOAT NOT NULL, + c12 DOUBLE NOT NULL, + c13 VARCHAR NOT NULL +) +STORED AS CSV +WITH HEADER ROW +LOCATION '../../testing/data/csv/aggregate_test_100.csv' + +# Basic example: distinct on the first column project the second one, and +# order by the third +query TI +SELECT DISTINCT ON (c1) c1, c2 FROM aggregate_test_100 ORDER BY c1, c3; +---- +a 5 +b 4 +c 2 +d 1 +e 3 + +# Basic example + reverse order of the selected column +query TI +SELECT DISTINCT ON (c1) c1, c2 FROM aggregate_test_100 ORDER BY c1, c3 DESC; +---- +a 1 +b 5 +c 4 +d 1 +e 1 + +# Basic example + reverse order of the ON column +query TI +SELECT DISTINCT ON (c1) c1, c2 FROM aggregate_test_100 ORDER BY c1 DESC, c3; +---- +e 3 +d 1 +c 2 +b 4 +a 4 + +# Basic example + reverse order of both columns + limit +query TI +SELECT DISTINCT ON (c1) c1, c2 FROM aggregate_test_100 ORDER BY c1 DESC, c3 DESC LIMIT 3; +---- +e 1 +d 1 +c 4 + +# Basic example + omit ON column from selection +query I +SELECT DISTINCT ON (c1) c2 FROM aggregate_test_100 ORDER BY c1, c3; +---- +5 +4 +2 +1 +3 + +# Test explain makes sense +query TT +EXPLAIN SELECT DISTINCT ON (c1) c3, c2 FROM aggregate_test_100 ORDER BY c1, c3; +---- +logical_plan +Projection: FIRST_VALUE(aggregate_test_100.c3) ORDER BY [aggregate_test_100.c1 ASC NULLS LAST, aggregate_test_100.c3 ASC NULLS LAST] AS c3, FIRST_VALUE(aggregate_test_100.c2) ORDER BY [aggregate_test_100.c1 ASC NULLS LAST, aggregate_test_100.c3 ASC NULLS LAST] AS c2 +--Sort: aggregate_test_100.c1 ASC NULLS LAST +----Aggregate: groupBy=[[aggregate_test_100.c1]], aggr=[[FIRST_VALUE(aggregate_test_100.c3) ORDER BY [aggregate_test_100.c1 ASC NULLS LAST, aggregate_test_100.c3 ASC NULLS LAST], FIRST_VALUE(aggregate_test_100.c2) ORDER BY [aggregate_test_100.c1 ASC NULLS LAST, aggregate_test_100.c3 ASC NULLS LAST]]] +------TableScan: aggregate_test_100 projection=[c1, c2, c3] +physical_plan +ProjectionExec: expr=[FIRST_VALUE(aggregate_test_100.c3) ORDER BY [aggregate_test_100.c1 ASC NULLS LAST, aggregate_test_100.c3 ASC NULLS LAST]@1 as c3, FIRST_VALUE(aggregate_test_100.c2) ORDER BY [aggregate_test_100.c1 ASC NULLS LAST, aggregate_test_100.c3 ASC NULLS LAST]@2 as c2] +--SortPreservingMergeExec: [c1@0 ASC NULLS LAST] +----SortExec: expr=[c1@0 ASC NULLS LAST] +------AggregateExec: mode=FinalPartitioned, gby=[c1@0 as c1], aggr=[FIRST_VALUE(aggregate_test_100.c3), FIRST_VALUE(aggregate_test_100.c2)] +--------CoalesceBatchesExec: target_batch_size=8192 +----------RepartitionExec: partitioning=Hash([c1@0], 4), input_partitions=4 +------------AggregateExec: mode=Partial, gby=[c1@0 as c1], aggr=[FIRST_VALUE(aggregate_test_100.c3), FIRST_VALUE(aggregate_test_100.c2)], ordering_mode=Sorted +--------------SortExec: expr=[c1@0 ASC NULLS LAST,c3@2 ASC NULLS LAST] +----------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1, c2, c3], has_header=true + +# ON expressions are not a sub-set of the ORDER BY expressions +query error SELECT DISTINCT ON expressions must match initial ORDER BY expressions +SELECT DISTINCT ON (c2 % 2 = 0) c2, c3 - 100 FROM aggregate_test_100 ORDER BY c2, c3; + +# ON expressions are empty +query error DataFusion error: Error during planning: No `ON` expressions provided +SELECT DISTINCT ON () c1, c2 FROM aggregate_test_100 ORDER BY c1, c2; + +# Use expressions in the ON and ORDER BY clauses, as well as the selection +query II +SELECT DISTINCT ON (c2 % 2 = 0) c2, c3 - 100 FROM aggregate_test_100 ORDER BY c2 % 2 = 0, c3 DESC; +---- +1 25 +4 23 + +# Multiple complex expressions +query TIB +SELECT DISTINCT ON (chr(ascii(c1) + 3), c2 % 2) chr(ascii(upper(c1)) + 3), c2 % 2, c3 > 80 AND c2 % 2 = 1 +FROM aggregate_test_100 +WHERE c1 IN ('a', 'b') +ORDER BY chr(ascii(c1) + 3), c2 % 2, c3 DESC; +---- +D 0 false +D 1 true +E 0 false +E 1 false + +# Joins using CTEs +query II +WITH t1 AS (SELECT * FROM aggregate_test_100), +t2 AS (SELECT * FROM aggregate_test_100) +SELECT DISTINCT ON (t1.c1, t2.c2) t2.c3, t1.c4 +FROM t1 INNER JOIN t2 ON t1.c13 = t2.c13 +ORDER BY t1.c1, t2.c2, t2.c5 +LIMIT 3; +---- +-25 15295 +45 15673 +-72 -11122 diff --git a/datafusion/substrait/src/logical_plan/producer.rs b/datafusion/substrait/src/logical_plan/producer.rs index 6fe8eca33705..9356a7753427 100644 --- a/datafusion/substrait/src/logical_plan/producer.rs +++ b/datafusion/substrait/src/logical_plan/producer.rs @@ -19,7 +19,7 @@ use std::collections::HashMap; use std::ops::Deref; use std::sync::Arc; -use datafusion::logical_expr::{Like, WindowFrameUnits}; +use datafusion::logical_expr::{Distinct, Like, WindowFrameUnits}; use datafusion::{ arrow::datatypes::{DataType, TimeUnit}, error::{DataFusionError, Result}, @@ -244,11 +244,11 @@ pub fn to_substrait_rel( }))), })) } - LogicalPlan::Distinct(distinct) => { + LogicalPlan::Distinct(Distinct::All(plan)) => { // Use Substrait's AggregateRel with empty measures to represent `select distinct` - let input = to_substrait_rel(distinct.input.as_ref(), ctx, extension_info)?; + let input = to_substrait_rel(plan.as_ref(), ctx, extension_info)?; // Get grouping keys from the input relation's number of output fields - let grouping = (0..distinct.input.schema().fields().len()) + let grouping = (0..plan.schema().fields().len()) .map(substrait_field_ref) .collect::>>()?; From 3df895597d8c2073081fd9d990048c7aefb3b62e Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Mon, 13 Nov 2023 09:44:35 -0700 Subject: [PATCH 241/572] Prepare 33.0.0-rc2 (#8144) * Update changelog for 33.0.0-rc2 * Update changelog for 33.0.0-rc2 * Use arrow-rs 48.0.1 --- Cargo.toml | 14 ++--- datafusion-cli/Cargo.lock | 98 ++++++++++++++----------------- datafusion-cli/Cargo.toml | 2 +- dev/changelog/33.0.0.md | 70 +++++++++++++++++++++- dev/release/generate-changelog.py | 4 ++ 5 files changed, 123 insertions(+), 65 deletions(-) diff --git a/Cargo.toml b/Cargo.toml index e7a4126743f2..7294c934b72b 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -49,12 +49,12 @@ rust-version = "1.70" version = "33.0.0" [workspace.dependencies] -arrow = { version = "48.0.0", features = ["prettyprint"] } -arrow-array = { version = "48.0.0", default-features = false, features = ["chrono-tz"] } -arrow-buffer = { version = "48.0.0", default-features = false } -arrow-flight = { version = "48.0.0", features = ["flight-sql-experimental"] } -arrow-ord = { version = "48.0.0", default-features = false } -arrow-schema = { version = "48.0.0", default-features = false } +arrow = { version = "~48.0.1", features = ["prettyprint"] } +arrow-array = { version = "~48.0.1", default-features = false, features = ["chrono-tz"] } +arrow-buffer = { version = "~48.0.1", default-features = false } +arrow-flight = { version = "~48.0.1", features = ["flight-sql-experimental"] } +arrow-ord = { version = "~48.0.1", default-features = false } +arrow-schema = { version = "~48.0.1", default-features = false } async-trait = "0.1.73" bigdecimal = "0.4.1" bytes = "1.4" @@ -81,7 +81,7 @@ log = "^0.4" num_cpus = "1.13.0" object_store = { version = "0.7.0", default-features = false } parking_lot = "0.12" -parquet = { version = "48.0.0", default-features = false, features = ["arrow", "async", "object_store"] } +parquet = { version = "~48.0.1", default-features = false, features = ["arrow", "async", "object_store"] } rand = "0.8" rstest = "0.18.0" serde_json = "1" diff --git a/datafusion-cli/Cargo.lock b/datafusion-cli/Cargo.lock index 629293e4839b..f0bb28469d2d 100644 --- a/datafusion-cli/Cargo.lock +++ b/datafusion-cli/Cargo.lock @@ -130,9 +130,9 @@ checksum = "96d30a06541fbafbc7f82ed10c06164cfbd2c401138f6addd8404629c4b16711" [[package]] name = "arrow" -version = "48.0.0" +version = "48.0.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "edb738d83750ec705808f6d44046d165e6bb8623f64e29a4d53fcb136ab22dfb" +checksum = "a8919668503a4f2d8b6da96fa7c16e93046bfb3412ffcfa1e5dc7d2e3adcb378" dependencies = [ "ahash", "arrow-arith", @@ -152,9 +152,9 @@ dependencies = [ [[package]] name = "arrow-arith" -version = "48.0.0" +version = "48.0.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "c5c3d17fc5b006e7beeaebfb1d2edfc92398b981f82d9744130437909b72a468" +checksum = "ef983914f477d4278b068f13b3224b7d19eb2b807ac9048544d3bfebdf2554c4" dependencies = [ "arrow-array", "arrow-buffer", @@ -167,9 +167,9 @@ dependencies = [ [[package]] name = "arrow-array" -version = "48.0.0" +version = "48.0.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "55705ada5cdde4cb0f202ffa6aa756637e33fea30e13d8d0d0fd6a24ffcee1e3" +checksum = "d6eaf89041fa5937940ae390294ece29e1db584f46d995608d6e5fe65a2e0e9b" dependencies = [ "ahash", "arrow-buffer", @@ -184,9 +184,9 @@ dependencies = [ [[package]] name = "arrow-buffer" -version = "48.0.0" +version = "48.0.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "a722f90a09b94f295ab7102542e97199d3500128843446ef63e410ad546c5333" +checksum = "55512d988c6fbd76e514fd3ff537ac50b0a675da5a245e4fdad77ecfd654205f" dependencies = [ "bytes", "half", @@ -195,9 +195,9 @@ dependencies = [ [[package]] name = "arrow-cast" -version = "48.0.0" +version = "48.0.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "af01fc1a06f6f2baf31a04776156d47f9f31ca5939fe6d00cd7a059f95a46ff1" +checksum = "655ee51a2156ba5375931ce21c1b2494b1d9260e6dcdc6d4db9060c37dc3325b" dependencies = [ "arrow-array", "arrow-buffer", @@ -213,9 +213,9 @@ dependencies = [ [[package]] name = "arrow-csv" -version = "48.0.0" +version = "48.0.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "83cbbfde86f9ecd3f875c42a73d8aeab3d95149cd80129b18d09e039ecf5391b" +checksum = "258bb689997ad5b6660b3ce3638bd6b383d668ec555ed41ad7c6559cbb2e4f91" dependencies = [ "arrow-array", "arrow-buffer", @@ -232,9 +232,9 @@ dependencies = [ [[package]] name = "arrow-data" -version = "48.0.0" +version = "48.0.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "d0a547195e607e625e7fafa1a7269b8df1a4a612c919efd9b26bd86e74538f3a" +checksum = "6dc2b9fec74763427e2e5575b8cc31ce96ba4c9b4eb05ce40e0616d9fad12461" dependencies = [ "arrow-buffer", "arrow-schema", @@ -244,9 +244,9 @@ dependencies = [ [[package]] name = "arrow-ipc" -version = "48.0.0" +version = "48.0.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "e36bf091502ab7e37775ff448413ef1ffff28ff93789acb669fffdd51b394d51" +checksum = "6eaa6ab203cc6d89b7eaa1ac781c1dfeef325454c5d5a0419017f95e6bafc03c" dependencies = [ "arrow-array", "arrow-buffer", @@ -258,9 +258,9 @@ dependencies = [ [[package]] name = "arrow-json" -version = "48.0.0" +version = "48.0.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "7ac346bc84846ab425ab3c8c7b6721db90643bc218939677ed7e071ccbfb919d" +checksum = "fb64e30d9b73f66fdc5c52d5f4cf69bbf03d62f64ffeafa0715590a5320baed7" dependencies = [ "arrow-array", "arrow-buffer", @@ -278,9 +278,9 @@ dependencies = [ [[package]] name = "arrow-ord" -version = "48.0.0" +version = "48.0.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "4502123d2397319f3a13688432bc678c61cb1582f2daa01253186da650bf5841" +checksum = "f9a818951c0d11c428dda03e908175969c262629dd20bd0850bd6c7a8c3bfe48" dependencies = [ "arrow-array", "arrow-buffer", @@ -293,9 +293,9 @@ dependencies = [ [[package]] name = "arrow-row" -version = "48.0.0" +version = "48.0.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "249fc5a07906ab3f3536a6e9f118ec2883fbcde398a97a5ba70053f0276abda4" +checksum = "a5d664318bc05f930559fc088888f0f7174d3c5bc888c0f4f9ae8f23aa398ba3" dependencies = [ "ahash", "arrow-array", @@ -308,15 +308,15 @@ dependencies = [ [[package]] name = "arrow-schema" -version = "48.0.0" +version = "48.0.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "9d7a8c3f97f5ef6abd862155a6f39aaba36b029322462d72bbcfa69782a50614" +checksum = "aaf4d737bba93da59f16129bec21e087aed0be84ff840e74146d4703879436cb" [[package]] name = "arrow-select" -version = "48.0.0" +version = "48.0.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "f868f4a5001429e20f7c1994b5cd1aa68b82e3db8cf96c559cdb56dc8be21410" +checksum = "374c4c3b812ecc2118727b892252a4a4308f87a8aca1dbf09f3ce4bc578e668a" dependencies = [ "ahash", "arrow-array", @@ -328,9 +328,9 @@ dependencies = [ [[package]] name = "arrow-string" -version = "48.0.0" +version = "48.0.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "a27fdf8fc70040a2dee78af2e217479cb5b263bd7ab8711c7999e74056eb688a" +checksum = "b15aed5624bb23da09142f58502b59c23f5bea607393298bb81dab1ce60fc769" dependencies = [ "arrow-array", "arrow-buffer", @@ -790,9 +790,9 @@ dependencies = [ [[package]] name = "bstr" -version = "1.7.0" +version = "1.8.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "c79ad7fb2dd38f3dabd76b09c6a5a20c038fc0213ef1e9afd30eb777f120f019" +checksum = "542f33a8835a0884b006a0c3df3dadd99c0c3f296ed26c2fdc8028e01ad6230c" dependencies = [ "memchr", "regex-automata", @@ -850,11 +850,10 @@ dependencies = [ [[package]] name = "cc" -version = "1.0.83" +version = "1.0.84" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "f1174fb0b6ec23863f8b971027804a42614e347eafb0a95bf0b12cdae21fc4d0" +checksum = "0f8e7c90afad890484a21653d08b6e209ae34770fb5ee298f9c699fcc1e5c856" dependencies = [ - "jobserver", "libc", ] @@ -1737,9 +1736,9 @@ dependencies = [ [[package]] name = "http" -version = "0.2.9" +version = "0.2.10" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "bd6effc99afb63425aff9b05836f029929e345a6148a14b7ecd5ab67af944482" +checksum = "f95b9abcae896730d42b78e09c155ed4ddf82c07b4de772c64aee5b2d8b7c150" dependencies = [ "bytes", "fnv", @@ -1917,15 +1916,6 @@ version = "1.0.9" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "af150ab688ff2122fcef229be89cb50dd66af9e01a4ff320cc137eecc9bacc38" -[[package]] -name = "jobserver" -version = "0.1.27" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "8c37f63953c4c63420ed5fd3d6d398c719489b9f872b9fa683262f8edd363c7d" -dependencies = [ - "libc", -] - [[package]] name = "js-sys" version = "0.3.65" @@ -2365,9 +2355,9 @@ dependencies = [ [[package]] name = "parquet" -version = "48.0.0" +version = "48.0.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "239229e6a668ab50c61de3dce61cf0fa1069345f7aa0f4c934491f92205a4945" +checksum = "6bfe55df96e3f02f11bf197ae37d91bb79801631f82f6195dd196ef521df3597" dependencies = [ "ahash", "arrow-array", @@ -2869,9 +2859,9 @@ dependencies = [ [[package]] name = "rustls-pemfile" -version = "1.0.3" +version = "1.0.4" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "2d3987094b1d07b653b7dfdc3f70ce9a1da9c51ac18c1b06b662e4f9a0e9f4b2" +checksum = "1c74cae0a4cf6ccbbf5f359f08efdf8ee7e1dc532573bf0db71968cb56b1448c" dependencies = [ "base64", ] @@ -3061,9 +3051,9 @@ dependencies = [ [[package]] name = "smallvec" -version = "1.11.1" +version = "1.11.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "942b4a808e05215192e39f4ab80813e599068285906cc91aa64f923db842bd5a" +checksum = "4dccd0940a2dcdf68d092b8cbab7dc0ad8fa938bf95787e1b916b0e3d0e8e970" [[package]] name = "snafu" @@ -3354,9 +3344,9 @@ checksum = "1f3ccbac311fea05f86f61904b462b55fb3df8837a366dfc601a0161d0532f20" [[package]] name = "tokio" -version = "1.33.0" +version = "1.34.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "4f38200e3ef7995e5ef13baec2f432a6da0aa9ac495b2c0e8f3b7eec2c92d653" +checksum = "d0c014766411e834f7af5b8f4cf46257aab4036ca95e9d2c144a10f59ad6f5b9" dependencies = [ "backtrace", "bytes", @@ -3372,9 +3362,9 @@ dependencies = [ [[package]] name = "tokio-macros" -version = "2.1.0" +version = "2.2.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "630bdcf245f78637c13ec01ffae6187cca34625e8c63150d424b59e55af2675e" +checksum = "5b8a1e28f2deaa14e508979454cb3a223b10b938b45af148bc0986de36f1923b" dependencies = [ "proc-macro2", "quote", diff --git a/datafusion-cli/Cargo.toml b/datafusion-cli/Cargo.toml index 73c4431f4352..890f84522c26 100644 --- a/datafusion-cli/Cargo.toml +++ b/datafusion-cli/Cargo.toml @@ -29,7 +29,7 @@ rust-version = "1.70" readme = "README.md" [dependencies] -arrow = "48.0.0" +arrow = "~48.0.1" async-trait = "0.1.41" aws-config = "0.55" aws-credential-types = "0.55" diff --git a/dev/changelog/33.0.0.md b/dev/changelog/33.0.0.md index 9acf40705264..17862a64a951 100644 --- a/dev/changelog/33.0.0.md +++ b/dev/changelog/33.0.0.md @@ -17,9 +17,9 @@ under the License. --> -## [33.0.0](https://github.com/apache/arrow-datafusion/tree/33.0.0) (2023-11-05) +## [33.0.0](https://github.com/apache/arrow-datafusion/tree/33.0.0) (2023-11-12) -[Full Changelog](https://github.com/apache/arrow-datafusion/compare/31.0.0...32.0.0) +[Full Changelog](https://github.com/apache/arrow-datafusion/compare/32.0.0...33.0.0) **Breaking changes:** @@ -28,6 +28,14 @@ - Add `parquet` feature flag, enabled by default, and make parquet conditional [#7745](https://github.com/apache/arrow-datafusion/pull/7745) (ongchi) - Change input for `to_timestamp` function to be seconds rather than nanoseconds, add `to_timestamp_nanos` [#7844](https://github.com/apache/arrow-datafusion/pull/7844) (comphead) - Percent Decode URL Paths (#8009) [#8012](https://github.com/apache/arrow-datafusion/pull/8012) (tustvold) +- chore: remove panics in datafusion-common::scalar by making more operations return `Result` [#7901](https://github.com/apache/arrow-datafusion/pull/7901) (junjunjd) +- Combine `Expr::Wildcard` and `Wxpr::QualifiedWildcard`, add `wildcard()` expr fn [#8105](https://github.com/apache/arrow-datafusion/pull/8105) (alamb) + +**Performance related:** + +- Add distinct union optimization [#7788](https://github.com/apache/arrow-datafusion/pull/7788) (maruschin) +- Fix join order for TPCH Q17 & Q18 by improving FilterExec statistics [#8126](https://github.com/apache/arrow-datafusion/pull/8126) (andygrove) +- feat: add column statistics into explain [#8112](https://github.com/apache/arrow-datafusion/pull/8112) (NGA-TRAN) **Implemented enhancements:** @@ -36,7 +44,6 @@ - add interval arithmetic for timestamp types [#7758](https://github.com/apache/arrow-datafusion/pull/7758) (mhilton) - Interval Arithmetic NegativeExpr Support [#7804](https://github.com/apache/arrow-datafusion/pull/7804) (berkaysynnada) - Exactness Indicator of Parameters: Precision [#7809](https://github.com/apache/arrow-datafusion/pull/7809) (berkaysynnada) -- Add distinct union optimization [#7788](https://github.com/apache/arrow-datafusion/pull/7788) (maruschin) - Implement GetIndexedField for map-typed columns [#7825](https://github.com/apache/arrow-datafusion/pull/7825) (swgillespie) - Fix precision loss when coercing date_part utf8 argument [#7846](https://github.com/apache/arrow-datafusion/pull/7846) (Dandandan) - Support `Binary`/`LargeBinary` --> `Utf8`/`LargeUtf8` in ilike and string functions [#7840](https://github.com/apache/arrow-datafusion/pull/7840) (alamb) @@ -49,6 +56,10 @@ - feat: Use bloom filter when reading parquet to skip row groups [#7821](https://github.com/apache/arrow-datafusion/pull/7821) (hengfeiyang) - Support Partitioning Data by Dictionary Encoded String Array Types [#7896](https://github.com/apache/arrow-datafusion/pull/7896) (devinjdangelo) - Read only enough bytes to infer Arrow IPC file schema via stream [#7962](https://github.com/apache/arrow-datafusion/pull/7962) (Jefffrey) +- feat: Support determining extensions from names like `foo.parquet.snappy` as well as `foo.parquet` [#7972](https://github.com/apache/arrow-datafusion/pull/7972) (Weijun-H) +- feat: Protobuf serde for Json file sink [#8062](https://github.com/apache/arrow-datafusion/pull/8062) (Jefffrey) +- feat: support target table alias in update statement [#8080](https://github.com/apache/arrow-datafusion/pull/8080) (jonahgao) +- feat: support UDAF in substrait producer/consumer [#8119](https://github.com/apache/arrow-datafusion/pull/8119) (waynexia) **Fixed bugs:** @@ -57,6 +68,8 @@ - fix: generate logical plan for `UPDATE SET FROM` statement [#7984](https://github.com/apache/arrow-datafusion/pull/7984) (jonahgao) - fix: single_distinct_aggretation_to_group_by fail [#7997](https://github.com/apache/arrow-datafusion/pull/7997) (haohuaijin) - fix: clippy warnings from nightly rust 1.75 [#8025](https://github.com/apache/arrow-datafusion/pull/8025) (waynexia) +- fix: DataFusion suggests invalid functions [#8083](https://github.com/apache/arrow-datafusion/pull/8083) (jonahgao) +- fix: add encode/decode to protobuf encoding [#8089](https://github.com/apache/arrow-datafusion/pull/8089) (Syleechan) **Documentation updates:** @@ -69,6 +82,10 @@ - Minor: Improve documentation for Filter Pushdown [#8023](https://github.com/apache/arrow-datafusion/pull/8023) (alamb) - Minor: Improve `ExecutionPlan` documentation [#8019](https://github.com/apache/arrow-datafusion/pull/8019) (alamb) - Improve comments for `PartitionSearchMode` struct [#8047](https://github.com/apache/arrow-datafusion/pull/8047) (ozankabak) +- Prepare 33.0.0 Release [#8057](https://github.com/apache/arrow-datafusion/pull/8057) (andygrove) +- Improve documentation for calculate_prune_length method in `SymmetricHashJoin` [#8125](https://github.com/apache/arrow-datafusion/pull/8125) (Asura7969) +- docs: show creation of DFSchema [#8132](https://github.com/apache/arrow-datafusion/pull/8132) (wjones127) +- Improve documentation site to make it easier to find communication on Slack/Discord [#8138](https://github.com/apache/arrow-datafusion/pull/8138) (alamb) **Merged pull requests:** @@ -226,3 +243,50 @@ - General approach for Array replace [#8050](https://github.com/apache/arrow-datafusion/pull/8050) (jayzhan211) - Minor: Remove the irrelevant note from the Expression API doc [#8053](https://github.com/apache/arrow-datafusion/pull/8053) (ongchi) - Minor: Add more documentation about Partitioning [#8022](https://github.com/apache/arrow-datafusion/pull/8022) (alamb) +- Minor: improve documentation for IsNotNull, DISTINCT, etc [#8052](https://github.com/apache/arrow-datafusion/pull/8052) (alamb) +- Prepare 33.0.0 Release [#8057](https://github.com/apache/arrow-datafusion/pull/8057) (andygrove) +- Minor: improve error message by adding types to message [#8065](https://github.com/apache/arrow-datafusion/pull/8065) (alamb) +- Minor: Remove redundant BuiltinScalarFunction::supports_zero_argument() [#8059](https://github.com/apache/arrow-datafusion/pull/8059) (2010YOUY01) +- Add example to ci [#8060](https://github.com/apache/arrow-datafusion/pull/8060) (smallzhongfeng) +- Update substrait requirement from 0.18.0 to 0.19.0 [#8076](https://github.com/apache/arrow-datafusion/pull/8076) (dependabot[bot]) +- Fix incorrect results in COUNT(\*) queries with LIMIT [#8049](https://github.com/apache/arrow-datafusion/pull/8049) (msirek) +- feat: Support determining extensions from names like `foo.parquet.snappy` as well as `foo.parquet` [#7972](https://github.com/apache/arrow-datafusion/pull/7972) (Weijun-H) +- Use FairSpillPool for TaskContext with spillable config [#8072](https://github.com/apache/arrow-datafusion/pull/8072) (viirya) +- Minor: Improve HashJoinStream docstrings [#8070](https://github.com/apache/arrow-datafusion/pull/8070) (alamb) +- Fixing broken link [#8085](https://github.com/apache/arrow-datafusion/pull/8085) (edmondop) +- fix: DataFusion suggests invalid functions [#8083](https://github.com/apache/arrow-datafusion/pull/8083) (jonahgao) +- Replace macro with function for `array_repeat` [#8071](https://github.com/apache/arrow-datafusion/pull/8071) (jayzhan211) +- Minor: remove unnecessary projection in `single_distinct_to_group_by` rule [#8061](https://github.com/apache/arrow-datafusion/pull/8061) (haohuaijin) +- minor: Remove duplicate version numbers for arrow, object_store, and parquet dependencies [#8095](https://github.com/apache/arrow-datafusion/pull/8095) (andygrove) +- fix: add encode/decode to protobuf encoding [#8089](https://github.com/apache/arrow-datafusion/pull/8089) (Syleechan) +- feat: Protobuf serde for Json file sink [#8062](https://github.com/apache/arrow-datafusion/pull/8062) (Jefffrey) +- Minor: use `Expr::alias` in a few places to make the code more concise [#8097](https://github.com/apache/arrow-datafusion/pull/8097) (alamb) +- Minor: Cleanup BuiltinScalarFunction::return_type() [#8088](https://github.com/apache/arrow-datafusion/pull/8088) (2010YOUY01) +- Update sqllogictest requirement from 0.17.0 to 0.18.0 [#8102](https://github.com/apache/arrow-datafusion/pull/8102) (dependabot[bot]) +- Projection Pushdown in PhysicalPlan [#8073](https://github.com/apache/arrow-datafusion/pull/8073) (berkaysynnada) +- Push limit into aggregation for DISTINCT ... LIMIT queries [#8038](https://github.com/apache/arrow-datafusion/pull/8038) (msirek) +- Bug-fix in Filter and Limit statistics [#8094](https://github.com/apache/arrow-datafusion/pull/8094) (berkaysynnada) +- feat: support target table alias in update statement [#8080](https://github.com/apache/arrow-datafusion/pull/8080) (jonahgao) +- Minor: Simlify downcast functions in cast.rs. [#8103](https://github.com/apache/arrow-datafusion/pull/8103) (Weijun-H) +- Fix ArrayAgg schema mismatch issue [#8055](https://github.com/apache/arrow-datafusion/pull/8055) (jayzhan211) +- Minor: Support `nulls` in `array_replace`, avoid a copy [#8054](https://github.com/apache/arrow-datafusion/pull/8054) (alamb) +- Minor: Improve the document format of JoinHashMap [#8090](https://github.com/apache/arrow-datafusion/pull/8090) (Asura7969) +- Simplify ProjectionPushdown and make it more general [#8109](https://github.com/apache/arrow-datafusion/pull/8109) (alamb) +- Minor: clean up the code regarding clippy [#8122](https://github.com/apache/arrow-datafusion/pull/8122) (Weijun-H) +- Support remaining functions in protobuf serialization, add `expr_fn` for `StructFunction` [#8100](https://github.com/apache/arrow-datafusion/pull/8100) (JacobOgle) +- Minor: Cleanup BuiltinScalarFunction's phys-expr creation [#8114](https://github.com/apache/arrow-datafusion/pull/8114) (2010YOUY01) +- rewrite `array_append/array_prepend` to remove deplicate codes [#8108](https://github.com/apache/arrow-datafusion/pull/8108) (Veeupup) +- Implementation of `array_intersect` [#8081](https://github.com/apache/arrow-datafusion/pull/8081) (Veeupup) +- Minor: fix ci break [#8136](https://github.com/apache/arrow-datafusion/pull/8136) (haohuaijin) +- Improve documentation for calculate_prune_length method in `SymmetricHashJoin` [#8125](https://github.com/apache/arrow-datafusion/pull/8125) (Asura7969) +- Minor: remove duplicated `array_replace` tests [#8066](https://github.com/apache/arrow-datafusion/pull/8066) (alamb) +- Minor: Fix temporary files created but not deleted during testing [#8115](https://github.com/apache/arrow-datafusion/pull/8115) (2010YOUY01) +- chore: remove panics in datafusion-common::scalar by making more operations return `Result` [#7901](https://github.com/apache/arrow-datafusion/pull/7901) (junjunjd) +- Fix join order for TPCH Q17 & Q18 by improving FilterExec statistics [#8126](https://github.com/apache/arrow-datafusion/pull/8126) (andygrove) +- Fix: Do not try and preserve order when there is no order to preserve in RepartitionExec [#8127](https://github.com/apache/arrow-datafusion/pull/8127) (alamb) +- feat: add column statistics into explain [#8112](https://github.com/apache/arrow-datafusion/pull/8112) (NGA-TRAN) +- Add subtrait support for `IS NULL` and `IS NOT NULL` [#8093](https://github.com/apache/arrow-datafusion/pull/8093) (tgujar) +- Combine `Expr::Wildcard` and `Wxpr::QualifiedWildcard`, add `wildcard()` expr fn [#8105](https://github.com/apache/arrow-datafusion/pull/8105) (alamb) +- docs: show creation of DFSchema [#8132](https://github.com/apache/arrow-datafusion/pull/8132) (wjones127) +- feat: support UDAF in substrait producer/consumer [#8119](https://github.com/apache/arrow-datafusion/pull/8119) (waynexia) +- Improve documentation site to make it easier to find communication on Slack/Discord [#8138](https://github.com/apache/arrow-datafusion/pull/8138) (alamb) diff --git a/dev/release/generate-changelog.py b/dev/release/generate-changelog.py index ff9e8d4754b2..f419bdb3a1ac 100755 --- a/dev/release/generate-changelog.py +++ b/dev/release/generate-changelog.py @@ -57,6 +57,7 @@ def generate_changelog(repo, repo_name, tag1, tag2): bugs = [] docs = [] enhancements = [] + performance = [] # categorize the pull requests based on GitHub labels print("Categorizing pull requests", file=sys.stderr) @@ -79,6 +80,8 @@ def generate_changelog(repo, repo_name, tag1, tag2): breaking.append((pull, commit)) elif 'bug' in labels or cc_type == 'fix': bugs.append((pull, commit)) + elif 'performance' in labels or cc_type == 'perf': + performance.append((pull, commit)) elif 'enhancement' in labels or cc_type == 'feat': enhancements.append((pull, commit)) elif 'documentation' in labels or cc_type == 'docs': @@ -87,6 +90,7 @@ def generate_changelog(repo, repo_name, tag1, tag2): # produce the changelog content print("Generating changelog content", file=sys.stderr) print_pulls(repo_name, "Breaking changes", breaking) + print_pulls(repo_name, "Performance related", performance) print_pulls(repo_name, "Implemented enhancements", enhancements) print_pulls(repo_name, "Fixed bugs", bugs) print_pulls(repo_name, "Documentation updates", docs) From d21a40c64e8227eb1ef44b40e9f97e22b9b9f838 Mon Sep 17 00:00:00 2001 From: Jay Zhan Date: Tue, 14 Nov 2023 02:27:44 +0800 Subject: [PATCH 242/572] Avoid concat in `array_append` (#8137) * clean array_append Signed-off-by: jayzhan211 * done Signed-off-by: jayzhan211 --------- Signed-off-by: jayzhan211 --- .../physical-expr/src/array_expressions.rs | 177 +++++++++--------- 1 file changed, 85 insertions(+), 92 deletions(-) diff --git a/datafusion/physical-expr/src/array_expressions.rs b/datafusion/physical-expr/src/array_expressions.rs index 54452e3653a8..73ef0ea6da9f 100644 --- a/datafusion/physical-expr/src/array_expressions.rs +++ b/datafusion/physical-expr/src/array_expressions.rs @@ -579,57 +579,85 @@ pub fn array_pop_back(args: &[ArrayRef]) -> Result { ) } +/// Appends or prepends elements to a ListArray. +/// +/// This function takes a ListArray, an ArrayRef, a FieldRef, and a boolean flag +/// indicating whether to append or prepend the elements. It returns a `Result` +/// representing the resulting ListArray after the operation. +/// +/// # Arguments +/// +/// * `list_array` - A reference to the ListArray to which elements will be appended/prepended. +/// * `element_array` - A reference to the Array containing elements to be appended/prepended. +/// * `field` - A reference to the Field describing the data type of the arrays. +/// * `is_append` - A boolean flag indicating whether to append (`true`) or prepend (`false`) elements. +/// +/// # Examples +/// +/// general_append_and_prepend( +/// [1, 2, 3], 4, append => [1, 2, 3, 4] +/// 5, [6, 7, 8], prepend => [5, 6, 7, 8] +/// ) +fn general_append_and_prepend( + list_array: &ListArray, + element_array: &ArrayRef, + data_type: &DataType, + is_append: bool, +) -> Result { + let mut offsets = vec![0]; + let values = list_array.values(); + let original_data = values.to_data(); + let element_data = element_array.to_data(); + let capacity = Capacities::Array(original_data.len() + element_data.len()); + + let mut mutable = MutableArrayData::with_capacities( + vec![&original_data, &element_data], + false, + capacity, + ); + + let values_index = 0; + let element_index = 1; + + for (row_index, offset_window) in list_array.offsets().windows(2).enumerate() { + let start = offset_window[0] as usize; + let end = offset_window[1] as usize; + if is_append { + mutable.extend(values_index, start, end); + mutable.extend(element_index, row_index, row_index + 1); + } else { + mutable.extend(element_index, row_index, row_index + 1); + mutable.extend(values_index, start, end); + } + offsets.push(offsets[row_index] + (end - start + 1) as i32); + } + + let data = mutable.freeze(); + + Ok(Arc::new(ListArray::try_new( + Arc::new(Field::new("item", data_type.to_owned(), true)), + OffsetBuffer::new(offsets.into()), + arrow_array::make_array(data), + None, + )?)) +} + /// Array_append SQL function pub fn array_append(args: &[ArrayRef]) -> Result { - let arr = as_list_array(&args[0])?; - let element = &args[1]; + let list_array = as_list_array(&args[0])?; + let element_array = &args[1]; - check_datatypes("array_append", &[arr.values(), element])?; - let res = match arr.value_type() { + check_datatypes("array_append", &[list_array.values(), element_array])?; + let res = match list_array.value_type() { DataType::List(_) => concat_internal(args)?, - DataType::Null => return make_array(&[element.to_owned()]), + DataType::Null => return make_array(&[element_array.to_owned()]), data_type => { - let mut new_values = vec![]; - let mut offsets = vec![0]; - - let elem_data = element.to_data(); - for (row_index, arr) in arr.iter().enumerate() { - let new_array = if let Some(arr) = arr { - let original_data = arr.to_data(); - let capacity = Capacities::Array(original_data.len() + 1); - let mut mutable = MutableArrayData::with_capacities( - vec![&original_data, &elem_data], - false, - capacity, - ); - mutable.extend(0, 0, original_data.len()); - mutable.extend(1, row_index, row_index + 1); - let data = mutable.freeze(); - arrow_array::make_array(data) - } else { - let capacity = Capacities::Array(1); - let mut mutable = MutableArrayData::with_capacities( - vec![&elem_data], - false, - capacity, - ); - mutable.extend(0, row_index, row_index + 1); - let data = mutable.freeze(); - arrow_array::make_array(data) - }; - offsets.push(offsets[row_index] + new_array.len() as i32); - new_values.push(new_array); - } - - let new_values: Vec<_> = new_values.iter().map(|a| a.as_ref()).collect(); - let values = arrow::compute::concat(&new_values)?; - - Arc::new(ListArray::try_new( - Arc::new(Field::new("item", data_type.to_owned(), true)), - OffsetBuffer::new(offsets.into()), - values, - None, - )?) + return general_append_and_prepend( + list_array, + element_array, + &data_type, + true, + ); } }; @@ -638,55 +666,20 @@ pub fn array_append(args: &[ArrayRef]) -> Result { /// Array_prepend SQL function pub fn array_prepend(args: &[ArrayRef]) -> Result { - let element = &args[0]; - let arr = as_list_array(&args[1])?; + let list_array = as_list_array(&args[1])?; + let element_array = &args[0]; - check_datatypes("array_prepend", &[element, arr.values()])?; - let res = match arr.value_type() { + check_datatypes("array_prepend", &[element_array, list_array.values()])?; + let res = match list_array.value_type() { DataType::List(_) => concat_internal(args)?, - DataType::Null => return make_array(&[element.to_owned()]), + DataType::Null => return make_array(&[element_array.to_owned()]), data_type => { - let mut new_values = vec![]; - let mut offsets = vec![0]; - - let elem_data = element.to_data(); - for (row_index, arr) in arr.iter().enumerate() { - let new_array = if let Some(arr) = arr { - let original_data = arr.to_data(); - let capacity = Capacities::Array(original_data.len() + 1); - let mut mutable = MutableArrayData::with_capacities( - vec![&original_data, &elem_data], - false, - capacity, - ); - mutable.extend(1, row_index, row_index + 1); - mutable.extend(0, 0, original_data.len()); - let data = mutable.freeze(); - arrow_array::make_array(data) - } else { - let capacity = Capacities::Array(1); - let mut mutable = MutableArrayData::with_capacities( - vec![&elem_data], - false, - capacity, - ); - mutable.extend(0, row_index, row_index + 1); - let data = mutable.freeze(); - arrow_array::make_array(data) - }; - offsets.push(offsets[row_index] + new_array.len() as i32); - new_values.push(new_array); - } - - let new_values: Vec<_> = new_values.iter().map(|a| a.as_ref()).collect(); - let values = arrow::compute::concat(&new_values)?; - - Arc::new(ListArray::try_new( - Arc::new(Field::new("item", data_type.to_owned(), true)), - OffsetBuffer::new(offsets.into()), - values, - None, - )?) + return general_append_and_prepend( + list_array, + element_array, + &data_type, + false, + ); } }; From 93a95775f51d21445958067b1da0991879464bb9 Mon Sep 17 00:00:00 2001 From: Jay Zhan Date: Tue, 14 Nov 2023 02:41:24 +0800 Subject: [PATCH 243/572] Replace macro with function for array_remove (#8106) * checkpoint Signed-off-by: jayzhan211 * done Signed-off-by: jayzhan211 * remove test and add null test Signed-off-by: jayzhan211 * fmt Signed-off-by: jayzhan211 * remove old code Signed-off-by: jayzhan211 * cleanup comment Signed-off-by: jayzhan211 * extend to large list and fix clippy Signed-off-by: jayzhan211 --------- Signed-off-by: jayzhan211 --- .../physical-expr/src/array_expressions.rs | 419 +++++------------- datafusion/sqllogictest/test_files/array.slt | 14 + 2 files changed, 135 insertions(+), 298 deletions(-) diff --git a/datafusion/physical-expr/src/array_expressions.rs b/datafusion/physical-expr/src/array_expressions.rs index 73ef0ea6da9f..60e09c5a9c90 100644 --- a/datafusion/physical-expr/src/array_expressions.rs +++ b/datafusion/physical-expr/src/array_expressions.rs @@ -1078,100 +1078,149 @@ pub fn array_positions(args: &[ArrayRef]) -> Result { Ok(res) } -macro_rules! general_remove { - ($ARRAY:expr, $ELEMENT:expr, $MAX:expr, $ARRAY_TYPE:ident) => {{ - let mut offsets: Vec = vec![0]; - let mut values = - downcast_arg!(new_empty_array($ELEMENT.data_type()), $ARRAY_TYPE).clone(); +/// For each element of `list_array[i]`, removed up to `arr_n[i]` occurences +/// of `element_array[i]`. +/// +/// The type of each **element** in `list_array` must be the same as the type of +/// `element_array`. This function also handles nested arrays +/// ([`ListArray`] of [`ListArray`]s) +/// +/// For example, when called to remove a list array (where each element is a +/// list of int32s, the second argument are int32 arrays, and the +/// third argument is the number of occurrences to remove +/// +/// ```text +/// general_remove( +/// [1, 2, 3, 2], 2, 1 ==> [1, 3, 2] (only the first 2 is removed) +/// [4, 5, 6, 5], 5, 2 ==> [4, 6] (both 5s are removed) +/// ) +/// ``` +fn general_remove( + list_array: &GenericListArray, + element_array: &ArrayRef, + arr_n: Vec, +) -> Result { + let data_type = list_array.value_type(); + let mut new_values = vec![]; + // Build up the offsets for the final output array + let mut offsets = Vec::::with_capacity(arr_n.len() + 1); + offsets.push(OffsetSize::zero()); - let element = downcast_arg!($ELEMENT, $ARRAY_TYPE); - for ((arr, el), max) in $ARRAY.iter().zip(element.iter()).zip($MAX.iter()) { - let last_offset: i32 = offsets.last().copied().ok_or_else(|| { - DataFusionError::Internal(format!("offsets should not be empty")) - })?; - match arr { - Some(arr) => { - let child_array = downcast_arg!(arr, $ARRAY_TYPE); - let mut counter = 0; - let max = if max < Some(1) { 1 } else { max.unwrap() }; + // n is the number of elements to remove in this row + for (row_index, (list_array_row, n)) in + list_array.iter().zip(arr_n.iter()).enumerate() + { + match list_array_row { + Some(list_array_row) => { + let indices = UInt32Array::from(vec![row_index as u32]); + let element_array_row = + arrow::compute::take(element_array, &indices, None)?; + + let eq_array = match element_array_row.data_type() { + // arrow_ord::cmp::distinct does not support ListArray, so we need to compare it by loop + DataType::List(_) => { + // compare each element of the from array + let element_array_row_inner = + as_list_array(&element_array_row)?.value(0); + let list_array_row_inner = as_list_array(&list_array_row)?; + + list_array_row_inner + .iter() + // compare element by element the current row of list_array + .map(|row| row.map(|row| row.ne(&element_array_row_inner))) + .collect::() + } + _ => { + let from_arr = Scalar::new(element_array_row); + // use distinct so Null = Null is false + arrow_ord::cmp::distinct(&list_array_row, &from_arr)? + } + }; - let filter_array = child_array + // We need to keep at most first n elements as `false`, which represent the elements to remove. + let eq_array = if eq_array.false_count() < *n as usize { + eq_array + } else { + let mut count = 0; + eq_array .iter() - .map(|element| { - if counter != max && element == el { - counter += 1; - Some(false) + .map(|e| { + // Keep first n `false` elements, and reverse other elements to `true`. + if let Some(false) = e { + if count < *n { + count += 1; + e + } else { + Some(true) + } } else { - Some(true) + e } }) - .collect::(); + .collect::() + }; - let filtered_array = compute::filter(&child_array, &filter_array)?; - values = downcast_arg!( - compute::concat(&[&values, &filtered_array,])?.clone(), - $ARRAY_TYPE - ) - .clone(); - offsets.push(last_offset + filtered_array.len() as i32); - } - None => offsets.push(last_offset), + let filtered_array = arrow::compute::filter(&list_array_row, &eq_array)?; + offsets.push( + offsets[row_index] + OffsetSize::usize_as(filtered_array.len()), + ); + new_values.push(filtered_array); + } + None => { + // Null element results in a null row (no new offsets) + offsets.push(offsets[row_index]); } } + } - let field = Arc::new(Field::new("item", $ELEMENT.data_type().clone(), true)); + let values = if new_values.is_empty() { + new_empty_array(&data_type) + } else { + let new_values = new_values.iter().map(|x| x.as_ref()).collect::>(); + arrow::compute::concat(&new_values)? + }; - Arc::new(ListArray::try_new( - field, - OffsetBuffer::new(offsets.into()), - Arc::new(values), - None, - )?) - }}; + Ok(Arc::new(GenericListArray::::try_new( + Arc::new(Field::new("item", data_type, true)), + OffsetBuffer::new(offsets.into()), + values, + list_array.nulls().cloned(), + )?)) } -macro_rules! array_removement_function { - ($FUNC:ident, $MAX_FUNC:expr, $DOC:expr) => { - #[doc = $DOC] - pub fn $FUNC(args: &[ArrayRef]) -> Result { - let arr = as_list_array(&args[0])?; - let element = &args[1]; - let max = $MAX_FUNC(args)?; - - check_datatypes(stringify!($FUNC), &[arr.values(), element])?; - macro_rules! array_function { - ($ARRAY_TYPE:ident) => { - general_remove!(arr, element, max, $ARRAY_TYPE) - }; - } - let res = call_array_function!(arr.value_type(), true); - - Ok(res) +fn array_remove_internal( + array: &ArrayRef, + element_array: &ArrayRef, + arr_n: Vec, +) -> Result { + match array.data_type() { + DataType::List(_) => { + let list_array = array.as_list::(); + general_remove::(list_array, element_array, arr_n) } - }; + DataType::LargeList(_) => { + let list_array = array.as_list::(); + general_remove::(list_array, element_array, arr_n) + } + _ => internal_err!("array_remove_all expects a list array"), + } } -fn remove_one(args: &[ArrayRef]) -> Result { - Ok(Int64Array::from_value(1, args[0].len())) +pub fn array_remove_all(args: &[ArrayRef]) -> Result { + let arr_n = vec![i64::MAX; args[0].len()]; + array_remove_internal(&args[0], &args[1], arr_n) } -fn remove_n(args: &[ArrayRef]) -> Result { - as_int64_array(&args[2]).cloned() +pub fn array_remove(args: &[ArrayRef]) -> Result { + let arr_n = vec![1; args[0].len()]; + array_remove_internal(&args[0], &args[1], arr_n) } -fn remove_all(args: &[ArrayRef]) -> Result { - Ok(Int64Array::from_value(i64::MAX, args[0].len())) +pub fn array_remove_n(args: &[ArrayRef]) -> Result { + let arr_n = as_int64_array(&args[2])?.values().to_vec(); + array_remove_internal(&args[0], &args[1], arr_n) } -// array removement functions -array_removement_function!(array_remove, remove_one, "Array_remove SQL function"); -array_removement_function!(array_remove_n, remove_n, "Array_remove_n SQL function"); -array_removement_function!( - array_remove_all, - remove_all, - "Array_remove_all SQL function" -); - /// For each element of `list_array[i]`, replaces up to `arr_n[i]` occurences /// of `from_array[i]`, `to_array[i]`. /// @@ -2601,173 +2650,6 @@ mod tests { ); } - #[test] - fn test_array_remove() { - // array_remove([3, 1, 2, 3, 2, 3], 3) = [1, 2, 3, 2, 3] - let list_array = return_array_with_repeating_elements(); - let array = array_remove(&[list_array, Arc::new(Int64Array::from_value(3, 1))]) - .expect("failed to initialize function array_remove"); - let result = - as_list_array(&array).expect("failed to initialize function array_remove"); - - assert_eq!(result.len(), 1); - assert_eq!( - &[1, 2, 3, 2, 3], - result - .value(0) - .as_any() - .downcast_ref::() - .unwrap() - .values() - ); - } - - #[test] - fn test_nested_array_remove() { - // array_remove( - // [[1, 2, 3, 4], [5, 6, 7, 8], [1, 2, 3, 4], [9, 10, 11, 12], [5, 6, 7, 8]], - // [1, 2, 3, 4], - // ) = [[5, 6, 7, 8], [1, 2, 3, 4], [9, 10, 11, 12], [5, 6, 7, 8]] - let list_array = return_nested_array_with_repeating_elements(); - let element_array = return_array(); - let array = array_remove(&[list_array, element_array]) - .expect("failed to initialize function array_remove"); - let result = - as_list_array(&array).expect("failed to initialize function array_remove"); - - assert_eq!(result.len(), 1); - let data = vec![ - Some(vec![Some(5), Some(6), Some(7), Some(8)]), - Some(vec![Some(1), Some(2), Some(3), Some(4)]), - Some(vec![Some(9), Some(10), Some(11), Some(12)]), - Some(vec![Some(5), Some(6), Some(7), Some(8)]), - ]; - let expected = ListArray::from_iter_primitive::(data); - assert_eq!( - expected, - result - .value(0) - .as_any() - .downcast_ref::() - .unwrap() - .clone() - ); - } - - #[test] - fn test_array_remove_n() { - // array_remove_n([3, 1, 2, 3, 2, 3], 3, 2) = [1, 2, 2, 3] - let list_array = return_array_with_repeating_elements(); - let array = array_remove_n(&[ - list_array, - Arc::new(Int64Array::from_value(3, 1)), - Arc::new(Int64Array::from_value(2, 1)), - ]) - .expect("failed to initialize function array_remove_n"); - let result = - as_list_array(&array).expect("failed to initialize function array_remove_n"); - - assert_eq!(result.len(), 1); - assert_eq!( - &[1, 2, 2, 3], - result - .value(0) - .as_any() - .downcast_ref::() - .unwrap() - .values() - ); - } - - #[test] - fn test_nested_array_remove_n() { - // array_remove_n( - // [[1, 2, 3, 4], [5, 6, 7, 8], [1, 2, 3, 4], [9, 10, 11, 12], [5, 6, 7, 8]], - // [1, 2, 3, 4], - // 3, - // ) = [[5, 6, 7, 8], [9, 10, 11, 12], [5, 6, 7, 8]] - let list_array = return_nested_array_with_repeating_elements(); - let element_array = return_array(); - let array = array_remove_n(&[ - list_array, - element_array, - Arc::new(Int64Array::from_value(3, 1)), - ]) - .expect("failed to initialize function array_remove_n"); - let result = - as_list_array(&array).expect("failed to initialize function array_remove_n"); - - assert_eq!(result.len(), 1); - let data = vec![ - Some(vec![Some(5), Some(6), Some(7), Some(8)]), - Some(vec![Some(9), Some(10), Some(11), Some(12)]), - Some(vec![Some(5), Some(6), Some(7), Some(8)]), - ]; - let expected = ListArray::from_iter_primitive::(data); - assert_eq!( - expected, - result - .value(0) - .as_any() - .downcast_ref::() - .unwrap() - .clone() - ); - } - - #[test] - fn test_array_remove_all() { - // array_remove_all([3, 1, 2, 3, 2, 3], 3) = [1, 2, 2] - let list_array = return_array_with_repeating_elements(); - let array = - array_remove_all(&[list_array, Arc::new(Int64Array::from_value(3, 1))]) - .expect("failed to initialize function array_remove_all"); - let result = as_list_array(&array) - .expect("failed to initialize function array_remove_all"); - - assert_eq!(result.len(), 1); - assert_eq!( - &[1, 2, 2], - result - .value(0) - .as_any() - .downcast_ref::() - .unwrap() - .values() - ); - } - - #[test] - fn test_nested_array_remove_all() { - // array_remove_all( - // [[1, 2, 3, 4], [5, 6, 7, 8], [1, 2, 3, 4], [9, 10, 11, 12], [5, 6, 7, 8]], - // [1, 2, 3, 4], - // ) = [[5, 6, 7, 8], [9, 10, 11, 12], [5, 6, 7, 8]] - let list_array = return_nested_array_with_repeating_elements(); - let element_array = return_array(); - let array = array_remove_all(&[list_array, element_array]) - .expect("failed to initialize function array_remove_all"); - let result = as_list_array(&array) - .expect("failed to initialize function array_remove_all"); - - assert_eq!(result.len(), 1); - let data = vec![ - Some(vec![Some(5), Some(6), Some(7), Some(8)]), - Some(vec![Some(9), Some(10), Some(11), Some(12)]), - Some(vec![Some(5), Some(6), Some(7), Some(8)]), - ]; - let expected = ListArray::from_iter_primitive::(data); - assert_eq!( - expected, - result - .value(0) - .as_any() - .downcast_ref::() - .unwrap() - .clone() - ); - } - #[test] fn test_array_to_string() { // array_to_string([1, 2, 3, 4], ',') = 1,2,3,4 @@ -3052,63 +2934,4 @@ mod tests { make_array(&[arr1, arr2]).expect("failed to initialize function array") } - - fn return_array_with_repeating_elements() -> ArrayRef { - // Returns: [3, 1, 2, 3, 2, 3] - let args = [ - Arc::new(Int64Array::from(vec![Some(3)])) as ArrayRef, - Arc::new(Int64Array::from(vec![Some(1)])) as ArrayRef, - Arc::new(Int64Array::from(vec![Some(2)])) as ArrayRef, - Arc::new(Int64Array::from(vec![Some(3)])) as ArrayRef, - Arc::new(Int64Array::from(vec![Some(2)])) as ArrayRef, - Arc::new(Int64Array::from(vec![Some(3)])) as ArrayRef, - ]; - make_array(&args).expect("failed to initialize function array") - } - - fn return_nested_array_with_repeating_elements() -> ArrayRef { - // Returns: [[1, 2, 3, 4], [5, 6, 7, 8], [1, 2, 3, 4], [9, 10, 11, 12], [5, 6, 7, 8]] - let args = [ - Arc::new(Int64Array::from(vec![Some(1)])) as ArrayRef, - Arc::new(Int64Array::from(vec![Some(2)])) as ArrayRef, - Arc::new(Int64Array::from(vec![Some(3)])) as ArrayRef, - Arc::new(Int64Array::from(vec![Some(4)])) as ArrayRef, - ]; - let arr1 = make_array(&args).expect("failed to initialize function array"); - - let args = [ - Arc::new(Int64Array::from(vec![Some(5)])) as ArrayRef, - Arc::new(Int64Array::from(vec![Some(6)])) as ArrayRef, - Arc::new(Int64Array::from(vec![Some(7)])) as ArrayRef, - Arc::new(Int64Array::from(vec![Some(8)])) as ArrayRef, - ]; - let arr2 = make_array(&args).expect("failed to initialize function array"); - - let args = [ - Arc::new(Int64Array::from(vec![Some(1)])) as ArrayRef, - Arc::new(Int64Array::from(vec![Some(2)])) as ArrayRef, - Arc::new(Int64Array::from(vec![Some(3)])) as ArrayRef, - Arc::new(Int64Array::from(vec![Some(4)])) as ArrayRef, - ]; - let arr3 = make_array(&args).expect("failed to initialize function array"); - - let args = [ - Arc::new(Int64Array::from(vec![Some(9)])) as ArrayRef, - Arc::new(Int64Array::from(vec![Some(10)])) as ArrayRef, - Arc::new(Int64Array::from(vec![Some(11)])) as ArrayRef, - Arc::new(Int64Array::from(vec![Some(12)])) as ArrayRef, - ]; - let arr4 = make_array(&args).expect("failed to initialize function array"); - - let args = [ - Arc::new(Int64Array::from(vec![Some(5)])) as ArrayRef, - Arc::new(Int64Array::from(vec![Some(6)])) as ArrayRef, - Arc::new(Int64Array::from(vec![Some(7)])) as ArrayRef, - Arc::new(Int64Array::from(vec![Some(8)])) as ArrayRef, - ]; - let arr5 = make_array(&args).expect("failed to initialize function array"); - - make_array(&[arr1, arr2, arr3, arr4, arr5]) - .expect("failed to initialize function array") - } } diff --git a/datafusion/sqllogictest/test_files/array.slt b/datafusion/sqllogictest/test_files/array.slt index ad81f37e0764..9207f0f0e359 100644 --- a/datafusion/sqllogictest/test_files/array.slt +++ b/datafusion/sqllogictest/test_files/array.slt @@ -2039,6 +2039,20 @@ select array_remove(make_array(1, 2, 2, 1, 1), 2), array_remove(make_array(1.0, ---- [1, 2, 1, 1] [2.0, 2.0, 1.0, 1.0] [h, e, l, o] +query ??? +select + array_remove(make_array(1, null, 2, 3), 2), + array_remove(make_array(1.1, null, 2.2, 3.3), 1.1), + array_remove(make_array('a', null, 'bc'), 'a'); +---- +[1, , 3] [, 2.2, 3.3] [, bc] + +# TODO: https://github.com/apache/arrow-datafusion/issues/7142 +# query +# select +# array_remove(make_array(1, null, 2), null), +# array_remove(make_array(1, null, 2, null), null); + # array_remove scalar function #2 (element is list) query ?? select array_remove(make_array([1, 2, 3], [4, 5, 6], [5, 5, 5], [4, 5, 6], [7, 8, 9]), [4, 5, 6]), array_remove(make_array([1, 3, 2], [2, 3, 4], [2, 3, 4], [5, 3, 1], [1, 3, 2]), [2, 3, 4]); From cbb2fd784655b24424cb811a5e215a522d1961b9 Mon Sep 17 00:00:00 2001 From: Edmondo Porcu Date: Mon, 13 Nov 2023 11:02:30 -0800 Subject: [PATCH 244/572] Implement `array_union` (#7897) * Initial implementation of array union without deduplication * Update datafusion/physical-expr/src/array_expressions.rs Co-authored-by: comphead * Update docs/source/user-guide/expressions.md Co-authored-by: comphead * Row based implementation of array_union * Added asymmetrical test * Addressing PR comments * Implementing code review feedback * Added script * Added tests for array * Additional tests * Removing spurious import from array_intersect --------- Co-authored-by: comphead --- datafusion/expr/src/built_in_function.rs | 6 ++ datafusion/expr/src/expr_fn.rs | 2 + .../physical-expr/src/array_expressions.rs | 83 +++++++++++++++- datafusion/physical-expr/src/functions.rs | 4 +- datafusion/proto/proto/datafusion.proto | 1 + datafusion/proto/src/generated/pbjson.rs | 3 + datafusion/proto/src/generated/prost.rs | 3 + .../proto/src/logical_plan/from_proto.rs | 7 ++ datafusion/proto/src/logical_plan/to_proto.rs | 1 + datafusion/sqllogictest/test_files/array.slt | 95 +++++++++++++++++++ docs/source/user-guide/expressions.md | 1 + .../source/user-guide/sql/scalar_functions.md | 38 ++++++++ 12 files changed, 242 insertions(+), 2 deletions(-) diff --git a/datafusion/expr/src/built_in_function.rs b/datafusion/expr/src/built_in_function.rs index ca3ca18e4d77..0d2c1f2e3cb7 100644 --- a/datafusion/expr/src/built_in_function.rs +++ b/datafusion/expr/src/built_in_function.rs @@ -176,6 +176,8 @@ pub enum BuiltinScalarFunction { ArrayToString, /// array_intersect ArrayIntersect, + /// array_union + ArrayUnion, /// cardinality Cardinality, /// construct an array from columns @@ -401,6 +403,7 @@ impl BuiltinScalarFunction { BuiltinScalarFunction::ArraySlice => Volatility::Immutable, BuiltinScalarFunction::ArrayToString => Volatility::Immutable, BuiltinScalarFunction::ArrayIntersect => Volatility::Immutable, + BuiltinScalarFunction::ArrayUnion => Volatility::Immutable, BuiltinScalarFunction::Cardinality => Volatility::Immutable, BuiltinScalarFunction::MakeArray => Volatility::Immutable, BuiltinScalarFunction::Ascii => Volatility::Immutable, @@ -581,6 +584,7 @@ impl BuiltinScalarFunction { BuiltinScalarFunction::ArraySlice => Ok(input_expr_types[0].clone()), BuiltinScalarFunction::ArrayToString => Ok(Utf8), BuiltinScalarFunction::ArrayIntersect => Ok(input_expr_types[0].clone()), + BuiltinScalarFunction::ArrayUnion => Ok(input_expr_types[0].clone()), BuiltinScalarFunction::Cardinality => Ok(UInt64), BuiltinScalarFunction::MakeArray => match input_expr_types.len() { 0 => Ok(List(Arc::new(Field::new("item", Null, true)))), @@ -885,6 +889,7 @@ impl BuiltinScalarFunction { Signature::variadic_any(self.volatility()) } BuiltinScalarFunction::ArrayIntersect => Signature::any(2, self.volatility()), + BuiltinScalarFunction::ArrayUnion => Signature::any(2, self.volatility()), BuiltinScalarFunction::Cardinality => Signature::any(1, self.volatility()), BuiltinScalarFunction::MakeArray => { // 0 or more arguments of arbitrary type @@ -1508,6 +1513,7 @@ fn aliases(func: &BuiltinScalarFunction) -> &'static [&'static str] { "array_join", "list_join", ], + BuiltinScalarFunction::ArrayUnion => &["array_union", "list_union"], BuiltinScalarFunction::Cardinality => &["cardinality"], BuiltinScalarFunction::MakeArray => &["make_array", "make_list"], BuiltinScalarFunction::ArrayIntersect => &["array_intersect", "list_intersect"], diff --git a/datafusion/expr/src/expr_fn.rs b/datafusion/expr/src/expr_fn.rs index 0e0ad46da101..0d920beb416f 100644 --- a/datafusion/expr/src/expr_fn.rs +++ b/datafusion/expr/src/expr_fn.rs @@ -717,6 +717,8 @@ scalar_expr!( array delimiter, "converts each element to its text representation." ); +scalar_expr!(ArrayUnion, array_union, array1 array2, "returns an array of the elements in the union of array1 and array2 without duplicates."); + scalar_expr!( Cardinality, cardinality, diff --git a/datafusion/physical-expr/src/array_expressions.rs b/datafusion/physical-expr/src/array_expressions.rs index 60e09c5a9c90..9b074ff0ee0d 100644 --- a/datafusion/physical-expr/src/array_expressions.rs +++ b/datafusion/physical-expr/src/array_expressions.rs @@ -27,6 +27,7 @@ use arrow::datatypes::{DataType, Field, UInt64Type}; use arrow::row::{RowConverter, SortField}; use arrow_buffer::NullBuffer; +use arrow_schema::FieldRef; use datafusion_common::cast::{ as_generic_string_array, as_int64_array, as_list_array, as_string_array, }; @@ -36,8 +37,8 @@ use datafusion_common::{ DataFusionError, Result, }; -use hashbrown::HashSet; use itertools::Itertools; +use std::collections::HashSet; macro_rules! downcast_arg { ($ARG:expr, $ARRAY_TYPE:ident) => {{ @@ -1382,6 +1383,86 @@ macro_rules! to_string { }}; } +fn union_generic_lists( + l: &GenericListArray, + r: &GenericListArray, + field: &FieldRef, +) -> Result> { + let converter = RowConverter::new(vec![SortField::new(l.value_type().clone())])?; + + let nulls = NullBuffer::union(l.nulls(), r.nulls()); + let l_values = l.values().clone(); + let r_values = r.values().clone(); + let l_values = converter.convert_columns(&[l_values])?; + let r_values = converter.convert_columns(&[r_values])?; + + // Might be worth adding an upstream OffsetBufferBuilder + let mut offsets = Vec::::with_capacity(l.len() + 1); + offsets.push(OffsetSize::usize_as(0)); + let mut rows = Vec::with_capacity(l_values.num_rows() + r_values.num_rows()); + let mut dedup = HashSet::new(); + for (l_w, r_w) in l.offsets().windows(2).zip(r.offsets().windows(2)) { + let l_slice = l_w[0].as_usize()..l_w[1].as_usize(); + let r_slice = r_w[0].as_usize()..r_w[1].as_usize(); + for i in l_slice { + let left_row = l_values.row(i); + if dedup.insert(left_row) { + rows.push(left_row); + } + } + for i in r_slice { + let right_row = r_values.row(i); + if dedup.insert(right_row) { + rows.push(right_row); + } + } + offsets.push(OffsetSize::usize_as(rows.len())); + dedup.clear(); + } + + let values = converter.convert_rows(rows)?; + let offsets = OffsetBuffer::new(offsets.into()); + let result = values[0].clone(); + Ok(GenericListArray::::new( + field.clone(), + offsets, + result, + nulls, + )) +} + +/// Array_union SQL function +pub fn array_union(args: &[ArrayRef]) -> Result { + if args.len() != 2 { + return exec_err!("array_union needs two arguments"); + } + let array1 = &args[0]; + let array2 = &args[1]; + match (array1.data_type(), array2.data_type()) { + (DataType::Null, _) => Ok(array2.clone()), + (_, DataType::Null) => Ok(array1.clone()), + (DataType::List(field_ref), DataType::List(_)) => { + check_datatypes("array_union", &[&array1, &array2])?; + let list1 = array1.as_list::(); + let list2 = array2.as_list::(); + let result = union_generic_lists::(list1, list2, field_ref)?; + Ok(Arc::new(result)) + } + (DataType::LargeList(field_ref), DataType::LargeList(_)) => { + check_datatypes("array_union", &[&array1, &array2])?; + let list1 = array1.as_list::(); + let list2 = array2.as_list::(); + let result = union_generic_lists::(list1, list2, field_ref)?; + Ok(Arc::new(result)) + } + _ => { + internal_err!( + "array_union only support list with offsets of type int32 and int64" + ) + } + } +} + /// Array_to_string SQL function pub fn array_to_string(args: &[ArrayRef]) -> Result { let arr = &args[0]; diff --git a/datafusion/physical-expr/src/functions.rs b/datafusion/physical-expr/src/functions.rs index 9185ade313eb..80c0eaf054fd 100644 --- a/datafusion/physical-expr/src/functions.rs +++ b/datafusion/physical-expr/src/functions.rs @@ -407,7 +407,9 @@ pub fn create_physical_fun( BuiltinScalarFunction::MakeArray => { Arc::new(|args| make_scalar_function(array_expressions::make_array)(args)) } - + BuiltinScalarFunction::ArrayUnion => { + Arc::new(|args| make_scalar_function(array_expressions::array_union)(args)) + } // struct functions BuiltinScalarFunction::Struct => Arc::new(struct_expressions::struct_expr), diff --git a/datafusion/proto/proto/datafusion.proto b/datafusion/proto/proto/datafusion.proto index 62b226e33339..793378a1ea87 100644 --- a/datafusion/proto/proto/datafusion.proto +++ b/datafusion/proto/proto/datafusion.proto @@ -635,6 +635,7 @@ enum ScalarFunction { StringToArray = 117; ToTimestampNanos = 118; ArrayIntersect = 119; + ArrayUnion = 120; } message ScalarFunctionNode { diff --git a/datafusion/proto/src/generated/pbjson.rs b/datafusion/proto/src/generated/pbjson.rs index 7602e1a36657..a78da2a51c9d 100644 --- a/datafusion/proto/src/generated/pbjson.rs +++ b/datafusion/proto/src/generated/pbjson.rs @@ -20908,6 +20908,7 @@ impl serde::Serialize for ScalarFunction { Self::StringToArray => "StringToArray", Self::ToTimestampNanos => "ToTimestampNanos", Self::ArrayIntersect => "ArrayIntersect", + Self::ArrayUnion => "ArrayUnion", }; serializer.serialize_str(variant) } @@ -21039,6 +21040,7 @@ impl<'de> serde::Deserialize<'de> for ScalarFunction { "StringToArray", "ToTimestampNanos", "ArrayIntersect", + "ArrayUnion", ]; struct GeneratedVisitor; @@ -21199,6 +21201,7 @@ impl<'de> serde::Deserialize<'de> for ScalarFunction { "StringToArray" => Ok(ScalarFunction::StringToArray), "ToTimestampNanos" => Ok(ScalarFunction::ToTimestampNanos), "ArrayIntersect" => Ok(ScalarFunction::ArrayIntersect), + "ArrayUnion" => Ok(ScalarFunction::ArrayUnion), _ => Err(serde::de::Error::unknown_variant(value, FIELDS)), } } diff --git a/datafusion/proto/src/generated/prost.rs b/datafusion/proto/src/generated/prost.rs index 825481a18822..7b7b0afb9216 100644 --- a/datafusion/proto/src/generated/prost.rs +++ b/datafusion/proto/src/generated/prost.rs @@ -2562,6 +2562,7 @@ pub enum ScalarFunction { StringToArray = 117, ToTimestampNanos = 118, ArrayIntersect = 119, + ArrayUnion = 120, } impl ScalarFunction { /// String value of the enum field names used in the ProtoBuf definition. @@ -2690,6 +2691,7 @@ impl ScalarFunction { ScalarFunction::StringToArray => "StringToArray", ScalarFunction::ToTimestampNanos => "ToTimestampNanos", ScalarFunction::ArrayIntersect => "ArrayIntersect", + ScalarFunction::ArrayUnion => "ArrayUnion", } } /// Creates an enum from field names used in the ProtoBuf definition. @@ -2815,6 +2817,7 @@ impl ScalarFunction { "StringToArray" => Some(Self::StringToArray), "ToTimestampNanos" => Some(Self::ToTimestampNanos), "ArrayIntersect" => Some(Self::ArrayIntersect), + "ArrayUnion" => Some(Self::ArrayUnion), _ => None, } } diff --git a/datafusion/proto/src/logical_plan/from_proto.rs b/datafusion/proto/src/logical_plan/from_proto.rs index 674492edef43..f7e38757e923 100644 --- a/datafusion/proto/src/logical_plan/from_proto.rs +++ b/datafusion/proto/src/logical_plan/from_proto.rs @@ -484,6 +484,7 @@ impl From<&protobuf::ScalarFunction> for BuiltinScalarFunction { ScalarFunction::ArraySlice => Self::ArraySlice, ScalarFunction::ArrayToString => Self::ArrayToString, ScalarFunction::ArrayIntersect => Self::ArrayIntersect, + ScalarFunction::ArrayUnion => Self::ArrayUnion, ScalarFunction::Cardinality => Self::Cardinality, ScalarFunction::Array => Self::MakeArray, ScalarFunction::NullIf => Self::NullIf, @@ -1424,6 +1425,12 @@ pub fn parse_expr( ScalarFunction::ArrayNdims => { Ok(array_ndims(parse_expr(&args[0], registry)?)) } + ScalarFunction::ArrayUnion => Ok(array( + args.to_owned() + .iter() + .map(|expr| parse_expr(expr, registry)) + .collect::, _>>()?, + )), ScalarFunction::Sqrt => Ok(sqrt(parse_expr(&args[0], registry)?)), ScalarFunction::Cbrt => Ok(cbrt(parse_expr(&args[0], registry)?)), ScalarFunction::Sin => Ok(sin(parse_expr(&args[0], registry)?)), diff --git a/datafusion/proto/src/logical_plan/to_proto.rs b/datafusion/proto/src/logical_plan/to_proto.rs index 946f2c6964a5..2bb7f89c7d4d 100644 --- a/datafusion/proto/src/logical_plan/to_proto.rs +++ b/datafusion/proto/src/logical_plan/to_proto.rs @@ -1487,6 +1487,7 @@ impl TryFrom<&BuiltinScalarFunction> for protobuf::ScalarFunction { BuiltinScalarFunction::ArraySlice => Self::ArraySlice, BuiltinScalarFunction::ArrayToString => Self::ArrayToString, BuiltinScalarFunction::ArrayIntersect => Self::ArrayIntersect, + BuiltinScalarFunction::ArrayUnion => Self::ArrayUnion, BuiltinScalarFunction::Cardinality => Self::Cardinality, BuiltinScalarFunction::MakeArray => Self::Array, BuiltinScalarFunction::NullIf => Self::NullIf, diff --git a/datafusion/sqllogictest/test_files/array.slt b/datafusion/sqllogictest/test_files/array.slt index 9207f0f0e359..54741afdf83a 100644 --- a/datafusion/sqllogictest/test_files/array.slt +++ b/datafusion/sqllogictest/test_files/array.slt @@ -1919,6 +1919,101 @@ select array_to_string(make_array(), ',') ---- (empty) + +## array_union (aliases: `list_union`) + +# array_union scalar function #1 +query ? +select array_union([1, 2, 3, 4], [5, 6, 3, 4]); +---- +[1, 2, 3, 4, 5, 6] + +# array_union scalar function #2 +query ? +select array_union([1, 2, 3, 4], [5, 6, 7, 8]); +---- +[1, 2, 3, 4, 5, 6, 7, 8] + +# array_union scalar function #3 +query ? +select array_union([1,2,3], []); +---- +[1, 2, 3] + +# array_union scalar function #4 +query ? +select array_union([1, 2, 3, 4], [5, 4]); +---- +[1, 2, 3, 4, 5] + +# array_union scalar function #5 +statement ok +CREATE TABLE arrays_with_repeating_elements_for_union +AS VALUES + ([1], [2]), + ([2, 3], [3]), + ([3], [3, 4]) +; + +query ? +select array_union(column1, column2) from arrays_with_repeating_elements_for_union; +---- +[1, 2] +[2, 3] +[3, 4] + +statement ok +drop table arrays_with_repeating_elements_for_union; + +# array_union scalar function #6 +query ? +select array_union([], []); +---- +NULL + +# array_union scalar function #7 +query ? +select array_union([[null]], []); +---- +[[]] + +# array_union scalar function #8 +query ? +select array_union([null], [null]); +---- +[] + +# array_union scalar function #9 +query ? +select array_union(null, []); +---- +NULL + +# array_union scalar function #10 +query ? +select array_union(null, null); +---- +NULL + +# array_union scalar function #11 +query ? +select array_union([1.2, 3.0], [1.2, 3.0, 5.7]); +---- +[1.2, 3.0, 5.7] + +# array_union scalar function #12 +query ? +select array_union(['hello'], ['hello','datafusion']); +---- +[hello, datafusion] + + + + + + + + # list_to_string scalar function #4 (function alias `array_to_string`) query TTT select list_to_string(['h', 'e', 'l', 'l', 'o'], ','), list_to_string([1, 2, 3, 4, 5], '-'), list_to_string([1.0, 2.0, 3.0], '|'); diff --git a/docs/source/user-guide/expressions.md b/docs/source/user-guide/expressions.md index 27384dccffe0..bec3ba9bb28c 100644 --- a/docs/source/user-guide/expressions.md +++ b/docs/source/user-guide/expressions.md @@ -233,6 +233,7 @@ Unlike to some databases the math functions in Datafusion works the same way as | array_slice(array, index) | Returns a slice of the array. `array_slice([1, 2, 3, 4, 5, 6, 7, 8], 3, 6) -> [3, 4, 5, 6]` | | array_to_string(array, delimiter) | Converts each element to its text representation. `array_to_string([1, 2, 3, 4], ',') -> 1,2,3,4` | | array_intersect(array1, array2) | Returns an array of the elements in the intersection of array1 and array2. `array_intersect([1, 2, 3, 4], [5, 6, 3, 4]) -> [3, 4]` | +| array_union(array1, array2) | Returns an array of the elements in the union of array1 and array2 without duplicates. `array_union([1, 2, 3, 4], [5, 6, 3, 4]) -> [1, 2, 3, 4, 5, 6]` | | cardinality(array) | Returns the total number of elements in the array. `cardinality([[1, 2, 3], [4, 5, 6]]) -> 6` | | make_array(value1, [value2 [, ...]]) | Returns an Arrow array using the specified input expressions. `make_array(1, 2, 3) -> [1, 2, 3]` | | trim_array(array, n) | Deprecated | diff --git a/docs/source/user-guide/sql/scalar_functions.md b/docs/source/user-guide/sql/scalar_functions.md index be05084fb249..2959e8202437 100644 --- a/docs/source/user-guide/sql/scalar_functions.md +++ b/docs/source/user-guide/sql/scalar_functions.md @@ -2211,6 +2211,44 @@ array_to_string(array, delimiter) - list_join - list_to_string +### `array_union` + +Returns an array of elements that are present in both arrays (all elements from both arrays) with out duplicates. + +``` +array_union(array1, array2) +``` + +#### Arguments + +- **array1**: Array expression. + Can be a constant, column, or function, and any combination of array operators. +- **array2**: Array expression. + Can be a constant, column, or function, and any combination of array operators. + +#### Example + +``` +❯ select array_union([1, 2, 3, 4], [5, 6, 3, 4]); ++----------------------------------------------------+ +| array_union([1, 2, 3, 4], [5, 6, 3, 4]); | ++----------------------------------------------------+ +| [1, 2, 3, 4, 5, 6] | ++----------------------------------------------------+ +❯ select array_union([1, 2, 3, 4], [5, 6, 7, 8]); ++----------------------------------------------------+ +| array_union([1, 2, 3, 4], [5, 6, 7, 8]); | ++----------------------------------------------------+ +| [1, 2, 3, 4, 5, 6] | ++----------------------------------------------------+ +``` + +--- + +#### Aliases + +- list_union + ### `cardinality` Returns the total number of elements in the array. From 4fb4b216fb02349133f4204cf600004342e2e5c5 Mon Sep 17 00:00:00 2001 From: Andrew Lamb Date: Mon, 13 Nov 2023 14:04:37 -0500 Subject: [PATCH 245/572] Minor: Document `ExecutionPlan::equivalence_properties` more thoroughly (#8128) * Minor: Document ExecutionPlan::equivalence_properties more thoroughly * Apply suggestions from code review Co-authored-by: Liang-Chi Hsieh Co-authored-by: Mustafa Akur <106137913+mustafasrepo@users.noreply.github.com> --------- Co-authored-by: Liang-Chi Hsieh Co-authored-by: Mustafa Akur <106137913+mustafasrepo@users.noreply.github.com> --- datafusion/physical-plan/src/lib.rs | 18 +++++++++++++++++- 1 file changed, 17 insertions(+), 1 deletion(-) diff --git a/datafusion/physical-plan/src/lib.rs b/datafusion/physical-plan/src/lib.rs index 9519f6a5a1dd..e5cd5e674cb1 100644 --- a/datafusion/physical-plan/src/lib.rs +++ b/datafusion/physical-plan/src/lib.rs @@ -203,7 +203,23 @@ pub trait ExecutionPlan: Debug + DisplayAs + Send + Sync { .collect() } - /// Get the [`EquivalenceProperties`] within the plan + /// Get the [`EquivalenceProperties`] within the plan. + /// + /// Equivalence properties tell DataFusion what columns are known to be + /// equal, during various optimization passes. By default, this returns "no + /// known equivalences" which is always correct, but may cause DataFusion to + /// unnecessarily resort data. + /// + /// If this ExecutionPlan makes no changes to the schema of the rows flowing + /// through it or how columns within each row relate to each other, it + /// should return the equivalence properties of its input. For + /// example, since `FilterExec` may remove rows from its input, but does not + /// otherwise modify them, it preserves its input equivalence properties. + /// However, since `ProjectionExec` may calculate derived expressions, it + /// needs special handling. + /// + /// See also [`Self::maintains_input_order`] and [`Self::output_ordering`] + /// for related concepts. fn equivalence_properties(&self) -> EquivalenceProperties { EquivalenceProperties::new(self.schema()) } From a38ac202f8db5bb7e184b57a59875b05990ee7c3 Mon Sep 17 00:00:00 2001 From: Nga Tran Date: Mon, 13 Nov 2023 17:33:25 -0500 Subject: [PATCH 246/572] feat: show statistics in explain verbose (#8113) * feat: show statistics in explain verbose * chore: address review comments * chore: address review comments * fix: add new enum types in pbjson * fix: add new types into message proto * Update explain plan --------- Co-authored-by: Andrew Lamb --- datafusion/common/src/display/mod.rs | 8 +++ datafusion/core/src/physical_planner.rs | 38 ++++++++-- datafusion/proto/proto/datafusion.proto | 2 + datafusion/proto/src/generated/pbjson.rs | 26 +++++++ datafusion/proto/src/generated/prost.rs | 6 +- .../proto/src/logical_plan/from_proto.rs | 5 +- datafusion/proto/src/logical_plan/to_proto.rs | 9 ++- .../sqllogictest/test_files/explain.slt | 70 ++++++++++++++++++- 8 files changed, 154 insertions(+), 10 deletions(-) diff --git a/datafusion/common/src/display/mod.rs b/datafusion/common/src/display/mod.rs index 766b37ce2891..4d1d48bf9fcc 100644 --- a/datafusion/common/src/display/mod.rs +++ b/datafusion/common/src/display/mod.rs @@ -47,6 +47,8 @@ pub enum PlanType { FinalLogicalPlan, /// The initial physical plan, prepared for execution InitialPhysicalPlan, + /// The initial physical plan with stats, prepared for execution + InitialPhysicalPlanWithStats, /// The ExecutionPlan which results from applying an optimizer pass OptimizedPhysicalPlan { /// The name of the optimizer which produced this plan @@ -54,6 +56,8 @@ pub enum PlanType { }, /// The final, fully optimized physical which would be executed FinalPhysicalPlan, + /// The final with stats, fully optimized physical which would be executed + FinalPhysicalPlanWithStats, } impl Display for PlanType { @@ -69,10 +73,14 @@ impl Display for PlanType { } PlanType::FinalLogicalPlan => write!(f, "logical_plan"), PlanType::InitialPhysicalPlan => write!(f, "initial_physical_plan"), + PlanType::InitialPhysicalPlanWithStats => { + write!(f, "initial_physical_plan_with_stats") + } PlanType::OptimizedPhysicalPlan { optimizer_name } => { write!(f, "physical_plan after {optimizer_name}") } PlanType::FinalPhysicalPlan => write!(f, "physical_plan"), + PlanType::FinalPhysicalPlanWithStats => write!(f, "physical_plan_with_stats"), } } } diff --git a/datafusion/core/src/physical_planner.rs b/datafusion/core/src/physical_planner.rs index 9f9b529ace03..9c1d978acc24 100644 --- a/datafusion/core/src/physical_planner.rs +++ b/datafusion/core/src/physical_planner.rs @@ -1893,12 +1893,25 @@ impl DefaultPhysicalPlanner { .await { Ok(input) => { + // This plan will includes statistics if show_statistics is on stringified_plans.push( displayable(input.as_ref()) .set_show_statistics(config.show_statistics) .to_stringified(e.verbose, InitialPhysicalPlan), ); + // If the show_statisitcs is off, add another line to show statsitics in the case of explain verbose + if e.verbose && !config.show_statistics { + stringified_plans.push( + displayable(input.as_ref()) + .set_show_statistics(true) + .to_stringified( + e.verbose, + InitialPhysicalPlanWithStats, + ), + ); + } + match self.optimize_internal( input, session_state, @@ -1912,11 +1925,26 @@ impl DefaultPhysicalPlanner { ); }, ) { - Ok(input) => stringified_plans.push( - displayable(input.as_ref()) - .set_show_statistics(config.show_statistics) - .to_stringified(e.verbose, FinalPhysicalPlan), - ), + Ok(input) => { + // This plan will includes statistics if show_statistics is on + stringified_plans.push( + displayable(input.as_ref()) + .set_show_statistics(config.show_statistics) + .to_stringified(e.verbose, FinalPhysicalPlan), + ); + + // If the show_statisitcs is off, add another line to show statsitics in the case of explain verbose + if e.verbose && !config.show_statistics { + stringified_plans.push( + displayable(input.as_ref()) + .set_show_statistics(true) + .to_stringified( + e.verbose, + FinalPhysicalPlanWithStats, + ), + ); + } + } Err(DataFusionError::Context(optimizer_name, e)) => { let plan_type = OptimizedPhysicalPlan { optimizer_name }; stringified_plans diff --git a/datafusion/proto/proto/datafusion.proto b/datafusion/proto/proto/datafusion.proto index 793378a1ea87..5d7c570bc173 100644 --- a/datafusion/proto/proto/datafusion.proto +++ b/datafusion/proto/proto/datafusion.proto @@ -1085,8 +1085,10 @@ message PlanType { OptimizedLogicalPlanType OptimizedLogicalPlan = 2; EmptyMessage FinalLogicalPlan = 3; EmptyMessage InitialPhysicalPlan = 4; + EmptyMessage InitialPhysicalPlanWithStats = 9; OptimizedPhysicalPlanType OptimizedPhysicalPlan = 5; EmptyMessage FinalPhysicalPlan = 6; + EmptyMessage FinalPhysicalPlanWithStats = 10; } } diff --git a/datafusion/proto/src/generated/pbjson.rs b/datafusion/proto/src/generated/pbjson.rs index a78da2a51c9d..12fa73205d49 100644 --- a/datafusion/proto/src/generated/pbjson.rs +++ b/datafusion/proto/src/generated/pbjson.rs @@ -19295,12 +19295,18 @@ impl serde::Serialize for PlanType { plan_type::PlanTypeEnum::InitialPhysicalPlan(v) => { struct_ser.serialize_field("InitialPhysicalPlan", v)?; } + plan_type::PlanTypeEnum::InitialPhysicalPlanWithStats(v) => { + struct_ser.serialize_field("InitialPhysicalPlanWithStats", v)?; + } plan_type::PlanTypeEnum::OptimizedPhysicalPlan(v) => { struct_ser.serialize_field("OptimizedPhysicalPlan", v)?; } plan_type::PlanTypeEnum::FinalPhysicalPlan(v) => { struct_ser.serialize_field("FinalPhysicalPlan", v)?; } + plan_type::PlanTypeEnum::FinalPhysicalPlanWithStats(v) => { + struct_ser.serialize_field("FinalPhysicalPlanWithStats", v)?; + } } } struct_ser.end() @@ -19319,8 +19325,10 @@ impl<'de> serde::Deserialize<'de> for PlanType { "OptimizedLogicalPlan", "FinalLogicalPlan", "InitialPhysicalPlan", + "InitialPhysicalPlanWithStats", "OptimizedPhysicalPlan", "FinalPhysicalPlan", + "FinalPhysicalPlanWithStats", ]; #[allow(clippy::enum_variant_names)] @@ -19331,8 +19339,10 @@ impl<'de> serde::Deserialize<'de> for PlanType { OptimizedLogicalPlan, FinalLogicalPlan, InitialPhysicalPlan, + InitialPhysicalPlanWithStats, OptimizedPhysicalPlan, FinalPhysicalPlan, + FinalPhysicalPlanWithStats, } impl<'de> serde::Deserialize<'de> for GeneratedField { fn deserialize(deserializer: D) -> std::result::Result @@ -19360,8 +19370,10 @@ impl<'de> serde::Deserialize<'de> for PlanType { "OptimizedLogicalPlan" => Ok(GeneratedField::OptimizedLogicalPlan), "FinalLogicalPlan" => Ok(GeneratedField::FinalLogicalPlan), "InitialPhysicalPlan" => Ok(GeneratedField::InitialPhysicalPlan), + "InitialPhysicalPlanWithStats" => Ok(GeneratedField::InitialPhysicalPlanWithStats), "OptimizedPhysicalPlan" => Ok(GeneratedField::OptimizedPhysicalPlan), "FinalPhysicalPlan" => Ok(GeneratedField::FinalPhysicalPlan), + "FinalPhysicalPlanWithStats" => Ok(GeneratedField::FinalPhysicalPlanWithStats), _ => Err(serde::de::Error::unknown_field(value, FIELDS)), } } @@ -19424,6 +19436,13 @@ impl<'de> serde::Deserialize<'de> for PlanType { return Err(serde::de::Error::duplicate_field("InitialPhysicalPlan")); } plan_type_enum__ = map_.next_value::<::std::option::Option<_>>()?.map(plan_type::PlanTypeEnum::InitialPhysicalPlan) +; + } + GeneratedField::InitialPhysicalPlanWithStats => { + if plan_type_enum__.is_some() { + return Err(serde::de::Error::duplicate_field("InitialPhysicalPlanWithStats")); + } + plan_type_enum__ = map_.next_value::<::std::option::Option<_>>()?.map(plan_type::PlanTypeEnum::InitialPhysicalPlanWithStats) ; } GeneratedField::OptimizedPhysicalPlan => { @@ -19438,6 +19457,13 @@ impl<'de> serde::Deserialize<'de> for PlanType { return Err(serde::de::Error::duplicate_field("FinalPhysicalPlan")); } plan_type_enum__ = map_.next_value::<::std::option::Option<_>>()?.map(plan_type::PlanTypeEnum::FinalPhysicalPlan) +; + } + GeneratedField::FinalPhysicalPlanWithStats => { + if plan_type_enum__.is_some() { + return Err(serde::de::Error::duplicate_field("FinalPhysicalPlanWithStats")); + } + plan_type_enum__ = map_.next_value::<::std::option::Option<_>>()?.map(plan_type::PlanTypeEnum::FinalPhysicalPlanWithStats) ; } } diff --git a/datafusion/proto/src/generated/prost.rs b/datafusion/proto/src/generated/prost.rs index 7b7b0afb9216..23be5d908866 100644 --- a/datafusion/proto/src/generated/prost.rs +++ b/datafusion/proto/src/generated/prost.rs @@ -1423,7 +1423,7 @@ pub struct OptimizedPhysicalPlanType { #[allow(clippy::derive_partial_eq_without_eq)] #[derive(Clone, PartialEq, ::prost::Message)] pub struct PlanType { - #[prost(oneof = "plan_type::PlanTypeEnum", tags = "1, 7, 8, 2, 3, 4, 5, 6")] + #[prost(oneof = "plan_type::PlanTypeEnum", tags = "1, 7, 8, 2, 3, 4, 9, 5, 6, 10")] pub plan_type_enum: ::core::option::Option, } /// Nested message and enum types in `PlanType`. @@ -1443,10 +1443,14 @@ pub mod plan_type { FinalLogicalPlan(super::EmptyMessage), #[prost(message, tag = "4")] InitialPhysicalPlan(super::EmptyMessage), + #[prost(message, tag = "9")] + InitialPhysicalPlanWithStats(super::EmptyMessage), #[prost(message, tag = "5")] OptimizedPhysicalPlan(super::OptimizedPhysicalPlanType), #[prost(message, tag = "6")] FinalPhysicalPlan(super::EmptyMessage), + #[prost(message, tag = "10")] + FinalPhysicalPlanWithStats(super::EmptyMessage), } } #[allow(clippy::derive_partial_eq_without_eq)] diff --git a/datafusion/proto/src/logical_plan/from_proto.rs b/datafusion/proto/src/logical_plan/from_proto.rs index f7e38757e923..0ecbe05e7903 100644 --- a/datafusion/proto/src/logical_plan/from_proto.rs +++ b/datafusion/proto/src/logical_plan/from_proto.rs @@ -19,7 +19,8 @@ use crate::protobuf::{ self, plan_type::PlanTypeEnum::{ AnalyzedLogicalPlan, FinalAnalyzedLogicalPlan, FinalLogicalPlan, - FinalPhysicalPlan, InitialLogicalPlan, InitialPhysicalPlan, OptimizedLogicalPlan, + FinalPhysicalPlan, FinalPhysicalPlanWithStats, InitialLogicalPlan, + InitialPhysicalPlan, InitialPhysicalPlanWithStats, OptimizedLogicalPlan, OptimizedPhysicalPlan, }, AnalyzedLogicalPlanType, CubeNode, GroupingSetNode, OptimizedLogicalPlanType, @@ -406,12 +407,14 @@ impl From<&protobuf::StringifiedPlan> for StringifiedPlan { } FinalLogicalPlan(_) => PlanType::FinalLogicalPlan, InitialPhysicalPlan(_) => PlanType::InitialPhysicalPlan, + InitialPhysicalPlanWithStats(_) => PlanType::InitialPhysicalPlanWithStats, OptimizedPhysicalPlan(OptimizedPhysicalPlanType { optimizer_name }) => { PlanType::OptimizedPhysicalPlan { optimizer_name: optimizer_name.clone(), } } FinalPhysicalPlan(_) => PlanType::FinalPhysicalPlan, + FinalPhysicalPlanWithStats(_) => PlanType::FinalPhysicalPlanWithStats, }, plan: Arc::new(stringified_plan.plan.clone()), } diff --git a/datafusion/proto/src/logical_plan/to_proto.rs b/datafusion/proto/src/logical_plan/to_proto.rs index 2bb7f89c7d4d..4c81ab954a71 100644 --- a/datafusion/proto/src/logical_plan/to_proto.rs +++ b/datafusion/proto/src/logical_plan/to_proto.rs @@ -24,7 +24,8 @@ use crate::protobuf::{ arrow_type::ArrowTypeEnum, plan_type::PlanTypeEnum::{ AnalyzedLogicalPlan, FinalAnalyzedLogicalPlan, FinalLogicalPlan, - FinalPhysicalPlan, InitialLogicalPlan, InitialPhysicalPlan, OptimizedLogicalPlan, + FinalPhysicalPlan, FinalPhysicalPlanWithStats, InitialLogicalPlan, + InitialPhysicalPlan, InitialPhysicalPlanWithStats, OptimizedLogicalPlan, OptimizedPhysicalPlan, }, AnalyzedLogicalPlanType, CubeNode, EmptyMessage, GroupingSetNode, LogicalExprList, @@ -352,6 +353,12 @@ impl From<&StringifiedPlan> for protobuf::StringifiedPlan { PlanType::FinalPhysicalPlan => Some(protobuf::PlanType { plan_type_enum: Some(FinalPhysicalPlan(EmptyMessage {})), }), + PlanType::InitialPhysicalPlanWithStats => Some(protobuf::PlanType { + plan_type_enum: Some(InitialPhysicalPlanWithStats(EmptyMessage {})), + }), + PlanType::FinalPhysicalPlanWithStats => Some(protobuf::PlanType { + plan_type_enum: Some(FinalPhysicalPlanWithStats(EmptyMessage {})), + }), }, plan: stringified_plan.plan.to_string(), } diff --git a/datafusion/sqllogictest/test_files/explain.slt b/datafusion/sqllogictest/test_files/explain.slt index 1db24efd9b4a..9726c35a319e 100644 --- a/datafusion/sqllogictest/test_files/explain.slt +++ b/datafusion/sqllogictest/test_files/explain.slt @@ -245,6 +245,7 @@ logical_plan after eliminate_projection SAME TEXT AS ABOVE logical_plan after push_down_limit SAME TEXT AS ABOVE logical_plan TableScan: simple_explain_test projection=[a, b, c] initial_physical_plan CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/example.csv]]}, projection=[a, b, c], has_header=true +initial_physical_plan_with_stats CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/example.csv]]}, projection=[a, b, c], has_header=true, statistics=[Rows=Absent, Bytes=Absent, [(Col[0]:),(Col[1]:),(Col[2]:)]] physical_plan after OutputRequirements OutputRequirementExec --CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/example.csv]]}, projection=[a, b, c], has_header=true @@ -260,6 +261,7 @@ physical_plan after PipelineChecker SAME TEXT AS ABOVE physical_plan after LimitAggregation SAME TEXT AS ABOVE physical_plan after ProjectionPushdown SAME TEXT AS ABOVE physical_plan CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/example.csv]]}, projection=[a, b, c], has_header=true +physical_plan_with_stats CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/example.csv]]}, projection=[a, b, c], has_header=true, statistics=[Rows=Absent, Bytes=Absent, [(Col[0]:),(Col[1]:),(Col[2]:)]] ### tests for EXPLAIN with display statistics enabled @@ -291,8 +293,72 @@ physical_plan GlobalLimitExec: skip=0, fetch=10, statistics=[Rows=Exact(8), Bytes=Absent, [(Col[0]:),(Col[1]:),(Col[2]:),(Col[3]:),(Col[4]:),(Col[5]:),(Col[6]:),(Col[7]:),(Col[8]:),(Col[9]:),(Col[10]:)]] --ParquetExec: file_groups={1 group: [[WORKSPACE_ROOT/parquet-testing/data/alltypes_plain.parquet]]}, projection=[id, bool_col, tinyint_col, smallint_col, int_col, bigint_col, float_col, double_col, date_string_col, string_col, timestamp_col], limit=10, statistics=[Rows=Exact(8), Bytes=Absent, [(Col[0]:),(Col[1]:),(Col[2]:),(Col[3]:),(Col[4]:),(Col[5]:),(Col[6]:),(Col[7]:),(Col[8]:),(Col[9]:),(Col[10]:)]] -statement ok -set datafusion.execution.collect_statistics = false; +# explain verbose with both collect & show statistics on +query TT +EXPLAIN VERBOSE SELECT * FROM alltypes_plain limit 10; +---- +initial_physical_plan +GlobalLimitExec: skip=0, fetch=10, statistics=[Rows=Exact(8), Bytes=Absent, [(Col[0]:),(Col[1]:),(Col[2]:),(Col[3]:),(Col[4]:),(Col[5]:),(Col[6]:),(Col[7]:),(Col[8]:),(Col[9]:),(Col[10]:)]] +--ParquetExec: file_groups={1 group: [[WORKSPACE_ROOT/parquet-testing/data/alltypes_plain.parquet]]}, projection=[id, bool_col, tinyint_col, smallint_col, int_col, bigint_col, float_col, double_col, date_string_col, string_col, timestamp_col], limit=10, statistics=[Rows=Exact(8), Bytes=Absent, [(Col[0]:),(Col[1]:),(Col[2]:),(Col[3]:),(Col[4]:),(Col[5]:),(Col[6]:),(Col[7]:),(Col[8]:),(Col[9]:),(Col[10]:)]] +physical_plan after OutputRequirements +OutputRequirementExec, statistics=[Rows=Exact(8), Bytes=Absent, [(Col[0]:),(Col[1]:),(Col[2]:),(Col[3]:),(Col[4]:),(Col[5]:),(Col[6]:),(Col[7]:),(Col[8]:),(Col[9]:),(Col[10]:)]] +--GlobalLimitExec: skip=0, fetch=10, statistics=[Rows=Exact(8), Bytes=Absent, [(Col[0]:),(Col[1]:),(Col[2]:),(Col[3]:),(Col[4]:),(Col[5]:),(Col[6]:),(Col[7]:),(Col[8]:),(Col[9]:),(Col[10]:)]] +----ParquetExec: file_groups={1 group: [[WORKSPACE_ROOT/parquet-testing/data/alltypes_plain.parquet]]}, projection=[id, bool_col, tinyint_col, smallint_col, int_col, bigint_col, float_col, double_col, date_string_col, string_col, timestamp_col], limit=10, statistics=[Rows=Exact(8), Bytes=Absent, [(Col[0]:),(Col[1]:),(Col[2]:),(Col[3]:),(Col[4]:),(Col[5]:),(Col[6]:),(Col[7]:),(Col[8]:),(Col[9]:),(Col[10]:)]] +physical_plan after aggregate_statistics SAME TEXT AS ABOVE +physical_plan after join_selection SAME TEXT AS ABOVE +physical_plan after LimitedDistinctAggregation SAME TEXT AS ABOVE +physical_plan after EnforceDistribution SAME TEXT AS ABOVE +physical_plan after CombinePartialFinalAggregate SAME TEXT AS ABOVE +physical_plan after EnforceSorting SAME TEXT AS ABOVE +physical_plan after coalesce_batches SAME TEXT AS ABOVE +physical_plan after OutputRequirements +GlobalLimitExec: skip=0, fetch=10, statistics=[Rows=Exact(8), Bytes=Absent, [(Col[0]:),(Col[1]:),(Col[2]:),(Col[3]:),(Col[4]:),(Col[5]:),(Col[6]:),(Col[7]:),(Col[8]:),(Col[9]:),(Col[10]:)]] +--ParquetExec: file_groups={1 group: [[WORKSPACE_ROOT/parquet-testing/data/alltypes_plain.parquet]]}, projection=[id, bool_col, tinyint_col, smallint_col, int_col, bigint_col, float_col, double_col, date_string_col, string_col, timestamp_col], limit=10, statistics=[Rows=Exact(8), Bytes=Absent, [(Col[0]:),(Col[1]:),(Col[2]:),(Col[3]:),(Col[4]:),(Col[5]:),(Col[6]:),(Col[7]:),(Col[8]:),(Col[9]:),(Col[10]:)]] +physical_plan after PipelineChecker SAME TEXT AS ABOVE +physical_plan after LimitAggregation SAME TEXT AS ABOVE +physical_plan after ProjectionPushdown SAME TEXT AS ABOVE +physical_plan +GlobalLimitExec: skip=0, fetch=10, statistics=[Rows=Exact(8), Bytes=Absent, [(Col[0]:),(Col[1]:),(Col[2]:),(Col[3]:),(Col[4]:),(Col[5]:),(Col[6]:),(Col[7]:),(Col[8]:),(Col[9]:),(Col[10]:)]] +--ParquetExec: file_groups={1 group: [[WORKSPACE_ROOT/parquet-testing/data/alltypes_plain.parquet]]}, projection=[id, bool_col, tinyint_col, smallint_col, int_col, bigint_col, float_col, double_col, date_string_col, string_col, timestamp_col], limit=10, statistics=[Rows=Exact(8), Bytes=Absent, [(Col[0]:),(Col[1]:),(Col[2]:),(Col[3]:),(Col[4]:),(Col[5]:),(Col[6]:),(Col[7]:),(Col[8]:),(Col[9]:),(Col[10]:)]] + statement ok set datafusion.explain.show_statistics = false; + +# explain verbose with collect on and & show statistics off: still has stats +query TT +EXPLAIN VERBOSE SELECT * FROM alltypes_plain limit 10; +---- +initial_physical_plan +GlobalLimitExec: skip=0, fetch=10 +--ParquetExec: file_groups={1 group: [[WORKSPACE_ROOT/parquet-testing/data/alltypes_plain.parquet]]}, projection=[id, bool_col, tinyint_col, smallint_col, int_col, bigint_col, float_col, double_col, date_string_col, string_col, timestamp_col], limit=10 +initial_physical_plan_with_stats +GlobalLimitExec: skip=0, fetch=10, statistics=[Rows=Exact(8), Bytes=Absent, [(Col[0]:),(Col[1]:),(Col[2]:),(Col[3]:),(Col[4]:),(Col[5]:),(Col[6]:),(Col[7]:),(Col[8]:),(Col[9]:),(Col[10]:)]] +--ParquetExec: file_groups={1 group: [[WORKSPACE_ROOT/parquet-testing/data/alltypes_plain.parquet]]}, projection=[id, bool_col, tinyint_col, smallint_col, int_col, bigint_col, float_col, double_col, date_string_col, string_col, timestamp_col], limit=10, statistics=[Rows=Exact(8), Bytes=Absent, [(Col[0]:),(Col[1]:),(Col[2]:),(Col[3]:),(Col[4]:),(Col[5]:),(Col[6]:),(Col[7]:),(Col[8]:),(Col[9]:),(Col[10]:)]] +physical_plan after OutputRequirements +OutputRequirementExec +--GlobalLimitExec: skip=0, fetch=10 +----ParquetExec: file_groups={1 group: [[WORKSPACE_ROOT/parquet-testing/data/alltypes_plain.parquet]]}, projection=[id, bool_col, tinyint_col, smallint_col, int_col, bigint_col, float_col, double_col, date_string_col, string_col, timestamp_col], limit=10 +physical_plan after aggregate_statistics SAME TEXT AS ABOVE +physical_plan after join_selection SAME TEXT AS ABOVE +physical_plan after LimitedDistinctAggregation SAME TEXT AS ABOVE +physical_plan after EnforceDistribution SAME TEXT AS ABOVE +physical_plan after CombinePartialFinalAggregate SAME TEXT AS ABOVE +physical_plan after EnforceSorting SAME TEXT AS ABOVE +physical_plan after coalesce_batches SAME TEXT AS ABOVE +physical_plan after OutputRequirements +GlobalLimitExec: skip=0, fetch=10 +--ParquetExec: file_groups={1 group: [[WORKSPACE_ROOT/parquet-testing/data/alltypes_plain.parquet]]}, projection=[id, bool_col, tinyint_col, smallint_col, int_col, bigint_col, float_col, double_col, date_string_col, string_col, timestamp_col], limit=10 +physical_plan after PipelineChecker SAME TEXT AS ABOVE +physical_plan after LimitAggregation SAME TEXT AS ABOVE +physical_plan after ProjectionPushdown SAME TEXT AS ABOVE +physical_plan +GlobalLimitExec: skip=0, fetch=10 +--ParquetExec: file_groups={1 group: [[WORKSPACE_ROOT/parquet-testing/data/alltypes_plain.parquet]]}, projection=[id, bool_col, tinyint_col, smallint_col, int_col, bigint_col, float_col, double_col, date_string_col, string_col, timestamp_col], limit=10 +physical_plan_with_stats +GlobalLimitExec: skip=0, fetch=10, statistics=[Rows=Exact(8), Bytes=Absent, [(Col[0]:),(Col[1]:),(Col[2]:),(Col[3]:),(Col[4]:),(Col[5]:),(Col[6]:),(Col[7]:),(Col[8]:),(Col[9]:),(Col[10]:)]] +--ParquetExec: file_groups={1 group: [[WORKSPACE_ROOT/parquet-testing/data/alltypes_plain.parquet]]}, projection=[id, bool_col, tinyint_col, smallint_col, int_col, bigint_col, float_col, double_col, date_string_col, string_col, timestamp_col], limit=10, statistics=[Rows=Exact(8), Bytes=Absent, [(Col[0]:),(Col[1]:),(Col[2]:),(Col[3]:),(Col[4]:),(Col[5]:),(Col[6]:),(Col[7]:),(Col[8]:),(Col[9]:),(Col[10]:)]] + + +statement ok +set datafusion.execution.collect_statistics = false; From 4535551120dd4c31c160a35851b9e4a33514a44f Mon Sep 17 00:00:00 2001 From: Syleechan <38198463+Syleechan@users.noreply.github.com> Date: Tue, 14 Nov 2023 15:31:25 +0800 Subject: [PATCH 247/572] feat:implement postgres style 'overlay' string function (#8117) * feat:implement posgres style 'overlay' string function * code format * code format * code format * code format * add sql slt test * fix modify other case issue * add test expr * add annotation * add overlay function sql reference doc * add sql case and format doc --- datafusion/expr/src/built_in_function.rs | 18 ++- datafusion/expr/src/expr_fn.rs | 7 ++ datafusion/physical-expr/src/functions.rs | 11 ++ .../physical-expr/src/string_expressions.rs | 108 ++++++++++++++++++ datafusion/proto/proto/datafusion.proto | 1 + datafusion/proto/src/generated/pbjson.rs | 3 + datafusion/proto/src/generated/prost.rs | 3 + .../proto/src/logical_plan/from_proto.rs | 15 ++- datafusion/proto/src/logical_plan/to_proto.rs | 1 + datafusion/sql/src/expr/mod.rs | 40 ++++++- .../sqllogictest/test_files/functions.slt | 42 +++++++ .../source/user-guide/sql/scalar_functions.md | 17 +++ 12 files changed, 260 insertions(+), 6 deletions(-) diff --git a/datafusion/expr/src/built_in_function.rs b/datafusion/expr/src/built_in_function.rs index 0d2c1f2e3cb7..77c64128e156 100644 --- a/datafusion/expr/src/built_in_function.rs +++ b/datafusion/expr/src/built_in_function.rs @@ -292,6 +292,8 @@ pub enum BuiltinScalarFunction { RegexpMatch, /// arrow_typeof ArrowTypeof, + /// overlay + OverLay, } /// Maps the sql function name to `BuiltinScalarFunction` @@ -455,6 +457,7 @@ impl BuiltinScalarFunction { BuiltinScalarFunction::Struct => Volatility::Immutable, BuiltinScalarFunction::FromUnixtime => Volatility::Immutable, BuiltinScalarFunction::ArrowTypeof => Volatility::Immutable, + BuiltinScalarFunction::OverLay => Volatility::Immutable, // Stable builtin functions BuiltinScalarFunction::Now => Volatility::Stable, @@ -812,6 +815,10 @@ impl BuiltinScalarFunction { BuiltinScalarFunction::Abs => Ok(input_expr_types[0].clone()), + BuiltinScalarFunction::OverLay => { + utf8_to_str_type(&input_expr_types[0], "overlay") + } + BuiltinScalarFunction::Acos | BuiltinScalarFunction::Asin | BuiltinScalarFunction::Atan @@ -1258,7 +1265,15 @@ impl BuiltinScalarFunction { } BuiltinScalarFunction::ArrowTypeof => Signature::any(1, self.volatility()), BuiltinScalarFunction::Abs => Signature::any(1, self.volatility()), - + BuiltinScalarFunction::OverLay => Signature::one_of( + vec![ + Exact(vec![Utf8, Utf8, Int64, Int64]), + Exact(vec![LargeUtf8, LargeUtf8, Int64, Int64]), + Exact(vec![Utf8, Utf8, Int64]), + Exact(vec![LargeUtf8, LargeUtf8, Int64]), + ], + self.volatility(), + ), BuiltinScalarFunction::Acos | BuiltinScalarFunction::Asin | BuiltinScalarFunction::Atan @@ -1517,6 +1532,7 @@ fn aliases(func: &BuiltinScalarFunction) -> &'static [&'static str] { BuiltinScalarFunction::Cardinality => &["cardinality"], BuiltinScalarFunction::MakeArray => &["make_array", "make_list"], BuiltinScalarFunction::ArrayIntersect => &["array_intersect", "list_intersect"], + BuiltinScalarFunction::OverLay => &["overlay"], // struct functions BuiltinScalarFunction::Struct => &["struct"], diff --git a/datafusion/expr/src/expr_fn.rs b/datafusion/expr/src/expr_fn.rs index 0d920beb416f..91674cc092e6 100644 --- a/datafusion/expr/src/expr_fn.rs +++ b/datafusion/expr/src/expr_fn.rs @@ -838,6 +838,11 @@ nary_scalar_expr!( "concatenates several strings, placing a seperator between each one" ); nary_scalar_expr!(Concat, concat_expr, "concatenates several strings"); +nary_scalar_expr!( + OverLay, + overlay, + "replace the substring of string that starts at the start'th character and extends for count characters with new substring" +); // date functions scalar_expr!(DatePart, date_part, part date, "extracts a subfield from the date"); @@ -1174,6 +1179,8 @@ mod test { test_nary_scalar_expr!(MakeArray, array, input); test_unary_scalar_expr!(ArrowTypeof, arrow_typeof); + test_nary_scalar_expr!(OverLay, overlay, string, characters, position, len); + test_nary_scalar_expr!(OverLay, overlay, string, characters, position); } #[test] diff --git a/datafusion/physical-expr/src/functions.rs b/datafusion/physical-expr/src/functions.rs index 80c0eaf054fd..7f8921e86c38 100644 --- a/datafusion/physical-expr/src/functions.rs +++ b/datafusion/physical-expr/src/functions.rs @@ -829,6 +829,17 @@ pub fn create_physical_fun( "{input_data_type}" ))))) }), + BuiltinScalarFunction::OverLay => Arc::new(|args| match args[0].data_type() { + DataType::Utf8 => { + make_scalar_function(string_expressions::overlay::)(args) + } + DataType::LargeUtf8 => { + make_scalar_function(string_expressions::overlay::)(args) + } + other => Err(DataFusionError::Internal(format!( + "Unsupported data type {other:?} for function overlay", + ))), + }), }) } diff --git a/datafusion/physical-expr/src/string_expressions.rs b/datafusion/physical-expr/src/string_expressions.rs index e6a3d5c331a5..7e954fdcfdc4 100644 --- a/datafusion/physical-expr/src/string_expressions.rs +++ b/datafusion/physical-expr/src/string_expressions.rs @@ -553,11 +553,102 @@ pub fn uuid(args: &[ColumnarValue]) -> Result { Ok(ColumnarValue::Array(Arc::new(array))) } +/// OVERLAY(string1 PLACING string2 FROM integer FOR integer2) +/// Replaces a substring of string1 with string2 starting at the integer bit +/// pgsql overlay('Txxxxas' placing 'hom' from 2 for 4) → Thomas +/// overlay('Txxxxas' placing 'hom' from 2) -> Thomxas, without for option, str2's len is instead +pub fn overlay(args: &[ArrayRef]) -> Result { + match args.len() { + 3 => { + let string_array = as_generic_string_array::(&args[0])?; + let characters_array = as_generic_string_array::(&args[1])?; + let pos_num = as_int64_array(&args[2])?; + + let result = string_array + .iter() + .zip(characters_array.iter()) + .zip(pos_num.iter()) + .map(|((string, characters), start_pos)| { + match (string, characters, start_pos) { + (Some(string), Some(characters), Some(start_pos)) => { + let string_len = string.chars().count(); + let characters_len = characters.chars().count(); + let replace_len = characters_len as i64; + let mut res = + String::with_capacity(string_len.max(characters_len)); + + //as sql replace index start from 1 while string index start from 0 + if start_pos > 1 && start_pos - 1 < string_len as i64 { + let start = (start_pos - 1) as usize; + res.push_str(&string[..start]); + } + res.push_str(characters); + // if start + replace_len - 1 >= string_length, just to string end + if start_pos + replace_len - 1 < string_len as i64 { + let end = (start_pos + replace_len - 1) as usize; + res.push_str(&string[end..]); + } + Ok(Some(res)) + } + _ => Ok(None), + } + }) + .collect::>>()?; + Ok(Arc::new(result) as ArrayRef) + } + 4 => { + let string_array = as_generic_string_array::(&args[0])?; + let characters_array = as_generic_string_array::(&args[1])?; + let pos_num = as_int64_array(&args[2])?; + let len_num = as_int64_array(&args[3])?; + + let result = string_array + .iter() + .zip(characters_array.iter()) + .zip(pos_num.iter()) + .zip(len_num.iter()) + .map(|(((string, characters), start_pos), len)| { + match (string, characters, start_pos, len) { + (Some(string), Some(characters), Some(start_pos), Some(len)) => { + let string_len = string.chars().count(); + let characters_len = characters.chars().count(); + let replace_len = len.min(string_len as i64); + let mut res = + String::with_capacity(string_len.max(characters_len)); + + //as sql replace index start from 1 while string index start from 0 + if start_pos > 1 && start_pos - 1 < string_len as i64 { + let start = (start_pos - 1) as usize; + res.push_str(&string[..start]); + } + res.push_str(characters); + // if start + replace_len - 1 >= string_length, just to string end + if start_pos + replace_len - 1 < string_len as i64 { + let end = (start_pos + replace_len - 1) as usize; + res.push_str(&string[end..]); + } + Ok(Some(res)) + } + _ => Ok(None), + } + }) + .collect::>>()?; + Ok(Arc::new(result) as ArrayRef) + } + other => { + internal_err!( + "overlay was called with {other} arguments. It requires 3 or 4." + ) + } + } +} + #[cfg(test)] mod tests { use crate::string_expressions; use arrow::{array::Int32Array, datatypes::Int32Type}; + use arrow_array::Int64Array; use super::*; @@ -599,4 +690,21 @@ mod tests { Ok(()) } + + #[test] + fn to_overlay() -> Result<()> { + let string = + Arc::new(StringArray::from(vec!["123", "abcdefg", "xyz", "Txxxxas"])); + let replace_string = + Arc::new(StringArray::from(vec!["abc", "qwertyasdfg", "ijk", "hom"])); + let start = Arc::new(Int64Array::from(vec![4, 1, 1, 2])); // start + let end = Arc::new(Int64Array::from(vec![5, 7, 2, 4])); // replace len + + let res = overlay::(&[string, replace_string, start, end]).unwrap(); + let result = as_generic_string_array::(&res).unwrap(); + let expected = StringArray::from(vec!["abc", "qwertyasdfg", "ijkz", "Thomas"]); + assert_eq!(&expected, result); + + Ok(()) + } } diff --git a/datafusion/proto/proto/datafusion.proto b/datafusion/proto/proto/datafusion.proto index 5d7c570bc173..d85678a76bf1 100644 --- a/datafusion/proto/proto/datafusion.proto +++ b/datafusion/proto/proto/datafusion.proto @@ -636,6 +636,7 @@ enum ScalarFunction { ToTimestampNanos = 118; ArrayIntersect = 119; ArrayUnion = 120; + OverLay = 121; } message ScalarFunctionNode { diff --git a/datafusion/proto/src/generated/pbjson.rs b/datafusion/proto/src/generated/pbjson.rs index 12fa73205d49..64db9137d64f 100644 --- a/datafusion/proto/src/generated/pbjson.rs +++ b/datafusion/proto/src/generated/pbjson.rs @@ -20935,6 +20935,7 @@ impl serde::Serialize for ScalarFunction { Self::ToTimestampNanos => "ToTimestampNanos", Self::ArrayIntersect => "ArrayIntersect", Self::ArrayUnion => "ArrayUnion", + Self::OverLay => "OverLay", }; serializer.serialize_str(variant) } @@ -21067,6 +21068,7 @@ impl<'de> serde::Deserialize<'de> for ScalarFunction { "ToTimestampNanos", "ArrayIntersect", "ArrayUnion", + "OverLay", ]; struct GeneratedVisitor; @@ -21228,6 +21230,7 @@ impl<'de> serde::Deserialize<'de> for ScalarFunction { "ToTimestampNanos" => Ok(ScalarFunction::ToTimestampNanos), "ArrayIntersect" => Ok(ScalarFunction::ArrayIntersect), "ArrayUnion" => Ok(ScalarFunction::ArrayUnion), + "OverLay" => Ok(ScalarFunction::OverLay), _ => Err(serde::de::Error::unknown_variant(value, FIELDS)), } } diff --git a/datafusion/proto/src/generated/prost.rs b/datafusion/proto/src/generated/prost.rs index 23be5d908866..131ca11993c1 100644 --- a/datafusion/proto/src/generated/prost.rs +++ b/datafusion/proto/src/generated/prost.rs @@ -2567,6 +2567,7 @@ pub enum ScalarFunction { ToTimestampNanos = 118, ArrayIntersect = 119, ArrayUnion = 120, + OverLay = 121, } impl ScalarFunction { /// String value of the enum field names used in the ProtoBuf definition. @@ -2696,6 +2697,7 @@ impl ScalarFunction { ScalarFunction::ToTimestampNanos => "ToTimestampNanos", ScalarFunction::ArrayIntersect => "ArrayIntersect", ScalarFunction::ArrayUnion => "ArrayUnion", + ScalarFunction::OverLay => "OverLay", } } /// Creates an enum from field names used in the ProtoBuf definition. @@ -2822,6 +2824,7 @@ impl ScalarFunction { "ToTimestampNanos" => Some(Self::ToTimestampNanos), "ArrayIntersect" => Some(Self::ArrayIntersect), "ArrayUnion" => Some(Self::ArrayUnion), + "OverLay" => Some(Self::OverLay), _ => None, } } diff --git a/datafusion/proto/src/logical_plan/from_proto.rs b/datafusion/proto/src/logical_plan/from_proto.rs index 0ecbe05e7903..9ca7bb0e893a 100644 --- a/datafusion/proto/src/logical_plan/from_proto.rs +++ b/datafusion/proto/src/logical_plan/from_proto.rs @@ -52,10 +52,10 @@ use datafusion_expr::{ factorial, flatten, floor, from_unixtime, gcd, isnan, iszero, lcm, left, ln, log, log10, log2, logical_plan::{PlanType, StringifiedPlan}, - lower, lpad, ltrim, md5, nanvl, now, nullif, octet_length, pi, power, radians, - random, regexp_match, regexp_replace, repeat, replace, reverse, right, round, rpad, - rtrim, sha224, sha256, sha384, sha512, signum, sin, sinh, split_part, sqrt, - starts_with, string_to_array, strpos, struct_fun, substr, substring, tan, tanh, + lower, lpad, ltrim, md5, nanvl, now, nullif, octet_length, overlay, pi, power, + radians, random, regexp_match, regexp_replace, repeat, replace, reverse, right, + round, rpad, rtrim, sha224, sha256, sha384, sha512, signum, sin, sinh, split_part, + sqrt, starts_with, string_to_array, strpos, struct_fun, substr, substring, tan, tanh, to_hex, to_timestamp_micros, to_timestamp_millis, to_timestamp_nanos, to_timestamp_seconds, translate, trim, trunc, upper, uuid, window_frame::regularize, @@ -546,6 +546,7 @@ impl From<&protobuf::ScalarFunction> for BuiltinScalarFunction { ScalarFunction::Isnan => Self::Isnan, ScalarFunction::Iszero => Self::Iszero, ScalarFunction::ArrowTypeof => Self::ArrowTypeof, + ScalarFunction::OverLay => Self::OverLay, } } } @@ -1680,6 +1681,12 @@ pub fn parse_expr( parse_expr(&args[1], registry)?, parse_expr(&args[2], registry)?, )), + ScalarFunction::OverLay => Ok(overlay( + args.to_owned() + .iter() + .map(|expr| parse_expr(expr, registry)) + .collect::, _>>()?, + )), ScalarFunction::StructFun => { Ok(struct_fun(parse_expr(&args[0], registry)?)) } diff --git a/datafusion/proto/src/logical_plan/to_proto.rs b/datafusion/proto/src/logical_plan/to_proto.rs index 4c81ab954a71..974d6c5aaba8 100644 --- a/datafusion/proto/src/logical_plan/to_proto.rs +++ b/datafusion/proto/src/logical_plan/to_proto.rs @@ -1553,6 +1553,7 @@ impl TryFrom<&BuiltinScalarFunction> for protobuf::ScalarFunction { BuiltinScalarFunction::Isnan => Self::Isnan, BuiltinScalarFunction::Iszero => Self::Iszero, BuiltinScalarFunction::ArrowTypeof => Self::ArrowTypeof, + BuiltinScalarFunction::OverLay => Self::OverLay, }; Ok(scalar_function) diff --git a/datafusion/sql/src/expr/mod.rs b/datafusion/sql/src/expr/mod.rs index 1cf0fc133f04..7fa16ced39da 100644 --- a/datafusion/sql/src/expr/mod.rs +++ b/datafusion/sql/src/expr/mod.rs @@ -459,7 +459,19 @@ impl<'a, S: ContextProvider> SqlToRel<'a, S> { schema, planner_context, ), - + SQLExpr::Overlay { + expr, + overlay_what, + overlay_from, + overlay_for, + } => self.sql_overlay_to_expr( + *expr, + *overlay_what, + *overlay_from, + overlay_for, + schema, + planner_context, + ), SQLExpr::Nested(e) => { self.sql_expr_to_logical_expr(*e, schema, planner_context) } @@ -645,6 +657,32 @@ impl<'a, S: ContextProvider> SqlToRel<'a, S> { Ok(Expr::ScalarFunction(ScalarFunction::new(fun, args))) } + fn sql_overlay_to_expr( + &self, + expr: SQLExpr, + overlay_what: SQLExpr, + overlay_from: SQLExpr, + overlay_for: Option>, + schema: &DFSchema, + planner_context: &mut PlannerContext, + ) -> Result { + let fun = BuiltinScalarFunction::OverLay; + let arg = self.sql_expr_to_logical_expr(expr, schema, planner_context)?; + let what_arg = + self.sql_expr_to_logical_expr(overlay_what, schema, planner_context)?; + let from_arg = + self.sql_expr_to_logical_expr(overlay_from, schema, planner_context)?; + let args = match overlay_for { + Some(for_expr) => { + let for_expr = + self.sql_expr_to_logical_expr(*for_expr, schema, planner_context)?; + vec![arg, what_arg, from_arg, for_expr] + } + None => vec![arg, what_arg, from_arg], + }; + Ok(Expr::ScalarFunction(ScalarFunction::new(fun, args))) + } + fn sql_agg_with_filter_to_expr( &self, expr: SQLExpr, diff --git a/datafusion/sqllogictest/test_files/functions.slt b/datafusion/sqllogictest/test_files/functions.slt index 2054752cc59c..8f4230438480 100644 --- a/datafusion/sqllogictest/test_files/functions.slt +++ b/datafusion/sqllogictest/test_files/functions.slt @@ -815,3 +815,45 @@ SELECT products.* REPLACE (price*2 AS price, product_id+1000 AS product_id) FROM 1002 OldBrand Product 2 59.98 1003 OldBrand Product 3 79.98 1004 OldBrand Product 4 99.98 + +#overlay tests +statement ok +CREATE TABLE over_test( + str TEXT, + characters TEXT, + pos INT, + len INT +) as VALUES + ('123', 'abc', 4, 5), + ('abcdefg', 'qwertyasdfg', 1, 7), + ('xyz', 'ijk', 1, 2), + ('Txxxxas', 'hom', 2, 4), + (NULL, 'hom', 2, 4), + ('Txxxxas', 'hom', NULL, 4), + ('Txxxxas', 'hom', 2, NULL), + ('Txxxxas', NULL, 2, 4) +; + +query T +SELECT overlay(str placing characters from pos for len) from over_test +---- +abc +qwertyasdfg +ijkz +Thomas +NULL +NULL +NULL +NULL + +query T +SELECT overlay(str placing characters from pos) from over_test +---- +abc +qwertyasdfg +ijk +Thomxas +NULL +NULL +Thomxas +NULL diff --git a/docs/source/user-guide/sql/scalar_functions.md b/docs/source/user-guide/sql/scalar_functions.md index 2959e8202437..099c90312227 100644 --- a/docs/source/user-guide/sql/scalar_functions.md +++ b/docs/source/user-guide/sql/scalar_functions.md @@ -635,6 +635,7 @@ nullif(expression1, expression2) - [trim](#trim) - [upper](#upper) - [uuid](#uuid) +- [overlay](#overlay) ### `ascii` @@ -1120,6 +1121,22 @@ Returns UUID v4 string value which is unique per row. uuid() ``` +### `overlay` + +Returns the string which is replaced by another string from the specified position and specified count length. +For example, `overlay('Txxxxas' placing 'hom' from 2 for 4) → Thomas` + +``` +overlay(str PLACING substr FROM pos [FOR count]) +``` + +#### Arguments + +- **str**: String expression to operate on. +- **substr**: the string to replace part of str. +- **pos**: the start position to replace of str. +- **count**: the count of characters to be replaced from start position of str. If not specified, will use substr length instead. + ## Binary String Functions - [decode](#decode) From fcd17c85c2eba1c5c8d92beb52f4351286f2dcea Mon Sep 17 00:00:00 2001 From: Andrew Lamb Date: Tue, 14 Nov 2023 03:12:04 -0500 Subject: [PATCH 248/572] Minor: Encapsulate `LeftJoinData` into a struct (rather than anonymous enum) and add comments (#8153) * Minor: Encapsulate LeftJoinData into a struct (rather than anonymous enum) * clippy --- .../physical-plan/src/joins/hash_join.rs | 72 ++++++++++++++----- .../src/joins/hash_join_utils.rs | 9 ++- 2 files changed, 61 insertions(+), 20 deletions(-) diff --git a/datafusion/physical-plan/src/joins/hash_join.rs b/datafusion/physical-plan/src/joins/hash_join.rs index 546a929bf939..da57fa07ccd9 100644 --- a/datafusion/physical-plan/src/joins/hash_join.rs +++ b/datafusion/physical-plan/src/joins/hash_join.rs @@ -73,7 +73,47 @@ use datafusion_physical_expr::EquivalenceProperties; use ahash::RandomState; use futures::{ready, Stream, StreamExt, TryStreamExt}; -type JoinLeftData = (JoinHashMap, RecordBatch, MemoryReservation); +/// HashTable and input data for the left (build side) of a join +struct JoinLeftData { + /// The hash table with indices into `batch` + hash_map: JoinHashMap, + /// The input rows for the build side + batch: RecordBatch, + /// Memory reservation that tracks memory used by `hash_map` hash table + /// `batch`. Cleared on drop. + #[allow(dead_code)] + reservation: MemoryReservation, +} + +impl JoinLeftData { + /// Create a new `JoinLeftData` from its parts + fn new( + hash_map: JoinHashMap, + batch: RecordBatch, + reservation: MemoryReservation, + ) -> Self { + Self { + hash_map, + batch, + reservation, + } + } + + /// Returns the number of rows in the build side + fn num_rows(&self) -> usize { + self.batch.num_rows() + } + + /// return a reference to the hash map + fn hash_map(&self) -> &JoinHashMap { + &self.hash_map + } + + /// returns a reference to the build side batch + fn batch(&self) -> &RecordBatch { + &self.batch + } +} /// Join execution plan: Evaluates eqijoin predicates in parallel on multiple /// partitions using a hash table and an optional filter list to apply post @@ -692,8 +732,9 @@ async fn collect_left_input( // Merge all batches into a single batch, so we // can directly index into the arrays let single_batch = concat_batches(&schema, &batches, num_rows)?; + let data = JoinLeftData::new(hashmap, single_batch, reservation); - Ok((hashmap, single_batch, reservation)) + Ok(data) } /// Updates `hash` with new entries from [RecordBatch] evaluated against the expressions `on`, @@ -770,7 +811,7 @@ struct HashJoinStream { left_fut: OnceFut, /// Which left (probe) side rows have been matches while creating output. /// For some OUTER joins, we need to know which rows have not been matched - /// to produce the correct. + /// to produce the correct output. visited_left_side: Option, /// right (probe) input right: SendableRecordBatchStream, @@ -1042,13 +1083,13 @@ impl HashJoinStream { { // TODO: Replace `ceil` wrapper with stable `div_cell` after // https://github.com/rust-lang/rust/issues/88581 - let visited_bitmap_size = bit_util::ceil(left_data.1.num_rows(), 8); + let visited_bitmap_size = bit_util::ceil(left_data.num_rows(), 8); self.reservation.try_grow(visited_bitmap_size)?; self.join_metrics.build_mem_used.add(visited_bitmap_size); } let visited_left_side = self.visited_left_side.get_or_insert_with(|| { - let num_rows = left_data.1.num_rows(); + let num_rows = left_data.num_rows(); if need_produce_result_in_final(self.join_type) { // Some join types need to track which row has be matched or unmatched: // `left semi` join: need to use the bitmap to produce the matched row in the left side @@ -1075,8 +1116,8 @@ impl HashJoinStream { // get the matched two indices for the on condition let left_right_indices = build_equal_condition_join_indices( - &left_data.0, - &left_data.1, + left_data.hash_map(), + left_data.batch(), &batch, &self.on_left, &self.on_right, @@ -1108,7 +1149,7 @@ impl HashJoinStream { let result = build_batch_from_indices( &self.schema, - &left_data.1, + left_data.batch(), &batch, &left_side, &right_side, @@ -1140,7 +1181,7 @@ impl HashJoinStream { // use the left and right indices to produce the batch result let result = build_batch_from_indices( &self.schema, - &left_data.1, + left_data.batch(), &empty_right_batch, &left_side, &right_side, @@ -2519,16 +2560,11 @@ mod tests { ("c", &vec![30, 40]), ); - let left_data = ( - JoinHashMap { - map: hashmap_left, - next, - }, - left, - ); + let join_hash_map = JoinHashMap::new(hashmap_left, next); + let (l, r) = build_equal_condition_join_indices( - &left_data.0, - &left_data.1, + &join_hash_map, + &left, &right, &[Column::new("a", 0)], &[Column::new("a", 0)], diff --git a/datafusion/physical-plan/src/joins/hash_join_utils.rs b/datafusion/physical-plan/src/joins/hash_join_utils.rs index 5ebf370b6d71..fecbf96f0895 100644 --- a/datafusion/physical-plan/src/joins/hash_join_utils.rs +++ b/datafusion/physical-plan/src/joins/hash_join_utils.rs @@ -103,12 +103,17 @@ use hashbrown::HashSet; /// ``` pub struct JoinHashMap { // Stores hash value to last row index - pub map: RawTable<(u64, u64)>, + map: RawTable<(u64, u64)>, // Stores indices in chained list data structure - pub next: Vec, + next: Vec, } impl JoinHashMap { + #[cfg(test)] + pub(crate) fn new(map: RawTable<(u64, u64)>, next: Vec) -> Self { + Self { map, next } + } + pub(crate) fn with_capacity(capacity: usize) -> Self { JoinHashMap { map: RawTable::with_capacity(capacity), From aaaf698b5e46dd1c4d29bce69dc539a667d2dda6 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Tue, 14 Nov 2023 01:05:13 -0800 Subject: [PATCH 249/572] Update sqllogictest requirement from 0.18.0 to 0.19.0 (#8163) Updates the requirements on [sqllogictest](https://github.com/risinglightdb/sqllogictest-rs) to permit the latest version. - [Release notes](https://github.com/risinglightdb/sqllogictest-rs/releases) - [Changelog](https://github.com/risinglightdb/sqllogictest-rs/blob/main/CHANGELOG.md) - [Commits](https://github.com/risinglightdb/sqllogictest-rs/compare/v0.18.0...v0.19.0) --- updated-dependencies: - dependency-name: sqllogictest dependency-type: direct:production ... Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- datafusion/sqllogictest/Cargo.toml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/datafusion/sqllogictest/Cargo.toml b/datafusion/sqllogictest/Cargo.toml index 4caec0e84b7f..436c6159e7a3 100644 --- a/datafusion/sqllogictest/Cargo.toml +++ b/datafusion/sqllogictest/Cargo.toml @@ -46,7 +46,7 @@ object_store = { workspace = true } postgres-protocol = { version = "0.6.4", optional = true } postgres-types = { version = "0.2.4", optional = true } rust_decimal = { version = "1.27.0" } -sqllogictest = "0.18.0" +sqllogictest = "0.19.0" sqlparser = { workspace = true } tempfile = { workspace = true } thiserror = { workspace = true } From 64057fd9ed763c79c3068c548be6bdd058f04608 Mon Sep 17 00:00:00 2001 From: Jonah Gao Date: Tue, 14 Nov 2023 22:28:03 +0800 Subject: [PATCH 250/572] feat: fill missing values with NULLs while inserting (#8146) * feat: fill missing values with NULLs while inserting * add test comment * update to re-trigger ci --- datafusion/sql/src/statement.rs | 46 +++++++++++-------- datafusion/sql/tests/sql_integration.rs | 33 ++++++------- datafusion/sqllogictest/test_files/insert.slt | 17 ++++++- .../test_files/insert_to_external.slt | 17 ++++++- 4 files changed, 75 insertions(+), 38 deletions(-) diff --git a/datafusion/sql/src/statement.rs b/datafusion/sql/src/statement.rs index ecc77b044223..49755729d2d5 100644 --- a/datafusion/sql/src/statement.rs +++ b/datafusion/sql/src/statement.rs @@ -33,7 +33,7 @@ use datafusion_common::parsers::CompressionTypeVariant; use datafusion_common::{ not_impl_err, plan_datafusion_err, plan_err, unqualified_field_not_found, Column, Constraints, DFField, DFSchema, DFSchemaRef, DataFusionError, OwnedTableReference, - Result, SchemaReference, TableReference, ToDFSchema, + Result, ScalarValue, SchemaReference, TableReference, ToDFSchema, }; use datafusion_expr::dml::{CopyOptions, CopyTo}; use datafusion_expr::expr_rewriter::normalize_col_with_schemas_and_ambiguity_check; @@ -1087,9 +1087,14 @@ impl<'a, S: ContextProvider> SqlToRel<'a, S> { let arrow_schema = (*table_source.schema()).clone(); let table_schema = DFSchema::try_from(arrow_schema)?; - // Get insert fields and index_mapping - // The i-th field of the table is `fields[index_mapping[i]]` - let (fields, index_mapping) = if columns.is_empty() { + // Get insert fields and target table's value indices + // + // if value_indices[i] = Some(j), it means that the value of the i-th target table's column is + // derived from the j-th output of the source. + // + // if value_indices[i] = None, it means that the value of the i-th target table's column is + // not provided, and should be filled with a default value later. + let (fields, value_indices) = if columns.is_empty() { // Empty means we're inserting into all columns of the table ( table_schema.fields().clone(), @@ -1098,7 +1103,7 @@ impl<'a, S: ContextProvider> SqlToRel<'a, S> { .collect::>(), ) } else { - let mut mapping = vec![None; table_schema.fields().len()]; + let mut value_indices = vec![None; table_schema.fields().len()]; let fields = columns .into_iter() .map(|c| self.normalizer.normalize(c)) @@ -1107,19 +1112,19 @@ impl<'a, S: ContextProvider> SqlToRel<'a, S> { let column_index = table_schema .index_of_column_by_name(None, &c)? .ok_or_else(|| unqualified_field_not_found(&c, &table_schema))?; - if mapping[column_index].is_some() { + if value_indices[column_index].is_some() { return Err(DataFusionError::SchemaError( datafusion_common::SchemaError::DuplicateUnqualifiedField { name: c, }, )); } else { - mapping[column_index] = Some(i); + value_indices[column_index] = Some(i); } Ok(table_schema.field(column_index).clone()) }) .collect::>>()?; - (fields, mapping) + (fields, value_indices) }; // infer types for Values clause... other types should be resolvable the regular way @@ -1154,17 +1159,22 @@ impl<'a, S: ContextProvider> SqlToRel<'a, S> { plan_err!("Column count doesn't match insert query!")?; } - let exprs = index_mapping + let exprs = value_indices .into_iter() - .flatten() - .map(|i| { - let target_field = &fields[i]; - let source_field = source.schema().field(i); - let expr = - datafusion_expr::Expr::Column(source_field.unqualified_column()) - .cast_to(target_field.data_type(), source.schema())? - .alias(target_field.name()); - Ok(expr) + .enumerate() + .map(|(i, value_index)| { + let target_field = table_schema.field(i); + let expr = match value_index { + Some(v) => { + let source_field = source.schema().field(v); + datafusion_expr::Expr::Column(source_field.qualified_column()) + .cast_to(target_field.data_type(), source.schema())? + } + // Fill the default value for the column, currently only supports NULL. + None => datafusion_expr::Expr::Literal(ScalarValue::Null) + .cast_to(target_field.data_type(), &DFSchema::empty())?, + }; + Ok(expr.alias(target_field.name())) }) .collect::>>()?; let source = project(source, exprs)?; diff --git a/datafusion/sql/tests/sql_integration.rs b/datafusion/sql/tests/sql_integration.rs index ff6dca7eef2a..4c2bad1c719e 100644 --- a/datafusion/sql/tests/sql_integration.rs +++ b/datafusion/sql/tests/sql_integration.rs @@ -422,12 +422,11 @@ CopyTo: format=csv output_url=output.csv single_file_output=true options: () fn plan_insert() { let sql = "insert into person (id, first_name, last_name) values (1, 'Alan', 'Turing')"; - let plan = r#" -Dml: op=[Insert Into] table=[person] - Projection: CAST(column1 AS UInt32) AS id, column2 AS first_name, column3 AS last_name - Values: (Int64(1), Utf8("Alan"), Utf8("Turing")) - "# - .trim(); + let plan = "Dml: op=[Insert Into] table=[person]\ + \n Projection: CAST(column1 AS UInt32) AS id, column2 AS first_name, column3 AS last_name, \ + CAST(NULL AS Int32) AS age, CAST(NULL AS Utf8) AS state, CAST(NULL AS Float64) AS salary, \ + CAST(NULL AS Timestamp(Nanosecond, None)) AS birth_date, CAST(NULL AS Int32) AS 😀\ + \n Values: (Int64(1), Utf8(\"Alan\"), Utf8(\"Turing\"))"; quick_test(sql, plan); } @@ -4037,12 +4036,11 @@ Dml: op=[Update] table=[person] fn test_prepare_statement_insert_infer() { let sql = "insert into person (id, first_name, last_name) values ($1, $2, $3)"; - let expected_plan = r#" -Dml: op=[Insert Into] table=[person] - Projection: column1 AS id, column2 AS first_name, column3 AS last_name - Values: ($1, $2, $3) - "# - .trim(); + let expected_plan = "Dml: op=[Insert Into] table=[person]\ + \n Projection: column1 AS id, column2 AS first_name, column3 AS last_name, \ + CAST(NULL AS Int32) AS age, CAST(NULL AS Utf8) AS state, CAST(NULL AS Float64) AS salary, \ + CAST(NULL AS Timestamp(Nanosecond, None)) AS birth_date, CAST(NULL AS Int32) AS 😀\ + \n Values: ($1, $2, $3)"; let expected_dt = "[Int32]"; let plan = prepare_stmt_quick_test(sql, expected_plan, expected_dt); @@ -4061,12 +4059,11 @@ Dml: op=[Insert Into] table=[person] ScalarValue::Utf8(Some("Alan".to_string())), ScalarValue::Utf8(Some("Turing".to_string())), ]; - let expected_plan = r#" -Dml: op=[Insert Into] table=[person] - Projection: column1 AS id, column2 AS first_name, column3 AS last_name - Values: (UInt32(1), Utf8("Alan"), Utf8("Turing")) - "# - .trim(); + let expected_plan = "Dml: op=[Insert Into] table=[person]\ + \n Projection: column1 AS id, column2 AS first_name, column3 AS last_name, \ + CAST(NULL AS Int32) AS age, CAST(NULL AS Utf8) AS state, CAST(NULL AS Float64) AS salary, \ + CAST(NULL AS Timestamp(Nanosecond, None)) AS birth_date, CAST(NULL AS Int32) AS 😀\ + \n Values: (UInt32(1), Utf8(\"Alan\"), Utf8(\"Turing\"))"; let plan = plan.replace_params_with_values(¶m_values).unwrap(); prepare_stmt_replace_params_quick_test(plan, param_values, expected_plan); diff --git a/datafusion/sqllogictest/test_files/insert.slt b/datafusion/sqllogictest/test_files/insert.slt index 8b9fd52e0d94..9860bdcae05c 100644 --- a/datafusion/sqllogictest/test_files/insert.slt +++ b/datafusion/sqllogictest/test_files/insert.slt @@ -258,14 +258,18 @@ insert into table_without_values(name, id) values(4, 'zoo'); statement error Error during planning: Column count doesn't match insert query! insert into table_without_values(id) values(4, 'zoo'); -statement error Error during planning: Inserting query must have the same schema with the table. +# insert NULL values for the missing column (name) +query IT insert into table_without_values(id) values(4); +---- +1 query IT rowsort select * from table_without_values; ---- 1 foo 2 bar +4 NULL statement ok drop table table_without_values; @@ -285,6 +289,16 @@ insert into table_without_values values(2, NULL); ---- 1 +# insert NULL values for the missing column (field2) +query II +insert into table_without_values(field1) values(3); +---- +1 + +# insert NULL values for the missing column (field1), but column is non-nullable +statement error Execution error: Invalid batch column at '0' has null but schema specifies non-nullable +insert into table_without_values(field2) values(300); + statement error Execution error: Invalid batch column at '0' has null but schema specifies non-nullable insert into table_without_values values(NULL, 300); @@ -296,6 +310,7 @@ select * from table_without_values; ---- 1 100 2 NULL +3 NULL statement ok drop table table_without_values; diff --git a/datafusion/sqllogictest/test_files/insert_to_external.slt b/datafusion/sqllogictest/test_files/insert_to_external.slt index d6449bc2726e..44410362412c 100644 --- a/datafusion/sqllogictest/test_files/insert_to_external.slt +++ b/datafusion/sqllogictest/test_files/insert_to_external.slt @@ -468,14 +468,18 @@ insert into table_without_values(name, id) values(4, 'zoo'); statement error Error during planning: Column count doesn't match insert query! insert into table_without_values(id) values(4, 'zoo'); -statement error Error during planning: Inserting query must have the same schema with the table. +# insert NULL values for the missing column (name) +query IT insert into table_without_values(id) values(4); +---- +1 query IT rowsort select * from table_without_values; ---- 1 foo 2 bar +4 NULL statement ok drop table table_without_values; @@ -498,6 +502,16 @@ insert into table_without_values values(2, NULL); ---- 1 +# insert NULL values for the missing column (field2) +query II +insert into table_without_values(field1) values(3); +---- +1 + +# insert NULL values for the missing column (field1), but column is non-nullable +statement error Execution error: Invalid batch column at '0' has null but schema specifies non-nullable +insert into table_without_values(field2) values(300); + statement error Execution error: Invalid batch column at '0' has null but schema specifies non-nullable insert into table_without_values values(NULL, 300); @@ -509,6 +523,7 @@ select * from table_without_values; ---- 1 100 2 NULL +3 NULL statement ok drop table table_without_values; From eef654c3b0c22b1f845b1441320b8bb718ddd605 Mon Sep 17 00:00:00 2001 From: Jay Zhan Date: Tue, 14 Nov 2023 22:29:47 +0800 Subject: [PATCH 251/572] Introduce return type for aggregate sum (#8141) * introduce return type for aggregate sum Signed-off-by: jayzhan211 * fmt Signed-off-by: jayzhan211 * fix state field type Signed-off-by: jayzhan211 * fix state field Signed-off-by: jayzhan211 --------- Signed-off-by: jayzhan211 --- datafusion/physical-expr/src/aggregate/sum.rs | 26 +++++++++++-------- .../src/aggregate/sum_distinct.rs | 11 +++++--- 2 files changed, 22 insertions(+), 15 deletions(-) diff --git a/datafusion/physical-expr/src/aggregate/sum.rs b/datafusion/physical-expr/src/aggregate/sum.rs index d6c23d0dfafd..03f666cc4e5d 100644 --- a/datafusion/physical-expr/src/aggregate/sum.rs +++ b/datafusion/physical-expr/src/aggregate/sum.rs @@ -41,7 +41,10 @@ use datafusion_expr::Accumulator; #[derive(Debug, Clone)] pub struct Sum { name: String, + // The DataType for the input expression data_type: DataType, + // The DataType for the final sum + return_type: DataType, expr: Arc, nullable: bool, } @@ -53,11 +56,12 @@ impl Sum { name: impl Into, data_type: DataType, ) -> Self { - let data_type = sum_return_type(&data_type).unwrap(); + let return_type = sum_return_type(&data_type).unwrap(); Self { name: name.into(), - expr, data_type, + return_type, + expr, nullable: true, } } @@ -70,13 +74,13 @@ impl Sum { /// `s` is a `Sum`, `helper` is a macro accepting (ArrowPrimitiveType, DataType) macro_rules! downcast_sum { ($s:ident, $helper:ident) => { - match $s.data_type { - DataType::UInt64 => $helper!(UInt64Type, $s.data_type), - DataType::Int64 => $helper!(Int64Type, $s.data_type), - DataType::Float64 => $helper!(Float64Type, $s.data_type), - DataType::Decimal128(_, _) => $helper!(Decimal128Type, $s.data_type), - DataType::Decimal256(_, _) => $helper!(Decimal256Type, $s.data_type), - _ => not_impl_err!("Sum not supported for {}: {}", $s.name, $s.data_type), + match $s.return_type { + DataType::UInt64 => $helper!(UInt64Type, $s.return_type), + DataType::Int64 => $helper!(Int64Type, $s.return_type), + DataType::Float64 => $helper!(Float64Type, $s.return_type), + DataType::Decimal128(_, _) => $helper!(Decimal128Type, $s.return_type), + DataType::Decimal256(_, _) => $helper!(Decimal256Type, $s.return_type), + _ => not_impl_err!("Sum not supported for {}: {}", $s.name, $s.return_type), } }; } @@ -91,7 +95,7 @@ impl AggregateExpr for Sum { fn field(&self) -> Result { Ok(Field::new( &self.name, - self.data_type.clone(), + self.return_type.clone(), self.nullable, )) } @@ -108,7 +112,7 @@ impl AggregateExpr for Sum { fn state_fields(&self) -> Result> { Ok(vec![Field::new( format_state_name(&self.name, "sum"), - self.data_type.clone(), + self.return_type.clone(), self.nullable, )]) } diff --git a/datafusion/physical-expr/src/aggregate/sum_distinct.rs b/datafusion/physical-expr/src/aggregate/sum_distinct.rs index ef1bd039a5ea..0cf4a90ab8cc 100644 --- a/datafusion/physical-expr/src/aggregate/sum_distinct.rs +++ b/datafusion/physical-expr/src/aggregate/sum_distinct.rs @@ -40,8 +40,10 @@ use datafusion_expr::Accumulator; pub struct DistinctSum { /// Column name name: String, - /// The DataType for the final sum + // The DataType for the input expression data_type: DataType, + // The DataType for the final sum + return_type: DataType, /// The input arguments, only contains 1 item for sum exprs: Vec>, } @@ -53,10 +55,11 @@ impl DistinctSum { name: String, data_type: DataType, ) -> Self { - let data_type = sum_return_type(&data_type).unwrap(); + let return_type = sum_return_type(&data_type).unwrap(); Self { name, data_type, + return_type, exprs, } } @@ -68,14 +71,14 @@ impl AggregateExpr for DistinctSum { } fn field(&self) -> Result { - Ok(Field::new(&self.name, self.data_type.clone(), true)) + Ok(Field::new(&self.name, self.return_type.clone(), true)) } fn state_fields(&self) -> Result> { // State field is a List which stores items to rebuild hash set. Ok(vec![Field::new_list( format_state_name(&self.name, "sum distinct"), - Field::new("item", self.data_type.clone(), true), + Field::new("item", self.return_type.clone(), true), false, )]) } From 31e54f00c71da3e4441cf59a5deee7fc1d2727f2 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E8=B0=AD=E5=B7=8D?= Date: Wed, 15 Nov 2023 00:57:08 +0800 Subject: [PATCH 252/572] implement range/generate_series func (#8140) * implement range/generate_series func Signed-off-by: veeupup * explain details for range func Signed-off-by: veeupup * fix ci * fix doc fmt * fix comments * regenerate proto * add comment for gen_range usage Signed-off-by: veeupup --------- Signed-off-by: veeupup Co-authored-by: Andrew Lamb --- datafusion/expr/src/built_in_function.rs | 15 ++++++ datafusion/expr/src/expr_fn.rs | 6 +++ .../physical-expr/src/array_expressions.rs | 52 +++++++++++++++++++ datafusion/physical-expr/src/functions.rs | 3 ++ datafusion/proto/proto/datafusion.proto | 1 + datafusion/proto/src/generated/pbjson.rs | 3 ++ datafusion/proto/src/generated/prost.rs | 3 ++ .../proto/src/logical_plan/from_proto.rs | 11 +++- datafusion/proto/src/logical_plan/to_proto.rs | 1 + datafusion/sqllogictest/test_files/array.slt | 36 +++++++++++++ docs/source/user-guide/expressions.md | 1 + .../source/user-guide/sql/scalar_functions.md | 15 ++++++ 12 files changed, 145 insertions(+), 2 deletions(-) diff --git a/datafusion/expr/src/built_in_function.rs b/datafusion/expr/src/built_in_function.rs index 77c64128e156..473094c00174 100644 --- a/datafusion/expr/src/built_in_function.rs +++ b/datafusion/expr/src/built_in_function.rs @@ -184,6 +184,8 @@ pub enum BuiltinScalarFunction { MakeArray, /// Flatten Flatten, + /// Range + Range, // struct functions /// struct @@ -406,6 +408,7 @@ impl BuiltinScalarFunction { BuiltinScalarFunction::ArrayToString => Volatility::Immutable, BuiltinScalarFunction::ArrayIntersect => Volatility::Immutable, BuiltinScalarFunction::ArrayUnion => Volatility::Immutable, + BuiltinScalarFunction::Range => Volatility::Immutable, BuiltinScalarFunction::Cardinality => Volatility::Immutable, BuiltinScalarFunction::MakeArray => Volatility::Immutable, BuiltinScalarFunction::Ascii => Volatility::Immutable, @@ -588,6 +591,9 @@ impl BuiltinScalarFunction { BuiltinScalarFunction::ArrayToString => Ok(Utf8), BuiltinScalarFunction::ArrayIntersect => Ok(input_expr_types[0].clone()), BuiltinScalarFunction::ArrayUnion => Ok(input_expr_types[0].clone()), + BuiltinScalarFunction::Range => { + Ok(List(Arc::new(Field::new("item", Int64, true)))) + } BuiltinScalarFunction::Cardinality => Ok(UInt64), BuiltinScalarFunction::MakeArray => match input_expr_types.len() { 0 => Ok(List(Arc::new(Field::new("item", Null, true)))), @@ -902,6 +908,14 @@ impl BuiltinScalarFunction { // 0 or more arguments of arbitrary type Signature::one_of(vec![VariadicAny, Any(0)], self.volatility()) } + BuiltinScalarFunction::Range => Signature::one_of( + vec![ + Exact(vec![Int64]), + Exact(vec![Int64, Int64]), + Exact(vec![Int64, Int64, Int64]), + ], + self.volatility(), + ), BuiltinScalarFunction::Struct => Signature::variadic( struct_expressions::SUPPORTED_STRUCT_TYPES.to_vec(), self.volatility(), @@ -1533,6 +1547,7 @@ fn aliases(func: &BuiltinScalarFunction) -> &'static [&'static str] { BuiltinScalarFunction::MakeArray => &["make_array", "make_list"], BuiltinScalarFunction::ArrayIntersect => &["array_intersect", "list_intersect"], BuiltinScalarFunction::OverLay => &["overlay"], + BuiltinScalarFunction::Range => &["range", "generate_series"], // struct functions BuiltinScalarFunction::Struct => &["struct"], diff --git a/datafusion/expr/src/expr_fn.rs b/datafusion/expr/src/expr_fn.rs index 91674cc092e6..e70a4a90f767 100644 --- a/datafusion/expr/src/expr_fn.rs +++ b/datafusion/expr/src/expr_fn.rs @@ -737,6 +737,12 @@ scalar_expr!( "Returns an array of the elements in the intersection of array1 and array2." ); +nary_scalar_expr!( + Range, + gen_range, + "Returns a list of values in the range between start and stop with step." +); + // string functions scalar_expr!(Ascii, ascii, chr, "ASCII code value of the character"); scalar_expr!( diff --git a/datafusion/physical-expr/src/array_expressions.rs b/datafusion/physical-expr/src/array_expressions.rs index 9b074ff0ee0d..6415bd5391d5 100644 --- a/datafusion/physical-expr/src/array_expressions.rs +++ b/datafusion/physical-expr/src/array_expressions.rs @@ -643,6 +643,58 @@ fn general_append_and_prepend( )?)) } +/// Generates an array of integers from start to stop with a given step. +/// +/// This function takes 1 to 3 ArrayRefs as arguments, representing start, stop, and step values. +/// It returns a `Result` representing the resulting ListArray after the operation. +/// +/// # Arguments +/// +/// * `args` - An array of 1 to 3 ArrayRefs representing start, stop, and step(step value can not be zero.) values. +/// +/// # Examples +/// +/// gen_range(3) => [0, 1, 2] +/// gen_range(1, 4) => [1, 2, 3] +/// gen_range(1, 7, 2) => [1, 3, 5] +pub fn gen_range(args: &[ArrayRef]) -> Result { + let (start_array, stop_array, step_array) = match args.len() { + 1 => (None, as_int64_array(&args[0])?, None), + 2 => ( + Some(as_int64_array(&args[0])?), + as_int64_array(&args[1])?, + None, + ), + 3 => ( + Some(as_int64_array(&args[0])?), + as_int64_array(&args[1])?, + Some(as_int64_array(&args[2])?), + ), + _ => return internal_err!("gen_range expects 1 to 3 arguments"), + }; + + let mut values = vec![]; + let mut offsets = vec![0]; + for (idx, stop) in stop_array.iter().enumerate() { + let stop = stop.unwrap_or(0); + let start = start_array.as_ref().map(|arr| arr.value(idx)).unwrap_or(0); + let step = step_array.as_ref().map(|arr| arr.value(idx)).unwrap_or(1); + if step == 0 { + return exec_err!("step can't be 0 for function range(start [, stop, step]"); + } + let value = (start..stop).step_by(step as usize); + values.extend(value); + offsets.push(values.len() as i32); + } + let arr = Arc::new(ListArray::try_new( + Arc::new(Field::new("item", DataType::Int64, true)), + OffsetBuffer::new(offsets.into()), + Arc::new(Int64Array::from(values)), + None, + )?); + Ok(arr) +} + /// Array_append SQL function pub fn array_append(args: &[ArrayRef]) -> Result { let list_array = as_list_array(&args[0])?; diff --git a/datafusion/physical-expr/src/functions.rs b/datafusion/physical-expr/src/functions.rs index 7f8921e86c38..799127c95c98 100644 --- a/datafusion/physical-expr/src/functions.rs +++ b/datafusion/physical-expr/src/functions.rs @@ -401,6 +401,9 @@ pub fn create_physical_fun( BuiltinScalarFunction::ArrayIntersect => Arc::new(|args| { make_scalar_function(array_expressions::array_intersect)(args) }), + BuiltinScalarFunction::Range => { + Arc::new(|args| make_scalar_function(array_expressions::gen_range)(args)) + } BuiltinScalarFunction::Cardinality => { Arc::new(|args| make_scalar_function(array_expressions::cardinality)(args)) } diff --git a/datafusion/proto/proto/datafusion.proto b/datafusion/proto/proto/datafusion.proto index d85678a76bf1..fa080518d50c 100644 --- a/datafusion/proto/proto/datafusion.proto +++ b/datafusion/proto/proto/datafusion.proto @@ -637,6 +637,7 @@ enum ScalarFunction { ArrayIntersect = 119; ArrayUnion = 120; OverLay = 121; + Range = 122; } message ScalarFunctionNode { diff --git a/datafusion/proto/src/generated/pbjson.rs b/datafusion/proto/src/generated/pbjson.rs index 64db9137d64f..08e7413102e8 100644 --- a/datafusion/proto/src/generated/pbjson.rs +++ b/datafusion/proto/src/generated/pbjson.rs @@ -20936,6 +20936,7 @@ impl serde::Serialize for ScalarFunction { Self::ArrayIntersect => "ArrayIntersect", Self::ArrayUnion => "ArrayUnion", Self::OverLay => "OverLay", + Self::Range => "Range", }; serializer.serialize_str(variant) } @@ -21069,6 +21070,7 @@ impl<'de> serde::Deserialize<'de> for ScalarFunction { "ArrayIntersect", "ArrayUnion", "OverLay", + "Range", ]; struct GeneratedVisitor; @@ -21231,6 +21233,7 @@ impl<'de> serde::Deserialize<'de> for ScalarFunction { "ArrayIntersect" => Ok(ScalarFunction::ArrayIntersect), "ArrayUnion" => Ok(ScalarFunction::ArrayUnion), "OverLay" => Ok(ScalarFunction::OverLay), + "Range" => Ok(ScalarFunction::Range), _ => Err(serde::de::Error::unknown_variant(value, FIELDS)), } } diff --git a/datafusion/proto/src/generated/prost.rs b/datafusion/proto/src/generated/prost.rs index 131ca11993c1..15606488b33a 100644 --- a/datafusion/proto/src/generated/prost.rs +++ b/datafusion/proto/src/generated/prost.rs @@ -2568,6 +2568,7 @@ pub enum ScalarFunction { ArrayIntersect = 119, ArrayUnion = 120, OverLay = 121, + Range = 122, } impl ScalarFunction { /// String value of the enum field names used in the ProtoBuf definition. @@ -2698,6 +2699,7 @@ impl ScalarFunction { ScalarFunction::ArrayIntersect => "ArrayIntersect", ScalarFunction::ArrayUnion => "ArrayUnion", ScalarFunction::OverLay => "OverLay", + ScalarFunction::Range => "Range", } } /// Creates an enum from field names used in the ProtoBuf definition. @@ -2825,6 +2827,7 @@ impl ScalarFunction { "ArrayIntersect" => Some(Self::ArrayIntersect), "ArrayUnion" => Some(Self::ArrayUnion), "OverLay" => Some(Self::OverLay), + "Range" => Some(Self::Range), _ => None, } } diff --git a/datafusion/proto/src/logical_plan/from_proto.rs b/datafusion/proto/src/logical_plan/from_proto.rs index 9ca7bb0e893a..b3d68570038c 100644 --- a/datafusion/proto/src/logical_plan/from_proto.rs +++ b/datafusion/proto/src/logical_plan/from_proto.rs @@ -49,8 +49,8 @@ use datafusion_expr::{ concat_expr, concat_ws_expr, cos, cosh, cot, current_date, current_time, date_bin, date_part, date_trunc, decode, degrees, digest, encode, exp, expr::{self, InList, Sort, WindowFunction}, - factorial, flatten, floor, from_unixtime, gcd, isnan, iszero, lcm, left, ln, log, - log10, log2, + factorial, flatten, floor, from_unixtime, gcd, gen_range, isnan, iszero, lcm, left, + ln, log, log10, log2, logical_plan::{PlanType, StringifiedPlan}, lower, lpad, ltrim, md5, nanvl, now, nullif, octet_length, overlay, pi, power, radians, random, regexp_match, regexp_replace, repeat, replace, reverse, right, @@ -488,6 +488,7 @@ impl From<&protobuf::ScalarFunction> for BuiltinScalarFunction { ScalarFunction::ArrayToString => Self::ArrayToString, ScalarFunction::ArrayIntersect => Self::ArrayIntersect, ScalarFunction::ArrayUnion => Self::ArrayUnion, + ScalarFunction::Range => Self::Range, ScalarFunction::Cardinality => Self::Cardinality, ScalarFunction::Array => Self::MakeArray, ScalarFunction::NullIf => Self::NullIf, @@ -1409,6 +1410,12 @@ pub fn parse_expr( parse_expr(&args[0], registry)?, parse_expr(&args[1], registry)?, )), + ScalarFunction::Range => Ok(gen_range( + args.to_owned() + .iter() + .map(|expr| parse_expr(expr, registry)) + .collect::, _>>()?, + )), ScalarFunction::Cardinality => { Ok(cardinality(parse_expr(&args[0], registry)?)) } diff --git a/datafusion/proto/src/logical_plan/to_proto.rs b/datafusion/proto/src/logical_plan/to_proto.rs index 974d6c5aaba8..491b7f666430 100644 --- a/datafusion/proto/src/logical_plan/to_proto.rs +++ b/datafusion/proto/src/logical_plan/to_proto.rs @@ -1495,6 +1495,7 @@ impl TryFrom<&BuiltinScalarFunction> for protobuf::ScalarFunction { BuiltinScalarFunction::ArrayToString => Self::ArrayToString, BuiltinScalarFunction::ArrayIntersect => Self::ArrayIntersect, BuiltinScalarFunction::ArrayUnion => Self::ArrayUnion, + BuiltinScalarFunction::Range => Self::Range, BuiltinScalarFunction::Cardinality => Self::Cardinality, BuiltinScalarFunction::MakeArray => Self::Array, BuiltinScalarFunction::NullIf => Self::NullIf, diff --git a/datafusion/sqllogictest/test_files/array.slt b/datafusion/sqllogictest/test_files/array.slt index 54741afdf83a..92013f37d36c 100644 --- a/datafusion/sqllogictest/test_files/array.slt +++ b/datafusion/sqllogictest/test_files/array.slt @@ -240,6 +240,13 @@ AS VALUES (make_array(31, 32, 33, 34, 35, 26, 37, 38, 39, 40), 34, 4, 'ok', [8,9]) ; +statement ok +CREATE TABLE arrays_range +AS VALUES + (3, 10, 2), + (4, 13, 3) +; + statement ok CREATE TABLE arrays_with_repeating_elements AS VALUES @@ -2662,6 +2669,32 @@ select list_has_all(make_array(1,2,3), make_array(4,5,6)), ---- false true false true +query ??? +select range(column2), + range(column1, column2), + range(column1, column2, column3) +from arrays_range; +---- +[0, 1, 2, 3, 4, 5, 6, 7, 8, 9] [3, 4, 5, 6, 7, 8, 9] [3, 5, 7, 9] +[0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12] [4, 5, 6, 7, 8, 9, 10, 11, 12] [4, 7, 10] + +query ????? +select range(5), + range(2, 5), + range(2, 10, 3), + range(1, 5, -1), + range(1, -5, 1) +; +---- +[0, 1, 2, 3, 4] [2, 3, 4] [2, 5, 8] [1] [] + +query ??? +select generate_series(5), + generate_series(2, 5), + generate_series(2, 10, 3) +; +---- +[0, 1, 2, 3, 4] [2, 3, 4] [2, 5, 8] ### Array operators tests @@ -2969,6 +3002,9 @@ drop table array_intersect_table_3D; statement ok drop table arrays_values_without_nulls; +statement ok +drop table arrays_range; + statement ok drop table arrays_with_repeating_elements; diff --git a/docs/source/user-guide/expressions.md b/docs/source/user-guide/expressions.md index bec3ba9bb28c..6b2ab46eb343 100644 --- a/docs/source/user-guide/expressions.md +++ b/docs/source/user-guide/expressions.md @@ -236,6 +236,7 @@ Unlike to some databases the math functions in Datafusion works the same way as | array_union(array1, array2) | Returns an array of the elements in the union of array1 and array2 without duplicates. `array_union([1, 2, 3, 4], [5, 6, 3, 4]) -> [1, 2, 3, 4, 5, 6]` | | cardinality(array) | Returns the total number of elements in the array. `cardinality([[1, 2, 3], [4, 5, 6]]) -> 6` | | make_array(value1, [value2 [, ...]]) | Returns an Arrow array using the specified input expressions. `make_array(1, 2, 3) -> [1, 2, 3]` | +| range(start [, stop, step]) | Returns an Arrow array between start and stop with step. `SELECT range(2, 10, 3) -> [2, 5, 8]` | | trim_array(array, n) | Deprecated | ## Regular Expressions diff --git a/docs/source/user-guide/sql/scalar_functions.md b/docs/source/user-guide/sql/scalar_functions.md index 099c90312227..826782e1a051 100644 --- a/docs/source/user-guide/sql/scalar_functions.md +++ b/docs/source/user-guide/sql/scalar_functions.md @@ -1560,6 +1560,7 @@ from_unixtime(expression) - [string_to_array](#string_to_array) - [string_to_list](#string_to_list) - [trim_array](#trim_array) +- [range](#range) ### `array_append` @@ -2481,6 +2482,20 @@ trim_array(array, n) Can be a constant, column, or function, and any combination of array operators. - **n**: Element to trim the array. +### `range` + +Returns an Arrow array between start and stop with step. `SELECT range(2, 10, 3) -> [2, 5, 8]` + +The range start..end contains all values with start <= x < end. It is empty if start >= end. + +Step can not be 0 (then the range will be nonsense.). + +#### Arguments + +- **start**: start of the range +- **end**: end of the range (not included) +- **step**: increase by step (can not be 0) + ## Struct Functions - [struct](#struct) From f390f159ac6f77d66adcf8409442f74177dbdd64 Mon Sep 17 00:00:00 2001 From: Andrew Lamb Date: Tue, 14 Nov 2023 13:41:38 -0500 Subject: [PATCH 253/572] Encapsulate `EquivalenceClass` into a struct (#8034) * Minor: Encapsulate EquivalenceClass * Rename inner to exprs * Rename new_from_vec to new * Apply suggestions from code review Co-authored-by: Mehmet Ozan Kabak * treat as set rather than vec * fmt * clippy --------- Co-authored-by: Mehmet Ozan Kabak --- datafusion/physical-expr/src/equivalence.rs | 235 +++++++++++++----- datafusion/physical-expr/src/physical_expr.rs | 35 +-- 2 files changed, 177 insertions(+), 93 deletions(-) diff --git a/datafusion/physical-expr/src/equivalence.rs b/datafusion/physical-expr/src/equivalence.rs index d8aa09b90460..84291653fb4f 100644 --- a/datafusion/physical-expr/src/equivalence.rs +++ b/datafusion/physical-expr/src/equivalence.rs @@ -20,11 +20,11 @@ use std::hash::Hash; use std::sync::Arc; use crate::expressions::Column; -use crate::physical_expr::{deduplicate_physical_exprs, have_common_entries}; use crate::sort_properties::{ExprOrdering, SortProperties}; use crate::{ - physical_exprs_contains, LexOrdering, LexOrderingRef, LexRequirement, - LexRequirementRef, PhysicalExpr, PhysicalSortExpr, PhysicalSortRequirement, + physical_exprs_bag_equal, physical_exprs_contains, LexOrdering, LexOrderingRef, + LexRequirement, LexRequirementRef, PhysicalExpr, PhysicalSortExpr, + PhysicalSortRequirement, }; use arrow::datatypes::SchemaRef; @@ -32,14 +32,110 @@ use arrow_schema::SortOptions; use datafusion_common::tree_node::{Transformed, TreeNode}; use datafusion_common::{JoinSide, JoinType, Result}; +use crate::physical_expr::deduplicate_physical_exprs; use indexmap::map::Entry; use indexmap::IndexMap; /// An `EquivalenceClass` is a set of [`Arc`]s that are known /// to have the same value for all tuples in a relation. These are generated by -/// equality predicates, typically equi-join conditions and equality conditions -/// in filters. -pub type EquivalenceClass = Vec>; +/// equality predicates (e.g. `a = b`), typically equi-join conditions and +/// equality conditions in filters. +/// +/// Two `EquivalenceClass`es are equal if they contains the same expressions in +/// without any ordering. +#[derive(Debug, Clone)] +pub struct EquivalenceClass { + /// The expressions in this equivalence class. The order doesn't + /// matter for equivalence purposes + /// + /// TODO: use a HashSet for this instead of a Vec + exprs: Vec>, +} + +impl PartialEq for EquivalenceClass { + /// Returns true if other is equal in the sense + /// of bags (multi-sets), disregarding their orderings. + fn eq(&self, other: &Self) -> bool { + physical_exprs_bag_equal(&self.exprs, &other.exprs) + } +} + +impl EquivalenceClass { + /// Create a new empty equivalence class + pub fn new_empty() -> Self { + Self { exprs: vec![] } + } + + // Create a new equivalence class from a pre-existing `Vec` + pub fn new(mut exprs: Vec>) -> Self { + deduplicate_physical_exprs(&mut exprs); + Self { exprs } + } + + /// Return the inner vector of expressions + pub fn into_vec(self) -> Vec> { + self.exprs + } + + /// Return the "canonical" expression for this class (the first element) + /// if any + fn canonical_expr(&self) -> Option> { + self.exprs.first().cloned() + } + + /// Insert the expression into this class, meaning it is known to be equal to + /// all other expressions in this class + pub fn push(&mut self, expr: Arc) { + if !self.contains(&expr) { + self.exprs.push(expr); + } + } + + /// Inserts all the expressions from other into this class + pub fn extend(&mut self, other: Self) { + for expr in other.exprs { + // use push so entries are deduplicated + self.push(expr); + } + } + + /// Returns true if this equivalence class contains t expression + pub fn contains(&self, expr: &Arc) -> bool { + physical_exprs_contains(&self.exprs, expr) + } + + /// Returns true if this equivalence class has any entries in common with `other` + pub fn contains_any(&self, other: &Self) -> bool { + self.exprs.iter().any(|e| other.contains(e)) + } + + /// return the number of items in this class + pub fn len(&self) -> usize { + self.exprs.len() + } + + /// return true if this class is empty + pub fn is_empty(&self) -> bool { + self.exprs.is_empty() + } + + /// Iterate over all elements in this class, in some arbitrary order + pub fn iter(&self) -> impl Iterator> { + self.exprs.iter() + } + + /// Return a new equivalence class that have the specified offset added to + /// each expression (used when schemas are appended such as in joins) + pub fn with_offset(&self, offset: usize) -> Self { + let new_exprs = self + .exprs + .iter() + .cloned() + .map(|e| add_offset_to_expr(e, offset)) + .collect(); + Self::new(new_exprs) + } +} /// Stores the mapping between source expressions and target expressions for a /// projection. @@ -148,10 +244,10 @@ impl EquivalenceGroup { let mut first_class = None; let mut second_class = None; for (idx, cls) in self.classes.iter().enumerate() { - if physical_exprs_contains(cls, left) { + if cls.contains(left) { first_class = Some(idx); } - if physical_exprs_contains(cls, right) { + if cls.contains(right) { second_class = Some(idx); } } @@ -181,7 +277,8 @@ impl EquivalenceGroup { (None, None) => { // None of the expressions is among existing classes. // Create a new equivalence class and extend the group. - self.classes.push(vec![left.clone(), right.clone()]); + self.classes + .push(EquivalenceClass::new(vec![left.clone(), right.clone()])); } } } @@ -192,7 +289,6 @@ impl EquivalenceGroup { self.classes.retain_mut(|cls| { // Keep groups that have at least two entries as singleton class is // meaningless (i.e. it contains no non-trivial information): - deduplicate_physical_exprs(cls); cls.len() > 1 }); // Unify/bridge groups that have common expressions: @@ -209,7 +305,7 @@ impl EquivalenceGroup { let mut next_idx = idx + 1; let start_size = self.classes[idx].len(); while next_idx < self.classes.len() { - if have_common_entries(&self.classes[idx], &self.classes[next_idx]) { + if self.classes[idx].contains_any(&self.classes[next_idx]) { let extension = self.classes.swap_remove(next_idx); self.classes[idx].extend(extension); } else { @@ -217,10 +313,7 @@ impl EquivalenceGroup { } } if self.classes[idx].len() > start_size { - deduplicate_physical_exprs(&mut self.classes[idx]); - if self.classes[idx].len() > start_size { - continue; - } + continue; } idx += 1; } @@ -239,8 +332,8 @@ impl EquivalenceGroup { expr.clone() .transform(&|expr| { for cls in self.iter() { - if physical_exprs_contains(cls, &expr) { - return Ok(Transformed::Yes(cls[0].clone())); + if cls.contains(&expr) { + return Ok(Transformed::Yes(cls.canonical_expr().unwrap())); } } Ok(Transformed::No(expr)) @@ -330,7 +423,7 @@ impl EquivalenceGroup { if source.eq(expr) || self .get_equivalence_class(source) - .map_or(false, |group| physical_exprs_contains(group, expr)) + .map_or(false, |group| group.contains(expr)) { return Some(target.clone()); } @@ -380,7 +473,7 @@ impl EquivalenceGroup { .iter() .filter_map(|expr| self.project_expr(mapping, expr)) .collect::>(); - (new_class.len() > 1).then_some(new_class) + (new_class.len() > 1).then_some(EquivalenceClass::new(new_class)) }); // TODO: Convert the algorithm below to a version that uses `HashMap`. // once `Arc` can be stored in `HashMap`. @@ -402,7 +495,9 @@ impl EquivalenceGroup { // equivalence classes are meaningless. let new_classes = new_classes .into_iter() - .filter_map(|(_, values)| (values.len() > 1).then_some(values)); + .filter_map(|(_, values)| (values.len() > 1).then_some(values)) + .map(EquivalenceClass::new); + let classes = projected_classes.chain(new_classes).collect(); Self::new(classes) } @@ -412,10 +507,8 @@ impl EquivalenceGroup { fn get_equivalence_class( &self, expr: &Arc, - ) -> Option<&[Arc]> { - self.iter() - .map(|cls| cls.as_slice()) - .find(|cls| physical_exprs_contains(cls, expr)) + ) -> Option<&EquivalenceClass> { + self.iter().find(|cls| cls.contains(expr)) } /// Combine equivalence groups of the given join children. @@ -431,12 +524,11 @@ impl EquivalenceGroup { let mut result = Self::new( self.iter() .cloned() - .chain(right_equivalences.iter().map(|item| { - item.iter() - .cloned() - .map(|expr| add_offset_to_expr(expr, left_size)) - .collect() - })) + .chain( + right_equivalences + .iter() + .map(|cls| cls.with_offset(left_size)), + ) .collect(), ); // In we have an inner join, expressions in the "on" condition @@ -1246,14 +1338,13 @@ mod tests { use std::sync::Arc; use super::*; - use crate::expressions::{col, lit, BinaryExpr, Column}; - use crate::physical_expr::{physical_exprs_bag_equal, physical_exprs_equal}; + use crate::expressions::{col, lit, BinaryExpr, Column, Literal}; use arrow::compute::{lexsort_to_indices, SortColumn}; use arrow::datatypes::{DataType, Field, Schema}; use arrow_array::{ArrayRef, RecordBatch, UInt32Array, UInt64Array}; use arrow_schema::{Fields, SortOptions}; - use datafusion_common::Result; + use datafusion_common::{Result, ScalarValue}; use datafusion_expr::Operator; use itertools::{izip, Itertools}; @@ -1440,8 +1531,8 @@ mod tests { assert_eq!(eq_properties.eq_group().len(), 1); let eq_groups = &eq_properties.eq_group().classes[0]; assert_eq!(eq_groups.len(), 2); - assert!(physical_exprs_contains(eq_groups, &col_a_expr)); - assert!(physical_exprs_contains(eq_groups, &col_b_expr)); + assert!(eq_groups.contains(&col_a_expr)); + assert!(eq_groups.contains(&col_b_expr)); // b and c are aliases. Exising equivalence class should expand, // however there shouldn't be any new equivalence class @@ -1449,9 +1540,9 @@ mod tests { assert_eq!(eq_properties.eq_group().len(), 1); let eq_groups = &eq_properties.eq_group().classes[0]; assert_eq!(eq_groups.len(), 3); - assert!(physical_exprs_contains(eq_groups, &col_a_expr)); - assert!(physical_exprs_contains(eq_groups, &col_b_expr)); - assert!(physical_exprs_contains(eq_groups, &col_c_expr)); + assert!(eq_groups.contains(&col_a_expr)); + assert!(eq_groups.contains(&col_b_expr)); + assert!(eq_groups.contains(&col_c_expr)); // This is a new set of equality. Hence equivalent class count should be 2. eq_properties.add_equal_conditions(&col_x_expr, &col_y_expr); @@ -1463,11 +1554,11 @@ mod tests { assert_eq!(eq_properties.eq_group().len(), 1); let eq_groups = &eq_properties.eq_group().classes[0]; assert_eq!(eq_groups.len(), 5); - assert!(physical_exprs_contains(eq_groups, &col_a_expr)); - assert!(physical_exprs_contains(eq_groups, &col_b_expr)); - assert!(physical_exprs_contains(eq_groups, &col_c_expr)); - assert!(physical_exprs_contains(eq_groups, &col_x_expr)); - assert!(physical_exprs_contains(eq_groups, &col_y_expr)); + assert!(eq_groups.contains(&col_a_expr)); + assert!(eq_groups.contains(&col_b_expr)); + assert!(eq_groups.contains(&col_c_expr)); + assert!(eq_groups.contains(&col_x_expr)); + assert!(eq_groups.contains(&col_y_expr)); Ok(()) } @@ -1509,10 +1600,10 @@ mod tests { assert_eq!(out_properties.eq_group().len(), 1); let eq_class = &out_properties.eq_group().classes[0]; assert_eq!(eq_class.len(), 4); - assert!(physical_exprs_contains(eq_class, col_a1)); - assert!(physical_exprs_contains(eq_class, col_a2)); - assert!(physical_exprs_contains(eq_class, col_a3)); - assert!(physical_exprs_contains(eq_class, col_a4)); + assert!(eq_class.contains(col_a1)); + assert!(eq_class.contains(col_a2)); + assert!(eq_class.contains(col_a3)); + assert!(eq_class.contains(col_a4)); Ok(()) } @@ -1852,10 +1943,12 @@ mod tests { let entries = entries .into_iter() .map(|entry| entry.into_iter().map(lit).collect::>()) + .map(EquivalenceClass::new) .collect::>(); let expected = expected .into_iter() .map(|entry| entry.into_iter().map(lit).collect::>()) + .map(EquivalenceClass::new) .collect::>(); let mut eq_groups = EquivalenceGroup::new(entries.clone()); eq_groups.bridge_classes(); @@ -1866,11 +1959,7 @@ mod tests { ); assert_eq!(eq_groups.len(), expected.len(), "{}", err_msg); for idx in 0..eq_groups.len() { - assert!( - physical_exprs_bag_equal(&eq_groups[idx], &expected[idx]), - "{}", - err_msg - ); + assert_eq!(&eq_groups[idx], &expected[idx], "{}", err_msg); } } Ok(()) @@ -1879,14 +1968,17 @@ mod tests { #[test] fn test_remove_redundant_entries_eq_group() -> Result<()> { let entries = vec![ - vec![lit(1), lit(1), lit(2)], + EquivalenceClass::new(vec![lit(1), lit(1), lit(2)]), // This group is meaningless should be removed - vec![lit(3), lit(3)], - vec![lit(4), lit(5), lit(6)], + EquivalenceClass::new(vec![lit(3), lit(3)]), + EquivalenceClass::new(vec![lit(4), lit(5), lit(6)]), ]; // Given equivalences classes are not in succinct form. // Expected form is the most plain representation that is functionally same. - let expected = vec![vec![lit(1), lit(2)], vec![lit(4), lit(5), lit(6)]]; + let expected = vec![ + EquivalenceClass::new(vec![lit(1), lit(2)]), + EquivalenceClass::new(vec![lit(4), lit(5), lit(6)]), + ]; let mut eq_groups = EquivalenceGroup::new(entries); eq_groups.remove_redundant_entries(); @@ -1894,8 +1986,8 @@ mod tests { assert_eq!(eq_groups.len(), expected.len()); assert_eq!(eq_groups.len(), 2); - assert!(physical_exprs_equal(&eq_groups[0], &expected[0])); - assert!(physical_exprs_equal(&eq_groups[1], &expected[1])); + assert_eq!(eq_groups[0], expected[0]); + assert_eq!(eq_groups[1], expected[1]); Ok(()) } @@ -2151,7 +2243,7 @@ mod tests { // expressions in the equivalence classes. For other expressions in the same // equivalence class use same result. This util gets already calculated result, when available. fn get_representative_arr( - eq_group: &[Arc], + eq_group: &EquivalenceClass, existing_vec: &[Option], schema: SchemaRef, ) -> Option { @@ -2224,7 +2316,7 @@ mod tests { get_representative_arr(eq_group, &schema_vec, schema.clone()) .unwrap_or_else(|| generate_random_array(n_elem, n_distinct)); - for expr in eq_group { + for expr in eq_group.iter() { let col = expr.as_any().downcast_ref::().unwrap(); let (idx, _field) = schema.column_with_name(col.name()).unwrap(); schema_vec[idx] = Some(representative_array.clone()); @@ -2626,6 +2718,29 @@ mod tests { Ok(()) } + #[test] + fn test_contains_any() { + let lit_true = Arc::new(Literal::new(ScalarValue::Boolean(Some(true)))) + as Arc; + let lit_false = Arc::new(Literal::new(ScalarValue::Boolean(Some(false)))) + as Arc; + let lit2 = + Arc::new(Literal::new(ScalarValue::Int32(Some(2)))) as Arc; + let lit1 = + Arc::new(Literal::new(ScalarValue::Int32(Some(1)))) as Arc; + let col_b_expr = Arc::new(Column::new("b", 1)) as Arc; + + let cls1 = EquivalenceClass::new(vec![lit_true.clone(), lit_false.clone()]); + let cls2 = EquivalenceClass::new(vec![lit_true.clone(), col_b_expr.clone()]); + let cls3 = EquivalenceClass::new(vec![lit2.clone(), lit1.clone()]); + + // lit_true is common + assert!(cls1.contains_any(&cls2)); + // there is no common entry + assert!(!cls1.contains_any(&cls3)); + assert!(!cls2.contains_any(&cls3)); + } + #[test] fn test_get_indices_of_matching_sort_exprs_with_order_eq() -> Result<()> { let sort_options = SortOptions::default(); diff --git a/datafusion/physical-expr/src/physical_expr.rs b/datafusion/physical-expr/src/physical_expr.rs index 79cbe6828b64..455ca84a792f 100644 --- a/datafusion/physical-expr/src/physical_expr.rs +++ b/datafusion/physical-expr/src/physical_expr.rs @@ -228,14 +228,6 @@ pub fn physical_exprs_contains( .any(|physical_expr| physical_expr.eq(expr)) } -/// Checks whether the given slices have any common entries. -pub fn have_common_entries( - lhs: &[Arc], - rhs: &[Arc], -) -> bool { - lhs.iter().any(|expr| physical_exprs_contains(rhs, expr)) -} - /// Checks whether the given physical expression slices are equal. pub fn physical_exprs_equal( lhs: &[Arc], @@ -293,8 +285,8 @@ mod tests { use crate::expressions::{Column, Literal}; use crate::physical_expr::{ - deduplicate_physical_exprs, have_common_entries, physical_exprs_bag_equal, - physical_exprs_contains, physical_exprs_equal, PhysicalExpr, + deduplicate_physical_exprs, physical_exprs_bag_equal, physical_exprs_contains, + physical_exprs_equal, PhysicalExpr, }; use datafusion_common::ScalarValue; @@ -334,29 +326,6 @@ mod tests { assert!(!physical_exprs_contains(&physical_exprs, &lit1)); } - #[test] - fn test_have_common_entries() { - let lit_true = Arc::new(Literal::new(ScalarValue::Boolean(Some(true)))) - as Arc; - let lit_false = Arc::new(Literal::new(ScalarValue::Boolean(Some(false)))) - as Arc; - let lit2 = - Arc::new(Literal::new(ScalarValue::Int32(Some(2)))) as Arc; - let lit1 = - Arc::new(Literal::new(ScalarValue::Int32(Some(1)))) as Arc; - let col_b_expr = Arc::new(Column::new("b", 1)) as Arc; - - let vec1 = vec![lit_true.clone(), lit_false.clone()]; - let vec2 = vec![lit_true.clone(), col_b_expr.clone()]; - let vec3 = vec![lit2.clone(), lit1.clone()]; - - // lit_true is common - assert!(have_common_entries(&vec1, &vec2)); - // there is no common entry - assert!(!have_common_entries(&vec1, &vec3)); - assert!(!have_common_entries(&vec2, &vec3)); - } - #[test] fn test_physical_exprs_equal() { let lit_true = Arc::new(Literal::new(ScalarValue::Boolean(Some(true)))) From abb2ae76d963ee50b08730928302aa9e20031919 Mon Sep 17 00:00:00 2001 From: Nga Tran Date: Tue, 14 Nov 2023 16:27:31 -0500 Subject: [PATCH 254/572] =?UTF-8?q?Revert=20"Minor:=20remove=20unnecessary?= =?UTF-8?q?=20projection=20in=20`single=5Fdistinct=5Fto=5Fg=E2=80=A6=20(#8?= =?UTF-8?q?176)?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit * Revert "Minor: remove unnecessary projection in `single_distinct_to_group_by` rule (#8061)" This reverts commit 15d8c9bf48a56ae9de34d18becab13fd1942dc4a. * Add regression test --------- Co-authored-by: Andrew Lamb --- .../src/single_distinct_to_groupby.rs | 95 ++++++++++++++----- .../sqllogictest/test_files/groupby.slt | 33 ++++++- datafusion/sqllogictest/test_files/joins.slt | 47 ++++----- .../sqllogictest/test_files/tpch/q16.slt.part | 10 +- 4 files changed, 130 insertions(+), 55 deletions(-) diff --git a/datafusion/optimizer/src/single_distinct_to_groupby.rs b/datafusion/optimizer/src/single_distinct_to_groupby.rs index 414217612d1e..be76c069f0b7 100644 --- a/datafusion/optimizer/src/single_distinct_to_groupby.rs +++ b/datafusion/optimizer/src/single_distinct_to_groupby.rs @@ -22,12 +22,13 @@ use std::sync::Arc; use crate::optimizer::ApplyOrder; use crate::{OptimizerConfig, OptimizerRule}; -use datafusion_common::Result; +use datafusion_common::{DFSchema, Result}; use datafusion_expr::{ col, expr::AggregateFunction, - logical_plan::{Aggregate, LogicalPlan}, - Expr, + logical_plan::{Aggregate, LogicalPlan, Projection}, + utils::columnize_expr, + Expr, ExprSchemable, }; use hashbrown::HashSet; @@ -152,7 +153,7 @@ impl OptimizerRule for SingleDistinctToGroupBy { // replace the distinct arg with alias let mut group_fields_set = HashSet::new(); - let outer_aggr_exprs = aggr_expr + let new_aggr_exprs = aggr_expr .iter() .map(|aggr_expr| match aggr_expr { Expr::AggregateFunction(AggregateFunction { @@ -174,24 +175,67 @@ impl OptimizerRule for SingleDistinctToGroupBy { false, // intentional to remove distinct here filter.clone(), order_by.clone(), - )) - .alias(aggr_expr.display_name()?)) + ))) } _ => Ok(aggr_expr.clone()), }) .collect::>>()?; // construct the inner AggrPlan + let inner_fields = inner_group_exprs + .iter() + .map(|expr| expr.to_field(input.schema())) + .collect::>>()?; + let inner_schema = DFSchema::new_with_metadata( + inner_fields, + input.schema().metadata().clone(), + )?; let inner_agg = LogicalPlan::Aggregate(Aggregate::try_new( input.clone(), inner_group_exprs, Vec::new(), )?); - Ok(Some(LogicalPlan::Aggregate(Aggregate::try_new( + let outer_fields = outer_group_exprs + .iter() + .chain(new_aggr_exprs.iter()) + .map(|expr| expr.to_field(&inner_schema)) + .collect::>>()?; + let outer_aggr_schema = Arc::new(DFSchema::new_with_metadata( + outer_fields, + input.schema().metadata().clone(), + )?); + + // so the aggregates are displayed in the same way even after the rewrite + // this optimizer has two kinds of alias: + // - group_by aggr + // - aggr expr + let group_size = group_expr.len(); + let alias_expr = out_group_expr_with_alias + .into_iter() + .map(|(group_expr, original_field)| { + if let Some(name) = original_field { + group_expr.alias(name) + } else { + group_expr + } + }) + .chain(new_aggr_exprs.iter().enumerate().map(|(idx, expr)| { + let idx = idx + group_size; + let name = fields[idx].qualified_name(); + columnize_expr(expr.clone().alias(name), &outer_aggr_schema) + })) + .collect(); + + let outer_aggr = LogicalPlan::Aggregate(Aggregate::try_new( Arc::new(inner_agg), outer_group_exprs, - outer_aggr_exprs, + new_aggr_exprs, + )?); + + Ok(Some(LogicalPlan::Projection(Projection::try_new( + alias_expr, + Arc::new(outer_aggr), )?))) } else { Ok(None) @@ -255,9 +299,10 @@ mod tests { .build()?; // Should work - let expected = "Aggregate: groupBy=[[]], aggr=[[COUNT(alias1) AS COUNT(DISTINCT test.b)]] [COUNT(DISTINCT test.b):Int64;N]\ - \n Aggregate: groupBy=[[test.b AS alias1]], aggr=[[]] [alias1:UInt32]\ - \n TableScan: test [a:UInt32, b:UInt32, c:UInt32]"; + let expected = "Projection: COUNT(alias1) AS COUNT(DISTINCT test.b) [COUNT(DISTINCT test.b):Int64;N]\ + \n Aggregate: groupBy=[[]], aggr=[[COUNT(alias1)]] [COUNT(alias1):Int64;N]\ + \n Aggregate: groupBy=[[test.b AS alias1]], aggr=[[]] [alias1:UInt32]\ + \n TableScan: test [a:UInt32, b:UInt32, c:UInt32]"; assert_optimized_plan_equal(&plan, expected) } @@ -328,9 +373,10 @@ mod tests { .aggregate(Vec::::new(), vec![count_distinct(lit(2) * col("b"))])? .build()?; - let expected = "Aggregate: groupBy=[[]], aggr=[[COUNT(alias1) AS COUNT(DISTINCT Int32(2) * test.b)]] [COUNT(DISTINCT Int32(2) * test.b):Int64;N]\ - \n Aggregate: groupBy=[[Int32(2) * test.b AS alias1]], aggr=[[]] [alias1:Int32]\ - \n TableScan: test [a:UInt32, b:UInt32, c:UInt32]"; + let expected = "Projection: COUNT(alias1) AS COUNT(DISTINCT Int32(2) * test.b) [COUNT(DISTINCT Int32(2) * test.b):Int64;N]\ + \n Aggregate: groupBy=[[]], aggr=[[COUNT(alias1)]] [COUNT(alias1):Int64;N]\ + \n Aggregate: groupBy=[[Int32(2) * test.b AS alias1]], aggr=[[]] [alias1:Int32]\ + \n TableScan: test [a:UInt32, b:UInt32, c:UInt32]"; assert_optimized_plan_equal(&plan, expected) } @@ -344,9 +390,10 @@ mod tests { .build()?; // Should work - let expected = "Aggregate: groupBy=[[test.a]], aggr=[[COUNT(alias1) AS COUNT(DISTINCT test.b)]] [a:UInt32, COUNT(DISTINCT test.b):Int64;N]\ - \n Aggregate: groupBy=[[test.a, test.b AS alias1]], aggr=[[]] [a:UInt32, alias1:UInt32]\ - \n TableScan: test [a:UInt32, b:UInt32, c:UInt32]"; + let expected = "Projection: test.a, COUNT(alias1) AS COUNT(DISTINCT test.b) [a:UInt32, COUNT(DISTINCT test.b):Int64;N]\ + \n Aggregate: groupBy=[[test.a]], aggr=[[COUNT(alias1)]] [a:UInt32, COUNT(alias1):Int64;N]\ + \n Aggregate: groupBy=[[test.a, test.b AS alias1]], aggr=[[]] [a:UInt32, alias1:UInt32]\ + \n TableScan: test [a:UInt32, b:UInt32, c:UInt32]"; assert_optimized_plan_equal(&plan, expected) } @@ -389,9 +436,10 @@ mod tests { )? .build()?; // Should work - let expected = "Aggregate: groupBy=[[test.a]], aggr=[[COUNT(alias1) AS COUNT(DISTINCT test.b), MAX(alias1) AS MAX(DISTINCT test.b)]] [a:UInt32, COUNT(DISTINCT test.b):Int64;N, MAX(DISTINCT test.b):UInt32;N]\ - \n Aggregate: groupBy=[[test.a, test.b AS alias1]], aggr=[[]] [a:UInt32, alias1:UInt32]\ - \n TableScan: test [a:UInt32, b:UInt32, c:UInt32]"; + let expected = "Projection: test.a, COUNT(alias1) AS COUNT(DISTINCT test.b), MAX(alias1) AS MAX(DISTINCT test.b) [a:UInt32, COUNT(DISTINCT test.b):Int64;N, MAX(DISTINCT test.b):UInt32;N]\ + \n Aggregate: groupBy=[[test.a]], aggr=[[COUNT(alias1), MAX(alias1)]] [a:UInt32, COUNT(alias1):Int64;N, MAX(alias1):UInt32;N]\ + \n Aggregate: groupBy=[[test.a, test.b AS alias1]], aggr=[[]] [a:UInt32, alias1:UInt32]\ + \n TableScan: test [a:UInt32, b:UInt32, c:UInt32]"; assert_optimized_plan_equal(&plan, expected) } @@ -423,9 +471,10 @@ mod tests { .build()?; // Should work - let expected = "Aggregate: groupBy=[[group_alias_0]], aggr=[[COUNT(alias1) AS COUNT(DISTINCT test.c)]] [group_alias_0:Int32, COUNT(DISTINCT test.c):Int64;N]\ - \n Aggregate: groupBy=[[test.a + Int32(1) AS group_alias_0, test.c AS alias1]], aggr=[[]] [group_alias_0:Int32, alias1:UInt32]\ - \n TableScan: test [a:UInt32, b:UInt32, c:UInt32]"; + let expected = "Projection: group_alias_0 AS test.a + Int32(1), COUNT(alias1) AS COUNT(DISTINCT test.c) [test.a + Int32(1):Int32, COUNT(DISTINCT test.c):Int64;N]\ + \n Aggregate: groupBy=[[group_alias_0]], aggr=[[COUNT(alias1)]] [group_alias_0:Int32, COUNT(alias1):Int64;N]\ + \n Aggregate: groupBy=[[test.a + Int32(1) AS group_alias_0, test.c AS alias1]], aggr=[[]] [group_alias_0:Int32, alias1:UInt32]\ + \n TableScan: test [a:UInt32, b:UInt32, c:UInt32]"; assert_optimized_plan_equal(&plan, expected) } diff --git a/datafusion/sqllogictest/test_files/groupby.slt b/datafusion/sqllogictest/test_files/groupby.slt index 105f11f21628..300e92a7352f 100644 --- a/datafusion/sqllogictest/test_files/groupby.slt +++ b/datafusion/sqllogictest/test_files/groupby.slt @@ -3823,17 +3823,17 @@ query TT EXPLAIN SELECT SUM(DISTINCT CAST(x AS DOUBLE)), MAX(DISTINCT CAST(x AS DOUBLE)) FROM t1 GROUP BY y; ---- logical_plan -Projection: SUM(DISTINCT t1.x), MAX(DISTINCT t1.x) ---Aggregate: groupBy=[[t1.y]], aggr=[[SUM(alias1) AS SUM(DISTINCT t1.x), MAX(alias1) AS MAX(DISTINCT t1.x)]] +Projection: SUM(alias1) AS SUM(DISTINCT t1.x), MAX(alias1) AS MAX(DISTINCT t1.x) +--Aggregate: groupBy=[[t1.y]], aggr=[[SUM(alias1), MAX(alias1)]] ----Aggregate: groupBy=[[t1.y, CAST(t1.x AS Float64)t1.x AS t1.x AS alias1]], aggr=[[]] ------Projection: CAST(t1.x AS Float64) AS CAST(t1.x AS Float64)t1.x, t1.y --------TableScan: t1 projection=[x, y] physical_plan -ProjectionExec: expr=[SUM(DISTINCT t1.x)@1 as SUM(DISTINCT t1.x), MAX(DISTINCT t1.x)@2 as MAX(DISTINCT t1.x)] ---AggregateExec: mode=FinalPartitioned, gby=[y@0 as y], aggr=[SUM(DISTINCT t1.x), MAX(DISTINCT t1.x)] +ProjectionExec: expr=[SUM(alias1)@1 as SUM(DISTINCT t1.x), MAX(alias1)@2 as MAX(DISTINCT t1.x)] +--AggregateExec: mode=FinalPartitioned, gby=[y@0 as y], aggr=[SUM(alias1), MAX(alias1)] ----CoalesceBatchesExec: target_batch_size=2 ------RepartitionExec: partitioning=Hash([y@0], 8), input_partitions=8 ---------AggregateExec: mode=Partial, gby=[y@0 as y], aggr=[SUM(DISTINCT t1.x), MAX(DISTINCT t1.x)] +--------AggregateExec: mode=Partial, gby=[y@0 as y], aggr=[SUM(alias1), MAX(alias1)] ----------AggregateExec: mode=FinalPartitioned, gby=[y@0 as y, alias1@1 as alias1], aggr=[] ------------CoalesceBatchesExec: target_batch_size=2 --------------RepartitionExec: partitioning=Hash([y@0, alias1@1], 8), input_partitions=8 @@ -3841,3 +3841,26 @@ ProjectionExec: expr=[SUM(DISTINCT t1.x)@1 as SUM(DISTINCT t1.x), MAX(DISTINCT t ------------------AggregateExec: mode=Partial, gby=[y@1 as y, CAST(t1.x AS Float64)t1.x@0 as alias1], aggr=[] --------------------ProjectionExec: expr=[CAST(x@0 AS Float64) as CAST(t1.x AS Float64)t1.x, y@1 as y] ----------------------MemoryExec: partitions=1, partition_sizes=[1] + +statement ok +drop table t1 + +# Reproducer for https://github.com/apache/arrow-datafusion/issues/8175 + +statement ok +create table t1(state string, city string, min_temp float, area int, time timestamp) as values + ('MA', 'Boston', 70.4, 1, 50), + ('MA', 'Bedford', 71.59, 2, 150); + +query RI +select date_part('year', time) as bla, count(distinct state) as count from t1 group by bla; +---- +1970 1 + +query PI +select date_bin(interval '1 year', time) as bla, count(distinct state) as count from t1 group by bla; +---- +1970-01-01T00:00:00 1 + +statement ok +drop table t1 diff --git a/datafusion/sqllogictest/test_files/joins.slt b/datafusion/sqllogictest/test_files/joins.slt index 24893297f163..fa3a6cff8c4a 100644 --- a/datafusion/sqllogictest/test_files/joins.slt +++ b/datafusion/sqllogictest/test_files/joins.slt @@ -1361,29 +1361,31 @@ from join_t1 inner join join_t2 on join_t1.t1_id = join_t2.t2_id ---- logical_plan -Aggregate: groupBy=[[]], aggr=[[COUNT(alias1) AS COUNT(DISTINCT join_t1.t1_id)]] ---Aggregate: groupBy=[[join_t1.t1_id AS alias1]], aggr=[[]] -----Projection: join_t1.t1_id -------Inner Join: join_t1.t1_id = join_t2.t2_id ---------TableScan: join_t1 projection=[t1_id] ---------TableScan: join_t2 projection=[t2_id] +Projection: COUNT(alias1) AS COUNT(DISTINCT join_t1.t1_id) +--Aggregate: groupBy=[[]], aggr=[[COUNT(alias1)]] +----Aggregate: groupBy=[[join_t1.t1_id AS alias1]], aggr=[[]] +------Projection: join_t1.t1_id +--------Inner Join: join_t1.t1_id = join_t2.t2_id +----------TableScan: join_t1 projection=[t1_id] +----------TableScan: join_t2 projection=[t2_id] physical_plan -AggregateExec: mode=Final, gby=[], aggr=[COUNT(DISTINCT join_t1.t1_id)] ---CoalescePartitionsExec -----AggregateExec: mode=Partial, gby=[], aggr=[COUNT(DISTINCT join_t1.t1_id)] -------AggregateExec: mode=FinalPartitioned, gby=[alias1@0 as alias1], aggr=[] ---------AggregateExec: mode=Partial, gby=[t1_id@0 as alias1], aggr=[] -----------ProjectionExec: expr=[t1_id@0 as t1_id] -------------CoalesceBatchesExec: target_batch_size=2 ---------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(t1_id@0, t2_id@0)] -----------------CoalesceBatchesExec: target_batch_size=2 -------------------RepartitionExec: partitioning=Hash([t1_id@0], 2), input_partitions=2 ---------------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -----------------------MemoryExec: partitions=1, partition_sizes=[1] -----------------CoalesceBatchesExec: target_batch_size=2 -------------------RepartitionExec: partitioning=Hash([t2_id@0], 2), input_partitions=2 ---------------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -----------------------MemoryExec: partitions=1, partition_sizes=[1] +ProjectionExec: expr=[COUNT(alias1)@0 as COUNT(DISTINCT join_t1.t1_id)] +--AggregateExec: mode=Final, gby=[], aggr=[COUNT(alias1)] +----CoalescePartitionsExec +------AggregateExec: mode=Partial, gby=[], aggr=[COUNT(alias1)] +--------AggregateExec: mode=FinalPartitioned, gby=[alias1@0 as alias1], aggr=[] +----------AggregateExec: mode=Partial, gby=[t1_id@0 as alias1], aggr=[] +------------ProjectionExec: expr=[t1_id@0 as t1_id] +--------------CoalesceBatchesExec: target_batch_size=2 +----------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(t1_id@0, t2_id@0)] +------------------CoalesceBatchesExec: target_batch_size=2 +--------------------RepartitionExec: partitioning=Hash([t1_id@0], 2), input_partitions=2 +----------------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +------------------------MemoryExec: partitions=1, partition_sizes=[1] +------------------CoalesceBatchesExec: target_batch_size=2 +--------------------RepartitionExec: partitioning=Hash([t2_id@0], 2), input_partitions=2 +----------------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +------------------------MemoryExec: partitions=1, partition_sizes=[1] statement ok set datafusion.explain.logical_plan_only = true; @@ -3407,3 +3409,4 @@ set datafusion.optimizer.prefer_existing_sort = false; statement ok drop table annotated_data; + diff --git a/datafusion/sqllogictest/test_files/tpch/q16.slt.part b/datafusion/sqllogictest/test_files/tpch/q16.slt.part index c04782958917..b93872929fe5 100644 --- a/datafusion/sqllogictest/test_files/tpch/q16.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/q16.slt.part @@ -52,8 +52,8 @@ limit 10; logical_plan Limit: skip=0, fetch=10 --Sort: supplier_cnt DESC NULLS FIRST, part.p_brand ASC NULLS LAST, part.p_type ASC NULLS LAST, part.p_size ASC NULLS LAST, fetch=10 -----Projection: part.p_brand, part.p_type, part.p_size, COUNT(DISTINCT partsupp.ps_suppkey) AS supplier_cnt -------Aggregate: groupBy=[[part.p_brand, part.p_type, part.p_size]], aggr=[[COUNT(alias1) AS COUNT(DISTINCT partsupp.ps_suppkey)]] +----Projection: part.p_brand, part.p_type, part.p_size, COUNT(alias1) AS supplier_cnt +------Aggregate: groupBy=[[part.p_brand, part.p_type, part.p_size]], aggr=[[COUNT(alias1)]] --------Aggregate: groupBy=[[part.p_brand, part.p_type, part.p_size, partsupp.ps_suppkey AS alias1]], aggr=[[]] ----------LeftAnti Join: partsupp.ps_suppkey = __correlated_sq_1.s_suppkey ------------Projection: partsupp.ps_suppkey, part.p_brand, part.p_type, part.p_size @@ -69,11 +69,11 @@ physical_plan GlobalLimitExec: skip=0, fetch=10 --SortPreservingMergeExec: [supplier_cnt@3 DESC,p_brand@0 ASC NULLS LAST,p_type@1 ASC NULLS LAST,p_size@2 ASC NULLS LAST], fetch=10 ----SortExec: TopK(fetch=10), expr=[supplier_cnt@3 DESC,p_brand@0 ASC NULLS LAST,p_type@1 ASC NULLS LAST,p_size@2 ASC NULLS LAST] -------ProjectionExec: expr=[p_brand@0 as p_brand, p_type@1 as p_type, p_size@2 as p_size, COUNT(DISTINCT partsupp.ps_suppkey)@3 as supplier_cnt] ---------AggregateExec: mode=FinalPartitioned, gby=[p_brand@0 as p_brand, p_type@1 as p_type, p_size@2 as p_size], aggr=[COUNT(DISTINCT partsupp.ps_suppkey)] +------ProjectionExec: expr=[p_brand@0 as p_brand, p_type@1 as p_type, p_size@2 as p_size, COUNT(alias1)@3 as supplier_cnt] +--------AggregateExec: mode=FinalPartitioned, gby=[p_brand@0 as p_brand, p_type@1 as p_type, p_size@2 as p_size], aggr=[COUNT(alias1)] ----------CoalesceBatchesExec: target_batch_size=8192 ------------RepartitionExec: partitioning=Hash([p_brand@0, p_type@1, p_size@2], 4), input_partitions=4 ---------------AggregateExec: mode=Partial, gby=[p_brand@0 as p_brand, p_type@1 as p_type, p_size@2 as p_size], aggr=[COUNT(DISTINCT partsupp.ps_suppkey)] +--------------AggregateExec: mode=Partial, gby=[p_brand@0 as p_brand, p_type@1 as p_type, p_size@2 as p_size], aggr=[COUNT(alias1)] ----------------AggregateExec: mode=FinalPartitioned, gby=[p_brand@0 as p_brand, p_type@1 as p_type, p_size@2 as p_size, alias1@3 as alias1], aggr=[] ------------------CoalesceBatchesExec: target_batch_size=8192 --------------------RepartitionExec: partitioning=Hash([p_brand@0, p_type@1, p_size@2, alias1@3], 4), input_partitions=4 From 6ecb6cd78dcd0508d9c5e8543275cd67ea4bbad6 Mon Sep 17 00:00:00 2001 From: Mustafa Akur <106137913+mustafasrepo@users.noreply.github.com> Date: Wed, 15 Nov 2023 11:04:29 +0300 Subject: [PATCH 255/572] Preserve all of the valid orderings during merging. (#8169) * Preserve all of the valid orderings during merging. * Update datafusion/physical-expr/src/equivalence.rs Co-authored-by: Mehmet Ozan Kabak * Address reviews --------- Co-authored-by: Mehmet Ozan Kabak --- .../sort_preserving_repartition_fuzz.rs | 276 +++++++++++++++++- datafusion/physical-expr/src/equivalence.rs | 31 +- .../physical-plan/src/repartition/mod.rs | 3 - .../src/sorts/sort_preserving_merge.rs | 3 +- datafusion/sqllogictest/test_files/window.slt | 45 +++ 5 files changed, 335 insertions(+), 23 deletions(-) diff --git a/datafusion/core/tests/fuzz_cases/sort_preserving_repartition_fuzz.rs b/datafusion/core/tests/fuzz_cases/sort_preserving_repartition_fuzz.rs index 818698d6c041..5bc29ba1c277 100644 --- a/datafusion/core/tests/fuzz_cases/sort_preserving_repartition_fuzz.rs +++ b/datafusion/core/tests/fuzz_cases/sort_preserving_repartition_fuzz.rs @@ -17,22 +17,272 @@ #[cfg(test)] mod sp_repartition_fuzz_tests { - use arrow::compute::concat_batches; - use arrow_array::{ArrayRef, Int64Array, RecordBatch}; - use arrow_schema::SortOptions; - use datafusion::physical_plan::memory::MemoryExec; - use datafusion::physical_plan::repartition::RepartitionExec; - use datafusion::physical_plan::sorts::sort_preserving_merge::SortPreservingMergeExec; - use datafusion::physical_plan::{collect, ExecutionPlan, Partitioning}; - use datafusion::prelude::SessionContext; - use datafusion_execution::config::SessionConfig; - use datafusion_physical_expr::expressions::col; - use datafusion_physical_expr::{PhysicalExpr, PhysicalSortExpr}; - use rand::rngs::StdRng; - use rand::{Rng, SeedableRng}; use std::sync::Arc; + + use arrow::compute::{concat_batches, lexsort, SortColumn}; + use arrow_array::{ArrayRef, Int64Array, RecordBatch, UInt64Array}; + use arrow_schema::{DataType, Field, Schema, SchemaRef, SortOptions}; + + use datafusion::physical_plan::{ + collect, + memory::MemoryExec, + metrics::{BaselineMetrics, ExecutionPlanMetricsSet}, + repartition::RepartitionExec, + sorts::sort_preserving_merge::SortPreservingMergeExec, + sorts::streaming_merge::streaming_merge, + stream::RecordBatchStreamAdapter, + ExecutionPlan, Partitioning, + }; + use datafusion::prelude::SessionContext; + use datafusion_common::Result; + use datafusion_execution::{ + config::SessionConfig, memory_pool::MemoryConsumer, SendableRecordBatchStream, + }; + use datafusion_physical_expr::{ + expressions::{col, Column}, + EquivalenceProperties, PhysicalExpr, PhysicalSortExpr, + }; use test_utils::add_empty_batches; + use itertools::izip; + use rand::{rngs::StdRng, seq::SliceRandom, Rng, SeedableRng}; + + // Generate a schema which consists of 6 columns (a, b, c, d, e, f) + fn create_test_schema() -> Result { + let a = Field::new("a", DataType::Int32, true); + let b = Field::new("b", DataType::Int32, true); + let c = Field::new("c", DataType::Int32, true); + let d = Field::new("d", DataType::Int32, true); + let e = Field::new("e", DataType::Int32, true); + let f = Field::new("f", DataType::Int32, true); + let schema = Arc::new(Schema::new(vec![a, b, c, d, e, f])); + + Ok(schema) + } + + /// Construct a schema with random ordering + /// among column a, b, c, d + /// where + /// Column [a=f] (e.g they are aliases). + /// Column e is constant. + fn create_random_schema(seed: u64) -> Result<(SchemaRef, EquivalenceProperties)> { + let test_schema = create_test_schema()?; + let col_a = &col("a", &test_schema)?; + let col_b = &col("b", &test_schema)?; + let col_c = &col("c", &test_schema)?; + let col_d = &col("d", &test_schema)?; + let col_e = &col("e", &test_schema)?; + let col_f = &col("f", &test_schema)?; + let col_exprs = [col_a, col_b, col_c, col_d, col_e, col_f]; + + let mut eq_properties = EquivalenceProperties::new(test_schema.clone()); + // Define a and f are aliases + eq_properties.add_equal_conditions(col_a, col_f); + // Column e has constant value. + eq_properties = eq_properties.add_constants([col_e.clone()]); + + // Randomly order columns for sorting + let mut rng = StdRng::seed_from_u64(seed); + let mut remaining_exprs = col_exprs[0..4].to_vec(); // only a, b, c, d are sorted + + let options_asc = SortOptions { + descending: false, + nulls_first: false, + }; + + while !remaining_exprs.is_empty() { + let n_sort_expr = rng.gen_range(0..remaining_exprs.len() + 1); + remaining_exprs.shuffle(&mut rng); + + let ordering = remaining_exprs + .drain(0..n_sort_expr) + .map(|expr| PhysicalSortExpr { + expr: expr.clone(), + options: options_asc, + }) + .collect(); + + eq_properties.add_new_orderings([ordering]); + } + + Ok((test_schema, eq_properties)) + } + + // If we already generated a random result for one of the + // expressions in the equivalence classes. For other expressions in the same + // equivalence class use same result. This util gets already calculated result, when available. + fn get_representative_arr( + eq_group: &[Arc], + existing_vec: &[Option], + schema: SchemaRef, + ) -> Option { + for expr in eq_group.iter() { + let col = expr.as_any().downcast_ref::().unwrap(); + let (idx, _field) = schema.column_with_name(col.name()).unwrap(); + if let Some(res) = &existing_vec[idx] { + return Some(res.clone()); + } + } + None + } + + // Generate a table that satisfies the given equivalence properties; i.e. + // equivalences, ordering equivalences, and constants. + fn generate_table_for_eq_properties( + eq_properties: &EquivalenceProperties, + n_elem: usize, + n_distinct: usize, + ) -> Result { + let mut rng = StdRng::seed_from_u64(23); + + let schema = eq_properties.schema(); + let mut schema_vec = vec![None; schema.fields.len()]; + + // Utility closure to generate random array + let mut generate_random_array = |num_elems: usize, max_val: usize| -> ArrayRef { + let values: Vec = (0..num_elems) + .map(|_| rng.gen_range(0..max_val) as u64) + .collect(); + Arc::new(UInt64Array::from_iter_values(values)) + }; + + // Fill constant columns + for constant in eq_properties.constants() { + let col = constant.as_any().downcast_ref::().unwrap(); + let (idx, _field) = schema.column_with_name(col.name()).unwrap(); + let arr = + Arc::new(UInt64Array::from_iter_values(vec![0; n_elem])) as ArrayRef; + schema_vec[idx] = Some(arr); + } + + // Fill columns based on ordering equivalences + for ordering in eq_properties.oeq_class().iter() { + let (sort_columns, indices): (Vec<_>, Vec<_>) = ordering + .iter() + .map(|PhysicalSortExpr { expr, options }| { + let col = expr.as_any().downcast_ref::().unwrap(); + let (idx, _field) = schema.column_with_name(col.name()).unwrap(); + let arr = generate_random_array(n_elem, n_distinct); + ( + SortColumn { + values: arr, + options: Some(*options), + }, + idx, + ) + }) + .unzip(); + + let sort_arrs = arrow::compute::lexsort(&sort_columns, None)?; + for (idx, arr) in izip!(indices, sort_arrs) { + schema_vec[idx] = Some(arr); + } + } + + // Fill columns based on equivalence groups + for eq_group in eq_properties.eq_group().iter() { + let representative_array = + get_representative_arr(eq_group, &schema_vec, schema.clone()) + .unwrap_or_else(|| generate_random_array(n_elem, n_distinct)); + + for expr in eq_group { + let col = expr.as_any().downcast_ref::().unwrap(); + let (idx, _field) = schema.column_with_name(col.name()).unwrap(); + schema_vec[idx] = Some(representative_array.clone()); + } + } + + let res: Vec<_> = schema_vec + .into_iter() + .zip(schema.fields.iter()) + .map(|(elem, field)| { + ( + field.name(), + // Generate random values for columns that do not occur in any of the groups (equivalence, ordering equivalence, constants) + elem.unwrap_or_else(|| generate_random_array(n_elem, n_distinct)), + ) + }) + .collect(); + + Ok(RecordBatch::try_from_iter(res)?) + } + + // This test checks for whether during sort preserving merge we can preserve all of the valid orderings + // successfully. If at the input we have orderings [a ASC, b ASC], [c ASC, d ASC] + // After sort preserving merge orderings [a ASC, b ASC], [c ASC, d ASC] should still be valid. + #[tokio::test] + async fn stream_merge_multi_order_preserve() -> Result<()> { + const N_PARTITION: usize = 8; + const N_ELEM: usize = 25; + const N_DISTINCT: usize = 5; + const N_DIFF_SCHEMA: usize = 20; + + use datafusion::physical_plan::common::collect; + for seed in 0..N_DIFF_SCHEMA { + // Create a schema with random equivalence properties + let (_test_schema, eq_properties) = create_random_schema(seed as u64)?; + let table_data_with_properties = + generate_table_for_eq_properties(&eq_properties, N_ELEM, N_DISTINCT)?; + let schema = table_data_with_properties.schema(); + let streams: Vec = (0..N_PARTITION) + .map(|_idx| { + let batch = table_data_with_properties.clone(); + Box::pin(RecordBatchStreamAdapter::new( + schema.clone(), + futures::stream::once(async { Ok(batch) }), + )) as SendableRecordBatchStream + }) + .collect::>(); + + // Returns concatenated version of the all available orderings + let exprs = eq_properties + .oeq_class() + .output_ordering() + .unwrap_or_default(); + + let context = SessionContext::new().task_ctx(); + let mem_reservation = + MemoryConsumer::new("test".to_string()).register(context.memory_pool()); + + // Internally SortPreservingMergeExec uses this function for merging. + let res = streaming_merge( + streams, + schema, + &exprs, + BaselineMetrics::new(&ExecutionPlanMetricsSet::new(), 0), + 1, + None, + mem_reservation, + )?; + let res = collect(res).await?; + // Contains the merged result. + let res = concat_batches(&res[0].schema(), &res)?; + + for ordering in eq_properties.oeq_class().iter() { + let err_msg = format!("error in eq properties: {:?}", eq_properties); + let sort_solumns = ordering + .iter() + .map(|sort_expr| sort_expr.evaluate_to_sort_column(&res)) + .collect::>>()?; + let orig_columns = sort_solumns + .iter() + .map(|sort_column| sort_column.values.clone()) + .collect::>(); + let sorted_columns = lexsort(&sort_solumns, None)?; + + // Make sure after merging ordering is still valid. + assert_eq!(orig_columns.len(), sorted_columns.len(), "{}", err_msg); + assert!( + izip!(orig_columns.into_iter(), sorted_columns.into_iter()) + .all(|(lhs, rhs)| { lhs == rhs }), + "{}", + err_msg + ) + } + } + Ok(()) + } + #[tokio::test(flavor = "multi_thread", worker_threads = 8)] async fn sort_preserving_repartition_test() { let seed_start = 0; diff --git a/datafusion/physical-expr/src/equivalence.rs b/datafusion/physical-expr/src/equivalence.rs index 84291653fb4f..f3bfe4961622 100644 --- a/datafusion/physical-expr/src/equivalence.rs +++ b/datafusion/physical-expr/src/equivalence.rs @@ -229,7 +229,7 @@ impl EquivalenceGroup { } /// Returns an iterator over the equivalence classes in this group. - fn iter(&self) -> impl Iterator { + pub fn iter(&self) -> impl Iterator { self.classes.iter() } @@ -551,7 +551,7 @@ impl EquivalenceGroup { /// This function constructs a duplicate-free `LexOrderingReq` by filtering out /// duplicate entries that have same physical expression inside. For example, -/// `vec![a Some(Asc), a Some(Desc)]` collapses to `vec![a Some(Asc)]`. +/// `vec![a Some(ASC), a Some(DESC)]` collapses to `vec![a Some(ASC)]`. pub fn collapse_lex_req(input: LexRequirement) -> LexRequirement { let mut output = Vec::::new(); for item in input { @@ -562,6 +562,19 @@ pub fn collapse_lex_req(input: LexRequirement) -> LexRequirement { output } +/// This function constructs a duplicate-free `LexOrdering` by filtering out +/// duplicate entries that have same physical expression inside. For example, +/// `vec![a ASC, a DESC]` collapses to `vec![a ASC]`. +pub fn collapse_lex_ordering(input: LexOrdering) -> LexOrdering { + let mut output = Vec::::new(); + for item in input { + if !output.iter().any(|req| req.expr.eq(&item.expr)) { + output.push(item); + } + } + output +} + /// An `OrderingEquivalenceClass` object keeps track of different alternative /// orderings than can describe a schema. For example, consider the following table: /// @@ -667,10 +680,13 @@ impl OrderingEquivalenceClass { } } - /// Gets the first ordering entry in this ordering equivalence class. - /// This is one of the many valid orderings (if there are multiple). + /// Returns the concatenation of all the orderings. This enables merge + /// operations to preserve all equivalent orderings simultaneously. pub fn output_ordering(&self) -> Option { - self.orderings.first().cloned() + let output_ordering = + self.orderings.iter().flatten().cloned().collect::>(); + let output_ordering = collapse_lex_ordering(output_ordering); + (!output_ordering.is_empty()).then_some(output_ordering) } // Append orderings in `other` to all existing orderings in this equivalence @@ -825,6 +841,11 @@ impl EquivalenceProperties { &self.eq_group } + /// Returns a reference to the constant expressions + pub fn constants(&self) -> &[Arc] { + &self.constants + } + /// Returns the normalized version of the ordering equivalence class within. /// Normalization removes constants and duplicates as well as standardizing /// expressions according to the equivalence group within. diff --git a/datafusion/physical-plan/src/repartition/mod.rs b/datafusion/physical-plan/src/repartition/mod.rs index 9719446d78d7..24f227d8a535 100644 --- a/datafusion/physical-plan/src/repartition/mod.rs +++ b/datafusion/physical-plan/src/repartition/mod.rs @@ -472,9 +472,6 @@ impl ExecutionPlan for RepartitionExec { if !self.maintains_input_order()[0] { result.clear_orderings(); } - if self.preserve_order { - result = result.with_reorder(self.sort_exprs().unwrap_or_default().to_vec()) - } result } diff --git a/datafusion/physical-plan/src/sorts/sort_preserving_merge.rs b/datafusion/physical-plan/src/sorts/sort_preserving_merge.rs index 65cd8e41480e..f4b57e8bfb45 100644 --- a/datafusion/physical-plan/src/sorts/sort_preserving_merge.rs +++ b/datafusion/physical-plan/src/sorts/sort_preserving_merge.rs @@ -174,8 +174,7 @@ impl ExecutionPlan for SortPreservingMergeExec { } fn equivalence_properties(&self) -> EquivalenceProperties { - let output_oeq = self.input.equivalence_properties(); - output_oeq.with_reorder(self.expr.to_vec()) + self.input.equivalence_properties() } fn children(&self) -> Vec> { diff --git a/datafusion/sqllogictest/test_files/window.slt b/datafusion/sqllogictest/test_files/window.slt index 2eb0576d559b..8be02b846cda 100644 --- a/datafusion/sqllogictest/test_files/window.slt +++ b/datafusion/sqllogictest/test_files/window.slt @@ -3396,6 +3396,21 @@ WITH ORDER (a ASC, b ASC) WITH ORDER (c ASC) LOCATION '../core/tests/data/window_2.csv'; +# Create an unbounded source where there is multiple orderings. +statement ok +CREATE UNBOUNDED EXTERNAL TABLE multiple_ordered_table_inf ( + a0 INTEGER, + a INTEGER, + b INTEGER, + c INTEGER, + d INTEGER +) +STORED AS CSV +WITH HEADER ROW +WITH ORDER (a ASC, b ASC) +WITH ORDER (c ASC) +LOCATION '../core/tests/data/window_2.csv'; + # All of the window execs in the physical plan should work in the # sorted mode. query TT @@ -3477,3 +3492,33 @@ query II select sum(1) over() x, sum(1) over () y ---- 1 1 + +statement ok +set datafusion.execution.target_partitions = 2; + +# source is ordered by [a ASC, b ASC], [c ASC] +# after sort preserving repartition and sort preserving merge +# we should still have the orderings [a ASC, b ASC], [c ASC]. +query TT +EXPLAIN SELECT *, + AVG(d) OVER sliding_window AS avg_d +FROM multiple_ordered_table_inf +WINDOW sliding_window AS ( + PARTITION BY d + ORDER BY a RANGE 10 PRECEDING +) +ORDER BY c +---- +logical_plan +Sort: multiple_ordered_table_inf.c ASC NULLS LAST +--Projection: multiple_ordered_table_inf.a0, multiple_ordered_table_inf.a, multiple_ordered_table_inf.b, multiple_ordered_table_inf.c, multiple_ordered_table_inf.d, AVG(multiple_ordered_table_inf.d) PARTITION BY [multiple_ordered_table_inf.d] ORDER BY [multiple_ordered_table_inf.a ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND CURRENT ROW AS avg_d +----WindowAggr: windowExpr=[[AVG(CAST(multiple_ordered_table_inf.d AS Float64)) PARTITION BY [multiple_ordered_table_inf.d] ORDER BY [multiple_ordered_table_inf.a ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND CURRENT ROW]] +------TableScan: multiple_ordered_table_inf projection=[a0, a, b, c, d] +physical_plan +SortPreservingMergeExec: [c@3 ASC NULLS LAST] +--ProjectionExec: expr=[a0@0 as a0, a@1 as a, b@2 as b, c@3 as c, d@4 as d, AVG(multiple_ordered_table_inf.d) PARTITION BY [multiple_ordered_table_inf.d] ORDER BY [multiple_ordered_table_inf.a ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND CURRENT ROW@5 as avg_d] +----BoundedWindowAggExec: wdw=[AVG(multiple_ordered_table_inf.d) PARTITION BY [multiple_ordered_table_inf.d] ORDER BY [multiple_ordered_table_inf.a ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND CURRENT ROW: Ok(Field { name: "AVG(multiple_ordered_table_inf.d) PARTITION BY [multiple_ordered_table_inf.d] ORDER BY [multiple_ordered_table_inf.a ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND CURRENT ROW", data_type: Float64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(10)), end_bound: CurrentRow }], mode=[Linear] +------CoalesceBatchesExec: target_batch_size=4096 +--------SortPreservingRepartitionExec: partitioning=Hash([d@4], 2), input_partitions=2, sort_exprs=a@1 ASC NULLS LAST,b@2 ASC NULLS LAST +----------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a0, a, b, c, d], infinite_source=true, output_ordering=[a@1 ASC NULLS LAST, b@2 ASC NULLS LAST], has_header=true From 7f111257443d79259eacbe3cb2ace1bdd276e5fc Mon Sep 17 00:00:00 2001 From: Andrew Lamb Date: Wed, 15 Nov 2023 04:59:23 -0500 Subject: [PATCH 256/572] Make fields of `ScalarUDF` , `AggregateUDF` and `WindowUDF` non `pub` (#8079) * Make fields of ScalarUDF non pub * Make fields of `WindowUDF` and `AggregateUDF` non pub. * fix doc --- .../core/src/datasource/listing/helpers.rs | 2 +- datafusion/core/src/execution/context/mod.rs | 6 +-- datafusion/core/src/physical_planner.rs | 4 +- datafusion/expr/src/expr.rs | 14 +++--- datafusion/expr/src/expr_schema.rs | 4 +- datafusion/expr/src/udaf.rs | 47 ++++++++++++++--- datafusion/expr/src/udf.rs | 50 +++++++++++++++---- datafusion/expr/src/udwf.rs | 44 ++++++++++++---- datafusion/expr/src/window_function.rs | 12 ++--- .../optimizer/src/analyzer/type_coercion.rs | 4 +- .../simplify_expressions/expr_simplifier.rs | 2 +- datafusion/physical-expr/src/functions.rs | 2 +- .../physical-expr/src/scalar_function.rs | 6 +-- datafusion/physical-expr/src/udf.rs | 6 +-- datafusion/physical-plan/src/udaf.rs | 10 ++-- datafusion/physical-plan/src/windows/mod.rs | 12 ++--- datafusion/proto/src/logical_plan/to_proto.rs | 8 +-- .../proto/src/physical_plan/from_proto.rs | 7 ++- .../proto/src/physical_plan/to_proto.rs | 3 +- .../tests/cases/roundtrip_logical_plan.rs | 12 ++--- .../tests/cases/roundtrip_physical_plan.rs | 4 +- 21 files changed, 172 insertions(+), 87 deletions(-) diff --git a/datafusion/core/src/datasource/listing/helpers.rs b/datafusion/core/src/datasource/listing/helpers.rs index 1d929f4bd4b1..3d2a3dc928b6 100644 --- a/datafusion/core/src/datasource/listing/helpers.rs +++ b/datafusion/core/src/datasource/listing/helpers.rs @@ -102,7 +102,7 @@ pub fn expr_applicable_for_cols(col_names: &[String], expr: &Expr) -> bool { } } Expr::ScalarUDF(ScalarUDF { fun, .. }) => { - match fun.signature.volatility { + match fun.signature().volatility { Volatility::Immutable => VisitRecursion::Continue, // TODO: Stable functions could be `applicable`, but that would require access to the context Volatility::Stable | Volatility::Volatile => { diff --git a/datafusion/core/src/execution/context/mod.rs b/datafusion/core/src/execution/context/mod.rs index 9c500ec07293..5c79c407b757 100644 --- a/datafusion/core/src/execution/context/mod.rs +++ b/datafusion/core/src/execution/context/mod.rs @@ -806,7 +806,7 @@ impl SessionContext { self.state .write() .scalar_functions - .insert(f.name.clone(), Arc::new(f)); + .insert(f.name().to_string(), Arc::new(f)); } /// Registers an aggregate UDF within this context. @@ -820,7 +820,7 @@ impl SessionContext { self.state .write() .aggregate_functions - .insert(f.name.clone(), Arc::new(f)); + .insert(f.name().to_string(), Arc::new(f)); } /// Registers a window UDF within this context. @@ -834,7 +834,7 @@ impl SessionContext { self.state .write() .window_functions - .insert(f.name.clone(), Arc::new(f)); + .insert(f.name().to_string(), Arc::new(f)); } /// Creates a [`DataFrame`] for reading a data source. diff --git a/datafusion/core/src/physical_planner.rs b/datafusion/core/src/physical_planner.rs index 9c1d978acc24..fffc51abeb67 100644 --- a/datafusion/core/src/physical_planner.rs +++ b/datafusion/core/src/physical_planner.rs @@ -222,7 +222,7 @@ fn create_physical_name(e: &Expr, is_first_expr: bool) -> Result { create_function_physical_name(&func.fun.to_string(), false, &func.args) } Expr::ScalarUDF(ScalarUDF { fun, args }) => { - create_function_physical_name(&fun.name, false, args) + create_function_physical_name(fun.name(), false, args) } Expr::WindowFunction(WindowFunction { fun, args, .. }) => { create_function_physical_name(&fun.to_string(), false, args) @@ -250,7 +250,7 @@ fn create_physical_name(e: &Expr, is_first_expr: bool) -> Result { for e in args { names.push(create_physical_name(e, false)?); } - Ok(format!("{}({})", fun.name, names.join(","))) + Ok(format!("{}({})", fun.name(), names.join(","))) } Expr::GroupingSet(grouping_set) => match grouping_set { GroupingSet::Rollup(exprs) => Ok(format!( diff --git a/datafusion/expr/src/expr.rs b/datafusion/expr/src/expr.rs index 97e4fcc327c3..2b2d30af3bc2 100644 --- a/datafusion/expr/src/expr.rs +++ b/datafusion/expr/src/expr.rs @@ -338,7 +338,7 @@ impl Between { } } -/// ScalarFunction expression +/// ScalarFunction expression invokes a built-in scalar function #[derive(Clone, PartialEq, Eq, Hash, Debug)] pub struct ScalarFunction { /// The function @@ -354,7 +354,9 @@ impl ScalarFunction { } } -/// ScalarUDF expression +/// ScalarUDF expression invokes a user-defined scalar function [`ScalarUDF`] +/// +/// [`ScalarUDF`]: crate::ScalarUDF #[derive(Clone, PartialEq, Eq, Hash, Debug)] pub struct ScalarUDF { /// The function @@ -1200,7 +1202,7 @@ impl fmt::Display for Expr { fmt_function(f, &func.fun.to_string(), false, &func.args, true) } Expr::ScalarUDF(ScalarUDF { fun, args }) => { - fmt_function(f, &fun.name, false, args, true) + fmt_function(f, fun.name(), false, args, true) } Expr::WindowFunction(WindowFunction { fun, @@ -1247,7 +1249,7 @@ impl fmt::Display for Expr { order_by, .. }) => { - fmt_function(f, &fun.name, false, args, true)?; + fmt_function(f, fun.name(), false, args, true)?; if let Some(fe) = filter { write!(f, " FILTER (WHERE {fe})")?; } @@ -1536,7 +1538,7 @@ fn create_name(e: &Expr) -> Result { create_function_name(&func.fun.to_string(), false, &func.args) } Expr::ScalarUDF(ScalarUDF { fun, args }) => { - create_function_name(&fun.name, false, args) + create_function_name(fun.name(), false, args) } Expr::WindowFunction(WindowFunction { fun, @@ -1589,7 +1591,7 @@ fn create_name(e: &Expr) -> Result { if let Some(ob) = order_by { info += &format!(" ORDER BY ([{}])", expr_vec_fmt!(ob)); } - Ok(format!("{}({}){}", fun.name, names.join(","), info)) + Ok(format!("{}({}){}", fun.name(), names.join(","), info)) } Expr::GroupingSet(grouping_set) => match grouping_set { GroupingSet::Rollup(exprs) => { diff --git a/datafusion/expr/src/expr_schema.rs b/datafusion/expr/src/expr_schema.rs index 5881feece1fc..0d06a1295199 100644 --- a/datafusion/expr/src/expr_schema.rs +++ b/datafusion/expr/src/expr_schema.rs @@ -87,7 +87,7 @@ impl ExprSchemable for Expr { .iter() .map(|e| e.get_type(schema)) .collect::>>()?; - Ok((fun.return_type)(&data_types)?.as_ref().clone()) + Ok(fun.return_type(&data_types)?) } Expr::ScalarFunction(ScalarFunction { fun, args }) => { let arg_data_types = args @@ -128,7 +128,7 @@ impl ExprSchemable for Expr { .iter() .map(|e| e.get_type(schema)) .collect::>>()?; - Ok((fun.return_type)(&data_types)?.as_ref().clone()) + fun.return_type(&data_types) } Expr::Not(_) | Expr::IsNull(_) diff --git a/datafusion/expr/src/udaf.rs b/datafusion/expr/src/udaf.rs index 84e238a1215b..b06e97acc283 100644 --- a/datafusion/expr/src/udaf.rs +++ b/datafusion/expr/src/udaf.rs @@ -15,12 +15,14 @@ // specific language governing permissions and limitations // under the License. -//! Udaf module contains functions and structs supporting user-defined aggregate functions. +//! [`AggregateUDF`]: User Defined Aggregate Functions -use crate::Expr; +use crate::{Accumulator, Expr}; use crate::{ AccumulatorFactoryFunction, ReturnTypeFunction, Signature, StateTypeFunction, }; +use arrow::datatypes::DataType; +use datafusion_common::Result; use std::fmt::{self, Debug, Formatter}; use std::sync::Arc; @@ -46,15 +48,15 @@ use std::sync::Arc; #[derive(Clone)] pub struct AggregateUDF { /// name - pub name: String, + name: String, /// Signature (input arguments) - pub signature: Signature, + signature: Signature, /// Return type - pub return_type: ReturnTypeFunction, + return_type: ReturnTypeFunction, /// actual implementation - pub accumulator: AccumulatorFactoryFunction, + accumulator: AccumulatorFactoryFunction, /// the accumulator's state's description as a function of the return type - pub state_type: StateTypeFunction, + state_type: StateTypeFunction, } impl Debug for AggregateUDF { @@ -112,4 +114,35 @@ impl AggregateUDF { order_by: None, }) } + + /// Returns this function's name + pub fn name(&self) -> &str { + &self.name + } + + /// Returns this function's signature (what input types are accepted) + pub fn signature(&self) -> &Signature { + &self.signature + } + + /// Return the type of the function given its input types + pub fn return_type(&self, args: &[DataType]) -> Result { + // Old API returns an Arc of the datatype for some reason + let res = (self.return_type)(args)?; + Ok(res.as_ref().clone()) + } + + /// Return an accumualator the given aggregate, given + /// its return datatype. + pub fn accumulator(&self, return_type: &DataType) -> Result> { + (self.accumulator)(return_type) + } + + /// Return the type of the intermediate state used by this aggregator, given + /// its return datatype. Supports multi-phase aggregations + pub fn state_type(&self, return_type: &DataType) -> Result> { + // old API returns an Arc for some reason, try and unwrap it here + let res = (self.state_type)(return_type)?; + Ok(Arc::try_unwrap(res).unwrap_or_else(|res| res.as_ref().clone())) + } } diff --git a/datafusion/expr/src/udf.rs b/datafusion/expr/src/udf.rs index be6c90aa5985..22e56caaaf5f 100644 --- a/datafusion/expr/src/udf.rs +++ b/datafusion/expr/src/udf.rs @@ -15,23 +15,31 @@ // specific language governing permissions and limitations // under the License. -//! Udf module contains foundational types that are used to represent UDFs in DataFusion. +//! [`ScalarUDF`]: Scalar User Defined Functions use crate::{Expr, ReturnTypeFunction, ScalarFunctionImplementation, Signature}; +use arrow::datatypes::DataType; +use datafusion_common::Result; use std::fmt; use std::fmt::Debug; use std::fmt::Formatter; use std::sync::Arc; -/// Logical representation of a UDF. +/// Logical representation of a Scalar User Defined Function. +/// +/// A scalar function produces a single row output for each row of input. +/// +/// This struct contains the information DataFusion needs to plan and invoke +/// functions such name, type signature, return type, and actual implementation. +/// #[derive(Clone)] pub struct ScalarUDF { - /// name - pub name: String, - /// signature - pub signature: Signature, - /// Return type - pub return_type: ReturnTypeFunction, + /// The name of the function + name: String, + /// The signature (the types of arguments that are supported) + signature: Signature, + /// Function that returns the return type given the argument types + return_type: ReturnTypeFunction, /// actual implementation /// /// The fn param is the wrapped function but be aware that the function will @@ -40,7 +48,7 @@ pub struct ScalarUDF { /// will be passed. In that case the single element is a null array to indicate /// the batch's row count (so that the generative zero-argument function can know /// the result array size). - pub fun: ScalarFunctionImplementation, + fun: ScalarFunctionImplementation, } impl Debug for ScalarUDF { @@ -89,4 +97,28 @@ impl ScalarUDF { pub fn call(&self, args: Vec) -> Expr { Expr::ScalarUDF(crate::expr::ScalarUDF::new(Arc::new(self.clone()), args)) } + + /// Returns this function's name + pub fn name(&self) -> &str { + &self.name + } + + /// Returns this function's signature (what input types are accepted) + pub fn signature(&self) -> &Signature { + &self.signature + } + + /// Return the type of the function given its input types + pub fn return_type(&self, args: &[DataType]) -> Result { + // Old API returns an Arc of the datatype for some reason + let res = (self.return_type)(args)?; + Ok(res.as_ref().clone()) + } + + /// Return the actual implementation + pub fn fun(&self) -> ScalarFunctionImplementation { + self.fun.clone() + } + + // TODO maybe add an invoke() method that runs the actual function? } diff --git a/datafusion/expr/src/udwf.rs b/datafusion/expr/src/udwf.rs index c0a2a8205a08..c233ee84b32d 100644 --- a/datafusion/expr/src/udwf.rs +++ b/datafusion/expr/src/udwf.rs @@ -15,17 +15,19 @@ // specific language governing permissions and limitations // under the License. -//! Support for user-defined window (UDWF) window functions +//! [`WindowUDF`]: User Defined Window Functions +use crate::{ + Expr, PartitionEvaluator, PartitionEvaluatorFactory, ReturnTypeFunction, Signature, + WindowFrame, +}; +use arrow::datatypes::DataType; +use datafusion_common::Result; use std::{ fmt::{self, Debug, Display, Formatter}, sync::Arc, }; -use crate::{ - Expr, PartitionEvaluatorFactory, ReturnTypeFunction, Signature, WindowFrame, -}; - /// Logical representation of a user-defined window function (UDWF) /// A UDWF is different from a UDF in that it is stateful across batches. /// @@ -35,13 +37,13 @@ use crate::{ #[derive(Clone)] pub struct WindowUDF { /// name - pub name: String, + name: String, /// signature - pub signature: Signature, + signature: Signature, /// Return type - pub return_type: ReturnTypeFunction, + return_type: ReturnTypeFunction, /// Return the partition evaluator - pub partition_evaluator_factory: PartitionEvaluatorFactory, + partition_evaluator_factory: PartitionEvaluatorFactory, } impl Debug for WindowUDF { @@ -86,7 +88,7 @@ impl WindowUDF { partition_evaluator_factory: &PartitionEvaluatorFactory, ) -> Self { Self { - name: name.to_owned(), + name: name.to_string(), signature: signature.clone(), return_type: return_type.clone(), partition_evaluator_factory: partition_evaluator_factory.clone(), @@ -115,4 +117,26 @@ impl WindowUDF { window_frame, }) } + + /// Returns this function's name + pub fn name(&self) -> &str { + &self.name + } + + /// Returns this function's signature (what input types are accepted) + pub fn signature(&self) -> &Signature { + &self.signature + } + + /// Return the type of the function given its input types + pub fn return_type(&self, args: &[DataType]) -> Result { + // Old API returns an Arc of the datatype for some reason + let res = (self.return_type)(args)?; + Ok(res.as_ref().clone()) + } + + /// Return a `PartitionEvaluator` for evaluating this window function + pub fn partition_evaluator_factory(&self) -> Result> { + (self.partition_evaluator_factory)() + } } diff --git a/datafusion/expr/src/window_function.rs b/datafusion/expr/src/window_function.rs index 463cceafeb6e..35b7bded70d3 100644 --- a/datafusion/expr/src/window_function.rs +++ b/datafusion/expr/src/window_function.rs @@ -171,12 +171,8 @@ impl WindowFunction { WindowFunction::BuiltInWindowFunction(fun) => { fun.return_type(input_expr_types) } - WindowFunction::AggregateUDF(fun) => { - Ok((*(fun.return_type)(input_expr_types)?).clone()) - } - WindowFunction::WindowUDF(fun) => { - Ok((*(fun.return_type)(input_expr_types)?).clone()) - } + WindowFunction::AggregateUDF(fun) => fun.return_type(input_expr_types), + WindowFunction::WindowUDF(fun) => fun.return_type(input_expr_types), } } } @@ -234,8 +230,8 @@ impl WindowFunction { match self { WindowFunction::AggregateFunction(fun) => fun.signature(), WindowFunction::BuiltInWindowFunction(fun) => fun.signature(), - WindowFunction::AggregateUDF(fun) => fun.signature.clone(), - WindowFunction::WindowUDF(fun) => fun.signature.clone(), + WindowFunction::AggregateUDF(fun) => fun.signature().clone(), + WindowFunction::WindowUDF(fun) => fun.signature().clone(), } } } diff --git a/datafusion/optimizer/src/analyzer/type_coercion.rs b/datafusion/optimizer/src/analyzer/type_coercion.rs index bfdbec390199..57dabbfee41c 100644 --- a/datafusion/optimizer/src/analyzer/type_coercion.rs +++ b/datafusion/optimizer/src/analyzer/type_coercion.rs @@ -323,7 +323,7 @@ impl TreeNodeRewriter for TypeCoercionRewriter { let new_expr = coerce_arguments_for_signature( args.as_slice(), &self.schema, - &fun.signature, + fun.signature(), )?; Ok(Expr::ScalarUDF(ScalarUDF::new(fun, new_expr))) } @@ -364,7 +364,7 @@ impl TreeNodeRewriter for TypeCoercionRewriter { let new_expr = coerce_arguments_for_signature( args.as_slice(), &self.schema, - &fun.signature, + fun.signature(), )?; let expr = Expr::AggregateUDF(expr::AggregateUDF::new( fun, new_expr, filter, order_by, diff --git a/datafusion/optimizer/src/simplify_expressions/expr_simplifier.rs b/datafusion/optimizer/src/simplify_expressions/expr_simplifier.rs index c5a1aacce745..947a6f6070d2 100644 --- a/datafusion/optimizer/src/simplify_expressions/expr_simplifier.rs +++ b/datafusion/optimizer/src/simplify_expressions/expr_simplifier.rs @@ -349,7 +349,7 @@ impl<'a> ConstEvaluator<'a> { Self::volatility_ok(fun.volatility()) } Expr::ScalarUDF(expr::ScalarUDF { fun, .. }) => { - Self::volatility_ok(fun.signature.volatility) + Self::volatility_ok(fun.signature().volatility) } Expr::Literal(_) | Expr::BinaryExpr { .. } diff --git a/datafusion/physical-expr/src/functions.rs b/datafusion/physical-expr/src/functions.rs index 799127c95c98..543d7eb654e2 100644 --- a/datafusion/physical-expr/src/functions.rs +++ b/datafusion/physical-expr/src/functions.rs @@ -78,7 +78,7 @@ pub fn create_physical_expr( &format!("{fun}"), fun_expr, input_phy_exprs.to_vec(), - &data_type, + data_type, monotonicity, ))) } diff --git a/datafusion/physical-expr/src/scalar_function.rs b/datafusion/physical-expr/src/scalar_function.rs index 63101c03bc4a..0a9d69720e19 100644 --- a/datafusion/physical-expr/src/scalar_function.rs +++ b/datafusion/physical-expr/src/scalar_function.rs @@ -77,14 +77,14 @@ impl ScalarFunctionExpr { name: &str, fun: ScalarFunctionImplementation, args: Vec>, - return_type: &DataType, + return_type: DataType, monotonicity: Option, ) -> Self { Self { fun, name: name.to_owned(), args, - return_type: return_type.clone(), + return_type, monotonicity, } } @@ -173,7 +173,7 @@ impl PhysicalExpr for ScalarFunctionExpr { &self.name, self.fun.clone(), children, - self.return_type(), + self.return_type().clone(), self.monotonicity.clone(), ))) } diff --git a/datafusion/physical-expr/src/udf.rs b/datafusion/physical-expr/src/udf.rs index af1e77cbf566..0ec1cf3f256b 100644 --- a/datafusion/physical-expr/src/udf.rs +++ b/datafusion/physical-expr/src/udf.rs @@ -35,10 +35,10 @@ pub fn create_physical_expr( .collect::>>()?; Ok(Arc::new(ScalarFunctionExpr::new( - &fun.name, - fun.fun.clone(), + fun.name(), + fun.fun().clone(), input_phy_exprs.to_vec(), - (fun.return_type)(&input_exprs_types)?.as_ref(), + fun.return_type(&input_exprs_types)?, None, ))) } diff --git a/datafusion/physical-plan/src/udaf.rs b/datafusion/physical-plan/src/udaf.rs index 7cc3cc7d59fe..94017efe97aa 100644 --- a/datafusion/physical-plan/src/udaf.rs +++ b/datafusion/physical-plan/src/udaf.rs @@ -50,7 +50,7 @@ pub fn create_aggregate_expr( Ok(Arc::new(AggregateFunctionExpr { fun: fun.clone(), args: input_phy_exprs.to_vec(), - data_type: (fun.return_type)(&input_exprs_types)?.as_ref().clone(), + data_type: fun.return_type(&input_exprs_types)?, name: name.into(), })) } @@ -83,7 +83,9 @@ impl AggregateExpr for AggregateFunctionExpr { } fn state_fields(&self) -> Result> { - let fields = (self.fun.state_type)(&self.data_type)? + let fields = self + .fun + .state_type(&self.data_type)? .iter() .enumerate() .map(|(i, data_type)| { @@ -103,11 +105,11 @@ impl AggregateExpr for AggregateFunctionExpr { } fn create_accumulator(&self) -> Result> { - (self.fun.accumulator)(&self.data_type) + self.fun.accumulator(&self.data_type) } fn create_sliding_accumulator(&self) -> Result> { - let accumulator = (self.fun.accumulator)(&self.data_type)?; + let accumulator = self.fun.accumulator(&self.data_type)?; // Accumulators that have window frame startings different // than `UNBOUNDED PRECEDING`, such as `1 PRECEEDING`, need to diff --git a/datafusion/physical-plan/src/windows/mod.rs b/datafusion/physical-plan/src/windows/mod.rs index b6ed6e482ff5..541192c00d0c 100644 --- a/datafusion/physical-plan/src/windows/mod.rs +++ b/datafusion/physical-plan/src/windows/mod.rs @@ -255,7 +255,7 @@ fn create_udwf_window_expr( .collect::>()?; // figure out the output type - let data_type = (fun.return_type)(&input_types)?; + let data_type = fun.return_type(&input_types)?; Ok(Arc::new(WindowUDFExpr { fun: Arc::clone(fun), args: args.to_vec(), @@ -272,7 +272,7 @@ struct WindowUDFExpr { /// Display name name: String, /// result type - data_type: Arc, + data_type: DataType, } impl BuiltInWindowFunctionExpr for WindowUDFExpr { @@ -282,11 +282,7 @@ impl BuiltInWindowFunctionExpr for WindowUDFExpr { fn field(&self) -> Result { let nullable = true; - Ok(Field::new( - &self.name, - self.data_type.as_ref().clone(), - nullable, - )) + Ok(Field::new(&self.name, self.data_type.clone(), nullable)) } fn expressions(&self) -> Vec> { @@ -294,7 +290,7 @@ impl BuiltInWindowFunctionExpr for WindowUDFExpr { } fn create_evaluator(&self) -> Result> { - (self.fun.partition_evaluator_factory)() + self.fun.partition_evaluator_factory() } fn name(&self) -> &str { diff --git a/datafusion/proto/src/logical_plan/to_proto.rs b/datafusion/proto/src/logical_plan/to_proto.rs index 491b7f666430..144f28531041 100644 --- a/datafusion/proto/src/logical_plan/to_proto.rs +++ b/datafusion/proto/src/logical_plan/to_proto.rs @@ -613,12 +613,12 @@ impl TryFrom<&Expr> for protobuf::LogicalExprNode { } WindowFunction::AggregateUDF(aggr_udf) => { protobuf::window_expr_node::WindowFunction::Udaf( - aggr_udf.name.clone(), + aggr_udf.name().to_string(), ) } WindowFunction::WindowUDF(window_udf) => { protobuf::window_expr_node::WindowFunction::Udwf( - window_udf.name.clone(), + window_udf.name().to_string(), ) } }; @@ -769,7 +769,7 @@ impl TryFrom<&Expr> for protobuf::LogicalExprNode { } Expr::ScalarUDF(ScalarUDF { fun, args }) => Self { expr_type: Some(ExprType::ScalarUdfExpr(protobuf::ScalarUdfExprNode { - fun_name: fun.name.clone(), + fun_name: fun.name().to_string(), args: args .iter() .map(|expr| expr.try_into()) @@ -784,7 +784,7 @@ impl TryFrom<&Expr> for protobuf::LogicalExprNode { }) => Self { expr_type: Some(ExprType::AggregateUdfExpr(Box::new( protobuf::AggregateUdfExprNode { - fun_name: fun.name.clone(), + fun_name: fun.name().to_string(), args: args.iter().map(|expr| expr.try_into()).collect::, Error, diff --git a/datafusion/proto/src/physical_plan/from_proto.rs b/datafusion/proto/src/physical_plan/from_proto.rs index a628523f0e74..22b74db9afd2 100644 --- a/datafusion/proto/src/physical_plan/from_proto.rs +++ b/datafusion/proto/src/physical_plan/from_proto.rs @@ -18,7 +18,6 @@ //! Serde code to convert from protocol buffers to Rust data structures. use std::convert::{TryFrom, TryInto}; -use std::ops::Deref; use std::sync::Arc; use arrow::compute::SortOptions; @@ -314,12 +313,12 @@ pub fn parse_physical_expr( &e.name, fun_expr, args, - &convert_required!(e.return_type)?, + convert_required!(e.return_type)?, None, )) } ExprType::ScalarUdf(e) => { - let scalar_fun = registry.udf(e.name.as_str())?.deref().clone().fun; + let scalar_fun = registry.udf(e.name.as_str())?.fun().clone(); let args = e .args @@ -331,7 +330,7 @@ pub fn parse_physical_expr( e.name.as_str(), scalar_fun, args, - &convert_required!(e.return_type)?, + convert_required!(e.return_type)?, None, )) } diff --git a/datafusion/proto/src/physical_plan/to_proto.rs b/datafusion/proto/src/physical_plan/to_proto.rs index 8201ef86b528..b8a590b0dc1a 100644 --- a/datafusion/proto/src/physical_plan/to_proto.rs +++ b/datafusion/proto/src/physical_plan/to_proto.rs @@ -84,10 +84,11 @@ impl TryFrom> for protobuf::PhysicalExprNode { .collect::>>()?; if let Some(a) = a.as_any().downcast_ref::() { + let name = a.fun().name().to_string(); return Ok(protobuf::PhysicalExprNode { expr_type: Some(protobuf::physical_expr_node::ExprType::AggregateExpr( protobuf::PhysicalAggregateExprNode { - aggregate_function: Some(physical_aggregate_expr_node::AggregateFunction::UserDefinedAggrFunction(a.fun().name.clone())), + aggregate_function: Some(physical_aggregate_expr_node::AggregateFunction::UserDefinedAggrFunction(name)), expr: expressions, ordering_req, distinct: false, diff --git a/datafusion/proto/tests/cases/roundtrip_logical_plan.rs b/datafusion/proto/tests/cases/roundtrip_logical_plan.rs index cc76e8a19e98..75af9d2e0acb 100644 --- a/datafusion/proto/tests/cases/roundtrip_logical_plan.rs +++ b/datafusion/proto/tests/cases/roundtrip_logical_plan.rs @@ -1559,12 +1559,12 @@ fn roundtrip_window() { Ok(Box::new(DummyWindow {})) } - let dummy_window_udf = WindowUDF { - name: String::from("dummy_udwf"), - signature: Signature::exact(vec![DataType::Float64], Volatility::Immutable), - return_type: Arc::new(return_type), - partition_evaluator_factory: Arc::new(make_partition_evaluator), - }; + let dummy_window_udf = WindowUDF::new( + "dummy_udwf", + &Signature::exact(vec![DataType::Float64], Volatility::Immutable), + &(Arc::new(return_type) as _), + &(Arc::new(make_partition_evaluator) as _), + ); let test_expr6 = Expr::WindowFunction(expr::WindowFunction::new( WindowFunction::WindowUDF(Arc::new(dummy_window_udf.clone())), diff --git a/datafusion/proto/tests/cases/roundtrip_physical_plan.rs b/datafusion/proto/tests/cases/roundtrip_physical_plan.rs index 81e66d5ead36..076ca415810a 100644 --- a/datafusion/proto/tests/cases/roundtrip_physical_plan.rs +++ b/datafusion/proto/tests/cases/roundtrip_physical_plan.rs @@ -522,7 +522,7 @@ fn roundtrip_builtin_scalar_function() -> Result<()> { "acos", fun_expr, vec![col("a", &schema)?], - &DataType::Int64, + DataType::Int64, None, ); @@ -556,7 +556,7 @@ fn roundtrip_scalar_udf() -> Result<()> { "dummy", scalar_fn, vec![col("a", &schema)?], - &DataType::Int64, + DataType::Int64, None, ); From 849d85f5661938fe625f593306716e99611d7705 Mon Sep 17 00:00:00 2001 From: Raphael Taylor-Davies <1781103+tustvold@users.noreply.github.com> Date: Wed, 15 Nov 2023 10:47:12 +0000 Subject: [PATCH 257/572] Fix logical conflicts (#8187) --- .../core/src/physical_optimizer/projection_pushdown.rs | 8 ++++---- .../tests/fuzz_cases/sort_preserving_repartition_fuzz.rs | 5 +++-- datafusion/substrait/src/logical_plan/producer.rs | 2 +- 3 files changed, 8 insertions(+), 7 deletions(-) diff --git a/datafusion/core/src/physical_optimizer/projection_pushdown.rs b/datafusion/core/src/physical_optimizer/projection_pushdown.rs index 8e50492ae5e5..74d0de507e4c 100644 --- a/datafusion/core/src/physical_optimizer/projection_pushdown.rs +++ b/datafusion/core/src/physical_optimizer/projection_pushdown.rs @@ -1146,7 +1146,7 @@ mod tests { Arc::new(Column::new("b", 1)), )), ], - &DataType::Int32, + DataType::Int32, None, )), Arc::new(CaseExpr::try_new( @@ -1212,7 +1212,7 @@ mod tests { Arc::new(Column::new("b", 1)), )), ], - &DataType::Int32, + DataType::Int32, None, )), Arc::new(CaseExpr::try_new( @@ -1281,7 +1281,7 @@ mod tests { Arc::new(Column::new("b", 1)), )), ], - &DataType::Int32, + DataType::Int32, None, )), Arc::new(CaseExpr::try_new( @@ -1347,7 +1347,7 @@ mod tests { Arc::new(Column::new("b_new", 1)), )), ], - &DataType::Int32, + DataType::Int32, None, )), Arc::new(CaseExpr::try_new( diff --git a/datafusion/core/tests/fuzz_cases/sort_preserving_repartition_fuzz.rs b/datafusion/core/tests/fuzz_cases/sort_preserving_repartition_fuzz.rs index 5bc29ba1c277..df6499e9b1e4 100644 --- a/datafusion/core/tests/fuzz_cases/sort_preserving_repartition_fuzz.rs +++ b/datafusion/core/tests/fuzz_cases/sort_preserving_repartition_fuzz.rs @@ -44,6 +44,7 @@ mod sp_repartition_fuzz_tests { }; use test_utils::add_empty_batches; + use datafusion_physical_expr::equivalence::EquivalenceClass; use itertools::izip; use rand::{rngs::StdRng, seq::SliceRandom, Rng, SeedableRng}; @@ -112,7 +113,7 @@ mod sp_repartition_fuzz_tests { // expressions in the equivalence classes. For other expressions in the same // equivalence class use same result. This util gets already calculated result, when available. fn get_representative_arr( - eq_group: &[Arc], + eq_group: &EquivalenceClass, existing_vec: &[Option], schema: SchemaRef, ) -> Option { @@ -185,7 +186,7 @@ mod sp_repartition_fuzz_tests { get_representative_arr(eq_group, &schema_vec, schema.clone()) .unwrap_or_else(|| generate_random_array(n_elem, n_distinct)); - for expr in eq_group { + for expr in eq_group.iter() { let col = expr.as_any().downcast_ref::().unwrap(); let (idx, _field) = schema.column_with_name(col.name()).unwrap(); schema_vec[idx] = Some(representative_array.clone()); diff --git a/datafusion/substrait/src/logical_plan/producer.rs b/datafusion/substrait/src/logical_plan/producer.rs index 9356a7753427..4b6aded78b49 100644 --- a/datafusion/substrait/src/logical_plan/producer.rs +++ b/datafusion/substrait/src/logical_plan/producer.rs @@ -619,7 +619,7 @@ pub fn to_substrait_agg_measure( for arg in args { arguments.push(FunctionArgument { arg_type: Some(ArgType::Value(to_substrait_rex(arg, schema, 0, extension_info)?)) }); } - let function_anchor = _register_function(fun.name.clone(), extension_info); + let function_anchor = _register_function(fun.name().to_string(), extension_info); Ok(Measure { measure: Some(AggregateFunction { function_reference: function_anchor, From 77a63260d19306c4e9235694d1f38eae2bab9ef6 Mon Sep 17 00:00:00 2001 From: Andrew Lamb Date: Wed, 15 Nov 2023 06:28:17 -0500 Subject: [PATCH 258/572] Minor: Update JoinHashMap example to make it clearer (#8154) --- .../physical-plan/src/joins/hash_join.rs | 6 +- .../src/joins/hash_join_utils.rs | 55 ++++++++++--------- 2 files changed, 32 insertions(+), 29 deletions(-) diff --git a/datafusion/physical-plan/src/joins/hash_join.rs b/datafusion/physical-plan/src/joins/hash_join.rs index da57fa07ccd9..7a08b56a6ea7 100644 --- a/datafusion/physical-plan/src/joins/hash_join.rs +++ b/datafusion/physical-plan/src/joins/hash_join.rs @@ -649,6 +649,8 @@ impl ExecutionPlan for HashJoinExec { } } +/// Reads the left (build) side of the input, buffering it in memory, to build a +/// hash table (`LeftJoinData`) async fn collect_left_input( partition: Option, random_state: RandomState, @@ -842,7 +844,7 @@ impl RecordBatchStream for HashJoinStream { /// # Example /// /// For `LEFT.b1 = RIGHT.b2`: -/// LEFT Table: +/// LEFT (build) Table: /// ```text /// a1 b1 c1 /// 1 1 10 @@ -854,7 +856,7 @@ impl RecordBatchStream for HashJoinStream { /// 13 10 130 /// ``` /// -/// RIGHT Table: +/// RIGHT (probe) Table: /// ```text /// a2 b2 c2 /// 2 2 20 diff --git a/datafusion/physical-plan/src/joins/hash_join_utils.rs b/datafusion/physical-plan/src/joins/hash_join_utils.rs index fecbf96f0895..db65c8bf083f 100644 --- a/datafusion/physical-plan/src/joins/hash_join_utils.rs +++ b/datafusion/physical-plan/src/joins/hash_join_utils.rs @@ -61,44 +61,45 @@ use hashbrown::HashSet; /// /// ``` text /// See the example below: -/// Insert (1,1) +/// +/// Insert (10,1) <-- insert hash value 10 with row index 1 /// map: -/// --------- -/// | 1 | 2 | -/// --------- +/// ---------- +/// | 10 | 2 | +/// ---------- /// next: /// --------------------- /// | 0 | 0 | 0 | 0 | 0 | /// --------------------- -/// Insert (2,2) +/// Insert (20,2) /// map: -/// --------- -/// | 1 | 2 | -/// | 2 | 3 | -/// --------- +/// ---------- +/// | 10 | 2 | +/// | 20 | 3 | +/// ---------- /// next: /// --------------------- /// | 0 | 0 | 0 | 0 | 0 | /// --------------------- -/// Insert (1,3) +/// Insert (10,3) <-- collision! row index 3 has a hash value of 10 as well /// map: -/// --------- -/// | 1 | 4 | -/// | 2 | 3 | -/// --------- +/// ---------- +/// | 10 | 4 | +/// | 20 | 3 | +/// ---------- /// next: /// --------------------- -/// | 0 | 0 | 0 | 2 | 0 | <--- hash value 1 maps to 4,2 (which means indices values 3,1) +/// | 0 | 0 | 0 | 2 | 0 | <--- hash value 10 maps to 4,2 (which means indices values 3,1) /// --------------------- -/// Insert (1,4) +/// Insert (10,4) <-- another collision! row index 4 ALSO has a hash value of 10 /// map: /// --------- -/// | 1 | 5 | -/// | 2 | 3 | +/// | 10 | 5 | +/// | 20 | 3 | /// --------- /// next: /// --------------------- -/// | 0 | 0 | 0 | 2 | 4 | <--- hash value 1 maps to 5,4,2 (which means indices values 4,3,1) +/// | 0 | 0 | 0 | 2 | 4 | <--- hash value 10 maps to 5,4,2 (which means indices values 4,3,1) /// --------------------- /// ``` pub struct JoinHashMap { @@ -124,7 +125,7 @@ impl JoinHashMap { /// Trait defining methods that must be implemented by a hash map type to be used for joins. pub trait JoinHashMapType { - /// The type of list used to store the hash values. + /// The type of list used to store the next list type NextType: IndexMut; /// Extend with zero fn extend_zero(&mut self, len: usize); @@ -201,15 +202,15 @@ impl fmt::Debug for JoinHashMap { /// Let's continue the example of `JoinHashMap` and then show how `PruningJoinHashMap` would /// handle the pruning scenario. /// -/// Insert the pair (1,4) into the `PruningJoinHashMap`: +/// Insert the pair (10,4) into the `PruningJoinHashMap`: /// map: -/// --------- -/// | 1 | 5 | -/// | 2 | 3 | -/// --------- +/// ---------- +/// | 10 | 5 | +/// | 20 | 3 | +/// ---------- /// list: /// --------------------- -/// | 0 | 0 | 0 | 2 | 4 | <--- hash value 1 maps to 5,4,2 (which means indices values 4,3,1) +/// | 0 | 0 | 0 | 2 | 4 | <--- hash value 10 maps to 5,4,2 (which means indices values 4,3,1) /// --------------------- /// /// Now, let's prune 3 rows from `PruningJoinHashMap`: @@ -219,7 +220,7 @@ impl fmt::Debug for JoinHashMap { /// --------- /// list: /// --------- -/// | 2 | 4 | <--- hash value 1 maps to 2 (5 - 3), 1 (4 - 3), NA (2 - 3) (which means indices values 1,0) +/// | 2 | 4 | <--- hash value 10 maps to 2 (5 - 3), 1 (4 - 3), NA (2 - 3) (which means indices values 1,0) /// --------- /// /// After pruning, the | 2 | 3 | entry is deleted from `PruningJoinHashMap` since From 020b8fc7619cfa392638da76456331b034479874 Mon Sep 17 00:00:00 2001 From: Raphael Taylor-Davies <1781103+tustvold@users.noreply.github.com> Date: Wed, 15 Nov 2023 12:48:27 +0000 Subject: [PATCH 259/572] Implement StreamTable and StreamTableProvider (#7994) (#8021) * Implement FIFO using extension points (#7994) * Clippy * Rename to StreamTable and make public * Add StreamEncoding * Rework sort order * Fix logical conflicts * Format * Add DefaultTableProvider * Fix doc * Fix project sort keys and CSV headers * Respect batch size on read * Tests are updated * Resolving clippy --------- Co-authored-by: metesynnada <100111937+metesynnada@users.noreply.github.com> --- .../core/src/datasource/listing/table.rs | 37 +- .../src/datasource/listing_table_factory.rs | 9 +- datafusion/core/src/datasource/mod.rs | 44 +++ datafusion/core/src/datasource/provider.rs | 40 +++ datafusion/core/src/datasource/stream.rs | 326 ++++++++++++++++++ datafusion/core/src/execution/context/mod.rs | 14 +- datafusion/core/tests/fifo.rs | 226 ++++++------ datafusion/physical-plan/src/streaming.rs | 21 +- datafusion/sqllogictest/test_files/ddl.slt | 2 +- .../sqllogictest/test_files/groupby.slt | 10 +- datafusion/sqllogictest/test_files/window.slt | 14 +- 11 files changed, 553 insertions(+), 190 deletions(-) create mode 100644 datafusion/core/src/datasource/stream.rs diff --git a/datafusion/core/src/datasource/listing/table.rs b/datafusion/core/src/datasource/listing/table.rs index d26d417bd8b2..c22eb58e88fa 100644 --- a/datafusion/core/src/datasource/listing/table.rs +++ b/datafusion/core/src/datasource/listing/table.rs @@ -26,6 +26,7 @@ use super::PartitionedFile; #[cfg(feature = "parquet")] use crate::datasource::file_format::parquet::ParquetFormat; use crate::datasource::{ + create_ordering, file_format::{ arrow::ArrowFormat, avro::AvroFormat, @@ -40,7 +41,6 @@ use crate::datasource::{ TableProvider, TableType, }; use crate::logical_expr::TableProviderFilterPushDown; -use crate::physical_plan; use crate::{ error::{DataFusionError, Result}, execution::context::SessionState, @@ -48,7 +48,6 @@ use crate::{ physical_plan::{empty::EmptyExec, ExecutionPlan, Statistics}, }; -use arrow::compute::SortOptions; use arrow::datatypes::{DataType, Field, SchemaBuilder, SchemaRef}; use arrow_schema::Schema; use datafusion_common::{ @@ -57,10 +56,9 @@ use datafusion_common::{ }; use datafusion_execution::cache::cache_manager::FileStatisticsCache; use datafusion_execution::cache::cache_unit::DefaultFileStatisticsCache; -use datafusion_expr::expr::Sort; use datafusion_optimizer::utils::conjunction; use datafusion_physical_expr::{ - create_physical_expr, LexOrdering, PhysicalSortExpr, PhysicalSortRequirement, + create_physical_expr, LexOrdering, PhysicalSortRequirement, }; use async_trait::async_trait; @@ -677,34 +675,7 @@ impl ListingTable { /// If file_sort_order is specified, creates the appropriate physical expressions fn try_create_output_ordering(&self) -> Result> { - let mut all_sort_orders = vec![]; - - for exprs in &self.options.file_sort_order { - // Construct PhsyicalSortExpr objects from Expr objects: - let sort_exprs = exprs - .iter() - .map(|expr| { - if let Expr::Sort(Sort { expr, asc, nulls_first }) = expr { - if let Expr::Column(col) = expr.as_ref() { - let expr = physical_plan::expressions::col(&col.name, self.table_schema.as_ref())?; - Ok(PhysicalSortExpr { - expr, - options: SortOptions { - descending: !asc, - nulls_first: *nulls_first, - }, - }) - } else { - plan_err!("Expected single column references in output_ordering, got {expr}") - } - } else { - plan_err!("Expected Expr::Sort in output_ordering, but got {expr}") - } - }) - .collect::>>()?; - all_sort_orders.push(sort_exprs); - } - Ok(all_sort_orders) + create_ordering(&self.table_schema, &self.options.file_sort_order) } } @@ -1040,9 +1011,11 @@ mod tests { use arrow::datatypes::{DataType, Schema}; use arrow::record_batch::RecordBatch; + use arrow_schema::SortOptions; use datafusion_common::stats::Precision; use datafusion_common::{assert_contains, GetExt, ScalarValue}; use datafusion_expr::{BinaryExpr, LogicalPlanBuilder, Operator}; + use datafusion_physical_expr::PhysicalSortExpr; use rstest::*; use tempfile::TempDir; diff --git a/datafusion/core/src/datasource/listing_table_factory.rs b/datafusion/core/src/datasource/listing_table_factory.rs index 26f40518979a..f9a7ab04ce68 100644 --- a/datafusion/core/src/datasource/listing_table_factory.rs +++ b/datafusion/core/src/datasource/listing_table_factory.rs @@ -44,18 +44,13 @@ use datafusion_expr::CreateExternalTable; use async_trait::async_trait; /// A `TableProviderFactory` capable of creating new `ListingTable`s +#[derive(Debug, Default)] pub struct ListingTableFactory {} impl ListingTableFactory { /// Creates a new `ListingTableFactory` pub fn new() -> Self { - Self {} - } -} - -impl Default for ListingTableFactory { - fn default() -> Self { - Self::new() + Self::default() } } diff --git a/datafusion/core/src/datasource/mod.rs b/datafusion/core/src/datasource/mod.rs index 48e9d6992124..45f9bee6a58b 100644 --- a/datafusion/core/src/datasource/mod.rs +++ b/datafusion/core/src/datasource/mod.rs @@ -29,6 +29,7 @@ pub mod memory; pub mod physical_plan; pub mod provider; mod statistics; +pub mod stream; pub mod streaming; pub mod view; @@ -43,3 +44,46 @@ pub use self::provider::TableProvider; pub use self::view::ViewTable; pub use crate::logical_expr::TableType; pub use statistics::get_statistics_with_limit; + +use arrow_schema::{Schema, SortOptions}; +use datafusion_common::{plan_err, DataFusionError, Result}; +use datafusion_expr::Expr; +use datafusion_physical_expr::{expressions, LexOrdering, PhysicalSortExpr}; + +fn create_ordering( + schema: &Schema, + sort_order: &[Vec], +) -> Result> { + let mut all_sort_orders = vec![]; + + for exprs in sort_order { + // Construct PhysicalSortExpr objects from Expr objects: + let mut sort_exprs = vec![]; + for expr in exprs { + match expr { + Expr::Sort(sort) => match sort.expr.as_ref() { + Expr::Column(col) => match expressions::col(&col.name, schema) { + Ok(expr) => { + sort_exprs.push(PhysicalSortExpr { + expr, + options: SortOptions { + descending: !sort.asc, + nulls_first: sort.nulls_first, + }, + }); + } + // Cannot find expression in the projected_schema, stop iterating + // since rest of the orderings are violated + Err(_) => break, + } + expr => return plan_err!("Expected single column references in output_ordering, got {expr}"), + } + expr => return plan_err!("Expected Expr::Sort in output_ordering, but got {expr}"), + } + } + if !sort_exprs.is_empty() { + all_sort_orders.push(sort_exprs); + } + } + Ok(all_sort_orders) +} diff --git a/datafusion/core/src/datasource/provider.rs b/datafusion/core/src/datasource/provider.rs index 7d9f9e86d603..4fe433044e6c 100644 --- a/datafusion/core/src/datasource/provider.rs +++ b/datafusion/core/src/datasource/provider.rs @@ -26,6 +26,8 @@ use datafusion_expr::{CreateExternalTable, LogicalPlan}; pub use datafusion_expr::{TableProviderFilterPushDown, TableType}; use crate::arrow::datatypes::SchemaRef; +use crate::datasource::listing_table_factory::ListingTableFactory; +use crate::datasource::stream::StreamTableFactory; use crate::error::Result; use crate::execution::context::SessionState; use crate::logical_expr::Expr; @@ -214,3 +216,41 @@ pub trait TableProviderFactory: Sync + Send { cmd: &CreateExternalTable, ) -> Result>; } + +/// The default [`TableProviderFactory`] +/// +/// If [`CreateExternalTable`] is unbounded calls [`StreamTableFactory::create`], +/// otherwise calls [`ListingTableFactory::create`] +#[derive(Debug, Default)] +pub struct DefaultTableFactory { + stream: StreamTableFactory, + listing: ListingTableFactory, +} + +impl DefaultTableFactory { + /// Creates a new [`DefaultTableFactory`] + pub fn new() -> Self { + Self::default() + } +} + +#[async_trait] +impl TableProviderFactory for DefaultTableFactory { + async fn create( + &self, + state: &SessionState, + cmd: &CreateExternalTable, + ) -> Result> { + let mut unbounded = cmd.unbounded; + for (k, v) in &cmd.options { + if k.eq_ignore_ascii_case("unbounded") && v.eq_ignore_ascii_case("true") { + unbounded = true + } + } + + match unbounded { + true => self.stream.create(state, cmd).await, + false => self.listing.create(state, cmd).await, + } + } +} diff --git a/datafusion/core/src/datasource/stream.rs b/datafusion/core/src/datasource/stream.rs new file mode 100644 index 000000000000..cf95dd249a7f --- /dev/null +++ b/datafusion/core/src/datasource/stream.rs @@ -0,0 +1,326 @@ +// 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. + +//! TableProvider for stream sources, such as FIFO files + +use std::any::Any; +use std::fmt::Formatter; +use std::fs::{File, OpenOptions}; +use std::io::BufReader; +use std::path::PathBuf; +use std::str::FromStr; +use std::sync::Arc; + +use arrow_array::{RecordBatch, RecordBatchReader, RecordBatchWriter}; +use arrow_schema::SchemaRef; +use async_trait::async_trait; +use futures::StreamExt; +use tokio::task::spawn_blocking; + +use datafusion_common::{plan_err, DataFusionError, Result}; +use datafusion_execution::{SendableRecordBatchStream, TaskContext}; +use datafusion_expr::{CreateExternalTable, Expr, TableType}; +use datafusion_physical_plan::common::AbortOnDropSingle; +use datafusion_physical_plan::insert::{DataSink, FileSinkExec}; +use datafusion_physical_plan::metrics::MetricsSet; +use datafusion_physical_plan::stream::RecordBatchReceiverStreamBuilder; +use datafusion_physical_plan::streaming::{PartitionStream, StreamingTableExec}; +use datafusion_physical_plan::{DisplayAs, DisplayFormatType, ExecutionPlan}; + +use crate::datasource::provider::TableProviderFactory; +use crate::datasource::{create_ordering, TableProvider}; +use crate::execution::context::SessionState; + +/// A [`TableProviderFactory`] for [`StreamTable`] +#[derive(Debug, Default)] +pub struct StreamTableFactory {} + +#[async_trait] +impl TableProviderFactory for StreamTableFactory { + async fn create( + &self, + state: &SessionState, + cmd: &CreateExternalTable, + ) -> Result> { + let schema: SchemaRef = Arc::new(cmd.schema.as_ref().into()); + let location = cmd.location.clone(); + let encoding = cmd.file_type.parse()?; + + let config = StreamConfig::new_file(schema, location.into()) + .with_encoding(encoding) + .with_order(cmd.order_exprs.clone()) + .with_header(cmd.has_header) + .with_batch_size(state.config().batch_size()); + + Ok(Arc::new(StreamTable(Arc::new(config)))) + } +} + +/// The data encoding for [`StreamTable`] +#[derive(Debug, Clone)] +pub enum StreamEncoding { + /// CSV records + Csv, + /// Newline-delimited JSON records + Json, +} + +impl FromStr for StreamEncoding { + type Err = DataFusionError; + + fn from_str(s: &str) -> std::result::Result { + match s.to_ascii_lowercase().as_str() { + "csv" => Ok(Self::Csv), + "json" => Ok(Self::Json), + _ => plan_err!("Unrecognised StreamEncoding {}", s), + } + } +} + +/// The configuration for a [`StreamTable`] +#[derive(Debug)] +pub struct StreamConfig { + schema: SchemaRef, + location: PathBuf, + batch_size: usize, + encoding: StreamEncoding, + header: bool, + order: Vec>, +} + +impl StreamConfig { + /// Stream data from the file at `location` + pub fn new_file(schema: SchemaRef, location: PathBuf) -> Self { + Self { + schema, + location, + batch_size: 1024, + encoding: StreamEncoding::Csv, + order: vec![], + header: false, + } + } + + /// Specify a sort order for the stream + pub fn with_order(mut self, order: Vec>) -> Self { + self.order = order; + self + } + + /// Specify the batch size + pub fn with_batch_size(mut self, batch_size: usize) -> Self { + self.batch_size = batch_size; + self + } + + /// Specify whether the file has a header (only applicable for [`StreamEncoding::Csv`]) + pub fn with_header(mut self, header: bool) -> Self { + self.header = header; + self + } + + /// Specify an encoding for the stream + pub fn with_encoding(mut self, encoding: StreamEncoding) -> Self { + self.encoding = encoding; + self + } + + fn reader(&self) -> Result> { + let file = File::open(&self.location)?; + let schema = self.schema.clone(); + match &self.encoding { + StreamEncoding::Csv => { + let reader = arrow::csv::ReaderBuilder::new(schema) + .with_header(self.header) + .with_batch_size(self.batch_size) + .build(file)?; + + Ok(Box::new(reader)) + } + StreamEncoding::Json => { + let reader = arrow::json::ReaderBuilder::new(schema) + .with_batch_size(self.batch_size) + .build(BufReader::new(file))?; + + Ok(Box::new(reader)) + } + } + } + + fn writer(&self) -> Result> { + match &self.encoding { + StreamEncoding::Csv => { + let header = self.header && !self.location.exists(); + let file = OpenOptions::new().write(true).open(&self.location)?; + let writer = arrow::csv::WriterBuilder::new() + .with_header(header) + .build(file); + + Ok(Box::new(writer)) + } + StreamEncoding::Json => { + let file = OpenOptions::new().write(true).open(&self.location)?; + Ok(Box::new(arrow::json::LineDelimitedWriter::new(file))) + } + } + } +} + +/// A [`TableProvider`] for a stream source, such as a FIFO file +pub struct StreamTable(Arc); + +impl StreamTable { + /// Create a new [`StreamTable`] for the given `StreamConfig` + pub fn new(config: Arc) -> Self { + Self(config) + } +} + +#[async_trait] +impl TableProvider for StreamTable { + fn as_any(&self) -> &dyn Any { + self + } + + fn schema(&self) -> SchemaRef { + self.0.schema.clone() + } + + fn table_type(&self) -> TableType { + TableType::Base + } + + async fn scan( + &self, + _state: &SessionState, + projection: Option<&Vec>, + _filters: &[Expr], + _limit: Option, + ) -> Result> { + let projected_schema = match projection { + Some(p) => { + let projected = self.0.schema.project(p)?; + create_ordering(&projected, &self.0.order)? + } + None => create_ordering(self.0.schema.as_ref(), &self.0.order)?, + }; + + Ok(Arc::new(StreamingTableExec::try_new( + self.0.schema.clone(), + vec![Arc::new(StreamRead(self.0.clone())) as _], + projection, + projected_schema, + true, + )?)) + } + + async fn insert_into( + &self, + _state: &SessionState, + input: Arc, + _overwrite: bool, + ) -> Result> { + let ordering = match self.0.order.first() { + Some(x) => { + let schema = self.0.schema.as_ref(); + let orders = create_ordering(schema, std::slice::from_ref(x))?; + let ordering = orders.into_iter().next().unwrap(); + Some(ordering.into_iter().map(Into::into).collect()) + } + None => None, + }; + + Ok(Arc::new(FileSinkExec::new( + input, + Arc::new(StreamWrite(self.0.clone())), + self.0.schema.clone(), + ordering, + ))) + } +} + +struct StreamRead(Arc); + +impl PartitionStream for StreamRead { + fn schema(&self) -> &SchemaRef { + &self.0.schema + } + + fn execute(&self, _ctx: Arc) -> SendableRecordBatchStream { + let config = self.0.clone(); + let schema = self.0.schema.clone(); + let mut builder = RecordBatchReceiverStreamBuilder::new(schema, 2); + let tx = builder.tx(); + builder.spawn_blocking(move || { + let reader = config.reader()?; + for b in reader { + if tx.blocking_send(b.map_err(Into::into)).is_err() { + break; + } + } + Ok(()) + }); + builder.build() + } +} + +#[derive(Debug)] +struct StreamWrite(Arc); + +impl DisplayAs for StreamWrite { + fn fmt_as(&self, _t: DisplayFormatType, f: &mut Formatter) -> std::fmt::Result { + write!(f, "{self:?}") + } +} + +#[async_trait] +impl DataSink for StreamWrite { + fn as_any(&self) -> &dyn Any { + self + } + + fn metrics(&self) -> Option { + None + } + + async fn write_all( + &self, + mut data: SendableRecordBatchStream, + _context: &Arc, + ) -> Result { + let config = self.0.clone(); + let (sender, mut receiver) = tokio::sync::mpsc::channel::(2); + // Note: FIFO Files support poll so this could use AsyncFd + let write = AbortOnDropSingle::new(spawn_blocking(move || { + let mut count = 0_u64; + let mut writer = config.writer()?; + while let Some(batch) = receiver.blocking_recv() { + count += batch.num_rows() as u64; + writer.write(&batch)?; + } + Ok(count) + })); + + while let Some(b) = data.next().await.transpose()? { + if sender.send(b).await.is_err() { + break; + } + } + drop(sender); + write.await.unwrap() + } +} diff --git a/datafusion/core/src/execution/context/mod.rs b/datafusion/core/src/execution/context/mod.rs index 5c79c407b757..b8e111d361b1 100644 --- a/datafusion/core/src/execution/context/mod.rs +++ b/datafusion/core/src/execution/context/mod.rs @@ -27,7 +27,6 @@ use crate::{ catalog::{CatalogList, MemoryCatalogList}, datasource::{ listing::{ListingOptions, ListingTable}, - listing_table_factory::ListingTableFactory, provider::TableProviderFactory, }, datasource::{MemTable, ViewTable}, @@ -111,6 +110,7 @@ use datafusion_sql::planner::object_name_to_table_reference; use uuid::Uuid; // backwards compatibility +use crate::datasource::provider::DefaultTableFactory; use crate::execution::options::ArrowReadOptions; pub use datafusion_execution::config::SessionConfig; pub use datafusion_execution::TaskContext; @@ -1285,12 +1285,12 @@ impl SessionState { let mut table_factories: HashMap> = HashMap::new(); #[cfg(feature = "parquet")] - table_factories.insert("PARQUET".into(), Arc::new(ListingTableFactory::new())); - table_factories.insert("CSV".into(), Arc::new(ListingTableFactory::new())); - table_factories.insert("JSON".into(), Arc::new(ListingTableFactory::new())); - table_factories.insert("NDJSON".into(), Arc::new(ListingTableFactory::new())); - table_factories.insert("AVRO".into(), Arc::new(ListingTableFactory::new())); - table_factories.insert("ARROW".into(), Arc::new(ListingTableFactory::new())); + table_factories.insert("PARQUET".into(), Arc::new(DefaultTableFactory::new())); + table_factories.insert("CSV".into(), Arc::new(DefaultTableFactory::new())); + table_factories.insert("JSON".into(), Arc::new(DefaultTableFactory::new())); + table_factories.insert("NDJSON".into(), Arc::new(DefaultTableFactory::new())); + table_factories.insert("AVRO".into(), Arc::new(DefaultTableFactory::new())); + table_factories.insert("ARROW".into(), Arc::new(DefaultTableFactory::new())); if config.create_default_catalog_and_schema() { let default_catalog = MemoryCatalogProvider::new(); diff --git a/datafusion/core/tests/fifo.rs b/datafusion/core/tests/fifo.rs index 7d9ea97f7b5b..93c7f7368065 100644 --- a/datafusion/core/tests/fifo.rs +++ b/datafusion/core/tests/fifo.rs @@ -17,42 +17,48 @@ //! This test demonstrates the DataFusion FIFO capabilities. //! -#[cfg(not(target_os = "windows"))] +#[cfg(target_family = "unix")] #[cfg(test)] mod unix_test { - use arrow::array::Array; - use arrow::csv::ReaderBuilder; - use arrow::datatypes::{DataType, Field, Schema}; - use datafusion::test_util::register_unbounded_file_with_ordering; - use datafusion::{ - prelude::{CsvReadOptions, SessionConfig, SessionContext}, - test_util::{aggr_test_schema, arrow_test_data}, - }; - use datafusion_common::{exec_err, DataFusionError, Result}; - use futures::StreamExt; - use itertools::enumerate; - use nix::sys::stat; - use nix::unistd; - use rstest::*; use std::fs::{File, OpenOptions}; use std::io::Write; use std::path::PathBuf; use std::sync::atomic::{AtomicBool, Ordering}; use std::sync::Arc; use std::thread; - use std::thread::JoinHandle; use std::time::{Duration, Instant}; + + use arrow::array::Array; + use arrow::csv::ReaderBuilder; + use arrow::datatypes::{DataType, Field, Schema}; + use arrow_schema::SchemaRef; + use futures::StreamExt; + use nix::sys::stat; + use nix::unistd; use tempfile::TempDir; + use tokio::task::{spawn_blocking, JoinHandle}; - // ! For the sake of the test, do not alter the numbers. ! - // Session batch size - const TEST_BATCH_SIZE: usize = 20; - // Number of lines written to FIFO - const TEST_DATA_SIZE: usize = 20_000; - // Number of lines what can be joined. Each joinable key produced 20 lines with - // aggregate_test_100 dataset. We will use these joinable keys for understanding - // incremental execution. - const TEST_JOIN_RATIO: f64 = 0.01; + use datafusion::datasource::stream::{StreamConfig, StreamTable}; + use datafusion::datasource::TableProvider; + use datafusion::{ + prelude::{CsvReadOptions, SessionConfig, SessionContext}, + test_util::{aggr_test_schema, arrow_test_data}, + }; + use datafusion_common::{exec_err, DataFusionError, Result}; + use datafusion_expr::Expr; + + /// Makes a TableProvider for a fifo file + fn fifo_table( + schema: SchemaRef, + path: impl Into, + sort: Vec>, + ) -> Arc { + let config = StreamConfig::new_file(schema, path.into()) + .with_order(sort) + .with_batch_size(TEST_BATCH_SIZE) + .with_header(true); + Arc::new(StreamTable::new(Arc::new(config))) + } fn create_fifo_file(tmp_dir: &TempDir, file_name: &str) -> Result { let file_path = tmp_dir.path().join(file_name); @@ -86,14 +92,46 @@ mod unix_test { Ok(()) } + fn create_writing_thread( + file_path: PathBuf, + header: String, + lines: Vec, + waiting_lock: Arc, + wait_until: usize, + ) -> JoinHandle<()> { + // Timeout for a long period of BrokenPipe error + let broken_pipe_timeout = Duration::from_secs(10); + let sa = file_path.clone(); + // Spawn a new thread to write to the FIFO file + spawn_blocking(move || { + let file = OpenOptions::new().write(true).open(sa).unwrap(); + // Reference time to use when deciding to fail the test + let execution_start = Instant::now(); + write_to_fifo(&file, &header, execution_start, broken_pipe_timeout).unwrap(); + for (cnt, line) in lines.iter().enumerate() { + while waiting_lock.load(Ordering::SeqCst) && cnt > wait_until { + thread::sleep(Duration::from_millis(50)); + } + write_to_fifo(&file, line, execution_start, broken_pipe_timeout).unwrap(); + } + drop(file); + }) + } + + // ! For the sake of the test, do not alter the numbers. ! + // Session batch size + const TEST_BATCH_SIZE: usize = 20; + // Number of lines written to FIFO + const TEST_DATA_SIZE: usize = 20_000; + // Number of lines what can be joined. Each joinable key produced 20 lines with + // aggregate_test_100 dataset. We will use these joinable keys for understanding + // incremental execution. + const TEST_JOIN_RATIO: f64 = 0.01; + // This test provides a relatively realistic end-to-end scenario where // we swap join sides to accommodate a FIFO source. - #[rstest] - #[timeout(std::time::Duration::from_secs(30))] #[tokio::test(flavor = "multi_thread", worker_threads = 8)] - async fn unbounded_file_with_swapped_join( - #[values(true, false)] unbounded_file: bool, - ) -> Result<()> { + async fn unbounded_file_with_swapped_join() -> Result<()> { // Create session context let config = SessionConfig::new() .with_batch_size(TEST_BATCH_SIZE) @@ -101,11 +139,10 @@ mod unix_test { .with_target_partitions(1); let ctx = SessionContext::new_with_config(config); // To make unbounded deterministic - let waiting = Arc::new(AtomicBool::new(unbounded_file)); + let waiting = Arc::new(AtomicBool::new(true)); // Create a new temporary FIFO file let tmp_dir = TempDir::new()?; - let fifo_path = - create_fifo_file(&tmp_dir, &format!("fifo_{unbounded_file:?}.csv"))?; + let fifo_path = create_fifo_file(&tmp_dir, "fifo_unbounded.csv")?; // Execution can calculated at least one RecordBatch after the number of // "joinable_lines_length" lines are read. let joinable_lines_length = @@ -129,7 +166,7 @@ mod unix_test { "a1,a2\n".to_owned(), lines, waiting.clone(), - joinable_lines_length, + joinable_lines_length * 2, ); // Data Schema @@ -137,15 +174,10 @@ mod unix_test { Field::new("a1", DataType::Utf8, false), Field::new("a2", DataType::UInt32, false), ])); - // Create a file with bounded or unbounded flag. - ctx.register_csv( - "left", - fifo_path.as_os_str().to_str().unwrap(), - CsvReadOptions::new() - .schema(schema.as_ref()) - .mark_infinite(unbounded_file), - ) - .await?; + + let provider = fifo_table(schema, fifo_path, vec![]); + ctx.register_table("left", provider).unwrap(); + // Register right table let schema = aggr_test_schema(); let test_data = arrow_test_data(); @@ -161,7 +193,7 @@ mod unix_test { while (stream.next().await).is_some() { waiting.store(false, Ordering::SeqCst); } - task.join().unwrap(); + task.await.unwrap(); Ok(()) } @@ -172,39 +204,10 @@ mod unix_test { Equal, } - fn create_writing_thread( - file_path: PathBuf, - header: String, - lines: Vec, - waiting_lock: Arc, - wait_until: usize, - ) -> JoinHandle<()> { - // Timeout for a long period of BrokenPipe error - let broken_pipe_timeout = Duration::from_secs(10); - // Spawn a new thread to write to the FIFO file - thread::spawn(move || { - let file = OpenOptions::new().write(true).open(file_path).unwrap(); - // Reference time to use when deciding to fail the test - let execution_start = Instant::now(); - write_to_fifo(&file, &header, execution_start, broken_pipe_timeout).unwrap(); - for (cnt, line) in enumerate(lines) { - while waiting_lock.load(Ordering::SeqCst) && cnt > wait_until { - thread::sleep(Duration::from_millis(50)); - } - write_to_fifo(&file, &line, execution_start, broken_pipe_timeout) - .unwrap(); - } - drop(file); - }) - } - // This test provides a relatively realistic end-to-end scenario where // we change the join into a [SymmetricHashJoin] to accommodate two // unbounded (FIFO) sources. - #[rstest] - #[timeout(std::time::Duration::from_secs(30))] - #[tokio::test(flavor = "multi_thread")] - #[ignore] + #[tokio::test] async fn unbounded_file_with_symmetric_join() -> Result<()> { // Create session context let config = SessionConfig::new() @@ -254,47 +257,30 @@ mod unix_test { Field::new("a1", DataType::UInt32, false), Field::new("a2", DataType::UInt32, false), ])); + // Specify the ordering: - let file_sort_order = vec![[datafusion_expr::col("a1")] - .into_iter() - .map(|e| { - let ascending = true; - let nulls_first = false; - e.sort(ascending, nulls_first) - }) - .collect::>()]; + let order = vec![vec![datafusion_expr::col("a1").sort(true, false)]]; + // Set unbounded sorted files read configuration - register_unbounded_file_with_ordering( - &ctx, - schema.clone(), - &left_fifo, - "left", - file_sort_order.clone(), - true, - ) - .await?; - register_unbounded_file_with_ordering( - &ctx, - schema, - &right_fifo, - "right", - file_sort_order, - true, - ) - .await?; + let provider = fifo_table(schema.clone(), left_fifo, order.clone()); + ctx.register_table("left", provider)?; + + let provider = fifo_table(schema.clone(), right_fifo, order); + ctx.register_table("right", provider)?; + // Execute the query, with no matching rows. (since key is modulus 10) let df = ctx .sql( "SELECT - t1.a1, - t1.a2, - t2.a1, - t2.a2 - FROM - left as t1 FULL - JOIN right as t2 ON t1.a2 = t2.a2 - AND t1.a1 > t2.a1 + 4 - AND t1.a1 < t2.a1 + 9", + t1.a1, + t1.a2, + t2.a1, + t2.a2 + FROM + left as t1 FULL + JOIN right as t2 ON t1.a2 = t2.a2 + AND t1.a1 > t2.a1 + 4 + AND t1.a1 < t2.a1 + 9", ) .await?; let mut stream = df.execute_stream().await?; @@ -313,7 +299,8 @@ mod unix_test { }; operations.push(op); } - tasks.into_iter().for_each(|jh| jh.join().unwrap()); + futures::future::try_join_all(tasks).await.unwrap(); + // The SymmetricHashJoin executor produces FULL join results at every // pruning, which happens before it reaches the end of input and more // than once. In this test, we feed partially joinable data to both @@ -368,8 +355,9 @@ mod unix_test { // Prevent move let (sink_fifo_path_thread, sink_display_fifo_path) = (sink_fifo_path.clone(), sink_fifo_path.display()); + // Spawn a new thread to read sink EXTERNAL TABLE. - tasks.push(thread::spawn(move || { + tasks.push(spawn_blocking(move || { let file = File::open(sink_fifo_path_thread).unwrap(); let schema = Arc::new(Schema::new(vec![ Field::new("a1", DataType::Utf8, false), @@ -377,7 +365,6 @@ mod unix_test { ])); let mut reader = ReaderBuilder::new(schema) - .with_header(true) .with_batch_size(TEST_BATCH_SIZE) .build(file) .map_err(|e| DataFusionError::Internal(e.to_string())) @@ -389,38 +376,35 @@ mod unix_test { })); // register second csv file with the SQL (create an empty file if not found) ctx.sql(&format!( - "CREATE EXTERNAL TABLE source_table ( + "CREATE UNBOUNDED EXTERNAL TABLE source_table ( a1 VARCHAR NOT NULL, a2 INT NOT NULL ) STORED AS CSV WITH HEADER ROW - OPTIONS ('UNBOUNDED' 'TRUE') LOCATION '{source_display_fifo_path}'" )) .await?; // register csv file with the SQL ctx.sql(&format!( - "CREATE EXTERNAL TABLE sink_table ( + "CREATE UNBOUNDED EXTERNAL TABLE sink_table ( a1 VARCHAR NOT NULL, a2 INT NOT NULL ) STORED AS CSV WITH HEADER ROW - OPTIONS ('UNBOUNDED' 'TRUE') LOCATION '{sink_display_fifo_path}'" )) .await?; let df = ctx - .sql( - "INSERT INTO sink_table - SELECT a1, a2 FROM source_table", - ) + .sql("INSERT INTO sink_table SELECT a1, a2 FROM source_table") .await?; + + // Start execution df.collect().await?; - tasks.into_iter().for_each(|jh| jh.join().unwrap()); + futures::future::try_join_all(tasks).await.unwrap(); Ok(()) } } diff --git a/datafusion/physical-plan/src/streaming.rs b/datafusion/physical-plan/src/streaming.rs index 1923a5f3abad..b0eaa2b42f42 100644 --- a/datafusion/physical-plan/src/streaming.rs +++ b/datafusion/physical-plan/src/streaming.rs @@ -55,7 +55,7 @@ pub struct StreamingTableExec { partitions: Vec>, projection: Option>, projected_schema: SchemaRef, - projected_output_ordering: Option, + projected_output_ordering: Vec, infinite: bool, } @@ -65,7 +65,7 @@ impl StreamingTableExec { schema: SchemaRef, partitions: Vec>, projection: Option<&Vec>, - projected_output_ordering: Option, + projected_output_ordering: impl IntoIterator, infinite: bool, ) -> Result { for x in partitions.iter() { @@ -88,7 +88,7 @@ impl StreamingTableExec { partitions, projected_schema, projection: projection.cloned().map(Into::into), - projected_output_ordering, + projected_output_ordering: projected_output_ordering.into_iter().collect(), infinite, }) } @@ -125,7 +125,7 @@ impl DisplayAs for StreamingTableExec { } self.projected_output_ordering - .as_deref() + .first() .map_or(Ok(()), |ordering| { if !ordering.is_empty() { write!( @@ -160,15 +160,16 @@ impl ExecutionPlan for StreamingTableExec { } fn output_ordering(&self) -> Option<&[PhysicalSortExpr]> { - self.projected_output_ordering.as_deref() + self.projected_output_ordering + .first() + .map(|ordering| ordering.as_slice()) } fn equivalence_properties(&self) -> EquivalenceProperties { - let mut result = EquivalenceProperties::new(self.schema()); - if let Some(ordering) = &self.projected_output_ordering { - result.add_new_orderings([ordering.clone()]) - } - result + EquivalenceProperties::new_with_orderings( + self.schema(), + &self.projected_output_ordering, + ) } fn children(&self) -> Vec> { diff --git a/datafusion/sqllogictest/test_files/ddl.slt b/datafusion/sqllogictest/test_files/ddl.slt index ed4f4b4a11ac..682972b5572a 100644 --- a/datafusion/sqllogictest/test_files/ddl.slt +++ b/datafusion/sqllogictest/test_files/ddl.slt @@ -750,7 +750,7 @@ query TT explain select c1 from t; ---- logical_plan TableScan: t projection=[c1] -physical_plan CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/empty.csv]]}, projection=[c1], infinite_source=true, has_header=true +physical_plan StreamingTableExec: partition_sizes=1, projection=[c1], infinite_source=true statement ok drop table t; diff --git a/datafusion/sqllogictest/test_files/groupby.slt b/datafusion/sqllogictest/test_files/groupby.slt index 300e92a7352f..4438d69af306 100644 --- a/datafusion/sqllogictest/test_files/groupby.slt +++ b/datafusion/sqllogictest/test_files/groupby.slt @@ -2115,7 +2115,7 @@ Projection: annotated_data_infinite2.a, annotated_data_infinite2.b, SUM(annotate physical_plan ProjectionExec: expr=[a@1 as a, b@0 as b, SUM(annotated_data_infinite2.c)@2 as summation1] --AggregateExec: mode=Single, gby=[b@1 as b, a@0 as a], aggr=[SUM(annotated_data_infinite2.c)], ordering_mode=Sorted -----CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, b, c], infinite_source=true, output_ordering=[a@0 ASC NULLS LAST, b@1 ASC NULLS LAST, c@2 ASC NULLS LAST], has_header=true +----StreamingTableExec: partition_sizes=1, projection=[a, b, c], infinite_source=true, output_ordering=[a@0 ASC NULLS LAST, b@1 ASC NULLS LAST, c@2 ASC NULLS LAST] query III @@ -2146,7 +2146,7 @@ Projection: annotated_data_infinite2.a, annotated_data_infinite2.d, SUM(annotate physical_plan ProjectionExec: expr=[a@1 as a, d@0 as d, SUM(annotated_data_infinite2.c) ORDER BY [annotated_data_infinite2.a DESC NULLS FIRST]@2 as summation1] --AggregateExec: mode=Single, gby=[d@2 as d, a@0 as a], aggr=[SUM(annotated_data_infinite2.c)], ordering_mode=PartiallySorted([1]) -----CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, c, d], infinite_source=true, output_ordering=[a@0 ASC NULLS LAST], has_header=true +----StreamingTableExec: partition_sizes=1, projection=[a, c, d], infinite_source=true, output_ordering=[a@0 ASC NULLS LAST] query III SELECT a, d, @@ -2179,7 +2179,7 @@ Projection: annotated_data_infinite2.a, annotated_data_infinite2.b, FIRST_VALUE( physical_plan ProjectionExec: expr=[a@0 as a, b@1 as b, FIRST_VALUE(annotated_data_infinite2.c) ORDER BY [annotated_data_infinite2.a DESC NULLS FIRST]@2 as first_c] --AggregateExec: mode=Single, gby=[a@0 as a, b@1 as b], aggr=[FIRST_VALUE(annotated_data_infinite2.c)], ordering_mode=Sorted -----CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, b, c], infinite_source=true, output_ordering=[a@0 ASC NULLS LAST, b@1 ASC NULLS LAST, c@2 ASC NULLS LAST], has_header=true +----StreamingTableExec: partition_sizes=1, projection=[a, b, c], infinite_source=true, output_ordering=[a@0 ASC NULLS LAST, b@1 ASC NULLS LAST, c@2 ASC NULLS LAST] query III SELECT a, b, FIRST_VALUE(c ORDER BY a DESC) as first_c @@ -2205,7 +2205,7 @@ Projection: annotated_data_infinite2.a, annotated_data_infinite2.b, LAST_VALUE(a physical_plan ProjectionExec: expr=[a@0 as a, b@1 as b, LAST_VALUE(annotated_data_infinite2.c) ORDER BY [annotated_data_infinite2.a DESC NULLS FIRST]@2 as last_c] --AggregateExec: mode=Single, gby=[a@0 as a, b@1 as b], aggr=[LAST_VALUE(annotated_data_infinite2.c)], ordering_mode=Sorted -----CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, b, c], infinite_source=true, output_ordering=[a@0 ASC NULLS LAST, b@1 ASC NULLS LAST, c@2 ASC NULLS LAST], has_header=true +----StreamingTableExec: partition_sizes=1, projection=[a, b, c], infinite_source=true, output_ordering=[a@0 ASC NULLS LAST, b@1 ASC NULLS LAST, c@2 ASC NULLS LAST] query III SELECT a, b, LAST_VALUE(c ORDER BY a DESC) as last_c @@ -2232,7 +2232,7 @@ Projection: annotated_data_infinite2.a, annotated_data_infinite2.b, LAST_VALUE(a physical_plan ProjectionExec: expr=[a@0 as a, b@1 as b, LAST_VALUE(annotated_data_infinite2.c)@2 as last_c] --AggregateExec: mode=Single, gby=[a@0 as a, b@1 as b], aggr=[LAST_VALUE(annotated_data_infinite2.c)], ordering_mode=Sorted -----CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, b, c], infinite_source=true, output_ordering=[a@0 ASC NULLS LAST, b@1 ASC NULLS LAST, c@2 ASC NULLS LAST], has_header=true +----StreamingTableExec: partition_sizes=1, projection=[a, b, c], infinite_source=true, output_ordering=[a@0 ASC NULLS LAST, b@1 ASC NULLS LAST, c@2 ASC NULLS LAST] query III SELECT a, b, LAST_VALUE(c) as last_c diff --git a/datafusion/sqllogictest/test_files/window.slt b/datafusion/sqllogictest/test_files/window.slt index 8be02b846cda..a3c57a67a6f0 100644 --- a/datafusion/sqllogictest/test_files/window.slt +++ b/datafusion/sqllogictest/test_files/window.slt @@ -2812,7 +2812,7 @@ ProjectionExec: expr=[sum1@0 as sum1, sum2@1 as sum2, count1@2 as count1, count2 ----ProjectionExec: expr=[SUM(annotated_data_infinite.inc_col) ORDER BY [annotated_data_infinite.ts ASC NULLS LAST] ROWS BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING@4 as sum1, SUM(annotated_data_infinite.inc_col) ORDER BY [annotated_data_infinite.ts DESC NULLS FIRST] ROWS BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING@2 as sum2, COUNT(annotated_data_infinite.inc_col) ORDER BY [annotated_data_infinite.ts ASC NULLS LAST] ROWS BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING@5 as count1, COUNT(annotated_data_infinite.inc_col) ORDER BY [annotated_data_infinite.ts DESC NULLS FIRST] ROWS BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING@3 as count2, ts@0 as ts] ------BoundedWindowAggExec: wdw=[SUM(annotated_data_infinite.inc_col) ORDER BY [annotated_data_infinite.ts ASC NULLS LAST] ROWS BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING: Ok(Field { name: "SUM(annotated_data_infinite.inc_col) ORDER BY [annotated_data_infinite.ts ASC NULLS LAST] ROWS BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(NULL)), end_bound: Following(UInt64(1)) }, COUNT(annotated_data_infinite.inc_col) ORDER BY [annotated_data_infinite.ts ASC NULLS LAST] ROWS BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING: Ok(Field { name: "COUNT(annotated_data_infinite.inc_col) ORDER BY [annotated_data_infinite.ts ASC NULLS LAST] ROWS BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(NULL)), end_bound: Following(UInt64(1)) }], mode=[Sorted] --------BoundedWindowAggExec: wdw=[SUM(annotated_data_infinite.inc_col) ORDER BY [annotated_data_infinite.ts DESC NULLS FIRST] ROWS BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING: Ok(Field { name: "SUM(annotated_data_infinite.inc_col) ORDER BY [annotated_data_infinite.ts DESC NULLS FIRST] ROWS BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(NULL)), end_bound: Following(UInt64(3)) }, COUNT(annotated_data_infinite.inc_col) ORDER BY [annotated_data_infinite.ts DESC NULLS FIRST] ROWS BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING: Ok(Field { name: "COUNT(annotated_data_infinite.inc_col) ORDER BY [annotated_data_infinite.ts DESC NULLS FIRST] ROWS BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(NULL)), end_bound: Following(UInt64(3)) }], mode=[Sorted] -----------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_1.csv]]}, projection=[ts, inc_col], infinite_source=true, output_ordering=[ts@0 ASC NULLS LAST], has_header=true +----------StreamingTableExec: partition_sizes=1, projection=[ts, inc_col], infinite_source=true, output_ordering=[ts@0 ASC NULLS LAST] query IIII @@ -2858,7 +2858,7 @@ ProjectionExec: expr=[sum1@0 as sum1, sum2@1 as sum2, count1@2 as count1, count2 ----ProjectionExec: expr=[SUM(annotated_data_infinite.inc_col) ORDER BY [annotated_data_infinite.ts ASC NULLS LAST] ROWS BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING@4 as sum1, SUM(annotated_data_infinite.inc_col) ORDER BY [annotated_data_infinite.ts DESC NULLS FIRST] ROWS BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING@2 as sum2, COUNT(annotated_data_infinite.inc_col) ORDER BY [annotated_data_infinite.ts ASC NULLS LAST] ROWS BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING@5 as count1, COUNT(annotated_data_infinite.inc_col) ORDER BY [annotated_data_infinite.ts DESC NULLS FIRST] ROWS BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING@3 as count2, ts@0 as ts] ------BoundedWindowAggExec: wdw=[SUM(annotated_data_infinite.inc_col) ORDER BY [annotated_data_infinite.ts ASC NULLS LAST] ROWS BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING: Ok(Field { name: "SUM(annotated_data_infinite.inc_col) ORDER BY [annotated_data_infinite.ts ASC NULLS LAST] ROWS BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(NULL)), end_bound: Following(UInt64(1)) }, COUNT(annotated_data_infinite.inc_col) ORDER BY [annotated_data_infinite.ts ASC NULLS LAST] ROWS BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING: Ok(Field { name: "COUNT(annotated_data_infinite.inc_col) ORDER BY [annotated_data_infinite.ts ASC NULLS LAST] ROWS BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(NULL)), end_bound: Following(UInt64(1)) }], mode=[Sorted] --------BoundedWindowAggExec: wdw=[SUM(annotated_data_infinite.inc_col) ORDER BY [annotated_data_infinite.ts DESC NULLS FIRST] ROWS BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING: Ok(Field { name: "SUM(annotated_data_infinite.inc_col) ORDER BY [annotated_data_infinite.ts DESC NULLS FIRST] ROWS BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(NULL)), end_bound: Following(UInt64(3)) }, COUNT(annotated_data_infinite.inc_col) ORDER BY [annotated_data_infinite.ts DESC NULLS FIRST] ROWS BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING: Ok(Field { name: "COUNT(annotated_data_infinite.inc_col) ORDER BY [annotated_data_infinite.ts DESC NULLS FIRST] ROWS BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(NULL)), end_bound: Following(UInt64(3)) }], mode=[Sorted] -----------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_1.csv]]}, projection=[ts, inc_col], infinite_source=true, output_ordering=[ts@0 ASC NULLS LAST], has_header=true +----------StreamingTableExec: partition_sizes=1, projection=[ts, inc_col], infinite_source=true, output_ordering=[ts@0 ASC NULLS LAST] query IIII @@ -2962,7 +2962,7 @@ ProjectionExec: expr=[a@1 as a, b@2 as b, c@3 as c, SUM(annotated_data_infinite2 ------------BoundedWindowAggExec: wdw=[SUM(annotated_data_infinite2.c) PARTITION BY [annotated_data_infinite2.a, annotated_data_infinite2.b, annotated_data_infinite2.d] ORDER BY [annotated_data_infinite2.c ASC NULLS LAST] ROWS BETWEEN 2 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "SUM(annotated_data_infinite2.c) PARTITION BY [annotated_data_infinite2.a, annotated_data_infinite2.b, annotated_data_infinite2.d] ORDER BY [annotated_data_infinite2.c ASC NULLS LAST] ROWS BETWEEN 2 PRECEDING AND 1 FOLLOWING", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(2)), end_bound: Following(UInt64(1)) }, SUM(annotated_data_infinite2.c) PARTITION BY [annotated_data_infinite2.a, annotated_data_infinite2.b, annotated_data_infinite2.d] ORDER BY [annotated_data_infinite2.c ASC NULLS LAST] ROWS BETWEEN 5 PRECEDING AND CURRENT ROW: Ok(Field { name: "SUM(annotated_data_infinite2.c) PARTITION BY [annotated_data_infinite2.a, annotated_data_infinite2.b, annotated_data_infinite2.d] ORDER BY [annotated_data_infinite2.c ASC NULLS LAST] ROWS BETWEEN 5 PRECEDING AND CURRENT ROW", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(5)), end_bound: CurrentRow }], mode=[PartiallySorted([0, 1])] --------------BoundedWindowAggExec: wdw=[SUM(annotated_data_infinite2.c) PARTITION BY [annotated_data_infinite2.a, annotated_data_infinite2.b] ORDER BY [annotated_data_infinite2.c ASC NULLS LAST] ROWS BETWEEN 2 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "SUM(annotated_data_infinite2.c) PARTITION BY [annotated_data_infinite2.a, annotated_data_infinite2.b] ORDER BY [annotated_data_infinite2.c ASC NULLS LAST] ROWS BETWEEN 2 PRECEDING AND 1 FOLLOWING", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(2)), end_bound: Following(UInt64(1)) }, SUM(annotated_data_infinite2.c) PARTITION BY [annotated_data_infinite2.a, annotated_data_infinite2.b] ORDER BY [annotated_data_infinite2.c ASC NULLS LAST] ROWS BETWEEN 5 PRECEDING AND 5 FOLLOWING: Ok(Field { name: "SUM(annotated_data_infinite2.c) PARTITION BY [annotated_data_infinite2.a, annotated_data_infinite2.b] ORDER BY [annotated_data_infinite2.c ASC NULLS LAST] ROWS BETWEEN 5 PRECEDING AND 5 FOLLOWING", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(5)), end_bound: Following(UInt64(5)) }], mode=[Sorted] ----------------ProjectionExec: expr=[CAST(c@2 AS Int64) as CAST(annotated_data_infinite2.c AS Int64)annotated_data_infinite2.c, a@0 as a, b@1 as b, c@2 as c, d@3 as d] -------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, b, c, d], infinite_source=true, output_ordering=[a@0 ASC NULLS LAST, b@1 ASC NULLS LAST, c@2 ASC NULLS LAST], has_header=true +------------------StreamingTableExec: partition_sizes=1, projection=[a, b, c, d], infinite_source=true, output_ordering=[a@0 ASC NULLS LAST, b@1 ASC NULLS LAST, c@2 ASC NULLS LAST] query IIIIIIIIIIIIIII @@ -3104,7 +3104,7 @@ CoalesceBatchesExec: target_batch_size=4096 ----GlobalLimitExec: skip=0, fetch=5 ------ProjectionExec: expr=[a0@0 as a0, a@1 as a, b@2 as b, c@3 as c, d@4 as d, ROW_NUMBER() ORDER BY [annotated_data_infinite2.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@5 as rn1] --------BoundedWindowAggExec: wdw=[ROW_NUMBER() ORDER BY [annotated_data_infinite2.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "ROW_NUMBER() ORDER BY [annotated_data_infinite2.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW", data_type: UInt64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(NULL)), end_bound: CurrentRow }], mode=[Sorted] -----------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a0, a, b, c, d], infinite_source=true, output_ordering=[a@1 ASC NULLS LAST, b@2 ASC NULLS LAST, c@3 ASC NULLS LAST], has_header=true +----------StreamingTableExec: partition_sizes=1, projection=[a0, a, b, c, d], infinite_source=true, output_ordering=[a@1 ASC NULLS LAST, b@2 ASC NULLS LAST, c@3 ASC NULLS LAST] # this is a negative test for asserting that window functions (other than ROW_NUMBER) # are not added to ordering equivalence @@ -3217,7 +3217,7 @@ ProjectionExec: expr=[SUM(annotated_data_infinite2.a) PARTITION BY [annotated_da ------BoundedWindowAggExec: wdw=[SUM(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.b, annotated_data_infinite2.a] ORDER BY [annotated_data_infinite2.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "SUM(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.b, annotated_data_infinite2.a] ORDER BY [annotated_data_infinite2.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(NULL)), end_bound: CurrentRow }], mode=[Sorted] --------BoundedWindowAggExec: wdw=[SUM(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.a, annotated_data_infinite2.d] ORDER BY [annotated_data_infinite2.b ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "SUM(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.a, annotated_data_infinite2.d] ORDER BY [annotated_data_infinite2.b ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(NULL)), end_bound: CurrentRow }], mode=[PartiallySorted([0])] ----------BoundedWindowAggExec: wdw=[SUM(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.a, annotated_data_infinite2.b] ORDER BY [annotated_data_infinite2.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "SUM(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.a, annotated_data_infinite2.b] ORDER BY [annotated_data_infinite2.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(NULL)), end_bound: CurrentRow }], mode=[Sorted] -------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, b, c, d], infinite_source=true, output_ordering=[a@0 ASC NULLS LAST, b@1 ASC NULLS LAST, c@2 ASC NULLS LAST], has_header=true +------------StreamingTableExec: partition_sizes=1, projection=[a, b, c, d], infinite_source=true, output_ordering=[a@0 ASC NULLS LAST, b@1 ASC NULLS LAST, c@2 ASC NULLS LAST] statement ok set datafusion.execution.target_partitions = 2; @@ -3255,7 +3255,7 @@ ProjectionExec: expr=[SUM(annotated_data_infinite2.a) PARTITION BY [annotated_da ------------------------CoalesceBatchesExec: target_batch_size=4096 --------------------------SortPreservingRepartitionExec: partitioning=Hash([a@0, b@1], 2), input_partitions=2, sort_exprs=a@0 ASC NULLS LAST,b@1 ASC NULLS LAST,c@2 ASC NULLS LAST ----------------------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, b, c, d], infinite_source=true, output_ordering=[a@0 ASC NULLS LAST, b@1 ASC NULLS LAST, c@2 ASC NULLS LAST], has_header=true +------------------------------StreamingTableExec: partition_sizes=1, projection=[a, b, c, d], infinite_source=true, output_ordering=[a@0 ASC NULLS LAST, b@1 ASC NULLS LAST, c@2 ASC NULLS LAST] # reset the partition number 1 again statement ok @@ -3521,4 +3521,4 @@ SortPreservingMergeExec: [c@3 ASC NULLS LAST] ------CoalesceBatchesExec: target_batch_size=4096 --------SortPreservingRepartitionExec: partitioning=Hash([d@4], 2), input_partitions=2, sort_exprs=a@1 ASC NULLS LAST,b@2 ASC NULLS LAST ----------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a0, a, b, c, d], infinite_source=true, output_ordering=[a@1 ASC NULLS LAST, b@2 ASC NULLS LAST], has_header=true +------------StreamingTableExec: partition_sizes=1, projection=[a0, a, b, c, d], infinite_source=true, output_ordering=[a@1 ASC NULLS LAST, b@2 ASC NULLS LAST] From 67d66faa829ea2fe102384a7534f86e66a3027b7 Mon Sep 17 00:00:00 2001 From: Mustafa Akur <106137913+mustafasrepo@users.noreply.github.com> Date: Wed, 15 Nov 2023 18:02:00 +0300 Subject: [PATCH 260/572] Remove unused Results (#8189) --- datafusion/physical-plan/src/aggregates/mod.rs | 10 +++++----- datafusion/physical-plan/src/windows/mod.rs | 14 +++++++------- 2 files changed, 12 insertions(+), 12 deletions(-) diff --git a/datafusion/physical-plan/src/aggregates/mod.rs b/datafusion/physical-plan/src/aggregates/mod.rs index 3ac812929772..7d7fba6ef6c3 100644 --- a/datafusion/physical-plan/src/aggregates/mod.rs +++ b/datafusion/physical-plan/src/aggregates/mod.rs @@ -405,7 +405,7 @@ fn get_aggregate_search_mode( aggr_expr: &mut [Arc], order_by_expr: &mut [Option], ordering_req: &mut Vec, -) -> Result { +) -> PartitionSearchMode { let groupby_exprs = group_by .expr .iter() @@ -413,11 +413,11 @@ fn get_aggregate_search_mode( .collect::>(); let mut partition_search_mode = PartitionSearchMode::Linear; if !group_by.is_single() || groupby_exprs.is_empty() { - return Ok(partition_search_mode); + return partition_search_mode; } if let Some((should_reverse, mode)) = - get_window_mode(&groupby_exprs, ordering_req, input)? + get_window_mode(&groupby_exprs, ordering_req, input) { let all_reversible = aggr_expr .iter() @@ -437,7 +437,7 @@ fn get_aggregate_search_mode( } partition_search_mode = mode; } - Ok(partition_search_mode) + partition_search_mode } /// Check whether group by expression contains all of the expression inside `requirement` @@ -513,7 +513,7 @@ impl AggregateExec { &mut aggr_expr, &mut order_by_expr, &mut ordering_req, - )?; + ); // Get GROUP BY expressions: let groupby_exprs = group_by.input_exprs(); diff --git a/datafusion/physical-plan/src/windows/mod.rs b/datafusion/physical-plan/src/windows/mod.rs index 541192c00d0c..d97e3c93a136 100644 --- a/datafusion/physical-plan/src/windows/mod.rs +++ b/datafusion/physical-plan/src/windows/mod.rs @@ -405,7 +405,7 @@ pub fn get_best_fitting_window( let orderby_keys = window_exprs[0].order_by(); let (should_reverse, partition_search_mode) = if let Some((should_reverse, partition_search_mode)) = - get_window_mode(partitionby_exprs, orderby_keys, input)? + get_window_mode(partitionby_exprs, orderby_keys, input) { (should_reverse, partition_search_mode) } else { @@ -467,12 +467,12 @@ pub fn get_best_fitting_window( /// can run with existing input ordering, so we can remove `SortExec` before it. /// The `bool` field in the return value represents whether we should reverse window /// operator to remove `SortExec` before it. The `PartitionSearchMode` field represents -/// the mode this window operator should work in to accomodate the existing ordering. +/// the mode this window operator should work in to accommodate the existing ordering. pub fn get_window_mode( partitionby_exprs: &[Arc], orderby_keys: &[PhysicalSortExpr], input: &Arc, -) -> Result> { +) -> Option<(bool, PartitionSearchMode)> { let input_eqs = input.equivalence_properties(); let mut partition_by_reqs: Vec = vec![]; let (_, indices) = input_eqs.find_longest_permutation(partitionby_exprs); @@ -499,10 +499,10 @@ pub fn get_window_mode( } else { PartitionSearchMode::PartiallySorted(indices) }; - return Ok(Some((should_swap, mode))); + return Some((should_swap, mode)); } } - Ok(None) + None } #[cfg(test)] @@ -869,7 +869,7 @@ mod tests { order_by_exprs.push(PhysicalSortExpr { expr, options }); } let res = - get_window_mode(&partition_by_exprs, &order_by_exprs, &exec_unbounded)?; + get_window_mode(&partition_by_exprs, &order_by_exprs, &exec_unbounded); // Since reversibility is not important in this test. Convert Option<(bool, PartitionSearchMode)> to Option let res = res.map(|(_, mode)| mode); assert_eq!( @@ -1033,7 +1033,7 @@ mod tests { } assert_eq!( - get_window_mode(&partition_by_exprs, &order_by_exprs, &exec_unbounded)?, + get_window_mode(&partition_by_exprs, &order_by_exprs, &exec_unbounded), *expected, "Unexpected result for in unbounded test case#: {case_idx:?}, case: {test_case:?}" ); From 841a9a6e4120395ed3df3423b2831531ba8a3fad Mon Sep 17 00:00:00 2001 From: Alex Huang Date: Wed, 15 Nov 2023 16:02:21 +0100 Subject: [PATCH 261/572] Minor: clean up the code based on clippy (#8179) --- datafusion/physical-expr/src/array_expressions.rs | 8 ++++---- datafusion/physical-plan/src/filter.rs | 2 +- datafusion/sql/src/select.rs | 6 +----- 3 files changed, 6 insertions(+), 10 deletions(-) diff --git a/datafusion/physical-expr/src/array_expressions.rs b/datafusion/physical-expr/src/array_expressions.rs index 6415bd5391d5..01d495ee7f6b 100644 --- a/datafusion/physical-expr/src/array_expressions.rs +++ b/datafusion/physical-expr/src/array_expressions.rs @@ -1440,7 +1440,7 @@ fn union_generic_lists( r: &GenericListArray, field: &FieldRef, ) -> Result> { - let converter = RowConverter::new(vec![SortField::new(l.value_type().clone())])?; + let converter = RowConverter::new(vec![SortField::new(l.value_type())])?; let nulls = NullBuffer::union(l.nulls(), r.nulls()); let l_values = l.values().clone(); @@ -1494,14 +1494,14 @@ pub fn array_union(args: &[ArrayRef]) -> Result { (DataType::Null, _) => Ok(array2.clone()), (_, DataType::Null) => Ok(array1.clone()), (DataType::List(field_ref), DataType::List(_)) => { - check_datatypes("array_union", &[&array1, &array2])?; + check_datatypes("array_union", &[array1, array2])?; let list1 = array1.as_list::(); let list2 = array2.as_list::(); let result = union_generic_lists::(list1, list2, field_ref)?; Ok(Arc::new(result)) } (DataType::LargeList(field_ref), DataType::LargeList(_)) => { - check_datatypes("array_union", &[&array1, &array2])?; + check_datatypes("array_union", &[array1, array2])?; let list1 = array1.as_list::(); let list2 = array2.as_list::(); let result = union_generic_lists::(list1, list2, field_ref)?; @@ -1985,7 +1985,7 @@ pub fn array_intersect(args: &[ArrayRef]) -> Result { if first_array.value_type() != second_array.value_type() { return internal_err!("array_intersect is not implemented for '{first_array:?}' and '{second_array:?}'"); } - let dt = first_array.value_type().clone(); + let dt = first_array.value_type(); let mut offsets = vec![0]; let mut new_arrays = vec![]; diff --git a/datafusion/physical-plan/src/filter.rs b/datafusion/physical-plan/src/filter.rs index 822ddfdf3eb0..52bff880b127 100644 --- a/datafusion/physical-plan/src/filter.rs +++ b/datafusion/physical-plan/src/filter.rs @@ -201,7 +201,7 @@ impl ExecutionPlan for FilterExec { // tracking issue for making this configurable: // https://github.com/apache/arrow-datafusion/issues/8133 let selectivity = 0.2_f32; - let mut stats = input_stats.clone().into_inexact(); + let mut stats = input_stats.into_inexact(); if let Precision::Inexact(n) = stats.num_rows { stats.num_rows = Precision::Inexact((selectivity * n as f32) as usize); } diff --git a/datafusion/sql/src/select.rs b/datafusion/sql/src/select.rs index 31333affe0af..356c53605131 100644 --- a/datafusion/sql/src/select.rs +++ b/datafusion/sql/src/select.rs @@ -245,11 +245,7 @@ impl<'a, S: ContextProvider> SqlToRel<'a, S> { let on_expr = on_expr .into_iter() .map(|e| { - self.sql_expr_to_logical_expr( - e.clone(), - plan.schema(), - planner_context, - ) + self.sql_expr_to_logical_expr(e, plan.schema(), planner_context) }) .collect::>>()?; From e1c2f9583015db326b3439897376f14f6b83a99a Mon Sep 17 00:00:00 2001 From: Andrew Lamb Date: Wed, 15 Nov 2023 12:49:31 -0500 Subject: [PATCH 262/572] Minor: simplify filter statistics code (#8174) * Minor: simplify filter statistics code * remove comment --- .../core/src/physical_optimizer/join_selection.rs | 1 + datafusion/physical-plan/src/filter.rs | 10 ++-------- 2 files changed, 3 insertions(+), 8 deletions(-) diff --git a/datafusion/core/src/physical_optimizer/join_selection.rs b/datafusion/core/src/physical_optimizer/join_selection.rs index 876a464257cc..a7ecd1ca655c 100644 --- a/datafusion/core/src/physical_optimizer/join_selection.rs +++ b/datafusion/core/src/physical_optimizer/join_selection.rs @@ -95,6 +95,7 @@ fn supports_collect_by_size( let Ok(stats) = plan.statistics() else { return false; }; + if let Some(size) = stats.total_byte_size.get_value() { *size != 0 && *size < collection_size_threshold } else if let Some(row_count) = stats.num_rows.get_value() { diff --git a/datafusion/physical-plan/src/filter.rs b/datafusion/physical-plan/src/filter.rs index 52bff880b127..597e1d523a24 100644 --- a/datafusion/physical-plan/src/filter.rs +++ b/datafusion/physical-plan/src/filter.rs @@ -277,16 +277,10 @@ fn collect_new_statistics( ) }; ColumnStatistics { - null_count: match input_column_stats[idx].null_count.get_value() { - Some(nc) => Precision::Inexact(*nc), - None => Precision::Absent, - }, + null_count: input_column_stats[idx].null_count.clone().to_inexact(), max_value, min_value, - distinct_count: match distinct_count.get_value() { - Some(dc) => Precision::Inexact(*dc), - None => Precision::Absent, - }, + distinct_count: distinct_count.to_inexact(), } }, ) From 7c2c2f029730756d433602a3cc501f695792e58d Mon Sep 17 00:00:00 2001 From: Jay Zhan Date: Thu, 16 Nov 2023 01:52:53 +0800 Subject: [PATCH 263/572] Replace macro with function for `array_position` and `array_positions` (#8170) * basic one Signed-off-by: jayzhan211 * complete n Signed-off-by: jayzhan211 * positions done Signed-off-by: jayzhan211 * compare_element_to_list Signed-off-by: jayzhan211 * fmt Signed-off-by: jayzhan211 * resolve rebase Signed-off-by: jayzhan211 * fmt Signed-off-by: jayzhan211 --------- Signed-off-by: jayzhan211 --- .../physical-expr/src/array_expressions.rs | 309 +++++++++--------- datafusion/sqllogictest/test_files/array.slt | 12 +- 2 files changed, 168 insertions(+), 153 deletions(-) diff --git a/datafusion/physical-expr/src/array_expressions.rs b/datafusion/physical-expr/src/array_expressions.rs index 01d495ee7f6b..515df2a970a4 100644 --- a/datafusion/physical-expr/src/array_expressions.rs +++ b/datafusion/physical-expr/src/array_expressions.rs @@ -131,6 +131,78 @@ macro_rules! array { }}; } +/// Computes a BooleanArray indicating equality or inequality between elements in a list array and a specified element array. +/// +/// # Arguments +/// +/// * `list_array_row` - A reference to a trait object implementing the Arrow `Array` trait. It represents the list array for which the equality or inequality will be compared. +/// +/// * `element_array` - A reference to a trait object implementing the Arrow `Array` trait. It represents the array with which each element in the `list_array_row` will be compared. +/// +/// * `row_index` - The index of the row in the `element_array` and `list_array` to use for the comparison. +/// +/// * `eq` - A boolean flag. If `true`, the function computes equality; if `false`, it computes inequality. +/// +/// # Returns +/// +/// Returns a `Result` representing the comparison results. The result may contain an error if there are issues with the computation. +/// +/// # Example +/// +/// ```text +/// compare_element_to_list( +/// [1, 2, 3], [1, 2, 3], 0, true => [true, false, false] +/// [1, 2, 3, 3, 2, 1], [1, 2, 3], 1, true => [false, true, false, false, true, false] +/// +/// [[1, 2, 3], [2, 3, 4], [3, 4, 5]], [[1, 2, 3], [2, 3, 4], [3, 4, 5]], 0, true => [true, false, false] +/// [[1, 2, 3], [2, 3, 4], [2, 3, 4]], [[1, 2, 3], [2, 3, 4], [3, 4, 5]], 1, false => [true, false, false] +/// ) +/// ``` +fn compare_element_to_list( + list_array_row: &dyn Array, + element_array: &dyn Array, + row_index: usize, + eq: bool, +) -> Result { + let indices = UInt32Array::from(vec![row_index as u32]); + let element_array_row = arrow::compute::take(element_array, &indices, None)?; + // Compute all positions in list_row_array (that is itself an + // array) that are equal to `from_array_row` + let res = match element_array_row.data_type() { + // arrow_ord::cmp::eq does not support ListArray, so we need to compare it by loop + DataType::List(_) => { + // compare each element of the from array + let element_array_row_inner = as_list_array(&element_array_row)?.value(0); + let list_array_row_inner = as_list_array(list_array_row)?; + + list_array_row_inner + .iter() + // compare element by element the current row of list_array + .map(|row| { + row.map(|row| { + if eq { + row.eq(&element_array_row_inner) + } else { + row.ne(&element_array_row_inner) + } + }) + }) + .collect::() + } + _ => { + let element_arr = Scalar::new(element_array_row); + // use not_distinct so we can compare NULL + if eq { + arrow_ord::cmp::not_distinct(&list_array_row, &element_arr)? + } else { + arrow_ord::cmp::distinct(&list_array_row, &element_arr)? + } + } + }; + + Ok(res) +} + /// Returns the length of a concrete array dimension fn compute_array_length( arr: Option, @@ -1005,114 +1077,68 @@ fn general_list_repeat( )?)) } -macro_rules! position { - ($ARRAY:expr, $ELEMENT:expr, $INDEX:expr, $ARRAY_TYPE:ident) => {{ - let element = downcast_arg!($ELEMENT, $ARRAY_TYPE); - $ARRAY - .iter() - .zip(element.iter()) - .zip($INDEX.iter()) - .map(|((arr, el), i)| { - let index = match i { - Some(i) => { - if i <= 0 { - 0 - } else { - i - 1 - } - } - None => return exec_err!("initial position must not be null"), - }; - - match arr { - Some(arr) => { - let child_array = downcast_arg!(arr, $ARRAY_TYPE); - - match child_array - .iter() - .skip(index as usize) - .position(|x| x == el) - { - Some(value) => Ok(Some(value as u64 + index as u64 + 1u64)), - None => Ok(None), - } - } - None => Ok(None), - } - }) - .collect::>()? - }}; -} - /// Array_position SQL function pub fn array_position(args: &[ArrayRef]) -> Result { - let arr = as_list_array(&args[0])?; - let element = &args[1]; + let list_array = as_list_array(&args[0])?; + let element_array = &args[1]; - let index = if args.len() == 3 { - as_int64_array(&args[2])?.clone() + check_datatypes("array_position", &[list_array.values(), element_array])?; + + let arr_from = if args.len() == 3 { + as_int64_array(&args[2])? + .values() + .to_vec() + .iter() + .map(|&x| x - 1) + .collect::>() } else { - Int64Array::from_value(0, arr.len()) + vec![0; list_array.len()] }; - check_datatypes("array_position", &[arr.values(), element])?; - macro_rules! array_function { - ($ARRAY_TYPE:ident) => { - position!(arr, element, index, $ARRAY_TYPE) - }; + // if `start_from` index is out of bounds, return error + for (arr, &from) in list_array.iter().zip(arr_from.iter()) { + if let Some(arr) = arr { + if from < 0 || from as usize >= arr.len() { + return internal_err!("start_from index out of bounds"); + } + } else { + // We will get null if we got null in the array, so we don't need to check + } } - let res = call_array_function!(arr.value_type(), true); - Ok(Arc::new(res)) + general_position::(list_array, element_array, arr_from) } -macro_rules! positions { - ($ARRAY:expr, $ELEMENT:expr, $ARRAY_TYPE:ident) => {{ - let element = downcast_arg!($ELEMENT, $ARRAY_TYPE); - let mut offsets: Vec = vec![0]; - let mut values = - downcast_arg!(new_empty_array(&DataType::UInt64), UInt64Array).clone(); - for comp in $ARRAY - .iter() - .zip(element.iter()) - .map(|(arr, el)| match arr { - Some(arr) => { - let child_array = downcast_arg!(arr, $ARRAY_TYPE); - let res = child_array - .iter() - .enumerate() - .filter(|(_, x)| *x == el) - .flat_map(|(i, _)| Some((i + 1) as u64)) - .collect::(); +fn general_position( + list_array: &GenericListArray, + element_array: &ArrayRef, + arr_from: Vec, // 0-indexed +) -> Result { + let mut data = Vec::with_capacity(list_array.len()); - Ok(res) - } - None => Ok(downcast_arg!( - new_empty_array(&DataType::UInt64), - UInt64Array - ) - .clone()), - }) - .collect::>>()? - { - let last_offset: i32 = offsets.last().copied().ok_or_else(|| { - DataFusionError::Internal(format!("offsets should not be empty",)) - })?; - values = - downcast_arg!(compute::concat(&[&values, &comp,])?.clone(), UInt64Array) - .clone(); - offsets.push(last_offset + comp.len() as i32); - } + for (row_index, (list_array_row, &from)) in + list_array.iter().zip(arr_from.iter()).enumerate() + { + let from = from as usize; - let field = Arc::new(Field::new("item", DataType::UInt64, true)); + if let Some(list_array_row) = list_array_row { + let eq_array = + compare_element_to_list(&list_array_row, element_array, row_index, true)?; - Arc::new(ListArray::try_new( - field, - OffsetBuffer::new(offsets.into()), - Arc::new(values), - None, - )?) - }}; + // Collect `true`s in 1-indexed positions + let index = eq_array + .iter() + .skip(from) + .position(|e| e == Some(true)) + .map(|index| (from + index + 1) as u64); + + data.push(index); + } else { + data.push(None); + } + } + + Ok(Arc::new(UInt64Array::from(data))) } /// Array_positions SQL function @@ -1121,14 +1147,37 @@ pub fn array_positions(args: &[ArrayRef]) -> Result { let element = &args[1]; check_datatypes("array_positions", &[arr.values(), element])?; - macro_rules! array_function { - ($ARRAY_TYPE:ident) => { - positions!(arr, element, $ARRAY_TYPE) - }; + + general_positions::(arr, element) +} + +fn general_positions( + list_array: &GenericListArray, + element_array: &ArrayRef, +) -> Result { + let mut data = Vec::with_capacity(list_array.len()); + + for (row_index, list_array_row) in list_array.iter().enumerate() { + if let Some(list_array_row) = list_array_row { + let eq_array = + compare_element_to_list(&list_array_row, element_array, row_index, true)?; + + // Collect `true`s in 1-indexed positions + let indexes = eq_array + .iter() + .positions(|e| e == Some(true)) + .map(|index| Some(index as u64 + 1)) + .collect::>(); + + data.push(Some(indexes)); + } else { + data.push(None); + } } - let res = call_array_function!(arr.value_type(), true); - Ok(res) + Ok(Arc::new( + ListArray::from_iter_primitive::(data), + )) } /// For each element of `list_array[i]`, removed up to `arr_n[i]` occurences @@ -1165,30 +1214,12 @@ fn general_remove( { match list_array_row { Some(list_array_row) => { - let indices = UInt32Array::from(vec![row_index as u32]); - let element_array_row = - arrow::compute::take(element_array, &indices, None)?; - - let eq_array = match element_array_row.data_type() { - // arrow_ord::cmp::distinct does not support ListArray, so we need to compare it by loop - DataType::List(_) => { - // compare each element of the from array - let element_array_row_inner = - as_list_array(&element_array_row)?.value(0); - let list_array_row_inner = as_list_array(&list_array_row)?; - - list_array_row_inner - .iter() - // compare element by element the current row of list_array - .map(|row| row.map(|row| row.ne(&element_array_row_inner))) - .collect::() - } - _ => { - let from_arr = Scalar::new(element_array_row); - // use distinct so Null = Null is false - arrow_ord::cmp::distinct(&list_array_row, &from_arr)? - } - }; + let eq_array = compare_element_to_list( + &list_array_row, + element_array, + row_index, + false, + )?; // We need to keep at most first n elements as `false`, which represent the elements to remove. let eq_array = if eq_array.false_count() < *n as usize { @@ -1313,30 +1344,14 @@ fn general_replace( match list_array_row { Some(list_array_row) => { - let indices = UInt32Array::from(vec![row_index as u32]); - let from_array_row = arrow::compute::take(from_array, &indices, None)?; // Compute all positions in list_row_array (that is itself an // array) that are equal to `from_array_row` - let eq_array = match from_array_row.data_type() { - // arrow_ord::cmp::eq does not support ListArray, so we need to compare it by loop - DataType::List(_) => { - // compare each element of the from array - let from_array_row_inner = - as_list_array(&from_array_row)?.value(0); - let list_array_row_inner = as_list_array(&list_array_row)?; - - list_array_row_inner - .iter() - // compare element by element the current row of list_array - .map(|row| row.map(|row| row.eq(&from_array_row_inner))) - .collect::() - } - _ => { - let from_arr = Scalar::new(from_array_row); - // use not_distinct so NULL = NULL - arrow_ord::cmp::not_distinct(&list_array_row, &from_arr)? - } - }; + let eq_array = compare_element_to_list( + &list_array_row, + &from_array, + row_index, + true, + )?; // Use MutableArrayData to build the replaced array let original_data = list_array_row.to_data(); diff --git a/datafusion/sqllogictest/test_files/array.slt b/datafusion/sqllogictest/test_files/array.slt index 92013f37d36c..67cabb0988fd 100644 --- a/datafusion/sqllogictest/test_files/array.slt +++ b/datafusion/sqllogictest/test_files/array.slt @@ -702,7 +702,7 @@ select array_element(make_array(1, 2, 3, 4, 5), 0), array_element(make_array('h' NULL NULL # array_element scalar function #4 (with NULL) -query error +query error select array_element(make_array(1, 2, 3, 4, 5), NULL), array_element(make_array('h', 'e', 'l', 'l', 'o'), NULL); # array_element scalar function #5 (with negative index) @@ -871,11 +871,11 @@ select array_slice(make_array(1, 2, 3, 4, 5), 0, 4), array_slice(make_array('h', [1, 2, 3, 4] [h, e, l] # array_slice scalar function #8 (with NULL and positive number) -query error +query error select array_slice(make_array(1, 2, 3, 4, 5), NULL, 4), array_slice(make_array('h', 'e', 'l', 'l', 'o'), NULL, 3); # array_slice scalar function #9 (with positive number and NULL) -query error +query error select array_slice(make_array(1, 2, 3, 4, 5), 2, NULL), array_slice(make_array('h', 'e', 'l', 'l', 'o'), 3, NULL); # array_slice scalar function #10 (with zero-zero) @@ -885,7 +885,7 @@ select array_slice(make_array(1, 2, 3, 4, 5), 0, 0), array_slice(make_array('h', [] [] # array_slice scalar function #11 (with NULL-NULL) -query error +query error select array_slice(make_array(1, 2, 3, 4, 5), NULL), array_slice(make_array('h', 'e', 'l', 'l', 'o'), NULL); # array_slice scalar function #12 (with zero and negative number) @@ -895,11 +895,11 @@ select array_slice(make_array(1, 2, 3, 4, 5), 0, -4), array_slice(make_array('h' [1] [h, e] # array_slice scalar function #13 (with negative number and NULL) -query error +query error select array_slice(make_array(1, 2, 3, 4, 5), 2, NULL), array_slice(make_array('h', 'e', 'l', 'l', 'o'), 3, NULL); # array_slice scalar function #14 (with NULL and negative number) -query error +query error select array_slice(make_array(1, 2, 3, 4, 5), NULL, -4), array_slice(make_array('h', 'e', 'l', 'l', 'o'), NULL, -3); # array_slice scalar function #15 (with negative indexes) From cd1c648e719fdfacbd7da586fed5251f5f26abde Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E8=B0=AD=E5=B7=8D?= Date: Thu, 16 Nov 2023 02:33:18 +0800 Subject: [PATCH 264/572] Add Library Guide for User Defined Functions: Window/Aggregate (#8171) * udwf doc Signed-off-by: veeupup * Add Library Guide for User Defined Functions: Window/Aggregate Signed-off-by: veeupup * make docs prettier Signed-off-by: veeupup --------- Signed-off-by: veeupup --- datafusion-examples/examples/simple_udaf.rs | 4 + datafusion-examples/examples/simple_udwf.rs | 4 +- docs/source/library-user-guide/adding-udfs.md | 316 +++++++++++++++++- 3 files changed, 319 insertions(+), 5 deletions(-) diff --git a/datafusion-examples/examples/simple_udaf.rs b/datafusion-examples/examples/simple_udaf.rs index 7aec9698d92f..2c797f221b2c 100644 --- a/datafusion-examples/examples/simple_udaf.rs +++ b/datafusion-examples/examples/simple_udaf.rs @@ -154,6 +154,10 @@ async fn main() -> Result<()> { // This is the description of the state. `state()` must match the types here. Arc::new(vec![DataType::Float64, DataType::UInt32]), ); + ctx.register_udaf(geometric_mean.clone()); + + let sql_df = ctx.sql("SELECT geo_mean(a) FROM t").await?; + sql_df.show().await?; // get a DataFrame from the context // this table has 1 column `a` f32 with values {2,4,8,64}, whose geometric mean is 8.0. diff --git a/datafusion-examples/examples/simple_udwf.rs b/datafusion-examples/examples/simple_udwf.rs index d1cbcc7c4389..0d04c093e147 100644 --- a/datafusion-examples/examples/simple_udwf.rs +++ b/datafusion-examples/examples/simple_udwf.rs @@ -89,7 +89,7 @@ async fn main() -> Result<()> { "SELECT \ car, \ speed, \ - smooth_it(speed) OVER (PARTITION BY car ORDER BY time),\ + smooth_it(speed) OVER (PARTITION BY car ORDER BY time) AS smooth_speed,\ time \ from cars \ ORDER BY \ @@ -109,7 +109,7 @@ async fn main() -> Result<()> { "SELECT \ car, \ speed, \ - smooth_it(speed) OVER (PARTITION BY car ORDER BY time ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING),\ + smooth_it(speed) OVER (PARTITION BY car ORDER BY time ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING) AS smooth_speed,\ time \ from cars \ ORDER BY \ diff --git a/docs/source/library-user-guide/adding-udfs.md b/docs/source/library-user-guide/adding-udfs.md index a4b5ed0b40f1..1e710bc321a2 100644 --- a/docs/source/library-user-guide/adding-udfs.md +++ b/docs/source/library-user-guide/adding-udfs.md @@ -38,7 +38,7 @@ A Scalar UDF is a function that takes a row of data and returns a single value. ```rust use std::sync::Arc; -use arrow::array::{ArrayRef, Int64Array}; +use datafusion::arrow::array::{ArrayRef, Int64Array}; use datafusion::common::Result; use datafusion::common::cast::as_int64_array; @@ -78,6 +78,11 @@ The challenge however is that DataFusion doesn't know about this function. We ne To register a Scalar UDF, you need to wrap the function implementation in a `ScalarUDF` struct and then register it with the `SessionContext`. DataFusion provides the `create_udf` and `make_scalar_function` helper functions to make this easier. ```rust +use datafusion::logical_expr::{Volatility, create_udf}; +use datafusion::physical_plan::functions::make_scalar_function; +use datafusion::arrow::datatypes::DataType; +use std::sync::Arc; + let udf = create_udf( "add_one", vec![DataType::Int64], @@ -98,6 +103,8 @@ A few things to note: That gives us a `ScalarUDF` that we can register with the `SessionContext`: ```rust +use datafusion::execution::context::SessionContext; + let mut ctx = SessionContext::new(); ctx.register_udf(udf); @@ -115,10 +122,313 @@ let df = ctx.sql(&sql).await.unwrap(); Scalar UDFs are functions that take a row of data and return a single value. Window UDFs are similar, but they also have access to the rows around them. Access to the the proximal rows is helpful, but adds some complexity to the implementation. -Body coming soon. +For example, we will declare a user defined window function that computes a moving average. + +```rust +use datafusion::arrow::{array::{ArrayRef, Float64Array, AsArray}, datatypes::Float64Type}; +use datafusion::logical_expr::{PartitionEvaluator}; +use datafusion::common::ScalarValue; +use datafusion::error::Result; +/// This implements the lowest level evaluation for a window function +/// +/// It handles calculating the value of the window function for each +/// distinct values of `PARTITION BY` +#[derive(Clone, Debug)] +struct MyPartitionEvaluator {} + +impl MyPartitionEvaluator { + fn new() -> Self { + Self {} + } +} + +/// Different evaluation methods are called depending on the various +/// settings of WindowUDF. This example uses the simplest and most +/// general, `evaluate`. See `PartitionEvaluator` for the other more +/// advanced uses. +impl PartitionEvaluator for MyPartitionEvaluator { + /// Tell DataFusion the window function varies based on the value + /// of the window frame. + fn uses_window_frame(&self) -> bool { + true + } + + /// This function is called once per input row. + /// + /// `range`specifies which indexes of `values` should be + /// considered for the calculation. + /// + /// Note this is the SLOWEST, but simplest, way to evaluate a + /// window function. It is much faster to implement + /// evaluate_all or evaluate_all_with_rank, if possible + fn evaluate( + &mut self, + values: &[ArrayRef], + range: &std::ops::Range, + ) -> Result { + // Again, the input argument is an array of floating + // point numbers to calculate a moving average + let arr: &Float64Array = values[0].as_ref().as_primitive::(); + + let range_len = range.end - range.start; + + // our smoothing function will average all the values in the + let output = if range_len > 0 { + let sum: f64 = arr.values().iter().skip(range.start).take(range_len).sum(); + Some(sum / range_len as f64) + } else { + None + }; + + Ok(ScalarValue::Float64(output)) + } +} + +/// Create a `PartitionEvalutor` to evaluate this function on a new +/// partition. +fn make_partition_evaluator() -> Result> { + Ok(Box::new(MyPartitionEvaluator::new())) +} +``` + +### Registering a Window UDF + +To register a Window UDF, you need to wrap the function implementation in a `WindowUDF` struct and then register it with the `SessionContext`. DataFusion provides the `create_udwf` helper functions to make this easier. + +```rust +use datafusion::logical_expr::{Volatility, create_udwf}; +use datafusion::arrow::datatypes::DataType; +use std::sync::Arc; + +// here is where we define the UDWF. We also declare its signature: +let smooth_it = create_udwf( + "smooth_it", + DataType::Float64, + Arc::new(DataType::Float64), + Volatility::Immutable, + Arc::new(make_partition_evaluator), +); +``` + +The `create_udwf` has five arguments to check: + +- The first argument is the name of the function. This is the name that will be used in SQL queries. +- **The second argument** is the `DataType` of input array (attention: this is not a list of arrays). I.e. in this case, the function accepts `Float64` as argument. +- The third argument is the return type of the function. I.e. in this case, the function returns an `Float64`. +- The fourth argument is the volatility of the function. In short, this is used to determine if the function's performance can be optimized in some situations. In this case, the function is `Immutable` because it always returns the same value for the same input. A random number generator would be `Volatile` because it returns a different value for the same input. +- **The fifth argument** is the function implementation. This is the function that we defined above. + +That gives us a `WindowUDF` that we can register with the `SessionContext`: + +```rust +use datafusion::execution::context::SessionContext; + +let ctx = SessionContext::new(); + +ctx.register_udwf(smooth_it); +``` + +At this point, you can use the `smooth_it` function in your query: + +For example, if we have a [`cars.csv`](https://github.com/apache/arrow-datafusion/blob/main/datafusion/core/tests/data/cars.csv) whose contents like + +```csv +car,speed,time +red,20.0,1996-04-12T12:05:03.000000000 +red,20.3,1996-04-12T12:05:04.000000000 +green,10.0,1996-04-12T12:05:03.000000000 +green,10.3,1996-04-12T12:05:04.000000000 +... +``` + +Then, we can query like below: + +```rust +use datafusion::datasource::file_format::options::CsvReadOptions; +// register csv table first +let csv_path = "cars.csv".to_string(); +ctx.register_csv("cars", &csv_path, CsvReadOptions::default().has_header(true)).await?; +// do query with smooth_it +let df = ctx + .sql( + "SELECT \ + car, \ + speed, \ + smooth_it(speed) OVER (PARTITION BY car ORDER BY time) as smooth_speed,\ + time \ + from cars \ + ORDER BY \ + car", + ) + .await?; +// print the results +df.show().await?; +``` + +the output will be like: + +```csv ++-------+-------+--------------------+---------------------+ +| car | speed | smooth_speed | time | ++-------+-------+--------------------+---------------------+ +| green | 10.0 | 10.0 | 1996-04-12T12:05:03 | +| green | 10.3 | 10.15 | 1996-04-12T12:05:04 | +| green | 10.4 | 10.233333333333334 | 1996-04-12T12:05:05 | +| green | 10.5 | 10.3 | 1996-04-12T12:05:06 | +| green | 11.0 | 10.440000000000001 | 1996-04-12T12:05:07 | +| green | 12.0 | 10.700000000000001 | 1996-04-12T12:05:08 | +| green | 14.0 | 11.171428571428573 | 1996-04-12T12:05:09 | +| green | 15.0 | 11.65 | 1996-04-12T12:05:10 | +| green | 15.1 | 12.033333333333333 | 1996-04-12T12:05:11 | +| green | 15.2 | 12.35 | 1996-04-12T12:05:12 | +| green | 8.0 | 11.954545454545455 | 1996-04-12T12:05:13 | +| green | 2.0 | 11.125 | 1996-04-12T12:05:14 | +| red | 20.0 | 20.0 | 1996-04-12T12:05:03 | +| red | 20.3 | 20.15 | 1996-04-12T12:05:04 | +... +``` ## Adding an Aggregate UDF Aggregate UDFs are functions that take a group of rows and return a single value. These are akin to SQL's `SUM` or `COUNT` functions. -Body coming soon. +For example, we will declare a single-type, single return type UDAF that computes the geometric mean. + +```rust +use datafusion::arrow::array::ArrayRef; +use datafusion::scalar::ScalarValue; +use datafusion::{error::Result, physical_plan::Accumulator}; + +/// A UDAF has state across multiple rows, and thus we require a `struct` with that state. +#[derive(Debug)] +struct GeometricMean { + n: u32, + prod: f64, +} + +impl GeometricMean { + // how the struct is initialized + pub fn new() -> Self { + GeometricMean { n: 0, prod: 1.0 } + } +} + +// UDAFs are built using the trait `Accumulator`, that offers DataFusion the necessary functions +// to use them. +impl Accumulator for GeometricMean { + // This function serializes our state to `ScalarValue`, which DataFusion uses + // to pass this state between execution stages. + // Note that this can be arbitrary data. + fn state(&self) -> Result> { + Ok(vec![ + ScalarValue::from(self.prod), + ScalarValue::from(self.n), + ]) + } + + // DataFusion expects this function to return the final value of this aggregator. + // in this case, this is the formula of the geometric mean + fn evaluate(&self) -> Result { + let value = self.prod.powf(1.0 / self.n as f64); + Ok(ScalarValue::from(value)) + } + + // DataFusion calls this function to update the accumulator's state for a batch + // of inputs rows. In this case the product is updated with values from the first column + // and the count is updated based on the row count + fn update_batch(&mut self, values: &[ArrayRef]) -> Result<()> { + if values.is_empty() { + return Ok(()); + } + let arr = &values[0]; + (0..arr.len()).try_for_each(|index| { + let v = ScalarValue::try_from_array(arr, index)?; + + if let ScalarValue::Float64(Some(value)) = v { + self.prod *= value; + self.n += 1; + } else { + unreachable!("") + } + Ok(()) + }) + } + + // Optimization hint: this trait also supports `update_batch` and `merge_batch`, + // that can be used to perform these operations on arrays instead of single values. + fn merge_batch(&mut self, states: &[ArrayRef]) -> Result<()> { + if states.is_empty() { + return Ok(()); + } + let arr = &states[0]; + (0..arr.len()).try_for_each(|index| { + let v = states + .iter() + .map(|array| ScalarValue::try_from_array(array, index)) + .collect::>>()?; + if let (ScalarValue::Float64(Some(prod)), ScalarValue::UInt32(Some(n))) = (&v[0], &v[1]) + { + self.prod *= prod; + self.n += n; + } else { + unreachable!("") + } + Ok(()) + }) + } + + fn size(&self) -> usize { + std::mem::size_of_val(self) + } +} +``` + +### registering an Aggregate UDF + +To register a Aggreate UDF, you need to wrap the function implementation in a `AggregateUDF` struct and then register it with the `SessionContext`. DataFusion provides the `create_udaf` helper functions to make this easier. + +```rust +use datafusion::logical_expr::{Volatility, create_udaf}; +use datafusion::arrow::datatypes::DataType; +use std::sync::Arc; + +// here is where we define the UDAF. We also declare its signature: +let geometric_mean = create_udaf( + // the name; used to represent it in plan descriptions and in the registry, to use in SQL. + "geo_mean", + // the input type; DataFusion guarantees that the first entry of `values` in `update` has this type. + vec![DataType::Float64], + // the return type; DataFusion expects this to match the type returned by `evaluate`. + Arc::new(DataType::Float64), + Volatility::Immutable, + // This is the accumulator factory; DataFusion uses it to create new accumulators. + Arc::new(|_| Ok(Box::new(GeometricMean::new()))), + // This is the description of the state. `state()` must match the types here. + Arc::new(vec![DataType::Float64, DataType::UInt32]), +); +``` + +The `create_udaf` has six arguments to check: + +- The first argument is the name of the function. This is the name that will be used in SQL queries. +- The second argument is a vector of `DataType`s. This is the list of argument types that the function accepts. I.e. in this case, the function accepts a single `Float64` argument. +- The third argument is the return type of the function. I.e. in this case, the function returns an `Int64`. +- The fourth argument is the volatility of the function. In short, this is used to determine if the function's performance can be optimized in some situations. In this case, the function is `Immutable` because it always returns the same value for the same input. A random number generator would be `Volatile` because it returns a different value for the same input. +- The fifth argument is the function implementation. This is the function that we defined above. +- The sixth argument is the description of the state, which will by passed between execution stages. + +That gives us a `AggregateUDF` that we can register with the `SessionContext`: + +```rust +use datafusion::execution::context::SessionContext; + +let ctx = SessionContext::new(); + +ctx.register_udaf(geometric_mean); +``` + +Then, we can query like below: + +```rust +let df = ctx.sql("SELECT geo_mean(a) FROM t").await?; +``` From a1c96634bd182e6cd90115544c7bdfeb30d752fb Mon Sep 17 00:00:00 2001 From: Raphael Taylor-Davies <1781103+tustvold@users.noreply.github.com> Date: Wed, 15 Nov 2023 18:43:06 +0000 Subject: [PATCH 265/572] Add more stream docs (#8192) --- datafusion/core/src/datasource/stream.rs | 19 +++++++++++++++++-- 1 file changed, 17 insertions(+), 2 deletions(-) diff --git a/datafusion/core/src/datasource/stream.rs b/datafusion/core/src/datasource/stream.rs index cf95dd249a7f..fc19ff954d8e 100644 --- a/datafusion/core/src/datasource/stream.rs +++ b/datafusion/core/src/datasource/stream.rs @@ -104,6 +104,12 @@ pub struct StreamConfig { impl StreamConfig { /// Stream data from the file at `location` + /// + /// * Data will be read sequentially from the provided `location` + /// * New data will be appended to the end of the file + /// + /// The encoding can be configured with [`Self::with_encoding`] and + /// defaults to [`StreamEncoding::Csv`] pub fn new_file(schema: SchemaRef, location: PathBuf) -> Self { Self { schema, @@ -180,11 +186,20 @@ impl StreamConfig { } } -/// A [`TableProvider`] for a stream source, such as a FIFO file +/// A [`TableProvider`] for an unbounded stream source +/// +/// Currently only reading from / appending to a single file in-place is supported, but +/// other stream sources and sinks may be added in future. +/// +/// Applications looking to read/write datasets comprising multiple files, e.g. [Hadoop]-style +/// data stored in object storage, should instead consider [`ListingTable`]. +/// +/// [Hadoop]: https://hadoop.apache.org/ +/// [`ListingTable`]: crate::datasource::listing::ListingTable pub struct StreamTable(Arc); impl StreamTable { - /// Create a new [`StreamTable`] for the given `StreamConfig` + /// Create a new [`StreamTable`] for the given [`StreamConfig`] pub fn new(config: Arc) -> Self { Self(config) } From 6b945a4409e1c8e9c50124e30a0996b65e9d31c6 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E8=B0=AD=E5=B7=8D?= Date: Thu, 16 Nov 2023 05:17:05 +0800 Subject: [PATCH 266/572] Implement func `array_pop_front` (#8142) * implement array_pop_front Signed-off-by: veeupup * abstract array_pop * fix cargo check * add docs for array_pop_front Signed-off-by: veeupup * fix comments --------- Signed-off-by: veeupup --- datafusion/expr/src/built_in_function.rs | 6 +++ datafusion/expr/src/expr_fn.rs | 8 ++++ .../physical-expr/src/array_expressions.rs | 42 ++++++++++++++++--- datafusion/physical-expr/src/functions.rs | 3 ++ datafusion/proto/proto/datafusion.proto | 1 + datafusion/proto/src/generated/pbjson.rs | 3 ++ datafusion/proto/src/generated/prost.rs | 3 ++ .../proto/src/logical_plan/from_proto.rs | 6 ++- datafusion/proto/src/logical_plan/to_proto.rs | 1 + datafusion/sqllogictest/test_files/array.slt | 38 +++++++++++++++++ docs/source/user-guide/expressions.md | 1 + .../source/user-guide/sql/scalar_functions.md | 25 +++++++++++ 12 files changed, 131 insertions(+), 6 deletions(-) diff --git a/datafusion/expr/src/built_in_function.rs b/datafusion/expr/src/built_in_function.rs index 473094c00174..1b48c37406d3 100644 --- a/datafusion/expr/src/built_in_function.rs +++ b/datafusion/expr/src/built_in_function.rs @@ -138,6 +138,8 @@ pub enum BuiltinScalarFunction { ArrayHasAll, /// array_has_any ArrayHasAny, + /// array_pop_front + ArrayPopFront, /// array_pop_back ArrayPopBack, /// array_dims @@ -392,6 +394,7 @@ impl BuiltinScalarFunction { BuiltinScalarFunction::ArrayElement => Volatility::Immutable, BuiltinScalarFunction::ArrayLength => Volatility::Immutable, BuiltinScalarFunction::ArrayNdims => Volatility::Immutable, + BuiltinScalarFunction::ArrayPopFront => Volatility::Immutable, BuiltinScalarFunction::ArrayPopBack => Volatility::Immutable, BuiltinScalarFunction::ArrayPosition => Volatility::Immutable, BuiltinScalarFunction::ArrayPositions => Volatility::Immutable, @@ -570,6 +573,7 @@ impl BuiltinScalarFunction { }, BuiltinScalarFunction::ArrayLength => Ok(UInt64), BuiltinScalarFunction::ArrayNdims => Ok(UInt64), + BuiltinScalarFunction::ArrayPopFront => Ok(input_expr_types[0].clone()), BuiltinScalarFunction::ArrayPopBack => Ok(input_expr_types[0].clone()), BuiltinScalarFunction::ArrayPosition => Ok(UInt64), BuiltinScalarFunction::ArrayPositions => { @@ -868,6 +872,7 @@ impl BuiltinScalarFunction { // for now, the list is small, as we do not have many built-in functions. match self { BuiltinScalarFunction::ArrayAppend => Signature::any(2, self.volatility()), + BuiltinScalarFunction::ArrayPopFront => Signature::any(1, self.volatility()), BuiltinScalarFunction::ArrayPopBack => Signature::any(1, self.volatility()), BuiltinScalarFunction::ArrayConcat => { Signature::variadic_any(self.volatility()) @@ -1512,6 +1517,7 @@ fn aliases(func: &BuiltinScalarFunction) -> &'static [&'static str] { } BuiltinScalarFunction::ArrayLength => &["array_length", "list_length"], BuiltinScalarFunction::ArrayNdims => &["array_ndims", "list_ndims"], + BuiltinScalarFunction::ArrayPopFront => &["array_pop_front", "list_pop_front"], BuiltinScalarFunction::ArrayPopBack => &["array_pop_back", "list_pop_back"], BuiltinScalarFunction::ArrayPosition => &[ "array_position", diff --git a/datafusion/expr/src/expr_fn.rs b/datafusion/expr/src/expr_fn.rs index e70a4a90f767..bcf1aa0ca7e5 100644 --- a/datafusion/expr/src/expr_fn.rs +++ b/datafusion/expr/src/expr_fn.rs @@ -590,6 +590,13 @@ scalar_expr!( "returns the array without the last element." ); +scalar_expr!( + ArrayPopFront, + array_pop_front, + array, + "returns the array without the first element." +); + nary_scalar_expr!(ArrayConcat, array_concat, "concatenates arrays."); scalar_expr!( ArrayHas, @@ -1166,6 +1173,7 @@ mod test { test_scalar_expr!(FromUnixtime, from_unixtime, unixtime); test_scalar_expr!(ArrayAppend, array_append, array, element); + test_scalar_expr!(ArrayPopFront, array_pop_front, array); test_scalar_expr!(ArrayPopBack, array_pop_back, array); test_unary_scalar_expr!(ArrayDims, array_dims); test_scalar_expr!(ArrayLength, array_length, array, dimension); diff --git a/datafusion/physical-expr/src/array_expressions.rs b/datafusion/physical-expr/src/array_expressions.rs index 515df2a970a4..ded606c3b705 100644 --- a/datafusion/physical-expr/src/array_expressions.rs +++ b/datafusion/physical-expr/src/array_expressions.rs @@ -636,13 +636,33 @@ pub fn array_slice(args: &[ArrayRef]) -> Result { define_array_slice(list_array, key, extra_key, false) } +fn general_array_pop( + list_array: &GenericListArray, + from_back: bool, +) -> Result<(Vec, Vec)> { + if from_back { + let key = vec![0; list_array.len()]; + // Atttetion: `arr.len() - 1` in extra key defines the last element position (position = index + 1, not inclusive) we want in the new array. + let extra_key: Vec<_> = list_array + .iter() + .map(|x| x.map_or(0, |arr| arr.len() as i64 - 1)) + .collect(); + Ok((key, extra_key)) + } else { + // Atttetion: 2 in the `key`` defines the first element position (position = index + 1) we want in the new array. + // We only handle two cases of the first element index: if the old array has any elements, starts from 2 (index + 1), or starts from initial. + let key: Vec<_> = list_array.iter().map(|x| x.map_or(0, |_| 2)).collect(); + let extra_key: Vec<_> = list_array + .iter() + .map(|x| x.map_or(0, |arr| arr.len() as i64)) + .collect(); + Ok((key, extra_key)) + } +} + pub fn array_pop_back(args: &[ArrayRef]) -> Result { let list_array = as_list_array(&args[0])?; - let key = vec![0; list_array.len()]; - let extra_key: Vec<_> = list_array - .iter() - .map(|x| x.map_or(0, |arr| arr.len() as i64 - 1)) - .collect(); + let (key, extra_key) = general_array_pop(list_array, true)?; define_array_slice( list_array, @@ -767,6 +787,18 @@ pub fn gen_range(args: &[ArrayRef]) -> Result { Ok(arr) } +pub fn array_pop_front(args: &[ArrayRef]) -> Result { + let list_array = as_list_array(&args[0])?; + let (key, extra_key) = general_array_pop(list_array, false)?; + + define_array_slice( + list_array, + &Int64Array::from(key), + &Int64Array::from(extra_key), + false, + ) +} + /// Array_append SQL function pub fn array_append(args: &[ArrayRef]) -> Result { let list_array = as_list_array(&args[0])?; diff --git a/datafusion/physical-expr/src/functions.rs b/datafusion/physical-expr/src/functions.rs index 543d7eb654e2..1e8500079f21 100644 --- a/datafusion/physical-expr/src/functions.rs +++ b/datafusion/physical-expr/src/functions.rs @@ -359,6 +359,9 @@ pub fn create_physical_fun( BuiltinScalarFunction::ArrayNdims => { Arc::new(|args| make_scalar_function(array_expressions::array_ndims)(args)) } + BuiltinScalarFunction::ArrayPopFront => Arc::new(|args| { + make_scalar_function(array_expressions::array_pop_front)(args) + }), BuiltinScalarFunction::ArrayPopBack => { Arc::new(|args| make_scalar_function(array_expressions::array_pop_back)(args)) } diff --git a/datafusion/proto/proto/datafusion.proto b/datafusion/proto/proto/datafusion.proto index fa080518d50c..66c34c7a12ec 100644 --- a/datafusion/proto/proto/datafusion.proto +++ b/datafusion/proto/proto/datafusion.proto @@ -638,6 +638,7 @@ enum ScalarFunction { ArrayUnion = 120; OverLay = 121; Range = 122; + ArrayPopFront = 123; } message ScalarFunctionNode { diff --git a/datafusion/proto/src/generated/pbjson.rs b/datafusion/proto/src/generated/pbjson.rs index 08e7413102e8..628adcc41189 100644 --- a/datafusion/proto/src/generated/pbjson.rs +++ b/datafusion/proto/src/generated/pbjson.rs @@ -20937,6 +20937,7 @@ impl serde::Serialize for ScalarFunction { Self::ArrayUnion => "ArrayUnion", Self::OverLay => "OverLay", Self::Range => "Range", + Self::ArrayPopFront => "ArrayPopFront", }; serializer.serialize_str(variant) } @@ -21071,6 +21072,7 @@ impl<'de> serde::Deserialize<'de> for ScalarFunction { "ArrayUnion", "OverLay", "Range", + "ArrayPopFront", ]; struct GeneratedVisitor; @@ -21234,6 +21236,7 @@ impl<'de> serde::Deserialize<'de> for ScalarFunction { "ArrayUnion" => Ok(ScalarFunction::ArrayUnion), "OverLay" => Ok(ScalarFunction::OverLay), "Range" => Ok(ScalarFunction::Range), + "ArrayPopFront" => Ok(ScalarFunction::ArrayPopFront), _ => Err(serde::de::Error::unknown_variant(value, FIELDS)), } } diff --git a/datafusion/proto/src/generated/prost.rs b/datafusion/proto/src/generated/prost.rs index 15606488b33a..317b888447a0 100644 --- a/datafusion/proto/src/generated/prost.rs +++ b/datafusion/proto/src/generated/prost.rs @@ -2569,6 +2569,7 @@ pub enum ScalarFunction { ArrayUnion = 120, OverLay = 121, Range = 122, + ArrayPopFront = 123, } impl ScalarFunction { /// String value of the enum field names used in the ProtoBuf definition. @@ -2700,6 +2701,7 @@ impl ScalarFunction { ScalarFunction::ArrayUnion => "ArrayUnion", ScalarFunction::OverLay => "OverLay", ScalarFunction::Range => "Range", + ScalarFunction::ArrayPopFront => "ArrayPopFront", } } /// Creates an enum from field names used in the ProtoBuf definition. @@ -2828,6 +2830,7 @@ impl ScalarFunction { "ArrayUnion" => Some(Self::ArrayUnion), "OverLay" => Some(Self::OverLay), "Range" => Some(Self::Range), + "ArrayPopFront" => Some(Self::ArrayPopFront), _ => None, } } diff --git a/datafusion/proto/src/logical_plan/from_proto.rs b/datafusion/proto/src/logical_plan/from_proto.rs index b3d68570038c..94c9f9806621 100644 --- a/datafusion/proto/src/logical_plan/from_proto.rs +++ b/datafusion/proto/src/logical_plan/from_proto.rs @@ -66,7 +66,7 @@ use datafusion_expr::{ WindowFrameUnits, }; use datafusion_expr::{ - array_empty, array_pop_back, + array_empty, array_pop_back, array_pop_front, expr::{Alias, Placeholder}, }; use std::sync::Arc; @@ -473,6 +473,7 @@ impl From<&protobuf::ScalarFunction> for BuiltinScalarFunction { ScalarFunction::Flatten => Self::Flatten, ScalarFunction::ArrayLength => Self::ArrayLength, ScalarFunction::ArrayNdims => Self::ArrayNdims, + ScalarFunction::ArrayPopFront => Self::ArrayPopFront, ScalarFunction::ArrayPopBack => Self::ArrayPopBack, ScalarFunction::ArrayPosition => Self::ArrayPosition, ScalarFunction::ArrayPositions => Self::ArrayPositions, @@ -1330,6 +1331,9 @@ pub fn parse_expr( parse_expr(&args[0], registry)?, parse_expr(&args[1], registry)?, )), + ScalarFunction::ArrayPopFront => { + Ok(array_pop_front(parse_expr(&args[0], registry)?)) + } ScalarFunction::ArrayPopBack => { Ok(array_pop_back(parse_expr(&args[0], registry)?)) } diff --git a/datafusion/proto/src/logical_plan/to_proto.rs b/datafusion/proto/src/logical_plan/to_proto.rs index 144f28531041..53be5f7bd498 100644 --- a/datafusion/proto/src/logical_plan/to_proto.rs +++ b/datafusion/proto/src/logical_plan/to_proto.rs @@ -1480,6 +1480,7 @@ impl TryFrom<&BuiltinScalarFunction> for protobuf::ScalarFunction { BuiltinScalarFunction::Flatten => Self::Flatten, BuiltinScalarFunction::ArrayLength => Self::ArrayLength, BuiltinScalarFunction::ArrayNdims => Self::ArrayNdims, + BuiltinScalarFunction::ArrayPopFront => Self::ArrayPopFront, BuiltinScalarFunction::ArrayPopBack => Self::ArrayPopBack, BuiltinScalarFunction::ArrayPosition => Self::ArrayPosition, BuiltinScalarFunction::ArrayPositions => Self::ArrayPositions, diff --git a/datafusion/sqllogictest/test_files/array.slt b/datafusion/sqllogictest/test_files/array.slt index 67cabb0988fd..99ed94883629 100644 --- a/datafusion/sqllogictest/test_files/array.slt +++ b/datafusion/sqllogictest/test_files/array.slt @@ -826,6 +826,44 @@ select array_pop_back(column1) from arrayspop; [] [, 10, 11] +## array_pop_front (aliases: `list_pop_front`) + +# array_pop_front scalar function #1 +query ?? +select array_pop_front(make_array(1, 2, 3, 4, 5)), array_pop_front(make_array('h', 'e', 'l', 'l', 'o')); +---- +[2, 3, 4, 5] [e, l, l, o] + +# array_pop_front scalar function #2 (after array_pop_front, array is empty) +query ? +select array_pop_front(make_array(1)); +---- +[] + +# array_pop_front scalar function #3 (array_pop_front the empty array) +query ? +select array_pop_front(array_pop_front(make_array(1))); +---- +[] + +# array_pop_front scalar function #5 (array_pop_front the nested arrays) +query ? +select array_pop_front(make_array(make_array(1, 2, 3), make_array(2, 9, 1), make_array(7, 8, 9), make_array(1, 2, 3), make_array(1, 7, 4), make_array(4, 5, 6))); +---- +[[2, 9, 1], [7, 8, 9], [1, 2, 3], [1, 7, 4], [4, 5, 6]] + +# array_pop_front scalar function #6 (array_pop_front the nested arrays with NULL) +query ? +select array_pop_front(make_array(NULL, make_array(1, 2, 3), make_array(2, 9, 1), make_array(7, 8, 9), make_array(1, 2, 3), make_array(1, 7, 4))); +---- +[[1, 2, 3], [2, 9, 1], [7, 8, 9], [1, 2, 3], [1, 7, 4]] + +# array_pop_front scalar function #8 (after array_pop_front, nested array is empty) +query ? +select array_pop_front(make_array(make_array(1, 2, 3))); +---- +[] + ## array_slice (aliases: list_slice) # array_slice scalar function #1 (with positive indexes) diff --git a/docs/source/user-guide/expressions.md b/docs/source/user-guide/expressions.md index 6b2ab46eb343..191ef6cd9116 100644 --- a/docs/source/user-guide/expressions.md +++ b/docs/source/user-guide/expressions.md @@ -219,6 +219,7 @@ Unlike to some databases the math functions in Datafusion works the same way as | flatten(array) | Converts an array of arrays to a flat array `flatten([[1], [2, 3], [4, 5, 6]]) -> [1, 2, 3, 4, 5, 6]` | | array_length(array, dimension) | Returns the length of the array dimension. `array_length([1, 2, 3, 4, 5]) -> 5` | | array_ndims(array) | Returns the number of dimensions of the array. `array_ndims([[1, 2, 3], [4, 5, 6]]) -> 2` | +| array_pop_front(array) | Returns the array without the first element. `array_pop_front([1, 2, 3]) -> [2, 3]` | | array_pop_back(array) | Returns the array without the last element. `array_pop_back([1, 2, 3]) -> [1, 2]` | | array_position(array, element) | Searches for an element in the array, returns first occurrence. `array_position([1, 2, 2, 3, 4], 2) -> 2` | | array_positions(array, element) | Searches for an element in the array, returns all occurrences. `array_positions([1, 2, 2, 3, 4], 2) -> [2, 3]` | diff --git a/docs/source/user-guide/sql/scalar_functions.md b/docs/source/user-guide/sql/scalar_functions.md index 826782e1a051..baaea3926f7d 100644 --- a/docs/source/user-guide/sql/scalar_functions.md +++ b/docs/source/user-guide/sql/scalar_functions.md @@ -1515,6 +1515,7 @@ from_unixtime(expression) - [array_length](#array_length) - [array_ndims](#array_ndims) - [array_prepend](#array_prepend) +- [array_pop_front](#array_pop_front) - [array_pop_back](#array_pop_back) - [array_position](#array_position) - [array_positions](#array_positions) @@ -1868,6 +1869,30 @@ array_prepend(element, array) - list_prepend - list_push_front +### `array_pop_front` + +Returns the array without the first element. + +``` +array_pop_first(array) +``` + +#### Arguments + +- **array**: Array expression. + Can be a constant, column, or function, and any combination of array operators. + +#### Example + +``` +❯ select array_pop_first([1, 2, 3]); ++-------------------------------+ +| array_pop_first(List([1,2,3])) | ++-------------------------------+ +| [2, 3] | ++-------------------------------+ +``` + ### `array_pop_back` Returns the array without the last element. From 937bb44fd05307e368dddd8533f25b02709b20c0 Mon Sep 17 00:00:00 2001 From: Edmondo Porcu Date: Wed, 15 Nov 2023 13:28:49 -0800 Subject: [PATCH 267/572] Moving arrow_files SQL tests to sqllogictest (#8217) * Moving arrow_files SQL tests to sqllogictest * Removed module --- datafusion/core/tests/sql/arrow_files.rs | 70 ------------------- datafusion/core/tests/sql/mod.rs | 1 - .../sqllogictest/test_files/arrow_files.slt | 44 ++++++++++++ 3 files changed, 44 insertions(+), 71 deletions(-) delete mode 100644 datafusion/core/tests/sql/arrow_files.rs create mode 100644 datafusion/sqllogictest/test_files/arrow_files.slt diff --git a/datafusion/core/tests/sql/arrow_files.rs b/datafusion/core/tests/sql/arrow_files.rs deleted file mode 100644 index fc90fe3c3464..000000000000 --- a/datafusion/core/tests/sql/arrow_files.rs +++ /dev/null @@ -1,70 +0,0 @@ -// Licensed to the Apache Software Foundation (ASF) under one -// or more contributor license agreements. See the NOTICE file -// distributed with this work for additional information -// regarding copyright ownership. The ASF licenses this file -// to you under the Apache License, Version 2.0 (the -// "License"); you may not use this file except in compliance -// with the License. You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, -// software distributed under the License is distributed on an -// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -// KIND, either express or implied. See the License for the -// specific language governing permissions and limitations -// under the License. -use datafusion::execution::options::ArrowReadOptions; - -use super::*; - -async fn register_arrow(ctx: &mut SessionContext) { - ctx.register_arrow( - "arrow_simple", - "tests/data/example.arrow", - ArrowReadOptions::default(), - ) - .await - .unwrap(); -} - -#[tokio::test] -async fn arrow_query() { - let mut ctx = SessionContext::new(); - register_arrow(&mut ctx).await; - let sql = "SELECT * FROM arrow_simple"; - let actual = execute_to_batches(&ctx, sql).await; - let expected = [ - "+----+-----+-------+", - "| f0 | f1 | f2 |", - "+----+-----+-------+", - "| 1 | foo | true |", - "| 2 | bar | |", - "| 3 | baz | false |", - "| 4 | | true |", - "+----+-----+-------+", - ]; - - assert_batches_eq!(expected, &actual); -} - -#[tokio::test] -async fn arrow_explain() { - let mut ctx = SessionContext::new(); - register_arrow(&mut ctx).await; - let sql = "EXPLAIN SELECT * FROM arrow_simple"; - let actual = execute(&ctx, sql).await; - let actual = normalize_vec_for_explain(actual); - let expected = vec![ - vec![ - "logical_plan", - "TableScan: arrow_simple projection=[f0, f1, f2]", - ], - vec![ - "physical_plan", - "ArrowExec: file_groups={1 group: [[WORKING_DIR/tests/data/example.arrow]]}, projection=[f0, f1, f2]\n", - ], - ]; - - assert_eq!(expected, actual); -} diff --git a/datafusion/core/tests/sql/mod.rs b/datafusion/core/tests/sql/mod.rs index d44513e69a9f..4bd42c4688df 100644 --- a/datafusion/core/tests/sql/mod.rs +++ b/datafusion/core/tests/sql/mod.rs @@ -73,7 +73,6 @@ macro_rules! test_expression { } pub mod aggregates; -pub mod arrow_files; pub mod create_drop; pub mod csv_files; pub mod describe; diff --git a/datafusion/sqllogictest/test_files/arrow_files.slt b/datafusion/sqllogictest/test_files/arrow_files.slt new file mode 100644 index 000000000000..5c1b6fb726ed --- /dev/null +++ b/datafusion/sqllogictest/test_files/arrow_files.slt @@ -0,0 +1,44 @@ +# 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. + +############# +## Arrow Files Format support +############# + + +statement ok + +CREATE EXTERNAL TABLE arrow_simple +STORED AS ARROW +LOCATION '../core/tests/data/example.arrow'; + + +# physical plan +query TT +EXPLAIN SELECT * FROM arrow_simple +---- +logical_plan TableScan: arrow_simple projection=[f0, f1, f2] +physical_plan ArrowExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/example.arrow]]}, projection=[f0, f1, f2] + +# correct content +query ITB +SELECT * FROM arrow_simple +---- +1 foo true +2 bar NULL +3 baz false +4 NULL true From 04c77ca3ec9f56d3ded52e15aa4de3f08e261478 Mon Sep 17 00:00:00 2001 From: Andrew Lamb Date: Wed, 15 Nov 2023 17:38:56 -0500 Subject: [PATCH 268/572] fix use of name in Column (#8219) --- .../optimizer/src/push_down_projection.rs | 27 ++++++++++++++++++- 1 file changed, 26 insertions(+), 1 deletion(-) diff --git a/datafusion/optimizer/src/push_down_projection.rs b/datafusion/optimizer/src/push_down_projection.rs index 2c314bf7651c..59a5357c97dd 100644 --- a/datafusion/optimizer/src/push_down_projection.rs +++ b/datafusion/optimizer/src/push_down_projection.rs @@ -228,7 +228,7 @@ impl OptimizerRule for PushDownProjection { // Gather all columns needed for expressions in this Aggregate let mut new_aggr_expr = vec![]; for e in agg.aggr_expr.iter() { - let column = Column::from(e.display_name()?); + let column = Column::from_name(e.display_name()?); if required_columns.contains(&column) { new_aggr_expr.push(e.clone()); } @@ -605,6 +605,31 @@ mod tests { assert_optimized_plan_eq(&plan, expected) } + #[test] + fn aggregate_with_periods() -> Result<()> { + let schema = Schema::new(vec![Field::new("tag.one", DataType::Utf8, false)]); + + // Build a plan that looks as follows (note "tag.one" is a column named + // "tag.one", not a column named "one" in a table named "tag"): + // + // Projection: tag.one + // Aggregate: groupBy=[], aggr=[MAX("tag.one") AS "tag.one"] + // TableScan + let plan = table_scan(Some("m4"), &schema, None)? + .aggregate( + Vec::::new(), + vec![max(col(Column::new_unqualified("tag.one"))).alias("tag.one")], + )? + .project([col(Column::new_unqualified("tag.one"))])? + .build()?; + + let expected = "\ + Aggregate: groupBy=[[]], aggr=[[MAX(m4.tag.one) AS tag.one]]\ + \n TableScan: m4 projection=[tag.one]"; + + assert_optimized_plan_eq(&plan, expected) + } + #[test] fn redundant_project() -> Result<()> { let table_scan = test_table_scan()?; From 4c6f5c5310f82e7aed3c5634b4d6c58d8780d9e5 Mon Sep 17 00:00:00 2001 From: Mustafa Akur <106137913+mustafasrepo@users.noreply.github.com> Date: Thu, 16 Nov 2023 10:01:20 +0300 Subject: [PATCH 269/572] Fix column indices in the planning tests (#8191) --- .../enforce_distribution.rs | 10 +- .../replace_with_order_preserving_variants.rs | 185 ++++++++++-------- 2 files changed, 108 insertions(+), 87 deletions(-) diff --git a/datafusion/core/src/physical_optimizer/enforce_distribution.rs b/datafusion/core/src/physical_optimizer/enforce_distribution.rs index 12f27ab18fbd..4aedc3b0d1a9 100644 --- a/datafusion/core/src/physical_optimizer/enforce_distribution.rs +++ b/datafusion/core/src/physical_optimizer/enforce_distribution.rs @@ -3794,7 +3794,11 @@ pub(crate) mod tests { sort_key, projection_exec_with_alias( filter_exec(parquet_exec()), - vec![("a".to_string(), "a".to_string())], + vec![ + ("a".to_string(), "a".to_string()), + ("b".to_string(), "b".to_string()), + ("c".to_string(), "c".to_string()), + ], ), false, ); @@ -3803,7 +3807,7 @@ pub(crate) mod tests { "SortPreservingMergeExec: [c@2 ASC]", // Expect repartition on the input to the sort (as it can benefit from additional parallelism) "SortExec: expr=[c@2 ASC]", - "ProjectionExec: expr=[a@0 as a]", + "ProjectionExec: expr=[a@0 as a, b@1 as b, c@2 as c]", "FilterExec: c@2 = 0", // repartition is lowest down "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", @@ -3815,7 +3819,7 @@ pub(crate) mod tests { let expected_first_sort_enforcement = &[ "SortExec: expr=[c@2 ASC]", "CoalescePartitionsExec", - "ProjectionExec: expr=[a@0 as a]", + "ProjectionExec: expr=[a@0 as a, b@1 as b, c@2 as c]", "FilterExec: c@2 = 0", "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", diff --git a/datafusion/core/src/physical_optimizer/replace_with_order_preserving_variants.rs b/datafusion/core/src/physical_optimizer/replace_with_order_preserving_variants.rs index 58806be6d411..7f8c9b852cb1 100644 --- a/datafusion/core/src/physical_optimizer/replace_with_order_preserving_variants.rs +++ b/datafusion/core/src/physical_optimizer/replace_with_order_preserving_variants.rs @@ -359,11 +359,11 @@ mod tests { let expected_input = ["SortPreservingMergeExec: [a@0 ASC NULLS LAST]", " SortExec: expr=[a@0 ASC NULLS LAST]", - " RepartitionExec: partitioning=Hash([c1@0], 8), input_partitions=8", + " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8", " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", " CsvExec: file_groups={1 group: [[file_path]]}, projection=[a, c, d], infinite_source=true, output_ordering=[a@0 ASC NULLS LAST], has_header=true"]; let expected_optimized = ["SortPreservingMergeExec: [a@0 ASC NULLS LAST]", - " SortPreservingRepartitionExec: partitioning=Hash([c1@0], 8), input_partitions=8, sort_exprs=a@0 ASC NULLS LAST", + " SortPreservingRepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8, sort_exprs=a@0 ASC NULLS LAST", " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", " CsvExec: file_groups={1 group: [[file_path]]}, projection=[a, c, d], infinite_source=true, output_ordering=[a@0 ASC NULLS LAST], has_header=true"]; assert_optimized!(expected_input, expected_optimized, physical_plan); @@ -379,38 +379,41 @@ mod tests { let repartition_hash = repartition_exec_hash(repartition_rr); let coalesce_partitions = coalesce_partitions_exec(repartition_hash); let sort = sort_exec( - vec![sort_expr_default("a", &schema)], + vec![sort_expr_default("a", &coalesce_partitions.schema())], coalesce_partitions, false, ); let repartition_rr2 = repartition_exec_round_robin(sort); let repartition_hash2 = repartition_exec_hash(repartition_rr2); - let filter = filter_exec(repartition_hash2, &schema); - let sort2 = sort_exec(vec![sort_expr_default("a", &schema)], filter, true); + let filter = filter_exec(repartition_hash2); + let sort2 = + sort_exec(vec![sort_expr_default("a", &filter.schema())], filter, true); - let physical_plan = - sort_preserving_merge_exec(vec![sort_expr_default("a", &schema)], sort2); + let physical_plan = sort_preserving_merge_exec( + vec![sort_expr_default("a", &sort2.schema())], + sort2, + ); let expected_input = [ "SortPreservingMergeExec: [a@0 ASC]", " SortExec: expr=[a@0 ASC]", - " FilterExec: c@2 > 3", - " RepartitionExec: partitioning=Hash([c1@0], 8), input_partitions=8", + " FilterExec: c@1 > 3", + " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8", " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", " SortExec: expr=[a@0 ASC]", " CoalescePartitionsExec", - " RepartitionExec: partitioning=Hash([c1@0], 8), input_partitions=8", + " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8", " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", " CsvExec: file_groups={1 group: [[file_path]]}, projection=[a, c, d], infinite_source=true, output_ordering=[a@0 ASC], has_header=true", ]; let expected_optimized = [ "SortPreservingMergeExec: [a@0 ASC]", - " FilterExec: c@2 > 3", - " SortPreservingRepartitionExec: partitioning=Hash([c1@0], 8), input_partitions=8, sort_exprs=a@0 ASC", + " FilterExec: c@1 > 3", + " SortPreservingRepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8, sort_exprs=a@0 ASC", " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", " SortPreservingMergeExec: [a@0 ASC]", - " SortPreservingRepartitionExec: partitioning=Hash([c1@0], 8), input_partitions=8, sort_exprs=a@0 ASC", + " SortPreservingRepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8, sort_exprs=a@0 ASC", " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", " CsvExec: file_groups={1 group: [[file_path]]}, projection=[a, c, d], infinite_source=true, output_ordering=[a@0 ASC], has_header=true", ]; @@ -424,7 +427,7 @@ mod tests { let sort_exprs = vec![sort_expr("a", &schema)]; let source = csv_exec_sorted(&schema, sort_exprs, true); let repartition_rr = repartition_exec_round_robin(source); - let filter = filter_exec(repartition_rr, &schema); + let filter = filter_exec(repartition_rr); let repartition_hash = repartition_exec_hash(filter); let sort = sort_exec(vec![sort_expr("a", &schema)], repartition_hash, true); @@ -433,14 +436,14 @@ mod tests { let expected_input = ["SortPreservingMergeExec: [a@0 ASC NULLS LAST]", " SortExec: expr=[a@0 ASC NULLS LAST]", - " RepartitionExec: partitioning=Hash([c1@0], 8), input_partitions=8", - " FilterExec: c@2 > 3", + " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8", + " FilterExec: c@1 > 3", " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", " CsvExec: file_groups={1 group: [[file_path]]}, projection=[a, c, d], infinite_source=true, output_ordering=[a@0 ASC NULLS LAST], has_header=true"]; let expected_optimized = [ "SortPreservingMergeExec: [a@0 ASC NULLS LAST]", - " SortPreservingRepartitionExec: partitioning=Hash([c1@0], 8), input_partitions=8, sort_exprs=a@0 ASC NULLS LAST", - " FilterExec: c@2 > 3", + " SortPreservingRepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8, sort_exprs=a@0 ASC NULLS LAST", + " FilterExec: c@1 > 3", " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", " CsvExec: file_groups={1 group: [[file_path]]}, projection=[a, c, d], infinite_source=true, output_ordering=[a@0 ASC NULLS LAST], has_header=true", @@ -456,7 +459,7 @@ mod tests { let source = csv_exec_sorted(&schema, sort_exprs, true); let repartition_rr = repartition_exec_round_robin(source); let repartition_hash = repartition_exec_hash(repartition_rr); - let filter = filter_exec(repartition_hash, &schema); + let filter = filter_exec(repartition_hash); let coalesce_batches_exec: Arc = coalesce_batches_exec(filter); let sort = sort_exec(vec![sort_expr("a", &schema)], coalesce_batches_exec, true); @@ -466,14 +469,14 @@ mod tests { let expected_input = ["SortPreservingMergeExec: [a@0 ASC NULLS LAST]", " SortExec: expr=[a@0 ASC NULLS LAST]", " CoalesceBatchesExec: target_batch_size=8192", - " FilterExec: c@2 > 3", - " RepartitionExec: partitioning=Hash([c1@0], 8), input_partitions=8", + " FilterExec: c@1 > 3", + " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8", " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", " CsvExec: file_groups={1 group: [[file_path]]}, projection=[a, c, d], infinite_source=true, output_ordering=[a@0 ASC NULLS LAST], has_header=true"]; let expected_optimized = ["SortPreservingMergeExec: [a@0 ASC NULLS LAST]", " CoalesceBatchesExec: target_batch_size=8192", - " FilterExec: c@2 > 3", - " SortPreservingRepartitionExec: partitioning=Hash([c1@0], 8), input_partitions=8, sort_exprs=a@0 ASC NULLS LAST", + " FilterExec: c@1 > 3", + " SortPreservingRepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8, sort_exprs=a@0 ASC NULLS LAST", " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", " CsvExec: file_groups={1 group: [[file_path]]}, projection=[a, c, d], infinite_source=true, output_ordering=[a@0 ASC NULLS LAST], has_header=true"]; assert_optimized!(expected_input, expected_optimized, physical_plan); @@ -488,7 +491,7 @@ mod tests { let repartition_rr = repartition_exec_round_robin(source); let coalesce_batches_exec_1 = coalesce_batches_exec(repartition_rr); let repartition_hash = repartition_exec_hash(coalesce_batches_exec_1); - let filter = filter_exec(repartition_hash, &schema); + let filter = filter_exec(repartition_hash); let coalesce_batches_exec_2 = coalesce_batches_exec(filter); let sort = sort_exec(vec![sort_expr("a", &schema)], coalesce_batches_exec_2, true); @@ -499,15 +502,15 @@ mod tests { let expected_input = ["SortPreservingMergeExec: [a@0 ASC NULLS LAST]", " SortExec: expr=[a@0 ASC NULLS LAST]", " CoalesceBatchesExec: target_batch_size=8192", - " FilterExec: c@2 > 3", - " RepartitionExec: partitioning=Hash([c1@0], 8), input_partitions=8", + " FilterExec: c@1 > 3", + " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8", " CoalesceBatchesExec: target_batch_size=8192", " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", " CsvExec: file_groups={1 group: [[file_path]]}, projection=[a, c, d], infinite_source=true, output_ordering=[a@0 ASC NULLS LAST], has_header=true"]; let expected_optimized = ["SortPreservingMergeExec: [a@0 ASC NULLS LAST]", " CoalesceBatchesExec: target_batch_size=8192", - " FilterExec: c@2 > 3", - " SortPreservingRepartitionExec: partitioning=Hash([c1@0], 8), input_partitions=8, sort_exprs=a@0 ASC NULLS LAST", + " FilterExec: c@1 > 3", + " SortPreservingRepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8, sort_exprs=a@0 ASC NULLS LAST", " CoalesceBatchesExec: target_batch_size=8192", " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", " CsvExec: file_groups={1 group: [[file_path]]}, projection=[a, c, d], infinite_source=true, output_ordering=[a@0 ASC NULLS LAST], has_header=true"]; @@ -522,7 +525,7 @@ mod tests { let source = csv_exec_sorted(&schema, sort_exprs, true); let repartition_rr = repartition_exec_round_robin(source); let repartition_hash = repartition_exec_hash(repartition_rr); - let filter = filter_exec(repartition_hash, &schema); + let filter = filter_exec(repartition_hash); let coalesce_batches_exec: Arc = coalesce_batches_exec(filter); let physical_plan: Arc = @@ -530,14 +533,14 @@ mod tests { let expected_input = ["CoalescePartitionsExec", " CoalesceBatchesExec: target_batch_size=8192", - " FilterExec: c@2 > 3", - " RepartitionExec: partitioning=Hash([c1@0], 8), input_partitions=8", + " FilterExec: c@1 > 3", + " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8", " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", " CsvExec: file_groups={1 group: [[file_path]]}, projection=[a, c, d], infinite_source=true, output_ordering=[a@0 ASC NULLS LAST], has_header=true"]; let expected_optimized = ["CoalescePartitionsExec", " CoalesceBatchesExec: target_batch_size=8192", - " FilterExec: c@2 > 3", - " RepartitionExec: partitioning=Hash([c1@0], 8), input_partitions=8", + " FilterExec: c@1 > 3", + " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8", " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", " CsvExec: file_groups={1 group: [[file_path]]}, projection=[a, c, d], infinite_source=true, output_ordering=[a@0 ASC NULLS LAST], has_header=true"]; assert_optimized!(expected_input, expected_optimized, physical_plan); @@ -551,7 +554,7 @@ mod tests { let source = csv_exec_sorted(&schema, sort_exprs, true); let repartition_rr = repartition_exec_round_robin(source); let repartition_hash = repartition_exec_hash(repartition_rr); - let filter = filter_exec(repartition_hash, &schema); + let filter = filter_exec(repartition_hash); let coalesce_batches = coalesce_batches_exec(filter); let repartition_hash_2 = repartition_exec_hash(coalesce_batches); let sort = sort_exec(vec![sort_expr("a", &schema)], repartition_hash_2, true); @@ -562,19 +565,19 @@ mod tests { let expected_input = [ "SortPreservingMergeExec: [a@0 ASC NULLS LAST]", " SortExec: expr=[a@0 ASC NULLS LAST]", - " RepartitionExec: partitioning=Hash([c1@0], 8), input_partitions=8", + " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8", " CoalesceBatchesExec: target_batch_size=8192", - " FilterExec: c@2 > 3", - " RepartitionExec: partitioning=Hash([c1@0], 8), input_partitions=8", + " FilterExec: c@1 > 3", + " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8", " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", " CsvExec: file_groups={1 group: [[file_path]]}, projection=[a, c, d], infinite_source=true, output_ordering=[a@0 ASC NULLS LAST], has_header=true" ]; let expected_optimized = [ "SortPreservingMergeExec: [a@0 ASC NULLS LAST]", - " SortPreservingRepartitionExec: partitioning=Hash([c1@0], 8), input_partitions=8, sort_exprs=a@0 ASC NULLS LAST", + " SortPreservingRepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8, sort_exprs=a@0 ASC NULLS LAST", " CoalesceBatchesExec: target_batch_size=8192", - " FilterExec: c@2 > 3", - " SortPreservingRepartitionExec: partitioning=Hash([c1@0], 8), input_partitions=8, sort_exprs=a@0 ASC NULLS LAST", + " FilterExec: c@1 > 3", + " SortPreservingRepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8, sort_exprs=a@0 ASC NULLS LAST", " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", " CsvExec: file_groups={1 group: [[file_path]]}, projection=[a, c, d], infinite_source=true, output_ordering=[a@0 ASC NULLS LAST], has_header=true", ]; @@ -590,22 +593,24 @@ mod tests { let repartition_rr = repartition_exec_round_robin(source); let repartition_hash = repartition_exec_hash(repartition_rr); let sort = sort_exec( - vec![sort_expr_default("c", &schema)], + vec![sort_expr_default("c", &repartition_hash.schema())], repartition_hash, true, ); - let physical_plan = - sort_preserving_merge_exec(vec![sort_expr_default("c", &schema)], sort); + let physical_plan = sort_preserving_merge_exec( + vec![sort_expr_default("c", &sort.schema())], + sort, + ); - let expected_input = ["SortPreservingMergeExec: [c@2 ASC]", - " SortExec: expr=[c@2 ASC]", - " RepartitionExec: partitioning=Hash([c1@0], 8), input_partitions=8", + let expected_input = ["SortPreservingMergeExec: [c@1 ASC]", + " SortExec: expr=[c@1 ASC]", + " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8", " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", " CsvExec: file_groups={1 group: [[file_path]]}, projection=[a, c, d], infinite_source=true, output_ordering=[a@0 ASC NULLS LAST], has_header=true"]; - let expected_optimized = ["SortPreservingMergeExec: [c@2 ASC]", - " SortExec: expr=[c@2 ASC]", - " RepartitionExec: partitioning=Hash([c1@0], 8), input_partitions=8", + let expected_optimized = ["SortPreservingMergeExec: [c@1 ASC]", + " SortExec: expr=[c@1 ASC]", + " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8", " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", " CsvExec: file_groups={1 group: [[file_path]]}, projection=[a, c, d], infinite_source=true, output_ordering=[a@0 ASC NULLS LAST], has_header=true"]; assert_optimized!(expected_input, expected_optimized, physical_plan); @@ -625,11 +630,11 @@ mod tests { let expected_input = ["SortExec: expr=[a@0 ASC NULLS LAST]", " CoalescePartitionsExec", - " RepartitionExec: partitioning=Hash([c1@0], 8), input_partitions=8", + " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8", " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", " CsvExec: file_groups={1 group: [[file_path]]}, projection=[a, c, d], infinite_source=true, output_ordering=[a@0 ASC NULLS LAST], has_header=true"]; let expected_optimized = ["SortPreservingMergeExec: [a@0 ASC NULLS LAST]", - " SortPreservingRepartitionExec: partitioning=Hash([c1@0], 8), input_partitions=8, sort_exprs=a@0 ASC NULLS LAST", + " SortPreservingRepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8, sort_exprs=a@0 ASC NULLS LAST", " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", " CsvExec: file_groups={1 group: [[file_path]]}, projection=[a, c, d], infinite_source=true, output_ordering=[a@0 ASC NULLS LAST], has_header=true"]; assert_optimized!(expected_input, expected_optimized, physical_plan); @@ -645,39 +650,42 @@ mod tests { let repartition_hash = repartition_exec_hash(repartition_rr); let coalesce_partitions = coalesce_partitions_exec(repartition_hash); let sort = sort_exec( - vec![sort_expr_default("c", &schema)], + vec![sort_expr_default("c", &coalesce_partitions.schema())], coalesce_partitions, false, ); let repartition_rr2 = repartition_exec_round_robin(sort); let repartition_hash2 = repartition_exec_hash(repartition_rr2); - let filter = filter_exec(repartition_hash2, &schema); - let sort2 = sort_exec(vec![sort_expr_default("c", &schema)], filter, true); + let filter = filter_exec(repartition_hash2); + let sort2 = + sort_exec(vec![sort_expr_default("c", &filter.schema())], filter, true); - let physical_plan = - sort_preserving_merge_exec(vec![sort_expr_default("c", &schema)], sort2); + let physical_plan = sort_preserving_merge_exec( + vec![sort_expr_default("c", &sort2.schema())], + sort2, + ); let expected_input = [ - "SortPreservingMergeExec: [c@2 ASC]", - " SortExec: expr=[c@2 ASC]", - " FilterExec: c@2 > 3", - " RepartitionExec: partitioning=Hash([c1@0], 8), input_partitions=8", + "SortPreservingMergeExec: [c@1 ASC]", + " SortExec: expr=[c@1 ASC]", + " FilterExec: c@1 > 3", + " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8", " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", - " SortExec: expr=[c@2 ASC]", + " SortExec: expr=[c@1 ASC]", " CoalescePartitionsExec", - " RepartitionExec: partitioning=Hash([c1@0], 8), input_partitions=8", + " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8", " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", " CsvExec: file_groups={1 group: [[file_path]]}, projection=[a, c, d], infinite_source=true, output_ordering=[a@0 ASC NULLS LAST], has_header=true", ]; let expected_optimized = [ - "SortPreservingMergeExec: [c@2 ASC]", - " FilterExec: c@2 > 3", - " SortPreservingRepartitionExec: partitioning=Hash([c1@0], 8), input_partitions=8, sort_exprs=c@2 ASC", + "SortPreservingMergeExec: [c@1 ASC]", + " FilterExec: c@1 > 3", + " SortPreservingRepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8, sort_exprs=c@1 ASC", " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", - " SortExec: expr=[c@2 ASC]", + " SortExec: expr=[c@1 ASC]", " CoalescePartitionsExec", - " RepartitionExec: partitioning=Hash([c1@0], 8), input_partitions=8", + " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8", " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", " CsvExec: file_groups={1 group: [[file_path]]}, projection=[a, c, d], infinite_source=true, output_ordering=[a@0 ASC NULLS LAST], has_header=true", ]; @@ -705,21 +713,27 @@ mod tests { let hash_join_exec = hash_join_exec(left_coalesce_partitions, right_coalesce_partitions); - let sort = sort_exec(vec![sort_expr_default("a", &schema)], hash_join_exec, true); + let sort = sort_exec( + vec![sort_expr_default("a", &hash_join_exec.schema())], + hash_join_exec, + true, + ); - let physical_plan = - sort_preserving_merge_exec(vec![sort_expr_default("a", &schema)], sort); + let physical_plan = sort_preserving_merge_exec( + vec![sort_expr_default("a", &sort.schema())], + sort, + ); let expected_input = [ "SortPreservingMergeExec: [a@0 ASC]", " SortExec: expr=[a@0 ASC]", " HashJoinExec: mode=Partitioned, join_type=Inner, on=[(c@1, c@1)]", " CoalesceBatchesExec: target_batch_size=4096", - " RepartitionExec: partitioning=Hash([c1@0], 8), input_partitions=8", + " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8", " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", " CsvExec: file_groups={1 group: [[file_path]]}, projection=[a, c, d], infinite_source=true, output_ordering=[a@0 ASC NULLS LAST], has_header=true", " CoalesceBatchesExec: target_batch_size=4096", - " RepartitionExec: partitioning=Hash([c1@0], 8), input_partitions=8", + " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8", " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", " CsvExec: file_groups={1 group: [[file_path]]}, projection=[a, c, d], infinite_source=true, output_ordering=[a@0 ASC NULLS LAST], has_header=true", ]; @@ -729,11 +743,11 @@ mod tests { " SortExec: expr=[a@0 ASC]", " HashJoinExec: mode=Partitioned, join_type=Inner, on=[(c@1, c@1)]", " CoalesceBatchesExec: target_batch_size=4096", - " RepartitionExec: partitioning=Hash([c1@0], 8), input_partitions=8", + " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8", " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", " CsvExec: file_groups={1 group: [[file_path]]}, projection=[a, c, d], infinite_source=true, output_ordering=[a@0 ASC NULLS LAST], has_header=true", " CoalesceBatchesExec: target_batch_size=4096", - " RepartitionExec: partitioning=Hash([c1@0], 8), input_partitions=8", + " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8", " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", " CsvExec: file_groups={1 group: [[file_path]]}, projection=[a, c, d], infinite_source=true, output_ordering=[a@0 ASC NULLS LAST], has_header=true", ]; @@ -754,11 +768,11 @@ mod tests { let expected_input = ["SortPreservingMergeExec: [a@0 ASC NULLS LAST]", " SortExec: expr=[a@0 ASC NULLS LAST]", - " RepartitionExec: partitioning=Hash([c1@0], 8), input_partitions=8", + " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8", " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", " CsvExec: file_groups={1 group: [[file_path]]}, projection=[a, c, d], output_ordering=[a@0 ASC NULLS LAST], has_header=true"]; let expected_optimized = ["SortPreservingMergeExec: [a@0 ASC NULLS LAST]", - " SortPreservingRepartitionExec: partitioning=Hash([c1@0], 8), input_partitions=8, sort_exprs=a@0 ASC NULLS LAST", + " SortPreservingRepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8, sort_exprs=a@0 ASC NULLS LAST", " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", " CsvExec: file_groups={1 group: [[file_path]]}, projection=[a, c, d], output_ordering=[a@0 ASC NULLS LAST], has_header=true"]; assert_optimized!(expected_input, expected_optimized, physical_plan, true); @@ -821,24 +835,23 @@ mod tests { } fn repartition_exec_hash(input: Arc) -> Arc { + let input_schema = input.schema(); Arc::new( RepartitionExec::try_new( input, - Partitioning::Hash(vec![Arc::new(Column::new("c1", 0))], 8), + Partitioning::Hash(vec![col("c", &input_schema).unwrap()], 8), ) .unwrap(), ) } - fn filter_exec( - input: Arc, - schema: &SchemaRef, - ) -> Arc { + fn filter_exec(input: Arc) -> Arc { + let input_schema = input.schema(); let predicate = expressions::binary( - col("c", schema).unwrap(), + col("c", &input_schema).unwrap(), Operator::Gt, expressions::lit(3i32), - schema, + &input_schema, ) .unwrap(); Arc::new(FilterExec::try_new(predicate, input).unwrap()) @@ -856,11 +869,15 @@ mod tests { left: Arc, right: Arc, ) -> Arc { + let left_on = col("c", &left.schema()).unwrap(); + let right_on = col("c", &right.schema()).unwrap(); + let left_col = left_on.as_any().downcast_ref::().unwrap(); + let right_col = right_on.as_any().downcast_ref::().unwrap(); Arc::new( HashJoinExec::try_new( left, right, - vec![(Column::new("c", 1), Column::new("c", 1))], + vec![(left_col.clone(), right_col.clone())], None, &JoinType::Inner, PartitionMode::Partitioned, From 37eecfe6ef023b9496a7c781339e458b1fc7692d Mon Sep 17 00:00:00 2001 From: Kirill Zaborsky Date: Thu, 16 Nov 2023 13:52:25 +0300 Subject: [PATCH 270/572] Remove unnecessary reassignment (#8232) --- datafusion/core/src/physical_planner.rs | 9 +++------ 1 file changed, 3 insertions(+), 6 deletions(-) diff --git a/datafusion/core/src/physical_planner.rs b/datafusion/core/src/physical_planner.rs index fffc51abeb67..1f1ef73cae34 100644 --- a/datafusion/core/src/physical_planner.rs +++ b/datafusion/core/src/physical_planner.rs @@ -820,16 +820,13 @@ impl DefaultPhysicalPlanner { let updated_aggregates = initial_aggr.aggr_expr().to_vec(); let updated_order_bys = initial_aggr.order_by_expr().to_vec(); - let (initial_aggr, next_partition_mode): ( - Arc, - AggregateMode, - ) = if can_repartition { + let next_partition_mode = if can_repartition { // construct a second aggregation with 'AggregateMode::FinalPartitioned' - (initial_aggr, AggregateMode::FinalPartitioned) + AggregateMode::FinalPartitioned } else { // construct a second aggregation, keeping the final column name equal to the // first aggregation and the expressions corresponding to the respective aggregate - (initial_aggr, AggregateMode::Final) + AggregateMode::Final }; let final_grouping_set = PhysicalGroupBy::new_single( From 9fd0f4e3f84b1a4bb4e42279a41ae809d41cce35 Mon Sep 17 00:00:00 2001 From: Marco Neumann Date: Thu, 16 Nov 2023 12:26:11 +0100 Subject: [PATCH 271/572] Update itertools requirement from 0.11 to 0.12 (#8233) * Update itertools requirement from 0.11 to 0.12 Updates the requirements on [itertools](https://github.com/rust-itertools/itertools) to permit the latest version. - [Changelog](https://github.com/rust-itertools/itertools/blob/master/CHANGELOG.md) - [Commits](https://github.com/rust-itertools/itertools/compare/v0.11.0...v0.12.0) --- updated-dependencies: - dependency-name: itertools dependency-type: direct:production ... Signed-off-by: dependabot[bot] * chore: cargo update ```console $ cargo update Updating crates.io index Updating async-compression v0.4.4 -> v0.4.5 Downgrading cc v1.0.84 -> v1.0.83 Updating errno v0.3.6 -> v0.3.7 Updating h2 v0.3.21 -> v0.3.22 Updating http v0.2.10 -> v0.2.11 Adding itertools v0.12.0 Adding jobserver v0.1.27 Updating rustix v0.38.21 -> v0.38.24 Updating termcolor v1.3.0 -> v1.4.0 Updating zerocopy v0.7.25 -> v0.7.26 Updating zerocopy-derive v0.7.25 -> v0.7.26 Updating zeroize v1.6.0 -> v1.6.1 ``` --------- Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- Cargo.toml | 2 +- datafusion-cli/Cargo.lock | 73 ++++++++++++++++++----------- datafusion/physical-expr/Cargo.toml | 2 +- datafusion/physical-plan/Cargo.toml | 2 +- 4 files changed, 49 insertions(+), 30 deletions(-) diff --git a/Cargo.toml b/Cargo.toml index 7294c934b72b..11c48acffd75 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -76,7 +76,7 @@ env_logger = "0.10" futures = "0.3" half = "2.2.1" indexmap = "2.0.0" -itertools = "0.11" +itertools = "0.12" log = "^0.4" num_cpus = "1.13.0" object_store = { version = "0.7.0", default-features = false } diff --git a/datafusion-cli/Cargo.lock b/datafusion-cli/Cargo.lock index f0bb28469d2d..4bc61a48a36e 100644 --- a/datafusion-cli/Cargo.lock +++ b/datafusion-cli/Cargo.lock @@ -359,9 +359,9 @@ dependencies = [ [[package]] name = "async-compression" -version = "0.4.4" +version = "0.4.5" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "f658e2baef915ba0f26f1f7c42bfb8e12f532a01f449a090ded75ae7a07e9ba2" +checksum = "bc2d0cfb2a7388d34f590e76686704c494ed7aaceed62ee1ba35cbf363abc2a5" dependencies = [ "bzip2", "flate2", @@ -850,10 +850,11 @@ dependencies = [ [[package]] name = "cc" -version = "1.0.84" +version = "1.0.83" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "0f8e7c90afad890484a21653d08b6e209ae34770fb5ee298f9c699fcc1e5c856" +checksum = "f1174fb0b6ec23863f8b971027804a42614e347eafb0a95bf0b12cdae21fc4d0" dependencies = [ + "jobserver", "libc", ] @@ -1122,7 +1123,7 @@ dependencies = [ "half", "hashbrown 0.14.2", "indexmap 2.1.0", - "itertools", + "itertools 0.12.0", "log", "num-traits", "num_cpus", @@ -1227,7 +1228,7 @@ dependencies = [ "datafusion-expr", "datafusion-physical-expr", "hashbrown 0.14.2", - "itertools", + "itertools 0.12.0", "log", "regex-syntax", ] @@ -1252,7 +1253,7 @@ dependencies = [ "hashbrown 0.14.2", "hex", "indexmap 2.1.0", - "itertools", + "itertools 0.12.0", "libc", "log", "md-5", @@ -1284,7 +1285,7 @@ dependencies = [ "half", "hashbrown 0.14.2", "indexmap 2.1.0", - "itertools", + "itertools 0.12.0", "log", "once_cell", "parking_lot", @@ -1421,9 +1422,9 @@ checksum = "5443807d6dff69373d433ab9ef5378ad8df50ca6298caf15de6e52e24aaf54d5" [[package]] name = "errno" -version = "0.3.6" +version = "0.3.7" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "7c18ee0ed65a5f1f81cac6b1d213b69c35fa47d4252ad41f1486dbd8226fe36e" +checksum = "f258a7194e7f7c2a7837a8913aeab7fd8c383457034fa20ce4dd3dcb813e8eb8" dependencies = [ "libc", "windows-sys", @@ -1645,9 +1646,9 @@ checksum = "d2fabcfbdc87f4758337ca535fb41a6d701b65693ce38287d856d1674551ec9b" [[package]] name = "h2" -version = "0.3.21" +version = "0.3.22" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "91fc23aa11be92976ef4729127f1a74adf36d8436f7816b185d18df956790833" +checksum = "4d6250322ef6e60f93f9a2162799302cd6f68f79f6e5d85c8c16f14d1d958178" dependencies = [ "bytes", "fnv", @@ -1655,7 +1656,7 @@ dependencies = [ "futures-sink", "futures-util", "http", - "indexmap 1.9.3", + "indexmap 2.1.0", "slab", "tokio", "tokio-util", @@ -1736,9 +1737,9 @@ dependencies = [ [[package]] name = "http" -version = "0.2.10" +version = "0.2.11" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "f95b9abcae896730d42b78e09c155ed4ddf82c07b4de772c64aee5b2d8b7c150" +checksum = "8947b1a6fad4393052c7ba1f4cd97bed3e953a95c79c92ad9b051a04611d9fbb" dependencies = [ "bytes", "fnv", @@ -1910,12 +1911,30 @@ dependencies = [ "either", ] +[[package]] +name = "itertools" +version = "0.12.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "25db6b064527c5d482d0423354fcd07a89a2dfe07b67892e62411946db7f07b0" +dependencies = [ + "either", +] + [[package]] name = "itoa" version = "1.0.9" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "af150ab688ff2122fcef229be89cb50dd66af9e01a4ff320cc137eecc9bacc38" +[[package]] +name = "jobserver" +version = "0.1.27" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "8c37f63953c4c63420ed5fd3d6d398c719489b9f872b9fa683262f8edd363c7d" +dependencies = [ + "libc", +] + [[package]] name = "js-sys" version = "0.3.65" @@ -2280,7 +2299,7 @@ dependencies = [ "futures", "humantime", "hyper", - "itertools", + "itertools 0.11.0", "parking_lot", "percent-encoding", "quick-xml", @@ -2515,7 +2534,7 @@ dependencies = [ "anstyle", "difflib", "float-cmp", - "itertools", + "itertools 0.11.0", "normalize-line-endings", "predicates-core", "regex", @@ -2810,9 +2829,9 @@ dependencies = [ [[package]] name = "rustix" -version = "0.38.21" +version = "0.38.24" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "2b426b0506e5d50a7d8dafcf2e81471400deb602392c7dd110815afb4eaf02a3" +checksum = "9ad981d6c340a49cdc40a1028d9c6084ec7e9fa33fcb839cab656a267071e234" dependencies = [ "bitflags 2.4.1", "errno", @@ -3240,9 +3259,9 @@ dependencies = [ [[package]] name = "termcolor" -version = "1.3.0" +version = "1.4.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "6093bad37da69aab9d123a8091e4be0aa4a03e4d601ec641c327398315f62b64" +checksum = "ff1bc3d3f05aff0403e8ac0d92ced918ec05b666a43f83297ccef5bea8a3d449" dependencies = [ "winapi-util", ] @@ -3881,18 +3900,18 @@ dependencies = [ [[package]] name = "zerocopy" -version = "0.7.25" +version = "0.7.26" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "8cd369a67c0edfef15010f980c3cbe45d7f651deac2cd67ce097cd801de16557" +checksum = "e97e415490559a91254a2979b4829267a57d2fcd741a98eee8b722fb57289aa0" dependencies = [ "zerocopy-derive", ] [[package]] name = "zerocopy-derive" -version = "0.7.25" +version = "0.7.26" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "c2f140bda219a26ccc0cdb03dba58af72590c53b22642577d88a927bc5c87d6b" +checksum = "dd7e48ccf166952882ca8bd778a43502c64f33bf94c12ebe2a7f08e5a0f6689f" dependencies = [ "proc-macro2", "quote", @@ -3901,9 +3920,9 @@ dependencies = [ [[package]] name = "zeroize" -version = "1.6.0" +version = "1.6.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "2a0956f1ba7c7909bfb66c2e9e4124ab6f6482560f6628b5aaeba39207c9aad9" +checksum = "12a3946ecfc929b583800f4629b6c25b88ac6e92a40ea5670f77112a85d40a8b" [[package]] name = "zstd" diff --git a/datafusion/physical-expr/Cargo.toml b/datafusion/physical-expr/Cargo.toml index 4496e7215204..caa812d0751c 100644 --- a/datafusion/physical-expr/Cargo.toml +++ b/datafusion/physical-expr/Cargo.toml @@ -56,7 +56,7 @@ half = { version = "2.1", default-features = false } hashbrown = { version = "0.14", features = ["raw"] } hex = { version = "0.4", optional = true } indexmap = { workspace = true } -itertools = { version = "0.11", features = ["use_std"] } +itertools = { version = "0.12", features = ["use_std"] } libc = "0.2.140" log = { workspace = true } md-5 = { version = "^0.10.0", optional = true } diff --git a/datafusion/physical-plan/Cargo.toml b/datafusion/physical-plan/Cargo.toml index 82c8f49a764f..6c761fc9687c 100644 --- a/datafusion/physical-plan/Cargo.toml +++ b/datafusion/physical-plan/Cargo.toml @@ -48,7 +48,7 @@ futures = { workspace = true } half = { version = "2.1", default-features = false } hashbrown = { version = "0.14", features = ["raw"] } indexmap = { workspace = true } -itertools = { version = "0.11", features = ["use_std"] } +itertools = { version = "0.12", features = ["use_std"] } log = { workspace = true } once_cell = "1.18.0" parking_lot = { workspace = true } From b126bca13c9a868a3bff7d4c7d1a7546c687b201 Mon Sep 17 00:00:00 2001 From: Chojan Shang Date: Fri, 17 Nov 2023 00:42:19 +0800 Subject: [PATCH 272/572] Port tests in subqueries.rs to sqllogictest (#8231) * Port tests in subqueries.rs to sqllogictest Signed-off-by: Chojan Shang * Follow rowsort Signed-off-by: Chojan Shang --------- Signed-off-by: Chojan Shang --- datafusion/core/tests/sql/mod.rs | 1 - datafusion/core/tests/sql/subqueries.rs | 63 ------------------- .../sqllogictest/test_files/subquery.slt | 24 +++++++ 3 files changed, 24 insertions(+), 64 deletions(-) delete mode 100644 datafusion/core/tests/sql/subqueries.rs diff --git a/datafusion/core/tests/sql/mod.rs b/datafusion/core/tests/sql/mod.rs index 4bd42c4688df..40a9e627a72a 100644 --- a/datafusion/core/tests/sql/mod.rs +++ b/datafusion/core/tests/sql/mod.rs @@ -92,7 +92,6 @@ pub mod references; pub mod repartition; pub mod select; mod sql_api; -pub mod subqueries; pub mod timestamp; fn create_join_context( diff --git a/datafusion/core/tests/sql/subqueries.rs b/datafusion/core/tests/sql/subqueries.rs deleted file mode 100644 index 01f8dd684b23..000000000000 --- a/datafusion/core/tests/sql/subqueries.rs +++ /dev/null @@ -1,63 +0,0 @@ -// Licensed to the Apache Software Foundation (ASF) under one -// or more contributor license agreements. See the NOTICE file -// distributed with this work for additional information -// regarding copyright ownership. The ASF licenses this file -// to you under the Apache License, Version 2.0 (the -// "License"); you may not use this file except in compliance -// with the License. You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, -// software distributed under the License is distributed on an -// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -// KIND, either express or implied. See the License for the -// specific language governing permissions and limitations -// under the License. - -use super::*; -use crate::sql::execute_to_batches; - -#[tokio::test] -#[ignore] -async fn correlated_scalar_subquery_sum_agg_bug() -> Result<()> { - let ctx = create_join_context("t1_id", "t2_id", true)?; - - let sql = "select t1.t1_int from t1 where (select sum(t2_int) is null from t2 where t1.t1_id = t2.t2_id)"; - - let msg = format!("Creating logical plan for '{sql}'"); - let dataframe = ctx.sql(sql).await.expect(&msg); - let plan = dataframe.into_optimized_plan()?; - - let expected = vec![ - "Projection: t1.t1_int [t1_int:UInt32;N]", - " Inner Join: t1.t1_id = __scalar_sq_1.t2_id [t1_id:UInt32;N, t1_int:UInt32;N, t2_id:UInt32;N]", - " TableScan: t1 projection=[t1_id, t1_int] [t1_id:UInt32;N, t1_int:UInt32;N]", - " SubqueryAlias: __scalar_sq_1 [t2_id:UInt32;N]", - " Projection: t2.t2_id [t2_id:UInt32;N]", - " Filter: SUM(t2.t2_int) IS NULL [t2_id:UInt32;N, SUM(t2.t2_int):UInt64;N]", - " Aggregate: groupBy=[[t2.t2_id]], aggr=[[SUM(t2.t2_int)]] [t2_id:UInt32;N, SUM(t2.t2_int):UInt64;N]", - " TableScan: t2 projection=[t2_id, t2_int] [t2_id:UInt32;N, t2_int:UInt32;N]", - ]; - let formatted = plan.display_indent_schema().to_string(); - let actual: Vec<&str> = formatted.trim().lines().collect(); - assert_eq!( - expected, actual, - "\n\nexpected:\n\n{expected:#?}\nactual:\n\n{actual:#?}\n\n" - ); - - // assert data - let results = execute_to_batches(&ctx, sql).await; - let expected = [ - "+--------+", - "| t1_int |", - "+--------+", - "| 2 |", - "| 4 |", - "| 3 |", - "+--------+", - ]; - assert_batches_sorted_eq!(expected, &results); - - Ok(()) -} diff --git a/datafusion/sqllogictest/test_files/subquery.slt b/datafusion/sqllogictest/test_files/subquery.slt index ef08c88a9d20..ef25d960c954 100644 --- a/datafusion/sqllogictest/test_files/subquery.slt +++ b/datafusion/sqllogictest/test_files/subquery.slt @@ -988,3 +988,27 @@ SELECT * FROM ON (severity.cron_job_name = jobs.cron_job_name); ---- catan-prod1-daily success catan-prod1-daily high + +##correlated_scalar_subquery_sum_agg_bug +#query TT +#explain +#select t1.t1_int from t1 where +# (select sum(t2_int) is null from t2 where t1.t1_id = t2.t2_id) +#---- +#logical_plan +#Projection: t1.t1_int +#--Inner Join: t1.t1_id = __scalar_sq_1.t2_id +#----TableScan: t1 projection=[t1_id, t1_int] +#----SubqueryAlias: __scalar_sq_1 +#------Projection: t2.t2_id +#--------Filter: SUM(t2.t2_int) IS NULL +#----------Aggregate: groupBy=[[t2.t2_id]], aggr=[[SUM(t2.t2_int)]] +#------------TableScan: t2 projection=[t2_id, t2_int] + +#query I rowsort +#select t1.t1_int from t1 where +# (select sum(t2_int) is null from t2 where t1.t1_id = t2.t2_id) +#---- +#2 +#3 +#4 From b013087b505db037811a292c99a307032b81d52b Mon Sep 17 00:00:00 2001 From: Will Jones Date: Thu, 16 Nov 2023 09:04:57 -0800 Subject: [PATCH 273/572] feat: make FSL scalar also an arrayref (#8221) --- datafusion/common/src/scalar.rs | 77 +++++++------------ .../optimizer/src/analyzer/type_coercion.rs | 18 ++--- datafusion/proto/src/logical_plan/to_proto.rs | 2 +- 3 files changed, 36 insertions(+), 61 deletions(-) diff --git a/datafusion/common/src/scalar.rs b/datafusion/common/src/scalar.rs index cdcc9aa4fbc5..211ac13e197e 100644 --- a/datafusion/common/src/scalar.rs +++ b/datafusion/common/src/scalar.rs @@ -33,7 +33,7 @@ use crate::hash_utils::create_hashes; use crate::utils::array_into_list_array; use arrow::buffer::{NullBuffer, OffsetBuffer}; use arrow::compute::kernels::numeric::*; -use arrow::datatypes::{i256, FieldRef, Fields, SchemaBuilder}; +use arrow::datatypes::{i256, Fields, SchemaBuilder}; use arrow::{ array::*, compute::kernels::cast::{cast_with_options, CastOptions}, @@ -95,9 +95,13 @@ pub enum ScalarValue { FixedSizeBinary(i32, Option>), /// large binary LargeBinary(Option>), - /// Fixed size list of nested ScalarValue - Fixedsizelist(Option>, FieldRef, i32), + /// Fixed size list scalar. + /// + /// The array must be a FixedSizeListArray with length 1. + FixedSizeList(ArrayRef), /// Represents a single element of a [`ListArray`] as an [`ArrayRef`] + /// + /// The array must be a ListArray with length 1. List(ArrayRef), /// Date stored as a signed 32bit int days since UNIX epoch 1970-01-01 Date32(Option), @@ -196,10 +200,8 @@ impl PartialEq for ScalarValue { (FixedSizeBinary(_, _), _) => false, (LargeBinary(v1), LargeBinary(v2)) => v1.eq(v2), (LargeBinary(_), _) => false, - (Fixedsizelist(v1, t1, l1), Fixedsizelist(v2, t2, l2)) => { - v1.eq(v2) && t1.eq(t2) && l1.eq(l2) - } - (Fixedsizelist(_, _, _), _) => false, + (FixedSizeList(v1), FixedSizeList(v2)) => v1.eq(v2), + (FixedSizeList(_), _) => false, (List(v1), List(v2)) => v1.eq(v2), (List(_), _) => false, (Date32(v1), Date32(v2)) => v1.eq(v2), @@ -310,15 +312,7 @@ impl PartialOrd for ScalarValue { (FixedSizeBinary(_, _), _) => None, (LargeBinary(v1), LargeBinary(v2)) => v1.partial_cmp(v2), (LargeBinary(_), _) => None, - (Fixedsizelist(v1, t1, l1), Fixedsizelist(v2, t2, l2)) => { - if t1.eq(t2) && l1.eq(l2) { - v1.partial_cmp(v2) - } else { - None - } - } - (Fixedsizelist(_, _, _), _) => None, - (List(arr1), List(arr2)) => { + (List(arr1), List(arr2)) | (FixedSizeList(arr1), FixedSizeList(arr2)) => { if arr1.data_type() == arr2.data_type() { let list_arr1 = as_list_array(arr1); let list_arr2 = as_list_array(arr2); @@ -349,6 +343,7 @@ impl PartialOrd for ScalarValue { } } (List(_), _) => None, + (FixedSizeList(_), _) => None, (Date32(v1), Date32(v2)) => v1.partial_cmp(v2), (Date32(_), _) => None, (Date64(v1), Date64(v2)) => v1.partial_cmp(v2), @@ -465,12 +460,7 @@ impl std::hash::Hash for ScalarValue { Binary(v) => v.hash(state), FixedSizeBinary(_, v) => v.hash(state), LargeBinary(v) => v.hash(state), - Fixedsizelist(v, t, l) => { - v.hash(state); - t.hash(state); - l.hash(state); - } - List(arr) => { + List(arr) | FixedSizeList(arr) => { let arrays = vec![arr.to_owned()]; let hashes_buffer = &mut vec![0; arr.len()]; let random_state = ahash::RandomState::with_seeds(0, 0, 0, 0); @@ -881,11 +871,9 @@ impl ScalarValue { ScalarValue::Binary(_) => DataType::Binary, ScalarValue::FixedSizeBinary(sz, _) => DataType::FixedSizeBinary(*sz), ScalarValue::LargeBinary(_) => DataType::LargeBinary, - ScalarValue::Fixedsizelist(_, field, length) => DataType::FixedSizeList( - Arc::new(Field::new("item", field.data_type().clone(), true)), - *length, - ), - ScalarValue::List(arr) => arr.data_type().to_owned(), + ScalarValue::List(arr) | ScalarValue::FixedSizeList(arr) => { + arr.data_type().to_owned() + } ScalarValue::Date32(_) => DataType::Date32, ScalarValue::Date64(_) => DataType::Date64, ScalarValue::Time32Second(_) => DataType::Time32(TimeUnit::Second), @@ -1032,8 +1020,11 @@ impl ScalarValue { ScalarValue::Binary(v) => v.is_none(), ScalarValue::FixedSizeBinary(_, v) => v.is_none(), ScalarValue::LargeBinary(v) => v.is_none(), - ScalarValue::Fixedsizelist(v, ..) => v.is_none(), - ScalarValue::List(arr) => arr.len() == arr.null_count(), + // arr.len() should be 1 for a list scalar, but we don't seem to + // enforce that anywhere, so we still check against array length. + ScalarValue::List(arr) | ScalarValue::FixedSizeList(arr) => { + arr.len() == arr.null_count() + } ScalarValue::Date32(v) => v.is_none(), ScalarValue::Date64(v) => v.is_none(), ScalarValue::Time32Second(v) => v.is_none(), @@ -1855,7 +1846,7 @@ impl ScalarValue { .collect::(), ), }, - ScalarValue::Fixedsizelist(..) => { + ScalarValue::FixedSizeList(..) => { return _not_impl_err!("FixedSizeList is not supported yet") } ScalarValue::List(arr) => { @@ -2407,7 +2398,7 @@ impl ScalarValue { ScalarValue::LargeBinary(val) => { eq_array_primitive!(array, index, LargeBinaryArray, val)? } - ScalarValue::Fixedsizelist(..) => { + ScalarValue::FixedSizeList(..) => { return _not_impl_err!("FixedSizeList is not supported yet") } ScalarValue::List(_) => return _not_impl_err!("List is not supported yet"), @@ -2533,14 +2524,9 @@ impl ScalarValue { | ScalarValue::LargeBinary(b) => { b.as_ref().map(|b| b.capacity()).unwrap_or_default() } - ScalarValue::Fixedsizelist(vals, field, _) => { - vals.as_ref() - .map(|vals| Self::size_of_vec(vals) - std::mem::size_of_val(vals)) - .unwrap_or_default() - // `field` is boxed, so it is NOT already included in `self` - + field.size() + ScalarValue::List(arr) | ScalarValue::FixedSizeList(arr) => { + arr.get_array_memory_size() } - ScalarValue::List(arr) => arr.get_array_memory_size(), ScalarValue::Struct(vals, fields) => { vals.as_ref() .map(|vals| { @@ -2908,18 +2894,7 @@ impl fmt::Display for ScalarValue { )?, None => write!(f, "NULL")?, }, - ScalarValue::Fixedsizelist(e, ..) => match e { - Some(l) => write!( - f, - "{}", - l.iter() - .map(|v| format!("{v}")) - .collect::>() - .join(",") - )?, - None => write!(f, "NULL")?, - }, - ScalarValue::List(arr) => write!( + ScalarValue::List(arr) | ScalarValue::FixedSizeList(arr) => write!( f, "{}", arrow::util::pretty::pretty_format_columns("col", &[arr.to_owned()]) @@ -2999,7 +2974,7 @@ impl fmt::Debug for ScalarValue { } ScalarValue::LargeBinary(None) => write!(f, "LargeBinary({self})"), ScalarValue::LargeBinary(Some(_)) => write!(f, "LargeBinary(\"{self}\")"), - ScalarValue::Fixedsizelist(..) => write!(f, "FixedSizeList([{self}])"), + ScalarValue::FixedSizeList(arr) => write!(f, "FixedSizeList([{arr:?}])"), ScalarValue::List(arr) => write!(f, "List([{arr:?}])"), ScalarValue::Date32(_) => write!(f, "Date32(\"{self}\")"), ScalarValue::Date64(_) => write!(f, "Date64(\"{self}\")"), diff --git a/datafusion/optimizer/src/analyzer/type_coercion.rs b/datafusion/optimizer/src/analyzer/type_coercion.rs index 57dabbfee41c..2c5e8c8b1c45 100644 --- a/datafusion/optimizer/src/analyzer/type_coercion.rs +++ b/datafusion/optimizer/src/analyzer/type_coercion.rs @@ -763,6 +763,7 @@ fn coerce_case_expression(case: Case, schema: &DFSchemaRef) -> Result { mod test { use std::sync::Arc; + use arrow::array::{FixedSizeListArray, Int32Array}; use arrow::datatypes::{DataType, TimeUnit}; use arrow::datatypes::Field; @@ -1237,15 +1238,14 @@ mod test { #[test] fn test_casting_for_fixed_size_list() -> Result<()> { - let val = lit(ScalarValue::Fixedsizelist( - Some(vec![ - ScalarValue::from(1i32), - ScalarValue::from(2i32), - ScalarValue::from(3i32), - ]), - Arc::new(Field::new("item", DataType::Int32, true)), - 3, - )); + let val = lit(ScalarValue::FixedSizeList(Arc::new( + FixedSizeListArray::new( + Arc::new(Field::new("item", DataType::Int32, true)), + 3, + Arc::new(Int32Array::from(vec![1, 2, 3])), + None, + ), + ))); let expr = Expr::ScalarFunction(ScalarFunction { fun: BuiltinScalarFunction::MakeArray, args: vec![val.clone()], diff --git a/datafusion/proto/src/logical_plan/to_proto.rs b/datafusion/proto/src/logical_plan/to_proto.rs index 53be5f7bd498..649be05b88c3 100644 --- a/datafusion/proto/src/logical_plan/to_proto.rs +++ b/datafusion/proto/src/logical_plan/to_proto.rs @@ -1134,7 +1134,7 @@ impl TryFrom<&ScalarValue> for protobuf::ScalarValue { Value::LargeUtf8Value(s.to_owned()) }) } - ScalarValue::Fixedsizelist(..) => Err(Error::General( + ScalarValue::FixedSizeList(..) => Err(Error::General( "Proto serialization error: ScalarValue::Fixedsizelist not supported" .to_string(), )), From 2938d1437e7fff4d306ff2d6eb26846f8f03ccc3 Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Thu, 16 Nov 2023 10:48:37 -0700 Subject: [PATCH 274/572] Add versions to datafusion dependencies (#8238) --- Cargo.toml | 22 +++++++++++----------- 1 file changed, 11 insertions(+), 11 deletions(-) diff --git a/Cargo.toml b/Cargo.toml index 11c48acffd75..f25c24fd3e1c 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -59,17 +59,17 @@ async-trait = "0.1.73" bigdecimal = "0.4.1" bytes = "1.4" ctor = "0.2.0" -datafusion = { path = "datafusion/core" } -datafusion-common = { path = "datafusion/common" } -datafusion-expr = { path = "datafusion/expr" } -datafusion-sql = { path = "datafusion/sql" } -datafusion-optimizer = { path = "datafusion/optimizer" } -datafusion-physical-expr = { path = "datafusion/physical-expr" } -datafusion-physical-plan = { path = "datafusion/physical-plan" } -datafusion-execution = { path = "datafusion/execution" } -datafusion-proto = { path = "datafusion/proto" } -datafusion-sqllogictest = { path = "datafusion/sqllogictest" } -datafusion-substrait = { path = "datafusion/substrait" } +datafusion = { path = "datafusion/core", version = "33.0.0" } +datafusion-common = { path = "datafusion/common", version = "33.0.0" } +datafusion-expr = { path = "datafusion/expr", version = "33.0.0" } +datafusion-sql = { path = "datafusion/sql", version = "33.0.0" } +datafusion-optimizer = { path = "datafusion/optimizer", version = "33.0.0" } +datafusion-physical-expr = { path = "datafusion/physical-expr", version = "33.0.0" } +datafusion-physical-plan = { path = "datafusion/physical-plan", version = "33.0.0" } +datafusion-execution = { path = "datafusion/execution", version = "33.0.0" } +datafusion-proto = { path = "datafusion/proto", version = "33.0.0" } +datafusion-sqllogictest = { path = "datafusion/sqllogictest", version = "33.0.0" } +datafusion-substrait = { path = "datafusion/substrait", version = "33.0.0" } dashmap = "5.4.0" doc-comment = "0.3" env_logger = "0.10" From 1e6ff64151edd4d16362b05598885cd02ef04453 Mon Sep 17 00:00:00 2001 From: Will Jones Date: Thu, 16 Nov 2023 11:24:40 -0800 Subject: [PATCH 275/572] feat: support eq_array and to_array_of_size for FSL (#8225) --- datafusion/common/src/scalar.rs | 63 +++++++++++++++++++++++++++------ 1 file changed, 53 insertions(+), 10 deletions(-) diff --git a/datafusion/common/src/scalar.rs b/datafusion/common/src/scalar.rs index 211ac13e197e..e8dac2a7f486 100644 --- a/datafusion/common/src/scalar.rs +++ b/datafusion/common/src/scalar.rs @@ -1725,7 +1725,7 @@ impl ScalarValue { /// /// Errors if `self` is /// - a decimal that fails be converted to a decimal array of size - /// - a `Fixedsizelist` that is not supported yet + /// - a `Fixedsizelist` that fails to be concatenated into an array of size /// - a `List` that fails to be concatenated into an array of size /// - a `Dictionary` that fails be converted to a dictionary array of size pub fn to_array_of_size(&self, size: usize) -> Result { @@ -1846,10 +1846,7 @@ impl ScalarValue { .collect::(), ), }, - ScalarValue::FixedSizeList(..) => { - return _not_impl_err!("FixedSizeList is not supported yet") - } - ScalarValue::List(arr) => { + ScalarValue::List(arr) | ScalarValue::FixedSizeList(arr) => { let arrays = std::iter::repeat(arr.as_ref()) .take(size) .collect::>(); @@ -2324,8 +2321,6 @@ impl ScalarValue { /// /// Errors if /// - it fails to downcast `array` to the data type of `self` - /// - `self` is a `Fixedsizelist` - /// - `self` is a `List` /// - `self` is a `Struct` /// /// # Panics @@ -2398,10 +2393,10 @@ impl ScalarValue { ScalarValue::LargeBinary(val) => { eq_array_primitive!(array, index, LargeBinaryArray, val)? } - ScalarValue::FixedSizeList(..) => { - return _not_impl_err!("FixedSizeList is not supported yet") + ScalarValue::List(arr) | ScalarValue::FixedSizeList(arr) => { + let right = array.slice(index, 1); + arr == &right } - ScalarValue::List(_) => return _not_impl_err!("List is not supported yet"), ScalarValue::Date32(val) => { eq_array_primitive!(array, index, Date32Array, val)? } @@ -3103,6 +3098,27 @@ mod tests { assert_eq!(&arr, actual_list_arr); } + #[test] + fn test_to_array_of_size_for_fsl() { + let values = Int32Array::from_iter([Some(1), None, Some(2)]); + let field = Arc::new(Field::new("item", DataType::Int32, true)); + let arr = FixedSizeListArray::new(field.clone(), 3, Arc::new(values), None); + let sv = ScalarValue::FixedSizeList(Arc::new(arr)); + let actual_arr = sv + .to_array_of_size(2) + .expect("Failed to convert to array of size"); + + let expected_values = + Int32Array::from_iter([Some(1), None, Some(2), Some(1), None, Some(2)]); + let expected_arr = + FixedSizeListArray::new(field, 3, Arc::new(expected_values), None); + + assert_eq!( + &expected_arr, + as_fixed_size_list_array(actual_arr.as_ref()).unwrap() + ); + } + #[test] fn test_list_to_array_string() { let scalars = vec![ @@ -3181,6 +3197,33 @@ mod tests { assert_eq!(result, &expected); } + #[test] + fn test_list_scalar_eq_to_array() { + let list_array: ArrayRef = + Arc::new(ListArray::from_iter_primitive::(vec![ + Some(vec![Some(0), Some(1), Some(2)]), + None, + Some(vec![None, Some(5)]), + ])); + + let fsl_array: ArrayRef = + Arc::new(FixedSizeListArray::from_iter_primitive::( + vec![ + Some(vec![Some(0), Some(1), Some(2)]), + None, + Some(vec![Some(3), None, Some(5)]), + ], + 3, + )); + + for arr in [list_array, fsl_array] { + for i in 0..arr.len() { + let scalar = ScalarValue::List(arr.slice(i, 1)); + assert!(scalar.eq_array(&arr, i).unwrap()); + } + } + } + #[test] fn scalar_add_trait_test() -> Result<()> { let float_value = ScalarValue::Float64(Some(123.)); From 7618e4d9c1801d76335164a1e70960d37012c516 Mon Sep 17 00:00:00 2001 From: Syleechan <38198463+Syleechan@users.noreply.github.com> Date: Fri, 17 Nov 2023 11:35:03 +0800 Subject: [PATCH 276/572] feat:implement calcite style 'levenshtein' string function (#8168) * feat:implement calcite style 'levenshtein' string function * format doc style * cargo lock --- datafusion/expr/src/built_in_function.rs | 12 ++++ datafusion/expr/src/expr_fn.rs | 2 + datafusion/physical-expr/src/functions.rs | 13 ++++ .../physical-expr/src/string_expressions.rs | 67 ++++++++++++++++++- datafusion/proto/proto/datafusion.proto | 1 + datafusion/proto/src/generated/pbjson.rs | 3 + datafusion/proto/src/generated/prost.rs | 3 + .../proto/src/logical_plan/from_proto.rs | 7 +- datafusion/proto/src/logical_plan/to_proto.rs | 1 + .../sqllogictest/test_files/functions.slt | 22 +++++- .../source/user-guide/sql/scalar_functions.md | 15 +++++ 11 files changed, 142 insertions(+), 4 deletions(-) diff --git a/datafusion/expr/src/built_in_function.rs b/datafusion/expr/src/built_in_function.rs index 1b48c37406d3..fc6f9c28e105 100644 --- a/datafusion/expr/src/built_in_function.rs +++ b/datafusion/expr/src/built_in_function.rs @@ -298,6 +298,8 @@ pub enum BuiltinScalarFunction { ArrowTypeof, /// overlay OverLay, + /// levenshtein + Levenshtein, } /// Maps the sql function name to `BuiltinScalarFunction` @@ -464,6 +466,7 @@ impl BuiltinScalarFunction { BuiltinScalarFunction::FromUnixtime => Volatility::Immutable, BuiltinScalarFunction::ArrowTypeof => Volatility::Immutable, BuiltinScalarFunction::OverLay => Volatility::Immutable, + BuiltinScalarFunction::Levenshtein => Volatility::Immutable, // Stable builtin functions BuiltinScalarFunction::Now => Volatility::Stable, @@ -829,6 +832,10 @@ impl BuiltinScalarFunction { utf8_to_str_type(&input_expr_types[0], "overlay") } + BuiltinScalarFunction::Levenshtein => { + utf8_to_int_type(&input_expr_types[0], "levenshtein") + } + BuiltinScalarFunction::Acos | BuiltinScalarFunction::Asin | BuiltinScalarFunction::Atan @@ -1293,6 +1300,10 @@ impl BuiltinScalarFunction { ], self.volatility(), ), + BuiltinScalarFunction::Levenshtein => Signature::one_of( + vec![Exact(vec![Utf8, Utf8]), Exact(vec![LargeUtf8, LargeUtf8])], + self.volatility(), + ), BuiltinScalarFunction::Acos | BuiltinScalarFunction::Asin | BuiltinScalarFunction::Atan @@ -1457,6 +1468,7 @@ fn aliases(func: &BuiltinScalarFunction) -> &'static [&'static str] { BuiltinScalarFunction::Trim => &["trim"], BuiltinScalarFunction::Upper => &["upper"], BuiltinScalarFunction::Uuid => &["uuid"], + BuiltinScalarFunction::Levenshtein => &["levenshtein"], // regex functions BuiltinScalarFunction::RegexpMatch => &["regexp_match"], diff --git a/datafusion/expr/src/expr_fn.rs b/datafusion/expr/src/expr_fn.rs index bcf1aa0ca7e5..75b762804427 100644 --- a/datafusion/expr/src/expr_fn.rs +++ b/datafusion/expr/src/expr_fn.rs @@ -909,6 +909,7 @@ scalar_expr!( ); scalar_expr!(ArrowTypeof, arrow_typeof, val, "data type"); +scalar_expr!(Levenshtein, levenshtein, string1 string2, "Returns the Levenshtein distance between the two given strings"); scalar_expr!( Struct, @@ -1195,6 +1196,7 @@ mod test { test_unary_scalar_expr!(ArrowTypeof, arrow_typeof); test_nary_scalar_expr!(OverLay, overlay, string, characters, position, len); test_nary_scalar_expr!(OverLay, overlay, string, characters, position); + test_scalar_expr!(Levenshtein, levenshtein, string1, string2); } #[test] diff --git a/datafusion/physical-expr/src/functions.rs b/datafusion/physical-expr/src/functions.rs index 1e8500079f21..b46249d26dde 100644 --- a/datafusion/physical-expr/src/functions.rs +++ b/datafusion/physical-expr/src/functions.rs @@ -846,6 +846,19 @@ pub fn create_physical_fun( "Unsupported data type {other:?} for function overlay", ))), }), + BuiltinScalarFunction::Levenshtein => { + Arc::new(|args| match args[0].data_type() { + DataType::Utf8 => { + make_scalar_function(string_expressions::levenshtein::)(args) + } + DataType::LargeUtf8 => { + make_scalar_function(string_expressions::levenshtein::)(args) + } + other => Err(DataFusionError::Internal(format!( + "Unsupported data type {other:?} for function levenshtein", + ))), + }) + } }) } diff --git a/datafusion/physical-expr/src/string_expressions.rs b/datafusion/physical-expr/src/string_expressions.rs index 7e954fdcfdc4..91d21f95e41f 100644 --- a/datafusion/physical-expr/src/string_expressions.rs +++ b/datafusion/physical-expr/src/string_expressions.rs @@ -23,11 +23,12 @@ use arrow::{ array::{ - Array, ArrayRef, BooleanArray, GenericStringArray, Int32Array, OffsetSizeTrait, - StringArray, + Array, ArrayRef, BooleanArray, GenericStringArray, Int32Array, Int64Array, + OffsetSizeTrait, StringArray, }, datatypes::{ArrowNativeType, ArrowPrimitiveType, DataType}, }; +use datafusion_common::utils::datafusion_strsim; use datafusion_common::{ cast::{ as_generic_string_array, as_int64_array, as_primitive_array, as_string_array, @@ -643,12 +644,59 @@ pub fn overlay(args: &[ArrayRef]) -> Result { } } +///Returns the Levenshtein distance between the two given strings. +/// LEVENSHTEIN('kitten', 'sitting') = 3 +pub fn levenshtein(args: &[ArrayRef]) -> Result { + if args.len() != 2 { + return Err(DataFusionError::Internal(format!( + "levenshtein function requires two arguments, got {}", + args.len() + ))); + } + let str1_array = as_generic_string_array::(&args[0])?; + let str2_array = as_generic_string_array::(&args[1])?; + match args[0].data_type() { + DataType::Utf8 => { + let result = str1_array + .iter() + .zip(str2_array.iter()) + .map(|(string1, string2)| match (string1, string2) { + (Some(string1), Some(string2)) => { + Some(datafusion_strsim::levenshtein(string1, string2) as i32) + } + _ => None, + }) + .collect::(); + Ok(Arc::new(result) as ArrayRef) + } + DataType::LargeUtf8 => { + let result = str1_array + .iter() + .zip(str2_array.iter()) + .map(|(string1, string2)| match (string1, string2) { + (Some(string1), Some(string2)) => { + Some(datafusion_strsim::levenshtein(string1, string2) as i64) + } + _ => None, + }) + .collect::(); + Ok(Arc::new(result) as ArrayRef) + } + other => { + internal_err!( + "levenshtein was called with {other} datatype arguments. It requires Utf8 or LargeUtf8." + ) + } + } +} + #[cfg(test)] mod tests { use crate::string_expressions; use arrow::{array::Int32Array, datatypes::Int32Type}; use arrow_array::Int64Array; + use datafusion_common::cast::as_int32_array; use super::*; @@ -707,4 +755,19 @@ mod tests { Ok(()) } + + #[test] + fn to_levenshtein() -> Result<()> { + let string1_array = + Arc::new(StringArray::from(vec!["123", "abc", "xyz", "kitten"])); + let string2_array = + Arc::new(StringArray::from(vec!["321", "def", "zyx", "sitting"])); + let res = levenshtein::(&[string1_array, string2_array]).unwrap(); + let result = + as_int32_array(&res).expect("failed to initialized function levenshtein"); + let expected = Int32Array::from(vec![2, 3, 2, 3]); + assert_eq!(&expected, result); + + Ok(()) + } } diff --git a/datafusion/proto/proto/datafusion.proto b/datafusion/proto/proto/datafusion.proto index 66c34c7a12ec..a5c3d3b603df 100644 --- a/datafusion/proto/proto/datafusion.proto +++ b/datafusion/proto/proto/datafusion.proto @@ -639,6 +639,7 @@ enum ScalarFunction { OverLay = 121; Range = 122; ArrayPopFront = 123; + Levenshtein = 124; } message ScalarFunctionNode { diff --git a/datafusion/proto/src/generated/pbjson.rs b/datafusion/proto/src/generated/pbjson.rs index 628adcc41189..3faacca18c60 100644 --- a/datafusion/proto/src/generated/pbjson.rs +++ b/datafusion/proto/src/generated/pbjson.rs @@ -20938,6 +20938,7 @@ impl serde::Serialize for ScalarFunction { Self::OverLay => "OverLay", Self::Range => "Range", Self::ArrayPopFront => "ArrayPopFront", + Self::Levenshtein => "Levenshtein", }; serializer.serialize_str(variant) } @@ -21073,6 +21074,7 @@ impl<'de> serde::Deserialize<'de> for ScalarFunction { "OverLay", "Range", "ArrayPopFront", + "Levenshtein", ]; struct GeneratedVisitor; @@ -21237,6 +21239,7 @@ impl<'de> serde::Deserialize<'de> for ScalarFunction { "OverLay" => Ok(ScalarFunction::OverLay), "Range" => Ok(ScalarFunction::Range), "ArrayPopFront" => Ok(ScalarFunction::ArrayPopFront), + "Levenshtein" => Ok(ScalarFunction::Levenshtein), _ => Err(serde::de::Error::unknown_variant(value, FIELDS)), } } diff --git a/datafusion/proto/src/generated/prost.rs b/datafusion/proto/src/generated/prost.rs index 317b888447a0..2555a31f6fe2 100644 --- a/datafusion/proto/src/generated/prost.rs +++ b/datafusion/proto/src/generated/prost.rs @@ -2570,6 +2570,7 @@ pub enum ScalarFunction { OverLay = 121, Range = 122, ArrayPopFront = 123, + Levenshtein = 124, } impl ScalarFunction { /// String value of the enum field names used in the ProtoBuf definition. @@ -2702,6 +2703,7 @@ impl ScalarFunction { ScalarFunction::OverLay => "OverLay", ScalarFunction::Range => "Range", ScalarFunction::ArrayPopFront => "ArrayPopFront", + ScalarFunction::Levenshtein => "Levenshtein", } } /// Creates an enum from field names used in the ProtoBuf definition. @@ -2831,6 +2833,7 @@ impl ScalarFunction { "OverLay" => Some(Self::OverLay), "Range" => Some(Self::Range), "ArrayPopFront" => Some(Self::ArrayPopFront), + "Levenshtein" => Some(Self::Levenshtein), _ => None, } } diff --git a/datafusion/proto/src/logical_plan/from_proto.rs b/datafusion/proto/src/logical_plan/from_proto.rs index 94c9f9806621..f14da70485ab 100644 --- a/datafusion/proto/src/logical_plan/from_proto.rs +++ b/datafusion/proto/src/logical_plan/from_proto.rs @@ -50,7 +50,7 @@ use datafusion_expr::{ date_part, date_trunc, decode, degrees, digest, encode, exp, expr::{self, InList, Sort, WindowFunction}, factorial, flatten, floor, from_unixtime, gcd, gen_range, isnan, iszero, lcm, left, - ln, log, log10, log2, + levenshtein, ln, log, log10, log2, logical_plan::{PlanType, StringifiedPlan}, lower, lpad, ltrim, md5, nanvl, now, nullif, octet_length, overlay, pi, power, radians, random, regexp_match, regexp_replace, repeat, replace, reverse, right, @@ -549,6 +549,7 @@ impl From<&protobuf::ScalarFunction> for BuiltinScalarFunction { ScalarFunction::Iszero => Self::Iszero, ScalarFunction::ArrowTypeof => Self::ArrowTypeof, ScalarFunction::OverLay => Self::OverLay, + ScalarFunction::Levenshtein => Self::Levenshtein, } } } @@ -1630,6 +1631,10 @@ pub fn parse_expr( )) } } + ScalarFunction::Levenshtein => Ok(levenshtein( + parse_expr(&args[0], registry)?, + parse_expr(&args[1], registry)?, + )), ScalarFunction::ToHex => Ok(to_hex(parse_expr(&args[0], registry)?)), ScalarFunction::ToTimestampMillis => { Ok(to_timestamp_millis(parse_expr(&args[0], registry)?)) diff --git a/datafusion/proto/src/logical_plan/to_proto.rs b/datafusion/proto/src/logical_plan/to_proto.rs index 649be05b88c3..de81a1f4caef 100644 --- a/datafusion/proto/src/logical_plan/to_proto.rs +++ b/datafusion/proto/src/logical_plan/to_proto.rs @@ -1556,6 +1556,7 @@ impl TryFrom<&BuiltinScalarFunction> for protobuf::ScalarFunction { BuiltinScalarFunction::Iszero => Self::Iszero, BuiltinScalarFunction::ArrowTypeof => Self::ArrowTypeof, BuiltinScalarFunction::OverLay => Self::OverLay, + BuiltinScalarFunction::Levenshtein => Self::Levenshtein, }; Ok(scalar_function) diff --git a/datafusion/sqllogictest/test_files/functions.slt b/datafusion/sqllogictest/test_files/functions.slt index 8f4230438480..9c8bb2c5f844 100644 --- a/datafusion/sqllogictest/test_files/functions.slt +++ b/datafusion/sqllogictest/test_files/functions.slt @@ -788,7 +788,7 @@ INSERT INTO products (product_id, product_name, price) VALUES (1, 'OldBrand Product 1', 19.99), (2, 'OldBrand Product 2', 29.99), (3, 'OldBrand Product 3', 39.99), -(4, 'OldBrand Product 4', 49.99) +(4, 'OldBrand Product 4', 49.99) query ITR SELECT * REPLACE (price*2 AS price) FROM products @@ -857,3 +857,23 @@ NULL NULL Thomxas NULL + +query I +SELECT levenshtein('kitten', 'sitting') +---- +3 + +query I +SELECT levenshtein('kitten', NULL) +---- +NULL + +query ? +SELECT levenshtein(NULL, 'sitting') +---- +NULL + +query ? +SELECT levenshtein(NULL, NULL) +---- +NULL diff --git a/docs/source/user-guide/sql/scalar_functions.md b/docs/source/user-guide/sql/scalar_functions.md index baaea3926f7d..f9f45a1b0a97 100644 --- a/docs/source/user-guide/sql/scalar_functions.md +++ b/docs/source/user-guide/sql/scalar_functions.md @@ -636,6 +636,7 @@ nullif(expression1, expression2) - [upper](#upper) - [uuid](#uuid) - [overlay](#overlay) +- [levenshtein](#levenshtein) ### `ascii` @@ -1137,6 +1138,20 @@ overlay(str PLACING substr FROM pos [FOR count]) - **pos**: the start position to replace of str. - **count**: the count of characters to be replaced from start position of str. If not specified, will use substr length instead. +### `levenshtein` + +Returns the Levenshtein distance between the two given strings. +For example, `levenshtein('kitten', 'sitting') = 3` + +``` +levenshtein(str1, str2) +``` + +#### Arguments + +- **str1**: String expression to compute Levenshtein distance with str2. +- **str2**: String expression to compute Levenshtein distance with str1. + ## Binary String Functions - [decode](#decode) From 2c5e237ab43cb6ba48c4f892120a2a7558466e76 Mon Sep 17 00:00:00 2001 From: Will Jones Date: Fri, 17 Nov 2023 06:46:21 -0800 Subject: [PATCH 277/572] feat: roundtrip FixedSizeList Scalar to protobuf (#8239) --- datafusion/proto/proto/datafusion.proto | 1 + datafusion/proto/src/generated/pbjson.rs | 14 ++++++++++ datafusion/proto/src/generated/prost.rs | 4 ++- .../proto/src/logical_plan/from_proto.rs | 8 ++++-- datafusion/proto/src/logical_plan/to_proto.rs | 28 +++++++++++-------- .../tests/cases/roundtrip_logical_plan.rs | 14 ++++++++-- 6 files changed, 51 insertions(+), 18 deletions(-) diff --git a/datafusion/proto/proto/datafusion.proto b/datafusion/proto/proto/datafusion.proto index a5c3d3b603df..8cab62acde04 100644 --- a/datafusion/proto/proto/datafusion.proto +++ b/datafusion/proto/proto/datafusion.proto @@ -983,6 +983,7 @@ message ScalarValue{ int32 date_32_value = 14; ScalarTime32Value time32_value = 15; ScalarListValue list_value = 17; + ScalarListValue fixed_size_list_value = 18; Decimal128 decimal128_value = 20; Decimal256 decimal256_value = 39; diff --git a/datafusion/proto/src/generated/pbjson.rs b/datafusion/proto/src/generated/pbjson.rs index 3faacca18c60..c50571dca0bb 100644 --- a/datafusion/proto/src/generated/pbjson.rs +++ b/datafusion/proto/src/generated/pbjson.rs @@ -22042,6 +22042,9 @@ impl serde::Serialize for ScalarValue { scalar_value::Value::ListValue(v) => { struct_ser.serialize_field("listValue", v)?; } + scalar_value::Value::FixedSizeListValue(v) => { + struct_ser.serialize_field("fixedSizeListValue", v)?; + } scalar_value::Value::Decimal128Value(v) => { struct_ser.serialize_field("decimal128Value", v)?; } @@ -22147,6 +22150,8 @@ impl<'de> serde::Deserialize<'de> for ScalarValue { "time32Value", "list_value", "listValue", + "fixed_size_list_value", + "fixedSizeListValue", "decimal128_value", "decimal128Value", "decimal256_value", @@ -22202,6 +22207,7 @@ impl<'de> serde::Deserialize<'de> for ScalarValue { Date32Value, Time32Value, ListValue, + FixedSizeListValue, Decimal128Value, Decimal256Value, Date64Value, @@ -22257,6 +22263,7 @@ impl<'de> serde::Deserialize<'de> for ScalarValue { "date32Value" | "date_32_value" => Ok(GeneratedField::Date32Value), "time32Value" | "time32_value" => Ok(GeneratedField::Time32Value), "listValue" | "list_value" => Ok(GeneratedField::ListValue), + "fixedSizeListValue" | "fixed_size_list_value" => Ok(GeneratedField::FixedSizeListValue), "decimal128Value" | "decimal128_value" => Ok(GeneratedField::Decimal128Value), "decimal256Value" | "decimal256_value" => Ok(GeneratedField::Decimal256Value), "date64Value" | "date_64_value" => Ok(GeneratedField::Date64Value), @@ -22399,6 +22406,13 @@ impl<'de> serde::Deserialize<'de> for ScalarValue { return Err(serde::de::Error::duplicate_field("listValue")); } value__ = map_.next_value::<::std::option::Option<_>>()?.map(scalar_value::Value::ListValue) +; + } + GeneratedField::FixedSizeListValue => { + if value__.is_some() { + return Err(serde::de::Error::duplicate_field("fixedSizeListValue")); + } + value__ = map_.next_value::<::std::option::Option<_>>()?.map(scalar_value::Value::FixedSizeListValue) ; } GeneratedField::Decimal128Value => { diff --git a/datafusion/proto/src/generated/prost.rs b/datafusion/proto/src/generated/prost.rs index 2555a31f6fe2..213be1c395c1 100644 --- a/datafusion/proto/src/generated/prost.rs +++ b/datafusion/proto/src/generated/prost.rs @@ -1200,7 +1200,7 @@ pub struct ScalarFixedSizeBinary { pub struct ScalarValue { #[prost( oneof = "scalar_value::Value", - tags = "33, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 17, 20, 39, 21, 24, 25, 35, 36, 37, 38, 26, 27, 28, 29, 30, 31, 32, 34" + tags = "33, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 17, 18, 20, 39, 21, 24, 25, 35, 36, 37, 38, 26, 27, 28, 29, 30, 31, 32, 34" )] pub value: ::core::option::Option, } @@ -1246,6 +1246,8 @@ pub mod scalar_value { Time32Value(super::ScalarTime32Value), #[prost(message, tag = "17")] ListValue(super::ScalarListValue), + #[prost(message, tag = "18")] + FixedSizeListValue(super::ScalarListValue), #[prost(message, tag = "20")] Decimal128Value(super::Decimal128), #[prost(message, tag = "39")] diff --git a/datafusion/proto/src/logical_plan/from_proto.rs b/datafusion/proto/src/logical_plan/from_proto.rs index f14da70485ab..a34b1b7beb74 100644 --- a/datafusion/proto/src/logical_plan/from_proto.rs +++ b/datafusion/proto/src/logical_plan/from_proto.rs @@ -658,7 +658,7 @@ impl TryFrom<&protobuf::ScalarValue> for ScalarValue { Value::Float64Value(v) => Self::Float64(Some(*v)), Value::Date32Value(v) => Self::Date32(Some(*v)), // ScalarValue::List is serialized using arrow IPC format - Value::ListValue(scalar_list) => { + Value::ListValue(scalar_list) | Value::FixedSizeListValue(scalar_list) => { let protobuf::ScalarListValue { ipc_message, arrow_data, @@ -699,7 +699,11 @@ impl TryFrom<&protobuf::ScalarValue> for ScalarValue { .map_err(DataFusionError::ArrowError) .map_err(|e| e.context("Decoding ScalarValue::List Value"))?; let arr = record_batch.column(0); - Self::List(arr.to_owned()) + match value { + Value::ListValue(_) => Self::List(arr.to_owned()), + Value::FixedSizeListValue(_) => Self::FixedSizeList(arr.to_owned()), + _ => unreachable!(), + } } Value::NullValue(v) => { let null_type: DataType = v.try_into()?; diff --git a/datafusion/proto/src/logical_plan/to_proto.rs b/datafusion/proto/src/logical_plan/to_proto.rs index de81a1f4caef..433c99403e2d 100644 --- a/datafusion/proto/src/logical_plan/to_proto.rs +++ b/datafusion/proto/src/logical_plan/to_proto.rs @@ -1134,13 +1134,9 @@ impl TryFrom<&ScalarValue> for protobuf::ScalarValue { Value::LargeUtf8Value(s.to_owned()) }) } - ScalarValue::FixedSizeList(..) => Err(Error::General( - "Proto serialization error: ScalarValue::Fixedsizelist not supported" - .to_string(), - )), - // ScalarValue::List is serialized using Arrow IPC messages. - // as a single column RecordBatch - ScalarValue::List(arr) => { + // ScalarValue::List and ScalarValue::FixedSizeList are serialized using + // Arrow IPC messages as a single column RecordBatch + ScalarValue::List(arr) | ScalarValue::FixedSizeList(arr) => { // Wrap in a "field_name" column let batch = RecordBatch::try_from_iter(vec![( "field_name", @@ -1168,11 +1164,19 @@ impl TryFrom<&ScalarValue> for protobuf::ScalarValue { schema: Some(schema), }; - Ok(protobuf::ScalarValue { - value: Some(protobuf::scalar_value::Value::ListValue( - scalar_list_value, - )), - }) + match val { + ScalarValue::List(_) => Ok(protobuf::ScalarValue { + value: Some(protobuf::scalar_value::Value::ListValue( + scalar_list_value, + )), + }), + ScalarValue::FixedSizeList(_) => Ok(protobuf::ScalarValue { + value: Some(protobuf::scalar_value::Value::FixedSizeListValue( + scalar_list_value, + )), + }), + _ => unreachable!(), + } } ScalarValue::Date32(val) => { create_proto_scalar(val.as_ref(), &data_type, |s| Value::Date32Value(*s)) diff --git a/datafusion/proto/tests/cases/roundtrip_logical_plan.rs b/datafusion/proto/tests/cases/roundtrip_logical_plan.rs index 75af9d2e0acb..2d56967ecffa 100644 --- a/datafusion/proto/tests/cases/roundtrip_logical_plan.rs +++ b/datafusion/proto/tests/cases/roundtrip_logical_plan.rs @@ -19,10 +19,10 @@ use std::collections::HashMap; use std::fmt::{self, Debug, Formatter}; use std::sync::Arc; -use arrow::array::ArrayRef; +use arrow::array::{ArrayRef, FixedSizeListArray}; use arrow::datatypes::{ - DataType, Field, Fields, IntervalDayTimeType, IntervalMonthDayNanoType, IntervalUnit, - Schema, SchemaRef, TimeUnit, UnionFields, UnionMode, + DataType, Field, Fields, Int32Type, IntervalDayTimeType, IntervalMonthDayNanoType, + IntervalUnit, Schema, SchemaRef, TimeUnit, UnionFields, UnionMode, }; use prost::Message; @@ -690,6 +690,14 @@ fn round_trip_scalar_values() { ], &DataType::List(new_arc_field("item", DataType::Float32, true)), )), + ScalarValue::FixedSizeList(Arc::new(FixedSizeListArray::from_iter_primitive::< + Int32Type, + _, + _, + >( + vec![Some(vec![Some(1), Some(2), Some(3)])], + 3, + ))), ScalarValue::Dictionary( Box::new(DataType::Int32), Box::new(ScalarValue::Utf8(Some("foo".into()))), From 49614333e29bff2ff6ce296ac0db341f19adb64e Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Fri, 17 Nov 2023 14:03:00 -0500 Subject: [PATCH 278/572] Update prost-build requirement from =0.12.1 to =0.12.2 (#8244) Updates the requirements on [prost-build](https://github.com/tokio-rs/prost) to permit the latest version. - [Release notes](https://github.com/tokio-rs/prost/releases) - [Commits](https://github.com/tokio-rs/prost/compare/v0.12.1...v0.12.2) --- updated-dependencies: - dependency-name: prost-build dependency-type: direct:production ... Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- datafusion/proto/gen/Cargo.toml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/datafusion/proto/gen/Cargo.toml b/datafusion/proto/gen/Cargo.toml index 37c49666d3d7..f58357c6c5d9 100644 --- a/datafusion/proto/gen/Cargo.toml +++ b/datafusion/proto/gen/Cargo.toml @@ -32,4 +32,4 @@ publish = false [dependencies] # Pin these dependencies so that the generated output is deterministic pbjson-build = "=0.6.2" -prost-build = "=0.12.1" +prost-build = "=0.12.2" From c14a765ac286ed5a70983c1c318b7e411aabb8d1 Mon Sep 17 00:00:00 2001 From: Alex Huang Date: Fri, 17 Nov 2023 20:04:48 +0100 Subject: [PATCH 279/572] Minor: Port tests in `displayable.rs` to sqllogictest (#8246) * remove test file * fix: update mod --- datafusion/core/tests/sql/displayable.rs | 57 ------------------------ datafusion/core/tests/sql/mod.rs | 1 - 2 files changed, 58 deletions(-) delete mode 100644 datafusion/core/tests/sql/displayable.rs diff --git a/datafusion/core/tests/sql/displayable.rs b/datafusion/core/tests/sql/displayable.rs deleted file mode 100644 index 3255d514c5e4..000000000000 --- a/datafusion/core/tests/sql/displayable.rs +++ /dev/null @@ -1,57 +0,0 @@ -// Licensed to the Apache Software Foundation (ASF) under one -// or more contributor license agreements. See the NOTICE file -// distributed with this work for additional information -// regarding copyright ownership. The ASF licenses this file -// to you under the Apache License, Version 2.0 (the -// "License"); you may not use this file except in compliance -// with the License. You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, -// software distributed under the License is distributed on an -// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -// KIND, either express or implied. See the License for the -// specific language governing permissions and limitations -// under the License. - -use object_store::path::Path; - -use datafusion::prelude::*; -use datafusion_physical_plan::displayable; - -#[tokio::test] -async fn teset_displayable() { - // Hard code target_partitions as it appears in the RepartitionExec output - let config = SessionConfig::new().with_target_partitions(3); - let ctx = SessionContext::new_with_config(config); - - // register the a table - ctx.register_csv("example", "tests/data/example.csv", CsvReadOptions::new()) - .await - .unwrap(); - - // create a plan to run a SQL query - let dataframe = ctx.sql("SELECT a FROM example WHERE a < 5").await.unwrap(); - let physical_plan = dataframe.create_physical_plan().await.unwrap(); - - // Format using display string in verbose mode - let displayable_plan = displayable(physical_plan.as_ref()); - let plan_string = format!("{}", displayable_plan.indent(true)); - - let working_directory = std::env::current_dir().unwrap(); - let normalized = Path::from_filesystem_path(working_directory).unwrap(); - let plan_string = plan_string.replace(normalized.as_ref(), "WORKING_DIR"); - - assert_eq!("CoalesceBatchesExec: target_batch_size=8192\ - \n FilterExec: a@0 < 5\ - \n RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=1\ - \n CsvExec: file_groups={1 group: [[WORKING_DIR/tests/data/example.csv]]}, projection=[a], has_header=true", - plan_string.trim()); - - let one_line = format!("{}", displayable_plan.one_line()); - assert_eq!( - "CoalesceBatchesExec: target_batch_size=8192", - one_line.trim() - ); -} diff --git a/datafusion/core/tests/sql/mod.rs b/datafusion/core/tests/sql/mod.rs index 40a9e627a72a..1d58bba876f1 100644 --- a/datafusion/core/tests/sql/mod.rs +++ b/datafusion/core/tests/sql/mod.rs @@ -76,7 +76,6 @@ pub mod aggregates; pub mod create_drop; pub mod csv_files; pub mod describe; -pub mod displayable; pub mod explain_analyze; pub mod expr; pub mod group_by; From a2b9ab82f8b9905e538f1b5ff27345998ab98fb9 Mon Sep 17 00:00:00 2001 From: Andrew Lamb Date: Fri, 17 Nov 2023 14:15:17 -0500 Subject: [PATCH 280/572] Minor: add `with_estimated_selectivity ` to Precision (#8177) * Minor: add apply_filter to Precision * fix: use inexact * Rename to with_estimated_selectivity --- datafusion/common/src/stats.rs | 9 +++++++++ datafusion/physical-plan/src/filter.rs | 25 ++++++++----------------- 2 files changed, 17 insertions(+), 17 deletions(-) diff --git a/datafusion/common/src/stats.rs b/datafusion/common/src/stats.rs index 1c7a4fd4d553..7ad8992ca9ae 100644 --- a/datafusion/common/src/stats.rs +++ b/datafusion/common/src/stats.rs @@ -151,6 +151,15 @@ impl Precision { (_, _) => Precision::Absent, } } + + /// Return the estimate of applying a filter with estimated selectivity + /// `selectivity` to this Precision. A selectivity of `1.0` means that all + /// rows are selected. A selectivity of `0.5` means half the rows are + /// selected. Will always return inexact statistics. + pub fn with_estimated_selectivity(self, selectivity: f64) -> Self { + self.map(|v| ((v as f64 * selectivity).ceil()) as usize) + .to_inexact() + } } impl Precision { diff --git a/datafusion/physical-plan/src/filter.rs b/datafusion/physical-plan/src/filter.rs index 597e1d523a24..107c95eff7f1 100644 --- a/datafusion/physical-plan/src/filter.rs +++ b/datafusion/physical-plan/src/filter.rs @@ -200,15 +200,12 @@ impl ExecutionPlan for FilterExec { // assume filter selects 20% of rows if we cannot do anything smarter // tracking issue for making this configurable: // https://github.com/apache/arrow-datafusion/issues/8133 - let selectivity = 0.2_f32; - let mut stats = input_stats.into_inexact(); - if let Precision::Inexact(n) = stats.num_rows { - stats.num_rows = Precision::Inexact((selectivity * n as f32) as usize); - } - if let Precision::Inexact(n) = stats.total_byte_size { - stats.total_byte_size = - Precision::Inexact((selectivity * n as f32) as usize); - } + let selectivity = 0.2_f64; + let mut stats = input_stats.clone().into_inexact(); + stats.num_rows = stats.num_rows.with_estimated_selectivity(selectivity); + stats.total_byte_size = stats + .total_byte_size + .with_estimated_selectivity(selectivity); return Ok(stats); } @@ -222,14 +219,8 @@ impl ExecutionPlan for FilterExec { // Estimate (inexact) selectivity of predicate let selectivity = analysis_ctx.selectivity.unwrap_or(1.0); - let num_rows = match num_rows.get_value() { - Some(nr) => Precision::Inexact((*nr as f64 * selectivity).ceil() as usize), - None => Precision::Absent, - }; - let total_byte_size = match total_byte_size.get_value() { - Some(tbs) => Precision::Inexact((*tbs as f64 * selectivity).ceil() as usize), - None => Precision::Absent, - }; + let num_rows = num_rows.with_estimated_selectivity(selectivity); + let total_byte_size = total_byte_size.with_estimated_selectivity(selectivity); let column_statistics = collect_new_statistics( &input_stats.column_statistics, From bc0ed23b9ecc113cdfd9a5d68d581be9c4cbf914 Mon Sep 17 00:00:00 2001 From: L_B__ Date: Sat, 18 Nov 2023 03:34:47 +0800 Subject: [PATCH 281/572] fix: Timestamp with timezone not considered `join on` (#8150) * fix: Timestamp with timezone not considerd in * Add Test For Explain HashJoin On Timestamp with Tz --------- Co-authored-by: ackingliu --- datafusion/expr/src/utils.rs | 2 +- datafusion/sqllogictest/test_files/joins.slt | 77 ++++++++++++++++++++ 2 files changed, 78 insertions(+), 1 deletion(-) diff --git a/datafusion/expr/src/utils.rs b/datafusion/expr/src/utils.rs index 8f13bf5f61be..ff95ff10e79b 100644 --- a/datafusion/expr/src/utils.rs +++ b/datafusion/expr/src/utils.rs @@ -901,7 +901,7 @@ pub fn can_hash(data_type: &DataType) -> bool { DataType::UInt64 => true, DataType::Float32 => true, DataType::Float64 => true, - DataType::Timestamp(time_unit, None) => match time_unit { + DataType::Timestamp(time_unit, _) => match time_unit { TimeUnit::Second => true, TimeUnit::Millisecond => true, TimeUnit::Microsecond => true, diff --git a/datafusion/sqllogictest/test_files/joins.slt b/datafusion/sqllogictest/test_files/joins.slt index fa3a6cff8c4a..737b43b5a903 100644 --- a/datafusion/sqllogictest/test_files/joins.slt +++ b/datafusion/sqllogictest/test_files/joins.slt @@ -140,6 +140,17 @@ SELECT FROM test_timestamps_table_source; +# create a table of timestamps with time zone +statement ok +CREATE TABLE test_timestamps_tz_table as +SELECT + arrow_cast(ts::timestamp::bigint, 'Timestamp(Nanosecond, Some("UTC"))') as nanos, + arrow_cast(ts::timestamp::bigint / 1000, 'Timestamp(Microsecond, Some("UTC"))') as micros, + arrow_cast(ts::timestamp::bigint / 1000000, 'Timestamp(Millisecond, Some("UTC"))') as millis, + arrow_cast(ts::timestamp::bigint / 1000000000, 'Timestamp(Second, Some("UTC"))') as secs, + names +FROM + test_timestamps_table_source; statement ok @@ -2462,6 +2473,16 @@ test_timestamps_table NULL NULL NULL NULL Row 2 2021-01-01T05:11:10.432 2021-01-01T05:11:10.432 2021-01-01T05:11:10.432 2021-01-01T05:11:10 Row 3 +# show the contents of the timestamp with timezone table +query PPPPT +select * from +test_timestamps_tz_table +---- +2018-11-13T17:11:10.011375885Z 2018-11-13T17:11:10.011375Z 2018-11-13T17:11:10.011Z 2018-11-13T17:11:10Z Row 0 +2011-12-13T11:13:10.123450Z 2011-12-13T11:13:10.123450Z 2011-12-13T11:13:10.123Z 2011-12-13T11:13:10Z Row 1 +NULL NULL NULL NULL Row 2 +2021-01-01T05:11:10.432Z 2021-01-01T05:11:10.432Z 2021-01-01T05:11:10.432Z 2021-01-01T05:11:10Z Row 3 + # test timestamp join on nanos datatype query PPPPTPPPPT rowsort SELECT * FROM test_timestamps_table as t1 JOIN (SELECT * FROM test_timestamps_table ) as t2 ON t1.nanos = t2.nanos; @@ -2470,6 +2491,14 @@ SELECT * FROM test_timestamps_table as t1 JOIN (SELECT * FROM test_timestamps_ta 2018-11-13T17:11:10.011375885 2018-11-13T17:11:10.011375 2018-11-13T17:11:10.011 2018-11-13T17:11:10 Row 0 2018-11-13T17:11:10.011375885 2018-11-13T17:11:10.011375 2018-11-13T17:11:10.011 2018-11-13T17:11:10 Row 0 2021-01-01T05:11:10.432 2021-01-01T05:11:10.432 2021-01-01T05:11:10.432 2021-01-01T05:11:10 Row 3 2021-01-01T05:11:10.432 2021-01-01T05:11:10.432 2021-01-01T05:11:10.432 2021-01-01T05:11:10 Row 3 +# test timestamp with timezone join on nanos datatype +query PPPPTPPPPT rowsort +SELECT * FROM test_timestamps_tz_table as t1 JOIN (SELECT * FROM test_timestamps_tz_table ) as t2 ON t1.nanos = t2.nanos; +---- +2011-12-13T11:13:10.123450Z 2011-12-13T11:13:10.123450Z 2011-12-13T11:13:10.123Z 2011-12-13T11:13:10Z Row 1 2011-12-13T11:13:10.123450Z 2011-12-13T11:13:10.123450Z 2011-12-13T11:13:10.123Z 2011-12-13T11:13:10Z Row 1 +2018-11-13T17:11:10.011375885Z 2018-11-13T17:11:10.011375Z 2018-11-13T17:11:10.011Z 2018-11-13T17:11:10Z Row 0 2018-11-13T17:11:10.011375885Z 2018-11-13T17:11:10.011375Z 2018-11-13T17:11:10.011Z 2018-11-13T17:11:10Z Row 0 +2021-01-01T05:11:10.432Z 2021-01-01T05:11:10.432Z 2021-01-01T05:11:10.432Z 2021-01-01T05:11:10Z Row 3 2021-01-01T05:11:10.432Z 2021-01-01T05:11:10.432Z 2021-01-01T05:11:10.432Z 2021-01-01T05:11:10Z Row 3 + # test timestamp join on micros datatype query PPPPTPPPPT rowsort SELECT * FROM test_timestamps_table as t1 JOIN (SELECT * FROM test_timestamps_table ) as t2 ON t1.micros = t2.micros @@ -2478,6 +2507,14 @@ SELECT * FROM test_timestamps_table as t1 JOIN (SELECT * FROM test_timestamps_ta 2018-11-13T17:11:10.011375885 2018-11-13T17:11:10.011375 2018-11-13T17:11:10.011 2018-11-13T17:11:10 Row 0 2018-11-13T17:11:10.011375885 2018-11-13T17:11:10.011375 2018-11-13T17:11:10.011 2018-11-13T17:11:10 Row 0 2021-01-01T05:11:10.432 2021-01-01T05:11:10.432 2021-01-01T05:11:10.432 2021-01-01T05:11:10 Row 3 2021-01-01T05:11:10.432 2021-01-01T05:11:10.432 2021-01-01T05:11:10.432 2021-01-01T05:11:10 Row 3 +# test timestamp with timezone join on micros datatype +query PPPPTPPPPT rowsort +SELECT * FROM test_timestamps_tz_table as t1 JOIN (SELECT * FROM test_timestamps_tz_table ) as t2 ON t1.micros = t2.micros +---- +2011-12-13T11:13:10.123450Z 2011-12-13T11:13:10.123450Z 2011-12-13T11:13:10.123Z 2011-12-13T11:13:10Z Row 1 2011-12-13T11:13:10.123450Z 2011-12-13T11:13:10.123450Z 2011-12-13T11:13:10.123Z 2011-12-13T11:13:10Z Row 1 +2018-11-13T17:11:10.011375885Z 2018-11-13T17:11:10.011375Z 2018-11-13T17:11:10.011Z 2018-11-13T17:11:10Z Row 0 2018-11-13T17:11:10.011375885Z 2018-11-13T17:11:10.011375Z 2018-11-13T17:11:10.011Z 2018-11-13T17:11:10Z Row 0 +2021-01-01T05:11:10.432Z 2021-01-01T05:11:10.432Z 2021-01-01T05:11:10.432Z 2021-01-01T05:11:10Z Row 3 2021-01-01T05:11:10.432Z 2021-01-01T05:11:10.432Z 2021-01-01T05:11:10.432Z 2021-01-01T05:11:10Z Row 3 + # test timestamp join on millis datatype query PPPPTPPPPT rowsort SELECT * FROM test_timestamps_table as t1 JOIN (SELECT * FROM test_timestamps_table ) as t2 ON t1.millis = t2.millis @@ -2486,6 +2523,46 @@ SELECT * FROM test_timestamps_table as t1 JOIN (SELECT * FROM test_timestamps_ta 2018-11-13T17:11:10.011375885 2018-11-13T17:11:10.011375 2018-11-13T17:11:10.011 2018-11-13T17:11:10 Row 0 2018-11-13T17:11:10.011375885 2018-11-13T17:11:10.011375 2018-11-13T17:11:10.011 2018-11-13T17:11:10 Row 0 2021-01-01T05:11:10.432 2021-01-01T05:11:10.432 2021-01-01T05:11:10.432 2021-01-01T05:11:10 Row 3 2021-01-01T05:11:10.432 2021-01-01T05:11:10.432 2021-01-01T05:11:10.432 2021-01-01T05:11:10 Row 3 +# test timestamp with timezone join on millis datatype +query PPPPTPPPPT rowsort +SELECT * FROM test_timestamps_tz_table as t1 JOIN (SELECT * FROM test_timestamps_tz_table ) as t2 ON t1.millis = t2.millis +---- +2011-12-13T11:13:10.123450Z 2011-12-13T11:13:10.123450Z 2011-12-13T11:13:10.123Z 2011-12-13T11:13:10Z Row 1 2011-12-13T11:13:10.123450Z 2011-12-13T11:13:10.123450Z 2011-12-13T11:13:10.123Z 2011-12-13T11:13:10Z Row 1 +2018-11-13T17:11:10.011375885Z 2018-11-13T17:11:10.011375Z 2018-11-13T17:11:10.011Z 2018-11-13T17:11:10Z Row 0 2018-11-13T17:11:10.011375885Z 2018-11-13T17:11:10.011375Z 2018-11-13T17:11:10.011Z 2018-11-13T17:11:10Z Row 0 +2021-01-01T05:11:10.432Z 2021-01-01T05:11:10.432Z 2021-01-01T05:11:10.432Z 2021-01-01T05:11:10Z Row 3 2021-01-01T05:11:10.432Z 2021-01-01T05:11:10.432Z 2021-01-01T05:11:10.432Z 2021-01-01T05:11:10Z Row 3 + +#### +# Config setup +#### + +statement ok +set datafusion.explain.logical_plan_only = false; + +statement ok +set datafusion.optimizer.prefer_hash_join = true; + +# explain hash join on timestamp with timezone type +query TT +EXPLAIN SELECT * FROM test_timestamps_tz_table as t1 JOIN test_timestamps_tz_table as t2 ON t1.millis = t2.millis +---- +logical_plan +Inner Join: t1.millis = t2.millis +--SubqueryAlias: t1 +----TableScan: test_timestamps_tz_table projection=[nanos, micros, millis, secs, names] +--SubqueryAlias: t2 +----TableScan: test_timestamps_tz_table projection=[nanos, micros, millis, secs, names] +physical_plan +CoalesceBatchesExec: target_batch_size=2 +--HashJoinExec: mode=Partitioned, join_type=Inner, on=[(millis@2, millis@2)] +----CoalesceBatchesExec: target_batch_size=2 +------RepartitionExec: partitioning=Hash([millis@2], 2), input_partitions=2 +--------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +----------MemoryExec: partitions=1, partition_sizes=[1] +----CoalesceBatchesExec: target_batch_size=2 +------RepartitionExec: partitioning=Hash([millis@2], 2), input_partitions=2 +--------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +----------MemoryExec: partitions=1, partition_sizes=[1] + # left_join_using_2 query II SELECT t1.c1, t2.c2 FROM test_partition_table t1 JOIN test_partition_table t2 USING (c2) ORDER BY t2.c2; From db92d4e5734bab686713740a7b3976a6b61ec073 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E8=B0=AD=E5=B7=8D?= Date: Sat, 18 Nov 2023 03:36:58 +0800 Subject: [PATCH 282/572] Replace macro in array_array to remove duplicate codes (#8252) Signed-off-by: veeupup --- .../physical-expr/src/array_expressions.rs | 180 ++++-------------- 1 file changed, 37 insertions(+), 143 deletions(-) diff --git a/datafusion/physical-expr/src/array_expressions.rs b/datafusion/physical-expr/src/array_expressions.rs index ded606c3b705..c5e8b0e75c83 100644 --- a/datafusion/physical-expr/src/array_expressions.rs +++ b/datafusion/physical-expr/src/array_expressions.rs @@ -67,70 +67,6 @@ macro_rules! downcast_vec { }}; } -macro_rules! new_builder { - (BooleanBuilder, $len:expr) => { - BooleanBuilder::with_capacity($len) - }; - (StringBuilder, $len:expr) => { - StringBuilder::new() - }; - (LargeStringBuilder, $len:expr) => { - LargeStringBuilder::new() - }; - ($el:ident, $len:expr) => {{ - <$el>::with_capacity($len) - }}; -} - -/// Combines multiple arrays into a single ListArray -/// -/// $ARGS: slice of arrays, each with $ARRAY_TYPE -/// $ARRAY_TYPE: the type of the list elements -/// $BUILDER_TYPE: the type of ArrayBuilder for the list elements -/// -/// Returns: a ListArray where the elements each have the same type as -/// $ARRAY_TYPE and each element have a length of $ARGS.len() -macro_rules! array { - ($ARGS:expr, $ARRAY_TYPE:ident, $BUILDER_TYPE:ident) => {{ - let builder = new_builder!($BUILDER_TYPE, $ARGS[0].len()); - let mut builder = - ListBuilder::<$BUILDER_TYPE>::with_capacity(builder, $ARGS.len()); - - let num_rows = $ARGS[0].len(); - assert!( - $ARGS.iter().all(|a| a.len() == num_rows), - "all arguments must have the same number of rows" - ); - - // for each entry in the array - for index in 0..num_rows { - // for each column - for arg in $ARGS { - match arg.as_any().downcast_ref::<$ARRAY_TYPE>() { - // Copy the source array value into the target ListArray - Some(arr) => { - if arr.is_valid(index) { - builder.values().append_value(arr.value(index)); - } else { - builder.values().append_null(); - } - } - None => match arg.as_any().downcast_ref::() { - Some(arr) => { - for _ in 0..arr.len() { - builder.values().append_null(); - } - } - None => return internal_err!("failed to downcast"), - }, - } - } - builder.append(true); - } - Arc::new(builder.finish()) - }}; -} - /// Computes a BooleanArray indicating equality or inequality between elements in a list array and a specified element array. /// /// # Arguments @@ -389,88 +325,46 @@ fn array_array(args: &[ArrayRef], data_type: DataType) -> Result { return plan_err!("Array requires at least one argument"); } - let res = match data_type { - DataType::List(..) => { - let row_count = args[0].len(); - let column_count = args.len(); - let mut list_arrays = vec![]; - let mut list_array_lengths = vec![]; - let mut list_valid = BooleanBufferBuilder::new(row_count); - // Construct ListArray per row - for index in 0..row_count { - let mut arrays = vec![]; - let mut array_lengths = vec![]; - let mut valid = BooleanBufferBuilder::new(column_count); - for arg in args { - if arg.as_any().downcast_ref::().is_some() { - array_lengths.push(0); - valid.append(false); - } else { - let list_arr = as_list_array(arg)?; - let arr = list_arr.value(index); - array_lengths.push(arr.len()); - arrays.push(arr); - valid.append(true); - } - } - if arrays.is_empty() { - list_valid.append(false); - list_array_lengths.push(0); - } else { - let buffer = valid.finish(); - // Assume all list arrays have the same data type - let data_type = arrays[0].data_type(); - let field = Arc::new(Field::new("item", data_type.to_owned(), true)); - let elements = arrays.iter().map(|x| x.as_ref()).collect::>(); - let values = compute::concat(elements.as_slice())?; - let list_arr = ListArray::new( - field, - OffsetBuffer::from_lengths(array_lengths), - values, - Some(NullBuffer::new(buffer)), - ); - list_valid.append(true); - list_array_lengths.push(list_arr.len()); - list_arrays.push(list_arr); - } + let mut data = vec![]; + let mut total_len = 0; + for arg in args { + let arg_data = if arg.as_any().is::() { + ArrayData::new_empty(&data_type) + } else { + arg.to_data() + }; + total_len += arg_data.len(); + data.push(arg_data); + } + let mut offsets = Vec::with_capacity(total_len); + offsets.push(0); + + let capacity = Capacities::Array(total_len); + let data_ref = data.iter().collect::>(); + let mut mutable = MutableArrayData::with_capacities(data_ref, true, capacity); + + let num_rows = args[0].len(); + for row_idx in 0..num_rows { + for (arr_idx, arg) in args.iter().enumerate() { + if !arg.as_any().is::() + && !arg.is_null(row_idx) + && arg.is_valid(row_idx) + { + mutable.extend(arr_idx, row_idx, row_idx + 1); + } else { + mutable.extend_nulls(1); } - // Construct ListArray for all rows - let buffer = list_valid.finish(); - // Assume all list arrays have the same data type - let data_type = list_arrays[0].data_type(); - let field = Arc::new(Field::new("item", data_type.to_owned(), true)); - let elements = list_arrays - .iter() - .map(|x| x as &dyn Array) - .collect::>(); - let values = compute::concat(elements.as_slice())?; - let list_arr = ListArray::new( - field, - OffsetBuffer::from_lengths(list_array_lengths), - values, - Some(NullBuffer::new(buffer)), - ); - Arc::new(list_arr) - } - DataType::Utf8 => array!(args, StringArray, StringBuilder), - DataType::LargeUtf8 => array!(args, LargeStringArray, LargeStringBuilder), - DataType::Boolean => array!(args, BooleanArray, BooleanBuilder), - DataType::Float32 => array!(args, Float32Array, Float32Builder), - DataType::Float64 => array!(args, Float64Array, Float64Builder), - DataType::Int8 => array!(args, Int8Array, Int8Builder), - DataType::Int16 => array!(args, Int16Array, Int16Builder), - DataType::Int32 => array!(args, Int32Array, Int32Builder), - DataType::Int64 => array!(args, Int64Array, Int64Builder), - DataType::UInt8 => array!(args, UInt8Array, UInt8Builder), - DataType::UInt16 => array!(args, UInt16Array, UInt16Builder), - DataType::UInt32 => array!(args, UInt32Array, UInt32Builder), - DataType::UInt64 => array!(args, UInt64Array, UInt64Builder), - data_type => { - return not_impl_err!("Array is not implemented for type '{data_type:?}'.") } - }; + offsets.push(mutable.len() as i32); + } - Ok(res) + let data = mutable.freeze(); + Ok(Arc::new(ListArray::try_new( + Arc::new(Field::new("item", data_type, true)), + OffsetBuffer::new(offsets.into()), + arrow_array::make_array(data), + None, + )?)) } /// `make_array` SQL function From 91eec3f92567f979e6b104793669dfe5bf35390b Mon Sep 17 00:00:00 2001 From: Chojan Shang Date: Sat, 18 Nov 2023 03:38:06 +0800 Subject: [PATCH 283/572] Port tests in projection.rs to sqllogictest (#8240) * Port tests in projection.rs to sqllogictest Signed-off-by: Chojan Shang * Minor update Signed-off-by: Chojan Shang * Make test happy Signed-off-by: Chojan Shang * Minor update Signed-off-by: Chojan Shang * Refine tests Signed-off-by: Chojan Shang * chore: remove unused code Signed-off-by: Chojan Shang --------- Signed-off-by: Chojan Shang --- datafusion/core/tests/sql/mod.rs | 18 - datafusion/core/tests/sql/partitioned_csv.rs | 20 +- datafusion/core/tests/sql/projection.rs | 373 ------------------ .../sqllogictest/test_files/projection.slt | 235 +++++++++++ 4 files changed, 236 insertions(+), 410 deletions(-) delete mode 100644 datafusion/core/tests/sql/projection.rs create mode 100644 datafusion/sqllogictest/test_files/projection.slt diff --git a/datafusion/core/tests/sql/mod.rs b/datafusion/core/tests/sql/mod.rs index 1d58bba876f1..b04ba573afad 100644 --- a/datafusion/core/tests/sql/mod.rs +++ b/datafusion/core/tests/sql/mod.rs @@ -86,7 +86,6 @@ pub mod parquet; pub mod parquet_schema; pub mod partitioned_csv; pub mod predicates; -pub mod projection; pub mod references; pub mod repartition; pub mod select; @@ -455,23 +454,6 @@ async fn register_aggregate_csv_by_sql(ctx: &SessionContext) { ); } -async fn register_aggregate_simple_csv(ctx: &SessionContext) -> Result<()> { - // It's not possible to use aggregate_test_100 as it doesn't have enough similar values to test grouping on floats. - let schema = Arc::new(Schema::new(vec![ - Field::new("c1", DataType::Float32, false), - Field::new("c2", DataType::Float64, false), - Field::new("c3", DataType::Boolean, false), - ])); - - ctx.register_csv( - "aggregate_simple", - "tests/data/aggregate_simple.csv", - CsvReadOptions::new().schema(&schema), - ) - .await?; - Ok(()) -} - async fn register_aggregate_csv(ctx: &SessionContext) -> Result<()> { let testdata = datafusion::test_util::arrow_test_data(); let schema = test_util::aggr_test_schema(); diff --git a/datafusion/core/tests/sql/partitioned_csv.rs b/datafusion/core/tests/sql/partitioned_csv.rs index d5a1c2f0b4f8..b77557a66cd8 100644 --- a/datafusion/core/tests/sql/partitioned_csv.rs +++ b/datafusion/core/tests/sql/partitioned_csv.rs @@ -19,31 +19,13 @@ use std::{io::Write, sync::Arc}; -use arrow::{ - datatypes::{DataType, Field, Schema, SchemaRef}, - record_batch::RecordBatch, -}; +use arrow::datatypes::{DataType, Field, Schema, SchemaRef}; use datafusion::{ error::Result, prelude::{CsvReadOptions, SessionConfig, SessionContext}, }; use tempfile::TempDir; -/// Execute SQL and return results -async fn plan_and_collect( - ctx: &mut SessionContext, - sql: &str, -) -> Result> { - ctx.sql(sql).await?.collect().await -} - -/// Execute SQL and return results -pub async fn execute(sql: &str, partition_count: usize) -> Result> { - let tmp_dir = TempDir::new()?; - let mut ctx = create_ctx(&tmp_dir, partition_count).await?; - plan_and_collect(&mut ctx, sql).await -} - /// Generate CSV partitions within the supplied directory fn populate_csv_partitions( tmp_dir: &TempDir, diff --git a/datafusion/core/tests/sql/projection.rs b/datafusion/core/tests/sql/projection.rs deleted file mode 100644 index b31cb34f5210..000000000000 --- a/datafusion/core/tests/sql/projection.rs +++ /dev/null @@ -1,373 +0,0 @@ -// Licensed to the Apache Software Foundation (ASF) under one -// or more contributor license agreements. See the NOTICE file -// distributed with this work for additional information -// regarding copyright ownership. The ASF licenses this file -// to you under the Apache License, Version 2.0 (the -// "License"); you may not use this file except in compliance -// with the License. You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, -// software distributed under the License is distributed on an -// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -// KIND, either express or implied. See the License for the -// specific language governing permissions and limitations -// under the License. - -use datafusion::datasource::provider_as_source; -use datafusion::test_util::scan_empty; -use datafusion_expr::{when, LogicalPlanBuilder, UNNAMED_TABLE}; -use tempfile::TempDir; - -use super::*; - -#[tokio::test] -async fn projection_same_fields() -> Result<()> { - let ctx = SessionContext::new(); - - let sql = "select (1+1) as a from (select 1 as a) as b;"; - let actual = execute_to_batches(&ctx, sql).await; - - #[rustfmt::skip] - let expected = ["+---+", - "| a |", - "+---+", - "| 2 |", - "+---+"]; - assert_batches_eq!(expected, &actual); - - Ok(()) -} - -#[tokio::test] -async fn projection_type_alias() -> Result<()> { - let ctx = SessionContext::new(); - register_aggregate_simple_csv(&ctx).await?; - - // Query that aliases one column to the name of a different column - // that also has a different type (c1 == float32, c3 == boolean) - let sql = "SELECT c1 as c3 FROM aggregate_simple ORDER BY c3 LIMIT 2"; - let actual = execute_to_batches(&ctx, sql).await; - - let expected = [ - "+---------+", - "| c3 |", - "+---------+", - "| 0.00001 |", - "| 0.00002 |", - "+---------+", - ]; - assert_batches_eq!(expected, &actual); - - Ok(()) -} - -#[tokio::test] -async fn csv_query_group_by_avg_with_projection() -> Result<()> { - let ctx = SessionContext::new(); - register_aggregate_csv(&ctx).await?; - let sql = "SELECT avg(c12), c1 FROM aggregate_test_100 GROUP BY c1"; - let actual = execute_to_batches(&ctx, sql).await; - let expected = [ - "+-----------------------------+----+", - "| AVG(aggregate_test_100.c12) | c1 |", - "+-----------------------------+----+", - "| 0.41040709263815384 | b |", - "| 0.48600669271341534 | e |", - "| 0.48754517466109415 | a |", - "| 0.48855379387549824 | d |", - "| 0.6600456536439784 | c |", - "+-----------------------------+----+", - ]; - assert_batches_sorted_eq!(expected, &actual); - Ok(()) -} - -#[tokio::test] -async fn parallel_projection() -> Result<()> { - let partition_count = 4; - let results = - partitioned_csv::execute("SELECT c1, c2 FROM test", partition_count).await?; - - let expected = vec![ - "+----+----+", - "| c1 | c2 |", - "+----+----+", - "| 3 | 1 |", - "| 3 | 2 |", - "| 3 | 3 |", - "| 3 | 4 |", - "| 3 | 5 |", - "| 3 | 6 |", - "| 3 | 7 |", - "| 3 | 8 |", - "| 3 | 9 |", - "| 3 | 10 |", - "| 2 | 1 |", - "| 2 | 2 |", - "| 2 | 3 |", - "| 2 | 4 |", - "| 2 | 5 |", - "| 2 | 6 |", - "| 2 | 7 |", - "| 2 | 8 |", - "| 2 | 9 |", - "| 2 | 10 |", - "| 1 | 1 |", - "| 1 | 2 |", - "| 1 | 3 |", - "| 1 | 4 |", - "| 1 | 5 |", - "| 1 | 6 |", - "| 1 | 7 |", - "| 1 | 8 |", - "| 1 | 9 |", - "| 1 | 10 |", - "| 0 | 1 |", - "| 0 | 2 |", - "| 0 | 3 |", - "| 0 | 4 |", - "| 0 | 5 |", - "| 0 | 6 |", - "| 0 | 7 |", - "| 0 | 8 |", - "| 0 | 9 |", - "| 0 | 10 |", - "+----+----+", - ]; - assert_batches_sorted_eq!(expected, &results); - - Ok(()) -} - -#[tokio::test] -async fn subquery_alias_case_insensitive() -> Result<()> { - let partition_count = 1; - let results = - partitioned_csv::execute("SELECT V1.c1, v1.C2 FROM (SELECT test.C1, TEST.c2 FROM test) V1 ORDER BY v1.c1, V1.C2 LIMIT 1", partition_count).await?; - - let expected = [ - "+----+----+", - "| c1 | c2 |", - "+----+----+", - "| 0 | 1 |", - "+----+----+", - ]; - assert_batches_sorted_eq!(expected, &results); - - Ok(()) -} - -#[tokio::test] -async fn projection_on_table_scan() -> Result<()> { - let tmp_dir = TempDir::new()?; - let partition_count = 4; - let ctx = partitioned_csv::create_ctx(&tmp_dir, partition_count).await?; - - let table = ctx.table("test").await?; - let logical_plan = LogicalPlanBuilder::from(table.into_optimized_plan()?) - .project(vec![col("c2")])? - .build()?; - - let state = ctx.state(); - let optimized_plan = state.optimize(&logical_plan)?; - match &optimized_plan { - LogicalPlan::TableScan(TableScan { - source, - projected_schema, - .. - }) => { - assert_eq!(source.schema().fields().len(), 3); - assert_eq!(projected_schema.fields().len(), 1); - } - _ => panic!("input to projection should be TableScan"), - } - - let expected = "TableScan: test projection=[c2]"; - assert_eq!(format!("{optimized_plan:?}"), expected); - - let physical_plan = state.create_physical_plan(&optimized_plan).await?; - - assert_eq!(1, physical_plan.schema().fields().len()); - assert_eq!("c2", physical_plan.schema().field(0).name().as_str()); - let batches = collect(physical_plan, state.task_ctx()).await?; - assert_eq!(40, batches.iter().map(|x| x.num_rows()).sum::()); - - Ok(()) -} - -#[tokio::test] -async fn preserve_nullability_on_projection() -> Result<()> { - let tmp_dir = TempDir::new()?; - let ctx = partitioned_csv::create_ctx(&tmp_dir, 1).await?; - - let schema: Schema = ctx.table("test").await.unwrap().schema().clone().into(); - assert!(!schema.field_with_name("c1")?.is_nullable()); - - let plan = scan_empty(None, &schema, None)? - .project(vec![col("c1")])? - .build()?; - - let dataframe = DataFrame::new(ctx.state(), plan); - let physical_plan = dataframe.create_physical_plan().await?; - assert!(!physical_plan.schema().field_with_name("c1")?.is_nullable()); - Ok(()) -} - -#[tokio::test] -async fn project_cast_dictionary() { - let ctx = SessionContext::new(); - - let host: DictionaryArray = vec![Some("host1"), None, Some("host2")] - .into_iter() - .collect(); - - let batch = RecordBatch::try_from_iter(vec![("host", Arc::new(host) as _)]).unwrap(); - - let t = MemTable::try_new(batch.schema(), vec![vec![batch]]).unwrap(); - - // Note that `host` is a dictionary array but `lit("")` is a DataType::Utf8 that needs to be cast - let expr = when(col("host").is_null(), lit("")) - .otherwise(col("host")) - .unwrap(); - - let projection = None; - let builder = LogicalPlanBuilder::scan( - "cpu_load_short", - provider_as_source(Arc::new(t)), - projection, - ) - .unwrap(); - - let logical_plan = builder.project(vec![expr]).unwrap().build().unwrap(); - let df = DataFrame::new(ctx.state(), logical_plan); - let actual = df.collect().await.unwrap(); - - let expected = ["+----------------------------------------------------------------------------------+", - "| CASE WHEN cpu_load_short.host IS NULL THEN Utf8(\"\") ELSE cpu_load_short.host END |", - "+----------------------------------------------------------------------------------+", - "| host1 |", - "| |", - "| host2 |", - "+----------------------------------------------------------------------------------+"]; - assert_batches_eq!(expected, &actual); -} - -#[tokio::test] -async fn projection_on_memory_scan() -> Result<()> { - let schema = Schema::new(vec![ - Field::new("a", DataType::Int32, false), - Field::new("b", DataType::Int32, false), - Field::new("c", DataType::Int32, false), - ]); - let schema = SchemaRef::new(schema); - - let partitions = vec![vec![RecordBatch::try_new( - schema.clone(), - vec![ - Arc::new(Int32Array::from(vec![1, 10, 10, 100])), - Arc::new(Int32Array::from(vec![2, 12, 12, 120])), - Arc::new(Int32Array::from(vec![3, 12, 12, 120])), - ], - )?]]; - - let provider = Arc::new(MemTable::try_new(schema, partitions)?); - let plan = - LogicalPlanBuilder::scan(UNNAMED_TABLE, provider_as_source(provider), None)? - .project(vec![col("b")])? - .build()?; - assert_fields_eq(&plan, vec!["b"]); - - let ctx = SessionContext::new(); - let state = ctx.state(); - let optimized_plan = state.optimize(&plan)?; - match &optimized_plan { - LogicalPlan::TableScan(TableScan { - source, - projected_schema, - .. - }) => { - assert_eq!(source.schema().fields().len(), 3); - assert_eq!(projected_schema.fields().len(), 1); - } - _ => panic!("input to projection should be InMemoryScan"), - } - - let expected = format!("TableScan: {UNNAMED_TABLE} projection=[b]"); - assert_eq!(format!("{optimized_plan:?}"), expected); - - let physical_plan = state.create_physical_plan(&optimized_plan).await?; - - assert_eq!(1, physical_plan.schema().fields().len()); - assert_eq!("b", physical_plan.schema().field(0).name().as_str()); - - let batches = collect(physical_plan, state.task_ctx()).await?; - assert_eq!(1, batches.len()); - assert_eq!(1, batches[0].num_columns()); - assert_eq!(4, batches[0].num_rows()); - - Ok(()) -} - -fn assert_fields_eq(plan: &LogicalPlan, expected: Vec<&str>) { - let actual: Vec = plan - .schema() - .fields() - .iter() - .map(|f| f.name().clone()) - .collect(); - assert_eq!(actual, expected); -} - -#[tokio::test] -async fn project_column_with_same_name_as_relation() -> Result<()> { - let ctx = SessionContext::new(); - - let sql = "select a.a from (select 1 as a) as a;"; - let actual = execute_to_batches(&ctx, sql).await; - - let expected = ["+---+", "| a |", "+---+", "| 1 |", "+---+"]; - assert_batches_sorted_eq!(expected, &actual); - - Ok(()) -} - -#[tokio::test] -async fn project_column_with_filters_that_cant_pushed_down_always_false() -> Result<()> { - let ctx = SessionContext::new(); - - let sql = "select * from (select 1 as a) f where f.a=2;"; - let actual = execute_to_batches(&ctx, sql).await; - - let expected = ["++", "++"]; - assert_batches_sorted_eq!(expected, &actual); - - Ok(()) -} - -#[tokio::test] -async fn project_column_with_filters_that_cant_pushed_down_always_true() -> Result<()> { - let ctx = SessionContext::new(); - - let sql = "select * from (select 1 as a) f where f.a=1;"; - let actual = execute_to_batches(&ctx, sql).await; - - let expected = ["+---+", "| a |", "+---+", "| 1 |", "+---+"]; - assert_batches_sorted_eq!(expected, &actual); - - Ok(()) -} - -#[tokio::test] -async fn project_columns_in_memory_without_propagation() -> Result<()> { - let ctx = SessionContext::new(); - - let sql = "select column1 as a from (values (1), (2)) f where f.column1 = 2;"; - let actual = execute_to_batches(&ctx, sql).await; - - let expected = ["+---+", "| a |", "+---+", "| 2 |", "+---+"]; - assert_batches_sorted_eq!(expected, &actual); - - Ok(()) -} diff --git a/datafusion/sqllogictest/test_files/projection.slt b/datafusion/sqllogictest/test_files/projection.slt new file mode 100644 index 000000000000..b752f5644b7f --- /dev/null +++ b/datafusion/sqllogictest/test_files/projection.slt @@ -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. + +########## +## Projection Statement Tests +########## + +# prepare data +statement ok +CREATE EXTERNAL TABLE aggregate_test_100 ( + c1 VARCHAR NOT NULL, + c2 TINYINT NOT NULL, + c3 SMALLINT NOT NULL, + c4 SMALLINT, + c5 INT, + c6 BIGINT NOT NULL, + c7 SMALLINT NOT NULL, + c8 INT NOT NULL, + c9 BIGINT UNSIGNED NOT NULL, + c10 VARCHAR NOT NULL, + c11 FLOAT NOT NULL, + c12 DOUBLE NOT NULL, + c13 VARCHAR NOT NULL +) +STORED AS CSV +WITH HEADER ROW +LOCATION '../../testing/data/csv/aggregate_test_100.csv' + +statement ok +CREATE EXTERNAL TABLE aggregate_simple ( + c1 FLOAT NOT NULL, + c2 DOUBLE NOT NULL, + c3 BOOLEAN NOT NULL +) +STORED AS CSV +WITH HEADER ROW +LOCATION '../core/tests/data/aggregate_simple.csv' + +statement ok +CREATE TABLE memory_table(a INT NOT NULL, b INT NOT NULL, c INT NOT NULL) AS VALUES +(1, 2, 3), +(10, 12, 12), +(10, 12, 12), +(100, 120, 120); + +statement ok +CREATE TABLE cpu_load_short(host STRING NOT NULL) AS VALUES +('host1'), +('host2'); + +statement ok +CREATE EXTERNAL TABLE test (c1 int, c2 bigint, c3 boolean) +STORED AS CSV LOCATION '../core/tests/data/partitioned_csv'; + +statement ok +CREATE EXTERNAL TABLE test_simple (c1 int, c2 bigint, c3 boolean) +STORED AS CSV LOCATION '../core/tests/data/partitioned_csv/partition-0.csv'; + +# projection same fields +query I rowsort +select (1+1) as a from (select 1 as a) as b; +---- +2 + +# projection type alias +query R rowsort +SELECT c1 as c3 FROM aggregate_simple ORDER BY c3 LIMIT 2; +---- +0.00001 +0.00002 + +# csv query group by avg with projection +query RT rowsort +SELECT avg(c12), c1 FROM aggregate_test_100 GROUP BY c1; +---- +0.410407092638 b +0.486006692713 e +0.487545174661 a +0.488553793875 d +0.660045653644 c + +# parallel projection +query II +SELECT c1, c2 FROM test ORDER BY c1 DESC, c2 ASC +---- +3 0 +3 1 +3 2 +3 3 +3 4 +3 5 +3 6 +3 7 +3 8 +3 9 +3 10 +2 0 +2 1 +2 2 +2 3 +2 4 +2 5 +2 6 +2 7 +2 8 +2 9 +2 10 +1 0 +1 1 +1 2 +1 3 +1 4 +1 5 +1 6 +1 7 +1 8 +1 9 +1 10 +0 0 +0 1 +0 2 +0 3 +0 4 +0 5 +0 6 +0 7 +0 8 +0 9 +0 10 + +# subquery alias case insensitive +query II +SELECT V1.c1, v1.C2 FROM (SELECT test_simple.C1, TEST_SIMPLE.c2 FROM test_simple) V1 ORDER BY v1.c1, V1.C2 LIMIT 1; +---- +0 0 + +# projection on table scan +statement ok +set datafusion.explain.logical_plan_only = true + +query TT +EXPLAIN SELECT c2 FROM test; +---- +logical_plan TableScan: test projection=[c2] + +statement count 44 +select c2 from test; + +statement ok +set datafusion.explain.logical_plan_only = false + +# project cast dictionary +query T +SELECT + CASE + WHEN cpu_load_short.host IS NULL THEN '' + ELSE cpu_load_short.host + END AS host +FROM + cpu_load_short; +---- +host1 +host2 + +# projection on memory scan +query TT +explain select b from memory_table; +---- +logical_plan TableScan: memory_table projection=[b] +physical_plan MemoryExec: partitions=1, partition_sizes=[1] + +query I +select b from memory_table; +---- +2 +12 +12 +120 + +# project column with same name as relation +query I +select a.a from (select 1 as a) as a; +---- +1 + +# project column with filters that cant pushed down always false +query I +select * from (select 1 as a) f where f.a=2; +---- + + +# project column with filters that cant pushed down always true +query I +select * from (select 1 as a) f where f.a=1; +---- +1 + +# project columns in memory without propagation +query I +SELECT column1 as a from (values (1), (2)) f where f.column1 = 2; +---- +2 + +# clean data +statement ok +DROP TABLE aggregate_simple; + +statement ok +DROP TABLE aggregate_test_100; + +statement ok +DROP TABLE memory_table; + +statement ok +DROP TABLE cpu_load_short; + +statement ok +DROP TABLE test; + +statement ok +DROP TABLE test_simple; From 729376442138f85e135b28010ca2c0d018955292 Mon Sep 17 00:00:00 2001 From: Jay Zhan Date: Sat, 18 Nov 2023 03:40:47 +0800 Subject: [PATCH 284/572] Introduce `array_except` function (#8135) * squash commits for rebase Signed-off-by: jayzhan211 * address comment Signed-off-by: jayzhan211 * rename Signed-off-by: jayzhan211 * fix rebase Signed-off-by: jayzhan211 --------- Signed-off-by: jayzhan211 Co-authored-by: Andrew Lamb --- datafusion/expr/src/built_in_function.rs | 6 + datafusion/expr/src/expr_fn.rs | 6 + .../physical-expr/src/array_expressions.rs | 80 ++++++++++++- datafusion/physical-expr/src/functions.rs | 3 + datafusion/proto/proto/datafusion.proto | 5 +- datafusion/proto/src/generated/pbjson.rs | 3 + datafusion/proto/src/generated/prost.rs | 7 +- .../proto/src/logical_plan/from_proto.rs | 27 +++-- datafusion/proto/src/logical_plan/to_proto.rs | 1 + datafusion/sqllogictest/test_files/array.slt | 108 ++++++++++++++++++ docs/source/user-guide/expressions.md | 1 + .../source/user-guide/sql/scalar_functions.md | 38 ++++++ 12 files changed, 269 insertions(+), 16 deletions(-) diff --git a/datafusion/expr/src/built_in_function.rs b/datafusion/expr/src/built_in_function.rs index fc6f9c28e105..e9030ebcc00f 100644 --- a/datafusion/expr/src/built_in_function.rs +++ b/datafusion/expr/src/built_in_function.rs @@ -180,6 +180,8 @@ pub enum BuiltinScalarFunction { ArrayIntersect, /// array_union ArrayUnion, + /// array_except + ArrayExcept, /// cardinality Cardinality, /// construct an array from columns @@ -394,6 +396,7 @@ impl BuiltinScalarFunction { BuiltinScalarFunction::ArrayHas => Volatility::Immutable, BuiltinScalarFunction::ArrayDims => Volatility::Immutable, BuiltinScalarFunction::ArrayElement => Volatility::Immutable, + BuiltinScalarFunction::ArrayExcept => Volatility::Immutable, BuiltinScalarFunction::ArrayLength => Volatility::Immutable, BuiltinScalarFunction::ArrayNdims => Volatility::Immutable, BuiltinScalarFunction::ArrayPopFront => Volatility::Immutable, @@ -601,6 +604,7 @@ impl BuiltinScalarFunction { BuiltinScalarFunction::Range => { Ok(List(Arc::new(Field::new("item", Int64, true)))) } + BuiltinScalarFunction::ArrayExcept => Ok(input_expr_types[0].clone()), BuiltinScalarFunction::Cardinality => Ok(UInt64), BuiltinScalarFunction::MakeArray => match input_expr_types.len() { 0 => Ok(List(Arc::new(Field::new("item", Null, true)))), @@ -887,6 +891,7 @@ impl BuiltinScalarFunction { BuiltinScalarFunction::ArrayDims => Signature::any(1, self.volatility()), BuiltinScalarFunction::ArrayEmpty => Signature::any(1, self.volatility()), BuiltinScalarFunction::ArrayElement => Signature::any(2, self.volatility()), + BuiltinScalarFunction::ArrayExcept => Signature::any(2, self.volatility()), BuiltinScalarFunction::Flatten => Signature::any(1, self.volatility()), BuiltinScalarFunction::ArrayHasAll | BuiltinScalarFunction::ArrayHasAny @@ -1521,6 +1526,7 @@ fn aliases(func: &BuiltinScalarFunction) -> &'static [&'static str] { "list_element", "list_extract", ], + BuiltinScalarFunction::ArrayExcept => &["array_except", "list_except"], BuiltinScalarFunction::Flatten => &["flatten"], BuiltinScalarFunction::ArrayHasAll => &["array_has_all", "list_has_all"], BuiltinScalarFunction::ArrayHasAny => &["array_has_any", "list_has_any"], diff --git a/datafusion/expr/src/expr_fn.rs b/datafusion/expr/src/expr_fn.rs index 75b762804427..674d2a34df38 100644 --- a/datafusion/expr/src/expr_fn.rs +++ b/datafusion/expr/src/expr_fn.rs @@ -640,6 +640,12 @@ scalar_expr!( array element, "extracts the element with the index n from the array." ); +scalar_expr!( + ArrayExcept, + array_except, + first_array second_array, + "Returns an array of the elements that appear in the first array but not in the second." +); scalar_expr!( ArrayLength, array_length, diff --git a/datafusion/physical-expr/src/array_expressions.rs b/datafusion/physical-expr/src/array_expressions.rs index c5e8b0e75c83..8bb70c316879 100644 --- a/datafusion/physical-expr/src/array_expressions.rs +++ b/datafusion/physical-expr/src/array_expressions.rs @@ -18,6 +18,7 @@ //! Array expressions use std::any::type_name; +use std::collections::HashSet; use std::sync::Arc; use arrow::array::*; @@ -38,7 +39,6 @@ use datafusion_common::{ }; use itertools::Itertools; -use std::collections::HashSet; macro_rules! downcast_arg { ($ARG:expr, $ARRAY_TYPE:ident) => {{ @@ -523,6 +523,84 @@ pub fn array_element(args: &[ArrayRef]) -> Result { define_array_slice(list_array, key, key, true) } +fn general_except( + l: &GenericListArray, + r: &GenericListArray, + field: &FieldRef, +) -> Result> { + let converter = RowConverter::new(vec![SortField::new(l.value_type())])?; + + let l_values = l.values().to_owned(); + let r_values = r.values().to_owned(); + let l_values = converter.convert_columns(&[l_values])?; + let r_values = converter.convert_columns(&[r_values])?; + + let mut offsets = Vec::::with_capacity(l.len() + 1); + offsets.push(OffsetSize::usize_as(0)); + + let mut rows = Vec::with_capacity(l_values.num_rows()); + let mut dedup = HashSet::new(); + + for (l_w, r_w) in l.offsets().windows(2).zip(r.offsets().windows(2)) { + let l_slice = l_w[0].as_usize()..l_w[1].as_usize(); + let r_slice = r_w[0].as_usize()..r_w[1].as_usize(); + for i in r_slice { + let right_row = r_values.row(i); + dedup.insert(right_row); + } + for i in l_slice { + let left_row = l_values.row(i); + if dedup.insert(left_row) { + rows.push(left_row); + } + } + + offsets.push(OffsetSize::usize_as(rows.len())); + dedup.clear(); + } + + if let Some(values) = converter.convert_rows(rows)?.get(0) { + Ok(GenericListArray::::new( + field.to_owned(), + OffsetBuffer::new(offsets.into()), + values.to_owned(), + l.nulls().cloned(), + )) + } else { + internal_err!("array_except failed to convert rows") + } +} + +pub fn array_except(args: &[ArrayRef]) -> Result { + if args.len() != 2 { + return internal_err!("array_except needs two arguments"); + } + + let array1 = &args[0]; + let array2 = &args[1]; + + match (array1.data_type(), array2.data_type()) { + (DataType::Null, _) | (_, DataType::Null) => Ok(array1.to_owned()), + (DataType::List(field), DataType::List(_)) => { + check_datatypes("array_except", &[&array1, &array2])?; + let list1 = array1.as_list::(); + let list2 = array2.as_list::(); + let result = general_except::(list1, list2, field)?; + Ok(Arc::new(result)) + } + (DataType::LargeList(field), DataType::LargeList(_)) => { + check_datatypes("array_except", &[&array1, &array2])?; + let list1 = array1.as_list::(); + let list2 = array2.as_list::(); + let result = general_except::(list1, list2, field)?; + Ok(Arc::new(result)) + } + (dt1, dt2) => { + internal_err!("array_except got unexpected types: {dt1:?} and {dt2:?}") + } + } +} + pub fn array_slice(args: &[ArrayRef]) -> Result { let list_array = as_list_array(&args[0])?; let key = as_int64_array(&args[1])?; diff --git a/datafusion/physical-expr/src/functions.rs b/datafusion/physical-expr/src/functions.rs index b46249d26dde..5a1a68dd2127 100644 --- a/datafusion/physical-expr/src/functions.rs +++ b/datafusion/physical-expr/src/functions.rs @@ -350,6 +350,9 @@ pub fn create_physical_fun( BuiltinScalarFunction::ArrayElement => { Arc::new(|args| make_scalar_function(array_expressions::array_element)(args)) } + BuiltinScalarFunction::ArrayExcept => { + Arc::new(|args| make_scalar_function(array_expressions::array_except)(args)) + } BuiltinScalarFunction::ArrayLength => { Arc::new(|args| make_scalar_function(array_expressions::array_length)(args)) } diff --git a/datafusion/proto/proto/datafusion.proto b/datafusion/proto/proto/datafusion.proto index 8cab62acde04..ad83ea1fce49 100644 --- a/datafusion/proto/proto/datafusion.proto +++ b/datafusion/proto/proto/datafusion.proto @@ -638,8 +638,9 @@ enum ScalarFunction { ArrayUnion = 120; OverLay = 121; Range = 122; - ArrayPopFront = 123; - Levenshtein = 124; + ArrayExcept = 123; + ArrayPopFront = 124; + Levenshtein = 125; } message ScalarFunctionNode { diff --git a/datafusion/proto/src/generated/pbjson.rs b/datafusion/proto/src/generated/pbjson.rs index c50571dca0bb..016719a6001a 100644 --- a/datafusion/proto/src/generated/pbjson.rs +++ b/datafusion/proto/src/generated/pbjson.rs @@ -20937,6 +20937,7 @@ impl serde::Serialize for ScalarFunction { Self::ArrayUnion => "ArrayUnion", Self::OverLay => "OverLay", Self::Range => "Range", + Self::ArrayExcept => "ArrayExcept", Self::ArrayPopFront => "ArrayPopFront", Self::Levenshtein => "Levenshtein", }; @@ -21073,6 +21074,7 @@ impl<'de> serde::Deserialize<'de> for ScalarFunction { "ArrayUnion", "OverLay", "Range", + "ArrayExcept", "ArrayPopFront", "Levenshtein", ]; @@ -21238,6 +21240,7 @@ impl<'de> serde::Deserialize<'de> for ScalarFunction { "ArrayUnion" => Ok(ScalarFunction::ArrayUnion), "OverLay" => Ok(ScalarFunction::OverLay), "Range" => Ok(ScalarFunction::Range), + "ArrayExcept" => Ok(ScalarFunction::ArrayExcept), "ArrayPopFront" => Ok(ScalarFunction::ArrayPopFront), "Levenshtein" => Ok(ScalarFunction::Levenshtein), _ => Err(serde::de::Error::unknown_variant(value, FIELDS)), diff --git a/datafusion/proto/src/generated/prost.rs b/datafusion/proto/src/generated/prost.rs index 213be1c395c1..647f814fda8d 100644 --- a/datafusion/proto/src/generated/prost.rs +++ b/datafusion/proto/src/generated/prost.rs @@ -2571,8 +2571,9 @@ pub enum ScalarFunction { ArrayUnion = 120, OverLay = 121, Range = 122, - ArrayPopFront = 123, - Levenshtein = 124, + ArrayExcept = 123, + ArrayPopFront = 124, + Levenshtein = 125, } impl ScalarFunction { /// String value of the enum field names used in the ProtoBuf definition. @@ -2704,6 +2705,7 @@ impl ScalarFunction { ScalarFunction::ArrayUnion => "ArrayUnion", ScalarFunction::OverLay => "OverLay", ScalarFunction::Range => "Range", + ScalarFunction::ArrayExcept => "ArrayExcept", ScalarFunction::ArrayPopFront => "ArrayPopFront", ScalarFunction::Levenshtein => "Levenshtein", } @@ -2834,6 +2836,7 @@ impl ScalarFunction { "ArrayUnion" => Some(Self::ArrayUnion), "OverLay" => Some(Self::OverLay), "Range" => Some(Self::Range), + "ArrayExcept" => Some(Self::ArrayExcept), "ArrayPopFront" => Some(Self::ArrayPopFront), "Levenshtein" => Some(Self::Levenshtein), _ => None, diff --git a/datafusion/proto/src/logical_plan/from_proto.rs b/datafusion/proto/src/logical_plan/from_proto.rs index a34b1b7beb74..f59a59f3c08b 100644 --- a/datafusion/proto/src/logical_plan/from_proto.rs +++ b/datafusion/proto/src/logical_plan/from_proto.rs @@ -41,13 +41,13 @@ use datafusion_common::{ }; use datafusion_expr::{ abs, acos, acosh, array, array_append, array_concat, array_dims, array_element, - array_has, array_has_all, array_has_any, array_intersect, array_length, array_ndims, - array_position, array_positions, array_prepend, array_remove, array_remove_all, - array_remove_n, array_repeat, array_replace, array_replace_all, array_replace_n, - array_slice, array_to_string, arrow_typeof, ascii, asin, asinh, atan, atan2, atanh, - bit_length, btrim, cardinality, cbrt, ceil, character_length, chr, coalesce, - concat_expr, concat_ws_expr, cos, cosh, cot, current_date, current_time, date_bin, - date_part, date_trunc, decode, degrees, digest, encode, exp, + array_except, array_has, array_has_all, array_has_any, array_intersect, array_length, + array_ndims, array_position, array_positions, array_prepend, array_remove, + array_remove_all, array_remove_n, array_repeat, array_replace, array_replace_all, + array_replace_n, array_slice, array_to_string, arrow_typeof, ascii, asin, asinh, + atan, atan2, atanh, bit_length, btrim, cardinality, cbrt, ceil, character_length, + chr, coalesce, concat_expr, concat_ws_expr, cos, cosh, cot, current_date, + current_time, date_bin, date_part, date_trunc, decode, degrees, digest, encode, exp, expr::{self, InList, Sort, WindowFunction}, factorial, flatten, floor, from_unixtime, gcd, gen_range, isnan, iszero, lcm, left, levenshtein, ln, log, log10, log2, @@ -465,6 +465,7 @@ impl From<&protobuf::ScalarFunction> for BuiltinScalarFunction { ScalarFunction::ArrayAppend => Self::ArrayAppend, ScalarFunction::ArrayConcat => Self::ArrayConcat, ScalarFunction::ArrayEmpty => Self::ArrayEmpty, + ScalarFunction::ArrayExcept => Self::ArrayExcept, ScalarFunction::ArrayHasAll => Self::ArrayHasAll, ScalarFunction::ArrayHasAny => Self::ArrayHasAny, ScalarFunction::ArrayHas => Self::ArrayHas, @@ -1352,6 +1353,10 @@ pub fn parse_expr( .map(|expr| parse_expr(expr, registry)) .collect::, _>>()?, )), + ScalarFunction::ArrayExcept => Ok(array_except( + parse_expr(&args[0], registry)?, + parse_expr(&args[1], registry)?, + )), ScalarFunction::ArrayHasAll => Ok(array_has_all( parse_expr(&args[0], registry)?, parse_expr(&args[1], registry)?, @@ -1364,6 +1369,10 @@ pub fn parse_expr( parse_expr(&args[0], registry)?, parse_expr(&args[1], registry)?, )), + ScalarFunction::ArrayIntersect => Ok(array_intersect( + parse_expr(&args[0], registry)?, + parse_expr(&args[1], registry)?, + )), ScalarFunction::ArrayPosition => Ok(array_position( parse_expr(&args[0], registry)?, parse_expr(&args[1], registry)?, @@ -1415,10 +1424,6 @@ pub fn parse_expr( parse_expr(&args[0], registry)?, parse_expr(&args[1], registry)?, )), - ScalarFunction::ArrayIntersect => Ok(array_intersect( - parse_expr(&args[0], registry)?, - parse_expr(&args[1], registry)?, - )), ScalarFunction::Range => Ok(gen_range( args.to_owned() .iter() diff --git a/datafusion/proto/src/logical_plan/to_proto.rs b/datafusion/proto/src/logical_plan/to_proto.rs index 433c99403e2d..8bf42582360d 100644 --- a/datafusion/proto/src/logical_plan/to_proto.rs +++ b/datafusion/proto/src/logical_plan/to_proto.rs @@ -1476,6 +1476,7 @@ impl TryFrom<&BuiltinScalarFunction> for protobuf::ScalarFunction { BuiltinScalarFunction::ArrayAppend => Self::ArrayAppend, BuiltinScalarFunction::ArrayConcat => Self::ArrayConcat, BuiltinScalarFunction::ArrayEmpty => Self::ArrayEmpty, + BuiltinScalarFunction::ArrayExcept => Self::ArrayExcept, BuiltinScalarFunction::ArrayHasAll => Self::ArrayHasAll, BuiltinScalarFunction::ArrayHasAny => Self::ArrayHasAny, BuiltinScalarFunction::ArrayHas => Self::ArrayHas, diff --git a/datafusion/sqllogictest/test_files/array.slt b/datafusion/sqllogictest/test_files/array.slt index 99ed94883629..61f190e7baf6 100644 --- a/datafusion/sqllogictest/test_files/array.slt +++ b/datafusion/sqllogictest/test_files/array.slt @@ -2734,6 +2734,114 @@ select generate_series(5), ---- [0, 1, 2, 3, 4] [2, 3, 4] [2, 5, 8] +## array_except + +statement ok +CREATE TABLE array_except_table +AS VALUES + ([1, 2, 2, 3], [2, 3, 4]), + ([2, 3, 3], [3]), + ([3], [3, 3, 4]), + (null, [3, 4]), + ([1, 2], null), + (null, null) +; + +query ? +select array_except(column1, column2) from array_except_table; +---- +[1] +[2] +[] +NULL +[1, 2] +NULL + +statement ok +drop table array_except_table; + +statement ok +CREATE TABLE array_except_nested_list_table +AS VALUES + ([[1, 2], [3]], [[2], [3], [4, 5]]), + ([[1, 2], [3]], [[2], [1, 2]]), + ([[1, 2], [3]], null), + (null, [[1], [2, 3], [4, 5, 6]]), + ([[1], [2, 3], [4, 5, 6]], [[2, 3], [4, 5, 6], [1]]) +; + +query ? +select array_except(column1, column2) from array_except_nested_list_table; +---- +[[1, 2]] +[[3]] +[[1, 2], [3]] +NULL +[] + +statement ok +drop table array_except_nested_list_table; + +statement ok +CREATE TABLE array_except_table_float +AS VALUES + ([1.1, 2.2, 3.3], [2.2]), + ([1.1, 2.2, 3.3], [4.4]), + ([1.1, 2.2, 3.3], [3.3, 2.2, 1.1]) +; + +query ? +select array_except(column1, column2) from array_except_table_float; +---- +[1.1, 3.3] +[1.1, 2.2, 3.3] +[] + +statement ok +drop table array_except_table_float; + +statement ok +CREATE TABLE array_except_table_ut8 +AS VALUES + (['a', 'b', 'c'], ['a']), + (['a', 'bc', 'def'], ['g', 'def']), + (['a', 'bc', 'def'], null), + (null, ['a']) +; + +query ? +select array_except(column1, column2) from array_except_table_ut8; +---- +[b, c] +[a, bc] +[a, bc, def] +NULL + +statement ok +drop table array_except_table_ut8; + +statement ok +CREATE TABLE array_except_table_bool +AS VALUES + ([true, false, false], [false]), + ([true, true, true], [false]), + ([false, false, false], [true]), + ([true, false], null), + (null, [true, false]) +; + +query ? +select array_except(column1, column2) from array_except_table_bool; +---- +[true] +[true] +[false] +[true, false] +NULL + +statement ok +drop table array_except_table_bool; + ### Array operators tests diff --git a/docs/source/user-guide/expressions.md b/docs/source/user-guide/expressions.md index 191ef6cd9116..257c50dfa497 100644 --- a/docs/source/user-guide/expressions.md +++ b/docs/source/user-guide/expressions.md @@ -235,6 +235,7 @@ Unlike to some databases the math functions in Datafusion works the same way as | array_to_string(array, delimiter) | Converts each element to its text representation. `array_to_string([1, 2, 3, 4], ',') -> 1,2,3,4` | | array_intersect(array1, array2) | Returns an array of the elements in the intersection of array1 and array2. `array_intersect([1, 2, 3, 4], [5, 6, 3, 4]) -> [3, 4]` | | array_union(array1, array2) | Returns an array of the elements in the union of array1 and array2 without duplicates. `array_union([1, 2, 3, 4], [5, 6, 3, 4]) -> [1, 2, 3, 4, 5, 6]` | +| array_except(array1, array2) | Returns an array of the elements that appear in the first array but not in the second. `array_except([1, 2, 3, 4], [5, 6, 3, 4]) -> [3, 4]` | | cardinality(array) | Returns the total number of elements in the array. `cardinality([[1, 2, 3], [4, 5, 6]]) -> 6` | | make_array(value1, [value2 [, ...]]) | Returns an Arrow array using the specified input expressions. `make_array(1, 2, 3) -> [1, 2, 3]` | | range(start [, stop, step]) | Returns an Arrow array between start and stop with step. `SELECT range(2, 10, 3) -> [2, 5, 8]` | diff --git a/docs/source/user-guide/sql/scalar_functions.md b/docs/source/user-guide/sql/scalar_functions.md index f9f45a1b0a97..eda46ef8a73b 100644 --- a/docs/source/user-guide/sql/scalar_functions.md +++ b/docs/source/user-guide/sql/scalar_functions.md @@ -2307,6 +2307,44 @@ array_union(array1, array2) - list_union +### `array_except` + +Returns an array of the elements that appear in the first array but not in the second. + +``` +array_except(array1, array2) +``` + +#### Arguments + +- **array1**: Array expression. + Can be a constant, column, or function, and any combination of array operators. +- **array2**: Array expression. + Can be a constant, column, or function, and any combination of array operators. + +#### Example + +``` +❯ select array_except([1, 2, 3, 4], [5, 6, 3, 4]); ++----------------------------------------------------+ +| array_except([1, 2, 3, 4], [5, 6, 3, 4]); | ++----------------------------------------------------+ +| [1, 2] | ++----------------------------------------------------+ +❯ select array_except([1, 2, 3, 4], [3, 4, 5, 6]); ++----------------------------------------------------+ +| array_except([1, 2, 3, 4], [3, 4, 5, 6]); | ++----------------------------------------------------+ +| [3, 4] | ++----------------------------------------------------+ +``` + +--- + +#### Aliases + +- list_except + ### `cardinality` Returns the total number of elements in the array. From 1836fb23cc21628f38b248ab993339e20bdb0d9d Mon Sep 17 00:00:00 2001 From: Asura7969 <1402357969@qq.com> Date: Sat, 18 Nov 2023 03:50:06 +0800 Subject: [PATCH 285/572] Port tests in `describe.rs` to sqllogictest (#8242) * Minor: Improve the document format of JoinHashMap * Port tests in describe.rs to sqllogictest --------- Co-authored-by: Andrew Lamb --- datafusion/core/tests/sql/describe.rs | 72 ------------------- datafusion/core/tests/sql/mod.rs | 1 - .../sqllogictest/test_files/describe.slt | 24 +++++++ 3 files changed, 24 insertions(+), 73 deletions(-) delete mode 100644 datafusion/core/tests/sql/describe.rs diff --git a/datafusion/core/tests/sql/describe.rs b/datafusion/core/tests/sql/describe.rs deleted file mode 100644 index cd8e79b2c93b..000000000000 --- a/datafusion/core/tests/sql/describe.rs +++ /dev/null @@ -1,72 +0,0 @@ -// Licensed to the Apache Software Foundation (ASF) under one -// or more contributor license agreements. See the NOTICE file -// distributed with this work for additional information -// regarding copyright ownership. The ASF licenses this file -// to you under the Apache License, Version 2.0 (the -// "License"); you may not use this file except in compliance -// with the License. You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, -// software distributed under the License is distributed on an -// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -// KIND, either express or implied. See the License for the -// specific language governing permissions and limitations -// under the License. - -use datafusion::assert_batches_eq; -use datafusion::prelude::*; -use datafusion_common::test_util::parquet_test_data; - -#[tokio::test] -async fn describe_plan() { - let ctx = parquet_context().await; - - let query = "describe alltypes_tiny_pages"; - let results = ctx.sql(query).await.unwrap().collect().await.unwrap(); - - let expected = vec![ - "+-----------------+-----------------------------+-------------+", - "| column_name | data_type | is_nullable |", - "+-----------------+-----------------------------+-------------+", - "| id | Int32 | YES |", - "| bool_col | Boolean | YES |", - "| tinyint_col | Int8 | YES |", - "| smallint_col | Int16 | YES |", - "| int_col | Int32 | YES |", - "| bigint_col | Int64 | YES |", - "| float_col | Float32 | YES |", - "| double_col | Float64 | YES |", - "| date_string_col | Utf8 | YES |", - "| string_col | Utf8 | YES |", - "| timestamp_col | Timestamp(Nanosecond, None) | YES |", - "| year | Int32 | YES |", - "| month | Int32 | YES |", - "+-----------------+-----------------------------+-------------+", - ]; - - assert_batches_eq!(expected, &results); - - // also ensure we plan Describe via SessionState - let state = ctx.state(); - let plan = state.create_logical_plan(query).await.unwrap(); - let df = DataFrame::new(state, plan); - let results = df.collect().await.unwrap(); - - assert_batches_eq!(expected, &results); -} - -/// Return a SessionContext with parquet file registered -async fn parquet_context() -> SessionContext { - let ctx = SessionContext::new(); - let testdata = parquet_test_data(); - ctx.register_parquet( - "alltypes_tiny_pages", - &format!("{testdata}/alltypes_tiny_pages.parquet"), - ParquetReadOptions::default(), - ) - .await - .unwrap(); - ctx -} diff --git a/datafusion/core/tests/sql/mod.rs b/datafusion/core/tests/sql/mod.rs index b04ba573afad..6d783a503184 100644 --- a/datafusion/core/tests/sql/mod.rs +++ b/datafusion/core/tests/sql/mod.rs @@ -75,7 +75,6 @@ macro_rules! test_expression { pub mod aggregates; pub mod create_drop; pub mod csv_files; -pub mod describe; pub mod explain_analyze; pub mod expr; pub mod group_by; diff --git a/datafusion/sqllogictest/test_files/describe.slt b/datafusion/sqllogictest/test_files/describe.slt index 007aec443cbc..f94a2e453884 100644 --- a/datafusion/sqllogictest/test_files/describe.slt +++ b/datafusion/sqllogictest/test_files/describe.slt @@ -62,3 +62,27 @@ DROP TABLE aggregate_simple; statement error Error during planning: table 'datafusion.public.../core/tests/data/aggregate_simple.csv' not found DESCRIBE '../core/tests/data/aggregate_simple.csv'; + +########## +# Describe command +########## + +statement ok +CREATE EXTERNAL TABLE alltypes_tiny_pages STORED AS PARQUET LOCATION '../../parquet-testing/data/alltypes_tiny_pages.parquet'; + +query TTT +describe alltypes_tiny_pages; +---- +id Int32 YES +bool_col Boolean YES +tinyint_col Int8 YES +smallint_col Int16 YES +int_col Int32 YES +bigint_col Int64 YES +float_col Float32 YES +double_col Float64 YES +date_string_col Utf8 YES +string_col Utf8 YES +timestamp_col Timestamp(Nanosecond, None) YES +year Int32 YES +month Int32 YES From 325a3fbe7623d3df0ab64867545c4d93a0c96015 Mon Sep 17 00:00:00 2001 From: Raphael Taylor-Davies <1781103+tustvold@users.noreply.github.com> Date: Fri, 17 Nov 2023 22:14:27 +0000 Subject: [PATCH 286/572] Remove FileWriterMode and ListingTableInsertMode (#7994) (#8017) * Remove FileWriterMode Support (#7994) * Don't ignore test * Error on insert to single file * Improve DisplayAs --- .../core/src/datasource/file_format/csv.rs | 74 +--- .../core/src/datasource/file_format/json.rs | 59 +-- .../src/datasource/file_format/options.rs | 32 +- .../src/datasource/file_format/parquet.rs | 67 +--- .../src/datasource/file_format/write/mod.rs | 204 ++--------- .../file_format/write/orchestration.rs | 111 +----- datafusion/core/src/datasource/listing/mod.rs | 4 +- .../core/src/datasource/listing/table.rs | 343 +----------------- datafusion/core/src/datasource/listing/url.rs | 8 +- .../src/datasource/listing_table_factory.rs | 24 +- .../core/src/datasource/physical_plan/mod.rs | 7 +- datafusion/core/src/datasource/stream.rs | 11 +- datafusion/core/src/physical_planner.rs | 2 - datafusion/proto/proto/datafusion.proto | 9 +- datafusion/proto/src/generated/pbjson.rs | 94 ----- datafusion/proto/src/generated/prost.rs | 31 -- .../proto/src/physical_plan/from_proto.rs | 12 - .../proto/src/physical_plan/to_proto.rs | 13 - .../tests/cases/roundtrip_physical_plan.rs | 2 - datafusion/sqllogictest/test_files/copy.slt | 2 +- datafusion/sqllogictest/test_files/errors.slt | 2 +- .../sqllogictest/test_files/explain.slt | 4 +- datafusion/sqllogictest/test_files/insert.slt | 2 +- .../test_files/insert_to_external.slt | 24 +- datafusion/sqllogictest/test_files/joins.slt | 1 - .../sqllogictest/test_files/options.slt | 4 +- datafusion/sqllogictest/test_files/order.slt | 2 +- .../sqllogictest/test_files/predicates.slt | 1 + .../sqllogictest/test_files/set_variable.slt | 2 +- datafusion/sqllogictest/test_files/update.slt | 2 +- 30 files changed, 127 insertions(+), 1026 deletions(-) diff --git a/datafusion/core/src/datasource/file_format/csv.rs b/datafusion/core/src/datasource/file_format/csv.rs index 5f2084bc80a8..684f416f771a 100644 --- a/datafusion/core/src/datasource/file_format/csv.rs +++ b/datafusion/core/src/datasource/file_format/csv.rs @@ -34,10 +34,10 @@ use futures::stream::BoxStream; use futures::{pin_mut, Stream, StreamExt, TryStreamExt}; use object_store::{delimited::newline_delimited_stream, ObjectMeta, ObjectStore}; -use super::write::orchestration::{stateless_append_all, stateless_multipart_put}; +use super::write::orchestration::stateless_multipart_put; use super::{FileFormat, DEFAULT_SCHEMA_INFER_MAX_RECORD}; use crate::datasource::file_format::file_compression_type::FileCompressionType; -use crate::datasource::file_format::write::{BatchSerializer, FileWriterMode}; +use crate::datasource::file_format::write::BatchSerializer; use crate::datasource::physical_plan::{ CsvExec, FileGroupDisplay, FileScanConfig, FileSinkConfig, }; @@ -465,11 +465,7 @@ impl DisplayAs for CsvSink { fn fmt_as(&self, t: DisplayFormatType, f: &mut fmt::Formatter<'_>) -> fmt::Result { match t { DisplayFormatType::Default | DisplayFormatType::Verbose => { - write!( - f, - "CsvSink(writer_mode={:?}, file_groups=", - self.config.writer_mode - )?; + write!(f, "CsvSink(file_groups=",)?; FileGroupDisplay(&self.config.file_groups).fmt_as(t, f)?; write!(f, ")") } @@ -481,55 +477,6 @@ impl CsvSink { fn new(config: FileSinkConfig) -> Self { Self { config } } - - async fn append_all( - &self, - data: SendableRecordBatchStream, - context: &Arc, - ) -> Result { - if !self.config.table_partition_cols.is_empty() { - return Err(DataFusionError::NotImplemented("Inserting in append mode to hive style partitioned tables is not supported".into())); - } - let writer_options = self.config.file_type_writer_options.try_into_csv()?; - let (builder, compression) = - (&writer_options.writer_options, &writer_options.compression); - let compression = FileCompressionType::from(*compression); - - let object_store = context - .runtime_env() - .object_store(&self.config.object_store_url)?; - let file_groups = &self.config.file_groups; - - let builder_clone = builder.clone(); - let options_clone = writer_options.clone(); - let get_serializer = move |file_size| { - let inner_clone = builder_clone.clone(); - // In append mode, consider has_header flag only when file is empty (at the start). - // For other modes, use has_header flag as is. - let serializer: Box = Box::new(if file_size > 0 { - CsvSerializer::new() - .with_builder(inner_clone) - .with_header(false) - } else { - CsvSerializer::new() - .with_builder(inner_clone) - .with_header(options_clone.writer_options.header()) - }); - serializer - }; - - stateless_append_all( - data, - context, - object_store, - file_groups, - self.config.unbounded_input, - compression, - Box::new(get_serializer), - ) - .await - } - async fn multipartput_all( &self, data: SendableRecordBatchStream, @@ -577,19 +524,8 @@ impl DataSink for CsvSink { data: SendableRecordBatchStream, context: &Arc, ) -> Result { - match self.config.writer_mode { - FileWriterMode::Append => { - let total_count = self.append_all(data, context).await?; - Ok(total_count) - } - FileWriterMode::PutMultipart => { - let total_count = self.multipartput_all(data, context).await?; - Ok(total_count) - } - FileWriterMode::Put => { - return not_impl_err!("FileWriterMode::Put is not supported yet!") - } - } + let total_count = self.multipartput_all(data, context).await?; + Ok(total_count) } } diff --git a/datafusion/core/src/datasource/file_format/json.rs b/datafusion/core/src/datasource/file_format/json.rs index 8d62d0a858ac..9893a1db45de 100644 --- a/datafusion/core/src/datasource/file_format/json.rs +++ b/datafusion/core/src/datasource/file_format/json.rs @@ -45,10 +45,10 @@ use crate::physical_plan::insert::FileSinkExec; use crate::physical_plan::SendableRecordBatchStream; use crate::physical_plan::{DisplayAs, DisplayFormatType, Statistics}; -use super::write::orchestration::{stateless_append_all, stateless_multipart_put}; +use super::write::orchestration::stateless_multipart_put; use crate::datasource::file_format::file_compression_type::FileCompressionType; -use crate::datasource::file_format::write::{BatchSerializer, FileWriterMode}; +use crate::datasource::file_format::write::BatchSerializer; use crate::datasource::file_format::DEFAULT_SCHEMA_INFER_MAX_RECORD; use crate::datasource::physical_plan::{FileSinkConfig, NdJsonExec}; use crate::error::Result; @@ -245,11 +245,7 @@ impl DisplayAs for JsonSink { fn fmt_as(&self, t: DisplayFormatType, f: &mut fmt::Formatter<'_>) -> fmt::Result { match t { DisplayFormatType::Default | DisplayFormatType::Verbose => { - write!( - f, - "JsonSink(writer_mode={:?}, file_groups=", - self.config.writer_mode - )?; + write!(f, "JsonSink(file_groups=",)?; FileGroupDisplay(&self.config.file_groups).fmt_as(t, f)?; write!(f, ")") } @@ -268,40 +264,6 @@ impl JsonSink { &self.config } - async fn append_all( - &self, - data: SendableRecordBatchStream, - context: &Arc, - ) -> Result { - if !self.config.table_partition_cols.is_empty() { - return Err(DataFusionError::NotImplemented("Inserting in append mode to hive style partitioned tables is not supported".into())); - } - - let writer_options = self.config.file_type_writer_options.try_into_json()?; - let compression = &writer_options.compression; - - let object_store = context - .runtime_env() - .object_store(&self.config.object_store_url)?; - let file_groups = &self.config.file_groups; - - let get_serializer = move |_| { - let serializer: Box = Box::new(JsonSerializer::new()); - serializer - }; - - stateless_append_all( - data, - context, - object_store, - file_groups, - self.config.unbounded_input, - (*compression).into(), - Box::new(get_serializer), - ) - .await - } - async fn multipartput_all( &self, data: SendableRecordBatchStream, @@ -342,19 +304,8 @@ impl DataSink for JsonSink { data: SendableRecordBatchStream, context: &Arc, ) -> Result { - match self.config.writer_mode { - FileWriterMode::Append => { - let total_count = self.append_all(data, context).await?; - Ok(total_count) - } - FileWriterMode::PutMultipart => { - let total_count = self.multipartput_all(data, context).await?; - Ok(total_count) - } - FileWriterMode::Put => { - return not_impl_err!("FileWriterMode::Put is not supported yet!") - } - } + let total_count = self.multipartput_all(data, context).await?; + Ok(total_count) } } diff --git a/datafusion/core/src/datasource/file_format/options.rs b/datafusion/core/src/datasource/file_format/options.rs index 41a70e6d2f8f..4c7557a4a9c0 100644 --- a/datafusion/core/src/datasource/file_format/options.rs +++ b/datafusion/core/src/datasource/file_format/options.rs @@ -28,7 +28,7 @@ use crate::datasource::file_format::file_compression_type::FileCompressionType; #[cfg(feature = "parquet")] use crate::datasource::file_format::parquet::ParquetFormat; use crate::datasource::file_format::DEFAULT_SCHEMA_INFER_MAX_RECORD; -use crate::datasource::listing::{ListingTableInsertMode, ListingTableUrl}; +use crate::datasource::listing::ListingTableUrl; use crate::datasource::{ file_format::{avro::AvroFormat, csv::CsvFormat, json::JsonFormat}, listing::ListingOptions, @@ -76,8 +76,6 @@ pub struct CsvReadOptions<'a> { pub infinite: bool, /// Indicates how the file is sorted pub file_sort_order: Vec>, - /// Setting controls how inserts to this file should be handled - pub insert_mode: ListingTableInsertMode, } impl<'a> Default for CsvReadOptions<'a> { @@ -101,7 +99,6 @@ impl<'a> CsvReadOptions<'a> { file_compression_type: FileCompressionType::UNCOMPRESSED, infinite: false, file_sort_order: vec![], - insert_mode: ListingTableInsertMode::AppendToFile, } } @@ -184,12 +181,6 @@ impl<'a> CsvReadOptions<'a> { self.file_sort_order = file_sort_order; self } - - /// Configure how insertions to this table should be handled - pub fn insert_mode(mut self, insert_mode: ListingTableInsertMode) -> Self { - self.insert_mode = insert_mode; - self - } } /// Options that control the reading of Parquet files. @@ -219,8 +210,6 @@ pub struct ParquetReadOptions<'a> { pub schema: Option<&'a Schema>, /// Indicates how the file is sorted pub file_sort_order: Vec>, - /// Setting controls how inserts to this file should be handled - pub insert_mode: ListingTableInsertMode, } impl<'a> Default for ParquetReadOptions<'a> { @@ -232,7 +221,6 @@ impl<'a> Default for ParquetReadOptions<'a> { skip_metadata: None, schema: None, file_sort_order: vec![], - insert_mode: ListingTableInsertMode::AppendNewFiles, } } } @@ -272,12 +260,6 @@ impl<'a> ParquetReadOptions<'a> { self.file_sort_order = file_sort_order; self } - - /// Configure how insertions to this table should be handled - pub fn insert_mode(mut self, insert_mode: ListingTableInsertMode) -> Self { - self.insert_mode = insert_mode; - self - } } /// Options that control the reading of ARROW files. @@ -403,8 +385,6 @@ pub struct NdJsonReadOptions<'a> { pub infinite: bool, /// Indicates how the file is sorted pub file_sort_order: Vec>, - /// Setting controls how inserts to this file should be handled - pub insert_mode: ListingTableInsertMode, } impl<'a> Default for NdJsonReadOptions<'a> { @@ -417,7 +397,6 @@ impl<'a> Default for NdJsonReadOptions<'a> { file_compression_type: FileCompressionType::UNCOMPRESSED, infinite: false, file_sort_order: vec![], - insert_mode: ListingTableInsertMode::AppendToFile, } } } @@ -464,12 +443,6 @@ impl<'a> NdJsonReadOptions<'a> { self.file_sort_order = file_sort_order; self } - - /// Configure how insertions to this table should be handled - pub fn insert_mode(mut self, insert_mode: ListingTableInsertMode) -> Self { - self.insert_mode = insert_mode; - self - } } #[async_trait] @@ -528,7 +501,6 @@ impl ReadOptions<'_> for CsvReadOptions<'_> { .with_table_partition_cols(self.table_partition_cols.clone()) .with_file_sort_order(self.file_sort_order.clone()) .with_infinite_source(self.infinite) - .with_insert_mode(self.insert_mode.clone()) } async fn get_resolved_schema( @@ -555,7 +527,6 @@ impl ReadOptions<'_> for ParquetReadOptions<'_> { .with_target_partitions(config.target_partitions()) .with_table_partition_cols(self.table_partition_cols.clone()) .with_file_sort_order(self.file_sort_order.clone()) - .with_insert_mode(self.insert_mode.clone()) } async fn get_resolved_schema( @@ -582,7 +553,6 @@ impl ReadOptions<'_> for NdJsonReadOptions<'_> { .with_table_partition_cols(self.table_partition_cols.clone()) .with_infinite_source(self.infinite) .with_file_sort_order(self.file_sort_order.clone()) - .with_insert_mode(self.insert_mode.clone()) } async fn get_resolved_schema( diff --git a/datafusion/core/src/datasource/file_format/parquet.rs b/datafusion/core/src/datasource/file_format/parquet.rs index 2cba474e559e..c4d05adfc6bc 100644 --- a/datafusion/core/src/datasource/file_format/parquet.rs +++ b/datafusion/core/src/datasource/file_format/parquet.rs @@ -40,11 +40,12 @@ use crate::datasource::statistics::{create_max_min_accs, get_col_stats}; use arrow::datatypes::SchemaRef; use arrow::datatypes::{Fields, Schema}; use bytes::{BufMut, BytesMut}; -use datafusion_common::{exec_err, not_impl_err, plan_err, DataFusionError, FileType}; +use datafusion_common::{exec_err, not_impl_err, DataFusionError, FileType}; use datafusion_execution::TaskContext; use datafusion_physical_expr::{PhysicalExpr, PhysicalSortRequirement}; use futures::{StreamExt, TryStreamExt}; use hashbrown::HashMap; +use object_store::path::Path; use object_store::{ObjectMeta, ObjectStore}; use parquet::arrow::{ arrow_to_parquet_schema, parquet_to_arrow_schema, AsyncArrowWriter, @@ -55,7 +56,7 @@ use parquet::file::properties::WriterProperties; use parquet::file::statistics::Statistics as ParquetStatistics; use super::write::demux::start_demuxer_task; -use super::write::{create_writer, AbortableWrite, FileWriterMode}; +use super::write::{create_writer, AbortableWrite}; use super::{FileFormat, FileScanConfig}; use crate::arrow::array::{ BooleanArray, Float32Array, Float64Array, Int32Array, Int64Array, @@ -64,7 +65,7 @@ use crate::arrow::datatypes::DataType; use crate::config::ConfigOptions; use crate::datasource::physical_plan::{ - FileGroupDisplay, FileMeta, FileSinkConfig, ParquetExec, SchemaAdapter, + FileGroupDisplay, FileSinkConfig, ParquetExec, SchemaAdapter, }; use crate::error::Result; use crate::execution::context::SessionState; @@ -596,11 +597,7 @@ impl DisplayAs for ParquetSink { fn fmt_as(&self, t: DisplayFormatType, f: &mut fmt::Formatter<'_>) -> fmt::Result { match t { DisplayFormatType::Default | DisplayFormatType::Verbose => { - write!( - f, - "ParquetSink(writer_mode={:?}, file_groups=", - self.config.writer_mode - )?; + write!(f, "ParquetSink(file_groups=",)?; FileGroupDisplay(&self.config.file_groups).fmt_as(t, f)?; write!(f, ")") } @@ -642,36 +639,23 @@ impl ParquetSink { /// AsyncArrowWriters are used when individual parquet file serialization is not parallelized async fn create_async_arrow_writer( &self, - file_meta: FileMeta, + location: &Path, object_store: Arc, parquet_props: WriterProperties, ) -> Result< AsyncArrowWriter>, > { - let object = &file_meta.object_meta; - match self.config.writer_mode { - FileWriterMode::Append => { - plan_err!( - "Appending to Parquet files is not supported by the file format!" - ) - } - FileWriterMode::Put => { - not_impl_err!("FileWriterMode::Put is not implemented for ParquetSink") - } - FileWriterMode::PutMultipart => { - let (_, multipart_writer) = object_store - .put_multipart(&object.location) - .await - .map_err(DataFusionError::ObjectStore)?; - let writer = AsyncArrowWriter::try_new( - multipart_writer, - self.get_writer_schema(), - 10485760, - Some(parquet_props), - )?; - Ok(writer) - } - } + let (_, multipart_writer) = object_store + .put_multipart(location) + .await + .map_err(DataFusionError::ObjectStore)?; + let writer = AsyncArrowWriter::try_new( + multipart_writer, + self.get_writer_schema(), + 10485760, + Some(parquet_props), + )?; + Ok(writer) } } @@ -730,13 +714,7 @@ impl DataSink for ParquetSink { if !allow_single_file_parallelism { let mut writer = self .create_async_arrow_writer( - ObjectMeta { - location: path, - last_modified: chrono::offset::Utc::now(), - size: 0, - e_tag: None, - } - .into(), + &path, object_store.clone(), parquet_props.clone(), ) @@ -752,17 +730,10 @@ impl DataSink for ParquetSink { }); } else { let writer = create_writer( - FileWriterMode::PutMultipart, // Parquet files as a whole are never compressed, since they // manage compressed blocks themselves. FileCompressionType::UNCOMPRESSED, - ObjectMeta { - location: path, - last_modified: chrono::offset::Utc::now(), - size: 0, - e_tag: None, - } - .into(), + &path, object_store.clone(), ) .await?; diff --git a/datafusion/core/src/datasource/file_format/write/mod.rs b/datafusion/core/src/datasource/file_format/write/mod.rs index 770c7a49c326..cfcdbd8c464e 100644 --- a/datafusion/core/src/datasource/file_format/write/mod.rs +++ b/datafusion/core/src/datasource/file_format/write/mod.rs @@ -19,128 +19,32 @@ //! write support for the various file formats use std::io::Error; -use std::mem; use std::pin::Pin; use std::sync::Arc; use std::task::{Context, Poll}; use crate::datasource::file_format::file_compression_type::FileCompressionType; -use crate::datasource::physical_plan::FileMeta; use crate::error::Result; use arrow_array::RecordBatch; -use datafusion_common::{exec_err, DataFusionError}; +use datafusion_common::DataFusionError; use async_trait::async_trait; use bytes::Bytes; use futures::future::BoxFuture; -use futures::ready; -use futures::FutureExt; use object_store::path::Path; -use object_store::{MultipartId, ObjectMeta, ObjectStore}; +use object_store::{MultipartId, ObjectStore}; use tokio::io::AsyncWrite; pub(crate) mod demux; pub(crate) mod orchestration; -/// `AsyncPutWriter` is an object that facilitates asynchronous writing to object stores. -/// It is specifically designed for the `object_store` crate's `put` method and sends -/// whole bytes at once when the buffer is flushed. -pub struct AsyncPutWriter { - /// Object metadata - object_meta: ObjectMeta, - /// A shared reference to the object store - store: Arc, - /// A buffer that stores the bytes to be sent - current_buffer: Vec, - /// Used for async handling in flush method - inner_state: AsyncPutState, -} - -impl AsyncPutWriter { - /// Constructor for the `AsyncPutWriter` object - pub fn new(object_meta: ObjectMeta, store: Arc) -> Self { - Self { - object_meta, - store, - current_buffer: vec![], - // The writer starts out in buffering mode - inner_state: AsyncPutState::Buffer, - } - } - - /// Separate implementation function that unpins the [`AsyncPutWriter`] so - /// that partial borrows work correctly - fn poll_shutdown_inner( - &mut self, - cx: &mut Context<'_>, - ) -> Poll> { - loop { - match &mut self.inner_state { - AsyncPutState::Buffer => { - // Convert the current buffer to bytes and take ownership of it - let bytes = Bytes::from(mem::take(&mut self.current_buffer)); - // Set the inner state to Put variant with the bytes - self.inner_state = AsyncPutState::Put { bytes } - } - AsyncPutState::Put { bytes } => { - // Send the bytes to the object store's put method - return Poll::Ready( - ready!(self - .store - .put(&self.object_meta.location, bytes.clone()) - .poll_unpin(cx)) - .map_err(Error::from), - ); - } - } - } - } -} - -/// An enum that represents the inner state of AsyncPut -enum AsyncPutState { - /// Building Bytes struct in this state - Buffer, - /// Data in the buffer is being sent to the object store - Put { bytes: Bytes }, -} - -impl AsyncWrite for AsyncPutWriter { - // Define the implementation of the AsyncWrite trait for the `AsyncPutWriter` struct - fn poll_write( - mut self: Pin<&mut Self>, - _: &mut Context<'_>, - buf: &[u8], - ) -> Poll> { - // Extend the current buffer with the incoming buffer - self.current_buffer.extend_from_slice(buf); - // Return a ready poll with the length of the incoming buffer - Poll::Ready(Ok(buf.len())) - } - - fn poll_flush( - self: Pin<&mut Self>, - _: &mut Context<'_>, - ) -> Poll> { - // Return a ready poll with an empty result - Poll::Ready(Ok(())) - } - - fn poll_shutdown( - mut self: Pin<&mut Self>, - cx: &mut Context<'_>, - ) -> Poll> { - // Call the poll_shutdown_inner method to handle the actual sending of data to the object store - self.poll_shutdown_inner(cx) - } -} - /// Stores data needed during abortion of MultiPart writers +#[derive(Clone)] pub(crate) struct MultiPart { /// A shared reference to the object store store: Arc, @@ -163,45 +67,28 @@ impl MultiPart { } } -pub(crate) enum AbortMode { - Put, - Append, - MultiPart(MultiPart), -} - /// A wrapper struct with abort method and writer pub(crate) struct AbortableWrite { writer: W, - mode: AbortMode, + multipart: MultiPart, } impl AbortableWrite { /// Create a new `AbortableWrite` instance with the given writer, and write mode. - pub(crate) fn new(writer: W, mode: AbortMode) -> Self { - Self { writer, mode } + pub(crate) fn new(writer: W, multipart: MultiPart) -> Self { + Self { writer, multipart } } /// handling of abort for different write modes pub(crate) fn abort_writer(&self) -> Result>> { - match &self.mode { - AbortMode::Put => Ok(async { Ok(()) }.boxed()), - AbortMode::Append => exec_err!("Cannot abort in append mode"), - AbortMode::MultiPart(MultiPart { - store, - multipart_id, - location, - }) => { - let location = location.clone(); - let multipart_id = multipart_id.clone(); - let store = store.clone(); - Ok(Box::pin(async move { - store - .abort_multipart(&location, &multipart_id) - .await - .map_err(DataFusionError::ObjectStore) - })) - } - } + let multi = self.multipart.clone(); + Ok(Box::pin(async move { + multi + .store + .abort_multipart(&multi.location, &multi.multipart_id) + .await + .map_err(DataFusionError::ObjectStore) + })) } } @@ -229,16 +116,6 @@ impl AsyncWrite for AbortableWrite { } } -/// An enum that defines different file writer modes. -#[derive(Debug, Clone, Copy)] -pub enum FileWriterMode { - /// Data is appended to an existing file. - Append, - /// Data is written to a new file. - Put, - /// Data is written to a new file in multiple parts. - PutMultipart, -} /// A trait that defines the methods required for a RecordBatch serializer. #[async_trait] pub trait BatchSerializer: Unpin + Send { @@ -255,51 +132,16 @@ pub trait BatchSerializer: Unpin + Send { /// Returns an [`AbortableWrite`] which writes to the given object store location /// with the specified compression pub(crate) async fn create_writer( - writer_mode: FileWriterMode, file_compression_type: FileCompressionType, - file_meta: FileMeta, + location: &Path, object_store: Arc, ) -> Result>> { - let object = &file_meta.object_meta; - match writer_mode { - // If the mode is append, call the store's append method and return wrapped in - // a boxed trait object. - FileWriterMode::Append => { - let writer = object_store - .append(&object.location) - .await - .map_err(DataFusionError::ObjectStore)?; - let writer = AbortableWrite::new( - file_compression_type.convert_async_writer(writer)?, - AbortMode::Append, - ); - Ok(writer) - } - // If the mode is put, create a new AsyncPut writer and return it wrapped in - // a boxed trait object - FileWriterMode::Put => { - let writer = Box::new(AsyncPutWriter::new(object.clone(), object_store)); - let writer = AbortableWrite::new( - file_compression_type.convert_async_writer(writer)?, - AbortMode::Put, - ); - Ok(writer) - } - // If the mode is put multipart, call the store's put_multipart method and - // return the writer wrapped in a boxed trait object. - FileWriterMode::PutMultipart => { - let (multipart_id, writer) = object_store - .put_multipart(&object.location) - .await - .map_err(DataFusionError::ObjectStore)?; - Ok(AbortableWrite::new( - file_compression_type.convert_async_writer(writer)?, - AbortMode::MultiPart(MultiPart::new( - object_store, - multipart_id, - object.location.clone(), - )), - )) - } - } + let (multipart_id, writer) = object_store + .put_multipart(location) + .await + .map_err(DataFusionError::ObjectStore)?; + Ok(AbortableWrite::new( + file_compression_type.convert_async_writer(writer)?, + MultiPart::new(object_store, multipart_id, location.clone()), + )) } diff --git a/datafusion/core/src/datasource/file_format/write/orchestration.rs b/datafusion/core/src/datasource/file_format/write/orchestration.rs index f84baa9ac225..2ae6b70ed1c5 100644 --- a/datafusion/core/src/datasource/file_format/write/orchestration.rs +++ b/datafusion/core/src/datasource/file_format/write/orchestration.rs @@ -22,7 +22,6 @@ use std::sync::Arc; use crate::datasource::file_format::file_compression_type::FileCompressionType; -use crate::datasource::listing::PartitionedFile; use crate::datasource::physical_plan::FileSinkConfig; use crate::error::Result; use crate::physical_plan::SendableRecordBatchStream; @@ -34,17 +33,13 @@ use datafusion_common::DataFusionError; use bytes::Bytes; use datafusion_execution::TaskContext; -use futures::StreamExt; - -use object_store::{ObjectMeta, ObjectStore}; - use tokio::io::{AsyncWrite, AsyncWriteExt}; use tokio::sync::mpsc::{self, Receiver}; use tokio::task::{JoinHandle, JoinSet}; use tokio::try_join; use super::demux::start_demuxer_task; -use super::{create_writer, AbortableWrite, BatchSerializer, FileWriterMode}; +use super::{create_writer, AbortableWrite, BatchSerializer}; type WriterType = AbortableWrite>; type SerializerType = Box; @@ -274,21 +269,9 @@ pub(crate) async fn stateless_multipart_put( stateless_serialize_and_write_files(rx_file_bundle, tx_row_cnt, unbounded_input) .await }); - while let Some((output_location, rb_stream)) = file_stream_rx.recv().await { + while let Some((location, rb_stream)) = file_stream_rx.recv().await { let serializer = get_serializer(); - let object_meta = ObjectMeta { - location: output_location, - last_modified: chrono::offset::Utc::now(), - size: 0, - e_tag: None, - }; - let writer = create_writer( - FileWriterMode::PutMultipart, - compression, - object_meta.into(), - object_store.clone(), - ) - .await?; + let writer = create_writer(compression, &location, object_store.clone()).await?; tx_file_bundle .send((rb_stream, serializer, writer)) @@ -325,91 +308,3 @@ pub(crate) async fn stateless_multipart_put( Ok(total_count) } - -/// Orchestrates append_all for any statelessly serialized file type. Appends to all files provided -/// in a round robin fashion. -pub(crate) async fn stateless_append_all( - mut data: SendableRecordBatchStream, - context: &Arc, - object_store: Arc, - file_groups: &Vec, - unbounded_input: bool, - compression: FileCompressionType, - get_serializer: Box Box + Send>, -) -> Result { - let rb_buffer_size = &context - .session_config() - .options() - .execution - .max_buffered_batches_per_output_file; - - let (tx_file_bundle, rx_file_bundle) = tokio::sync::mpsc::channel(file_groups.len()); - let mut send_channels = vec![]; - for file_group in file_groups { - let serializer = get_serializer(file_group.object_meta.size); - - let file = file_group.clone(); - let writer = create_writer( - FileWriterMode::Append, - compression, - file.object_meta.clone().into(), - object_store.clone(), - ) - .await?; - - let (tx, rx) = tokio::sync::mpsc::channel(rb_buffer_size / 2); - send_channels.push(tx); - tx_file_bundle - .send((rx, serializer, writer)) - .await - .map_err(|_| { - DataFusionError::Internal( - "Writer receive file bundle channel closed unexpectedly!".into(), - ) - })?; - } - - let (tx_row_cnt, rx_row_cnt) = tokio::sync::oneshot::channel(); - let write_coordinater_task = tokio::spawn(async move { - stateless_serialize_and_write_files(rx_file_bundle, tx_row_cnt, unbounded_input) - .await - }); - - // Append to file groups in round robin - let mut next_file_idx = 0; - while let Some(rb) = data.next().await.transpose()? { - send_channels[next_file_idx].send(rb).await.map_err(|_| { - DataFusionError::Internal( - "Recordbatch file append stream closed unexpectedly!".into(), - ) - })?; - next_file_idx = (next_file_idx + 1) % send_channels.len(); - if unbounded_input { - tokio::task::yield_now().await; - } - } - // Signal to the write coordinater that no more files are coming - drop(tx_file_bundle); - drop(send_channels); - - let total_count = rx_row_cnt.await.map_err(|_| { - DataFusionError::Internal( - "Did not receieve row count from write coordinater".into(), - ) - })?; - - match try_join!(write_coordinater_task) { - Ok(r1) => { - r1.0?; - } - Err(e) => { - if e.is_panic() { - std::panic::resume_unwind(e.into_panic()); - } else { - unreachable!(); - } - } - } - - Ok(total_count) -} diff --git a/datafusion/core/src/datasource/listing/mod.rs b/datafusion/core/src/datasource/listing/mod.rs index 8b0f021f0277..aa2e20164b5e 100644 --- a/datafusion/core/src/datasource/listing/mod.rs +++ b/datafusion/core/src/datasource/listing/mod.rs @@ -31,9 +31,7 @@ use std::pin::Pin; use std::sync::Arc; pub use self::url::ListingTableUrl; -pub use table::{ - ListingOptions, ListingTable, ListingTableConfig, ListingTableInsertMode, -}; +pub use table::{ListingOptions, ListingTable, ListingTableConfig}; /// Stream of files get listed from object store pub type PartitionedFileStream = diff --git a/datafusion/core/src/datasource/listing/table.rs b/datafusion/core/src/datasource/listing/table.rs index c22eb58e88fa..515bc8a9e612 100644 --- a/datafusion/core/src/datasource/listing/table.rs +++ b/datafusion/core/src/datasource/listing/table.rs @@ -214,33 +214,6 @@ impl ListingTableConfig { } } -#[derive(Debug, Clone)] -///controls how new data should be inserted to a ListingTable -pub enum ListingTableInsertMode { - ///Data should be appended to an existing file - AppendToFile, - ///Data is appended as new files in existing TablePaths - AppendNewFiles, - ///Throw an error if insert into is attempted on this table - Error, -} - -impl FromStr for ListingTableInsertMode { - type Err = DataFusionError; - fn from_str(s: &str) -> Result { - let s_lower = s.to_lowercase(); - match s_lower.as_str() { - "append_to_file" => Ok(ListingTableInsertMode::AppendToFile), - "append_new_files" => Ok(ListingTableInsertMode::AppendNewFiles), - "error" => Ok(ListingTableInsertMode::Error), - _ => plan_err!( - "Unknown or unsupported insert mode {s}. Supported options are \ - append_to_file, append_new_files, and error." - ), - } - } -} - /// Options for creating a [`ListingTable`] #[derive(Clone, Debug)] pub struct ListingOptions { @@ -279,8 +252,6 @@ pub struct ListingOptions { /// In order to support infinite inputs, DataFusion may adjust query /// plans (e.g. joins) to run the given query in full pipelining mode. pub infinite_source: bool, - /// This setting controls how inserts to this table should be handled - pub insert_mode: ListingTableInsertMode, /// This setting when true indicates that the table is backed by a single file. /// Any inserts to the table may only append to this existing file. pub single_file: bool, @@ -305,7 +276,6 @@ impl ListingOptions { target_partitions: 1, file_sort_order: vec![], infinite_source: false, - insert_mode: ListingTableInsertMode::AppendToFile, single_file: false, file_type_write_options: None, } @@ -476,12 +446,6 @@ impl ListingOptions { self } - /// Configure how insertions to this table should be handled. - pub fn with_insert_mode(mut self, insert_mode: ListingTableInsertMode) -> Self { - self.insert_mode = insert_mode; - self - } - /// Configure if this table is backed by a sigle file pub fn with_single_file(mut self, single_file: bool) -> Self { self.single_file = single_file; @@ -806,6 +770,13 @@ impl TableProvider for ListingTable { } let table_path = &self.table_paths()[0]; + if !table_path.is_collection() { + return plan_err!( + "Inserting into a ListingTable backed by a single file is not supported, URL is possibly missing a trailing `/`. \ + To append to an existing file use StreamTable, e.g. by using CREATE UNBOUNDED EXTERNAL TABLE" + ); + } + // Get the object store for the table path. let store = state.runtime_env().object_store(table_path)?; @@ -820,31 +791,6 @@ impl TableProvider for ListingTable { .await?; let file_groups = file_list_stream.try_collect::>().await?; - //if we are writing a single output_partition to a table backed by a single file - //we can append to that file. Otherwise, we can write new files into the directory - //adding new files to the listing table in order to insert to the table. - let input_partitions = input.output_partitioning().partition_count(); - let writer_mode = match self.options.insert_mode { - ListingTableInsertMode::AppendToFile => { - if input_partitions > file_groups.len() { - return plan_err!( - "Cannot append {input_partitions} partitions to {} files!", - file_groups.len() - ); - } - - crate::datasource::file_format::write::FileWriterMode::Append - } - ListingTableInsertMode::AppendNewFiles => { - crate::datasource::file_format::write::FileWriterMode::PutMultipart - } - ListingTableInsertMode::Error => { - return plan_err!( - "Invalid plan attempting write to table with TableWriteMode::Error!" - ); - } - }; - let file_format = self.options().format.as_ref(); let file_type_writer_options = match &self.options().file_type_write_options { @@ -862,7 +808,6 @@ impl TableProvider for ListingTable { file_groups, output_schema: self.schema(), table_partition_cols: self.options.table_partition_cols.clone(), - writer_mode, // A plan can produce finite number of rows even if it has unbounded sources, like LIMIT // queries. Thus, we can check if the plan is streaming to ensure file sink input is // unbounded. When `unbounded_input` flag is `true` for sink, we occasionally call `yield_now` @@ -877,14 +822,6 @@ impl TableProvider for ListingTable { let unsorted: Vec> = vec![]; let order_requirements = if self.options().file_sort_order != unsorted { - if matches!( - self.options().insert_mode, - ListingTableInsertMode::AppendToFile - ) { - return plan_err!( - "Cannot insert into a sorted ListingTable with mode append!" - ); - } // Multiple sort orders in outer vec are equivalent, so we pass only the first one let ordering = self .try_create_output_ordering()? @@ -1003,7 +940,7 @@ mod tests { use crate::prelude::*; use crate::{ assert_batches_eq, - datasource::file_format::{avro::AvroFormat, file_compression_type::FileTypeExt}, + datasource::file_format::avro::AvroFormat, execution::options::ReadOptions, logical_expr::{col, lit}, test::{columns, object_store::register_test_store}, @@ -1567,17 +1504,6 @@ mod tests { Ok(()) } - #[tokio::test] - async fn test_insert_into_append_to_json_file() -> Result<()> { - helper_test_insert_into_append_to_existing_files( - FileType::JSON, - FileCompressionType::UNCOMPRESSED, - None, - ) - .await?; - Ok(()) - } - #[tokio::test] async fn test_insert_into_append_new_json_files() -> Result<()> { let mut config_map: HashMap = HashMap::new(); @@ -1596,17 +1522,6 @@ mod tests { Ok(()) } - #[tokio::test] - async fn test_insert_into_append_to_csv_file() -> Result<()> { - helper_test_insert_into_append_to_existing_files( - FileType::CSV, - FileCompressionType::UNCOMPRESSED, - None, - ) - .await?; - Ok(()) - } - #[tokio::test] async fn test_insert_into_append_new_csv_files() -> Result<()> { let mut config_map: HashMap = HashMap::new(); @@ -1663,13 +1578,8 @@ mod tests { #[tokio::test] async fn test_insert_into_sql_csv_defaults() -> Result<()> { - helper_test_insert_into_sql( - "csv", - FileCompressionType::UNCOMPRESSED, - "OPTIONS (insert_mode 'append_new_files')", - None, - ) - .await?; + helper_test_insert_into_sql("csv", FileCompressionType::UNCOMPRESSED, "", None) + .await?; Ok(()) } @@ -1678,8 +1588,7 @@ mod tests { helper_test_insert_into_sql( "csv", FileCompressionType::UNCOMPRESSED, - "WITH HEADER ROW \ - OPTIONS (insert_mode 'append_new_files')", + "WITH HEADER ROW", None, ) .await?; @@ -1688,13 +1597,8 @@ mod tests { #[tokio::test] async fn test_insert_into_sql_json_defaults() -> Result<()> { - helper_test_insert_into_sql( - "json", - FileCompressionType::UNCOMPRESSED, - "OPTIONS (insert_mode 'append_new_files')", - None, - ) - .await?; + helper_test_insert_into_sql("json", FileCompressionType::UNCOMPRESSED, "", None) + .await?; Ok(()) } @@ -1879,211 +1783,6 @@ mod tests { Ok(()) } - #[tokio::test] - async fn test_insert_into_append_to_parquet_file_fails() -> Result<()> { - let maybe_err = helper_test_insert_into_append_to_existing_files( - FileType::PARQUET, - FileCompressionType::UNCOMPRESSED, - None, - ) - .await; - let _err = - maybe_err.expect_err("Appending to existing parquet file did not fail!"); - Ok(()) - } - - fn load_empty_schema_table( - schema: SchemaRef, - temp_path: &str, - insert_mode: ListingTableInsertMode, - file_format: Arc, - ) -> Result> { - File::create(temp_path)?; - let table_path = ListingTableUrl::parse(temp_path).unwrap(); - - let listing_options = - ListingOptions::new(file_format.clone()).with_insert_mode(insert_mode); - - let config = ListingTableConfig::new(table_path) - .with_listing_options(listing_options) - .with_schema(schema); - - let table = ListingTable::try_new(config)?; - Ok(Arc::new(table)) - } - - /// Logic of testing inserting into listing table by Appending to existing files - /// is the same for all formats/options which support this. This helper allows - /// passing different options to execute the same test with different settings. - async fn helper_test_insert_into_append_to_existing_files( - file_type: FileType, - file_compression_type: FileCompressionType, - session_config_map: Option>, - ) -> Result<()> { - // Create the initial context, schema, and batch. - let session_ctx = match session_config_map { - Some(cfg) => { - let config = SessionConfig::from_string_hash_map(cfg)?; - SessionContext::new_with_config(config) - } - None => SessionContext::new(), - }; - // Create a new schema with one field called "a" of type Int32 - let schema = Arc::new(Schema::new(vec![Field::new( - "column1", - DataType::Int32, - false, - )])); - - // Create a new batch of data to insert into the table - let batch = RecordBatch::try_new( - schema.clone(), - vec![Arc::new(arrow_array::Int32Array::from(vec![1, 2, 3]))], - )?; - - // Filename with extension - let filename = format!( - "path{}", - file_type - .to_owned() - .get_ext_with_compression(file_compression_type) - .unwrap() - ); - - // Create a temporary directory and a CSV file within it. - let tmp_dir = TempDir::new()?; - let path = tmp_dir.path().join(filename); - - let file_format: Arc = match file_type { - FileType::CSV => Arc::new( - CsvFormat::default().with_file_compression_type(file_compression_type), - ), - FileType::JSON => Arc::new( - JsonFormat::default().with_file_compression_type(file_compression_type), - ), - FileType::PARQUET => Arc::new(ParquetFormat::default()), - FileType::AVRO => Arc::new(AvroFormat {}), - FileType::ARROW => Arc::new(ArrowFormat {}), - }; - - let initial_table = load_empty_schema_table( - schema.clone(), - path.to_str().unwrap(), - ListingTableInsertMode::AppendToFile, - file_format, - )?; - session_ctx.register_table("t", initial_table)?; - // Create and register the source table with the provided schema and inserted data - let source_table = Arc::new(MemTable::try_new( - schema.clone(), - vec![vec![batch.clone(), batch.clone()]], - )?); - session_ctx.register_table("source", source_table.clone())?; - // Convert the source table into a provider so that it can be used in a query - let source = provider_as_source(source_table); - // Create a table scan logical plan to read from the source table - let scan_plan = LogicalPlanBuilder::scan("source", source, None)?.build()?; - // Create an insert plan to insert the source data into the initial table - let insert_into_table = - LogicalPlanBuilder::insert_into(scan_plan, "t", &schema, false)?.build()?; - // Create a physical plan from the insert plan - let plan = session_ctx - .state() - .create_physical_plan(&insert_into_table) - .await?; - - // Execute the physical plan and collect the results - let res = collect(plan, session_ctx.task_ctx()).await?; - // Insert returns the number of rows written, in our case this would be 6. - let expected = [ - "+-------+", - "| count |", - "+-------+", - "| 6 |", - "+-------+", - ]; - - // Assert that the batches read from the file match the expected result. - assert_batches_eq!(expected, &res); - - // Read the records in the table - let batches = session_ctx.sql("select * from t").await?.collect().await?; - - // Define the expected result as a vector of strings. - let expected = [ - "+---------+", - "| column1 |", - "+---------+", - "| 1 |", - "| 2 |", - "| 3 |", - "| 1 |", - "| 2 |", - "| 3 |", - "+---------+", - ]; - - // Assert that the batches read from the file match the expected result. - assert_batches_eq!(expected, &batches); - - // Assert that only 1 file was added to the table - let num_files = tmp_dir.path().read_dir()?.count(); - assert_eq!(num_files, 1); - - // Create a physical plan from the insert plan - let plan = session_ctx - .state() - .create_physical_plan(&insert_into_table) - .await?; - - // Again, execute the physical plan and collect the results - let res = collect(plan, session_ctx.task_ctx()).await?; - // Insert returns the number of rows written, in our case this would be 6. - let expected = [ - "+-------+", - "| count |", - "+-------+", - "| 6 |", - "+-------+", - ]; - - // Assert that the batches read from the file match the expected result. - assert_batches_eq!(expected, &res); - - // Open the CSV file, read its contents as a record batch, and collect the batches into a vector. - let batches = session_ctx.sql("select * from t").await?.collect().await?; - - // Define the expected result after the second append. - let expected = vec![ - "+---------+", - "| column1 |", - "+---------+", - "| 1 |", - "| 2 |", - "| 3 |", - "| 1 |", - "| 2 |", - "| 3 |", - "| 1 |", - "| 2 |", - "| 3 |", - "| 1 |", - "| 2 |", - "| 3 |", - "+---------+", - ]; - - // Assert that the batches read from the file after the second append match the expected result. - assert_batches_eq!(expected, &batches); - - // Assert that no additional files were added to the table - let num_files = tmp_dir.path().read_dir()?.count(); - assert_eq!(num_files, 1); - - // Return Ok if the function - Ok(()) - } - async fn helper_test_append_new_files_to_table( file_type: FileType, file_compression_type: FileCompressionType, @@ -2129,7 +1828,6 @@ mod tests { "t", tmp_dir.path().to_str().unwrap(), CsvReadOptions::new() - .insert_mode(ListingTableInsertMode::AppendNewFiles) .schema(schema.as_ref()) .file_compression_type(file_compression_type), ) @@ -2141,7 +1839,6 @@ mod tests { "t", tmp_dir.path().to_str().unwrap(), NdJsonReadOptions::default() - .insert_mode(ListingTableInsertMode::AppendNewFiles) .schema(schema.as_ref()) .file_compression_type(file_compression_type), ) @@ -2152,9 +1849,7 @@ mod tests { .register_parquet( "t", tmp_dir.path().to_str().unwrap(), - ParquetReadOptions::default() - .insert_mode(ListingTableInsertMode::AppendNewFiles) - .schema(schema.as_ref()), + ParquetReadOptions::default().schema(schema.as_ref()), ) .await?; } @@ -2163,10 +1858,7 @@ mod tests { .register_avro( "t", tmp_dir.path().to_str().unwrap(), - AvroReadOptions::default() - // TODO implement insert_mode for avro - //.insert_mode(ListingTableInsertMode::AppendNewFiles) - .schema(schema.as_ref()), + AvroReadOptions::default().schema(schema.as_ref()), ) .await?; } @@ -2175,10 +1867,7 @@ mod tests { .register_arrow( "t", tmp_dir.path().to_str().unwrap(), - ArrowReadOptions::default() - // TODO implement insert_mode for arrow - //.insert_mode(ListingTableInsertMode::AppendNewFiles) - .schema(schema.as_ref()), + ArrowReadOptions::default().schema(schema.as_ref()), ) .await?; } diff --git a/datafusion/core/src/datasource/listing/url.rs b/datafusion/core/src/datasource/listing/url.rs index 9197e37adbd5..ba3c3fae21e2 100644 --- a/datafusion/core/src/datasource/listing/url.rs +++ b/datafusion/core/src/datasource/listing/url.rs @@ -181,6 +181,11 @@ impl ListingTableUrl { } } + /// Returns `true` if `path` refers to a collection of objects + pub fn is_collection(&self) -> bool { + self.url.as_str().ends_with('/') + } + /// Strips the prefix of this [`ListingTableUrl`] from the provided path, returning /// an iterator of the remaining path segments pub(crate) fn strip_prefix<'a, 'b: 'a>( @@ -203,8 +208,7 @@ impl ListingTableUrl { file_extension: &'a str, ) -> Result>> { // If the prefix is a file, use a head request, otherwise list - let is_dir = self.url.as_str().ends_with('/'); - let list = match is_dir { + let list = match self.is_collection() { true => match ctx.runtime_env().cache_manager.get_list_files_cache() { None => futures::stream::once(store.list(Some(&self.prefix))) .try_flatten() diff --git a/datafusion/core/src/datasource/listing_table_factory.rs b/datafusion/core/src/datasource/listing_table_factory.rs index f9a7ab04ce68..543a3a83f7c5 100644 --- a/datafusion/core/src/datasource/listing_table_factory.rs +++ b/datafusion/core/src/datasource/listing_table_factory.rs @@ -21,8 +21,6 @@ use std::path::Path; use std::str::FromStr; use std::sync::Arc; -use super::listing::ListingTableInsertMode; - #[cfg(feature = "parquet")] use crate::datasource::file_format::parquet::ParquetFormat; use crate::datasource::file_format::{ @@ -38,7 +36,7 @@ use crate::execution::context::SessionState; use arrow::datatypes::{DataType, SchemaRef}; use datafusion_common::file_options::{FileTypeWriterOptions, StatementOptions}; -use datafusion_common::{DataFusionError, FileType}; +use datafusion_common::{plan_err, DataFusionError, FileType}; use datafusion_expr::CreateExternalTable; use async_trait::async_trait; @@ -149,19 +147,12 @@ impl TableProviderFactory for ListingTableFactory { .take_bool_option("single_file")? .unwrap_or(false); - let explicit_insert_mode = statement_options.take_str_option("insert_mode"); - let insert_mode = match explicit_insert_mode { - Some(mode) => ListingTableInsertMode::from_str(mode.as_str()), - None => match file_type { - FileType::CSV => Ok(ListingTableInsertMode::AppendToFile), - #[cfg(feature = "parquet")] - FileType::PARQUET => Ok(ListingTableInsertMode::AppendNewFiles), - FileType::AVRO => Ok(ListingTableInsertMode::AppendNewFiles), - FileType::JSON => Ok(ListingTableInsertMode::AppendToFile), - FileType::ARROW => Ok(ListingTableInsertMode::AppendNewFiles), - }, - }?; - + // Backwards compatibility + if let Some(s) = statement_options.take_str_option("insert_mode") { + if !s.eq_ignore_ascii_case("append_new_files") { + return plan_err!("Unknown or unsupported insert mode {s}. Only append_to_file supported"); + } + } let file_type = file_format.file_type(); // Use remaining options and session state to build FileTypeWriterOptions @@ -214,7 +205,6 @@ impl TableProviderFactory for ListingTableFactory { .with_target_partitions(state.config().target_partitions()) .with_table_partition_cols(table_partition_cols) .with_file_sort_order(cmd.order_exprs.clone()) - .with_insert_mode(insert_mode) .with_single_file(single_file) .with_write_options(file_type_writer_options) .with_infinite_source(unbounded); diff --git a/datafusion/core/src/datasource/physical_plan/mod.rs b/datafusion/core/src/datasource/physical_plan/mod.rs index ea0a9698ff5c..738e70966bce 100644 --- a/datafusion/core/src/datasource/physical_plan/mod.rs +++ b/datafusion/core/src/datasource/physical_plan/mod.rs @@ -49,10 +49,7 @@ use std::{ use super::listing::ListingTableUrl; use crate::error::{DataFusionError, Result}; -use crate::{ - datasource::file_format::write::FileWriterMode, - physical_plan::{DisplayAs, DisplayFormatType}, -}; +use crate::physical_plan::{DisplayAs, DisplayFormatType}; use crate::{ datasource::{ listing::{FileRange, PartitionedFile}, @@ -90,8 +87,6 @@ pub struct FileSinkConfig { /// A vector of column names and their corresponding data types, /// representing the partitioning columns for the file pub table_partition_cols: Vec<(String, DataType)>, - /// A writer mode that determines how data is written to the file - pub writer_mode: FileWriterMode, /// If true, it is assumed there is a single table_path which is a file to which all data should be written /// regardless of input partitioning. Otherwise, each table path is assumed to be a directory /// to which each output partition is written to its own output file. diff --git a/datafusion/core/src/datasource/stream.rs b/datafusion/core/src/datasource/stream.rs index fc19ff954d8e..6965968b6f25 100644 --- a/datafusion/core/src/datasource/stream.rs +++ b/datafusion/core/src/datasource/stream.rs @@ -171,7 +171,7 @@ impl StreamConfig { match &self.encoding { StreamEncoding::Csv => { let header = self.header && !self.location.exists(); - let file = OpenOptions::new().write(true).open(&self.location)?; + let file = OpenOptions::new().append(true).open(&self.location)?; let writer = arrow::csv::WriterBuilder::new() .with_header(header) .build(file); @@ -179,7 +179,7 @@ impl StreamConfig { Ok(Box::new(writer)) } StreamEncoding::Json => { - let file = OpenOptions::new().write(true).open(&self.location)?; + let file = OpenOptions::new().append(true).open(&self.location)?; Ok(Box::new(arrow::json::LineDelimitedWriter::new(file))) } } @@ -298,7 +298,12 @@ struct StreamWrite(Arc); impl DisplayAs for StreamWrite { fn fmt_as(&self, _t: DisplayFormatType, f: &mut Formatter) -> std::fmt::Result { - write!(f, "{self:?}") + f.debug_struct("StreamWrite") + .field("location", &self.0.location) + .field("batch_size", &self.0.batch_size) + .field("encoding", &self.0.encoding) + .field("header", &self.0.header) + .finish_non_exhaustive() } } diff --git a/datafusion/core/src/physical_planner.rs b/datafusion/core/src/physical_planner.rs index 1f1ef73cae34..82d96c98e688 100644 --- a/datafusion/core/src/physical_planner.rs +++ b/datafusion/core/src/physical_planner.rs @@ -27,7 +27,6 @@ use crate::datasource::file_format::csv::CsvFormat; use crate::datasource::file_format::json::JsonFormat; #[cfg(feature = "parquet")] use crate::datasource::file_format::parquet::ParquetFormat; -use crate::datasource::file_format::write::FileWriterMode; use crate::datasource::file_format::FileFormat; use crate::datasource::listing::ListingTableUrl; use crate::datasource::physical_plan::FileSinkConfig; @@ -591,7 +590,6 @@ impl DefaultPhysicalPlanner { output_schema: Arc::new(schema), table_partition_cols: vec![], unbounded_input: false, - writer_mode: FileWriterMode::PutMultipart, single_file_output: *single_file_output, overwrite: false, file_type_writer_options diff --git a/datafusion/proto/proto/datafusion.proto b/datafusion/proto/proto/datafusion.proto index ad83ea1fce49..750d12bd776d 100644 --- a/datafusion/proto/proto/datafusion.proto +++ b/datafusion/proto/proto/datafusion.proto @@ -1157,12 +1157,6 @@ message PhysicalPlanNode { } } -enum FileWriterMode { - APPEND = 0; - PUT = 1; - PUT_MULTIPART = 2; -} - enum CompressionTypeVariant { GZIP = 0; BZIP2 = 1; @@ -1187,12 +1181,13 @@ message JsonWriterOptions { } message FileSinkConfig { + reserved 6; // writer_mode + string object_store_url = 1; repeated PartitionedFile file_groups = 2; repeated string table_paths = 3; Schema output_schema = 4; repeated PartitionColumn table_partition_cols = 5; - FileWriterMode writer_mode = 6; bool single_file_output = 7; bool unbounded_input = 8; bool overwrite = 9; diff --git a/datafusion/proto/src/generated/pbjson.rs b/datafusion/proto/src/generated/pbjson.rs index 016719a6001a..af64bd68deb1 100644 --- a/datafusion/proto/src/generated/pbjson.rs +++ b/datafusion/proto/src/generated/pbjson.rs @@ -7471,9 +7471,6 @@ impl serde::Serialize for FileSinkConfig { if !self.table_partition_cols.is_empty() { len += 1; } - if self.writer_mode != 0 { - len += 1; - } if self.single_file_output { len += 1; } @@ -7502,11 +7499,6 @@ impl serde::Serialize for FileSinkConfig { if !self.table_partition_cols.is_empty() { struct_ser.serialize_field("tablePartitionCols", &self.table_partition_cols)?; } - if self.writer_mode != 0 { - let v = FileWriterMode::try_from(self.writer_mode) - .map_err(|_| serde::ser::Error::custom(format!("Invalid variant {}", self.writer_mode)))?; - struct_ser.serialize_field("writerMode", &v)?; - } if self.single_file_output { struct_ser.serialize_field("singleFileOutput", &self.single_file_output)?; } @@ -7539,8 +7531,6 @@ impl<'de> serde::Deserialize<'de> for FileSinkConfig { "outputSchema", "table_partition_cols", "tablePartitionCols", - "writer_mode", - "writerMode", "single_file_output", "singleFileOutput", "unbounded_input", @@ -7557,7 +7547,6 @@ impl<'de> serde::Deserialize<'de> for FileSinkConfig { TablePaths, OutputSchema, TablePartitionCols, - WriterMode, SingleFileOutput, UnboundedInput, Overwrite, @@ -7588,7 +7577,6 @@ impl<'de> serde::Deserialize<'de> for FileSinkConfig { "tablePaths" | "table_paths" => Ok(GeneratedField::TablePaths), "outputSchema" | "output_schema" => Ok(GeneratedField::OutputSchema), "tablePartitionCols" | "table_partition_cols" => Ok(GeneratedField::TablePartitionCols), - "writerMode" | "writer_mode" => Ok(GeneratedField::WriterMode), "singleFileOutput" | "single_file_output" => Ok(GeneratedField::SingleFileOutput), "unboundedInput" | "unbounded_input" => Ok(GeneratedField::UnboundedInput), "overwrite" => Ok(GeneratedField::Overwrite), @@ -7617,7 +7605,6 @@ impl<'de> serde::Deserialize<'de> for FileSinkConfig { let mut table_paths__ = None; let mut output_schema__ = None; let mut table_partition_cols__ = None; - let mut writer_mode__ = None; let mut single_file_output__ = None; let mut unbounded_input__ = None; let mut overwrite__ = None; @@ -7654,12 +7641,6 @@ impl<'de> serde::Deserialize<'de> for FileSinkConfig { } table_partition_cols__ = Some(map_.next_value()?); } - GeneratedField::WriterMode => { - if writer_mode__.is_some() { - return Err(serde::de::Error::duplicate_field("writerMode")); - } - writer_mode__ = Some(map_.next_value::()? as i32); - } GeneratedField::SingleFileOutput => { if single_file_output__.is_some() { return Err(serde::de::Error::duplicate_field("singleFileOutput")); @@ -7692,7 +7673,6 @@ impl<'de> serde::Deserialize<'de> for FileSinkConfig { table_paths: table_paths__.unwrap_or_default(), output_schema: output_schema__, table_partition_cols: table_partition_cols__.unwrap_or_default(), - writer_mode: writer_mode__.unwrap_or_default(), single_file_output: single_file_output__.unwrap_or_default(), unbounded_input: unbounded_input__.unwrap_or_default(), overwrite: overwrite__.unwrap_or_default(), @@ -7800,80 +7780,6 @@ impl<'de> serde::Deserialize<'de> for FileTypeWriterOptions { deserializer.deserialize_struct("datafusion.FileTypeWriterOptions", FIELDS, GeneratedVisitor) } } -impl serde::Serialize for FileWriterMode { - #[allow(deprecated)] - fn serialize(&self, serializer: S) -> std::result::Result - where - S: serde::Serializer, - { - let variant = match self { - Self::Append => "APPEND", - Self::Put => "PUT", - Self::PutMultipart => "PUT_MULTIPART", - }; - serializer.serialize_str(variant) - } -} -impl<'de> serde::Deserialize<'de> for FileWriterMode { - #[allow(deprecated)] - fn deserialize(deserializer: D) -> std::result::Result - where - D: serde::Deserializer<'de>, - { - const FIELDS: &[&str] = &[ - "APPEND", - "PUT", - "PUT_MULTIPART", - ]; - - struct GeneratedVisitor; - - impl<'de> serde::de::Visitor<'de> for GeneratedVisitor { - type Value = FileWriterMode; - - fn expecting(&self, formatter: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { - write!(formatter, "expected one of: {:?}", &FIELDS) - } - - fn visit_i64(self, v: i64) -> std::result::Result - where - E: serde::de::Error, - { - i32::try_from(v) - .ok() - .and_then(|x| x.try_into().ok()) - .ok_or_else(|| { - serde::de::Error::invalid_value(serde::de::Unexpected::Signed(v), &self) - }) - } - - fn visit_u64(self, v: u64) -> std::result::Result - where - E: serde::de::Error, - { - i32::try_from(v) - .ok() - .and_then(|x| x.try_into().ok()) - .ok_or_else(|| { - serde::de::Error::invalid_value(serde::de::Unexpected::Unsigned(v), &self) - }) - } - - fn visit_str(self, value: &str) -> std::result::Result - where - E: serde::de::Error, - { - match value { - "APPEND" => Ok(FileWriterMode::Append), - "PUT" => Ok(FileWriterMode::Put), - "PUT_MULTIPART" => Ok(FileWriterMode::PutMultipart), - _ => Err(serde::de::Error::unknown_variant(value, FIELDS)), - } - } - } - deserializer.deserialize_any(GeneratedVisitor) - } -} impl serde::Serialize for FilterExecNode { #[allow(deprecated)] fn serialize(&self, serializer: S) -> std::result::Result diff --git a/datafusion/proto/src/generated/prost.rs b/datafusion/proto/src/generated/prost.rs index 647f814fda8d..b23f09e91b26 100644 --- a/datafusion/proto/src/generated/prost.rs +++ b/datafusion/proto/src/generated/prost.rs @@ -1615,8 +1615,6 @@ pub struct FileSinkConfig { pub output_schema: ::core::option::Option, #[prost(message, repeated, tag = "5")] pub table_partition_cols: ::prost::alloc::vec::Vec, - #[prost(enumeration = "FileWriterMode", tag = "6")] - pub writer_mode: i32, #[prost(bool, tag = "7")] pub single_file_output: bool, #[prost(bool, tag = "8")] @@ -3200,35 +3198,6 @@ impl UnionMode { } #[derive(Clone, Copy, Debug, PartialEq, Eq, Hash, PartialOrd, Ord, ::prost::Enumeration)] #[repr(i32)] -pub enum FileWriterMode { - Append = 0, - Put = 1, - PutMultipart = 2, -} -impl FileWriterMode { - /// String value of the enum field names used in the ProtoBuf definition. - /// - /// The values are not transformed in any way and thus are considered stable - /// (if the ProtoBuf definition does not change) and safe for programmatic use. - pub fn as_str_name(&self) -> &'static str { - match self { - FileWriterMode::Append => "APPEND", - FileWriterMode::Put => "PUT", - FileWriterMode::PutMultipart => "PUT_MULTIPART", - } - } - /// Creates an enum from field names used in the ProtoBuf definition. - pub fn from_str_name(value: &str) -> ::core::option::Option { - match value { - "APPEND" => Some(Self::Append), - "PUT" => Some(Self::Put), - "PUT_MULTIPART" => Some(Self::PutMultipart), - _ => None, - } - } -} -#[derive(Clone, Copy, Debug, PartialEq, Eq, Hash, PartialOrd, Ord, ::prost::Enumeration)] -#[repr(i32)] pub enum CompressionTypeVariant { Gzip = 0, Bzip2 = 1, diff --git a/datafusion/proto/src/physical_plan/from_proto.rs b/datafusion/proto/src/physical_plan/from_proto.rs index 22b74db9afd2..f5771ddb155b 100644 --- a/datafusion/proto/src/physical_plan/from_proto.rs +++ b/datafusion/proto/src/physical_plan/from_proto.rs @@ -23,7 +23,6 @@ use std::sync::Arc; use arrow::compute::SortOptions; use datafusion::arrow::datatypes::Schema; use datafusion::datasource::file_format::json::JsonSink; -use datafusion::datasource::file_format::write::FileWriterMode; use datafusion::datasource::listing::{FileRange, ListingTableUrl, PartitionedFile}; use datafusion::datasource::object_store::ObjectStoreUrl; use datafusion::datasource::physical_plan::{FileScanConfig, FileSinkConfig}; @@ -739,7 +738,6 @@ impl TryFrom<&protobuf::FileSinkConfig> for FileSinkConfig { table_paths, output_schema: Arc::new(convert_required!(conf.output_schema)?), table_partition_cols, - writer_mode: conf.writer_mode().into(), single_file_output: conf.single_file_output, unbounded_input: conf.unbounded_input, overwrite: conf.overwrite, @@ -748,16 +746,6 @@ impl TryFrom<&protobuf::FileSinkConfig> for FileSinkConfig { } } -impl From for FileWriterMode { - fn from(value: protobuf::FileWriterMode) -> Self { - match value { - protobuf::FileWriterMode::Append => Self::Append, - protobuf::FileWriterMode::Put => Self::Put, - protobuf::FileWriterMode::PutMultipart => Self::PutMultipart, - } - } -} - impl From for CompressionTypeVariant { fn from(value: protobuf::CompressionTypeVariant) -> Self { match value { diff --git a/datafusion/proto/src/physical_plan/to_proto.rs b/datafusion/proto/src/physical_plan/to_proto.rs index b8a590b0dc1a..44864be947d5 100644 --- a/datafusion/proto/src/physical_plan/to_proto.rs +++ b/datafusion/proto/src/physical_plan/to_proto.rs @@ -31,7 +31,6 @@ use datafusion::datasource::{ file_format::json::JsonSink, physical_plan::FileScanConfig, }; use datafusion::datasource::{ - file_format::write::FileWriterMode, listing::{FileRange, PartitionedFile}, physical_plan::FileSinkConfig, }; @@ -819,7 +818,6 @@ impl TryFrom<&FileSinkConfig> for protobuf::FileSinkConfig { type Error = DataFusionError; fn try_from(conf: &FileSinkConfig) -> Result { - let writer_mode: protobuf::FileWriterMode = conf.writer_mode.into(); let file_groups = conf .file_groups .iter() @@ -847,7 +845,6 @@ impl TryFrom<&FileSinkConfig> for protobuf::FileSinkConfig { table_paths, output_schema: Some(conf.output_schema.as_ref().try_into()?), table_partition_cols, - writer_mode: writer_mode.into(), single_file_output: conf.single_file_output, unbounded_input: conf.unbounded_input, overwrite: conf.overwrite, @@ -856,16 +853,6 @@ impl TryFrom<&FileSinkConfig> for protobuf::FileSinkConfig { } } -impl From for protobuf::FileWriterMode { - fn from(value: FileWriterMode) -> Self { - match value { - FileWriterMode::Append => Self::Append, - FileWriterMode::Put => Self::Put, - FileWriterMode::PutMultipart => Self::PutMultipart, - } - } -} - impl From<&CompressionTypeVariant> for protobuf::CompressionTypeVariant { fn from(value: &CompressionTypeVariant) -> Self { match value { diff --git a/datafusion/proto/tests/cases/roundtrip_physical_plan.rs b/datafusion/proto/tests/cases/roundtrip_physical_plan.rs index 076ca415810a..23b0ea43c73a 100644 --- a/datafusion/proto/tests/cases/roundtrip_physical_plan.rs +++ b/datafusion/proto/tests/cases/roundtrip_physical_plan.rs @@ -22,7 +22,6 @@ use datafusion::arrow::array::ArrayRef; use datafusion::arrow::compute::kernels::sort::SortOptions; use datafusion::arrow::datatypes::{DataType, Field, Fields, IntervalUnit, Schema}; use datafusion::datasource::file_format::json::JsonSink; -use datafusion::datasource::file_format::write::FileWriterMode; use datafusion::datasource::listing::{ListingTableUrl, PartitionedFile}; use datafusion::datasource::object_store::ObjectStoreUrl; use datafusion::datasource::physical_plan::{ @@ -732,7 +731,6 @@ fn roundtrip_json_sink() -> Result<()> { table_paths: vec![ListingTableUrl::parse("file:///")?], output_schema: schema.clone(), table_partition_cols: vec![("plan_type".to_string(), DataType::Utf8)], - writer_mode: FileWriterMode::Put, single_file_output: true, unbounded_input: false, overwrite: true, diff --git a/datafusion/sqllogictest/test_files/copy.slt b/datafusion/sqllogictest/test_files/copy.slt index 6e4a711a0115..fbf1523477b1 100644 --- a/datafusion/sqllogictest/test_files/copy.slt +++ b/datafusion/sqllogictest/test_files/copy.slt @@ -32,7 +32,7 @@ logical_plan CopyTo: format=parquet output_url=test_files/scratch/copy/table single_file_output=false options: (compression 'zstd(10)') --TableScan: source_table projection=[col1, col2] physical_plan -FileSinkExec: sink=ParquetSink(writer_mode=PutMultipart, file_groups=[]) +FileSinkExec: sink=ParquetSink(file_groups=[]) --MemoryExec: partitions=1, partition_sizes=[1] # Error case diff --git a/datafusion/sqllogictest/test_files/errors.slt b/datafusion/sqllogictest/test_files/errors.slt index 4aded8a576fb..e3b2610e51be 100644 --- a/datafusion/sqllogictest/test_files/errors.slt +++ b/datafusion/sqllogictest/test_files/errors.slt @@ -133,4 +133,4 @@ order by c9 statement error Inconsistent data type across values list at row 1 column 0. Was Int64 but found Utf8 -create table foo as values (1), ('foo'); \ No newline at end of file +create table foo as values (1), ('foo'); diff --git a/datafusion/sqllogictest/test_files/explain.slt b/datafusion/sqllogictest/test_files/explain.slt index 9726c35a319e..129814767ca2 100644 --- a/datafusion/sqllogictest/test_files/explain.slt +++ b/datafusion/sqllogictest/test_files/explain.slt @@ -140,7 +140,7 @@ physical_plan CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/te # create a sink table, path is same with aggregate_test_100 table # we do not overwrite this file, we only assert plan. statement ok -CREATE EXTERNAL TABLE sink_table ( +CREATE UNBOUNDED EXTERNAL TABLE sink_table ( c1 VARCHAR NOT NULL, c2 TINYINT NOT NULL, c3 SMALLINT NOT NULL, @@ -168,7 +168,7 @@ Dml: op=[Insert Into] table=[sink_table] ----Sort: aggregate_test_100.c1 ASC NULLS LAST ------TableScan: aggregate_test_100 projection=[c1, c2, c3, c4, c5, c6, c7, c8, c9, c10, c11, c12, c13] physical_plan -FileSinkExec: sink=CsvSink(writer_mode=Append, file_groups=[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]) +FileSinkExec: sink=StreamWrite { location: "../../testing/data/csv/aggregate_test_100.csv", batch_size: 8192, encoding: Csv, header: true, .. } --SortExec: expr=[c1@0 ASC NULLS LAST] ----CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1, c2, c3, c4, c5, c6, c7, c8, c9, c10, c11, c12, c13], has_header=true diff --git a/datafusion/sqllogictest/test_files/insert.slt b/datafusion/sqllogictest/test_files/insert.slt index 9860bdcae05c..a100b5ac6b85 100644 --- a/datafusion/sqllogictest/test_files/insert.slt +++ b/datafusion/sqllogictest/test_files/insert.slt @@ -289,7 +289,7 @@ insert into table_without_values values(2, NULL); ---- 1 -# insert NULL values for the missing column (field2) +# insert NULL values for the missing column (field2) query II insert into table_without_values(field1) values(3); ---- diff --git a/datafusion/sqllogictest/test_files/insert_to_external.slt b/datafusion/sqllogictest/test_files/insert_to_external.slt index 44410362412c..39323479ff74 100644 --- a/datafusion/sqllogictest/test_files/insert_to_external.slt +++ b/datafusion/sqllogictest/test_files/insert_to_external.slt @@ -100,7 +100,7 @@ Dml: op=[Insert Into] table=[ordered_insert_test] --Projection: column1 AS a, column2 AS b ----Values: (Int64(5), Int64(1)), (Int64(4), Int64(2)), (Int64(7), Int64(7)), (Int64(7), Int64(8)), (Int64(7), Int64(9))... physical_plan -FileSinkExec: sink=CsvSink(writer_mode=PutMultipart, file_groups=[]) +FileSinkExec: sink=CsvSink(file_groups=[]) --SortExec: expr=[a@0 ASC NULLS LAST,b@1 DESC] ----ProjectionExec: expr=[column1@0 as a, column2@1 as b] ------ValuesExec @@ -254,6 +254,22 @@ create_local_path 'true', single_file 'true', ); +query error DataFusion error: Error during planning: Inserting into a ListingTable backed by a single file is not supported, URL is possibly missing a trailing `/`\. To append to an existing file use StreamTable, e\.g\. by using CREATE UNBOUNDED EXTERNAL TABLE +INSERT INTO single_file_test values (1, 2), (3, 4); + +statement ok +drop table single_file_test; + +statement ok +CREATE UNBOUNDED EXTERNAL TABLE +single_file_test(a bigint, b bigint) +STORED AS csv +LOCATION 'test_files/scratch/insert_to_external/single_csv_table.csv' +OPTIONS( +create_local_path 'true', +single_file 'true', +); + query II INSERT INTO single_file_test values (1, 2), (3, 4); ---- @@ -315,7 +331,7 @@ Dml: op=[Insert Into] table=[table_without_values] --------WindowAggr: windowExpr=[[SUM(CAST(aggregate_test_100.c4 AS Int64)) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING, COUNT(UInt8(1)) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING AS COUNT(*) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING]] ----------TableScan: aggregate_test_100 projection=[c1, c4, c9] physical_plan -FileSinkExec: sink=ParquetSink(writer_mode=PutMultipart, file_groups=[]) +FileSinkExec: sink=ParquetSink(file_groups=[]) --ProjectionExec: expr=[SUM(aggregate_test_100.c4) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING@0 as field1, COUNT(*) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING@1 as field2] ----SortPreservingMergeExec: [c1@2 ASC NULLS LAST] ------ProjectionExec: expr=[SUM(aggregate_test_100.c4) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING@3 as SUM(aggregate_test_100.c4) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING, COUNT(*) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING@4 as COUNT(*) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING, c1@0 as c1] @@ -378,7 +394,7 @@ Dml: op=[Insert Into] table=[table_without_values] ----WindowAggr: windowExpr=[[SUM(CAST(aggregate_test_100.c4 AS Int64)) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING, COUNT(UInt8(1)) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING AS COUNT(*) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING]] ------TableScan: aggregate_test_100 projection=[c1, c4, c9] physical_plan -FileSinkExec: sink=ParquetSink(writer_mode=PutMultipart, file_groups=[]) +FileSinkExec: sink=ParquetSink(file_groups=[]) --CoalescePartitionsExec ----ProjectionExec: expr=[SUM(aggregate_test_100.c4) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING@3 as field1, COUNT(*) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING@4 as field2] ------BoundedWindowAggExec: wdw=[SUM(aggregate_test_100.c4) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "SUM(aggregate_test_100.c4) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(1)), end_bound: Following(UInt64(1)) }, COUNT(*) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "COUNT(*) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(1)), end_bound: Following(UInt64(1)) }], mode=[Sorted] @@ -422,7 +438,7 @@ Dml: op=[Insert Into] table=[table_without_values] ----Sort: aggregate_test_100.c1 ASC NULLS LAST ------TableScan: aggregate_test_100 projection=[c1] physical_plan -FileSinkExec: sink=ParquetSink(writer_mode=PutMultipart, file_groups=[]) +FileSinkExec: sink=ParquetSink(file_groups=[]) --SortExec: expr=[c1@0 ASC NULLS LAST] ----CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1], has_header=true diff --git a/datafusion/sqllogictest/test_files/joins.slt b/datafusion/sqllogictest/test_files/joins.slt index 737b43b5a903..0fea8da5a342 100644 --- a/datafusion/sqllogictest/test_files/joins.slt +++ b/datafusion/sqllogictest/test_files/joins.slt @@ -3486,4 +3486,3 @@ set datafusion.optimizer.prefer_existing_sort = false; statement ok drop table annotated_data; - diff --git a/datafusion/sqllogictest/test_files/options.slt b/datafusion/sqllogictest/test_files/options.slt index 83fe85745ef8..9366a9b3b3c8 100644 --- a/datafusion/sqllogictest/test_files/options.slt +++ b/datafusion/sqllogictest/test_files/options.slt @@ -84,7 +84,7 @@ statement ok drop table a # test datafusion.sql_parser.parse_float_as_decimal -# +# # default option value is false query RR select 10000000000000000000.01, -10000000000000000000.01 @@ -209,5 +209,3 @@ select -123456789.0123456789012345678901234567890 # Restore option to default value statement ok set datafusion.sql_parser.parse_float_as_decimal = false; - - diff --git a/datafusion/sqllogictest/test_files/order.slt b/datafusion/sqllogictest/test_files/order.slt index 8148f1c4c7c9..9c5d1704f42b 100644 --- a/datafusion/sqllogictest/test_files/order.slt +++ b/datafusion/sqllogictest/test_files/order.slt @@ -447,7 +447,7 @@ statement ok drop table multiple_ordered_table; # Create tables having some ordered columns. In the next step, we will expect to observe that scalar -# functions, such as mathematical functions like atan(), ceil(), sqrt(), or date_time functions +# functions, such as mathematical functions like atan(), ceil(), sqrt(), or date_time functions # like date_bin() and date_trunc(), will maintain the order of its argument columns. statement ok CREATE EXTERNAL TABLE csv_with_timestamps ( diff --git a/datafusion/sqllogictest/test_files/predicates.slt b/datafusion/sqllogictest/test_files/predicates.slt index d22b2ff953b7..e992a440d0a2 100644 --- a/datafusion/sqllogictest/test_files/predicates.slt +++ b/datafusion/sqllogictest/test_files/predicates.slt @@ -495,6 +495,7 @@ set datafusion.execution.parquet.bloom_filter_enabled=true; query T SELECT * FROM data_index_bloom_encoding_stats WHERE "String" = 'foo'; +---- query T SELECT * FROM data_index_bloom_encoding_stats WHERE "String" = 'test'; diff --git a/datafusion/sqllogictest/test_files/set_variable.slt b/datafusion/sqllogictest/test_files/set_variable.slt index 714e1e995e26..440fb2c6ef2b 100644 --- a/datafusion/sqllogictest/test_files/set_variable.slt +++ b/datafusion/sqllogictest/test_files/set_variable.slt @@ -243,4 +243,4 @@ statement ok SET TIME ZONE = 'Asia/Taipei2' statement error Arrow error: Parser error: Invalid timezone "Asia/Taipei2": 'Asia/Taipei2' is not a valid timezone -SELECT '2000-01-01T00:00:00'::TIMESTAMP::TIMESTAMPTZ \ No newline at end of file +SELECT '2000-01-01T00:00:00'::TIMESTAMP::TIMESTAMPTZ diff --git a/datafusion/sqllogictest/test_files/update.slt b/datafusion/sqllogictest/test_files/update.slt index c88082fc7272..6412c3ca859e 100644 --- a/datafusion/sqllogictest/test_files/update.slt +++ b/datafusion/sqllogictest/test_files/update.slt @@ -89,4 +89,4 @@ Dml: op=[Update] table=[t1] ------CrossJoin: --------SubqueryAlias: t ----------TableScan: t1 ---------TableScan: t2 \ No newline at end of file +--------TableScan: t2 From 2e3f4344be4b520a1c58ab82b4171303b6826b65 Mon Sep 17 00:00:00 2001 From: Alex Huang Date: Sat, 18 Nov 2023 11:01:42 +0100 Subject: [PATCH 287/572] Minor: clean up the code based on Clippy (#8257) --- datafusion/physical-expr/src/array_expressions.rs | 4 ++-- datafusion/physical-plan/src/filter.rs | 2 +- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/datafusion/physical-expr/src/array_expressions.rs b/datafusion/physical-expr/src/array_expressions.rs index 8bb70c316879..e2d22a0d3328 100644 --- a/datafusion/physical-expr/src/array_expressions.rs +++ b/datafusion/physical-expr/src/array_expressions.rs @@ -582,14 +582,14 @@ pub fn array_except(args: &[ArrayRef]) -> Result { match (array1.data_type(), array2.data_type()) { (DataType::Null, _) | (_, DataType::Null) => Ok(array1.to_owned()), (DataType::List(field), DataType::List(_)) => { - check_datatypes("array_except", &[&array1, &array2])?; + check_datatypes("array_except", &[array1, array2])?; let list1 = array1.as_list::(); let list2 = array2.as_list::(); let result = general_except::(list1, list2, field)?; Ok(Arc::new(result)) } (DataType::LargeList(field), DataType::LargeList(_)) => { - check_datatypes("array_except", &[&array1, &array2])?; + check_datatypes("array_except", &[array1, array2])?; let list1 = array1.as_list::(); let list2 = array2.as_list::(); let result = general_except::(list1, list2, field)?; diff --git a/datafusion/physical-plan/src/filter.rs b/datafusion/physical-plan/src/filter.rs index 107c95eff7f1..b6cd9fe79c85 100644 --- a/datafusion/physical-plan/src/filter.rs +++ b/datafusion/physical-plan/src/filter.rs @@ -201,7 +201,7 @@ impl ExecutionPlan for FilterExec { // tracking issue for making this configurable: // https://github.com/apache/arrow-datafusion/issues/8133 let selectivity = 0.2_f64; - let mut stats = input_stats.clone().into_inexact(); + let mut stats = input_stats.into_inexact(); stats.num_rows = stats.num_rows.with_estimated_selectivity(selectivity); stats.total_byte_size = stats .total_byte_size From a984f08989a1d59b04992f6325a2b707629e8873 Mon Sep 17 00:00:00 2001 From: Raphael Taylor-Davies <1781103+tustvold@users.noreply.github.com> Date: Sat, 18 Nov 2023 10:52:48 +0000 Subject: [PATCH 288/572] Update arrow 49.0.0 and object_store 0.8.0 (#8029) * POC: Remove ListingTable Append Support (#7994) * Prepare object_store 0.8.0 * Fix datafusion-cli test * Update arrow version * Update tests * Update pin * Unignore fifo test * Update lockfile --- Cargo.toml | 17 +++-- datafusion-cli/Cargo.lock | 71 ++++++++++--------- datafusion-cli/Cargo.toml | 4 +- datafusion-cli/src/exec.rs | 7 +- datafusion/core/src/catalog/listing_schema.rs | 7 +- .../core/src/datasource/file_format/arrow.rs | 2 + .../core/src/datasource/file_format/csv.rs | 1 + .../core/src/datasource/file_format/mod.rs | 16 +++-- .../src/datasource/file_format/parquet.rs | 20 ++++-- .../core/src/datasource/listing/helpers.rs | 3 +- datafusion/core/src/datasource/listing/mod.rs | 2 + datafusion/core/src/datasource/listing/url.rs | 12 ++-- .../core/src/datasource/physical_plan/mod.rs | 1 + .../src/datasource/physical_plan/parquet.rs | 1 + .../physical_plan/parquet/row_groups.rs | 1 + datafusion/core/src/test/object_store.rs | 1 + datafusion/core/src/test_util/parquet.rs | 1 + .../core/tests/parquet/custom_reader.rs | 1 + datafusion/core/tests/parquet/page_pruning.rs | 1 + .../core/tests/parquet/schema_coercion.rs | 1 + datafusion/core/tests/path_partition.rs | 30 ++++---- datafusion/execution/src/cache/cache_unit.rs | 2 + .../physical-expr/src/expressions/cast.rs | 6 +- .../physical-expr/src/expressions/try_cast.rs | 6 +- .../proto/src/physical_plan/from_proto.rs | 1 + datafusion/sqllogictest/test_files/copy.slt | 18 ++--- .../substrait/src/physical_plan/consumer.rs | 1 + 27 files changed, 132 insertions(+), 102 deletions(-) diff --git a/Cargo.toml b/Cargo.toml index f25c24fd3e1c..60befdf1cfb7 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -49,12 +49,12 @@ rust-version = "1.70" version = "33.0.0" [workspace.dependencies] -arrow = { version = "~48.0.1", features = ["prettyprint"] } -arrow-array = { version = "~48.0.1", default-features = false, features = ["chrono-tz"] } -arrow-buffer = { version = "~48.0.1", default-features = false } -arrow-flight = { version = "~48.0.1", features = ["flight-sql-experimental"] } -arrow-ord = { version = "~48.0.1", default-features = false } -arrow-schema = { version = "~48.0.1", default-features = false } +arrow = { version = "49.0.0", features = ["prettyprint"] } +arrow-array = { version = "49.0.0", default-features = false, features = ["chrono-tz"] } +arrow-buffer = { version = "49.0.0", default-features = false } +arrow-flight = { version = "49.0.0", features = ["flight-sql-experimental"] } +arrow-ord = { version = "49.0.0", default-features = false } +arrow-schema = { version = "49.0.0", default-features = false } async-trait = "0.1.73" bigdecimal = "0.4.1" bytes = "1.4" @@ -79,9 +79,9 @@ indexmap = "2.0.0" itertools = "0.12" log = "^0.4" num_cpus = "1.13.0" -object_store = { version = "0.7.0", default-features = false } +object_store = { version = "0.8.0", default-features = false } parking_lot = "0.12" -parquet = { version = "~48.0.1", default-features = false, features = ["arrow", "async", "object_store"] } +parquet = { version = "49.0.0", default-features = false, features = ["arrow", "async", "object_store"] } rand = "0.8" rstest = "0.18.0" serde_json = "1" @@ -108,4 +108,3 @@ opt-level = 3 overflow-checks = false panic = 'unwind' rpath = false - diff --git a/datafusion-cli/Cargo.lock b/datafusion-cli/Cargo.lock index 4bc61a48a36e..06bc14c5b656 100644 --- a/datafusion-cli/Cargo.lock +++ b/datafusion-cli/Cargo.lock @@ -130,9 +130,9 @@ checksum = "96d30a06541fbafbc7f82ed10c06164cfbd2c401138f6addd8404629c4b16711" [[package]] name = "arrow" -version = "48.0.1" +version = "49.0.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "a8919668503a4f2d8b6da96fa7c16e93046bfb3412ffcfa1e5dc7d2e3adcb378" +checksum = "5bc25126d18a012146a888a0298f2c22e1150327bd2765fc76d710a556b2d614" dependencies = [ "ahash", "arrow-arith", @@ -152,9 +152,9 @@ dependencies = [ [[package]] name = "arrow-arith" -version = "48.0.1" +version = "49.0.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "ef983914f477d4278b068f13b3224b7d19eb2b807ac9048544d3bfebdf2554c4" +checksum = "34ccd45e217ffa6e53bbb0080990e77113bdd4e91ddb84e97b77649810bcf1a7" dependencies = [ "arrow-array", "arrow-buffer", @@ -167,9 +167,9 @@ dependencies = [ [[package]] name = "arrow-array" -version = "48.0.1" +version = "49.0.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "d6eaf89041fa5937940ae390294ece29e1db584f46d995608d6e5fe65a2e0e9b" +checksum = "6bda9acea48b25123c08340f3a8ac361aa0f74469bb36f5ee9acf923fce23e9d" dependencies = [ "ahash", "arrow-buffer", @@ -184,9 +184,9 @@ dependencies = [ [[package]] name = "arrow-buffer" -version = "48.0.1" +version = "49.0.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "55512d988c6fbd76e514fd3ff537ac50b0a675da5a245e4fdad77ecfd654205f" +checksum = "01a0fc21915b00fc6c2667b069c1b64bdd920982f426079bc4a7cab86822886c" dependencies = [ "bytes", "half", @@ -195,15 +195,16 @@ dependencies = [ [[package]] name = "arrow-cast" -version = "48.0.1" +version = "49.0.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "655ee51a2156ba5375931ce21c1b2494b1d9260e6dcdc6d4db9060c37dc3325b" +checksum = "5dc0368ed618d509636c1e3cc20db1281148190a78f43519487b2daf07b63b4a" dependencies = [ "arrow-array", "arrow-buffer", "arrow-data", "arrow-schema", "arrow-select", + "base64", "chrono", "comfy-table", "half", @@ -213,9 +214,9 @@ dependencies = [ [[package]] name = "arrow-csv" -version = "48.0.1" +version = "49.0.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "258bb689997ad5b6660b3ce3638bd6b383d668ec555ed41ad7c6559cbb2e4f91" +checksum = "2e09aa6246a1d6459b3f14baeaa49606cfdbca34435c46320e14054d244987ca" dependencies = [ "arrow-array", "arrow-buffer", @@ -232,9 +233,9 @@ dependencies = [ [[package]] name = "arrow-data" -version = "48.0.1" +version = "49.0.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "6dc2b9fec74763427e2e5575b8cc31ce96ba4c9b4eb05ce40e0616d9fad12461" +checksum = "907fafe280a3874474678c1858b9ca4cb7fd83fb8034ff5b6d6376205a08c634" dependencies = [ "arrow-buffer", "arrow-schema", @@ -244,9 +245,9 @@ dependencies = [ [[package]] name = "arrow-ipc" -version = "48.0.1" +version = "49.0.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "6eaa6ab203cc6d89b7eaa1ac781c1dfeef325454c5d5a0419017f95e6bafc03c" +checksum = "79a43d6808411886b8c7d4f6f7dd477029c1e77ffffffb7923555cc6579639cd" dependencies = [ "arrow-array", "arrow-buffer", @@ -258,9 +259,9 @@ dependencies = [ [[package]] name = "arrow-json" -version = "48.0.1" +version = "49.0.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "fb64e30d9b73f66fdc5c52d5f4cf69bbf03d62f64ffeafa0715590a5320baed7" +checksum = "d82565c91fd627922ebfe2810ee4e8346841b6f9361b87505a9acea38b614fee" dependencies = [ "arrow-array", "arrow-buffer", @@ -278,9 +279,9 @@ dependencies = [ [[package]] name = "arrow-ord" -version = "48.0.1" +version = "49.0.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "f9a818951c0d11c428dda03e908175969c262629dd20bd0850bd6c7a8c3bfe48" +checksum = "9b23b0e53c0db57c6749997fd343d4c0354c994be7eca67152dd2bdb9a3e1bb4" dependencies = [ "arrow-array", "arrow-buffer", @@ -293,9 +294,9 @@ dependencies = [ [[package]] name = "arrow-row" -version = "48.0.1" +version = "49.0.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "a5d664318bc05f930559fc088888f0f7174d3c5bc888c0f4f9ae8f23aa398ba3" +checksum = "361249898d2d6d4a6eeb7484be6ac74977e48da12a4dd81a708d620cc558117a" dependencies = [ "ahash", "arrow-array", @@ -308,15 +309,15 @@ dependencies = [ [[package]] name = "arrow-schema" -version = "48.0.1" +version = "49.0.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "aaf4d737bba93da59f16129bec21e087aed0be84ff840e74146d4703879436cb" +checksum = "09e28a5e781bf1b0f981333684ad13f5901f4cd2f20589eab7cf1797da8fc167" [[package]] name = "arrow-select" -version = "48.0.1" +version = "49.0.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "374c4c3b812ecc2118727b892252a4a4308f87a8aca1dbf09f3ce4bc578e668a" +checksum = "4f6208466590960efc1d2a7172bc4ff18a67d6e25c529381d7f96ddaf0dc4036" dependencies = [ "ahash", "arrow-array", @@ -328,9 +329,9 @@ dependencies = [ [[package]] name = "arrow-string" -version = "48.0.1" +version = "49.0.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "b15aed5624bb23da09142f58502b59c23f5bea607393298bb81dab1ce60fc769" +checksum = "a4a48149c63c11c9ff571e50ab8f017d2a7cb71037a882b42f6354ed2da9acc7" dependencies = [ "arrow-array", "arrow-buffer", @@ -2288,9 +2289,9 @@ dependencies = [ [[package]] name = "object_store" -version = "0.7.1" +version = "0.8.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "f930c88a43b1c3f6e776dfe495b4afab89882dbc81530c632db2ed65451ebcb4" +checksum = "2524735495ea1268be33d200e1ee97455096a0846295a21548cd2f3541de7050" dependencies = [ "async-trait", "base64", @@ -2305,7 +2306,7 @@ dependencies = [ "quick-xml", "rand", "reqwest", - "ring 0.16.20", + "ring 0.17.5", "rustls-pemfile", "serde", "serde_json", @@ -2374,9 +2375,9 @@ dependencies = [ [[package]] name = "parquet" -version = "48.0.1" +version = "49.0.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "6bfe55df96e3f02f11bf197ae37d91bb79801631f82f6195dd196ef521df3597" +checksum = "af88740a842787da39b3d69ce5fbf6fce97d20211d3b299fee0a0da6430c74d4" dependencies = [ "ahash", "arrow-array", @@ -2597,9 +2598,9 @@ checksum = "658fa1faf7a4cc5f057c9ee5ef560f717ad9d8dc66d975267f709624d6e1ab88" [[package]] name = "quick-xml" -version = "0.30.0" +version = "0.31.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "eff6510e86862b57b210fd8cbe8ed3f0d7d600b9c2863cd4549a2e033c66e956" +checksum = "1004a344b30a54e2ee58d66a71b32d2db2feb0a31f9a2d302bf0536f15de2a33" dependencies = [ "memchr", "serde", diff --git a/datafusion-cli/Cargo.toml b/datafusion-cli/Cargo.toml index 890f84522c26..dd7a077988cb 100644 --- a/datafusion-cli/Cargo.toml +++ b/datafusion-cli/Cargo.toml @@ -29,7 +29,7 @@ rust-version = "1.70" readme = "README.md" [dependencies] -arrow = "~48.0.1" +arrow = "49.0.0" async-trait = "0.1.41" aws-config = "0.55" aws-credential-types = "0.55" @@ -38,7 +38,7 @@ datafusion = { path = "../datafusion/core", version = "33.0.0", features = ["avr dirs = "4.0.0" env_logger = "0.9" mimalloc = { version = "0.1", default-features = false } -object_store = { version = "0.7.0", features = ["aws", "gcp"] } +object_store = { version = "0.8.0", features = ["aws", "gcp"] } parking_lot = { version = "0.12" } regex = "1.8" rustyline = "11.0" diff --git a/datafusion-cli/src/exec.rs b/datafusion-cli/src/exec.rs index b62ad12dbfbb..14ac22687bf4 100644 --- a/datafusion-cli/src/exec.rs +++ b/datafusion-cli/src/exec.rs @@ -350,7 +350,7 @@ mod tests { async fn create_object_store_table_gcs() -> Result<()> { let service_account_path = "fake_service_account_path"; let service_account_key = - "{\"private_key\": \"fake_private_key.pem\",\"client_email\":\"fake_client_email\"}"; + "{\"private_key\": \"fake_private_key.pem\",\"client_email\":\"fake_client_email\", \"private_key_id\":\"id\"}"; let application_credentials_path = "fake_application_credentials_path"; let location = "gcs://bucket/path/file.parquet"; @@ -366,8 +366,9 @@ mod tests { let sql = format!("CREATE EXTERNAL TABLE test STORED AS PARQUET OPTIONS('service_account_key' '{service_account_key}') LOCATION '{location}'"); let err = create_external_table_test(location, &sql) .await - .unwrap_err(); - assert!(err.to_string().contains("No RSA key found in pem file")); + .unwrap_err() + .to_string(); + assert!(err.contains("No RSA key found in pem file"), "{err}"); // for application_credentials_path let sql = format!("CREATE EXTERNAL TABLE test STORED AS PARQUET diff --git a/datafusion/core/src/catalog/listing_schema.rs b/datafusion/core/src/catalog/listing_schema.rs index 7e527642be16..0d5c49f377d0 100644 --- a/datafusion/core/src/catalog/listing_schema.rs +++ b/datafusion/core/src/catalog/listing_schema.rs @@ -92,12 +92,7 @@ impl ListingSchemaProvider { /// Reload table information from ObjectStore pub async fn refresh(&self, state: &SessionState) -> datafusion_common::Result<()> { - let entries: Vec<_> = self - .store - .list(Some(&self.path)) - .await? - .try_collect() - .await?; + let entries: Vec<_> = self.store.list(Some(&self.path)).try_collect().await?; let base = Path::new(self.path.as_ref()); let mut tables = HashSet::new(); for file in entries.iter() { diff --git a/datafusion/core/src/datasource/file_format/arrow.rs b/datafusion/core/src/datasource/file_format/arrow.rs index a9bd7d0e27bb..07c96bdae1b4 100644 --- a/datafusion/core/src/datasource/file_format/arrow.rs +++ b/datafusion/core/src/datasource/file_format/arrow.rs @@ -214,6 +214,7 @@ mod tests { last_modified: DateTime::default(), size: usize::MAX, e_tag: None, + version: None, }; let arrow_format = ArrowFormat {}; @@ -256,6 +257,7 @@ mod tests { last_modified: DateTime::default(), size: usize::MAX, e_tag: None, + version: None, }; let arrow_format = ArrowFormat {}; diff --git a/datafusion/core/src/datasource/file_format/csv.rs b/datafusion/core/src/datasource/file_format/csv.rs index 684f416f771a..df6689af6b73 100644 --- a/datafusion/core/src/datasource/file_format/csv.rs +++ b/datafusion/core/src/datasource/file_format/csv.rs @@ -673,6 +673,7 @@ mod tests { last_modified: DateTime::default(), size: usize::MAX, e_tag: None, + version: None, }; let num_rows_to_read = 100; diff --git a/datafusion/core/src/datasource/file_format/mod.rs b/datafusion/core/src/datasource/file_format/mod.rs index b541e2a1d44c..7c2331548e5e 100644 --- a/datafusion/core/src/datasource/file_format/mod.rs +++ b/datafusion/core/src/datasource/file_format/mod.rs @@ -124,7 +124,8 @@ pub(crate) mod test_util { use object_store::local::LocalFileSystem; use object_store::path::Path; use object_store::{ - GetOptions, GetResult, GetResultPayload, ListResult, MultipartId, + GetOptions, GetResult, GetResultPayload, ListResult, MultipartId, PutOptions, + PutResult, }; use tokio::io::AsyncWrite; @@ -189,7 +190,12 @@ pub(crate) mod test_util { #[async_trait] impl ObjectStore for VariableStream { - async fn put(&self, _location: &Path, _bytes: Bytes) -> object_store::Result<()> { + async fn put_opts( + &self, + _location: &Path, + _bytes: Bytes, + _opts: PutOptions, + ) -> object_store::Result { unimplemented!() } @@ -228,6 +234,7 @@ pub(crate) mod test_util { last_modified: Default::default(), size: range.end, e_tag: None, + version: None, }, range: Default::default(), }) @@ -257,11 +264,10 @@ pub(crate) mod test_util { unimplemented!() } - async fn list( + fn list( &self, _prefix: Option<&Path>, - ) -> object_store::Result>> - { + ) -> BoxStream<'_, object_store::Result> { unimplemented!() } diff --git a/datafusion/core/src/datasource/file_format/parquet.rs b/datafusion/core/src/datasource/file_format/parquet.rs index c4d05adfc6bc..cf6b87408107 100644 --- a/datafusion/core/src/datasource/file_format/parquet.rs +++ b/datafusion/core/src/datasource/file_format/parquet.rs @@ -1199,7 +1199,9 @@ mod tests { use log::error; use object_store::local::LocalFileSystem; use object_store::path::Path; - use object_store::{GetOptions, GetResult, ListResult, MultipartId}; + use object_store::{ + GetOptions, GetResult, ListResult, MultipartId, PutOptions, PutResult, + }; use parquet::arrow::arrow_reader::ArrowReaderOptions; use parquet::arrow::ParquetRecordBatchStreamBuilder; use parquet::file::metadata::{ParquetColumnIndex, ParquetOffsetIndex}; @@ -1283,7 +1285,12 @@ mod tests { #[async_trait] impl ObjectStore for RequestCountingObjectStore { - async fn put(&self, _location: &Path, _bytes: Bytes) -> object_store::Result<()> { + async fn put_opts( + &self, + _location: &Path, + _bytes: Bytes, + _opts: PutOptions, + ) -> object_store::Result { Err(object_store::Error::NotImplemented) } @@ -1320,12 +1327,13 @@ mod tests { Err(object_store::Error::NotImplemented) } - async fn list( + fn list( &self, _prefix: Option<&Path>, - ) -> object_store::Result>> - { - Err(object_store::Error::NotImplemented) + ) -> BoxStream<'_, object_store::Result> { + Box::pin(futures::stream::once(async { + Err(object_store::Error::NotImplemented) + })) } async fn list_with_delimiter( diff --git a/datafusion/core/src/datasource/listing/helpers.rs b/datafusion/core/src/datasource/listing/helpers.rs index 3d2a3dc928b6..322d65d5645d 100644 --- a/datafusion/core/src/datasource/listing/helpers.rs +++ b/datafusion/core/src/datasource/listing/helpers.rs @@ -361,8 +361,7 @@ pub async fn pruned_partition_list<'a>( Some(files) => files, None => { trace!("Recursively listing partition {}", partition.path); - let s = store.list(Some(&partition.path)).await?; - s.try_collect().await? + store.list(Some(&partition.path)).try_collect().await? } }; diff --git a/datafusion/core/src/datasource/listing/mod.rs b/datafusion/core/src/datasource/listing/mod.rs index aa2e20164b5e..87c1663ae718 100644 --- a/datafusion/core/src/datasource/listing/mod.rs +++ b/datafusion/core/src/datasource/listing/mod.rs @@ -80,6 +80,7 @@ impl PartitionedFile { last_modified: chrono::Utc.timestamp_nanos(0), size: size as usize, e_tag: None, + version: None, }, partition_values: vec![], range: None, @@ -95,6 +96,7 @@ impl PartitionedFile { last_modified: chrono::Utc.timestamp_nanos(0), size: size as usize, e_tag: None, + version: None, }, partition_values: vec![], range: Some(FileRange { start, end }), diff --git a/datafusion/core/src/datasource/listing/url.rs b/datafusion/core/src/datasource/listing/url.rs index ba3c3fae21e2..45845916a971 100644 --- a/datafusion/core/src/datasource/listing/url.rs +++ b/datafusion/core/src/datasource/listing/url.rs @@ -210,17 +210,15 @@ impl ListingTableUrl { // If the prefix is a file, use a head request, otherwise list let list = match self.is_collection() { true => match ctx.runtime_env().cache_manager.get_list_files_cache() { - None => futures::stream::once(store.list(Some(&self.prefix))) - .try_flatten() - .boxed(), + None => store.list(Some(&self.prefix)), Some(cache) => { if let Some(res) = cache.get(&self.prefix) { debug!("Hit list all files cache"); futures::stream::iter(res.as_ref().clone().into_iter().map(Ok)) .boxed() } else { - let list_res = store.list(Some(&self.prefix)).await; - let vec = list_res?.try_collect::>().await?; + let list_res = store.list(Some(&self.prefix)); + let vec = list_res.try_collect::>().await?; cache.put(&self.prefix, Arc::new(vec.clone())); futures::stream::iter(vec.into_iter().map(Ok)).boxed() } @@ -330,8 +328,8 @@ mod tests { let url = ListingTableUrl::parse("file:///foo/bar?").unwrap(); assert_eq!(url.prefix.as_ref(), "foo/bar"); - let err = ListingTableUrl::parse("file:///foo/😺").unwrap_err(); - assert_eq!(err.to_string(), "Object Store error: Encountered object with invalid path: Error parsing Path \"/foo/😺\": Encountered illegal character sequence \"😺\" whilst parsing path segment \"😺\""); + let url = ListingTableUrl::parse("file:///foo/😺").unwrap(); + assert_eq!(url.prefix.as_ref(), "foo/😺"); let url = ListingTableUrl::parse("file:///foo/bar%2Efoo").unwrap(); assert_eq!(url.prefix.as_ref(), "foo/bar.foo"); diff --git a/datafusion/core/src/datasource/physical_plan/mod.rs b/datafusion/core/src/datasource/physical_plan/mod.rs index 738e70966bce..aca71678d98b 100644 --- a/datafusion/core/src/datasource/physical_plan/mod.rs +++ b/datafusion/core/src/datasource/physical_plan/mod.rs @@ -784,6 +784,7 @@ mod tests { last_modified: Utc::now(), size: 42, e_tag: None, + version: None, }; PartitionedFile { diff --git a/datafusion/core/src/datasource/physical_plan/parquet.rs b/datafusion/core/src/datasource/physical_plan/parquet.rs index 960b2ec7337d..731672ceb8b8 100644 --- a/datafusion/core/src/datasource/physical_plan/parquet.rs +++ b/datafusion/core/src/datasource/physical_plan/parquet.rs @@ -1718,6 +1718,7 @@ mod tests { last_modified: Utc.timestamp_nanos(0), size: 1337, e_tag: None, + version: None, }, partition_values: vec![], range: None, diff --git a/datafusion/core/src/datasource/physical_plan/parquet/row_groups.rs b/datafusion/core/src/datasource/physical_plan/parquet/row_groups.rs index dc6ef50bc101..0079368f9cdd 100644 --- a/datafusion/core/src/datasource/physical_plan/parquet/row_groups.rs +++ b/datafusion/core/src/datasource/physical_plan/parquet/row_groups.rs @@ -1243,6 +1243,7 @@ mod tests { last_modified: chrono::DateTime::from(std::time::SystemTime::now()), size: data.len(), e_tag: None, + version: None, }; let in_memory = object_store::memory::InMemory::new(); in_memory diff --git a/datafusion/core/src/test/object_store.rs b/datafusion/core/src/test/object_store.rs index 08cebb56cc77..d6f324a7f1f9 100644 --- a/datafusion/core/src/test/object_store.rs +++ b/datafusion/core/src/test/object_store.rs @@ -61,5 +61,6 @@ pub fn local_unpartitioned_file(path: impl AsRef) -> ObjectMeta last_modified: metadata.modified().map(chrono::DateTime::from).unwrap(), size: metadata.len() as usize, e_tag: None, + version: None, } } diff --git a/datafusion/core/src/test_util/parquet.rs b/datafusion/core/src/test_util/parquet.rs index 0d11526703b4..f3c0d2987a46 100644 --- a/datafusion/core/src/test_util/parquet.rs +++ b/datafusion/core/src/test_util/parquet.rs @@ -113,6 +113,7 @@ impl TestParquetFile { last_modified: Default::default(), size, e_tag: None, + version: None, }; Ok(Self { diff --git a/datafusion/core/tests/parquet/custom_reader.rs b/datafusion/core/tests/parquet/custom_reader.rs index 37481b936d24..3752d42dbf43 100644 --- a/datafusion/core/tests/parquet/custom_reader.rs +++ b/datafusion/core/tests/parquet/custom_reader.rs @@ -188,6 +188,7 @@ async fn store_parquet_in_memory( last_modified: chrono::DateTime::from(SystemTime::now()), size: buf.len(), e_tag: None, + version: None, }; (meta, Bytes::from(buf)) diff --git a/datafusion/core/tests/parquet/page_pruning.rs b/datafusion/core/tests/parquet/page_pruning.rs index b77643c35e84..e1e8b8e66edd 100644 --- a/datafusion/core/tests/parquet/page_pruning.rs +++ b/datafusion/core/tests/parquet/page_pruning.rs @@ -50,6 +50,7 @@ async fn get_parquet_exec(state: &SessionState, filter: Expr) -> ParquetExec { last_modified: metadata.modified().map(chrono::DateTime::from).unwrap(), size: metadata.len() as usize, e_tag: None, + version: None, }; let schema = ParquetFormat::default() diff --git a/datafusion/core/tests/parquet/schema_coercion.rs b/datafusion/core/tests/parquet/schema_coercion.rs index b3134d470b56..25c62f18f5ba 100644 --- a/datafusion/core/tests/parquet/schema_coercion.rs +++ b/datafusion/core/tests/parquet/schema_coercion.rs @@ -194,5 +194,6 @@ pub fn local_unpartitioned_file(path: impl AsRef) -> ObjectMeta last_modified: metadata.modified().map(chrono::DateTime::from).unwrap(), size: metadata.len() as usize, e_tag: None, + version: None, } } diff --git a/datafusion/core/tests/path_partition.rs b/datafusion/core/tests/path_partition.rs index 27d146de798d..dd8eb52f67c7 100644 --- a/datafusion/core/tests/path_partition.rs +++ b/datafusion/core/tests/path_partition.rs @@ -46,7 +46,7 @@ use futures::stream; use futures::stream::BoxStream; use object_store::{ path::Path, GetOptions, GetResult, GetResultPayload, ListResult, MultipartId, - ObjectMeta, ObjectStore, + ObjectMeta, ObjectStore, PutOptions, PutResult, }; use tokio::io::AsyncWrite; use url::Url; @@ -620,7 +620,12 @@ impl MirroringObjectStore { #[async_trait] impl ObjectStore for MirroringObjectStore { - async fn put(&self, _location: &Path, _bytes: Bytes) -> object_store::Result<()> { + async fn put_opts( + &self, + _location: &Path, + _bytes: Bytes, + _opts: PutOptions, + ) -> object_store::Result { unimplemented!() } @@ -653,6 +658,7 @@ impl ObjectStore for MirroringObjectStore { last_modified: metadata.modified().map(chrono::DateTime::from).unwrap(), size: metadata.len() as usize, e_tag: None, + version: None, }; Ok(GetResult { @@ -680,26 +686,16 @@ impl ObjectStore for MirroringObjectStore { Ok(data.into()) } - async fn head(&self, location: &Path) -> object_store::Result { - self.files.iter().find(|x| *x == location).unwrap(); - Ok(ObjectMeta { - location: location.clone(), - last_modified: Utc.timestamp_nanos(0), - size: self.file_size as usize, - e_tag: None, - }) - } - async fn delete(&self, _location: &Path) -> object_store::Result<()> { unimplemented!() } - async fn list( + fn list( &self, prefix: Option<&Path>, - ) -> object_store::Result>> { + ) -> BoxStream<'_, object_store::Result> { let prefix = prefix.cloned().unwrap_or_default(); - Ok(Box::pin(stream::iter(self.files.iter().filter_map( + Box::pin(stream::iter(self.files.iter().filter_map( move |location| { // Don't return for exact prefix match let filter = location @@ -713,10 +709,11 @@ impl ObjectStore for MirroringObjectStore { last_modified: Utc.timestamp_nanos(0), size: self.file_size as usize, e_tag: None, + version: None, }) }) }, - )))) + ))) } async fn list_with_delimiter( @@ -750,6 +747,7 @@ impl ObjectStore for MirroringObjectStore { last_modified: Utc.timestamp_nanos(0), size: self.file_size as usize, e_tag: None, + version: None, }; objects.push(object); } diff --git a/datafusion/execution/src/cache/cache_unit.rs b/datafusion/execution/src/cache/cache_unit.rs index 4a21dc02bd13..c54839061c8a 100644 --- a/datafusion/execution/src/cache/cache_unit.rs +++ b/datafusion/execution/src/cache/cache_unit.rs @@ -176,6 +176,7 @@ mod tests { .into(), size: 1024, e_tag: None, + version: None, }; let cache = DefaultFileStatisticsCache::default(); assert!(cache.get_with_extra(&meta.location, &meta).is_none()); @@ -219,6 +220,7 @@ mod tests { .into(), size: 1024, e_tag: None, + version: None, }; let cache = DefaultListFilesCache::default(); diff --git a/datafusion/physical-expr/src/expressions/cast.rs b/datafusion/physical-expr/src/expressions/cast.rs index 780e042156b8..cbc82cc77628 100644 --- a/datafusion/physical-expr/src/expressions/cast.rs +++ b/datafusion/physical-expr/src/expressions/cast.rs @@ -680,7 +680,11 @@ mod tests { // Ensure a useful error happens at plan time if invalid casts are used let schema = Schema::new(vec![Field::new("a", DataType::Int32, false)]); - let result = cast(col("a", &schema).unwrap(), &schema, DataType::LargeBinary); + let result = cast( + col("a", &schema).unwrap(), + &schema, + DataType::Interval(IntervalUnit::MonthDayNano), + ); result.expect_err("expected Invalid CAST"); } diff --git a/datafusion/physical-expr/src/expressions/try_cast.rs b/datafusion/physical-expr/src/expressions/try_cast.rs index dea7f9f86a62..0f7909097a10 100644 --- a/datafusion/physical-expr/src/expressions/try_cast.rs +++ b/datafusion/physical-expr/src/expressions/try_cast.rs @@ -555,7 +555,11 @@ mod tests { // Ensure a useful error happens at plan time if invalid casts are used let schema = Schema::new(vec![Field::new("a", DataType::Int32, false)]); - let result = try_cast(col("a", &schema).unwrap(), &schema, DataType::LargeBinary); + let result = try_cast( + col("a", &schema).unwrap(), + &schema, + DataType::Interval(IntervalUnit::MonthDayNano), + ); result.expect_err("expected Invalid TRY_CAST"); } diff --git a/datafusion/proto/src/physical_plan/from_proto.rs b/datafusion/proto/src/physical_plan/from_proto.rs index f5771ddb155b..dcebfbf2dabb 100644 --- a/datafusion/proto/src/physical_plan/from_proto.rs +++ b/datafusion/proto/src/physical_plan/from_proto.rs @@ -540,6 +540,7 @@ impl TryFrom<&protobuf::PartitionedFile> for PartitionedFile { last_modified: Utc.timestamp_nanos(val.last_modified_ns as i64), size: val.size as usize, e_tag: None, + version: None, }, partition_values: val .partition_values diff --git a/datafusion/sqllogictest/test_files/copy.slt b/datafusion/sqllogictest/test_files/copy.slt index fbf1523477b1..02ab33083315 100644 --- a/datafusion/sqllogictest/test_files/copy.slt +++ b/datafusion/sqllogictest/test_files/copy.slt @@ -66,8 +66,8 @@ select * from validate_parquet; # Copy parquet with all supported statment overrides query IT -COPY source_table -TO 'test_files/scratch/copy/table_with_options' +COPY source_table +TO 'test_files/scratch/copy/table_with_options' (format parquet, single_file_output false, compression snappy, @@ -206,11 +206,11 @@ select * from validate_single_json; # COPY csv files with all options set query IT -COPY source_table -to 'test_files/scratch/copy/table_csv_with_options' -(format csv, -single_file_output false, -header false, +COPY source_table +to 'test_files/scratch/copy/table_csv_with_options' +(format csv, +single_file_output false, +header false, compression 'uncompressed', datetime_format '%FT%H:%M:%S.%9f', delimiter ';', @@ -220,8 +220,8 @@ null_value 'NULLVAL'); # Validate single csv output statement ok -CREATE EXTERNAL TABLE validate_csv_with_options -STORED AS csv +CREATE EXTERNAL TABLE validate_csv_with_options +STORED AS csv LOCATION 'test_files/scratch/copy/table_csv_with_options'; query T diff --git a/datafusion/substrait/src/physical_plan/consumer.rs b/datafusion/substrait/src/physical_plan/consumer.rs index 1dab1f9d5e39..942798173e0e 100644 --- a/datafusion/substrait/src/physical_plan/consumer.rs +++ b/datafusion/substrait/src/physical_plan/consumer.rs @@ -89,6 +89,7 @@ pub async fn from_substrait_rel( location: path.into(), size, e_tag: None, + version: None, }, partition_values: vec![], range: None, From 76ced31429a4e324f9f57cb3e521e75739171e38 Mon Sep 17 00:00:00 2001 From: Huaijin Date: Sat, 18 Nov 2023 18:58:17 +0800 Subject: [PATCH 289/572] feat: impl the basic `string_agg` function (#8148) * init impl * add support for larget utf8 * add some test * support null * remove redundance code * remove redundance code * add more test * Update datafusion/physical-expr/src/aggregate/string_agg.rs Co-authored-by: universalmind303 * Update datafusion/physical-expr/src/aggregate/string_agg.rs Co-authored-by: universalmind303 * add suggest * Update datafusion/physical-expr/src/aggregate/string_agg.rs Co-authored-by: Andrew Lamb * Update datafusion/sqllogictest/test_files/aggregate.slt Co-authored-by: Andrew Lamb * Update datafusion/sqllogictest/test_files/aggregate.slt Co-authored-by: Andrew Lamb * fix ci --------- Co-authored-by: universalmind303 Co-authored-by: Andrew Lamb --- datafusion/expr/src/aggregate_function.rs | 8 + .../expr/src/type_coercion/aggregates.rs | 26 ++ .../physical-expr/src/aggregate/build_in.rs | 16 ++ datafusion/physical-expr/src/aggregate/mod.rs | 1 + .../physical-expr/src/aggregate/string_agg.rs | 246 ++++++++++++++++++ .../physical-expr/src/expressions/mod.rs | 1 + datafusion/proto/proto/datafusion.proto | 1 + datafusion/proto/src/generated/pbjson.rs | 3 + datafusion/proto/src/generated/prost.rs | 3 + .../proto/src/logical_plan/from_proto.rs | 1 + datafusion/proto/src/logical_plan/to_proto.rs | 4 + .../sqllogictest/test_files/aggregate.slt | 76 ++++++ 12 files changed, 386 insertions(+) create mode 100644 datafusion/physical-expr/src/aggregate/string_agg.rs diff --git a/datafusion/expr/src/aggregate_function.rs b/datafusion/expr/src/aggregate_function.rs index ea0b01825170..4611c7fb10d7 100644 --- a/datafusion/expr/src/aggregate_function.rs +++ b/datafusion/expr/src/aggregate_function.rs @@ -100,6 +100,8 @@ pub enum AggregateFunction { BoolAnd, /// Bool Or BoolOr, + /// string_agg + StringAgg, } impl AggregateFunction { @@ -141,6 +143,7 @@ impl AggregateFunction { BitXor => "BIT_XOR", BoolAnd => "BOOL_AND", BoolOr => "BOOL_OR", + StringAgg => "STRING_AGG", } } } @@ -171,6 +174,7 @@ impl FromStr for AggregateFunction { "array_agg" => AggregateFunction::ArrayAgg, "first_value" => AggregateFunction::FirstValue, "last_value" => AggregateFunction::LastValue, + "string_agg" => AggregateFunction::StringAgg, // statistical "corr" => AggregateFunction::Correlation, "covar" => AggregateFunction::Covariance, @@ -299,6 +303,7 @@ impl AggregateFunction { AggregateFunction::FirstValue | AggregateFunction::LastValue => { Ok(coerced_data_types[0].clone()) } + AggregateFunction::StringAgg => Ok(DataType::LargeUtf8), } } } @@ -408,6 +413,9 @@ impl AggregateFunction { .collect(), Volatility::Immutable, ), + AggregateFunction::StringAgg => { + Signature::uniform(2, STRINGS.to_vec(), Volatility::Immutable) + } } } } diff --git a/datafusion/expr/src/type_coercion/aggregates.rs b/datafusion/expr/src/type_coercion/aggregates.rs index 261c406d5d5e..7128b575978a 100644 --- a/datafusion/expr/src/type_coercion/aggregates.rs +++ b/datafusion/expr/src/type_coercion/aggregates.rs @@ -298,6 +298,23 @@ pub fn coerce_types( | AggregateFunction::FirstValue | AggregateFunction::LastValue => Ok(input_types.to_vec()), AggregateFunction::Grouping => Ok(vec![input_types[0].clone()]), + AggregateFunction::StringAgg => { + if !is_string_agg_supported_arg_type(&input_types[0]) { + return plan_err!( + "The function {:?} does not support inputs of type {:?}", + agg_fun, + input_types[0] + ); + } + if !is_string_agg_supported_arg_type(&input_types[1]) { + return plan_err!( + "The function {:?} does not support inputs of type {:?}", + agg_fun, + input_types[1] + ); + } + Ok(vec![LargeUtf8, input_types[1].clone()]) + } } } @@ -565,6 +582,15 @@ pub fn is_approx_percentile_cont_supported_arg_type(arg_type: &DataType) -> bool ) } +/// Return `true` if `arg_type` is of a [`DataType`] that the +/// [`AggregateFunction::StringAgg`] aggregation can operate on. +pub fn is_string_agg_supported_arg_type(arg_type: &DataType) -> bool { + matches!( + arg_type, + DataType::Utf8 | DataType::LargeUtf8 | DataType::Null + ) +} + #[cfg(test)] mod tests { use super::*; diff --git a/datafusion/physical-expr/src/aggregate/build_in.rs b/datafusion/physical-expr/src/aggregate/build_in.rs index 596197b4eebe..c40f0db19405 100644 --- a/datafusion/physical-expr/src/aggregate/build_in.rs +++ b/datafusion/physical-expr/src/aggregate/build_in.rs @@ -369,6 +369,22 @@ pub fn create_aggregate_expr( ordering_req.to_vec(), ordering_types, )), + (AggregateFunction::StringAgg, false) => { + if !ordering_req.is_empty() { + return not_impl_err!( + "STRING_AGG(ORDER BY a ASC) order-sensitive aggregations are not available" + ); + } + Arc::new(expressions::StringAgg::new( + input_phy_exprs[0].clone(), + input_phy_exprs[1].clone(), + name, + data_type, + )) + } + (AggregateFunction::StringAgg, true) => { + return not_impl_err!("STRING_AGG(DISTINCT) aggregations are not available"); + } }) } diff --git a/datafusion/physical-expr/src/aggregate/mod.rs b/datafusion/physical-expr/src/aggregate/mod.rs index 442d018b87d5..329bb1e6415e 100644 --- a/datafusion/physical-expr/src/aggregate/mod.rs +++ b/datafusion/physical-expr/src/aggregate/mod.rs @@ -43,6 +43,7 @@ pub(crate) mod covariance; pub(crate) mod first_last; pub(crate) mod grouping; pub(crate) mod median; +pub(crate) mod string_agg; #[macro_use] pub(crate) mod min_max; pub mod build_in; diff --git a/datafusion/physical-expr/src/aggregate/string_agg.rs b/datafusion/physical-expr/src/aggregate/string_agg.rs new file mode 100644 index 000000000000..74c083959ed8 --- /dev/null +++ b/datafusion/physical-expr/src/aggregate/string_agg.rs @@ -0,0 +1,246 @@ +// 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. + +//! [`StringAgg`] and [`StringAggAccumulator`] accumulator for the `string_agg` function + +use crate::aggregate::utils::down_cast_any_ref; +use crate::expressions::{format_state_name, Literal}; +use crate::{AggregateExpr, PhysicalExpr}; +use arrow::array::ArrayRef; +use arrow::datatypes::{DataType, Field}; +use datafusion_common::cast::as_generic_string_array; +use datafusion_common::{not_impl_err, DataFusionError, Result, ScalarValue}; +use datafusion_expr::Accumulator; +use std::any::Any; +use std::sync::Arc; + +/// STRING_AGG aggregate expression +#[derive(Debug)] +pub struct StringAgg { + name: String, + data_type: DataType, + expr: Arc, + delimiter: Arc, + nullable: bool, +} + +impl StringAgg { + /// Create a new StringAgg aggregate function + pub fn new( + expr: Arc, + delimiter: Arc, + name: impl Into, + data_type: DataType, + ) -> Self { + Self { + name: name.into(), + data_type, + delimiter, + expr, + nullable: true, + } + } +} + +impl AggregateExpr for StringAgg { + fn as_any(&self) -> &dyn Any { + self + } + + fn field(&self) -> Result { + Ok(Field::new( + &self.name, + self.data_type.clone(), + self.nullable, + )) + } + + fn create_accumulator(&self) -> Result> { + if let Some(delimiter) = self.delimiter.as_any().downcast_ref::() { + match delimiter.value() { + ScalarValue::Utf8(Some(delimiter)) + | ScalarValue::LargeUtf8(Some(delimiter)) => { + return Ok(Box::new(StringAggAccumulator::new(delimiter))); + } + ScalarValue::Null => { + return Ok(Box::new(StringAggAccumulator::new(""))); + } + _ => return not_impl_err!("StringAgg not supported for {}", self.name), + } + } + not_impl_err!("StringAgg not supported for {}", self.name) + } + + fn state_fields(&self) -> Result> { + Ok(vec![Field::new( + format_state_name(&self.name, "string_agg"), + self.data_type.clone(), + self.nullable, + )]) + } + + fn expressions(&self) -> Vec> { + vec![self.expr.clone(), self.delimiter.clone()] + } + + fn name(&self) -> &str { + &self.name + } +} + +impl PartialEq for StringAgg { + fn eq(&self, other: &dyn Any) -> bool { + down_cast_any_ref(other) + .downcast_ref::() + .map(|x| { + self.name == x.name + && self.data_type == x.data_type + && self.expr.eq(&x.expr) + && self.delimiter.eq(&x.delimiter) + }) + .unwrap_or(false) + } +} + +#[derive(Debug)] +pub(crate) struct StringAggAccumulator { + values: Option, + delimiter: String, +} + +impl StringAggAccumulator { + pub fn new(delimiter: &str) -> Self { + Self { + values: None, + delimiter: delimiter.to_string(), + } + } +} + +impl Accumulator for StringAggAccumulator { + fn update_batch(&mut self, values: &[ArrayRef]) -> Result<()> { + let string_array: Vec<_> = as_generic_string_array::(&values[0])? + .iter() + .filter_map(|v| v.as_ref().map(ToString::to_string)) + .collect(); + if !string_array.is_empty() { + let s = string_array.join(self.delimiter.as_str()); + let v = self.values.get_or_insert("".to_string()); + if !v.is_empty() { + v.push_str(self.delimiter.as_str()); + } + v.push_str(s.as_str()); + } + Ok(()) + } + + fn merge_batch(&mut self, values: &[ArrayRef]) -> Result<()> { + self.update_batch(values)?; + Ok(()) + } + + fn state(&self) -> Result> { + Ok(vec![self.evaluate()?]) + } + + fn evaluate(&self) -> Result { + Ok(ScalarValue::LargeUtf8(self.values.clone())) + } + + fn size(&self) -> usize { + std::mem::size_of_val(self) + + self.values.as_ref().map(|v| v.capacity()).unwrap_or(0) + + self.delimiter.capacity() + } +} + +#[cfg(test)] +mod tests { + use super::*; + use crate::expressions::tests::aggregate; + use crate::expressions::{col, create_aggregate_expr, try_cast}; + use arrow::array::ArrayRef; + use arrow::datatypes::*; + use arrow::record_batch::RecordBatch; + use arrow_array::LargeStringArray; + use arrow_array::StringArray; + use datafusion_expr::type_coercion::aggregates::coerce_types; + use datafusion_expr::AggregateFunction; + + fn assert_string_aggregate( + array: ArrayRef, + function: AggregateFunction, + distinct: bool, + expected: ScalarValue, + delimiter: String, + ) { + let data_type = array.data_type(); + let sig = function.signature(); + let coerced = + coerce_types(&function, &[data_type.clone(), DataType::Utf8], &sig).unwrap(); + + let input_schema = Schema::new(vec![Field::new("a", data_type.clone(), true)]); + let batch = + RecordBatch::try_new(Arc::new(input_schema.clone()), vec![array]).unwrap(); + + let input = try_cast( + col("a", &input_schema).unwrap(), + &input_schema, + coerced[0].clone(), + ) + .unwrap(); + + let delimiter = Arc::new(Literal::new(ScalarValue::Utf8(Some(delimiter)))); + let schema = Schema::new(vec![Field::new("a", coerced[0].clone(), true)]); + let agg = create_aggregate_expr( + &function, + distinct, + &[input, delimiter], + &[], + &schema, + "agg", + ) + .unwrap(); + + let result = aggregate(&batch, agg).unwrap(); + assert_eq!(expected, result); + } + + #[test] + fn string_agg_utf8() { + let a: ArrayRef = Arc::new(StringArray::from(vec!["h", "e", "l", "l", "o"])); + assert_string_aggregate( + a, + AggregateFunction::StringAgg, + false, + ScalarValue::LargeUtf8(Some("h,e,l,l,o".to_owned())), + ",".to_owned(), + ); + } + + #[test] + fn string_agg_largeutf8() { + let a: ArrayRef = Arc::new(LargeStringArray::from(vec!["h", "e", "l", "l", "o"])); + assert_string_aggregate( + a, + AggregateFunction::StringAgg, + false, + ScalarValue::LargeUtf8(Some("h|e|l|l|o".to_owned())), + "|".to_owned(), + ); + } +} diff --git a/datafusion/physical-expr/src/expressions/mod.rs b/datafusion/physical-expr/src/expressions/mod.rs index 1919cac97986..b6d0ad5b9104 100644 --- a/datafusion/physical-expr/src/expressions/mod.rs +++ b/datafusion/physical-expr/src/expressions/mod.rs @@ -63,6 +63,7 @@ pub use crate::aggregate::min_max::{MaxAccumulator, MinAccumulator}; pub use crate::aggregate::regr::{Regr, RegrType}; pub use crate::aggregate::stats::StatsType; pub use crate::aggregate::stddev::{Stddev, StddevPop}; +pub use crate::aggregate::string_agg::StringAgg; pub use crate::aggregate::sum::Sum; pub use crate::aggregate::sum_distinct::DistinctSum; pub use crate::aggregate::variance::{Variance, VariancePop}; diff --git a/datafusion/proto/proto/datafusion.proto b/datafusion/proto/proto/datafusion.proto index 750d12bd776d..9d508078c705 100644 --- a/datafusion/proto/proto/datafusion.proto +++ b/datafusion/proto/proto/datafusion.proto @@ -686,6 +686,7 @@ enum AggregateFunction { REGR_SXX = 32; REGR_SYY = 33; REGR_SXY = 34; + STRING_AGG = 35; } message AggregateExprNode { diff --git a/datafusion/proto/src/generated/pbjson.rs b/datafusion/proto/src/generated/pbjson.rs index af64bd68deb1..0a8f415e20c5 100644 --- a/datafusion/proto/src/generated/pbjson.rs +++ b/datafusion/proto/src/generated/pbjson.rs @@ -474,6 +474,7 @@ impl serde::Serialize for AggregateFunction { Self::RegrSxx => "REGR_SXX", Self::RegrSyy => "REGR_SYY", Self::RegrSxy => "REGR_SXY", + Self::StringAgg => "STRING_AGG", }; serializer.serialize_str(variant) } @@ -520,6 +521,7 @@ impl<'de> serde::Deserialize<'de> for AggregateFunction { "REGR_SXX", "REGR_SYY", "REGR_SXY", + "STRING_AGG", ]; struct GeneratedVisitor; @@ -595,6 +597,7 @@ impl<'de> serde::Deserialize<'de> for AggregateFunction { "REGR_SXX" => Ok(AggregateFunction::RegrSxx), "REGR_SYY" => Ok(AggregateFunction::RegrSyy), "REGR_SXY" => Ok(AggregateFunction::RegrSxy), + "STRING_AGG" => Ok(AggregateFunction::StringAgg), _ => Err(serde::de::Error::unknown_variant(value, FIELDS)), } } diff --git a/datafusion/proto/src/generated/prost.rs b/datafusion/proto/src/generated/prost.rs index b23f09e91b26..84fb84b9487e 100644 --- a/datafusion/proto/src/generated/prost.rs +++ b/datafusion/proto/src/generated/prost.rs @@ -2881,6 +2881,7 @@ pub enum AggregateFunction { RegrSxx = 32, RegrSyy = 33, RegrSxy = 34, + StringAgg = 35, } impl AggregateFunction { /// String value of the enum field names used in the ProtoBuf definition. @@ -2926,6 +2927,7 @@ impl AggregateFunction { AggregateFunction::RegrSxx => "REGR_SXX", AggregateFunction::RegrSyy => "REGR_SYY", AggregateFunction::RegrSxy => "REGR_SXY", + AggregateFunction::StringAgg => "STRING_AGG", } } /// Creates an enum from field names used in the ProtoBuf definition. @@ -2968,6 +2970,7 @@ impl AggregateFunction { "REGR_SXX" => Some(Self::RegrSxx), "REGR_SYY" => Some(Self::RegrSyy), "REGR_SXY" => Some(Self::RegrSxy), + "STRING_AGG" => Some(Self::StringAgg), _ => None, } } diff --git a/datafusion/proto/src/logical_plan/from_proto.rs b/datafusion/proto/src/logical_plan/from_proto.rs index f59a59f3c08b..4ae45fa52162 100644 --- a/datafusion/proto/src/logical_plan/from_proto.rs +++ b/datafusion/proto/src/logical_plan/from_proto.rs @@ -597,6 +597,7 @@ impl From for AggregateFunction { protobuf::AggregateFunction::Median => Self::Median, protobuf::AggregateFunction::FirstValueAgg => Self::FirstValue, protobuf::AggregateFunction::LastValueAgg => Self::LastValue, + protobuf::AggregateFunction::StringAgg => Self::StringAgg, } } } diff --git a/datafusion/proto/src/logical_plan/to_proto.rs b/datafusion/proto/src/logical_plan/to_proto.rs index 8bf42582360d..cf66e3ddd5b5 100644 --- a/datafusion/proto/src/logical_plan/to_proto.rs +++ b/datafusion/proto/src/logical_plan/to_proto.rs @@ -405,6 +405,7 @@ impl From<&AggregateFunction> for protobuf::AggregateFunction { AggregateFunction::Median => Self::Median, AggregateFunction::FirstValue => Self::FirstValueAgg, AggregateFunction::LastValue => Self::LastValueAgg, + AggregateFunction::StringAgg => Self::StringAgg, } } } @@ -721,6 +722,9 @@ impl TryFrom<&Expr> for protobuf::LogicalExprNode { AggregateFunction::LastValue => { protobuf::AggregateFunction::LastValueAgg } + AggregateFunction::StringAgg => { + protobuf::AggregateFunction::StringAgg + } }; let aggregate_expr = protobuf::AggregateExprNode { diff --git a/datafusion/sqllogictest/test_files/aggregate.slt b/datafusion/sqllogictest/test_files/aggregate.slt index a1bb93ed53c4..0a495dd2b0c9 100644 --- a/datafusion/sqllogictest/test_files/aggregate.slt +++ b/datafusion/sqllogictest/test_files/aggregate.slt @@ -2987,3 +2987,79 @@ NULL NULL 1 NULL 3 6 0 0 0 NULL NULL 1 NULL 5 15 0 0 0 3 0 2 1 5.5 16.5 0.5 4.5 1.5 3 0 3 1 6 18 2 18 6 + +statement error +SELECT STRING_AGG() + +statement error +SELECT STRING_AGG(1,2,3) + +statement error +SELECT STRING_AGG(STRING_AGG('a', ',')) + +query T +SELECT STRING_AGG('a', ',') +---- +a + +query TTTT +SELECT STRING_AGG('a',','), STRING_AGG('a', NULL), STRING_AGG(NULL, ','), STRING_AGG(NULL, NULL) +---- +a a NULL NULL + +query TT +select string_agg('', '|'), string_agg('a', ''); +---- +(empty) a + +query T +SELECT STRING_AGG(column1, '|') FROM (values (''), (null), ('')); +---- +| + +statement ok +CREATE TABLE strings(g INTEGER, x VARCHAR, y VARCHAR) + +query ITT +INSERT INTO strings VALUES (1,'a','/'), (1,'b','-'), (2,'i','/'), (2,NULL,'-'), (2,'j','+'), (3,'p','/'), (4,'x','/'), (4,'y','-'), (4,'z','+') +---- +9 + +query IT +SELECT g, STRING_AGG(x,'|') FROM strings GROUP BY g ORDER BY g +---- +1 a|b +2 i|j +3 p +4 x|y|z + +query T +SELECT STRING_AGG(x,',') FROM strings WHERE g > 100 +---- +NULL + +statement ok +drop table strings + +query T +WITH my_data as ( +SELECT 'text1'::varchar(1000) as my_column union all +SELECT 'text1'::varchar(1000) as my_column union all +SELECT 'text1'::varchar(1000) as my_column +) +SELECT string_agg(my_column,', ') as my_string_agg +FROM my_data +---- +text1, text1, text1 + +query T +WITH my_data as ( +SELECT 1 as dummy, 'text1'::varchar(1000) as my_column union all +SELECT 1 as dummy, 'text1'::varchar(1000) as my_column union all +SELECT 1 as dummy, 'text1'::varchar(1000) as my_column +) +SELECT string_agg(my_column,', ') as my_string_agg +FROM my_data +GROUP BY dummy +---- +text1, text1, text1 From 8f48053fc5f6fc3de27b69cd6f229558d8fc8990 Mon Sep 17 00:00:00 2001 From: Markus Appel Date: Sat, 18 Nov 2023 14:41:29 +0100 Subject: [PATCH 290/572] Minor: Make schema of grouping set columns nullable (#8248) * Make output schema of aggregation grouping sets nullable * Improve * Fix tests --- datafusion/expr/src/logical_plan/plan.rs | 56 +++++++++++++++++-- .../src/single_distinct_to_groupby.rs | 6 +- .../sqllogictest/test_files/aggregate.slt | 7 ++- 3 files changed, 57 insertions(+), 12 deletions(-) diff --git a/datafusion/expr/src/logical_plan/plan.rs b/datafusion/expr/src/logical_plan/plan.rs index b7537dc02e9d..a024824c7a5a 100644 --- a/datafusion/expr/src/logical_plan/plan.rs +++ b/datafusion/expr/src/logical_plan/plan.rs @@ -2294,13 +2294,25 @@ impl Aggregate { aggr_expr: Vec, ) -> Result { let group_expr = enumerate_grouping_sets(group_expr)?; + + let is_grouping_set = matches!(group_expr.as_slice(), [Expr::GroupingSet(_)]); + let grouping_expr: Vec = grouping_set_to_exprlist(group_expr.as_slice())?; - let all_expr = grouping_expr.iter().chain(aggr_expr.iter()); - let schema = DFSchema::new_with_metadata( - exprlist_to_fields(all_expr, &input)?, - input.schema().metadata().clone(), - )?; + let mut fields = exprlist_to_fields(grouping_expr.iter(), &input)?; + + // Even columns that cannot be null will become nullable when used in a grouping set. + if is_grouping_set { + fields = fields + .into_iter() + .map(|field| field.with_nullable(true)) + .collect::>(); + } + + fields.extend(exprlist_to_fields(aggr_expr.iter(), &input)?); + + let schema = + DFSchema::new_with_metadata(fields, input.schema().metadata().clone())?; Self::try_new_with_schema(input, group_expr, aggr_expr, Arc::new(schema)) } @@ -2539,7 +2551,7 @@ pub struct Unnest { mod tests { use super::*; use crate::logical_plan::table_scan; - use crate::{col, exists, in_subquery, lit, placeholder}; + use crate::{col, count, exists, in_subquery, lit, placeholder, GroupingSet}; use arrow::datatypes::{DataType, Field, Schema}; use datafusion_common::tree_node::TreeNodeVisitor; use datafusion_common::{not_impl_err, DFSchema, TableReference}; @@ -3006,4 +3018,36 @@ digraph { plan.replace_params_with_values(&[42i32.into()]) .expect_err("unexpectedly succeeded to replace an invalid placeholder"); } + + #[test] + fn test_nullable_schema_after_grouping_set() { + let schema = Schema::new(vec![ + Field::new("foo", DataType::Int32, false), + Field::new("bar", DataType::Int32, false), + ]); + + let plan = table_scan(TableReference::none(), &schema, None) + .unwrap() + .aggregate( + vec![Expr::GroupingSet(GroupingSet::GroupingSets(vec![ + vec![col("foo")], + vec![col("bar")], + ]))], + vec![count(lit(true))], + ) + .unwrap() + .build() + .unwrap(); + + let output_schema = plan.schema(); + + assert!(output_schema + .field_with_name(None, "foo") + .unwrap() + .is_nullable(),); + assert!(output_schema + .field_with_name(None, "bar") + .unwrap() + .is_nullable()); + } } diff --git a/datafusion/optimizer/src/single_distinct_to_groupby.rs b/datafusion/optimizer/src/single_distinct_to_groupby.rs index be76c069f0b7..ac18e596b7bd 100644 --- a/datafusion/optimizer/src/single_distinct_to_groupby.rs +++ b/datafusion/optimizer/src/single_distinct_to_groupby.rs @@ -322,7 +322,7 @@ mod tests { .build()?; // Should not be optimized - let expected = "Aggregate: groupBy=[[GROUPING SETS ((test.a), (test.b))]], aggr=[[COUNT(DISTINCT test.c)]] [a:UInt32, b:UInt32, COUNT(DISTINCT test.c):Int64;N]\ + let expected = "Aggregate: groupBy=[[GROUPING SETS ((test.a), (test.b))]], aggr=[[COUNT(DISTINCT test.c)]] [a:UInt32;N, b:UInt32;N, COUNT(DISTINCT test.c):Int64;N]\ \n TableScan: test [a:UInt32, b:UInt32, c:UInt32]"; assert_optimized_plan_equal(&plan, expected) @@ -340,7 +340,7 @@ mod tests { .build()?; // Should not be optimized - let expected = "Aggregate: groupBy=[[CUBE (test.a, test.b)]], aggr=[[COUNT(DISTINCT test.c)]] [a:UInt32, b:UInt32, COUNT(DISTINCT test.c):Int64;N]\ + let expected = "Aggregate: groupBy=[[CUBE (test.a, test.b)]], aggr=[[COUNT(DISTINCT test.c)]] [a:UInt32;N, b:UInt32;N, COUNT(DISTINCT test.c):Int64;N]\ \n TableScan: test [a:UInt32, b:UInt32, c:UInt32]"; assert_optimized_plan_equal(&plan, expected) @@ -359,7 +359,7 @@ mod tests { .build()?; // Should not be optimized - let expected = "Aggregate: groupBy=[[ROLLUP (test.a, test.b)]], aggr=[[COUNT(DISTINCT test.c)]] [a:UInt32, b:UInt32, COUNT(DISTINCT test.c):Int64;N]\ + let expected = "Aggregate: groupBy=[[ROLLUP (test.a, test.b)]], aggr=[[COUNT(DISTINCT test.c)]] [a:UInt32;N, b:UInt32;N, COUNT(DISTINCT test.c):Int64;N]\ \n TableScan: test [a:UInt32, b:UInt32, c:UInt32]"; assert_optimized_plan_equal(&plan, expected) diff --git a/datafusion/sqllogictest/test_files/aggregate.slt b/datafusion/sqllogictest/test_files/aggregate.slt index 0a495dd2b0c9..faad6feb3f33 100644 --- a/datafusion/sqllogictest/test_files/aggregate.slt +++ b/datafusion/sqllogictest/test_files/aggregate.slt @@ -2672,9 +2672,10 @@ query TT EXPLAIN SELECT c2, c3 FROM aggregate_test_100 group by rollup(c2, c3) limit 3; ---- logical_plan -Limit: skip=0, fetch=3 ---Aggregate: groupBy=[[ROLLUP (aggregate_test_100.c2, aggregate_test_100.c3)]], aggr=[[]] -----TableScan: aggregate_test_100 projection=[c2, c3] +Projection: aggregate_test_100.c2, aggregate_test_100.c3 +--Limit: skip=0, fetch=3 +----Aggregate: groupBy=[[ROLLUP (aggregate_test_100.c2, aggregate_test_100.c3)]], aggr=[[]] +------TableScan: aggregate_test_100 projection=[c2, c3] physical_plan GlobalLimitExec: skip=0, fetch=3 --AggregateExec: mode=Final, gby=[c2@0 as c2, c3@1 as c3], aggr=[], lim=[3] From 393e48f98872c696a90fce033fa584533d2326fa Mon Sep 17 00:00:00 2001 From: Will Jones Date: Sat, 18 Nov 2023 08:30:05 -0800 Subject: [PATCH 291/572] feat: support arbitrary binaryexpr simplifications (#8256) --- .../src/simplify_expressions/guarantees.rs | 73 +++++++++++-------- 1 file changed, 44 insertions(+), 29 deletions(-) diff --git a/datafusion/optimizer/src/simplify_expressions/guarantees.rs b/datafusion/optimizer/src/simplify_expressions/guarantees.rs index 5504d7d76e35..0204698571b4 100644 --- a/datafusion/optimizer/src/simplify_expressions/guarantees.rs +++ b/datafusion/optimizer/src/simplify_expressions/guarantees.rs @@ -20,7 +20,7 @@ //! [`ExprSimplifier::with_guarantees()`]: crate::simplify_expressions::expr_simplifier::ExprSimplifier::with_guarantees use datafusion_common::{tree_node::TreeNodeRewriter, DataFusionError, Result}; use datafusion_expr::{expr::InList, lit, Between, BinaryExpr, Expr}; -use std::collections::HashMap; +use std::{borrow::Cow, collections::HashMap}; use datafusion_physical_expr::intervals::{Interval, IntervalBound, NullableInterval}; @@ -103,37 +103,44 @@ impl<'a> TreeNodeRewriter for GuaranteeRewriter<'a> { } Expr::BinaryExpr(BinaryExpr { left, op, right }) => { - // We only support comparisons for now - if !op.is_comparison_operator() { - return Ok(expr); - }; - - // Check if this is a comparison between a column and literal - let (col, op, value) = match (left.as_ref(), right.as_ref()) { - (Expr::Column(_), Expr::Literal(value)) => (left, *op, value), - (Expr::Literal(value), Expr::Column(_)) => { - // If we can swap the op, we can simplify the expression - if let Some(op) = op.swap() { - (right, op, value) + // The left or right side of expression might either have a guarantee + // or be a literal. Either way, we can resolve them to a NullableInterval. + let left_interval = self + .guarantees + .get(left.as_ref()) + .map(|interval| Cow::Borrowed(*interval)) + .or_else(|| { + if let Expr::Literal(value) = left.as_ref() { + Some(Cow::Owned(value.clone().into())) } else { - return Ok(expr); + None + } + }); + let right_interval = self + .guarantees + .get(right.as_ref()) + .map(|interval| Cow::Borrowed(*interval)) + .or_else(|| { + if let Expr::Literal(value) = right.as_ref() { + Some(Cow::Owned(value.clone().into())) + } else { + None + } + }); + + match (left_interval, right_interval) { + (Some(left_interval), Some(right_interval)) => { + let result = + left_interval.apply_operator(op, right_interval.as_ref())?; + if result.is_certainly_true() { + Ok(lit(true)) + } else if result.is_certainly_false() { + Ok(lit(false)) + } else { + Ok(expr) } } - _ => return Ok(expr), - }; - - if let Some(col_interval) = self.guarantees.get(col.as_ref()) { - let result = - col_interval.apply_operator(&op, &value.clone().into())?; - if result.is_certainly_true() { - Ok(lit(true)) - } else if result.is_certainly_false() { - Ok(lit(false)) - } else { - Ok(expr) - } - } else { - Ok(expr) + _ => Ok(expr), } } @@ -262,6 +269,13 @@ mod tests { values: Interval::make(Some(1_i32), Some(3_i32), (true, false)), }, ), + // s.y ∈ (1, 3] (not null) + ( + col("s").field("y"), + NullableInterval::NotNull { + values: Interval::make(Some(1_i32), Some(3_i32), (true, false)), + }, + ), ]; let mut rewriter = GuaranteeRewriter::new(guarantees.iter()); @@ -269,6 +283,7 @@ mod tests { // (original_expr, expected_simplification) let simplified_cases = &[ (col("x").lt_eq(lit(1)), false), + (col("s").field("y").lt_eq(lit(1)), false), (col("x").lt_eq(lit(3)), true), (col("x").gt(lit(3)), false), (col("x").gt(lit(1)), true), From 2156dde54623d26635d4388d161d94ac79918cdc Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Metehan=20Y=C4=B1ld=C4=B1r=C4=B1m?= <100111937+metesynnada@users.noreply.github.com> Date: Mon, 20 Nov 2023 13:40:25 +0200 Subject: [PATCH 292/572] Making stream joins extensible: A new Trait implementation for SHJ (#8234) * Upstream * Update utils.rs * Review * Name change and remove ignore on test * Comment revisions * Improve comments --------- Co-authored-by: Mehmet Ozan Kabak --- .../physical-plan/src/joins/hash_join.rs | 3 +- datafusion/physical-plan/src/joins/mod.rs | 2 +- ...ash_join_utils.rs => stream_join_utils.rs} | 550 +++++++++++++----- .../src/joins/symmetric_hash_join.rs | 503 +++++++++------- .../physical-plan/src/joins/test_utils.rs | 61 +- datafusion/physical-plan/src/joins/utils.rs | 131 ++++- datafusion/proto/proto/datafusion.proto | 16 + datafusion/proto/src/generated/pbjson.rs | 285 +++++++++ datafusion/proto/src/generated/prost.rs | 48 +- datafusion/proto/src/physical_plan/mod.rs | 168 +++++- .../tests/cases/roundtrip_physical_plan.rs | 46 +- 11 files changed, 1463 insertions(+), 350 deletions(-) rename datafusion/physical-plan/src/joins/{hash_join_utils.rs => stream_join_utils.rs} (67%) diff --git a/datafusion/physical-plan/src/joins/hash_join.rs b/datafusion/physical-plan/src/joins/hash_join.rs index 7a08b56a6ea7..4846d0a5e046 100644 --- a/datafusion/physical-plan/src/joins/hash_join.rs +++ b/datafusion/physical-plan/src/joins/hash_join.rs @@ -26,7 +26,7 @@ use std::{any::Any, usize, vec}; use crate::joins::utils::{ adjust_indices_by_join_type, apply_join_filter_to_indices, build_batch_from_indices, calculate_join_output_ordering, get_final_indices_from_bit_map, - need_produce_result_in_final, + need_produce_result_in_final, JoinHashMap, JoinHashMapType, }; use crate::DisplayAs; use crate::{ @@ -35,7 +35,6 @@ use crate::{ expressions::Column, expressions::PhysicalSortExpr, hash_utils::create_hashes, - joins::hash_join_utils::{JoinHashMap, JoinHashMapType}, joins::utils::{ adjust_right_output_partitioning, build_join_schema, check_join_is_valid, estimate_join_statistics, partitioned_join_output_partitioning, diff --git a/datafusion/physical-plan/src/joins/mod.rs b/datafusion/physical-plan/src/joins/mod.rs index 19f10d06e1ef..6ddf19c51193 100644 --- a/datafusion/physical-plan/src/joins/mod.rs +++ b/datafusion/physical-plan/src/joins/mod.rs @@ -25,9 +25,9 @@ pub use sort_merge_join::SortMergeJoinExec; pub use symmetric_hash_join::SymmetricHashJoinExec; mod cross_join; mod hash_join; -mod hash_join_utils; mod nested_loop_join; mod sort_merge_join; +mod stream_join_utils; mod symmetric_hash_join; pub mod utils; diff --git a/datafusion/physical-plan/src/joins/hash_join_utils.rs b/datafusion/physical-plan/src/joins/stream_join_utils.rs similarity index 67% rename from datafusion/physical-plan/src/joins/hash_join_utils.rs rename to datafusion/physical-plan/src/joins/stream_join_utils.rs index db65c8bf083f..aa57a4f89606 100644 --- a/datafusion/physical-plan/src/joins/hash_join_utils.rs +++ b/datafusion/physical-plan/src/joins/stream_join_utils.rs @@ -15,151 +15,34 @@ // specific language governing permissions and limitations // under the License. -//! This file contains common subroutines for regular and symmetric hash join +//! This file contains common subroutines for symmetric hash join //! related functionality, used both in join calculations and optimization rules. use std::collections::{HashMap, VecDeque}; -use std::fmt::Debug; -use std::ops::IndexMut; use std::sync::Arc; -use std::{fmt, usize}; +use std::task::{Context, Poll}; +use std::usize; -use crate::joins::utils::JoinFilter; +use crate::handle_async_state; +use crate::joins::utils::{JoinFilter, JoinHashMapType}; use arrow::compute::concat_batches; -use arrow::datatypes::{ArrowNativeType, SchemaRef}; -use arrow_array::builder::BooleanBufferBuilder; use arrow_array::{ArrowPrimitiveType, NativeAdapter, PrimitiveArray, RecordBatch}; +use arrow_buffer::{ArrowNativeType, BooleanBufferBuilder}; +use arrow_schema::SchemaRef; +use async_trait::async_trait; use datafusion_common::tree_node::{Transformed, TreeNode}; use datafusion_common::{DataFusionError, JoinSide, Result, ScalarValue}; +use datafusion_execution::SendableRecordBatchStream; use datafusion_physical_expr::expressions::Column; use datafusion_physical_expr::intervals::{Interval, IntervalBound}; use datafusion_physical_expr::utils::collect_columns; use datafusion_physical_expr::{PhysicalExpr, PhysicalSortExpr}; +use futures::{ready, FutureExt, StreamExt}; use hashbrown::raw::RawTable; use hashbrown::HashSet; -/// Maps a `u64` hash value based on the build side ["on" values] to a list of indices with this key's value. -/// -/// By allocating a `HashMap` with capacity for *at least* the number of rows for entries at the build side, -/// we make sure that we don't have to re-hash the hashmap, which needs access to the key (the hash in this case) value. -/// -/// E.g. 1 -> [3, 6, 8] indicates that the column values map to rows 3, 6 and 8 for hash value 1 -/// As the key is a hash value, we need to check possible hash collisions in the probe stage -/// During this stage it might be the case that a row is contained the same hashmap value, -/// but the values don't match. Those are checked in the [`equal_rows_arr`](crate::joins::hash_join::equal_rows_arr) method. -/// -/// The indices (values) are stored in a separate chained list stored in the `Vec`. -/// -/// The first value (+1) is stored in the hashmap, whereas the next value is stored in array at the position value. -/// -/// The chain can be followed until the value "0" has been reached, meaning the end of the list. -/// Also see chapter 5.3 of [Balancing vectorized query execution with bandwidth-optimized storage](https://dare.uva.nl/search?identifier=5ccbb60a-38b8-4eeb-858a-e7735dd37487) -/// -/// # Example -/// -/// ``` text -/// See the example below: -/// -/// Insert (10,1) <-- insert hash value 10 with row index 1 -/// map: -/// ---------- -/// | 10 | 2 | -/// ---------- -/// next: -/// --------------------- -/// | 0 | 0 | 0 | 0 | 0 | -/// --------------------- -/// Insert (20,2) -/// map: -/// ---------- -/// | 10 | 2 | -/// | 20 | 3 | -/// ---------- -/// next: -/// --------------------- -/// | 0 | 0 | 0 | 0 | 0 | -/// --------------------- -/// Insert (10,3) <-- collision! row index 3 has a hash value of 10 as well -/// map: -/// ---------- -/// | 10 | 4 | -/// | 20 | 3 | -/// ---------- -/// next: -/// --------------------- -/// | 0 | 0 | 0 | 2 | 0 | <--- hash value 10 maps to 4,2 (which means indices values 3,1) -/// --------------------- -/// Insert (10,4) <-- another collision! row index 4 ALSO has a hash value of 10 -/// map: -/// --------- -/// | 10 | 5 | -/// | 20 | 3 | -/// --------- -/// next: -/// --------------------- -/// | 0 | 0 | 0 | 2 | 4 | <--- hash value 10 maps to 5,4,2 (which means indices values 4,3,1) -/// --------------------- -/// ``` -pub struct JoinHashMap { - // Stores hash value to last row index - map: RawTable<(u64, u64)>, - // Stores indices in chained list data structure - next: Vec, -} - -impl JoinHashMap { - #[cfg(test)] - pub(crate) fn new(map: RawTable<(u64, u64)>, next: Vec) -> Self { - Self { map, next } - } - - pub(crate) fn with_capacity(capacity: usize) -> Self { - JoinHashMap { - map: RawTable::with_capacity(capacity), - next: vec![0; capacity], - } - } -} - -/// Trait defining methods that must be implemented by a hash map type to be used for joins. -pub trait JoinHashMapType { - /// The type of list used to store the next list - type NextType: IndexMut; - /// Extend with zero - fn extend_zero(&mut self, len: usize); - /// Returns mutable references to the hash map and the next. - fn get_mut(&mut self) -> (&mut RawTable<(u64, u64)>, &mut Self::NextType); - /// Returns a reference to the hash map. - fn get_map(&self) -> &RawTable<(u64, u64)>; - /// Returns a reference to the next. - fn get_list(&self) -> &Self::NextType; -} - -/// Implementation of `JoinHashMapType` for `JoinHashMap`. -impl JoinHashMapType for JoinHashMap { - type NextType = Vec; - - // Void implementation - fn extend_zero(&mut self, _: usize) {} - - /// Get mutable references to the hash map and the next. - fn get_mut(&mut self) -> (&mut RawTable<(u64, u64)>, &mut Self::NextType) { - (&mut self.map, &mut self.next) - } - - /// Get a reference to the hash map. - fn get_map(&self) -> &RawTable<(u64, u64)> { - &self.map - } - - /// Get a reference to the next. - fn get_list(&self) -> &Self::NextType { - &self.next - } -} - /// Implementation of `JoinHashMapType` for `PruningJoinHashMap`. impl JoinHashMapType for PruningJoinHashMap { type NextType = VecDeque; @@ -185,12 +68,6 @@ impl JoinHashMapType for PruningJoinHashMap { } } -impl fmt::Debug for JoinHashMap { - fn fmt(&self, _f: &mut fmt::Formatter) -> fmt::Result { - Ok(()) - } -} - /// The `PruningJoinHashMap` is similar to a regular `JoinHashMap`, but with /// the capability of pruning elements in an efficient manner. This structure /// is particularly useful for cases where it's necessary to remove elements @@ -322,7 +199,7 @@ impl PruningJoinHashMap { } } -fn check_filter_expr_contains_sort_information( +pub fn check_filter_expr_contains_sort_information( expr: &Arc, reference: &Arc, ) -> bool { @@ -740,20 +617,423 @@ pub fn record_visited_indices( } } +/// The `handle_state` macro is designed to process the result of a state-changing +/// operation, typically encountered in implementations of `EagerJoinStream`. It +/// operates on a `StreamJoinStateResult` by matching its variants and executing +/// corresponding actions. This macro is used to streamline code that deals with +/// state transitions, reducing boilerplate and improving readability. +/// +/// # Cases +/// +/// - `Ok(StreamJoinStateResult::Continue)`: Continues the loop, indicating the +/// stream join operation should proceed to the next step. +/// - `Ok(StreamJoinStateResult::Ready(result))`: Returns a `Poll::Ready` with the +/// result, either yielding a value or indicating the stream is awaiting more +/// data. +/// - `Err(e)`: Returns a `Poll::Ready` containing an error, signaling an issue +/// during the stream join operation. +/// +/// # Arguments +/// +/// * `$match_case`: An expression that evaluates to a `Result>`. +#[macro_export] +macro_rules! handle_state { + ($match_case:expr) => { + match $match_case { + Ok(StreamJoinStateResult::Continue) => continue, + Ok(StreamJoinStateResult::Ready(result)) => { + Poll::Ready(Ok(result).transpose()) + } + Err(e) => Poll::Ready(Some(Err(e))), + } + }; +} + +/// The `handle_async_state` macro adapts the `handle_state` macro for use in +/// asynchronous operations, particularly when dealing with `Poll` results within +/// async traits like `EagerJoinStream`. It polls the asynchronous state-changing +/// function using `poll_unpin` and then passes the result to `handle_state` for +/// further processing. +/// +/// # Arguments +/// +/// * `$state_func`: An async function or future that returns a +/// `Result>`. +/// * `$cx`: The context to be passed for polling, usually of type `&mut Context`. +/// +#[macro_export] +macro_rules! handle_async_state { + ($state_func:expr, $cx:expr) => { + $crate::handle_state!(ready!($state_func.poll_unpin($cx))) + }; +} + +/// Represents the result of a stateful operation on `EagerJoinStream`. +/// +/// This enumueration indicates whether the state produced a result that is +/// ready for use (`Ready`) or if the operation requires continuation (`Continue`). +/// +/// Variants: +/// - `Ready(T)`: Indicates that the operation is complete with a result of type `T`. +/// - `Continue`: Indicates that the operation is not yet complete and requires further +/// processing or more data. When this variant is returned, it typically means that the +/// current invocation of the state did not produce a final result, and the operation +/// should be invoked again later with more data and possibly with a different state. +pub enum StreamJoinStateResult { + Ready(T), + Continue, +} + +/// Represents the various states of an eager join stream operation. +/// +/// This enum is used to track the current state of streaming during a join +/// operation. It provides indicators as to which side of the join needs to be +/// pulled next or if one (or both) sides have been exhausted. This allows +/// for efficient management of resources and optimal performance during the +/// join process. +#[derive(Clone, Debug)] +pub enum EagerJoinStreamState { + /// Indicates that the next step should pull from the right side of the join. + PullRight, + + /// Indicates that the next step should pull from the left side of the join. + PullLeft, + + /// State representing that the right side of the join has been fully processed. + RightExhausted, + + /// State representing that the left side of the join has been fully processed. + LeftExhausted, + + /// Represents a state where both sides of the join are exhausted. + /// + /// The `final_result` field indicates whether the join operation has + /// produced a final result or not. + BothExhausted { final_result: bool }, +} + +/// `EagerJoinStream` is an asynchronous trait designed for managing incremental +/// join operations between two streams, such as those used in `SymmetricHashJoinExec` +/// and `SortMergeJoinExec`. Unlike traditional join approaches that need to scan +/// one side of the join fully before proceeding, `EagerJoinStream` facilitates +/// more dynamic join operations by working with streams as they emit data. This +/// approach allows for more efficient processing, particularly in scenarios +/// where waiting for complete data materialization is not feasible or optimal. +/// The trait provides a framework for handling various states of such a join +/// process, ensuring that join logic is efficiently executed as data becomes +/// available from either stream. +/// +/// Implementors of this trait can perform eager joins of data from two different +/// asynchronous streams, typically referred to as left and right streams. The +/// trait provides a comprehensive set of methods to control and execute the join +/// process, leveraging the states defined in `EagerJoinStreamState`. Methods are +/// primarily focused on asynchronously fetching data batches from each stream, +/// processing them, and managing transitions between various states of the join. +/// +/// This trait's default implementations use a state machine approach to navigate +/// different stages of the join operation, handling data from both streams and +/// determining when the join completes. +/// +/// State Transitions: +/// - From `PullLeft` to `PullRight` or `LeftExhausted`: +/// - In `fetch_next_from_left_stream`, when fetching a batch from the left stream: +/// - On success (`Some(Ok(batch))`), state transitions to `PullRight` for +/// processing the batch. +/// - On error (`Some(Err(e))`), the error is returned, and the state remains +/// unchanged. +/// - On no data (`None`), state changes to `LeftExhausted`, returning `Continue` +/// to proceed with the join process. +/// - From `PullRight` to `PullLeft` or `RightExhausted`: +/// - In `fetch_next_from_right_stream`, when fetching from the right stream: +/// - If a batch is available, state changes to `PullLeft` for processing. +/// - On error, the error is returned without changing the state. +/// - If right stream is exhausted (`None`), state transitions to `RightExhausted`, +/// with a `Continue` result. +/// - Handling `RightExhausted` and `LeftExhausted`: +/// - Methods `handle_right_stream_end` and `handle_left_stream_end` manage scenarios +/// when streams are exhausted: +/// - They attempt to continue processing with the other stream. +/// - If both streams are exhausted, state changes to `BothExhausted { final_result: false }`. +/// - Transition to `BothExhausted { final_result: true }`: +/// - Occurs in `prepare_for_final_results_after_exhaustion` when both streams are +/// exhausted, indicating completion of processing and availability of final results. +#[async_trait] +pub trait EagerJoinStream { + /// Implements the main polling logic for the join stream. + /// + /// This method continuously checks the state of the join stream and + /// acts accordingly by delegating the handling to appropriate sub-methods + /// depending on the current state. + /// + /// # Arguments + /// + /// * `cx` - A context that facilitates cooperative non-blocking execution within a task. + /// + /// # Returns + /// + /// * `Poll>>` - A polled result, either a `RecordBatch` or None. + fn poll_next_impl( + &mut self, + cx: &mut Context<'_>, + ) -> Poll>> + where + Self: Send, + { + loop { + return match self.state() { + EagerJoinStreamState::PullRight => { + handle_async_state!(self.fetch_next_from_right_stream(), cx) + } + EagerJoinStreamState::PullLeft => { + handle_async_state!(self.fetch_next_from_left_stream(), cx) + } + EagerJoinStreamState::RightExhausted => { + handle_async_state!(self.handle_right_stream_end(), cx) + } + EagerJoinStreamState::LeftExhausted => { + handle_async_state!(self.handle_left_stream_end(), cx) + } + EagerJoinStreamState::BothExhausted { + final_result: false, + } => { + handle_state!(self.prepare_for_final_results_after_exhaustion()) + } + EagerJoinStreamState::BothExhausted { final_result: true } => { + Poll::Ready(None) + } + }; + } + } + /// Asynchronously pulls the next batch from the right stream. + /// + /// This default implementation checks for the next value in the right stream. + /// If a batch is found, the state is switched to `PullLeft`, and the batch handling + /// is delegated to `process_batch_from_right`. If the stream ends, the state is set to `RightExhausted`. + /// + /// # Returns + /// + /// * `Result>>` - The state result after pulling the batch. + async fn fetch_next_from_right_stream( + &mut self, + ) -> Result>> { + match self.right_stream().next().await { + Some(Ok(batch)) => { + self.set_state(EagerJoinStreamState::PullLeft); + self.process_batch_from_right(batch) + } + Some(Err(e)) => Err(e), + None => { + self.set_state(EagerJoinStreamState::RightExhausted); + Ok(StreamJoinStateResult::Continue) + } + } + } + + /// Asynchronously pulls the next batch from the left stream. + /// + /// This default implementation checks for the next value in the left stream. + /// If a batch is found, the state is switched to `PullRight`, and the batch handling + /// is delegated to `process_batch_from_left`. If the stream ends, the state is set to `LeftExhausted`. + /// + /// # Returns + /// + /// * `Result>>` - The state result after pulling the batch. + async fn fetch_next_from_left_stream( + &mut self, + ) -> Result>> { + match self.left_stream().next().await { + Some(Ok(batch)) => { + self.set_state(EagerJoinStreamState::PullRight); + self.process_batch_from_left(batch) + } + Some(Err(e)) => Err(e), + None => { + self.set_state(EagerJoinStreamState::LeftExhausted); + Ok(StreamJoinStateResult::Continue) + } + } + } + + /// Asynchronously handles the scenario when the right stream is exhausted. + /// + /// In this default implementation, when the right stream is exhausted, it attempts + /// to pull from the left stream. If a batch is found in the left stream, it delegates + /// the handling to `process_batch_from_left`. If both streams are exhausted, the state is set + /// to indicate both streams are exhausted without final results yet. + /// + /// # Returns + /// + /// * `Result>>` - The state result after checking the exhaustion state. + async fn handle_right_stream_end( + &mut self, + ) -> Result>> { + match self.left_stream().next().await { + Some(Ok(batch)) => self.process_batch_after_right_end(batch), + Some(Err(e)) => Err(e), + None => { + self.set_state(EagerJoinStreamState::BothExhausted { + final_result: false, + }); + Ok(StreamJoinStateResult::Continue) + } + } + } + + /// Asynchronously handles the scenario when the left stream is exhausted. + /// + /// When the left stream is exhausted, this default + /// implementation tries to pull from the right stream and delegates the batch + /// handling to `process_batch_after_left_end`. If both streams are exhausted, the state + /// is updated to indicate so. + /// + /// # Returns + /// + /// * `Result>>` - The state result after checking the exhaustion state. + async fn handle_left_stream_end( + &mut self, + ) -> Result>> { + match self.right_stream().next().await { + Some(Ok(batch)) => self.process_batch_after_left_end(batch), + Some(Err(e)) => Err(e), + None => { + self.set_state(EagerJoinStreamState::BothExhausted { + final_result: false, + }); + Ok(StreamJoinStateResult::Continue) + } + } + } + + /// Handles the state when both streams are exhausted and final results are yet to be produced. + /// + /// This default implementation switches the state to indicate both streams are + /// exhausted with final results and then invokes the handling for this specific + /// scenario via `process_batches_before_finalization`. + /// + /// # Returns + /// + /// * `Result>>` - The state result after both streams are exhausted. + fn prepare_for_final_results_after_exhaustion( + &mut self, + ) -> Result>> { + self.set_state(EagerJoinStreamState::BothExhausted { final_result: true }); + self.process_batches_before_finalization() + } + + /// Handles a pulled batch from the right stream. + /// + /// # Arguments + /// + /// * `batch` - The pulled `RecordBatch` from the right stream. + /// + /// # Returns + /// + /// * `Result>>` - The state result after processing the batch. + fn process_batch_from_right( + &mut self, + batch: RecordBatch, + ) -> Result>>; + + /// Handles a pulled batch from the left stream. + /// + /// # Arguments + /// + /// * `batch` - The pulled `RecordBatch` from the left stream. + /// + /// # Returns + /// + /// * `Result>>` - The state result after processing the batch. + fn process_batch_from_left( + &mut self, + batch: RecordBatch, + ) -> Result>>; + + /// Handles the situation when only the left stream is exhausted. + /// + /// # Arguments + /// + /// * `right_batch` - The `RecordBatch` from the right stream. + /// + /// # Returns + /// + /// * `Result>>` - The state result after the left stream is exhausted. + fn process_batch_after_left_end( + &mut self, + right_batch: RecordBatch, + ) -> Result>>; + + /// Handles the situation when only the right stream is exhausted. + /// + /// # Arguments + /// + /// * `left_batch` - The `RecordBatch` from the left stream. + /// + /// # Returns + /// + /// * `Result>>` - The state result after the right stream is exhausted. + fn process_batch_after_right_end( + &mut self, + left_batch: RecordBatch, + ) -> Result>>; + + /// Handles the final state after both streams are exhausted. + /// + /// # Returns + /// + /// * `Result>>` - The final state result after processing. + fn process_batches_before_finalization( + &mut self, + ) -> Result>>; + + /// Provides mutable access to the right stream. + /// + /// # Returns + /// + /// * `&mut SendableRecordBatchStream` - Returns a mutable reference to the right stream. + fn right_stream(&mut self) -> &mut SendableRecordBatchStream; + + /// Provides mutable access to the left stream. + /// + /// # Returns + /// + /// * `&mut SendableRecordBatchStream` - Returns a mutable reference to the left stream. + fn left_stream(&mut self) -> &mut SendableRecordBatchStream; + + /// Sets the current state of the join stream. + /// + /// # Arguments + /// + /// * `state` - The new state to be set. + fn set_state(&mut self, state: EagerJoinStreamState); + + /// Fetches the current state of the join stream. + /// + /// # Returns + /// + /// * `EagerJoinStreamState` - The current state of the join stream. + fn state(&mut self) -> EagerJoinStreamState; +} + #[cfg(test)] pub mod tests { + use std::sync::Arc; + use super::*; + use crate::joins::stream_join_utils::{ + build_filter_input_order, check_filter_expr_contains_sort_information, + convert_sort_expr_with_filter_schema, PruningJoinHashMap, + }; use crate::{ expressions::Column, expressions::PhysicalSortExpr, joins::utils::{ColumnIndex, JoinFilter}, }; + use arrow::compute::SortOptions; use arrow::datatypes::{DataType, Field, Schema}; use datafusion_common::ScalarValue; use datafusion_expr::Operator; use datafusion_physical_expr::expressions::{binary, cast, col, lit}; - use std::sync::Arc; /// Filter expr for a + b > c + 10 AND a + b < c + 100 pub(crate) fn complicated_filter( diff --git a/datafusion/physical-plan/src/joins/symmetric_hash_join.rs b/datafusion/physical-plan/src/joins/symmetric_hash_join.rs index 51561f5dab24..d653297abea7 100644 --- a/datafusion/physical-plan/src/joins/symmetric_hash_join.rs +++ b/datafusion/physical-plan/src/joins/symmetric_hash_join.rs @@ -25,20 +25,19 @@ //! This plan uses the [`OneSideHashJoiner`] object to facilitate join calculations //! for both its children. -use std::fmt; -use std::fmt::Debug; +use std::any::Any; +use std::fmt::{self, Debug}; use std::sync::Arc; use std::task::Poll; -use std::vec; -use std::{any::Any, usize}; +use std::{usize, vec}; use crate::common::SharedMemoryReservation; use crate::joins::hash_join::{build_equal_condition_join_indices, update_hash}; -use crate::joins::hash_join_utils::{ +use crate::joins::stream_join_utils::{ calculate_filter_expr_intervals, combine_two_batches, convert_sort_expr_with_filter_schema, get_pruning_anti_indices, - get_pruning_semi_indices, record_visited_indices, PruningJoinHashMap, - SortedFilterExpr, + get_pruning_semi_indices, record_visited_indices, EagerJoinStream, + EagerJoinStreamState, PruningJoinHashMap, SortedFilterExpr, StreamJoinStateResult, }; use crate::joins::utils::{ build_batch_from_indices, build_join_schema, check_join_is_valid, @@ -67,8 +66,7 @@ use datafusion_physical_expr::equivalence::join_equivalence_properties; use datafusion_physical_expr::intervals::ExprIntervalGraph; use ahash::RandomState; -use futures::stream::{select, BoxStream}; -use futures::{Stream, StreamExt}; +use futures::Stream; use hashbrown::HashSet; use parking_lot::Mutex; @@ -186,34 +184,34 @@ pub struct SymmetricHashJoinExec { } #[derive(Debug)] -struct SymmetricHashJoinSideMetrics { +pub struct StreamJoinSideMetrics { /// Number of batches consumed by this operator - input_batches: metrics::Count, + pub(crate) input_batches: metrics::Count, /// Number of rows consumed by this operator - input_rows: metrics::Count, + pub(crate) input_rows: metrics::Count, } /// Metrics for HashJoinExec #[derive(Debug)] -struct SymmetricHashJoinMetrics { +pub struct StreamJoinMetrics { /// Number of left batches/rows consumed by this operator - left: SymmetricHashJoinSideMetrics, + pub(crate) left: StreamJoinSideMetrics, /// Number of right batches/rows consumed by this operator - right: SymmetricHashJoinSideMetrics, + pub(crate) right: StreamJoinSideMetrics, /// Memory used by sides in bytes pub(crate) stream_memory_usage: metrics::Gauge, /// Number of batches produced by this operator - output_batches: metrics::Count, + pub(crate) output_batches: metrics::Count, /// Number of rows produced by this operator - output_rows: metrics::Count, + pub(crate) output_rows: metrics::Count, } -impl SymmetricHashJoinMetrics { +impl StreamJoinMetrics { pub fn new(partition: usize, metrics: &ExecutionPlanMetricsSet) -> Self { let input_batches = MetricBuilder::new(metrics).counter("input_batches", partition); let input_rows = MetricBuilder::new(metrics).counter("input_rows", partition); - let left = SymmetricHashJoinSideMetrics { + let left = StreamJoinSideMetrics { input_batches, input_rows, }; @@ -221,7 +219,7 @@ impl SymmetricHashJoinMetrics { let input_batches = MetricBuilder::new(metrics).counter("input_batches", partition); let input_rows = MetricBuilder::new(metrics).counter("input_rows", partition); - let right = SymmetricHashJoinSideMetrics { + let right = StreamJoinSideMetrics { input_batches, input_rows, }; @@ -516,21 +514,9 @@ impl ExecutionPlan for SymmetricHashJoinExec { let right_side_joiner = OneSideHashJoiner::new(JoinSide::Right, on_right, self.right.schema()); - let left_stream = self - .left - .execute(partition, context.clone())? - .map(|val| (JoinSide::Left, val)); - - let right_stream = self - .right - .execute(partition, context.clone())? - .map(|val| (JoinSide::Right, val)); - // This function will attempt to pull items from both streams. - // Each stream will be polled in a round-robin fashion, and whenever a stream is - // ready to yield an item that item is yielded. - // After one of the two input streams completes, the remaining one will be polled exclusively. - // The returned stream completes when both input streams have completed. - let input_stream = select(left_stream, right_stream).boxed(); + let left_stream = self.left.execute(partition, context.clone())?; + + let right_stream = self.right.execute(partition, context.clone())?; let reservation = Arc::new(Mutex::new( MemoryConsumer::new(format!("SymmetricHashJoinStream[{partition}]")) @@ -541,7 +527,8 @@ impl ExecutionPlan for SymmetricHashJoinExec { } Ok(Box::pin(SymmetricHashJoinStream { - input_stream, + left_stream, + right_stream, schema: self.schema(), filter: self.filter.clone(), join_type: self.join_type, @@ -549,12 +536,12 @@ impl ExecutionPlan for SymmetricHashJoinExec { left: left_side_joiner, right: right_side_joiner, column_indices: self.column_indices.clone(), - metrics: SymmetricHashJoinMetrics::new(partition, &self.metrics), + metrics: StreamJoinMetrics::new(partition, &self.metrics), graph, left_sorted_filter_expr, right_sorted_filter_expr, null_equals_null: self.null_equals_null, - final_result: false, + state: EagerJoinStreamState::PullRight, reservation, })) } @@ -562,8 +549,9 @@ impl ExecutionPlan for SymmetricHashJoinExec { /// A stream that issues [RecordBatch]es as they arrive from the right of the join. struct SymmetricHashJoinStream { - /// Input stream - input_stream: BoxStream<'static, (JoinSide, Result)>, + /// Input streams + left_stream: SendableRecordBatchStream, + right_stream: SendableRecordBatchStream, /// Input schema schema: Arc, /// join filter @@ -587,11 +575,11 @@ struct SymmetricHashJoinStream { /// If null_equals_null is true, null == null else null != null null_equals_null: bool, /// Metrics - metrics: SymmetricHashJoinMetrics, + metrics: StreamJoinMetrics, /// Memory reservation reservation: SharedMemoryReservation, - /// Flag indicating whether there is nothing to process anymore - final_result: bool, + /// State machine for input execution + state: EagerJoinStreamState, } impl RecordBatchStream for SymmetricHashJoinStream { @@ -763,7 +751,9 @@ pub(crate) fn build_side_determined_results( column_indices: &[ColumnIndex], ) -> Result> { // Check if we need to produce a result in the final output: - if need_to_produce_result_in_final(build_hash_joiner.build_side, join_type) { + if prune_length > 0 + && need_to_produce_result_in_final(build_hash_joiner.build_side, join_type) + { // Calculate the indices for build and probe sides based on join type and build side: let (build_indices, probe_indices) = calculate_indices_by_join_type( build_hash_joiner.build_side, @@ -1019,10 +1009,104 @@ impl OneSideHashJoiner { } } +impl EagerJoinStream for SymmetricHashJoinStream { + fn process_batch_from_right( + &mut self, + batch: RecordBatch, + ) -> Result>> { + self.perform_join_for_given_side(batch, JoinSide::Right) + .map(|maybe_batch| { + if maybe_batch.is_some() { + StreamJoinStateResult::Ready(maybe_batch) + } else { + StreamJoinStateResult::Continue + } + }) + } + + fn process_batch_from_left( + &mut self, + batch: RecordBatch, + ) -> Result>> { + self.perform_join_for_given_side(batch, JoinSide::Left) + .map(|maybe_batch| { + if maybe_batch.is_some() { + StreamJoinStateResult::Ready(maybe_batch) + } else { + StreamJoinStateResult::Continue + } + }) + } + + fn process_batch_after_left_end( + &mut self, + right_batch: RecordBatch, + ) -> Result>> { + self.process_batch_from_right(right_batch) + } + + fn process_batch_after_right_end( + &mut self, + left_batch: RecordBatch, + ) -> Result>> { + self.process_batch_from_left(left_batch) + } + + fn process_batches_before_finalization( + &mut self, + ) -> Result>> { + // Get the left side results: + let left_result = build_side_determined_results( + &self.left, + &self.schema, + self.left.input_buffer.num_rows(), + self.right.input_buffer.schema(), + self.join_type, + &self.column_indices, + )?; + // Get the right side results: + let right_result = build_side_determined_results( + &self.right, + &self.schema, + self.right.input_buffer.num_rows(), + self.left.input_buffer.schema(), + self.join_type, + &self.column_indices, + )?; + + // Combine the left and right results: + let result = combine_two_batches(&self.schema, left_result, right_result)?; + + // Update the metrics and return the result: + if let Some(batch) = &result { + // Update the metrics: + self.metrics.output_batches.add(1); + self.metrics.output_rows.add(batch.num_rows()); + return Ok(StreamJoinStateResult::Ready(result)); + } + Ok(StreamJoinStateResult::Continue) + } + + fn right_stream(&mut self) -> &mut SendableRecordBatchStream { + &mut self.right_stream + } + + fn left_stream(&mut self) -> &mut SendableRecordBatchStream { + &mut self.left_stream + } + + fn set_state(&mut self, state: EagerJoinStreamState) { + self.state = state; + } + + fn state(&mut self) -> EagerJoinStreamState { + self.state.clone() + } +} + impl SymmetricHashJoinStream { fn size(&self) -> usize { let mut size = 0; - size += std::mem::size_of_val(&self.input_stream); size += std::mem::size_of_val(&self.schema); size += std::mem::size_of_val(&self.filter); size += std::mem::size_of_val(&self.join_type); @@ -1035,165 +1119,111 @@ impl SymmetricHashJoinStream { size += std::mem::size_of_val(&self.random_state); size += std::mem::size_of_val(&self.null_equals_null); size += std::mem::size_of_val(&self.metrics); - size += std::mem::size_of_val(&self.final_result); size } - /// Polls the next result of the join operation. - /// - /// If the result of the join is ready, it returns the next record batch. - /// If the join has completed and there are no more results, it returns - /// `Poll::Ready(None)`. If the join operation is not complete, but the - /// current stream is not ready yet, it returns `Poll::Pending`. - fn poll_next_impl( + + /// Performs a join operation for the specified `probe_side` (either left or right). + /// This function: + /// 1. Determines which side is the probe and which is the build side. + /// 2. Updates metrics based on the batch that was polled. + /// 3. Executes the join with the given `probe_batch`. + /// 4. Optionally computes anti-join results if all conditions are met. + /// 5. Combines the results and returns a combined batch or `None` if no batch was produced. + fn perform_join_for_given_side( &mut self, - cx: &mut std::task::Context<'_>, - ) -> Poll>> { - loop { - // Poll the next batch from `input_stream`: - match self.input_stream.poll_next_unpin(cx) { - // Batch is available - Poll::Ready(Some((side, Ok(probe_batch)))) => { - // Determine which stream should be polled next. The side the - // RecordBatch comes from becomes the probe side. - let ( - probe_hash_joiner, - build_hash_joiner, - probe_side_sorted_filter_expr, - build_side_sorted_filter_expr, - probe_side_metrics, - ) = if side.eq(&JoinSide::Left) { - ( - &mut self.left, - &mut self.right, - &mut self.left_sorted_filter_expr, - &mut self.right_sorted_filter_expr, - &mut self.metrics.left, - ) - } else { - ( - &mut self.right, - &mut self.left, - &mut self.right_sorted_filter_expr, - &mut self.left_sorted_filter_expr, - &mut self.metrics.right, - ) - }; - // Update the metrics for the stream that was polled: - probe_side_metrics.input_batches.add(1); - probe_side_metrics.input_rows.add(probe_batch.num_rows()); - // Update the internal state of the hash joiner for the build side: - probe_hash_joiner - .update_internal_state(&probe_batch, &self.random_state)?; - // Join the two sides: - let equal_result = join_with_probe_batch( - build_hash_joiner, - probe_hash_joiner, - &self.schema, - self.join_type, - self.filter.as_ref(), - &probe_batch, - &self.column_indices, - &self.random_state, - self.null_equals_null, - )?; - // Increment the offset for the probe hash joiner: - probe_hash_joiner.offset += probe_batch.num_rows(); - - let anti_result = if let ( - Some(build_side_sorted_filter_expr), - Some(probe_side_sorted_filter_expr), - Some(graph), - ) = ( - build_side_sorted_filter_expr.as_mut(), - probe_side_sorted_filter_expr.as_mut(), - self.graph.as_mut(), - ) { - // Calculate filter intervals: - calculate_filter_expr_intervals( - &build_hash_joiner.input_buffer, - build_side_sorted_filter_expr, - &probe_batch, - probe_side_sorted_filter_expr, - )?; - let prune_length = build_hash_joiner - .calculate_prune_length_with_probe_batch( - build_side_sorted_filter_expr, - probe_side_sorted_filter_expr, - graph, - )?; - - if prune_length > 0 { - let res = build_side_determined_results( - build_hash_joiner, - &self.schema, - prune_length, - probe_batch.schema(), - self.join_type, - &self.column_indices, - )?; - build_hash_joiner.prune_internal_state(prune_length)?; - res - } else { - None - } - } else { - None - }; - - // Combine results: - let result = - combine_two_batches(&self.schema, equal_result, anti_result)?; - let capacity = self.size(); - self.metrics.stream_memory_usage.set(capacity); - self.reservation.lock().try_resize(capacity)?; - // Update the metrics if we have a batch; otherwise, continue the loop. - if let Some(batch) = &result { - self.metrics.output_batches.add(1); - self.metrics.output_rows.add(batch.num_rows()); - return Poll::Ready(Ok(result).transpose()); - } - } - Poll::Ready(Some((_, Err(e)))) => return Poll::Ready(Some(Err(e))), - Poll::Ready(None) => { - // If the final result has already been obtained, return `Poll::Ready(None)`: - if self.final_result { - return Poll::Ready(None); - } - self.final_result = true; - // Get the left side results: - let left_result = build_side_determined_results( - &self.left, - &self.schema, - self.left.input_buffer.num_rows(), - self.right.input_buffer.schema(), - self.join_type, - &self.column_indices, - )?; - // Get the right side results: - let right_result = build_side_determined_results( - &self.right, - &self.schema, - self.right.input_buffer.num_rows(), - self.left.input_buffer.schema(), - self.join_type, - &self.column_indices, - )?; - - // Combine the left and right results: - let result = - combine_two_batches(&self.schema, left_result, right_result)?; - - // Update the metrics and return the result: - if let Some(batch) = &result { - // Update the metrics: - self.metrics.output_batches.add(1); - self.metrics.output_rows.add(batch.num_rows()); - return Poll::Ready(Ok(result).transpose()); - } - } - Poll::Pending => return Poll::Pending, - } + probe_batch: RecordBatch, + probe_side: JoinSide, + ) -> Result> { + let ( + probe_hash_joiner, + build_hash_joiner, + probe_side_sorted_filter_expr, + build_side_sorted_filter_expr, + probe_side_metrics, + ) = if probe_side.eq(&JoinSide::Left) { + ( + &mut self.left, + &mut self.right, + &mut self.left_sorted_filter_expr, + &mut self.right_sorted_filter_expr, + &mut self.metrics.left, + ) + } else { + ( + &mut self.right, + &mut self.left, + &mut self.right_sorted_filter_expr, + &mut self.left_sorted_filter_expr, + &mut self.metrics.right, + ) + }; + // Update the metrics for the stream that was polled: + probe_side_metrics.input_batches.add(1); + probe_side_metrics.input_rows.add(probe_batch.num_rows()); + // Update the internal state of the hash joiner for the build side: + probe_hash_joiner.update_internal_state(&probe_batch, &self.random_state)?; + // Join the two sides: + let equal_result = join_with_probe_batch( + build_hash_joiner, + probe_hash_joiner, + &self.schema, + self.join_type, + self.filter.as_ref(), + &probe_batch, + &self.column_indices, + &self.random_state, + self.null_equals_null, + )?; + // Increment the offset for the probe hash joiner: + probe_hash_joiner.offset += probe_batch.num_rows(); + + let anti_result = if let ( + Some(build_side_sorted_filter_expr), + Some(probe_side_sorted_filter_expr), + Some(graph), + ) = ( + build_side_sorted_filter_expr.as_mut(), + probe_side_sorted_filter_expr.as_mut(), + self.graph.as_mut(), + ) { + // Calculate filter intervals: + calculate_filter_expr_intervals( + &build_hash_joiner.input_buffer, + build_side_sorted_filter_expr, + &probe_batch, + probe_side_sorted_filter_expr, + )?; + let prune_length = build_hash_joiner + .calculate_prune_length_with_probe_batch( + build_side_sorted_filter_expr, + probe_side_sorted_filter_expr, + graph, + )?; + let result = build_side_determined_results( + build_hash_joiner, + &self.schema, + prune_length, + probe_batch.schema(), + self.join_type, + &self.column_indices, + )?; + build_hash_joiner.prune_internal_state(prune_length)?; + result + } else { + None + }; + + // Combine results: + let result = combine_two_batches(&self.schema, equal_result, anti_result)?; + let capacity = self.size(); + self.metrics.stream_memory_usage.set(capacity); + self.reservation.lock().try_resize(capacity)?; + // Update the metrics if we have a batch; otherwise, continue the loop. + if let Some(batch) = &result { + self.metrics.output_batches.add(1); + self.metrics.output_rows.add(batch.num_rows()); } + Ok(result) } } @@ -1203,10 +1233,9 @@ mod tests { use std::sync::Mutex; use super::*; - use crate::joins::hash_join_utils::tests::complicated_filter; use crate::joins::test_utils::{ - build_sides_record_batches, compare_batches, create_memory_table, - join_expr_tests_fixture_f64, join_expr_tests_fixture_i32, + build_sides_record_batches, compare_batches, complicated_filter, + create_memory_table, join_expr_tests_fixture_f64, join_expr_tests_fixture_i32, join_expr_tests_fixture_temporal, partitioned_hash_join_with_filter, partitioned_sym_join_with_filter, split_record_batches, }; @@ -1833,6 +1862,73 @@ mod tests { Ok(()) } + #[tokio::test(flavor = "multi_thread")] + async fn complex_join_all_one_ascending_equivalence() -> Result<()> { + let cardinality = (3, 4); + let join_type = JoinType::Full; + + // a + b > c + 10 AND a + b < c + 100 + let config = SessionConfig::new().with_repartition_joins(false); + // let session_ctx = SessionContext::with_config(config); + // let task_ctx = session_ctx.task_ctx(); + let task_ctx = Arc::new(TaskContext::default().with_session_config(config)); + let (left_partition, right_partition) = get_or_create_table(cardinality, 8)?; + let left_schema = &left_partition[0].schema(); + let right_schema = &right_partition[0].schema(); + let left_sorted = vec![ + vec![PhysicalSortExpr { + expr: col("la1", left_schema)?, + options: SortOptions::default(), + }], + vec![PhysicalSortExpr { + expr: col("la2", left_schema)?, + options: SortOptions::default(), + }], + ]; + + let right_sorted = vec![PhysicalSortExpr { + expr: col("ra1", right_schema)?, + options: SortOptions::default(), + }]; + + let (left, right) = create_memory_table( + left_partition, + right_partition, + left_sorted, + vec![right_sorted], + )?; + + let on = vec![( + Column::new_with_schema("lc1", left_schema)?, + Column::new_with_schema("rc1", right_schema)?, + )]; + + let intermediate_schema = Schema::new(vec![ + Field::new("0", DataType::Int32, true), + Field::new("1", DataType::Int32, true), + Field::new("2", DataType::Int32, true), + ]); + let filter_expr = complicated_filter(&intermediate_schema)?; + let column_indices = vec![ + ColumnIndex { + index: 0, + side: JoinSide::Left, + }, + ColumnIndex { + index: 4, + side: JoinSide::Left, + }, + ColumnIndex { + index: 0, + side: JoinSide::Right, + }, + ]; + let filter = JoinFilter::new(filter_expr, column_indices, intermediate_schema); + + experiment(left, right, Some(filter), join_type, on, task_ctx).await?; + Ok(()) + } + #[rstest] #[tokio::test(flavor = "multi_thread")] async fn testing_with_temporal_columns( @@ -1917,6 +2013,7 @@ mod tests { experiment(left, right, Some(filter), join_type, on, task_ctx).await?; Ok(()) } + #[rstest] #[tokio::test(flavor = "multi_thread")] async fn test_with_interval_columns( diff --git a/datafusion/physical-plan/src/joins/test_utils.rs b/datafusion/physical-plan/src/joins/test_utils.rs index bb4a86199112..6deaa9ba1b9c 100644 --- a/datafusion/physical-plan/src/joins/test_utils.rs +++ b/datafusion/physical-plan/src/joins/test_utils.rs @@ -17,6 +17,9 @@ //! This file has test utils for hash joins +use std::sync::Arc; +use std::usize; + use crate::joins::utils::{JoinFilter, JoinOn}; use crate::joins::{ HashJoinExec, PartitionMode, StreamJoinPartitionMode, SymmetricHashJoinExec, @@ -24,24 +27,24 @@ use crate::joins::{ use crate::memory::MemoryExec; use crate::repartition::RepartitionExec; use crate::{common, ExecutionPlan, Partitioning}; + use arrow::util::pretty::pretty_format_batches; use arrow_array::{ ArrayRef, Float64Array, Int32Array, IntervalDayTimeArray, RecordBatch, TimestampMillisecondArray, }; -use arrow_schema::Schema; -use datafusion_common::Result; -use datafusion_common::ScalarValue; +use arrow_schema::{DataType, Schema}; +use datafusion_common::{Result, ScalarValue}; use datafusion_execution::TaskContext; use datafusion_expr::{JoinType, Operator}; +use datafusion_physical_expr::expressions::{binary, cast, col, lit}; use datafusion_physical_expr::intervals::test_utils::{ gen_conjunctive_numerical_expr, gen_conjunctive_temporal_expr, }; use datafusion_physical_expr::{LexOrdering, PhysicalExpr}; + use rand::prelude::StdRng; use rand::{Rng, SeedableRng}; -use std::sync::Arc; -use std::usize; pub fn compare_batches(collected_1: &[RecordBatch], collected_2: &[RecordBatch]) { // compare @@ -500,3 +503,51 @@ pub fn create_memory_table( .with_sort_information(right_sorted); Ok((Arc::new(left), Arc::new(right))) } + +/// Filter expr for a + b > c + 10 AND a + b < c + 100 +pub(crate) fn complicated_filter( + filter_schema: &Schema, +) -> Result> { + let left_expr = binary( + cast( + binary( + col("0", filter_schema)?, + Operator::Plus, + col("1", filter_schema)?, + filter_schema, + )?, + filter_schema, + DataType::Int64, + )?, + Operator::Gt, + binary( + cast(col("2", filter_schema)?, filter_schema, DataType::Int64)?, + Operator::Plus, + lit(ScalarValue::Int64(Some(10))), + filter_schema, + )?, + filter_schema, + )?; + + let right_expr = binary( + cast( + binary( + col("0", filter_schema)?, + Operator::Plus, + col("1", filter_schema)?, + filter_schema, + )?, + filter_schema, + DataType::Int64, + )?, + Operator::Lt, + binary( + cast(col("2", filter_schema)?, filter_schema, DataType::Int64)?, + Operator::Plus, + lit(ScalarValue::Int64(Some(100))), + filter_schema, + )?, + filter_schema, + )?; + binary(left_expr, Operator::And, right_expr, filter_schema) +} diff --git a/datafusion/physical-plan/src/joins/utils.rs b/datafusion/physical-plan/src/joins/utils.rs index f93f08255e0c..0729d365d6a0 100644 --- a/datafusion/physical-plan/src/joins/utils.rs +++ b/datafusion/physical-plan/src/joins/utils.rs @@ -18,12 +18,14 @@ //! Join related functionality used both on logical and physical plans use std::collections::HashSet; +use std::fmt::{self, Debug}; use std::future::Future; +use std::ops::IndexMut; use std::sync::Arc; use std::task::{Context, Poll}; use std::usize; -use crate::joins::hash_join_utils::{build_filter_input_order, SortedFilterExpr}; +use crate::joins::stream_join_utils::{build_filter_input_order, SortedFilterExpr}; use crate::metrics::{self, ExecutionPlanMetricsSet, MetricBuilder}; use crate::{ColumnStatistics, ExecutionPlan, Partitioning, Statistics}; @@ -50,8 +52,135 @@ use datafusion_physical_expr::{ use futures::future::{BoxFuture, Shared}; use futures::{ready, FutureExt}; +use hashbrown::raw::RawTable; use parking_lot::Mutex; +/// Maps a `u64` hash value based on the build side ["on" values] to a list of indices with this key's value. +/// +/// By allocating a `HashMap` with capacity for *at least* the number of rows for entries at the build side, +/// we make sure that we don't have to re-hash the hashmap, which needs access to the key (the hash in this case) value. +/// +/// E.g. 1 -> [3, 6, 8] indicates that the column values map to rows 3, 6 and 8 for hash value 1 +/// As the key is a hash value, we need to check possible hash collisions in the probe stage +/// During this stage it might be the case that a row is contained the same hashmap value, +/// but the values don't match. Those are checked in the [`equal_rows_arr`](crate::joins::hash_join::equal_rows_arr) method. +/// +/// The indices (values) are stored in a separate chained list stored in the `Vec`. +/// +/// The first value (+1) is stored in the hashmap, whereas the next value is stored in array at the position value. +/// +/// The chain can be followed until the value "0" has been reached, meaning the end of the list. +/// Also see chapter 5.3 of [Balancing vectorized query execution with bandwidth-optimized storage](https://dare.uva.nl/search?identifier=5ccbb60a-38b8-4eeb-858a-e7735dd37487) +/// +/// # Example +/// +/// ``` text +/// See the example below: +/// +/// Insert (10,1) <-- insert hash value 10 with row index 1 +/// map: +/// ---------- +/// | 10 | 2 | +/// ---------- +/// next: +/// --------------------- +/// | 0 | 0 | 0 | 0 | 0 | +/// --------------------- +/// Insert (20,2) +/// map: +/// ---------- +/// | 10 | 2 | +/// | 20 | 3 | +/// ---------- +/// next: +/// --------------------- +/// | 0 | 0 | 0 | 0 | 0 | +/// --------------------- +/// Insert (10,3) <-- collision! row index 3 has a hash value of 10 as well +/// map: +/// ---------- +/// | 10 | 4 | +/// | 20 | 3 | +/// ---------- +/// next: +/// --------------------- +/// | 0 | 0 | 0 | 2 | 0 | <--- hash value 10 maps to 4,2 (which means indices values 3,1) +/// --------------------- +/// Insert (10,4) <-- another collision! row index 4 ALSO has a hash value of 10 +/// map: +/// --------- +/// | 10 | 5 | +/// | 20 | 3 | +/// --------- +/// next: +/// --------------------- +/// | 0 | 0 | 0 | 2 | 4 | <--- hash value 10 maps to 5,4,2 (which means indices values 4,3,1) +/// --------------------- +/// ``` +pub struct JoinHashMap { + // Stores hash value to last row index + map: RawTable<(u64, u64)>, + // Stores indices in chained list data structure + next: Vec, +} + +impl JoinHashMap { + #[cfg(test)] + pub(crate) fn new(map: RawTable<(u64, u64)>, next: Vec) -> Self { + Self { map, next } + } + + pub(crate) fn with_capacity(capacity: usize) -> Self { + JoinHashMap { + map: RawTable::with_capacity(capacity), + next: vec![0; capacity], + } + } +} + +// Trait defining methods that must be implemented by a hash map type to be used for joins. +pub trait JoinHashMapType { + /// The type of list used to store the next list + type NextType: IndexMut; + /// Extend with zero + fn extend_zero(&mut self, len: usize); + /// Returns mutable references to the hash map and the next. + fn get_mut(&mut self) -> (&mut RawTable<(u64, u64)>, &mut Self::NextType); + /// Returns a reference to the hash map. + fn get_map(&self) -> &RawTable<(u64, u64)>; + /// Returns a reference to the next. + fn get_list(&self) -> &Self::NextType; +} + +/// Implementation of `JoinHashMapType` for `JoinHashMap`. +impl JoinHashMapType for JoinHashMap { + type NextType = Vec; + + // Void implementation + fn extend_zero(&mut self, _: usize) {} + + /// Get mutable references to the hash map and the next. + fn get_mut(&mut self) -> (&mut RawTable<(u64, u64)>, &mut Self::NextType) { + (&mut self.map, &mut self.next) + } + + /// Get a reference to the hash map. + fn get_map(&self) -> &RawTable<(u64, u64)> { + &self.map + } + + /// Get a reference to the next. + fn get_list(&self) -> &Self::NextType { + &self.next + } +} + +impl fmt::Debug for JoinHashMap { + fn fmt(&self, _f: &mut fmt::Formatter) -> fmt::Result { + Ok(()) + } +} + /// The on clause of the join, as vector of (left, right) columns. pub type JoinOn = Vec<(Column, Column)>; /// Reference for JoinOn. diff --git a/datafusion/proto/proto/datafusion.proto b/datafusion/proto/proto/datafusion.proto index 9d508078c705..9197343d749e 100644 --- a/datafusion/proto/proto/datafusion.proto +++ b/datafusion/proto/proto/datafusion.proto @@ -1155,6 +1155,7 @@ message PhysicalPlanNode { NestedLoopJoinExecNode nested_loop_join = 22; AnalyzeExecNode analyze = 23; JsonSinkExecNode json_sink = 24; + SymmetricHashJoinExecNode symmetric_hash_join = 25; } } @@ -1432,6 +1433,21 @@ message HashJoinExecNode { JoinFilter filter = 8; } +enum StreamPartitionMode { + SINGLE_PARTITION = 0; + PARTITIONED_EXEC = 1; +} + +message SymmetricHashJoinExecNode { + PhysicalPlanNode left = 1; + PhysicalPlanNode right = 2; + repeated JoinOn on = 3; + JoinType join_type = 4; + StreamPartitionMode partition_mode = 6; + bool null_equals_null = 7; + JoinFilter filter = 8; +} + message UnionExecNode { repeated PhysicalPlanNode inputs = 1; } diff --git a/datafusion/proto/src/generated/pbjson.rs b/datafusion/proto/src/generated/pbjson.rs index 0a8f415e20c5..8a6360023794 100644 --- a/datafusion/proto/src/generated/pbjson.rs +++ b/datafusion/proto/src/generated/pbjson.rs @@ -17844,6 +17844,9 @@ impl serde::Serialize for PhysicalPlanNode { physical_plan_node::PhysicalPlanType::JsonSink(v) => { struct_ser.serialize_field("jsonSink", v)?; } + physical_plan_node::PhysicalPlanType::SymmetricHashJoin(v) => { + struct_ser.serialize_field("symmetricHashJoin", v)?; + } } } struct_ser.end() @@ -17890,6 +17893,8 @@ impl<'de> serde::Deserialize<'de> for PhysicalPlanNode { "analyze", "json_sink", "jsonSink", + "symmetric_hash_join", + "symmetricHashJoin", ]; #[allow(clippy::enum_variant_names)] @@ -17917,6 +17922,7 @@ impl<'de> serde::Deserialize<'de> for PhysicalPlanNode { NestedLoopJoin, Analyze, JsonSink, + SymmetricHashJoin, } impl<'de> serde::Deserialize<'de> for GeneratedField { fn deserialize(deserializer: D) -> std::result::Result @@ -17961,6 +17967,7 @@ impl<'de> serde::Deserialize<'de> for PhysicalPlanNode { "nestedLoopJoin" | "nested_loop_join" => Ok(GeneratedField::NestedLoopJoin), "analyze" => Ok(GeneratedField::Analyze), "jsonSink" | "json_sink" => Ok(GeneratedField::JsonSink), + "symmetricHashJoin" | "symmetric_hash_join" => Ok(GeneratedField::SymmetricHashJoin), _ => Err(serde::de::Error::unknown_field(value, FIELDS)), } } @@ -18142,6 +18149,13 @@ impl<'de> serde::Deserialize<'de> for PhysicalPlanNode { return Err(serde::de::Error::duplicate_field("jsonSink")); } physical_plan_type__ = map_.next_value::<::std::option::Option<_>>()?.map(physical_plan_node::PhysicalPlanType::JsonSink) +; + } + GeneratedField::SymmetricHashJoin => { + if physical_plan_type__.is_some() { + return Err(serde::de::Error::duplicate_field("symmetricHashJoin")); + } + physical_plan_type__ = map_.next_value::<::std::option::Option<_>>()?.map(physical_plan_node::PhysicalPlanType::SymmetricHashJoin) ; } } @@ -23648,6 +23662,77 @@ impl<'de> serde::Deserialize<'de> for Statistics { deserializer.deserialize_struct("datafusion.Statistics", FIELDS, GeneratedVisitor) } } +impl serde::Serialize for StreamPartitionMode { + #[allow(deprecated)] + fn serialize(&self, serializer: S) -> std::result::Result + where + S: serde::Serializer, + { + let variant = match self { + Self::SinglePartition => "SINGLE_PARTITION", + Self::PartitionedExec => "PARTITIONED_EXEC", + }; + serializer.serialize_str(variant) + } +} +impl<'de> serde::Deserialize<'de> for StreamPartitionMode { + #[allow(deprecated)] + fn deserialize(deserializer: D) -> std::result::Result + where + D: serde::Deserializer<'de>, + { + const FIELDS: &[&str] = &[ + "SINGLE_PARTITION", + "PARTITIONED_EXEC", + ]; + + struct GeneratedVisitor; + + impl<'de> serde::de::Visitor<'de> for GeneratedVisitor { + type Value = StreamPartitionMode; + + fn expecting(&self, formatter: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + write!(formatter, "expected one of: {:?}", &FIELDS) + } + + fn visit_i64(self, v: i64) -> std::result::Result + where + E: serde::de::Error, + { + i32::try_from(v) + .ok() + .and_then(|x| x.try_into().ok()) + .ok_or_else(|| { + serde::de::Error::invalid_value(serde::de::Unexpected::Signed(v), &self) + }) + } + + fn visit_u64(self, v: u64) -> std::result::Result + where + E: serde::de::Error, + { + i32::try_from(v) + .ok() + .and_then(|x| x.try_into().ok()) + .ok_or_else(|| { + serde::de::Error::invalid_value(serde::de::Unexpected::Unsigned(v), &self) + }) + } + + fn visit_str(self, value: &str) -> std::result::Result + where + E: serde::de::Error, + { + match value { + "SINGLE_PARTITION" => Ok(StreamPartitionMode::SinglePartition), + "PARTITIONED_EXEC" => Ok(StreamPartitionMode::PartitionedExec), + _ => Err(serde::de::Error::unknown_variant(value, FIELDS)), + } + } + } + deserializer.deserialize_any(GeneratedVisitor) + } +} impl serde::Serialize for StringifiedPlan { #[allow(deprecated)] fn serialize(&self, serializer: S) -> std::result::Result @@ -24066,6 +24151,206 @@ impl<'de> serde::Deserialize<'de> for SubqueryAliasNode { deserializer.deserialize_struct("datafusion.SubqueryAliasNode", FIELDS, GeneratedVisitor) } } +impl serde::Serialize for SymmetricHashJoinExecNode { + #[allow(deprecated)] + fn serialize(&self, serializer: S) -> std::result::Result + where + S: serde::Serializer, + { + use serde::ser::SerializeStruct; + let mut len = 0; + if self.left.is_some() { + len += 1; + } + if self.right.is_some() { + len += 1; + } + if !self.on.is_empty() { + len += 1; + } + if self.join_type != 0 { + len += 1; + } + if self.partition_mode != 0 { + len += 1; + } + if self.null_equals_null { + len += 1; + } + if self.filter.is_some() { + len += 1; + } + let mut struct_ser = serializer.serialize_struct("datafusion.SymmetricHashJoinExecNode", len)?; + if let Some(v) = self.left.as_ref() { + struct_ser.serialize_field("left", v)?; + } + if let Some(v) = self.right.as_ref() { + struct_ser.serialize_field("right", v)?; + } + if !self.on.is_empty() { + struct_ser.serialize_field("on", &self.on)?; + } + if self.join_type != 0 { + let v = JoinType::try_from(self.join_type) + .map_err(|_| serde::ser::Error::custom(format!("Invalid variant {}", self.join_type)))?; + struct_ser.serialize_field("joinType", &v)?; + } + if self.partition_mode != 0 { + let v = StreamPartitionMode::try_from(self.partition_mode) + .map_err(|_| serde::ser::Error::custom(format!("Invalid variant {}", self.partition_mode)))?; + struct_ser.serialize_field("partitionMode", &v)?; + } + if self.null_equals_null { + struct_ser.serialize_field("nullEqualsNull", &self.null_equals_null)?; + } + if let Some(v) = self.filter.as_ref() { + struct_ser.serialize_field("filter", v)?; + } + struct_ser.end() + } +} +impl<'de> serde::Deserialize<'de> for SymmetricHashJoinExecNode { + #[allow(deprecated)] + fn deserialize(deserializer: D) -> std::result::Result + where + D: serde::Deserializer<'de>, + { + const FIELDS: &[&str] = &[ + "left", + "right", + "on", + "join_type", + "joinType", + "partition_mode", + "partitionMode", + "null_equals_null", + "nullEqualsNull", + "filter", + ]; + + #[allow(clippy::enum_variant_names)] + enum GeneratedField { + Left, + Right, + On, + JoinType, + PartitionMode, + NullEqualsNull, + Filter, + } + impl<'de> serde::Deserialize<'de> for GeneratedField { + fn deserialize(deserializer: D) -> std::result::Result + where + D: serde::Deserializer<'de>, + { + struct GeneratedVisitor; + + impl<'de> serde::de::Visitor<'de> for GeneratedVisitor { + type Value = GeneratedField; + + fn expecting(&self, formatter: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + write!(formatter, "expected one of: {:?}", &FIELDS) + } + + #[allow(unused_variables)] + fn visit_str(self, value: &str) -> std::result::Result + where + E: serde::de::Error, + { + match value { + "left" => Ok(GeneratedField::Left), + "right" => Ok(GeneratedField::Right), + "on" => Ok(GeneratedField::On), + "joinType" | "join_type" => Ok(GeneratedField::JoinType), + "partitionMode" | "partition_mode" => Ok(GeneratedField::PartitionMode), + "nullEqualsNull" | "null_equals_null" => Ok(GeneratedField::NullEqualsNull), + "filter" => Ok(GeneratedField::Filter), + _ => Err(serde::de::Error::unknown_field(value, FIELDS)), + } + } + } + deserializer.deserialize_identifier(GeneratedVisitor) + } + } + struct GeneratedVisitor; + impl<'de> serde::de::Visitor<'de> for GeneratedVisitor { + type Value = SymmetricHashJoinExecNode; + + fn expecting(&self, formatter: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + formatter.write_str("struct datafusion.SymmetricHashJoinExecNode") + } + + fn visit_map(self, mut map_: V) -> std::result::Result + where + V: serde::de::MapAccess<'de>, + { + let mut left__ = None; + let mut right__ = None; + let mut on__ = None; + let mut join_type__ = None; + let mut partition_mode__ = None; + let mut null_equals_null__ = None; + let mut filter__ = None; + while let Some(k) = map_.next_key()? { + match k { + GeneratedField::Left => { + if left__.is_some() { + return Err(serde::de::Error::duplicate_field("left")); + } + left__ = map_.next_value()?; + } + GeneratedField::Right => { + if right__.is_some() { + return Err(serde::de::Error::duplicate_field("right")); + } + right__ = map_.next_value()?; + } + GeneratedField::On => { + if on__.is_some() { + return Err(serde::de::Error::duplicate_field("on")); + } + on__ = Some(map_.next_value()?); + } + GeneratedField::JoinType => { + if join_type__.is_some() { + return Err(serde::de::Error::duplicate_field("joinType")); + } + join_type__ = Some(map_.next_value::()? as i32); + } + GeneratedField::PartitionMode => { + if partition_mode__.is_some() { + return Err(serde::de::Error::duplicate_field("partitionMode")); + } + partition_mode__ = Some(map_.next_value::()? as i32); + } + GeneratedField::NullEqualsNull => { + if null_equals_null__.is_some() { + return Err(serde::de::Error::duplicate_field("nullEqualsNull")); + } + null_equals_null__ = Some(map_.next_value()?); + } + GeneratedField::Filter => { + if filter__.is_some() { + return Err(serde::de::Error::duplicate_field("filter")); + } + filter__ = map_.next_value()?; + } + } + } + Ok(SymmetricHashJoinExecNode { + left: left__, + right: right__, + on: on__.unwrap_or_default(), + join_type: join_type__.unwrap_or_default(), + partition_mode: partition_mode__.unwrap_or_default(), + null_equals_null: null_equals_null__.unwrap_or_default(), + filter: filter__, + }) + } + } + deserializer.deserialize_struct("datafusion.SymmetricHashJoinExecNode", FIELDS, GeneratedVisitor) + } +} impl serde::Serialize for TimeUnit { #[allow(deprecated)] fn serialize(&self, serializer: S) -> std::result::Result diff --git a/datafusion/proto/src/generated/prost.rs b/datafusion/proto/src/generated/prost.rs index 84fb84b9487e..4fb8e1599e4b 100644 --- a/datafusion/proto/src/generated/prost.rs +++ b/datafusion/proto/src/generated/prost.rs @@ -1514,7 +1514,7 @@ pub mod owned_table_reference { pub struct PhysicalPlanNode { #[prost( oneof = "physical_plan_node::PhysicalPlanType", - tags = "1, 2, 3, 4, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24" + tags = "1, 2, 3, 4, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25" )] pub physical_plan_type: ::core::option::Option, } @@ -1571,6 +1571,8 @@ pub mod physical_plan_node { Analyze(::prost::alloc::boxed::Box), #[prost(message, tag = "24")] JsonSink(::prost::alloc::boxed::Box), + #[prost(message, tag = "25")] + SymmetricHashJoin(::prost::alloc::boxed::Box), } } #[allow(clippy::derive_partial_eq_without_eq)] @@ -2009,6 +2011,24 @@ pub struct HashJoinExecNode { } #[allow(clippy::derive_partial_eq_without_eq)] #[derive(Clone, PartialEq, ::prost::Message)] +pub struct SymmetricHashJoinExecNode { + #[prost(message, optional, boxed, tag = "1")] + pub left: ::core::option::Option<::prost::alloc::boxed::Box>, + #[prost(message, optional, boxed, tag = "2")] + pub right: ::core::option::Option<::prost::alloc::boxed::Box>, + #[prost(message, repeated, tag = "3")] + pub on: ::prost::alloc::vec::Vec, + #[prost(enumeration = "JoinType", tag = "4")] + pub join_type: i32, + #[prost(enumeration = "StreamPartitionMode", tag = "6")] + pub partition_mode: i32, + #[prost(bool, tag = "7")] + pub null_equals_null: bool, + #[prost(message, optional, tag = "8")] + pub filter: ::core::option::Option, +} +#[allow(clippy::derive_partial_eq_without_eq)] +#[derive(Clone, PartialEq, ::prost::Message)] pub struct UnionExecNode { #[prost(message, repeated, tag = "1")] pub inputs: ::prost::alloc::vec::Vec, @@ -3265,6 +3285,32 @@ impl PartitionMode { } #[derive(Clone, Copy, Debug, PartialEq, Eq, Hash, PartialOrd, Ord, ::prost::Enumeration)] #[repr(i32)] +pub enum StreamPartitionMode { + SinglePartition = 0, + PartitionedExec = 1, +} +impl StreamPartitionMode { + /// String value of the enum field names used in the ProtoBuf definition. + /// + /// The values are not transformed in any way and thus are considered stable + /// (if the ProtoBuf definition does not change) and safe for programmatic use. + pub fn as_str_name(&self) -> &'static str { + match self { + StreamPartitionMode::SinglePartition => "SINGLE_PARTITION", + StreamPartitionMode::PartitionedExec => "PARTITIONED_EXEC", + } + } + /// Creates an enum from field names used in the ProtoBuf definition. + pub fn from_str_name(value: &str) -> ::core::option::Option { + match value { + "SINGLE_PARTITION" => Some(Self::SinglePartition), + "PARTITIONED_EXEC" => Some(Self::PartitionedExec), + _ => None, + } + } +} +#[derive(Clone, Copy, Debug, PartialEq, Eq, Hash, PartialOrd, Ord, ::prost::Enumeration)] +#[repr(i32)] pub enum AggregateMode { Partial = 0, Final = 1, diff --git a/datafusion/proto/src/physical_plan/mod.rs b/datafusion/proto/src/physical_plan/mod.rs index 1eedbe987ec1..6714c35dc615 100644 --- a/datafusion/proto/src/physical_plan/mod.rs +++ b/datafusion/proto/src/physical_plan/mod.rs @@ -39,7 +39,9 @@ use datafusion::physical_plan::expressions::{Column, PhysicalSortExpr}; use datafusion::physical_plan::filter::FilterExec; use datafusion::physical_plan::insert::FileSinkExec; use datafusion::physical_plan::joins::utils::{ColumnIndex, JoinFilter}; -use datafusion::physical_plan::joins::{CrossJoinExec, NestedLoopJoinExec}; +use datafusion::physical_plan::joins::{ + CrossJoinExec, NestedLoopJoinExec, StreamJoinPartitionMode, SymmetricHashJoinExec, +}; use datafusion::physical_plan::joins::{HashJoinExec, PartitionMode}; use datafusion::physical_plan::limit::{GlobalLimitExec, LocalLimitExec}; use datafusion::physical_plan::projection::ProjectionExec; @@ -583,6 +585,97 @@ impl AsExecutionPlan for PhysicalPlanNode { hashjoin.null_equals_null, )?)) } + PhysicalPlanType::SymmetricHashJoin(sym_join) => { + let left = into_physical_plan( + &sym_join.left, + registry, + runtime, + extension_codec, + )?; + let right = into_physical_plan( + &sym_join.right, + registry, + runtime, + extension_codec, + )?; + let on = sym_join + .on + .iter() + .map(|col| { + let left = into_required!(col.left)?; + let right = into_required!(col.right)?; + Ok((left, right)) + }) + .collect::>()?; + let join_type = protobuf::JoinType::try_from(sym_join.join_type) + .map_err(|_| { + proto_error(format!( + "Received a SymmetricHashJoin message with unknown JoinType {}", + sym_join.join_type + )) + })?; + let filter = sym_join + .filter + .as_ref() + .map(|f| { + let schema = f + .schema + .as_ref() + .ok_or_else(|| proto_error("Missing JoinFilter schema"))? + .try_into()?; + + let expression = parse_physical_expr( + f.expression.as_ref().ok_or_else(|| { + proto_error("Unexpected empty filter expression") + })?, + registry, &schema + )?; + let column_indices = f.column_indices + .iter() + .map(|i| { + let side = protobuf::JoinSide::try_from(i.side) + .map_err(|_| proto_error(format!( + "Received a HashJoinNode message with JoinSide in Filter {}", + i.side)) + )?; + + Ok(ColumnIndex{ + index: i.index as usize, + side: side.into(), + }) + }) + .collect::>()?; + + Ok(JoinFilter::new(expression, column_indices, schema)) + }) + .map_or(Ok(None), |v: Result| v.map(Some))?; + + let partition_mode = + protobuf::StreamPartitionMode::try_from(sym_join.partition_mode).map_err(|_| { + proto_error(format!( + "Received a SymmetricHashJoin message with unknown PartitionMode {}", + sym_join.partition_mode + )) + })?; + let partition_mode = match partition_mode { + protobuf::StreamPartitionMode::SinglePartition => { + StreamJoinPartitionMode::SinglePartition + } + protobuf::StreamPartitionMode::PartitionedExec => { + StreamJoinPartitionMode::Partitioned + } + }; + SymmetricHashJoinExec::try_new( + left, + right, + on, + filter, + &join_type.into(), + sym_join.null_equals_null, + partition_mode, + ) + .map(|e| Arc::new(e) as _) + } PhysicalPlanType::Union(union) => { let mut inputs: Vec> = vec![]; for input in &union.inputs { @@ -1008,6 +1101,79 @@ impl AsExecutionPlan for PhysicalPlanNode { }); } + if let Some(exec) = plan.downcast_ref::() { + let left = protobuf::PhysicalPlanNode::try_from_physical_plan( + exec.left().to_owned(), + extension_codec, + )?; + let right = protobuf::PhysicalPlanNode::try_from_physical_plan( + exec.right().to_owned(), + extension_codec, + )?; + let on = exec + .on() + .iter() + .map(|tuple| protobuf::JoinOn { + left: Some(protobuf::PhysicalColumn { + name: tuple.0.name().to_string(), + index: tuple.0.index() as u32, + }), + right: Some(protobuf::PhysicalColumn { + name: tuple.1.name().to_string(), + index: tuple.1.index() as u32, + }), + }) + .collect(); + let join_type: protobuf::JoinType = exec.join_type().to_owned().into(); + let filter = exec + .filter() + .as_ref() + .map(|f| { + let expression = f.expression().to_owned().try_into()?; + let column_indices = f + .column_indices() + .iter() + .map(|i| { + let side: protobuf::JoinSide = i.side.to_owned().into(); + protobuf::ColumnIndex { + index: i.index as u32, + side: side.into(), + } + }) + .collect(); + let schema = f.schema().try_into()?; + Ok(protobuf::JoinFilter { + expression: Some(expression), + column_indices, + schema: Some(schema), + }) + }) + .map_or(Ok(None), |v: Result| v.map(Some))?; + + let partition_mode = match exec.partition_mode() { + StreamJoinPartitionMode::SinglePartition => { + protobuf::StreamPartitionMode::SinglePartition + } + StreamJoinPartitionMode::Partitioned => { + protobuf::StreamPartitionMode::PartitionedExec + } + }; + + return Ok(protobuf::PhysicalPlanNode { + physical_plan_type: Some(PhysicalPlanType::SymmetricHashJoin(Box::new( + protobuf::SymmetricHashJoinExecNode { + left: Some(Box::new(left)), + right: Some(Box::new(right)), + on, + join_type: join_type.into(), + partition_mode: partition_mode.into(), + null_equals_null: exec.null_equals_null(), + filter, + }, + ))), + }); + } + if let Some(exec) = plan.downcast_ref::() { let left = protobuf::PhysicalPlanNode::try_from_physical_plan( exec.left().to_owned(), diff --git a/datafusion/proto/tests/cases/roundtrip_physical_plan.rs b/datafusion/proto/tests/cases/roundtrip_physical_plan.rs index 23b0ea43c73a..d7d762d470d7 100644 --- a/datafusion/proto/tests/cases/roundtrip_physical_plan.rs +++ b/datafusion/proto/tests/cases/roundtrip_physical_plan.rs @@ -45,7 +45,9 @@ use datafusion::physical_plan::expressions::{ use datafusion::physical_plan::filter::FilterExec; use datafusion::physical_plan::functions::make_scalar_function; use datafusion::physical_plan::insert::FileSinkExec; -use datafusion::physical_plan::joins::{HashJoinExec, NestedLoopJoinExec, PartitionMode}; +use datafusion::physical_plan::joins::{ + HashJoinExec, NestedLoopJoinExec, PartitionMode, StreamJoinPartitionMode, +}; use datafusion::physical_plan::limit::{GlobalLimitExec, LocalLimitExec}; use datafusion::physical_plan::projection::ProjectionExec; use datafusion::physical_plan::sorts::sort::SortExec; @@ -754,3 +756,45 @@ fn roundtrip_json_sink() -> Result<()> { Some(sort_order), ))) } + +#[test] +fn roundtrip_sym_hash_join() -> Result<()> { + let field_a = Field::new("col", DataType::Int64, false); + let schema_left = Schema::new(vec![field_a.clone()]); + let schema_right = Schema::new(vec![field_a]); + let on = vec![( + Column::new("col", schema_left.index_of("col")?), + Column::new("col", schema_right.index_of("col")?), + )]; + + let schema_left = Arc::new(schema_left); + let schema_right = Arc::new(schema_right); + for join_type in &[ + JoinType::Inner, + JoinType::Left, + JoinType::Right, + JoinType::Full, + JoinType::LeftAnti, + JoinType::RightAnti, + JoinType::LeftSemi, + JoinType::RightSemi, + ] { + for partition_mode in &[ + StreamJoinPartitionMode::Partitioned, + StreamJoinPartitionMode::SinglePartition, + ] { + roundtrip_test(Arc::new( + datafusion::physical_plan::joins::SymmetricHashJoinExec::try_new( + Arc::new(EmptyExec::new(false, schema_left.clone())), + Arc::new(EmptyExec::new(false, schema_right.clone())), + on.clone(), + None, + join_type, + false, + *partition_mode, + )?, + ))?; + } + } + Ok(()) +} From 53d5df2d5c97667729ac94dde9f1a3c1d5d3b4e0 Mon Sep 17 00:00:00 2001 From: Raphael Taylor-Davies <1781103+tustvold@users.noreply.github.com> Date: Mon, 20 Nov 2023 13:15:00 +0000 Subject: [PATCH 293/572] Don't canonicalize ListingTableUrl (#8014) --- datafusion-cli/src/exec.rs | 10 +- datafusion/core/src/datasource/listing/url.rs | 110 ++++++++++++++---- 2 files changed, 91 insertions(+), 29 deletions(-) diff --git a/datafusion-cli/src/exec.rs b/datafusion-cli/src/exec.rs index 14ac22687bf4..1869e15ef584 100644 --- a/datafusion-cli/src/exec.rs +++ b/datafusion-cli/src/exec.rs @@ -388,15 +388,7 @@ mod tests { // Ensure that local files are also registered let sql = format!("CREATE EXTERNAL TABLE test STORED AS PARQUET LOCATION '{location}'"); - let err = create_external_table_test(location, &sql) - .await - .unwrap_err(); - - if let DataFusionError::IoError(e) = err { - assert_eq!(e.kind(), std::io::ErrorKind::NotFound); - } else { - return Err(err); - } + create_external_table_test(location, &sql).await.unwrap(); Ok(()) } diff --git a/datafusion/core/src/datasource/listing/url.rs b/datafusion/core/src/datasource/listing/url.rs index 45845916a971..9e9fb9210071 100644 --- a/datafusion/core/src/datasource/listing/url.rs +++ b/datafusion/core/src/datasource/listing/url.rs @@ -45,6 +45,17 @@ pub struct ListingTableUrl { impl ListingTableUrl { /// Parse a provided string as a `ListingTableUrl` /// + /// A URL can either refer to a single object, or a collection of objects with a + /// common prefix, with the presence of a trailing `/` indicating a collection. + /// + /// For example, `file:///foo.txt` refers to the file at `/foo.txt`, whereas + /// `file:///foo/` refers to all the files under the directory `/foo` and its + /// subdirectories. + /// + /// Similarly `s3://BUCKET/blob.csv` refers to `blob.csv` in the S3 bucket `BUCKET`, + /// wherease `s3://BUCKET/foo/` refers to all objects with the prefix `foo/` in the + /// S3 bucket `BUCKET` + /// /// # URL Encoding /// /// URL paths are expected to be URL-encoded. That is, the URL for a file named `bar%2Efoo` @@ -58,19 +69,21 @@ impl ListingTableUrl { /// # Paths without a Scheme /// /// If no scheme is provided, or the string is an absolute filesystem path - /// as determined [`std::path::Path::is_absolute`], the string will be + /// as determined by [`std::path::Path::is_absolute`], the string will be /// interpreted as a path on the local filesystem using the operating /// system's standard path delimiter, i.e. `\` on Windows, `/` on Unix. /// /// If the path contains any of `'?', '*', '['`, it will be considered /// a glob expression and resolved as described in the section below. /// - /// Otherwise, the path will be resolved to an absolute path, returning - /// an error if it does not exist, and converted to a [file URI] + /// Otherwise, the path will be resolved to an absolute path based on the current + /// working directory, and converted to a [file URI]. /// - /// If you wish to specify a path that does not exist on the local - /// machine you must provide it as a fully-qualified [file URI] - /// e.g. `file:///myfile.txt` + /// If the path already exists in the local filesystem this will be used to determine if this + /// [`ListingTableUrl`] refers to a collection or a single object, otherwise the presence + /// of a trailing path delimiter will be used to indicate a directory. For the avoidance + /// of ambiguity it is recommended users always include trailing `/` when intending to + /// refer to a directory. /// /// ## Glob File Paths /// @@ -78,9 +91,7 @@ impl ListingTableUrl { /// be resolved as follows. /// /// The string up to the first path segment containing a glob expression will be extracted, - /// and resolved in the same manner as a normal scheme-less path. That is, resolved to - /// an absolute path on the local filesystem, returning an error if it does not exist, - /// and converted to a [file URI] + /// and resolved in the same manner as a normal scheme-less path above. /// /// The remaining string will be interpreted as a [`glob::Pattern`] and used as a /// filter when listing files from object storage @@ -130,7 +141,7 @@ impl ListingTableUrl { /// Creates a new [`ListingTableUrl`] interpreting `s` as a filesystem path fn parse_path(s: &str) -> Result { - let (prefix, glob) = match split_glob_expression(s) { + let (path, glob) = match split_glob_expression(s) { Some((prefix, glob)) => { let glob = Pattern::new(glob) .map_err(|e| DataFusionError::External(Box::new(e)))?; @@ -139,15 +150,12 @@ impl ListingTableUrl { None => (s, None), }; - let path = std::path::Path::new(prefix).canonicalize()?; - let url = if path.is_dir() { - Url::from_directory_path(path) - } else { - Url::from_file_path(path) - } - .map_err(|_| DataFusionError::Internal(format!("Can not open path: {s}")))?; - // TODO: Currently we do not have an IO-related error variant that accepts () - // or a string. Once we have such a variant, change the error type above. + let url = url_from_filesystem_path(path).ok_or_else(|| { + DataFusionError::External( + format!("Failed to convert path to URL: {path}").into(), + ) + })?; + Self::try_new(url, glob) } @@ -162,7 +170,10 @@ impl ListingTableUrl { self.url.scheme() } - /// Return the prefix from which to list files + /// Return the URL path not excluding any glob expression + /// + /// If [`Self::is_collection`], this is the listing prefix + /// Otherwise, this is the path to the object pub fn prefix(&self) -> &Path { &self.prefix } @@ -249,6 +260,34 @@ impl ListingTableUrl { } } +/// Creates a file URL from a potentially relative filesystem path +fn url_from_filesystem_path(s: &str) -> Option { + let path = std::path::Path::new(s); + let is_dir = match path.exists() { + true => path.is_dir(), + // Fallback to inferring from trailing separator + false => std::path::is_separator(s.chars().last()?), + }; + + let from_absolute_path = |p| { + let first = match is_dir { + true => Url::from_directory_path(p).ok(), + false => Url::from_file_path(p).ok(), + }?; + + // By default from_*_path preserve relative path segments + // We therefore parse the URL again to resolve these + Url::parse(first.as_str()).ok() + }; + + if path.is_absolute() { + return from_absolute_path(path); + } + + let absolute = std::env::current_dir().ok()?.join(path); + from_absolute_path(&absolute) +} + impl AsRef for ListingTableUrl { fn as_ref(&self) -> &str { self.url.as_ref() @@ -349,6 +388,37 @@ mod tests { let url = ListingTableUrl::parse(path.to_str().unwrap()).unwrap(); assert!(url.prefix.as_ref().ends_with("bar%2Ffoo"), "{}", url.prefix); + + let url = ListingTableUrl::parse("file:///foo/../a%252Fb.txt").unwrap(); + assert_eq!(url.prefix.as_ref(), "a%2Fb.txt"); + + let url = + ListingTableUrl::parse("file:///foo/./bar/../../baz/./test.txt").unwrap(); + assert_eq!(url.prefix.as_ref(), "baz/test.txt"); + + let workdir = std::env::current_dir().unwrap(); + let t = workdir.join("non-existent"); + let a = ListingTableUrl::parse(t.to_str().unwrap()).unwrap(); + let b = ListingTableUrl::parse("non-existent").unwrap(); + assert_eq!(a, b); + assert!(a.prefix.as_ref().ends_with("non-existent")); + + let t = workdir.parent().unwrap(); + let a = ListingTableUrl::parse(t.to_str().unwrap()).unwrap(); + let b = ListingTableUrl::parse("..").unwrap(); + assert_eq!(a, b); + + let t = t.join("bar"); + let a = ListingTableUrl::parse(t.to_str().unwrap()).unwrap(); + let b = ListingTableUrl::parse("../bar").unwrap(); + assert_eq!(a, b); + assert!(a.prefix.as_ref().ends_with("bar")); + + let t = t.join(".").join("foo").join("..").join("baz"); + let a = ListingTableUrl::parse(t.to_str().unwrap()).unwrap(); + let b = ListingTableUrl::parse("../bar/./foo/../baz").unwrap(); + assert_eq!(a, b); + assert!(a.prefix.as_ref().ends_with("bar/baz")); } #[test] From 4195f2f367648f9c2ed990f2948248e53f503f9b Mon Sep 17 00:00:00 2001 From: Andrew Lamb Date: Mon, 20 Nov 2023 11:08:12 -0500 Subject: [PATCH 294/572] Minor: Add sql level test for inserting into non-existent directory (#8278) --- datafusion/sqllogictest/test_files/insert.slt | 36 +++++++++++++++++++ 1 file changed, 36 insertions(+) diff --git a/datafusion/sqllogictest/test_files/insert.slt b/datafusion/sqllogictest/test_files/insert.slt index a100b5ac6b85..aacd227cdb76 100644 --- a/datafusion/sqllogictest/test_files/insert.slt +++ b/datafusion/sqllogictest/test_files/insert.slt @@ -314,3 +314,39 @@ select * from table_without_values; statement ok drop table table_without_values; + + +### Test for creating tables into directories that do not already exist +# note use of `scratch` directory (which is cleared between runs) + +statement ok +create external table new_empty_table(x int) stored as parquet location 'test_files/scratch/insert/new_empty_table/'; -- needs trailing slash + +# should start empty +query I +select * from new_empty_table; +---- + +# should succeed and the table should create the direectory +statement ok +insert into new_empty_table values (1); + +# Now has values +query I +select * from new_empty_table; +---- +1 + +statement ok +drop table new_empty_table; + +## test we get an error if the path doesn't end in slash +statement ok +create external table bad_new_empty_table(x int) stored as parquet location 'test_files/scratch/insert/bad_new_empty_table'; -- no trailing slash + +# should fail +query error DataFusion error: Error during planning: Inserting into a ListingTable backed by a single file is not supported, URL is possibly missing a trailing `/`\. To append to an existing file use StreamTable, e\.g\. by using CREATE UNBOUNDED EXTERNAL TABLE +insert into bad_new_empty_table values (1); + +statement ok +drop table bad_new_empty_table; From f310db31b801553bae16ac6a3ef9dc76de7b016a Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E8=B0=AD=E5=B7=8D?= Date: Tue, 21 Nov 2023 04:55:18 +0800 Subject: [PATCH 295/572] Replace `array_has/array_has_all/array_has_any` macro to remove duplicate code (#8263) * Replcate array_has macro to remove deplicate codes Signed-off-by: veeupup * Replcate array_has_all macro to remove deplicate codes Signed-off-by: veeupup * Replcate array_has_any macro to remove deplicate codes Signed-off-by: veeupup --------- Signed-off-by: veeupup --- .../physical-expr/src/array_expressions.rs | 149 ++++-------------- 1 file changed, 35 insertions(+), 114 deletions(-) diff --git a/datafusion/physical-expr/src/array_expressions.rs b/datafusion/physical-expr/src/array_expressions.rs index e2d22a0d3328..c0f6c67263a7 100644 --- a/datafusion/physical-expr/src/array_expressions.rs +++ b/datafusion/physical-expr/src/array_expressions.rs @@ -1748,70 +1748,27 @@ pub fn array_ndims(args: &[ArrayRef]) -> Result { Ok(Arc::new(result) as ArrayRef) } -macro_rules! non_list_contains { - ($ARRAY:expr, $SUB_ARRAY:expr, $ARRAY_TYPE:ident) => {{ - let sub_array = downcast_arg!($SUB_ARRAY, $ARRAY_TYPE); - let mut boolean_builder = BooleanArray::builder($ARRAY.len()); - - for (arr, elem) in $ARRAY.iter().zip(sub_array.iter()) { - if let (Some(arr), Some(elem)) = (arr, elem) { - let arr = downcast_arg!(arr, $ARRAY_TYPE); - let res = arr.iter().dedup().flatten().any(|x| x == elem); - boolean_builder.append_value(res); - } - } - Ok(Arc::new(boolean_builder.finish())) - }}; -} - /// Array_has SQL function pub fn array_has(args: &[ArrayRef]) -> Result { let array = as_list_array(&args[0])?; let element = &args[1]; check_datatypes("array_has", &[array.values(), element])?; - match element.data_type() { - DataType::List(_) => { - let sub_array = as_list_array(element)?; - let mut boolean_builder = BooleanArray::builder(array.len()); - - for (arr, elem) in array.iter().zip(sub_array.iter()) { - if let (Some(arr), Some(elem)) = (arr, elem) { - let list_arr = as_list_array(&arr)?; - let res = list_arr.iter().dedup().flatten().any(|x| *x == *elem); - boolean_builder.append_value(res); - } - } - Ok(Arc::new(boolean_builder.finish())) - } - data_type => { - macro_rules! array_function { - ($ARRAY_TYPE:ident) => { - non_list_contains!(array, element, $ARRAY_TYPE) - }; - } - call_array_function!(data_type, false) - } - } -} - -macro_rules! array_has_any_non_list_check { - ($ARRAY:expr, $SUB_ARRAY:expr, $ARRAY_TYPE:ident) => {{ - let arr = downcast_arg!($ARRAY, $ARRAY_TYPE); - let sub_arr = downcast_arg!($SUB_ARRAY, $ARRAY_TYPE); + let mut boolean_builder = BooleanArray::builder(array.len()); - let mut res = false; - for elem in sub_arr.iter().dedup() { - if let Some(elem) = elem { - res |= arr.iter().dedup().flatten().any(|x| x == elem); - } else { - return internal_err!( - "array_has_any does not support Null type for element in sub_array" - ); - } + let converter = RowConverter::new(vec![SortField::new(array.value_type())])?; + let r_values = converter.convert_columns(&[element.clone()])?; + for (row_idx, arr) in array.iter().enumerate() { + if let Some(arr) = arr { + let arr_values = converter.convert_columns(&[arr])?; + let res = arr_values + .iter() + .dedup() + .any(|x| x == r_values.row(row_idx)); + boolean_builder.append_value(res); } - res - }}; + } + Ok(Arc::new(boolean_builder.finish())) } /// Array_has_any SQL function @@ -1820,55 +1777,27 @@ pub fn array_has_any(args: &[ArrayRef]) -> Result { let array = as_list_array(&args[0])?; let sub_array = as_list_array(&args[1])?; - let mut boolean_builder = BooleanArray::builder(array.len()); + + let converter = RowConverter::new(vec![SortField::new(array.value_type())])?; for (arr, sub_arr) in array.iter().zip(sub_array.iter()) { if let (Some(arr), Some(sub_arr)) = (arr, sub_arr) { - let res = match arr.data_type() { - DataType::List(_) => { - let arr = downcast_arg!(arr, ListArray); - let sub_arr = downcast_arg!(sub_arr, ListArray); - - let mut res = false; - for elem in sub_arr.iter().dedup().flatten() { - res |= arr.iter().dedup().flatten().any(|x| *x == *elem); - } - res + let arr_values = converter.convert_columns(&[arr])?; + let sub_arr_values = converter.convert_columns(&[sub_arr])?; + + let mut res = false; + for elem in sub_arr_values.iter().dedup() { + res |= arr_values.iter().dedup().any(|x| x == elem); + if res { + break; } - data_type => { - macro_rules! array_function { - ($ARRAY_TYPE:ident) => { - array_has_any_non_list_check!(arr, sub_arr, $ARRAY_TYPE) - }; - } - call_array_function!(data_type, false) - } - }; + } boolean_builder.append_value(res); } } Ok(Arc::new(boolean_builder.finish())) } -macro_rules! array_has_all_non_list_check { - ($ARRAY:expr, $SUB_ARRAY:expr, $ARRAY_TYPE:ident) => {{ - let arr = downcast_arg!($ARRAY, $ARRAY_TYPE); - let sub_arr = downcast_arg!($SUB_ARRAY, $ARRAY_TYPE); - - let mut res = true; - for elem in sub_arr.iter().dedup() { - if let Some(elem) = elem { - res &= arr.iter().dedup().flatten().any(|x| x == elem); - } else { - return internal_err!( - "array_has_all does not support Null type for element in sub_array" - ); - } - } - res - }}; -} - /// Array_has_all SQL function pub fn array_has_all(args: &[ArrayRef]) -> Result { check_datatypes("array_has_all", &[&args[0], &args[1]])?; @@ -1877,28 +1806,20 @@ pub fn array_has_all(args: &[ArrayRef]) -> Result { let sub_array = as_list_array(&args[1])?; let mut boolean_builder = BooleanArray::builder(array.len()); + + let converter = RowConverter::new(vec![SortField::new(array.value_type())])?; for (arr, sub_arr) in array.iter().zip(sub_array.iter()) { if let (Some(arr), Some(sub_arr)) = (arr, sub_arr) { - let res = match arr.data_type() { - DataType::List(_) => { - let arr = downcast_arg!(arr, ListArray); - let sub_arr = downcast_arg!(sub_arr, ListArray); - - let mut res = true; - for elem in sub_arr.iter().dedup().flatten() { - res &= arr.iter().dedup().flatten().any(|x| *x == *elem); - } - res - } - data_type => { - macro_rules! array_function { - ($ARRAY_TYPE:ident) => { - array_has_all_non_list_check!(arr, sub_arr, $ARRAY_TYPE) - }; - } - call_array_function!(data_type, false) + let arr_values = converter.convert_columns(&[arr])?; + let sub_arr_values = converter.convert_columns(&[sub_arr])?; + + let mut res = true; + for elem in sub_arr_values.iter().dedup() { + res &= arr_values.iter().dedup().any(|x| x == elem); + if !res { + break; } - }; + } boolean_builder.append_value(res); } } From 58483fbbbe732cca070209c82ae7e5cfd031f6ae Mon Sep 17 00:00:00 2001 From: Andrew Lamb Date: Tue, 21 Nov 2023 05:57:35 -0500 Subject: [PATCH 296/572] Fix bug in field level metadata matching code (#8286) * Fix bug in field level metadata matching code * improve comment * single map --- datafusion/physical-plan/src/projection.rs | 16 ++--- datafusion/sqllogictest/src/test_context.rs | 44 ++++++++++--- .../sqllogictest/test_files/metadata.slt | 62 +++++++++++++++++++ 3 files changed, 104 insertions(+), 18 deletions(-) create mode 100644 datafusion/sqllogictest/test_files/metadata.slt diff --git a/datafusion/physical-plan/src/projection.rs b/datafusion/physical-plan/src/projection.rs index b8e2d0e425d4..dfb860bc8cf3 100644 --- a/datafusion/physical-plan/src/projection.rs +++ b/datafusion/physical-plan/src/projection.rs @@ -257,16 +257,12 @@ fn get_field_metadata( e: &Arc, input_schema: &Schema, ) -> Option> { - let name = if let Some(column) = e.as_any().downcast_ref::() { - column.name() - } else { - return None; - }; - - input_schema - .field_with_name(name) - .ok() - .map(|f| f.metadata().clone()) + // Look up field by index in schema (not NAME as there can be more than one + // column with the same name) + e.as_any() + .downcast_ref::() + .map(|column| input_schema.field(column.index()).metadata()) + .cloned() } fn stats_projection( diff --git a/datafusion/sqllogictest/src/test_context.rs b/datafusion/sqllogictest/src/test_context.rs index b2314f34f360..f5ab8f71aaaf 100644 --- a/datafusion/sqllogictest/src/test_context.rs +++ b/datafusion/sqllogictest/src/test_context.rs @@ -35,6 +35,7 @@ use datafusion::{ }; use datafusion_common::DataFusionError; use log::info; +use std::collections::HashMap; use std::fs::File; use std::io::Write; use std::path::Path; @@ -57,8 +58,8 @@ impl TestContext { } } - /// Create a SessionContext, configured for the specific test, if - /// possible. + /// Create a SessionContext, configured for the specific sqllogictest + /// test(.slt file) , if possible. /// /// If `None` is returned (e.g. because some needed feature is not /// enabled), the file should be skipped @@ -67,7 +68,7 @@ impl TestContext { // hardcode target partitions so plans are deterministic .with_target_partitions(4); - let test_ctx = TestContext::new(SessionContext::new_with_config(config)); + let mut test_ctx = TestContext::new(SessionContext::new_with_config(config)); let file_name = relative_path.file_name().unwrap().to_str().unwrap(); match file_name { @@ -86,10 +87,8 @@ impl TestContext { "avro.slt" => { #[cfg(feature = "avro")] { - let mut test_ctx = test_ctx; info!("Registering avro tables"); register_avro_tables(&mut test_ctx).await; - return Some(test_ctx); } #[cfg(not(feature = "avro"))] { @@ -99,10 +98,11 @@ impl TestContext { } "joins.slt" => { info!("Registering partition table tables"); - - let mut test_ctx = test_ctx; register_partition_table(&mut test_ctx).await; - return Some(test_ctx); + } + "metadata.slt" => { + info!("Registering metadata table tables"); + register_metadata_tables(test_ctx.session_ctx()).await; } _ => { info!("Using default SessionContext"); @@ -299,3 +299,31 @@ fn table_with_many_types() -> Arc { let provider = MemTable::try_new(Arc::new(schema), vec![vec![batch]]).unwrap(); Arc::new(provider) } + +/// Registers a table_with_metadata that contains both field level and Table level metadata +pub async fn register_metadata_tables(ctx: &SessionContext) { + let id = Field::new("id", DataType::Int32, true).with_metadata(HashMap::from([( + String::from("metadata_key"), + String::from("the id field"), + )])); + let name = Field::new("name", DataType::Utf8, true).with_metadata(HashMap::from([( + String::from("metadata_key"), + String::from("the name field"), + )])); + + let schema = Schema::new(vec![id, name]).with_metadata(HashMap::from([( + String::from("metadata_key"), + String::from("the entire schema"), + )])); + + let batch = RecordBatch::try_new( + Arc::new(schema), + vec![ + Arc::new(Int32Array::from(vec![Some(1), None, Some(3)])) as _, + Arc::new(StringArray::from(vec![None, Some("bar"), Some("baz")])) as _, + ], + ) + .unwrap(); + + ctx.register_batch("table_with_metadata", batch).unwrap(); +} diff --git a/datafusion/sqllogictest/test_files/metadata.slt b/datafusion/sqllogictest/test_files/metadata.slt new file mode 100644 index 000000000000..3b2b219244f5 --- /dev/null +++ b/datafusion/sqllogictest/test_files/metadata.slt @@ -0,0 +1,62 @@ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at + +# http://www.apache.org/licenses/LICENSE-2.0 + +# Unless required by applicable law or agreed to in writing, +# software distributed under the License is distributed on an +# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +# KIND, either express or implied. See the License for the +# specific language governing permissions and limitations +# under the License. + +########## +## Tests for tables that has both metadata on each field as well as metadata on +## the schema itself. +########## + +## Note that table_with_metadata is defined using Rust code +## in the test harness as there is no way to define schema +## with metadata in SQL. + +query IT +select * from table_with_metadata; +---- +1 NULL +NULL bar +3 baz + +query I rowsort +SELECT ( + SELECT id FROM table_with_metadata + ) UNION ( + SELECT id FROM table_with_metadata + ); +---- +1 +3 +NULL + +query I rowsort +SELECT "data"."id" +FROM + ( + (SELECT "id" FROM "table_with_metadata") + UNION + (SELECT "id" FROM "table_with_metadata") + ) as "data", + ( + SELECT "id" FROM "table_with_metadata" + ) as "samples" +WHERE "data"."id" = "samples"."id"; +---- +1 +3 + +statement ok +drop table table_with_metadata; From e9b9645ca0da5c6ce3d1e7d8210e442cf565cea0 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Berkay=20=C5=9Eahin?= <124376117+berkaysynnada@users.noreply.github.com> Date: Tue, 21 Nov 2023 19:36:51 +0300 Subject: [PATCH 297/572] Interval Arithmetic Updates (#8276) * Interval lib can be accessible from logical plan * committed to merge precision PR * minor fix after merge, adding ts-interval handling * bound openness removed * Remove all interval bound related code * test fix * fix docstrings * Fix after merge * Minor changes * Simplifications * Resolve linter errors * Addressing reviews * Fix win tests * Update interval_arithmetic.rs * Code simplifications * Review Part 1 * Review Part 2 * Addressing Ozan's feedback * Resolving conflicts * Review Part 3 * Better cardinality calculation * fix clippy * Review Part 4 * type check, test polish, bug fix * Constructs filter graph with datatypes * Update Cargo.lock * Update Cargo.toml * Review * Other expectations of AND * OR operator implementation * Certainly false asserting comparison operators * Update interval_arithmetic.rs * Tests added, is_superset renamed * Final review * Resolving conflicts * Address review feedback --------- Co-authored-by: Mustafa Akur Co-authored-by: Mehmet Ozan Kabak --- datafusion-cli/Cargo.lock | 55 +- datafusion/common/Cargo.toml | 12 +- datafusion/common/src/lib.rs | 1 + .../src/intervals => common/src}/rounding.rs | 8 +- datafusion/common/src/scalar.rs | 42 + datafusion/expr/Cargo.toml | 5 +- datafusion/expr/src/interval_arithmetic.rs | 3307 +++++++++++++++++ datafusion/expr/src/lib.rs | 24 +- datafusion/expr/src/type_coercion/binary.rs | 1 - .../simplify_expressions/expr_simplifier.rs | 44 +- .../src/simplify_expressions/guarantees.rs | 69 +- datafusion/physical-expr/Cargo.toml | 8 +- datafusion/physical-expr/src/analysis.rs | 158 +- .../physical-expr/src/expressions/binary.rs | 117 +- .../physical-expr/src/expressions/cast.rs | 18 +- .../physical-expr/src/expressions/negative.rs | 52 +- .../physical-expr/src/intervals/cp_solver.rs | 1038 +++--- .../src/intervals/interval_aritmetic.rs | 1886 ---------- datafusion/physical-expr/src/intervals/mod.rs | 5 - .../physical-expr/src/intervals/utils.rs | 104 +- datafusion/physical-expr/src/physical_expr.rs | 30 +- .../physical-expr/src/sort_properties.rs | 2 +- datafusion/physical-expr/src/utils.rs | 14 +- datafusion/physical-plan/src/filter.rs | 23 +- .../src/joins/stream_join_utils.rs | 33 +- .../src/joins/symmetric_hash_join.rs | 11 +- .../physical-plan/src/joins/test_utils.rs | 14 + datafusion/physical-plan/src/joins/utils.rs | 98 +- 28 files changed, 4406 insertions(+), 2773 deletions(-) rename datafusion/{physical-expr/src/intervals => common/src}/rounding.rs (98%) create mode 100644 datafusion/expr/src/interval_arithmetic.rs delete mode 100644 datafusion/physical-expr/src/intervals/interval_aritmetic.rs diff --git a/datafusion-cli/Cargo.lock b/datafusion-cli/Cargo.lock index 06bc14c5b656..fa2832ab3fc6 100644 --- a/datafusion-cli/Cargo.lock +++ b/datafusion-cli/Cargo.lock @@ -360,9 +360,9 @@ dependencies = [ [[package]] name = "async-compression" -version = "0.4.5" +version = "0.4.4" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "bc2d0cfb2a7388d34f590e76686704c494ed7aaceed62ee1ba35cbf363abc2a5" +checksum = "f658e2baef915ba0f26f1f7c42bfb8e12f532a01f449a090ded75ae7a07e9ba2" dependencies = [ "bzip2", "flate2", @@ -851,11 +851,10 @@ dependencies = [ [[package]] name = "cc" -version = "1.0.83" +version = "1.0.84" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "f1174fb0b6ec23863f8b971027804a42614e347eafb0a95bf0b12cdae21fc4d0" +checksum = "0f8e7c90afad890484a21653d08b6e209ae34770fb5ee298f9c699fcc1e5c856" dependencies = [ - "jobserver", "libc", ] @@ -1180,6 +1179,7 @@ dependencies = [ "arrow-schema", "chrono", "half", + "libc", "num_cpus", "object_store", "parquet", @@ -1213,6 +1213,7 @@ dependencies = [ "arrow", "arrow-array", "datafusion-common", + "paste", "sqlparser", "strum", "strum_macros", @@ -1255,7 +1256,6 @@ dependencies = [ "hex", "indexmap 2.1.0", "itertools 0.12.0", - "libc", "log", "md-5", "paste", @@ -1423,9 +1423,9 @@ checksum = "5443807d6dff69373d433ab9ef5378ad8df50ca6298caf15de6e52e24aaf54d5" [[package]] name = "errno" -version = "0.3.7" +version = "0.3.6" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "f258a7194e7f7c2a7837a8913aeab7fd8c383457034fa20ce4dd3dcb813e8eb8" +checksum = "7c18ee0ed65a5f1f81cac6b1d213b69c35fa47d4252ad41f1486dbd8226fe36e" dependencies = [ "libc", "windows-sys", @@ -1647,9 +1647,9 @@ checksum = "d2fabcfbdc87f4758337ca535fb41a6d701b65693ce38287d856d1674551ec9b" [[package]] name = "h2" -version = "0.3.22" +version = "0.3.21" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "4d6250322ef6e60f93f9a2162799302cd6f68f79f6e5d85c8c16f14d1d958178" +checksum = "91fc23aa11be92976ef4729127f1a74adf36d8436f7816b185d18df956790833" dependencies = [ "bytes", "fnv", @@ -1657,7 +1657,7 @@ dependencies = [ "futures-sink", "futures-util", "http", - "indexmap 2.1.0", + "indexmap 1.9.3", "slab", "tokio", "tokio-util", @@ -1738,9 +1738,9 @@ dependencies = [ [[package]] name = "http" -version = "0.2.11" +version = "0.2.10" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "8947b1a6fad4393052c7ba1f4cd97bed3e953a95c79c92ad9b051a04611d9fbb" +checksum = "f95b9abcae896730d42b78e09c155ed4ddf82c07b4de772c64aee5b2d8b7c150" dependencies = [ "bytes", "fnv", @@ -1927,15 +1927,6 @@ version = "1.0.9" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "af150ab688ff2122fcef229be89cb50dd66af9e01a4ff320cc137eecc9bacc38" -[[package]] -name = "jobserver" -version = "0.1.27" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "8c37f63953c4c63420ed5fd3d6d398c719489b9f872b9fa683262f8edd363c7d" -dependencies = [ - "libc", -] - [[package]] name = "js-sys" version = "0.3.65" @@ -2830,9 +2821,9 @@ dependencies = [ [[package]] name = "rustix" -version = "0.38.24" +version = "0.38.21" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "9ad981d6c340a49cdc40a1028d9c6084ec7e9fa33fcb839cab656a267071e234" +checksum = "2b426b0506e5d50a7d8dafcf2e81471400deb602392c7dd110815afb4eaf02a3" dependencies = [ "bitflags 2.4.1", "errno", @@ -3260,9 +3251,9 @@ dependencies = [ [[package]] name = "termcolor" -version = "1.4.0" +version = "1.3.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "ff1bc3d3f05aff0403e8ac0d92ced918ec05b666a43f83297ccef5bea8a3d449" +checksum = "6093bad37da69aab9d123a8091e4be0aa4a03e4d601ec641c327398315f62b64" dependencies = [ "winapi-util", ] @@ -3901,18 +3892,18 @@ dependencies = [ [[package]] name = "zerocopy" -version = "0.7.26" +version = "0.7.25" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "e97e415490559a91254a2979b4829267a57d2fcd741a98eee8b722fb57289aa0" +checksum = "8cd369a67c0edfef15010f980c3cbe45d7f651deac2cd67ce097cd801de16557" dependencies = [ "zerocopy-derive", ] [[package]] name = "zerocopy-derive" -version = "0.7.26" +version = "0.7.25" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "dd7e48ccf166952882ca8bd778a43502c64f33bf94c12ebe2a7f08e5a0f6689f" +checksum = "c2f140bda219a26ccc0cdb03dba58af72590c53b22642577d88a927bc5c87d6b" dependencies = [ "proc-macro2", "quote", @@ -3921,9 +3912,9 @@ dependencies = [ [[package]] name = "zeroize" -version = "1.6.1" +version = "1.6.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "12a3946ecfc929b583800f4629b6c25b88ac6e92a40ea5670f77112a85d40a8b" +checksum = "2a0956f1ba7c7909bfb66c2e9e4124ab6f6482560f6628b5aaeba39207c9aad9" [[package]] name = "zstd" diff --git a/datafusion/common/Cargo.toml b/datafusion/common/Cargo.toml index b3a810153923..b69e1f7f3d10 100644 --- a/datafusion/common/Cargo.toml +++ b/datafusion/common/Cargo.toml @@ -38,14 +38,22 @@ backtrace = [] pyarrow = ["pyo3", "arrow/pyarrow", "parquet"] [dependencies] -ahash = { version = "0.8", default-features = false, features = ["runtime-rng"] } -apache-avro = { version = "0.16", default-features = false, features = ["bzip", "snappy", "xz", "zstandard"], optional = true } +ahash = { version = "0.8", default-features = false, features = [ + "runtime-rng", +] } +apache-avro = { version = "0.16", default-features = false, features = [ + "bzip", + "snappy", + "xz", + "zstandard", +], optional = true } arrow = { workspace = true } arrow-array = { workspace = true } arrow-buffer = { workspace = true } arrow-schema = { workspace = true } chrono = { workspace = true } half = { version = "2.1", default-features = false } +libc = "0.2.140" num_cpus = { workspace = true } object_store = { workspace = true, optional = true } parquet = { workspace = true, optional = true, default-features = true } diff --git a/datafusion/common/src/lib.rs b/datafusion/common/src/lib.rs index 53c3cfddff8d..90fb4a88149c 100644 --- a/datafusion/common/src/lib.rs +++ b/datafusion/common/src/lib.rs @@ -34,6 +34,7 @@ pub mod file_options; pub mod format; pub mod hash_utils; pub mod parsers; +pub mod rounding; pub mod scalar; pub mod stats; pub mod test_util; diff --git a/datafusion/physical-expr/src/intervals/rounding.rs b/datafusion/common/src/rounding.rs similarity index 98% rename from datafusion/physical-expr/src/intervals/rounding.rs rename to datafusion/common/src/rounding.rs index c1172fba9152..413067ecd61e 100644 --- a/datafusion/physical-expr/src/intervals/rounding.rs +++ b/datafusion/common/src/rounding.rs @@ -22,8 +22,8 @@ use std::ops::{Add, BitAnd, Sub}; -use datafusion_common::Result; -use datafusion_common::ScalarValue; +use crate::Result; +use crate::ScalarValue; // Define constants for ARM #[cfg(all(target_arch = "aarch64", not(target_os = "windows")))] @@ -162,7 +162,7 @@ impl FloatBits for f64 { /// # Examples /// /// ``` -/// use datafusion_physical_expr::intervals::rounding::next_up; +/// use datafusion_common::rounding::next_up; /// /// let f: f32 = 1.0; /// let next_f = next_up(f); @@ -195,7 +195,7 @@ pub fn next_up(float: F) -> F { /// # Examples /// /// ``` -/// use datafusion_physical_expr::intervals::rounding::next_down; +/// use datafusion_common::rounding::next_down; /// /// let f: f32 = 1.0; /// let next_f = next_down(f); diff --git a/datafusion/common/src/scalar.rs b/datafusion/common/src/scalar.rs index e8dac2a7f486..fd1ceb5fad78 100644 --- a/datafusion/common/src/scalar.rs +++ b/datafusion/common/src/scalar.rs @@ -988,6 +988,48 @@ impl ScalarValue { Self::try_from_array(r.as_ref(), 0) } + /// Wrapping multiplication of `ScalarValue` + /// + /// NB: operating on `ScalarValue` directly is not efficient, performance sensitive code + /// should operate on Arrays directly, using vectorized array kernels. + pub fn mul>(&self, other: T) -> Result { + let r = mul_wrapping(&self.to_scalar()?, &other.borrow().to_scalar()?)?; + Self::try_from_array(r.as_ref(), 0) + } + + /// Checked multiplication of `ScalarValue` + /// + /// NB: operating on `ScalarValue` directly is not efficient, performance sensitive code + /// should operate on Arrays directly, using vectorized array kernels. + pub fn mul_checked>(&self, other: T) -> Result { + let r = mul(&self.to_scalar()?, &other.borrow().to_scalar()?)?; + Self::try_from_array(r.as_ref(), 0) + } + + /// Performs `lhs / rhs` + /// + /// Overflow or division by zero will result in an error, with exception to + /// floating point numbers, which instead follow the IEEE 754 rules. + /// + /// NB: operating on `ScalarValue` directly is not efficient, performance sensitive code + /// should operate on Arrays directly, using vectorized array kernels. + pub fn div>(&self, other: T) -> Result { + let r = div(&self.to_scalar()?, &other.borrow().to_scalar()?)?; + Self::try_from_array(r.as_ref(), 0) + } + + /// Performs `lhs % rhs` + /// + /// Overflow or division by zero will result in an error, with exception to + /// floating point numbers, which instead follow the IEEE 754 rules. + /// + /// NB: operating on `ScalarValue` directly is not efficient, performance sensitive code + /// should operate on Arrays directly, using vectorized array kernels. + pub fn rem>(&self, other: T) -> Result { + let r = rem(&self.to_scalar()?, &other.borrow().to_scalar()?)?; + Self::try_from_array(r.as_ref(), 0) + } + pub fn is_unsigned(&self) -> bool { matches!( self, diff --git a/datafusion/expr/Cargo.toml b/datafusion/expr/Cargo.toml index 5b1b42153877..3e05dae61954 100644 --- a/datafusion/expr/Cargo.toml +++ b/datafusion/expr/Cargo.toml @@ -35,10 +35,13 @@ path = "src/lib.rs" [features] [dependencies] -ahash = { version = "0.8", default-features = false, features = ["runtime-rng"] } +ahash = { version = "0.8", default-features = false, features = [ + "runtime-rng", +] } arrow = { workspace = true } arrow-array = { workspace = true } datafusion-common = { workspace = true } +paste = "^1.0" sqlparser = { workspace = true } strum = { version = "0.25.0", features = ["derive"] } strum_macros = "0.25.0" diff --git a/datafusion/expr/src/interval_arithmetic.rs b/datafusion/expr/src/interval_arithmetic.rs new file mode 100644 index 000000000000..c85c6fc66bc8 --- /dev/null +++ b/datafusion/expr/src/interval_arithmetic.rs @@ -0,0 +1,3307 @@ +// 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. + +//! Interval arithmetic library + +use std::borrow::Borrow; +use std::fmt::{self, Display, Formatter}; +use std::ops::{AddAssign, SubAssign}; + +use crate::type_coercion::binary::get_result_type; +use crate::Operator; + +use arrow::compute::{cast_with_options, CastOptions}; +use arrow::datatypes::DataType; +use arrow::datatypes::{IntervalUnit, TimeUnit}; +use datafusion_common::rounding::{alter_fp_rounding_mode, next_down, next_up}; +use datafusion_common::{internal_err, DataFusionError, Result, ScalarValue}; + +macro_rules! get_extreme_value { + ($extreme:ident, $value:expr) => { + match $value { + DataType::UInt8 => ScalarValue::UInt8(Some(u8::$extreme)), + DataType::UInt16 => ScalarValue::UInt16(Some(u16::$extreme)), + DataType::UInt32 => ScalarValue::UInt32(Some(u32::$extreme)), + DataType::UInt64 => ScalarValue::UInt64(Some(u64::$extreme)), + DataType::Int8 => ScalarValue::Int8(Some(i8::$extreme)), + DataType::Int16 => ScalarValue::Int16(Some(i16::$extreme)), + DataType::Int32 => ScalarValue::Int32(Some(i32::$extreme)), + DataType::Int64 => ScalarValue::Int64(Some(i64::$extreme)), + DataType::Float32 => ScalarValue::Float32(Some(f32::$extreme)), + DataType::Float64 => ScalarValue::Float64(Some(f64::$extreme)), + DataType::Duration(TimeUnit::Second) => { + ScalarValue::DurationSecond(Some(i64::$extreme)) + } + DataType::Duration(TimeUnit::Millisecond) => { + ScalarValue::DurationMillisecond(Some(i64::$extreme)) + } + DataType::Duration(TimeUnit::Microsecond) => { + ScalarValue::DurationMicrosecond(Some(i64::$extreme)) + } + DataType::Duration(TimeUnit::Nanosecond) => { + ScalarValue::DurationNanosecond(Some(i64::$extreme)) + } + DataType::Timestamp(TimeUnit::Second, _) => { + ScalarValue::TimestampSecond(Some(i64::$extreme), None) + } + DataType::Timestamp(TimeUnit::Millisecond, _) => { + ScalarValue::TimestampMillisecond(Some(i64::$extreme), None) + } + DataType::Timestamp(TimeUnit::Microsecond, _) => { + ScalarValue::TimestampMicrosecond(Some(i64::$extreme), None) + } + DataType::Timestamp(TimeUnit::Nanosecond, _) => { + ScalarValue::TimestampNanosecond(Some(i64::$extreme), None) + } + DataType::Interval(IntervalUnit::YearMonth) => { + ScalarValue::IntervalYearMonth(Some(i32::$extreme)) + } + DataType::Interval(IntervalUnit::DayTime) => { + ScalarValue::IntervalDayTime(Some(i64::$extreme)) + } + DataType::Interval(IntervalUnit::MonthDayNano) => { + ScalarValue::IntervalMonthDayNano(Some(i128::$extreme)) + } + _ => unreachable!(), + } + }; +} + +macro_rules! value_transition { + ($bound:ident, $direction:expr, $value:expr) => { + match $value { + UInt8(Some(value)) if value == u8::$bound => UInt8(None), + UInt16(Some(value)) if value == u16::$bound => UInt16(None), + UInt32(Some(value)) if value == u32::$bound => UInt32(None), + UInt64(Some(value)) if value == u64::$bound => UInt64(None), + Int8(Some(value)) if value == i8::$bound => Int8(None), + Int16(Some(value)) if value == i16::$bound => Int16(None), + Int32(Some(value)) if value == i32::$bound => Int32(None), + Int64(Some(value)) if value == i64::$bound => Int64(None), + Float32(Some(value)) if value == f32::$bound => Float32(None), + Float64(Some(value)) if value == f64::$bound => Float64(None), + DurationSecond(Some(value)) if value == i64::$bound => DurationSecond(None), + DurationMillisecond(Some(value)) if value == i64::$bound => { + DurationMillisecond(None) + } + DurationMicrosecond(Some(value)) if value == i64::$bound => { + DurationMicrosecond(None) + } + DurationNanosecond(Some(value)) if value == i64::$bound => { + DurationNanosecond(None) + } + TimestampSecond(Some(value), tz) if value == i64::$bound => { + TimestampSecond(None, tz) + } + TimestampMillisecond(Some(value), tz) if value == i64::$bound => { + TimestampMillisecond(None, tz) + } + TimestampMicrosecond(Some(value), tz) if value == i64::$bound => { + TimestampMicrosecond(None, tz) + } + TimestampNanosecond(Some(value), tz) if value == i64::$bound => { + TimestampNanosecond(None, tz) + } + IntervalYearMonth(Some(value)) if value == i32::$bound => { + IntervalYearMonth(None) + } + IntervalDayTime(Some(value)) if value == i64::$bound => IntervalDayTime(None), + IntervalMonthDayNano(Some(value)) if value == i128::$bound => { + IntervalMonthDayNano(None) + } + _ => next_value_helper::<$direction>($value), + } + }; +} + +/// The `Interval` type represents a closed interval used for computing +/// reliable bounds for mathematical expressions. +/// +/// Conventions: +/// +/// 1. **Closed bounds**: The interval always encompasses its endpoints. We +/// accommodate operations resulting in open intervals by incrementing or +/// decrementing the interval endpoint value to its successor/predecessor. +/// +/// 2. **Unbounded endpoints**: If the `lower` or `upper` bounds are indeterminate, +/// they are labeled as *unbounded*. This is represented using a `NULL`. +/// +/// 3. **Overflow handling**: If the `lower` or `upper` endpoints exceed their +/// limits after any operation, they either become unbounded or they are fixed +/// to the maximum/minimum value of the datatype, depending on the direction +/// of the overflowing endpoint, opting for the safer choice. +/// +/// 4. **Floating-point special cases**: +/// - `INF` values are converted to `NULL`s while constructing an interval to +/// ensure consistency, with other data types. +/// - `NaN` (Not a Number) results are conservatively result in unbounded +/// endpoints. +#[derive(Debug, Clone, PartialEq, Eq)] +pub struct Interval { + lower: ScalarValue, + upper: ScalarValue, +} + +/// This macro handles the `NaN` and `INF` floating point values. +/// +/// - `NaN` values are always converted to unbounded i.e. `NULL` values. +/// - For lower bounds: +/// - A `NEG_INF` value is converted to a `NULL`. +/// - An `INF` value is conservatively converted to the maximum representable +/// number for the floating-point type in question. In this case, converting +/// to `NULL` doesn't make sense as it would be interpreted as a `NEG_INF`. +/// - For upper bounds: +/// - An `INF` value is converted to a `NULL`. +/// - An `NEG_INF` value is conservatively converted to the minimum representable +/// number for the floating-point type in question. In this case, converting +/// to `NULL` doesn't make sense as it would be interpreted as an `INF`. +macro_rules! handle_float_intervals { + ($scalar_type:ident, $primitive_type:ident, $lower:expr, $upper:expr) => {{ + let lower = match $lower { + ScalarValue::$scalar_type(Some(l_val)) + if l_val == $primitive_type::NEG_INFINITY || l_val.is_nan() => + { + ScalarValue::$scalar_type(None) + } + ScalarValue::$scalar_type(Some(l_val)) + if l_val == $primitive_type::INFINITY => + { + ScalarValue::$scalar_type(Some($primitive_type::MAX)) + } + value @ ScalarValue::$scalar_type(Some(_)) => value, + _ => ScalarValue::$scalar_type(None), + }; + + let upper = match $upper { + ScalarValue::$scalar_type(Some(r_val)) + if r_val == $primitive_type::INFINITY || r_val.is_nan() => + { + ScalarValue::$scalar_type(None) + } + ScalarValue::$scalar_type(Some(r_val)) + if r_val == $primitive_type::NEG_INFINITY => + { + ScalarValue::$scalar_type(Some($primitive_type::MIN)) + } + value @ ScalarValue::$scalar_type(Some(_)) => value, + _ => ScalarValue::$scalar_type(None), + }; + + Interval { lower, upper } + }}; +} + +/// Ordering floating-point numbers according to their binary representations +/// contradicts with their natural ordering. Floating-point number ordering +/// after unsigned integer transmutation looks like: +/// +/// ```text +/// 0, 1, 2, 3, ..., MAX, -0, -1, -2, ..., -MAX +/// ``` +/// +/// This macro applies a one-to-one map that fixes the ordering above. +macro_rules! map_floating_point_order { + ($value:expr, $ty:ty) => {{ + let num_bits = std::mem::size_of::<$ty>() * 8; + let sign_bit = 1 << (num_bits - 1); + if $value & sign_bit == sign_bit { + // Negative numbers: + !$value + } else { + // Positive numbers: + $value | sign_bit + } + }}; +} + +impl Interval { + /// Attempts to create a new `Interval` from the given lower and upper bounds. + /// + /// # Notes + /// + /// This constructor creates intervals in a "canonical" form where: + /// - **Boolean intervals**: + /// - Unboundedness (`NULL`) for boolean endpoints is converted to `false` + /// for lower and `true` for upper bounds. + /// - **Floating-point intervals**: + /// - Floating-point endpoints with `NaN`, `INF`, or `NEG_INF` are converted + /// to `NULL`s. + pub fn try_new(lower: ScalarValue, upper: ScalarValue) -> Result { + if lower.data_type() != upper.data_type() { + return internal_err!("Endpoints of an Interval should have the same type"); + } + + let interval = Self::new(lower, upper); + + if interval.lower.is_null() + || interval.upper.is_null() + || interval.lower <= interval.upper + { + Ok(interval) + } else { + internal_err!( + "Interval's lower bound {} is greater than the upper bound {}", + interval.lower, + interval.upper + ) + } + } + + /// Only for internal usage. Responsible for standardizing booleans and + /// floating-point values, as well as fixing NaNs. It doesn't validate + /// the given bounds for ordering, or verify that they have the same data + /// type. For its user-facing counterpart and more details, see + /// [`Interval::try_new`]. + fn new(lower: ScalarValue, upper: ScalarValue) -> Self { + if let ScalarValue::Boolean(lower_bool) = lower { + let ScalarValue::Boolean(upper_bool) = upper else { + // We are sure that upper and lower bounds have the same type. + unreachable!(); + }; + // Standardize boolean interval endpoints: + Self { + lower: ScalarValue::Boolean(Some(lower_bool.unwrap_or(false))), + upper: ScalarValue::Boolean(Some(upper_bool.unwrap_or(true))), + } + } + // Standardize floating-point endpoints: + else if lower.data_type() == DataType::Float32 { + handle_float_intervals!(Float32, f32, lower, upper) + } else if lower.data_type() == DataType::Float64 { + handle_float_intervals!(Float64, f64, lower, upper) + } else { + // Other data types do not require standardization: + Self { lower, upper } + } + } + + /// Convenience function to create a new `Interval` from the given (optional) + /// bounds, for use in tests only. Absence of either endpoint indicates + /// unboundedness on that side. See [`Interval::try_new`] for more information. + pub fn make(lower: Option, upper: Option) -> Result + where + ScalarValue: From>, + { + Self::try_new(ScalarValue::from(lower), ScalarValue::from(upper)) + } + + /// Creates an unbounded interval from both sides if the datatype supported. + pub fn make_unbounded(data_type: &DataType) -> Result { + let unbounded_endpoint = ScalarValue::try_from(data_type)?; + Ok(Self::new(unbounded_endpoint.clone(), unbounded_endpoint)) + } + + /// Returns a reference to the lower bound. + pub fn lower(&self) -> &ScalarValue { + &self.lower + } + + /// Returns a reference to the upper bound. + pub fn upper(&self) -> &ScalarValue { + &self.upper + } + + /// Converts this `Interval` into its boundary scalar values. It's useful + /// when you need to work with the individual bounds directly. + pub fn into_bounds(self) -> (ScalarValue, ScalarValue) { + (self.lower, self.upper) + } + + /// This function returns the data type of this interval. + pub fn data_type(&self) -> DataType { + let lower_type = self.lower.data_type(); + let upper_type = self.upper.data_type(); + + // There must be no way to create an interval whose endpoints have + // different types. + assert!( + lower_type == upper_type, + "Interval bounds have different types: {lower_type} != {upper_type}" + ); + lower_type + } + + /// Casts this interval to `data_type` using `cast_options`. + pub fn cast_to( + &self, + data_type: &DataType, + cast_options: &CastOptions, + ) -> Result { + Self::try_new( + cast_scalar_value(&self.lower, data_type, cast_options)?, + cast_scalar_value(&self.upper, data_type, cast_options)?, + ) + } + + pub const CERTAINLY_FALSE: Self = Self { + lower: ScalarValue::Boolean(Some(false)), + upper: ScalarValue::Boolean(Some(false)), + }; + + pub const UNCERTAIN: Self = Self { + lower: ScalarValue::Boolean(Some(false)), + upper: ScalarValue::Boolean(Some(true)), + }; + + pub const CERTAINLY_TRUE: Self = Self { + lower: ScalarValue::Boolean(Some(true)), + upper: ScalarValue::Boolean(Some(true)), + }; + + /// Decide if this interval is certainly greater than, possibly greater than, + /// or can't be greater than `other` by returning `[true, true]`, + /// `[false, true]` or `[false, false]` respectively. + /// + /// NOTE: This function only works with intervals of the same data type. + /// Attempting to compare intervals of different data types will lead + /// to an error. + pub(crate) fn gt>(&self, other: T) -> Result { + let rhs = other.borrow(); + if self.data_type().ne(&rhs.data_type()) { + internal_err!( + "Only intervals with the same data type are comparable, lhs:{}, rhs:{}", + self.data_type(), + rhs.data_type() + ) + } else if !(self.upper.is_null() || rhs.lower.is_null()) + && self.upper <= rhs.lower + { + // Values in this interval are certainly less than or equal to + // those in the given interval. + Ok(Self::CERTAINLY_FALSE) + } else if !(self.lower.is_null() || rhs.upper.is_null()) + && (self.lower > rhs.upper) + { + // Values in this interval are certainly greater than those in the + // given interval. + Ok(Self::CERTAINLY_TRUE) + } else { + // All outcomes are possible. + Ok(Self::UNCERTAIN) + } + } + + /// Decide if this interval is certainly greater than or equal to, possibly + /// greater than or equal to, or can't be greater than or equal to `other` + /// by returning `[true, true]`, `[false, true]` or `[false, false]` respectively. + /// + /// NOTE: This function only works with intervals of the same data type. + /// Attempting to compare intervals of different data types will lead + /// to an error. + pub(crate) fn gt_eq>(&self, other: T) -> Result { + let rhs = other.borrow(); + if self.data_type().ne(&rhs.data_type()) { + internal_err!( + "Only intervals with the same data type are comparable, lhs:{}, rhs:{}", + self.data_type(), + rhs.data_type() + ) + } else if !(self.lower.is_null() || rhs.upper.is_null()) + && self.lower >= rhs.upper + { + // Values in this interval are certainly greater than or equal to + // those in the given interval. + Ok(Self::CERTAINLY_TRUE) + } else if !(self.upper.is_null() || rhs.lower.is_null()) + && (self.upper < rhs.lower) + { + // Values in this interval are certainly less than those in the + // given interval. + Ok(Self::CERTAINLY_FALSE) + } else { + // All outcomes are possible. + Ok(Self::UNCERTAIN) + } + } + + /// Decide if this interval is certainly less than, possibly less than, or + /// can't be less than `other` by returning `[true, true]`, `[false, true]` + /// or `[false, false]` respectively. + /// + /// NOTE: This function only works with intervals of the same data type. + /// Attempting to compare intervals of different data types will lead + /// to an error. + pub(crate) fn lt>(&self, other: T) -> Result { + other.borrow().gt(self) + } + + /// Decide if this interval is certainly less than or equal to, possibly + /// less than or equal to, or can't be less than or equal to `other` by + /// returning `[true, true]`, `[false, true]` or `[false, false]` respectively. + /// + /// NOTE: This function only works with intervals of the same data type. + /// Attempting to compare intervals of different data types will lead + /// to an error. + pub(crate) fn lt_eq>(&self, other: T) -> Result { + other.borrow().gt_eq(self) + } + + /// Decide if this interval is certainly equal to, possibly equal to, or + /// can't be equal to `other` by returning `[true, true]`, `[false, true]` + /// or `[false, false]` respectively. + /// + /// NOTE: This function only works with intervals of the same data type. + /// Attempting to compare intervals of different data types will lead + /// to an error. + pub(crate) fn equal>(&self, other: T) -> Result { + let rhs = other.borrow(); + if get_result_type(&self.data_type(), &Operator::Eq, &rhs.data_type()).is_err() { + internal_err!( + "Interval data types must be compatible for equality checks, lhs:{}, rhs:{}", + self.data_type(), + rhs.data_type() + ) + } else if !self.lower.is_null() + && (self.lower == self.upper) + && (rhs.lower == rhs.upper) + && (self.lower == rhs.lower) + { + Ok(Self::CERTAINLY_TRUE) + } else if self.intersect(rhs)?.is_none() { + Ok(Self::CERTAINLY_FALSE) + } else { + Ok(Self::UNCERTAIN) + } + } + + /// Compute the logical conjunction of this (boolean) interval with the + /// given boolean interval. + pub(crate) fn and>(&self, other: T) -> Result { + let rhs = other.borrow(); + match (&self.lower, &self.upper, &rhs.lower, &rhs.upper) { + ( + &ScalarValue::Boolean(Some(self_lower)), + &ScalarValue::Boolean(Some(self_upper)), + &ScalarValue::Boolean(Some(other_lower)), + &ScalarValue::Boolean(Some(other_upper)), + ) => { + let lower = self_lower && other_lower; + let upper = self_upper && other_upper; + + Ok(Self { + lower: ScalarValue::Boolean(Some(lower)), + upper: ScalarValue::Boolean(Some(upper)), + }) + } + _ => internal_err!("Incompatible data types for logical conjunction"), + } + } + + /// Compute the logical negation of this (boolean) interval. + pub(crate) fn not(&self) -> Result { + if self.data_type().ne(&DataType::Boolean) { + internal_err!("Cannot apply logical negation to a non-boolean interval") + } else if self == &Self::CERTAINLY_TRUE { + Ok(Self::CERTAINLY_FALSE) + } else if self == &Self::CERTAINLY_FALSE { + Ok(Self::CERTAINLY_TRUE) + } else { + Ok(Self::UNCERTAIN) + } + } + + /// Compute the intersection of this interval with the given interval. + /// If the intersection is empty, return `None`. + /// + /// NOTE: This function only works with intervals of the same data type. + /// Attempting to compare intervals of different data types will lead + /// to an error. + pub fn intersect>(&self, other: T) -> Result> { + let rhs = other.borrow(); + if self.data_type().ne(&rhs.data_type()) { + return internal_err!( + "Only intervals with the same data type are intersectable, lhs:{}, rhs:{}", + self.data_type(), + rhs.data_type() + ); + }; + + // If it is evident that the result is an empty interval, short-circuit + // and directly return `None`. + if (!(self.lower.is_null() || rhs.upper.is_null()) && self.lower > rhs.upper) + || (!(self.upper.is_null() || rhs.lower.is_null()) && self.upper < rhs.lower) + { + return Ok(None); + } + + let lower = max_of_bounds(&self.lower, &rhs.lower); + let upper = min_of_bounds(&self.upper, &rhs.upper); + + // New lower and upper bounds must always construct a valid interval. + assert!( + (lower.is_null() || upper.is_null() || (lower <= upper)), + "The intersection of two intervals can not be an invalid interval" + ); + + Ok(Some(Self { lower, upper })) + } + + /// Decide if this interval certainly contains, possibly contains, or can't + /// contain a [`ScalarValue`] (`other`) by returning `[true, true]`, + /// `[false, true]` or `[false, false]` respectively. + /// + /// NOTE: This function only works with intervals of the same data type. + /// Attempting to compare intervals of different data types will lead + /// to an error. + pub fn contains_value>(&self, other: T) -> Result { + let rhs = other.borrow(); + if self.data_type().ne(&rhs.data_type()) { + return internal_err!( + "Data types must be compatible for containment checks, lhs:{}, rhs:{}", + self.data_type(), + rhs.data_type() + ); + } + + // We only check the upper bound for a `None` value because `None` + // values are less than `Some` values according to Rust. + Ok(&self.lower <= rhs && (self.upper.is_null() || rhs <= &self.upper)) + } + + /// Decide if this interval is a superset of, overlaps with, or + /// disjoint with `other` by returning `[true, true]`, `[false, true]` or + /// `[false, false]` respectively. + /// + /// NOTE: This function only works with intervals of the same data type. + /// Attempting to compare intervals of different data types will lead + /// to an error. + pub fn contains>(&self, other: T) -> Result { + let rhs = other.borrow(); + if self.data_type().ne(&rhs.data_type()) { + return internal_err!( + "Interval data types must match for containment checks, lhs:{}, rhs:{}", + self.data_type(), + rhs.data_type() + ); + }; + + match self.intersect(rhs)? { + Some(intersection) => { + if &intersection == rhs { + Ok(Self::CERTAINLY_TRUE) + } else { + Ok(Self::UNCERTAIN) + } + } + None => Ok(Self::CERTAINLY_FALSE), + } + } + + /// Add the given interval (`other`) to this interval. Say we have intervals + /// `[a1, b1]` and `[a2, b2]`, then their sum is `[a1 + a2, b1 + b2]`. Note + /// that this represents all possible values the sum can take if one can + /// choose single values arbitrarily from each of the operands. + pub fn add>(&self, other: T) -> Result { + let rhs = other.borrow(); + let dt = get_result_type(&self.data_type(), &Operator::Plus, &rhs.data_type())?; + + Ok(Self::new( + add_bounds::(&dt, &self.lower, &rhs.lower), + add_bounds::(&dt, &self.upper, &rhs.upper), + )) + } + + /// Subtract the given interval (`other`) from this interval. Say we have + /// intervals `[a1, b1]` and `[a2, b2]`, then their difference is + /// `[a1 - b2, b1 - a2]`. Note that this represents all possible values the + /// difference can take if one can choose single values arbitrarily from + /// each of the operands. + pub fn sub>(&self, other: T) -> Result { + let rhs = other.borrow(); + let dt = get_result_type(&self.data_type(), &Operator::Minus, &rhs.data_type())?; + + Ok(Self::new( + sub_bounds::(&dt, &self.lower, &rhs.upper), + sub_bounds::(&dt, &self.upper, &rhs.lower), + )) + } + + /// Multiply the given interval (`other`) with this interval. Say we have + /// intervals `[a1, b1]` and `[a2, b2]`, then their product is `[min(a1 * a2, + /// a1 * b2, b1 * a2, b1 * b2), max(a1 * a2, a1 * b2, b1 * a2, b1 * b2)]`. + /// Note that this represents all possible values the product can take if + /// one can choose single values arbitrarily from each of the operands. + /// + /// NOTE: This function only works with intervals of the same data type. + /// Attempting to compare intervals of different data types will lead + /// to an error. + pub fn mul>(&self, other: T) -> Result { + let rhs = other.borrow(); + let dt = if self.data_type().eq(&rhs.data_type()) { + self.data_type() + } else { + return internal_err!( + "Intervals must have the same data type for multiplication, lhs:{}, rhs:{}", + self.data_type(), + rhs.data_type() + ); + }; + + let zero = ScalarValue::new_zero(&dt)?; + + let result = match ( + self.contains_value(&zero)?, + rhs.contains_value(&zero)?, + dt.is_unsigned_integer(), + ) { + (true, true, false) => mul_helper_multi_zero_inclusive(&dt, self, rhs), + (true, false, false) => { + mul_helper_single_zero_inclusive(&dt, self, rhs, zero) + } + (false, true, false) => { + mul_helper_single_zero_inclusive(&dt, rhs, self, zero) + } + _ => mul_helper_zero_exclusive(&dt, self, rhs, zero), + }; + Ok(result) + } + + /// Divide this interval by the given interval (`other`). Say we have intervals + /// `[a1, b1]` and `[a2, b2]`, then their division is `[a1, b1] * [1 / b2, 1 / a2]` + /// if `0 ∉ [a2, b2]` and `[NEG_INF, INF]` otherwise. Note that this represents + /// all possible values the quotient can take if one can choose single values + /// arbitrarily from each of the operands. + /// + /// NOTE: This function only works with intervals of the same data type. + /// Attempting to compare intervals of different data types will lead + /// to an error. + /// + /// **TODO**: Once interval sets are supported, cases where the divisor contains + /// zero should result in an interval set, not the universal set. + pub fn div>(&self, other: T) -> Result { + let rhs = other.borrow(); + let dt = if self.data_type().eq(&rhs.data_type()) { + self.data_type() + } else { + return internal_err!( + "Intervals must have the same data type for division, lhs:{}, rhs:{}", + self.data_type(), + rhs.data_type() + ); + }; + + let zero = ScalarValue::new_zero(&dt)?; + // We want 0 to be approachable from both negative and positive sides. + let zero_point = match &dt { + DataType::Float32 | DataType::Float64 => Self::new(zero.clone(), zero), + _ => Self::new(prev_value(zero.clone()), next_value(zero.clone())), + }; + + // Exit early with an unbounded interval if zero is strictly inside the + // right hand side: + if rhs.contains(&zero_point)? == Self::CERTAINLY_TRUE && !dt.is_unsigned_integer() + { + Self::make_unbounded(&dt) + } + // At this point, we know that only one endpoint of the right hand side + // can be zero. + else if self.contains(&zero_point)? == Self::CERTAINLY_TRUE + && !dt.is_unsigned_integer() + { + Ok(div_helper_lhs_zero_inclusive(&dt, self, rhs, &zero_point)) + } else { + Ok(div_helper_zero_exclusive(&dt, self, rhs, &zero_point)) + } + } + + /// Returns the cardinality of this interval, which is the number of all + /// distinct points inside it. This function returns `None` if: + /// - The interval is unbounded from either side, or + /// - Cardinality calculations for the datatype in question is not + /// implemented yet, or + /// - An overflow occurs during the calculation: This case can only arise + /// when the calculated cardinality does not fit in an `u64`. + pub fn cardinality(&self) -> Option { + let data_type = self.data_type(); + if data_type.is_integer() { + self.upper.distance(&self.lower).map(|diff| diff as u64) + } else if data_type.is_floating() { + // Negative numbers are sorted in the reverse order. To + // always have a positive difference after the subtraction, + // we perform following transformation: + match (&self.lower, &self.upper) { + // Exploit IEEE 754 ordering properties to calculate the correct + // cardinality in all cases (including subnormals). + ( + ScalarValue::Float32(Some(lower)), + ScalarValue::Float32(Some(upper)), + ) => { + let lower_bits = map_floating_point_order!(lower.to_bits(), u32); + let upper_bits = map_floating_point_order!(upper.to_bits(), u32); + Some((upper_bits - lower_bits) as u64) + } + ( + ScalarValue::Float64(Some(lower)), + ScalarValue::Float64(Some(upper)), + ) => { + let lower_bits = map_floating_point_order!(lower.to_bits(), u64); + let upper_bits = map_floating_point_order!(upper.to_bits(), u64); + let count = upper_bits - lower_bits; + (count != u64::MAX).then_some(count) + } + _ => None, + } + } else { + // Cardinality calculations are not implemented for this data type yet: + None + } + .map(|result| result + 1) + } +} + +impl Display for Interval { + fn fmt(&self, f: &mut Formatter<'_>) -> fmt::Result { + write!(f, "[{}, {}]", self.lower, self.upper) + } +} + +/// Applies the given binary operator the `lhs` and `rhs` arguments. +pub fn apply_operator(op: &Operator, lhs: &Interval, rhs: &Interval) -> Result { + match *op { + Operator::Eq => lhs.equal(rhs), + Operator::NotEq => lhs.equal(rhs)?.not(), + Operator::Gt => lhs.gt(rhs), + Operator::GtEq => lhs.gt_eq(rhs), + Operator::Lt => lhs.lt(rhs), + Operator::LtEq => lhs.lt_eq(rhs), + Operator::And => lhs.and(rhs), + Operator::Plus => lhs.add(rhs), + Operator::Minus => lhs.sub(rhs), + Operator::Multiply => lhs.mul(rhs), + Operator::Divide => lhs.div(rhs), + _ => internal_err!("Interval arithmetic does not support the operator {op}"), + } +} + +/// Helper function used for adding the end-point values of intervals. +/// +/// **Caution:** This function contains multiple calls to `unwrap()`, and may +/// return non-standardized interval bounds. Therefore, it should be used +/// with caution. Currently, it is used in contexts where the `DataType` +/// (`dt`) is validated prior to calling this function, and the following +/// interval creation is standardized with `Interval::new`. +fn add_bounds( + dt: &DataType, + lhs: &ScalarValue, + rhs: &ScalarValue, +) -> ScalarValue { + if lhs.is_null() || rhs.is_null() { + return ScalarValue::try_from(dt).unwrap(); + } + + match dt { + DataType::Float64 | DataType::Float32 => { + alter_fp_rounding_mode::(lhs, rhs, |lhs, rhs| lhs.add_checked(rhs)) + } + _ => lhs.add_checked(rhs), + } + .unwrap_or_else(|_| handle_overflow::(dt, Operator::Plus, lhs, rhs)) +} + +/// Helper function used for subtracting the end-point values of intervals. +/// +/// **Caution:** This function contains multiple calls to `unwrap()`, and may +/// return non-standardized interval bounds. Therefore, it should be used +/// with caution. Currently, it is used in contexts where the `DataType` +/// (`dt`) is validated prior to calling this function, and the following +/// interval creation is standardized with `Interval::new`. +fn sub_bounds( + dt: &DataType, + lhs: &ScalarValue, + rhs: &ScalarValue, +) -> ScalarValue { + if lhs.is_null() || rhs.is_null() { + return ScalarValue::try_from(dt).unwrap(); + } + + match dt { + DataType::Float64 | DataType::Float32 => { + alter_fp_rounding_mode::(lhs, rhs, |lhs, rhs| lhs.sub_checked(rhs)) + } + _ => lhs.sub_checked(rhs), + } + .unwrap_or_else(|_| handle_overflow::(dt, Operator::Minus, lhs, rhs)) +} + +/// Helper function used for multiplying the end-point values of intervals. +/// +/// **Caution:** This function contains multiple calls to `unwrap()`, and may +/// return non-standardized interval bounds. Therefore, it should be used +/// with caution. Currently, it is used in contexts where the `DataType` +/// (`dt`) is validated prior to calling this function, and the following +/// interval creation is standardized with `Interval::new`. +fn mul_bounds( + dt: &DataType, + lhs: &ScalarValue, + rhs: &ScalarValue, +) -> ScalarValue { + if lhs.is_null() || rhs.is_null() { + return ScalarValue::try_from(dt).unwrap(); + } + + match dt { + DataType::Float64 | DataType::Float32 => { + alter_fp_rounding_mode::(lhs, rhs, |lhs, rhs| lhs.mul_checked(rhs)) + } + _ => lhs.mul_checked(rhs), + } + .unwrap_or_else(|_| handle_overflow::(dt, Operator::Multiply, lhs, rhs)) +} + +/// Helper function used for dividing the end-point values of intervals. +/// +/// **Caution:** This function contains multiple calls to `unwrap()`, and may +/// return non-standardized interval bounds. Therefore, it should be used +/// with caution. Currently, it is used in contexts where the `DataType` +/// (`dt`) is validated prior to calling this function, and the following +/// interval creation is standardized with `Interval::new`. +fn div_bounds( + dt: &DataType, + lhs: &ScalarValue, + rhs: &ScalarValue, +) -> ScalarValue { + let zero = ScalarValue::new_zero(dt).unwrap(); + + if (lhs.is_null() || rhs.eq(&zero)) || (dt.is_unsigned_integer() && rhs.is_null()) { + return ScalarValue::try_from(dt).unwrap(); + } else if rhs.is_null() { + return zero; + } + + match dt { + DataType::Float64 | DataType::Float32 => { + alter_fp_rounding_mode::(lhs, rhs, |lhs, rhs| lhs.div(rhs)) + } + _ => lhs.div(rhs), + } + .unwrap_or_else(|_| handle_overflow::(dt, Operator::Divide, lhs, rhs)) +} + +/// This function handles cases where an operation results in an overflow. Such +/// results are converted to an *unbounded endpoint* if: +/// - We are calculating an upper bound and we have a positive overflow. +/// - We are calculating a lower bound and we have a negative overflow. +/// Otherwise; the function sets the endpoint as: +/// - The minimum representable number with the given datatype (`dt`) if +/// we are calculating an upper bound and we have a negative overflow. +/// - The maximum representable number with the given datatype (`dt`) if +/// we are calculating a lower bound and we have a positive overflow. +/// +/// **Caution:** This function contains multiple calls to `unwrap()`, and may +/// return non-standardized interval bounds. Therefore, it should be used +/// with caution. Currently, it is used in contexts where the `DataType` +/// (`dt`) is validated prior to calling this function, `op` is supported by +/// interval library, and the following interval creation is standardized with +/// `Interval::new`. +fn handle_overflow( + dt: &DataType, + op: Operator, + lhs: &ScalarValue, + rhs: &ScalarValue, +) -> ScalarValue { + let zero = ScalarValue::new_zero(dt).unwrap(); + let positive_sign = match op { + Operator::Multiply | Operator::Divide => { + lhs.lt(&zero) && rhs.lt(&zero) || lhs.gt(&zero) && rhs.gt(&zero) + } + Operator::Plus => lhs.ge(&zero), + Operator::Minus => lhs.ge(rhs), + _ => { + unreachable!() + } + }; + match (UPPER, positive_sign) { + (true, true) | (false, false) => ScalarValue::try_from(dt).unwrap(), + (true, false) => { + get_extreme_value!(MIN, dt) + } + (false, true) => { + get_extreme_value!(MAX, dt) + } + } +} + +// This function should remain private since it may corrupt the an interval if +// used without caution. +fn next_value(value: ScalarValue) -> ScalarValue { + use ScalarValue::*; + value_transition!(MAX, true, value) +} + +// This function should remain private since it may corrupt the an interval if +// used without caution. +fn prev_value(value: ScalarValue) -> ScalarValue { + use ScalarValue::*; + value_transition!(MIN, false, value) +} + +trait OneTrait: Sized + std::ops::Add + std::ops::Sub { + fn one() -> Self; +} +macro_rules! impl_OneTrait{ + ($($m:ty),*) => {$( impl OneTrait for $m { fn one() -> Self { 1 as $m } })*} +} +impl_OneTrait! {u8, u16, u32, u64, i8, i16, i32, i64, i128} + +/// This function either increments or decrements its argument, depending on +/// the `INC` value (where a `true` value corresponds to the increment). +fn increment_decrement( + mut value: T, +) -> T { + if INC { + value.add_assign(T::one()); + } else { + value.sub_assign(T::one()); + } + value +} + +/// This function returns the next/previous value depending on the `INC` value. +/// If `true`, it returns the next value; otherwise it returns the previous value. +fn next_value_helper(value: ScalarValue) -> ScalarValue { + use ScalarValue::*; + match value { + // f32/f64::NEG_INF/INF and f32/f64::NaN values should not emerge at this point. + Float32(Some(val)) => { + assert!(val.is_finite(), "Non-standardized floating point usage"); + Float32(Some(if INC { next_up(val) } else { next_down(val) })) + } + Float64(Some(val)) => { + assert!(val.is_finite(), "Non-standardized floating point usage"); + Float64(Some(if INC { next_up(val) } else { next_down(val) })) + } + Int8(Some(val)) => Int8(Some(increment_decrement::(val))), + Int16(Some(val)) => Int16(Some(increment_decrement::(val))), + Int32(Some(val)) => Int32(Some(increment_decrement::(val))), + Int64(Some(val)) => Int64(Some(increment_decrement::(val))), + UInt8(Some(val)) => UInt8(Some(increment_decrement::(val))), + UInt16(Some(val)) => UInt16(Some(increment_decrement::(val))), + UInt32(Some(val)) => UInt32(Some(increment_decrement::(val))), + UInt64(Some(val)) => UInt64(Some(increment_decrement::(val))), + DurationSecond(Some(val)) => { + DurationSecond(Some(increment_decrement::(val))) + } + DurationMillisecond(Some(val)) => { + DurationMillisecond(Some(increment_decrement::(val))) + } + DurationMicrosecond(Some(val)) => { + DurationMicrosecond(Some(increment_decrement::(val))) + } + DurationNanosecond(Some(val)) => { + DurationNanosecond(Some(increment_decrement::(val))) + } + TimestampSecond(Some(val), tz) => { + TimestampSecond(Some(increment_decrement::(val)), tz) + } + TimestampMillisecond(Some(val), tz) => { + TimestampMillisecond(Some(increment_decrement::(val)), tz) + } + TimestampMicrosecond(Some(val), tz) => { + TimestampMicrosecond(Some(increment_decrement::(val)), tz) + } + TimestampNanosecond(Some(val), tz) => { + TimestampNanosecond(Some(increment_decrement::(val)), tz) + } + IntervalYearMonth(Some(val)) => { + IntervalYearMonth(Some(increment_decrement::(val))) + } + IntervalDayTime(Some(val)) => { + IntervalDayTime(Some(increment_decrement::(val))) + } + IntervalMonthDayNano(Some(val)) => { + IntervalMonthDayNano(Some(increment_decrement::(val))) + } + _ => value, // Unbounded values return without change. + } +} + +/// Returns the greater of the given interval bounds. Assumes that a `NULL` +/// value represents `NEG_INF`. +fn max_of_bounds(first: &ScalarValue, second: &ScalarValue) -> ScalarValue { + if !first.is_null() && (second.is_null() || first >= second) { + first.clone() + } else { + second.clone() + } +} + +/// Returns the lesser of the given interval bounds. Assumes that a `NULL` +/// value represents `INF`. +fn min_of_bounds(first: &ScalarValue, second: &ScalarValue) -> ScalarValue { + if !first.is_null() && (second.is_null() || first <= second) { + first.clone() + } else { + second.clone() + } +} + +/// This function updates the given intervals by enforcing (i.e. propagating) +/// the inequality `left > right` (or the `left >= right` inequality, if `strict` +/// is `true`). +/// +/// Returns a `Result` wrapping an `Option` containing the tuple of resulting +/// intervals. If the comparison is infeasible, returns `None`. +/// +/// Example usage: +/// ``` +/// use datafusion_common::DataFusionError; +/// use datafusion_expr::interval_arithmetic::{satisfy_greater, Interval}; +/// +/// let left = Interval::make(Some(-1000.0_f32), Some(1000.0_f32))?; +/// let right = Interval::make(Some(500.0_f32), Some(2000.0_f32))?; +/// let strict = false; +/// assert_eq!( +/// satisfy_greater(&left, &right, strict)?, +/// Some(( +/// Interval::make(Some(500.0_f32), Some(1000.0_f32))?, +/// Interval::make(Some(500.0_f32), Some(1000.0_f32))? +/// )) +/// ); +/// Ok::<(), DataFusionError>(()) +/// ``` +/// +/// NOTE: This function only works with intervals of the same data type. +/// Attempting to compare intervals of different data types will lead +/// to an error. +pub fn satisfy_greater( + left: &Interval, + right: &Interval, + strict: bool, +) -> Result> { + if left.data_type().ne(&right.data_type()) { + return internal_err!( + "Intervals must have the same data type, lhs:{}, rhs:{}", + left.data_type(), + right.data_type() + ); + } + + if !left.upper.is_null() && left.upper <= right.lower { + if !strict && left.upper == right.lower { + // Singleton intervals: + return Ok(Some(( + Interval::new(left.upper.clone(), left.upper.clone()), + Interval::new(left.upper.clone(), left.upper.clone()), + ))); + } else { + // Left-hand side: <--======----0------------> + // Right-hand side: <------------0--======----> + // No intersection, infeasible to propagate: + return Ok(None); + } + } + + // Only the lower bound of left hand side and the upper bound of the right + // hand side can change after propagating the greater-than operation. + let new_left_lower = if left.lower.is_null() || left.lower <= right.lower { + if strict { + next_value(right.lower.clone()) + } else { + right.lower.clone() + } + } else { + left.lower.clone() + }; + // Below code is asymmetric relative to the above if statement, because + // `None` compares less than `Some` in Rust. + let new_right_upper = if right.upper.is_null() + || (!left.upper.is_null() && left.upper <= right.upper) + { + if strict { + prev_value(left.upper.clone()) + } else { + left.upper.clone() + } + } else { + right.upper.clone() + }; + + Ok(Some(( + Interval::new(new_left_lower, left.upper.clone()), + Interval::new(right.lower.clone(), new_right_upper), + ))) +} + +/// Multiplies two intervals that both contain zero. +/// +/// This function takes in two intervals (`lhs` and `rhs`) as arguments and +/// returns their product (whose data type is known to be `dt`). It is +/// specifically designed to handle intervals that contain zero within their +/// ranges. Returns an error if the multiplication of bounds fails. +/// +/// ```text +/// Left-hand side: <-------=====0=====-------> +/// Right-hand side: <-------=====0=====-------> +/// ``` +/// +/// **Caution:** This function contains multiple calls to `unwrap()`. Therefore, +/// it should be used with caution. Currently, it is used in contexts where the +/// `DataType` (`dt`) is validated prior to calling this function. +fn mul_helper_multi_zero_inclusive( + dt: &DataType, + lhs: &Interval, + rhs: &Interval, +) -> Interval { + if lhs.lower.is_null() + || lhs.upper.is_null() + || rhs.lower.is_null() + || rhs.upper.is_null() + { + return Interval::make_unbounded(dt).unwrap(); + } + // Since unbounded cases are handled above, we can safely + // use the utility functions here to eliminate code duplication. + let lower = min_of_bounds( + &mul_bounds::(dt, &lhs.lower, &rhs.upper), + &mul_bounds::(dt, &rhs.lower, &lhs.upper), + ); + let upper = max_of_bounds( + &mul_bounds::(dt, &lhs.upper, &rhs.upper), + &mul_bounds::(dt, &lhs.lower, &rhs.lower), + ); + // There is no possibility to create an invalid interval. + Interval::new(lower, upper) +} + +/// Multiplies two intervals when only left-hand side interval contains zero. +/// +/// This function takes in two intervals (`lhs` and `rhs`) as arguments and +/// returns their product (whose data type is known to be `dt`). This function +/// serves as a subroutine that handles the specific case when only `lhs` contains +/// zero within its range. The interval not containing zero, i.e. rhs, can lie +/// on either side of zero. Returns an error if the multiplication of bounds fails. +/// +/// ``` text +/// Left-hand side: <-------=====0=====-------> +/// Right-hand side: <--======----0------------> +/// +/// or +/// +/// Left-hand side: <-------=====0=====-------> +/// Right-hand side: <------------0--======----> +/// ``` +/// +/// **Caution:** This function contains multiple calls to `unwrap()`. Therefore, +/// it should be used with caution. Currently, it is used in contexts where the +/// `DataType` (`dt`) is validated prior to calling this function. +fn mul_helper_single_zero_inclusive( + dt: &DataType, + lhs: &Interval, + rhs: &Interval, + zero: ScalarValue, +) -> Interval { + // With the following interval bounds, there is no possibility to create an invalid interval. + if rhs.upper <= zero && !rhs.upper.is_null() { + // <-------=====0=====-------> + // <--======----0------------> + let lower = mul_bounds::(dt, &lhs.upper, &rhs.lower); + let upper = mul_bounds::(dt, &lhs.lower, &rhs.lower); + Interval::new(lower, upper) + } else { + // <-------=====0=====-------> + // <------------0--======----> + let lower = mul_bounds::(dt, &lhs.lower, &rhs.upper); + let upper = mul_bounds::(dt, &lhs.upper, &rhs.upper); + Interval::new(lower, upper) + } +} + +/// Multiplies two intervals when neither of them contains zero. +/// +/// This function takes in two intervals (`lhs` and `rhs`) as arguments and +/// returns their product (whose data type is known to be `dt`). It is +/// specifically designed to handle intervals that do not contain zero within +/// their ranges. Returns an error if the multiplication of bounds fails. +/// +/// ``` text +/// Left-hand side: <--======----0------------> +/// Right-hand side: <--======----0------------> +/// +/// or +/// +/// Left-hand side: <--======----0------------> +/// Right-hand side: <------------0--======----> +/// +/// or +/// +/// Left-hand side: <------------0--======----> +/// Right-hand side: <--======----0------------> +/// +/// or +/// +/// Left-hand side: <------------0--======----> +/// Right-hand side: <------------0--======----> +/// ``` +/// +/// **Caution:** This function contains multiple calls to `unwrap()`. Therefore, +/// it should be used with caution. Currently, it is used in contexts where the +/// `DataType` (`dt`) is validated prior to calling this function. +fn mul_helper_zero_exclusive( + dt: &DataType, + lhs: &Interval, + rhs: &Interval, + zero: ScalarValue, +) -> Interval { + let (lower, upper) = match ( + lhs.upper <= zero && !lhs.upper.is_null(), + rhs.upper <= zero && !rhs.upper.is_null(), + ) { + // With the following interval bounds, there is no possibility to create an invalid interval. + (true, true) => ( + // <--======----0------------> + // <--======----0------------> + mul_bounds::(dt, &lhs.upper, &rhs.upper), + mul_bounds::(dt, &lhs.lower, &rhs.lower), + ), + (true, false) => ( + // <--======----0------------> + // <------------0--======----> + mul_bounds::(dt, &lhs.lower, &rhs.upper), + mul_bounds::(dt, &lhs.upper, &rhs.lower), + ), + (false, true) => ( + // <------------0--======----> + // <--======----0------------> + mul_bounds::(dt, &rhs.lower, &lhs.upper), + mul_bounds::(dt, &rhs.upper, &lhs.lower), + ), + (false, false) => ( + // <------------0--======----> + // <------------0--======----> + mul_bounds::(dt, &lhs.lower, &rhs.lower), + mul_bounds::(dt, &lhs.upper, &rhs.upper), + ), + }; + Interval::new(lower, upper) +} + +/// Divides the left-hand side interval by the right-hand side interval when +/// the former contains zero. +/// +/// This function takes in two intervals (`lhs` and `rhs`) as arguments and +/// returns their quotient (whose data type is known to be `dt`). This function +/// serves as a subroutine that handles the specific case when only `lhs` contains +/// zero within its range. Returns an error if the division of bounds fails. +/// +/// ``` text +/// Left-hand side: <-------=====0=====-------> +/// Right-hand side: <--======----0------------> +/// +/// or +/// +/// Left-hand side: <-------=====0=====-------> +/// Right-hand side: <------------0--======----> +/// ``` +/// +/// **Caution:** This function contains multiple calls to `unwrap()`. Therefore, +/// it should be used with caution. Currently, it is used in contexts where the +/// `DataType` (`dt`) is validated prior to calling this function. +fn div_helper_lhs_zero_inclusive( + dt: &DataType, + lhs: &Interval, + rhs: &Interval, + zero_point: &Interval, +) -> Interval { + // With the following interval bounds, there is no possibility to create an invalid interval. + if rhs.upper <= zero_point.lower && !rhs.upper.is_null() { + // <-------=====0=====-------> + // <--======----0------------> + let lower = div_bounds::(dt, &lhs.upper, &rhs.upper); + let upper = div_bounds::(dt, &lhs.lower, &rhs.upper); + Interval::new(lower, upper) + } else { + // <-------=====0=====-------> + // <------------0--======----> + let lower = div_bounds::(dt, &lhs.lower, &rhs.lower); + let upper = div_bounds::(dt, &lhs.upper, &rhs.lower); + Interval::new(lower, upper) + } +} + +/// Divides the left-hand side interval by the right-hand side interval when +/// neither interval contains zero. +/// +/// This function takes in two intervals (`lhs` and `rhs`) as arguments and +/// returns their quotient (whose data type is known to be `dt`). It is +/// specifically designed to handle intervals that do not contain zero within +/// their ranges. Returns an error if the division of bounds fails. +/// +/// ``` text +/// Left-hand side: <--======----0------------> +/// Right-hand side: <--======----0------------> +/// +/// or +/// +/// Left-hand side: <--======----0------------> +/// Right-hand side: <------------0--======----> +/// +/// or +/// +/// Left-hand side: <------------0--======----> +/// Right-hand side: <--======----0------------> +/// +/// or +/// +/// Left-hand side: <------------0--======----> +/// Right-hand side: <------------0--======----> +/// ``` +/// +/// **Caution:** This function contains multiple calls to `unwrap()`. Therefore, +/// it should be used with caution. Currently, it is used in contexts where the +/// `DataType` (`dt`) is validated prior to calling this function. +fn div_helper_zero_exclusive( + dt: &DataType, + lhs: &Interval, + rhs: &Interval, + zero_point: &Interval, +) -> Interval { + let (lower, upper) = match ( + lhs.upper <= zero_point.lower && !lhs.upper.is_null(), + rhs.upper <= zero_point.lower && !rhs.upper.is_null(), + ) { + // With the following interval bounds, there is no possibility to create an invalid interval. + (true, true) => ( + // <--======----0------------> + // <--======----0------------> + div_bounds::(dt, &lhs.upper, &rhs.lower), + div_bounds::(dt, &lhs.lower, &rhs.upper), + ), + (true, false) => ( + // <--======----0------------> + // <------------0--======----> + div_bounds::(dt, &lhs.lower, &rhs.lower), + div_bounds::(dt, &lhs.upper, &rhs.upper), + ), + (false, true) => ( + // <------------0--======----> + // <--======----0------------> + div_bounds::(dt, &lhs.upper, &rhs.upper), + div_bounds::(dt, &lhs.lower, &rhs.lower), + ), + (false, false) => ( + // <------------0--======----> + // <------------0--======----> + div_bounds::(dt, &lhs.lower, &rhs.upper), + div_bounds::(dt, &lhs.upper, &rhs.lower), + ), + }; + Interval::new(lower, upper) +} + +/// This function computes the selectivity of an operation by computing the +/// cardinality ratio of the given input/output intervals. If this can not be +/// calculated for some reason, it returns `1.0` meaning fully selective (no +/// filtering). +pub fn cardinality_ratio(initial_interval: &Interval, final_interval: &Interval) -> f64 { + match (final_interval.cardinality(), initial_interval.cardinality()) { + (Some(final_interval), Some(initial_interval)) => { + (final_interval as f64) / (initial_interval as f64) + } + _ => 1.0, + } +} + +/// Cast scalar value to the given data type using an arrow kernel. +fn cast_scalar_value( + value: &ScalarValue, + data_type: &DataType, + cast_options: &CastOptions, +) -> Result { + let cast_array = cast_with_options(&value.to_array()?, data_type, cast_options)?; + ScalarValue::try_from_array(&cast_array, 0) +} + +/// An [Interval] that also tracks null status using a boolean interval. +/// +/// This represents values that may be in a particular range or be null. +/// +/// # Examples +/// +/// ``` +/// use arrow::datatypes::DataType; +/// use datafusion_common::ScalarValue; +/// use datafusion_expr::interval_arithmetic::Interval; +/// use datafusion_expr::interval_arithmetic::NullableInterval; +/// +/// // [1, 2) U {NULL} +/// let maybe_null = NullableInterval::MaybeNull { +/// values: Interval::try_new( +/// ScalarValue::Int32(Some(1)), +/// ScalarValue::Int32(Some(2)), +/// ).unwrap(), +/// }; +/// +/// // (0, ∞) +/// let not_null = NullableInterval::NotNull { +/// values: Interval::try_new( +/// ScalarValue::Int32(Some(0)), +/// ScalarValue::Int32(None), +/// ).unwrap(), +/// }; +/// +/// // {NULL} +/// let null_interval = NullableInterval::Null { datatype: DataType::Int32 }; +/// +/// // {4} +/// let single_value = NullableInterval::from(ScalarValue::Int32(Some(4))); +/// ``` +#[derive(Debug, Clone, PartialEq, Eq)] +pub enum NullableInterval { + /// The value is always null. This is typed so it can be used in physical + /// expressions, which don't do type coercion. + Null { datatype: DataType }, + /// The value may or may not be null. If it is non-null, its is within the + /// specified range. + MaybeNull { values: Interval }, + /// The value is definitely not null, and is within the specified range. + NotNull { values: Interval }, +} + +impl Display for NullableInterval { + fn fmt(&self, f: &mut Formatter<'_>) -> fmt::Result { + match self { + Self::Null { .. } => write!(f, "NullableInterval: {{NULL}}"), + Self::MaybeNull { values } => { + write!(f, "NullableInterval: {} U {{NULL}}", values) + } + Self::NotNull { values } => write!(f, "NullableInterval: {}", values), + } + } +} + +impl From for NullableInterval { + /// Create an interval that represents a single value. + fn from(value: ScalarValue) -> Self { + if value.is_null() { + Self::Null { + datatype: value.data_type(), + } + } else { + Self::NotNull { + values: Interval { + lower: value.clone(), + upper: value, + }, + } + } + } +} + +impl NullableInterval { + /// Get the values interval, or None if this interval is definitely null. + pub fn values(&self) -> Option<&Interval> { + match self { + Self::Null { .. } => None, + Self::MaybeNull { values } | Self::NotNull { values } => Some(values), + } + } + + /// Get the data type + pub fn data_type(&self) -> DataType { + match self { + Self::Null { datatype } => datatype.clone(), + Self::MaybeNull { values } | Self::NotNull { values } => values.data_type(), + } + } + + /// Return true if the value is definitely true (and not null). + pub fn is_certainly_true(&self) -> bool { + match self { + Self::Null { .. } | Self::MaybeNull { .. } => false, + Self::NotNull { values } => values == &Interval::CERTAINLY_TRUE, + } + } + + /// Return true if the value is definitely false (and not null). + pub fn is_certainly_false(&self) -> bool { + match self { + Self::Null { .. } => false, + Self::MaybeNull { .. } => false, + Self::NotNull { values } => values == &Interval::CERTAINLY_FALSE, + } + } + + /// Perform logical negation on a boolean nullable interval. + fn not(&self) -> Result { + match self { + Self::Null { datatype } => Ok(Self::Null { + datatype: datatype.clone(), + }), + Self::MaybeNull { values } => Ok(Self::MaybeNull { + values: values.not()?, + }), + Self::NotNull { values } => Ok(Self::NotNull { + values: values.not()?, + }), + } + } + + /// Apply the given operator to this interval and the given interval. + /// + /// # Examples + /// + /// ``` + /// use datafusion_common::ScalarValue; + /// use datafusion_expr::Operator; + /// use datafusion_expr::interval_arithmetic::Interval; + /// use datafusion_expr::interval_arithmetic::NullableInterval; + /// + /// // 4 > 3 -> true + /// let lhs = NullableInterval::from(ScalarValue::Int32(Some(4))); + /// let rhs = NullableInterval::from(ScalarValue::Int32(Some(3))); + /// let result = lhs.apply_operator(&Operator::Gt, &rhs).unwrap(); + /// assert_eq!(result, NullableInterval::from(ScalarValue::Boolean(Some(true)))); + /// + /// // [1, 3) > NULL -> NULL + /// let lhs = NullableInterval::NotNull { + /// values: Interval::try_new( + /// ScalarValue::Int32(Some(1)), + /// ScalarValue::Int32(Some(3)), + /// ).unwrap(), + /// }; + /// let rhs = NullableInterval::from(ScalarValue::Int32(None)); + /// let result = lhs.apply_operator(&Operator::Gt, &rhs).unwrap(); + /// assert_eq!(result.single_value(), Some(ScalarValue::Boolean(None))); + /// + /// // [1, 3] > [2, 4] -> [false, true] + /// let lhs = NullableInterval::NotNull { + /// values: Interval::try_new( + /// ScalarValue::Int32(Some(1)), + /// ScalarValue::Int32(Some(3)), + /// ).unwrap(), + /// }; + /// let rhs = NullableInterval::NotNull { + /// values: Interval::try_new( + /// ScalarValue::Int32(Some(2)), + /// ScalarValue::Int32(Some(4)), + /// ).unwrap(), + /// }; + /// let result = lhs.apply_operator(&Operator::Gt, &rhs).unwrap(); + /// // Both inputs are valid (non-null), so result must be non-null + /// assert_eq!(result, NullableInterval::NotNull { + /// // Uncertain whether inequality is true or false + /// values: Interval::UNCERTAIN, + /// }); + /// ``` + pub fn apply_operator(&self, op: &Operator, rhs: &Self) -> Result { + match op { + Operator::IsDistinctFrom => { + let values = match (self, rhs) { + // NULL is distinct from NULL -> False + (Self::Null { .. }, Self::Null { .. }) => Interval::CERTAINLY_FALSE, + // x is distinct from y -> x != y, + // if at least one of them is never null. + (Self::NotNull { .. }, _) | (_, Self::NotNull { .. }) => { + let lhs_values = self.values(); + let rhs_values = rhs.values(); + match (lhs_values, rhs_values) { + (Some(lhs_values), Some(rhs_values)) => { + lhs_values.equal(rhs_values)?.not()? + } + (Some(_), None) | (None, Some(_)) => Interval::CERTAINLY_TRUE, + (None, None) => unreachable!("Null case handled above"), + } + } + _ => Interval::UNCERTAIN, + }; + // IsDistinctFrom never returns null. + Ok(Self::NotNull { values }) + } + Operator::IsNotDistinctFrom => self + .apply_operator(&Operator::IsDistinctFrom, rhs) + .map(|i| i.not())?, + _ => { + if let (Some(left_values), Some(right_values)) = + (self.values(), rhs.values()) + { + let values = apply_operator(op, left_values, right_values)?; + match (self, rhs) { + (Self::NotNull { .. }, Self::NotNull { .. }) => { + Ok(Self::NotNull { values }) + } + _ => Ok(Self::MaybeNull { values }), + } + } else if op.is_comparison_operator() { + Ok(Self::Null { + datatype: DataType::Boolean, + }) + } else { + Ok(Self::Null { + datatype: self.data_type(), + }) + } + } + } + } + + /// Decide if this interval is a superset of, overlaps with, or + /// disjoint with `other` by returning `[true, true]`, `[false, true]` or + /// `[false, false]` respectively. + /// + /// NOTE: This function only works with intervals of the same data type. + /// Attempting to compare intervals of different data types will lead + /// to an error. + pub fn contains>(&self, other: T) -> Result { + let rhs = other.borrow(); + if let (Some(left_values), Some(right_values)) = (self.values(), rhs.values()) { + left_values + .contains(right_values) + .map(|values| match (self, rhs) { + (Self::NotNull { .. }, Self::NotNull { .. }) => { + Self::NotNull { values } + } + _ => Self::MaybeNull { values }, + }) + } else { + Ok(Self::Null { + datatype: DataType::Boolean, + }) + } + } + + /// If the interval has collapsed to a single value, return that value. + /// Otherwise, returns `None`. + /// + /// # Examples + /// + /// ``` + /// use datafusion_common::ScalarValue; + /// use datafusion_expr::interval_arithmetic::Interval; + /// use datafusion_expr::interval_arithmetic::NullableInterval; + /// + /// let interval = NullableInterval::from(ScalarValue::Int32(Some(4))); + /// assert_eq!(interval.single_value(), Some(ScalarValue::Int32(Some(4)))); + /// + /// let interval = NullableInterval::from(ScalarValue::Int32(None)); + /// assert_eq!(interval.single_value(), Some(ScalarValue::Int32(None))); + /// + /// let interval = NullableInterval::MaybeNull { + /// values: Interval::try_new( + /// ScalarValue::Int32(Some(1)), + /// ScalarValue::Int32(Some(4)), + /// ).unwrap(), + /// }; + /// assert_eq!(interval.single_value(), None); + /// ``` + pub fn single_value(&self) -> Option { + match self { + Self::Null { datatype } => { + Some(ScalarValue::try_from(datatype).unwrap_or(ScalarValue::Null)) + } + Self::MaybeNull { values } | Self::NotNull { values } + if values.lower == values.upper && !values.lower.is_null() => + { + Some(values.lower.clone()) + } + _ => None, + } + } +} + +#[cfg(test)] +mod tests { + use crate::interval_arithmetic::{next_value, prev_value, satisfy_greater, Interval}; + + use arrow::datatypes::DataType; + use datafusion_common::{Result, ScalarValue}; + + #[test] + fn test_next_prev_value() -> Result<()> { + let zeros = vec![ + ScalarValue::new_zero(&DataType::UInt8)?, + ScalarValue::new_zero(&DataType::UInt16)?, + ScalarValue::new_zero(&DataType::UInt32)?, + ScalarValue::new_zero(&DataType::UInt64)?, + ScalarValue::new_zero(&DataType::Int8)?, + ScalarValue::new_zero(&DataType::Int16)?, + ScalarValue::new_zero(&DataType::Int32)?, + ScalarValue::new_zero(&DataType::Int64)?, + ]; + let ones = vec![ + ScalarValue::new_one(&DataType::UInt8)?, + ScalarValue::new_one(&DataType::UInt16)?, + ScalarValue::new_one(&DataType::UInt32)?, + ScalarValue::new_one(&DataType::UInt64)?, + ScalarValue::new_one(&DataType::Int8)?, + ScalarValue::new_one(&DataType::Int16)?, + ScalarValue::new_one(&DataType::Int32)?, + ScalarValue::new_one(&DataType::Int64)?, + ]; + zeros.into_iter().zip(ones).for_each(|(z, o)| { + assert_eq!(next_value(z.clone()), o); + assert_eq!(prev_value(o), z); + }); + + let values = vec![ + ScalarValue::new_zero(&DataType::Float32)?, + ScalarValue::new_zero(&DataType::Float64)?, + ]; + let eps = vec![ + ScalarValue::Float32(Some(1e-6)), + ScalarValue::Float64(Some(1e-6)), + ]; + values.into_iter().zip(eps).for_each(|(value, eps)| { + assert!(next_value(value.clone()) + .sub(value.clone()) + .unwrap() + .lt(&eps)); + assert!(value + .clone() + .sub(prev_value(value.clone())) + .unwrap() + .lt(&eps)); + assert_ne!(next_value(value.clone()), value); + assert_ne!(prev_value(value.clone()), value); + }); + + let min_max = vec![ + ( + ScalarValue::UInt64(Some(u64::MIN)), + ScalarValue::UInt64(Some(u64::MAX)), + ), + ( + ScalarValue::Int8(Some(i8::MIN)), + ScalarValue::Int8(Some(i8::MAX)), + ), + ( + ScalarValue::Float32(Some(f32::MIN)), + ScalarValue::Float32(Some(f32::MAX)), + ), + ( + ScalarValue::Float64(Some(f64::MIN)), + ScalarValue::Float64(Some(f64::MAX)), + ), + ]; + let inf = vec![ + ScalarValue::UInt64(None), + ScalarValue::Int8(None), + ScalarValue::Float32(None), + ScalarValue::Float64(None), + ]; + min_max.into_iter().zip(inf).for_each(|((min, max), inf)| { + assert_eq!(next_value(max.clone()), inf); + assert_ne!(prev_value(max.clone()), max); + assert_ne!(prev_value(max.clone()), inf); + + assert_eq!(prev_value(min.clone()), inf); + assert_ne!(next_value(min.clone()), min); + assert_ne!(next_value(min.clone()), inf); + + assert_eq!(next_value(inf.clone()), inf); + assert_eq!(prev_value(inf.clone()), inf); + }); + + Ok(()) + } + + #[test] + fn test_new_interval() -> Result<()> { + use ScalarValue::*; + + let cases = vec![ + ( + (Boolean(None), Boolean(Some(false))), + Boolean(Some(false)), + Boolean(Some(false)), + ), + ( + (Boolean(Some(false)), Boolean(None)), + Boolean(Some(false)), + Boolean(Some(true)), + ), + ( + (Boolean(Some(false)), Boolean(Some(true))), + Boolean(Some(false)), + Boolean(Some(true)), + ), + ( + (UInt16(Some(u16::MAX)), UInt16(None)), + UInt16(Some(u16::MAX)), + UInt16(None), + ), + ( + (Int16(None), Int16(Some(-1000))), + Int16(None), + Int16(Some(-1000)), + ), + ( + (Float32(Some(f32::MAX)), Float32(Some(f32::MAX))), + Float32(Some(f32::MAX)), + Float32(Some(f32::MAX)), + ), + ( + (Float32(Some(f32::NAN)), Float32(Some(f32::MIN))), + Float32(None), + Float32(Some(f32::MIN)), + ), + ( + ( + Float64(Some(f64::NEG_INFINITY)), + Float64(Some(f64::INFINITY)), + ), + Float64(None), + Float64(None), + ), + ]; + for (inputs, lower, upper) in cases { + let result = Interval::try_new(inputs.0, inputs.1)?; + assert_eq!(result.clone().lower(), &lower); + assert_eq!(result.upper(), &upper); + } + + let invalid_intervals = vec![ + (Float32(Some(f32::INFINITY)), Float32(Some(100_f32))), + (Float64(Some(0_f64)), Float64(Some(f64::NEG_INFINITY))), + (Boolean(Some(true)), Boolean(Some(false))), + (Int32(Some(1000)), Int32(Some(-2000))), + (UInt64(Some(1)), UInt64(Some(0))), + ]; + for (lower, upper) in invalid_intervals { + Interval::try_new(lower, upper).expect_err( + "Given parameters should have given an invalid interval error", + ); + } + + Ok(()) + } + + #[test] + fn test_make_unbounded() -> Result<()> { + use ScalarValue::*; + + let unbounded_cases = vec![ + (DataType::Boolean, Boolean(Some(false)), Boolean(Some(true))), + (DataType::UInt8, UInt8(None), UInt8(None)), + (DataType::UInt16, UInt16(None), UInt16(None)), + (DataType::UInt32, UInt32(None), UInt32(None)), + (DataType::UInt64, UInt64(None), UInt64(None)), + (DataType::Int8, Int8(None), Int8(None)), + (DataType::Int16, Int16(None), Int16(None)), + (DataType::Int32, Int32(None), Int32(None)), + (DataType::Int64, Int64(None), Int64(None)), + (DataType::Float32, Float32(None), Float32(None)), + (DataType::Float64, Float64(None), Float64(None)), + ]; + for (dt, lower, upper) in unbounded_cases { + let inf = Interval::make_unbounded(&dt)?; + assert_eq!(inf.clone().lower(), &lower); + assert_eq!(inf.upper(), &upper); + } + + Ok(()) + } + + #[test] + fn gt_lt_test() -> Result<()> { + let exactly_gt_cases = vec![ + ( + Interval::make(Some(1000_i64), None)?, + Interval::make(None, Some(999_i64))?, + ), + ( + Interval::make(Some(1000_i64), Some(1000_i64))?, + Interval::make(None, Some(999_i64))?, + ), + ( + Interval::make(Some(501_i64), Some(1000_i64))?, + Interval::make(Some(500_i64), Some(500_i64))?, + ), + ( + Interval::make(Some(-1000_i64), Some(1000_i64))?, + Interval::make(None, Some(-1500_i64))?, + ), + ( + Interval::try_new( + next_value(ScalarValue::Float32(Some(0.0))), + next_value(ScalarValue::Float32(Some(0.0))), + )?, + Interval::make(Some(0.0_f32), Some(0.0_f32))?, + ), + ( + Interval::make(Some(-1.0_f32), Some(-1.0_f32))?, + Interval::try_new( + prev_value(ScalarValue::Float32(Some(-1.0))), + prev_value(ScalarValue::Float32(Some(-1.0))), + )?, + ), + ]; + for (first, second) in exactly_gt_cases { + assert_eq!(first.gt(second.clone())?, Interval::CERTAINLY_TRUE); + assert_eq!(second.lt(first)?, Interval::CERTAINLY_TRUE); + } + + let possibly_gt_cases = vec![ + ( + Interval::make(Some(1000_i64), Some(2000_i64))?, + Interval::make(Some(1000_i64), Some(1000_i64))?, + ), + ( + Interval::make(Some(500_i64), Some(1000_i64))?, + Interval::make(Some(500_i64), Some(1000_i64))?, + ), + ( + Interval::make(Some(1000_i64), None)?, + Interval::make(Some(1000_i64), None)?, + ), + ( + Interval::make::(None, None)?, + Interval::make::(None, None)?, + ), + ( + Interval::try_new( + ScalarValue::Float32(Some(0.0_f32)), + next_value(ScalarValue::Float32(Some(0.0_f32))), + )?, + Interval::make(Some(0.0_f32), Some(0.0_f32))?, + ), + ( + Interval::make(Some(-1.0_f32), Some(-1.0_f32))?, + Interval::try_new( + prev_value(ScalarValue::Float32(Some(-1.0_f32))), + ScalarValue::Float32(Some(-1.0_f32)), + )?, + ), + ]; + for (first, second) in possibly_gt_cases { + assert_eq!(first.gt(second.clone())?, Interval::UNCERTAIN); + assert_eq!(second.lt(first)?, Interval::UNCERTAIN); + } + + let not_gt_cases = vec![ + ( + Interval::make(Some(1000_i64), Some(1000_i64))?, + Interval::make(Some(1000_i64), Some(1000_i64))?, + ), + ( + Interval::make(Some(500_i64), Some(1000_i64))?, + Interval::make(Some(1000_i64), None)?, + ), + ( + Interval::make(None, Some(1000_i64))?, + Interval::make(Some(1000_i64), Some(1500_i64))?, + ), + ( + Interval::try_new( + prev_value(ScalarValue::Float32(Some(0.0_f32))), + ScalarValue::Float32(Some(0.0_f32)), + )?, + Interval::make(Some(0.0_f32), Some(0.0_f32))?, + ), + ( + Interval::make(Some(-1.0_f32), Some(-1.0_f32))?, + Interval::try_new( + ScalarValue::Float32(Some(-1.0_f32)), + next_value(ScalarValue::Float32(Some(-1.0_f32))), + )?, + ), + ]; + for (first, second) in not_gt_cases { + assert_eq!(first.gt(second.clone())?, Interval::CERTAINLY_FALSE); + assert_eq!(second.lt(first)?, Interval::CERTAINLY_FALSE); + } + + Ok(()) + } + + #[test] + fn gteq_lteq_test() -> Result<()> { + let exactly_gteq_cases = vec![ + ( + Interval::make(Some(1000_i64), None)?, + Interval::make(None, Some(1000_i64))?, + ), + ( + Interval::make(Some(1000_i64), Some(1000_i64))?, + Interval::make(None, Some(1000_i64))?, + ), + ( + Interval::make(Some(500_i64), Some(1000_i64))?, + Interval::make(Some(500_i64), Some(500_i64))?, + ), + ( + Interval::make(Some(-1000_i64), Some(1000_i64))?, + Interval::make(None, Some(-1500_i64))?, + ), + ( + Interval::make(Some(0.0_f32), Some(0.0_f32))?, + Interval::make(Some(0.0_f32), Some(0.0_f32))?, + ), + ( + Interval::try_new( + ScalarValue::Float32(Some(-1.0)), + next_value(ScalarValue::Float32(Some(-1.0))), + )?, + Interval::try_new( + prev_value(ScalarValue::Float32(Some(-1.0))), + ScalarValue::Float32(Some(-1.0)), + )?, + ), + ]; + for (first, second) in exactly_gteq_cases { + assert_eq!(first.gt_eq(second.clone())?, Interval::CERTAINLY_TRUE); + assert_eq!(second.lt_eq(first)?, Interval::CERTAINLY_TRUE); + } + + let possibly_gteq_cases = vec![ + ( + Interval::make(Some(999_i64), Some(2000_i64))?, + Interval::make(Some(1000_i64), Some(1000_i64))?, + ), + ( + Interval::make(Some(500_i64), Some(1000_i64))?, + Interval::make(Some(500_i64), Some(1001_i64))?, + ), + ( + Interval::make(Some(0_i64), None)?, + Interval::make(Some(1000_i64), None)?, + ), + ( + Interval::make::(None, None)?, + Interval::make::(None, None)?, + ), + ( + Interval::try_new( + prev_value(ScalarValue::Float32(Some(0.0))), + ScalarValue::Float32(Some(0.0)), + )?, + Interval::make(Some(0.0_f32), Some(0.0_f32))?, + ), + ( + Interval::make(Some(-1.0_f32), Some(-1.0_f32))?, + Interval::try_new( + prev_value(ScalarValue::Float32(Some(-1.0_f32))), + next_value(ScalarValue::Float32(Some(-1.0_f32))), + )?, + ), + ]; + for (first, second) in possibly_gteq_cases { + assert_eq!(first.gt_eq(second.clone())?, Interval::UNCERTAIN); + assert_eq!(second.lt_eq(first)?, Interval::UNCERTAIN); + } + + let not_gteq_cases = vec![ + ( + Interval::make(Some(1000_i64), Some(1000_i64))?, + Interval::make(Some(2000_i64), Some(2000_i64))?, + ), + ( + Interval::make(Some(500_i64), Some(999_i64))?, + Interval::make(Some(1000_i64), None)?, + ), + ( + Interval::make(None, Some(1000_i64))?, + Interval::make(Some(1001_i64), Some(1500_i64))?, + ), + ( + Interval::try_new( + prev_value(ScalarValue::Float32(Some(0.0_f32))), + prev_value(ScalarValue::Float32(Some(0.0_f32))), + )?, + Interval::make(Some(0.0_f32), Some(0.0_f32))?, + ), + ( + Interval::make(Some(-1.0_f32), Some(-1.0_f32))?, + Interval::try_new( + next_value(ScalarValue::Float32(Some(-1.0))), + next_value(ScalarValue::Float32(Some(-1.0))), + )?, + ), + ]; + for (first, second) in not_gteq_cases { + assert_eq!(first.gt_eq(second.clone())?, Interval::CERTAINLY_FALSE); + assert_eq!(second.lt_eq(first)?, Interval::CERTAINLY_FALSE); + } + + Ok(()) + } + + #[test] + fn equal_test() -> Result<()> { + let exactly_eq_cases = vec![ + ( + Interval::make(Some(1000_i64), Some(1000_i64))?, + Interval::make(Some(1000_i64), Some(1000_i64))?, + ), + ( + Interval::make(Some(0_u64), Some(0_u64))?, + Interval::make(Some(0_u64), Some(0_u64))?, + ), + ( + Interval::make(Some(f32::MAX), Some(f32::MAX))?, + Interval::make(Some(f32::MAX), Some(f32::MAX))?, + ), + ( + Interval::make(Some(f64::MIN), Some(f64::MIN))?, + Interval::make(Some(f64::MIN), Some(f64::MIN))?, + ), + ]; + for (first, second) in exactly_eq_cases { + assert_eq!(first.equal(second.clone())?, Interval::CERTAINLY_TRUE); + assert_eq!(second.equal(first)?, Interval::CERTAINLY_TRUE); + } + + let possibly_eq_cases = vec![ + ( + Interval::make::(None, None)?, + Interval::make::(None, None)?, + ), + ( + Interval::make(Some(0_i64), Some(0_i64))?, + Interval::make(Some(0_i64), Some(1000_i64))?, + ), + ( + Interval::make(Some(0_i64), Some(0_i64))?, + Interval::make(Some(0_i64), Some(1000_i64))?, + ), + ( + Interval::make(Some(100.0_f32), Some(200.0_f32))?, + Interval::make(Some(0.0_f32), Some(1000.0_f32))?, + ), + ( + Interval::try_new( + prev_value(ScalarValue::Float32(Some(0.0))), + ScalarValue::Float32(Some(0.0)), + )?, + Interval::make(Some(0.0_f32), Some(0.0_f32))?, + ), + ( + Interval::make(Some(-1.0_f32), Some(-1.0_f32))?, + Interval::try_new( + prev_value(ScalarValue::Float32(Some(-1.0))), + next_value(ScalarValue::Float32(Some(-1.0))), + )?, + ), + ]; + for (first, second) in possibly_eq_cases { + assert_eq!(first.equal(second.clone())?, Interval::UNCERTAIN); + assert_eq!(second.equal(first)?, Interval::UNCERTAIN); + } + + let not_eq_cases = vec![ + ( + Interval::make(Some(1000_i64), Some(1000_i64))?, + Interval::make(Some(2000_i64), Some(2000_i64))?, + ), + ( + Interval::make(Some(500_i64), Some(999_i64))?, + Interval::make(Some(1000_i64), None)?, + ), + ( + Interval::make(None, Some(1000_i64))?, + Interval::make(Some(1001_i64), Some(1500_i64))?, + ), + ( + Interval::try_new( + prev_value(ScalarValue::Float32(Some(0.0))), + prev_value(ScalarValue::Float32(Some(0.0))), + )?, + Interval::make(Some(0.0_f32), Some(0.0_f32))?, + ), + ( + Interval::make(Some(-1.0_f32), Some(-1.0_f32))?, + Interval::try_new( + next_value(ScalarValue::Float32(Some(-1.0))), + next_value(ScalarValue::Float32(Some(-1.0))), + )?, + ), + ]; + for (first, second) in not_eq_cases { + assert_eq!(first.equal(second.clone())?, Interval::CERTAINLY_FALSE); + assert_eq!(second.equal(first)?, Interval::CERTAINLY_FALSE); + } + + Ok(()) + } + + #[test] + fn and_test() -> Result<()> { + let cases = vec![ + (false, true, false, false, false, false), + (false, false, false, true, false, false), + (false, true, false, true, false, true), + (false, true, true, true, false, true), + (false, false, false, false, false, false), + (true, true, true, true, true, true), + ]; + + for case in cases { + assert_eq!( + Interval::make(Some(case.0), Some(case.1))? + .and(Interval::make(Some(case.2), Some(case.3))?)?, + Interval::make(Some(case.4), Some(case.5))? + ); + } + Ok(()) + } + + #[test] + fn not_test() -> Result<()> { + let cases = vec![ + (false, true, false, true), + (false, false, true, true), + (true, true, false, false), + ]; + + for case in cases { + assert_eq!( + Interval::make(Some(case.0), Some(case.1))?.not()?, + Interval::make(Some(case.2), Some(case.3))? + ); + } + Ok(()) + } + + #[test] + fn intersect_test() -> Result<()> { + let possible_cases = vec![ + ( + Interval::make(Some(1000_i64), None)?, + Interval::make::(None, None)?, + Interval::make(Some(1000_i64), None)?, + ), + ( + Interval::make(Some(1000_i64), None)?, + Interval::make(None, Some(1000_i64))?, + Interval::make(Some(1000_i64), Some(1000_i64))?, + ), + ( + Interval::make(Some(1000_i64), None)?, + Interval::make(None, Some(2000_i64))?, + Interval::make(Some(1000_i64), Some(2000_i64))?, + ), + ( + Interval::make(Some(1000_i64), Some(2000_i64))?, + Interval::make(Some(1000_i64), None)?, + Interval::make(Some(1000_i64), Some(2000_i64))?, + ), + ( + Interval::make(Some(1000_i64), Some(2000_i64))?, + Interval::make(Some(1000_i64), Some(1500_i64))?, + Interval::make(Some(1000_i64), Some(1500_i64))?, + ), + ( + Interval::make(Some(1000_i64), Some(2000_i64))?, + Interval::make(Some(500_i64), Some(1500_i64))?, + Interval::make(Some(1000_i64), Some(1500_i64))?, + ), + ( + Interval::make::(None, None)?, + Interval::make::(None, None)?, + Interval::make::(None, None)?, + ), + ( + Interval::make(None, Some(2000_u64))?, + Interval::make(Some(500_u64), None)?, + Interval::make(Some(500_u64), Some(2000_u64))?, + ), + ( + Interval::make(Some(0_u64), Some(0_u64))?, + Interval::make(Some(0_u64), None)?, + Interval::make(Some(0_u64), Some(0_u64))?, + ), + ( + Interval::make(Some(1000.0_f32), None)?, + Interval::make(None, Some(1000.0_f32))?, + Interval::make(Some(1000.0_f32), Some(1000.0_f32))?, + ), + ( + Interval::make(Some(1000.0_f32), Some(1500.0_f32))?, + Interval::make(Some(0.0_f32), Some(1500.0_f32))?, + Interval::make(Some(1000.0_f32), Some(1500.0_f32))?, + ), + ( + Interval::make(Some(-1000.0_f64), Some(1500.0_f64))?, + Interval::make(Some(-1500.0_f64), Some(2000.0_f64))?, + Interval::make(Some(-1000.0_f64), Some(1500.0_f64))?, + ), + ( + Interval::make(Some(16.0_f64), Some(32.0_f64))?, + Interval::make(Some(32.0_f64), Some(64.0_f64))?, + Interval::make(Some(32.0_f64), Some(32.0_f64))?, + ), + ]; + for (first, second, expected) in possible_cases { + assert_eq!(first.intersect(second)?.unwrap(), expected) + } + + let empty_cases = vec![ + ( + Interval::make(Some(1000_i64), None)?, + Interval::make(None, Some(0_i64))?, + ), + ( + Interval::make(Some(1000_i64), None)?, + Interval::make(None, Some(999_i64))?, + ), + ( + Interval::make(Some(1500_i64), Some(2000_i64))?, + Interval::make(Some(1000_i64), Some(1499_i64))?, + ), + ( + Interval::make(Some(0_i64), Some(1000_i64))?, + Interval::make(Some(2000_i64), Some(3000_i64))?, + ), + ( + Interval::try_new( + prev_value(ScalarValue::Float32(Some(1.0))), + prev_value(ScalarValue::Float32(Some(1.0))), + )?, + Interval::make(Some(1.0_f32), Some(1.0_f32))?, + ), + ( + Interval::try_new( + next_value(ScalarValue::Float32(Some(1.0))), + next_value(ScalarValue::Float32(Some(1.0))), + )?, + Interval::make(Some(1.0_f32), Some(1.0_f32))?, + ), + ]; + for (first, second) in empty_cases { + assert_eq!(first.intersect(second)?, None) + } + + Ok(()) + } + + #[test] + fn test_contains() -> Result<()> { + let possible_cases = vec![ + ( + Interval::make::(None, None)?, + Interval::make::(None, None)?, + Interval::CERTAINLY_TRUE, + ), + ( + Interval::make(Some(1500_i64), Some(2000_i64))?, + Interval::make(Some(1501_i64), Some(1999_i64))?, + Interval::CERTAINLY_TRUE, + ), + ( + Interval::make(Some(1000_i64), None)?, + Interval::make::(None, None)?, + Interval::UNCERTAIN, + ), + ( + Interval::make(Some(1000_i64), Some(2000_i64))?, + Interval::make(Some(500), Some(1500_i64))?, + Interval::UNCERTAIN, + ), + ( + Interval::make(Some(16.0), Some(32.0))?, + Interval::make(Some(32.0), Some(64.0))?, + Interval::UNCERTAIN, + ), + ( + Interval::make(Some(1000_i64), None)?, + Interval::make(None, Some(0_i64))?, + Interval::CERTAINLY_FALSE, + ), + ( + Interval::make(Some(1500_i64), Some(2000_i64))?, + Interval::make(Some(1000_i64), Some(1499_i64))?, + Interval::CERTAINLY_FALSE, + ), + ( + Interval::try_new( + prev_value(ScalarValue::Float32(Some(1.0))), + prev_value(ScalarValue::Float32(Some(1.0))), + )?, + Interval::make(Some(1.0_f32), Some(1.0_f32))?, + Interval::CERTAINLY_FALSE, + ), + ( + Interval::try_new( + next_value(ScalarValue::Float32(Some(1.0))), + next_value(ScalarValue::Float32(Some(1.0))), + )?, + Interval::make(Some(1.0_f32), Some(1.0_f32))?, + Interval::CERTAINLY_FALSE, + ), + ]; + for (first, second, expected) in possible_cases { + assert_eq!(first.contains(second)?, expected) + } + + Ok(()) + } + + #[test] + fn test_add() -> Result<()> { + let cases = vec![ + ( + Interval::make(Some(100_i64), Some(200_i64))?, + Interval::make(None, Some(200_i64))?, + Interval::make(None, Some(400_i64))?, + ), + ( + Interval::make(Some(100_i64), Some(200_i64))?, + Interval::make(Some(200_i64), None)?, + Interval::make(Some(300_i64), None)?, + ), + ( + Interval::make(None, Some(200_i64))?, + Interval::make(Some(100_i64), Some(200_i64))?, + Interval::make(None, Some(400_i64))?, + ), + ( + Interval::make(Some(200_i64), None)?, + Interval::make(Some(100_i64), Some(200_i64))?, + Interval::make(Some(300_i64), None)?, + ), + ( + Interval::make(Some(100_i64), Some(200_i64))?, + Interval::make(Some(-300_i64), Some(150_i64))?, + Interval::make(Some(-200_i64), Some(350_i64))?, + ), + ( + Interval::make(Some(f32::MAX), Some(f32::MAX))?, + Interval::make(Some(11_f32), Some(11_f32))?, + Interval::make(Some(f32::MAX), None)?, + ), + ( + Interval::make(Some(f32::MIN), Some(f32::MIN))?, + Interval::make(Some(-10_f32), Some(10_f32))?, + // Since rounding mode is up, the result would be much greater than f32::MIN + // (f32::MIN = -3.4_028_235e38, the result is -3.4_028_233e38) + Interval::make( + None, + Some(-340282330000000000000000000000000000000.0_f32), + )?, + ), + ( + Interval::make(Some(f32::MIN), Some(f32::MIN))?, + Interval::make(Some(-10_f32), Some(-10_f32))?, + Interval::make(None, Some(f32::MIN))?, + ), + ( + Interval::make(Some(1.0), Some(f32::MAX))?, + Interval::make(Some(f32::MAX), Some(f32::MAX))?, + Interval::make(Some(f32::MAX), None)?, + ), + ( + Interval::make(Some(f32::MIN), Some(f32::MIN))?, + Interval::make(Some(f32::MAX), Some(f32::MAX))?, + Interval::make(Some(-0.0_f32), Some(0.0_f32))?, + ), + ( + Interval::make(Some(100_f64), None)?, + Interval::make(None, Some(200_f64))?, + Interval::make::(None, None)?, + ), + ( + Interval::make(None, Some(100_f64))?, + Interval::make(None, Some(200_f64))?, + Interval::make(None, Some(300_f64))?, + ), + ]; + for case in cases { + let result = case.0.add(case.1)?; + if case.0.data_type().is_floating() { + assert!( + result.lower().is_null() && case.2.lower().is_null() + || result.lower().le(case.2.lower()) + ); + assert!( + result.upper().is_null() && case.2.upper().is_null() + || result.upper().ge(case.2.upper()) + ); + } else { + assert_eq!(result, case.2); + } + } + + Ok(()) + } + + #[test] + fn test_sub() -> Result<()> { + let cases = vec![ + ( + Interval::make(Some(i32::MAX), Some(i32::MAX))?, + Interval::make(Some(11_i32), Some(11_i32))?, + Interval::make(Some(i32::MAX - 11), Some(i32::MAX - 11))?, + ), + ( + Interval::make(Some(100_i64), Some(200_i64))?, + Interval::make(None, Some(200_i64))?, + Interval::make(Some(-100_i64), None)?, + ), + ( + Interval::make(Some(100_i64), Some(200_i64))?, + Interval::make(Some(200_i64), None)?, + Interval::make(None, Some(0_i64))?, + ), + ( + Interval::make(None, Some(200_i64))?, + Interval::make(Some(100_i64), Some(200_i64))?, + Interval::make(None, Some(100_i64))?, + ), + ( + Interval::make(Some(200_i64), None)?, + Interval::make(Some(100_i64), Some(200_i64))?, + Interval::make(Some(0_i64), None)?, + ), + ( + Interval::make(Some(100_i64), Some(200_i64))?, + Interval::make(Some(-300_i64), Some(150_i64))?, + Interval::make(Some(-50_i64), Some(500_i64))?, + ), + ( + Interval::make(Some(i64::MIN), Some(i64::MIN))?, + Interval::make(Some(-10_i64), Some(-10_i64))?, + Interval::make(Some(i64::MIN + 10), Some(i64::MIN + 10))?, + ), + ( + Interval::make(Some(1), Some(i64::MAX))?, + Interval::make(Some(i64::MAX), Some(i64::MAX))?, + Interval::make(Some(1 - i64::MAX), Some(0))?, + ), + ( + Interval::make(Some(i64::MIN), Some(i64::MIN))?, + Interval::make(Some(i64::MAX), Some(i64::MAX))?, + Interval::make(None, Some(i64::MIN))?, + ), + ( + Interval::make(Some(2_u32), Some(10_u32))?, + Interval::make(Some(4_u32), Some(6_u32))?, + Interval::make(None, Some(6_u32))?, + ), + ( + Interval::make(Some(2_u32), Some(10_u32))?, + Interval::make(Some(20_u32), Some(30_u32))?, + Interval::make(None, Some(0_u32))?, + ), + ( + Interval::make(Some(f32::MIN), Some(f32::MIN))?, + Interval::make(Some(-10_f32), Some(10_f32))?, + // Since rounding mode is up, the result would be much larger than f32::MIN + // (f32::MIN = -3.4_028_235e38, the result is -3.4_028_233e38) + Interval::make( + None, + Some(-340282330000000000000000000000000000000.0_f32), + )?, + ), + ( + Interval::make(Some(100_f64), None)?, + Interval::make(None, Some(200_f64))?, + Interval::make(Some(-100_f64), None)?, + ), + ( + Interval::make(None, Some(100_f64))?, + Interval::make(None, Some(200_f64))?, + Interval::make::(None, None)?, + ), + ]; + for case in cases { + let result = case.0.sub(case.1)?; + if case.0.data_type().is_floating() { + assert!( + result.lower().is_null() && case.2.lower().is_null() + || result.lower().le(case.2.lower()) + ); + assert!( + result.upper().is_null() && case.2.upper().is_null() + || result.upper().ge(case.2.upper(),) + ); + } else { + assert_eq!(result, case.2); + } + } + + Ok(()) + } + + #[test] + fn test_mul() -> Result<()> { + let cases = vec![ + ( + Interval::make(Some(1_i64), Some(2_i64))?, + Interval::make(None, Some(2_i64))?, + Interval::make(None, Some(4_i64))?, + ), + ( + Interval::make(Some(1_i64), Some(2_i64))?, + Interval::make(Some(2_i64), None)?, + Interval::make(Some(2_i64), None)?, + ), + ( + Interval::make(None, Some(2_i64))?, + Interval::make(Some(1_i64), Some(2_i64))?, + Interval::make(None, Some(4_i64))?, + ), + ( + Interval::make(Some(2_i64), None)?, + Interval::make(Some(1_i64), Some(2_i64))?, + Interval::make(Some(2_i64), None)?, + ), + ( + Interval::make(Some(1_i64), Some(2_i64))?, + Interval::make(Some(-3_i64), Some(15_i64))?, + Interval::make(Some(-6_i64), Some(30_i64))?, + ), + ( + Interval::make(Some(-0.0), Some(0.0))?, + Interval::make(None, Some(0.0))?, + Interval::make::(None, None)?, + ), + ( + Interval::make(Some(f32::MIN), Some(f32::MIN))?, + Interval::make(Some(-10_f32), Some(10_f32))?, + Interval::make::(None, None)?, + ), + ( + Interval::make(Some(1_u32), Some(2_u32))?, + Interval::make(Some(0_u32), Some(1_u32))?, + Interval::make(Some(0_u32), Some(2_u32))?, + ), + ( + Interval::make(None, Some(2_u32))?, + Interval::make(Some(0_u32), Some(1_u32))?, + Interval::make(None, Some(2_u32))?, + ), + ( + Interval::make(None, Some(2_u32))?, + Interval::make(Some(1_u32), Some(2_u32))?, + Interval::make(None, Some(4_u32))?, + ), + ( + Interval::make(None, Some(2_u32))?, + Interval::make(Some(1_u32), None)?, + Interval::make::(None, None)?, + ), + ( + Interval::make::(None, None)?, + Interval::make(Some(0_u32), None)?, + Interval::make::(None, None)?, + ), + ( + Interval::make(Some(f32::MAX), Some(f32::MAX))?, + Interval::make(Some(11_f32), Some(11_f32))?, + Interval::make(Some(f32::MAX), None)?, + ), + ( + Interval::make(Some(f32::MIN), Some(f32::MIN))?, + Interval::make(Some(-10_f32), Some(-10_f32))?, + Interval::make(Some(f32::MAX), None)?, + ), + ( + Interval::make(Some(1.0), Some(f32::MAX))?, + Interval::make(Some(f32::MAX), Some(f32::MAX))?, + Interval::make(Some(f32::MAX), None)?, + ), + ( + Interval::make(Some(f32::MIN), Some(f32::MIN))?, + Interval::make(Some(f32::MAX), Some(f32::MAX))?, + Interval::make(None, Some(f32::MIN))?, + ), + ( + Interval::make(Some(-0.0_f32), Some(0.0_f32))?, + Interval::make(Some(f32::MAX), None)?, + Interval::make::(None, None)?, + ), + ( + Interval::make(Some(0.0_f32), Some(0.0_f32))?, + Interval::make(Some(f32::MAX), None)?, + Interval::make(Some(0.0_f32), None)?, + ), + ( + Interval::make(Some(1_f64), None)?, + Interval::make(None, Some(2_f64))?, + Interval::make::(None, None)?, + ), + ( + Interval::make(None, Some(1_f64))?, + Interval::make(None, Some(2_f64))?, + Interval::make::(None, None)?, + ), + ( + Interval::make(Some(-0.0_f64), Some(-0.0_f64))?, + Interval::make(Some(1_f64), Some(2_f64))?, + Interval::make(Some(-0.0_f64), Some(-0.0_f64))?, + ), + ( + Interval::make(Some(0.0_f64), Some(0.0_f64))?, + Interval::make(Some(1_f64), Some(2_f64))?, + Interval::make(Some(0.0_f64), Some(0.0_f64))?, + ), + ( + Interval::make(Some(-0.0_f64), Some(0.0_f64))?, + Interval::make(Some(1_f64), Some(2_f64))?, + Interval::make(Some(-0.0_f64), Some(0.0_f64))?, + ), + ( + Interval::make(Some(-0.0_f64), Some(1.0_f64))?, + Interval::make(Some(1_f64), Some(2_f64))?, + Interval::make(Some(-0.0_f64), Some(2.0_f64))?, + ), + ( + Interval::make(Some(0.0_f64), Some(1.0_f64))?, + Interval::make(Some(1_f64), Some(2_f64))?, + Interval::make(Some(0.0_f64), Some(2.0_f64))?, + ), + ( + Interval::make(Some(-0.0_f64), Some(1.0_f64))?, + Interval::make(Some(-1_f64), Some(2_f64))?, + Interval::make(Some(-1.0_f64), Some(2.0_f64))?, + ), + ( + Interval::make::(None, None)?, + Interval::make(Some(-0.0_f64), Some(0.0_f64))?, + Interval::make::(None, None)?, + ), + ( + Interval::make::(None, Some(10.0_f64))?, + Interval::make(Some(-0.0_f64), Some(0.0_f64))?, + Interval::make::(None, None)?, + ), + ]; + for case in cases { + let result = case.0.mul(case.1)?; + if case.0.data_type().is_floating() { + assert!( + result.lower().is_null() && case.2.lower().is_null() + || result.lower().le(case.2.lower()) + ); + assert!( + result.upper().is_null() && case.2.upper().is_null() + || result.upper().ge(case.2.upper()) + ); + } else { + assert_eq!(result, case.2); + } + } + + Ok(()) + } + + #[test] + fn test_div() -> Result<()> { + let cases = vec![ + ( + Interval::make(Some(100_i64), Some(200_i64))?, + Interval::make(Some(1_i64), Some(2_i64))?, + Interval::make(Some(50_i64), Some(200_i64))?, + ), + ( + Interval::make(Some(-200_i64), Some(-100_i64))?, + Interval::make(Some(-2_i64), Some(-1_i64))?, + Interval::make(Some(50_i64), Some(200_i64))?, + ), + ( + Interval::make(Some(100_i64), Some(200_i64))?, + Interval::make(Some(-2_i64), Some(-1_i64))?, + Interval::make(Some(-200_i64), Some(-50_i64))?, + ), + ( + Interval::make(Some(-200_i64), Some(-100_i64))?, + Interval::make(Some(1_i64), Some(2_i64))?, + Interval::make(Some(-200_i64), Some(-50_i64))?, + ), + ( + Interval::make(Some(-200_i64), Some(100_i64))?, + Interval::make(Some(1_i64), Some(2_i64))?, + Interval::make(Some(-200_i64), Some(100_i64))?, + ), + ( + Interval::make(Some(-100_i64), Some(200_i64))?, + Interval::make(Some(1_i64), Some(2_i64))?, + Interval::make(Some(-100_i64), Some(200_i64))?, + ), + ( + Interval::make(Some(10_i64), Some(20_i64))?, + Interval::make::(None, None)?, + Interval::make::(None, None)?, + ), + ( + Interval::make(Some(-100_i64), Some(200_i64))?, + Interval::make(Some(-1_i64), Some(2_i64))?, + Interval::make::(None, None)?, + ), + ( + Interval::make(Some(-100_i64), Some(200_i64))?, + Interval::make(Some(-2_i64), Some(1_i64))?, + Interval::make::(None, None)?, + ), + ( + Interval::make(Some(100_i64), Some(200_i64))?, + Interval::make(Some(0_i64), Some(1_i64))?, + Interval::make(Some(100_i64), None)?, + ), + ( + Interval::make(Some(100_i64), Some(200_i64))?, + Interval::make(None, Some(0_i64))?, + Interval::make(None, Some(0_i64))?, + ), + ( + Interval::make(Some(100_i64), Some(200_i64))?, + Interval::make(Some(0_i64), Some(0_i64))?, + Interval::make::(None, None)?, + ), + ( + Interval::make(Some(0_i64), Some(1_i64))?, + Interval::make(Some(100_i64), Some(200_i64))?, + Interval::make(Some(0_i64), Some(0_i64))?, + ), + ( + Interval::make(Some(0_i64), Some(1_i64))?, + Interval::make(Some(100_i64), Some(200_i64))?, + Interval::make(Some(0_i64), Some(0_i64))?, + ), + ( + Interval::make(Some(1_u32), Some(2_u32))?, + Interval::make(Some(0_u32), Some(0_u32))?, + Interval::make::(None, None)?, + ), + ( + Interval::make(Some(10_u32), Some(20_u32))?, + Interval::make(None, Some(2_u32))?, + Interval::make(Some(5_u32), None)?, + ), + ( + Interval::make(Some(10_u32), Some(20_u32))?, + Interval::make(Some(0_u32), Some(2_u32))?, + Interval::make(Some(5_u32), None)?, + ), + ( + Interval::make(Some(10_u32), Some(20_u32))?, + Interval::make(Some(0_u32), Some(0_u32))?, + Interval::make::(None, None)?, + ), + ( + Interval::make(Some(12_u64), Some(48_u64))?, + Interval::make(Some(10_u64), Some(20_u64))?, + Interval::make(Some(0_u64), Some(4_u64))?, + ), + ( + Interval::make(Some(12_u64), Some(48_u64))?, + Interval::make(None, Some(2_u64))?, + Interval::make(Some(6_u64), None)?, + ), + ( + Interval::make(Some(12_u64), Some(48_u64))?, + Interval::make(Some(0_u64), Some(2_u64))?, + Interval::make(Some(6_u64), None)?, + ), + ( + Interval::make(None, Some(48_u64))?, + Interval::make(Some(0_u64), Some(2_u64))?, + Interval::make::(None, None)?, + ), + ( + Interval::make(Some(f32::MAX), Some(f32::MAX))?, + Interval::make(Some(-0.1_f32), Some(0.1_f32))?, + Interval::make::(None, None)?, + ), + ( + Interval::make(Some(f32::MIN), None)?, + Interval::make(Some(0.1_f32), Some(0.1_f32))?, + Interval::make::(None, None)?, + ), + ( + Interval::make(Some(-10.0_f32), Some(10.0_f32))?, + Interval::make(Some(-0.1_f32), Some(-0.1_f32))?, + Interval::make(Some(-100.0_f32), Some(100.0_f32))?, + ), + ( + Interval::make(Some(-10.0_f32), Some(f32::MAX))?, + Interval::make::(None, None)?, + Interval::make::(None, None)?, + ), + ( + Interval::make(Some(f32::MIN), Some(10.0_f32))?, + Interval::make(Some(1.0_f32), None)?, + Interval::make(Some(f32::MIN), Some(10.0_f32))?, + ), + ( + Interval::make(Some(-0.0_f32), Some(0.0_f32))?, + Interval::make(Some(f32::MAX), None)?, + Interval::make(Some(-0.0_f32), Some(0.0_f32))?, + ), + ( + Interval::make(Some(-0.0_f32), Some(0.0_f32))?, + Interval::make(None, Some(-0.0_f32))?, + Interval::make::(None, None)?, + ), + ( + Interval::make(Some(0.0_f32), Some(0.0_f32))?, + Interval::make(Some(f32::MAX), None)?, + Interval::make(Some(0.0_f32), Some(0.0_f32))?, + ), + ( + Interval::make(Some(1.0_f32), Some(2.0_f32))?, + Interval::make(Some(0.0_f32), Some(4.0_f32))?, + Interval::make(Some(0.25_f32), None)?, + ), + ( + Interval::make(Some(1.0_f32), Some(2.0_f32))?, + Interval::make(Some(-4.0_f32), Some(-0.0_f32))?, + Interval::make(None, Some(-0.25_f32))?, + ), + ( + Interval::make(Some(-4.0_f64), Some(2.0_f64))?, + Interval::make(Some(10.0_f64), Some(20.0_f64))?, + Interval::make(Some(-0.4_f64), Some(0.2_f64))?, + ), + ( + Interval::make(Some(-0.0_f64), Some(-0.0_f64))?, + Interval::make(None, Some(-0.0_f64))?, + Interval::make(Some(0.0_f64), None)?, + ), + ( + Interval::make(Some(1.0_f64), Some(2.0_f64))?, + Interval::make::(None, None)?, + Interval::make(Some(0.0_f64), None)?, + ), + ]; + for case in cases { + let result = case.0.div(case.1)?; + if case.0.data_type().is_floating() { + assert!( + result.lower().is_null() && case.2.lower().is_null() + || result.lower().le(case.2.lower()) + ); + assert!( + result.upper().is_null() && case.2.upper().is_null() + || result.upper().ge(case.2.upper()) + ); + } else { + assert_eq!(result, case.2); + } + } + + Ok(()) + } + + #[test] + fn test_cardinality_of_intervals() -> Result<()> { + // In IEEE 754 standard for floating-point arithmetic, if we keep the sign and exponent fields same, + // we can represent 4503599627370496+1 different numbers by changing the mantissa + // (4503599627370496 = 2^52, since there are 52 bits in mantissa, and 2^23 = 8388608 for f32). + // TODO: Add tests for non-exponential boundary aligned intervals too. + let distinct_f64 = 4503599627370497; + let distinct_f32 = 8388609; + let intervals = [ + Interval::make(Some(0.25_f64), Some(0.50_f64))?, + Interval::make(Some(0.5_f64), Some(1.0_f64))?, + Interval::make(Some(1.0_f64), Some(2.0_f64))?, + Interval::make(Some(32.0_f64), Some(64.0_f64))?, + Interval::make(Some(-0.50_f64), Some(-0.25_f64))?, + Interval::make(Some(-32.0_f64), Some(-16.0_f64))?, + ]; + for interval in intervals { + assert_eq!(interval.cardinality().unwrap(), distinct_f64); + } + + let intervals = [ + Interval::make(Some(0.25_f32), Some(0.50_f32))?, + Interval::make(Some(-1_f32), Some(-0.5_f32))?, + ]; + for interval in intervals { + assert_eq!(interval.cardinality().unwrap(), distinct_f32); + } + + // The regular logarithmic distribution of floating-point numbers are + // only applicable outside of the `(-phi, phi)` interval where `phi` + // denotes the largest positive subnormal floating-point number. Since + // the following intervals include such subnormal points, we cannot use + // a simple powers-of-two type formula for our expectations. Therefore, + // we manually supply the actual expected cardinality. + let interval = Interval::make(Some(-0.0625), Some(0.0625))?; + assert_eq!(interval.cardinality().unwrap(), 9178336040581070850); + + let interval = Interval::try_new( + ScalarValue::UInt64(Some(u64::MIN + 1)), + ScalarValue::UInt64(Some(u64::MAX)), + )?; + assert_eq!(interval.cardinality().unwrap(), u64::MAX); + + let interval = Interval::try_new( + ScalarValue::Int64(Some(i64::MIN + 1)), + ScalarValue::Int64(Some(i64::MAX)), + )?; + assert_eq!(interval.cardinality().unwrap(), u64::MAX); + + let interval = Interval::try_new( + ScalarValue::Float32(Some(-0.0_f32)), + ScalarValue::Float32(Some(0.0_f32)), + )?; + assert_eq!(interval.cardinality().unwrap(), 2); + + Ok(()) + } + + #[test] + fn test_satisfy_comparison() -> Result<()> { + let cases = vec![ + ( + Interval::make(Some(1000_i64), None)?, + Interval::make(None, Some(1000_i64))?, + true, + Interval::make(Some(1000_i64), None)?, + Interval::make(None, Some(1000_i64))?, + ), + ( + Interval::make(None, Some(1000_i64))?, + Interval::make(Some(1000_i64), None)?, + true, + Interval::make(Some(1000_i64), Some(1000_i64))?, + Interval::make(Some(1000_i64), Some(1000_i64))?, + ), + ( + Interval::make(Some(1000_i64), None)?, + Interval::make(None, Some(1000_i64))?, + false, + Interval::make(Some(1000_i64), None)?, + Interval::make(None, Some(1000_i64))?, + ), + ( + Interval::make(Some(0_i64), Some(1000_i64))?, + Interval::make(Some(500_i64), Some(1500_i64))?, + true, + Interval::make(Some(500_i64), Some(1000_i64))?, + Interval::make(Some(500_i64), Some(1000_i64))?, + ), + ( + Interval::make(Some(500_i64), Some(1500_i64))?, + Interval::make(Some(0_i64), Some(1000_i64))?, + true, + Interval::make(Some(500_i64), Some(1500_i64))?, + Interval::make(Some(0_i64), Some(1000_i64))?, + ), + ( + Interval::make(Some(0_i64), Some(1000_i64))?, + Interval::make(Some(500_i64), Some(1500_i64))?, + false, + Interval::make(Some(501_i64), Some(1000_i64))?, + Interval::make(Some(500_i64), Some(999_i64))?, + ), + ( + Interval::make(Some(500_i64), Some(1500_i64))?, + Interval::make(Some(0_i64), Some(1000_i64))?, + false, + Interval::make(Some(500_i64), Some(1500_i64))?, + Interval::make(Some(0_i64), Some(1000_i64))?, + ), + ( + Interval::make::(None, None)?, + Interval::make(Some(1_i64), Some(1_i64))?, + false, + Interval::make(Some(2_i64), None)?, + Interval::make(Some(1_i64), Some(1_i64))?, + ), + ( + Interval::make::(None, None)?, + Interval::make(Some(1_i64), Some(1_i64))?, + true, + Interval::make(Some(1_i64), None)?, + Interval::make(Some(1_i64), Some(1_i64))?, + ), + ( + Interval::make(Some(1_i64), Some(1_i64))?, + Interval::make::(None, None)?, + false, + Interval::make(Some(1_i64), Some(1_i64))?, + Interval::make(None, Some(0_i64))?, + ), + ( + Interval::make(Some(1_i64), Some(1_i64))?, + Interval::make::(None, None)?, + true, + Interval::make(Some(1_i64), Some(1_i64))?, + Interval::make(None, Some(1_i64))?, + ), + ( + Interval::make(Some(1_i64), Some(1_i64))?, + Interval::make::(None, None)?, + false, + Interval::make(Some(1_i64), Some(1_i64))?, + Interval::make(None, Some(0_i64))?, + ), + ( + Interval::make(Some(1_i64), Some(1_i64))?, + Interval::make::(None, None)?, + true, + Interval::make(Some(1_i64), Some(1_i64))?, + Interval::make(None, Some(1_i64))?, + ), + ( + Interval::make::(None, None)?, + Interval::make(Some(1_i64), Some(1_i64))?, + false, + Interval::make(Some(2_i64), None)?, + Interval::make(Some(1_i64), Some(1_i64))?, + ), + ( + Interval::make::(None, None)?, + Interval::make(Some(1_i64), Some(1_i64))?, + true, + Interval::make(Some(1_i64), None)?, + Interval::make(Some(1_i64), Some(1_i64))?, + ), + ( + Interval::make(Some(-1000.0_f32), Some(1000.0_f32))?, + Interval::make(Some(-500.0_f32), Some(500.0_f32))?, + false, + Interval::try_new( + next_value(ScalarValue::Float32(Some(-500.0))), + ScalarValue::Float32(Some(1000.0)), + )?, + Interval::make(Some(-500_f32), Some(500.0_f32))?, + ), + ( + Interval::make(Some(-500.0_f32), Some(500.0_f32))?, + Interval::make(Some(-1000.0_f32), Some(1000.0_f32))?, + true, + Interval::make(Some(-500.0_f32), Some(500.0_f32))?, + Interval::make(Some(-1000.0_f32), Some(500.0_f32))?, + ), + ( + Interval::make(Some(-500.0_f32), Some(500.0_f32))?, + Interval::make(Some(-1000.0_f32), Some(1000.0_f32))?, + false, + Interval::make(Some(-500.0_f32), Some(500.0_f32))?, + Interval::try_new( + ScalarValue::Float32(Some(-1000.0_f32)), + prev_value(ScalarValue::Float32(Some(500.0_f32))), + )?, + ), + ( + Interval::make(Some(-1000.0_f64), Some(1000.0_f64))?, + Interval::make(Some(-500.0_f64), Some(500.0_f64))?, + true, + Interval::make(Some(-500.0_f64), Some(1000.0_f64))?, + Interval::make(Some(-500.0_f64), Some(500.0_f64))?, + ), + ]; + for (first, second, includes_endpoints, left_modified, right_modified) in cases { + assert_eq!( + satisfy_greater(&first, &second, !includes_endpoints)?.unwrap(), + (left_modified, right_modified) + ); + } + + let infeasible_cases = vec![ + ( + Interval::make(None, Some(1000_i64))?, + Interval::make(Some(1000_i64), None)?, + false, + ), + ( + Interval::make(Some(-1000.0_f32), Some(1000.0_f32))?, + Interval::make(Some(1500.0_f32), Some(2000.0_f32))?, + false, + ), + ]; + for (first, second, includes_endpoints) in infeasible_cases { + assert_eq!(satisfy_greater(&first, &second, !includes_endpoints)?, None); + } + + Ok(()) + } + + #[test] + fn test_interval_display() { + let interval = Interval::make(Some(0.25_f32), Some(0.50_f32)).unwrap(); + assert_eq!(format!("{}", interval), "[0.25, 0.5]"); + + let interval = Interval::try_new( + ScalarValue::Float32(Some(f32::NEG_INFINITY)), + ScalarValue::Float32(Some(f32::INFINITY)), + ) + .unwrap(); + assert_eq!(format!("{}", interval), "[NULL, NULL]"); + } + + macro_rules! capture_mode_change { + ($TYPE:ty) => { + paste::item! { + capture_mode_change_helper!([], + [], + $TYPE); + } + }; + } + + macro_rules! capture_mode_change_helper { + ($TEST_FN_NAME:ident, $CREATE_FN_NAME:ident, $TYPE:ty) => { + fn $CREATE_FN_NAME(lower: $TYPE, upper: $TYPE) -> Interval { + Interval::try_new( + ScalarValue::try_from(Some(lower as $TYPE)).unwrap(), + ScalarValue::try_from(Some(upper as $TYPE)).unwrap(), + ) + .unwrap() + } + + fn $TEST_FN_NAME(input: ($TYPE, $TYPE), expect_low: bool, expect_high: bool) { + assert!(expect_low || expect_high); + let interval1 = $CREATE_FN_NAME(input.0, input.0); + let interval2 = $CREATE_FN_NAME(input.1, input.1); + let result = interval1.add(&interval2).unwrap(); + let without_fe = $CREATE_FN_NAME(input.0 + input.1, input.0 + input.1); + assert!( + (!expect_low || result.lower < without_fe.lower) + && (!expect_high || result.upper > without_fe.upper) + ); + } + }; + } + + capture_mode_change!(f32); + capture_mode_change!(f64); + + #[cfg(all( + any(target_arch = "x86_64", target_arch = "aarch64"), + not(target_os = "windows") + ))] + #[test] + fn test_add_intervals_lower_affected_f32() { + // Lower is affected + let lower = f32::from_bits(1073741887); //1000000000000000000000000111111 + let upper = f32::from_bits(1098907651); //1000001100000000000000000000011 + capture_mode_change_f32((lower, upper), true, false); + + // Upper is affected + let lower = f32::from_bits(1072693248); //111111111100000000000000000000 + let upper = f32::from_bits(715827883); //101010101010101010101010101011 + capture_mode_change_f32((lower, upper), false, true); + + // Lower is affected + let lower = 1.0; // 0x3FF0000000000000 + let upper = 0.3; // 0x3FD3333333333333 + capture_mode_change_f64((lower, upper), true, false); + + // Upper is affected + let lower = 1.4999999999999998; // 0x3FF7FFFFFFFFFFFF + let upper = 0.000_000_000_000_000_022_044_604_925_031_31; // 0x3C796A6B413BB21F + capture_mode_change_f64((lower, upper), false, true); + } + + #[cfg(any( + not(any(target_arch = "x86_64", target_arch = "aarch64")), + target_os = "windows" + ))] + #[test] + fn test_next_impl_add_intervals_f64() { + let lower = 1.5; + let upper = 1.5; + capture_mode_change_f64((lower, upper), true, true); + + let lower = 1.5; + let upper = 1.5; + capture_mode_change_f32((lower, upper), true, true); + } +} diff --git a/datafusion/expr/src/lib.rs b/datafusion/expr/src/lib.rs index 21c0d750a36d..b9976f90c547 100644 --- a/datafusion/expr/src/lib.rs +++ b/datafusion/expr/src/lib.rs @@ -26,10 +26,20 @@ //! The [expr_fn] module contains functions for creating expressions. mod accumulator; -pub mod aggregate_function; -pub mod array_expressions; mod built_in_function; mod columnar_value; +mod literal; +mod nullif; +mod operator; +mod partition_evaluator; +mod signature; +mod table_source; +mod udaf; +mod udf; +mod udwf; + +pub mod aggregate_function; +pub mod array_expressions; pub mod conditional_expressions; pub mod expr; pub mod expr_fn; @@ -37,19 +47,11 @@ pub mod expr_rewriter; pub mod expr_schema; pub mod field_util; pub mod function; -mod literal; +pub mod interval_arithmetic; pub mod logical_plan; -mod nullif; -mod operator; -mod partition_evaluator; -mod signature; pub mod struct_expressions; -mod table_source; pub mod tree_node; pub mod type_coercion; -mod udaf; -mod udf; -mod udwf; pub mod utils; pub mod window_frame; pub mod window_function; diff --git a/datafusion/expr/src/type_coercion/binary.rs b/datafusion/expr/src/type_coercion/binary.rs index cf93d15e23f0..9ccddbfce068 100644 --- a/datafusion/expr/src/type_coercion/binary.rs +++ b/datafusion/expr/src/type_coercion/binary.rs @@ -782,7 +782,6 @@ fn temporal_coercion(lhs_type: &DataType, rhs_type: &DataType) -> Option Some(Interval(MonthDayNano)), (Date64, Date32) | (Date32, Date64) => Some(Date64), (Timestamp(_, None), Date32) | (Date32, Timestamp(_, None)) => { diff --git a/datafusion/optimizer/src/simplify_expressions/expr_simplifier.rs b/datafusion/optimizer/src/simplify_expressions/expr_simplifier.rs index 947a6f6070d2..ad64625f7f77 100644 --- a/datafusion/optimizer/src/simplify_expressions/expr_simplifier.rs +++ b/datafusion/optimizer/src/simplify_expressions/expr_simplifier.rs @@ -21,9 +21,11 @@ use std::ops::Not; use super::or_in_list_simplifier::OrInListSimplifier; use super::utils::*; - use crate::analyzer::type_coercion::TypeCoercionRewriter; +use crate::simplify_expressions::guarantees::GuaranteeRewriter; use crate::simplify_expressions::regex::simplify_regex_expr; +use crate::simplify_expressions::SimplifyInfo; + use arrow::{ array::new_null_array, datatypes::{DataType, Field, Schema}, @@ -37,18 +39,15 @@ use datafusion_common::{ use datafusion_common::{ exec_err, internal_err, DFSchema, DFSchemaRef, DataFusionError, Result, ScalarValue, }; -use datafusion_expr::expr::{InList, InSubquery, ScalarFunction}; use datafusion_expr::{ and, expr, lit, or, BinaryExpr, BuiltinScalarFunction, Case, ColumnarValue, Expr, Like, Volatility, }; -use datafusion_physical_expr::{ - create_physical_expr, execution_props::ExecutionProps, intervals::NullableInterval, +use datafusion_expr::{ + expr::{InList, InSubquery, ScalarFunction}, + interval_arithmetic::NullableInterval, }; - -use crate::simplify_expressions::SimplifyInfo; - -use crate::simplify_expressions::guarantees::GuaranteeRewriter; +use datafusion_physical_expr::{create_physical_expr, execution_props::ExecutionProps}; /// This structure handles API for expression simplification pub struct ExprSimplifier { @@ -178,9 +177,9 @@ impl ExprSimplifier { /// ```rust /// use arrow::datatypes::{DataType, Field, Schema}; /// use datafusion_expr::{col, lit, Expr}; + /// use datafusion_expr::interval_arithmetic::{Interval, NullableInterval}; /// use datafusion_common::{Result, ScalarValue, ToDFSchema}; /// use datafusion_physical_expr::execution_props::ExecutionProps; - /// use datafusion_physical_expr::intervals::{Interval, NullableInterval}; /// use datafusion_optimizer::simplify_expressions::{ /// ExprSimplifier, SimplifyContext}; /// @@ -207,7 +206,7 @@ impl ExprSimplifier { /// ( /// col("x"), /// NullableInterval::NotNull { - /// values: Interval::make(Some(3_i64), Some(5_i64), (false, false)), + /// values: Interval::make(Some(3_i64), Some(5_i64)).unwrap() /// } /// ), /// // y = 3 @@ -1300,26 +1299,25 @@ mod tests { sync::Arc, }; + use super::*; use crate::simplify_expressions::{ utils::for_test::{cast_to_int64_expr, now_expr, to_timestamp_expr}, SimplifyContext, }; - - use super::*; use crate::test::test_table_scan_with_name; + use arrow::{ array::{ArrayRef, Int32Array}, datatypes::{DataType, Field, Schema}, }; - use chrono::{DateTime, TimeZone, Utc}; use datafusion_common::{assert_contains, cast::as_int32_array, DFField, ToDFSchema}; - use datafusion_expr::*; + use datafusion_expr::{interval_arithmetic::Interval, *}; use datafusion_physical_expr::{ - execution_props::ExecutionProps, - functions::make_scalar_function, - intervals::{Interval, NullableInterval}, + execution_props::ExecutionProps, functions::make_scalar_function, }; + use chrono::{DateTime, TimeZone, Utc}; + // ------------------------------ // --- ExprSimplifier tests ----- // ------------------------------ @@ -3281,7 +3279,7 @@ mod tests { ( col("c3"), NullableInterval::NotNull { - values: Interval::make(Some(0_i64), Some(2_i64), (false, false)), + values: Interval::make(Some(0_i64), Some(2_i64)).unwrap(), }, ), ( @@ -3301,19 +3299,23 @@ mod tests { ( col("c3"), NullableInterval::MaybeNull { - values: Interval::make(Some(0_i64), Some(2_i64), (false, false)), + values: Interval::make(Some(0_i64), Some(2_i64)).unwrap(), }, ), ( col("c4"), NullableInterval::MaybeNull { - values: Interval::make(Some(9_u32), Some(9_u32), (false, false)), + values: Interval::make(Some(9_u32), Some(9_u32)).unwrap(), }, ), ( col("c1"), NullableInterval::NotNull { - values: Interval::make(Some("d"), Some("f"), (false, false)), + values: Interval::try_new( + ScalarValue::Utf8(Some("d".to_string())), + ScalarValue::Utf8(Some("f".to_string())), + ) + .unwrap(), }, ), ]; diff --git a/datafusion/optimizer/src/simplify_expressions/guarantees.rs b/datafusion/optimizer/src/simplify_expressions/guarantees.rs index 0204698571b4..3cfaae858e2d 100644 --- a/datafusion/optimizer/src/simplify_expressions/guarantees.rs +++ b/datafusion/optimizer/src/simplify_expressions/guarantees.rs @@ -18,11 +18,12 @@ //! Simplifier implementation for [`ExprSimplifier::with_guarantees()`] //! //! [`ExprSimplifier::with_guarantees()`]: crate::simplify_expressions::expr_simplifier::ExprSimplifier::with_guarantees -use datafusion_common::{tree_node::TreeNodeRewriter, DataFusionError, Result}; -use datafusion_expr::{expr::InList, lit, Between, BinaryExpr, Expr}; + use std::{borrow::Cow, collections::HashMap}; -use datafusion_physical_expr::intervals::{Interval, IntervalBound, NullableInterval}; +use datafusion_common::{tree_node::TreeNodeRewriter, DataFusionError, Result}; +use datafusion_expr::interval_arithmetic::{Interval, NullableInterval}; +use datafusion_expr::{expr::InList, lit, Between, BinaryExpr, Expr}; /// Rewrite expressions to incorporate guarantees. /// @@ -82,10 +83,7 @@ impl<'a> TreeNodeRewriter for GuaranteeRewriter<'a> { high.as_ref(), ) { let expr_interval = NullableInterval::NotNull { - values: Interval::new( - IntervalBound::new(low.clone(), false), - IntervalBound::new(high.clone(), false), - ), + values: Interval::try_new(low.clone(), high.clone())?, }; let contains = expr_interval.contains(*interval)?; @@ -146,12 +144,8 @@ impl<'a> TreeNodeRewriter for GuaranteeRewriter<'a> { // Columns (if interval is collapsed to a single value) Expr::Column(_) => { - if let Some(col_interval) = self.guarantees.get(&expr) { - if let Some(value) = col_interval.single_value() { - Ok(lit(value)) - } else { - Ok(expr) - } + if let Some(interval) = self.guarantees.get(&expr) { + Ok(interval.single_value().map_or(expr, lit)) } else { Ok(expr) } @@ -215,7 +209,7 @@ mod tests { ( col("x"), NullableInterval::NotNull { - values: Default::default(), + values: Interval::make_unbounded(&DataType::Boolean).unwrap(), }, ), ]; @@ -262,18 +256,18 @@ mod tests { #[test] fn test_inequalities_non_null_bounded() { let guarantees = vec![ - // x ∈ (1, 3] (not null) + // x ∈ [1, 3] (not null) ( col("x"), NullableInterval::NotNull { - values: Interval::make(Some(1_i32), Some(3_i32), (true, false)), + values: Interval::make(Some(1_i32), Some(3_i32)).unwrap(), }, ), - // s.y ∈ (1, 3] (not null) + // s.y ∈ [1, 3] (not null) ( col("s").field("y"), NullableInterval::NotNull { - values: Interval::make(Some(1_i32), Some(3_i32), (true, false)), + values: Interval::make(Some(1_i32), Some(3_i32)).unwrap(), }, ), ]; @@ -282,18 +276,16 @@ mod tests { // (original_expr, expected_simplification) let simplified_cases = &[ - (col("x").lt_eq(lit(1)), false), - (col("s").field("y").lt_eq(lit(1)), false), + (col("x").lt(lit(0)), false), + (col("s").field("y").lt(lit(0)), false), (col("x").lt_eq(lit(3)), true), (col("x").gt(lit(3)), false), - (col("x").gt(lit(1)), true), + (col("x").gt(lit(0)), true), (col("x").eq(lit(0)), false), (col("x").not_eq(lit(0)), true), - (col("x").between(lit(2), lit(5)), true), - (col("x").between(lit(2), lit(3)), true), + (col("x").between(lit(0), lit(5)), true), (col("x").between(lit(5), lit(10)), false), - (col("x").not_between(lit(2), lit(5)), false), - (col("x").not_between(lit(2), lit(3)), false), + (col("x").not_between(lit(0), lit(5)), false), (col("x").not_between(lit(5), lit(10)), true), ( Expr::BinaryExpr(BinaryExpr { @@ -334,10 +326,11 @@ mod tests { ( col("x"), NullableInterval::NotNull { - values: Interval::new( - IntervalBound::new(ScalarValue::Date32(Some(18628)), false), - IntervalBound::make_unbounded(DataType::Date32).unwrap(), - ), + values: Interval::try_new( + ScalarValue::Date32(Some(18628)), + ScalarValue::Date32(None), + ) + .unwrap(), }, ), ]; @@ -412,7 +405,11 @@ mod tests { ( col("x"), NullableInterval::MaybeNull { - values: Interval::make(Some("abc"), Some("def"), (true, false)), + values: Interval::try_new( + ScalarValue::Utf8(Some("abc".to_string())), + ScalarValue::Utf8(Some("def".to_string())), + ) + .unwrap(), }, ), ]; @@ -485,11 +482,15 @@ mod tests { #[test] fn test_in_list() { let guarantees = vec![ - // x ∈ [1, 10) (not null) + // x ∈ [1, 10] (not null) ( col("x"), NullableInterval::NotNull { - values: Interval::make(Some(1_i32), Some(10_i32), (false, true)), + values: Interval::try_new( + ScalarValue::Int32(Some(1)), + ScalarValue::Int32(Some(10)), + ) + .unwrap(), }, ), ]; @@ -501,8 +502,8 @@ mod tests { let cases = &[ // x IN (9, 11) => x IN (9) ("x", vec![9, 11], false, vec![9]), - // x IN (10, 2) => x IN (2) - ("x", vec![10, 2], false, vec![2]), + // x IN (10, 2) => x IN (10, 2) + ("x", vec![10, 2], false, vec![10, 2]), // x NOT IN (9, 11) => x NOT IN (9) ("x", vec![9, 11], true, vec![9]), // x NOT IN (0, 22) => x NOT IN () diff --git a/datafusion/physical-expr/Cargo.toml b/datafusion/physical-expr/Cargo.toml index caa812d0751c..d237c68657a1 100644 --- a/datafusion/physical-expr/Cargo.toml +++ b/datafusion/physical-expr/Cargo.toml @@ -34,13 +34,16 @@ path = "src/lib.rs" [features] crypto_expressions = ["md-5", "sha2", "blake2", "blake3"] -default = ["crypto_expressions", "regex_expressions", "unicode_expressions", "encoding_expressions"] +default = ["crypto_expressions", "regex_expressions", "unicode_expressions", "encoding_expressions", +] encoding_expressions = ["base64", "hex"] regex_expressions = ["regex"] unicode_expressions = ["unicode-segmentation"] [dependencies] -ahash = { version = "0.8", default-features = false, features = ["runtime-rng"] } +ahash = { version = "0.8", default-features = false, features = [ + "runtime-rng", +] } arrow = { workspace = true } arrow-array = { workspace = true } arrow-buffer = { workspace = true } @@ -57,7 +60,6 @@ hashbrown = { version = "0.14", features = ["raw"] } hex = { version = "0.4", optional = true } indexmap = { workspace = true } itertools = { version = "0.12", features = ["use_std"] } -libc = "0.2.140" log = { workspace = true } md-5 = { version = "^0.10.0", optional = true } paste = "^1.0" diff --git a/datafusion/physical-expr/src/analysis.rs b/datafusion/physical-expr/src/analysis.rs index 93c24014fd3e..dc12bdf46acd 100644 --- a/datafusion/physical-expr/src/analysis.rs +++ b/datafusion/physical-expr/src/analysis.rs @@ -21,8 +21,7 @@ use std::fmt::Debug; use std::sync::Arc; use crate::expressions::Column; -use crate::intervals::cp_solver::PropagationResult; -use crate::intervals::{cardinality_ratio, ExprIntervalGraph, Interval, IntervalBound}; +use crate::intervals::cp_solver::{ExprIntervalGraph, PropagationResult}; use crate::utils::collect_columns; use crate::PhysicalExpr; @@ -31,6 +30,7 @@ use datafusion_common::stats::Precision; use datafusion_common::{ internal_err, ColumnStatistics, DataFusionError, Result, ScalarValue, }; +use datafusion_expr::interval_arithmetic::{cardinality_ratio, Interval}; /// The shared context used during the analysis of an expression. Includes /// the boundaries for all known columns. @@ -92,22 +92,18 @@ impl ExprBoundaries { ) -> Result { let field = &schema.fields()[col_index]; let empty_field = ScalarValue::try_from(field.data_type())?; - let interval = Interval::new( - IntervalBound::new_closed( - col_stats - .min_value - .get_value() - .cloned() - .unwrap_or(empty_field.clone()), - ), - IntervalBound::new_closed( - col_stats - .max_value - .get_value() - .cloned() - .unwrap_or(empty_field), - ), - ); + let interval = Interval::try_new( + col_stats + .min_value + .get_value() + .cloned() + .unwrap_or(empty_field.clone()), + col_stats + .max_value + .get_value() + .cloned() + .unwrap_or(empty_field), + )?; let column = Column::new(field.name(), col_index); Ok(ExprBoundaries { column, @@ -135,47 +131,44 @@ impl ExprBoundaries { pub fn analyze( expr: &Arc, context: AnalysisContext, + schema: &Schema, ) -> Result { let target_boundaries = context.boundaries; - let mut graph = ExprIntervalGraph::try_new(expr.clone())?; + let mut graph = ExprIntervalGraph::try_new(expr.clone(), schema)?; - let columns: Vec> = collect_columns(expr) + let columns = collect_columns(expr) .into_iter() - .map(|c| Arc::new(c) as Arc) - .collect(); - - let target_expr_and_indices: Vec<(Arc, usize)> = - graph.gather_node_indices(columns.as_slice()); - - let mut target_indices_and_boundaries: Vec<(usize, Interval)> = - target_expr_and_indices - .iter() - .filter_map(|(expr, i)| { - target_boundaries.iter().find_map(|bound| { - expr.as_any() - .downcast_ref::() - .filter(|expr_column| bound.column.eq(*expr_column)) - .map(|_| (*i, bound.interval.clone())) - }) + .map(|c| Arc::new(c) as _) + .collect::>(); + + let target_expr_and_indices = graph.gather_node_indices(columns.as_slice()); + + let mut target_indices_and_boundaries = target_expr_and_indices + .iter() + .filter_map(|(expr, i)| { + target_boundaries.iter().find_map(|bound| { + expr.as_any() + .downcast_ref::() + .filter(|expr_column| bound.column.eq(*expr_column)) + .map(|_| (*i, bound.interval.clone())) }) - .collect(); - Ok( - match graph.update_ranges(&mut target_indices_and_boundaries)? { - PropagationResult::Success => shrink_boundaries( - expr, - graph, - target_boundaries, - target_expr_and_indices, - )?, - PropagationResult::Infeasible => { - AnalysisContext::new(target_boundaries).with_selectivity(0.0) - } - PropagationResult::CannotPropagate => { - AnalysisContext::new(target_boundaries).with_selectivity(1.0) - } - }, - ) + }) + .collect::>(); + + match graph + .update_ranges(&mut target_indices_and_boundaries, Interval::CERTAINLY_TRUE)? + { + PropagationResult::Success => { + shrink_boundaries(graph, target_boundaries, target_expr_and_indices) + } + PropagationResult::Infeasible => { + Ok(AnalysisContext::new(target_boundaries).with_selectivity(0.0)) + } + PropagationResult::CannotPropagate => { + Ok(AnalysisContext::new(target_boundaries).with_selectivity(1.0)) + } + } } /// If the `PropagationResult` indicates success, this function calculates the @@ -183,8 +176,7 @@ pub fn analyze( /// Following this, it constructs and returns a new `AnalysisContext` with the /// updated parameters. fn shrink_boundaries( - expr: &Arc, - mut graph: ExprIntervalGraph, + graph: ExprIntervalGraph, mut target_boundaries: Vec, target_expr_and_indices: Vec<(Arc, usize)>, ) -> Result { @@ -199,20 +191,12 @@ fn shrink_boundaries( }; } }); - let graph_nodes = graph.gather_node_indices(&[expr.clone()]); - let Some((_, root_index)) = graph_nodes.get(0) else { - return internal_err!( - "The ExprIntervalGraph under investigation does not have any nodes." - ); - }; - let final_result = graph.get_interval(*root_index); - - let selectivity = calculate_selectivity( - &final_result.lower.value, - &final_result.upper.value, - &target_boundaries, - &initial_boundaries, - )?; + + let selectivity = calculate_selectivity(&target_boundaries, &initial_boundaries); + + if !(0.0..=1.0).contains(&selectivity) { + return internal_err!("Selectivity is out of limit: {}", selectivity); + } Ok(AnalysisContext::new(target_boundaries).with_selectivity(selectivity)) } @@ -220,33 +204,17 @@ fn shrink_boundaries( /// This function calculates the filter predicate's selectivity by comparing /// the initial and pruned column boundaries. Selectivity is defined as the /// ratio of rows in a table that satisfy the filter's predicate. -/// -/// An exact propagation result at the root, i.e. `[true, true]` or `[false, false]`, -/// leads to early exit (returning a selectivity value of either 1.0 or 0.0). In such -/// a case, `[true, true]` indicates that all data values satisfy the predicate (hence, -/// selectivity is 1.0), and `[false, false]` suggests that no data value meets the -/// predicate (therefore, selectivity is 0.0). fn calculate_selectivity( - lower_value: &ScalarValue, - upper_value: &ScalarValue, target_boundaries: &[ExprBoundaries], initial_boundaries: &[ExprBoundaries], -) -> Result { - match (lower_value, upper_value) { - (ScalarValue::Boolean(Some(true)), ScalarValue::Boolean(Some(true))) => Ok(1.0), - (ScalarValue::Boolean(Some(false)), ScalarValue::Boolean(Some(false))) => Ok(0.0), - _ => { - // Since the intervals are assumed uniform and the values - // are not correlated, we need to multiply the selectivities - // of multiple columns to get the overall selectivity. - target_boundaries.iter().enumerate().try_fold( - 1.0, - |acc, (i, ExprBoundaries { interval, .. })| { - let temp = - cardinality_ratio(&initial_boundaries[i].interval, interval)?; - Ok(acc * temp) - }, - ) - } - } +) -> f64 { + // Since the intervals are assumed uniform and the values + // are not correlated, we need to multiply the selectivities + // of multiple columns to get the overall selectivity. + initial_boundaries + .iter() + .zip(target_boundaries.iter()) + .fold(1.0, |acc, (initial, target)| { + acc * cardinality_ratio(&initial.interval, &target.interval) + }) } diff --git a/datafusion/physical-expr/src/expressions/binary.rs b/datafusion/physical-expr/src/expressions/binary.rs index 0a05a479e5a7..9c7fdd2e814b 100644 --- a/datafusion/physical-expr/src/expressions/binary.rs +++ b/datafusion/physical-expr/src/expressions/binary.rs @@ -23,8 +23,8 @@ use std::{any::Any, sync::Arc}; use crate::array_expressions::{ array_append, array_concat, array_has_all, array_prepend, }; +use crate::expressions::datum::{apply, apply_cmp}; use crate::intervals::cp_solver::{propagate_arithmetic, propagate_comparison}; -use crate::intervals::{apply_operator, Interval}; use crate::physical_expr::down_cast_any_ref; use crate::sort_properties::SortProperties; use crate::PhysicalExpr; @@ -38,12 +38,13 @@ use arrow::compute::kernels::comparison::regexp_is_match_utf8_scalar; use arrow::compute::kernels::concat_elements::concat_elements_utf8; use arrow::datatypes::*; use arrow::record_batch::RecordBatch; + use datafusion_common::cast::as_boolean_array; use datafusion_common::{internal_err, DataFusionError, Result, ScalarValue}; +use datafusion_expr::interval_arithmetic::{apply_operator, Interval}; use datafusion_expr::type_coercion::binary::get_result_type; use datafusion_expr::{ColumnarValue, Operator}; -use crate::expressions::datum::{apply, apply_cmp}; use kernels::{ bitwise_and_dyn, bitwise_and_dyn_scalar, bitwise_or_dyn, bitwise_or_dyn_scalar, bitwise_shift_left_dyn, bitwise_shift_left_dyn_scalar, bitwise_shift_right_dyn, @@ -338,32 +339,102 @@ impl PhysicalExpr for BinaryExpr { &self, interval: &Interval, children: &[&Interval], - ) -> Result>> { + ) -> Result>> { // Get children intervals. let left_interval = children[0]; let right_interval = children[1]; - let (left, right) = if self.op.is_logic_operator() { - // TODO: Currently, this implementation only supports the AND operator - // and does not require any further propagation. In the future, - // upon adding support for additional logical operators, this - // method will require modification to support propagating the - // changes accordingly. - return Ok(vec![]); - } else if self.op.is_comparison_operator() { - if interval == &Interval::CERTAINLY_FALSE { - // TODO: We will handle strictly false clauses by negating - // the comparison operator (e.g. GT to LE, LT to GE) - // once open/closed intervals are supported. - return Ok(vec![]); + if self.op.eq(&Operator::And) { + if interval.eq(&Interval::CERTAINLY_TRUE) { + // A certainly true logical conjunction can only derive from possibly + // true operands. Otherwise, we prove infeasability. + Ok((!left_interval.eq(&Interval::CERTAINLY_FALSE) + && !right_interval.eq(&Interval::CERTAINLY_FALSE)) + .then(|| vec![Interval::CERTAINLY_TRUE, Interval::CERTAINLY_TRUE])) + } else if interval.eq(&Interval::CERTAINLY_FALSE) { + // If the logical conjunction is certainly false, one of the + // operands must be false. However, it's not always possible to + // determine which operand is false, leading to different scenarios. + + // If one operand is certainly true and the other one is uncertain, + // then the latter must be certainly false. + if left_interval.eq(&Interval::CERTAINLY_TRUE) + && right_interval.eq(&Interval::UNCERTAIN) + { + Ok(Some(vec![ + Interval::CERTAINLY_TRUE, + Interval::CERTAINLY_FALSE, + ])) + } else if right_interval.eq(&Interval::CERTAINLY_TRUE) + && left_interval.eq(&Interval::UNCERTAIN) + { + Ok(Some(vec![ + Interval::CERTAINLY_FALSE, + Interval::CERTAINLY_TRUE, + ])) + } + // If both children are uncertain, or if one is certainly false, + // we cannot conclusively refine their intervals. In this case, + // propagation does not result in any interval changes. + else { + Ok(Some(vec![])) + } + } else { + // An uncertain logical conjunction result can not shrink the + // end-points of its children. + Ok(Some(vec![])) + } + } else if self.op.eq(&Operator::Or) { + if interval.eq(&Interval::CERTAINLY_FALSE) { + // A certainly false logical conjunction can only derive from certainly + // false operands. Otherwise, we prove infeasability. + Ok((!left_interval.eq(&Interval::CERTAINLY_TRUE) + && !right_interval.eq(&Interval::CERTAINLY_TRUE)) + .then(|| vec![Interval::CERTAINLY_FALSE, Interval::CERTAINLY_FALSE])) + } else if interval.eq(&Interval::CERTAINLY_TRUE) { + // If the logical disjunction is certainly true, one of the + // operands must be true. However, it's not always possible to + // determine which operand is true, leading to different scenarios. + + // If one operand is certainly false and the other one is uncertain, + // then the latter must be certainly true. + if left_interval.eq(&Interval::CERTAINLY_FALSE) + && right_interval.eq(&Interval::UNCERTAIN) + { + Ok(Some(vec![ + Interval::CERTAINLY_FALSE, + Interval::CERTAINLY_TRUE, + ])) + } else if right_interval.eq(&Interval::CERTAINLY_FALSE) + && left_interval.eq(&Interval::UNCERTAIN) + { + Ok(Some(vec![ + Interval::CERTAINLY_TRUE, + Interval::CERTAINLY_FALSE, + ])) + } + // If both children are uncertain, or if one is certainly true, + // we cannot conclusively refine their intervals. In this case, + // propagation does not result in any interval changes. + else { + Ok(Some(vec![])) + } + } else { + // An uncertain logical disjunction result can not shrink the + // end-points of its children. + Ok(Some(vec![])) } - // Propagate the comparison operator. - propagate_comparison(&self.op, left_interval, right_interval)? + } else if self.op.is_comparison_operator() { + Ok( + propagate_comparison(&self.op, interval, left_interval, right_interval)? + .map(|(left, right)| vec![left, right]), + ) } else { - // Propagate the arithmetic operator. - propagate_arithmetic(&self.op, interval, left_interval, right_interval)? - }; - Ok(vec![left, right]) + Ok( + propagate_arithmetic(&self.op, interval, left_interval, right_interval)? + .map(|(left, right)| vec![left, right]), + ) + } } fn dyn_hash(&self, state: &mut dyn Hasher) { @@ -380,7 +451,7 @@ impl PhysicalExpr for BinaryExpr { Operator::Minus => left_child.sub(right_child), Operator::Gt | Operator::GtEq => left_child.gt_or_gteq(right_child), Operator::Lt | Operator::LtEq => right_child.gt_or_gteq(left_child), - Operator::And => left_child.and(right_child), + Operator::And | Operator::Or => left_child.and_or(right_child), _ => SortProperties::Unordered, } } diff --git a/datafusion/physical-expr/src/expressions/cast.rs b/datafusion/physical-expr/src/expressions/cast.rs index cbc82cc77628..b718b5017c5e 100644 --- a/datafusion/physical-expr/src/expressions/cast.rs +++ b/datafusion/physical-expr/src/expressions/cast.rs @@ -20,18 +20,16 @@ use std::fmt; use std::hash::{Hash, Hasher}; use std::sync::Arc; -use crate::intervals::Interval; use crate::physical_expr::down_cast_any_ref; use crate::sort_properties::SortProperties; use crate::PhysicalExpr; -use arrow::compute; -use arrow::compute::{kernels, CastOptions}; +use arrow::compute::{can_cast_types, kernels, CastOptions}; use arrow::datatypes::{DataType, Schema}; use arrow::record_batch::RecordBatch; -use compute::can_cast_types; use datafusion_common::format::DEFAULT_FORMAT_OPTIONS; use datafusion_common::{not_impl_err, DataFusionError, Result, ScalarValue}; +use datafusion_expr::interval_arithmetic::Interval; use datafusion_expr::ColumnarValue; const DEFAULT_CAST_OPTIONS: CastOptions<'static> = CastOptions { @@ -129,13 +127,13 @@ impl PhysicalExpr for CastExpr { &self, interval: &Interval, children: &[&Interval], - ) -> Result>> { + ) -> Result>> { let child_interval = children[0]; // Get child's datatype: - let cast_type = child_interval.get_datatype()?; - Ok(vec![Some( - interval.cast_to(&cast_type, &self.cast_options)?, - )]) + let cast_type = child_interval.data_type(); + Ok(Some( + vec![interval.cast_to(&cast_type, &self.cast_options)?], + )) } fn dyn_hash(&self, state: &mut dyn Hasher) { @@ -226,6 +224,7 @@ pub fn cast( mod tests { use super::*; use crate::expressions::col; + use arrow::{ array::{ Array, Decimal128Array, Float32Array, Float64Array, Int16Array, Int32Array, @@ -234,6 +233,7 @@ mod tests { }, datatypes::*, }; + use datafusion_common::Result; // runs an end-to-end test of physical type cast diff --git a/datafusion/physical-expr/src/expressions/negative.rs b/datafusion/physical-expr/src/expressions/negative.rs index 65b347941163..a59fd1ae3f20 100644 --- a/datafusion/physical-expr/src/expressions/negative.rs +++ b/datafusion/physical-expr/src/expressions/negative.rs @@ -17,25 +17,26 @@ //! Negation (-) expression -use crate::intervals::Interval; +use std::any::Any; +use std::hash::{Hash, Hasher}; +use std::sync::Arc; + use crate::physical_expr::down_cast_any_ref; use crate::sort_properties::SortProperties; use crate::PhysicalExpr; + use arrow::{ compute::kernels::numeric::neg_wrapping, datatypes::{DataType, Schema}, record_batch::RecordBatch, }; use datafusion_common::{internal_err, DataFusionError, Result}; +use datafusion_expr::interval_arithmetic::Interval; use datafusion_expr::{ type_coercion::{is_interval, is_null, is_signed_numeric}, ColumnarValue, }; -use std::any::Any; -use std::hash::{Hash, Hasher}; -use std::sync::Arc; - /// Negative expression #[derive(Debug, Hash)] pub struct NegativeExpr { @@ -108,10 +109,10 @@ impl PhysicalExpr for NegativeExpr { /// It replaces the upper and lower bounds after multiplying them with -1. /// Ex: `(a, b]` => `[-b, -a)` fn evaluate_bounds(&self, children: &[&Interval]) -> Result { - Ok(Interval::new( - children[0].upper.negate()?, - children[0].lower.negate()?, - )) + Interval::try_new( + children[0].upper().arithmetic_negate()?, + children[0].lower().arithmetic_negate()?, + ) } /// Returns a new [`Interval`] of a NegativeExpr that has the existing `interval` given that @@ -120,12 +121,16 @@ impl PhysicalExpr for NegativeExpr { &self, interval: &Interval, children: &[&Interval], - ) -> Result>> { + ) -> Result>> { let child_interval = children[0]; - let negated_interval = - Interval::new(interval.upper.negate()?, interval.lower.negate()?); + let negated_interval = Interval::try_new( + interval.upper().arithmetic_negate()?, + interval.lower().arithmetic_negate()?, + )?; - Ok(vec![child_interval.intersect(negated_interval)?]) + Ok(child_interval + .intersect(negated_interval)? + .map(|result| vec![result])) } /// The ordering of a [`NegativeExpr`] is simply the reverse of its child. @@ -167,14 +172,14 @@ pub fn negative( #[cfg(test)] mod tests { use super::*; - use crate::{ - expressions::{col, Column}, - intervals::Interval, - }; + use crate::expressions::{col, Column}; + use arrow::array::*; use arrow::datatypes::*; use arrow_schema::DataType::{Float32, Float64, Int16, Int32, Int64, Int8}; - use datafusion_common::{cast::as_primitive_array, Result}; + use datafusion_common::cast::as_primitive_array; + use datafusion_common::Result; + use paste::paste; macro_rules! test_array_negative_op { @@ -218,8 +223,8 @@ mod tests { let negative_expr = NegativeExpr { arg: Arc::new(Column::new("a", 0)), }; - let child_interval = Interval::make(Some(-2), Some(1), (true, false)); - let negative_expr_interval = Interval::make(Some(-1), Some(2), (false, true)); + let child_interval = Interval::make(Some(-2), Some(1))?; + let negative_expr_interval = Interval::make(Some(-1), Some(2))?; assert_eq!( negative_expr.evaluate_bounds(&[&child_interval])?, negative_expr_interval @@ -232,10 +237,9 @@ mod tests { let negative_expr = NegativeExpr { arg: Arc::new(Column::new("a", 0)), }; - let original_child_interval = Interval::make(Some(-2), Some(3), (false, false)); - let negative_expr_interval = Interval::make(Some(0), Some(4), (true, false)); - let after_propagation = - vec![Some(Interval::make(Some(-2), Some(0), (false, true)))]; + let original_child_interval = Interval::make(Some(-2), Some(3))?; + let negative_expr_interval = Interval::make(Some(0), Some(4))?; + let after_propagation = Some(vec![Interval::make(Some(-2), Some(0))?]); assert_eq!( negative_expr.propagate_constraints( &negative_expr_interval, diff --git a/datafusion/physical-expr/src/intervals/cp_solver.rs b/datafusion/physical-expr/src/intervals/cp_solver.rs index e7515341c52c..5064ad8d5c48 100644 --- a/datafusion/physical-expr/src/intervals/cp_solver.rs +++ b/datafusion/physical-expr/src/intervals/cp_solver.rs @@ -24,15 +24,13 @@ use std::sync::Arc; use super::utils::{ convert_duration_type_to_interval, convert_interval_type_to_duration, get_inverse_op, }; -use super::IntervalBound; use crate::expressions::Literal; -use crate::intervals::interval_aritmetic::{apply_operator, Interval}; use crate::utils::{build_dag, ExprTreeNode}; use crate::PhysicalExpr; -use arrow_schema::DataType; -use datafusion_common::{DataFusionError, Result, ScalarValue}; -use datafusion_expr::type_coercion::binary::get_result_type; +use arrow_schema::{DataType, Schema}; +use datafusion_common::{internal_err, DataFusionError, Result}; +use datafusion_expr::interval_arithmetic::{apply_operator, satisfy_greater, Interval}; use datafusion_expr::Operator; use petgraph::graph::NodeIndex; @@ -148,7 +146,7 @@ pub enum PropagationResult { } /// This is a node in the DAEG; it encapsulates a reference to the actual -/// [PhysicalExpr] as well as an interval containing expression bounds. +/// [`PhysicalExpr`] as well as an interval containing expression bounds. #[derive(Clone, Debug)] pub struct ExprIntervalGraphNode { expr: Arc, @@ -163,11 +161,9 @@ impl Display for ExprIntervalGraphNode { impl ExprIntervalGraphNode { /// Constructs a new DAEG node with an [-∞, ∞] range. - pub fn new(expr: Arc) -> Self { - ExprIntervalGraphNode { - expr, - interval: Interval::default(), - } + pub fn new_unbounded(expr: Arc, dt: &DataType) -> Result { + Interval::make_unbounded(dt) + .map(|interval| ExprIntervalGraphNode { expr, interval }) } /// Constructs a new DAEG node with the given range. @@ -180,26 +176,24 @@ impl ExprIntervalGraphNode { &self.interval } - /// This function creates a DAEG node from Datafusion's [ExprTreeNode] + /// This function creates a DAEG node from Datafusion's [`ExprTreeNode`] /// object. Literals are created with definite, singleton intervals while /// any other expression starts with an indefinite interval ([-∞, ∞]). - pub fn make_node(node: &ExprTreeNode) -> ExprIntervalGraphNode { + pub fn make_node(node: &ExprTreeNode, schema: &Schema) -> Result { let expr = node.expression().clone(); if let Some(literal) = expr.as_any().downcast_ref::() { let value = literal.value(); - let interval = Interval::new( - IntervalBound::new_closed(value.clone()), - IntervalBound::new_closed(value.clone()), - ); - ExprIntervalGraphNode::new_with_interval(expr, interval) + Interval::try_new(value.clone(), value.clone()) + .map(|interval| Self::new_with_interval(expr, interval)) } else { - ExprIntervalGraphNode::new(expr) + expr.data_type(schema) + .and_then(|dt| Self::new_unbounded(expr, &dt)) } } } impl PartialEq for ExprIntervalGraphNode { - fn eq(&self, other: &ExprIntervalGraphNode) -> bool { + fn eq(&self, other: &Self) -> bool { self.expr.eq(&other.expr) } } @@ -216,16 +210,23 @@ impl PartialEq for ExprIntervalGraphNode { /// - For minus operation, specifically, we would first do /// - [xL, xU] <- ([yL, yU] + [pL, pU]) ∩ [xL, xU], and then /// - [yL, yU] <- ([xL, xU] - [pL, pU]) ∩ [yL, yU]. +/// - For multiplication operation, specifically, we would first do +/// - [xL, xU] <- ([pL, pU] / [yL, yU]) ∩ [xL, xU], and then +/// - [yL, yU] <- ([pL, pU] / [xL, xU]) ∩ [yL, yU]. +/// - For division operation, specifically, we would first do +/// - [xL, xU] <- ([yL, yU] * [pL, pU]) ∩ [xL, xU], and then +/// - [yL, yU] <- ([xL, xU] / [pL, pU]) ∩ [yL, yU]. pub fn propagate_arithmetic( op: &Operator, parent: &Interval, left_child: &Interval, right_child: &Interval, -) -> Result<(Option, Option)> { - let inverse_op = get_inverse_op(*op); - match (left_child.get_datatype()?, right_child.get_datatype()?) { - // If we have a child whose type is a time interval (i.e. DataType::Interval), we need special handling - // since timestamp differencing results in a Duration type. +) -> Result> { + let inverse_op = get_inverse_op(*op)?; + match (left_child.data_type(), right_child.data_type()) { + // If we have a child whose type is a time interval (i.e. DataType::Interval), + // we need special handling since timestamp differencing results in a + // Duration type. (DataType::Timestamp(..), DataType::Interval(_)) => { propagate_time_interval_at_right( left_child, @@ -250,87 +251,109 @@ pub fn propagate_arithmetic( .intersect(left_child)? { // Left is feasible: - Some(value) => { + Some(value) => Ok( // Propagate to the right using the new left. - let right = - propagate_right(&value, parent, right_child, op, &inverse_op)?; - - // Return intervals for both children: - Ok((Some(value), right)) - } + propagate_right(&value, parent, right_child, op, &inverse_op)? + .map(|right| (value, right)), + ), // If the left child is infeasible, short-circuit. - None => Ok((None, None)), + None => Ok(None), } } } } -/// This function provides a target parent interval for comparison operators. -/// If we have expression > 0, expression must have the range (0, ∞). -/// If we have expression >= 0, expression must have the range [0, ∞). -/// If we have expression < 0, expression must have the range (-∞, 0). -/// If we have expression <= 0, expression must have the range (-∞, 0]. -fn comparison_operator_target( - left_datatype: &DataType, - op: &Operator, - right_datatype: &DataType, -) -> Result { - let datatype = get_result_type(left_datatype, &Operator::Minus, right_datatype)?; - let unbounded = IntervalBound::make_unbounded(&datatype)?; - let zero = ScalarValue::new_zero(&datatype)?; - Ok(match *op { - Operator::GtEq => Interval::new(IntervalBound::new_closed(zero), unbounded), - Operator::Gt => Interval::new(IntervalBound::new_open(zero), unbounded), - Operator::LtEq => Interval::new(unbounded, IntervalBound::new_closed(zero)), - Operator::Lt => Interval::new(unbounded, IntervalBound::new_open(zero)), - Operator::Eq => Interval::new( - IntervalBound::new_closed(zero.clone()), - IntervalBound::new_closed(zero), - ), - _ => unreachable!(), - }) -} - -/// This function propagates constraints arising from comparison operators. -/// The main idea is that we can analyze an inequality like x > y through the -/// equivalent inequality x - y > 0. Assuming that x and y has ranges [xL, xU] -/// and [yL, yU], we simply apply constraint propagation across [xL, xU], -/// [yL, yH] and [0, ∞]. Specifically, we would first do -/// - [xL, xU] <- ([yL, yU] + [0, ∞]) ∩ [xL, xU], and then -/// - [yL, yU] <- ([xL, xU] - [0, ∞]) ∩ [yL, yU]. +/// This function refines intervals `left_child` and `right_child` by applying +/// comparison propagation through `parent` via operation. The main idea is +/// that we can shrink ranges of variables x and y using parent interval p. +/// Two intervals can be ordered in 6 ways for a Gt `>` operator: +/// ```text +/// (1): Infeasible, short-circuit +/// left: | ================ | +/// right: | ======================== | +/// +/// (2): Update both interval +/// left: | ====================== | +/// right: | ====================== | +/// | +/// V +/// left: | ======= | +/// right: | ======= | +/// +/// (3): Update left interval +/// left: | ============================== | +/// right: | ========== | +/// | +/// V +/// left: | ===================== | +/// right: | ========== | +/// +/// (4): Update right interval +/// left: | ========== | +/// right: | =========================== | +/// | +/// V +/// left: | ========== | +/// right | ================== | +/// +/// (5): No change +/// left: | ============================ | +/// right: | =================== | +/// +/// (6): No change +/// left: | ==================== | +/// right: | =============== | +/// +/// -inf --------------------------------------------------------------- +inf +/// ``` pub fn propagate_comparison( op: &Operator, + parent: &Interval, left_child: &Interval, right_child: &Interval, -) -> Result<(Option, Option)> { - let left_type = left_child.get_datatype()?; - let right_type = right_child.get_datatype()?; - let parent = comparison_operator_target(&left_type, op, &right_type)?; - match (&left_type, &right_type) { - // We can not compare a Duration type with a time interval type - // without a reference timestamp unless the latter has a zero month field. - (DataType::Interval(_), DataType::Duration(_)) => { - propagate_comparison_to_time_interval_at_left( - left_child, - &parent, - right_child, - ) +) -> Result> { + if parent == &Interval::CERTAINLY_TRUE { + match op { + Operator::Eq => left_child.intersect(right_child).map(|result| { + result.map(|intersection| (intersection.clone(), intersection)) + }), + Operator::Gt => satisfy_greater(left_child, right_child, true), + Operator::GtEq => satisfy_greater(left_child, right_child, false), + Operator::Lt => satisfy_greater(right_child, left_child, true) + .map(|t| t.map(reverse_tuple)), + Operator::LtEq => satisfy_greater(right_child, left_child, false) + .map(|t| t.map(reverse_tuple)), + _ => internal_err!( + "The operator must be a comparison operator to propagate intervals" + ), } - (DataType::Duration(_), DataType::Interval(_)) => { - propagate_comparison_to_time_interval_at_left( - left_child, - &parent, - right_child, - ) + } else if parent == &Interval::CERTAINLY_FALSE { + match op { + Operator::Eq => { + // TODO: Propagation is not possible until we support interval sets. + Ok(None) + } + Operator::Gt => satisfy_greater(right_child, left_child, false), + Operator::GtEq => satisfy_greater(right_child, left_child, true), + Operator::Lt => satisfy_greater(left_child, right_child, false) + .map(|t| t.map(reverse_tuple)), + Operator::LtEq => satisfy_greater(left_child, right_child, true) + .map(|t| t.map(reverse_tuple)), + _ => internal_err!( + "The operator must be a comparison operator to propagate intervals" + ), } - _ => propagate_arithmetic(&Operator::Minus, &parent, left_child, right_child), + } else { + // Uncertainty cannot change any end-point of the intervals. + Ok(None) } } impl ExprIntervalGraph { - pub fn try_new(expr: Arc) -> Result { + pub fn try_new(expr: Arc, schema: &Schema) -> Result { // Build the full graph: - let (root, graph) = build_dag(expr, &ExprIntervalGraphNode::make_node)?; + let (root, graph) = + build_dag(expr, &|node| ExprIntervalGraphNode::make_node(node, schema))?; Ok(Self { graph, root }) } @@ -383,7 +406,7 @@ impl ExprIntervalGraph { // // ``` - /// This function associates stable node indices with [PhysicalExpr]s so + /// This function associates stable node indices with [`PhysicalExpr`]s so /// that we can match `Arc` and NodeIndex objects during /// membership tests. pub fn gather_node_indices( @@ -437,6 +460,33 @@ impl ExprIntervalGraph { nodes } + /// Updates intervals for all expressions in the DAEG by successive + /// bottom-up and top-down traversals. + pub fn update_ranges( + &mut self, + leaf_bounds: &mut [(usize, Interval)], + given_range: Interval, + ) -> Result { + self.assign_intervals(leaf_bounds); + let bounds = self.evaluate_bounds()?; + // There are three possible cases to consider: + // (1) given_range ⊇ bounds => Nothing to propagate + // (2) ∅ ⊂ (given_range ∩ bounds) ⊂ bounds => Can propagate + // (3) Disjoint sets => Infeasible + if given_range.contains(bounds)? == Interval::CERTAINLY_TRUE { + // First case: + Ok(PropagationResult::CannotPropagate) + } else if bounds.contains(&given_range)? != Interval::CERTAINLY_FALSE { + // Second case: + let result = self.propagate_constraints(given_range); + self.update_intervals(leaf_bounds); + result + } else { + // Third case: + Ok(PropagationResult::Infeasible) + } + } + /// This function assigns given ranges to expressions in the DAEG. /// The argument `assignments` associates indices of sought expressions /// with their corresponding new ranges. @@ -466,34 +516,43 @@ impl ExprIntervalGraph { /// # Examples /// /// ``` - /// use std::sync::Arc; - /// use datafusion_common::ScalarValue; - /// use datafusion_expr::Operator; - /// use datafusion_physical_expr::expressions::{BinaryExpr, Column, Literal}; - /// use datafusion_physical_expr::intervals::{Interval, IntervalBound, ExprIntervalGraph}; - /// use datafusion_physical_expr::PhysicalExpr; - /// let expr = Arc::new(BinaryExpr::new( - /// Arc::new(Column::new("gnz", 0)), - /// Operator::Plus, - /// Arc::new(Literal::new(ScalarValue::Int32(Some(10)))), - /// )); - /// let mut graph = ExprIntervalGraph::try_new(expr).unwrap(); - /// // Do it once, while constructing. - /// let node_indices = graph + /// use arrow::datatypes::DataType; + /// use arrow::datatypes::Field; + /// use arrow::datatypes::Schema; + /// use datafusion_common::ScalarValue; + /// use datafusion_expr::interval_arithmetic::Interval; + /// use datafusion_expr::Operator; + /// use datafusion_physical_expr::expressions::{BinaryExpr, Column, Literal}; + /// use datafusion_physical_expr::intervals::cp_solver::ExprIntervalGraph; + /// use datafusion_physical_expr::PhysicalExpr; + /// use std::sync::Arc; + /// + /// let expr = Arc::new(BinaryExpr::new( + /// Arc::new(Column::new("gnz", 0)), + /// Operator::Plus, + /// Arc::new(Literal::new(ScalarValue::Int32(Some(10)))), + /// )); + /// + /// let schema = Schema::new(vec![Field::new("gnz".to_string(), DataType::Int32, true)]); + /// + /// let mut graph = ExprIntervalGraph::try_new(expr, &schema).unwrap(); + /// // Do it once, while constructing. + /// let node_indices = graph /// .gather_node_indices(&[Arc::new(Column::new("gnz", 0))]); - /// let left_index = node_indices.get(0).unwrap().1; - /// // Provide intervals for leaf variables (here, there is only one). - /// let intervals = vec![( + /// let left_index = node_indices.get(0).unwrap().1; + /// + /// // Provide intervals for leaf variables (here, there is only one). + /// let intervals = vec![( /// left_index, - /// Interval::make(Some(10), Some(20), (true, true)), - /// )]; - /// // Evaluate bounds for the composite expression: - /// graph.assign_intervals(&intervals); - /// assert_eq!( - /// graph.evaluate_bounds().unwrap(), - /// &Interval::make(Some(20), Some(30), (true, true)), - /// ) + /// Interval::make(Some(10), Some(20)).unwrap(), + /// )]; /// + /// // Evaluate bounds for the composite expression: + /// graph.assign_intervals(&intervals); + /// assert_eq!( + /// graph.evaluate_bounds().unwrap(), + /// &Interval::make(Some(20), Some(30)).unwrap(), + /// ) /// ``` pub fn evaluate_bounds(&mut self) -> Result<&Interval> { let mut dfs = DfsPostOrder::new(&self.graph, self.root); @@ -505,7 +564,7 @@ impl ExprIntervalGraph { // If the current expression is a leaf, its interval should already // be set externally, just continue with the evaluation procedure: if !children_intervals.is_empty() { - // Reverse to align with [PhysicalExpr]'s children: + // Reverse to align with `PhysicalExpr`'s children: children_intervals.reverse(); self.graph[node].interval = self.graph[node].expr.evaluate_bounds(&children_intervals)?; @@ -516,8 +575,19 @@ impl ExprIntervalGraph { /// Updates/shrinks bounds for leaf expressions using interval arithmetic /// via a top-down traversal. - fn propagate_constraints(&mut self) -> Result { + fn propagate_constraints( + &mut self, + given_range: Interval, + ) -> Result { let mut bfs = Bfs::new(&self.graph, self.root); + + // Adjust the root node with the given range: + if let Some(interval) = self.graph[self.root].interval.intersect(given_range)? { + self.graph[self.root].interval = interval; + } else { + return Ok(PropagationResult::Infeasible); + } + while let Some(node) = bfs.next(&self.graph) { let neighbors = self.graph.neighbors_directed(node, Outgoing); let mut children = neighbors.collect::>(); @@ -526,7 +596,7 @@ impl ExprIntervalGraph { if children.is_empty() { continue; } - // Reverse to align with [PhysicalExpr]'s children: + // Reverse to align with `PhysicalExpr`'s children: children.reverse(); let children_intervals = children .iter() @@ -536,164 +606,132 @@ impl ExprIntervalGraph { let propagated_intervals = self.graph[node] .expr .propagate_constraints(node_interval, &children_intervals)?; - for (child, interval) in children.into_iter().zip(propagated_intervals) { - if let Some(interval) = interval { + if let Some(propagated_intervals) = propagated_intervals { + for (child, interval) in children.into_iter().zip(propagated_intervals) { self.graph[child].interval = interval; - } else { - // The constraint is infeasible, report: - return Ok(PropagationResult::Infeasible); } + } else { + // The constraint is infeasible, report: + return Ok(PropagationResult::Infeasible); } } Ok(PropagationResult::Success) } - /// Updates intervals for all expressions in the DAEG by successive - /// bottom-up and top-down traversals. - pub fn update_ranges( - &mut self, - leaf_bounds: &mut [(usize, Interval)], - ) -> Result { - self.assign_intervals(leaf_bounds); - let bounds = self.evaluate_bounds()?; - if bounds == &Interval::CERTAINLY_FALSE { - Ok(PropagationResult::Infeasible) - } else if bounds == &Interval::UNCERTAIN { - let result = self.propagate_constraints(); - self.update_intervals(leaf_bounds); - result - } else { - Ok(PropagationResult::CannotPropagate) - } - } - /// Returns the interval associated with the node at the given `index`. pub fn get_interval(&self, index: usize) -> Interval { self.graph[NodeIndex::new(index)].interval.clone() } } -/// During the propagation of [`Interval`] values on an [`ExprIntervalGraph`], if there exists a `timestamp - timestamp` -/// operation, the result would be of type `Duration`. However, we may encounter a situation where a time interval -/// is involved in an arithmetic operation with a `Duration` type. This function offers special handling for such cases, -/// where the time interval resides on the left side of the operation. +/// This is a subfunction of the `propagate_arithmetic` function that propagates to the right child. +fn propagate_right( + left: &Interval, + parent: &Interval, + right: &Interval, + op: &Operator, + inverse_op: &Operator, +) -> Result> { + match op { + Operator::Minus => apply_operator(op, left, parent), + Operator::Plus => apply_operator(inverse_op, parent, left), + Operator::Divide => apply_operator(op, left, parent), + Operator::Multiply => apply_operator(inverse_op, parent, left), + _ => internal_err!("Interval arithmetic does not support the operator {}", op), + }? + .intersect(right) +} + +/// During the propagation of [`Interval`] values on an [`ExprIntervalGraph`], +/// if there exists a `timestamp - timestamp` operation, the result would be +/// of type `Duration`. However, we may encounter a situation where a time interval +/// is involved in an arithmetic operation with a `Duration` type. This function +/// offers special handling for such cases, where the time interval resides on +/// the left side of the operation. fn propagate_time_interval_at_left( left_child: &Interval, right_child: &Interval, parent: &Interval, op: &Operator, inverse_op: &Operator, -) -> Result<(Option, Option)> { +) -> Result> { // We check if the child's time interval(s) has a non-zero month or day field(s). // If so, we return it as is without propagating. Otherwise, we first convert - // the time intervals to the Duration type, then propagate, and then convert the bounds to time intervals again. - if let Some(duration) = convert_interval_type_to_duration(left_child) { + // the time intervals to the `Duration` type, then propagate, and then convert + // the bounds to time intervals again. + let result = if let Some(duration) = convert_interval_type_to_duration(left_child) { match apply_operator(inverse_op, parent, right_child)?.intersect(duration)? { Some(value) => { + let left = convert_duration_type_to_interval(&value); let right = propagate_right(&value, parent, right_child, op, inverse_op)?; - let new_interval = convert_duration_type_to_interval(&value); - Ok((new_interval, right)) + match (left, right) { + (Some(left), Some(right)) => Some((left, right)), + _ => None, + } } - None => Ok((None, None)), + None => None, } } else { - let right = propagate_right(left_child, parent, right_child, op, inverse_op)?; - Ok((Some(left_child.clone()), right)) - } + propagate_right(left_child, parent, right_child, op, inverse_op)? + .map(|right| (left_child.clone(), right)) + }; + Ok(result) } -/// During the propagation of [`Interval`] values on an [`ExprIntervalGraph`], if there exists a `timestamp - timestamp` -/// operation, the result would be of type `Duration`. However, we may encounter a situation where a time interval -/// is involved in an arithmetic operation with a `Duration` type. This function offers special handling for such cases, -/// where the time interval resides on the right side of the operation. +/// During the propagation of [`Interval`] values on an [`ExprIntervalGraph`], +/// if there exists a `timestamp - timestamp` operation, the result would be +/// of type `Duration`. However, we may encounter a situation where a time interval +/// is involved in an arithmetic operation with a `Duration` type. This function +/// offers special handling for such cases, where the time interval resides on +/// the right side of the operation. fn propagate_time_interval_at_right( left_child: &Interval, right_child: &Interval, parent: &Interval, op: &Operator, inverse_op: &Operator, -) -> Result<(Option, Option)> { +) -> Result> { // We check if the child's time interval(s) has a non-zero month or day field(s). // If so, we return it as is without propagating. Otherwise, we first convert - // the time intervals to the Duration type, then propagate, and then convert the bounds to time intervals again. - if let Some(duration) = convert_interval_type_to_duration(right_child) { + // the time intervals to the `Duration` type, then propagate, and then convert + // the bounds to time intervals again. + let result = if let Some(duration) = convert_interval_type_to_duration(right_child) { match apply_operator(inverse_op, parent, &duration)?.intersect(left_child)? { Some(value) => { - let right = - propagate_right(left_child, parent, &duration, op, inverse_op)?; - let right = - right.and_then(|right| convert_duration_type_to_interval(&right)); - Ok((Some(value), right)) + propagate_right(left_child, parent, &duration, op, inverse_op)? + .and_then(|right| convert_duration_type_to_interval(&right)) + .map(|right| (value, right)) } - None => Ok((None, None)), + None => None, } } else { - match apply_operator(inverse_op, parent, right_child)?.intersect(left_child)? { - Some(value) => Ok((Some(value), Some(right_child.clone()))), - None => Ok((None, None)), - } - } + apply_operator(inverse_op, parent, right_child)? + .intersect(left_child)? + .map(|value| (value, right_child.clone())) + }; + Ok(result) } -/// This is a subfunction of the `propagate_arithmetic` function that propagates to the right child. -fn propagate_right( - left: &Interval, - parent: &Interval, - right: &Interval, - op: &Operator, - inverse_op: &Operator, -) -> Result> { - match op { - Operator::Minus => apply_operator(op, left, parent), - Operator::Plus => apply_operator(inverse_op, parent, left), - _ => unreachable!(), - }? - .intersect(right) -} - -/// Converts the `time interval` (as the left child) to duration, then performs the propagation rule for comparison operators. -pub fn propagate_comparison_to_time_interval_at_left( - left_child: &Interval, - parent: &Interval, - right_child: &Interval, -) -> Result<(Option, Option)> { - if let Some(converted) = convert_interval_type_to_duration(left_child) { - propagate_arithmetic(&Operator::Minus, parent, &converted, right_child) - } else { - Err(DataFusionError::Internal( - "Interval type has a non-zero month field, cannot compare with a Duration type".to_string(), - )) - } -} - -/// Converts the `time interval` (as the right child) to duration, then performs the propagation rule for comparison operators. -pub fn propagate_comparison_to_time_interval_at_right( - left_child: &Interval, - parent: &Interval, - right_child: &Interval, -) -> Result<(Option, Option)> { - if let Some(converted) = convert_interval_type_to_duration(right_child) { - propagate_arithmetic(&Operator::Minus, parent, left_child, &converted) - } else { - Err(DataFusionError::Internal( - "Interval type has a non-zero month field, cannot compare with a Duration type".to_string(), - )) - } +fn reverse_tuple((first, second): (T, U)) -> (U, T) { + (second, first) } #[cfg(test)] mod tests { use super::*; - use itertools::Itertools; - use crate::expressions::{BinaryExpr, Column}; use crate::intervals::test_utils::gen_conjunctive_numerical_expr; + use arrow::datatypes::TimeUnit; + use arrow_schema::{DataType, Field}; use datafusion_common::ScalarValue; + + use itertools::Itertools; use rand::rngs::StdRng; use rand::{Rng, SeedableRng}; use rstest::*; + #[allow(clippy::too_many_arguments)] fn experiment( expr: Arc, exprs_with_interval: (Arc, Arc), @@ -702,6 +740,7 @@ mod tests { left_expected: Interval, right_expected: Interval, result: PropagationResult, + schema: &Schema, ) -> Result<()> { let col_stats = vec![ (exprs_with_interval.0.clone(), left_interval), @@ -711,7 +750,7 @@ mod tests { (exprs_with_interval.0.clone(), left_expected), (exprs_with_interval.1.clone(), right_expected), ]; - let mut graph = ExprIntervalGraph::try_new(expr)?; + let mut graph = ExprIntervalGraph::try_new(expr, schema)?; let expr_indexes = graph .gather_node_indices(&col_stats.iter().map(|(e, _)| e.clone()).collect_vec()); @@ -726,14 +765,37 @@ mod tests { .map(|((_, interval), (_, index))| (*index, interval.clone())) .collect_vec(); - let exp_result = graph.update_ranges(&mut col_stat_nodes[..])?; + let exp_result = + graph.update_ranges(&mut col_stat_nodes[..], Interval::CERTAINLY_TRUE)?; assert_eq!(exp_result, result); col_stat_nodes.iter().zip(expected_nodes.iter()).for_each( |((_, calculated_interval_node), (_, expected))| { // NOTE: These randomized tests only check for conservative containment, // not openness/closedness of endpoints. - assert!(calculated_interval_node.lower.value <= expected.lower.value); - assert!(calculated_interval_node.upper.value >= expected.upper.value); + + // Calculated bounds are relaxed by 1 to cover all strict and + // and non-strict comparison cases since we have only closed bounds. + let one = ScalarValue::new_one(&expected.data_type()).unwrap(); + assert!( + calculated_interval_node.lower() + <= &expected.lower().add(&one).unwrap(), + "{}", + format!( + "Calculated {} must be less than or equal {}", + calculated_interval_node.lower(), + expected.lower() + ) + ); + assert!( + calculated_interval_node.upper() + >= &expected.upper().sub(&one).unwrap(), + "{}", + format!( + "Calculated {} must be greater than or equal {}", + calculated_interval_node.upper(), + expected.upper() + ) + ); }, ); Ok(()) @@ -773,12 +835,24 @@ mod tests { experiment( expr, - (left_col, right_col), - Interval::make(left_given.0, left_given.1, (true, true)), - Interval::make(right_given.0, right_given.1, (true, true)), - Interval::make(left_expected.0, left_expected.1, (true, true)), - Interval::make(right_expected.0, right_expected.1, (true, true)), + (left_col.clone(), right_col.clone()), + Interval::make(left_given.0, left_given.1).unwrap(), + Interval::make(right_given.0, right_given.1).unwrap(), + Interval::make(left_expected.0, left_expected.1).unwrap(), + Interval::make(right_expected.0, right_expected.1).unwrap(), PropagationResult::Success, + &Schema::new(vec![ + Field::new( + left_col.as_any().downcast_ref::().unwrap().name(), + DataType::$SCALAR, + true, + ), + Field::new( + right_col.as_any().downcast_ref::().unwrap().name(), + DataType::$SCALAR, + true, + ), + ]), ) } }; @@ -802,12 +876,24 @@ mod tests { let expr = Arc::new(BinaryExpr::new(left_and_1, Operator::Gt, right_col.clone())); experiment( expr, - (left_col, right_col), - Interval::make(Some(10), Some(20), (true, true)), - Interval::make(Some(100), None, (true, true)), - Interval::make(Some(10), Some(20), (true, true)), - Interval::make(Some(100), None, (true, true)), + (left_col.clone(), right_col.clone()), + Interval::make(Some(10_i32), Some(20_i32))?, + Interval::make(Some(100), None)?, + Interval::make(Some(10), Some(20))?, + Interval::make(Some(100), None)?, PropagationResult::Infeasible, + &Schema::new(vec![ + Field::new( + left_col.as_any().downcast_ref::().unwrap().name(), + DataType::Int32, + true, + ), + Field::new( + right_col.as_any().downcast_ref::().unwrap().name(), + DataType::Int32, + true, + ), + ]), ) } @@ -1112,7 +1198,14 @@ mod tests { Arc::new(Column::new("b", 1)), )); let expr = Arc::new(BinaryExpr::new(left_expr, Operator::Gt, right_expr)); - let mut graph = ExprIntervalGraph::try_new(expr).unwrap(); + let mut graph = ExprIntervalGraph::try_new( + expr, + &Schema::new(vec![ + Field::new("a", DataType::Int32, true), + Field::new("b", DataType::Int32, true), + ]), + ) + .unwrap(); // Define a test leaf node. let leaf_node = Arc::new(BinaryExpr::new( Arc::new(Column::new("a", 0)), @@ -1151,7 +1244,16 @@ mod tests { Arc::new(Column::new("z", 1)), )); let expr = Arc::new(BinaryExpr::new(left_expr, Operator::Gt, right_expr)); - let mut graph = ExprIntervalGraph::try_new(expr).unwrap(); + let mut graph = ExprIntervalGraph::try_new( + expr, + &Schema::new(vec![ + Field::new("a", DataType::Int32, true), + Field::new("b", DataType::Int32, true), + Field::new("y", DataType::Int32, true), + Field::new("z", DataType::Int32, true), + ]), + ) + .unwrap(); // Define a test leaf node. let leaf_node = Arc::new(BinaryExpr::new( Arc::new(Column::new("a", 0)), @@ -1190,7 +1292,15 @@ mod tests { Arc::new(Column::new("z", 1)), )); let expr = Arc::new(BinaryExpr::new(left_expr, Operator::Gt, right_expr)); - let mut graph = ExprIntervalGraph::try_new(expr).unwrap(); + let mut graph = ExprIntervalGraph::try_new( + expr, + &Schema::new(vec![ + Field::new("a", DataType::Int32, true), + Field::new("b", DataType::Int32, true), + Field::new("z", DataType::Int32, true), + ]), + ) + .unwrap(); // Define a test leaf node. let leaf_node = Arc::new(BinaryExpr::new( Arc::new(Column::new("a", 0)), @@ -1213,9 +1323,9 @@ mod tests { fn test_gather_node_indices_cannot_provide() -> Result<()> { // Expression: a@0 + 1 + b@1 > y@0 - z@1 -> provide a@0 + b@1 // TODO: We expect nodes a@0 and b@1 to be pruned, and intervals to be provided from the a@0 + b@1 node. - // However, we do not have an exact node for a@0 + b@1 due to the binary tree structure of the expressions. - // Pruning and interval providing for BinaryExpr expressions are more challenging without exact matches. - // Currently, we only support exact matches for BinaryExprs, but we plan to extend support beyond exact matches in the future. + // However, we do not have an exact node for a@0 + b@1 due to the binary tree structure of the expressions. + // Pruning and interval providing for BinaryExpr expressions are more challenging without exact matches. + // Currently, we only support exact matches for BinaryExprs, but we plan to extend support beyond exact matches in the future. let left_expr = Arc::new(BinaryExpr::new( Arc::new(BinaryExpr::new( Arc::new(Column::new("a", 0)), @@ -1232,7 +1342,16 @@ mod tests { Arc::new(Column::new("z", 1)), )); let expr = Arc::new(BinaryExpr::new(left_expr, Operator::Gt, right_expr)); - let mut graph = ExprIntervalGraph::try_new(expr).unwrap(); + let mut graph = ExprIntervalGraph::try_new( + expr, + &Schema::new(vec![ + Field::new("a", DataType::Int32, true), + Field::new("b", DataType::Int32, true), + Field::new("y", DataType::Int32, true), + Field::new("z", DataType::Int32, true), + ]), + ) + .unwrap(); // Define a test leaf node. let leaf_node = Arc::new(BinaryExpr::new( Arc::new(Column::new("a", 0)), @@ -1257,80 +1376,51 @@ mod tests { Operator::Plus, Arc::new(Literal::new(ScalarValue::new_interval_mdn(0, 1, 321))), ); - let parent = Interval::new( - IntervalBound::new( - // 15.10.2020 - 10:11:12.000_000_321 AM - ScalarValue::TimestampNanosecond(Some(1_602_756_672_000_000_321), None), - false, - ), - IntervalBound::new( - // 16.10.2020 - 10:11:12.000_000_321 AM - ScalarValue::TimestampNanosecond(Some(1_602_843_072_000_000_321), None), - false, - ), - ); - let left_child = Interval::new( - IntervalBound::new( - // 10.10.2020 - 10:11:12 AM - ScalarValue::TimestampNanosecond(Some(1_602_324_672_000_000_000), None), - false, - ), - IntervalBound::new( - // 20.10.2020 - 10:11:12 AM - ScalarValue::TimestampNanosecond(Some(1_603_188_672_000_000_000), None), - false, - ), - ); - let right_child = Interval::new( - IntervalBound::new( - // 1 day 321 ns - ScalarValue::IntervalMonthDayNano(Some(0x1_0000_0000_0000_0141)), - false, - ), - IntervalBound::new( - // 1 day 321 ns - ScalarValue::IntervalMonthDayNano(Some(0x1_0000_0000_0000_0141)), - false, - ), - ); + let parent = Interval::try_new( + // 15.10.2020 - 10:11:12.000_000_321 AM + ScalarValue::TimestampNanosecond(Some(1_602_756_672_000_000_321), None), + // 16.10.2020 - 10:11:12.000_000_321 AM + ScalarValue::TimestampNanosecond(Some(1_602_843_072_000_000_321), None), + )?; + let left_child = Interval::try_new( + // 10.10.2020 - 10:11:12 AM + ScalarValue::TimestampNanosecond(Some(1_602_324_672_000_000_000), None), + // 20.10.2020 - 10:11:12 AM + ScalarValue::TimestampNanosecond(Some(1_603_188_672_000_000_000), None), + )?; + let right_child = Interval::try_new( + // 1 day 321 ns + ScalarValue::IntervalMonthDayNano(Some(0x1_0000_0000_0000_0141)), + // 1 day 321 ns + ScalarValue::IntervalMonthDayNano(Some(0x1_0000_0000_0000_0141)), + )?; let children = vec![&left_child, &right_child]; - let result = expression.propagate_constraints(&parent, &children)?; + let result = expression + .propagate_constraints(&parent, &children)? + .unwrap(); assert_eq!( - Some(Interval::new( - // 14.10.2020 - 10:11:12 AM - IntervalBound::new( + vec![ + Interval::try_new( + // 14.10.2020 - 10:11:12 AM ScalarValue::TimestampNanosecond( Some(1_602_670_272_000_000_000), None ), - false, - ), - // 15.10.2020 - 10:11:12 AM - IntervalBound::new( + // 15.10.2020 - 10:11:12 AM ScalarValue::TimestampNanosecond( Some(1_602_756_672_000_000_000), None ), - false, - ), - )), - result[0] - ); - assert_eq!( - Some(Interval::new( - // 1 day 321 ns in Duration type - IntervalBound::new( + )?, + Interval::try_new( + // 1 day 321 ns in Duration type ScalarValue::IntervalMonthDayNano(Some(0x1_0000_0000_0000_0141)), - false, - ), - // 1 day 321 ns in Duration type - IntervalBound::new( + // 1 day 321 ns in Duration type ScalarValue::IntervalMonthDayNano(Some(0x1_0000_0000_0000_0141)), - false, - ), - )), - result[1] + )? + ], + result ); Ok(()) @@ -1343,206 +1433,216 @@ mod tests { Operator::Plus, Arc::new(Column::new("ts_column", 0)), ); - let parent = Interval::new( - IntervalBound::new( - // 15.10.2020 - 10:11:12 AM - ScalarValue::TimestampMillisecond(Some(1_602_756_672_000), None), - false, - ), - IntervalBound::new( - // 16.10.2020 - 10:11:12 AM - ScalarValue::TimestampMillisecond(Some(1_602_843_072_000), None), - false, - ), - ); - let right_child = Interval::new( - IntervalBound::new( - // 10.10.2020 - 10:11:12 AM - ScalarValue::TimestampMillisecond(Some(1_602_324_672_000), None), - false, - ), - IntervalBound::new( - // 20.10.2020 - 10:11:12 AM - ScalarValue::TimestampMillisecond(Some(1_603_188_672_000), None), - false, - ), - ); - let left_child = Interval::new( - IntervalBound::new( - // 2 days - ScalarValue::IntervalDayTime(Some(172_800_000)), - false, - ), - IntervalBound::new( - // 10 days - ScalarValue::IntervalDayTime(Some(864_000_000)), - false, - ), - ); + let parent = Interval::try_new( + // 15.10.2020 - 10:11:12 AM + ScalarValue::TimestampMillisecond(Some(1_602_756_672_000), None), + // 16.10.2020 - 10:11:12 AM + ScalarValue::TimestampMillisecond(Some(1_602_843_072_000), None), + )?; + let right_child = Interval::try_new( + // 10.10.2020 - 10:11:12 AM + ScalarValue::TimestampMillisecond(Some(1_602_324_672_000), None), + // 20.10.2020 - 10:11:12 AM + ScalarValue::TimestampMillisecond(Some(1_603_188_672_000), None), + )?; + let left_child = Interval::try_new( + // 2 days + ScalarValue::IntervalDayTime(Some(172_800_000)), + // 10 days + ScalarValue::IntervalDayTime(Some(864_000_000)), + )?; let children = vec![&left_child, &right_child]; - let result = expression.propagate_constraints(&parent, &children)?; + let result = expression + .propagate_constraints(&parent, &children)? + .unwrap(); assert_eq!( - Some(Interval::new( - // 10.10.2020 - 10:11:12 AM - IntervalBound::new( - ScalarValue::TimestampMillisecond(Some(1_602_324_672_000), None), - false, - ), - // 14.10.2020 - 10:11:12 AM - IntervalBound::new( - ScalarValue::TimestampMillisecond(Some(1_602_670_272_000), None), - false, - ) - )), - result[1] - ); - assert_eq!( - Some(Interval::new( - IntervalBound::new( + vec![ + Interval::try_new( // 2 days ScalarValue::IntervalDayTime(Some(172_800_000)), - false, - ), - IntervalBound::new( // 6 days ScalarValue::IntervalDayTime(Some(518_400_000)), - false, - ), - )), - result[0] + )?, + Interval::try_new( + // 10.10.2020 - 10:11:12 AM + ScalarValue::TimestampMillisecond(Some(1_602_324_672_000), None), + // 14.10.2020 - 10:11:12 AM + ScalarValue::TimestampMillisecond(Some(1_602_670_272_000), None), + )? + ], + result ); Ok(()) } #[test] - fn test_propagate_comparison() { + fn test_propagate_comparison() -> Result<()> { // In the examples below: // `left` is unbounded: [?, ?], // `right` is known to be [1000,1000] - // so `left` < `right` results in no new knowledge of `right` but knowing that `left` is now < 1000:` [?, 1000) - let left = Interval::new( - IntervalBound::make_unbounded(DataType::Int64).unwrap(), - IntervalBound::make_unbounded(DataType::Int64).unwrap(), - ); - let right = Interval::new( - IntervalBound::new(ScalarValue::Int64(Some(1000)), false), - IntervalBound::new(ScalarValue::Int64(Some(1000)), false), - ); + // so `left` < `right` results in no new knowledge of `right` but knowing that `left` is now < 1000:` [?, 999] + let left = Interval::make_unbounded(&DataType::Int64)?; + let right = Interval::make(Some(1000_i64), Some(1000_i64))?; assert_eq!( - ( - Some(Interval::new( - IntervalBound::make_unbounded(DataType::Int64).unwrap(), - IntervalBound::new(ScalarValue::Int64(Some(1000)), true) - )), - Some(Interval::new( - IntervalBound::new(ScalarValue::Int64(Some(1000)), false), - IntervalBound::new(ScalarValue::Int64(Some(1000)), false) - )), - ), - propagate_comparison(&Operator::Lt, &left, &right).unwrap() + (Some(( + Interval::make(None, Some(999_i64))?, + Interval::make(Some(1000_i64), Some(1000_i64))?, + ))), + propagate_comparison( + &Operator::Lt, + &Interval::CERTAINLY_TRUE, + &left, + &right + )? ); - let left = Interval::new( - IntervalBound::make_unbounded(DataType::Timestamp( - TimeUnit::Nanosecond, - None, - )) - .unwrap(), - IntervalBound::make_unbounded(DataType::Timestamp( - TimeUnit::Nanosecond, - None, - )) - .unwrap(), - ); - let right = Interval::new( - IntervalBound::new(ScalarValue::TimestampNanosecond(Some(1000), None), false), - IntervalBound::new(ScalarValue::TimestampNanosecond(Some(1000), None), false), - ); + let left = + Interval::make_unbounded(&DataType::Timestamp(TimeUnit::Nanosecond, None))?; + let right = Interval::try_new( + ScalarValue::TimestampNanosecond(Some(1000), None), + ScalarValue::TimestampNanosecond(Some(1000), None), + )?; assert_eq!( - ( - Some(Interval::new( - IntervalBound::make_unbounded(DataType::Timestamp( + (Some(( + Interval::try_new( + ScalarValue::try_from(&DataType::Timestamp( TimeUnit::Nanosecond, None )) .unwrap(), - IntervalBound::new( - ScalarValue::TimestampNanosecond(Some(1000), None), - true - ) - )), - Some(Interval::new( - IntervalBound::new( - ScalarValue::TimestampNanosecond(Some(1000), None), - false - ), - IntervalBound::new( - ScalarValue::TimestampNanosecond(Some(1000), None), - false - ) - )), - ), - propagate_comparison(&Operator::Lt, &left, &right).unwrap() + ScalarValue::TimestampNanosecond(Some(999), None), + )?, + Interval::try_new( + ScalarValue::TimestampNanosecond(Some(1000), None), + ScalarValue::TimestampNanosecond(Some(1000), None), + )? + ))), + propagate_comparison( + &Operator::Lt, + &Interval::CERTAINLY_TRUE, + &left, + &right + )? ); - let left = Interval::new( - IntervalBound::make_unbounded(DataType::Timestamp( - TimeUnit::Nanosecond, - Some("+05:00".into()), - )) - .unwrap(), - IntervalBound::make_unbounded(DataType::Timestamp( - TimeUnit::Nanosecond, - Some("+05:00".into()), - )) - .unwrap(), - ); - let right = Interval::new( - IntervalBound::new( - ScalarValue::TimestampNanosecond(Some(1000), Some("+05:00".into())), - false, - ), - IntervalBound::new( - ScalarValue::TimestampNanosecond(Some(1000), Some("+05:00".into())), - false, - ), - ); + let left = Interval::make_unbounded(&DataType::Timestamp( + TimeUnit::Nanosecond, + Some("+05:00".into()), + ))?; + let right = Interval::try_new( + ScalarValue::TimestampNanosecond(Some(1000), Some("+05:00".into())), + ScalarValue::TimestampNanosecond(Some(1000), Some("+05:00".into())), + )?; assert_eq!( - ( - Some(Interval::new( - IntervalBound::make_unbounded(DataType::Timestamp( + (Some(( + Interval::try_new( + ScalarValue::try_from(&DataType::Timestamp( TimeUnit::Nanosecond, Some("+05:00".into()), )) .unwrap(), - IntervalBound::new( - ScalarValue::TimestampNanosecond( - Some(1000), - Some("+05:00".into()) - ), - true - ) - )), - Some(Interval::new( - IntervalBound::new( - ScalarValue::TimestampNanosecond( - Some(1000), - Some("+05:00".into()) - ), - false - ), - IntervalBound::new( - ScalarValue::TimestampNanosecond( - Some(1000), - Some("+05:00".into()) - ), - false - ) - )), - ), - propagate_comparison(&Operator::Lt, &left, &right).unwrap() + ScalarValue::TimestampNanosecond(Some(999), Some("+05:00".into())), + )?, + Interval::try_new( + ScalarValue::TimestampNanosecond(Some(1000), Some("+05:00".into())), + ScalarValue::TimestampNanosecond(Some(1000), Some("+05:00".into())), + )? + ))), + propagate_comparison( + &Operator::Lt, + &Interval::CERTAINLY_TRUE, + &left, + &right + )? ); + + Ok(()) + } + + #[test] + fn test_propagate_or() -> Result<()> { + let expr = Arc::new(BinaryExpr::new( + Arc::new(Column::new("a", 0)), + Operator::Or, + Arc::new(Column::new("b", 1)), + )); + let parent = Interval::CERTAINLY_FALSE; + let children_set = vec![ + vec![&Interval::CERTAINLY_FALSE, &Interval::UNCERTAIN], + vec![&Interval::UNCERTAIN, &Interval::CERTAINLY_FALSE], + vec![&Interval::CERTAINLY_FALSE, &Interval::CERTAINLY_FALSE], + vec![&Interval::UNCERTAIN, &Interval::UNCERTAIN], + ]; + for children in children_set { + assert_eq!( + expr.propagate_constraints(&parent, &children)?.unwrap(), + vec![Interval::CERTAINLY_FALSE, Interval::CERTAINLY_FALSE], + ); + } + + let parent = Interval::CERTAINLY_FALSE; + let children_set = vec![ + vec![&Interval::CERTAINLY_TRUE, &Interval::UNCERTAIN], + vec![&Interval::UNCERTAIN, &Interval::CERTAINLY_TRUE], + ]; + for children in children_set { + assert_eq!(expr.propagate_constraints(&parent, &children)?, None,); + } + + let parent = Interval::CERTAINLY_TRUE; + let children = vec![&Interval::CERTAINLY_FALSE, &Interval::UNCERTAIN]; + assert_eq!( + expr.propagate_constraints(&parent, &children)?.unwrap(), + vec![Interval::CERTAINLY_FALSE, Interval::CERTAINLY_TRUE] + ); + + let parent = Interval::CERTAINLY_TRUE; + let children = vec![&Interval::UNCERTAIN, &Interval::UNCERTAIN]; + assert_eq!( + expr.propagate_constraints(&parent, &children)?.unwrap(), + // Empty means unchanged intervals. + vec![] + ); + + Ok(()) + } + + #[test] + fn test_propagate_certainly_false_and() -> Result<()> { + let expr = Arc::new(BinaryExpr::new( + Arc::new(Column::new("a", 0)), + Operator::And, + Arc::new(Column::new("b", 1)), + )); + let parent = Interval::CERTAINLY_FALSE; + let children_and_results_set = vec![ + ( + vec![&Interval::CERTAINLY_TRUE, &Interval::UNCERTAIN], + vec![Interval::CERTAINLY_TRUE, Interval::CERTAINLY_FALSE], + ), + ( + vec![&Interval::UNCERTAIN, &Interval::CERTAINLY_TRUE], + vec![Interval::CERTAINLY_FALSE, Interval::CERTAINLY_TRUE], + ), + ( + vec![&Interval::UNCERTAIN, &Interval::UNCERTAIN], + // Empty means unchanged intervals. + vec![], + ), + ( + vec![&Interval::CERTAINLY_FALSE, &Interval::UNCERTAIN], + vec![], + ), + ]; + for (children, result) in children_and_results_set { + assert_eq!( + expr.propagate_constraints(&parent, &children)?.unwrap(), + result + ); + } + + Ok(()) } } diff --git a/datafusion/physical-expr/src/intervals/interval_aritmetic.rs b/datafusion/physical-expr/src/intervals/interval_aritmetic.rs deleted file mode 100644 index 4b81adfbb1f8..000000000000 --- a/datafusion/physical-expr/src/intervals/interval_aritmetic.rs +++ /dev/null @@ -1,1886 +0,0 @@ -// Licensed to the Apache Software Foundation (ASF) under one -// or more contributor license agreements. See the NOTICE file -// distributed with this work for additional information -// regarding copyright ownership. The ASF licenses this file -// to you under the Apache License, Version 2.0 (the -// "License"); you may not use this file except in compliance -// with the License. You may obtain a copy of the License at -// -//http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, -// software distributed under the License is distributed on an -// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -// KIND, either express or implied. See the License for the -// specific language governing permissions and limitations -// under the License. - -//! Interval arithmetic library - -use std::borrow::Borrow; -use std::fmt::{self, Display, Formatter}; -use std::ops::{AddAssign, SubAssign}; - -use crate::aggregate::min_max::{max, min}; -use crate::intervals::rounding::{alter_fp_rounding_mode, next_down, next_up}; - -use arrow::compute::{cast_with_options, CastOptions}; -use arrow::datatypes::DataType; -use arrow_array::ArrowNativeTypeOp; -use datafusion_common::{internal_err, DataFusionError, Result, ScalarValue}; -use datafusion_expr::type_coercion::binary::get_result_type; -use datafusion_expr::Operator; - -/// This type represents a single endpoint of an [`Interval`]. An -/// endpoint can be open (does not include the endpoint) or closed -/// (includes the endpoint). -#[derive(Debug, Clone, PartialEq, Eq)] -pub struct IntervalBound { - pub value: ScalarValue, - /// If true, interval does not include `value` - pub open: bool, -} - -impl IntervalBound { - /// Creates a new `IntervalBound` object using the given value. - pub const fn new(value: ScalarValue, open: bool) -> IntervalBound { - IntervalBound { value, open } - } - - /// Creates a new "open" interval (does not include the `value` - /// bound) - pub const fn new_open(value: ScalarValue) -> IntervalBound { - IntervalBound::new(value, true) - } - - /// Creates a new "closed" interval (includes the `value` - /// bound) - pub const fn new_closed(value: ScalarValue) -> IntervalBound { - IntervalBound::new(value, false) - } - - /// This convenience function creates an unbounded interval endpoint. - pub fn make_unbounded>(data_type: T) -> Result { - ScalarValue::try_from(data_type.borrow()).map(|v| IntervalBound::new(v, true)) - } - - /// This convenience function returns the data type associated with this - /// `IntervalBound`. - pub fn get_datatype(&self) -> DataType { - self.value.data_type() - } - - /// This convenience function checks whether the `IntervalBound` represents - /// an unbounded interval endpoint. - pub fn is_unbounded(&self) -> bool { - self.value.is_null() - } - - /// This function casts the `IntervalBound` to the given data type. - pub(crate) fn cast_to( - &self, - data_type: &DataType, - cast_options: &CastOptions, - ) -> Result { - cast_scalar_value(&self.value, data_type, cast_options) - .map(|value| IntervalBound::new(value, self.open)) - } - - /// Returns a new bound with a negated value, if any, and the same open/closed. - /// For example negating `[5` would return `[-5`, or `-1)` would return `1)`. - pub fn negate(&self) -> Result { - self.value.arithmetic_negate().map(|value| IntervalBound { - value, - open: self.open, - }) - } - - /// This function adds the given `IntervalBound` to this `IntervalBound`. - /// The result is unbounded if either is; otherwise, their values are - /// added. The result is closed if both original bounds are closed, or open - /// otherwise. - pub fn add>( - &self, - other: T, - ) -> Result { - let rhs = other.borrow(); - if self.is_unbounded() || rhs.is_unbounded() { - return IntervalBound::make_unbounded(get_result_type( - &self.get_datatype(), - &Operator::Plus, - &rhs.get_datatype(), - )?); - } - match self.get_datatype() { - DataType::Float64 | DataType::Float32 => { - alter_fp_rounding_mode::(&self.value, &rhs.value, |lhs, rhs| { - lhs.add(rhs) - }) - } - _ => self.value.add(&rhs.value), - } - .map(|v| IntervalBound::new(v, self.open || rhs.open)) - } - - /// This function subtracts the given `IntervalBound` from `self`. - /// The result is unbounded if either is; otherwise, their values are - /// subtracted. The result is closed if both original bounds are closed, - /// or open otherwise. - pub fn sub>( - &self, - other: T, - ) -> Result { - let rhs = other.borrow(); - if self.is_unbounded() || rhs.is_unbounded() { - return IntervalBound::make_unbounded(get_result_type( - &self.get_datatype(), - &Operator::Minus, - &rhs.get_datatype(), - )?); - } - match self.get_datatype() { - DataType::Float64 | DataType::Float32 => { - alter_fp_rounding_mode::(&self.value, &rhs.value, |lhs, rhs| { - lhs.sub(rhs) - }) - } - _ => self.value.sub(&rhs.value), - } - .map(|v| IntervalBound::new(v, self.open || rhs.open)) - } - - /// This function chooses one of the given `IntervalBound`s according to - /// the given function `decide`. The result is unbounded if both are. If - /// only one of the arguments is unbounded, the other one is chosen by - /// default. If neither is unbounded, the function `decide` is used. - pub fn choose( - first: &IntervalBound, - second: &IntervalBound, - decide: fn(&ScalarValue, &ScalarValue) -> Result, - ) -> Result { - Ok(if first.is_unbounded() { - second.clone() - } else if second.is_unbounded() { - first.clone() - } else if first.value != second.value { - let chosen = decide(&first.value, &second.value)?; - if chosen.eq(&first.value) { - first.clone() - } else { - second.clone() - } - } else { - IntervalBound::new(second.value.clone(), first.open || second.open) - }) - } -} - -impl Display for IntervalBound { - fn fmt(&self, f: &mut Formatter<'_>) -> fmt::Result { - write!(f, "IntervalBound [{}]", self.value) - } -} - -/// This type represents an interval, which is used to calculate reliable -/// bounds for expressions: -/// -/// * An *open* interval does not include the endpoint and is written using a -/// `(` or `)`. -/// -/// * A *closed* interval does include the endpoint and is written using `[` or -/// `]`. -/// -/// * If the interval's `lower` and/or `upper` bounds are not known, they are -/// called *unbounded* endpoint and represented using a `NULL` and written using -/// `∞`. -/// -/// # Examples -/// -/// A `Int64` `Interval` of `[10, 20)` represents the values `10, 11, ... 18, -/// 19` (includes 10, but does not include 20). -/// -/// A `Int64` `Interval` of `[10, ∞)` represents a value known to be either -/// `10` or higher. -/// -/// An `Interval` of `(-∞, ∞)` represents that the range is entirely unknown. -/// -#[derive(Debug, Clone, PartialEq, Eq)] -pub struct Interval { - pub lower: IntervalBound, - pub upper: IntervalBound, -} - -impl Default for Interval { - fn default() -> Self { - Interval::new( - IntervalBound::new(ScalarValue::Null, true), - IntervalBound::new(ScalarValue::Null, true), - ) - } -} - -impl Display for Interval { - fn fmt(&self, f: &mut Formatter<'_>) -> fmt::Result { - write!( - f, - "{}{}, {}{}", - if self.lower.open { "(" } else { "[" }, - self.lower.value, - self.upper.value, - if self.upper.open { ")" } else { "]" } - ) - } -} - -impl Interval { - /// Creates a new interval object using the given bounds. - /// - /// # Boolean intervals need special handling - /// - /// For boolean intervals, having an open false lower bound is equivalent to - /// having a true closed lower bound. Similarly, open true upper bound is - /// equivalent to having a false closed upper bound. Also for boolean - /// intervals, having an unbounded left endpoint is equivalent to having a - /// false closed lower bound, while having an unbounded right endpoint is - /// equivalent to having a true closed upper bound. Therefore; input - /// parameters to construct an Interval can have different types, but they - /// all result in `[false, false]`, `[false, true]` or `[true, true]`. - pub fn new(lower: IntervalBound, upper: IntervalBound) -> Interval { - // Boolean intervals need a special handling. - if let ScalarValue::Boolean(_) = lower.value { - let standardized_lower = match lower.value { - ScalarValue::Boolean(None) if lower.open => { - ScalarValue::Boolean(Some(false)) - } - ScalarValue::Boolean(Some(false)) if lower.open => { - ScalarValue::Boolean(Some(true)) - } - // The rest may include some invalid interval cases. The validation of - // interval construction parameters will be implemented later. - // For now, let's return them unchanged. - _ => lower.value, - }; - let standardized_upper = match upper.value { - ScalarValue::Boolean(None) if upper.open => { - ScalarValue::Boolean(Some(true)) - } - ScalarValue::Boolean(Some(true)) if upper.open => { - ScalarValue::Boolean(Some(false)) - } - _ => upper.value, - }; - Interval { - lower: IntervalBound::new(standardized_lower, false), - upper: IntervalBound::new(standardized_upper, false), - } - } else { - Interval { lower, upper } - } - } - - pub fn make(lower: Option, upper: Option, open: (bool, bool)) -> Interval - where - ScalarValue: From>, - { - Interval::new( - IntervalBound::new(ScalarValue::from(lower), open.0), - IntervalBound::new(ScalarValue::from(upper), open.1), - ) - } - - /// Casts this interval to `data_type` using `cast_options`. - pub(crate) fn cast_to( - &self, - data_type: &DataType, - cast_options: &CastOptions, - ) -> Result { - let lower = self.lower.cast_to(data_type, cast_options)?; - let upper = self.upper.cast_to(data_type, cast_options)?; - Ok(Interval::new(lower, upper)) - } - - /// This function returns the data type of this interval. If both endpoints - /// do not have the same data type, returns an error. - pub fn get_datatype(&self) -> Result { - let lower_type = self.lower.get_datatype(); - let upper_type = self.upper.get_datatype(); - if lower_type == upper_type { - Ok(lower_type) - } else { - internal_err!( - "Interval bounds have different types: {lower_type} != {upper_type}" - ) - } - } - - /// Decide if this interval is certainly greater than, possibly greater than, - /// or can't be greater than `other` by returning [true, true], - /// [false, true] or [false, false] respectively. - pub(crate) fn gt>(&self, other: T) -> Interval { - let rhs = other.borrow(); - let flags = if !self.upper.is_unbounded() - && !rhs.lower.is_unbounded() - && self.upper.value <= rhs.lower.value - { - // Values in this interval are certainly less than or equal to those - // in the given interval. - (false, false) - } else if !self.lower.is_unbounded() - && !rhs.upper.is_unbounded() - && self.lower.value >= rhs.upper.value - && (self.lower.value > rhs.upper.value || self.lower.open || rhs.upper.open) - { - // Values in this interval are certainly greater than those in the - // given interval. - (true, true) - } else { - // All outcomes are possible. - (false, true) - }; - - Interval::make(Some(flags.0), Some(flags.1), (false, false)) - } - - /// Decide if this interval is certainly greater than or equal to, possibly greater than - /// or equal to, or can't be greater than or equal to `other` by returning [true, true], - /// [false, true] or [false, false] respectively. - pub(crate) fn gt_eq>(&self, other: T) -> Interval { - let rhs = other.borrow(); - let flags = if !self.lower.is_unbounded() - && !rhs.upper.is_unbounded() - && self.lower.value >= rhs.upper.value - { - // Values in this interval are certainly greater than or equal to those - // in the given interval. - (true, true) - } else if !self.upper.is_unbounded() - && !rhs.lower.is_unbounded() - && self.upper.value <= rhs.lower.value - && (self.upper.value < rhs.lower.value || self.upper.open || rhs.lower.open) - { - // Values in this interval are certainly less than those in the - // given interval. - (false, false) - } else { - // All outcomes are possible. - (false, true) - }; - - Interval::make(Some(flags.0), Some(flags.1), (false, false)) - } - - /// Decide if this interval is certainly less than, possibly less than, - /// or can't be less than `other` by returning [true, true], - /// [false, true] or [false, false] respectively. - pub(crate) fn lt>(&self, other: T) -> Interval { - other.borrow().gt(self) - } - - /// Decide if this interval is certainly less than or equal to, possibly - /// less than or equal to, or can't be less than or equal to `other` by returning - /// [true, true], [false, true] or [false, false] respectively. - pub(crate) fn lt_eq>(&self, other: T) -> Interval { - other.borrow().gt_eq(self) - } - - /// Decide if this interval is certainly equal to, possibly equal to, - /// or can't be equal to `other` by returning [true, true], - /// [false, true] or [false, false] respectively. - pub(crate) fn equal>(&self, other: T) -> Interval { - let rhs = other.borrow(); - let flags = if !self.lower.is_unbounded() - && (self.lower.value == self.upper.value) - && (rhs.lower.value == rhs.upper.value) - && (self.lower.value == rhs.lower.value) - { - (true, true) - } else if self.gt(rhs) == Interval::CERTAINLY_TRUE - || self.lt(rhs) == Interval::CERTAINLY_TRUE - { - (false, false) - } else { - (false, true) - }; - - Interval::make(Some(flags.0), Some(flags.1), (false, false)) - } - - /// Compute the logical conjunction of this (boolean) interval with the given boolean interval. - pub(crate) fn and>(&self, other: T) -> Result { - let rhs = other.borrow(); - match ( - &self.lower.value, - &self.upper.value, - &rhs.lower.value, - &rhs.upper.value, - ) { - ( - ScalarValue::Boolean(Some(self_lower)), - ScalarValue::Boolean(Some(self_upper)), - ScalarValue::Boolean(Some(other_lower)), - ScalarValue::Boolean(Some(other_upper)), - ) => { - let lower = *self_lower && *other_lower; - let upper = *self_upper && *other_upper; - - Ok(Interval { - lower: IntervalBound::new(ScalarValue::Boolean(Some(lower)), false), - upper: IntervalBound::new(ScalarValue::Boolean(Some(upper)), false), - }) - } - _ => internal_err!("Incompatible types for logical conjunction"), - } - } - - /// Compute the logical negation of this (boolean) interval. - pub(crate) fn not(&self) -> Result { - if !matches!(self.get_datatype()?, DataType::Boolean) { - return internal_err!( - "Cannot apply logical negation to non-boolean interval" - ); - } - if self == &Interval::CERTAINLY_TRUE { - Ok(Interval::CERTAINLY_FALSE) - } else if self == &Interval::CERTAINLY_FALSE { - Ok(Interval::CERTAINLY_TRUE) - } else { - Ok(Interval::UNCERTAIN) - } - } - - /// Compute the intersection of the interval with the given interval. - /// If the intersection is empty, return None. - pub(crate) fn intersect>( - &self, - other: T, - ) -> Result> { - let rhs = other.borrow(); - // If it is evident that the result is an empty interval, - // do not make any calculation and directly return None. - if (!self.lower.is_unbounded() - && !rhs.upper.is_unbounded() - && self.lower.value > rhs.upper.value) - || (!self.upper.is_unbounded() - && !rhs.lower.is_unbounded() - && self.upper.value < rhs.lower.value) - { - // This None value signals an empty interval. - return Ok(None); - } - - let lower = IntervalBound::choose(&self.lower, &rhs.lower, max)?; - let upper = IntervalBound::choose(&self.upper, &rhs.upper, min)?; - - let non_empty = lower.is_unbounded() - || upper.is_unbounded() - || lower.value != upper.value - || (!lower.open && !upper.open); - Ok(non_empty.then_some(Interval::new(lower, upper))) - } - - /// Decide if this interval is certainly contains, possibly contains, - /// or can't can't `other` by returning [true, true], - /// [false, true] or [false, false] respectively. - pub fn contains>(&self, other: T) -> Result { - match self.intersect(other.borrow())? { - Some(intersection) => { - // Need to compare with same bounds close-ness. - if intersection.close_bounds() == other.borrow().clone().close_bounds() { - Ok(Interval::CERTAINLY_TRUE) - } else { - Ok(Interval::UNCERTAIN) - } - } - None => Ok(Interval::CERTAINLY_FALSE), - } - } - - /// Add the given interval (`other`) to this interval. Say we have - /// intervals [a1, b1] and [a2, b2], then their sum is [a1 + a2, b1 + b2]. - /// Note that this represents all possible values the sum can take if - /// one can choose single values arbitrarily from each of the operands. - pub fn add>(&self, other: T) -> Result { - let rhs = other.borrow(); - Ok(Interval::new( - self.lower.add::(&rhs.lower)?, - self.upper.add::(&rhs.upper)?, - )) - } - - /// Subtract the given interval (`other`) from this interval. Say we have - /// intervals [a1, b1] and [a2, b2], then their sum is [a1 - b2, b1 - a2]. - /// Note that this represents all possible values the difference can take - /// if one can choose single values arbitrarily from each of the operands. - pub fn sub>(&self, other: T) -> Result { - let rhs = other.borrow(); - Ok(Interval::new( - self.lower.sub::(&rhs.upper)?, - self.upper.sub::(&rhs.lower)?, - )) - } - - pub const CERTAINLY_FALSE: Interval = Interval { - lower: IntervalBound::new_closed(ScalarValue::Boolean(Some(false))), - upper: IntervalBound::new_closed(ScalarValue::Boolean(Some(false))), - }; - - pub const UNCERTAIN: Interval = Interval { - lower: IntervalBound::new_closed(ScalarValue::Boolean(Some(false))), - upper: IntervalBound::new_closed(ScalarValue::Boolean(Some(true))), - }; - - pub const CERTAINLY_TRUE: Interval = Interval { - lower: IntervalBound::new_closed(ScalarValue::Boolean(Some(true))), - upper: IntervalBound::new_closed(ScalarValue::Boolean(Some(true))), - }; - - /// Returns the cardinality of this interval, which is the number of all - /// distinct points inside it. This function returns `None` if: - /// - The interval is unbounded from either side, or - /// - Cardinality calculations for the datatype in question is not - /// implemented yet, or - /// - An overflow occurs during the calculation. - /// - /// This function returns an error if the given interval is malformed. - pub fn cardinality(&self) -> Result> { - let data_type = self.get_datatype()?; - if data_type.is_integer() { - Ok(self.upper.value.distance(&self.lower.value).map(|diff| { - calculate_cardinality_based_on_bounds( - self.lower.open, - self.upper.open, - diff as u64, - ) - })) - } - // Ordering floating-point numbers according to their binary representations - // coincide with their natural ordering. Therefore, we can consider their - // binary representations as "indices" and subtract them. For details, see: - // https://stackoverflow.com/questions/8875064/how-many-distinct-floating-point-numbers-in-a-specific-range - else if data_type.is_floating() { - match (&self.lower.value, &self.upper.value) { - ( - ScalarValue::Float32(Some(lower)), - ScalarValue::Float32(Some(upper)), - ) => { - // Negative numbers are sorted in the reverse order. To always have a positive difference after the subtraction, - // we perform following transformation: - let lower_bits = lower.to_bits() as i32; - let upper_bits = upper.to_bits() as i32; - let transformed_lower = - lower_bits ^ ((lower_bits >> 31) & 0x7fffffff); - let transformed_upper = - upper_bits ^ ((upper_bits >> 31) & 0x7fffffff); - let Ok(count) = transformed_upper.sub_checked(transformed_lower) - else { - return Ok(None); - }; - Ok(Some(calculate_cardinality_based_on_bounds( - self.lower.open, - self.upper.open, - count as u64, - ))) - } - ( - ScalarValue::Float64(Some(lower)), - ScalarValue::Float64(Some(upper)), - ) => { - let lower_bits = lower.to_bits() as i64; - let upper_bits = upper.to_bits() as i64; - let transformed_lower = - lower_bits ^ ((lower_bits >> 63) & 0x7fffffffffffffff); - let transformed_upper = - upper_bits ^ ((upper_bits >> 63) & 0x7fffffffffffffff); - let Ok(count) = transformed_upper.sub_checked(transformed_lower) - else { - return Ok(None); - }; - Ok(Some(calculate_cardinality_based_on_bounds( - self.lower.open, - self.upper.open, - count as u64, - ))) - } - _ => Ok(None), - } - } else { - // Cardinality calculations are not implemented for this data type yet: - Ok(None) - } - } - - /// This function "closes" this interval; i.e. it modifies the endpoints so - /// that we end up with the narrowest possible closed interval containing - /// the original interval. - pub fn close_bounds(mut self) -> Interval { - if self.lower.open { - // Get next value - self.lower.value = next_value::(self.lower.value); - self.lower.open = false; - } - - if self.upper.open { - // Get previous value - self.upper.value = next_value::(self.upper.value); - self.upper.open = false; - } - - self - } -} - -trait OneTrait: Sized + std::ops::Add + std::ops::Sub { - fn one() -> Self; -} - -macro_rules! impl_OneTrait{ - ($($m:ty),*) => {$( impl OneTrait for $m { fn one() -> Self { 1 as $m } })*} -} -impl_OneTrait! {u8, u16, u32, u64, i8, i16, i32, i64} - -/// This function either increments or decrements its argument, depending on the `INC` value. -/// If `true`, it increments; otherwise it decrements the argument. -fn increment_decrement( - mut val: T, -) -> T { - if INC { - val.add_assign(T::one()); - } else { - val.sub_assign(T::one()); - } - val -} - -macro_rules! check_infinite_bounds { - ($value:expr, $val:expr, $type:ident, $inc:expr) => { - if ($val == $type::MAX && $inc) || ($val == $type::MIN && !$inc) { - return $value; - } - }; -} - -/// This function returns the next/previous value depending on the `ADD` value. -/// If `true`, it returns the next value; otherwise it returns the previous value. -fn next_value(value: ScalarValue) -> ScalarValue { - use ScalarValue::*; - match value { - Float32(Some(val)) => { - let new_float = if INC { next_up(val) } else { next_down(val) }; - Float32(Some(new_float)) - } - Float64(Some(val)) => { - let new_float = if INC { next_up(val) } else { next_down(val) }; - Float64(Some(new_float)) - } - Int8(Some(val)) => { - check_infinite_bounds!(value, val, i8, INC); - Int8(Some(increment_decrement::(val))) - } - Int16(Some(val)) => { - check_infinite_bounds!(value, val, i16, INC); - Int16(Some(increment_decrement::(val))) - } - Int32(Some(val)) => { - check_infinite_bounds!(value, val, i32, INC); - Int32(Some(increment_decrement::(val))) - } - Int64(Some(val)) => { - check_infinite_bounds!(value, val, i64, INC); - Int64(Some(increment_decrement::(val))) - } - UInt8(Some(val)) => { - check_infinite_bounds!(value, val, u8, INC); - UInt8(Some(increment_decrement::(val))) - } - UInt16(Some(val)) => { - check_infinite_bounds!(value, val, u16, INC); - UInt16(Some(increment_decrement::(val))) - } - UInt32(Some(val)) => { - check_infinite_bounds!(value, val, u32, INC); - UInt32(Some(increment_decrement::(val))) - } - UInt64(Some(val)) => { - check_infinite_bounds!(value, val, u64, INC); - UInt64(Some(increment_decrement::(val))) - } - _ => value, // Unsupported datatypes - } -} - -/// This function computes the selectivity of an operation by computing the -/// cardinality ratio of the given input/output intervals. If this can not be -/// calculated for some reason, it returns `1.0` meaning fullly selective (no -/// filtering). -pub fn cardinality_ratio( - initial_interval: &Interval, - final_interval: &Interval, -) -> Result { - Ok( - match ( - final_interval.cardinality()?, - initial_interval.cardinality()?, - ) { - (Some(final_interval), Some(initial_interval)) => { - final_interval as f64 / initial_interval as f64 - } - _ => 1.0, - }, - ) -} - -pub fn apply_operator(op: &Operator, lhs: &Interval, rhs: &Interval) -> Result { - match *op { - Operator::Eq => Ok(lhs.equal(rhs)), - Operator::NotEq => Ok(lhs.equal(rhs).not()?), - Operator::Gt => Ok(lhs.gt(rhs)), - Operator::GtEq => Ok(lhs.gt_eq(rhs)), - Operator::Lt => Ok(lhs.lt(rhs)), - Operator::LtEq => Ok(lhs.lt_eq(rhs)), - Operator::And => lhs.and(rhs), - Operator::Plus => lhs.add(rhs), - Operator::Minus => lhs.sub(rhs), - _ => Ok(Interval::default()), - } -} - -/// Cast scalar value to the given data type using an arrow kernel. -fn cast_scalar_value( - value: &ScalarValue, - data_type: &DataType, - cast_options: &CastOptions, -) -> Result { - let cast_array = cast_with_options(&value.to_array()?, data_type, cast_options)?; - ScalarValue::try_from_array(&cast_array, 0) -} - -/// This function calculates the final cardinality result by inspecting the endpoints of the interval. -fn calculate_cardinality_based_on_bounds( - lower_open: bool, - upper_open: bool, - diff: u64, -) -> u64 { - match (lower_open, upper_open) { - (false, false) => diff + 1, - (true, true) => diff - 1, - _ => diff, - } -} - -/// An [Interval] that also tracks null status using a boolean interval. -/// -/// This represents values that may be in a particular range or be null. -/// -/// # Examples -/// -/// ``` -/// use arrow::datatypes::DataType; -/// use datafusion_physical_expr::intervals::{Interval, NullableInterval}; -/// use datafusion_common::ScalarValue; -/// -/// // [1, 2) U {NULL} -/// NullableInterval::MaybeNull { -/// values: Interval::make(Some(1), Some(2), (false, true)), -/// }; -/// -/// // (0, ∞) -/// NullableInterval::NotNull { -/// values: Interval::make(Some(0), None, (true, true)), -/// }; -/// -/// // {NULL} -/// NullableInterval::Null { datatype: DataType::Int32 }; -/// -/// // {4} -/// NullableInterval::from(ScalarValue::Int32(Some(4))); -/// ``` -#[derive(Debug, Clone, PartialEq, Eq)] -pub enum NullableInterval { - /// The value is always null in this interval - /// - /// This is typed so it can be used in physical expressions, which don't do - /// type coercion. - Null { datatype: DataType }, - /// The value may or may not be null in this interval. If it is non null its value is within - /// the specified values interval - MaybeNull { values: Interval }, - /// The value is definitely not null in this interval and is within values - NotNull { values: Interval }, -} - -impl Default for NullableInterval { - fn default() -> Self { - NullableInterval::MaybeNull { - values: Interval::default(), - } - } -} - -impl Display for NullableInterval { - fn fmt(&self, f: &mut Formatter<'_>) -> fmt::Result { - match self { - Self::Null { .. } => write!(f, "NullableInterval: {{NULL}}"), - Self::MaybeNull { values } => { - write!(f, "NullableInterval: {} U {{NULL}}", values) - } - Self::NotNull { values } => write!(f, "NullableInterval: {}", values), - } - } -} - -impl From for NullableInterval { - /// Create an interval that represents a single value. - fn from(value: ScalarValue) -> Self { - if value.is_null() { - Self::Null { - datatype: value.data_type(), - } - } else { - Self::NotNull { - values: Interval::new( - IntervalBound::new(value.clone(), false), - IntervalBound::new(value, false), - ), - } - } - } -} - -impl NullableInterval { - /// Get the values interval, or None if this interval is definitely null. - pub fn values(&self) -> Option<&Interval> { - match self { - Self::Null { .. } => None, - Self::MaybeNull { values } | Self::NotNull { values } => Some(values), - } - } - - /// Get the data type - pub fn get_datatype(&self) -> Result { - match self { - Self::Null { datatype } => Ok(datatype.clone()), - Self::MaybeNull { values } | Self::NotNull { values } => { - values.get_datatype() - } - } - } - - /// Return true if the value is definitely true (and not null). - pub fn is_certainly_true(&self) -> bool { - match self { - Self::Null { .. } | Self::MaybeNull { .. } => false, - Self::NotNull { values } => values == &Interval::CERTAINLY_TRUE, - } - } - - /// Return true if the value is definitely false (and not null). - pub fn is_certainly_false(&self) -> bool { - match self { - Self::Null { .. } => false, - Self::MaybeNull { .. } => false, - Self::NotNull { values } => values == &Interval::CERTAINLY_FALSE, - } - } - - /// Perform logical negation on a boolean nullable interval. - fn not(&self) -> Result { - match self { - Self::Null { datatype } => Ok(Self::Null { - datatype: datatype.clone(), - }), - Self::MaybeNull { values } => Ok(Self::MaybeNull { - values: values.not()?, - }), - Self::NotNull { values } => Ok(Self::NotNull { - values: values.not()?, - }), - } - } - - /// Apply the given operator to this interval and the given interval. - /// - /// # Examples - /// - /// ``` - /// use datafusion_common::ScalarValue; - /// use datafusion_expr::Operator; - /// use datafusion_physical_expr::intervals::{Interval, NullableInterval}; - /// - /// // 4 > 3 -> true - /// let lhs = NullableInterval::from(ScalarValue::Int32(Some(4))); - /// let rhs = NullableInterval::from(ScalarValue::Int32(Some(3))); - /// let result = lhs.apply_operator(&Operator::Gt, &rhs).unwrap(); - /// assert_eq!(result, NullableInterval::from(ScalarValue::Boolean(Some(true)))); - /// - /// // [1, 3) > NULL -> NULL - /// let lhs = NullableInterval::NotNull { - /// values: Interval::make(Some(1), Some(3), (false, true)), - /// }; - /// let rhs = NullableInterval::from(ScalarValue::Int32(None)); - /// let result = lhs.apply_operator(&Operator::Gt, &rhs).unwrap(); - /// assert_eq!(result.single_value(), Some(ScalarValue::Boolean(None))); - /// - /// // [1, 3] > [2, 4] -> [false, true] - /// let lhs = NullableInterval::NotNull { - /// values: Interval::make(Some(1), Some(3), (false, false)), - /// }; - /// let rhs = NullableInterval::NotNull { - /// values: Interval::make(Some(2), Some(4), (false, false)), - /// }; - /// let result = lhs.apply_operator(&Operator::Gt, &rhs).unwrap(); - /// // Both inputs are valid (non-null), so result must be non-null - /// assert_eq!(result, NullableInterval::NotNull { - /// // Uncertain whether inequality is true or false - /// values: Interval::UNCERTAIN, - /// }); - /// - /// ``` - pub fn apply_operator(&self, op: &Operator, rhs: &Self) -> Result { - match op { - Operator::IsDistinctFrom => { - let values = match (self, rhs) { - // NULL is distinct from NULL -> False - (Self::Null { .. }, Self::Null { .. }) => Interval::CERTAINLY_FALSE, - // x is distinct from y -> x != y, - // if at least one of them is never null. - (Self::NotNull { .. }, _) | (_, Self::NotNull { .. }) => { - let lhs_values = self.values(); - let rhs_values = rhs.values(); - match (lhs_values, rhs_values) { - (Some(lhs_values), Some(rhs_values)) => { - lhs_values.equal(rhs_values).not()? - } - (Some(_), None) | (None, Some(_)) => Interval::CERTAINLY_TRUE, - (None, None) => unreachable!("Null case handled above"), - } - } - _ => Interval::UNCERTAIN, - }; - // IsDistinctFrom never returns null. - Ok(Self::NotNull { values }) - } - Operator::IsNotDistinctFrom => self - .apply_operator(&Operator::IsDistinctFrom, rhs) - .map(|i| i.not())?, - _ => { - if let (Some(left_values), Some(right_values)) = - (self.values(), rhs.values()) - { - let values = apply_operator(op, left_values, right_values)?; - match (self, rhs) { - (Self::NotNull { .. }, Self::NotNull { .. }) => { - Ok(Self::NotNull { values }) - } - _ => Ok(Self::MaybeNull { values }), - } - } else if op.is_comparison_operator() { - Ok(Self::Null { - datatype: DataType::Boolean, - }) - } else { - Ok(Self::Null { - datatype: self.get_datatype()?, - }) - } - } - } - } - - /// Determine if this interval contains the given interval. Returns a boolean - /// interval that is [true, true] if this interval is a superset of the - /// given interval, [false, false] if this interval is disjoint from the - /// given interval, and [false, true] otherwise. - pub fn contains>(&self, other: T) -> Result { - let rhs = other.borrow(); - if let (Some(left_values), Some(right_values)) = (self.values(), rhs.values()) { - let values = left_values.contains(right_values)?; - match (self, rhs) { - (Self::NotNull { .. }, Self::NotNull { .. }) => { - Ok(Self::NotNull { values }) - } - _ => Ok(Self::MaybeNull { values }), - } - } else { - Ok(Self::Null { - datatype: DataType::Boolean, - }) - } - } - - /// If the interval has collapsed to a single value, return that value. - /// - /// Otherwise returns None. - /// - /// # Examples - /// - /// ``` - /// use datafusion_common::ScalarValue; - /// use datafusion_physical_expr::intervals::{Interval, NullableInterval}; - /// - /// let interval = NullableInterval::from(ScalarValue::Int32(Some(4))); - /// assert_eq!(interval.single_value(), Some(ScalarValue::Int32(Some(4)))); - /// - /// let interval = NullableInterval::from(ScalarValue::Int32(None)); - /// assert_eq!(interval.single_value(), Some(ScalarValue::Int32(None))); - /// - /// let interval = NullableInterval::MaybeNull { - /// values: Interval::make(Some(1), Some(4), (false, true)), - /// }; - /// assert_eq!(interval.single_value(), None); - /// ``` - pub fn single_value(&self) -> Option { - match self { - Self::Null { datatype } => { - Some(ScalarValue::try_from(datatype).unwrap_or(ScalarValue::Null)) - } - Self::MaybeNull { values } | Self::NotNull { values } - if values.lower.value == values.upper.value - && !values.lower.is_unbounded() => - { - Some(values.lower.value.clone()) - } - _ => None, - } - } -} - -#[cfg(test)] -mod tests { - use super::next_value; - use crate::intervals::{Interval, IntervalBound}; - use arrow_schema::DataType; - use datafusion_common::{Result, ScalarValue}; - - fn open_open(lower: Option, upper: Option) -> Interval - where - ScalarValue: From>, - { - Interval::make(lower, upper, (true, true)) - } - - fn open_closed(lower: Option, upper: Option) -> Interval - where - ScalarValue: From>, - { - Interval::make(lower, upper, (true, false)) - } - - fn closed_open(lower: Option, upper: Option) -> Interval - where - ScalarValue: From>, - { - Interval::make(lower, upper, (false, true)) - } - - fn closed_closed(lower: Option, upper: Option) -> Interval - where - ScalarValue: From>, - { - Interval::make(lower, upper, (false, false)) - } - - #[test] - fn intersect_test() -> Result<()> { - let possible_cases = vec![ - (Some(1000_i64), None, None, None, Some(1000_i64), None), - (None, Some(1000_i64), None, None, None, Some(1000_i64)), - (None, None, Some(1000_i64), None, Some(1000_i64), None), - (None, None, None, Some(1000_i64), None, Some(1000_i64)), - ( - Some(1000_i64), - None, - Some(1000_i64), - None, - Some(1000_i64), - None, - ), - ( - None, - Some(1000_i64), - Some(999_i64), - Some(1002_i64), - Some(999_i64), - Some(1000_i64), - ), - (None, None, None, None, None, None), - ]; - - for case in possible_cases { - assert_eq!( - open_open(case.0, case.1).intersect(open_open(case.2, case.3))?, - Some(open_open(case.4, case.5)) - ) - } - - let empty_cases = vec![ - (None, Some(1000_i64), Some(1001_i64), None), - (Some(1001_i64), None, None, Some(1000_i64)), - (None, Some(1000_i64), Some(1001_i64), Some(1002_i64)), - (Some(1001_i64), Some(1002_i64), None, Some(1000_i64)), - ]; - - for case in empty_cases { - assert_eq!( - open_open(case.0, case.1).intersect(open_open(case.2, case.3))?, - None - ) - } - - Ok(()) - } - - #[test] - fn gt_test() { - let cases = vec![ - (Some(1000_i64), None, None, None, false, true), - (None, Some(1000_i64), None, None, false, true), - (None, None, Some(1000_i64), None, false, true), - (None, None, None, Some(1000_i64), false, true), - (None, Some(1000_i64), Some(1000_i64), None, false, false), - (None, Some(1000_i64), Some(1001_i64), None, false, false), - (Some(1000_i64), None, Some(1000_i64), None, false, true), - ( - None, - Some(1000_i64), - Some(1001_i64), - Some(1002_i64), - false, - false, - ), - ( - None, - Some(1000_i64), - Some(999_i64), - Some(1002_i64), - false, - true, - ), - ( - Some(1002_i64), - None, - Some(999_i64), - Some(1002_i64), - true, - true, - ), - ( - Some(1003_i64), - None, - Some(999_i64), - Some(1002_i64), - true, - true, - ), - (None, None, None, None, false, true), - ]; - - for case in cases { - assert_eq!( - open_open(case.0, case.1).gt(open_open(case.2, case.3)), - closed_closed(Some(case.4), Some(case.5)) - ); - } - } - - #[test] - fn lt_test() { - let cases = vec![ - (Some(1000_i64), None, None, None, false, true), - (None, Some(1000_i64), None, None, false, true), - (None, None, Some(1000_i64), None, false, true), - (None, None, None, Some(1000_i64), false, true), - (None, Some(1000_i64), Some(1000_i64), None, true, true), - (None, Some(1000_i64), Some(1001_i64), None, true, true), - (Some(1000_i64), None, Some(1000_i64), None, false, true), - ( - None, - Some(1000_i64), - Some(1001_i64), - Some(1002_i64), - true, - true, - ), - ( - None, - Some(1000_i64), - Some(999_i64), - Some(1002_i64), - false, - true, - ), - (None, None, None, None, false, true), - ]; - - for case in cases { - assert_eq!( - open_open(case.0, case.1).lt(open_open(case.2, case.3)), - closed_closed(Some(case.4), Some(case.5)) - ); - } - } - - #[test] - fn and_test() -> Result<()> { - let cases = vec![ - (false, true, false, false, false, false), - (false, false, false, true, false, false), - (false, true, false, true, false, true), - (false, true, true, true, false, true), - (false, false, false, false, false, false), - (true, true, true, true, true, true), - ]; - - for case in cases { - assert_eq!( - open_open(Some(case.0), Some(case.1)) - .and(open_open(Some(case.2), Some(case.3)))?, - open_open(Some(case.4), Some(case.5)) - ); - } - Ok(()) - } - - #[test] - fn add_test() -> Result<()> { - let cases = vec![ - (Some(1000_i64), None, None, None, None, None), - (None, Some(1000_i64), None, None, None, None), - (None, None, Some(1000_i64), None, None, None), - (None, None, None, Some(1000_i64), None, None), - ( - Some(1000_i64), - None, - Some(1000_i64), - None, - Some(2000_i64), - None, - ), - ( - None, - Some(1000_i64), - Some(999_i64), - Some(1002_i64), - None, - Some(2002_i64), - ), - (None, Some(1000_i64), Some(1000_i64), None, None, None), - ( - Some(2001_i64), - Some(1_i64), - Some(1005_i64), - Some(-999_i64), - Some(3006_i64), - Some(-998_i64), - ), - (None, None, None, None, None, None), - ]; - - for case in cases { - assert_eq!( - open_open(case.0, case.1).add(open_open(case.2, case.3))?, - open_open(case.4, case.5) - ); - } - Ok(()) - } - - #[test] - fn sub_test() -> Result<()> { - let cases = vec![ - (Some(1000_i64), None, None, None, None, None), - (None, Some(1000_i64), None, None, None, None), - (None, None, Some(1000_i64), None, None, None), - (None, None, None, Some(1000_i64), None, None), - (Some(1000_i64), None, Some(1000_i64), None, None, None), - ( - None, - Some(1000_i64), - Some(999_i64), - Some(1002_i64), - None, - Some(1_i64), - ), - ( - None, - Some(1000_i64), - Some(1000_i64), - None, - None, - Some(0_i64), - ), - ( - Some(2001_i64), - Some(1000_i64), - Some(1005), - Some(999_i64), - Some(1002_i64), - Some(-5_i64), - ), - (None, None, None, None, None, None), - ]; - - for case in cases { - assert_eq!( - open_open(case.0, case.1).sub(open_open(case.2, case.3))?, - open_open(case.4, case.5) - ); - } - Ok(()) - } - - #[test] - fn sub_test_various_bounds() -> Result<()> { - let cases = vec![ - ( - closed_closed(Some(100_i64), Some(200_i64)), - closed_open(Some(200_i64), None), - open_closed(None, Some(0_i64)), - ), - ( - closed_open(Some(100_i64), Some(200_i64)), - open_closed(Some(300_i64), Some(150_i64)), - closed_open(Some(-50_i64), Some(-100_i64)), - ), - ( - closed_open(Some(100_i64), Some(200_i64)), - open_open(Some(200_i64), None), - open_open(None, Some(0_i64)), - ), - ( - closed_closed(Some(1_i64), Some(1_i64)), - closed_closed(Some(11_i64), Some(11_i64)), - closed_closed(Some(-10_i64), Some(-10_i64)), - ), - ]; - for case in cases { - assert_eq!(case.0.sub(case.1)?, case.2) - } - Ok(()) - } - - #[test] - fn add_test_various_bounds() -> Result<()> { - let cases = vec![ - ( - closed_closed(Some(100_i64), Some(200_i64)), - open_closed(None, Some(200_i64)), - open_closed(None, Some(400_i64)), - ), - ( - closed_open(Some(100_i64), Some(200_i64)), - closed_open(Some(-300_i64), Some(150_i64)), - closed_open(Some(-200_i64), Some(350_i64)), - ), - ( - closed_open(Some(100_i64), Some(200_i64)), - open_open(Some(200_i64), None), - open_open(Some(300_i64), None), - ), - ( - closed_closed(Some(1_i64), Some(1_i64)), - closed_closed(Some(11_i64), Some(11_i64)), - closed_closed(Some(12_i64), Some(12_i64)), - ), - ]; - for case in cases { - assert_eq!(case.0.add(case.1)?, case.2) - } - Ok(()) - } - - #[test] - fn lt_test_various_bounds() -> Result<()> { - let cases = vec![ - ( - closed_closed(Some(100_i64), Some(200_i64)), - open_closed(None, Some(100_i64)), - closed_closed(Some(false), Some(false)), - ), - ( - closed_closed(Some(100_i64), Some(200_i64)), - open_open(None, Some(100_i64)), - closed_closed(Some(false), Some(false)), - ), - ( - open_open(Some(100_i64), Some(200_i64)), - closed_closed(Some(0_i64), Some(100_i64)), - closed_closed(Some(false), Some(false)), - ), - ( - closed_closed(Some(2_i64), Some(2_i64)), - closed_closed(Some(1_i64), Some(2_i64)), - closed_closed(Some(false), Some(false)), - ), - ( - closed_closed(Some(2_i64), Some(2_i64)), - closed_open(Some(1_i64), Some(2_i64)), - closed_closed(Some(false), Some(false)), - ), - ( - closed_closed(Some(1_i64), Some(1_i64)), - open_open(Some(1_i64), Some(2_i64)), - closed_closed(Some(true), Some(true)), - ), - ]; - for case in cases { - assert_eq!(case.0.lt(case.1), case.2) - } - Ok(()) - } - - #[test] - fn gt_test_various_bounds() -> Result<()> { - let cases = vec![ - ( - closed_closed(Some(100_i64), Some(200_i64)), - open_closed(None, Some(100_i64)), - closed_closed(Some(false), Some(true)), - ), - ( - closed_closed(Some(100_i64), Some(200_i64)), - open_open(None, Some(100_i64)), - closed_closed(Some(true), Some(true)), - ), - ( - open_open(Some(100_i64), Some(200_i64)), - closed_closed(Some(0_i64), Some(100_i64)), - closed_closed(Some(true), Some(true)), - ), - ( - closed_closed(Some(2_i64), Some(2_i64)), - closed_closed(Some(1_i64), Some(2_i64)), - closed_closed(Some(false), Some(true)), - ), - ( - closed_closed(Some(2_i64), Some(2_i64)), - closed_open(Some(1_i64), Some(2_i64)), - closed_closed(Some(true), Some(true)), - ), - ( - closed_closed(Some(1_i64), Some(1_i64)), - open_open(Some(1_i64), Some(2_i64)), - closed_closed(Some(false), Some(false)), - ), - ]; - for case in cases { - assert_eq!(case.0.gt(case.1), case.2) - } - Ok(()) - } - - #[test] - fn lt_eq_test_various_bounds() -> Result<()> { - let cases = vec![ - ( - closed_closed(Some(100_i64), Some(200_i64)), - open_closed(None, Some(100_i64)), - closed_closed(Some(false), Some(true)), - ), - ( - closed_closed(Some(100_i64), Some(200_i64)), - open_open(None, Some(100_i64)), - closed_closed(Some(false), Some(false)), - ), - ( - closed_closed(Some(2_i64), Some(2_i64)), - closed_closed(Some(1_i64), Some(2_i64)), - closed_closed(Some(false), Some(true)), - ), - ( - closed_closed(Some(2_i64), Some(2_i64)), - closed_open(Some(1_i64), Some(2_i64)), - closed_closed(Some(false), Some(false)), - ), - ( - closed_closed(Some(1_i64), Some(1_i64)), - closed_open(Some(1_i64), Some(2_i64)), - closed_closed(Some(true), Some(true)), - ), - ( - closed_closed(Some(1_i64), Some(1_i64)), - open_open(Some(1_i64), Some(2_i64)), - closed_closed(Some(true), Some(true)), - ), - ]; - for case in cases { - assert_eq!(case.0.lt_eq(case.1), case.2) - } - Ok(()) - } - - #[test] - fn gt_eq_test_various_bounds() -> Result<()> { - let cases = vec![ - ( - closed_closed(Some(100_i64), Some(200_i64)), - open_closed(None, Some(100_i64)), - closed_closed(Some(true), Some(true)), - ), - ( - closed_closed(Some(100_i64), Some(200_i64)), - open_open(None, Some(100_i64)), - closed_closed(Some(true), Some(true)), - ), - ( - closed_closed(Some(2_i64), Some(2_i64)), - closed_closed(Some(1_i64), Some(2_i64)), - closed_closed(Some(true), Some(true)), - ), - ( - closed_closed(Some(2_i64), Some(2_i64)), - closed_open(Some(1_i64), Some(2_i64)), - closed_closed(Some(true), Some(true)), - ), - ( - closed_closed(Some(1_i64), Some(1_i64)), - closed_open(Some(1_i64), Some(2_i64)), - closed_closed(Some(false), Some(true)), - ), - ( - closed_closed(Some(1_i64), Some(1_i64)), - open_open(Some(1_i64), Some(2_i64)), - closed_closed(Some(false), Some(false)), - ), - ]; - for case in cases { - assert_eq!(case.0.gt_eq(case.1), case.2) - } - Ok(()) - } - - #[test] - fn intersect_test_various_bounds() -> Result<()> { - let cases = vec![ - ( - closed_closed(Some(100_i64), Some(200_i64)), - open_closed(None, Some(100_i64)), - Some(closed_closed(Some(100_i64), Some(100_i64))), - ), - ( - closed_closed(Some(100_i64), Some(200_i64)), - open_open(None, Some(100_i64)), - None, - ), - ( - open_open(Some(100_i64), Some(200_i64)), - closed_closed(Some(0_i64), Some(100_i64)), - None, - ), - ( - closed_closed(Some(2_i64), Some(2_i64)), - closed_closed(Some(1_i64), Some(2_i64)), - Some(closed_closed(Some(2_i64), Some(2_i64))), - ), - ( - closed_closed(Some(2_i64), Some(2_i64)), - closed_open(Some(1_i64), Some(2_i64)), - None, - ), - ( - closed_closed(Some(1_i64), Some(1_i64)), - open_open(Some(1_i64), Some(2_i64)), - None, - ), - ( - closed_closed(Some(1_i64), Some(3_i64)), - open_open(Some(1_i64), Some(2_i64)), - Some(open_open(Some(1_i64), Some(2_i64))), - ), - ]; - for case in cases { - assert_eq!(case.0.intersect(case.1)?, case.2) - } - Ok(()) - } - - // This function tests if valid constructions produce standardized objects - // ([false, false], [false, true], [true, true]) for boolean intervals. - #[test] - fn non_standard_interval_constructs() { - use ScalarValue::Boolean; - let cases = vec![ - ( - IntervalBound::new(Boolean(None), true), - IntervalBound::new(Boolean(Some(true)), false), - closed_closed(Some(false), Some(true)), - ), - ( - IntervalBound::new(Boolean(None), true), - IntervalBound::new(Boolean(Some(true)), true), - closed_closed(Some(false), Some(false)), - ), - ( - IntervalBound::new(Boolean(Some(false)), false), - IntervalBound::new(Boolean(None), true), - closed_closed(Some(false), Some(true)), - ), - ( - IntervalBound::new(Boolean(Some(true)), false), - IntervalBound::new(Boolean(None), true), - closed_closed(Some(true), Some(true)), - ), - ( - IntervalBound::new(Boolean(None), true), - IntervalBound::new(Boolean(None), true), - closed_closed(Some(false), Some(true)), - ), - ( - IntervalBound::new(Boolean(Some(false)), true), - IntervalBound::new(Boolean(None), true), - closed_closed(Some(true), Some(true)), - ), - ]; - - for case in cases { - assert_eq!(Interval::new(case.0, case.1), case.2) - } - } - - macro_rules! capture_mode_change { - ($TYPE:ty) => { - paste::item! { - capture_mode_change_helper!([], - [], - $TYPE); - } - }; - } - - macro_rules! capture_mode_change_helper { - ($TEST_FN_NAME:ident, $CREATE_FN_NAME:ident, $TYPE:ty) => { - fn $CREATE_FN_NAME(lower: $TYPE, upper: $TYPE) -> Interval { - Interval::make(Some(lower as $TYPE), Some(upper as $TYPE), (true, true)) - } - - fn $TEST_FN_NAME(input: ($TYPE, $TYPE), expect_low: bool, expect_high: bool) { - assert!(expect_low || expect_high); - let interval1 = $CREATE_FN_NAME(input.0, input.0); - let interval2 = $CREATE_FN_NAME(input.1, input.1); - let result = interval1.add(&interval2).unwrap(); - let without_fe = $CREATE_FN_NAME(input.0 + input.1, input.0 + input.1); - assert!( - (!expect_low || result.lower.value < without_fe.lower.value) - && (!expect_high || result.upper.value > without_fe.upper.value) - ); - } - }; - } - - capture_mode_change!(f32); - capture_mode_change!(f64); - - #[cfg(all( - any(target_arch = "x86_64", target_arch = "aarch64"), - not(target_os = "windows") - ))] - #[test] - fn test_add_intervals_lower_affected_f32() { - // Lower is affected - let lower = f32::from_bits(1073741887); //1000000000000000000000000111111 - let upper = f32::from_bits(1098907651); //1000001100000000000000000000011 - capture_mode_change_f32((lower, upper), true, false); - - // Upper is affected - let lower = f32::from_bits(1072693248); //111111111100000000000000000000 - let upper = f32::from_bits(715827883); //101010101010101010101010101011 - capture_mode_change_f32((lower, upper), false, true); - - // Lower is affected - let lower = 1.0; // 0x3FF0000000000000 - let upper = 0.3; // 0x3FD3333333333333 - capture_mode_change_f64((lower, upper), true, false); - - // Upper is affected - let lower = 1.4999999999999998; // 0x3FF7FFFFFFFFFFFF - let upper = 0.000_000_000_000_000_022_044_604_925_031_31; // 0x3C796A6B413BB21F - capture_mode_change_f64((lower, upper), false, true); - } - - #[cfg(any( - not(any(target_arch = "x86_64", target_arch = "aarch64")), - target_os = "windows" - ))] - #[test] - fn test_next_impl_add_intervals_f64() { - let lower = 1.5; - let upper = 1.5; - capture_mode_change_f64((lower, upper), true, true); - - let lower = 1.5; - let upper = 1.5; - capture_mode_change_f32((lower, upper), true, true); - } - - #[test] - fn test_cardinality_of_intervals() -> Result<()> { - // In IEEE 754 standard for floating-point arithmetic, if we keep the sign and exponent fields same, - // we can represent 4503599627370496 different numbers by changing the mantissa - // (4503599627370496 = 2^52, since there are 52 bits in mantissa, and 2^23 = 8388608 for f32). - let distinct_f64 = 4503599627370496; - let distinct_f32 = 8388608; - let intervals = [ - Interval::new( - IntervalBound::new(ScalarValue::from(0.25), false), - IntervalBound::new(ScalarValue::from(0.50), true), - ), - Interval::new( - IntervalBound::new(ScalarValue::from(0.5), false), - IntervalBound::new(ScalarValue::from(1.0), true), - ), - Interval::new( - IntervalBound::new(ScalarValue::from(1.0), false), - IntervalBound::new(ScalarValue::from(2.0), true), - ), - Interval::new( - IntervalBound::new(ScalarValue::from(32.0), false), - IntervalBound::new(ScalarValue::from(64.0), true), - ), - Interval::new( - IntervalBound::new(ScalarValue::from(-0.50), false), - IntervalBound::new(ScalarValue::from(-0.25), true), - ), - Interval::new( - IntervalBound::new(ScalarValue::from(-32.0), false), - IntervalBound::new(ScalarValue::from(-16.0), true), - ), - ]; - for interval in intervals { - assert_eq!(interval.cardinality()?.unwrap(), distinct_f64); - } - - let intervals = [ - Interval::new( - IntervalBound::new(ScalarValue::from(0.25_f32), false), - IntervalBound::new(ScalarValue::from(0.50_f32), true), - ), - Interval::new( - IntervalBound::new(ScalarValue::from(-1_f32), false), - IntervalBound::new(ScalarValue::from(-0.5_f32), true), - ), - ]; - for interval in intervals { - assert_eq!(interval.cardinality()?.unwrap(), distinct_f32); - } - - // The regular logarithmic distribution of floating-point numbers are - // only applicable outside of the `(-phi, phi)` interval where `phi` - // denotes the largest positive subnormal floating-point number. Since - // the following intervals include such subnormal points, we cannot use - // a simple powers-of-two type formula for our expectations. Therefore, - // we manually supply the actual expected cardinality. - let interval = Interval::new( - IntervalBound::new(ScalarValue::from(-0.0625), false), - IntervalBound::new(ScalarValue::from(0.0625), true), - ); - assert_eq!(interval.cardinality()?.unwrap(), 9178336040581070849); - - let interval = Interval::new( - IntervalBound::new(ScalarValue::from(-0.0625_f32), false), - IntervalBound::new(ScalarValue::from(0.0625_f32), true), - ); - assert_eq!(interval.cardinality()?.unwrap(), 2063597569); - - Ok(()) - } - - #[test] - fn test_next_value() -> Result<()> { - // integer increment / decrement - let zeros = vec![ - ScalarValue::new_zero(&DataType::UInt8)?, - ScalarValue::new_zero(&DataType::UInt16)?, - ScalarValue::new_zero(&DataType::UInt32)?, - ScalarValue::new_zero(&DataType::UInt64)?, - ScalarValue::new_zero(&DataType::Int8)?, - ScalarValue::new_zero(&DataType::Int8)?, - ScalarValue::new_zero(&DataType::Int8)?, - ScalarValue::new_zero(&DataType::Int8)?, - ]; - - let ones = vec![ - ScalarValue::new_one(&DataType::UInt8)?, - ScalarValue::new_one(&DataType::UInt16)?, - ScalarValue::new_one(&DataType::UInt32)?, - ScalarValue::new_one(&DataType::UInt64)?, - ScalarValue::new_one(&DataType::Int8)?, - ScalarValue::new_one(&DataType::Int8)?, - ScalarValue::new_one(&DataType::Int8)?, - ScalarValue::new_one(&DataType::Int8)?, - ]; - - zeros.into_iter().zip(ones).for_each(|(z, o)| { - assert_eq!(next_value::(z.clone()), o); - assert_eq!(next_value::(o), z); - }); - - // floating value increment / decrement - let values = vec![ - ScalarValue::new_zero(&DataType::Float32)?, - ScalarValue::new_zero(&DataType::Float64)?, - ]; - - let eps = vec![ - ScalarValue::Float32(Some(1e-6)), - ScalarValue::Float64(Some(1e-6)), - ]; - - values.into_iter().zip(eps).for_each(|(v, e)| { - assert!(next_value::(v.clone()).sub(v.clone()).unwrap().lt(&e)); - assert!(v.clone().sub(next_value::(v)).unwrap().lt(&e)); - }); - - // Min / Max values do not change for integer values - let min = vec![ - ScalarValue::UInt64(Some(u64::MIN)), - ScalarValue::Int8(Some(i8::MIN)), - ]; - let max = vec![ - ScalarValue::UInt64(Some(u64::MAX)), - ScalarValue::Int8(Some(i8::MAX)), - ]; - - min.into_iter().zip(max).for_each(|(min, max)| { - assert_eq!(next_value::(max.clone()), max); - assert_eq!(next_value::(min.clone()), min); - }); - - // Min / Max values results in infinity for floating point values - assert_eq!( - next_value::(ScalarValue::Float32(Some(f32::MAX))), - ScalarValue::Float32(Some(f32::INFINITY)) - ); - assert_eq!( - next_value::(ScalarValue::Float64(Some(f64::MIN))), - ScalarValue::Float64(Some(f64::NEG_INFINITY)) - ); - - Ok(()) - } - - #[test] - fn test_interval_display() { - let interval = Interval::new( - IntervalBound::new(ScalarValue::from(0.25_f32), true), - IntervalBound::new(ScalarValue::from(0.50_f32), false), - ); - assert_eq!(format!("{}", interval), "(0.25, 0.5]"); - - let interval = Interval::new( - IntervalBound::new(ScalarValue::from(0.25_f32), false), - IntervalBound::new(ScalarValue::from(0.50_f32), true), - ); - assert_eq!(format!("{}", interval), "[0.25, 0.5)"); - - let interval = Interval::new( - IntervalBound::new(ScalarValue::from(0.25_f32), true), - IntervalBound::new(ScalarValue::from(0.50_f32), true), - ); - assert_eq!(format!("{}", interval), "(0.25, 0.5)"); - - let interval = Interval::new( - IntervalBound::new(ScalarValue::from(0.25_f32), false), - IntervalBound::new(ScalarValue::from(0.50_f32), false), - ); - assert_eq!(format!("{}", interval), "[0.25, 0.5]"); - } -} diff --git a/datafusion/physical-expr/src/intervals/mod.rs b/datafusion/physical-expr/src/intervals/mod.rs index b89d1c59dc64..9752ca27b5a3 100644 --- a/datafusion/physical-expr/src/intervals/mod.rs +++ b/datafusion/physical-expr/src/intervals/mod.rs @@ -18,10 +18,5 @@ //! Interval arithmetic and constraint propagation library pub mod cp_solver; -pub mod interval_aritmetic; -pub mod rounding; pub mod test_utils; pub mod utils; - -pub use cp_solver::ExprIntervalGraph; -pub use interval_aritmetic::*; diff --git a/datafusion/physical-expr/src/intervals/utils.rs b/datafusion/physical-expr/src/intervals/utils.rs index 7a4ccff950e6..03d13632104d 100644 --- a/datafusion/physical-expr/src/intervals/utils.rs +++ b/datafusion/physical-expr/src/intervals/utils.rs @@ -19,14 +19,16 @@ use std::sync::Arc; -use super::{Interval, IntervalBound}; use crate::{ expressions::{BinaryExpr, CastExpr, Column, Literal, NegativeExpr}, PhysicalExpr, }; use arrow_schema::{DataType, SchemaRef}; -use datafusion_common::{DataFusionError, Result, ScalarValue}; +use datafusion_common::{ + internal_datafusion_err, internal_err, DataFusionError, Result, ScalarValue, +}; +use datafusion_expr::interval_arithmetic::Interval; use datafusion_expr::Operator; const MDN_DAY_MASK: i128 = 0xFFFF_FFFF_0000_0000_0000_0000; @@ -66,11 +68,13 @@ pub fn check_support(expr: &Arc, schema: &SchemaRef) -> bool { } // This function returns the inverse operator of the given operator. -pub fn get_inverse_op(op: Operator) -> Operator { +pub fn get_inverse_op(op: Operator) -> Result { match op { - Operator::Plus => Operator::Minus, - Operator::Minus => Operator::Plus, - _ => unreachable!(), + Operator::Plus => Ok(Operator::Minus), + Operator::Minus => Ok(Operator::Plus), + Operator::Multiply => Ok(Operator::Divide), + Operator::Divide => Ok(Operator::Multiply), + _ => internal_err!("Interval arithmetic does not support the operator {}", op), } } @@ -86,6 +90,8 @@ pub fn is_operator_supported(op: &Operator) -> bool { | &Operator::Lt | &Operator::LtEq | &Operator::Eq + | &Operator::Multiply + | &Operator::Divide ) } @@ -109,36 +115,26 @@ pub fn is_datatype_supported(data_type: &DataType) -> bool { /// Converts an [`Interval`] of time intervals to one of `Duration`s, if applicable. Otherwise, returns [`None`]. pub fn convert_interval_type_to_duration(interval: &Interval) -> Option { if let (Some(lower), Some(upper)) = ( - convert_interval_bound_to_duration(&interval.lower), - convert_interval_bound_to_duration(&interval.upper), + convert_interval_bound_to_duration(interval.lower()), + convert_interval_bound_to_duration(interval.upper()), ) { - Some(Interval::new(lower, upper)) + Interval::try_new(lower, upper).ok() } else { None } } -/// Converts an [`IntervalBound`] containing a time interval to one containing a `Duration`, if applicable. Otherwise, returns [`None`]. +/// Converts an [`ScalarValue`] containing a time interval to one containing a `Duration`, if applicable. Otherwise, returns [`None`]. fn convert_interval_bound_to_duration( - interval_bound: &IntervalBound, -) -> Option { - match interval_bound.value { - ScalarValue::IntervalMonthDayNano(Some(mdn)) => { - interval_mdn_to_duration_ns(&mdn).ok().map(|duration| { - IntervalBound::new( - ScalarValue::DurationNanosecond(Some(duration)), - interval_bound.open, - ) - }) - } - ScalarValue::IntervalDayTime(Some(dt)) => { - interval_dt_to_duration_ms(&dt).ok().map(|duration| { - IntervalBound::new( - ScalarValue::DurationMillisecond(Some(duration)), - interval_bound.open, - ) - }) - } + interval_bound: &ScalarValue, +) -> Option { + match interval_bound { + ScalarValue::IntervalMonthDayNano(Some(mdn)) => interval_mdn_to_duration_ns(mdn) + .ok() + .map(|duration| ScalarValue::DurationNanosecond(Some(duration))), + ScalarValue::IntervalDayTime(Some(dt)) => interval_dt_to_duration_ms(dt) + .ok() + .map(|duration| ScalarValue::DurationMillisecond(Some(duration))), _ => None, } } @@ -146,28 +142,32 @@ fn convert_interval_bound_to_duration( /// Converts an [`Interval`] of `Duration`s to one of time intervals, if applicable. Otherwise, returns [`None`]. pub fn convert_duration_type_to_interval(interval: &Interval) -> Option { if let (Some(lower), Some(upper)) = ( - convert_duration_bound_to_interval(&interval.lower), - convert_duration_bound_to_interval(&interval.upper), + convert_duration_bound_to_interval(interval.lower()), + convert_duration_bound_to_interval(interval.upper()), ) { - Some(Interval::new(lower, upper)) + Interval::try_new(lower, upper).ok() } else { None } } -/// Converts an [`IntervalBound`] containing a `Duration` to one containing a time interval, if applicable. Otherwise, returns [`None`]. +/// Converts a [`ScalarValue`] containing a `Duration` to one containing a time interval, if applicable. Otherwise, returns [`None`]. fn convert_duration_bound_to_interval( - interval_bound: &IntervalBound, -) -> Option { - match interval_bound.value { - ScalarValue::DurationNanosecond(Some(duration)) => Some(IntervalBound::new( - ScalarValue::new_interval_mdn(0, 0, duration), - interval_bound.open, - )), - ScalarValue::DurationMillisecond(Some(duration)) => Some(IntervalBound::new( - ScalarValue::new_interval_dt(0, duration as i32), - interval_bound.open, - )), + interval_bound: &ScalarValue, +) -> Option { + match interval_bound { + ScalarValue::DurationNanosecond(Some(duration)) => { + Some(ScalarValue::new_interval_mdn(0, 0, *duration)) + } + ScalarValue::DurationMicrosecond(Some(duration)) => { + Some(ScalarValue::new_interval_mdn(0, 0, *duration * 1000)) + } + ScalarValue::DurationMillisecond(Some(duration)) => { + Some(ScalarValue::new_interval_dt(0, *duration as i32)) + } + ScalarValue::DurationSecond(Some(duration)) => { + Some(ScalarValue::new_interval_dt(0, *duration as i32 * 1000)) + } _ => None, } } @@ -180,14 +180,13 @@ fn interval_mdn_to_duration_ns(mdn: &i128) -> Result { let nanoseconds = mdn & MDN_NS_MASK; if months == 0 && days == 0 { - nanoseconds.try_into().map_err(|_| { - DataFusionError::Internal("Resulting duration exceeds i64::MAX".to_string()) - }) + nanoseconds + .try_into() + .map_err(|_| internal_datafusion_err!("Resulting duration exceeds i64::MAX")) } else { - Err(DataFusionError::Internal( + internal_err!( "The interval cannot have a non-zero month or day value for duration convertibility" - .to_string(), - )) + ) } } @@ -200,9 +199,8 @@ fn interval_dt_to_duration_ms(dt: &i64) -> Result { if days == 0 { Ok(milliseconds) } else { - Err(DataFusionError::Internal( + internal_err!( "The interval cannot have a non-zero day value for duration convertibility" - .to_string(), - )) + ) } } diff --git a/datafusion/physical-expr/src/physical_expr.rs b/datafusion/physical-expr/src/physical_expr.rs index 455ca84a792f..a8d1e3638a17 100644 --- a/datafusion/physical-expr/src/physical_expr.rs +++ b/datafusion/physical-expr/src/physical_expr.rs @@ -20,7 +20,6 @@ use std::fmt::{Debug, Display}; use std::hash::{Hash, Hasher}; use std::sync::Arc; -use crate::intervals::Interval; use crate::sort_properties::SortProperties; use crate::utils::scatter; @@ -30,6 +29,7 @@ use arrow::datatypes::{DataType, Schema}; use arrow::record_batch::RecordBatch; use datafusion_common::utils::DataPtr; use datafusion_common::{internal_err, not_impl_err, DataFusionError, Result}; +use datafusion_expr::interval_arithmetic::Interval; use datafusion_expr::ColumnarValue; use itertools::izip; @@ -95,36 +95,34 @@ pub trait PhysicalExpr: Send + Sync + Display + Debug + PartialEq { /// Updates bounds for child expressions, given a known interval for this /// expression. /// - /// This is used to propagate constraints down through an - /// expression tree. + /// This is used to propagate constraints down through an expression tree. /// /// # Arguments /// /// * `interval` is the currently known interval for this expression. - /// * `children` are the current intervals for the children of this expression + /// * `children` are the current intervals for the children of this expression. /// /// # Returns /// - /// A Vec of new intervals for the children, in order. + /// A `Vec` of new intervals for the children, in order. /// - /// If constraint propagation reveals an infeasibility, returns [None] for - /// the child causing infeasibility. - /// - /// If none of the child intervals change as a result of propagation, may - /// return an empty vector instead of cloning `children`. + /// If constraint propagation reveals an infeasibility for any child, returns + /// [`None`]. If none of the children intervals change as a result of propagation, + /// may return an empty vector instead of cloning `children`. This is the default + /// (and conservative) return value. /// /// # Example /// - /// If the expression is `a + b`, the current `interval` is `[4, 5] and the - /// inputs are given [`a: [0, 2], `b: [-∞, 4]]`, then propagation would - /// would return `[a: [0, 2], b: [2, 4]]` as `b` must be at least 2 to - /// make the output at least `4`. + /// If the expression is `a + b`, the current `interval` is `[4, 5]` and the + /// inputs `a` and `b` are respectively given as `[0, 2]` and `[-∞, 4]`, then + /// propagation would would return `[0, 2]` and `[2, 4]` as `b` must be at + /// least `2` to make the output at least `4`. fn propagate_constraints( &self, _interval: &Interval, _children: &[&Interval], - ) -> Result>> { - not_impl_err!("Not implemented for {self}") + ) -> Result>> { + Ok(Some(vec![])) } /// Update the hash `state` with this expression requirements from diff --git a/datafusion/physical-expr/src/sort_properties.rs b/datafusion/physical-expr/src/sort_properties.rs index a3b201f84e9d..25729640ec99 100644 --- a/datafusion/physical-expr/src/sort_properties.rs +++ b/datafusion/physical-expr/src/sort_properties.rs @@ -99,7 +99,7 @@ impl SortProperties { } } - pub fn and(&self, rhs: &Self) -> Self { + pub fn and_or(&self, rhs: &Self) -> Self { match (self, rhs) { (Self::Ordered(lhs), Self::Ordered(rhs)) if lhs.descending == rhs.descending => diff --git a/datafusion/physical-expr/src/utils.rs b/datafusion/physical-expr/src/utils.rs index 2f4ee89463a8..ed62956de8e0 100644 --- a/datafusion/physical-expr/src/utils.rs +++ b/datafusion/physical-expr/src/utils.rs @@ -183,7 +183,7 @@ impl TreeNode for ExprTreeNode { /// identical expressions in one node. Caller specifies the node type in the /// DAEG via the `constructor` argument, which constructs nodes in the DAEG /// from the [ExprTreeNode] ancillary object. -struct PhysicalExprDAEGBuilder<'a, T, F: Fn(&ExprTreeNode) -> T> { +struct PhysicalExprDAEGBuilder<'a, T, F: Fn(&ExprTreeNode) -> Result> { // The resulting DAEG (expression DAG). graph: StableGraph, // A vector of visited expression nodes and their corresponding node indices. @@ -192,7 +192,7 @@ struct PhysicalExprDAEGBuilder<'a, T, F: Fn(&ExprTreeNode) -> T> { constructor: &'a F, } -impl<'a, T, F: Fn(&ExprTreeNode) -> T> TreeNodeRewriter +impl<'a, T, F: Fn(&ExprTreeNode) -> Result> TreeNodeRewriter for PhysicalExprDAEGBuilder<'a, T, F> { type N = ExprTreeNode; @@ -213,7 +213,7 @@ impl<'a, T, F: Fn(&ExprTreeNode) -> T> TreeNodeRewriter // add edges to its child nodes. Add the visited expression to the vector // of visited expressions and return the newly created node index. None => { - let node_idx = self.graph.add_node((self.constructor)(&node)); + let node_idx = self.graph.add_node((self.constructor)(&node)?); for expr_node in node.child_nodes.iter() { self.graph.add_edge(node_idx, expr_node.data.unwrap(), 0); } @@ -234,7 +234,7 @@ pub fn build_dag( constructor: &F, ) -> Result<(NodeIndex, StableGraph)> where - F: Fn(&ExprTreeNode) -> T, + F: Fn(&ExprTreeNode) -> Result, { // Create a new expression tree node from the input expression. let init = ExprTreeNode::new(expr); @@ -394,7 +394,7 @@ mod tests { } } - fn make_dummy_node(node: &ExprTreeNode) -> PhysicalExprDummyNode { + fn make_dummy_node(node: &ExprTreeNode) -> Result { let expr = node.expression().clone(); let dummy_property = if expr.as_any().is::() { "Binary" @@ -406,12 +406,12 @@ mod tests { "Other" } .to_owned(); - PhysicalExprDummyNode { + Ok(PhysicalExprDummyNode { expr, property: DummyProperty { expr_type: dummy_property, }, - } + }) } #[test] diff --git a/datafusion/physical-plan/src/filter.rs b/datafusion/physical-plan/src/filter.rs index b6cd9fe79c85..903f4c972ebd 100644 --- a/datafusion/physical-plan/src/filter.rs +++ b/datafusion/physical-plan/src/filter.rs @@ -27,7 +27,6 @@ use super::expressions::PhysicalSortExpr; use super::{ ColumnStatistics, DisplayAs, RecordBatchStream, SendableRecordBatchStream, Statistics, }; - use crate::{ metrics::{BaselineMetrics, ExecutionPlanMetricsSet, MetricsSet}, Column, DisplayFormatType, ExecutionPlan, Partitioning, @@ -215,7 +214,8 @@ impl ExecutionPlan for FilterExec { &self.input.schema(), &input_stats.column_statistics, )?; - let analysis_ctx = analyze(predicate, input_analysis_ctx)?; + + let analysis_ctx = analyze(predicate, input_analysis_ctx, &self.schema())?; // Estimate (inexact) selectivity of predicate let selectivity = analysis_ctx.selectivity.unwrap_or(1.0); @@ -254,19 +254,12 @@ fn collect_new_statistics( .. }, )| { - let closed_interval = interval.close_bounds(); - let (min_value, max_value) = - if closed_interval.lower.value.eq(&closed_interval.upper.value) { - ( - Precision::Exact(closed_interval.lower.value), - Precision::Exact(closed_interval.upper.value), - ) - } else { - ( - Precision::Inexact(closed_interval.lower.value), - Precision::Inexact(closed_interval.upper.value), - ) - }; + let (lower, upper) = interval.into_bounds(); + let (min_value, max_value) = if lower.eq(&upper) { + (Precision::Exact(lower), Precision::Exact(upper)) + } else { + (Precision::Inexact(lower), Precision::Inexact(upper)) + }; ColumnStatistics { null_count: input_column_stats[idx].null_count.clone().to_inexact(), max_value, diff --git a/datafusion/physical-plan/src/joins/stream_join_utils.rs b/datafusion/physical-plan/src/joins/stream_join_utils.rs index aa57a4f89606..5083f96b01fb 100644 --- a/datafusion/physical-plan/src/joins/stream_join_utils.rs +++ b/datafusion/physical-plan/src/joins/stream_join_utils.rs @@ -29,13 +29,13 @@ use crate::joins::utils::{JoinFilter, JoinHashMapType}; use arrow::compute::concat_batches; use arrow_array::{ArrowPrimitiveType, NativeAdapter, PrimitiveArray, RecordBatch}; use arrow_buffer::{ArrowNativeType, BooleanBufferBuilder}; -use arrow_schema::SchemaRef; +use arrow_schema::{Schema, SchemaRef}; use async_trait::async_trait; use datafusion_common::tree_node::{Transformed, TreeNode}; use datafusion_common::{DataFusionError, JoinSide, Result, ScalarValue}; use datafusion_execution::SendableRecordBatchStream; +use datafusion_expr::interval_arithmetic::Interval; use datafusion_physical_expr::expressions::Column; -use datafusion_physical_expr::intervals::{Interval, IntervalBound}; use datafusion_physical_expr::utils::collect_columns; use datafusion_physical_expr::{PhysicalExpr, PhysicalSortExpr}; @@ -316,7 +316,11 @@ pub fn build_filter_input_order( order: &PhysicalSortExpr, ) -> Result> { let opt_expr = convert_sort_expr_with_filter_schema(&side, filter, schema, order)?; - Ok(opt_expr.map(|filter_expr| SortedFilterExpr::new(order.clone(), filter_expr))) + opt_expr + .map(|filter_expr| { + SortedFilterExpr::try_new(order.clone(), filter_expr, filter.schema()) + }) + .transpose() } /// Convert a physical expression into a filter expression using the given @@ -359,16 +363,18 @@ pub struct SortedFilterExpr { impl SortedFilterExpr { /// Constructor - pub fn new( + pub fn try_new( origin_sorted_expr: PhysicalSortExpr, filter_expr: Arc, - ) -> Self { - Self { + filter_schema: &Schema, + ) -> Result { + let dt = &filter_expr.data_type(filter_schema)?; + Ok(Self { origin_sorted_expr, filter_expr, - interval: Interval::default(), + interval: Interval::make_unbounded(dt)?, node_index: 0, - } + }) } /// Get origin expr information pub fn origin_sorted_expr(&self) -> &PhysicalSortExpr { @@ -494,12 +500,12 @@ pub fn update_filter_expr_interval( // Convert the array to a ScalarValue: let value = ScalarValue::try_from_array(&array, 0)?; // Create a ScalarValue representing positive or negative infinity for the same data type: - let unbounded = IntervalBound::make_unbounded(value.data_type())?; + let inf = ScalarValue::try_from(value.data_type())?; // Update the interval with lower and upper bounds based on the sort option: let interval = if sorted_expr.origin_sorted_expr().options.descending { - Interval::new(unbounded, IntervalBound::new(value, false)) + Interval::try_new(inf, value)? } else { - Interval::new(IntervalBound::new(value, false), unbounded) + Interval::try_new(value, inf)? }; // Set the calculated interval for the sorted filter expression: sorted_expr.set_interval(interval); @@ -1024,14 +1030,13 @@ pub mod tests { convert_sort_expr_with_filter_schema, PruningJoinHashMap, }; use crate::{ - expressions::Column, - expressions::PhysicalSortExpr, + expressions::{Column, PhysicalSortExpr}, joins::utils::{ColumnIndex, JoinFilter}, }; use arrow::compute::SortOptions; use arrow::datatypes::{DataType, Field, Schema}; - use datafusion_common::ScalarValue; + use datafusion_common::{JoinSide, ScalarValue}; use datafusion_expr::Operator; use datafusion_physical_expr::expressions::{binary, cast, col, lit}; diff --git a/datafusion/physical-plan/src/joins/symmetric_hash_join.rs b/datafusion/physical-plan/src/joins/symmetric_hash_join.rs index d653297abea7..95f15877b960 100644 --- a/datafusion/physical-plan/src/joins/symmetric_hash_join.rs +++ b/datafusion/physical-plan/src/joins/symmetric_hash_join.rs @@ -62,8 +62,9 @@ use datafusion_common::{ }; use datafusion_execution::memory_pool::MemoryConsumer; use datafusion_execution::TaskContext; +use datafusion_expr::interval_arithmetic::Interval; use datafusion_physical_expr::equivalence::join_equivalence_properties; -use datafusion_physical_expr::intervals::ExprIntervalGraph; +use datafusion_physical_expr::intervals::cp_solver::ExprIntervalGraph; use ahash::RandomState; use futures::Stream; @@ -631,9 +632,9 @@ fn determine_prune_length( // Get the lower or upper interval based on the sort direction let target = if origin_sorted_expr.options.descending { - interval.upper.value.clone() + interval.upper().clone() } else { - interval.lower.value.clone() + interval.lower().clone() }; // Perform binary search on the array to determine the length of the record batch to be pruned @@ -975,7 +976,7 @@ impl OneSideHashJoiner { filter_intervals.push((expr.node_index(), expr.interval().clone())) } // Update the physical expression graph using the join filter intervals: - graph.update_ranges(&mut filter_intervals)?; + graph.update_ranges(&mut filter_intervals, Interval::CERTAINLY_TRUE)?; // Extract the new join filter interval for the build side: let calculated_build_side_interval = filter_intervals.remove(0).1; // If the intervals have not changed, return early without pruning: @@ -1948,7 +1949,7 @@ mod tests { (12, 17), )] cardinality: (i32, i32), - #[values(0, 1)] case_expr: usize, + #[values(0, 1, 2)] case_expr: usize, ) -> Result<()> { let session_config = SessionConfig::new().with_repartition_joins(false); let task_ctx = TaskContext::default().with_session_config(session_config); diff --git a/datafusion/physical-plan/src/joins/test_utils.rs b/datafusion/physical-plan/src/joins/test_utils.rs index 6deaa9ba1b9c..fbd52ddf0c70 100644 --- a/datafusion/physical-plan/src/joins/test_utils.rs +++ b/datafusion/physical-plan/src/joins/test_utils.rs @@ -243,6 +243,20 @@ pub fn join_expr_tests_fixture_temporal( ScalarValue::TimestampMillisecond(Some(1672574402000), None), // 2023-01-01:12.00.02 schema, ), + // constructs ((left_col - DURATION '3 secs') > (right_col - DURATION '2 secs')) AND ((left_col - DURATION '5 secs') < (right_col - DURATION '4 secs')) + 2 => gen_conjunctive_temporal_expr( + left_col, + right_col, + Operator::Minus, + Operator::Minus, + Operator::Minus, + Operator::Minus, + ScalarValue::DurationMillisecond(Some(3000)), // 3 secs + ScalarValue::DurationMillisecond(Some(2000)), // 2 secs + ScalarValue::DurationMillisecond(Some(5000)), // 5 secs + ScalarValue::DurationMillisecond(Some(4000)), // 4 secs + schema, + ), _ => unreachable!(), } } diff --git a/datafusion/physical-plan/src/joins/utils.rs b/datafusion/physical-plan/src/joins/utils.rs index 0729d365d6a0..5e01ca227cf5 100644 --- a/datafusion/physical-plan/src/joins/utils.rs +++ b/datafusion/physical-plan/src/joins/utils.rs @@ -42,9 +42,10 @@ use datafusion_common::{ plan_datafusion_err, plan_err, DataFusionError, JoinSide, JoinType, Result, SharedResult, }; +use datafusion_expr::interval_arithmetic::Interval; use datafusion_physical_expr::equivalence::add_offset_to_expr; use datafusion_physical_expr::expressions::Column; -use datafusion_physical_expr::intervals::{ExprIntervalGraph, Interval, IntervalBound}; +use datafusion_physical_expr::intervals::cp_solver::ExprIntervalGraph; use datafusion_physical_expr::utils::merge_vectors; use datafusion_physical_expr::{ LexOrdering, LexOrderingRef, PhysicalExpr, PhysicalSortExpr, @@ -713,8 +714,8 @@ fn estimate_inner_join_cardinality( ); } - let left_max_distinct = max_distinct_count(&left_stats.num_rows, left_stat)?; - let right_max_distinct = max_distinct_count(&right_stats.num_rows, right_stat)?; + let left_max_distinct = max_distinct_count(&left_stats.num_rows, left_stat); + let right_max_distinct = max_distinct_count(&right_stats.num_rows, right_stat); let max_distinct = left_max_distinct.max(&right_max_distinct); if max_distinct.get_value().is_some() { // Seems like there are a few implementations of this algorithm that implement @@ -745,48 +746,60 @@ fn estimate_inner_join_cardinality( } /// Estimate the number of maximum distinct values that can be present in the -/// given column from its statistics. -/// -/// If distinct_count is available, uses it directly. If the column numeric, and -/// has min/max values, then they might be used as a fallback option. Otherwise, -/// returns None. +/// given column from its statistics. If distinct_count is available, uses it +/// directly. Otherwise, if the column is numeric and has min/max values, it +/// estimates the maximum distinct count from those. fn max_distinct_count( num_rows: &Precision, stats: &ColumnStatistics, -) -> Option> { - match ( - &stats.distinct_count, - stats.max_value.get_value(), - stats.min_value.get_value(), - ) { - (Precision::Exact(_), _, _) | (Precision::Inexact(_), _, _) => { - Some(stats.distinct_count.clone()) - } - (_, Some(max), Some(min)) => { - let numeric_range = Interval::new( - IntervalBound::new(min.clone(), false), - IntervalBound::new(max.clone(), false), - ) - .cardinality() - .ok() - .flatten()? as usize; - - // The number can never be greater than the number of rows we have (minus - // the nulls, since they don't count as distinct values). - let ceiling = - num_rows.get_value()? - stats.null_count.get_value().unwrap_or(&0); - Some( - if num_rows.is_exact().unwrap_or(false) - && stats.max_value.is_exact().unwrap_or(false) - && stats.min_value.is_exact().unwrap_or(false) +) -> Precision { + match &stats.distinct_count { + dc @ (Precision::Exact(_) | Precision::Inexact(_)) => dc.clone(), + _ => { + // The number can never be greater than the number of rows we have + // minus the nulls (since they don't count as distinct values). + let result = match num_rows { + Precision::Absent => Precision::Absent, + Precision::Inexact(count) => { + Precision::Inexact(count - stats.null_count.get_value().unwrap_or(&0)) + } + Precision::Exact(count) => { + let count = count - stats.null_count.get_value().unwrap_or(&0); + if stats.null_count.is_exact().unwrap_or(false) { + Precision::Exact(count) + } else { + Precision::Inexact(count) + } + } + }; + // Cap the estimate using the number of possible values: + if let (Some(min), Some(max)) = + (stats.min_value.get_value(), stats.max_value.get_value()) + { + if let Some(range_dc) = Interval::try_new(min.clone(), max.clone()) + .ok() + .and_then(|e| e.cardinality()) { - Precision::Exact(numeric_range.min(ceiling)) - } else { - Precision::Inexact(numeric_range.min(ceiling)) - }, - ) + let range_dc = range_dc as usize; + // Note that the `unwrap` calls in the below statement are safe. + return if matches!(result, Precision::Absent) + || &range_dc < result.get_value().unwrap() + { + if stats.min_value.is_exact().unwrap() + && stats.max_value.is_exact().unwrap() + { + Precision::Exact(range_dc) + } else { + Precision::Inexact(range_dc) + } + } else { + result + }; + } + } + + result } - _ => None, } } @@ -1251,7 +1264,8 @@ pub fn prepare_sorted_exprs( vec![left_temp_sorted_filter_expr, right_temp_sorted_filter_expr]; // Build the expression interval graph - let mut graph = ExprIntervalGraph::try_new(filter.expression().clone())?; + let mut graph = + ExprIntervalGraph::try_new(filter.expression().clone(), filter.schema())?; // Update sorted expressions with node indices update_sorted_exprs_with_node_indices(&mut graph, &mut sorted_exprs); @@ -1697,7 +1711,7 @@ mod tests { column_statistics: right_col_stats, }, ), - None + Some(Precision::Inexact(100)) ); Ok(()) } From ffbc6896b0f4f1b417991d1a13266be10c3f3709 Mon Sep 17 00:00:00 2001 From: Mustafa Akur <106137913+mustafasrepo@users.noreply.github.com> Date: Tue, 21 Nov 2023 22:41:53 +0300 Subject: [PATCH 298/572] [MINOR]: Remove unecessary orderings from the final plan (#8289) * Remove lost orderings from the final plan * Improve comments --------- Co-authored-by: Mehmet Ozan Kabak --- .../src/physical_optimizer/enforce_sorting.rs | 4 +++- datafusion/physical-plan/src/insert.rs | 23 +++++++------------ datafusion/sqllogictest/test_files/select.slt | 23 +++++++++++++++++++ 3 files changed, 34 insertions(+), 16 deletions(-) diff --git a/datafusion/core/src/physical_optimizer/enforce_sorting.rs b/datafusion/core/src/physical_optimizer/enforce_sorting.rs index 2590948d3b3e..6fec74f608ae 100644 --- a/datafusion/core/src/physical_optimizer/enforce_sorting.rs +++ b/datafusion/core/src/physical_optimizer/enforce_sorting.rs @@ -476,7 +476,9 @@ fn ensure_sorting( update_child_to_remove_unnecessary_sort(child, sort_onwards, &plan)?; } } - (None, None) => {} + (None, None) => { + update_child_to_remove_unnecessary_sort(child, sort_onwards, &plan)?; + } } } // For window expressions, we can remove some sorts when we can diff --git a/datafusion/physical-plan/src/insert.rs b/datafusion/physical-plan/src/insert.rs index 4eeb58974aba..81cdfd753fe6 100644 --- a/datafusion/physical-plan/src/insert.rs +++ b/datafusion/physical-plan/src/insert.rs @@ -219,24 +219,17 @@ impl ExecutionPlan for FileSinkExec { } fn required_input_ordering(&self) -> Vec>> { - // The input order is either exlicitly set (such as by a ListingTable), - // or require that the [FileSinkExec] gets the data in the order the - // input produced it (otherwise the optimizer may chose to reorder - // the input which could result in unintended / poor UX) - // - // More rationale: - // https://github.com/apache/arrow-datafusion/pull/6354#discussion_r1195284178 - match &self.sort_order { - Some(requirements) => vec![Some(requirements.clone())], - None => vec![self - .input - .output_ordering() - .map(PhysicalSortRequirement::from_sort_exprs)], - } + // The required input ordering is set externally (e.g. by a `ListingTable`). + // Otherwise, there is no specific requirement (i.e. `sort_expr` is `None`). + vec![self.sort_order.as_ref().cloned()] } fn maintains_input_order(&self) -> Vec { - vec![false] + // Maintains ordering in the sense that the written file will reflect + // the ordering of the input. For more context, see: + // + // https://github.com/apache/arrow-datafusion/pull/6354#discussion_r1195284178 + vec![true] } fn children(&self) -> Vec> { diff --git a/datafusion/sqllogictest/test_files/select.slt b/datafusion/sqllogictest/test_files/select.slt index 98ea061c731b..bb81c5a9a138 100644 --- a/datafusion/sqllogictest/test_files/select.slt +++ b/datafusion/sqllogictest/test_files/select.slt @@ -1013,6 +1013,29 @@ SortPreservingMergeExec: [c@3 ASC NULLS LAST] --------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 ----------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a0, a, b, c, d], output_ordering=[a@1 ASC NULLS LAST, b@2 ASC NULLS LAST, c@3 ASC NULLS LAST], has_header=true +# When ordering lost during projection, we shouldn't keep the SortExec. +# in the final physical plan. +query TT +EXPLAIN SELECT c2, COUNT(*) +FROM (SELECT c2 +FROM aggregate_test_100 +ORDER BY c1, c2) +GROUP BY c2; +---- +logical_plan +Aggregate: groupBy=[[aggregate_test_100.c2]], aggr=[[COUNT(UInt8(1)) AS COUNT(*)]] +--Projection: aggregate_test_100.c2 +----Sort: aggregate_test_100.c1 ASC NULLS LAST, aggregate_test_100.c2 ASC NULLS LAST +------Projection: aggregate_test_100.c2, aggregate_test_100.c1 +--------TableScan: aggregate_test_100 projection=[c1, c2] +physical_plan +AggregateExec: mode=FinalPartitioned, gby=[c2@0 as c2], aggr=[COUNT(*)] +--CoalesceBatchesExec: target_batch_size=8192 +----RepartitionExec: partitioning=Hash([c2@0], 2), input_partitions=2 +------AggregateExec: mode=Partial, gby=[c2@0 as c2], aggr=[COUNT(*)] +--------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +----------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c2], has_header=true + statement ok drop table annotated_data_finite2; From 47b4972329be053d20801887e34d978cd5f99448 Mon Sep 17 00:00:00 2001 From: Eduard Karacharov <13005055+korowa@users.noreply.github.com> Date: Tue, 21 Nov 2023 22:20:23 +0200 Subject: [PATCH 299/572] consistent logical & physical NTILE return types (#8270) --- datafusion/expr/src/window_function.rs | 11 ++++++++++- datafusion/sqllogictest/test_files/window.slt | 7 +++++++ 2 files changed, 17 insertions(+), 1 deletion(-) diff --git a/datafusion/expr/src/window_function.rs b/datafusion/expr/src/window_function.rs index 35b7bded70d3..946a80dd844a 100644 --- a/datafusion/expr/src/window_function.rs +++ b/datafusion/expr/src/window_function.rs @@ -205,7 +205,7 @@ impl BuiltInWindowFunction { BuiltInWindowFunction::PercentRank | BuiltInWindowFunction::CumeDist => { Ok(DataType::Float64) } - BuiltInWindowFunction::Ntile => Ok(DataType::UInt32), + BuiltInWindowFunction::Ntile => Ok(DataType::UInt64), BuiltInWindowFunction::Lag | BuiltInWindowFunction::Lead | BuiltInWindowFunction::FirstValue @@ -369,6 +369,15 @@ mod tests { Ok(()) } + #[test] + fn test_ntile_return_type() -> Result<()> { + let fun = find_df_window_func("ntile").unwrap(); + let observed = fun.return_type(&[DataType::Int16])?; + assert_eq!(DataType::UInt64, observed); + + Ok(()) + } + #[test] fn test_window_function_case_insensitive() -> Result<()> { let names = vec![ diff --git a/datafusion/sqllogictest/test_files/window.slt b/datafusion/sqllogictest/test_files/window.slt index a3c57a67a6f0..1ef0ba0d10e3 100644 --- a/datafusion/sqllogictest/test_files/window.slt +++ b/datafusion/sqllogictest/test_files/window.slt @@ -3522,3 +3522,10 @@ SortPreservingMergeExec: [c@3 ASC NULLS LAST] --------SortPreservingRepartitionExec: partitioning=Hash([d@4], 2), input_partitions=2, sort_exprs=a@1 ASC NULLS LAST,b@2 ASC NULLS LAST ----------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 ------------StreamingTableExec: partition_sizes=1, projection=[a0, a, b, c, d], infinite_source=true, output_ordering=[a@1 ASC NULLS LAST, b@2 ASC NULLS LAST] + +# CTAS with NTILE function +statement ok +CREATE TABLE new_table AS SELECT NTILE(2) OVER(ORDER BY c1) AS ntile_2 FROM aggregate_test_100; + +statement ok +DROP TABLE new_table; From 54a02470fc9304110a0995b5e540bc247e0a2c6e Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E8=B0=AD=E5=B7=8D?= Date: Wed, 22 Nov 2023 04:28:04 +0800 Subject: [PATCH 300/572] make `array_union`/`array_except`/`array_intersect` handle empty/null arrays rightly (#8269) * make array_union handle empty/null arrays rightly Signed-off-by: veeupup * make array_except handle empty/null arrays rightly Signed-off-by: veeupup * make array_intersect handle empty/null arrays rightly Signed-off-by: veeupup * fix sql_array_literal Signed-off-by: veeupup * fix comments --------- Signed-off-by: veeupup --- datafusion/expr/src/built_in_function.rs | 18 ++- .../physical-expr/src/array_expressions.rs | 137 ++++++++++-------- datafusion/sql/src/expr/value.rs | 34 +++-- datafusion/sql/tests/sql_integration.rs | 22 --- .../sqllogictest/test_files/aggregate.slt | 4 +- datafusion/sqllogictest/test_files/array.slt | 52 ++++++- 6 files changed, 164 insertions(+), 103 deletions(-) diff --git a/datafusion/expr/src/built_in_function.rs b/datafusion/expr/src/built_in_function.rs index e9030ebcc00f..cbf5d400bab5 100644 --- a/datafusion/expr/src/built_in_function.rs +++ b/datafusion/expr/src/built_in_function.rs @@ -599,12 +599,24 @@ impl BuiltinScalarFunction { BuiltinScalarFunction::ArrayReplaceAll => Ok(input_expr_types[0].clone()), BuiltinScalarFunction::ArraySlice => Ok(input_expr_types[0].clone()), BuiltinScalarFunction::ArrayToString => Ok(Utf8), - BuiltinScalarFunction::ArrayIntersect => Ok(input_expr_types[0].clone()), - BuiltinScalarFunction::ArrayUnion => Ok(input_expr_types[0].clone()), + BuiltinScalarFunction::ArrayUnion | BuiltinScalarFunction::ArrayIntersect => { + match (input_expr_types[0].clone(), input_expr_types[1].clone()) { + (DataType::Null, dt) => Ok(dt), + (dt, DataType::Null) => Ok(dt), + (dt, _) => Ok(dt), + } + } BuiltinScalarFunction::Range => { Ok(List(Arc::new(Field::new("item", Int64, true)))) } - BuiltinScalarFunction::ArrayExcept => Ok(input_expr_types[0].clone()), + BuiltinScalarFunction::ArrayExcept => { + match (input_expr_types[0].clone(), input_expr_types[1].clone()) { + (DataType::Null, _) | (_, DataType::Null) => { + Ok(input_expr_types[0].clone()) + } + (dt, _) => Ok(dt), + } + } BuiltinScalarFunction::Cardinality => Ok(UInt64), BuiltinScalarFunction::MakeArray => match input_expr_types.len() { 0 => Ok(List(Arc::new(Field::new("item", Null, true)))), diff --git a/datafusion/physical-expr/src/array_expressions.rs b/datafusion/physical-expr/src/array_expressions.rs index c0f6c67263a7..8968bcf2ea4e 100644 --- a/datafusion/physical-expr/src/array_expressions.rs +++ b/datafusion/physical-expr/src/array_expressions.rs @@ -228,10 +228,10 @@ fn compute_array_dims(arr: Option) -> Result>>> fn check_datatypes(name: &str, args: &[&ArrayRef]) -> Result<()> { let data_type = args[0].data_type(); - if !args - .iter() - .all(|arg| arg.data_type().equals_datatype(data_type)) - { + if !args.iter().all(|arg| { + arg.data_type().equals_datatype(data_type) + || arg.data_type().equals_datatype(&DataType::Null) + }) { let types = args.iter().map(|arg| arg.data_type()).collect::>(); return plan_err!("{name} received incompatible types: '{types:?}'."); } @@ -1512,19 +1512,29 @@ pub fn array_union(args: &[ArrayRef]) -> Result { match (array1.data_type(), array2.data_type()) { (DataType::Null, _) => Ok(array2.clone()), (_, DataType::Null) => Ok(array1.clone()), - (DataType::List(field_ref), DataType::List(_)) => { - check_datatypes("array_union", &[array1, array2])?; - let list1 = array1.as_list::(); - let list2 = array2.as_list::(); - let result = union_generic_lists::(list1, list2, field_ref)?; - Ok(Arc::new(result)) + (DataType::List(l_field_ref), DataType::List(r_field_ref)) => { + match (l_field_ref.data_type(), r_field_ref.data_type()) { + (DataType::Null, _) => Ok(array2.clone()), + (_, DataType::Null) => Ok(array1.clone()), + (_, _) => { + let list1 = array1.as_list::(); + let list2 = array2.as_list::(); + let result = union_generic_lists::(list1, list2, l_field_ref)?; + Ok(Arc::new(result)) + } + } } - (DataType::LargeList(field_ref), DataType::LargeList(_)) => { - check_datatypes("array_union", &[array1, array2])?; - let list1 = array1.as_list::(); - let list2 = array2.as_list::(); - let result = union_generic_lists::(list1, list2, field_ref)?; - Ok(Arc::new(result)) + (DataType::LargeList(l_field_ref), DataType::LargeList(r_field_ref)) => { + match (l_field_ref.data_type(), r_field_ref.data_type()) { + (DataType::Null, _) => Ok(array2.clone()), + (_, DataType::Null) => Ok(array1.clone()), + (_, _) => { + let list1 = array1.as_list::(); + let list2 = array2.as_list::(); + let result = union_generic_lists::(list1, list2, l_field_ref)?; + Ok(Arc::new(result)) + } + } } _ => { internal_err!( @@ -1919,55 +1929,66 @@ pub fn string_to_array(args: &[ArrayRef]) -> Result Result { assert_eq!(args.len(), 2); - let first_array = as_list_array(&args[0])?; - let second_array = as_list_array(&args[1])?; + let first_array = &args[0]; + let second_array = &args[1]; - if first_array.value_type() != second_array.value_type() { - return internal_err!("array_intersect is not implemented for '{first_array:?}' and '{second_array:?}'"); - } - let dt = first_array.value_type(); + match (first_array.data_type(), second_array.data_type()) { + (DataType::Null, _) => Ok(second_array.clone()), + (_, DataType::Null) => Ok(first_array.clone()), + _ => { + let first_array = as_list_array(&first_array)?; + let second_array = as_list_array(&second_array)?; - let mut offsets = vec![0]; - let mut new_arrays = vec![]; - - let converter = RowConverter::new(vec![SortField::new(dt.clone())])?; - for (first_arr, second_arr) in first_array.iter().zip(second_array.iter()) { - if let (Some(first_arr), Some(second_arr)) = (first_arr, second_arr) { - let l_values = converter.convert_columns(&[first_arr])?; - let r_values = converter.convert_columns(&[second_arr])?; - - let values_set: HashSet<_> = l_values.iter().collect(); - let mut rows = Vec::with_capacity(r_values.num_rows()); - for r_val in r_values.iter().sorted().dedup() { - if values_set.contains(&r_val) { - rows.push(r_val); - } + if first_array.value_type() != second_array.value_type() { + return internal_err!("array_intersect is not implemented for '{first_array:?}' and '{second_array:?}'"); } - let last_offset: i32 = match offsets.last().copied() { - Some(offset) => offset, - None => return internal_err!("offsets should not be empty"), - }; - offsets.push(last_offset + rows.len() as i32); - let arrays = converter.convert_rows(rows)?; - let array = match arrays.get(0) { - Some(array) => array.clone(), - None => { - return internal_err!( - "array_intersect: failed to get array from rows" - ) + let dt = first_array.value_type(); + + let mut offsets = vec![0]; + let mut new_arrays = vec![]; + + let converter = RowConverter::new(vec![SortField::new(dt.clone())])?; + for (first_arr, second_arr) in first_array.iter().zip(second_array.iter()) { + if let (Some(first_arr), Some(second_arr)) = (first_arr, second_arr) { + let l_values = converter.convert_columns(&[first_arr])?; + let r_values = converter.convert_columns(&[second_arr])?; + + let values_set: HashSet<_> = l_values.iter().collect(); + let mut rows = Vec::with_capacity(r_values.num_rows()); + for r_val in r_values.iter().sorted().dedup() { + if values_set.contains(&r_val) { + rows.push(r_val); + } + } + + let last_offset: i32 = match offsets.last().copied() { + Some(offset) => offset, + None => return internal_err!("offsets should not be empty"), + }; + offsets.push(last_offset + rows.len() as i32); + let arrays = converter.convert_rows(rows)?; + let array = match arrays.get(0) { + Some(array) => array.clone(), + None => { + return internal_err!( + "array_intersect: failed to get array from rows" + ) + } + }; + new_arrays.push(array); } - }; - new_arrays.push(array); + } + + let field = Arc::new(Field::new("item", dt, true)); + let offsets = OffsetBuffer::new(offsets.into()); + let new_arrays_ref = + new_arrays.iter().map(|v| v.as_ref()).collect::>(); + let values = compute::concat(&new_arrays_ref)?; + let arr = Arc::new(ListArray::try_new(field, offsets, values, None)?); + Ok(arr) } } - - let field = Arc::new(Field::new("item", dt, true)); - let offsets = OffsetBuffer::new(offsets.into()); - let new_arrays_ref = new_arrays.iter().map(|v| v.as_ref()).collect::>(); - let values = compute::concat(&new_arrays_ref)?; - let arr = Arc::new(ListArray::try_new(field, offsets, values, None)?); - Ok(arr) } #[cfg(test)] diff --git a/datafusion/sql/src/expr/value.rs b/datafusion/sql/src/expr/value.rs index 3a06fdb158f7..0f086bca6819 100644 --- a/datafusion/sql/src/expr/value.rs +++ b/datafusion/sql/src/expr/value.rs @@ -16,20 +16,20 @@ // under the License. use crate::planner::{ContextProvider, PlannerContext, SqlToRel}; -use arrow::array::new_null_array; use arrow::compute::kernels::cast_utils::parse_interval_month_day_nano; use arrow::datatypes::DECIMAL128_MAX_PRECISION; use arrow_schema::DataType; use datafusion_common::{ not_impl_err, plan_err, DFSchema, DataFusionError, Result, ScalarValue, }; +use datafusion_expr::expr::ScalarFunction; use datafusion_expr::expr::{BinaryExpr, Placeholder}; +use datafusion_expr::BuiltinScalarFunction; use datafusion_expr::{lit, Expr, Operator}; use log::debug; use sqlparser::ast::{BinaryOperator, Expr as SQLExpr, Interval, Value}; use sqlparser::parser::ParserError::ParserError; use std::borrow::Cow; -use std::collections::HashSet; impl<'a, S: ContextProvider> SqlToRel<'a, S> { pub(crate) fn parse_value( @@ -138,9 +138,19 @@ impl<'a, S: ContextProvider> SqlToRel<'a, S> { schema, &mut PlannerContext::new(), )?; + match value { - Expr::Literal(scalar) => { - values.push(scalar); + Expr::Literal(_) => { + values.push(value); + } + Expr::ScalarFunction(ref scalar_function) => { + if scalar_function.fun == BuiltinScalarFunction::MakeArray { + values.push(value); + } else { + return not_impl_err!( + "ScalarFunctions without MakeArray are not supported: {value}" + ); + } } _ => { return not_impl_err!( @@ -150,18 +160,10 @@ impl<'a, S: ContextProvider> SqlToRel<'a, S> { } } - let data_types: HashSet = - values.iter().map(|e| e.data_type()).collect(); - - if data_types.is_empty() { - Ok(lit(ScalarValue::List(new_null_array(&DataType::Null, 0)))) - } else if data_types.len() > 1 { - not_impl_err!("Arrays with different types are not supported: {data_types:?}") - } else { - let data_type = values[0].data_type(); - let arr = ScalarValue::new_list(&values, &data_type); - Ok(lit(ScalarValue::List(arr))) - } + Ok(Expr::ScalarFunction(ScalarFunction::new( + BuiltinScalarFunction::MakeArray, + values, + ))) } /// Convert a SQL interval expression to a DataFusion logical plan diff --git a/datafusion/sql/tests/sql_integration.rs b/datafusion/sql/tests/sql_integration.rs index 4c2bad1c719e..a56e9a50f054 100644 --- a/datafusion/sql/tests/sql_integration.rs +++ b/datafusion/sql/tests/sql_integration.rs @@ -1383,18 +1383,6 @@ fn select_interval_out_of_range() { ); } -#[test] -fn select_array_no_common_type() { - let sql = "SELECT [1, true, null]"; - let err = logical_plan(sql).expect_err("query should have failed"); - - // HashSet doesn't guarantee order - assert_contains!( - err.strip_backtrace(), - "This feature is not implemented: Arrays with different types are not supported: " - ); -} - #[test] fn recursive_ctes() { let sql = " @@ -1411,16 +1399,6 @@ fn recursive_ctes() { ); } -#[test] -fn select_array_non_literal_type() { - let sql = "SELECT [now()]"; - let err = logical_plan(sql).expect_err("query should have failed"); - assert_eq!( - "This feature is not implemented: Arrays with elements other than literal are not supported: now()", - err.strip_backtrace() - ); -} - #[test] fn select_simple_aggregate_with_groupby_and_column_is_in_aggregate_and_groupby() { quick_test( diff --git a/datafusion/sqllogictest/test_files/aggregate.slt b/datafusion/sqllogictest/test_files/aggregate.slt index faad6feb3f33..7157be948914 100644 --- a/datafusion/sqllogictest/test_files/aggregate.slt +++ b/datafusion/sqllogictest/test_files/aggregate.slt @@ -1396,7 +1396,7 @@ SELECT COUNT(DISTINCT c1) FROM test query ? SELECT ARRAY_AGG([]) ---- -[] +[[]] # array_agg_one query ? @@ -1419,7 +1419,7 @@ e 4 query ? SELECT ARRAY_AGG([]); ---- -[] +[[]] # array_agg_one query ? diff --git a/datafusion/sqllogictest/test_files/array.slt b/datafusion/sqllogictest/test_files/array.slt index 61f190e7baf6..d33555509e6c 100644 --- a/datafusion/sqllogictest/test_files/array.slt +++ b/datafusion/sqllogictest/test_files/array.slt @@ -265,6 +265,14 @@ AS VALUES (make_array([28, 29, 30], [31, 32, 33], [34, 35, 36], [28, 29, 30], [31, 32, 33], [34, 35, 36], [28, 29, 30], [31, 32, 33], [34, 35, 36], [28, 29, 30]), [28, 29, 30], [37, 38, 39], 10) ; +query ? +select [1, true, null] +---- +[1, 1, ] + +query error DataFusion error: This feature is not implemented: ScalarFunctions without MakeArray are not supported: now() +SELECT [now()] + query TTT select arrow_typeof(column1), arrow_typeof(column2), arrow_typeof(column3) from arrays; ---- @@ -2014,7 +2022,7 @@ drop table arrays_with_repeating_elements_for_union; query ? select array_union([], []); ---- -NULL +[] # array_union scalar function #7 query ? @@ -2032,7 +2040,7 @@ select array_union([null], [null]); query ? select array_union(null, []); ---- -NULL +[] # array_union scalar function #10 query ? @@ -2687,6 +2695,26 @@ SELECT array_intersect(make_array(1,2,3), make_array(2,3,4)), ---- [2, 3] [] [aa, cc] [true] [2.2, 3.3] [[2, 2], [3, 3]] +query ? +select array_intersect([], []); +---- +[] + +query ? +select array_intersect([], null); +---- +[] + +query ? +select array_intersect(null, []); +---- +[] + +query ? +select array_intersect(null, null); +---- +NULL + query ?????? SELECT list_intersect(make_array(1,2,3), make_array(2,3,4)), list_intersect(make_array(1,3,5), make_array(2,4,6)), @@ -2842,6 +2870,26 @@ NULL statement ok drop table array_except_table_bool; +query ? +select array_except([], null); +---- +[] + +query ? +select array_except([], []); +---- +[] + +query ? +select array_except(null, []); +---- +NULL + +query ? +select array_except(null, null) +---- +NULL + ### Array operators tests From 952e7c302bcdc05d090fe334269f41705f28ceea Mon Sep 17 00:00:00 2001 From: Alex Huang Date: Tue, 21 Nov 2023 21:40:09 +0100 Subject: [PATCH 301/572] improve file path validation when reading parquet (#8267) * improve file path validation * fix cli * update test * update test --- datafusion/core/src/execution/context/mod.rs | 8 ++- .../core/src/execution/context/parquet.rs | 69 ++++++++++++++++++- 2 files changed, 71 insertions(+), 6 deletions(-) diff --git a/datafusion/core/src/execution/context/mod.rs b/datafusion/core/src/execution/context/mod.rs index b8e111d361b1..f829092570bb 100644 --- a/datafusion/core/src/execution/context/mod.rs +++ b/datafusion/core/src/execution/context/mod.rs @@ -858,10 +858,12 @@ impl SessionContext { // check if the file extension matches the expected extension for path in &table_paths { - let file_name = path.prefix().filename().unwrap_or_default(); - if !path.as_str().ends_with(&option_extension) && file_name.contains('.') { + let file_path = path.as_str(); + if !file_path.ends_with(option_extension.clone().as_str()) + && !path.is_collection() + { return exec_err!( - "File '{file_name}' does not match the expected extension '{option_extension}'" + "File path '{file_path}' does not match the expected extension '{option_extension}'" ); } } diff --git a/datafusion/core/src/execution/context/parquet.rs b/datafusion/core/src/execution/context/parquet.rs index 821b1ccf1823..5d649d3e6df8 100644 --- a/datafusion/core/src/execution/context/parquet.rs +++ b/datafusion/core/src/execution/context/parquet.rs @@ -138,10 +138,10 @@ mod tests { Ok(()) } - #[cfg(not(target_family = "windows"))] #[tokio::test] async fn read_from_different_file_extension() -> Result<()> { let ctx = SessionContext::new(); + let sep = std::path::MAIN_SEPARATOR.to_string(); // Make up a new dataframe. let write_df = ctx.read_batch(RecordBatch::try_new( @@ -175,6 +175,25 @@ mod tests { .unwrap() .to_string(); + let path4 = temp_dir_path + .join("output4.parquet".to_owned() + &sep) + .to_str() + .unwrap() + .to_string(); + + let path5 = temp_dir_path + .join("bbb..bbb") + .join("filename.parquet") + .to_str() + .unwrap() + .to_string(); + let dir = temp_dir_path + .join("bbb..bbb".to_owned() + &sep) + .to_str() + .unwrap() + .to_string(); + std::fs::create_dir(dir).expect("create dir failed"); + // Write the dataframe to a parquet file named 'output1.parquet' write_df .clone() @@ -205,6 +224,7 @@ mod tests { // Write the dataframe to a parquet file named 'output3.parquet.snappy.parquet' write_df + .clone() .write_parquet( &path3, DataFrameWriteOptions::new().with_single_file_output(true), @@ -216,6 +236,19 @@ mod tests { ) .await?; + // Write the dataframe to a parquet file named 'bbb..bbb/filename.parquet' + write_df + .write_parquet( + &path5, + DataFrameWriteOptions::new().with_single_file_output(true), + Some( + WriterProperties::builder() + .set_compression(Compression::SNAPPY) + .build(), + ), + ) + .await?; + // Read the dataframe from 'output1.parquet' with the default file extension. let read_df = ctx .read_parquet( @@ -253,10 +286,11 @@ mod tests { }, ) .await; - + let binding = DataFilePaths::to_urls(&path2).unwrap(); + let expexted_path = binding[0].as_str(); assert_eq!( read_df.unwrap_err().strip_backtrace(), - "Execution error: File 'output2.parquet.snappy' does not match the expected extension '.parquet'" + format!("Execution error: File path '{}' does not match the expected extension '.parquet'", expexted_path) ); // Read the dataframe from 'output3.parquet.snappy.parquet' with the correct file extension. @@ -269,6 +303,35 @@ mod tests { ) .await?; + let results = read_df.collect().await?; + let total_rows: usize = results.iter().map(|rb| rb.num_rows()).sum(); + assert_eq!(total_rows, 5); + + // Read the dataframe from 'output4/' + std::fs::create_dir(&path4)?; + let read_df = ctx + .read_parquet( + &path4, + ParquetReadOptions { + ..Default::default() + }, + ) + .await?; + + let results = read_df.collect().await?; + let total_rows: usize = results.iter().map(|rb| rb.num_rows()).sum(); + assert_eq!(total_rows, 0); + + // Read the datafram from doule dot folder; + let read_df = ctx + .read_parquet( + &path5, + ParquetReadOptions { + ..Default::default() + }, + ) + .await?; + let results = read_df.collect().await?; let total_rows: usize = results.iter().map(|rb| rb.num_rows()).sum(); assert_eq!(total_rows, 5); From afdabb260a32e1d3e2119b48b93e47d851cf765f Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Wed, 22 Nov 2023 00:24:42 -0700 Subject: [PATCH 302/572] [Benchmarks] Make `partitions` default to number of cores instead of 2 (#8292) * Default partitions to num cores * update test --- benchmarks/src/sort.rs | 5 +++-- benchmarks/src/tpch/run.rs | 6 +++--- benchmarks/src/util/options.rs | 8 ++++---- 3 files changed, 10 insertions(+), 9 deletions(-) diff --git a/benchmarks/src/sort.rs b/benchmarks/src/sort.rs index 5643c8561944..224f2b19c72e 100644 --- a/benchmarks/src/sort.rs +++ b/benchmarks/src/sort.rs @@ -148,8 +148,9 @@ impl RunOpt { println!("Executing '{title}' (sorting by: {expr:?})"); rundata.start_new_case(title); for i in 0..self.common.iterations { - let config = - SessionConfig::new().with_target_partitions(self.common.partitions); + let config = SessionConfig::new().with_target_partitions( + self.common.partitions.unwrap_or(num_cpus::get()), + ); let ctx = SessionContext::new_with_config(config); let (rows, elapsed) = exec_sort(&ctx, &expr, &test_file, self.common.debug).await?; diff --git a/benchmarks/src/tpch/run.rs b/benchmarks/src/tpch/run.rs index 171b074d2a1b..5193d578fb48 100644 --- a/benchmarks/src/tpch/run.rs +++ b/benchmarks/src/tpch/run.rs @@ -285,7 +285,7 @@ impl RunOpt { } fn partitions(&self) -> usize { - self.common.partitions + self.common.partitions.unwrap_or(num_cpus::get()) } } @@ -325,7 +325,7 @@ mod tests { let path = get_tpch_data_path()?; let common = CommonOpt { iterations: 1, - partitions: 2, + partitions: Some(2), batch_size: 8192, debug: false, }; @@ -357,7 +357,7 @@ mod tests { let path = get_tpch_data_path()?; let common = CommonOpt { iterations: 1, - partitions: 2, + partitions: Some(2), batch_size: 8192, debug: false, }; diff --git a/benchmarks/src/util/options.rs b/benchmarks/src/util/options.rs index 1d86d10fb88c..b9398e5b522f 100644 --- a/benchmarks/src/util/options.rs +++ b/benchmarks/src/util/options.rs @@ -26,9 +26,9 @@ pub struct CommonOpt { #[structopt(short = "i", long = "iterations", default_value = "3")] pub iterations: usize, - /// Number of partitions to process in parallel - #[structopt(short = "n", long = "partitions", default_value = "2")] - pub partitions: usize, + /// Number of partitions to process in parallel. Defaults to number of available cores. + #[structopt(short = "n", long = "partitions")] + pub partitions: Option, /// Batch size when reading CSV or Parquet files #[structopt(short = "s", long = "batch-size", default_value = "8192")] @@ -48,7 +48,7 @@ impl CommonOpt { /// Modify the existing config appropriately pub fn update_config(&self, config: SessionConfig) -> SessionConfig { config - .with_target_partitions(self.partitions) + .with_target_partitions(self.partitions.unwrap_or(num_cpus::get())) .with_batch_size(self.batch_size) } } From 1ba87248912254bba073ecf6c65eaaf4845e9285 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Wed, 22 Nov 2023 10:53:34 +0100 Subject: [PATCH 303/572] Update prost-build requirement from =0.12.2 to =0.12.3 (#8298) Updates the requirements on [prost-build](https://github.com/tokio-rs/prost) to permit the latest version. - [Release notes](https://github.com/tokio-rs/prost/releases) - [Commits](https://github.com/tokio-rs/prost/compare/v0.12.2...v0.12.3) --- updated-dependencies: - dependency-name: prost-build dependency-type: direct:production ... Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- datafusion/proto/gen/Cargo.toml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/datafusion/proto/gen/Cargo.toml b/datafusion/proto/gen/Cargo.toml index f58357c6c5d9..8b3f3f98a8a1 100644 --- a/datafusion/proto/gen/Cargo.toml +++ b/datafusion/proto/gen/Cargo.toml @@ -32,4 +32,4 @@ publish = false [dependencies] # Pin these dependencies so that the generated output is deterministic pbjson-build = "=0.6.2" -prost-build = "=0.12.2" +prost-build = "=0.12.3" From b46b7c0ea27e7c5ec63f5367ed04c9612a32d717 Mon Sep 17 00:00:00 2001 From: Jay Zhan Date: Wed, 22 Nov 2023 19:31:14 +0800 Subject: [PATCH 304/572] Fix Display for List (#8261) * fix display for list Signed-off-by: jayzhan211 * fmt Signed-off-by: jayzhan211 * address comment Signed-off-by: jayzhan211 --------- Signed-off-by: jayzhan211 --- datafusion/common/src/scalar.rs | 21 ++++++++++------ .../sqllogictest/test_files/explain.slt | 25 +++++++++++++++++++ 2 files changed, 38 insertions(+), 8 deletions(-) diff --git a/datafusion/common/src/scalar.rs b/datafusion/common/src/scalar.rs index fd1ceb5fad78..21cd50dea8c7 100644 --- a/datafusion/common/src/scalar.rs +++ b/datafusion/common/src/scalar.rs @@ -34,6 +34,7 @@ use crate::utils::array_into_list_array; use arrow::buffer::{NullBuffer, OffsetBuffer}; use arrow::compute::kernels::numeric::*; use arrow::datatypes::{i256, Fields, SchemaBuilder}; +use arrow::util::display::{ArrayFormatter, FormatOptions}; use arrow::{ array::*, compute::kernels::cast::{cast_with_options, CastOptions}, @@ -2931,12 +2932,14 @@ impl fmt::Display for ScalarValue { )?, None => write!(f, "NULL")?, }, - ScalarValue::List(arr) | ScalarValue::FixedSizeList(arr) => write!( - f, - "{}", - arrow::util::pretty::pretty_format_columns("col", &[arr.to_owned()]) - .unwrap() - )?, + ScalarValue::List(arr) | ScalarValue::FixedSizeList(arr) => { + // ScalarValue List should always have a single element + assert_eq!(arr.len(), 1); + let options = FormatOptions::default().with_display_error(true); + let formatter = ArrayFormatter::try_new(arr, &options).unwrap(); + let value_formatter = formatter.value(0); + write!(f, "{value_formatter}")? + } ScalarValue::Date32(e) => format_option!(f, e)?, ScalarValue::Date64(e) => format_option!(f, e)?, ScalarValue::Time32Second(e) => format_option!(f, e)?, @@ -3011,8 +3014,10 @@ impl fmt::Debug for ScalarValue { } ScalarValue::LargeBinary(None) => write!(f, "LargeBinary({self})"), ScalarValue::LargeBinary(Some(_)) => write!(f, "LargeBinary(\"{self}\")"), - ScalarValue::FixedSizeList(arr) => write!(f, "FixedSizeList([{arr:?}])"), - ScalarValue::List(arr) => write!(f, "List([{arr:?}])"), + ScalarValue::FixedSizeList(_) => write!(f, "FixedSizeList({self})"), + ScalarValue::List(_) => { + write!(f, "List({self})") + } ScalarValue::Date32(_) => write!(f, "Date32(\"{self}\")"), ScalarValue::Date64(_) => write!(f, "Date64(\"{self}\")"), ScalarValue::Time32Second(_) => write!(f, "Time32Second(\"{self}\")"), diff --git a/datafusion/sqllogictest/test_files/explain.slt b/datafusion/sqllogictest/test_files/explain.slt index 129814767ca2..c8eff2f301aa 100644 --- a/datafusion/sqllogictest/test_files/explain.slt +++ b/datafusion/sqllogictest/test_files/explain.slt @@ -362,3 +362,28 @@ GlobalLimitExec: skip=0, fetch=10, statistics=[Rows=Exact(8), Bytes=Absent, [(Co statement ok set datafusion.execution.collect_statistics = false; + +# Explain ArrayFuncions + +statement ok +set datafusion.explain.physical_plan_only = false + +query TT +explain select make_array(make_array(1, 2, 3), make_array(4, 5, 6)); +---- +logical_plan +Projection: List([[1, 2, 3], [4, 5, 6]]) AS make_array(make_array(Int64(1),Int64(2),Int64(3)),make_array(Int64(4),Int64(5),Int64(6))) +--EmptyRelation +physical_plan +ProjectionExec: expr=[[[1, 2, 3], [4, 5, 6]] as make_array(make_array(Int64(1),Int64(2),Int64(3)),make_array(Int64(4),Int64(5),Int64(6)))] +--EmptyExec: produce_one_row=true + +query TT +explain select [[1, 2, 3], [4, 5, 6]]; +---- +logical_plan +Projection: List([[1, 2, 3], [4, 5, 6]]) AS make_array(make_array(Int64(1),Int64(2),Int64(3)),make_array(Int64(4),Int64(5),Int64(6))) +--EmptyRelation +physical_plan +ProjectionExec: expr=[[[1, 2, 3], [4, 5, 6]] as make_array(make_array(Int64(1),Int64(2),Int64(3)),make_array(Int64(4),Int64(5),Int64(6)))] +--EmptyExec: produce_one_row=true From 3dbda1e2cfdbfb974c268887294e6cf3de350f71 Mon Sep 17 00:00:00 2001 From: Jonah Gao Date: Wed, 22 Nov 2023 19:49:00 +0800 Subject: [PATCH 305/572] feat: support customizing column default values for inserting (#8283) * parse column default values * fix clippy * Impl for memroy table * Add tests * Add test * Use plan_datafusion_err * Add comment * Update datafusion/sql/src/planner.rs Co-authored-by: comphead * Fix ci --------- Co-authored-by: comphead --- .../src/datasource/default_table_source.rs | 4 ++ datafusion/core/src/datasource/memory.rs | 16 +++++ datafusion/core/src/datasource/provider.rs | 5 ++ datafusion/core/src/execution/context/mod.rs | 14 ++++- datafusion/expr/src/logical_plan/ddl.rs | 2 + datafusion/expr/src/logical_plan/plan.rs | 2 + datafusion/expr/src/table_source.rs | 5 ++ datafusion/sql/src/planner.rs | 41 +++++++++++- datafusion/sql/src/query.rs | 1 + datafusion/sql/src/statement.rs | 15 ++++- datafusion/sqllogictest/test_files/insert.slt | 62 +++++++++++++++++++ 11 files changed, 160 insertions(+), 7 deletions(-) diff --git a/datafusion/core/src/datasource/default_table_source.rs b/datafusion/core/src/datasource/default_table_source.rs index 00a9c123ceee..fadf01c74c5d 100644 --- a/datafusion/core/src/datasource/default_table_source.rs +++ b/datafusion/core/src/datasource/default_table_source.rs @@ -73,6 +73,10 @@ impl TableSource for DefaultTableSource { fn get_logical_plan(&self) -> Option<&datafusion_expr::LogicalPlan> { self.table_provider.get_logical_plan() } + + fn get_column_default(&self, column: &str) -> Option<&Expr> { + self.table_provider.get_column_default(column) + } } /// Wrap TableProvider in TableSource diff --git a/datafusion/core/src/datasource/memory.rs b/datafusion/core/src/datasource/memory.rs index 6bcaa97a408f..a841518d9c8f 100644 --- a/datafusion/core/src/datasource/memory.rs +++ b/datafusion/core/src/datasource/memory.rs @@ -19,6 +19,7 @@ use datafusion_physical_plan::metrics::MetricsSet; use futures::StreamExt; +use hashbrown::HashMap; use log::debug; use std::any::Any; use std::fmt::{self, Debug}; @@ -56,6 +57,7 @@ pub struct MemTable { schema: SchemaRef, pub(crate) batches: Vec, constraints: Constraints, + column_defaults: HashMap, } impl MemTable { @@ -79,6 +81,7 @@ impl MemTable { .map(|e| Arc::new(RwLock::new(e))) .collect::>(), constraints: Constraints::empty(), + column_defaults: HashMap::new(), }) } @@ -88,6 +91,15 @@ impl MemTable { self } + /// Assign column defaults + pub fn with_column_defaults( + mut self, + column_defaults: HashMap, + ) -> Self { + self.column_defaults = column_defaults; + self + } + /// Create a mem table by reading from another data source pub async fn load( t: Arc, @@ -228,6 +240,10 @@ impl TableProvider for MemTable { None, ))) } + + fn get_column_default(&self, column: &str) -> Option<&Expr> { + self.column_defaults.get(column) + } } /// Implements for writing to a [`MemTable`] diff --git a/datafusion/core/src/datasource/provider.rs b/datafusion/core/src/datasource/provider.rs index 4fe433044e6c..275523405a09 100644 --- a/datafusion/core/src/datasource/provider.rs +++ b/datafusion/core/src/datasource/provider.rs @@ -66,6 +66,11 @@ pub trait TableProvider: Sync + Send { None } + /// Get the default value for a column, if available. + fn get_column_default(&self, _column: &str) -> Option<&Expr> { + None + } + /// Create an [`ExecutionPlan`] for scanning the table with optionally /// specified `projection`, `filter` and `limit`, described below. /// diff --git a/datafusion/core/src/execution/context/mod.rs b/datafusion/core/src/execution/context/mod.rs index f829092570bb..46388f990a9a 100644 --- a/datafusion/core/src/execution/context/mod.rs +++ b/datafusion/core/src/execution/context/mod.rs @@ -529,6 +529,7 @@ impl SessionContext { if_not_exists, or_replace, constraints, + column_defaults, } = cmd; let input = Arc::try_unwrap(input).unwrap_or_else(|e| e.as_ref().clone()); @@ -542,7 +543,12 @@ impl SessionContext { let physical = DataFrame::new(self.state(), input); let batches: Vec<_> = physical.collect_partitioned().await?; - let table = Arc::new(MemTable::try_new(schema, batches)?); + let table = Arc::new( + // pass constraints and column defaults to the mem table. + MemTable::try_new(schema, batches)? + .with_constraints(constraints) + .with_column_defaults(column_defaults.into_iter().collect()), + ); self.register_table(&name, table)?; self.return_empty_dataframe() @@ -557,8 +563,10 @@ impl SessionContext { let batches: Vec<_> = physical.collect_partitioned().await?; let table = Arc::new( - // pass constraints to the mem table. - MemTable::try_new(schema, batches)?.with_constraints(constraints), + // pass constraints and column defaults to the mem table. + MemTable::try_new(schema, batches)? + .with_constraints(constraints) + .with_column_defaults(column_defaults.into_iter().collect()), ); self.register_table(&name, table)?; diff --git a/datafusion/expr/src/logical_plan/ddl.rs b/datafusion/expr/src/logical_plan/ddl.rs index 2c90a3aca754..97551a941abf 100644 --- a/datafusion/expr/src/logical_plan/ddl.rs +++ b/datafusion/expr/src/logical_plan/ddl.rs @@ -228,6 +228,8 @@ pub struct CreateMemoryTable { pub if_not_exists: bool, /// Option to replace table content if table already exists pub or_replace: bool, + /// Default values for columns + pub column_defaults: Vec<(String, Expr)>, } /// Creates a view. diff --git a/datafusion/expr/src/logical_plan/plan.rs b/datafusion/expr/src/logical_plan/plan.rs index a024824c7a5a..69ba42d34a70 100644 --- a/datafusion/expr/src/logical_plan/plan.rs +++ b/datafusion/expr/src/logical_plan/plan.rs @@ -811,6 +811,7 @@ impl LogicalPlan { name, if_not_exists, or_replace, + column_defaults, .. })) => Ok(LogicalPlan::Ddl(DdlStatement::CreateMemoryTable( CreateMemoryTable { @@ -819,6 +820,7 @@ impl LogicalPlan { name: name.clone(), if_not_exists: *if_not_exists, or_replace: *or_replace, + column_defaults: column_defaults.clone(), }, ))), LogicalPlan::Ddl(DdlStatement::CreateView(CreateView { diff --git a/datafusion/expr/src/table_source.rs b/datafusion/expr/src/table_source.rs index 94f26d9158cd..565f48c1c5a9 100644 --- a/datafusion/expr/src/table_source.rs +++ b/datafusion/expr/src/table_source.rs @@ -103,4 +103,9 @@ pub trait TableSource: Sync + Send { fn get_logical_plan(&self) -> Option<&LogicalPlan> { None } + + /// Get the default value for a column, if available. + fn get_column_default(&self, _column: &str) -> Option<&Expr> { + None + } } diff --git a/datafusion/sql/src/planner.rs b/datafusion/sql/src/planner.rs index ca5e260aee05..622e5aca799a 100644 --- a/datafusion/sql/src/planner.rs +++ b/datafusion/sql/src/planner.rs @@ -21,8 +21,9 @@ use std::sync::Arc; use std::vec; use arrow_schema::*; -use datafusion_common::field_not_found; -use datafusion_common::internal_err; +use datafusion_common::{ + field_not_found, internal_err, plan_datafusion_err, SchemaError, +}; use datafusion_expr::WindowUDF; use sqlparser::ast::TimezoneInfo; use sqlparser::ast::{ArrayElemTypeDef, ExactNumberInfo}; @@ -230,6 +231,42 @@ impl<'a, S: ContextProvider> SqlToRel<'a, S> { Ok(Schema::new(fields)) } + /// Returns a vector of (column_name, default_expr) pairs + pub(super) fn build_column_defaults( + &self, + columns: &Vec, + planner_context: &mut PlannerContext, + ) -> Result> { + let mut column_defaults = vec![]; + // Default expressions are restricted, column references are not allowed + let empty_schema = DFSchema::empty(); + let error_desc = |e: DataFusionError| match e { + DataFusionError::SchemaError(SchemaError::FieldNotFound { .. }) => { + plan_datafusion_err!( + "Column reference is not allowed in the DEFAULT expression : {}", + e + ) + } + _ => e, + }; + + for column in columns { + if let Some(default_sql_expr) = + column.options.iter().find_map(|o| match &o.option { + ColumnOption::Default(expr) => Some(expr), + _ => None, + }) + { + let default_expr = self + .sql_to_expr(default_sql_expr.clone(), &empty_schema, planner_context) + .map_err(error_desc)?; + column_defaults + .push((self.normalizer.normalize(column.name.clone()), default_expr)); + } + } + Ok(column_defaults) + } + /// Apply the given TableAlias to the input plan pub(crate) fn apply_table_alias( &self, diff --git a/datafusion/sql/src/query.rs b/datafusion/sql/src/query.rs index 832e2da9c6ec..643f41d84485 100644 --- a/datafusion/sql/src/query.rs +++ b/datafusion/sql/src/query.rs @@ -90,6 +90,7 @@ impl<'a, S: ContextProvider> SqlToRel<'a, S> { input: Arc::new(plan), if_not_exists: false, or_replace: false, + column_defaults: vec![], })) } _ => plan, diff --git a/datafusion/sql/src/statement.rs b/datafusion/sql/src/statement.rs index 49755729d2d5..aa2f0583cb99 100644 --- a/datafusion/sql/src/statement.rs +++ b/datafusion/sql/src/statement.rs @@ -204,6 +204,9 @@ impl<'a, S: ContextProvider> SqlToRel<'a, S> { let mut all_constraints = constraints; let inline_constraints = calc_inline_constraints_from_columns(&columns); all_constraints.extend(inline_constraints); + // Build column default values + let column_defaults = + self.build_column_defaults(&columns, planner_context)?; match query { Some(query) => { let plan = self.query_to_plan(*query, planner_context)?; @@ -250,6 +253,7 @@ impl<'a, S: ContextProvider> SqlToRel<'a, S> { input: Arc::new(plan), if_not_exists, or_replace, + column_defaults, }, ))) } @@ -272,6 +276,7 @@ impl<'a, S: ContextProvider> SqlToRel<'a, S> { input: Arc::new(plan), if_not_exists, or_replace, + column_defaults, }, ))) } @@ -1170,8 +1175,14 @@ impl<'a, S: ContextProvider> SqlToRel<'a, S> { datafusion_expr::Expr::Column(source_field.qualified_column()) .cast_to(target_field.data_type(), source.schema())? } - // Fill the default value for the column, currently only supports NULL. - None => datafusion_expr::Expr::Literal(ScalarValue::Null) + // The value is not specified. Fill in the default value for the column. + None => table_source + .get_column_default(target_field.name()) + .cloned() + .unwrap_or_else(|| { + // If there is no default for the column, then the default is NULL + datafusion_expr::Expr::Literal(ScalarValue::Null) + }) .cast_to(target_field.data_type(), &DFSchema::empty())?, }; Ok(expr.alias(target_field.name())) diff --git a/datafusion/sqllogictest/test_files/insert.slt b/datafusion/sqllogictest/test_files/insert.slt index aacd227cdb76..9734aab9ab07 100644 --- a/datafusion/sqllogictest/test_files/insert.slt +++ b/datafusion/sqllogictest/test_files/insert.slt @@ -350,3 +350,65 @@ insert into bad_new_empty_table values (1); statement ok drop table bad_new_empty_table; + + +### Test for specifying column's default value + +statement ok +create table test_column_defaults( + a int, + b int not null default null, + c int default 100*2+300, + d text default lower('DEFAULT_TEXT'), + e timestamp default now() +) + +query IIITP +insert into test_column_defaults values(1, 10, 100, 'ABC', now()) +---- +1 + +statement error DataFusion error: Execution error: Invalid batch column at '1' has null but schema specifies non-nullable +insert into test_column_defaults(a) values(2) + +query IIITP +insert into test_column_defaults(b) values(20) +---- +1 + +query IIIT rowsort +select a,b,c,d from test_column_defaults +---- +1 10 100 ABC +NULL 20 500 default_text + +statement ok +drop table test_column_defaults + + +# test create table as +statement ok +create table test_column_defaults( + a int, + b int not null default null, + c int default 100*2+300, + d text default lower('DEFAULT_TEXT'), + e timestamp default now() +) as values(1, 10, 100, 'ABC', now()) + +query IIITP +insert into test_column_defaults(b) values(20) +---- +1 + +query IIIT rowsort +select a,b,c,d from test_column_defaults +---- +1 10 100 ABC +NULL 20 500 default_text + +statement ok +drop table test_column_defaults + +statement error DataFusion error: Error during planning: Column reference is not allowed in the DEFAULT expression : Schema error: No field named a. +create table test_column_defaults(a int, b int default a+1) From f2b03443260cade7e43eba568d94d8c09cd002f4 Mon Sep 17 00:00:00 2001 From: Alex Huang Date: Wed, 22 Nov 2023 13:30:11 +0100 Subject: [PATCH 306/572] support `LargeList` for `arrow_cast`, support `ScalarValue::LargeList` (#8290) * support largelist for arrow_cast * fix cli * update tests; * add new_large_list in ScalarValue * fix ci * support LargeList in scalar * modify comment * support largelist for proto --- datafusion/common/src/scalar.rs | 461 ++++++++++++++++-- datafusion/common/src/utils.rs | 14 +- datafusion/proto/proto/datafusion.proto | 1 + datafusion/proto/src/generated/pbjson.rs | 14 + datafusion/proto/src/generated/prost.rs | 4 +- .../proto/src/logical_plan/from_proto.rs | 5 +- datafusion/proto/src/logical_plan/to_proto.rs | 9 +- .../tests/cases/roundtrip_logical_plan.rs | 30 ++ datafusion/sql/src/expr/arrow_cast.rs | 14 + .../sqllogictest/test_files/arrow_typeof.slt | 38 ++ 10 files changed, 546 insertions(+), 44 deletions(-) diff --git a/datafusion/common/src/scalar.rs b/datafusion/common/src/scalar.rs index 21cd50dea8c7..ffa8ab50f862 100644 --- a/datafusion/common/src/scalar.rs +++ b/datafusion/common/src/scalar.rs @@ -30,7 +30,7 @@ use crate::cast::{ }; use crate::error::{DataFusionError, Result, _internal_err, _not_impl_err}; use crate::hash_utils::create_hashes; -use crate::utils::array_into_list_array; +use crate::utils::{array_into_large_list_array, array_into_list_array}; use arrow::buffer::{NullBuffer, OffsetBuffer}; use arrow::compute::kernels::numeric::*; use arrow::datatypes::{i256, Fields, SchemaBuilder}; @@ -104,6 +104,8 @@ pub enum ScalarValue { /// /// The array must be a ListArray with length 1. List(ArrayRef), + /// The array must be a LargeListArray with length 1. + LargeList(ArrayRef), /// Date stored as a signed 32bit int days since UNIX epoch 1970-01-01 Date32(Option), /// Date stored as a signed 64bit int milliseconds since UNIX epoch 1970-01-01 @@ -205,6 +207,8 @@ impl PartialEq for ScalarValue { (FixedSizeList(_), _) => false, (List(v1), List(v2)) => v1.eq(v2), (List(_), _) => false, + (LargeList(v1), LargeList(v2)) => v1.eq(v2), + (LargeList(_), _) => false, (Date32(v1), Date32(v2)) => v1.eq(v2), (Date32(_), _) => false, (Date64(v1), Date64(v2)) => v1.eq(v2), @@ -343,7 +347,38 @@ impl PartialOrd for ScalarValue { None } } + (LargeList(arr1), LargeList(arr2)) => { + if arr1.data_type() == arr2.data_type() { + let list_arr1 = as_large_list_array(arr1); + let list_arr2 = as_large_list_array(arr2); + if list_arr1.len() != list_arr2.len() { + return None; + } + for i in 0..list_arr1.len() { + let arr1 = list_arr1.value(i); + let arr2 = list_arr2.value(i); + + let lt_res = + arrow::compute::kernels::cmp::lt(&arr1, &arr2).ok()?; + let eq_res = + arrow::compute::kernels::cmp::eq(&arr1, &arr2).ok()?; + + for j in 0..lt_res.len() { + if lt_res.is_valid(j) && lt_res.value(j) { + return Some(Ordering::Less); + } + if eq_res.is_valid(j) && !eq_res.value(j) { + return Some(Ordering::Greater); + } + } + } + Some(Ordering::Equal) + } else { + None + } + } (List(_), _) => None, + (LargeList(_), _) => None, (FixedSizeList(_), _) => None, (Date32(v1), Date32(v2)) => v1.partial_cmp(v2), (Date32(_), _) => None, @@ -461,7 +496,7 @@ impl std::hash::Hash for ScalarValue { Binary(v) => v.hash(state), FixedSizeBinary(_, v) => v.hash(state), LargeBinary(v) => v.hash(state), - List(arr) | FixedSizeList(arr) => { + List(arr) | LargeList(arr) | FixedSizeList(arr) => { let arrays = vec![arr.to_owned()]; let hashes_buffer = &mut vec![0; arr.len()]; let random_state = ahash::RandomState::with_seeds(0, 0, 0, 0); @@ -872,9 +907,9 @@ impl ScalarValue { ScalarValue::Binary(_) => DataType::Binary, ScalarValue::FixedSizeBinary(sz, _) => DataType::FixedSizeBinary(*sz), ScalarValue::LargeBinary(_) => DataType::LargeBinary, - ScalarValue::List(arr) | ScalarValue::FixedSizeList(arr) => { - arr.data_type().to_owned() - } + ScalarValue::List(arr) + | ScalarValue::LargeList(arr) + | ScalarValue::FixedSizeList(arr) => arr.data_type().to_owned(), ScalarValue::Date32(_) => DataType::Date32, ScalarValue::Date64(_) => DataType::Date64, ScalarValue::Time32Second(_) => DataType::Time32(TimeUnit::Second), @@ -1065,9 +1100,9 @@ impl ScalarValue { ScalarValue::LargeBinary(v) => v.is_none(), // arr.len() should be 1 for a list scalar, but we don't seem to // enforce that anywhere, so we still check against array length. - ScalarValue::List(arr) | ScalarValue::FixedSizeList(arr) => { - arr.len() == arr.null_count() - } + ScalarValue::List(arr) + | ScalarValue::LargeList(arr) + | ScalarValue::FixedSizeList(arr) => arr.len() == arr.null_count(), ScalarValue::Date32(v) => v.is_none(), ScalarValue::Date64(v) => v.is_none(), ScalarValue::Time32Second(v) => v.is_none(), @@ -1279,10 +1314,10 @@ impl ScalarValue { } macro_rules! build_array_list_primitive { - ($ARRAY_TY:ident, $SCALAR_TY:ident, $NATIVE_TYPE:ident) => {{ - Ok::(Arc::new(ListArray::from_iter_primitive::<$ARRAY_TY, _, _>( - scalars.into_iter().map(|x| match x { - ScalarValue::List(arr) => { + ($ARRAY_TY:ident, $SCALAR_TY:ident, $NATIVE_TYPE:ident, $LIST_TY:ident, $SCALAR_LIST:pat) => {{ + Ok::(Arc::new($LIST_TY::from_iter_primitive::<$ARRAY_TY, _, _>( + scalars.into_iter().map(|x| match x{ + ScalarValue::List(arr) if matches!(x, $SCALAR_LIST) => { // `ScalarValue::List` contains a single element `ListArray`. let list_arr = as_list_array(&arr); if list_arr.is_null(0) { @@ -1295,6 +1330,19 @@ impl ScalarValue { )) } } + ScalarValue::LargeList(arr) if matches!(x, $SCALAR_LIST) =>{ + // `ScalarValue::List` contains a single element `ListArray`. + let list_arr = as_large_list_array(&arr); + if list_arr.is_null(0) { + Ok(None) + } else { + let primitive_arr = + list_arr.values().as_primitive::<$ARRAY_TY>(); + Ok(Some( + primitive_arr.into_iter().collect::>>(), + )) + } + } sv => _internal_err!( "Inconsistent types in ScalarValue::iter_to_array. \ Expected {:?}, got {:?}", @@ -1307,11 +1355,11 @@ impl ScalarValue { } macro_rules! build_array_list_string { - ($BUILDER:ident, $STRING_ARRAY:ident) => {{ - let mut builder = ListBuilder::new($BUILDER::new()); + ($BUILDER:ident, $STRING_ARRAY:ident,$LIST_BUILDER:ident,$SCALAR_LIST:pat) => {{ + let mut builder = $LIST_BUILDER::new($BUILDER::new()); for scalar in scalars.into_iter() { match scalar { - ScalarValue::List(arr) => { + ScalarValue::List(arr) if matches!(scalar, $SCALAR_LIST) => { // `ScalarValue::List` contains a single element `ListArray`. let list_arr = as_list_array(&arr); @@ -1331,6 +1379,26 @@ impl ScalarValue { } builder.append(true); } + ScalarValue::LargeList(arr) if matches!(scalar, $SCALAR_LIST) => { + // `ScalarValue::List` contains a single element `ListArray`. + let list_arr = as_large_list_array(&arr); + + if list_arr.is_null(0) { + builder.append(false); + continue; + } + + let string_arr = $STRING_ARRAY(list_arr.values()); + + for v in string_arr.iter() { + if let Some(v) = v { + builder.values().append_value(v); + } else { + builder.values().append_null(); + } + } + builder.append(true); + } sv => { return _internal_err!( "Inconsistent types in ScalarValue::iter_to_array. \ @@ -1419,46 +1487,227 @@ impl ScalarValue { build_array_primitive!(IntervalMonthDayNanoArray, IntervalMonthDayNano) } DataType::List(fields) if fields.data_type() == &DataType::Int8 => { - build_array_list_primitive!(Int8Type, Int8, i8)? + build_array_list_primitive!( + Int8Type, + Int8, + i8, + ListArray, + ScalarValue::List(_) + )? } DataType::List(fields) if fields.data_type() == &DataType::Int16 => { - build_array_list_primitive!(Int16Type, Int16, i16)? + build_array_list_primitive!( + Int16Type, + Int16, + i16, + ListArray, + ScalarValue::List(_) + )? } DataType::List(fields) if fields.data_type() == &DataType::Int32 => { - build_array_list_primitive!(Int32Type, Int32, i32)? + build_array_list_primitive!( + Int32Type, + Int32, + i32, + ListArray, + ScalarValue::List(_) + )? } DataType::List(fields) if fields.data_type() == &DataType::Int64 => { - build_array_list_primitive!(Int64Type, Int64, i64)? + build_array_list_primitive!( + Int64Type, + Int64, + i64, + ListArray, + ScalarValue::List(_) + )? } DataType::List(fields) if fields.data_type() == &DataType::UInt8 => { - build_array_list_primitive!(UInt8Type, UInt8, u8)? + build_array_list_primitive!( + UInt8Type, + UInt8, + u8, + ListArray, + ScalarValue::List(_) + )? } DataType::List(fields) if fields.data_type() == &DataType::UInt16 => { - build_array_list_primitive!(UInt16Type, UInt16, u16)? + build_array_list_primitive!( + UInt16Type, + UInt16, + u16, + ListArray, + ScalarValue::List(_) + )? } DataType::List(fields) if fields.data_type() == &DataType::UInt32 => { - build_array_list_primitive!(UInt32Type, UInt32, u32)? + build_array_list_primitive!( + UInt32Type, + UInt32, + u32, + ListArray, + ScalarValue::List(_) + )? } DataType::List(fields) if fields.data_type() == &DataType::UInt64 => { - build_array_list_primitive!(UInt64Type, UInt64, u64)? + build_array_list_primitive!( + UInt64Type, + UInt64, + u64, + ListArray, + ScalarValue::List(_) + )? } DataType::List(fields) if fields.data_type() == &DataType::Float32 => { - build_array_list_primitive!(Float32Type, Float32, f32)? + build_array_list_primitive!( + Float32Type, + Float32, + f32, + ListArray, + ScalarValue::List(_) + )? } DataType::List(fields) if fields.data_type() == &DataType::Float64 => { - build_array_list_primitive!(Float64Type, Float64, f64)? + build_array_list_primitive!( + Float64Type, + Float64, + f64, + ListArray, + ScalarValue::List(_) + )? } DataType::List(fields) if fields.data_type() == &DataType::Utf8 => { - build_array_list_string!(StringBuilder, as_string_array) + build_array_list_string!( + StringBuilder, + as_string_array, + ListBuilder, + ScalarValue::List(_) + ) } DataType::List(fields) if fields.data_type() == &DataType::LargeUtf8 => { - build_array_list_string!(LargeStringBuilder, as_largestring_array) + build_array_list_string!( + LargeStringBuilder, + as_largestring_array, + ListBuilder, + ScalarValue::List(_) + ) } DataType::List(_) => { // Fallback case handling homogeneous lists with any ScalarValue element type let list_array = ScalarValue::iter_to_array_list(scalars)?; Arc::new(list_array) } + DataType::LargeList(fields) if fields.data_type() == &DataType::Int8 => { + build_array_list_primitive!( + Int8Type, + Int8, + i8, + LargeListArray, + ScalarValue::LargeList(_) + )? + } + DataType::LargeList(fields) if fields.data_type() == &DataType::Int16 => { + build_array_list_primitive!( + Int16Type, + Int16, + i16, + LargeListArray, + ScalarValue::LargeList(_) + )? + } + DataType::LargeList(fields) if fields.data_type() == &DataType::Int32 => { + build_array_list_primitive!( + Int32Type, + Int32, + i32, + LargeListArray, + ScalarValue::LargeList(_) + )? + } + DataType::LargeList(fields) if fields.data_type() == &DataType::Int64 => { + build_array_list_primitive!( + Int64Type, + Int64, + i64, + LargeListArray, + ScalarValue::LargeList(_) + )? + } + DataType::LargeList(fields) if fields.data_type() == &DataType::UInt8 => { + build_array_list_primitive!( + UInt8Type, + UInt8, + u8, + LargeListArray, + ScalarValue::LargeList(_) + )? + } + DataType::LargeList(fields) if fields.data_type() == &DataType::UInt16 => { + build_array_list_primitive!( + UInt16Type, + UInt16, + u16, + LargeListArray, + ScalarValue::LargeList(_) + )? + } + DataType::LargeList(fields) if fields.data_type() == &DataType::UInt32 => { + build_array_list_primitive!( + UInt32Type, + UInt32, + u32, + LargeListArray, + ScalarValue::LargeList(_) + )? + } + DataType::LargeList(fields) if fields.data_type() == &DataType::UInt64 => { + build_array_list_primitive!( + UInt64Type, + UInt64, + u64, + LargeListArray, + ScalarValue::LargeList(_) + )? + } + DataType::LargeList(fields) if fields.data_type() == &DataType::Float32 => { + build_array_list_primitive!( + Float32Type, + Float32, + f32, + LargeListArray, + ScalarValue::LargeList(_) + )? + } + DataType::LargeList(fields) if fields.data_type() == &DataType::Float64 => { + build_array_list_primitive!( + Float64Type, + Float64, + f64, + LargeListArray, + ScalarValue::LargeList(_) + )? + } + DataType::LargeList(fields) if fields.data_type() == &DataType::Utf8 => { + build_array_list_string!( + StringBuilder, + as_string_array, + LargeListBuilder, + ScalarValue::LargeList(_) + ) + } + DataType::LargeList(fields) if fields.data_type() == &DataType::LargeUtf8 => { + build_array_list_string!( + LargeStringBuilder, + as_largestring_array, + LargeListBuilder, + ScalarValue::LargeList(_) + ) + } + DataType::LargeList(_) => { + // Fallback case handling homogeneous lists with any ScalarValue element type + let list_array = ScalarValue::iter_to_large_array_list(scalars)?; + Arc::new(list_array) + } DataType::Struct(fields) => { // Initialize a Vector to store the ScalarValues for each column let mut columns: Vec> = @@ -1571,7 +1820,6 @@ impl ScalarValue { | DataType::Time64(TimeUnit::Millisecond) | DataType::Duration(_) | DataType::FixedSizeList(_, _) - | DataType::LargeList(_) | DataType::Union(_, _) | DataType::Map(_, _) | DataType::RunEndEncoded(_, _) => { @@ -1639,10 +1887,10 @@ impl ScalarValue { Ok(array) } - /// This function build with nulls with nulls buffer. + /// This function build ListArray with nulls with nulls buffer. fn iter_to_array_list( scalars: impl IntoIterator, - ) -> Result> { + ) -> Result { let mut elements: Vec = vec![]; let mut valid = BooleanBufferBuilder::new(0); let mut offsets = vec![]; @@ -1686,7 +1934,62 @@ impl ScalarValue { let list_array = ListArray::new( Arc::new(Field::new("item", flat_array.data_type().clone(), true)), - OffsetBuffer::::from_lengths(offsets), + OffsetBuffer::from_lengths(offsets), + flat_array, + Some(NullBuffer::new(buffer)), + ); + + Ok(list_array) + } + + /// This function build LargeListArray with nulls with nulls buffer. + fn iter_to_large_array_list( + scalars: impl IntoIterator, + ) -> Result { + let mut elements: Vec = vec![]; + let mut valid = BooleanBufferBuilder::new(0); + let mut offsets = vec![]; + + for scalar in scalars { + if let ScalarValue::List(arr) = scalar { + // `ScalarValue::List` contains a single element `ListArray`. + let list_arr = as_list_array(&arr); + + if list_arr.is_null(0) { + // Repeat previous offset index + offsets.push(0); + + // Element is null + valid.append(false); + } else { + let arr = list_arr.values().to_owned(); + offsets.push(arr.len()); + elements.push(arr); + + // Element is valid + valid.append(true); + } + } else { + return _internal_err!( + "Expected ScalarValue::List element. Received {scalar:?}" + ); + } + } + + // Concatenate element arrays to create single flat array + let element_arrays: Vec<&dyn Array> = + elements.iter().map(|a| a.as_ref()).collect(); + + let flat_array = match arrow::compute::concat(&element_arrays) { + Ok(flat_array) => flat_array, + Err(err) => return Err(DataFusionError::ArrowError(err)), + }; + + let buffer = valid.finish(); + + let list_array = LargeListArray::new( + Arc::new(Field::new("item", flat_array.data_type().clone(), true)), + OffsetBuffer::from_lengths(offsets), flat_array, Some(NullBuffer::new(buffer)), ); @@ -1762,6 +2065,41 @@ impl ScalarValue { Arc::new(array_into_list_array(values)) } + /// Converts `Vec` where each element has type corresponding to + /// `data_type`, to a [`LargeListArray`]. + /// + /// Example + /// ``` + /// use datafusion_common::ScalarValue; + /// use arrow::array::{LargeListArray, Int32Array}; + /// use arrow::datatypes::{DataType, Int32Type}; + /// use datafusion_common::cast::as_large_list_array; + /// + /// let scalars = vec![ + /// ScalarValue::Int32(Some(1)), + /// ScalarValue::Int32(None), + /// ScalarValue::Int32(Some(2)) + /// ]; + /// + /// let array = ScalarValue::new_large_list(&scalars, &DataType::Int32); + /// let result = as_large_list_array(&array).unwrap(); + /// + /// let expected = LargeListArray::from_iter_primitive::( + /// vec![ + /// Some(vec![Some(1), None, Some(2)]) + /// ]); + /// + /// assert_eq!(result, &expected); + /// ``` + pub fn new_large_list(values: &[ScalarValue], data_type: &DataType) -> ArrayRef { + let values = if values.is_empty() { + new_empty_array(data_type) + } else { + Self::iter_to_array(values.iter().cloned()).unwrap() + }; + Arc::new(array_into_large_list_array(values)) + } + /// Converts a scalar value into an array of `size` rows. /// /// # Errors @@ -1889,7 +2227,9 @@ impl ScalarValue { .collect::(), ), }, - ScalarValue::List(arr) | ScalarValue::FixedSizeList(arr) => { + ScalarValue::List(arr) + | ScalarValue::LargeList(arr) + | ScalarValue::FixedSizeList(arr) => { let arrays = std::iter::repeat(arr.as_ref()) .take(size) .collect::>(); @@ -2162,6 +2502,14 @@ impl ScalarValue { ScalarValue::List(arr) } + DataType::LargeList(_) => { + let list_array = as_large_list_array(array); + let nested_array = list_array.value(index); + // Produces a single element `LargeListArray` with the value at `index`. + let arr = Arc::new(array_into_large_list_array(nested_array)); + + ScalarValue::LargeList(arr) + } // TODO: There is no test for FixedSizeList now, add it later DataType::FixedSizeList(_, _) => { let list_array = as_fixed_size_list_array(array)?; @@ -2436,7 +2784,9 @@ impl ScalarValue { ScalarValue::LargeBinary(val) => { eq_array_primitive!(array, index, LargeBinaryArray, val)? } - ScalarValue::List(arr) | ScalarValue::FixedSizeList(arr) => { + ScalarValue::List(arr) + | ScalarValue::LargeList(arr) + | ScalarValue::FixedSizeList(arr) => { let right = array.slice(index, 1); arr == &right } @@ -2562,9 +2912,9 @@ impl ScalarValue { | ScalarValue::LargeBinary(b) => { b.as_ref().map(|b| b.capacity()).unwrap_or_default() } - ScalarValue::List(arr) | ScalarValue::FixedSizeList(arr) => { - arr.get_array_memory_size() - } + ScalarValue::List(arr) + | ScalarValue::LargeList(arr) + | ScalarValue::FixedSizeList(arr) => arr.get_array_memory_size(), ScalarValue::Struct(vals, fields) => { vals.as_ref() .map(|vals| { @@ -2932,7 +3282,9 @@ impl fmt::Display for ScalarValue { )?, None => write!(f, "NULL")?, }, - ScalarValue::List(arr) | ScalarValue::FixedSizeList(arr) => { + ScalarValue::List(arr) + | ScalarValue::LargeList(arr) + | ScalarValue::FixedSizeList(arr) => { // ScalarValue List should always have a single element assert_eq!(arr.len(), 1); let options = FormatOptions::default().with_display_error(true); @@ -3015,9 +3367,8 @@ impl fmt::Debug for ScalarValue { ScalarValue::LargeBinary(None) => write!(f, "LargeBinary({self})"), ScalarValue::LargeBinary(Some(_)) => write!(f, "LargeBinary(\"{self}\")"), ScalarValue::FixedSizeList(_) => write!(f, "FixedSizeList({self})"), - ScalarValue::List(_) => { - write!(f, "List({self})") - } + ScalarValue::List(_) => write!(f, "List({self})"), + ScalarValue::LargeList(_) => write!(f, "LargeList({self})"), ScalarValue::Date32(_) => write!(f, "Date32(\"{self}\")"), ScalarValue::Date64(_) => write!(f, "Date64(\"{self}\")"), ScalarValue::Time32Second(_) => write!(f, "Time32Second(\"{self}\")"), @@ -3644,6 +3995,15 @@ mod tests { assert_eq!(list_array.values().len(), 0); } + #[test] + fn scalar_large_list_null_to_array() { + let list_array_ref = ScalarValue::new_large_list(&[], &DataType::UInt64); + let list_array = as_large_list_array(&list_array_ref); + + assert_eq!(list_array.len(), 1); + assert_eq!(list_array.values().len(), 0); + } + #[test] fn scalar_list_to_array() -> Result<()> { let values = vec![ @@ -3665,6 +4025,27 @@ mod tests { Ok(()) } + #[test] + fn scalar_large_list_to_array() -> Result<()> { + let values = vec![ + ScalarValue::UInt64(Some(100)), + ScalarValue::UInt64(None), + ScalarValue::UInt64(Some(101)), + ]; + let list_array_ref = ScalarValue::new_large_list(&values, &DataType::UInt64); + let list_array = as_large_list_array(&list_array_ref); + assert_eq!(list_array.len(), 1); + assert_eq!(list_array.values().len(), 3); + + let prim_array_ref = list_array.value(0); + let prim_array = as_uint64_array(&prim_array_ref)?; + assert_eq!(prim_array.len(), 3); + assert_eq!(prim_array.value(0), 100); + assert!(prim_array.is_null(1)); + assert_eq!(prim_array.value(2), 101); + Ok(()) + } + /// Creates array directly and via ScalarValue and ensures they are the same macro_rules! check_scalar_iter { ($SCALAR_T:ident, $ARRAYTYPE:ident, $INPUT:expr) => {{ diff --git a/datafusion/common/src/utils.rs b/datafusion/common/src/utils.rs index f031f7880436..12d4f516b4d0 100644 --- a/datafusion/common/src/utils.rs +++ b/datafusion/common/src/utils.rs @@ -25,7 +25,7 @@ use arrow::compute; use arrow::compute::{partition, SortColumn, SortOptions}; use arrow::datatypes::{Field, SchemaRef, UInt32Type}; use arrow::record_batch::RecordBatch; -use arrow_array::{Array, ListArray}; +use arrow_array::{Array, LargeListArray, ListArray}; use sqlparser::ast::Ident; use sqlparser::dialect::GenericDialect; use sqlparser::parser::Parser; @@ -349,6 +349,18 @@ pub fn array_into_list_array(arr: ArrayRef) -> ListArray { ) } +/// Wrap an array into a single element `LargeListArray`. +/// For example `[1, 2, 3]` would be converted into `[[1, 2, 3]]` +pub fn array_into_large_list_array(arr: ArrayRef) -> LargeListArray { + let offsets = OffsetBuffer::from_lengths([arr.len()]); + LargeListArray::new( + Arc::new(Field::new("item", arr.data_type().to_owned(), true)), + offsets, + arr, + None, + ) +} + /// Wrap arrays into a single element `ListArray`. /// /// Example: diff --git a/datafusion/proto/proto/datafusion.proto b/datafusion/proto/proto/datafusion.proto index 9197343d749e..d43d19f85842 100644 --- a/datafusion/proto/proto/datafusion.proto +++ b/datafusion/proto/proto/datafusion.proto @@ -984,6 +984,7 @@ message ScalarValue{ // Literal Date32 value always has a unit of day int32 date_32_value = 14; ScalarTime32Value time32_value = 15; + ScalarListValue large_list_value = 16; ScalarListValue list_value = 17; ScalarListValue fixed_size_list_value = 18; diff --git a/datafusion/proto/src/generated/pbjson.rs b/datafusion/proto/src/generated/pbjson.rs index 8a6360023794..133bbbee8920 100644 --- a/datafusion/proto/src/generated/pbjson.rs +++ b/datafusion/proto/src/generated/pbjson.rs @@ -21965,6 +21965,9 @@ impl serde::Serialize for ScalarValue { scalar_value::Value::Time32Value(v) => { struct_ser.serialize_field("time32Value", v)?; } + scalar_value::Value::LargeListValue(v) => { + struct_ser.serialize_field("largeListValue", v)?; + } scalar_value::Value::ListValue(v) => { struct_ser.serialize_field("listValue", v)?; } @@ -22074,6 +22077,8 @@ impl<'de> serde::Deserialize<'de> for ScalarValue { "date32Value", "time32_value", "time32Value", + "large_list_value", + "largeListValue", "list_value", "listValue", "fixed_size_list_value", @@ -22132,6 +22137,7 @@ impl<'de> serde::Deserialize<'de> for ScalarValue { Float64Value, Date32Value, Time32Value, + LargeListValue, ListValue, FixedSizeListValue, Decimal128Value, @@ -22188,6 +22194,7 @@ impl<'de> serde::Deserialize<'de> for ScalarValue { "float64Value" | "float64_value" => Ok(GeneratedField::Float64Value), "date32Value" | "date_32_value" => Ok(GeneratedField::Date32Value), "time32Value" | "time32_value" => Ok(GeneratedField::Time32Value), + "largeListValue" | "large_list_value" => Ok(GeneratedField::LargeListValue), "listValue" | "list_value" => Ok(GeneratedField::ListValue), "fixedSizeListValue" | "fixed_size_list_value" => Ok(GeneratedField::FixedSizeListValue), "decimal128Value" | "decimal128_value" => Ok(GeneratedField::Decimal128Value), @@ -22325,6 +22332,13 @@ impl<'de> serde::Deserialize<'de> for ScalarValue { return Err(serde::de::Error::duplicate_field("time32Value")); } value__ = map_.next_value::<::std::option::Option<_>>()?.map(scalar_value::Value::Time32Value) +; + } + GeneratedField::LargeListValue => { + if value__.is_some() { + return Err(serde::de::Error::duplicate_field("largeListValue")); + } + value__ = map_.next_value::<::std::option::Option<_>>()?.map(scalar_value::Value::LargeListValue) ; } GeneratedField::ListValue => { diff --git a/datafusion/proto/src/generated/prost.rs b/datafusion/proto/src/generated/prost.rs index 4fb8e1599e4b..503c4b6c73f1 100644 --- a/datafusion/proto/src/generated/prost.rs +++ b/datafusion/proto/src/generated/prost.rs @@ -1200,7 +1200,7 @@ pub struct ScalarFixedSizeBinary { pub struct ScalarValue { #[prost( oneof = "scalar_value::Value", - tags = "33, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 17, 18, 20, 39, 21, 24, 25, 35, 36, 37, 38, 26, 27, 28, 29, 30, 31, 32, 34" + tags = "33, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 20, 39, 21, 24, 25, 35, 36, 37, 38, 26, 27, 28, 29, 30, 31, 32, 34" )] pub value: ::core::option::Option, } @@ -1244,6 +1244,8 @@ pub mod scalar_value { Date32Value(i32), #[prost(message, tag = "15")] Time32Value(super::ScalarTime32Value), + #[prost(message, tag = "16")] + LargeListValue(super::ScalarListValue), #[prost(message, tag = "17")] ListValue(super::ScalarListValue), #[prost(message, tag = "18")] diff --git a/datafusion/proto/src/logical_plan/from_proto.rs b/datafusion/proto/src/logical_plan/from_proto.rs index 4ae45fa52162..8069e017f797 100644 --- a/datafusion/proto/src/logical_plan/from_proto.rs +++ b/datafusion/proto/src/logical_plan/from_proto.rs @@ -660,7 +660,9 @@ impl TryFrom<&protobuf::ScalarValue> for ScalarValue { Value::Float64Value(v) => Self::Float64(Some(*v)), Value::Date32Value(v) => Self::Date32(Some(*v)), // ScalarValue::List is serialized using arrow IPC format - Value::ListValue(scalar_list) | Value::FixedSizeListValue(scalar_list) => { + Value::ListValue(scalar_list) + | Value::FixedSizeListValue(scalar_list) + | Value::LargeListValue(scalar_list) => { let protobuf::ScalarListValue { ipc_message, arrow_data, @@ -703,6 +705,7 @@ impl TryFrom<&protobuf::ScalarValue> for ScalarValue { let arr = record_batch.column(0); match value { Value::ListValue(_) => Self::List(arr.to_owned()), + Value::LargeListValue(_) => Self::LargeList(arr.to_owned()), Value::FixedSizeListValue(_) => Self::FixedSizeList(arr.to_owned()), _ => unreachable!(), } diff --git a/datafusion/proto/src/logical_plan/to_proto.rs b/datafusion/proto/src/logical_plan/to_proto.rs index cf66e3ddd5b5..750eb03e8347 100644 --- a/datafusion/proto/src/logical_plan/to_proto.rs +++ b/datafusion/proto/src/logical_plan/to_proto.rs @@ -1140,7 +1140,9 @@ impl TryFrom<&ScalarValue> for protobuf::ScalarValue { } // ScalarValue::List and ScalarValue::FixedSizeList are serialized using // Arrow IPC messages as a single column RecordBatch - ScalarValue::List(arr) | ScalarValue::FixedSizeList(arr) => { + ScalarValue::List(arr) + | ScalarValue::LargeList(arr) + | ScalarValue::FixedSizeList(arr) => { // Wrap in a "field_name" column let batch = RecordBatch::try_from_iter(vec![( "field_name", @@ -1174,6 +1176,11 @@ impl TryFrom<&ScalarValue> for protobuf::ScalarValue { scalar_list_value, )), }), + ScalarValue::LargeList(_) => Ok(protobuf::ScalarValue { + value: Some(protobuf::scalar_value::Value::LargeListValue( + scalar_list_value, + )), + }), ScalarValue::FixedSizeList(_) => Ok(protobuf::ScalarValue { value: Some(protobuf::scalar_value::Value::FixedSizeListValue( scalar_list_value, diff --git a/datafusion/proto/tests/cases/roundtrip_logical_plan.rs b/datafusion/proto/tests/cases/roundtrip_logical_plan.rs index 2d56967ecffa..acc7f07bfa9f 100644 --- a/datafusion/proto/tests/cases/roundtrip_logical_plan.rs +++ b/datafusion/proto/tests/cases/roundtrip_logical_plan.rs @@ -574,6 +574,7 @@ fn round_trip_scalar_values() { ScalarValue::Utf8(None), ScalarValue::LargeUtf8(None), ScalarValue::List(ScalarValue::new_list(&[], &DataType::Boolean)), + ScalarValue::LargeList(ScalarValue::new_large_list(&[], &DataType::Boolean)), ScalarValue::Date32(None), ScalarValue::Boolean(Some(true)), ScalarValue::Boolean(Some(false)), @@ -674,6 +675,16 @@ fn round_trip_scalar_values() { ], &DataType::Float32, )), + ScalarValue::LargeList(ScalarValue::new_large_list( + &[ + ScalarValue::Float32(Some(-213.1)), + ScalarValue::Float32(None), + ScalarValue::Float32(Some(5.5)), + ScalarValue::Float32(Some(2.0)), + ScalarValue::Float32(Some(1.0)), + ], + &DataType::Float32, + )), ScalarValue::List(ScalarValue::new_list( &[ ScalarValue::List(ScalarValue::new_list(&[], &DataType::Float32)), @@ -690,6 +701,25 @@ fn round_trip_scalar_values() { ], &DataType::List(new_arc_field("item", DataType::Float32, true)), )), + ScalarValue::LargeList(ScalarValue::new_large_list( + &[ + ScalarValue::LargeList(ScalarValue::new_large_list( + &[], + &DataType::Float32, + )), + ScalarValue::LargeList(ScalarValue::new_large_list( + &[ + ScalarValue::Float32(Some(-213.1)), + ScalarValue::Float32(None), + ScalarValue::Float32(Some(5.5)), + ScalarValue::Float32(Some(2.0)), + ScalarValue::Float32(Some(1.0)), + ], + &DataType::Float32, + )), + ], + &DataType::LargeList(new_arc_field("item", DataType::Float32, true)), + )), ScalarValue::FixedSizeList(Arc::new(FixedSizeListArray::from_iter_primitive::< Int32Type, _, diff --git a/datafusion/sql/src/expr/arrow_cast.rs b/datafusion/sql/src/expr/arrow_cast.rs index 8c0184b6d119..ade8b96b5cc2 100644 --- a/datafusion/sql/src/expr/arrow_cast.rs +++ b/datafusion/sql/src/expr/arrow_cast.rs @@ -149,6 +149,7 @@ impl<'a> Parser<'a> { Token::Decimal256 => self.parse_decimal_256(), Token::Dictionary => self.parse_dictionary(), Token::List => self.parse_list(), + Token::LargeList => self.parse_large_list(), tok => Err(make_error( self.val, &format!("finding next type, got unexpected '{tok}'"), @@ -166,6 +167,16 @@ impl<'a> Parser<'a> { )))) } + /// Parses the LargeList type + fn parse_large_list(&mut self) -> Result { + self.expect_token(Token::LParen)?; + let data_type = self.parse_next_type()?; + self.expect_token(Token::RParen)?; + Ok(DataType::LargeList(Arc::new(Field::new( + "item", data_type, true, + )))) + } + /// Parses the next timeunit fn parse_time_unit(&mut self, context: &str) -> Result { match self.next_token()? { @@ -496,6 +507,7 @@ impl<'a> Tokenizer<'a> { "Date64" => Token::SimpleType(DataType::Date64), "List" => Token::List, + "LargeList" => Token::LargeList, "Second" => Token::TimeUnit(TimeUnit::Second), "Millisecond" => Token::TimeUnit(TimeUnit::Millisecond), @@ -585,6 +597,7 @@ enum Token { Integer(i64), DoubleQuotedString(String), List, + LargeList, } impl Display for Token { @@ -592,6 +605,7 @@ impl Display for Token { match self { Token::SimpleType(t) => write!(f, "{t}"), Token::List => write!(f, "List"), + Token::LargeList => write!(f, "LargeList"), Token::Timestamp => write!(f, "Timestamp"), Token::Time32 => write!(f, "Time32"), Token::Time64 => write!(f, "Time64"), diff --git a/datafusion/sqllogictest/test_files/arrow_typeof.slt b/datafusion/sqllogictest/test_files/arrow_typeof.slt index e485251b7342..3fad4d0f61b9 100644 --- a/datafusion/sqllogictest/test_files/arrow_typeof.slt +++ b/datafusion/sqllogictest/test_files/arrow_typeof.slt @@ -338,3 +338,41 @@ select arrow_cast(timestamp '2000-01-01T00:00:00Z', 'Timestamp(Nanosecond, Some( statement error Arrow error: Parser error: Invalid timezone "\+25:00": '\+25:00' is not a valid timezone select arrow_cast(timestamp '2000-01-01T00:00:00', 'Timestamp(Nanosecond, Some( "+25:00" ))'); + + +## List + + +query ? +select arrow_cast('1', 'List(Int64)'); +---- +[1] + +query ? +select arrow_cast(make_array(1, 2, 3), 'List(Int64)'); +---- +[1, 2, 3] + +query T +select arrow_typeof(arrow_cast(make_array(1, 2, 3), 'List(Int64)')); +---- +List(Field { name: "item", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }) + + +## LargeList + + +query ? +select arrow_cast('1', 'LargeList(Int64)'); +---- +[1] + +query ? +select arrow_cast(make_array(1, 2, 3), 'LargeList(Int64)'); +---- +[1, 2, 3] + +query T +select arrow_typeof(arrow_cast(make_array(1, 2, 3), 'LargeList(Int64)')); +---- +LargeList(Field { name: "item", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }) \ No newline at end of file From 98f1bc0171874d181aee8bc654bc81ab22314a29 Mon Sep 17 00:00:00 2001 From: Alex Huang Date: Wed, 22 Nov 2023 16:07:39 +0100 Subject: [PATCH 307/572] Minor: remove useless clone based on Clippy (#8300) --- datafusion/expr/src/interval_arithmetic.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/datafusion/expr/src/interval_arithmetic.rs b/datafusion/expr/src/interval_arithmetic.rs index c85c6fc66bc8..5d34fe91c3ac 100644 --- a/datafusion/expr/src/interval_arithmetic.rs +++ b/datafusion/expr/src/interval_arithmetic.rs @@ -698,7 +698,7 @@ impl Interval { // We want 0 to be approachable from both negative and positive sides. let zero_point = match &dt { DataType::Float32 | DataType::Float64 => Self::new(zero.clone(), zero), - _ => Self::new(prev_value(zero.clone()), next_value(zero.clone())), + _ => Self::new(prev_value(zero.clone()), next_value(zero)), }; // Exit early with an unbounded interval if zero is strictly inside the From 9619f02db79c794212437415b1e6a53b44eef4c9 Mon Sep 17 00:00:00 2001 From: Mustafa Akur <106137913+mustafasrepo@users.noreply.github.com> Date: Thu, 23 Nov 2023 10:01:22 +0300 Subject: [PATCH 308/572] Calculate ordering equivalence for expressions (rather than just columns) (#8281) * Complex exprs requirement support (#215) * Discover ordering of complex expressions in group by and window partition by * Remove unnecessary tests * Update comments * Minor changes * Better projection support complex expression support * Fix failing test * Simplifications * Simplifications * Add is end flag * Simplifications * Simplifications * Simplifications * Minor changes * Minor changes * Minor changes * All tests pass * Change implementation of find_longest_permutation * Minor changes * Minor changes * Remove projection section * Remove projection implementation * Fix linter errors * Remove projection sections * Minor changes * Add docstring comments * Add comments * Minor changes * Minor changes * Add comments * simplifications * Minor changes * Review Part 1 * Add new tests * Review Part 2 * Address review feedback * Remove error message check in the test --------- Co-authored-by: Mehmet Ozan Kabak --- .../enforce_distribution.rs | 8 +- .../replace_with_order_preserving_variants.rs | 93 +- datafusion/physical-expr/src/equivalence.rs | 1255 +++++++++++++---- .../physical-expr/src/sort_properties.rs | 1 - datafusion/physical-plan/src/projection.rs | 2 +- .../sqllogictest/test_files/groupby.slt | 101 ++ 6 files changed, 1155 insertions(+), 305 deletions(-) diff --git a/datafusion/core/src/physical_optimizer/enforce_distribution.rs b/datafusion/core/src/physical_optimizer/enforce_distribution.rs index 4aedc3b0d1a9..a34958a6c96d 100644 --- a/datafusion/core/src/physical_optimizer/enforce_distribution.rs +++ b/datafusion/core/src/physical_optimizer/enforce_distribution.rs @@ -3787,7 +3787,7 @@ pub(crate) mod tests { fn repartition_transitively_past_sort_with_projection_and_filter() -> Result<()> { let schema = schema(); let sort_key = vec![PhysicalSortExpr { - expr: col("c", &schema).unwrap(), + expr: col("a", &schema).unwrap(), options: SortOptions::default(), }]; let plan = sort_exec( @@ -3804,9 +3804,9 @@ pub(crate) mod tests { ); let expected = &[ - "SortPreservingMergeExec: [c@2 ASC]", + "SortPreservingMergeExec: [a@0 ASC]", // Expect repartition on the input to the sort (as it can benefit from additional parallelism) - "SortExec: expr=[c@2 ASC]", + "SortExec: expr=[a@0 ASC]", "ProjectionExec: expr=[a@0 as a, b@1 as b, c@2 as c]", "FilterExec: c@2 = 0", // repartition is lowest down @@ -3817,7 +3817,7 @@ pub(crate) mod tests { assert_optimized!(expected, plan.clone(), true); let expected_first_sort_enforcement = &[ - "SortExec: expr=[c@2 ASC]", + "SortExec: expr=[a@0 ASC]", "CoalescePartitionsExec", "ProjectionExec: expr=[a@0 as a, b@1 as b, c@2 as c]", "FilterExec: c@2 = 0", diff --git a/datafusion/core/src/physical_optimizer/replace_with_order_preserving_variants.rs b/datafusion/core/src/physical_optimizer/replace_with_order_preserving_variants.rs index 7f8c9b852cb1..5f130848de11 100644 --- a/datafusion/core/src/physical_optimizer/replace_with_order_preserving_variants.rs +++ b/datafusion/core/src/physical_optimizer/replace_with_order_preserving_variants.rs @@ -357,15 +357,19 @@ mod tests { let physical_plan = sort_preserving_merge_exec(vec![sort_expr("a", &schema)], sort); - let expected_input = ["SortPreservingMergeExec: [a@0 ASC NULLS LAST]", + let expected_input = [ + "SortPreservingMergeExec: [a@0 ASC NULLS LAST]", " SortExec: expr=[a@0 ASC NULLS LAST]", " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8", " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", - " CsvExec: file_groups={1 group: [[file_path]]}, projection=[a, c, d], infinite_source=true, output_ordering=[a@0 ASC NULLS LAST], has_header=true"]; - let expected_optimized = ["SortPreservingMergeExec: [a@0 ASC NULLS LAST]", + " CsvExec: file_groups={1 group: [[file_path]]}, projection=[a, c, d], infinite_source=true, output_ordering=[a@0 ASC NULLS LAST], has_header=true", + ]; + let expected_optimized = [ + "SortPreservingMergeExec: [a@0 ASC NULLS LAST]", " SortPreservingRepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8, sort_exprs=a@0 ASC NULLS LAST", " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", - " CsvExec: file_groups={1 group: [[file_path]]}, projection=[a, c, d], infinite_source=true, output_ordering=[a@0 ASC NULLS LAST], has_header=true"]; + " CsvExec: file_groups={1 group: [[file_path]]}, projection=[a, c, d], infinite_source=true, output_ordering=[a@0 ASC NULLS LAST], has_header=true", + ]; assert_optimized!(expected_input, expected_optimized, physical_plan); Ok(()) } @@ -434,19 +438,20 @@ mod tests { let physical_plan = sort_preserving_merge_exec(vec![sort_expr("a", &schema)], sort); - let expected_input = ["SortPreservingMergeExec: [a@0 ASC NULLS LAST]", + let expected_input = [ + "SortPreservingMergeExec: [a@0 ASC NULLS LAST]", " SortExec: expr=[a@0 ASC NULLS LAST]", " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8", " FilterExec: c@1 > 3", " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", - " CsvExec: file_groups={1 group: [[file_path]]}, projection=[a, c, d], infinite_source=true, output_ordering=[a@0 ASC NULLS LAST], has_header=true"]; + " CsvExec: file_groups={1 group: [[file_path]]}, projection=[a, c, d], infinite_source=true, output_ordering=[a@0 ASC NULLS LAST], has_header=true", + ]; let expected_optimized = [ "SortPreservingMergeExec: [a@0 ASC NULLS LAST]", " SortPreservingRepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8, sort_exprs=a@0 ASC NULLS LAST", " FilterExec: c@1 > 3", " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", " CsvExec: file_groups={1 group: [[file_path]]}, projection=[a, c, d], infinite_source=true, output_ordering=[a@0 ASC NULLS LAST], has_header=true", - ]; assert_optimized!(expected_input, expected_optimized, physical_plan); Ok(()) @@ -466,19 +471,23 @@ mod tests { let physical_plan = sort_preserving_merge_exec(vec![sort_expr("a", &schema)], sort); - let expected_input = ["SortPreservingMergeExec: [a@0 ASC NULLS LAST]", + let expected_input = [ + "SortPreservingMergeExec: [a@0 ASC NULLS LAST]", " SortExec: expr=[a@0 ASC NULLS LAST]", " CoalesceBatchesExec: target_batch_size=8192", " FilterExec: c@1 > 3", " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8", " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", - " CsvExec: file_groups={1 group: [[file_path]]}, projection=[a, c, d], infinite_source=true, output_ordering=[a@0 ASC NULLS LAST], has_header=true"]; - let expected_optimized = ["SortPreservingMergeExec: [a@0 ASC NULLS LAST]", + " CsvExec: file_groups={1 group: [[file_path]]}, projection=[a, c, d], infinite_source=true, output_ordering=[a@0 ASC NULLS LAST], has_header=true", + ]; + let expected_optimized = [ + "SortPreservingMergeExec: [a@0 ASC NULLS LAST]", " CoalesceBatchesExec: target_batch_size=8192", " FilterExec: c@1 > 3", " SortPreservingRepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8, sort_exprs=a@0 ASC NULLS LAST", " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", - " CsvExec: file_groups={1 group: [[file_path]]}, projection=[a, c, d], infinite_source=true, output_ordering=[a@0 ASC NULLS LAST], has_header=true"]; + " CsvExec: file_groups={1 group: [[file_path]]}, projection=[a, c, d], infinite_source=true, output_ordering=[a@0 ASC NULLS LAST], has_header=true", + ]; assert_optimized!(expected_input, expected_optimized, physical_plan); Ok(()) } @@ -499,21 +508,25 @@ mod tests { let physical_plan = sort_preserving_merge_exec(vec![sort_expr("a", &schema)], sort); - let expected_input = ["SortPreservingMergeExec: [a@0 ASC NULLS LAST]", + let expected_input = [ + "SortPreservingMergeExec: [a@0 ASC NULLS LAST]", " SortExec: expr=[a@0 ASC NULLS LAST]", " CoalesceBatchesExec: target_batch_size=8192", " FilterExec: c@1 > 3", " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8", " CoalesceBatchesExec: target_batch_size=8192", " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", - " CsvExec: file_groups={1 group: [[file_path]]}, projection=[a, c, d], infinite_source=true, output_ordering=[a@0 ASC NULLS LAST], has_header=true"]; - let expected_optimized = ["SortPreservingMergeExec: [a@0 ASC NULLS LAST]", + " CsvExec: file_groups={1 group: [[file_path]]}, projection=[a, c, d], infinite_source=true, output_ordering=[a@0 ASC NULLS LAST], has_header=true", + ]; + let expected_optimized = [ + "SortPreservingMergeExec: [a@0 ASC NULLS LAST]", " CoalesceBatchesExec: target_batch_size=8192", " FilterExec: c@1 > 3", " SortPreservingRepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8, sort_exprs=a@0 ASC NULLS LAST", " CoalesceBatchesExec: target_batch_size=8192", " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", - " CsvExec: file_groups={1 group: [[file_path]]}, projection=[a, c, d], infinite_source=true, output_ordering=[a@0 ASC NULLS LAST], has_header=true"]; + " CsvExec: file_groups={1 group: [[file_path]]}, projection=[a, c, d], infinite_source=true, output_ordering=[a@0 ASC NULLS LAST], has_header=true", + ]; assert_optimized!(expected_input, expected_optimized, physical_plan); Ok(()) } @@ -531,18 +544,22 @@ mod tests { let physical_plan: Arc = coalesce_partitions_exec(coalesce_batches_exec); - let expected_input = ["CoalescePartitionsExec", + let expected_input = [ + "CoalescePartitionsExec", " CoalesceBatchesExec: target_batch_size=8192", " FilterExec: c@1 > 3", " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8", " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", - " CsvExec: file_groups={1 group: [[file_path]]}, projection=[a, c, d], infinite_source=true, output_ordering=[a@0 ASC NULLS LAST], has_header=true"]; - let expected_optimized = ["CoalescePartitionsExec", + " CsvExec: file_groups={1 group: [[file_path]]}, projection=[a, c, d], infinite_source=true, output_ordering=[a@0 ASC NULLS LAST], has_header=true", + ]; + let expected_optimized = [ + "CoalescePartitionsExec", " CoalesceBatchesExec: target_batch_size=8192", " FilterExec: c@1 > 3", " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8", " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", - " CsvExec: file_groups={1 group: [[file_path]]}, projection=[a, c, d], infinite_source=true, output_ordering=[a@0 ASC NULLS LAST], has_header=true"]; + " CsvExec: file_groups={1 group: [[file_path]]}, projection=[a, c, d], infinite_source=true, output_ordering=[a@0 ASC NULLS LAST], has_header=true", + ]; assert_optimized!(expected_input, expected_optimized, physical_plan); Ok(()) } @@ -570,7 +587,7 @@ mod tests { " FilterExec: c@1 > 3", " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8", " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", - " CsvExec: file_groups={1 group: [[file_path]]}, projection=[a, c, d], infinite_source=true, output_ordering=[a@0 ASC NULLS LAST], has_header=true" + " CsvExec: file_groups={1 group: [[file_path]]}, projection=[a, c, d], infinite_source=true, output_ordering=[a@0 ASC NULLS LAST], has_header=true", ]; let expected_optimized = [ "SortPreservingMergeExec: [a@0 ASC NULLS LAST]", @@ -603,16 +620,20 @@ mod tests { sort, ); - let expected_input = ["SortPreservingMergeExec: [c@1 ASC]", + let expected_input = [ + "SortPreservingMergeExec: [c@1 ASC]", " SortExec: expr=[c@1 ASC]", " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8", " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", - " CsvExec: file_groups={1 group: [[file_path]]}, projection=[a, c, d], infinite_source=true, output_ordering=[a@0 ASC NULLS LAST], has_header=true"]; - let expected_optimized = ["SortPreservingMergeExec: [c@1 ASC]", + " CsvExec: file_groups={1 group: [[file_path]]}, projection=[a, c, d], infinite_source=true, output_ordering=[a@0 ASC NULLS LAST], has_header=true", + ]; + let expected_optimized = [ + "SortPreservingMergeExec: [c@1 ASC]", " SortExec: expr=[c@1 ASC]", " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8", " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", - " CsvExec: file_groups={1 group: [[file_path]]}, projection=[a, c, d], infinite_source=true, output_ordering=[a@0 ASC NULLS LAST], has_header=true"]; + " CsvExec: file_groups={1 group: [[file_path]]}, projection=[a, c, d], infinite_source=true, output_ordering=[a@0 ASC NULLS LAST], has_header=true", + ]; assert_optimized!(expected_input, expected_optimized, physical_plan); Ok(()) } @@ -628,15 +649,19 @@ mod tests { let physical_plan = sort_exec(vec![sort_expr("a", &schema)], coalesce_partitions, false); - let expected_input = ["SortExec: expr=[a@0 ASC NULLS LAST]", + let expected_input = [ + "SortExec: expr=[a@0 ASC NULLS LAST]", " CoalescePartitionsExec", " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8", " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", - " CsvExec: file_groups={1 group: [[file_path]]}, projection=[a, c, d], infinite_source=true, output_ordering=[a@0 ASC NULLS LAST], has_header=true"]; - let expected_optimized = ["SortPreservingMergeExec: [a@0 ASC NULLS LAST]", + " CsvExec: file_groups={1 group: [[file_path]]}, projection=[a, c, d], infinite_source=true, output_ordering=[a@0 ASC NULLS LAST], has_header=true", + ]; + let expected_optimized = [ + "SortPreservingMergeExec: [a@0 ASC NULLS LAST]", " SortPreservingRepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8, sort_exprs=a@0 ASC NULLS LAST", " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", - " CsvExec: file_groups={1 group: [[file_path]]}, projection=[a, c, d], infinite_source=true, output_ordering=[a@0 ASC NULLS LAST], has_header=true"]; + " CsvExec: file_groups={1 group: [[file_path]]}, projection=[a, c, d], infinite_source=true, output_ordering=[a@0 ASC NULLS LAST], has_header=true", + ]; assert_optimized!(expected_input, expected_optimized, physical_plan); Ok(()) } @@ -766,15 +791,19 @@ mod tests { let physical_plan = sort_preserving_merge_exec(vec![sort_expr("a", &schema)], sort); - let expected_input = ["SortPreservingMergeExec: [a@0 ASC NULLS LAST]", + let expected_input = [ + "SortPreservingMergeExec: [a@0 ASC NULLS LAST]", " SortExec: expr=[a@0 ASC NULLS LAST]", " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8", " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", - " CsvExec: file_groups={1 group: [[file_path]]}, projection=[a, c, d], output_ordering=[a@0 ASC NULLS LAST], has_header=true"]; - let expected_optimized = ["SortPreservingMergeExec: [a@0 ASC NULLS LAST]", + " CsvExec: file_groups={1 group: [[file_path]]}, projection=[a, c, d], output_ordering=[a@0 ASC NULLS LAST], has_header=true", + ]; + let expected_optimized = [ + "SortPreservingMergeExec: [a@0 ASC NULLS LAST]", " SortPreservingRepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8, sort_exprs=a@0 ASC NULLS LAST", " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", - " CsvExec: file_groups={1 group: [[file_path]]}, projection=[a, c, d], output_ordering=[a@0 ASC NULLS LAST], has_header=true"]; + " CsvExec: file_groups={1 group: [[file_path]]}, projection=[a, c, d], output_ordering=[a@0 ASC NULLS LAST], has_header=true", + ]; assert_optimized!(expected_input, expected_optimized, physical_plan, true); Ok(()) } diff --git a/datafusion/physical-expr/src/equivalence.rs b/datafusion/physical-expr/src/equivalence.rs index f3bfe4961622..f9f03300f5e9 100644 --- a/datafusion/physical-expr/src/equivalence.rs +++ b/datafusion/physical-expr/src/equivalence.rs @@ -15,26 +15,23 @@ // specific language governing permissions and limitations // under the License. -use std::collections::HashSet; use std::hash::Hash; use std::sync::Arc; -use crate::expressions::Column; +use crate::expressions::{Column, Literal}; +use crate::physical_expr::deduplicate_physical_exprs; use crate::sort_properties::{ExprOrdering, SortProperties}; use crate::{ physical_exprs_bag_equal, physical_exprs_contains, LexOrdering, LexOrderingRef, LexRequirement, LexRequirementRef, PhysicalExpr, PhysicalSortExpr, PhysicalSortRequirement, }; - use arrow::datatypes::SchemaRef; use arrow_schema::SortOptions; use datafusion_common::tree_node::{Transformed, TreeNode}; use datafusion_common::{JoinSide, JoinType, Result}; -use crate::physical_expr::deduplicate_physical_exprs; -use indexmap::map::Entry; -use indexmap::IndexMap; +use indexmap::IndexSet; /// An `EquivalenceClass` is a set of [`Arc`]s that are known /// to have the same value for all tuples in a relation. These are generated by @@ -141,60 +138,81 @@ impl EquivalenceClass { /// projection. #[derive(Debug, Clone)] pub struct ProjectionMapping { - /// `(source expression)` --> `(target expression)` - /// Indices in the vector corresponds to the indices after projection. - inner: Vec<(Arc, Arc)>, + /// Mapping between source expressions and target expressions. + /// Vector indices correspond to the indices after projection. + map: Vec<(Arc, Arc)>, } impl ProjectionMapping { /// Constructs the mapping between a projection's input and output /// expressions. /// - /// For example, given the input projection expressions (`a+b`, `c+d`) - /// and an output schema with two columns `"c+d"` and `"a+b"` - /// the projection mapping would be + /// For example, given the input projection expressions (`a + b`, `c + d`) + /// and an output schema with two columns `"c + d"` and `"a + b"`, the + /// projection mapping would be: + /// /// ```text - /// [0]: (c+d, col("c+d")) - /// [1]: (a+b, col("a+b")) + /// [0]: (c + d, col("c + d")) + /// [1]: (a + b, col("a + b")) /// ``` - /// where `col("c+d")` means the column named "c+d". + /// + /// where `col("c + d")` means the column named `"c + d"`. pub fn try_new( expr: &[(Arc, String)], input_schema: &SchemaRef, ) -> Result { // Construct a map from the input expressions to the output expression of the projection: - let mut inner = vec![]; - for (expr_idx, (expression, name)) in expr.iter().enumerate() { - let target_expr = Arc::new(Column::new(name, expr_idx)) as _; - - let source_expr = expression.clone().transform_down(&|e| match e - .as_any() - .downcast_ref::( - ) { - Some(col) => { - // Sometimes, expression and its name in the input_schema doesn't match. - // This can cause problems. Hence in here we make sure that expression name - // matches with the name in the inout_schema. - // Conceptually, source_expr and expression should be same. - let idx = col.index(); - let matching_input_field = input_schema.field(idx); - let matching_input_column = - Column::new(matching_input_field.name(), idx); - Ok(Transformed::Yes(Arc::new(matching_input_column))) - } - None => Ok(Transformed::No(e)), - })?; - - inner.push((source_expr, target_expr)); - } - Ok(Self { inner }) + expr.iter() + .enumerate() + .map(|(expr_idx, (expression, name))| { + let target_expr = Arc::new(Column::new(name, expr_idx)) as _; + expression + .clone() + .transform_down(&|e| match e.as_any().downcast_ref::() { + Some(col) => { + // Sometimes, an expression and its name in the input_schema + // doesn't match. This can cause problems, so we make sure + // that the expression name matches with the name in `input_schema`. + // Conceptually, `source_expr` and `expression` should be the same. + let idx = col.index(); + let matching_input_field = input_schema.field(idx); + let matching_input_column = + Column::new(matching_input_field.name(), idx); + Ok(Transformed::Yes(Arc::new(matching_input_column))) + } + None => Ok(Transformed::No(e)), + }) + .map(|source_expr| (source_expr, target_expr)) + }) + .collect::>>() + .map(|map| Self { map }) } /// Iterate over pairs of (source, target) expressions pub fn iter( &self, ) -> impl Iterator, Arc)> + '_ { - self.inner.iter() + self.map.iter() + } + + /// This function returns the target expression for a given source expression. + /// + /// # Arguments + /// + /// * `expr` - Source physical expression. + /// + /// # Returns + /// + /// An `Option` containing the target for the given source expression, + /// where a `None` value means that `expr` is not inside the mapping. + pub fn target_expr( + &self, + expr: &Arc, + ) -> Option> { + self.map + .iter() + .find(|(source, _)| source.eq(expr)) + .map(|(_, target)| target.clone()) } } @@ -213,7 +231,7 @@ impl EquivalenceGroup { /// Creates an equivalence group from the given equivalence classes. fn new(classes: Vec) -> Self { - let mut result = EquivalenceGroup { classes }; + let mut result = Self { classes }; result.remove_redundant_entries(); result } @@ -256,12 +274,13 @@ impl EquivalenceGroup { // If the given left and right sides belong to different classes, // we should unify/bridge these classes. if first_idx != second_idx { - // By convention make sure second_idx is larger than first_idx. + // By convention, make sure `second_idx` is larger than `first_idx`. if first_idx > second_idx { (first_idx, second_idx) = (second_idx, first_idx); } - // Remove second_idx from self.classes then merge its values with class at first_idx. - // Convention above makes sure that first_idx is still valid after second_idx removal. + // Remove the class at `second_idx` and merge its values with + // the class at `first_idx`. The convention above makes sure + // that `first_idx` is still valid after removing `second_idx`. let other_class = self.classes.swap_remove(second_idx); self.classes[first_idx].extend(other_class); } @@ -413,32 +432,37 @@ impl EquivalenceGroup { mapping: &ProjectionMapping, expr: &Arc, ) -> Option> { - let children = expr.children(); - if children.is_empty() { + // First, we try to project expressions with an exact match. If we are + // unable to do this, we consult equivalence classes. + if let Some(target) = mapping.target_expr(expr) { + // If we match the source, we can project directly: + return Some(target); + } else { + // If the given expression is not inside the mapping, try to project + // expressions considering the equivalence classes. for (source, target) in mapping.iter() { - // If we match the source, or an equivalent expression to source, - // then we can project. For example, if we have the mapping - // (a as a1, a + c) and the equivalence class (a, b), expression - // b also projects to a1. - if source.eq(expr) - || self - .get_equivalence_class(source) - .map_or(false, |group| group.contains(expr)) + // If we match an equivalent expression to `source`, then we can + // project. For example, if we have the mapping `(a as a1, a + c)` + // and the equivalence class `(a, b)`, expression `b` projects to `a1`. + if self + .get_equivalence_class(source) + .map_or(false, |group| group.contains(expr)) { return Some(target.clone()); } } } // Project a non-leaf expression by projecting its children. - else if let Some(children) = children + let children = expr.children(); + if children.is_empty() { + // Leaf expression should be inside mapping. + return None; + } + children .into_iter() .map(|child| self.project_expr(mapping, &child)) .collect::>>() - { - return Some(expr.clone().with_new_children(children).unwrap()); - } - // Arriving here implies the expression was invalid after projection. - None + .map(|children| expr.clone().with_new_children(children).unwrap()) } /// Projects `ordering` according to the given projection mapping. @@ -502,8 +526,8 @@ impl EquivalenceGroup { Self::new(classes) } - /// Returns the equivalence class that contains `expr`. - /// If none of the equivalence classes contains `expr`, returns `None`. + /// Returns the equivalence class containing `expr`. If no equivalence class + /// contains `expr`, returns `None`. fn get_equivalence_class( &self, expr: &Arc, @@ -656,26 +680,35 @@ impl OrderingEquivalenceClass { self.remove_redundant_entries(); } - /// Removes redundant orderings from this equivalence class. - /// For instance, If we already have the ordering [a ASC, b ASC, c DESC], - /// then there is no need to keep ordering [a ASC, b ASC] in the state. + /// Removes redundant orderings from this equivalence class. For instance, + /// if we already have the ordering `[a ASC, b ASC, c DESC]`, then there is + /// no need to keep ordering `[a ASC, b ASC]` in the state. fn remove_redundant_entries(&mut self) { - let mut idx = 0; - while idx < self.orderings.len() { - let mut removal = false; - for (ordering_idx, ordering) in self.orderings[0..idx].iter().enumerate() { - if let Some(right_finer) = finer_side(ordering, &self.orderings[idx]) { - if right_finer { - self.orderings.swap(ordering_idx, idx); + let mut work = true; + while work { + work = false; + let mut idx = 0; + while idx < self.orderings.len() { + let mut ordering_idx = idx + 1; + let mut removal = self.orderings[idx].is_empty(); + while ordering_idx < self.orderings.len() { + work |= resolve_overlap(&mut self.orderings, idx, ordering_idx); + if self.orderings[idx].is_empty() { + removal = true; + break; + } + work |= resolve_overlap(&mut self.orderings, ordering_idx, idx); + if self.orderings[ordering_idx].is_empty() { + self.orderings.swap_remove(ordering_idx); + } else { + ordering_idx += 1; } - removal = true; - break; } - } - if removal { - self.orderings.swap_remove(idx); - } else { - idx += 1; + if removal { + self.orderings.swap_remove(idx); + } else { + idx += 1; + } } } } @@ -683,8 +716,7 @@ impl OrderingEquivalenceClass { /// Returns the concatenation of all the orderings. This enables merge /// operations to preserve all equivalent orderings simultaneously. pub fn output_ordering(&self) -> Option { - let output_ordering = - self.orderings.iter().flatten().cloned().collect::>(); + let output_ordering = self.orderings.iter().flatten().cloned().collect(); let output_ordering = collapse_lex_ordering(output_ordering); (!output_ordering.is_empty()).then_some(output_ordering) } @@ -741,12 +773,18 @@ pub fn add_offset_to_expr( // an `Ok` value. } -/// Returns `true` if the ordering `rhs` is strictly finer than the ordering `rhs`, -/// `false` if the ordering `lhs` is at least as fine as the ordering `lhs`, and -/// `None` otherwise (i.e. when given orderings are incomparable). -fn finer_side(lhs: LexOrderingRef, rhs: LexOrderingRef) -> Option { - let all_equal = lhs.iter().zip(rhs.iter()).all(|(lhs, rhs)| lhs.eq(rhs)); - all_equal.then_some(lhs.len() < rhs.len()) +/// Trims `orderings[idx]` if some suffix of it overlaps with a prefix of +/// `orderings[pre_idx]`. Returns `true` if there is any overlap, `false` otherwise. +fn resolve_overlap(orderings: &mut [LexOrdering], idx: usize, pre_idx: usize) -> bool { + let length = orderings[idx].len(); + let other_length = orderings[pre_idx].len(); + for overlap in 1..=length.min(other_length) { + if orderings[idx][length - overlap..] == orderings[pre_idx][..overlap] { + orderings[idx].truncate(length - overlap); + return true; + } + } + false } /// A `EquivalenceProperties` object stores useful information related to a schema. @@ -985,36 +1023,56 @@ impl EquivalenceProperties { /// Checks whether the given sort requirements are satisfied by any of the /// existing orderings. pub fn ordering_satisfy_requirement(&self, reqs: LexRequirementRef) -> bool { + let mut eq_properties = self.clone(); // First, standardize the given requirement: - let normalized_reqs = self.normalize_sort_requirements(reqs); - if normalized_reqs.is_empty() { - // Requirements are tautologically satisfied if empty. - return true; + let normalized_reqs = eq_properties.normalize_sort_requirements(reqs); + for normalized_req in normalized_reqs { + // Check whether given ordering is satisfied + if !eq_properties.ordering_satisfy_single(&normalized_req) { + return false; + } + // Treat satisfied keys as constants in subsequent iterations. We + // can do this because the "next" key only matters in a lexicographical + // ordering when the keys to its left have the same values. + // + // Note that these expressions are not properly "constants". This is just + // an implementation strategy confined to this function. + // + // For example, assume that the requirement is `[a ASC, (b + c) ASC]`, + // and existing equivalent orderings are `[a ASC, b ASC]` and `[c ASC]`. + // From the analysis above, we know that `[a ASC]` is satisfied. Then, + // we add column `a` as constant to the algorithm state. This enables us + // to deduce that `(b + c) ASC` is satisfied, given `a` is constant. + eq_properties = + eq_properties.add_constants(std::iter::once(normalized_req.expr)); } - let mut indices = HashSet::new(); - for ordering in self.normalized_oeq_class().iter() { - let match_indices = ordering - .iter() - .map(|sort_expr| { - normalized_reqs - .iter() - .position(|sort_req| sort_expr.satisfy(sort_req, &self.schema)) - }) - .collect::>(); - // Find the largest contiguous increasing sequence starting from the first index: - if let Some(&Some(first)) = match_indices.first() { - indices.insert(first); - let mut iter = match_indices.windows(2); - while let Some([Some(current), Some(next)]) = iter.next() { - if next > current { - indices.insert(*next); - } else { - break; - } - } + true + } + + /// Determines whether the ordering specified by the given sort requirement + /// is satisfied based on the orderings within, equivalence classes, and + /// constant expressions. + /// + /// # Arguments + /// + /// - `req`: A reference to a `PhysicalSortRequirement` for which the ordering + /// satisfaction check will be done. + /// + /// # Returns + /// + /// Returns `true` if the specified ordering is satisfied, `false` otherwise. + fn ordering_satisfy_single(&self, req: &PhysicalSortRequirement) -> bool { + let expr_ordering = self.get_expr_ordering(req.expr.clone()); + let ExprOrdering { expr, state, .. } = expr_ordering; + match state { + SortProperties::Ordered(options) => { + let sort_expr = PhysicalSortExpr { expr, options }; + sort_expr.satisfy(req, self.schema()) } + // Singleton expressions satisfies any ordering. + SortProperties::Singleton => true, + SortProperties::Unordered => false, } - indices.len() == normalized_reqs.len() } /// Checks whether the `given`` sort requirements are equal or more specific @@ -1138,6 +1196,43 @@ impl EquivalenceProperties { self.eq_group.project_expr(projection_mapping, expr) } + /// Projects constants based on the provided `ProjectionMapping`. + /// + /// This function takes a `ProjectionMapping` and identifies/projects + /// constants based on the existing constants and the mapping. It ensures + /// that constants are appropriately propagated through the projection. + /// + /// # Arguments + /// + /// - `mapping`: A reference to a `ProjectionMapping` representing the + /// mapping of source expressions to target expressions in the projection. + /// + /// # Returns + /// + /// Returns a `Vec>` containing the projected constants. + fn projected_constants( + &self, + mapping: &ProjectionMapping, + ) -> Vec> { + // First, project existing constants. For example, assume that `a + b` + // is known to be constant. If the projection were `a as a_new`, `b as b_new`, + // then we would project constant `a + b` as `a_new + b_new`. + let mut projected_constants = self + .constants + .iter() + .flat_map(|expr| self.eq_group.project_expr(mapping, expr)) + .collect::>(); + // Add projection expressions that are known to be constant: + for (source, target) in mapping.iter() { + if self.is_expr_constant(source) + && !physical_exprs_contains(&projected_constants, target) + { + projected_constants.push(target.clone()); + } + } + projected_constants + } + /// Projects the equivalences within according to `projection_mapping` /// and `output_schema`. pub fn project( @@ -1152,7 +1247,8 @@ impl EquivalenceProperties { .collect::>(); for (source, target) in projection_mapping.iter() { let expr_ordering = ExprOrdering::new(source.clone()) - .transform_up(&|expr| update_ordering(expr, self)) + .transform_up(&|expr| Ok(update_ordering(expr, self))) + // Guaranteed to always return `Ok`. .unwrap(); if let SortProperties::Ordered(options) = expr_ordering.state { // Push new ordering to the state. @@ -1165,7 +1261,7 @@ impl EquivalenceProperties { Self { eq_group: self.eq_group.project(projection_mapping), oeq_class: OrderingEquivalenceClass::new(projected_orderings), - constants: vec![], + constants: self.projected_constants(projection_mapping), schema: output_schema, } } @@ -1184,41 +1280,123 @@ impl EquivalenceProperties { &self, exprs: &[Arc], ) -> (LexOrdering, Vec) { - let normalized_exprs = self.eq_group.normalize_exprs(exprs.to_vec()); - // Use a map to associate expression indices with sort options: - let mut ordered_exprs = IndexMap::::new(); - for ordering in self.normalized_oeq_class().iter() { - for sort_expr in ordering { - if let Some(idx) = normalized_exprs - .iter() - .position(|expr| sort_expr.expr.eq(expr)) - { - if let Entry::Vacant(e) = ordered_exprs.entry(idx) { - e.insert(sort_expr.options); + let mut eq_properties = self.clone(); + let mut result = vec![]; + // The algorithm is as follows: + // - Iterate over all the expressions and insert ordered expressions + // into the result. + // - Treat inserted expressions as constants (i.e. add them as constants + // to the state). + // - Continue the above procedure until no expression is inserted; i.e. + // the algorithm reaches a fixed point. + // This algorithm should reach a fixed point in at most `exprs.len()` + // iterations. + let mut search_indices = (0..exprs.len()).collect::>(); + for _idx in 0..exprs.len() { + // Get ordered expressions with their indices. + let ordered_exprs = search_indices + .iter() + .flat_map(|&idx| { + let ExprOrdering { expr, state, .. } = + eq_properties.get_expr_ordering(exprs[idx].clone()); + if let SortProperties::Ordered(options) = state { + Some((PhysicalSortExpr { expr, options }, idx)) + } else { + None } - } else { - // We only consider expressions that correspond to a prefix - // of one of the equivalent orderings we have. - break; - } + }) + .collect::>(); + // We reached a fixed point, exit. + if ordered_exprs.is_empty() { + break; + } + // Remove indices that have an ordering from `search_indices`, and + // treat ordered expressions as constants in subsequent iterations. + // We can do this because the "next" key only matters in a lexicographical + // ordering when the keys to its left have the same values. + // + // Note that these expressions are not properly "constants". This is just + // an implementation strategy confined to this function. + for (PhysicalSortExpr { expr, .. }, idx) in &ordered_exprs { + eq_properties = + eq_properties.add_constants(std::iter::once(expr.clone())); + search_indices.remove(idx); } + // Add new ordered section to the state. + result.extend(ordered_exprs); } - // Construct the lexicographical ordering according to the permutation: - ordered_exprs - .into_iter() - .map(|(idx, options)| { - ( - PhysicalSortExpr { - expr: exprs[idx].clone(), - options, - }, - idx, - ) - }) - .unzip() + result.into_iter().unzip() + } + + /// This function determines whether the provided expression is constant + /// based on the known constants. + /// + /// # Arguments + /// + /// - `expr`: A reference to a `Arc` representing the + /// expression to be checked. + /// + /// # Returns + /// + /// Returns `true` if the expression is constant according to equivalence + /// group, `false` otherwise. + fn is_expr_constant(&self, expr: &Arc) -> bool { + // As an example, assume that we know columns `a` and `b` are constant. + // Then, `a`, `b` and `a + b` will all return `true` whereas `c` will + // return `false`. + let normalized_constants = self.eq_group.normalize_exprs(self.constants.to_vec()); + let normalized_expr = self.eq_group.normalize_expr(expr.clone()); + is_constant_recurse(&normalized_constants, &normalized_expr) + } + + /// Retrieves the ordering information for a given physical expression. + /// + /// This function constructs an `ExprOrdering` object for the provided + /// expression, which encapsulates information about the expression's + /// ordering, including its [`SortProperties`]. + /// + /// # Arguments + /// + /// - `expr`: An `Arc` representing the physical expression + /// for which ordering information is sought. + /// + /// # Returns + /// + /// Returns an `ExprOrdering` object containing the ordering information for + /// the given expression. + pub fn get_expr_ordering(&self, expr: Arc) -> ExprOrdering { + ExprOrdering::new(expr.clone()) + .transform_up(&|expr| Ok(update_ordering(expr, self))) + // Guaranteed to always return `Ok`. + .unwrap() } } +/// This function determines whether the provided expression is constant +/// based on the known constants. +/// +/// # Arguments +/// +/// - `constants`: A `&[Arc]` containing expressions known to +/// be a constant. +/// - `expr`: A reference to a `Arc` representing the expression +/// to check. +/// +/// # Returns +/// +/// Returns `true` if the expression is constant according to equivalence +/// group, `false` otherwise. +fn is_constant_recurse( + constants: &[Arc], + expr: &Arc, +) -> bool { + if physical_exprs_contains(constants, expr) { + return true; + } + let children = expr.children(); + !children.is_empty() && children.iter().all(|c| is_constant_recurse(constants, c)) +} + /// Calculate ordering equivalence properties for the given join operation. pub fn join_equivalence_properties( left: EquivalenceProperties, @@ -1330,27 +1508,26 @@ fn updated_right_ordering_equivalence_class( fn update_ordering( mut node: ExprOrdering, eq_properties: &EquivalenceProperties, -) -> Result> { - if !node.expr.children().is_empty() { +) -> Transformed { + // We have a Column, which is one of the two possible leaf node types: + let normalized_expr = eq_properties.eq_group.normalize_expr(node.expr.clone()); + if eq_properties.is_expr_constant(&normalized_expr) { + node.state = SortProperties::Singleton; + } else if let Some(options) = eq_properties + .normalized_oeq_class() + .get_options(&normalized_expr) + { + node.state = SortProperties::Ordered(options); + } else if !node.expr.children().is_empty() { // We have an intermediate (non-leaf) node, account for its children: node.state = node.expr.get_ordering(&node.children_states); - Ok(Transformed::Yes(node)) - } else if node.expr.as_any().is::() { - // We have a Column, which is one of the two possible leaf node types: - let eq_group = &eq_properties.eq_group; - let normalized_expr = eq_group.normalize_expr(node.expr.clone()); - let oeq_class = &eq_properties.oeq_class; - if let Some(options) = oeq_class.get_options(&normalized_expr) { - node.state = SortProperties::Ordered(options); - Ok(Transformed::Yes(node)) - } else { - Ok(Transformed::No(node)) - } - } else { + } else if node.expr.as_any().is::() { // We have a Literal, which is the other possible leaf node type: node.state = node.expr.get_ordering(&[]); - Ok(Transformed::Yes(node)) + } else { + return Transformed::No(node); } + Transformed::Yes(node) } #[cfg(test)] @@ -1359,14 +1536,16 @@ mod tests { use std::sync::Arc; use super::*; + use crate::execution_props::ExecutionProps; use crate::expressions::{col, lit, BinaryExpr, Column, Literal}; + use crate::functions::create_physical_expr; use arrow::compute::{lexsort_to_indices, SortColumn}; use arrow::datatypes::{DataType, Field, Schema}; - use arrow_array::{ArrayRef, RecordBatch, UInt32Array, UInt64Array}; - use arrow_schema::{Fields, SortOptions}; + use arrow_array::{ArrayRef, Float64Array, RecordBatch, UInt32Array}; + use arrow_schema::{Fields, SortOptions, TimeUnit}; use datafusion_common::{Result, ScalarValue}; - use datafusion_expr::Operator; + use datafusion_expr::{BuiltinScalarFunction, Operator}; use itertools::{izip, Itertools}; use rand::rngs::StdRng; @@ -1432,12 +1611,12 @@ mod tests { // Generate a schema which consists of 6 columns (a, b, c, d, e, f) fn create_test_schema_2() -> Result { - let a = Field::new("a", DataType::Int32, true); - let b = Field::new("b", DataType::Int32, true); - let c = Field::new("c", DataType::Int32, true); - let d = Field::new("d", DataType::Int32, true); - let e = Field::new("e", DataType::Int32, true); - let f = Field::new("f", DataType::Int32, true); + let a = Field::new("a", DataType::Float64, true); + let b = Field::new("b", DataType::Float64, true); + let c = Field::new("c", DataType::Float64, true); + let d = Field::new("d", DataType::Float64, true); + let e = Field::new("e", DataType::Float64, true); + let f = Field::new("f", DataType::Float64, true); let schema = Arc::new(Schema::new(vec![a, b, c, d, e, f])); Ok(schema) @@ -1602,19 +1781,20 @@ mod tests { Field::new("a4", DataType::Int64, true), ])); + // a as a1, a as a2, a as a3, a as a3 + let proj_exprs = vec![ + (col_a.clone(), "a1".to_string()), + (col_a.clone(), "a2".to_string()), + (col_a.clone(), "a3".to_string()), + (col_a.clone(), "a4".to_string()), + ]; + let projection_mapping = ProjectionMapping::try_new(&proj_exprs, &input_schema)?; + // a as a1, a as a2, a as a3, a as a3 let col_a1 = &col("a1", &out_schema)?; let col_a2 = &col("a2", &out_schema)?; let col_a3 = &col("a3", &out_schema)?; let col_a4 = &col("a4", &out_schema)?; - let projection_mapping = ProjectionMapping { - inner: vec![ - (col_a.clone(), col_a1.clone()), - (col_a.clone(), col_a2.clone()), - (col_a.clone(), col_a3.clone()), - (col_a.clone(), col_a4.clone()), - ], - }; let out_properties = input_properties.project(&projection_mapping, out_schema); // At the output a1=a2=a3=a4 @@ -1631,6 +1811,10 @@ mod tests { #[test] fn test_ordering_satisfy() -> Result<()> { + let input_schema = Arc::new(Schema::new(vec![ + Field::new("a", DataType::Int64, true), + Field::new("b", DataType::Int64, true), + ])); let crude = vec![PhysicalSortExpr { expr: Arc::new(Column::new("a", 0)), options: SortOptions::default(), @@ -1646,13 +1830,12 @@ mod tests { }, ]; // finer ordering satisfies, crude ordering should return true - let empty_schema = &Arc::new(Schema::empty()); - let mut eq_properties_finer = EquivalenceProperties::new(empty_schema.clone()); + let mut eq_properties_finer = EquivalenceProperties::new(input_schema.clone()); eq_properties_finer.oeq_class.push(finer.clone()); assert!(eq_properties_finer.ordering_satisfy(&crude)); // Crude ordering doesn't satisfy finer ordering. should return false - let mut eq_properties_crude = EquivalenceProperties::new(empty_schema.clone()); + let mut eq_properties_crude = EquivalenceProperties::new(input_schema.clone()); eq_properties_crude.oeq_class.push(crude.clone()); assert!(!eq_properties_crude.ordering_satisfy(&finer)); Ok(()) @@ -1826,6 +2009,296 @@ mod tests { Ok(()) } + #[test] + fn test_ordering_satisfy_with_equivalence2() -> Result<()> { + let test_schema = create_test_schema()?; + let col_a = &col("a", &test_schema)?; + let col_b = &col("b", &test_schema)?; + let col_c = &col("c", &test_schema)?; + let col_d = &col("d", &test_schema)?; + let col_e = &col("e", &test_schema)?; + let col_f = &col("f", &test_schema)?; + let floor_a = &create_physical_expr( + &BuiltinScalarFunction::Floor, + &[col("a", &test_schema)?], + &test_schema, + &ExecutionProps::default(), + )?; + let floor_f = &create_physical_expr( + &BuiltinScalarFunction::Floor, + &[col("f", &test_schema)?], + &test_schema, + &ExecutionProps::default(), + )?; + let exp_a = &create_physical_expr( + &BuiltinScalarFunction::Exp, + &[col("a", &test_schema)?], + &test_schema, + &ExecutionProps::default(), + )?; + let a_plus_b = Arc::new(BinaryExpr::new( + col_a.clone(), + Operator::Plus, + col_b.clone(), + )) as Arc; + let options = SortOptions { + descending: false, + nulls_first: false, + }; + + let test_cases = vec![ + // ------------ TEST CASE 1 ------------ + ( + // orderings + vec![ + // [a ASC, d ASC, b ASC] + vec![(col_a, options), (col_d, options), (col_b, options)], + // [c ASC] + vec![(col_c, options)], + ], + // equivalence classes + vec![vec![col_a, col_f]], + // constants + vec![col_e], + // requirement [a ASC, b ASC], requirement is not satisfied. + vec![(col_a, options), (col_b, options)], + // expected: requirement is not satisfied. + false, + ), + // ------------ TEST CASE 2 ------------ + ( + // orderings + vec![ + // [a ASC, c ASC, b ASC] + vec![(col_a, options), (col_c, options), (col_b, options)], + // [d ASC] + vec![(col_d, options)], + ], + // equivalence classes + vec![vec![col_a, col_f]], + // constants + vec![col_e], + // requirement [floor(a) ASC], + vec![(floor_a, options)], + // expected: requirement is satisfied. + true, + ), + // ------------ TEST CASE 2.1 ------------ + ( + // orderings + vec![ + // [a ASC, c ASC, b ASC] + vec![(col_a, options), (col_c, options), (col_b, options)], + // [d ASC] + vec![(col_d, options)], + ], + // equivalence classes + vec![vec![col_a, col_f]], + // constants + vec![col_e], + // requirement [floor(f) ASC], (Please note that a=f) + vec![(floor_f, options)], + // expected: requirement is satisfied. + true, + ), + // ------------ TEST CASE 3 ------------ + ( + // orderings + vec![ + // [a ASC, c ASC, b ASC] + vec![(col_a, options), (col_c, options), (col_b, options)], + // [d ASC] + vec![(col_d, options)], + ], + // equivalence classes + vec![vec![col_a, col_f]], + // constants + vec![col_e], + // requirement [a ASC, c ASC, a+b ASC], + vec![(col_a, options), (col_c, options), (&a_plus_b, options)], + // expected: requirement is satisfied. + true, + ), + // ------------ TEST CASE 4 ------------ + ( + // orderings + vec![ + // [a ASC, b ASC, c ASC, d ASC] + vec![ + (col_a, options), + (col_b, options), + (col_c, options), + (col_d, options), + ], + ], + // equivalence classes + vec![vec![col_a, col_f]], + // constants + vec![col_e], + // requirement [floor(a) ASC, a+b ASC], + vec![(floor_a, options), (&a_plus_b, options)], + // expected: requirement is satisfied. + false, + ), + // ------------ TEST CASE 5 ------------ + ( + // orderings + vec![ + // [a ASC, b ASC, c ASC, d ASC] + vec![ + (col_a, options), + (col_b, options), + (col_c, options), + (col_d, options), + ], + ], + // equivalence classes + vec![vec![col_a, col_f]], + // constants + vec![col_e], + // requirement [exp(a) ASC, a+b ASC], + vec![(exp_a, options), (&a_plus_b, options)], + // expected: requirement is not satisfied. + // TODO: If we know that exp function is 1-to-1 function. + // we could have deduced that above requirement is satisfied. + false, + ), + // ------------ TEST CASE 6 ------------ + ( + // orderings + vec![ + // [a ASC, d ASC, b ASC] + vec![(col_a, options), (col_d, options), (col_b, options)], + // [c ASC] + vec![(col_c, options)], + ], + // equivalence classes + vec![vec![col_a, col_f]], + // constants + vec![col_e], + // requirement [a ASC, d ASC, floor(a) ASC], + vec![(col_a, options), (col_d, options), (floor_a, options)], + // expected: requirement is satisfied. + true, + ), + // ------------ TEST CASE 7 ------------ + ( + // orderings + vec![ + // [a ASC, c ASC, b ASC] + vec![(col_a, options), (col_c, options), (col_b, options)], + // [d ASC] + vec![(col_d, options)], + ], + // equivalence classes + vec![vec![col_a, col_f]], + // constants + vec![col_e], + // requirement [a ASC, floor(a) ASC, a + b ASC], + vec![(col_a, options), (floor_a, options), (&a_plus_b, options)], + // expected: requirement is not satisfied. + false, + ), + // ------------ TEST CASE 8 ------------ + ( + // orderings + vec![ + // [a ASC, b ASC, c ASC] + vec![(col_a, options), (col_b, options), (col_c, options)], + // [d ASC] + vec![(col_d, options)], + ], + // equivalence classes + vec![vec![col_a, col_f]], + // constants + vec![col_e], + // requirement [a ASC, c ASC, floor(a) ASC, a + b ASC], + vec![ + (col_a, options), + (col_c, options), + (&floor_a, options), + (&a_plus_b, options), + ], + // expected: requirement is not satisfied. + false, + ), + // ------------ TEST CASE 9 ------------ + ( + // orderings + vec![ + // [a ASC, b ASC, c ASC, d ASC] + vec![ + (col_a, options), + (col_b, options), + (col_c, options), + (col_d, options), + ], + ], + // equivalence classes + vec![vec![col_a, col_f]], + // constants + vec![col_e], + // requirement [a ASC, b ASC, c ASC, floor(a) ASC], + vec![ + (col_a, options), + (col_b, options), + (&col_c, options), + (&floor_a, options), + ], + // expected: requirement is satisfied. + true, + ), + // ------------ TEST CASE 10 ------------ + ( + // orderings + vec![ + // [d ASC, b ASC] + vec![(col_d, options), (col_b, options)], + // [c ASC, a ASC] + vec![(col_c, options), (col_a, options)], + ], + // equivalence classes + vec![vec![col_a, col_f]], + // constants + vec![col_e], + // requirement [c ASC, d ASC, a + b ASC], + vec![(col_c, options), (col_d, options), (&a_plus_b, options)], + // expected: requirement is satisfied. + true, + ), + ]; + + for (orderings, eq_group, constants, reqs, expected) in test_cases { + let err_msg = + format!("error in test orderings: {orderings:?}, eq_group: {eq_group:?}, constants: {constants:?}, reqs: {reqs:?}, expected: {expected:?}"); + let mut eq_properties = EquivalenceProperties::new(test_schema.clone()); + let orderings = convert_to_orderings(&orderings); + eq_properties.add_new_orderings(orderings); + let eq_group = eq_group + .into_iter() + .map(|eq_class| { + let eq_classes = eq_class.into_iter().cloned().collect::>(); + EquivalenceClass::new(eq_classes) + }) + .collect::>(); + let eq_group = EquivalenceGroup::new(eq_group); + eq_properties.add_equivalence_group(eq_group); + + let constants = constants.into_iter().cloned(); + eq_properties = eq_properties.add_constants(constants); + + let reqs = convert_to_sort_exprs(&reqs); + assert_eq!( + eq_properties.ordering_satisfy(&reqs), + expected, + "{}", + err_msg + ); + } + + Ok(()) + } + #[test] fn test_ordering_satisfy_with_equivalence_random() -> Result<()> { const N_RANDOM_SCHEMA: usize = 5; @@ -1865,8 +2338,8 @@ mod tests { table_data_with_properties.clone(), )?; let err_msg = format!( - "Error in test case requirement:{:?}, expected: {:?}", - requirement, expected + "Error in test case requirement:{:?}, expected: {:?}, eq_properties.oeq_class: {:?}, eq_properties.eq_group: {:?}, eq_properties.constants: {:?}", + requirement, expected, eq_properties.oeq_class, eq_properties.eq_group, eq_properties.constants ); // Check whether ordering_satisfy API result and // experimental result matches. @@ -1883,6 +2356,78 @@ mod tests { Ok(()) } + #[test] + fn test_ordering_satisfy_with_equivalence_complex_random() -> Result<()> { + const N_RANDOM_SCHEMA: usize = 100; + const N_ELEMENTS: usize = 125; + const N_DISTINCT: usize = 5; + const SORT_OPTIONS: SortOptions = SortOptions { + descending: false, + nulls_first: false, + }; + + for seed in 0..N_RANDOM_SCHEMA { + // Create a random schema with random properties + let (test_schema, eq_properties) = create_random_schema(seed as u64)?; + // Generate a data that satisfies properties given + let table_data_with_properties = + generate_table_for_eq_properties(&eq_properties, N_ELEMENTS, N_DISTINCT)?; + + let floor_a = create_physical_expr( + &BuiltinScalarFunction::Floor, + &[col("a", &test_schema)?], + &test_schema, + &ExecutionProps::default(), + )?; + let a_plus_b = Arc::new(BinaryExpr::new( + col("a", &test_schema)?, + Operator::Plus, + col("b", &test_schema)?, + )) as Arc; + let exprs = vec![ + col("a", &test_schema)?, + col("b", &test_schema)?, + col("c", &test_schema)?, + col("d", &test_schema)?, + col("e", &test_schema)?, + col("f", &test_schema)?, + floor_a, + a_plus_b, + ]; + + for n_req in 0..=exprs.len() { + for exprs in exprs.iter().combinations(n_req) { + let requirement = exprs + .into_iter() + .map(|expr| PhysicalSortExpr { + expr: expr.clone(), + options: SORT_OPTIONS, + }) + .collect::>(); + let expected = is_table_same_after_sort( + requirement.clone(), + table_data_with_properties.clone(), + )?; + let err_msg = format!( + "Error in test case requirement:{:?}, expected: {:?}, eq_properties.oeq_class: {:?}, eq_properties.eq_group: {:?}, eq_properties.constants: {:?}", + requirement, expected, eq_properties.oeq_class, eq_properties.eq_group, eq_properties.constants + ); + // Check whether ordering_satisfy API result and + // experimental result matches. + + assert_eq!( + eq_properties.ordering_satisfy(&requirement), + (expected | false), + "{}", + err_msg + ); + } + } + } + + Ok(()) + } + #[test] fn test_ordering_satisfy_different_lengths() -> Result<()> { let test_schema = create_test_schema()?; @@ -2018,6 +2563,8 @@ mod tests { let col_a = &col("a", &schema)?; let col_b = &col("b", &schema)?; let col_c = &col("c", &schema)?; + let col_d = &col("d", &schema)?; + let col_e = &col("e", &schema)?; let option_asc = SortOptions { descending: false, @@ -2111,6 +2658,124 @@ mod tests { ], ], ), + // ------- TEST CASE 5 --------- + // Empty ordering + ( + vec![vec![]], + // No ordering in the state (empty ordering is ignored). + vec![], + ), + // ------- TEST CASE 6 --------- + ( + // ORDERINGS GIVEN + vec![ + // [a ASC, b ASC] + vec![(col_a, option_asc), (col_b, option_asc)], + // [b ASC] + vec![(col_b, option_asc)], + ], + // EXPECTED orderings that is succinct. + vec![ + // [a ASC] + vec![(col_a, option_asc)], + // [b ASC] + vec![(col_b, option_asc)], + ], + ), + // ------- TEST CASE 7 --------- + // b, a + // c, a + // d, b, c + ( + // ORDERINGS GIVEN + vec![ + // [b ASC, a ASC] + vec![(col_b, option_asc), (col_a, option_asc)], + // [c ASC, a ASC] + vec![(col_c, option_asc), (col_a, option_asc)], + // [d ASC, b ASC, c ASC] + vec![ + (col_d, option_asc), + (col_b, option_asc), + (col_c, option_asc), + ], + ], + // EXPECTED orderings that is succinct. + vec![ + // [b ASC, a ASC] + vec![(col_b, option_asc), (col_a, option_asc)], + // [c ASC, a ASC] + vec![(col_c, option_asc), (col_a, option_asc)], + // [d ASC] + vec![(col_d, option_asc)], + ], + ), + // ------- TEST CASE 8 --------- + // b, e + // c, a + // d, b, e, c, a + ( + // ORDERINGS GIVEN + vec![ + // [b ASC, e ASC] + vec![(col_b, option_asc), (col_e, option_asc)], + // [c ASC, a ASC] + vec![(col_c, option_asc), (col_a, option_asc)], + // [d ASC, b ASC, e ASC, c ASC, a ASC] + vec![ + (col_d, option_asc), + (col_b, option_asc), + (col_e, option_asc), + (col_c, option_asc), + (col_a, option_asc), + ], + ], + // EXPECTED orderings that is succinct. + vec![ + // [b ASC, e ASC] + vec![(col_b, option_asc), (col_e, option_asc)], + // [c ASC, a ASC] + vec![(col_c, option_asc), (col_a, option_asc)], + // [d ASC] + vec![(col_d, option_asc)], + ], + ), + // ------- TEST CASE 9 --------- + // b + // a, b, c + // d, a, b + ( + // ORDERINGS GIVEN + vec![ + // [b ASC] + vec![(col_b, option_asc)], + // [a ASC, b ASC, c ASC] + vec![ + (col_a, option_asc), + (col_b, option_asc), + (col_c, option_asc), + ], + // [d ASC, a ASC, b ASC] + vec![ + (col_d, option_asc), + (col_a, option_asc), + (col_b, option_asc), + ], + ], + // EXPECTED orderings that is succinct. + vec![ + // [b ASC] + vec![(col_b, option_asc)], + // [a ASC, b ASC, c ASC] + vec![ + (col_a, option_asc), + (col_b, option_asc), + (col_c, option_asc), + ], + // [d ASC] + vec![(col_d, option_asc)], + ], + ), ]; for (orderings, expected) in test_cases { let orderings = convert_to_orderings(&orderings); @@ -2216,13 +2881,16 @@ mod tests { let mut columns = batch.columns().to_vec(); // Create a new unique column - let n_row = batch.num_rows() as u64; - let unique_col = Arc::new(UInt64Array::from_iter_values(0..n_row)) as ArrayRef; + let n_row = batch.num_rows(); + let vals: Vec = (0..n_row).collect::>(); + let vals: Vec = vals.into_iter().map(|val| val as f64).collect(); + let unique_col = Arc::new(Float64Array::from_iter_values(vals)) as ArrayRef; columns.push(unique_col.clone()); // Create a new schema with the added unique column let unique_col_name = "unique"; - let unique_field = Arc::new(Field::new(unique_col_name, DataType::UInt64, false)); + let unique_field = + Arc::new(Field::new(unique_col_name, DataType::Float64, false)); let fields: Vec<_> = original_schema .fields() .iter() @@ -2241,17 +2909,17 @@ mod tests { }); // Convert the required ordering to a list of SortColumn - let sort_columns: Vec<_> = required_ordering + let sort_columns = required_ordering .iter() - .filter_map(|order_expr| { - let col = order_expr.expr.as_any().downcast_ref::()?; - let col_index = schema.column_with_name(col.name())?.0; - Some(SortColumn { - values: new_batch.column(col_index).clone(), + .map(|order_expr| { + let expr_result = order_expr.expr.evaluate(&new_batch)?; + let values = expr_result.into_array(new_batch.num_rows())?; + Ok(SortColumn { + values, options: Some(order_expr.options), }) }) - .collect(); + .collect::>>()?; // Check if the indices after sorting match the initial ordering let sorted_indices = lexsort_to_indices(&sort_columns, None)?; @@ -2292,18 +2960,18 @@ mod tests { // Utility closure to generate random array let mut generate_random_array = |num_elems: usize, max_val: usize| -> ArrayRef { - let values: Vec = (0..num_elems) - .map(|_| rng.gen_range(0..max_val) as u64) + let values: Vec = (0..num_elems) + .map(|_| rng.gen_range(0..max_val) as f64 / 2.0) .collect(); - Arc::new(UInt64Array::from_iter_values(values)) + Arc::new(Float64Array::from_iter_values(values)) }; // Fill constant columns for constant in &eq_properties.constants { let col = constant.as_any().downcast_ref::().unwrap(); let (idx, _field) = schema.column_with_name(col.name()).unwrap(); - let arr = - Arc::new(UInt64Array::from_iter_values(vec![0; n_elem])) as ArrayRef; + let arr = Arc::new(Float64Array::from_iter_values(vec![0 as f64; n_elem])) + as ArrayRef; schema_vec[idx] = Some(arr); } @@ -2620,6 +3288,12 @@ mod tests { let col_d = &col("d", &test_schema)?; let col_e = &col("e", &test_schema)?; let col_h = &col("h", &test_schema)?; + // a + d + let a_plus_d = Arc::new(BinaryExpr::new( + col_a.clone(), + Operator::Plus, + col_d.clone(), + )) as Arc; let option_asc = SortOptions { descending: false, @@ -2650,14 +3324,26 @@ mod tests { vec![col_d, col_e, col_b], vec![ (col_d, option_asc), - (col_b, option_asc), (col_e, option_desc), + (col_b, option_asc), ], ), // TEST CASE 4 (vec![col_b], vec![]), // TEST CASE 5 (vec![col_d], vec![(col_d, option_asc)]), + // TEST CASE 5 + (vec![&a_plus_d], vec![(&a_plus_d, option_asc)]), + // TEST CASE 6 + ( + vec![col_b, col_d], + vec![(col_d, option_asc), (col_b, option_asc)], + ), + // TEST CASE 6 + ( + vec![col_c, col_e], + vec![(col_c, option_asc), (col_e, option_desc)], + ), ]; for (exprs, expected) in test_cases { let exprs = exprs.into_iter().cloned().collect::>(); @@ -2669,6 +3355,82 @@ mod tests { Ok(()) } + #[test] + fn test_find_longest_permutation_random() -> Result<()> { + const N_RANDOM_SCHEMA: usize = 100; + const N_ELEMENTS: usize = 125; + const N_DISTINCT: usize = 5; + + for seed in 0..N_RANDOM_SCHEMA { + // Create a random schema with random properties + let (test_schema, eq_properties) = create_random_schema(seed as u64)?; + // Generate a data that satisfies properties given + let table_data_with_properties = + generate_table_for_eq_properties(&eq_properties, N_ELEMENTS, N_DISTINCT)?; + + let floor_a = create_physical_expr( + &BuiltinScalarFunction::Floor, + &[col("a", &test_schema)?], + &test_schema, + &ExecutionProps::default(), + )?; + let a_plus_b = Arc::new(BinaryExpr::new( + col("a", &test_schema)?, + Operator::Plus, + col("b", &test_schema)?, + )) as Arc; + let exprs = vec![ + col("a", &test_schema)?, + col("b", &test_schema)?, + col("c", &test_schema)?, + col("d", &test_schema)?, + col("e", &test_schema)?, + col("f", &test_schema)?, + floor_a, + a_plus_b, + ]; + + for n_req in 0..=exprs.len() { + for exprs in exprs.iter().combinations(n_req) { + let exprs = exprs.into_iter().cloned().collect::>(); + let (ordering, indices) = + eq_properties.find_longest_permutation(&exprs); + // Make sure that find_longest_permutation return values are consistent + let ordering2 = indices + .iter() + .zip(ordering.iter()) + .map(|(&idx, sort_expr)| PhysicalSortExpr { + expr: exprs[idx].clone(), + options: sort_expr.options, + }) + .collect::>(); + assert_eq!( + ordering, ordering2, + "indices and lexicographical ordering do not match" + ); + + let err_msg = format!( + "Error in test case ordering:{:?}, eq_properties.oeq_class: {:?}, eq_properties.eq_group: {:?}, eq_properties.constants: {:?}", + ordering, eq_properties.oeq_class, eq_properties.eq_group, eq_properties.constants + ); + assert_eq!(ordering.len(), indices.len(), "{}", err_msg); + // Since ordered section satisfies schema, we expect + // that result will be same after sort (e.g sort was unnecessary). + assert!( + is_table_same_after_sort( + ordering.clone(), + table_data_with_properties.clone(), + )?, + "{}", + err_msg + ); + } + } + } + + Ok(()) + } + #[test] fn test_update_ordering() -> Result<()> { let schema = Schema::new(vec![ @@ -2726,11 +3488,14 @@ mod tests { ), ]; for (expr, expected) in test_cases { - let expr_ordering = ExprOrdering::new(expr.clone()); - let expr_ordering = expr_ordering - .transform_up(&|expr| update_ordering(expr, &eq_properties))?; + let leading_orderings = eq_properties + .oeq_class() + .iter() + .flat_map(|ordering| ordering.first().cloned()) + .collect::>(); + let expr_ordering = eq_properties.get_expr_ordering(expr.clone()); let err_msg = format!( - "expr:{:?}, expected: {:?}, actual: {:?}", + "expr:{:?}, expected: {:?}, actual: {:?}, leading_orderings: {leading_orderings:?}", expr, expected, expr_ordering.state ); assert_eq!(expr_ordering.state, expected, "{}", err_msg); @@ -2921,74 +3686,30 @@ mod tests { } #[test] - fn project_empty_output_ordering() -> Result<()> { - let schema = Schema::new(vec![ - Field::new("a", DataType::Int32, true), - Field::new("b", DataType::Int32, true), - Field::new("c", DataType::Int32, true), - ]); - let mut eq_properties = EquivalenceProperties::new(Arc::new(schema.clone())); - let ordering = vec![PhysicalSortExpr { - expr: Arc::new(Column::new("b", 1)), - options: SortOptions::default(), - }]; - eq_properties.add_new_orderings([ordering]); - let projection_mapping = ProjectionMapping { - inner: vec![ - ( - Arc::new(Column::new("b", 1)) as _, - Arc::new(Column::new("b_new", 0)) as _, - ), - ( - Arc::new(Column::new("a", 0)) as _, - Arc::new(Column::new("a_new", 1)) as _, - ), - ], - }; - let projection_schema = Arc::new(Schema::new(vec![ - Field::new("b_new", DataType::Int32, true), - Field::new("a_new", DataType::Int32, true), - ])); - let orderings = eq_properties - .project(&projection_mapping, projection_schema) - .oeq_class() - .output_ordering() - .unwrap_or_default(); - - assert_eq!( - vec![PhysicalSortExpr { - expr: Arc::new(Column::new("b_new", 0)), - options: SortOptions::default(), - }], - orderings - ); - - let schema = Schema::new(vec![ + fn test_expr_consists_of_constants() -> Result<()> { + let schema = Arc::new(Schema::new(vec![ Field::new("a", DataType::Int32, true), Field::new("b", DataType::Int32, true), Field::new("c", DataType::Int32, true), - ]); - let eq_properties = EquivalenceProperties::new(Arc::new(schema)); - let projection_mapping = ProjectionMapping { - inner: vec![ - ( - Arc::new(Column::new("c", 2)) as _, - Arc::new(Column::new("c_new", 0)) as _, - ), - ( - Arc::new(Column::new("b", 1)) as _, - Arc::new(Column::new("b_new", 1)) as _, - ), - ], - }; - let projection_schema = Arc::new(Schema::new(vec![ - Field::new("c_new", DataType::Int32, true), - Field::new("b_new", DataType::Int32, true), + Field::new("d", DataType::Int32, true), + Field::new("ts", DataType::Timestamp(TimeUnit::Nanosecond, None), true), ])); - let projected = eq_properties.project(&projection_mapping, projection_schema); - // After projection there is no ordering. - assert!(projected.oeq_class().output_ordering().is_none()); - + let col_a = col("a", &schema)?; + let col_b = col("b", &schema)?; + let col_d = col("d", &schema)?; + let b_plus_d = Arc::new(BinaryExpr::new( + col_b.clone(), + Operator::Plus, + col_d.clone(), + )) as Arc; + + let constants = vec![col_a.clone(), col_b.clone()]; + let expr = b_plus_d.clone(); + assert!(!is_constant_recurse(&constants, &expr)); + + let constants = vec![col_a.clone(), col_b.clone(), col_d.clone()]; + let expr = b_plus_d.clone(); + assert!(is_constant_recurse(&constants, &expr)); Ok(()) } } diff --git a/datafusion/physical-expr/src/sort_properties.rs b/datafusion/physical-expr/src/sort_properties.rs index 25729640ec99..f8648abdf7a7 100644 --- a/datafusion/physical-expr/src/sort_properties.rs +++ b/datafusion/physical-expr/src/sort_properties.rs @@ -18,7 +18,6 @@ use std::{ops::Neg, sync::Arc}; use crate::PhysicalExpr; - use arrow_schema::SortOptions; use datafusion_common::tree_node::{TreeNode, VisitRecursion}; use datafusion_common::Result; diff --git a/datafusion/physical-plan/src/projection.rs b/datafusion/physical-plan/src/projection.rs index dfb860bc8cf3..2e1d3dbf94f5 100644 --- a/datafusion/physical-plan/src/projection.rs +++ b/datafusion/physical-plan/src/projection.rs @@ -38,10 +38,10 @@ use arrow::record_batch::{RecordBatch, RecordBatchOptions}; use datafusion_common::stats::Precision; use datafusion_common::Result; use datafusion_execution::TaskContext; +use datafusion_physical_expr::equivalence::ProjectionMapping; use datafusion_physical_expr::expressions::{Literal, UnKnownColumn}; use datafusion_physical_expr::EquivalenceProperties; -use datafusion_physical_expr::equivalence::ProjectionMapping; use futures::stream::{Stream, StreamExt}; use log::trace; diff --git a/datafusion/sqllogictest/test_files/groupby.slt b/datafusion/sqllogictest/test_files/groupby.slt index 4438d69af306..756d3f737439 100644 --- a/datafusion/sqllogictest/test_files/groupby.slt +++ b/datafusion/sqllogictest/test_files/groupby.slt @@ -3842,6 +3842,107 @@ ProjectionExec: expr=[SUM(alias1)@1 as SUM(DISTINCT t1.x), MAX(alias1)@2 as MAX( --------------------ProjectionExec: expr=[CAST(x@0 AS Float64) as CAST(t1.x AS Float64)t1.x, y@1 as y] ----------------------MemoryExec: partitions=1, partition_sizes=[1] +# create an unbounded table that contains ordered timestamp. +statement ok +CREATE UNBOUNDED EXTERNAL TABLE unbounded_csv_with_timestamps ( + name VARCHAR, + ts TIMESTAMP +) +STORED AS CSV +WITH ORDER (ts DESC) +LOCATION '../core/tests/data/timestamps.csv' + +# below query should work in streaming mode. +query TT +EXPLAIN SELECT date_bin('15 minutes', ts) as time_chunks + FROM unbounded_csv_with_timestamps + GROUP BY date_bin('15 minutes', ts) + ORDER BY time_chunks DESC + LIMIT 5; +---- +logical_plan +Limit: skip=0, fetch=5 +--Sort: time_chunks DESC NULLS FIRST, fetch=5 +----Projection: date_bin(Utf8("15 minutes"),unbounded_csv_with_timestamps.ts) AS time_chunks +------Aggregate: groupBy=[[date_bin(IntervalMonthDayNano("900000000000"), unbounded_csv_with_timestamps.ts) AS date_bin(Utf8("15 minutes"),unbounded_csv_with_timestamps.ts)]], aggr=[[]] +--------TableScan: unbounded_csv_with_timestamps projection=[ts] +physical_plan +GlobalLimitExec: skip=0, fetch=5 +--SortPreservingMergeExec: [time_chunks@0 DESC], fetch=5 +----ProjectionExec: expr=[date_bin(Utf8("15 minutes"),unbounded_csv_with_timestamps.ts)@0 as time_chunks] +------AggregateExec: mode=FinalPartitioned, gby=[date_bin(Utf8("15 minutes"),unbounded_csv_with_timestamps.ts)@0 as date_bin(Utf8("15 minutes"),unbounded_csv_with_timestamps.ts)], aggr=[], ordering_mode=Sorted +--------CoalesceBatchesExec: target_batch_size=2 +----------SortPreservingRepartitionExec: partitioning=Hash([date_bin(Utf8("15 minutes"),unbounded_csv_with_timestamps.ts)@0], 8), input_partitions=8, sort_exprs=date_bin(Utf8("15 minutes"),unbounded_csv_with_timestamps.ts)@0 DESC +------------AggregateExec: mode=Partial, gby=[date_bin(900000000000, ts@0) as date_bin(Utf8("15 minutes"),unbounded_csv_with_timestamps.ts)], aggr=[], ordering_mode=Sorted +--------------RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1 +----------------StreamingTableExec: partition_sizes=1, projection=[ts], infinite_source=true, output_ordering=[ts@0 DESC] + +query P +SELECT date_bin('15 minutes', ts) as time_chunks + FROM unbounded_csv_with_timestamps + GROUP BY date_bin('15 minutes', ts) + ORDER BY time_chunks DESC + LIMIT 5; +---- +2018-12-13T12:00:00 +2018-11-13T17:00:00 + +# Since extract is not a monotonic function, below query should not run. +# when source is unbounded. +query error +SELECT extract(month from ts) as months + FROM unbounded_csv_with_timestamps + GROUP BY extract(month from ts) + ORDER BY months DESC + LIMIT 5; + +# Create a table where timestamp is ordered +statement ok +CREATE EXTERNAL TABLE csv_with_timestamps ( + name VARCHAR, + ts TIMESTAMP +) +STORED AS CSV +WITH ORDER (ts DESC) +LOCATION '../core/tests/data/timestamps.csv'; + +# below query should run since it operates on a bounded source and have a sort +# at the top of its plan. +query TT +EXPLAIN SELECT extract(month from ts) as months + FROM csv_with_timestamps + GROUP BY extract(month from ts) + ORDER BY months DESC + LIMIT 5; +---- +logical_plan +Limit: skip=0, fetch=5 +--Sort: months DESC NULLS FIRST, fetch=5 +----Projection: date_part(Utf8("MONTH"),csv_with_timestamps.ts) AS months +------Aggregate: groupBy=[[date_part(Utf8("MONTH"), csv_with_timestamps.ts)]], aggr=[[]] +--------TableScan: csv_with_timestamps projection=[ts] +physical_plan +GlobalLimitExec: skip=0, fetch=5 +--SortPreservingMergeExec: [months@0 DESC], fetch=5 +----SortExec: TopK(fetch=5), expr=[months@0 DESC] +------ProjectionExec: expr=[date_part(Utf8("MONTH"),csv_with_timestamps.ts)@0 as months] +--------AggregateExec: mode=FinalPartitioned, gby=[date_part(Utf8("MONTH"),csv_with_timestamps.ts)@0 as date_part(Utf8("MONTH"),csv_with_timestamps.ts)], aggr=[] +----------CoalesceBatchesExec: target_batch_size=2 +------------RepartitionExec: partitioning=Hash([date_part(Utf8("MONTH"),csv_with_timestamps.ts)@0], 8), input_partitions=8 +--------------AggregateExec: mode=Partial, gby=[date_part(MONTH, ts@0) as date_part(Utf8("MONTH"),csv_with_timestamps.ts)], aggr=[] +----------------RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1 +------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/timestamps.csv]]}, projection=[ts], output_ordering=[ts@0 DESC], has_header=false + +query R +SELECT extract(month from ts) as months + FROM csv_with_timestamps + GROUP BY extract(month from ts) + ORDER BY months DESC + LIMIT 5; +---- +12 +11 + statement ok drop table t1 From b648d4e22e82989c65523e62312e1995a1543888 Mon Sep 17 00:00:00 2001 From: Kirill Zaborsky Date: Thu, 23 Nov 2023 15:50:19 +0300 Subject: [PATCH 309/572] Fix sqllogictests links in contributor-guide/index.md and README.md (#8314) The crate was moved so the links need to be updated --- datafusion/sqllogictest/README.md | 2 +- docs/source/contributor-guide/index.md | 6 +++--- 2 files changed, 4 insertions(+), 4 deletions(-) diff --git a/datafusion/sqllogictest/README.md b/datafusion/sqllogictest/README.md index 0349ed852f46..bda00a2dce0f 100644 --- a/datafusion/sqllogictest/README.md +++ b/datafusion/sqllogictest/README.md @@ -240,7 +240,7 @@ query - NULL values are rendered as `NULL`, - empty strings are rendered as `(empty)`, - boolean values are rendered as `true`/`false`, - - this list can be not exhaustive, check the `datafusion/core/tests/sqllogictests/src/engines/conversion.rs` for + - this list can be not exhaustive, check the `datafusion/sqllogictest/src/engines/conversion.rs` for details. - `sort_mode`: If included, it must be one of `nosort` (**default**), `rowsort`, or `valuesort`. In `nosort` mode, the results appear in exactly the order in which they were received from the database engine. The `nosort` mode should diff --git a/docs/source/contributor-guide/index.md b/docs/source/contributor-guide/index.md index 1a8b5e427087..8d69ade83d72 100644 --- a/docs/source/contributor-guide/index.md +++ b/docs/source/contributor-guide/index.md @@ -151,7 +151,7 @@ Tests for code in an individual module are defined in the same source file with ### sqllogictests Tests -DataFusion's SQL implementation is tested using [sqllogictest](https://github.com/apache/arrow-datafusion/tree/main/datafusion/core/tests/sqllogictests) which are run like any other Rust test using `cargo test --test sqllogictests`. +DataFusion's SQL implementation is tested using [sqllogictest](https://github.com/apache/arrow-datafusion/tree/main/datafusion/sqllogictest) which are run like any other Rust test using `cargo test --test sqllogictests`. `sqllogictests` tests may be less convenient for new contributors who are familiar with writing `.rs` tests as they require learning another tool. However, `sqllogictest` based tests are much easier to develop and maintain as they 1) do not require a slow recompile/link cycle and 2) can be automatically updated via `cargo test --test sqllogictests -- --complete`. @@ -243,8 +243,8 @@ Below is a checklist of what you need to do to add a new aggregate function to D - a new line in `signature` with the signature of the function (number and types of its arguments) - a new line in `create_aggregate_expr` mapping the built-in to the implementation - tests to the function. -- In [core/tests/sqllogictests/test_files](../../../datafusion/core/tests/sqllogictests/test_files), add new `sqllogictest` integration tests where the function is called through SQL against well known data and returns the expected result. - - Documentation for `sqllogictest` [here](../../../datafusion/core/tests/sqllogictests/README.md) +- In [sqllogictest/test_files](../../../datafusion/sqllogictest/test_files), add new `sqllogictest` integration tests where the function is called through SQL against well known data and returns the expected result. + - Documentation for `sqllogictest` [here](../../../datafusion/sqllogictest/README.md) - Add SQL reference documentation [here](../../../docs/source/user-guide/sql/aggregate_functions.md) ### How to display plans graphically From f8dcc64ca3be4db315aa2e4d4da953ec8a3c87bb Mon Sep 17 00:00:00 2001 From: Yongting You <2010youy01@gmail.com> Date: Sun, 26 Nov 2023 03:49:18 -0800 Subject: [PATCH 310/572] Refactor: Unify `Expr::ScalarFunction` and `Expr::ScalarUDF`, introduce unresolved functions by name (#8258) * Refactor Expr::ScalarFunction * Remove Expr::ScalarUDF * review comments * make name() return &str * fix fmt * fix after merge --- .../core/src/datasource/listing/helpers.rs | 54 +++++----- datafusion/core/src/physical_planner.rs | 18 ++-- datafusion/expr/src/expr.rs | 79 +++++++++------ datafusion/expr/src/expr_fn.rs | 98 +++++++++++-------- datafusion/expr/src/expr_schema.rs | 64 ++++++------ datafusion/expr/src/lib.rs | 2 +- datafusion/expr/src/tree_node/expr.rs | 25 +++-- datafusion/expr/src/udf.rs | 5 +- datafusion/expr/src/utils.rs | 1 - .../optimizer/src/analyzer/type_coercion.rs | 75 +++++++------- datafusion/optimizer/src/push_down_filter.rs | 32 ++++-- .../simplify_expressions/expr_simplifier.rs | 59 +++++++---- .../src/simplify_expressions/utils.rs | 14 ++- datafusion/physical-expr/src/planner.rs | 70 +++++++------ .../proto/src/logical_plan/from_proto.rs | 2 +- datafusion/proto/src/logical_plan/to_proto.rs | 52 ++++++---- .../tests/cases/roundtrip_logical_plan.rs | 7 +- datafusion/sql/src/expr/function.rs | 4 +- datafusion/sql/src/expr/value.rs | 9 +- .../substrait/src/logical_plan/consumer.rs | 7 +- .../substrait/src/logical_plan/producer.rs | 13 ++- 21 files changed, 419 insertions(+), 271 deletions(-) diff --git a/datafusion/core/src/datasource/listing/helpers.rs b/datafusion/core/src/datasource/listing/helpers.rs index 322d65d5645d..f9b02f4d0c10 100644 --- a/datafusion/core/src/datasource/listing/helpers.rs +++ b/datafusion/core/src/datasource/listing/helpers.rs @@ -38,9 +38,8 @@ use super::PartitionedFile; use crate::datasource::listing::ListingTableUrl; use crate::execution::context::SessionState; use datafusion_common::tree_node::{TreeNode, VisitRecursion}; -use datafusion_common::{Column, DFField, DFSchema, DataFusionError}; -use datafusion_expr::expr::ScalarUDF; -use datafusion_expr::{Expr, Volatility}; +use datafusion_common::{internal_err, Column, DFField, DFSchema, DataFusionError}; +use datafusion_expr::{Expr, ScalarFunctionDefinition, Volatility}; use datafusion_physical_expr::create_physical_expr; use datafusion_physical_expr::execution_props::ExecutionProps; use object_store::path::Path; @@ -54,13 +53,13 @@ use object_store::{ObjectMeta, ObjectStore}; pub fn expr_applicable_for_cols(col_names: &[String], expr: &Expr) -> bool { let mut is_applicable = true; expr.apply(&mut |expr| { - Ok(match expr { + match expr { Expr::Column(Column { ref name, .. }) => { is_applicable &= col_names.contains(name); if is_applicable { - VisitRecursion::Skip + Ok(VisitRecursion::Skip) } else { - VisitRecursion::Stop + Ok(VisitRecursion::Stop) } } Expr::Literal(_) @@ -89,25 +88,32 @@ pub fn expr_applicable_for_cols(col_names: &[String], expr: &Expr) -> bool { | Expr::ScalarSubquery(_) | Expr::GetIndexedField { .. } | Expr::GroupingSet(_) - | Expr::Case { .. } => VisitRecursion::Continue, + | Expr::Case { .. } => Ok(VisitRecursion::Continue), Expr::ScalarFunction(scalar_function) => { - match scalar_function.fun.volatility() { - Volatility::Immutable => VisitRecursion::Continue, - // TODO: Stable functions could be `applicable`, but that would require access to the context - Volatility::Stable | Volatility::Volatile => { - is_applicable = false; - VisitRecursion::Stop + match &scalar_function.func_def { + ScalarFunctionDefinition::BuiltIn { fun, .. } => { + match fun.volatility() { + Volatility::Immutable => Ok(VisitRecursion::Continue), + // TODO: Stable functions could be `applicable`, but that would require access to the context + Volatility::Stable | Volatility::Volatile => { + is_applicable = false; + Ok(VisitRecursion::Stop) + } + } } - } - } - Expr::ScalarUDF(ScalarUDF { fun, .. }) => { - match fun.signature().volatility { - Volatility::Immutable => VisitRecursion::Continue, - // TODO: Stable functions could be `applicable`, but that would require access to the context - Volatility::Stable | Volatility::Volatile => { - is_applicable = false; - VisitRecursion::Stop + ScalarFunctionDefinition::UDF(fun) => { + match fun.signature().volatility { + Volatility::Immutable => Ok(VisitRecursion::Continue), + // TODO: Stable functions could be `applicable`, but that would require access to the context + Volatility::Stable | Volatility::Volatile => { + is_applicable = false; + Ok(VisitRecursion::Stop) + } + } + } + ScalarFunctionDefinition::Name(_) => { + internal_err!("Function `Expr` with name should be resolved.") } } } @@ -123,9 +129,9 @@ pub fn expr_applicable_for_cols(col_names: &[String], expr: &Expr) -> bool { | Expr::Wildcard { .. } | Expr::Placeholder(_) => { is_applicable = false; - VisitRecursion::Stop + Ok(VisitRecursion::Stop) } - }) + } }) .unwrap(); is_applicable diff --git a/datafusion/core/src/physical_planner.rs b/datafusion/core/src/physical_planner.rs index 82d96c98e688..09f0e11dc2b5 100644 --- a/datafusion/core/src/physical_planner.rs +++ b/datafusion/core/src/physical_planner.rs @@ -83,13 +83,13 @@ use datafusion_common::{ use datafusion_expr::dml::{CopyOptions, CopyTo}; use datafusion_expr::expr::{ self, AggregateFunction, AggregateUDF, Alias, Between, BinaryExpr, Cast, - GetFieldAccess, GetIndexedField, GroupingSet, InList, Like, ScalarUDF, TryCast, - WindowFunction, + GetFieldAccess, GetIndexedField, GroupingSet, InList, Like, TryCast, WindowFunction, }; use datafusion_expr::expr_rewriter::{unalias, unnormalize_cols}; use datafusion_expr::logical_plan::builder::wrap_projection_for_join_if_necessary; use datafusion_expr::{ - DescribeTable, DmlStatement, StringifiedPlan, WindowFrame, WindowFrameBound, WriteOp, + DescribeTable, DmlStatement, ScalarFunctionDefinition, StringifiedPlan, WindowFrame, + WindowFrameBound, WriteOp, }; use datafusion_physical_expr::expressions::Literal; use datafusion_sql::utils::window_expr_common_partition_keys; @@ -217,11 +217,13 @@ fn create_physical_name(e: &Expr, is_first_expr: bool) -> Result { Ok(name) } - Expr::ScalarFunction(func) => { - create_function_physical_name(&func.fun.to_string(), false, &func.args) - } - Expr::ScalarUDF(ScalarUDF { fun, args }) => { - create_function_physical_name(fun.name(), false, args) + Expr::ScalarFunction(expr::ScalarFunction { func_def, args }) => { + // function should be resolved during `AnalyzerRule`s + if let ScalarFunctionDefinition::Name(_) = func_def { + return internal_err!("Function `Expr` with name should be resolved."); + } + + create_function_physical_name(func_def.name(), false, args) } Expr::WindowFunction(WindowFunction { fun, args, .. }) => { create_function_physical_name(&fun.to_string(), false, args) diff --git a/datafusion/expr/src/expr.rs b/datafusion/expr/src/expr.rs index 2b2d30af3bc2..13e488dac042 100644 --- a/datafusion/expr/src/expr.rs +++ b/datafusion/expr/src/expr.rs @@ -148,10 +148,8 @@ pub enum Expr { TryCast(TryCast), /// A sort expression, that can be used to sort values. Sort(Sort), - /// Represents the call of a built-in scalar function with a set of arguments. + /// Represents the call of a scalar function with a set of arguments. ScalarFunction(ScalarFunction), - /// Represents the call of a user-defined scalar function with arguments. - ScalarUDF(ScalarUDF), /// Represents the call of an aggregate built-in function with arguments. AggregateFunction(AggregateFunction), /// Represents the call of a window function with arguments. @@ -338,37 +336,61 @@ impl Between { } } +#[derive(Debug, Clone, PartialEq, Eq, Hash)] +/// Defines which implementation of a function for DataFusion to call. +pub enum ScalarFunctionDefinition { + /// Resolved to a `BuiltinScalarFunction` + /// There is plan to migrate `BuiltinScalarFunction` to UDF-based implementation (issue#8045) + /// This variant is planned to be removed in long term + BuiltIn { + fun: built_in_function::BuiltinScalarFunction, + name: Arc, + }, + /// Resolved to a user defined function + UDF(Arc), + /// A scalar function constructed with name. This variant can not be executed directly + /// and instead must be resolved to one of the other variants prior to physical planning. + Name(Arc), +} + /// ScalarFunction expression invokes a built-in scalar function #[derive(Clone, PartialEq, Eq, Hash, Debug)] pub struct ScalarFunction { /// The function - pub fun: built_in_function::BuiltinScalarFunction, + pub func_def: ScalarFunctionDefinition, /// List of expressions to feed to the functions as arguments pub args: Vec, } +impl ScalarFunctionDefinition { + /// Function's name for display + pub fn name(&self) -> &str { + match self { + ScalarFunctionDefinition::BuiltIn { name, .. } => name.as_ref(), + ScalarFunctionDefinition::UDF(udf) => udf.name(), + ScalarFunctionDefinition::Name(func_name) => func_name.as_ref(), + } + } +} + impl ScalarFunction { /// Create a new ScalarFunction expression pub fn new(fun: built_in_function::BuiltinScalarFunction, args: Vec) -> Self { - Self { fun, args } + Self { + func_def: ScalarFunctionDefinition::BuiltIn { + fun, + name: Arc::from(fun.to_string()), + }, + args, + } } -} -/// ScalarUDF expression invokes a user-defined scalar function [`ScalarUDF`] -/// -/// [`ScalarUDF`]: crate::ScalarUDF -#[derive(Clone, PartialEq, Eq, Hash, Debug)] -pub struct ScalarUDF { - /// The function - pub fun: Arc, - /// List of expressions to feed to the functions as arguments - pub args: Vec, -} - -impl ScalarUDF { - /// Create a new ScalarUDF expression - pub fn new(fun: Arc, args: Vec) -> Self { - Self { fun, args } + /// Create a new ScalarFunction expression with a user-defined function (UDF) + pub fn new_udf(udf: Arc, args: Vec) -> Self { + Self { + func_def: ScalarFunctionDefinition::UDF(udf), + args, + } } } @@ -736,7 +758,6 @@ impl Expr { Expr::Placeholder(_) => "Placeholder", Expr::ScalarFunction(..) => "ScalarFunction", Expr::ScalarSubquery { .. } => "ScalarSubquery", - Expr::ScalarUDF(..) => "ScalarUDF", Expr::ScalarVariable(..) => "ScalarVariable", Expr::Sort { .. } => "Sort", Expr::TryCast { .. } => "TryCast", @@ -1198,11 +1219,8 @@ impl fmt::Display for Expr { write!(f, " NULLS LAST") } } - Expr::ScalarFunction(func) => { - fmt_function(f, &func.fun.to_string(), false, &func.args, true) - } - Expr::ScalarUDF(ScalarUDF { fun, args }) => { - fmt_function(f, fun.name(), false, args, true) + Expr::ScalarFunction(ScalarFunction { func_def, args }) => { + fmt_function(f, func_def.name(), false, args, true) } Expr::WindowFunction(WindowFunction { fun, @@ -1534,11 +1552,8 @@ fn create_name(e: &Expr) -> Result { } } } - Expr::ScalarFunction(func) => { - create_function_name(&func.fun.to_string(), false, &func.args) - } - Expr::ScalarUDF(ScalarUDF { fun, args }) => { - create_function_name(fun.name(), false, args) + Expr::ScalarFunction(ScalarFunction { func_def, args }) => { + create_function_name(func_def.name(), false, args) } Expr::WindowFunction(WindowFunction { fun, diff --git a/datafusion/expr/src/expr_fn.rs b/datafusion/expr/src/expr_fn.rs index 674d2a34df38..4da68575946a 100644 --- a/datafusion/expr/src/expr_fn.rs +++ b/datafusion/expr/src/expr_fn.rs @@ -1014,7 +1014,7 @@ pub fn call_fn(name: impl AsRef, args: Vec) -> Result { #[cfg(test)] mod test { use super::*; - use crate::lit; + use crate::{lit, ScalarFunctionDefinition}; #[test] fn filter_is_null_and_is_not_null() { @@ -1029,8 +1029,10 @@ mod test { macro_rules! test_unary_scalar_expr { ($ENUM:ident, $FUNC:ident) => {{ - if let Expr::ScalarFunction(ScalarFunction { fun, args }) = - $FUNC(col("tableA.a")) + if let Expr::ScalarFunction(ScalarFunction { + func_def: ScalarFunctionDefinition::BuiltIn { fun, .. }, + args, + }) = $FUNC(col("tableA.a")) { let name = built_in_function::BuiltinScalarFunction::$ENUM; assert_eq!(name, fun); @@ -1042,42 +1044,42 @@ mod test { } macro_rules! test_scalar_expr { - ($ENUM:ident, $FUNC:ident, $($arg:ident),*) => { - let expected = [$(stringify!($arg)),*]; - let result = $FUNC( + ($ENUM:ident, $FUNC:ident, $($arg:ident),*) => { + let expected = [$(stringify!($arg)),*]; + let result = $FUNC( + $( + col(stringify!($arg.to_string())) + ),* + ); + if let Expr::ScalarFunction(ScalarFunction { func_def: ScalarFunctionDefinition::BuiltIn{fun, ..}, args }) = result { + let name = built_in_function::BuiltinScalarFunction::$ENUM; + assert_eq!(name, fun); + assert_eq!(expected.len(), args.len()); + } else { + assert!(false, "unexpected: {:?}", result); + } + }; +} + + macro_rules! test_nary_scalar_expr { + ($ENUM:ident, $FUNC:ident, $($arg:ident),*) => { + let expected = [$(stringify!($arg)),*]; + let result = $FUNC( + vec![ $( col(stringify!($arg.to_string())) ),* - ); - if let Expr::ScalarFunction(ScalarFunction { fun, args }) = result { - let name = built_in_function::BuiltinScalarFunction::$ENUM; - assert_eq!(name, fun); - assert_eq!(expected.len(), args.len()); - } else { - assert!(false, "unexpected: {:?}", result); - } - }; - } - - macro_rules! test_nary_scalar_expr { - ($ENUM:ident, $FUNC:ident, $($arg:ident),*) => { - let expected = [$(stringify!($arg)),*]; - let result = $FUNC( - vec![ - $( - col(stringify!($arg.to_string())) - ),* - ] - ); - if let Expr::ScalarFunction(ScalarFunction { fun, args }) = result { - let name = built_in_function::BuiltinScalarFunction::$ENUM; - assert_eq!(name, fun); - assert_eq!(expected.len(), args.len()); - } else { - assert!(false, "unexpected: {:?}", result); - } - }; - } + ] + ); + if let Expr::ScalarFunction(ScalarFunction { func_def: ScalarFunctionDefinition::BuiltIn{fun, ..}, args }) = result { + let name = built_in_function::BuiltinScalarFunction::$ENUM; + assert_eq!(name, fun); + assert_eq!(expected.len(), args.len()); + } else { + assert!(false, "unexpected: {:?}", result); + } + }; +} #[test] fn scalar_function_definitions() { @@ -1207,7 +1209,11 @@ mod test { #[test] fn uuid_function_definitions() { - if let Expr::ScalarFunction(ScalarFunction { fun, args }) = uuid() { + if let Expr::ScalarFunction(ScalarFunction { + func_def: ScalarFunctionDefinition::BuiltIn { fun, .. }, + args, + }) = uuid() + { let name = BuiltinScalarFunction::Uuid; assert_eq!(name, fun); assert_eq!(0, args.len()); @@ -1218,8 +1224,10 @@ mod test { #[test] fn digest_function_definitions() { - if let Expr::ScalarFunction(ScalarFunction { fun, args }) = - digest(col("tableA.a"), lit("md5")) + if let Expr::ScalarFunction(ScalarFunction { + func_def: ScalarFunctionDefinition::BuiltIn { fun, .. }, + args, + }) = digest(col("tableA.a"), lit("md5")) { let name = BuiltinScalarFunction::Digest; assert_eq!(name, fun); @@ -1231,8 +1239,10 @@ mod test { #[test] fn encode_function_definitions() { - if let Expr::ScalarFunction(ScalarFunction { fun, args }) = - encode(col("tableA.a"), lit("base64")) + if let Expr::ScalarFunction(ScalarFunction { + func_def: ScalarFunctionDefinition::BuiltIn { fun, .. }, + args, + }) = encode(col("tableA.a"), lit("base64")) { let name = BuiltinScalarFunction::Encode; assert_eq!(name, fun); @@ -1244,8 +1254,10 @@ mod test { #[test] fn decode_function_definitions() { - if let Expr::ScalarFunction(ScalarFunction { fun, args }) = - decode(col("tableA.a"), lit("hex")) + if let Expr::ScalarFunction(ScalarFunction { + func_def: ScalarFunctionDefinition::BuiltIn { fun, .. }, + args, + }) = decode(col("tableA.a"), lit("hex")) { let name = BuiltinScalarFunction::Decode; assert_eq!(name, fun); diff --git a/datafusion/expr/src/expr_schema.rs b/datafusion/expr/src/expr_schema.rs index 0d06a1295199..d5d9c848b2e9 100644 --- a/datafusion/expr/src/expr_schema.rs +++ b/datafusion/expr/src/expr_schema.rs @@ -18,8 +18,8 @@ use super::{Between, Expr, Like}; use crate::expr::{ AggregateFunction, AggregateUDF, Alias, BinaryExpr, Cast, GetFieldAccess, - GetIndexedField, InList, InSubquery, Placeholder, ScalarFunction, ScalarUDF, Sort, - TryCast, WindowFunction, + GetIndexedField, InList, InSubquery, Placeholder, ScalarFunction, + ScalarFunctionDefinition, Sort, TryCast, WindowFunction, }; use crate::field_util::GetFieldAccessSchema; use crate::type_coercion::binary::get_result_type; @@ -82,32 +82,39 @@ impl ExprSchemable for Expr { Expr::Case(case) => case.when_then_expr[0].1.get_type(schema), Expr::Cast(Cast { data_type, .. }) | Expr::TryCast(TryCast { data_type, .. }) => Ok(data_type.clone()), - Expr::ScalarUDF(ScalarUDF { fun, args }) => { - let data_types = args - .iter() - .map(|e| e.get_type(schema)) - .collect::>>()?; - Ok(fun.return_type(&data_types)?) - } - Expr::ScalarFunction(ScalarFunction { fun, args }) => { - let arg_data_types = args - .iter() - .map(|e| e.get_type(schema)) - .collect::>>()?; - - // verify that input data types is consistent with function's `TypeSignature` - data_types(&arg_data_types, &fun.signature()).map_err(|_| { - plan_datafusion_err!( - "{}", - utils::generate_signature_error_msg( - &format!("{fun}"), - fun.signature(), - &arg_data_types, - ) - ) - })?; - - fun.return_type(&arg_data_types) + Expr::ScalarFunction(ScalarFunction { func_def, args }) => { + match func_def { + ScalarFunctionDefinition::BuiltIn { fun, .. } => { + let arg_data_types = args + .iter() + .map(|e| e.get_type(schema)) + .collect::>>()?; + + // verify that input data types is consistent with function's `TypeSignature` + data_types(&arg_data_types, &fun.signature()).map_err(|_| { + plan_datafusion_err!( + "{}", + utils::generate_signature_error_msg( + &format!("{fun}"), + fun.signature(), + &arg_data_types, + ) + ) + })?; + + fun.return_type(&arg_data_types) + } + ScalarFunctionDefinition::UDF(fun) => { + let data_types = args + .iter() + .map(|e| e.get_type(schema)) + .collect::>>()?; + Ok(fun.return_type(&data_types)?) + } + ScalarFunctionDefinition::Name(_) => { + internal_err!("Function `Expr` with name should be resolved.") + } + } } Expr::WindowFunction(WindowFunction { fun, args, .. }) => { let data_types = args @@ -243,7 +250,6 @@ impl ExprSchemable for Expr { Expr::ScalarVariable(_, _) | Expr::TryCast { .. } | Expr::ScalarFunction(..) - | Expr::ScalarUDF(..) | Expr::WindowFunction { .. } | Expr::AggregateFunction { .. } | Expr::AggregateUDF { .. } diff --git a/datafusion/expr/src/lib.rs b/datafusion/expr/src/lib.rs index b9976f90c547..6172d17365ad 100644 --- a/datafusion/expr/src/lib.rs +++ b/datafusion/expr/src/lib.rs @@ -63,7 +63,7 @@ pub use built_in_function::BuiltinScalarFunction; pub use columnar_value::ColumnarValue; pub use expr::{ Between, BinaryExpr, Case, Cast, Expr, GetFieldAccess, GetIndexedField, GroupingSet, - Like, TryCast, + Like, ScalarFunctionDefinition, TryCast, }; pub use expr_fn::*; pub use expr_schema::ExprSchemable; diff --git a/datafusion/expr/src/tree_node/expr.rs b/datafusion/expr/src/tree_node/expr.rs index 6b86de37ba44..474b5f7689b9 100644 --- a/datafusion/expr/src/tree_node/expr.rs +++ b/datafusion/expr/src/tree_node/expr.rs @@ -20,12 +20,12 @@ use crate::expr::{ AggregateFunction, AggregateUDF, Alias, Between, BinaryExpr, Case, Cast, GetIndexedField, GroupingSet, InList, InSubquery, Like, Placeholder, ScalarFunction, - ScalarUDF, Sort, TryCast, WindowFunction, + ScalarFunctionDefinition, Sort, TryCast, WindowFunction, }; use crate::{Expr, GetFieldAccess}; use datafusion_common::tree_node::{TreeNode, VisitRecursion}; -use datafusion_common::Result; +use datafusion_common::{internal_err, DataFusionError, Result}; impl TreeNode for Expr { fn apply_children(&self, op: &mut F) -> Result @@ -64,7 +64,7 @@ impl TreeNode for Expr { } Expr::GroupingSet(GroupingSet::Rollup(exprs)) | Expr::GroupingSet(GroupingSet::Cube(exprs)) => exprs.clone(), - Expr::ScalarFunction (ScalarFunction{ args, .. } )| Expr::ScalarUDF(ScalarUDF { args, .. }) => { + Expr::ScalarFunction (ScalarFunction{ args, .. } ) => { args.clone() } Expr::GroupingSet(GroupingSet::GroupingSets(lists_of_exprs)) => { @@ -276,12 +276,19 @@ impl TreeNode for Expr { asc, nulls_first, )), - Expr::ScalarFunction(ScalarFunction { args, fun }) => Expr::ScalarFunction( - ScalarFunction::new(fun, transform_vec(args, &mut transform)?), - ), - Expr::ScalarUDF(ScalarUDF { args, fun }) => { - Expr::ScalarUDF(ScalarUDF::new(fun, transform_vec(args, &mut transform)?)) - } + Expr::ScalarFunction(ScalarFunction { func_def, args }) => match func_def { + ScalarFunctionDefinition::BuiltIn { fun, .. } => Expr::ScalarFunction( + ScalarFunction::new(fun, transform_vec(args, &mut transform)?), + ), + ScalarFunctionDefinition::UDF(fun) => Expr::ScalarFunction( + ScalarFunction::new_udf(fun, transform_vec(args, &mut transform)?), + ), + ScalarFunctionDefinition::Name(_) => { + return internal_err!( + "Function `Expr` with name should be resolved." + ); + } + }, Expr::WindowFunction(WindowFunction { args, fun, diff --git a/datafusion/expr/src/udf.rs b/datafusion/expr/src/udf.rs index 22e56caaaf5f..bc910b928a5d 100644 --- a/datafusion/expr/src/udf.rs +++ b/datafusion/expr/src/udf.rs @@ -95,7 +95,10 @@ impl ScalarUDF { /// creates a logical expression with a call of the UDF /// This utility allows using the UDF without requiring access to the registry. pub fn call(&self, args: Vec) -> Expr { - Expr::ScalarUDF(crate::expr::ScalarUDF::new(Arc::new(self.clone()), args)) + Expr::ScalarFunction(crate::expr::ScalarFunction::new_udf( + Arc::new(self.clone()), + args, + )) } /// Returns this function's name diff --git a/datafusion/expr/src/utils.rs b/datafusion/expr/src/utils.rs index ff95ff10e79b..d8668fba8e1e 100644 --- a/datafusion/expr/src/utils.rs +++ b/datafusion/expr/src/utils.rs @@ -283,7 +283,6 @@ pub fn expr_to_columns(expr: &Expr, accum: &mut HashSet) -> Result<()> { | Expr::TryCast { .. } | Expr::Sort { .. } | Expr::ScalarFunction(..) - | Expr::ScalarUDF(..) | Expr::WindowFunction { .. } | Expr::AggregateFunction { .. } | Expr::GroupingSet(_) diff --git a/datafusion/optimizer/src/analyzer/type_coercion.rs b/datafusion/optimizer/src/analyzer/type_coercion.rs index 2c5e8c8b1c45..6628e8961e26 100644 --- a/datafusion/optimizer/src/analyzer/type_coercion.rs +++ b/datafusion/optimizer/src/analyzer/type_coercion.rs @@ -29,7 +29,7 @@ use datafusion_common::{ }; use datafusion_expr::expr::{ self, Between, BinaryExpr, Case, Exists, InList, InSubquery, Like, ScalarFunction, - ScalarUDF, WindowFunction, + WindowFunction, }; use datafusion_expr::expr_rewriter::rewrite_preserving_name; use datafusion_expr::expr_schema::cast_subquery; @@ -45,7 +45,8 @@ use datafusion_expr::type_coercion::{is_datetime, is_utf8_or_large_utf8}; use datafusion_expr::{ is_false, is_not_false, is_not_true, is_not_unknown, is_true, is_unknown, type_coercion, window_function, AggregateFunction, BuiltinScalarFunction, Expr, - LogicalPlan, Operator, Projection, WindowFrame, WindowFrameBound, WindowFrameUnits, + LogicalPlan, Operator, Projection, ScalarFunctionDefinition, WindowFrame, + WindowFrameBound, WindowFrameUnits, }; use datafusion_expr::{ExprSchemable, Signature}; @@ -319,24 +320,32 @@ impl TreeNodeRewriter for TypeCoercionRewriter { let case = coerce_case_expression(case, &self.schema)?; Ok(Expr::Case(case)) } - Expr::ScalarUDF(ScalarUDF { fun, args }) => { - let new_expr = coerce_arguments_for_signature( - args.as_slice(), - &self.schema, - fun.signature(), - )?; - Ok(Expr::ScalarUDF(ScalarUDF::new(fun, new_expr))) - } - Expr::ScalarFunction(ScalarFunction { fun, args }) => { - let new_args = coerce_arguments_for_signature( - args.as_slice(), - &self.schema, - &fun.signature(), - )?; - let new_args = - coerce_arguments_for_fun(new_args.as_slice(), &self.schema, &fun)?; - Ok(Expr::ScalarFunction(ScalarFunction::new(fun, new_args))) - } + Expr::ScalarFunction(ScalarFunction { func_def, args }) => match func_def { + ScalarFunctionDefinition::BuiltIn { fun, .. } => { + let new_args = coerce_arguments_for_signature( + args.as_slice(), + &self.schema, + &fun.signature(), + )?; + let new_args = coerce_arguments_for_fun( + new_args.as_slice(), + &self.schema, + &fun, + )?; + Ok(Expr::ScalarFunction(ScalarFunction::new(fun, new_args))) + } + ScalarFunctionDefinition::UDF(fun) => { + let new_expr = coerce_arguments_for_signature( + args.as_slice(), + &self.schema, + fun.signature(), + )?; + Ok(Expr::ScalarFunction(ScalarFunction::new_udf(fun, new_expr))) + } + ScalarFunctionDefinition::Name(_) => { + internal_err!("Function `Expr` with name should be resolved.") + } + }, Expr::AggregateFunction(expr::AggregateFunction { fun, args, @@ -838,7 +847,7 @@ mod test { Arc::new(move |_| Ok(Arc::new(DataType::Utf8))); let fun: ScalarFunctionImplementation = Arc::new(move |_| Ok(ColumnarValue::Scalar(ScalarValue::new_utf8("a")))); - let udf = Expr::ScalarUDF(expr::ScalarUDF::new( + let udf = Expr::ScalarFunction(expr::ScalarFunction::new_udf( Arc::new(ScalarUDF::new( "TestScalarUDF", &Signature::uniform(1, vec![DataType::Float32], Volatility::Stable), @@ -859,7 +868,7 @@ mod test { let return_type: ReturnTypeFunction = Arc::new(move |_| Ok(Arc::new(DataType::Utf8))); let fun: ScalarFunctionImplementation = Arc::new(move |_| unimplemented!()); - let udf = Expr::ScalarUDF(expr::ScalarUDF::new( + let udf = Expr::ScalarFunction(expr::ScalarFunction::new_udf( Arc::new(ScalarUDF::new( "TestScalarUDF", &Signature::uniform(1, vec![DataType::Int32], Volatility::Stable), @@ -873,9 +882,9 @@ mod test { .err() .unwrap(); assert_eq!( - "type_coercion\ncaused by\nError during planning: Coercion from [Utf8] to the signature Uniform(1, [Int32]) failed.", - err.strip_backtrace() - ); + "type_coercion\ncaused by\nError during planning: Coercion from [Utf8] to the signature Uniform(1, [Int32]) failed.", + err.strip_backtrace() + ); Ok(()) } @@ -1246,10 +1255,10 @@ mod test { None, ), ))); - let expr = Expr::ScalarFunction(ScalarFunction { - fun: BuiltinScalarFunction::MakeArray, - args: vec![val.clone()], - }); + let expr = Expr::ScalarFunction(ScalarFunction::new( + BuiltinScalarFunction::MakeArray, + vec![val.clone()], + )); let schema = Arc::new(DFSchema::new_with_metadata( vec![DFField::new_unqualified( "item", @@ -1278,10 +1287,10 @@ mod test { &schema, )?; - let expected = Expr::ScalarFunction(ScalarFunction { - fun: BuiltinScalarFunction::MakeArray, - args: vec![expected_casted_expr], - }); + let expected = Expr::ScalarFunction(ScalarFunction::new( + BuiltinScalarFunction::MakeArray, + vec![expected_casted_expr], + )); assert_eq!(result, expected); Ok(()) diff --git a/datafusion/optimizer/src/push_down_filter.rs b/datafusion/optimizer/src/push_down_filter.rs index 05f4072e3857..7a2c6a8d8ccd 100644 --- a/datafusion/optimizer/src/push_down_filter.rs +++ b/datafusion/optimizer/src/push_down_filter.rs @@ -28,7 +28,8 @@ use datafusion_expr::{ and, expr_rewriter::replace_col, logical_plan::{CrossJoin, Join, JoinType, LogicalPlan, TableScan, Union}, - or, BinaryExpr, Expr, Filter, Operator, TableProviderFilterPushDown, + or, BinaryExpr, Expr, Filter, Operator, ScalarFunctionDefinition, + TableProviderFilterPushDown, }; use itertools::Itertools; use std::collections::{HashMap, HashSet}; @@ -221,7 +222,10 @@ fn can_evaluate_as_join_condition(predicate: &Expr) -> Result { | Expr::InSubquery(_) | Expr::ScalarSubquery(_) | Expr::OuterReferenceColumn(_, _) - | Expr::ScalarUDF(..) => { + | Expr::ScalarFunction(datafusion_expr::expr::ScalarFunction { + func_def: ScalarFunctionDefinition::UDF(_), + .. + }) => { is_evaluate = false; Ok(VisitRecursion::Stop) } @@ -977,10 +981,26 @@ fn is_volatile_expression(e: &Expr) -> bool { let mut is_volatile = false; e.apply(&mut |expr| { Ok(match expr { - Expr::ScalarFunction(f) if f.fun.volatility() == Volatility::Volatile => { - is_volatile = true; - VisitRecursion::Stop - } + Expr::ScalarFunction(f) => match &f.func_def { + ScalarFunctionDefinition::BuiltIn { fun, .. } + if fun.volatility() == Volatility::Volatile => + { + is_volatile = true; + VisitRecursion::Stop + } + ScalarFunctionDefinition::UDF(fun) + if fun.signature().volatility == Volatility::Volatile => + { + is_volatile = true; + VisitRecursion::Stop + } + ScalarFunctionDefinition::Name(_) => { + return internal_err!( + "Function `Expr` with name should be resolved." + ); + } + _ => VisitRecursion::Continue, + }, _ => VisitRecursion::Continue, }) }) diff --git a/datafusion/optimizer/src/simplify_expressions/expr_simplifier.rs b/datafusion/optimizer/src/simplify_expressions/expr_simplifier.rs index ad64625f7f77..3310bfed75bf 100644 --- a/datafusion/optimizer/src/simplify_expressions/expr_simplifier.rs +++ b/datafusion/optimizer/src/simplify_expressions/expr_simplifier.rs @@ -40,8 +40,8 @@ use datafusion_common::{ exec_err, internal_err, DFSchema, DFSchemaRef, DataFusionError, Result, ScalarValue, }; use datafusion_expr::{ - and, expr, lit, or, BinaryExpr, BuiltinScalarFunction, Case, ColumnarValue, Expr, - Like, Volatility, + and, lit, or, BinaryExpr, BuiltinScalarFunction, Case, ColumnarValue, Expr, Like, + ScalarFunctionDefinition, Volatility, }; use datafusion_expr::{ expr::{InList, InSubquery, ScalarFunction}, @@ -344,12 +344,15 @@ impl<'a> ConstEvaluator<'a> { | Expr::GroupingSet(_) | Expr::Wildcard { .. } | Expr::Placeholder(_) => false, - Expr::ScalarFunction(ScalarFunction { fun, .. }) => { - Self::volatility_ok(fun.volatility()) - } - Expr::ScalarUDF(expr::ScalarUDF { fun, .. }) => { - Self::volatility_ok(fun.signature().volatility) - } + Expr::ScalarFunction(ScalarFunction { func_def, .. }) => match func_def { + ScalarFunctionDefinition::BuiltIn { fun, .. } => { + Self::volatility_ok(fun.volatility()) + } + ScalarFunctionDefinition::UDF(fun) => { + Self::volatility_ok(fun.signature().volatility) + } + ScalarFunctionDefinition::Name(_) => false, + }, Expr::Literal(_) | Expr::BinaryExpr { .. } | Expr::Not(_) @@ -1200,25 +1203,41 @@ impl<'a, S: SimplifyInfo> TreeNodeRewriter for Simplifier<'a, S> { // log Expr::ScalarFunction(ScalarFunction { - fun: BuiltinScalarFunction::Log, + func_def: + ScalarFunctionDefinition::BuiltIn { + fun: BuiltinScalarFunction::Log, + .. + }, args, }) => simpl_log(args, <&S>::clone(&info))?, // power Expr::ScalarFunction(ScalarFunction { - fun: BuiltinScalarFunction::Power, + func_def: + ScalarFunctionDefinition::BuiltIn { + fun: BuiltinScalarFunction::Power, + .. + }, args, }) => simpl_power(args, <&S>::clone(&info))?, // concat Expr::ScalarFunction(ScalarFunction { - fun: BuiltinScalarFunction::Concat, + func_def: + ScalarFunctionDefinition::BuiltIn { + fun: BuiltinScalarFunction::Concat, + .. + }, args, }) => simpl_concat(args)?, // concat_ws Expr::ScalarFunction(ScalarFunction { - fun: BuiltinScalarFunction::ConcatWithSeparator, + func_def: + ScalarFunctionDefinition::BuiltIn { + fun: BuiltinScalarFunction::ConcatWithSeparator, + .. + }, args, }) => match &args[..] { [delimiter, vals @ ..] => simpl_concat_ws(delimiter, vals)?, @@ -1550,7 +1569,7 @@ mod tests { // immutable UDF should get folded // udf_add(1+2, 30+40) --> 73 - let expr = Expr::ScalarUDF(expr::ScalarUDF::new( + let expr = Expr::ScalarFunction(expr::ScalarFunction::new_udf( make_udf_add(Volatility::Immutable), args.clone(), )); @@ -1559,15 +1578,21 @@ mod tests { // stable UDF should be entirely folded // udf_add(1+2, 30+40) --> 73 let fun = make_udf_add(Volatility::Stable); - let expr = Expr::ScalarUDF(expr::ScalarUDF::new(Arc::clone(&fun), args.clone())); + let expr = Expr::ScalarFunction(expr::ScalarFunction::new_udf( + Arc::clone(&fun), + args.clone(), + )); test_evaluate(expr, lit(73)); // volatile UDF should have args folded // udf_add(1+2, 30+40) --> udf_add(3, 70) let fun = make_udf_add(Volatility::Volatile); - let expr = Expr::ScalarUDF(expr::ScalarUDF::new(Arc::clone(&fun), args)); - let expected_expr = - Expr::ScalarUDF(expr::ScalarUDF::new(Arc::clone(&fun), folded_args)); + let expr = + Expr::ScalarFunction(expr::ScalarFunction::new_udf(Arc::clone(&fun), args)); + let expected_expr = Expr::ScalarFunction(expr::ScalarFunction::new_udf( + Arc::clone(&fun), + folded_args, + )); test_evaluate(expr, expected_expr); } diff --git a/datafusion/optimizer/src/simplify_expressions/utils.rs b/datafusion/optimizer/src/simplify_expressions/utils.rs index 17e5d97c3006..e69207b6889a 100644 --- a/datafusion/optimizer/src/simplify_expressions/utils.rs +++ b/datafusion/optimizer/src/simplify_expressions/utils.rs @@ -23,7 +23,7 @@ use datafusion_expr::expr::ScalarFunction; use datafusion_expr::{ expr::{Between, BinaryExpr, InList}, expr_fn::{and, bitwise_and, bitwise_or, concat_ws, or}, - lit, BuiltinScalarFunction, Expr, Like, Operator, + lit, BuiltinScalarFunction, Expr, Like, Operator, ScalarFunctionDefinition, }; pub static POWS_OF_TEN: [i128; 38] = [ @@ -365,7 +365,11 @@ pub fn simpl_log(current_args: Vec, info: &dyn SimplifyInfo) -> Result Ok(args[1].clone()), _ => { @@ -405,7 +409,11 @@ pub fn simpl_power(current_args: Vec, info: &dyn SimplifyInfo) -> Result Ok(args[1].clone()), _ => Ok(Expr::ScalarFunction(ScalarFunction::new( diff --git a/datafusion/physical-expr/src/planner.rs b/datafusion/physical-expr/src/planner.rs index f318cd3b0f4d..5c5cc8e36fa7 100644 --- a/datafusion/physical-expr/src/planner.rs +++ b/datafusion/physical-expr/src/planner.rs @@ -29,10 +29,10 @@ use datafusion_common::{ exec_err, internal_err, not_impl_err, plan_err, DFSchema, DataFusionError, Result, ScalarValue, }; -use datafusion_expr::expr::{Alias, Cast, InList, ScalarFunction, ScalarUDF}; +use datafusion_expr::expr::{Alias, Cast, InList, ScalarFunction}; use datafusion_expr::{ binary_expr, Between, BinaryExpr, Expr, GetFieldAccess, GetIndexedField, Like, - Operator, TryCast, + Operator, ScalarFunctionDefinition, TryCast, }; use std::sync::Arc; @@ -348,36 +348,50 @@ pub fn create_physical_expr( ))) } - Expr::ScalarFunction(ScalarFunction { fun, args }) => { - let physical_args = args - .iter() - .map(|e| { - create_physical_expr(e, input_dfschema, input_schema, execution_props) - }) - .collect::>>()?; - functions::create_physical_expr( - fun, - &physical_args, - input_schema, - execution_props, - ) - } - Expr::ScalarUDF(ScalarUDF { fun, args }) => { - let mut physical_args = vec![]; - for e in args { - physical_args.push(create_physical_expr( - e, - input_dfschema, + Expr::ScalarFunction(ScalarFunction { func_def, args }) => match func_def { + ScalarFunctionDefinition::BuiltIn { fun, .. } => { + let physical_args = args + .iter() + .map(|e| { + create_physical_expr( + e, + input_dfschema, + input_schema, + execution_props, + ) + }) + .collect::>>()?; + functions::create_physical_expr( + fun, + &physical_args, input_schema, execution_props, - )?); + ) + } + ScalarFunctionDefinition::UDF(fun) => { + let mut physical_args = vec![]; + for e in args { + physical_args.push(create_physical_expr( + e, + input_dfschema, + input_schema, + execution_props, + )?); + } + // udfs with zero params expect null array as input + if args.is_empty() { + physical_args.push(Arc::new(Literal::new(ScalarValue::Null))); + } + udf::create_physical_expr( + fun.clone().as_ref(), + &physical_args, + input_schema, + ) } - // udfs with zero params expect null array as input - if args.is_empty() { - physical_args.push(Arc::new(Literal::new(ScalarValue::Null))); + ScalarFunctionDefinition::Name(_) => { + internal_err!("Function `Expr` with name should be resolved.") } - udf::create_physical_expr(fun.clone().as_ref(), &physical_args, input_schema) - } + }, Expr::Between(Between { expr, negated, diff --git a/datafusion/proto/src/logical_plan/from_proto.rs b/datafusion/proto/src/logical_plan/from_proto.rs index 8069e017f797..d4a64287b07e 100644 --- a/datafusion/proto/src/logical_plan/from_proto.rs +++ b/datafusion/proto/src/logical_plan/from_proto.rs @@ -1723,7 +1723,7 @@ pub fn parse_expr( } ExprType::ScalarUdfExpr(protobuf::ScalarUdfExprNode { fun_name, args }) => { let scalar_fn = registry.udf(fun_name.as_str())?; - Ok(Expr::ScalarUDF(expr::ScalarUDF::new( + Ok(Expr::ScalarFunction(expr::ScalarFunction::new_udf( scalar_fn, args.iter() .map(|expr| parse_expr(expr, registry)) diff --git a/datafusion/proto/src/logical_plan/to_proto.rs b/datafusion/proto/src/logical_plan/to_proto.rs index 750eb03e8347..508cde98ae2a 100644 --- a/datafusion/proto/src/logical_plan/to_proto.rs +++ b/datafusion/proto/src/logical_plan/to_proto.rs @@ -45,7 +45,7 @@ use datafusion_common::{ }; use datafusion_expr::expr::{ self, Alias, Between, BinaryExpr, Cast, GetFieldAccess, GetIndexedField, GroupingSet, - InList, Like, Placeholder, ScalarFunction, ScalarUDF, Sort, + InList, Like, Placeholder, ScalarFunction, ScalarFunctionDefinition, Sort, }; use datafusion_expr::{ logical_plan::PlanType, logical_plan::StringifiedPlan, AggregateFunction, @@ -756,29 +756,39 @@ impl TryFrom<&Expr> for protobuf::LogicalExprNode { .to_string(), )) } - Expr::ScalarFunction(ScalarFunction { fun, args }) => { - let fun: protobuf::ScalarFunction = fun.try_into()?; - let args: Vec = args - .iter() - .map(|e| e.try_into()) - .collect::, Error>>()?; - Self { - expr_type: Some(ExprType::ScalarFunction( - protobuf::ScalarFunctionNode { - fun: fun.into(), - args, + Expr::ScalarFunction(ScalarFunction { func_def, args }) => match func_def { + ScalarFunctionDefinition::BuiltIn { fun, .. } => { + let fun: protobuf::ScalarFunction = fun.try_into()?; + let args: Vec = args + .iter() + .map(|e| e.try_into()) + .collect::, Error>>()?; + Self { + expr_type: Some(ExprType::ScalarFunction( + protobuf::ScalarFunctionNode { + fun: fun.into(), + args, + }, + )), + } + } + ScalarFunctionDefinition::UDF(fun) => Self { + expr_type: Some(ExprType::ScalarUdfExpr( + protobuf::ScalarUdfExprNode { + fun_name: fun.name().to_string(), + args: args + .iter() + .map(|expr| expr.try_into()) + .collect::, Error>>()?, }, )), + }, + ScalarFunctionDefinition::Name(_) => { + return Err(Error::NotImplemented( + "Proto serialization error: Trying to serialize a unresolved function" + .to_string(), + )); } - } - Expr::ScalarUDF(ScalarUDF { fun, args }) => Self { - expr_type: Some(ExprType::ScalarUdfExpr(protobuf::ScalarUdfExprNode { - fun_name: fun.name().to_string(), - args: args - .iter() - .map(|expr| expr.try_into()) - .collect::, Error>>()?, - })), }, Expr::AggregateUDF(expr::AggregateUDF { fun, diff --git a/datafusion/proto/tests/cases/roundtrip_logical_plan.rs b/datafusion/proto/tests/cases/roundtrip_logical_plan.rs index acc7f07bfa9f..3ab001298ed2 100644 --- a/datafusion/proto/tests/cases/roundtrip_logical_plan.rs +++ b/datafusion/proto/tests/cases/roundtrip_logical_plan.rs @@ -39,7 +39,7 @@ use datafusion_common::{internal_err, not_impl_err, plan_err}; use datafusion_common::{DFField, DFSchema, DFSchemaRef, DataFusionError, ScalarValue}; use datafusion_expr::expr::{ self, Between, BinaryExpr, Case, Cast, GroupingSet, InList, Like, ScalarFunction, - ScalarUDF, Sort, + Sort, }; use datafusion_expr::logical_plan::{Extension, UserDefinedLogicalNodeCore}; use datafusion_expr::{ @@ -1402,7 +1402,10 @@ fn roundtrip_scalar_udf() { scalar_fn, ); - let test_expr = Expr::ScalarUDF(ScalarUDF::new(Arc::new(udf.clone()), vec![lit("")])); + let test_expr = Expr::ScalarFunction(ScalarFunction::new_udf( + Arc::new(udf.clone()), + vec![lit("")], + )); let ctx = SessionContext::new(); ctx.register_udf(udf); diff --git a/datafusion/sql/src/expr/function.rs b/datafusion/sql/src/expr/function.rs index c77ef64718bb..24ba4d1b506a 100644 --- a/datafusion/sql/src/expr/function.rs +++ b/datafusion/sql/src/expr/function.rs @@ -19,7 +19,7 @@ use crate::planner::{ContextProvider, PlannerContext, SqlToRel}; use datafusion_common::{ not_impl_err, plan_datafusion_err, plan_err, DFSchema, DataFusionError, Result, }; -use datafusion_expr::expr::{ScalarFunction, ScalarUDF}; +use datafusion_expr::expr::ScalarFunction; use datafusion_expr::function::suggest_valid_function; use datafusion_expr::window_frame::regularize; use datafusion_expr::{ @@ -66,7 +66,7 @@ impl<'a, S: ContextProvider> SqlToRel<'a, S> { // user-defined function (UDF) should have precedence in case it has the same name as a scalar built-in function if let Some(fm) = self.context_provider.get_function_meta(&name) { let args = self.function_args_to_expr(args, schema, planner_context)?; - return Ok(Expr::ScalarUDF(ScalarUDF::new(fm, args))); + return Ok(Expr::ScalarFunction(ScalarFunction::new_udf(fm, args))); } // next, scalar built-in diff --git a/datafusion/sql/src/expr/value.rs b/datafusion/sql/src/expr/value.rs index 0f086bca6819..f33e9e8ddf78 100644 --- a/datafusion/sql/src/expr/value.rs +++ b/datafusion/sql/src/expr/value.rs @@ -24,8 +24,8 @@ use datafusion_common::{ }; use datafusion_expr::expr::ScalarFunction; use datafusion_expr::expr::{BinaryExpr, Placeholder}; -use datafusion_expr::BuiltinScalarFunction; use datafusion_expr::{lit, Expr, Operator}; +use datafusion_expr::{BuiltinScalarFunction, ScalarFunctionDefinition}; use log::debug; use sqlparser::ast::{BinaryOperator, Expr as SQLExpr, Interval, Value}; use sqlparser::parser::ParserError::ParserError; @@ -143,8 +143,11 @@ impl<'a, S: ContextProvider> SqlToRel<'a, S> { Expr::Literal(_) => { values.push(value); } - Expr::ScalarFunction(ref scalar_function) => { - if scalar_function.fun == BuiltinScalarFunction::MakeArray { + Expr::ScalarFunction(ScalarFunction { + func_def: ScalarFunctionDefinition::BuiltIn { fun, .. }, + .. + }) => { + if fun == BuiltinScalarFunction::MakeArray { values.push(value); } else { return not_impl_err!( diff --git a/datafusion/substrait/src/logical_plan/consumer.rs b/datafusion/substrait/src/logical_plan/consumer.rs index f4c36557dac8..5cb72adaca4d 100644 --- a/datafusion/substrait/src/logical_plan/consumer.rs +++ b/datafusion/substrait/src/logical_plan/consumer.rs @@ -843,10 +843,9 @@ pub async fn from_substrait_rex( }; args.push(arg_expr?.as_ref().clone()); } - Ok(Arc::new(Expr::ScalarFunction(expr::ScalarFunction { - fun, - args, - }))) + Ok(Arc::new(Expr::ScalarFunction(expr::ScalarFunction::new( + fun, args, + )))) } ScalarFunctionType::Op(op) => { if f.arguments.len() != 2 { diff --git a/datafusion/substrait/src/logical_plan/producer.rs b/datafusion/substrait/src/logical_plan/producer.rs index 4b6aded78b49..95604e6d2db9 100644 --- a/datafusion/substrait/src/logical_plan/producer.rs +++ b/datafusion/substrait/src/logical_plan/producer.rs @@ -34,7 +34,7 @@ use datafusion::common::{exec_err, internal_err, not_impl_err}; use datafusion::logical_expr::aggregate_function; use datafusion::logical_expr::expr::{ Alias, BinaryExpr, Case, Cast, GroupingSet, InList, - ScalarFunction as DFScalarFunction, Sort, WindowFunction, + ScalarFunction as DFScalarFunction, ScalarFunctionDefinition, Sort, WindowFunction, }; use datafusion::logical_expr::{expr, Between, JoinConstraint, LogicalPlan, Operator}; use datafusion::prelude::Expr; @@ -822,7 +822,7 @@ pub fn to_substrait_rex( Ok(substrait_or_list) } } - Expr::ScalarFunction(DFScalarFunction { fun, args }) => { + Expr::ScalarFunction(DFScalarFunction { func_def, args }) => { let mut arguments: Vec = vec![]; for arg in args { arguments.push(FunctionArgument { @@ -834,7 +834,14 @@ pub fn to_substrait_rex( )?)), }); } - let function_anchor = _register_function(fun.to_string(), extension_info); + + // function should be resolved during `AnalyzerRule` + if let ScalarFunctionDefinition::Name(_) = func_def { + return internal_err!("Function `Expr` with name should be resolved."); + } + + let function_anchor = + _register_function(func_def.name().to_string(), extension_info); Ok(Expression { rex_type: Some(RexType::ScalarFunction(ScalarFunction { function_reference: function_anchor, From f29bcf36184691dc0417b6be2eb3e33fa8a6f1cc Mon Sep 17 00:00:00 2001 From: Huaijin Date: Sun, 26 Nov 2023 19:53:46 +0800 Subject: [PATCH 311/572] Support no distinct aggregate sum/min/max in `single_distinct_to_group_by` rule (#8266) * init impl * add some tests * add filter tests * minor * add more tests * update test --- .../src/single_distinct_to_groupby.rs | 280 ++++++++++++++++-- .../sqllogictest/test_files/groupby.slt | 82 +++++ 2 files changed, 330 insertions(+), 32 deletions(-) diff --git a/datafusion/optimizer/src/single_distinct_to_groupby.rs b/datafusion/optimizer/src/single_distinct_to_groupby.rs index ac18e596b7bd..fa142438c4a3 100644 --- a/datafusion/optimizer/src/single_distinct_to_groupby.rs +++ b/datafusion/optimizer/src/single_distinct_to_groupby.rs @@ -24,6 +24,7 @@ use crate::{OptimizerConfig, OptimizerRule}; use datafusion_common::{DFSchema, Result}; use datafusion_expr::{ + aggregate_function::AggregateFunction::{Max, Min, Sum}, col, expr::AggregateFunction, logical_plan::{Aggregate, LogicalPlan, Projection}, @@ -35,17 +36,19 @@ use hashbrown::HashSet; /// single distinct to group by optimizer rule /// ```text -/// SELECT F1(DISTINCT s),F2(DISTINCT s) -/// ... -/// GROUP BY k +/// Before: +/// SELECT a, COUNT(DINSTINCT b), SUM(c) +/// FROM t +/// GROUP BY a /// -/// Into -/// -/// SELECT F1(alias1),F2(alias1) +/// After: +/// SELECT a, COUNT(alias1), SUM(alias2) /// FROM ( -/// SELECT s as alias1, k ... GROUP BY s, k +/// SELECT a, b as alias1, SUM(c) as alias2 +/// FROM t +/// GROUP BY a, b /// ) -/// GROUP BY k +/// GROUP BY a /// ``` #[derive(Default)] pub struct SingleDistinctToGroupBy {} @@ -64,22 +67,30 @@ fn is_single_distinct_agg(plan: &LogicalPlan) -> Result { match plan { LogicalPlan::Aggregate(Aggregate { aggr_expr, .. }) => { let mut fields_set = HashSet::new(); - let mut distinct_count = 0; + let mut aggregate_count = 0; for expr in aggr_expr { if let Expr::AggregateFunction(AggregateFunction { - distinct, args, .. + fun, + distinct, + args, + filter, + order_by, }) = expr { - if *distinct { - distinct_count += 1; + if filter.is_some() || order_by.is_some() { + return Ok(false); } - for e in args { - fields_set.insert(e.canonical_name()); + aggregate_count += 1; + if *distinct { + for e in args { + fields_set.insert(e.canonical_name()); + } + } else if !matches!(fun, Sum | Min | Max) { + return Ok(false); } } } - let res = distinct_count == aggr_expr.len() && fields_set.len() == 1; - Ok(res) + Ok(aggregate_count == aggr_expr.len() && fields_set.len() == 1) } _ => Ok(false), } @@ -152,30 +163,57 @@ impl OptimizerRule for SingleDistinctToGroupBy { .collect::>(); // replace the distinct arg with alias + let mut index = 1; let mut group_fields_set = HashSet::new(); - let new_aggr_exprs = aggr_expr + let mut inner_aggr_exprs = vec![]; + let outer_aggr_exprs = aggr_expr .iter() .map(|aggr_expr| match aggr_expr { Expr::AggregateFunction(AggregateFunction { fun, args, - filter, - order_by, + distinct, .. }) => { // is_single_distinct_agg ensure args.len=1 - if group_fields_set.insert(args[0].display_name()?) { + if *distinct + && group_fields_set.insert(args[0].display_name()?) + { inner_group_exprs.push( args[0].clone().alias(SINGLE_DISTINCT_ALIAS), ); } - Ok(Expr::AggregateFunction(AggregateFunction::new( - fun.clone(), - vec![col(SINGLE_DISTINCT_ALIAS)], - false, // intentional to remove distinct here - filter.clone(), - order_by.clone(), - ))) + + // if the aggregate function is not distinct, we need to rewrite it like two phase aggregation + if !(*distinct) { + index += 1; + let alias_str = format!("alias{}", index); + inner_aggr_exprs.push( + Expr::AggregateFunction(AggregateFunction::new( + fun.clone(), + args.clone(), + false, + None, + None, + )) + .alias(&alias_str), + ); + Ok(Expr::AggregateFunction(AggregateFunction::new( + fun.clone(), + vec![col(&alias_str)], + false, + None, + None, + ))) + } else { + Ok(Expr::AggregateFunction(AggregateFunction::new( + fun.clone(), + vec![col(SINGLE_DISTINCT_ALIAS)], + false, // intentional to remove distinct here + None, + None, + ))) + } } _ => Ok(aggr_expr.clone()), }) @@ -184,6 +222,7 @@ impl OptimizerRule for SingleDistinctToGroupBy { // construct the inner AggrPlan let inner_fields = inner_group_exprs .iter() + .chain(inner_aggr_exprs.iter()) .map(|expr| expr.to_field(input.schema())) .collect::>>()?; let inner_schema = DFSchema::new_with_metadata( @@ -193,12 +232,12 @@ impl OptimizerRule for SingleDistinctToGroupBy { let inner_agg = LogicalPlan::Aggregate(Aggregate::try_new( input.clone(), inner_group_exprs, - Vec::new(), + inner_aggr_exprs, )?); let outer_fields = outer_group_exprs .iter() - .chain(new_aggr_exprs.iter()) + .chain(outer_aggr_exprs.iter()) .map(|expr| expr.to_field(&inner_schema)) .collect::>>()?; let outer_aggr_schema = Arc::new(DFSchema::new_with_metadata( @@ -220,7 +259,7 @@ impl OptimizerRule for SingleDistinctToGroupBy { group_expr } }) - .chain(new_aggr_exprs.iter().enumerate().map(|(idx, expr)| { + .chain(outer_aggr_exprs.iter().enumerate().map(|(idx, expr)| { let idx = idx + group_size; let name = fields[idx].qualified_name(); columnize_expr(expr.clone().alias(name), &outer_aggr_schema) @@ -230,7 +269,7 @@ impl OptimizerRule for SingleDistinctToGroupBy { let outer_aggr = LogicalPlan::Aggregate(Aggregate::try_new( Arc::new(inner_agg), outer_group_exprs, - new_aggr_exprs, + outer_aggr_exprs, )?); Ok(Some(LogicalPlan::Projection(Projection::try_new( @@ -262,7 +301,7 @@ mod tests { use datafusion_expr::expr::GroupingSet; use datafusion_expr::{ col, count, count_distinct, lit, logical_plan::builder::LogicalPlanBuilder, max, - AggregateFunction, + min, sum, AggregateFunction, }; fn assert_optimized_plan_equal(plan: &LogicalPlan, expected: &str) -> Result<()> { @@ -478,4 +517,181 @@ mod tests { assert_optimized_plan_equal(&plan, expected) } + + #[test] + fn two_distinct_and_one_common() -> Result<()> { + let table_scan = test_table_scan()?; + + let plan = LogicalPlanBuilder::from(table_scan) + .aggregate( + vec![col("a")], + vec![ + sum(col("c")), + count_distinct(col("b")), + Expr::AggregateFunction(expr::AggregateFunction::new( + AggregateFunction::Max, + vec![col("b")], + true, + None, + None, + )), + ], + )? + .build()?; + // Should work + let expected = "Projection: test.a, SUM(alias2) AS SUM(test.c), COUNT(alias1) AS COUNT(DISTINCT test.b), MAX(alias1) AS MAX(DISTINCT test.b) [a:UInt32, SUM(test.c):UInt64;N, COUNT(DISTINCT test.b):Int64;N, MAX(DISTINCT test.b):UInt32;N]\ + \n Aggregate: groupBy=[[test.a]], aggr=[[SUM(alias2), COUNT(alias1), MAX(alias1)]] [a:UInt32, SUM(alias2):UInt64;N, COUNT(alias1):Int64;N, MAX(alias1):UInt32;N]\ + \n Aggregate: groupBy=[[test.a, test.b AS alias1]], aggr=[[SUM(test.c) AS alias2]] [a:UInt32, alias1:UInt32, alias2:UInt64;N]\ + \n TableScan: test [a:UInt32, b:UInt32, c:UInt32]"; + + assert_optimized_plan_equal(&plan, expected) + } + + #[test] + fn one_distinctand_and_two_common() -> Result<()> { + let table_scan = test_table_scan()?; + + let plan = LogicalPlanBuilder::from(table_scan) + .aggregate( + vec![col("a")], + vec![sum(col("c")), max(col("c")), count_distinct(col("b"))], + )? + .build()?; + // Should work + let expected = "Projection: test.a, SUM(alias2) AS SUM(test.c), MAX(alias3) AS MAX(test.c), COUNT(alias1) AS COUNT(DISTINCT test.b) [a:UInt32, SUM(test.c):UInt64;N, MAX(test.c):UInt32;N, COUNT(DISTINCT test.b):Int64;N]\ + \n Aggregate: groupBy=[[test.a]], aggr=[[SUM(alias2), MAX(alias3), COUNT(alias1)]] [a:UInt32, SUM(alias2):UInt64;N, MAX(alias3):UInt32;N, COUNT(alias1):Int64;N]\ + \n Aggregate: groupBy=[[test.a, test.b AS alias1]], aggr=[[SUM(test.c) AS alias2, MAX(test.c) AS alias3]] [a:UInt32, alias1:UInt32, alias2:UInt64;N, alias3:UInt32;N]\ + \n TableScan: test [a:UInt32, b:UInt32, c:UInt32]"; + + assert_optimized_plan_equal(&plan, expected) + } + + #[test] + fn one_distinct_and_one_common() -> Result<()> { + let table_scan = test_table_scan()?; + + let plan = LogicalPlanBuilder::from(table_scan) + .aggregate( + vec![col("c")], + vec![min(col("a")), count_distinct(col("b"))], + )? + .build()?; + // Should work + let expected = "Projection: test.c, MIN(alias2) AS MIN(test.a), COUNT(alias1) AS COUNT(DISTINCT test.b) [c:UInt32, MIN(test.a):UInt32;N, COUNT(DISTINCT test.b):Int64;N]\ + \n Aggregate: groupBy=[[test.c]], aggr=[[MIN(alias2), COUNT(alias1)]] [c:UInt32, MIN(alias2):UInt32;N, COUNT(alias1):Int64;N]\ + \n Aggregate: groupBy=[[test.c, test.b AS alias1]], aggr=[[MIN(test.a) AS alias2]] [c:UInt32, alias1:UInt32, alias2:UInt32;N]\ + \n TableScan: test [a:UInt32, b:UInt32, c:UInt32]"; + + assert_optimized_plan_equal(&plan, expected) + } + + #[test] + fn common_with_filter() -> Result<()> { + let table_scan = test_table_scan()?; + + // SUM(a) FILTER (WHERE a > 5) + let expr = Expr::AggregateFunction(expr::AggregateFunction::new( + AggregateFunction::Sum, + vec![col("a")], + false, + Some(Box::new(col("a").gt(lit(5)))), + None, + )); + let plan = LogicalPlanBuilder::from(table_scan) + .aggregate(vec![col("c")], vec![expr, count_distinct(col("b"))])? + .build()?; + // Do nothing + let expected = "Aggregate: groupBy=[[test.c]], aggr=[[SUM(test.a) FILTER (WHERE test.a > Int32(5)), COUNT(DISTINCT test.b)]] [c:UInt32, SUM(test.a) FILTER (WHERE test.a > Int32(5)):UInt64;N, COUNT(DISTINCT test.b):Int64;N]\ + \n TableScan: test [a:UInt32, b:UInt32, c:UInt32]"; + + assert_optimized_plan_equal(&plan, expected) + } + + #[test] + fn distinct_with_filter() -> Result<()> { + let table_scan = test_table_scan()?; + + // COUNT(DISTINCT a) FILTER (WHERE a > 5) + let expr = Expr::AggregateFunction(expr::AggregateFunction::new( + AggregateFunction::Count, + vec![col("a")], + true, + Some(Box::new(col("a").gt(lit(5)))), + None, + )); + let plan = LogicalPlanBuilder::from(table_scan) + .aggregate(vec![col("c")], vec![sum(col("a")), expr])? + .build()?; + // Do nothing + let expected = "Aggregate: groupBy=[[test.c]], aggr=[[SUM(test.a), COUNT(DISTINCT test.a) FILTER (WHERE test.a > Int32(5))]] [c:UInt32, SUM(test.a):UInt64;N, COUNT(DISTINCT test.a) FILTER (WHERE test.a > Int32(5)):Int64;N]\ + \n TableScan: test [a:UInt32, b:UInt32, c:UInt32]"; + + assert_optimized_plan_equal(&plan, expected) + } + + #[test] + fn common_with_order_by() -> Result<()> { + let table_scan = test_table_scan()?; + + // SUM(a ORDER BY a) + let expr = Expr::AggregateFunction(expr::AggregateFunction::new( + AggregateFunction::Sum, + vec![col("a")], + false, + None, + Some(vec![col("a")]), + )); + let plan = LogicalPlanBuilder::from(table_scan) + .aggregate(vec![col("c")], vec![expr, count_distinct(col("b"))])? + .build()?; + // Do nothing + let expected = "Aggregate: groupBy=[[test.c]], aggr=[[SUM(test.a) ORDER BY [test.a], COUNT(DISTINCT test.b)]] [c:UInt32, SUM(test.a) ORDER BY [test.a]:UInt64;N, COUNT(DISTINCT test.b):Int64;N]\ + \n TableScan: test [a:UInt32, b:UInt32, c:UInt32]"; + + assert_optimized_plan_equal(&plan, expected) + } + + #[test] + fn distinct_with_order_by() -> Result<()> { + let table_scan = test_table_scan()?; + + // COUNT(DISTINCT a ORDER BY a) + let expr = Expr::AggregateFunction(expr::AggregateFunction::new( + AggregateFunction::Count, + vec![col("a")], + true, + None, + Some(vec![col("a")]), + )); + let plan = LogicalPlanBuilder::from(table_scan) + .aggregate(vec![col("c")], vec![sum(col("a")), expr])? + .build()?; + // Do nothing + let expected = "Aggregate: groupBy=[[test.c]], aggr=[[SUM(test.a), COUNT(DISTINCT test.a) ORDER BY [test.a]]] [c:UInt32, SUM(test.a):UInt64;N, COUNT(DISTINCT test.a) ORDER BY [test.a]:Int64;N]\ + \n TableScan: test [a:UInt32, b:UInt32, c:UInt32]"; + + assert_optimized_plan_equal(&plan, expected) + } + + #[test] + fn aggregate_with_filter_and_order_by() -> Result<()> { + let table_scan = test_table_scan()?; + + // COUNT(DISTINCT a ORDER BY a) FILTER (WHERE a > 5) + let expr = Expr::AggregateFunction(expr::AggregateFunction::new( + AggregateFunction::Count, + vec![col("a")], + true, + Some(Box::new(col("a").gt(lit(5)))), + Some(vec![col("a")]), + )); + let plan = LogicalPlanBuilder::from(table_scan) + .aggregate(vec![col("c")], vec![sum(col("a")), expr])? + .build()?; + // Do nothing + let expected = "Aggregate: groupBy=[[test.c]], aggr=[[SUM(test.a), COUNT(DISTINCT test.a) FILTER (WHERE test.a > Int32(5)) ORDER BY [test.a]]] [c:UInt32, SUM(test.a):UInt64;N, COUNT(DISTINCT test.a) FILTER (WHERE test.a > Int32(5)) ORDER BY [test.a]:Int64;N]\ + \n TableScan: test [a:UInt32, b:UInt32, c:UInt32]"; + + assert_optimized_plan_equal(&plan, expected) + } } diff --git a/datafusion/sqllogictest/test_files/groupby.slt b/datafusion/sqllogictest/test_files/groupby.slt index 756d3f737439..d6f9adb02335 100644 --- a/datafusion/sqllogictest/test_files/groupby.slt +++ b/datafusion/sqllogictest/test_files/groupby.slt @@ -3965,3 +3965,85 @@ select date_bin(interval '1 year', time) as bla, count(distinct state) as count statement ok drop table t1 + +statement ok +CREATE EXTERNAL TABLE aggregate_test_100 ( + c1 VARCHAR NOT NULL, + c2 TINYINT NOT NULL, + c3 SMALLINT NOT NULL, + c4 SMALLINT, + c5 INT, + c6 BIGINT NOT NULL, + c7 SMALLINT NOT NULL, + c8 INT NOT NULL, + c9 INT UNSIGNED NOT NULL, + c10 BIGINT UNSIGNED NOT NULL, + c11 FLOAT NOT NULL, + c12 DOUBLE NOT NULL, + c13 VARCHAR NOT NULL +) +STORED AS CSV +WITH HEADER ROW +LOCATION '../../testing/data/csv/aggregate_test_100.csv' + +query TIIII +SELECT c1, count(distinct c2), min(distinct c2), min(c3), max(c4) FROM aggregate_test_100 GROUP BY c1 ORDER BY c1; +---- +a 5 1 -101 32064 +b 5 1 -117 25286 +c 5 1 -117 29106 +d 5 1 -99 31106 +e 5 1 -95 32514 + +query TT +EXPLAIN SELECT c1, count(distinct c2), min(distinct c2), sum(c3), max(c4) FROM aggregate_test_100 GROUP BY c1 ORDER BY c1; +---- +logical_plan +Sort: aggregate_test_100.c1 ASC NULLS LAST +--Projection: aggregate_test_100.c1, COUNT(alias1) AS COUNT(DISTINCT aggregate_test_100.c2), MIN(alias1) AS MIN(DISTINCT aggregate_test_100.c2), SUM(alias2) AS SUM(aggregate_test_100.c3), MAX(alias3) AS MAX(aggregate_test_100.c4) +----Aggregate: groupBy=[[aggregate_test_100.c1]], aggr=[[COUNT(alias1), MIN(alias1), SUM(alias2), MAX(alias3)]] +------Aggregate: groupBy=[[aggregate_test_100.c1, aggregate_test_100.c2 AS alias1]], aggr=[[SUM(CAST(aggregate_test_100.c3 AS Int64)) AS alias2, MAX(aggregate_test_100.c4) AS alias3]] +--------TableScan: aggregate_test_100 projection=[c1, c2, c3, c4] +physical_plan +SortPreservingMergeExec: [c1@0 ASC NULLS LAST] +--SortExec: expr=[c1@0 ASC NULLS LAST] +----ProjectionExec: expr=[c1@0 as c1, COUNT(alias1)@1 as COUNT(DISTINCT aggregate_test_100.c2), MIN(alias1)@2 as MIN(DISTINCT aggregate_test_100.c2), SUM(alias2)@3 as SUM(aggregate_test_100.c3), MAX(alias3)@4 as MAX(aggregate_test_100.c4)] +------AggregateExec: mode=FinalPartitioned, gby=[c1@0 as c1], aggr=[COUNT(alias1), MIN(alias1), SUM(alias2), MAX(alias3)] +--------CoalesceBatchesExec: target_batch_size=2 +----------RepartitionExec: partitioning=Hash([c1@0], 8), input_partitions=8 +------------AggregateExec: mode=Partial, gby=[c1@0 as c1], aggr=[COUNT(alias1), MIN(alias1), SUM(alias2), MAX(alias3)] +--------------AggregateExec: mode=FinalPartitioned, gby=[c1@0 as c1, alias1@1 as alias1], aggr=[alias2, alias3] +----------------CoalesceBatchesExec: target_batch_size=2 +------------------RepartitionExec: partitioning=Hash([c1@0, alias1@1], 8), input_partitions=8 +--------------------AggregateExec: mode=Partial, gby=[c1@0 as c1, c2@1 as alias1], aggr=[alias2, alias3] +----------------------RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1 +------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1, c2, c3, c4], has_header=true + +# Use PostgreSQL dialect +statement ok +set datafusion.sql_parser.dialect = 'Postgres'; + +query II +SELECT c2, count(distinct c3) FILTER (WHERE c1 != 'a') FROM aggregate_test_100 GROUP BY c2 ORDER BY c2; +---- +1 17 +2 17 +3 13 +4 19 +5 11 + +query III +SELECT c2, count(distinct c3) FILTER (WHERE c1 != 'a'), count(c5) FILTER (WHERE c1 != 'b') FROM aggregate_test_100 GROUP BY c2 ORDER BY c2; +---- +1 17 19 +2 17 18 +3 13 17 +4 19 18 +5 11 9 + +# Restore the default dialect +statement ok +set datafusion.sql_parser.dialect = 'Generic'; + +statement ok +drop table aggregate_test_100; From 234217e439ccf598d704fd7645560f04f25e8a6f Mon Sep 17 00:00:00 2001 From: Syleechan <38198463+Syleechan@users.noreply.github.com> Date: Sun, 26 Nov 2023 20:00:20 +0800 Subject: [PATCH 312/572] feat:implement sql style 'substr_index' string function (#8272) * feat:implement sql style 'substr_index' string function * code format * code format * code format * fix index bound issue * code format * code format * add args len check * add sql tests * code format * doc format --- datafusion/expr/src/built_in_function.rs | 15 ++++ datafusion/expr/src/expr_fn.rs | 2 + datafusion/physical-expr/src/functions.rs | 23 ++++++ .../physical-expr/src/unicode_expressions.rs | 65 ++++++++++++++++ datafusion/proto/proto/datafusion.proto | 1 + datafusion/proto/src/generated/pbjson.rs | 3 + datafusion/proto/src/generated/prost.rs | 3 + .../proto/src/logical_plan/from_proto.rs | 12 ++- datafusion/proto/src/logical_plan/to_proto.rs | 1 + .../sqllogictest/test_files/functions.slt | 75 +++++++++++++++++++ .../source/user-guide/sql/scalar_functions.md | 18 +++++ 11 files changed, 215 insertions(+), 3 deletions(-) diff --git a/datafusion/expr/src/built_in_function.rs b/datafusion/expr/src/built_in_function.rs index cbf5d400bab5..d92067501657 100644 --- a/datafusion/expr/src/built_in_function.rs +++ b/datafusion/expr/src/built_in_function.rs @@ -302,6 +302,8 @@ pub enum BuiltinScalarFunction { OverLay, /// levenshtein Levenshtein, + /// substr_index + SubstrIndex, } /// Maps the sql function name to `BuiltinScalarFunction` @@ -470,6 +472,7 @@ impl BuiltinScalarFunction { BuiltinScalarFunction::ArrowTypeof => Volatility::Immutable, BuiltinScalarFunction::OverLay => Volatility::Immutable, BuiltinScalarFunction::Levenshtein => Volatility::Immutable, + BuiltinScalarFunction::SubstrIndex => Volatility::Immutable, // Stable builtin functions BuiltinScalarFunction::Now => Volatility::Stable, @@ -773,6 +776,9 @@ impl BuiltinScalarFunction { return plan_err!("The to_hex function can only accept integers."); } }), + BuiltinScalarFunction::SubstrIndex => { + utf8_to_str_type(&input_expr_types[0], "substr_index") + } BuiltinScalarFunction::ToTimestamp => Ok(match &input_expr_types[0] { Int64 => Timestamp(Second, None), _ => Timestamp(Nanosecond, None), @@ -1235,6 +1241,14 @@ impl BuiltinScalarFunction { self.volatility(), ), + BuiltinScalarFunction::SubstrIndex => Signature::one_of( + vec![ + Exact(vec![Utf8, Utf8, Int64]), + Exact(vec![LargeUtf8, LargeUtf8, Int64]), + ], + self.volatility(), + ), + BuiltinScalarFunction::Replace | BuiltinScalarFunction::Translate => { Signature::one_of(vec![Exact(vec![Utf8, Utf8, Utf8])], self.volatility()) } @@ -1486,6 +1500,7 @@ fn aliases(func: &BuiltinScalarFunction) -> &'static [&'static str] { BuiltinScalarFunction::Upper => &["upper"], BuiltinScalarFunction::Uuid => &["uuid"], BuiltinScalarFunction::Levenshtein => &["levenshtein"], + BuiltinScalarFunction::SubstrIndex => &["substr_index", "substring_index"], // regex functions BuiltinScalarFunction::RegexpMatch => &["regexp_match"], diff --git a/datafusion/expr/src/expr_fn.rs b/datafusion/expr/src/expr_fn.rs index 4da68575946a..d2c5e5cddbf3 100644 --- a/datafusion/expr/src/expr_fn.rs +++ b/datafusion/expr/src/expr_fn.rs @@ -916,6 +916,7 @@ scalar_expr!( scalar_expr!(ArrowTypeof, arrow_typeof, val, "data type"); scalar_expr!(Levenshtein, levenshtein, string1 string2, "Returns the Levenshtein distance between the two given strings"); +scalar_expr!(SubstrIndex, substr_index, string delimiter count, "Returns the substring from str before count occurrences of the delimiter"); scalar_expr!( Struct, @@ -1205,6 +1206,7 @@ mod test { test_nary_scalar_expr!(OverLay, overlay, string, characters, position, len); test_nary_scalar_expr!(OverLay, overlay, string, characters, position); test_scalar_expr!(Levenshtein, levenshtein, string1, string2); + test_scalar_expr!(SubstrIndex, substr_index, string, delimiter, count); } #[test] diff --git a/datafusion/physical-expr/src/functions.rs b/datafusion/physical-expr/src/functions.rs index 5a1a68dd2127..40b21347edf5 100644 --- a/datafusion/physical-expr/src/functions.rs +++ b/datafusion/physical-expr/src/functions.rs @@ -862,6 +862,29 @@ pub fn create_physical_fun( ))), }) } + BuiltinScalarFunction::SubstrIndex => { + Arc::new(|args| match args[0].data_type() { + DataType::Utf8 => { + let func = invoke_if_unicode_expressions_feature_flag!( + substr_index, + i32, + "substr_index" + ); + make_scalar_function(func)(args) + } + DataType::LargeUtf8 => { + let func = invoke_if_unicode_expressions_feature_flag!( + substr_index, + i64, + "substr_index" + ); + make_scalar_function(func)(args) + } + other => Err(DataFusionError::Internal(format!( + "Unsupported data type {other:?} for function substr_index", + ))), + }) + } }) } diff --git a/datafusion/physical-expr/src/unicode_expressions.rs b/datafusion/physical-expr/src/unicode_expressions.rs index e28700a25ce4..f27b3c157741 100644 --- a/datafusion/physical-expr/src/unicode_expressions.rs +++ b/datafusion/physical-expr/src/unicode_expressions.rs @@ -455,3 +455,68 @@ pub fn translate(args: &[ArrayRef]) -> Result { Ok(Arc::new(result) as ArrayRef) } + +/// Returns the substring from str before count occurrences of the delimiter delim. If count is positive, everything to the left of the final delimiter (counting from the left) is returned. If count is negative, everything to the right of the final delimiter (counting from the right) is returned. +/// SUBSTRING_INDEX('www.apache.org', '.', 1) = www +/// SUBSTRING_INDEX('www.apache.org', '.', 2) = www.apache +/// SUBSTRING_INDEX('www.apache.org', '.', -2) = apache.org +/// SUBSTRING_INDEX('www.apache.org', '.', -1) = org +pub fn substr_index(args: &[ArrayRef]) -> Result { + if args.len() != 3 { + return internal_err!( + "substr_index was called with {} arguments. It requires 3.", + args.len() + ); + } + + let string_array = as_generic_string_array::(&args[0])?; + let delimiter_array = as_generic_string_array::(&args[1])?; + let count_array = as_int64_array(&args[2])?; + + let result = string_array + .iter() + .zip(delimiter_array.iter()) + .zip(count_array.iter()) + .map(|((string, delimiter), n)| match (string, delimiter, n) { + (Some(string), Some(delimiter), Some(n)) => { + let mut res = String::new(); + match n { + 0 => { + "".to_string(); + } + _other => { + if n > 0 { + let idx = string + .split(delimiter) + .take(n as usize) + .fold(0, |len, x| len + x.len() + delimiter.len()) + - delimiter.len(); + res.push_str(if idx >= string.len() { + string + } else { + &string[..idx] + }); + } else { + let idx = (string.split(delimiter).take((-n) as usize).fold( + string.len() as isize, + |len, x| { + len - x.len() as isize - delimiter.len() as isize + }, + ) + delimiter.len() as isize) + as usize; + res.push_str(if idx >= string.len() { + string + } else { + &string[idx..] + }); + } + } + } + Some(res) + } + _ => None, + }) + .collect::>(); + + Ok(Arc::new(result) as ArrayRef) +} diff --git a/datafusion/proto/proto/datafusion.proto b/datafusion/proto/proto/datafusion.proto index d43d19f85842..5c33b10f1395 100644 --- a/datafusion/proto/proto/datafusion.proto +++ b/datafusion/proto/proto/datafusion.proto @@ -641,6 +641,7 @@ enum ScalarFunction { ArrayExcept = 123; ArrayPopFront = 124; Levenshtein = 125; + SubstrIndex = 126; } message ScalarFunctionNode { diff --git a/datafusion/proto/src/generated/pbjson.rs b/datafusion/proto/src/generated/pbjson.rs index 133bbbee8920..598719dc8ac6 100644 --- a/datafusion/proto/src/generated/pbjson.rs +++ b/datafusion/proto/src/generated/pbjson.rs @@ -20863,6 +20863,7 @@ impl serde::Serialize for ScalarFunction { Self::ArrayExcept => "ArrayExcept", Self::ArrayPopFront => "ArrayPopFront", Self::Levenshtein => "Levenshtein", + Self::SubstrIndex => "SubstrIndex", }; serializer.serialize_str(variant) } @@ -21000,6 +21001,7 @@ impl<'de> serde::Deserialize<'de> for ScalarFunction { "ArrayExcept", "ArrayPopFront", "Levenshtein", + "SubstrIndex", ]; struct GeneratedVisitor; @@ -21166,6 +21168,7 @@ impl<'de> serde::Deserialize<'de> for ScalarFunction { "ArrayExcept" => Ok(ScalarFunction::ArrayExcept), "ArrayPopFront" => Ok(ScalarFunction::ArrayPopFront), "Levenshtein" => Ok(ScalarFunction::Levenshtein), + "SubstrIndex" => Ok(ScalarFunction::SubstrIndex), _ => Err(serde::de::Error::unknown_variant(value, FIELDS)), } } diff --git a/datafusion/proto/src/generated/prost.rs b/datafusion/proto/src/generated/prost.rs index 503c4b6c73f1..e79a17fc5c9c 100644 --- a/datafusion/proto/src/generated/prost.rs +++ b/datafusion/proto/src/generated/prost.rs @@ -2594,6 +2594,7 @@ pub enum ScalarFunction { ArrayExcept = 123, ArrayPopFront = 124, Levenshtein = 125, + SubstrIndex = 126, } impl ScalarFunction { /// String value of the enum field names used in the ProtoBuf definition. @@ -2728,6 +2729,7 @@ impl ScalarFunction { ScalarFunction::ArrayExcept => "ArrayExcept", ScalarFunction::ArrayPopFront => "ArrayPopFront", ScalarFunction::Levenshtein => "Levenshtein", + ScalarFunction::SubstrIndex => "SubstrIndex", } } /// Creates an enum from field names used in the ProtoBuf definition. @@ -2859,6 +2861,7 @@ impl ScalarFunction { "ArrayExcept" => Some(Self::ArrayExcept), "ArrayPopFront" => Some(Self::ArrayPopFront), "Levenshtein" => Some(Self::Levenshtein), + "SubstrIndex" => Some(Self::SubstrIndex), _ => None, } } diff --git a/datafusion/proto/src/logical_plan/from_proto.rs b/datafusion/proto/src/logical_plan/from_proto.rs index d4a64287b07e..b2455d5a0d13 100644 --- a/datafusion/proto/src/logical_plan/from_proto.rs +++ b/datafusion/proto/src/logical_plan/from_proto.rs @@ -55,9 +55,9 @@ use datafusion_expr::{ lower, lpad, ltrim, md5, nanvl, now, nullif, octet_length, overlay, pi, power, radians, random, regexp_match, regexp_replace, repeat, replace, reverse, right, round, rpad, rtrim, sha224, sha256, sha384, sha512, signum, sin, sinh, split_part, - sqrt, starts_with, string_to_array, strpos, struct_fun, substr, substring, tan, tanh, - to_hex, to_timestamp_micros, to_timestamp_millis, to_timestamp_nanos, - to_timestamp_seconds, translate, trim, trunc, upper, uuid, + sqrt, starts_with, string_to_array, strpos, struct_fun, substr, substr_index, + substring, tan, tanh, to_hex, to_timestamp_micros, to_timestamp_millis, + to_timestamp_nanos, to_timestamp_seconds, translate, trim, trunc, upper, uuid, window_frame::regularize, AggregateFunction, Between, BinaryExpr, BuiltInWindowFunction, BuiltinScalarFunction, Case, Cast, Expr, GetFieldAccess, GetIndexedField, GroupingSet, @@ -551,6 +551,7 @@ impl From<&protobuf::ScalarFunction> for BuiltinScalarFunction { ScalarFunction::ArrowTypeof => Self::ArrowTypeof, ScalarFunction::OverLay => Self::OverLay, ScalarFunction::Levenshtein => Self::Levenshtein, + ScalarFunction::SubstrIndex => Self::SubstrIndex, } } } @@ -1716,6 +1717,11 @@ pub fn parse_expr( .map(|expr| parse_expr(expr, registry)) .collect::, _>>()?, )), + ScalarFunction::SubstrIndex => Ok(substr_index( + parse_expr(&args[0], registry)?, + parse_expr(&args[1], registry)?, + parse_expr(&args[2], registry)?, + )), ScalarFunction::StructFun => { Ok(struct_fun(parse_expr(&args[0], registry)?)) } diff --git a/datafusion/proto/src/logical_plan/to_proto.rs b/datafusion/proto/src/logical_plan/to_proto.rs index 508cde98ae2a..9be4a532bb5b 100644 --- a/datafusion/proto/src/logical_plan/to_proto.rs +++ b/datafusion/proto/src/logical_plan/to_proto.rs @@ -1583,6 +1583,7 @@ impl TryFrom<&BuiltinScalarFunction> for protobuf::ScalarFunction { BuiltinScalarFunction::ArrowTypeof => Self::ArrowTypeof, BuiltinScalarFunction::OverLay => Self::OverLay, BuiltinScalarFunction::Levenshtein => Self::Levenshtein, + BuiltinScalarFunction::SubstrIndex => Self::SubstrIndex, }; Ok(scalar_function) diff --git a/datafusion/sqllogictest/test_files/functions.slt b/datafusion/sqllogictest/test_files/functions.slt index 9c8bb2c5f844..91072a49cd46 100644 --- a/datafusion/sqllogictest/test_files/functions.slt +++ b/datafusion/sqllogictest/test_files/functions.slt @@ -877,3 +877,78 @@ query ? SELECT levenshtein(NULL, NULL) ---- NULL + +query T +SELECT substr_index('www.apache.org', '.', 1) +---- +www + +query T +SELECT substr_index('www.apache.org', '.', 2) +---- +www.apache + +query T +SELECT substr_index('www.apache.org', '.', -1) +---- +org + +query T +SELECT substr_index('www.apache.org', '.', -2) +---- +apache.org + +query T +SELECT substr_index('www.apache.org', 'ac', 1) +---- +www.ap + +query T +SELECT substr_index('www.apache.org', 'ac', -1) +---- +he.org + +query T +SELECT substr_index('www.apache.org', 'ac', 2) +---- +www.apache.org + +query T +SELECT substr_index('www.apache.org', 'ac', -2) +---- +www.apache.org + +query ? +SELECT substr_index(NULL, 'ac', 1) +---- +NULL + +query T +SELECT substr_index('www.apache.org', NULL, 1) +---- +NULL + +query T +SELECT substr_index('www.apache.org', 'ac', NULL) +---- +NULL + +query T +SELECT substr_index('', 'ac', 1) +---- +(empty) + +query T +SELECT substr_index('www.apache.org', '', 1) +---- +(empty) + +query T +SELECT substr_index('www.apache.org', 'ac', 0) +---- +(empty) + +query ? +SELECT substr_index(NULL, NULL, NULL) +---- +NULL diff --git a/docs/source/user-guide/sql/scalar_functions.md b/docs/source/user-guide/sql/scalar_functions.md index eda46ef8a73b..e7ebbc9f1fe7 100644 --- a/docs/source/user-guide/sql/scalar_functions.md +++ b/docs/source/user-guide/sql/scalar_functions.md @@ -637,6 +637,7 @@ nullif(expression1, expression2) - [uuid](#uuid) - [overlay](#overlay) - [levenshtein](#levenshtein) +- [substr_index](#substr_index) ### `ascii` @@ -1152,6 +1153,23 @@ levenshtein(str1, str2) - **str1**: String expression to compute Levenshtein distance with str2. - **str2**: String expression to compute Levenshtein distance with str1. +### `substr_index` + +Returns the substring from str before count occurrences of the delimiter delim. +If count is positive, everything to the left of the final delimiter (counting from the left) is returned. +If count is negative, everything to the right of the final delimiter (counting from the right) is returned. +For example, `substr_index('www.apache.org', '.', 1) = www`, `substr_index('www.apache.org', '.', -1) = org` + +``` +substr_index(str, delim, count) +``` + +#### Arguments + +- **str**: String expression to operate on. +- **delim**: the string to find in str to split str. +- **count**: The number of times to search for the delimiter. Can be both a positive or negative number. + ## Binary String Functions - [decode](#decode) From 2071259e23f75d94678cc0a54bad154d3748b8cb Mon Sep 17 00:00:00 2001 From: comphead Date: Sun, 26 Nov 2023 14:28:01 -0800 Subject: [PATCH 313/572] Fixing issues with for timestamp literals (#8193) Fixing issue for timestamp literals --- datafusion/common/src/scalar.rs | 12 +++++ datafusion/core/tests/sql/timestamp.rs | 2 +- datafusion/expr/src/built_in_function.rs | 7 +-- .../physical-expr/src/datetime_expressions.rs | 8 +-- .../physical-expr/src/expressions/negative.rs | 7 ++- datafusion/sql/src/expr/mod.rs | 30 ++++++++--- datafusion/sql/tests/sql_integration.rs | 4 +- .../sqllogictest/test_files/timestamps.slt | 52 +++++++++++++++++-- datafusion/sqllogictest/test_files/window.slt | 16 +++--- 9 files changed, 103 insertions(+), 35 deletions(-) diff --git a/datafusion/common/src/scalar.rs b/datafusion/common/src/scalar.rs index ffa8ab50f862..3431d71468ea 100644 --- a/datafusion/common/src/scalar.rs +++ b/datafusion/common/src/scalar.rs @@ -983,6 +983,18 @@ impl ScalarValue { ScalarValue::Decimal256(Some(v), precision, scale) => Ok( ScalarValue::Decimal256(Some(v.neg_wrapping()), *precision, *scale), ), + ScalarValue::TimestampSecond(Some(v), tz) => { + Ok(ScalarValue::TimestampSecond(Some(-v), tz.clone())) + } + ScalarValue::TimestampNanosecond(Some(v), tz) => { + Ok(ScalarValue::TimestampNanosecond(Some(-v), tz.clone())) + } + ScalarValue::TimestampMicrosecond(Some(v), tz) => { + Ok(ScalarValue::TimestampMicrosecond(Some(-v), tz.clone())) + } + ScalarValue::TimestampMillisecond(Some(v), tz) => { + Ok(ScalarValue::TimestampMillisecond(Some(-v), tz.clone())) + } value => _internal_err!( "Can not run arithmetic negative on scalar value {value:?}" ), diff --git a/datafusion/core/tests/sql/timestamp.rs b/datafusion/core/tests/sql/timestamp.rs index a18e6831b615..ada66503a181 100644 --- a/datafusion/core/tests/sql/timestamp.rs +++ b/datafusion/core/tests/sql/timestamp.rs @@ -742,7 +742,7 @@ async fn test_arrow_typeof() -> Result<()> { "+-----------------------------------------------------------------------+", "| arrow_typeof(date_trunc(Utf8(\"microsecond\"),to_timestamp(Int64(61)))) |", "+-----------------------------------------------------------------------+", - "| Timestamp(Second, None) |", + "| Timestamp(Nanosecond, None) |", "+-----------------------------------------------------------------------+", ]; assert_batches_eq!(expected, &actual); diff --git a/datafusion/expr/src/built_in_function.rs b/datafusion/expr/src/built_in_function.rs index d92067501657..c511c752b4d7 100644 --- a/datafusion/expr/src/built_in_function.rs +++ b/datafusion/expr/src/built_in_function.rs @@ -779,13 +779,10 @@ impl BuiltinScalarFunction { BuiltinScalarFunction::SubstrIndex => { utf8_to_str_type(&input_expr_types[0], "substr_index") } - BuiltinScalarFunction::ToTimestamp => Ok(match &input_expr_types[0] { - Int64 => Timestamp(Second, None), - _ => Timestamp(Nanosecond, None), - }), + BuiltinScalarFunction::ToTimestamp + | BuiltinScalarFunction::ToTimestampNanos => Ok(Timestamp(Nanosecond, None)), BuiltinScalarFunction::ToTimestampMillis => Ok(Timestamp(Millisecond, None)), BuiltinScalarFunction::ToTimestampMicros => Ok(Timestamp(Microsecond, None)), - BuiltinScalarFunction::ToTimestampNanos => Ok(Timestamp(Nanosecond, None)), BuiltinScalarFunction::ToTimestampSeconds => Ok(Timestamp(Second, None)), BuiltinScalarFunction::FromUnixtime => Ok(Timestamp(Second, None)), BuiltinScalarFunction::Now => { diff --git a/datafusion/physical-expr/src/datetime_expressions.rs b/datafusion/physical-expr/src/datetime_expressions.rs index 5b597de78ac9..0d42708c97ec 100644 --- a/datafusion/physical-expr/src/datetime_expressions.rs +++ b/datafusion/physical-expr/src/datetime_expressions.rs @@ -966,9 +966,11 @@ pub fn to_timestamp_invoke(args: &[ColumnarValue]) -> Result { } match args[0].data_type() { - DataType::Int64 => { - cast_column(&args[0], &DataType::Timestamp(TimeUnit::Second, None), None) - } + DataType::Int64 => cast_column( + &cast_column(&args[0], &DataType::Timestamp(TimeUnit::Second, None), None)?, + &DataType::Timestamp(TimeUnit::Nanosecond, None), + None, + ), DataType::Timestamp(_, None) => cast_column( &args[0], &DataType::Timestamp(TimeUnit::Nanosecond, None), diff --git a/datafusion/physical-expr/src/expressions/negative.rs b/datafusion/physical-expr/src/expressions/negative.rs index a59fd1ae3f20..b64b4a0c86de 100644 --- a/datafusion/physical-expr/src/expressions/negative.rs +++ b/datafusion/physical-expr/src/expressions/negative.rs @@ -33,7 +33,7 @@ use arrow::{ use datafusion_common::{internal_err, DataFusionError, Result}; use datafusion_expr::interval_arithmetic::Interval; use datafusion_expr::{ - type_coercion::{is_interval, is_null, is_signed_numeric}, + type_coercion::{is_interval, is_null, is_signed_numeric, is_timestamp}, ColumnarValue, }; @@ -160,7 +160,10 @@ pub fn negative( let data_type = arg.data_type(input_schema)?; if is_null(&data_type) { Ok(arg) - } else if !is_signed_numeric(&data_type) && !is_interval(&data_type) { + } else if !is_signed_numeric(&data_type) + && !is_interval(&data_type) + && !is_timestamp(&data_type) + { internal_err!( "Can't create negative physical expr for (- '{arg:?}'), the type of child expr is {data_type}, not signed numeric" ) diff --git a/datafusion/sql/src/expr/mod.rs b/datafusion/sql/src/expr/mod.rs index 7fa16ced39da..25fe6b6633c2 100644 --- a/datafusion/sql/src/expr/mod.rs +++ b/datafusion/sql/src/expr/mod.rs @@ -29,6 +29,7 @@ mod value; use crate::planner::{ContextProvider, PlannerContext, SqlToRel}; use arrow_schema::DataType; +use arrow_schema::TimeUnit; use datafusion_common::{ internal_err, not_impl_err, plan_err, Column, DFSchema, DataFusionError, Result, ScalarValue, @@ -224,14 +225,27 @@ impl<'a, S: ContextProvider> SqlToRel<'a, S> { SQLExpr::Cast { expr, data_type, .. - } => Ok(Expr::Cast(Cast::new( - Box::new(self.sql_expr_to_logical_expr( - *expr, - schema, - planner_context, - )?), - self.convert_data_type(&data_type)?, - ))), + } => { + let dt = self.convert_data_type(&data_type)?; + let expr = + self.sql_expr_to_logical_expr(*expr, schema, planner_context)?; + + // numeric constants are treated as seconds (rather as nanoseconds) + // to align with postgres / duckdb semantics + let expr = match &dt { + DataType::Timestamp(TimeUnit::Nanosecond, tz) + if expr.get_type(schema)? == DataType::Int64 => + { + Expr::Cast(Cast::new( + Box::new(expr), + DataType::Timestamp(TimeUnit::Second, tz.clone()), + )) + } + _ => expr, + }; + + Ok(Expr::Cast(Cast::new(Box::new(expr), dt))) + } SQLExpr::TryCast { expr, data_type, .. diff --git a/datafusion/sql/tests/sql_integration.rs b/datafusion/sql/tests/sql_integration.rs index a56e9a50f054..d5b06bcf815f 100644 --- a/datafusion/sql/tests/sql_integration.rs +++ b/datafusion/sql/tests/sql_integration.rs @@ -606,11 +606,9 @@ fn select_compound_filter() { #[test] fn test_timestamp_filter() { let sql = "SELECT state FROM person WHERE birth_date < CAST (158412331400600000 as timestamp)"; - let expected = "Projection: person.state\ - \n Filter: person.birth_date < CAST(Int64(158412331400600000) AS Timestamp(Nanosecond, None))\ + \n Filter: person.birth_date < CAST(CAST(Int64(158412331400600000) AS Timestamp(Second, None)) AS Timestamp(Nanosecond, None))\ \n TableScan: person"; - quick_test(sql, expected); } diff --git a/datafusion/sqllogictest/test_files/timestamps.slt b/datafusion/sqllogictest/test_files/timestamps.slt index e186aa12f7a9..3830d8f86812 100644 --- a/datafusion/sqllogictest/test_files/timestamps.slt +++ b/datafusion/sqllogictest/test_files/timestamps.slt @@ -1788,8 +1788,50 @@ SELECT TIMESTAMPTZ '2020-01-01 00:00:00Z' = TIMESTAMP '2020-01-01' ---- true -# verify to_timestamp edge cases to be in sync with postgresql -query PPPPP -SELECT to_timestamp(null), to_timestamp(-62125747200), to_timestamp(0), to_timestamp(1926632005177), to_timestamp(1926632005) ----- -NULL 0001-04-25T00:00:00 1970-01-01T00:00:00 +63022-07-16T12:59:37 2031-01-19T23:33:25 +# verify timestamp cast with integer input +query PPPPPP +SELECT to_timestamp(null), to_timestamp(0), to_timestamp(1926632005), to_timestamp(1), to_timestamp(-1), to_timestamp(0-1) +---- +NULL 1970-01-01T00:00:00 2031-01-19T23:33:25 1970-01-01T00:00:01 1969-12-31T23:59:59 1969-12-31T23:59:59 + +# verify timestamp syntax stlyes are consistent +query BBBBBBBBBBBBB +SELECT to_timestamp(null) is null as c1, + null::timestamp is null as c2, + cast(null as timestamp) is null as c3, + to_timestamp(0) = 0::timestamp as c4, + to_timestamp(1926632005) = 1926632005::timestamp as c5, + to_timestamp(1) = 1::timestamp as c6, + to_timestamp(-1) = -1::timestamp as c7, + to_timestamp(0-1) = (0-1)::timestamp as c8, + to_timestamp(0) = cast(0 as timestamp) as c9, + to_timestamp(1926632005) = cast(1926632005 as timestamp) as c10, + to_timestamp(1) = cast(1 as timestamp) as c11, + to_timestamp(-1) = cast(-1 as timestamp) as c12, + to_timestamp(0-1) = cast(0-1 as timestamp) as c13 +---- +true true true true true true true true true true true true true + +# verify timestamp output types +query TTT +SELECT arrow_typeof(to_timestamp(1)), arrow_typeof(to_timestamp(null)), arrow_typeof(to_timestamp('2023-01-10 12:34:56.000')) +---- +Timestamp(Nanosecond, None) Timestamp(Nanosecond, None) Timestamp(Nanosecond, None) + +# verify timestamp output types using timestamp literal syntax +query BBBBBB +SELECT arrow_typeof(to_timestamp(1)) = arrow_typeof(1::timestamp) as c1, + arrow_typeof(to_timestamp(null)) = arrow_typeof(null::timestamp) as c2, + arrow_typeof(to_timestamp('2023-01-10 12:34:56.000')) = arrow_typeof('2023-01-10 12:34:56.000'::timestamp) as c3, + arrow_typeof(to_timestamp(1)) = arrow_typeof(cast(1 as timestamp)) as c4, + arrow_typeof(to_timestamp(null)) = arrow_typeof(cast(null as timestamp)) as c5, + arrow_typeof(to_timestamp('2023-01-10 12:34:56.000')) = arrow_typeof(cast('2023-01-10 12:34:56.000' as timestamp)) as c6 +---- +true true true true true true + +# known issues. currently overflows (expects default precision to be microsecond instead of nanoseconds. Work pending) +#verify extreme values +#query PPPPPPPP +#SELECT to_timestamp(-62125747200), to_timestamp(1926632005177), -62125747200::timestamp, 1926632005177::timestamp, cast(-62125747200 as timestamp), cast(1926632005177 as timestamp) +#---- +#0001-04-25T00:00:00 +63022-07-16T12:59:37 0001-04-25T00:00:00 +63022-07-16T12:59:37 0001-04-25T00:00:00 +63022-07-16T12:59:37 diff --git a/datafusion/sqllogictest/test_files/window.slt b/datafusion/sqllogictest/test_files/window.slt index 1ef0ba0d10e3..319c08407661 100644 --- a/datafusion/sqllogictest/test_files/window.slt +++ b/datafusion/sqllogictest/test_files/window.slt @@ -895,14 +895,14 @@ SELECT statement ok create table temp as values -(1664264591000000000), -(1664264592000000000), -(1664264592000000000), -(1664264593000000000), -(1664264594000000000), -(1664364594000000000), -(1664464594000000000), -(1664564594000000000); +(1664264591), +(1664264592), +(1664264592), +(1664264593), +(1664264594), +(1664364594), +(1664464594), +(1664564594); statement ok create table t as select cast(column1 as timestamp) as ts from temp; From d81c961ccb301bf12fba002474c3d2092d66d032 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Berkay=20=C5=9Eahin?= <124376117+berkaysynnada@users.noreply.github.com> Date: Mon, 27 Nov 2023 09:21:26 +0300 Subject: [PATCH 314/572] Projection Pushdown over StreamingTableExec (#8299) * Projection above streaming table can be removed * Review --------- Co-authored-by: Mehmet Ozan Kabak --- .../physical_optimizer/projection_pushdown.rs | 186 ++++++++++++++++-- datafusion/physical-plan/src/streaming.rs | 29 ++- 2 files changed, 199 insertions(+), 16 deletions(-) diff --git a/datafusion/core/src/physical_optimizer/projection_pushdown.rs b/datafusion/core/src/physical_optimizer/projection_pushdown.rs index 74d0de507e4c..c0e512ffe57b 100644 --- a/datafusion/core/src/physical_optimizer/projection_pushdown.rs +++ b/datafusion/core/src/physical_optimizer/projection_pushdown.rs @@ -20,6 +20,8 @@ //! projections one by one if the operator below is amenable to this. If a //! projection reaches a source, it can even dissappear from the plan entirely. +use std::sync::Arc; + use super::output_requirements::OutputRequirementExec; use super::PhysicalOptimizerRule; use crate::datasource::physical_plan::CsvExec; @@ -39,7 +41,6 @@ use crate::physical_plan::sorts::sort_preserving_merge::SortPreservingMergeExec; use crate::physical_plan::{Distribution, ExecutionPlan}; use arrow_schema::SchemaRef; - use datafusion_common::config::ConfigOptions; use datafusion_common::tree_node::{Transformed, TreeNode}; use datafusion_common::JoinSide; @@ -47,10 +48,10 @@ use datafusion_physical_expr::expressions::Column; use datafusion_physical_expr::{ Partitioning, PhysicalExpr, PhysicalSortExpr, PhysicalSortRequirement, }; +use datafusion_physical_plan::streaming::StreamingTableExec; use datafusion_physical_plan::union::UnionExec; use itertools::Itertools; -use std::sync::Arc; /// This rule inspects [`ProjectionExec`]'s in the given physical plan and tries to /// remove or swap with its child. @@ -135,6 +136,8 @@ pub fn remove_unnecessary_projections( try_swapping_with_sort_merge_join(projection, sm_join)? } else if let Some(sym_join) = input.downcast_ref::() { try_swapping_with_sym_hash_join(projection, sym_join)? + } else if let Some(ste) = input.downcast_ref::() { + try_swapping_with_streaming_table(projection, ste)? } else { // If the input plan of the projection is not one of the above, we // conservatively assume that pushing the projection down may hurt. @@ -149,8 +152,8 @@ pub fn remove_unnecessary_projections( Ok(maybe_modified.map_or(Transformed::No(plan), Transformed::Yes)) } -/// Tries to swap `projection` with its input (`csv`). If possible, performs -/// the swap and returns [`CsvExec`] as the top plan. Otherwise, returns `None`. +/// Tries to embed `projection` to its input (`csv`). If possible, returns +/// [`CsvExec`] as the top plan. Otherwise, returns `None`. fn try_swapping_with_csv( projection: &ProjectionExec, csv: &CsvExec, @@ -174,8 +177,8 @@ fn try_swapping_with_csv( }) } -/// Tries to swap `projection` with its input (`memory`). If possible, performs -/// the swap and returns [`MemoryExec`] as the top plan. Otherwise, returns `None`. +/// Tries to embed `projection` to its input (`memory`). If possible, returns +/// [`MemoryExec`] as the top plan. Otherwise, returns `None`. fn try_swapping_with_memory( projection: &ProjectionExec, memory: &MemoryExec, @@ -197,10 +200,52 @@ fn try_swapping_with_memory( .transpose() } +/// Tries to embed `projection` to its input (`streaming table`). +/// If possible, returns [`StreamingTableExec`] as the top plan. Otherwise, +/// returns `None`. +fn try_swapping_with_streaming_table( + projection: &ProjectionExec, + streaming_table: &StreamingTableExec, +) -> Result>> { + if !all_alias_free_columns(projection.expr()) { + return Ok(None); + } + + let streaming_table_projections = streaming_table + .projection() + .as_ref() + .map(|i| i.as_ref().to_vec()); + let new_projections = + new_projections_for_columns(projection, &streaming_table_projections); + + let mut lex_orderings = vec![]; + for lex_ordering in streaming_table.projected_output_ordering().into_iter() { + let mut orderings = vec![]; + for order in lex_ordering { + let Some(new_ordering) = update_expr(&order.expr, projection.expr(), false)? + else { + return Ok(None); + }; + orderings.push(PhysicalSortExpr { + expr: new_ordering, + options: order.options, + }); + } + lex_orderings.push(orderings); + } + + StreamingTableExec::try_new( + streaming_table.partition_schema().clone(), + streaming_table.partitions().clone(), + Some(&new_projections), + lex_orderings, + streaming_table.is_infinite(), + ) + .map(|e| Some(Arc::new(e) as _)) +} + /// Unifies `projection` with its input (which is also a [`ProjectionExec`]). -/// Two consecutive projections can always merge into a single projection unless -/// the [`update_expr`] function does not support one of the expression -/// types involved in the projection. +/// Two consecutive projections can always merge into a single projection. fn try_unifying_projections( projection: &ProjectionExec, child: &ProjectionExec, @@ -779,10 +824,6 @@ fn new_projections_for_columns( /// given the expressions `c@0`, `a@1` and `b@2`, and the [`ProjectionExec`] with /// an output schema of `a, c_new`, then `c@0` becomes `c_new@1`, `a@1` becomes /// `a@0`, but `b@2` results in `None` since the projection does not include `b`. -/// -/// If the expression contains a `PhysicalExpr` variant that this function does -/// not support, it will return `None`. An error can only be introduced if -/// `CaseExpr::try_new` returns an error. fn update_expr( expr: &Arc, projected_exprs: &[(Arc, String)], @@ -1102,10 +1143,11 @@ mod tests { use crate::physical_plan::sorts::sort_preserving_merge::SortPreservingMergeExec; use crate::physical_plan::ExecutionPlan; - use arrow_schema::{DataType, Field, Schema, SortOptions}; + use arrow_schema::{DataType, Field, Schema, SchemaRef, SortOptions}; use datafusion_common::config::ConfigOptions; use datafusion_common::{JoinSide, JoinType, Result, ScalarValue, Statistics}; use datafusion_execution::object_store::ObjectStoreUrl; + use datafusion_execution::{SendableRecordBatchStream, TaskContext}; use datafusion_expr::{ColumnarValue, Operator}; use datafusion_physical_expr::expressions::{ BinaryExpr, CaseExpr, CastExpr, Column, Literal, NegativeExpr, @@ -1115,8 +1157,11 @@ mod tests { PhysicalSortRequirement, ScalarFunctionExpr, }; use datafusion_physical_plan::joins::SymmetricHashJoinExec; + use datafusion_physical_plan::streaming::{PartitionStream, StreamingTableExec}; use datafusion_physical_plan::union::UnionExec; + use itertools::Itertools; + #[test] fn test_update_matching_exprs() -> Result<()> { let exprs: Vec> = vec![ @@ -1575,6 +1620,119 @@ mod tests { Ok(()) } + #[test] + fn test_streaming_table_after_projection() -> Result<()> { + struct DummyStreamPartition { + schema: SchemaRef, + } + impl PartitionStream for DummyStreamPartition { + fn schema(&self) -> &SchemaRef { + &self.schema + } + fn execute(&self, _ctx: Arc) -> SendableRecordBatchStream { + unreachable!() + } + } + + let streaming_table = StreamingTableExec::try_new( + Arc::new(Schema::new(vec![ + Field::new("a", DataType::Int32, true), + Field::new("b", DataType::Int32, true), + Field::new("c", DataType::Int32, true), + Field::new("d", DataType::Int32, true), + Field::new("e", DataType::Int32, true), + ])), + vec![Arc::new(DummyStreamPartition { + schema: Arc::new(Schema::new(vec![ + Field::new("a", DataType::Int32, true), + Field::new("b", DataType::Int32, true), + Field::new("c", DataType::Int32, true), + Field::new("d", DataType::Int32, true), + Field::new("e", DataType::Int32, true), + ])), + }) as _], + Some(&vec![0_usize, 2, 4, 3]), + vec![ + vec![ + PhysicalSortExpr { + expr: Arc::new(Column::new("e", 2)), + options: SortOptions::default(), + }, + PhysicalSortExpr { + expr: Arc::new(Column::new("a", 0)), + options: SortOptions::default(), + }, + ], + vec![PhysicalSortExpr { + expr: Arc::new(Column::new("d", 3)), + options: SortOptions::default(), + }], + ] + .into_iter(), + true, + )?; + let projection = Arc::new(ProjectionExec::try_new( + vec![ + (Arc::new(Column::new("d", 3)), "d".to_string()), + (Arc::new(Column::new("e", 2)), "e".to_string()), + (Arc::new(Column::new("a", 0)), "a".to_string()), + ], + Arc::new(streaming_table) as _, + )?) as _; + + let after_optimize = + ProjectionPushdown::new().optimize(projection, &ConfigOptions::new())?; + + let result = after_optimize + .as_any() + .downcast_ref::() + .unwrap(); + assert_eq!( + result.partition_schema(), + &Arc::new(Schema::new(vec![ + Field::new("a", DataType::Int32, true), + Field::new("b", DataType::Int32, true), + Field::new("c", DataType::Int32, true), + Field::new("d", DataType::Int32, true), + Field::new("e", DataType::Int32, true), + ])) + ); + assert_eq!( + result.projection().clone().unwrap().to_vec(), + vec![3_usize, 4, 0] + ); + assert_eq!( + result.projected_schema(), + &Schema::new(vec![ + Field::new("d", DataType::Int32, true), + Field::new("e", DataType::Int32, true), + Field::new("a", DataType::Int32, true), + ]) + ); + assert_eq!( + result.projected_output_ordering().into_iter().collect_vec(), + vec![ + vec![ + PhysicalSortExpr { + expr: Arc::new(Column::new("e", 1)), + options: SortOptions::default(), + }, + PhysicalSortExpr { + expr: Arc::new(Column::new("a", 2)), + options: SortOptions::default(), + }, + ], + vec![PhysicalSortExpr { + expr: Arc::new(Column::new("d", 0)), + options: SortOptions::default(), + }], + ] + ); + assert!(result.is_infinite()); + + Ok(()) + } + #[test] fn test_projection_after_projection() -> Result<()> { let csv = create_simple_csv_exec(); diff --git a/datafusion/physical-plan/src/streaming.rs b/datafusion/physical-plan/src/streaming.rs index b0eaa2b42f42..59819c6921fb 100644 --- a/datafusion/physical-plan/src/streaming.rs +++ b/datafusion/physical-plan/src/streaming.rs @@ -26,6 +26,7 @@ use crate::stream::RecordBatchStreamAdapter; use crate::{ExecutionPlan, Partitioning, SendableRecordBatchStream}; use arrow::datatypes::SchemaRef; +use arrow_schema::Schema; use datafusion_common::{internal_err, plan_err, DataFusionError, Result}; use datafusion_execution::TaskContext; use datafusion_physical_expr::{EquivalenceProperties, LexOrdering, PhysicalSortExpr}; @@ -70,9 +71,9 @@ impl StreamingTableExec { ) -> Result { for x in partitions.iter() { let partition_schema = x.schema(); - if !schema.contains(partition_schema) { + if !schema.eq(partition_schema) { debug!( - "target schema does not contain partition schema. \ + "Target schema does not match with partition schema. \ Target_schema: {schema:?}. Partiton Schema: {partition_schema:?}" ); return plan_err!("Mismatch between schema and batches"); @@ -92,6 +93,30 @@ impl StreamingTableExec { infinite, }) } + + pub fn partitions(&self) -> &Vec> { + &self.partitions + } + + pub fn partition_schema(&self) -> &SchemaRef { + self.partitions[0].schema() + } + + pub fn projection(&self) -> &Option> { + &self.projection + } + + pub fn projected_schema(&self) -> &Schema { + &self.projected_schema + } + + pub fn projected_output_ordering(&self) -> impl IntoIterator { + self.projected_output_ordering.clone() + } + + pub fn is_infinite(&self) -> bool { + self.infinite + } } impl std::fmt::Debug for StreamingTableExec { From 6d5a350db07503f8b8e22102b001aadad56b7ec9 Mon Sep 17 00:00:00 2001 From: comphead Date: Mon, 27 Nov 2023 14:34:08 -0800 Subject: [PATCH 315/572] minor: fix documentation (#8323) --- docs/source/user-guide/sql/scalar_functions.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/source/user-guide/sql/scalar_functions.md b/docs/source/user-guide/sql/scalar_functions.md index e7ebbc9f1fe7..74dceb221ad2 100644 --- a/docs/source/user-guide/sql/scalar_functions.md +++ b/docs/source/user-guide/sql/scalar_functions.md @@ -2315,7 +2315,7 @@ array_union(array1, array2) +----------------------------------------------------+ | array_union([1, 2, 3, 4], [5, 6, 7, 8]); | +----------------------------------------------------+ -| [1, 2, 3, 4, 5, 6] | +| [1, 2, 3, 4, 5, 6, 7, 8] | +----------------------------------------------------+ ``` From 9dfb224058b084a69274905e216c8043d7e1d79f Mon Sep 17 00:00:00 2001 From: jokercurry <982458633@qq.com> Date: Tue, 28 Nov 2023 06:34:41 +0800 Subject: [PATCH 316/572] fix: wrong result of range function (#8313) * fix: wrong result of range function * fix test * add ut * add ut * nit * nit --------- Co-authored-by: zhongjingxiong --- .../physical-expr/src/array_expressions.rs | 71 ++++++++++++++++++- datafusion/sqllogictest/test_files/array.slt | 7 +- 2 files changed, 73 insertions(+), 5 deletions(-) diff --git a/datafusion/physical-expr/src/array_expressions.rs b/datafusion/physical-expr/src/array_expressions.rs index 8968bcf2ea4e..6b7bef8e6a36 100644 --- a/datafusion/physical-expr/src/array_expressions.rs +++ b/datafusion/physical-expr/src/array_expressions.rs @@ -746,8 +746,14 @@ pub fn gen_range(args: &[ArrayRef]) -> Result { if step == 0 { return exec_err!("step can't be 0 for function range(start [, stop, step]"); } - let value = (start..stop).step_by(step as usize); - values.extend(value); + if step < 0 { + // Decreasing range + values.extend((stop + 1..start + 1).rev().step_by((-step) as usize)); + } else { + // Increasing range + values.extend((start..stop).step_by(step as usize)); + } + offsets.push(values.len() as i32); } let arr = Arc::new(ListArray::try_new( @@ -2514,6 +2520,67 @@ mod tests { .is_null(0)); } + #[test] + fn test_array_range() { + // range(1, 5, 1) = [1, 2, 3, 4] + let args1 = Arc::new(Int64Array::from(vec![Some(1)])) as ArrayRef; + let args2 = Arc::new(Int64Array::from(vec![Some(5)])) as ArrayRef; + let args3 = Arc::new(Int64Array::from(vec![Some(1)])) as ArrayRef; + let arr = gen_range(&[args1, args2, args3]).unwrap(); + + let result = as_list_array(&arr).expect("failed to initialize function range"); + assert_eq!( + &[1, 2, 3, 4], + result + .value(0) + .as_any() + .downcast_ref::() + .unwrap() + .values() + ); + + // range(1, -5, -1) = [1, 0, -1, -2, -3, -4] + let args1 = Arc::new(Int64Array::from(vec![Some(1)])) as ArrayRef; + let args2 = Arc::new(Int64Array::from(vec![Some(-5)])) as ArrayRef; + let args3 = Arc::new(Int64Array::from(vec![Some(-1)])) as ArrayRef; + let arr = gen_range(&[args1, args2, args3]).unwrap(); + + let result = as_list_array(&arr).expect("failed to initialize function range"); + assert_eq!( + &[1, 0, -1, -2, -3, -4], + result + .value(0) + .as_any() + .downcast_ref::() + .unwrap() + .values() + ); + + // range(1, 5, -1) = [] + let args1 = Arc::new(Int64Array::from(vec![Some(1)])) as ArrayRef; + let args2 = Arc::new(Int64Array::from(vec![Some(5)])) as ArrayRef; + let args3 = Arc::new(Int64Array::from(vec![Some(-1)])) as ArrayRef; + let arr = gen_range(&[args1, args2, args3]).unwrap(); + + let result = as_list_array(&arr).expect("failed to initialize function range"); + assert_eq!( + &[], + result + .value(0) + .as_any() + .downcast_ref::() + .unwrap() + .values() + ); + + // range(1, 5, 0) = [] + let args1 = Arc::new(Int64Array::from(vec![Some(1)])) as ArrayRef; + let args2 = Arc::new(Int64Array::from(vec![Some(5)])) as ArrayRef; + let args3 = Arc::new(Int64Array::from(vec![Some(0)])) as ArrayRef; + let is_err = gen_range(&[args1, args2, args3]).is_err(); + assert!(is_err) + } + #[test] fn test_nested_array_slice() { // array_slice([[1, 2, 3, 4], [5, 6, 7, 8]], 1, 1) = [[1, 2, 3, 4]] diff --git a/datafusion/sqllogictest/test_files/array.slt b/datafusion/sqllogictest/test_files/array.slt index d33555509e6c..db657ff22bd5 100644 --- a/datafusion/sqllogictest/test_files/array.slt +++ b/datafusion/sqllogictest/test_files/array.slt @@ -2744,15 +2744,16 @@ from arrays_range; [0, 1, 2, 3, 4, 5, 6, 7, 8, 9] [3, 4, 5, 6, 7, 8, 9] [3, 5, 7, 9] [0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12] [4, 5, 6, 7, 8, 9, 10, 11, 12] [4, 7, 10] -query ????? +query ?????? select range(5), range(2, 5), range(2, 10, 3), range(1, 5, -1), - range(1, -5, 1) + range(1, -5, 1), + range(1, -5, -1) ; ---- -[0, 1, 2, 3, 4] [2, 3, 4] [2, 5, 8] [1] [] +[0, 1, 2, 3, 4] [2, 3, 4] [2, 5, 8] [] [] [1, 0, -1, -2, -3, -4] query ??? select generate_series(5), From a7cc08967c6c8dfbee92f9fb8273f30efefb75aa Mon Sep 17 00:00:00 2001 From: Andrew Lamb Date: Mon, 27 Nov 2023 17:43:04 -0500 Subject: [PATCH 317/572] Minor: rename parquet.rs to parquet/mod.rs (#8301) --- .../src/datasource/physical_plan/{parquet.rs => parquet/mod.rs} | 0 1 file changed, 0 insertions(+), 0 deletions(-) rename datafusion/core/src/datasource/physical_plan/{parquet.rs => parquet/mod.rs} (100%) diff --git a/datafusion/core/src/datasource/physical_plan/parquet.rs b/datafusion/core/src/datasource/physical_plan/parquet/mod.rs similarity index 100% rename from datafusion/core/src/datasource/physical_plan/parquet.rs rename to datafusion/core/src/datasource/physical_plan/parquet/mod.rs From d22ed692c37a4cb2142fbab29e59642c3409262a Mon Sep 17 00:00:00 2001 From: Wei Date: Tue, 28 Nov 2023 06:45:10 +0800 Subject: [PATCH 318/572] refactor: output ordering (#8304) * refactor: output-ordering * chore: test * chore: cr comment Co-authored-by: Alex Huang --------- Co-authored-by: Alex Huang --- .../core/src/datasource/physical_plan/mod.rs | 19 +++++++++++++++++-- .../sqllogictest/test_files/groupby.slt | 2 +- datafusion/sqllogictest/test_files/insert.slt | 2 +- datafusion/sqllogictest/test_files/order.slt | 6 +++--- .../sqllogictest/test_files/subquery.slt | 4 ++-- datafusion/sqllogictest/test_files/window.slt | 6 +++--- 6 files changed, 27 insertions(+), 12 deletions(-) diff --git a/datafusion/core/src/datasource/physical_plan/mod.rs b/datafusion/core/src/datasource/physical_plan/mod.rs index aca71678d98b..4cf115d03a9b 100644 --- a/datafusion/core/src/datasource/physical_plan/mod.rs +++ b/datafusion/core/src/datasource/physical_plan/mod.rs @@ -135,9 +135,24 @@ impl DisplayAs for FileScanConfig { write!(f, ", infinite_source=true")?; } - if let Some(ordering) = orderings.first() { + if let Some(ordering) = orderings.get(0) { if !ordering.is_empty() { - write!(f, ", output_ordering={}", OutputOrderingDisplay(ordering))?; + let start = if orderings.len() == 1 { + ", output_ordering=" + } else { + ", output_orderings=[" + }; + write!(f, "{}", start)?; + for (idx, ordering) in + orderings.iter().enumerate().filter(|(_, o)| !o.is_empty()) + { + match idx { + 0 => write!(f, "{}", OutputOrderingDisplay(ordering))?, + _ => write!(f, ", {}", OutputOrderingDisplay(ordering))?, + } + } + let end = if orderings.len() == 1 { "" } else { "]" }; + write!(f, "{}", end)?; } } diff --git a/datafusion/sqllogictest/test_files/groupby.slt b/datafusion/sqllogictest/test_files/groupby.slt index d6f9adb02335..1d6d7dc671fa 100644 --- a/datafusion/sqllogictest/test_files/groupby.slt +++ b/datafusion/sqllogictest/test_files/groupby.slt @@ -3628,7 +3628,7 @@ ProjectionExec: expr=[FIRST_VALUE(multiple_ordered_table.a) ORDER BY [multiple_o ------RepartitionExec: partitioning=Hash([d@0], 8), input_partitions=8 --------AggregateExec: mode=Partial, gby=[d@2 as d], aggr=[FIRST_VALUE(multiple_ordered_table.a), FIRST_VALUE(multiple_ordered_table.c)] ----------RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1 -------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, c, d], output_ordering=[a@0 ASC NULLS LAST], has_header=true +------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, c, d], output_orderings=[[a@0 ASC NULLS LAST], [c@1 ASC NULLS LAST]], has_header=true query II rowsort SELECT FIRST_VALUE(a ORDER BY a ASC) as first_a, diff --git a/datafusion/sqllogictest/test_files/insert.slt b/datafusion/sqllogictest/test_files/insert.slt index 9734aab9ab07..75252b3b7c35 100644 --- a/datafusion/sqllogictest/test_files/insert.slt +++ b/datafusion/sqllogictest/test_files/insert.slt @@ -386,7 +386,7 @@ statement ok drop table test_column_defaults -# test create table as +# test create table as statement ok create table test_column_defaults( a int, diff --git a/datafusion/sqllogictest/test_files/order.slt b/datafusion/sqllogictest/test_files/order.slt index 9c5d1704f42b..77df9e0bb493 100644 --- a/datafusion/sqllogictest/test_files/order.slt +++ b/datafusion/sqllogictest/test_files/order.slt @@ -441,7 +441,7 @@ physical_plan SortPreservingMergeExec: [result@0 ASC NULLS LAST] --ProjectionExec: expr=[b@1 + a@0 + c@2 as result] ----RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, b, c], output_ordering=[a@0 ASC NULLS LAST], has_header=true +------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, b, c], output_orderings=[[a@0 ASC NULLS LAST], [b@1 ASC NULLS LAST], [c@2 ASC NULLS LAST]], has_header=true statement ok drop table multiple_ordered_table; @@ -559,7 +559,7 @@ physical_plan SortPreservingMergeExec: [log_c11_base_c12@0 ASC NULLS LAST] --ProjectionExec: expr=[log(CAST(c11@0 AS Float64), c12@1) as log_c11_base_c12] ----RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c11, c12], output_ordering=[c11@0 ASC NULLS LAST], has_header=true +------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c11, c12], output_orderings=[[c11@0 ASC NULLS LAST], [c12@1 DESC]], has_header=true query TT EXPLAIN SELECT LOG(c12, c11) as log_c12_base_c11 @@ -574,7 +574,7 @@ physical_plan SortPreservingMergeExec: [log_c12_base_c11@0 DESC] --ProjectionExec: expr=[log(c12@1, CAST(c11@0 AS Float64)) as log_c12_base_c11] ----RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c11, c12], output_ordering=[c11@0 ASC NULLS LAST], has_header=true +------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c11, c12], output_orderings=[[c11@0 ASC NULLS LAST], [c12@1 DESC]], has_header=true statement ok drop table aggregate_test_100; diff --git a/datafusion/sqllogictest/test_files/subquery.slt b/datafusion/sqllogictest/test_files/subquery.slt index ef25d960c954..4729c3f01054 100644 --- a/datafusion/sqllogictest/test_files/subquery.slt +++ b/datafusion/sqllogictest/test_files/subquery.slt @@ -992,7 +992,7 @@ catan-prod1-daily success catan-prod1-daily high ##correlated_scalar_subquery_sum_agg_bug #query TT #explain -#select t1.t1_int from t1 where +#select t1.t1_int from t1 where # (select sum(t2_int) is null from t2 where t1.t1_id = t2.t2_id) #---- #logical_plan @@ -1006,7 +1006,7 @@ catan-prod1-daily success catan-prod1-daily high #------------TableScan: t2 projection=[t2_id, t2_int] #query I rowsort -#select t1.t1_int from t1 where +#select t1.t1_int from t1 where # (select sum(t2_int) is null from t2 where t1.t1_id = t2.t2_id) #---- #2 diff --git a/datafusion/sqllogictest/test_files/window.slt b/datafusion/sqllogictest/test_files/window.slt index 319c08407661..4edac211b370 100644 --- a/datafusion/sqllogictest/test_files/window.slt +++ b/datafusion/sqllogictest/test_files/window.slt @@ -3429,7 +3429,7 @@ ProjectionExec: expr=[MIN(multiple_ordered_table.d) ORDER BY [multiple_ordered_t --BoundedWindowAggExec: wdw=[MIN(multiple_ordered_table.d) ORDER BY [multiple_ordered_table.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "MIN(multiple_ordered_table.d) ORDER BY [multiple_ordered_table.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(NULL)), end_bound: CurrentRow }], mode=[Sorted] ----ProjectionExec: expr=[c@2 as c, d@3 as d, MAX(multiple_ordered_table.d) PARTITION BY [multiple_ordered_table.b, multiple_ordered_table.a] ORDER BY [multiple_ordered_table.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@4 as MAX(multiple_ordered_table.d) PARTITION BY [multiple_ordered_table.b, multiple_ordered_table.a] ORDER BY [multiple_ordered_table.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW] ------BoundedWindowAggExec: wdw=[MAX(multiple_ordered_table.d) PARTITION BY [multiple_ordered_table.b, multiple_ordered_table.a] ORDER BY [multiple_ordered_table.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "MAX(multiple_ordered_table.d) PARTITION BY [multiple_ordered_table.b, multiple_ordered_table.a] ORDER BY [multiple_ordered_table.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(NULL)), end_bound: CurrentRow }], mode=[Sorted] ---------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, b, c, d], output_ordering=[a@0 ASC NULLS LAST, b@1 ASC NULLS LAST], has_header=true +--------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, b, c, d], output_orderings=[[a@0 ASC NULLS LAST, b@1 ASC NULLS LAST], [c@2 ASC NULLS LAST]], has_header=true query TT EXPLAIN SELECT MAX(c) OVER(PARTITION BY d ORDER BY c ASC) as max_c @@ -3461,7 +3461,7 @@ Projection: SUM(multiple_ordered_table.d) PARTITION BY [multiple_ordered_table.c physical_plan ProjectionExec: expr=[SUM(multiple_ordered_table.d) PARTITION BY [multiple_ordered_table.c] ORDER BY [multiple_ordered_table.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@3 as SUM(multiple_ordered_table.d) PARTITION BY [multiple_ordered_table.c] ORDER BY [multiple_ordered_table.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW] --BoundedWindowAggExec: wdw=[SUM(multiple_ordered_table.d) PARTITION BY [multiple_ordered_table.c] ORDER BY [multiple_ordered_table.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "SUM(multiple_ordered_table.d) PARTITION BY [multiple_ordered_table.c] ORDER BY [multiple_ordered_table.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(NULL)), end_bound: CurrentRow }], mode=[Sorted] -----CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, c, d], output_ordering=[a@0 ASC NULLS LAST], has_header=true +----CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, c, d], output_orderings=[[a@0 ASC NULLS LAST], [c@1 ASC NULLS LAST]], has_header=true query TT explain SELECT SUM(d) OVER(PARTITION BY c, a ORDER BY b ASC) @@ -3474,7 +3474,7 @@ Projection: SUM(multiple_ordered_table.d) PARTITION BY [multiple_ordered_table.c physical_plan ProjectionExec: expr=[SUM(multiple_ordered_table.d) PARTITION BY [multiple_ordered_table.c, multiple_ordered_table.a] ORDER BY [multiple_ordered_table.b ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@4 as SUM(multiple_ordered_table.d) PARTITION BY [multiple_ordered_table.c, multiple_ordered_table.a] ORDER BY [multiple_ordered_table.b ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW] --BoundedWindowAggExec: wdw=[SUM(multiple_ordered_table.d) PARTITION BY [multiple_ordered_table.c, multiple_ordered_table.a] ORDER BY [multiple_ordered_table.b ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "SUM(multiple_ordered_table.d) PARTITION BY [multiple_ordered_table.c, multiple_ordered_table.a] ORDER BY [multiple_ordered_table.b ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(NULL)), end_bound: CurrentRow }], mode=[Sorted] -----CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, b, c, d], output_ordering=[a@0 ASC NULLS LAST, b@1 ASC NULLS LAST], has_header=true +----CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, b, c, d], output_orderings=[[a@0 ASC NULLS LAST, b@1 ASC NULLS LAST], [c@2 ASC NULLS LAST]], has_header=true query I SELECT SUM(d) OVER(PARTITION BY c, a ORDER BY b ASC) From 88204bfb6d13db02928402c7abd763de9ce9a4af Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Tue, 28 Nov 2023 07:44:04 -0500 Subject: [PATCH 319/572] Update substrait requirement from 0.19.0 to 0.20.0 (#8339) Updates the requirements on [substrait](https://github.com/substrait-io/substrait-rs) to permit the latest version. - [Release notes](https://github.com/substrait-io/substrait-rs/releases) - [Changelog](https://github.com/substrait-io/substrait-rs/blob/main/CHANGELOG.md) - [Commits](https://github.com/substrait-io/substrait-rs/compare/v0.19.0...v0.20.0) --- updated-dependencies: - dependency-name: substrait dependency-type: direct:production ... Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- datafusion/substrait/Cargo.toml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/datafusion/substrait/Cargo.toml b/datafusion/substrait/Cargo.toml index 102b0a7c58f1..42ebe56c298b 100644 --- a/datafusion/substrait/Cargo.toml +++ b/datafusion/substrait/Cargo.toml @@ -35,7 +35,7 @@ itertools = { workspace = true } object_store = { workspace = true } prost = "0.12" prost-types = "0.12" -substrait = "0.19.0" +substrait = "0.20.0" tokio = "1.17" [features] From 3c12deeb1faa28b9512eb2343cf91855030c7bde Mon Sep 17 00:00:00 2001 From: Edmondo Porcu Date: Tue, 28 Nov 2023 04:49:02 -0800 Subject: [PATCH 320/572] First iteration of aggregates.rs tests to sqllogictests (#8316) --- datafusion/core/tests/sql/aggregates.rs | 501 ------------------ .../sqllogictest/test_files/aggregate.slt | 165 +++++- 2 files changed, 146 insertions(+), 520 deletions(-) diff --git a/datafusion/core/tests/sql/aggregates.rs b/datafusion/core/tests/sql/aggregates.rs index 03864e9efef8..af6d0d5f4e24 100644 --- a/datafusion/core/tests/sql/aggregates.rs +++ b/datafusion/core/tests/sql/aggregates.rs @@ -17,8 +17,6 @@ use super::*; use datafusion::scalar::ScalarValue; -use datafusion::test_util::scan_empty; -use datafusion_common::cast::as_float64_array; #[tokio::test] async fn csv_query_array_agg_distinct() -> Result<()> { @@ -68,324 +66,6 @@ async fn csv_query_array_agg_distinct() -> Result<()> { Ok(()) } -#[tokio::test] -async fn aggregate() -> Result<()> { - let results = execute_with_partition("SELECT SUM(c1), SUM(c2) FROM test", 4).await?; - assert_eq!(results.len(), 1); - - let expected = [ - "+--------------+--------------+", - "| SUM(test.c1) | SUM(test.c2) |", - "+--------------+--------------+", - "| 60 | 220 |", - "+--------------+--------------+", - ]; - assert_batches_sorted_eq!(expected, &results); - - Ok(()) -} - -#[tokio::test] -async fn aggregate_empty() -> Result<()> { - // The predicate on this query purposely generates no results - let results = - execute_with_partition("SELECT SUM(c1), SUM(c2) FROM test where c1 > 100000", 4) - .await - .unwrap(); - - assert_eq!(results.len(), 1); - - let expected = [ - "+--------------+--------------+", - "| SUM(test.c1) | SUM(test.c2) |", - "+--------------+--------------+", - "| | |", - "+--------------+--------------+", - ]; - assert_batches_sorted_eq!(expected, &results); - - Ok(()) -} - -#[tokio::test] -async fn aggregate_avg() -> Result<()> { - let results = execute_with_partition("SELECT AVG(c1), AVG(c2) FROM test", 4).await?; - assert_eq!(results.len(), 1); - - let expected = [ - "+--------------+--------------+", - "| AVG(test.c1) | AVG(test.c2) |", - "+--------------+--------------+", - "| 1.5 | 5.5 |", - "+--------------+--------------+", - ]; - assert_batches_sorted_eq!(expected, &results); - - Ok(()) -} - -#[tokio::test] -async fn aggregate_max() -> Result<()> { - let results = execute_with_partition("SELECT MAX(c1), MAX(c2) FROM test", 4).await?; - assert_eq!(results.len(), 1); - - let expected = [ - "+--------------+--------------+", - "| MAX(test.c1) | MAX(test.c2) |", - "+--------------+--------------+", - "| 3 | 10 |", - "+--------------+--------------+", - ]; - assert_batches_sorted_eq!(expected, &results); - - Ok(()) -} - -#[tokio::test] -async fn aggregate_min() -> Result<()> { - let results = execute_with_partition("SELECT MIN(c1), MIN(c2) FROM test", 4).await?; - assert_eq!(results.len(), 1); - - let expected = [ - "+--------------+--------------+", - "| MIN(test.c1) | MIN(test.c2) |", - "+--------------+--------------+", - "| 0 | 1 |", - "+--------------+--------------+", - ]; - assert_batches_sorted_eq!(expected, &results); - - Ok(()) -} - -#[tokio::test] -async fn aggregate_grouped() -> Result<()> { - let results = - execute_with_partition("SELECT c1, SUM(c2) FROM test GROUP BY c1", 4).await?; - - let expected = [ - "+----+--------------+", - "| c1 | SUM(test.c2) |", - "+----+--------------+", - "| 0 | 55 |", - "| 1 | 55 |", - "| 2 | 55 |", - "| 3 | 55 |", - "+----+--------------+", - ]; - assert_batches_sorted_eq!(expected, &results); - - Ok(()) -} - -#[tokio::test] -async fn aggregate_grouped_avg() -> Result<()> { - let results = - execute_with_partition("SELECT c1, AVG(c2) FROM test GROUP BY c1", 4).await?; - - let expected = [ - "+----+--------------+", - "| c1 | AVG(test.c2) |", - "+----+--------------+", - "| 0 | 5.5 |", - "| 1 | 5.5 |", - "| 2 | 5.5 |", - "| 3 | 5.5 |", - "+----+--------------+", - ]; - assert_batches_sorted_eq!(expected, &results); - - Ok(()) -} - -#[tokio::test] -async fn aggregate_grouped_empty() -> Result<()> { - let results = execute_with_partition( - "SELECT c1, AVG(c2) FROM test WHERE c1 = 123 GROUP BY c1", - 4, - ) - .await?; - - let expected = [ - "+----+--------------+", - "| c1 | AVG(test.c2) |", - "+----+--------------+", - "+----+--------------+", - ]; - assert_batches_sorted_eq!(expected, &results); - - Ok(()) -} - -#[tokio::test] -async fn aggregate_grouped_max() -> Result<()> { - let results = - execute_with_partition("SELECT c1, MAX(c2) FROM test GROUP BY c1", 4).await?; - - let expected = [ - "+----+--------------+", - "| c1 | MAX(test.c2) |", - "+----+--------------+", - "| 0 | 10 |", - "| 1 | 10 |", - "| 2 | 10 |", - "| 3 | 10 |", - "+----+--------------+", - ]; - assert_batches_sorted_eq!(expected, &results); - - Ok(()) -} - -#[tokio::test] -async fn aggregate_grouped_min() -> Result<()> { - let results = - execute_with_partition("SELECT c1, MIN(c2) FROM test GROUP BY c1", 4).await?; - - let expected = [ - "+----+--------------+", - "| c1 | MIN(test.c2) |", - "+----+--------------+", - "| 0 | 1 |", - "| 1 | 1 |", - "| 2 | 1 |", - "| 3 | 1 |", - "+----+--------------+", - ]; - assert_batches_sorted_eq!(expected, &results); - - Ok(()) -} - -#[tokio::test] -async fn aggregate_min_max_w_custom_window_frames() -> Result<()> { - let ctx = SessionContext::new(); - register_aggregate_csv(&ctx).await?; - let sql = - "SELECT - MIN(c12) OVER (ORDER BY C12 RANGE BETWEEN 0.3 PRECEDING AND 0.2 FOLLOWING) as min1, - MAX(c12) OVER (ORDER BY C11 RANGE BETWEEN 0.1 PRECEDING AND 0.2 FOLLOWING) as max1 - FROM aggregate_test_100 - ORDER BY C9 - LIMIT 5"; - let actual = execute_to_batches(&ctx, sql).await; - let expected = [ - "+---------------------+--------------------+", - "| min1 | max1 |", - "+---------------------+--------------------+", - "| 0.01479305307777301 | 0.9965400387585364 |", - "| 0.01479305307777301 | 0.9800193410444061 |", - "| 0.01479305307777301 | 0.9706712283358269 |", - "| 0.2667177795079635 | 0.9965400387585364 |", - "| 0.3600766362333053 | 0.9706712283358269 |", - "+---------------------+--------------------+", - ]; - assert_batches_eq!(expected, &actual); - Ok(()) -} - -#[tokio::test] -async fn aggregate_min_max_w_custom_window_frames_unbounded_start() -> Result<()> { - let ctx = SessionContext::new(); - register_aggregate_csv(&ctx).await?; - let sql = - "SELECT - MIN(c12) OVER (ORDER BY C12 RANGE BETWEEN UNBOUNDED PRECEDING AND 0.2 FOLLOWING) as min1, - MAX(c12) OVER (ORDER BY C11 RANGE BETWEEN UNBOUNDED PRECEDING AND 0.2 FOLLOWING) as max1 - FROM aggregate_test_100 - ORDER BY C9 - LIMIT 5"; - let actual = execute_to_batches(&ctx, sql).await; - let expected = [ - "+---------------------+--------------------+", - "| min1 | max1 |", - "+---------------------+--------------------+", - "| 0.01479305307777301 | 0.9965400387585364 |", - "| 0.01479305307777301 | 0.9800193410444061 |", - "| 0.01479305307777301 | 0.9800193410444061 |", - "| 0.01479305307777301 | 0.9965400387585364 |", - "| 0.01479305307777301 | 0.9800193410444061 |", - "+---------------------+--------------------+", - ]; - assert_batches_eq!(expected, &actual); - Ok(()) -} - -#[tokio::test] -async fn aggregate_avg_add() -> Result<()> { - let results = execute_with_partition( - "SELECT AVG(c1), AVG(c1) + 1, AVG(c1) + 2, 1 + AVG(c1) FROM test", - 4, - ) - .await?; - assert_eq!(results.len(), 1); - - let expected = ["+--------------+-------------------------+-------------------------+-------------------------+", - "| AVG(test.c1) | AVG(test.c1) + Int64(1) | AVG(test.c1) + Int64(2) | Int64(1) + AVG(test.c1) |", - "+--------------+-------------------------+-------------------------+-------------------------+", - "| 1.5 | 2.5 | 3.5 | 2.5 |", - "+--------------+-------------------------+-------------------------+-------------------------+"]; - assert_batches_sorted_eq!(expected, &results); - - Ok(()) -} - -#[tokio::test] -async fn case_sensitive_identifiers_aggregates() { - let ctx = SessionContext::new(); - ctx.register_table("t", table_with_sequence(1, 1).unwrap()) - .unwrap(); - - let expected = [ - "+----------+", - "| MAX(t.i) |", - "+----------+", - "| 1 |", - "+----------+", - ]; - - let results = plan_and_collect(&ctx, "SELECT max(i) FROM t") - .await - .unwrap(); - - assert_batches_sorted_eq!(expected, &results); - - let results = plan_and_collect(&ctx, "SELECT MAX(i) FROM t") - .await - .unwrap(); - assert_batches_sorted_eq!(expected, &results); - - // Using double quotes allows specifying the function name with capitalization - let err = plan_and_collect(&ctx, "SELECT \"MAX\"(i) FROM t") - .await - .unwrap_err(); - assert!(err - .to_string() - .contains("Error during planning: Invalid function 'MAX'")); - - let results = plan_and_collect(&ctx, "SELECT \"max\"(i) FROM t") - .await - .unwrap(); - assert_batches_sorted_eq!(expected, &results); -} - -#[tokio::test] -async fn count_basic() -> Result<()> { - let results = - execute_with_partition("SELECT COUNT(c1), COUNT(c2) FROM test", 1).await?; - assert_eq!(results.len(), 1); - - let expected = [ - "+----------------+----------------+", - "| COUNT(test.c1) | COUNT(test.c2) |", - "+----------------+----------------+", - "| 10 | 10 |", - "+----------------+----------------+", - ]; - assert_batches_sorted_eq!(expected, &results); - Ok(()) -} - #[tokio::test] async fn count_partitioned() -> Result<()> { let results = @@ -495,162 +175,6 @@ async fn count_aggregated_cube() -> Result<()> { Ok(()) } -#[tokio::test] -async fn count_multi_expr() -> Result<()> { - let schema = Arc::new(Schema::new(vec![ - Field::new("c1", DataType::Int32, true), - Field::new("c2", DataType::Int32, true), - ])); - - let data = RecordBatch::try_new( - schema.clone(), - vec![ - Arc::new(Int32Array::from(vec![ - Some(0), - None, - Some(1), - Some(2), - None, - ])), - Arc::new(Int32Array::from(vec![ - Some(1), - Some(1), - Some(0), - None, - None, - ])), - ], - )?; - - let ctx = SessionContext::new(); - ctx.register_batch("test", data)?; - let sql = "SELECT count(c1, c2) FROM test"; - let actual = execute_to_batches(&ctx, sql).await; - - let expected = [ - "+------------------------+", - "| COUNT(test.c1,test.c2) |", - "+------------------------+", - "| 2 |", - "+------------------------+", - ]; - assert_batches_sorted_eq!(expected, &actual); - Ok(()) -} - -#[tokio::test] -async fn count_multi_expr_group_by() -> Result<()> { - let schema = Arc::new(Schema::new(vec![ - Field::new("c1", DataType::Int32, true), - Field::new("c2", DataType::Int32, true), - Field::new("c3", DataType::Int32, true), - ])); - - let data = RecordBatch::try_new( - schema.clone(), - vec![ - Arc::new(Int32Array::from(vec![ - Some(0), - None, - Some(1), - Some(2), - None, - ])), - Arc::new(Int32Array::from(vec![ - Some(1), - Some(1), - Some(0), - None, - None, - ])), - Arc::new(Int32Array::from(vec![ - Some(10), - Some(10), - Some(10), - Some(10), - Some(10), - ])), - ], - )?; - - let ctx = SessionContext::new(); - ctx.register_batch("test", data)?; - let sql = "SELECT c3, count(c1, c2) FROM test group by c3"; - let actual = execute_to_batches(&ctx, sql).await; - - let expected = [ - "+----+------------------------+", - "| c3 | COUNT(test.c1,test.c2) |", - "+----+------------------------+", - "| 10 | 2 |", - "+----+------------------------+", - ]; - assert_batches_sorted_eq!(expected, &actual); - Ok(()) -} - -#[tokio::test] -async fn simple_avg() -> Result<()> { - let schema = Schema::new(vec![Field::new("a", DataType::Int32, false)]); - - let batch1 = RecordBatch::try_new( - Arc::new(schema.clone()), - vec![Arc::new(Int32Array::from(vec![1, 2, 3]))], - )?; - let batch2 = RecordBatch::try_new( - Arc::new(schema.clone()), - vec![Arc::new(Int32Array::from(vec![4, 5]))], - )?; - - let ctx = SessionContext::new(); - - let provider = MemTable::try_new(Arc::new(schema), vec![vec![batch1], vec![batch2]])?; - ctx.register_table("t", Arc::new(provider))?; - - let result = plan_and_collect(&ctx, "SELECT AVG(a) FROM t").await?; - - let batch = &result[0]; - assert_eq!(1, batch.num_columns()); - assert_eq!(1, batch.num_rows()); - - let values = as_float64_array(batch.column(0)).expect("failed to cast version"); - assert_eq!(values.len(), 1); - // avg(1,2,3,4,5) = 3.0 - assert_eq!(values.value(0), 3.0_f64); - Ok(()) -} - -#[tokio::test] -async fn simple_mean() -> Result<()> { - let schema = Schema::new(vec![Field::new("a", DataType::Int32, false)]); - - let batch1 = RecordBatch::try_new( - Arc::new(schema.clone()), - vec![Arc::new(Int32Array::from(vec![1, 2, 3]))], - )?; - let batch2 = RecordBatch::try_new( - Arc::new(schema.clone()), - vec![Arc::new(Int32Array::from(vec![4, 5]))], - )?; - - let ctx = SessionContext::new(); - - let provider = MemTable::try_new(Arc::new(schema), vec![vec![batch1], vec![batch2]])?; - ctx.register_table("t", Arc::new(provider))?; - - let result = plan_and_collect(&ctx, "SELECT MEAN(a) FROM t").await?; - - let batch = &result[0]; - assert_eq!(1, batch.num_columns()); - assert_eq!(1, batch.num_rows()); - - let values = as_float64_array(batch.column(0)).expect("failed to cast version"); - assert_eq!(values.len(), 1); - // mean(1,2,3,4,5) = 3.0 - assert_eq!(values.value(0), 3.0_f64); - Ok(()) -} - async fn run_count_distinct_integers_aggregated_scenario( partitions: Vec>, ) -> Result> { @@ -771,31 +295,6 @@ async fn count_distinct_integers_aggregated_multiple_partitions() -> Result<()> Ok(()) } -#[tokio::test] -async fn aggregate_with_alias() -> Result<()> { - let ctx = SessionContext::new(); - let state = ctx.state(); - - let schema = Arc::new(Schema::new(vec![ - Field::new("c1", DataType::Utf8, false), - Field::new("c2", DataType::UInt32, false), - ])); - - let plan = scan_empty(None, schema.as_ref(), None)? - .aggregate(vec![col("c1")], vec![sum(col("c2"))])? - .project(vec![col("c1"), sum(col("c2")).alias("total_salary")])? - .build()?; - - let plan = state.optimize(&plan)?; - let physical_plan = state.create_physical_plan(&Arc::new(plan)).await?; - assert_eq!("c1", physical_plan.schema().field(0).name().as_str()); - assert_eq!( - "total_salary", - physical_plan.schema().field(1).name().as_str() - ); - Ok(()) -} - #[tokio::test] async fn test_accumulator_row_accumulator() -> Result<()> { let config = SessionConfig::new(); diff --git a/datafusion/sqllogictest/test_files/aggregate.slt b/datafusion/sqllogictest/test_files/aggregate.slt index 7157be948914..a14c179326bb 100644 --- a/datafusion/sqllogictest/test_files/aggregate.slt +++ b/datafusion/sqllogictest/test_files/aggregate.slt @@ -1327,36 +1327,128 @@ select avg(c1), arrow_typeof(avg(c1)) from d_table ---- 5 Decimal128(14, 7) -# FIX: different test table + # aggregate -# query I -# SELECT SUM(c1), SUM(c2) FROM test -# ---- -# 60 220 +query II +SELECT SUM(c1), SUM(c2) FROM test +---- +7 6 + +# aggregate_empty + +query II +SELECT SUM(c1), SUM(c2) FROM test where c1 > 100000 +---- +NULL NULL + +# aggregate_avg +query RR +SELECT AVG(c1), AVG(c2) FROM test +---- +1.75 1.5 + +# aggregate_max +query II +SELECT MAX(c1), MAX(c2) FROM test +---- +3 2 + +# aggregate_min +query II +SELECT MIN(c1), MIN(c2) FROM test +---- +0 1 -# TODO: aggregate_empty +# aggregate_grouped +query II +SELECT c1, SUM(c2) FROM test GROUP BY c1 order by c1 +---- +0 NULL +1 1 +3 4 +NULL 1 -# TODO: aggregate_avg +# aggregate_grouped_avg +query IR +SELECT c1, AVG(c2) FROM test GROUP BY c1 order by c1 +---- +0 NULL +1 1 +3 2 +NULL 1 -# TODO: aggregate_max +# aggregate_grouped_empty +query IR +SELECT c1, AVG(c2) FROM test WHERE c1 = 123 GROUP BY c1 +---- -# TODO: aggregate_min +# aggregate_grouped_max +query II +SELECT c1, MAX(c2) FROM test GROUP BY c1 order by c1 +---- +0 NULL +1 1 +3 2 +NULL 1 -# TODO: aggregate_grouped +# aggregate_grouped_min +query II +SELECT c1, MIN(c2) FROM test GROUP BY c1 order by c1 +---- +0 NULL +1 1 +3 2 +NULL 1 -# TODO: aggregate_grouped_avg +# aggregate_min_max_w_custom_window_frames +query RR +SELECT +MIN(c12) OVER (ORDER BY C12 RANGE BETWEEN 0.3 PRECEDING AND 0.2 FOLLOWING) as min1, +MAX(c12) OVER (ORDER BY C11 RANGE BETWEEN 0.1 PRECEDING AND 0.2 FOLLOWING) as max1 +FROM aggregate_test_100 +ORDER BY C9 +LIMIT 5 +---- +0.014793053078 0.996540038759 +0.014793053078 0.980019341044 +0.014793053078 0.970671228336 +0.266717779508 0.996540038759 +0.360076636233 0.970671228336 -# TODO: aggregate_grouped_empty +# aggregate_min_max_with_custom_window_frames_unbounded_start +query RR +SELECT +MIN(c12) OVER (ORDER BY C12 RANGE BETWEEN UNBOUNDED PRECEDING AND 0.2 FOLLOWING) as min1, +MAX(c12) OVER (ORDER BY C11 RANGE BETWEEN UNBOUNDED PRECEDING AND 0.2 FOLLOWING) as max1 +FROM aggregate_test_100 +ORDER BY C9 +LIMIT 5 +---- +0.014793053078 0.996540038759 +0.014793053078 0.980019341044 +0.014793053078 0.980019341044 +0.014793053078 0.996540038759 +0.014793053078 0.980019341044 -# TODO: aggregate_grouped_max +# aggregate_avg_add +query RRRR +SELECT AVG(c1), AVG(c1) + 1, AVG(c1) + 2, 1 + AVG(c1) FROM test +---- +1.75 2.75 3.75 2.75 -# TODO: aggregate_grouped_min +# case_sensitive_identifiers_aggregates +query I +SELECT max(c1) FROM test; +---- +3 -# TODO: aggregate_avg_add -# TODO: case_sensitive_identifiers_aggregates -# TODO: count_basic +# count_basic +query II +SELECT COUNT(c1), COUNT(c2) FROM test +---- +4 4 # TODO: count_partitioned @@ -1364,9 +1456,44 @@ select avg(c1), arrow_typeof(avg(c1)) from d_table # TODO: count_aggregated_cube -# TODO: simple_avg +# count_multi_expr +query I +SELECT count(c1, c2) FROM test +---- +3 + +# count_multi_expr_group_by +query I +SELECT count(c1, c2) FROM test group by c1 order by c1 +---- +0 +1 +2 +0 + +# aggreggte_with_alias +query II +select c1, sum(c2) as `Total Salary` from test group by c1 order by c1 +---- +0 NULL +1 1 +3 4 +NULL 1 + +# simple_avg + +query R +select avg(c1) from test +---- +1.75 + +# simple_mean +query R +select mean(c1) from test +---- +1.75 + -# TODO: simple_mean # query_sum_distinct - 2 different aggregate functions: avg and sum(distinct) query RI From f1dbb2dad9aaec2fa85d75daefad3926d3df976a Mon Sep 17 00:00:00 2001 From: Tan Wei Date: Tue, 28 Nov 2023 20:56:00 +0800 Subject: [PATCH 321/572] Library Guide: Add Using the DataFrame API (#8319) * Library Guide: Add Using the DataFrame API Signed-off-by: veeupup * fix comments * fix comments Signed-off-by: veeupup --------- Signed-off-by: veeupup --- .../using-the-dataframe-api.md | 127 +++++++++++++++++- 1 file changed, 126 insertions(+), 1 deletion(-) diff --git a/docs/source/library-user-guide/using-the-dataframe-api.md b/docs/source/library-user-guide/using-the-dataframe-api.md index fdf309980dc2..c4f4ecd4f137 100644 --- a/docs/source/library-user-guide/using-the-dataframe-api.md +++ b/docs/source/library-user-guide/using-the-dataframe-api.md @@ -19,4 +19,129 @@ # Using the DataFrame API -Coming Soon +## What is a DataFrame + +`DataFrame` in `DataFrame` is modeled after the Pandas DataFrame interface, and is a thin wrapper over LogicalPlan that adds functionality for building and executing those plans. + +```rust +pub struct DataFrame { + session_state: SessionState, + plan: LogicalPlan, +} +``` + +You can build up `DataFrame`s using its methods, similarly to building `LogicalPlan`s using `LogicalPlanBuilder`: + +```rust +let df = ctx.table("users").await?; + +// Create a new DataFrame sorted by `id`, `bank_account` +let new_df = df.select(vec![col("id"), col("bank_account")])? + .sort(vec![col("id")])?; + +// Build the same plan using the LogicalPlanBuilder +let plan = LogicalPlanBuilder::from(&df.to_logical_plan()) + .project(vec![col("id"), col("bank_account")])? + .sort(vec![col("id")])? + .build()?; +``` + +You can use `collect` or `execute_stream` to execute the query. + +## How to generate a DataFrame + +You can directly use the `DataFrame` API or generate a `DataFrame` from a SQL query. + +For example, to use `sql` to construct `DataFrame`: + +```rust +let ctx = SessionContext::new(); +// Register the in-memory table containing the data +ctx.register_table("users", Arc::new(create_memtable()?))?; +let dataframe = ctx.sql("SELECT * FROM users;").await?; +``` + +To construct `DataFrame` using the API: + +```rust +let ctx = SessionContext::new(); +// Register the in-memory table containing the data +ctx.register_table("users", Arc::new(create_memtable()?))?; +let dataframe = ctx + .table("users") + .filter(col("a").lt_eq(col("b")))? + .sort(vec![col("a").sort(true, true), col("b").sort(false, false)])?; +``` + +## Collect / Streaming Exec + +DataFusion `DataFrame`s are "lazy", meaning they do not do any processing until they are executed, which allows for additional optimizations. + +When you have a `DataFrame`, you can run it in one of three ways: + +1. `collect` which executes the query and buffers all the output into a `Vec` +2. `streaming_exec`, which begins executions and returns a `SendableRecordBatchStream` which incrementally computes output on each call to `next()` +3. `cache` which executes the query and buffers the output into a new in memory DataFrame. + +You can just collect all outputs once like: + +```rust +let ctx = SessionContext::new(); +let df = ctx.read_csv("tests/data/example.csv", CsvReadOptions::new()).await?; +let batches = df.collect().await?; +``` + +You can also use stream output to incrementally generate output one `RecordBatch` at a time + +```rust +let ctx = SessionContext::new(); +let df = ctx.read_csv("tests/data/example.csv", CsvReadOptions::new()).await?; +let mut stream = df.execute_stream().await?; +while let Some(rb) = stream.next().await { + println!("{rb:?}"); +} +``` + +# Write DataFrame to Files + +You can also serialize `DataFrame` to a file. For now, `Datafusion` supports write `DataFrame` to `csv`, `json` and `parquet`. + +When writing a file, DataFusion will execute the DataFrame and stream the results to a file. + +For example, to write a csv_file + +```rust +let ctx = SessionContext::new(); +// Register the in-memory table containing the data +ctx.register_table("users", Arc::new(mem_table))?; +let dataframe = ctx.sql("SELECT * FROM users;").await?; + +dataframe + .write_csv("user_dataframe.csv", DataFrameWriteOptions::default(), None) + .await; +``` + +and the file will look like (Example Output): + +``` +id,bank_account +1,9000 +``` + +## Transform between LogicalPlan and DataFrame + +As shown above, `DataFrame` is just a very thin wrapper of `LogicalPlan`, so you can easily go back and forth between them. + +```rust +// Just combine LogicalPlan with SessionContext and you get a DataFrame +let ctx = SessionContext::new(); +// Register the in-memory table containing the data +ctx.register_table("users", Arc::new(mem_table))?; +let dataframe = ctx.sql("SELECT * FROM users;").await?; + +// get LogicalPlan in dataframe +let plan = dataframe.logical_plan().clone(); + +// construct a DataFrame with LogicalPlan +let new_df = DataFrame::new(ctx.state(), plan); +``` From 975b012c7fb3fda0c0d1a55c71b923803229d348 Mon Sep 17 00:00:00 2001 From: Xiaofeng Zhang Date: Tue, 28 Nov 2023 23:00:17 +0800 Subject: [PATCH 322/572] Port tests in limit.rs to sqllogictest (#8315) * Port tests in limit.rs to sqllogictest * Minor: Add rationale comments and an explain to limit.slt * Fix clippy --------- Co-authored-by: xiaohan.zxf Co-authored-by: Andrew Lamb --- datafusion/core/tests/sql/limit.rs | 101 ---------------- datafusion/core/tests/sql/mod.rs | 13 -- datafusion/sqllogictest/test_files/limit.slt | 119 +++++++++++++++++++ 3 files changed, 119 insertions(+), 114 deletions(-) delete mode 100644 datafusion/core/tests/sql/limit.rs diff --git a/datafusion/core/tests/sql/limit.rs b/datafusion/core/tests/sql/limit.rs deleted file mode 100644 index 1c8ea4fd3468..000000000000 --- a/datafusion/core/tests/sql/limit.rs +++ /dev/null @@ -1,101 +0,0 @@ -// Licensed to the Apache Software Foundation (ASF) under one -// or more contributor license agreements. See the NOTICE file -// distributed with this work for additional information -// regarding copyright ownership. The ASF licenses this file -// to you under the Apache License, Version 2.0 (the -// "License"); you may not use this file except in compliance -// with the License. You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, -// software distributed under the License is distributed on an -// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -// KIND, either express or implied. See the License for the -// specific language governing permissions and limitations -// under the License. - -use super::*; - -#[tokio::test] -async fn limit() -> Result<()> { - let tmp_dir = TempDir::new()?; - let ctx = create_ctx_with_partition(&tmp_dir, 1).await?; - ctx.register_table("t", table_with_sequence(1, 1000).unwrap()) - .unwrap(); - - let results = plan_and_collect(&ctx, "SELECT i FROM t ORDER BY i DESC limit 3") - .await - .unwrap(); - - #[rustfmt::skip] - let expected = ["+------+", - "| i |", - "+------+", - "| 1000 |", - "| 999 |", - "| 998 |", - "+------+"]; - - assert_batches_eq!(expected, &results); - - let results = plan_and_collect(&ctx, "SELECT i FROM t ORDER BY i limit 3") - .await - .unwrap(); - - #[rustfmt::skip] - let expected = ["+---+", - "| i |", - "+---+", - "| 1 |", - "| 2 |", - "| 3 |", - "+---+"]; - - assert_batches_eq!(expected, &results); - - let results = plan_and_collect(&ctx, "SELECT i FROM t limit 3") - .await - .unwrap(); - - // the actual rows are not guaranteed, so only check the count (should be 3) - let num_rows: usize = results.into_iter().map(|b| b.num_rows()).sum(); - assert_eq!(num_rows, 3); - - Ok(()) -} - -#[tokio::test] -async fn limit_multi_partitions() -> Result<()> { - let tmp_dir = TempDir::new()?; - let ctx = create_ctx_with_partition(&tmp_dir, 1).await?; - - let partitions = vec![ - vec![make_partition(0)], - vec![make_partition(1)], - vec![make_partition(2)], - vec![make_partition(3)], - vec![make_partition(4)], - vec![make_partition(5)], - ]; - let schema = partitions[0][0].schema(); - let provider = Arc::new(MemTable::try_new(schema, partitions).unwrap()); - - ctx.register_table("t", provider).unwrap(); - - // select all rows - let results = plan_and_collect(&ctx, "SELECT i FROM t").await.unwrap(); - - let num_rows: usize = results.into_iter().map(|b| b.num_rows()).sum(); - assert_eq!(num_rows, 15); - - for limit in 1..10 { - let query = format!("SELECT i FROM t limit {limit}"); - let results = plan_and_collect(&ctx, &query).await.unwrap(); - - let num_rows: usize = results.into_iter().map(|b| b.num_rows()).sum(); - assert_eq!(num_rows, limit, "mismatch with query {query}"); - } - - Ok(()) -} diff --git a/datafusion/core/tests/sql/mod.rs b/datafusion/core/tests/sql/mod.rs index 6d783a503184..47de6ec857da 100644 --- a/datafusion/core/tests/sql/mod.rs +++ b/datafusion/core/tests/sql/mod.rs @@ -79,7 +79,6 @@ pub mod explain_analyze; pub mod expr; pub mod group_by; pub mod joins; -pub mod limit; pub mod order; pub mod parquet; pub mod parquet_schema; @@ -546,18 +545,6 @@ fn populate_csv_partitions( Ok(schema) } -/// Return a RecordBatch with a single Int32 array with values (0..sz) -pub fn make_partition(sz: i32) -> RecordBatch { - let seq_start = 0; - let seq_end = sz; - let values = (seq_start..seq_end).collect::>(); - let schema = Arc::new(Schema::new(vec![Field::new("i", DataType::Int32, true)])); - let arr = Arc::new(Int32Array::from(values)); - let arr = arr as ArrayRef; - - RecordBatch::try_new(schema, vec![arr]).unwrap() -} - /// Specialised String representation fn col_str(column: &ArrayRef, row_index: usize) -> String { // NullArray::is_null() does not work on NullArray. diff --git a/datafusion/sqllogictest/test_files/limit.slt b/datafusion/sqllogictest/test_files/limit.slt index 21248ddbd8d7..9e093336a15d 100644 --- a/datafusion/sqllogictest/test_files/limit.slt +++ b/datafusion/sqllogictest/test_files/limit.slt @@ -379,6 +379,125 @@ SELECT COUNT(*) FROM (SELECT a FROM t1 WHERE a > 3 LIMIT 3 OFFSET 6); ---- 1 +# generate BIGINT data from 1 to 1000 in multiple partitions +statement ok +CREATE TABLE t1000 (i BIGINT) AS +WITH t AS (VALUES (0), (0), (0), (0), (0), (0), (0), (0), (0), (0)) +SELECT ROW_NUMBER() OVER (PARTITION BY t1.column1) FROM t t1, t t2, t t3; + +# verify that there are multiple partitions in the input (i.e. MemoryExec says +# there are 4 partitions) so that this tests multi-partition limit. +query TT +EXPLAIN SELECT DISTINCT i FROM t1000; +---- +logical_plan +Aggregate: groupBy=[[t1000.i]], aggr=[[]] +--TableScan: t1000 projection=[i] +physical_plan +AggregateExec: mode=FinalPartitioned, gby=[i@0 as i], aggr=[] +--CoalesceBatchesExec: target_batch_size=8192 +----RepartitionExec: partitioning=Hash([i@0], 4), input_partitions=4 +------AggregateExec: mode=Partial, gby=[i@0 as i], aggr=[] +--------MemoryExec: partitions=4, partition_sizes=[1, 1, 2, 1] + +query I +SELECT i FROM t1000 ORDER BY i DESC LIMIT 3; +---- +1000 +999 +998 + +query I +SELECT i FROM t1000 ORDER BY i LIMIT 3; +---- +1 +2 +3 + +query I +SELECT COUNT(*) FROM (SELECT i FROM t1000 LIMIT 3); +---- +3 + +# limit_multi_partitions +statement ok +CREATE TABLE t15 (i BIGINT); + +query I +INSERT INTO t15 VALUES (1); +---- +1 + +query I +INSERT INTO t15 VALUES (1), (2); +---- +2 + +query I +INSERT INTO t15 VALUES (1), (2), (3); +---- +3 + +query I +INSERT INTO t15 VALUES (1), (2), (3), (4); +---- +4 + +query I +INSERT INTO t15 VALUES (1), (2), (3), (4), (5); +---- +5 + +query I +SELECT COUNT(*) FROM t15; +---- +15 + +query I +SELECT COUNT(*) FROM (SELECT i FROM t15 LIMIT 1); +---- +1 + +query I +SELECT COUNT(*) FROM (SELECT i FROM t15 LIMIT 2); +---- +2 + +query I +SELECT COUNT(*) FROM (SELECT i FROM t15 LIMIT 3); +---- +3 + +query I +SELECT COUNT(*) FROM (SELECT i FROM t15 LIMIT 4); +---- +4 + +query I +SELECT COUNT(*) FROM (SELECT i FROM t15 LIMIT 5); +---- +5 + +query I +SELECT COUNT(*) FROM (SELECT i FROM t15 LIMIT 6); +---- +6 + +query I +SELECT COUNT(*) FROM (SELECT i FROM t15 LIMIT 7); +---- +7 + +query I +SELECT COUNT(*) FROM (SELECT i FROM t15 LIMIT 8); +---- +8 + +query I +SELECT COUNT(*) FROM (SELECT i FROM t15 LIMIT 9); +---- +9 + ######## # Clean up after the test ######## From 2a692446f46ef96f48eb9ba19231e9576be9ff5a Mon Sep 17 00:00:00 2001 From: Tan Wei Date: Tue, 28 Nov 2023 23:02:02 +0800 Subject: [PATCH 323/572] move array function unit_tests to sqllogictest (#8332) * move array function unit_tests to sqllogictest Signed-off-by: veeupup * add comment for array_expression internal test --------- Signed-off-by: veeupup --- .../physical-expr/src/array_expressions.rs | 1040 +---------------- datafusion/sqllogictest/test_files/array.slt | 20 +- 2 files changed, 14 insertions(+), 1046 deletions(-) diff --git a/datafusion/physical-expr/src/array_expressions.rs b/datafusion/physical-expr/src/array_expressions.rs index 6b7bef8e6a36..e6543808b97a 100644 --- a/datafusion/physical-expr/src/array_expressions.rs +++ b/datafusion/physical-expr/src/array_expressions.rs @@ -2001,8 +2001,8 @@ pub fn array_intersect(args: &[ArrayRef]) -> Result { mod tests { use super::*; use arrow::datatypes::Int64Type; - use datafusion_common::cast::as_uint64_array; + /// Only test internal functions, array-related sql functions will be tested in sqllogictest `array.slt` #[test] fn test_align_array_dimensions() { let array1d_1 = @@ -2044,980 +2044,6 @@ mod tests { ); } - #[test] - fn test_array() { - // make_array(1, 2, 3) = [1, 2, 3] - let args = [ - Arc::new(Int64Array::from(vec![1])) as ArrayRef, - Arc::new(Int64Array::from(vec![2])), - Arc::new(Int64Array::from(vec![3])), - ]; - let array = make_array(&args).expect("failed to initialize function array"); - let result = as_list_array(&array).expect("failed to initialize function array"); - assert_eq!(result.len(), 1); - assert_eq!( - &[1, 2, 3], - as_int64_array(&result.value(0)) - .expect("failed to cast to primitive array") - .values() - ) - } - - #[test] - fn test_nested_array() { - // make_array([1, 3, 5], [2, 4, 6]) = [[1, 3, 5], [2, 4, 6]] - let args = [ - Arc::new(Int64Array::from(vec![1, 2])) as ArrayRef, - Arc::new(Int64Array::from(vec![3, 4])), - Arc::new(Int64Array::from(vec![5, 6])), - ]; - let array = make_array(&args).expect("failed to initialize function array"); - let result = as_list_array(&array).expect("failed to initialize function array"); - assert_eq!(result.len(), 2); - assert_eq!( - &[1, 3, 5], - as_int64_array(&result.value(0)) - .expect("failed to cast to primitive array") - .values() - ); - assert_eq!( - &[2, 4, 6], - as_int64_array(&result.value(1)) - .expect("failed to cast to primitive array") - .values() - ); - } - - #[test] - fn test_array_element() { - // array_element([1, 2, 3, 4], 1) = 1 - let list_array = return_array(); - let arr = array_element(&[list_array, Arc::new(Int64Array::from_value(1, 1))]) - .expect("failed to initialize function array_element"); - let result = - as_int64_array(&arr).expect("failed to initialize function array_element"); - - assert_eq!(result, &Int64Array::from_value(1, 1)); - - // array_element([1, 2, 3, 4], 3) = 3 - let list_array = return_array(); - let arr = array_element(&[list_array, Arc::new(Int64Array::from_value(3, 1))]) - .expect("failed to initialize function array_element"); - let result = - as_int64_array(&arr).expect("failed to initialize function array_element"); - - assert_eq!(result, &Int64Array::from_value(3, 1)); - - // array_element([1, 2, 3, 4], 0) = NULL - let list_array = return_array(); - let arr = array_element(&[list_array, Arc::new(Int64Array::from_value(0, 1))]) - .expect("failed to initialize function array_element"); - let result = - as_int64_array(&arr).expect("failed to initialize function array_element"); - - assert_eq!(result, &Int64Array::from(vec![None])); - - // array_element([1, 2, 3, 4], NULL) = NULL - let list_array = return_array(); - let arr = array_element(&[list_array, Arc::new(Int64Array::from(vec![None]))]) - .expect("failed to initialize function array_element"); - let result = - as_int64_array(&arr).expect("failed to initialize function array_element"); - - assert_eq!(result, &Int64Array::from(vec![None])); - - // array_element([1, 2, 3, 4], -1) = 4 - let list_array = return_array(); - let arr = array_element(&[list_array, Arc::new(Int64Array::from_value(-1, 1))]) - .expect("failed to initialize function array_element"); - let result = - as_int64_array(&arr).expect("failed to initialize function array_element"); - - assert_eq!(result, &Int64Array::from_value(4, 1)); - - // array_element([1, 2, 3, 4], -3) = 2 - let list_array = return_array(); - let arr = array_element(&[list_array, Arc::new(Int64Array::from_value(-3, 1))]) - .expect("failed to initialize function array_element"); - let result = - as_int64_array(&arr).expect("failed to initialize function array_element"); - - assert_eq!(result, &Int64Array::from_value(2, 1)); - - // array_element([1, 2, 3, 4], 10) = NULL - let list_array = return_array(); - let arr = array_element(&[list_array, Arc::new(Int64Array::from_value(10, 1))]) - .expect("failed to initialize function array_element"); - let result = - as_int64_array(&arr).expect("failed to initialize function array_element"); - - assert_eq!(result, &Int64Array::from(vec![None])); - } - - #[test] - fn test_nested_array_element() { - // array_element([[1, 2, 3, 4], [5, 6, 7, 8]], 2) = [5, 6, 7, 8] - let list_array = return_nested_array(); - let arr = array_element(&[list_array, Arc::new(Int64Array::from_value(2, 1))]) - .expect("failed to initialize function array_element"); - let result = - as_list_array(&arr).expect("failed to initialize function array_element"); - - assert_eq!( - &[5, 6, 7, 8], - result - .value(0) - .as_any() - .downcast_ref::() - .unwrap() - .values() - ); - } - - #[test] - fn test_array_pop_back() { - // array_pop_back([1, 2, 3, 4]) = [1, 2, 3] - let list_array = return_array(); - let arr = array_pop_back(&[list_array]) - .expect("failed to initialize function array_pop_back"); - let result = - as_list_array(&arr).expect("failed to initialize function array_pop_back"); - assert_eq!( - &[1, 2, 3], - result - .value(0) - .as_any() - .downcast_ref::() - .unwrap() - .values() - ); - - // array_pop_back([1, 2, 3]) = [1, 2] - let list_array = Arc::new(result.clone()); - let arr = array_pop_back(&[list_array]) - .expect("failed to initialize function array_pop_back"); - let result = - as_list_array(&arr).expect("failed to initialize function array_pop_back"); - assert_eq!( - &[1, 2], - result - .value(0) - .as_any() - .downcast_ref::() - .unwrap() - .values() - ); - - // array_pop_back([1, 2]) = [1] - let list_array = Arc::new(result.clone()); - let arr = array_pop_back(&[list_array]) - .expect("failed to initialize function array_pop_back"); - let result = - as_list_array(&arr).expect("failed to initialize function array_pop_back"); - assert_eq!( - &[1], - result - .value(0) - .as_any() - .downcast_ref::() - .unwrap() - .values() - ); - - // array_pop_back([1]) = [] - let list_array = Arc::new(result.clone()); - let arr = array_pop_back(&[list_array]) - .expect("failed to initialize function array_pop_back"); - let result = - as_list_array(&arr).expect("failed to initialize function array_pop_back"); - assert_eq!( - &[], - result - .value(0) - .as_any() - .downcast_ref::() - .unwrap() - .values() - ); - // array_pop_back([]) = [] - let list_array = Arc::new(result.clone()); - let arr = array_pop_back(&[list_array]) - .expect("failed to initialize function array_pop_back"); - let result = - as_list_array(&arr).expect("failed to initialize function array_pop_back"); - assert_eq!( - &[], - result - .value(0) - .as_any() - .downcast_ref::() - .unwrap() - .values() - ); - - // array_pop_back([1, NULL, 3, NULL]) = [1, NULL, 3] - let list_array = return_array_with_nulls(); - let arr = array_pop_back(&[list_array]) - .expect("failed to initialize function array_pop_back"); - let result = - as_list_array(&arr).expect("failed to initialize function array_pop_back"); - assert_eq!(3, result.values().len()); - assert_eq!( - &[false, true, false], - &[ - result.values().is_null(0), - result.values().is_null(1), - result.values().is_null(2) - ] - ); - } - #[test] - fn test_nested_array_pop_back() { - // array_pop_back([[1, 2, 3, 4], [5, 6, 7, 8]]) = [[1, 2, 3, 4]] - let list_array = return_nested_array(); - let arr = array_pop_back(&[list_array]) - .expect("failed to initialize function array_slice"); - let result = - as_list_array(&arr).expect("failed to initialize function array_slice"); - assert_eq!( - &[1, 2, 3, 4], - result - .value(0) - .as_any() - .downcast_ref::() - .unwrap() - .value(0) - .as_any() - .downcast_ref::() - .unwrap() - .values() - ); - - // array_pop_back([[1, 2, 3, 4]]) = [] - let list_array = Arc::new(result.clone()); - let arr = array_pop_back(&[list_array]) - .expect("failed to initialize function array_pop_back"); - let result = - as_list_array(&arr).expect("failed to initialize function array_pop_back"); - assert!(result - .value(0) - .as_any() - .downcast_ref::() - .unwrap() - .is_empty()); - // array_pop_back([]) = [] - let list_array = Arc::new(result.clone()); - let arr = array_pop_back(&[list_array]) - .expect("failed to initialize function array_pop_back"); - let result = - as_list_array(&arr).expect("failed to initialize function array_pop_back"); - assert!(result - .value(0) - .as_any() - .downcast_ref::() - .unwrap() - .is_empty()); - } - - #[test] - fn test_array_slice() { - // array_slice([1, 2, 3, 4], 1, 3) = [1, 2, 3] - let list_array = return_array(); - let arr = array_slice(&[ - list_array, - Arc::new(Int64Array::from_value(1, 1)), - Arc::new(Int64Array::from_value(3, 1)), - ]) - .expect("failed to initialize function array_slice"); - let result = - as_list_array(&arr).expect("failed to initialize function array_slice"); - - assert_eq!( - &[1, 2, 3], - result - .value(0) - .as_any() - .downcast_ref::() - .unwrap() - .values() - ); - - // array_slice([1, 2, 3, 4], 2, 2) = [2] - let list_array = return_array(); - let arr = array_slice(&[ - list_array, - Arc::new(Int64Array::from_value(2, 1)), - Arc::new(Int64Array::from_value(2, 1)), - ]) - .expect("failed to initialize function array_slice"); - let result = - as_list_array(&arr).expect("failed to initialize function array_slice"); - - assert_eq!( - &[2], - result - .value(0) - .as_any() - .downcast_ref::() - .unwrap() - .values() - ); - - // array_slice([1, 2, 3, 4], 0, 0) = [] - let list_array = return_array(); - let arr = array_slice(&[ - list_array, - Arc::new(Int64Array::from_value(0, 1)), - Arc::new(Int64Array::from_value(0, 1)), - ]) - .expect("failed to initialize function array_slice"); - let result = - as_list_array(&arr).expect("failed to initialize function array_slice"); - - assert!(result - .value(0) - .as_any() - .downcast_ref::() - .unwrap() - .is_empty()); - - // array_slice([1, 2, 3, 4], 0, 6) = [1, 2, 3, 4] - let list_array = return_array(); - let arr = array_slice(&[ - list_array, - Arc::new(Int64Array::from_value(0, 1)), - Arc::new(Int64Array::from_value(6, 1)), - ]) - .expect("failed to initialize function array_slice"); - let result = - as_list_array(&arr).expect("failed to initialize function array_slice"); - - assert_eq!( - &[1, 2, 3, 4], - result - .value(0) - .as_any() - .downcast_ref::() - .unwrap() - .values() - ); - - // array_slice([1, 2, 3, 4], -2, -2) = [] - let list_array = return_array(); - let arr = array_slice(&[ - list_array, - Arc::new(Int64Array::from_value(-2, 1)), - Arc::new(Int64Array::from_value(-2, 1)), - ]) - .expect("failed to initialize function array_slice"); - let result = - as_list_array(&arr).expect("failed to initialize function array_slice"); - - assert!(result - .value(0) - .as_any() - .downcast_ref::() - .unwrap() - .is_empty()); - - // array_slice([1, 2, 3, 4], -3, -1) = [2, 3] - let list_array = return_array(); - let arr = array_slice(&[ - list_array, - Arc::new(Int64Array::from_value(-3, 1)), - Arc::new(Int64Array::from_value(-1, 1)), - ]) - .expect("failed to initialize function array_slice"); - let result = - as_list_array(&arr).expect("failed to initialize function array_slice"); - - assert_eq!( - &[2, 3], - result - .value(0) - .as_any() - .downcast_ref::() - .unwrap() - .values() - ); - - // array_slice([1, 2, 3, 4], -3, 2) = [2] - let list_array = return_array(); - let arr = array_slice(&[ - list_array, - Arc::new(Int64Array::from_value(-3, 1)), - Arc::new(Int64Array::from_value(2, 1)), - ]) - .expect("failed to initialize function array_slice"); - let result = - as_list_array(&arr).expect("failed to initialize function array_slice"); - - assert_eq!( - &[2], - result - .value(0) - .as_any() - .downcast_ref::() - .unwrap() - .values() - ); - - // array_slice([1, 2, 3, 4], 2, 11) = [2, 3, 4] - let list_array = return_array(); - let arr = array_slice(&[ - list_array, - Arc::new(Int64Array::from_value(2, 1)), - Arc::new(Int64Array::from_value(11, 1)), - ]) - .expect("failed to initialize function array_slice"); - let result = - as_list_array(&arr).expect("failed to initialize function array_slice"); - - assert_eq!( - &[2, 3, 4], - result - .value(0) - .as_any() - .downcast_ref::() - .unwrap() - .values() - ); - - // array_slice([1, 2, 3, 4], 3, 1) = [] - let list_array = return_array(); - let arr = array_slice(&[ - list_array, - Arc::new(Int64Array::from_value(3, 1)), - Arc::new(Int64Array::from_value(1, 1)), - ]) - .expect("failed to initialize function array_slice"); - let result = - as_list_array(&arr).expect("failed to initialize function array_slice"); - - assert!(result - .value(0) - .as_any() - .downcast_ref::() - .unwrap() - .is_empty()); - - // array_slice([1, 2, 3, 4], -7, -2) = NULL - let list_array = return_array(); - let arr = array_slice(&[ - list_array, - Arc::new(Int64Array::from_value(-7, 1)), - Arc::new(Int64Array::from_value(-2, 1)), - ]) - .expect("failed to initialize function array_slice"); - let result = - as_list_array(&arr).expect("failed to initialize function array_slice"); - - assert!(result - .value(0) - .as_any() - .downcast_ref::() - .unwrap() - .is_null(0)); - } - - #[test] - fn test_array_range() { - // range(1, 5, 1) = [1, 2, 3, 4] - let args1 = Arc::new(Int64Array::from(vec![Some(1)])) as ArrayRef; - let args2 = Arc::new(Int64Array::from(vec![Some(5)])) as ArrayRef; - let args3 = Arc::new(Int64Array::from(vec![Some(1)])) as ArrayRef; - let arr = gen_range(&[args1, args2, args3]).unwrap(); - - let result = as_list_array(&arr).expect("failed to initialize function range"); - assert_eq!( - &[1, 2, 3, 4], - result - .value(0) - .as_any() - .downcast_ref::() - .unwrap() - .values() - ); - - // range(1, -5, -1) = [1, 0, -1, -2, -3, -4] - let args1 = Arc::new(Int64Array::from(vec![Some(1)])) as ArrayRef; - let args2 = Arc::new(Int64Array::from(vec![Some(-5)])) as ArrayRef; - let args3 = Arc::new(Int64Array::from(vec![Some(-1)])) as ArrayRef; - let arr = gen_range(&[args1, args2, args3]).unwrap(); - - let result = as_list_array(&arr).expect("failed to initialize function range"); - assert_eq!( - &[1, 0, -1, -2, -3, -4], - result - .value(0) - .as_any() - .downcast_ref::() - .unwrap() - .values() - ); - - // range(1, 5, -1) = [] - let args1 = Arc::new(Int64Array::from(vec![Some(1)])) as ArrayRef; - let args2 = Arc::new(Int64Array::from(vec![Some(5)])) as ArrayRef; - let args3 = Arc::new(Int64Array::from(vec![Some(-1)])) as ArrayRef; - let arr = gen_range(&[args1, args2, args3]).unwrap(); - - let result = as_list_array(&arr).expect("failed to initialize function range"); - assert_eq!( - &[], - result - .value(0) - .as_any() - .downcast_ref::() - .unwrap() - .values() - ); - - // range(1, 5, 0) = [] - let args1 = Arc::new(Int64Array::from(vec![Some(1)])) as ArrayRef; - let args2 = Arc::new(Int64Array::from(vec![Some(5)])) as ArrayRef; - let args3 = Arc::new(Int64Array::from(vec![Some(0)])) as ArrayRef; - let is_err = gen_range(&[args1, args2, args3]).is_err(); - assert!(is_err) - } - - #[test] - fn test_nested_array_slice() { - // array_slice([[1, 2, 3, 4], [5, 6, 7, 8]], 1, 1) = [[1, 2, 3, 4]] - let list_array = return_nested_array(); - let arr = array_slice(&[ - list_array, - Arc::new(Int64Array::from_value(1, 1)), - Arc::new(Int64Array::from_value(1, 1)), - ]) - .expect("failed to initialize function array_slice"); - let result = - as_list_array(&arr).expect("failed to initialize function array_slice"); - - assert_eq!( - &[1, 2, 3, 4], - result - .value(0) - .as_any() - .downcast_ref::() - .unwrap() - .value(0) - .as_any() - .downcast_ref::() - .unwrap() - .values() - ); - - // array_slice([[1, 2, 3, 4], [5, 6, 7, 8]], -1, -1) = [] - let list_array = return_nested_array(); - let arr = array_slice(&[ - list_array, - Arc::new(Int64Array::from_value(-1, 1)), - Arc::new(Int64Array::from_value(-1, 1)), - ]) - .expect("failed to initialize function array_slice"); - let result = - as_list_array(&arr).expect("failed to initialize function array_slice"); - - assert!(result - .value(0) - .as_any() - .downcast_ref::() - .unwrap() - .is_empty()); - - // array_slice([[1, 2, 3, 4], [5, 6, 7, 8]], -1, 2) = [[5, 6, 7, 8]] - let list_array = return_nested_array(); - let arr = array_slice(&[ - list_array, - Arc::new(Int64Array::from_value(-1, 1)), - Arc::new(Int64Array::from_value(2, 1)), - ]) - .expect("failed to initialize function array_slice"); - let result = - as_list_array(&arr).expect("failed to initialize function array_slice"); - - assert_eq!( - &[5, 6, 7, 8], - result - .value(0) - .as_any() - .downcast_ref::() - .unwrap() - .value(0) - .as_any() - .downcast_ref::() - .unwrap() - .values() - ); - } - - #[test] - fn test_array_append() { - // array_append([1, 2, 3], 4) = [1, 2, 3, 4] - let data = vec![Some(vec![Some(1), Some(2), Some(3)])]; - let list_array = - Arc::new(ListArray::from_iter_primitive::(data)) as ArrayRef; - let int64_array = Arc::new(Int64Array::from(vec![Some(4)])) as ArrayRef; - - let args = [list_array, int64_array]; - - let array = - array_append(&args).expect("failed to initialize function array_append"); - let result = - as_list_array(&array).expect("failed to initialize function array_append"); - - assert_eq!( - &[1, 2, 3, 4], - result - .value(0) - .as_any() - .downcast_ref::() - .unwrap() - .values() - ); - } - - #[test] - fn test_array_prepend() { - // array_prepend(1, [2, 3, 4]) = [1, 2, 3, 4] - let data = vec![Some(vec![Some(2), Some(3), Some(4)])]; - let list_array = - Arc::new(ListArray::from_iter_primitive::(data)) as ArrayRef; - let int64_array = Arc::new(Int64Array::from(vec![Some(1)])) as ArrayRef; - - let args = [int64_array, list_array]; - - let array = - array_prepend(&args).expect("failed to initialize function array_append"); - let result = - as_list_array(&array).expect("failed to initialize function array_append"); - - assert_eq!( - &[1, 2, 3, 4], - result - .value(0) - .as_any() - .downcast_ref::() - .unwrap() - .values() - ); - } - - #[test] - fn test_array_concat() { - // array_concat([1, 2, 3], [4, 5, 6], [7, 8, 9]) = [1, 2, 3, 4, 5, 6, 7, 8, 9] - let data = vec![Some(vec![Some(1), Some(2), Some(3)])]; - let list_array1 = - Arc::new(ListArray::from_iter_primitive::(data)) as ArrayRef; - let data = vec![Some(vec![Some(4), Some(5), Some(6)])]; - let list_array2 = - Arc::new(ListArray::from_iter_primitive::(data)) as ArrayRef; - let data = vec![Some(vec![Some(7), Some(8), Some(9)])]; - let list_array3 = - Arc::new(ListArray::from_iter_primitive::(data)) as ArrayRef; - - let args = [list_array1, list_array2, list_array3]; - - let array = - array_concat(&args).expect("failed to initialize function array_concat"); - let result = - as_list_array(&array).expect("failed to initialize function array_concat"); - - assert_eq!( - &[1, 2, 3, 4, 5, 6, 7, 8, 9], - result - .value(0) - .as_any() - .downcast_ref::() - .unwrap() - .values() - ); - } - - #[test] - fn test_nested_array_concat() { - // array_concat([1, 2, 3, 4], [1, 2, 3, 4]) = [1, 2, 3, 4, 1, 2, 3, 4] - let list_array = return_array(); - let arr = array_concat(&[list_array.clone(), list_array.clone()]) - .expect("failed to initialize function array_concat"); - let result = - as_list_array(&arr).expect("failed to initialize function array_concat"); - - assert_eq!( - &[1, 2, 3, 4, 1, 2, 3, 4], - result - .value(0) - .as_any() - .downcast_ref::() - .unwrap() - .values() - ); - - // array_concat([[1, 2, 3, 4], [5, 6, 7, 8]], [1, 2, 3, 4]) = [[1, 2, 3, 4], [5, 6, 7, 8], [1, 2, 3, 4]] - let list_nested_array = return_nested_array(); - let list_array = return_array(); - let arr = array_concat(&[list_nested_array, list_array]) - .expect("failed to initialize function array_concat"); - let result = - as_list_array(&arr).expect("failed to initialize function array_concat"); - - assert_eq!( - &[1, 2, 3, 4], - result - .value(0) - .as_any() - .downcast_ref::() - .unwrap() - .value(2) - .as_any() - .downcast_ref::() - .unwrap() - .values() - ); - } - - #[test] - fn test_array_position() { - // array_position([1, 2, 3, 4], 3) = 3 - let list_array = return_array(); - let array = array_position(&[list_array, Arc::new(Int64Array::from_value(3, 1))]) - .expect("failed to initialize function array_position"); - let result = as_uint64_array(&array) - .expect("failed to initialize function array_position"); - - assert_eq!(result, &UInt64Array::from(vec![3])); - } - - #[test] - fn test_array_positions() { - // array_positions([1, 2, 3, 4], 3) = [3] - let list_array = return_array(); - let array = - array_positions(&[list_array, Arc::new(Int64Array::from_value(3, 1))]) - .expect("failed to initialize function array_position"); - let result = - as_list_array(&array).expect("failed to initialize function array_position"); - - assert_eq!(result.len(), 1); - assert_eq!( - &[3], - result - .value(0) - .as_any() - .downcast_ref::() - .unwrap() - .values() - ); - } - - #[test] - fn test_array_to_string() { - // array_to_string([1, 2, 3, 4], ',') = 1,2,3,4 - let list_array = return_array(); - let array = - array_to_string(&[list_array, Arc::new(StringArray::from(vec![Some(",")]))]) - .expect("failed to initialize function array_to_string"); - let result = as_string_array(&array) - .expect("failed to initialize function array_to_string"); - - assert_eq!(result.len(), 1); - assert_eq!("1,2,3,4", result.value(0)); - - // array_to_string([1, NULL, 3, NULL], ',', '*') = 1,*,3,* - let list_array = return_array_with_nulls(); - let array = array_to_string(&[ - list_array, - Arc::new(StringArray::from(vec![Some(",")])), - Arc::new(StringArray::from(vec![Some("*")])), - ]) - .expect("failed to initialize function array_to_string"); - let result = as_string_array(&array) - .expect("failed to initialize function array_to_string"); - - assert_eq!(result.len(), 1); - assert_eq!("1,*,3,*", result.value(0)); - } - - #[test] - fn test_nested_array_to_string() { - // array_to_string([[1, 2, 3, 4], [5, 6, 7, 8]], '-') = 1-2-3-4-5-6-7-8 - let list_array = return_nested_array(); - let array = - array_to_string(&[list_array, Arc::new(StringArray::from(vec![Some("-")]))]) - .expect("failed to initialize function array_to_string"); - let result = as_string_array(&array) - .expect("failed to initialize function array_to_string"); - - assert_eq!(result.len(), 1); - assert_eq!("1-2-3-4-5-6-7-8", result.value(0)); - - // array_to_string([[1, NULL, 3, NULL], [NULL, 6, 7, NULL]], '-', '*') = 1-*-3-*-*-6-7-* - let list_array = return_nested_array_with_nulls(); - let array = array_to_string(&[ - list_array, - Arc::new(StringArray::from(vec![Some("-")])), - Arc::new(StringArray::from(vec![Some("*")])), - ]) - .expect("failed to initialize function array_to_string"); - let result = as_string_array(&array) - .expect("failed to initialize function array_to_string"); - - assert_eq!(result.len(), 1); - assert_eq!("1-*-3-*-*-6-7-*", result.value(0)); - } - - #[test] - fn test_cardinality() { - // cardinality([1, 2, 3, 4]) = 4 - let list_array = return_array(); - let arr = cardinality(&[list_array]) - .expect("failed to initialize function cardinality"); - let result = - as_uint64_array(&arr).expect("failed to initialize function cardinality"); - - assert_eq!(result, &UInt64Array::from(vec![4])); - } - - #[test] - fn test_nested_cardinality() { - // cardinality([[1, 2, 3, 4], [5, 6, 7, 8]]) = 8 - let list_array = return_nested_array(); - let arr = cardinality(&[list_array]) - .expect("failed to initialize function cardinality"); - let result = - as_uint64_array(&arr).expect("failed to initialize function cardinality"); - - assert_eq!(result, &UInt64Array::from(vec![8])); - } - - #[test] - fn test_array_length() { - // array_length([1, 2, 3, 4]) = 4 - let list_array = return_array(); - let arr = array_length(&[list_array.clone()]) - .expect("failed to initialize function array_ndims"); - let result = - as_uint64_array(&arr).expect("failed to initialize function array_ndims"); - - assert_eq!(result, &UInt64Array::from_value(4, 1)); - - // array_length([1, 2, 3, 4], 1) = 4 - let array = array_length(&[list_array, Arc::new(Int64Array::from_value(1, 1))]) - .expect("failed to initialize function array_ndims"); - let result = - as_uint64_array(&array).expect("failed to initialize function array_ndims"); - - assert_eq!(result, &UInt64Array::from_value(4, 1)); - } - - #[test] - fn test_nested_array_length() { - let list_array = return_nested_array(); - - // array_length([[1, 2, 3, 4], [5, 6, 7, 8]]) = 2 - let arr = array_length(&[list_array.clone()]) - .expect("failed to initialize function array_length"); - let result = - as_uint64_array(&arr).expect("failed to initialize function array_length"); - - assert_eq!(result, &UInt64Array::from_value(2, 1)); - - // array_length([[1, 2, 3, 4], [5, 6, 7, 8]], 1) = 2 - let arr = - array_length(&[list_array.clone(), Arc::new(Int64Array::from_value(1, 1))]) - .expect("failed to initialize function array_length"); - let result = - as_uint64_array(&arr).expect("failed to initialize function array_length"); - - assert_eq!(result, &UInt64Array::from_value(2, 1)); - - // array_length([[1, 2, 3, 4], [5, 6, 7, 8]], 2) = 4 - let arr = - array_length(&[list_array.clone(), Arc::new(Int64Array::from_value(2, 1))]) - .expect("failed to initialize function array_length"); - let result = - as_uint64_array(&arr).expect("failed to initialize function array_length"); - - assert_eq!(result, &UInt64Array::from_value(4, 1)); - - // array_length([[1, 2, 3, 4], [5, 6, 7, 8]], 3) = NULL - let arr = array_length(&[list_array, Arc::new(Int64Array::from_value(3, 1))]) - .expect("failed to initialize function array_length"); - let result = - as_uint64_array(&arr).expect("failed to initialize function array_length"); - - assert_eq!(result, &UInt64Array::from(vec![None])); - } - - #[test] - fn test_array_dims() { - // array_dims([1, 2, 3, 4]) = [4] - let list_array = return_array(); - - let array = - array_dims(&[list_array]).expect("failed to initialize function array_dims"); - let result = - as_list_array(&array).expect("failed to initialize function array_dims"); - - assert_eq!( - &[4], - result - .value(0) - .as_any() - .downcast_ref::() - .unwrap() - .values() - ); - } - - #[test] - fn test_nested_array_dims() { - // array_dims([[1, 2, 3, 4], [5, 6, 7, 8]]) = [2, 4] - let list_array = return_nested_array(); - - let array = - array_dims(&[list_array]).expect("failed to initialize function array_dims"); - let result = - as_list_array(&array).expect("failed to initialize function array_dims"); - - assert_eq!( - &[2, 4], - result - .value(0) - .as_any() - .downcast_ref::() - .unwrap() - .values() - ); - } - - #[test] - fn test_array_ndims() { - // array_ndims([1, 2, 3, 4]) = 1 - let list_array = return_array(); - - let array = array_ndims(&[list_array]) - .expect("failed to initialize function array_ndims"); - let result = - as_uint64_array(&array).expect("failed to initialize function array_ndims"); - - assert_eq!(result, &UInt64Array::from_value(1, 1)); - } - - #[test] - fn test_nested_array_ndims() { - // array_ndims([[1, 2, 3, 4], [5, 6, 7, 8]]) = 2 - let list_array = return_nested_array(); - - let array = array_ndims(&[list_array]) - .expect("failed to initialize function array_ndims"); - let result = - as_uint64_array(&array).expect("failed to initialize function array_ndims"); - - assert_eq!(result, &UInt64Array::from_value(2, 1)); - } - #[test] fn test_check_invalid_datatypes() { let data = vec![Some(vec![Some(1), Some(2), Some(3)])]; @@ -3031,68 +2057,4 @@ mod tests { assert_eq!(array.unwrap_err().strip_backtrace(), "Error during planning: array_append received incompatible types: '[Int64, Utf8]'."); } - - fn return_array() -> ArrayRef { - // Returns: [1, 2, 3, 4] - let args = [ - Arc::new(Int64Array::from(vec![Some(1)])) as ArrayRef, - Arc::new(Int64Array::from(vec![Some(2)])) as ArrayRef, - Arc::new(Int64Array::from(vec![Some(3)])) as ArrayRef, - Arc::new(Int64Array::from(vec![Some(4)])) as ArrayRef, - ]; - make_array(&args).expect("failed to initialize function array") - } - - fn return_nested_array() -> ArrayRef { - // Returns: [[1, 2, 3, 4], [5, 6, 7, 8]] - let args = [ - Arc::new(Int64Array::from(vec![Some(1)])) as ArrayRef, - Arc::new(Int64Array::from(vec![Some(2)])) as ArrayRef, - Arc::new(Int64Array::from(vec![Some(3)])) as ArrayRef, - Arc::new(Int64Array::from(vec![Some(4)])) as ArrayRef, - ]; - let arr1 = make_array(&args).expect("failed to initialize function array"); - - let args = [ - Arc::new(Int64Array::from(vec![Some(5)])) as ArrayRef, - Arc::new(Int64Array::from(vec![Some(6)])) as ArrayRef, - Arc::new(Int64Array::from(vec![Some(7)])) as ArrayRef, - Arc::new(Int64Array::from(vec![Some(8)])) as ArrayRef, - ]; - let arr2 = make_array(&args).expect("failed to initialize function array"); - - make_array(&[arr1, arr2]).expect("failed to initialize function array") - } - - fn return_array_with_nulls() -> ArrayRef { - // Returns: [1, NULL, 3, NULL] - let args = [ - Arc::new(Int64Array::from(vec![Some(1)])) as ArrayRef, - Arc::new(Int64Array::from(vec![None])) as ArrayRef, - Arc::new(Int64Array::from(vec![Some(3)])) as ArrayRef, - Arc::new(Int64Array::from(vec![None])) as ArrayRef, - ]; - make_array(&args).expect("failed to initialize function array") - } - - fn return_nested_array_with_nulls() -> ArrayRef { - // Returns: [[1, NULL, 3, NULL], [NULL, 6, 7, NULL]] - let args = [ - Arc::new(Int64Array::from(vec![Some(1)])) as ArrayRef, - Arc::new(Int64Array::from(vec![None])) as ArrayRef, - Arc::new(Int64Array::from(vec![Some(3)])) as ArrayRef, - Arc::new(Int64Array::from(vec![None])) as ArrayRef, - ]; - let arr1 = make_array(&args).expect("failed to initialize function array"); - - let args = [ - Arc::new(Int64Array::from(vec![None])) as ArrayRef, - Arc::new(Int64Array::from(vec![Some(6)])) as ArrayRef, - Arc::new(Int64Array::from(vec![Some(7)])) as ArrayRef, - Arc::new(Int64Array::from(vec![None])) as ArrayRef, - ]; - let arr2 = make_array(&args).expect("failed to initialize function array"); - - make_array(&[arr1, arr2]).expect("failed to initialize function array") - } } diff --git a/datafusion/sqllogictest/test_files/array.slt b/datafusion/sqllogictest/test_files/array.slt index db657ff22bd5..9e3ac3bf08f6 100644 --- a/datafusion/sqllogictest/test_files/array.slt +++ b/datafusion/sqllogictest/test_files/array.slt @@ -942,7 +942,7 @@ select array_slice(make_array(1, 2, 3, 4, 5), 0, -4), array_slice(make_array('h' # array_slice scalar function #13 (with negative number and NULL) query error -select array_slice(make_array(1, 2, 3, 4, 5), 2, NULL), array_slice(make_array('h', 'e', 'l', 'l', 'o'), 3, NULL); +select array_slice(make_array(1, 2, 3, 4, 5), -2, NULL), array_slice(make_array('h', 'e', 'l', 'l', 'o'), -3, NULL); # array_slice scalar function #14 (with NULL and negative number) query error @@ -979,10 +979,10 @@ select array_slice(make_array(1, 2, 3, 4, 5), -7, -2), array_slice(make_array('h [] [] # array_slice scalar function #20 (with negative indexes; nested array) -query ? -select array_slice(make_array(make_array(1, 2, 3, 4, 5), make_array(6, 7, 8, 9, 10)), -2, -1); +query ?? +select array_slice(make_array(make_array(1, 2, 3, 4, 5), make_array(6, 7, 8, 9, 10)), -2, -1), array_slice(make_array(make_array(1, 2, 3), make_array(6, 7, 8)), -1, -1); ---- -[[1, 2, 3, 4, 5]] +[[1, 2, 3, 4, 5]] [] # array_slice scalar function #21 (with first positive index and last negative index) query ?? @@ -2395,11 +2395,17 @@ select array_length(make_array()), array_length(make_array(), 1), array_length(m ---- 0 0 NULL -# list_length scalar function #6 (function alias `array_length`) +# array_length scalar function #6 nested array query III -select list_length(make_array(1, 2, 3, 4, 5)), list_length(make_array(1, 2, 3)), list_length(make_array([1, 2], [3, 4], [5, 6])); +select array_length([[1, 2, 3, 4], [5, 6, 7, 8]]), array_length([[1, 2, 3, 4], [5, 6, 7, 8]], 1), array_length([[1, 2, 3, 4], [5, 6, 7, 8]], 2); ---- -5 3 3 +2 2 4 + +# list_length scalar function #7 (function alias `array_length`) +query IIII +select list_length(make_array(1, 2, 3, 4, 5)), list_length(make_array(1, 2, 3)), list_length(make_array([1, 2], [3, 4], [5, 6])), array_length([[1, 2, 3, 4], [5, 6, 7, 8]], 3); +---- +5 3 3 NULL # array_length with columns query I From e21b03154511cd61e03e299a595db6be6b1852c1 Mon Sep 17 00:00:00 2001 From: Mustafa Akur <106137913+mustafasrepo@users.noreply.github.com> Date: Wed, 29 Nov 2023 08:48:26 +0300 Subject: [PATCH 324/572] NTH_VALUE reverse support (#8327) Co-authored-by: Mehmet Ozan Kabak --- .../enforce_distribution.rs | 6 +- .../src/physical_optimizer/enforce_sorting.rs | 3 +- .../physical_optimizer/projection_pushdown.rs | 3 +- .../replace_with_order_preserving_variants.rs | 9 +- .../core/src/physical_optimizer/utils.rs | 9 +- .../physical-expr/src/window/nth_value.rs | 89 ++++++++---- .../physical-expr/src/window/window_expr.rs | 16 ++- datafusion/physical-plan/src/lib.rs | 7 + .../src/windows/bounded_window_agg_exec.rs | 128 ++++++++++++++++++ .../proto/src/physical_plan/to_proto.rs | 8 +- datafusion/sqllogictest/test_files/window.slt | 50 +++++++ 11 files changed, 269 insertions(+), 59 deletions(-) diff --git a/datafusion/core/src/physical_optimizer/enforce_distribution.rs b/datafusion/core/src/physical_optimizer/enforce_distribution.rs index a34958a6c96d..4befea741c8c 100644 --- a/datafusion/core/src/physical_optimizer/enforce_distribution.rs +++ b/datafusion/core/src/physical_optimizer/enforce_distribution.rs @@ -28,8 +28,8 @@ use std::sync::Arc; use crate::config::ConfigOptions; use crate::error::Result; use crate::physical_optimizer::utils::{ - add_sort_above, get_children_exectrees, get_plan_string, is_coalesce_partitions, - is_repartition, is_sort_preserving_merge, ExecTree, + add_sort_above, get_children_exectrees, is_coalesce_partitions, is_repartition, + is_sort_preserving_merge, ExecTree, }; use crate::physical_optimizer::PhysicalOptimizerRule; use crate::physical_plan::aggregates::{AggregateExec, AggregateMode, PhysicalGroupBy}; @@ -54,8 +54,8 @@ use datafusion_physical_expr::utils::map_columns_before_projection; use datafusion_physical_expr::{ physical_exprs_equal, EquivalenceProperties, PhysicalExpr, }; -use datafusion_physical_plan::unbounded_output; use datafusion_physical_plan::windows::{get_best_fitting_window, BoundedWindowAggExec}; +use datafusion_physical_plan::{get_plan_string, unbounded_output}; use itertools::izip; diff --git a/datafusion/core/src/physical_optimizer/enforce_sorting.rs b/datafusion/core/src/physical_optimizer/enforce_sorting.rs index 6fec74f608ae..ff052b5f040c 100644 --- a/datafusion/core/src/physical_optimizer/enforce_sorting.rs +++ b/datafusion/core/src/physical_optimizer/enforce_sorting.rs @@ -765,9 +765,8 @@ mod tests { repartition_exec, sort_exec, sort_expr, sort_expr_options, sort_merge_join_exec, sort_preserving_merge_exec, spr_repartition_exec, union_exec, }; - use crate::physical_optimizer::utils::get_plan_string; use crate::physical_plan::repartition::RepartitionExec; - use crate::physical_plan::{displayable, Partitioning}; + use crate::physical_plan::{displayable, get_plan_string, Partitioning}; use crate::prelude::{SessionConfig, SessionContext}; use crate::test::csv_exec_sorted; diff --git a/datafusion/core/src/physical_optimizer/projection_pushdown.rs b/datafusion/core/src/physical_optimizer/projection_pushdown.rs index c0e512ffe57b..7ebb64ab858a 100644 --- a/datafusion/core/src/physical_optimizer/projection_pushdown.rs +++ b/datafusion/core/src/physical_optimizer/projection_pushdown.rs @@ -1130,7 +1130,6 @@ mod tests { use crate::physical_optimizer::projection_pushdown::{ join_table_borders, update_expr, ProjectionPushdown, }; - use crate::physical_optimizer::utils::get_plan_string; use crate::physical_optimizer::PhysicalOptimizerRule; use crate::physical_plan::coalesce_partitions::CoalescePartitionsExec; use crate::physical_plan::filter::FilterExec; @@ -1141,7 +1140,7 @@ mod tests { use crate::physical_plan::repartition::RepartitionExec; use crate::physical_plan::sorts::sort::SortExec; use crate::physical_plan::sorts::sort_preserving_merge::SortPreservingMergeExec; - use crate::physical_plan::ExecutionPlan; + use crate::physical_plan::{get_plan_string, ExecutionPlan}; use arrow_schema::{DataType, Field, Schema, SchemaRef, SortOptions}; use datafusion_common::config::ConfigOptions; diff --git a/datafusion/core/src/physical_optimizer/replace_with_order_preserving_variants.rs b/datafusion/core/src/physical_optimizer/replace_with_order_preserving_variants.rs index 5f130848de11..09274938cbce 100644 --- a/datafusion/core/src/physical_optimizer/replace_with_order_preserving_variants.rs +++ b/datafusion/core/src/physical_optimizer/replace_with_order_preserving_variants.rs @@ -286,7 +286,7 @@ mod tests { use crate::physical_plan::repartition::RepartitionExec; use crate::physical_plan::sorts::sort::SortExec; use crate::physical_plan::sorts::sort_preserving_merge::SortPreservingMergeExec; - use crate::physical_plan::{displayable, Partitioning}; + use crate::physical_plan::{displayable, get_plan_string, Partitioning}; use crate::prelude::SessionConfig; use arrow::compute::SortOptions; @@ -958,11 +958,4 @@ mod tests { FileCompressionType::UNCOMPRESSED, )) } - - // Util function to get string representation of a physical plan - fn get_plan_string(plan: &Arc) -> Vec { - let formatted = displayable(plan.as_ref()).indent(true).to_string(); - let actual: Vec<&str> = formatted.trim().lines().collect(); - actual.iter().map(|elem| elem.to_string()).collect() - } } diff --git a/datafusion/core/src/physical_optimizer/utils.rs b/datafusion/core/src/physical_optimizer/utils.rs index 530df374ca7c..fccc1db0d359 100644 --- a/datafusion/core/src/physical_optimizer/utils.rs +++ b/datafusion/core/src/physical_optimizer/utils.rs @@ -28,7 +28,7 @@ use crate::physical_plan::sorts::sort::SortExec; use crate::physical_plan::sorts::sort_preserving_merge::SortPreservingMergeExec; use crate::physical_plan::union::UnionExec; use crate::physical_plan::windows::{BoundedWindowAggExec, WindowAggExec}; -use crate::physical_plan::{displayable, ExecutionPlan}; +use crate::physical_plan::{get_plan_string, ExecutionPlan}; use datafusion_physical_expr::{LexRequirementRef, PhysicalSortRequirement}; @@ -154,10 +154,3 @@ pub fn is_union(plan: &Arc) -> bool { pub fn is_repartition(plan: &Arc) -> bool { plan.as_any().is::() } - -/// Utility function yielding a string representation of the given [`ExecutionPlan`]. -pub fn get_plan_string(plan: &Arc) -> Vec { - let formatted = displayable(plan.as_ref()).indent(true).to_string(); - let actual: Vec<&str> = formatted.trim().lines().collect(); - actual.iter().map(|elem| elem.to_string()).collect() -} diff --git a/datafusion/physical-expr/src/window/nth_value.rs b/datafusion/physical-expr/src/window/nth_value.rs index 262a50969b82..b3c89122ebad 100644 --- a/datafusion/physical-expr/src/window/nth_value.rs +++ b/datafusion/physical-expr/src/window/nth_value.rs @@ -15,21 +15,24 @@ // specific language governing permissions and limitations // under the License. -//! Defines physical expressions for `first_value`, `last_value`, and `nth_value` -//! that can evaluated at runtime during query execution +//! Defines physical expressions for `FIRST_VALUE`, `LAST_VALUE`, and `NTH_VALUE` +//! functions that can be evaluated at run time during query execution. + +use std::any::Any; +use std::cmp::Ordering; +use std::ops::Range; +use std::sync::Arc; use crate::window::window_expr::{NthValueKind, NthValueState}; use crate::window::BuiltInWindowFunctionExpr; use crate::PhysicalExpr; + use arrow::array::{Array, ArrayRef}; use arrow::datatypes::{DataType, Field}; use datafusion_common::{exec_err, ScalarValue}; use datafusion_common::{DataFusionError, Result}; use datafusion_expr::window_state::WindowAggState; use datafusion_expr::PartitionEvaluator; -use std::any::Any; -use std::ops::Range; -use std::sync::Arc; /// nth_value expression #[derive(Debug)] @@ -77,17 +80,17 @@ impl NthValue { n: u32, ) -> Result { match n { - 0 => exec_err!("nth_value expect n to be > 0"), + 0 => exec_err!("NTH_VALUE expects n to be non-zero"), _ => Ok(Self { name: name.into(), expr, data_type, - kind: NthValueKind::Nth(n), + kind: NthValueKind::Nth(n as i64), }), } } - /// Get nth_value kind + /// Get the NTH_VALUE kind pub fn get_kind(&self) -> NthValueKind { self.kind } @@ -125,7 +128,7 @@ impl BuiltInWindowFunctionExpr for NthValue { let reversed_kind = match self.kind { NthValueKind::First => NthValueKind::Last, NthValueKind::Last => NthValueKind::First, - NthValueKind::Nth(_) => return None, + NthValueKind::Nth(idx) => NthValueKind::Nth(-idx), }; Some(Arc::new(Self { name: self.name.clone(), @@ -143,16 +146,17 @@ pub(crate) struct NthValueEvaluator { } impl PartitionEvaluator for NthValueEvaluator { - /// When the window frame has a fixed beginning (e.g UNBOUNDED - /// PRECEDING), for some functions such as FIRST_VALUE, LAST_VALUE and - /// NTH_VALUE we can memoize result. Once result is calculated it - /// will always stay same. Hence, we do not need to keep past data - /// as we process the entire dataset. This feature enables us to - /// prune rows from table. The default implementation does nothing + /// When the window frame has a fixed beginning (e.g UNBOUNDED PRECEDING), + /// for some functions such as FIRST_VALUE, LAST_VALUE and NTH_VALUE, we + /// can memoize the result. Once result is calculated, it will always stay + /// same. Hence, we do not need to keep past data as we process the entire + /// dataset. fn memoize(&mut self, state: &mut WindowAggState) -> Result<()> { let out = &state.out_col; let size = out.len(); - let (is_prunable, is_last) = match self.state.kind { + let mut buffer_size = 1; + // Decide if we arrived at a final result yet: + let (is_prunable, is_reverse_direction) = match self.state.kind { NthValueKind::First => { let n_range = state.window_frame_range.end - state.window_frame_range.start; @@ -162,16 +166,30 @@ impl PartitionEvaluator for NthValueEvaluator { NthValueKind::Nth(n) => { let n_range = state.window_frame_range.end - state.window_frame_range.start; - (n_range >= (n as usize) && size >= (n as usize), false) + match n.cmp(&0) { + Ordering::Greater => { + (n_range >= (n as usize) && size > (n as usize), false) + } + Ordering::Less => { + let reverse_index = (-n) as usize; + buffer_size = reverse_index; + // Negative index represents reverse direction. + (n_range >= reverse_index, true) + } + Ordering::Equal => { + // The case n = 0 is not valid for the NTH_VALUE function. + unreachable!(); + } + } } }; if is_prunable { - if self.state.finalized_result.is_none() && !is_last { + if self.state.finalized_result.is_none() && !is_reverse_direction { let result = ScalarValue::try_from_array(out, size - 1)?; self.state.finalized_result = Some(result); } state.window_frame_range.start = - state.window_frame_range.end.saturating_sub(1); + state.window_frame_range.end.saturating_sub(buffer_size); } Ok(()) } @@ -195,12 +213,33 @@ impl PartitionEvaluator for NthValueEvaluator { NthValueKind::First => ScalarValue::try_from_array(arr, range.start), NthValueKind::Last => ScalarValue::try_from_array(arr, range.end - 1), NthValueKind::Nth(n) => { - // We are certain that n > 0. - let index = (n as usize) - 1; - if index >= n_range { - ScalarValue::try_from(arr.data_type()) - } else { - ScalarValue::try_from_array(arr, range.start + index) + match n.cmp(&0) { + Ordering::Greater => { + // SQL indices are not 0-based. + let index = (n as usize) - 1; + if index >= n_range { + // Outside the range, return NULL: + ScalarValue::try_from(arr.data_type()) + } else { + ScalarValue::try_from_array(arr, range.start + index) + } + } + Ordering::Less => { + let reverse_index = (-n) as usize; + if n_range >= reverse_index { + ScalarValue::try_from_array( + arr, + range.start + n_range - reverse_index, + ) + } else { + // Outside the range, return NULL: + ScalarValue::try_from(arr.data_type()) + } + } + Ordering::Equal => { + // The case n = 0 is not valid for the NTH_VALUE function. + unreachable!(); + } } } } diff --git a/datafusion/physical-expr/src/window/window_expr.rs b/datafusion/physical-expr/src/window/window_expr.rs index b282e3579754..4211a616e100 100644 --- a/datafusion/physical-expr/src/window/window_expr.rs +++ b/datafusion/physical-expr/src/window/window_expr.rs @@ -15,7 +15,13 @@ // specific language governing permissions and limitations // under the License. +use std::any::Any; +use std::fmt::Debug; +use std::ops::Range; +use std::sync::Arc; + use crate::{PhysicalExpr, PhysicalSortExpr}; + use arrow::array::{new_empty_array, Array, ArrayRef}; use arrow::compute::kernels::sort::SortColumn; use arrow::compute::SortOptions; @@ -25,13 +31,9 @@ use datafusion_common::{internal_err, DataFusionError, Result, ScalarValue}; use datafusion_expr::window_state::{ PartitionBatchState, WindowAggState, WindowFrameContext, }; -use datafusion_expr::PartitionEvaluator; -use datafusion_expr::{Accumulator, WindowFrame}; +use datafusion_expr::{Accumulator, PartitionEvaluator, WindowFrame}; + use indexmap::IndexMap; -use std::any::Any; -use std::fmt::Debug; -use std::ops::Range; -use std::sync::Arc; /// Common trait for [window function] implementations /// @@ -292,7 +294,7 @@ pub struct NumRowsState { pub enum NthValueKind { First, Last, - Nth(u32), + Nth(i64), } #[derive(Debug, Clone)] diff --git a/datafusion/physical-plan/src/lib.rs b/datafusion/physical-plan/src/lib.rs index e5cd5e674cb1..b2c69b467e9c 100644 --- a/datafusion/physical-plan/src/lib.rs +++ b/datafusion/physical-plan/src/lib.rs @@ -570,5 +570,12 @@ pub fn unbounded_output(plan: &Arc) -> bool { .unwrap_or(true) } +/// Utility function yielding a string representation of the given [`ExecutionPlan`]. +pub fn get_plan_string(plan: &Arc) -> Vec { + let formatted = displayable(plan.as_ref()).indent(true).to_string(); + let actual: Vec<&str> = formatted.trim().lines().collect(); + actual.iter().map(|elem| elem.to_string()).collect() +} + #[cfg(test)] pub mod test; diff --git a/datafusion/physical-plan/src/windows/bounded_window_agg_exec.rs b/datafusion/physical-plan/src/windows/bounded_window_agg_exec.rs index fb679b013863..8156ab1fa31b 100644 --- a/datafusion/physical-plan/src/windows/bounded_window_agg_exec.rs +++ b/datafusion/physical-plan/src/windows/bounded_window_agg_exec.rs @@ -1109,3 +1109,131 @@ fn get_aggregate_result_out_column( result .ok_or_else(|| DataFusionError::Execution("Should contain something".to_string())) } + +#[cfg(test)] +mod tests { + use crate::common::collect; + use crate::memory::MemoryExec; + use crate::windows::{BoundedWindowAggExec, PartitionSearchMode}; + use crate::{get_plan_string, ExecutionPlan}; + use arrow_array::RecordBatch; + use arrow_schema::{DataType, Field, Schema}; + use datafusion_common::{assert_batches_eq, Result, ScalarValue}; + use datafusion_execution::config::SessionConfig; + use datafusion_execution::TaskContext; + use datafusion_expr::{WindowFrame, WindowFrameBound, WindowFrameUnits}; + use datafusion_physical_expr::expressions::col; + use datafusion_physical_expr::expressions::NthValue; + use datafusion_physical_expr::window::BuiltInWindowExpr; + use datafusion_physical_expr::window::BuiltInWindowFunctionExpr; + use std::sync::Arc; + + // Tests NTH_VALUE(negative index) with memoize feature. + // To be able to trigger memoize feature for NTH_VALUE we need to + // - feed BoundedWindowAggExec with batch stream data. + // - Window frame should contain UNBOUNDED PRECEDING. + // It hard to ensure these conditions are met, from the sql query. + #[tokio::test] + async fn test_window_nth_value_bounded_memoize() -> Result<()> { + let config = SessionConfig::new().with_target_partitions(1); + let task_ctx = Arc::new(TaskContext::default().with_session_config(config)); + + let schema = Arc::new(Schema::new(vec![Field::new("a", DataType::Int32, false)])); + // Create a new batch of data to insert into the table + let batch = RecordBatch::try_new( + schema.clone(), + vec![Arc::new(arrow_array::Int32Array::from(vec![1, 2, 3]))], + )?; + + let memory_exec = MemoryExec::try_new( + &[vec![batch.clone(), batch.clone(), batch.clone()]], + schema.clone(), + None, + ) + .map(|e| Arc::new(e) as Arc)?; + let col_a = col("a", &schema)?; + let nth_value_func1 = + NthValue::nth("nth_value(-1)", col_a.clone(), DataType::Int32, 1)? + .reverse_expr() + .unwrap(); + let nth_value_func2 = + NthValue::nth("nth_value(-2)", col_a.clone(), DataType::Int32, 2)? + .reverse_expr() + .unwrap(); + let last_value_func = + Arc::new(NthValue::last("last", col_a.clone(), DataType::Int32)) as _; + let window_exprs = vec![ + // LAST_VALUE(a) + Arc::new(BuiltInWindowExpr::new( + last_value_func, + &[], + &[], + Arc::new(WindowFrame { + units: WindowFrameUnits::Rows, + start_bound: WindowFrameBound::Preceding(ScalarValue::UInt64(None)), + end_bound: WindowFrameBound::CurrentRow, + }), + )) as _, + // NTH_VALUE(a, -1) + Arc::new(BuiltInWindowExpr::new( + nth_value_func1, + &[], + &[], + Arc::new(WindowFrame { + units: WindowFrameUnits::Rows, + start_bound: WindowFrameBound::Preceding(ScalarValue::UInt64(None)), + end_bound: WindowFrameBound::CurrentRow, + }), + )) as _, + // NTH_VALUE(a, -2) + Arc::new(BuiltInWindowExpr::new( + nth_value_func2, + &[], + &[], + Arc::new(WindowFrame { + units: WindowFrameUnits::Rows, + start_bound: WindowFrameBound::Preceding(ScalarValue::UInt64(None)), + end_bound: WindowFrameBound::CurrentRow, + }), + )) as _, + ]; + let physical_plan = BoundedWindowAggExec::try_new( + window_exprs, + memory_exec, + vec![], + PartitionSearchMode::Sorted, + ) + .map(|e| Arc::new(e) as Arc)?; + + let batches = collect(physical_plan.execute(0, task_ctx)?).await?; + + let expected = vec![ + "BoundedWindowAggExec: wdw=[last: Ok(Field { name: \"last\", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(NULL)), end_bound: CurrentRow }, nth_value(-1): Ok(Field { name: \"nth_value(-1)\", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(NULL)), end_bound: CurrentRow }, nth_value(-2): Ok(Field { name: \"nth_value(-2)\", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(NULL)), end_bound: CurrentRow }], mode=[Sorted]", + " MemoryExec: partitions=1, partition_sizes=[3]", + ]; + // Get string representation of the plan + let actual = get_plan_string(&physical_plan); + assert_eq!( + expected, actual, + "\n**Optimized Plan Mismatch\n\nexpected:\n\n{expected:#?}\nactual:\n\n{actual:#?}\n\n" + ); + + let expected = [ + "+---+------+---------------+---------------+", + "| a | last | nth_value(-1) | nth_value(-2) |", + "+---+------+---------------+---------------+", + "| 1 | 1 | 1 | |", + "| 2 | 2 | 2 | 1 |", + "| 3 | 3 | 3 | 2 |", + "| 1 | 1 | 1 | 3 |", + "| 2 | 2 | 2 | 1 |", + "| 3 | 3 | 3 | 2 |", + "| 1 | 1 | 1 | 3 |", + "| 2 | 2 | 2 | 1 |", + "| 3 | 3 | 3 | 2 |", + "+---+------+---------------+---------------+", + ]; + assert_batches_eq!(expected, &batches); + Ok(()) + } +} diff --git a/datafusion/proto/src/physical_plan/to_proto.rs b/datafusion/proto/src/physical_plan/to_proto.rs index 44864be947d5..ea00b726b9d6 100644 --- a/datafusion/proto/src/physical_plan/to_proto.rs +++ b/datafusion/proto/src/physical_plan/to_proto.rs @@ -27,11 +27,11 @@ use crate::protobuf::{ physical_aggregate_expr_node, PhysicalSortExprNode, PhysicalSortExprNodeCollection, ScalarValue, }; + use datafusion::datasource::{ - file_format::json::JsonSink, physical_plan::FileScanConfig, -}; -use datafusion::datasource::{ + file_format::json::JsonSink, listing::{FileRange, PartitionedFile}, + physical_plan::FileScanConfig, physical_plan::FileSinkConfig, }; use datafusion::logical_expr::BuiltinScalarFunction; @@ -180,7 +180,7 @@ impl TryFrom> for protobuf::PhysicalWindowExprNode { args.insert( 1, Arc::new(Literal::new( - datafusion_common::ScalarValue::Int64(Some(n as i64)), + datafusion_common::ScalarValue::Int64(Some(n)), )), ); protobuf::BuiltInWindowFunction::NthValue diff --git a/datafusion/sqllogictest/test_files/window.slt b/datafusion/sqllogictest/test_files/window.slt index 4edac211b370..55b8843a0b9c 100644 --- a/datafusion/sqllogictest/test_files/window.slt +++ b/datafusion/sqllogictest/test_files/window.slt @@ -3493,6 +3493,56 @@ select sum(1) over() x, sum(1) over () y ---- 1 1 +# NTH_VALUE requirement is c DESC, However existing ordering is c ASC +# if we reverse window expression: "NTH_VALUE(c, 2) OVER(order by c DESC ) as nv1" +# as "NTH_VALUE(c, -2) OVER(order by c ASC RANGE BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING) as nv1" +# Please note that: "NTH_VALUE(c, 2) OVER(order by c DESC ) as nv1" is same with +# "NTH_VALUE(c, 2) OVER(order by c DESC RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW) as nv1" " +# we can produce same result without re-sorting the table. +# Unfortunately since window expression names are string, this change is not seen the plan (we do not do string manipulation). +# TODO: Reflect window expression reversal in the plans. +query TT +EXPLAIN SELECT c, NTH_VALUE(c, 2) OVER(order by c DESC) as nv1 + FROM multiple_ordered_table + ORDER BY c ASC + LIMIT 5 +---- +logical_plan +Limit: skip=0, fetch=5 +--Sort: multiple_ordered_table.c ASC NULLS LAST, fetch=5 +----Projection: multiple_ordered_table.c, NTH_VALUE(multiple_ordered_table.c,Int64(2)) ORDER BY [multiple_ordered_table.c DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW AS nv1 +------WindowAggr: windowExpr=[[NTH_VALUE(multiple_ordered_table.c, Int64(2)) ORDER BY [multiple_ordered_table.c DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW]] +--------TableScan: multiple_ordered_table projection=[c] +physical_plan +GlobalLimitExec: skip=0, fetch=5 +--ProjectionExec: expr=[c@0 as c, NTH_VALUE(multiple_ordered_table.c,Int64(2)) ORDER BY [multiple_ordered_table.c DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@1 as nv1] +----WindowAggExec: wdw=[NTH_VALUE(multiple_ordered_table.c,Int64(2)) ORDER BY [multiple_ordered_table.c DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "NTH_VALUE(multiple_ordered_table.c,Int64(2)) ORDER BY [multiple_ordered_table.c DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: CurrentRow, end_bound: Following(Int32(NULL)) }] +------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[c], output_ordering=[c@0 ASC NULLS LAST], has_header=true + +query II +SELECT c, NTH_VALUE(c, 2) OVER(order by c DESC) as nv1 + FROM multiple_ordered_table + ORDER BY c ASC + LIMIT 5 +---- +0 98 +1 98 +2 98 +3 98 +4 98 + +query II +SELECT c, NTH_VALUE(c, 2) OVER(order by c DESC) as nv1 + FROM multiple_ordered_table + ORDER BY c DESC + LIMIT 5 +---- +99 NULL +98 98 +97 98 +96 98 +95 98 + statement ok set datafusion.execution.target_partitions = 2; From 19bdcdc4140f0b36023626195d84bfbf970b752d Mon Sep 17 00:00:00 2001 From: Mustafa Akur <106137913+mustafasrepo@users.noreply.github.com> Date: Wed, 29 Nov 2023 10:58:51 +0300 Subject: [PATCH 325/572] Refactor optimize projections rule, combines (eliminate, merge, pushdown projections) (#8340) --- datafusion/core/tests/sql/explain_analyze.rs | 2 +- datafusion/expr/src/logical_plan/mod.rs | 9 +- datafusion/expr/src/logical_plan/plan.rs | 74 +- datafusion/optimizer/src/eliminate_project.rs | 94 -- datafusion/optimizer/src/lib.rs | 2 +- datafusion/optimizer/src/merge_projection.rs | 106 +-- .../optimizer/src/optimize_projections.rs | 848 ++++++++++++++++++ datafusion/optimizer/src/optimizer.rs | 10 +- .../optimizer/src/push_down_projection.rs | 530 +---------- .../optimizer/tests/optimizer_integration.rs | 10 +- .../sqllogictest/test_files/aggregate.slt | 7 +- .../sqllogictest/test_files/explain.slt | 12 +- datafusion/sqllogictest/test_files/limit.slt | 16 +- .../sqllogictest/test_files/subquery.slt | 62 +- datafusion/sqllogictest/test_files/window.slt | 30 +- .../tests/cases/roundtrip_logical_plan.rs | 18 +- 16 files changed, 1011 insertions(+), 819 deletions(-) delete mode 100644 datafusion/optimizer/src/eliminate_project.rs create mode 100644 datafusion/optimizer/src/optimize_projections.rs diff --git a/datafusion/core/tests/sql/explain_analyze.rs b/datafusion/core/tests/sql/explain_analyze.rs index 0ebd3a0c69d1..ecb5766a3bb5 100644 --- a/datafusion/core/tests/sql/explain_analyze.rs +++ b/datafusion/core/tests/sql/explain_analyze.rs @@ -560,7 +560,7 @@ async fn csv_explain_verbose_plans() { // Since the plan contains path that are environmentally // dependant(e.g. full path of the test file), only verify // important content - assert_contains!(&actual, "logical_plan after push_down_projection"); + assert_contains!(&actual, "logical_plan after optimize_projections"); assert_contains!(&actual, "physical_plan"); assert_contains!(&actual, "FilterExec: c2@1 > 10"); assert_contains!(actual, "ProjectionExec: expr=[c1@0 as c1]"); diff --git a/datafusion/expr/src/logical_plan/mod.rs b/datafusion/expr/src/logical_plan/mod.rs index 51d78cd721b6..bc722dd69ace 100644 --- a/datafusion/expr/src/logical_plan/mod.rs +++ b/datafusion/expr/src/logical_plan/mod.rs @@ -33,10 +33,11 @@ pub use ddl::{ }; pub use dml::{DmlStatement, WriteOp}; pub use plan::{ - Aggregate, Analyze, CrossJoin, DescribeTable, Distinct, DistinctOn, EmptyRelation, - Explain, Extension, Filter, Join, JoinConstraint, JoinType, Limit, LogicalPlan, - Partitioning, PlanType, Prepare, Projection, Repartition, Sort, StringifiedPlan, - Subquery, SubqueryAlias, TableScan, ToStringifiedPlan, Union, Unnest, Values, Window, + projection_schema, Aggregate, Analyze, CrossJoin, DescribeTable, Distinct, + DistinctOn, EmptyRelation, Explain, Extension, Filter, Join, JoinConstraint, + JoinType, Limit, LogicalPlan, Partitioning, PlanType, Prepare, Projection, + Repartition, Sort, StringifiedPlan, Subquery, SubqueryAlias, TableScan, + ToStringifiedPlan, Union, Unnest, Values, Window, }; pub use statement::{ SetVariable, Statement, TransactionAccessMode, TransactionConclusion, TransactionEnd, diff --git a/datafusion/expr/src/logical_plan/plan.rs b/datafusion/expr/src/logical_plan/plan.rs index 69ba42d34a70..ea7a48d2c4f4 100644 --- a/datafusion/expr/src/logical_plan/plan.rs +++ b/datafusion/expr/src/logical_plan/plan.rs @@ -551,15 +551,9 @@ impl LogicalPlan { Projection::try_new(projection.expr.to_vec(), Arc::new(inputs[0].clone())) .map(LogicalPlan::Projection) } - LogicalPlan::Window(Window { - window_expr, - schema, - .. - }) => Ok(LogicalPlan::Window(Window { - input: Arc::new(inputs[0].clone()), - window_expr: window_expr.to_vec(), - schema: schema.clone(), - })), + LogicalPlan::Window(Window { window_expr, .. }) => Ok(LogicalPlan::Window( + Window::try_new(window_expr.to_vec(), Arc::new(inputs[0].clone()))?, + )), LogicalPlan::Aggregate(Aggregate { group_expr, aggr_expr, @@ -837,10 +831,19 @@ impl LogicalPlan { LogicalPlan::Extension(e) => Ok(LogicalPlan::Extension(Extension { node: e.node.from_template(&expr, inputs), })), - LogicalPlan::Union(Union { schema, .. }) => Ok(LogicalPlan::Union(Union { - inputs: inputs.iter().cloned().map(Arc::new).collect(), - schema: schema.clone(), - })), + LogicalPlan::Union(Union { schema, .. }) => { + let input_schema = inputs[0].schema(); + // If inputs are not pruned do not change schema. + let schema = if schema.fields().len() == input_schema.fields().len() { + schema + } else { + input_schema + }; + Ok(LogicalPlan::Union(Union { + inputs: inputs.iter().cloned().map(Arc::new).collect(), + schema: schema.clone(), + })) + } LogicalPlan::Distinct(distinct) => { let distinct = match distinct { Distinct::All(_) => Distinct::All(Arc::new(inputs[0].clone())), @@ -1792,11 +1795,8 @@ pub struct Projection { impl Projection { /// Create a new Projection pub fn try_new(expr: Vec, input: Arc) -> Result { - let schema = Arc::new(DFSchema::new_with_metadata( - exprlist_to_fields(&expr, &input)?, - input.schema().metadata().clone(), - )?); - Self::try_new_with_schema(expr, input, schema) + let projection_schema = projection_schema(&input, &expr)?; + Self::try_new_with_schema(expr, input, projection_schema) } /// Create a new Projection using the specified output schema @@ -1808,11 +1808,6 @@ impl Projection { if expr.len() != schema.fields().len() { return plan_err!("Projection has mismatch between number of expressions ({}) and number of fields in schema ({})", expr.len(), schema.fields().len()); } - // Update functional dependencies of `input` according to projection - // expressions: - let id_key_groups = calc_func_dependencies_for_project(&expr, &input)?; - let schema = schema.as_ref().clone(); - let schema = Arc::new(schema.with_functional_dependencies(id_key_groups)); Ok(Self { expr, input, @@ -1836,6 +1831,29 @@ impl Projection { } } +/// Computes the schema of the result produced by applying a projection to the input logical plan. +/// +/// # Arguments +/// +/// * `input`: A reference to the input `LogicalPlan` for which the projection schema +/// will be computed. +/// * `exprs`: A slice of `Expr` expressions representing the projection operation to apply. +/// +/// # Returns +/// +/// A `Result` containing an `Arc` representing the schema of the result +/// produced by the projection operation. If the schema computation is successful, +/// the `Result` will contain the schema; otherwise, it will contain an error. +pub fn projection_schema(input: &LogicalPlan, exprs: &[Expr]) -> Result> { + let mut schema = DFSchema::new_with_metadata( + exprlist_to_fields(exprs, input)?, + input.schema().metadata().clone(), + )?; + schema = schema + .with_functional_dependencies(calc_func_dependencies_for_project(exprs, input)?); + Ok(Arc::new(schema)) +} + /// Aliased subquery #[derive(Clone, PartialEq, Eq, Hash)] // mark non_exhaustive to encourage use of try_new/new() @@ -1934,8 +1952,7 @@ impl Window { /// Create a new window operator. pub fn try_new(window_expr: Vec, input: Arc) -> Result { let mut window_fields: Vec = input.schema().fields().clone(); - window_fields - .extend_from_slice(&exprlist_to_fields(window_expr.iter(), input.as_ref())?); + window_fields.extend_from_slice(&exprlist_to_fields(window_expr.iter(), &input)?); let metadata = input.schema().metadata().clone(); // Update functional dependencies for window: @@ -2357,6 +2374,13 @@ impl Aggregate { schema, }) } + + /// Get the length of the group by expression in the output schema + /// This is not simply group by expression length. Expression may be + /// GroupingSet, etc. In these case we need to get inner expression lengths. + pub fn group_expr_len(&self) -> Result { + grouping_set_expr_count(&self.group_expr) + } } /// Checks whether any expression in `group_expr` contains `Expr::GroupingSet`. diff --git a/datafusion/optimizer/src/eliminate_project.rs b/datafusion/optimizer/src/eliminate_project.rs deleted file mode 100644 index d3226eaa78cf..000000000000 --- a/datafusion/optimizer/src/eliminate_project.rs +++ /dev/null @@ -1,94 +0,0 @@ -// Licensed to the Apache Software Foundation (ASF) under one -// or more contributor license agreements. See the NOTICE file -// distributed with this work for additional information -// regarding copyright ownership. The ASF licenses this file -// to you under the Apache License, Version 2.0 (the -// "License"); you may not use this file except in compliance -// with the License. You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, -// software distributed under the License is distributed on an -// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -// KIND, either express or implied. See the License for the -// specific language governing permissions and limitations -// under the License. - -use crate::optimizer::ApplyOrder; -use crate::{OptimizerConfig, OptimizerRule}; -use datafusion_common::{DFSchemaRef, Result}; -use datafusion_expr::logical_plan::LogicalPlan; -use datafusion_expr::{Expr, Projection}; - -/// Optimization rule that eliminate unnecessary [LogicalPlan::Projection]. -#[derive(Default)] -pub struct EliminateProjection; - -impl EliminateProjection { - #[allow(missing_docs)] - pub fn new() -> Self { - Self {} - } -} - -impl OptimizerRule for EliminateProjection { - fn try_optimize( - &self, - plan: &LogicalPlan, - _config: &dyn OptimizerConfig, - ) -> Result> { - match plan { - LogicalPlan::Projection(projection) => { - let child_plan = projection.input.as_ref(); - match child_plan { - LogicalPlan::Union(_) - | LogicalPlan::Filter(_) - | LogicalPlan::TableScan(_) - | LogicalPlan::SubqueryAlias(_) - | LogicalPlan::Sort(_) => { - if can_eliminate(projection, child_plan.schema()) { - Ok(Some(child_plan.clone())) - } else { - Ok(None) - } - } - _ => { - if plan.schema() == child_plan.schema() { - Ok(Some(child_plan.clone())) - } else { - Ok(None) - } - } - } - } - _ => Ok(None), - } - } - - fn name(&self) -> &str { - "eliminate_projection" - } - - fn apply_order(&self) -> Option { - Some(ApplyOrder::TopDown) - } -} - -pub(crate) fn can_eliminate(projection: &Projection, schema: &DFSchemaRef) -> bool { - if projection.expr.len() != schema.fields().len() { - return false; - } - for (i, e) in projection.expr.iter().enumerate() { - match e { - Expr::Column(c) => { - let d = schema.fields().get(i).unwrap(); - if c != &d.qualified_column() && c != &d.unqualified_column() { - return false; - } - } - _ => return false, - } - } - true -} diff --git a/datafusion/optimizer/src/lib.rs b/datafusion/optimizer/src/lib.rs index ede0ac5c7164..d8b0c14589a2 100644 --- a/datafusion/optimizer/src/lib.rs +++ b/datafusion/optimizer/src/lib.rs @@ -27,10 +27,10 @@ pub mod eliminate_limit; pub mod eliminate_nested_union; pub mod eliminate_one_union; pub mod eliminate_outer_join; -pub mod eliminate_project; pub mod extract_equijoin_predicate; pub mod filter_null_join_keys; pub mod merge_projection; +pub mod optimize_projections; pub mod optimizer; pub mod propagate_empty_relation; pub mod push_down_filter; diff --git a/datafusion/optimizer/src/merge_projection.rs b/datafusion/optimizer/src/merge_projection.rs index ec040cba6fe4..f7b750011e44 100644 --- a/datafusion/optimizer/src/merge_projection.rs +++ b/datafusion/optimizer/src/merge_projection.rs @@ -15,105 +15,9 @@ // specific language governing permissions and limitations // under the License. -use std::collections::HashMap; - -use crate::optimizer::ApplyOrder; -use crate::push_down_filter::replace_cols_by_name; -use crate::{OptimizerConfig, OptimizerRule}; - -use datafusion_common::Result; -use datafusion_expr::{Expr, LogicalPlan, Projection}; - -/// Optimization rule that merge [LogicalPlan::Projection]. -#[derive(Default)] -pub struct MergeProjection; - -impl MergeProjection { - #[allow(missing_docs)] - pub fn new() -> Self { - Self {} - } -} - -impl OptimizerRule for MergeProjection { - fn try_optimize( - &self, - plan: &LogicalPlan, - _config: &dyn OptimizerConfig, - ) -> Result> { - match plan { - LogicalPlan::Projection(parent_projection) => { - match parent_projection.input.as_ref() { - LogicalPlan::Projection(child_projection) => { - let new_plan = - merge_projection(parent_projection, child_projection)?; - Ok(Some( - self.try_optimize(&new_plan, _config)?.unwrap_or(new_plan), - )) - } - _ => Ok(None), - } - } - _ => Ok(None), - } - } - - fn name(&self) -> &str { - "merge_projection" - } - - fn apply_order(&self) -> Option { - Some(ApplyOrder::TopDown) - } -} - -pub(super) fn merge_projection( - parent_projection: &Projection, - child_projection: &Projection, -) -> Result { - let replace_map = collect_projection_expr(child_projection); - let new_exprs = parent_projection - .expr - .iter() - .map(|expr| replace_cols_by_name(expr.clone(), &replace_map)) - .enumerate() - .map(|(i, e)| match e { - Ok(e) => { - let parent_expr = parent_projection.schema.fields()[i].qualified_name(); - e.alias_if_changed(parent_expr) - } - Err(e) => Err(e), - }) - .collect::>>()?; - // Use try_new, since schema changes with changing expressions. - let new_plan = LogicalPlan::Projection(Projection::try_new( - new_exprs, - child_projection.input.clone(), - )?); - Ok(new_plan) -} - -pub fn collect_projection_expr(projection: &Projection) -> HashMap { - projection - .schema - .fields() - .iter() - .enumerate() - .flat_map(|(i, field)| { - // strip alias - let expr = projection.expr[i].clone().unalias(); - // Convert both qualified and unqualified fields - [ - (field.name().clone(), expr.clone()), - (field.qualified_name(), expr), - ] - }) - .collect::>() -} - #[cfg(test)] mod tests { - use crate::merge_projection::MergeProjection; + use crate::optimize_projections::OptimizeProjections; use datafusion_common::Result; use datafusion_expr::{ binary_expr, col, lit, logical_plan::builder::LogicalPlanBuilder, LogicalPlan, @@ -124,7 +28,7 @@ mod tests { use crate::test::*; fn assert_optimized_plan_equal(plan: &LogicalPlan, expected: &str) -> Result<()> { - assert_optimized_plan_eq(Arc::new(MergeProjection::new()), plan, expected) + assert_optimized_plan_eq(Arc::new(OptimizeProjections::new()), plan, expected) } #[test] @@ -136,7 +40,7 @@ mod tests { .build()?; let expected = "Projection: Int32(1) + test.a\ - \n TableScan: test"; + \n TableScan: test projection=[a]"; assert_optimized_plan_equal(&plan, expected) } @@ -150,7 +54,7 @@ mod tests { .build()?; let expected = "Projection: Int32(1) + test.a\ - \n TableScan: test"; + \n TableScan: test projection=[a]"; assert_optimized_plan_equal(&plan, expected) } @@ -163,7 +67,7 @@ mod tests { .build()?; let expected = "Projection: test.a AS alias\ - \n TableScan: test"; + \n TableScan: test projection=[a]"; assert_optimized_plan_equal(&plan, expected) } } diff --git a/datafusion/optimizer/src/optimize_projections.rs b/datafusion/optimizer/src/optimize_projections.rs new file mode 100644 index 000000000000..3d0565a6af41 --- /dev/null +++ b/datafusion/optimizer/src/optimize_projections.rs @@ -0,0 +1,848 @@ +// 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. + +//! Optimizer rule to prune unnecessary Columns from the intermediate schemas inside the [LogicalPlan]. +//! This rule +//! - Removes unnecessary columns that are not showed at the output, and that are not used during computation. +//! - Adds projection to decrease table column size before operators that benefits from less memory at its input. +//! - Removes unnecessary [LogicalPlan::Projection] from the [LogicalPlan]. +use crate::optimizer::ApplyOrder; +use datafusion_common::{Column, DFSchema, DFSchemaRef, JoinType, Result}; +use datafusion_expr::expr::{Alias, ScalarFunction}; +use datafusion_expr::{ + logical_plan::LogicalPlan, projection_schema, Aggregate, BinaryExpr, Cast, Distinct, + Expr, Projection, ScalarFunctionDefinition, TableScan, Window, +}; +use hashbrown::HashMap; +use itertools::{izip, Itertools}; +use std::collections::HashSet; +use std::sync::Arc; + +use crate::{OptimizerConfig, OptimizerRule}; + +/// A rule for optimizing logical plans by removing unused Columns/Fields. +/// +/// `OptimizeProjections` is an optimizer rule that identifies and eliminates columns from a logical plan +/// that are not used in any downstream operations. This can improve query performance and reduce unnecessary +/// data processing. +/// +/// The rule analyzes the input logical plan, determines the necessary column indices, and then removes any +/// unnecessary columns. Additionally, it eliminates any unnecessary projections in the plan. +#[derive(Default)] +pub struct OptimizeProjections {} + +impl OptimizeProjections { + #[allow(missing_docs)] + pub fn new() -> Self { + Self {} + } +} + +impl OptimizerRule for OptimizeProjections { + fn try_optimize( + &self, + plan: &LogicalPlan, + config: &dyn OptimizerConfig, + ) -> Result> { + // All of the fields at the output are necessary. + let indices = require_all_indices(plan); + optimize_projections(plan, config, &indices) + } + + fn name(&self) -> &str { + "optimize_projections" + } + + fn apply_order(&self) -> Option { + None + } +} + +/// Removes unnecessary columns (e.g Columns that are not referred at the output schema and +/// Columns that are not used during any computation, expression evaluation) from the logical plan and its inputs. +/// +/// # Arguments +/// +/// - `plan`: A reference to the input `LogicalPlan` to be optimized. +/// - `_config`: A reference to the optimizer configuration (not currently used). +/// - `indices`: A slice of column indices that represent the necessary column indices for downstream operations. +/// +/// # Returns +/// +/// - `Ok(Some(LogicalPlan))`: An optimized `LogicalPlan` with unnecessary columns removed. +/// - `Ok(None)`: If the optimization process results in a logical plan that doesn't require further propagation. +/// - `Err(error)`: If an error occurs during the optimization process. +fn optimize_projections( + plan: &LogicalPlan, + _config: &dyn OptimizerConfig, + indices: &[usize], +) -> Result> { + // `child_required_indices` stores + // - indices of the columns required for each child + // - a flag indicating whether putting a projection above children is beneficial for the parent. + // As an example LogicalPlan::Filter benefits from small tables. Hence for filter child this flag would be `true`. + let child_required_indices: Option, bool)>> = match plan { + LogicalPlan::Sort(_) + | LogicalPlan::Filter(_) + | LogicalPlan::Repartition(_) + | LogicalPlan::Unnest(_) + | LogicalPlan::Union(_) + | LogicalPlan::SubqueryAlias(_) + | LogicalPlan::Distinct(Distinct::On(_)) => { + // Re-route required indices from the parent + column indices referred by expressions in the plan + // to the child. + // All of these operators benefits from small tables at their inputs. Hence projection_beneficial flag is `true`. + let exprs = plan.expressions(); + let child_req_indices = plan + .inputs() + .into_iter() + .map(|input| { + let required_indices = + get_all_required_indices(indices, input, exprs.iter())?; + Ok((required_indices, true)) + }) + .collect::>>()?; + Some(child_req_indices) + } + LogicalPlan::Limit(_) | LogicalPlan::Prepare(_) => { + // Re-route required indices from the parent + column indices referred by expressions in the plan + // to the child. + // Limit, Prepare doesn't benefit from small column numbers. Hence projection_beneficial flag is `false`. + let exprs = plan.expressions(); + let child_req_indices = plan + .inputs() + .into_iter() + .map(|input| { + let required_indices = + get_all_required_indices(indices, input, exprs.iter())?; + Ok((required_indices, false)) + }) + .collect::>>()?; + Some(child_req_indices) + } + LogicalPlan::Copy(_) + | LogicalPlan::Ddl(_) + | LogicalPlan::Dml(_) + | LogicalPlan::Explain(_) + | LogicalPlan::Analyze(_) + | LogicalPlan::Subquery(_) + | LogicalPlan::Distinct(Distinct::All(_)) => { + // Require all of the fields of the Dml, Ddl, Copy, Explain, Analyze, Subquery, Distinct::All input(s). + // Their child plan can be treated as final plan. Otherwise expected schema may not match. + // TODO: For some subquery variants we may not need to require all indices for its input. + // such as Exists. + let child_requirements = plan + .inputs() + .iter() + .map(|input| { + // Require all of the fields for each input. + // No projection since all of the fields at the child is required + (require_all_indices(input), false) + }) + .collect::>(); + Some(child_requirements) + } + LogicalPlan::EmptyRelation(_) + | LogicalPlan::Statement(_) + | LogicalPlan::Values(_) + | LogicalPlan::Extension(_) + | LogicalPlan::DescribeTable(_) => { + // EmptyRelation, Values, DescribeTable, Statement has no inputs stop iteration + + // TODO: Add support for extension + // It is not known how to direct requirements to children for LogicalPlan::Extension. + // Safest behaviour is to stop propagation. + None + } + LogicalPlan::Projection(proj) => { + return if let Some(proj) = merge_consecutive_projections(proj)? { + rewrite_projection_given_requirements(&proj, _config, indices)? + .map(|res| Ok(Some(res))) + // Even if projection cannot be optimized, return merged version + .unwrap_or_else(|| Ok(Some(LogicalPlan::Projection(proj)))) + } else { + rewrite_projection_given_requirements(proj, _config, indices) + }; + } + LogicalPlan::Aggregate(aggregate) => { + // Split parent requirements to group by and aggregate sections + let group_expr_len = aggregate.group_expr_len()?; + let (_group_by_reqs, mut aggregate_reqs): (Vec, Vec) = + indices.iter().partition(|&&idx| idx < group_expr_len); + // Offset aggregate indices so that they point to valid indices at the `aggregate.aggr_expr` + aggregate_reqs + .iter_mut() + .for_each(|idx| *idx -= group_expr_len); + + // Group by expressions are same + let new_group_bys = aggregate.group_expr.clone(); + + // Only use absolutely necessary aggregate expressions required by parent. + let new_aggr_expr = get_at_indices(&aggregate.aggr_expr, &aggregate_reqs); + let all_exprs_iter = new_group_bys.iter().chain(new_aggr_expr.iter()); + let necessary_indices = + indices_referred_by_exprs(&aggregate.input, all_exprs_iter)?; + + let aggregate_input = if let Some(input) = + optimize_projections(&aggregate.input, _config, &necessary_indices)? + { + input + } else { + aggregate.input.as_ref().clone() + }; + + // Simplify input of the aggregation by adding a projection so that its input only contains + // absolutely necessary columns for the aggregate expressions. Please no that we use aggregate.input.schema() + // because necessary_indices refers to fields in this schema. + let necessary_exprs = + get_required_exprs(aggregate.input.schema(), &necessary_indices); + let (aggregate_input, _is_added) = + add_projection_on_top_if_helpful(aggregate_input, necessary_exprs, true)?; + + // Create new aggregate plan with updated input, and absolutely necessary fields. + return Aggregate::try_new( + Arc::new(aggregate_input), + new_group_bys, + new_aggr_expr, + ) + .map(|aggregate| Some(LogicalPlan::Aggregate(aggregate))); + } + LogicalPlan::Window(window) => { + // Split parent requirements to child and window expression sections. + let n_input_fields = window.input.schema().fields().len(); + let (child_reqs, mut window_reqs): (Vec, Vec) = + indices.iter().partition(|&&idx| idx < n_input_fields); + // Offset window expr indices so that they point to valid indices at the `window.window_expr` + window_reqs + .iter_mut() + .for_each(|idx| *idx -= n_input_fields); + + // Only use window expressions that are absolutely necessary by parent requirements. + let new_window_expr = get_at_indices(&window.window_expr, &window_reqs); + + // All of the required column indices at the input of the window by parent, and window expression requirements. + let required_indices = get_all_required_indices( + &child_reqs, + &window.input, + new_window_expr.iter(), + )?; + let window_child = if let Some(new_window_child) = + optimize_projections(&window.input, _config, &required_indices)? + { + new_window_child + } else { + window.input.as_ref().clone() + }; + // When no window expression is necessary, just use window input. (Remove window operator) + return if new_window_expr.is_empty() { + Ok(Some(window_child)) + } else { + // Calculate required expressions at the input of the window. + // Please note that we use `old_child`, because `required_indices` refers to `old_child`. + let required_exprs = + get_required_exprs(window.input.schema(), &required_indices); + let (window_child, _is_added) = + add_projection_on_top_if_helpful(window_child, required_exprs, true)?; + let window = Window::try_new(new_window_expr, Arc::new(window_child))?; + Ok(Some(LogicalPlan::Window(window))) + }; + } + LogicalPlan::Join(join) => { + let left_len = join.left.schema().fields().len(); + let (left_req_indices, right_req_indices) = + split_join_requirements(left_len, indices, &join.join_type); + let exprs = plan.expressions(); + let left_indices = + get_all_required_indices(&left_req_indices, &join.left, exprs.iter())?; + let right_indices = + get_all_required_indices(&right_req_indices, &join.right, exprs.iter())?; + // Join benefits from small columns numbers at its input (decreases memory usage) + // Hence each child benefits from projection. + Some(vec![(left_indices, true), (right_indices, true)]) + } + LogicalPlan::CrossJoin(cross_join) => { + let left_len = cross_join.left.schema().fields().len(); + let (left_child_indices, right_child_indices) = + split_join_requirements(left_len, indices, &JoinType::Inner); + // Join benefits from small columns numbers at its input (decreases memory usage) + // Hence each child benefits from projection. + Some(vec![ + (left_child_indices, true), + (right_child_indices, true), + ]) + } + LogicalPlan::TableScan(table_scan) => { + let projection_fields = table_scan.projected_schema.fields(); + let schema = table_scan.source.schema(); + // We expect to find all of the required indices of the projected schema fields. + // among original schema. If at least one of them cannot be found. Use all of the fields in the file. + // (No projection at the source) + let projection = indices + .iter() + .map(|&idx| { + schema.fields().iter().position(|field_source| { + projection_fields[idx].field() == field_source + }) + }) + .collect::>>(); + + return Ok(Some(LogicalPlan::TableScan(TableScan::try_new( + table_scan.table_name.clone(), + table_scan.source.clone(), + projection, + table_scan.filters.clone(), + table_scan.fetch, + )?))); + } + }; + + let child_required_indices = + if let Some(child_required_indices) = child_required_indices { + child_required_indices + } else { + // Stop iteration, cannot propagate requirement down below this operator. + return Ok(None); + }; + + let new_inputs = izip!(child_required_indices, plan.inputs().into_iter()) + .map(|((required_indices, projection_beneficial), child)| { + let (input, mut is_changed) = if let Some(new_input) = + optimize_projections(child, _config, &required_indices)? + { + (new_input, true) + } else { + (child.clone(), false) + }; + let project_exprs = get_required_exprs(child.schema(), &required_indices); + let (input, is_projection_added) = add_projection_on_top_if_helpful( + input, + project_exprs, + projection_beneficial, + )?; + is_changed |= is_projection_added; + Ok(is_changed.then_some(input)) + }) + .collect::>>>()?; + // All of the children are same in this case, no need to change plan + if new_inputs.iter().all(|child| child.is_none()) { + Ok(None) + } else { + // At least one of the children is changed. + let new_inputs = izip!(new_inputs, plan.inputs()) + // If new_input is `None`, this means child is not changed. Hence use `old_child` during construction. + .map(|(new_input, old_child)| new_input.unwrap_or_else(|| old_child.clone())) + .collect::>(); + let res = plan.with_new_inputs(&new_inputs)?; + Ok(Some(res)) + } +} + +/// Merge Consecutive Projections +/// +/// Given a projection `proj`, this function attempts to merge it with a previous +/// projection if it exists and if the merging is beneficial. Merging is considered +/// beneficial when expressions in the current projection are non-trivial and referred to +/// more than once in its input fields. This can act as a caching mechanism for non-trivial +/// computations. +/// +/// # Arguments +/// +/// * `proj` - A reference to the `Projection` to be merged. +/// +/// # Returns +/// +/// A `Result` containing an `Option` of the merged `Projection`. If merging is not beneficial +/// it returns `Ok(None)`. +fn merge_consecutive_projections(proj: &Projection) -> Result> { + let prev_projection = if let LogicalPlan::Projection(prev) = proj.input.as_ref() { + prev + } else { + return Ok(None); + }; + + // Count usages (referral counts) of each projection expression in its input fields + let column_referral_map: HashMap = proj + .expr + .iter() + .flat_map(|expr| expr.to_columns()) + .fold(HashMap::new(), |mut map, cols| { + cols.into_iter() + .for_each(|col| *map.entry(col.clone()).or_default() += 1); + map + }); + + // Merging these projections is not beneficial, e.g + // If an expression is not trivial and it is referred more than 1, consecutive projections will be + // beneficial as caching mechanism for non-trivial computations. + // See discussion in: https://github.com/apache/arrow-datafusion/issues/8296 + if column_referral_map.iter().any(|(col, usage)| { + *usage > 1 + && !is_expr_trivial( + &prev_projection.expr + [prev_projection.schema.index_of_column(col).unwrap()], + ) + }) { + return Ok(None); + } + + // If all of the expression of the top projection can be rewritten. Rewrite expressions and create a new projection + let new_exprs = proj + .expr + .iter() + .map(|expr| rewrite_expr(expr, prev_projection)) + .collect::>>>()?; + new_exprs + .map(|exprs| Projection::try_new(exprs, prev_projection.input.clone())) + .transpose() +} + +/// Trim Expression +/// +/// Trim the given expression by removing any unnecessary layers of abstraction. +/// If the expression is an alias, the function returns the underlying expression. +/// Otherwise, it returns the original expression unchanged. +/// +/// # Arguments +/// +/// * `expr` - The input expression to be trimmed. +/// +/// # Returns +/// +/// The trimmed expression. If the input is an alias, the underlying expression is returned. +/// +/// Without trimming, during projection merge we can end up unnecessary indirections inside the expressions. +/// Consider: +/// +/// Projection (a1 + b1 as sum1) +/// --Projection (a as a1, b as b1) +/// ----Source (a, b) +/// +/// After merge we want to produce +/// +/// Projection (a + b as sum1) +/// --Source(a, b) +/// +/// Without trimming we would end up +/// +/// Projection (a as a1 + b as b1 as sum1) +/// --Source(a, b) +fn trim_expr(expr: Expr) -> Expr { + match expr { + Expr::Alias(alias) => *alias.expr, + _ => expr, + } +} + +// Check whether expression is trivial (e.g it doesn't include computation.) +fn is_expr_trivial(expr: &Expr) -> bool { + matches!(expr, Expr::Column(_) | Expr::Literal(_)) +} + +// Exit early when None is seen. +macro_rules! rewrite_expr_with_check { + ($expr:expr, $input:expr) => { + if let Some(val) = rewrite_expr($expr, $input)? { + val + } else { + return Ok(None); + } + }; +} + +// Rewrites expression using its input projection (Merges consecutive projection expressions). +/// Rewrites an projections expression using its input projection +/// (Helper during merging consecutive projection expressions). +/// +/// # Arguments +/// +/// * `expr` - A reference to the expression to be rewritten. +/// * `input` - A reference to the input (itself a projection) of the projection expression. +/// +/// # Returns +/// +/// A `Result` containing an `Option` of the rewritten expression. If the rewrite is successful, +/// it returns `Ok(Some)` with the modified expression. If the expression cannot be rewritten +/// it returns `Ok(None)`. +fn rewrite_expr(expr: &Expr, input: &Projection) -> Result> { + Ok(match expr { + Expr::Column(col) => { + // Find index of column + let idx = input.schema.index_of_column(col)?; + Some(input.expr[idx].clone()) + } + Expr::BinaryExpr(binary) => { + let lhs = trim_expr(rewrite_expr_with_check!(&binary.left, input)); + let rhs = trim_expr(rewrite_expr_with_check!(&binary.right, input)); + Some(Expr::BinaryExpr(BinaryExpr::new( + Box::new(lhs), + binary.op, + Box::new(rhs), + ))) + } + Expr::Alias(alias) => { + let new_expr = trim_expr(rewrite_expr_with_check!(&alias.expr, input)); + Some(Expr::Alias(Alias::new( + new_expr, + alias.relation.clone(), + alias.name.clone(), + ))) + } + Expr::Literal(_val) => Some(expr.clone()), + Expr::Cast(cast) => { + let new_expr = rewrite_expr_with_check!(&cast.expr, input); + Some(Expr::Cast(Cast::new( + Box::new(new_expr), + cast.data_type.clone(), + ))) + } + Expr::ScalarFunction(scalar_fn) => { + let fun = if let ScalarFunctionDefinition::BuiltIn { fun, .. } = + scalar_fn.func_def + { + fun + } else { + return Ok(None); + }; + scalar_fn + .args + .iter() + .map(|expr| rewrite_expr(expr, input)) + .collect::>>>()? + .map(|new_args| Expr::ScalarFunction(ScalarFunction::new(fun, new_args))) + } + _ => { + // Unsupported type to merge in consecutive projections + None + } + }) +} + +/// Retrieves a set of outer-referenced columns from an expression. +/// Please note that `expr.to_columns()` API doesn't return these columns. +/// +/// # Arguments +/// +/// * `expr` - The expression to be analyzed for outer-referenced columns. +/// +/// # Returns +/// +/// A `HashSet` containing columns that are referenced by the expression. +fn outer_columns(expr: &Expr) -> HashSet { + let mut columns = HashSet::new(); + outer_columns_helper(expr, &mut columns); + columns +} + +/// Helper function to accumulate outer-referenced columns referred by the `expr`. +/// +/// # Arguments +/// +/// * `expr` - The expression to be analyzed for outer-referenced columns. +/// * `columns` - A mutable reference to a `HashSet` where the detected columns are collected. +fn outer_columns_helper(expr: &Expr, columns: &mut HashSet) { + match expr { + Expr::OuterReferenceColumn(_, col) => { + columns.insert(col.clone()); + } + Expr::BinaryExpr(binary_expr) => { + outer_columns_helper(&binary_expr.left, columns); + outer_columns_helper(&binary_expr.right, columns); + } + Expr::ScalarSubquery(subquery) => { + for expr in &subquery.outer_ref_columns { + outer_columns_helper(expr, columns); + } + } + Expr::Exists(exists) => { + for expr in &exists.subquery.outer_ref_columns { + outer_columns_helper(expr, columns); + } + } + Expr::Alias(alias) => { + outer_columns_helper(&alias.expr, columns); + } + _ => {} + } +} + +/// Generates the required expressions(Column) that resides at `indices` of the `input_schema`. +/// +/// # Arguments +/// +/// * `input_schema` - A reference to the input schema. +/// * `indices` - A slice of `usize` indices specifying which columns are required. +/// +/// # Returns +/// +/// A vector of `Expr::Column` expressions, that sits at `indices` of the `input_schema`. +fn get_required_exprs(input_schema: &Arc, indices: &[usize]) -> Vec { + let fields = input_schema.fields(); + indices + .iter() + .map(|&idx| Expr::Column(fields[idx].qualified_column())) + .collect() +} + +/// Get indices of the necessary fields referred by all of the `exprs` among input LogicalPlan. +/// +/// # Arguments +/// +/// * `input`: The input logical plan to analyze for index requirements. +/// * `exprs`: An iterator of expressions for which we want to find necessary field indices at the input. +/// +/// # Returns +/// +/// A [Result] object that contains the required field indices for the `input` operator, to be able to calculate +/// successfully all of the `exprs`. +fn indices_referred_by_exprs<'a, I: Iterator>( + input: &LogicalPlan, + exprs: I, +) -> Result> { + let new_indices = exprs + .flat_map(|expr| indices_referred_by_expr(input.schema(), expr)) + .flatten() + // Make sure no duplicate entries exists and indices are ordered. + .sorted() + .dedup() + .collect::>(); + Ok(new_indices) +} + +/// Get indices of the necessary fields referred by the `expr` among input schema. +/// +/// # Arguments +/// +/// * `input_schema`: The input schema to search for indices referred by expr. +/// * `expr`: An expression for which we want to find necessary field indices at the input schema. +/// +/// # Returns +/// +/// A [Result] object that contains the required field indices of the `input_schema`, to be able to calculate +/// the `expr` successfully. +fn indices_referred_by_expr( + input_schema: &DFSchemaRef, + expr: &Expr, +) -> Result> { + let mut cols = expr.to_columns()?; + // Get outer referenced columns (expr.to_columns() doesn't return these columns). + cols.extend(outer_columns(expr)); + cols.iter() + .filter(|&col| input_schema.has_column(col)) + .map(|col| input_schema.index_of_column(col)) + .collect::>>() +} + +/// Get all required indices for the input (indices required by parent + indices referred by `exprs`) +/// +/// # Arguments +/// +/// * `parent_required_indices` - A slice of indices required by the parent plan. +/// * `input` - The input logical plan to analyze for index requirements. +/// * `exprs` - An iterator of expressions used to determine required indices. +/// +/// # Returns +/// +/// A `Result` containing a vector of `usize` indices containing all required indices. +fn get_all_required_indices<'a, I: Iterator>( + parent_required_indices: &[usize], + input: &LogicalPlan, + exprs: I, +) -> Result> { + let referred_indices = indices_referred_by_exprs(input, exprs)?; + Ok(merge_vectors(parent_required_indices, &referred_indices)) +} + +/// Retrieves a list of expressions at specified indices from a slice of expressions. +/// +/// This function takes a slice of expressions `exprs` and a slice of `usize` indices `indices`. +/// It returns a new vector containing the expressions from `exprs` that correspond to the provided indices (with bound check). +/// +/// # Arguments +/// +/// * `exprs` - A slice of expressions from which expressions are to be retrieved. +/// * `indices` - A slice of `usize` indices specifying the positions of the expressions to be retrieved. +/// +/// # Returns +/// +/// A vector of expressions that correspond to the specified indices. If any index is out of bounds, +/// the associated expression is skipped in the result. +fn get_at_indices(exprs: &[Expr], indices: &[usize]) -> Vec { + indices + .iter() + // Indices may point to further places than `exprs` len. + .filter_map(|&idx| exprs.get(idx).cloned()) + .collect() +} + +/// Merges two slices of `usize` values into a single vector with sorted (ascending) and deduplicated elements. +/// +/// # Arguments +/// +/// * `lhs` - The first slice of `usize` values to be merged. +/// * `rhs` - The second slice of `usize` values to be merged. +/// +/// # Returns +/// +/// A vector of `usize` values containing the merged, sorted, and deduplicated elements from `lhs` and `rhs`. +/// As an example merge of [3, 2, 4] and [3, 6, 1] will produce [1, 2, 3, 6] +fn merge_vectors(lhs: &[usize], rhs: &[usize]) -> Vec { + let mut merged = lhs.to_vec(); + merged.extend(rhs); + // Make sure to run sort before dedup. + // Dedup removes consecutive same entries + // If sort is run before it, all duplicates are removed. + merged.sort(); + merged.dedup(); + merged +} + +/// Splits requirement indices for a join into left and right children based on the join type. +/// +/// This function takes the length of the left child, a slice of requirement indices, and the type +/// of join (e.g., INNER, LEFT, RIGHT, etc.) as arguments. Depending on the join type, it divides +/// the requirement indices into those that apply to the left child and those that apply to the right child. +/// +/// - For INNER, LEFT, RIGHT, and FULL joins, the requirements are split between left and right children. +/// The right child indices are adjusted to point to valid positions in the right child by subtracting +/// the length of the left child. +/// +/// - For LEFT ANTI, LEFT SEMI, RIGHT SEMI, and RIGHT ANTI joins, all requirements are re-routed to either +/// the left child or the right child directly, depending on the join type. +/// +/// # Arguments +/// +/// * `left_len` - The length of the left child. +/// * `indices` - A slice of requirement indices. +/// * `join_type` - The type of join (e.g., INNER, LEFT, RIGHT, etc.). +/// +/// # Returns +/// +/// A tuple containing two vectors of `usize` indices: the first vector represents the requirements for +/// the left child, and the second vector represents the requirements for the right child. The indices +/// are appropriately split and adjusted based on the join type. +fn split_join_requirements( + left_len: usize, + indices: &[usize], + join_type: &JoinType, +) -> (Vec, Vec) { + match join_type { + // In these cases requirements split to left and right child. + JoinType::Inner | JoinType::Left | JoinType::Right | JoinType::Full => { + let (left_child_reqs, mut right_child_reqs): (Vec, Vec) = + indices.iter().partition(|&&idx| idx < left_len); + // Decrease right side index by `left_len` so that they point to valid positions in the right child. + right_child_reqs.iter_mut().for_each(|idx| *idx -= left_len); + (left_child_reqs, right_child_reqs) + } + // All requirements can be re-routed to left child directly. + JoinType::LeftAnti | JoinType::LeftSemi => (indices.to_vec(), vec![]), + // All requirements can be re-routed to right side directly. (No need to change index, join schema is right child schema.) + JoinType::RightSemi | JoinType::RightAnti => (vec![], indices.to_vec()), + } +} + +/// Adds a projection on top of a logical plan if it is beneficial and reduces the number of columns for the parent operator. +/// +/// This function takes a `LogicalPlan`, a list of projection expressions, and a flag indicating whether +/// the projection is beneficial. If the projection is beneficial and reduces the number of columns in +/// the plan, a new `LogicalPlan` with the projection is created and returned, along with a `true` flag. +/// If the projection is unnecessary or doesn't reduce the number of columns, the original plan is returned +/// with a `false` flag. +/// +/// # Arguments +/// +/// * `plan` - The input `LogicalPlan` to potentially add a projection to. +/// * `project_exprs` - A list of expressions for the projection. +/// * `projection_beneficial` - A flag indicating whether the projection is beneficial. +/// +/// # Returns +/// +/// A `Result` containing a tuple with two values: the resulting `LogicalPlan` (with or without +/// the added projection) and a `bool` flag indicating whether the projection was added (`true`) or not (`false`). +fn add_projection_on_top_if_helpful( + plan: LogicalPlan, + project_exprs: Vec, + projection_beneficial: bool, +) -> Result<(LogicalPlan, bool)> { + // Make sure projection decreases table column size, otherwise it is unnecessary. + if !projection_beneficial || project_exprs.len() >= plan.schema().fields().len() { + Ok((plan, false)) + } else { + let new_plan = Projection::try_new(project_exprs, Arc::new(plan)) + .map(LogicalPlan::Projection)?; + Ok((new_plan, true)) + } +} + +/// Collects and returns a vector of all indices of the fields in the schema of a logical plan. +/// +/// # Arguments +/// +/// * `plan` - A reference to the `LogicalPlan` for which indices are required. +/// +/// # Returns +/// +/// A vector of `usize` indices representing all fields in the schema of the provided logical plan. +fn require_all_indices(plan: &LogicalPlan) -> Vec { + (0..plan.schema().fields().len()).collect() +} + +/// Rewrite Projection Given Required fields by its parent(s). +/// +/// # Arguments +/// +/// * `proj` - A reference to the original projection to be rewritten. +/// * `_config` - A reference to the optimizer configuration (unused in the function). +/// * `indices` - A slice of indices representing the required columns by the parent(s) of projection. +/// +/// # Returns +/// +/// A `Result` containing an `Option` of the rewritten logical plan. If the +/// rewrite is successful, it returns `Some` with the optimized logical plan. +/// If the logical plan remains unchanged it returns `Ok(None)`. +fn rewrite_projection_given_requirements( + proj: &Projection, + _config: &dyn OptimizerConfig, + indices: &[usize], +) -> Result> { + let exprs_used = get_at_indices(&proj.expr, indices); + let required_indices = indices_referred_by_exprs(&proj.input, exprs_used.iter())?; + return if let Some(input) = + optimize_projections(&proj.input, _config, &required_indices)? + { + if &projection_schema(&input, &exprs_used)? == input.schema() { + Ok(Some(input)) + } else { + let new_proj = Projection::try_new(exprs_used, Arc::new(input.clone()))?; + let new_proj = LogicalPlan::Projection(new_proj); + Ok(Some(new_proj)) + } + } else if exprs_used.len() < proj.expr.len() { + // Projection expression used is different than the existing projection + // In this case, even if child doesn't change we should update projection to use less columns. + if &projection_schema(&proj.input, &exprs_used)? == proj.input.schema() { + Ok(Some(proj.input.as_ref().clone())) + } else { + let new_proj = Projection::try_new(exprs_used, proj.input.clone())?; + let new_proj = LogicalPlan::Projection(new_proj); + Ok(Some(new_proj)) + } + } else { + // Projection doesn't change. + Ok(None) + }; +} diff --git a/datafusion/optimizer/src/optimizer.rs b/datafusion/optimizer/src/optimizer.rs index e93565fef0a0..c7ad31f39b00 100644 --- a/datafusion/optimizer/src/optimizer.rs +++ b/datafusion/optimizer/src/optimizer.rs @@ -27,15 +27,13 @@ use crate::eliminate_limit::EliminateLimit; use crate::eliminate_nested_union::EliminateNestedUnion; use crate::eliminate_one_union::EliminateOneUnion; use crate::eliminate_outer_join::EliminateOuterJoin; -use crate::eliminate_project::EliminateProjection; use crate::extract_equijoin_predicate::ExtractEquijoinPredicate; use crate::filter_null_join_keys::FilterNullJoinKeys; -use crate::merge_projection::MergeProjection; +use crate::optimize_projections::OptimizeProjections; use crate::plan_signature::LogicalPlanSignature; use crate::propagate_empty_relation::PropagateEmptyRelation; use crate::push_down_filter::PushDownFilter; use crate::push_down_limit::PushDownLimit; -use crate::push_down_projection::PushDownProjection; use crate::replace_distinct_aggregate::ReplaceDistinctWithAggregate; use crate::rewrite_disjunctive_predicate::RewriteDisjunctivePredicate; use crate::scalar_subquery_to_join::ScalarSubqueryToJoin; @@ -234,7 +232,6 @@ impl Optimizer { // run it again after running the optimizations that potentially converted // subqueries to joins Arc::new(SimplifyExpressions::new()), - Arc::new(MergeProjection::new()), Arc::new(RewriteDisjunctivePredicate::new()), Arc::new(EliminateDuplicatedExpr::new()), Arc::new(EliminateFilter::new()), @@ -255,10 +252,7 @@ impl Optimizer { Arc::new(SimplifyExpressions::new()), Arc::new(UnwrapCastInComparison::new()), Arc::new(CommonSubexprEliminate::new()), - Arc::new(PushDownProjection::new()), - Arc::new(EliminateProjection::new()), - // PushDownProjection can pushdown Projections through Limits, do PushDownLimit again. - Arc::new(PushDownLimit::new()), + Arc::new(OptimizeProjections::new()), ]; Self::with_rules(rules) diff --git a/datafusion/optimizer/src/push_down_projection.rs b/datafusion/optimizer/src/push_down_projection.rs index 59a5357c97dd..bdd66347631c 100644 --- a/datafusion/optimizer/src/push_down_projection.rs +++ b/datafusion/optimizer/src/push_down_projection.rs @@ -18,530 +18,26 @@ //! Projection Push Down optimizer rule ensures that only referenced columns are //! loaded into memory -use std::collections::{BTreeSet, HashMap, HashSet}; -use std::sync::Arc; - -use crate::eliminate_project::can_eliminate; -use crate::merge_projection::merge_projection; -use crate::optimizer::ApplyOrder; -use crate::push_down_filter::replace_cols_by_name; -use crate::{OptimizerConfig, OptimizerRule}; -use arrow::error::Result as ArrowResult; -use datafusion_common::ScalarValue::UInt8; -use datafusion_common::{ - plan_err, Column, DFSchema, DFSchemaRef, DataFusionError, Result, -}; -use datafusion_expr::expr::{AggregateFunction, Alias}; -use datafusion_expr::{ - logical_plan::{Aggregate, LogicalPlan, Projection, TableScan, Union}, - utils::{expr_to_columns, exprlist_to_columns, exprlist_to_fields}, - Expr, LogicalPlanBuilder, SubqueryAlias, -}; - -// if projection is empty return projection-new_plan, else return new_plan. -#[macro_export] -macro_rules! generate_plan { - ($projection_is_empty:expr, $plan:expr, $new_plan:expr) => { - if $projection_is_empty { - $new_plan - } else { - $plan.with_new_inputs(&[$new_plan])? - } - }; -} - -/// Optimizer that removes unused projections and aggregations from plans -/// This reduces both scans and -#[derive(Default)] -pub struct PushDownProjection {} - -impl OptimizerRule for PushDownProjection { - fn try_optimize( - &self, - plan: &LogicalPlan, - _config: &dyn OptimizerConfig, - ) -> Result> { - let projection = match plan { - LogicalPlan::Projection(projection) => projection, - LogicalPlan::Aggregate(agg) => { - let mut required_columns = HashSet::new(); - for e in agg.aggr_expr.iter().chain(agg.group_expr.iter()) { - expr_to_columns(e, &mut required_columns)? - } - let new_expr = get_expr(&required_columns, agg.input.schema())?; - let projection = LogicalPlan::Projection(Projection::try_new( - new_expr, - agg.input.clone(), - )?); - let optimized_child = self - .try_optimize(&projection, _config)? - .unwrap_or(projection); - return Ok(Some(plan.with_new_inputs(&[optimized_child])?)); - } - LogicalPlan::TableScan(scan) if scan.projection.is_none() => { - return Ok(Some(push_down_scan(&HashSet::new(), scan, false)?)); - } - _ => return Ok(None), - }; - - let child_plan = &*projection.input; - let projection_is_empty = projection.expr.is_empty(); - - let new_plan = match child_plan { - LogicalPlan::Projection(child_projection) => { - let new_plan = merge_projection(projection, child_projection)?; - self.try_optimize(&new_plan, _config)?.unwrap_or(new_plan) - } - LogicalPlan::Join(join) => { - // collect column in on/filter in join and projection. - let mut push_columns: HashSet = HashSet::new(); - for e in projection.expr.iter() { - expr_to_columns(e, &mut push_columns)?; - } - for (l, r) in join.on.iter() { - expr_to_columns(l, &mut push_columns)?; - expr_to_columns(r, &mut push_columns)?; - } - if let Some(expr) = &join.filter { - expr_to_columns(expr, &mut push_columns)?; - } - - let new_left = generate_projection( - &push_columns, - join.left.schema(), - join.left.clone(), - )?; - let new_right = generate_projection( - &push_columns, - join.right.schema(), - join.right.clone(), - )?; - let new_join = child_plan.with_new_inputs(&[new_left, new_right])?; - - generate_plan!(projection_is_empty, plan, new_join) - } - LogicalPlan::CrossJoin(join) => { - // collect column in on/filter in join and projection. - let mut push_columns: HashSet = HashSet::new(); - for e in projection.expr.iter() { - expr_to_columns(e, &mut push_columns)?; - } - let new_left = generate_projection( - &push_columns, - join.left.schema(), - join.left.clone(), - )?; - let new_right = generate_projection( - &push_columns, - join.right.schema(), - join.right.clone(), - )?; - let new_join = child_plan.with_new_inputs(&[new_left, new_right])?; - - generate_plan!(projection_is_empty, plan, new_join) - } - LogicalPlan::TableScan(scan) - if !scan.projected_schema.fields().is_empty() => - { - let mut used_columns: HashSet = HashSet::new(); - if projection_is_empty { - push_down_scan(&used_columns, scan, true)? - } else { - for expr in projection.expr.iter() { - expr_to_columns(expr, &mut used_columns)?; - } - let new_scan = push_down_scan(&used_columns, scan, true)?; - - plan.with_new_inputs(&[new_scan])? - } - } - LogicalPlan::Union(union) => { - let mut required_columns = HashSet::new(); - exprlist_to_columns(&projection.expr, &mut required_columns)?; - // When there is no projection, we need to add the first column to the projection - // Because if push empty down, children may output different columns. - if required_columns.is_empty() { - required_columns.insert(union.schema.fields()[0].qualified_column()); - } - // we don't push down projection expr, we just prune columns, so we just push column - // because push expr may cause more cost. - let projection_column_exprs = get_expr(&required_columns, &union.schema)?; - let mut inputs = Vec::with_capacity(union.inputs.len()); - for input in &union.inputs { - let mut replace_map = HashMap::new(); - for (i, field) in input.schema().fields().iter().enumerate() { - replace_map.insert( - union.schema.fields()[i].qualified_name(), - Expr::Column(field.qualified_column()), - ); - } - - let exprs = projection_column_exprs - .iter() - .map(|expr| replace_cols_by_name(expr.clone(), &replace_map)) - .collect::>>()?; - - inputs.push(Arc::new(LogicalPlan::Projection(Projection::try_new( - exprs, - input.clone(), - )?))) - } - // create schema of all used columns - let schema = DFSchema::new_with_metadata( - exprlist_to_fields(&projection_column_exprs, child_plan)?, - union.schema.metadata().clone(), - )?; - let new_union = LogicalPlan::Union(Union { - inputs, - schema: Arc::new(schema), - }); - - generate_plan!(projection_is_empty, plan, new_union) - } - LogicalPlan::SubqueryAlias(subquery_alias) => { - let replace_map = generate_column_replace_map(subquery_alias); - let mut required_columns = HashSet::new(); - exprlist_to_columns(&projection.expr, &mut required_columns)?; - - let new_required_columns = required_columns - .iter() - .map(|c| { - replace_map.get(c).cloned().ok_or_else(|| { - DataFusionError::Internal("replace column failed".to_string()) - }) - }) - .collect::>>()?; - - let new_expr = - get_expr(&new_required_columns, subquery_alias.input.schema())?; - let new_projection = LogicalPlan::Projection(Projection::try_new( - new_expr, - subquery_alias.input.clone(), - )?); - let new_alias = child_plan.with_new_inputs(&[new_projection])?; - - generate_plan!(projection_is_empty, plan, new_alias) - } - LogicalPlan::Aggregate(agg) => { - let mut required_columns = HashSet::new(); - exprlist_to_columns(&projection.expr, &mut required_columns)?; - // Gather all columns needed for expressions in this Aggregate - let mut new_aggr_expr = vec![]; - for e in agg.aggr_expr.iter() { - let column = Column::from_name(e.display_name()?); - if required_columns.contains(&column) { - new_aggr_expr.push(e.clone()); - } - } - - // if new_aggr_expr emtpy and aggr is COUNT(UInt8(1)), push it - if new_aggr_expr.is_empty() && agg.aggr_expr.len() == 1 { - if let Expr::AggregateFunction(AggregateFunction { - fun, args, .. - }) = &agg.aggr_expr[0] - { - if matches!(fun, datafusion_expr::AggregateFunction::Count) - && args.len() == 1 - && args[0] == Expr::Literal(UInt8(Some(1))) - { - new_aggr_expr.push(agg.aggr_expr[0].clone()); - } - } - } - - let new_agg = LogicalPlan::Aggregate(Aggregate::try_new( - agg.input.clone(), - agg.group_expr.clone(), - new_aggr_expr, - )?); - - generate_plan!(projection_is_empty, plan, new_agg) - } - LogicalPlan::Window(window) => { - let mut required_columns = HashSet::new(); - exprlist_to_columns(&projection.expr, &mut required_columns)?; - // Gather all columns needed for expressions in this Window - let mut new_window_expr = vec![]; - for e in window.window_expr.iter() { - let column = Column::from_name(e.display_name()?); - if required_columns.contains(&column) { - new_window_expr.push(e.clone()); - } - } - - if new_window_expr.is_empty() { - // none columns in window expr are needed, remove the window expr - let input = window.input.clone(); - let new_window = restrict_outputs(input.clone(), &required_columns)? - .unwrap_or((*input).clone()); - - generate_plan!(projection_is_empty, plan, new_window) - } else { - let mut referenced_inputs = HashSet::new(); - exprlist_to_columns(&new_window_expr, &mut referenced_inputs)?; - window - .input - .schema() - .fields() - .iter() - .filter(|f| required_columns.contains(&f.qualified_column())) - .for_each(|f| { - referenced_inputs.insert(f.qualified_column()); - }); - - let input = window.input.clone(); - let new_input = restrict_outputs(input.clone(), &referenced_inputs)? - .unwrap_or((*input).clone()); - let new_window = LogicalPlanBuilder::from(new_input) - .window(new_window_expr)? - .build()?; - - generate_plan!(projection_is_empty, plan, new_window) - } - } - LogicalPlan::Filter(filter) => { - if can_eliminate(projection, child_plan.schema()) { - // when projection schema == filter schema, we can commute directly. - let new_proj = - plan.with_new_inputs(&[filter.input.as_ref().clone()])?; - child_plan.with_new_inputs(&[new_proj])? - } else { - let mut required_columns = HashSet::new(); - exprlist_to_columns(&projection.expr, &mut required_columns)?; - exprlist_to_columns( - &[filter.predicate.clone()], - &mut required_columns, - )?; - - let new_expr = get_expr(&required_columns, filter.input.schema())?; - let new_projection = LogicalPlan::Projection(Projection::try_new( - new_expr, - filter.input.clone(), - )?); - let new_filter = child_plan.with_new_inputs(&[new_projection])?; - - generate_plan!(projection_is_empty, plan, new_filter) - } - } - LogicalPlan::Sort(sort) => { - if can_eliminate(projection, child_plan.schema()) { - // can commute - let new_proj = plan.with_new_inputs(&[(*sort.input).clone()])?; - child_plan.with_new_inputs(&[new_proj])? - } else { - let mut required_columns = HashSet::new(); - exprlist_to_columns(&projection.expr, &mut required_columns)?; - exprlist_to_columns(&sort.expr, &mut required_columns)?; - - let new_expr = get_expr(&required_columns, sort.input.schema())?; - let new_projection = LogicalPlan::Projection(Projection::try_new( - new_expr, - sort.input.clone(), - )?); - let new_sort = child_plan.with_new_inputs(&[new_projection])?; - - generate_plan!(projection_is_empty, plan, new_sort) - } - } - LogicalPlan::Limit(limit) => { - // can commute - let new_proj = plan.with_new_inputs(&[limit.input.as_ref().clone()])?; - child_plan.with_new_inputs(&[new_proj])? - } - _ => return Ok(None), - }; - - Ok(Some(new_plan)) - } - - fn name(&self) -> &str { - "push_down_projection" - } - - fn apply_order(&self) -> Option { - Some(ApplyOrder::TopDown) - } -} - -impl PushDownProjection { - #[allow(missing_docs)] - pub fn new() -> Self { - Self {} - } -} - -fn generate_column_replace_map( - subquery_alias: &SubqueryAlias, -) -> HashMap { - subquery_alias - .input - .schema() - .fields() - .iter() - .enumerate() - .map(|(i, field)| { - ( - subquery_alias.schema.fields()[i].qualified_column(), - field.qualified_column(), - ) - }) - .collect() -} - -pub fn collect_projection_expr(projection: &Projection) -> HashMap { - projection - .schema - .fields() - .iter() - .enumerate() - .flat_map(|(i, field)| { - // strip alias, as they should not be part of filters - let expr = match &projection.expr[i] { - Expr::Alias(Alias { expr, .. }) => expr.as_ref().clone(), - expr => expr.clone(), - }; - - // Convert both qualified and unqualified fields - [ - (field.name().clone(), expr.clone()), - (field.qualified_name(), expr), - ] - }) - .collect::>() -} - -/// Get the projection exprs from columns in the order of the schema -fn get_expr(columns: &HashSet, schema: &DFSchemaRef) -> Result> { - let expr = schema - .fields() - .iter() - .flat_map(|field| { - let qc = field.qualified_column(); - let uqc = field.unqualified_column(); - if columns.contains(&qc) || columns.contains(&uqc) { - Some(Expr::Column(qc)) - } else { - None - } - }) - .collect::>(); - if columns.len() != expr.len() { - plan_err!("required columns can't push down, columns: {columns:?}") - } else { - Ok(expr) - } -} - -fn generate_projection( - used_columns: &HashSet, - schema: &DFSchemaRef, - input: Arc, -) -> Result { - let expr = schema - .fields() - .iter() - .flat_map(|field| { - let column = field.qualified_column(); - if used_columns.contains(&column) { - Some(Expr::Column(column)) - } else { - None - } - }) - .collect::>(); - - Ok(LogicalPlan::Projection(Projection::try_new(expr, input)?)) -} - -fn push_down_scan( - used_columns: &HashSet, - scan: &TableScan, - has_projection: bool, -) -> Result { - // once we reach the table scan, we can use the accumulated set of column - // names to construct the set of column indexes in the scan - // - // we discard non-existing columns because some column names are not part of the schema, - // e.g. when the column derives from an aggregation - // - // Use BTreeSet to remove potential duplicates (e.g. union) as - // well as to sort the projection to ensure deterministic behavior - let schema = scan.source.schema(); - let mut projection: BTreeSet = used_columns - .iter() - .filter(|c| { - c.relation.is_none() || c.relation.as_ref().unwrap() == &scan.table_name - }) - .map(|c| schema.index_of(&c.name)) - .filter_map(ArrowResult::ok) - .collect(); - - if !has_projection && projection.is_empty() { - // for table scan without projection, we default to return all columns - projection = schema - .fields() - .iter() - .enumerate() - .map(|(i, _)| i) - .collect::>(); - } - - // Building new projection from BTreeSet - // preserving source projection order if it exists - let projection = if let Some(original_projection) = &scan.projection { - original_projection - .clone() - .into_iter() - .filter(|idx| projection.contains(idx)) - .collect::>() - } else { - projection.into_iter().collect::>() - }; - - TableScan::try_new( - scan.table_name.clone(), - scan.source.clone(), - Some(projection), - scan.filters.clone(), - scan.fetch, - ) - .map(LogicalPlan::TableScan) -} - -fn restrict_outputs( - plan: Arc, - permitted_outputs: &HashSet, -) -> Result> { - let schema = plan.schema(); - if permitted_outputs.len() == schema.fields().len() { - return Ok(None); - } - Ok(Some(generate_projection( - permitted_outputs, - schema, - plan.clone(), - )?)) -} - #[cfg(test)] mod tests { use std::collections::HashMap; + use std::sync::Arc; use std::vec; - use super::*; - use crate::eliminate_project::EliminateProjection; + use crate::optimize_projections::OptimizeProjections; use crate::optimizer::Optimizer; use crate::test::*; use crate::OptimizerContext; use arrow::datatypes::{DataType, Field, Schema}; - use datafusion_common::{DFField, DFSchema}; + use datafusion_common::{Column, DFField, DFSchema, Result}; use datafusion_expr::builder::table_scan_with_filters; use datafusion_expr::expr::{self, Cast}; use datafusion_expr::logical_plan::{ builder::LogicalPlanBuilder, table_scan, JoinType, }; use datafusion_expr::{ - col, count, lit, max, min, AggregateFunction, Expr, WindowFrame, WindowFunction, + col, count, lit, max, min, AggregateFunction, Expr, LogicalPlan, Projection, + WindowFrame, WindowFunction, }; #[test] @@ -867,7 +363,7 @@ mod tests { // Build the LogicalPlan directly (don't use PlanBuilder), so // that the Column references are unqualified (e.g. their // relation is `None`). PlanBuilder resolves the expressions - let expr = vec![col("a"), col("b")]; + let expr = vec![col("test.a"), col("test.b")]; let plan = LogicalPlan::Projection(Projection::try_new(expr, Arc::new(table_scan))?); @@ -1126,24 +622,14 @@ mod tests { } fn optimize(plan: &LogicalPlan) -> Result { - let optimizer = Optimizer::with_rules(vec![ - Arc::new(PushDownProjection::new()), - Arc::new(EliminateProjection::new()), - ]); - let mut optimized_plan = optimizer + let optimizer = Optimizer::with_rules(vec![Arc::new(OptimizeProjections::new())]); + let optimized_plan = optimizer .optimize_recursively( optimizer.rules.get(0).unwrap(), plan, &OptimizerContext::new(), )? .unwrap_or_else(|| plan.clone()); - optimized_plan = optimizer - .optimize_recursively( - optimizer.rules.get(1).unwrap(), - &optimized_plan, - &OptimizerContext::new(), - )? - .unwrap_or(optimized_plan); Ok(optimized_plan) } } diff --git a/datafusion/optimizer/tests/optimizer_integration.rs b/datafusion/optimizer/tests/optimizer_integration.rs index 872071e52fa7..e593b07361e2 100644 --- a/datafusion/optimizer/tests/optimizer_integration.rs +++ b/datafusion/optimizer/tests/optimizer_integration.rs @@ -185,8 +185,9 @@ fn between_date32_plus_interval() -> Result<()> { let plan = test_sql(sql)?; let expected = "Aggregate: groupBy=[[]], aggr=[[COUNT(Int64(1))]]\ - \n Filter: test.col_date32 >= Date32(\"10303\") AND test.col_date32 <= Date32(\"10393\")\ - \n TableScan: test projection=[col_date32]"; + \n Projection: \ + \n Filter: test.col_date32 >= Date32(\"10303\") AND test.col_date32 <= Date32(\"10393\")\ + \n TableScan: test projection=[col_date32]"; assert_eq!(expected, format!("{plan:?}")); Ok(()) } @@ -198,8 +199,9 @@ fn between_date64_plus_interval() -> Result<()> { let plan = test_sql(sql)?; let expected = "Aggregate: groupBy=[[]], aggr=[[COUNT(Int64(1))]]\ - \n Filter: test.col_date64 >= Date64(\"890179200000\") AND test.col_date64 <= Date64(\"897955200000\")\ - \n TableScan: test projection=[col_date64]"; + \n Projection: \ + \n Filter: test.col_date64 >= Date64(\"890179200000\") AND test.col_date64 <= Date64(\"897955200000\")\ + \n TableScan: test projection=[col_date64]"; assert_eq!(expected, format!("{plan:?}")); Ok(()) } diff --git a/datafusion/sqllogictest/test_files/aggregate.slt b/datafusion/sqllogictest/test_files/aggregate.slt index a14c179326bb..88590055484f 100644 --- a/datafusion/sqllogictest/test_files/aggregate.slt +++ b/datafusion/sqllogictest/test_files/aggregate.slt @@ -2799,10 +2799,9 @@ query TT EXPLAIN SELECT c2, c3 FROM aggregate_test_100 group by rollup(c2, c3) limit 3; ---- logical_plan -Projection: aggregate_test_100.c2, aggregate_test_100.c3 ---Limit: skip=0, fetch=3 -----Aggregate: groupBy=[[ROLLUP (aggregate_test_100.c2, aggregate_test_100.c3)]], aggr=[[]] -------TableScan: aggregate_test_100 projection=[c2, c3] +Limit: skip=0, fetch=3 +--Aggregate: groupBy=[[ROLLUP (aggregate_test_100.c2, aggregate_test_100.c3)]], aggr=[[]] +----TableScan: aggregate_test_100 projection=[c2, c3] physical_plan GlobalLimitExec: skip=0, fetch=3 --AggregateExec: mode=Final, gby=[c2@0 as c2, c3@1 as c3], aggr=[], lim=[3] diff --git a/datafusion/sqllogictest/test_files/explain.slt b/datafusion/sqllogictest/test_files/explain.slt index c8eff2f301aa..18792735ffed 100644 --- a/datafusion/sqllogictest/test_files/explain.slt +++ b/datafusion/sqllogictest/test_files/explain.slt @@ -192,7 +192,6 @@ logical_plan after decorrelate_predicate_subquery SAME TEXT AS ABOVE logical_plan after scalar_subquery_to_join SAME TEXT AS ABOVE logical_plan after extract_equijoin_predicate SAME TEXT AS ABOVE logical_plan after simplify_expressions SAME TEXT AS ABOVE -logical_plan after merge_projection SAME TEXT AS ABOVE logical_plan after rewrite_disjunctive_predicate SAME TEXT AS ABOVE logical_plan after eliminate_duplicated_expr SAME TEXT AS ABOVE logical_plan after eliminate_filter SAME TEXT AS ABOVE @@ -209,11 +208,7 @@ logical_plan after single_distinct_aggregation_to_group_by SAME TEXT AS ABOVE logical_plan after simplify_expressions SAME TEXT AS ABOVE logical_plan after unwrap_cast_in_comparison SAME TEXT AS ABOVE logical_plan after common_sub_expression_eliminate SAME TEXT AS ABOVE -logical_plan after push_down_projection -Projection: simple_explain_test.a, simple_explain_test.b, simple_explain_test.c ---TableScan: simple_explain_test projection=[a, b, c] -logical_plan after eliminate_projection TableScan: simple_explain_test projection=[a, b, c] -logical_plan after push_down_limit SAME TEXT AS ABOVE +logical_plan after optimize_projections TableScan: simple_explain_test projection=[a, b, c] logical_plan after eliminate_nested_union SAME TEXT AS ABOVE logical_plan after simplify_expressions SAME TEXT AS ABOVE logical_plan after unwrap_cast_in_comparison SAME TEXT AS ABOVE @@ -223,7 +218,6 @@ logical_plan after decorrelate_predicate_subquery SAME TEXT AS ABOVE logical_plan after scalar_subquery_to_join SAME TEXT AS ABOVE logical_plan after extract_equijoin_predicate SAME TEXT AS ABOVE logical_plan after simplify_expressions SAME TEXT AS ABOVE -logical_plan after merge_projection SAME TEXT AS ABOVE logical_plan after rewrite_disjunctive_predicate SAME TEXT AS ABOVE logical_plan after eliminate_duplicated_expr SAME TEXT AS ABOVE logical_plan after eliminate_filter SAME TEXT AS ABOVE @@ -240,9 +234,7 @@ logical_plan after single_distinct_aggregation_to_group_by SAME TEXT AS ABOVE logical_plan after simplify_expressions SAME TEXT AS ABOVE logical_plan after unwrap_cast_in_comparison SAME TEXT AS ABOVE logical_plan after common_sub_expression_eliminate SAME TEXT AS ABOVE -logical_plan after push_down_projection SAME TEXT AS ABOVE -logical_plan after eliminate_projection SAME TEXT AS ABOVE -logical_plan after push_down_limit SAME TEXT AS ABOVE +logical_plan after optimize_projections SAME TEXT AS ABOVE logical_plan TableScan: simple_explain_test projection=[a, b, c] initial_physical_plan CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/example.csv]]}, projection=[a, b, c], has_header=true initial_physical_plan_with_stats CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/example.csv]]}, projection=[a, b, c], has_header=true, statistics=[Rows=Absent, Bytes=Absent, [(Col[0]:),(Col[1]:),(Col[2]:)]] diff --git a/datafusion/sqllogictest/test_files/limit.slt b/datafusion/sqllogictest/test_files/limit.slt index 9e093336a15d..182195112e87 100644 --- a/datafusion/sqllogictest/test_files/limit.slt +++ b/datafusion/sqllogictest/test_files/limit.slt @@ -361,18 +361,20 @@ EXPLAIN SELECT COUNT(*) FROM (SELECT a FROM t1 WHERE a > 3 LIMIT 3 OFFSET 6); ---- logical_plan Aggregate: groupBy=[[]], aggr=[[COUNT(UInt8(1)) AS COUNT(*)]] ---Limit: skip=6, fetch=3 -----Filter: t1.a > Int32(3) -------TableScan: t1 projection=[a] +--Projection: +----Limit: skip=6, fetch=3 +------Filter: t1.a > Int32(3) +--------TableScan: t1 projection=[a] physical_plan AggregateExec: mode=Final, gby=[], aggr=[COUNT(*)] --CoalescePartitionsExec ----AggregateExec: mode=Partial, gby=[], aggr=[COUNT(*)] ------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 ---------GlobalLimitExec: skip=6, fetch=3 -----------CoalesceBatchesExec: target_batch_size=8192 -------------FilterExec: a@0 > 3 ---------------MemoryExec: partitions=1, partition_sizes=[1] +--------ProjectionExec: expr=[] +----------GlobalLimitExec: skip=6, fetch=3 +------------CoalesceBatchesExec: target_batch_size=8192 +--------------FilterExec: a@0 > 3 +----------------MemoryExec: partitions=1, partition_sizes=[1] query I SELECT COUNT(*) FROM (SELECT a FROM t1 WHERE a > 3 LIMIT 3 OFFSET 6); diff --git a/datafusion/sqllogictest/test_files/subquery.slt b/datafusion/sqllogictest/test_files/subquery.slt index 4729c3f01054..430e676fa477 100644 --- a/datafusion/sqllogictest/test_files/subquery.slt +++ b/datafusion/sqllogictest/test_files/subquery.slt @@ -437,7 +437,7 @@ Projection: t1.t1_id, () AS t2_int ------Projection: t2.t2_int --------Filter: t2.t2_int = outer_ref(t1.t1_int) ----------TableScan: t2 ---TableScan: t1 projection=[t1_id] +--TableScan: t1 projection=[t1_id, t1_int] query TT explain SELECT t1_id from t1 where t1_int = (SELECT t2_int FROM t2 WHERE t2.t2_int = t1.t1_int limit 1) @@ -484,27 +484,29 @@ query TT explain SELECT t1_id, t1_name FROM t1 WHERE EXISTS (SELECT sum(t1.t1_int + t2.t2_id) FROM t2 WHERE t1.t1_name = t2.t2_name) ---- logical_plan -Filter: EXISTS () ---Subquery: -----Projection: SUM(outer_ref(t1.t1_int) + t2.t2_id) -------Aggregate: groupBy=[[]], aggr=[[SUM(CAST(outer_ref(t1.t1_int) + t2.t2_id AS Int64))]] ---------Filter: outer_ref(t1.t1_name) = t2.t2_name -----------TableScan: t2 ---TableScan: t1 projection=[t1_id, t1_name] +Projection: t1.t1_id, t1.t1_name +--Filter: EXISTS () +----Subquery: +------Projection: SUM(outer_ref(t1.t1_int) + t2.t2_id) +--------Aggregate: groupBy=[[]], aggr=[[SUM(CAST(outer_ref(t1.t1_int) + t2.t2_id AS Int64))]] +----------Filter: outer_ref(t1.t1_name) = t2.t2_name +------------TableScan: t2 +----TableScan: t1 projection=[t1_id, t1_name, t1_int] #support_agg_correlated_columns2 query TT explain SELECT t1_id, t1_name FROM t1 WHERE EXISTS (SELECT count(*) FROM t2 WHERE t1.t1_name = t2.t2_name having sum(t1_int + t2_id) >0) ---- logical_plan -Filter: EXISTS () ---Subquery: -----Projection: COUNT(*) -------Filter: SUM(outer_ref(t1.t1_int) + t2.t2_id) > Int64(0) ---------Aggregate: groupBy=[[]], aggr=[[COUNT(UInt8(1)) AS COUNT(*), SUM(CAST(outer_ref(t1.t1_int) + t2.t2_id AS Int64))]] -----------Filter: outer_ref(t1.t1_name) = t2.t2_name -------------TableScan: t2 ---TableScan: t1 projection=[t1_id, t1_name] +Projection: t1.t1_id, t1.t1_name +--Filter: EXISTS () +----Subquery: +------Projection: COUNT(*) +--------Filter: SUM(outer_ref(t1.t1_int) + t2.t2_id) > Int64(0) +----------Aggregate: groupBy=[[]], aggr=[[COUNT(UInt8(1)) AS COUNT(*), SUM(CAST(outer_ref(t1.t1_int) + t2.t2_id AS Int64))]] +------------Filter: outer_ref(t1.t1_name) = t2.t2_name +--------------TableScan: t2 +----TableScan: t1 projection=[t1_id, t1_name, t1_int] #support_join_correlated_columns query TT @@ -1012,3 +1014,31 @@ catan-prod1-daily success catan-prod1-daily high #2 #3 #4 + +statement ok +create table t(a bigint); + +# Result of query below shouldn't depend on +# number of optimization passes +# See issue: https://github.com/apache/arrow-datafusion/issues/8296 +statement ok +set datafusion.optimizer.max_passes = 1; + +query TT +explain select a/2, a/2 + 1 from t +---- +logical_plan +Projection: t.a / Int64(2)Int64(2)t.a AS t.a / Int64(2), t.a / Int64(2)Int64(2)t.a AS t.a / Int64(2) + Int64(1) +--Projection: t.a / Int64(2) AS t.a / Int64(2)Int64(2)t.a +----TableScan: t projection=[a] + +statement ok +set datafusion.optimizer.max_passes = 3; + +query TT +explain select a/2, a/2 + 1 from t +---- +logical_plan +Projection: t.a / Int64(2)Int64(2)t.a AS t.a / Int64(2), t.a / Int64(2)Int64(2)t.a AS t.a / Int64(2) + Int64(1) +--Projection: t.a / Int64(2) AS t.a / Int64(2)Int64(2)t.a +----TableScan: t projection=[a] diff --git a/datafusion/sqllogictest/test_files/window.slt b/datafusion/sqllogictest/test_files/window.slt index 55b8843a0b9c..b2491478d84e 100644 --- a/datafusion/sqllogictest/test_files/window.slt +++ b/datafusion/sqllogictest/test_files/window.slt @@ -1731,26 +1731,28 @@ logical_plan Projection: COUNT(*) AS global_count --Aggregate: groupBy=[[]], aggr=[[COUNT(UInt8(1)) AS COUNT(*)]] ----SubqueryAlias: a -------Sort: aggregate_test_100.c1 ASC NULLS LAST ---------Aggregate: groupBy=[[aggregate_test_100.c1]], aggr=[[]] -----------Projection: aggregate_test_100.c1 -------------Filter: aggregate_test_100.c13 != Utf8("C2GT5KVyOPZpgKVl110TyZO0NcJ434") ---------------TableScan: aggregate_test_100 projection=[c1, c13], partial_filters=[aggregate_test_100.c13 != Utf8("C2GT5KVyOPZpgKVl110TyZO0NcJ434")] +------Projection: +--------Sort: aggregate_test_100.c1 ASC NULLS LAST +----------Aggregate: groupBy=[[aggregate_test_100.c1]], aggr=[[]] +------------Projection: aggregate_test_100.c1 +--------------Filter: aggregate_test_100.c13 != Utf8("C2GT5KVyOPZpgKVl110TyZO0NcJ434") +----------------TableScan: aggregate_test_100 projection=[c1, c13], partial_filters=[aggregate_test_100.c13 != Utf8("C2GT5KVyOPZpgKVl110TyZO0NcJ434")] physical_plan ProjectionExec: expr=[COUNT(*)@0 as global_count] --AggregateExec: mode=Final, gby=[], aggr=[COUNT(*)] ----CoalescePartitionsExec ------AggregateExec: mode=Partial, gby=[], aggr=[COUNT(*)] --------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=2 -----------AggregateExec: mode=FinalPartitioned, gby=[c1@0 as c1], aggr=[] -------------CoalesceBatchesExec: target_batch_size=4096 ---------------RepartitionExec: partitioning=Hash([c1@0], 2), input_partitions=2 -----------------AggregateExec: mode=Partial, gby=[c1@0 as c1], aggr=[] -------------------ProjectionExec: expr=[c1@0 as c1] ---------------------CoalesceBatchesExec: target_batch_size=4096 -----------------------FilterExec: c13@1 != C2GT5KVyOPZpgKVl110TyZO0NcJ434 -------------------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 ---------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1, c13], has_header=true +----------ProjectionExec: expr=[] +------------AggregateExec: mode=FinalPartitioned, gby=[c1@0 as c1], aggr=[] +--------------CoalesceBatchesExec: target_batch_size=4096 +----------------RepartitionExec: partitioning=Hash([c1@0], 2), input_partitions=2 +------------------AggregateExec: mode=Partial, gby=[c1@0 as c1], aggr=[] +--------------------ProjectionExec: expr=[c1@0 as c1] +----------------------CoalesceBatchesExec: target_batch_size=4096 +------------------------FilterExec: c13@1 != C2GT5KVyOPZpgKVl110TyZO0NcJ434 +--------------------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +----------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1, c13], has_header=true query I SELECT count(*) as global_count FROM diff --git a/datafusion/substrait/tests/cases/roundtrip_logical_plan.rs b/datafusion/substrait/tests/cases/roundtrip_logical_plan.rs index cee3a346495b..1c5dbe9ce884 100644 --- a/datafusion/substrait/tests/cases/roundtrip_logical_plan.rs +++ b/datafusion/substrait/tests/cases/roundtrip_logical_plan.rs @@ -501,10 +501,11 @@ async fn simple_intersect() -> Result<()> { assert_expected_plan( "SELECT COUNT(*) FROM (SELECT data.a FROM data INTERSECT SELECT data2.a FROM data2);", "Aggregate: groupBy=[[]], aggr=[[COUNT(UInt8(1))]]\ - \n LeftSemi Join: data.a = data2.a\ - \n Aggregate: groupBy=[[data.a]], aggr=[[]]\ - \n TableScan: data projection=[a]\ - \n TableScan: data2 projection=[a]", + \n Projection: \ + \n LeftSemi Join: data.a = data2.a\ + \n Aggregate: groupBy=[[data.a]], aggr=[[]]\ + \n TableScan: data projection=[a]\ + \n TableScan: data2 projection=[a]", ) .await } @@ -514,10 +515,11 @@ async fn simple_intersect_table_reuse() -> Result<()> { assert_expected_plan( "SELECT COUNT(*) FROM (SELECT data.a FROM data INTERSECT SELECT data.a FROM data);", "Aggregate: groupBy=[[]], aggr=[[COUNT(UInt8(1))]]\ - \n LeftSemi Join: data.a = data.a\ - \n Aggregate: groupBy=[[data.a]], aggr=[[]]\ - \n TableScan: data projection=[a]\ - \n TableScan: data projection=[a]", + \n Projection: \ + \n LeftSemi Join: data.a = data.a\ + \n Aggregate: groupBy=[[data.a]], aggr=[[]]\ + \n TableScan: data projection=[a]\ + \n TableScan: data projection=[a]", ) .await } From 4c914ea1e5d3dc61f3552adcffb8356c90d73bac Mon Sep 17 00:00:00 2001 From: Mustafa Akur <106137913+mustafasrepo@users.noreply.github.com> Date: Wed, 29 Nov 2023 15:11:34 +0300 Subject: [PATCH 326/572] Move merge projections tests to under optimize projections (#8352) --- datafusion/optimizer/src/lib.rs | 1 - datafusion/optimizer/src/merge_projection.rs | 73 ------------------- .../optimizer/src/optimize_projections.rs | 57 +++++++++++++++ 3 files changed, 57 insertions(+), 74 deletions(-) delete mode 100644 datafusion/optimizer/src/merge_projection.rs diff --git a/datafusion/optimizer/src/lib.rs b/datafusion/optimizer/src/lib.rs index d8b0c14589a2..b54facc5d682 100644 --- a/datafusion/optimizer/src/lib.rs +++ b/datafusion/optimizer/src/lib.rs @@ -29,7 +29,6 @@ pub mod eliminate_one_union; pub mod eliminate_outer_join; pub mod extract_equijoin_predicate; pub mod filter_null_join_keys; -pub mod merge_projection; pub mod optimize_projections; pub mod optimizer; pub mod propagate_empty_relation; diff --git a/datafusion/optimizer/src/merge_projection.rs b/datafusion/optimizer/src/merge_projection.rs deleted file mode 100644 index f7b750011e44..000000000000 --- a/datafusion/optimizer/src/merge_projection.rs +++ /dev/null @@ -1,73 +0,0 @@ -// Licensed to the Apache Software Foundation (ASF) under one -// or more contributor license agreements. See the NOTICE file -// distributed with this work for additional information -// regarding copyright ownership. The ASF licenses this file -// to you under the Apache License, Version 2.0 (the -// "License"); you may not use this file except in compliance -// with the License. You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, -// software distributed under the License is distributed on an -// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -// KIND, either express or implied. See the License for the -// specific language governing permissions and limitations -// under the License. - -#[cfg(test)] -mod tests { - use crate::optimize_projections::OptimizeProjections; - use datafusion_common::Result; - use datafusion_expr::{ - binary_expr, col, lit, logical_plan::builder::LogicalPlanBuilder, LogicalPlan, - Operator, - }; - use std::sync::Arc; - - use crate::test::*; - - fn assert_optimized_plan_equal(plan: &LogicalPlan, expected: &str) -> Result<()> { - assert_optimized_plan_eq(Arc::new(OptimizeProjections::new()), plan, expected) - } - - #[test] - fn merge_two_projection() -> Result<()> { - let table_scan = test_table_scan()?; - let plan = LogicalPlanBuilder::from(table_scan) - .project(vec![col("a")])? - .project(vec![binary_expr(lit(1), Operator::Plus, col("a"))])? - .build()?; - - let expected = "Projection: Int32(1) + test.a\ - \n TableScan: test projection=[a]"; - assert_optimized_plan_equal(&plan, expected) - } - - #[test] - fn merge_three_projection() -> Result<()> { - let table_scan = test_table_scan()?; - let plan = LogicalPlanBuilder::from(table_scan) - .project(vec![col("a"), col("b")])? - .project(vec![col("a")])? - .project(vec![binary_expr(lit(1), Operator::Plus, col("a"))])? - .build()?; - - let expected = "Projection: Int32(1) + test.a\ - \n TableScan: test projection=[a]"; - assert_optimized_plan_equal(&plan, expected) - } - - #[test] - fn merge_alias() -> Result<()> { - let table_scan = test_table_scan()?; - let plan = LogicalPlanBuilder::from(table_scan) - .project(vec![col("a")])? - .project(vec![col("a").alias("alias")])? - .build()?; - - let expected = "Projection: test.a AS alias\ - \n TableScan: test projection=[a]"; - assert_optimized_plan_equal(&plan, expected) - } -} diff --git a/datafusion/optimizer/src/optimize_projections.rs b/datafusion/optimizer/src/optimize_projections.rs index 3d0565a6af41..1e98ee76d2d9 100644 --- a/datafusion/optimizer/src/optimize_projections.rs +++ b/datafusion/optimizer/src/optimize_projections.rs @@ -846,3 +846,60 @@ fn rewrite_projection_given_requirements( Ok(None) }; } + +#[cfg(test)] +mod tests { + use crate::optimize_projections::OptimizeProjections; + use datafusion_common::Result; + use datafusion_expr::{ + binary_expr, col, lit, logical_plan::builder::LogicalPlanBuilder, LogicalPlan, + Operator, + }; + use std::sync::Arc; + + use crate::test::*; + + fn assert_optimized_plan_equal(plan: &LogicalPlan, expected: &str) -> Result<()> { + assert_optimized_plan_eq(Arc::new(OptimizeProjections::new()), plan, expected) + } + + #[test] + fn merge_two_projection() -> Result<()> { + let table_scan = test_table_scan()?; + let plan = LogicalPlanBuilder::from(table_scan) + .project(vec![col("a")])? + .project(vec![binary_expr(lit(1), Operator::Plus, col("a"))])? + .build()?; + + let expected = "Projection: Int32(1) + test.a\ + \n TableScan: test projection=[a]"; + assert_optimized_plan_equal(&plan, expected) + } + + #[test] + fn merge_three_projection() -> Result<()> { + let table_scan = test_table_scan()?; + let plan = LogicalPlanBuilder::from(table_scan) + .project(vec![col("a"), col("b")])? + .project(vec![col("a")])? + .project(vec![binary_expr(lit(1), Operator::Plus, col("a"))])? + .build()?; + + let expected = "Projection: Int32(1) + test.a\ + \n TableScan: test projection=[a]"; + assert_optimized_plan_equal(&plan, expected) + } + + #[test] + fn merge_alias() -> Result<()> { + let table_scan = test_table_scan()?; + let plan = LogicalPlanBuilder::from(table_scan) + .project(vec![col("a")])? + .project(vec![col("a").alias("alias")])? + .build()?; + + let expected = "Projection: test.a AS alias\ + \n TableScan: test projection=[a]"; + assert_optimized_plan_equal(&plan, expected) + } +} From aeb012e78115bf69d9a58407ac24bc20bd9e0bf0 Mon Sep 17 00:00:00 2001 From: Asura7969 <1402357969@qq.com> Date: Wed, 29 Nov 2023 23:50:18 +0800 Subject: [PATCH 327/572] Add `quote` and `escape` attributes to create csv external table (#8351) * Minor: Improve the document format of JoinHashMap * sql csv_with_quote_escape * fix --- .../common/src/file_options/csv_writer.rs | 6 ++ .../src/datasource/listing_table_factory.rs | 16 +++-- datafusion/core/tests/data/escape.csv | 11 ++++ datafusion/core/tests/data/quote.csv | 11 ++++ .../sqllogictest/test_files/csv_files.slt | 65 +++++++++++++++++++ 5 files changed, 105 insertions(+), 4 deletions(-) create mode 100644 datafusion/core/tests/data/escape.csv create mode 100644 datafusion/core/tests/data/quote.csv create mode 100644 datafusion/sqllogictest/test_files/csv_files.slt diff --git a/datafusion/common/src/file_options/csv_writer.rs b/datafusion/common/src/file_options/csv_writer.rs index fef4a1d21b4b..d6046f0219dd 100644 --- a/datafusion/common/src/file_options/csv_writer.rs +++ b/datafusion/common/src/file_options/csv_writer.rs @@ -91,6 +91,12 @@ impl TryFrom<(&ConfigOptions, &StatementOptions)> for CsvWriterOptions { ) })?) }, + "quote" | "escape" => { + // https://github.com/apache/arrow-rs/issues/5146 + // These two attributes are only available when reading csv files. + // To avoid error + builder + }, _ => return Err(DataFusionError::Configuration(format!("Found unsupported option {option} with value {value} for CSV format!"))) } } diff --git a/datafusion/core/src/datasource/listing_table_factory.rs b/datafusion/core/src/datasource/listing_table_factory.rs index 543a3a83f7c5..f70a82035108 100644 --- a/datafusion/core/src/datasource/listing_table_factory.rs +++ b/datafusion/core/src/datasource/listing_table_factory.rs @@ -67,12 +67,20 @@ impl TableProviderFactory for ListingTableFactory { let file_extension = get_extension(cmd.location.as_str()); let file_format: Arc = match file_type { - FileType::CSV => Arc::new( - CsvFormat::default() + FileType::CSV => { + let mut statement_options = StatementOptions::from(&cmd.options); + let mut csv_format = CsvFormat::default() .with_has_header(cmd.has_header) .with_delimiter(cmd.delimiter as u8) - .with_file_compression_type(file_compression_type), - ), + .with_file_compression_type(file_compression_type); + if let Some(quote) = statement_options.take_str_option("quote") { + csv_format = csv_format.with_quote(quote.as_bytes()[0]) + } + if let Some(escape) = statement_options.take_str_option("escape") { + csv_format = csv_format.with_escape(Some(escape.as_bytes()[0])) + } + Arc::new(csv_format) + } #[cfg(feature = "parquet")] FileType::PARQUET => Arc::new(ParquetFormat::default()), FileType::AVRO => Arc::new(AvroFormat), diff --git a/datafusion/core/tests/data/escape.csv b/datafusion/core/tests/data/escape.csv new file mode 100644 index 000000000000..331a1e697329 --- /dev/null +++ b/datafusion/core/tests/data/escape.csv @@ -0,0 +1,11 @@ +c1,c2 +"id0","value\"0" +"id1","value\"1" +"id2","value\"2" +"id3","value\"3" +"id4","value\"4" +"id5","value\"5" +"id6","value\"6" +"id7","value\"7" +"id8","value\"8" +"id9","value\"9" diff --git a/datafusion/core/tests/data/quote.csv b/datafusion/core/tests/data/quote.csv new file mode 100644 index 000000000000..d81488436409 --- /dev/null +++ b/datafusion/core/tests/data/quote.csv @@ -0,0 +1,11 @@ +c1,c2 +~id0~,~value0~ +~id1~,~value1~ +~id2~,~value2~ +~id3~,~value3~ +~id4~,~value4~ +~id5~,~value5~ +~id6~,~value6~ +~id7~,~value7~ +~id8~,~value8~ +~id9~,~value9~ diff --git a/datafusion/sqllogictest/test_files/csv_files.slt b/datafusion/sqllogictest/test_files/csv_files.slt new file mode 100644 index 000000000000..9facb064bf32 --- /dev/null +++ b/datafusion/sqllogictest/test_files/csv_files.slt @@ -0,0 +1,65 @@ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at + +# http://www.apache.org/licenses/LICENSE-2.0 + +# Unless required by applicable law or agreed to in writing, +# software distributed under the License is distributed on an +# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +# KIND, either express or implied. See the License for the +# specific language governing permissions and limitations +# under the License. + +# create_external_table_with_quote_escape +statement ok +CREATE EXTERNAL TABLE csv_with_quote ( +c1 VARCHAR, +c2 VARCHAR +) STORED AS CSV +WITH HEADER ROW +DELIMITER ',' +OPTIONS ('quote' '~') +LOCATION '../core/tests/data/quote.csv'; + +statement ok +CREATE EXTERNAL TABLE csv_with_escape ( +c1 VARCHAR, +c2 VARCHAR +) STORED AS CSV +WITH HEADER ROW +DELIMITER ',' +OPTIONS ('escape' '\"') +LOCATION '../core/tests/data/escape.csv'; + +query TT +select * from csv_with_quote; +---- +id0 value0 +id1 value1 +id2 value2 +id3 value3 +id4 value4 +id5 value5 +id6 value6 +id7 value7 +id8 value8 +id9 value9 + +query TT +select * from csv_with_escape; +---- +id0 value"0 +id1 value"1 +id2 value"2 +id3 value"3 +id4 value"4 +id5 value"5 +id6 value"6 +id7 value"7 +id8 value"8 +id9 value"9 From d22403a062e9d1f0cdb89b0e80cc05d6318b4e12 Mon Sep 17 00:00:00 2001 From: Andrew Lamb Date: Wed, 29 Nov 2023 11:12:29 -0500 Subject: [PATCH 328/572] Minor: Add DataFrame test (#8341) * Minor: restore DataFrame test * Move test to a better location * simplify test --- datafusion/core/src/physical_planner.rs | 21 +++++++++++++++++++++ 1 file changed, 21 insertions(+) diff --git a/datafusion/core/src/physical_planner.rs b/datafusion/core/src/physical_planner.rs index 09f0e11dc2b5..e0f1201aea01 100644 --- a/datafusion/core/src/physical_planner.rs +++ b/datafusion/core/src/physical_planner.rs @@ -2540,6 +2540,27 @@ mod tests { Ok(()) } + #[tokio::test] + async fn aggregate_with_alias() -> Result<()> { + let schema = Arc::new(Schema::new(vec![ + Field::new("c1", DataType::Utf8, false), + Field::new("c2", DataType::UInt32, false), + ])); + + let logical_plan = scan_empty(None, schema.as_ref(), None)? + .aggregate(vec![col("c1")], vec![sum(col("c2"))])? + .project(vec![col("c1"), sum(col("c2")).alias("total_salary")])? + .build()?; + + let physical_plan = plan(&logical_plan).await?; + assert_eq!("c1", physical_plan.schema().field(0).name().as_str()); + assert_eq!( + "total_salary", + physical_plan.schema().field(1).name().as_str() + ); + Ok(()) + } + #[tokio::test] async fn test_explain() { let schema = Schema::new(vec![Field::new("id", DataType::Int32, false)]); From e93f8e13a7f34f1d17f299ffcc1cbb103246d602 Mon Sep 17 00:00:00 2001 From: Alex Huang Date: Wed, 29 Nov 2023 17:40:49 +0100 Subject: [PATCH 329/572] clean up the code based on Clippy (#8359) --- datafusion/optimizer/src/optimize_projections.rs | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/datafusion/optimizer/src/optimize_projections.rs b/datafusion/optimizer/src/optimize_projections.rs index 1e98ee76d2d9..b6d026279aa6 100644 --- a/datafusion/optimizer/src/optimize_projections.rs +++ b/datafusion/optimizer/src/optimize_projections.rs @@ -381,7 +381,7 @@ fn merge_consecutive_projections(proj: &Projection) -> Result .flat_map(|expr| expr.to_columns()) .fold(HashMap::new(), |mut map, cols| { cols.into_iter() - .for_each(|col| *map.entry(col.clone()).or_default() += 1); + .for_each(|col| *map.entry(col).or_default() += 1); map }); @@ -827,7 +827,7 @@ fn rewrite_projection_given_requirements( if &projection_schema(&input, &exprs_used)? == input.schema() { Ok(Some(input)) } else { - let new_proj = Projection::try_new(exprs_used, Arc::new(input.clone()))?; + let new_proj = Projection::try_new(exprs_used, Arc::new(input))?; let new_proj = LogicalPlan::Projection(new_proj); Ok(Some(new_proj)) } From bbec7870e69d130690690238e4b1d6bd49c31cac Mon Sep 17 00:00:00 2001 From: Andrew Lamb Date: Wed, 29 Nov 2023 12:18:04 -0500 Subject: [PATCH 330/572] Minor: Make it easier to work with Expr::ScalarFunction (#8350) --- datafusion/core/src/physical_planner.rs | 6 +++--- datafusion/expr/src/expr.rs | 15 ++++++++++----- datafusion/substrait/src/logical_plan/producer.rs | 12 ++++++------ 3 files changed, 19 insertions(+), 14 deletions(-) diff --git a/datafusion/core/src/physical_planner.rs b/datafusion/core/src/physical_planner.rs index e0f1201aea01..ef364c22ee7d 100644 --- a/datafusion/core/src/physical_planner.rs +++ b/datafusion/core/src/physical_planner.rs @@ -217,13 +217,13 @@ fn create_physical_name(e: &Expr, is_first_expr: bool) -> Result { Ok(name) } - Expr::ScalarFunction(expr::ScalarFunction { func_def, args }) => { + Expr::ScalarFunction(fun) => { // function should be resolved during `AnalyzerRule`s - if let ScalarFunctionDefinition::Name(_) = func_def { + if let ScalarFunctionDefinition::Name(_) = fun.func_def { return internal_err!("Function `Expr` with name should be resolved."); } - create_function_physical_name(func_def.name(), false, args) + create_function_physical_name(fun.name(), false, &fun.args) } Expr::WindowFunction(WindowFunction { fun, args, .. }) => { create_function_physical_name(&fun.to_string(), false, args) diff --git a/datafusion/expr/src/expr.rs b/datafusion/expr/src/expr.rs index 13e488dac042..b46d204faafb 100644 --- a/datafusion/expr/src/expr.rs +++ b/datafusion/expr/src/expr.rs @@ -362,6 +362,13 @@ pub struct ScalarFunction { pub args: Vec, } +impl ScalarFunction { + // return the Function's name + pub fn name(&self) -> &str { + self.func_def.name() + } +} + impl ScalarFunctionDefinition { /// Function's name for display pub fn name(&self) -> &str { @@ -1219,8 +1226,8 @@ impl fmt::Display for Expr { write!(f, " NULLS LAST") } } - Expr::ScalarFunction(ScalarFunction { func_def, args }) => { - fmt_function(f, func_def.name(), false, args, true) + Expr::ScalarFunction(fun) => { + fmt_function(f, fun.name(), false, &fun.args, true) } Expr::WindowFunction(WindowFunction { fun, @@ -1552,9 +1559,7 @@ fn create_name(e: &Expr) -> Result { } } } - Expr::ScalarFunction(ScalarFunction { func_def, args }) => { - create_function_name(func_def.name(), false, args) - } + Expr::ScalarFunction(fun) => create_function_name(fun.name(), false, &fun.args), Expr::WindowFunction(WindowFunction { fun, args, diff --git a/datafusion/substrait/src/logical_plan/producer.rs b/datafusion/substrait/src/logical_plan/producer.rs index 95604e6d2db9..2be3e7b4e884 100644 --- a/datafusion/substrait/src/logical_plan/producer.rs +++ b/datafusion/substrait/src/logical_plan/producer.rs @@ -33,8 +33,8 @@ use datafusion::common::{exec_err, internal_err, not_impl_err}; #[allow(unused_imports)] use datafusion::logical_expr::aggregate_function; use datafusion::logical_expr::expr::{ - Alias, BinaryExpr, Case, Cast, GroupingSet, InList, - ScalarFunction as DFScalarFunction, ScalarFunctionDefinition, Sort, WindowFunction, + Alias, BinaryExpr, Case, Cast, GroupingSet, InList, ScalarFunctionDefinition, Sort, + WindowFunction, }; use datafusion::logical_expr::{expr, Between, JoinConstraint, LogicalPlan, Operator}; use datafusion::prelude::Expr; @@ -822,9 +822,9 @@ pub fn to_substrait_rex( Ok(substrait_or_list) } } - Expr::ScalarFunction(DFScalarFunction { func_def, args }) => { + Expr::ScalarFunction(fun) => { let mut arguments: Vec = vec![]; - for arg in args { + for arg in &fun.args { arguments.push(FunctionArgument { arg_type: Some(ArgType::Value(to_substrait_rex( arg, @@ -836,12 +836,12 @@ pub fn to_substrait_rex( } // function should be resolved during `AnalyzerRule` - if let ScalarFunctionDefinition::Name(_) = func_def { + if let ScalarFunctionDefinition::Name(_) = fun.func_def { return internal_err!("Function `Expr` with name should be resolved."); } let function_anchor = - _register_function(func_def.name().to_string(), extension_info); + _register_function(fun.name().to_string(), extension_info); Ok(Expression { rex_type: Some(RexType::ScalarFunction(ScalarFunction { function_reference: function_anchor, From 11f164c5f3568c3d4fa796bb63a7db36a7b1e821 Mon Sep 17 00:00:00 2001 From: Jesse Date: Wed, 29 Nov 2023 19:07:26 +0100 Subject: [PATCH 331/572] Move some datafusion-optimizer::utils down to datafusion-expr::utils (#8354) These utils manipulate `LogicalPlan`s and `Expr`s and may be useful in projects that only depend on `datafusion-expr` --- benchmarks/src/parquet_filter.rs | 2 +- .../core/src/datasource/listing/table.rs | 4 +- .../core/tests/parquet/filter_pushdown.rs | 2 +- datafusion/expr/src/utils.rs | 381 ++++++++++++++++- datafusion/optimizer/src/analyzer/subquery.rs | 3 +- .../optimizer/src/analyzer/type_coercion.rs | 7 +- datafusion/optimizer/src/decorrelate.rs | 5 +- .../src/decorrelate_predicate_subquery.rs | 3 +- .../src/extract_equijoin_predicate.rs | 3 +- datafusion/optimizer/src/optimizer.rs | 2 +- datafusion/optimizer/src/push_down_filter.rs | 16 +- .../optimizer/src/scalar_subquery_to_join.rs | 3 +- .../simplify_expressions/simplify_exprs.rs | 2 +- .../src/unwrap_cast_in_comparison.rs | 2 +- datafusion/optimizer/src/utils.rs | 399 +++++------------- .../substrait/src/logical_plan/consumer.rs | 2 +- 16 files changed, 499 insertions(+), 337 deletions(-) diff --git a/benchmarks/src/parquet_filter.rs b/benchmarks/src/parquet_filter.rs index e19596b80f54..1d816908e2b0 100644 --- a/benchmarks/src/parquet_filter.rs +++ b/benchmarks/src/parquet_filter.rs @@ -19,8 +19,8 @@ use crate::AccessLogOpt; use crate::{BenchmarkRun, CommonOpt}; use arrow::util::pretty; use datafusion::common::Result; +use datafusion::logical_expr::utils::disjunction; use datafusion::logical_expr::{lit, or, Expr}; -use datafusion::optimizer::utils::disjunction; use datafusion::physical_plan::collect; use datafusion::prelude::{col, SessionContext}; use datafusion::test_util::parquet::{ParquetScanOptions, TestParquetFile}; diff --git a/datafusion/core/src/datasource/listing/table.rs b/datafusion/core/src/datasource/listing/table.rs index 515bc8a9e612..a3be57db3a83 100644 --- a/datafusion/core/src/datasource/listing/table.rs +++ b/datafusion/core/src/datasource/listing/table.rs @@ -40,11 +40,10 @@ use crate::datasource::{ physical_plan::{is_plan_streaming, FileScanConfig, FileSinkConfig}, TableProvider, TableType, }; -use crate::logical_expr::TableProviderFilterPushDown; use crate::{ error::{DataFusionError, Result}, execution::context::SessionState, - logical_expr::Expr, + logical_expr::{utils::conjunction, Expr, TableProviderFilterPushDown}, physical_plan::{empty::EmptyExec, ExecutionPlan, Statistics}, }; @@ -56,7 +55,6 @@ use datafusion_common::{ }; use datafusion_execution::cache::cache_manager::FileStatisticsCache; use datafusion_execution::cache::cache_unit::DefaultFileStatisticsCache; -use datafusion_optimizer::utils::conjunction; use datafusion_physical_expr::{ create_physical_expr, LexOrdering, PhysicalSortRequirement, }; diff --git a/datafusion/core/tests/parquet/filter_pushdown.rs b/datafusion/core/tests/parquet/filter_pushdown.rs index 61a8f87b9ea5..f214e8903a4f 100644 --- a/datafusion/core/tests/parquet/filter_pushdown.rs +++ b/datafusion/core/tests/parquet/filter_pushdown.rs @@ -34,7 +34,7 @@ use datafusion::physical_plan::collect; use datafusion::physical_plan::metrics::MetricsSet; use datafusion::prelude::{col, lit, lit_timestamp_nano, Expr, SessionContext}; use datafusion::test_util::parquet::{ParquetScanOptions, TestParquetFile}; -use datafusion_optimizer::utils::{conjunction, disjunction, split_conjunction}; +use datafusion_expr::utils::{conjunction, disjunction, split_conjunction}; use itertools::Itertools; use parquet::file::properties::WriterProperties; use tempfile::TempDir; diff --git a/datafusion/expr/src/utils.rs b/datafusion/expr/src/utils.rs index d8668fba8e1e..7deb13c89be5 100644 --- a/datafusion/expr/src/utils.rs +++ b/datafusion/expr/src/utils.rs @@ -18,9 +18,13 @@ //! Expression utilities use crate::expr::{Alias, Sort, WindowFunction}; +use crate::expr_rewriter::strip_outer_reference; use crate::logical_plan::Aggregate; use crate::signature::{Signature, TypeSignature}; -use crate::{Cast, Expr, ExprSchemable, GroupingSet, LogicalPlan, TryCast}; +use crate::{ + and, BinaryExpr, Cast, Expr, ExprSchemable, Filter, GroupingSet, LogicalPlan, + Operator, TryCast, +}; use arrow::datatypes::{DataType, TimeUnit}; use datafusion_common::tree_node::{TreeNode, VisitRecursion}; use datafusion_common::{ @@ -30,6 +34,7 @@ use datafusion_common::{ use sqlparser::ast::{ExceptSelectItem, ExcludeSelectItem, WildcardAdditionalOptions}; use std::cmp::Ordering; use std::collections::HashSet; +use std::sync::Arc; /// The value to which `COUNT(*)` is expanded to in /// `COUNT()` expressions @@ -1004,12 +1009,245 @@ pub fn generate_signature_error_msg( ) } +/// Splits a conjunctive [`Expr`] such as `A AND B AND C` => `[A, B, C]` +/// +/// See [`split_conjunction_owned`] for more details and an example. +pub fn split_conjunction(expr: &Expr) -> Vec<&Expr> { + split_conjunction_impl(expr, vec![]) +} + +fn split_conjunction_impl<'a>(expr: &'a Expr, mut exprs: Vec<&'a Expr>) -> Vec<&'a Expr> { + match expr { + Expr::BinaryExpr(BinaryExpr { + right, + op: Operator::And, + left, + }) => { + let exprs = split_conjunction_impl(left, exprs); + split_conjunction_impl(right, exprs) + } + Expr::Alias(Alias { expr, .. }) => split_conjunction_impl(expr, exprs), + other => { + exprs.push(other); + exprs + } + } +} + +/// Splits an owned conjunctive [`Expr`] such as `A AND B AND C` => `[A, B, C]` +/// +/// This is often used to "split" filter expressions such as `col1 = 5 +/// AND col2 = 10` into [`col1 = 5`, `col2 = 10`]; +/// +/// # Example +/// ``` +/// # use datafusion_expr::{col, lit}; +/// # use datafusion_expr::utils::split_conjunction_owned; +/// // a=1 AND b=2 +/// let expr = col("a").eq(lit(1)).and(col("b").eq(lit(2))); +/// +/// // [a=1, b=2] +/// let split = vec![ +/// col("a").eq(lit(1)), +/// col("b").eq(lit(2)), +/// ]; +/// +/// // use split_conjunction_owned to split them +/// assert_eq!(split_conjunction_owned(expr), split); +/// ``` +pub fn split_conjunction_owned(expr: Expr) -> Vec { + split_binary_owned(expr, Operator::And) +} + +/// Splits an owned binary operator tree [`Expr`] such as `A B C` => `[A, B, C]` +/// +/// This is often used to "split" expressions such as `col1 = 5 +/// AND col2 = 10` into [`col1 = 5`, `col2 = 10`]; +/// +/// # Example +/// ``` +/// # use datafusion_expr::{col, lit, Operator}; +/// # use datafusion_expr::utils::split_binary_owned; +/// # use std::ops::Add; +/// // a=1 + b=2 +/// let expr = col("a").eq(lit(1)).add(col("b").eq(lit(2))); +/// +/// // [a=1, b=2] +/// let split = vec![ +/// col("a").eq(lit(1)), +/// col("b").eq(lit(2)), +/// ]; +/// +/// // use split_binary_owned to split them +/// assert_eq!(split_binary_owned(expr, Operator::Plus), split); +/// ``` +pub fn split_binary_owned(expr: Expr, op: Operator) -> Vec { + split_binary_owned_impl(expr, op, vec![]) +} + +fn split_binary_owned_impl( + expr: Expr, + operator: Operator, + mut exprs: Vec, +) -> Vec { + match expr { + Expr::BinaryExpr(BinaryExpr { right, op, left }) if op == operator => { + let exprs = split_binary_owned_impl(*left, operator, exprs); + split_binary_owned_impl(*right, operator, exprs) + } + Expr::Alias(Alias { expr, .. }) => { + split_binary_owned_impl(*expr, operator, exprs) + } + other => { + exprs.push(other); + exprs + } + } +} + +/// Splits an binary operator tree [`Expr`] such as `A B C` => `[A, B, C]` +/// +/// See [`split_binary_owned`] for more details and an example. +pub fn split_binary(expr: &Expr, op: Operator) -> Vec<&Expr> { + split_binary_impl(expr, op, vec![]) +} + +fn split_binary_impl<'a>( + expr: &'a Expr, + operator: Operator, + mut exprs: Vec<&'a Expr>, +) -> Vec<&'a Expr> { + match expr { + Expr::BinaryExpr(BinaryExpr { right, op, left }) if *op == operator => { + let exprs = split_binary_impl(left, operator, exprs); + split_binary_impl(right, operator, exprs) + } + Expr::Alias(Alias { expr, .. }) => split_binary_impl(expr, operator, exprs), + other => { + exprs.push(other); + exprs + } + } +} + +/// Combines an array of filter expressions into a single filter +/// expression consisting of the input filter expressions joined with +/// logical AND. +/// +/// Returns None if the filters array is empty. +/// +/// # Example +/// ``` +/// # use datafusion_expr::{col, lit}; +/// # use datafusion_expr::utils::conjunction; +/// // a=1 AND b=2 +/// let expr = col("a").eq(lit(1)).and(col("b").eq(lit(2))); +/// +/// // [a=1, b=2] +/// let split = vec![ +/// col("a").eq(lit(1)), +/// col("b").eq(lit(2)), +/// ]; +/// +/// // use conjunction to join them together with `AND` +/// assert_eq!(conjunction(split), Some(expr)); +/// ``` +pub fn conjunction(filters: impl IntoIterator) -> Option { + filters.into_iter().reduce(|accum, expr| accum.and(expr)) +} + +/// Combines an array of filter expressions into a single filter +/// expression consisting of the input filter expressions joined with +/// logical OR. +/// +/// Returns None if the filters array is empty. +pub fn disjunction(filters: impl IntoIterator) -> Option { + filters.into_iter().reduce(|accum, expr| accum.or(expr)) +} + +/// returns a new [LogicalPlan] that wraps `plan` in a [LogicalPlan::Filter] with +/// its predicate be all `predicates` ANDed. +pub fn add_filter(plan: LogicalPlan, predicates: &[&Expr]) -> Result { + // reduce filters to a single filter with an AND + let predicate = predicates + .iter() + .skip(1) + .fold(predicates[0].clone(), |acc, predicate| { + and(acc, (*predicate).to_owned()) + }); + + Ok(LogicalPlan::Filter(Filter::try_new( + predicate, + Arc::new(plan), + )?)) +} + +/// Looks for correlating expressions: for example, a binary expression with one field from the subquery, and +/// one not in the subquery (closed upon from outer scope) +/// +/// # Arguments +/// +/// * `exprs` - List of expressions that may or may not be joins +/// +/// # Return value +/// +/// Tuple of (expressions containing joins, remaining non-join expressions) +pub fn find_join_exprs(exprs: Vec<&Expr>) -> Result<(Vec, Vec)> { + let mut joins = vec![]; + let mut others = vec![]; + for filter in exprs.into_iter() { + // If the expression contains correlated predicates, add it to join filters + if filter.contains_outer() { + if !matches!(filter, Expr::BinaryExpr(BinaryExpr{ left, op: Operator::Eq, right }) if left.eq(right)) + { + joins.push(strip_outer_reference((*filter).clone())); + } + } else { + others.push((*filter).clone()); + } + } + + Ok((joins, others)) +} + +/// Returns the first (and only) element in a slice, or an error +/// +/// # Arguments +/// +/// * `slice` - The slice to extract from +/// +/// # Return value +/// +/// The first element, or an error +pub fn only_or_err(slice: &[T]) -> Result<&T> { + match slice { + [it] => Ok(it), + [] => plan_err!("No items found!"), + _ => plan_err!("More than one item found!"), + } +} + +/// merge inputs schema into a single schema. +pub fn merge_schema(inputs: Vec<&LogicalPlan>) -> DFSchema { + if inputs.len() == 1 { + inputs[0].schema().clone().as_ref().clone() + } else { + inputs.iter().map(|input| input.schema()).fold( + DFSchema::empty(), + |mut lhs, rhs| { + lhs.merge(rhs); + lhs + }, + ) + } +} + #[cfg(test)] mod tests { use super::*; use crate::expr_vec_fmt; use crate::{ - col, cube, expr, grouping_set, rollup, AggregateFunction, WindowFrame, + col, cube, expr, grouping_set, lit, rollup, AggregateFunction, WindowFrame, WindowFunction, }; @@ -1322,4 +1560,143 @@ mod tests { Ok(()) } + #[test] + fn test_split_conjunction() { + let expr = col("a"); + let result = split_conjunction(&expr); + assert_eq!(result, vec![&expr]); + } + + #[test] + fn test_split_conjunction_two() { + let expr = col("a").eq(lit(5)).and(col("b")); + let expr1 = col("a").eq(lit(5)); + let expr2 = col("b"); + + let result = split_conjunction(&expr); + assert_eq!(result, vec![&expr1, &expr2]); + } + + #[test] + fn test_split_conjunction_alias() { + let expr = col("a").eq(lit(5)).and(col("b").alias("the_alias")); + let expr1 = col("a").eq(lit(5)); + let expr2 = col("b"); // has no alias + + let result = split_conjunction(&expr); + assert_eq!(result, vec![&expr1, &expr2]); + } + + #[test] + fn test_split_conjunction_or() { + let expr = col("a").eq(lit(5)).or(col("b")); + let result = split_conjunction(&expr); + assert_eq!(result, vec![&expr]); + } + + #[test] + fn test_split_binary_owned() { + let expr = col("a"); + assert_eq!(split_binary_owned(expr.clone(), Operator::And), vec![expr]); + } + + #[test] + fn test_split_binary_owned_two() { + assert_eq!( + split_binary_owned(col("a").eq(lit(5)).and(col("b")), Operator::And), + vec![col("a").eq(lit(5)), col("b")] + ); + } + + #[test] + fn test_split_binary_owned_different_op() { + let expr = col("a").eq(lit(5)).or(col("b")); + assert_eq!( + // expr is connected by OR, but pass in AND + split_binary_owned(expr.clone(), Operator::And), + vec![expr] + ); + } + + #[test] + fn test_split_conjunction_owned() { + let expr = col("a"); + assert_eq!(split_conjunction_owned(expr.clone()), vec![expr]); + } + + #[test] + fn test_split_conjunction_owned_two() { + assert_eq!( + split_conjunction_owned(col("a").eq(lit(5)).and(col("b"))), + vec![col("a").eq(lit(5)), col("b")] + ); + } + + #[test] + fn test_split_conjunction_owned_alias() { + assert_eq!( + split_conjunction_owned(col("a").eq(lit(5)).and(col("b").alias("the_alias"))), + vec![ + col("a").eq(lit(5)), + // no alias on b + col("b"), + ] + ); + } + + #[test] + fn test_conjunction_empty() { + assert_eq!(conjunction(vec![]), None); + } + + #[test] + fn test_conjunction() { + // `[A, B, C]` + let expr = conjunction(vec![col("a"), col("b"), col("c")]); + + // --> `(A AND B) AND C` + assert_eq!(expr, Some(col("a").and(col("b")).and(col("c")))); + + // which is different than `A AND (B AND C)` + assert_ne!(expr, Some(col("a").and(col("b").and(col("c"))))); + } + + #[test] + fn test_disjunction_empty() { + assert_eq!(disjunction(vec![]), None); + } + + #[test] + fn test_disjunction() { + // `[A, B, C]` + let expr = disjunction(vec![col("a"), col("b"), col("c")]); + + // --> `(A OR B) OR C` + assert_eq!(expr, Some(col("a").or(col("b")).or(col("c")))); + + // which is different than `A OR (B OR C)` + assert_ne!(expr, Some(col("a").or(col("b").or(col("c"))))); + } + + #[test] + fn test_split_conjunction_owned_or() { + let expr = col("a").eq(lit(5)).or(col("b")); + assert_eq!(split_conjunction_owned(expr.clone()), vec![expr]); + } + + #[test] + fn test_collect_expr() -> Result<()> { + let mut accum: HashSet = HashSet::new(); + expr_to_columns( + &Expr::Cast(Cast::new(Box::new(col("a")), DataType::Float64)), + &mut accum, + )?; + expr_to_columns( + &Expr::Cast(Cast::new(Box::new(col("a")), DataType::Float64)), + &mut accum, + )?; + assert_eq!(1, accum.len()); + assert!(accum.contains(&Column::from_name("a"))); + Ok(()) + } } diff --git a/datafusion/optimizer/src/analyzer/subquery.rs b/datafusion/optimizer/src/analyzer/subquery.rs index 6b8b1020cd6d..7c5b70b19af0 100644 --- a/datafusion/optimizer/src/analyzer/subquery.rs +++ b/datafusion/optimizer/src/analyzer/subquery.rs @@ -16,10 +16,11 @@ // under the License. use crate::analyzer::check_plan; -use crate::utils::{collect_subquery_cols, split_conjunction}; +use crate::utils::collect_subquery_cols; use datafusion_common::tree_node::{TreeNode, VisitRecursion}; use datafusion_common::{plan_err, DataFusionError, Result}; use datafusion_expr::expr_rewriter::strip_outer_reference; +use datafusion_expr::utils::split_conjunction; use datafusion_expr::{ Aggregate, BinaryExpr, Cast, Expr, Filter, Join, JoinType, LogicalPlan, Operator, Window, diff --git a/datafusion/optimizer/src/analyzer/type_coercion.rs b/datafusion/optimizer/src/analyzer/type_coercion.rs index 6628e8961e26..eb5d8c53a5e0 100644 --- a/datafusion/optimizer/src/analyzer/type_coercion.rs +++ b/datafusion/optimizer/src/analyzer/type_coercion.rs @@ -42,16 +42,15 @@ use datafusion_expr::type_coercion::other::{ get_coerce_type_for_case_expression, get_coerce_type_for_list, }; use datafusion_expr::type_coercion::{is_datetime, is_utf8_or_large_utf8}; +use datafusion_expr::utils::merge_schema; use datafusion_expr::{ is_false, is_not_false, is_not_true, is_not_unknown, is_true, is_unknown, type_coercion, window_function, AggregateFunction, BuiltinScalarFunction, Expr, - LogicalPlan, Operator, Projection, ScalarFunctionDefinition, WindowFrame, - WindowFrameBound, WindowFrameUnits, + ExprSchemable, LogicalPlan, Operator, Projection, ScalarFunctionDefinition, + Signature, WindowFrame, WindowFrameBound, WindowFrameUnits, }; -use datafusion_expr::{ExprSchemable, Signature}; use crate::analyzer::AnalyzerRule; -use crate::utils::merge_schema; #[derive(Default)] pub struct TypeCoercion {} diff --git a/datafusion/optimizer/src/decorrelate.rs b/datafusion/optimizer/src/decorrelate.rs index c8162683f39e..ed6f472186d4 100644 --- a/datafusion/optimizer/src/decorrelate.rs +++ b/datafusion/optimizer/src/decorrelate.rs @@ -16,15 +16,14 @@ // under the License. use crate::simplify_expressions::{ExprSimplifier, SimplifyContext}; -use crate::utils::{ - collect_subquery_cols, conjunction, find_join_exprs, split_conjunction, -}; +use crate::utils::collect_subquery_cols; use datafusion_common::tree_node::{ RewriteRecursion, Transformed, TreeNode, TreeNodeRewriter, }; use datafusion_common::{plan_err, Result}; use datafusion_common::{Column, DFSchemaRef, DataFusionError, ScalarValue}; use datafusion_expr::expr::Alias; +use datafusion_expr::utils::{conjunction, find_join_exprs, split_conjunction}; use datafusion_expr::{expr, EmptyRelation, Expr, LogicalPlan, LogicalPlanBuilder}; use datafusion_physical_expr::execution_props::ExecutionProps; use std::collections::{BTreeSet, HashMap}; diff --git a/datafusion/optimizer/src/decorrelate_predicate_subquery.rs b/datafusion/optimizer/src/decorrelate_predicate_subquery.rs index 96b46663d8e4..450336376a23 100644 --- a/datafusion/optimizer/src/decorrelate_predicate_subquery.rs +++ b/datafusion/optimizer/src/decorrelate_predicate_subquery.rs @@ -17,7 +17,7 @@ use crate::decorrelate::PullUpCorrelatedExpr; use crate::optimizer::ApplyOrder; -use crate::utils::{conjunction, replace_qualified_name, split_conjunction}; +use crate::utils::replace_qualified_name; use crate::{OptimizerConfig, OptimizerRule}; use datafusion_common::alias::AliasGenerator; use datafusion_common::tree_node::TreeNode; @@ -25,6 +25,7 @@ use datafusion_common::{plan_err, DataFusionError, Result}; use datafusion_expr::expr::{Exists, InSubquery}; use datafusion_expr::expr_rewriter::create_col_from_scalar_expr; use datafusion_expr::logical_plan::{JoinType, Subquery}; +use datafusion_expr::utils::{conjunction, split_conjunction}; use datafusion_expr::{ exists, in_subquery, not_exists, not_in_subquery, BinaryExpr, Expr, Filter, LogicalPlan, LogicalPlanBuilder, Operator, diff --git a/datafusion/optimizer/src/extract_equijoin_predicate.rs b/datafusion/optimizer/src/extract_equijoin_predicate.rs index 575969fbf73c..24664d57c38d 100644 --- a/datafusion/optimizer/src/extract_equijoin_predicate.rs +++ b/datafusion/optimizer/src/extract_equijoin_predicate.rs @@ -17,11 +17,10 @@ //! [`ExtractEquijoinPredicate`] rule that extracts equijoin predicates use crate::optimizer::ApplyOrder; -use crate::utils::split_conjunction; use crate::{OptimizerConfig, OptimizerRule}; use datafusion_common::DFSchema; use datafusion_common::Result; -use datafusion_expr::utils::{can_hash, find_valid_equijoin_key_pair}; +use datafusion_expr::utils::{can_hash, find_valid_equijoin_key_pair, split_conjunction}; use datafusion_expr::{BinaryExpr, Expr, ExprSchemable, Join, LogicalPlan, Operator}; use std::sync::Arc; diff --git a/datafusion/optimizer/src/optimizer.rs b/datafusion/optimizer/src/optimizer.rs index c7ad31f39b00..7af46ed70adf 100644 --- a/datafusion/optimizer/src/optimizer.rs +++ b/datafusion/optimizer/src/optimizer.rs @@ -45,7 +45,7 @@ use chrono::{DateTime, Utc}; use datafusion_common::alias::AliasGenerator; use datafusion_common::config::ConfigOptions; use datafusion_common::{DataFusionError, Result}; -use datafusion_expr::logical_plan::LogicalPlan; +use datafusion_expr::LogicalPlan; use log::{debug, warn}; use std::collections::HashSet; use std::sync::Arc; diff --git a/datafusion/optimizer/src/push_down_filter.rs b/datafusion/optimizer/src/push_down_filter.rs index 7a2c6a8d8ccd..95eeee931b4f 100644 --- a/datafusion/optimizer/src/push_down_filter.rs +++ b/datafusion/optimizer/src/push_down_filter.rs @@ -16,13 +16,13 @@ //! the plan. use crate::optimizer::ApplyOrder; -use crate::utils::{conjunction, split_conjunction, split_conjunction_owned}; -use crate::{utils, OptimizerConfig, OptimizerRule}; +use crate::{OptimizerConfig, OptimizerRule}; use datafusion_common::tree_node::{Transformed, TreeNode, VisitRecursion}; use datafusion_common::{ internal_err, plan_datafusion_err, Column, DFSchema, DataFusionError, Result, }; use datafusion_expr::expr::Alias; +use datafusion_expr::utils::{conjunction, split_conjunction, split_conjunction_owned}; use datafusion_expr::Volatility; use datafusion_expr::{ and, @@ -546,9 +546,7 @@ fn push_down_join( parent_predicate: Option<&Expr>, ) -> Result> { let predicates = match parent_predicate { - Some(parent_predicate) => { - utils::split_conjunction_owned(parent_predicate.clone()) - } + Some(parent_predicate) => split_conjunction_owned(parent_predicate.clone()), None => vec![], }; @@ -556,7 +554,7 @@ fn push_down_join( let on_filters = join .filter .as_ref() - .map(|e| utils::split_conjunction_owned(e.clone())) + .map(|e| split_conjunction_owned(e.clone())) .unwrap_or_default(); let mut is_inner_join = false; @@ -805,7 +803,7 @@ impl OptimizerRule for PushDownFilter { .map(|e| Ok(Column::from_qualified_name(e.display_name()?))) .collect::>>()?; - let predicates = utils::split_conjunction_owned(filter.predicate.clone()); + let predicates = split_conjunction_owned(filter.predicate.clone()); let mut keep_predicates = vec![]; let mut push_predicates = vec![]; @@ -853,7 +851,7 @@ impl OptimizerRule for PushDownFilter { } } LogicalPlan::CrossJoin(CrossJoin { left, right, .. }) => { - let predicates = utils::split_conjunction_owned(filter.predicate.clone()); + let predicates = split_conjunction_owned(filter.predicate.clone()); push_down_all_join( predicates, vec![], @@ -908,7 +906,7 @@ impl OptimizerRule for PushDownFilter { let prevent_cols = extension_plan.node.prevent_predicate_push_down_columns(); - let predicates = utils::split_conjunction_owned(filter.predicate.clone()); + let predicates = split_conjunction_owned(filter.predicate.clone()); let mut keep_predicates = vec![]; let mut push_predicates = vec![]; diff --git a/datafusion/optimizer/src/scalar_subquery_to_join.rs b/datafusion/optimizer/src/scalar_subquery_to_join.rs index 7ac0c25119c3..34ed4a9475cb 100644 --- a/datafusion/optimizer/src/scalar_subquery_to_join.rs +++ b/datafusion/optimizer/src/scalar_subquery_to_join.rs @@ -17,7 +17,7 @@ use crate::decorrelate::{PullUpCorrelatedExpr, UN_MATCHED_ROW_INDICATOR}; use crate::optimizer::ApplyOrder; -use crate::utils::{conjunction, replace_qualified_name}; +use crate::utils::replace_qualified_name; use crate::{OptimizerConfig, OptimizerRule}; use datafusion_common::alias::AliasGenerator; use datafusion_common::tree_node::{ @@ -26,6 +26,7 @@ use datafusion_common::tree_node::{ use datafusion_common::{plan_err, Column, DataFusionError, Result, ScalarValue}; use datafusion_expr::expr_rewriter::create_col_from_scalar_expr; use datafusion_expr::logical_plan::{JoinType, Subquery}; +use datafusion_expr::utils::conjunction; use datafusion_expr::{expr, EmptyRelation, Expr, LogicalPlan, LogicalPlanBuilder}; use std::collections::{BTreeSet, HashMap}; use std::sync::Arc; diff --git a/datafusion/optimizer/src/simplify_expressions/simplify_exprs.rs b/datafusion/optimizer/src/simplify_expressions/simplify_exprs.rs index 9dc83e0fadf5..43a41b1185a3 100644 --- a/datafusion/optimizer/src/simplify_expressions/simplify_exprs.rs +++ b/datafusion/optimizer/src/simplify_expressions/simplify_exprs.rs @@ -20,10 +20,10 @@ use std::sync::Arc; use super::{ExprSimplifier, SimplifyContext}; -use crate::utils::merge_schema; use crate::{OptimizerConfig, OptimizerRule}; use datafusion_common::{DFSchema, DFSchemaRef, Result}; use datafusion_expr::logical_plan::LogicalPlan; +use datafusion_expr::utils::merge_schema; use datafusion_physical_expr::execution_props::ExecutionProps; /// Optimizer Pass that simplifies [`LogicalPlan`]s by rewriting diff --git a/datafusion/optimizer/src/unwrap_cast_in_comparison.rs b/datafusion/optimizer/src/unwrap_cast_in_comparison.rs index 907c12b7afb1..91603e82a54f 100644 --- a/datafusion/optimizer/src/unwrap_cast_in_comparison.rs +++ b/datafusion/optimizer/src/unwrap_cast_in_comparison.rs @@ -19,7 +19,6 @@ //! of expr can be added if needed. //! This rule can reduce adding the `Expr::Cast` the expr instead of adding the `Expr::Cast` to literal expr. use crate::optimizer::ApplyOrder; -use crate::utils::merge_schema; use crate::{OptimizerConfig, OptimizerRule}; use arrow::datatypes::{ DataType, TimeUnit, MAX_DECIMAL_FOR_EACH_PRECISION, MIN_DECIMAL_FOR_EACH_PRECISION, @@ -31,6 +30,7 @@ use datafusion_common::{ }; use datafusion_expr::expr::{BinaryExpr, Cast, InList, TryCast}; use datafusion_expr::expr_rewriter::rewrite_preserving_name; +use datafusion_expr::utils::merge_schema; use datafusion_expr::{ binary_expr, in_list, lit, Expr, ExprSchemable, LogicalPlan, Operator, }; diff --git a/datafusion/optimizer/src/utils.rs b/datafusion/optimizer/src/utils.rs index a3e7e42875d7..48f72ee7a0f8 100644 --- a/datafusion/optimizer/src/utils.rs +++ b/datafusion/optimizer/src/utils.rs @@ -18,19 +18,13 @@ //! Collection of utility functions that are leveraged by the query optimizer rules use crate::{OptimizerConfig, OptimizerRule}; -use datafusion_common::DataFusionError; -use datafusion_common::{plan_err, Column, DFSchemaRef}; +use datafusion_common::{Column, DFSchemaRef}; use datafusion_common::{DFSchema, Result}; -use datafusion_expr::expr::{Alias, BinaryExpr}; -use datafusion_expr::expr_rewriter::{replace_col, strip_outer_reference}; -use datafusion_expr::{ - and, - logical_plan::{Filter, LogicalPlan}, - Expr, Operator, -}; +use datafusion_expr::expr_rewriter::replace_col; +use datafusion_expr::utils as expr_utils; +use datafusion_expr::{logical_plan::LogicalPlan, Expr, Operator}; use log::{debug, trace}; use std::collections::{BTreeSet, HashMap}; -use std::sync::Arc; /// Convenience rule for writing optimizers: recursively invoke /// optimize on plan's children and then return a node of the same @@ -58,29 +52,55 @@ pub fn optimize_children( } } +pub(crate) fn collect_subquery_cols( + exprs: &[Expr], + subquery_schema: DFSchemaRef, +) -> Result> { + exprs.iter().try_fold(BTreeSet::new(), |mut cols, expr| { + let mut using_cols: Vec = vec![]; + for col in expr.to_columns()?.into_iter() { + if subquery_schema.has_column(&col) { + using_cols.push(col); + } + } + + cols.extend(using_cols); + Result::<_>::Ok(cols) + }) +} + +pub(crate) fn replace_qualified_name( + expr: Expr, + cols: &BTreeSet, + subquery_alias: &str, +) -> Result { + let alias_cols: Vec = cols + .iter() + .map(|col| { + Column::from_qualified_name(format!("{}.{}", subquery_alias, col.name)) + }) + .collect(); + let replace_map: HashMap<&Column, &Column> = + cols.iter().zip(alias_cols.iter()).collect(); + + replace_col(expr, &replace_map) +} + +/// Log the plan in debug/tracing mode after some part of the optimizer runs +pub fn log_plan(description: &str, plan: &LogicalPlan) { + debug!("{description}:\n{}\n", plan.display_indent()); + trace!("{description}::\n{}\n", plan.display_indent_schema()); +} + /// Splits a conjunctive [`Expr`] such as `A AND B AND C` => `[A, B, C]` /// /// See [`split_conjunction_owned`] for more details and an example. +#[deprecated( + since = "34.0.0", + note = "use `datafusion_expr::utils::split_conjunction` instead" +)] pub fn split_conjunction(expr: &Expr) -> Vec<&Expr> { - split_conjunction_impl(expr, vec![]) -} - -fn split_conjunction_impl<'a>(expr: &'a Expr, mut exprs: Vec<&'a Expr>) -> Vec<&'a Expr> { - match expr { - Expr::BinaryExpr(BinaryExpr { - right, - op: Operator::And, - left, - }) => { - let exprs = split_conjunction_impl(left, exprs); - split_conjunction_impl(right, exprs) - } - Expr::Alias(Alias { expr, .. }) => split_conjunction_impl(expr, exprs), - other => { - exprs.push(other); - exprs - } - } + expr_utils::split_conjunction(expr) } /// Splits an owned conjunctive [`Expr`] such as `A AND B AND C` => `[A, B, C]` @@ -104,8 +124,12 @@ fn split_conjunction_impl<'a>(expr: &'a Expr, mut exprs: Vec<&'a Expr>) -> Vec<& /// // use split_conjunction_owned to split them /// assert_eq!(split_conjunction_owned(expr), split); /// ``` +#[deprecated( + since = "34.0.0", + note = "use `datafusion_expr::utils::split_conjunction_owned` instead" +)] pub fn split_conjunction_owned(expr: Expr) -> Vec { - split_binary_owned(expr, Operator::And) + expr_utils::split_conjunction_owned(expr) } /// Splits an owned binary operator tree [`Expr`] such as `A B C` => `[A, B, C]` @@ -130,53 +154,23 @@ pub fn split_conjunction_owned(expr: Expr) -> Vec { /// // use split_binary_owned to split them /// assert_eq!(split_binary_owned(expr, Operator::Plus), split); /// ``` +#[deprecated( + since = "34.0.0", + note = "use `datafusion_expr::utils::split_binary_owned` instead" +)] pub fn split_binary_owned(expr: Expr, op: Operator) -> Vec { - split_binary_owned_impl(expr, op, vec![]) -} - -fn split_binary_owned_impl( - expr: Expr, - operator: Operator, - mut exprs: Vec, -) -> Vec { - match expr { - Expr::BinaryExpr(BinaryExpr { right, op, left }) if op == operator => { - let exprs = split_binary_owned_impl(*left, operator, exprs); - split_binary_owned_impl(*right, operator, exprs) - } - Expr::Alias(Alias { expr, .. }) => { - split_binary_owned_impl(*expr, operator, exprs) - } - other => { - exprs.push(other); - exprs - } - } + expr_utils::split_binary_owned(expr, op) } /// Splits an binary operator tree [`Expr`] such as `A B C` => `[A, B, C]` /// /// See [`split_binary_owned`] for more details and an example. +#[deprecated( + since = "34.0.0", + note = "use `datafusion_expr::utils::split_binary` instead" +)] pub fn split_binary(expr: &Expr, op: Operator) -> Vec<&Expr> { - split_binary_impl(expr, op, vec![]) -} - -fn split_binary_impl<'a>( - expr: &'a Expr, - operator: Operator, - mut exprs: Vec<&'a Expr>, -) -> Vec<&'a Expr> { - match expr { - Expr::BinaryExpr(BinaryExpr { right, op, left }) if *op == operator => { - let exprs = split_binary_impl(left, operator, exprs); - split_binary_impl(right, operator, exprs) - } - Expr::Alias(Alias { expr, .. }) => split_binary_impl(expr, operator, exprs), - other => { - exprs.push(other); - exprs - } - } + expr_utils::split_binary(expr, op) } /// Combines an array of filter expressions into a single filter @@ -201,8 +195,12 @@ fn split_binary_impl<'a>( /// // use conjunction to join them together with `AND` /// assert_eq!(conjunction(split), Some(expr)); /// ``` +#[deprecated( + since = "34.0.0", + note = "use `datafusion_expr::utils::conjunction` instead" +)] pub fn conjunction(filters: impl IntoIterator) -> Option { - filters.into_iter().reduce(|accum, expr| accum.and(expr)) + expr_utils::conjunction(filters) } /// Combines an array of filter expressions into a single filter @@ -210,25 +208,22 @@ pub fn conjunction(filters: impl IntoIterator) -> Option { /// logical OR. /// /// Returns None if the filters array is empty. +#[deprecated( + since = "34.0.0", + note = "use `datafusion_expr::utils::disjunction` instead" +)] pub fn disjunction(filters: impl IntoIterator) -> Option { - filters.into_iter().reduce(|accum, expr| accum.or(expr)) + expr_utils::disjunction(filters) } /// returns a new [LogicalPlan] that wraps `plan` in a [LogicalPlan::Filter] with /// its predicate be all `predicates` ANDed. +#[deprecated( + since = "34.0.0", + note = "use `datafusion_expr::utils::add_filter` instead" +)] pub fn add_filter(plan: LogicalPlan, predicates: &[&Expr]) -> Result { - // reduce filters to a single filter with an AND - let predicate = predicates - .iter() - .skip(1) - .fold(predicates[0].clone(), |acc, predicate| { - and(acc, (*predicate).to_owned()) - }); - - Ok(LogicalPlan::Filter(Filter::try_new( - predicate, - Arc::new(plan), - )?)) + expr_utils::add_filter(plan, predicates) } /// Looks for correlating expressions: for example, a binary expression with one field from the subquery, and @@ -241,22 +236,12 @@ pub fn add_filter(plan: LogicalPlan, predicates: &[&Expr]) -> Result) -> Result<(Vec, Vec)> { - let mut joins = vec![]; - let mut others = vec![]; - for filter in exprs.into_iter() { - // If the expression contains correlated predicates, add it to join filters - if filter.contains_outer() { - if !matches!(filter, Expr::BinaryExpr(BinaryExpr{ left, op: Operator::Eq, right }) if left.eq(right)) - { - joins.push(strip_outer_reference((*filter).clone())); - } - } else { - others.push((*filter).clone()); - } - } - - Ok((joins, others)) + expr_utils::find_join_exprs(exprs) } /// Returns the first (and only) element in a slice, or an error @@ -268,215 +253,19 @@ pub fn find_join_exprs(exprs: Vec<&Expr>) -> Result<(Vec, Vec)> { /// # Return value /// /// The first element, or an error +#[deprecated( + since = "34.0.0", + note = "use `datafusion_expr::utils::only_or_err` instead" +)] pub fn only_or_err(slice: &[T]) -> Result<&T> { - match slice { - [it] => Ok(it), - [] => plan_err!("No items found!"), - _ => plan_err!("More than one item found!"), - } + expr_utils::only_or_err(slice) } /// merge inputs schema into a single schema. +#[deprecated( + since = "34.0.0", + note = "use `datafusion_expr::utils::merge_schema` instead" +)] pub fn merge_schema(inputs: Vec<&LogicalPlan>) -> DFSchema { - if inputs.len() == 1 { - inputs[0].schema().clone().as_ref().clone() - } else { - inputs.iter().map(|input| input.schema()).fold( - DFSchema::empty(), - |mut lhs, rhs| { - lhs.merge(rhs); - lhs - }, - ) - } -} - -pub(crate) fn collect_subquery_cols( - exprs: &[Expr], - subquery_schema: DFSchemaRef, -) -> Result> { - exprs.iter().try_fold(BTreeSet::new(), |mut cols, expr| { - let mut using_cols: Vec = vec![]; - for col in expr.to_columns()?.into_iter() { - if subquery_schema.has_column(&col) { - using_cols.push(col); - } - } - - cols.extend(using_cols); - Result::<_>::Ok(cols) - }) -} - -pub(crate) fn replace_qualified_name( - expr: Expr, - cols: &BTreeSet, - subquery_alias: &str, -) -> Result { - let alias_cols: Vec = cols - .iter() - .map(|col| { - Column::from_qualified_name(format!("{}.{}", subquery_alias, col.name)) - }) - .collect(); - let replace_map: HashMap<&Column, &Column> = - cols.iter().zip(alias_cols.iter()).collect(); - - replace_col(expr, &replace_map) -} - -/// Log the plan in debug/tracing mode after some part of the optimizer runs -pub fn log_plan(description: &str, plan: &LogicalPlan) { - debug!("{description}:\n{}\n", plan.display_indent()); - trace!("{description}::\n{}\n", plan.display_indent_schema()); -} - -#[cfg(test)] -mod tests { - use super::*; - use arrow::datatypes::DataType; - use datafusion_common::Column; - use datafusion_expr::expr::Cast; - use datafusion_expr::{col, lit, utils::expr_to_columns}; - use std::collections::HashSet; - - #[test] - fn test_split_conjunction() { - let expr = col("a"); - let result = split_conjunction(&expr); - assert_eq!(result, vec![&expr]); - } - - #[test] - fn test_split_conjunction_two() { - let expr = col("a").eq(lit(5)).and(col("b")); - let expr1 = col("a").eq(lit(5)); - let expr2 = col("b"); - - let result = split_conjunction(&expr); - assert_eq!(result, vec![&expr1, &expr2]); - } - - #[test] - fn test_split_conjunction_alias() { - let expr = col("a").eq(lit(5)).and(col("b").alias("the_alias")); - let expr1 = col("a").eq(lit(5)); - let expr2 = col("b"); // has no alias - - let result = split_conjunction(&expr); - assert_eq!(result, vec![&expr1, &expr2]); - } - - #[test] - fn test_split_conjunction_or() { - let expr = col("a").eq(lit(5)).or(col("b")); - let result = split_conjunction(&expr); - assert_eq!(result, vec![&expr]); - } - - #[test] - fn test_split_binary_owned() { - let expr = col("a"); - assert_eq!(split_binary_owned(expr.clone(), Operator::And), vec![expr]); - } - - #[test] - fn test_split_binary_owned_two() { - assert_eq!( - split_binary_owned(col("a").eq(lit(5)).and(col("b")), Operator::And), - vec![col("a").eq(lit(5)), col("b")] - ); - } - - #[test] - fn test_split_binary_owned_different_op() { - let expr = col("a").eq(lit(5)).or(col("b")); - assert_eq!( - // expr is connected by OR, but pass in AND - split_binary_owned(expr.clone(), Operator::And), - vec![expr] - ); - } - - #[test] - fn test_split_conjunction_owned() { - let expr = col("a"); - assert_eq!(split_conjunction_owned(expr.clone()), vec![expr]); - } - - #[test] - fn test_split_conjunction_owned_two() { - assert_eq!( - split_conjunction_owned(col("a").eq(lit(5)).and(col("b"))), - vec![col("a").eq(lit(5)), col("b")] - ); - } - - #[test] - fn test_split_conjunction_owned_alias() { - assert_eq!( - split_conjunction_owned(col("a").eq(lit(5)).and(col("b").alias("the_alias"))), - vec![ - col("a").eq(lit(5)), - // no alias on b - col("b"), - ] - ); - } - - #[test] - fn test_conjunction_empty() { - assert_eq!(conjunction(vec![]), None); - } - - #[test] - fn test_conjunction() { - // `[A, B, C]` - let expr = conjunction(vec![col("a"), col("b"), col("c")]); - - // --> `(A AND B) AND C` - assert_eq!(expr, Some(col("a").and(col("b")).and(col("c")))); - - // which is different than `A AND (B AND C)` - assert_ne!(expr, Some(col("a").and(col("b").and(col("c"))))); - } - - #[test] - fn test_disjunction_empty() { - assert_eq!(disjunction(vec![]), None); - } - - #[test] - fn test_disjunction() { - // `[A, B, C]` - let expr = disjunction(vec![col("a"), col("b"), col("c")]); - - // --> `(A OR B) OR C` - assert_eq!(expr, Some(col("a").or(col("b")).or(col("c")))); - - // which is different than `A OR (B OR C)` - assert_ne!(expr, Some(col("a").or(col("b").or(col("c"))))); - } - - #[test] - fn test_split_conjunction_owned_or() { - let expr = col("a").eq(lit(5)).or(col("b")); - assert_eq!(split_conjunction_owned(expr.clone()), vec![expr]); - } - - #[test] - fn test_collect_expr() -> Result<()> { - let mut accum: HashSet = HashSet::new(); - expr_to_columns( - &Expr::Cast(Cast::new(Box::new(col("a")), DataType::Float64)), - &mut accum, - )?; - expr_to_columns( - &Expr::Cast(Cast::new(Box::new(col("a")), DataType::Float64)), - &mut accum, - )?; - assert_eq!(1, accum.len()); - assert!(accum.contains(&Column::from_name("a"))); - Ok(()) - } + expr_utils::merge_schema(inputs) } diff --git a/datafusion/substrait/src/logical_plan/consumer.rs b/datafusion/substrait/src/logical_plan/consumer.rs index 5cb72adaca4d..b7a51032dcd9 100644 --- a/datafusion/substrait/src/logical_plan/consumer.rs +++ b/datafusion/substrait/src/logical_plan/consumer.rs @@ -32,7 +32,7 @@ use datafusion::prelude::JoinType; use datafusion::sql::TableReference; use datafusion::{ error::{DataFusionError, Result}, - optimizer::utils::split_conjunction, + logical_expr::utils::split_conjunction, prelude::{Column, SessionContext}, scalar::ScalarValue, }; From 167b5b75a06d4b5f4bb38a87de1092c6251089a2 Mon Sep 17 00:00:00 2001 From: Andrew Lamb Date: Wed, 29 Nov 2023 14:07:52 -0500 Subject: [PATCH 332/572] Minor: Make BuiltInScalarFunction::alias a method (#8349) --- datafusion/expr/src/built_in_function.rs | 362 ++++++++++++----------- 1 file changed, 187 insertions(+), 175 deletions(-) diff --git a/datafusion/expr/src/built_in_function.rs b/datafusion/expr/src/built_in_function.rs index c511c752b4d7..53f9e850d303 100644 --- a/datafusion/expr/src/built_in_function.rs +++ b/datafusion/expr/src/built_in_function.rs @@ -313,8 +313,7 @@ fn name_to_function() -> &'static HashMap<&'static str, BuiltinScalarFunction> { NAME_TO_FUNCTION_LOCK.get_or_init(|| { let mut map = HashMap::new(); BuiltinScalarFunction::iter().for_each(|func| { - let a = aliases(&func); - a.iter().for_each(|&a| { + func.aliases().iter().for_each(|&a| { map.insert(a, func); }); }); @@ -330,7 +329,7 @@ fn function_to_name() -> &'static HashMap { FUNCTION_TO_NAME_LOCK.get_or_init(|| { let mut map = HashMap::new(); BuiltinScalarFunction::iter().for_each(|func| { - map.insert(func, *aliases(&func).first().unwrap_or(&"NO_ALIAS")); + map.insert(func, *func.aliases().first().unwrap_or(&"NO_ALIAS")); }); map }) @@ -1417,188 +1416,201 @@ impl BuiltinScalarFunction { None } } -} -fn aliases(func: &BuiltinScalarFunction) -> &'static [&'static str] { - match func { - BuiltinScalarFunction::Abs => &["abs"], - BuiltinScalarFunction::Acos => &["acos"], - BuiltinScalarFunction::Acosh => &["acosh"], - BuiltinScalarFunction::Asin => &["asin"], - BuiltinScalarFunction::Asinh => &["asinh"], - BuiltinScalarFunction::Atan => &["atan"], - BuiltinScalarFunction::Atanh => &["atanh"], - BuiltinScalarFunction::Atan2 => &["atan2"], - BuiltinScalarFunction::Cbrt => &["cbrt"], - BuiltinScalarFunction::Ceil => &["ceil"], - BuiltinScalarFunction::Cos => &["cos"], - BuiltinScalarFunction::Cot => &["cot"], - BuiltinScalarFunction::Cosh => &["cosh"], - BuiltinScalarFunction::Degrees => &["degrees"], - BuiltinScalarFunction::Exp => &["exp"], - BuiltinScalarFunction::Factorial => &["factorial"], - BuiltinScalarFunction::Floor => &["floor"], - BuiltinScalarFunction::Gcd => &["gcd"], - BuiltinScalarFunction::Isnan => &["isnan"], - BuiltinScalarFunction::Iszero => &["iszero"], - BuiltinScalarFunction::Lcm => &["lcm"], - BuiltinScalarFunction::Ln => &["ln"], - BuiltinScalarFunction::Log => &["log"], - BuiltinScalarFunction::Log10 => &["log10"], - BuiltinScalarFunction::Log2 => &["log2"], - BuiltinScalarFunction::Nanvl => &["nanvl"], - BuiltinScalarFunction::Pi => &["pi"], - BuiltinScalarFunction::Power => &["power", "pow"], - BuiltinScalarFunction::Radians => &["radians"], - BuiltinScalarFunction::Random => &["random"], - BuiltinScalarFunction::Round => &["round"], - BuiltinScalarFunction::Signum => &["signum"], - BuiltinScalarFunction::Sin => &["sin"], - BuiltinScalarFunction::Sinh => &["sinh"], - BuiltinScalarFunction::Sqrt => &["sqrt"], - BuiltinScalarFunction::Tan => &["tan"], - BuiltinScalarFunction::Tanh => &["tanh"], - BuiltinScalarFunction::Trunc => &["trunc"], + /// Returns all names that can be used to call this function + pub fn aliases(&self) -> &'static [&'static str] { + match self { + BuiltinScalarFunction::Abs => &["abs"], + BuiltinScalarFunction::Acos => &["acos"], + BuiltinScalarFunction::Acosh => &["acosh"], + BuiltinScalarFunction::Asin => &["asin"], + BuiltinScalarFunction::Asinh => &["asinh"], + BuiltinScalarFunction::Atan => &["atan"], + BuiltinScalarFunction::Atanh => &["atanh"], + BuiltinScalarFunction::Atan2 => &["atan2"], + BuiltinScalarFunction::Cbrt => &["cbrt"], + BuiltinScalarFunction::Ceil => &["ceil"], + BuiltinScalarFunction::Cos => &["cos"], + BuiltinScalarFunction::Cot => &["cot"], + BuiltinScalarFunction::Cosh => &["cosh"], + BuiltinScalarFunction::Degrees => &["degrees"], + BuiltinScalarFunction::Exp => &["exp"], + BuiltinScalarFunction::Factorial => &["factorial"], + BuiltinScalarFunction::Floor => &["floor"], + BuiltinScalarFunction::Gcd => &["gcd"], + BuiltinScalarFunction::Isnan => &["isnan"], + BuiltinScalarFunction::Iszero => &["iszero"], + BuiltinScalarFunction::Lcm => &["lcm"], + BuiltinScalarFunction::Ln => &["ln"], + BuiltinScalarFunction::Log => &["log"], + BuiltinScalarFunction::Log10 => &["log10"], + BuiltinScalarFunction::Log2 => &["log2"], + BuiltinScalarFunction::Nanvl => &["nanvl"], + BuiltinScalarFunction::Pi => &["pi"], + BuiltinScalarFunction::Power => &["power", "pow"], + BuiltinScalarFunction::Radians => &["radians"], + BuiltinScalarFunction::Random => &["random"], + BuiltinScalarFunction::Round => &["round"], + BuiltinScalarFunction::Signum => &["signum"], + BuiltinScalarFunction::Sin => &["sin"], + BuiltinScalarFunction::Sinh => &["sinh"], + BuiltinScalarFunction::Sqrt => &["sqrt"], + BuiltinScalarFunction::Tan => &["tan"], + BuiltinScalarFunction::Tanh => &["tanh"], + BuiltinScalarFunction::Trunc => &["trunc"], - // conditional functions - BuiltinScalarFunction::Coalesce => &["coalesce"], - BuiltinScalarFunction::NullIf => &["nullif"], + // conditional functions + BuiltinScalarFunction::Coalesce => &["coalesce"], + BuiltinScalarFunction::NullIf => &["nullif"], - // string functions - BuiltinScalarFunction::Ascii => &["ascii"], - BuiltinScalarFunction::BitLength => &["bit_length"], - BuiltinScalarFunction::Btrim => &["btrim"], - BuiltinScalarFunction::CharacterLength => { - &["character_length", "char_length", "length"] - } - BuiltinScalarFunction::Concat => &["concat"], - BuiltinScalarFunction::ConcatWithSeparator => &["concat_ws"], - BuiltinScalarFunction::Chr => &["chr"], - BuiltinScalarFunction::InitCap => &["initcap"], - BuiltinScalarFunction::Left => &["left"], - BuiltinScalarFunction::Lower => &["lower"], - BuiltinScalarFunction::Lpad => &["lpad"], - BuiltinScalarFunction::Ltrim => &["ltrim"], - BuiltinScalarFunction::OctetLength => &["octet_length"], - BuiltinScalarFunction::Repeat => &["repeat"], - BuiltinScalarFunction::Replace => &["replace"], - BuiltinScalarFunction::Reverse => &["reverse"], - BuiltinScalarFunction::Right => &["right"], - BuiltinScalarFunction::Rpad => &["rpad"], - BuiltinScalarFunction::Rtrim => &["rtrim"], - BuiltinScalarFunction::SplitPart => &["split_part"], - BuiltinScalarFunction::StringToArray => &["string_to_array", "string_to_list"], - BuiltinScalarFunction::StartsWith => &["starts_with"], - BuiltinScalarFunction::Strpos => &["strpos"], - BuiltinScalarFunction::Substr => &["substr"], - BuiltinScalarFunction::ToHex => &["to_hex"], - BuiltinScalarFunction::Translate => &["translate"], - BuiltinScalarFunction::Trim => &["trim"], - BuiltinScalarFunction::Upper => &["upper"], - BuiltinScalarFunction::Uuid => &["uuid"], - BuiltinScalarFunction::Levenshtein => &["levenshtein"], - BuiltinScalarFunction::SubstrIndex => &["substr_index", "substring_index"], + // string functions + BuiltinScalarFunction::Ascii => &["ascii"], + BuiltinScalarFunction::BitLength => &["bit_length"], + BuiltinScalarFunction::Btrim => &["btrim"], + BuiltinScalarFunction::CharacterLength => { + &["character_length", "char_length", "length"] + } + BuiltinScalarFunction::Concat => &["concat"], + BuiltinScalarFunction::ConcatWithSeparator => &["concat_ws"], + BuiltinScalarFunction::Chr => &["chr"], + BuiltinScalarFunction::InitCap => &["initcap"], + BuiltinScalarFunction::Left => &["left"], + BuiltinScalarFunction::Lower => &["lower"], + BuiltinScalarFunction::Lpad => &["lpad"], + BuiltinScalarFunction::Ltrim => &["ltrim"], + BuiltinScalarFunction::OctetLength => &["octet_length"], + BuiltinScalarFunction::Repeat => &["repeat"], + BuiltinScalarFunction::Replace => &["replace"], + BuiltinScalarFunction::Reverse => &["reverse"], + BuiltinScalarFunction::Right => &["right"], + BuiltinScalarFunction::Rpad => &["rpad"], + BuiltinScalarFunction::Rtrim => &["rtrim"], + BuiltinScalarFunction::SplitPart => &["split_part"], + BuiltinScalarFunction::StringToArray => { + &["string_to_array", "string_to_list"] + } + BuiltinScalarFunction::StartsWith => &["starts_with"], + BuiltinScalarFunction::Strpos => &["strpos"], + BuiltinScalarFunction::Substr => &["substr"], + BuiltinScalarFunction::ToHex => &["to_hex"], + BuiltinScalarFunction::Translate => &["translate"], + BuiltinScalarFunction::Trim => &["trim"], + BuiltinScalarFunction::Upper => &["upper"], + BuiltinScalarFunction::Uuid => &["uuid"], + BuiltinScalarFunction::Levenshtein => &["levenshtein"], + BuiltinScalarFunction::SubstrIndex => &["substr_index", "substring_index"], - // regex functions - BuiltinScalarFunction::RegexpMatch => &["regexp_match"], - BuiltinScalarFunction::RegexpReplace => &["regexp_replace"], + // regex functions + BuiltinScalarFunction::RegexpMatch => &["regexp_match"], + BuiltinScalarFunction::RegexpReplace => &["regexp_replace"], - // time/date functions - BuiltinScalarFunction::Now => &["now"], - BuiltinScalarFunction::CurrentDate => &["current_date"], - BuiltinScalarFunction::CurrentTime => &["current_time"], - BuiltinScalarFunction::DateBin => &["date_bin"], - BuiltinScalarFunction::DateTrunc => &["date_trunc", "datetrunc"], - BuiltinScalarFunction::DatePart => &["date_part", "datepart"], - BuiltinScalarFunction::ToTimestamp => &["to_timestamp"], - BuiltinScalarFunction::ToTimestampMillis => &["to_timestamp_millis"], - BuiltinScalarFunction::ToTimestampMicros => &["to_timestamp_micros"], - BuiltinScalarFunction::ToTimestampSeconds => &["to_timestamp_seconds"], - BuiltinScalarFunction::ToTimestampNanos => &["to_timestamp_nanos"], - BuiltinScalarFunction::FromUnixtime => &["from_unixtime"], + // time/date functions + BuiltinScalarFunction::Now => &["now"], + BuiltinScalarFunction::CurrentDate => &["current_date"], + BuiltinScalarFunction::CurrentTime => &["current_time"], + BuiltinScalarFunction::DateBin => &["date_bin"], + BuiltinScalarFunction::DateTrunc => &["date_trunc", "datetrunc"], + BuiltinScalarFunction::DatePart => &["date_part", "datepart"], + BuiltinScalarFunction::ToTimestamp => &["to_timestamp"], + BuiltinScalarFunction::ToTimestampMillis => &["to_timestamp_millis"], + BuiltinScalarFunction::ToTimestampMicros => &["to_timestamp_micros"], + BuiltinScalarFunction::ToTimestampSeconds => &["to_timestamp_seconds"], + BuiltinScalarFunction::ToTimestampNanos => &["to_timestamp_nanos"], + BuiltinScalarFunction::FromUnixtime => &["from_unixtime"], - // hashing functions - BuiltinScalarFunction::Digest => &["digest"], - BuiltinScalarFunction::MD5 => &["md5"], - BuiltinScalarFunction::SHA224 => &["sha224"], - BuiltinScalarFunction::SHA256 => &["sha256"], - BuiltinScalarFunction::SHA384 => &["sha384"], - BuiltinScalarFunction::SHA512 => &["sha512"], + // hashing functions + BuiltinScalarFunction::Digest => &["digest"], + BuiltinScalarFunction::MD5 => &["md5"], + BuiltinScalarFunction::SHA224 => &["sha224"], + BuiltinScalarFunction::SHA256 => &["sha256"], + BuiltinScalarFunction::SHA384 => &["sha384"], + BuiltinScalarFunction::SHA512 => &["sha512"], - // encode/decode - BuiltinScalarFunction::Encode => &["encode"], - BuiltinScalarFunction::Decode => &["decode"], + // encode/decode + BuiltinScalarFunction::Encode => &["encode"], + BuiltinScalarFunction::Decode => &["decode"], - // other functions - BuiltinScalarFunction::ArrowTypeof => &["arrow_typeof"], + // other functions + BuiltinScalarFunction::ArrowTypeof => &["arrow_typeof"], - // array functions - BuiltinScalarFunction::ArrayAppend => &[ - "array_append", - "list_append", - "array_push_back", - "list_push_back", - ], - BuiltinScalarFunction::ArrayConcat => { - &["array_concat", "array_cat", "list_concat", "list_cat"] - } - BuiltinScalarFunction::ArrayDims => &["array_dims", "list_dims"], - BuiltinScalarFunction::ArrayEmpty => &["empty"], - BuiltinScalarFunction::ArrayElement => &[ - "array_element", - "array_extract", - "list_element", - "list_extract", - ], - BuiltinScalarFunction::ArrayExcept => &["array_except", "list_except"], - BuiltinScalarFunction::Flatten => &["flatten"], - BuiltinScalarFunction::ArrayHasAll => &["array_has_all", "list_has_all"], - BuiltinScalarFunction::ArrayHasAny => &["array_has_any", "list_has_any"], - BuiltinScalarFunction::ArrayHas => { - &["array_has", "list_has", "array_contains", "list_contains"] - } - BuiltinScalarFunction::ArrayLength => &["array_length", "list_length"], - BuiltinScalarFunction::ArrayNdims => &["array_ndims", "list_ndims"], - BuiltinScalarFunction::ArrayPopFront => &["array_pop_front", "list_pop_front"], - BuiltinScalarFunction::ArrayPopBack => &["array_pop_back", "list_pop_back"], - BuiltinScalarFunction::ArrayPosition => &[ - "array_position", - "list_position", - "array_indexof", - "list_indexof", - ], - BuiltinScalarFunction::ArrayPositions => &["array_positions", "list_positions"], - BuiltinScalarFunction::ArrayPrepend => &[ - "array_prepend", - "list_prepend", - "array_push_front", - "list_push_front", - ], - BuiltinScalarFunction::ArrayRepeat => &["array_repeat", "list_repeat"], - BuiltinScalarFunction::ArrayRemove => &["array_remove", "list_remove"], - BuiltinScalarFunction::ArrayRemoveN => &["array_remove_n", "list_remove_n"], - BuiltinScalarFunction::ArrayRemoveAll => &["array_remove_all", "list_remove_all"], - BuiltinScalarFunction::ArrayReplace => &["array_replace", "list_replace"], - BuiltinScalarFunction::ArrayReplaceN => &["array_replace_n", "list_replace_n"], - BuiltinScalarFunction::ArrayReplaceAll => { - &["array_replace_all", "list_replace_all"] - } - BuiltinScalarFunction::ArraySlice => &["array_slice", "list_slice"], - BuiltinScalarFunction::ArrayToString => &[ - "array_to_string", - "list_to_string", - "array_join", - "list_join", - ], - BuiltinScalarFunction::ArrayUnion => &["array_union", "list_union"], - BuiltinScalarFunction::Cardinality => &["cardinality"], - BuiltinScalarFunction::MakeArray => &["make_array", "make_list"], - BuiltinScalarFunction::ArrayIntersect => &["array_intersect", "list_intersect"], - BuiltinScalarFunction::OverLay => &["overlay"], - BuiltinScalarFunction::Range => &["range", "generate_series"], + // array functions + BuiltinScalarFunction::ArrayAppend => &[ + "array_append", + "list_append", + "array_push_back", + "list_push_back", + ], + BuiltinScalarFunction::ArrayConcat => { + &["array_concat", "array_cat", "list_concat", "list_cat"] + } + BuiltinScalarFunction::ArrayDims => &["array_dims", "list_dims"], + BuiltinScalarFunction::ArrayEmpty => &["empty"], + BuiltinScalarFunction::ArrayElement => &[ + "array_element", + "array_extract", + "list_element", + "list_extract", + ], + BuiltinScalarFunction::ArrayExcept => &["array_except", "list_except"], + BuiltinScalarFunction::Flatten => &["flatten"], + BuiltinScalarFunction::ArrayHasAll => &["array_has_all", "list_has_all"], + BuiltinScalarFunction::ArrayHasAny => &["array_has_any", "list_has_any"], + BuiltinScalarFunction::ArrayHas => { + &["array_has", "list_has", "array_contains", "list_contains"] + } + BuiltinScalarFunction::ArrayLength => &["array_length", "list_length"], + BuiltinScalarFunction::ArrayNdims => &["array_ndims", "list_ndims"], + BuiltinScalarFunction::ArrayPopFront => { + &["array_pop_front", "list_pop_front"] + } + BuiltinScalarFunction::ArrayPopBack => &["array_pop_back", "list_pop_back"], + BuiltinScalarFunction::ArrayPosition => &[ + "array_position", + "list_position", + "array_indexof", + "list_indexof", + ], + BuiltinScalarFunction::ArrayPositions => { + &["array_positions", "list_positions"] + } + BuiltinScalarFunction::ArrayPrepend => &[ + "array_prepend", + "list_prepend", + "array_push_front", + "list_push_front", + ], + BuiltinScalarFunction::ArrayRepeat => &["array_repeat", "list_repeat"], + BuiltinScalarFunction::ArrayRemove => &["array_remove", "list_remove"], + BuiltinScalarFunction::ArrayRemoveN => &["array_remove_n", "list_remove_n"], + BuiltinScalarFunction::ArrayRemoveAll => { + &["array_remove_all", "list_remove_all"] + } + BuiltinScalarFunction::ArrayReplace => &["array_replace", "list_replace"], + BuiltinScalarFunction::ArrayReplaceN => { + &["array_replace_n", "list_replace_n"] + } + BuiltinScalarFunction::ArrayReplaceAll => { + &["array_replace_all", "list_replace_all"] + } + BuiltinScalarFunction::ArraySlice => &["array_slice", "list_slice"], + BuiltinScalarFunction::ArrayToString => &[ + "array_to_string", + "list_to_string", + "array_join", + "list_join", + ], + BuiltinScalarFunction::ArrayUnion => &["array_union", "list_union"], + BuiltinScalarFunction::Cardinality => &["cardinality"], + BuiltinScalarFunction::MakeArray => &["make_array", "make_list"], + BuiltinScalarFunction::ArrayIntersect => { + &["array_intersect", "list_intersect"] + } + BuiltinScalarFunction::OverLay => &["overlay"], + BuiltinScalarFunction::Range => &["range", "generate_series"], - // struct functions - BuiltinScalarFunction::Struct => &["struct"], + // struct functions + BuiltinScalarFunction::Struct => &["struct"], + } } } From 06bbe1298fa8aa042b6a6462e55b2890969d884a Mon Sep 17 00:00:00 2001 From: Andrew Lamb Date: Wed, 29 Nov 2023 14:10:06 -0500 Subject: [PATCH 333/572] Extract parquet statistics to its own module, add tests (#8294) * Extract parquet statistics to its own module, add tests * Update datafusion/core/src/datasource/physical_plan/parquet/statistics.rs Co-authored-by: Raphael Taylor-Davies <1781103+tustvold@users.noreply.github.com> * rename enum * Improve API * Add test for reading struct array statistics * Add test for column after statistics * improve tests * simplify * clippy * Update datafusion/core/src/datasource/physical_plan/parquet/statistics.rs * Update datafusion/core/src/datasource/physical_plan/parquet/statistics.rs * Add test showing incorrect statistics * Rework statistics * Fix clippy * Update documentation and make it clear the statistics are not publically accessable * Add link to upstream arrow ticket --------- Co-authored-by: Raphael Taylor-Davies <1781103+tustvold@users.noreply.github.com> Co-authored-by: Raphael Taylor-Davies --- .../datasource/physical_plan/parquet/mod.rs | 24 +- .../physical_plan/parquet/page_filter.rs | 5 +- .../physical_plan/parquet/row_groups.rs | 189 +--- .../physical_plan/parquet/statistics.rs | 899 ++++++++++++++++++ 4 files changed, 951 insertions(+), 166 deletions(-) create mode 100644 datafusion/core/src/datasource/physical_plan/parquet/statistics.rs diff --git a/datafusion/core/src/datasource/physical_plan/parquet/mod.rs b/datafusion/core/src/datasource/physical_plan/parquet/mod.rs index 731672ceb8b8..95aae71c779e 100644 --- a/datafusion/core/src/datasource/physical_plan/parquet/mod.rs +++ b/datafusion/core/src/datasource/physical_plan/parquet/mod.rs @@ -66,6 +66,7 @@ mod metrics; pub mod page_filter; mod row_filter; mod row_groups; +mod statistics; pub use metrics::ParquetFileMetrics; @@ -506,6 +507,7 @@ impl FileOpener for ParquetOpener { let file_metadata = builder.metadata().clone(); let predicate = pruning_predicate.as_ref().map(|p| p.as_ref()); let mut row_groups = row_groups::prune_row_groups_by_statistics( + builder.parquet_schema(), file_metadata.row_groups(), file_range, predicate, @@ -718,28 +720,6 @@ pub async fn plan_to_parquet( Ok(()) } -// Copy from the arrow-rs -// https://github.com/apache/arrow-rs/blob/733b7e7fd1e8c43a404c3ce40ecf741d493c21b4/parquet/src/arrow/buffer/bit_util.rs#L55 -// Convert the byte slice to fixed length byte array with the length of 16 -fn sign_extend_be(b: &[u8]) -> [u8; 16] { - assert!(b.len() <= 16, "Array too large, expected less than 16"); - let is_negative = (b[0] & 128u8) == 128u8; - let mut result = if is_negative { [255u8; 16] } else { [0u8; 16] }; - for (d, s) in result.iter_mut().skip(16 - b.len()).zip(b) { - *d = *s; - } - result -} - -// Convert the bytes array to i128. -// The endian of the input bytes array must be big-endian. -pub(crate) fn from_bytes_to_i128(b: &[u8]) -> i128 { - // The bytes array are from parquet file and must be the big-endian. - // The endian is defined by parquet format, and the reference document - // https://github.com/apache/parquet-format/blob/54e53e5d7794d383529dd30746378f19a12afd58/src/main/thrift/parquet.thrift#L66 - i128::from_be_bytes(sign_extend_be(b)) -} - // Convert parquet column schema to arrow data type, and just consider the // decimal data type. pub(crate) fn parquet_to_arrow_decimal_type( diff --git a/datafusion/core/src/datasource/physical_plan/parquet/page_filter.rs b/datafusion/core/src/datasource/physical_plan/parquet/page_filter.rs index b5b5f154f7a0..42bfef35996e 100644 --- a/datafusion/core/src/datasource/physical_plan/parquet/page_filter.rs +++ b/datafusion/core/src/datasource/physical_plan/parquet/page_filter.rs @@ -39,9 +39,8 @@ use parquet::{ }; use std::sync::Arc; -use crate::datasource::physical_plan::parquet::{ - from_bytes_to_i128, parquet_to_arrow_decimal_type, -}; +use crate::datasource::physical_plan::parquet::parquet_to_arrow_decimal_type; +use crate::datasource::physical_plan::parquet::statistics::from_bytes_to_i128; use crate::physical_optimizer::pruning::{PruningPredicate, PruningStatistics}; use super::metrics::ParquetFileMetrics; diff --git a/datafusion/core/src/datasource/physical_plan/parquet/row_groups.rs b/datafusion/core/src/datasource/physical_plan/parquet/row_groups.rs index 0079368f9cdd..0ab2046097c4 100644 --- a/datafusion/core/src/datasource/physical_plan/parquet/row_groups.rs +++ b/datafusion/core/src/datasource/physical_plan/parquet/row_groups.rs @@ -15,25 +15,25 @@ // specific language governing permissions and limitations // under the License. -use arrow::{ - array::ArrayRef, - datatypes::{DataType, Schema}, -}; +use arrow::{array::ArrayRef, datatypes::Schema}; +use arrow_schema::FieldRef; use datafusion_common::tree_node::{TreeNode, VisitRecursion}; use datafusion_common::{Column, DataFusionError, Result, ScalarValue}; +use parquet::file::metadata::ColumnChunkMetaData; +use parquet::schema::types::SchemaDescriptor; use parquet::{ arrow::{async_reader::AsyncFileReader, ParquetRecordBatchStreamBuilder}, bloom_filter::Sbbf, - file::{metadata::RowGroupMetaData, statistics::Statistics as ParquetStatistics}, + file::metadata::RowGroupMetaData, }; use std::{ collections::{HashMap, HashSet}, sync::Arc, }; -use crate::datasource::{ - listing::FileRange, - physical_plan::parquet::{from_bytes_to_i128, parquet_to_arrow_decimal_type}, +use crate::datasource::listing::FileRange; +use crate::datasource::physical_plan::parquet::statistics::{ + max_statistics, min_statistics, parquet_column, }; use crate::logical_expr::Operator; use crate::physical_expr::expressions as phys_expr; @@ -51,7 +51,11 @@ use super::ParquetFileMetrics; /// /// If an index IS present in the returned Vec it means the predicate /// did not filter out that row group. +/// +/// Note: This method currently ignores ColumnOrder +/// pub(crate) fn prune_row_groups_by_statistics( + parquet_schema: &SchemaDescriptor, groups: &[RowGroupMetaData], range: Option, predicate: Option<&PruningPredicate>, @@ -74,8 +78,9 @@ pub(crate) fn prune_row_groups_by_statistics( if let Some(predicate) = predicate { let pruning_stats = RowGroupPruningStatistics { + parquet_schema, row_group_metadata: metadata, - parquet_schema: predicate.schema().as_ref(), + arrow_schema: predicate.schema().as_ref(), }; match predicate.prune(&pruning_stats) { Ok(values) => { @@ -296,146 +301,33 @@ impl BloomFilterPruningPredicate { } } -/// Wraps parquet statistics in a way -/// that implements [`PruningStatistics`] +/// Wraps [`RowGroupMetaData`] in a way that implements [`PruningStatistics`] +/// +/// Note: This should be implemented for an array of [`RowGroupMetaData`] instead +/// of per row-group struct RowGroupPruningStatistics<'a> { + parquet_schema: &'a SchemaDescriptor, row_group_metadata: &'a RowGroupMetaData, - parquet_schema: &'a Schema, + arrow_schema: &'a Schema, } -/// Extract the min/max statistics from a `ParquetStatistics` object -macro_rules! get_statistic { - ($column_statistics:expr, $func:ident, $bytes_func:ident, $target_arrow_type:expr) => {{ - if !$column_statistics.has_min_max_set() { - return None; - } - match $column_statistics { - ParquetStatistics::Boolean(s) => Some(ScalarValue::Boolean(Some(*s.$func()))), - ParquetStatistics::Int32(s) => { - match $target_arrow_type { - // int32 to decimal with the precision and scale - Some(DataType::Decimal128(precision, scale)) => { - Some(ScalarValue::Decimal128( - Some(*s.$func() as i128), - precision, - scale, - )) - } - _ => Some(ScalarValue::Int32(Some(*s.$func()))), - } - } - ParquetStatistics::Int64(s) => { - match $target_arrow_type { - // int64 to decimal with the precision and scale - Some(DataType::Decimal128(precision, scale)) => { - Some(ScalarValue::Decimal128( - Some(*s.$func() as i128), - precision, - scale, - )) - } - _ => Some(ScalarValue::Int64(Some(*s.$func()))), - } - } - // 96 bit ints not supported - ParquetStatistics::Int96(_) => None, - ParquetStatistics::Float(s) => Some(ScalarValue::Float32(Some(*s.$func()))), - ParquetStatistics::Double(s) => Some(ScalarValue::Float64(Some(*s.$func()))), - ParquetStatistics::ByteArray(s) => { - match $target_arrow_type { - // decimal data type - Some(DataType::Decimal128(precision, scale)) => { - Some(ScalarValue::Decimal128( - Some(from_bytes_to_i128(s.$bytes_func())), - precision, - scale, - )) - } - _ => { - let s = std::str::from_utf8(s.$bytes_func()) - .map(|s| s.to_string()) - .ok(); - Some(ScalarValue::Utf8(s)) - } - } - } - // type not supported yet - ParquetStatistics::FixedLenByteArray(s) => { - match $target_arrow_type { - // just support the decimal data type - Some(DataType::Decimal128(precision, scale)) => { - Some(ScalarValue::Decimal128( - Some(from_bytes_to_i128(s.$bytes_func())), - precision, - scale, - )) - } - _ => None, - } - } - } - }}; -} - -// Extract the min or max value calling `func` or `bytes_func` on the ParquetStatistics as appropriate -macro_rules! get_min_max_values { - ($self:expr, $column:expr, $func:ident, $bytes_func:ident) => {{ - let (_column_index, field) = - if let Some((v, f)) = $self.parquet_schema.column_with_name(&$column.name) { - (v, f) - } else { - // Named column was not present - return None; - }; - - let data_type = field.data_type(); - // The result may be None, because DataFusion doesn't have support for ScalarValues of the column type - let null_scalar: ScalarValue = data_type.try_into().ok()?; - - $self.row_group_metadata - .columns() - .iter() - .find(|c| c.column_descr().name() == &$column.name) - .and_then(|c| if c.statistics().is_some() {Some((c.statistics().unwrap(), c.column_descr()))} else {None}) - .map(|(stats, column_descr)| - { - let target_data_type = parquet_to_arrow_decimal_type(column_descr); - get_statistic!(stats, $func, $bytes_func, target_data_type) - }) - .flatten() - // column either didn't have statistics at all or didn't have min/max values - .or_else(|| Some(null_scalar.clone())) - .and_then(|s| s.to_array().ok()) - }} -} - -// Extract the null count value on the ParquetStatistics -macro_rules! get_null_count_values { - ($self:expr, $column:expr) => {{ - let value = ScalarValue::UInt64( - if let Some(col) = $self - .row_group_metadata - .columns() - .iter() - .find(|c| c.column_descr().name() == &$column.name) - { - col.statistics().map(|s| s.null_count()) - } else { - Some($self.row_group_metadata.num_rows() as u64) - }, - ); - - value.to_array().ok() - }}; +impl<'a> RowGroupPruningStatistics<'a> { + /// Lookups up the parquet column by name + fn column(&self, name: &str) -> Option<(&ColumnChunkMetaData, &FieldRef)> { + let (idx, field) = parquet_column(self.parquet_schema, self.arrow_schema, name)?; + Some((self.row_group_metadata.column(idx), field)) + } } impl<'a> PruningStatistics for RowGroupPruningStatistics<'a> { fn min_values(&self, column: &Column) -> Option { - get_min_max_values!(self, column, min, min_bytes) + let (column, field) = self.column(&column.name)?; + min_statistics(field.data_type(), std::iter::once(column.statistics())).ok() } fn max_values(&self, column: &Column) -> Option { - get_min_max_values!(self, column, max, max_bytes) + let (column, field) = self.column(&column.name)?; + max_statistics(field.data_type(), std::iter::once(column.statistics())).ok() } fn num_containers(&self) -> usize { @@ -443,7 +335,9 @@ impl<'a> PruningStatistics for RowGroupPruningStatistics<'a> { } fn null_counts(&self, column: &Column) -> Option { - get_null_count_values!(self, column) + let (c, _) = self.column(&column.name)?; + let scalar = ScalarValue::UInt64(Some(c.statistics()?.null_count())); + scalar.to_array().ok() } } @@ -463,6 +357,7 @@ mod tests { use datafusion_physical_expr::execution_props::ExecutionProps; use datafusion_physical_expr::{create_physical_expr, PhysicalExpr}; use datafusion_sql::planner::ContextProvider; + use parquet::arrow::arrow_to_parquet_schema; use parquet::arrow::async_reader::ParquetObjectReader; use parquet::basic::LogicalType; use parquet::data_type::{ByteArray, FixedLenByteArray}; @@ -540,6 +435,7 @@ mod tests { let metrics = parquet_file_metrics(); assert_eq!( prune_row_groups_by_statistics( + &schema_descr, &[rgm1, rgm2], None, Some(&pruning_predicate), @@ -574,6 +470,7 @@ mod tests { // is null / undefined so the first row group can't be filtered out assert_eq!( prune_row_groups_by_statistics( + &schema_descr, &[rgm1, rgm2], None, Some(&pruning_predicate), @@ -621,6 +518,7 @@ mod tests { // when conditions are joined using AND assert_eq!( prune_row_groups_by_statistics( + &schema_descr, groups, None, Some(&pruning_predicate), @@ -639,6 +537,7 @@ mod tests { // this bypasses the entire predicate expression and no row groups are filtered out assert_eq!( prune_row_groups_by_statistics( + &schema_descr, groups, None, Some(&pruning_predicate), @@ -678,6 +577,7 @@ mod tests { Field::new("c1", DataType::Int32, false), Field::new("c2", DataType::Boolean, false), ])); + let schema_descr = arrow_to_parquet_schema(&schema).unwrap(); let expr = col("c1").gt(lit(15)).and(col("c2").is_null()); let expr = logical2physical(&expr, &schema); let pruning_predicate = PruningPredicate::try_new(expr, schema).unwrap(); @@ -687,6 +587,7 @@ mod tests { // First row group was filtered out because it contains no null value on "c2". assert_eq!( prune_row_groups_by_statistics( + &schema_descr, &groups, None, Some(&pruning_predicate), @@ -706,6 +607,7 @@ mod tests { Field::new("c1", DataType::Int32, false), Field::new("c2", DataType::Boolean, false), ])); + let schema_descr = arrow_to_parquet_schema(&schema).unwrap(); let expr = col("c1") .gt(lit(15)) .and(col("c2").eq(lit(ScalarValue::Boolean(None)))); @@ -718,6 +620,7 @@ mod tests { // pass predicates. Ideally these should both be false assert_eq!( prune_row_groups_by_statistics( + &schema_descr, &groups, None, Some(&pruning_predicate), @@ -776,6 +679,7 @@ mod tests { let metrics = parquet_file_metrics(); assert_eq!( prune_row_groups_by_statistics( + &schema_descr, &[rgm1, rgm2, rgm3], None, Some(&pruning_predicate), @@ -839,6 +743,7 @@ mod tests { let metrics = parquet_file_metrics(); assert_eq!( prune_row_groups_by_statistics( + &schema_descr, &[rgm1, rgm2, rgm3, rgm4], None, Some(&pruning_predicate), @@ -886,6 +791,7 @@ mod tests { let metrics = parquet_file_metrics(); assert_eq!( prune_row_groups_by_statistics( + &schema_descr, &[rgm1, rgm2, rgm3], None, Some(&pruning_predicate), @@ -956,6 +862,7 @@ mod tests { let metrics = parquet_file_metrics(); assert_eq!( prune_row_groups_by_statistics( + &schema_descr, &[rgm1, rgm2, rgm3], None, Some(&pruning_predicate), @@ -1015,6 +922,7 @@ mod tests { let metrics = parquet_file_metrics(); assert_eq!( prune_row_groups_by_statistics( + &schema_descr, &[rgm1, rgm2, rgm3], None, Some(&pruning_predicate), @@ -1028,7 +936,6 @@ mod tests { schema_descr: &SchemaDescPtr, column_statistics: Vec, ) -> RowGroupMetaData { - use parquet::file::metadata::ColumnChunkMetaData; let mut columns = vec![]; for (i, s) in column_statistics.iter().enumerate() { let column = ColumnChunkMetaData::builder(schema_descr.column(i)) @@ -1046,7 +953,7 @@ mod tests { } fn get_test_schema_descr(fields: Vec) -> SchemaDescPtr { - use parquet::schema::types::{SchemaDescriptor, Type as SchemaType}; + use parquet::schema::types::Type as SchemaType; let schema_fields = fields .iter() .map(|field| { diff --git a/datafusion/core/src/datasource/physical_plan/parquet/statistics.rs b/datafusion/core/src/datasource/physical_plan/parquet/statistics.rs new file mode 100644 index 000000000000..4e472606da51 --- /dev/null +++ b/datafusion/core/src/datasource/physical_plan/parquet/statistics.rs @@ -0,0 +1,899 @@ +// 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. + +//! [`min_statistics`] and [`max_statistics`] convert statistics in parquet format to arrow [`ArrayRef`]. + +// TODO: potentially move this to arrow-rs: https://github.com/apache/arrow-rs/issues/4328 + +use arrow::{array::ArrayRef, datatypes::DataType}; +use arrow_array::new_empty_array; +use arrow_schema::{FieldRef, Schema}; +use datafusion_common::{Result, ScalarValue}; +use parquet::file::statistics::Statistics as ParquetStatistics; +use parquet::schema::types::SchemaDescriptor; + +// Convert the bytes array to i128. +// The endian of the input bytes array must be big-endian. +pub(crate) fn from_bytes_to_i128(b: &[u8]) -> i128 { + // The bytes array are from parquet file and must be the big-endian. + // The endian is defined by parquet format, and the reference document + // https://github.com/apache/parquet-format/blob/54e53e5d7794d383529dd30746378f19a12afd58/src/main/thrift/parquet.thrift#L66 + i128::from_be_bytes(sign_extend_be(b)) +} + +// Copy from arrow-rs +// https://github.com/apache/arrow-rs/blob/733b7e7fd1e8c43a404c3ce40ecf741d493c21b4/parquet/src/arrow/buffer/bit_util.rs#L55 +// Convert the byte slice to fixed length byte array with the length of 16 +fn sign_extend_be(b: &[u8]) -> [u8; 16] { + assert!(b.len() <= 16, "Array too large, expected less than 16"); + let is_negative = (b[0] & 128u8) == 128u8; + let mut result = if is_negative { [255u8; 16] } else { [0u8; 16] }; + for (d, s) in result.iter_mut().skip(16 - b.len()).zip(b) { + *d = *s; + } + result +} + +/// Extract a single min/max statistics from a [`ParquetStatistics`] object +/// +/// * `$column_statistics` is the `ParquetStatistics` object +/// * `$func is the function` (`min`/`max`) to call to get the value +/// * `$bytes_func` is the function (`min_bytes`/`max_bytes`) to call to get the value as bytes +/// * `$target_arrow_type` is the [`DataType`] of the target statistics +macro_rules! get_statistic { + ($column_statistics:expr, $func:ident, $bytes_func:ident, $target_arrow_type:expr) => {{ + if !$column_statistics.has_min_max_set() { + return None; + } + match $column_statistics { + ParquetStatistics::Boolean(s) => Some(ScalarValue::Boolean(Some(*s.$func()))), + ParquetStatistics::Int32(s) => { + match $target_arrow_type { + // int32 to decimal with the precision and scale + Some(DataType::Decimal128(precision, scale)) => { + Some(ScalarValue::Decimal128( + Some(*s.$func() as i128), + *precision, + *scale, + )) + } + _ => Some(ScalarValue::Int32(Some(*s.$func()))), + } + } + ParquetStatistics::Int64(s) => { + match $target_arrow_type { + // int64 to decimal with the precision and scale + Some(DataType::Decimal128(precision, scale)) => { + Some(ScalarValue::Decimal128( + Some(*s.$func() as i128), + *precision, + *scale, + )) + } + _ => Some(ScalarValue::Int64(Some(*s.$func()))), + } + } + // 96 bit ints not supported + ParquetStatistics::Int96(_) => None, + ParquetStatistics::Float(s) => Some(ScalarValue::Float32(Some(*s.$func()))), + ParquetStatistics::Double(s) => Some(ScalarValue::Float64(Some(*s.$func()))), + ParquetStatistics::ByteArray(s) => { + match $target_arrow_type { + // decimal data type + Some(DataType::Decimal128(precision, scale)) => { + Some(ScalarValue::Decimal128( + Some(from_bytes_to_i128(s.$bytes_func())), + *precision, + *scale, + )) + } + _ => { + let s = std::str::from_utf8(s.$bytes_func()) + .map(|s| s.to_string()) + .ok(); + Some(ScalarValue::Utf8(s)) + } + } + } + // type not supported yet + ParquetStatistics::FixedLenByteArray(s) => { + match $target_arrow_type { + // just support the decimal data type + Some(DataType::Decimal128(precision, scale)) => { + Some(ScalarValue::Decimal128( + Some(from_bytes_to_i128(s.$bytes_func())), + *precision, + *scale, + )) + } + _ => None, + } + } + } + }}; +} + +/// Lookups up the parquet column by name +/// +/// Returns the parquet column index and the corresponding arrow field +pub(crate) fn parquet_column<'a>( + parquet_schema: &SchemaDescriptor, + arrow_schema: &'a Schema, + name: &str, +) -> Option<(usize, &'a FieldRef)> { + let (root_idx, field) = arrow_schema.fields.find(name)?; + if field.data_type().is_nested() { + // Nested fields are not supported and require non-trivial logic + // to correctly walk the parquet schema accounting for the + // logical type rules - + // + // For example a ListArray could correspond to anything from 1 to 3 levels + // in the parquet schema + return None; + } + + // This could be made more efficient (#TBD) + let parquet_idx = (0..parquet_schema.columns().len()) + .find(|x| parquet_schema.get_column_root_idx(*x) == root_idx)?; + Some((parquet_idx, field)) +} + +/// Extracts the min statistics from an iterator of [`ParquetStatistics`] to an [`ArrayRef`] +pub(crate) fn min_statistics<'a, I: Iterator>>( + data_type: &DataType, + iterator: I, +) -> Result { + let scalars = iterator + .map(|x| x.and_then(|s| get_statistic!(s, min, min_bytes, Some(data_type)))); + collect_scalars(data_type, scalars) +} + +/// Extracts the max statistics from an iterator of [`ParquetStatistics`] to an [`ArrayRef`] +pub(crate) fn max_statistics<'a, I: Iterator>>( + data_type: &DataType, + iterator: I, +) -> Result { + let scalars = iterator + .map(|x| x.and_then(|s| get_statistic!(s, max, max_bytes, Some(data_type)))); + collect_scalars(data_type, scalars) +} + +/// Builds an array from an iterator of ScalarValue +fn collect_scalars>>( + data_type: &DataType, + iterator: I, +) -> Result { + let mut scalars = iterator.peekable(); + match scalars.peek().is_none() { + true => Ok(new_empty_array(data_type)), + false => { + let null = ScalarValue::try_from(data_type)?; + ScalarValue::iter_to_array(scalars.map(|x| x.unwrap_or_else(|| null.clone()))) + } + } +} + +#[cfg(test)] +mod test { + use super::*; + use arrow_array::{ + new_null_array, Array, BinaryArray, BooleanArray, Decimal128Array, Float32Array, + Float64Array, Int32Array, Int64Array, RecordBatch, StringArray, StructArray, + TimestampNanosecondArray, + }; + use arrow_schema::{Field, SchemaRef}; + use bytes::Bytes; + use datafusion_common::test_util::parquet_test_data; + use parquet::arrow::arrow_reader::ArrowReaderBuilder; + use parquet::arrow::arrow_writer::ArrowWriter; + use parquet::file::metadata::{ParquetMetaData, RowGroupMetaData}; + use parquet::file::properties::{EnabledStatistics, WriterProperties}; + use std::path::PathBuf; + use std::sync::Arc; + + // TODO error cases (with parquet statistics that are mismatched in expected type) + + #[test] + fn roundtrip_empty() { + let empty_bool_array = new_empty_array(&DataType::Boolean); + Test { + input: empty_bool_array.clone(), + expected_min: empty_bool_array.clone(), + expected_max: empty_bool_array.clone(), + } + .run() + } + + #[test] + fn roundtrip_bool() { + Test { + input: bool_array([ + // row group 1 + Some(true), + None, + Some(true), + // row group 2 + Some(true), + Some(false), + None, + // row group 3 + None, + None, + None, + ]), + expected_min: bool_array([Some(true), Some(false), None]), + expected_max: bool_array([Some(true), Some(true), None]), + } + .run() + } + + #[test] + fn roundtrip_int32() { + Test { + input: i32_array([ + // row group 1 + Some(1), + None, + Some(3), + // row group 2 + Some(0), + Some(5), + None, + // row group 3 + None, + None, + None, + ]), + expected_min: i32_array([Some(1), Some(0), None]), + expected_max: i32_array([Some(3), Some(5), None]), + } + .run() + } + + #[test] + fn roundtrip_int64() { + Test { + input: i64_array([ + // row group 1 + Some(1), + None, + Some(3), + // row group 2 + Some(0), + Some(5), + None, + // row group 3 + None, + None, + None, + ]), + expected_min: i64_array([Some(1), Some(0), None]), + expected_max: i64_array(vec![Some(3), Some(5), None]), + } + .run() + } + + #[test] + fn roundtrip_f32() { + Test { + input: f32_array([ + // row group 1 + Some(1.0), + None, + Some(3.0), + // row group 2 + Some(-1.0), + Some(5.0), + None, + // row group 3 + None, + None, + None, + ]), + expected_min: f32_array([Some(1.0), Some(-1.0), None]), + expected_max: f32_array([Some(3.0), Some(5.0), None]), + } + .run() + } + + #[test] + fn roundtrip_f64() { + Test { + input: f64_array([ + // row group 1 + Some(1.0), + None, + Some(3.0), + // row group 2 + Some(-1.0), + Some(5.0), + None, + // row group 3 + None, + None, + None, + ]), + expected_min: f64_array([Some(1.0), Some(-1.0), None]), + expected_max: f64_array([Some(3.0), Some(5.0), None]), + } + .run() + } + + #[test] + #[should_panic( + expected = "Inconsistent types in ScalarValue::iter_to_array. Expected Int64, got TimestampNanosecond(NULL, None)" + )] + // Due to https://github.com/apache/arrow-datafusion/issues/8295 + fn roundtrip_timestamp() { + Test { + input: timestamp_array([ + // row group 1 + Some(1), + None, + Some(3), + // row group 2 + Some(9), + Some(5), + None, + // row group 3 + None, + None, + None, + ]), + expected_min: timestamp_array([Some(1), Some(5), None]), + expected_max: timestamp_array([Some(3), Some(9), None]), + } + .run() + } + + #[test] + fn roundtrip_decimal() { + Test { + input: Arc::new( + Decimal128Array::from(vec![ + // row group 1 + Some(100), + None, + Some(22000), + // row group 2 + Some(500000), + Some(330000), + None, + // row group 3 + None, + None, + None, + ]) + .with_precision_and_scale(9, 2) + .unwrap(), + ), + expected_min: Arc::new( + Decimal128Array::from(vec![Some(100), Some(330000), None]) + .with_precision_and_scale(9, 2) + .unwrap(), + ), + expected_max: Arc::new( + Decimal128Array::from(vec![Some(22000), Some(500000), None]) + .with_precision_and_scale(9, 2) + .unwrap(), + ), + } + .run() + } + + #[test] + fn roundtrip_utf8() { + Test { + input: utf8_array([ + // row group 1 + Some("A"), + None, + Some("Q"), + // row group 2 + Some("ZZ"), + Some("AA"), + None, + // row group 3 + None, + None, + None, + ]), + expected_min: utf8_array([Some("A"), Some("AA"), None]), + expected_max: utf8_array([Some("Q"), Some("ZZ"), None]), + } + .run() + } + + #[test] + fn roundtrip_struct() { + let mut test = Test { + input: struct_array(vec![ + // row group 1 + (Some(true), Some(1)), + (None, None), + (Some(true), Some(3)), + // row group 2 + (Some(true), Some(0)), + (Some(false), Some(5)), + (None, None), + // row group 3 + (None, None), + (None, None), + (None, None), + ]), + expected_min: struct_array(vec![ + (Some(true), Some(1)), + (Some(true), Some(0)), + (None, None), + ]), + + expected_max: struct_array(vec![ + (Some(true), Some(3)), + (Some(true), Some(0)), + (None, None), + ]), + }; + // Due to https://github.com/apache/arrow-datafusion/issues/8334, + // statistics for struct arrays are not supported + test.expected_min = + new_null_array(test.input.data_type(), test.expected_min.len()); + test.expected_max = + new_null_array(test.input.data_type(), test.expected_min.len()); + test.run() + } + + #[test] + #[should_panic( + expected = "Inconsistent types in ScalarValue::iter_to_array. Expected Utf8, got Binary(NULL)" + )] + // Due to https://github.com/apache/arrow-datafusion/issues/8295 + fn roundtrip_binary() { + Test { + input: Arc::new(BinaryArray::from_opt_vec(vec![ + // row group 1 + Some(b"A"), + None, + Some(b"Q"), + // row group 2 + Some(b"ZZ"), + Some(b"AA"), + None, + // row group 3 + None, + None, + None, + ])), + expected_min: Arc::new(BinaryArray::from_opt_vec(vec![ + Some(b"A"), + Some(b"AA"), + None, + ])), + expected_max: Arc::new(BinaryArray::from_opt_vec(vec![ + Some(b"Q"), + Some(b"ZZ"), + None, + ])), + } + .run() + } + + #[test] + fn struct_and_non_struct() { + // Ensures that statistics for an array that appears *after* a struct + // array are not wrong + let struct_col = struct_array(vec![ + // row group 1 + (Some(true), Some(1)), + (None, None), + (Some(true), Some(3)), + ]); + let int_col = i32_array([Some(100), Some(200), Some(300)]); + let expected_min = i32_array([Some(100)]); + let expected_max = i32_array(vec![Some(300)]); + + // use a name that shadows a name in the struct column + match struct_col.data_type() { + DataType::Struct(fields) => { + assert_eq!(fields.get(1).unwrap().name(), "int_col") + } + _ => panic!("unexpected data type for struct column"), + }; + + let input_batch = RecordBatch::try_from_iter([ + ("struct_col", struct_col), + ("int_col", int_col), + ]) + .unwrap(); + + let schema = input_batch.schema(); + + let metadata = parquet_metadata(schema.clone(), input_batch); + let parquet_schema = metadata.file_metadata().schema_descr(); + + // read the int_col statistics + let (idx, _) = parquet_column(parquet_schema, &schema, "int_col").unwrap(); + assert_eq!(idx, 2); + + let row_groups = metadata.row_groups(); + let iter = row_groups.iter().map(|x| x.column(idx).statistics()); + + let min = min_statistics(&DataType::Int32, iter.clone()).unwrap(); + assert_eq!( + &min, + &expected_min, + "Min. Statistics\n\n{}\n\n", + DisplayStats(row_groups) + ); + + let max = max_statistics(&DataType::Int32, iter).unwrap(); + assert_eq!( + &max, + &expected_max, + "Max. Statistics\n\n{}\n\n", + DisplayStats(row_groups) + ); + } + + #[test] + fn nan_in_stats() { + // /parquet-testing/data/nan_in_stats.parquet + // row_groups: 1 + // "x": Double({min: Some(1.0), max: Some(NaN), distinct_count: None, null_count: 0, min_max_deprecated: false, min_max_backwards_compatible: false}) + + TestFile::new("nan_in_stats.parquet") + .with_column(ExpectedColumn { + name: "x", + expected_min: Arc::new(Float64Array::from(vec![Some(1.0)])), + expected_max: Arc::new(Float64Array::from(vec![Some(f64::NAN)])), + }) + .run(); + } + + #[test] + fn alltypes_plain() { + // /parquet-testing/data/datapage_v1-snappy-compressed-checksum.parquet + // row_groups: 1 + // (has no statistics) + TestFile::new("alltypes_plain.parquet") + // No column statistics should be read as NULL, but with the right type + .with_column(ExpectedColumn { + name: "id", + expected_min: i32_array([None]), + expected_max: i32_array([None]), + }) + .with_column(ExpectedColumn { + name: "bool_col", + expected_min: bool_array([None]), + expected_max: bool_array([None]), + }) + .run(); + } + + #[test] + fn alltypes_tiny_pages() { + // /parquet-testing/data/alltypes_tiny_pages.parquet + // row_groups: 1 + // "id": Int32({min: Some(0), max: Some(7299), distinct_count: None, null_count: 0, min_max_deprecated: false, min_max_backwards_compatible: false}) + // "bool_col": Boolean({min: Some(false), max: Some(true), distinct_count: None, null_count: 0, min_max_deprecated: false, min_max_backwards_compatible: false}) + // "tinyint_col": Int32({min: Some(0), max: Some(9), distinct_count: None, null_count: 0, min_max_deprecated: false, min_max_backwards_compatible: false}) + // "smallint_col": Int32({min: Some(0), max: Some(9), distinct_count: None, null_count: 0, min_max_deprecated: false, min_max_backwards_compatible: false}) + // "int_col": Int32({min: Some(0), max: Some(9), distinct_count: None, null_count: 0, min_max_deprecated: false, min_max_backwards_compatible: false}) + // "bigint_col": Int64({min: Some(0), max: Some(90), distinct_count: None, null_count: 0, min_max_deprecated: false, min_max_backwards_compatible: false}) + // "float_col": Float({min: Some(0.0), max: Some(9.9), distinct_count: None, null_count: 0, min_max_deprecated: false, min_max_backwards_compatible: false}) + // "double_col": Double({min: Some(0.0), max: Some(90.89999999999999), distinct_count: None, null_count: 0, min_max_deprecated: false, min_max_backwards_compatible: false}) + // "date_string_col": ByteArray({min: Some(ByteArray { data: "01/01/09" }), max: Some(ByteArray { data: "12/31/10" }), distinct_count: None, null_count: 0, min_max_deprecated: false, min_max_backwards_compatible: false}) + // "string_col": ByteArray({min: Some(ByteArray { data: "0" }), max: Some(ByteArray { data: "9" }), distinct_count: None, null_count: 0, min_max_deprecated: false, min_max_backwards_compatible: false}) + // "timestamp_col": Int96({min: None, max: None, distinct_count: None, null_count: 0, min_max_deprecated: true, min_max_backwards_compatible: true}) + // "year": Int32({min: Some(2009), max: Some(2010), distinct_count: None, null_count: 0, min_max_deprecated: false, min_max_backwards_compatible: false}) + // "month": Int32({min: Some(1), max: Some(12), distinct_count: None, null_count: 0, min_max_deprecated: false, min_max_backwards_compatible: false}) + TestFile::new("alltypes_tiny_pages.parquet") + .with_column(ExpectedColumn { + name: "id", + expected_min: i32_array([Some(0)]), + expected_max: i32_array([Some(7299)]), + }) + .with_column(ExpectedColumn { + name: "bool_col", + expected_min: bool_array([Some(false)]), + expected_max: bool_array([Some(true)]), + }) + .with_column(ExpectedColumn { + name: "tinyint_col", + expected_min: i32_array([Some(0)]), + expected_max: i32_array([Some(9)]), + }) + .with_column(ExpectedColumn { + name: "smallint_col", + expected_min: i32_array([Some(0)]), + expected_max: i32_array([Some(9)]), + }) + .with_column(ExpectedColumn { + name: "int_col", + expected_min: i32_array([Some(0)]), + expected_max: i32_array([Some(9)]), + }) + .with_column(ExpectedColumn { + name: "bigint_col", + expected_min: i64_array([Some(0)]), + expected_max: i64_array([Some(90)]), + }) + .with_column(ExpectedColumn { + name: "float_col", + expected_min: f32_array([Some(0.0)]), + expected_max: f32_array([Some(9.9)]), + }) + .with_column(ExpectedColumn { + name: "double_col", + expected_min: f64_array([Some(0.0)]), + expected_max: f64_array([Some(90.89999999999999)]), + }) + .with_column(ExpectedColumn { + name: "date_string_col", + expected_min: utf8_array([Some("01/01/09")]), + expected_max: utf8_array([Some("12/31/10")]), + }) + .with_column(ExpectedColumn { + name: "string_col", + expected_min: utf8_array([Some("0")]), + expected_max: utf8_array([Some("9")]), + }) + // File has no min/max for timestamp_col + .with_column(ExpectedColumn { + name: "timestamp_col", + expected_min: timestamp_array([None]), + expected_max: timestamp_array([None]), + }) + .with_column(ExpectedColumn { + name: "year", + expected_min: i32_array([Some(2009)]), + expected_max: i32_array([Some(2010)]), + }) + .with_column(ExpectedColumn { + name: "month", + expected_min: i32_array([Some(1)]), + expected_max: i32_array([Some(12)]), + }) + .run(); + } + + #[test] + fn fixed_length_decimal_legacy() { + // /parquet-testing/data/fixed_length_decimal_legacy.parquet + // row_groups: 1 + // "value": FixedLenByteArray({min: Some(FixedLenByteArray(ByteArray { data: Some(ByteBufferPtr { data: b"\0\0\0\0\0\xc8" }) })), max: Some(FixedLenByteArray(ByteArray { data: "\0\0\0\0\t`" })), distinct_count: None, null_count: 0, min_max_deprecated: true, min_max_backwards_compatible: true}) + + TestFile::new("fixed_length_decimal_legacy.parquet") + .with_column(ExpectedColumn { + name: "value", + expected_min: Arc::new( + Decimal128Array::from(vec![Some(200)]) + .with_precision_and_scale(13, 2) + .unwrap(), + ), + expected_max: Arc::new( + Decimal128Array::from(vec![Some(2400)]) + .with_precision_and_scale(13, 2) + .unwrap(), + ), + }) + .run(); + } + + const ROWS_PER_ROW_GROUP: usize = 3; + + /// Writes the input batch into a parquet file, with every every three rows as + /// their own row group, and compares the min/maxes to the expected values + struct Test { + input: ArrayRef, + expected_min: ArrayRef, + expected_max: ArrayRef, + } + + impl Test { + fn run(self) { + let Self { + input, + expected_min, + expected_max, + } = self; + + let input_batch = RecordBatch::try_from_iter([("c1", input)]).unwrap(); + + let schema = input_batch.schema(); + + let metadata = parquet_metadata(schema.clone(), input_batch); + let parquet_schema = metadata.file_metadata().schema_descr(); + + let row_groups = metadata.row_groups(); + + for field in schema.fields() { + if field.data_type().is_nested() { + let lookup = parquet_column(parquet_schema, &schema, field.name()); + assert_eq!(lookup, None); + continue; + } + + let (idx, f) = + parquet_column(parquet_schema, &schema, field.name()).unwrap(); + assert_eq!(f, field); + + let iter = row_groups.iter().map(|x| x.column(idx).statistics()); + let min = min_statistics(f.data_type(), iter.clone()).unwrap(); + assert_eq!( + &min, + &expected_min, + "Min. Statistics\n\n{}\n\n", + DisplayStats(row_groups) + ); + + let max = max_statistics(f.data_type(), iter).unwrap(); + assert_eq!( + &max, + &expected_max, + "Max. Statistics\n\n{}\n\n", + DisplayStats(row_groups) + ); + } + } + } + + /// Write the specified batches out as parquet and return the metadata + fn parquet_metadata(schema: SchemaRef, batch: RecordBatch) -> Arc { + let props = WriterProperties::builder() + .set_statistics_enabled(EnabledStatistics::Chunk) + .set_max_row_group_size(ROWS_PER_ROW_GROUP) + .build(); + + let mut buffer = Vec::new(); + let mut writer = ArrowWriter::try_new(&mut buffer, schema, Some(props)).unwrap(); + writer.write(&batch).unwrap(); + writer.close().unwrap(); + + let reader = ArrowReaderBuilder::try_new(Bytes::from(buffer)).unwrap(); + reader.metadata().clone() + } + + /// Formats the statistics nicely for display + struct DisplayStats<'a>(&'a [RowGroupMetaData]); + impl<'a> std::fmt::Display for DisplayStats<'a> { + fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + let row_groups = self.0; + writeln!(f, " row_groups: {}", row_groups.len())?; + for rg in row_groups { + for col in rg.columns() { + if let Some(statistics) = col.statistics() { + writeln!(f, " {}: {:?}", col.column_path(), statistics)?; + } + } + } + Ok(()) + } + } + + struct ExpectedColumn { + name: &'static str, + expected_min: ArrayRef, + expected_max: ArrayRef, + } + + /// Reads statistics out of the specified, and compares them to the expected values + struct TestFile { + file_name: &'static str, + expected_columns: Vec, + } + + impl TestFile { + fn new(file_name: &'static str) -> Self { + Self { + file_name, + expected_columns: Vec::new(), + } + } + + fn with_column(mut self, column: ExpectedColumn) -> Self { + self.expected_columns.push(column); + self + } + + /// Reads the specified parquet file and validates that the exepcted min/max + /// values for the specified columns are as expected. + fn run(self) { + let path = PathBuf::from(parquet_test_data()).join(self.file_name); + let file = std::fs::File::open(path).unwrap(); + let reader = ArrowReaderBuilder::try_new(file).unwrap(); + let arrow_schema = reader.schema(); + let metadata = reader.metadata(); + let row_groups = metadata.row_groups(); + let parquet_schema = metadata.file_metadata().schema_descr(); + + for expected_column in self.expected_columns { + let ExpectedColumn { + name, + expected_min, + expected_max, + } = expected_column; + + let (idx, field) = + parquet_column(parquet_schema, arrow_schema, name).unwrap(); + + let iter = row_groups.iter().map(|x| x.column(idx).statistics()); + let actual_min = min_statistics(field.data_type(), iter.clone()).unwrap(); + assert_eq!(&expected_min, &actual_min, "column {name}"); + + let actual_max = max_statistics(field.data_type(), iter).unwrap(); + assert_eq!(&expected_max, &actual_max, "column {name}"); + } + } + } + + fn bool_array(input: impl IntoIterator>) -> ArrayRef { + let array: BooleanArray = input.into_iter().collect(); + Arc::new(array) + } + + fn i32_array(input: impl IntoIterator>) -> ArrayRef { + let array: Int32Array = input.into_iter().collect(); + Arc::new(array) + } + + fn i64_array(input: impl IntoIterator>) -> ArrayRef { + let array: Int64Array = input.into_iter().collect(); + Arc::new(array) + } + + fn f32_array(input: impl IntoIterator>) -> ArrayRef { + let array: Float32Array = input.into_iter().collect(); + Arc::new(array) + } + + fn f64_array(input: impl IntoIterator>) -> ArrayRef { + let array: Float64Array = input.into_iter().collect(); + Arc::new(array) + } + + fn timestamp_array(input: impl IntoIterator>) -> ArrayRef { + let array: TimestampNanosecondArray = input.into_iter().collect(); + Arc::new(array) + } + + fn utf8_array<'a>(input: impl IntoIterator>) -> ArrayRef { + let array: StringArray = input + .into_iter() + .map(|s| s.map(|s| s.to_string())) + .collect(); + Arc::new(array) + } + + // returns a struct array with columns "bool_col" and "int_col" with the specified values + fn struct_array(input: Vec<(Option, Option)>) -> ArrayRef { + let boolean: BooleanArray = input.iter().map(|(b, _i)| b).collect(); + let int: Int32Array = input.iter().map(|(_b, i)| i).collect(); + + let nullable = true; + let struct_array = StructArray::from(vec![ + ( + Arc::new(Field::new("bool_col", DataType::Boolean, nullable)), + Arc::new(boolean) as ArrayRef, + ), + ( + Arc::new(Field::new("int_col", DataType::Int32, nullable)), + Arc::new(int) as ArrayRef, + ), + ]); + Arc::new(struct_array) + } +} From c079a927e072b031a57df44c3ff0736c9679061c Mon Sep 17 00:00:00 2001 From: Syleechan <38198463+Syleechan@users.noreply.github.com> Date: Fri, 1 Dec 2023 06:03:57 +0800 Subject: [PATCH 334/572] feat:implement sql style 'find_in_set' string function (#8328) * feat:implement sql style 'find_in_set' string function * format code * modify test case --- datafusion/expr/src/built_in_function.rs | 11 +++++ datafusion/expr/src/expr_fn.rs | 2 + datafusion/physical-expr/src/functions.rs | 21 +++++++++ .../physical-expr/src/unicode_expressions.rs | 39 +++++++++++++++++ datafusion/proto/proto/datafusion.proto | 1 + datafusion/proto/src/generated/pbjson.rs | 3 ++ datafusion/proto/src/generated/prost.rs | 3 ++ .../proto/src/logical_plan/from_proto.rs | 9 +++- datafusion/proto/src/logical_plan/to_proto.rs | 1 + .../sqllogictest/test_files/functions.slt | 43 +++++++++++++++++++ .../source/user-guide/sql/scalar_functions.md | 15 +++++++ 11 files changed, 146 insertions(+), 2 deletions(-) diff --git a/datafusion/expr/src/built_in_function.rs b/datafusion/expr/src/built_in_function.rs index 53f9e850d303..2f67783201f5 100644 --- a/datafusion/expr/src/built_in_function.rs +++ b/datafusion/expr/src/built_in_function.rs @@ -304,6 +304,8 @@ pub enum BuiltinScalarFunction { Levenshtein, /// substr_index SubstrIndex, + /// find_in_set + FindInSet, } /// Maps the sql function name to `BuiltinScalarFunction` @@ -472,6 +474,7 @@ impl BuiltinScalarFunction { BuiltinScalarFunction::OverLay => Volatility::Immutable, BuiltinScalarFunction::Levenshtein => Volatility::Immutable, BuiltinScalarFunction::SubstrIndex => Volatility::Immutable, + BuiltinScalarFunction::FindInSet => Volatility::Immutable, // Stable builtin functions BuiltinScalarFunction::Now => Volatility::Stable, @@ -778,6 +781,9 @@ impl BuiltinScalarFunction { BuiltinScalarFunction::SubstrIndex => { utf8_to_str_type(&input_expr_types[0], "substr_index") } + BuiltinScalarFunction::FindInSet => { + utf8_to_int_type(&input_expr_types[0], "find_in_set") + } BuiltinScalarFunction::ToTimestamp | BuiltinScalarFunction::ToTimestampNanos => Ok(Timestamp(Nanosecond, None)), BuiltinScalarFunction::ToTimestampMillis => Ok(Timestamp(Millisecond, None)), @@ -1244,6 +1250,10 @@ impl BuiltinScalarFunction { ], self.volatility(), ), + BuiltinScalarFunction::FindInSet => Signature::one_of( + vec![Exact(vec![Utf8, Utf8]), Exact(vec![LargeUtf8, LargeUtf8])], + self.volatility(), + ), BuiltinScalarFunction::Replace | BuiltinScalarFunction::Translate => { Signature::one_of(vec![Exact(vec![Utf8, Utf8, Utf8])], self.volatility()) @@ -1499,6 +1509,7 @@ impl BuiltinScalarFunction { BuiltinScalarFunction::Uuid => &["uuid"], BuiltinScalarFunction::Levenshtein => &["levenshtein"], BuiltinScalarFunction::SubstrIndex => &["substr_index", "substring_index"], + BuiltinScalarFunction::FindInSet => &["find_in_set"], // regex functions BuiltinScalarFunction::RegexpMatch => &["regexp_match"], diff --git a/datafusion/expr/src/expr_fn.rs b/datafusion/expr/src/expr_fn.rs index d2c5e5cddbf3..1f4ab7bb4ad3 100644 --- a/datafusion/expr/src/expr_fn.rs +++ b/datafusion/expr/src/expr_fn.rs @@ -917,6 +917,7 @@ scalar_expr!( scalar_expr!(ArrowTypeof, arrow_typeof, val, "data type"); scalar_expr!(Levenshtein, levenshtein, string1 string2, "Returns the Levenshtein distance between the two given strings"); scalar_expr!(SubstrIndex, substr_index, string delimiter count, "Returns the substring from str before count occurrences of the delimiter"); +scalar_expr!(FindInSet, find_in_set, str strlist, "Returns a value in the range of 1 to N if the string str is in the string list strlist consisting of N substrings"); scalar_expr!( Struct, @@ -1207,6 +1208,7 @@ mod test { test_nary_scalar_expr!(OverLay, overlay, string, characters, position); test_scalar_expr!(Levenshtein, levenshtein, string1, string2); test_scalar_expr!(SubstrIndex, substr_index, string, delimiter, count); + test_scalar_expr!(FindInSet, find_in_set, string, stringlist); } #[test] diff --git a/datafusion/physical-expr/src/functions.rs b/datafusion/physical-expr/src/functions.rs index 40b21347edf5..72c7f492166d 100644 --- a/datafusion/physical-expr/src/functions.rs +++ b/datafusion/physical-expr/src/functions.rs @@ -885,6 +885,27 @@ pub fn create_physical_fun( ))), }) } + BuiltinScalarFunction::FindInSet => Arc::new(|args| match args[0].data_type() { + DataType::Utf8 => { + let func = invoke_if_unicode_expressions_feature_flag!( + find_in_set, + Int32Type, + "find_in_set" + ); + make_scalar_function(func)(args) + } + DataType::LargeUtf8 => { + let func = invoke_if_unicode_expressions_feature_flag!( + find_in_set, + Int64Type, + "find_in_set" + ); + make_scalar_function(func)(args) + } + other => Err(DataFusionError::Internal(format!( + "Unsupported data type {other:?} for function find_in_set", + ))), + }), }) } diff --git a/datafusion/physical-expr/src/unicode_expressions.rs b/datafusion/physical-expr/src/unicode_expressions.rs index f27b3c157741..240efe4223c3 100644 --- a/datafusion/physical-expr/src/unicode_expressions.rs +++ b/datafusion/physical-expr/src/unicode_expressions.rs @@ -520,3 +520,42 @@ pub fn substr_index(args: &[ArrayRef]) -> Result { Ok(Arc::new(result) as ArrayRef) } + +///Returns a value in the range of 1 to N if the string str is in the string list strlist consisting of N substrings +///A string list is a string composed of substrings separated by , characters. +pub fn find_in_set(args: &[ArrayRef]) -> Result +where + T::Native: OffsetSizeTrait, +{ + if args.len() != 2 { + return internal_err!( + "find_in_set was called with {} arguments. It requires 2.", + args.len() + ); + } + + let str_array: &GenericStringArray = + as_generic_string_array::(&args[0])?; + let str_list_array: &GenericStringArray = + as_generic_string_array::(&args[1])?; + + let result = str_array + .iter() + .zip(str_list_array.iter()) + .map(|(string, str_list)| match (string, str_list) { + (Some(string), Some(str_list)) => { + let mut res = 0; + let str_set: Vec<&str> = str_list.split(',').collect(); + for (idx, str) in str_set.iter().enumerate() { + if str == &string { + res = idx + 1; + break; + } + } + T::Native::from_usize(res) + } + _ => None, + }) + .collect::>(); + Ok(Arc::new(result) as ArrayRef) +} diff --git a/datafusion/proto/proto/datafusion.proto b/datafusion/proto/proto/datafusion.proto index 5c33b10f1395..8c2fd5369e33 100644 --- a/datafusion/proto/proto/datafusion.proto +++ b/datafusion/proto/proto/datafusion.proto @@ -642,6 +642,7 @@ enum ScalarFunction { ArrayPopFront = 124; Levenshtein = 125; SubstrIndex = 126; + FindInSet = 127; } message ScalarFunctionNode { diff --git a/datafusion/proto/src/generated/pbjson.rs b/datafusion/proto/src/generated/pbjson.rs index 598719dc8ac6..b8c5f6a4aae8 100644 --- a/datafusion/proto/src/generated/pbjson.rs +++ b/datafusion/proto/src/generated/pbjson.rs @@ -20864,6 +20864,7 @@ impl serde::Serialize for ScalarFunction { Self::ArrayPopFront => "ArrayPopFront", Self::Levenshtein => "Levenshtein", Self::SubstrIndex => "SubstrIndex", + Self::FindInSet => "FindInSet", }; serializer.serialize_str(variant) } @@ -21002,6 +21003,7 @@ impl<'de> serde::Deserialize<'de> for ScalarFunction { "ArrayPopFront", "Levenshtein", "SubstrIndex", + "FindInSet", ]; struct GeneratedVisitor; @@ -21169,6 +21171,7 @@ impl<'de> serde::Deserialize<'de> for ScalarFunction { "ArrayPopFront" => Ok(ScalarFunction::ArrayPopFront), "Levenshtein" => Ok(ScalarFunction::Levenshtein), "SubstrIndex" => Ok(ScalarFunction::SubstrIndex), + "FindInSet" => Ok(ScalarFunction::FindInSet), _ => Err(serde::de::Error::unknown_variant(value, FIELDS)), } } diff --git a/datafusion/proto/src/generated/prost.rs b/datafusion/proto/src/generated/prost.rs index e79a17fc5c9c..c31bc4ab5948 100644 --- a/datafusion/proto/src/generated/prost.rs +++ b/datafusion/proto/src/generated/prost.rs @@ -2595,6 +2595,7 @@ pub enum ScalarFunction { ArrayPopFront = 124, Levenshtein = 125, SubstrIndex = 126, + FindInSet = 127, } impl ScalarFunction { /// String value of the enum field names used in the ProtoBuf definition. @@ -2730,6 +2731,7 @@ impl ScalarFunction { ScalarFunction::ArrayPopFront => "ArrayPopFront", ScalarFunction::Levenshtein => "Levenshtein", ScalarFunction::SubstrIndex => "SubstrIndex", + ScalarFunction::FindInSet => "FindInSet", } } /// Creates an enum from field names used in the ProtoBuf definition. @@ -2862,6 +2864,7 @@ impl ScalarFunction { "ArrayPopFront" => Some(Self::ArrayPopFront), "Levenshtein" => Some(Self::Levenshtein), "SubstrIndex" => Some(Self::SubstrIndex), + "FindInSet" => Some(Self::FindInSet), _ => None, } } diff --git a/datafusion/proto/src/logical_plan/from_proto.rs b/datafusion/proto/src/logical_plan/from_proto.rs index b2455d5a0d13..d596998c1de3 100644 --- a/datafusion/proto/src/logical_plan/from_proto.rs +++ b/datafusion/proto/src/logical_plan/from_proto.rs @@ -49,8 +49,8 @@ use datafusion_expr::{ chr, coalesce, concat_expr, concat_ws_expr, cos, cosh, cot, current_date, current_time, date_bin, date_part, date_trunc, decode, degrees, digest, encode, exp, expr::{self, InList, Sort, WindowFunction}, - factorial, flatten, floor, from_unixtime, gcd, gen_range, isnan, iszero, lcm, left, - levenshtein, ln, log, log10, log2, + factorial, find_in_set, flatten, floor, from_unixtime, gcd, gen_range, isnan, iszero, + lcm, left, levenshtein, ln, log, log10, log2, logical_plan::{PlanType, StringifiedPlan}, lower, lpad, ltrim, md5, nanvl, now, nullif, octet_length, overlay, pi, power, radians, random, regexp_match, regexp_replace, repeat, replace, reverse, right, @@ -552,6 +552,7 @@ impl From<&protobuf::ScalarFunction> for BuiltinScalarFunction { ScalarFunction::OverLay => Self::OverLay, ScalarFunction::Levenshtein => Self::Levenshtein, ScalarFunction::SubstrIndex => Self::SubstrIndex, + ScalarFunction::FindInSet => Self::FindInSet, } } } @@ -1722,6 +1723,10 @@ pub fn parse_expr( parse_expr(&args[1], registry)?, parse_expr(&args[2], registry)?, )), + ScalarFunction::FindInSet => Ok(find_in_set( + parse_expr(&args[0], registry)?, + parse_expr(&args[1], registry)?, + )), ScalarFunction::StructFun => { Ok(struct_fun(parse_expr(&args[0], registry)?)) } diff --git a/datafusion/proto/src/logical_plan/to_proto.rs b/datafusion/proto/src/logical_plan/to_proto.rs index 9be4a532bb5b..54be6460c392 100644 --- a/datafusion/proto/src/logical_plan/to_proto.rs +++ b/datafusion/proto/src/logical_plan/to_proto.rs @@ -1584,6 +1584,7 @@ impl TryFrom<&BuiltinScalarFunction> for protobuf::ScalarFunction { BuiltinScalarFunction::OverLay => Self::OverLay, BuiltinScalarFunction::Levenshtein => Self::Levenshtein, BuiltinScalarFunction::SubstrIndex => Self::SubstrIndex, + BuiltinScalarFunction::FindInSet => Self::FindInSet, }; Ok(scalar_function) diff --git a/datafusion/sqllogictest/test_files/functions.slt b/datafusion/sqllogictest/test_files/functions.slt index 91072a49cd46..4f55ea316bb9 100644 --- a/datafusion/sqllogictest/test_files/functions.slt +++ b/datafusion/sqllogictest/test_files/functions.slt @@ -952,3 +952,46 @@ query ? SELECT substr_index(NULL, NULL, NULL) ---- NULL + +query I +SELECT find_in_set('b', 'a,b,c,d') +---- +2 + + +query I +SELECT find_in_set('a', 'a,b,c,d,a') +---- +1 + +query I +SELECT find_in_set('', 'a,b,c,d,a') +---- +0 + +query I +SELECT find_in_set('a', '') +---- +0 + + +query I +SELECT find_in_set('', '') +---- +1 + +query ? +SELECT find_in_set(NULL, 'a,b,c,d') +---- +NULL + +query I +SELECT find_in_set('a', NULL) +---- +NULL + + +query ? +SELECT find_in_set(NULL, NULL) +---- +NULL diff --git a/docs/source/user-guide/sql/scalar_functions.md b/docs/source/user-guide/sql/scalar_functions.md index 74dceb221ad2..c0889d94dbac 100644 --- a/docs/source/user-guide/sql/scalar_functions.md +++ b/docs/source/user-guide/sql/scalar_functions.md @@ -638,6 +638,7 @@ nullif(expression1, expression2) - [overlay](#overlay) - [levenshtein](#levenshtein) - [substr_index](#substr_index) +- [find_in_set](#find_in_set) ### `ascii` @@ -1170,6 +1171,20 @@ substr_index(str, delim, count) - **delim**: the string to find in str to split str. - **count**: The number of times to search for the delimiter. Can be both a positive or negative number. +### `find_in_set` + +Returns a value in the range of 1 to N if the string str is in the string list strlist consisting of N substrings. +For example, `find_in_set('b', 'a,b,c,d') = 2` + +``` +find_in_set(str, strlist) +``` + +#### Arguments + +- **str**: String expression to find in strlist. +- **strlist**: A string list is a string composed of substrings separated by , characters. + ## Binary String Functions - [decode](#decode) From a588123759786c4037e0b077a13cb62b4b2afa42 Mon Sep 17 00:00:00 2001 From: Jay Zhan Date: Fri, 1 Dec 2023 06:04:09 +0800 Subject: [PATCH 335/572] largeutf to temporal (#8357) Signed-off-by: jayzhan211 --- datafusion/expr/src/type_coercion/binary.rs | 37 ++++++++++--------- datafusion/sqllogictest/test_files/scalar.slt | 17 +++++++++ 2 files changed, 36 insertions(+), 18 deletions(-) diff --git a/datafusion/expr/src/type_coercion/binary.rs b/datafusion/expr/src/type_coercion/binary.rs index 9ccddbfce068..1027e97d061a 100644 --- a/datafusion/expr/src/type_coercion/binary.rs +++ b/datafusion/expr/src/type_coercion/binary.rs @@ -331,26 +331,27 @@ fn string_temporal_coercion( rhs_type: &DataType, ) -> Option { use arrow::datatypes::DataType::*; - match (lhs_type, rhs_type) { - (Utf8, Date32) | (Date32, Utf8) => Some(Date32), - (Utf8, Date64) | (Date64, Utf8) => Some(Date64), - (Utf8, Time32(unit)) | (Time32(unit), Utf8) => { - match is_time_with_valid_unit(Time32(unit.clone())) { - false => None, - true => Some(Time32(unit.clone())), - } - } - (Utf8, Time64(unit)) | (Time64(unit), Utf8) => { - match is_time_with_valid_unit(Time64(unit.clone())) { - false => None, - true => Some(Time64(unit.clone())), - } - } - (Timestamp(_, tz), Utf8) | (Utf8, Timestamp(_, tz)) => { - Some(Timestamp(TimeUnit::Nanosecond, tz.clone())) + + fn match_rule(l: &DataType, r: &DataType) -> Option { + match (l, r) { + // Coerce Utf8/LargeUtf8 to Date32/Date64/Time32/Time64/Timestamp + (Utf8, temporal) | (LargeUtf8, temporal) => match temporal { + Date32 | Date64 => Some(temporal.clone()), + Time32(_) | Time64(_) => { + if is_time_with_valid_unit(temporal.to_owned()) { + Some(temporal.to_owned()) + } else { + None + } + } + Timestamp(_, tz) => Some(Timestamp(TimeUnit::Nanosecond, tz.clone())), + _ => None, + }, + _ => None, } - _ => None, } + + match_rule(lhs_type, rhs_type).or_else(|| match_rule(rhs_type, lhs_type)) } /// Coerce `lhs_type` and `rhs_type` to a common type for the purposes of a comparison operation diff --git a/datafusion/sqllogictest/test_files/scalar.slt b/datafusion/sqllogictest/test_files/scalar.slt index ecb7fe13fcf4..b3597c664fbb 100644 --- a/datafusion/sqllogictest/test_files/scalar.slt +++ b/datafusion/sqllogictest/test_files/scalar.slt @@ -1926,3 +1926,20 @@ A true B false C false D false + +# test string_temporal_coercion +query BBBBBBBBBB +select + arrow_cast(to_timestamp('2020-01-01 01:01:11.1234567890Z'), 'Timestamp(Second, None)') == '2020-01-01T01:01:11', + arrow_cast(to_timestamp('2020-01-02 01:01:11.1234567890Z'), 'Timestamp(Second, None)') == arrow_cast('2020-01-02T01:01:11', 'LargeUtf8'), + arrow_cast(to_timestamp('2020-01-03 01:01:11.1234567890Z'), 'Time32(Second)') == '01:01:11', + arrow_cast(to_timestamp('2020-01-04 01:01:11.1234567890Z'), 'Time32(Second)') == arrow_cast('01:01:11', 'LargeUtf8'), + arrow_cast(to_timestamp('2020-01-05 01:01:11.1234567890Z'), 'Time64(Microsecond)') == '01:01:11.123456', + arrow_cast(to_timestamp('2020-01-06 01:01:11.1234567890Z'), 'Time64(Microsecond)') == arrow_cast('01:01:11.123456', 'LargeUtf8'), + arrow_cast('2020-01-07', 'Date32') == '2020-01-07', + arrow_cast('2020-01-08', 'Date64') == '2020-01-08', + arrow_cast('2020-01-09', 'Date32') == arrow_cast('2020-01-09', 'LargeUtf8'), + arrow_cast('2020-01-10', 'Date64') == arrow_cast('2020-01-10', 'LargeUtf8') +; +---- +true true true true true true true true true true From a49740f675b2279e60b1898114f2e4d81ed43441 Mon Sep 17 00:00:00 2001 From: Alex Huang Date: Thu, 30 Nov 2023 23:04:56 +0100 Subject: [PATCH 336/572] Refactor aggregate function handling (#8358) * Refactor aggregate function handling * fix ci * update comment * fix ci * simplify the code * fix fmt * fix ci * fix clippy --- .../core/src/datasource/listing/helpers.rs | 3 +- datafusion/core/src/physical_planner.rs | 128 ++++----- datafusion/expr/src/aggregate_function.rs | 2 +- datafusion/expr/src/expr.rs | 112 ++++---- datafusion/expr/src/expr_schema.rs | 26 +- datafusion/expr/src/tree_node/expr.rs | 66 ++--- datafusion/expr/src/udaf.rs | 11 +- datafusion/expr/src/utils.rs | 10 +- .../src/analyzer/count_wildcard_rule.rs | 15 +- .../optimizer/src/analyzer/type_coercion.rs | 68 ++--- .../optimizer/src/common_subexpr_eliminate.rs | 11 +- datafusion/optimizer/src/decorrelate.rs | 27 +- datafusion/optimizer/src/push_down_filter.rs | 1 - .../simplify_expressions/expr_simplifier.rs | 1 - .../src/single_distinct_to_groupby.rs | 5 +- .../proto/src/logical_plan/from_proto.rs | 3 +- datafusion/proto/src/logical_plan/to_proto.rs | 246 +++++++++--------- .../tests/cases/roundtrip_logical_plan.rs | 3 +- datafusion/sql/src/expr/function.rs | 4 +- datafusion/sql/src/expr/mod.rs | 3 +- datafusion/sql/src/select.rs | 9 +- .../substrait/src/logical_plan/consumer.rs | 19 +- .../substrait/src/logical_plan/producer.rs | 126 ++++----- 23 files changed, 462 insertions(+), 437 deletions(-) diff --git a/datafusion/core/src/datasource/listing/helpers.rs b/datafusion/core/src/datasource/listing/helpers.rs index f9b02f4d0c10..0c39877cd11e 100644 --- a/datafusion/core/src/datasource/listing/helpers.rs +++ b/datafusion/core/src/datasource/listing/helpers.rs @@ -122,8 +122,7 @@ pub fn expr_applicable_for_cols(col_names: &[String], expr: &Expr) -> bool { // - AGGREGATE, WINDOW and SORT should not end up in filter conditions, except maybe in some edge cases // - Can `Wildcard` be considered as a `Literal`? // - ScalarVariable could be `applicable`, but that would require access to the context - Expr::AggregateUDF { .. } - | Expr::AggregateFunction { .. } + Expr::AggregateFunction { .. } | Expr::Sort { .. } | Expr::WindowFunction { .. } | Expr::Wildcard { .. } diff --git a/datafusion/core/src/physical_planner.rs b/datafusion/core/src/physical_planner.rs index ef364c22ee7d..9e64eb9c5108 100644 --- a/datafusion/core/src/physical_planner.rs +++ b/datafusion/core/src/physical_planner.rs @@ -82,8 +82,9 @@ use datafusion_common::{ }; use datafusion_expr::dml::{CopyOptions, CopyTo}; use datafusion_expr::expr::{ - self, AggregateFunction, AggregateUDF, Alias, Between, BinaryExpr, Cast, - GetFieldAccess, GetIndexedField, GroupingSet, InList, Like, TryCast, WindowFunction, + self, AggregateFunction, AggregateFunctionDefinition, Alias, Between, BinaryExpr, + Cast, GetFieldAccess, GetIndexedField, GroupingSet, InList, Like, TryCast, + WindowFunction, }; use datafusion_expr::expr_rewriter::{unalias, unnormalize_cols}; use datafusion_expr::logical_plan::builder::wrap_projection_for_join_if_necessary; @@ -229,30 +230,37 @@ fn create_physical_name(e: &Expr, is_first_expr: bool) -> Result { create_function_physical_name(&fun.to_string(), false, args) } Expr::AggregateFunction(AggregateFunction { - fun, + func_def, distinct, args, - .. - }) => create_function_physical_name(&fun.to_string(), *distinct, args), - Expr::AggregateUDF(AggregateUDF { - fun, - args, filter, order_by, - }) => { - // TODO: Add support for filter and order by in AggregateUDF - if filter.is_some() { - return exec_err!("aggregate expression with filter is not supported"); + }) => match func_def { + AggregateFunctionDefinition::BuiltIn(..) => { + create_function_physical_name(func_def.name(), *distinct, args) } - if order_by.is_some() { - return exec_err!("aggregate expression with order_by is not supported"); + AggregateFunctionDefinition::UDF(fun) => { + // TODO: Add support for filter and order by in AggregateUDF + if filter.is_some() { + return exec_err!( + "aggregate expression with filter is not supported" + ); + } + if order_by.is_some() { + return exec_err!( + "aggregate expression with order_by is not supported" + ); + } + let names = args + .iter() + .map(|e| create_physical_name(e, false)) + .collect::>>()?; + Ok(format!("{}({})", fun.name(), names.join(","))) } - let mut names = Vec::with_capacity(args.len()); - for e in args { - names.push(create_physical_name(e, false)?); + AggregateFunctionDefinition::Name(_) => { + internal_err!("Aggregate function `Expr` with name should be resolved.") } - Ok(format!("{}({})", fun.name(), names.join(","))) - } + }, Expr::GroupingSet(grouping_set) => match grouping_set { GroupingSet::Rollup(exprs) => Ok(format!( "ROLLUP ({})", @@ -1705,7 +1713,7 @@ pub fn create_aggregate_expr_with_name_and_maybe_filter( ) -> Result { match e { Expr::AggregateFunction(AggregateFunction { - fun, + func_def, distinct, args, filter, @@ -1746,63 +1754,35 @@ pub fn create_aggregate_expr_with_name_and_maybe_filter( ), None => None, }; - let ordering_reqs = order_by.clone().unwrap_or(vec![]); - let agg_expr = aggregates::create_aggregate_expr( - fun, - *distinct, - &args, - &ordering_reqs, - physical_input_schema, - name, - )?; - Ok((agg_expr, filter, order_by)) - } - Expr::AggregateUDF(AggregateUDF { - fun, - args, - filter, - order_by, - }) => { - let args = args - .iter() - .map(|e| { - create_physical_expr( - e, - logical_input_schema, + let (agg_expr, filter, order_by) = match func_def { + AggregateFunctionDefinition::BuiltIn(fun) => { + let ordering_reqs = order_by.clone().unwrap_or(vec![]); + let agg_expr = aggregates::create_aggregate_expr( + fun, + *distinct, + &args, + &ordering_reqs, physical_input_schema, - execution_props, + name, + )?; + (agg_expr, filter, order_by) + } + AggregateFunctionDefinition::UDF(fun) => { + let agg_expr = udaf::create_aggregate_expr( + fun, + &args, + physical_input_schema, + name, + ); + (agg_expr?, filter, order_by) + } + AggregateFunctionDefinition::Name(_) => { + return internal_err!( + "Aggregate function name should have been resolved" ) - }) - .collect::>>()?; - - let filter = match filter { - Some(e) => Some(create_physical_expr( - e, - logical_input_schema, - physical_input_schema, - execution_props, - )?), - None => None, - }; - let order_by = match order_by { - Some(e) => Some( - e.iter() - .map(|expr| { - create_physical_sort_expr( - expr, - logical_input_schema, - physical_input_schema, - execution_props, - ) - }) - .collect::>>()?, - ), - None => None, + } }; - - let agg_expr = - udaf::create_aggregate_expr(fun, &args, physical_input_schema, name); - Ok((agg_expr?, filter, order_by)) + Ok((agg_expr, filter, order_by)) } other => internal_err!("Invalid aggregate expression '{other:?}'"), } diff --git a/datafusion/expr/src/aggregate_function.rs b/datafusion/expr/src/aggregate_function.rs index 4611c7fb10d7..cea72c3cb5e6 100644 --- a/datafusion/expr/src/aggregate_function.rs +++ b/datafusion/expr/src/aggregate_function.rs @@ -105,7 +105,7 @@ pub enum AggregateFunction { } impl AggregateFunction { - fn name(&self) -> &str { + pub fn name(&self) -> &str { use AggregateFunction::*; match self { Count => "COUNT", diff --git a/datafusion/expr/src/expr.rs b/datafusion/expr/src/expr.rs index b46d204faafb..256f5b210ec2 100644 --- a/datafusion/expr/src/expr.rs +++ b/datafusion/expr/src/expr.rs @@ -154,8 +154,6 @@ pub enum Expr { AggregateFunction(AggregateFunction), /// Represents the call of a window function with arguments. WindowFunction(WindowFunction), - /// aggregate function - AggregateUDF(AggregateUDF), /// Returns whether the list contains the expr value. InList(InList), /// EXISTS subquery @@ -484,11 +482,33 @@ impl Sort { } } +#[derive(Debug, Clone, PartialEq, Eq, Hash)] +/// Defines which implementation of an aggregate function DataFusion should call. +pub enum AggregateFunctionDefinition { + BuiltIn(aggregate_function::AggregateFunction), + /// Resolved to a user defined aggregate function + UDF(Arc), + /// A aggregation function constructed with name. This variant can not be executed directly + /// and instead must be resolved to one of the other variants prior to physical planning. + Name(Arc), +} + +impl AggregateFunctionDefinition { + /// Function's name for display + pub fn name(&self) -> &str { + match self { + AggregateFunctionDefinition::BuiltIn(fun) => fun.name(), + AggregateFunctionDefinition::UDF(udf) => udf.name(), + AggregateFunctionDefinition::Name(func_name) => func_name.as_ref(), + } + } +} + /// Aggregate function #[derive(Clone, PartialEq, Eq, Hash, Debug)] pub struct AggregateFunction { /// Name of the function - pub fun: aggregate_function::AggregateFunction, + pub func_def: AggregateFunctionDefinition, /// List of expressions to feed to the functions as arguments pub args: Vec, /// Whether this is a DISTINCT aggregation or not @@ -508,7 +528,24 @@ impl AggregateFunction { order_by: Option>, ) -> Self { Self { - fun, + func_def: AggregateFunctionDefinition::BuiltIn(fun), + args, + distinct, + filter, + order_by, + } + } + + /// Create a new AggregateFunction expression with a user-defined function (UDF) + pub fn new_udf( + udf: Arc, + args: Vec, + distinct: bool, + filter: Option>, + order_by: Option>, + ) -> Self { + Self { + func_def: AggregateFunctionDefinition::UDF(udf), args, distinct, filter, @@ -736,7 +773,6 @@ impl Expr { pub fn variant_name(&self) -> &str { match self { Expr::AggregateFunction { .. } => "AggregateFunction", - Expr::AggregateUDF { .. } => "AggregateUDF", Expr::Alias(..) => "Alias", Expr::Between { .. } => "Between", Expr::BinaryExpr { .. } => "BinaryExpr", @@ -1251,30 +1287,14 @@ impl fmt::Display for Expr { Ok(()) } Expr::AggregateFunction(AggregateFunction { - fun, + func_def, distinct, ref args, filter, order_by, .. }) => { - fmt_function(f, &fun.to_string(), *distinct, args, true)?; - if let Some(fe) = filter { - write!(f, " FILTER (WHERE {fe})")?; - } - if let Some(ob) = order_by { - write!(f, " ORDER BY [{}]", expr_vec_fmt!(ob))?; - } - Ok(()) - } - Expr::AggregateUDF(AggregateUDF { - fun, - ref args, - filter, - order_by, - .. - }) => { - fmt_function(f, fun.name(), false, args, true)?; + fmt_function(f, func_def.name(), *distinct, args, true)?; if let Some(fe) = filter { write!(f, " FILTER (WHERE {fe})")?; } @@ -1579,39 +1599,39 @@ fn create_name(e: &Expr) -> Result { Ok(parts.join(" ")) } Expr::AggregateFunction(AggregateFunction { - fun, + func_def, distinct, args, filter, order_by, }) => { - let mut name = create_function_name(&fun.to_string(), *distinct, args)?; - if let Some(fe) = filter { - name = format!("{name} FILTER (WHERE {fe})"); - }; - if let Some(order_by) = order_by { - name = format!("{name} ORDER BY [{}]", expr_vec_fmt!(order_by)); + let name = match func_def { + AggregateFunctionDefinition::BuiltIn(..) + | AggregateFunctionDefinition::Name(..) => { + create_function_name(func_def.name(), *distinct, args)? + } + AggregateFunctionDefinition::UDF(..) => { + let names: Vec = + args.iter().map(create_name).collect::>()?; + names.join(",") + } }; - Ok(name) - } - Expr::AggregateUDF(AggregateUDF { - fun, - args, - filter, - order_by, - }) => { - let mut names = Vec::with_capacity(args.len()); - for e in args { - names.push(create_name(e)?); - } let mut info = String::new(); if let Some(fe) = filter { info += &format!(" FILTER (WHERE {fe})"); + }; + if let Some(order_by) = order_by { + info += &format!(" ORDER BY [{}]", expr_vec_fmt!(order_by)); + }; + match func_def { + AggregateFunctionDefinition::BuiltIn(..) + | AggregateFunctionDefinition::Name(..) => { + Ok(format!("{}{}", name, info)) + } + AggregateFunctionDefinition::UDF(fun) => { + Ok(format!("{}({}){}", fun.name(), name, info)) + } } - if let Some(ob) = order_by { - info += &format!(" ORDER BY ([{}])", expr_vec_fmt!(ob)); - } - Ok(format!("{}({}){}", fun.name(), names.join(","), info)) } Expr::GroupingSet(grouping_set) => match grouping_set { GroupingSet::Rollup(exprs) => { diff --git a/datafusion/expr/src/expr_schema.rs b/datafusion/expr/src/expr_schema.rs index d5d9c848b2e9..99b27e8912bc 100644 --- a/datafusion/expr/src/expr_schema.rs +++ b/datafusion/expr/src/expr_schema.rs @@ -17,8 +17,8 @@ use super::{Between, Expr, Like}; use crate::expr::{ - AggregateFunction, AggregateUDF, Alias, BinaryExpr, Cast, GetFieldAccess, - GetIndexedField, InList, InSubquery, Placeholder, ScalarFunction, + AggregateFunction, AggregateFunctionDefinition, Alias, BinaryExpr, Cast, + GetFieldAccess, GetIndexedField, InList, InSubquery, Placeholder, ScalarFunction, ScalarFunctionDefinition, Sort, TryCast, WindowFunction, }; use crate::field_util::GetFieldAccessSchema; @@ -123,19 +123,22 @@ impl ExprSchemable for Expr { .collect::>>()?; fun.return_type(&data_types) } - Expr::AggregateFunction(AggregateFunction { fun, args, .. }) => { + Expr::AggregateFunction(AggregateFunction { func_def, args, .. }) => { let data_types = args .iter() .map(|e| e.get_type(schema)) .collect::>>()?; - fun.return_type(&data_types) - } - Expr::AggregateUDF(AggregateUDF { fun, args, .. }) => { - let data_types = args - .iter() - .map(|e| e.get_type(schema)) - .collect::>>()?; - fun.return_type(&data_types) + match func_def { + AggregateFunctionDefinition::BuiltIn(fun) => { + fun.return_type(&data_types) + } + AggregateFunctionDefinition::UDF(fun) => { + Ok(fun.return_type(&data_types)?) + } + AggregateFunctionDefinition::Name(_) => { + internal_err!("Function `Expr` with name should be resolved.") + } + } } Expr::Not(_) | Expr::IsNull(_) @@ -252,7 +255,6 @@ impl ExprSchemable for Expr { | Expr::ScalarFunction(..) | Expr::WindowFunction { .. } | Expr::AggregateFunction { .. } - | Expr::AggregateUDF { .. } | Expr::Placeholder(_) => Ok(true), Expr::IsNull(_) | Expr::IsNotNull(_) diff --git a/datafusion/expr/src/tree_node/expr.rs b/datafusion/expr/src/tree_node/expr.rs index 474b5f7689b9..fcb0a4cd93f3 100644 --- a/datafusion/expr/src/tree_node/expr.rs +++ b/datafusion/expr/src/tree_node/expr.rs @@ -18,9 +18,9 @@ //! Tree node implementation for logical expr use crate::expr::{ - AggregateFunction, AggregateUDF, Alias, Between, BinaryExpr, Case, Cast, - GetIndexedField, GroupingSet, InList, InSubquery, Like, Placeholder, ScalarFunction, - ScalarFunctionDefinition, Sort, TryCast, WindowFunction, + AggregateFunction, AggregateFunctionDefinition, Alias, Between, BinaryExpr, Case, + Cast, GetIndexedField, GroupingSet, InList, InSubquery, Like, Placeholder, + ScalarFunction, ScalarFunctionDefinition, Sort, TryCast, WindowFunction, }; use crate::{Expr, GetFieldAccess}; @@ -108,7 +108,7 @@ impl TreeNode for Expr { expr_vec } Expr::AggregateFunction(AggregateFunction { args, filter, order_by, .. }) - | Expr::AggregateUDF(AggregateUDF { args, filter, order_by, .. }) => { + => { let mut expr_vec = args.clone(); if let Some(f) = filter { @@ -304,17 +304,40 @@ impl TreeNode for Expr { )), Expr::AggregateFunction(AggregateFunction { args, - fun, + func_def, distinct, filter, order_by, - }) => Expr::AggregateFunction(AggregateFunction::new( - fun, - transform_vec(args, &mut transform)?, - distinct, - transform_option_box(filter, &mut transform)?, - transform_option_vec(order_by, &mut transform)?, - )), + }) => match func_def { + AggregateFunctionDefinition::BuiltIn(fun) => { + Expr::AggregateFunction(AggregateFunction::new( + fun, + transform_vec(args, &mut transform)?, + distinct, + transform_option_box(filter, &mut transform)?, + transform_option_vec(order_by, &mut transform)?, + )) + } + AggregateFunctionDefinition::UDF(fun) => { + let order_by = if let Some(order_by) = order_by { + Some(transform_vec(order_by, &mut transform)?) + } else { + None + }; + Expr::AggregateFunction(AggregateFunction::new_udf( + fun, + transform_vec(args, &mut transform)?, + false, + transform_option_box(filter, &mut transform)?, + transform_option_vec(order_by, &mut transform)?, + )) + } + AggregateFunctionDefinition::Name(_) => { + return internal_err!( + "Function `Expr` with name should be resolved." + ); + } + }, Expr::GroupingSet(grouping_set) => match grouping_set { GroupingSet::Rollup(exprs) => Expr::GroupingSet(GroupingSet::Rollup( transform_vec(exprs, &mut transform)?, @@ -331,24 +354,7 @@ impl TreeNode for Expr { )) } }, - Expr::AggregateUDF(AggregateUDF { - args, - fun, - filter, - order_by, - }) => { - let order_by = if let Some(order_by) = order_by { - Some(transform_vec(order_by, &mut transform)?) - } else { - None - }; - Expr::AggregateUDF(AggregateUDF::new( - fun, - transform_vec(args, &mut transform)?, - transform_option_box(filter, &mut transform)?, - transform_option_vec(order_by, &mut transform)?, - )) - } + Expr::InList(InList { expr, list, diff --git a/datafusion/expr/src/udaf.rs b/datafusion/expr/src/udaf.rs index b06e97acc283..cfbca4ab1337 100644 --- a/datafusion/expr/src/udaf.rs +++ b/datafusion/expr/src/udaf.rs @@ -107,12 +107,13 @@ impl AggregateUDF { /// This utility allows using the UDAF without requiring access to /// the registry, such as with the DataFrame API. pub fn call(&self, args: Vec) -> Expr { - Expr::AggregateUDF(crate::expr::AggregateUDF { - fun: Arc::new(self.clone()), + Expr::AggregateFunction(crate::expr::AggregateFunction::new_udf( + Arc::new(self.clone()), args, - filter: None, - order_by: None, - }) + false, + None, + None, + )) } /// Returns this function's name diff --git a/datafusion/expr/src/utils.rs b/datafusion/expr/src/utils.rs index 7deb13c89be5..7d126a0f3373 100644 --- a/datafusion/expr/src/utils.rs +++ b/datafusion/expr/src/utils.rs @@ -291,7 +291,6 @@ pub fn expr_to_columns(expr: &Expr, accum: &mut HashSet) -> Result<()> { | Expr::WindowFunction { .. } | Expr::AggregateFunction { .. } | Expr::GroupingSet(_) - | Expr::AggregateUDF { .. } | Expr::InList { .. } | Expr::Exists { .. } | Expr::InSubquery(_) @@ -595,15 +594,12 @@ pub fn group_window_expr_by_sort_keys( Ok(result) } -/// Collect all deeply nested `Expr::AggregateFunction` and -/// `Expr::AggregateUDF`. They are returned in order of occurrence (depth +/// Collect all deeply nested `Expr::AggregateFunction`. +/// They are returned in order of occurrence (depth /// first), with duplicates omitted. pub fn find_aggregate_exprs(exprs: &[Expr]) -> Vec { find_exprs_in_exprs(exprs, &|nested_expr| { - matches!( - nested_expr, - Expr::AggregateFunction { .. } | Expr::AggregateUDF { .. } - ) + matches!(nested_expr, Expr::AggregateFunction { .. }) }) } diff --git a/datafusion/optimizer/src/analyzer/count_wildcard_rule.rs b/datafusion/optimizer/src/analyzer/count_wildcard_rule.rs index b4de322f76f6..fd84bb80160b 100644 --- a/datafusion/optimizer/src/analyzer/count_wildcard_rule.rs +++ b/datafusion/optimizer/src/analyzer/count_wildcard_rule.rs @@ -19,7 +19,7 @@ use crate::analyzer::AnalyzerRule; use datafusion_common::config::ConfigOptions; use datafusion_common::tree_node::{Transformed, TreeNode, TreeNodeRewriter}; use datafusion_common::Result; -use datafusion_expr::expr::{AggregateFunction, InSubquery}; +use datafusion_expr::expr::{AggregateFunction, AggregateFunctionDefinition, InSubquery}; use datafusion_expr::expr_rewriter::rewrite_preserving_name; use datafusion_expr::utils::COUNT_STAR_EXPANSION; use datafusion_expr::Expr::ScalarSubquery; @@ -144,20 +144,23 @@ impl TreeNodeRewriter for CountWildcardRewriter { _ => old_expr, }, Expr::AggregateFunction(AggregateFunction { - fun: aggregate_function::AggregateFunction::Count, + func_def: + AggregateFunctionDefinition::BuiltIn( + aggregate_function::AggregateFunction::Count, + ), args, distinct, filter, order_by, }) if args.len() == 1 => match args[0] { Expr::Wildcard { qualifier: None } => { - Expr::AggregateFunction(AggregateFunction { - fun: aggregate_function::AggregateFunction::Count, - args: vec![lit(COUNT_STAR_EXPANSION)], + Expr::AggregateFunction(AggregateFunction::new( + aggregate_function::AggregateFunction::Count, + vec![lit(COUNT_STAR_EXPANSION)], distinct, filter, order_by, - }) + )) } _ => old_expr, }, diff --git a/datafusion/optimizer/src/analyzer/type_coercion.rs b/datafusion/optimizer/src/analyzer/type_coercion.rs index eb5d8c53a5e0..bedc86e2f4f1 100644 --- a/datafusion/optimizer/src/analyzer/type_coercion.rs +++ b/datafusion/optimizer/src/analyzer/type_coercion.rs @@ -28,8 +28,8 @@ use datafusion_common::{ DataFusionError, Result, ScalarValue, }; use datafusion_expr::expr::{ - self, Between, BinaryExpr, Case, Exists, InList, InSubquery, Like, ScalarFunction, - WindowFunction, + self, AggregateFunctionDefinition, Between, BinaryExpr, Case, Exists, InList, + InSubquery, Like, ScalarFunction, WindowFunction, }; use datafusion_expr::expr_rewriter::rewrite_preserving_name; use datafusion_expr::expr_schema::cast_subquery; @@ -346,39 +346,39 @@ impl TreeNodeRewriter for TypeCoercionRewriter { } }, Expr::AggregateFunction(expr::AggregateFunction { - fun, + func_def, args, distinct, filter, order_by, - }) => { - let new_expr = coerce_agg_exprs_for_signature( - &fun, - &args, - &self.schema, - &fun.signature(), - )?; - let expr = Expr::AggregateFunction(expr::AggregateFunction::new( - fun, new_expr, distinct, filter, order_by, - )); - Ok(expr) - } - Expr::AggregateUDF(expr::AggregateUDF { - fun, - args, - filter, - order_by, - }) => { - let new_expr = coerce_arguments_for_signature( - args.as_slice(), - &self.schema, - fun.signature(), - )?; - let expr = Expr::AggregateUDF(expr::AggregateUDF::new( - fun, new_expr, filter, order_by, - )); - Ok(expr) - } + }) => match func_def { + AggregateFunctionDefinition::BuiltIn(fun) => { + let new_expr = coerce_agg_exprs_for_signature( + &fun, + &args, + &self.schema, + &fun.signature(), + )?; + let expr = Expr::AggregateFunction(expr::AggregateFunction::new( + fun, new_expr, distinct, filter, order_by, + )); + Ok(expr) + } + AggregateFunctionDefinition::UDF(fun) => { + let new_expr = coerce_arguments_for_signature( + args.as_slice(), + &self.schema, + fun.signature(), + )?; + let expr = Expr::AggregateFunction(expr::AggregateFunction::new_udf( + fun, new_expr, false, filter, order_by, + )); + Ok(expr) + } + AggregateFunctionDefinition::Name(_) => { + internal_err!("Function `Expr` with name should be resolved.") + } + }, Expr::WindowFunction(WindowFunction { fun, args, @@ -914,9 +914,10 @@ mod test { Arc::new(|_| Ok(Box::::default())), Arc::new(vec![DataType::UInt64, DataType::Float64]), ); - let udaf = Expr::AggregateUDF(expr::AggregateUDF::new( + let udaf = Expr::AggregateFunction(expr::AggregateFunction::new_udf( Arc::new(my_avg), vec![lit(10i64)], + false, None, None, )); @@ -941,9 +942,10 @@ mod test { &accumulator, &state_type, ); - let udaf = Expr::AggregateUDF(expr::AggregateUDF::new( + let udaf = Expr::AggregateFunction(expr::AggregateFunction::new_udf( Arc::new(my_avg), vec![lit("10")], + false, None, None, )); diff --git a/datafusion/optimizer/src/common_subexpr_eliminate.rs b/datafusion/optimizer/src/common_subexpr_eliminate.rs index f5ad767c5016..1d21407a6985 100644 --- a/datafusion/optimizer/src/common_subexpr_eliminate.rs +++ b/datafusion/optimizer/src/common_subexpr_eliminate.rs @@ -509,10 +509,9 @@ enum ExprMask { /// - [`Sort`](Expr::Sort) /// - [`Wildcard`](Expr::Wildcard) /// - [`AggregateFunction`](Expr::AggregateFunction) - /// - [`AggregateUDF`](Expr::AggregateUDF) Normal, - /// Like [`Normal`](Self::Normal), but includes [`AggregateFunction`](Expr::AggregateFunction) and [`AggregateUDF`](Expr::AggregateUDF). + /// Like [`Normal`](Self::Normal), but includes [`AggregateFunction`](Expr::AggregateFunction). NormalAndAggregates, } @@ -528,10 +527,7 @@ impl ExprMask { | Expr::Wildcard { .. } ); - let is_aggr = matches!( - expr, - Expr::AggregateFunction(..) | Expr::AggregateUDF { .. } - ); + let is_aggr = matches!(expr, Expr::AggregateFunction(..)); match self { Self::Normal => is_normal_minus_aggregates || is_aggr, @@ -908,7 +904,7 @@ mod test { let accumulator: AccumulatorFactoryFunction = Arc::new(|_| unimplemented!()); let state_type: StateTypeFunction = Arc::new(|_| unimplemented!()); let udf_agg = |inner: Expr| { - Expr::AggregateUDF(datafusion_expr::expr::AggregateUDF::new( + Expr::AggregateFunction(datafusion_expr::expr::AggregateFunction::new_udf( Arc::new(AggregateUDF::new( "my_agg", &Signature::exact(vec![DataType::UInt32], Volatility::Stable), @@ -917,6 +913,7 @@ mod test { &state_type, )), vec![inner], + false, None, None, )) diff --git a/datafusion/optimizer/src/decorrelate.rs b/datafusion/optimizer/src/decorrelate.rs index ed6f472186d4..b1000f042c98 100644 --- a/datafusion/optimizer/src/decorrelate.rs +++ b/datafusion/optimizer/src/decorrelate.rs @@ -22,7 +22,7 @@ use datafusion_common::tree_node::{ }; use datafusion_common::{plan_err, Result}; use datafusion_common::{Column, DFSchemaRef, DataFusionError, ScalarValue}; -use datafusion_expr::expr::Alias; +use datafusion_expr::expr::{AggregateFunctionDefinition, Alias}; use datafusion_expr::utils::{conjunction, find_join_exprs, split_conjunction}; use datafusion_expr::{expr, EmptyRelation, Expr, LogicalPlan, LogicalPlanBuilder}; use datafusion_physical_expr::execution_props::ExecutionProps; @@ -372,16 +372,25 @@ fn agg_exprs_evaluation_result_on_empty_batch( for e in agg_expr.iter() { let result_expr = e.clone().transform_up(&|expr| { let new_expr = match expr { - Expr::AggregateFunction(expr::AggregateFunction { fun, .. }) => { - if matches!(fun, datafusion_expr::AggregateFunction::Count) { - Transformed::Yes(Expr::Literal(ScalarValue::Int64(Some(0)))) - } else { - Transformed::Yes(Expr::Literal(ScalarValue::Null)) + Expr::AggregateFunction(expr::AggregateFunction { func_def, .. }) => { + match func_def { + AggregateFunctionDefinition::BuiltIn(fun) => { + if matches!(fun, datafusion_expr::AggregateFunction::Count) { + Transformed::Yes(Expr::Literal(ScalarValue::Int64(Some( + 0, + )))) + } else { + Transformed::Yes(Expr::Literal(ScalarValue::Null)) + } + } + AggregateFunctionDefinition::UDF { .. } => { + Transformed::Yes(Expr::Literal(ScalarValue::Null)) + } + AggregateFunctionDefinition::Name(_) => { + Transformed::Yes(Expr::Literal(ScalarValue::Null)) + } } } - Expr::AggregateUDF(_) => { - Transformed::Yes(Expr::Literal(ScalarValue::Null)) - } _ => Transformed::No(expr), }; Ok(new_expr) diff --git a/datafusion/optimizer/src/push_down_filter.rs b/datafusion/optimizer/src/push_down_filter.rs index 95eeee931b4f..bad6e24715c9 100644 --- a/datafusion/optimizer/src/push_down_filter.rs +++ b/datafusion/optimizer/src/push_down_filter.rs @@ -253,7 +253,6 @@ fn can_evaluate_as_join_condition(predicate: &Expr) -> Result { Expr::Sort(_) | Expr::AggregateFunction(_) | Expr::WindowFunction(_) - | Expr::AggregateUDF { .. } | Expr::Wildcard { .. } | Expr::GroupingSet(_) => internal_err!("Unsupported predicate type"), })?; diff --git a/datafusion/optimizer/src/simplify_expressions/expr_simplifier.rs b/datafusion/optimizer/src/simplify_expressions/expr_simplifier.rs index 3310bfed75bf..c7366e17619c 100644 --- a/datafusion/optimizer/src/simplify_expressions/expr_simplifier.rs +++ b/datafusion/optimizer/src/simplify_expressions/expr_simplifier.rs @@ -332,7 +332,6 @@ impl<'a> ConstEvaluator<'a> { // Has no runtime cost, but needed during planning Expr::Alias(..) | Expr::AggregateFunction { .. } - | Expr::AggregateUDF { .. } | Expr::ScalarVariable(_, _) | Expr::Column(_) | Expr::OuterReferenceColumn(_, _) diff --git a/datafusion/optimizer/src/single_distinct_to_groupby.rs b/datafusion/optimizer/src/single_distinct_to_groupby.rs index fa142438c4a3..7e6fb6b355ab 100644 --- a/datafusion/optimizer/src/single_distinct_to_groupby.rs +++ b/datafusion/optimizer/src/single_distinct_to_groupby.rs @@ -23,6 +23,7 @@ use crate::optimizer::ApplyOrder; use crate::{OptimizerConfig, OptimizerRule}; use datafusion_common::{DFSchema, Result}; +use datafusion_expr::expr::AggregateFunctionDefinition; use datafusion_expr::{ aggregate_function::AggregateFunction::{Max, Min, Sum}, col, @@ -70,7 +71,7 @@ fn is_single_distinct_agg(plan: &LogicalPlan) -> Result { let mut aggregate_count = 0; for expr in aggr_expr { if let Expr::AggregateFunction(AggregateFunction { - fun, + func_def: AggregateFunctionDefinition::BuiltIn(fun), distinct, args, filter, @@ -170,7 +171,7 @@ impl OptimizerRule for SingleDistinctToGroupBy { .iter() .map(|aggr_expr| match aggr_expr { Expr::AggregateFunction(AggregateFunction { - fun, + func_def: AggregateFunctionDefinition::BuiltIn(fun), args, distinct, .. diff --git a/datafusion/proto/src/logical_plan/from_proto.rs b/datafusion/proto/src/logical_plan/from_proto.rs index d596998c1de3..ae3628bddeb2 100644 --- a/datafusion/proto/src/logical_plan/from_proto.rs +++ b/datafusion/proto/src/logical_plan/from_proto.rs @@ -1744,12 +1744,13 @@ pub fn parse_expr( ExprType::AggregateUdfExpr(pb) => { let agg_fn = registry.udaf(pb.fun_name.as_str())?; - Ok(Expr::AggregateUDF(expr::AggregateUDF::new( + Ok(Expr::AggregateFunction(expr::AggregateFunction::new_udf( agg_fn, pb.args .iter() .map(|expr| parse_expr(expr, registry)) .collect::, Error>>()?, + false, parse_optional_expr(pb.filter.as_deref(), registry)?.map(Box::new), parse_vec_expr(&pb.order_by, registry)?, ))) diff --git a/datafusion/proto/src/logical_plan/to_proto.rs b/datafusion/proto/src/logical_plan/to_proto.rs index 54be6460c392..b619339674fd 100644 --- a/datafusion/proto/src/logical_plan/to_proto.rs +++ b/datafusion/proto/src/logical_plan/to_proto.rs @@ -44,8 +44,9 @@ use datafusion_common::{ ScalarValue, }; use datafusion_expr::expr::{ - self, Alias, Between, BinaryExpr, Cast, GetFieldAccess, GetIndexedField, GroupingSet, - InList, Like, Placeholder, ScalarFunction, ScalarFunctionDefinition, Sort, + self, AggregateFunctionDefinition, Alias, Between, BinaryExpr, Cast, GetFieldAccess, + GetIndexedField, GroupingSet, InList, Like, Placeholder, ScalarFunction, + ScalarFunctionDefinition, Sort, }; use datafusion_expr::{ logical_plan::PlanType, logical_plan::StringifiedPlan, AggregateFunction, @@ -652,104 +653,139 @@ impl TryFrom<&Expr> for protobuf::LogicalExprNode { } } Expr::AggregateFunction(expr::AggregateFunction { - ref fun, + ref func_def, ref args, ref distinct, ref filter, ref order_by, }) => { - let aggr_function = match fun { - AggregateFunction::ApproxDistinct => { - protobuf::AggregateFunction::ApproxDistinct - } - AggregateFunction::ApproxPercentileCont => { - protobuf::AggregateFunction::ApproxPercentileCont - } - AggregateFunction::ApproxPercentileContWithWeight => { - protobuf::AggregateFunction::ApproxPercentileContWithWeight - } - AggregateFunction::ArrayAgg => protobuf::AggregateFunction::ArrayAgg, - AggregateFunction::Min => protobuf::AggregateFunction::Min, - AggregateFunction::Max => protobuf::AggregateFunction::Max, - AggregateFunction::Sum => protobuf::AggregateFunction::Sum, - AggregateFunction::BitAnd => protobuf::AggregateFunction::BitAnd, - AggregateFunction::BitOr => protobuf::AggregateFunction::BitOr, - AggregateFunction::BitXor => protobuf::AggregateFunction::BitXor, - AggregateFunction::BoolAnd => protobuf::AggregateFunction::BoolAnd, - AggregateFunction::BoolOr => protobuf::AggregateFunction::BoolOr, - AggregateFunction::Avg => protobuf::AggregateFunction::Avg, - AggregateFunction::Count => protobuf::AggregateFunction::Count, - AggregateFunction::Variance => protobuf::AggregateFunction::Variance, - AggregateFunction::VariancePop => { - protobuf::AggregateFunction::VariancePop - } - AggregateFunction::Covariance => { - protobuf::AggregateFunction::Covariance - } - AggregateFunction::CovariancePop => { - protobuf::AggregateFunction::CovariancePop - } - AggregateFunction::Stddev => protobuf::AggregateFunction::Stddev, - AggregateFunction::StddevPop => { - protobuf::AggregateFunction::StddevPop - } - AggregateFunction::Correlation => { - protobuf::AggregateFunction::Correlation - } - AggregateFunction::RegrSlope => { - protobuf::AggregateFunction::RegrSlope - } - AggregateFunction::RegrIntercept => { - protobuf::AggregateFunction::RegrIntercept - } - AggregateFunction::RegrR2 => protobuf::AggregateFunction::RegrR2, - AggregateFunction::RegrAvgx => protobuf::AggregateFunction::RegrAvgx, - AggregateFunction::RegrAvgy => protobuf::AggregateFunction::RegrAvgy, - AggregateFunction::RegrCount => { - protobuf::AggregateFunction::RegrCount - } - AggregateFunction::RegrSXX => protobuf::AggregateFunction::RegrSxx, - AggregateFunction::RegrSYY => protobuf::AggregateFunction::RegrSyy, - AggregateFunction::RegrSXY => protobuf::AggregateFunction::RegrSxy, - AggregateFunction::ApproxMedian => { - protobuf::AggregateFunction::ApproxMedian - } - AggregateFunction::Grouping => protobuf::AggregateFunction::Grouping, - AggregateFunction::Median => protobuf::AggregateFunction::Median, - AggregateFunction::FirstValue => { - protobuf::AggregateFunction::FirstValueAgg - } - AggregateFunction::LastValue => { - protobuf::AggregateFunction::LastValueAgg - } - AggregateFunction::StringAgg => { - protobuf::AggregateFunction::StringAgg + match func_def { + AggregateFunctionDefinition::BuiltIn(fun) => { + let aggr_function = match fun { + AggregateFunction::ApproxDistinct => { + protobuf::AggregateFunction::ApproxDistinct + } + AggregateFunction::ApproxPercentileCont => { + protobuf::AggregateFunction::ApproxPercentileCont + } + AggregateFunction::ApproxPercentileContWithWeight => { + protobuf::AggregateFunction::ApproxPercentileContWithWeight + } + AggregateFunction::ArrayAgg => protobuf::AggregateFunction::ArrayAgg, + AggregateFunction::Min => protobuf::AggregateFunction::Min, + AggregateFunction::Max => protobuf::AggregateFunction::Max, + AggregateFunction::Sum => protobuf::AggregateFunction::Sum, + AggregateFunction::BitAnd => protobuf::AggregateFunction::BitAnd, + AggregateFunction::BitOr => protobuf::AggregateFunction::BitOr, + AggregateFunction::BitXor => protobuf::AggregateFunction::BitXor, + AggregateFunction::BoolAnd => protobuf::AggregateFunction::BoolAnd, + AggregateFunction::BoolOr => protobuf::AggregateFunction::BoolOr, + AggregateFunction::Avg => protobuf::AggregateFunction::Avg, + AggregateFunction::Count => protobuf::AggregateFunction::Count, + AggregateFunction::Variance => protobuf::AggregateFunction::Variance, + AggregateFunction::VariancePop => { + protobuf::AggregateFunction::VariancePop + } + AggregateFunction::Covariance => { + protobuf::AggregateFunction::Covariance + } + AggregateFunction::CovariancePop => { + protobuf::AggregateFunction::CovariancePop + } + AggregateFunction::Stddev => protobuf::AggregateFunction::Stddev, + AggregateFunction::StddevPop => { + protobuf::AggregateFunction::StddevPop + } + AggregateFunction::Correlation => { + protobuf::AggregateFunction::Correlation + } + AggregateFunction::RegrSlope => { + protobuf::AggregateFunction::RegrSlope + } + AggregateFunction::RegrIntercept => { + protobuf::AggregateFunction::RegrIntercept + } + AggregateFunction::RegrR2 => protobuf::AggregateFunction::RegrR2, + AggregateFunction::RegrAvgx => protobuf::AggregateFunction::RegrAvgx, + AggregateFunction::RegrAvgy => protobuf::AggregateFunction::RegrAvgy, + AggregateFunction::RegrCount => { + protobuf::AggregateFunction::RegrCount + } + AggregateFunction::RegrSXX => protobuf::AggregateFunction::RegrSxx, + AggregateFunction::RegrSYY => protobuf::AggregateFunction::RegrSyy, + AggregateFunction::RegrSXY => protobuf::AggregateFunction::RegrSxy, + AggregateFunction::ApproxMedian => { + protobuf::AggregateFunction::ApproxMedian + } + AggregateFunction::Grouping => protobuf::AggregateFunction::Grouping, + AggregateFunction::Median => protobuf::AggregateFunction::Median, + AggregateFunction::FirstValue => { + protobuf::AggregateFunction::FirstValueAgg + } + AggregateFunction::LastValue => { + protobuf::AggregateFunction::LastValueAgg + } + AggregateFunction::StringAgg => { + protobuf::AggregateFunction::StringAgg + } + }; + + let aggregate_expr = protobuf::AggregateExprNode { + aggr_function: aggr_function.into(), + expr: args + .iter() + .map(|v| v.try_into()) + .collect::, _>>()?, + distinct: *distinct, + filter: match filter { + Some(e) => Some(Box::new(e.as_ref().try_into()?)), + None => None, + }, + order_by: match order_by { + Some(e) => e + .iter() + .map(|expr| expr.try_into()) + .collect::, _>>()?, + None => vec![], + }, + }; + Self { + expr_type: Some(ExprType::AggregateExpr(Box::new( + aggregate_expr, + ))), + } } - }; - - let aggregate_expr = protobuf::AggregateExprNode { - aggr_function: aggr_function.into(), - expr: args - .iter() - .map(|v| v.try_into()) - .collect::, _>>()?, - distinct: *distinct, - filter: match filter { - Some(e) => Some(Box::new(e.as_ref().try_into()?)), - None => None, - }, - order_by: match order_by { - Some(e) => e - .iter() - .map(|expr| expr.try_into()) - .collect::, _>>()?, - None => vec![], + AggregateFunctionDefinition::UDF(fun) => Self { + expr_type: Some(ExprType::AggregateUdfExpr(Box::new( + protobuf::AggregateUdfExprNode { + fun_name: fun.name().to_string(), + args: args + .iter() + .map(|expr| expr.try_into()) + .collect::, Error>>()?, + filter: match filter { + Some(e) => Some(Box::new(e.as_ref().try_into()?)), + None => None, + }, + order_by: match order_by { + Some(e) => e + .iter() + .map(|expr| expr.try_into()) + .collect::, _>>()?, + None => vec![], + }, + }, + ))), }, - }; - Self { - expr_type: Some(ExprType::AggregateExpr(Box::new(aggregate_expr))), + AggregateFunctionDefinition::Name(_) => { + return Err(Error::NotImplemented( + "Proto serialization error: Trying to serialize a unresolved function" + .to_string(), + )); + } } } + Expr::ScalarVariable(_, _) => { return Err(Error::General( "Proto serialization error: Scalar Variable not supported" @@ -790,34 +826,6 @@ impl TryFrom<&Expr> for protobuf::LogicalExprNode { )); } }, - Expr::AggregateUDF(expr::AggregateUDF { - fun, - args, - filter, - order_by, - }) => Self { - expr_type: Some(ExprType::AggregateUdfExpr(Box::new( - protobuf::AggregateUdfExprNode { - fun_name: fun.name().to_string(), - args: args.iter().map(|expr| expr.try_into()).collect::, - Error, - >>( - )?, - filter: match filter { - Some(e) => Some(Box::new(e.as_ref().try_into()?)), - None => None, - }, - order_by: match order_by { - Some(e) => e - .iter() - .map(|expr| expr.try_into()) - .collect::, _>>()?, - None => vec![], - }, - }, - ))), - }, Expr::Not(expr) => { let expr = Box::new(protobuf::Not { expr: Some(Box::new(expr.as_ref().try_into()?)), diff --git a/datafusion/proto/tests/cases/roundtrip_logical_plan.rs b/datafusion/proto/tests/cases/roundtrip_logical_plan.rs index 3ab001298ed2..45727c39a373 100644 --- a/datafusion/proto/tests/cases/roundtrip_logical_plan.rs +++ b/datafusion/proto/tests/cases/roundtrip_logical_plan.rs @@ -1375,9 +1375,10 @@ fn roundtrip_aggregate_udf() { Arc::new(vec![DataType::Float64, DataType::UInt32]), ); - let test_expr = Expr::AggregateUDF(expr::AggregateUDF::new( + let test_expr = Expr::AggregateFunction(expr::AggregateFunction::new_udf( Arc::new(dummy_agg.clone()), vec![lit(1.0_f64)], + false, Some(Box::new(lit(true))), None, )); diff --git a/datafusion/sql/src/expr/function.rs b/datafusion/sql/src/expr/function.rs index 24ba4d1b506a..958e03879842 100644 --- a/datafusion/sql/src/expr/function.rs +++ b/datafusion/sql/src/expr/function.rs @@ -135,8 +135,8 @@ impl<'a, S: ContextProvider> SqlToRel<'a, S> { // User defined aggregate functions (UDAF) have precedence in case it has the same name as a scalar built-in function if let Some(fm) = self.context_provider.get_aggregate_meta(&name) { let args = self.function_args_to_expr(args, schema, planner_context)?; - return Ok(Expr::AggregateUDF(expr::AggregateUDF::new( - fm, args, None, None, + return Ok(Expr::AggregateFunction(expr::AggregateFunction::new_udf( + fm, args, false, None, None, ))); } diff --git a/datafusion/sql/src/expr/mod.rs b/datafusion/sql/src/expr/mod.rs index 25fe6b6633c2..b8c130055a5a 100644 --- a/datafusion/sql/src/expr/mod.rs +++ b/datafusion/sql/src/expr/mod.rs @@ -34,6 +34,7 @@ use datafusion_common::{ internal_err, not_impl_err, plan_err, Column, DFSchema, DataFusionError, Result, ScalarValue, }; +use datafusion_expr::expr::AggregateFunctionDefinition; use datafusion_expr::expr::InList; use datafusion_expr::expr::ScalarFunction; use datafusion_expr::{ @@ -706,7 +707,7 @@ impl<'a, S: ContextProvider> SqlToRel<'a, S> { ) -> Result { match self.sql_expr_to_logical_expr(expr, schema, planner_context)? { Expr::AggregateFunction(expr::AggregateFunction { - fun, + func_def: AggregateFunctionDefinition::BuiltIn(fun), args, distinct, order_by, diff --git a/datafusion/sql/src/select.rs b/datafusion/sql/src/select.rs index 356c53605131..c546ca755206 100644 --- a/datafusion/sql/src/select.rs +++ b/datafusion/sql/src/select.rs @@ -170,11 +170,10 @@ impl<'a, S: ContextProvider> SqlToRel<'a, S> { select_exprs .iter() .filter(|select_expr| match select_expr { - Expr::AggregateFunction(_) | Expr::AggregateUDF(_) => false, - Expr::Alias(Alias { expr, name: _, .. }) => !matches!( - **expr, - Expr::AggregateFunction(_) | Expr::AggregateUDF(_) - ), + Expr::AggregateFunction(_) => false, + Expr::Alias(Alias { expr, name: _, .. }) => { + !matches!(**expr, Expr::AggregateFunction(_)) + } _ => true, }) .cloned() diff --git a/datafusion/substrait/src/logical_plan/consumer.rs b/datafusion/substrait/src/logical_plan/consumer.rs index b7a51032dcd9..cf05d814a5cb 100644 --- a/datafusion/substrait/src/logical_plan/consumer.rs +++ b/datafusion/substrait/src/logical_plan/consumer.rs @@ -692,21 +692,14 @@ pub async fn from_substrait_agg_func( // try udaf first, then built-in aggr fn. if let Ok(fun) = ctx.udaf(function_name) { - Ok(Arc::new(Expr::AggregateUDF(expr::AggregateUDF { - fun, - args, - filter, - order_by, - }))) + Ok(Arc::new(Expr::AggregateFunction( + expr::AggregateFunction::new_udf(fun, args, distinct, filter, order_by), + ))) } else if let Ok(fun) = aggregate_function::AggregateFunction::from_str(function_name) { - Ok(Arc::new(Expr::AggregateFunction(expr::AggregateFunction { - fun, - args, - distinct, - filter, - order_by, - }))) + Ok(Arc::new(Expr::AggregateFunction( + expr::AggregateFunction::new(fun, args, distinct, filter, order_by), + ))) } else { not_impl_err!( "Aggregated function {} is not supported: function anchor = {:?}", diff --git a/datafusion/substrait/src/logical_plan/producer.rs b/datafusion/substrait/src/logical_plan/producer.rs index 2be3e7b4e884..d576e70711df 100644 --- a/datafusion/substrait/src/logical_plan/producer.rs +++ b/datafusion/substrait/src/logical_plan/producer.rs @@ -33,8 +33,8 @@ use datafusion::common::{exec_err, internal_err, not_impl_err}; #[allow(unused_imports)] use datafusion::logical_expr::aggregate_function; use datafusion::logical_expr::expr::{ - Alias, BinaryExpr, Case, Cast, GroupingSet, InList, ScalarFunctionDefinition, Sort, - WindowFunction, + AggregateFunctionDefinition, Alias, BinaryExpr, Case, Cast, GroupingSet, InList, + ScalarFunctionDefinition, Sort, WindowFunction, }; use datafusion::logical_expr::{expr, Between, JoinConstraint, LogicalPlan, Operator}; use datafusion::prelude::Expr; @@ -578,65 +578,73 @@ pub fn to_substrait_agg_measure( ), ) -> Result { match expr { - Expr::AggregateFunction(expr::AggregateFunction { fun, args, distinct, filter, order_by }) => { - let sorts = if let Some(order_by) = order_by { - order_by.iter().map(|expr| to_substrait_sort_field(expr, schema, extension_info)).collect::>>()? - } else { - vec![] - }; - let mut arguments: Vec = vec![]; - for arg in args { - arguments.push(FunctionArgument { arg_type: Some(ArgType::Value(to_substrait_rex(arg, schema, 0, extension_info)?)) }); - } - let function_anchor = _register_function(fun.to_string(), extension_info); - Ok(Measure { - measure: Some(AggregateFunction { - function_reference: function_anchor, - arguments, - sorts, - output_type: None, - invocation: match distinct { - true => AggregationInvocation::Distinct as i32, - false => AggregationInvocation::All as i32, - }, - phase: AggregationPhase::Unspecified as i32, - args: vec![], - options: vec![], - }), - filter: match filter { - Some(f) => Some(to_substrait_rex(f, schema, 0, extension_info)?), - None => None + Expr::AggregateFunction(expr::AggregateFunction { func_def, args, distinct, filter, order_by }) => { + match func_def { + AggregateFunctionDefinition::BuiltIn (fun) => { + let sorts = if let Some(order_by) = order_by { + order_by.iter().map(|expr| to_substrait_sort_field(expr, schema, extension_info)).collect::>>()? + } else { + vec![] + }; + let mut arguments: Vec = vec![]; + for arg in args { + arguments.push(FunctionArgument { arg_type: Some(ArgType::Value(to_substrait_rex(arg, schema, 0, extension_info)?)) }); + } + let function_anchor = _register_function(fun.to_string(), extension_info); + Ok(Measure { + measure: Some(AggregateFunction { + function_reference: function_anchor, + arguments, + sorts, + output_type: None, + invocation: match distinct { + true => AggregationInvocation::Distinct as i32, + false => AggregationInvocation::All as i32, + }, + phase: AggregationPhase::Unspecified as i32, + args: vec![], + options: vec![], + }), + filter: match filter { + Some(f) => Some(to_substrait_rex(f, schema, 0, extension_info)?), + None => None + } + }) } - }) - } - Expr::AggregateUDF(expr::AggregateUDF{ fun, args, filter, order_by }) =>{ - let sorts = if let Some(order_by) = order_by { - order_by.iter().map(|expr| to_substrait_sort_field(expr, schema, extension_info)).collect::>>()? - } else { - vec![] - }; - let mut arguments: Vec = vec![]; - for arg in args { - arguments.push(FunctionArgument { arg_type: Some(ArgType::Value(to_substrait_rex(arg, schema, 0, extension_info)?)) }); - } - let function_anchor = _register_function(fun.name().to_string(), extension_info); - Ok(Measure { - measure: Some(AggregateFunction { - function_reference: function_anchor, - arguments, - sorts, - output_type: None, - invocation: AggregationInvocation::All as i32, - phase: AggregationPhase::Unspecified as i32, - args: vec![], - options: vec![], - }), - filter: match filter { - Some(f) => Some(to_substrait_rex(f, schema, 0, extension_info)?), - None => None + AggregateFunctionDefinition::UDF(fun) => { + let sorts = if let Some(order_by) = order_by { + order_by.iter().map(|expr| to_substrait_sort_field(expr, schema, extension_info)).collect::>>()? + } else { + vec![] + }; + let mut arguments: Vec = vec![]; + for arg in args { + arguments.push(FunctionArgument { arg_type: Some(ArgType::Value(to_substrait_rex(arg, schema, 0, extension_info)?)) }); + } + let function_anchor = _register_function(fun.name().to_string(), extension_info); + Ok(Measure { + measure: Some(AggregateFunction { + function_reference: function_anchor, + arguments, + sorts, + output_type: None, + invocation: AggregationInvocation::All as i32, + phase: AggregationPhase::Unspecified as i32, + args: vec![], + options: vec![], + }), + filter: match filter { + Some(f) => Some(to_substrait_rex(f, schema, 0, extension_info)?), + None => None + } + }) } - }) - }, + AggregateFunctionDefinition::Name(name) => { + internal_err!("AggregateFunctionDefinition::Name({:?}) should be resolved during `AnalyzerRule`", name) + } + } + + } Expr::Alias(Alias{expr,..})=> { to_substrait_agg_measure(expr, schema, extension_info) } From d45cf00771064ffea934c476eb12b86eb4ad75b1 Mon Sep 17 00:00:00 2001 From: Tan Wei Date: Fri, 1 Dec 2023 06:06:05 +0800 Subject: [PATCH 337/572] Implement Aliases for ScalarUDF (#8360) * Implement Aliases for ScalarUDF Signed-off-by: veeupup * fix comments Signed-off-by: veeupup --------- Signed-off-by: veeupup --- datafusion/core/src/execution/context/mod.rs | 11 +++++- .../user_defined_scalar_functions.rs | 37 +++++++++++++++++++ datafusion/expr/src/udf.rs | 18 +++++++++ 3 files changed, 64 insertions(+), 2 deletions(-) diff --git a/datafusion/core/src/execution/context/mod.rs b/datafusion/core/src/execution/context/mod.rs index 46388f990a9a..dbebedce3c97 100644 --- a/datafusion/core/src/execution/context/mod.rs +++ b/datafusion/core/src/execution/context/mod.rs @@ -810,9 +810,16 @@ impl SessionContext { /// /// - `SELECT MY_FUNC(x)...` will look for a function named `"my_func"` /// - `SELECT "my_FUNC"(x)` will look for a function named `"my_FUNC"` + /// Any functions registered with the udf name or its aliases will be overwritten with this new function pub fn register_udf(&self, f: ScalarUDF) { - self.state - .write() + let mut state = self.state.write(); + let aliases = f.aliases(); + for alias in aliases { + state + .scalar_functions + .insert(alias.to_string(), Arc::new(f.clone())); + } + state .scalar_functions .insert(f.name().to_string(), Arc::new(f)); } diff --git a/datafusion/core/tests/user_defined/user_defined_scalar_functions.rs b/datafusion/core/tests/user_defined/user_defined_scalar_functions.rs index 1c7e7137290f..985b0bd5bc76 100644 --- a/datafusion/core/tests/user_defined/user_defined_scalar_functions.rs +++ b/datafusion/core/tests/user_defined/user_defined_scalar_functions.rs @@ -341,6 +341,43 @@ async fn case_sensitive_identifiers_user_defined_functions() -> Result<()> { Ok(()) } +#[tokio::test] +async fn test_user_defined_functions_with_alias() -> Result<()> { + let ctx = SessionContext::new(); + let arr = Int32Array::from(vec![1]); + let batch = RecordBatch::try_from_iter(vec![("i", Arc::new(arr) as _)])?; + ctx.register_batch("t", batch).unwrap(); + + let myfunc = |args: &[ArrayRef]| Ok(Arc::clone(&args[0])); + let myfunc = make_scalar_function(myfunc); + + let udf = create_udf( + "dummy", + vec![DataType::Int32], + Arc::new(DataType::Int32), + Volatility::Immutable, + myfunc, + ) + .with_aliases(vec!["dummy_alias"]); + + ctx.register_udf(udf); + + let expected = [ + "+------------+", + "| dummy(t.i) |", + "+------------+", + "| 1 |", + "+------------+", + ]; + let result = plan_and_collect(&ctx, "SELECT dummy(i) FROM t").await?; + assert_batches_eq!(expected, &result); + + let alias_result = plan_and_collect(&ctx, "SELECT dummy_alias(i) FROM t").await?; + assert_batches_eq!(expected, &alias_result); + + Ok(()) +} + fn create_udf_context() -> SessionContext { let ctx = SessionContext::new(); // register a custom UDF diff --git a/datafusion/expr/src/udf.rs b/datafusion/expr/src/udf.rs index bc910b928a5d..3a18ca2d25e8 100644 --- a/datafusion/expr/src/udf.rs +++ b/datafusion/expr/src/udf.rs @@ -49,6 +49,8 @@ pub struct ScalarUDF { /// the batch's row count (so that the generative zero-argument function can know /// the result array size). fun: ScalarFunctionImplementation, + /// Optional aliases for the function. This list should NOT include the value of `name` as well + aliases: Vec, } impl Debug for ScalarUDF { @@ -89,9 +91,20 @@ impl ScalarUDF { signature: signature.clone(), return_type: return_type.clone(), fun: fun.clone(), + aliases: vec![], } } + /// Adds additional names that can be used to invoke this function, in addition to `name` + pub fn with_aliases( + mut self, + aliases: impl IntoIterator, + ) -> Self { + self.aliases + .extend(aliases.into_iter().map(|s| s.to_string())); + self + } + /// creates a logical expression with a call of the UDF /// This utility allows using the UDF without requiring access to the registry. pub fn call(&self, args: Vec) -> Expr { @@ -106,6 +119,11 @@ impl ScalarUDF { &self.name } + /// Returns the aliases for this function. See [`ScalarUDF::with_aliases`] for more details + pub fn aliases(&self) -> &[String] { + &self.aliases + } + /// Returns this function's signature (what input types are accepted) pub fn signature(&self) -> &Signature { &self.signature From 513fd052bdbf5c7a73de544a876961f780b90a92 Mon Sep 17 00:00:00 2001 From: Andrew Lamb Date: Thu, 30 Nov 2023 17:11:19 -0500 Subject: [PATCH 338/572] Minor: Remove uncessary name field in ScalarFunctionDefintion (#8365) --- .../core/src/datasource/listing/helpers.rs | 2 +- datafusion/expr/src/built_in_function.rs | 9 +++++-- datafusion/expr/src/expr.rs | 14 +++------- datafusion/expr/src/expr_fn.rs | 14 +++++----- datafusion/expr/src/expr_schema.rs | 2 +- datafusion/expr/src/tree_node/expr.rs | 2 +- .../optimizer/src/analyzer/type_coercion.rs | 2 +- .../optimizer/src/optimize_projections.rs | 4 +-- datafusion/optimizer/src/push_down_filter.rs | 2 +- .../simplify_expressions/expr_simplifier.rs | 27 +++++-------------- .../src/simplify_expressions/utils.rs | 12 ++------- datafusion/physical-expr/src/planner.rs | 2 +- datafusion/proto/src/logical_plan/to_proto.rs | 2 +- datafusion/sql/src/expr/value.rs | 2 +- 14 files changed, 36 insertions(+), 60 deletions(-) diff --git a/datafusion/core/src/datasource/listing/helpers.rs b/datafusion/core/src/datasource/listing/helpers.rs index 0c39877cd11e..a4505cf62d6a 100644 --- a/datafusion/core/src/datasource/listing/helpers.rs +++ b/datafusion/core/src/datasource/listing/helpers.rs @@ -92,7 +92,7 @@ pub fn expr_applicable_for_cols(col_names: &[String], expr: &Expr) -> bool { Expr::ScalarFunction(scalar_function) => { match &scalar_function.func_def { - ScalarFunctionDefinition::BuiltIn { fun, .. } => { + ScalarFunctionDefinition::BuiltIn(fun) => { match fun.volatility() { Volatility::Immutable => Ok(VisitRecursion::Continue), // TODO: Stable functions could be `applicable`, but that would require access to the context diff --git a/datafusion/expr/src/built_in_function.rs b/datafusion/expr/src/built_in_function.rs index 2f67783201f5..a51941fdee11 100644 --- a/datafusion/expr/src/built_in_function.rs +++ b/datafusion/expr/src/built_in_function.rs @@ -348,6 +348,12 @@ impl BuiltinScalarFunction { self.signature().type_signature.supports_zero_argument() } + /// Returns the name of this function + pub fn name(&self) -> &str { + // .unwrap is safe here because compiler makes sure the map will have matches for each BuiltinScalarFunction + function_to_name().get(self).unwrap() + } + /// Returns the [Volatility] of the builtin function. pub fn volatility(&self) -> Volatility { match self { @@ -1627,8 +1633,7 @@ impl BuiltinScalarFunction { impl fmt::Display for BuiltinScalarFunction { fn fmt(&self, f: &mut fmt::Formatter) -> fmt::Result { - // .unwrap is safe here because compiler makes sure the map will have matches for each BuiltinScalarFunction - write!(f, "{}", function_to_name().get(self).unwrap()) + write!(f, "{}", self.name()) } } diff --git a/datafusion/expr/src/expr.rs b/datafusion/expr/src/expr.rs index 256f5b210ec2..ee9b0ad6f967 100644 --- a/datafusion/expr/src/expr.rs +++ b/datafusion/expr/src/expr.rs @@ -17,7 +17,6 @@ //! Expr module contains core type definition for `Expr`. -use crate::built_in_function; use crate::expr_fn::binary_expr; use crate::logical_plan::Subquery; use crate::udaf; @@ -26,6 +25,7 @@ use crate::window_frame; use crate::window_function; use crate::Operator; use crate::{aggregate_function, ExprSchemable}; +use crate::{built_in_function, BuiltinScalarFunction}; use arrow::datatypes::DataType; use datafusion_common::tree_node::{Transformed, TreeNode}; use datafusion_common::{internal_err, DFSchema, OwnedTableReference}; @@ -340,10 +340,7 @@ pub enum ScalarFunctionDefinition { /// Resolved to a `BuiltinScalarFunction` /// There is plan to migrate `BuiltinScalarFunction` to UDF-based implementation (issue#8045) /// This variant is planned to be removed in long term - BuiltIn { - fun: built_in_function::BuiltinScalarFunction, - name: Arc, - }, + BuiltIn(BuiltinScalarFunction), /// Resolved to a user defined function UDF(Arc), /// A scalar function constructed with name. This variant can not be executed directly @@ -371,7 +368,7 @@ impl ScalarFunctionDefinition { /// Function's name for display pub fn name(&self) -> &str { match self { - ScalarFunctionDefinition::BuiltIn { name, .. } => name.as_ref(), + ScalarFunctionDefinition::BuiltIn(fun) => fun.name(), ScalarFunctionDefinition::UDF(udf) => udf.name(), ScalarFunctionDefinition::Name(func_name) => func_name.as_ref(), } @@ -382,10 +379,7 @@ impl ScalarFunction { /// Create a new ScalarFunction expression pub fn new(fun: built_in_function::BuiltinScalarFunction, args: Vec) -> Self { Self { - func_def: ScalarFunctionDefinition::BuiltIn { - fun, - name: Arc::from(fun.to_string()), - }, + func_def: ScalarFunctionDefinition::BuiltIn(fun), args, } } diff --git a/datafusion/expr/src/expr_fn.rs b/datafusion/expr/src/expr_fn.rs index 1f4ab7bb4ad3..6148226f6b1a 100644 --- a/datafusion/expr/src/expr_fn.rs +++ b/datafusion/expr/src/expr_fn.rs @@ -1032,7 +1032,7 @@ mod test { macro_rules! test_unary_scalar_expr { ($ENUM:ident, $FUNC:ident) => {{ if let Expr::ScalarFunction(ScalarFunction { - func_def: ScalarFunctionDefinition::BuiltIn { fun, .. }, + func_def: ScalarFunctionDefinition::BuiltIn(fun), args, }) = $FUNC(col("tableA.a")) { @@ -1053,7 +1053,7 @@ mod test { col(stringify!($arg.to_string())) ),* ); - if let Expr::ScalarFunction(ScalarFunction { func_def: ScalarFunctionDefinition::BuiltIn{fun, ..}, args }) = result { + if let Expr::ScalarFunction(ScalarFunction { func_def: ScalarFunctionDefinition::BuiltIn(fun), args }) = result { let name = built_in_function::BuiltinScalarFunction::$ENUM; assert_eq!(name, fun); assert_eq!(expected.len(), args.len()); @@ -1073,7 +1073,7 @@ mod test { ),* ] ); - if let Expr::ScalarFunction(ScalarFunction { func_def: ScalarFunctionDefinition::BuiltIn{fun, ..}, args }) = result { + if let Expr::ScalarFunction(ScalarFunction { func_def: ScalarFunctionDefinition::BuiltIn(fun), args }) = result { let name = built_in_function::BuiltinScalarFunction::$ENUM; assert_eq!(name, fun); assert_eq!(expected.len(), args.len()); @@ -1214,7 +1214,7 @@ mod test { #[test] fn uuid_function_definitions() { if let Expr::ScalarFunction(ScalarFunction { - func_def: ScalarFunctionDefinition::BuiltIn { fun, .. }, + func_def: ScalarFunctionDefinition::BuiltIn(fun), args, }) = uuid() { @@ -1229,7 +1229,7 @@ mod test { #[test] fn digest_function_definitions() { if let Expr::ScalarFunction(ScalarFunction { - func_def: ScalarFunctionDefinition::BuiltIn { fun, .. }, + func_def: ScalarFunctionDefinition::BuiltIn(fun), args, }) = digest(col("tableA.a"), lit("md5")) { @@ -1244,7 +1244,7 @@ mod test { #[test] fn encode_function_definitions() { if let Expr::ScalarFunction(ScalarFunction { - func_def: ScalarFunctionDefinition::BuiltIn { fun, .. }, + func_def: ScalarFunctionDefinition::BuiltIn(fun), args, }) = encode(col("tableA.a"), lit("base64")) { @@ -1259,7 +1259,7 @@ mod test { #[test] fn decode_function_definitions() { if let Expr::ScalarFunction(ScalarFunction { - func_def: ScalarFunctionDefinition::BuiltIn { fun, .. }, + func_def: ScalarFunctionDefinition::BuiltIn(fun), args, }) = decode(col("tableA.a"), lit("hex")) { diff --git a/datafusion/expr/src/expr_schema.rs b/datafusion/expr/src/expr_schema.rs index 99b27e8912bc..2795ac5f0962 100644 --- a/datafusion/expr/src/expr_schema.rs +++ b/datafusion/expr/src/expr_schema.rs @@ -84,7 +84,7 @@ impl ExprSchemable for Expr { | Expr::TryCast(TryCast { data_type, .. }) => Ok(data_type.clone()), Expr::ScalarFunction(ScalarFunction { func_def, args }) => { match func_def { - ScalarFunctionDefinition::BuiltIn { fun, .. } => { + ScalarFunctionDefinition::BuiltIn(fun) => { let arg_data_types = args .iter() .map(|e| e.get_type(schema)) diff --git a/datafusion/expr/src/tree_node/expr.rs b/datafusion/expr/src/tree_node/expr.rs index fcb0a4cd93f3..1098842716b9 100644 --- a/datafusion/expr/src/tree_node/expr.rs +++ b/datafusion/expr/src/tree_node/expr.rs @@ -277,7 +277,7 @@ impl TreeNode for Expr { nulls_first, )), Expr::ScalarFunction(ScalarFunction { func_def, args }) => match func_def { - ScalarFunctionDefinition::BuiltIn { fun, .. } => Expr::ScalarFunction( + ScalarFunctionDefinition::BuiltIn(fun) => Expr::ScalarFunction( ScalarFunction::new(fun, transform_vec(args, &mut transform)?), ), ScalarFunctionDefinition::UDF(fun) => Expr::ScalarFunction( diff --git a/datafusion/optimizer/src/analyzer/type_coercion.rs b/datafusion/optimizer/src/analyzer/type_coercion.rs index bedc86e2f4f1..e3b86f5db78f 100644 --- a/datafusion/optimizer/src/analyzer/type_coercion.rs +++ b/datafusion/optimizer/src/analyzer/type_coercion.rs @@ -320,7 +320,7 @@ impl TreeNodeRewriter for TypeCoercionRewriter { Ok(Expr::Case(case)) } Expr::ScalarFunction(ScalarFunction { func_def, args }) => match func_def { - ScalarFunctionDefinition::BuiltIn { fun, .. } => { + ScalarFunctionDefinition::BuiltIn(fun) => { let new_args = coerce_arguments_for_signature( args.as_slice(), &self.schema, diff --git a/datafusion/optimizer/src/optimize_projections.rs b/datafusion/optimizer/src/optimize_projections.rs index b6d026279aa6..bbf704a83c55 100644 --- a/datafusion/optimizer/src/optimize_projections.rs +++ b/datafusion/optimizer/src/optimize_projections.rs @@ -510,9 +510,7 @@ fn rewrite_expr(expr: &Expr, input: &Projection) -> Result> { ))) } Expr::ScalarFunction(scalar_fn) => { - let fun = if let ScalarFunctionDefinition::BuiltIn { fun, .. } = - scalar_fn.func_def - { + let fun = if let ScalarFunctionDefinition::BuiltIn(fun) = scalar_fn.func_def { fun } else { return Ok(None); diff --git a/datafusion/optimizer/src/push_down_filter.rs b/datafusion/optimizer/src/push_down_filter.rs index bad6e24715c9..e8f116d89466 100644 --- a/datafusion/optimizer/src/push_down_filter.rs +++ b/datafusion/optimizer/src/push_down_filter.rs @@ -979,7 +979,7 @@ fn is_volatile_expression(e: &Expr) -> bool { e.apply(&mut |expr| { Ok(match expr { Expr::ScalarFunction(f) => match &f.func_def { - ScalarFunctionDefinition::BuiltIn { fun, .. } + ScalarFunctionDefinition::BuiltIn(fun) if fun.volatility() == Volatility::Volatile => { is_volatile = true; diff --git a/datafusion/optimizer/src/simplify_expressions/expr_simplifier.rs b/datafusion/optimizer/src/simplify_expressions/expr_simplifier.rs index c7366e17619c..41c71c9d9aff 100644 --- a/datafusion/optimizer/src/simplify_expressions/expr_simplifier.rs +++ b/datafusion/optimizer/src/simplify_expressions/expr_simplifier.rs @@ -344,7 +344,7 @@ impl<'a> ConstEvaluator<'a> { | Expr::Wildcard { .. } | Expr::Placeholder(_) => false, Expr::ScalarFunction(ScalarFunction { func_def, .. }) => match func_def { - ScalarFunctionDefinition::BuiltIn { fun, .. } => { + ScalarFunctionDefinition::BuiltIn(fun) => { Self::volatility_ok(fun.volatility()) } ScalarFunctionDefinition::UDF(fun) => { @@ -1202,41 +1202,28 @@ impl<'a, S: SimplifyInfo> TreeNodeRewriter for Simplifier<'a, S> { // log Expr::ScalarFunction(ScalarFunction { - func_def: - ScalarFunctionDefinition::BuiltIn { - fun: BuiltinScalarFunction::Log, - .. - }, + func_def: ScalarFunctionDefinition::BuiltIn(BuiltinScalarFunction::Log), args, }) => simpl_log(args, <&S>::clone(&info))?, // power Expr::ScalarFunction(ScalarFunction { - func_def: - ScalarFunctionDefinition::BuiltIn { - fun: BuiltinScalarFunction::Power, - .. - }, + func_def: ScalarFunctionDefinition::BuiltIn(BuiltinScalarFunction::Power), args, }) => simpl_power(args, <&S>::clone(&info))?, // concat Expr::ScalarFunction(ScalarFunction { - func_def: - ScalarFunctionDefinition::BuiltIn { - fun: BuiltinScalarFunction::Concat, - .. - }, + func_def: ScalarFunctionDefinition::BuiltIn(BuiltinScalarFunction::Concat), args, }) => simpl_concat(args)?, // concat_ws Expr::ScalarFunction(ScalarFunction { func_def: - ScalarFunctionDefinition::BuiltIn { - fun: BuiltinScalarFunction::ConcatWithSeparator, - .. - }, + ScalarFunctionDefinition::BuiltIn( + BuiltinScalarFunction::ConcatWithSeparator, + ), args, }) => match &args[..] { [delimiter, vals @ ..] => simpl_concat_ws(delimiter, vals)?, diff --git a/datafusion/optimizer/src/simplify_expressions/utils.rs b/datafusion/optimizer/src/simplify_expressions/utils.rs index e69207b6889a..fa91a3ace2a2 100644 --- a/datafusion/optimizer/src/simplify_expressions/utils.rs +++ b/datafusion/optimizer/src/simplify_expressions/utils.rs @@ -365,11 +365,7 @@ pub fn simpl_log(current_args: Vec, info: &dyn SimplifyInfo) -> Result Ok(args[1].clone()), _ => { @@ -409,11 +405,7 @@ pub fn simpl_power(current_args: Vec, info: &dyn SimplifyInfo) -> Result Ok(args[1].clone()), _ => Ok(Expr::ScalarFunction(ScalarFunction::new( diff --git a/datafusion/physical-expr/src/planner.rs b/datafusion/physical-expr/src/planner.rs index 5c5cc8e36fa7..5501647da2c3 100644 --- a/datafusion/physical-expr/src/planner.rs +++ b/datafusion/physical-expr/src/planner.rs @@ -349,7 +349,7 @@ pub fn create_physical_expr( } Expr::ScalarFunction(ScalarFunction { func_def, args }) => match func_def { - ScalarFunctionDefinition::BuiltIn { fun, .. } => { + ScalarFunctionDefinition::BuiltIn(fun) => { let physical_args = args .iter() .map(|e| { diff --git a/datafusion/proto/src/logical_plan/to_proto.rs b/datafusion/proto/src/logical_plan/to_proto.rs index b619339674fd..ab8e850014e5 100644 --- a/datafusion/proto/src/logical_plan/to_proto.rs +++ b/datafusion/proto/src/logical_plan/to_proto.rs @@ -793,7 +793,7 @@ impl TryFrom<&Expr> for protobuf::LogicalExprNode { )) } Expr::ScalarFunction(ScalarFunction { func_def, args }) => match func_def { - ScalarFunctionDefinition::BuiltIn { fun, .. } => { + ScalarFunctionDefinition::BuiltIn(fun) => { let fun: protobuf::ScalarFunction = fun.try_into()?; let args: Vec = args .iter() diff --git a/datafusion/sql/src/expr/value.rs b/datafusion/sql/src/expr/value.rs index f33e9e8ddf78..a3f29da488ba 100644 --- a/datafusion/sql/src/expr/value.rs +++ b/datafusion/sql/src/expr/value.rs @@ -144,7 +144,7 @@ impl<'a, S: ContextProvider> SqlToRel<'a, S> { values.push(value); } Expr::ScalarFunction(ScalarFunction { - func_def: ScalarFunctionDefinition::BuiltIn { fun, .. }, + func_def: ScalarFunctionDefinition::BuiltIn(fun), .. }) => { if fun == BuiltinScalarFunction::MakeArray { From e52d1507c25dc0c71c7168a99872f098359beb21 Mon Sep 17 00:00:00 2001 From: Alex Huang Date: Thu, 30 Nov 2023 23:12:19 +0100 Subject: [PATCH 339/572] feat: support `LargeList` in `array_empty` (#8321) * support LargeList in array_empty * update err info --- .../physical-expr/src/array_expressions.rs | 16 +++++++++--- datafusion/sqllogictest/test_files/array.slt | 26 +++++++++++++++++++ 2 files changed, 39 insertions(+), 3 deletions(-) diff --git a/datafusion/physical-expr/src/array_expressions.rs b/datafusion/physical-expr/src/array_expressions.rs index e6543808b97a..103a392b199d 100644 --- a/datafusion/physical-expr/src/array_expressions.rs +++ b/datafusion/physical-expr/src/array_expressions.rs @@ -30,7 +30,8 @@ use arrow_buffer::NullBuffer; use arrow_schema::FieldRef; use datafusion_common::cast::{ - as_generic_string_array, as_int64_array, as_list_array, as_string_array, + as_generic_list_array, as_generic_string_array, as_int64_array, as_list_array, + as_null_array, as_string_array, }; use datafusion_common::utils::array_into_list_array; use datafusion_common::{ @@ -939,12 +940,21 @@ pub fn array_concat(args: &[ArrayRef]) -> Result { /// Array_empty SQL function pub fn array_empty(args: &[ArrayRef]) -> Result { - if args[0].as_any().downcast_ref::().is_some() { + if as_null_array(&args[0]).is_ok() { // Make sure to return Boolean type. return Ok(Arc::new(BooleanArray::new_null(args[0].len()))); } + let array_type = args[0].data_type(); - let array = as_list_array(&args[0])?; + match array_type { + DataType::List(_) => array_empty_dispatch::(&args[0]), + DataType::LargeList(_) => array_empty_dispatch::(&args[0]), + _ => internal_err!("array_empty does not support type '{array_type:?}'."), + } +} + +fn array_empty_dispatch(array: &ArrayRef) -> Result { + let array = as_generic_list_array::(array)?; let builder = array .iter() .map(|arr| arr.map(|arr| arr.len() == arr.null_count())) diff --git a/datafusion/sqllogictest/test_files/array.slt b/datafusion/sqllogictest/test_files/array.slt index 9e3ac3bf08f6..3b45d995e1a2 100644 --- a/datafusion/sqllogictest/test_files/array.slt +++ b/datafusion/sqllogictest/test_files/array.slt @@ -3056,18 +3056,33 @@ select empty(make_array(1)); ---- false +query B +select empty(arrow_cast(make_array(1), 'LargeList(Int64)')); +---- +false + # empty scalar function #2 query B select empty(make_array()); ---- true +query B +select empty(arrow_cast(make_array(), 'LargeList(Null)')); +---- +true + # empty scalar function #3 query B select empty(make_array(NULL)); ---- false +query B +select empty(arrow_cast(make_array(NULL), 'LargeList(Null)')); +---- +false + # empty scalar function #4 query B select empty(NULL); @@ -3086,6 +3101,17 @@ NULL false false +query B +select empty(arrow_cast(column1, 'LargeList(List(Int64))')) from arrays; +---- +false +false +false +false +NULL +false +false + query ? SELECT string_to_array('abcxxxdef', 'xxx') ---- From 5c02664674215f6a012901ec976860544189d265 Mon Sep 17 00:00:00 2001 From: Seth Paydar <29551413+spaydar@users.noreply.github.com> Date: Thu, 30 Nov 2023 14:45:58 -0800 Subject: [PATCH 340/572] Double type argument for to_timestamp function (#8159) * feat: test queries for to_timestamp(float) WIP * feat: Float64 input for to_timestamp * cargo fmt * clippy * docs: double input type for to_timestamp * feat: cast floats to timestamp * style: cargo fmt * fix: float64 cast for timestamp nanos only --- datafusion/expr/src/built_in_function.rs | 1 + .../physical-expr/src/datetime_expressions.rs | 5 ++++ .../physical-expr/src/expressions/cast.rs | 20 +++++++++++-- .../sqllogictest/test_files/timestamps.slt | 29 +++++++++++++++++++ .../source/user-guide/sql/scalar_functions.md | 4 +-- 5 files changed, 55 insertions(+), 4 deletions(-) diff --git a/datafusion/expr/src/built_in_function.rs b/datafusion/expr/src/built_in_function.rs index a51941fdee11..d48e9e7a67fe 100644 --- a/datafusion/expr/src/built_in_function.rs +++ b/datafusion/expr/src/built_in_function.rs @@ -1023,6 +1023,7 @@ impl BuiltinScalarFunction { 1, vec![ Int64, + Float64, Timestamp(Nanosecond, None), Timestamp(Microsecond, None), Timestamp(Millisecond, None), diff --git a/datafusion/physical-expr/src/datetime_expressions.rs b/datafusion/physical-expr/src/datetime_expressions.rs index 0d42708c97ec..bc0385cd8915 100644 --- a/datafusion/physical-expr/src/datetime_expressions.rs +++ b/datafusion/physical-expr/src/datetime_expressions.rs @@ -971,6 +971,11 @@ pub fn to_timestamp_invoke(args: &[ColumnarValue]) -> Result { &DataType::Timestamp(TimeUnit::Nanosecond, None), None, ), + DataType::Float64 => cast_column( + &args[0], + &DataType::Timestamp(TimeUnit::Nanosecond, None), + None, + ), DataType::Timestamp(_, None) => cast_column( &args[0], &DataType::Timestamp(TimeUnit::Nanosecond, None), diff --git a/datafusion/physical-expr/src/expressions/cast.rs b/datafusion/physical-expr/src/expressions/cast.rs index b718b5017c5e..b3ca95292a37 100644 --- a/datafusion/physical-expr/src/expressions/cast.rs +++ b/datafusion/physical-expr/src/expressions/cast.rs @@ -176,7 +176,20 @@ pub fn cast_column( kernels::cast::cast_with_options(array, cast_type, &cast_options)?, )), ColumnarValue::Scalar(scalar) => { - let scalar_array = scalar.to_array()?; + let scalar_array = if cast_type + == &DataType::Timestamp(arrow_schema::TimeUnit::Nanosecond, None) + { + if let ScalarValue::Float64(Some(float_ts)) = scalar { + ScalarValue::Int64( + Some((float_ts * 1_000_000_000_f64).trunc() as i64), + ) + .to_array()? + } else { + scalar.to_array()? + } + } else { + scalar.to_array()? + }; let cast_array = kernels::cast::cast_with_options( &scalar_array, cast_type, @@ -201,7 +214,10 @@ pub fn cast_with_options( let expr_type = expr.data_type(input_schema)?; if expr_type == cast_type { Ok(expr.clone()) - } else if can_cast_types(&expr_type, &cast_type) { + } else if can_cast_types(&expr_type, &cast_type) + || (expr_type == DataType::Float64 + && cast_type == DataType::Timestamp(arrow_schema::TimeUnit::Nanosecond, None)) + { Ok(Arc::new(CastExpr::new(expr, cast_type, cast_options))) } else { not_impl_err!("Unsupported CAST from {expr_type:?} to {cast_type:?}") diff --git a/datafusion/sqllogictest/test_files/timestamps.slt b/datafusion/sqllogictest/test_files/timestamps.slt index 3830d8f86812..71b6ddf33f39 100644 --- a/datafusion/sqllogictest/test_files/timestamps.slt +++ b/datafusion/sqllogictest/test_files/timestamps.slt @@ -291,6 +291,35 @@ SELECT COUNT(*) FROM ts_data_secs where ts > to_timestamp_seconds('2020-09-08T12 ---- 2 + +# to_timestamp float inputs + +query PPP +SELECT to_timestamp(1.1) as c1, cast(1.1 as timestamp) as c2, 1.1::timestamp as c3; +---- +1970-01-01T00:00:01.100 1970-01-01T00:00:01.100 1970-01-01T00:00:01.100 + +query PPP +SELECT to_timestamp(-1.1) as c1, cast(-1.1 as timestamp) as c2, (-1.1)::timestamp as c3; +---- +1969-12-31T23:59:58.900 1969-12-31T23:59:58.900 1969-12-31T23:59:58.900 + +query PPP +SELECT to_timestamp(0.0) as c1, cast(0.0 as timestamp) as c2, 0.0::timestamp as c3; +---- +1970-01-01T00:00:00 1970-01-01T00:00:00 1970-01-01T00:00:00 + +query PPP +SELECT to_timestamp(1.23456789) as c1, cast(1.23456789 as timestamp) as c2, 1.23456789::timestamp as c3; +---- +1970-01-01T00:00:01.234567890 1970-01-01T00:00:01.234567890 1970-01-01T00:00:01.234567890 + +query PPP +SELECT to_timestamp(123456789.123456789) as c1, cast(123456789.123456789 as timestamp) as c2, 123456789.123456789::timestamp as c3; +---- +1973-11-29T21:33:09.123456784 1973-11-29T21:33:09.123456784 1973-11-29T21:33:09.123456784 + + # from_unixtime # 1599566400 is '2020-09-08T12:00:00+00:00' diff --git a/docs/source/user-guide/sql/scalar_functions.md b/docs/source/user-guide/sql/scalar_functions.md index c0889d94dbac..49e850ba90a8 100644 --- a/docs/source/user-guide/sql/scalar_functions.md +++ b/docs/source/user-guide/sql/scalar_functions.md @@ -1457,9 +1457,9 @@ extract(field FROM source) ### `to_timestamp` Converts a value to a timestamp (`YYYY-MM-DDT00:00:00Z`). -Supports strings, integer, and unsigned integer types as input. +Supports strings, integer, unsigned integer, and double types as input. Strings are parsed as RFC3339 (e.g. '2023-07-20T05:44:00') -Integers and unsigned integers are interpreted as seconds since the unix epoch (`1970-01-01T00:00:00Z`) +Integers, unsigned integers, and doubles are interpreted as seconds since the unix epoch (`1970-01-01T00:00:00Z`) return the corresponding timestamp. ``` From e19c669855baa8b78ff86755803944d2ddf65536 Mon Sep 17 00:00:00 2001 From: Tan Wei Date: Fri, 1 Dec 2023 06:56:03 +0800 Subject: [PATCH 341/572] Support User Defined Table Function (#8306) * Support User Defined Table Function Signed-off-by: veeupup * fix comments Signed-off-by: veeupup * add udtf test Signed-off-by: veeupup * add file header * Simply table function example, add some comments * Simplfy exprs * make clippy happy * Update datafusion/core/tests/user_defined/user_defined_table_functions.rs --------- Signed-off-by: veeupup Co-authored-by: Andrew Lamb --- datafusion-examples/examples/simple_udtf.rs | 177 ++++++++++++++ datafusion/core/src/datasource/function.rs | 56 +++++ datafusion/core/src/datasource/mod.rs | 1 + datafusion/core/src/execution/context/mod.rs | 30 ++- datafusion/core/tests/user_defined/mod.rs | 3 + .../user_defined_table_functions.rs | 219 ++++++++++++++++++ datafusion/sql/src/planner.rs | 9 + datafusion/sql/src/relation/mod.rs | 76 ++++-- 8 files changed, 550 insertions(+), 21 deletions(-) create mode 100644 datafusion-examples/examples/simple_udtf.rs create mode 100644 datafusion/core/src/datasource/function.rs create mode 100644 datafusion/core/tests/user_defined/user_defined_table_functions.rs diff --git a/datafusion-examples/examples/simple_udtf.rs b/datafusion-examples/examples/simple_udtf.rs new file mode 100644 index 000000000000..bce633765281 --- /dev/null +++ b/datafusion-examples/examples/simple_udtf.rs @@ -0,0 +1,177 @@ +// 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::csv::reader::Format; +use arrow::csv::ReaderBuilder; +use async_trait::async_trait; +use datafusion::arrow::datatypes::SchemaRef; +use datafusion::arrow::record_batch::RecordBatch; +use datafusion::datasource::function::TableFunctionImpl; +use datafusion::datasource::TableProvider; +use datafusion::error::Result; +use datafusion::execution::context::{ExecutionProps, SessionState}; +use datafusion::physical_plan::memory::MemoryExec; +use datafusion::physical_plan::ExecutionPlan; +use datafusion::prelude::SessionContext; +use datafusion_common::{plan_err, DataFusionError, ScalarValue}; +use datafusion_expr::{Expr, TableType}; +use datafusion_optimizer::simplify_expressions::{ExprSimplifier, SimplifyContext}; +use std::fs::File; +use std::io::Seek; +use std::path::Path; +use std::sync::Arc; + +// To define your own table function, you only need to do the following 3 things: +// 1. Implement your own [`TableProvider`] +// 2. Implement your own [`TableFunctionImpl`] and return your [`TableProvider`] +// 3. Register the function using [`SessionContext::register_udtf`] + +/// This example demonstrates how to register a TableFunction +#[tokio::main] +async fn main() -> Result<()> { + // create local execution context + let ctx = SessionContext::new(); + + // register the table function that will be called in SQL statements by `read_csv` + ctx.register_udtf("read_csv", Arc::new(LocalCsvTableFunc {})); + + let testdata = datafusion::test_util::arrow_test_data(); + let csv_file = format!("{testdata}/csv/aggregate_test_100.csv"); + + // Pass 2 arguments, read csv with at most 2 rows (simplify logic makes 1+1 --> 2) + let df = ctx + .sql(format!("SELECT * FROM read_csv('{csv_file}', 1 + 1);").as_str()) + .await?; + df.show().await?; + + // just run, return all rows + let df = ctx + .sql(format!("SELECT * FROM read_csv('{csv_file}');").as_str()) + .await?; + df.show().await?; + + Ok(()) +} + +/// Table Function that mimics the [`read_csv`] function in DuckDB. +/// +/// Usage: `read_csv(filename, [limit])` +/// +/// [`read_csv`]: https://duckdb.org/docs/data/csv/overview.html +struct LocalCsvTable { + schema: SchemaRef, + limit: Option, + batches: Vec, +} + +#[async_trait] +impl TableProvider for LocalCsvTable { + fn as_any(&self) -> &dyn std::any::Any { + self + } + + fn schema(&self) -> SchemaRef { + self.schema.clone() + } + + fn table_type(&self) -> TableType { + TableType::Base + } + + async fn scan( + &self, + _state: &SessionState, + projection: Option<&Vec>, + _filters: &[Expr], + _limit: Option, + ) -> Result> { + let batches = if let Some(max_return_lines) = self.limit { + // get max return rows from self.batches + let mut batches = vec![]; + let mut lines = 0; + for batch in &self.batches { + let batch_lines = batch.num_rows(); + if lines + batch_lines > max_return_lines { + let batch_lines = max_return_lines - lines; + batches.push(batch.slice(0, batch_lines)); + break; + } else { + batches.push(batch.clone()); + lines += batch_lines; + } + } + batches + } else { + self.batches.clone() + }; + Ok(Arc::new(MemoryExec::try_new( + &[batches], + TableProvider::schema(self), + projection.cloned(), + )?)) + } +} +struct LocalCsvTableFunc {} + +impl TableFunctionImpl for LocalCsvTableFunc { + fn call(&self, exprs: &[Expr]) -> Result> { + let Some(Expr::Literal(ScalarValue::Utf8(Some(ref path)))) = exprs.get(0) else { + return plan_err!("read_csv requires at least one string argument"); + }; + + let limit = exprs + .get(1) + .map(|expr| { + // try to simpify the expression, so 1+2 becomes 3, for example + let execution_props = ExecutionProps::new(); + let info = SimplifyContext::new(&execution_props); + let expr = ExprSimplifier::new(info).simplify(expr.clone())?; + + if let Expr::Literal(ScalarValue::Int64(Some(limit))) = expr { + Ok(limit as usize) + } else { + plan_err!("Limit must be an integer") + } + }) + .transpose()?; + + let (schema, batches) = read_csv_batches(path)?; + + let table = LocalCsvTable { + schema, + limit, + batches, + }; + Ok(Arc::new(table)) + } +} + +fn read_csv_batches(csv_path: impl AsRef) -> Result<(SchemaRef, Vec)> { + let mut file = File::open(csv_path)?; + let (schema, _) = Format::default().infer_schema(&mut file, None)?; + file.rewind()?; + + let reader = ReaderBuilder::new(Arc::new(schema.clone())) + .with_header(true) + .build(file)?; + let mut batches = vec![]; + for bacth in reader { + batches.push(bacth?); + } + let schema = Arc::new(schema); + Ok((schema, batches)) +} diff --git a/datafusion/core/src/datasource/function.rs b/datafusion/core/src/datasource/function.rs new file mode 100644 index 000000000000..2fd352ee4eb3 --- /dev/null +++ b/datafusion/core/src/datasource/function.rs @@ -0,0 +1,56 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +//! A table that uses a function to generate data + +use super::TableProvider; + +use datafusion_common::Result; +use datafusion_expr::Expr; + +use std::sync::Arc; + +/// A trait for table function implementations +pub trait TableFunctionImpl: Sync + Send { + /// Create a table provider + fn call(&self, args: &[Expr]) -> Result>; +} + +/// A table that uses a function to generate data +pub struct TableFunction { + /// Name of the table function + name: String, + /// Function implementation + fun: Arc, +} + +impl TableFunction { + /// Create a new table function + pub fn new(name: String, fun: Arc) -> Self { + Self { name, fun } + } + + /// Get the name of the table function + pub fn name(&self) -> &str { + &self.name + } + + /// Get the function implementation and generate a table + pub fn create_table_provider(&self, args: &[Expr]) -> Result> { + self.fun.call(args) + } +} diff --git a/datafusion/core/src/datasource/mod.rs b/datafusion/core/src/datasource/mod.rs index 45f9bee6a58b..2e516cc36a01 100644 --- a/datafusion/core/src/datasource/mod.rs +++ b/datafusion/core/src/datasource/mod.rs @@ -23,6 +23,7 @@ pub mod avro_to_arrow; pub mod default_table_source; pub mod empty; pub mod file_format; +pub mod function; pub mod listing; pub mod listing_table_factory; pub mod memory; diff --git a/datafusion/core/src/execution/context/mod.rs b/datafusion/core/src/execution/context/mod.rs index dbebedce3c97..58a4f08341d6 100644 --- a/datafusion/core/src/execution/context/mod.rs +++ b/datafusion/core/src/execution/context/mod.rs @@ -26,6 +26,7 @@ mod parquet; use crate::{ catalog::{CatalogList, MemoryCatalogList}, datasource::{ + function::{TableFunction, TableFunctionImpl}, listing::{ListingOptions, ListingTable}, provider::TableProviderFactory, }, @@ -42,7 +43,7 @@ use datafusion_common::{ use datafusion_execution::registry::SerializerRegistry; use datafusion_expr::{ logical_plan::{DdlStatement, Statement}, - StringifiedPlan, UserDefinedLogicalNode, WindowUDF, + Expr, StringifiedPlan, UserDefinedLogicalNode, WindowUDF, }; pub use datafusion_physical_expr::execution_props::ExecutionProps; use datafusion_physical_expr::var_provider::is_system_variables; @@ -803,6 +804,14 @@ impl SessionContext { .add_var_provider(variable_type, provider); } + /// Register a table UDF with this context + pub fn register_udtf(&self, name: &str, fun: Arc) { + self.state.write().table_functions.insert( + name.to_owned(), + Arc::new(TableFunction::new(name.to_owned(), fun)), + ); + } + /// Registers a scalar UDF within this context. /// /// Note in SQL queries, function names are looked up using @@ -1241,6 +1250,8 @@ pub struct SessionState { query_planner: Arc, /// Collection of catalogs containing schemas and ultimately TableProviders catalog_list: Arc, + /// Table Functions + table_functions: HashMap>, /// Scalar functions that are registered with the context scalar_functions: HashMap>, /// Aggregate functions registered in the context @@ -1339,6 +1350,7 @@ impl SessionState { physical_optimizers: PhysicalOptimizer::new(), query_planner: Arc::new(DefaultQueryPlanner {}), catalog_list, + table_functions: HashMap::new(), scalar_functions: HashMap::new(), aggregate_functions: HashMap::new(), window_functions: HashMap::new(), @@ -1877,6 +1889,22 @@ impl<'a> ContextProvider for SessionContextProvider<'a> { .ok_or_else(|| plan_datafusion_err!("table '{name}' not found")) } + fn get_table_function_source( + &self, + name: &str, + args: Vec, + ) -> Result> { + let tbl_func = self + .state + .table_functions + .get(name) + .cloned() + .ok_or_else(|| plan_datafusion_err!("table function '{name}' not found"))?; + let provider = tbl_func.create_table_provider(&args)?; + + Ok(provider_as_source(provider)) + } + fn get_function_meta(&self, name: &str) -> Option> { self.state.scalar_functions().get(name).cloned() } diff --git a/datafusion/core/tests/user_defined/mod.rs b/datafusion/core/tests/user_defined/mod.rs index 09c7c3d3266b..6c6d966cc3aa 100644 --- a/datafusion/core/tests/user_defined/mod.rs +++ b/datafusion/core/tests/user_defined/mod.rs @@ -26,3 +26,6 @@ mod user_defined_plan; /// Tests for User Defined Window Functions mod user_defined_window_functions; + +/// Tests for User Defined Table Functions +mod user_defined_table_functions; diff --git a/datafusion/core/tests/user_defined/user_defined_table_functions.rs b/datafusion/core/tests/user_defined/user_defined_table_functions.rs new file mode 100644 index 000000000000..b5d10b1c5b9b --- /dev/null +++ b/datafusion/core/tests/user_defined/user_defined_table_functions.rs @@ -0,0 +1,219 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +use arrow::array::Int64Array; +use arrow::csv::reader::Format; +use arrow::csv::ReaderBuilder; +use async_trait::async_trait; +use datafusion::arrow::datatypes::SchemaRef; +use datafusion::arrow::record_batch::RecordBatch; +use datafusion::datasource::function::TableFunctionImpl; +use datafusion::datasource::TableProvider; +use datafusion::error::Result; +use datafusion::execution::context::SessionState; +use datafusion::execution::TaskContext; +use datafusion::physical_plan::memory::MemoryExec; +use datafusion::physical_plan::{collect, ExecutionPlan}; +use datafusion::prelude::SessionContext; +use datafusion_common::{assert_batches_eq, DFSchema, ScalarValue}; +use datafusion_expr::{EmptyRelation, Expr, LogicalPlan, Projection, TableType}; +use std::fs::File; +use std::io::Seek; +use std::path::Path; +use std::sync::Arc; + +/// test simple udtf with define read_csv with parameters +#[tokio::test] +async fn test_simple_read_csv_udtf() -> Result<()> { + let ctx = SessionContext::new(); + + ctx.register_udtf("read_csv", Arc::new(SimpleCsvTableFunc {})); + + let csv_file = "tests/tpch-csv/nation.csv"; + // read csv with at most 5 rows + let rbs = ctx + .sql(format!("SELECT * FROM read_csv('{csv_file}', 5);").as_str()) + .await? + .collect() + .await?; + + let excepted = [ + "+-------------+-----------+-------------+-------------------------------------------------------------------------------------------------------------+", + "| n_nationkey | n_name | n_regionkey | n_comment |", + "+-------------+-----------+-------------+-------------------------------------------------------------------------------------------------------------+", + "| 1 | ARGENTINA | 1 | al foxes promise slyly according to the regular accounts. bold requests alon |", + "| 2 | BRAZIL | 1 | y alongside of the pending deposits. carefully special packages are about the ironic forges. slyly special |", + "| 3 | CANADA | 1 | eas hang ironic, silent packages. slyly regular packages are furiously over the tithes. fluffily bold |", + "| 4 | EGYPT | 4 | y above the carefully unusual theodolites. final dugouts are quickly across the furiously regular d |", + "| 5 | ETHIOPIA | 0 | ven packages wake quickly. regu |", + "+-------------+-----------+-------------+-------------------------------------------------------------------------------------------------------------+", ]; + assert_batches_eq!(excepted, &rbs); + + // just run, return all rows + let rbs = ctx + .sql(format!("SELECT * FROM read_csv('{csv_file}');").as_str()) + .await? + .collect() + .await?; + let excepted = [ + "+-------------+-----------+-------------+--------------------------------------------------------------------------------------------------------------------+", + "| n_nationkey | n_name | n_regionkey | n_comment |", + "+-------------+-----------+-------------+--------------------------------------------------------------------------------------------------------------------+", + "| 1 | ARGENTINA | 1 | al foxes promise slyly according to the regular accounts. bold requests alon |", + "| 2 | BRAZIL | 1 | y alongside of the pending deposits. carefully special packages are about the ironic forges. slyly special |", + "| 3 | CANADA | 1 | eas hang ironic, silent packages. slyly regular packages are furiously over the tithes. fluffily bold |", + "| 4 | EGYPT | 4 | y above the carefully unusual theodolites. final dugouts are quickly across the furiously regular d |", + "| 5 | ETHIOPIA | 0 | ven packages wake quickly. regu |", + "| 6 | FRANCE | 3 | refully final requests. regular, ironi |", + "| 7 | GERMANY | 3 | l platelets. regular accounts x-ray: unusual, regular acco |", + "| 8 | INDIA | 2 | ss excuses cajole slyly across the packages. deposits print aroun |", + "| 9 | INDONESIA | 2 | slyly express asymptotes. regular deposits haggle slyly. carefully ironic hockey players sleep blithely. carefull |", + "| 10 | IRAN | 4 | efully alongside of the slyly final dependencies. |", + "+-------------+-----------+-------------+--------------------------------------------------------------------------------------------------------------------+" + ]; + assert_batches_eq!(excepted, &rbs); + + Ok(()) +} + +struct SimpleCsvTable { + schema: SchemaRef, + exprs: Vec, + batches: Vec, +} + +#[async_trait] +impl TableProvider for SimpleCsvTable { + fn as_any(&self) -> &dyn std::any::Any { + self + } + + fn schema(&self) -> SchemaRef { + self.schema.clone() + } + + fn table_type(&self) -> TableType { + TableType::Base + } + + async fn scan( + &self, + state: &SessionState, + projection: Option<&Vec>, + _filters: &[Expr], + _limit: Option, + ) -> Result> { + let batches = if !self.exprs.is_empty() { + let max_return_lines = self.interpreter_expr(state).await?; + // get max return rows from self.batches + let mut batches = vec![]; + let mut lines = 0; + for batch in &self.batches { + let batch_lines = batch.num_rows(); + if lines + batch_lines > max_return_lines as usize { + let batch_lines = max_return_lines as usize - lines; + batches.push(batch.slice(0, batch_lines)); + break; + } else { + batches.push(batch.clone()); + lines += batch_lines; + } + } + batches + } else { + self.batches.clone() + }; + Ok(Arc::new(MemoryExec::try_new( + &[batches], + TableProvider::schema(self), + projection.cloned(), + )?)) + } +} + +impl SimpleCsvTable { + async fn interpreter_expr(&self, state: &SessionState) -> Result { + use datafusion::logical_expr::expr_rewriter::normalize_col; + use datafusion::logical_expr::utils::columnize_expr; + let plan = LogicalPlan::EmptyRelation(EmptyRelation { + produce_one_row: true, + schema: Arc::new(DFSchema::empty()), + }); + let logical_plan = Projection::try_new( + vec![columnize_expr( + normalize_col(self.exprs[0].clone(), &plan)?, + plan.schema(), + )], + Arc::new(plan), + ) + .map(LogicalPlan::Projection)?; + let rbs = collect( + state.create_physical_plan(&logical_plan).await?, + Arc::new(TaskContext::from(state)), + ) + .await?; + let limit = rbs[0] + .column(0) + .as_any() + .downcast_ref::() + .unwrap() + .value(0); + Ok(limit) + } +} + +struct SimpleCsvTableFunc {} + +impl TableFunctionImpl for SimpleCsvTableFunc { + fn call(&self, exprs: &[Expr]) -> Result> { + let mut new_exprs = vec![]; + let mut filepath = String::new(); + for expr in exprs { + match expr { + Expr::Literal(ScalarValue::Utf8(Some(ref path))) => { + filepath = path.clone() + } + expr => new_exprs.push(expr.clone()), + } + } + let (schema, batches) = read_csv_batches(filepath)?; + let table = SimpleCsvTable { + schema, + exprs: new_exprs.clone(), + batches, + }; + Ok(Arc::new(table)) + } +} + +fn read_csv_batches(csv_path: impl AsRef) -> Result<(SchemaRef, Vec)> { + let mut file = File::open(csv_path)?; + let (schema, _) = Format::default() + .with_header(true) + .infer_schema(&mut file, None)?; + file.rewind()?; + + let reader = ReaderBuilder::new(Arc::new(schema.clone())) + .with_header(true) + .build(file)?; + let mut batches = vec![]; + for bacth in reader { + batches.push(bacth?); + } + let schema = Arc::new(schema); + Ok((schema, batches)) +} diff --git a/datafusion/sql/src/planner.rs b/datafusion/sql/src/planner.rs index 622e5aca799a..c5c30e3a2253 100644 --- a/datafusion/sql/src/planner.rs +++ b/datafusion/sql/src/planner.rs @@ -52,6 +52,15 @@ pub trait ContextProvider { } /// Getter for a datasource fn get_table_source(&self, name: TableReference) -> Result>; + /// Getter for a table function + fn get_table_function_source( + &self, + _name: &str, + _args: Vec, + ) -> Result> { + not_impl_err!("Table Functions are not supported") + } + /// Getter for a UDF description fn get_function_meta(&self, name: &str) -> Option>; /// Getter for a UDAF description diff --git a/datafusion/sql/src/relation/mod.rs b/datafusion/sql/src/relation/mod.rs index 180743d19b7b..6fc7e9601243 100644 --- a/datafusion/sql/src/relation/mod.rs +++ b/datafusion/sql/src/relation/mod.rs @@ -16,9 +16,11 @@ // under the License. use crate::planner::{ContextProvider, PlannerContext, SqlToRel}; -use datafusion_common::{not_impl_err, DataFusionError, Result}; +use datafusion_common::{ + not_impl_err, plan_err, DFSchema, DataFusionError, Result, TableReference, +}; use datafusion_expr::{LogicalPlan, LogicalPlanBuilder}; -use sqlparser::ast::TableFactor; +use sqlparser::ast::{FunctionArg, FunctionArgExpr, TableFactor}; mod join; @@ -30,24 +32,58 @@ impl<'a, S: ContextProvider> SqlToRel<'a, S> { planner_context: &mut PlannerContext, ) -> Result { let (plan, alias) = match relation { - TableFactor::Table { name, alias, .. } => { - // normalize name and alias - let table_ref = self.object_name_to_table_reference(name)?; - let table_name = table_ref.to_string(); - let cte = planner_context.get_cte(&table_name); - ( - match ( - cte, - self.context_provider.get_table_source(table_ref.clone()), - ) { - (Some(cte_plan), _) => Ok(cte_plan.clone()), - (_, Ok(provider)) => { - LogicalPlanBuilder::scan(table_ref, provider, None)?.build() - } - (None, Err(e)) => Err(e), - }?, - alias, - ) + TableFactor::Table { + name, alias, args, .. + } => { + if let Some(func_args) = args { + let tbl_func_name = name.0.get(0).unwrap().value.to_string(); + let args = func_args + .into_iter() + .flat_map(|arg| { + if let FunctionArg::Unnamed(FunctionArgExpr::Expr(expr)) = arg + { + self.sql_expr_to_logical_expr( + expr, + &DFSchema::empty(), + planner_context, + ) + } else { + plan_err!("Unsupported function argument type: {:?}", arg) + } + }) + .collect::>(); + let provider = self + .context_provider + .get_table_function_source(&tbl_func_name, args)?; + let plan = LogicalPlanBuilder::scan( + TableReference::Bare { + table: std::borrow::Cow::Borrowed("tmp_table"), + }, + provider, + None, + )? + .build()?; + (plan, alias) + } else { + // normalize name and alias + let table_ref = self.object_name_to_table_reference(name)?; + let table_name = table_ref.to_string(); + let cte = planner_context.get_cte(&table_name); + ( + match ( + cte, + self.context_provider.get_table_source(table_ref.clone()), + ) { + (Some(cte_plan), _) => Ok(cte_plan.clone()), + (_, Ok(provider)) => { + LogicalPlanBuilder::scan(table_ref, provider, None)? + .build() + } + (None, Err(e)) => Err(e), + }?, + alias, + ) + } } TableFactor::Derived { subquery, alias, .. From c19260d6b6cf294cf05d98cc2d7e41855ca358c7 Mon Sep 17 00:00:00 2001 From: comphead Date: Fri, 1 Dec 2023 11:52:39 -0800 Subject: [PATCH 342/572] Document timestamp input limits (#8369) * document timestamp input limis * fix text * prettier * remove doc for nanoseconds * Update datafusion/physical-expr/src/datetime_expressions.rs Co-authored-by: Andrew Lamb --------- Co-authored-by: Andrew Lamb --- datafusion/physical-expr/src/datetime_expressions.rs | 4 ++++ docs/source/user-guide/sql/scalar_functions.md | 3 +++ 2 files changed, 7 insertions(+) diff --git a/datafusion/physical-expr/src/datetime_expressions.rs b/datafusion/physical-expr/src/datetime_expressions.rs index bc0385cd8915..a4d8118cf86b 100644 --- a/datafusion/physical-expr/src/datetime_expressions.rs +++ b/datafusion/physical-expr/src/datetime_expressions.rs @@ -130,6 +130,10 @@ fn string_to_timestamp_nanos_shim(s: &str) -> Result { } /// to_timestamp SQL function +/// +/// Note: `to_timestamp` returns `Timestamp(Nanosecond)` though its arguments are interpreted as **seconds**. The supported range for integer input is between `-9223372037` and `9223372036`. +/// Supported range for string input is between `1677-09-21T00:12:44.0` and `2262-04-11T23:47:16.0`. +/// Please use `to_timestamp_seconds` for the input outside of supported bounds. pub fn to_timestamp(args: &[ColumnarValue]) -> Result { handle::( args, diff --git a/docs/source/user-guide/sql/scalar_functions.md b/docs/source/user-guide/sql/scalar_functions.md index 49e850ba90a8..0d9725203c3d 100644 --- a/docs/source/user-guide/sql/scalar_functions.md +++ b/docs/source/user-guide/sql/scalar_functions.md @@ -1462,6 +1462,9 @@ Strings are parsed as RFC3339 (e.g. '2023-07-20T05:44:00') Integers, unsigned integers, and doubles are interpreted as seconds since the unix epoch (`1970-01-01T00:00:00Z`) return the corresponding timestamp. +Note: `to_timestamp` returns `Timestamp(Nanosecond)`. The supported range for integer input is between `-9223372037` and `9223372036`. +Supported range for string input is between `1677-09-21T00:12:44.0` and `2262-04-11T23:47:16.0`. Please use `to_timestamp_seconds` for the input outside of supported bounds. + ``` to_timestamp(expression) ``` From eb5aa220261e9f5ec3e4cce3098f6de92f820785 Mon Sep 17 00:00:00 2001 From: Huaijin Date: Sat, 2 Dec 2023 03:53:58 +0800 Subject: [PATCH 343/572] fix: make `ntile` work in some corner cases (#8371) * fix: make ntile work in some corner cases * fix comments * minor * Update datafusion/sqllogictest/test_files/window.slt Co-authored-by: Mustafa Akur <106137913+mustafasrepo@users.noreply.github.com> --------- Co-authored-by: Mustafa Akur <106137913+mustafasrepo@users.noreply.github.com> --- datafusion/expr/src/window_function.rs | 15 +- datafusion/physical-expr/src/window/ntile.rs | 3 +- datafusion/physical-plan/src/windows/mod.rs | 29 ++-- datafusion/sqllogictest/test_files/window.slt | 146 ++++++++++++++++++ 4 files changed, 182 insertions(+), 11 deletions(-) diff --git a/datafusion/expr/src/window_function.rs b/datafusion/expr/src/window_function.rs index 946a80dd844a..610f1ecaeae9 100644 --- a/datafusion/expr/src/window_function.rs +++ b/datafusion/expr/src/window_function.rs @@ -268,7 +268,20 @@ impl BuiltInWindowFunction { BuiltInWindowFunction::FirstValue | BuiltInWindowFunction::LastValue => { Signature::any(1, Volatility::Immutable) } - BuiltInWindowFunction::Ntile => Signature::any(1, Volatility::Immutable), + BuiltInWindowFunction::Ntile => Signature::uniform( + 1, + vec![ + DataType::UInt64, + DataType::UInt32, + DataType::UInt16, + DataType::UInt8, + DataType::Int64, + DataType::Int32, + DataType::Int16, + DataType::Int8, + ], + Volatility::Immutable, + ), BuiltInWindowFunction::NthValue => Signature::any(2, Volatility::Immutable), } } diff --git a/datafusion/physical-expr/src/window/ntile.rs b/datafusion/physical-expr/src/window/ntile.rs index 49aac0877ab3..f5442e1b0fee 100644 --- a/datafusion/physical-expr/src/window/ntile.rs +++ b/datafusion/physical-expr/src/window/ntile.rs @@ -96,8 +96,9 @@ impl PartitionEvaluator for NtileEvaluator { ) -> Result { let num_rows = num_rows as u64; let mut vec: Vec = Vec::new(); + let n = u64::min(self.n, num_rows); for i in 0..num_rows { - let res = i * self.n / num_rows; + let res = i * n / num_rows; vec.push(res + 1) } Ok(Arc::new(UInt64Array::from(vec))) diff --git a/datafusion/physical-plan/src/windows/mod.rs b/datafusion/physical-plan/src/windows/mod.rs index d97e3c93a136..828dcb4b130c 100644 --- a/datafusion/physical-plan/src/windows/mod.rs +++ b/datafusion/physical-plan/src/windows/mod.rs @@ -189,15 +189,26 @@ fn create_built_in_window_expr( BuiltInWindowFunction::PercentRank => Arc::new(percent_rank(name)), BuiltInWindowFunction::CumeDist => Arc::new(cume_dist(name)), BuiltInWindowFunction::Ntile => { - let n: i64 = get_scalar_value_from_args(args, 0)? - .ok_or_else(|| { - DataFusionError::Execution( - "NTILE requires at least 1 argument".to_string(), - ) - })? - .try_into()?; - let n: u64 = n as u64; - Arc::new(Ntile::new(name, n)) + let n = get_scalar_value_from_args(args, 0)?.ok_or_else(|| { + DataFusionError::Execution( + "NTILE requires a positive integer".to_string(), + ) + })?; + + if n.is_null() { + return exec_err!("NTILE requires a positive integer, but finds NULL"); + } + + if n.is_unsigned() { + let n: u64 = n.try_into()?; + Arc::new(Ntile::new(name, n)) + } else { + let n: i64 = n.try_into()?; + if n <= 0 { + return exec_err!("NTILE requires a positive integer"); + } + Arc::new(Ntile::new(name, n as u64)) + } } BuiltInWindowFunction::Lag => { let arg = args[0].clone(); diff --git a/datafusion/sqllogictest/test_files/window.slt b/datafusion/sqllogictest/test_files/window.slt index b2491478d84e..bb6ca119480d 100644 --- a/datafusion/sqllogictest/test_files/window.slt +++ b/datafusion/sqllogictest/test_files/window.slt @@ -3581,3 +3581,149 @@ CREATE TABLE new_table AS SELECT NTILE(2) OVER(ORDER BY c1) AS ntile_2 FROM aggr statement ok DROP TABLE new_table; + +statement ok +CREATE TABLE t1 (a int) AS VALUES (1), (2), (3); + +query I +SELECT NTILE(9223377) OVER(ORDER BY a) FROM t1; +---- +1 +2 +3 + +query I +SELECT NTILE(9223372036854775809) OVER(ORDER BY a) FROM t1; +---- +1 +2 +3 + +query error DataFusion error: Execution error: NTILE requires a positive integer +SELECT NTILE(-922337203685477580) OVER(ORDER BY a) FROM t1; + +query error DataFusion error: Execution error: Table 't' doesn't exist\. +DROP TABLE t; + +# NTILE with PARTITION BY, those tests from duckdb: https://github.com/duckdb/duckdb/blob/main/test/sql/window/test_ntile.test +statement ok +CREATE TABLE score_board (team_name VARCHAR, player VARCHAR, score INTEGER) as VALUES + ('Mongrels', 'Apu', 350), + ('Mongrels', 'Ned', 666), + ('Mongrels', 'Meg', 1030), + ('Mongrels', 'Burns', 1270), + ('Simpsons', 'Homer', 1), + ('Simpsons', 'Lisa', 710), + ('Simpsons', 'Marge', 990), + ('Simpsons', 'Bart', 2010) + +query TTII +SELECT + team_name, + player, + score, + NTILE(2) OVER (PARTITION BY team_name ORDER BY score ASC) AS NTILE +FROM score_board s +ORDER BY team_name, score; +---- +Mongrels Apu 350 1 +Mongrels Ned 666 1 +Mongrels Meg 1030 2 +Mongrels Burns 1270 2 +Simpsons Homer 1 1 +Simpsons Lisa 710 1 +Simpsons Marge 990 2 +Simpsons Bart 2010 2 + +query TTII +SELECT + team_name, + player, + score, + NTILE(2) OVER (ORDER BY score ASC) AS NTILE +FROM score_board s +ORDER BY score; +---- +Simpsons Homer 1 1 +Mongrels Apu 350 1 +Mongrels Ned 666 1 +Simpsons Lisa 710 1 +Simpsons Marge 990 2 +Mongrels Meg 1030 2 +Mongrels Burns 1270 2 +Simpsons Bart 2010 2 + +query TTII +SELECT + team_name, + player, + score, + NTILE(1000) OVER (PARTITION BY team_name ORDER BY score ASC) AS NTILE +FROM score_board s +ORDER BY team_name, score; +---- +Mongrels Apu 350 1 +Mongrels Ned 666 2 +Mongrels Meg 1030 3 +Mongrels Burns 1270 4 +Simpsons Homer 1 1 +Simpsons Lisa 710 2 +Simpsons Marge 990 3 +Simpsons Bart 2010 4 + +query TTII +SELECT + team_name, + player, + score, + NTILE(1) OVER (PARTITION BY team_name ORDER BY score ASC) AS NTILE +FROM score_board s +ORDER BY team_name, score; +---- +Mongrels Apu 350 1 +Mongrels Ned 666 1 +Mongrels Meg 1030 1 +Mongrels Burns 1270 1 +Simpsons Homer 1 1 +Simpsons Lisa 710 1 +Simpsons Marge 990 1 +Simpsons Bart 2010 1 + +# incorrect number of parameters for ntile +query error DataFusion error: Execution error: NTILE requires a positive integer, but finds NULL +SELECT + NTILE(NULL) OVER (PARTITION BY team_name ORDER BY score ASC) AS NTILE +FROM score_board s + +query error DataFusion error: Execution error: NTILE requires a positive integer +SELECT + NTILE(-1) OVER (PARTITION BY team_name ORDER BY score ASC) AS NTILE +FROM score_board s + +query error DataFusion error: Execution error: NTILE requires a positive integer +SELECT + NTILE(0) OVER (PARTITION BY team_name ORDER BY score ASC) AS NTILE +FROM score_board s + +statement error +SELECT + NTILE() OVER (PARTITION BY team_name ORDER BY score ASC) AS NTILE +FROM score_board s + +statement error +SELECT + NTILE(1,2) OVER (PARTITION BY team_name ORDER BY score ASC) AS NTILE +FROM score_board s + +statement error +SELECT + NTILE(1,2,3) OVER (PARTITION BY team_name ORDER BY score ASC) AS NTILE +FROM score_board s + +statement error +SELECT + NTILE(1,2,3,4) OVER (PARTITION BY team_name ORDER BY score ASC) AS NTILE +FROM score_board s + +statement ok +DROP TABLE score_board; From 8882f1bbd4254c4aeb73b758f73ed98fbaeac6aa Mon Sep 17 00:00:00 2001 From: Alex Huang Date: Fri, 1 Dec 2023 21:07:23 +0100 Subject: [PATCH 344/572] Refactor array_union function to use a generic (#8381) union_arrays function --- .../physical-expr/src/array_expressions.rs | 41 ++++++++++--------- 1 file changed, 21 insertions(+), 20 deletions(-) diff --git a/datafusion/physical-expr/src/array_expressions.rs b/datafusion/physical-expr/src/array_expressions.rs index 103a392b199d..a36f485d7ba3 100644 --- a/datafusion/physical-expr/src/array_expressions.rs +++ b/datafusion/physical-expr/src/array_expressions.rs @@ -1525,32 +1525,33 @@ pub fn array_union(args: &[ArrayRef]) -> Result { } let array1 = &args[0]; let array2 = &args[1]; + + fn union_arrays( + array1: &ArrayRef, + array2: &ArrayRef, + l_field_ref: &Arc, + r_field_ref: &Arc, + ) -> Result { + match (l_field_ref.data_type(), r_field_ref.data_type()) { + (DataType::Null, _) => Ok(array2.clone()), + (_, DataType::Null) => Ok(array1.clone()), + (_, _) => { + let list1 = array1.as_list::(); + let list2 = array2.as_list::(); + let result = union_generic_lists::(list1, list2, l_field_ref)?; + Ok(Arc::new(result)) + } + } + } + match (array1.data_type(), array2.data_type()) { (DataType::Null, _) => Ok(array2.clone()), (_, DataType::Null) => Ok(array1.clone()), (DataType::List(l_field_ref), DataType::List(r_field_ref)) => { - match (l_field_ref.data_type(), r_field_ref.data_type()) { - (DataType::Null, _) => Ok(array2.clone()), - (_, DataType::Null) => Ok(array1.clone()), - (_, _) => { - let list1 = array1.as_list::(); - let list2 = array2.as_list::(); - let result = union_generic_lists::(list1, list2, l_field_ref)?; - Ok(Arc::new(result)) - } - } + union_arrays::(array1, array2, l_field_ref, r_field_ref) } (DataType::LargeList(l_field_ref), DataType::LargeList(r_field_ref)) => { - match (l_field_ref.data_type(), r_field_ref.data_type()) { - (DataType::Null, _) => Ok(array2.clone()), - (_, DataType::Null) => Ok(array1.clone()), - (_, _) => { - let list1 = array1.as_list::(); - let list2 = array2.as_list::(); - let result = union_generic_lists::(list1, list2, l_field_ref)?; - Ok(Arc::new(result)) - } - } + union_arrays::(array1, array2, l_field_ref, r_field_ref) } _ => { internal_err!( From a6e6d3fab083839239ef81cf3a3546dd8929a541 Mon Sep 17 00:00:00 2001 From: Alex Huang Date: Fri, 1 Dec 2023 21:14:19 +0100 Subject: [PATCH 345/572] Refactor function argument handling in (#8387) ScalarFunctionDefinition --- datafusion/expr/src/expr_schema.rs | 15 ++--- datafusion/physical-expr/src/planner.rs | 66 ++++++++----------- datafusion/proto/src/logical_plan/to_proto.rs | 53 ++++++++------- 3 files changed, 58 insertions(+), 76 deletions(-) diff --git a/datafusion/expr/src/expr_schema.rs b/datafusion/expr/src/expr_schema.rs index 2795ac5f0962..e5b0185d90e0 100644 --- a/datafusion/expr/src/expr_schema.rs +++ b/datafusion/expr/src/expr_schema.rs @@ -83,13 +83,12 @@ impl ExprSchemable for Expr { Expr::Cast(Cast { data_type, .. }) | Expr::TryCast(TryCast { data_type, .. }) => Ok(data_type.clone()), Expr::ScalarFunction(ScalarFunction { func_def, args }) => { + let arg_data_types = args + .iter() + .map(|e| e.get_type(schema)) + .collect::>>()?; match func_def { ScalarFunctionDefinition::BuiltIn(fun) => { - let arg_data_types = args - .iter() - .map(|e| e.get_type(schema)) - .collect::>>()?; - // verify that input data types is consistent with function's `TypeSignature` data_types(&arg_data_types, &fun.signature()).map_err(|_| { plan_datafusion_err!( @@ -105,11 +104,7 @@ impl ExprSchemable for Expr { fun.return_type(&arg_data_types) } ScalarFunctionDefinition::UDF(fun) => { - let data_types = args - .iter() - .map(|e| e.get_type(schema)) - .collect::>>()?; - Ok(fun.return_type(&data_types)?) + Ok(fun.return_type(&arg_data_types)?) } ScalarFunctionDefinition::Name(_) => { internal_err!("Function `Expr` with name should be resolved.") diff --git a/datafusion/physical-expr/src/planner.rs b/datafusion/physical-expr/src/planner.rs index 5501647da2c3..9c212cb81f6b 100644 --- a/datafusion/physical-expr/src/planner.rs +++ b/datafusion/physical-expr/src/planner.rs @@ -348,50 +348,38 @@ pub fn create_physical_expr( ))) } - Expr::ScalarFunction(ScalarFunction { func_def, args }) => match func_def { - ScalarFunctionDefinition::BuiltIn(fun) => { - let physical_args = args - .iter() - .map(|e| { - create_physical_expr( - e, - input_dfschema, - input_schema, - execution_props, - ) - }) - .collect::>>()?; - functions::create_physical_expr( - fun, - &physical_args, - input_schema, - execution_props, - ) - } - ScalarFunctionDefinition::UDF(fun) => { - let mut physical_args = vec![]; - for e in args { - physical_args.push(create_physical_expr( - e, - input_dfschema, + Expr::ScalarFunction(ScalarFunction { func_def, args }) => { + let mut physical_args = args + .iter() + .map(|e| { + create_physical_expr(e, input_dfschema, input_schema, execution_props) + }) + .collect::>>()?; + match func_def { + ScalarFunctionDefinition::BuiltIn(fun) => { + functions::create_physical_expr( + fun, + &physical_args, input_schema, execution_props, - )?); + ) + } + ScalarFunctionDefinition::UDF(fun) => { + // udfs with zero params expect null array as input + if args.is_empty() { + physical_args.push(Arc::new(Literal::new(ScalarValue::Null))); + } + udf::create_physical_expr( + fun.clone().as_ref(), + &physical_args, + input_schema, + ) } - // udfs with zero params expect null array as input - if args.is_empty() { - physical_args.push(Arc::new(Literal::new(ScalarValue::Null))); + ScalarFunctionDefinition::Name(_) => { + internal_err!("Function `Expr` with name should be resolved.") } - udf::create_physical_expr( - fun.clone().as_ref(), - &physical_args, - input_schema, - ) } - ScalarFunctionDefinition::Name(_) => { - internal_err!("Function `Expr` with name should be resolved.") - } - }, + } Expr::Between(Between { expr, negated, diff --git a/datafusion/proto/src/logical_plan/to_proto.rs b/datafusion/proto/src/logical_plan/to_proto.rs index ab8e850014e5..ecbfaca5dbfe 100644 --- a/datafusion/proto/src/logical_plan/to_proto.rs +++ b/datafusion/proto/src/logical_plan/to_proto.rs @@ -792,40 +792,39 @@ impl TryFrom<&Expr> for protobuf::LogicalExprNode { .to_string(), )) } - Expr::ScalarFunction(ScalarFunction { func_def, args }) => match func_def { - ScalarFunctionDefinition::BuiltIn(fun) => { - let fun: protobuf::ScalarFunction = fun.try_into()?; - let args: Vec = args - .iter() - .map(|e| e.try_into()) - .collect::, Error>>()?; - Self { - expr_type: Some(ExprType::ScalarFunction( - protobuf::ScalarFunctionNode { - fun: fun.into(), + Expr::ScalarFunction(ScalarFunction { func_def, args }) => { + let args = args + .iter() + .map(|expr| expr.try_into()) + .collect::, Error>>()?; + match func_def { + ScalarFunctionDefinition::BuiltIn(fun) => { + let fun: protobuf::ScalarFunction = fun.try_into()?; + Self { + expr_type: Some(ExprType::ScalarFunction( + protobuf::ScalarFunctionNode { + fun: fun.into(), + args, + }, + )), + } + } + ScalarFunctionDefinition::UDF(fun) => Self { + expr_type: Some(ExprType::ScalarUdfExpr( + protobuf::ScalarUdfExprNode { + fun_name: fun.name().to_string(), args, }, )), - } - } - ScalarFunctionDefinition::UDF(fun) => Self { - expr_type: Some(ExprType::ScalarUdfExpr( - protobuf::ScalarUdfExprNode { - fun_name: fun.name().to_string(), - args: args - .iter() - .map(|expr| expr.try_into()) - .collect::, Error>>()?, - }, - )), - }, - ScalarFunctionDefinition::Name(_) => { - return Err(Error::NotImplemented( + }, + ScalarFunctionDefinition::Name(_) => { + return Err(Error::NotImplemented( "Proto serialization error: Trying to serialize a unresolved function" .to_string(), )); + } } - }, + } Expr::Not(expr) => { let expr = Box::new(protobuf::Not { expr: Some(Box::new(expr.as_ref().try_into()?)), From eb8aff7becaf5d4a44c723b29445deb958fbe3b4 Mon Sep 17 00:00:00 2001 From: Kirill Zaborsky Date: Sat, 2 Dec 2023 01:28:24 +0300 Subject: [PATCH 346/572] Materialize dictionaries in group keys (#7647) (#8291) Given that group keys inherently have few repeated values, especially when grouping on a single column, the use of dictionary encoding is unlikely to be yielding significant returns --- datafusion/core/tests/path_partition.rs | 15 ++------- .../src/aggregates/group_values/row.rs | 27 +++------------- .../physical-plan/src/aggregates/mod.rs | 31 +++++++++++++++++-- .../physical-plan/src/aggregates/row_hash.rs | 4 ++- .../sqllogictest/test_files/aggregate.slt | 9 ++++++ 5 files changed, 48 insertions(+), 38 deletions(-) diff --git a/datafusion/core/tests/path_partition.rs b/datafusion/core/tests/path_partition.rs index dd8eb52f67c7..abe6ab283aff 100644 --- a/datafusion/core/tests/path_partition.rs +++ b/datafusion/core/tests/path_partition.rs @@ -168,9 +168,9 @@ async fn parquet_distinct_partition_col() -> Result<()> { assert_eq!(min_limit, resulting_limit); let s = ScalarValue::try_from_array(results[0].column(1), 0)?; - let month = match extract_as_utf(&s) { - Some(month) => month, - s => panic!("Expected month as Dict(_, Utf8) found {s:?}"), + let month = match s { + ScalarValue::Utf8(Some(month)) => month, + s => panic!("Expected month as Utf8 found {s:?}"), }; let sql_on_partition_boundary = format!( @@ -191,15 +191,6 @@ async fn parquet_distinct_partition_col() -> Result<()> { Ok(()) } -fn extract_as_utf(v: &ScalarValue) -> Option { - if let ScalarValue::Dictionary(_, v) = v { - if let ScalarValue::Utf8(v) = v.as_ref() { - return v.clone(); - } - } - None -} - #[tokio::test] async fn csv_filter_with_file_col() -> Result<()> { let ctx = SessionContext::new(); diff --git a/datafusion/physical-plan/src/aggregates/group_values/row.rs b/datafusion/physical-plan/src/aggregates/group_values/row.rs index 10ff9edb8912..e7c7a42cf902 100644 --- a/datafusion/physical-plan/src/aggregates/group_values/row.rs +++ b/datafusion/physical-plan/src/aggregates/group_values/row.rs @@ -17,22 +17,18 @@ use crate::aggregates::group_values::GroupValues; use ahash::RandomState; -use arrow::compute::cast; use arrow::record_batch::RecordBatch; use arrow::row::{RowConverter, Rows, SortField}; -use arrow_array::{Array, ArrayRef}; -use arrow_schema::{DataType, SchemaRef}; +use arrow_array::ArrayRef; +use arrow_schema::SchemaRef; use datafusion_common::hash_utils::create_hashes; -use datafusion_common::{DataFusionError, Result}; +use datafusion_common::Result; use datafusion_execution::memory_pool::proxy::{RawTableAllocExt, VecAllocExt}; use datafusion_physical_expr::EmitTo; use hashbrown::raw::RawTable; /// A [`GroupValues`] making use of [`Rows`] pub struct GroupValuesRows { - /// The output schema - schema: SchemaRef, - /// Converter for the group values row_converter: RowConverter, @@ -79,7 +75,6 @@ impl GroupValuesRows { let map = RawTable::with_capacity(0); Ok(Self { - schema, row_converter, map, map_size: 0, @@ -170,7 +165,7 @@ impl GroupValues for GroupValuesRows { .take() .expect("Can not emit from empty rows"); - let mut output = match emit_to { + let output = match emit_to { EmitTo::All => { let output = self.row_converter.convert_rows(&group_values)?; group_values.clear(); @@ -203,20 +198,6 @@ impl GroupValues for GroupValuesRows { } }; - // TODO: Materialize dictionaries in group keys (#7647) - for (field, array) in self.schema.fields.iter().zip(&mut output) { - let expected = field.data_type(); - if let DataType::Dictionary(_, v) = expected { - let actual = array.data_type(); - if v.as_ref() != actual { - return Err(DataFusionError::Internal(format!( - "Converted group rows expected dictionary of {v} got {actual}" - ))); - } - *array = cast(array.as_ref(), expected)?; - } - } - self.group_values = Some(group_values); Ok(output) } diff --git a/datafusion/physical-plan/src/aggregates/mod.rs b/datafusion/physical-plan/src/aggregates/mod.rs index 7d7fba6ef6c3..d594335af44f 100644 --- a/datafusion/physical-plan/src/aggregates/mod.rs +++ b/datafusion/physical-plan/src/aggregates/mod.rs @@ -38,6 +38,7 @@ use crate::{ use arrow::array::ArrayRef; use arrow::datatypes::{Field, Schema, SchemaRef}; use arrow::record_batch::RecordBatch; +use arrow_schema::DataType; use datafusion_common::stats::Precision; use datafusion_common::{not_impl_err, plan_err, DataFusionError, Result}; use datafusion_execution::TaskContext; @@ -286,6 +287,9 @@ pub struct AggregateExec { limit: Option, /// Input plan, could be a partial aggregate or the input to the aggregate pub input: Arc, + /// Original aggregation schema, could be different from `schema` before dictionary group + /// keys get materialized + original_schema: SchemaRef, /// Schema after the aggregate is applied schema: SchemaRef, /// Input schema before any aggregation is applied. For partial aggregate this will be the @@ -469,7 +473,7 @@ impl AggregateExec { input: Arc, input_schema: SchemaRef, ) -> Result { - let schema = create_schema( + let original_schema = create_schema( &input.schema(), &group_by.expr, &aggr_expr, @@ -477,7 +481,11 @@ impl AggregateExec { mode, )?; - let schema = Arc::new(schema); + let schema = Arc::new(materialize_dict_group_keys( + &original_schema, + group_by.expr.len(), + )); + let original_schema = Arc::new(original_schema); // Reset ordering requirement to `None` if aggregator is not order-sensitive order_by_expr = aggr_expr .iter() @@ -552,6 +560,7 @@ impl AggregateExec { filter_expr, order_by_expr, input, + original_schema, schema, input_schema, projection_mapping, @@ -973,6 +982,24 @@ fn create_schema( Ok(Schema::new(fields)) } +/// returns schema with dictionary group keys materialized as their value types +/// The actual convertion happens in `RowConverter` and we don't do unnecessary +/// conversion back into dictionaries +fn materialize_dict_group_keys(schema: &Schema, group_count: usize) -> Schema { + let fields = schema + .fields + .iter() + .enumerate() + .map(|(i, field)| match field.data_type() { + DataType::Dictionary(_, value_data_type) if i < group_count => { + Field::new(field.name(), *value_data_type.clone(), field.is_nullable()) + } + _ => Field::clone(field), + }) + .collect::>(); + Schema::new(fields) +} + fn group_schema(schema: &Schema, group_count: usize) -> SchemaRef { let group_fields = schema.fields()[0..group_count].to_vec(); Arc::new(Schema::new(group_fields)) diff --git a/datafusion/physical-plan/src/aggregates/row_hash.rs b/datafusion/physical-plan/src/aggregates/row_hash.rs index f96417fc323b..2f94c3630c33 100644 --- a/datafusion/physical-plan/src/aggregates/row_hash.rs +++ b/datafusion/physical-plan/src/aggregates/row_hash.rs @@ -324,7 +324,9 @@ impl GroupedHashAggregateStream { .map(create_group_accumulator) .collect::>()?; - let group_schema = group_schema(&agg_schema, agg_group_by.expr.len()); + // we need to use original schema so RowConverter in group_values below + // will do the proper coversion of dictionaries into value types + let group_schema = group_schema(&agg.original_schema, agg_group_by.expr.len()); let spill_expr = group_schema .fields .into_iter() diff --git a/datafusion/sqllogictest/test_files/aggregate.slt b/datafusion/sqllogictest/test_files/aggregate.slt index 88590055484f..e4718035a58d 100644 --- a/datafusion/sqllogictest/test_files/aggregate.slt +++ b/datafusion/sqllogictest/test_files/aggregate.slt @@ -2421,6 +2421,15 @@ select max(x_dict) from value_dict group by x_dict % 2 order by max(x_dict); 4 5 +query T +select arrow_typeof(x_dict) from value_dict group by x_dict; +---- +Int32 +Int32 +Int32 +Int32 +Int32 + statement ok drop table value From f5d10e55d575e1eec58b993cab2d8a7ca2370ff9 Mon Sep 17 00:00:00 2001 From: Jay Zhan Date: Sat, 2 Dec 2023 06:28:32 +0800 Subject: [PATCH 347/572] Rewrite `array_ndims` to fix List(Null) handling (#8320) * done Signed-off-by: jayzhan211 * add more test Signed-off-by: jayzhan211 * cleanup Signed-off-by: jayzhan211 --------- Signed-off-by: jayzhan211 Co-authored-by: Andrew Lamb --- datafusion/common/src/utils.rs | 32 ++++++++ .../physical-expr/src/array_expressions.rs | 76 +++++++------------ datafusion/sqllogictest/test_files/array.slt | 42 +++++++++- 3 files changed, 97 insertions(+), 53 deletions(-) diff --git a/datafusion/common/src/utils.rs b/datafusion/common/src/utils.rs index 12d4f516b4d0..7f2dc61c07bf 100644 --- a/datafusion/common/src/utils.rs +++ b/datafusion/common/src/utils.rs @@ -26,6 +26,7 @@ use arrow::compute::{partition, SortColumn, SortOptions}; use arrow::datatypes::{Field, SchemaRef, UInt32Type}; use arrow::record_batch::RecordBatch; use arrow_array::{Array, LargeListArray, ListArray}; +use arrow_schema::DataType; use sqlparser::ast::Ident; use sqlparser::dialect::GenericDialect; use sqlparser::parser::Parser; @@ -402,6 +403,37 @@ pub fn arrays_into_list_array( )) } +/// Get the base type of a data type. +/// +/// Example +/// ``` +/// use arrow::datatypes::{DataType, Field}; +/// use datafusion_common::utils::base_type; +/// use std::sync::Arc; +/// +/// let data_type = DataType::List(Arc::new(Field::new("item", DataType::Int32, true))); +/// assert_eq!(base_type(&data_type), DataType::Int32); +/// +/// let data_type = DataType::Int32; +/// assert_eq!(base_type(&data_type), DataType::Int32); +/// ``` +pub fn base_type(data_type: &DataType) -> DataType { + if let DataType::List(field) = data_type { + base_type(field.data_type()) + } else { + data_type.to_owned() + } +} + +/// Compute the number of dimensions in a list data type. +pub fn list_ndims(data_type: &DataType) -> u64 { + if let DataType::List(field) = data_type { + 1 + list_ndims(field.data_type()) + } else { + 0 + } +} + /// An extension trait for smart pointers. Provides an interface to get a /// raw pointer to the data (with metadata stripped away). /// diff --git a/datafusion/physical-expr/src/array_expressions.rs b/datafusion/physical-expr/src/array_expressions.rs index a36f485d7ba3..7059c6a9f37f 100644 --- a/datafusion/physical-expr/src/array_expressions.rs +++ b/datafusion/physical-expr/src/array_expressions.rs @@ -33,7 +33,7 @@ use datafusion_common::cast::{ as_generic_list_array, as_generic_string_array, as_int64_array, as_list_array, as_null_array, as_string_array, }; -use datafusion_common::utils::array_into_list_array; +use datafusion_common::utils::{array_into_list_array, list_ndims}; use datafusion_common::{ exec_err, internal_datafusion_err, internal_err, not_impl_err, plan_err, DataFusionError, Result, @@ -103,6 +103,7 @@ fn compare_element_to_list( ) -> Result { let indices = UInt32Array::from(vec![row_index as u32]); let element_array_row = arrow::compute::take(element_array, &indices, None)?; + // Compute all positions in list_row_array (that is itself an // array) that are equal to `from_array_row` let res = match element_array_row.data_type() { @@ -176,35 +177,6 @@ fn compute_array_length( } } -/// Returns the dimension of the array -fn compute_array_ndims(arr: Option) -> Result> { - Ok(compute_array_ndims_with_datatype(arr)?.0) -} - -/// Returns the dimension and the datatype of elements of the array -fn compute_array_ndims_with_datatype( - arr: Option, -) -> Result<(Option, DataType)> { - let mut res: u64 = 1; - let mut value = match arr { - Some(arr) => arr, - None => return Ok((None, DataType::Null)), - }; - if value.is_empty() { - return Ok((None, DataType::Null)); - } - - loop { - match value.data_type() { - DataType::List(..) => { - value = downcast_arg!(value, ListArray).value(0); - res += 1; - } - data_type => return Ok((Some(res), data_type.clone())), - } - } -} - /// Returns the length of each array dimension fn compute_array_dims(arr: Option) -> Result>>> { let mut value = match arr { @@ -825,10 +797,7 @@ pub fn array_prepend(args: &[ArrayRef]) -> Result { fn align_array_dimensions(args: Vec) -> Result> { let args_ndim = args .iter() - .map(|arg| compute_array_ndims(Some(arg.to_owned()))) - .collect::>>()? - .into_iter() - .map(|x| x.unwrap_or(0)) + .map(|arg| datafusion_common::utils::list_ndims(arg.data_type())) .collect::>(); let max_ndim = args_ndim.iter().max().unwrap_or(&0); @@ -919,6 +888,7 @@ fn concat_internal(args: &[ArrayRef]) -> Result { Arc::new(compute::concat(elements.as_slice())?), Some(NullBuffer::new(buffer)), ); + Ok(Arc::new(list_arr)) } @@ -926,11 +896,11 @@ fn concat_internal(args: &[ArrayRef]) -> Result { pub fn array_concat(args: &[ArrayRef]) -> Result { let mut new_args = vec![]; for arg in args { - let (ndim, lower_data_type) = - compute_array_ndims_with_datatype(Some(arg.clone()))?; - if ndim.is_none() || ndim == Some(1) { - return not_impl_err!("Array is not type '{lower_data_type:?}'."); - } else if !lower_data_type.equals_datatype(&DataType::Null) { + let ndim = list_ndims(arg.data_type()); + let base_type = datafusion_common::utils::base_type(arg.data_type()); + if ndim == 0 { + return not_impl_err!("Array is not type '{base_type:?}'."); + } else if !base_type.eq(&DataType::Null) { new_args.push(arg.clone()); } } @@ -1765,14 +1735,22 @@ pub fn array_dims(args: &[ArrayRef]) -> Result { /// Array_ndims SQL function pub fn array_ndims(args: &[ArrayRef]) -> Result { - let list_array = as_list_array(&args[0])?; + if let Some(list_array) = args[0].as_list_opt::() { + let ndims = datafusion_common::utils::list_ndims(list_array.data_type()); - let result = list_array - .iter() - .map(compute_array_ndims) - .collect::>()?; + let mut data = vec![]; + for arr in list_array.iter() { + if arr.is_some() { + data.push(Some(ndims)) + } else { + data.push(None) + } + } - Ok(Arc::new(result) as ArrayRef) + Ok(Arc::new(UInt64Array::from(data)) as ArrayRef) + } else { + Ok(Arc::new(UInt64Array::from(vec![0; args[0].len()])) as ArrayRef) + } } /// Array_has SQL function @@ -2034,10 +2012,10 @@ mod tests { .unwrap(); let expected = as_list_array(&array2d_1).unwrap(); - let expected_dim = compute_array_ndims(Some(array2d_1.to_owned())).unwrap(); + let expected_dim = datafusion_common::utils::list_ndims(array2d_1.data_type()); assert_ne!(as_list_array(&res[0]).unwrap(), expected); assert_eq!( - compute_array_ndims(Some(res[0].clone())).unwrap(), + datafusion_common::utils::list_ndims(res[0].data_type()), expected_dim ); @@ -2047,10 +2025,10 @@ mod tests { align_array_dimensions(vec![array1d_1, Arc::new(array3d_2.clone())]).unwrap(); let expected = as_list_array(&array3d_1).unwrap(); - let expected_dim = compute_array_ndims(Some(array3d_1.to_owned())).unwrap(); + let expected_dim = datafusion_common::utils::list_ndims(array3d_1.data_type()); assert_ne!(as_list_array(&res[0]).unwrap(), expected); assert_eq!( - compute_array_ndims(Some(res[0].clone())).unwrap(), + datafusion_common::utils::list_ndims(res[0].data_type()), expected_dim ); } diff --git a/datafusion/sqllogictest/test_files/array.slt b/datafusion/sqllogictest/test_files/array.slt index 3b45d995e1a2..092bc697a197 100644 --- a/datafusion/sqllogictest/test_files/array.slt +++ b/datafusion/sqllogictest/test_files/array.slt @@ -2479,10 +2479,44 @@ NULL [3] [4] ## array_ndims (aliases: `list_ndims`) # array_ndims scalar function #1 + query III -select array_ndims(make_array(1, 2, 3)), array_ndims(make_array([1, 2], [3, 4])), array_ndims(make_array([[[[1], [2]]]])); +select + array_ndims(1), + array_ndims(null), + array_ndims([2, 3]); ---- -1 2 5 +0 0 1 + +statement ok +CREATE TABLE array_ndims_table +AS VALUES + (1, [1, 2, 3], [[7]], [[[[[10]]]]]), + (2, [4, 5], [[8]], [[[[[10]]]]]), + (null, [6], [[9]], [[[[[10]]]]]), + (3, [6], [[9]], [[[[[10]]]]]) +; + +query IIII +select + array_ndims(column1), + array_ndims(column2), + array_ndims(column3), + array_ndims(column4) +from array_ndims_table; +---- +0 1 2 5 +0 1 2 5 +0 1 2 5 +0 1 2 5 + +statement ok +drop table array_ndims_table; + +query I +select array_ndims(arrow_cast([null], 'List(List(List(Int64)))')); +---- +3 # array_ndims scalar function #2 query II @@ -2494,7 +2528,7 @@ select array_ndims(array_repeat(array_repeat(array_repeat(1, 3), 2), 1)), array_ query II select array_ndims(make_array()), array_ndims(make_array(make_array())) ---- -NULL 2 +1 2 # list_ndims scalar function #4 (function alias `array_ndims`) query III @@ -2505,7 +2539,7 @@ select list_ndims(make_array(1, 2, 3)), list_ndims(make_array([1, 2], [3, 4])), query II select array_ndims(make_array()), array_ndims(make_array(make_array())) ---- -NULL 2 +1 2 # array_ndims with columns query III From 3b298374f9706fd15e21744b3ffa00ae9e100377 Mon Sep 17 00:00:00 2001 From: Andrew Lamb Date: Sat, 2 Dec 2023 04:17:18 -0500 Subject: [PATCH 348/572] Docs: Improve the documentation on `ScalarValue` (#8378) * Minor: Improve the documentation on `ScalarValue` * Update datafusion/common/src/scalar.rs Co-authored-by: Liang-Chi Hsieh * Update datafusion/common/src/scalar.rs Co-authored-by: Liang-Chi Hsieh --------- Co-authored-by: Liang-Chi Hsieh --- datafusion/common/src/scalar.rs | 47 +++++++++++++++++++++++++++++++-- 1 file changed, 45 insertions(+), 2 deletions(-) diff --git a/datafusion/common/src/scalar.rs b/datafusion/common/src/scalar.rs index 3431d71468ea..ef0edbd9e09f 100644 --- a/datafusion/common/src/scalar.rs +++ b/datafusion/common/src/scalar.rs @@ -50,9 +50,52 @@ use arrow::{ use arrow_array::cast::as_list_array; use arrow_array::{ArrowNativeTypeOp, Scalar}; -/// Represents a dynamically typed, nullable single value. -/// This is the single-valued counter-part to arrow's [`Array`]. +/// A dynamically typed, nullable single value, (the single-valued counter-part +/// to arrow's [`Array`]) /// +/// # Performance +/// +/// In general, please use arrow [`Array`]s rather than [`ScalarValue`] whenever +/// possible, as it is far more efficient for multiple values. +/// +/// # Example +/// ``` +/// # use datafusion_common::ScalarValue; +/// // Create single scalar value for an Int32 value +/// let s1 = ScalarValue::Int32(Some(10)); +/// +/// // You can also create values using the From impl: +/// let s2 = ScalarValue::from(10i32); +/// assert_eq!(s1, s2); +/// ``` +/// +/// # Null Handling +/// +/// `ScalarValue` represents null values in the same way as Arrow. Nulls are +/// "typed" in the sense that a null value in an [`Int32Array`] is different +/// than a null value in a [`Float64Array`], and is different than the values in +/// a [`NullArray`]. +/// +/// ``` +/// # fn main() -> datafusion_common::Result<()> { +/// # use std::collections::hash_set::Difference; +/// # use datafusion_common::ScalarValue; +/// # use arrow::datatypes::DataType; +/// // You can create a 'null' Int32 value directly: +/// let s1 = ScalarValue::Int32(None); +/// +/// // You can also create a null value for a given datatype: +/// let s2 = ScalarValue::try_from(&DataType::Int32)?; +/// assert_eq!(s1, s2); +/// +/// // Note that this is DIFFERENT than a `ScalarValue::Null` +/// let s3 = ScalarValue::Null; +/// assert_ne!(s1, s3); +/// # Ok(()) +/// # } +/// ``` +/// +/// # Further Reading /// See [datatypes](https://arrow.apache.org/docs/python/api/datatypes.html) for /// details on datatypes and the [format](https://github.com/apache/arrow/blob/master/format/Schema.fbs#L354-L375) /// for the definitive reference. From 340ecfdfe0e6667d2c9f528a60d8ee7fa5c34805 Mon Sep 17 00:00:00 2001 From: Jay Zhan Date: Sat, 2 Dec 2023 17:19:17 +0800 Subject: [PATCH 349/572] Avoid concat for `array_replace` (#8337) * add benchmark Signed-off-by: jayzhan211 * fmt Signed-off-by: jayzhan211 * address clippy Signed-off-by: jayzhan211 * cleanup Signed-off-by: jayzhan211 * fix comment Signed-off-by: jayzhan211 --------- Signed-off-by: jayzhan211 Co-authored-by: Andrew Lamb --- datafusion/core/Cargo.toml | 4 + datafusion/core/benches/array_expression.rs | 73 ++++++++++ .../physical-expr/src/array_expressions.rs | 125 ++++++++---------- 3 files changed, 135 insertions(+), 67 deletions(-) create mode 100644 datafusion/core/benches/array_expression.rs diff --git a/datafusion/core/Cargo.toml b/datafusion/core/Cargo.toml index 0b7aa1509820..7caf91e24f2f 100644 --- a/datafusion/core/Cargo.toml +++ b/datafusion/core/Cargo.toml @@ -167,3 +167,7 @@ name = "sort" [[bench]] harness = false name = "topk_aggregate" + +[[bench]] +harness = false +name = "array_expression" diff --git a/datafusion/core/benches/array_expression.rs b/datafusion/core/benches/array_expression.rs new file mode 100644 index 000000000000..95bc93e0e353 --- /dev/null +++ b/datafusion/core/benches/array_expression.rs @@ -0,0 +1,73 @@ +// 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. + +#[macro_use] +extern crate criterion; +extern crate arrow; +extern crate datafusion; + +mod data_utils; +use crate::criterion::Criterion; +use arrow_array::cast::AsArray; +use arrow_array::types::Int64Type; +use arrow_array::{ArrayRef, Int64Array, ListArray}; +use datafusion_physical_expr::array_expressions; +use std::sync::Arc; + +fn criterion_benchmark(c: &mut Criterion) { + // Construct large arrays for benchmarking + + let array_len = 100000000; + + let array = (0..array_len).map(|_| Some(2_i64)).collect::>(); + let list_array = ListArray::from_iter_primitive::(vec![ + Some(array.clone()), + Some(array.clone()), + Some(array), + ]); + let from_array = Int64Array::from_value(2, 3); + let to_array = Int64Array::from_value(-2, 3); + + let args = vec![ + Arc::new(list_array) as ArrayRef, + Arc::new(from_array) as ArrayRef, + Arc::new(to_array) as ArrayRef, + ]; + + let array = (0..array_len).map(|_| Some(-2_i64)).collect::>(); + let expected_array = ListArray::from_iter_primitive::(vec![ + Some(array.clone()), + Some(array.clone()), + Some(array), + ]); + + // Benchmark array functions + + c.bench_function("array_replace", |b| { + b.iter(|| { + assert_eq!( + array_expressions::array_replace_all(args.as_slice()) + .unwrap() + .as_list::(), + criterion::black_box(&expected_array) + ) + }) + }); +} + +criterion_group!(benches, criterion_benchmark); +criterion_main!(benches); diff --git a/datafusion/physical-expr/src/array_expressions.rs b/datafusion/physical-expr/src/array_expressions.rs index 7059c6a9f37f..84dfe3b9ff75 100644 --- a/datafusion/physical-expr/src/array_expressions.rs +++ b/datafusion/physical-expr/src/array_expressions.rs @@ -35,8 +35,7 @@ use datafusion_common::cast::{ }; use datafusion_common::utils::{array_into_list_array, list_ndims}; use datafusion_common::{ - exec_err, internal_datafusion_err, internal_err, not_impl_err, plan_err, - DataFusionError, Result, + exec_err, internal_err, not_impl_err, plan_err, DataFusionError, Result, }; use itertools::Itertools; @@ -1320,84 +1319,76 @@ fn general_replace( ) -> Result { // Build up the offsets for the final output array let mut offsets: Vec = vec![0]; - let data_type = list_array.value_type(); - let mut new_values = vec![]; + let values = list_array.values(); + let original_data = values.to_data(); + let to_data = to_array.to_data(); + let capacity = Capacities::Array(original_data.len()); - // n is the number of elements to replace in this row - for (row_index, (list_array_row, n)) in - list_array.iter().zip(arr_n.iter()).enumerate() - { - let last_offset: i32 = offsets - .last() - .copied() - .ok_or_else(|| internal_datafusion_err!("offsets should not be empty"))?; + // First array is the original array, second array is the element to replace with. + let mut mutable = MutableArrayData::with_capacities( + vec![&original_data, &to_data], + false, + capacity, + ); - match list_array_row { - Some(list_array_row) => { - // Compute all positions in list_row_array (that is itself an - // array) that are equal to `from_array_row` - let eq_array = compare_element_to_list( - &list_array_row, - &from_array, - row_index, - true, - )?; + let mut valid = BooleanBufferBuilder::new(list_array.len()); - // Use MutableArrayData to build the replaced array - let original_data = list_array_row.to_data(); - let to_data = to_array.to_data(); - let capacity = Capacities::Array(original_data.len() + to_data.len()); + for (row_index, offset_window) in list_array.offsets().windows(2).enumerate() { + if list_array.is_null(row_index) { + offsets.push(offsets[row_index]); + valid.append(false); + continue; + } - // First array is the original array, second array is the element to replace with. - let mut mutable = MutableArrayData::with_capacities( - vec![&original_data, &to_data], - false, - capacity, - ); - let original_idx = 0; - let replace_idx = 1; - - let mut counter = 0; - for (i, to_replace) in eq_array.iter().enumerate() { - if let Some(true) = to_replace { - mutable.extend(replace_idx, row_index, row_index + 1); - counter += 1; - if counter == *n { - // copy original data for any matches past n - mutable.extend(original_idx, i + 1, eq_array.len()); - break; - } - } else { - // copy original data for false / null matches - mutable.extend(original_idx, i, i + 1); - } - } + let start = offset_window[0] as usize; + let end = offset_window[1] as usize; - let data = mutable.freeze(); - let replaced_array = arrow_array::make_array(data); + let list_array_row = list_array.value(row_index); - offsets.push(last_offset + replaced_array.len() as i32); - new_values.push(replaced_array); - } - None => { - // Null element results in a null row (no new offsets) - offsets.push(last_offset); + // Compute all positions in list_row_array (that is itself an + // array) that are equal to `from_array_row` + let eq_array = + compare_element_to_list(&list_array_row, &from_array, row_index, true)?; + + let original_idx = 0; + let replace_idx = 1; + let n = arr_n[row_index]; + let mut counter = 0; + + // All elements are false, no need to replace, just copy original data + if eq_array.false_count() == eq_array.len() { + mutable.extend(original_idx, start, end); + offsets.push(offsets[row_index] + (end - start) as i32); + valid.append(true); + continue; + } + + for (i, to_replace) in eq_array.iter().enumerate() { + if let Some(true) = to_replace { + mutable.extend(replace_idx, row_index, row_index + 1); + counter += 1; + if counter == n { + // copy original data for any matches past n + mutable.extend(original_idx, start + i + 1, end); + break; + } + } else { + // copy original data for false / null matches + mutable.extend(original_idx, start + i, start + i + 1); } } + + offsets.push(offsets[row_index] + (end - start) as i32); + valid.append(true); } - let values = if new_values.is_empty() { - new_empty_array(&data_type) - } else { - let new_values: Vec<_> = new_values.iter().map(|a| a.as_ref()).collect(); - arrow::compute::concat(&new_values)? - }; + let data = mutable.freeze(); Ok(Arc::new(ListArray::try_new( - Arc::new(Field::new("item", data_type, true)), + Arc::new(Field::new("item", list_array.value_type(), true)), OffsetBuffer::new(offsets.into()), - values, - list_array.nulls().cloned(), + arrow_array::make_array(data), + Some(NullBuffer::new(valid.finish())), )?)) } From bb2ea4b7830556b072a09bc17f85ea7d90553b73 Mon Sep 17 00:00:00 2001 From: Mohammad Razeghi Date: Sat, 2 Dec 2023 18:03:53 +0100 Subject: [PATCH 350/572] add a summary table to benchmark compare output (#8399) --- benchmarks/compare.py | 39 ++++++++++++++++++++++++++++++++++----- 1 file changed, 34 insertions(+), 5 deletions(-) diff --git a/benchmarks/compare.py b/benchmarks/compare.py index 80aa3c76b754..ec2b28fa0556 100755 --- a/benchmarks/compare.py +++ b/benchmarks/compare.py @@ -109,7 +109,6 @@ def compare( noise_threshold: float, ) -> None: baseline = BenchmarkRun.load_from_file(baseline_path) - comparison = BenchmarkRun.load_from_file(comparison_path) console = Console() @@ -124,27 +123,57 @@ def compare( table.add_column(comparison_header, justify="right", style="dim") table.add_column("Change", justify="right", style="dim") + faster_count = 0 + slower_count = 0 + no_change_count = 0 + total_baseline_time = 0 + total_comparison_time = 0 + for baseline_result, comparison_result in zip(baseline.queries, comparison.queries): assert baseline_result.query == comparison_result.query + total_baseline_time += baseline_result.execution_time + total_comparison_time += comparison_result.execution_time + change = comparison_result.execution_time / baseline_result.execution_time if (1.0 - noise_threshold) <= change <= (1.0 + noise_threshold): - change = "no change" + change_text = "no change" + no_change_count += 1 elif change < 1.0: - change = f"+{(1 / change):.2f}x faster" + change_text = f"+{(1 / change):.2f}x faster" + faster_count += 1 else: - change = f"{change:.2f}x slower" + change_text = f"{change:.2f}x slower" + slower_count += 1 table.add_row( f"Q{baseline_result.query}", f"{baseline_result.execution_time:.2f}ms", f"{comparison_result.execution_time:.2f}ms", - change, + change_text, ) console.print(table) + # Calculate averages + avg_baseline_time = total_baseline_time / len(baseline.queries) + avg_comparison_time = total_comparison_time / len(comparison.queries) + + # Summary table + summary_table = Table(show_header=True, header_style="bold magenta") + summary_table.add_column("Benchmark Summary", justify="left", style="dim") + summary_table.add_column("", justify="right", style="dim") + + summary_table.add_row(f"Total Time ({baseline_header})", f"{total_baseline_time:.2f}ms") + summary_table.add_row(f"Total Time ({comparison_header})", f"{total_comparison_time:.2f}ms") + summary_table.add_row(f"Average Time ({baseline_header})", f"{avg_baseline_time:.2f}ms") + summary_table.add_row(f"Average Time ({comparison_header})", f"{avg_comparison_time:.2f}ms") + summary_table.add_row("Queries Faster", str(faster_count)) + summary_table.add_row("Queries Slower", str(slower_count)) + summary_table.add_row("Queries with No Change", str(no_change_count)) + + console.print(summary_table) def main() -> None: parser = ArgumentParser() From 075ff3ddfc78680d5da424ed63ffea1e38a6c57d Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Berkay=20=C5=9Eahin?= <124376117+berkaysynnada@users.noreply.github.com> Date: Sun, 3 Dec 2023 01:44:18 +0300 Subject: [PATCH 351/572] Refactors on TreeNode Implementations (#8395) * minor changes * PipelineStatePropagator tree refactor * Remove duplications by children_unbounded() * Remove on-the-fly tree construction * Minor changes --------- Co-authored-by: Mustafa Akur --- .../src/physical_optimizer/join_selection.rs | 21 +++++-- .../physical_optimizer/pipeline_checker.rs | 40 ++++++------- datafusion/physical-expr/src/equivalence.rs | 2 +- .../physical-expr/src/sort_properties.rs | 58 +++++++------------ datafusion/physical-expr/src/utils.rs | 15 ++--- 5 files changed, 65 insertions(+), 71 deletions(-) diff --git a/datafusion/core/src/physical_optimizer/join_selection.rs b/datafusion/core/src/physical_optimizer/join_selection.rs index a7ecd1ca655c..0c3ac2d24529 100644 --- a/datafusion/core/src/physical_optimizer/join_selection.rs +++ b/datafusion/core/src/physical_optimizer/join_selection.rs @@ -434,7 +434,7 @@ fn hash_join_convert_symmetric_subrule( config_options: &ConfigOptions, ) -> Option> { if let Some(hash_join) = input.plan.as_any().downcast_ref::() { - let ub_flags = &input.children_unbounded; + let ub_flags = input.children_unbounded(); let (left_unbounded, right_unbounded) = (ub_flags[0], ub_flags[1]); input.unbounded = left_unbounded || right_unbounded; let result = if left_unbounded && right_unbounded { @@ -511,7 +511,7 @@ fn hash_join_swap_subrule( _config_options: &ConfigOptions, ) -> Option> { if let Some(hash_join) = input.plan.as_any().downcast_ref::() { - let ub_flags = &input.children_unbounded; + let ub_flags = input.children_unbounded(); let (left_unbounded, right_unbounded) = (ub_flags[0], ub_flags[1]); input.unbounded = left_unbounded || right_unbounded; let result = if left_unbounded @@ -577,7 +577,7 @@ fn apply_subrules( } let is_unbounded = input .plan - .unbounded_output(&input.children_unbounded) + .unbounded_output(&input.children_unbounded()) // Treat the case where an operator can not run on unbounded data as // if it can and it outputs unbounded data. Do not raise an error yet. // Such operators may be fixed, adjusted or replaced later on during @@ -1253,6 +1253,7 @@ mod hash_join_tests { use arrow::record_batch::RecordBatch; use datafusion_common::utils::DataPtr; use datafusion_common::JoinType; + use datafusion_physical_plan::empty::EmptyExec; use std::sync::Arc; struct TestCase { @@ -1620,10 +1621,22 @@ mod hash_join_tests { false, )?; + let children = vec![ + PipelineStatePropagator { + plan: Arc::new(EmptyExec::new(false, Arc::new(Schema::empty()))), + unbounded: left_unbounded, + children: vec![], + }, + PipelineStatePropagator { + plan: Arc::new(EmptyExec::new(false, Arc::new(Schema::empty()))), + unbounded: right_unbounded, + children: vec![], + }, + ]; let initial_hash_join_state = PipelineStatePropagator { plan: Arc::new(join), unbounded: false, - children_unbounded: vec![left_unbounded, right_unbounded], + children, }; let optimized_hash_join = diff --git a/datafusion/core/src/physical_optimizer/pipeline_checker.rs b/datafusion/core/src/physical_optimizer/pipeline_checker.rs index 43ae7dbfe7b6..d59248aadf05 100644 --- a/datafusion/core/src/physical_optimizer/pipeline_checker.rs +++ b/datafusion/core/src/physical_optimizer/pipeline_checker.rs @@ -70,19 +70,27 @@ impl PhysicalOptimizerRule for PipelineChecker { pub struct PipelineStatePropagator { pub(crate) plan: Arc, pub(crate) unbounded: bool, - pub(crate) children_unbounded: Vec, + pub(crate) children: Vec, } impl PipelineStatePropagator { /// Constructs a new, default pipelining state. pub fn new(plan: Arc) -> Self { - let length = plan.children().len(); + let children = plan.children(); PipelineStatePropagator { plan, unbounded: false, - children_unbounded: vec![false; length], + children: children.into_iter().map(Self::new).collect(), } } + + /// Returns the children unboundedness information. + pub fn children_unbounded(&self) -> Vec { + self.children + .iter() + .map(|c| c.unbounded) + .collect::>() + } } impl TreeNode for PipelineStatePropagator { @@ -90,9 +98,8 @@ impl TreeNode for PipelineStatePropagator { where F: FnMut(&Self) -> Result, { - let children = self.plan.children(); - for child in children { - match op(&PipelineStatePropagator::new(child))? { + for child in &self.children { + match op(child)? { VisitRecursion::Continue => {} VisitRecursion::Skip => return Ok(VisitRecursion::Continue), VisitRecursion::Stop => return Ok(VisitRecursion::Stop), @@ -106,25 +113,18 @@ impl TreeNode for PipelineStatePropagator { where F: FnMut(Self) -> Result, { - let children = self.plan.children(); - if !children.is_empty() { - let new_children = children + if !self.children.is_empty() { + let new_children = self + .children .into_iter() - .map(PipelineStatePropagator::new) .map(transform) .collect::>>()?; - let children_unbounded = new_children - .iter() - .map(|c| c.unbounded) - .collect::>(); - let children_plans = new_children - .into_iter() - .map(|child| child.plan) - .collect::>(); + let children_plans = new_children.iter().map(|c| c.plan.clone()).collect(); + Ok(PipelineStatePropagator { plan: with_new_children_if_necessary(self.plan, children_plans)?.into(), unbounded: self.unbounded, - children_unbounded, + children: new_children, }) } else { Ok(self) @@ -149,7 +149,7 @@ pub fn check_finiteness_requirements( } input .plan - .unbounded_output(&input.children_unbounded) + .unbounded_output(&input.children_unbounded()) .map(|value| { input.unbounded = value; Transformed::Yes(input) diff --git a/datafusion/physical-expr/src/equivalence.rs b/datafusion/physical-expr/src/equivalence.rs index f9f03300f5e9..4a562f4ef101 100644 --- a/datafusion/physical-expr/src/equivalence.rs +++ b/datafusion/physical-expr/src/equivalence.rs @@ -1520,7 +1520,7 @@ fn update_ordering( node.state = SortProperties::Ordered(options); } else if !node.expr.children().is_empty() { // We have an intermediate (non-leaf) node, account for its children: - node.state = node.expr.get_ordering(&node.children_states); + node.state = node.expr.get_ordering(&node.children_state()); } else if node.expr.as_any().is::() { // We have a Literal, which is the other possible leaf node type: node.state = node.expr.get_ordering(&[]); diff --git a/datafusion/physical-expr/src/sort_properties.rs b/datafusion/physical-expr/src/sort_properties.rs index f8648abdf7a7..f51374461776 100644 --- a/datafusion/physical-expr/src/sort_properties.rs +++ b/datafusion/physical-expr/src/sort_properties.rs @@ -17,13 +17,12 @@ use std::{ops::Neg, sync::Arc}; -use crate::PhysicalExpr; use arrow_schema::SortOptions; + +use crate::PhysicalExpr; use datafusion_common::tree_node::{TreeNode, VisitRecursion}; use datafusion_common::Result; -use itertools::Itertools; - /// To propagate [`SortOptions`] across the [`PhysicalExpr`], it is insufficient /// to simply use `Option`: There must be a differentiation between /// unordered columns and literal values, since literals may not break the ordering @@ -35,11 +34,12 @@ use itertools::Itertools; /// sorted data; however the ((a_ordered + 999) + c_ordered) expression can. Therefore, /// we need two different variants for literals and unordered columns as literals are /// often more ordering-friendly under most mathematical operations. -#[derive(PartialEq, Debug, Clone, Copy)] +#[derive(PartialEq, Debug, Clone, Copy, Default)] pub enum SortProperties { /// Use the ordinary [`SortOptions`] struct to represent ordered data: Ordered(SortOptions), // This alternative represents unordered data: + #[default] Unordered, // Singleton is used for single-valued literal numbers: Singleton, @@ -151,34 +151,24 @@ impl Neg for SortProperties { pub struct ExprOrdering { pub expr: Arc, pub state: SortProperties, - pub children_states: Vec, + pub children: Vec, } impl ExprOrdering { /// Creates a new [`ExprOrdering`] with [`SortProperties::Unordered`] states /// for `expr` and its children. pub fn new(expr: Arc) -> Self { - let size = expr.children().len(); + let children = expr.children(); Self { expr, - state: SortProperties::Unordered, - children_states: vec![SortProperties::Unordered; size], + state: Default::default(), + children: children.into_iter().map(Self::new).collect(), } } - /// Updates this [`ExprOrdering`]'s children states with the given states. - pub fn with_new_children(mut self, children_states: Vec) -> Self { - self.children_states = children_states; - self - } - - /// Creates new [`ExprOrdering`] objects for each child of the expression. - pub fn children_expr_orderings(&self) -> Vec { - self.expr - .children() - .into_iter() - .map(ExprOrdering::new) - .collect() + /// Get a reference to each child state. + pub fn children_state(&self) -> Vec { + self.children.iter().map(|c| c.state).collect() } } @@ -187,8 +177,8 @@ impl TreeNode for ExprOrdering { where F: FnMut(&Self) -> Result, { - for child in self.children_expr_orderings() { - match op(&child)? { + for child in &self.children { + match op(child)? { VisitRecursion::Continue => {} VisitRecursion::Skip => return Ok(VisitRecursion::Continue), VisitRecursion::Stop => return Ok(VisitRecursion::Stop), @@ -197,25 +187,19 @@ impl TreeNode for ExprOrdering { Ok(VisitRecursion::Continue) } - fn map_children(self, transform: F) -> Result + fn map_children(mut self, transform: F) -> Result where F: FnMut(Self) -> Result, { - if self.children_states.is_empty() { + if self.children.is_empty() { Ok(self) } else { - let child_expr_orderings = self.children_expr_orderings(); - // After mapping over the children, the function `F` applies to the - // current object and updates its state. - Ok(self.with_new_children( - child_expr_orderings - .into_iter() - // Update children states after this transformation: - .map(transform) - // Extract the state (i.e. sort properties) information: - .map_ok(|c| c.state) - .collect::>>()?, - )) + self.children = self + .children + .into_iter() + .map(transform) + .collect::>>()?; + Ok(self) } } } diff --git a/datafusion/physical-expr/src/utils.rs b/datafusion/physical-expr/src/utils.rs index ed62956de8e0..71a7ff5fb778 100644 --- a/datafusion/physical-expr/src/utils.rs +++ b/datafusion/physical-expr/src/utils.rs @@ -129,10 +129,11 @@ pub struct ExprTreeNode { impl ExprTreeNode { pub fn new(expr: Arc) -> Self { + let children = expr.children(); ExprTreeNode { expr, data: None, - child_nodes: vec![], + child_nodes: children.into_iter().map(Self::new).collect_vec(), } } @@ -140,12 +141,8 @@ impl ExprTreeNode { &self.expr } - pub fn children(&self) -> Vec> { - self.expr - .children() - .into_iter() - .map(ExprTreeNode::new) - .collect() + pub fn children(&self) -> &[ExprTreeNode] { + &self.child_nodes } } @@ -155,7 +152,7 @@ impl TreeNode for ExprTreeNode { F: FnMut(&Self) -> Result, { for child in self.children() { - match op(&child)? { + match op(child)? { VisitRecursion::Continue => {} VisitRecursion::Skip => return Ok(VisitRecursion::Continue), VisitRecursion::Stop => return Ok(VisitRecursion::Stop), @@ -170,7 +167,7 @@ impl TreeNode for ExprTreeNode { F: FnMut(Self) -> Result, { self.child_nodes = self - .children() + .child_nodes .into_iter() .map(transform) .collect::>>()?; From f6af014860e1b6041e434b3fe6fccee09cb0e6d1 Mon Sep 17 00:00:00 2001 From: Alex Huang Date: Sun, 3 Dec 2023 13:35:17 +0100 Subject: [PATCH 352/572] feat: support `LargeList` in `make_array` and `array_length` (#8121) * feat: support LargeList in make_array and array_length * chore: add tests * fix: update tests for nested array * use usise_as * add new_large_list * refactor array_length * add comment * update test in sqllogictest * fix ci * fix macro * use usize_as * update comment * return based on data_type in make_array --- .../physical-expr/src/array_expressions.rs | 47 +++++++++++++----- datafusion/sqllogictest/test_files/array.slt | 49 +++++++++++++++++++ 2 files changed, 83 insertions(+), 13 deletions(-) diff --git a/datafusion/physical-expr/src/array_expressions.rs b/datafusion/physical-expr/src/array_expressions.rs index 84dfe3b9ff75..0601c22ecfb4 100644 --- a/datafusion/physical-expr/src/array_expressions.rs +++ b/datafusion/physical-expr/src/array_expressions.rs @@ -171,6 +171,10 @@ fn compute_array_length( value = downcast_arg!(value, ListArray).value(0); current_dimension += 1; } + DataType::LargeList(..) => { + value = downcast_arg!(value, LargeListArray).value(0); + current_dimension += 1; + } _ => return Ok(None), } } @@ -252,7 +256,7 @@ macro_rules! call_array_function { } /// Convert one or more [`ArrayRef`] of the same type into a -/// `ListArray` +/// `ListArray` or 'LargeListArray' depending on the offset size. /// /// # Example (non nested) /// @@ -291,7 +295,10 @@ macro_rules! call_array_function { /// └──────────────┘ └──────────────┘ └─────────────────────────────┘ /// col1 col2 output /// ``` -fn array_array(args: &[ArrayRef], data_type: DataType) -> Result { +fn array_array( + args: &[ArrayRef], + data_type: DataType, +) -> Result { // do not accept 0 arguments. if args.is_empty() { return plan_err!("Array requires at least one argument"); @@ -308,8 +315,9 @@ fn array_array(args: &[ArrayRef], data_type: DataType) -> Result { total_len += arg_data.len(); data.push(arg_data); } - let mut offsets = Vec::with_capacity(total_len); - offsets.push(0); + + let mut offsets: Vec = Vec::with_capacity(total_len); + offsets.push(O::usize_as(0)); let capacity = Capacities::Array(total_len); let data_ref = data.iter().collect::>(); @@ -327,11 +335,11 @@ fn array_array(args: &[ArrayRef], data_type: DataType) -> Result { mutable.extend_nulls(1); } } - offsets.push(mutable.len() as i32); + offsets.push(O::usize_as(mutable.len())); } - let data = mutable.freeze(); - Ok(Arc::new(ListArray::try_new( + + Ok(Arc::new(GenericListArray::::try_new( Arc::new(Field::new("item", data_type, true)), OffsetBuffer::new(offsets.into()), arrow_array::make_array(data), @@ -356,7 +364,8 @@ pub fn make_array(arrays: &[ArrayRef]) -> Result { let array = new_null_array(&DataType::Null, arrays.len()); Ok(Arc::new(array_into_list_array(array))) } - data_type => array_array(arrays, data_type), + DataType::LargeList(..) => array_array::(arrays, data_type), + _ => array_array::(arrays, data_type), } } @@ -1693,11 +1702,11 @@ pub fn flatten(args: &[ArrayRef]) -> Result { Ok(Arc::new(flattened_array) as ArrayRef) } -/// Array_length SQL function -pub fn array_length(args: &[ArrayRef]) -> Result { - let list_array = as_list_array(&args[0])?; - let dimension = if args.len() == 2 { - as_int64_array(&args[1])?.clone() +/// Dispatch array length computation based on the offset type. +fn array_length_dispatch(array: &[ArrayRef]) -> Result { + let list_array = as_generic_list_array::(&array[0])?; + let dimension = if array.len() == 2 { + as_int64_array(&array[1])?.clone() } else { Int64Array::from_value(1, list_array.len()) }; @@ -1711,6 +1720,18 @@ pub fn array_length(args: &[ArrayRef]) -> Result { Ok(Arc::new(result) as ArrayRef) } +/// Array_length SQL function +pub fn array_length(args: &[ArrayRef]) -> Result { + match &args[0].data_type() { + DataType::List(_) => array_length_dispatch::(args), + DataType::LargeList(_) => array_length_dispatch::(args), + _ => internal_err!( + "array_length does not support type '{:?}'", + args[0].data_type() + ), + } +} + /// Array_dims SQL function pub fn array_dims(args: &[ArrayRef]) -> Result { let list_array = as_list_array(&args[0])?; diff --git a/datafusion/sqllogictest/test_files/array.slt b/datafusion/sqllogictest/test_files/array.slt index 092bc697a197..6ec2b2cb013b 100644 --- a/datafusion/sqllogictest/test_files/array.slt +++ b/datafusion/sqllogictest/test_files/array.slt @@ -2371,24 +2371,44 @@ select array_length(make_array(1, 2, 3, 4, 5)), array_length(make_array(1, 2, 3) ---- 5 3 3 +query III +select array_length(arrow_cast(make_array(1, 2, 3, 4, 5), 'LargeList(Int64)')), array_length(arrow_cast(make_array(1, 2, 3), 'LargeList(Int64)')), array_length(arrow_cast(make_array([1, 2], [3, 4], [5, 6]), 'LargeList(List(Int64))')); +---- +5 3 3 + # array_length scalar function #2 query III select array_length(make_array(1, 2, 3, 4, 5), 1), array_length(make_array(1, 2, 3), 1), array_length(make_array([1, 2], [3, 4], [5, 6]), 1); ---- 5 3 3 +query III +select array_length(arrow_cast(make_array(1, 2, 3, 4, 5), 'LargeList(Int64)'), 1), array_length(arrow_cast(make_array(1, 2, 3), 'LargeList(Int64)'), 1), array_length(arrow_cast(make_array([1, 2], [3, 4], [5, 6]), 'LargeList(List(Int64))'), 1); +---- +5 3 3 + # array_length scalar function #3 query III select array_length(make_array(1, 2, 3, 4, 5), 2), array_length(make_array(1, 2, 3), 2), array_length(make_array([1, 2], [3, 4], [5, 6]), 2); ---- NULL NULL 2 +query III +select array_length(arrow_cast(make_array(1, 2, 3, 4, 5), 'LargeList(Int64)'), 2), array_length(arrow_cast(make_array(1, 2, 3), 'LargeList(Int64)'), 2), array_length(arrow_cast(make_array([1, 2], [3, 4], [5, 6]), 'LargeList(List(Int64))'), 2); +---- +NULL NULL 2 + # array_length scalar function #4 query II select array_length(array_repeat(array_repeat(array_repeat(3, 5), 2), 3), 1), array_length(array_repeat(array_repeat(array_repeat(3, 5), 2), 3), 2); ---- 3 2 +query II +select array_length(arrow_cast(array_repeat(array_repeat(array_repeat(3, 5), 2), 3), 'LargeList(List(List(Int64)))'), 1), array_length(arrow_cast(array_repeat(array_repeat(array_repeat(3, 5), 2), 3), 'LargeList(List(List(Int64)))'), 2); +---- +3 2 + # array_length scalar function #5 query III select array_length(make_array()), array_length(make_array(), 1), array_length(make_array(), 2) @@ -2407,6 +2427,11 @@ select list_length(make_array(1, 2, 3, 4, 5)), list_length(make_array(1, 2, 3)), ---- 5 3 3 NULL +query III +select list_length(arrow_cast(make_array(1, 2, 3, 4, 5), 'LargeList(Int64)')), list_length(arrow_cast(make_array(1, 2, 3), 'LargeList(Int64)')), list_length(arrow_cast(make_array([1, 2], [3, 4], [5, 6]), 'LargeList(List(Int64))')); +---- +5 3 3 + # array_length with columns query I select array_length(column1, column3) from arrays_values; @@ -2420,6 +2445,18 @@ NULL NULL NULL +query I +select array_length(arrow_cast(column1, 'LargeList(Int64)'), column3) from arrays_values; +---- +10 +NULL +NULL +NULL +NULL +NULL +NULL +NULL + # array_length with columns and scalars query II select array_length(array[array[1, 2], array[3, 4]], column3), array_length(column1, 1) from arrays_values; @@ -2433,6 +2470,18 @@ NULL 10 NULL 10 NULL 10 +query II +select array_length(arrow_cast(array[array[1, 2], array[3, 4]], 'LargeList(List(Int64))'), column3), array_length(arrow_cast(column1, 'LargeList(Int64)'), 1) from arrays_values; +---- +2 10 +2 10 +NULL 10 +NULL 10 +NULL NULL +NULL 10 +NULL 10 +NULL 10 + ## array_dims (aliases: `list_dims`) # array dims error From 26196e648ecfee168265ddf498563abd2651b5e3 Mon Sep 17 00:00:00 2001 From: jakevin Date: Sun, 3 Dec 2023 21:44:15 +0800 Subject: [PATCH 353/572] remove `unalias()` TableScan filters when create Physical Filter (#8404) - remove `unalias` TableScan filters - refactor CreateExternalTable - fix typo --- datafusion-cli/src/exec.rs | 11 ++++------- datafusion/core/src/physical_planner.rs | 5 ++--- datafusion/physical-expr/src/array_expressions.rs | 6 +++--- 3 files changed, 9 insertions(+), 13 deletions(-) diff --git a/datafusion-cli/src/exec.rs b/datafusion-cli/src/exec.rs index 1869e15ef584..63862caab82a 100644 --- a/datafusion-cli/src/exec.rs +++ b/datafusion-cli/src/exec.rs @@ -221,14 +221,11 @@ async fn exec_and_print( | LogicalPlan::Analyze(_) ); - let df = match &plan { - LogicalPlan::Ddl(DdlStatement::CreateExternalTable(cmd)) => { - create_external_table(ctx, cmd).await?; - ctx.execute_logical_plan(plan).await? - } - _ => ctx.execute_logical_plan(plan).await?, - }; + if let LogicalPlan::Ddl(DdlStatement::CreateExternalTable(cmd)) = &plan { + create_external_table(ctx, cmd).await?; + } + let df = ctx.execute_logical_plan(plan).await?; let results = df.collect().await?; let print_options = if should_ignore_maxrows { diff --git a/datafusion/core/src/physical_planner.rs b/datafusion/core/src/physical_planner.rs index 9e64eb9c5108..0e96b126b967 100644 --- a/datafusion/core/src/physical_planner.rs +++ b/datafusion/core/src/physical_planner.rs @@ -86,7 +86,7 @@ use datafusion_expr::expr::{ Cast, GetFieldAccess, GetIndexedField, GroupingSet, InList, Like, TryCast, WindowFunction, }; -use datafusion_expr::expr_rewriter::{unalias, unnormalize_cols}; +use datafusion_expr::expr_rewriter::unnormalize_cols; use datafusion_expr::logical_plan::builder::wrap_projection_for_join_if_necessary; use datafusion_expr::{ DescribeTable, DmlStatement, ScalarFunctionDefinition, StringifiedPlan, WindowFrame, @@ -562,8 +562,7 @@ impl DefaultPhysicalPlanner { // doesn't know (nor should care) how the relation was // referred to in the query let filters = unnormalize_cols(filters.iter().cloned()); - let unaliased: Vec = filters.into_iter().map(unalias).collect(); - source.scan(session_state, projection.as_ref(), &unaliased, *fetch).await + source.scan(session_state, projection.as_ref(), &filters, *fetch).await } LogicalPlan::Copy(CopyTo{ input, diff --git a/datafusion/physical-expr/src/array_expressions.rs b/datafusion/physical-expr/src/array_expressions.rs index 0601c22ecfb4..9489a51fa385 100644 --- a/datafusion/physical-expr/src/array_expressions.rs +++ b/datafusion/physical-expr/src/array_expressions.rs @@ -595,14 +595,14 @@ fn general_array_pop( ) -> Result<(Vec, Vec)> { if from_back { let key = vec![0; list_array.len()]; - // Atttetion: `arr.len() - 1` in extra key defines the last element position (position = index + 1, not inclusive) we want in the new array. + // Attention: `arr.len() - 1` in extra key defines the last element position (position = index + 1, not inclusive) we want in the new array. let extra_key: Vec<_> = list_array .iter() .map(|x| x.map_or(0, |arr| arr.len() as i64 - 1)) .collect(); Ok((key, extra_key)) } else { - // Atttetion: 2 in the `key`` defines the first element position (position = index + 1) we want in the new array. + // Attention: 2 in the `key`` defines the first element position (position = index + 1) we want in the new array. // We only handle two cases of the first element index: if the old array has any elements, starts from 2 (index + 1), or starts from initial. let key: Vec<_> = list_array.iter().map(|x| x.map_or(0, |_| 2)).collect(); let extra_key: Vec<_> = list_array @@ -1414,7 +1414,7 @@ pub fn array_replace_n(args: &[ArrayRef]) -> Result { } pub fn array_replace_all(args: &[ArrayRef]) -> Result { - // replace all occurences (up to "i64::MAX") + // replace all occurrences (up to "i64::MAX") let arr_n = vec![i64::MAX; args[0].len()]; general_replace(as_list_array(&args[0])?, &args[1], &args[2], arr_n) } From e5a95b17a9013b3d1c9bbd719e2ba7c17fabeaf3 Mon Sep 17 00:00:00 2001 From: Nick Poorman Date: Mon, 4 Dec 2023 01:40:50 -0700 Subject: [PATCH 354/572] Update custom-table-providers.md (#8409) --- docs/source/library-user-guide/custom-table-providers.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/source/library-user-guide/custom-table-providers.md b/docs/source/library-user-guide/custom-table-providers.md index ca0e9de779ef..9da207da68f3 100644 --- a/docs/source/library-user-guide/custom-table-providers.md +++ b/docs/source/library-user-guide/custom-table-providers.md @@ -25,7 +25,7 @@ This section will also touch on how to have DataFusion use the new `TableProvide ## Table Provider and Scan -The `scan` method on the `TableProvider` is likely its most important. It returns an `ExecutionPlan` that DataFusion will use to read the actual data during execution o the query. +The `scan` method on the `TableProvider` is likely its most important. It returns an `ExecutionPlan` that DataFusion will use to read the actual data during execution of the query. ### Scan From a73be00aa7397d1e4dcbb74e5ba898f8006b98bf Mon Sep 17 00:00:00 2001 From: Huaijin Date: Mon, 4 Dec 2023 19:58:14 +0800 Subject: [PATCH 355/572] fix transforming `LogicalPlan::Explain` use `TreeNode::transform` fails (#8400) * fix transforming LogicalPlan::Explain use TreeNode::transform fails * Update datafusion/expr/src/logical_plan/plan.rs Co-authored-by: Andrew Lamb --------- Co-authored-by: Andrew Lamb --- datafusion/expr/src/logical_plan/plan.rs | 66 +++++++++++++++++++----- 1 file changed, 53 insertions(+), 13 deletions(-) diff --git a/datafusion/expr/src/logical_plan/plan.rs b/datafusion/expr/src/logical_plan/plan.rs index ea7a48d2c4f4..9bb47c7da058 100644 --- a/datafusion/expr/src/logical_plan/plan.rs +++ b/datafusion/expr/src/logical_plan/plan.rs @@ -877,19 +877,19 @@ impl LogicalPlan { input: Arc::new(inputs[0].clone()), })) } - LogicalPlan::Explain(_) => { - // Explain should be handled specially in the optimizers; - // If this check cannot pass it means some optimizer pass is - // trying to optimize Explain directly - if expr.is_empty() { - return plan_err!("Invalid EXPLAIN command. Expression is empty"); - } - - if inputs.is_empty() { - return plan_err!("Invalid EXPLAIN command. Inputs are empty"); - } - - Ok(self.clone()) + LogicalPlan::Explain(e) => { + assert!( + expr.is_empty(), + "Invalid EXPLAIN command. Expression should empty" + ); + assert_eq!(inputs.len(), 1, "Invalid EXPLAIN command. Inputs are empty"); + Ok(LogicalPlan::Explain(Explain { + verbose: e.verbose, + plan: Arc::new(inputs[0].clone()), + stringified_plans: e.stringified_plans.clone(), + schema: e.schema.clone(), + logical_optimization_succeeded: e.logical_optimization_succeeded, + })) } LogicalPlan::Prepare(Prepare { name, data_types, .. @@ -3076,4 +3076,44 @@ digraph { .unwrap() .is_nullable()); } + + #[test] + fn test_transform_explain() { + let schema = Schema::new(vec![ + Field::new("foo", DataType::Int32, false), + Field::new("bar", DataType::Int32, false), + ]); + + let plan = table_scan(TableReference::none(), &schema, None) + .unwrap() + .explain(false, false) + .unwrap() + .build() + .unwrap(); + + let external_filter = + col("foo").eq(Expr::Literal(ScalarValue::Boolean(Some(true)))); + + // after transformation, because plan is not the same anymore, + // the parent plan is built again with call to LogicalPlan::with_new_inputs -> with_new_exprs + let plan = plan + .transform(&|plan| match plan { + LogicalPlan::TableScan(table) => { + let filter = Filter::try_new( + external_filter.clone(), + Arc::new(LogicalPlan::TableScan(table)), + ) + .unwrap(); + Ok(Transformed::Yes(LogicalPlan::Filter(filter))) + } + x => Ok(Transformed::No(x)), + }) + .unwrap(); + + let expected = "Explain\ + \n Filter: foo = Boolean(true)\ + \n TableScan: ?table?"; + let actual = format!("{}", plan.display_indent()); + assert_eq!(expected.to_string(), actual) + } } From 4b4af65444761f2f2c87b74b0fac8a19db4912a9 Mon Sep 17 00:00:00 2001 From: Asura7969 <1402357969@qq.com> Date: Mon, 4 Dec 2023 20:38:02 +0800 Subject: [PATCH 356/572] Docs: Fix `array_except` documentation example (#8407) * Minor: Improve the document format of JoinHashMap * Docs: Fix `array_except` documentation example --- docs/source/user-guide/sql/scalar_functions.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/source/user-guide/sql/scalar_functions.md b/docs/source/user-guide/sql/scalar_functions.md index 0d9725203c3d..46920f1c4d0b 100644 --- a/docs/source/user-guide/sql/scalar_functions.md +++ b/docs/source/user-guide/sql/scalar_functions.md @@ -2371,7 +2371,7 @@ array_except(array1, array2) +----------------------------------------------------+ | array_except([1, 2, 3, 4], [3, 4, 5, 6]); | +----------------------------------------------------+ -| [3, 4] | +| [1, 2] | +----------------------------------------------------+ ``` From 37bbd665439f8227971a3657a01205544694bed1 Mon Sep 17 00:00:00 2001 From: Asura7969 <1402357969@qq.com> Date: Tue, 5 Dec 2023 05:41:40 +0800 Subject: [PATCH 357/572] Support named query parameters (#8384) * Minor: Improve the document format of JoinHashMap * support named query parameters * cargo fmt * add `ParamValues` conversion * improve doc --- datafusion/common/src/lib.rs | 2 + datafusion/common/src/param_value.rs | 149 +++++++++++++++++++++++ datafusion/core/src/dataframe/mod.rs | 30 ++++- datafusion/core/tests/sql/select.rs | 47 +++++++ datafusion/expr/src/expr.rs | 2 +- datafusion/expr/src/logical_plan/plan.rs | 66 ++-------- datafusion/sql/src/expr/value.rs | 7 +- datafusion/sql/tests/sql_integration.rs | 27 ++-- 8 files changed, 261 insertions(+), 69 deletions(-) create mode 100644 datafusion/common/src/param_value.rs diff --git a/datafusion/common/src/lib.rs b/datafusion/common/src/lib.rs index 90fb4a88149c..6df89624fc51 100644 --- a/datafusion/common/src/lib.rs +++ b/datafusion/common/src/lib.rs @@ -20,6 +20,7 @@ mod dfschema; mod error; mod functional_dependencies; mod join_type; +mod param_value; #[cfg(feature = "pyarrow")] mod pyarrow; mod schema_reference; @@ -59,6 +60,7 @@ pub use functional_dependencies::{ Constraints, Dependency, FunctionalDependence, FunctionalDependencies, }; pub use join_type::{JoinConstraint, JoinSide, JoinType}; +pub use param_value::ParamValues; pub use scalar::{ScalarType, ScalarValue}; pub use schema_reference::{OwnedSchemaReference, SchemaReference}; pub use stats::{ColumnStatistics, Statistics}; diff --git a/datafusion/common/src/param_value.rs b/datafusion/common/src/param_value.rs new file mode 100644 index 000000000000..253c312b66d5 --- /dev/null +++ b/datafusion/common/src/param_value.rs @@ -0,0 +1,149 @@ +// 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 crate::error::{_internal_err, _plan_err}; +use crate::{DataFusionError, Result, ScalarValue}; +use arrow_schema::DataType; +use std::collections::HashMap; + +/// The parameter value corresponding to the placeholder +#[derive(Debug, Clone)] +pub enum ParamValues { + /// for positional query parameters, like select * from test where a > $1 and b = $2 + LIST(Vec), + /// for named query parameters, like select * from test where a > $foo and b = $goo + MAP(HashMap), +} + +impl ParamValues { + /// Verify parameter list length and type + pub fn verify(&self, expect: &Vec) -> Result<()> { + match self { + ParamValues::LIST(list) => { + // Verify if the number of params matches the number of values + if expect.len() != list.len() { + return _plan_err!( + "Expected {} parameters, got {}", + expect.len(), + list.len() + ); + } + + // Verify if the types of the params matches the types of the values + let iter = expect.iter().zip(list.iter()); + for (i, (param_type, value)) in iter.enumerate() { + if *param_type != value.data_type() { + return _plan_err!( + "Expected parameter of type {:?}, got {:?} at index {}", + param_type, + value.data_type(), + i + ); + } + } + Ok(()) + } + ParamValues::MAP(_) => { + // If it is a named query, variables can be reused, + // but the lengths are not necessarily equal + Ok(()) + } + } + } + + pub fn get_placeholders_with_values( + &self, + id: &String, + data_type: &Option, + ) -> Result { + match self { + ParamValues::LIST(list) => { + if id.is_empty() || id == "$0" { + return _plan_err!("Empty placeholder id"); + } + // convert id (in format $1, $2, ..) to idx (0, 1, ..) + let idx = id[1..].parse::().map_err(|e| { + DataFusionError::Internal(format!( + "Failed to parse placeholder id: {e}" + )) + })? - 1; + // value at the idx-th position in param_values should be the value for the placeholder + let value = list.get(idx).ok_or_else(|| { + DataFusionError::Internal(format!( + "No value found for placeholder with id {id}" + )) + })?; + // check if the data type of the value matches the data type of the placeholder + if Some(value.data_type()) != *data_type { + return _internal_err!( + "Placeholder value type mismatch: expected {:?}, got {:?}", + data_type, + value.data_type() + ); + } + Ok(value.clone()) + } + ParamValues::MAP(map) => { + // convert name (in format $a, $b, ..) to mapped values (a, b, ..) + let name = &id[1..]; + // value at the name position in param_values should be the value for the placeholder + let value = map.get(name).ok_or_else(|| { + DataFusionError::Internal(format!( + "No value found for placeholder with name {id}" + )) + })?; + // check if the data type of the value matches the data type of the placeholder + if Some(value.data_type()) != *data_type { + return _internal_err!( + "Placeholder value type mismatch: expected {:?}, got {:?}", + data_type, + value.data_type() + ); + } + Ok(value.clone()) + } + } + } +} + +impl From> for ParamValues { + fn from(value: Vec) -> Self { + Self::LIST(value) + } +} + +impl From> for ParamValues +where + K: Into, +{ + fn from(value: Vec<(K, ScalarValue)>) -> Self { + let value: HashMap = + value.into_iter().map(|(k, v)| (k.into(), v)).collect(); + Self::MAP(value) + } +} + +impl From> for ParamValues +where + K: Into, +{ + fn from(value: HashMap) -> Self { + let value: HashMap = + value.into_iter().map(|(k, v)| (k.into(), v)).collect(); + Self::MAP(value) + } +} diff --git a/datafusion/core/src/dataframe/mod.rs b/datafusion/core/src/dataframe/mod.rs index 89e82fa952bb..52b5157b7313 100644 --- a/datafusion/core/src/dataframe/mod.rs +++ b/datafusion/core/src/dataframe/mod.rs @@ -32,11 +32,12 @@ use datafusion_common::file_options::csv_writer::CsvWriterOptions; use datafusion_common::file_options::json_writer::JsonWriterOptions; use datafusion_common::parsers::CompressionTypeVariant; use datafusion_common::{ - DataFusionError, FileType, FileTypeWriterOptions, SchemaError, UnnestOptions, + DataFusionError, FileType, FileTypeWriterOptions, ParamValues, SchemaError, + UnnestOptions, }; use datafusion_expr::dml::CopyOptions; -use datafusion_common::{Column, DFSchema, ScalarValue}; +use datafusion_common::{Column, DFSchema}; use datafusion_expr::{ avg, count, is_null, max, median, min, stddev, utils::COUNT_STAR_EXPANSION, TableProviderFilterPushDown, UNNAMED_TABLE, @@ -1227,11 +1228,32 @@ impl DataFrame { /// ], /// &results /// ); + /// // Note you can also provide named parameters + /// let results = ctx + /// .sql("SELECT a FROM example WHERE b = $my_param") + /// .await? + /// // replace $my_param with value 2 + /// // Note you can also use a HashMap as well + /// .with_param_values(vec![ + /// ("my_param", ScalarValue::from(2i64)) + /// ])? + /// .collect() + /// .await?; + /// assert_batches_eq!( + /// &[ + /// "+---+", + /// "| a |", + /// "+---+", + /// "| 1 |", + /// "+---+", + /// ], + /// &results + /// ); /// # Ok(()) /// # } /// ``` - pub fn with_param_values(self, param_values: Vec) -> Result { - let plan = self.plan.with_param_values(param_values)?; + pub fn with_param_values(self, query_values: impl Into) -> Result { + let plan = self.plan.with_param_values(query_values)?; Ok(Self::new(self.session_state, plan)) } diff --git a/datafusion/core/tests/sql/select.rs b/datafusion/core/tests/sql/select.rs index 63f3e979305a..cbdea9d72948 100644 --- a/datafusion/core/tests/sql/select.rs +++ b/datafusion/core/tests/sql/select.rs @@ -525,6 +525,53 @@ async fn test_prepare_statement() -> Result<()> { Ok(()) } +#[tokio::test] +async fn test_named_query_parameters() -> Result<()> { + let tmp_dir = TempDir::new()?; + let partition_count = 4; + let ctx = partitioned_csv::create_ctx(&tmp_dir, partition_count).await?; + + // sql to statement then to logical plan with parameters + // c1 defined as UINT32, c2 defined as UInt64 + let results = ctx + .sql("SELECT c1, c2 FROM test WHERE c1 > $coo AND c1 < $foo") + .await? + .with_param_values(vec![ + ("foo", ScalarValue::UInt32(Some(3))), + ("coo", ScalarValue::UInt32(Some(0))), + ])? + .collect() + .await?; + let expected = vec![ + "+----+----+", + "| c1 | c2 |", + "+----+----+", + "| 1 | 1 |", + "| 1 | 2 |", + "| 1 | 3 |", + "| 1 | 4 |", + "| 1 | 5 |", + "| 1 | 6 |", + "| 1 | 7 |", + "| 1 | 8 |", + "| 1 | 9 |", + "| 1 | 10 |", + "| 2 | 1 |", + "| 2 | 2 |", + "| 2 | 3 |", + "| 2 | 4 |", + "| 2 | 5 |", + "| 2 | 6 |", + "| 2 | 7 |", + "| 2 | 8 |", + "| 2 | 9 |", + "| 2 | 10 |", + "+----+----+", + ]; + assert_batches_sorted_eq!(expected, &results); + Ok(()) +} + #[tokio::test] async fn parallel_query_with_filter() -> Result<()> { let tmp_dir = TempDir::new()?; diff --git a/datafusion/expr/src/expr.rs b/datafusion/expr/src/expr.rs index ee9b0ad6f967..6fa400454dff 100644 --- a/datafusion/expr/src/expr.rs +++ b/datafusion/expr/src/expr.rs @@ -671,7 +671,7 @@ impl InSubquery { } } -/// Placeholder, representing bind parameter values such as `$1`. +/// Placeholder, representing bind parameter values such as `$1` or `$name`. /// /// The type of these parameters is inferred using [`Expr::infer_placeholder_types`] /// or can be specified directly using `PREPARE` statements. diff --git a/datafusion/expr/src/logical_plan/plan.rs b/datafusion/expr/src/logical_plan/plan.rs index 9bb47c7da058..fc8590294fe9 100644 --- a/datafusion/expr/src/logical_plan/plan.rs +++ b/datafusion/expr/src/logical_plan/plan.rs @@ -48,7 +48,7 @@ use datafusion_common::tree_node::{ use datafusion_common::{ aggregate_functional_dependencies, internal_err, plan_err, Column, Constraints, DFField, DFSchema, DFSchemaRef, DataFusionError, FunctionalDependencies, - OwnedTableReference, Result, ScalarValue, UnnestOptions, + OwnedTableReference, ParamValues, Result, UnnestOptions, }; // backwards compatibility pub use datafusion_common::display::{PlanType, StringifiedPlan, ToStringifiedPlan}; @@ -993,32 +993,12 @@ impl LogicalPlan { /// ``` pub fn with_param_values( self, - param_values: Vec, + param_values: impl Into, ) -> Result { + let param_values = param_values.into(); match self { LogicalPlan::Prepare(prepare_lp) => { - // Verify if the number of params matches the number of values - if prepare_lp.data_types.len() != param_values.len() { - return plan_err!( - "Expected {} parameters, got {}", - prepare_lp.data_types.len(), - param_values.len() - ); - } - - // Verify if the types of the params matches the types of the values - let iter = prepare_lp.data_types.iter().zip(param_values.iter()); - for (i, (param_type, value)) in iter.enumerate() { - if *param_type != value.data_type() { - return plan_err!( - "Expected parameter of type {:?}, got {:?} at index {}", - param_type, - value.data_type(), - i - ); - } - } - + param_values.verify(&prepare_lp.data_types)?; let input_plan = prepare_lp.input; input_plan.replace_params_with_values(¶m_values) } @@ -1182,7 +1162,7 @@ impl LogicalPlan { /// See [`Self::with_param_values`] for examples and usage pub fn replace_params_with_values( &self, - param_values: &[ScalarValue], + param_values: &ParamValues, ) -> Result { let new_exprs = self .expressions() @@ -1239,36 +1219,15 @@ impl LogicalPlan { /// corresponding values provided in the params_values fn replace_placeholders_with_values( expr: Expr, - param_values: &[ScalarValue], + param_values: &ParamValues, ) -> Result { expr.transform(&|expr| { match &expr { Expr::Placeholder(Placeholder { id, data_type }) => { - if id.is_empty() || id == "$0" { - return plan_err!("Empty placeholder id"); - } - // convert id (in format $1, $2, ..) to idx (0, 1, ..) - let idx = id[1..].parse::().map_err(|e| { - DataFusionError::Internal(format!( - "Failed to parse placeholder id: {e}" - )) - })? - 1; - // value at the idx-th position in param_values should be the value for the placeholder - let value = param_values.get(idx).ok_or_else(|| { - DataFusionError::Internal(format!( - "No value found for placeholder with id {id}" - )) - })?; - // check if the data type of the value matches the data type of the placeholder - if Some(value.data_type()) != *data_type { - return internal_err!( - "Placeholder value type mismatch: expected {:?}, got {:?}", - data_type, - value.data_type() - ); - } + let value = + param_values.get_placeholders_with_values(id, data_type)?; // Replace the placeholder with the value - Ok(Transformed::Yes(Expr::Literal(value.clone()))) + Ok(Transformed::Yes(Expr::Literal(value))) } Expr::ScalarSubquery(qry) => { let subquery = @@ -2580,7 +2539,7 @@ mod tests { use crate::{col, count, exists, in_subquery, lit, placeholder, GroupingSet}; use arrow::datatypes::{DataType, Field, Schema}; use datafusion_common::tree_node::TreeNodeVisitor; - use datafusion_common::{not_impl_err, DFSchema, TableReference}; + use datafusion_common::{not_impl_err, DFSchema, ScalarValue, TableReference}; use std::collections::HashMap; fn employee_schema() -> Schema { @@ -3028,7 +2987,8 @@ digraph { .build() .unwrap(); - plan.replace_params_with_values(&[42i32.into()]) + let param_values = vec![ScalarValue::Int32(Some(42))]; + plan.replace_params_with_values(¶m_values.clone().into()) .expect_err("unexpectedly succeeded to replace an invalid placeholder"); // test $0 placeholder @@ -3041,7 +3001,7 @@ digraph { .build() .unwrap(); - plan.replace_params_with_values(&[42i32.into()]) + plan.replace_params_with_values(¶m_values.into()) .expect_err("unexpectedly succeeded to replace an invalid placeholder"); } diff --git a/datafusion/sql/src/expr/value.rs b/datafusion/sql/src/expr/value.rs index a3f29da488ba..708f7c60011a 100644 --- a/datafusion/sql/src/expr/value.rs +++ b/datafusion/sql/src/expr/value.rs @@ -108,7 +108,12 @@ impl<'a, S: ContextProvider> SqlToRel<'a, S> { } Ok(index) => index - 1, Err(_) => { - return plan_err!("Invalid placeholder, not a number: {param}"); + return if param_data_types.is_empty() { + Ok(Expr::Placeholder(Placeholder::new(param, None))) + } else { + // when PREPARE Statement, param_data_types length is always 0 + plan_err!("Invalid placeholder, not a number: {param}") + }; } }; // Check if the placeholder is in the parameter list diff --git a/datafusion/sql/tests/sql_integration.rs b/datafusion/sql/tests/sql_integration.rs index d5b06bcf815f..83bdb954b134 100644 --- a/datafusion/sql/tests/sql_integration.rs +++ b/datafusion/sql/tests/sql_integration.rs @@ -22,11 +22,11 @@ use std::{sync::Arc, vec}; use arrow_schema::*; use sqlparser::dialect::{Dialect, GenericDialect, HiveDialect, MySqlDialect}; -use datafusion_common::plan_err; use datafusion_common::{ assert_contains, config::ConfigOptions, DataFusionError, Result, ScalarValue, TableReference, }; +use datafusion_common::{plan_err, ParamValues}; use datafusion_expr::{ logical_plan::{LogicalPlan, Prepare}, AggregateUDF, ScalarUDF, TableSource, WindowUDF, @@ -471,6 +471,10 @@ Dml: op=[Insert Into] table=[test_decimal] "INSERT INTO person (id, first_name, last_name) VALUES ($2, $4, $6)", "Error during planning: Placeholder type could not be resolved" )] +#[case::placeholder_type_unresolved( + "INSERT INTO person (id, first_name, last_name) VALUES ($id, $first_name, $last_name)", + "Error during planning: Can't parse placeholder: $id" +)] #[test] fn test_insert_schema_errors(#[case] sql: &str, #[case] error: &str) { let err = logical_plan(sql).unwrap_err(); @@ -2674,7 +2678,7 @@ fn prepare_stmt_quick_test( fn prepare_stmt_replace_params_quick_test( plan: LogicalPlan, - param_values: Vec, + param_values: impl Into, expected_plan: &str, ) -> LogicalPlan { // replace params @@ -3726,7 +3730,7 @@ fn test_prepare_statement_to_plan_no_param() { /////////////////// // replace params with values - let param_values = vec![]; + let param_values: Vec = vec![]; let expected_plan = "Projection: person.id, person.age\ \n Filter: person.age = Int64(10)\ \n TableScan: person"; @@ -3740,7 +3744,7 @@ fn test_prepare_statement_to_plan_one_param_no_value_panic() { let sql = "PREPARE my_plan(INT) AS SELECT id, age FROM person WHERE age = 10"; let plan = logical_plan(sql).unwrap(); // declare 1 param but provide 0 - let param_values = vec![]; + let param_values: Vec = vec![]; assert_eq!( plan.with_param_values(param_values) .unwrap_err() @@ -3853,7 +3857,7 @@ Projection: person.id, orders.order_id assert_eq!(actual_types, expected_types); // replace params with values - let param_values = vec![ScalarValue::Int32(Some(10))]; + let param_values = vec![ScalarValue::Int32(Some(10))].into(); let expected_plan = r#" Projection: person.id, orders.order_id Inner Join: Filter: person.id = orders.customer_id AND person.age = Int32(10) @@ -3885,7 +3889,7 @@ Projection: person.id, person.age assert_eq!(actual_types, expected_types); // replace params with values - let param_values = vec![ScalarValue::Int32(Some(10))]; + let param_values = vec![ScalarValue::Int32(Some(10))].into(); let expected_plan = r#" Projection: person.id, person.age Filter: person.age = Int32(10) @@ -3919,7 +3923,8 @@ Projection: person.id, person.age assert_eq!(actual_types, expected_types); // replace params with values - let param_values = vec![ScalarValue::Int32(Some(10)), ScalarValue::Int32(Some(30))]; + let param_values = + vec![ScalarValue::Int32(Some(10)), ScalarValue::Int32(Some(30))].into(); let expected_plan = r#" Projection: person.id, person.age Filter: person.age BETWEEN Int32(10) AND Int32(30) @@ -3955,7 +3960,7 @@ Projection: person.id, person.age assert_eq!(actual_types, expected_types); // replace params with values - let param_values = vec![ScalarValue::UInt32(Some(10))]; + let param_values = vec![ScalarValue::UInt32(Some(10))].into(); let expected_plan = r#" Projection: person.id, person.age Filter: person.age = () @@ -3995,7 +4000,8 @@ Dml: op=[Update] table=[person] assert_eq!(actual_types, expected_types); // replace params with values - let param_values = vec![ScalarValue::Int32(Some(42)), ScalarValue::UInt32(Some(1))]; + let param_values = + vec![ScalarValue::Int32(Some(42)), ScalarValue::UInt32(Some(1))].into(); let expected_plan = r#" Dml: op=[Update] table=[person] Projection: person.id AS id, person.first_name AS first_name, person.last_name AS last_name, Int32(42) AS age, person.state AS state, person.salary AS salary, person.birth_date AS birth_date, person.😀 AS 😀 @@ -4034,7 +4040,8 @@ fn test_prepare_statement_insert_infer() { ScalarValue::UInt32(Some(1)), ScalarValue::Utf8(Some("Alan".to_string())), ScalarValue::Utf8(Some("Turing".to_string())), - ]; + ] + .into(); let expected_plan = "Dml: op=[Insert Into] table=[person]\ \n Projection: column1 AS id, column2 AS first_name, column3 AS last_name, \ CAST(NULL AS Int32) AS age, CAST(NULL AS Utf8) AS state, CAST(NULL AS Float64) AS salary, \ From 49dc1f2467a18fc5c2ac3d4d1a404a9ae2ffa908 Mon Sep 17 00:00:00 2001 From: Alex Huang Date: Tue, 5 Dec 2023 00:04:19 +0100 Subject: [PATCH 358/572] Minor: Add installation link to README.md (#8389) * Add installation link to README.md --- README.md | 1 + docs/source/user-guide/cli.md | 32 +++++++++++++++++++++++++++++++- 2 files changed, 32 insertions(+), 1 deletion(-) diff --git a/README.md b/README.md index f5ee1d6d806f..883700a39355 100644 --- a/README.md +++ b/README.md @@ -28,6 +28,7 @@ in-memory format. [Python Bindings](https://github.com/apache/arrow-datafusion-p Here are links to some important information - [Project Site](https://arrow.apache.org/datafusion) +- [Installation](https://arrow.apache.org/datafusion/user-guide/cli.html#installation) - [Rust Getting Started](https://arrow.apache.org/datafusion/user-guide/example-usage.html) - [Rust DataFrame API](https://arrow.apache.org/datafusion/user-guide/dataframe.html) - [Rust API docs](https://docs.rs/datafusion/latest/datafusion) diff --git a/docs/source/user-guide/cli.md b/docs/source/user-guide/cli.md index e8fdae7bb097..525ab090ce51 100644 --- a/docs/source/user-guide/cli.md +++ b/docs/source/user-guide/cli.md @@ -31,7 +31,9 @@ The easiest way to install DataFusion CLI a spin is via `cargo install datafusio ### Install and run using Homebrew (on MacOS) -DataFusion CLI can also be installed via Homebrew (on MacOS). Install it as any other pre-built software like this: +DataFusion CLI can also be installed via Homebrew (on MacOS). If you don't have Homebrew installed, you can check how to install it [here](https://docs.brew.sh/Installation). + +Install it as any other pre-built software like this: ```bash brew install datafusion @@ -46,6 +48,34 @@ brew install datafusion datafusion-cli ``` +### Install and run using PyPI + +DataFusion CLI can also be installed via PyPI. You can check how to install PyPI [here](https://pip.pypa.io/en/latest/installation/). + +Install it as any other pre-built software like this: + +```bash +pip3 install datafusion +# Defaulting to user installation because normal site-packages is not writeable +# Collecting datafusion +# Downloading datafusion-33.0.0-cp38-abi3-macosx_11_0_arm64.whl.metadata (9.6 kB) +# Collecting pyarrow>=11.0.0 (from datafusion) +# Downloading pyarrow-14.0.1-cp39-cp39-macosx_11_0_arm64.whl.metadata (3.0 kB) +# Requirement already satisfied: numpy>=1.16.6 in /Users/Library/Python/3.9/lib/python/site-packages (from pyarrow>=11.0.0->datafusion) (1.23.4) +# Downloading datafusion-33.0.0-cp38-abi3-macosx_11_0_arm64.whl (13.5 MB) +# ━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━ 13.5/13.5 MB 3.6 MB/s eta 0:00:00 +# Downloading pyarrow-14.0.1-cp39-cp39-macosx_11_0_arm64.whl (24.0 MB) +# ━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━ 24.0/24.0 MB 36.4 MB/s eta 0:00:00 +# Installing collected packages: pyarrow, datafusion +# Attempting uninstall: pyarrow +# Found existing installation: pyarrow 10.0.1 +# Uninstalling pyarrow-10.0.1: +# Successfully uninstalled pyarrow-10.0.1 +# Successfully installed datafusion-33.0.0 pyarrow-14.0.1 + +datafusion-cli +``` + ### Run using Docker There is no officially published Docker image for the DataFusion CLI, so it is necessary to build from source From 0bcf4627d6ab9faddbfbe11817c55b7a0e0686eb Mon Sep 17 00:00:00 2001 From: Liang-Chi Hsieh Date: Mon, 4 Dec 2023 15:24:18 -0800 Subject: [PATCH 359/572] Update code comment for the cases of regularized RANGE frame and add tests for ORDER BY cases with RANGE frame (#8410) * fix: RANGE frame can be regularized to ROWS frame only if empty ORDER BY clause * Fix flaky test * Update test comment * Add code comment * Update --- datafusion/expr/src/window_frame.rs | 12 +++- datafusion/sqllogictest/test_files/window.slt | 58 +++++++++++++++++++ 2 files changed, 69 insertions(+), 1 deletion(-) diff --git a/datafusion/expr/src/window_frame.rs b/datafusion/expr/src/window_frame.rs index 5f161b85dd9a..2a64f21b856b 100644 --- a/datafusion/expr/src/window_frame.rs +++ b/datafusion/expr/src/window_frame.rs @@ -148,12 +148,22 @@ impl WindowFrame { pub fn regularize(mut frame: WindowFrame, order_bys: usize) -> Result { if frame.units == WindowFrameUnits::Range && order_bys != 1 { // Normally, RANGE frames require an ORDER BY clause with exactly one - // column. However, an ORDER BY clause may be absent in two edge cases. + // column. However, an ORDER BY clause may be absent or present but with + // more than one column in two edge cases: + // 1. start bound is UNBOUNDED or CURRENT ROW + // 2. end bound is CURRENT ROW or UNBOUNDED. + // In these cases, we regularize the RANGE frame to be equivalent to a ROWS + // frame with the UNBOUNDED bounds. + // Note that this follows Postgres behavior. if (frame.start_bound.is_unbounded() || frame.start_bound == WindowFrameBound::CurrentRow) && (frame.end_bound == WindowFrameBound::CurrentRow || frame.end_bound.is_unbounded()) { + // If an ORDER BY clause is absent, the frame is equivalent to a ROWS + // frame with the UNBOUNDED bounds. + // If an ORDER BY clause is present but has more than one column, the + // frame is unchanged. if order_bys == 0 { frame.units = WindowFrameUnits::Rows; frame.start_bound = diff --git a/datafusion/sqllogictest/test_files/window.slt b/datafusion/sqllogictest/test_files/window.slt index bb6ca119480d..c0dcd4ae1ea5 100644 --- a/datafusion/sqllogictest/test_files/window.slt +++ b/datafusion/sqllogictest/test_files/window.slt @@ -3727,3 +3727,61 @@ FROM score_board s statement ok DROP TABLE score_board; + +# Regularize RANGE frame +query error DataFusion error: Error during planning: RANGE requires exactly one ORDER BY column +select a, + rank() over (order by a, a + 1 RANGE BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING) rnk + from (select 1 a union select 2 a) q ORDER BY a + +query II +select a, + rank() over (order by a RANGE BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING) rnk + from (select 1 a union select 2 a) q ORDER BY a +---- +1 1 +2 2 + +query error DataFusion error: Error during planning: RANGE requires exactly one ORDER BY column +select a, + rank() over (RANGE BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING) rnk + from (select 1 a union select 2 a) q ORDER BY a + +query II +select a, + rank() over (order by a, a + 1 RANGE BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING) rnk + from (select 1 a union select 2 a) q ORDER BY a +---- +1 1 +2 2 + +query II +select a, + rank() over (order by a RANGE BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING) rnk + from (select 1 a union select 2 a) q ORDER BY a +---- +1 1 +2 2 + +# TODO: this is different to Postgres which returns [1, 1] for `rnk`. +# Comment it because it is flaky now as it depends on the order of the `a` column. +# query II +# select a, +# rank() over (RANGE BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING) rnk +# from (select 1 a union select 2 a) q ORDER BY rnk +# ---- +# 1 1 +# 2 2 + +# TODO: this works in Postgres which returns [1, 1]. +query error DataFusion error: Arrow error: Invalid argument error: must either specify a row count or at least one column +select rank() over (RANGE between UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING) rnk + from (select 1 a union select 2 a) q; + +# TODO: this is different to Postgres which returns [1, 1] for `rnk`. +query I +select rank() over (order by 1 RANGE BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING) rnk + from (select 1 a union select 2 a) q ORDER BY rnk +---- +1 +2 From 08fff2dc9fd4aa4eafe5cd69f84a41c3ed338f1b Mon Sep 17 00:00:00 2001 From: Andrew Lamb Date: Mon, 4 Dec 2023 20:25:51 -0500 Subject: [PATCH 360/572] Minor: Add example with parameters to LogicalPlan (#8418) --- datafusion/expr/src/logical_plan/plan.rs | 31 +++++++++++++++++++----- 1 file changed, 25 insertions(+), 6 deletions(-) diff --git a/datafusion/expr/src/logical_plan/plan.rs b/datafusion/expr/src/logical_plan/plan.rs index fc8590294fe9..2988e7536bce 100644 --- a/datafusion/expr/src/logical_plan/plan.rs +++ b/datafusion/expr/src/logical_plan/plan.rs @@ -976,9 +976,10 @@ impl LogicalPlan { /// .filter(col("id").eq(placeholder("$1"))).unwrap() /// .build().unwrap(); /// - /// assert_eq!("Filter: t1.id = $1\ - /// \n TableScan: t1", - /// plan.display_indent().to_string() + /// assert_eq!( + /// "Filter: t1.id = $1\ + /// \n TableScan: t1", + /// plan.display_indent().to_string() /// ); /// /// // Fill in the parameter $1 with a literal 3 @@ -986,10 +987,28 @@ impl LogicalPlan { /// ScalarValue::from(3i32) // value at index 0 --> $1 /// ]).unwrap(); /// - /// assert_eq!("Filter: t1.id = Int32(3)\ - /// \n TableScan: t1", - /// plan.display_indent().to_string() + /// assert_eq!( + /// "Filter: t1.id = Int32(3)\ + /// \n TableScan: t1", + /// plan.display_indent().to_string() /// ); + /// + /// // Note you can also used named parameters + /// // Build SELECT * FROM t1 WHRERE id = $my_param + /// let plan = table_scan(Some("t1"), &schema, None).unwrap() + /// .filter(col("id").eq(placeholder("$my_param"))).unwrap() + /// .build().unwrap() + /// // Fill in the parameter $my_param with a literal 3 + /// .with_param_values(vec![ + /// ("my_param", ScalarValue::from(3i32)), + /// ]).unwrap(); + /// + /// assert_eq!( + /// "Filter: t1.id = Int32(3)\ + /// \n TableScan: t1", + /// plan.display_indent().to_string() + /// ); + /// /// ``` pub fn with_param_values( self, From d1554c85c0da2342c5247ed22a728617e8f69142 Mon Sep 17 00:00:00 2001 From: Andrew Lamb Date: Mon, 4 Dec 2023 20:26:30 -0500 Subject: [PATCH 361/572] Minor: Improve `PruningPredicate` documentation (#8394) * Minor: Improve PruningPredicate documentation * tweaks * Apply suggestions from code review Co-authored-by: Liang-Chi Hsieh --------- Co-authored-by: Liang-Chi Hsieh --- .../core/src/physical_optimizer/pruning.rs | 57 ++++++++++++------- 1 file changed, 38 insertions(+), 19 deletions(-) diff --git a/datafusion/core/src/physical_optimizer/pruning.rs b/datafusion/core/src/physical_optimizer/pruning.rs index de508327fade..b2ba7596db8d 100644 --- a/datafusion/core/src/physical_optimizer/pruning.rs +++ b/datafusion/core/src/physical_optimizer/pruning.rs @@ -66,43 +66,57 @@ use log::trace; /// min_values("X") -> None /// ``` pub trait PruningStatistics { - /// return the minimum values for the named column, if known. - /// Note: the returned array must contain `num_containers()` rows + /// Return the minimum values for the named column, if known. + /// + /// If the minimum value for a particular container is not known, the + /// returned array should have `null` in that row. If the minimum value is + /// not known for any row, return `None`. + /// + /// Note: the returned array must contain [`Self::num_containers`] rows fn min_values(&self, column: &Column) -> Option; - /// return the maximum values for the named column, if known. - /// Note: the returned array must contain `num_containers()` rows. + /// Return the maximum values for the named column, if known. + /// + /// See [`Self::min_values`] for when to return `None` and null values. + /// + /// Note: the returned array must contain [`Self::num_containers`] rows fn max_values(&self, column: &Column) -> Option; - /// return the number of containers (e.g. row groups) being - /// pruned with these statistics + /// Return the number of containers (e.g. row groups) being + /// pruned with these statistics (the number of rows in each returned array) fn num_containers(&self) -> usize; - /// return the number of null values for the named column as an + /// Return the number of null values for the named column as an /// `Option`. /// - /// Note: the returned array must contain `num_containers()` rows. + /// See [`Self::min_values`] for when to return `None` and null values. + /// + /// Note: the returned array must contain [`Self::num_containers`] rows fn null_counts(&self, column: &Column) -> Option; } -/// Evaluates filter expressions on statistics, rather than the actual data. If -/// no rows could possibly pass the filter entire containers can be "pruned" -/// (skipped), without reading any actual data, leading to significant +/// Evaluates filter expressions on statistics such as min/max values and null +/// counts, attempting to prove a "container" (e.g. Parquet Row Group) can be +/// skipped without reading the actual data, potentially leading to significant /// performance improvements. /// -/// [`PruningPredicate`]s are used to prune (avoid scanning) Parquet Row Groups +/// For example, [`PruningPredicate`]s are used to prune Parquet Row Groups /// based on the min/max values found in the Parquet metadata. If the /// `PruningPredicate` can guarantee that no rows in the Row Group match the /// filter, the entire Row Group is skipped during query execution. /// -/// Note that this API is designed to be general, as it works: +/// The `PruningPredicate` API is general, allowing it to be used for pruning +/// other types of containers (e.g. files) based on statistics that may be +/// known from external catalogs (e.g. Delta Lake) or other sources. Thus it +/// supports: /// /// 1. Arbitrary expressions expressions (including user defined functions) /// -/// 2. Anything that implements the [`PruningStatistics`] trait, not just -/// Parquet metadata, allowing it to be used by other systems to prune entities -/// (e.g. entire files) if the statistics are known via some other source, such -/// as a catalog. +/// 2. Vectorized evaluation (provide more than one set of statistics at a time) +/// so it is suitable for pruning 1000s of containers. +/// +/// 3. Anything that implements the [`PruningStatistics`] trait, not just +/// Parquet metadata. /// /// # Example /// @@ -122,6 +136,7 @@ pub trait PruningStatistics { /// B: true (rows might match x = 5) /// C: true (rows might match x = 5) /// ``` +/// /// See [`PruningPredicate::try_new`] and [`PruningPredicate::prune`] for more information. #[derive(Debug, Clone)] pub struct PruningPredicate { @@ -251,8 +266,12 @@ fn is_always_true(expr: &Arc) -> bool { .unwrap_or_default() } -/// Records for which columns statistics are necessary to evaluate a -/// pruning predicate. +/// Describes which columns statistics are necessary to evaluate a +/// [`PruningPredicate`]. +/// +/// This structure permits reading and creating the minimum number statistics, +/// which is important since statistics may be non trivial to read (e.g. large +/// strings or when there are 1000s of columns). /// /// Handles creating references to the min/max statistics /// for columns as well as recording which statistics are needed From 2e5ad7a3cb3b3f3e96f931b903eb8bb6639329ff Mon Sep 17 00:00:00 2001 From: Wei Date: Wed, 6 Dec 2023 03:30:43 +0800 Subject: [PATCH 362/572] feat: ScalarValue from String (#8411) * feat: scalar from string * chore: cr comment --- datafusion/common/src/scalar.rs | 16 ++++++++++++++++ 1 file changed, 16 insertions(+) diff --git a/datafusion/common/src/scalar.rs b/datafusion/common/src/scalar.rs index ef0edbd9e09f..177fe00a6a3c 100644 --- a/datafusion/common/src/scalar.rs +++ b/datafusion/common/src/scalar.rs @@ -3065,6 +3065,12 @@ impl FromStr for ScalarValue { } } +impl From for ScalarValue { + fn from(value: String) -> Self { + ScalarValue::Utf8(Some(value)) + } +} + impl From> for ScalarValue { fn from(value: Vec<(&str, ScalarValue)>) -> Self { let (fields, scalars): (SchemaBuilder, Vec<_>) = value @@ -4688,6 +4694,16 @@ mod tests { ); } + #[test] + fn test_scalar_value_from_string() { + let scalar = ScalarValue::from("foo"); + assert_eq!(scalar, ScalarValue::Utf8(Some("foo".to_string()))); + let scalar = ScalarValue::from("foo".to_string()); + assert_eq!(scalar, ScalarValue::Utf8(Some("foo".to_string()))); + let scalar = ScalarValue::from_str("foo").unwrap(); + assert_eq!(scalar, ScalarValue::Utf8(Some("foo".to_string()))); + } + #[test] fn test_scalar_struct() { let field_a = Arc::new(Field::new("A", DataType::Int32, false)); From 2d5f30efba8231766368af6016d58752902287e4 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Tue, 5 Dec 2023 15:28:44 -0500 Subject: [PATCH 363/572] Bump actions/labeler from 4.3.0 to 5.0.0 (#8422) Bumps [actions/labeler](https://github.com/actions/labeler) from 4.3.0 to 5.0.0. - [Release notes](https://github.com/actions/labeler/releases) - [Commits](https://github.com/actions/labeler/compare/v4.3.0...v5.0.0) --- updated-dependencies: - dependency-name: actions/labeler dependency-type: direct:production update-type: version-update:semver-major ... Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- .github/workflows/dev_pr.yml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/.github/workflows/dev_pr.yml b/.github/workflows/dev_pr.yml index 85aabc188934..77b257743331 100644 --- a/.github/workflows/dev_pr.yml +++ b/.github/workflows/dev_pr.yml @@ -46,7 +46,7 @@ jobs: github.event_name == 'pull_request_target' && (github.event.action == 'opened' || github.event.action == 'synchronize') - uses: actions/labeler@v4.3.0 + uses: actions/labeler@v5.0.0 with: repo-token: ${{ secrets.GITHUB_TOKEN }} configuration-path: .github/workflows/dev_pr/labeler.yml From a3f34c960e8c24f88a1bc9297733885cbacffc3f Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Tue, 5 Dec 2023 15:43:27 -0500 Subject: [PATCH 364/572] Update sqlparser requirement from 0.39.0 to 0.40.0 (#8338) * Update sqlparser requirement from 0.39.0 to 0.40.0 Updates the requirements on [sqlparser](https://github.com/sqlparser-rs/sqlparser-rs) to permit the latest version. - [Changelog](https://github.com/sqlparser-rs/sqlparser-rs/blob/main/CHANGELOG.md) - [Commits](https://github.com/sqlparser-rs/sqlparser-rs/compare/v0.39.0...v0.39.0) --- updated-dependencies: - dependency-name: sqlparser dependency-type: direct:production ... Signed-off-by: dependabot[bot] * Update for new API * Update datafusion-cli Cargo.check --------- Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> Co-authored-by: Andrew Lamb --- Cargo.toml | 2 +- datafusion-cli/Cargo.lock | 312 ++++++++++++++++++++------------ datafusion/sql/src/statement.rs | 8 +- 3 files changed, 202 insertions(+), 120 deletions(-) diff --git a/Cargo.toml b/Cargo.toml index 60befdf1cfb7..2bcbe059ab25 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -85,7 +85,7 @@ parquet = { version = "49.0.0", default-features = false, features = ["arrow", " rand = "0.8" rstest = "0.18.0" serde_json = "1" -sqlparser = { version = "0.39.0", features = ["visitor"] } +sqlparser = { version = "0.40.0", features = ["visitor"] } tempfile = "3" thiserror = "1.0.44" chrono = { version = "0.4.31", default-features = false } diff --git a/datafusion-cli/Cargo.lock b/datafusion-cli/Cargo.lock index fa2832ab3fc6..474d85ac4603 100644 --- a/datafusion-cli/Cargo.lock +++ b/datafusion-cli/Cargo.lock @@ -178,7 +178,7 @@ dependencies = [ "chrono", "chrono-tz", "half", - "hashbrown 0.14.2", + "hashbrown 0.14.3", "num", ] @@ -304,7 +304,7 @@ dependencies = [ "arrow-data", "arrow-schema", "half", - "hashbrown 0.14.2", + "hashbrown 0.14.3", ] [[package]] @@ -360,9 +360,9 @@ dependencies = [ [[package]] name = "async-compression" -version = "0.4.4" +version = "0.4.5" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "f658e2baef915ba0f26f1f7c42bfb8e12f532a01f449a090ded75ae7a07e9ba2" +checksum = "bc2d0cfb2a7388d34f590e76686704c494ed7aaceed62ee1ba35cbf363abc2a5" dependencies = [ "bzip2", "flate2", @@ -820,9 +820,9 @@ checksum = "a2bd12c1caf447e69cd4528f47f94d203fd2582878ecb9e9465484c4148a8223" [[package]] name = "bytes-utils" -version = "0.1.3" +version = "0.1.4" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "e47d3a8076e283f3acd27400535992edb3ba4b5bb72f8891ad8fbe7932a7d4b9" +checksum = "7dafe3a8757b027e2be6e4e5601ed563c55989fcf1546e933c66c8eb3a058d35" dependencies = [ "bytes", "either", @@ -851,10 +851,11 @@ dependencies = [ [[package]] name = "cc" -version = "1.0.84" +version = "1.0.83" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "0f8e7c90afad890484a21653d08b6e209ae34770fb5ee298f9c699fcc1e5c856" +checksum = "f1174fb0b6ec23863f8b971027804a42614e347eafb0a95bf0b12cdae21fc4d0" dependencies = [ + "jobserver", "libc", ] @@ -874,7 +875,7 @@ dependencies = [ "iana-time-zone", "num-traits", "serde", - "windows-targets", + "windows-targets 0.48.5", ] [[package]] @@ -988,9 +989,9 @@ checksum = "f7144d30dcf0fafbce74250a3963025d8d52177934239851c917d29f1df280c2" [[package]] name = "core-foundation" -version = "0.9.3" +version = "0.9.4" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "194a7a9e6de53fa55116934067c844d9d749312f75c6f6d0980e8c252f8c2146" +checksum = "91e195e091a93c46f7102ec7818a2aa394e1e1771c3ab4825963fa03e45afb8f" dependencies = [ "core-foundation-sys", "libc", @@ -998,9 +999,9 @@ dependencies = [ [[package]] name = "core-foundation-sys" -version = "0.8.4" +version = "0.8.6" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "e496a50fda8aacccc86d7529e2c1e0892dbd0f898a6b5645b5561b89c3210efa" +checksum = "06ea2b9bc92be3c2baa9334a323ebca2d6f074ff852cd1d7b11064035cd3868f" [[package]] name = "core2" @@ -1089,7 +1090,7 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "978747c1d849a7d2ee5e8adc0159961c48fb7e5db2f06af6723b80123bb53856" dependencies = [ "cfg-if", - "hashbrown 0.14.2", + "hashbrown 0.14.3", "lock_api", "once_cell", "parking_lot_core", @@ -1121,7 +1122,7 @@ dependencies = [ "futures", "glob", "half", - "hashbrown 0.14.2", + "hashbrown 0.14.3", "indexmap 2.1.0", "itertools 0.12.0", "log", @@ -1196,7 +1197,7 @@ dependencies = [ "datafusion-common", "datafusion-expr", "futures", - "hashbrown 0.14.2", + "hashbrown 0.14.3", "log", "object_store", "parking_lot", @@ -1229,7 +1230,7 @@ dependencies = [ "datafusion-common", "datafusion-expr", "datafusion-physical-expr", - "hashbrown 0.14.2", + "hashbrown 0.14.3", "itertools 0.12.0", "log", "regex-syntax", @@ -1252,7 +1253,7 @@ dependencies = [ "datafusion-common", "datafusion-expr", "half", - "hashbrown 0.14.2", + "hashbrown 0.14.3", "hex", "indexmap 2.1.0", "itertools 0.12.0", @@ -1284,7 +1285,7 @@ dependencies = [ "datafusion-physical-expr", "futures", "half", - "hashbrown 0.14.2", + "hashbrown 0.14.3", "indexmap 2.1.0", "itertools 0.12.0", "log", @@ -1310,9 +1311,9 @@ dependencies = [ [[package]] name = "deranged" -version = "0.3.9" +version = "0.3.10" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "0f32d04922c60427da6f9fef14d042d9edddef64cb9d4ce0d64d0685fbeb1fd3" +checksum = "8eb30d70a07a3b04884d2677f06bec33509dc67ca60d92949e5535352d3191dc" dependencies = [ "powerfmt", ] @@ -1423,12 +1424,12 @@ checksum = "5443807d6dff69373d433ab9ef5378ad8df50ca6298caf15de6e52e24aaf54d5" [[package]] name = "errno" -version = "0.3.6" +version = "0.3.8" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "7c18ee0ed65a5f1f81cac6b1d213b69c35fa47d4252ad41f1486dbd8226fe36e" +checksum = "a258e46cdc063eb8519c00b9fc845fc47bcfca4130e2f08e88665ceda8474245" dependencies = [ "libc", - "windows-sys", + "windows-sys 0.52.0", ] [[package]] @@ -1464,7 +1465,7 @@ checksum = "ef033ed5e9bad94e55838ca0ca906db0e043f517adda0c8b79c7a8c66c93c1b5" dependencies = [ "cfg-if", "rustix", - "windows-sys", + "windows-sys 0.48.0", ] [[package]] @@ -1510,9 +1511,9 @@ checksum = "3f9eec918d3f24069decb9af1554cad7c880e2da24a9afd88aca000531ab82c1" [[package]] name = "form_urlencoded" -version = "1.2.0" +version = "1.2.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "a62bc1cf6f830c2ec14a513a9fb124d0a213a629668a4186f329db21fe045652" +checksum = "e13624c2627564efccf4934284bdd98cbaa14e79b0b5a141218e507b3a823456" dependencies = [ "percent-encoding", ] @@ -1635,9 +1636,9 @@ dependencies = [ [[package]] name = "gimli" -version = "0.28.0" +version = "0.28.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "6fb8d784f27acf97159b40fc4db5ecd8aa23b9ad5ef69cdd136d3bc80665f0c0" +checksum = "4271d37baee1b8c7e4b708028c57d816cf9d2434acb33a549475f78c181f6253" [[package]] name = "glob" @@ -1647,9 +1648,9 @@ checksum = "d2fabcfbdc87f4758337ca535fb41a6d701b65693ce38287d856d1674551ec9b" [[package]] name = "h2" -version = "0.3.21" +version = "0.3.22" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "91fc23aa11be92976ef4729127f1a74adf36d8436f7816b185d18df956790833" +checksum = "4d6250322ef6e60f93f9a2162799302cd6f68f79f6e5d85c8c16f14d1d958178" dependencies = [ "bytes", "fnv", @@ -1657,7 +1658,7 @@ dependencies = [ "futures-sink", "futures-util", "http", - "indexmap 1.9.3", + "indexmap 2.1.0", "slab", "tokio", "tokio-util", @@ -1692,9 +1693,9 @@ dependencies = [ [[package]] name = "hashbrown" -version = "0.14.2" +version = "0.14.3" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "f93e7192158dbcda357bdec5fb5788eebf8bbac027f3f33e719d29135ae84156" +checksum = "290f1a1d9242c78d09ce40a5e87e7554ee637af1351968159f4952f028f75604" dependencies = [ "ahash", "allocator-api2", @@ -1738,9 +1739,9 @@ dependencies = [ [[package]] name = "http" -version = "0.2.10" +version = "0.2.11" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "f95b9abcae896730d42b78e09c155ed4ddf82c07b4de772c64aee5b2d8b7c150" +checksum = "8947b1a6fad4393052c7ba1f4cd97bed3e953a95c79c92ad9b051a04611d9fbb" dependencies = [ "bytes", "fnv", @@ -1824,7 +1825,7 @@ dependencies = [ "futures-util", "http", "hyper", - "rustls 0.21.8", + "rustls 0.21.9", "tokio", "tokio-rustls 0.24.1", ] @@ -1854,9 +1855,9 @@ dependencies = [ [[package]] name = "idna" -version = "0.4.0" +version = "0.5.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "7d20d6b07bfbc108882d88ed8e37d39636dcc260e15e30c45e6ba089610b917c" +checksum = "634d9b1461af396cad843f47fdba5597a4f9e6ddd4bfb6ff5d85028c25cb12f6" dependencies = [ "unicode-bidi", "unicode-normalization", @@ -1879,7 +1880,7 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "d530e1a18b1cb4c484e6e34556a0d948706958449fca0cab753d649f2bce3d1f" dependencies = [ "equivalent", - "hashbrown 0.14.2", + "hashbrown 0.14.3", ] [[package]] @@ -1927,11 +1928,20 @@ version = "1.0.9" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "af150ab688ff2122fcef229be89cb50dd66af9e01a4ff320cc137eecc9bacc38" +[[package]] +name = "jobserver" +version = "0.1.27" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "8c37f63953c4c63420ed5fd3d6d398c719489b9f872b9fa683262f8edd363c7d" +dependencies = [ + "libc", +] + [[package]] name = "js-sys" -version = "0.3.65" +version = "0.3.66" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "54c0c35952f67de54bb584e9fd912b3023117cbafc0a77d8f3dee1fb5f572fe8" +checksum = "cee9c64da59eae3b50095c18d3e74f8b73c0b86d2792824ff01bbce68ba229ca" dependencies = [ "wasm-bindgen", ] @@ -2065,9 +2075,9 @@ dependencies = [ [[package]] name = "linux-raw-sys" -version = "0.4.11" +version = "0.4.12" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "969488b55f8ac402214f3f5fd243ebb7206cf82de60d3172994707a4bcc2b829" +checksum = "c4cd1a83af159aa67994778be9070f0ae1bd732942279cabb14f86f986a21456" [[package]] name = "lock_api" @@ -2153,7 +2163,7 @@ checksum = "3dce281c5e46beae905d4de1870d8b1509a9142b62eedf18b443b011ca8343d0" dependencies = [ "libc", "wasi", - "windows-sys", + "windows-sys 0.48.0", ] [[package]] @@ -2297,7 +2307,7 @@ dependencies = [ "quick-xml", "rand", "reqwest", - "ring 0.17.5", + "ring 0.17.6", "rustls-pemfile", "serde", "serde_json", @@ -2361,7 +2371,7 @@ dependencies = [ "libc", "redox_syscall", "smallvec", - "windows-targets", + "windows-targets 0.48.5", ] [[package]] @@ -2384,7 +2394,7 @@ dependencies = [ "chrono", "flate2", "futures", - "hashbrown 0.14.2", + "hashbrown 0.14.3", "lz4_flex", "num", "num-bigint", @@ -2415,9 +2425,9 @@ checksum = "de3145af08024dea9fa9914f381a17b8fc6034dfb00f3a84013f7ff43f29ed4c" [[package]] name = "percent-encoding" -version = "2.3.0" +version = "2.3.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "9b2a4787296e9989611394c33f193f676704af1686e70b8f8033ab5ba9a35a94" +checksum = "e3148f5046208a5d56bcfc03053e3ca6334e51da8dfb19b6cdc8b306fae3283e" [[package]] name = "petgraph" @@ -2574,9 +2584,9 @@ dependencies = [ [[package]] name = "proc-macro2" -version = "1.0.69" +version = "1.0.70" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "134c189feb4956b20f6f547d2cf727d4c0fe06722b20a0eec87ed445a97f92da" +checksum = "39278fbbf5fb4f646ce651690877f89d1c5811a3d4acb27700c1cb3cdb78fd3b" dependencies = [ "unicode-ident", ] @@ -2724,7 +2734,7 @@ dependencies = [ "once_cell", "percent-encoding", "pin-project-lite", - "rustls 0.21.8", + "rustls 0.21.9", "rustls-pemfile", "serde", "serde_json", @@ -2760,16 +2770,16 @@ dependencies = [ [[package]] name = "ring" -version = "0.17.5" +version = "0.17.6" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "fb0205304757e5d899b9c2e448b867ffd03ae7f988002e47cd24954391394d0b" +checksum = "684d5e6e18f669ccebf64a92236bb7db9a34f07be010e3627368182027180866" dependencies = [ "cc", "getrandom", "libc", "spin 0.9.8", "untrusted 0.9.0", - "windows-sys", + "windows-sys 0.48.0", ] [[package]] @@ -2821,15 +2831,15 @@ dependencies = [ [[package]] name = "rustix" -version = "0.38.21" +version = "0.38.26" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "2b426b0506e5d50a7d8dafcf2e81471400deb602392c7dd110815afb4eaf02a3" +checksum = "9470c4bf8246c8daf25f9598dca807fb6510347b1e1cfa55749113850c79d88a" dependencies = [ "bitflags 2.4.1", "errno", "libc", "linux-raw-sys", - "windows-sys", + "windows-sys 0.52.0", ] [[package]] @@ -2846,12 +2856,12 @@ dependencies = [ [[package]] name = "rustls" -version = "0.21.8" +version = "0.21.9" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "446e14c5cda4f3f30fe71863c34ec70f5ac79d6087097ad0bb433e1be5edf04c" +checksum = "629648aced5775d558af50b2b4c7b02983a04b312126d45eeead26e7caa498b9" dependencies = [ "log", - "ring 0.17.5", + "ring 0.17.6", "rustls-webpki", "sct", ] @@ -2883,7 +2893,7 @@ version = "0.101.7" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "8b6275d1ee7a1cd780b64aca7726599a1dbc893b1e64144529e55c3c2f745765" dependencies = [ - "ring 0.17.5", + "ring 0.17.6", "untrusted 0.9.0", ] @@ -2937,7 +2947,7 @@ version = "0.1.22" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "0c3733bf4cf7ea0880754e19cb5a462007c4a8c1914bff372ccc95b464f1df88" dependencies = [ - "windows-sys", + "windows-sys 0.48.0", ] [[package]] @@ -2952,7 +2962,7 @@ version = "0.7.1" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "da046153aa2352493d6cb7da4b6e5c0c057d8a1d0a9aa8560baffdd945acd414" dependencies = [ - "ring 0.17.5", + "ring 0.17.6", "untrusted 0.9.0", ] @@ -2993,18 +3003,18 @@ checksum = "a3f0bf26fd526d2a95683cd0f87bf103b8539e2ca1ef48ce002d67aad59aa0b4" [[package]] name = "serde" -version = "1.0.192" +version = "1.0.193" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "bca2a08484b285dcb282d0f67b26cadc0df8b19f8c12502c13d966bf9482f001" +checksum = "25dd9975e68d0cb5aa1120c288333fc98731bd1dd12f561e468ea4728c042b89" dependencies = [ "serde_derive", ] [[package]] name = "serde_derive" -version = "1.0.192" +version = "1.0.193" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "d6c7207fbec9faa48073f3e3074cbe553af6ea512d7c21ba46e434e70ea9fbc1" +checksum = "43576ca501357b9b071ac53cdc7da8ef0cbd9493d8df094cd821777ea6e894d3" dependencies = [ "proc-macro2", "quote", @@ -3090,9 +3100,9 @@ dependencies = [ [[package]] name = "snap" -version = "1.1.0" +version = "1.1.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "5e9f0ab6ef7eb7353d9119c170a436d1bf248eea575ac42d19d12f4e34130831" +checksum = "1b6b67fb9a61334225b5b790716f609cd58395f895b3fe8b328786812a40bc3b" [[package]] name = "socket2" @@ -3111,7 +3121,7 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "7b5fac59a5cb5dd637972e5fca70daf0523c9067fcdc4842f053dae04a18f8e9" dependencies = [ "libc", - "windows-sys", + "windows-sys 0.48.0", ] [[package]] @@ -3128,9 +3138,9 @@ checksum = "6980e8d7511241f8acf4aebddbb1ff938df5eebe98691418c4468d0b72a96a67" [[package]] name = "sqlparser" -version = "0.39.0" +version = "0.40.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "743b4dc2cbde11890ccb254a8fc9d537fa41b36da00de2a1c5e9848c9bc42bd7" +checksum = "7c80afe31cdb649e56c0d9bb5503be9166600d68a852c38dd445636d126858e5" dependencies = [ "log", "sqlparser_derive", @@ -3138,9 +3148,9 @@ dependencies = [ [[package]] name = "sqlparser_derive" -version = "0.1.1" +version = "0.2.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "55fe75cb4a364c7f7ae06c7dbbc8d84bddd85d6cdf9975963c3935bc1991761e" +checksum = "3e9c2e1dde0efa87003e7923d94a90f46e3274ad1649f51de96812be561f041f" dependencies = [ "proc-macro2", "quote", @@ -3246,14 +3256,14 @@ dependencies = [ "fastrand 2.0.1", "redox_syscall", "rustix", - "windows-sys", + "windows-sys 0.48.0", ] [[package]] name = "termcolor" -version = "1.3.0" +version = "1.4.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "6093bad37da69aab9d123a8091e4be0aa4a03e4d601ec641c327398315f62b64" +checksum = "ff1bc3d3f05aff0403e8ac0d92ced918ec05b666a43f83297ccef5bea8a3d449" dependencies = [ "winapi-util", ] @@ -3368,7 +3378,7 @@ dependencies = [ "pin-project-lite", "socket2 0.5.5", "tokio-macros", - "windows-sys", + "windows-sys 0.48.0", ] [[package]] @@ -3399,7 +3409,7 @@ version = "0.24.1" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "c28327cf380ac148141087fbfb9de9d7bd4e84ab5d2c28fbc911d753de8a7081" dependencies = [ - "rustls 0.21.8", + "rustls 0.21.9", "tokio", ] @@ -3577,9 +3587,9 @@ checksum = "8ecb6da28b8a351d773b68d5825ac39017e680750f980f3a1a85cd8dd28a47c1" [[package]] name = "url" -version = "2.4.1" +version = "2.5.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "143b538f18257fac9cad154828a57c6bf5157e1aa604d4816b5995bf6de87ae5" +checksum = "31e6302e3bb753d46e83516cae55ae196fc0c309407cf11ab35cc51a4c2a4633" dependencies = [ "form_urlencoded", "idna", @@ -3600,9 +3610,9 @@ checksum = "711b9620af191e0cdc7468a8d14e709c3dcdb115b36f838e601583af800a370a" [[package]] name = "uuid" -version = "1.5.0" +version = "1.6.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "88ad59a7560b41a70d191093a945f0b87bc1deeda46fb237479708a1d6b6cdfc" +checksum = "5e395fcf16a7a3d8127ec99782007af141946b4795001f876d54fb0d55978560" dependencies = [ "getrandom", "serde", @@ -3656,9 +3666,9 @@ checksum = "9c8d87e72b64a3b4db28d11ce29237c246188f4f51057d65a7eab63b7987e423" [[package]] name = "wasm-bindgen" -version = "0.2.88" +version = "0.2.89" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "7daec296f25a1bae309c0cd5c29c4b260e510e6d813c286b19eaadf409d40fce" +checksum = "0ed0d4f68a3015cc185aff4db9506a015f4b96f95303897bfa23f846db54064e" dependencies = [ "cfg-if", "wasm-bindgen-macro", @@ -3666,9 +3676,9 @@ dependencies = [ [[package]] name = "wasm-bindgen-backend" -version = "0.2.88" +version = "0.2.89" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "e397f4664c0e4e428e8313a469aaa58310d302159845980fd23b0f22a847f217" +checksum = "1b56f625e64f3a1084ded111c4d5f477df9f8c92df113852fa5a374dbda78826" dependencies = [ "bumpalo", "log", @@ -3681,9 +3691,9 @@ dependencies = [ [[package]] name = "wasm-bindgen-futures" -version = "0.4.38" +version = "0.4.39" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "9afec9963e3d0994cac82455b2b3502b81a7f40f9a0d32181f7528d9f4b43e02" +checksum = "ac36a15a220124ac510204aec1c3e5db8a22ab06fd6706d881dc6149f8ed9a12" dependencies = [ "cfg-if", "js-sys", @@ -3693,9 +3703,9 @@ dependencies = [ [[package]] name = "wasm-bindgen-macro" -version = "0.2.88" +version = "0.2.89" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "5961017b3b08ad5f3fe39f1e79877f8ee7c23c5e5fd5eb80de95abc41f1f16b2" +checksum = "0162dbf37223cd2afce98f3d0785506dcb8d266223983e4b5b525859e6e182b2" dependencies = [ "quote", "wasm-bindgen-macro-support", @@ -3703,9 +3713,9 @@ dependencies = [ [[package]] name = "wasm-bindgen-macro-support" -version = "0.2.88" +version = "0.2.89" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "c5353b8dab669f5e10f5bd76df26a9360c748f054f862ff5f3f8aae0c7fb3907" +checksum = "f0eb82fcb7930ae6219a7ecfd55b217f5f0893484b7a13022ebb2b2bf20b5283" dependencies = [ "proc-macro2", "quote", @@ -3716,9 +3726,9 @@ dependencies = [ [[package]] name = "wasm-bindgen-shared" -version = "0.2.88" +version = "0.2.89" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "0d046c5d029ba91a1ed14da14dca44b68bf2f124cfbaf741c54151fdb3e0750b" +checksum = "7ab9b36309365056cd639da3134bf87fa8f3d86008abf99e612384a6eecd459f" [[package]] name = "wasm-streams" @@ -3735,9 +3745,9 @@ dependencies = [ [[package]] name = "web-sys" -version = "0.3.65" +version = "0.3.66" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "5db499c5f66323272151db0e666cd34f78617522fb0c1604d31a27c50c206a85" +checksum = "50c24a44ec86bb68fbecd1b3efed7e85ea5621b39b35ef2766b66cd984f8010f" dependencies = [ "js-sys", "wasm-bindgen", @@ -3749,15 +3759,15 @@ version = "0.22.4" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "ed63aea5ce73d0ff405984102c42de94fc55a6b75765d621c65262469b3c9b53" dependencies = [ - "ring 0.17.5", + "ring 0.17.6", "untrusted 0.9.0", ] [[package]] name = "webpki-roots" -version = "0.25.2" +version = "0.25.3" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "14247bb57be4f377dfb94c72830b8ce8fc6beac03cf4bf7b9732eadd414123fc" +checksum = "1778a42e8b3b90bff8d0f5032bf22250792889a5cdc752aa0020c84abe3aaf10" [[package]] name = "winapi" @@ -3796,7 +3806,7 @@ version = "0.51.1" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "f1f8cf84f35d2db49a46868f947758c7a1138116f7fac3bc844f43ade1292e64" dependencies = [ - "windows-targets", + "windows-targets 0.48.5", ] [[package]] @@ -3805,7 +3815,16 @@ version = "0.48.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "677d2418bec65e3338edb076e806bc1ec15693c5d0104683f2efe857f61056a9" dependencies = [ - "windows-targets", + "windows-targets 0.48.5", +] + +[[package]] +name = "windows-sys" +version = "0.52.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "282be5f36a8ce781fad8c8ae18fa3f9beff57ec1b52cb3de0789201425d9a33d" +dependencies = [ + "windows-targets 0.52.0", ] [[package]] @@ -3814,13 +3833,28 @@ version = "0.48.5" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "9a2fa6e2155d7247be68c096456083145c183cbbbc2764150dda45a87197940c" dependencies = [ - "windows_aarch64_gnullvm", - "windows_aarch64_msvc", - "windows_i686_gnu", - "windows_i686_msvc", - "windows_x86_64_gnu", - "windows_x86_64_gnullvm", - "windows_x86_64_msvc", + "windows_aarch64_gnullvm 0.48.5", + "windows_aarch64_msvc 0.48.5", + "windows_i686_gnu 0.48.5", + "windows_i686_msvc 0.48.5", + "windows_x86_64_gnu 0.48.5", + "windows_x86_64_gnullvm 0.48.5", + "windows_x86_64_msvc 0.48.5", +] + +[[package]] +name = "windows-targets" +version = "0.52.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "8a18201040b24831fbb9e4eb208f8892e1f50a37feb53cc7ff887feb8f50e7cd" +dependencies = [ + "windows_aarch64_gnullvm 0.52.0", + "windows_aarch64_msvc 0.52.0", + "windows_i686_gnu 0.52.0", + "windows_i686_msvc 0.52.0", + "windows_x86_64_gnu 0.52.0", + "windows_x86_64_gnullvm 0.52.0", + "windows_x86_64_msvc 0.52.0", ] [[package]] @@ -3829,42 +3863,84 @@ version = "0.48.5" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "2b38e32f0abccf9987a4e3079dfb67dcd799fb61361e53e2882c3cbaf0d905d8" +[[package]] +name = "windows_aarch64_gnullvm" +version = "0.52.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "cb7764e35d4db8a7921e09562a0304bf2f93e0a51bfccee0bd0bb0b666b015ea" + [[package]] name = "windows_aarch64_msvc" version = "0.48.5" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "dc35310971f3b2dbbf3f0690a219f40e2d9afcf64f9ab7cc1be722937c26b4bc" +[[package]] +name = "windows_aarch64_msvc" +version = "0.52.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "bbaa0368d4f1d2aaefc55b6fcfee13f41544ddf36801e793edbbfd7d7df075ef" + [[package]] name = "windows_i686_gnu" version = "0.48.5" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "a75915e7def60c94dcef72200b9a8e58e5091744960da64ec734a6c6e9b3743e" +[[package]] +name = "windows_i686_gnu" +version = "0.52.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "a28637cb1fa3560a16915793afb20081aba2c92ee8af57b4d5f28e4b3e7df313" + [[package]] name = "windows_i686_msvc" version = "0.48.5" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "8f55c233f70c4b27f66c523580f78f1004e8b5a8b659e05a4eb49d4166cca406" +[[package]] +name = "windows_i686_msvc" +version = "0.52.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "ffe5e8e31046ce6230cc7215707b816e339ff4d4d67c65dffa206fd0f7aa7b9a" + [[package]] name = "windows_x86_64_gnu" version = "0.48.5" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "53d40abd2583d23e4718fddf1ebec84dbff8381c07cae67ff7768bbf19c6718e" +[[package]] +name = "windows_x86_64_gnu" +version = "0.52.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "3d6fa32db2bc4a2f5abeacf2b69f7992cd09dca97498da74a151a3132c26befd" + [[package]] name = "windows_x86_64_gnullvm" version = "0.48.5" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "0b7b52767868a23d5bab768e390dc5f5c55825b6d30b86c844ff2dc7414044cc" +[[package]] +name = "windows_x86_64_gnullvm" +version = "0.52.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "1a657e1e9d3f514745a572a6846d3c7aa7dbe1658c056ed9c3344c4109a6949e" + [[package]] name = "windows_x86_64_msvc" version = "0.48.5" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "ed94fce61571a4006852b7389a063ab983c02eb1bb37b47f8272ce92d06d9538" +[[package]] +name = "windows_x86_64_msvc" +version = "0.52.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "dff9641d1cd4be8d1a070daf9e3773c5f67e78b4d9d42263020c057706765c04" + [[package]] name = "winreg" version = "0.50.0" @@ -3872,7 +3948,7 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "524e57b2c537c0f9b1e69f1965311ec12182b4122e45035b1508cd24d2adadb1" dependencies = [ "cfg-if", - "windows-sys", + "windows-sys 0.48.0", ] [[package]] @@ -3892,18 +3968,18 @@ dependencies = [ [[package]] name = "zerocopy" -version = "0.7.25" +version = "0.7.29" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "8cd369a67c0edfef15010f980c3cbe45d7f651deac2cd67ce097cd801de16557" +checksum = "5d075cf85bbb114e933343e087b92f2146bac0d55b534cbb8188becf0039948e" dependencies = [ "zerocopy-derive", ] [[package]] name = "zerocopy-derive" -version = "0.7.25" +version = "0.7.29" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "c2f140bda219a26ccc0cdb03dba58af72590c53b22642577d88a927bc5c87d6b" +checksum = "86cd5ca076997b97ef09d3ad65efe811fa68c9e874cb636ccb211223a813b0c2" dependencies = [ "proc-macro2", "quote", @@ -3912,9 +3988,9 @@ dependencies = [ [[package]] name = "zeroize" -version = "1.6.0" +version = "1.7.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "2a0956f1ba7c7909bfb66c2e9e4124ab6f6482560f6628b5aaeba39207c9aad9" +checksum = "525b4ec142c6b68a2d10f01f7bbf6755599ca3f81ea53b8431b7dd348f5fdb2d" [[package]] name = "zstd" diff --git a/datafusion/sql/src/statement.rs b/datafusion/sql/src/statement.rs index aa2f0583cb99..a64010a7c3db 100644 --- a/datafusion/sql/src/statement.rs +++ b/datafusion/sql/src/statement.rs @@ -458,6 +458,9 @@ impl<'a, S: ContextProvider> SqlToRel<'a, S> { if ignore { plan_err!("Insert-ignore clause not supported")?; } + let Some(source) = source else { + plan_err!("Inserts without a source not supported")? + }; let _ = into; // optional keyword doesn't change behavior self.insert_to_plan(table_name, columns, source, overwrite) } @@ -566,7 +569,10 @@ impl<'a, S: ContextProvider> SqlToRel<'a, S> { }); Ok(LogicalPlan::Statement(statement)) } - Statement::Rollback { chain } => { + Statement::Rollback { chain, savepoint } => { + if savepoint.is_some() { + plan_err!("Savepoints not supported")?; + } let statement = PlanStatement::TransactionEnd(TransactionEnd { conclusion: TransactionConclusion::Rollback, chain, From 6dd3c95bed44b6af6cc9b59d8198693f9dc92339 Mon Sep 17 00:00:00 2001 From: Alex Huang Date: Tue, 5 Dec 2023 21:43:41 +0100 Subject: [PATCH 365/572] feat: support `LargeList` for `array_has`, `array_has_all` and `array_has_any` (#8322) * support LargeList for array_has, array_has_all and array_has_any * simplify the code --------- Co-authored-by: Andrew Lamb --- .../physical-expr/src/array_expressions.rs | 143 +++++++++++------- datafusion/sqllogictest/test_files/array.slt | 111 ++++++++++++++ 2 files changed, 201 insertions(+), 53 deletions(-) diff --git a/datafusion/physical-expr/src/array_expressions.rs b/datafusion/physical-expr/src/array_expressions.rs index 9489a51fa385..6104566450c3 100644 --- a/datafusion/physical-expr/src/array_expressions.rs +++ b/datafusion/physical-expr/src/array_expressions.rs @@ -1765,82 +1765,119 @@ pub fn array_ndims(args: &[ArrayRef]) -> Result { } } -/// Array_has SQL function -pub fn array_has(args: &[ArrayRef]) -> Result { - let array = as_list_array(&args[0])?; - let element = &args[1]; +/// Represents the type of comparison for array_has. +#[derive(Debug, PartialEq)] +enum ComparisonType { + // array_has_all + All, + // array_has_any + Any, + // array_has + Single, +} + +fn general_array_has_dispatch( + array: &ArrayRef, + sub_array: &ArrayRef, + comparison_type: ComparisonType, +) -> Result { + let array = if comparison_type == ComparisonType::Single { + let arr = as_generic_list_array::(array)?; + check_datatypes("array_has", &[arr.values(), sub_array])?; + arr + } else { + check_datatypes("array_has", &[array, sub_array])?; + as_generic_list_array::(array)? + }; - check_datatypes("array_has", &[array.values(), element])?; let mut boolean_builder = BooleanArray::builder(array.len()); let converter = RowConverter::new(vec![SortField::new(array.value_type())])?; - let r_values = converter.convert_columns(&[element.clone()])?; - for (row_idx, arr) in array.iter().enumerate() { - if let Some(arr) = arr { + + let element = sub_array.clone(); + let sub_array = if comparison_type != ComparisonType::Single { + as_generic_list_array::(sub_array)? + } else { + array + }; + + for (row_idx, (arr, sub_arr)) in array.iter().zip(sub_array.iter()).enumerate() { + if let (Some(arr), Some(sub_arr)) = (arr, sub_arr) { let arr_values = converter.convert_columns(&[arr])?; - let res = arr_values - .iter() - .dedup() - .any(|x| x == r_values.row(row_idx)); + let sub_arr_values = if comparison_type != ComparisonType::Single { + converter.convert_columns(&[sub_arr])? + } else { + converter.convert_columns(&[element.clone()])? + }; + + let mut res = match comparison_type { + ComparisonType::All => sub_arr_values + .iter() + .dedup() + .all(|elem| arr_values.iter().dedup().any(|x| x == elem)), + ComparisonType::Any => sub_arr_values + .iter() + .dedup() + .any(|elem| arr_values.iter().dedup().any(|x| x == elem)), + ComparisonType::Single => arr_values + .iter() + .dedup() + .any(|x| x == sub_arr_values.row(row_idx)), + }; + + if comparison_type == ComparisonType::Any { + res |= res; + } + boolean_builder.append_value(res); } } Ok(Arc::new(boolean_builder.finish())) } -/// Array_has_any SQL function -pub fn array_has_any(args: &[ArrayRef]) -> Result { - check_datatypes("array_has_any", &[&args[0], &args[1]])?; +/// Array_has SQL function +pub fn array_has(args: &[ArrayRef]) -> Result { + let array_type = args[0].data_type(); - let array = as_list_array(&args[0])?; - let sub_array = as_list_array(&args[1])?; - let mut boolean_builder = BooleanArray::builder(array.len()); + match array_type { + DataType::List(_) => { + general_array_has_dispatch::(&args[0], &args[1], ComparisonType::Single) + } + DataType::LargeList(_) => { + general_array_has_dispatch::(&args[0], &args[1], ComparisonType::Single) + } + _ => internal_err!("array_has does not support type '{array_type:?}'."), + } +} - let converter = RowConverter::new(vec![SortField::new(array.value_type())])?; - for (arr, sub_arr) in array.iter().zip(sub_array.iter()) { - if let (Some(arr), Some(sub_arr)) = (arr, sub_arr) { - let arr_values = converter.convert_columns(&[arr])?; - let sub_arr_values = converter.convert_columns(&[sub_arr])?; +/// Array_has_any SQL function +pub fn array_has_any(args: &[ArrayRef]) -> Result { + let array_type = args[0].data_type(); - let mut res = false; - for elem in sub_arr_values.iter().dedup() { - res |= arr_values.iter().dedup().any(|x| x == elem); - if res { - break; - } - } - boolean_builder.append_value(res); + match array_type { + DataType::List(_) => { + general_array_has_dispatch::(&args[0], &args[1], ComparisonType::Any) } + DataType::LargeList(_) => { + general_array_has_dispatch::(&args[0], &args[1], ComparisonType::Any) + } + _ => internal_err!("array_has_any does not support type '{array_type:?}'."), } - Ok(Arc::new(boolean_builder.finish())) } /// Array_has_all SQL function pub fn array_has_all(args: &[ArrayRef]) -> Result { - check_datatypes("array_has_all", &[&args[0], &args[1]])?; - - let array = as_list_array(&args[0])?; - let sub_array = as_list_array(&args[1])?; - - let mut boolean_builder = BooleanArray::builder(array.len()); - - let converter = RowConverter::new(vec![SortField::new(array.value_type())])?; - for (arr, sub_arr) in array.iter().zip(sub_array.iter()) { - if let (Some(arr), Some(sub_arr)) = (arr, sub_arr) { - let arr_values = converter.convert_columns(&[arr])?; - let sub_arr_values = converter.convert_columns(&[sub_arr])?; + let array_type = args[0].data_type(); - let mut res = true; - for elem in sub_arr_values.iter().dedup() { - res &= arr_values.iter().dedup().any(|x| x == elem); - if !res { - break; - } - } - boolean_builder.append_value(res); + match array_type { + DataType::List(_) => { + general_array_has_dispatch::(&args[0], &args[1], ComparisonType::All) + } + DataType::LargeList(_) => { + general_array_has_dispatch::(&args[0], &args[1], ComparisonType::All) } + _ => internal_err!("array_has_all does not support type '{array_type:?}'."), } - Ok(Arc::new(boolean_builder.finish())) } /// Splits string at occurrences of delimiter and returns an array of parts diff --git a/datafusion/sqllogictest/test_files/array.slt b/datafusion/sqllogictest/test_files/array.slt index 6ec2b2cb013b..d8bf441d7169 100644 --- a/datafusion/sqllogictest/test_files/array.slt +++ b/datafusion/sqllogictest/test_files/array.slt @@ -2621,6 +2621,23 @@ select array_has(make_array(1,2), 1), ---- true true true true true false true false true false true false +query BBBBBBBBBBBB +select array_has(arrow_cast(make_array(1,2), 'LargeList(Int64)'), 1), + array_has(arrow_cast(make_array(1,2,NULL), 'LargeList(Int64)'), 1), + array_has(arrow_cast(make_array([2,3], [3,4]), 'LargeList(List(Int64))'), make_array(2,3)), + array_has(arrow_cast(make_array([[1], [2,3]], [[4,5], [6]]), 'LargeList(List(List(Int64)))'), make_array([1], [2,3])), + array_has(arrow_cast(make_array([[1], [2,3]], [[4,5], [6]]), 'LargeList(List(List(Int64)))'), make_array([4,5], [6])), + array_has(arrow_cast(make_array([[1], [2,3]], [[4,5], [6]]), 'LargeList(List(List(Int64)))'), make_array([1])), + array_has(arrow_cast(make_array([[[1]]]), 'LargeList(List(List(List(Int64))))'), make_array([[1]])), + array_has(arrow_cast(make_array([[[1]]], [[[1], [2]]]), 'LargeList(List(List(List(Int64))))'), make_array([[2]])), + array_has(arrow_cast(make_array([[[1]]], [[[1], [2]]]), 'LargeList(List(List(List(Int64))))'), make_array([[1], [2]])), + list_has(arrow_cast(make_array(1,2,3), 'LargeList(Int64)'), 4), + array_contains(arrow_cast(make_array(1,2,3), 'LargeList(Int64)'), 3), + list_contains(arrow_cast(make_array(1,2,3), 'LargeList(Int64)'), 0) +; +---- +true true true true true false true false true false true false + query BBB select array_has(column1, column2), array_has_all(column3, column4), @@ -2630,6 +2647,15 @@ from array_has_table_1D; true true true false false false +query BBB +select array_has(arrow_cast(column1, 'LargeList(Int64)'), column2), + array_has_all(arrow_cast(column3, 'LargeList(Int64)'), arrow_cast(column4, 'LargeList(Int64)')), + array_has_any(arrow_cast(column5, 'LargeList(Int64)'), arrow_cast(column6, 'LargeList(Int64)')) +from array_has_table_1D; +---- +true true true +false false false + query BBB select array_has(column1, column2), array_has_all(column3, column4), @@ -2639,6 +2665,15 @@ from array_has_table_1D_Float; true true false false false true +query BBB +select array_has(arrow_cast(column1, 'LargeList(Float64)'), column2), + array_has_all(arrow_cast(column3, 'LargeList(Float64)'), arrow_cast(column4, 'LargeList(Float64)')), + array_has_any(arrow_cast(column5, 'LargeList(Float64)'), arrow_cast(column6, 'LargeList(Float64)')) +from array_has_table_1D_Float; +---- +true true false +false false true + query BBB select array_has(column1, column2), array_has_all(column3, column4), @@ -2648,6 +2683,15 @@ from array_has_table_1D_Boolean; false true true true true true +query BBB +select array_has(arrow_cast(column1, 'LargeList(Boolean)'), column2), + array_has_all(arrow_cast(column3, 'LargeList(Boolean)'), arrow_cast(column4, 'LargeList(Boolean)')), + array_has_any(arrow_cast(column5, 'LargeList(Boolean)'), arrow_cast(column6, 'LargeList(Boolean)')) +from array_has_table_1D_Boolean; +---- +false true true +true true true + query BBB select array_has(column1, column2), array_has_all(column3, column4), @@ -2657,6 +2701,15 @@ from array_has_table_1D_UTF8; true true false false false true +query BBB +select array_has(arrow_cast(column1, 'LargeList(Utf8)'), column2), + array_has_all(arrow_cast(column3, 'LargeList(Utf8)'), arrow_cast(column4, 'LargeList(Utf8)')), + array_has_any(arrow_cast(column5, 'LargeList(Utf8)'), arrow_cast(column6, 'LargeList(Utf8)')) +from array_has_table_1D_UTF8; +---- +true true false +false false true + query BB select array_has(column1, column2), array_has_all(column3, column4) @@ -2665,6 +2718,14 @@ from array_has_table_2D; false true true false +query BB +select array_has(arrow_cast(column1, 'LargeList(List(Int64))'), column2), + array_has_all(arrow_cast(column3, 'LargeList(List(Int64))'), arrow_cast(column4, 'LargeList(List(Int64))')) +from array_has_table_2D; +---- +false true +true false + query B select array_has_all(column1, column2) from array_has_table_2D_float; @@ -2672,6 +2733,13 @@ from array_has_table_2D_float; true false +query B +select array_has_all(arrow_cast(column1, 'LargeList(List(Float64))'), arrow_cast(column2, 'LargeList(List(Float64))')) +from array_has_table_2D_float; +---- +true +false + query B select array_has(column1, column2) from array_has_table_3D; ---- @@ -2683,6 +2751,17 @@ true false true +query B +select array_has(arrow_cast(column1, 'LargeList(List(List(Int64)))'), column2) from array_has_table_3D; +---- +false +true +false +false +true +false +true + query BBBB select array_has(column1, make_array(5, 6)), array_has(column1, make_array(7, NULL)), @@ -2697,6 +2776,20 @@ false true false false false false false false false false false false +query BBBB +select array_has(arrow_cast(column1, 'LargeList(List(Int64))'), make_array(5, 6)), + array_has(arrow_cast(column1, 'LargeList(List(Int64))'), make_array(7, NULL)), + array_has(arrow_cast(column2, 'LargeList(Float64)'), 5.5), + array_has(arrow_cast(column3, 'LargeList(Utf8)'), 'o') +from arrays; +---- +false false false true +true false true false +true false false true +false true false false +false false false false +false false false false + query BBBBBBBBBBBBB select array_has_all(make_array(1,2,3), make_array(1,3)), array_has_all(make_array(1,2,3), make_array(1,4)), @@ -2715,6 +2808,24 @@ select array_has_all(make_array(1,2,3), make_array(1,3)), ---- true false true false false false true true false false true false true +query BBBBBBBBBBBBB +select array_has_all(arrow_cast(make_array(1,2,3), 'LargeList(Int64)'), arrow_cast(make_array(1,3), 'LargeList(Int64)')), + array_has_all(arrow_cast(make_array(1,2,3),'LargeList(Int64)'), arrow_cast(make_array(1,4), 'LargeList(Int64)')), + array_has_all(arrow_cast(make_array([1,2], [3,4]), 'LargeList(List(Int64))'), arrow_cast(make_array([1,2]), 'LargeList(List(Int64))')), + array_has_all(arrow_cast(make_array([1,2], [3,4]), 'LargeList(List(Int64))'), arrow_cast(make_array([1,3]), 'LargeList(List(Int64))')), + array_has_all(arrow_cast(make_array([1,2], [3,4]), 'LargeList(List(Int64))'), arrow_cast(make_array([1,2], [3,4], [5,6]), 'LargeList(List(Int64))')), + array_has_all(arrow_cast(make_array([[1,2,3]]), 'LargeList(List(List(Int64)))'), arrow_cast(make_array([[1]]), 'LargeList(List(List(Int64)))')), + array_has_all(arrow_cast(make_array([[1,2,3]]), 'LargeList(List(List(Int64)))'), arrow_cast(make_array([[1,2,3]]), 'LargeList(List(List(Int64)))')), + array_has_any(arrow_cast(make_array(1,2,3),'LargeList(Int64)'), arrow_cast(make_array(1,10,100), 'LargeList(Int64)')), + array_has_any(arrow_cast(make_array(1,2,3),'LargeList(Int64)'), arrow_cast(make_array(10,100),'LargeList(Int64)')), + array_has_any(arrow_cast(make_array([1,2], [3,4]), 'LargeList(List(Int64))'), arrow_cast(make_array([1,10], [10,4]), 'LargeList(List(Int64))')), + array_has_any(arrow_cast(make_array([1,2], [3,4]), 'LargeList(List(Int64))'), arrow_cast(make_array([10,20], [3,4]), 'LargeList(List(Int64))')), + array_has_any(arrow_cast(make_array([[1,2,3]]), 'LargeList(List(List(Int64)))'), arrow_cast(make_array([[1,2,3], [4,5,6]]), 'LargeList(List(List(Int64)))')), + array_has_any(arrow_cast(make_array([[1,2,3]]), 'LargeList(List(List(Int64)))'), arrow_cast(make_array([[1,2,3]], [[4,5,6]]), 'LargeList(List(List(Int64)))')) +; +---- +true false true false false false true true false false true false true + query ??? select array_intersect(column1, column2), array_intersect(column3, column4), From 4ceb2dec7a709a1ecb36955673574ce2df36baf2 Mon Sep 17 00:00:00 2001 From: jakevin Date: Wed, 6 Dec 2023 04:43:50 +0800 Subject: [PATCH 366/572] Union `schema` can't be a subset of the child schema (#8408) Co-authored-by: Andrew Lamb --- datafusion/core/src/physical_planner.rs | 11 ++----- datafusion/physical-plan/src/union.rs | 34 +------------------- datafusion/sqllogictest/test_files/union.slt | 5 +++ 3 files changed, 8 insertions(+), 42 deletions(-) diff --git a/datafusion/core/src/physical_planner.rs b/datafusion/core/src/physical_planner.rs index 0e96b126b967..47d071d533e3 100644 --- a/datafusion/core/src/physical_planner.rs +++ b/datafusion/core/src/physical_planner.rs @@ -919,17 +919,10 @@ impl DefaultPhysicalPlanner { )?; Ok(Arc::new(FilterExec::try_new(runtime_expr, physical_input)?)) } - LogicalPlan::Union(Union { inputs, schema }) => { + LogicalPlan::Union(Union { inputs, .. }) => { let physical_plans = self.create_initial_plan_multi(inputs.iter().map(|lp| lp.as_ref()), session_state).await?; - if schema.fields().len() < physical_plans[0].schema().fields().len() { - // `schema` could be a subset of the child schema. For example - // for query "select count(*) from (select a from t union all select a from t)" - // `schema` is empty but child schema contains one field `a`. - Ok(Arc::new(UnionExec::try_new_with_schema(physical_plans, schema.clone())?)) - } else { - Ok(Arc::new(UnionExec::new(physical_plans))) - } + Ok(Arc::new(UnionExec::new(physical_plans))) } LogicalPlan::Repartition(Repartition { input, diff --git a/datafusion/physical-plan/src/union.rs b/datafusion/physical-plan/src/union.rs index 9700605ce406..92ad0f4e65db 100644 --- a/datafusion/physical-plan/src/union.rs +++ b/datafusion/physical-plan/src/union.rs @@ -38,7 +38,7 @@ use crate::stream::ObservedStream; use arrow::datatypes::{Field, Schema, SchemaRef}; use arrow::record_batch::RecordBatch; use datafusion_common::stats::Precision; -use datafusion_common::{exec_err, internal_err, DFSchemaRef, DataFusionError, Result}; +use datafusion_common::{exec_err, internal_err, DataFusionError, Result}; use datafusion_execution::TaskContext; use datafusion_physical_expr::EquivalenceProperties; @@ -95,38 +95,6 @@ pub struct UnionExec { } impl UnionExec { - /// Create a new UnionExec with specified schema. - /// The `schema` should always be a subset of the schema of `inputs`, - /// otherwise, an error will be returned. - pub fn try_new_with_schema( - inputs: Vec>, - schema: DFSchemaRef, - ) -> Result { - let mut exec = Self::new(inputs); - let exec_schema = exec.schema(); - let fields = schema - .fields() - .iter() - .map(|dff| { - exec_schema - .field_with_name(dff.name()) - .cloned() - .map_err(|_| { - DataFusionError::Internal(format!( - "Cannot find the field {:?} in child schema", - dff.name() - )) - }) - }) - .collect::>>()?; - let schema = Arc::new(Schema::new_with_metadata( - fields, - exec.schema().metadata().clone(), - )); - exec.schema = schema; - Ok(exec) - } - /// Create a new UnionExec pub fn new(inputs: Vec>) -> Self { let schema = union_schema(&inputs); diff --git a/datafusion/sqllogictest/test_files/union.slt b/datafusion/sqllogictest/test_files/union.slt index 0f255cdb9fb9..2c8970a13927 100644 --- a/datafusion/sqllogictest/test_files/union.slt +++ b/datafusion/sqllogictest/test_files/union.slt @@ -82,6 +82,11 @@ SELECT 2 as x 1 2 +query I +select count(*) from (select id from t1 union all select id from t2) +---- +6 + # csv_union_all statement ok CREATE EXTERNAL TABLE aggregate_test_100 ( From e322839df4ed89e24d1650436975e03645696793 Mon Sep 17 00:00:00 2001 From: Andrew Lamb Date: Tue, 5 Dec 2023 15:43:58 -0500 Subject: [PATCH 367/572] Move `PartitionSearchMode` into datafusion_physical_plan, rename to `InputOrderMode` (#8364) * Move PartitionSearchMode into datafusion_physical_plan * Improve comments * Rename to InputOrderMode * Update prost --- .../src/physical_optimizer/enforce_sorting.rs | 7 +- .../core/src/physical_optimizer/test_utils.rs | 5 +- datafusion/core/src/physical_planner.rs | 10 +-- .../core/tests/fuzz_cases/window_fuzz.rs | 12 ++- .../physical-plan/src/aggregates/mod.rs | 30 ++++---- .../physical-plan/src/aggregates/order/mod.rs | 12 ++- .../physical-plan/src/aggregates/row_hash.rs | 2 +- datafusion/physical-plan/src/lib.rs | 2 + datafusion/physical-plan/src/ordering.rs | 51 +++++++++++++ .../src/windows/bounded_window_agg_exec.rs | 39 +++++----- datafusion/physical-plan/src/windows/mod.rs | 76 +++++++------------ datafusion/proto/proto/datafusion.proto | 6 +- datafusion/proto/src/generated/pbjson.rs | 42 +++++----- datafusion/proto/src/generated/prost.rs | 12 ++- datafusion/proto/src/physical_plan/mod.rs | 57 ++++++-------- 15 files changed, 188 insertions(+), 175 deletions(-) create mode 100644 datafusion/physical-plan/src/ordering.rs diff --git a/datafusion/core/src/physical_optimizer/enforce_sorting.rs b/datafusion/core/src/physical_optimizer/enforce_sorting.rs index ff052b5f040c..14715ede500a 100644 --- a/datafusion/core/src/physical_optimizer/enforce_sorting.rs +++ b/datafusion/core/src/physical_optimizer/enforce_sorting.rs @@ -53,14 +53,15 @@ use crate::physical_plan::sorts::sort_preserving_merge::SortPreservingMergeExec; use crate::physical_plan::windows::{ get_best_fitting_window, BoundedWindowAggExec, WindowAggExec, }; -use crate::physical_plan::{with_new_children_if_necessary, Distribution, ExecutionPlan}; +use crate::physical_plan::{ + with_new_children_if_necessary, Distribution, ExecutionPlan, InputOrderMode, +}; use datafusion_common::tree_node::{Transformed, TreeNode, VisitRecursion}; use datafusion_common::{plan_err, DataFusionError}; use datafusion_physical_expr::{PhysicalSortExpr, PhysicalSortRequirement}; use datafusion_physical_plan::repartition::RepartitionExec; -use datafusion_physical_plan::windows::PartitionSearchMode; use itertools::izip; /// This rule inspects [`SortExec`]'s in the given physical plan and removes the @@ -611,7 +612,7 @@ fn analyze_window_sort_removal( window_expr.to_vec(), window_child, partitionby_exprs.to_vec(), - PartitionSearchMode::Sorted, + InputOrderMode::Sorted, )?) as _ } else { Arc::new(WindowAggExec::try_new( diff --git a/datafusion/core/src/physical_optimizer/test_utils.rs b/datafusion/core/src/physical_optimizer/test_utils.rs index cc62cda41266..37a76eff1ee2 100644 --- a/datafusion/core/src/physical_optimizer/test_utils.rs +++ b/datafusion/core/src/physical_optimizer/test_utils.rs @@ -35,7 +35,7 @@ use crate::physical_plan::sorts::sort::SortExec; use crate::physical_plan::sorts::sort_preserving_merge::SortPreservingMergeExec; use crate::physical_plan::union::UnionExec; use crate::physical_plan::windows::create_window_expr; -use crate::physical_plan::{ExecutionPlan, Partitioning}; +use crate::physical_plan::{ExecutionPlan, InputOrderMode, Partitioning}; use crate::prelude::{CsvReadOptions, SessionContext}; use arrow_schema::{Schema, SchemaRef, SortOptions}; @@ -44,7 +44,6 @@ use datafusion_execution::object_store::ObjectStoreUrl; use datafusion_expr::{AggregateFunction, WindowFrame, WindowFunction}; use datafusion_physical_expr::expressions::col; use datafusion_physical_expr::{PhysicalExpr, PhysicalSortExpr}; -use datafusion_physical_plan::windows::PartitionSearchMode; use async_trait::async_trait; @@ -240,7 +239,7 @@ pub fn bounded_window_exec( .unwrap()], input.clone(), vec![], - PartitionSearchMode::Sorted, + InputOrderMode::Sorted, ) .unwrap(), ) diff --git a/datafusion/core/src/physical_planner.rs b/datafusion/core/src/physical_planner.rs index 47d071d533e3..8ef433173edd 100644 --- a/datafusion/core/src/physical_planner.rs +++ b/datafusion/core/src/physical_planner.rs @@ -63,12 +63,10 @@ use crate::physical_plan::sorts::sort::SortExec; use crate::physical_plan::union::UnionExec; use crate::physical_plan::unnest::UnnestExec; use crate::physical_plan::values::ValuesExec; -use crate::physical_plan::windows::{ - BoundedWindowAggExec, PartitionSearchMode, WindowAggExec, -}; +use crate::physical_plan::windows::{BoundedWindowAggExec, WindowAggExec}; use crate::physical_plan::{ - aggregates, displayable, udaf, windows, AggregateExpr, ExecutionPlan, Partitioning, - PhysicalExpr, WindowExpr, + aggregates, displayable, udaf, windows, AggregateExpr, ExecutionPlan, InputOrderMode, + Partitioning, PhysicalExpr, WindowExpr, }; use arrow::compute::SortOptions; @@ -761,7 +759,7 @@ impl DefaultPhysicalPlanner { window_expr, input_exec, physical_partition_keys, - PartitionSearchMode::Sorted, + InputOrderMode::Sorted, )?) } else { Arc::new(WindowAggExec::try_new( diff --git a/datafusion/core/tests/fuzz_cases/window_fuzz.rs b/datafusion/core/tests/fuzz_cases/window_fuzz.rs index af96063ffb5f..44ff71d02392 100644 --- a/datafusion/core/tests/fuzz_cases/window_fuzz.rs +++ b/datafusion/core/tests/fuzz_cases/window_fuzz.rs @@ -25,9 +25,9 @@ use arrow::util::pretty::pretty_format_batches; use datafusion::physical_plan::memory::MemoryExec; use datafusion::physical_plan::sorts::sort::SortExec; use datafusion::physical_plan::windows::{ - create_window_expr, BoundedWindowAggExec, PartitionSearchMode, WindowAggExec, + create_window_expr, BoundedWindowAggExec, WindowAggExec, }; -use datafusion::physical_plan::{collect, ExecutionPlan}; +use datafusion::physical_plan::{collect, ExecutionPlan, InputOrderMode}; use datafusion::prelude::{SessionConfig, SessionContext}; use datafusion_common::{Result, ScalarValue}; use datafusion_expr::type_coercion::aggregates::coerce_types; @@ -43,9 +43,7 @@ use hashbrown::HashMap; use rand::rngs::StdRng; use rand::{Rng, SeedableRng}; -use datafusion_physical_plan::windows::PartitionSearchMode::{ - Linear, PartiallySorted, Sorted, -}; +use datafusion_physical_plan::InputOrderMode::{Linear, PartiallySorted, Sorted}; #[tokio::test(flavor = "multi_thread", worker_threads = 16)] async fn window_bounded_window_random_comparison() -> Result<()> { @@ -385,9 +383,9 @@ async fn run_window_test( random_seed: u64, partition_by_columns: Vec<&str>, orderby_columns: Vec<&str>, - search_mode: PartitionSearchMode, + search_mode: InputOrderMode, ) -> Result<()> { - let is_linear = !matches!(search_mode, PartitionSearchMode::Sorted); + let is_linear = !matches!(search_mode, InputOrderMode::Sorted); let mut rng = StdRng::seed_from_u64(random_seed); let schema = input1[0].schema(); let session_config = SessionConfig::new().with_batch_size(50); diff --git a/datafusion/physical-plan/src/aggregates/mod.rs b/datafusion/physical-plan/src/aggregates/mod.rs index d594335af44f..2f69ed061ce1 100644 --- a/datafusion/physical-plan/src/aggregates/mod.rs +++ b/datafusion/physical-plan/src/aggregates/mod.rs @@ -27,11 +27,9 @@ use crate::aggregates::{ }; use crate::metrics::{ExecutionPlanMetricsSet, MetricsSet}; -use crate::windows::{ - get_ordered_partition_by_indices, get_window_mode, PartitionSearchMode, -}; +use crate::windows::{get_ordered_partition_by_indices, get_window_mode}; use crate::{ - DisplayFormatType, Distribution, ExecutionPlan, Partitioning, + DisplayFormatType, Distribution, ExecutionPlan, InputOrderMode, Partitioning, SendableRecordBatchStream, Statistics, }; @@ -304,7 +302,9 @@ pub struct AggregateExec { /// Execution metrics metrics: ExecutionPlanMetricsSet, required_input_ordering: Option, - partition_search_mode: PartitionSearchMode, + /// Describes how the input is ordered relative to the group by columns + input_order_mode: InputOrderMode, + /// Describe how the output is ordered output_ordering: Option, } @@ -409,15 +409,15 @@ fn get_aggregate_search_mode( aggr_expr: &mut [Arc], order_by_expr: &mut [Option], ordering_req: &mut Vec, -) -> PartitionSearchMode { +) -> InputOrderMode { let groupby_exprs = group_by .expr .iter() .map(|(item, _)| item.clone()) .collect::>(); - let mut partition_search_mode = PartitionSearchMode::Linear; + let mut input_order_mode = InputOrderMode::Linear; if !group_by.is_single() || groupby_exprs.is_empty() { - return partition_search_mode; + return input_order_mode; } if let Some((should_reverse, mode)) = @@ -439,9 +439,9 @@ fn get_aggregate_search_mode( ); *ordering_req = reverse_order_bys(ordering_req); } - partition_search_mode = mode; + input_order_mode = mode; } - partition_search_mode + input_order_mode } /// Check whether group by expression contains all of the expression inside `requirement` @@ -515,7 +515,7 @@ impl AggregateExec { &input.equivalence_properties(), )?; let mut ordering_req = requirement.unwrap_or(vec![]); - let partition_search_mode = get_aggregate_search_mode( + let input_order_mode = get_aggregate_search_mode( &group_by, &input, &mut aggr_expr, @@ -567,7 +567,7 @@ impl AggregateExec { metrics: ExecutionPlanMetricsSet::new(), required_input_ordering, limit: None, - partition_search_mode, + input_order_mode, output_ordering, }) } @@ -767,8 +767,8 @@ impl DisplayAs for AggregateExec { write!(f, ", lim=[{limit}]")?; } - if self.partition_search_mode != PartitionSearchMode::Linear { - write!(f, ", ordering_mode={:?}", self.partition_search_mode)?; + if self.input_order_mode != InputOrderMode::Linear { + write!(f, ", ordering_mode={:?}", self.input_order_mode)?; } } } @@ -819,7 +819,7 @@ impl ExecutionPlan for AggregateExec { /// infinite, returns an error to indicate this. fn unbounded_output(&self, children: &[bool]) -> Result { if children[0] { - if self.partition_search_mode == PartitionSearchMode::Linear { + if self.input_order_mode == InputOrderMode::Linear { // Cannot run without breaking pipeline. plan_err!( "Aggregate Error: `GROUP BY` clauses with columns without ordering and GROUPING SETS are not supported for unbounded inputs." diff --git a/datafusion/physical-plan/src/aggregates/order/mod.rs b/datafusion/physical-plan/src/aggregates/order/mod.rs index f72d2f06e459..b258b97a9e84 100644 --- a/datafusion/physical-plan/src/aggregates/order/mod.rs +++ b/datafusion/physical-plan/src/aggregates/order/mod.rs @@ -23,7 +23,7 @@ use datafusion_physical_expr::{EmitTo, PhysicalSortExpr}; mod full; mod partial; -use crate::windows::PartitionSearchMode; +use crate::InputOrderMode; pub(crate) use full::GroupOrderingFull; pub(crate) use partial::GroupOrderingPartial; @@ -42,18 +42,16 @@ impl GroupOrdering { /// Create a `GroupOrdering` for the the specified ordering pub fn try_new( input_schema: &Schema, - mode: &PartitionSearchMode, + mode: &InputOrderMode, ordering: &[PhysicalSortExpr], ) -> Result { match mode { - PartitionSearchMode::Linear => Ok(GroupOrdering::None), - PartitionSearchMode::PartiallySorted(order_indices) => { + InputOrderMode::Linear => Ok(GroupOrdering::None), + InputOrderMode::PartiallySorted(order_indices) => { GroupOrderingPartial::try_new(input_schema, order_indices, ordering) .map(GroupOrdering::Partial) } - PartitionSearchMode::Sorted => { - Ok(GroupOrdering::Full(GroupOrderingFull::new())) - } + InputOrderMode::Sorted => Ok(GroupOrdering::Full(GroupOrderingFull::new())), } } diff --git a/datafusion/physical-plan/src/aggregates/row_hash.rs b/datafusion/physical-plan/src/aggregates/row_hash.rs index 2f94c3630c33..89614fd3020c 100644 --- a/datafusion/physical-plan/src/aggregates/row_hash.rs +++ b/datafusion/physical-plan/src/aggregates/row_hash.rs @@ -346,7 +346,7 @@ impl GroupedHashAggregateStream { .find_longest_permutation(&agg_group_by.output_exprs()); let group_ordering = GroupOrdering::try_new( &group_schema, - &agg.partition_search_mode, + &agg.input_order_mode, ordering.as_slice(), )?; diff --git a/datafusion/physical-plan/src/lib.rs b/datafusion/physical-plan/src/lib.rs index b2c69b467e9c..f40911c10168 100644 --- a/datafusion/physical-plan/src/lib.rs +++ b/datafusion/physical-plan/src/lib.rs @@ -58,6 +58,7 @@ pub mod joins; pub mod limit; pub mod memory; pub mod metrics; +mod ordering; pub mod projection; pub mod repartition; pub mod sorts; @@ -72,6 +73,7 @@ pub mod windows; pub use crate::display::{DefaultDisplay, DisplayAs, DisplayFormatType, VerboseDisplay}; pub use crate::metrics::Metric; +pub use crate::ordering::InputOrderMode; pub use crate::topk::TopK; pub use crate::visitor::{accept, visit_execution_plan, ExecutionPlanVisitor}; diff --git a/datafusion/physical-plan/src/ordering.rs b/datafusion/physical-plan/src/ordering.rs new file mode 100644 index 000000000000..047f89eef193 --- /dev/null +++ b/datafusion/physical-plan/src/ordering.rs @@ -0,0 +1,51 @@ +// 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. + +/// Specifies how the input to an aggregation or window operator is ordered +/// relative to their `GROUP BY` or `PARTITION BY` expressions. +/// +/// For example, if the existing ordering is `[a ASC, b ASC, c ASC]` +/// +/// ## Window Functions +/// - A `PARTITION BY b` clause can use `Linear` mode. +/// - A `PARTITION BY a, c` or a `PARTITION BY c, a` can use +/// `PartiallySorted([0])` or `PartiallySorted([1])` modes, respectively. +/// (The vector stores the index of `a` in the respective PARTITION BY expression.) +/// - A `PARTITION BY a, b` or a `PARTITION BY b, a` can use `Sorted` mode. +/// +/// ## Aggregations +/// - A `GROUP BY b` clause can use `Linear` mode. +/// - A `GROUP BY a, c` or a `GROUP BY BY c, a` can use +/// `PartiallySorted([0])` or `PartiallySorted([1])` modes, respectively. +/// (The vector stores the index of `a` in the respective PARTITION BY expression.) +/// - A `GROUP BY a, b` or a `GROUP BY b, a` can use `Sorted` mode. +/// +/// Note these are the same examples as above, but with `GROUP BY` instead of +/// `PARTITION BY` to make the examples easier to read. +#[derive(Debug, Clone, PartialEq)] +pub enum InputOrderMode { + /// There is no partial permutation of the expressions satisfying the + /// existing ordering. + Linear, + /// There is a partial permutation of the expressions satisfying the + /// existing ordering. Indices describing the longest partial permutation + /// are stored in the vector. + PartiallySorted(Vec), + /// There is a (full) permutation of the expressions satisfying the + /// existing ordering. + Sorted, +} diff --git a/datafusion/physical-plan/src/windows/bounded_window_agg_exec.rs b/datafusion/physical-plan/src/windows/bounded_window_agg_exec.rs index 8156ab1fa31b..9e4d6c137067 100644 --- a/datafusion/physical-plan/src/windows/bounded_window_agg_exec.rs +++ b/datafusion/physical-plan/src/windows/bounded_window_agg_exec.rs @@ -31,11 +31,12 @@ use crate::expressions::PhysicalSortExpr; use crate::metrics::{BaselineMetrics, ExecutionPlanMetricsSet, MetricsSet}; use crate::windows::{ calc_requirements, get_ordered_partition_by_indices, get_partition_by_sort_exprs, - window_equivalence_properties, PartitionSearchMode, + window_equivalence_properties, }; use crate::{ ColumnStatistics, DisplayAs, DisplayFormatType, Distribution, ExecutionPlan, - Partitioning, RecordBatchStream, SendableRecordBatchStream, Statistics, WindowExpr, + InputOrderMode, Partitioning, RecordBatchStream, SendableRecordBatchStream, + Statistics, WindowExpr, }; use arrow::{ @@ -81,8 +82,8 @@ pub struct BoundedWindowAggExec { pub partition_keys: Vec>, /// Execution metrics metrics: ExecutionPlanMetricsSet, - /// Partition by search mode - pub partition_search_mode: PartitionSearchMode, + /// Describes how the input is ordered relative to the partition keys + pub input_order_mode: InputOrderMode, /// Partition by indices that define ordering // For example, if input ordering is ORDER BY a, b and window expression // contains PARTITION BY b, a; `ordered_partition_by_indices` would be 1, 0. @@ -98,13 +99,13 @@ impl BoundedWindowAggExec { window_expr: Vec>, input: Arc, partition_keys: Vec>, - partition_search_mode: PartitionSearchMode, + input_order_mode: InputOrderMode, ) -> Result { let schema = create_schema(&input.schema(), &window_expr)?; let schema = Arc::new(schema); let partition_by_exprs = window_expr[0].partition_by(); - let ordered_partition_by_indices = match &partition_search_mode { - PartitionSearchMode::Sorted => { + let ordered_partition_by_indices = match &input_order_mode { + InputOrderMode::Sorted => { let indices = get_ordered_partition_by_indices( window_expr[0].partition_by(), &input, @@ -115,10 +116,8 @@ impl BoundedWindowAggExec { (0..partition_by_exprs.len()).collect::>() } } - PartitionSearchMode::PartiallySorted(ordered_indices) => { - ordered_indices.clone() - } - PartitionSearchMode::Linear => { + InputOrderMode::PartiallySorted(ordered_indices) => ordered_indices.clone(), + InputOrderMode::Linear => { vec![] } }; @@ -128,7 +127,7 @@ impl BoundedWindowAggExec { schema, partition_keys, metrics: ExecutionPlanMetricsSet::new(), - partition_search_mode, + input_order_mode, ordered_partition_by_indices, }) } @@ -162,8 +161,8 @@ impl BoundedWindowAggExec { fn get_search_algo(&self) -> Result> { let partition_by_sort_keys = self.partition_by_sort_keys()?; let ordered_partition_by_indices = self.ordered_partition_by_indices.clone(); - Ok(match &self.partition_search_mode { - PartitionSearchMode::Sorted => { + Ok(match &self.input_order_mode { + InputOrderMode::Sorted => { // In Sorted mode, all partition by columns should be ordered. if self.window_expr()[0].partition_by().len() != ordered_partition_by_indices.len() @@ -175,7 +174,7 @@ impl BoundedWindowAggExec { ordered_partition_by_indices, }) } - PartitionSearchMode::Linear | PartitionSearchMode::PartiallySorted(_) => { + InputOrderMode::Linear | InputOrderMode::PartiallySorted(_) => { Box::new(LinearSearch::new(ordered_partition_by_indices)) } }) @@ -203,7 +202,7 @@ impl DisplayAs for BoundedWindowAggExec { ) }) .collect(); - let mode = &self.partition_search_mode; + let mode = &self.input_order_mode; write!(f, "wdw=[{}], mode=[{:?}]", g.join(", "), mode)?; } } @@ -244,7 +243,7 @@ impl ExecutionPlan for BoundedWindowAggExec { fn required_input_ordering(&self) -> Vec>> { let partition_bys = self.window_expr()[0].partition_by(); let order_keys = self.window_expr()[0].order_by(); - if self.partition_search_mode != PartitionSearchMode::Sorted + if self.input_order_mode != InputOrderMode::Sorted || self.ordered_partition_by_indices.len() >= partition_bys.len() { let partition_bys = self @@ -283,7 +282,7 @@ impl ExecutionPlan for BoundedWindowAggExec { self.window_expr.clone(), children[0].clone(), self.partition_keys.clone(), - self.partition_search_mode.clone(), + self.input_order_mode.clone(), )?)) } @@ -1114,7 +1113,7 @@ fn get_aggregate_result_out_column( mod tests { use crate::common::collect; use crate::memory::MemoryExec; - use crate::windows::{BoundedWindowAggExec, PartitionSearchMode}; + use crate::windows::{BoundedWindowAggExec, InputOrderMode}; use crate::{get_plan_string, ExecutionPlan}; use arrow_array::RecordBatch; use arrow_schema::{DataType, Field, Schema}; @@ -1201,7 +1200,7 @@ mod tests { window_exprs, memory_exec, vec![], - PartitionSearchMode::Sorted, + InputOrderMode::Sorted, ) .map(|e| Arc::new(e) as Arc)?; diff --git a/datafusion/physical-plan/src/windows/mod.rs b/datafusion/physical-plan/src/windows/mod.rs index 828dcb4b130c..3187e6b0fbd3 100644 --- a/datafusion/physical-plan/src/windows/mod.rs +++ b/datafusion/physical-plan/src/windows/mod.rs @@ -27,7 +27,7 @@ use crate::{ cume_dist, dense_rank, lag, lead, percent_rank, rank, Literal, NthValue, Ntile, PhysicalSortExpr, RowNumber, }, - udaf, unbounded_output, ExecutionPlan, PhysicalExpr, + udaf, unbounded_output, ExecutionPlan, InputOrderMode, PhysicalExpr, }; use arrow::datatypes::Schema; @@ -54,30 +54,6 @@ pub use datafusion_physical_expr::window::{ BuiltInWindowExpr, PlainAggregateWindowExpr, WindowExpr, }; -#[derive(Debug, Clone, PartialEq)] -/// Specifies aggregation grouping and/or window partitioning properties of a -/// set of expressions in terms of the existing ordering. -/// For example, if the existing ordering is `[a ASC, b ASC, c ASC]`: -/// - A `PARTITION BY b` clause will result in `Linear` mode. -/// - A `PARTITION BY a, c` or a `PARTITION BY c, a` clause will result in -/// `PartiallySorted([0])` or `PartiallySorted([1])` modes, respectively. -/// The vector stores the index of `a` in the respective PARTITION BY expression. -/// - A `PARTITION BY a, b` or a `PARTITION BY b, a` clause will result in -/// `Sorted` mode. -/// Note that the examples above are applicable for `GROUP BY` clauses too. -pub enum PartitionSearchMode { - /// There is no partial permutation of the expressions satisfying the - /// existing ordering. - Linear, - /// There is a partial permutation of the expressions satisfying the - /// existing ordering. Indices describing the longest partial permutation - /// are stored in the vector. - PartiallySorted(Vec), - /// There is a (full) permutation of the expressions satisfying the - /// existing ordering. - Sorted, -} - /// Create a physical expression for window function pub fn create_window_expr( fun: &WindowFunction, @@ -414,17 +390,17 @@ pub fn get_best_fitting_window( // of the window_exprs are same. let partitionby_exprs = window_exprs[0].partition_by(); let orderby_keys = window_exprs[0].order_by(); - let (should_reverse, partition_search_mode) = - if let Some((should_reverse, partition_search_mode)) = + let (should_reverse, input_order_mode) = + if let Some((should_reverse, input_order_mode)) = get_window_mode(partitionby_exprs, orderby_keys, input) { - (should_reverse, partition_search_mode) + (should_reverse, input_order_mode) } else { return Ok(None); }; let is_unbounded = unbounded_output(input); - if !is_unbounded && partition_search_mode != PartitionSearchMode::Sorted { - // Executor has bounded input and `partition_search_mode` is not `PartitionSearchMode::Sorted` + if !is_unbounded && input_order_mode != InputOrderMode::Sorted { + // Executor has bounded input and `input_order_mode` is not `InputOrderMode::Sorted` // in this case removing the sort is not helpful, return: return Ok(None); }; @@ -452,13 +428,13 @@ pub fn get_best_fitting_window( window_expr, input.clone(), physical_partition_keys.to_vec(), - partition_search_mode, + input_order_mode, )?) as _)) - } else if partition_search_mode != PartitionSearchMode::Sorted { + } else if input_order_mode != InputOrderMode::Sorted { // For `WindowAggExec` to work correctly PARTITION BY columns should be sorted. - // Hence, if `partition_search_mode` is not `PartitionSearchMode::Sorted` we should convert - // input ordering such that it can work with PartitionSearchMode::Sorted (add `SortExec`). - // Effectively `WindowAggExec` works only in PartitionSearchMode::Sorted mode. + // Hence, if `input_order_mode` is not `Sorted` we should convert + // input ordering such that it can work with `Sorted` (add `SortExec`). + // Effectively `WindowAggExec` works only in `Sorted` mode. Ok(None) } else { Ok(Some(Arc::new(WindowAggExec::try_new( @@ -474,16 +450,16 @@ pub fn get_best_fitting_window( /// is sufficient to run the current window operator. /// - A `None` return value indicates that we can not remove the sort in question /// (input ordering is not sufficient to run current window executor). -/// - A `Some((bool, PartitionSearchMode))` value indicates that the window operator +/// - A `Some((bool, InputOrderMode))` value indicates that the window operator /// can run with existing input ordering, so we can remove `SortExec` before it. /// The `bool` field in the return value represents whether we should reverse window -/// operator to remove `SortExec` before it. The `PartitionSearchMode` field represents +/// operator to remove `SortExec` before it. The `InputOrderMode` field represents /// the mode this window operator should work in to accommodate the existing ordering. pub fn get_window_mode( partitionby_exprs: &[Arc], orderby_keys: &[PhysicalSortExpr], input: &Arc, -) -> Option<(bool, PartitionSearchMode)> { +) -> Option<(bool, InputOrderMode)> { let input_eqs = input.equivalence_properties(); let mut partition_by_reqs: Vec = vec![]; let (_, indices) = input_eqs.find_longest_permutation(partitionby_exprs); @@ -504,11 +480,11 @@ pub fn get_window_mode( if partition_by_eqs.ordering_satisfy_requirement(&req) { // Window can be run with existing ordering let mode = if indices.len() == partitionby_exprs.len() { - PartitionSearchMode::Sorted + InputOrderMode::Sorted } else if indices.is_empty() { - PartitionSearchMode::Linear + InputOrderMode::Linear } else { - PartitionSearchMode::PartiallySorted(indices) + InputOrderMode::PartiallySorted(indices) }; return Some((should_swap, mode)); } @@ -532,7 +508,7 @@ mod tests { use futures::FutureExt; - use PartitionSearchMode::{Linear, PartiallySorted, Sorted}; + use InputOrderMode::{Linear, PartiallySorted, Sorted}; fn create_test_schema() -> Result { let nullable_column = Field::new("nullable_col", DataType::Int32, true); @@ -792,11 +768,11 @@ mod tests { // Second field in the tuple is Vec where each element in the vector represents ORDER BY columns // For instance, vec!["c"], corresponds to ORDER BY c ASC NULLS FIRST, (ordering is default ordering. We do not check // for reversibility in this test). - // Third field in the tuple is Option, which corresponds to expected algorithm mode. + // Third field in the tuple is Option, which corresponds to expected algorithm mode. // None represents that existing ordering is not sufficient to run executor with any one of the algorithms // (We need to add SortExec to be able to run it). - // Some(PartitionSearchMode) represents, we can run algorithm with existing ordering; and algorithm should work in - // PartitionSearchMode. + // Some(InputOrderMode) represents, we can run algorithm with existing ordering; and algorithm should work in + // InputOrderMode. let test_cases = vec![ (vec!["a"], vec!["a"], Some(Sorted)), (vec!["a"], vec!["b"], Some(Sorted)), @@ -881,7 +857,7 @@ mod tests { } let res = get_window_mode(&partition_by_exprs, &order_by_exprs, &exec_unbounded); - // Since reversibility is not important in this test. Convert Option<(bool, PartitionSearchMode)> to Option + // Since reversibility is not important in this test. Convert Option<(bool, InputOrderMode)> to Option let res = res.map(|(_, mode)| mode); assert_eq!( res, *expected, @@ -912,12 +888,12 @@ mod tests { // Second field in the tuple is Vec<(str, bool, bool)> where each element in the vector represents ORDER BY columns // For instance, vec![("c", false, false)], corresponds to ORDER BY c ASC NULLS LAST, // similarly, vec![("c", true, true)], corresponds to ORDER BY c DESC NULLS FIRST, - // Third field in the tuple is Option<(bool, PartitionSearchMode)>, which corresponds to expected result. + // Third field in the tuple is Option<(bool, InputOrderMode)>, which corresponds to expected result. // None represents that existing ordering is not sufficient to run executor with any one of the algorithms // (We need to add SortExec to be able to run it). - // Some((bool, PartitionSearchMode)) represents, we can run algorithm with existing ordering. Algorithm should work in - // PartitionSearchMode, bool field represents whether we should reverse window expressions to run executor with existing ordering. - // For instance, `Some((false, PartitionSearchMode::Sorted))`, represents that we shouldn't reverse window expressions. And algorithm + // Some((bool, InputOrderMode)) represents, we can run algorithm with existing ordering. Algorithm should work in + // InputOrderMode, bool field represents whether we should reverse window expressions to run executor with existing ordering. + // For instance, `Some((false, InputOrderMode::Sorted))`, represents that we shouldn't reverse window expressions. And algorithm // should work in Sorted mode to work with existing ordering. let test_cases = vec![ // PARTITION BY a, b ORDER BY c ASC NULLS LAST diff --git a/datafusion/proto/proto/datafusion.proto b/datafusion/proto/proto/datafusion.proto index 8c2fd5369e33..daf539f219de 100644 --- a/datafusion/proto/proto/datafusion.proto +++ b/datafusion/proto/proto/datafusion.proto @@ -1502,7 +1502,7 @@ enum AggregateMode { SINGLE_PARTITIONED = 4; } -message PartiallySortedPartitionSearchMode { +message PartiallySortedInputOrderMode { repeated uint64 columns = 6; } @@ -1511,9 +1511,9 @@ message WindowAggExecNode { repeated PhysicalWindowExprNode window_expr = 2; repeated PhysicalExprNode partition_keys = 5; // Set optional to `None` for `BoundedWindowAggExec`. - oneof partition_search_mode { + oneof input_order_mode { EmptyMessage linear = 7; - PartiallySortedPartitionSearchMode partially_sorted = 8; + PartiallySortedInputOrderMode partially_sorted = 8; EmptyMessage sorted = 9; } } diff --git a/datafusion/proto/src/generated/pbjson.rs b/datafusion/proto/src/generated/pbjson.rs index b8c5f6a4aae8..f453875d71d4 100644 --- a/datafusion/proto/src/generated/pbjson.rs +++ b/datafusion/proto/src/generated/pbjson.rs @@ -14967,7 +14967,7 @@ impl<'de> serde::Deserialize<'de> for PartialTableReference { deserializer.deserialize_struct("datafusion.PartialTableReference", FIELDS, GeneratedVisitor) } } -impl serde::Serialize for PartiallySortedPartitionSearchMode { +impl serde::Serialize for PartiallySortedInputOrderMode { #[allow(deprecated)] fn serialize(&self, serializer: S) -> std::result::Result where @@ -14978,14 +14978,14 @@ impl serde::Serialize for PartiallySortedPartitionSearchMode { if !self.columns.is_empty() { len += 1; } - let mut struct_ser = serializer.serialize_struct("datafusion.PartiallySortedPartitionSearchMode", len)?; + let mut struct_ser = serializer.serialize_struct("datafusion.PartiallySortedInputOrderMode", len)?; if !self.columns.is_empty() { struct_ser.serialize_field("columns", &self.columns.iter().map(ToString::to_string).collect::>())?; } struct_ser.end() } } -impl<'de> serde::Deserialize<'de> for PartiallySortedPartitionSearchMode { +impl<'de> serde::Deserialize<'de> for PartiallySortedInputOrderMode { #[allow(deprecated)] fn deserialize(deserializer: D) -> std::result::Result where @@ -15029,13 +15029,13 @@ impl<'de> serde::Deserialize<'de> for PartiallySortedPartitionSearchMode { } struct GeneratedVisitor; impl<'de> serde::de::Visitor<'de> for GeneratedVisitor { - type Value = PartiallySortedPartitionSearchMode; + type Value = PartiallySortedInputOrderMode; fn expecting(&self, formatter: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { - formatter.write_str("struct datafusion.PartiallySortedPartitionSearchMode") + formatter.write_str("struct datafusion.PartiallySortedInputOrderMode") } - fn visit_map(self, mut map_: V) -> std::result::Result + fn visit_map(self, mut map_: V) -> std::result::Result where V: serde::de::MapAccess<'de>, { @@ -15053,12 +15053,12 @@ impl<'de> serde::Deserialize<'de> for PartiallySortedPartitionSearchMode { } } } - Ok(PartiallySortedPartitionSearchMode { + Ok(PartiallySortedInputOrderMode { columns: columns__.unwrap_or_default(), }) } } - deserializer.deserialize_struct("datafusion.PartiallySortedPartitionSearchMode", FIELDS, GeneratedVisitor) + deserializer.deserialize_struct("datafusion.PartiallySortedInputOrderMode", FIELDS, GeneratedVisitor) } } impl serde::Serialize for PartitionColumn { @@ -25639,7 +25639,7 @@ impl serde::Serialize for WindowAggExecNode { if !self.partition_keys.is_empty() { len += 1; } - if self.partition_search_mode.is_some() { + if self.input_order_mode.is_some() { len += 1; } let mut struct_ser = serializer.serialize_struct("datafusion.WindowAggExecNode", len)?; @@ -25652,15 +25652,15 @@ impl serde::Serialize for WindowAggExecNode { if !self.partition_keys.is_empty() { struct_ser.serialize_field("partitionKeys", &self.partition_keys)?; } - if let Some(v) = self.partition_search_mode.as_ref() { + if let Some(v) = self.input_order_mode.as_ref() { match v { - window_agg_exec_node::PartitionSearchMode::Linear(v) => { + window_agg_exec_node::InputOrderMode::Linear(v) => { struct_ser.serialize_field("linear", v)?; } - window_agg_exec_node::PartitionSearchMode::PartiallySorted(v) => { + window_agg_exec_node::InputOrderMode::PartiallySorted(v) => { struct_ser.serialize_field("partiallySorted", v)?; } - window_agg_exec_node::PartitionSearchMode::Sorted(v) => { + window_agg_exec_node::InputOrderMode::Sorted(v) => { struct_ser.serialize_field("sorted", v)?; } } @@ -25743,7 +25743,7 @@ impl<'de> serde::Deserialize<'de> for WindowAggExecNode { let mut input__ = None; let mut window_expr__ = None; let mut partition_keys__ = None; - let mut partition_search_mode__ = None; + let mut input_order_mode__ = None; while let Some(k) = map_.next_key()? { match k { GeneratedField::Input => { @@ -25765,24 +25765,24 @@ impl<'de> serde::Deserialize<'de> for WindowAggExecNode { partition_keys__ = Some(map_.next_value()?); } GeneratedField::Linear => { - if partition_search_mode__.is_some() { + if input_order_mode__.is_some() { return Err(serde::de::Error::duplicate_field("linear")); } - partition_search_mode__ = map_.next_value::<::std::option::Option<_>>()?.map(window_agg_exec_node::PartitionSearchMode::Linear) + input_order_mode__ = map_.next_value::<::std::option::Option<_>>()?.map(window_agg_exec_node::InputOrderMode::Linear) ; } GeneratedField::PartiallySorted => { - if partition_search_mode__.is_some() { + if input_order_mode__.is_some() { return Err(serde::de::Error::duplicate_field("partiallySorted")); } - partition_search_mode__ = map_.next_value::<::std::option::Option<_>>()?.map(window_agg_exec_node::PartitionSearchMode::PartiallySorted) + input_order_mode__ = map_.next_value::<::std::option::Option<_>>()?.map(window_agg_exec_node::InputOrderMode::PartiallySorted) ; } GeneratedField::Sorted => { - if partition_search_mode__.is_some() { + if input_order_mode__.is_some() { return Err(serde::de::Error::duplicate_field("sorted")); } - partition_search_mode__ = map_.next_value::<::std::option::Option<_>>()?.map(window_agg_exec_node::PartitionSearchMode::Sorted) + input_order_mode__ = map_.next_value::<::std::option::Option<_>>()?.map(window_agg_exec_node::InputOrderMode::Sorted) ; } } @@ -25791,7 +25791,7 @@ impl<'de> serde::Deserialize<'de> for WindowAggExecNode { input: input__, window_expr: window_expr__.unwrap_or_default(), partition_keys: partition_keys__.unwrap_or_default(), - partition_search_mode: partition_search_mode__, + input_order_mode: input_order_mode__, }) } } diff --git a/datafusion/proto/src/generated/prost.rs b/datafusion/proto/src/generated/prost.rs index c31bc4ab5948..9e78b7c8d6dd 100644 --- a/datafusion/proto/src/generated/prost.rs +++ b/datafusion/proto/src/generated/prost.rs @@ -2101,7 +2101,7 @@ pub struct ProjectionExecNode { } #[allow(clippy::derive_partial_eq_without_eq)] #[derive(Clone, PartialEq, ::prost::Message)] -pub struct PartiallySortedPartitionSearchMode { +pub struct PartiallySortedInputOrderMode { #[prost(uint64, repeated, tag = "6")] pub columns: ::prost::alloc::vec::Vec, } @@ -2115,21 +2115,19 @@ pub struct WindowAggExecNode { #[prost(message, repeated, tag = "5")] pub partition_keys: ::prost::alloc::vec::Vec, /// Set optional to `None` for `BoundedWindowAggExec`. - #[prost(oneof = "window_agg_exec_node::PartitionSearchMode", tags = "7, 8, 9")] - pub partition_search_mode: ::core::option::Option< - window_agg_exec_node::PartitionSearchMode, - >, + #[prost(oneof = "window_agg_exec_node::InputOrderMode", tags = "7, 8, 9")] + pub input_order_mode: ::core::option::Option, } /// Nested message and enum types in `WindowAggExecNode`. pub mod window_agg_exec_node { /// Set optional to `None` for `BoundedWindowAggExec`. #[allow(clippy::derive_partial_eq_without_eq)] #[derive(Clone, PartialEq, ::prost::Oneof)] - pub enum PartitionSearchMode { + pub enum InputOrderMode { #[prost(message, tag = "7")] Linear(super::EmptyMessage), #[prost(message, tag = "8")] - PartiallySorted(super::PartiallySortedPartitionSearchMode), + PartiallySorted(super::PartiallySortedInputOrderMode), #[prost(message, tag = "9")] Sorted(super::EmptyMessage), } diff --git a/datafusion/proto/src/physical_plan/mod.rs b/datafusion/proto/src/physical_plan/mod.rs index 6714c35dc615..907ba04ebc20 100644 --- a/datafusion/proto/src/physical_plan/mod.rs +++ b/datafusion/proto/src/physical_plan/mod.rs @@ -49,11 +49,10 @@ use datafusion::physical_plan::repartition::RepartitionExec; use datafusion::physical_plan::sorts::sort::SortExec; use datafusion::physical_plan::sorts::sort_preserving_merge::SortPreservingMergeExec; use datafusion::physical_plan::union::UnionExec; -use datafusion::physical_plan::windows::{ - BoundedWindowAggExec, PartitionSearchMode, WindowAggExec, -}; +use datafusion::physical_plan::windows::{BoundedWindowAggExec, WindowAggExec}; use datafusion::physical_plan::{ - udaf, AggregateExpr, ExecutionPlan, Partitioning, PhysicalExpr, WindowExpr, + udaf, AggregateExpr, ExecutionPlan, InputOrderMode, Partitioning, PhysicalExpr, + WindowExpr, }; use datafusion_common::{internal_err, not_impl_err, DataFusionError, Result}; use prost::bytes::BufMut; @@ -313,20 +312,18 @@ impl AsExecutionPlan for PhysicalPlanNode { }) .collect::>>>()?; - if let Some(partition_search_mode) = - window_agg.partition_search_mode.as_ref() - { - let partition_search_mode = match partition_search_mode { - window_agg_exec_node::PartitionSearchMode::Linear(_) => { - PartitionSearchMode::Linear + if let Some(input_order_mode) = window_agg.input_order_mode.as_ref() { + let input_order_mode = match input_order_mode { + window_agg_exec_node::InputOrderMode::Linear(_) => { + InputOrderMode::Linear } - window_agg_exec_node::PartitionSearchMode::PartiallySorted( - protobuf::PartiallySortedPartitionSearchMode { columns }, - ) => PartitionSearchMode::PartiallySorted( + window_agg_exec_node::InputOrderMode::PartiallySorted( + protobuf::PartiallySortedInputOrderMode { columns }, + ) => InputOrderMode::PartiallySorted( columns.iter().map(|c| *c as usize).collect(), ), - window_agg_exec_node::PartitionSearchMode::Sorted(_) => { - PartitionSearchMode::Sorted + window_agg_exec_node::InputOrderMode::Sorted(_) => { + InputOrderMode::Sorted } }; @@ -334,7 +331,7 @@ impl AsExecutionPlan for PhysicalPlanNode { physical_window_expr, input, partition_keys, - partition_search_mode, + input_order_mode, )?)) } else { Ok(Arc::new(WindowAggExec::try_new( @@ -1560,7 +1557,7 @@ impl AsExecutionPlan for PhysicalPlanNode { input: Some(Box::new(input)), window_expr, partition_keys, - partition_search_mode: None, + input_order_mode: None, }, ))), }); @@ -1584,24 +1581,20 @@ impl AsExecutionPlan for PhysicalPlanNode { .map(|e| e.clone().try_into()) .collect::>>()?; - let partition_search_mode = match &exec.partition_search_mode { - PartitionSearchMode::Linear => { - window_agg_exec_node::PartitionSearchMode::Linear( - protobuf::EmptyMessage {}, - ) - } - PartitionSearchMode::PartiallySorted(columns) => { - window_agg_exec_node::PartitionSearchMode::PartiallySorted( - protobuf::PartiallySortedPartitionSearchMode { + let input_order_mode = match &exec.input_order_mode { + InputOrderMode::Linear => window_agg_exec_node::InputOrderMode::Linear( + protobuf::EmptyMessage {}, + ), + InputOrderMode::PartiallySorted(columns) => { + window_agg_exec_node::InputOrderMode::PartiallySorted( + protobuf::PartiallySortedInputOrderMode { columns: columns.iter().map(|c| *c as u64).collect(), }, ) } - PartitionSearchMode::Sorted => { - window_agg_exec_node::PartitionSearchMode::Sorted( - protobuf::EmptyMessage {}, - ) - } + InputOrderMode::Sorted => window_agg_exec_node::InputOrderMode::Sorted( + protobuf::EmptyMessage {}, + ), }; return Ok(protobuf::PhysicalPlanNode { @@ -1610,7 +1603,7 @@ impl AsExecutionPlan for PhysicalPlanNode { input: Some(Box::new(input)), window_expr, partition_keys, - partition_search_mode: Some(partition_search_mode), + input_order_mode: Some(input_order_mode), }, ))), }); From c7a69658c8df1d738f761aa9a9d2401d29369025 Mon Sep 17 00:00:00 2001 From: Edmondo Porcu Date: Tue, 5 Dec 2023 13:36:48 -0800 Subject: [PATCH 368/572] Make filter selectivity for statistics configurable (#8243) * Turning filter selectivity as a configurable parameter * Renaming API to be more consistent with struct value * Adding a filter with custom selectivity --- datafusion/common/src/config.rs | 6 ++ .../physical_optimizer/projection_pushdown.rs | 4 + datafusion/core/src/physical_planner.rs | 4 +- datafusion/physical-plan/src/filter.rs | 78 ++++++++++++++++++- datafusion/proto/proto/datafusion.proto | 1 + datafusion/proto/src/generated/pbjson.rs | 20 +++++ datafusion/proto/src/generated/prost.rs | 2 + datafusion/proto/src/physical_plan/mod.rs | 12 ++- .../test_files/information_schema.slt | 2 + docs/source/user-guide/configs.md | 1 + 10 files changed, 124 insertions(+), 6 deletions(-) diff --git a/datafusion/common/src/config.rs b/datafusion/common/src/config.rs index ba2072ecc151..03fb5ea320a0 100644 --- a/datafusion/common/src/config.rs +++ b/datafusion/common/src/config.rs @@ -524,6 +524,11 @@ config_namespace! { /// The maximum estimated size in bytes for one input side of a HashJoin /// will be collected into a single partition pub hash_join_single_partition_threshold: usize, default = 1024 * 1024 + + /// The default filter selectivity used by Filter Statistics + /// when an exact selectivity cannot be determined. Valid values are + /// between 0 (no selectivity) and 100 (all rows are selected). + pub default_filter_selectivity: u8, default = 20 } } @@ -877,6 +882,7 @@ config_field!(String); config_field!(bool); config_field!(usize); config_field!(f64); +config_field!(u8); config_field!(u64); /// An implementation trait used to recursively walk configuration diff --git a/datafusion/core/src/physical_optimizer/projection_pushdown.rs b/datafusion/core/src/physical_optimizer/projection_pushdown.rs index 7ebb64ab858a..f6c94edd8ca3 100644 --- a/datafusion/core/src/physical_optimizer/projection_pushdown.rs +++ b/datafusion/core/src/physical_optimizer/projection_pushdown.rs @@ -348,6 +348,10 @@ fn try_swapping_with_filter( }; FilterExec::try_new(new_predicate, make_with_child(projection, filter.input())?) + .and_then(|e| { + let selectivity = filter.default_selectivity(); + e.with_default_selectivity(selectivity) + }) .map(|e| Some(Arc::new(e) as _)) } diff --git a/datafusion/core/src/physical_planner.rs b/datafusion/core/src/physical_planner.rs index 8ef433173edd..65a2e4e0a4f3 100644 --- a/datafusion/core/src/physical_planner.rs +++ b/datafusion/core/src/physical_planner.rs @@ -915,7 +915,9 @@ impl DefaultPhysicalPlanner { &input_schema, session_state, )?; - Ok(Arc::new(FilterExec::try_new(runtime_expr, physical_input)?)) + let selectivity = session_state.config().options().optimizer.default_filter_selectivity; + let filter = FilterExec::try_new(runtime_expr, physical_input)?; + Ok(Arc::new(filter.with_default_selectivity(selectivity)?)) } LogicalPlan::Union(Union { inputs, .. }) => { let physical_plans = self.create_initial_plan_multi(inputs.iter().map(|lp| lp.as_ref()), session_state).await?; diff --git a/datafusion/physical-plan/src/filter.rs b/datafusion/physical-plan/src/filter.rs index 903f4c972ebd..56a1b4e17821 100644 --- a/datafusion/physical-plan/src/filter.rs +++ b/datafusion/physical-plan/src/filter.rs @@ -61,6 +61,8 @@ pub struct FilterExec { input: Arc, /// Execution metrics metrics: ExecutionPlanMetricsSet, + /// Selectivity for statistics. 0 = no rows, 100 all rows + default_selectivity: u8, } impl FilterExec { @@ -74,6 +76,7 @@ impl FilterExec { predicate, input: input.clone(), metrics: ExecutionPlanMetricsSet::new(), + default_selectivity: 20, }), other => { plan_err!("Filter predicate must return boolean values, not {other:?}") @@ -81,6 +84,17 @@ impl FilterExec { } } + pub fn with_default_selectivity( + mut self, + default_selectivity: u8, + ) -> Result { + if default_selectivity > 100 { + return plan_err!("Default flter selectivity needs to be less than 100"); + } + self.default_selectivity = default_selectivity; + Ok(self) + } + /// The expression to filter on. This expression must evaluate to a boolean value. pub fn predicate(&self) -> &Arc { &self.predicate @@ -90,6 +104,11 @@ impl FilterExec { pub fn input(&self) -> &Arc { &self.input } + + /// The default selectivity + pub fn default_selectivity(&self) -> u8 { + self.default_selectivity + } } impl DisplayAs for FilterExec { @@ -166,6 +185,10 @@ impl ExecutionPlan for FilterExec { mut children: Vec>, ) -> Result> { FilterExec::try_new(self.predicate.clone(), children.swap_remove(0)) + .and_then(|e| { + let selectivity = e.default_selectivity(); + e.with_default_selectivity(selectivity) + }) .map(|e| Arc::new(e) as _) } @@ -196,10 +219,7 @@ impl ExecutionPlan for FilterExec { let input_stats = self.input.statistics()?; let schema = self.schema(); if !check_support(predicate, &schema) { - // assume filter selects 20% of rows if we cannot do anything smarter - // tracking issue for making this configurable: - // https://github.com/apache/arrow-datafusion/issues/8133 - let selectivity = 0.2_f64; + let selectivity = self.default_selectivity as f64 / 100.0; let mut stats = input_stats.into_inexact(); stats.num_rows = stats.num_rows.with_estimated_selectivity(selectivity); stats.total_byte_size = stats @@ -987,4 +1007,54 @@ mod tests { Ok(()) } + + #[tokio::test] + async fn test_validation_filter_selectivity() -> Result<()> { + let schema = Schema::new(vec![Field::new("a", DataType::Int32, false)]); + let input = Arc::new(StatisticsExec::new( + Statistics::new_unknown(&schema), + schema, + )); + // WHERE a = 10 + let predicate = Arc::new(BinaryExpr::new( + Arc::new(Column::new("a", 0)), + Operator::Eq, + Arc::new(Literal::new(ScalarValue::Int32(Some(10)))), + )); + let filter = FilterExec::try_new(predicate, input)?; + assert!(filter.with_default_selectivity(120).is_err()); + Ok(()) + } + + #[tokio::test] + async fn test_custom_filter_selectivity() -> Result<()> { + // Need a decimal to trigger inexact selectivity + let schema = + Schema::new(vec![Field::new("a", DataType::Decimal128(2, 3), false)]); + let input = Arc::new(StatisticsExec::new( + Statistics { + num_rows: Precision::Inexact(1000), + total_byte_size: Precision::Inexact(4000), + column_statistics: vec![ColumnStatistics { + ..Default::default() + }], + }, + schema, + )); + // WHERE a = 10 + let predicate = Arc::new(BinaryExpr::new( + Arc::new(Column::new("a", 0)), + Operator::Eq, + Arc::new(Literal::new(ScalarValue::Decimal128(Some(10), 10, 10))), + )); + let filter = FilterExec::try_new(predicate, input)?; + let statistics = filter.statistics()?; + assert_eq!(statistics.num_rows, Precision::Inexact(200)); + assert_eq!(statistics.total_byte_size, Precision::Inexact(800)); + let filter = filter.with_default_selectivity(40)?; + let statistics = filter.statistics()?; + assert_eq!(statistics.num_rows, Precision::Inexact(400)); + assert_eq!(statistics.total_byte_size, Precision::Inexact(1600)); + Ok(()) + } } diff --git a/datafusion/proto/proto/datafusion.proto b/datafusion/proto/proto/datafusion.proto index daf539f219de..e46e70a1396b 100644 --- a/datafusion/proto/proto/datafusion.proto +++ b/datafusion/proto/proto/datafusion.proto @@ -1368,6 +1368,7 @@ message PhysicalNegativeNode { message FilterExecNode { PhysicalPlanNode input = 1; PhysicalExprNode expr = 2; + uint32 default_filter_selectivity = 3; } message FileGroup { diff --git a/datafusion/proto/src/generated/pbjson.rs b/datafusion/proto/src/generated/pbjson.rs index f453875d71d4..a1c177541981 100644 --- a/datafusion/proto/src/generated/pbjson.rs +++ b/datafusion/proto/src/generated/pbjson.rs @@ -7797,6 +7797,9 @@ impl serde::Serialize for FilterExecNode { if self.expr.is_some() { len += 1; } + if self.default_filter_selectivity != 0 { + len += 1; + } let mut struct_ser = serializer.serialize_struct("datafusion.FilterExecNode", len)?; if let Some(v) = self.input.as_ref() { struct_ser.serialize_field("input", v)?; @@ -7804,6 +7807,9 @@ impl serde::Serialize for FilterExecNode { if let Some(v) = self.expr.as_ref() { struct_ser.serialize_field("expr", v)?; } + if self.default_filter_selectivity != 0 { + struct_ser.serialize_field("defaultFilterSelectivity", &self.default_filter_selectivity)?; + } struct_ser.end() } } @@ -7816,12 +7822,15 @@ impl<'de> serde::Deserialize<'de> for FilterExecNode { const FIELDS: &[&str] = &[ "input", "expr", + "default_filter_selectivity", + "defaultFilterSelectivity", ]; #[allow(clippy::enum_variant_names)] enum GeneratedField { Input, Expr, + DefaultFilterSelectivity, } impl<'de> serde::Deserialize<'de> for GeneratedField { fn deserialize(deserializer: D) -> std::result::Result @@ -7845,6 +7854,7 @@ impl<'de> serde::Deserialize<'de> for FilterExecNode { match value { "input" => Ok(GeneratedField::Input), "expr" => Ok(GeneratedField::Expr), + "defaultFilterSelectivity" | "default_filter_selectivity" => Ok(GeneratedField::DefaultFilterSelectivity), _ => Err(serde::de::Error::unknown_field(value, FIELDS)), } } @@ -7866,6 +7876,7 @@ impl<'de> serde::Deserialize<'de> for FilterExecNode { { let mut input__ = None; let mut expr__ = None; + let mut default_filter_selectivity__ = None; while let Some(k) = map_.next_key()? { match k { GeneratedField::Input => { @@ -7880,11 +7891,20 @@ impl<'de> serde::Deserialize<'de> for FilterExecNode { } expr__ = map_.next_value()?; } + GeneratedField::DefaultFilterSelectivity => { + if default_filter_selectivity__.is_some() { + return Err(serde::de::Error::duplicate_field("defaultFilterSelectivity")); + } + default_filter_selectivity__ = + Some(map_.next_value::<::pbjson::private::NumberDeserialize<_>>()?.0) + ; + } } } Ok(FilterExecNode { input: input__, expr: expr__, + default_filter_selectivity: default_filter_selectivity__.unwrap_or_default(), }) } } diff --git a/datafusion/proto/src/generated/prost.rs b/datafusion/proto/src/generated/prost.rs index 9e78b7c8d6dd..b9fb616b3133 100644 --- a/datafusion/proto/src/generated/prost.rs +++ b/datafusion/proto/src/generated/prost.rs @@ -1916,6 +1916,8 @@ pub struct FilterExecNode { pub input: ::core::option::Option<::prost::alloc::boxed::Box>, #[prost(message, optional, tag = "2")] pub expr: ::core::option::Option, + #[prost(uint32, tag = "3")] + pub default_filter_selectivity: u32, } #[allow(clippy::derive_partial_eq_without_eq)] #[derive(Clone, PartialEq, ::prost::Message)] diff --git a/datafusion/proto/src/physical_plan/mod.rs b/datafusion/proto/src/physical_plan/mod.rs index 907ba04ebc20..74c8ec894ff2 100644 --- a/datafusion/proto/src/physical_plan/mod.rs +++ b/datafusion/proto/src/physical_plan/mod.rs @@ -158,7 +158,16 @@ impl AsExecutionPlan for PhysicalPlanNode { .to_owned(), ) })?; - Ok(Arc::new(FilterExec::try_new(predicate, input)?)) + let filter_selectivity = filter.default_filter_selectivity.try_into(); + let filter = FilterExec::try_new(predicate, input)?; + match filter_selectivity { + Ok(filter_selectivity) => Ok(Arc::new( + filter.with_default_selectivity(filter_selectivity)?, + )), + Err(_) => Err(DataFusionError::Internal( + "filter_selectivity in PhysicalPlanNode is invalid ".to_owned(), + )), + } } PhysicalPlanType::CsvScan(scan) => Ok(Arc::new(CsvExec::new( parse_protobuf_file_scan_config( @@ -988,6 +997,7 @@ impl AsExecutionPlan for PhysicalPlanNode { protobuf::FilterExecNode { input: Some(Box::new(input)), expr: Some(exec.predicate().clone().try_into()?), + default_filter_selectivity: exec.default_selectivity() as u32, }, ))), }); diff --git a/datafusion/sqllogictest/test_files/information_schema.slt b/datafusion/sqllogictest/test_files/information_schema.slt index 741ff724781f..5c6bf6e2dac1 100644 --- a/datafusion/sqllogictest/test_files/information_schema.slt +++ b/datafusion/sqllogictest/test_files/information_schema.slt @@ -188,6 +188,7 @@ datafusion.explain.logical_plan_only false datafusion.explain.physical_plan_only false datafusion.explain.show_statistics false datafusion.optimizer.allow_symmetric_joins_without_pruning true +datafusion.optimizer.default_filter_selectivity 20 datafusion.optimizer.enable_distinct_aggregation_soft_limit true datafusion.optimizer.enable_round_robin_repartition true datafusion.optimizer.enable_topk_aggregation true @@ -261,6 +262,7 @@ datafusion.explain.logical_plan_only false When set to true, the explain stateme datafusion.explain.physical_plan_only false When set to true, the explain statement will only print physical plans datafusion.explain.show_statistics false When set to true, the explain statement will print operator statistics for physical plans datafusion.optimizer.allow_symmetric_joins_without_pruning true Should DataFusion allow symmetric hash joins for unbounded data sources even when its inputs do not have any ordering or filtering If the flag is not enabled, the SymmetricHashJoin operator will be unable to prune its internal buffers, resulting in certain join types - such as Full, Left, LeftAnti, LeftSemi, Right, RightAnti, and RightSemi - being produced only at the end of the execution. This is not typical in stream processing. Additionally, without proper design for long runner execution, all types of joins may encounter out-of-memory errors. +datafusion.optimizer.default_filter_selectivity 20 The default filter selectivity used by Filter Statistics when an exact selectivity cannot be determined. Valid values are between 0 (no selectivity) and 100 (all rows are selected). datafusion.optimizer.enable_distinct_aggregation_soft_limit true When set to true, the optimizer will push a limit operation into grouped aggregations which have no aggregate expressions, as a soft limit, emitting groups once the limit is reached, before all rows in the group are read. datafusion.optimizer.enable_round_robin_repartition true When set to true, the physical plan optimizer will try to add round robin repartitioning to increase parallelism to leverage more CPU cores datafusion.optimizer.enable_topk_aggregation true When set to true, the optimizer will attempt to perform limit operations during aggregations, if possible diff --git a/docs/source/user-guide/configs.md b/docs/source/user-guide/configs.md index 11363f0657f6..d5a43e429e09 100644 --- a/docs/source/user-guide/configs.md +++ b/docs/source/user-guide/configs.md @@ -99,6 +99,7 @@ Environment variables are read during `SessionConfig` initialisation so they mus | datafusion.optimizer.top_down_join_key_reordering | true | When set to true, the physical plan optimizer will run a top down process to reorder the join keys | | datafusion.optimizer.prefer_hash_join | true | When set to true, the physical plan optimizer will prefer HashJoin over SortMergeJoin. HashJoin can work more efficiently than SortMergeJoin but consumes more memory | | datafusion.optimizer.hash_join_single_partition_threshold | 1048576 | The maximum estimated size in bytes for one input side of a HashJoin will be collected into a single partition | +| datafusion.optimizer.default_filter_selectivity | 20 | The default filter selectivity used by Filter Statistics when an exact selectivity cannot be determined. Valid values are between 0 (no selectivity) and 100 (all rows are selected). | | datafusion.explain.logical_plan_only | false | When set to true, the explain statement will only print logical plans | | datafusion.explain.physical_plan_only | false | When set to true, the explain statement will only print physical plans | | datafusion.explain.show_statistics | false | When set to true, the explain statement will print operator statistics for physical plans | From 3ddd5ebc80444a5b40fa6b916f6ae69e5ed78d7d Mon Sep 17 00:00:00 2001 From: Liang-Chi Hsieh Date: Tue, 5 Dec 2023 16:47:27 -0800 Subject: [PATCH 369/572] fix: Changed labeler.yml to latest format (#8431) * fix: Changed labeler.yml to latest format * Use all * More * More * Try * More --- .github/workflows/dev_pr/labeler.yml | 34 +++++++++++++++------------- 1 file changed, 18 insertions(+), 16 deletions(-) diff --git a/.github/workflows/dev_pr/labeler.yml b/.github/workflows/dev_pr/labeler.yml index e84cf5efb1d8..34a37948785b 100644 --- a/.github/workflows/dev_pr/labeler.yml +++ b/.github/workflows/dev_pr/labeler.yml @@ -16,35 +16,37 @@ # under the License. development-process: - - dev/**.* - - .github/**.* - - ci/**.* - - .asf.yaml +- changed-files: + - any-glob-to-any-file: ['dev/**.*', '.github/**.*', 'ci/**.*', '.asf.yaml'] documentation: - - docs/**.* - - README.md - - ./**/README.md - - DEVELOPERS.md - - datafusion/docs/**.* +- changed-files: + - any-glob-to-any-file: ['docs/**.*', 'README.md', './**/README.md', 'DEVELOPERS.md', 'datafusion/docs/**.*'] sql: - - datafusion/sql/**/* +- changed-files: + - any-glob-to-any-file: ['datafusion/sql/**/*'] logical-expr: - - datafusion/expr/**/* +- changed-files: + - any-glob-to-any-file: ['datafusion/expr/**/*'] physical-expr: - - datafusion/physical-expr/**/* +- changed-files: + - any-glob-to-any-file: ['datafusion/physical-expr/**/*'] optimizer: - - datafusion/optimizer/**/* +- changed-files: + - any-glob-to-any-file: ['datafusion/optimizer/**/*'] core: - - datafusion/core/**/* +- changed-files: + - any-glob-to-any-file: ['datafusion/core/**/*'] substrait: - - datafusion/substrait/**/* +- changed-files: + - any-glob-to-any-file: ['datafusion/substrait/**/*'] sqllogictest: - - datafusion/sqllogictest/**/* +- changed-files: + - any-glob-to-any-file: ['datafusion/sqllogictest/**/*'] From fd92bcb225ded5b9c4c8b2661a8b3a33868dda0f Mon Sep 17 00:00:00 2001 From: Andrew Lamb Date: Wed, 6 Dec 2023 03:12:39 -0500 Subject: [PATCH 370/572] Minor: Use `ScalarValue::from` impl for strings (#8429) * Minor: Use ScalarValue::from impl for strings * fix typo --- datafusion/common/src/pyarrow.rs | 2 +- datafusion/common/src/scalar.rs | 25 +++++------ .../core/src/datasource/listing/helpers.rs | 20 ++------- .../core/src/datasource/physical_plan/avro.rs | 3 +- .../core/src/datasource/physical_plan/csv.rs | 3 +- .../physical_plan/file_scan_config.rs | 42 ++++++------------- .../datasource/physical_plan/parquet/mod.rs | 4 +- datafusion/core/src/test/variable.rs | 4 +- datafusion/execution/src/config.rs | 2 +- datafusion/expr/src/expr.rs | 2 +- datafusion/expr/src/literal.rs | 6 +-- .../simplify_expressions/expr_simplifier.rs | 9 ++-- .../src/simplify_expressions/guarantees.rs | 6 +-- .../src/simplify_expressions/regex.rs | 2 +- datafusion/physical-expr/benches/in_list.rs | 2 +- .../physical-expr/src/aggregate/min_max.rs | 14 +------ .../physical-expr/src/aggregate/string_agg.rs | 2 +- .../physical-expr/src/datetime_expressions.rs | 2 +- .../src/expressions/get_indexed_field.rs | 2 +- .../physical-expr/src/expressions/nullif.rs | 2 +- datafusion/physical-expr/src/functions.rs | 4 +- .../physical-plan/src/joins/cross_join.rs | 32 ++++---------- datafusion/physical-plan/src/projection.rs | 16 ++----- datafusion/physical-plan/src/union.rs | 24 +++-------- .../tests/cases/roundtrip_logical_plan.rs | 2 +- .../tests/cases/roundtrip_physical_plan.rs | 2 +- datafusion/sql/src/expr/mod.rs | 4 +- datafusion/sql/tests/sql_integration.rs | 12 +++--- 28 files changed, 83 insertions(+), 167 deletions(-) diff --git a/datafusion/common/src/pyarrow.rs b/datafusion/common/src/pyarrow.rs index aa0153919360..f4356477532f 100644 --- a/datafusion/common/src/pyarrow.rs +++ b/datafusion/common/src/pyarrow.rs @@ -119,7 +119,7 @@ mod tests { ScalarValue::Boolean(Some(true)), ScalarValue::Int32(Some(23)), ScalarValue::Float64(Some(12.34)), - ScalarValue::Utf8(Some("Hello!".to_string())), + ScalarValue::from("Hello!"), ScalarValue::Date32(Some(1234)), ]; diff --git a/datafusion/common/src/scalar.rs b/datafusion/common/src/scalar.rs index 177fe00a6a3c..10f052b90923 100644 --- a/datafusion/common/src/scalar.rs +++ b/datafusion/common/src/scalar.rs @@ -774,7 +774,7 @@ impl ScalarValue { /// Returns a [`ScalarValue::Utf8`] representing `val` pub fn new_utf8(val: impl Into) -> Self { - ScalarValue::Utf8(Some(val.into())) + ScalarValue::from(val.into()) } /// Returns a [`ScalarValue::IntervalYearMonth`] representing @@ -2699,7 +2699,7 @@ impl ScalarValue { /// Try to parse `value` into a ScalarValue of type `target_type` pub fn try_from_string(value: String, target_type: &DataType) -> Result { - let value = ScalarValue::Utf8(Some(value)); + let value = ScalarValue::from(value); let cast_options = CastOptions { safe: false, format_options: Default::default(), @@ -3581,9 +3581,9 @@ mod tests { #[test] fn test_list_to_array_string() { let scalars = vec![ - ScalarValue::Utf8(Some(String::from("rust"))), - ScalarValue::Utf8(Some(String::from("arrow"))), - ScalarValue::Utf8(Some(String::from("data-fusion"))), + ScalarValue::from("rust"), + ScalarValue::from("arrow"), + ScalarValue::from("data-fusion"), ]; let array = ScalarValue::new_list(scalars.as_slice(), &DataType::Utf8); @@ -4722,7 +4722,7 @@ mod tests { Some(vec![ ScalarValue::Int32(Some(23)), ScalarValue::Boolean(Some(false)), - ScalarValue::Utf8(Some("Hello".to_string())), + ScalarValue::from("Hello"), ScalarValue::from(vec![ ("e", ScalarValue::from(2i16)), ("f", ScalarValue::from(3i64)), @@ -4915,17 +4915,17 @@ mod tests { // Define struct scalars let s0 = ScalarValue::from(vec![ - ("A", ScalarValue::Utf8(Some(String::from("First")))), + ("A", ScalarValue::from("First")), ("primitive_list", l0), ]); let s1 = ScalarValue::from(vec![ - ("A", ScalarValue::Utf8(Some(String::from("Second")))), + ("A", ScalarValue::from("Second")), ("primitive_list", l1), ]); let s2 = ScalarValue::from(vec![ - ("A", ScalarValue::Utf8(Some(String::from("Third")))), + ("A", ScalarValue::from("Third")), ("primitive_list", l2), ]); @@ -5212,7 +5212,7 @@ mod tests { check_scalar_cast(ScalarValue::Float64(None), DataType::Int16); check_scalar_cast( - ScalarValue::Utf8(Some("foo".to_string())), + ScalarValue::from("foo"), DataType::Dictionary(Box::new(DataType::Int32), Box::new(DataType::Utf8)), ); @@ -5493,10 +5493,7 @@ mod tests { (ScalarValue::Int8(None), ScalarValue::Int16(Some(1))), (ScalarValue::Int8(Some(1)), ScalarValue::Int16(None)), // Unsupported types - ( - ScalarValue::Utf8(Some("foo".to_string())), - ScalarValue::Utf8(Some("bar".to_string())), - ), + (ScalarValue::from("foo"), ScalarValue::from("bar")), ( ScalarValue::Boolean(Some(true)), ScalarValue::Boolean(Some(false)), diff --git a/datafusion/core/src/datasource/listing/helpers.rs b/datafusion/core/src/datasource/listing/helpers.rs index a4505cf62d6a..3536c098bd76 100644 --- a/datafusion/core/src/datasource/listing/helpers.rs +++ b/datafusion/core/src/datasource/listing/helpers.rs @@ -526,19 +526,13 @@ mod tests { f1.object_meta.location.as_ref(), "tablepath/mypartition=val1/file.parquet" ); - assert_eq!( - &f1.partition_values, - &[ScalarValue::Utf8(Some(String::from("val1"))),] - ); + assert_eq!(&f1.partition_values, &[ScalarValue::from("val1")]); let f2 = &pruned[1]; assert_eq!( f2.object_meta.location.as_ref(), "tablepath/mypartition=val1/other=val3/file.parquet" ); - assert_eq!( - f2.partition_values, - &[ScalarValue::Utf8(Some(String::from("val1"))),] - ); + assert_eq!(f2.partition_values, &[ScalarValue::from("val1"),]); } #[tokio::test] @@ -579,10 +573,7 @@ mod tests { ); assert_eq!( &f1.partition_values, - &[ - ScalarValue::Utf8(Some(String::from("p1v2"))), - ScalarValue::Utf8(Some(String::from("p2v1"))) - ] + &[ScalarValue::from("p1v2"), ScalarValue::from("p2v1"),] ); let f2 = &pruned[1]; assert_eq!( @@ -591,10 +582,7 @@ mod tests { ); assert_eq!( &f2.partition_values, - &[ - ScalarValue::Utf8(Some(String::from("p1v2"))), - ScalarValue::Utf8(Some(String::from("p2v1"))) - ] + &[ScalarValue::from("p1v2"), ScalarValue::from("p2v1")] ); } diff --git a/datafusion/core/src/datasource/physical_plan/avro.rs b/datafusion/core/src/datasource/physical_plan/avro.rs index b97f162fd2f5..885b4c5d3911 100644 --- a/datafusion/core/src/datasource/physical_plan/avro.rs +++ b/datafusion/core/src/datasource/physical_plan/avro.rs @@ -406,8 +406,7 @@ mod tests { .await?; let mut partitioned_file = PartitionedFile::from(meta); - partitioned_file.partition_values = - vec![ScalarValue::Utf8(Some("2021-10-26".to_owned()))]; + partitioned_file.partition_values = vec![ScalarValue::from("2021-10-26")]; let avro_exec = AvroExec::new(FileScanConfig { // select specific columns of the files as well as the partitioning diff --git a/datafusion/core/src/datasource/physical_plan/csv.rs b/datafusion/core/src/datasource/physical_plan/csv.rs index 75aa343ffbfc..816a82543bab 100644 --- a/datafusion/core/src/datasource/physical_plan/csv.rs +++ b/datafusion/core/src/datasource/physical_plan/csv.rs @@ -872,8 +872,7 @@ mod tests { // Add partition columns config.table_partition_cols = vec![Field::new("date", DataType::Utf8, false)]; - config.file_groups[0][0].partition_values = - vec![ScalarValue::Utf8(Some("2021-10-26".to_owned()))]; + config.file_groups[0][0].partition_values = vec![ScalarValue::from("2021-10-26")]; // We should be able to project on the partition column // Which is supposed to be after the file fields diff --git a/datafusion/core/src/datasource/physical_plan/file_scan_config.rs b/datafusion/core/src/datasource/physical_plan/file_scan_config.rs index 68e996391cc3..d308397ab6e2 100644 --- a/datafusion/core/src/datasource/physical_plan/file_scan_config.rs +++ b/datafusion/core/src/datasource/physical_plan/file_scan_config.rs @@ -654,15 +654,9 @@ mod tests { // file_batch is ok here because we kept all the file cols in the projection file_batch, &[ - wrap_partition_value_in_dict(ScalarValue::Utf8(Some( - "2021".to_owned(), - ))), - wrap_partition_value_in_dict(ScalarValue::Utf8(Some( - "10".to_owned(), - ))), - wrap_partition_value_in_dict(ScalarValue::Utf8(Some( - "26".to_owned(), - ))), + wrap_partition_value_in_dict(ScalarValue::from("2021")), + wrap_partition_value_in_dict(ScalarValue::from("10")), + wrap_partition_value_in_dict(ScalarValue::from("26")), ], ) .expect("Projection of partition columns into record batch failed"); @@ -688,15 +682,9 @@ mod tests { // file_batch is ok here because we kept all the file cols in the projection file_batch, &[ - wrap_partition_value_in_dict(ScalarValue::Utf8(Some( - "2021".to_owned(), - ))), - wrap_partition_value_in_dict(ScalarValue::Utf8(Some( - "10".to_owned(), - ))), - wrap_partition_value_in_dict(ScalarValue::Utf8(Some( - "27".to_owned(), - ))), + wrap_partition_value_in_dict(ScalarValue::from("2021")), + wrap_partition_value_in_dict(ScalarValue::from("10")), + wrap_partition_value_in_dict(ScalarValue::from("27")), ], ) .expect("Projection of partition columns into record batch failed"); @@ -724,15 +712,9 @@ mod tests { // file_batch is ok here because we kept all the file cols in the projection file_batch, &[ - wrap_partition_value_in_dict(ScalarValue::Utf8(Some( - "2021".to_owned(), - ))), - wrap_partition_value_in_dict(ScalarValue::Utf8(Some( - "10".to_owned(), - ))), - wrap_partition_value_in_dict(ScalarValue::Utf8(Some( - "28".to_owned(), - ))), + wrap_partition_value_in_dict(ScalarValue::from("2021")), + wrap_partition_value_in_dict(ScalarValue::from("10")), + wrap_partition_value_in_dict(ScalarValue::from("28")), ], ) .expect("Projection of partition columns into record batch failed"); @@ -758,9 +740,9 @@ mod tests { // file_batch is ok here because we kept all the file cols in the projection file_batch, &[ - ScalarValue::Utf8(Some("2021".to_owned())), - ScalarValue::Utf8(Some("10".to_owned())), - ScalarValue::Utf8(Some("26".to_owned())), + ScalarValue::from("2021"), + ScalarValue::from("10"), + ScalarValue::from("26"), ], ) .expect("Projection of partition columns into record batch failed"); diff --git a/datafusion/core/src/datasource/physical_plan/parquet/mod.rs b/datafusion/core/src/datasource/physical_plan/parquet/mod.rs index 95aae71c779e..718f9f820af1 100644 --- a/datafusion/core/src/datasource/physical_plan/parquet/mod.rs +++ b/datafusion/core/src/datasource/physical_plan/parquet/mod.rs @@ -1603,11 +1603,11 @@ mod tests { let partitioned_file = PartitionedFile { object_meta: meta, partition_values: vec![ - ScalarValue::Utf8(Some("2021".to_owned())), + ScalarValue::from("2021"), ScalarValue::UInt8(Some(10)), ScalarValue::Dictionary( Box::new(DataType::UInt16), - Box::new(ScalarValue::Utf8(Some("26".to_owned()))), + Box::new(ScalarValue::from("26")), ), ], range: None, diff --git a/datafusion/core/src/test/variable.rs b/datafusion/core/src/test/variable.rs index a55513841561..38207b42cb7b 100644 --- a/datafusion/core/src/test/variable.rs +++ b/datafusion/core/src/test/variable.rs @@ -37,7 +37,7 @@ impl VarProvider for SystemVar { /// get system variable value fn get_value(&self, var_names: Vec) -> Result { let s = format!("{}-{}", "system-var", var_names.concat()); - Ok(ScalarValue::Utf8(Some(s))) + Ok(ScalarValue::from(s)) } fn get_type(&self, _: &[String]) -> Option { @@ -61,7 +61,7 @@ impl VarProvider for UserDefinedVar { fn get_value(&self, var_names: Vec) -> Result { if var_names[0] != "@integer" { let s = format!("{}-{}", "user-defined-var", var_names.concat()); - Ok(ScalarValue::Utf8(Some(s))) + Ok(ScalarValue::from(s)) } else { Ok(ScalarValue::Int32(Some(41))) } diff --git a/datafusion/execution/src/config.rs b/datafusion/execution/src/config.rs index cfcc205b5625..8556335b395a 100644 --- a/datafusion/execution/src/config.rs +++ b/datafusion/execution/src/config.rs @@ -86,7 +86,7 @@ impl SessionConfig { /// Set a generic `str` configuration option pub fn set_str(self, key: &str, value: &str) -> Self { - self.set(key, ScalarValue::Utf8(Some(value.to_string()))) + self.set(key, ScalarValue::from(value)) } /// Customize batch size diff --git a/datafusion/expr/src/expr.rs b/datafusion/expr/src/expr.rs index 6fa400454dff..958f4f4a3456 100644 --- a/datafusion/expr/src/expr.rs +++ b/datafusion/expr/src/expr.rs @@ -1044,7 +1044,7 @@ impl Expr { Expr::GetIndexedField(GetIndexedField { expr: Box::new(self), field: GetFieldAccess::NamedStructField { - name: ScalarValue::Utf8(Some(name.into())), + name: ScalarValue::from(name.into()), }, }) } diff --git a/datafusion/expr/src/literal.rs b/datafusion/expr/src/literal.rs index effc31553819..2f04729af2ed 100644 --- a/datafusion/expr/src/literal.rs +++ b/datafusion/expr/src/literal.rs @@ -43,19 +43,19 @@ pub trait TimestampLiteral { impl Literal for &str { fn lit(&self) -> Expr { - Expr::Literal(ScalarValue::Utf8(Some((*self).to_owned()))) + Expr::Literal(ScalarValue::from(*self)) } } impl Literal for String { fn lit(&self) -> Expr { - Expr::Literal(ScalarValue::Utf8(Some((*self).to_owned()))) + Expr::Literal(ScalarValue::from(self.as_ref())) } } impl Literal for &String { fn lit(&self) -> Expr { - Expr::Literal(ScalarValue::Utf8(Some((*self).to_owned()))) + Expr::Literal(ScalarValue::from(self.as_ref())) } } diff --git a/datafusion/optimizer/src/simplify_expressions/expr_simplifier.rs b/datafusion/optimizer/src/simplify_expressions/expr_simplifier.rs index 41c71c9d9aff..e2fbd5e927a1 100644 --- a/datafusion/optimizer/src/simplify_expressions/expr_simplifier.rs +++ b/datafusion/optimizer/src/simplify_expressions/expr_simplifier.rs @@ -3297,10 +3297,7 @@ mod tests { col("c4"), NullableInterval::from(ScalarValue::UInt32(Some(9))), ), - ( - col("c1"), - NullableInterval::from(ScalarValue::Utf8(Some("a".to_string()))), - ), + (col("c1"), NullableInterval::from(ScalarValue::from("a"))), ]; let output = simplify_with_guarantee(expr.clone(), guarantees); assert_eq!(output, lit(false)); @@ -3323,8 +3320,8 @@ mod tests { col("c1"), NullableInterval::NotNull { values: Interval::try_new( - ScalarValue::Utf8(Some("d".to_string())), - ScalarValue::Utf8(Some("f".to_string())), + ScalarValue::from("d"), + ScalarValue::from("f"), ) .unwrap(), }, diff --git a/datafusion/optimizer/src/simplify_expressions/guarantees.rs b/datafusion/optimizer/src/simplify_expressions/guarantees.rs index 3cfaae858e2d..860dc326b9b0 100644 --- a/datafusion/optimizer/src/simplify_expressions/guarantees.rs +++ b/datafusion/optimizer/src/simplify_expressions/guarantees.rs @@ -406,8 +406,8 @@ mod tests { col("x"), NullableInterval::MaybeNull { values: Interval::try_new( - ScalarValue::Utf8(Some("abc".to_string())), - ScalarValue::Utf8(Some("def".to_string())), + ScalarValue::from("abc"), + ScalarValue::from("def"), ) .unwrap(), }, @@ -463,7 +463,7 @@ mod tests { ScalarValue::Int32(Some(1)), ScalarValue::Boolean(Some(true)), ScalarValue::Boolean(None), - ScalarValue::Utf8(Some("abc".to_string())), + ScalarValue::from("abc"), ScalarValue::LargeUtf8(Some("def".to_string())), ScalarValue::Date32(Some(18628)), ScalarValue::Date32(None), diff --git a/datafusion/optimizer/src/simplify_expressions/regex.rs b/datafusion/optimizer/src/simplify_expressions/regex.rs index b9d9821b43f0..175b70f2b10e 100644 --- a/datafusion/optimizer/src/simplify_expressions/regex.rs +++ b/datafusion/optimizer/src/simplify_expressions/regex.rs @@ -84,7 +84,7 @@ impl OperatorMode { let like = Like { negated: self.not, expr, - pattern: Box::new(Expr::Literal(ScalarValue::Utf8(Some(pattern)))), + pattern: Box::new(Expr::Literal(ScalarValue::from(pattern))), escape_char: None, case_insensitive: self.i, }; diff --git a/datafusion/physical-expr/benches/in_list.rs b/datafusion/physical-expr/benches/in_list.rs index db017326083a..90bfc5efb61e 100644 --- a/datafusion/physical-expr/benches/in_list.rs +++ b/datafusion/physical-expr/benches/in_list.rs @@ -57,7 +57,7 @@ fn do_benches( .collect(); let in_list: Vec<_> = (0..in_list_length) - .map(|_| ScalarValue::Utf8(Some(random_string(&mut rng, string_length)))) + .map(|_| ScalarValue::from(random_string(&mut rng, string_length))) .collect(); do_bench( diff --git a/datafusion/physical-expr/src/aggregate/min_max.rs b/datafusion/physical-expr/src/aggregate/min_max.rs index f5b708e8894e..7e3ef2a2abab 100644 --- a/datafusion/physical-expr/src/aggregate/min_max.rs +++ b/datafusion/physical-expr/src/aggregate/min_max.rs @@ -1297,12 +1297,7 @@ mod tests { #[test] fn max_utf8() -> Result<()> { let a: ArrayRef = Arc::new(StringArray::from(vec!["d", "a", "c", "b"])); - generic_test_op!( - a, - DataType::Utf8, - Max, - ScalarValue::Utf8(Some("d".to_string())) - ) + generic_test_op!(a, DataType::Utf8, Max, ScalarValue::from("d")) } #[test] @@ -1319,12 +1314,7 @@ mod tests { #[test] fn min_utf8() -> Result<()> { let a: ArrayRef = Arc::new(StringArray::from(vec!["d", "a", "c", "b"])); - generic_test_op!( - a, - DataType::Utf8, - Min, - ScalarValue::Utf8(Some("a".to_string())) - ) + generic_test_op!(a, DataType::Utf8, Min, ScalarValue::from("a")) } #[test] diff --git a/datafusion/physical-expr/src/aggregate/string_agg.rs b/datafusion/physical-expr/src/aggregate/string_agg.rs index 74c083959ed8..7adc736932ad 100644 --- a/datafusion/physical-expr/src/aggregate/string_agg.rs +++ b/datafusion/physical-expr/src/aggregate/string_agg.rs @@ -204,7 +204,7 @@ mod tests { ) .unwrap(); - let delimiter = Arc::new(Literal::new(ScalarValue::Utf8(Some(delimiter)))); + let delimiter = Arc::new(Literal::new(ScalarValue::from(delimiter))); let schema = Schema::new(vec![Field::new("a", coerced[0].clone(), true)]); let agg = create_aggregate_expr( &function, diff --git a/datafusion/physical-expr/src/datetime_expressions.rs b/datafusion/physical-expr/src/datetime_expressions.rs index a4d8118cf86b..04cfec29ea8a 100644 --- a/datafusion/physical-expr/src/datetime_expressions.rs +++ b/datafusion/physical-expr/src/datetime_expressions.rs @@ -1349,7 +1349,7 @@ mod tests { .collect::() .with_timezone_opt(tz_opt.clone()); let result = date_trunc(&[ - ColumnarValue::Scalar(ScalarValue::Utf8(Some("day".to_string()))), + ColumnarValue::Scalar(ScalarValue::from("day")), ColumnarValue::Array(Arc::new(input)), ]) .unwrap(); diff --git a/datafusion/physical-expr/src/expressions/get_indexed_field.rs b/datafusion/physical-expr/src/expressions/get_indexed_field.rs index 7d5f16c454d6..9c2a64723dc6 100644 --- a/datafusion/physical-expr/src/expressions/get_indexed_field.rs +++ b/datafusion/physical-expr/src/expressions/get_indexed_field.rs @@ -110,7 +110,7 @@ impl GetIndexedFieldExpr { Self::new( arg, GetFieldAccessExpr::NamedStructField { - name: ScalarValue::Utf8(Some(name.into())), + name: ScalarValue::from(name.into()), }, ) } diff --git a/datafusion/physical-expr/src/expressions/nullif.rs b/datafusion/physical-expr/src/expressions/nullif.rs index 252bd10c3e73..dcd883f92965 100644 --- a/datafusion/physical-expr/src/expressions/nullif.rs +++ b/datafusion/physical-expr/src/expressions/nullif.rs @@ -154,7 +154,7 @@ mod tests { let a = StringArray::from(vec![Some("foo"), Some("bar"), None, Some("baz")]); let a = ColumnarValue::Array(Arc::new(a)); - let lit_array = ColumnarValue::Scalar(ScalarValue::Utf8(Some("bar".to_string()))); + let lit_array = ColumnarValue::Scalar(ScalarValue::from("bar")); let result = nullif_func(&[a, lit_array])?; let result = result.into_array(0).expect("Failed to convert to array"); diff --git a/datafusion/physical-expr/src/functions.rs b/datafusion/physical-expr/src/functions.rs index 72c7f492166d..b5d7a8e97dd6 100644 --- a/datafusion/physical-expr/src/functions.rs +++ b/datafusion/physical-expr/src/functions.rs @@ -834,9 +834,9 @@ pub fn create_physical_fun( } let input_data_type = args[0].data_type(); - Ok(ColumnarValue::Scalar(ScalarValue::Utf8(Some(format!( + Ok(ColumnarValue::Scalar(ScalarValue::from(format!( "{input_data_type}" - ))))) + )))) }), BuiltinScalarFunction::OverLay => Arc::new(|args| match args[0].data_type() { DataType::Utf8 => { diff --git a/datafusion/physical-plan/src/joins/cross_join.rs b/datafusion/physical-plan/src/joins/cross_join.rs index 4c928d44caf4..938c9e4d343d 100644 --- a/datafusion/physical-plan/src/joins/cross_join.rs +++ b/datafusion/physical-plan/src/joins/cross_join.rs @@ -476,12 +476,8 @@ mod tests { }, ColumnStatistics { distinct_count: Precision::Exact(1), - max_value: Precision::Exact(ScalarValue::Utf8(Some(String::from( - "x", - )))), - min_value: Precision::Exact(ScalarValue::Utf8(Some(String::from( - "a", - )))), + max_value: Precision::Exact(ScalarValue::from("x")), + min_value: Precision::Exact(ScalarValue::from("a")), null_count: Precision::Exact(3), }, ], @@ -512,12 +508,8 @@ mod tests { }, ColumnStatistics { distinct_count: Precision::Exact(1), - max_value: Precision::Exact(ScalarValue::Utf8(Some(String::from( - "x", - )))), - min_value: Precision::Exact(ScalarValue::Utf8(Some(String::from( - "a", - )))), + max_value: Precision::Exact(ScalarValue::from("x")), + min_value: Precision::Exact(ScalarValue::from("a")), null_count: Precision::Exact(3 * right_row_count), }, ColumnStatistics { @@ -548,12 +540,8 @@ mod tests { }, ColumnStatistics { distinct_count: Precision::Exact(1), - max_value: Precision::Exact(ScalarValue::Utf8(Some(String::from( - "x", - )))), - min_value: Precision::Exact(ScalarValue::Utf8(Some(String::from( - "a", - )))), + max_value: Precision::Exact(ScalarValue::from("x")), + min_value: Precision::Exact(ScalarValue::from("a")), null_count: Precision::Exact(3), }, ], @@ -584,12 +572,8 @@ mod tests { }, ColumnStatistics { distinct_count: Precision::Exact(1), - max_value: Precision::Exact(ScalarValue::Utf8(Some(String::from( - "x", - )))), - min_value: Precision::Exact(ScalarValue::Utf8(Some(String::from( - "a", - )))), + max_value: Precision::Exact(ScalarValue::from("x")), + min_value: Precision::Exact(ScalarValue::from("a")), null_count: Precision::Absent, // we don't know the row count on the right }, ColumnStatistics { diff --git a/datafusion/physical-plan/src/projection.rs b/datafusion/physical-plan/src/projection.rs index 2e1d3dbf94f5..cc2ab62049ed 100644 --- a/datafusion/physical-plan/src/projection.rs +++ b/datafusion/physical-plan/src/projection.rs @@ -397,12 +397,8 @@ mod tests { }, ColumnStatistics { distinct_count: Precision::Exact(1), - max_value: Precision::Exact(ScalarValue::Utf8(Some(String::from( - "x", - )))), - min_value: Precision::Exact(ScalarValue::Utf8(Some(String::from( - "a", - )))), + max_value: Precision::Exact(ScalarValue::from("x")), + min_value: Precision::Exact(ScalarValue::from("a")), null_count: Precision::Exact(3), }, ColumnStatistics { @@ -439,12 +435,8 @@ mod tests { column_statistics: vec![ ColumnStatistics { distinct_count: Precision::Exact(1), - max_value: Precision::Exact(ScalarValue::Utf8(Some(String::from( - "x", - )))), - min_value: Precision::Exact(ScalarValue::Utf8(Some(String::from( - "a", - )))), + max_value: Precision::Exact(ScalarValue::from("x")), + min_value: Precision::Exact(ScalarValue::from("a")), null_count: Precision::Exact(3), }, ColumnStatistics { diff --git a/datafusion/physical-plan/src/union.rs b/datafusion/physical-plan/src/union.rs index 92ad0f4e65db..14ef9c2ec27b 100644 --- a/datafusion/physical-plan/src/union.rs +++ b/datafusion/physical-plan/src/union.rs @@ -674,12 +674,8 @@ mod tests { }, ColumnStatistics { distinct_count: Precision::Exact(1), - max_value: Precision::Exact(ScalarValue::Utf8(Some(String::from( - "x", - )))), - min_value: Precision::Exact(ScalarValue::Utf8(Some(String::from( - "a", - )))), + max_value: Precision::Exact(ScalarValue::from("x")), + min_value: Precision::Exact(ScalarValue::from("a")), null_count: Precision::Exact(3), }, ColumnStatistics { @@ -703,12 +699,8 @@ mod tests { }, ColumnStatistics { distinct_count: Precision::Absent, - max_value: Precision::Exact(ScalarValue::Utf8(Some(String::from( - "c", - )))), - min_value: Precision::Exact(ScalarValue::Utf8(Some(String::from( - "b", - )))), + max_value: Precision::Exact(ScalarValue::from("c")), + min_value: Precision::Exact(ScalarValue::from("b")), null_count: Precision::Absent, }, ColumnStatistics { @@ -733,12 +725,8 @@ mod tests { }, ColumnStatistics { distinct_count: Precision::Absent, - max_value: Precision::Exact(ScalarValue::Utf8(Some(String::from( - "x", - )))), - min_value: Precision::Exact(ScalarValue::Utf8(Some(String::from( - "a", - )))), + max_value: Precision::Exact(ScalarValue::from("x")), + min_value: Precision::Exact(ScalarValue::from("a")), null_count: Precision::Absent, }, ColumnStatistics { diff --git a/datafusion/proto/tests/cases/roundtrip_logical_plan.rs b/datafusion/proto/tests/cases/roundtrip_logical_plan.rs index 45727c39a373..e04a7a9c9d03 100644 --- a/datafusion/proto/tests/cases/roundtrip_logical_plan.rs +++ b/datafusion/proto/tests/cases/roundtrip_logical_plan.rs @@ -730,7 +730,7 @@ fn round_trip_scalar_values() { ))), ScalarValue::Dictionary( Box::new(DataType::Int32), - Box::new(ScalarValue::Utf8(Some("foo".into()))), + Box::new(ScalarValue::from("foo")), ), ScalarValue::Dictionary( Box::new(DataType::Int32), diff --git a/datafusion/proto/tests/cases/roundtrip_physical_plan.rs b/datafusion/proto/tests/cases/roundtrip_physical_plan.rs index d7d762d470d7..287207bae5f6 100644 --- a/datafusion/proto/tests/cases/roundtrip_physical_plan.rs +++ b/datafusion/proto/tests/cases/roundtrip_physical_plan.rs @@ -636,7 +636,7 @@ fn roundtrip_get_indexed_field_named_struct_field() -> Result<()> { let get_indexed_field_expr = Arc::new(GetIndexedFieldExpr::new( col_arg, GetFieldAccessExpr::NamedStructField { - name: ScalarValue::Utf8(Some(String::from("name"))), + name: ScalarValue::from("name"), }, )); diff --git a/datafusion/sql/src/expr/mod.rs b/datafusion/sql/src/expr/mod.rs index b8c130055a5a..2f44466c79c3 100644 --- a/datafusion/sql/src/expr/mod.rs +++ b/datafusion/sql/src/expr/mod.rs @@ -171,7 +171,7 @@ impl<'a, S: ContextProvider> SqlToRel<'a, S> { Ok(Expr::ScalarFunction(ScalarFunction::new( BuiltinScalarFunction::DatePart, vec![ - Expr::Literal(ScalarValue::Utf8(Some(format!("{field}")))), + Expr::Literal(ScalarValue::from(format!("{field}"))), self.sql_expr_to_logical_expr(*expr, schema, planner_context)?, ], ))) @@ -739,7 +739,7 @@ impl<'a, S: ContextProvider> SqlToRel<'a, S> { SQLExpr::Value( Value::SingleQuotedString(s) | Value::DoubleQuotedString(s), ) => GetFieldAccess::NamedStructField { - name: ScalarValue::Utf8(Some(s)), + name: ScalarValue::from(s), }, SQLExpr::JsonAccess { left, diff --git a/datafusion/sql/tests/sql_integration.rs b/datafusion/sql/tests/sql_integration.rs index 83bdb954b134..944a383ee4b8 100644 --- a/datafusion/sql/tests/sql_integration.rs +++ b/datafusion/sql/tests/sql_integration.rs @@ -4038,8 +4038,8 @@ fn test_prepare_statement_insert_infer() { // replace params with values let param_values = vec![ ScalarValue::UInt32(Some(1)), - ScalarValue::Utf8(Some("Alan".to_string())), - ScalarValue::Utf8(Some("Turing".to_string())), + ScalarValue::from("Alan"), + ScalarValue::from("Turing"), ] .into(); let expected_plan = "Dml: op=[Insert Into] table=[person]\ @@ -4120,11 +4120,11 @@ fn test_prepare_statement_to_plan_multi_params() { // replace params with values let param_values = vec![ ScalarValue::Int32(Some(10)), - ScalarValue::Utf8(Some("abc".to_string())), + ScalarValue::from("abc"), ScalarValue::Float64(Some(100.0)), ScalarValue::Int32(Some(20)), ScalarValue::Float64(Some(200.0)), - ScalarValue::Utf8(Some("xyz".to_string())), + ScalarValue::from("xyz"), ]; let expected_plan = "Projection: person.id, person.age, Utf8(\"xyz\")\ @@ -4190,8 +4190,8 @@ fn test_prepare_statement_to_plan_value_list() { /////////////////// // replace params with values let param_values = vec![ - ScalarValue::Utf8(Some("a".to_string())), - ScalarValue::Utf8(Some("b".to_string())), + ScalarValue::from("a".to_string()), + ScalarValue::from("b".to_string()), ]; let expected_plan = "Projection: t.num, t.letter\ \n SubqueryAlias: t\ From 0d7cab055cb39d6df751e070af5a0bf5444e3849 Mon Sep 17 00:00:00 2001 From: yi wang <48236141+my-vegetable-has-exploded@users.noreply.github.com> Date: Wed, 6 Dec 2023 19:40:01 +0800 Subject: [PATCH 371/572] Support crossjoin in substrait. (#8427) * Support crossjoin in substrait. * use struct destructuring * remove useless builder. --- .../substrait/src/logical_plan/consumer.rs | 9 +++++++++ .../substrait/src/logical_plan/producer.rs | 20 ++++++++++++++++++- .../tests/cases/roundtrip_logical_plan.rs | 5 +++++ 3 files changed, 33 insertions(+), 1 deletion(-) diff --git a/datafusion/substrait/src/logical_plan/consumer.rs b/datafusion/substrait/src/logical_plan/consumer.rs index cf05d814a5cb..ffc9d094ab91 100644 --- a/datafusion/substrait/src/logical_plan/consumer.rs +++ b/datafusion/substrait/src/logical_plan/consumer.rs @@ -434,6 +434,15 @@ pub async fn from_substrait_rel( None => plan_err!("JoinRel without join condition is not allowed"), } } + Some(RelType::Cross(cross)) => { + let left: LogicalPlanBuilder = LogicalPlanBuilder::from( + from_substrait_rel(ctx, cross.left.as_ref().unwrap(), extensions).await?, + ); + let right = + from_substrait_rel(ctx, cross.right.as_ref().unwrap(), extensions) + .await?; + left.cross_join(right)?.build() + } Some(RelType::Read(read)) => match &read.as_ref().read_type { Some(ReadType::NamedTable(nt)) => { let table_reference = match nt.names.len() { diff --git a/datafusion/substrait/src/logical_plan/producer.rs b/datafusion/substrait/src/logical_plan/producer.rs index d576e70711df..c5f1278be6e0 100644 --- a/datafusion/substrait/src/logical_plan/producer.rs +++ b/datafusion/substrait/src/logical_plan/producer.rs @@ -19,7 +19,7 @@ use std::collections::HashMap; use std::ops::Deref; use std::sync::Arc; -use datafusion::logical_expr::{Distinct, Like, WindowFrameUnits}; +use datafusion::logical_expr::{CrossJoin, Distinct, Like, WindowFrameUnits}; use datafusion::{ arrow::datatypes::{DataType, TimeUnit}, error::{DataFusionError, Result}, @@ -40,6 +40,7 @@ use datafusion::logical_expr::{expr, Between, JoinConstraint, LogicalPlan, Opera use datafusion::prelude::Expr; use prost_types::Any as ProtoAny; use substrait::proto::expression::window_function::BoundsType; +use substrait::proto::CrossRel; use substrait::{ proto::{ aggregate_function::AggregationInvocation, @@ -332,6 +333,23 @@ pub fn to_substrait_rel( }))), })) } + LogicalPlan::CrossJoin(cross_join) => { + let CrossJoin { + left, + right, + schema: _, + } = cross_join; + let left = to_substrait_rel(left.as_ref(), ctx, extension_info)?; + let right = to_substrait_rel(right.as_ref(), ctx, extension_info)?; + Ok(Box::new(Rel { + rel_type: Some(RelType::Cross(Box::new(CrossRel { + common: None, + left: Some(left), + right: Some(right), + advanced_extension: None, + }))), + })) + } LogicalPlan::SubqueryAlias(alias) => { // Do nothing if encounters SubqueryAlias // since there is no corresponding relation type in Substrait diff --git a/datafusion/substrait/tests/cases/roundtrip_logical_plan.rs b/datafusion/substrait/tests/cases/roundtrip_logical_plan.rs index 1c5dbe9ce884..691fba864449 100644 --- a/datafusion/substrait/tests/cases/roundtrip_logical_plan.rs +++ b/datafusion/substrait/tests/cases/roundtrip_logical_plan.rs @@ -394,6 +394,11 @@ async fn roundtrip_inlist_4() -> Result<()> { roundtrip("SELECT * FROM data WHERE f NOT IN ('a', 'b', 'c', 'd')").await } +#[tokio::test] +async fn roundtrip_cross_join() -> Result<()> { + roundtrip("SELECT * FROM data CROSS JOIN data2").await +} + #[tokio::test] async fn roundtrip_inner_join() -> Result<()> { roundtrip("SELECT data.a FROM data JOIN data2 ON data.a = data2.a").await From eb08846c71c3435515b7dba496cb0cbe7f968995 Mon Sep 17 00:00:00 2001 From: Asura7969 <1402357969@qq.com> Date: Thu, 7 Dec 2023 05:17:25 +0800 Subject: [PATCH 372/572] Fix ambiguous reference when aliasing in combination with `ORDER BY` (#8425) * Minor: Improve the document format of JoinHashMap * ambiguous reference * ignore aliases * fix * fix * add test * add test * add test * add test --- .../optimizer/tests/optimizer_integration.rs | 9 ++++----- datafusion/sql/src/select.rs | 7 ++++++- datafusion/sql/tests/sql_integration.rs | 17 ++++++++++++++--- datafusion/sqllogictest/test_files/select.slt | 15 +++++++++++++++ 4 files changed, 39 insertions(+), 9 deletions(-) diff --git a/datafusion/optimizer/tests/optimizer_integration.rs b/datafusion/optimizer/tests/optimizer_integration.rs index e593b07361e2..4172881c0aad 100644 --- a/datafusion/optimizer/tests/optimizer_integration.rs +++ b/datafusion/optimizer/tests/optimizer_integration.rs @@ -324,11 +324,10 @@ fn push_down_filter_groupby_expr_contains_alias() { fn test_same_name_but_not_ambiguous() { let sql = "SELECT t1.col_int32 AS col_int32 FROM test t1 intersect SELECT col_int32 FROM test t2"; let plan = test_sql(sql).unwrap(); - let expected = "LeftSemi Join: col_int32 = t2.col_int32\ - \n Aggregate: groupBy=[[col_int32]], aggr=[[]]\ - \n Projection: t1.col_int32 AS col_int32\ - \n SubqueryAlias: t1\ - \n TableScan: test projection=[col_int32]\ + let expected = "LeftSemi Join: t1.col_int32 = t2.col_int32\ + \n Aggregate: groupBy=[[t1.col_int32]], aggr=[[]]\ + \n SubqueryAlias: t1\ + \n TableScan: test projection=[col_int32]\ \n SubqueryAlias: t2\ \n TableScan: test projection=[col_int32]"; assert_eq!(expected, format!("{plan:?}")); diff --git a/datafusion/sql/src/select.rs b/datafusion/sql/src/select.rs index c546ca755206..15f720d75652 100644 --- a/datafusion/sql/src/select.rs +++ b/datafusion/sql/src/select.rs @@ -384,7 +384,12 @@ impl<'a, S: ContextProvider> SqlToRel<'a, S> { &[&[plan.schema()]], &plan.using_columns()?, )?; - let expr = col.alias(self.normalizer.normalize(alias)); + let name = self.normalizer.normalize(alias); + // avoiding adding an alias if the column name is the same. + let expr = match &col { + Expr::Column(column) if column.name.eq(&name) => col, + _ => col.alias(name), + }; Ok(vec![expr]) } SelectItem::Wildcard(options) => { diff --git a/datafusion/sql/tests/sql_integration.rs b/datafusion/sql/tests/sql_integration.rs index 944a383ee4b8..48ba50145308 100644 --- a/datafusion/sql/tests/sql_integration.rs +++ b/datafusion/sql/tests/sql_integration.rs @@ -3546,13 +3546,24 @@ fn test_select_unsupported_syntax_errors(#[case] sql: &str, #[case] error: &str) fn select_order_by_with_cast() { let sql = "SELECT first_name AS first_name FROM (SELECT first_name AS first_name FROM person) ORDER BY CAST(first_name as INT)"; - let expected = "Sort: CAST(first_name AS first_name AS Int32) ASC NULLS LAST\ - \n Projection: first_name AS first_name\ - \n Projection: person.first_name AS first_name\ + let expected = "Sort: CAST(person.first_name AS Int32) ASC NULLS LAST\ + \n Projection: person.first_name\ + \n Projection: person.first_name\ \n TableScan: person"; quick_test(sql, expected); } +#[test] +fn test_avoid_add_alias() { + // avoiding adding an alias if the column name is the same. + // plan1 = plan2 + let sql = "select person.id as id from person order by person.id"; + let plan1 = logical_plan(sql).unwrap(); + let sql = "select id from person order by id"; + let plan2 = logical_plan(sql).unwrap(); + assert_eq!(format!("{plan1:?}"), format!("{plan2:?}")); +} + #[test] fn test_duplicated_left_join_key_inner_join() { // person.id * 2 happen twice in left side. diff --git a/datafusion/sqllogictest/test_files/select.slt b/datafusion/sqllogictest/test_files/select.slt index bb81c5a9a138..3f3befd85a59 100644 --- a/datafusion/sqllogictest/test_files/select.slt +++ b/datafusion/sqllogictest/test_files/select.slt @@ -868,6 +868,21 @@ statement error DataFusion error: Error during planning: EXCLUDE or EXCEPT conta SELECT * EXCLUDE(d, b, c, a, a, b, c, d) FROM table1 +# avoiding adding an alias if the column name is the same +query TT +EXPLAIN select a as a FROM table1 order by a +---- +logical_plan +Sort: table1.a ASC NULLS LAST +--TableScan: table1 projection=[a] +physical_plan +SortExec: expr=[a@0 ASC NULLS LAST] +--MemoryExec: partitions=1, partition_sizes=[1] + +# ambiguous column references in on join +query error DataFusion error: Schema error: Ambiguous reference to unqualified field a +EXPLAIN select a as a FROM table1 t1 CROSS JOIN table1 t2 order by a + # run below query in multi partitions statement ok set datafusion.execution.target_partitions = 2; From 4a46f31c853c019e671c49c1713049691e77c252 Mon Sep 17 00:00:00 2001 From: Alex Huang Date: Wed, 6 Dec 2023 22:24:06 +0100 Subject: [PATCH 373/572] Minor: convert marcro `list-slice` and `slice` to function (#8424) * remove marcro list-slice * fix cast dyn Array * remove macro slice --- .../physical-expr/src/array_expressions.rs | 181 +++++++++--------- 1 file changed, 89 insertions(+), 92 deletions(-) diff --git a/datafusion/physical-expr/src/array_expressions.rs b/datafusion/physical-expr/src/array_expressions.rs index 6104566450c3..f254274edde6 100644 --- a/datafusion/physical-expr/src/array_expressions.rs +++ b/datafusion/physical-expr/src/array_expressions.rs @@ -18,6 +18,7 @@ //! Array expressions use std::any::type_name; +use std::cmp::Ordering; use std::collections::HashSet; use std::sync::Arc; @@ -377,111 +378,107 @@ fn return_empty(return_null: bool, data_type: DataType) -> Arc { } } -macro_rules! list_slice { - ($ARRAY:expr, $I:expr, $J:expr, $RETURN_ELEMENT:expr, $ARRAY_TYPE:ident) => {{ - let array = $ARRAY.as_any().downcast_ref::<$ARRAY_TYPE>().unwrap(); - if $I == 0 && $J == 0 || $ARRAY.is_empty() { - return return_empty($RETURN_ELEMENT, $ARRAY.data_type().clone()); - } +fn list_slice( + array: &dyn Array, + i: i64, + j: i64, + return_element: bool, +) -> ArrayRef { + let array = array.as_any().downcast_ref::().unwrap(); - let i = if $I < 0 { - if $I.abs() as usize > array.len() { - return return_empty(true, $ARRAY.data_type().clone()); - } + let array_type = array.data_type().clone(); - (array.len() as i64 + $I + 1) as usize - } else { - if $I == 0 { - 1 - } else { - $I as usize - } - }; - let j = if $J < 0 { - if $J.abs() as usize > array.len() { - return return_empty(true, $ARRAY.data_type().clone()); + if i == 0 && j == 0 || array.is_empty() { + return return_empty(return_element, array_type); + } + + let i = match i.cmp(&0) { + Ordering::Less => { + if i.unsigned_abs() > array.len() as u64 { + return return_empty(true, array_type); } - if $RETURN_ELEMENT { - (array.len() as i64 + $J + 1) as usize - } else { - (array.len() as i64 + $J) as usize + (array.len() as i64 + i + 1) as usize + } + Ordering::Equal => 1, + Ordering::Greater => i as usize, + }; + + let j = match j.cmp(&0) { + Ordering::Less => { + if j.unsigned_abs() as usize > array.len() { + return return_empty(true, array_type); } - } else { - if $J == 0 { - 1 + if return_element { + (array.len() as i64 + j + 1) as usize } else { - if $J as usize > array.len() { - array.len() - } else { - $J as usize - } + (array.len() as i64 + j) as usize } - }; - - if i > j || i as usize > $ARRAY.len() { - return_empty($RETURN_ELEMENT, $ARRAY.data_type().clone()) - } else { - Arc::new(array.slice((i - 1), (j + 1 - i))) } - }}; + Ordering::Equal => 1, + Ordering::Greater => j.min(array.len() as i64) as usize, + }; + + if i > j || i > array.len() { + return_empty(return_element, array_type) + } else { + Arc::new(array.slice(i - 1, j + 1 - i)) + } } -macro_rules! slice { - ($ARRAY:expr, $KEY:expr, $EXTRA_KEY:expr, $RETURN_ELEMENT:expr, $ARRAY_TYPE:ident) => {{ - let sliced_array: Vec> = $ARRAY +fn slice( + array: &ListArray, + key: &Int64Array, + extra_key: &Int64Array, + return_element: bool, +) -> Result> { + let sliced_array: Vec> = array + .iter() + .zip(key.iter()) + .zip(extra_key.iter()) + .map(|((arr, i), j)| match (arr, i, j) { + (Some(arr), Some(i), Some(j)) => list_slice::(&arr, i, j, return_element), + (Some(arr), None, Some(j)) => list_slice::(&arr, 1i64, j, return_element), + (Some(arr), Some(i), None) => { + list_slice::(&arr, i, arr.len() as i64, return_element) + } + (Some(arr), None, None) if !return_element => arr.clone(), + _ => return_empty(return_element, array.value_type()), + }) + .collect(); + + // concat requires input of at least one array + if sliced_array.is_empty() { + Ok(return_empty(return_element, array.value_type())) + } else { + let vec = sliced_array .iter() - .zip($KEY.iter()) - .zip($EXTRA_KEY.iter()) - .map(|((arr, i), j)| match (arr, i, j) { - (Some(arr), Some(i), Some(j)) => { - list_slice!(arr, i, j, $RETURN_ELEMENT, $ARRAY_TYPE) - } - (Some(arr), None, Some(j)) => { - list_slice!(arr, 1i64, j, $RETURN_ELEMENT, $ARRAY_TYPE) - } - (Some(arr), Some(i), None) => { - list_slice!(arr, i, arr.len() as i64, $RETURN_ELEMENT, $ARRAY_TYPE) - } - (Some(arr), None, None) if !$RETURN_ELEMENT => arr, - _ => return_empty($RETURN_ELEMENT, $ARRAY.value_type().clone()), - }) - .collect(); + .map(|a| a.as_ref()) + .collect::>(); + let mut i: i32 = 0; + let mut offsets = vec![i]; + offsets.extend( + vec.iter() + .map(|a| { + i += a.len() as i32; + i + }) + .collect::>(), + ); + let values = compute::concat(vec.as_slice()).unwrap(); - // concat requires input of at least one array - if sliced_array.is_empty() { - Ok(return_empty($RETURN_ELEMENT, $ARRAY.value_type())) + if return_element { + Ok(values) } else { - let vec = sliced_array - .iter() - .map(|a| a.as_ref()) - .collect::>(); - let mut i: i32 = 0; - let mut offsets = vec![i]; - offsets.extend( - vec.iter() - .map(|a| { - i += a.len() as i32; - i - }) - .collect::>(), - ); - let values = compute::concat(vec.as_slice()).unwrap(); - - if $RETURN_ELEMENT { - Ok(values) - } else { - let field = - Arc::new(Field::new("item", $ARRAY.value_type().clone(), true)); - Ok(Arc::new(ListArray::try_new( - field, - OffsetBuffer::new(offsets.into()), - values, - None, - )?)) - } + let field = Arc::new(Field::new("item", array.value_type(), true)); + Ok(Arc::new(ListArray::try_new( + field, + OffsetBuffer::new(offsets.into()), + values, + None, + )?)) } - }}; + } } fn define_array_slice( @@ -492,7 +489,7 @@ fn define_array_slice( ) -> Result { macro_rules! array_function { ($ARRAY_TYPE:ident) => { - slice!(list_array, key, extra_key, return_element, $ARRAY_TYPE) + slice::<$ARRAY_TYPE>(list_array, key, extra_key, return_element) }; } call_array_function!(list_array.value_type(), true) From 107791ae8cf1282865d0f8c017d5ef24c9f1408c Mon Sep 17 00:00:00 2001 From: Jay Zhan Date: Thu, 7 Dec 2023 05:42:18 +0800 Subject: [PATCH 374/572] Remove macro in iter_to_array for List (#8414) * introduce build_array_list_primitive Signed-off-by: jayzhan211 * introduce large list Signed-off-by: jayzhan211 * fix test Signed-off-by: jayzhan211 * add null Signed-off-by: jayzhan211 * clippy Signed-off-by: jayzhan211 --------- Signed-off-by: jayzhan211 --- datafusion/common/src/scalar.rs | 634 +++++++------------------------- 1 file changed, 123 insertions(+), 511 deletions(-) diff --git a/datafusion/common/src/scalar.rs b/datafusion/common/src/scalar.rs index 10f052b90923..1f302c750916 100644 --- a/datafusion/common/src/scalar.rs +++ b/datafusion/common/src/scalar.rs @@ -31,7 +31,6 @@ use crate::cast::{ use crate::error::{DataFusionError, Result, _internal_err, _not_impl_err}; use crate::hash_utils::create_hashes; use crate::utils::{array_into_large_list_array, array_into_list_array}; -use arrow::buffer::{NullBuffer, OffsetBuffer}; use arrow::compute::kernels::numeric::*; use arrow::datatypes::{i256, Fields, SchemaBuilder}; use arrow::util::display::{ArrayFormatter, FormatOptions}; @@ -39,12 +38,11 @@ use arrow::{ array::*, compute::kernels::cast::{cast_with_options, CastOptions}, datatypes::{ - ArrowDictionaryKeyType, ArrowNativeType, DataType, Field, Float32Type, - Float64Type, Int16Type, Int32Type, Int64Type, Int8Type, IntervalDayTimeType, - IntervalMonthDayNanoType, IntervalUnit, IntervalYearMonthType, TimeUnit, - TimestampMicrosecondType, TimestampMillisecondType, TimestampNanosecondType, - TimestampSecondType, UInt16Type, UInt32Type, UInt64Type, UInt8Type, - DECIMAL128_MAX_PRECISION, + ArrowDictionaryKeyType, ArrowNativeType, DataType, Field, Float32Type, Int16Type, + Int32Type, Int64Type, Int8Type, IntervalDayTimeType, IntervalMonthDayNanoType, + IntervalUnit, IntervalYearMonthType, TimeUnit, TimestampMicrosecondType, + TimestampMillisecondType, TimestampNanosecondType, TimestampSecondType, + UInt16Type, UInt32Type, UInt64Type, UInt8Type, DECIMAL128_MAX_PRECISION, }, }; use arrow_array::cast::as_list_array; @@ -1368,103 +1366,36 @@ impl ScalarValue { }}; } - macro_rules! build_array_list_primitive { - ($ARRAY_TY:ident, $SCALAR_TY:ident, $NATIVE_TYPE:ident, $LIST_TY:ident, $SCALAR_LIST:pat) => {{ - Ok::(Arc::new($LIST_TY::from_iter_primitive::<$ARRAY_TY, _, _>( - scalars.into_iter().map(|x| match x{ - ScalarValue::List(arr) if matches!(x, $SCALAR_LIST) => { - // `ScalarValue::List` contains a single element `ListArray`. - let list_arr = as_list_array(&arr); - if list_arr.is_null(0) { - Ok(None) - } else { - let primitive_arr = - list_arr.values().as_primitive::<$ARRAY_TY>(); - Ok(Some( - primitive_arr.into_iter().collect::>>(), - )) - } - } - ScalarValue::LargeList(arr) if matches!(x, $SCALAR_LIST) =>{ - // `ScalarValue::List` contains a single element `ListArray`. - let list_arr = as_large_list_array(&arr); - if list_arr.is_null(0) { - Ok(None) - } else { - let primitive_arr = - list_arr.values().as_primitive::<$ARRAY_TY>(); - Ok(Some( - primitive_arr.into_iter().collect::>>(), - )) - } - } - sv => _internal_err!( - "Inconsistent types in ScalarValue::iter_to_array. \ - Expected {:?}, got {:?}", - data_type, sv - ), - }) - .collect::>>()?, - ))) - }}; - } - - macro_rules! build_array_list_string { - ($BUILDER:ident, $STRING_ARRAY:ident,$LIST_BUILDER:ident,$SCALAR_LIST:pat) => {{ - let mut builder = $LIST_BUILDER::new($BUILDER::new()); - for scalar in scalars.into_iter() { - match scalar { - ScalarValue::List(arr) if matches!(scalar, $SCALAR_LIST) => { - // `ScalarValue::List` contains a single element `ListArray`. - let list_arr = as_list_array(&arr); - - if list_arr.is_null(0) { - builder.append(false); - continue; - } - - let string_arr = $STRING_ARRAY(list_arr.values()); - - for v in string_arr.iter() { - if let Some(v) = v { - builder.values().append_value(v); - } else { - builder.values().append_null(); - } - } - builder.append(true); - } - ScalarValue::LargeList(arr) if matches!(scalar, $SCALAR_LIST) => { - // `ScalarValue::List` contains a single element `ListArray`. - let list_arr = as_large_list_array(&arr); - - if list_arr.is_null(0) { - builder.append(false); - continue; - } - - let string_arr = $STRING_ARRAY(list_arr.values()); - - for v in string_arr.iter() { - if let Some(v) = v { - builder.values().append_value(v); - } else { - builder.values().append_null(); - } - } - builder.append(true); - } - sv => { - return _internal_err!( - "Inconsistent types in ScalarValue::iter_to_array. \ - Expected List, got {:?}", - sv - ) - } - } + fn build_list_array( + scalars: impl IntoIterator, + ) -> Result { + let arrays = scalars + .into_iter() + .map(|s| s.to_array()) + .collect::>>()?; + + let capacity = Capacities::Array(arrays.iter().map(|arr| arr.len()).sum()); + // ScalarValue::List contains a single element ListArray. + let nulls = arrays + .iter() + .map(|arr| arr.is_null(0)) + .collect::>(); + let arrays_data = arrays.iter().map(|arr| arr.to_data()).collect::>(); + + let arrays_ref = arrays_data.iter().collect::>(); + let mut mutable = + MutableArrayData::with_capacities(arrays_ref, true, capacity); + + // ScalarValue::List contains a single element ListArray. + for (index, is_null) in (0..arrays.len()).zip(nulls.into_iter()) { + if is_null { + mutable.extend_nulls(1) + } else { + mutable.extend(index, 0, 1); } - Arc::new(builder.finish()) - }}; + } + let data = mutable.freeze(); + Ok(arrow_array::make_array(data)) } let array: ArrayRef = match &data_type { @@ -1541,228 +1472,7 @@ impl ScalarValue { DataType::Interval(IntervalUnit::MonthDayNano) => { build_array_primitive!(IntervalMonthDayNanoArray, IntervalMonthDayNano) } - DataType::List(fields) if fields.data_type() == &DataType::Int8 => { - build_array_list_primitive!( - Int8Type, - Int8, - i8, - ListArray, - ScalarValue::List(_) - )? - } - DataType::List(fields) if fields.data_type() == &DataType::Int16 => { - build_array_list_primitive!( - Int16Type, - Int16, - i16, - ListArray, - ScalarValue::List(_) - )? - } - DataType::List(fields) if fields.data_type() == &DataType::Int32 => { - build_array_list_primitive!( - Int32Type, - Int32, - i32, - ListArray, - ScalarValue::List(_) - )? - } - DataType::List(fields) if fields.data_type() == &DataType::Int64 => { - build_array_list_primitive!( - Int64Type, - Int64, - i64, - ListArray, - ScalarValue::List(_) - )? - } - DataType::List(fields) if fields.data_type() == &DataType::UInt8 => { - build_array_list_primitive!( - UInt8Type, - UInt8, - u8, - ListArray, - ScalarValue::List(_) - )? - } - DataType::List(fields) if fields.data_type() == &DataType::UInt16 => { - build_array_list_primitive!( - UInt16Type, - UInt16, - u16, - ListArray, - ScalarValue::List(_) - )? - } - DataType::List(fields) if fields.data_type() == &DataType::UInt32 => { - build_array_list_primitive!( - UInt32Type, - UInt32, - u32, - ListArray, - ScalarValue::List(_) - )? - } - DataType::List(fields) if fields.data_type() == &DataType::UInt64 => { - build_array_list_primitive!( - UInt64Type, - UInt64, - u64, - ListArray, - ScalarValue::List(_) - )? - } - DataType::List(fields) if fields.data_type() == &DataType::Float32 => { - build_array_list_primitive!( - Float32Type, - Float32, - f32, - ListArray, - ScalarValue::List(_) - )? - } - DataType::List(fields) if fields.data_type() == &DataType::Float64 => { - build_array_list_primitive!( - Float64Type, - Float64, - f64, - ListArray, - ScalarValue::List(_) - )? - } - DataType::List(fields) if fields.data_type() == &DataType::Utf8 => { - build_array_list_string!( - StringBuilder, - as_string_array, - ListBuilder, - ScalarValue::List(_) - ) - } - DataType::List(fields) if fields.data_type() == &DataType::LargeUtf8 => { - build_array_list_string!( - LargeStringBuilder, - as_largestring_array, - ListBuilder, - ScalarValue::List(_) - ) - } - DataType::List(_) => { - // Fallback case handling homogeneous lists with any ScalarValue element type - let list_array = ScalarValue::iter_to_array_list(scalars)?; - Arc::new(list_array) - } - DataType::LargeList(fields) if fields.data_type() == &DataType::Int8 => { - build_array_list_primitive!( - Int8Type, - Int8, - i8, - LargeListArray, - ScalarValue::LargeList(_) - )? - } - DataType::LargeList(fields) if fields.data_type() == &DataType::Int16 => { - build_array_list_primitive!( - Int16Type, - Int16, - i16, - LargeListArray, - ScalarValue::LargeList(_) - )? - } - DataType::LargeList(fields) if fields.data_type() == &DataType::Int32 => { - build_array_list_primitive!( - Int32Type, - Int32, - i32, - LargeListArray, - ScalarValue::LargeList(_) - )? - } - DataType::LargeList(fields) if fields.data_type() == &DataType::Int64 => { - build_array_list_primitive!( - Int64Type, - Int64, - i64, - LargeListArray, - ScalarValue::LargeList(_) - )? - } - DataType::LargeList(fields) if fields.data_type() == &DataType::UInt8 => { - build_array_list_primitive!( - UInt8Type, - UInt8, - u8, - LargeListArray, - ScalarValue::LargeList(_) - )? - } - DataType::LargeList(fields) if fields.data_type() == &DataType::UInt16 => { - build_array_list_primitive!( - UInt16Type, - UInt16, - u16, - LargeListArray, - ScalarValue::LargeList(_) - )? - } - DataType::LargeList(fields) if fields.data_type() == &DataType::UInt32 => { - build_array_list_primitive!( - UInt32Type, - UInt32, - u32, - LargeListArray, - ScalarValue::LargeList(_) - )? - } - DataType::LargeList(fields) if fields.data_type() == &DataType::UInt64 => { - build_array_list_primitive!( - UInt64Type, - UInt64, - u64, - LargeListArray, - ScalarValue::LargeList(_) - )? - } - DataType::LargeList(fields) if fields.data_type() == &DataType::Float32 => { - build_array_list_primitive!( - Float32Type, - Float32, - f32, - LargeListArray, - ScalarValue::LargeList(_) - )? - } - DataType::LargeList(fields) if fields.data_type() == &DataType::Float64 => { - build_array_list_primitive!( - Float64Type, - Float64, - f64, - LargeListArray, - ScalarValue::LargeList(_) - )? - } - DataType::LargeList(fields) if fields.data_type() == &DataType::Utf8 => { - build_array_list_string!( - StringBuilder, - as_string_array, - LargeListBuilder, - ScalarValue::LargeList(_) - ) - } - DataType::LargeList(fields) if fields.data_type() == &DataType::LargeUtf8 => { - build_array_list_string!( - LargeStringBuilder, - as_largestring_array, - LargeListBuilder, - ScalarValue::LargeList(_) - ) - } - DataType::LargeList(_) => { - // Fallback case handling homogeneous lists with any ScalarValue element type - let list_array = ScalarValue::iter_to_large_array_list(scalars)?; - Arc::new(list_array) - } + DataType::List(_) | DataType::LargeList(_) => build_list_array(scalars)?, DataType::Struct(fields) => { // Initialize a Vector to store the ScalarValues for each column let mut columns: Vec> = @@ -1942,116 +1652,6 @@ impl ScalarValue { Ok(array) } - /// This function build ListArray with nulls with nulls buffer. - fn iter_to_array_list( - scalars: impl IntoIterator, - ) -> Result { - let mut elements: Vec = vec![]; - let mut valid = BooleanBufferBuilder::new(0); - let mut offsets = vec![]; - - for scalar in scalars { - if let ScalarValue::List(arr) = scalar { - // `ScalarValue::List` contains a single element `ListArray`. - let list_arr = as_list_array(&arr); - - if list_arr.is_null(0) { - // Repeat previous offset index - offsets.push(0); - - // Element is null - valid.append(false); - } else { - let arr = list_arr.values().to_owned(); - offsets.push(arr.len()); - elements.push(arr); - - // Element is valid - valid.append(true); - } - } else { - return _internal_err!( - "Expected ScalarValue::List element. Received {scalar:?}" - ); - } - } - - // Concatenate element arrays to create single flat array - let element_arrays: Vec<&dyn Array> = - elements.iter().map(|a| a.as_ref()).collect(); - - let flat_array = match arrow::compute::concat(&element_arrays) { - Ok(flat_array) => flat_array, - Err(err) => return Err(DataFusionError::ArrowError(err)), - }; - - let buffer = valid.finish(); - - let list_array = ListArray::new( - Arc::new(Field::new("item", flat_array.data_type().clone(), true)), - OffsetBuffer::from_lengths(offsets), - flat_array, - Some(NullBuffer::new(buffer)), - ); - - Ok(list_array) - } - - /// This function build LargeListArray with nulls with nulls buffer. - fn iter_to_large_array_list( - scalars: impl IntoIterator, - ) -> Result { - let mut elements: Vec = vec![]; - let mut valid = BooleanBufferBuilder::new(0); - let mut offsets = vec![]; - - for scalar in scalars { - if let ScalarValue::List(arr) = scalar { - // `ScalarValue::List` contains a single element `ListArray`. - let list_arr = as_list_array(&arr); - - if list_arr.is_null(0) { - // Repeat previous offset index - offsets.push(0); - - // Element is null - valid.append(false); - } else { - let arr = list_arr.values().to_owned(); - offsets.push(arr.len()); - elements.push(arr); - - // Element is valid - valid.append(true); - } - } else { - return _internal_err!( - "Expected ScalarValue::List element. Received {scalar:?}" - ); - } - } - - // Concatenate element arrays to create single flat array - let element_arrays: Vec<&dyn Array> = - elements.iter().map(|a| a.as_ref()).collect(); - - let flat_array = match arrow::compute::concat(&element_arrays) { - Ok(flat_array) => flat_array, - Err(err) => return Err(DataFusionError::ArrowError(err)), - }; - - let buffer = valid.finish(); - - let list_array = LargeListArray::new( - Arc::new(Field::new("item", flat_array.data_type().clone(), true)), - OffsetBuffer::from_lengths(offsets), - flat_array, - Some(NullBuffer::new(buffer)), - ); - - Ok(list_array) - } - fn build_decimal_array( value: Option, precision: u8, @@ -3520,21 +3120,23 @@ impl ScalarType for TimestampNanosecondType { #[cfg(test)] mod tests { + use super::*; + use std::cmp::Ordering; use std::sync::Arc; + use chrono::NaiveDate; + use rand::Rng; + + use arrow::buffer::OffsetBuffer; use arrow::compute::kernels; use arrow::compute::{concat, is_null}; use arrow::datatypes::ArrowPrimitiveType; use arrow::util::pretty::pretty_format_columns; use arrow_array::ArrowNumericType; - use chrono::NaiveDate; - use rand::Rng; use crate::cast::{as_string_array, as_uint32_array, as_uint64_array}; - use super::*; - #[test] fn test_to_array_of_size_for_list() { let arr = ListArray::from_iter_primitive::(vec![Some(vec![ @@ -3597,28 +3199,77 @@ mod tests { assert_eq!(result, &expected); } + fn build_list( + values: Vec>>>, + ) -> Vec { + values + .into_iter() + .map(|v| { + let arr = if v.is_some() { + Arc::new( + GenericListArray::::from_iter_primitive::( + vec![v], + ), + ) + } else if O::IS_LARGE { + new_null_array( + &DataType::LargeList(Arc::new(Field::new( + "item", + DataType::Int64, + true, + ))), + 1, + ) + } else { + new_null_array( + &DataType::List(Arc::new(Field::new( + "item", + DataType::Int64, + true, + ))), + 1, + ) + }; + + if O::IS_LARGE { + ScalarValue::LargeList(arr) + } else { + ScalarValue::List(arr) + } + }) + .collect() + } + #[test] fn iter_to_array_primitive_test() { - let scalars = vec![ - ScalarValue::List(Arc::new( - ListArray::from_iter_primitive::(vec![Some(vec![ - Some(1), - Some(2), - Some(3), - ])]), - )), - ScalarValue::List(Arc::new( - ListArray::from_iter_primitive::(vec![Some(vec![ - Some(4), - Some(5), - ])]), - )), - ]; + // List[[1,2,3]], List[null], List[[4,5]] + let scalars = build_list::(vec![ + Some(vec![Some(1), Some(2), Some(3)]), + None, + Some(vec![Some(4), Some(5)]), + ]); let array = ScalarValue::iter_to_array(scalars).unwrap(); let list_array = as_list_array(&array); + // List[[1,2,3], null, [4,5]] let expected = ListArray::from_iter_primitive::(vec![ Some(vec![Some(1), Some(2), Some(3)]), + None, + Some(vec![Some(4), Some(5)]), + ]); + assert_eq!(list_array, &expected); + + let scalars = build_list::(vec![ + Some(vec![Some(1), Some(2), Some(3)]), + None, + Some(vec![Some(4), Some(5)]), + ]); + + let array = ScalarValue::iter_to_array(scalars).unwrap(); + let list_array = as_large_list_array(&array); + let expected = LargeListArray::from_iter_primitive::(vec![ + Some(vec![Some(1), Some(2), Some(3)]), + None, Some(vec![Some(4), Some(5)]), ]); assert_eq!(list_array, &expected); @@ -5083,69 +4734,37 @@ mod tests { assert_eq!(array, &expected); } - #[test] - fn test_nested_lists() { - // Define inner list scalars - let a1 = ListArray::from_iter_primitive::(vec![Some(vec![ - Some(1), - Some(2), - Some(3), - ])]); - let a2 = ListArray::from_iter_primitive::(vec![Some(vec![ - Some(4), - Some(5), - ])]); - let l1 = ListArray::new( - Arc::new(Field::new( - "item", - DataType::List(Arc::new(Field::new("item", DataType::Int32, true))), - true, - )), - OffsetBuffer::::from_lengths([1, 1]), - arrow::compute::concat(&[&a1, &a2]).unwrap(), - None, - ); - - let a1 = - ListArray::from_iter_primitive::(vec![Some(vec![Some(6)])]); - let a2 = ListArray::from_iter_primitive::(vec![Some(vec![ - Some(7), - Some(8), - ])]); - let l2 = ListArray::new( - Arc::new(Field::new( - "item", - DataType::List(Arc::new(Field::new("item", DataType::Int32, true))), - true, - )), - OffsetBuffer::::from_lengths([1, 1]), - arrow::compute::concat(&[&a1, &a2]).unwrap(), - None, - ); - - let a1 = - ListArray::from_iter_primitive::(vec![Some(vec![Some(9)])]); - let l3 = ListArray::new( + fn build_2d_list(data: Vec>) -> ListArray { + let a1 = ListArray::from_iter_primitive::(vec![Some(data)]); + ListArray::new( Arc::new(Field::new( "item", DataType::List(Arc::new(Field::new("item", DataType::Int32, true))), true, )), OffsetBuffer::::from_lengths([1]), - arrow::compute::concat(&[&a1]).unwrap(), + Arc::new(a1), None, - ); + ) + } + + #[test] + fn test_nested_lists() { + // Define inner list scalars + let arr1 = build_2d_list(vec![Some(1), Some(2), Some(3)]); + let arr2 = build_2d_list(vec![Some(4), Some(5)]); + let arr3 = build_2d_list(vec![Some(6)]); let array = ScalarValue::iter_to_array(vec![ - ScalarValue::List(Arc::new(l1)), - ScalarValue::List(Arc::new(l2)), - ScalarValue::List(Arc::new(l3)), + ScalarValue::List(Arc::new(arr1)), + ScalarValue::List(Arc::new(arr2)), + ScalarValue::List(Arc::new(arr3)), ]) .unwrap(); let array = as_list_array(&array); // Construct expected array with array builders - let inner_builder = Int32Array::builder(8); + let inner_builder = Int32Array::builder(6); let middle_builder = ListBuilder::new(inner_builder); let mut outer_builder = ListBuilder::new(middle_builder); @@ -5153,6 +4772,7 @@ mod tests { outer_builder.values().values().append_value(2); outer_builder.values().values().append_value(3); outer_builder.values().append(true); + outer_builder.append(true); outer_builder.values().values().append_value(4); outer_builder.values().values().append_value(5); @@ -5161,14 +4781,6 @@ mod tests { outer_builder.values().values().append_value(6); outer_builder.values().append(true); - - outer_builder.values().values().append_value(7); - outer_builder.values().values().append_value(8); - outer_builder.values().append(true); - outer_builder.append(true); - - outer_builder.values().values().append_value(9); - outer_builder.values().append(true); outer_builder.append(true); let expected = outer_builder.finish(); From 439339a6519f48b672615ce6acac8d48b8be4b8f Mon Sep 17 00:00:00 2001 From: Liang-Chi Hsieh Date: Wed, 6 Dec 2023 13:45:28 -0800 Subject: [PATCH 375/572] fix: Literal in `ORDER BY` window definition should not be an ordinal referring to relation column (#8419) * fix: RANGE frame can be regularized to ROWS frame only if empty ORDER BY clause * Fix flaky test * Update test comment * Add code comment * Update * fix: Literal in window definition should not refer to relation column * Remove unused import * Update datafusion/sql/src/expr/function.rs Co-authored-by: Andrew Lamb * Add code comment * Fix format --------- Co-authored-by: Andrew Lamb --- datafusion/physical-expr/src/sort_expr.rs | 8 ++------ .../src/windows/bounded_window_agg_exec.rs | 4 ++-- datafusion/sql/src/expr/function.rs | 11 ++++++++--- datafusion/sql/src/expr/mod.rs | 7 ++++++- datafusion/sql/src/expr/order_by.rs | 9 +++++++-- datafusion/sql/src/query.rs | 2 +- datafusion/sql/src/statement.rs | 3 ++- datafusion/sqllogictest/test_files/window.slt | 12 ++++++------ 8 files changed, 34 insertions(+), 22 deletions(-) diff --git a/datafusion/physical-expr/src/sort_expr.rs b/datafusion/physical-expr/src/sort_expr.rs index 664a6b65b7f7..914d76f9261a 100644 --- a/datafusion/physical-expr/src/sort_expr.rs +++ b/datafusion/physical-expr/src/sort_expr.rs @@ -26,7 +26,7 @@ use crate::PhysicalExpr; use arrow::compute::kernels::sort::{SortColumn, SortOptions}; use arrow::record_batch::RecordBatch; use arrow_schema::Schema; -use datafusion_common::{exec_err, DataFusionError, Result}; +use datafusion_common::Result; use datafusion_expr::ColumnarValue; /// Represents Sort operation for a column in a RecordBatch @@ -65,11 +65,7 @@ impl PhysicalSortExpr { let value_to_sort = self.expr.evaluate(batch)?; let array_to_sort = match value_to_sort { ColumnarValue::Array(array) => array, - ColumnarValue::Scalar(scalar) => { - return exec_err!( - "Sort operation is not applicable to scalar value {scalar}" - ); - } + ColumnarValue::Scalar(scalar) => scalar.to_array_of_size(batch.num_rows())?, }; Ok(SortColumn { values: array_to_sort, diff --git a/datafusion/physical-plan/src/windows/bounded_window_agg_exec.rs b/datafusion/physical-plan/src/windows/bounded_window_agg_exec.rs index 9e4d6c137067..f988b28cce0d 100644 --- a/datafusion/physical-plan/src/windows/bounded_window_agg_exec.rs +++ b/datafusion/physical-plan/src/windows/bounded_window_agg_exec.rs @@ -51,7 +51,7 @@ use datafusion_common::utils::{ evaluate_partition_ranges, get_arrayref_at_indices, get_at_indices, get_record_batch_at_indices, get_row_at_idx, }; -use datafusion_common::{exec_err, plan_err, DataFusionError, Result}; +use datafusion_common::{exec_err, DataFusionError, Result}; use datafusion_execution::TaskContext; use datafusion_expr::window_state::{PartitionBatchState, WindowAggState}; use datafusion_expr::ColumnarValue; @@ -585,7 +585,7 @@ impl LinearSearch { .map(|item| match item.evaluate(record_batch)? { ColumnarValue::Array(array) => Ok(array), ColumnarValue::Scalar(scalar) => { - plan_err!("Sort operation is not applicable to scalar value {scalar}") + scalar.to_array_of_size(record_batch.num_rows()) } }) .collect() diff --git a/datafusion/sql/src/expr/function.rs b/datafusion/sql/src/expr/function.rs index 958e03879842..14ea20c3fa5f 100644 --- a/datafusion/sql/src/expr/function.rs +++ b/datafusion/sql/src/expr/function.rs @@ -92,8 +92,13 @@ impl<'a, S: ContextProvider> SqlToRel<'a, S> { .into_iter() .map(|e| self.sql_expr_to_logical_expr(e, schema, planner_context)) .collect::>>()?; - let order_by = - self.order_by_to_sort_expr(&window.order_by, schema, planner_context)?; + let order_by = self.order_by_to_sort_expr( + &window.order_by, + schema, + planner_context, + // Numeric literals in window function ORDER BY are treated as constants + false, + )?; let window_frame = window .window_frame .as_ref() @@ -143,7 +148,7 @@ impl<'a, S: ContextProvider> SqlToRel<'a, S> { // next, aggregate built-ins if let Ok(fun) = AggregateFunction::from_str(&name) { let order_by = - self.order_by_to_sort_expr(&order_by, schema, planner_context)?; + self.order_by_to_sort_expr(&order_by, schema, planner_context, true)?; let order_by = (!order_by.is_empty()).then_some(order_by); let args = self.function_args_to_expr(args, schema, planner_context)?; let filter: Option> = filter diff --git a/datafusion/sql/src/expr/mod.rs b/datafusion/sql/src/expr/mod.rs index 2f44466c79c3..27351e10eb34 100644 --- a/datafusion/sql/src/expr/mod.rs +++ b/datafusion/sql/src/expr/mod.rs @@ -555,7 +555,12 @@ impl<'a, S: ContextProvider> SqlToRel<'a, S> { } = array_agg; let order_by = if let Some(order_by) = order_by { - Some(self.order_by_to_sort_expr(&order_by, input_schema, planner_context)?) + Some(self.order_by_to_sort_expr( + &order_by, + input_schema, + planner_context, + true, + )?) } else { None }; diff --git a/datafusion/sql/src/expr/order_by.rs b/datafusion/sql/src/expr/order_by.rs index 1dccc2376f0b..772255bd9773 100644 --- a/datafusion/sql/src/expr/order_by.rs +++ b/datafusion/sql/src/expr/order_by.rs @@ -24,12 +24,17 @@ use datafusion_expr::Expr; use sqlparser::ast::{Expr as SQLExpr, OrderByExpr, Value}; impl<'a, S: ContextProvider> SqlToRel<'a, S> { - /// convert sql [OrderByExpr] to `Vec` + /// Convert sql [OrderByExpr] to `Vec`. + /// + /// If `literal_to_column` is true, treat any numeric literals (e.g. `2`) as a 1 based index + /// into the SELECT list (e.g. `SELECT a, b FROM table ORDER BY 2`). + /// If false, interpret numeric literals as constant values. pub(crate) fn order_by_to_sort_expr( &self, exprs: &[OrderByExpr], schema: &DFSchema, planner_context: &mut PlannerContext, + literal_to_column: bool, ) -> Result> { let mut expr_vec = vec![]; for e in exprs { @@ -40,7 +45,7 @@ impl<'a, S: ContextProvider> SqlToRel<'a, S> { } = e; let expr = match expr { - SQLExpr::Value(Value::Number(v, _)) => { + SQLExpr::Value(Value::Number(v, _)) if literal_to_column => { let field_index = v .parse::() .map_err(|err| plan_datafusion_err!("{}", err))?; diff --git a/datafusion/sql/src/query.rs b/datafusion/sql/src/query.rs index 643f41d84485..dd4cab126261 100644 --- a/datafusion/sql/src/query.rs +++ b/datafusion/sql/src/query.rs @@ -161,7 +161,7 @@ impl<'a, S: ContextProvider> SqlToRel<'a, S> { } let order_by_rex = - self.order_by_to_sort_expr(&order_by, plan.schema(), planner_context)?; + self.order_by_to_sort_expr(&order_by, plan.schema(), planner_context, true)?; if let LogicalPlan::Distinct(Distinct::On(ref distinct_on)) = plan { // In case of `DISTINCT ON` we must capture the sort expressions since during the plan diff --git a/datafusion/sql/src/statement.rs b/datafusion/sql/src/statement.rs index a64010a7c3db..4220e83316d8 100644 --- a/datafusion/sql/src/statement.rs +++ b/datafusion/sql/src/statement.rs @@ -710,7 +710,8 @@ impl<'a, S: ContextProvider> SqlToRel<'a, S> { let mut all_results = vec![]; for expr in order_exprs { // Convert each OrderByExpr to a SortExpr: - let expr_vec = self.order_by_to_sort_expr(&expr, schema, planner_context)?; + let expr_vec = + self.order_by_to_sort_expr(&expr, schema, planner_context, true)?; // Verify that columns of all SortExprs exist in the schema: for expr in expr_vec.iter() { for column in expr.to_columns()?.iter() { diff --git a/datafusion/sqllogictest/test_files/window.slt b/datafusion/sqllogictest/test_files/window.slt index c0dcd4ae1ea5..0179431ac8ad 100644 --- a/datafusion/sqllogictest/test_files/window.slt +++ b/datafusion/sqllogictest/test_files/window.slt @@ -3778,10 +3778,10 @@ query error DataFusion error: Arrow error: Invalid argument error: must either s select rank() over (RANGE between UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING) rnk from (select 1 a union select 2 a) q; -# TODO: this is different to Postgres which returns [1, 1] for `rnk`. -query I -select rank() over (order by 1 RANGE BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING) rnk - from (select 1 a union select 2 a) q ORDER BY rnk +query II +select a, + rank() over (order by 1 RANGE BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING) rnk + from (select 1 a union select 2 a) q ORDER BY a ---- -1 -2 +1 1 +2 1 From fa8a0d9fd609efd24e866a191339075f169f90bd Mon Sep 17 00:00:00 2001 From: Jonah Gao Date: Thu, 7 Dec 2023 05:55:13 +0800 Subject: [PATCH 376/572] feat: customize column default values for external tables (#8415) * feat: customize column default values for external tables * fix test * tests from reviewing --- datafusion/core/src/catalog/listing_schema.rs | 1 + .../core/src/datasource/listing/table.rs | 16 +++++ .../src/datasource/listing_table_factory.rs | 4 +- datafusion/core/src/datasource/memory.rs | 2 +- datafusion/expr/src/logical_plan/ddl.rs | 2 + datafusion/proto/proto/datafusion.proto | 1 + datafusion/proto/src/generated/pbjson.rs | 20 ++++++ datafusion/proto/src/generated/prost.rs | 5 ++ datafusion/proto/src/logical_plan/mod.rs | 17 +++++ .../tests/cases/roundtrip_logical_plan.rs | 11 +-- datafusion/sql/src/statement.rs | 10 ++- datafusion/sqllogictest/test_files/insert.slt | 21 ++++++ .../test_files/insert_to_external.slt | 67 +++++++++++++++++++ 13 files changed, 169 insertions(+), 8 deletions(-) diff --git a/datafusion/core/src/catalog/listing_schema.rs b/datafusion/core/src/catalog/listing_schema.rs index 0d5c49f377d0..c3c682689542 100644 --- a/datafusion/core/src/catalog/listing_schema.rs +++ b/datafusion/core/src/catalog/listing_schema.rs @@ -149,6 +149,7 @@ impl ListingSchemaProvider { unbounded: false, options: Default::default(), constraints: Constraints::empty(), + column_defaults: Default::default(), }, ) .await?; diff --git a/datafusion/core/src/datasource/listing/table.rs b/datafusion/core/src/datasource/listing/table.rs index a3be57db3a83..effeacc4804f 100644 --- a/datafusion/core/src/datasource/listing/table.rs +++ b/datafusion/core/src/datasource/listing/table.rs @@ -17,6 +17,7 @@ //! The table implementation. +use std::collections::HashMap; use std::str::FromStr; use std::{any::Any, sync::Arc}; @@ -558,6 +559,7 @@ pub struct ListingTable { collected_statistics: FileStatisticsCache, infinite_source: bool, constraints: Constraints, + column_defaults: HashMap, } impl ListingTable { @@ -596,6 +598,7 @@ impl ListingTable { collected_statistics: Arc::new(DefaultFileStatisticsCache::default()), infinite_source, constraints: Constraints::empty(), + column_defaults: HashMap::new(), }; Ok(table) @@ -607,6 +610,15 @@ impl ListingTable { self } + /// Assign column defaults + pub fn with_column_defaults( + mut self, + column_defaults: HashMap, + ) -> Self { + self.column_defaults = column_defaults; + self + } + /// Set the [`FileStatisticsCache`] used to cache parquet file statistics. /// /// Setting a statistics cache on the `SessionContext` can avoid refetching statistics @@ -844,6 +856,10 @@ impl TableProvider for ListingTable { .create_writer_physical_plan(input, state, config, order_requirements) .await } + + fn get_column_default(&self, column: &str) -> Option<&Expr> { + self.column_defaults.get(column) + } } impl ListingTable { diff --git a/datafusion/core/src/datasource/listing_table_factory.rs b/datafusion/core/src/datasource/listing_table_factory.rs index f70a82035108..96436306c641 100644 --- a/datafusion/core/src/datasource/listing_table_factory.rs +++ b/datafusion/core/src/datasource/listing_table_factory.rs @@ -228,7 +228,8 @@ impl TableProviderFactory for ListingTableFactory { .with_cache(state.runtime_env().cache_manager.get_file_statistic_cache()); let table = provider .with_definition(cmd.definition.clone()) - .with_constraints(cmd.constraints.clone()); + .with_constraints(cmd.constraints.clone()) + .with_column_defaults(cmd.column_defaults.clone()); Ok(Arc::new(table)) } } @@ -279,6 +280,7 @@ mod tests { unbounded: false, options: HashMap::new(), constraints: Constraints::empty(), + column_defaults: HashMap::new(), }; let table_provider = factory.create(&state, &cmd).await.unwrap(); let listing_table = table_provider diff --git a/datafusion/core/src/datasource/memory.rs b/datafusion/core/src/datasource/memory.rs index a841518d9c8f..7c044b29366d 100644 --- a/datafusion/core/src/datasource/memory.rs +++ b/datafusion/core/src/datasource/memory.rs @@ -19,9 +19,9 @@ use datafusion_physical_plan::metrics::MetricsSet; use futures::StreamExt; -use hashbrown::HashMap; use log::debug; use std::any::Any; +use std::collections::HashMap; use std::fmt::{self, Debug}; use std::sync::Arc; diff --git a/datafusion/expr/src/logical_plan/ddl.rs b/datafusion/expr/src/logical_plan/ddl.rs index 97551a941abf..e74992d99373 100644 --- a/datafusion/expr/src/logical_plan/ddl.rs +++ b/datafusion/expr/src/logical_plan/ddl.rs @@ -194,6 +194,8 @@ pub struct CreateExternalTable { pub options: HashMap, /// The list of constraints in the schema, such as primary key, unique, etc. pub constraints: Constraints, + /// Default values for columns + pub column_defaults: HashMap, } // Hashing refers to a subset of fields considered in PartialEq. diff --git a/datafusion/proto/proto/datafusion.proto b/datafusion/proto/proto/datafusion.proto index e46e70a1396b..64b8e2807476 100644 --- a/datafusion/proto/proto/datafusion.proto +++ b/datafusion/proto/proto/datafusion.proto @@ -216,6 +216,7 @@ message CreateExternalTableNode { bool unbounded = 14; map options = 11; Constraints constraints = 15; + map column_defaults = 16; } message PrepareNode { diff --git a/datafusion/proto/src/generated/pbjson.rs b/datafusion/proto/src/generated/pbjson.rs index a1c177541981..34ad63d819e5 100644 --- a/datafusion/proto/src/generated/pbjson.rs +++ b/datafusion/proto/src/generated/pbjson.rs @@ -4026,6 +4026,9 @@ impl serde::Serialize for CreateExternalTableNode { if self.constraints.is_some() { len += 1; } + if !self.column_defaults.is_empty() { + len += 1; + } let mut struct_ser = serializer.serialize_struct("datafusion.CreateExternalTableNode", len)?; if let Some(v) = self.name.as_ref() { struct_ser.serialize_field("name", v)?; @@ -4069,6 +4072,9 @@ impl serde::Serialize for CreateExternalTableNode { if let Some(v) = self.constraints.as_ref() { struct_ser.serialize_field("constraints", v)?; } + if !self.column_defaults.is_empty() { + struct_ser.serialize_field("columnDefaults", &self.column_defaults)?; + } struct_ser.end() } } @@ -4099,6 +4105,8 @@ impl<'de> serde::Deserialize<'de> for CreateExternalTableNode { "unbounded", "options", "constraints", + "column_defaults", + "columnDefaults", ]; #[allow(clippy::enum_variant_names)] @@ -4117,6 +4125,7 @@ impl<'de> serde::Deserialize<'de> for CreateExternalTableNode { Unbounded, Options, Constraints, + ColumnDefaults, } impl<'de> serde::Deserialize<'de> for GeneratedField { fn deserialize(deserializer: D) -> std::result::Result @@ -4152,6 +4161,7 @@ impl<'de> serde::Deserialize<'de> for CreateExternalTableNode { "unbounded" => Ok(GeneratedField::Unbounded), "options" => Ok(GeneratedField::Options), "constraints" => Ok(GeneratedField::Constraints), + "columnDefaults" | "column_defaults" => Ok(GeneratedField::ColumnDefaults), _ => Err(serde::de::Error::unknown_field(value, FIELDS)), } } @@ -4185,6 +4195,7 @@ impl<'de> serde::Deserialize<'de> for CreateExternalTableNode { let mut unbounded__ = None; let mut options__ = None; let mut constraints__ = None; + let mut column_defaults__ = None; while let Some(k) = map_.next_key()? { match k { GeneratedField::Name => { @@ -4273,6 +4284,14 @@ impl<'de> serde::Deserialize<'de> for CreateExternalTableNode { } constraints__ = map_.next_value()?; } + GeneratedField::ColumnDefaults => { + if column_defaults__.is_some() { + return Err(serde::de::Error::duplicate_field("columnDefaults")); + } + column_defaults__ = Some( + map_.next_value::>()? + ); + } } } Ok(CreateExternalTableNode { @@ -4290,6 +4309,7 @@ impl<'de> serde::Deserialize<'de> for CreateExternalTableNode { unbounded: unbounded__.unwrap_or_default(), options: options__.unwrap_or_default(), constraints: constraints__, + column_defaults: column_defaults__.unwrap_or_default(), }) } } diff --git a/datafusion/proto/src/generated/prost.rs b/datafusion/proto/src/generated/prost.rs index b9fb616b3133..8b4dd1b759d6 100644 --- a/datafusion/proto/src/generated/prost.rs +++ b/datafusion/proto/src/generated/prost.rs @@ -360,6 +360,11 @@ pub struct CreateExternalTableNode { >, #[prost(message, optional, tag = "15")] pub constraints: ::core::option::Option, + #[prost(map = "string, message", tag = "16")] + pub column_defaults: ::std::collections::HashMap< + ::prost::alloc::string::String, + LogicalExprNode, + >, } #[allow(clippy::derive_partial_eq_without_eq)] #[derive(Clone, PartialEq, ::prost::Message)] diff --git a/datafusion/proto/src/logical_plan/mod.rs b/datafusion/proto/src/logical_plan/mod.rs index 851f062bd51f..50bca0295def 100644 --- a/datafusion/proto/src/logical_plan/mod.rs +++ b/datafusion/proto/src/logical_plan/mod.rs @@ -15,6 +15,7 @@ // specific language governing permissions and limitations // under the License. +use std::collections::HashMap; use std::fmt::Debug; use std::str::FromStr; use std::sync::Arc; @@ -521,6 +522,13 @@ impl AsLogicalPlan for LogicalPlanNode { order_exprs.push(order_expr) } + let mut column_defaults = + HashMap::with_capacity(create_extern_table.column_defaults.len()); + for (col_name, expr) in &create_extern_table.column_defaults { + let expr = from_proto::parse_expr(expr, ctx)?; + column_defaults.insert(col_name.clone(), expr); + } + Ok(LogicalPlan::Ddl(DdlStatement::CreateExternalTable(CreateExternalTable { schema: pb_schema.try_into()?, name: from_owned_table_reference(create_extern_table.name.as_ref(), "CreateExternalTable")?, @@ -540,6 +548,7 @@ impl AsLogicalPlan for LogicalPlanNode { unbounded: create_extern_table.unbounded, options: create_extern_table.options.clone(), constraints: constraints.into(), + column_defaults, }))) } LogicalPlanType::CreateView(create_view) => { @@ -1298,6 +1307,7 @@ impl AsLogicalPlan for LogicalPlanNode { unbounded, options, constraints, + column_defaults, }, )) => { let mut converted_order_exprs: Vec = vec![]; @@ -1312,6 +1322,12 @@ impl AsLogicalPlan for LogicalPlanNode { converted_order_exprs.push(temp); } + let mut converted_column_defaults = + HashMap::with_capacity(column_defaults.len()); + for (col_name, expr) in column_defaults { + converted_column_defaults.insert(col_name.clone(), expr.try_into()?); + } + Ok(protobuf::LogicalPlanNode { logical_plan_type: Some(LogicalPlanType::CreateExternalTable( protobuf::CreateExternalTableNode { @@ -1329,6 +1345,7 @@ impl AsLogicalPlan for LogicalPlanNode { unbounded: *unbounded, options: options.clone(), constraints: Some(constraints.clone().into()), + column_defaults: converted_column_defaults, }, )), }) diff --git a/datafusion/proto/tests/cases/roundtrip_logical_plan.rs b/datafusion/proto/tests/cases/roundtrip_logical_plan.rs index e04a7a9c9d03..5e36a838f311 100644 --- a/datafusion/proto/tests/cases/roundtrip_logical_plan.rs +++ b/datafusion/proto/tests/cases/roundtrip_logical_plan.rs @@ -217,11 +217,10 @@ async fn roundtrip_custom_memory_tables() -> Result<()> { async fn roundtrip_custom_listing_tables() -> Result<()> { let ctx = SessionContext::new(); - // Make sure during round-trip, constraint information is preserved let query = "CREATE EXTERNAL TABLE multiple_ordered_table_with_pk ( a0 INTEGER, - a INTEGER, - b INTEGER, + a INTEGER DEFAULT 1*2 + 3, + b INTEGER DEFAULT NULL, c INTEGER, d INTEGER, primary key(c) @@ -232,11 +231,13 @@ async fn roundtrip_custom_listing_tables() -> Result<()> { WITH ORDER (c ASC) LOCATION '../core/tests/data/window_2.csv';"; - let plan = ctx.sql(query).await?.into_optimized_plan()?; + let plan = ctx.state().create_logical_plan(query).await?; let bytes = logical_plan_to_bytes(&plan)?; let logical_round_trip = logical_plan_from_bytes(&bytes, &ctx)?; - assert_eq!(format!("{plan:?}"), format!("{logical_round_trip:?}")); + // Use exact matching to verify everything. Make sure during round-trip, + // information like constraints, column defaults, and other aspects of the plan are preserved. + assert_eq!(plan, logical_round_trip); Ok(()) } diff --git a/datafusion/sql/src/statement.rs b/datafusion/sql/src/statement.rs index 4220e83316d8..12083554f093 100644 --- a/datafusion/sql/src/statement.rs +++ b/datafusion/sql/src/statement.rs @@ -762,11 +762,18 @@ impl<'a, S: ContextProvider> SqlToRel<'a, S> { )?; } + let mut planner_context = PlannerContext::new(); + + let column_defaults = self + .build_column_defaults(&columns, &mut planner_context)? + .into_iter() + .collect(); + let schema = self.build_schema(columns)?; let df_schema = schema.to_dfschema_ref()?; let ordered_exprs = - self.build_order_by(order_exprs, &df_schema, &mut PlannerContext::new())?; + self.build_order_by(order_exprs, &df_schema, &mut planner_context)?; // External tables do not support schemas at the moment, so the name is just a table name let name = OwnedTableReference::bare(name); @@ -788,6 +795,7 @@ impl<'a, S: ContextProvider> SqlToRel<'a, S> { unbounded, options, constraints, + column_defaults, }, ))) } diff --git a/datafusion/sqllogictest/test_files/insert.slt b/datafusion/sqllogictest/test_files/insert.slt index 75252b3b7c35..e20b3779459b 100644 --- a/datafusion/sqllogictest/test_files/insert.slt +++ b/datafusion/sqllogictest/test_files/insert.slt @@ -382,6 +382,27 @@ select a,b,c,d from test_column_defaults 1 10 100 ABC NULL 20 500 default_text +# fill the timestamp column with default value `now()` again, it should be different from the previous one +query IIITP +insert into test_column_defaults(a, b, c, d) values(2, 20, 200, 'DEF') +---- +1 + +# Ensure that the default expression `now()` is evaluated during insertion, not optimized away. +# Rows are inserted during different time, so their timestamp values should be different. +query I rowsort +select count(distinct e) from test_column_defaults +---- +3 + +# Expect all rows to be true as now() was inserted into the table +query B rowsort +select e < now() from test_column_defaults +---- +true +true +true + statement ok drop table test_column_defaults diff --git a/datafusion/sqllogictest/test_files/insert_to_external.slt b/datafusion/sqllogictest/test_files/insert_to_external.slt index 39323479ff74..85c2db7faaf6 100644 --- a/datafusion/sqllogictest/test_files/insert_to_external.slt +++ b/datafusion/sqllogictest/test_files/insert_to_external.slt @@ -543,3 +543,70 @@ select * from table_without_values; statement ok drop table table_without_values; + + +### Test for specifying column's default value + +statement ok +CREATE EXTERNAL TABLE test_column_defaults( + a int, + b int not null default null, + c int default 100*2+300, + d text default lower('DEFAULT_TEXT'), + e timestamp default now() +) STORED AS parquet +LOCATION 'test_files/scratch/insert_to_external/external_parquet_table_q6' +OPTIONS (create_local_path 'true'); + +# fill in all column values +query IIITP +insert into test_column_defaults values(1, 10, 100, 'ABC', now()) +---- +1 + +statement error DataFusion error: Execution error: Invalid batch column at '1' has null but schema specifies non-nullable +insert into test_column_defaults(a) values(2) + +query IIITP +insert into test_column_defaults(b) values(20) +---- +1 + +query IIIT rowsort +select a,b,c,d from test_column_defaults +---- +1 10 100 ABC +NULL 20 500 default_text + +# fill the timestamp column with default value `now()` again, it should be different from the previous one +query IIITP +insert into test_column_defaults(a, b, c, d) values(2, 20, 200, 'DEF') +---- +1 + +# Ensure that the default expression `now()` is evaluated during insertion, not optimized away. +# Rows are inserted during different time, so their timestamp values should be different. +query I rowsort +select count(distinct e) from test_column_defaults +---- +3 + +# Expect all rows to be true as now() was inserted into the table +query B rowsort +select e < now() from test_column_defaults +---- +true +true +true + +statement ok +drop table test_column_defaults + +# test invalid default value +statement error DataFusion error: Error during planning: Column reference is not allowed in the DEFAULT expression : Schema error: No field named a. +CREATE EXTERNAL TABLE test_column_defaults( + a int, + b int default a+1 +) STORED AS parquet +LOCATION 'test_files/scratch/insert_to_external/external_parquet_table_q7' +OPTIONS (create_local_path 'true'); From d9d8ddd5f770817f325190c4c0cc02436e7777e6 Mon Sep 17 00:00:00 2001 From: Asura7969 <1402357969@qq.com> Date: Thu, 7 Dec 2023 06:00:16 +0800 Subject: [PATCH 377/572] feat: Support `array_sort`(`list_sort`) (#8279) * Minor: Improve the document format of JoinHashMap * list sort * fix: example doc * fix: ci * fix: doc error * fix pb * like DuckDB function semantics * fix ci * fix pb * fix: doc * add table test * fix: not as expected * fix: return null * resolve conflicts * doc * merge --- datafusion/expr/src/built_in_function.rs | 8 ++ datafusion/expr/src/expr_fn.rs | 3 + .../physical-expr/src/array_expressions.rs | 83 ++++++++++++++++++- datafusion/physical-expr/src/functions.rs | 3 + datafusion/proto/proto/datafusion.proto | 1 + datafusion/proto/src/generated/pbjson.rs | 3 + datafusion/proto/src/generated/prost.rs | 3 + .../proto/src/logical_plan/from_proto.rs | 15 +++- datafusion/proto/src/logical_plan/to_proto.rs | 1 + datafusion/sqllogictest/test_files/array.slt | 50 +++++++++-- .../source/user-guide/sql/scalar_functions.md | 36 ++++++++ 11 files changed, 194 insertions(+), 12 deletions(-) diff --git a/datafusion/expr/src/built_in_function.rs b/datafusion/expr/src/built_in_function.rs index d48e9e7a67fe..44fbf45525d4 100644 --- a/datafusion/expr/src/built_in_function.rs +++ b/datafusion/expr/src/built_in_function.rs @@ -130,6 +130,8 @@ pub enum BuiltinScalarFunction { // array functions /// array_append ArrayAppend, + /// array_sort + ArraySort, /// array_concat ArrayConcat, /// array_has @@ -398,6 +400,7 @@ impl BuiltinScalarFunction { BuiltinScalarFunction::Tanh => Volatility::Immutable, BuiltinScalarFunction::Trunc => Volatility::Immutable, BuiltinScalarFunction::ArrayAppend => Volatility::Immutable, + BuiltinScalarFunction::ArraySort => Volatility::Immutable, BuiltinScalarFunction::ArrayConcat => Volatility::Immutable, BuiltinScalarFunction::ArrayEmpty => Volatility::Immutable, BuiltinScalarFunction::ArrayHasAll => Volatility::Immutable, @@ -545,6 +548,7 @@ impl BuiltinScalarFunction { Ok(data_type) } BuiltinScalarFunction::ArrayAppend => Ok(input_expr_types[0].clone()), + BuiltinScalarFunction::ArraySort => Ok(input_expr_types[0].clone()), BuiltinScalarFunction::ArrayConcat => { let mut expr_type = Null; let mut max_dims = 0; @@ -909,6 +913,9 @@ impl BuiltinScalarFunction { // for now, the list is small, as we do not have many built-in functions. match self { BuiltinScalarFunction::ArrayAppend => Signature::any(2, self.volatility()), + BuiltinScalarFunction::ArraySort => { + Signature::variadic_any(self.volatility()) + } BuiltinScalarFunction::ArrayPopFront => Signature::any(1, self.volatility()), BuiltinScalarFunction::ArrayPopBack => Signature::any(1, self.volatility()), BuiltinScalarFunction::ArrayConcat => { @@ -1558,6 +1565,7 @@ impl BuiltinScalarFunction { "array_push_back", "list_push_back", ], + BuiltinScalarFunction::ArraySort => &["array_sort", "list_sort"], BuiltinScalarFunction::ArrayConcat => { &["array_concat", "array_cat", "list_concat", "list_cat"] } diff --git a/datafusion/expr/src/expr_fn.rs b/datafusion/expr/src/expr_fn.rs index 6148226f6b1a..8d25619c07d1 100644 --- a/datafusion/expr/src/expr_fn.rs +++ b/datafusion/expr/src/expr_fn.rs @@ -583,6 +583,8 @@ scalar_expr!( "appends an element to the end of an array." ); +scalar_expr!(ArraySort, array_sort, array desc null_first, "returns sorted array."); + scalar_expr!( ArrayPopBack, array_pop_back, @@ -1184,6 +1186,7 @@ mod test { test_scalar_expr!(FromUnixtime, from_unixtime, unixtime); test_scalar_expr!(ArrayAppend, array_append, array, element); + test_scalar_expr!(ArraySort, array_sort, array, desc, null_first); test_scalar_expr!(ArrayPopFront, array_pop_front, array); test_scalar_expr!(ArrayPopBack, array_pop_back, array); test_unary_scalar_expr!(ArrayDims, array_dims); diff --git a/datafusion/physical-expr/src/array_expressions.rs b/datafusion/physical-expr/src/array_expressions.rs index f254274edde6..269bbf7dcf10 100644 --- a/datafusion/physical-expr/src/array_expressions.rs +++ b/datafusion/physical-expr/src/array_expressions.rs @@ -29,7 +29,7 @@ use arrow::datatypes::{DataType, Field, UInt64Type}; use arrow::row::{RowConverter, SortField}; use arrow_buffer::NullBuffer; -use arrow_schema::FieldRef; +use arrow_schema::{FieldRef, SortOptions}; use datafusion_common::cast::{ as_generic_list_array, as_generic_string_array, as_int64_array, as_list_array, as_null_array, as_string_array, @@ -693,7 +693,7 @@ fn general_append_and_prepend( /// # Arguments /// /// * `args` - An array of 1 to 3 ArrayRefs representing start, stop, and step(step value can not be zero.) values. -/// +/// /// # Examples /// /// gen_range(3) => [0, 1, 2] @@ -777,6 +777,85 @@ pub fn array_append(args: &[ArrayRef]) -> Result { Ok(res) } +/// Array_sort SQL function +pub fn array_sort(args: &[ArrayRef]) -> Result { + let sort_option = match args.len() { + 1 => None, + 2 => { + let sort = as_string_array(&args[1])?.value(0); + Some(SortOptions { + descending: order_desc(sort)?, + nulls_first: true, + }) + } + 3 => { + let sort = as_string_array(&args[1])?.value(0); + let nulls_first = as_string_array(&args[2])?.value(0); + Some(SortOptions { + descending: order_desc(sort)?, + nulls_first: order_nulls_first(nulls_first)?, + }) + } + _ => return internal_err!("array_sort expects 1 to 3 arguments"), + }; + + let list_array = as_list_array(&args[0])?; + let row_count = list_array.len(); + + let mut array_lengths = vec![]; + let mut arrays = vec![]; + let mut valid = BooleanBufferBuilder::new(row_count); + for i in 0..row_count { + if list_array.is_null(i) { + array_lengths.push(0); + valid.append(false); + } else { + let arr_ref = list_array.value(i); + let arr_ref = arr_ref.as_ref(); + + let sorted_array = compute::sort(arr_ref, sort_option)?; + array_lengths.push(sorted_array.len()); + arrays.push(sorted_array); + valid.append(true); + } + } + + // Assume all arrays have the same data type + let data_type = list_array.value_type(); + let buffer = valid.finish(); + + let elements = arrays + .iter() + .map(|a| a.as_ref()) + .collect::>(); + + let list_arr = ListArray::new( + Arc::new(Field::new("item", data_type, true)), + OffsetBuffer::from_lengths(array_lengths), + Arc::new(compute::concat(elements.as_slice())?), + Some(NullBuffer::new(buffer)), + ); + Ok(Arc::new(list_arr)) +} + +fn order_desc(modifier: &str) -> Result { + match modifier.to_uppercase().as_str() { + "DESC" => Ok(true), + "ASC" => Ok(false), + _ => internal_err!("the second parameter of array_sort expects DESC or ASC"), + } +} + +fn order_nulls_first(modifier: &str) -> Result { + match modifier.to_uppercase().as_str() { + "NULLS FIRST" => Ok(true), + "NULLS LAST" => Ok(false), + _ => internal_err!( + "the third parameter of array_sort expects NULLS FIRST or NULLS LAST" + ), + } +} + /// Array_prepend SQL function pub fn array_prepend(args: &[ArrayRef]) -> Result { let list_array = as_list_array(&args[1])?; diff --git a/datafusion/physical-expr/src/functions.rs b/datafusion/physical-expr/src/functions.rs index b5d7a8e97dd6..873864a57a6f 100644 --- a/datafusion/physical-expr/src/functions.rs +++ b/datafusion/physical-expr/src/functions.rs @@ -329,6 +329,9 @@ pub fn create_physical_fun( BuiltinScalarFunction::ArrayAppend => { Arc::new(|args| make_scalar_function(array_expressions::array_append)(args)) } + BuiltinScalarFunction::ArraySort => { + Arc::new(|args| make_scalar_function(array_expressions::array_sort)(args)) + } BuiltinScalarFunction::ArrayConcat => { Arc::new(|args| make_scalar_function(array_expressions::array_concat)(args)) } diff --git a/datafusion/proto/proto/datafusion.proto b/datafusion/proto/proto/datafusion.proto index 64b8e2807476..863e3c315c82 100644 --- a/datafusion/proto/proto/datafusion.proto +++ b/datafusion/proto/proto/datafusion.proto @@ -644,6 +644,7 @@ enum ScalarFunction { Levenshtein = 125; SubstrIndex = 126; FindInSet = 127; + ArraySort = 128; } message ScalarFunctionNode { diff --git a/datafusion/proto/src/generated/pbjson.rs b/datafusion/proto/src/generated/pbjson.rs index 34ad63d819e5..74798ee8e94c 100644 --- a/datafusion/proto/src/generated/pbjson.rs +++ b/datafusion/proto/src/generated/pbjson.rs @@ -20905,6 +20905,7 @@ impl serde::Serialize for ScalarFunction { Self::Levenshtein => "Levenshtein", Self::SubstrIndex => "SubstrIndex", Self::FindInSet => "FindInSet", + Self::ArraySort => "ArraySort", }; serializer.serialize_str(variant) } @@ -21044,6 +21045,7 @@ impl<'de> serde::Deserialize<'de> for ScalarFunction { "Levenshtein", "SubstrIndex", "FindInSet", + "ArraySort", ]; struct GeneratedVisitor; @@ -21212,6 +21214,7 @@ impl<'de> serde::Deserialize<'de> for ScalarFunction { "Levenshtein" => Ok(ScalarFunction::Levenshtein), "SubstrIndex" => Ok(ScalarFunction::SubstrIndex), "FindInSet" => Ok(ScalarFunction::FindInSet), + "ArraySort" => Ok(ScalarFunction::ArraySort), _ => Err(serde::de::Error::unknown_variant(value, FIELDS)), } } diff --git a/datafusion/proto/src/generated/prost.rs b/datafusion/proto/src/generated/prost.rs index 8b4dd1b759d6..ae20913e3dd7 100644 --- a/datafusion/proto/src/generated/prost.rs +++ b/datafusion/proto/src/generated/prost.rs @@ -2601,6 +2601,7 @@ pub enum ScalarFunction { Levenshtein = 125, SubstrIndex = 126, FindInSet = 127, + ArraySort = 128, } impl ScalarFunction { /// String value of the enum field names used in the ProtoBuf definition. @@ -2737,6 +2738,7 @@ impl ScalarFunction { ScalarFunction::Levenshtein => "Levenshtein", ScalarFunction::SubstrIndex => "SubstrIndex", ScalarFunction::FindInSet => "FindInSet", + ScalarFunction::ArraySort => "ArraySort", } } /// Creates an enum from field names used in the ProtoBuf definition. @@ -2870,6 +2872,7 @@ impl ScalarFunction { "Levenshtein" => Some(Self::Levenshtein), "SubstrIndex" => Some(Self::SubstrIndex), "FindInSet" => Some(Self::FindInSet), + "ArraySort" => Some(Self::ArraySort), _ => None, } } diff --git a/datafusion/proto/src/logical_plan/from_proto.rs b/datafusion/proto/src/logical_plan/from_proto.rs index ae3628bddeb2..13576aaa089a 100644 --- a/datafusion/proto/src/logical_plan/from_proto.rs +++ b/datafusion/proto/src/logical_plan/from_proto.rs @@ -44,10 +44,11 @@ use datafusion_expr::{ array_except, array_has, array_has_all, array_has_any, array_intersect, array_length, array_ndims, array_position, array_positions, array_prepend, array_remove, array_remove_all, array_remove_n, array_repeat, array_replace, array_replace_all, - array_replace_n, array_slice, array_to_string, arrow_typeof, ascii, asin, asinh, - atan, atan2, atanh, bit_length, btrim, cardinality, cbrt, ceil, character_length, - chr, coalesce, concat_expr, concat_ws_expr, cos, cosh, cot, current_date, - current_time, date_bin, date_part, date_trunc, decode, degrees, digest, encode, exp, + array_replace_n, array_slice, array_sort, array_to_string, arrow_typeof, ascii, asin, + asinh, atan, atan2, atanh, bit_length, btrim, cardinality, cbrt, ceil, + character_length, chr, coalesce, concat_expr, concat_ws_expr, cos, cosh, cot, + current_date, current_time, date_bin, date_part, date_trunc, decode, degrees, digest, + encode, exp, expr::{self, InList, Sort, WindowFunction}, factorial, find_in_set, flatten, floor, from_unixtime, gcd, gen_range, isnan, iszero, lcm, left, levenshtein, ln, log, log10, log2, @@ -463,6 +464,7 @@ impl From<&protobuf::ScalarFunction> for BuiltinScalarFunction { ScalarFunction::Rtrim => Self::Rtrim, ScalarFunction::ToTimestamp => Self::ToTimestamp, ScalarFunction::ArrayAppend => Self::ArrayAppend, + ScalarFunction::ArraySort => Self::ArraySort, ScalarFunction::ArrayConcat => Self::ArrayConcat, ScalarFunction::ArrayEmpty => Self::ArrayEmpty, ScalarFunction::ArrayExcept => Self::ArrayExcept, @@ -1343,6 +1345,11 @@ pub fn parse_expr( parse_expr(&args[0], registry)?, parse_expr(&args[1], registry)?, )), + ScalarFunction::ArraySort => Ok(array_sort( + parse_expr(&args[0], registry)?, + parse_expr(&args[1], registry)?, + parse_expr(&args[2], registry)?, + )), ScalarFunction::ArrayPopFront => { Ok(array_pop_front(parse_expr(&args[0], registry)?)) } diff --git a/datafusion/proto/src/logical_plan/to_proto.rs b/datafusion/proto/src/logical_plan/to_proto.rs index ecbfaca5dbfe..0af8d9f3e719 100644 --- a/datafusion/proto/src/logical_plan/to_proto.rs +++ b/datafusion/proto/src/logical_plan/to_proto.rs @@ -1502,6 +1502,7 @@ impl TryFrom<&BuiltinScalarFunction> for protobuf::ScalarFunction { BuiltinScalarFunction::Rtrim => Self::Rtrim, BuiltinScalarFunction::ToTimestamp => Self::ToTimestamp, BuiltinScalarFunction::ArrayAppend => Self::ArrayAppend, + BuiltinScalarFunction::ArraySort => Self::ArraySort, BuiltinScalarFunction::ArrayConcat => Self::ArrayConcat, BuiltinScalarFunction::ArrayEmpty => Self::ArrayEmpty, BuiltinScalarFunction::ArrayExcept => Self::ArrayExcept, diff --git a/datafusion/sqllogictest/test_files/array.slt b/datafusion/sqllogictest/test_files/array.slt index d8bf441d7169..3c23dd369ae5 100644 --- a/datafusion/sqllogictest/test_files/array.slt +++ b/datafusion/sqllogictest/test_files/array.slt @@ -1052,6 +1052,44 @@ select make_array(['a','b'], null); ---- [[a, b], ] +## array_sort (aliases: `list_sort`) +query ??? +select array_sort(make_array(1, 3, null, 5, NULL, -5)), array_sort(make_array(1, 3, null, 2), 'ASC'), array_sort(make_array(1, 3, null, 2), 'desc', 'NULLS FIRST'); +---- +[, , -5, 1, 3, 5] [, 1, 2, 3] [, 3, 2, 1] + +query ? +select array_sort(column1, 'DESC', 'NULLS LAST') from arrays_values; +---- +[10, 9, 8, 7, 6, 5, 4, 3, 2, ] +[20, 18, 17, 16, 15, 14, 13, 12, 11, ] +[30, 29, 28, 27, 26, 25, 23, 22, 21, ] +[40, 39, 38, 37, 35, 34, 33, 32, 31, ] +NULL +[50, 49, 48, 47, 46, 45, 44, 43, 42, 41] +[60, 59, 58, 57, 56, 55, 54, 52, 51, ] +[70, 69, 68, 67, 66, 65, 64, 63, 62, 61] + +query ? +select array_sort(column1, 'ASC', 'NULLS FIRST') from arrays_values; +---- +[, 2, 3, 4, 5, 6, 7, 8, 9, 10] +[, 11, 12, 13, 14, 15, 16, 17, 18, 20] +[, 21, 22, 23, 25, 26, 27, 28, 29, 30] +[, 31, 32, 33, 34, 35, 37, 38, 39, 40] +NULL +[41, 42, 43, 44, 45, 46, 47, 48, 49, 50] +[, 51, 52, 54, 55, 56, 57, 58, 59, 60] +[61, 62, 63, 64, 65, 66, 67, 68, 69, 70] + + +## list_sort (aliases: `array_sort`) +query ??? +select list_sort(make_array(1, 3, null, 5, NULL, -5)), list_sort(make_array(1, 3, null, 2), 'ASC'), list_sort(make_array(1, 3, null, 2), 'desc', 'NULLS FIRST'); +---- +[, , -5, 1, 3, 5] [, 1, 2, 3] [, 3, 2, 1] + + ## array_append (aliases: `list_append`, `array_push_back`, `list_push_back`) # TODO: array_append with NULLs @@ -1224,7 +1262,7 @@ select array_prepend(make_array(1, 11, 111), column1), array_prepend(column2, ma # array_repeat scalar function #1 query ???????? -select +select array_repeat(1, 5), array_repeat(3.14, 3), array_repeat('l', 4), @@ -1257,7 +1295,7 @@ AS VALUES (0, 3, 3.3, 'datafusion', make_array(8, 9)); query ?????? -select +select array_repeat(column2, column1), array_repeat(column3, column1), array_repeat(column4, column1), @@ -1272,7 +1310,7 @@ from array_repeat_table; [] [] [] [] [3, 3, 3] [] statement ok -drop table array_repeat_table; +drop table array_repeat_table; ## array_concat (aliases: `array_cat`, `list_concat`, `list_cat`) @@ -2188,7 +2226,7 @@ select array_remove(make_array(1, 2, 2, 1, 1), 2), array_remove(make_array(1.0, [1, 2, 1, 1] [2.0, 2.0, 1.0, 1.0] [h, e, l, o] query ??? -select +select array_remove(make_array(1, null, 2, 3), 2), array_remove(make_array(1.1, null, 2.2, 3.3), 1.1), array_remove(make_array('a', null, 'bc'), 'a'); @@ -2887,7 +2925,7 @@ from array_intersect_table_3D; query ?????? SELECT array_intersect(make_array(1,2,3), make_array(2,3,4)), array_intersect(make_array(1,3,5), make_array(2,4,6)), - array_intersect(make_array('aa','bb','cc'), make_array('cc','aa','dd')), + array_intersect(make_array('aa','bb','cc'), make_array('cc','aa','dd')), array_intersect(make_array(true, false), make_array(true)), array_intersect(make_array(1.1, 2.2, 3.3), make_array(2.2, 3.3, 4.4)), array_intersect(make_array([1, 1], [2, 2], [3, 3]), make_array([2, 2], [3, 3], [4, 4])) @@ -2918,7 +2956,7 @@ NULL query ?????? SELECT list_intersect(make_array(1,2,3), make_array(2,3,4)), list_intersect(make_array(1,3,5), make_array(2,4,6)), - list_intersect(make_array('aa','bb','cc'), make_array('cc','aa','dd')), + list_intersect(make_array('aa','bb','cc'), make_array('cc','aa','dd')), list_intersect(make_array(true, false), make_array(true)), list_intersect(make_array(1.1, 2.2, 3.3), make_array(2.2, 3.3, 4.4)), list_intersect(make_array([1, 1], [2, 2], [3, 3]), make_array([2, 2], [3, 3], [4, 4])) diff --git a/docs/source/user-guide/sql/scalar_functions.md b/docs/source/user-guide/sql/scalar_functions.md index 46920f1c4d0b..9a9bec9df77b 100644 --- a/docs/source/user-guide/sql/scalar_functions.md +++ b/docs/source/user-guide/sql/scalar_functions.md @@ -1555,6 +1555,7 @@ from_unixtime(expression) ## Array Functions - [array_append](#array_append) +- [array_sort](#array_sort) - [array_cat](#array_cat) - [array_concat](#array_concat) - [array_contains](#array_contains) @@ -1584,6 +1585,7 @@ from_unixtime(expression) - [cardinality](#cardinality) - [empty](#empty) - [list_append](#list_append) +- [list_sort](#list_sort) - [list_cat](#list_cat) - [list_concat](#list_concat) - [list_dims](#list_dims) @@ -1645,6 +1647,36 @@ array_append(array, element) - list_append - list_push_back +### `array_sort` + +Sort array. + +``` +array_sort(array, desc, nulls_first) +``` + +#### Arguments + +- **array**: Array expression. + Can be a constant, column, or function, and any combination of array operators. +- **desc**: Whether to sort in descending order(`ASC` or `DESC`). +- **nulls_first**: Whether to sort nulls first(`NULLS FIRST` or `NULLS LAST`). + +#### Example + +``` +❯ select array_sort([3, 1, 2]); ++-----------------------------+ +| array_sort(List([3,1,2])) | ++-----------------------------+ +| [1, 2, 3] | ++-----------------------------+ +``` + +#### Aliases + +- list_sort + ### `array_cat` _Alias of [array_concat](#array_concat)._ @@ -2433,6 +2465,10 @@ empty(array) _Alias of [array_append](#array_append)._ +### `list_sort` + +_Alias of [array_sort](#array_sort)._ + ### `list_cat` _Alias of [array_concat](#array_concat)._ From 99bf509bc5ef7e49c32ab19e261ab662276c8968 Mon Sep 17 00:00:00 2001 From: Devin D'Angelo Date: Wed, 6 Dec 2023 17:29:45 -0500 Subject: [PATCH 378/572] Bugfix: Remove df-cli specific SQL statment options before executing with DataFusion (#8426) * remove df-cli specific options from create external table options * add test and comments * cargo fmt * merge main * cargo toml format --- datafusion-cli/Cargo.lock | 19 +++++++------ datafusion-cli/Cargo.toml | 1 + datafusion-cli/src/exec.rs | 31 +++++++++++++++++---- datafusion-cli/src/object_storage.rs | 41 +++++++++++++++------------- 4 files changed, 58 insertions(+), 34 deletions(-) diff --git a/datafusion-cli/Cargo.lock b/datafusion-cli/Cargo.lock index 474d85ac4603..f88c907b052f 100644 --- a/datafusion-cli/Cargo.lock +++ b/datafusion-cli/Cargo.lock @@ -1155,6 +1155,7 @@ dependencies = [ "clap", "ctor", "datafusion", + "datafusion-common", "dirs", "env_logger", "mimalloc", @@ -2157,9 +2158,9 @@ dependencies = [ [[package]] name = "mio" -version = "0.8.9" +version = "0.8.10" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "3dce281c5e46beae905d4de1870d8b1509a9142b62eedf18b443b011ca8343d0" +checksum = "8f3d0b296e374a4e6f3c7b0a1f5a51d748a0d34c85e7dc48fc3fa9a87657fe09" dependencies = [ "libc", "wasi", @@ -2307,7 +2308,7 @@ dependencies = [ "quick-xml", "rand", "reqwest", - "ring 0.17.6", + "ring 0.17.7", "rustls-pemfile", "serde", "serde_json", @@ -2770,9 +2771,9 @@ dependencies = [ [[package]] name = "ring" -version = "0.17.6" +version = "0.17.7" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "684d5e6e18f669ccebf64a92236bb7db9a34f07be010e3627368182027180866" +checksum = "688c63d65483050968b2a8937f7995f443e27041a0f7700aa59b0822aedebb74" dependencies = [ "cc", "getrandom", @@ -2861,7 +2862,7 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "629648aced5775d558af50b2b4c7b02983a04b312126d45eeead26e7caa498b9" dependencies = [ "log", - "ring 0.17.6", + "ring 0.17.7", "rustls-webpki", "sct", ] @@ -2893,7 +2894,7 @@ version = "0.101.7" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "8b6275d1ee7a1cd780b64aca7726599a1dbc893b1e64144529e55c3c2f745765" dependencies = [ - "ring 0.17.6", + "ring 0.17.7", "untrusted 0.9.0", ] @@ -2962,7 +2963,7 @@ version = "0.7.1" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "da046153aa2352493d6cb7da4b6e5c0c057d8a1d0a9aa8560baffdd945acd414" dependencies = [ - "ring 0.17.6", + "ring 0.17.7", "untrusted 0.9.0", ] @@ -3759,7 +3760,7 @@ version = "0.22.4" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "ed63aea5ce73d0ff405984102c42de94fc55a6b75765d621c65262469b3c9b53" dependencies = [ - "ring 0.17.6", + "ring 0.17.7", "untrusted 0.9.0", ] diff --git a/datafusion-cli/Cargo.toml b/datafusion-cli/Cargo.toml index dd7a077988cb..fd2dfd76c20e 100644 --- a/datafusion-cli/Cargo.toml +++ b/datafusion-cli/Cargo.toml @@ -48,5 +48,6 @@ url = "2.2" [dev-dependencies] assert_cmd = "2.0" ctor = "0.2.0" +datafusion-common = { path = "../datafusion/common" } predicates = "3.0" rstest = "0.17" diff --git a/datafusion-cli/src/exec.rs b/datafusion-cli/src/exec.rs index 63862caab82a..8af534cd1375 100644 --- a/datafusion-cli/src/exec.rs +++ b/datafusion-cli/src/exec.rs @@ -211,7 +211,7 @@ async fn exec_and_print( })?; let statements = DFParser::parse_sql_with_dialect(&sql, dialect.as_ref())?; for statement in statements { - let plan = ctx.state().statement_to_plan(statement).await?; + let mut plan = ctx.state().statement_to_plan(statement).await?; // For plans like `Explain` ignore `MaxRows` option and always display all rows let should_ignore_maxrows = matches!( @@ -221,10 +221,12 @@ async fn exec_and_print( | LogicalPlan::Analyze(_) ); - if let LogicalPlan::Ddl(DdlStatement::CreateExternalTable(cmd)) = &plan { + // Note that cmd is a mutable reference so that create_external_table function can remove all + // datafusion-cli specific options before passing through to datafusion. Otherwise, datafusion + // will raise Configuration errors. + if let LogicalPlan::Ddl(DdlStatement::CreateExternalTable(cmd)) = &mut plan { create_external_table(ctx, cmd).await?; } - let df = ctx.execute_logical_plan(plan).await?; let results = df.collect().await?; @@ -244,7 +246,7 @@ async fn exec_and_print( async fn create_external_table( ctx: &SessionContext, - cmd: &CreateExternalTable, + cmd: &mut CreateExternalTable, ) -> Result<()> { let table_path = ListingTableUrl::parse(&cmd.location)?; let scheme = table_path.scheme(); @@ -285,15 +287,32 @@ async fn create_external_table( #[cfg(test)] mod tests { + use std::str::FromStr; + use super::*; use datafusion::common::plan_err; + use datafusion_common::{file_options::StatementOptions, FileTypeWriterOptions}; async fn create_external_table_test(location: &str, sql: &str) -> Result<()> { let ctx = SessionContext::new(); - let plan = ctx.state().create_logical_plan(sql).await?; + let mut plan = ctx.state().create_logical_plan(sql).await?; - if let LogicalPlan::Ddl(DdlStatement::CreateExternalTable(cmd)) = &plan { + if let LogicalPlan::Ddl(DdlStatement::CreateExternalTable(cmd)) = &mut plan { create_external_table(&ctx, cmd).await?; + let options: Vec<_> = cmd + .options + .iter() + .map(|(k, v)| (k.clone(), v.clone())) + .collect(); + let statement_options = StatementOptions::new(options); + let file_type = + datafusion_common::FileType::from_str(cmd.file_type.as_str())?; + + let _file_type_writer_options = FileTypeWriterOptions::build( + &file_type, + ctx.state().config_options(), + &statement_options, + )?; } else { return plan_err!("LogicalPlan is not a CreateExternalTable"); } diff --git a/datafusion-cli/src/object_storage.rs b/datafusion-cli/src/object_storage.rs index c39d1915eb43..9d79c7e0ec78 100644 --- a/datafusion-cli/src/object_storage.rs +++ b/datafusion-cli/src/object_storage.rs @@ -30,20 +30,23 @@ use url::Url; pub async fn get_s3_object_store_builder( url: &Url, - cmd: &CreateExternalTable, + cmd: &mut CreateExternalTable, ) -> Result { let bucket_name = get_bucket_name(url)?; let mut builder = AmazonS3Builder::from_env().with_bucket_name(bucket_name); if let (Some(access_key_id), Some(secret_access_key)) = ( - cmd.options.get("access_key_id"), - cmd.options.get("secret_access_key"), + // These options are datafusion-cli specific and must be removed before passing through to datafusion. + // Otherwise, a Configuration error will be raised. + cmd.options.remove("access_key_id"), + cmd.options.remove("secret_access_key"), ) { + println!("removing secret access key!"); builder = builder .with_access_key_id(access_key_id) .with_secret_access_key(secret_access_key); - if let Some(session_token) = cmd.options.get("session_token") { + if let Some(session_token) = cmd.options.remove("session_token") { builder = builder.with_token(session_token); } } else { @@ -66,7 +69,7 @@ pub async fn get_s3_object_store_builder( builder = builder.with_credentials(credentials); } - if let Some(region) = cmd.options.get("region") { + if let Some(region) = cmd.options.remove("region") { builder = builder.with_region(region); } @@ -99,7 +102,7 @@ impl CredentialProvider for S3CredentialProvider { pub fn get_oss_object_store_builder( url: &Url, - cmd: &CreateExternalTable, + cmd: &mut CreateExternalTable, ) -> Result { let bucket_name = get_bucket_name(url)?; let mut builder = AmazonS3Builder::from_env() @@ -109,15 +112,15 @@ pub fn get_oss_object_store_builder( .with_region("do_not_care"); if let (Some(access_key_id), Some(secret_access_key)) = ( - cmd.options.get("access_key_id"), - cmd.options.get("secret_access_key"), + cmd.options.remove("access_key_id"), + cmd.options.remove("secret_access_key"), ) { builder = builder .with_access_key_id(access_key_id) .with_secret_access_key(secret_access_key); } - if let Some(endpoint) = cmd.options.get("endpoint") { + if let Some(endpoint) = cmd.options.remove("endpoint") { builder = builder.with_endpoint(endpoint); } @@ -126,21 +129,21 @@ pub fn get_oss_object_store_builder( pub fn get_gcs_object_store_builder( url: &Url, - cmd: &CreateExternalTable, + cmd: &mut CreateExternalTable, ) -> Result { let bucket_name = get_bucket_name(url)?; let mut builder = GoogleCloudStorageBuilder::from_env().with_bucket_name(bucket_name); - if let Some(service_account_path) = cmd.options.get("service_account_path") { + if let Some(service_account_path) = cmd.options.remove("service_account_path") { builder = builder.with_service_account_path(service_account_path); } - if let Some(service_account_key) = cmd.options.get("service_account_key") { + if let Some(service_account_key) = cmd.options.remove("service_account_key") { builder = builder.with_service_account_key(service_account_key); } if let Some(application_credentials_path) = - cmd.options.get("application_credentials_path") + cmd.options.remove("application_credentials_path") { builder = builder.with_application_credentials(application_credentials_path); } @@ -180,9 +183,9 @@ mod tests { let sql = format!("CREATE EXTERNAL TABLE test STORED AS PARQUET OPTIONS('access_key_id' '{access_key_id}', 'secret_access_key' '{secret_access_key}', 'region' '{region}', 'session_token' {session_token}) LOCATION '{location}'"); let ctx = SessionContext::new(); - let plan = ctx.state().create_logical_plan(&sql).await?; + let mut plan = ctx.state().create_logical_plan(&sql).await?; - if let LogicalPlan::Ddl(DdlStatement::CreateExternalTable(cmd)) = &plan { + if let LogicalPlan::Ddl(DdlStatement::CreateExternalTable(cmd)) = &mut plan { let builder = get_s3_object_store_builder(table_url.as_ref(), cmd).await?; // get the actual configuration information, then assert_eq! let config = [ @@ -212,9 +215,9 @@ mod tests { let sql = format!("CREATE EXTERNAL TABLE test STORED AS PARQUET OPTIONS('access_key_id' '{access_key_id}', 'secret_access_key' '{secret_access_key}', 'endpoint' '{endpoint}') LOCATION '{location}'"); let ctx = SessionContext::new(); - let plan = ctx.state().create_logical_plan(&sql).await?; + let mut plan = ctx.state().create_logical_plan(&sql).await?; - if let LogicalPlan::Ddl(DdlStatement::CreateExternalTable(cmd)) = &plan { + if let LogicalPlan::Ddl(DdlStatement::CreateExternalTable(cmd)) = &mut plan { let builder = get_oss_object_store_builder(table_url.as_ref(), cmd)?; // get the actual configuration information, then assert_eq! let config = [ @@ -244,9 +247,9 @@ mod tests { let sql = format!("CREATE EXTERNAL TABLE test STORED AS PARQUET OPTIONS('service_account_path' '{service_account_path}', 'service_account_key' '{service_account_key}', 'application_credentials_path' '{application_credentials_path}') LOCATION '{location}'"); let ctx = SessionContext::new(); - let plan = ctx.state().create_logical_plan(&sql).await?; + let mut plan = ctx.state().create_logical_plan(&sql).await?; - if let LogicalPlan::Ddl(DdlStatement::CreateExternalTable(cmd)) = &plan { + if let LogicalPlan::Ddl(DdlStatement::CreateExternalTable(cmd)) = &mut plan { let builder = get_gcs_object_store_builder(table_url.as_ref(), cmd)?; // get the actual configuration information, then assert_eq! let config = [ From c8e1c84e6b4f1292afa6f5517bc6978b55758723 Mon Sep 17 00:00:00 2001 From: Jesse Date: Wed, 6 Dec 2023 23:33:53 +0100 Subject: [PATCH 379/572] Detect when filters make subqueries scalar (#8312) Co-authored-by: Andrew Lamb --- .../common/src/functional_dependencies.rs | 8 + datafusion/expr/src/logical_plan/plan.rs | 141 +++++++++++++++++- .../sqllogictest/test_files/subquery.slt | 18 +++ 3 files changed, 164 insertions(+), 3 deletions(-) diff --git a/datafusion/common/src/functional_dependencies.rs b/datafusion/common/src/functional_dependencies.rs index fbddcddab4bc..4587677e7726 100644 --- a/datafusion/common/src/functional_dependencies.rs +++ b/datafusion/common/src/functional_dependencies.rs @@ -413,6 +413,14 @@ impl FunctionalDependencies { } } +impl Deref for FunctionalDependencies { + type Target = [FunctionalDependence]; + + fn deref(&self) -> &Self::Target { + self.deps.as_slice() + } +} + /// Calculates functional dependencies for aggregate output, when there is a GROUP BY expression. pub fn aggregate_functional_dependencies( aggr_input_schema: &DFSchema, diff --git a/datafusion/expr/src/logical_plan/plan.rs b/datafusion/expr/src/logical_plan/plan.rs index 2988e7536bce..d85e0b5b0a40 100644 --- a/datafusion/expr/src/logical_plan/plan.rs +++ b/datafusion/expr/src/logical_plan/plan.rs @@ -33,6 +33,7 @@ use crate::logical_plan::{DmlStatement, Statement}; use crate::utils::{ enumerate_grouping_sets, exprlist_to_fields, find_out_reference_exprs, grouping_set_expr_count, grouping_set_to_exprlist, inspect_expr_pre, + split_conjunction, }; use crate::{ build_join_schema, expr_vec_fmt, BinaryExpr, CreateMemoryTable, CreateView, Expr, @@ -47,7 +48,7 @@ use datafusion_common::tree_node::{ }; use datafusion_common::{ aggregate_functional_dependencies, internal_err, plan_err, Column, Constraints, - DFField, DFSchema, DFSchemaRef, DataFusionError, FunctionalDependencies, + DFField, DFSchema, DFSchemaRef, DataFusionError, Dependency, FunctionalDependencies, OwnedTableReference, ParamValues, Result, UnnestOptions, }; // backwards compatibility @@ -1032,7 +1033,13 @@ impl LogicalPlan { pub fn max_rows(self: &LogicalPlan) -> Option { match self { LogicalPlan::Projection(Projection { input, .. }) => input.max_rows(), - LogicalPlan::Filter(Filter { input, .. }) => input.max_rows(), + LogicalPlan::Filter(filter) => { + if filter.is_scalar() { + Some(1) + } else { + filter.input.max_rows() + } + } LogicalPlan::Window(Window { input, .. }) => input.max_rows(), LogicalPlan::Aggregate(Aggregate { input, group_expr, .. @@ -1913,6 +1920,73 @@ impl Filter { Ok(Self { predicate, input }) } + + /// Is this filter guaranteed to return 0 or 1 row in a given instantiation? + /// + /// This function will return `true` if its predicate contains a conjunction of + /// `col(a) = `, where its schema has a unique filter that is covered + /// by this conjunction. + /// + /// For example, for the table: + /// ```sql + /// CREATE TABLE t (a INTEGER PRIMARY KEY, b INTEGER); + /// ``` + /// `Filter(a = 2).is_scalar() == true` + /// , whereas + /// `Filter(b = 2).is_scalar() == false` + /// and + /// `Filter(a = 2 OR b = 2).is_scalar() == false` + fn is_scalar(&self) -> bool { + let schema = self.input.schema(); + + let functional_dependencies = self.input.schema().functional_dependencies(); + let unique_keys = functional_dependencies.iter().filter(|dep| { + let nullable = dep.nullable + && dep + .source_indices + .iter() + .any(|&source| schema.field(source).is_nullable()); + !nullable + && dep.mode == Dependency::Single + && dep.target_indices.len() == schema.fields().len() + }); + + let exprs = split_conjunction(&self.predicate); + let eq_pred_cols: HashSet<_> = exprs + .iter() + .filter_map(|expr| { + let Expr::BinaryExpr(BinaryExpr { + left, + op: Operator::Eq, + right, + }) = expr + else { + return None; + }; + // This is a no-op filter expression + if left == right { + return None; + } + + match (left.as_ref(), right.as_ref()) { + (Expr::Column(_), Expr::Column(_)) => None, + (Expr::Column(c), _) | (_, Expr::Column(c)) => { + Some(schema.index_of_column(c).unwrap()) + } + _ => None, + } + }) + .collect(); + + // If we have a functional dependence that is a subset of our predicate, + // this filter is scalar + for key in unique_keys { + if key.source_indices.iter().all(|c| eq_pred_cols.contains(c)) { + return true; + } + } + false + } } /// Window its input based on a set of window spec and window function (e.g. SUM or RANK) @@ -2554,12 +2628,16 @@ pub struct Unnest { #[cfg(test)] mod tests { use super::*; + use crate::builder::LogicalTableSource; use crate::logical_plan::table_scan; use crate::{col, count, exists, in_subquery, lit, placeholder, GroupingSet}; use arrow::datatypes::{DataType, Field, Schema}; use datafusion_common::tree_node::TreeNodeVisitor; - use datafusion_common::{not_impl_err, DFSchema, ScalarValue, TableReference}; + use datafusion_common::{ + not_impl_err, Constraint, DFSchema, ScalarValue, TableReference, + }; use std::collections::HashMap; + use std::sync::Arc; fn employee_schema() -> Schema { Schema::new(vec![ @@ -3056,6 +3134,63 @@ digraph { .is_nullable()); } + #[test] + fn test_filter_is_scalar() { + // test empty placeholder + let schema = + Arc::new(Schema::new(vec![Field::new("id", DataType::Int32, false)])); + + let source = Arc::new(LogicalTableSource::new(schema)); + let schema = Arc::new( + DFSchema::try_from_qualified_schema( + TableReference::bare("tab"), + &source.schema(), + ) + .unwrap(), + ); + let scan = Arc::new(LogicalPlan::TableScan(TableScan { + table_name: TableReference::bare("tab"), + source: source.clone(), + projection: None, + projected_schema: schema.clone(), + filters: vec![], + fetch: None, + })); + let col = schema.field(0).qualified_column(); + + let filter = Filter::try_new( + Expr::Column(col).eq(Expr::Literal(ScalarValue::Int32(Some(1)))), + scan, + ) + .unwrap(); + assert!(!filter.is_scalar()); + let unique_schema = + Arc::new(schema.as_ref().clone().with_functional_dependencies( + FunctionalDependencies::new_from_constraints( + Some(&Constraints::new_unverified(vec![Constraint::Unique( + vec![0], + )])), + 1, + ), + )); + let scan = Arc::new(LogicalPlan::TableScan(TableScan { + table_name: TableReference::bare("tab"), + source, + projection: None, + projected_schema: unique_schema.clone(), + filters: vec![], + fetch: None, + })); + let col = schema.field(0).qualified_column(); + + let filter = Filter::try_new( + Expr::Column(col).eq(Expr::Literal(ScalarValue::Int32(Some(1)))), + scan, + ) + .unwrap(); + assert!(filter.is_scalar()); + } + #[test] fn test_transform_explain() { let schema = Schema::new(vec![ diff --git a/datafusion/sqllogictest/test_files/subquery.slt b/datafusion/sqllogictest/test_files/subquery.slt index 430e676fa477..3e0fcb7aa96e 100644 --- a/datafusion/sqllogictest/test_files/subquery.slt +++ b/datafusion/sqllogictest/test_files/subquery.slt @@ -49,6 +49,13 @@ CREATE TABLE t2(t2_id INT, t2_name TEXT, t2_int INT) AS VALUES (44, 'x', 3), (55, 'w', 3); +statement ok +CREATE TABLE t3(t3_id INT PRIMARY KEY, t3_name TEXT, t3_int INT) AS VALUES +(11, 'e', 3), +(22, 'f', 1), +(44, 'g', 3), +(55, 'h', 3); + statement ok CREATE EXTERNAL TABLE IF NOT EXISTS customer ( c_custkey BIGINT, @@ -419,6 +426,17 @@ SELECT t1_id, t1_name, t1_int FROM t1 order by t1_int in (SELECT t2_int FROM t2 statement error DataFusion error: check_analyzed_plan\ncaused by\nError during planning: Correlated scalar subquery must be aggregated to return at most one row SELECT t1_id, (SELECT t2_int FROM t2 WHERE t2.t2_int = t1.t1_int) as t2_int from t1 +#non_aggregated_correlated_scalar_subquery_unique +query II rowsort +SELECT t1_id, (SELECT t3_int FROM t3 WHERE t3.t3_id = t1.t1_id) as t3_int from t1 +---- +11 3 +22 1 +33 NULL +44 3 + + +#non_aggregated_correlated_scalar_subquery statement error DataFusion error: check_analyzed_plan\ncaused by\nError during planning: Correlated scalar subquery must be aggregated to return at most one row SELECT t1_id, (SELECT t2_int FROM t2 WHERE t2.t2_int = t1_int group by t2_int) as t2_int from t1 From 33fc1104c199904fb0ee019546ac6587e7088316 Mon Sep 17 00:00:00 2001 From: Mustafa Akur <106137913+mustafasrepo@users.noreply.github.com> Date: Thu, 7 Dec 2023 09:23:40 +0300 Subject: [PATCH 380/572] Add alias check to optimize projections merge (#8438) * Relax schema check for optimize projections. * Minor changes * Update datafusion/optimizer/src/optimize_projections.rs Co-authored-by: jakevin --------- Co-authored-by: jakevin --- datafusion/optimizer/src/optimize_projections.rs | 15 ++++++++++++--- datafusion/sqllogictest/test_files/select.slt | 9 +++++++++ 2 files changed, 21 insertions(+), 3 deletions(-) diff --git a/datafusion/optimizer/src/optimize_projections.rs b/datafusion/optimizer/src/optimize_projections.rs index bbf704a83c55..440e12cc26d7 100644 --- a/datafusion/optimizer/src/optimize_projections.rs +++ b/datafusion/optimizer/src/optimize_projections.rs @@ -405,9 +405,18 @@ fn merge_consecutive_projections(proj: &Projection) -> Result .iter() .map(|expr| rewrite_expr(expr, prev_projection)) .collect::>>>()?; - new_exprs - .map(|exprs| Projection::try_new(exprs, prev_projection.input.clone())) - .transpose() + if let Some(new_exprs) = new_exprs { + let new_exprs = new_exprs + .into_iter() + .zip(proj.expr.iter()) + .map(|(new_expr, old_expr)| { + new_expr.alias_if_changed(old_expr.name_for_alias()?) + }) + .collect::>>()?; + Projection::try_new(new_exprs, prev_projection.input.clone()).map(Some) + } else { + Ok(None) + } } /// Trim Expression diff --git a/datafusion/sqllogictest/test_files/select.slt b/datafusion/sqllogictest/test_files/select.slt index 3f3befd85a59..bbb05b6cffa7 100644 --- a/datafusion/sqllogictest/test_files/select.slt +++ b/datafusion/sqllogictest/test_files/select.slt @@ -1056,3 +1056,12 @@ drop table annotated_data_finite2; statement ok drop table t; + +statement ok +create table t(x bigint, y bigint) as values (1,2), (1,3); + +query II +select z+1, y from (select x+1 as z, y from t) where y > 1; +---- +3 2 +3 3 From 5e8b0e09228925b01c8bcc7afe448a7487347872 Mon Sep 17 00:00:00 2001 From: Jay Zhan Date: Thu, 7 Dec 2023 23:23:00 +0800 Subject: [PATCH 381/572] Fix PartialOrd for ScalarValue::List/FixSizeList/LargeList (#8253) * list cmp Signed-off-by: jayzhan211 * remove cfg Signed-off-by: jayzhan211 --------- Signed-off-by: jayzhan211 Co-authored-by: Andrew Lamb --- datafusion/common/src/scalar.rs | 110 ++++++++++---------------------- 1 file changed, 35 insertions(+), 75 deletions(-) diff --git a/datafusion/common/src/scalar.rs b/datafusion/common/src/scalar.rs index 1f302c750916..7e18c313e090 100644 --- a/datafusion/common/src/scalar.rs +++ b/datafusion/common/src/scalar.rs @@ -358,69 +358,47 @@ impl PartialOrd for ScalarValue { (FixedSizeBinary(_, _), _) => None, (LargeBinary(v1), LargeBinary(v2)) => v1.partial_cmp(v2), (LargeBinary(_), _) => None, - (List(arr1), List(arr2)) | (FixedSizeList(arr1), FixedSizeList(arr2)) => { - if arr1.data_type() == arr2.data_type() { - let list_arr1 = as_list_array(arr1); - let list_arr2 = as_list_array(arr2); - if list_arr1.len() != list_arr2.len() { - return None; - } - for i in 0..list_arr1.len() { - let arr1 = list_arr1.value(i); - let arr2 = list_arr2.value(i); - - let lt_res = - arrow::compute::kernels::cmp::lt(&arr1, &arr2).ok()?; - let eq_res = - arrow::compute::kernels::cmp::eq(&arr1, &arr2).ok()?; - - for j in 0..lt_res.len() { - if lt_res.is_valid(j) && lt_res.value(j) { - return Some(Ordering::Less); - } - if eq_res.is_valid(j) && !eq_res.value(j) { - return Some(Ordering::Greater); - } - } + (List(arr1), List(arr2)) + | (FixedSizeList(arr1), FixedSizeList(arr2)) + | (LargeList(arr1), LargeList(arr2)) => { + // ScalarValue::List / ScalarValue::FixedSizeList / ScalarValue::LargeList are ensure to have length 1 + assert_eq!(arr1.len(), 1); + assert_eq!(arr2.len(), 1); + + if arr1.data_type() != arr2.data_type() { + return None; + } + + fn first_array_for_list(arr: &ArrayRef) -> ArrayRef { + if let Some(arr) = arr.as_list_opt::() { + arr.value(0) + } else if let Some(arr) = arr.as_list_opt::() { + arr.value(0) + } else if let Some(arr) = arr.as_fixed_size_list_opt() { + arr.value(0) + } else { + unreachable!("Since only List / LargeList / FixedSizeList are supported, this should never happen") } - Some(Ordering::Equal) - } else { - None } - } - (LargeList(arr1), LargeList(arr2)) => { - if arr1.data_type() == arr2.data_type() { - let list_arr1 = as_large_list_array(arr1); - let list_arr2 = as_large_list_array(arr2); - if list_arr1.len() != list_arr2.len() { - return None; + + let arr1 = first_array_for_list(arr1); + let arr2 = first_array_for_list(arr2); + + let lt_res = arrow::compute::kernels::cmp::lt(&arr1, &arr2).ok()?; + let eq_res = arrow::compute::kernels::cmp::eq(&arr1, &arr2).ok()?; + + for j in 0..lt_res.len() { + if lt_res.is_valid(j) && lt_res.value(j) { + return Some(Ordering::Less); } - for i in 0..list_arr1.len() { - let arr1 = list_arr1.value(i); - let arr2 = list_arr2.value(i); - - let lt_res = - arrow::compute::kernels::cmp::lt(&arr1, &arr2).ok()?; - let eq_res = - arrow::compute::kernels::cmp::eq(&arr1, &arr2).ok()?; - - for j in 0..lt_res.len() { - if lt_res.is_valid(j) && lt_res.value(j) { - return Some(Ordering::Less); - } - if eq_res.is_valid(j) && !eq_res.value(j) { - return Some(Ordering::Greater); - } - } + if eq_res.is_valid(j) && !eq_res.value(j) { + return Some(Ordering::Greater); } - Some(Ordering::Equal) - } else { - None } + + Some(Ordering::Equal) } - (List(_), _) => None, - (LargeList(_), _) => None, - (FixedSizeList(_), _) => None, + (List(_), _) | (LargeList(_), _) | (FixedSizeList(_), _) => None, (Date32(v1), Date32(v2)) => v1.partial_cmp(v2), (Date32(_), _) => None, (Date64(v1), Date64(v2)) => v1.partial_cmp(v2), @@ -3644,24 +3622,6 @@ mod tests { ])]), )); assert_eq!(a.partial_cmp(&b), Some(Ordering::Less)); - - let a = - ScalarValue::List(Arc::new( - ListArray::from_iter_primitive::(vec![ - Some(vec![Some(10), Some(2), Some(3)]), - None, - Some(vec![Some(10), Some(2), Some(3)]), - ]), - )); - let b = - ScalarValue::List(Arc::new( - ListArray::from_iter_primitive::(vec![ - Some(vec![Some(10), Some(2), Some(3)]), - None, - Some(vec![Some(10), Some(2), Some(3)]), - ]), - )); - assert_eq!(a.partial_cmp(&b), Some(Ordering::Equal)); } #[test] From 6767ea347209be4e366cf8fc258343cf8f0be175 Mon Sep 17 00:00:00 2001 From: Tan Wei Date: Fri, 8 Dec 2023 05:23:34 +0800 Subject: [PATCH 382/572] Support parquet_metadata for datafusion-cli (#8413) * Support parquet_metadata for datafusion-cli Signed-off-by: veeupup * make tomlfmt happy * display like duckdb Signed-off-by: veeupup * add test & fix single quote --------- Signed-off-by: veeupup --- datafusion-cli/Cargo.lock | 1 + datafusion-cli/Cargo.toml | 1 + datafusion-cli/src/functions.rs | 223 +++++++++++++++++++++++++++++++- datafusion-cli/src/main.rs | 35 +++++ 4 files changed, 258 insertions(+), 2 deletions(-) diff --git a/datafusion-cli/Cargo.lock b/datafusion-cli/Cargo.lock index f88c907b052f..76be04d5ef67 100644 --- a/datafusion-cli/Cargo.lock +++ b/datafusion-cli/Cargo.lock @@ -1161,6 +1161,7 @@ dependencies = [ "mimalloc", "object_store", "parking_lot", + "parquet", "predicates", "regex", "rstest", diff --git a/datafusion-cli/Cargo.toml b/datafusion-cli/Cargo.toml index fd2dfd76c20e..5ce318aea3ac 100644 --- a/datafusion-cli/Cargo.toml +++ b/datafusion-cli/Cargo.toml @@ -40,6 +40,7 @@ env_logger = "0.9" mimalloc = { version = "0.1", default-features = false } object_store = { version = "0.8.0", features = ["aws", "gcp"] } parking_lot = { version = "0.12" } +parquet = { version = "49.0.0", default-features = false } regex = "1.8" rustyline = "11.0" tokio = { version = "1.24", features = ["macros", "rt", "rt-multi-thread", "sync", "parking_lot"] } diff --git a/datafusion-cli/src/functions.rs b/datafusion-cli/src/functions.rs index eeebe713d716..24f3399ee2be 100644 --- a/datafusion-cli/src/functions.rs +++ b/datafusion-cli/src/functions.rs @@ -16,12 +16,26 @@ // under the License. //! Functions that are query-able and searchable via the `\h` command -use arrow::array::StringArray; -use arrow::datatypes::{DataType, Field, Schema}; +use arrow::array::{Int64Array, StringArray}; +use arrow::datatypes::{DataType, Field, Schema, SchemaRef}; use arrow::record_batch::RecordBatch; use arrow::util::pretty::pretty_format_batches; +use async_trait::async_trait; +use datafusion::common::DataFusionError; +use datafusion::common::{plan_err, Column}; +use datafusion::datasource::function::TableFunctionImpl; +use datafusion::datasource::TableProvider; use datafusion::error::Result; +use datafusion::execution::context::SessionState; +use datafusion::logical_expr::Expr; +use datafusion::physical_plan::memory::MemoryExec; +use datafusion::physical_plan::ExecutionPlan; +use datafusion::scalar::ScalarValue; +use parquet::file::reader::FileReader; +use parquet::file::serialized_reader::SerializedFileReader; +use parquet::file::statistics::Statistics; use std::fmt; +use std::fs::File; use std::str::FromStr; use std::sync::Arc; @@ -196,3 +210,208 @@ pub fn display_all_functions() -> Result<()> { println!("{}", pretty_format_batches(&[batch]).unwrap()); Ok(()) } + +/// PARQUET_META table function +struct ParquetMetadataTable { + schema: SchemaRef, + batch: RecordBatch, +} + +#[async_trait] +impl TableProvider for ParquetMetadataTable { + fn as_any(&self) -> &dyn std::any::Any { + self + } + + fn schema(&self) -> arrow::datatypes::SchemaRef { + self.schema.clone() + } + + fn table_type(&self) -> datafusion::logical_expr::TableType { + datafusion::logical_expr::TableType::Base + } + + async fn scan( + &self, + _state: &SessionState, + projection: Option<&Vec>, + _filters: &[Expr], + _limit: Option, + ) -> Result> { + Ok(Arc::new(MemoryExec::try_new( + &[vec![self.batch.clone()]], + TableProvider::schema(self), + projection.cloned(), + )?)) + } +} + +pub struct ParquetMetadataFunc {} + +impl TableFunctionImpl for ParquetMetadataFunc { + fn call(&self, exprs: &[Expr]) -> Result> { + let filename = match exprs.get(0) { + Some(Expr::Literal(ScalarValue::Utf8(Some(s)))) => s, // single quote: parquet_metadata('x.parquet') + Some(Expr::Column(Column { name, .. })) => name, // double quote: parquet_metadata("x.parquet") + _ => { + return plan_err!( + "parquet_metadata requires string argument as its input" + ); + } + }; + + let file = File::open(filename.clone())?; + let reader = SerializedFileReader::new(file)?; + let metadata = reader.metadata(); + + let schema = Arc::new(Schema::new(vec![ + Field::new("filename", DataType::Utf8, true), + Field::new("row_group_id", DataType::Int64, true), + Field::new("row_group_num_rows", DataType::Int64, true), + Field::new("row_group_num_columns", DataType::Int64, true), + Field::new("row_group_bytes", DataType::Int64, true), + Field::new("column_id", DataType::Int64, true), + Field::new("file_offset", DataType::Int64, true), + Field::new("num_values", DataType::Int64, true), + Field::new("path_in_schema", DataType::Utf8, true), + Field::new("type", DataType::Utf8, true), + Field::new("stats_min", DataType::Utf8, true), + Field::new("stats_max", DataType::Utf8, true), + Field::new("stats_null_count", DataType::Int64, true), + Field::new("stats_distinct_count", DataType::Int64, true), + Field::new("stats_min_value", DataType::Utf8, true), + Field::new("stats_max_value", DataType::Utf8, true), + Field::new("compression", DataType::Utf8, true), + Field::new("encodings", DataType::Utf8, true), + Field::new("index_page_offset", DataType::Int64, true), + Field::new("dictionary_page_offset", DataType::Int64, true), + Field::new("data_page_offset", DataType::Int64, true), + Field::new("total_compressed_size", DataType::Int64, true), + Field::new("total_uncompressed_size", DataType::Int64, true), + ])); + + // construct recordbatch from metadata + let mut filename_arr = vec![]; + let mut row_group_id_arr = vec![]; + let mut row_group_num_rows_arr = vec![]; + let mut row_group_num_columns_arr = vec![]; + let mut row_group_bytes_arr = vec![]; + let mut column_id_arr = vec![]; + let mut file_offset_arr = vec![]; + let mut num_values_arr = vec![]; + let mut path_in_schema_arr = vec![]; + let mut type_arr = vec![]; + let mut stats_min_arr = vec![]; + let mut stats_max_arr = vec![]; + let mut stats_null_count_arr = vec![]; + let mut stats_distinct_count_arr = vec![]; + let mut stats_min_value_arr = vec![]; + let mut stats_max_value_arr = vec![]; + let mut compression_arr = vec![]; + let mut encodings_arr = vec![]; + let mut index_page_offset_arr = vec![]; + let mut dictionary_page_offset_arr = vec![]; + let mut data_page_offset_arr = vec![]; + let mut total_compressed_size_arr = vec![]; + let mut total_uncompressed_size_arr = vec![]; + for (rg_idx, row_group) in metadata.row_groups().iter().enumerate() { + for (col_idx, column) in row_group.columns().iter().enumerate() { + filename_arr.push(filename.clone()); + row_group_id_arr.push(rg_idx as i64); + row_group_num_rows_arr.push(row_group.num_rows()); + row_group_num_columns_arr.push(row_group.num_columns() as i64); + row_group_bytes_arr.push(row_group.total_byte_size()); + column_id_arr.push(col_idx as i64); + file_offset_arr.push(column.file_offset()); + num_values_arr.push(column.num_values()); + path_in_schema_arr.push(column.column_path().to_string()); + type_arr.push(column.column_type().to_string()); + if let Some(s) = column.statistics() { + let (min_val, max_val) = if s.has_min_max_set() { + let (min_val, max_val) = match s { + Statistics::Boolean(val) => { + (val.min().to_string(), val.max().to_string()) + } + Statistics::Int32(val) => { + (val.min().to_string(), val.max().to_string()) + } + Statistics::Int64(val) => { + (val.min().to_string(), val.max().to_string()) + } + Statistics::Int96(val) => { + (val.min().to_string(), val.max().to_string()) + } + Statistics::Float(val) => { + (val.min().to_string(), val.max().to_string()) + } + Statistics::Double(val) => { + (val.min().to_string(), val.max().to_string()) + } + Statistics::ByteArray(val) => { + (val.min().to_string(), val.max().to_string()) + } + Statistics::FixedLenByteArray(val) => { + (val.min().to_string(), val.max().to_string()) + } + }; + (Some(min_val), Some(max_val)) + } else { + (None, None) + }; + stats_min_arr.push(min_val.clone()); + stats_max_arr.push(max_val.clone()); + stats_null_count_arr.push(Some(s.null_count() as i64)); + stats_distinct_count_arr.push(s.distinct_count().map(|c| c as i64)); + stats_min_value_arr.push(min_val); + stats_max_value_arr.push(max_val); + } else { + stats_min_arr.push(None); + stats_max_arr.push(None); + stats_null_count_arr.push(None); + stats_distinct_count_arr.push(None); + stats_min_value_arr.push(None); + stats_max_value_arr.push(None); + }; + compression_arr.push(format!("{:?}", column.compression())); + encodings_arr.push(format!("{:?}", column.encodings())); + index_page_offset_arr.push(column.index_page_offset()); + dictionary_page_offset_arr.push(column.dictionary_page_offset()); + data_page_offset_arr.push(column.data_page_offset()); + total_compressed_size_arr.push(column.compressed_size()); + total_uncompressed_size_arr.push(column.uncompressed_size()); + } + } + + let rb = RecordBatch::try_new( + schema.clone(), + vec![ + Arc::new(StringArray::from(filename_arr)), + Arc::new(Int64Array::from(row_group_id_arr)), + Arc::new(Int64Array::from(row_group_num_rows_arr)), + Arc::new(Int64Array::from(row_group_num_columns_arr)), + Arc::new(Int64Array::from(row_group_bytes_arr)), + Arc::new(Int64Array::from(column_id_arr)), + Arc::new(Int64Array::from(file_offset_arr)), + Arc::new(Int64Array::from(num_values_arr)), + Arc::new(StringArray::from(path_in_schema_arr)), + Arc::new(StringArray::from(type_arr)), + Arc::new(StringArray::from(stats_min_arr)), + Arc::new(StringArray::from(stats_max_arr)), + Arc::new(Int64Array::from(stats_null_count_arr)), + Arc::new(Int64Array::from(stats_distinct_count_arr)), + Arc::new(StringArray::from(stats_min_value_arr)), + Arc::new(StringArray::from(stats_max_value_arr)), + Arc::new(StringArray::from(compression_arr)), + Arc::new(StringArray::from(encodings_arr)), + Arc::new(Int64Array::from(index_page_offset_arr)), + Arc::new(Int64Array::from(dictionary_page_offset_arr)), + Arc::new(Int64Array::from(data_page_offset_arr)), + Arc::new(Int64Array::from(total_compressed_size_arr)), + Arc::new(Int64Array::from(total_uncompressed_size_arr)), + ], + )?; + + let parquet_metadata = ParquetMetadataTable { schema, batch: rb }; + Ok(Arc::new(parquet_metadata)) + } +} diff --git a/datafusion-cli/src/main.rs b/datafusion-cli/src/main.rs index c069f458f196..8b1a9816afc0 100644 --- a/datafusion-cli/src/main.rs +++ b/datafusion-cli/src/main.rs @@ -22,6 +22,7 @@ use datafusion::execution::memory_pool::{FairSpillPool, GreedyMemoryPool}; use datafusion::execution::runtime_env::{RuntimeConfig, RuntimeEnv}; use datafusion::prelude::SessionContext; use datafusion_cli::catalog::DynamicFileCatalog; +use datafusion_cli::functions::ParquetMetadataFunc; use datafusion_cli::{ exec, print_format::PrintFormat, @@ -185,6 +186,8 @@ pub async fn main() -> Result<()> { ctx.state().catalog_list(), ctx.state_weak_ref(), ))); + // register `parquet_metadata` table function to get metadata from parquet files + ctx.register_udtf("parquet_metadata", Arc::new(ParquetMetadataFunc {})); let mut print_options = PrintOptions { format: args.format, @@ -328,6 +331,8 @@ fn extract_memory_pool_size(size: &str) -> Result { #[cfg(test)] mod tests { + use datafusion::assert_batches_eq; + use super::*; fn assert_conversion(input: &str, expected: Result) { @@ -385,4 +390,34 @@ mod tests { Ok(()) } + + #[tokio::test] + async fn test_parquet_metadata_works() -> Result<(), DataFusionError> { + let ctx = SessionContext::new(); + ctx.register_udtf("parquet_metadata", Arc::new(ParquetMetadataFunc {})); + + // input with single quote + let sql = + "SELECT * FROM parquet_metadata('../datafusion/core/tests/data/fixed_size_list_array.parquet')"; + let df = ctx.sql(sql).await?; + let rbs = df.collect().await?; + + let excepted = [ + "+-------------------------------------------------------------+--------------+--------------------+-----------------------+-----------------+-----------+-------------+------------+----------------+-------+-----------+-----------+------------------+----------------------+-----------------+-----------------+-------------+------------------------------+-------------------+------------------------+------------------+-----------------------+-------------------------+", + "| filename | row_group_id | row_group_num_rows | row_group_num_columns | row_group_bytes | column_id | file_offset | num_values | path_in_schema | type | stats_min | stats_max | stats_null_count | stats_distinct_count | stats_min_value | stats_max_value | compression | encodings | index_page_offset | dictionary_page_offset | data_page_offset | total_compressed_size | total_uncompressed_size |", + "+-------------------------------------------------------------+--------------+--------------------+-----------------------+-----------------+-----------+-------------+------------+----------------+-------+-----------+-----------+------------------+----------------------+-----------------+-----------------+-------------+------------------------------+-------------------+------------------------+------------------+-----------------------+-------------------------+", + "| ../datafusion/core/tests/data/fixed_size_list_array.parquet | 0 | 2 | 1 | 123 | 0 | 125 | 4 | \"f0.list.item\" | INT64 | 1 | 4 | 0 | | 1 | 4 | SNAPPY | [RLE_DICTIONARY, PLAIN, RLE] | | 4 | 46 | 121 | 123 |", + "+-------------------------------------------------------------+--------------+--------------------+-----------------------+-----------------+-----------+-------------+------------+----------------+-------+-----------+-----------+------------------+----------------------+-----------------+-----------------+-------------+------------------------------+-------------------+------------------------+------------------+-----------------------+-------------------------+", + ]; + assert_batches_eq!(excepted, &rbs); + + // input with double quote + let sql = + "SELECT * FROM parquet_metadata(\"../datafusion/core/tests/data/fixed_size_list_array.parquet\")"; + let df = ctx.sql(sql).await?; + let rbs = df.collect().await?; + assert_batches_eq!(excepted, &rbs); + + Ok(()) + } } From d5dd5351995bb09797827d879af070631b6f58c7 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Dani=C3=ABl=20Heres?= Date: Thu, 7 Dec 2023 22:47:34 +0100 Subject: [PATCH 383/572] Fix bug in optimizing a nested count (#8459) * Fix nested count optimization * fmt * extend comment * Clippy * Update datafusion/optimizer/src/optimize_projections.rs Co-authored-by: Liang-Chi Hsieh * Add sqllogictests * Fmt --------- Co-authored-by: Liang-Chi Hsieh --- .../optimizer/src/optimize_projections.rs | 40 +++++++++++++++++-- .../sqllogictest/test_files/aggregate.slt | 13 ++++++ 2 files changed, 49 insertions(+), 4 deletions(-) diff --git a/datafusion/optimizer/src/optimize_projections.rs b/datafusion/optimizer/src/optimize_projections.rs index 440e12cc26d7..8bee2951541d 100644 --- a/datafusion/optimizer/src/optimize_projections.rs +++ b/datafusion/optimizer/src/optimize_projections.rs @@ -192,7 +192,7 @@ fn optimize_projections( let new_group_bys = aggregate.group_expr.clone(); // Only use absolutely necessary aggregate expressions required by parent. - let new_aggr_expr = get_at_indices(&aggregate.aggr_expr, &aggregate_reqs); + let mut new_aggr_expr = get_at_indices(&aggregate.aggr_expr, &aggregate_reqs); let all_exprs_iter = new_group_bys.iter().chain(new_aggr_expr.iter()); let necessary_indices = indices_referred_by_exprs(&aggregate.input, all_exprs_iter)?; @@ -213,6 +213,16 @@ fn optimize_projections( let (aggregate_input, _is_added) = add_projection_on_top_if_helpful(aggregate_input, necessary_exprs, true)?; + // Aggregate always needs at least one aggregate expression. + // With a nested count we don't require any column as input, but still need to create a correct aggregate + // The aggregate may be optimized out later (select count(*) from (select count(*) from [...]) always returns 1 + if new_aggr_expr.is_empty() + && new_group_bys.is_empty() + && !aggregate.aggr_expr.is_empty() + { + new_aggr_expr = vec![aggregate.aggr_expr[0].clone()]; + } + // Create new aggregate plan with updated input, and absolutely necessary fields. return Aggregate::try_new( Arc::new(aggregate_input), @@ -857,10 +867,11 @@ fn rewrite_projection_given_requirements( #[cfg(test)] mod tests { use crate::optimize_projections::OptimizeProjections; - use datafusion_common::Result; + use arrow::datatypes::{DataType, Field, Schema}; + use datafusion_common::{Result, TableReference}; use datafusion_expr::{ - binary_expr, col, lit, logical_plan::builder::LogicalPlanBuilder, LogicalPlan, - Operator, + binary_expr, col, count, lit, logical_plan::builder::LogicalPlanBuilder, + table_scan, Expr, LogicalPlan, Operator, }; use std::sync::Arc; @@ -909,4 +920,25 @@ mod tests { \n TableScan: test projection=[a]"; assert_optimized_plan_equal(&plan, expected) } + #[test] + fn test_nested_count() -> Result<()> { + let schema = Schema::new(vec![Field::new("foo", DataType::Int32, false)]); + + let groups: Vec = vec![]; + + let plan = table_scan(TableReference::none(), &schema, None) + .unwrap() + .aggregate(groups.clone(), vec![count(lit(1))]) + .unwrap() + .aggregate(groups, vec![count(lit(1))]) + .unwrap() + .build() + .unwrap(); + + let expected = "Aggregate: groupBy=[[]], aggr=[[COUNT(Int32(1))]]\ + \n Projection: \ + \n Aggregate: groupBy=[[]], aggr=[[COUNT(Int32(1))]]\ + \n TableScan: ?table? projection=[]"; + assert_optimized_plan_equal(&plan, expected) + } } diff --git a/datafusion/sqllogictest/test_files/aggregate.slt b/datafusion/sqllogictest/test_files/aggregate.slt index e4718035a58d..7cfc9c707d43 100644 --- a/datafusion/sqllogictest/test_files/aggregate.slt +++ b/datafusion/sqllogictest/test_files/aggregate.slt @@ -3199,3 +3199,16 @@ FROM my_data GROUP BY dummy ---- text1, text1, text1 + + +# Queries with nested count(*) + +query I +select count(*) from (select count(*) from (select 1)); +---- +1 + +query I +select count(*) from (select count(*) a, count(*) b from (select 1)); +---- +1 \ No newline at end of file From 1aedf8d42c95d10c02a08977ebae226fc0a37aea Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Thu, 7 Dec 2023 17:24:38 -0500 Subject: [PATCH 384/572] Bump actions/setup-python from 4 to 5 (#8449) Bumps [actions/setup-python](https://github.com/actions/setup-python) from 4 to 5. - [Release notes](https://github.com/actions/setup-python/releases) - [Commits](https://github.com/actions/setup-python/compare/v4...v5) --- updated-dependencies: - dependency-name: actions/setup-python dependency-type: direct:production update-type: version-update:semver-major ... Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- .github/workflows/dev.yml | 2 +- .github/workflows/docs.yaml | 2 +- .github/workflows/rust.yml | 2 +- 3 files changed, 3 insertions(+), 3 deletions(-) diff --git a/.github/workflows/dev.yml b/.github/workflows/dev.yml index cc23e99e8cba..19af21ec910b 100644 --- a/.github/workflows/dev.yml +++ b/.github/workflows/dev.yml @@ -30,7 +30,7 @@ jobs: - name: Checkout uses: actions/checkout@v4 - name: Setup Python - uses: actions/setup-python@v4 + uses: actions/setup-python@v5 with: python-version: "3.10" - name: Audit licenses diff --git a/.github/workflows/docs.yaml b/.github/workflows/docs.yaml index 14b2038e8794..ab6a615ab60b 100644 --- a/.github/workflows/docs.yaml +++ b/.github/workflows/docs.yaml @@ -24,7 +24,7 @@ jobs: path: asf-site - name: Setup Python - uses: actions/setup-python@v4 + uses: actions/setup-python@v5 with: python-version: "3.10" diff --git a/.github/workflows/rust.yml b/.github/workflows/rust.yml index 485d179571e3..099aab061435 100644 --- a/.github/workflows/rust.yml +++ b/.github/workflows/rust.yml @@ -348,7 +348,7 @@ jobs: - uses: actions/checkout@v4 with: submodules: true - - uses: actions/setup-python@v4 + - uses: actions/setup-python@v5 with: python-version: "3.8" - name: Install PyArrow From 9be9073703e439eef8fe25c375a32bc40da7ce21 Mon Sep 17 00:00:00 2001 From: Liang-Chi Hsieh Date: Thu, 7 Dec 2023 14:33:58 -0800 Subject: [PATCH 385/572] fix: ORDER BY window definition should work on null (#8444) --- datafusion/optimizer/src/analyzer/type_coercion.rs | 5 ++++- datafusion/sqllogictest/test_files/window.slt | 8 ++++++++ 2 files changed, 12 insertions(+), 1 deletion(-) diff --git a/datafusion/optimizer/src/analyzer/type_coercion.rs b/datafusion/optimizer/src/analyzer/type_coercion.rs index e3b86f5db78f..91611251d9dd 100644 --- a/datafusion/optimizer/src/analyzer/type_coercion.rs +++ b/datafusion/optimizer/src/analyzer/type_coercion.rs @@ -503,7 +503,10 @@ fn coerce_window_frame( let target_type = match window_frame.units { WindowFrameUnits::Range => { if let Some(col_type) = current_types.first() { - if col_type.is_numeric() || is_utf8_or_large_utf8(col_type) { + if col_type.is_numeric() + || is_utf8_or_large_utf8(col_type) + || matches!(col_type, DataType::Null) + { col_type } else if is_datetime(col_type) { &DataType::Interval(IntervalUnit::MonthDayNano) diff --git a/datafusion/sqllogictest/test_files/window.slt b/datafusion/sqllogictest/test_files/window.slt index 0179431ac8ad..5b69ead0ff36 100644 --- a/datafusion/sqllogictest/test_files/window.slt +++ b/datafusion/sqllogictest/test_files/window.slt @@ -3785,3 +3785,11 @@ select a, ---- 1 1 2 1 + +query II +select a, + rank() over (order by null RANGE BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING) rnk + from (select 1 a union select 2 a) q ORDER BY a +---- +1 1 +2 1 From c0c9e8888878c5d7f2586cf605702430c94ea425 Mon Sep 17 00:00:00 2001 From: Ruihang Xia Date: Fri, 8 Dec 2023 10:23:12 +0800 Subject: [PATCH 386/572] flx clippy warnings (#8455) * change get zero to first() Signed-off-by: Ruihang Xia * wake clone to wake_by_ref Signed-off-by: Ruihang Xia * more first() Signed-off-by: Ruihang Xia * try_from() to from() Signed-off-by: Ruihang Xia --------- Signed-off-by: Ruihang Xia --- .../examples/custom_datasource.rs | 2 +- datafusion-examples/examples/memtable.rs | 2 +- datafusion-examples/examples/simple_udtf.rs | 2 +- datafusion/common/src/dfschema.rs | 4 ++-- datafusion/common/src/error.rs | 16 +++++++--------- datafusion/common/src/utils.rs | 4 ++-- datafusion/core/benches/sort_limit_query_sql.rs | 2 +- datafusion/core/benches/sql_query_with_io.rs | 5 ++--- .../core/src/datasource/file_format/parquet.rs | 4 ++-- .../src/datasource/file_format/write/demux.rs | 7 ++----- datafusion/core/src/datasource/listing/table.rs | 12 ++++++------ .../core/src/datasource/physical_plan/json.rs | 8 ++++---- .../core/src/datasource/physical_plan/mod.rs | 2 +- datafusion/core/src/physical_planner.rs | 2 +- datafusion/core/tests/parquet/file_statistics.rs | 6 +++--- datafusion/core/tests/sql/mod.rs | 1 - datafusion/core/tests/sql/parquet.rs | 2 +- datafusion/execution/src/cache/cache_unit.rs | 2 +- datafusion/expr/src/utils.rs | 1 - datafusion/optimizer/src/eliminate_limit.rs | 2 +- datafusion/optimizer/src/push_down_filter.rs | 2 +- datafusion/optimizer/src/push_down_projection.rs | 2 +- datafusion/optimizer/src/test/mod.rs | 8 ++++---- .../src/aggregate/groups_accumulator/adapter.rs | 2 +- .../physical-expr/src/array_expressions.rs | 4 ++-- datafusion/physical-plan/src/memory.rs | 2 +- datafusion/physical-plan/src/test/exec.rs | 2 +- datafusion/sql/src/relation/mod.rs | 2 +- 28 files changed, 51 insertions(+), 59 deletions(-) diff --git a/datafusion-examples/examples/custom_datasource.rs b/datafusion-examples/examples/custom_datasource.rs index 9f25a0b2fa47..69f9c9530e87 100644 --- a/datafusion-examples/examples/custom_datasource.rs +++ b/datafusion-examples/examples/custom_datasource.rs @@ -80,7 +80,7 @@ async fn search_accounts( timeout(Duration::from_secs(10), async move { let result = dataframe.collect().await.unwrap(); - let record_batch = result.get(0).unwrap(); + let record_batch = result.first().unwrap(); assert_eq!(expected_result_length, record_batch.column(1).len()); dbg!(record_batch.columns()); diff --git a/datafusion-examples/examples/memtable.rs b/datafusion-examples/examples/memtable.rs index bef8f3e5bb8f..5cce578039e7 100644 --- a/datafusion-examples/examples/memtable.rs +++ b/datafusion-examples/examples/memtable.rs @@ -40,7 +40,7 @@ async fn main() -> Result<()> { timeout(Duration::from_secs(10), async move { let result = dataframe.collect().await.unwrap(); - let record_batch = result.get(0).unwrap(); + let record_batch = result.first().unwrap(); assert_eq!(1, record_batch.column(0).len()); dbg!(record_batch.columns()); diff --git a/datafusion-examples/examples/simple_udtf.rs b/datafusion-examples/examples/simple_udtf.rs index bce633765281..e120c5e7bf8e 100644 --- a/datafusion-examples/examples/simple_udtf.rs +++ b/datafusion-examples/examples/simple_udtf.rs @@ -129,7 +129,7 @@ struct LocalCsvTableFunc {} impl TableFunctionImpl for LocalCsvTableFunc { fn call(&self, exprs: &[Expr]) -> Result> { - let Some(Expr::Literal(ScalarValue::Utf8(Some(ref path)))) = exprs.get(0) else { + let Some(Expr::Literal(ScalarValue::Utf8(Some(ref path)))) = exprs.first() else { return plan_err!("read_csv requires at least one string argument"); }; diff --git a/datafusion/common/src/dfschema.rs b/datafusion/common/src/dfschema.rs index 52cd85675824..9819ae795b74 100644 --- a/datafusion/common/src/dfschema.rs +++ b/datafusion/common/src/dfschema.rs @@ -1476,8 +1476,8 @@ mod tests { DFSchema::new_with_metadata([a, b].to_vec(), HashMap::new()).unwrap(), ); let schema: Schema = df_schema.as_ref().clone().into(); - let a_df = df_schema.fields.get(0).unwrap().field(); - let a_arrow = schema.fields.get(0).unwrap(); + let a_df = df_schema.fields.first().unwrap().field(); + let a_arrow = schema.fields.first().unwrap(); assert_eq!(a_df.metadata(), a_arrow.metadata()) } diff --git a/datafusion/common/src/error.rs b/datafusion/common/src/error.rs index 9114c669ab8b..4ae30ae86cdd 100644 --- a/datafusion/common/src/error.rs +++ b/datafusion/common/src/error.rs @@ -564,18 +564,16 @@ mod test { assert_eq!( err.split(DataFusionError::BACK_TRACE_SEP) .collect::>() - .get(0) + .first() .unwrap(), &"Error during planning: Err" ); - assert!( - err.split(DataFusionError::BACK_TRACE_SEP) - .collect::>() - .get(1) - .unwrap() - .len() - > 0 - ); + assert!(!err + .split(DataFusionError::BACK_TRACE_SEP) + .collect::>() + .get(1) + .unwrap() + .is_empty()); } #[cfg(not(feature = "backtrace"))] diff --git a/datafusion/common/src/utils.rs b/datafusion/common/src/utils.rs index 7f2dc61c07bf..9094ecd06361 100644 --- a/datafusion/common/src/utils.rs +++ b/datafusion/common/src/utils.rs @@ -135,7 +135,7 @@ pub fn bisect( ) -> Result { let low: usize = 0; let high: usize = item_columns - .get(0) + .first() .ok_or_else(|| { DataFusionError::Internal("Column array shouldn't be empty".to_string()) })? @@ -186,7 +186,7 @@ pub fn linear_search( ) -> Result { let low: usize = 0; let high: usize = item_columns - .get(0) + .first() .ok_or_else(|| { DataFusionError::Internal("Column array shouldn't be empty".to_string()) })? diff --git a/datafusion/core/benches/sort_limit_query_sql.rs b/datafusion/core/benches/sort_limit_query_sql.rs index efed5a04e7a5..cfd4b8bc4bba 100644 --- a/datafusion/core/benches/sort_limit_query_sql.rs +++ b/datafusion/core/benches/sort_limit_query_sql.rs @@ -99,7 +99,7 @@ fn create_context() -> Arc> { ctx_holder.lock().push(Arc::new(Mutex::new(ctx))) }); - let ctx = ctx_holder.lock().get(0).unwrap().clone(); + let ctx = ctx_holder.lock().first().unwrap().clone(); ctx } diff --git a/datafusion/core/benches/sql_query_with_io.rs b/datafusion/core/benches/sql_query_with_io.rs index 1f9b4dc6ccf7..c7a838385bd6 100644 --- a/datafusion/core/benches/sql_query_with_io.rs +++ b/datafusion/core/benches/sql_query_with_io.rs @@ -93,10 +93,9 @@ async fn setup_files(store: Arc) { for partition in 0..TABLE_PARTITIONS { for file in 0..PARTITION_FILES { let data = create_parquet_file(&mut rng, file * FILE_ROWS); - let location = Path::try_from(format!( + let location = Path::from(format!( "{table_name}/partition={partition}/{file}.parquet" - )) - .unwrap(); + )); store.put(&location, data).await.unwrap(); } } diff --git a/datafusion/core/src/datasource/file_format/parquet.rs b/datafusion/core/src/datasource/file_format/parquet.rs index cf6b87408107..09e54558f12e 100644 --- a/datafusion/core/src/datasource/file_format/parquet.rs +++ b/datafusion/core/src/datasource/file_format/parquet.rs @@ -1803,8 +1803,8 @@ mod tests { // there is only one row group in one file. assert_eq!(page_index.len(), 1); assert_eq!(offset_index.len(), 1); - let page_index = page_index.get(0).unwrap(); - let offset_index = offset_index.get(0).unwrap(); + let page_index = page_index.first().unwrap(); + let offset_index = offset_index.first().unwrap(); // 13 col in one row group assert_eq!(page_index.len(), 13); diff --git a/datafusion/core/src/datasource/file_format/write/demux.rs b/datafusion/core/src/datasource/file_format/write/demux.rs index 27c65dd459ec..fa4ed8437015 100644 --- a/datafusion/core/src/datasource/file_format/write/demux.rs +++ b/datafusion/core/src/datasource/file_format/write/demux.rs @@ -264,12 +264,9 @@ async fn hive_style_partitions_demuxer( // TODO: upstream RecordBatch::take to arrow-rs let take_indices = builder.finish(); let struct_array: StructArray = rb.clone().into(); - let parted_batch = RecordBatch::try_from( + let parted_batch = RecordBatch::from( arrow::compute::take(&struct_array, &take_indices, None)?.as_struct(), - ) - .map_err(|_| { - DataFusionError::Internal("Unexpected error partitioning batch!".into()) - })?; + ); // Get or create channel for this batch let part_tx = match value_map.get_mut(&part_key) { diff --git a/datafusion/core/src/datasource/listing/table.rs b/datafusion/core/src/datasource/listing/table.rs index effeacc4804f..a7f69a1d3cc8 100644 --- a/datafusion/core/src/datasource/listing/table.rs +++ b/datafusion/core/src/datasource/listing/table.rs @@ -157,7 +157,7 @@ impl ListingTableConfig { /// Infer `ListingOptions` based on `table_path` suffix. pub async fn infer_options(self, state: &SessionState) -> Result { - let store = if let Some(url) = self.table_paths.get(0) { + let store = if let Some(url) = self.table_paths.first() { state.runtime_env().object_store(url)? } else { return Ok(self); @@ -165,7 +165,7 @@ impl ListingTableConfig { let file = self .table_paths - .get(0) + .first() .unwrap() .list_all_files(state, store.as_ref(), "") .await? @@ -191,7 +191,7 @@ impl ListingTableConfig { pub async fn infer_schema(self, state: &SessionState) -> Result { match self.options { Some(options) => { - let schema = if let Some(url) = self.table_paths.get(0) { + let schema = if let Some(url) = self.table_paths.first() { options.infer_schema(state, url).await? } else { Arc::new(Schema::empty()) @@ -710,7 +710,7 @@ impl TableProvider for ListingTable { None }; - let object_store_url = if let Some(url) = self.table_paths.get(0) { + let object_store_url = if let Some(url) = self.table_paths.first() { url.object_store() } else { return Ok(Arc::new(EmptyExec::new(false, Arc::new(Schema::empty())))); @@ -835,7 +835,7 @@ impl TableProvider for ListingTable { // Multiple sort orders in outer vec are equivalent, so we pass only the first one let ordering = self .try_create_output_ordering()? - .get(0) + .first() .ok_or(DataFusionError::Internal( "Expected ListingTable to have a sort order, but none found!".into(), ))? @@ -872,7 +872,7 @@ impl ListingTable { filters: &'a [Expr], limit: Option, ) -> Result<(Vec>, Statistics)> { - let store = if let Some(url) = self.table_paths.get(0) { + let store = if let Some(url) = self.table_paths.first() { ctx.runtime_env().object_store(url)? } else { return Ok((vec![], Statistics::new_unknown(&self.file_schema))); diff --git a/datafusion/core/src/datasource/physical_plan/json.rs b/datafusion/core/src/datasource/physical_plan/json.rs index 73dcb32ac81f..9c3b523a652c 100644 --- a/datafusion/core/src/datasource/physical_plan/json.rs +++ b/datafusion/core/src/datasource/physical_plan/json.rs @@ -357,9 +357,9 @@ mod tests { ) .unwrap(); let meta = file_groups - .get(0) + .first() .unwrap() - .get(0) + .first() .unwrap() .clone() .object_meta; @@ -391,9 +391,9 @@ mod tests { ) .unwrap(); let path = file_groups - .get(0) + .first() .unwrap() - .get(0) + .first() .unwrap() .object_meta .location diff --git a/datafusion/core/src/datasource/physical_plan/mod.rs b/datafusion/core/src/datasource/physical_plan/mod.rs index 4cf115d03a9b..14e550eab1d5 100644 --- a/datafusion/core/src/datasource/physical_plan/mod.rs +++ b/datafusion/core/src/datasource/physical_plan/mod.rs @@ -135,7 +135,7 @@ impl DisplayAs for FileScanConfig { write!(f, ", infinite_source=true")?; } - if let Some(ordering) = orderings.get(0) { + if let Some(ordering) = orderings.first() { if !ordering.is_empty() { let start = if orderings.len() == 1 { ", output_ordering=" diff --git a/datafusion/core/src/physical_planner.rs b/datafusion/core/src/physical_planner.rs index 65a2e4e0a4f3..38532002a634 100644 --- a/datafusion/core/src/physical_planner.rs +++ b/datafusion/core/src/physical_planner.rs @@ -2012,7 +2012,7 @@ impl DefaultPhysicalPlanner { let mut column_names = StringBuilder::new(); let mut data_types = StringBuilder::new(); let mut is_nullables = StringBuilder::new(); - for (_, field) in table_schema.fields().iter().enumerate() { + for field in table_schema.fields() { column_names.append_value(field.name()); // "System supplied type" --> Use debug format of the datatype diff --git a/datafusion/core/tests/parquet/file_statistics.rs b/datafusion/core/tests/parquet/file_statistics.rs index 1ea154303d69..9f94a59a3e59 100644 --- a/datafusion/core/tests/parquet/file_statistics.rs +++ b/datafusion/core/tests/parquet/file_statistics.rs @@ -133,7 +133,7 @@ async fn list_files_with_session_level_cache() { assert_eq!(get_list_file_cache_size(&state1), 1); let fg = &parquet1.base_config().file_groups; assert_eq!(fg.len(), 1); - assert_eq!(fg.get(0).unwrap().len(), 1); + assert_eq!(fg.first().unwrap().len(), 1); //Session 2 first time list files //check session 1 cache result not show in session 2 @@ -144,7 +144,7 @@ async fn list_files_with_session_level_cache() { assert_eq!(get_list_file_cache_size(&state2), 1); let fg2 = &parquet2.base_config().file_groups; assert_eq!(fg2.len(), 1); - assert_eq!(fg2.get(0).unwrap().len(), 1); + assert_eq!(fg2.first().unwrap().len(), 1); //Session 1 second time list files //check session 1 cache result not show in session 2 @@ -155,7 +155,7 @@ async fn list_files_with_session_level_cache() { assert_eq!(get_list_file_cache_size(&state1), 1); let fg = &parquet3.base_config().file_groups; assert_eq!(fg.len(), 1); - assert_eq!(fg.get(0).unwrap().len(), 1); + assert_eq!(fg.first().unwrap().len(), 1); // List same file no increase assert_eq!(get_list_file_cache_size(&state1), 1); } diff --git a/datafusion/core/tests/sql/mod.rs b/datafusion/core/tests/sql/mod.rs index 47de6ec857da..94fc8015a78a 100644 --- a/datafusion/core/tests/sql/mod.rs +++ b/datafusion/core/tests/sql/mod.rs @@ -15,7 +15,6 @@ // specific language governing permissions and limitations // under the License. -use std::convert::TryFrom; use std::sync::Arc; use arrow::{ diff --git a/datafusion/core/tests/sql/parquet.rs b/datafusion/core/tests/sql/parquet.rs index c2844a2b762a..8f810a929df3 100644 --- a/datafusion/core/tests/sql/parquet.rs +++ b/datafusion/core/tests/sql/parquet.rs @@ -263,7 +263,7 @@ async fn parquet_list_columns() { assert_eq!( as_string_array(&utf8_list_array.value(0)).unwrap(), - &StringArray::try_from(vec![Some("abc"), Some("efg"), Some("hij"),]).unwrap() + &StringArray::from(vec![Some("abc"), Some("efg"), Some("hij"),]) ); assert_eq!( diff --git a/datafusion/execution/src/cache/cache_unit.rs b/datafusion/execution/src/cache/cache_unit.rs index c54839061c8a..25f9b9fa4d68 100644 --- a/datafusion/execution/src/cache/cache_unit.rs +++ b/datafusion/execution/src/cache/cache_unit.rs @@ -228,7 +228,7 @@ mod tests { cache.put(&meta.location, vec![meta.clone()].into()); assert_eq!( - cache.get(&meta.location).unwrap().get(0).unwrap().clone(), + cache.get(&meta.location).unwrap().first().unwrap().clone(), meta.clone() ); } diff --git a/datafusion/expr/src/utils.rs b/datafusion/expr/src/utils.rs index 7d126a0f3373..c30c734fcf1f 100644 --- a/datafusion/expr/src/utils.rs +++ b/datafusion/expr/src/utils.rs @@ -501,7 +501,6 @@ pub fn generate_sort_key( let res = final_sort_keys .into_iter() .zip(is_partition_flag) - .map(|(lhs, rhs)| (lhs, rhs)) .collect::>(); Ok(res) } diff --git a/datafusion/optimizer/src/eliminate_limit.rs b/datafusion/optimizer/src/eliminate_limit.rs index 7844ca7909fc..4386253740aa 100644 --- a/datafusion/optimizer/src/eliminate_limit.rs +++ b/datafusion/optimizer/src/eliminate_limit.rs @@ -97,7 +97,7 @@ mod tests { let optimizer = Optimizer::with_rules(vec![Arc::new(EliminateLimit::new())]); let optimized_plan = optimizer .optimize_recursively( - optimizer.rules.get(0).unwrap(), + optimizer.rules.first().unwrap(), plan, &OptimizerContext::new(), )? diff --git a/datafusion/optimizer/src/push_down_filter.rs b/datafusion/optimizer/src/push_down_filter.rs index e8f116d89466..c090fb849a82 100644 --- a/datafusion/optimizer/src/push_down_filter.rs +++ b/datafusion/optimizer/src/push_down_filter.rs @@ -1062,7 +1062,7 @@ mod tests { ]); let mut optimized_plan = optimizer .optimize_recursively( - optimizer.rules.get(0).unwrap(), + optimizer.rules.first().unwrap(), plan, &OptimizerContext::new(), )? diff --git a/datafusion/optimizer/src/push_down_projection.rs b/datafusion/optimizer/src/push_down_projection.rs index bdd66347631c..10cc1879aeeb 100644 --- a/datafusion/optimizer/src/push_down_projection.rs +++ b/datafusion/optimizer/src/push_down_projection.rs @@ -625,7 +625,7 @@ mod tests { let optimizer = Optimizer::with_rules(vec![Arc::new(OptimizeProjections::new())]); let optimized_plan = optimizer .optimize_recursively( - optimizer.rules.get(0).unwrap(), + optimizer.rules.first().unwrap(), plan, &OptimizerContext::new(), )? diff --git a/datafusion/optimizer/src/test/mod.rs b/datafusion/optimizer/src/test/mod.rs index 917ddc565c9e..e691fe9a5351 100644 --- a/datafusion/optimizer/src/test/mod.rs +++ b/datafusion/optimizer/src/test/mod.rs @@ -158,7 +158,7 @@ pub fn assert_optimized_plan_eq( let optimizer = Optimizer::with_rules(vec![rule.clone()]); let optimized_plan = optimizer .optimize_recursively( - optimizer.rules.get(0).unwrap(), + optimizer.rules.first().unwrap(), plan, &OptimizerContext::new(), )? @@ -199,7 +199,7 @@ pub fn assert_optimized_plan_eq_display_indent( let optimizer = Optimizer::with_rules(vec![rule]); let optimized_plan = optimizer .optimize_recursively( - optimizer.rules.get(0).unwrap(), + optimizer.rules.first().unwrap(), plan, &OptimizerContext::new(), ) @@ -233,7 +233,7 @@ pub fn assert_optimizer_err( ) { let optimizer = Optimizer::with_rules(vec![rule]); let res = optimizer.optimize_recursively( - optimizer.rules.get(0).unwrap(), + optimizer.rules.first().unwrap(), plan, &OptimizerContext::new(), ); @@ -255,7 +255,7 @@ pub fn assert_optimization_skipped( let optimizer = Optimizer::with_rules(vec![rule]); let new_plan = optimizer .optimize_recursively( - optimizer.rules.get(0).unwrap(), + optimizer.rules.first().unwrap(), plan, &OptimizerContext::new(), )? diff --git a/datafusion/physical-expr/src/aggregate/groups_accumulator/adapter.rs b/datafusion/physical-expr/src/aggregate/groups_accumulator/adapter.rs index dcc8c37e7484..cf980f4c3f16 100644 --- a/datafusion/physical-expr/src/aggregate/groups_accumulator/adapter.rs +++ b/datafusion/physical-expr/src/aggregate/groups_accumulator/adapter.rs @@ -309,7 +309,7 @@ impl GroupsAccumulator for GroupsAccumulatorAdapter { // double check each array has the same length (aka the // accumulator was implemented correctly - if let Some(first_col) = arrays.get(0) { + if let Some(first_col) = arrays.first() { for arr in &arrays { assert_eq!(arr.len(), first_col.len()) } diff --git a/datafusion/physical-expr/src/array_expressions.rs b/datafusion/physical-expr/src/array_expressions.rs index 269bbf7dcf10..08df3ef9f613 100644 --- a/datafusion/physical-expr/src/array_expressions.rs +++ b/datafusion/physical-expr/src/array_expressions.rs @@ -537,7 +537,7 @@ fn general_except( dedup.clear(); } - if let Some(values) = converter.convert_rows(rows)?.get(0) { + if let Some(values) = converter.convert_rows(rows)?.first() { Ok(GenericListArray::::new( field.to_owned(), OffsetBuffer::new(offsets.into()), @@ -2088,7 +2088,7 @@ pub fn array_intersect(args: &[ArrayRef]) -> Result { }; offsets.push(last_offset + rows.len() as i32); let arrays = converter.convert_rows(rows)?; - let array = match arrays.get(0) { + let array = match arrays.first() { Some(array) => array.clone(), None => { return internal_err!( diff --git a/datafusion/physical-plan/src/memory.rs b/datafusion/physical-plan/src/memory.rs index 39cd47452eff..7de474fda11c 100644 --- a/datafusion/physical-plan/src/memory.rs +++ b/datafusion/physical-plan/src/memory.rs @@ -55,7 +55,7 @@ impl fmt::Debug for MemoryExec { write!(f, "partitions: [...]")?; write!(f, "schema: {:?}", self.projected_schema)?; write!(f, "projection: {:?}", self.projection)?; - if let Some(sort_info) = &self.sort_information.get(0) { + if let Some(sort_info) = &self.sort_information.first() { write!(f, ", output_ordering: {:?}", sort_info)?; } Ok(()) diff --git a/datafusion/physical-plan/src/test/exec.rs b/datafusion/physical-plan/src/test/exec.rs index 71e6cba6741e..1f6ee1f117aa 100644 --- a/datafusion/physical-plan/src/test/exec.rs +++ b/datafusion/physical-plan/src/test/exec.rs @@ -790,7 +790,7 @@ impl Stream for PanicStream { } else { self.ready = true; // get called again - cx.waker().clone().wake(); + cx.waker().wake_by_ref(); return Poll::Pending; } } diff --git a/datafusion/sql/src/relation/mod.rs b/datafusion/sql/src/relation/mod.rs index 6fc7e9601243..b233f47a058f 100644 --- a/datafusion/sql/src/relation/mod.rs +++ b/datafusion/sql/src/relation/mod.rs @@ -36,7 +36,7 @@ impl<'a, S: ContextProvider> SqlToRel<'a, S> { name, alias, args, .. } => { if let Some(func_args) = args { - let tbl_func_name = name.0.get(0).unwrap().value.to_string(); + let tbl_func_name = name.0.first().unwrap().value.to_string(); let args = func_args .into_iter() .flat_map(|arg| { From 205e315ed3eafbb016ffc5ac62a3be07734a8885 Mon Sep 17 00:00:00 2001 From: Liang-Chi Hsieh Date: Fri, 8 Dec 2023 01:37:03 -0800 Subject: [PATCH 387/572] fix: RANGE frame for corner cases with empty ORDER BY clause should be treated as constant sort (#8445) * fix: RANGE frame for corner cases with empty ORDER BY clause should be treated as constant sort * fix * Make the test not flaky * fix clippy --- datafusion/expr/src/window_frame.rs | 56 ++++++++++++------- .../proto/src/logical_plan/from_proto.rs | 8 ++- datafusion/sql/src/expr/function.rs | 10 +++- datafusion/sqllogictest/test_files/window.slt | 16 +++--- 4 files changed, 56 insertions(+), 34 deletions(-) diff --git a/datafusion/expr/src/window_frame.rs b/datafusion/expr/src/window_frame.rs index 2a64f21b856b..2701ca1ecf3b 100644 --- a/datafusion/expr/src/window_frame.rs +++ b/datafusion/expr/src/window_frame.rs @@ -23,6 +23,8 @@ //! - An ending frame boundary, //! - An EXCLUDE clause. +use crate::expr::Sort; +use crate::Expr; use datafusion_common::{plan_err, sql_err, DataFusionError, Result, ScalarValue}; use sqlparser::ast; use sqlparser::parser::ParserError::ParserError; @@ -142,41 +144,57 @@ impl WindowFrame { } } -/// Construct equivalent explicit window frames for implicit corner cases. -/// With this processing, we may assume in downstream code that RANGE/GROUPS -/// frames contain an appropriate ORDER BY clause. -pub fn regularize(mut frame: WindowFrame, order_bys: usize) -> Result { - if frame.units == WindowFrameUnits::Range && order_bys != 1 { +/// Regularizes ORDER BY clause for window definition for implicit corner cases. +pub fn regularize_window_order_by( + frame: &WindowFrame, + order_by: &mut Vec, +) -> Result<()> { + if frame.units == WindowFrameUnits::Range && order_by.len() != 1 { // Normally, RANGE frames require an ORDER BY clause with exactly one // column. However, an ORDER BY clause may be absent or present but with // more than one column in two edge cases: // 1. start bound is UNBOUNDED or CURRENT ROW // 2. end bound is CURRENT ROW or UNBOUNDED. - // In these cases, we regularize the RANGE frame to be equivalent to a ROWS - // frame with the UNBOUNDED bounds. - // Note that this follows Postgres behavior. + // In these cases, we regularize the ORDER BY clause if the ORDER BY clause + // is absent. If an ORDER BY clause is present but has more than one column, + // the ORDER BY clause is unchanged. Note that this follows Postgres behavior. if (frame.start_bound.is_unbounded() || frame.start_bound == WindowFrameBound::CurrentRow) && (frame.end_bound == WindowFrameBound::CurrentRow || frame.end_bound.is_unbounded()) { - // If an ORDER BY clause is absent, the frame is equivalent to a ROWS - // frame with the UNBOUNDED bounds. - // If an ORDER BY clause is present but has more than one column, the - // frame is unchanged. - if order_bys == 0 { - frame.units = WindowFrameUnits::Rows; - frame.start_bound = - WindowFrameBound::Preceding(ScalarValue::UInt64(None)); - frame.end_bound = WindowFrameBound::Following(ScalarValue::UInt64(None)); + // If an ORDER BY clause is absent, it is equivalent to a ORDER BY clause + // with constant value as sort key. + // If an ORDER BY clause is present but has more than one column, it is + // unchanged. + if order_by.is_empty() { + order_by.push(Expr::Sort(Sort::new( + Box::new(Expr::Literal(ScalarValue::UInt64(Some(1)))), + true, + false, + ))); } - } else { + } + } + Ok(()) +} + +/// Checks if given window frame is valid. In particular, if the frame is RANGE +/// with offset PRECEDING/FOLLOWING, it must have exactly one ORDER BY column. +pub fn check_window_frame(frame: &WindowFrame, order_bys: usize) -> Result<()> { + if frame.units == WindowFrameUnits::Range && order_bys != 1 { + // See `regularize_window_order_by`. + if !(frame.start_bound.is_unbounded() + || frame.start_bound == WindowFrameBound::CurrentRow) + || !(frame.end_bound == WindowFrameBound::CurrentRow + || frame.end_bound.is_unbounded()) + { plan_err!("RANGE requires exactly one ORDER BY column")? } } else if frame.units == WindowFrameUnits::Groups && order_bys == 0 { plan_err!("GROUPS requires an ORDER BY clause")? }; - Ok(frame) + Ok(()) } /// There are five ways to describe starting and ending frame boundaries: diff --git a/datafusion/proto/src/logical_plan/from_proto.rs b/datafusion/proto/src/logical_plan/from_proto.rs index 13576aaa089a..22a3ed804a5c 100644 --- a/datafusion/proto/src/logical_plan/from_proto.rs +++ b/datafusion/proto/src/logical_plan/from_proto.rs @@ -39,6 +39,7 @@ use datafusion_common::{ internal_err, plan_datafusion_err, Column, Constraint, Constraints, DFField, DFSchema, DFSchemaRef, DataFusionError, OwnedTableReference, Result, ScalarValue, }; +use datafusion_expr::window_frame::{check_window_frame, regularize_window_order_by}; use datafusion_expr::{ abs, acos, acosh, array, array_append, array_concat, array_dims, array_element, array_except, array_has, array_has_all, array_has_any, array_intersect, array_length, @@ -59,7 +60,6 @@ use datafusion_expr::{ sqrt, starts_with, string_to_array, strpos, struct_fun, substr, substr_index, substring, tan, tanh, to_hex, to_timestamp_micros, to_timestamp_millis, to_timestamp_nanos, to_timestamp_seconds, translate, trim, trunc, upper, uuid, - window_frame::regularize, AggregateFunction, Between, BinaryExpr, BuiltInWindowFunction, BuiltinScalarFunction, Case, Cast, Expr, GetFieldAccess, GetIndexedField, GroupingSet, GroupingSet::GroupingSets, @@ -1072,7 +1072,7 @@ pub fn parse_expr( .iter() .map(|e| parse_expr(e, registry)) .collect::, _>>()?; - let order_by = expr + let mut order_by = expr .order_by .iter() .map(|e| parse_expr(e, registry)) @@ -1082,7 +1082,8 @@ pub fn parse_expr( .as_ref() .map::, _>(|window_frame| { let window_frame = window_frame.clone().try_into()?; - regularize(window_frame, order_by.len()) + check_window_frame(&window_frame, order_by.len()) + .map(|_| window_frame) }) .transpose()? .ok_or_else(|| { @@ -1090,6 +1091,7 @@ pub fn parse_expr( "missing window frame during deserialization".to_string(), ) })?; + regularize_window_order_by(&window_frame, &mut order_by)?; match window_function { window_expr_node::WindowFunction::AggrFunction(i) => { diff --git a/datafusion/sql/src/expr/function.rs b/datafusion/sql/src/expr/function.rs index 14ea20c3fa5f..73de4fa43907 100644 --- a/datafusion/sql/src/expr/function.rs +++ b/datafusion/sql/src/expr/function.rs @@ -21,7 +21,7 @@ use datafusion_common::{ }; use datafusion_expr::expr::ScalarFunction; use datafusion_expr::function::suggest_valid_function; -use datafusion_expr::window_frame::regularize; +use datafusion_expr::window_frame::{check_window_frame, regularize_window_order_by}; use datafusion_expr::{ expr, window_function, AggregateFunction, BuiltinScalarFunction, Expr, WindowFrame, WindowFunction, @@ -92,7 +92,7 @@ impl<'a, S: ContextProvider> SqlToRel<'a, S> { .into_iter() .map(|e| self.sql_expr_to_logical_expr(e, schema, planner_context)) .collect::>>()?; - let order_by = self.order_by_to_sort_expr( + let mut order_by = self.order_by_to_sort_expr( &window.order_by, schema, planner_context, @@ -104,14 +104,18 @@ impl<'a, S: ContextProvider> SqlToRel<'a, S> { .as_ref() .map(|window_frame| { let window_frame = window_frame.clone().try_into()?; - regularize(window_frame, order_by.len()) + check_window_frame(&window_frame, order_by.len()) + .map(|_| window_frame) }) .transpose()?; + let window_frame = if let Some(window_frame) = window_frame { + regularize_window_order_by(&window_frame, &mut order_by)?; window_frame } else { WindowFrame::new(!order_by.is_empty()) }; + if let Ok(fun) = self.find_window_func(&name) { let expr = match fun { WindowFunction::AggregateFunction(aggregate_fun) => { diff --git a/datafusion/sqllogictest/test_files/window.slt b/datafusion/sqllogictest/test_files/window.slt index 5b69ead0ff36..b660a9a0c2ae 100644 --- a/datafusion/sqllogictest/test_files/window.slt +++ b/datafusion/sqllogictest/test_files/window.slt @@ -3763,15 +3763,13 @@ select a, 1 1 2 2 -# TODO: this is different to Postgres which returns [1, 1] for `rnk`. -# Comment it because it is flaky now as it depends on the order of the `a` column. -# query II -# select a, -# rank() over (RANGE BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING) rnk -# from (select 1 a union select 2 a) q ORDER BY rnk -# ---- -# 1 1 -# 2 2 +query II +select a, + rank() over (RANGE BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING) rnk + from (select 1 a union select 2 a) q ORDER BY a +---- +1 1 +2 1 # TODO: this works in Postgres which returns [1, 1]. query error DataFusion error: Arrow error: Invalid argument error: must either specify a row count or at least one column From a8d74a7b141a63430f10c313c55aad059d81ecb5 Mon Sep 17 00:00:00 2001 From: Brent Gardner Date: Fri, 8 Dec 2023 04:10:39 -0700 Subject: [PATCH 388/572] Preserve `dict_id` on `Field` during serde roundtrip (#8457) * Failing test * Passing test --- datafusion/proto/proto/datafusion.proto | 2 + datafusion/proto/src/generated/pbjson.rs | 39 +++++++++++++++++++ datafusion/proto/src/generated/prost.rs | 4 ++ .../proto/src/logical_plan/from_proto.rs | 16 +++++++- datafusion/proto/src/logical_plan/to_proto.rs | 2 + .../tests/cases/roundtrip_logical_plan.rs | 39 +++++++++++++++++++ 6 files changed, 100 insertions(+), 2 deletions(-) diff --git a/datafusion/proto/proto/datafusion.proto b/datafusion/proto/proto/datafusion.proto index 863e3c315c82..de7afd5c7bb2 100644 --- a/datafusion/proto/proto/datafusion.proto +++ b/datafusion/proto/proto/datafusion.proto @@ -840,6 +840,8 @@ message Field { // for complex data types like structs, unions repeated Field children = 4; map metadata = 5; + int64 dict_id = 6; + bool dict_ordered = 7; } message FixedSizeBinary{ diff --git a/datafusion/proto/src/generated/pbjson.rs b/datafusion/proto/src/generated/pbjson.rs index 74798ee8e94c..3001a9c09503 100644 --- a/datafusion/proto/src/generated/pbjson.rs +++ b/datafusion/proto/src/generated/pbjson.rs @@ -6910,6 +6910,12 @@ impl serde::Serialize for Field { if !self.metadata.is_empty() { len += 1; } + if self.dict_id != 0 { + len += 1; + } + if self.dict_ordered { + len += 1; + } let mut struct_ser = serializer.serialize_struct("datafusion.Field", len)?; if !self.name.is_empty() { struct_ser.serialize_field("name", &self.name)?; @@ -6926,6 +6932,13 @@ impl serde::Serialize for Field { if !self.metadata.is_empty() { struct_ser.serialize_field("metadata", &self.metadata)?; } + if self.dict_id != 0 { + #[allow(clippy::needless_borrow)] + struct_ser.serialize_field("dictId", ToString::to_string(&self.dict_id).as_str())?; + } + if self.dict_ordered { + struct_ser.serialize_field("dictOrdered", &self.dict_ordered)?; + } struct_ser.end() } } @@ -6942,6 +6955,10 @@ impl<'de> serde::Deserialize<'de> for Field { "nullable", "children", "metadata", + "dict_id", + "dictId", + "dict_ordered", + "dictOrdered", ]; #[allow(clippy::enum_variant_names)] @@ -6951,6 +6968,8 @@ impl<'de> serde::Deserialize<'de> for Field { Nullable, Children, Metadata, + DictId, + DictOrdered, } impl<'de> serde::Deserialize<'de> for GeneratedField { fn deserialize(deserializer: D) -> std::result::Result @@ -6977,6 +6996,8 @@ impl<'de> serde::Deserialize<'de> for Field { "nullable" => Ok(GeneratedField::Nullable), "children" => Ok(GeneratedField::Children), "metadata" => Ok(GeneratedField::Metadata), + "dictId" | "dict_id" => Ok(GeneratedField::DictId), + "dictOrdered" | "dict_ordered" => Ok(GeneratedField::DictOrdered), _ => Err(serde::de::Error::unknown_field(value, FIELDS)), } } @@ -7001,6 +7022,8 @@ impl<'de> serde::Deserialize<'de> for Field { let mut nullable__ = None; let mut children__ = None; let mut metadata__ = None; + let mut dict_id__ = None; + let mut dict_ordered__ = None; while let Some(k) = map_.next_key()? { match k { GeneratedField::Name => { @@ -7035,6 +7058,20 @@ impl<'de> serde::Deserialize<'de> for Field { map_.next_value::>()? ); } + GeneratedField::DictId => { + if dict_id__.is_some() { + return Err(serde::de::Error::duplicate_field("dictId")); + } + dict_id__ = + Some(map_.next_value::<::pbjson::private::NumberDeserialize<_>>()?.0) + ; + } + GeneratedField::DictOrdered => { + if dict_ordered__.is_some() { + return Err(serde::de::Error::duplicate_field("dictOrdered")); + } + dict_ordered__ = Some(map_.next_value()?); + } } } Ok(Field { @@ -7043,6 +7080,8 @@ impl<'de> serde::Deserialize<'de> for Field { nullable: nullable__.unwrap_or_default(), children: children__.unwrap_or_default(), metadata: metadata__.unwrap_or_default(), + dict_id: dict_id__.unwrap_or_default(), + dict_ordered: dict_ordered__.unwrap_or_default(), }) } } diff --git a/datafusion/proto/src/generated/prost.rs b/datafusion/proto/src/generated/prost.rs index ae20913e3dd7..cfa424e444f8 100644 --- a/datafusion/proto/src/generated/prost.rs +++ b/datafusion/proto/src/generated/prost.rs @@ -1027,6 +1027,10 @@ pub struct Field { ::prost::alloc::string::String, ::prost::alloc::string::String, >, + #[prost(int64, tag = "6")] + pub dict_id: i64, + #[prost(bool, tag = "7")] + pub dict_ordered: bool, } #[allow(clippy::derive_partial_eq_without_eq)] #[derive(Clone, PartialEq, ::prost::Message)] diff --git a/datafusion/proto/src/logical_plan/from_proto.rs b/datafusion/proto/src/logical_plan/from_proto.rs index 22a3ed804a5c..7daab47837d6 100644 --- a/datafusion/proto/src/logical_plan/from_proto.rs +++ b/datafusion/proto/src/logical_plan/from_proto.rs @@ -377,8 +377,20 @@ impl TryFrom<&protobuf::Field> for Field { type Error = Error; fn try_from(field: &protobuf::Field) -> Result { let datatype = field.arrow_type.as_deref().required("arrow_type")?; - Ok(Self::new(field.name.as_str(), datatype, field.nullable) - .with_metadata(field.metadata.clone())) + let field = if field.dict_id != 0 { + Self::new_dict( + field.name.as_str(), + datatype, + field.nullable, + field.dict_id, + field.dict_ordered, + ) + .with_metadata(field.metadata.clone()) + } else { + Self::new(field.name.as_str(), datatype, field.nullable) + .with_metadata(field.metadata.clone()) + }; + Ok(field) } } diff --git a/datafusion/proto/src/logical_plan/to_proto.rs b/datafusion/proto/src/logical_plan/to_proto.rs index 0af8d9f3e719..4c6fdaa894ae 100644 --- a/datafusion/proto/src/logical_plan/to_proto.rs +++ b/datafusion/proto/src/logical_plan/to_proto.rs @@ -108,6 +108,8 @@ impl TryFrom<&Field> for protobuf::Field { nullable: field.is_nullable(), children: Vec::new(), metadata: field.metadata().clone(), + dict_id: field.dict_id().unwrap_or(0), + dict_ordered: field.dict_is_ordered().unwrap_or(false), }) } } diff --git a/datafusion/proto/tests/cases/roundtrip_logical_plan.rs b/datafusion/proto/tests/cases/roundtrip_logical_plan.rs index 5e36a838f311..8e15b5d0d480 100644 --- a/datafusion/proto/tests/cases/roundtrip_logical_plan.rs +++ b/datafusion/proto/tests/cases/roundtrip_logical_plan.rs @@ -972,6 +972,45 @@ fn round_trip_datatype() { } } +#[test] +fn roundtrip_dict_id() -> Result<()> { + let dict_id = 42; + let field = Field::new( + "keys", + DataType::List(Arc::new(Field::new_dict( + "item", + DataType::Dictionary(Box::new(DataType::UInt16), Box::new(DataType::Utf8)), + true, + dict_id, + false, + ))), + false, + ); + let schema = Arc::new(Schema::new(vec![field])); + + // encode + let mut buf: Vec = vec![]; + let schema_proto: datafusion_proto::generated::datafusion::Schema = + schema.try_into().unwrap(); + schema_proto.encode(&mut buf).unwrap(); + + // decode + let schema_proto = + datafusion_proto::generated::datafusion::Schema::decode(buf.as_slice()).unwrap(); + let decoded: Schema = (&schema_proto).try_into()?; + + // assert + let keys = decoded.fields().iter().last().unwrap(); + match keys.data_type() { + DataType::List(field) => { + assert_eq!(field.dict_id(), Some(dict_id), "dict_id should be retained"); + } + _ => panic!("Invalid type"), + } + + Ok(()) +} + #[test] fn roundtrip_null_scalar_values() { let test_types = vec![ From e2986f135890cf5d259d857316bb95574e904cf0 Mon Sep 17 00:00:00 2001 From: Kun Liu Date: Fri, 8 Dec 2023 19:37:13 +0800 Subject: [PATCH 389/572] support inter leave node (#8460) --- datafusion/proto/proto/datafusion.proto | 5 + datafusion/proto/src/generated/pbjson.rs | 104 ++++++++++++++++++ datafusion/proto/src/generated/prost.rs | 10 +- datafusion/proto/src/physical_plan/mod.rs | 44 ++++++-- .../tests/cases/roundtrip_physical_plan.rs | 65 ++++++++--- 5 files changed, 202 insertions(+), 26 deletions(-) diff --git a/datafusion/proto/proto/datafusion.proto b/datafusion/proto/proto/datafusion.proto index de7afd5c7bb2..55fb08042399 100644 --- a/datafusion/proto/proto/datafusion.proto +++ b/datafusion/proto/proto/datafusion.proto @@ -1163,6 +1163,7 @@ message PhysicalPlanNode { AnalyzeExecNode analyze = 23; JsonSinkExecNode json_sink = 24; SymmetricHashJoinExecNode symmetric_hash_join = 25; + InterleaveExecNode interleave = 26; } } @@ -1456,6 +1457,10 @@ message SymmetricHashJoinExecNode { JoinFilter filter = 8; } +message InterleaveExecNode { + repeated PhysicalPlanNode inputs = 1; +} + message UnionExecNode { repeated PhysicalPlanNode inputs = 1; } diff --git a/datafusion/proto/src/generated/pbjson.rs b/datafusion/proto/src/generated/pbjson.rs index 3001a9c09503..dea329cbea28 100644 --- a/datafusion/proto/src/generated/pbjson.rs +++ b/datafusion/proto/src/generated/pbjson.rs @@ -9244,6 +9244,97 @@ impl<'de> serde::Deserialize<'de> for InListNode { deserializer.deserialize_struct("datafusion.InListNode", FIELDS, GeneratedVisitor) } } +impl serde::Serialize for InterleaveExecNode { + #[allow(deprecated)] + fn serialize(&self, serializer: S) -> std::result::Result + where + S: serde::Serializer, + { + use serde::ser::SerializeStruct; + let mut len = 0; + if !self.inputs.is_empty() { + len += 1; + } + let mut struct_ser = serializer.serialize_struct("datafusion.InterleaveExecNode", len)?; + if !self.inputs.is_empty() { + struct_ser.serialize_field("inputs", &self.inputs)?; + } + struct_ser.end() + } +} +impl<'de> serde::Deserialize<'de> for InterleaveExecNode { + #[allow(deprecated)] + fn deserialize(deserializer: D) -> std::result::Result + where + D: serde::Deserializer<'de>, + { + const FIELDS: &[&str] = &[ + "inputs", + ]; + + #[allow(clippy::enum_variant_names)] + enum GeneratedField { + Inputs, + } + impl<'de> serde::Deserialize<'de> for GeneratedField { + fn deserialize(deserializer: D) -> std::result::Result + where + D: serde::Deserializer<'de>, + { + struct GeneratedVisitor; + + impl<'de> serde::de::Visitor<'de> for GeneratedVisitor { + type Value = GeneratedField; + + fn expecting(&self, formatter: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + write!(formatter, "expected one of: {:?}", &FIELDS) + } + + #[allow(unused_variables)] + fn visit_str(self, value: &str) -> std::result::Result + where + E: serde::de::Error, + { + match value { + "inputs" => Ok(GeneratedField::Inputs), + _ => Err(serde::de::Error::unknown_field(value, FIELDS)), + } + } + } + deserializer.deserialize_identifier(GeneratedVisitor) + } + } + struct GeneratedVisitor; + impl<'de> serde::de::Visitor<'de> for GeneratedVisitor { + type Value = InterleaveExecNode; + + fn expecting(&self, formatter: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + formatter.write_str("struct datafusion.InterleaveExecNode") + } + + fn visit_map(self, mut map_: V) -> std::result::Result + where + V: serde::de::MapAccess<'de>, + { + let mut inputs__ = None; + while let Some(k) = map_.next_key()? { + match k { + GeneratedField::Inputs => { + if inputs__.is_some() { + return Err(serde::de::Error::duplicate_field("inputs")); + } + inputs__ = Some(map_.next_value()?); + } + } + } + Ok(InterleaveExecNode { + inputs: inputs__.unwrap_or_default(), + }) + } + } + deserializer.deserialize_struct("datafusion.InterleaveExecNode", FIELDS, GeneratedVisitor) + } +} impl serde::Serialize for IntervalMonthDayNanoValue { #[allow(deprecated)] fn serialize(&self, serializer: S) -> std::result::Result @@ -17926,6 +18017,9 @@ impl serde::Serialize for PhysicalPlanNode { physical_plan_node::PhysicalPlanType::SymmetricHashJoin(v) => { struct_ser.serialize_field("symmetricHashJoin", v)?; } + physical_plan_node::PhysicalPlanType::Interleave(v) => { + struct_ser.serialize_field("interleave", v)?; + } } } struct_ser.end() @@ -17974,6 +18068,7 @@ impl<'de> serde::Deserialize<'de> for PhysicalPlanNode { "jsonSink", "symmetric_hash_join", "symmetricHashJoin", + "interleave", ]; #[allow(clippy::enum_variant_names)] @@ -18002,6 +18097,7 @@ impl<'de> serde::Deserialize<'de> for PhysicalPlanNode { Analyze, JsonSink, SymmetricHashJoin, + Interleave, } impl<'de> serde::Deserialize<'de> for GeneratedField { fn deserialize(deserializer: D) -> std::result::Result @@ -18047,6 +18143,7 @@ impl<'de> serde::Deserialize<'de> for PhysicalPlanNode { "analyze" => Ok(GeneratedField::Analyze), "jsonSink" | "json_sink" => Ok(GeneratedField::JsonSink), "symmetricHashJoin" | "symmetric_hash_join" => Ok(GeneratedField::SymmetricHashJoin), + "interleave" => Ok(GeneratedField::Interleave), _ => Err(serde::de::Error::unknown_field(value, FIELDS)), } } @@ -18235,6 +18332,13 @@ impl<'de> serde::Deserialize<'de> for PhysicalPlanNode { return Err(serde::de::Error::duplicate_field("symmetricHashJoin")); } physical_plan_type__ = map_.next_value::<::std::option::Option<_>>()?.map(physical_plan_node::PhysicalPlanType::SymmetricHashJoin) +; + } + GeneratedField::Interleave => { + if physical_plan_type__.is_some() { + return Err(serde::de::Error::duplicate_field("interleave")); + } + physical_plan_type__ = map_.next_value::<::std::option::Option<_>>()?.map(physical_plan_node::PhysicalPlanType::Interleave) ; } } diff --git a/datafusion/proto/src/generated/prost.rs b/datafusion/proto/src/generated/prost.rs index cfa424e444f8..41b94a2a3961 100644 --- a/datafusion/proto/src/generated/prost.rs +++ b/datafusion/proto/src/generated/prost.rs @@ -1525,7 +1525,7 @@ pub mod owned_table_reference { pub struct PhysicalPlanNode { #[prost( oneof = "physical_plan_node::PhysicalPlanType", - tags = "1, 2, 3, 4, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25" + tags = "1, 2, 3, 4, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26" )] pub physical_plan_type: ::core::option::Option, } @@ -1584,6 +1584,8 @@ pub mod physical_plan_node { JsonSink(::prost::alloc::boxed::Box), #[prost(message, tag = "25")] SymmetricHashJoin(::prost::alloc::boxed::Box), + #[prost(message, tag = "26")] + Interleave(super::InterleaveExecNode), } } #[allow(clippy::derive_partial_eq_without_eq)] @@ -2042,6 +2044,12 @@ pub struct SymmetricHashJoinExecNode { } #[allow(clippy::derive_partial_eq_without_eq)] #[derive(Clone, PartialEq, ::prost::Message)] +pub struct InterleaveExecNode { + #[prost(message, repeated, tag = "1")] + pub inputs: ::prost::alloc::vec::Vec, +} +#[allow(clippy::derive_partial_eq_without_eq)] +#[derive(Clone, PartialEq, ::prost::Message)] pub struct UnionExecNode { #[prost(message, repeated, tag = "1")] pub inputs: ::prost::alloc::vec::Vec, diff --git a/datafusion/proto/src/physical_plan/mod.rs b/datafusion/proto/src/physical_plan/mod.rs index 74c8ec894ff2..878a5bcb7f69 100644 --- a/datafusion/proto/src/physical_plan/mod.rs +++ b/datafusion/proto/src/physical_plan/mod.rs @@ -48,7 +48,7 @@ use datafusion::physical_plan::projection::ProjectionExec; use datafusion::physical_plan::repartition::RepartitionExec; use datafusion::physical_plan::sorts::sort::SortExec; use datafusion::physical_plan::sorts::sort_preserving_merge::SortPreservingMergeExec; -use datafusion::physical_plan::union::UnionExec; +use datafusion::physical_plan::union::{InterleaveExec, UnionExec}; use datafusion::physical_plan::windows::{BoundedWindowAggExec, WindowAggExec}; use datafusion::physical_plan::{ udaf, AggregateExpr, ExecutionPlan, InputOrderMode, Partitioning, PhysicalExpr, @@ -545,7 +545,7 @@ impl AsExecutionPlan for PhysicalPlanNode { f.expression.as_ref().ok_or_else(|| { proto_error("Unexpected empty filter expression") })?, - registry, &schema + registry, &schema, )?; let column_indices = f.column_indices .iter() @@ -556,7 +556,7 @@ impl AsExecutionPlan for PhysicalPlanNode { i.side)) )?; - Ok(ColumnIndex{ + Ok(ColumnIndex { index: i.index as usize, side: side.into(), }) @@ -634,7 +634,7 @@ impl AsExecutionPlan for PhysicalPlanNode { f.expression.as_ref().ok_or_else(|| { proto_error("Unexpected empty filter expression") })?, - registry, &schema + registry, &schema, )?; let column_indices = f.column_indices .iter() @@ -645,7 +645,7 @@ impl AsExecutionPlan for PhysicalPlanNode { i.side)) )?; - Ok(ColumnIndex{ + Ok(ColumnIndex { index: i.index as usize, side: side.into(), }) @@ -693,6 +693,17 @@ impl AsExecutionPlan for PhysicalPlanNode { } Ok(Arc::new(UnionExec::new(inputs))) } + PhysicalPlanType::Interleave(interleave) => { + let mut inputs: Vec> = vec![]; + for input in &interleave.inputs { + inputs.push(input.try_into_physical_plan( + registry, + runtime, + extension_codec, + )?); + } + Ok(Arc::new(InterleaveExec::try_new(inputs)?)) + } PhysicalPlanType::CrossJoin(crossjoin) => { let left: Arc = into_physical_plan( &crossjoin.left, @@ -735,7 +746,7 @@ impl AsExecutionPlan for PhysicalPlanNode { })? .as_ref(); Ok(PhysicalSortExpr { - expr: parse_physical_expr(expr,registry, input.schema().as_ref())?, + expr: parse_physical_expr(expr, registry, input.schema().as_ref())?, options: SortOptions { descending: !sort_expr.asc, nulls_first: sort_expr.nulls_first, @@ -782,7 +793,7 @@ impl AsExecutionPlan for PhysicalPlanNode { })? .as_ref(); Ok(PhysicalSortExpr { - expr: parse_physical_expr(expr,registry, input.schema().as_ref())?, + expr: parse_physical_expr(expr, registry, input.schema().as_ref())?, options: SortOptions { descending: !sort_expr.asc, nulls_first: sort_expr.nulls_first, @@ -845,7 +856,7 @@ impl AsExecutionPlan for PhysicalPlanNode { f.expression.as_ref().ok_or_else(|| { proto_error("Unexpected empty filter expression") })?, - registry, &schema + registry, &schema, )?; let column_indices = f.column_indices .iter() @@ -856,7 +867,7 @@ impl AsExecutionPlan for PhysicalPlanNode { i.side)) )?; - Ok(ColumnIndex{ + Ok(ColumnIndex { index: i.index as usize, side: side.into(), }) @@ -1463,6 +1474,21 @@ impl AsExecutionPlan for PhysicalPlanNode { }); } + if let Some(interleave) = plan.downcast_ref::() { + let mut inputs: Vec = vec![]; + for input in interleave.inputs() { + inputs.push(protobuf::PhysicalPlanNode::try_from_physical_plan( + input.to_owned(), + extension_codec, + )?); + } + return Ok(protobuf::PhysicalPlanNode { + physical_plan_type: Some(PhysicalPlanType::Interleave( + protobuf::InterleaveExecNode { inputs }, + )), + }); + } + if let Some(exec) = plan.downcast_ref::() { let input = protobuf::PhysicalPlanNode::try_from_physical_plan( exec.input().to_owned(), diff --git a/datafusion/proto/tests/cases/roundtrip_physical_plan.rs b/datafusion/proto/tests/cases/roundtrip_physical_plan.rs index 287207bae5f6..f46a29447dd6 100644 --- a/datafusion/proto/tests/cases/roundtrip_physical_plan.rs +++ b/datafusion/proto/tests/cases/roundtrip_physical_plan.rs @@ -50,12 +50,14 @@ use datafusion::physical_plan::joins::{ }; use datafusion::physical_plan::limit::{GlobalLimitExec, LocalLimitExec}; use datafusion::physical_plan::projection::ProjectionExec; +use datafusion::physical_plan::repartition::RepartitionExec; use datafusion::physical_plan::sorts::sort::SortExec; +use datafusion::physical_plan::union::{InterleaveExec, UnionExec}; use datafusion::physical_plan::windows::{ BuiltInWindowExpr, PlainAggregateWindowExpr, WindowAggExec, }; use datafusion::physical_plan::{ - functions, udaf, AggregateExpr, ExecutionPlan, PhysicalExpr, Statistics, + functions, udaf, AggregateExpr, ExecutionPlan, Partitioning, PhysicalExpr, Statistics, }; use datafusion::prelude::SessionContext; use datafusion::scalar::ScalarValue; @@ -231,21 +233,21 @@ fn roundtrip_window() -> Result<()> { }; let builtin_window_expr = Arc::new(BuiltInWindowExpr::new( - Arc::new(NthValue::first( - "FIRST_VALUE(a) PARTITION BY [b] ORDER BY [a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW", - col("a", &schema)?, - DataType::Int64, - )), - &[col("b", &schema)?], - &[PhysicalSortExpr { - expr: col("a", &schema)?, - options: SortOptions { - descending: false, - nulls_first: false, - }, - }], - Arc::new(window_frame), - )); + Arc::new(NthValue::first( + "FIRST_VALUE(a) PARTITION BY [b] ORDER BY [a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW", + col("a", &schema)?, + DataType::Int64, + )), + &[col("b", &schema)?], + &[PhysicalSortExpr { + expr: col("a", &schema)?, + options: SortOptions { + descending: false, + nulls_first: false, + }, + }], + Arc::new(window_frame), + )); let plain_aggr_window_expr = Arc::new(PlainAggregateWindowExpr::new( Arc::new(Avg::new( @@ -798,3 +800,34 @@ fn roundtrip_sym_hash_join() -> Result<()> { } Ok(()) } + +#[test] +fn roundtrip_union() -> Result<()> { + let field_a = Field::new("col", DataType::Int64, false); + let schema_left = Schema::new(vec![field_a.clone()]); + let schema_right = Schema::new(vec![field_a]); + let left = EmptyExec::new(false, Arc::new(schema_left)); + let right = EmptyExec::new(false, Arc::new(schema_right)); + let inputs: Vec> = vec![Arc::new(left), Arc::new(right)]; + let union = UnionExec::new(inputs); + roundtrip_test(Arc::new(union)) +} + +#[test] +fn roundtrip_interleave() -> Result<()> { + let field_a = Field::new("col", DataType::Int64, false); + let schema_left = Schema::new(vec![field_a.clone()]); + let schema_right = Schema::new(vec![field_a]); + let partition = Partitioning::Hash(vec![], 3); + let left = RepartitionExec::try_new( + Arc::new(EmptyExec::new(false, Arc::new(schema_left))), + partition.clone(), + )?; + let right = RepartitionExec::try_new( + Arc::new(EmptyExec::new(false, Arc::new(schema_right))), + partition.clone(), + )?; + let inputs: Vec> = vec![Arc::new(left), Arc::new(right)]; + let interleave = InterleaveExec::try_new(inputs)?; + roundtrip_test(Arc::new(interleave)) +} From ecb7c7da957d4cc9a772b7c9b9c36e57292ee699 Mon Sep 17 00:00:00 2001 From: Mustafa Akur <106137913+mustafasrepo@users.noreply.github.com> Date: Fri, 8 Dec 2023 14:44:53 +0300 Subject: [PATCH 390/572] Not fail when window input is empty record batch (#8466) --- datafusion/common/src/utils.rs | 10 +++++++--- .../src/windows/bounded_window_agg_exec.rs | 9 ++++++--- datafusion/sqllogictest/test_files/window.slt | 6 ++++-- 3 files changed, 17 insertions(+), 8 deletions(-) diff --git a/datafusion/common/src/utils.rs b/datafusion/common/src/utils.rs index 9094ecd06361..fecab8835e50 100644 --- a/datafusion/common/src/utils.rs +++ b/datafusion/common/src/utils.rs @@ -25,7 +25,7 @@ use arrow::compute; use arrow::compute::{partition, SortColumn, SortOptions}; use arrow::datatypes::{Field, SchemaRef, UInt32Type}; use arrow::record_batch::RecordBatch; -use arrow_array::{Array, LargeListArray, ListArray}; +use arrow_array::{Array, LargeListArray, ListArray, RecordBatchOptions}; use arrow_schema::DataType; use sqlparser::ast::Ident; use sqlparser::dialect::GenericDialect; @@ -90,8 +90,12 @@ pub fn get_record_batch_at_indices( indices: &PrimitiveArray, ) -> Result { let new_columns = get_arrayref_at_indices(record_batch.columns(), indices)?; - RecordBatch::try_new(record_batch.schema(), new_columns) - .map_err(DataFusionError::ArrowError) + RecordBatch::try_new_with_options( + record_batch.schema(), + new_columns, + &RecordBatchOptions::new().with_row_count(Some(indices.len())), + ) + .map_err(DataFusionError::ArrowError) } /// This function compares two tuples depending on the given sort options. diff --git a/datafusion/physical-plan/src/windows/bounded_window_agg_exec.rs b/datafusion/physical-plan/src/windows/bounded_window_agg_exec.rs index f988b28cce0d..431a43bc6055 100644 --- a/datafusion/physical-plan/src/windows/bounded_window_agg_exec.rs +++ b/datafusion/physical-plan/src/windows/bounded_window_agg_exec.rs @@ -40,7 +40,7 @@ use crate::{ }; use arrow::{ - array::{Array, ArrayRef, UInt32Builder}, + array::{Array, ArrayRef, RecordBatchOptions, UInt32Builder}, compute::{concat, concat_batches, sort_to_indices}, datatypes::{Schema, SchemaBuilder, SchemaRef}, record_batch::RecordBatch, @@ -1026,8 +1026,11 @@ impl BoundedWindowAggStream { .iter() .map(|elem| elem.slice(n_out, n_to_keep)) .collect::>(); - self.input_buffer = - RecordBatch::try_new(self.input_buffer.schema(), batch_to_keep)?; + self.input_buffer = RecordBatch::try_new_with_options( + self.input_buffer.schema(), + batch_to_keep, + &RecordBatchOptions::new().with_row_count(Some(n_to_keep)), + )?; Ok(()) } diff --git a/datafusion/sqllogictest/test_files/window.slt b/datafusion/sqllogictest/test_files/window.slt index b660a9a0c2ae..7846bb001a91 100644 --- a/datafusion/sqllogictest/test_files/window.slt +++ b/datafusion/sqllogictest/test_files/window.slt @@ -3771,10 +3771,12 @@ select a, 1 1 2 1 -# TODO: this works in Postgres which returns [1, 1]. -query error DataFusion error: Arrow error: Invalid argument error: must either specify a row count or at least one column +query I select rank() over (RANGE between UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING) rnk from (select 1 a union select 2 a) q; +---- +1 +1 query II select a, From 3f6ff22d40e0d3373e1538929ec54ee1ec330fc9 Mon Sep 17 00:00:00 2001 From: Alex Huang Date: Fri, 8 Dec 2023 12:46:29 +0100 Subject: [PATCH 391/572] update cast (#8458) --- datafusion/physical-expr/src/expressions/cast.rs | 6 ++---- 1 file changed, 2 insertions(+), 4 deletions(-) diff --git a/datafusion/physical-expr/src/expressions/cast.rs b/datafusion/physical-expr/src/expressions/cast.rs index b3ca95292a37..0c4ed3c12549 100644 --- a/datafusion/physical-expr/src/expressions/cast.rs +++ b/datafusion/physical-expr/src/expressions/cast.rs @@ -140,8 +140,7 @@ impl PhysicalExpr for CastExpr { let mut s = state; self.expr.hash(&mut s); self.cast_type.hash(&mut s); - // Add `self.cast_options` when hash is available - // https://github.com/apache/arrow-rs/pull/4395 + self.cast_options.hash(&mut s); } /// A [`CastExpr`] preserves the ordering of its child. @@ -157,8 +156,7 @@ impl PartialEq for CastExpr { .map(|x| { self.expr.eq(&x.expr) && self.cast_type == x.cast_type - // TODO: Use https://github.com/apache/arrow-rs/issues/2966 when available - && self.cast_options.safe == x.cast_options.safe + && self.cast_options == x.cast_options }) .unwrap_or(false) } From d771f266b1b46d5a0eeacc11b66c9bebd1ec257d Mon Sep 17 00:00:00 2001 From: Huaijin Date: Fri, 8 Dec 2023 20:06:24 +0800 Subject: [PATCH 392/572] fix: don't unifies projection if expr is non-trival (#8454) * fix: don't unifies projection if expr is non-trival * Update datafusion/core/src/physical_optimizer/projection_pushdown.rs Co-authored-by: Alex Huang --------- Co-authored-by: Alex Huang --- .../physical_optimizer/projection_pushdown.rs | 38 ++++++++++++++++-- datafusion/sqllogictest/test_files/select.slt | 39 +++++++++++++++++++ 2 files changed, 74 insertions(+), 3 deletions(-) diff --git a/datafusion/core/src/physical_optimizer/projection_pushdown.rs b/datafusion/core/src/physical_optimizer/projection_pushdown.rs index f6c94edd8ca3..67a2eaf0d9b3 100644 --- a/datafusion/core/src/physical_optimizer/projection_pushdown.rs +++ b/datafusion/core/src/physical_optimizer/projection_pushdown.rs @@ -20,6 +20,7 @@ //! projections one by one if the operator below is amenable to this. If a //! projection reaches a source, it can even dissappear from the plan entirely. +use std::collections::HashMap; use std::sync::Arc; use super::output_requirements::OutputRequirementExec; @@ -42,9 +43,9 @@ use crate::physical_plan::{Distribution, ExecutionPlan}; use arrow_schema::SchemaRef; use datafusion_common::config::ConfigOptions; -use datafusion_common::tree_node::{Transformed, TreeNode}; +use datafusion_common::tree_node::{Transformed, TreeNode, VisitRecursion}; use datafusion_common::JoinSide; -use datafusion_physical_expr::expressions::Column; +use datafusion_physical_expr::expressions::{Column, Literal}; use datafusion_physical_expr::{ Partitioning, PhysicalExpr, PhysicalSortExpr, PhysicalSortRequirement, }; @@ -245,12 +246,36 @@ fn try_swapping_with_streaming_table( } /// Unifies `projection` with its input (which is also a [`ProjectionExec`]). -/// Two consecutive projections can always merge into a single projection. fn try_unifying_projections( projection: &ProjectionExec, child: &ProjectionExec, ) -> Result>> { let mut projected_exprs = vec![]; + let mut column_ref_map: HashMap = HashMap::new(); + + // Collect the column references usage in the outer projection. + projection.expr().iter().for_each(|(expr, _)| { + expr.apply(&mut |expr| { + Ok({ + if let Some(column) = expr.as_any().downcast_ref::() { + *column_ref_map.entry(column.clone()).or_default() += 1; + } + VisitRecursion::Continue + }) + }) + .unwrap(); + }); + + // Merging these projections is not beneficial, e.g + // If an expression is not trivial and it is referred more than 1, unifies projections will be + // beneficial as caching mechanism for non-trivial computations. + // See discussion in: https://github.com/apache/arrow-datafusion/issues/8296 + if column_ref_map.iter().any(|(column, count)| { + *count > 1 && !is_expr_trivial(&child.expr()[column.index()].0.clone()) + }) { + return Ok(None); + } + for (expr, alias) in projection.expr() { // If there is no match in the input projection, we cannot unify these // projections. This case will arise if the projection expression contains @@ -265,6 +290,13 @@ fn try_unifying_projections( .map(|e| Some(Arc::new(e) as _)) } +/// Checks if the given expression is trivial. +/// An expression is considered trivial if it is either a `Column` or a `Literal`. +fn is_expr_trivial(expr: &Arc) -> bool { + expr.as_any().downcast_ref::().is_some() + || expr.as_any().downcast_ref::().is_some() +} + /// Tries to swap `projection` with its input (`output_req`). If possible, /// performs the swap and returns [`OutputRequirementExec`] as the top plan. /// Otherwise, returns `None`. diff --git a/datafusion/sqllogictest/test_files/select.slt b/datafusion/sqllogictest/test_files/select.slt index bbb05b6cffa7..ea570b99d4dd 100644 --- a/datafusion/sqllogictest/test_files/select.slt +++ b/datafusion/sqllogictest/test_files/select.slt @@ -1065,3 +1065,42 @@ select z+1, y from (select x+1 as z, y from t) where y > 1; ---- 3 2 3 3 + +query TT +EXPLAIN SELECT x/2, x/2+1 FROM t; +---- +logical_plan +Projection: t.x / Int64(2)Int64(2)t.x AS t.x / Int64(2), t.x / Int64(2)Int64(2)t.x AS t.x / Int64(2) + Int64(1) +--Projection: t.x / Int64(2) AS t.x / Int64(2)Int64(2)t.x +----TableScan: t projection=[x] +physical_plan +ProjectionExec: expr=[t.x / Int64(2)Int64(2)t.x@0 as t.x / Int64(2), t.x / Int64(2)Int64(2)t.x@0 + 1 as t.x / Int64(2) + Int64(1)] +--ProjectionExec: expr=[x@0 / 2 as t.x / Int64(2)Int64(2)t.x] +----MemoryExec: partitions=1, partition_sizes=[1] + +query II +SELECT x/2, x/2+1 FROM t; +---- +0 1 +0 1 + +query TT +EXPLAIN SELECT abs(x), abs(x) + abs(y) FROM t; +---- +logical_plan +Projection: abs(t.x)t.x AS abs(t.x), abs(t.x)t.x AS abs(t.x) + abs(t.y) +--Projection: abs(t.x) AS abs(t.x)t.x, t.y +----TableScan: t projection=[x, y] +physical_plan +ProjectionExec: expr=[abs(t.x)t.x@0 as abs(t.x), abs(t.x)t.x@0 + abs(y@1) as abs(t.x) + abs(t.y)] +--ProjectionExec: expr=[abs(x@0) as abs(t.x)t.x, y@1 as y] +----MemoryExec: partitions=1, partition_sizes=[1] + +query II +SELECT abs(x), abs(x) + abs(y) FROM t; +---- +1 3 +1 4 + +statement ok +DROP TABLE t; From d43a70d254410e0c2d16f5817dd36e66626aceca Mon Sep 17 00:00:00 2001 From: Andrew Lamb Date: Fri, 8 Dec 2023 07:08:10 -0500 Subject: [PATCH 393/572] Minor: Add new bloom filter predicate tests (#8433) * Minor: Add new bloom filter tests * fmt --- .../physical_plan/parquet/row_groups.rs | 117 +++++++++++++++++- 1 file changed, 113 insertions(+), 4 deletions(-) diff --git a/datafusion/core/src/datasource/physical_plan/parquet/row_groups.rs b/datafusion/core/src/datasource/physical_plan/parquet/row_groups.rs index 0ab2046097c4..65414f5619a5 100644 --- a/datafusion/core/src/datasource/physical_plan/parquet/row_groups.rs +++ b/datafusion/core/src/datasource/physical_plan/parquet/row_groups.rs @@ -350,6 +350,7 @@ mod tests { use arrow::datatypes::Schema; use arrow::datatypes::{DataType, Field}; use datafusion_common::{config::ConfigOptions, TableReference, ToDFSchema}; + use datafusion_common::{DataFusionError, Result}; use datafusion_expr::{ builder::LogicalTableSource, cast, col, lit, AggregateUDF, Expr, ScalarUDF, TableSource, WindowUDF, @@ -994,6 +995,26 @@ mod tests { create_physical_expr(expr, &df_schema, schema, &execution_props).unwrap() } + // Note the values in the `String` column are: + // ❯ select * from './parquet-testing/data/data_index_bloom_encoding_stats.parquet'; + // +-----------+ + // | String | + // +-----------+ + // | Hello | + // | This is | + // | a | + // | test | + // | How | + // | are you | + // | doing | + // | today | + // | the quick | + // | brown fox | + // | jumps | + // | over | + // | the lazy | + // | dog | + // +-----------+ #[tokio::test] async fn test_row_group_bloom_filter_pruning_predicate_simple_expr() { // load parquet file @@ -1002,7 +1023,7 @@ mod tests { let path = format!("{testdata}/{file_name}"); let data = bytes::Bytes::from(std::fs::read(path).unwrap()); - // generate pruning predicate + // generate pruning predicate `(String = "Hello_Not_exists")` let schema = Schema::new(vec![Field::new("String", DataType::Utf8, false)]); let expr = col(r#""String""#).eq(lit("Hello_Not_Exists")); let expr = logical2physical(&expr, &schema); @@ -1029,7 +1050,7 @@ mod tests { let path = format!("{testdata}/{file_name}"); let data = bytes::Bytes::from(std::fs::read(path).unwrap()); - // generate pruning predicate + // generate pruning predicate `(String = "Hello_Not_exists" OR String = "Hello_Not_exists2")` let schema = Schema::new(vec![Field::new("String", DataType::Utf8, false)]); let expr = lit("1").eq(lit("1")).and( col(r#""String""#) @@ -1091,7 +1112,7 @@ mod tests { let path = format!("{testdata}/{file_name}"); let data = bytes::Bytes::from(std::fs::read(path).unwrap()); - // generate pruning predicate + // generate pruning predicate `(String = "Hello")` let schema = Schema::new(vec![Field::new("String", DataType::Utf8, false)]); let expr = col(r#""String""#).eq(lit("Hello")); let expr = logical2physical(&expr, &schema); @@ -1110,6 +1131,94 @@ mod tests { assert_eq!(pruned_row_groups, row_groups); } + #[tokio::test] + async fn test_row_group_bloom_filter_pruning_predicate_with_exists_2_values() { + // load parquet file + let testdata = datafusion_common::test_util::parquet_test_data(); + let file_name = "data_index_bloom_encoding_stats.parquet"; + let path = format!("{testdata}/{file_name}"); + let data = bytes::Bytes::from(std::fs::read(path).unwrap()); + + // generate pruning predicate `(String = "Hello") OR (String = "the quick")` + let schema = Schema::new(vec![Field::new("String", DataType::Utf8, false)]); + let expr = col(r#""String""#) + .eq(lit("Hello")) + .or(col(r#""String""#).eq(lit("the quick"))); + let expr = logical2physical(&expr, &schema); + let pruning_predicate = + PruningPredicate::try_new(expr, Arc::new(schema)).unwrap(); + + let row_groups = vec![0]; + let pruned_row_groups = test_row_group_bloom_filter_pruning_predicate( + file_name, + data, + &pruning_predicate, + &row_groups, + ) + .await + .unwrap(); + assert_eq!(pruned_row_groups, row_groups); + } + + #[tokio::test] + async fn test_row_group_bloom_filter_pruning_predicate_with_exists_3_values() { + // load parquet file + let testdata = datafusion_common::test_util::parquet_test_data(); + let file_name = "data_index_bloom_encoding_stats.parquet"; + let path = format!("{testdata}/{file_name}"); + let data = bytes::Bytes::from(std::fs::read(path).unwrap()); + + // generate pruning predicate `(String = "Hello") OR (String = "the quick") OR (String = "are you")` + let schema = Schema::new(vec![Field::new("String", DataType::Utf8, false)]); + let expr = col(r#""String""#) + .eq(lit("Hello")) + .or(col(r#""String""#).eq(lit("the quick"))) + .or(col(r#""String""#).eq(lit("are you"))); + let expr = logical2physical(&expr, &schema); + let pruning_predicate = + PruningPredicate::try_new(expr, Arc::new(schema)).unwrap(); + + let row_groups = vec![0]; + let pruned_row_groups = test_row_group_bloom_filter_pruning_predicate( + file_name, + data, + &pruning_predicate, + &row_groups, + ) + .await + .unwrap(); + assert_eq!(pruned_row_groups, row_groups); + } + + #[tokio::test] + async fn test_row_group_bloom_filter_pruning_predicate_with_or_not_eq() { + // load parquet file + let testdata = datafusion_common::test_util::parquet_test_data(); + let file_name = "data_index_bloom_encoding_stats.parquet"; + let path = format!("{testdata}/{file_name}"); + let data = bytes::Bytes::from(std::fs::read(path).unwrap()); + + // generate pruning predicate `(String = "foo") OR (String != "bar")` + let schema = Schema::new(vec![Field::new("String", DataType::Utf8, false)]); + let expr = col(r#""String""#) + .not_eq(lit("foo")) + .or(col(r#""String""#).not_eq(lit("bar"))); + let expr = logical2physical(&expr, &schema); + let pruning_predicate = + PruningPredicate::try_new(expr, Arc::new(schema)).unwrap(); + + let row_groups = vec![0]; + let pruned_row_groups = test_row_group_bloom_filter_pruning_predicate( + file_name, + data, + &pruning_predicate, + &row_groups, + ) + .await + .unwrap(); + assert_eq!(pruned_row_groups, row_groups); + } + #[tokio::test] async fn test_row_group_bloom_filter_pruning_predicate_without_bloom_filter() { // load parquet file @@ -1118,7 +1227,7 @@ mod tests { let path = format!("{testdata}/{file_name}"); let data = bytes::Bytes::from(std::fs::read(path).unwrap()); - // generate pruning predicate + // generate pruning predicate on a column without a bloom filter let schema = Schema::new(vec![Field::new("string_col", DataType::Utf8, false)]); let expr = col(r#""string_col""#).eq(lit("0")); let expr = logical2physical(&expr, &schema); From 8f9d6e349627e7eaf818942aa039441bc2bd61a8 Mon Sep 17 00:00:00 2001 From: Mustafa Akur <106137913+mustafasrepo@users.noreply.github.com> Date: Fri, 8 Dec 2023 16:40:56 +0300 Subject: [PATCH 394/572] Add PRIMARY KEY Aggregate support to dataframe API (#8356) * Aggregate rewrite for dataframe API. * Simplifications * Minor changes * Minor changes * Add new test * Add new tests * Minor changes * Add rule, for aggregate simplification * Simplifications * Simplifications * Simplifications * Minor changes * Simplifications * Add new test condition * Tmp * Push requirement below aggregate * Add join and subqeury alias * Add cross join support * Minor changes * Add logical plan repartition support * Add union support * Add table scan * Add limit * Minor changes, buggy * Add new tests, fix existing bugs * change concat type array_concat * Resolve some of the bugs * Comment out a rule * All tests pass, when single distinct is closed * Fix aggregate bug * Change analyze and explain implementations * All tests pass * Resolve linter errors * Simplifications, remove unnecessary codes * Comment out tests * Remove pushdown projection * Pushdown empty projections * Fix failing tests * Simplifications * Update comments, simplifications * Remove eliminate projection rule, Add method for group expr len aggregate * Simplifications, subquery support * Update comments, add unnest support, simplifications * Remove eliminate projection pass * Change name * Minor changes * Minor changes * Add comments * Fix failing test * Minor simplifications * update * Minor * Remove ordering * Minor changes * add merge projections * Add comments, resolve linter errors * Minor changes * Minor changes * Minor changes * Minor changes * Minor changes * Minor changes * Minor changes * Minor changes * Review Part 1 * Review Part 2 * Fix quadratic search, Change trim_expr impl * Review Part 3 * Address reviews * Minor changes * Review Part 4 * Add case expr support * Review Part 5 * Review Part 6 * Finishing touch: Improve comments --------- Co-authored-by: berkaysynnada Co-authored-by: Mehmet Ozan Kabak --- datafusion/common/src/dfschema.rs | 13 +- .../common/src/functional_dependencies.rs | 117 ++- datafusion/common/src/lib.rs | 5 +- datafusion/core/src/dataframe/mod.rs | 346 ++++++- datafusion/core/tests/dataframe/mod.rs | 85 ++ datafusion/expr/src/logical_plan/builder.rs | 33 +- datafusion/expr/src/logical_plan/plan.rs | 56 +- datafusion/expr/src/type_coercion/binary.rs | 2 +- datafusion/expr/src/utils.rs | 34 +- .../optimizer/src/optimize_projections.rs | 958 ++++++++++-------- datafusion/optimizer/src/optimizer.rs | 20 +- .../optimizer/tests/optimizer_integration.rs | 10 +- datafusion/sql/src/select.rs | 76 +- .../sqllogictest/test_files/groupby.slt | 205 +++- 14 files changed, 1349 insertions(+), 611 deletions(-) diff --git a/datafusion/common/src/dfschema.rs b/datafusion/common/src/dfschema.rs index 9819ae795b74..e06f947ad5e7 100644 --- a/datafusion/common/src/dfschema.rs +++ b/datafusion/common/src/dfschema.rs @@ -199,9 +199,16 @@ impl DFSchema { pub fn with_functional_dependencies( mut self, functional_dependencies: FunctionalDependencies, - ) -> Self { - self.functional_dependencies = functional_dependencies; - self + ) -> Result { + if functional_dependencies.is_valid(self.fields.len()) { + self.functional_dependencies = functional_dependencies; + Ok(self) + } else { + _plan_err!( + "Invalid functional dependency: {:?}", + functional_dependencies + ) + } } /// Create a new schema that contains the fields from this schema followed by the fields diff --git a/datafusion/common/src/functional_dependencies.rs b/datafusion/common/src/functional_dependencies.rs index 4587677e7726..1cb1751d713e 100644 --- a/datafusion/common/src/functional_dependencies.rs +++ b/datafusion/common/src/functional_dependencies.rs @@ -24,6 +24,7 @@ use std::ops::Deref; use std::vec::IntoIter; use crate::error::_plan_err; +use crate::utils::{merge_and_order_indices, set_difference}; use crate::{DFSchema, DFSchemaRef, DataFusionError, JoinType, Result}; use sqlparser::ast::TableConstraint; @@ -271,6 +272,29 @@ impl FunctionalDependencies { self.deps.extend(other.deps); } + /// Sanity checks if functional dependencies are valid. For example, if + /// there are 10 fields, we cannot receive any index further than 9. + pub fn is_valid(&self, n_field: usize) -> bool { + self.deps.iter().all( + |FunctionalDependence { + source_indices, + target_indices, + .. + }| { + source_indices + .iter() + .max() + .map(|&max_index| max_index < n_field) + .unwrap_or(true) + && target_indices + .iter() + .max() + .map(|&max_index| max_index < n_field) + .unwrap_or(true) + }, + ) + } + /// Adds the `offset` value to `source_indices` and `target_indices` for /// each functional dependency. pub fn add_offset(&mut self, offset: usize) { @@ -442,44 +466,56 @@ pub fn aggregate_functional_dependencies( } in &func_dependencies.deps { // Keep source indices in a `HashSet` to prevent duplicate entries: - let mut new_source_indices = HashSet::new(); + let mut new_source_indices = vec![]; + let mut new_source_field_names = vec![]; let source_field_names = source_indices .iter() .map(|&idx| aggr_input_fields[idx].qualified_name()) .collect::>(); + for (idx, group_by_expr_name) in group_by_expr_names.iter().enumerate() { // When one of the input determinant expressions matches with // the GROUP BY expression, add the index of the GROUP BY // expression as a new determinant key: if source_field_names.contains(group_by_expr_name) { - new_source_indices.insert(idx); + new_source_indices.push(idx); + new_source_field_names.push(group_by_expr_name.clone()); } } + let existing_target_indices = + get_target_functional_dependencies(aggr_input_schema, group_by_expr_names); + let new_target_indices = get_target_functional_dependencies( + aggr_input_schema, + &new_source_field_names, + ); + let mode = if existing_target_indices == new_target_indices + && new_target_indices.is_some() + { + // If dependency covers all GROUP BY expressions, mode will be `Single`: + Dependency::Single + } else { + // Otherwise, existing mode is preserved: + *mode + }; // All of the composite indices occur in the GROUP BY expression: if new_source_indices.len() == source_indices.len() { aggregate_func_dependencies.push( FunctionalDependence::new( - new_source_indices.into_iter().collect(), + new_source_indices, target_indices.clone(), *nullable, ) - // input uniqueness stays the same when GROUP BY matches with input functional dependence determinants - .with_mode(*mode), + .with_mode(mode), ); } } + // If we have a single GROUP BY key, we can guarantee uniqueness after // aggregation: if group_by_expr_names.len() == 1 { // If `source_indices` contain 0, delete this functional dependency // as it will be added anyway with mode `Dependency::Single`: - if let Some(idx) = aggregate_func_dependencies - .iter() - .position(|item| item.source_indices.contains(&0)) - { - // Delete the functional dependency that contains zeroth idx: - aggregate_func_dependencies.remove(idx); - } + aggregate_func_dependencies.retain(|item| !item.source_indices.contains(&0)); // Add a new functional dependency associated with the whole table: aggregate_func_dependencies.push( // Use nullable property of the group by expression @@ -527,8 +563,61 @@ pub fn get_target_functional_dependencies( combined_target_indices.extend(target_indices.iter()); } } - (!combined_target_indices.is_empty()) - .then_some(combined_target_indices.iter().cloned().collect::>()) + (!combined_target_indices.is_empty()).then_some({ + let mut result = combined_target_indices.into_iter().collect::>(); + result.sort(); + result + }) +} + +/// Returns indices for the minimal subset of GROUP BY expressions that are +/// functionally equivalent to the original set of GROUP BY expressions. +pub fn get_required_group_by_exprs_indices( + schema: &DFSchema, + group_by_expr_names: &[String], +) -> Option> { + let dependencies = schema.functional_dependencies(); + let field_names = schema + .fields() + .iter() + .map(|item| item.qualified_name()) + .collect::>(); + let mut groupby_expr_indices = group_by_expr_names + .iter() + .map(|group_by_expr_name| { + field_names + .iter() + .position(|field_name| field_name == group_by_expr_name) + }) + .collect::>>()?; + + groupby_expr_indices.sort(); + for FunctionalDependence { + source_indices, + target_indices, + .. + } in &dependencies.deps + { + if source_indices + .iter() + .all(|source_idx| groupby_expr_indices.contains(source_idx)) + { + // If all source indices are among GROUP BY expression indices, we + // can remove target indices from GROUP BY expression indices and + // use source indices instead. + groupby_expr_indices = set_difference(&groupby_expr_indices, target_indices); + groupby_expr_indices = + merge_and_order_indices(groupby_expr_indices, source_indices); + } + } + groupby_expr_indices + .iter() + .map(|idx| { + group_by_expr_names + .iter() + .position(|name| &field_names[*idx] == name) + }) + .collect() } /// Updates entries inside the `entries` vector with their corresponding diff --git a/datafusion/common/src/lib.rs b/datafusion/common/src/lib.rs index 6df89624fc51..ed547782e4a5 100644 --- a/datafusion/common/src/lib.rs +++ b/datafusion/common/src/lib.rs @@ -56,8 +56,9 @@ pub use file_options::file_type::{ }; pub use file_options::FileTypeWriterOptions; pub use functional_dependencies::{ - aggregate_functional_dependencies, get_target_functional_dependencies, Constraint, - Constraints, Dependency, FunctionalDependence, FunctionalDependencies, + aggregate_functional_dependencies, get_required_group_by_exprs_indices, + get_target_functional_dependencies, Constraint, Constraints, Dependency, + FunctionalDependence, FunctionalDependencies, }; pub use join_type::{JoinConstraint, JoinSide, JoinType}; pub use param_value::ParamValues; diff --git a/datafusion/core/src/dataframe/mod.rs b/datafusion/core/src/dataframe/mod.rs index 52b5157b7313..c40dd522a457 100644 --- a/datafusion/core/src/dataframe/mod.rs +++ b/datafusion/core/src/dataframe/mod.rs @@ -23,44 +23,43 @@ mod parquet; use std::any::Any; use std::sync::Arc; +use crate::arrow::datatypes::{Schema, SchemaRef}; +use crate::arrow::record_batch::RecordBatch; +use crate::arrow::util::pretty; +use crate::datasource::{provider_as_source, MemTable, TableProvider}; +use crate::error::Result; +use crate::execution::{ + context::{SessionState, TaskContext}, + FunctionRegistry, +}; +use crate::logical_expr::utils::find_window_exprs; +use crate::logical_expr::{ + col, Expr, JoinType, LogicalPlan, LogicalPlanBuilder, Partitioning, TableType, +}; +use crate::physical_plan::{ + collect, collect_partitioned, execute_stream, execute_stream_partitioned, + ExecutionPlan, SendableRecordBatchStream, +}; +use crate::prelude::SessionContext; + use arrow::array::{Array, ArrayRef, Int64Array, StringArray}; use arrow::compute::{cast, concat}; use arrow::csv::WriterBuilder; use arrow::datatypes::{DataType, Field}; -use async_trait::async_trait; use datafusion_common::file_options::csv_writer::CsvWriterOptions; use datafusion_common::file_options::json_writer::JsonWriterOptions; use datafusion_common::parsers::CompressionTypeVariant; use datafusion_common::{ - DataFusionError, FileType, FileTypeWriterOptions, ParamValues, SchemaError, - UnnestOptions, + Column, DFSchema, DataFusionError, FileType, FileTypeWriterOptions, ParamValues, + SchemaError, UnnestOptions, }; use datafusion_expr::dml::CopyOptions; - -use datafusion_common::{Column, DFSchema}; use datafusion_expr::{ avg, count, is_null, max, median, min, stddev, utils::COUNT_STAR_EXPANSION, TableProviderFilterPushDown, UNNAMED_TABLE, }; -use crate::arrow::datatypes::Schema; -use crate::arrow::datatypes::SchemaRef; -use crate::arrow::record_batch::RecordBatch; -use crate::arrow::util::pretty; -use crate::datasource::{provider_as_source, MemTable, TableProvider}; -use crate::error::Result; -use crate::execution::{ - context::{SessionState, TaskContext}, - FunctionRegistry, -}; -use crate::logical_expr::{ - col, utils::find_window_exprs, Expr, JoinType, LogicalPlan, LogicalPlanBuilder, - Partitioning, TableType, -}; -use crate::physical_plan::SendableRecordBatchStream; -use crate::physical_plan::{collect, collect_partitioned}; -use crate::physical_plan::{execute_stream, execute_stream_partitioned, ExecutionPlan}; -use crate::prelude::SessionContext; +use async_trait::async_trait; /// Contains options that control how data is /// written out from a DataFrame @@ -1343,24 +1342,43 @@ impl TableProvider for DataFrameTableProvider { mod tests { use std::vec; - use arrow::array::Int32Array; - use arrow::datatypes::DataType; + use super::*; + use crate::execution::context::SessionConfig; + use crate::physical_plan::{ColumnarValue, Partitioning, PhysicalExpr}; + use crate::test_util::{register_aggregate_csv, test_table, test_table_with_name}; + use crate::{assert_batches_sorted_eq, execution::context::SessionContext}; + use arrow::array::{self, Int32Array}; + use arrow::datatypes::DataType; + use datafusion_common::{Constraint, Constraints, ScalarValue}; use datafusion_expr::{ avg, cast, count, count_distinct, create_udf, expr, lit, max, min, sum, - BuiltInWindowFunction, ScalarFunctionImplementation, Volatility, WindowFrame, - WindowFunction, + BinaryExpr, BuiltInWindowFunction, Operator, ScalarFunctionImplementation, + Volatility, WindowFrame, WindowFunction, }; use datafusion_physical_expr::expressions::Column; - - use crate::execution::context::SessionConfig; - use crate::physical_plan::ColumnarValue; - use crate::physical_plan::Partitioning; - use crate::physical_plan::PhysicalExpr; - use crate::test_util::{register_aggregate_csv, test_table, test_table_with_name}; - use crate::{assert_batches_sorted_eq, execution::context::SessionContext}; - - use super::*; + use datafusion_physical_plan::get_plan_string; + + pub fn table_with_constraints() -> Arc { + let dual_schema = Arc::new(Schema::new(vec![ + Field::new("id", DataType::Int32, false), + Field::new("name", DataType::Utf8, false), + ])); + let batch = RecordBatch::try_new( + dual_schema.clone(), + vec![ + Arc::new(array::Int32Array::from(vec![1])), + Arc::new(array::StringArray::from(vec!["a"])), + ], + ) + .unwrap(); + let provider = MemTable::try_new(dual_schema, vec![vec![batch]]) + .unwrap() + .with_constraints(Constraints::new_unverified(vec![Constraint::PrimaryKey( + vec![0], + )])); + Arc::new(provider) + } async fn assert_logical_expr_schema_eq_physical_expr_schema( df: DataFrame, @@ -1557,6 +1575,262 @@ mod tests { Ok(()) } + #[tokio::test] + async fn test_aggregate_with_pk() -> Result<()> { + // create the dataframe + let config = SessionConfig::new().with_target_partitions(1); + let ctx = SessionContext::new_with_config(config); + + let table1 = table_with_constraints(); + let df = ctx.read_table(table1)?; + let col_id = Expr::Column(datafusion_common::Column { + relation: None, + name: "id".to_string(), + }); + let col_name = Expr::Column(datafusion_common::Column { + relation: None, + name: "name".to_string(), + }); + + // group by contains id column + let group_expr = vec![col_id.clone()]; + let aggr_expr = vec![]; + let df = df.aggregate(group_expr, aggr_expr)?; + + // expr list contains id, name + let expr_list = vec![col_id, col_name]; + let df = df.select(expr_list)?; + let physical_plan = df.clone().create_physical_plan().await?; + let expected = vec![ + "AggregateExec: mode=Single, gby=[id@0 as id, name@1 as name], aggr=[]", + " MemoryExec: partitions=1, partition_sizes=[1]", + ]; + // Get string representation of the plan + let actual = get_plan_string(&physical_plan); + assert_eq!( + expected, actual, + "\n**Optimized Plan Mismatch\n\nexpected:\n\n{expected:#?}\nactual:\n\n{actual:#?}\n\n" + ); + // Since id and name are functionally dependant, we can use name among expression + // even if it is not part of the group by expression. + let df_results = collect(physical_plan, ctx.task_ctx()).await?; + + #[rustfmt::skip] + assert_batches_sorted_eq!( + ["+----+------+", + "| id | name |", + "+----+------+", + "| 1 | a |", + "+----+------+",], + &df_results + ); + + Ok(()) + } + + #[tokio::test] + async fn test_aggregate_with_pk2() -> Result<()> { + // create the dataframe + let config = SessionConfig::new().with_target_partitions(1); + let ctx = SessionContext::new_with_config(config); + + let table1 = table_with_constraints(); + let df = ctx.read_table(table1)?; + let col_id = Expr::Column(datafusion_common::Column { + relation: None, + name: "id".to_string(), + }); + let col_name = Expr::Column(datafusion_common::Column { + relation: None, + name: "name".to_string(), + }); + + // group by contains id column + let group_expr = vec![col_id.clone()]; + let aggr_expr = vec![]; + let df = df.aggregate(group_expr, aggr_expr)?; + + let condition1 = Expr::BinaryExpr(BinaryExpr::new( + Box::new(col_id.clone()), + Operator::Eq, + Box::new(Expr::Literal(ScalarValue::Int32(Some(1)))), + )); + let condition2 = Expr::BinaryExpr(BinaryExpr::new( + Box::new(col_name), + Operator::Eq, + Box::new(Expr::Literal(ScalarValue::Utf8(Some("a".to_string())))), + )); + // Predicate refers to id, and name fields + let predicate = Expr::BinaryExpr(BinaryExpr::new( + Box::new(condition1), + Operator::And, + Box::new(condition2), + )); + let df = df.filter(predicate)?; + let physical_plan = df.clone().create_physical_plan().await?; + + let expected = vec![ + "CoalesceBatchesExec: target_batch_size=8192", + " FilterExec: id@0 = 1 AND name@1 = a", + " AggregateExec: mode=Single, gby=[id@0 as id, name@1 as name], aggr=[]", + " MemoryExec: partitions=1, partition_sizes=[1]", + ]; + // Get string representation of the plan + let actual = get_plan_string(&physical_plan); + assert_eq!( + expected, actual, + "\n**Optimized Plan Mismatch\n\nexpected:\n\n{expected:#?}\nactual:\n\n{actual:#?}\n\n" + ); + + // Since id and name are functionally dependant, we can use name among expression + // even if it is not part of the group by expression. + let df_results = collect(physical_plan, ctx.task_ctx()).await?; + + #[rustfmt::skip] + assert_batches_sorted_eq!( + ["+----+------+", + "| id | name |", + "+----+------+", + "| 1 | a |", + "+----+------+",], + &df_results + ); + + Ok(()) + } + + #[tokio::test] + async fn test_aggregate_with_pk3() -> Result<()> { + // create the dataframe + let config = SessionConfig::new().with_target_partitions(1); + let ctx = SessionContext::new_with_config(config); + + let table1 = table_with_constraints(); + let df = ctx.read_table(table1)?; + let col_id = Expr::Column(datafusion_common::Column { + relation: None, + name: "id".to_string(), + }); + let col_name = Expr::Column(datafusion_common::Column { + relation: None, + name: "name".to_string(), + }); + + // group by contains id column + let group_expr = vec![col_id.clone()]; + let aggr_expr = vec![]; + // group by id, + let df = df.aggregate(group_expr, aggr_expr)?; + + let condition1 = Expr::BinaryExpr(BinaryExpr::new( + Box::new(col_id.clone()), + Operator::Eq, + Box::new(Expr::Literal(ScalarValue::Int32(Some(1)))), + )); + // Predicate refers to id field + let predicate = condition1; + // id=0 + let df = df.filter(predicate)?; + // Select expression refers to id, and name columns. + // id, name + let df = df.select(vec![col_id.clone(), col_name.clone()])?; + let physical_plan = df.clone().create_physical_plan().await?; + + let expected = vec![ + "CoalesceBatchesExec: target_batch_size=8192", + " FilterExec: id@0 = 1", + " AggregateExec: mode=Single, gby=[id@0 as id, name@1 as name], aggr=[]", + " MemoryExec: partitions=1, partition_sizes=[1]", + ]; + // Get string representation of the plan + let actual = get_plan_string(&physical_plan); + assert_eq!( + expected, actual, + "\n**Optimized Plan Mismatch\n\nexpected:\n\n{expected:#?}\nactual:\n\n{actual:#?}\n\n" + ); + + // Since id and name are functionally dependant, we can use name among expression + // even if it is not part of the group by expression. + let df_results = collect(physical_plan, ctx.task_ctx()).await?; + + #[rustfmt::skip] + assert_batches_sorted_eq!( + ["+----+------+", + "| id | name |", + "+----+------+", + "| 1 | a |", + "+----+------+",], + &df_results + ); + + Ok(()) + } + + #[tokio::test] + async fn test_aggregate_with_pk4() -> Result<()> { + // create the dataframe + let config = SessionConfig::new().with_target_partitions(1); + let ctx = SessionContext::new_with_config(config); + + let table1 = table_with_constraints(); + let df = ctx.read_table(table1)?; + let col_id = Expr::Column(datafusion_common::Column { + relation: None, + name: "id".to_string(), + }); + + // group by contains id column + let group_expr = vec![col_id.clone()]; + let aggr_expr = vec![]; + // group by id, + let df = df.aggregate(group_expr, aggr_expr)?; + + let condition1 = Expr::BinaryExpr(BinaryExpr::new( + Box::new(col_id.clone()), + Operator::Eq, + Box::new(Expr::Literal(ScalarValue::Int32(Some(1)))), + )); + // Predicate refers to id field + let predicate = condition1; + // id=1 + let df = df.filter(predicate)?; + // Select expression refers to id column. + // id + let df = df.select(vec![col_id.clone()])?; + let physical_plan = df.clone().create_physical_plan().await?; + + // In this case aggregate shouldn't be expanded, since these + // columns are not used. + let expected = vec![ + "CoalesceBatchesExec: target_batch_size=8192", + " FilterExec: id@0 = 1", + " AggregateExec: mode=Single, gby=[id@0 as id], aggr=[]", + " MemoryExec: partitions=1, partition_sizes=[1]", + ]; + // Get string representation of the plan + let actual = get_plan_string(&physical_plan); + assert_eq!( + expected, actual, + "\n**Optimized Plan Mismatch\n\nexpected:\n\n{expected:#?}\nactual:\n\n{actual:#?}\n\n" + ); + + // Since id and name are functionally dependant, we can use name among expression + // even if it is not part of the group by expression. + let df_results = collect(physical_plan, ctx.task_ctx()).await?; + + #[rustfmt::skip] + assert_batches_sorted_eq!( + [ "+----+", + "| id |", + "+----+", + "| 1 |", + "+----+",], + &df_results + ); + + Ok(()) + } + #[tokio::test] async fn test_distinct() -> Result<()> { let t = test_table().await?; diff --git a/datafusion/core/tests/dataframe/mod.rs b/datafusion/core/tests/dataframe/mod.rs index 10f4574020bf..c6b8e0e01b4f 100644 --- a/datafusion/core/tests/dataframe/mod.rs +++ b/datafusion/core/tests/dataframe/mod.rs @@ -1323,6 +1323,91 @@ async fn unnest_array_agg() -> Result<()> { Ok(()) } +#[tokio::test] +async fn unnest_with_redundant_columns() -> Result<()> { + let mut shape_id_builder = UInt32Builder::new(); + let mut tag_id_builder = UInt32Builder::new(); + + for shape_id in 1..=3 { + for tag_id in 1..=3 { + shape_id_builder.append_value(shape_id as u32); + tag_id_builder.append_value((shape_id * 10 + tag_id) as u32); + } + } + + let batch = RecordBatch::try_from_iter(vec![ + ("shape_id", Arc::new(shape_id_builder.finish()) as ArrayRef), + ("tag_id", Arc::new(tag_id_builder.finish()) as ArrayRef), + ])?; + + let ctx = SessionContext::new(); + ctx.register_batch("shapes", batch)?; + let df = ctx.table("shapes").await?; + + let results = df.clone().collect().await?; + let expected = vec![ + "+----------+--------+", + "| shape_id | tag_id |", + "+----------+--------+", + "| 1 | 11 |", + "| 1 | 12 |", + "| 1 | 13 |", + "| 2 | 21 |", + "| 2 | 22 |", + "| 2 | 23 |", + "| 3 | 31 |", + "| 3 | 32 |", + "| 3 | 33 |", + "+----------+--------+", + ]; + assert_batches_sorted_eq!(expected, &results); + + // Doing an `array_agg` by `shape_id` produces: + let df = df + .clone() + .aggregate( + vec![col("shape_id")], + vec![array_agg(col("shape_id")).alias("shape_id2")], + )? + .unnest_column("shape_id2")? + .select(vec![col("shape_id")])?; + + let optimized_plan = df.clone().into_optimized_plan()?; + let expected = vec![ + "Projection: shapes.shape_id [shape_id:UInt32]", + " Unnest: shape_id2 [shape_id:UInt32, shape_id2:UInt32;N]", + " Aggregate: groupBy=[[shapes.shape_id]], aggr=[[ARRAY_AGG(shapes.shape_id) AS shape_id2]] [shape_id:UInt32, shape_id2:List(Field { name: \"item\", data_type: UInt32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} });N]", + " TableScan: shapes projection=[shape_id] [shape_id:UInt32]", + ]; + + let formatted = optimized_plan.display_indent_schema().to_string(); + let actual: Vec<&str> = formatted.trim().lines().collect(); + assert_eq!( + expected, actual, + "\n\nexpected:\n\n{expected:#?}\nactual:\n\n{actual:#?}\n\n" + ); + + let results = df.collect().await?; + let expected = [ + "+----------+", + "| shape_id |", + "+----------+", + "| 1 |", + "| 1 |", + "| 1 |", + "| 2 |", + "| 2 |", + "| 2 |", + "| 3 |", + "| 3 |", + "| 3 |", + "+----------+", + ]; + assert_batches_sorted_eq!(expected, &results); + + Ok(()) +} + async fn create_test_table(name: &str) -> Result { let schema = Arc::new(Schema::new(vec![ Field::new("a", DataType::Utf8, false), diff --git a/datafusion/expr/src/logical_plan/builder.rs b/datafusion/expr/src/logical_plan/builder.rs index c4ff9fe95435..be2c45b901fa 100644 --- a/datafusion/expr/src/logical_plan/builder.rs +++ b/datafusion/expr/src/logical_plan/builder.rs @@ -50,9 +50,9 @@ use crate::{ use arrow::datatypes::{DataType, Schema, SchemaRef}; use datafusion_common::display::ToStringifiedPlan; use datafusion_common::{ - plan_datafusion_err, plan_err, Column, DFField, DFSchema, DFSchemaRef, - DataFusionError, FileType, OwnedTableReference, Result, ScalarValue, TableReference, - ToDFSchema, UnnestOptions, + get_target_functional_dependencies, plan_datafusion_err, plan_err, Column, DFField, + DFSchema, DFSchemaRef, DataFusionError, FileType, OwnedTableReference, Result, + ScalarValue, TableReference, ToDFSchema, UnnestOptions, }; /// Default table name for unnamed table @@ -904,8 +904,27 @@ impl LogicalPlanBuilder { group_expr: impl IntoIterator>, aggr_expr: impl IntoIterator>, ) -> Result { - let group_expr = normalize_cols(group_expr, &self.plan)?; + let mut group_expr = normalize_cols(group_expr, &self.plan)?; let aggr_expr = normalize_cols(aggr_expr, &self.plan)?; + + // Rewrite groupby exprs according to functional dependencies + let group_by_expr_names = group_expr + .iter() + .map(|group_by_expr| group_by_expr.display_name()) + .collect::>>()?; + let schema = self.plan.schema(); + if let Some(target_indices) = + get_target_functional_dependencies(schema, &group_by_expr_names) + { + for idx in target_indices { + let field = schema.field(idx); + let expr = + Expr::Column(Column::new(field.qualifier().cloned(), field.name())); + if !group_expr.contains(&expr) { + group_expr.push(expr); + } + } + } Aggregate::try_new(Arc::new(self.plan), group_expr, aggr_expr) .map(LogicalPlan::Aggregate) .map(Self::from) @@ -1166,8 +1185,8 @@ pub fn build_join_schema( ); let mut metadata = left.metadata().clone(); metadata.extend(right.metadata().clone()); - DFSchema::new_with_metadata(fields, metadata) - .map(|schema| schema.with_functional_dependencies(func_dependencies)) + let schema = DFSchema::new_with_metadata(fields, metadata)?; + schema.with_functional_dependencies(func_dependencies) } /// Errors if one or more expressions have equal names. @@ -1491,7 +1510,7 @@ pub fn unnest_with_options( let df_schema = DFSchema::new_with_metadata(fields, metadata)?; // We can use the existing functional dependencies: let deps = input_schema.functional_dependencies().clone(); - let schema = Arc::new(df_schema.with_functional_dependencies(deps)); + let schema = Arc::new(df_schema.with_functional_dependencies(deps)?); Ok(LogicalPlan::Unnest(Unnest { input: Arc::new(input), diff --git a/datafusion/expr/src/logical_plan/plan.rs b/datafusion/expr/src/logical_plan/plan.rs index d85e0b5b0a40..dfd4fbf65d8e 100644 --- a/datafusion/expr/src/logical_plan/plan.rs +++ b/datafusion/expr/src/logical_plan/plan.rs @@ -946,7 +946,7 @@ impl LogicalPlan { // We can use the existing functional dependencies as is: .with_functional_dependencies( input.schema().functional_dependencies().clone(), - ), + )?, ); Ok(LogicalPlan::Unnest(Unnest { @@ -1834,8 +1834,9 @@ pub fn projection_schema(input: &LogicalPlan, exprs: &[Expr]) -> Result Schema { Schema::new(vec![ @@ -3164,15 +3165,20 @@ digraph { ) .unwrap(); assert!(!filter.is_scalar()); - let unique_schema = - Arc::new(schema.as_ref().clone().with_functional_dependencies( - FunctionalDependencies::new_from_constraints( - Some(&Constraints::new_unverified(vec![Constraint::Unique( - vec![0], - )])), - 1, - ), - )); + let unique_schema = Arc::new( + schema + .as_ref() + .clone() + .with_functional_dependencies( + FunctionalDependencies::new_from_constraints( + Some(&Constraints::new_unverified(vec![Constraint::Unique( + vec![0], + )])), + 1, + ), + ) + .unwrap(), + ); let scan = Arc::new(LogicalPlan::TableScan(TableScan { table_name: TableReference::bare("tab"), source, diff --git a/datafusion/expr/src/type_coercion/binary.rs b/datafusion/expr/src/type_coercion/binary.rs index 1027e97d061a..dd9449198796 100644 --- a/datafusion/expr/src/type_coercion/binary.rs +++ b/datafusion/expr/src/type_coercion/binary.rs @@ -116,7 +116,7 @@ fn signature(lhs: &DataType, op: &Operator, rhs: &DataType) -> Result }) } AtArrow | ArrowAt => { - // ArrowAt and AtArrow check for whether one array ic contained in another. + // ArrowAt and AtArrow check for whether one array is contained in another. // The result type is boolean. Signature::comparison defines this signature. // Operation has nothing to do with comparison array_coercion(lhs, rhs).map(Signature::comparison).ok_or_else(|| { diff --git a/datafusion/expr/src/utils.rs b/datafusion/expr/src/utils.rs index c30c734fcf1f..abdd7f5f57f6 100644 --- a/datafusion/expr/src/utils.rs +++ b/datafusion/expr/src/utils.rs @@ -17,6 +17,10 @@ //! Expression utilities +use std::cmp::Ordering; +use std::collections::HashSet; +use std::sync::Arc; + use crate::expr::{Alias, Sort, WindowFunction}; use crate::expr_rewriter::strip_outer_reference; use crate::logical_plan::Aggregate; @@ -25,16 +29,15 @@ use crate::{ and, BinaryExpr, Cast, Expr, ExprSchemable, Filter, GroupingSet, LogicalPlan, Operator, TryCast, }; + use arrow::datatypes::{DataType, TimeUnit}; use datafusion_common::tree_node::{TreeNode, VisitRecursion}; use datafusion_common::{ internal_err, plan_datafusion_err, plan_err, Column, DFField, DFSchema, DFSchemaRef, DataFusionError, Result, ScalarValue, TableReference, }; + use sqlparser::ast::{ExceptSelectItem, ExcludeSelectItem, WildcardAdditionalOptions}; -use std::cmp::Ordering; -use std::collections::HashSet; -use std::sync::Arc; /// The value to which `COUNT(*)` is expanded to in /// `COUNT()` expressions @@ -433,7 +436,7 @@ pub fn expand_qualified_wildcard( let qualified_schema = DFSchema::new_with_metadata(qualified_fields, schema.metadata().clone())? // We can use the functional dependencies as is, since it only stores indices: - .with_functional_dependencies(schema.functional_dependencies().clone()); + .with_functional_dependencies(schema.functional_dependencies().clone())?; let excluded_columns = if let Some(WildcardAdditionalOptions { opt_exclude, opt_except, @@ -730,11 +733,7 @@ fn agg_cols(agg: &Aggregate) -> Vec { .collect() } -fn exprlist_to_fields_aggregate( - exprs: &[Expr], - plan: &LogicalPlan, - agg: &Aggregate, -) -> Result> { +fn exprlist_to_fields_aggregate(exprs: &[Expr], agg: &Aggregate) -> Result> { let agg_cols = agg_cols(agg); let mut fields = vec![]; for expr in exprs { @@ -743,7 +742,7 @@ fn exprlist_to_fields_aggregate( // resolve against schema of input to aggregate fields.push(expr.to_field(agg.input.schema())?); } - _ => fields.push(expr.to_field(plan.schema())?), + _ => fields.push(expr.to_field(&agg.schema)?), } } Ok(fields) @@ -760,15 +759,7 @@ pub fn exprlist_to_fields<'a>( // `GROUPING(person.state)` so in order to resolve `person.state` in this case we need to // look at the input to the aggregate instead. let fields = match plan { - LogicalPlan::Aggregate(agg) => { - Some(exprlist_to_fields_aggregate(&exprs, plan, agg)) - } - LogicalPlan::Window(window) => match window.input.as_ref() { - LogicalPlan::Aggregate(agg) => { - Some(exprlist_to_fields_aggregate(&exprs, plan, agg)) - } - _ => None, - }, + LogicalPlan::Aggregate(agg) => Some(exprlist_to_fields_aggregate(&exprs, agg)), _ => None, }; if let Some(fields) = fields { @@ -1240,10 +1231,9 @@ pub fn merge_schema(inputs: Vec<&LogicalPlan>) -> DFSchema { #[cfg(test)] mod tests { use super::*; - use crate::expr_vec_fmt; use crate::{ - col, cube, expr, grouping_set, lit, rollup, AggregateFunction, WindowFrame, - WindowFunction, + col, cube, expr, expr_vec_fmt, grouping_set, lit, rollup, AggregateFunction, + WindowFrame, WindowFunction, }; #[test] diff --git a/datafusion/optimizer/src/optimize_projections.rs b/datafusion/optimizer/src/optimize_projections.rs index 8bee2951541d..7ae9f7edf5e5 100644 --- a/datafusion/optimizer/src/optimize_projections.rs +++ b/datafusion/optimizer/src/optimize_projections.rs @@ -15,33 +15,42 @@ // specific language governing permissions and limitations // under the License. -//! Optimizer rule to prune unnecessary Columns from the intermediate schemas inside the [LogicalPlan]. -//! This rule -//! - Removes unnecessary columns that are not showed at the output, and that are not used during computation. -//! - Adds projection to decrease table column size before operators that benefits from less memory at its input. -//! - Removes unnecessary [LogicalPlan::Projection] from the [LogicalPlan]. +//! Optimizer rule to prune unnecessary columns from intermediate schemas +//! inside the [`LogicalPlan`]. This rule: +//! - Removes unnecessary columns that do not appear at the output and/or are +//! not used during any computation step. +//! - Adds projections to decrease table column size before operators that +//! benefit from a smaller memory footprint at its input. +//! - Removes unnecessary [`LogicalPlan::Projection`]s from the [`LogicalPlan`]. + +use std::collections::HashSet; +use std::sync::Arc; + use crate::optimizer::ApplyOrder; -use datafusion_common::{Column, DFSchema, DFSchemaRef, JoinType, Result}; -use datafusion_expr::expr::{Alias, ScalarFunction}; +use crate::{OptimizerConfig, OptimizerRule}; + +use arrow::datatypes::SchemaRef; +use datafusion_common::{ + get_required_group_by_exprs_indices, Column, DFSchema, DFSchemaRef, JoinType, Result, +}; +use datafusion_expr::expr::{Alias, ScalarFunction, ScalarFunctionDefinition}; use datafusion_expr::{ logical_plan::LogicalPlan, projection_schema, Aggregate, BinaryExpr, Cast, Distinct, - Expr, Projection, ScalarFunctionDefinition, TableScan, Window, + Expr, GroupingSet, Projection, TableScan, Window, }; + use hashbrown::HashMap; use itertools::{izip, Itertools}; -use std::collections::HashSet; -use std::sync::Arc; - -use crate::{OptimizerConfig, OptimizerRule}; -/// A rule for optimizing logical plans by removing unused Columns/Fields. +/// A rule for optimizing logical plans by removing unused columns/fields. /// -/// `OptimizeProjections` is an optimizer rule that identifies and eliminates columns from a logical plan -/// that are not used in any downstream operations. This can improve query performance and reduce unnecessary -/// data processing. +/// `OptimizeProjections` is an optimizer rule that identifies and eliminates +/// columns from a logical plan that are not used by downstream operations. +/// This can improve query performance and reduce unnecessary data processing. /// -/// The rule analyzes the input logical plan, determines the necessary column indices, and then removes any -/// unnecessary columns. Additionally, it eliminates any unnecessary projections in the plan. +/// The rule analyzes the input logical plan, determines the necessary column +/// indices, and then removes any unnecessary columns. It also removes any +/// unnecessary projections from the plan tree. #[derive(Default)] pub struct OptimizeProjections {} @@ -58,8 +67,8 @@ impl OptimizerRule for OptimizeProjections { plan: &LogicalPlan, config: &dyn OptimizerConfig, ) -> Result> { - // All of the fields at the output are necessary. - let indices = require_all_indices(plan); + // All output fields are necessary: + let indices = (0..plan.schema().fields().len()).collect::>(); optimize_projections(plan, config, &indices) } @@ -72,30 +81,35 @@ impl OptimizerRule for OptimizeProjections { } } -/// Removes unnecessary columns (e.g Columns that are not referred at the output schema and -/// Columns that are not used during any computation, expression evaluation) from the logical plan and its inputs. +/// Removes unnecessary columns (e.g. columns that do not appear in the output +/// schema and/or are not used during any computation step such as expression +/// evaluation) from the logical plan and its inputs. /// -/// # Arguments +/// # Parameters /// -/// - `plan`: A reference to the input `LogicalPlan` to be optimized. -/// - `_config`: A reference to the optimizer configuration (not currently used). -/// - `indices`: A slice of column indices that represent the necessary column indices for downstream operations. +/// - `plan`: A reference to the input `LogicalPlan` to optimize. +/// - `config`: A reference to the optimizer configuration. +/// - `indices`: A slice of column indices that represent the necessary column +/// indices for downstream operations. /// /// # Returns /// -/// - `Ok(Some(LogicalPlan))`: An optimized `LogicalPlan` with unnecessary columns removed. -/// - `Ok(None)`: If the optimization process results in a logical plan that doesn't require further propagation. -/// - `Err(error)`: If an error occurs during the optimization process. +/// A `Result` object with the following semantics: +/// +/// - `Ok(Some(LogicalPlan))`: An optimized `LogicalPlan` without unnecessary +/// columns. +/// - `Ok(None)`: Signal that the given logical plan did not require any change. +/// - `Err(error)`: An error occured during the optimization process. fn optimize_projections( plan: &LogicalPlan, - _config: &dyn OptimizerConfig, + config: &dyn OptimizerConfig, indices: &[usize], ) -> Result> { // `child_required_indices` stores // - indices of the columns required for each child // - a flag indicating whether putting a projection above children is beneficial for the parent. // As an example LogicalPlan::Filter benefits from small tables. Hence for filter child this flag would be `true`. - let child_required_indices: Option, bool)>> = match plan { + let child_required_indices: Vec<(Vec, bool)> = match plan { LogicalPlan::Sort(_) | LogicalPlan::Filter(_) | LogicalPlan::Repartition(_) @@ -103,36 +117,32 @@ fn optimize_projections( | LogicalPlan::Union(_) | LogicalPlan::SubqueryAlias(_) | LogicalPlan::Distinct(Distinct::On(_)) => { - // Re-route required indices from the parent + column indices referred by expressions in the plan - // to the child. - // All of these operators benefits from small tables at their inputs. Hence projection_beneficial flag is `true`. + // Pass index requirements from the parent as well as column indices + // that appear in this plan's expressions to its child. All these + // operators benefit from "small" inputs, so the projection_beneficial + // flag is `true`. let exprs = plan.expressions(); - let child_req_indices = plan - .inputs() + plan.inputs() .into_iter() .map(|input| { - let required_indices = - get_all_required_indices(indices, input, exprs.iter())?; - Ok((required_indices, true)) + get_all_required_indices(indices, input, exprs.iter()) + .map(|idxs| (idxs, true)) }) - .collect::>>()?; - Some(child_req_indices) + .collect::>()? } LogicalPlan::Limit(_) | LogicalPlan::Prepare(_) => { - // Re-route required indices from the parent + column indices referred by expressions in the plan - // to the child. - // Limit, Prepare doesn't benefit from small column numbers. Hence projection_beneficial flag is `false`. + // Pass index requirements from the parent as well as column indices + // that appear in this plan's expressions to its child. These operators + // do not benefit from "small" inputs, so the projection_beneficial + // flag is `false`. let exprs = plan.expressions(); - let child_req_indices = plan - .inputs() + plan.inputs() .into_iter() .map(|input| { - let required_indices = - get_all_required_indices(indices, input, exprs.iter())?; - Ok((required_indices, false)) + get_all_required_indices(indices, input, exprs.iter()) + .map(|idxs| (idxs, false)) }) - .collect::>>()?; - Some(child_req_indices) + .collect::>()? } LogicalPlan::Copy(_) | LogicalPlan::Ddl(_) @@ -141,81 +151,99 @@ fn optimize_projections( | LogicalPlan::Analyze(_) | LogicalPlan::Subquery(_) | LogicalPlan::Distinct(Distinct::All(_)) => { - // Require all of the fields of the Dml, Ddl, Copy, Explain, Analyze, Subquery, Distinct::All input(s). - // Their child plan can be treated as final plan. Otherwise expected schema may not match. - // TODO: For some subquery variants we may not need to require all indices for its input. - // such as Exists. - let child_requirements = plan - .inputs() + // These plans require all their fields, and their children should + // be treated as final plans -- otherwise, we may have schema a + // mismatch. + // TODO: For some subquery variants (e.g. a subquery arising from an + // EXISTS expression), we may not need to require all indices. + plan.inputs() .iter() - .map(|input| { - // Require all of the fields for each input. - // No projection since all of the fields at the child is required - (require_all_indices(input), false) - }) - .collect::>(); - Some(child_requirements) + .map(|input| ((0..input.schema().fields().len()).collect_vec(), false)) + .collect::>() } LogicalPlan::EmptyRelation(_) | LogicalPlan::Statement(_) | LogicalPlan::Values(_) | LogicalPlan::Extension(_) | LogicalPlan::DescribeTable(_) => { - // EmptyRelation, Values, DescribeTable, Statement has no inputs stop iteration - - // TODO: Add support for extension - // It is not known how to direct requirements to children for LogicalPlan::Extension. - // Safest behaviour is to stop propagation. - None + // These operators have no inputs, so stop the optimization process. + // TODO: Add support for `LogicalPlan::Extension`. + return Ok(None); } LogicalPlan::Projection(proj) => { return if let Some(proj) = merge_consecutive_projections(proj)? { - rewrite_projection_given_requirements(&proj, _config, indices)? - .map(|res| Ok(Some(res))) - // Even if projection cannot be optimized, return merged version - .unwrap_or_else(|| Ok(Some(LogicalPlan::Projection(proj)))) + Ok(Some( + rewrite_projection_given_requirements(&proj, config, indices)? + // Even if we cannot optimize the projection, merge if possible: + .unwrap_or_else(|| LogicalPlan::Projection(proj)), + )) } else { - rewrite_projection_given_requirements(proj, _config, indices) + rewrite_projection_given_requirements(proj, config, indices) }; } LogicalPlan::Aggregate(aggregate) => { - // Split parent requirements to group by and aggregate sections - let group_expr_len = aggregate.group_expr_len()?; - let (_group_by_reqs, mut aggregate_reqs): (Vec, Vec) = - indices.iter().partition(|&&idx| idx < group_expr_len); - // Offset aggregate indices so that they point to valid indices at the `aggregate.aggr_expr` - aggregate_reqs - .iter_mut() - .for_each(|idx| *idx -= group_expr_len); - - // Group by expressions are same - let new_group_bys = aggregate.group_expr.clone(); - - // Only use absolutely necessary aggregate expressions required by parent. + // Split parent requirements to GROUP BY and aggregate sections: + let n_group_exprs = aggregate.group_expr_len()?; + let (group_by_reqs, mut aggregate_reqs): (Vec, Vec) = + indices.iter().partition(|&&idx| idx < n_group_exprs); + // Offset aggregate indices so that they point to valid indices at + // `aggregate.aggr_expr`: + for idx in aggregate_reqs.iter_mut() { + *idx -= 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.display_name()) + .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. + let required_indices = + merge_slices(&simplest_groupby_indices, &group_by_reqs); + get_at_indices(&aggregate.group_expr, &required_indices) + } else { + aggregate.group_expr.clone() + }; + + // Only use the absolutely necessary aggregate expressions required + // by the parent: let mut new_aggr_expr = get_at_indices(&aggregate.aggr_expr, &aggregate_reqs); let all_exprs_iter = new_group_bys.iter().chain(new_aggr_expr.iter()); - let necessary_indices = - indices_referred_by_exprs(&aggregate.input, all_exprs_iter)?; + let schema = aggregate.input.schema(); + let necessary_indices = indices_referred_by_exprs(schema, all_exprs_iter)?; let aggregate_input = if let Some(input) = - optimize_projections(&aggregate.input, _config, &necessary_indices)? + optimize_projections(&aggregate.input, config, &necessary_indices)? { input } else { aggregate.input.as_ref().clone() }; - // Simplify input of the aggregation by adding a projection so that its input only contains - // absolutely necessary columns for the aggregate expressions. Please no that we use aggregate.input.schema() - // because necessary_indices refers to fields in this schema. - let necessary_exprs = - get_required_exprs(aggregate.input.schema(), &necessary_indices); - let (aggregate_input, _is_added) = - add_projection_on_top_if_helpful(aggregate_input, necessary_exprs, true)?; - - // Aggregate always needs at least one aggregate expression. - // With a nested count we don't require any column as input, but still need to create a correct aggregate - // The aggregate may be optimized out later (select count(*) from (select count(*) from [...]) always returns 1 + // Simplify the input of the aggregation by adding a projection so + // that its input only contains absolutely necessary columns for + // the aggregate expressions. Note that necessary_indices refer to + // fields in `aggregate.input.schema()`. + let necessary_exprs = get_required_exprs(schema, &necessary_indices); + let (aggregate_input, _) = + add_projection_on_top_if_helpful(aggregate_input, necessary_exprs)?; + + // Aggregations always need at least one aggregate expression. + // With a nested count, we don't require any column as input, but + // still need to create a correct aggregate, which may be optimized + // out later. As an example, consider the following query: + // + // SELECT COUNT(*) FROM (SELECT COUNT(*) FROM [...]) + // + // which always returns 1. if new_aggr_expr.is_empty() && new_group_bys.is_empty() && !aggregate.aggr_expr.is_empty() @@ -223,7 +251,8 @@ fn optimize_projections( new_aggr_expr = vec![aggregate.aggr_expr[0].clone()]; } - // Create new aggregate plan with updated input, and absolutely necessary fields. + // Create a new aggregate plan with the updated input and only the + // absolutely necessary fields: return Aggregate::try_new( Arc::new(aggregate_input), new_group_bys, @@ -232,43 +261,48 @@ fn optimize_projections( .map(|aggregate| Some(LogicalPlan::Aggregate(aggregate))); } LogicalPlan::Window(window) => { - // Split parent requirements to child and window expression sections. + // Split parent requirements to child and window expression sections: let n_input_fields = window.input.schema().fields().len(); let (child_reqs, mut window_reqs): (Vec, Vec) = indices.iter().partition(|&&idx| idx < n_input_fields); - // Offset window expr indices so that they point to valid indices at the `window.window_expr` - window_reqs - .iter_mut() - .for_each(|idx| *idx -= n_input_fields); + // Offset window expression indices so that they point to valid + // indices at `window.window_expr`: + for idx in window_reqs.iter_mut() { + *idx -= n_input_fields; + } - // Only use window expressions that are absolutely necessary by parent requirements. + // Only use window expressions that are absolutely necessary according + // to parent requirements: let new_window_expr = get_at_indices(&window.window_expr, &window_reqs); - // All of the required column indices at the input of the window by parent, and window expression requirements. + // Get all the required column indices at the input, either by the + // parent or window expression requirements. let required_indices = get_all_required_indices( &child_reqs, &window.input, new_window_expr.iter(), )?; let window_child = if let Some(new_window_child) = - optimize_projections(&window.input, _config, &required_indices)? + optimize_projections(&window.input, config, &required_indices)? { new_window_child } else { window.input.as_ref().clone() }; - // When no window expression is necessary, just use window input. (Remove window operator) + return if new_window_expr.is_empty() { + // When no window expression is necessary, use the input directly: Ok(Some(window_child)) } else { // Calculate required expressions at the input of the window. - // Please note that we use `old_child`, because `required_indices` refers to `old_child`. + // Please note that we use `old_child`, because `required_indices` + // refers to `old_child`. let required_exprs = get_required_exprs(window.input.schema(), &required_indices); - let (window_child, _is_added) = - add_projection_on_top_if_helpful(window_child, required_exprs, true)?; - let window = Window::try_new(new_window_expr, Arc::new(window_child))?; - Ok(Some(LogicalPlan::Window(window))) + let (window_child, _) = + add_projection_on_top_if_helpful(window_child, required_exprs)?; + Window::try_new(new_window_expr, Arc::new(window_child)) + .map(|window| Some(LogicalPlan::Window(window))) }; } LogicalPlan::Join(join) => { @@ -280,136 +314,137 @@ fn optimize_projections( get_all_required_indices(&left_req_indices, &join.left, exprs.iter())?; let right_indices = get_all_required_indices(&right_req_indices, &join.right, exprs.iter())?; - // Join benefits from small columns numbers at its input (decreases memory usage) - // Hence each child benefits from projection. - Some(vec![(left_indices, true), (right_indices, true)]) + // Joins benefit from "small" input tables (lower memory usage). + // Therefore, each child benefits from projection: + vec![(left_indices, true), (right_indices, true)] } LogicalPlan::CrossJoin(cross_join) => { let left_len = cross_join.left.schema().fields().len(); let (left_child_indices, right_child_indices) = split_join_requirements(left_len, indices, &JoinType::Inner); - // Join benefits from small columns numbers at its input (decreases memory usage) - // Hence each child benefits from projection. - Some(vec![ - (left_child_indices, true), - (right_child_indices, true), - ]) + // Joins benefit from "small" input tables (lower memory usage). + // Therefore, each child benefits from projection: + vec![(left_child_indices, true), (right_child_indices, true)] } LogicalPlan::TableScan(table_scan) => { - let projection_fields = table_scan.projected_schema.fields(); let schema = table_scan.source.schema(); - // We expect to find all of the required indices of the projected schema fields. - // among original schema. If at least one of them cannot be found. Use all of the fields in the file. - // (No projection at the source) - let projection = indices - .iter() - .map(|&idx| { - schema.fields().iter().position(|field_source| { - projection_fields[idx].field() == field_source - }) - }) - .collect::>>(); + // Get indices referred to in the original (schema with all fields) + // given projected indices. + let projection = with_indices(&table_scan.projection, schema, |map| { + indices.iter().map(|&idx| map[idx]).collect() + }); - return Ok(Some(LogicalPlan::TableScan(TableScan::try_new( + return TableScan::try_new( table_scan.table_name.clone(), table_scan.source.clone(), - projection, + Some(projection), table_scan.filters.clone(), table_scan.fetch, - )?))); + ) + .map(|table| Some(LogicalPlan::TableScan(table))); } }; - let child_required_indices = - if let Some(child_required_indices) = child_required_indices { - child_required_indices - } else { - // Stop iteration, cannot propagate requirement down below this operator. - return Ok(None); - }; - let new_inputs = izip!(child_required_indices, plan.inputs().into_iter()) .map(|((required_indices, projection_beneficial), child)| { - let (input, mut is_changed) = if let Some(new_input) = - optimize_projections(child, _config, &required_indices)? + let (input, is_changed) = if let Some(new_input) = + optimize_projections(child, config, &required_indices)? { (new_input, true) } else { (child.clone(), false) }; let project_exprs = get_required_exprs(child.schema(), &required_indices); - let (input, is_projection_added) = add_projection_on_top_if_helpful( - input, - project_exprs, - projection_beneficial, - )?; - is_changed |= is_projection_added; - Ok(is_changed.then_some(input)) + let (input, proj_added) = if projection_beneficial { + add_projection_on_top_if_helpful(input, project_exprs)? + } else { + (input, false) + }; + Ok((is_changed || proj_added).then_some(input)) }) - .collect::>>>()?; - // All of the children are same in this case, no need to change plan + .collect::>>()?; if new_inputs.iter().all(|child| child.is_none()) { + // All children are the same in this case, no need to change the plan: Ok(None) } else { - // At least one of the children is changed. + // At least one of the children is changed: let new_inputs = izip!(new_inputs, plan.inputs()) - // If new_input is `None`, this means child is not changed. Hence use `old_child` during construction. + // If new_input is `None`, this means child is not changed, so use + // `old_child` during construction: .map(|(new_input, old_child)| new_input.unwrap_or_else(|| old_child.clone())) .collect::>(); - let res = plan.with_new_inputs(&new_inputs)?; - Ok(Some(res)) + plan.with_new_inputs(&new_inputs).map(Some) } } -/// Merge Consecutive Projections +/// This function applies the given function `f` to the projection indices +/// `proj_indices` if they exist. Otherwise, applies `f` to a default set +/// of indices according to `schema`. +fn with_indices( + proj_indices: &Option>, + schema: SchemaRef, + mut f: F, +) -> Vec +where + F: FnMut(&[usize]) -> Vec, +{ + match proj_indices { + Some(indices) => f(indices.as_slice()), + None => { + let range: Vec = (0..schema.fields.len()).collect(); + f(range.as_slice()) + } + } +} + +/// Merges consecutive projections. /// /// Given a projection `proj`, this function attempts to merge it with a previous -/// projection if it exists and if the merging is beneficial. Merging is considered -/// beneficial when expressions in the current projection are non-trivial and referred to -/// more than once in its input fields. This can act as a caching mechanism for non-trivial -/// computations. +/// projection if it exists and if merging is beneficial. Merging is considered +/// beneficial when expressions in the current projection are non-trivial and +/// appear more than once in its input fields. This can act as a caching mechanism +/// for non-trivial computations. /// -/// # Arguments +/// # Parameters /// /// * `proj` - A reference to the `Projection` to be merged. /// /// # Returns /// -/// A `Result` containing an `Option` of the merged `Projection`. If merging is not beneficial -/// it returns `Ok(None)`. +/// A `Result` object with the following semantics: +/// +/// - `Ok(Some(Projection))`: Merge was beneficial and successful. Contains the +/// merged projection. +/// - `Ok(None)`: Signals that merge is not beneficial (and has not taken place). +/// - `Err(error)`: An error occured during the function call. fn merge_consecutive_projections(proj: &Projection) -> Result> { - let prev_projection = if let LogicalPlan::Projection(prev) = proj.input.as_ref() { - prev - } else { + let LogicalPlan::Projection(prev_projection) = proj.input.as_ref() else { return Ok(None); }; - // Count usages (referral counts) of each projection expression in its input fields - let column_referral_map: HashMap = proj - .expr - .iter() - .flat_map(|expr| expr.to_columns()) - .fold(HashMap::new(), |mut map, cols| { - cols.into_iter() - .for_each(|col| *map.entry(col).or_default() += 1); - map - }); - - // Merging these projections is not beneficial, e.g - // If an expression is not trivial and it is referred more than 1, consecutive projections will be - // beneficial as caching mechanism for non-trivial computations. - // See discussion in: https://github.com/apache/arrow-datafusion/issues/8296 - if column_referral_map.iter().any(|(col, usage)| { - *usage > 1 + // Count usages (referrals) of each projection expression in its input fields: + let mut column_referral_map = HashMap::::new(); + for columns in proj.expr.iter().flat_map(|expr| expr.to_columns()) { + for col in columns.into_iter() { + *column_referral_map.entry(col.clone()).or_default() += 1; + } + } + + // If an expression is non-trivial and appears more than once, consecutive + // projections will benefit from a compute-once approach. For details, see: + // https://github.com/apache/arrow-datafusion/issues/8296 + if column_referral_map.into_iter().any(|(col, usage)| { + usage > 1 && !is_expr_trivial( &prev_projection.expr - [prev_projection.schema.index_of_column(col).unwrap()], + [prev_projection.schema.index_of_column(&col).unwrap()], ) }) { return Ok(None); } - // If all of the expression of the top projection can be rewritten. Rewrite expressions and create a new projection + // If all the expression of the top projection can be rewritten, do so and + // create a new projection: let new_exprs = proj .expr .iter() @@ -429,183 +464,252 @@ fn merge_consecutive_projections(proj: &Projection) -> Result } } -/// Trim Expression -/// -/// Trim the given expression by removing any unnecessary layers of abstraction. +/// Trim the given expression by removing any unnecessary layers of aliasing. /// If the expression is an alias, the function returns the underlying expression. -/// Otherwise, it returns the original expression unchanged. -/// -/// # Arguments +/// Otherwise, it returns the given expression as is. /// -/// * `expr` - The input expression to be trimmed. +/// Without trimming, we can end up with unnecessary indirections inside expressions +/// during projection merges. /// -/// # Returns -/// -/// The trimmed expression. If the input is an alias, the underlying expression is returned. -/// -/// Without trimming, during projection merge we can end up unnecessary indirections inside the expressions. /// Consider: /// -/// Projection (a1 + b1 as sum1) -/// --Projection (a as a1, b as b1) -/// ----Source (a, b) +/// ```text +/// Projection(a1 + b1 as sum1) +/// --Projection(a as a1, b as b1) +/// ----Source(a, b) +/// ``` /// -/// After merge we want to produce +/// After merge, we want to produce: /// -/// Projection (a + b as sum1) +/// ```text +/// Projection(a + b as sum1) /// --Source(a, b) +/// ``` /// -/// Without trimming we would end up +/// Without trimming, we would end up with: /// -/// Projection (a as a1 + b as b1 as sum1) +/// ```text +/// Projection((a as a1 + b as b1) as sum1) /// --Source(a, b) +/// ``` fn trim_expr(expr: Expr) -> Expr { match expr { - Expr::Alias(alias) => *alias.expr, + Expr::Alias(alias) => trim_expr(*alias.expr), _ => expr, } } -// Check whether expression is trivial (e.g it doesn't include computation.) +// Check whether `expr` is trivial; i.e. it doesn't imply any computation. fn is_expr_trivial(expr: &Expr) -> bool { matches!(expr, Expr::Column(_) | Expr::Literal(_)) } -// Exit early when None is seen. +// Exit early when there is no rewrite to do. macro_rules! rewrite_expr_with_check { ($expr:expr, $input:expr) => { - if let Some(val) = rewrite_expr($expr, $input)? { - val + if let Some(value) = rewrite_expr($expr, $input)? { + value } else { return Ok(None); } }; } -// Rewrites expression using its input projection (Merges consecutive projection expressions). -/// Rewrites an projections expression using its input projection -/// (Helper during merging consecutive projection expressions). +/// Rewrites a projection expression using the projection before it (i.e. its input) +/// This is a subroutine to the `merge_consecutive_projections` function. /// -/// # Arguments +/// # Parameters /// -/// * `expr` - A reference to the expression to be rewritten. -/// * `input` - A reference to the input (itself a projection) of the projection expression. +/// * `expr` - A reference to the expression to rewrite. +/// * `input` - A reference to the input of the projection expression (itself +/// a projection). /// /// # Returns /// -/// A `Result` containing an `Option` of the rewritten expression. If the rewrite is successful, -/// it returns `Ok(Some)` with the modified expression. If the expression cannot be rewritten -/// it returns `Ok(None)`. +/// A `Result` object with the following semantics: +/// +/// - `Ok(Some(Expr))`: Rewrite was successful. Contains the rewritten result. +/// - `Ok(None)`: Signals that `expr` can not be rewritten. +/// - `Err(error)`: An error occured during the function call. fn rewrite_expr(expr: &Expr, input: &Projection) -> Result> { - Ok(match expr { + let result = match expr { Expr::Column(col) => { - // Find index of column + // Find index of column: let idx = input.schema.index_of_column(col)?; - Some(input.expr[idx].clone()) + input.expr[idx].clone() } - Expr::BinaryExpr(binary) => { - let lhs = trim_expr(rewrite_expr_with_check!(&binary.left, input)); - let rhs = trim_expr(rewrite_expr_with_check!(&binary.right, input)); - Some(Expr::BinaryExpr(BinaryExpr::new( - Box::new(lhs), - binary.op, - Box::new(rhs), - ))) - } - Expr::Alias(alias) => { - let new_expr = trim_expr(rewrite_expr_with_check!(&alias.expr, input)); - Some(Expr::Alias(Alias::new( - new_expr, - alias.relation.clone(), - alias.name.clone(), - ))) - } - Expr::Literal(_val) => Some(expr.clone()), + Expr::BinaryExpr(binary) => Expr::BinaryExpr(BinaryExpr::new( + Box::new(trim_expr(rewrite_expr_with_check!(&binary.left, input))), + binary.op, + Box::new(trim_expr(rewrite_expr_with_check!(&binary.right, input))), + )), + Expr::Alias(alias) => Expr::Alias(Alias::new( + trim_expr(rewrite_expr_with_check!(&alias.expr, input)), + alias.relation.clone(), + alias.name.clone(), + )), + Expr::Literal(_) => expr.clone(), Expr::Cast(cast) => { let new_expr = rewrite_expr_with_check!(&cast.expr, input); - Some(Expr::Cast(Cast::new( - Box::new(new_expr), - cast.data_type.clone(), - ))) + Expr::Cast(Cast::new(Box::new(new_expr), cast.data_type.clone())) } Expr::ScalarFunction(scalar_fn) => { - let fun = if let ScalarFunctionDefinition::BuiltIn(fun) = scalar_fn.func_def { - fun - } else { + // TODO: Support UDFs. + let ScalarFunctionDefinition::BuiltIn(fun) = scalar_fn.func_def else { return Ok(None); }; - scalar_fn + return Ok(scalar_fn .args .iter() .map(|expr| rewrite_expr(expr, input)) - .collect::>>>()? - .map(|new_args| Expr::ScalarFunction(ScalarFunction::new(fun, new_args))) + .collect::>>()? + .map(|new_args| { + Expr::ScalarFunction(ScalarFunction::new(fun, new_args)) + })); } - _ => { - // Unsupported type to merge in consecutive projections - None - } - }) + // Unsupported type for consecutive projection merge analysis. + _ => return Ok(None), + }; + Ok(Some(result)) } -/// Retrieves a set of outer-referenced columns from an expression. -/// Please note that `expr.to_columns()` API doesn't return these columns. +/// Retrieves a set of outer-referenced columns by the given expression, `expr`. +/// Note that the `Expr::to_columns()` function doesn't return these columns. /// -/// # Arguments +/// # Parameters /// -/// * `expr` - The expression to be analyzed for outer-referenced columns. +/// * `expr` - The expression to analyze for outer-referenced columns. /// /// # Returns /// -/// A `HashSet` containing columns that are referenced by the expression. -fn outer_columns(expr: &Expr) -> HashSet { +/// If the function can safely infer all outer-referenced columns, returns a +/// `Some(HashSet)` containing these columns. Otherwise, returns `None`. +fn outer_columns(expr: &Expr) -> Option> { let mut columns = HashSet::new(); - outer_columns_helper(expr, &mut columns); - columns + outer_columns_helper(expr, &mut columns).then_some(columns) } -/// Helper function to accumulate outer-referenced columns referred by the `expr`. +/// A recursive subroutine that accumulates outer-referenced columns by the +/// given expression, `expr`. /// -/// # Arguments +/// # Parameters /// -/// * `expr` - The expression to be analyzed for outer-referenced columns. -/// * `columns` - A mutable reference to a `HashSet` where the detected columns are collected. -fn outer_columns_helper(expr: &Expr, columns: &mut HashSet) { +/// * `expr` - The expression to analyze for outer-referenced columns. +/// * `columns` - A mutable reference to a `HashSet` where detected +/// columns are collected. +/// +/// Returns `true` if it can safely collect all outer-referenced columns. +/// Otherwise, returns `false`. +fn outer_columns_helper(expr: &Expr, columns: &mut HashSet) -> bool { match expr { Expr::OuterReferenceColumn(_, col) => { columns.insert(col.clone()); + true } Expr::BinaryExpr(binary_expr) => { - outer_columns_helper(&binary_expr.left, columns); - outer_columns_helper(&binary_expr.right, columns); + outer_columns_helper(&binary_expr.left, columns) + && outer_columns_helper(&binary_expr.right, columns) } Expr::ScalarSubquery(subquery) => { - for expr in &subquery.outer_ref_columns { - outer_columns_helper(expr, columns); - } + let exprs = subquery.outer_ref_columns.iter(); + outer_columns_helper_multi(exprs, columns) } Expr::Exists(exists) => { - for expr in &exists.subquery.outer_ref_columns { - outer_columns_helper(expr, columns); + let exprs = exists.subquery.outer_ref_columns.iter(); + outer_columns_helper_multi(exprs, columns) + } + Expr::Alias(alias) => outer_columns_helper(&alias.expr, columns), + Expr::InSubquery(insubquery) => { + let exprs = insubquery.subquery.outer_ref_columns.iter(); + outer_columns_helper_multi(exprs, columns) + } + Expr::IsNotNull(expr) | Expr::IsNull(expr) => outer_columns_helper(expr, columns), + Expr::Cast(cast) => outer_columns_helper(&cast.expr, columns), + Expr::Sort(sort) => outer_columns_helper(&sort.expr, columns), + Expr::AggregateFunction(aggregate_fn) => { + outer_columns_helper_multi(aggregate_fn.args.iter(), columns) + && aggregate_fn + .order_by + .as_ref() + .map_or(true, |obs| outer_columns_helper_multi(obs.iter(), columns)) + && aggregate_fn + .filter + .as_ref() + .map_or(true, |filter| outer_columns_helper(filter, columns)) + } + Expr::WindowFunction(window_fn) => { + outer_columns_helper_multi(window_fn.args.iter(), columns) + && outer_columns_helper_multi(window_fn.order_by.iter(), columns) + && outer_columns_helper_multi(window_fn.partition_by.iter(), columns) + } + Expr::GroupingSet(groupingset) => match groupingset { + GroupingSet::GroupingSets(multi_exprs) => multi_exprs + .iter() + .all(|e| outer_columns_helper_multi(e.iter(), columns)), + GroupingSet::Cube(exprs) | GroupingSet::Rollup(exprs) => { + outer_columns_helper_multi(exprs.iter(), columns) } + }, + Expr::ScalarFunction(scalar_fn) => { + outer_columns_helper_multi(scalar_fn.args.iter(), columns) } - Expr::Alias(alias) => { - outer_columns_helper(&alias.expr, columns); + Expr::Like(like) => { + outer_columns_helper(&like.expr, columns) + && outer_columns_helper(&like.pattern, columns) } - _ => {} + Expr::InList(in_list) => { + outer_columns_helper(&in_list.expr, columns) + && outer_columns_helper_multi(in_list.list.iter(), columns) + } + Expr::Case(case) => { + let when_then_exprs = case + .when_then_expr + .iter() + .flat_map(|(first, second)| [first.as_ref(), second.as_ref()]); + outer_columns_helper_multi(when_then_exprs, columns) + && case + .expr + .as_ref() + .map_or(true, |expr| outer_columns_helper(expr, columns)) + && case + .else_expr + .as_ref() + .map_or(true, |expr| outer_columns_helper(expr, columns)) + } + Expr::Column(_) | Expr::Literal(_) | Expr::Wildcard { .. } => true, + _ => false, } } -/// Generates the required expressions(Column) that resides at `indices` of the `input_schema`. +/// 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. +/// +/// Returns `true` if it can safely collect all outer-referenced columns. +/// Otherwise, returns `false`. +fn outer_columns_helper_multi<'a>( + mut exprs: impl Iterator, + columns: &mut HashSet, +) -> bool { + exprs.all(|e| outer_columns_helper(e, columns)) +} + +/// Generates the required expressions (columns) that reside at `indices` of +/// the given `input_schema`. /// /// # Arguments /// /// * `input_schema` - A reference to the input schema. -/// * `indices` - A slice of `usize` indices specifying which columns are required. +/// * `indices` - A slice of `usize` indices specifying required columns. /// /// # Returns /// -/// A vector of `Expr::Column` expressions, that sits at `indices` of the `input_schema`. +/// A vector of `Expr::Column` expressions residing at `indices` of the `input_schema`. fn get_required_exprs(input_schema: &Arc, indices: &[usize]) -> Vec { let fields = input_schema.fields(); indices @@ -614,58 +718,70 @@ fn get_required_exprs(input_schema: &Arc, indices: &[usize]) -> Vec>( - input: &LogicalPlan, - exprs: I, +/// A [`Result`] object containing the indices of all required fields in +/// `input_schema` to calculate all `exprs` successfully. +fn indices_referred_by_exprs<'a>( + input_schema: &DFSchemaRef, + exprs: impl Iterator, ) -> Result> { - let new_indices = exprs - .flat_map(|expr| indices_referred_by_expr(input.schema(), expr)) + let indices = exprs + .map(|expr| indices_referred_by_expr(input_schema, expr)) + .collect::>>()?; + Ok(indices + .into_iter() .flatten() - // Make sure no duplicate entries exists and indices are ordered. + // Make sure no duplicate entries exist and indices are ordered: .sorted() .dedup() - .collect::>(); - Ok(new_indices) + .collect()) } -/// Get indices of the necessary fields referred by the `expr` among input schema. +/// Get indices of the fields referred to by the given expression `expr` within +/// the given schema (`input_schema`). /// -/// # Arguments +/// # Parameters /// -/// * `input_schema`: The input schema to search for indices referred by expr. -/// * `expr`: An expression for which we want to find necessary field indices at the input schema. +/// * `input_schema`: The input schema to analyze for index requirements. +/// * `expr`: An expression for which we want to find necessary field indices. /// /// # Returns /// -/// A [Result] object that contains the required field indices of the `input_schema`, to be able to calculate -/// the `expr` successfully. +/// A [`Result`] object containing the indices of all required fields in +/// `input_schema` to calculate `expr` successfully. fn indices_referred_by_expr( input_schema: &DFSchemaRef, expr: &Expr, ) -> Result> { let mut cols = expr.to_columns()?; - // Get outer referenced columns (expr.to_columns() doesn't return these columns). - cols.extend(outer_columns(expr)); - cols.iter() - .filter(|&col| input_schema.has_column(col)) - .map(|col| input_schema.index_of_column(col)) - .collect::>>() + // Get outer-referenced columns: + if let Some(outer_cols) = outer_columns(expr) { + cols.extend(outer_cols); + } else { + // Expression is not known to contain outer columns or not. Hence, do + // not assume anything and require all the schema indices at the input: + return Ok((0..input_schema.fields().len()).collect()); + } + Ok(cols + .iter() + .flat_map(|col| input_schema.index_of_column(col)) + .collect()) } -/// Get all required indices for the input (indices required by parent + indices referred by `exprs`) +/// Gets all required indices for the input; i.e. those required by the parent +/// and those referred to by `exprs`. /// -/// # Arguments +/// # Parameters /// /// * `parent_required_indices` - A slice of indices required by the parent plan. /// * `input` - The input logical plan to analyze for index requirements. @@ -673,30 +789,28 @@ fn indices_referred_by_expr( /// /// # Returns /// -/// A `Result` containing a vector of `usize` indices containing all required indices. -fn get_all_required_indices<'a, I: Iterator>( +/// A `Result` containing a vector of `usize` indices containing all the required +/// indices. +fn get_all_required_indices<'a>( parent_required_indices: &[usize], input: &LogicalPlan, - exprs: I, + exprs: impl Iterator, ) -> Result> { - let referred_indices = indices_referred_by_exprs(input, exprs)?; - Ok(merge_vectors(parent_required_indices, &referred_indices)) + indices_referred_by_exprs(input.schema(), exprs) + .map(|indices| merge_slices(parent_required_indices, &indices)) } -/// Retrieves a list of expressions at specified indices from a slice of expressions. +/// Retrieves the expressions at specified indices within the given slice. Ignores +/// any invalid indices. /// -/// This function takes a slice of expressions `exprs` and a slice of `usize` indices `indices`. -/// It returns a new vector containing the expressions from `exprs` that correspond to the provided indices (with bound check). +/// # Parameters /// -/// # Arguments -/// -/// * `exprs` - A slice of expressions from which expressions are to be retrieved. -/// * `indices` - A slice of `usize` indices specifying the positions of the expressions to be retrieved. +/// * `exprs` - A slice of expressions to index into. +/// * `indices` - A slice of indices specifying the positions of expressions sought. /// /// # Returns /// -/// A vector of expressions that correspond to the specified indices. If any index is out of bounds, -/// the associated expression is skipped in the result. +/// A vector of expressions corresponding to specified indices. fn get_at_indices(exprs: &[Expr], indices: &[usize]) -> Vec { indices .iter() @@ -705,158 +819,148 @@ fn get_at_indices(exprs: &[Expr], indices: &[usize]) -> Vec { .collect() } -/// Merges two slices of `usize` values into a single vector with sorted (ascending) and deduplicated elements. -/// -/// # Arguments -/// -/// * `lhs` - The first slice of `usize` values to be merged. -/// * `rhs` - The second slice of `usize` values to be merged. -/// -/// # Returns -/// -/// A vector of `usize` values containing the merged, sorted, and deduplicated elements from `lhs` and `rhs`. -/// As an example merge of [3, 2, 4] and [3, 6, 1] will produce [1, 2, 3, 6] -fn merge_vectors(lhs: &[usize], rhs: &[usize]) -> Vec { - let mut merged = lhs.to_vec(); - merged.extend(rhs); - // Make sure to run sort before dedup. - // Dedup removes consecutive same entries - // If sort is run before it, all duplicates are removed. - merged.sort(); - merged.dedup(); - merged +/// Merges two slices into a single vector with sorted (ascending) and +/// deduplicated elements. For example, merging `[3, 2, 4]` and `[3, 6, 1]` +/// will produce `[1, 2, 3, 6]`. +fn merge_slices(left: &[T], right: &[T]) -> Vec { + // Make sure to sort before deduping, which removes the duplicates: + left.iter() + .cloned() + .chain(right.iter().cloned()) + .sorted() + .dedup() + .collect() } -/// Splits requirement indices for a join into left and right children based on the join type. +/// Splits requirement indices for a join into left and right children based on +/// the join type. /// -/// This function takes the length of the left child, a slice of requirement indices, and the type -/// of join (e.g., INNER, LEFT, RIGHT, etc.) as arguments. Depending on the join type, it divides -/// the requirement indices into those that apply to the left child and those that apply to the right child. +/// This function takes the length of the left child, a slice of requirement +/// indices, and the type of join (e.g. `INNER`, `LEFT`, `RIGHT`) as arguments. +/// Depending on the join type, it divides the requirement indices into those +/// that apply to the left child and those that apply to the right child. /// -/// - For INNER, LEFT, RIGHT, and FULL joins, the requirements are split between left and right children. -/// The right child indices are adjusted to point to valid positions in the right child by subtracting -/// the length of the left child. +/// - For `INNER`, `LEFT`, `RIGHT` and `FULL` joins, the requirements are split +/// between left and right children. The right child indices are adjusted to +/// point to valid positions within the right child by subtracting the length +/// of the left child. /// -/// - For LEFT ANTI, LEFT SEMI, RIGHT SEMI, and RIGHT ANTI joins, all requirements are re-routed to either -/// the left child or the right child directly, depending on the join type. +/// - For `LEFT ANTI`, `LEFT SEMI`, `RIGHT SEMI` and `RIGHT ANTI` joins, all +/// requirements are re-routed to either the left child or the right child +/// directly, depending on the join type. /// -/// # Arguments +/// # Parameters /// /// * `left_len` - The length of the left child. /// * `indices` - A slice of requirement indices. -/// * `join_type` - The type of join (e.g., INNER, LEFT, RIGHT, etc.). +/// * `join_type` - The type of join (e.g. `INNER`, `LEFT`, `RIGHT`). /// /// # Returns /// -/// A tuple containing two vectors of `usize` indices: the first vector represents the requirements for -/// the left child, and the second vector represents the requirements for the right child. The indices -/// are appropriately split and adjusted based on the join type. +/// A tuple containing two vectors of `usize` indices: The first vector represents +/// the requirements for the left child, and the second vector represents the +/// requirements for the right child. The indices are appropriately split and +/// adjusted based on the join type. fn split_join_requirements( left_len: usize, indices: &[usize], join_type: &JoinType, ) -> (Vec, Vec) { match join_type { - // In these cases requirements split to left and right child. + // In these cases requirements are split between left/right children: JoinType::Inner | JoinType::Left | JoinType::Right | JoinType::Full => { - let (left_child_reqs, mut right_child_reqs): (Vec, Vec) = + let (left_reqs, mut right_reqs): (Vec, Vec) = indices.iter().partition(|&&idx| idx < left_len); - // Decrease right side index by `left_len` so that they point to valid positions in the right child. - right_child_reqs.iter_mut().for_each(|idx| *idx -= left_len); - (left_child_reqs, right_child_reqs) + // Decrease right side indices by `left_len` so that they point to valid + // positions within the right child: + for idx in right_reqs.iter_mut() { + *idx -= left_len; + } + (left_reqs, right_reqs) } // All requirements can be re-routed to left child directly. JoinType::LeftAnti | JoinType::LeftSemi => (indices.to_vec(), vec![]), - // All requirements can be re-routed to right side directly. (No need to change index, join schema is right child schema.) + // All requirements can be re-routed to right side directly. + // No need to change index, join schema is right child schema. JoinType::RightSemi | JoinType::RightAnti => (vec![], indices.to_vec()), } } -/// Adds a projection on top of a logical plan if it is beneficial and reduces the number of columns for the parent operator. +/// Adds a projection on top of a logical plan if doing so reduces the number +/// of columns for the parent operator. /// -/// This function takes a `LogicalPlan`, a list of projection expressions, and a flag indicating whether -/// the projection is beneficial. If the projection is beneficial and reduces the number of columns in -/// the plan, a new `LogicalPlan` with the projection is created and returned, along with a `true` flag. -/// If the projection is unnecessary or doesn't reduce the number of columns, the original plan is returned -/// with a `false` flag. +/// This function takes a `LogicalPlan` and a list of projection expressions. +/// If the projection is beneficial (it reduces the number of columns in the +/// plan) a new `LogicalPlan` with the projection is created and returned, along +/// with a `true` flag. If the projection doesn't reduce the number of columns, +/// the original plan is returned with a `false` flag. /// -/// # Arguments +/// # Parameters /// /// * `plan` - The input `LogicalPlan` to potentially add a projection to. /// * `project_exprs` - A list of expressions for the projection. -/// * `projection_beneficial` - A flag indicating whether the projection is beneficial. /// /// # Returns /// -/// A `Result` containing a tuple with two values: the resulting `LogicalPlan` (with or without -/// the added projection) and a `bool` flag indicating whether the projection was added (`true`) or not (`false`). +/// A `Result` containing a tuple with two values: The resulting `LogicalPlan` +/// (with or without the added projection) and a `bool` flag indicating if a +/// projection was added (`true`) or not (`false`). fn add_projection_on_top_if_helpful( plan: LogicalPlan, project_exprs: Vec, - projection_beneficial: bool, ) -> Result<(LogicalPlan, bool)> { - // Make sure projection decreases table column size, otherwise it is unnecessary. - if !projection_beneficial || project_exprs.len() >= plan.schema().fields().len() { + // Make sure projection decreases the number of columns, otherwise it is unnecessary. + if project_exprs.len() >= plan.schema().fields().len() { Ok((plan, false)) } else { - let new_plan = Projection::try_new(project_exprs, Arc::new(plan)) - .map(LogicalPlan::Projection)?; - Ok((new_plan, true)) + Projection::try_new(project_exprs, Arc::new(plan)) + .map(|proj| (LogicalPlan::Projection(proj), true)) } } -/// Collects and returns a vector of all indices of the fields in the schema of a logical plan. +/// Rewrite the given projection according to the fields required by its +/// ancestors. /// -/// # Arguments +/// # Parameters /// -/// * `plan` - A reference to the `LogicalPlan` for which indices are required. +/// * `proj` - A reference to the original projection to rewrite. +/// * `config` - A reference to the optimizer configuration. +/// * `indices` - A slice of indices representing the columns required by the +/// ancestors of the given projection. /// /// # Returns /// -/// A vector of `usize` indices representing all fields in the schema of the provided logical plan. -fn require_all_indices(plan: &LogicalPlan) -> Vec { - (0..plan.schema().fields().len()).collect() -} - -/// Rewrite Projection Given Required fields by its parent(s). -/// -/// # Arguments -/// -/// * `proj` - A reference to the original projection to be rewritten. -/// * `_config` - A reference to the optimizer configuration (unused in the function). -/// * `indices` - A slice of indices representing the required columns by the parent(s) of projection. -/// -/// # Returns +/// A `Result` object with the following semantics: /// -/// A `Result` containing an `Option` of the rewritten logical plan. If the -/// rewrite is successful, it returns `Some` with the optimized logical plan. -/// If the logical plan remains unchanged it returns `Ok(None)`. +/// - `Ok(Some(LogicalPlan))`: Contains the rewritten projection +/// - `Ok(None)`: No rewrite necessary. +/// - `Err(error)`: An error occured during the function call. fn rewrite_projection_given_requirements( proj: &Projection, - _config: &dyn OptimizerConfig, + config: &dyn OptimizerConfig, indices: &[usize], ) -> Result> { let exprs_used = get_at_indices(&proj.expr, indices); - let required_indices = indices_referred_by_exprs(&proj.input, exprs_used.iter())?; + let required_indices = + indices_referred_by_exprs(proj.input.schema(), exprs_used.iter())?; return if let Some(input) = - optimize_projections(&proj.input, _config, &required_indices)? + optimize_projections(&proj.input, config, &required_indices)? { if &projection_schema(&input, &exprs_used)? == input.schema() { Ok(Some(input)) } else { - let new_proj = Projection::try_new(exprs_used, Arc::new(input))?; - let new_proj = LogicalPlan::Projection(new_proj); - Ok(Some(new_proj)) + Projection::try_new(exprs_used, Arc::new(input)) + .map(|proj| Some(LogicalPlan::Projection(proj))) } } else if exprs_used.len() < proj.expr.len() { - // Projection expression used is different than the existing projection - // In this case, even if child doesn't change we should update projection to use less columns. + // Projection expression used is different than the existing projection. + // In this case, even if the child doesn't change, we should update the + // projection to use fewer columns: if &projection_schema(&proj.input, &exprs_used)? == proj.input.schema() { Ok(Some(proj.input.as_ref().clone())) } else { - let new_proj = Projection::try_new(exprs_used, proj.input.clone())?; - let new_proj = LogicalPlan::Projection(new_proj); - Ok(Some(new_proj)) + Projection::try_new(exprs_used, proj.input.clone()) + .map(|proj| Some(LogicalPlan::Projection(proj))) } } else { // Projection doesn't change. @@ -866,16 +970,16 @@ fn rewrite_projection_given_requirements( #[cfg(test)] mod tests { + use std::sync::Arc; + use crate::optimize_projections::OptimizeProjections; + use crate::test::{assert_optimized_plan_eq, test_table_scan}; use arrow::datatypes::{DataType, Field, Schema}; use datafusion_common::{Result, TableReference}; use datafusion_expr::{ binary_expr, col, count, lit, logical_plan::builder::LogicalPlanBuilder, table_scan, Expr, LogicalPlan, Operator, }; - use std::sync::Arc; - - use crate::test::*; fn assert_optimized_plan_equal(plan: &LogicalPlan, expected: &str) -> Result<()> { assert_optimized_plan_eq(Arc::new(OptimizeProjections::new()), plan, expected) @@ -920,6 +1024,20 @@ mod tests { \n TableScan: test projection=[a]"; assert_optimized_plan_equal(&plan, expected) } + + #[test] + fn merge_nested_alias() -> Result<()> { + let table_scan = test_table_scan()?; + let plan = LogicalPlanBuilder::from(table_scan) + .project(vec![col("a").alias("alias1").alias("alias2")])? + .project(vec![col("alias2").alias("alias")])? + .build()?; + + let expected = "Projection: test.a AS alias\ + \n TableScan: test projection=[a]"; + assert_optimized_plan_equal(&plan, expected) + } + #[test] fn test_nested_count() -> Result<()> { let schema = Schema::new(vec![Field::new("foo", DataType::Int32, false)]); diff --git a/datafusion/optimizer/src/optimizer.rs b/datafusion/optimizer/src/optimizer.rs index 7af46ed70adf..0dc34cb809eb 100644 --- a/datafusion/optimizer/src/optimizer.rs +++ b/datafusion/optimizer/src/optimizer.rs @@ -17,6 +17,10 @@ //! Query optimizer traits +use std::collections::HashSet; +use std::sync::Arc; +use std::time::Instant; + use crate::common_subexpr_eliminate::CommonSubexprEliminate; use crate::decorrelate_predicate_subquery::DecorrelatePredicateSubquery; use crate::eliminate_cross_join::EliminateCrossJoin; @@ -41,15 +45,14 @@ use crate::simplify_expressions::SimplifyExpressions; use crate::single_distinct_to_groupby::SingleDistinctToGroupBy; use crate::unwrap_cast_in_comparison::UnwrapCastInComparison; use crate::utils::log_plan; -use chrono::{DateTime, Utc}; + use datafusion_common::alias::AliasGenerator; use datafusion_common::config::ConfigOptions; use datafusion_common::{DataFusionError, Result}; -use datafusion_expr::LogicalPlan; +use datafusion_expr::logical_plan::LogicalPlan; + +use chrono::{DateTime, Utc}; use log::{debug, warn}; -use std::collections::HashSet; -use std::sync::Arc; -use std::time::Instant; /// `OptimizerRule` transforms one [`LogicalPlan`] into another which /// computes the same results, but in a potentially more efficient @@ -447,17 +450,18 @@ pub(crate) fn assert_schema_is_the_same( #[cfg(test)] mod tests { + use std::sync::{Arc, Mutex}; + + use super::ApplyOrder; use crate::optimizer::Optimizer; use crate::test::test_table_scan; use crate::{OptimizerConfig, OptimizerContext, OptimizerRule}; + use datafusion_common::{ plan_err, DFField, DFSchema, DFSchemaRef, DataFusionError, Result, }; use datafusion_expr::logical_plan::EmptyRelation; use datafusion_expr::{col, lit, LogicalPlan, LogicalPlanBuilder, Projection}; - use std::sync::{Arc, Mutex}; - - use super::ApplyOrder; #[test] fn skip_failing_rule() { diff --git a/datafusion/optimizer/tests/optimizer_integration.rs b/datafusion/optimizer/tests/optimizer_integration.rs index 4172881c0aad..d857c6154ea9 100644 --- a/datafusion/optimizer/tests/optimizer_integration.rs +++ b/datafusion/optimizer/tests/optimizer_integration.rs @@ -15,8 +15,11 @@ // specific language governing permissions and limitations // under the License. +use std::any::Any; +use std::collections::HashMap; +use std::sync::Arc; + use arrow::datatypes::{DataType, Field, Schema, SchemaRef, TimeUnit}; -use chrono::{DateTime, NaiveDateTime, Utc}; use datafusion_common::config::ConfigOptions; use datafusion_common::{plan_err, DataFusionError, Result}; use datafusion_expr::{AggregateUDF, LogicalPlan, ScalarUDF, TableSource, WindowUDF}; @@ -28,9 +31,8 @@ use datafusion_sql::sqlparser::ast::Statement; use datafusion_sql::sqlparser::dialect::GenericDialect; use datafusion_sql::sqlparser::parser::Parser; use datafusion_sql::TableReference; -use std::any::Any; -use std::collections::HashMap; -use std::sync::Arc; + +use chrono::{DateTime, NaiveDateTime, Utc}; #[cfg(test)] #[ctor::ctor] diff --git a/datafusion/sql/src/select.rs b/datafusion/sql/src/select.rs index 15f720d75652..a0819e4aaf8e 100644 --- a/datafusion/sql/src/select.rs +++ b/datafusion/sql/src/select.rs @@ -25,10 +25,7 @@ use crate::utils::{ }; use datafusion_common::Column; -use datafusion_common::{ - get_target_functional_dependencies, not_impl_err, plan_err, DFSchemaRef, - DataFusionError, Result, -}; +use datafusion_common::{not_impl_err, plan_err, DataFusionError, Result}; use datafusion_expr::expr::Alias; use datafusion_expr::expr_rewriter::{ normalize_col, normalize_col_with_schemas_and_ambiguity_check, @@ -534,14 +531,17 @@ impl<'a, S: ContextProvider> SqlToRel<'a, S> { group_by_exprs: &[Expr], aggr_exprs: &[Expr], ) -> Result<(LogicalPlan, Vec, Option)> { - let group_by_exprs = - get_updated_group_by_exprs(group_by_exprs, select_exprs, input.schema())?; - // create the aggregate plan let plan = LogicalPlanBuilder::from(input.clone()) - .aggregate(group_by_exprs.clone(), aggr_exprs.to_vec())? + .aggregate(group_by_exprs.to_vec(), aggr_exprs.to_vec())? .build()?; + let group_by_exprs = if let LogicalPlan::Aggregate(agg) = &plan { + &agg.group_expr + } else { + unreachable!(); + }; + // in this next section of code we are re-writing the projection to refer to columns // output by the aggregate plan. For example, if the projection contains the expression // `SUM(a)` then we replace that with a reference to a column `SUM(a)` produced by @@ -550,7 +550,7 @@ impl<'a, S: ContextProvider> SqlToRel<'a, S> { // combine the original grouping and aggregate expressions into one list (note that // we do not add the "having" expression since that is not part of the projection) let mut aggr_projection_exprs = vec![]; - for expr in &group_by_exprs { + for expr in group_by_exprs { match expr { Expr::GroupingSet(GroupingSet::Rollup(exprs)) => { aggr_projection_exprs.extend_from_slice(exprs) @@ -659,61 +659,3 @@ fn match_window_definitions( } Ok(()) } - -/// Update group by exprs, according to functional dependencies -/// The query below -/// -/// SELECT sn, amount -/// FROM sales_global -/// GROUP BY sn -/// -/// cannot be calculated, because it has a column(`amount`) which is not -/// part of group by expression. -/// However, if we know that, `sn` is determinant of `amount`. We can -/// safely, determine value of `amount` for each distinct `sn`. For these cases -/// we rewrite the query above as -/// -/// SELECT sn, amount -/// FROM sales_global -/// GROUP BY sn, amount -/// -/// Both queries, are functionally same. \[Because, (`sn`, `amount`) and (`sn`) -/// defines the identical groups. \] -/// This function updates group by expressions such that select expressions that are -/// not in group by expression, are added to the group by expressions if they are dependent -/// of the sub-set of group by expressions. -fn get_updated_group_by_exprs( - group_by_exprs: &[Expr], - select_exprs: &[Expr], - schema: &DFSchemaRef, -) -> Result> { - let mut new_group_by_exprs = group_by_exprs.to_vec(); - let fields = schema.fields(); - let group_by_expr_names = group_by_exprs - .iter() - .map(|group_by_expr| group_by_expr.display_name()) - .collect::>>()?; - // Get targets that can be used in a select, even if they do not occur in aggregation: - if let Some(target_indices) = - get_target_functional_dependencies(schema, &group_by_expr_names) - { - // Calculate dependent fields names with determinant GROUP BY expression: - let associated_field_names = target_indices - .iter() - .map(|idx| fields[*idx].qualified_name()) - .collect::>(); - // Expand GROUP BY expressions with select expressions: If a GROUP - // BY expression is a determinant key, we can use its dependent - // columns in select statements also. - for expr in select_exprs { - let expr_name = format!("{}", expr); - if !new_group_by_exprs.contains(expr) - && associated_field_names.contains(&expr_name) - { - new_group_by_exprs.push(expr.clone()); - } - } - } - - Ok(new_group_by_exprs) -} diff --git a/datafusion/sqllogictest/test_files/groupby.slt b/datafusion/sqllogictest/test_files/groupby.slt index 1d6d7dc671fa..5248ac8c8531 100644 --- a/datafusion/sqllogictest/test_files/groupby.slt +++ b/datafusion/sqllogictest/test_files/groupby.slt @@ -3211,6 +3211,21 @@ SELECT s.sn, s.amount, 2*s.sn 3 200 6 4 100 8 +# we should be able to re-write group by expression +# using functional dependencies for complex expressions also. +# In this case, we use 2*s.amount instead of s.amount. +query IRI +SELECT s.sn, 2*s.amount, 2*s.sn + FROM sales_global_with_pk AS s + GROUP BY sn + ORDER BY sn +---- +0 60 0 +1 100 2 +2 150 4 +3 400 6 +4 200 8 + query IRI SELECT s.sn, s.amount, 2*s.sn FROM sales_global_with_pk_alternate AS s @@ -3364,7 +3379,7 @@ SELECT column1, COUNT(*) as column2 FROM (VALUES (['a', 'b'], 1), (['c', 'd', 'e # primary key should be aware from which columns it is associated -statement error DataFusion error: Error during planning: Projection references non-aggregate values: Expression r.sn could not be resolved from available columns: l.sn, SUM\(l.amount\) +statement error DataFusion error: Error during planning: Projection references non-aggregate values: Expression r.sn could not be resolved from available columns: l.sn, l.zip_code, l.country, l.ts, l.currency, l.amount, SUM\(l.amount\) SELECT l.sn, r.sn, SUM(l.amount), r.amount FROM sales_global_with_pk AS l JOIN sales_global_with_pk AS r @@ -3456,7 +3471,7 @@ ORDER BY r.sn 4 100 2022-01-03T10:00:00 # after join, new window expressions shouldn't be associated with primary keys -statement error DataFusion error: Error during planning: Projection references non-aggregate values: Expression rn1 could not be resolved from available columns: r.sn, SUM\(r.amount\) +statement error DataFusion error: Error during planning: Projection references non-aggregate values: Expression rn1 could not be resolved from available columns: r.sn, r.ts, r.amount, SUM\(r.amount\) SELECT r.sn, SUM(r.amount), rn1 FROM (SELECT r.ts, r.sn, r.amount, @@ -3784,6 +3799,192 @@ AggregateExec: mode=FinalPartitioned, gby=[c@0 as c, b@1 as b], aggr=[SUM(multip ----------RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1 ------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[b, c, d], output_ordering=[c@1 ASC NULLS LAST], has_header=true +statement ok +set datafusion.execution.target_partitions = 1; + +query TT +EXPLAIN SELECT c, sum1 + FROM + (SELECT c, b, a, SUM(d) as sum1 + FROM multiple_ordered_table_with_pk + GROUP BY c) +GROUP BY c; +---- +logical_plan +Aggregate: groupBy=[[multiple_ordered_table_with_pk.c, sum1]], aggr=[[]] +--Projection: multiple_ordered_table_with_pk.c, SUM(multiple_ordered_table_with_pk.d) AS sum1 +----Aggregate: groupBy=[[multiple_ordered_table_with_pk.c]], aggr=[[SUM(CAST(multiple_ordered_table_with_pk.d AS Int64))]] +------TableScan: multiple_ordered_table_with_pk projection=[c, d] +physical_plan +AggregateExec: mode=Single, gby=[c@0 as c, sum1@1 as sum1], aggr=[], ordering_mode=PartiallySorted([0]) +--ProjectionExec: expr=[c@0 as c, SUM(multiple_ordered_table_with_pk.d)@1 as sum1] +----AggregateExec: mode=Single, gby=[c@0 as c], aggr=[SUM(multiple_ordered_table_with_pk.d)], ordering_mode=Sorted +------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[c, d], output_ordering=[c@0 ASC NULLS LAST], has_header=true + +query TT +EXPLAIN SELECT c, sum1, SUM(b) OVER() as sumb + FROM + (SELECT c, b, a, SUM(d) as sum1 + FROM multiple_ordered_table_with_pk + GROUP BY c); +---- +logical_plan +Projection: multiple_ordered_table_with_pk.c, sum1, SUM(multiple_ordered_table_with_pk.b) ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING AS sumb +--WindowAggr: windowExpr=[[SUM(CAST(multiple_ordered_table_with_pk.b AS Int64)) ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING]] +----Projection: multiple_ordered_table_with_pk.c, multiple_ordered_table_with_pk.b, SUM(multiple_ordered_table_with_pk.d) AS sum1 +------Aggregate: groupBy=[[multiple_ordered_table_with_pk.c, multiple_ordered_table_with_pk.b]], aggr=[[SUM(CAST(multiple_ordered_table_with_pk.d AS Int64))]] +--------TableScan: multiple_ordered_table_with_pk projection=[b, c, d] +physical_plan +ProjectionExec: expr=[c@0 as c, sum1@2 as sum1, SUM(multiple_ordered_table_with_pk.b) ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING@3 as sumb] +--WindowAggExec: wdw=[SUM(multiple_ordered_table_with_pk.b) ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING: Ok(Field { name: "SUM(multiple_ordered_table_with_pk.b) ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(NULL)), end_bound: Following(UInt64(NULL)) }] +----ProjectionExec: expr=[c@0 as c, b@1 as b, SUM(multiple_ordered_table_with_pk.d)@2 as sum1] +------AggregateExec: mode=Single, gby=[c@1 as c, b@0 as b], aggr=[SUM(multiple_ordered_table_with_pk.d)], ordering_mode=PartiallySorted([0]) +--------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[b, c, d], output_ordering=[c@1 ASC NULLS LAST], has_header=true + +query TT +EXPLAIN SELECT lhs.c, rhs.c, lhs.sum1, rhs.sum1 + FROM + (SELECT c, b, a, SUM(d) as sum1 + FROM multiple_ordered_table_with_pk + GROUP BY c) as lhs + JOIN + (SELECT c, b, a, SUM(d) as sum1 + FROM multiple_ordered_table_with_pk + GROUP BY c) as rhs + ON lhs.b=rhs.b; +---- +logical_plan +Projection: lhs.c, rhs.c, lhs.sum1, rhs.sum1 +--Inner Join: lhs.b = rhs.b +----SubqueryAlias: lhs +------Projection: multiple_ordered_table_with_pk.c, multiple_ordered_table_with_pk.b, SUM(multiple_ordered_table_with_pk.d) AS sum1 +--------Aggregate: groupBy=[[multiple_ordered_table_with_pk.c, multiple_ordered_table_with_pk.b]], aggr=[[SUM(CAST(multiple_ordered_table_with_pk.d AS Int64))]] +----------TableScan: multiple_ordered_table_with_pk projection=[b, c, d] +----SubqueryAlias: rhs +------Projection: multiple_ordered_table_with_pk.c, multiple_ordered_table_with_pk.b, SUM(multiple_ordered_table_with_pk.d) AS sum1 +--------Aggregate: groupBy=[[multiple_ordered_table_with_pk.c, multiple_ordered_table_with_pk.b]], aggr=[[SUM(CAST(multiple_ordered_table_with_pk.d AS Int64))]] +----------TableScan: multiple_ordered_table_with_pk projection=[b, c, d] +physical_plan +ProjectionExec: expr=[c@0 as c, c@3 as c, sum1@2 as sum1, sum1@5 as sum1] +--CoalesceBatchesExec: target_batch_size=2 +----HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(b@1, b@1)] +------ProjectionExec: expr=[c@0 as c, b@1 as b, SUM(multiple_ordered_table_with_pk.d)@2 as sum1] +--------AggregateExec: mode=Single, gby=[c@1 as c, b@0 as b], aggr=[SUM(multiple_ordered_table_with_pk.d)], ordering_mode=PartiallySorted([0]) +----------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[b, c, d], output_ordering=[c@1 ASC NULLS LAST], has_header=true +------ProjectionExec: expr=[c@0 as c, b@1 as b, SUM(multiple_ordered_table_with_pk.d)@2 as sum1] +--------AggregateExec: mode=Single, gby=[c@1 as c, b@0 as b], aggr=[SUM(multiple_ordered_table_with_pk.d)], ordering_mode=PartiallySorted([0]) +----------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[b, c, d], output_ordering=[c@1 ASC NULLS LAST], has_header=true + +query TT +EXPLAIN SELECT lhs.c, rhs.c, lhs.sum1, rhs.sum1 + FROM + (SELECT c, b, a, SUM(d) as sum1 + FROM multiple_ordered_table_with_pk + GROUP BY c) as lhs + CROSS JOIN + (SELECT c, b, a, SUM(d) as sum1 + FROM multiple_ordered_table_with_pk + GROUP BY c) as rhs; +---- +logical_plan +Projection: lhs.c, rhs.c, lhs.sum1, rhs.sum1 +--CrossJoin: +----SubqueryAlias: lhs +------Projection: multiple_ordered_table_with_pk.c, SUM(multiple_ordered_table_with_pk.d) AS sum1 +--------Aggregate: groupBy=[[multiple_ordered_table_with_pk.c]], aggr=[[SUM(CAST(multiple_ordered_table_with_pk.d AS Int64))]] +----------TableScan: multiple_ordered_table_with_pk projection=[c, d] +----SubqueryAlias: rhs +------Projection: multiple_ordered_table_with_pk.c, SUM(multiple_ordered_table_with_pk.d) AS sum1 +--------Aggregate: groupBy=[[multiple_ordered_table_with_pk.c]], aggr=[[SUM(CAST(multiple_ordered_table_with_pk.d AS Int64))]] +----------TableScan: multiple_ordered_table_with_pk projection=[c, d] +physical_plan +ProjectionExec: expr=[c@0 as c, c@2 as c, sum1@1 as sum1, sum1@3 as sum1] +--CrossJoinExec +----ProjectionExec: expr=[c@0 as c, SUM(multiple_ordered_table_with_pk.d)@1 as sum1] +------AggregateExec: mode=Single, gby=[c@0 as c], aggr=[SUM(multiple_ordered_table_with_pk.d)], ordering_mode=Sorted +--------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[c, d], output_ordering=[c@0 ASC NULLS LAST], has_header=true +----ProjectionExec: expr=[c@0 as c, SUM(multiple_ordered_table_with_pk.d)@1 as sum1] +------AggregateExec: mode=Single, gby=[c@0 as c], aggr=[SUM(multiple_ordered_table_with_pk.d)], ordering_mode=Sorted +--------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[c, d], output_ordering=[c@0 ASC NULLS LAST], has_header=true + +# we do not generate physical plan for Repartition yet (e.g Distribute By queries). +query TT +EXPLAIN SELECT a, b, sum1 +FROM (SELECT c, b, a, SUM(d) as sum1 + FROM multiple_ordered_table_with_pk + GROUP BY c) +DISTRIBUTE BY a +---- +logical_plan +Repartition: DistributeBy(a) +--Projection: multiple_ordered_table_with_pk.a, multiple_ordered_table_with_pk.b, SUM(multiple_ordered_table_with_pk.d) AS sum1 +----Aggregate: groupBy=[[multiple_ordered_table_with_pk.c, multiple_ordered_table_with_pk.a, multiple_ordered_table_with_pk.b]], aggr=[[SUM(CAST(multiple_ordered_table_with_pk.d AS Int64))]] +------TableScan: multiple_ordered_table_with_pk projection=[a, b, c, d] + +# union with aggregate +query TT +EXPLAIN SELECT c, a, SUM(d) as sum1 + FROM multiple_ordered_table_with_pk + GROUP BY c +UNION ALL + SELECT c, a, SUM(d) as sum1 + FROM multiple_ordered_table_with_pk + GROUP BY c +---- +logical_plan +Union +--Projection: multiple_ordered_table_with_pk.c, multiple_ordered_table_with_pk.a, SUM(multiple_ordered_table_with_pk.d) AS sum1 +----Aggregate: groupBy=[[multiple_ordered_table_with_pk.c, multiple_ordered_table_with_pk.a]], aggr=[[SUM(CAST(multiple_ordered_table_with_pk.d AS Int64))]] +------TableScan: multiple_ordered_table_with_pk projection=[a, c, d] +--Projection: multiple_ordered_table_with_pk.c, multiple_ordered_table_with_pk.a, SUM(multiple_ordered_table_with_pk.d) AS sum1 +----Aggregate: groupBy=[[multiple_ordered_table_with_pk.c, multiple_ordered_table_with_pk.a]], aggr=[[SUM(CAST(multiple_ordered_table_with_pk.d AS Int64))]] +------TableScan: multiple_ordered_table_with_pk projection=[a, c, d] +physical_plan +UnionExec +--ProjectionExec: expr=[c@0 as c, a@1 as a, SUM(multiple_ordered_table_with_pk.d)@2 as sum1] +----AggregateExec: mode=Single, gby=[c@1 as c, a@0 as a], aggr=[SUM(multiple_ordered_table_with_pk.d)], ordering_mode=Sorted +------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, c, d], output_orderings=[[a@0 ASC NULLS LAST], [c@1 ASC NULLS LAST]], has_header=true +--ProjectionExec: expr=[c@0 as c, a@1 as a, SUM(multiple_ordered_table_with_pk.d)@2 as sum1] +----AggregateExec: mode=Single, gby=[c@1 as c, a@0 as a], aggr=[SUM(multiple_ordered_table_with_pk.d)], ordering_mode=Sorted +------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, c, d], output_orderings=[[a@0 ASC NULLS LAST], [c@1 ASC NULLS LAST]], has_header=true + +# table scan should be simplified. +query TT +EXPLAIN SELECT c, a, SUM(d) as sum1 + FROM multiple_ordered_table_with_pk + GROUP BY c +---- +logical_plan +Projection: multiple_ordered_table_with_pk.c, multiple_ordered_table_with_pk.a, SUM(multiple_ordered_table_with_pk.d) AS sum1 +--Aggregate: groupBy=[[multiple_ordered_table_with_pk.c, multiple_ordered_table_with_pk.a]], aggr=[[SUM(CAST(multiple_ordered_table_with_pk.d AS Int64))]] +----TableScan: multiple_ordered_table_with_pk projection=[a, c, d] +physical_plan +ProjectionExec: expr=[c@0 as c, a@1 as a, SUM(multiple_ordered_table_with_pk.d)@2 as sum1] +--AggregateExec: mode=Single, gby=[c@1 as c, a@0 as a], aggr=[SUM(multiple_ordered_table_with_pk.d)], ordering_mode=Sorted +----CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, c, d], output_orderings=[[a@0 ASC NULLS LAST], [c@1 ASC NULLS LAST]], has_header=true + +# limit should be simplified +query TT +EXPLAIN SELECT * + FROM (SELECT c, a, SUM(d) as sum1 + FROM multiple_ordered_table_with_pk + GROUP BY c + LIMIT 5) +---- +logical_plan +Projection: multiple_ordered_table_with_pk.c, multiple_ordered_table_with_pk.a, SUM(multiple_ordered_table_with_pk.d) AS sum1 +--Limit: skip=0, fetch=5 +----Aggregate: groupBy=[[multiple_ordered_table_with_pk.c, multiple_ordered_table_with_pk.a]], aggr=[[SUM(CAST(multiple_ordered_table_with_pk.d AS Int64))]] +------TableScan: multiple_ordered_table_with_pk projection=[a, c, d] +physical_plan +ProjectionExec: expr=[c@0 as c, a@1 as a, SUM(multiple_ordered_table_with_pk.d)@2 as sum1] +--GlobalLimitExec: skip=0, fetch=5 +----AggregateExec: mode=Single, gby=[c@1 as c, a@0 as a], aggr=[SUM(multiple_ordered_table_with_pk.d)], ordering_mode=Sorted +------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, c, d], output_orderings=[[a@0 ASC NULLS LAST], [c@1 ASC NULLS LAST]], has_header=true + +statement ok +set datafusion.execution.target_partitions = 8; + # Tests for single distinct to group by optimization rule statement ok CREATE TABLE t(x int) AS VALUES (1), (2), (1); From 047fb333683b2fbbc3da227480a5a4a8625038aa Mon Sep 17 00:00:00 2001 From: Alex Huang Date: Fri, 8 Dec 2023 18:13:30 +0100 Subject: [PATCH 395/572] Minor: refactor `data_trunc` to reduce duplicated code (#8430) * refactor data_trunc * fix cast to timestamp array * fix cast to timestamp scalar * fix doc --- datafusion/common/src/scalar.rs | 15 ++ .../physical-expr/src/datetime_expressions.rs | 137 +++++------------- 2 files changed, 53 insertions(+), 99 deletions(-) diff --git a/datafusion/common/src/scalar.rs b/datafusion/common/src/scalar.rs index 7e18c313e090..d730fbf89b72 100644 --- a/datafusion/common/src/scalar.rs +++ b/datafusion/common/src/scalar.rs @@ -46,6 +46,7 @@ use arrow::{ }, }; use arrow_array::cast::as_list_array; +use arrow_array::types::ArrowTimestampType; use arrow_array::{ArrowNativeTypeOp, Scalar}; /// A dynamically typed, nullable single value, (the single-valued counter-part @@ -774,6 +775,20 @@ impl ScalarValue { ScalarValue::IntervalMonthDayNano(Some(val)) } + /// Returns a [`ScalarValue`] representing + /// `value` and `tz_opt` timezone + pub fn new_timestamp( + value: Option, + tz_opt: Option>, + ) -> Self { + match T::UNIT { + TimeUnit::Second => ScalarValue::TimestampSecond(value, tz_opt), + TimeUnit::Millisecond => ScalarValue::TimestampMillisecond(value, tz_opt), + TimeUnit::Microsecond => ScalarValue::TimestampMicrosecond(value, tz_opt), + TimeUnit::Nanosecond => ScalarValue::TimestampNanosecond(value, tz_opt), + } + } + /// Create a zero value in the given type. pub fn new_zero(datatype: &DataType) -> Result { assert!(datatype.is_primitive()); diff --git a/datafusion/physical-expr/src/datetime_expressions.rs b/datafusion/physical-expr/src/datetime_expressions.rs index 04cfec29ea8a..d634b4d01918 100644 --- a/datafusion/physical-expr/src/datetime_expressions.rs +++ b/datafusion/physical-expr/src/datetime_expressions.rs @@ -36,6 +36,7 @@ use arrow::{ TimestampMillisecondType, TimestampNanosecondType, TimestampSecondType, }, }; +use arrow_array::types::ArrowTimestampType; use arrow_array::{ timezone::Tz, TimestampMicrosecondArray, TimestampMillisecondArray, TimestampSecondArray, @@ -43,7 +44,7 @@ use arrow_array::{ use chrono::prelude::*; use chrono::{Duration, Months, NaiveDate}; use datafusion_common::cast::{ - as_date32_array, as_date64_array, as_generic_string_array, + as_date32_array, as_date64_array, as_generic_string_array, as_primitive_array, as_timestamp_microsecond_array, as_timestamp_millisecond_array, as_timestamp_nanosecond_array, as_timestamp_second_array, }; @@ -335,7 +336,7 @@ fn date_trunc_coarse(granularity: &str, value: i64, tz: Option) -> Result, tz: Option, @@ -403,123 +404,61 @@ pub fn date_trunc(args: &[ColumnarValue]) -> Result { return exec_err!("Granularity of `date_trunc` must be non-null scalar Utf8"); }; + fn process_array( + array: &dyn Array, + granularity: String, + tz_opt: &Option>, + ) -> Result { + let parsed_tz = parse_tz(tz_opt)?; + let array = as_primitive_array::(array)?; + let array = array + .iter() + .map(|x| general_date_trunc(T::UNIT, &x, parsed_tz, granularity.as_str())) + .collect::>>()? + .with_timezone_opt(tz_opt.clone()); + Ok(ColumnarValue::Array(Arc::new(array))) + } + + fn process_scalr( + v: &Option, + granularity: String, + tz_opt: &Option>, + ) -> Result { + let parsed_tz = parse_tz(tz_opt)?; + let value = general_date_trunc(T::UNIT, v, parsed_tz, granularity.as_str())?; + let value = ScalarValue::new_timestamp::(value, tz_opt.clone()); + Ok(ColumnarValue::Scalar(value)) + } + Ok(match array { ColumnarValue::Scalar(ScalarValue::TimestampNanosecond(v, tz_opt)) => { - let parsed_tz = parse_tz(tz_opt)?; - let value = - _date_trunc(TimeUnit::Nanosecond, v, parsed_tz, granularity.as_str())?; - let value = ScalarValue::TimestampNanosecond(value, tz_opt.clone()); - ColumnarValue::Scalar(value) + process_scalr::(v, granularity, tz_opt)? } ColumnarValue::Scalar(ScalarValue::TimestampMicrosecond(v, tz_opt)) => { - let parsed_tz = parse_tz(tz_opt)?; - let value = - _date_trunc(TimeUnit::Microsecond, v, parsed_tz, granularity.as_str())?; - let value = ScalarValue::TimestampMicrosecond(value, tz_opt.clone()); - ColumnarValue::Scalar(value) + process_scalr::(v, granularity, tz_opt)? } ColumnarValue::Scalar(ScalarValue::TimestampMillisecond(v, tz_opt)) => { - let parsed_tz = parse_tz(tz_opt)?; - let value = - _date_trunc(TimeUnit::Millisecond, v, parsed_tz, granularity.as_str())?; - let value = ScalarValue::TimestampMillisecond(value, tz_opt.clone()); - ColumnarValue::Scalar(value) + process_scalr::(v, granularity, tz_opt)? } ColumnarValue::Scalar(ScalarValue::TimestampSecond(v, tz_opt)) => { - let parsed_tz = parse_tz(tz_opt)?; - let value = - _date_trunc(TimeUnit::Second, v, parsed_tz, granularity.as_str())?; - let value = ScalarValue::TimestampSecond(value, tz_opt.clone()); - ColumnarValue::Scalar(value) + process_scalr::(v, granularity, tz_opt)? } ColumnarValue::Array(array) => { let array_type = array.data_type(); match array_type { DataType::Timestamp(TimeUnit::Second, tz_opt) => { - let parsed_tz = parse_tz(tz_opt)?; - let array = as_timestamp_second_array(array)?; - let array = array - .iter() - .map(|x| { - _date_trunc( - TimeUnit::Second, - &x, - parsed_tz, - granularity.as_str(), - ) - }) - .collect::>()? - .with_timezone_opt(tz_opt.clone()); - ColumnarValue::Array(Arc::new(array)) + process_array::(array, granularity, tz_opt)? } DataType::Timestamp(TimeUnit::Millisecond, tz_opt) => { - let parsed_tz = parse_tz(tz_opt)?; - let array = as_timestamp_millisecond_array(array)?; - let array = array - .iter() - .map(|x| { - _date_trunc( - TimeUnit::Millisecond, - &x, - parsed_tz, - granularity.as_str(), - ) - }) - .collect::>()? - .with_timezone_opt(tz_opt.clone()); - ColumnarValue::Array(Arc::new(array)) + process_array::(array, granularity, tz_opt)? } DataType::Timestamp(TimeUnit::Microsecond, tz_opt) => { - let parsed_tz = parse_tz(tz_opt)?; - let array = as_timestamp_microsecond_array(array)?; - let array = array - .iter() - .map(|x| { - _date_trunc( - TimeUnit::Microsecond, - &x, - parsed_tz, - granularity.as_str(), - ) - }) - .collect::>()? - .with_timezone_opt(tz_opt.clone()); - ColumnarValue::Array(Arc::new(array)) + process_array::(array, granularity, tz_opt)? } DataType::Timestamp(TimeUnit::Nanosecond, tz_opt) => { - let parsed_tz = parse_tz(tz_opt)?; - let array = as_timestamp_nanosecond_array(array)?; - let array = array - .iter() - .map(|x| { - _date_trunc( - TimeUnit::Nanosecond, - &x, - parsed_tz, - granularity.as_str(), - ) - }) - .collect::>()? - .with_timezone_opt(tz_opt.clone()); - ColumnarValue::Array(Arc::new(array)) - } - _ => { - let parsed_tz = None; - let array = as_timestamp_nanosecond_array(array)?; - let array = array - .iter() - .map(|x| { - _date_trunc( - TimeUnit::Nanosecond, - &x, - parsed_tz, - granularity.as_str(), - ) - }) - .collect::>()?; - - ColumnarValue::Array(Arc::new(array)) + process_array::(array, granularity, tz_opt)? } + _ => process_array::(array, granularity, &None)?, } } _ => { From cd02c40f7575e331121a94cb217b71905e240f9f Mon Sep 17 00:00:00 2001 From: yi wang <48236141+my-vegetable-has-exploded@users.noreply.github.com> Date: Sat, 9 Dec 2023 02:06:52 +0800 Subject: [PATCH 396/572] Support array_distinct function. (#8268) * implement distinct func implement slt & proto fix null & empty list * add comment for slt Co-authored-by: Alex Huang * fix largelist * add largelist for slt * Use collect for rows & init capcity for offsets. * fixup: remove useless match * fix fmt * fix fmt --------- Co-authored-by: Alex Huang Co-authored-by: Andrew Lamb --- datafusion/expr/src/built_in_function.rs | 6 ++ datafusion/expr/src/expr_fn.rs | 6 ++ .../physical-expr/src/array_expressions.rs | 64 +++++++++++- datafusion/physical-expr/src/functions.rs | 3 + datafusion/proto/proto/datafusion.proto | 1 + datafusion/proto/src/generated/pbjson.rs | 3 + datafusion/proto/src/generated/prost.rs | 3 + .../proto/src/logical_plan/from_proto.rs | 22 +++-- datafusion/proto/src/logical_plan/to_proto.rs | 1 + datafusion/sqllogictest/test_files/array.slt | 99 +++++++++++++++++++ docs/source/user-guide/expressions.md | 1 + 11 files changed, 198 insertions(+), 11 deletions(-) diff --git a/datafusion/expr/src/built_in_function.rs b/datafusion/expr/src/built_in_function.rs index 44fbf45525d4..977b556b26cf 100644 --- a/datafusion/expr/src/built_in_function.rs +++ b/datafusion/expr/src/built_in_function.rs @@ -146,6 +146,8 @@ pub enum BuiltinScalarFunction { ArrayPopBack, /// array_dims ArrayDims, + /// array_distinct + ArrayDistinct, /// array_element ArrayElement, /// array_empty @@ -407,6 +409,7 @@ impl BuiltinScalarFunction { BuiltinScalarFunction::ArrayHasAny => Volatility::Immutable, BuiltinScalarFunction::ArrayHas => Volatility::Immutable, BuiltinScalarFunction::ArrayDims => Volatility::Immutable, + BuiltinScalarFunction::ArrayDistinct => Volatility::Immutable, BuiltinScalarFunction::ArrayElement => Volatility::Immutable, BuiltinScalarFunction::ArrayExcept => Volatility::Immutable, BuiltinScalarFunction::ArrayLength => Volatility::Immutable, @@ -586,6 +589,7 @@ impl BuiltinScalarFunction { BuiltinScalarFunction::ArrayDims => { Ok(List(Arc::new(Field::new("item", UInt64, true)))) } + BuiltinScalarFunction::ArrayDistinct => Ok(input_expr_types[0].clone()), BuiltinScalarFunction::ArrayElement => match &input_expr_types[0] { List(field) => Ok(field.data_type().clone()), _ => plan_err!( @@ -933,6 +937,7 @@ impl BuiltinScalarFunction { Signature::variadic_any(self.volatility()) } BuiltinScalarFunction::ArrayNdims => Signature::any(1, self.volatility()), + BuiltinScalarFunction::ArrayDistinct => Signature::any(1, self.volatility()), BuiltinScalarFunction::ArrayPosition => { Signature::variadic_any(self.volatility()) } @@ -1570,6 +1575,7 @@ impl BuiltinScalarFunction { &["array_concat", "array_cat", "list_concat", "list_cat"] } BuiltinScalarFunction::ArrayDims => &["array_dims", "list_dims"], + BuiltinScalarFunction::ArrayDistinct => &["array_distinct", "list_distinct"], BuiltinScalarFunction::ArrayEmpty => &["empty"], BuiltinScalarFunction::ArrayElement => &[ "array_element", diff --git a/datafusion/expr/src/expr_fn.rs b/datafusion/expr/src/expr_fn.rs index 8d25619c07d1..cedf1d845137 100644 --- a/datafusion/expr/src/expr_fn.rs +++ b/datafusion/expr/src/expr_fn.rs @@ -660,6 +660,12 @@ scalar_expr!( array, "returns the number of dimensions of the array." ); +scalar_expr!( + ArrayDistinct, + array_distinct, + array, + "return distinct values from the array after removing duplicates." +); scalar_expr!( ArrayPosition, array_position, diff --git a/datafusion/physical-expr/src/array_expressions.rs b/datafusion/physical-expr/src/array_expressions.rs index 08df3ef9f613..ae048694583b 100644 --- a/datafusion/physical-expr/src/array_expressions.rs +++ b/datafusion/physical-expr/src/array_expressions.rs @@ -31,8 +31,8 @@ use arrow_buffer::NullBuffer; use arrow_schema::{FieldRef, SortOptions}; use datafusion_common::cast::{ - as_generic_list_array, as_generic_string_array, as_int64_array, as_list_array, - as_null_array, as_string_array, + as_generic_list_array, as_generic_string_array, as_int64_array, as_large_list_array, + as_list_array, as_null_array, as_string_array, }; use datafusion_common::utils::{array_into_list_array, list_ndims}; use datafusion_common::{ @@ -2111,6 +2111,66 @@ pub fn array_intersect(args: &[ArrayRef]) -> Result { } } +pub fn general_array_distinct( + array: &GenericListArray, + field: &FieldRef, +) -> Result { + let dt = array.value_type(); + let mut offsets = Vec::with_capacity(array.len()); + offsets.push(OffsetSize::usize_as(0)); + let mut new_arrays = Vec::with_capacity(array.len()); + let converter = RowConverter::new(vec![SortField::new(dt.clone())])?; + // distinct for each list in ListArray + for arr in array.iter().flatten() { + let values = converter.convert_columns(&[arr])?; + // sort elements in list and remove duplicates + let rows = values.iter().sorted().dedup().collect::>(); + let last_offset: OffsetSize = offsets.last().copied().unwrap(); + offsets.push(last_offset + OffsetSize::usize_as(rows.len())); + let arrays = converter.convert_rows(rows)?; + let array = match arrays.get(0) { + Some(array) => array.clone(), + None => { + return internal_err!("array_distinct: failed to get array from rows") + } + }; + new_arrays.push(array); + } + let offsets = OffsetBuffer::new(offsets.into()); + let new_arrays_ref = new_arrays.iter().map(|v| v.as_ref()).collect::>(); + let values = compute::concat(&new_arrays_ref)?; + Ok(Arc::new(GenericListArray::::try_new( + field.clone(), + offsets, + values, + None, + )?)) +} + +/// array_distinct SQL function +/// example: from list [1, 3, 2, 3, 1, 2, 4] to [1, 2, 3, 4] +pub fn array_distinct(args: &[ArrayRef]) -> Result { + assert_eq!(args.len(), 1); + + // handle null + if args[0].data_type() == &DataType::Null { + return Ok(args[0].clone()); + } + + // handle for list & largelist + match args[0].data_type() { + DataType::List(field) => { + let array = as_list_array(&args[0])?; + general_array_distinct(array, field) + } + DataType::LargeList(field) => { + let array = as_large_list_array(&args[0])?; + general_array_distinct(array, field) + } + _ => internal_err!("array_distinct only support list array"), + } +} + #[cfg(test)] mod tests { use super::*; diff --git a/datafusion/physical-expr/src/functions.rs b/datafusion/physical-expr/src/functions.rs index 873864a57a6f..53de85843919 100644 --- a/datafusion/physical-expr/src/functions.rs +++ b/datafusion/physical-expr/src/functions.rs @@ -350,6 +350,9 @@ pub fn create_physical_fun( BuiltinScalarFunction::ArrayDims => { Arc::new(|args| make_scalar_function(array_expressions::array_dims)(args)) } + BuiltinScalarFunction::ArrayDistinct => { + Arc::new(|args| make_scalar_function(array_expressions::array_distinct)(args)) + } BuiltinScalarFunction::ArrayElement => { Arc::new(|args| make_scalar_function(array_expressions::array_element)(args)) } diff --git a/datafusion/proto/proto/datafusion.proto b/datafusion/proto/proto/datafusion.proto index 55fb08042399..13a54f2a5659 100644 --- a/datafusion/proto/proto/datafusion.proto +++ b/datafusion/proto/proto/datafusion.proto @@ -645,6 +645,7 @@ enum ScalarFunction { SubstrIndex = 126; FindInSet = 127; ArraySort = 128; + ArrayDistinct = 129; } message ScalarFunctionNode { diff --git a/datafusion/proto/src/generated/pbjson.rs b/datafusion/proto/src/generated/pbjson.rs index dea329cbea28..0d013c72d37f 100644 --- a/datafusion/proto/src/generated/pbjson.rs +++ b/datafusion/proto/src/generated/pbjson.rs @@ -21049,6 +21049,7 @@ impl serde::Serialize for ScalarFunction { Self::SubstrIndex => "SubstrIndex", Self::FindInSet => "FindInSet", Self::ArraySort => "ArraySort", + Self::ArrayDistinct => "ArrayDistinct", }; serializer.serialize_str(variant) } @@ -21189,6 +21190,7 @@ impl<'de> serde::Deserialize<'de> for ScalarFunction { "SubstrIndex", "FindInSet", "ArraySort", + "ArrayDistinct", ]; struct GeneratedVisitor; @@ -21358,6 +21360,7 @@ impl<'de> serde::Deserialize<'de> for ScalarFunction { "SubstrIndex" => Ok(ScalarFunction::SubstrIndex), "FindInSet" => Ok(ScalarFunction::FindInSet), "ArraySort" => Ok(ScalarFunction::ArraySort), + "ArrayDistinct" => Ok(ScalarFunction::ArrayDistinct), _ => Err(serde::de::Error::unknown_variant(value, FIELDS)), } } diff --git a/datafusion/proto/src/generated/prost.rs b/datafusion/proto/src/generated/prost.rs index 41b94a2a3961..d4b62d4b3fd8 100644 --- a/datafusion/proto/src/generated/prost.rs +++ b/datafusion/proto/src/generated/prost.rs @@ -2614,6 +2614,7 @@ pub enum ScalarFunction { SubstrIndex = 126, FindInSet = 127, ArraySort = 128, + ArrayDistinct = 129, } impl ScalarFunction { /// String value of the enum field names used in the ProtoBuf definition. @@ -2751,6 +2752,7 @@ impl ScalarFunction { ScalarFunction::SubstrIndex => "SubstrIndex", ScalarFunction::FindInSet => "FindInSet", ScalarFunction::ArraySort => "ArraySort", + ScalarFunction::ArrayDistinct => "ArrayDistinct", } } /// Creates an enum from field names used in the ProtoBuf definition. @@ -2885,6 +2887,7 @@ impl ScalarFunction { "SubstrIndex" => Some(Self::SubstrIndex), "FindInSet" => Some(Self::FindInSet), "ArraySort" => Some(Self::ArraySort), + "ArrayDistinct" => Some(Self::ArrayDistinct), _ => None, } } diff --git a/datafusion/proto/src/logical_plan/from_proto.rs b/datafusion/proto/src/logical_plan/from_proto.rs index 7daab47837d6..193e0947d6d9 100644 --- a/datafusion/proto/src/logical_plan/from_proto.rs +++ b/datafusion/proto/src/logical_plan/from_proto.rs @@ -41,15 +41,15 @@ use datafusion_common::{ }; use datafusion_expr::window_frame::{check_window_frame, regularize_window_order_by}; use datafusion_expr::{ - abs, acos, acosh, array, array_append, array_concat, array_dims, array_element, - array_except, array_has, array_has_all, array_has_any, array_intersect, array_length, - array_ndims, array_position, array_positions, array_prepend, array_remove, - array_remove_all, array_remove_n, array_repeat, array_replace, array_replace_all, - array_replace_n, array_slice, array_sort, array_to_string, arrow_typeof, ascii, asin, - asinh, atan, atan2, atanh, bit_length, btrim, cardinality, cbrt, ceil, - character_length, chr, coalesce, concat_expr, concat_ws_expr, cos, cosh, cot, - current_date, current_time, date_bin, date_part, date_trunc, decode, degrees, digest, - encode, exp, + abs, acos, acosh, array, array_append, array_concat, array_dims, array_distinct, + array_element, array_except, array_has, array_has_all, array_has_any, + array_intersect, array_length, array_ndims, array_position, array_positions, + array_prepend, array_remove, array_remove_all, array_remove_n, array_repeat, + array_replace, array_replace_all, array_replace_n, array_slice, array_sort, + array_to_string, arrow_typeof, ascii, asin, asinh, atan, atan2, atanh, bit_length, + btrim, cardinality, cbrt, ceil, character_length, chr, coalesce, concat_expr, + concat_ws_expr, cos, cosh, cot, current_date, current_time, date_bin, date_part, + date_trunc, decode, degrees, digest, encode, exp, expr::{self, InList, Sort, WindowFunction}, factorial, find_in_set, flatten, floor, from_unixtime, gcd, gen_range, isnan, iszero, lcm, left, levenshtein, ln, log, log10, log2, @@ -484,6 +484,7 @@ impl From<&protobuf::ScalarFunction> for BuiltinScalarFunction { ScalarFunction::ArrayHasAny => Self::ArrayHasAny, ScalarFunction::ArrayHas => Self::ArrayHas, ScalarFunction::ArrayDims => Self::ArrayDims, + ScalarFunction::ArrayDistinct => Self::ArrayDistinct, ScalarFunction::ArrayElement => Self::ArrayElement, ScalarFunction::Flatten => Self::Flatten, ScalarFunction::ArrayLength => Self::ArrayLength, @@ -1467,6 +1468,9 @@ pub fn parse_expr( ScalarFunction::ArrayDims => { Ok(array_dims(parse_expr(&args[0], registry)?)) } + ScalarFunction::ArrayDistinct => { + Ok(array_distinct(parse_expr(&args[0], registry)?)) + } ScalarFunction::ArrayElement => Ok(array_element( parse_expr(&args[0], registry)?, parse_expr(&args[1], registry)?, diff --git a/datafusion/proto/src/logical_plan/to_proto.rs b/datafusion/proto/src/logical_plan/to_proto.rs index 4c6fdaa894ae..2997d147424d 100644 --- a/datafusion/proto/src/logical_plan/to_proto.rs +++ b/datafusion/proto/src/logical_plan/to_proto.rs @@ -1512,6 +1512,7 @@ impl TryFrom<&BuiltinScalarFunction> for protobuf::ScalarFunction { BuiltinScalarFunction::ArrayHasAny => Self::ArrayHasAny, BuiltinScalarFunction::ArrayHas => Self::ArrayHas, BuiltinScalarFunction::ArrayDims => Self::ArrayDims, + BuiltinScalarFunction::ArrayDistinct => Self::ArrayDistinct, BuiltinScalarFunction::ArrayElement => Self::ArrayElement, BuiltinScalarFunction::Flatten => Self::Flatten, BuiltinScalarFunction::ArrayLength => Self::ArrayLength, diff --git a/datafusion/sqllogictest/test_files/array.slt b/datafusion/sqllogictest/test_files/array.slt index 3c23dd369ae5..1202a2b1e99d 100644 --- a/datafusion/sqllogictest/test_files/array.slt +++ b/datafusion/sqllogictest/test_files/array.slt @@ -182,6 +182,38 @@ AS VALUES (make_array([[1], [2]], [[2], [3]]), make_array([1], [2])) ; +statement ok +CREATE TABLE array_distinct_table_1D +AS VALUES + (make_array(1, 1, 2, 2, 3)), + (make_array(1, 2, 3, 4, 5)), + (make_array(3, 5, 3, 3, 3)) +; + +statement ok +CREATE TABLE array_distinct_table_1D_UTF8 +AS VALUES + (make_array('a', 'a', 'bc', 'bc', 'def')), + (make_array('a', 'bc', 'def', 'defg', 'defg')), + (make_array('defg', 'defg', 'defg', 'defg', 'defg')) +; + +statement ok +CREATE TABLE array_distinct_table_2D +AS VALUES + (make_array([1,2], [1,2], [3,4], [3,4], [5,6])), + (make_array([1,2], [3,4], [5,6], [7,8], [9,10])), + (make_array([5,6], [5,6], NULL)) +; + +statement ok +CREATE TABLE array_distinct_table_1D_large +AS VALUES + (arrow_cast(make_array(1, 1, 2, 2, 3), 'LargeList(Int64)')), + (arrow_cast(make_array(1, 2, 3, 4, 5), 'LargeList(Int64)')), + (arrow_cast(make_array(3, 5, 3, 3, 3), 'LargeList(Int64)')) +; + statement ok CREATE TABLE array_intersect_table_1D AS VALUES @@ -2864,6 +2896,73 @@ select array_has_all(arrow_cast(make_array(1,2,3), 'LargeList(Int64)'), arrow_ca ---- true false true false false false true true false false true false true +query BBBBBBBBBBBBB +select array_has_all(arrow_cast(make_array(1,2,3), 'LargeList(Int64)'), arrow_cast(make_array(1,3), 'LargeList(Int64)')), + array_has_all(arrow_cast(make_array(1,2,3),'LargeList(Int64)'), arrow_cast(make_array(1,4), 'LargeList(Int64)')), + array_has_all(arrow_cast(make_array([1,2], [3,4]), 'LargeList(List(Int64))'), arrow_cast(make_array([1,2]), 'LargeList(List(Int64))')), + array_has_all(arrow_cast(make_array([1,2], [3,4]), 'LargeList(List(Int64))'), arrow_cast(make_array([1,3]), 'LargeList(List(Int64))')), + array_has_all(arrow_cast(make_array([1,2], [3,4]), 'LargeList(List(Int64))'), arrow_cast(make_array([1,2], [3,4], [5,6]), 'LargeList(List(Int64))')), + array_has_all(arrow_cast(make_array([[1,2,3]]), 'LargeList(List(List(Int64)))'), arrow_cast(make_array([[1]]), 'LargeList(List(List(Int64)))')), + array_has_all(arrow_cast(make_array([[1,2,3]]), 'LargeList(List(List(Int64)))'), arrow_cast(make_array([[1,2,3]]), 'LargeList(List(List(Int64)))')), + array_has_any(arrow_cast(make_array(1,2,3),'LargeList(Int64)'), arrow_cast(make_array(1,10,100), 'LargeList(Int64)')), + array_has_any(arrow_cast(make_array(1,2,3),'LargeList(Int64)'), arrow_cast(make_array(10,100),'LargeList(Int64)')), + array_has_any(arrow_cast(make_array([1,2], [3,4]), 'LargeList(List(Int64))'), arrow_cast(make_array([1,10], [10,4]), 'LargeList(List(Int64))')), + array_has_any(arrow_cast(make_array([1,2], [3,4]), 'LargeList(List(Int64))'), arrow_cast(make_array([10,20], [3,4]), 'LargeList(List(Int64))')), + array_has_any(arrow_cast(make_array([[1,2,3]]), 'LargeList(List(List(Int64)))'), arrow_cast(make_array([[1,2,3], [4,5,6]]), 'LargeList(List(List(Int64)))')), + array_has_any(arrow_cast(make_array([[1,2,3]]), 'LargeList(List(List(Int64)))'), arrow_cast(make_array([[1,2,3]], [[4,5,6]]), 'LargeList(List(List(Int64)))')) +; +---- +true false true false false false true true false false true false true + +## array_distinct + +query ? +select array_distinct(null); +---- +NULL + +query ? +select array_distinct([]); +---- +[] + +query ? +select array_distinct([[], []]); +---- +[[]] + +query ? +select array_distinct(column1) +from array_distinct_table_1D; +---- +[1, 2, 3] +[1, 2, 3, 4, 5] +[3, 5] + +query ? +select array_distinct(column1) +from array_distinct_table_1D_UTF8; +---- +[a, bc, def] +[a, bc, def, defg] +[defg] + +query ? +select array_distinct(column1) +from array_distinct_table_2D; +---- +[[1, 2], [3, 4], [5, 6]] +[[1, 2], [3, 4], [5, 6], [7, 8], [9, 10]] +[, [5, 6]] + +query ? +select array_distinct(column1) +from array_distinct_table_1D_large; +---- +[1, 2, 3] +[1, 2, 3, 4, 5] +[3, 5] + query ??? select array_intersect(column1, column2), array_intersect(column3, column4), diff --git a/docs/source/user-guide/expressions.md b/docs/source/user-guide/expressions.md index 257c50dfa497..b8689e556741 100644 --- a/docs/source/user-guide/expressions.md +++ b/docs/source/user-guide/expressions.md @@ -215,6 +215,7 @@ Unlike to some databases the math functions in Datafusion works the same way as | array_has_all(array, sub-array) | Returns true if all elements of sub-array exist in array `array_has_all([1,2,3], [1,3]) -> true` | | array_has_any(array, sub-array) | Returns true if any elements exist in both arrays `array_has_any([1,2,3], [1,4]) -> true` | | array_dims(array) | Returns an array of the array's dimensions. `array_dims([[1, 2, 3], [4, 5, 6]]) -> [2, 3]` | +| array_distinct(array) | Returns distinct values from the array after removing duplicates. `array_distinct([1, 3, 2, 3, 1, 2, 4]) -> [1, 2, 3, 4]` | | array_element(array, index) | Extracts the element with the index n from the array `array_element([1, 2, 3, 4], 3) -> 3` | | flatten(array) | Converts an array of arrays to a flat array `flatten([[1], [2, 3], [4, 5, 6]]) -> [1, 2, 3, 4, 5, 6]` | | array_length(array, dimension) | Returns the length of the array dimension. `array_length([1, 2, 3, 4, 5]) -> 5` | From 34b0445b778c503f4e65b384ee9ec119ec90044a Mon Sep 17 00:00:00 2001 From: Mustafa Akur <106137913+mustafasrepo@users.noreply.github.com> Date: Fri, 8 Dec 2023 21:31:55 +0300 Subject: [PATCH 397/572] Add primary key support to stream table (#8467) --- datafusion/core/src/datasource/stream.rs | 14 ++++++++- .../sqllogictest/test_files/groupby.slt | 31 +++++++++++++++++++ 2 files changed, 44 insertions(+), 1 deletion(-) diff --git a/datafusion/core/src/datasource/stream.rs b/datafusion/core/src/datasource/stream.rs index 6965968b6f25..e7512499eb9d 100644 --- a/datafusion/core/src/datasource/stream.rs +++ b/datafusion/core/src/datasource/stream.rs @@ -31,7 +31,7 @@ use async_trait::async_trait; use futures::StreamExt; use tokio::task::spawn_blocking; -use datafusion_common::{plan_err, DataFusionError, Result}; +use datafusion_common::{plan_err, Constraints, DataFusionError, Result}; use datafusion_execution::{SendableRecordBatchStream, TaskContext}; use datafusion_expr::{CreateExternalTable, Expr, TableType}; use datafusion_physical_plan::common::AbortOnDropSingle; @@ -100,6 +100,7 @@ pub struct StreamConfig { encoding: StreamEncoding, header: bool, order: Vec>, + constraints: Constraints, } impl StreamConfig { @@ -118,6 +119,7 @@ impl StreamConfig { encoding: StreamEncoding::Csv, order: vec![], header: false, + constraints: Constraints::empty(), } } @@ -145,6 +147,12 @@ impl StreamConfig { self } + /// Assign constraints + pub fn with_constraints(mut self, constraints: Constraints) -> Self { + self.constraints = constraints; + self + } + fn reader(&self) -> Result> { let file = File::open(&self.location)?; let schema = self.schema.clone(); @@ -215,6 +223,10 @@ impl TableProvider for StreamTable { self.0.schema.clone() } + fn constraints(&self) -> Option<&Constraints> { + Some(&self.0.constraints) + } + fn table_type(&self) -> TableType { TableType::Base } diff --git a/datafusion/sqllogictest/test_files/groupby.slt b/datafusion/sqllogictest/test_files/groupby.slt index 5248ac8c8531..b7be4d78b583 100644 --- a/datafusion/sqllogictest/test_files/groupby.slt +++ b/datafusion/sqllogictest/test_files/groupby.slt @@ -4248,3 +4248,34 @@ set datafusion.sql_parser.dialect = 'Generic'; statement ok drop table aggregate_test_100; + + +# Create an unbounded external table with primary key +# column c +statement ok +CREATE EXTERNAL TABLE unbounded_multiple_ordered_table_with_pk ( + a0 INTEGER, + a INTEGER, + b INTEGER, + c INTEGER primary key, + d INTEGER +) +STORED AS CSV +WITH HEADER ROW +WITH ORDER (a ASC, b ASC) +WITH ORDER (c ASC) +LOCATION '../core/tests/data/window_2.csv'; + +# Query below can be executed, since c is primary key. +query III rowsort +SELECT c, a, SUM(d) +FROM unbounded_multiple_ordered_table_with_pk +GROUP BY c +ORDER BY c +LIMIT 5 +---- +0 0 0 +1 0 2 +2 0 0 +3 0 0 +4 0 1 From 91cc573d89fbcf9342b968760a4d0f9a47072527 Mon Sep 17 00:00:00 2001 From: Andrew Lamb Date: Fri, 8 Dec 2023 16:44:32 -0500 Subject: [PATCH 398/572] Add `evaluate_demo` and `range_analysis_demo` to Expr examples (#8377) * Add `evaluate_demo` and `range_analysis_demo` to Expr examples * Prettier * Update datafusion-examples/examples/expr_api.rs Co-authored-by: Raphael Taylor-Davies <1781103+tustvold@users.noreply.github.com> * rename ExprBoundaries::try_new_unknown --> ExprBoundaries::try_new_unbounded --------- Co-authored-by: Raphael Taylor-Davies <1781103+tustvold@users.noreply.github.com> --- datafusion-examples/README.md | 2 +- datafusion-examples/examples/expr_api.rs | 144 ++++++++++++++++-- datafusion/core/src/lib.rs | 10 +- datafusion/physical-expr/src/analysis.rs | 25 ++- .../library-user-guide/working-with-exprs.md | 13 +- 5 files changed, 174 insertions(+), 20 deletions(-) diff --git a/datafusion-examples/README.md b/datafusion-examples/README.md index 9f7c9f99d14e..305422ccd0be 100644 --- a/datafusion-examples/README.md +++ b/datafusion-examples/README.md @@ -50,7 +50,7 @@ cargo run --example csv_sql - [`dataframe-to-s3.rs`](examples/external_dependency/dataframe-to-s3.rs): Run a query using a DataFrame against a parquet file from s3 - [`dataframe_in_memory.rs`](examples/dataframe_in_memory.rs): Run a query using a DataFrame against data in memory - [`deserialize_to_struct.rs`](examples/deserialize_to_struct.rs): Convert query results into rust structs using serde -- [`expr_api.rs`](examples/expr_api.rs): Use the `Expr` construction and simplification API +- [`expr_api.rs`](examples/expr_api.rs): Create, execute, simplify and anaylze `Expr`s - [`flight_sql_server.rs`](examples/flight/flight_sql_server.rs): Run DataFusion as a standalone process and execute SQL queries from JDBC clients - [`memtable.rs`](examples/memtable.rs): Create an query data in memory using SQL and `RecordBatch`es - [`parquet_sql.rs`](examples/parquet_sql.rs): Build and run a query plan from a SQL statement against a local Parquet file diff --git a/datafusion-examples/examples/expr_api.rs b/datafusion-examples/examples/expr_api.rs index 97abf4d552a9..715e1ff2dce6 100644 --- a/datafusion-examples/examples/expr_api.rs +++ b/datafusion-examples/examples/expr_api.rs @@ -15,28 +15,43 @@ // specific language governing permissions and limitations // under the License. +use arrow::array::{BooleanArray, Int32Array}; +use arrow::record_batch::RecordBatch; use datafusion::arrow::datatypes::{DataType, Field, Schema, TimeUnit}; use datafusion::error::Result; use datafusion::optimizer::simplify_expressions::{ExprSimplifier, SimplifyContext}; use datafusion::physical_expr::execution_props::ExecutionProps; +use datafusion::physical_expr::{ + analyze, create_physical_expr, AnalysisContext, ExprBoundaries, PhysicalExpr, +}; use datafusion::prelude::*; use datafusion_common::{ScalarValue, ToDFSchema}; use datafusion_expr::expr::BinaryExpr; -use datafusion_expr::Operator; +use datafusion_expr::interval_arithmetic::Interval; +use datafusion_expr::{ColumnarValue, ExprSchemable, Operator}; +use std::sync::Arc; /// This example demonstrates the DataFusion [`Expr`] API. /// /// DataFusion comes with a powerful and extensive system for /// representing and manipulating expressions such as `A + 5` and `X -/// IN ('foo', 'bar', 'baz')` and many other constructs. +/// IN ('foo', 'bar', 'baz')`. +/// +/// In addition to building and manipulating [`Expr`]s, DataFusion +/// also comes with APIs for evaluation, simplification, and analysis. +/// +/// The code in this example shows how to: +/// 1. Create [`Exprs`] using different APIs: [`main`]` +/// 2. Evaluate [`Exprs`] against data: [`evaluate_demo`] +/// 3. Simplify expressions: [`simplify_demo`] +/// 4. Analyze predicates for boundary ranges: [`range_analysis_demo`] #[tokio::main] async fn main() -> Result<()> { // The easiest way to do create expressions is to use the - // "fluent"-style API, like this: + // "fluent"-style API: let expr = col("a") + lit(5); - // this creates the same expression as the following though with - // much less code, + // The same same expression can be created directly, with much more code: let expr2 = Expr::BinaryExpr(BinaryExpr::new( Box::new(col("a")), Operator::Plus, @@ -44,15 +59,51 @@ async fn main() -> Result<()> { )); assert_eq!(expr, expr2); + // See how to evaluate expressions + evaluate_demo()?; + + // See how to simplify expressions simplify_demo()?; + // See how to analyze ranges in expressions + range_analysis_demo()?; + + Ok(()) +} + +/// DataFusion can also evaluate arbitrary expressions on Arrow arrays. +fn evaluate_demo() -> Result<()> { + // For example, let's say you have some integers in an array + let batch = RecordBatch::try_from_iter([( + "a", + Arc::new(Int32Array::from(vec![4, 5, 6, 7, 8, 7, 4])) as _, + )])?; + + // If you want to find all rows where the expression `a < 5 OR a = 8` is true + let expr = col("a").lt(lit(5)).or(col("a").eq(lit(8))); + + // First, you make a "physical expression" from the logical `Expr` + let physical_expr = physical_expr(&batch.schema(), expr)?; + + // Now, you can evaluate the expression against the RecordBatch + let result = physical_expr.evaluate(&batch)?; + + // The result contain an array that is true only for where `a < 5 OR a = 8` + let expected_result = Arc::new(BooleanArray::from(vec![ + true, false, false, false, true, false, true, + ])) as _; + assert!( + matches!(&result, ColumnarValue::Array(r) if r == &expected_result), + "result: {:?}", + result + ); + Ok(()) } -/// In addition to easy construction, DataFusion exposes APIs for -/// working with and simplifying such expressions that call into the -/// same powerful and extensive implementation used for the query -/// engine. +/// In addition to easy construction, DataFusion exposes APIs for simplifying +/// such expression so they are more efficient to evaluate. This code is also +/// used by the query engine to optimize queries. fn simplify_demo() -> Result<()> { // For example, lets say you have has created an expression such // ts = to_timestamp("2020-09-08T12:00:00+00:00") @@ -94,7 +145,7 @@ fn simplify_demo() -> Result<()> { make_field("b", DataType::Boolean), ]) .to_dfschema_ref()?; - let context = SimplifyContext::new(&props).with_schema(schema); + let context = SimplifyContext::new(&props).with_schema(schema.clone()); let simplifier = ExprSimplifier::new(context); // basic arithmetic simplification @@ -120,6 +171,64 @@ fn simplify_demo() -> Result<()> { col("i").lt(lit(10)) ); + // String --> Date simplification + // `cast('2020-09-01' as date)` --> 18500 + assert_eq!( + simplifier.simplify(lit("2020-09-01").cast_to(&DataType::Date32, &schema)?)?, + lit(ScalarValue::Date32(Some(18506))) + ); + + Ok(()) +} + +/// DataFusion also has APIs for analyzing predicates (boolean expressions) to +/// determine any ranges restrictions on the inputs required for the predicate +/// evaluate to true. +fn range_analysis_demo() -> Result<()> { + // For example, let's say you are interested in finding data for all days + // in the month of September, 2020 + let september_1 = ScalarValue::Date32(Some(18506)); // 2020-09-01 + let october_1 = ScalarValue::Date32(Some(18536)); // 2020-10-01 + + // The predicate to find all such days could be + // `date > '2020-09-01' AND date < '2020-10-01'` + let expr = col("date") + .gt(lit(september_1.clone())) + .and(col("date").lt(lit(october_1.clone()))); + + // Using the analysis API, DataFusion can determine that the value of `date` + // must be in the range `['2020-09-01', '2020-10-01']`. If your data is + // organized in files according to day, this information permits skipping + // entire files without reading them. + // + // While this simple example could be handled with a special case, the + // DataFusion API handles arbitrary expressions (so for example, you don't + // have to handle the case where the predicate clauses are reversed such as + // `date < '2020-10-01' AND date > '2020-09-01'` + + // As always, we need to tell DataFusion the type of column "date" + let schema = Schema::new(vec![make_field("date", DataType::Date32)]); + + // You can provide DataFusion any known boundaries on the values of `date` + // (for example, maybe you know you only have data up to `2020-09-15`), but + // in this case, let's say we don't know any boundaries beforehand so we use + // `try_new_unknown` + let boundaries = ExprBoundaries::try_new_unbounded(&schema)?; + + // Now, we invoke the analysis code to perform the range analysis + let physical_expr = physical_expr(&schema, expr)?; + let analysis_result = + analyze(&physical_expr, AnalysisContext::new(boundaries), &schema)?; + + // The results of the analysis is an range, encoded as an `Interval`, for + // each column in the schema, that must be true in order for the predicate + // to be true. + // + // In this case, we can see that, as expected, `analyze` has figured out + // that in this case, `date` must be in the range `['2020-09-01', '2020-10-01']` + let expected_range = Interval::try_new(september_1, october_1)?; + assert_eq!(analysis_result.boundaries[0].interval, expected_range); + Ok(()) } @@ -132,3 +241,18 @@ fn make_ts_field(name: &str) -> Field { let tz = None; make_field(name, DataType::Timestamp(TimeUnit::Nanosecond, tz)) } + +/// Build a physical expression from a logical one, after applying simplification and type coercion +pub fn physical_expr(schema: &Schema, expr: Expr) -> Result> { + let df_schema = schema.clone().to_dfschema_ref()?; + + // Simplify + let props = ExecutionProps::new(); + let simplifier = + ExprSimplifier::new(SimplifyContext::new(&props).with_schema(df_schema.clone())); + + // apply type coercion here to ensure types match + let expr = simplifier.coerce(expr, df_schema.clone())?; + + create_physical_expr(&expr, df_schema.as_ref(), schema, &props) +} diff --git a/datafusion/core/src/lib.rs b/datafusion/core/src/lib.rs index bf9a4abf4f2d..b3ebbc6e3637 100644 --- a/datafusion/core/src/lib.rs +++ b/datafusion/core/src/lib.rs @@ -283,12 +283,20 @@ //! //! ## Plan Representations //! -//! Logical planning yields [`LogicalPlan`]s nodes and [`Expr`] +//! ### Logical Plans +//! Logical planning yields [`LogicalPlan`] nodes and [`Expr`] //! expressions which are [`Schema`] aware and represent statements //! independent of how they are physically executed. //! A [`LogicalPlan`] is a Directed Acyclic Graph (DAG) of other //! [`LogicalPlan`]s, each potentially containing embedded [`Expr`]s. //! +//! Examples of working with and executing `Expr`s can be found in the +//! [`expr_api`.rs] example +//! +//! [`expr_api`.rs]: https://github.com/apache/arrow-datafusion/blob/main/datafusion-examples/examples/expr_api.rs +//! +//! ### Physical Plans +//! //! An [`ExecutionPlan`] (sometimes referred to as a "physical plan") //! is a plan that can be executed against data. It a DAG of other //! [`ExecutionPlan`]s each potentially containing expressions of the diff --git a/datafusion/physical-expr/src/analysis.rs b/datafusion/physical-expr/src/analysis.rs index dc12bdf46acd..f43434362a19 100644 --- a/datafusion/physical-expr/src/analysis.rs +++ b/datafusion/physical-expr/src/analysis.rs @@ -72,8 +72,12 @@ impl AnalysisContext { } } -/// Represents the boundaries of the resulting value from a physical expression, -/// if it were to be an expression, if it were to be evaluated. +/// Represents the boundaries (e.g. min and max values) of a particular column +/// +/// This is used range analysis of expressions, to determine if the expression +/// limits the value of particular columns (e.g. analyzing an expression such as +/// `time < 50` would result in a boundary interval for `time` having a max +/// value of `50`). #[derive(Clone, Debug, PartialEq)] pub struct ExprBoundaries { pub column: Column, @@ -111,6 +115,23 @@ impl ExprBoundaries { distinct_count: col_stats.distinct_count.clone(), }) } + + /// Create `ExprBoundaries` that represent no known bounds for all the + /// columns in `schema` + pub fn try_new_unbounded(schema: &Schema) -> Result> { + schema + .fields() + .iter() + .enumerate() + .map(|(i, field)| { + Ok(Self { + column: Column::new(field.name(), i), + interval: Interval::make_unbounded(field.data_type())?, + distinct_count: Precision::Absent, + }) + }) + .collect() + } } /// Attempts to refine column boundaries and compute a selectivity value. diff --git a/docs/source/library-user-guide/working-with-exprs.md b/docs/source/library-user-guide/working-with-exprs.md index a8baf24d5f0a..96be8ef7f1ae 100644 --- a/docs/source/library-user-guide/working-with-exprs.md +++ b/docs/source/library-user-guide/working-with-exprs.md @@ -17,7 +17,7 @@ under the License. --> -# Working with Exprs +# Working with `Expr`s @@ -48,12 +48,11 @@ As another example, the SQL expression `a + b * c` would be represented as an `E └────────────────────┘ └────────────────────┘ ``` -As the writer of a library, you may want to use or create `Expr`s to represent computations that you want to perform. This guide will walk you through how to make your own scalar UDF as an `Expr` and how to rewrite `Expr`s to inline the simple UDF. +As the writer of a library, you can use `Expr`s to represent computations that you want to perform. This guide will walk you through how to make your own scalar UDF as an `Expr` and how to rewrite `Expr`s to inline the simple UDF. -There are also executable examples for working with `Expr`s: +## Creating and Evaluating `Expr`s -- [rewrite_expr.rs](https://github.com/apache/arrow-datafusion/blob/main/datafusion-examples/examples/rewrite_expr.rs) -- [expr_api.rs](https://github.com/apache/arrow-datafusion/blob/main/datafusion-examples/examples/expr_api.rs) +Please see [expr_api.rs](https://github.com/apache/arrow-datafusion/blob/main/datafusion-examples/examples/expr_api.rs) for well commented code for creating, evaluating, simplifying, and analyzing `Expr`s. ## A Scalar UDF Example @@ -79,7 +78,9 @@ let expr = add_one_udf.call(vec![col("my_column")]); If you'd like to learn more about `Expr`s, before we get into the details of creating and rewriting them, you can read the [expression user-guide](./../user-guide/expressions.md). -## Rewriting Exprs +## Rewriting `Expr`s + +[rewrite_expr.rs](https://github.com/apache/arrow-datafusion/blob/main/datafusion-examples/examples/rewrite_expr.rs) contains example code for rewriting `Expr`s. Rewriting Expressions is the process of taking an `Expr` and transforming it into another `Expr`. This is useful for a number of reasons, including: From ac4adfac3ea0306dfdbc23e1e0ee65d0e192f784 Mon Sep 17 00:00:00 2001 From: Alex Huang Date: Fri, 8 Dec 2023 22:44:50 +0100 Subject: [PATCH 399/572] fix typo (#8473) --- datafusion/physical-expr/src/datetime_expressions.rs | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/datafusion/physical-expr/src/datetime_expressions.rs b/datafusion/physical-expr/src/datetime_expressions.rs index d634b4d01918..bbeb2b0dce86 100644 --- a/datafusion/physical-expr/src/datetime_expressions.rs +++ b/datafusion/physical-expr/src/datetime_expressions.rs @@ -419,7 +419,7 @@ pub fn date_trunc(args: &[ColumnarValue]) -> Result { Ok(ColumnarValue::Array(Arc::new(array))) } - fn process_scalr( + fn process_scalar( v: &Option, granularity: String, tz_opt: &Option>, @@ -432,16 +432,16 @@ pub fn date_trunc(args: &[ColumnarValue]) -> Result { Ok(match array { ColumnarValue::Scalar(ScalarValue::TimestampNanosecond(v, tz_opt)) => { - process_scalr::(v, granularity, tz_opt)? + process_scalar::(v, granularity, tz_opt)? } ColumnarValue::Scalar(ScalarValue::TimestampMicrosecond(v, tz_opt)) => { - process_scalr::(v, granularity, tz_opt)? + process_scalar::(v, granularity, tz_opt)? } ColumnarValue::Scalar(ScalarValue::TimestampMillisecond(v, tz_opt)) => { - process_scalr::(v, granularity, tz_opt)? + process_scalar::(v, granularity, tz_opt)? } ColumnarValue::Scalar(ScalarValue::TimestampSecond(v, tz_opt)) => { - process_scalr::(v, granularity, tz_opt)? + process_scalar::(v, granularity, tz_opt)? } ColumnarValue::Array(array) => { let array_type = array.data_type(); From 2765fee4dd7518a4dc39244d1b495329b9771be1 Mon Sep 17 00:00:00 2001 From: Keunwoo Lee Date: Fri, 8 Dec 2023 13:45:24 -0800 Subject: [PATCH 400/572] Fix comment typo in table.rs: s/indentical/identical/ (#8469) --- datafusion/core/src/datasource/listing/table.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/datafusion/core/src/datasource/listing/table.rs b/datafusion/core/src/datasource/listing/table.rs index a7f69a1d3cc8..10ec9f8d8d3a 100644 --- a/datafusion/core/src/datasource/listing/table.rs +++ b/datafusion/core/src/datasource/listing/table.rs @@ -490,7 +490,7 @@ impl ListingOptions { /// /// # Features /// -/// 1. Merges schemas if the files have compatible but not indentical schemas +/// 1. Merges schemas if the files have compatible but not identical schemas /// /// 2. Hive-style partitioning support, where a path such as /// `/files/date=1/1/2022/data.parquet` is injected as a `date` column. From 182a37eaf48e89a84dcd241880bd970c8b0b9363 Mon Sep 17 00:00:00 2001 From: Jay Zhan Date: Sat, 9 Dec 2023 19:57:56 +0800 Subject: [PATCH 401/572] Remove `define_array_slice` and reuse `array_slice` for `array_pop_front/back` (#8401) * array_element done Signed-off-by: jayzhan211 * clippy Signed-off-by: jayzhan211 * replace array_slice Signed-off-by: jayzhan211 * fix get_indexed_field_empty_list Signed-off-by: jayzhan211 * replace pop front and pop back Signed-off-by: jayzhan211 * clippy Signed-off-by: jayzhan211 * add doc and comment Signed-off-by: jayzhan211 * fmt Signed-off-by: jayzhan211 --------- Signed-off-by: jayzhan211 --- .../physical-expr/src/array_expressions.rs | 337 +++++++++--------- .../src/expressions/get_indexed_field.rs | 2 +- 2 files changed, 179 insertions(+), 160 deletions(-) diff --git a/datafusion/physical-expr/src/array_expressions.rs b/datafusion/physical-expr/src/array_expressions.rs index ae048694583b..c2dc88b10773 100644 --- a/datafusion/physical-expr/src/array_expressions.rs +++ b/datafusion/physical-expr/src/array_expressions.rs @@ -18,7 +18,6 @@ //! Array expressions use std::any::type_name; -use std::cmp::Ordering; use std::collections::HashSet; use std::sync::Arc; @@ -370,135 +369,64 @@ pub fn make_array(arrays: &[ArrayRef]) -> Result { } } -fn return_empty(return_null: bool, data_type: DataType) -> Arc { - if return_null { - new_null_array(&data_type, 1) - } else { - new_empty_array(&data_type) - } -} - -fn list_slice( - array: &dyn Array, - i: i64, - j: i64, - return_element: bool, -) -> ArrayRef { - let array = array.as_any().downcast_ref::().unwrap(); - - let array_type = array.data_type().clone(); +/// array_element SQL function +/// +/// There are two arguments for array_element, the first one is the array, the second one is the 1-indexed index. +/// `array_element(array, index)` +/// +/// For example: +/// > array_element(\[1, 2, 3], 2) -> 2 +pub fn array_element(args: &[ArrayRef]) -> Result { + let list_array = as_list_array(&args[0])?; + let indexes = as_int64_array(&args[1])?; - if i == 0 && j == 0 || array.is_empty() { - return return_empty(return_element, array_type); - } + let values = list_array.values(); + let original_data = values.to_data(); + let capacity = Capacities::Array(original_data.len()); - let i = match i.cmp(&0) { - Ordering::Less => { - if i.unsigned_abs() > array.len() as u64 { - return return_empty(true, array_type); - } + // use_nulls: true, we don't construct List for array_element, so we need explicit nulls. + let mut mutable = + MutableArrayData::with_capacities(vec![&original_data], true, capacity); - (array.len() as i64 + i + 1) as usize - } - Ordering::Equal => 1, - Ordering::Greater => i as usize, - }; + fn adjusted_array_index(index: i64, len: usize) -> Option { + // 0 ~ len - 1 + let adjusted_zero_index = if index < 0 { + index + len as i64 + } else { + index - 1 + }; - let j = match j.cmp(&0) { - Ordering::Less => { - if j.unsigned_abs() as usize > array.len() { - return return_empty(true, array_type); - } - if return_element { - (array.len() as i64 + j + 1) as usize - } else { - (array.len() as i64 + j) as usize - } + if 0 <= adjusted_zero_index && adjusted_zero_index < len as i64 { + Some(adjusted_zero_index) + } else { + // Out of bounds + None } - Ordering::Equal => 1, - Ordering::Greater => j.min(array.len() as i64) as usize, - }; - - if i > j || i > array.len() { - return_empty(return_element, array_type) - } else { - Arc::new(array.slice(i - 1, j + 1 - i)) } -} -fn slice( - array: &ListArray, - key: &Int64Array, - extra_key: &Int64Array, - return_element: bool, -) -> Result> { - let sliced_array: Vec> = array - .iter() - .zip(key.iter()) - .zip(extra_key.iter()) - .map(|((arr, i), j)| match (arr, i, j) { - (Some(arr), Some(i), Some(j)) => list_slice::(&arr, i, j, return_element), - (Some(arr), None, Some(j)) => list_slice::(&arr, 1i64, j, return_element), - (Some(arr), Some(i), None) => { - list_slice::(&arr, i, arr.len() as i64, return_element) - } - (Some(arr), None, None) if !return_element => arr.clone(), - _ => return_empty(return_element, array.value_type()), - }) - .collect(); + for (row_index, offset_window) in list_array.offsets().windows(2).enumerate() { + let start = offset_window[0] as usize; + let end = offset_window[1] as usize; + let len = end - start; - // concat requires input of at least one array - if sliced_array.is_empty() { - Ok(return_empty(return_element, array.value_type())) - } else { - let vec = sliced_array - .iter() - .map(|a| a.as_ref()) - .collect::>(); - let mut i: i32 = 0; - let mut offsets = vec![i]; - offsets.extend( - vec.iter() - .map(|a| { - i += a.len() as i32; - i - }) - .collect::>(), - ); - let values = compute::concat(vec.as_slice()).unwrap(); + // array is null + if len == 0 { + mutable.extend_nulls(1); + continue; + } + + let index = adjusted_array_index(indexes.value(row_index), len); - if return_element { - Ok(values) + if let Some(index) = index { + mutable.extend(0, start + index as usize, start + index as usize + 1); } else { - let field = Arc::new(Field::new("item", array.value_type(), true)); - Ok(Arc::new(ListArray::try_new( - field, - OffsetBuffer::new(offsets.into()), - values, - None, - )?)) + // Index out of bounds + mutable.extend_nulls(1); } } -} - -fn define_array_slice( - list_array: &ListArray, - key: &Int64Array, - extra_key: &Int64Array, - return_element: bool, -) -> Result { - macro_rules! array_function { - ($ARRAY_TYPE:ident) => { - slice::<$ARRAY_TYPE>(list_array, key, extra_key, return_element) - }; - } - call_array_function!(list_array.value_type(), true) -} -pub fn array_element(args: &[ArrayRef]) -> Result { - let list_array = as_list_array(&args[0])?; - let key = as_int64_array(&args[1])?; - define_array_slice(list_array, key, key, true) + let data = mutable.freeze(); + Ok(arrow_array::make_array(data)) } fn general_except( @@ -579,47 +507,136 @@ pub fn array_except(args: &[ArrayRef]) -> Result { } } +/// array_slice SQL function +/// +/// We follow the behavior of array_slice in DuckDB +/// Note that array_slice is 1-indexed. And there are two additional arguments `from` and `to` in array_slice. +/// +/// > array_slice(array, from, to) +/// +/// Positive index is treated as the index from the start of the array. If the +/// `from` index is smaller than 1, it is treated as 1. If the `to` index is larger than the +/// length of the array, it is treated as the length of the array. +/// +/// Negative index is treated as the index from the end of the array. If the index +/// is larger than the length of the array, it is NOT VALID, either in `from` or `to`. +/// The `to` index is exclusive like python slice syntax. +/// +/// See test cases in `array.slt` for more details. pub fn array_slice(args: &[ArrayRef]) -> Result { let list_array = as_list_array(&args[0])?; - let key = as_int64_array(&args[1])?; - let extra_key = as_int64_array(&args[2])?; - define_array_slice(list_array, key, extra_key, false) -} - -fn general_array_pop( - list_array: &GenericListArray, - from_back: bool, -) -> Result<(Vec, Vec)> { - if from_back { - let key = vec![0; list_array.len()]; - // Attention: `arr.len() - 1` in extra key defines the last element position (position = index + 1, not inclusive) we want in the new array. - let extra_key: Vec<_> = list_array - .iter() - .map(|x| x.map_or(0, |arr| arr.len() as i64 - 1)) - .collect(); - Ok((key, extra_key)) - } else { - // Attention: 2 in the `key`` defines the first element position (position = index + 1) we want in the new array. - // We only handle two cases of the first element index: if the old array has any elements, starts from 2 (index + 1), or starts from initial. - let key: Vec<_> = list_array.iter().map(|x| x.map_or(0, |_| 2)).collect(); - let extra_key: Vec<_> = list_array - .iter() - .map(|x| x.map_or(0, |arr| arr.len() as i64)) - .collect(); - Ok((key, extra_key)) + let from_array = as_int64_array(&args[1])?; + let to_array = as_int64_array(&args[2])?; + + let values = list_array.values(); + let original_data = values.to_data(); + let capacity = Capacities::Array(original_data.len()); + + // use_nulls: false, we don't need nulls but empty array for array_slice, so we don't need explicit nulls but adjust offset to indicate nulls. + let mut mutable = + MutableArrayData::with_capacities(vec![&original_data], false, capacity); + + // We have the slice syntax compatible with DuckDB v0.8.1. + // The rule `adjusted_from_index` and `adjusted_to_index` follows the rule of array_slice in duckdb. + + fn adjusted_from_index(index: i64, len: usize) -> Option { + // 0 ~ len - 1 + let adjusted_zero_index = if index < 0 { + index + len as i64 + } else { + // array_slice(arr, 1, to) is the same as array_slice(arr, 0, to) + std::cmp::max(index - 1, 0) + }; + + if 0 <= adjusted_zero_index && adjusted_zero_index < len as i64 { + Some(adjusted_zero_index) + } else { + // Out of bounds + None + } + } + + fn adjusted_to_index(index: i64, len: usize) -> Option { + // 0 ~ len - 1 + let adjusted_zero_index = if index < 0 { + // array_slice in duckdb with negative to_index is python-like, so index itself is exclusive + index + len as i64 - 1 + } else { + // array_slice(arr, from, len + 1) is the same as array_slice(arr, from, len) + std::cmp::min(index - 1, len as i64 - 1) + }; + + if 0 <= adjusted_zero_index && adjusted_zero_index < len as i64 { + Some(adjusted_zero_index) + } else { + // Out of bounds + None + } + } + + let mut offsets = vec![0]; + + for (row_index, offset_window) in list_array.offsets().windows(2).enumerate() { + let start = offset_window[0] as usize; + let end = offset_window[1] as usize; + let len = end - start; + + // len 0 indicate array is null, return empty array in this row. + if len == 0 { + offsets.push(offsets[row_index]); + continue; + } + + // If index is null, we consider it as the minimum / maximum index of the array. + let from_index = if from_array.is_null(row_index) { + Some(0) + } else { + adjusted_from_index(from_array.value(row_index), len) + }; + + let to_index = if to_array.is_null(row_index) { + Some(len as i64 - 1) + } else { + adjusted_to_index(to_array.value(row_index), len) + }; + + if let (Some(from), Some(to)) = (from_index, to_index) { + if from <= to { + assert!(start + to as usize <= end); + mutable.extend(0, start + from as usize, start + to as usize + 1); + offsets.push(offsets[row_index] + (to - from + 1) as i32); + } else { + // invalid range, return empty array + offsets.push(offsets[row_index]); + } + } else { + // invalid range, return empty array + offsets.push(offsets[row_index]); + } } + + let data = mutable.freeze(); + + Ok(Arc::new(ListArray::try_new( + Arc::new(Field::new("item", list_array.value_type(), true)), + OffsetBuffer::new(offsets.into()), + arrow_array::make_array(data), + None, + )?)) } +/// array_pop_back SQL function pub fn array_pop_back(args: &[ArrayRef]) -> Result { let list_array = as_list_array(&args[0])?; - let (key, extra_key) = general_array_pop(list_array, true)?; - - define_array_slice( - list_array, - &Int64Array::from(key), - &Int64Array::from(extra_key), - false, - ) + let from_array = Int64Array::from(vec![1; list_array.len()]); + let to_array = Int64Array::from( + list_array + .iter() + .map(|arr| arr.map_or(0, |arr| arr.len() as i64 - 1)) + .collect::>(), + ); + let args = vec![args[0].clone(), Arc::new(from_array), Arc::new(to_array)]; + array_slice(args.as_slice()) } /// Appends or prepends elements to a ListArray. @@ -743,16 +760,18 @@ pub fn gen_range(args: &[ArrayRef]) -> Result { Ok(arr) } +/// array_pop_front SQL function pub fn array_pop_front(args: &[ArrayRef]) -> Result { let list_array = as_list_array(&args[0])?; - let (key, extra_key) = general_array_pop(list_array, false)?; - - define_array_slice( - list_array, - &Int64Array::from(key), - &Int64Array::from(extra_key), - false, - ) + let from_array = Int64Array::from(vec![2; list_array.len()]); + let to_array = Int64Array::from( + list_array + .iter() + .map(|arr| arr.map_or(0, |arr| arr.len() as i64)) + .collect::>(), + ); + let args = vec![args[0].clone(), Arc::new(from_array), Arc::new(to_array)]; + array_slice(args.as_slice()) } /// Array_append SQL function diff --git a/datafusion/physical-expr/src/expressions/get_indexed_field.rs b/datafusion/physical-expr/src/expressions/get_indexed_field.rs index 9c2a64723dc6..43fd5a812a16 100644 --- a/datafusion/physical-expr/src/expressions/get_indexed_field.rs +++ b/datafusion/physical-expr/src/expressions/get_indexed_field.rs @@ -453,7 +453,7 @@ mod tests { .evaluate(&batch)? .into_array(batch.num_rows()) .expect("Failed to convert to array"); - assert!(result.is_null(0)); + assert!(result.is_empty()); Ok(()) } From 62ee8fb048b8108a45f8a3ef06f1b2a56dce3d3f Mon Sep 17 00:00:00 2001 From: Alex Huang Date: Sat, 9 Dec 2023 12:58:41 +0100 Subject: [PATCH 402/572] Minor: refactor `trim` to clean up duplicated code (#8434) * refactor trim * add fmt for TrimType * fix closure * update comment --- .../physical-expr/src/string_expressions.rs | 169 +++++++----------- 1 file changed, 69 insertions(+), 100 deletions(-) diff --git a/datafusion/physical-expr/src/string_expressions.rs b/datafusion/physical-expr/src/string_expressions.rs index 91d21f95e41f..7d9fecf61407 100644 --- a/datafusion/physical-expr/src/string_expressions.rs +++ b/datafusion/physical-expr/src/string_expressions.rs @@ -37,8 +37,11 @@ use datafusion_common::{ }; use datafusion_common::{internal_err, DataFusionError, Result}; use datafusion_expr::ColumnarValue; -use std::iter; use std::sync::Arc; +use std::{ + fmt::{Display, Formatter}, + iter, +}; use uuid::Uuid; /// applies a unary expression to `args[0]` that is expected to be downcastable to @@ -133,53 +136,6 @@ pub fn ascii(args: &[ArrayRef]) -> Result { Ok(Arc::new(result) as ArrayRef) } -/// Removes the longest string containing only characters in characters (a space by default) from the start and end of string. -/// btrim('xyxtrimyyx', 'xyz') = 'trim' -pub fn btrim(args: &[ArrayRef]) -> Result { - match args.len() { - 1 => { - let string_array = as_generic_string_array::(&args[0])?; - - let result = string_array - .iter() - .map(|string| { - string.map(|string: &str| { - string.trim_start_matches(' ').trim_end_matches(' ') - }) - }) - .collect::>(); - - Ok(Arc::new(result) as ArrayRef) - } - 2 => { - let string_array = as_generic_string_array::(&args[0])?; - let characters_array = as_generic_string_array::(&args[1])?; - - let result = string_array - .iter() - .zip(characters_array.iter()) - .map(|(string, characters)| match (string, characters) { - (None, _) => None, - (_, None) => None, - (Some(string), Some(characters)) => { - let chars: Vec = characters.chars().collect(); - Some( - string - .trim_start_matches(&chars[..]) - .trim_end_matches(&chars[..]), - ) - } - }) - .collect::>(); - - Ok(Arc::new(result) as ArrayRef) - } - other => internal_err!( - "btrim was called with {other} arguments. It requires at least 1 and at most 2." - ), - } -} - /// Returns the character with the given code. chr(0) is disallowed because text data types cannot store that character. /// chr(65) = 'A' pub fn chr(args: &[ArrayRef]) -> Result { @@ -346,44 +302,95 @@ pub fn lower(args: &[ColumnarValue]) -> Result { handle(args, |string| string.to_ascii_lowercase(), "lower") } -/// Removes the longest string containing only characters in characters (a space by default) from the start of string. -/// ltrim('zzzytest', 'xyz') = 'test' -pub fn ltrim(args: &[ArrayRef]) -> Result { +enum TrimType { + Left, + Right, + Both, +} + +impl Display for TrimType { + fn fmt(&self, f: &mut Formatter<'_>) -> std::fmt::Result { + match self { + TrimType::Left => write!(f, "ltrim"), + TrimType::Right => write!(f, "rtrim"), + TrimType::Both => write!(f, "btrim"), + } + } +} + +fn general_trim( + args: &[ArrayRef], + trim_type: TrimType, +) -> Result { + let func = match trim_type { + TrimType::Left => |input, pattern: &str| { + let pattern = pattern.chars().collect::>(); + str::trim_start_matches::<&[char]>(input, pattern.as_ref()) + }, + TrimType::Right => |input, pattern: &str| { + let pattern = pattern.chars().collect::>(); + str::trim_end_matches::<&[char]>(input, pattern.as_ref()) + }, + TrimType::Both => |input, pattern: &str| { + let pattern = pattern.chars().collect::>(); + str::trim_end_matches::<&[char]>( + str::trim_start_matches::<&[char]>(input, pattern.as_ref()), + pattern.as_ref(), + ) + }, + }; + + let string_array = as_generic_string_array::(&args[0])?; + match args.len() { 1 => { - let string_array = as_generic_string_array::(&args[0])?; - let result = string_array .iter() - .map(|string| string.map(|string: &str| string.trim_start_matches(' '))) + .map(|string| string.map(|string: &str| func(string, " "))) .collect::>(); Ok(Arc::new(result) as ArrayRef) } 2 => { - let string_array = as_generic_string_array::(&args[0])?; let characters_array = as_generic_string_array::(&args[1])?; let result = string_array .iter() .zip(characters_array.iter()) .map(|(string, characters)| match (string, characters) { - (Some(string), Some(characters)) => { - let chars: Vec = characters.chars().collect(); - Some(string.trim_start_matches(&chars[..])) - } + (Some(string), Some(characters)) => Some(func(string, characters)), _ => None, }) .collect::>(); Ok(Arc::new(result) as ArrayRef) } - other => internal_err!( - "ltrim was called with {other} arguments. It requires at least 1 and at most 2." - ), + other => { + internal_err!( + "{trim_type} was called with {other} arguments. It requires at least 1 and at most 2." + ) + } } } +/// Returns the longest string with leading and trailing characters removed. If the characters are not specified, whitespace is removed. +/// btrim('xyxtrimyyx', 'xyz') = 'trim' +pub fn btrim(args: &[ArrayRef]) -> Result { + general_trim::(args, TrimType::Both) +} + +/// Returns the longest string with leading characters removed. If the characters are not specified, whitespace is removed. +/// ltrim('zzzytest', 'xyz') = 'test' +pub fn ltrim(args: &[ArrayRef]) -> Result { + general_trim::(args, TrimType::Left) +} + +/// Returns the longest string with trailing characters removed. If the characters are not specified, whitespace is removed. +/// rtrim('testxxzx', 'xyz') = 'test' +pub fn rtrim(args: &[ArrayRef]) -> Result { + general_trim::(args, TrimType::Right) +} + /// Repeats string the specified number of times. /// repeat('Pg', 4) = 'PgPgPgPg' pub fn repeat(args: &[ArrayRef]) -> Result { @@ -422,44 +429,6 @@ pub fn replace(args: &[ArrayRef]) -> Result { Ok(Arc::new(result) as ArrayRef) } -/// Removes the longest string containing only characters in characters (a space by default) from the end of string. -/// rtrim('testxxzx', 'xyz') = 'test' -pub fn rtrim(args: &[ArrayRef]) -> Result { - match args.len() { - 1 => { - let string_array = as_generic_string_array::(&args[0])?; - - let result = string_array - .iter() - .map(|string| string.map(|string: &str| string.trim_end_matches(' '))) - .collect::>(); - - Ok(Arc::new(result) as ArrayRef) - } - 2 => { - let string_array = as_generic_string_array::(&args[0])?; - let characters_array = as_generic_string_array::(&args[1])?; - - let result = string_array - .iter() - .zip(characters_array.iter()) - .map(|(string, characters)| match (string, characters) { - (Some(string), Some(characters)) => { - let chars: Vec = characters.chars().collect(); - Some(string.trim_end_matches(&chars[..])) - } - _ => None, - }) - .collect::>(); - - Ok(Arc::new(result) as ArrayRef) - } - other => internal_err!( - "rtrim was called with {other} arguments. It requires at least 1 and at most 2." - ), - } -} - /// Splits string at occurrences of delimiter and returns the n'th field (counting from one). /// split_part('abc~@~def~@~ghi', '~@~', 2) = 'def' pub fn split_part(args: &[ArrayRef]) -> Result { From d091b55be6a4ce552023ef162b5d081136d3ff6d Mon Sep 17 00:00:00 2001 From: Mohammad Razeghi Date: Sat, 9 Dec 2023 13:23:34 +0100 Subject: [PATCH 403/572] Split `EmptyExec` into `PlaceholderRowExec` (#8446) * add PlaceHolderRowExec * Change produce_one_row=true calls to use PlaceHolderRowExec * remove produce_one_row from EmptyExec, changes in proto serializer, working tests * PlaceHolder => Placeholder --------- Co-authored-by: Andrew Lamb --- datafusion/core/src/datasource/empty.rs | 2 +- .../core/src/datasource/listing/table.rs | 4 +- .../aggregate_statistics.rs | 4 +- .../src/physical_optimizer/join_selection.rs | 4 +- datafusion/core/src/physical_planner.rs | 12 +- datafusion/core/tests/custom_sources.rs | 12 +- datafusion/core/tests/sql/explain_analyze.rs | 4 +- datafusion/expr/src/logical_plan/plan.rs | 2 +- datafusion/optimizer/README.md | 6 +- datafusion/physical-plan/src/display.rs | 2 +- datafusion/physical-plan/src/empty.rs | 93 +------ datafusion/physical-plan/src/lib.rs | 1 + .../physical-plan/src/placeholder_row.rs | 229 ++++++++++++++++++ datafusion/proto/proto/datafusion.proto | 8 +- datafusion/proto/src/generated/pbjson.rs | 123 ++++++++-- datafusion/proto/src/generated/prost.rs | 14 +- datafusion/proto/src/physical_plan/mod.rs | 19 +- .../tests/cases/roundtrip_physical_plan.rs | 63 ++--- .../sqllogictest/test_files/explain.slt | 6 +- datafusion/sqllogictest/test_files/join.slt | 2 +- datafusion/sqllogictest/test_files/limit.slt | 6 +- datafusion/sqllogictest/test_files/union.slt | 10 +- datafusion/sqllogictest/test_files/window.slt | 16 +- 23 files changed, 459 insertions(+), 183 deletions(-) create mode 100644 datafusion/physical-plan/src/placeholder_row.rs diff --git a/datafusion/core/src/datasource/empty.rs b/datafusion/core/src/datasource/empty.rs index 77160aa5d1c0..5100987520ee 100644 --- a/datafusion/core/src/datasource/empty.rs +++ b/datafusion/core/src/datasource/empty.rs @@ -77,7 +77,7 @@ impl TableProvider for EmptyTable { // even though there is no data, projections apply let projected_schema = project_schema(&self.schema, projection)?; Ok(Arc::new( - EmptyExec::new(false, projected_schema).with_partitions(self.partitions), + EmptyExec::new(projected_schema).with_partitions(self.partitions), )) } } diff --git a/datafusion/core/src/datasource/listing/table.rs b/datafusion/core/src/datasource/listing/table.rs index 10ec9f8d8d3a..0ce1b43fe456 100644 --- a/datafusion/core/src/datasource/listing/table.rs +++ b/datafusion/core/src/datasource/listing/table.rs @@ -685,7 +685,7 @@ impl TableProvider for ListingTable { if partitioned_file_lists.is_empty() { let schema = self.schema(); let projected_schema = project_schema(&schema, projection)?; - return Ok(Arc::new(EmptyExec::new(false, projected_schema))); + return Ok(Arc::new(EmptyExec::new(projected_schema))); } // extract types of partition columns @@ -713,7 +713,7 @@ impl TableProvider for ListingTable { let object_store_url = if let Some(url) = self.table_paths.first() { url.object_store() } else { - return Ok(Arc::new(EmptyExec::new(false, Arc::new(Schema::empty())))); + return Ok(Arc::new(EmptyExec::new(Arc::new(Schema::empty())))); }; // create the execution plan self.options diff --git a/datafusion/core/src/physical_optimizer/aggregate_statistics.rs b/datafusion/core/src/physical_optimizer/aggregate_statistics.rs index 4265e3ff80d0..795857b10ef5 100644 --- a/datafusion/core/src/physical_optimizer/aggregate_statistics.rs +++ b/datafusion/core/src/physical_optimizer/aggregate_statistics.rs @@ -22,7 +22,6 @@ use super::optimizer::PhysicalOptimizerRule; use crate::config::ConfigOptions; use crate::error::Result; use crate::physical_plan::aggregates::AggregateExec; -use crate::physical_plan::empty::EmptyExec; use crate::physical_plan::projection::ProjectionExec; use crate::physical_plan::{expressions, AggregateExpr, ExecutionPlan, Statistics}; use crate::scalar::ScalarValue; @@ -30,6 +29,7 @@ use crate::scalar::ScalarValue; use datafusion_common::stats::Precision; use datafusion_common::tree_node::TreeNode; use datafusion_expr::utils::COUNT_STAR_EXPANSION; +use datafusion_physical_plan::placeholder_row::PlaceholderRowExec; /// Optimizer that uses available statistics for aggregate functions #[derive(Default)] @@ -82,7 +82,7 @@ impl PhysicalOptimizerRule for AggregateStatistics { // input can be entirely removed Ok(Arc::new(ProjectionExec::try_new( projections, - Arc::new(EmptyExec::new(true, plan.schema())), + Arc::new(PlaceholderRowExec::new(plan.schema())), )?)) } else { plan.map_children(|child| self.optimize(child, _config)) diff --git a/datafusion/core/src/physical_optimizer/join_selection.rs b/datafusion/core/src/physical_optimizer/join_selection.rs index 0c3ac2d24529..6b2fe24acf00 100644 --- a/datafusion/core/src/physical_optimizer/join_selection.rs +++ b/datafusion/core/src/physical_optimizer/join_selection.rs @@ -1623,12 +1623,12 @@ mod hash_join_tests { let children = vec![ PipelineStatePropagator { - plan: Arc::new(EmptyExec::new(false, Arc::new(Schema::empty()))), + plan: Arc::new(EmptyExec::new(Arc::new(Schema::empty()))), unbounded: left_unbounded, children: vec![], }, PipelineStatePropagator { - plan: Arc::new(EmptyExec::new(false, Arc::new(Schema::empty()))), + plan: Arc::new(EmptyExec::new(Arc::new(Schema::empty()))), unbounded: right_unbounded, children: vec![], }, diff --git a/datafusion/core/src/physical_planner.rs b/datafusion/core/src/physical_planner.rs index 38532002a634..ab38b3ec6d2f 100644 --- a/datafusion/core/src/physical_planner.rs +++ b/datafusion/core/src/physical_planner.rs @@ -91,6 +91,7 @@ use datafusion_expr::{ WindowFrameBound, WriteOp, }; use datafusion_physical_expr::expressions::Literal; +use datafusion_physical_plan::placeholder_row::PlaceholderRowExec; use datafusion_sql::utils::window_expr_common_partition_keys; use async_trait::async_trait; @@ -1196,10 +1197,15 @@ impl DefaultPhysicalPlanner { } LogicalPlan::Subquery(_) => todo!(), LogicalPlan::EmptyRelation(EmptyRelation { - produce_one_row, + produce_one_row: false, schema, }) => Ok(Arc::new(EmptyExec::new( - *produce_one_row, + SchemaRef::new(schema.as_ref().to_owned().into()), + ))), + LogicalPlan::EmptyRelation(EmptyRelation { + produce_one_row: true, + schema, + }) => Ok(Arc::new(PlaceholderRowExec::new( SchemaRef::new(schema.as_ref().to_owned().into()), ))), LogicalPlan::SubqueryAlias(SubqueryAlias { input, .. }) => { @@ -2767,7 +2773,7 @@ mod tests { digraph { 1[shape=box label="ProjectionExec: expr=[id@0 + 2 as employee.id + Int32(2)]", tooltip=""] - 2[shape=box label="EmptyExec: produce_one_row=false", tooltip=""] + 2[shape=box label="EmptyExec", tooltip=""] 1 -> 2 [arrowhead=none, arrowtail=normal, dir=back] } // End DataFusion GraphViz Plan diff --git a/datafusion/core/tests/custom_sources.rs b/datafusion/core/tests/custom_sources.rs index daf1ef41a297..a9ea5cc2a35c 100644 --- a/datafusion/core/tests/custom_sources.rs +++ b/datafusion/core/tests/custom_sources.rs @@ -30,7 +30,6 @@ use datafusion::execution::context::{SessionContext, SessionState, TaskContext}; use datafusion::logical_expr::{ col, Expr, LogicalPlan, LogicalPlanBuilder, TableScan, UNNAMED_TABLE, }; -use datafusion::physical_plan::empty::EmptyExec; use datafusion::physical_plan::expressions::PhysicalSortExpr; use datafusion::physical_plan::{ collect, ColumnStatistics, DisplayAs, DisplayFormatType, ExecutionPlan, Partitioning, @@ -42,6 +41,7 @@ use datafusion_common::project_schema; use datafusion_common::stats::Precision; use async_trait::async_trait; +use datafusion_physical_plan::placeholder_row::PlaceholderRowExec; use futures::stream::Stream; /// Also run all tests that are found in the `custom_sources_cases` directory @@ -256,9 +256,9 @@ async fn optimizers_catch_all_statistics() { let physical_plan = df.create_physical_plan().await.unwrap(); - // when the optimization kicks in, the source is replaced by an EmptyExec + // when the optimization kicks in, the source is replaced by an PlaceholderRowExec assert!( - contains_empty_exec(Arc::clone(&physical_plan)), + contains_place_holder_exec(Arc::clone(&physical_plan)), "Expected aggregate_statistics optimizations missing: {physical_plan:?}" ); @@ -283,12 +283,12 @@ async fn optimizers_catch_all_statistics() { assert_eq!(format!("{:?}", actual[0]), format!("{expected:?}")); } -fn contains_empty_exec(plan: Arc) -> bool { - if plan.as_any().is::() { +fn contains_place_holder_exec(plan: Arc) -> bool { + if plan.as_any().is::() { true } else if plan.children().len() != 1 { false } else { - contains_empty_exec(Arc::clone(&plan.children()[0])) + contains_place_holder_exec(Arc::clone(&plan.children()[0])) } } diff --git a/datafusion/core/tests/sql/explain_analyze.rs b/datafusion/core/tests/sql/explain_analyze.rs index ecb5766a3bb5..37f8cefc9080 100644 --- a/datafusion/core/tests/sql/explain_analyze.rs +++ b/datafusion/core/tests/sql/explain_analyze.rs @@ -575,7 +575,7 @@ async fn explain_analyze_runs_optimizers() { // This happens as an optimization pass where count(*) can be // answered using statistics only. - let expected = "EmptyExec: produce_one_row=true"; + let expected = "PlaceholderRowExec"; let sql = "EXPLAIN SELECT count(*) from alltypes_plain"; let actual = execute_to_batches(&ctx, sql).await; @@ -806,7 +806,7 @@ async fn explain_physical_plan_only() { let expected = vec![vec![ "physical_plan", "ProjectionExec: expr=[2 as COUNT(*)]\ - \n EmptyExec: produce_one_row=true\ + \n PlaceholderRowExec\ \n", ]]; assert_eq!(expected, actual); diff --git a/datafusion/expr/src/logical_plan/plan.rs b/datafusion/expr/src/logical_plan/plan.rs index dfd4fbf65d8e..d74015bf094d 100644 --- a/datafusion/expr/src/logical_plan/plan.rs +++ b/datafusion/expr/src/logical_plan/plan.rs @@ -1208,7 +1208,7 @@ impl LogicalPlan { self.with_new_exprs(new_exprs, &new_inputs_with_values) } - /// Walk the logical plan, find any `PlaceHolder` tokens, and return a map of their IDs and DataTypes + /// Walk the logical plan, find any `Placeholder` tokens, and return a map of their IDs and DataTypes pub fn get_parameter_types( &self, ) -> Result>, DataFusionError> { diff --git a/datafusion/optimizer/README.md b/datafusion/optimizer/README.md index b8e5b93e6692..4f9e0fb98526 100644 --- a/datafusion/optimizer/README.md +++ b/datafusion/optimizer/README.md @@ -153,7 +153,7 @@ Looking at the `EXPLAIN` output we can see that the optimizer has effectively re | logical_plan | Projection: Int64(3) AS Int64(1) + Int64(2) | | | EmptyRelation | | physical_plan | ProjectionExec: expr=[3 as Int64(1) + Int64(2)] | -| | EmptyExec: produce_one_row=true | +| | PlaceholderRowExec | | | | +---------------+-------------------------------------------------+ ``` @@ -318,7 +318,7 @@ In the following example, the `type_coercion` and `simplify_expressions` passes | logical_plan | Projection: Utf8("3.2") AS foo | | | EmptyRelation | | initial_physical_plan | ProjectionExec: expr=[3.2 as foo] | -| | EmptyExec: produce_one_row=true | +| | PlaceholderRowExec | | | | | physical_plan after aggregate_statistics | SAME TEXT AS ABOVE | | physical_plan after join_selection | SAME TEXT AS ABOVE | @@ -326,7 +326,7 @@ In the following example, the `type_coercion` and `simplify_expressions` passes | physical_plan after repartition | SAME TEXT AS ABOVE | | physical_plan after add_merge_exec | SAME TEXT AS ABOVE | | physical_plan | ProjectionExec: expr=[3.2 as foo] | -| | EmptyExec: produce_one_row=true | +| | PlaceholderRowExec | | | | +------------------------------------------------------------+---------------------------------------------------------------------------+ ``` diff --git a/datafusion/physical-plan/src/display.rs b/datafusion/physical-plan/src/display.rs index aa368251ebf3..612e164be0e2 100644 --- a/datafusion/physical-plan/src/display.rs +++ b/datafusion/physical-plan/src/display.rs @@ -132,7 +132,7 @@ impl<'a> DisplayableExecutionPlan<'a> { /// ```dot /// strict digraph dot_plan { // 0[label="ProjectionExec: expr=[id@0 + 2 as employee.id + Int32(2)]",tooltip=""] - // 1[label="EmptyExec: produce_one_row=false",tooltip=""] + // 1[label="EmptyExec",tooltip=""] // 0 -> 1 // } /// ``` diff --git a/datafusion/physical-plan/src/empty.rs b/datafusion/physical-plan/src/empty.rs index a3e1fb79edb5..41c8dbed1453 100644 --- a/datafusion/physical-plan/src/empty.rs +++ b/datafusion/physical-plan/src/empty.rs @@ -15,7 +15,7 @@ // specific language governing permissions and limitations // under the License. -//! EmptyRelation execution plan +//! EmptyRelation with produce_one_row=false execution plan use std::any::Any; use std::sync::Arc; @@ -24,19 +24,16 @@ use super::expressions::PhysicalSortExpr; use super::{common, DisplayAs, SendableRecordBatchStream, Statistics}; use crate::{memory::MemoryStream, DisplayFormatType, ExecutionPlan, Partitioning}; -use arrow::array::{ArrayRef, NullArray}; -use arrow::datatypes::{DataType, Field, Fields, Schema, SchemaRef}; +use arrow::datatypes::SchemaRef; use arrow::record_batch::RecordBatch; use datafusion_common::{internal_err, DataFusionError, Result}; use datafusion_execution::TaskContext; use log::trace; -/// Execution plan for empty relation (produces no rows) +/// Execution plan for empty relation with produce_one_row=false #[derive(Debug)] pub struct EmptyExec { - /// Specifies whether this exec produces a row or not - produce_one_row: bool, /// The schema for the produced row schema: SchemaRef, /// Number of partitions @@ -45,9 +42,8 @@ pub struct EmptyExec { impl EmptyExec { /// Create a new EmptyExec - pub fn new(produce_one_row: bool, schema: SchemaRef) -> Self { + pub fn new(schema: SchemaRef) -> Self { EmptyExec { - produce_one_row, schema, partitions: 1, } @@ -59,36 +55,8 @@ impl EmptyExec { self } - /// Specifies whether this exec produces a row or not - pub fn produce_one_row(&self) -> bool { - self.produce_one_row - } - fn data(&self) -> Result> { - let batch = if self.produce_one_row { - let n_field = self.schema.fields.len(); - // hack for https://github.com/apache/arrow-datafusion/pull/3242 - let n_field = if n_field == 0 { 1 } else { n_field }; - vec![RecordBatch::try_new( - Arc::new(Schema::new( - (0..n_field) - .map(|i| { - Field::new(format!("placeholder_{i}"), DataType::Null, true) - }) - .collect::(), - )), - (0..n_field) - .map(|_i| { - let ret: ArrayRef = Arc::new(NullArray::new(1)); - ret - }) - .collect(), - )?] - } else { - vec![] - }; - - Ok(batch) + Ok(vec![]) } } @@ -100,7 +68,7 @@ impl DisplayAs for EmptyExec { ) -> std::fmt::Result { match t { DisplayFormatType::Default | DisplayFormatType::Verbose => { - write!(f, "EmptyExec: produce_one_row={}", self.produce_one_row) + write!(f, "EmptyExec") } } } @@ -133,10 +101,7 @@ impl ExecutionPlan for EmptyExec { self: Arc, _: Vec>, ) -> Result> { - Ok(Arc::new(EmptyExec::new( - self.produce_one_row, - self.schema.clone(), - ))) + Ok(Arc::new(EmptyExec::new(self.schema.clone()))) } fn execute( @@ -184,7 +149,7 @@ mod tests { let task_ctx = Arc::new(TaskContext::default()); let schema = test::aggr_test_schema(); - let empty = EmptyExec::new(false, schema.clone()); + let empty = EmptyExec::new(schema.clone()); assert_eq!(empty.schema(), schema); // we should have no results @@ -198,16 +163,11 @@ mod tests { #[test] fn with_new_children() -> Result<()> { let schema = test::aggr_test_schema(); - let empty = Arc::new(EmptyExec::new(false, schema.clone())); - let empty_with_row = Arc::new(EmptyExec::new(true, schema)); + let empty = Arc::new(EmptyExec::new(schema.clone())); let empty2 = with_new_children_if_necessary(empty.clone(), vec![])?.into(); assert_eq!(empty.schema(), empty2.schema()); - let empty_with_row_2 = - with_new_children_if_necessary(empty_with_row.clone(), vec![])?.into(); - assert_eq!(empty_with_row.schema(), empty_with_row_2.schema()); - let too_many_kids = vec![empty2]; assert!( with_new_children_if_necessary(empty, too_many_kids).is_err(), @@ -220,44 +180,11 @@ mod tests { async fn invalid_execute() -> Result<()> { let task_ctx = Arc::new(TaskContext::default()); let schema = test::aggr_test_schema(); - let empty = EmptyExec::new(false, schema); + let empty = EmptyExec::new(schema); // ask for the wrong partition assert!(empty.execute(1, task_ctx.clone()).is_err()); assert!(empty.execute(20, task_ctx).is_err()); Ok(()) } - - #[tokio::test] - async fn produce_one_row() -> Result<()> { - let task_ctx = Arc::new(TaskContext::default()); - let schema = test::aggr_test_schema(); - let empty = EmptyExec::new(true, schema); - - let iter = empty.execute(0, task_ctx)?; - let batches = common::collect(iter).await?; - - // should have one item - assert_eq!(batches.len(), 1); - - Ok(()) - } - - #[tokio::test] - async fn produce_one_row_multiple_partition() -> Result<()> { - let task_ctx = Arc::new(TaskContext::default()); - let schema = test::aggr_test_schema(); - let partitions = 3; - let empty = EmptyExec::new(true, schema).with_partitions(partitions); - - for n in 0..partitions { - let iter = empty.execute(n, task_ctx.clone())?; - let batches = common::collect(iter).await?; - - // should have one item - assert_eq!(batches.len(), 1); - } - - Ok(()) - } } diff --git a/datafusion/physical-plan/src/lib.rs b/datafusion/physical-plan/src/lib.rs index f40911c10168..6c9e97e03cb7 100644 --- a/datafusion/physical-plan/src/lib.rs +++ b/datafusion/physical-plan/src/lib.rs @@ -59,6 +59,7 @@ pub mod limit; pub mod memory; pub mod metrics; mod ordering; +pub mod placeholder_row; pub mod projection; pub mod repartition; pub mod sorts; diff --git a/datafusion/physical-plan/src/placeholder_row.rs b/datafusion/physical-plan/src/placeholder_row.rs new file mode 100644 index 000000000000..94f32788530b --- /dev/null +++ b/datafusion/physical-plan/src/placeholder_row.rs @@ -0,0 +1,229 @@ +// 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. + +//! EmptyRelation produce_one_row=true execution plan + +use std::any::Any; +use std::sync::Arc; + +use super::expressions::PhysicalSortExpr; +use super::{common, DisplayAs, SendableRecordBatchStream, Statistics}; +use crate::{memory::MemoryStream, DisplayFormatType, ExecutionPlan, Partitioning}; + +use arrow::array::{ArrayRef, NullArray}; +use arrow::datatypes::{DataType, Field, Fields, Schema, SchemaRef}; +use arrow::record_batch::RecordBatch; +use datafusion_common::{internal_err, DataFusionError, Result}; +use datafusion_execution::TaskContext; + +use log::trace; + +/// Execution plan for empty relation with produce_one_row=true +#[derive(Debug)] +pub struct PlaceholderRowExec { + /// The schema for the produced row + schema: SchemaRef, + /// Number of partitions + partitions: usize, +} + +impl PlaceholderRowExec { + /// Create a new PlaceholderRowExec + pub fn new(schema: SchemaRef) -> Self { + PlaceholderRowExec { + schema, + partitions: 1, + } + } + + /// Create a new PlaceholderRowExecPlaceholderRowExec with specified partition number + pub fn with_partitions(mut self, partitions: usize) -> Self { + self.partitions = partitions; + self + } + + fn data(&self) -> Result> { + Ok({ + let n_field = self.schema.fields.len(); + // hack for https://github.com/apache/arrow-datafusion/pull/3242 + let n_field = if n_field == 0 { 1 } else { n_field }; + vec![RecordBatch::try_new( + Arc::new(Schema::new( + (0..n_field) + .map(|i| { + Field::new(format!("placeholder_{i}"), DataType::Null, true) + }) + .collect::(), + )), + (0..n_field) + .map(|_i| { + let ret: ArrayRef = Arc::new(NullArray::new(1)); + ret + }) + .collect(), + )?] + }) + } +} + +impl DisplayAs for PlaceholderRowExec { + fn fmt_as( + &self, + t: DisplayFormatType, + f: &mut std::fmt::Formatter, + ) -> std::fmt::Result { + match t { + DisplayFormatType::Default | DisplayFormatType::Verbose => { + write!(f, "PlaceholderRowExec") + } + } + } +} + +impl ExecutionPlan for PlaceholderRowExec { + /// Return a reference to Any that can be used for downcasting + fn as_any(&self) -> &dyn Any { + self + } + + fn schema(&self) -> SchemaRef { + self.schema.clone() + } + + fn children(&self) -> Vec> { + vec![] + } + + /// Get the output partitioning of this plan + fn output_partitioning(&self) -> Partitioning { + Partitioning::UnknownPartitioning(self.partitions) + } + + fn output_ordering(&self) -> Option<&[PhysicalSortExpr]> { + None + } + + fn with_new_children( + self: Arc, + _: Vec>, + ) -> Result> { + Ok(Arc::new(PlaceholderRowExec::new(self.schema.clone()))) + } + + fn execute( + &self, + partition: usize, + context: Arc, + ) -> Result { + trace!("Start PlaceholderRowExec::execute for partition {} of context session_id {} and task_id {:?}", partition, context.session_id(), context.task_id()); + + if partition >= self.partitions { + return internal_err!( + "PlaceholderRowExec invalid partition {} (expected less than {})", + partition, + self.partitions + ); + } + + Ok(Box::pin(MemoryStream::try_new( + self.data()?, + self.schema.clone(), + None, + )?)) + } + + fn statistics(&self) -> Result { + let batch = self + .data() + .expect("Create single row placeholder RecordBatch should not fail"); + Ok(common::compute_record_batch_statistics( + &[batch], + &self.schema, + None, + )) + } +} + +#[cfg(test)] +mod tests { + use super::*; + use crate::with_new_children_if_necessary; + use crate::{common, test}; + + #[test] + fn with_new_children() -> Result<()> { + let schema = test::aggr_test_schema(); + + let placeholder = Arc::new(PlaceholderRowExec::new(schema)); + + let placeholder_2 = + with_new_children_if_necessary(placeholder.clone(), vec![])?.into(); + assert_eq!(placeholder.schema(), placeholder_2.schema()); + + let too_many_kids = vec![placeholder_2]; + assert!( + with_new_children_if_necessary(placeholder, too_many_kids).is_err(), + "expected error when providing list of kids" + ); + Ok(()) + } + + #[tokio::test] + async fn invalid_execute() -> Result<()> { + let task_ctx = Arc::new(TaskContext::default()); + let schema = test::aggr_test_schema(); + let placeholder = PlaceholderRowExec::new(schema); + + // ask for the wrong partition + assert!(placeholder.execute(1, task_ctx.clone()).is_err()); + assert!(placeholder.execute(20, task_ctx).is_err()); + Ok(()) + } + + #[tokio::test] + async fn produce_one_row() -> Result<()> { + let task_ctx = Arc::new(TaskContext::default()); + let schema = test::aggr_test_schema(); + let placeholder = PlaceholderRowExec::new(schema); + + let iter = placeholder.execute(0, task_ctx)?; + let batches = common::collect(iter).await?; + + // should have one item + assert_eq!(batches.len(), 1); + + Ok(()) + } + + #[tokio::test] + async fn produce_one_row_multiple_partition() -> Result<()> { + let task_ctx = Arc::new(TaskContext::default()); + let schema = test::aggr_test_schema(); + let partitions = 3; + let placeholder = PlaceholderRowExec::new(schema).with_partitions(partitions); + + for n in 0..partitions { + let iter = placeholder.execute(n, task_ctx.clone())?; + let batches = common::collect(iter).await?; + + // should have one item + assert_eq!(batches.len(), 1); + } + + Ok(()) + } +} diff --git a/datafusion/proto/proto/datafusion.proto b/datafusion/proto/proto/datafusion.proto index 13a54f2a5659..f391592dfe76 100644 --- a/datafusion/proto/proto/datafusion.proto +++ b/datafusion/proto/proto/datafusion.proto @@ -1165,6 +1165,7 @@ message PhysicalPlanNode { JsonSinkExecNode json_sink = 24; SymmetricHashJoinExecNode symmetric_hash_join = 25; InterleaveExecNode interleave = 26; + PlaceholderRowExecNode placeholder_row = 27; } } @@ -1495,8 +1496,11 @@ message JoinOn { } message EmptyExecNode { - bool produce_one_row = 1; - Schema schema = 2; + Schema schema = 1; +} + +message PlaceholderRowExecNode { + Schema schema = 1; } message ProjectionExecNode { diff --git a/datafusion/proto/src/generated/pbjson.rs b/datafusion/proto/src/generated/pbjson.rs index 0d013c72d37f..d506b5dcce53 100644 --- a/datafusion/proto/src/generated/pbjson.rs +++ b/datafusion/proto/src/generated/pbjson.rs @@ -6389,16 +6389,10 @@ impl serde::Serialize for EmptyExecNode { { use serde::ser::SerializeStruct; let mut len = 0; - if self.produce_one_row { - len += 1; - } if self.schema.is_some() { len += 1; } let mut struct_ser = serializer.serialize_struct("datafusion.EmptyExecNode", len)?; - if self.produce_one_row { - struct_ser.serialize_field("produceOneRow", &self.produce_one_row)?; - } if let Some(v) = self.schema.as_ref() { struct_ser.serialize_field("schema", v)?; } @@ -6412,14 +6406,11 @@ impl<'de> serde::Deserialize<'de> for EmptyExecNode { D: serde::Deserializer<'de>, { const FIELDS: &[&str] = &[ - "produce_one_row", - "produceOneRow", "schema", ]; #[allow(clippy::enum_variant_names)] enum GeneratedField { - ProduceOneRow, Schema, } impl<'de> serde::Deserialize<'de> for GeneratedField { @@ -6442,7 +6433,6 @@ impl<'de> serde::Deserialize<'de> for EmptyExecNode { E: serde::de::Error, { match value { - "produceOneRow" | "produce_one_row" => Ok(GeneratedField::ProduceOneRow), "schema" => Ok(GeneratedField::Schema), _ => Err(serde::de::Error::unknown_field(value, FIELDS)), } @@ -6463,16 +6453,9 @@ impl<'de> serde::Deserialize<'de> for EmptyExecNode { where V: serde::de::MapAccess<'de>, { - let mut produce_one_row__ = None; let mut schema__ = None; while let Some(k) = map_.next_key()? { match k { - GeneratedField::ProduceOneRow => { - if produce_one_row__.is_some() { - return Err(serde::de::Error::duplicate_field("produceOneRow")); - } - produce_one_row__ = Some(map_.next_value()?); - } GeneratedField::Schema => { if schema__.is_some() { return Err(serde::de::Error::duplicate_field("schema")); @@ -6482,7 +6465,6 @@ impl<'de> serde::Deserialize<'de> for EmptyExecNode { } } Ok(EmptyExecNode { - produce_one_row: produce_one_row__.unwrap_or_default(), schema: schema__, }) } @@ -18020,6 +18002,9 @@ impl serde::Serialize for PhysicalPlanNode { physical_plan_node::PhysicalPlanType::Interleave(v) => { struct_ser.serialize_field("interleave", v)?; } + physical_plan_node::PhysicalPlanType::PlaceholderRow(v) => { + struct_ser.serialize_field("placeholderRow", v)?; + } } } struct_ser.end() @@ -18069,6 +18054,8 @@ impl<'de> serde::Deserialize<'de> for PhysicalPlanNode { "symmetric_hash_join", "symmetricHashJoin", "interleave", + "placeholder_row", + "placeholderRow", ]; #[allow(clippy::enum_variant_names)] @@ -18098,6 +18085,7 @@ impl<'de> serde::Deserialize<'de> for PhysicalPlanNode { JsonSink, SymmetricHashJoin, Interleave, + PlaceholderRow, } impl<'de> serde::Deserialize<'de> for GeneratedField { fn deserialize(deserializer: D) -> std::result::Result @@ -18144,6 +18132,7 @@ impl<'de> serde::Deserialize<'de> for PhysicalPlanNode { "jsonSink" | "json_sink" => Ok(GeneratedField::JsonSink), "symmetricHashJoin" | "symmetric_hash_join" => Ok(GeneratedField::SymmetricHashJoin), "interleave" => Ok(GeneratedField::Interleave), + "placeholderRow" | "placeholder_row" => Ok(GeneratedField::PlaceholderRow), _ => Err(serde::de::Error::unknown_field(value, FIELDS)), } } @@ -18339,6 +18328,13 @@ impl<'de> serde::Deserialize<'de> for PhysicalPlanNode { return Err(serde::de::Error::duplicate_field("interleave")); } physical_plan_type__ = map_.next_value::<::std::option::Option<_>>()?.map(physical_plan_node::PhysicalPlanType::Interleave) +; + } + GeneratedField::PlaceholderRow => { + if physical_plan_type__.is_some() { + return Err(serde::de::Error::duplicate_field("placeholderRow")); + } + physical_plan_type__ = map_.next_value::<::std::option::Option<_>>()?.map(physical_plan_node::PhysicalPlanType::PlaceholderRow) ; } } @@ -19369,6 +19365,97 @@ impl<'de> serde::Deserialize<'de> for PlaceholderNode { deserializer.deserialize_struct("datafusion.PlaceholderNode", FIELDS, GeneratedVisitor) } } +impl serde::Serialize for PlaceholderRowExecNode { + #[allow(deprecated)] + fn serialize(&self, serializer: S) -> std::result::Result + where + S: serde::Serializer, + { + use serde::ser::SerializeStruct; + let mut len = 0; + if self.schema.is_some() { + len += 1; + } + let mut struct_ser = serializer.serialize_struct("datafusion.PlaceholderRowExecNode", len)?; + if let Some(v) = self.schema.as_ref() { + struct_ser.serialize_field("schema", v)?; + } + struct_ser.end() + } +} +impl<'de> serde::Deserialize<'de> for PlaceholderRowExecNode { + #[allow(deprecated)] + fn deserialize(deserializer: D) -> std::result::Result + where + D: serde::Deserializer<'de>, + { + const FIELDS: &[&str] = &[ + "schema", + ]; + + #[allow(clippy::enum_variant_names)] + enum GeneratedField { + Schema, + } + impl<'de> serde::Deserialize<'de> for GeneratedField { + fn deserialize(deserializer: D) -> std::result::Result + where + D: serde::Deserializer<'de>, + { + struct GeneratedVisitor; + + impl<'de> serde::de::Visitor<'de> for GeneratedVisitor { + type Value = GeneratedField; + + fn expecting(&self, formatter: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + write!(formatter, "expected one of: {:?}", &FIELDS) + } + + #[allow(unused_variables)] + fn visit_str(self, value: &str) -> std::result::Result + where + E: serde::de::Error, + { + match value { + "schema" => Ok(GeneratedField::Schema), + _ => Err(serde::de::Error::unknown_field(value, FIELDS)), + } + } + } + deserializer.deserialize_identifier(GeneratedVisitor) + } + } + struct GeneratedVisitor; + impl<'de> serde::de::Visitor<'de> for GeneratedVisitor { + type Value = PlaceholderRowExecNode; + + fn expecting(&self, formatter: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + formatter.write_str("struct datafusion.PlaceholderRowExecNode") + } + + fn visit_map(self, mut map_: V) -> std::result::Result + where + V: serde::de::MapAccess<'de>, + { + let mut schema__ = None; + while let Some(k) = map_.next_key()? { + match k { + GeneratedField::Schema => { + if schema__.is_some() { + return Err(serde::de::Error::duplicate_field("schema")); + } + schema__ = map_.next_value()?; + } + } + } + Ok(PlaceholderRowExecNode { + schema: schema__, + }) + } + } + deserializer.deserialize_struct("datafusion.PlaceholderRowExecNode", FIELDS, GeneratedVisitor) + } +} impl serde::Serialize for PlanType { #[allow(deprecated)] fn serialize(&self, serializer: S) -> std::result::Result diff --git a/datafusion/proto/src/generated/prost.rs b/datafusion/proto/src/generated/prost.rs index d4b62d4b3fd8..8aadc96349ca 100644 --- a/datafusion/proto/src/generated/prost.rs +++ b/datafusion/proto/src/generated/prost.rs @@ -1525,7 +1525,7 @@ pub mod owned_table_reference { pub struct PhysicalPlanNode { #[prost( oneof = "physical_plan_node::PhysicalPlanType", - tags = "1, 2, 3, 4, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26" + tags = "1, 2, 3, 4, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27" )] pub physical_plan_type: ::core::option::Option, } @@ -1586,6 +1586,8 @@ pub mod physical_plan_node { SymmetricHashJoin(::prost::alloc::boxed::Box), #[prost(message, tag = "26")] Interleave(super::InterleaveExecNode), + #[prost(message, tag = "27")] + PlaceholderRow(super::PlaceholderRowExecNode), } } #[allow(clippy::derive_partial_eq_without_eq)] @@ -2103,9 +2105,13 @@ pub struct JoinOn { #[allow(clippy::derive_partial_eq_without_eq)] #[derive(Clone, PartialEq, ::prost::Message)] pub struct EmptyExecNode { - #[prost(bool, tag = "1")] - pub produce_one_row: bool, - #[prost(message, optional, tag = "2")] + #[prost(message, optional, tag = "1")] + pub schema: ::core::option::Option, +} +#[allow(clippy::derive_partial_eq_without_eq)] +#[derive(Clone, PartialEq, ::prost::Message)] +pub struct PlaceholderRowExecNode { + #[prost(message, optional, tag = "1")] pub schema: ::core::option::Option, } #[allow(clippy::derive_partial_eq_without_eq)] diff --git a/datafusion/proto/src/physical_plan/mod.rs b/datafusion/proto/src/physical_plan/mod.rs index 878a5bcb7f69..73091a6fced9 100644 --- a/datafusion/proto/src/physical_plan/mod.rs +++ b/datafusion/proto/src/physical_plan/mod.rs @@ -44,6 +44,7 @@ use datafusion::physical_plan::joins::{ }; use datafusion::physical_plan::joins::{HashJoinExec, PartitionMode}; use datafusion::physical_plan::limit::{GlobalLimitExec, LocalLimitExec}; +use datafusion::physical_plan::placeholder_row::PlaceholderRowExec; use datafusion::physical_plan::projection::ProjectionExec; use datafusion::physical_plan::repartition::RepartitionExec; use datafusion::physical_plan::sorts::sort::SortExec; @@ -721,7 +722,11 @@ impl AsExecutionPlan for PhysicalPlanNode { } PhysicalPlanType::Empty(empty) => { let schema = Arc::new(convert_required!(empty.schema)?); - Ok(Arc::new(EmptyExec::new(empty.produce_one_row, schema))) + Ok(Arc::new(EmptyExec::new(schema))) + } + PhysicalPlanType::PlaceholderRow(placeholder) => { + let schema = Arc::new(convert_required!(placeholder.schema)?); + Ok(Arc::new(PlaceholderRowExec::new(schema))) } PhysicalPlanType::Sort(sort) => { let input: Arc = @@ -1307,7 +1312,17 @@ impl AsExecutionPlan for PhysicalPlanNode { return Ok(protobuf::PhysicalPlanNode { physical_plan_type: Some(PhysicalPlanType::Empty( protobuf::EmptyExecNode { - produce_one_row: empty.produce_one_row(), + schema: Some(schema), + }, + )), + }); + } + + if let Some(empty) = plan.downcast_ref::() { + let schema = empty.schema().as_ref().try_into()?; + return Ok(protobuf::PhysicalPlanNode { + physical_plan_type: Some(PhysicalPlanType::PlaceholderRow( + protobuf::PlaceholderRowExecNode { schema: Some(schema), }, )), diff --git a/datafusion/proto/tests/cases/roundtrip_physical_plan.rs b/datafusion/proto/tests/cases/roundtrip_physical_plan.rs index f46a29447dd6..da76209dbb49 100644 --- a/datafusion/proto/tests/cases/roundtrip_physical_plan.rs +++ b/datafusion/proto/tests/cases/roundtrip_physical_plan.rs @@ -49,6 +49,7 @@ use datafusion::physical_plan::joins::{ HashJoinExec, NestedLoopJoinExec, PartitionMode, StreamJoinPartitionMode, }; use datafusion::physical_plan::limit::{GlobalLimitExec, LocalLimitExec}; +use datafusion::physical_plan::placeholder_row::PlaceholderRowExec; use datafusion::physical_plan::projection::ProjectionExec; use datafusion::physical_plan::repartition::RepartitionExec; use datafusion::physical_plan::sorts::sort::SortExec; @@ -104,7 +105,7 @@ fn roundtrip_test_with_context( #[test] fn roundtrip_empty() -> Result<()> { - roundtrip_test(Arc::new(EmptyExec::new(false, Arc::new(Schema::empty())))) + roundtrip_test(Arc::new(EmptyExec::new(Arc::new(Schema::empty())))) } #[test] @@ -117,7 +118,7 @@ fn roundtrip_date_time_interval() -> Result<()> { false, ), ]); - let input = Arc::new(EmptyExec::new(false, Arc::new(schema.clone()))); + let input = Arc::new(EmptyExec::new(Arc::new(schema.clone()))); let date_expr = col("some_date", &schema)?; let literal_expr = col("some_interval", &schema)?; let date_time_interval_expr = @@ -132,7 +133,7 @@ fn roundtrip_date_time_interval() -> Result<()> { #[test] fn roundtrip_local_limit() -> Result<()> { roundtrip_test(Arc::new(LocalLimitExec::new( - Arc::new(EmptyExec::new(false, Arc::new(Schema::empty()))), + Arc::new(EmptyExec::new(Arc::new(Schema::empty()))), 25, ))) } @@ -140,7 +141,7 @@ fn roundtrip_local_limit() -> Result<()> { #[test] fn roundtrip_global_limit() -> Result<()> { roundtrip_test(Arc::new(GlobalLimitExec::new( - Arc::new(EmptyExec::new(false, Arc::new(Schema::empty()))), + Arc::new(EmptyExec::new(Arc::new(Schema::empty()))), 0, Some(25), ))) @@ -149,7 +150,7 @@ fn roundtrip_global_limit() -> Result<()> { #[test] fn roundtrip_global_skip_no_limit() -> Result<()> { roundtrip_test(Arc::new(GlobalLimitExec::new( - Arc::new(EmptyExec::new(false, Arc::new(Schema::empty()))), + Arc::new(EmptyExec::new(Arc::new(Schema::empty()))), 10, None, // no limit ))) @@ -179,8 +180,8 @@ fn roundtrip_hash_join() -> Result<()> { ] { for partition_mode in &[PartitionMode::Partitioned, PartitionMode::CollectLeft] { roundtrip_test(Arc::new(HashJoinExec::try_new( - Arc::new(EmptyExec::new(false, schema_left.clone())), - Arc::new(EmptyExec::new(false, schema_right.clone())), + Arc::new(EmptyExec::new(schema_left.clone())), + Arc::new(EmptyExec::new(schema_right.clone())), on.clone(), None, join_type, @@ -211,8 +212,8 @@ fn roundtrip_nested_loop_join() -> Result<()> { JoinType::RightSemi, ] { roundtrip_test(Arc::new(NestedLoopJoinExec::try_new( - Arc::new(EmptyExec::new(false, schema_left.clone())), - Arc::new(EmptyExec::new(false, schema_right.clone())), + Arc::new(EmptyExec::new(schema_left.clone())), + Arc::new(EmptyExec::new(schema_right.clone())), None, join_type, )?))?; @@ -277,7 +278,7 @@ fn roundtrip_window() -> Result<()> { Arc::new(window_frame), )); - let input = Arc::new(EmptyExec::new(false, schema.clone())); + let input = Arc::new(EmptyExec::new(schema.clone())); roundtrip_test(Arc::new(WindowAggExec::try_new( vec![ @@ -311,7 +312,7 @@ fn rountrip_aggregate() -> Result<()> { aggregates.clone(), vec![None], vec![None], - Arc::new(EmptyExec::new(false, schema.clone())), + Arc::new(EmptyExec::new(schema.clone())), schema, )?)) } @@ -379,7 +380,7 @@ fn roundtrip_aggregate_udaf() -> Result<()> { aggregates.clone(), vec![None], vec![None], - Arc::new(EmptyExec::new(false, schema.clone())), + Arc::new(EmptyExec::new(schema.clone())), schema, )?), ctx, @@ -405,7 +406,7 @@ fn roundtrip_filter_with_not_and_in_list() -> Result<()> { let and = binary(not, Operator::And, in_list, &schema)?; roundtrip_test(Arc::new(FilterExec::try_new( and, - Arc::new(EmptyExec::new(false, schema.clone())), + Arc::new(EmptyExec::new(schema.clone())), )?)) } @@ -432,7 +433,7 @@ fn roundtrip_sort() -> Result<()> { ]; roundtrip_test(Arc::new(SortExec::new( sort_exprs, - Arc::new(EmptyExec::new(false, schema)), + Arc::new(EmptyExec::new(schema)), ))) } @@ -460,11 +461,11 @@ fn roundtrip_sort_preserve_partitioning() -> Result<()> { roundtrip_test(Arc::new(SortExec::new( sort_exprs.clone(), - Arc::new(EmptyExec::new(false, schema.clone())), + Arc::new(EmptyExec::new(schema.clone())), )))?; roundtrip_test(Arc::new( - SortExec::new(sort_exprs, Arc::new(EmptyExec::new(false, schema))) + SortExec::new(sort_exprs, Arc::new(EmptyExec::new(schema))) .with_preserve_partitioning(true), )) } @@ -514,7 +515,7 @@ fn roundtrip_builtin_scalar_function() -> Result<()> { let field_b = Field::new("b", DataType::Int64, false); let schema = Arc::new(Schema::new(vec![field_a, field_b])); - let input = Arc::new(EmptyExec::new(false, schema.clone())); + let input = Arc::new(EmptyExec::new(schema.clone())); let execution_props = ExecutionProps::new(); @@ -541,7 +542,7 @@ fn roundtrip_scalar_udf() -> Result<()> { let field_b = Field::new("b", DataType::Int64, false); let schema = Arc::new(Schema::new(vec![field_a, field_b])); - let input = Arc::new(EmptyExec::new(false, schema.clone())); + let input = Arc::new(EmptyExec::new(schema.clone())); let fn_impl = |args: &[ArrayRef]| Ok(Arc::new(args[0].clone()) as ArrayRef); @@ -594,7 +595,7 @@ fn roundtrip_distinct_count() -> Result<()> { aggregates.clone(), vec![None], vec![None], - Arc::new(EmptyExec::new(false, schema.clone())), + Arc::new(EmptyExec::new(schema.clone())), schema, )?)) } @@ -605,7 +606,7 @@ fn roundtrip_like() -> Result<()> { Field::new("a", DataType::Utf8, false), Field::new("b", DataType::Utf8, false), ]); - let input = Arc::new(EmptyExec::new(false, Arc::new(schema.clone()))); + let input = Arc::new(EmptyExec::new(Arc::new(schema.clone()))); let like_expr = like( false, false, @@ -632,7 +633,7 @@ fn roundtrip_get_indexed_field_named_struct_field() -> Result<()> { ]; let schema = Schema::new(fields); - let input = Arc::new(EmptyExec::new(false, Arc::new(schema.clone()))); + let input = Arc::new(EmptyExec::new(Arc::new(schema.clone()))); let col_arg = col("arg", &schema)?; let get_indexed_field_expr = Arc::new(GetIndexedFieldExpr::new( @@ -659,7 +660,7 @@ fn roundtrip_get_indexed_field_list_index() -> Result<()> { ]; let schema = Schema::new(fields); - let input = Arc::new(EmptyExec::new(true, Arc::new(schema.clone()))); + let input = Arc::new(PlaceholderRowExec::new(Arc::new(schema.clone()))); let col_arg = col("arg", &schema)?; let col_key = col("key", &schema)?; @@ -686,7 +687,7 @@ fn roundtrip_get_indexed_field_list_range() -> Result<()> { ]; let schema = Schema::new(fields); - let input = Arc::new(EmptyExec::new(false, Arc::new(schema.clone()))); + let input = Arc::new(EmptyExec::new(Arc::new(schema.clone()))); let col_arg = col("arg", &schema)?; let col_start = col("start", &schema)?; @@ -712,7 +713,7 @@ fn roundtrip_analyze() -> Result<()> { let field_a = Field::new("plan_type", DataType::Utf8, false); let field_b = Field::new("plan", DataType::Utf8, false); let schema = Schema::new(vec![field_a, field_b]); - let input = Arc::new(EmptyExec::new(true, Arc::new(schema.clone()))); + let input = Arc::new(PlaceholderRowExec::new(Arc::new(schema.clone()))); roundtrip_test(Arc::new(AnalyzeExec::new( false, @@ -727,7 +728,7 @@ fn roundtrip_json_sink() -> Result<()> { let field_a = Field::new("plan_type", DataType::Utf8, false); let field_b = Field::new("plan", DataType::Utf8, false); let schema = Arc::new(Schema::new(vec![field_a, field_b])); - let input = Arc::new(EmptyExec::new(true, schema.clone())); + let input = Arc::new(PlaceholderRowExec::new(schema.clone())); let file_sink_config = FileSinkConfig { object_store_url: ObjectStoreUrl::local_filesystem(), @@ -787,8 +788,8 @@ fn roundtrip_sym_hash_join() -> Result<()> { ] { roundtrip_test(Arc::new( datafusion::physical_plan::joins::SymmetricHashJoinExec::try_new( - Arc::new(EmptyExec::new(false, schema_left.clone())), - Arc::new(EmptyExec::new(false, schema_right.clone())), + Arc::new(EmptyExec::new(schema_left.clone())), + Arc::new(EmptyExec::new(schema_right.clone())), on.clone(), None, join_type, @@ -806,8 +807,8 @@ fn roundtrip_union() -> Result<()> { let field_a = Field::new("col", DataType::Int64, false); let schema_left = Schema::new(vec![field_a.clone()]); let schema_right = Schema::new(vec![field_a]); - let left = EmptyExec::new(false, Arc::new(schema_left)); - let right = EmptyExec::new(false, Arc::new(schema_right)); + let left = EmptyExec::new(Arc::new(schema_left)); + let right = EmptyExec::new(Arc::new(schema_right)); let inputs: Vec> = vec![Arc::new(left), Arc::new(right)]; let union = UnionExec::new(inputs); roundtrip_test(Arc::new(union)) @@ -820,11 +821,11 @@ fn roundtrip_interleave() -> Result<()> { let schema_right = Schema::new(vec![field_a]); let partition = Partitioning::Hash(vec![], 3); let left = RepartitionExec::try_new( - Arc::new(EmptyExec::new(false, Arc::new(schema_left))), + Arc::new(EmptyExec::new(Arc::new(schema_left))), partition.clone(), )?; let right = RepartitionExec::try_new( - Arc::new(EmptyExec::new(false, Arc::new(schema_right))), + Arc::new(EmptyExec::new(Arc::new(schema_right))), partition.clone(), )?; let inputs: Vec> = vec![Arc::new(left), Arc::new(right)]; diff --git a/datafusion/sqllogictest/test_files/explain.slt b/datafusion/sqllogictest/test_files/explain.slt index 18792735ffed..4583ef319b7f 100644 --- a/datafusion/sqllogictest/test_files/explain.slt +++ b/datafusion/sqllogictest/test_files/explain.slt @@ -94,7 +94,7 @@ EXPLAIN select count(*) from (values ('a', 1, 100), ('a', 2, 150)) as t (c1,c2,c ---- physical_plan ProjectionExec: expr=[2 as COUNT(*)] ---EmptyExec: produce_one_row=true +--PlaceholderRowExec statement ok set datafusion.explain.physical_plan_only = false @@ -368,7 +368,7 @@ Projection: List([[1, 2, 3], [4, 5, 6]]) AS make_array(make_array(Int64(1),Int64 --EmptyRelation physical_plan ProjectionExec: expr=[[[1, 2, 3], [4, 5, 6]] as make_array(make_array(Int64(1),Int64(2),Int64(3)),make_array(Int64(4),Int64(5),Int64(6)))] ---EmptyExec: produce_one_row=true +--PlaceholderRowExec query TT explain select [[1, 2, 3], [4, 5, 6]]; @@ -378,4 +378,4 @@ Projection: List([[1, 2, 3], [4, 5, 6]]) AS make_array(make_array(Int64(1),Int64 --EmptyRelation physical_plan ProjectionExec: expr=[[[1, 2, 3], [4, 5, 6]] as make_array(make_array(Int64(1),Int64(2),Int64(3)),make_array(Int64(4),Int64(5),Int64(6)))] ---EmptyExec: produce_one_row=true +--PlaceholderRowExec diff --git a/datafusion/sqllogictest/test_files/join.slt b/datafusion/sqllogictest/test_files/join.slt index 874d849e9a29..386ffe766b19 100644 --- a/datafusion/sqllogictest/test_files/join.slt +++ b/datafusion/sqllogictest/test_files/join.slt @@ -556,7 +556,7 @@ query TT explain select * from t1 join t2 on false; ---- logical_plan EmptyRelation -physical_plan EmptyExec: produce_one_row=false +physical_plan EmptyExec # Make batch size smaller than table row number. to introduce parallelism to the plan. statement ok diff --git a/datafusion/sqllogictest/test_files/limit.slt b/datafusion/sqllogictest/test_files/limit.slt index 182195112e87..e063d6e8960a 100644 --- a/datafusion/sqllogictest/test_files/limit.slt +++ b/datafusion/sqllogictest/test_files/limit.slt @@ -312,7 +312,7 @@ Aggregate: groupBy=[[]], aggr=[[COUNT(UInt8(1)) AS COUNT(*)]] ----TableScan: t1 projection=[], fetch=14 physical_plan ProjectionExec: expr=[0 as COUNT(*)] ---EmptyExec: produce_one_row=true +--PlaceholderRowExec query I SELECT COUNT(*) FROM (SELECT a FROM t1 LIMIT 3 OFFSET 11); @@ -330,7 +330,7 @@ Aggregate: groupBy=[[]], aggr=[[COUNT(UInt8(1)) AS COUNT(*)]] ----TableScan: t1 projection=[], fetch=11 physical_plan ProjectionExec: expr=[2 as COUNT(*)] ---EmptyExec: produce_one_row=true +--PlaceholderRowExec query I SELECT COUNT(*) FROM (SELECT a FROM t1 LIMIT 3 OFFSET 8); @@ -348,7 +348,7 @@ Aggregate: groupBy=[[]], aggr=[[COUNT(UInt8(1)) AS COUNT(*)]] ----TableScan: t1 projection=[] physical_plan ProjectionExec: expr=[2 as COUNT(*)] ---EmptyExec: produce_one_row=true +--PlaceholderRowExec query I SELECT COUNT(*) FROM (SELECT a FROM t1 LIMIT 3 OFFSET 8); diff --git a/datafusion/sqllogictest/test_files/union.slt b/datafusion/sqllogictest/test_files/union.slt index 2c8970a13927..b4e338875e24 100644 --- a/datafusion/sqllogictest/test_files/union.slt +++ b/datafusion/sqllogictest/test_files/union.slt @@ -551,11 +551,11 @@ UnionExec ------CoalesceBatchesExec: target_batch_size=2 --------RepartitionExec: partitioning=Hash([Int64(1)@0], 4), input_partitions=1 ----------AggregateExec: mode=Partial, gby=[1 as Int64(1)], aggr=[] -------------EmptyExec: produce_one_row=true +------------PlaceholderRowExec --ProjectionExec: expr=[2 as a] -----EmptyExec: produce_one_row=true +----PlaceholderRowExec --ProjectionExec: expr=[3 as a] -----EmptyExec: produce_one_row=true +----PlaceholderRowExec # test UNION ALL aliases correctly with aliased subquery query TT @@ -583,7 +583,7 @@ UnionExec --------RepartitionExec: partitioning=Hash([n@0], 4), input_partitions=1 ----------AggregateExec: mode=Partial, gby=[n@0 as n], aggr=[COUNT(*)] ------------ProjectionExec: expr=[5 as n] ---------------EmptyExec: produce_one_row=true +--------------PlaceholderRowExec --ProjectionExec: expr=[1 as count, MAX(Int64(10))@0 as n] ----AggregateExec: mode=Single, gby=[], aggr=[MAX(Int64(10))] -------EmptyExec: produce_one_row=true +------PlaceholderRowExec diff --git a/datafusion/sqllogictest/test_files/window.slt b/datafusion/sqllogictest/test_files/window.slt index 7846bb001a91..f3de5b54fc8b 100644 --- a/datafusion/sqllogictest/test_files/window.slt +++ b/datafusion/sqllogictest/test_files/window.slt @@ -279,13 +279,13 @@ SortPreservingMergeExec: [b@0 ASC NULLS LAST] ------------AggregateExec: mode=Partial, gby=[b@1 as b], aggr=[MAX(d.a)] --------------UnionExec ----------------ProjectionExec: expr=[1 as a, aa as b] -------------------EmptyExec: produce_one_row=true +------------------PlaceholderRowExec ----------------ProjectionExec: expr=[3 as a, aa as b] -------------------EmptyExec: produce_one_row=true +------------------PlaceholderRowExec ----------------ProjectionExec: expr=[5 as a, bb as b] -------------------EmptyExec: produce_one_row=true +------------------PlaceholderRowExec ----------------ProjectionExec: expr=[7 as a, bb as b] -------------------EmptyExec: produce_one_row=true +------------------PlaceholderRowExec # Check actual result: query TI @@ -365,13 +365,13 @@ SortPreservingMergeExec: [b@0 ASC NULLS LAST] --------------RepartitionExec: partitioning=Hash([b@1], 4), input_partitions=4 ----------------UnionExec ------------------ProjectionExec: expr=[1 as a, aa as b] ---------------------EmptyExec: produce_one_row=true +--------------------PlaceholderRowExec ------------------ProjectionExec: expr=[3 as a, aa as b] ---------------------EmptyExec: produce_one_row=true +--------------------PlaceholderRowExec ------------------ProjectionExec: expr=[5 as a, bb as b] ---------------------EmptyExec: produce_one_row=true +--------------------PlaceholderRowExec ------------------ProjectionExec: expr=[7 as a, bb as b] ---------------------EmptyExec: produce_one_row=true +--------------------PlaceholderRowExec # check actual result From 93b21bdcd3d465ed78b610b54edf1418a47fc497 Mon Sep 17 00:00:00 2001 From: Dan Lovell Date: Mon, 11 Dec 2023 06:21:24 -0500 Subject: [PATCH 404/572] Enable non-uniform field type for structs created in DataFusion (#8463) * feat: struct: implement variadic_any solution, enable all struct field types * fix: run cargo-fmt * cln: remove unused imports --- datafusion/expr/src/built_in_function.rs | 8 ++--- .../physical-expr/src/struct_expressions.rs | 35 +++++-------------- datafusion/sqllogictest/test_files/struct.slt | 11 ++++++ 3 files changed, 22 insertions(+), 32 deletions(-) diff --git a/datafusion/expr/src/built_in_function.rs b/datafusion/expr/src/built_in_function.rs index 977b556b26cf..5a903a73adc6 100644 --- a/datafusion/expr/src/built_in_function.rs +++ b/datafusion/expr/src/built_in_function.rs @@ -28,8 +28,7 @@ use crate::signature::TIMEZONE_WILDCARD; use crate::type_coercion::binary::get_wider_type; use crate::type_coercion::functions::data_types; use crate::{ - conditional_expressions, struct_expressions, FuncMonotonicity, Signature, - TypeSignature, Volatility, + conditional_expressions, FuncMonotonicity, Signature, TypeSignature, Volatility, }; use arrow::datatypes::{DataType, Field, Fields, IntervalUnit, TimeUnit}; @@ -971,10 +970,7 @@ impl BuiltinScalarFunction { ], self.volatility(), ), - BuiltinScalarFunction::Struct => Signature::variadic( - struct_expressions::SUPPORTED_STRUCT_TYPES.to_vec(), - self.volatility(), - ), + BuiltinScalarFunction::Struct => Signature::variadic_any(self.volatility()), BuiltinScalarFunction::Concat | BuiltinScalarFunction::ConcatWithSeparator => { Signature::variadic(vec![Utf8], self.volatility()) diff --git a/datafusion/physical-expr/src/struct_expressions.rs b/datafusion/physical-expr/src/struct_expressions.rs index 0eed1d16fba8..b0ccb2a3ccb6 100644 --- a/datafusion/physical-expr/src/struct_expressions.rs +++ b/datafusion/physical-expr/src/struct_expressions.rs @@ -18,8 +18,8 @@ //! Struct expressions use arrow::array::*; -use arrow::datatypes::{DataType, Field}; -use datafusion_common::{exec_err, not_impl_err, DataFusionError, Result}; +use arrow::datatypes::Field; +use datafusion_common::{exec_err, DataFusionError, Result}; use datafusion_expr::ColumnarValue; use std::sync::Arc; @@ -34,31 +34,14 @@ fn array_struct(args: &[ArrayRef]) -> Result { .enumerate() .map(|(i, arg)| { let field_name = format!("c{i}"); - match arg.data_type() { - DataType::Utf8 - | DataType::LargeUtf8 - | DataType::Boolean - | DataType::Float32 - | DataType::Float64 - | DataType::Int8 - | DataType::Int16 - | DataType::Int32 - | DataType::Int64 - | DataType::UInt8 - | DataType::UInt16 - | DataType::UInt32 - | DataType::UInt64 => Ok(( - Arc::new(Field::new( - field_name.as_str(), - arg.data_type().clone(), - true, - )), - arg.clone(), + Ok(( + Arc::new(Field::new( + field_name.as_str(), + arg.data_type().clone(), + true, )), - data_type => { - not_impl_err!("Struct is not implemented for type '{data_type:?}'.") - } - } + arg.clone(), + )) }) .collect::>>()?; diff --git a/datafusion/sqllogictest/test_files/struct.slt b/datafusion/sqllogictest/test_files/struct.slt index fc14798a3bfe..936dedcc896e 100644 --- a/datafusion/sqllogictest/test_files/struct.slt +++ b/datafusion/sqllogictest/test_files/struct.slt @@ -58,5 +58,16 @@ select struct(a, b, c) from values; {c0: 2, c1: 2.2, c2: b} {c0: 3, c1: 3.3, c2: c} +# explain struct scalar function with columns #1 +query TT +explain select struct(a, b, c) from values; +---- +logical_plan +Projection: struct(values.a, values.b, values.c) +--TableScan: values projection=[a, b, c] +physical_plan +ProjectionExec: expr=[struct(a@0, b@1, c@2) as struct(values.a,values.b,values.c)] +--MemoryExec: partitions=1, partition_sizes=[1] + statement ok drop table values; From ff65dee3ff4318da13f5f89bafddf446ffbf8803 Mon Sep 17 00:00:00 2001 From: Jay Zhan Date: Mon, 11 Dec 2023 21:38:23 +0800 Subject: [PATCH 405/572] add multi ordering test case (#8439) Signed-off-by: jayzhan211 --- .../tests/data/aggregate_agg_multi_order.csv | 11 +++++ .../src/aggregate/array_agg_ordered.rs | 49 +++++++------------ .../sqllogictest/test_files/aggregate.slt | 30 ++++++++++++ 3 files changed, 60 insertions(+), 30 deletions(-) create mode 100644 datafusion/core/tests/data/aggregate_agg_multi_order.csv diff --git a/datafusion/core/tests/data/aggregate_agg_multi_order.csv b/datafusion/core/tests/data/aggregate_agg_multi_order.csv new file mode 100644 index 000000000000..e9a65ceee4aa --- /dev/null +++ b/datafusion/core/tests/data/aggregate_agg_multi_order.csv @@ -0,0 +1,11 @@ +c1,c2,c3 +1,20,0 +2,20,1 +3,10,2 +4,10,3 +5,30,4 +6,30,5 +7,30,6 +8,30,7 +9,30,8 +10,10,9 \ No newline at end of file diff --git a/datafusion/physical-expr/src/aggregate/array_agg_ordered.rs b/datafusion/physical-expr/src/aggregate/array_agg_ordered.rs index 9ca83a781a01..eb5ae8b0b0c3 100644 --- a/datafusion/physical-expr/src/aggregate/array_agg_ordered.rs +++ b/datafusion/physical-expr/src/aggregate/array_agg_ordered.rs @@ -30,9 +30,9 @@ use crate::{AggregateExpr, LexOrdering, PhysicalExpr, PhysicalSortExpr}; use arrow::array::ArrayRef; use arrow::datatypes::{DataType, Field}; +use arrow_array::cast::AsArray; use arrow_array::Array; use arrow_schema::{Fields, SortOptions}; -use datafusion_common::cast::as_list_array; use datafusion_common::utils::{compare_rows, get_row_at_idx}; use datafusion_common::{exec_err, DataFusionError, Result, ScalarValue}; use datafusion_expr::Accumulator; @@ -214,7 +214,7 @@ impl Accumulator for OrderSensitiveArrayAggAccumulator { // values received from its ordering requirement expression. (This information is necessary for during merging). let agg_orderings = &states[1]; - if as_list_array(agg_orderings).is_ok() { + if let Some(agg_orderings) = agg_orderings.as_list_opt::() { // Stores ARRAY_AGG results coming from each partition let mut partition_values = vec![]; // Stores ordering requirement expression results coming from each partition @@ -232,10 +232,21 @@ impl Accumulator for OrderSensitiveArrayAggAccumulator { } let orderings = ScalarValue::convert_array_to_scalar_vec(agg_orderings)?; - // Ordering requirement expression values for each entry in the ARRAY_AGG list - let other_ordering_values = self.convert_array_agg_to_orderings(orderings)?; - for v in other_ordering_values.into_iter() { - partition_ordering_values.push(v); + + for partition_ordering_rows in orderings.into_iter() { + // Extract value from struct to ordering_rows for each group/partition + let ordering_value = partition_ordering_rows.into_iter().map(|ordering_row| { + if let ScalarValue::Struct(Some(ordering_columns_per_row), _) = ordering_row { + Ok(ordering_columns_per_row) + } else { + exec_err!( + "Expects to receive ScalarValue::Struct(Some(..), _) but got:{:?}", + ordering_row.data_type() + ) + } + }).collect::>>()?; + + partition_ordering_values.push(ordering_value); } let sort_options = self @@ -293,33 +304,10 @@ impl Accumulator for OrderSensitiveArrayAggAccumulator { } impl OrderSensitiveArrayAggAccumulator { - /// Inner Vec\ in the ordering_values can be thought as ordering information for the each ScalarValue in the values array. - /// See [`merge_ordered_arrays`] for more information. - fn convert_array_agg_to_orderings( - &self, - array_agg: Vec>, - ) -> Result>>> { - let mut orderings = vec![]; - // in_data is Vec where ScalarValue does not include ScalarValue::List - for in_data in array_agg.into_iter() { - let ordering = in_data.into_iter().map(|struct_vals| { - if let ScalarValue::Struct(Some(orderings), _) = struct_vals { - Ok(orderings) - } else { - exec_err!( - "Expects to receive ScalarValue::Struct(Some(..), _) but got:{:?}", - struct_vals.data_type() - ) - } - }).collect::>>()?; - orderings.push(ordering); - } - Ok(orderings) - } - fn evaluate_orderings(&self) -> Result { let fields = ordering_fields(&self.ordering_req, &self.datatypes[1..]); let struct_field = Fields::from(fields.clone()); + let orderings: Vec = self .ordering_values .iter() @@ -329,6 +317,7 @@ impl OrderSensitiveArrayAggAccumulator { .collect(); let struct_type = DataType::Struct(Fields::from(fields)); + // Wrap in List, so we have the same data structure ListArray(StructArray..) for group by cases let arr = ScalarValue::new_list(&orderings, &struct_type); Ok(ScalarValue::List(arr)) } diff --git a/datafusion/sqllogictest/test_files/aggregate.slt b/datafusion/sqllogictest/test_files/aggregate.slt index 7cfc9c707d43..bcda3464f49b 100644 --- a/datafusion/sqllogictest/test_files/aggregate.slt +++ b/datafusion/sqllogictest/test_files/aggregate.slt @@ -106,6 +106,36 @@ FROM ---- [0VVIHzxWtNOFLtnhjHEKjXaJOSLJfm, 0keZ5G8BffGwgF2RwQD59TFzMStxCB, 0og6hSkhbX8AC1ktFS4kounvTzy8Vo, 1aOcrEGd0cOqZe2I5XBOm0nDcwtBZO, 2T3wSlHdEmASmO0xcXHnndkKEt6bz8] +statement ok +CREATE EXTERNAL TABLE agg_order ( +c1 INT NOT NULL, +c2 INT NOT NULL, +c3 INT NOT NULL +) +STORED AS CSV +WITH HEADER ROW +LOCATION '../core/tests/data/aggregate_agg_multi_order.csv'; + +# test array_agg with order by multiple columns +query ? +select array_agg(c1 order by c2 desc, c3) from agg_order; +---- +[5, 6, 7, 8, 9, 1, 2, 3, 4, 10] + +query TT +explain select array_agg(c1 order by c2 desc, c3) from agg_order; +---- +logical_plan +Aggregate: groupBy=[[]], aggr=[[ARRAY_AGG(agg_order.c1) ORDER BY [agg_order.c2 DESC NULLS FIRST, agg_order.c3 ASC NULLS LAST]]] +--TableScan: agg_order projection=[c1, c2, c3] +physical_plan +AggregateExec: mode=Final, gby=[], aggr=[ARRAY_AGG(agg_order.c1)] +--CoalescePartitionsExec +----AggregateExec: mode=Partial, gby=[], aggr=[ARRAY_AGG(agg_order.c1)] +------SortExec: expr=[c2@1 DESC,c3@2 ASC NULLS LAST] +--------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +----------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/aggregate_agg_multi_order.csv]]}, projection=[c1, c2, c3], has_header=true + statement error This feature is not implemented: LIMIT not supported in ARRAY_AGG: 1 SELECT array_agg(c13 LIMIT 1) FROM aggregate_test_100 From 391f301efdd37cbacbf11bf9db2d335b21a53a57 Mon Sep 17 00:00:00 2001 From: Thomas Cameron Date: Mon, 11 Dec 2023 23:17:11 +0900 Subject: [PATCH 406/572] Sort filenames when reading parquet to ensure consistent schema (#6629) * update * FIXed * add parquet * update * update * update * try2 * update * update * Add comments * cargo fmt --------- Co-authored-by: Andrew Lamb --- .../src/datasource/file_format/parquet.rs | 85 +++++++++++++++++-- 1 file changed, 80 insertions(+), 5 deletions(-) diff --git a/datafusion/core/src/datasource/file_format/parquet.rs b/datafusion/core/src/datasource/file_format/parquet.rs index 09e54558f12e..9db320fb9da4 100644 --- a/datafusion/core/src/datasource/file_format/parquet.rs +++ b/datafusion/core/src/datasource/file_format/parquet.rs @@ -164,6 +164,16 @@ fn clear_metadata( }) } +async fn fetch_schema_with_location( + store: &dyn ObjectStore, + file: &ObjectMeta, + metadata_size_hint: Option, +) -> Result<(Path, Schema)> { + let loc_path = file.location.clone(); + let schema = fetch_schema(store, file, metadata_size_hint).await?; + Ok((loc_path, schema)) +} + #[async_trait] impl FileFormat for ParquetFormat { fn as_any(&self) -> &dyn Any { @@ -176,13 +186,32 @@ impl FileFormat for ParquetFormat { store: &Arc, objects: &[ObjectMeta], ) -> Result { - let schemas: Vec<_> = futures::stream::iter(objects) - .map(|object| fetch_schema(store.as_ref(), object, self.metadata_size_hint)) + let mut schemas: Vec<_> = futures::stream::iter(objects) + .map(|object| { + fetch_schema_with_location( + store.as_ref(), + object, + self.metadata_size_hint, + ) + }) .boxed() // Workaround https://github.com/rust-lang/rust/issues/64552 .buffered(state.config_options().execution.meta_fetch_concurrency) .try_collect() .await?; + // Schema inference adds fields based the order they are seen + // which depends on the order the files are processed. For some + // object stores (like local file systems) the order returned from list + // is not deterministic. Thus, to ensure deterministic schema inference + // sort the files first. + // https://github.com/apache/arrow-datafusion/pull/6629 + schemas.sort_by(|(location1, _), (location2, _)| location1.cmp(location2)); + + let schemas = schemas + .into_iter() + .map(|(_, schema)| schema) + .collect::>(); + let schema = if self.skip_metadata(state.config_options()) { Schema::try_merge(clear_metadata(schemas)) } else { @@ -1124,12 +1153,21 @@ pub(crate) mod test_util { batches: Vec, multi_page: bool, ) -> Result<(Vec, Vec)> { + // we need the tmp files to be sorted as some tests rely on the how the returning files are ordered + // https://github.com/apache/arrow-datafusion/pull/6629 + let tmp_files = { + let mut tmp_files: Vec<_> = (0..batches.len()) + .map(|_| NamedTempFile::new().expect("creating temp file")) + .collect(); + tmp_files.sort_by(|a, b| a.path().cmp(b.path())); + tmp_files + }; + // Each batch writes to their own file let files: Vec<_> = batches .into_iter() - .map(|batch| { - let mut output = NamedTempFile::new().expect("creating temp file"); - + .zip(tmp_files.into_iter()) + .map(|(batch, mut output)| { let builder = WriterProperties::builder(); let props = if multi_page { builder.set_data_page_row_count_limit(ROWS_PER_PAGE) @@ -1155,6 +1193,7 @@ pub(crate) mod test_util { .collect(); let meta: Vec<_> = files.iter().map(local_unpartitioned_file).collect(); + Ok((meta, files)) } @@ -1254,6 +1293,42 @@ mod tests { Ok(()) } + #[tokio::test] + async fn is_schema_stable() -> Result<()> { + let c1: ArrayRef = + Arc::new(StringArray::from(vec![Some("Foo"), None, Some("bar")])); + + let c2: ArrayRef = Arc::new(Int64Array::from(vec![Some(1), Some(2), None])); + + let batch1 = + RecordBatch::try_from_iter(vec![("a", c1.clone()), ("b", c1.clone())]) + .unwrap(); + let batch2 = + RecordBatch::try_from_iter(vec![("c", c2.clone()), ("d", c2.clone())]) + .unwrap(); + + let store = Arc::new(LocalFileSystem::new()) as _; + let (meta, _files) = store_parquet(vec![batch1, batch2], false).await?; + + let session = SessionContext::new(); + let ctx = session.state(); + let format = ParquetFormat::default(); + let schema = format.infer_schema(&ctx, &store, &meta).await.unwrap(); + + let order: Vec<_> = ["a", "b", "c", "d"] + .into_iter() + .map(|i| i.to_string()) + .collect(); + let coll: Vec<_> = schema + .all_fields() + .into_iter() + .map(|i| i.name().to_string()) + .collect(); + assert_eq!(coll, order); + + Ok(()) + } + #[derive(Debug)] struct RequestCountingObjectStore { inner: Arc, From 1861c3d42bbb0c4caa9c5a61c65065b87b32aa35 Mon Sep 17 00:00:00 2001 From: Andrew Lamb Date: Mon, 11 Dec 2023 10:10:50 -0500 Subject: [PATCH 407/572] Minor: Improve comments in EnforceDistribution tests (#8474) --- .../enforce_distribution.rs | 34 ++++++++++++------- 1 file changed, 22 insertions(+), 12 deletions(-) diff --git a/datafusion/core/src/physical_optimizer/enforce_distribution.rs b/datafusion/core/src/physical_optimizer/enforce_distribution.rs index 4befea741c8c..3aed6555f305 100644 --- a/datafusion/core/src/physical_optimizer/enforce_distribution.rs +++ b/datafusion/core/src/physical_optimizer/enforce_distribution.rs @@ -256,7 +256,7 @@ impl PhysicalOptimizerRule for EnforceDistribution { /// 1) If the current plan is Partitioned HashJoin, SortMergeJoin, check whether the requirements can be satisfied by adjusting join keys ordering: /// Requirements can not be satisfied, clear the current requirements, generate new requirements(to pushdown) based on the current join keys, return the unchanged plan. /// Requirements is already satisfied, clear the current requirements, generate new requirements(to pushdown) based on the current join keys, return the unchanged plan. -/// Requirements can be satisfied by adjusting keys ordering, clear the current requiements, generate new requirements(to pushdown) based on the adjusted join keys, return the changed plan. +/// Requirements can be satisfied by adjusting keys ordering, clear the current requirements, generate new requirements(to pushdown) based on the adjusted join keys, return the changed plan. /// /// 2) If the current plan is Aggregation, check whether the requirements can be satisfied by adjusting group by keys ordering: /// Requirements can not be satisfied, clear all the requirements, return the unchanged plan. @@ -928,7 +928,7 @@ fn add_roundrobin_on_top( // If any of the following conditions is true // - Preserving ordering is not helpful in terms of satisfying ordering requirements // - Usage of order preserving variants is not desirable - // (determined by flag `config.optimizer.bounded_order_preserving_variants`) + // (determined by flag `config.optimizer.prefer_existing_sort`) let partitioning = Partitioning::RoundRobinBatch(n_target); let repartition = RepartitionExec::try_new(input, partitioning)?.with_preserve_order(); @@ -996,7 +996,7 @@ fn add_hash_on_top( // - Preserving ordering is not helpful in terms of satisfying ordering // requirements. // - Usage of order preserving variants is not desirable (per the flag - // `config.optimizer.bounded_order_preserving_variants`). + // `config.optimizer.prefer_existing_sort`). let mut new_plan = if repartition_beneficial_stats { // Since hashing benefits from partitioning, add a round-robin repartition // before it: @@ -1045,7 +1045,7 @@ fn add_spm_on_top( // If any of the following conditions is true // - Preserving ordering is not helpful in terms of satisfying ordering requirements // - Usage of order preserving variants is not desirable - // (determined by flag `config.optimizer.bounded_order_preserving_variants`) + // (determined by flag `config.optimizer.prefer_existing_sort`) let should_preserve_ordering = input.output_ordering().is_some(); let new_plan: Arc = if should_preserve_ordering { let existing_ordering = input.output_ordering().unwrap_or(&[]); @@ -2026,7 +2026,7 @@ pub(crate) mod tests { fn ensure_distribution_helper( plan: Arc, target_partitions: usize, - bounded_order_preserving_variants: bool, + prefer_existing_sort: bool, ) -> Result> { let distribution_context = DistributionContext::new(plan); let mut config = ConfigOptions::new(); @@ -2034,7 +2034,7 @@ pub(crate) mod tests { config.optimizer.enable_round_robin_repartition = false; config.optimizer.repartition_file_scans = false; config.optimizer.repartition_file_min_size = 1024; - config.optimizer.prefer_existing_sort = bounded_order_preserving_variants; + config.optimizer.prefer_existing_sort = prefer_existing_sort; ensure_distribution(distribution_context, &config).map(|item| item.into().plan) } @@ -2056,23 +2056,33 @@ pub(crate) mod tests { } /// Runs the repartition optimizer and asserts the plan against the expected + /// Arguments + /// * `EXPECTED_LINES` - Expected output plan + /// * `PLAN` - Input plan + /// * `FIRST_ENFORCE_DIST` - + /// true: (EnforceDistribution, EnforceDistribution, EnforceSorting) + /// false: else runs (EnforceSorting, EnforceDistribution, EnforceDistribution) + /// * `PREFER_EXISTING_SORT` (optional) - if true, will not repartition / resort data if it is already sorted + /// * `TARGET_PARTITIONS` (optional) - number of partitions to repartition to + /// * `REPARTITION_FILE_SCANS` (optional) - if true, will repartition file scans + /// * `REPARTITION_FILE_MIN_SIZE` (optional) - minimum file size to repartition macro_rules! assert_optimized { ($EXPECTED_LINES: expr, $PLAN: expr, $FIRST_ENFORCE_DIST: expr) => { assert_optimized!($EXPECTED_LINES, $PLAN, $FIRST_ENFORCE_DIST, false, 10, false, 1024); }; - ($EXPECTED_LINES: expr, $PLAN: expr, $FIRST_ENFORCE_DIST: expr, $BOUNDED_ORDER_PRESERVING_VARIANTS: expr) => { - assert_optimized!($EXPECTED_LINES, $PLAN, $FIRST_ENFORCE_DIST, $BOUNDED_ORDER_PRESERVING_VARIANTS, 10, false, 1024); + ($EXPECTED_LINES: expr, $PLAN: expr, $FIRST_ENFORCE_DIST: expr, $PREFER_EXISTING_SORT: expr) => { + assert_optimized!($EXPECTED_LINES, $PLAN, $FIRST_ENFORCE_DIST, $PREFER_EXISTING_SORT, 10, false, 1024); }; - ($EXPECTED_LINES: expr, $PLAN: expr, $FIRST_ENFORCE_DIST: expr, $BOUNDED_ORDER_PRESERVING_VARIANTS: expr, $TARGET_PARTITIONS: expr, $REPARTITION_FILE_SCANS: expr, $REPARTITION_FILE_MIN_SIZE: expr) => { + ($EXPECTED_LINES: expr, $PLAN: expr, $FIRST_ENFORCE_DIST: expr, $PREFER_EXISTING_SORT: expr, $TARGET_PARTITIONS: expr, $REPARTITION_FILE_SCANS: expr, $REPARTITION_FILE_MIN_SIZE: expr) => { let expected_lines: Vec<&str> = $EXPECTED_LINES.iter().map(|s| *s).collect(); let mut config = ConfigOptions::new(); config.execution.target_partitions = $TARGET_PARTITIONS; config.optimizer.repartition_file_scans = $REPARTITION_FILE_SCANS; config.optimizer.repartition_file_min_size = $REPARTITION_FILE_MIN_SIZE; - config.optimizer.prefer_existing_sort = $BOUNDED_ORDER_PRESERVING_VARIANTS; + config.optimizer.prefer_existing_sort = $PREFER_EXISTING_SORT; // NOTE: These tests verify the joint `EnforceDistribution` + `EnforceSorting` cascade // because they were written prior to the separation of `BasicEnforcement` into @@ -3294,7 +3304,7 @@ pub(crate) mod tests { ]; assert_optimized!(expected, exec, true); // In this case preserving ordering through order preserving operators is not desirable - // (according to flag: bounded_order_preserving_variants) + // (according to flag: PREFER_EXISTING_SORT) // hence in this case ordering lost during CoalescePartitionsExec and re-introduced with // SortExec at the top. let expected = &[ @@ -4341,7 +4351,7 @@ pub(crate) mod tests { "ParquetExec: file_groups={2 groups: [[x], [y]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC]", ]; - // last flag sets config.optimizer.bounded_order_preserving_variants + // last flag sets config.optimizer.PREFER_EXISTING_SORT assert_optimized!(expected, physical_plan.clone(), true, true); assert_optimized!(expected, physical_plan, false, true); From 171a5fd18e6a4e742fc9763bffb70b1fcc21f3b3 Mon Sep 17 00:00:00 2001 From: Wei Date: Mon, 11 Dec 2023 23:40:36 +0800 Subject: [PATCH 408/572] fix: support uppercase when parsing `Interval` (#8478) * fix: interval uppercase unit * feat: add test * chore: fmt * chore: remove redundant test --- datafusion/sql/src/expr/value.rs | 1 + .../sqllogictest/test_files/interval.slt | 80 +++++++++++++++++++ 2 files changed, 81 insertions(+) diff --git a/datafusion/sql/src/expr/value.rs b/datafusion/sql/src/expr/value.rs index 708f7c60011a..9f88318ab21a 100644 --- a/datafusion/sql/src/expr/value.rs +++ b/datafusion/sql/src/expr/value.rs @@ -343,6 +343,7 @@ impl<'a, S: ContextProvider> SqlToRel<'a, S> { // TODO make interval parsing better in arrow-rs / expose `IntervalType` fn has_units(val: &str) -> bool { + let val = val.to_lowercase(); val.ends_with("century") || val.ends_with("centuries") || val.ends_with("decade") diff --git a/datafusion/sqllogictest/test_files/interval.slt b/datafusion/sqllogictest/test_files/interval.slt index 500876f76221..f2ae2984f07b 100644 --- a/datafusion/sqllogictest/test_files/interval.slt +++ b/datafusion/sqllogictest/test_files/interval.slt @@ -126,6 +126,86 @@ select interval '5' nanoseconds ---- 0 years 0 mons 0 days 0 hours 0 mins 0.000000005 secs +query ? +select interval '5 YEAR' +---- +0 years 60 mons 0 days 0 hours 0 mins 0.000000000 secs + +query ? +select interval '5 MONTH' +---- +0 years 5 mons 0 days 0 hours 0 mins 0.000000000 secs + +query ? +select interval '5 WEEK' +---- +0 years 0 mons 35 days 0 hours 0 mins 0.000000000 secs + +query ? +select interval '5 DAY' +---- +0 years 0 mons 5 days 0 hours 0 mins 0.000000000 secs + +query ? +select interval '5 HOUR' +---- +0 years 0 mons 0 days 5 hours 0 mins 0.000000000 secs + +query ? +select interval '5 HOURS' +---- +0 years 0 mons 0 days 5 hours 0 mins 0.000000000 secs + +query ? +select interval '5 MINUTE' +---- +0 years 0 mons 0 days 0 hours 5 mins 0.000000000 secs + +query ? +select interval '5 SECOND' +---- +0 years 0 mons 0 days 0 hours 0 mins 5.000000000 secs + +query ? +select interval '5 SECONDS' +---- +0 years 0 mons 0 days 0 hours 0 mins 5.000000000 secs + +query ? +select interval '5 MILLISECOND' +---- +0 years 0 mons 0 days 0 hours 0 mins 0.005000000 secs + +query ? +select interval '5 MILLISECONDS' +---- +0 years 0 mons 0 days 0 hours 0 mins 0.005000000 secs + +query ? +select interval '5 MICROSECOND' +---- +0 years 0 mons 0 days 0 hours 0 mins 0.000005000 secs + +query ? +select interval '5 MICROSECONDS' +---- +0 years 0 mons 0 days 0 hours 0 mins 0.000005000 secs + +query ? +select interval '5 NANOSECOND' +---- +0 years 0 mons 0 days 0 hours 0 mins 0.000000005 secs + +query ? +select interval '5 NANOSECONDS' +---- +0 years 0 mons 0 days 0 hours 0 mins 0.000000005 secs + +query ? +select interval '5 YEAR 5 MONTH 5 DAY 5 HOUR 5 MINUTE 5 SECOND 5 MILLISECOND 5 MICROSECOND 5 NANOSECOND' +---- +0 years 65 mons 5 days 5 hours 5 mins 5.005005005 secs + # Interval with string literal addition query ? select interval '1 month' + '1 month' From 95ba48bd2291dd5c303bdaf88cbb55c79d395930 Mon Sep 17 00:00:00 2001 From: Mustafa Akur <106137913+mustafasrepo@users.noreply.github.com> Date: Mon, 11 Dec 2023 20:05:55 +0300 Subject: [PATCH 409/572] Better Equivalence (ordering and exact equivalence) Propagation through ProjectionExec (#8484) * Better projection support complex expression support --------- Co-authored-by: metesynnada <100111937+metesynnada@users.noreply.github.com> Co-authored-by: Mehmet Ozan Kabak --- datafusion/physical-expr/src/equivalence.rs | 1746 ++++++++++++++++++- 1 file changed, 1679 insertions(+), 67 deletions(-) diff --git a/datafusion/physical-expr/src/equivalence.rs b/datafusion/physical-expr/src/equivalence.rs index 4a562f4ef101..defd7b5786a3 100644 --- a/datafusion/physical-expr/src/equivalence.rs +++ b/datafusion/physical-expr/src/equivalence.rs @@ -15,7 +15,8 @@ // specific language governing permissions and limitations // under the License. -use std::hash::Hash; +use std::collections::{HashMap, HashSet}; +use std::hash::{Hash, Hasher}; use std::sync::Arc; use crate::expressions::{Column, Literal}; @@ -26,12 +27,14 @@ use crate::{ LexRequirement, LexRequirementRef, PhysicalExpr, PhysicalSortExpr, PhysicalSortRequirement, }; + use arrow::datatypes::SchemaRef; use arrow_schema::SortOptions; use datafusion_common::tree_node::{Transformed, TreeNode}; use datafusion_common::{JoinSide, JoinType, Result}; use indexmap::IndexSet; +use itertools::Itertools; /// An `EquivalenceClass` is a set of [`Arc`]s that are known /// to have the same value for all tuples in a relation. These are generated by @@ -465,31 +468,6 @@ impl EquivalenceGroup { .map(|children| expr.clone().with_new_children(children).unwrap()) } - /// Projects `ordering` according to the given projection mapping. - /// If the resulting ordering is invalid after projection, returns `None`. - fn project_ordering( - &self, - mapping: &ProjectionMapping, - ordering: LexOrderingRef, - ) -> Option { - // If any sort expression is invalid after projection, rest of the - // ordering shouldn't be projected either. For example, if input ordering - // is [a ASC, b ASC, c ASC], and column b is not valid after projection, - // the result should be [a ASC], not [a ASC, c ASC], even if column c is - // valid after projection. - let result = ordering - .iter() - .map_while(|sort_expr| { - self.project_expr(mapping, &sort_expr.expr) - .map(|expr| PhysicalSortExpr { - expr, - options: sort_expr.options, - }) - }) - .collect::>(); - (!result.is_empty()).then_some(result) - } - /// Projects this equivalence group according to the given projection mapping. pub fn project(&self, mapping: &ProjectionMapping) -> Self { let projected_classes = self.iter().filter_map(|cls| { @@ -724,8 +702,21 @@ impl OrderingEquivalenceClass { // Append orderings in `other` to all existing orderings in this equivalence // class. pub fn join_suffix(mut self, other: &Self) -> Self { - for ordering in other.iter() { - for idx in 0..self.orderings.len() { + let n_ordering = self.orderings.len(); + // Replicate entries before cross product + let n_cross = std::cmp::max(n_ordering, other.len() * n_ordering); + self.orderings = self + .orderings + .iter() + .cloned() + .cycle() + .take(n_cross) + .collect(); + // Suffix orderings of other to the current orderings. + for (outer_idx, ordering) in other.iter().enumerate() { + for idx in 0..n_ordering { + // Calculate cross product index + let idx = outer_idx * n_ordering + idx; self.orderings[idx].extend(ordering.iter().cloned()); } } @@ -1196,6 +1187,181 @@ impl EquivalenceProperties { self.eq_group.project_expr(projection_mapping, expr) } + /// Constructs a dependency map based on existing orderings referred to in + /// the projection. + /// + /// This function analyzes the orderings in the normalized order-equivalence + /// class and builds a dependency map. The dependency map captures relationships + /// between expressions within the orderings, helping to identify dependencies + /// and construct valid projected orderings during projection operations. + /// + /// # Parameters + /// + /// - `mapping`: A reference to the `ProjectionMapping` that defines the + /// relationship between source and target expressions. + /// + /// # Returns + /// + /// A [`DependencyMap`] representing the dependency map, where each + /// [`DependencyNode`] contains dependencies for the key [`PhysicalSortExpr`]. + /// + /// # Example + /// + /// Assume we have two equivalent orderings: `[a ASC, b ASC]` and `[a ASC, c ASC]`, + /// and the projection mapping is `[a -> a_new, b -> b_new, b + c -> b + c]`. + /// Then, the dependency map will be: + /// + /// ```text + /// a ASC: Node {Some(a_new ASC), HashSet{}} + /// b ASC: Node {Some(b_new ASC), HashSet{a ASC}} + /// c ASC: Node {None, HashSet{a ASC}} + /// ``` + fn construct_dependency_map(&self, mapping: &ProjectionMapping) -> DependencyMap { + let mut dependency_map = HashMap::new(); + for ordering in self.normalized_oeq_class().iter() { + for (idx, sort_expr) in ordering.iter().enumerate() { + let target_sort_expr = + self.project_expr(&sort_expr.expr, mapping).map(|expr| { + PhysicalSortExpr { + expr, + options: sort_expr.options, + } + }); + let is_projected = target_sort_expr.is_some(); + if is_projected + || mapping + .iter() + .any(|(source, _)| expr_refers(source, &sort_expr.expr)) + { + // Previous ordering is a dependency. Note that there is no, + // dependency for a leading ordering (i.e. the first sort + // expression). + let dependency = idx.checked_sub(1).map(|a| &ordering[a]); + // Add sort expressions that can be projected or referred to + // by any of the projection expressions to the dependency map: + dependency_map + .entry(sort_expr.clone()) + .or_insert_with(|| DependencyNode { + target_sort_expr: target_sort_expr.clone(), + dependencies: HashSet::new(), + }) + .insert_dependency(dependency); + } + if !is_projected { + // If we can not project, stop constructing the dependency + // map as remaining dependencies will be invalid after projection. + break; + } + } + } + dependency_map + } + + /// Returns a new `ProjectionMapping` where source expressions are normalized. + /// + /// This normalization ensures that source expressions are transformed into a + /// consistent representation. This is beneficial for algorithms that rely on + /// exact equalities, as it allows for more precise and reliable comparisons. + /// + /// # Parameters + /// + /// - `mapping`: A reference to the original `ProjectionMapping` to be normalized. + /// + /// # Returns + /// + /// A new `ProjectionMapping` with normalized source expressions. + fn normalized_mapping(&self, mapping: &ProjectionMapping) -> ProjectionMapping { + // Construct the mapping where source expressions are normalized. In this way + // In the algorithms below we can work on exact equalities + ProjectionMapping { + map: mapping + .iter() + .map(|(source, target)| { + let normalized_source = self.eq_group.normalize_expr(source.clone()); + (normalized_source, target.clone()) + }) + .collect(), + } + } + + /// Computes projected orderings based on a given projection mapping. + /// + /// This function takes a `ProjectionMapping` and computes the possible + /// orderings for the projected expressions. It considers dependencies + /// between expressions and generates valid orderings according to the + /// specified sort properties. + /// + /// # Parameters + /// + /// - `mapping`: A reference to the `ProjectionMapping` that defines the + /// relationship between source and target expressions. + /// + /// # Returns + /// + /// A vector of `LexOrdering` containing all valid orderings after projection. + fn projected_orderings(&self, mapping: &ProjectionMapping) -> Vec { + let mapping = self.normalized_mapping(mapping); + + // Get dependency map for existing orderings: + let dependency_map = self.construct_dependency_map(&mapping); + + let orderings = mapping.iter().flat_map(|(source, target)| { + referred_dependencies(&dependency_map, source) + .into_iter() + .filter_map(|relevant_deps| { + if let SortProperties::Ordered(options) = + get_expr_ordering(source, &relevant_deps) + { + Some((options, relevant_deps)) + } else { + // Do not consider unordered cases + None + } + }) + .flat_map(|(options, relevant_deps)| { + let sort_expr = PhysicalSortExpr { + expr: target.clone(), + options, + }; + // Generate dependent orderings (i.e. prefixes for `sort_expr`): + let mut dependency_orderings = + generate_dependency_orderings(&relevant_deps, &dependency_map); + // Append `sort_expr` to the dependent orderings: + for ordering in dependency_orderings.iter_mut() { + ordering.push(sort_expr.clone()); + } + dependency_orderings + }) + }); + + // Add valid projected orderings. For example, if existing ordering is + // `a + b` and projection is `[a -> a_new, b -> b_new]`, we need to + // preserve `a_new + b_new` as ordered. Please note that `a_new` and + // `b_new` themselves need not be ordered. Such dependencies cannot be + // deduced via the pass above. + let projected_orderings = dependency_map.iter().flat_map(|(sort_expr, node)| { + let mut prefixes = construct_prefix_orderings(sort_expr, &dependency_map); + if prefixes.is_empty() { + // If prefix is empty, there is no dependency. Insert + // empty ordering: + prefixes = vec![vec![]]; + } + // Append current ordering on top its dependencies: + for ordering in prefixes.iter_mut() { + if let Some(target) = &node.target_sort_expr { + ordering.push(target.clone()) + } + } + prefixes + }); + + // Simplify each ordering by removing redundant sections: + orderings + .chain(projected_orderings) + .map(collapse_lex_ordering) + .collect() + } + /// Projects constants based on the provided `ProjectionMapping`. /// /// This function takes a `ProjectionMapping` and identifies/projects @@ -1240,28 +1406,13 @@ impl EquivalenceProperties { projection_mapping: &ProjectionMapping, output_schema: SchemaRef, ) -> Self { - let mut projected_orderings = self - .oeq_class - .iter() - .filter_map(|order| self.eq_group.project_ordering(projection_mapping, order)) - .collect::>(); - for (source, target) in projection_mapping.iter() { - let expr_ordering = ExprOrdering::new(source.clone()) - .transform_up(&|expr| Ok(update_ordering(expr, self))) - // Guaranteed to always return `Ok`. - .unwrap(); - if let SortProperties::Ordered(options) = expr_ordering.state { - // Push new ordering to the state. - projected_orderings.push(vec![PhysicalSortExpr { - expr: target.clone(), - options, - }]); - } - } + let projected_constants = self.projected_constants(projection_mapping); + let projected_eq_group = self.eq_group.project(projection_mapping); + let projected_orderings = self.projected_orderings(projection_mapping); Self { - eq_group: self.eq_group.project(projection_mapping), + eq_group: projected_eq_group, oeq_class: OrderingEquivalenceClass::new(projected_orderings), - constants: self.projected_constants(projection_mapping), + constants: projected_constants, schema: output_schema, } } @@ -1397,6 +1548,270 @@ fn is_constant_recurse( !children.is_empty() && children.iter().all(|c| is_constant_recurse(constants, c)) } +/// This function examines whether a referring expression directly refers to a +/// given referred expression or if any of its children in the expression tree +/// refer to the specified expression. +/// +/// # Parameters +/// +/// - `referring_expr`: A reference to the referring expression (`Arc`). +/// - `referred_expr`: A reference to the referred expression (`Arc`) +/// +/// # Returns +/// +/// A boolean value indicating whether `referring_expr` refers (needs it to evaluate its result) +/// `referred_expr` or not. +fn expr_refers( + referring_expr: &Arc, + referred_expr: &Arc, +) -> bool { + referring_expr.eq(referred_expr) + || referring_expr + .children() + .iter() + .any(|child| expr_refers(child, referred_expr)) +} + +/// Wrapper struct for `Arc` to use them as keys in a hash map. +#[derive(Debug, Clone)] +struct ExprWrapper(Arc); + +impl PartialEq for ExprWrapper { + fn eq(&self, other: &Self) -> bool { + self.0.eq(&other.0) + } +} + +impl Eq for ExprWrapper {} + +impl Hash for ExprWrapper { + fn hash(&self, state: &mut H) { + self.0.hash(state); + } +} + +/// This function analyzes the dependency map to collect referred dependencies for +/// a given source expression. +/// +/// # Parameters +/// +/// - `dependency_map`: A reference to the `DependencyMap` where each +/// `PhysicalSortExpr` is associated with a `DependencyNode`. +/// - `source`: A reference to the source expression (`Arc`) +/// for which relevant dependencies need to be identified. +/// +/// # Returns +/// +/// A `Vec` containing the dependencies for the given source +/// expression. These dependencies are expressions that are referred to by +/// the source expression based on the provided dependency map. +fn referred_dependencies( + dependency_map: &DependencyMap, + source: &Arc, +) -> Vec { + // Associate `PhysicalExpr`s with `PhysicalSortExpr`s that contain them: + let mut expr_to_sort_exprs = HashMap::::new(); + for sort_expr in dependency_map + .keys() + .filter(|sort_expr| expr_refers(source, &sort_expr.expr)) + { + let key = ExprWrapper(sort_expr.expr.clone()); + expr_to_sort_exprs + .entry(key) + .or_default() + .insert(sort_expr.clone()); + } + + // Generate all valid dependencies for the source. For example, if the source + // is `a + b` and the map is `[a -> (a ASC, a DESC), b -> (b ASC)]`, we get + // `vec![HashSet(a ASC, b ASC), HashSet(a DESC, b ASC)]`. + expr_to_sort_exprs + .values() + .multi_cartesian_product() + .map(|referred_deps| referred_deps.into_iter().cloned().collect()) + .collect() +} + +/// This function recursively analyzes the dependencies of the given sort +/// expression within the given dependency map to construct lexicographical +/// orderings that include the sort expression and its dependencies. +/// +/// # Parameters +/// +/// - `referred_sort_expr`: A reference to the sort expression (`PhysicalSortExpr`) +/// for which lexicographical orderings satisfying its dependencies are to be +/// constructed. +/// - `dependency_map`: A reference to the `DependencyMap` that contains +/// dependencies for different `PhysicalSortExpr`s. +/// +/// # Returns +/// +/// A vector of lexicographical orderings (`Vec`) based on the given +/// sort expression and its dependencies. +fn construct_orderings( + referred_sort_expr: &PhysicalSortExpr, + dependency_map: &DependencyMap, +) -> Vec { + // We are sure that `referred_sort_expr` is inside `dependency_map`. + let node = &dependency_map[referred_sort_expr]; + // Since we work on intermediate nodes, we are sure `val.target_sort_expr` + // exists. + let target_sort_expr = node.target_sort_expr.clone().unwrap(); + if node.dependencies.is_empty() { + vec![vec![target_sort_expr]] + } else { + node.dependencies + .iter() + .flat_map(|dep| { + let mut orderings = construct_orderings(dep, dependency_map); + for ordering in orderings.iter_mut() { + ordering.push(target_sort_expr.clone()) + } + orderings + }) + .collect() + } +} + +/// This function retrieves the dependencies of the given relevant sort expression +/// from the given dependency map. It then constructs prefix orderings by recursively +/// analyzing the dependencies and include them in the orderings. +/// +/// # Parameters +/// +/// - `relevant_sort_expr`: A reference to the relevant sort expression +/// (`PhysicalSortExpr`) for which prefix orderings are to be constructed. +/// - `dependency_map`: A reference to the `DependencyMap` containing dependencies. +/// +/// # Returns +/// +/// A vector of prefix orderings (`Vec`) based on the given relevant +/// sort expression and its dependencies. +fn construct_prefix_orderings( + relevant_sort_expr: &PhysicalSortExpr, + dependency_map: &DependencyMap, +) -> Vec { + dependency_map[relevant_sort_expr] + .dependencies + .iter() + .flat_map(|dep| construct_orderings(dep, dependency_map)) + .collect() +} + +/// Given a set of relevant dependencies (`relevant_deps`) and a map of dependencies +/// (`dependency_map`), this function generates all possible prefix orderings +/// based on the given dependencies. +/// +/// # Parameters +/// +/// * `dependencies` - A reference to the dependencies. +/// * `dependency_map` - A reference to the map of dependencies for expressions. +/// +/// # Returns +/// +/// A vector of lexical orderings (`Vec`) representing all valid orderings +/// based on the given dependencies. +fn generate_dependency_orderings( + dependencies: &Dependencies, + dependency_map: &DependencyMap, +) -> Vec { + // Construct all the valid prefix orderings for each expression appearing + // in the projection: + let relevant_prefixes = dependencies + .iter() + .flat_map(|dep| { + let prefixes = construct_prefix_orderings(dep, dependency_map); + (!prefixes.is_empty()).then_some(prefixes) + }) + .collect::>(); + + // No dependency, dependent is a leading ordering. + if relevant_prefixes.is_empty() { + // Return an empty ordering: + return vec![vec![]]; + } + + // Generate all possible orderings where dependencies are satisfied for the + // current projection expression. For example, if expression is `a + b ASC`, + // and the dependency for `a ASC` is `[c ASC]`, the dependency for `b ASC` + // is `[d DESC]`, then we generate `[c ASC, d DESC, a + b ASC]` and + // `[d DESC, c ASC, a + b ASC]`. + relevant_prefixes + .into_iter() + .multi_cartesian_product() + .flat_map(|prefix_orderings| { + prefix_orderings + .iter() + .permutations(prefix_orderings.len()) + .map(|prefixes| prefixes.into_iter().flatten().cloned().collect()) + .collect::>() + }) + .collect() +} + +/// This function examines the given expression and the sort expressions it +/// refers to determine the ordering properties of the expression. +/// +/// # Parameters +/// +/// - `expr`: A reference to the source expression (`Arc`) for +/// which ordering properties need to be determined. +/// - `dependencies`: A reference to `Dependencies`, containing sort expressions +/// referred to by `expr`. +/// +/// # Returns +/// +/// A `SortProperties` indicating the ordering information of the given expression. +fn get_expr_ordering( + expr: &Arc, + dependencies: &Dependencies, +) -> SortProperties { + if let Some(column_order) = dependencies.iter().find(|&order| expr.eq(&order.expr)) { + // If exact match is found, return its ordering. + SortProperties::Ordered(column_order.options) + } else { + // Find orderings of its children + let child_states = expr + .children() + .iter() + .map(|child| get_expr_ordering(child, dependencies)) + .collect::>(); + // Calculate expression ordering using ordering of its children. + expr.get_ordering(&child_states) + } +} + +/// Represents a node in the dependency map used to construct projected orderings. +/// +/// A `DependencyNode` contains information about a particular sort expression, +/// including its target sort expression and a set of dependencies on other sort +/// expressions. +/// +/// # Fields +/// +/// - `target_sort_expr`: An optional `PhysicalSortExpr` representing the target +/// sort expression associated with the node. It is `None` if the sort expression +/// cannot be projected. +/// - `dependencies`: A [`Dependencies`] containing dependencies on other sort +/// expressions that are referred to by the target sort expression. +#[derive(Debug, Clone, PartialEq, Eq)] +struct DependencyNode { + target_sort_expr: Option, + dependencies: Dependencies, +} + +impl DependencyNode { + // Insert dependency to the state (if exists). + fn insert_dependency(&mut self, dependency: Option<&PhysicalSortExpr>) { + if let Some(dep) = dependency { + self.dependencies.insert(dep.clone()); + } + } +} + +type DependencyMap = HashMap; +type Dependencies = HashSet; + /// Calculate ordering equivalence properties for the given join operation. pub fn join_equivalence_properties( left: EquivalenceProperties, @@ -1544,7 +1959,7 @@ mod tests { use arrow::datatypes::{DataType, Field, Schema}; use arrow_array::{ArrayRef, Float64Array, RecordBatch, UInt32Array}; use arrow_schema::{Fields, SortOptions, TimeUnit}; - use datafusion_common::{Result, ScalarValue}; + use datafusion_common::{plan_datafusion_err, DataFusionError, Result, ScalarValue}; use datafusion_expr::{BuiltinScalarFunction, Operator}; use itertools::{izip, Itertools}; @@ -1552,6 +1967,37 @@ mod tests { use rand::seq::SliceRandom; use rand::{Rng, SeedableRng}; + fn output_schema( + mapping: &ProjectionMapping, + input_schema: &Arc, + ) -> Result { + // Calculate output schema + let fields: Result> = mapping + .iter() + .map(|(source, target)| { + let name = target + .as_any() + .downcast_ref::() + .ok_or_else(|| plan_datafusion_err!("Expects to have column"))? + .name(); + let field = Field::new( + name, + source.data_type(input_schema)?, + source.nullable(input_schema)?, + ); + + Ok(field) + }) + .collect(); + + let output_schema = Arc::new(Schema::new_with_metadata( + fields?, + input_schema.metadata().clone(), + )); + + Ok(output_schema) + } + // Generate a schema which consists of 8 columns (a, b, c, d, e, f, g, h) fn create_test_schema() -> Result { let a = Field::new("a", DataType::Int32, true); @@ -1679,7 +2125,7 @@ mod tests { .map(|(expr, options)| { PhysicalSortRequirement::new((*expr).clone(), *options) }) - .collect::>() + .collect() } // Convert each tuple to PhysicalSortExpr @@ -1692,7 +2138,7 @@ mod tests { expr: (*expr).clone(), options: *options, }) - .collect::>() + .collect() } // Convert each inner tuple to PhysicalSortExpr @@ -1705,6 +2151,56 @@ mod tests { .collect() } + // Convert each tuple to PhysicalSortExpr + fn convert_to_sort_exprs_owned( + in_data: &[(Arc, SortOptions)], + ) -> Vec { + in_data + .iter() + .map(|(expr, options)| PhysicalSortExpr { + expr: (*expr).clone(), + options: *options, + }) + .collect() + } + + // Convert each inner tuple to PhysicalSortExpr + fn convert_to_orderings_owned( + orderings: &[Vec<(Arc, SortOptions)>], + ) -> Vec> { + orderings + .iter() + .map(|sort_exprs| convert_to_sort_exprs_owned(sort_exprs)) + .collect() + } + + // Apply projection to the input_data, return projected equivalence properties and record batch + fn apply_projection( + proj_exprs: Vec<(Arc, String)>, + input_data: &RecordBatch, + input_eq_properties: &EquivalenceProperties, + ) -> Result<(RecordBatch, EquivalenceProperties)> { + let input_schema = input_data.schema(); + let projection_mapping = ProjectionMapping::try_new(&proj_exprs, &input_schema)?; + + let output_schema = output_schema(&projection_mapping, &input_schema)?; + let num_rows = input_data.num_rows(); + // Apply projection to the input record batch. + let projected_values = projection_mapping + .iter() + .map(|(source, _target)| source.evaluate(input_data)?.into_array(num_rows)) + .collect::>>()?; + let projected_batch = if projected_values.is_empty() { + RecordBatch::new_empty(output_schema.clone()) + } else { + RecordBatch::try_new(output_schema.clone(), projected_values)? + }; + + let projected_eq = + input_eq_properties.project(&projection_mapping, output_schema); + Ok((projected_batch, projected_eq)) + } + #[test] fn add_equal_conditions_test() -> Result<()> { let schema = Arc::new(Schema::new(vec![ @@ -1774,13 +2270,16 @@ mod tests { let input_properties = EquivalenceProperties::new(input_schema.clone()); let col_a = col("a", &input_schema)?; - let out_schema = Arc::new(Schema::new(vec![ - Field::new("a1", DataType::Int64, true), - Field::new("a2", DataType::Int64, true), - Field::new("a3", DataType::Int64, true), - Field::new("a4", DataType::Int64, true), - ])); + // a as a1, a as a2, a as a3, a as a3 + let proj_exprs = vec![ + (col_a.clone(), "a1".to_string()), + (col_a.clone(), "a2".to_string()), + (col_a.clone(), "a3".to_string()), + (col_a.clone(), "a4".to_string()), + ]; + let projection_mapping = ProjectionMapping::try_new(&proj_exprs, &input_schema)?; + let out_schema = output_schema(&projection_mapping, &input_schema)?; // a as a1, a as a2, a as a3, a as a3 let proj_exprs = vec![ (col_a.clone(), "a1".to_string()), @@ -3686,30 +4185,1143 @@ mod tests { } #[test] - fn test_expr_consists_of_constants() -> Result<()> { + fn project_orderings() -> Result<()> { let schema = Arc::new(Schema::new(vec![ Field::new("a", DataType::Int32, true), Field::new("b", DataType::Int32, true), Field::new("c", DataType::Int32, true), Field::new("d", DataType::Int32, true), + Field::new("e", DataType::Int32, true), Field::new("ts", DataType::Timestamp(TimeUnit::Nanosecond, None), true), ])); - let col_a = col("a", &schema)?; - let col_b = col("b", &schema)?; - let col_d = col("d", &schema)?; + let col_a = &col("a", &schema)?; + let col_b = &col("b", &schema)?; + let col_c = &col("c", &schema)?; + let col_d = &col("d", &schema)?; + let col_e = &col("e", &schema)?; + let col_ts = &col("ts", &schema)?; + let interval = Arc::new(Literal::new(ScalarValue::IntervalDayTime(Some(2)))) + as Arc; + let date_bin_func = &create_physical_expr( + &BuiltinScalarFunction::DateBin, + &[interval, col_ts.clone()], + &schema, + &ExecutionProps::default(), + )?; + let a_plus_b = Arc::new(BinaryExpr::new( + col_a.clone(), + Operator::Plus, + col_b.clone(), + )) as Arc; let b_plus_d = Arc::new(BinaryExpr::new( col_b.clone(), Operator::Plus, col_d.clone(), )) as Arc; + let b_plus_e = Arc::new(BinaryExpr::new( + col_b.clone(), + Operator::Plus, + col_e.clone(), + )) as Arc; + let c_plus_d = Arc::new(BinaryExpr::new( + col_c.clone(), + Operator::Plus, + col_d.clone(), + )) as Arc; - let constants = vec![col_a.clone(), col_b.clone()]; - let expr = b_plus_d.clone(); - assert!(!is_constant_recurse(&constants, &expr)); + let option_asc = SortOptions { + descending: false, + nulls_first: false, + }; + let option_desc = SortOptions { + descending: true, + nulls_first: true, + }; - let constants = vec![col_a.clone(), col_b.clone(), col_d.clone()]; - let expr = b_plus_d.clone(); - assert!(is_constant_recurse(&constants, &expr)); + let test_cases = vec![ + // ---------- TEST CASE 1 ------------ + ( + // orderings + vec![ + // [b ASC] + vec![(col_b, option_asc)], + ], + // projection exprs + vec![(col_b, "b_new".to_string()), (col_a, "a_new".to_string())], + // expected + vec![ + // [b_new ASC] + vec![("b_new", option_asc)], + ], + ), + // ---------- TEST CASE 2 ------------ + ( + // orderings + vec![ + // empty ordering + ], + // projection exprs + vec![(col_c, "c_new".to_string()), (col_b, "b_new".to_string())], + // expected + vec![ + // no ordering at the output + ], + ), + // ---------- TEST CASE 3 ------------ + ( + // orderings + vec![ + // [ts ASC] + vec![(col_ts, option_asc)], + ], + // projection exprs + vec![ + (col_b, "b_new".to_string()), + (col_a, "a_new".to_string()), + (col_ts, "ts_new".to_string()), + (date_bin_func, "date_bin_res".to_string()), + ], + // expected + vec![ + // [date_bin_res ASC] + vec![("date_bin_res", option_asc)], + // [ts_new ASC] + vec![("ts_new", option_asc)], + ], + ), + // ---------- TEST CASE 4 ------------ + ( + // orderings + vec![ + // [a ASC, ts ASC] + vec![(col_a, option_asc), (col_ts, option_asc)], + // [b ASC, ts ASC] + vec![(col_b, option_asc), (col_ts, option_asc)], + ], + // projection exprs + vec![ + (col_b, "b_new".to_string()), + (col_a, "a_new".to_string()), + (col_ts, "ts_new".to_string()), + (date_bin_func, "date_bin_res".to_string()), + ], + // expected + vec![ + // [a_new ASC, ts_new ASC] + vec![("a_new", option_asc), ("ts_new", option_asc)], + // [a_new ASC, date_bin_res ASC] + vec![("a_new", option_asc), ("date_bin_res", option_asc)], + // [b_new ASC, ts_new ASC] + vec![("b_new", option_asc), ("ts_new", option_asc)], + // [b_new ASC, date_bin_res ASC] + vec![("b_new", option_asc), ("date_bin_res", option_asc)], + ], + ), + // ---------- TEST CASE 5 ------------ + ( + // orderings + vec![ + // [a + b ASC] + vec![(&a_plus_b, option_asc)], + ], + // projection exprs + vec![ + (col_b, "b_new".to_string()), + (col_a, "a_new".to_string()), + (&a_plus_b, "a+b".to_string()), + ], + // expected + vec![ + // [a + b ASC] + vec![("a+b", option_asc)], + ], + ), + // ---------- TEST CASE 6 ------------ + ( + // orderings + vec![ + // [a + b ASC, c ASC] + vec![(&a_plus_b, option_asc), (&col_c, option_asc)], + ], + // projection exprs + vec![ + (col_b, "b_new".to_string()), + (col_a, "a_new".to_string()), + (col_c, "c_new".to_string()), + (&a_plus_b, "a+b".to_string()), + ], + // expected + vec![ + // [a + b ASC, c_new ASC] + vec![("a+b", option_asc), ("c_new", option_asc)], + ], + ), + // ------- TEST CASE 7 ---------- + ( + vec![ + // [a ASC, b ASC, c ASC] + vec![(col_a, option_asc), (col_b, option_asc)], + // [a ASC, d ASC] + vec![(col_a, option_asc), (col_d, option_asc)], + ], + // b as b_new, a as a_new, d as d_new b+d + vec![ + (col_b, "b_new".to_string()), + (col_a, "a_new".to_string()), + (col_d, "d_new".to_string()), + (&b_plus_d, "b+d".to_string()), + ], + // expected + vec![ + // [a_new ASC, b_new ASC] + vec![("a_new", option_asc), ("b_new", option_asc)], + // [a_new ASC, d_new ASC] + vec![("a_new", option_asc), ("d_new", option_asc)], + // [a_new ASC, b+d ASC] + vec![("a_new", option_asc), ("b+d", option_asc)], + ], + ), + // ------- TEST CASE 8 ---------- + ( + // orderings + vec![ + // [b+d ASC] + vec![(&b_plus_d, option_asc)], + ], + // proj exprs + vec![ + (col_b, "b_new".to_string()), + (col_a, "a_new".to_string()), + (col_d, "d_new".to_string()), + (&b_plus_d, "b+d".to_string()), + ], + // expected + vec![ + // [b+d ASC] + vec![("b+d", option_asc)], + ], + ), + // ------- TEST CASE 9 ---------- + ( + // orderings + vec![ + // [a ASC, d ASC, b ASC] + vec![ + (col_a, option_asc), + (col_d, option_asc), + (col_b, option_asc), + ], + // [c ASC] + vec![(col_c, option_asc)], + ], + // proj exprs + vec![ + (col_b, "b_new".to_string()), + (col_a, "a_new".to_string()), + (col_d, "d_new".to_string()), + (col_c, "c_new".to_string()), + ], + // expected + vec![ + // [a_new ASC, d_new ASC, b_new ASC] + vec![ + ("a_new", option_asc), + ("d_new", option_asc), + ("b_new", option_asc), + ], + // [c_new ASC], + vec![("c_new", option_asc)], + ], + ), + // ------- TEST CASE 10 ---------- + ( + vec![ + // [a ASC, b ASC, c ASC] + vec![ + (col_a, option_asc), + (col_b, option_asc), + (col_c, option_asc), + ], + // [a ASC, d ASC] + vec![(col_a, option_asc), (col_d, option_asc)], + ], + // proj exprs + vec![ + (col_b, "b_new".to_string()), + (col_a, "a_new".to_string()), + (col_c, "c_new".to_string()), + (&c_plus_d, "c+d".to_string()), + ], + // expected + vec![ + // [a_new ASC, b_new ASC, c_new ASC] + vec![ + ("a_new", option_asc), + ("b_new", option_asc), + ("c_new", option_asc), + ], + // [a_new ASC, b_new ASC, c+d ASC] + vec![ + ("a_new", option_asc), + ("b_new", option_asc), + ("c+d", option_asc), + ], + ], + ), + // ------- TEST CASE 11 ---------- + ( + // orderings + vec![ + // [a ASC, b ASC] + vec![(col_a, option_asc), (col_b, option_asc)], + // [a ASC, d ASC] + vec![(col_a, option_asc), (col_d, option_asc)], + ], + // proj exprs + vec![ + (col_b, "b_new".to_string()), + (col_a, "a_new".to_string()), + (&b_plus_d, "b+d".to_string()), + ], + // expected + vec![ + // [a_new ASC, b_new ASC] + vec![("a_new", option_asc), ("b_new", option_asc)], + // [a_new ASC, b + d ASC] + vec![("a_new", option_asc), ("b+d", option_asc)], + ], + ), + // ------- TEST CASE 12 ---------- + ( + // orderings + vec![ + // [a ASC, b ASC, c ASC] + vec![ + (col_a, option_asc), + (col_b, option_asc), + (col_c, option_asc), + ], + ], + // proj exprs + vec![(col_c, "c_new".to_string()), (col_a, "a_new".to_string())], + // expected + vec![ + // [a_new ASC] + vec![("a_new", option_asc)], + ], + ), + // ------- TEST CASE 13 ---------- + ( + // orderings + vec![ + // [a ASC, b ASC, c ASC] + vec![ + (col_a, option_asc), + (col_b, option_asc), + (col_c, option_asc), + ], + // [a ASC, a + b ASC, c ASC] + vec![ + (col_a, option_asc), + (&a_plus_b, option_asc), + (col_c, option_asc), + ], + ], + // proj exprs + vec![ + (col_c, "c_new".to_string()), + (col_b, "b_new".to_string()), + (col_a, "a_new".to_string()), + (&a_plus_b, "a+b".to_string()), + ], + // expected + vec![ + // [a_new ASC, b_new ASC, c_new ASC] + vec![ + ("a_new", option_asc), + ("b_new", option_asc), + ("c_new", option_asc), + ], + // [a_new ASC, a+b ASC, c_new ASC] + vec![ + ("a_new", option_asc), + ("a+b", option_asc), + ("c_new", option_asc), + ], + ], + ), + // ------- TEST CASE 14 ---------- + ( + // orderings + vec![ + // [a ASC, b ASC] + vec![(col_a, option_asc), (col_b, option_asc)], + // [c ASC, b ASC] + vec![(col_c, option_asc), (col_b, option_asc)], + // [d ASC, e ASC] + vec![(col_d, option_asc), (col_e, option_asc)], + ], + // proj exprs + vec![ + (col_c, "c_new".to_string()), + (col_d, "d_new".to_string()), + (col_a, "a_new".to_string()), + (&b_plus_e, "b+e".to_string()), + ], + // expected + vec![ + // [a_new ASC, d_new ASC, b+e ASC] + vec![ + ("a_new", option_asc), + ("d_new", option_asc), + ("b+e", option_asc), + ], + // [d_new ASC, a_new ASC, b+e ASC] + vec![ + ("d_new", option_asc), + ("a_new", option_asc), + ("b+e", option_asc), + ], + // [c_new ASC, d_new ASC, b+e ASC] + vec![ + ("c_new", option_asc), + ("d_new", option_asc), + ("b+e", option_asc), + ], + // [d_new ASC, c_new ASC, b+e ASC] + vec![ + ("d_new", option_asc), + ("c_new", option_asc), + ("b+e", option_asc), + ], + ], + ), + // ------- TEST CASE 15 ---------- + ( + // orderings + vec![ + // [a ASC, c ASC, b ASC] + vec![ + (col_a, option_asc), + (col_c, option_asc), + (&col_b, option_asc), + ], + ], + // proj exprs + vec![ + (col_c, "c_new".to_string()), + (col_a, "a_new".to_string()), + (&a_plus_b, "a+b".to_string()), + ], + // expected + vec![ + // [a_new ASC, d_new ASC, b+e ASC] + vec![ + ("a_new", option_asc), + ("c_new", option_asc), + ("a+b", option_asc), + ], + ], + ), + // ------- TEST CASE 16 ---------- + ( + // orderings + vec![ + // [a ASC, b ASC] + vec![(col_a, option_asc), (col_b, option_asc)], + // [c ASC, b DESC] + vec![(col_c, option_asc), (col_b, option_desc)], + // [e ASC] + vec![(col_e, option_asc)], + ], + // proj exprs + vec![ + (col_c, "c_new".to_string()), + (col_a, "a_new".to_string()), + (col_b, "b_new".to_string()), + (&b_plus_e, "b+e".to_string()), + ], + // expected + vec![ + // [a_new ASC, b_new ASC] + vec![("a_new", option_asc), ("b_new", option_asc)], + // [a_new ASC, b_new ASC] + vec![("a_new", option_asc), ("b+e", option_asc)], + // [c_new ASC, b_new DESC] + vec![("c_new", option_asc), ("b_new", option_desc)], + ], + ), + ]; + + for (idx, (orderings, proj_exprs, expected)) in test_cases.into_iter().enumerate() + { + let mut eq_properties = EquivalenceProperties::new(schema.clone()); + + let orderings = convert_to_orderings(&orderings); + eq_properties.add_new_orderings(orderings); + + let proj_exprs = proj_exprs + .into_iter() + .map(|(expr, name)| (expr.clone(), name)) + .collect::>(); + let projection_mapping = ProjectionMapping::try_new(&proj_exprs, &schema)?; + let output_schema = output_schema(&projection_mapping, &schema)?; + + let expected = expected + .into_iter() + .map(|ordering| { + ordering + .into_iter() + .map(|(name, options)| { + (col(name, &output_schema).unwrap(), options) + }) + .collect::>() + }) + .collect::>(); + let expected = convert_to_orderings_owned(&expected); + + let projected_eq = eq_properties.project(&projection_mapping, output_schema); + let orderings = projected_eq.oeq_class(); + + let err_msg = format!( + "test_idx: {:?}, actual: {:?}, expected: {:?}, projection_mapping: {:?}", + idx, orderings.orderings, expected, projection_mapping + ); + + assert_eq!(orderings.len(), expected.len(), "{}", err_msg); + for expected_ordering in &expected { + assert!(orderings.contains(expected_ordering), "{}", err_msg) + } + } + + Ok(()) + } + + #[test] + fn project_orderings2() -> Result<()> { + let schema = Arc::new(Schema::new(vec![ + Field::new("a", DataType::Int32, true), + Field::new("b", DataType::Int32, true), + Field::new("c", DataType::Int32, true), + Field::new("d", DataType::Int32, true), + Field::new("ts", DataType::Timestamp(TimeUnit::Nanosecond, None), true), + ])); + let col_a = &col("a", &schema)?; + let col_b = &col("b", &schema)?; + let col_c = &col("c", &schema)?; + let col_ts = &col("ts", &schema)?; + let a_plus_b = Arc::new(BinaryExpr::new( + col_a.clone(), + Operator::Plus, + col_b.clone(), + )) as Arc; + let interval = Arc::new(Literal::new(ScalarValue::IntervalDayTime(Some(2)))) + as Arc; + let date_bin_ts = &create_physical_expr( + &BuiltinScalarFunction::DateBin, + &[interval, col_ts.clone()], + &schema, + &ExecutionProps::default(), + )?; + + let round_c = &create_physical_expr( + &BuiltinScalarFunction::Round, + &[col_c.clone()], + &schema, + &ExecutionProps::default(), + )?; + + let option_asc = SortOptions { + descending: false, + nulls_first: false, + }; + + let proj_exprs = vec![ + (col_b, "b_new".to_string()), + (col_a, "a_new".to_string()), + (col_c, "c_new".to_string()), + (date_bin_ts, "date_bin_res".to_string()), + (round_c, "round_c_res".to_string()), + ]; + let proj_exprs = proj_exprs + .into_iter() + .map(|(expr, name)| (expr.clone(), name)) + .collect::>(); + let projection_mapping = ProjectionMapping::try_new(&proj_exprs, &schema)?; + let output_schema = output_schema(&projection_mapping, &schema)?; + + let col_a_new = &col("a_new", &output_schema)?; + let col_b_new = &col("b_new", &output_schema)?; + let col_c_new = &col("c_new", &output_schema)?; + let col_date_bin_res = &col("date_bin_res", &output_schema)?; + let col_round_c_res = &col("round_c_res", &output_schema)?; + let a_new_plus_b_new = Arc::new(BinaryExpr::new( + col_a_new.clone(), + Operator::Plus, + col_b_new.clone(), + )) as Arc; + + let test_cases = vec![ + // ---------- TEST CASE 1 ------------ + ( + // orderings + vec![ + // [a ASC] + vec![(col_a, option_asc)], + ], + // expected + vec![ + // [b_new ASC] + vec![(col_a_new, option_asc)], + ], + ), + // ---------- TEST CASE 2 ------------ + ( + // orderings + vec![ + // [a+b ASC] + vec![(&a_plus_b, option_asc)], + ], + // expected + vec![ + // [b_new ASC] + vec![(&a_new_plus_b_new, option_asc)], + ], + ), + // ---------- TEST CASE 3 ------------ + ( + // orderings + vec![ + // [a ASC, ts ASC] + vec![(col_a, option_asc), (col_ts, option_asc)], + ], + // expected + vec![ + // [a_new ASC, date_bin_res ASC] + vec![(col_a_new, option_asc), (col_date_bin_res, option_asc)], + ], + ), + // ---------- TEST CASE 4 ------------ + ( + // orderings + vec![ + // [a ASC, ts ASC, b ASC] + vec![ + (col_a, option_asc), + (col_ts, option_asc), + (col_b, option_asc), + ], + ], + // expected + vec![ + // [a_new ASC, date_bin_res ASC] + // Please note that result is not [a_new ASC, date_bin_res ASC, b_new ASC] + // because, datebin_res may not be 1-1 function. Hence without introducing ts + // dependency we cannot guarantee any ordering after date_bin_res column. + vec![(col_a_new, option_asc), (col_date_bin_res, option_asc)], + ], + ), + // ---------- TEST CASE 5 ------------ + ( + // orderings + vec![ + // [a ASC, c ASC] + vec![(col_a, option_asc), (col_c, option_asc)], + ], + // expected + vec![ + // [a_new ASC, round_c_res ASC, c_new ASC] + vec![(col_a_new, option_asc), (col_round_c_res, option_asc)], + // [a_new ASC, c_new ASC] + vec![(col_a_new, option_asc), (col_c_new, option_asc)], + ], + ), + // ---------- TEST CASE 6 ------------ + ( + // orderings + vec![ + // [c ASC, b ASC] + vec![(col_c, option_asc), (col_b, option_asc)], + ], + // expected + vec![ + // [round_c_res ASC] + vec![(col_round_c_res, option_asc)], + // [c_new ASC, b_new ASC] + vec![(col_c_new, option_asc), (col_b_new, option_asc)], + ], + ), + // ---------- TEST CASE 7 ------------ + ( + // orderings + vec![ + // [a+b ASC, c ASC] + vec![(&a_plus_b, option_asc), (col_c, option_asc)], + ], + // expected + vec![ + // [a+b ASC, round(c) ASC, c_new ASC] + vec![ + (&a_new_plus_b_new, option_asc), + (&col_round_c_res, option_asc), + ], + // [a+b ASC, c_new ASC] + vec![(&a_new_plus_b_new, option_asc), (col_c_new, option_asc)], + ], + ), + ]; + + for (idx, (orderings, expected)) in test_cases.iter().enumerate() { + let mut eq_properties = EquivalenceProperties::new(schema.clone()); + + let orderings = convert_to_orderings(orderings); + eq_properties.add_new_orderings(orderings); + + let expected = convert_to_orderings(expected); + + let projected_eq = + eq_properties.project(&projection_mapping, output_schema.clone()); + let orderings = projected_eq.oeq_class(); + + let err_msg = format!( + "test idx: {:?}, actual: {:?}, expected: {:?}, projection_mapping: {:?}", + idx, orderings.orderings, expected, projection_mapping + ); + + assert_eq!(orderings.len(), expected.len(), "{}", err_msg); + for expected_ordering in &expected { + assert!(orderings.contains(expected_ordering), "{}", err_msg) + } + } + Ok(()) + } + + #[test] + fn project_orderings3() -> Result<()> { + let schema = Arc::new(Schema::new(vec![ + Field::new("a", DataType::Int32, true), + Field::new("b", DataType::Int32, true), + Field::new("c", DataType::Int32, true), + Field::new("d", DataType::Int32, true), + Field::new("e", DataType::Int32, true), + Field::new("f", DataType::Int32, true), + ])); + let col_a = &col("a", &schema)?; + let col_b = &col("b", &schema)?; + let col_c = &col("c", &schema)?; + let col_d = &col("d", &schema)?; + let col_e = &col("e", &schema)?; + let col_f = &col("f", &schema)?; + let a_plus_b = Arc::new(BinaryExpr::new( + col_a.clone(), + Operator::Plus, + col_b.clone(), + )) as Arc; + + let option_asc = SortOptions { + descending: false, + nulls_first: false, + }; + + let proj_exprs = vec![ + (col_c, "c_new".to_string()), + (col_d, "d_new".to_string()), + (&a_plus_b, "a+b".to_string()), + ]; + let proj_exprs = proj_exprs + .into_iter() + .map(|(expr, name)| (expr.clone(), name)) + .collect::>(); + let projection_mapping = ProjectionMapping::try_new(&proj_exprs, &schema)?; + let output_schema = output_schema(&projection_mapping, &schema)?; + + let col_a_plus_b_new = &col("a+b", &output_schema)?; + let col_c_new = &col("c_new", &output_schema)?; + let col_d_new = &col("d_new", &output_schema)?; + + let test_cases = vec![ + // ---------- TEST CASE 1 ------------ + ( + // orderings + vec![ + // [d ASC, b ASC] + vec![(col_d, option_asc), (col_b, option_asc)], + // [c ASC, a ASC] + vec![(col_c, option_asc), (col_a, option_asc)], + ], + // equal conditions + vec![], + // expected + vec![ + // [d_new ASC, c_new ASC, a+b ASC] + vec![ + (col_d_new, option_asc), + (col_c_new, option_asc), + (col_a_plus_b_new, option_asc), + ], + // [c_new ASC, d_new ASC, a+b ASC] + vec![ + (col_c_new, option_asc), + (col_d_new, option_asc), + (col_a_plus_b_new, option_asc), + ], + ], + ), + // ---------- TEST CASE 2 ------------ + ( + // orderings + vec![ + // [d ASC, b ASC] + vec![(col_d, option_asc), (col_b, option_asc)], + // [c ASC, e ASC], Please note that a=e + vec![(col_c, option_asc), (col_e, option_asc)], + ], + // equal conditions + vec![(col_e, col_a)], + // expected + vec![ + // [d_new ASC, c_new ASC, a+b ASC] + vec![ + (col_d_new, option_asc), + (col_c_new, option_asc), + (col_a_plus_b_new, option_asc), + ], + // [c_new ASC, d_new ASC, a+b ASC] + vec![ + (col_c_new, option_asc), + (col_d_new, option_asc), + (col_a_plus_b_new, option_asc), + ], + ], + ), + // ---------- TEST CASE 3 ------------ + ( + // orderings + vec![ + // [d ASC, b ASC] + vec![(col_d, option_asc), (col_b, option_asc)], + // [c ASC, e ASC], Please note that a=f + vec![(col_c, option_asc), (col_e, option_asc)], + ], + // equal conditions + vec![(col_a, col_f)], + // expected + vec![ + // [d_new ASC] + vec![(col_d_new, option_asc)], + // [c_new ASC] + vec![(col_c_new, option_asc)], + ], + ), + ]; + for (orderings, equal_columns, expected) in test_cases { + let mut eq_properties = EquivalenceProperties::new(schema.clone()); + for (lhs, rhs) in equal_columns { + eq_properties.add_equal_conditions(lhs, rhs); + } + + let orderings = convert_to_orderings(&orderings); + eq_properties.add_new_orderings(orderings); + + let expected = convert_to_orderings(&expected); + + let projected_eq = + eq_properties.project(&projection_mapping, output_schema.clone()); + let orderings = projected_eq.oeq_class(); + + let err_msg = format!( + "actual: {:?}, expected: {:?}, projection_mapping: {:?}", + orderings.orderings, expected, projection_mapping + ); + + assert_eq!(orderings.len(), expected.len(), "{}", err_msg); + for expected_ordering in &expected { + assert!(orderings.contains(expected_ordering), "{}", err_msg) + } + } + + Ok(()) + } + + #[test] + fn project_orderings_random() -> Result<()> { + const N_RANDOM_SCHEMA: usize = 20; + const N_ELEMENTS: usize = 125; + const N_DISTINCT: usize = 5; + + for seed in 0..N_RANDOM_SCHEMA { + // Create a random schema with random properties + let (test_schema, eq_properties) = create_random_schema(seed as u64)?; + // Generate a data that satisfies properties given + let table_data_with_properties = + generate_table_for_eq_properties(&eq_properties, N_ELEMENTS, N_DISTINCT)?; + // Floor(a) + let floor_a = create_physical_expr( + &BuiltinScalarFunction::Floor, + &[col("a", &test_schema)?], + &test_schema, + &ExecutionProps::default(), + )?; + // a + b + let a_plus_b = Arc::new(BinaryExpr::new( + col("a", &test_schema)?, + Operator::Plus, + col("b", &test_schema)?, + )) as Arc; + let proj_exprs = vec![ + (col("a", &test_schema)?, "a_new"), + (col("b", &test_schema)?, "b_new"), + (col("c", &test_schema)?, "c_new"), + (col("d", &test_schema)?, "d_new"), + (col("e", &test_schema)?, "e_new"), + (col("f", &test_schema)?, "f_new"), + (floor_a, "floor(a)"), + (a_plus_b, "a+b"), + ]; + + for n_req in 0..=proj_exprs.len() { + for proj_exprs in proj_exprs.iter().combinations(n_req) { + let proj_exprs = proj_exprs + .into_iter() + .map(|(expr, name)| (expr.clone(), name.to_string())) + .collect::>(); + let (projected_batch, projected_eq) = apply_projection( + proj_exprs.clone(), + &table_data_with_properties, + &eq_properties, + )?; + + // Make sure each ordering after projection is valid. + for ordering in projected_eq.oeq_class().iter() { + let err_msg = format!( + "Error in test case ordering:{:?}, eq_properties.oeq_class: {:?}, eq_properties.eq_group: {:?}, eq_properties.constants: {:?}, proj_exprs: {:?}", + ordering, eq_properties.oeq_class, eq_properties.eq_group, eq_properties.constants, proj_exprs + ); + // Since ordered section satisfies schema, we expect + // that result will be same after sort (e.g sort was unnecessary). + assert!( + is_table_same_after_sort( + ordering.clone(), + projected_batch.clone(), + )?, + "{}", + err_msg + ); + } + } + } + } + + Ok(()) + } + + #[test] + fn ordering_satisfy_after_projection_random() -> Result<()> { + const N_RANDOM_SCHEMA: usize = 20; + const N_ELEMENTS: usize = 125; + const N_DISTINCT: usize = 5; + const SORT_OPTIONS: SortOptions = SortOptions { + descending: false, + nulls_first: false, + }; + + for seed in 0..N_RANDOM_SCHEMA { + // Create a random schema with random properties + let (test_schema, eq_properties) = create_random_schema(seed as u64)?; + // Generate a data that satisfies properties given + let table_data_with_properties = + generate_table_for_eq_properties(&eq_properties, N_ELEMENTS, N_DISTINCT)?; + // Floor(a) + let floor_a = create_physical_expr( + &BuiltinScalarFunction::Floor, + &[col("a", &test_schema)?], + &test_schema, + &ExecutionProps::default(), + )?; + // a + b + let a_plus_b = Arc::new(BinaryExpr::new( + col("a", &test_schema)?, + Operator::Plus, + col("b", &test_schema)?, + )) as Arc; + let proj_exprs = vec![ + (col("a", &test_schema)?, "a_new"), + (col("b", &test_schema)?, "b_new"), + (col("c", &test_schema)?, "c_new"), + (col("d", &test_schema)?, "d_new"), + (col("e", &test_schema)?, "e_new"), + (col("f", &test_schema)?, "f_new"), + (floor_a, "floor(a)"), + (a_plus_b, "a+b"), + ]; + + for n_req in 0..=proj_exprs.len() { + for proj_exprs in proj_exprs.iter().combinations(n_req) { + let proj_exprs = proj_exprs + .into_iter() + .map(|(expr, name)| (expr.clone(), name.to_string())) + .collect::>(); + let (projected_batch, projected_eq) = apply_projection( + proj_exprs.clone(), + &table_data_with_properties, + &eq_properties, + )?; + + let projection_mapping = + ProjectionMapping::try_new(&proj_exprs, &test_schema)?; + + let projected_exprs = projection_mapping + .iter() + .map(|(_source, target)| target.clone()) + .collect::>(); + + for n_req in 0..=projected_exprs.len() { + for exprs in projected_exprs.iter().combinations(n_req) { + let requirement = exprs + .into_iter() + .map(|expr| PhysicalSortExpr { + expr: expr.clone(), + options: SORT_OPTIONS, + }) + .collect::>(); + let expected = is_table_same_after_sort( + requirement.clone(), + projected_batch.clone(), + )?; + let err_msg = format!( + "Error in test case requirement:{:?}, expected: {:?}, eq_properties.oeq_class: {:?}, eq_properties.eq_group: {:?}, eq_properties.constants: {:?}, projected_eq.oeq_class: {:?}, projected_eq.eq_group: {:?}, projected_eq.constants: {:?}, projection_mapping: {:?}", + requirement, expected, eq_properties.oeq_class, eq_properties.eq_group, eq_properties.constants, projected_eq.oeq_class, projected_eq.eq_group, projected_eq.constants, projection_mapping + ); + // Check whether ordering_satisfy API result and + // experimental result matches. + assert_eq!( + projected_eq.ordering_satisfy(&requirement), + expected, + "{}", + err_msg + ); + } + } + } + } + } + + Ok(()) + } + + #[test] + fn test_expr_consists_of_constants() -> Result<()> { + let schema = Arc::new(Schema::new(vec![ + Field::new("a", DataType::Int32, true), + Field::new("b", DataType::Int32, true), + Field::new("c", DataType::Int32, true), + Field::new("d", DataType::Int32, true), + Field::new("ts", DataType::Timestamp(TimeUnit::Nanosecond, None), true), + ])); + let col_a = col("a", &schema)?; + let col_b = col("b", &schema)?; + let col_d = col("d", &schema)?; + let b_plus_d = Arc::new(BinaryExpr::new( + col_b.clone(), + Operator::Plus, + col_d.clone(), + )) as Arc; + + let constants = vec![col_a.clone(), col_b.clone()]; + let expr = b_plus_d.clone(); + assert!(!is_constant_recurse(&constants, &expr)); + + let constants = vec![col_a.clone(), col_b.clone(), col_d.clone()]; + let expr = b_plus_d.clone(); + assert!(is_constant_recurse(&constants, &expr)); + Ok(()) + } + + #[test] + fn test_join_equivalence_properties() -> Result<()> { + let schema = create_test_schema()?; + let col_a = &col("a", &schema)?; + let col_b = &col("b", &schema)?; + let col_c = &col("c", &schema)?; + let offset = schema.fields.len(); + let col_a2 = &add_offset_to_expr(col_a.clone(), offset); + let col_b2 = &add_offset_to_expr(col_b.clone(), offset); + let option_asc = SortOptions { + descending: false, + nulls_first: false, + }; + let test_cases = vec![ + // ------- TEST CASE 1 -------- + // [a ASC], [b ASC] + ( + // [a ASC], [b ASC] + vec![vec![(col_a, option_asc)], vec![(col_b, option_asc)]], + // [a ASC], [b ASC] + vec![vec![(col_a, option_asc)], vec![(col_b, option_asc)]], + // expected [a ASC, a2 ASC], [a ASC, b2 ASC], [b ASC, a2 ASC], [b ASC, b2 ASC] + vec![ + vec![(col_a, option_asc), (col_a2, option_asc)], + vec![(col_a, option_asc), (col_b2, option_asc)], + vec![(col_b, option_asc), (col_a2, option_asc)], + vec![(col_b, option_asc), (col_b2, option_asc)], + ], + ), + // ------- TEST CASE 2 -------- + // [a ASC], [b ASC] + ( + // [a ASC], [b ASC], [c ASC] + vec![ + vec![(col_a, option_asc)], + vec![(col_b, option_asc)], + vec![(col_c, option_asc)], + ], + // [a ASC], [b ASC] + vec![vec![(col_a, option_asc)], vec![(col_b, option_asc)]], + // expected [a ASC, a2 ASC], [a ASC, b2 ASC], [b ASC, a2 ASC], [b ASC, b2 ASC], [c ASC, a2 ASC], [c ASC, b2 ASC] + vec![ + vec![(col_a, option_asc), (col_a2, option_asc)], + vec![(col_a, option_asc), (col_b2, option_asc)], + vec![(col_b, option_asc), (col_a2, option_asc)], + vec![(col_b, option_asc), (col_b2, option_asc)], + vec![(col_c, option_asc), (col_a2, option_asc)], + vec![(col_c, option_asc), (col_b2, option_asc)], + ], + ), + ]; + for (left_orderings, right_orderings, expected) in test_cases { + let mut left_eq_properties = EquivalenceProperties::new(schema.clone()); + let mut right_eq_properties = EquivalenceProperties::new(schema.clone()); + let left_orderings = convert_to_orderings(&left_orderings); + let right_orderings = convert_to_orderings(&right_orderings); + let expected = convert_to_orderings(&expected); + left_eq_properties.add_new_orderings(left_orderings); + right_eq_properties.add_new_orderings(right_orderings); + let join_eq = join_equivalence_properties( + left_eq_properties, + right_eq_properties, + &JoinType::Inner, + Arc::new(Schema::empty()), + &[true, false], + Some(JoinSide::Left), + &[], + ); + let orderings = &join_eq.oeq_class.orderings; + let err_msg = format!("expected: {:?}, actual:{:?}", expected, orderings); + assert_eq!( + join_eq.oeq_class.orderings.len(), + expected.len(), + "{}", + err_msg + ); + for ordering in orderings { + assert!( + expected.contains(ordering), + "{}, ordering: {:?}", + err_msg, + ordering + ); + } + } Ok(()) } } From 11542740a982d3aba36f43d93967a58bd9ab8d9b Mon Sep 17 00:00:00 2001 From: jokercurry <982458633@qq.com> Date: Tue, 12 Dec 2023 21:23:08 +0800 Subject: [PATCH 410/572] Add `today` alias for `current_date` (#8423) * fix conflict * add md * add test * add test * addr comments * Update datafusion/sqllogictest/test_files/timestamps.slt Co-authored-by: Alex Huang --------- Co-authored-by: zhongjingxiong Co-authored-by: Alex Huang --- datafusion/expr/src/built_in_function.rs | 2 +- .../sqllogictest/test_files/timestamps.slt | 24 +++++++++++++++++++ .../source/user-guide/sql/scalar_functions.md | 9 +++++++ 3 files changed, 34 insertions(+), 1 deletion(-) diff --git a/datafusion/expr/src/built_in_function.rs b/datafusion/expr/src/built_in_function.rs index 5a903a73adc6..fd899289ac82 100644 --- a/datafusion/expr/src/built_in_function.rs +++ b/datafusion/expr/src/built_in_function.rs @@ -1532,7 +1532,7 @@ impl BuiltinScalarFunction { // time/date functions BuiltinScalarFunction::Now => &["now"], - BuiltinScalarFunction::CurrentDate => &["current_date"], + BuiltinScalarFunction::CurrentDate => &["current_date", "today"], BuiltinScalarFunction::CurrentTime => &["current_time"], BuiltinScalarFunction::DateBin => &["date_bin"], BuiltinScalarFunction::DateTrunc => &["date_trunc", "datetrunc"], diff --git a/datafusion/sqllogictest/test_files/timestamps.slt b/datafusion/sqllogictest/test_files/timestamps.slt index 71b6ddf33f39..f956d59b1da0 100644 --- a/datafusion/sqllogictest/test_files/timestamps.slt +++ b/datafusion/sqllogictest/test_files/timestamps.slt @@ -46,6 +46,30 @@ statement ok create table ts_data_secs as select arrow_cast(ts / 1000000000, 'Timestamp(Second, None)') as ts, value from ts_data; +########## +## Current date Tests +########## + +query B +select cast(now() as date) = current_date(); +---- +true + +query B +select now() = current_date(); +---- +false + +query B +select current_date() = today(); +---- +true + +query B +select cast(now() as date) = today(); +---- +true + ########## ## Timestamp Handling Tests diff --git a/docs/source/user-guide/sql/scalar_functions.md b/docs/source/user-guide/sql/scalar_functions.md index 9a9bec9df77b..ad4c6ed083bf 100644 --- a/docs/source/user-guide/sql/scalar_functions.md +++ b/docs/source/user-guide/sql/scalar_functions.md @@ -1280,6 +1280,7 @@ regexp_replace(str, regexp, replacement, flags) - [datepart](#datepart) - [extract](#extract) - [to_timestamp](#to_timestamp) +- [today](#today) - [to_timestamp_millis](#to_timestamp_millis) - [to_timestamp_micros](#to_timestamp_micros) - [to_timestamp_seconds](#to_timestamp_seconds) @@ -1308,6 +1309,14 @@ no matter when in the query plan the function executes. current_date() ``` +#### Aliases + +- today + +### `today` + +_Alias of [current_date](#current_date)._ + ### `current_time` Returns the current UTC time. From 2102275f64012ec6eff4dd9c72eb87cc76921d74 Mon Sep 17 00:00:00 2001 From: Alex Huang Date: Tue, 12 Dec 2023 14:30:29 +0100 Subject: [PATCH 411/572] remove useless clone (#8495) --- datafusion/physical-expr/src/array_expressions.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/datafusion/physical-expr/src/array_expressions.rs b/datafusion/physical-expr/src/array_expressions.rs index c2dc88b10773..7fa97dad7aa6 100644 --- a/datafusion/physical-expr/src/array_expressions.rs +++ b/datafusion/physical-expr/src/array_expressions.rs @@ -2138,7 +2138,7 @@ pub fn general_array_distinct( let mut offsets = Vec::with_capacity(array.len()); offsets.push(OffsetSize::usize_as(0)); let mut new_arrays = Vec::with_capacity(array.len()); - let converter = RowConverter::new(vec![SortField::new(dt.clone())])?; + let converter = RowConverter::new(vec![SortField::new(dt)])?; // distinct for each list in ListArray for arr in array.iter().flatten() { let values = converter.convert_columns(&[arr])?; From 7f312c8a1d82b19f03c640e4a5d7d6437b2ee835 Mon Sep 17 00:00:00 2001 From: Huaijin Date: Tue, 12 Dec 2023 21:36:23 +0800 Subject: [PATCH 412/572] fix: incorrect set preserve_partitioning in SortExec (#8485) * fix: incorrect set preserve_partitioning in SortExec * add more comment --- .../physical_optimizer/projection_pushdown.rs | 3 +- datafusion/sqllogictest/test_files/join.slt | 37 +++++++++++++++++++ 2 files changed, 39 insertions(+), 1 deletion(-) diff --git a/datafusion/core/src/physical_optimizer/projection_pushdown.rs b/datafusion/core/src/physical_optimizer/projection_pushdown.rs index 67a2eaf0d9b3..664afbe822ff 100644 --- a/datafusion/core/src/physical_optimizer/projection_pushdown.rs +++ b/datafusion/core/src/physical_optimizer/projection_pushdown.rs @@ -454,7 +454,8 @@ fn try_swapping_with_sort( Ok(Some(Arc::new( SortExec::new(updated_exprs, make_with_child(projection, sort.input())?) - .with_fetch(sort.fetch()), + .with_fetch(sort.fetch()) + .with_preserve_partitioning(sort.preserve_partitioning()), ))) } diff --git a/datafusion/sqllogictest/test_files/join.slt b/datafusion/sqllogictest/test_files/join.slt index 386ffe766b19..c9dd7ca604ad 100644 --- a/datafusion/sqllogictest/test_files/join.slt +++ b/datafusion/sqllogictest/test_files/join.slt @@ -594,3 +594,40 @@ drop table IF EXISTS full_join_test; # batch size statement ok set datafusion.execution.batch_size = 8192; + +# related to: https://github.com/apache/arrow-datafusion/issues/8374 +statement ok +CREATE TABLE t1(a text, b int) AS VALUES ('Alice', 50), ('Alice', 100); + +statement ok +CREATE TABLE t2(a text, b int) AS VALUES ('Alice', 2), ('Alice', 1); + +# the current query results are incorrect, becuase the query was incorrectly rewritten as: +# SELECT t1.a, t1.b FROM t1 JOIN t2 ON t1.a = t2.a ORDER BY t1.a, t1.b; +# the difference is ORDER BY clause rewrite from t2.b to t1.b, it is incorrect. +# after https://github.com/apache/arrow-datafusion/issues/8374 fixed, the correct result should be: +# Alice 50 +# Alice 100 +# Alice 50 +# Alice 100 +query TI +SELECT t1.a, t1.b FROM t1 JOIN t2 ON t1.a = t2.a ORDER BY t1.a, t2.b; +---- +Alice 50 +Alice 50 +Alice 100 +Alice 100 + +query TITI +SELECT t1.a, t1.b, t2.a, t2.b FROM t1 JOIN t2 ON t1.a = t2.a ORDER BY t1.a, t2.b; +---- +Alice 50 Alice 1 +Alice 100 Alice 1 +Alice 50 Alice 2 +Alice 100 Alice 2 + +statement ok +DROP TABLE t1; + +statement ok +DROP TABLE t2; From 2919e32b8ce9e594f83a0a1268cc13a121a7963c Mon Sep 17 00:00:00 2001 From: Dennis Liu Date: Wed, 13 Dec 2023 05:01:07 +0800 Subject: [PATCH 413/572] Explicitly mark parquet for tests in datafusion-common (#8497) * Add cfg for test and reference requiring parquet * Check datafusion-common can be compiled without parquet * Update rust.yml * Remove unnecessary space --- .github/workflows/rust.yml | 3 +++ datafusion/common/src/file_options/file_type.rs | 1 + datafusion/common/src/file_options/mod.rs | 8 ++++++++ datafusion/common/src/test_util.rs | 1 + 4 files changed, 13 insertions(+) diff --git a/.github/workflows/rust.yml b/.github/workflows/rust.yml index 099aab061435..a541091e3a2b 100644 --- a/.github/workflows/rust.yml +++ b/.github/workflows/rust.yml @@ -68,6 +68,9 @@ jobs: - name: Check workspace without default features run: cargo check --no-default-features -p datafusion + - name: Check datafusion-common without default features + run: cargo check --tests --no-default-features -p datafusion-common + - name: Check workspace in debug mode run: cargo check diff --git a/datafusion/common/src/file_options/file_type.rs b/datafusion/common/src/file_options/file_type.rs index a07f2e0cb847..b1d61b1a2567 100644 --- a/datafusion/common/src/file_options/file_type.rs +++ b/datafusion/common/src/file_options/file_type.rs @@ -109,6 +109,7 @@ mod tests { use std::str::FromStr; #[test] + #[cfg(feature = "parquet")] fn from_str() { for (ext, file_type) in [ ("csv", FileType::CSV), diff --git a/datafusion/common/src/file_options/mod.rs b/datafusion/common/src/file_options/mod.rs index b7c1341e3046..f0e49dd85597 100644 --- a/datafusion/common/src/file_options/mod.rs +++ b/datafusion/common/src/file_options/mod.rs @@ -299,6 +299,7 @@ impl Display for FileTypeWriterOptions { mod tests { use std::collections::HashMap; + #[cfg(feature = "parquet")] use parquet::{ basic::{Compression, Encoding, ZstdLevel}, file::properties::{EnabledStatistics, WriterVersion}, @@ -313,9 +314,11 @@ mod tests { use crate::Result; + #[cfg(feature = "parquet")] use super::{parquet_writer::ParquetWriterOptions, StatementOptions}; #[test] + #[cfg(feature = "parquet")] fn test_writeroptions_parquet_from_statement_options() -> Result<()> { let mut option_map: HashMap = HashMap::new(); option_map.insert("max_row_group_size".to_owned(), "123".to_owned()); @@ -386,6 +389,7 @@ mod tests { } #[test] + #[cfg(feature = "parquet")] fn test_writeroptions_parquet_column_specific() -> Result<()> { let mut option_map: HashMap = HashMap::new(); @@ -506,6 +510,8 @@ mod tests { } #[test] + // for StatementOptions + #[cfg(feature = "parquet")] fn test_writeroptions_csv_from_statement_options() -> Result<()> { let mut option_map: HashMap = HashMap::new(); option_map.insert("header".to_owned(), "true".to_owned()); @@ -533,6 +539,8 @@ mod tests { } #[test] + // for StatementOptions + #[cfg(feature = "parquet")] fn test_writeroptions_json_from_statement_options() -> Result<()> { let mut option_map: HashMap = HashMap::new(); option_map.insert("compression".to_owned(), "gzip".to_owned()); diff --git a/datafusion/common/src/test_util.rs b/datafusion/common/src/test_util.rs index 9a4433782157..eeace97eebfa 100644 --- a/datafusion/common/src/test_util.rs +++ b/datafusion/common/src/test_util.rs @@ -285,6 +285,7 @@ mod tests { } #[test] + #[cfg(feature = "parquet")] fn test_happy() { let res = arrow_test_data(); assert!(PathBuf::from(res).is_dir()); From 861cc36eb66243079e0171637e65adccd5956c94 Mon Sep 17 00:00:00 2001 From: Devin D'Angelo Date: Tue, 12 Dec 2023 16:02:53 -0500 Subject: [PATCH 414/572] Minor/Doc: Clarify DataFrame::write_table Documentation (#8519) * update write_table doc * fmt --- datafusion/core/src/dataframe/mod.rs | 13 +++++++++---- 1 file changed, 9 insertions(+), 4 deletions(-) diff --git a/datafusion/core/src/dataframe/mod.rs b/datafusion/core/src/dataframe/mod.rs index c40dd522a457..3e286110f7f9 100644 --- a/datafusion/core/src/dataframe/mod.rs +++ b/datafusion/core/src/dataframe/mod.rs @@ -1013,11 +1013,16 @@ impl DataFrame { )) } - /// Write this DataFrame to the referenced table + /// Write this DataFrame to the referenced table by name. /// This method uses on the same underlying implementation - /// as the SQL Insert Into statement. - /// Unlike most other DataFrame methods, this method executes - /// eagerly, writing data, and returning the count of rows written. + /// as the SQL Insert Into statement. Unlike most other DataFrame methods, + /// this method executes eagerly. Data is written to the table using an + /// execution plan returned by the [TableProvider]'s insert_into method. + /// Refer to the documentation of the specific [TableProvider] to determine + /// the expected data returned by the insert_into plan via this method. + /// For the built in ListingTable provider, a single [RecordBatch] containing + /// a single column and row representing the count of total rows written + /// is returned. pub async fn write_table( self, table_name: &str, From 500ab40888a855bd021c302af9b27319a98ebc54 Mon Sep 17 00:00:00 2001 From: Vaibhav Rabber Date: Wed, 13 Dec 2023 03:37:53 +0530 Subject: [PATCH 415/572] fix: Pull stats in `IdentVisitor`/`GraphvizVisitor` only when requested (#8514) Signed-off-by: Vaibhav --- datafusion/physical-plan/src/display.rs | 127 +++++++++++++++++++++++- 1 file changed, 125 insertions(+), 2 deletions(-) diff --git a/datafusion/physical-plan/src/display.rs b/datafusion/physical-plan/src/display.rs index 612e164be0e2..19c2847b09dc 100644 --- a/datafusion/physical-plan/src/display.rs +++ b/datafusion/physical-plan/src/display.rs @@ -260,8 +260,8 @@ impl<'a, 'b> ExecutionPlanVisitor for IndentVisitor<'a, 'b> { } } } - let stats = plan.statistics().map_err(|_e| fmt::Error)?; if self.show_statistics { + let stats = plan.statistics().map_err(|_e| fmt::Error)?; write!(self.f, ", statistics=[{}]", stats)?; } writeln!(self.f)?; @@ -341,8 +341,8 @@ impl ExecutionPlanVisitor for GraphvizVisitor<'_, '_> { } }; - let stats = plan.statistics().map_err(|_e| fmt::Error)?; let statistics = if self.show_statistics { + let stats = plan.statistics().map_err(|_e| fmt::Error)?; format!("statistics=[{}]", stats) } else { "".to_string() @@ -436,3 +436,126 @@ impl<'a> fmt::Display for OutputOrderingDisplay<'a> { write!(f, "]") } } + +#[cfg(test)] +mod tests { + use std::fmt::Write; + use std::sync::Arc; + + use datafusion_common::DataFusionError; + + use crate::{DisplayAs, ExecutionPlan}; + + use super::DisplayableExecutionPlan; + + #[derive(Debug, Clone, Copy)] + enum TestStatsExecPlan { + Panic, + Error, + Ok, + } + + impl DisplayAs for TestStatsExecPlan { + fn fmt_as( + &self, + _t: crate::DisplayFormatType, + f: &mut std::fmt::Formatter, + ) -> std::fmt::Result { + write!(f, "TestStatsExecPlan") + } + } + + impl ExecutionPlan for TestStatsExecPlan { + fn as_any(&self) -> &dyn std::any::Any { + self + } + + fn schema(&self) -> arrow_schema::SchemaRef { + Arc::new(arrow_schema::Schema::empty()) + } + + fn output_partitioning(&self) -> datafusion_physical_expr::Partitioning { + datafusion_physical_expr::Partitioning::UnknownPartitioning(1) + } + + fn output_ordering( + &self, + ) -> Option<&[datafusion_physical_expr::PhysicalSortExpr]> { + None + } + + fn children(&self) -> Vec> { + vec![] + } + + fn with_new_children( + self: Arc, + _: Vec>, + ) -> datafusion_common::Result> { + unimplemented!() + } + + fn execute( + &self, + _: usize, + _: Arc, + ) -> datafusion_common::Result + { + todo!() + } + + fn statistics(&self) -> datafusion_common::Result { + match self { + Self::Panic => panic!("expected panic"), + Self::Error => { + Err(DataFusionError::Internal("expected error".to_string())) + } + Self::Ok => Ok(datafusion_common::Statistics::new_unknown( + self.schema().as_ref(), + )), + } + } + } + + fn test_stats_display(exec: TestStatsExecPlan, show_stats: bool) { + let display = + DisplayableExecutionPlan::new(&exec).set_show_statistics(show_stats); + + let mut buf = String::new(); + write!(&mut buf, "{}", display.one_line()).unwrap(); + let buf = buf.trim(); + assert_eq!(buf, "TestStatsExecPlan"); + } + + #[test] + fn test_display_when_stats_panic_with_no_show_stats() { + test_stats_display(TestStatsExecPlan::Panic, false); + } + + #[test] + fn test_display_when_stats_error_with_no_show_stats() { + test_stats_display(TestStatsExecPlan::Error, false); + } + + #[test] + fn test_display_when_stats_ok_with_no_show_stats() { + test_stats_display(TestStatsExecPlan::Ok, false); + } + + #[test] + #[should_panic(expected = "expected panic")] + fn test_display_when_stats_panic_with_show_stats() { + test_stats_display(TestStatsExecPlan::Panic, true); + } + + #[test] + #[should_panic(expected = "Error")] // fmt::Error + fn test_display_when_stats_error_with_show_stats() { + test_stats_display(TestStatsExecPlan::Error, true); + } + + #[test] + fn test_display_when_stats_ok_with_show_stats() { + test_stats_display(TestStatsExecPlan::Ok, false); + } +} From 4578f3daeefd84305d3f055c243827856e2c036c Mon Sep 17 00:00:00 2001 From: Jacob Ogle <123908271+JacobOgle@users.noreply.github.com> Date: Wed, 13 Dec 2023 09:11:29 -0500 Subject: [PATCH 416/572] Change display of RepartitionExec from SortPreservingRepartitionExec to RepartitionExec preserve_order=true (#8521) * Change display of RepartitionExec from SortPreservingRepartitionExec to RepartitionExec preserve_order=true #8129 * fix mod.rs with cargo fmt * test fix for repartition::test::test_preserve_order --- .../enforce_distribution.rs | 38 +++++++++---------- .../src/physical_optimizer/enforce_sorting.rs | 6 +-- .../replace_with_order_preserving_variants.rs | 22 +++++------ .../physical-plan/src/repartition/mod.rs | 12 +++--- .../sqllogictest/test_files/groupby.slt | 2 +- datafusion/sqllogictest/test_files/joins.slt | 2 +- datafusion/sqllogictest/test_files/window.slt | 10 ++--- 7 files changed, 46 insertions(+), 46 deletions(-) diff --git a/datafusion/core/src/physical_optimizer/enforce_distribution.rs b/datafusion/core/src/physical_optimizer/enforce_distribution.rs index 3aed6555f305..93cdbf858367 100644 --- a/datafusion/core/src/physical_optimizer/enforce_distribution.rs +++ b/datafusion/core/src/physical_optimizer/enforce_distribution.rs @@ -1129,8 +1129,8 @@ fn replace_order_preserving_variants( /// Assume that following plan is given: /// ```text /// "SortPreservingMergeExec: \[a@0 ASC]" -/// " SortPreservingRepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=10", -/// " SortPreservingRepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=2", +/// " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=10, preserve_order=true", +/// " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=2, preserve_order=true", /// " ParquetExec: file_groups={2 groups: \[\[x], \[y]]}, projection=\[a, b, c, d, e], output_ordering=\[a@0 ASC]", /// ``` /// @@ -3015,16 +3015,16 @@ pub(crate) mod tests { vec![ top_join_plan.as_str(), join_plan.as_str(), - "SortPreservingRepartitionExec: partitioning=Hash([a@0], 10), input_partitions=10, sort_exprs=a@0 ASC", + "RepartitionExec: partitioning=Hash([a@0], 10), input_partitions=10, preserve_order=true, sort_exprs=a@0 ASC", "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", "SortExec: expr=[a@0 ASC]", "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", - "SortPreservingRepartitionExec: partitioning=Hash([b1@1], 10), input_partitions=10, sort_exprs=b1@1 ASC", + "RepartitionExec: partitioning=Hash([b1@1], 10), input_partitions=10, preserve_order=true, sort_exprs=b1@1 ASC", "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", "SortExec: expr=[b1@1 ASC]", "ProjectionExec: expr=[a@0 as a1, b@1 as b1, c@2 as c1, d@3 as d1, e@4 as e1]", "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", - "SortPreservingRepartitionExec: partitioning=Hash([c@2], 10), input_partitions=10, sort_exprs=c@2 ASC", + "RepartitionExec: partitioning=Hash([c@2], 10), input_partitions=10, preserve_order=true, sort_exprs=c@2 ASC", "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", "SortExec: expr=[c@2 ASC]", "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", @@ -3041,21 +3041,21 @@ pub(crate) mod tests { _ => vec![ top_join_plan.as_str(), // Below 4 operators are differences introduced, when join mode is changed - "SortPreservingRepartitionExec: partitioning=Hash([a@0], 10), input_partitions=10, sort_exprs=a@0 ASC", + "RepartitionExec: partitioning=Hash([a@0], 10), input_partitions=10, preserve_order=true, sort_exprs=a@0 ASC", "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", "SortExec: expr=[a@0 ASC]", "CoalescePartitionsExec", join_plan.as_str(), - "SortPreservingRepartitionExec: partitioning=Hash([a@0], 10), input_partitions=10, sort_exprs=a@0 ASC", + "RepartitionExec: partitioning=Hash([a@0], 10), input_partitions=10, preserve_order=true, sort_exprs=a@0 ASC", "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", "SortExec: expr=[a@0 ASC]", "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", - "SortPreservingRepartitionExec: partitioning=Hash([b1@1], 10), input_partitions=10, sort_exprs=b1@1 ASC", + "RepartitionExec: partitioning=Hash([b1@1], 10), input_partitions=10, preserve_order=true, sort_exprs=b1@1 ASC", "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", "SortExec: expr=[b1@1 ASC]", "ProjectionExec: expr=[a@0 as a1, b@1 as b1, c@2 as c1, d@3 as d1, e@4 as e1]", "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", - "SortPreservingRepartitionExec: partitioning=Hash([c@2], 10), input_partitions=10, sort_exprs=c@2 ASC", + "RepartitionExec: partitioning=Hash([c@2], 10), input_partitions=10, preserve_order=true, sort_exprs=c@2 ASC", "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", "SortExec: expr=[c@2 ASC]", "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", @@ -3129,16 +3129,16 @@ pub(crate) mod tests { JoinType::Inner | JoinType::Right => vec![ top_join_plan.as_str(), join_plan.as_str(), - "SortPreservingRepartitionExec: partitioning=Hash([a@0], 10), input_partitions=10, sort_exprs=a@0 ASC", + "RepartitionExec: partitioning=Hash([a@0], 10), input_partitions=10, preserve_order=true, sort_exprs=a@0 ASC", "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", "SortExec: expr=[a@0 ASC]", "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", - "SortPreservingRepartitionExec: partitioning=Hash([b1@1], 10), input_partitions=10, sort_exprs=b1@1 ASC", + "RepartitionExec: partitioning=Hash([b1@1], 10), input_partitions=10, preserve_order=true, sort_exprs=b1@1 ASC", "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", "SortExec: expr=[b1@1 ASC]", "ProjectionExec: expr=[a@0 as a1, b@1 as b1, c@2 as c1, d@3 as d1, e@4 as e1]", "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", - "SortPreservingRepartitionExec: partitioning=Hash([c@2], 10), input_partitions=10, sort_exprs=c@2 ASC", + "RepartitionExec: partitioning=Hash([c@2], 10), input_partitions=10, preserve_order=true, sort_exprs=c@2 ASC", "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", "SortExec: expr=[c@2 ASC]", "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", @@ -3146,21 +3146,21 @@ pub(crate) mod tests { // Should include 8 RepartitionExecs (4 of them preserves order) and 4 SortExecs JoinType::Left | JoinType::Full => vec![ top_join_plan.as_str(), - "SortPreservingRepartitionExec: partitioning=Hash([b1@6], 10), input_partitions=10, sort_exprs=b1@6 ASC", + "RepartitionExec: partitioning=Hash([b1@6], 10), input_partitions=10, preserve_order=true, sort_exprs=b1@6 ASC", "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", "SortExec: expr=[b1@6 ASC]", "CoalescePartitionsExec", join_plan.as_str(), - "SortPreservingRepartitionExec: partitioning=Hash([a@0], 10), input_partitions=10, sort_exprs=a@0 ASC", + "RepartitionExec: partitioning=Hash([a@0], 10), input_partitions=10, preserve_order=true, sort_exprs=a@0 ASC", "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", "SortExec: expr=[a@0 ASC]", "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", - "SortPreservingRepartitionExec: partitioning=Hash([b1@1], 10), input_partitions=10, sort_exprs=b1@1 ASC", + "RepartitionExec: partitioning=Hash([b1@1], 10), input_partitions=10, preserve_order=true, sort_exprs=b1@1 ASC", "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", "SortExec: expr=[b1@1 ASC]", "ProjectionExec: expr=[a@0 as a1, b@1 as b1, c@2 as c1, d@3 as d1, e@4 as e1]", "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", - "SortPreservingRepartitionExec: partitioning=Hash([c@2], 10), input_partitions=10, sort_exprs=c@2 ASC", + "RepartitionExec: partitioning=Hash([c@2], 10), input_partitions=10, preserve_order=true, sort_exprs=c@2 ASC", "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", "SortExec: expr=[c@2 ASC]", "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", @@ -3251,7 +3251,7 @@ pub(crate) mod tests { let expected_first_sort_enforcement = &[ "SortMergeJoin: join_type=Inner, on=[(b3@1, b2@1), (a3@0, a2@0)]", - "SortPreservingRepartitionExec: partitioning=Hash([b3@1, a3@0], 10), input_partitions=10, sort_exprs=b3@1 ASC,a3@0 ASC", + "RepartitionExec: partitioning=Hash([b3@1, a3@0], 10), input_partitions=10, preserve_order=true, sort_exprs=b3@1 ASC,a3@0 ASC", "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", "SortExec: expr=[b3@1 ASC,a3@0 ASC]", "CoalescePartitionsExec", @@ -3262,7 +3262,7 @@ pub(crate) mod tests { "AggregateExec: mode=Partial, gby=[b@1 as b1, a@0 as a1], aggr=[]", "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", - "SortPreservingRepartitionExec: partitioning=Hash([b2@1, a2@0], 10), input_partitions=10, sort_exprs=b2@1 ASC,a2@0 ASC", + "RepartitionExec: partitioning=Hash([b2@1, a2@0], 10), input_partitions=10, preserve_order=true, sort_exprs=b2@1 ASC,a2@0 ASC", "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", "SortExec: expr=[b2@1 ASC,a2@0 ASC]", "CoalescePartitionsExec", @@ -4347,7 +4347,7 @@ pub(crate) mod tests { let expected = &[ "SortPreservingMergeExec: [c@2 ASC]", "FilterExec: c@2 = 0", - "SortPreservingRepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=2, sort_exprs=c@2 ASC", + "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=2, preserve_order=true, sort_exprs=c@2 ASC", "ParquetExec: file_groups={2 groups: [[x], [y]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC]", ]; diff --git a/datafusion/core/src/physical_optimizer/enforce_sorting.rs b/datafusion/core/src/physical_optimizer/enforce_sorting.rs index 14715ede500a..277404b301c4 100644 --- a/datafusion/core/src/physical_optimizer/enforce_sorting.rs +++ b/datafusion/core/src/physical_optimizer/enforce_sorting.rs @@ -2162,7 +2162,7 @@ mod tests { ]; let expected_optimized = [ "SortPreservingMergeExec: [a@0 ASC]", - " SortPreservingRepartitionExec: partitioning=Hash([c@2], 10), input_partitions=10, sort_exprs=a@0 ASC", + " RepartitionExec: partitioning=Hash([c@2], 10), input_partitions=10, preserve_order=true, sort_exprs=a@0 ASC", " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", " CsvExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], infinite_source=true, output_ordering=[a@0 ASC], has_header=false", ]; @@ -2191,7 +2191,7 @@ mod tests { ]; let expected_optimized = [ "SortPreservingMergeExec: [a@0 ASC]", - " SortPreservingRepartitionExec: partitioning=Hash([c@2], 10), input_partitions=10, sort_exprs=a@0 ASC", + " RepartitionExec: partitioning=Hash([c@2], 10), input_partitions=10, preserve_order=true, sort_exprs=a@0 ASC", " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", " CsvExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], infinite_source=true, output_ordering=[a@0 ASC], has_header=false", ]; @@ -2263,7 +2263,7 @@ mod tests { let expected_input = [ "BoundedWindowAggExec: wdw=[count: Ok(Field { name: \"count\", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(NULL), end_bound: CurrentRow }], mode=[Sorted]", " SortPreservingMergeExec: [a@0 ASC,b@1 ASC]", - " SortPreservingRepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=10, sort_exprs=a@0 ASC,b@1 ASC", + " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=10, preserve_order=true, sort_exprs=a@0 ASC,b@1 ASC", " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", " SortExec: expr=[a@0 ASC,b@1 ASC]", " CsvExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], has_header=false", diff --git a/datafusion/core/src/physical_optimizer/replace_with_order_preserving_variants.rs b/datafusion/core/src/physical_optimizer/replace_with_order_preserving_variants.rs index 09274938cbce..af45df7d8474 100644 --- a/datafusion/core/src/physical_optimizer/replace_with_order_preserving_variants.rs +++ b/datafusion/core/src/physical_optimizer/replace_with_order_preserving_variants.rs @@ -366,7 +366,7 @@ mod tests { ]; let expected_optimized = [ "SortPreservingMergeExec: [a@0 ASC NULLS LAST]", - " SortPreservingRepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8, sort_exprs=a@0 ASC NULLS LAST", + " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8, preserve_order=true, sort_exprs=a@0 ASC NULLS LAST", " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", " CsvExec: file_groups={1 group: [[file_path]]}, projection=[a, c, d], infinite_source=true, output_ordering=[a@0 ASC NULLS LAST], has_header=true", ]; @@ -414,10 +414,10 @@ mod tests { let expected_optimized = [ "SortPreservingMergeExec: [a@0 ASC]", " FilterExec: c@1 > 3", - " SortPreservingRepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8, sort_exprs=a@0 ASC", + " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8, preserve_order=true, sort_exprs=a@0 ASC", " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", " SortPreservingMergeExec: [a@0 ASC]", - " SortPreservingRepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8, sort_exprs=a@0 ASC", + " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8, preserve_order=true, sort_exprs=a@0 ASC", " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", " CsvExec: file_groups={1 group: [[file_path]]}, projection=[a, c, d], infinite_source=true, output_ordering=[a@0 ASC], has_header=true", ]; @@ -448,7 +448,7 @@ mod tests { ]; let expected_optimized = [ "SortPreservingMergeExec: [a@0 ASC NULLS LAST]", - " SortPreservingRepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8, sort_exprs=a@0 ASC NULLS LAST", + " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8, preserve_order=true, sort_exprs=a@0 ASC NULLS LAST", " FilterExec: c@1 > 3", " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", " CsvExec: file_groups={1 group: [[file_path]]}, projection=[a, c, d], infinite_source=true, output_ordering=[a@0 ASC NULLS LAST], has_header=true", @@ -484,7 +484,7 @@ mod tests { "SortPreservingMergeExec: [a@0 ASC NULLS LAST]", " CoalesceBatchesExec: target_batch_size=8192", " FilterExec: c@1 > 3", - " SortPreservingRepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8, sort_exprs=a@0 ASC NULLS LAST", + " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8, preserve_order=true, sort_exprs=a@0 ASC NULLS LAST", " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", " CsvExec: file_groups={1 group: [[file_path]]}, projection=[a, c, d], infinite_source=true, output_ordering=[a@0 ASC NULLS LAST], has_header=true", ]; @@ -522,7 +522,7 @@ mod tests { "SortPreservingMergeExec: [a@0 ASC NULLS LAST]", " CoalesceBatchesExec: target_batch_size=8192", " FilterExec: c@1 > 3", - " SortPreservingRepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8, sort_exprs=a@0 ASC NULLS LAST", + " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8, preserve_order=true, sort_exprs=a@0 ASC NULLS LAST", " CoalesceBatchesExec: target_batch_size=8192", " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", " CsvExec: file_groups={1 group: [[file_path]]}, projection=[a, c, d], infinite_source=true, output_ordering=[a@0 ASC NULLS LAST], has_header=true", @@ -591,10 +591,10 @@ mod tests { ]; let expected_optimized = [ "SortPreservingMergeExec: [a@0 ASC NULLS LAST]", - " SortPreservingRepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8, sort_exprs=a@0 ASC NULLS LAST", + " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8, preserve_order=true, sort_exprs=a@0 ASC NULLS LAST", " CoalesceBatchesExec: target_batch_size=8192", " FilterExec: c@1 > 3", - " SortPreservingRepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8, sort_exprs=a@0 ASC NULLS LAST", + " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8, preserve_order=true, sort_exprs=a@0 ASC NULLS LAST", " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", " CsvExec: file_groups={1 group: [[file_path]]}, projection=[a, c, d], infinite_source=true, output_ordering=[a@0 ASC NULLS LAST], has_header=true", ]; @@ -658,7 +658,7 @@ mod tests { ]; let expected_optimized = [ "SortPreservingMergeExec: [a@0 ASC NULLS LAST]", - " SortPreservingRepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8, sort_exprs=a@0 ASC NULLS LAST", + " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8, preserve_order=true, sort_exprs=a@0 ASC NULLS LAST", " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", " CsvExec: file_groups={1 group: [[file_path]]}, projection=[a, c, d], infinite_source=true, output_ordering=[a@0 ASC NULLS LAST], has_header=true", ]; @@ -706,7 +706,7 @@ mod tests { let expected_optimized = [ "SortPreservingMergeExec: [c@1 ASC]", " FilterExec: c@1 > 3", - " SortPreservingRepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8, sort_exprs=c@1 ASC", + " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8, preserve_order=true, sort_exprs=c@1 ASC", " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", " SortExec: expr=[c@1 ASC]", " CoalescePartitionsExec", @@ -800,7 +800,7 @@ mod tests { ]; let expected_optimized = [ "SortPreservingMergeExec: [a@0 ASC NULLS LAST]", - " SortPreservingRepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8, sort_exprs=a@0 ASC NULLS LAST", + " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8, preserve_order=true, sort_exprs=a@0 ASC NULLS LAST", " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", " CsvExec: file_groups={1 group: [[file_path]]}, projection=[a, c, d], output_ordering=[a@0 ASC NULLS LAST], has_header=true", ]; diff --git a/datafusion/physical-plan/src/repartition/mod.rs b/datafusion/physical-plan/src/repartition/mod.rs index 24f227d8a535..769dc5e0e197 100644 --- a/datafusion/physical-plan/src/repartition/mod.rs +++ b/datafusion/physical-plan/src/repartition/mod.rs @@ -370,11 +370,7 @@ impl RepartitionExec { /// Get name used to display this Exec pub fn name(&self) -> &str { - if self.preserve_order { - "SortPreservingRepartitionExec" - } else { - "RepartitionExec" - } + "RepartitionExec" } } @@ -394,6 +390,10 @@ impl DisplayAs for RepartitionExec { self.input.output_partitioning().partition_count() )?; + if self.preserve_order { + write!(f, ", preserve_order=true")?; + } + if let Some(sort_exprs) = self.sort_exprs() { write!( f, @@ -1491,7 +1491,7 @@ mod test { // Repartition should preserve order let expected_plan = [ - "SortPreservingRepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=2, sort_exprs=c0@0 ASC", + "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=2, preserve_order=true, sort_exprs=c0@0 ASC", " UnionExec", " MemoryExec: partitions=1, partition_sizes=[0], output_ordering=c0@0 ASC", " MemoryExec: partitions=1, partition_sizes=[0], output_ordering=c0@0 ASC", diff --git a/datafusion/sqllogictest/test_files/groupby.slt b/datafusion/sqllogictest/test_files/groupby.slt index b7be4d78b583..8ed5245ef09b 100644 --- a/datafusion/sqllogictest/test_files/groupby.slt +++ b/datafusion/sqllogictest/test_files/groupby.slt @@ -4073,7 +4073,7 @@ GlobalLimitExec: skip=0, fetch=5 ----ProjectionExec: expr=[date_bin(Utf8("15 minutes"),unbounded_csv_with_timestamps.ts)@0 as time_chunks] ------AggregateExec: mode=FinalPartitioned, gby=[date_bin(Utf8("15 minutes"),unbounded_csv_with_timestamps.ts)@0 as date_bin(Utf8("15 minutes"),unbounded_csv_with_timestamps.ts)], aggr=[], ordering_mode=Sorted --------CoalesceBatchesExec: target_batch_size=2 -----------SortPreservingRepartitionExec: partitioning=Hash([date_bin(Utf8("15 minutes"),unbounded_csv_with_timestamps.ts)@0], 8), input_partitions=8, sort_exprs=date_bin(Utf8("15 minutes"),unbounded_csv_with_timestamps.ts)@0 DESC +----------RepartitionExec: partitioning=Hash([date_bin(Utf8("15 minutes"),unbounded_csv_with_timestamps.ts)@0], 8), input_partitions=8, preserve_order=true, sort_exprs=date_bin(Utf8("15 minutes"),unbounded_csv_with_timestamps.ts)@0 DESC ------------AggregateExec: mode=Partial, gby=[date_bin(900000000000, ts@0) as date_bin(Utf8("15 minutes"),unbounded_csv_with_timestamps.ts)], aggr=[], ordering_mode=Sorted --------------RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1 ----------------StreamingTableExec: partition_sizes=1, projection=[ts], infinite_source=true, output_ordering=[ts@0 DESC] diff --git a/datafusion/sqllogictest/test_files/joins.slt b/datafusion/sqllogictest/test_files/joins.slt index 0fea8da5a342..67e3750113da 100644 --- a/datafusion/sqllogictest/test_files/joins.slt +++ b/datafusion/sqllogictest/test_files/joins.slt @@ -3464,7 +3464,7 @@ SortPreservingMergeExec: [a@0 ASC] ----------------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 ------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, b, c], output_ordering=[a@0 ASC, b@1 ASC NULLS LAST, c@2 ASC NULLS LAST], has_header=true ------------------CoalesceBatchesExec: target_batch_size=2 ---------------------SortPreservingRepartitionExec: partitioning=Hash([a@0], 2), input_partitions=2, sort_exprs=a@0 ASC,b@1 ASC NULLS LAST +--------------------RepartitionExec: partitioning=Hash([a@0], 2), input_partitions=2, preserve_order=true, sort_exprs=a@0 ASC,b@1 ASC NULLS LAST ----------------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 ------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, b], output_ordering=[a@0 ASC, b@1 ASC NULLS LAST], has_header=true diff --git a/datafusion/sqllogictest/test_files/window.slt b/datafusion/sqllogictest/test_files/window.slt index f3de5b54fc8b..7b628f9b6f14 100644 --- a/datafusion/sqllogictest/test_files/window.slt +++ b/datafusion/sqllogictest/test_files/window.slt @@ -3245,17 +3245,17 @@ physical_plan ProjectionExec: expr=[SUM(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.a, annotated_data_infinite2.b] ORDER BY [annotated_data_infinite2.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@2 as sum1, SUM(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.b, annotated_data_infinite2.a] ORDER BY [annotated_data_infinite2.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@4 as sum2, SUM(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.a, annotated_data_infinite2.d] ORDER BY [annotated_data_infinite2.b ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@3 as sum3, SUM(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.d] ORDER BY [annotated_data_infinite2.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@5 as sum4] --BoundedWindowAggExec: wdw=[SUM(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.d] ORDER BY [annotated_data_infinite2.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "SUM(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.d] ORDER BY [annotated_data_infinite2.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(NULL)), end_bound: CurrentRow }], mode=[Linear] ----CoalesceBatchesExec: target_batch_size=4096 -------SortPreservingRepartitionExec: partitioning=Hash([d@1], 2), input_partitions=2, sort_exprs=a@0 ASC NULLS LAST +------RepartitionExec: partitioning=Hash([d@1], 2), input_partitions=2, preserve_order=true, sort_exprs=a@0 ASC NULLS LAST --------ProjectionExec: expr=[a@0 as a, d@3 as d, SUM(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.a, annotated_data_infinite2.b] ORDER BY [annotated_data_infinite2.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@4 as SUM(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.a, annotated_data_infinite2.b] ORDER BY [annotated_data_infinite2.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW, SUM(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.a, annotated_data_infinite2.d] ORDER BY [annotated_data_infinite2.b ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@5 as SUM(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.a, annotated_data_infinite2.d] ORDER BY [annotated_data_infinite2.b ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW, SUM(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.b, annotated_data_infinite2.a] ORDER BY [annotated_data_infinite2.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@6 as SUM(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.b, annotated_data_infinite2.a] ORDER BY [annotated_data_infinite2.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW] ----------BoundedWindowAggExec: wdw=[SUM(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.b, annotated_data_infinite2.a] ORDER BY [annotated_data_infinite2.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "SUM(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.b, annotated_data_infinite2.a] ORDER BY [annotated_data_infinite2.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(NULL)), end_bound: CurrentRow }], mode=[Sorted] ------------CoalesceBatchesExec: target_batch_size=4096 ---------------SortPreservingRepartitionExec: partitioning=Hash([b@1, a@0], 2), input_partitions=2, sort_exprs=a@0 ASC NULLS LAST,b@1 ASC NULLS LAST,c@2 ASC NULLS LAST +--------------RepartitionExec: partitioning=Hash([b@1, a@0], 2), input_partitions=2, preserve_order=true, sort_exprs=a@0 ASC NULLS LAST,b@1 ASC NULLS LAST,c@2 ASC NULLS LAST ----------------BoundedWindowAggExec: wdw=[SUM(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.a, annotated_data_infinite2.d] ORDER BY [annotated_data_infinite2.b ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "SUM(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.a, annotated_data_infinite2.d] ORDER BY [annotated_data_infinite2.b ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(NULL)), end_bound: CurrentRow }], mode=[PartiallySorted([0])] ------------------CoalesceBatchesExec: target_batch_size=4096 ---------------------SortPreservingRepartitionExec: partitioning=Hash([a@0, d@3], 2), input_partitions=2, sort_exprs=a@0 ASC NULLS LAST,b@1 ASC NULLS LAST,c@2 ASC NULLS LAST +--------------------RepartitionExec: partitioning=Hash([a@0, d@3], 2), input_partitions=2, preserve_order=true, sort_exprs=a@0 ASC NULLS LAST,b@1 ASC NULLS LAST,c@2 ASC NULLS LAST ----------------------BoundedWindowAggExec: wdw=[SUM(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.a, annotated_data_infinite2.b] ORDER BY [annotated_data_infinite2.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "SUM(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.a, annotated_data_infinite2.b] ORDER BY [annotated_data_infinite2.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(NULL)), end_bound: CurrentRow }], mode=[Sorted] ------------------------CoalesceBatchesExec: target_batch_size=4096 ---------------------------SortPreservingRepartitionExec: partitioning=Hash([a@0, b@1], 2), input_partitions=2, sort_exprs=a@0 ASC NULLS LAST,b@1 ASC NULLS LAST,c@2 ASC NULLS LAST +--------------------------RepartitionExec: partitioning=Hash([a@0, b@1], 2), input_partitions=2, preserve_order=true, sort_exprs=a@0 ASC NULLS LAST,b@1 ASC NULLS LAST,c@2 ASC NULLS LAST ----------------------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 ------------------------------StreamingTableExec: partition_sizes=1, projection=[a, b, c, d], infinite_source=true, output_ordering=[a@0 ASC NULLS LAST, b@1 ASC NULLS LAST, c@2 ASC NULLS LAST] @@ -3571,7 +3571,7 @@ SortPreservingMergeExec: [c@3 ASC NULLS LAST] --ProjectionExec: expr=[a0@0 as a0, a@1 as a, b@2 as b, c@3 as c, d@4 as d, AVG(multiple_ordered_table_inf.d) PARTITION BY [multiple_ordered_table_inf.d] ORDER BY [multiple_ordered_table_inf.a ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND CURRENT ROW@5 as avg_d] ----BoundedWindowAggExec: wdw=[AVG(multiple_ordered_table_inf.d) PARTITION BY [multiple_ordered_table_inf.d] ORDER BY [multiple_ordered_table_inf.a ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND CURRENT ROW: Ok(Field { name: "AVG(multiple_ordered_table_inf.d) PARTITION BY [multiple_ordered_table_inf.d] ORDER BY [multiple_ordered_table_inf.a ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND CURRENT ROW", data_type: Float64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(10)), end_bound: CurrentRow }], mode=[Linear] ------CoalesceBatchesExec: target_batch_size=4096 ---------SortPreservingRepartitionExec: partitioning=Hash([d@4], 2), input_partitions=2, sort_exprs=a@1 ASC NULLS LAST,b@2 ASC NULLS LAST +--------RepartitionExec: partitioning=Hash([d@4], 2), input_partitions=2, preserve_order=true, sort_exprs=a@1 ASC NULLS LAST,b@2 ASC NULLS LAST ----------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 ------------StreamingTableExec: partition_sizes=1, projection=[a0, a, b, c, d], infinite_source=true, output_ordering=[a@1 ASC NULLS LAST, b@2 ASC NULLS LAST] From 2bc67ef5a9f1e9393e3fa7396ee3a53fa08ca415 Mon Sep 17 00:00:00 2001 From: Asura7969 <1402357969@qq.com> Date: Thu, 14 Dec 2023 02:54:55 +0800 Subject: [PATCH 417/572] Fix `DataFrame::cache` errors with `Plan("Mismatch between schema and batches")` (#8510) * type cast * add test * use physical plan * logic optimization --- datafusion/core/src/dataframe/mod.rs | 22 +++++++++++++++++----- 1 file changed, 17 insertions(+), 5 deletions(-) diff --git a/datafusion/core/src/dataframe/mod.rs b/datafusion/core/src/dataframe/mod.rs index 3e286110f7f9..4b8a9c5b7d79 100644 --- a/datafusion/core/src/dataframe/mod.rs +++ b/datafusion/core/src/dataframe/mod.rs @@ -1276,11 +1276,12 @@ impl DataFrame { /// ``` pub async fn cache(self) -> Result { let context = SessionContext::new_with_state(self.session_state.clone()); - let mem_table = MemTable::try_new( - SchemaRef::from(self.schema().clone()), - self.collect_partitioned().await?, - )?; - + // The schema is consistent with the output + let plan = self.clone().create_physical_plan().await?; + let schema = plan.schema(); + let task_ctx = Arc::new(self.task_ctx()); + let partitions = collect_partitioned(plan, task_ctx).await?; + let mem_table = MemTable::try_new(schema, partitions)?; context.read_table(Arc::new(mem_table)) } } @@ -2638,6 +2639,17 @@ mod tests { Ok(()) } + #[tokio::test] + async fn test_cache_mismatch() -> Result<()> { + let ctx = SessionContext::new(); + let df = ctx + .sql("SELECT CASE WHEN true THEN NULL ELSE 1 END") + .await?; + let cache_df = df.cache().await; + assert!(cache_df.is_ok()); + Ok(()) + } + #[tokio::test] async fn cache_test() -> Result<()> { let df = test_table() From 0678a6978528d0e2e1c85a1018b68ed974eb45cf Mon Sep 17 00:00:00 2001 From: Andrew Lamb Date: Wed, 13 Dec 2023 13:55:13 -0500 Subject: [PATCH 418/572] Minor: update pbjson_dependency (#8470) --- datafusion/proto/Cargo.toml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/datafusion/proto/Cargo.toml b/datafusion/proto/Cargo.toml index 4dda689fff4c..fbd412aedaa5 100644 --- a/datafusion/proto/Cargo.toml +++ b/datafusion/proto/Cargo.toml @@ -47,7 +47,7 @@ datafusion = { path = "../core", version = "33.0.0" } datafusion-common = { workspace = true } datafusion-expr = { workspace = true } object_store = { workspace = true } -pbjson = { version = "0.5", optional = true } +pbjson = { version = "0.6.0", optional = true } prost = "0.12.0" serde = { version = "1.0", optional = true } serde_json = { workspace = true, optional = true } From 2e93f079461fc3603df07af94ce616a5317af370 Mon Sep 17 00:00:00 2001 From: Andrew Lamb Date: Wed, 13 Dec 2023 13:55:43 -0500 Subject: [PATCH 419/572] Minor: Update prost-derive dependency (#8471) --- datafusion-examples/Cargo.toml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/datafusion-examples/Cargo.toml b/datafusion-examples/Cargo.toml index 676b4aaa78c0..59580bcb6a05 100644 --- a/datafusion-examples/Cargo.toml +++ b/datafusion-examples/Cargo.toml @@ -48,7 +48,7 @@ mimalloc = { version = "0.1", default-features = false } num_cpus = { workspace = true } object_store = { workspace = true, features = ["aws", "http"] } prost = { version = "0.12", default-features = false } -prost-derive = { version = "0.11", default-features = false } +prost-derive = { version = "0.12", default-features = false } serde = { version = "1.0.136", features = ["derive"] } serde_json = { workspace = true } tempfile = { workspace = true } From 9a322c8bdd5d62630936ac348bb935207675b99d Mon Sep 17 00:00:00 2001 From: Devin D'Angelo Date: Wed, 13 Dec 2023 15:39:37 -0500 Subject: [PATCH 420/572] Add write_table to dataframe actions in user guide (#8527) --- docs/source/user-guide/dataframe.md | 1 + 1 file changed, 1 insertion(+) diff --git a/docs/source/user-guide/dataframe.md b/docs/source/user-guide/dataframe.md index 4484b2c51019..c0210200a246 100644 --- a/docs/source/user-guide/dataframe.md +++ b/docs/source/user-guide/dataframe.md @@ -95,6 +95,7 @@ These methods execute the logical plan represented by the DataFrame and either c | write_csv | Execute this DataFrame and write the results to disk in CSV format. | | write_json | Execute this DataFrame and write the results to disk in JSON format. | | write_parquet | Execute this DataFrame and write the results to disk in Parquet format. | +| write_table | Execute this DataFrame and write the results via the insert_into method of the registered TableProvider | ## Other DataFrame Methods From 5bf80d655d545622d0272bc8e4bcf89dacaacf36 Mon Sep 17 00:00:00 2001 From: Andrew Lamb Date: Wed, 13 Dec 2023 16:07:01 -0500 Subject: [PATCH 421/572] Minor: Add repartition_file.slt end to end test for repartitioning files, and supporting tweaks (#8505) * Minor: Add repartition_file.slt end to end test for repartitioning files, and supporting tweaks * Sort files, update tests --- .../core/src/datasource/listing/helpers.rs | 8 +- datafusion/core/src/datasource/listing/mod.rs | 5 + datafusion/core/src/datasource/listing/url.rs | 4 + datafusion/sqllogictest/bin/sqllogictests.rs | 2 + .../test_files/repartition_scan.slt | 268 ++++++++++++++++++ 5 files changed, 286 insertions(+), 1 deletion(-) create mode 100644 datafusion/sqllogictest/test_files/repartition_scan.slt diff --git a/datafusion/core/src/datasource/listing/helpers.rs b/datafusion/core/src/datasource/listing/helpers.rs index 3536c098bd76..be74afa1f4d6 100644 --- a/datafusion/core/src/datasource/listing/helpers.rs +++ b/datafusion/core/src/datasource/listing/helpers.rs @@ -141,12 +141,18 @@ const CONCURRENCY_LIMIT: usize = 100; /// Partition the list of files into `n` groups pub fn split_files( - partitioned_files: Vec, + mut partitioned_files: Vec, n: usize, ) -> Vec> { if partitioned_files.is_empty() { return vec![]; } + + // ObjectStore::list does not guarantee any consistent order and for some + // implementations such as LocalFileSystem, it may be inconsistent. Thus + // Sort files by path to ensure consistent plans when run more than once. + partitioned_files.sort_by(|a, b| a.path().cmp(b.path())); + // effectively this is div with rounding up instead of truncating let chunk_size = (partitioned_files.len() + n - 1) / n; partitioned_files diff --git a/datafusion/core/src/datasource/listing/mod.rs b/datafusion/core/src/datasource/listing/mod.rs index 87c1663ae718..5e5b96f6ba8c 100644 --- a/datafusion/core/src/datasource/listing/mod.rs +++ b/datafusion/core/src/datasource/listing/mod.rs @@ -109,6 +109,11 @@ impl PartitionedFile { let size = std::fs::metadata(path.clone())?.len(); Ok(Self::new(path, size)) } + + /// Return the path of this partitioned file + pub fn path(&self) -> &Path { + &self.object_meta.location + } } impl From for PartitionedFile { diff --git a/datafusion/core/src/datasource/listing/url.rs b/datafusion/core/src/datasource/listing/url.rs index 9e9fb9210071..979ed9e975c4 100644 --- a/datafusion/core/src/datasource/listing/url.rs +++ b/datafusion/core/src/datasource/listing/url.rs @@ -131,6 +131,10 @@ impl ListingTableUrl { if is_directory { fs::create_dir_all(path)?; } else { + // ensure parent directory exists + if let Some(parent) = path.parent() { + fs::create_dir_all(parent)?; + } fs::File::create(path)?; } } diff --git a/datafusion/sqllogictest/bin/sqllogictests.rs b/datafusion/sqllogictest/bin/sqllogictests.rs index 618e3106c629..484677d58e79 100644 --- a/datafusion/sqllogictest/bin/sqllogictests.rs +++ b/datafusion/sqllogictest/bin/sqllogictests.rs @@ -159,6 +159,7 @@ async fn run_test_file_with_postgres(test_file: TestFile) -> Result<()> { relative_path, } = test_file; info!("Running with Postgres runner: {}", path.display()); + setup_scratch_dir(&relative_path)?; let mut runner = sqllogictest::Runner::new(|| Postgres::connect(relative_path.clone())); runner.with_column_validator(strict_column_validator); @@ -188,6 +189,7 @@ async fn run_complete_file(test_file: TestFile) -> Result<()> { info!("Skipping: {}", path.display()); return Ok(()); }; + setup_scratch_dir(&relative_path)?; let mut runner = sqllogictest::Runner::new(|| async { Ok(DataFusion::new( test_ctx.session_ctx().clone(), diff --git a/datafusion/sqllogictest/test_files/repartition_scan.slt b/datafusion/sqllogictest/test_files/repartition_scan.slt new file mode 100644 index 000000000000..551d6d9ed48a --- /dev/null +++ b/datafusion/sqllogictest/test_files/repartition_scan.slt @@ -0,0 +1,268 @@ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at + +# http://www.apache.org/licenses/LICENSE-2.0 + +# Unless required by applicable law or agreed to in writing, +# software distributed under the License is distributed on an +# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +# KIND, either express or implied. See the License for the +# specific language governing permissions and limitations +# under the License. + +########## +# Tests for automatically reading files in parallel during scan +########## + +# Set 4 partitions for deterministic output plans +statement ok +set datafusion.execution.target_partitions = 4; + +# automatically partition all files over 1 byte +statement ok +set datafusion.optimizer.repartition_file_min_size = 1; + +################### +### Parquet tests +################### + +# create a single parquet file +# Note filename 2.parquet to test sorting (on local file systems it is often listed before 1.parquet) +statement ok +COPY (VALUES (1), (2), (3), (4), (5)) TO 'test_files/scratch/repartition_scan/parquet_table/2.parquet' +(FORMAT PARQUET, SINGLE_FILE_OUTPUT true); + +statement ok +CREATE EXTERNAL TABLE parquet_table(column1 int) +STORED AS PARQUET +LOCATION 'test_files/scratch/repartition_scan/parquet_table/'; + +query I +select * from parquet_table; +---- +1 +2 +3 +4 +5 + +## Expect to see the scan read the file as "4" groups with even sizes (offsets) +query TT +EXPLAIN SELECT column1 FROM parquet_table WHERE column1 <> 42; +---- +logical_plan +Filter: parquet_table.column1 != Int32(42) +--TableScan: parquet_table projection=[column1], partial_filters=[parquet_table.column1 != Int32(42)] +physical_plan +CoalesceBatchesExec: target_batch_size=8192 +--FilterExec: column1@0 != 42 +----ParquetExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/parquet_table/2.parquet:0..101], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/parquet_table/2.parquet:101..202], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/parquet_table/2.parquet:202..303], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/parquet_table/2.parquet:303..403]]}, projection=[column1], predicate=column1@0 != 42, pruning_predicate=column1_min@0 != 42 OR 42 != column1_max@1 + +# create a second parquet file +statement ok +COPY (VALUES (100), (200)) TO 'test_files/scratch/repartition_scan/parquet_table/1.parquet' +(FORMAT PARQUET, SINGLE_FILE_OUTPUT true); + +## Still expect to see the scan read the file as "4" groups with even sizes. One group should read +## parts of both files. +query TT +EXPLAIN SELECT column1 FROM parquet_table WHERE column1 <> 42 ORDER BY column1; +---- +logical_plan +Sort: parquet_table.column1 ASC NULLS LAST +--Filter: parquet_table.column1 != Int32(42) +----TableScan: parquet_table projection=[column1], partial_filters=[parquet_table.column1 != Int32(42)] +physical_plan +SortPreservingMergeExec: [column1@0 ASC NULLS LAST] +--SortExec: expr=[column1@0 ASC NULLS LAST] +----CoalesceBatchesExec: target_batch_size=8192 +------FilterExec: column1@0 != 42 +--------ParquetExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/parquet_table/1.parquet:0..200], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/parquet_table/1.parquet:200..394, WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/parquet_table/2.parquet:0..6], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/parquet_table/2.parquet:6..206], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/parquet_table/2.parquet:206..403]]}, projection=[column1], predicate=column1@0 != 42, pruning_predicate=column1_min@0 != 42 OR 42 != column1_max@1 + + +## Read the files as though they are ordered + +statement ok +CREATE EXTERNAL TABLE parquet_table_with_order(column1 int) +STORED AS PARQUET +LOCATION 'test_files/scratch/repartition_scan/parquet_table' +WITH ORDER (column1 ASC); + +# output should be ordered +query I +SELECT column1 FROM parquet_table_with_order WHERE column1 <> 42 ORDER BY column1; +---- +1 +2 +3 +4 +5 +100 +200 + +# explain should not have any groups with more than one file +# https://github.com/apache/arrow-datafusion/issues/8451 +query TT +EXPLAIN SELECT column1 FROM parquet_table_with_order WHERE column1 <> 42 ORDER BY column1; +---- +logical_plan +Sort: parquet_table_with_order.column1 ASC NULLS LAST +--Filter: parquet_table_with_order.column1 != Int32(42) +----TableScan: parquet_table_with_order projection=[column1], partial_filters=[parquet_table_with_order.column1 != Int32(42)] +physical_plan +SortPreservingMergeExec: [column1@0 ASC NULLS LAST] +--CoalesceBatchesExec: target_batch_size=8192 +----FilterExec: column1@0 != 42 +------ParquetExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/parquet_table/1.parquet:0..200], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/parquet_table/1.parquet:200..394, WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/parquet_table/2.parquet:0..6], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/parquet_table/2.parquet:6..206], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/parquet_table/2.parquet:206..403]]}, projection=[column1], predicate=column1@0 != 42, pruning_predicate=column1_min@0 != 42 OR 42 != column1_max@1 + +# Cleanup +statement ok +DROP TABLE parquet_table; + +statement ok +DROP TABLE parquet_table_with_order; + + +################### +### CSV tests +################### + +# Since parquet and CSV share most of the same implementation, this test checks +# that the basics are connected properly + +# create a single csv file +statement ok +COPY (VALUES (1), (2), (3), (4), (5)) TO 'test_files/scratch/repartition_scan/csv_table/1.csv' +(FORMAT csv, SINGLE_FILE_OUTPUT true, HEADER true); + +statement ok +CREATE EXTERNAL TABLE csv_table(column1 int) +STORED AS csv +WITH HEADER ROW +LOCATION 'test_files/scratch/repartition_scan/csv_table/'; + +query I +select * from csv_table; +---- +1 +2 +3 +4 +5 + +## Expect to see the scan read the file as "4" groups with even sizes (offsets) +query TT +EXPLAIN SELECT column1 FROM csv_table WHERE column1 <> 42; +---- +logical_plan +Filter: csv_table.column1 != Int32(42) +--TableScan: csv_table projection=[column1], partial_filters=[csv_table.column1 != Int32(42)] +physical_plan +CoalesceBatchesExec: target_batch_size=8192 +--FilterExec: column1@0 != 42 +----CsvExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/csv_table/1.csv:0..5], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/csv_table/1.csv:5..10], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/csv_table/1.csv:10..15], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/csv_table/1.csv:15..18]]}, projection=[column1], has_header=true + +# Cleanup +statement ok +DROP TABLE csv_table; + + +################### +### JSON tests +################### + +# Since parquet and json share most of the same implementation, this test checks +# that the basics are connected properly + +# create a single json file +statement ok +COPY (VALUES (1), (2), (3), (4), (5)) TO 'test_files/scratch/repartition_scan/json_table/1.json' +(FORMAT json, SINGLE_FILE_OUTPUT true); + +statement ok +CREATE EXTERNAL TABLE json_table(column1 int) +STORED AS json +LOCATION 'test_files/scratch/repartition_scan/json_table/'; + +query I +select * from json_table; +---- +1 +2 +3 +4 +5 + +## In the future it would be cool to see the file read as "4" groups with even sizes (offsets) +## but for now it is just one group +## https://github.com/apache/arrow-datafusion/issues/8502 +query TT +EXPLAIN SELECT column1 FROM json_table WHERE column1 <> 42; +---- +logical_plan +Filter: json_table.column1 != Int32(42) +--TableScan: json_table projection=[column1], partial_filters=[json_table.column1 != Int32(42)] +physical_plan +CoalesceBatchesExec: target_batch_size=8192 +--FilterExec: column1@0 != 42 +----RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +------JsonExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/json_table/1.json]]}, projection=[column1] + + +# Cleanup +statement ok +DROP TABLE json_table; + + +################### +### Arrow File tests +################### + +## Use pre-existing files we don't have a way to create arrow files yet +## (https://github.com/apache/arrow-datafusion/issues/8504) +statement ok +CREATE EXTERNAL TABLE arrow_table +STORED AS ARROW +LOCATION '../core/tests/data/example.arrow'; + + +# It would be great to see the file read as "4" groups with even sizes (offsets) eventually +# https://github.com/apache/arrow-datafusion/issues/8503 +query TT +EXPLAIN SELECT * FROM arrow_table +---- +logical_plan TableScan: arrow_table projection=[f0, f1, f2] +physical_plan ArrowExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/example.arrow]]}, projection=[f0, f1, f2] + +# Cleanup +statement ok +DROP TABLE arrow_table; + +################### +### Avro File tests +################### + +## Use pre-existing files we don't have a way to create avro files yet + +statement ok +CREATE EXTERNAL TABLE avro_table +STORED AS AVRO +WITH HEADER ROW +LOCATION '../../testing/data/avro/simple_enum.avro' + + +# It would be great to see the file read as "4" groups with even sizes (offsets) eventually +query TT +EXPLAIN SELECT * FROM avro_table +---- +logical_plan TableScan: avro_table projection=[f1, f2, f3] +physical_plan AvroExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/avro/simple_enum.avro]]}, projection=[f1, f2, f3] + +# Cleanup +statement ok +DROP TABLE avro_table; From 898911b9952679a163247b88b2a79c47be2e226c Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Wed, 13 Dec 2023 14:22:24 -0700 Subject: [PATCH 422/572] Prepare version 34.0.0 (#8508) * bump version * changelog * changelog * Revert change --- Cargo.toml | 24 +-- benchmarks/Cargo.toml | 8 +- datafusion-cli/Cargo.lock | 102 ++++++------ datafusion-cli/Cargo.toml | 4 +- datafusion/CHANGELOG.md | 1 + datafusion/core/Cargo.toml | 6 +- datafusion/optimizer/Cargo.toml | 4 +- datafusion/proto/Cargo.toml | 2 +- datafusion/sqllogictest/Cargo.toml | 2 +- dev/changelog/34.0.0.md | 247 +++++++++++++++++++++++++++++ docs/Cargo.toml | 2 +- docs/source/user-guide/configs.md | 2 +- 12 files changed, 326 insertions(+), 78 deletions(-) create mode 100644 dev/changelog/34.0.0.md diff --git a/Cargo.toml b/Cargo.toml index 2bcbe059ab25..023dc6c6fc4f 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -46,7 +46,7 @@ license = "Apache-2.0" readme = "README.md" repository = "https://github.com/apache/arrow-datafusion" rust-version = "1.70" -version = "33.0.0" +version = "34.0.0" [workspace.dependencies] arrow = { version = "49.0.0", features = ["prettyprint"] } @@ -59,17 +59,17 @@ async-trait = "0.1.73" bigdecimal = "0.4.1" bytes = "1.4" ctor = "0.2.0" -datafusion = { path = "datafusion/core", version = "33.0.0" } -datafusion-common = { path = "datafusion/common", version = "33.0.0" } -datafusion-expr = { path = "datafusion/expr", version = "33.0.0" } -datafusion-sql = { path = "datafusion/sql", version = "33.0.0" } -datafusion-optimizer = { path = "datafusion/optimizer", version = "33.0.0" } -datafusion-physical-expr = { path = "datafusion/physical-expr", version = "33.0.0" } -datafusion-physical-plan = { path = "datafusion/physical-plan", version = "33.0.0" } -datafusion-execution = { path = "datafusion/execution", version = "33.0.0" } -datafusion-proto = { path = "datafusion/proto", version = "33.0.0" } -datafusion-sqllogictest = { path = "datafusion/sqllogictest", version = "33.0.0" } -datafusion-substrait = { path = "datafusion/substrait", version = "33.0.0" } +datafusion = { path = "datafusion/core", version = "34.0.0" } +datafusion-common = { path = "datafusion/common", version = "34.0.0" } +datafusion-expr = { path = "datafusion/expr", version = "34.0.0" } +datafusion-sql = { path = "datafusion/sql", version = "34.0.0" } +datafusion-optimizer = { path = "datafusion/optimizer", version = "34.0.0" } +datafusion-physical-expr = { path = "datafusion/physical-expr", version = "34.0.0" } +datafusion-physical-plan = { path = "datafusion/physical-plan", version = "34.0.0" } +datafusion-execution = { path = "datafusion/execution", version = "34.0.0" } +datafusion-proto = { path = "datafusion/proto", version = "34.0.0" } +datafusion-sqllogictest = { path = "datafusion/sqllogictest", version = "34.0.0" } +datafusion-substrait = { path = "datafusion/substrait", version = "34.0.0" } dashmap = "5.4.0" doc-comment = "0.3" env_logger = "0.10" diff --git a/benchmarks/Cargo.toml b/benchmarks/Cargo.toml index c5a24a0a5cf9..4ce46968e1f4 100644 --- a/benchmarks/Cargo.toml +++ b/benchmarks/Cargo.toml @@ -18,7 +18,7 @@ [package] name = "datafusion-benchmarks" description = "DataFusion Benchmarks" -version = "33.0.0" +version = "34.0.0" edition = { workspace = true } authors = ["Apache Arrow "] homepage = "https://github.com/apache/arrow-datafusion" @@ -34,8 +34,8 @@ snmalloc = ["snmalloc-rs"] [dependencies] arrow = { workspace = true } -datafusion = { path = "../datafusion/core", version = "33.0.0" } -datafusion-common = { path = "../datafusion/common", version = "33.0.0" } +datafusion = { path = "../datafusion/core", version = "34.0.0" } +datafusion-common = { path = "../datafusion/common", version = "34.0.0" } env_logger = { workspace = true } futures = { workspace = true } log = { workspace = true } @@ -50,4 +50,4 @@ test-utils = { path = "../test-utils/", version = "0.1.0" } tokio = { version = "^1.0", features = ["macros", "rt", "rt-multi-thread", "parking_lot"] } [dev-dependencies] -datafusion-proto = { path = "../datafusion/proto", version = "33.0.0" } +datafusion-proto = { path = "../datafusion/proto", version = "34.0.0" } diff --git a/datafusion-cli/Cargo.lock b/datafusion-cli/Cargo.lock index 76be04d5ef67..19ad6709362d 100644 --- a/datafusion-cli/Cargo.lock +++ b/datafusion-cli/Cargo.lock @@ -384,7 +384,7 @@ checksum = "a66537f1bb974b254c98ed142ff995236e81b9d0fe4db0575f46612cb15eb0f9" dependencies = [ "proc-macro2", "quote", - "syn 2.0.39", + "syn 2.0.40", ] [[package]] @@ -1074,7 +1074,7 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "37e366bff8cd32dd8754b0991fb66b279dc48f598c3a18914852a6673deef583" dependencies = [ "quote", - "syn 2.0.39", + "syn 2.0.40", ] [[package]] @@ -1098,7 +1098,7 @@ dependencies = [ [[package]] name = "datafusion" -version = "33.0.0" +version = "34.0.0" dependencies = [ "ahash", "apache-avro", @@ -1145,7 +1145,7 @@ dependencies = [ [[package]] name = "datafusion-cli" -version = "33.0.0" +version = "34.0.0" dependencies = [ "arrow", "assert_cmd", @@ -1172,7 +1172,7 @@ dependencies = [ [[package]] name = "datafusion-common" -version = "33.0.0" +version = "34.0.0" dependencies = [ "ahash", "apache-avro", @@ -1191,7 +1191,7 @@ dependencies = [ [[package]] name = "datafusion-execution" -version = "33.0.0" +version = "34.0.0" dependencies = [ "arrow", "chrono", @@ -1210,7 +1210,7 @@ dependencies = [ [[package]] name = "datafusion-expr" -version = "33.0.0" +version = "34.0.0" dependencies = [ "ahash", "arrow", @@ -1224,7 +1224,7 @@ dependencies = [ [[package]] name = "datafusion-optimizer" -version = "33.0.0" +version = "34.0.0" dependencies = [ "arrow", "async-trait", @@ -1240,7 +1240,7 @@ dependencies = [ [[package]] name = "datafusion-physical-expr" -version = "33.0.0" +version = "34.0.0" dependencies = [ "ahash", "arrow", @@ -1272,7 +1272,7 @@ dependencies = [ [[package]] name = "datafusion-physical-plan" -version = "33.0.0" +version = "34.0.0" dependencies = [ "ahash", "arrow", @@ -1301,7 +1301,7 @@ dependencies = [ [[package]] name = "datafusion-sql" -version = "33.0.0" +version = "34.0.0" dependencies = [ "arrow", "arrow-schema", @@ -1576,7 +1576,7 @@ checksum = "53b153fd91e4b0147f4aced87be237c98248656bb01050b96bf3ee89220a8ddb" dependencies = [ "proc-macro2", "quote", - "syn 2.0.39", + "syn 2.0.40", ] [[package]] @@ -1752,9 +1752,9 @@ dependencies = [ [[package]] name = "http-body" -version = "0.4.5" +version = "0.4.6" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "d5f38f16d184e36f2408a55281cd658ecbd3ca05cce6d6510a176eca393e26d1" +checksum = "7ceab25649e9960c0311ea418d17bee82c0dcec1bd053b5f9a66e265a693bed2" dependencies = [ "bytes", "http", @@ -1827,7 +1827,7 @@ dependencies = [ "futures-util", "http", "hyper", - "rustls 0.21.9", + "rustls 0.21.10", "tokio", "tokio-rustls 0.24.1", ] @@ -1926,9 +1926,9 @@ dependencies = [ [[package]] name = "itoa" -version = "1.0.9" +version = "1.0.10" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "af150ab688ff2122fcef229be89cb50dd66af9e01a4ff320cc137eecc9bacc38" +checksum = "b1a46d1a171d865aa5f83f92695765caa047a9b4cbae2cbf37dbd613a793fd4c" [[package]] name = "jobserver" @@ -2020,9 +2020,9 @@ dependencies = [ [[package]] name = "libc" -version = "0.2.150" +version = "0.2.151" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "89d92a4743f9a61002fae18374ed11e7973f530cb3a3255fb354818118b2203c" +checksum = "302d7ab3130588088d277783b1e2d2e10c9e9e4a16dd9050e6ec93fb3e7048f4" [[package]] name = "libflate" @@ -2322,9 +2322,9 @@ dependencies = [ [[package]] name = "once_cell" -version = "1.18.0" +version = "1.19.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "dd8b5dd2ae5ed71462c540258bedcb51965123ad7e7ccf4b9a8cafaa4a63576d" +checksum = "3fdb12b2476b595f9358c5161aa467c2438859caa136dec86c26fdd2efe17b92" [[package]] name = "openssl-probe" @@ -2496,7 +2496,7 @@ checksum = "4359fd9c9171ec6e8c62926d6faaf553a8dc3f64e1507e76da7911b4f6a04405" dependencies = [ "proc-macro2", "quote", - "syn 2.0.39", + "syn 2.0.40", ] [[package]] @@ -2736,7 +2736,7 @@ dependencies = [ "once_cell", "percent-encoding", "pin-project-lite", - "rustls 0.21.9", + "rustls 0.21.10", "rustls-pemfile", "serde", "serde_json", @@ -2833,9 +2833,9 @@ dependencies = [ [[package]] name = "rustix" -version = "0.38.26" +version = "0.38.28" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "9470c4bf8246c8daf25f9598dca807fb6510347b1e1cfa55749113850c79d88a" +checksum = "72e572a5e8ca657d7366229cdde4bd14c4eb5499a9573d4d366fe1b599daa316" dependencies = [ "bitflags 2.4.1", "errno", @@ -2858,9 +2858,9 @@ dependencies = [ [[package]] name = "rustls" -version = "0.21.9" +version = "0.21.10" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "629648aced5775d558af50b2b4c7b02983a04b312126d45eeead26e7caa498b9" +checksum = "f9d5a6813c0759e4609cd494e8e725babae6a2ca7b62a5536a13daaec6fcb7ba" dependencies = [ "log", "ring 0.17.7", @@ -2930,9 +2930,9 @@ dependencies = [ [[package]] name = "ryu" -version = "1.0.15" +version = "1.0.16" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "1ad4cc8da4ef723ed60bced201181d83791ad433213d8c24efffda1eec85d741" +checksum = "f98d2aa92eebf49b69786be48e4477826b256916e84a57ff2a4f21923b48eb4c" [[package]] name = "same-file" @@ -3020,7 +3020,7 @@ checksum = "43576ca501357b9b071ac53cdc7da8ef0cbd9493d8df094cd821777ea6e894d3" dependencies = [ "proc-macro2", "quote", - "syn 2.0.39", + "syn 2.0.40", ] [[package]] @@ -3196,7 +3196,7 @@ dependencies = [ "proc-macro2", "quote", "rustversion", - "syn 2.0.39", + "syn 2.0.40", ] [[package]] @@ -3218,9 +3218,9 @@ dependencies = [ [[package]] name = "syn" -version = "2.0.39" +version = "2.0.40" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "23e78b90f2fcf45d3e842032ce32e3f2d1545ba6636271dcbf24fa306d87be7a" +checksum = "13fa70a4ee923979ffb522cacce59d34421ebdea5625e1073c4326ef9d2dd42e" dependencies = [ "proc-macro2", "quote", @@ -3299,7 +3299,7 @@ checksum = "266b2e40bc00e5a6c09c3584011e08b06f123c00362c92b975ba9843aaaa14b8" dependencies = [ "proc-macro2", "quote", - "syn 2.0.39", + "syn 2.0.40", ] [[package]] @@ -3367,9 +3367,9 @@ checksum = "1f3ccbac311fea05f86f61904b462b55fb3df8837a366dfc601a0161d0532f20" [[package]] name = "tokio" -version = "1.34.0" +version = "1.35.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "d0c014766411e834f7af5b8f4cf46257aab4036ca95e9d2c144a10f59ad6f5b9" +checksum = "841d45b238a16291a4e1584e61820b8ae57d696cc5015c459c229ccc6990cc1c" dependencies = [ "backtrace", "bytes", @@ -3391,7 +3391,7 @@ checksum = "5b8a1e28f2deaa14e508979454cb3a223b10b938b45af148bc0986de36f1923b" dependencies = [ "proc-macro2", "quote", - "syn 2.0.39", + "syn 2.0.40", ] [[package]] @@ -3411,7 +3411,7 @@ version = "0.24.1" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "c28327cf380ac148141087fbfb9de9d7bd4e84ab5d2c28fbc911d753de8a7081" dependencies = [ - "rustls 0.21.9", + "rustls 0.21.10", "tokio", ] @@ -3488,7 +3488,7 @@ checksum = "34704c8d6ebcbc939824180af020566b01a7c01f80641264eba0999f6c2b6be7" dependencies = [ "proc-macro2", "quote", - "syn 2.0.39", + "syn 2.0.40", ] [[package]] @@ -3502,9 +3502,9 @@ dependencies = [ [[package]] name = "try-lock" -version = "0.2.4" +version = "0.2.5" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "3528ecfd12c466c6f163363caf2d02a71161dd5e1cc6ae7b34207ea2d42d81ed" +checksum = "e421abadd41a4225275504ea4d6566923418b7f05506fbc9c0fe86ba7396114b" [[package]] name = "twox-hash" @@ -3533,7 +3533,7 @@ checksum = "f03ca4cb38206e2bef0700092660bb74d696f808514dae47fa1467cbfe26e96e" dependencies = [ "proc-macro2", "quote", - "syn 2.0.39", + "syn 2.0.40", ] [[package]] @@ -3544,9 +3544,9 @@ checksum = "42ff0bf0c66b8238c6f3b578df37d0b7848e55df8577b3f74f92a69acceeb825" [[package]] name = "unicode-bidi" -version = "0.3.13" +version = "0.3.14" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "92888ba5573ff080736b3648696b70cafad7d250551175acbaa4e0385b3e1460" +checksum = "6f2528f27a9eb2b21e69c95319b30bd0efd85d09c379741b0f78ea1d86be2416" [[package]] name = "unicode-ident" @@ -3687,7 +3687,7 @@ dependencies = [ "once_cell", "proc-macro2", "quote", - "syn 2.0.39", + "syn 2.0.40", "wasm-bindgen-shared", ] @@ -3721,7 +3721,7 @@ checksum = "f0eb82fcb7930ae6219a7ecfd55b217f5f0893484b7a13022ebb2b2bf20b5283" dependencies = [ "proc-macro2", "quote", - "syn 2.0.39", + "syn 2.0.40", "wasm-bindgen-backend", "wasm-bindgen-shared", ] @@ -3970,22 +3970,22 @@ dependencies = [ [[package]] name = "zerocopy" -version = "0.7.29" +version = "0.7.30" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "5d075cf85bbb114e933343e087b92f2146bac0d55b534cbb8188becf0039948e" +checksum = "306dca4455518f1f31635ec308b6b3e4eb1b11758cefafc782827d0aa7acb5c7" dependencies = [ "zerocopy-derive", ] [[package]] name = "zerocopy-derive" -version = "0.7.29" +version = "0.7.30" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "86cd5ca076997b97ef09d3ad65efe811fa68c9e874cb636ccb211223a813b0c2" +checksum = "be912bf68235a88fbefd1b73415cb218405958d1655b2ece9035a19920bdf6ba" dependencies = [ "proc-macro2", "quote", - "syn 2.0.39", + "syn 2.0.40", ] [[package]] diff --git a/datafusion-cli/Cargo.toml b/datafusion-cli/Cargo.toml index 5ce318aea3ac..1bf24808fb90 100644 --- a/datafusion-cli/Cargo.toml +++ b/datafusion-cli/Cargo.toml @@ -18,7 +18,7 @@ [package] name = "datafusion-cli" description = "Command Line Client for DataFusion query engine." -version = "33.0.0" +version = "34.0.0" authors = ["Apache Arrow "] edition = "2021" keywords = ["arrow", "datafusion", "query", "sql"] @@ -34,7 +34,7 @@ async-trait = "0.1.41" aws-config = "0.55" aws-credential-types = "0.55" clap = { version = "3", features = ["derive", "cargo"] } -datafusion = { path = "../datafusion/core", version = "33.0.0", features = ["avro", "crypto_expressions", "encoding_expressions", "parquet", "regex_expressions", "unicode_expressions", "compression"] } +datafusion = { path = "../datafusion/core", version = "34.0.0", features = ["avro", "crypto_expressions", "encoding_expressions", "parquet", "regex_expressions", "unicode_expressions", "compression"] } dirs = "4.0.0" env_logger = "0.9" mimalloc = { version = "0.1", default-features = false } diff --git a/datafusion/CHANGELOG.md b/datafusion/CHANGELOG.md index e224b9387655..d64bbeda877d 100644 --- a/datafusion/CHANGELOG.md +++ b/datafusion/CHANGELOG.md @@ -19,6 +19,7 @@ # Changelog +- [34.0.0](../dev/changelog/34.0.0.md) - [33.0.0](../dev/changelog/33.0.0.md) - [32.0.0](../dev/changelog/32.0.0.md) - [31.0.0](../dev/changelog/31.0.0.md) diff --git a/datafusion/core/Cargo.toml b/datafusion/core/Cargo.toml index 7caf91e24f2f..0ee83e756745 100644 --- a/datafusion/core/Cargo.toml +++ b/datafusion/core/Cargo.toml @@ -62,11 +62,11 @@ bytes = { workspace = true } bzip2 = { version = "0.4.3", optional = true } chrono = { workspace = true } dashmap = { workspace = true } -datafusion-common = { path = "../common", version = "33.0.0", features = ["object_store"], default-features = false } +datafusion-common = { path = "../common", version = "34.0.0", features = ["object_store"], default-features = false } datafusion-execution = { workspace = true } datafusion-expr = { workspace = true } -datafusion-optimizer = { path = "../optimizer", version = "33.0.0", default-features = false } -datafusion-physical-expr = { path = "../physical-expr", version = "33.0.0", default-features = false } +datafusion-optimizer = { path = "../optimizer", version = "34.0.0", default-features = false } +datafusion-physical-expr = { path = "../physical-expr", version = "34.0.0", default-features = false } datafusion-physical-plan = { workspace = true } datafusion-sql = { workspace = true } flate2 = { version = "1.0.24", optional = true } diff --git a/datafusion/optimizer/Cargo.toml b/datafusion/optimizer/Cargo.toml index fac880867fef..b350d41d3fe3 100644 --- a/datafusion/optimizer/Cargo.toml +++ b/datafusion/optimizer/Cargo.toml @@ -44,7 +44,7 @@ async-trait = { workspace = true } chrono = { workspace = true } datafusion-common = { workspace = true } datafusion-expr = { workspace = true } -datafusion-physical-expr = { path = "../physical-expr", version = "33.0.0", default-features = false } +datafusion-physical-expr = { path = "../physical-expr", version = "34.0.0", default-features = false } hashbrown = { version = "0.14", features = ["raw"] } itertools = { workspace = true } log = { workspace = true } @@ -52,5 +52,5 @@ regex-syntax = "0.8.0" [dev-dependencies] ctor = { workspace = true } -datafusion-sql = { path = "../sql", version = "33.0.0" } +datafusion-sql = { path = "../sql", version = "34.0.0" } env_logger = "0.10.0" diff --git a/datafusion/proto/Cargo.toml b/datafusion/proto/Cargo.toml index fbd412aedaa5..f9f24b28db81 100644 --- a/datafusion/proto/Cargo.toml +++ b/datafusion/proto/Cargo.toml @@ -43,7 +43,7 @@ parquet = ["datafusion/parquet", "datafusion-common/parquet"] [dependencies] arrow = { workspace = true } chrono = { workspace = true } -datafusion = { path = "../core", version = "33.0.0" } +datafusion = { path = "../core", version = "34.0.0" } datafusion-common = { workspace = true } datafusion-expr = { workspace = true } object_store = { workspace = true } diff --git a/datafusion/sqllogictest/Cargo.toml b/datafusion/sqllogictest/Cargo.toml index 436c6159e7a3..e333dc816f66 100644 --- a/datafusion/sqllogictest/Cargo.toml +++ b/datafusion/sqllogictest/Cargo.toml @@ -36,7 +36,7 @@ async-trait = { workspace = true } bigdecimal = { workspace = true } bytes = { version = "1.4.0", optional = true } chrono = { workspace = true, optional = true } -datafusion = { path = "../core", version = "33.0.0" } +datafusion = { path = "../core", version = "34.0.0" } datafusion-common = { workspace = true } futures = { version = "0.3.28" } half = { workspace = true } diff --git a/dev/changelog/34.0.0.md b/dev/changelog/34.0.0.md new file mode 100644 index 000000000000..8b8933017cfb --- /dev/null +++ b/dev/changelog/34.0.0.md @@ -0,0 +1,247 @@ + + +## [34.0.0](https://github.com/apache/arrow-datafusion/tree/34.0.0) (2023-12-11) + +[Full Changelog](https://github.com/apache/arrow-datafusion/compare/33.0.0...34.0.0) + +**Breaking changes:** + +- Implement `DISTINCT ON` from Postgres [#7981](https://github.com/apache/arrow-datafusion/pull/7981) (gruuya) +- Encapsulate `EquivalenceClass` into a struct [#8034](https://github.com/apache/arrow-datafusion/pull/8034) (alamb) +- Make fields of `ScalarUDF` , `AggregateUDF` and `WindowUDF` non `pub` [#8079](https://github.com/apache/arrow-datafusion/pull/8079) (alamb) +- Implement StreamTable and StreamTableProvider (#7994) [#8021](https://github.com/apache/arrow-datafusion/pull/8021) (tustvold) +- feat: make FixedSizeList scalar also an ArrayRef [#8221](https://github.com/apache/arrow-datafusion/pull/8221) (wjones127) +- Remove FileWriterMode and ListingTableInsertMode (#7994) [#8017](https://github.com/apache/arrow-datafusion/pull/8017) (tustvold) +- Refactor: Unify `Expr::ScalarFunction` and `Expr::ScalarUDF`, introduce unresolved functions by name [#8258](https://github.com/apache/arrow-datafusion/pull/8258) (2010YOUY01) +- Refactor aggregate function handling [#8358](https://github.com/apache/arrow-datafusion/pull/8358) (Weijun-H) +- Move `PartitionSearchMode` into datafusion_physical_plan, rename to `InputOrderMode` [#8364](https://github.com/apache/arrow-datafusion/pull/8364) (alamb) +- Split `EmptyExec` into `PlaceholderRowExec` [#8446](https://github.com/apache/arrow-datafusion/pull/8446) (razeghi71) + +**Implemented enhancements:** + +- feat: show statistics in explain verbose [#8113](https://github.com/apache/arrow-datafusion/pull/8113) (NGA-TRAN) +- feat:implement postgres style 'overlay' string function [#8117](https://github.com/apache/arrow-datafusion/pull/8117) (Syleechan) +- feat: fill missing values with NULLs while inserting [#8146](https://github.com/apache/arrow-datafusion/pull/8146) (jonahgao) +- feat: to_array_of_size for ScalarValue::FixedSizeList [#8225](https://github.com/apache/arrow-datafusion/pull/8225) (wjones127) +- feat:implement calcite style 'levenshtein' string function [#8168](https://github.com/apache/arrow-datafusion/pull/8168) (Syleechan) +- feat: roundtrip FixedSizeList Scalar to protobuf [#8239](https://github.com/apache/arrow-datafusion/pull/8239) (wjones127) +- feat: impl the basic `string_agg` function [#8148](https://github.com/apache/arrow-datafusion/pull/8148) (haohuaijin) +- feat: support simplifying BinaryExpr with arbitrary guarantees in GuaranteeRewriter [#8256](https://github.com/apache/arrow-datafusion/pull/8256) (wjones127) +- feat: support customizing column default values for inserting [#8283](https://github.com/apache/arrow-datafusion/pull/8283) (jonahgao) +- feat:implement sql style 'substr_index' string function [#8272](https://github.com/apache/arrow-datafusion/pull/8272) (Syleechan) +- feat:implement sql style 'find_in_set' string function [#8328](https://github.com/apache/arrow-datafusion/pull/8328) (Syleechan) +- feat: support `LargeList` in `array_empty` [#8321](https://github.com/apache/arrow-datafusion/pull/8321) (Weijun-H) +- feat: support `LargeList` in `make_array` and `array_length` [#8121](https://github.com/apache/arrow-datafusion/pull/8121) (Weijun-H) +- feat: ScalarValue from String [#8411](https://github.com/apache/arrow-datafusion/pull/8411) (QuenKar) +- feat: support `LargeList` for `array_has`, `array_has_all` and `array_has_any` [#8322](https://github.com/apache/arrow-datafusion/pull/8322) (Weijun-H) +- feat: customize column default values for external tables [#8415](https://github.com/apache/arrow-datafusion/pull/8415) (jonahgao) +- feat: Support `array_sort`(`list_sort`) [#8279](https://github.com/apache/arrow-datafusion/pull/8279) (Asura7969) +- feat: support `InterleaveExecNode` in the proto [#8460](https://github.com/apache/arrow-datafusion/pull/8460) (liukun4515) + +**Fixed bugs:** + +- fix: Timestamp with timezone not considered `join on` [#8150](https://github.com/apache/arrow-datafusion/pull/8150) (ACking-you) +- fix: wrong result of range function [#8313](https://github.com/apache/arrow-datafusion/pull/8313) (smallzhongfeng) +- fix: make `ntile` work in some corner cases [#8371](https://github.com/apache/arrow-datafusion/pull/8371) (haohuaijin) +- fix: Changed labeler.yml to latest format [#8431](https://github.com/apache/arrow-datafusion/pull/8431) (viirya) +- fix: Literal in `ORDER BY` window definition should not be an ordinal referring to relation column [#8419](https://github.com/apache/arrow-datafusion/pull/8419) (viirya) +- fix: ORDER BY window definition should work on null literal [#8444](https://github.com/apache/arrow-datafusion/pull/8444) (viirya) +- fix: RANGE frame for corner cases with empty ORDER BY clause should be treated as constant sort [#8445](https://github.com/apache/arrow-datafusion/pull/8445) (viirya) +- fix: don't unifies projection if expr is non-trival [#8454](https://github.com/apache/arrow-datafusion/pull/8454) (haohuaijin) +- fix: support uppercase when parsing `Interval` [#8478](https://github.com/apache/arrow-datafusion/pull/8478) (QuenKar) + +**Documentation updates:** + +- Library Guide: Add Using the DataFrame API [#8319](https://github.com/apache/arrow-datafusion/pull/8319) (Veeupup) +- Minor: Add installation link to README.md [#8389](https://github.com/apache/arrow-datafusion/pull/8389) (Weijun-H) + +**Merged pull requests:** + +- Fix typo in partitioning.rs [#8134](https://github.com/apache/arrow-datafusion/pull/8134) (lewiszlw) +- Implement `DISTINCT ON` from Postgres [#7981](https://github.com/apache/arrow-datafusion/pull/7981) (gruuya) +- Prepare 33.0.0-rc2 [#8144](https://github.com/apache/arrow-datafusion/pull/8144) (andygrove) +- Avoid concat in `array_append` [#8137](https://github.com/apache/arrow-datafusion/pull/8137) (jayzhan211) +- Replace macro with function for array_remove [#8106](https://github.com/apache/arrow-datafusion/pull/8106) (jayzhan211) +- Implement `array_union` [#7897](https://github.com/apache/arrow-datafusion/pull/7897) (edmondop) +- Minor: Document `ExecutionPlan::equivalence_properties` more thoroughly [#8128](https://github.com/apache/arrow-datafusion/pull/8128) (alamb) +- feat: show statistics in explain verbose [#8113](https://github.com/apache/arrow-datafusion/pull/8113) (NGA-TRAN) +- feat:implement postgres style 'overlay' string function [#8117](https://github.com/apache/arrow-datafusion/pull/8117) (Syleechan) +- Minor: Encapsulate `LeftJoinData` into a struct (rather than anonymous enum) and add comments [#8153](https://github.com/apache/arrow-datafusion/pull/8153) (alamb) +- Update sqllogictest requirement from 0.18.0 to 0.19.0 [#8163](https://github.com/apache/arrow-datafusion/pull/8163) (dependabot[bot]) +- feat: fill missing values with NULLs while inserting [#8146](https://github.com/apache/arrow-datafusion/pull/8146) (jonahgao) +- Introduce return type for aggregate sum [#8141](https://github.com/apache/arrow-datafusion/pull/8141) (jayzhan211) +- implement range/generate_series func [#8140](https://github.com/apache/arrow-datafusion/pull/8140) (Veeupup) +- Encapsulate `EquivalenceClass` into a struct [#8034](https://github.com/apache/arrow-datafusion/pull/8034) (alamb) +- Revert "Minor: remove unnecessary projection in `single_distinct_to_g… [#8176](https://github.com/apache/arrow-datafusion/pull/8176) (NGA-TRAN) +- Preserve all of the valid orderings during merging. [#8169](https://github.com/apache/arrow-datafusion/pull/8169) (mustafasrepo) +- Make fields of `ScalarUDF` , `AggregateUDF` and `WindowUDF` non `pub` [#8079](https://github.com/apache/arrow-datafusion/pull/8079) (alamb) +- Fix logical conflicts [#8187](https://github.com/apache/arrow-datafusion/pull/8187) (tustvold) +- Minor: Update JoinHashMap comment example to make it clearer [#8154](https://github.com/apache/arrow-datafusion/pull/8154) (alamb) +- Implement StreamTable and StreamTableProvider (#7994) [#8021](https://github.com/apache/arrow-datafusion/pull/8021) (tustvold) +- [MINOR]: Remove unused Results [#8189](https://github.com/apache/arrow-datafusion/pull/8189) (mustafasrepo) +- Minor: clean up the code based on clippy [#8179](https://github.com/apache/arrow-datafusion/pull/8179) (Weijun-H) +- Minor: simplify filter statistics code [#8174](https://github.com/apache/arrow-datafusion/pull/8174) (alamb) +- Replace macro with function for `array_position` and `array_positions` [#8170](https://github.com/apache/arrow-datafusion/pull/8170) (jayzhan211) +- Add Library Guide for User Defined Functions: Window/Aggregate [#8171](https://github.com/apache/arrow-datafusion/pull/8171) (Veeupup) +- Add more stream docs [#8192](https://github.com/apache/arrow-datafusion/pull/8192) (tustvold) +- Implement func `array_pop_front` [#8142](https://github.com/apache/arrow-datafusion/pull/8142) (Veeupup) +- Moving arrow_files SQL tests to sqllogictest [#8217](https://github.com/apache/arrow-datafusion/pull/8217) (edmondop) +- fix regression in the use of name in ProjectionPushdown [#8219](https://github.com/apache/arrow-datafusion/pull/8219) (alamb) +- [MINOR]: Fix column indices in the planning tests [#8191](https://github.com/apache/arrow-datafusion/pull/8191) (mustafasrepo) +- Remove unnecessary reassignment [#8232](https://github.com/apache/arrow-datafusion/pull/8232) (qrilka) +- Update itertools requirement from 0.11 to 0.12 [#8233](https://github.com/apache/arrow-datafusion/pull/8233) (crepererum) +- Port tests in subqueries.rs to sqllogictest [#8231](https://github.com/apache/arrow-datafusion/pull/8231) (PsiACE) +- feat: make FixedSizeList scalar also an ArrayRef [#8221](https://github.com/apache/arrow-datafusion/pull/8221) (wjones127) +- Add versions to datafusion dependencies [#8238](https://github.com/apache/arrow-datafusion/pull/8238) (andygrove) +- feat: to_array_of_size for ScalarValue::FixedSizeList [#8225](https://github.com/apache/arrow-datafusion/pull/8225) (wjones127) +- feat:implement calcite style 'levenshtein' string function [#8168](https://github.com/apache/arrow-datafusion/pull/8168) (Syleechan) +- feat: roundtrip FixedSizeList Scalar to protobuf [#8239](https://github.com/apache/arrow-datafusion/pull/8239) (wjones127) +- Update prost-build requirement from =0.12.1 to =0.12.2 [#8244](https://github.com/apache/arrow-datafusion/pull/8244) (dependabot[bot]) +- Minor: Port tests in `displayable.rs` to sqllogictest [#8246](https://github.com/apache/arrow-datafusion/pull/8246) (Weijun-H) +- Minor: add `with_estimated_selectivity ` to Precision [#8177](https://github.com/apache/arrow-datafusion/pull/8177) (alamb) +- fix: Timestamp with timezone not considered `join on` [#8150](https://github.com/apache/arrow-datafusion/pull/8150) (ACking-you) +- Replace macro in array_array to remove duplicate codes [#8252](https://github.com/apache/arrow-datafusion/pull/8252) (Veeupup) +- Port tests in projection.rs to sqllogictest [#8240](https://github.com/apache/arrow-datafusion/pull/8240) (PsiACE) +- Introduce `array_except` function [#8135](https://github.com/apache/arrow-datafusion/pull/8135) (jayzhan211) +- Port tests in `describe.rs` to sqllogictest [#8242](https://github.com/apache/arrow-datafusion/pull/8242) (Asura7969) +- Remove FileWriterMode and ListingTableInsertMode (#7994) [#8017](https://github.com/apache/arrow-datafusion/pull/8017) (tustvold) +- Minor: clean up the code based on Clippy [#8257](https://github.com/apache/arrow-datafusion/pull/8257) (Weijun-H) +- Update arrow 49.0.0 and object_store 0.8.0 [#8029](https://github.com/apache/arrow-datafusion/pull/8029) (tustvold) +- feat: impl the basic `string_agg` function [#8148](https://github.com/apache/arrow-datafusion/pull/8148) (haohuaijin) +- Minor: Make schema of grouping set columns nullable [#8248](https://github.com/apache/arrow-datafusion/pull/8248) (markusa380) +- feat: support simplifying BinaryExpr with arbitrary guarantees in GuaranteeRewriter [#8256](https://github.com/apache/arrow-datafusion/pull/8256) (wjones127) +- Making stream joins extensible: A new Trait implementation for SHJ [#8234](https://github.com/apache/arrow-datafusion/pull/8234) (metesynnada) +- Don't Canonicalize Filesystem Paths in ListingTableUrl / support new external tables for files that do not (yet) exist [#8014](https://github.com/apache/arrow-datafusion/pull/8014) (tustvold) +- Minor: Add sql level test for inserting into non-existent directory [#8278](https://github.com/apache/arrow-datafusion/pull/8278) (alamb) +- Replace `array_has/array_has_all/array_has_any` macro to remove duplicate code [#8263](https://github.com/apache/arrow-datafusion/pull/8263) (Veeupup) +- Fix bug in field level metadata matching code [#8286](https://github.com/apache/arrow-datafusion/pull/8286) (alamb) +- Refactor Interval Arithmetic Updates [#8276](https://github.com/apache/arrow-datafusion/pull/8276) (berkaysynnada) +- [MINOR]: Remove unecessary orderings from the final plan [#8289](https://github.com/apache/arrow-datafusion/pull/8289) (mustafasrepo) +- consistent logical & physical `NTILE` return types [#8270](https://github.com/apache/arrow-datafusion/pull/8270) (korowa) +- make `array_union`/`array_except`/`array_intersect` handle empty/null arrays rightly [#8269](https://github.com/apache/arrow-datafusion/pull/8269) (Veeupup) +- improve file path validation when reading parquet [#8267](https://github.com/apache/arrow-datafusion/pull/8267) (Weijun-H) +- [Benchmarks] Make `partitions` default to number of cores instead of 2 [#8292](https://github.com/apache/arrow-datafusion/pull/8292) (andygrove) +- Update prost-build requirement from =0.12.2 to =0.12.3 [#8298](https://github.com/apache/arrow-datafusion/pull/8298) (dependabot[bot]) +- Fix Display for List [#8261](https://github.com/apache/arrow-datafusion/pull/8261) (jayzhan211) +- feat: support customizing column default values for inserting [#8283](https://github.com/apache/arrow-datafusion/pull/8283) (jonahgao) +- support `LargeList` for `arrow_cast`, support `ScalarValue::LargeList` [#8290](https://github.com/apache/arrow-datafusion/pull/8290) (Weijun-H) +- Minor: remove useless clone based on Clippy [#8300](https://github.com/apache/arrow-datafusion/pull/8300) (Weijun-H) +- Calculate ordering equivalence for expressions (rather than just columns) [#8281](https://github.com/apache/arrow-datafusion/pull/8281) (mustafasrepo) +- Fix sqllogictests link in contributor-guide/index.md [#8314](https://github.com/apache/arrow-datafusion/pull/8314) (qrilka) +- Refactor: Unify `Expr::ScalarFunction` and `Expr::ScalarUDF`, introduce unresolved functions by name [#8258](https://github.com/apache/arrow-datafusion/pull/8258) (2010YOUY01) +- Support no distinct aggregate sum/min/max in `single_distinct_to_group_by` rule [#8266](https://github.com/apache/arrow-datafusion/pull/8266) (haohuaijin) +- feat:implement sql style 'substr_index' string function [#8272](https://github.com/apache/arrow-datafusion/pull/8272) (Syleechan) +- Fixing issues with for timestamp literals [#8193](https://github.com/apache/arrow-datafusion/pull/8193) (comphead) +- Projection Pushdown over StreamingTableExec [#8299](https://github.com/apache/arrow-datafusion/pull/8299) (berkaysynnada) +- minor: fix documentation [#8323](https://github.com/apache/arrow-datafusion/pull/8323) (comphead) +- fix: wrong result of range function [#8313](https://github.com/apache/arrow-datafusion/pull/8313) (smallzhongfeng) +- Minor: rename parquet.rs to parquet/mod.rs [#8301](https://github.com/apache/arrow-datafusion/pull/8301) (alamb) +- refactor: output ordering [#8304](https://github.com/apache/arrow-datafusion/pull/8304) (QuenKar) +- Update substrait requirement from 0.19.0 to 0.20.0 [#8339](https://github.com/apache/arrow-datafusion/pull/8339) (dependabot[bot]) +- Port tests in `aggregates.rs` to sqllogictest [#8316](https://github.com/apache/arrow-datafusion/pull/8316) (edmondop) +- Library Guide: Add Using the DataFrame API [#8319](https://github.com/apache/arrow-datafusion/pull/8319) (Veeupup) +- Port tests in limit.rs to sqllogictest [#8315](https://github.com/apache/arrow-datafusion/pull/8315) (zhangxffff) +- move array function unit_tests to sqllogictest [#8332](https://github.com/apache/arrow-datafusion/pull/8332) (Veeupup) +- NTH_VALUE reverse support [#8327](https://github.com/apache/arrow-datafusion/pull/8327) (mustafasrepo) +- Optimize Projections during Logical Plan [#8340](https://github.com/apache/arrow-datafusion/pull/8340) (mustafasrepo) +- [MINOR]: Move merge projections tests to under optimize projections [#8352](https://github.com/apache/arrow-datafusion/pull/8352) (mustafasrepo) +- Add `quote` and `escape` attributes to create csv external table [#8351](https://github.com/apache/arrow-datafusion/pull/8351) (Asura7969) +- Minor: Add DataFrame test [#8341](https://github.com/apache/arrow-datafusion/pull/8341) (alamb) +- Minor: clean up the code based on Clippy [#8359](https://github.com/apache/arrow-datafusion/pull/8359) (Weijun-H) +- Minor: Make it easier to work with Expr::ScalarFunction [#8350](https://github.com/apache/arrow-datafusion/pull/8350) (alamb) +- Minor: Move some datafusion-optimizer::utils down to datafusion-expr::utils [#8354](https://github.com/apache/arrow-datafusion/pull/8354) (Jesse-Bakker) +- Minor: Make `BuiltInScalarFunction::alias` a method [#8349](https://github.com/apache/arrow-datafusion/pull/8349) (alamb) +- Extract parquet statistics to its own module, add tests [#8294](https://github.com/apache/arrow-datafusion/pull/8294) (alamb) +- feat:implement sql style 'find_in_set' string function [#8328](https://github.com/apache/arrow-datafusion/pull/8328) (Syleechan) +- Support LargeUtf8 to Temporal Coercion [#8357](https://github.com/apache/arrow-datafusion/pull/8357) (jayzhan211) +- Refactor aggregate function handling [#8358](https://github.com/apache/arrow-datafusion/pull/8358) (Weijun-H) +- Implement Aliases for ScalarUDF [#8360](https://github.com/apache/arrow-datafusion/pull/8360) (Veeupup) +- Minor: Remove unnecessary name field in `ScalarFunctionDefintion` [#8365](https://github.com/apache/arrow-datafusion/pull/8365) (alamb) +- feat: support `LargeList` in `array_empty` [#8321](https://github.com/apache/arrow-datafusion/pull/8321) (Weijun-H) +- Double type argument for to_timestamp function [#8159](https://github.com/apache/arrow-datafusion/pull/8159) (spaydar) +- Support User Defined Table Function [#8306](https://github.com/apache/arrow-datafusion/pull/8306) (Veeupup) +- Document timestamp input limits [#8369](https://github.com/apache/arrow-datafusion/pull/8369) (comphead) +- fix: make `ntile` work in some corner cases [#8371](https://github.com/apache/arrow-datafusion/pull/8371) (haohuaijin) +- Minor: Refactor array_union function to use a generic union_arrays function [#8381](https://github.com/apache/arrow-datafusion/pull/8381) (Weijun-H) +- Minor: Refactor function argument handling in `ScalarFunctionDefinition` [#8387](https://github.com/apache/arrow-datafusion/pull/8387) (Weijun-H) +- Materialize dictionaries in group keys [#8291](https://github.com/apache/arrow-datafusion/pull/8291) (qrilka) +- Rewrite `array_ndims` to fix List(Null) handling [#8320](https://github.com/apache/arrow-datafusion/pull/8320) (jayzhan211) +- Docs: Improve the documentation on `ScalarValue` [#8378](https://github.com/apache/arrow-datafusion/pull/8378) (alamb) +- Avoid concat for `array_replace` [#8337](https://github.com/apache/arrow-datafusion/pull/8337) (jayzhan211) +- add a summary table to benchmark compare output [#8399](https://github.com/apache/arrow-datafusion/pull/8399) (razeghi71) +- Refactors on TreeNode Implementations [#8395](https://github.com/apache/arrow-datafusion/pull/8395) (berkaysynnada) +- feat: support `LargeList` in `make_array` and `array_length` [#8121](https://github.com/apache/arrow-datafusion/pull/8121) (Weijun-H) +- remove `unalias` TableScan filters when create Physical Filter [#8404](https://github.com/apache/arrow-datafusion/pull/8404) (jackwener) +- Update custom-table-providers.md [#8409](https://github.com/apache/arrow-datafusion/pull/8409) (nickpoorman) +- fix transforming `LogicalPlan::Explain` use `TreeNode::transform` fails [#8400](https://github.com/apache/arrow-datafusion/pull/8400) (haohuaijin) +- Docs: Fix `array_except` documentation example error [#8407](https://github.com/apache/arrow-datafusion/pull/8407) (Asura7969) +- Support named query parameters [#8384](https://github.com/apache/arrow-datafusion/pull/8384) (Asura7969) +- Minor: Add installation link to README.md [#8389](https://github.com/apache/arrow-datafusion/pull/8389) (Weijun-H) +- Update code comment for the cases of regularized RANGE frame and add tests for ORDER BY cases with RANGE frame [#8410](https://github.com/apache/arrow-datafusion/pull/8410) (viirya) +- Minor: Add example with parameters to LogicalPlan [#8418](https://github.com/apache/arrow-datafusion/pull/8418) (alamb) +- Minor: Improve `PruningPredicate` documentation [#8394](https://github.com/apache/arrow-datafusion/pull/8394) (alamb) +- feat: ScalarValue from String [#8411](https://github.com/apache/arrow-datafusion/pull/8411) (QuenKar) +- Bump actions/labeler from 4.3.0 to 5.0.0 [#8422](https://github.com/apache/arrow-datafusion/pull/8422) (dependabot[bot]) +- Update sqlparser requirement from 0.39.0 to 0.40.0 [#8338](https://github.com/apache/arrow-datafusion/pull/8338) (dependabot[bot]) +- feat: support `LargeList` for `array_has`, `array_has_all` and `array_has_any` [#8322](https://github.com/apache/arrow-datafusion/pull/8322) (Weijun-H) +- Union `schema` can't be a subset of the child schema [#8408](https://github.com/apache/arrow-datafusion/pull/8408) (jackwener) +- Move `PartitionSearchMode` into datafusion_physical_plan, rename to `InputOrderMode` [#8364](https://github.com/apache/arrow-datafusion/pull/8364) (alamb) +- Make filter selectivity for statistics configurable [#8243](https://github.com/apache/arrow-datafusion/pull/8243) (edmondop) +- fix: Changed labeler.yml to latest format [#8431](https://github.com/apache/arrow-datafusion/pull/8431) (viirya) +- Minor: Use `ScalarValue::from` impl for strings [#8429](https://github.com/apache/arrow-datafusion/pull/8429) (alamb) +- Support crossjoin in substrait. [#8427](https://github.com/apache/arrow-datafusion/pull/8427) (my-vegetable-has-exploded) +- Fix ambiguous reference when aliasing in combination with `ORDER BY` [#8425](https://github.com/apache/arrow-datafusion/pull/8425) (Asura7969) +- Minor: convert marcro `list-slice` and `slice` to function [#8424](https://github.com/apache/arrow-datafusion/pull/8424) (Weijun-H) +- Remove macro in iter_to_array for List [#8414](https://github.com/apache/arrow-datafusion/pull/8414) (jayzhan211) +- fix: Literal in `ORDER BY` window definition should not be an ordinal referring to relation column [#8419](https://github.com/apache/arrow-datafusion/pull/8419) (viirya) +- feat: customize column default values for external tables [#8415](https://github.com/apache/arrow-datafusion/pull/8415) (jonahgao) +- feat: Support `array_sort`(`list_sort`) [#8279](https://github.com/apache/arrow-datafusion/pull/8279) (Asura7969) +- Bugfix: Remove df-cli specific SQL statment options before executing with DataFusion [#8426](https://github.com/apache/arrow-datafusion/pull/8426) (devinjdangelo) +- Detect when filters on unique constraints make subqueries scalar [#8312](https://github.com/apache/arrow-datafusion/pull/8312) (Jesse-Bakker) +- Add alias check to optimize projections merge [#8438](https://github.com/apache/arrow-datafusion/pull/8438) (mustafasrepo) +- Fix PartialOrd for ScalarValue::List/FixSizeList/LargeList [#8253](https://github.com/apache/arrow-datafusion/pull/8253) (jayzhan211) +- Support parquet_metadata for datafusion-cli [#8413](https://github.com/apache/arrow-datafusion/pull/8413) (Veeupup) +- Fix bug in optimizing a nested count [#8459](https://github.com/apache/arrow-datafusion/pull/8459) (Dandandan) +- Bump actions/setup-python from 4 to 5 [#8449](https://github.com/apache/arrow-datafusion/pull/8449) (dependabot[bot]) +- fix: ORDER BY window definition should work on null literal [#8444](https://github.com/apache/arrow-datafusion/pull/8444) (viirya) +- flx clippy warnings [#8455](https://github.com/apache/arrow-datafusion/pull/8455) (waynexia) +- fix: RANGE frame for corner cases with empty ORDER BY clause should be treated as constant sort [#8445](https://github.com/apache/arrow-datafusion/pull/8445) (viirya) +- Preserve `dict_id` on `Field` during serde roundtrip [#8457](https://github.com/apache/arrow-datafusion/pull/8457) (avantgardnerio) +- feat: support `InterleaveExecNode` in the proto [#8460](https://github.com/apache/arrow-datafusion/pull/8460) (liukun4515) +- [BUG FIX]: Proper Empty Batch handling in window execution [#8466](https://github.com/apache/arrow-datafusion/pull/8466) (mustafasrepo) +- Minor: update `cast` [#8458](https://github.com/apache/arrow-datafusion/pull/8458) (Weijun-H) +- fix: don't unifies projection if expr is non-trival [#8454](https://github.com/apache/arrow-datafusion/pull/8454) (haohuaijin) +- Minor: Add new bloom filter predicate tests [#8433](https://github.com/apache/arrow-datafusion/pull/8433) (alamb) +- Add PRIMARY KEY Aggregate support to dataframe API [#8356](https://github.com/apache/arrow-datafusion/pull/8356) (mustafasrepo) +- Minor: refactor `data_trunc` to reduce duplicated code [#8430](https://github.com/apache/arrow-datafusion/pull/8430) (Weijun-H) +- Support array_distinct function. [#8268](https://github.com/apache/arrow-datafusion/pull/8268) (my-vegetable-has-exploded) +- Add primary key support to stream table [#8467](https://github.com/apache/arrow-datafusion/pull/8467) (mustafasrepo) +- Add `evaluate_demo` and `range_analysis_demo` to Expr examples [#8377](https://github.com/apache/arrow-datafusion/pull/8377) (alamb) +- Minor: fix function name typo [#8473](https://github.com/apache/arrow-datafusion/pull/8473) (Weijun-H) +- Minor: Fix comment typo in table.rs: s/indentical/identical/ [#8469](https://github.com/apache/arrow-datafusion/pull/8469) (KeunwooLee-at) +- Remove `define_array_slice` and reuse `array_slice` for `array_pop_front/back` [#8401](https://github.com/apache/arrow-datafusion/pull/8401) (jayzhan211) +- Minor: refactor `trim` to clean up duplicated code [#8434](https://github.com/apache/arrow-datafusion/pull/8434) (Weijun-H) +- Split `EmptyExec` into `PlaceholderRowExec` [#8446](https://github.com/apache/arrow-datafusion/pull/8446) (razeghi71) +- Enable non-uniform field type for structs created in DataFusion [#8463](https://github.com/apache/arrow-datafusion/pull/8463) (dlovell) +- Minor: Add multi ordering test for array agg order [#8439](https://github.com/apache/arrow-datafusion/pull/8439) (jayzhan211) +- Sort filenames when reading parquet to ensure consistent schema [#6629](https://github.com/apache/arrow-datafusion/pull/6629) (thomas-k-cameron) +- Minor: Improve comments in EnforceDistribution tests [#8474](https://github.com/apache/arrow-datafusion/pull/8474) (alamb) +- fix: support uppercase when parsing `Interval` [#8478](https://github.com/apache/arrow-datafusion/pull/8478) (QuenKar) +- Better Equivalence (ordering and exact equivalence) Propagation through ProjectionExec [#8484](https://github.com/apache/arrow-datafusion/pull/8484) (mustafasrepo) diff --git a/docs/Cargo.toml b/docs/Cargo.toml index 4d01466924f9..813335e30f77 100644 --- a/docs/Cargo.toml +++ b/docs/Cargo.toml @@ -29,4 +29,4 @@ authors = { workspace = true } rust-version = "1.70" [dependencies] -datafusion = { path = "../datafusion/core", version = "33.0.0", default-features = false } +datafusion = { path = "../datafusion/core", version = "34.0.0", default-features = false } diff --git a/docs/source/user-guide/configs.md b/docs/source/user-guide/configs.md index d5a43e429e09..6fb5cc4ca870 100644 --- a/docs/source/user-guide/configs.md +++ b/docs/source/user-guide/configs.md @@ -64,7 +64,7 @@ Environment variables are read during `SessionConfig` initialisation so they mus | datafusion.execution.parquet.statistics_enabled | NULL | Sets if statistics are enabled for any column Valid values are: "none", "chunk", and "page" These values are not case sensitive. If NULL, uses default parquet writer setting | | datafusion.execution.parquet.max_statistics_size | NULL | Sets max statistics size for any column. If NULL, uses default parquet writer setting | | datafusion.execution.parquet.max_row_group_size | 1048576 | Sets maximum number of rows in a row group | -| datafusion.execution.parquet.created_by | datafusion version 33.0.0 | Sets "created by" property | +| datafusion.execution.parquet.created_by | datafusion version 34.0.0 | Sets "created by" property | | datafusion.execution.parquet.column_index_truncate_length | NULL | Sets column index truncate length | | datafusion.execution.parquet.data_page_row_count_limit | 18446744073709551615 | Sets best effort maximum number of rows in data page | | datafusion.execution.parquet.encoding | NULL | 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 | From cf2de9b22c8e7c4c4b80bdb82ae1353e36a5af51 Mon Sep 17 00:00:00 2001 From: Ruihang Xia Date: Thu, 14 Dec 2023 10:42:16 +0800 Subject: [PATCH 423/572] refactor: use ExprBuilder to consume substrait expr and use macro to generate error (#8515) * refactor: use ExprBuilder to consume substrait expr Signed-off-by: Ruihang Xia * use macro to generate error Signed-off-by: Ruihang Xia --------- Signed-off-by: Ruihang Xia --- datafusion/common/src/error.rs | 3 + .../substrait/src/logical_plan/consumer.rs | 324 +++++++++--------- 2 files changed, 158 insertions(+), 169 deletions(-) diff --git a/datafusion/common/src/error.rs b/datafusion/common/src/error.rs index 4ae30ae86cdd..56b52bd73f9b 100644 --- a/datafusion/common/src/error.rs +++ b/datafusion/common/src/error.rs @@ -517,6 +517,9 @@ make_error!(not_impl_err, not_impl_datafusion_err, NotImplemented); // Exposes a macro to create `DataFusionError::Execution` make_error!(exec_err, exec_datafusion_err, Execution); +// Exposes a macro to create `DataFusionError::Substrait` +make_error!(substrait_err, substrait_datafusion_err, Substrait); + // Exposes a macro to create `DataFusionError::SQL` #[macro_export] macro_rules! sql_err { diff --git a/datafusion/substrait/src/logical_plan/consumer.rs b/datafusion/substrait/src/logical_plan/consumer.rs index ffc9d094ab91..f6b556fc6448 100644 --- a/datafusion/substrait/src/logical_plan/consumer.rs +++ b/datafusion/substrait/src/logical_plan/consumer.rs @@ -17,7 +17,9 @@ use async_recursion::async_recursion; use datafusion::arrow::datatypes::{DataType, Field, TimeUnit}; -use datafusion::common::{not_impl_err, DFField, DFSchema, DFSchemaRef}; +use datafusion::common::{ + not_impl_err, substrait_datafusion_err, substrait_err, DFField, DFSchema, DFSchemaRef, +}; use datafusion::execution::FunctionRegistry; use datafusion::logical_expr::{ @@ -73,16 +75,7 @@ use crate::variation_const::{ enum ScalarFunctionType { Builtin(BuiltinScalarFunction), Op(Operator), - /// [Expr::Not] - Not, - /// [Expr::Like] Used for filtering rows based on the given wildcard pattern. Case sensitive - Like, - /// [Expr::Like] Case insensitive operator counterpart of `Like` - ILike, - /// [Expr::IsNull] - IsNull, - /// [Expr::IsNotNull] - IsNotNull, + Expr(BuiltinExprBuilder), } pub fn name_to_op(name: &str) -> Result { @@ -127,14 +120,11 @@ fn scalar_function_type_from_str(name: &str) -> Result { return Ok(ScalarFunctionType::Builtin(fun)); } - match name { - "not" => Ok(ScalarFunctionType::Not), - "like" => Ok(ScalarFunctionType::Like), - "ilike" => Ok(ScalarFunctionType::ILike), - "is_null" => Ok(ScalarFunctionType::IsNull), - "is_not_null" => Ok(ScalarFunctionType::IsNotNull), - others => not_impl_err!("Unsupported function name: {others:?}"), + if let Some(builder) = BuiltinExprBuilder::try_from_name(name) { + return Ok(ScalarFunctionType::Expr(builder)); } + + not_impl_err!("Unsupported function name: {name:?}") } fn split_eq_and_noneq_join_predicate_with_nulls_equality( @@ -519,9 +509,7 @@ pub async fn from_substrait_rel( }, Some(RelType::ExtensionLeaf(extension)) => { let Some(ext_detail) = &extension.detail else { - return Err(DataFusionError::Substrait( - "Unexpected empty detail in ExtensionLeafRel".to_string(), - )); + return substrait_err!("Unexpected empty detail in ExtensionLeafRel"); }; let plan = ctx .state() @@ -531,18 +519,16 @@ pub async fn from_substrait_rel( } Some(RelType::ExtensionSingle(extension)) => { let Some(ext_detail) = &extension.detail else { - return Err(DataFusionError::Substrait( - "Unexpected empty detail in ExtensionSingleRel".to_string(), - )); + return substrait_err!("Unexpected empty detail in ExtensionSingleRel"); }; let plan = ctx .state() .serializer_registry() .deserialize_logical_plan(&ext_detail.type_url, &ext_detail.value)?; let Some(input_rel) = &extension.input else { - return Err(DataFusionError::Substrait( - "ExtensionSingleRel doesn't contains input rel. Try use ExtensionLeafRel instead".to_string() - )); + return substrait_err!( + "ExtensionSingleRel doesn't contains input rel. Try use ExtensionLeafRel instead" + ); }; let input_plan = from_substrait_rel(ctx, input_rel, extensions).await?; let plan = plan.from_template(&plan.expressions(), &[input_plan]); @@ -550,9 +536,7 @@ pub async fn from_substrait_rel( } Some(RelType::ExtensionMulti(extension)) => { let Some(ext_detail) = &extension.detail else { - return Err(DataFusionError::Substrait( - "Unexpected empty detail in ExtensionSingleRel".to_string(), - )); + return substrait_err!("Unexpected empty detail in ExtensionSingleRel"); }; let plan = ctx .state() @@ -881,64 +865,8 @@ pub async fn from_substrait_rex( ), } } - ScalarFunctionType::Not => { - let arg = f.arguments.first().ok_or_else(|| { - DataFusionError::Substrait( - "expect one argument for `NOT` expr".to_string(), - ) - })?; - match &arg.arg_type { - Some(ArgType::Value(e)) => { - let expr = from_substrait_rex(e, input_schema, extensions) - .await? - .as_ref() - .clone(); - Ok(Arc::new(Expr::Not(Box::new(expr)))) - } - _ => not_impl_err!("Invalid arguments for Not expression"), - } - } - ScalarFunctionType::Like => { - make_datafusion_like(false, f, input_schema, extensions).await - } - ScalarFunctionType::ILike => { - make_datafusion_like(true, f, input_schema, extensions).await - } - ScalarFunctionType::IsNull => { - let arg = f.arguments.first().ok_or_else(|| { - DataFusionError::Substrait( - "expect one argument for `IS NULL` expr".to_string(), - ) - })?; - match &arg.arg_type { - Some(ArgType::Value(e)) => { - let expr = from_substrait_rex(e, input_schema, extensions) - .await? - .as_ref() - .clone(); - Ok(Arc::new(Expr::IsNull(Box::new(expr)))) - } - _ => not_impl_err!("Invalid arguments for IS NULL expression"), - } - } - ScalarFunctionType::IsNotNull => { - let arg = f.arguments.first().ok_or_else(|| { - DataFusionError::Substrait( - "expect one argument for `IS NOT NULL` expr".to_string(), - ) - })?; - match &arg.arg_type { - Some(ArgType::Value(e)) => { - let expr = from_substrait_rex(e, input_schema, extensions) - .await? - .as_ref() - .clone(); - Ok(Arc::new(Expr::IsNotNull(Box::new(expr)))) - } - _ => { - not_impl_err!("Invalid arguments for IS NOT NULL expression") - } - } + ScalarFunctionType::Expr(builder) => { + builder.build(f, input_schema, extensions).await } } } @@ -960,9 +888,7 @@ pub async fn from_substrait_rex( ), from_substrait_type(output_type)?, )))), - None => Err(DataFusionError::Substrait( - "Cast experssion without output type is not allowed".to_string(), - )), + None => substrait_err!("Cast experssion without output type is not allowed"), }, Some(RexType::WindowFunction(window)) => { let fun = match extensions.get(&window.function_reference) { @@ -1087,9 +1013,7 @@ fn from_substrait_type(dt: &substrait::proto::Type) -> Result { r#type::Kind::List(list) => { let inner_type = from_substrait_type(list.r#type.as_ref().ok_or_else(|| { - DataFusionError::Substrait( - "List type must have inner type".to_string(), - ) + substrait_datafusion_err!("List type must have inner type") })?)?; let field = Arc::new(Field::new("list_item", inner_type, true)); match list.type_variation_reference { @@ -1141,9 +1065,7 @@ fn from_substrait_bound( } } }, - None => Err(DataFusionError::Substrait( - "WindowFunction missing Substrait Bound kind".to_string(), - )), + None => substrait_err!("WindowFunction missing Substrait Bound kind"), }, None => { if is_lower { @@ -1162,36 +1084,28 @@ pub(crate) fn from_substrait_literal(lit: &Literal) -> Result { DEFAULT_TYPE_REF => ScalarValue::Int8(Some(*n as i8)), UNSIGNED_INTEGER_TYPE_REF => ScalarValue::UInt8(Some(*n as u8)), others => { - return Err(DataFusionError::Substrait(format!( - "Unknown type variation reference {others}", - ))); + return substrait_err!("Unknown type variation reference {others}"); } }, Some(LiteralType::I16(n)) => match lit.type_variation_reference { DEFAULT_TYPE_REF => ScalarValue::Int16(Some(*n as i16)), UNSIGNED_INTEGER_TYPE_REF => ScalarValue::UInt16(Some(*n as u16)), others => { - return Err(DataFusionError::Substrait(format!( - "Unknown type variation reference {others}", - ))); + return substrait_err!("Unknown type variation reference {others}"); } }, Some(LiteralType::I32(n)) => match lit.type_variation_reference { DEFAULT_TYPE_REF => ScalarValue::Int32(Some(*n)), UNSIGNED_INTEGER_TYPE_REF => ScalarValue::UInt32(Some(*n as u32)), others => { - return Err(DataFusionError::Substrait(format!( - "Unknown type variation reference {others}", - ))); + return substrait_err!("Unknown type variation reference {others}"); } }, Some(LiteralType::I64(n)) => match lit.type_variation_reference { DEFAULT_TYPE_REF => ScalarValue::Int64(Some(*n)), UNSIGNED_INTEGER_TYPE_REF => ScalarValue::UInt64(Some(*n as u64)), others => { - return Err(DataFusionError::Substrait(format!( - "Unknown type variation reference {others}", - ))); + return substrait_err!("Unknown type variation reference {others}"); } }, Some(LiteralType::Fp32(f)) => ScalarValue::Float32(Some(*f)), @@ -1202,9 +1116,7 @@ pub(crate) fn from_substrait_literal(lit: &Literal) -> Result { TIMESTAMP_MICRO_TYPE_REF => ScalarValue::TimestampMicrosecond(Some(*t), None), TIMESTAMP_NANO_TYPE_REF => ScalarValue::TimestampNanosecond(Some(*t), None), others => { - return Err(DataFusionError::Substrait(format!( - "Unknown type variation reference {others}", - ))); + return substrait_err!("Unknown type variation reference {others}"); } }, Some(LiteralType::Date(d)) => ScalarValue::Date32(Some(*d)), @@ -1212,38 +1124,30 @@ pub(crate) fn from_substrait_literal(lit: &Literal) -> Result { DEFAULT_CONTAINER_TYPE_REF => ScalarValue::Utf8(Some(s.clone())), LARGE_CONTAINER_TYPE_REF => ScalarValue::LargeUtf8(Some(s.clone())), others => { - return Err(DataFusionError::Substrait(format!( - "Unknown type variation reference {others}", - ))); + return substrait_err!("Unknown type variation reference {others}"); } }, Some(LiteralType::Binary(b)) => match lit.type_variation_reference { DEFAULT_CONTAINER_TYPE_REF => ScalarValue::Binary(Some(b.clone())), LARGE_CONTAINER_TYPE_REF => ScalarValue::LargeBinary(Some(b.clone())), others => { - return Err(DataFusionError::Substrait(format!( - "Unknown type variation reference {others}", - ))); + return substrait_err!("Unknown type variation reference {others}"); } }, Some(LiteralType::FixedBinary(b)) => { ScalarValue::FixedSizeBinary(b.len() as _, Some(b.clone())) } Some(LiteralType::Decimal(d)) => { - let value: [u8; 16] = - d.value - .clone() - .try_into() - .or(Err(DataFusionError::Substrait( - "Failed to parse decimal value".to_string(), - )))?; + let value: [u8; 16] = d + .value + .clone() + .try_into() + .or(substrait_err!("Failed to parse decimal value"))?; let p = d.precision.try_into().map_err(|e| { - DataFusionError::Substrait(format!( - "Failed to parse decimal precision: {e}" - )) + substrait_datafusion_err!("Failed to parse decimal precision: {e}") })?; let s = d.scale.try_into().map_err(|e| { - DataFusionError::Substrait(format!("Failed to parse decimal scale: {e}")) + substrait_datafusion_err!("Failed to parse decimal scale: {e}") })?; ScalarValue::Decimal128( Some(std::primitive::i128::from_le_bytes(value)), @@ -1341,50 +1245,132 @@ fn from_substrait_null(null_type: &Type) -> Result { } } -async fn make_datafusion_like( - case_insensitive: bool, - f: &ScalarFunction, - input_schema: &DFSchema, - extensions: &HashMap, -) -> Result> { - let fn_name = if case_insensitive { "ILIKE" } else { "LIKE" }; - if f.arguments.len() != 3 { - return not_impl_err!("Expect three arguments for `{fn_name}` expr"); +/// Build [`Expr`] from its name and required inputs. +struct BuiltinExprBuilder { + expr_name: String, +} + +impl BuiltinExprBuilder { + pub fn try_from_name(name: &str) -> Option { + match name { + "not" | "like" | "ilike" | "is_null" | "is_not_null" => Some(Self { + expr_name: name.to_string(), + }), + _ => None, + } } - let Some(ArgType::Value(expr_substrait)) = &f.arguments[0].arg_type else { - return not_impl_err!("Invalid arguments type for `{fn_name}` expr"); - }; - let expr = from_substrait_rex(expr_substrait, input_schema, extensions) - .await? - .as_ref() - .clone(); - let Some(ArgType::Value(pattern_substrait)) = &f.arguments[1].arg_type else { - return not_impl_err!("Invalid arguments type for `{fn_name}` expr"); - }; - let pattern = from_substrait_rex(pattern_substrait, input_schema, extensions) - .await? - .as_ref() - .clone(); - let Some(ArgType::Value(escape_char_substrait)) = &f.arguments[2].arg_type else { - return not_impl_err!("Invalid arguments type for `{fn_name}` expr"); - }; - let escape_char_expr = - from_substrait_rex(escape_char_substrait, input_schema, extensions) + pub async fn build( + self, + f: &ScalarFunction, + input_schema: &DFSchema, + extensions: &HashMap, + ) -> Result> { + match self.expr_name.as_str() { + "not" => Self::build_not_expr(f, input_schema, extensions).await, + "like" => Self::build_like_expr(false, f, input_schema, extensions).await, + "ilike" => Self::build_like_expr(true, f, input_schema, extensions).await, + "is_null" => { + Self::build_is_null_expr(false, f, input_schema, extensions).await + } + "is_not_null" => { + Self::build_is_null_expr(true, f, input_schema, extensions).await + } + _ => { + not_impl_err!("Unsupported builtin expression: {}", self.expr_name) + } + } + } + + async fn build_not_expr( + f: &ScalarFunction, + input_schema: &DFSchema, + extensions: &HashMap, + ) -> Result> { + if f.arguments.len() != 1 { + return not_impl_err!("Expect one argument for `NOT` expr"); + } + let Some(ArgType::Value(expr_substrait)) = &f.arguments[0].arg_type else { + return not_impl_err!("Invalid arguments type for `NOT` expr"); + }; + let expr = from_substrait_rex(expr_substrait, input_schema, extensions) .await? .as_ref() .clone(); - let Expr::Literal(ScalarValue::Utf8(escape_char)) = escape_char_expr else { - return Err(DataFusionError::Substrait(format!( - "Expect Utf8 literal for escape char, but found {escape_char_expr:?}", - ))); - }; + Ok(Arc::new(Expr::Not(Box::new(expr)))) + } + + async fn build_like_expr( + case_insensitive: bool, + f: &ScalarFunction, + input_schema: &DFSchema, + extensions: &HashMap, + ) -> Result> { + let fn_name = if case_insensitive { "ILIKE" } else { "LIKE" }; + if f.arguments.len() != 3 { + return not_impl_err!("Expect three arguments for `{fn_name}` expr"); + } + + let Some(ArgType::Value(expr_substrait)) = &f.arguments[0].arg_type else { + return not_impl_err!("Invalid arguments type for `{fn_name}` expr"); + }; + let expr = from_substrait_rex(expr_substrait, input_schema, extensions) + .await? + .as_ref() + .clone(); + let Some(ArgType::Value(pattern_substrait)) = &f.arguments[1].arg_type else { + return not_impl_err!("Invalid arguments type for `{fn_name}` expr"); + }; + let pattern = from_substrait_rex(pattern_substrait, input_schema, extensions) + .await? + .as_ref() + .clone(); + let Some(ArgType::Value(escape_char_substrait)) = &f.arguments[2].arg_type else { + return not_impl_err!("Invalid arguments type for `{fn_name}` expr"); + }; + let escape_char_expr = + from_substrait_rex(escape_char_substrait, input_schema, extensions) + .await? + .as_ref() + .clone(); + let Expr::Literal(ScalarValue::Utf8(escape_char)) = escape_char_expr else { + return substrait_err!( + "Expect Utf8 literal for escape char, but found {escape_char_expr:?}" + ); + }; - Ok(Arc::new(Expr::Like(Like { - negated: false, - expr: Box::new(expr), - pattern: Box::new(pattern), - escape_char: escape_char.map(|c| c.chars().next().unwrap()), - case_insensitive, - }))) + Ok(Arc::new(Expr::Like(Like { + negated: false, + expr: Box::new(expr), + pattern: Box::new(pattern), + escape_char: escape_char.map(|c| c.chars().next().unwrap()), + case_insensitive, + }))) + } + + async fn build_is_null_expr( + is_not: bool, + f: &ScalarFunction, + input_schema: &DFSchema, + extensions: &HashMap, + ) -> Result> { + let fn_name = if is_not { "IS NOT NULL" } else { "IS NULL" }; + let arg = f.arguments.first().ok_or_else(|| { + substrait_datafusion_err!("expect one argument for `{fn_name}` expr") + })?; + match &arg.arg_type { + Some(ArgType::Value(e)) => { + let expr = from_substrait_rex(e, input_schema, extensions) + .await? + .as_ref() + .clone(); + if is_not { + Ok(Arc::new(Expr::IsNotNull(Box::new(expr)))) + } else { + Ok(Arc::new(Expr::IsNull(Box::new(expr)))) + } + } + _ => substrait_err!("Invalid arguments for `{fn_name}` expression"), + } + } } From 79c17e3f1a9ddc95ad787963524b8c702548ac29 Mon Sep 17 00:00:00 2001 From: Mustafa Akur <106137913+mustafasrepo@users.noreply.github.com> Date: Thu, 14 Dec 2023 09:09:51 +0300 Subject: [PATCH 424/572] Make tests deterministic (#8525) --- .../sqllogictest/test_files/distinct_on.slt | 8 ++--- .../sqllogictest/test_files/groupby.slt | 32 +++++++++---------- 2 files changed, 20 insertions(+), 20 deletions(-) diff --git a/datafusion/sqllogictest/test_files/distinct_on.slt b/datafusion/sqllogictest/test_files/distinct_on.slt index 8a36b49b98c6..9a7117b69b99 100644 --- a/datafusion/sqllogictest/test_files/distinct_on.slt +++ b/datafusion/sqllogictest/test_files/distinct_on.slt @@ -38,9 +38,9 @@ LOCATION '../../testing/data/csv/aggregate_test_100.csv' # Basic example: distinct on the first column project the second one, and # order by the third query TI -SELECT DISTINCT ON (c1) c1, c2 FROM aggregate_test_100 ORDER BY c1, c3; +SELECT DISTINCT ON (c1) c1, c2 FROM aggregate_test_100 ORDER BY c1, c3, c9; ---- -a 5 +a 4 b 4 c 2 d 1 @@ -48,7 +48,7 @@ e 3 # Basic example + reverse order of the selected column query TI -SELECT DISTINCT ON (c1) c1, c2 FROM aggregate_test_100 ORDER BY c1, c3 DESC; +SELECT DISTINCT ON (c1) c1, c2 FROM aggregate_test_100 ORDER BY c1, c3 DESC, c9; ---- a 1 b 5 @@ -58,7 +58,7 @@ e 1 # Basic example + reverse order of the ON column query TI -SELECT DISTINCT ON (c1) c1, c2 FROM aggregate_test_100 ORDER BY c1 DESC, c3; +SELECT DISTINCT ON (c1) c1, c2 FROM aggregate_test_100 ORDER BY c1 DESC, c3, c9; ---- e 3 d 1 diff --git a/datafusion/sqllogictest/test_files/groupby.slt b/datafusion/sqllogictest/test_files/groupby.slt index 8ed5245ef09b..b915c439059b 100644 --- a/datafusion/sqllogictest/test_files/groupby.slt +++ b/datafusion/sqllogictest/test_files/groupby.slt @@ -2329,15 +2329,15 @@ ProjectionExec: expr=[country@0 as country, ARRAY_AGG(s.amount) ORDER BY [s.amou ----SortExec: expr=[amount@1 DESC] ------MemoryExec: partitions=1, partition_sizes=[1] -query T?R +query T?R rowsort SELECT s.country, ARRAY_AGG(s.amount ORDER BY s.amount DESC) AS amounts, SUM(s.amount) AS sum1 FROM sales_global AS s GROUP BY s.country ---- FRA [200.0, 50.0] 250 -TUR [100.0, 75.0] 175 GRC [80.0, 30.0] 110 +TUR [100.0, 75.0] 175 # test_ordering_sensitive_aggregation3 # When different aggregators have conflicting requirements, we cannot satisfy all of them in current implementation. @@ -2373,7 +2373,7 @@ ProjectionExec: expr=[country@0 as country, ARRAY_AGG(s.amount) ORDER BY [s.amou ----SortExec: expr=[country@0 ASC NULLS LAST,amount@1 DESC] ------MemoryExec: partitions=1, partition_sizes=[1] -query T?R +query T?R rowsort SELECT s.country, ARRAY_AGG(s.amount ORDER BY s.amount DESC) AS amounts, SUM(s.amount) AS sum1 FROM (SELECT * @@ -2409,7 +2409,7 @@ ProjectionExec: expr=[country@0 as country, zip_code@1 as zip_code, ARRAY_AGG(s. ----SortExec: expr=[country@1 ASC NULLS LAST,amount@2 DESC] ------MemoryExec: partitions=1, partition_sizes=[1] -query TI?R +query TI?R rowsort SELECT s.country, s.zip_code, ARRAY_AGG(s.amount ORDER BY s.amount DESC) AS amounts, SUM(s.amount) AS sum1 FROM (SELECT * @@ -2445,7 +2445,7 @@ ProjectionExec: expr=[country@0 as country, ARRAY_AGG(s.amount) ORDER BY [s.coun ----SortExec: expr=[country@0 ASC NULLS LAST] ------MemoryExec: partitions=1, partition_sizes=[1] -query T?R +query T?R rowsort SELECT s.country, ARRAY_AGG(s.amount ORDER BY s.amount DESC) AS amounts, SUM(s.amount) AS sum1 FROM (SELECT * @@ -2480,7 +2480,7 @@ ProjectionExec: expr=[country@0 as country, ARRAY_AGG(s.amount) ORDER BY [s.coun ----SortExec: expr=[country@0 ASC NULLS LAST,amount@1 DESC] ------MemoryExec: partitions=1, partition_sizes=[1] -query T?R +query T?R rowsort SELECT s.country, ARRAY_AGG(s.amount ORDER BY s.country DESC, s.amount DESC) AS amounts, SUM(s.amount) AS sum1 FROM (SELECT * @@ -2512,7 +2512,7 @@ ProjectionExec: expr=[country@0 as country, ARRAY_AGG(sales_global.amount) ORDER ----SortExec: expr=[amount@1 DESC] ------MemoryExec: partitions=1, partition_sizes=[1] -query T?RR +query T?RR rowsort SELECT country, ARRAY_AGG(amount ORDER BY amount DESC) AS amounts, FIRST_VALUE(amount ORDER BY amount ASC) AS fv1, LAST_VALUE(amount ORDER BY amount DESC) AS fv2 @@ -2520,8 +2520,8 @@ SELECT country, ARRAY_AGG(amount ORDER BY amount DESC) AS amounts, GROUP BY country ---- FRA [200.0, 50.0] 50 50 -TUR [100.0, 75.0] 75 75 GRC [80.0, 30.0] 30 30 +TUR [100.0, 75.0] 75 75 # test_reverse_aggregate_expr2 # Some of the Aggregators can be reversed, by this way we can still run aggregators without re-ordering @@ -2640,7 +2640,7 @@ ProjectionExec: expr=[country@0 as country, FIRST_VALUE(sales_global.amount) ORD ----SortExec: expr=[ts@1 ASC NULLS LAST] ------MemoryExec: partitions=1, partition_sizes=[1] -query TRRR +query TRRR rowsort SELECT country, FIRST_VALUE(amount ORDER BY ts DESC) as fv1, LAST_VALUE(amount ORDER BY ts DESC) as lv1, SUM(amount ORDER BY ts DESC) as sum1 @@ -2649,8 +2649,8 @@ SELECT country, FIRST_VALUE(amount ORDER BY ts DESC) as fv1, ORDER BY ts ASC) GROUP BY country ---- -GRC 80 30 110 FRA 200 50 250 +GRC 80 30 110 TUR 100 75 175 # If existing ordering doesn't satisfy requirement, we should do calculations @@ -2674,16 +2674,16 @@ ProjectionExec: expr=[country@0 as country, FIRST_VALUE(sales_global.amount) ORD ----SortExec: expr=[ts@1 DESC] ------MemoryExec: partitions=1, partition_sizes=[1] -query TRRR +query TRRR rowsort SELECT country, FIRST_VALUE(amount ORDER BY ts DESC) as fv1, LAST_VALUE(amount ORDER BY ts DESC) as lv1, SUM(amount ORDER BY ts DESC) as sum1 FROM sales_global GROUP BY country ---- -TUR 100 75 175 -GRC 80 30 110 FRA 200 50 250 +GRC 80 30 110 +TUR 100 75 175 query TT EXPLAIN SELECT s.zip_code, s.country, s.sn, s.ts, s.currency, LAST_VALUE(e.amount ORDER BY e.sn) AS last_rate @@ -2715,7 +2715,7 @@ SortExec: expr=[sn@2 ASC NULLS LAST] --------------MemoryExec: partitions=1, partition_sizes=[1] --------------MemoryExec: partitions=1, partition_sizes=[1] -query ITIPTR +query ITIPTR rowsort SELECT s.zip_code, s.country, s.sn, s.ts, s.currency, LAST_VALUE(e.amount ORDER BY e.sn) AS last_rate FROM sales_global AS s JOIN sales_global AS e @@ -2725,10 +2725,10 @@ GROUP BY s.sn, s.zip_code, s.country, s.ts, s.currency ORDER BY s.sn ---- 0 GRC 0 2022-01-01T06:00:00 EUR 30 +0 GRC 4 2022-01-03T10:00:00 EUR 80 1 FRA 1 2022-01-01T08:00:00 EUR 50 -1 TUR 2 2022-01-01T11:30:00 TRY 75 1 FRA 3 2022-01-02T12:00:00 EUR 200 -0 GRC 4 2022-01-03T10:00:00 EUR 80 +1 TUR 2 2022-01-01T11:30:00 TRY 75 1 TUR 4 2022-01-03T10:00:00 TRY 100 # Run order-sensitive aggregators in multiple partitions From 5909866bba3e23e5f807972b84de526a4eb16c4c Mon Sep 17 00:00:00 2001 From: Liang-Chi Hsieh Date: Thu, 14 Dec 2023 00:53:56 -0800 Subject: [PATCH 425/572] fix: volatile expressions should not be target of common subexpt elimination (#8520) * fix: volatile expressions should not be target of common subexpt elimination * Fix clippy * For review * Return error for unresolved scalar function * Improve error message --- datafusion/expr/src/expr.rs | 75 ++++++++++++++++++- .../optimizer/src/common_subexpr_eliminate.rs | 18 +++-- .../sqllogictest/test_files/functions.slt | 6 ++ 3 files changed, 91 insertions(+), 8 deletions(-) diff --git a/datafusion/expr/src/expr.rs b/datafusion/expr/src/expr.rs index 958f4f4a3456..f0aab95b8f0d 100644 --- a/datafusion/expr/src/expr.rs +++ b/datafusion/expr/src/expr.rs @@ -373,6 +373,24 @@ impl ScalarFunctionDefinition { ScalarFunctionDefinition::Name(func_name) => func_name.as_ref(), } } + + /// Whether this function is volatile, i.e. whether it can return different results + /// when evaluated multiple times with the same input. + pub fn is_volatile(&self) -> Result { + match self { + ScalarFunctionDefinition::BuiltIn(fun) => { + Ok(fun.volatility() == crate::Volatility::Volatile) + } + ScalarFunctionDefinition::UDF(udf) => { + Ok(udf.signature().volatility == crate::Volatility::Volatile) + } + ScalarFunctionDefinition::Name(func) => { + internal_err!( + "Cannot determine volatility of unresolved function: {func}" + ) + } + } + } } impl ScalarFunction { @@ -1692,14 +1710,28 @@ fn create_names(exprs: &[Expr]) -> Result { .join(", ")) } +/// Whether the given expression is volatile, i.e. whether it can return different results +/// when evaluated multiple times with the same input. +pub fn is_volatile(expr: &Expr) -> Result { + match expr { + Expr::ScalarFunction(func) => func.func_def.is_volatile(), + _ => Ok(false), + } +} + #[cfg(test)] mod test { use crate::expr::Cast; use crate::expr_fn::col; - use crate::{case, lit, Expr}; + use crate::{ + case, lit, BuiltinScalarFunction, ColumnarValue, Expr, ReturnTypeFunction, + ScalarFunctionDefinition, ScalarFunctionImplementation, ScalarUDF, Signature, + Volatility, + }; use arrow::datatypes::DataType; use datafusion_common::Column; use datafusion_common::{Result, ScalarValue}; + use std::sync::Arc; #[test] fn format_case_when() -> Result<()> { @@ -1800,4 +1832,45 @@ mod test { "UInt32(1) OR UInt32(2)" ); } + + #[test] + fn test_is_volatile_scalar_func_definition() { + // BuiltIn + assert!( + ScalarFunctionDefinition::BuiltIn(BuiltinScalarFunction::Random) + .is_volatile() + .unwrap() + ); + assert!( + !ScalarFunctionDefinition::BuiltIn(BuiltinScalarFunction::Abs) + .is_volatile() + .unwrap() + ); + + // UDF + let return_type: ReturnTypeFunction = + Arc::new(move |_| Ok(Arc::new(DataType::Utf8))); + let fun: ScalarFunctionImplementation = + Arc::new(move |_| Ok(ColumnarValue::Scalar(ScalarValue::new_utf8("a")))); + let udf = Arc::new(ScalarUDF::new( + "TestScalarUDF", + &Signature::uniform(1, vec![DataType::Float32], Volatility::Stable), + &return_type, + &fun, + )); + assert!(!ScalarFunctionDefinition::UDF(udf).is_volatile().unwrap()); + + let udf = Arc::new(ScalarUDF::new( + "TestScalarUDF", + &Signature::uniform(1, vec![DataType::Float32], Volatility::Volatile), + &return_type, + &fun, + )); + assert!(ScalarFunctionDefinition::UDF(udf).is_volatile().unwrap()); + + // Unresolved function + ScalarFunctionDefinition::Name(Arc::from("UnresolvedFunc")) + .is_volatile() + .expect_err("Shouldn't determine volatility of unresolved function"); + } } diff --git a/datafusion/optimizer/src/common_subexpr_eliminate.rs b/datafusion/optimizer/src/common_subexpr_eliminate.rs index 1d21407a6985..1e089257c61a 100644 --- a/datafusion/optimizer/src/common_subexpr_eliminate.rs +++ b/datafusion/optimizer/src/common_subexpr_eliminate.rs @@ -29,7 +29,7 @@ use datafusion_common::tree_node::{ use datafusion_common::{ internal_err, Column, DFField, DFSchema, DFSchemaRef, DataFusionError, Result, }; -use datafusion_expr::expr::Alias; +use datafusion_expr::expr::{is_volatile, Alias}; use datafusion_expr::logical_plan::{ Aggregate, Filter, LogicalPlan, Projection, Sort, Window, }; @@ -113,6 +113,8 @@ impl CommonSubexprEliminate { let Projection { expr, input, .. } = projection; let input_schema = Arc::clone(input.schema()); let mut expr_set = ExprSet::new(); + + // Visit expr list and build expr identifier to occuring count map (`expr_set`). let arrays = to_arrays(expr, input_schema, &mut expr_set, ExprMask::Normal)?; let (mut new_expr, new_input) = @@ -516,7 +518,7 @@ enum ExprMask { } impl ExprMask { - fn ignores(&self, expr: &Expr) -> bool { + fn ignores(&self, expr: &Expr) -> Result { let is_normal_minus_aggregates = matches!( expr, Expr::Literal(..) @@ -527,12 +529,14 @@ impl ExprMask { | Expr::Wildcard { .. } ); + let is_volatile = is_volatile(expr)?; + let is_aggr = matches!(expr, Expr::AggregateFunction(..)); - match self { - Self::Normal => is_normal_minus_aggregates || is_aggr, - Self::NormalAndAggregates => is_normal_minus_aggregates, - } + Ok(match self { + Self::Normal => is_volatile || is_normal_minus_aggregates || is_aggr, + Self::NormalAndAggregates => is_volatile || is_normal_minus_aggregates, + }) } } @@ -624,7 +628,7 @@ impl TreeNodeVisitor for ExprIdentifierVisitor<'_> { let (idx, sub_expr_desc) = self.pop_enter_mark(); // skip exprs should not be recognize. - if self.expr_mask.ignores(expr) { + if self.expr_mask.ignores(expr)? { self.id_array[idx].0 = self.series_number; let desc = Self::desc_expr(expr); self.visit_stack.push(VisitRecord::ExprItem(desc)); diff --git a/datafusion/sqllogictest/test_files/functions.slt b/datafusion/sqllogictest/test_files/functions.slt index 4f55ea316bb9..1903088b0748 100644 --- a/datafusion/sqllogictest/test_files/functions.slt +++ b/datafusion/sqllogictest/test_files/functions.slt @@ -995,3 +995,9 @@ query ? SELECT find_in_set(NULL, NULL) ---- NULL + +# Verify that multiple calls to volatile functions like `random()` are not combined / optimized away +query B +SELECT r FROM (SELECT r1 == r2 r, r1, r2 FROM (SELECT random() r1, random() r2) WHERE r1 > 0 AND r2 > 0) +---- +false From 831b2ba6e03a601d773a5ea8a3d2cb5fcb7a7da6 Mon Sep 17 00:00:00 2001 From: Xu Chen Date: Thu, 14 Dec 2023 22:40:51 +0800 Subject: [PATCH 426/572] Add LakeSoul to the list of Known Users (#8536) Signed-off-by: chenxu Co-authored-by: chenxu --- docs/source/user-guide/introduction.md | 1 + 1 file changed, 1 insertion(+) diff --git a/docs/source/user-guide/introduction.md b/docs/source/user-guide/introduction.md index da250fbb1f9c..6c1e54c2b701 100644 --- a/docs/source/user-guide/introduction.md +++ b/docs/source/user-guide/introduction.md @@ -106,6 +106,7 @@ Here are some active projects using DataFusion: - [GlareDB](https://github.com/GlareDB/glaredb) Fast SQL database for querying and analyzing distributed data. - [InfluxDB IOx](https://github.com/influxdata/influxdb_iox) Time Series Database - [Kamu](https://github.com/kamu-data/kamu-cli/) Planet-scale streaming data pipeline +- [LakeSoul](https://github.com/lakesoul-io/LakeSoul) Open source LakeHouse framework with native IO in Rust. - [Lance](https://github.com/lancedb/lance) Modern columnar data format for ML - [Parseable](https://github.com/parseablehq/parseable) Log storage and observability platform - [qv](https://github.com/timvw/qv) Quickly view your data From 974d49c907c5ddb250895efaef0952ed063f1831 Mon Sep 17 00:00:00 2001 From: Andrew Lamb Date: Thu, 14 Dec 2023 09:58:20 -0500 Subject: [PATCH 427/572] Fix regression with Incorrect results when reading parquet files with different schemas and statistics (#8533) * Add test for schema evolution * Fix reading parquet statistics * Update tests for fix * Add comments to help explain the test * Add another test --- .../datasource/physical_plan/parquet/mod.rs | 9 +- .../physical_plan/parquet/row_groups.rs | 140 ++++++++++++++---- .../test_files/schema_evolution.slt | 140 ++++++++++++++++++ 3 files changed, 256 insertions(+), 33 deletions(-) create mode 100644 datafusion/sqllogictest/test_files/schema_evolution.slt diff --git a/datafusion/core/src/datasource/physical_plan/parquet/mod.rs b/datafusion/core/src/datasource/physical_plan/parquet/mod.rs index 718f9f820af1..641b7bbb1596 100644 --- a/datafusion/core/src/datasource/physical_plan/parquet/mod.rs +++ b/datafusion/core/src/datasource/physical_plan/parquet/mod.rs @@ -468,8 +468,10 @@ impl FileOpener for ParquetOpener { ParquetRecordBatchStreamBuilder::new_with_options(reader, options) .await?; + let file_schema = builder.schema().clone(); + let (schema_mapping, adapted_projections) = - schema_adapter.map_schema(builder.schema())?; + schema_adapter.map_schema(&file_schema)?; // let predicate = predicate.map(|p| reassign_predicate_columns(p, builder.schema(), true)).transpose()?; let mask = ProjectionMask::roots( @@ -481,8 +483,8 @@ impl FileOpener for ParquetOpener { if let Some(predicate) = pushdown_filters.then_some(predicate).flatten() { let row_filter = row_filter::build_row_filter( &predicate, - builder.schema().as_ref(), - table_schema.as_ref(), + &file_schema, + &table_schema, builder.metadata(), reorder_predicates, &file_metrics, @@ -507,6 +509,7 @@ impl FileOpener for ParquetOpener { let file_metadata = builder.metadata().clone(); let predicate = pruning_predicate.as_ref().map(|p| p.as_ref()); let mut row_groups = row_groups::prune_row_groups_by_statistics( + &file_schema, builder.parquet_schema(), file_metadata.row_groups(), file_range, diff --git a/datafusion/core/src/datasource/physical_plan/parquet/row_groups.rs b/datafusion/core/src/datasource/physical_plan/parquet/row_groups.rs index 65414f5619a5..7c3f7d9384ab 100644 --- a/datafusion/core/src/datasource/physical_plan/parquet/row_groups.rs +++ b/datafusion/core/src/datasource/physical_plan/parquet/row_groups.rs @@ -55,6 +55,7 @@ use super::ParquetFileMetrics; /// Note: This method currently ignores ColumnOrder /// pub(crate) fn prune_row_groups_by_statistics( + arrow_schema: &Schema, parquet_schema: &SchemaDescriptor, groups: &[RowGroupMetaData], range: Option, @@ -80,7 +81,7 @@ pub(crate) fn prune_row_groups_by_statistics( let pruning_stats = RowGroupPruningStatistics { parquet_schema, row_group_metadata: metadata, - arrow_schema: predicate.schema().as_ref(), + arrow_schema, }; match predicate.prune(&pruning_stats) { Ok(values) => { @@ -416,11 +417,11 @@ mod tests { fn row_group_pruning_predicate_simple_expr() { use datafusion_expr::{col, lit}; // int > 1 => c1_max > 1 - let schema = Schema::new(vec![Field::new("c1", DataType::Int32, false)]); + let schema = + Arc::new(Schema::new(vec![Field::new("c1", DataType::Int32, false)])); let expr = col("c1").gt(lit(15)); let expr = logical2physical(&expr, &schema); - let pruning_predicate = - PruningPredicate::try_new(expr, Arc::new(schema)).unwrap(); + let pruning_predicate = PruningPredicate::try_new(expr, schema.clone()).unwrap(); let field = PrimitiveTypeField::new("c1", PhysicalType::INT32); let schema_descr = get_test_schema_descr(vec![field]); @@ -436,6 +437,7 @@ mod tests { let metrics = parquet_file_metrics(); assert_eq!( prune_row_groups_by_statistics( + &schema, &schema_descr, &[rgm1, rgm2], None, @@ -450,11 +452,11 @@ mod tests { fn row_group_pruning_predicate_missing_stats() { use datafusion_expr::{col, lit}; // int > 1 => c1_max > 1 - let schema = Schema::new(vec![Field::new("c1", DataType::Int32, false)]); + let schema = + Arc::new(Schema::new(vec![Field::new("c1", DataType::Int32, false)])); let expr = col("c1").gt(lit(15)); let expr = logical2physical(&expr, &schema); - let pruning_predicate = - PruningPredicate::try_new(expr, Arc::new(schema)).unwrap(); + let pruning_predicate = PruningPredicate::try_new(expr, schema.clone()).unwrap(); let field = PrimitiveTypeField::new("c1", PhysicalType::INT32); let schema_descr = get_test_schema_descr(vec![field]); @@ -471,6 +473,7 @@ mod tests { // is null / undefined so the first row group can't be filtered out assert_eq!( prune_row_groups_by_statistics( + &schema, &schema_descr, &[rgm1, rgm2], None, @@ -519,6 +522,7 @@ mod tests { // when conditions are joined using AND assert_eq!( prune_row_groups_by_statistics( + &schema, &schema_descr, groups, None, @@ -532,12 +536,13 @@ mod tests { // this bypasses the entire predicate expression and no row groups are filtered out let expr = col("c1").gt(lit(15)).or(col("c2").rem(lit(2)).eq(lit(0))); let expr = logical2physical(&expr, &schema); - let pruning_predicate = PruningPredicate::try_new(expr, schema).unwrap(); + let pruning_predicate = PruningPredicate::try_new(expr, schema.clone()).unwrap(); // if conditions in predicate are joined with OR and an unsupported expression is used // this bypasses the entire predicate expression and no row groups are filtered out assert_eq!( prune_row_groups_by_statistics( + &schema, &schema_descr, groups, None, @@ -548,6 +553,64 @@ mod tests { ); } + #[test] + fn row_group_pruning_predicate_file_schema() { + use datafusion_expr::{col, lit}; + // test row group predicate when file schema is different than table schema + // c1 > 0 + let table_schema = Arc::new(Schema::new(vec![ + Field::new("c1", DataType::Int32, false), + Field::new("c2", DataType::Int32, false), + ])); + let expr = col("c1").gt(lit(0)); + let expr = logical2physical(&expr, &table_schema); + let pruning_predicate = + PruningPredicate::try_new(expr, table_schema.clone()).unwrap(); + + // Model a file schema's column order c2 then c1, which is the opposite + // of the table schema + let file_schema = Arc::new(Schema::new(vec![ + Field::new("c2", DataType::Int32, false), + Field::new("c1", DataType::Int32, false), + ])); + let schema_descr = get_test_schema_descr(vec![ + PrimitiveTypeField::new("c2", PhysicalType::INT32), + PrimitiveTypeField::new("c1", PhysicalType::INT32), + ]); + // rg1 has c2 less than zero, c1 greater than zero + let rgm1 = get_row_group_meta_data( + &schema_descr, + vec![ + ParquetStatistics::int32(Some(-10), Some(-1), None, 0, false), // c2 + ParquetStatistics::int32(Some(1), Some(10), None, 0, false), + ], + ); + // rg1 has c2 greater than zero, c1 less than zero + let rgm2 = get_row_group_meta_data( + &schema_descr, + vec![ + ParquetStatistics::int32(Some(1), Some(10), None, 0, false), + ParquetStatistics::int32(Some(-10), Some(-1), None, 0, false), + ], + ); + + let metrics = parquet_file_metrics(); + let groups = &[rgm1, rgm2]; + // the first row group should be left because c1 is greater than zero + // the second should be filtered out because c1 is less than zero + assert_eq!( + prune_row_groups_by_statistics( + &file_schema, // NB must be file schema, not table_schema + &schema_descr, + groups, + None, + Some(&pruning_predicate), + &metrics + ), + vec![0] + ); + } + fn gen_row_group_meta_data_for_pruning_predicate() -> Vec { let schema_descr = get_test_schema_descr(vec![ PrimitiveTypeField::new("c1", PhysicalType::INT32), @@ -581,13 +644,14 @@ mod tests { let schema_descr = arrow_to_parquet_schema(&schema).unwrap(); let expr = col("c1").gt(lit(15)).and(col("c2").is_null()); let expr = logical2physical(&expr, &schema); - let pruning_predicate = PruningPredicate::try_new(expr, schema).unwrap(); + let pruning_predicate = PruningPredicate::try_new(expr, schema.clone()).unwrap(); let groups = gen_row_group_meta_data_for_pruning_predicate(); let metrics = parquet_file_metrics(); // First row group was filtered out because it contains no null value on "c2". assert_eq!( prune_row_groups_by_statistics( + &schema, &schema_descr, &groups, None, @@ -613,7 +677,7 @@ mod tests { .gt(lit(15)) .and(col("c2").eq(lit(ScalarValue::Boolean(None)))); let expr = logical2physical(&expr, &schema); - let pruning_predicate = PruningPredicate::try_new(expr, schema).unwrap(); + let pruning_predicate = PruningPredicate::try_new(expr, schema.clone()).unwrap(); let groups = gen_row_group_meta_data_for_pruning_predicate(); let metrics = parquet_file_metrics(); @@ -621,6 +685,7 @@ mod tests { // pass predicates. Ideally these should both be false assert_eq!( prune_row_groups_by_statistics( + &schema, &schema_descr, &groups, None, @@ -639,8 +704,11 @@ mod tests { // INT32: c1 > 5, the c1 is decimal(9,2) // The type of scalar value if decimal(9,2), don't need to do cast - let schema = - Schema::new(vec![Field::new("c1", DataType::Decimal128(9, 2), false)]); + let schema = Arc::new(Schema::new(vec![Field::new( + "c1", + DataType::Decimal128(9, 2), + false, + )])); let field = PrimitiveTypeField::new("c1", PhysicalType::INT32) .with_logical_type(LogicalType::Decimal { scale: 2, @@ -651,8 +719,7 @@ mod tests { let schema_descr = get_test_schema_descr(vec![field]); let expr = col("c1").gt(lit(ScalarValue::Decimal128(Some(500), 9, 2))); let expr = logical2physical(&expr, &schema); - let pruning_predicate = - PruningPredicate::try_new(expr, Arc::new(schema)).unwrap(); + let pruning_predicate = PruningPredicate::try_new(expr, schema.clone()).unwrap(); let rgm1 = get_row_group_meta_data( &schema_descr, // [1.00, 6.00] @@ -680,6 +747,7 @@ mod tests { let metrics = parquet_file_metrics(); assert_eq!( prune_row_groups_by_statistics( + &schema, &schema_descr, &[rgm1, rgm2, rgm3], None, @@ -693,8 +761,11 @@ mod tests { // The c1 type is decimal(9,0) in the parquet file, and the type of scalar is decimal(5,2). // We should convert all type to the coercion type, which is decimal(11,2) // The decimal of arrow is decimal(5,2), the decimal of parquet is decimal(9,0) - let schema = - Schema::new(vec![Field::new("c1", DataType::Decimal128(9, 0), false)]); + let schema = Arc::new(Schema::new(vec![Field::new( + "c1", + DataType::Decimal128(9, 0), + false, + )])); let field = PrimitiveTypeField::new("c1", PhysicalType::INT32) .with_logical_type(LogicalType::Decimal { @@ -709,8 +780,7 @@ mod tests { Decimal128(11, 2), )); let expr = logical2physical(&expr, &schema); - let pruning_predicate = - PruningPredicate::try_new(expr, Arc::new(schema)).unwrap(); + let pruning_predicate = PruningPredicate::try_new(expr, schema.clone()).unwrap(); let rgm1 = get_row_group_meta_data( &schema_descr, // [100, 600] @@ -744,6 +814,7 @@ mod tests { let metrics = parquet_file_metrics(); assert_eq!( prune_row_groups_by_statistics( + &schema, &schema_descr, &[rgm1, rgm2, rgm3, rgm4], None, @@ -754,8 +825,11 @@ mod tests { ); // INT64: c1 < 5, the c1 is decimal(18,2) - let schema = - Schema::new(vec![Field::new("c1", DataType::Decimal128(18, 2), false)]); + let schema = Arc::new(Schema::new(vec![Field::new( + "c1", + DataType::Decimal128(18, 2), + false, + )])); let field = PrimitiveTypeField::new("c1", PhysicalType::INT64) .with_logical_type(LogicalType::Decimal { scale: 2, @@ -766,8 +840,7 @@ mod tests { let schema_descr = get_test_schema_descr(vec![field]); let expr = col("c1").lt(lit(ScalarValue::Decimal128(Some(500), 18, 2))); let expr = logical2physical(&expr, &schema); - let pruning_predicate = - PruningPredicate::try_new(expr, Arc::new(schema)).unwrap(); + let pruning_predicate = PruningPredicate::try_new(expr, schema.clone()).unwrap(); let rgm1 = get_row_group_meta_data( &schema_descr, // [6.00, 8.00] @@ -792,6 +865,7 @@ mod tests { let metrics = parquet_file_metrics(); assert_eq!( prune_row_groups_by_statistics( + &schema, &schema_descr, &[rgm1, rgm2, rgm3], None, @@ -803,8 +877,11 @@ mod tests { // FIXED_LENGTH_BYTE_ARRAY: c1 = decimal128(100000, 28, 3), the c1 is decimal(18,2) // the type of parquet is decimal(18,2) - let schema = - Schema::new(vec![Field::new("c1", DataType::Decimal128(18, 2), false)]); + let schema = Arc::new(Schema::new(vec![Field::new( + "c1", + DataType::Decimal128(18, 2), + false, + )])); let field = PrimitiveTypeField::new("c1", PhysicalType::FIXED_LEN_BYTE_ARRAY) .with_logical_type(LogicalType::Decimal { scale: 2, @@ -818,8 +895,7 @@ mod tests { let left = cast(col("c1"), DataType::Decimal128(28, 3)); let expr = left.eq(lit(ScalarValue::Decimal128(Some(100000), 28, 3))); let expr = logical2physical(&expr, &schema); - let pruning_predicate = - PruningPredicate::try_new(expr, Arc::new(schema)).unwrap(); + let pruning_predicate = PruningPredicate::try_new(expr, schema.clone()).unwrap(); // we must use the big-endian when encode the i128 to bytes or vec[u8]. let rgm1 = get_row_group_meta_data( &schema_descr, @@ -863,6 +939,7 @@ mod tests { let metrics = parquet_file_metrics(); assert_eq!( prune_row_groups_by_statistics( + &schema, &schema_descr, &[rgm1, rgm2, rgm3], None, @@ -874,8 +951,11 @@ mod tests { // BYTE_ARRAY: c1 = decimal128(100000, 28, 3), the c1 is decimal(18,2) // the type of parquet is decimal(18,2) - let schema = - Schema::new(vec![Field::new("c1", DataType::Decimal128(18, 2), false)]); + let schema = Arc::new(Schema::new(vec![Field::new( + "c1", + DataType::Decimal128(18, 2), + false, + )])); let field = PrimitiveTypeField::new("c1", PhysicalType::BYTE_ARRAY) .with_logical_type(LogicalType::Decimal { scale: 2, @@ -889,8 +969,7 @@ mod tests { let left = cast(col("c1"), DataType::Decimal128(28, 3)); let expr = left.eq(lit(ScalarValue::Decimal128(Some(100000), 28, 3))); let expr = logical2physical(&expr, &schema); - let pruning_predicate = - PruningPredicate::try_new(expr, Arc::new(schema)).unwrap(); + let pruning_predicate = PruningPredicate::try_new(expr, schema.clone()).unwrap(); // we must use the big-endian when encode the i128 to bytes or vec[u8]. let rgm1 = get_row_group_meta_data( &schema_descr, @@ -923,6 +1002,7 @@ mod tests { let metrics = parquet_file_metrics(); assert_eq!( prune_row_groups_by_statistics( + &schema, &schema_descr, &[rgm1, rgm2, rgm3], None, diff --git a/datafusion/sqllogictest/test_files/schema_evolution.slt b/datafusion/sqllogictest/test_files/schema_evolution.slt new file mode 100644 index 000000000000..36d54159e24d --- /dev/null +++ b/datafusion/sqllogictest/test_files/schema_evolution.slt @@ -0,0 +1,140 @@ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at + +# http://www.apache.org/licenses/LICENSE-2.0 + +# Unless required by applicable law or agreed to in writing, +# software distributed under the License is distributed on an +# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +# KIND, either express or implied. See the License for the +# specific language governing permissions and limitations +# under the License. + +########## +# Tests for schema evolution -- reading +# data from different files with different schemas +########## + + +statement ok +CREATE EXTERNAL TABLE parquet_table(a varchar, b int, c float) STORED AS PARQUET +LOCATION 'test_files/scratch/schema_evolution/parquet_table/'; + +# File1 has only columns a and b +statement ok +COPY ( + SELECT column1 as a, column2 as b + FROM ( VALUES ('foo', 1), ('foo', 2), ('foo', 3) ) + ) TO 'test_files/scratch/schema_evolution/parquet_table/1.parquet' +(FORMAT PARQUET, SINGLE_FILE_OUTPUT true); + + +# File2 has only b +statement ok +COPY ( + SELECT column1 as b + FROM ( VALUES (10) ) + ) TO 'test_files/scratch/schema_evolution/parquet_table/2.parquet' +(FORMAT PARQUET, SINGLE_FILE_OUTPUT true); + +# File3 has a column from 'z' which does not appear in the table +# but also values from a which do appear in the table +statement ok +COPY ( + SELECT column1 as z, column2 as a + FROM ( VALUES ('bar', 'foo'), ('blarg', 'foo') ) + ) TO 'test_files/scratch/schema_evolution/parquet_table/3.parquet' +(FORMAT PARQUET, SINGLE_FILE_OUTPUT true); + +# File4 has data for b and a (reversed) and d +statement ok +COPY ( + SELECT column1 as b, column2 as a, column3 as c + FROM ( VALUES (100, 'foo', 10.5), (200, 'foo', 12.6), (300, 'bzz', 13.7) ) + ) TO 'test_files/scratch/schema_evolution/parquet_table/4.parquet' +(FORMAT PARQUET, SINGLE_FILE_OUTPUT true); + +# The logical distribution of `a`, `b` and `c` in the files is like this: +# +## File1: +# foo 1 NULL +# foo 2 NULL +# foo 3 NULL +# +## File2: +# NULL 10 NULL +# +## File3: +# foo NULL NULL +# foo NULL NULL +# +## File4: +# foo 100 10.5 +# foo 200 12.6 +# bzz 300 13.7 + +# Show all the data +query TIR rowsort +select * from parquet_table; +---- +NULL 10 NULL +bzz 300 13.7 +foo 1 NULL +foo 100 10.5 +foo 2 NULL +foo 200 12.6 +foo 3 NULL +foo NULL NULL +foo NULL NULL + +# Should see all 7 rows that have 'a=foo' +query TIR rowsort +select * from parquet_table where a = 'foo'; +---- +foo 1 NULL +foo 100 10.5 +foo 2 NULL +foo 200 12.6 +foo 3 NULL +foo NULL NULL +foo NULL NULL + +query TIR rowsort +select * from parquet_table where a != 'foo'; +---- +bzz 300 13.7 + +# this should produce at least one row +query TIR rowsort +select * from parquet_table where a is NULL; +---- +NULL 10 NULL + +query TIR rowsort +select * from parquet_table where b > 5; +---- +NULL 10 NULL +bzz 300 13.7 +foo 100 10.5 +foo 200 12.6 + + +query TIR rowsort +select * from parquet_table where b < 150; +---- +NULL 10 NULL +foo 1 NULL +foo 100 10.5 +foo 2 NULL +foo 3 NULL + +query TIR rowsort +select * from parquet_table where c > 11.0; +---- +bzz 300 13.7 +foo 200 12.6 From 1042095211caec2cbd0af93b8f4c8a78dff47259 Mon Sep 17 00:00:00 2001 From: Ashim Sedhain <38435962+asimsedhain@users.noreply.github.com> Date: Thu, 14 Dec 2023 10:40:53 -0600 Subject: [PATCH 428/572] feat: improve string statistics display (#8535) GH-8464 --- datafusion-cli/src/functions.rs | 77 ++++++++++++++++++++++----------- datafusion-cli/src/main.rs | 24 ++++++++++ 2 files changed, 75 insertions(+), 26 deletions(-) diff --git a/datafusion-cli/src/functions.rs b/datafusion-cli/src/functions.rs index 24f3399ee2be..f8d9ed238be4 100644 --- a/datafusion-cli/src/functions.rs +++ b/datafusion-cli/src/functions.rs @@ -31,6 +31,7 @@ use datafusion::logical_expr::Expr; use datafusion::physical_plan::memory::MemoryExec; use datafusion::physical_plan::ExecutionPlan; use datafusion::scalar::ScalarValue; +use parquet::basic::ConvertedType; use parquet::file::reader::FileReader; use parquet::file::serialized_reader::SerializedFileReader; use parquet::file::statistics::Statistics; @@ -246,6 +247,52 @@ impl TableProvider for ParquetMetadataTable { } } +fn convert_parquet_statistics( + value: &Statistics, + converted_type: ConvertedType, +) -> (String, String) { + match (value, converted_type) { + (Statistics::Boolean(val), _) => (val.min().to_string(), val.max().to_string()), + (Statistics::Int32(val), _) => (val.min().to_string(), val.max().to_string()), + (Statistics::Int64(val), _) => (val.min().to_string(), val.max().to_string()), + (Statistics::Int96(val), _) => (val.min().to_string(), val.max().to_string()), + (Statistics::Float(val), _) => (val.min().to_string(), val.max().to_string()), + (Statistics::Double(val), _) => (val.min().to_string(), val.max().to_string()), + (Statistics::ByteArray(val), ConvertedType::UTF8) => { + let min_bytes = val.min(); + let max_bytes = val.max(); + let min = min_bytes + .as_utf8() + .map(|v| v.to_string()) + .unwrap_or_else(|_| min_bytes.to_string()); + + let max = max_bytes + .as_utf8() + .map(|v| v.to_string()) + .unwrap_or_else(|_| max_bytes.to_string()); + (min, max) + } + (Statistics::ByteArray(val), _) => (val.min().to_string(), val.max().to_string()), + (Statistics::FixedLenByteArray(val), ConvertedType::UTF8) => { + let min_bytes = val.min(); + let max_bytes = val.max(); + let min = min_bytes + .as_utf8() + .map(|v| v.to_string()) + .unwrap_or_else(|_| min_bytes.to_string()); + + let max = max_bytes + .as_utf8() + .map(|v| v.to_string()) + .unwrap_or_else(|_| max_bytes.to_string()); + (min, max) + } + (Statistics::FixedLenByteArray(val), _) => { + (val.min().to_string(), val.max().to_string()) + } + } +} + pub struct ParquetMetadataFunc {} impl TableFunctionImpl for ParquetMetadataFunc { @@ -326,34 +373,12 @@ impl TableFunctionImpl for ParquetMetadataFunc { num_values_arr.push(column.num_values()); path_in_schema_arr.push(column.column_path().to_string()); type_arr.push(column.column_type().to_string()); + let converted_type = column.column_descr().converted_type(); + if let Some(s) = column.statistics() { let (min_val, max_val) = if s.has_min_max_set() { - let (min_val, max_val) = match s { - Statistics::Boolean(val) => { - (val.min().to_string(), val.max().to_string()) - } - Statistics::Int32(val) => { - (val.min().to_string(), val.max().to_string()) - } - Statistics::Int64(val) => { - (val.min().to_string(), val.max().to_string()) - } - Statistics::Int96(val) => { - (val.min().to_string(), val.max().to_string()) - } - Statistics::Float(val) => { - (val.min().to_string(), val.max().to_string()) - } - Statistics::Double(val) => { - (val.min().to_string(), val.max().to_string()) - } - Statistics::ByteArray(val) => { - (val.min().to_string(), val.max().to_string()) - } - Statistics::FixedLenByteArray(val) => { - (val.min().to_string(), val.max().to_string()) - } - }; + let (min_val, max_val) = + convert_parquet_statistics(s, converted_type); (Some(min_val), Some(max_val)) } else { (None, None) diff --git a/datafusion-cli/src/main.rs b/datafusion-cli/src/main.rs index 8b1a9816afc0..8b74a797b57b 100644 --- a/datafusion-cli/src/main.rs +++ b/datafusion-cli/src/main.rs @@ -420,4 +420,28 @@ mod tests { Ok(()) } + + #[tokio::test] + async fn test_parquet_metadata_works_with_strings() -> Result<(), DataFusionError> { + let ctx = SessionContext::new(); + ctx.register_udtf("parquet_metadata", Arc::new(ParquetMetadataFunc {})); + + // input with string columns + let sql = + "SELECT * FROM parquet_metadata('../parquet-testing/data/data_index_bloom_encoding_stats.parquet')"; + let df = ctx.sql(sql).await?; + let rbs = df.collect().await?; + + let excepted = [ + +"+-----------------------------------------------------------------+--------------+--------------------+-----------------------+-----------------+-----------+-------------+------------+----------------+------------+-----------+-----------+------------------+----------------------+-----------------+-----------------+--------------------+--------------------------+-------------------+------------------------+------------------+-----------------------+-------------------------+", +"| filename | row_group_id | row_group_num_rows | row_group_num_columns | row_group_bytes | column_id | file_offset | num_values | path_in_schema | type | stats_min | stats_max | stats_null_count | stats_distinct_count | stats_min_value | stats_max_value | compression | encodings | index_page_offset | dictionary_page_offset | data_page_offset | total_compressed_size | total_uncompressed_size |", +"+-----------------------------------------------------------------+--------------+--------------------+-----------------------+-----------------+-----------+-------------+------------+----------------+------------+-----------+-----------+------------------+----------------------+-----------------+-----------------+--------------------+--------------------------+-------------------+------------------------+------------------+-----------------------+-------------------------+", +"| ../parquet-testing/data/data_index_bloom_encoding_stats.parquet | 0 | 14 | 1 | 163 | 0 | 4 | 14 | \"String\" | BYTE_ARRAY | Hello | today | 0 | | Hello | today | GZIP(GzipLevel(6)) | [BIT_PACKED, RLE, PLAIN] | | | 4 | 152 | 163 |", +"+-----------------------------------------------------------------+--------------+--------------------+-----------------------+-----------------+-----------+-------------+------------+----------------+------------+-----------+-----------+------------------+----------------------+-----------------+-----------------+--------------------+--------------------------+-------------------+------------------------+------------------+-----------------------+-------------------------+" + ]; + assert_batches_eq!(excepted, &rbs); + + Ok(()) + } } From a971f1e7e379f5efe9fa3a5839c36ca2d797e201 Mon Sep 17 00:00:00 2001 From: Raphael Taylor-Davies <1781103+tustvold@users.noreply.github.com> Date: Thu, 14 Dec 2023 17:49:56 +0000 Subject: [PATCH 429/572] Defer file creation to write (#8539) * Defer file creation to write * Format * Remove INSERT_MODE * Format * Add ticket link --- datafusion/core/src/datasource/listing/url.rs | 1 + .../src/datasource/listing_table_factory.rs | 17 +++++------------ datafusion/core/src/datasource/stream.rs | 10 ++++++++-- datafusion/core/src/physical_planner.rs | 6 +----- .../test_files/insert_to_external.slt | 18 +++++++++--------- docs/source/user-guide/sql/write_options.md | 8 +++----- 6 files changed, 27 insertions(+), 33 deletions(-) diff --git a/datafusion/core/src/datasource/listing/url.rs b/datafusion/core/src/datasource/listing/url.rs index 979ed9e975c4..3ca7864f7f9e 100644 --- a/datafusion/core/src/datasource/listing/url.rs +++ b/datafusion/core/src/datasource/listing/url.rs @@ -116,6 +116,7 @@ impl ListingTableUrl { /// Get object store for specified input_url /// if input_url is actually not a url, we assume it is a local file path /// if we have a local path, create it if not exists so ListingTableUrl::parse works + #[deprecated(note = "Use parse")] pub fn parse_create_local_if_not_exists( s: impl AsRef, is_directory: bool, diff --git a/datafusion/core/src/datasource/listing_table_factory.rs b/datafusion/core/src/datasource/listing_table_factory.rs index 96436306c641..a9d0c3a0099e 100644 --- a/datafusion/core/src/datasource/listing_table_factory.rs +++ b/datafusion/core/src/datasource/listing_table_factory.rs @@ -148,19 +148,18 @@ impl TableProviderFactory for ListingTableFactory { .unwrap_or(false) }; - let create_local_path = statement_options - .take_bool_option("create_local_path")? - .unwrap_or(false); let single_file = statement_options .take_bool_option("single_file")? .unwrap_or(false); - // Backwards compatibility + // Backwards compatibility (#8547) if let Some(s) = statement_options.take_str_option("insert_mode") { if !s.eq_ignore_ascii_case("append_new_files") { - return plan_err!("Unknown or unsupported insert mode {s}. Only append_to_file supported"); + return plan_err!("Unknown or unsupported insert mode {s}. Only append_new_files supported"); } } + statement_options.take_bool_option("create_local_path")?; + let file_type = file_format.file_type(); // Use remaining options and session state to build FileTypeWriterOptions @@ -199,13 +198,7 @@ impl TableProviderFactory for ListingTableFactory { FileType::AVRO => file_type_writer_options, }; - let table_path = match create_local_path { - true => ListingTableUrl::parse_create_local_if_not_exists( - &cmd.location, - !single_file, - ), - false => ListingTableUrl::parse(&cmd.location), - }?; + let table_path = ListingTableUrl::parse(&cmd.location)?; let options = ListingOptions::new(file_format) .with_collect_stat(state.config().collect_statistics()) diff --git a/datafusion/core/src/datasource/stream.rs b/datafusion/core/src/datasource/stream.rs index e7512499eb9d..b9b45a6c7470 100644 --- a/datafusion/core/src/datasource/stream.rs +++ b/datafusion/core/src/datasource/stream.rs @@ -179,7 +179,10 @@ impl StreamConfig { match &self.encoding { StreamEncoding::Csv => { let header = self.header && !self.location.exists(); - let file = OpenOptions::new().append(true).open(&self.location)?; + let file = OpenOptions::new() + .create(true) + .append(true) + .open(&self.location)?; let writer = arrow::csv::WriterBuilder::new() .with_header(header) .build(file); @@ -187,7 +190,10 @@ impl StreamConfig { Ok(Box::new(writer)) } StreamEncoding::Json => { - let file = OpenOptions::new().append(true).open(&self.location)?; + let file = OpenOptions::new() + .create(true) + .append(true) + .open(&self.location)?; Ok(Box::new(arrow::json::LineDelimitedWriter::new(file))) } } diff --git a/datafusion/core/src/physical_planner.rs b/datafusion/core/src/physical_planner.rs index ab38b3ec6d2f..93f0b31e5234 100644 --- a/datafusion/core/src/physical_planner.rs +++ b/datafusion/core/src/physical_planner.rs @@ -571,11 +571,7 @@ impl DefaultPhysicalPlanner { copy_options, }) => { let input_exec = self.create_initial_plan(input, session_state).await?; - - // TODO: make this behavior configurable via options (should copy to create path/file as needed?) - // TODO: add additional configurable options for if existing files should be overwritten or - // appended to - let parsed_url = ListingTableUrl::parse_create_local_if_not_exists(output_url, !*single_file_output)?; + let parsed_url = ListingTableUrl::parse(output_url)?; let object_store_url = parsed_url.object_store(); let schema: Schema = (**input.schema()).clone().into(); diff --git a/datafusion/sqllogictest/test_files/insert_to_external.slt b/datafusion/sqllogictest/test_files/insert_to_external.slt index 85c2db7faaf6..cdaf0bb64339 100644 --- a/datafusion/sqllogictest/test_files/insert_to_external.slt +++ b/datafusion/sqllogictest/test_files/insert_to_external.slt @@ -57,7 +57,7 @@ CREATE EXTERNAL TABLE dictionary_encoded_parquet_partitioned( b varchar, ) STORED AS parquet -LOCATION 'test_files/scratch/insert_to_external/parquet_types_partitioned' +LOCATION 'test_files/scratch/insert_to_external/parquet_types_partitioned/' PARTITIONED BY (b) OPTIONS( create_local_path 'true', @@ -292,7 +292,7 @@ statement ok CREATE EXTERNAL TABLE directory_test(a bigint, b bigint) STORED AS parquet -LOCATION 'test_files/scratch/insert_to_external/external_parquet_table_q0' +LOCATION 'test_files/scratch/insert_to_external/external_parquet_table_q0/' OPTIONS( create_local_path 'true', ); @@ -312,7 +312,7 @@ statement ok CREATE EXTERNAL TABLE table_without_values(field1 BIGINT NULL, field2 BIGINT NULL) STORED AS parquet -LOCATION 'test_files/scratch/insert_to_external/external_parquet_table_q1' +LOCATION 'test_files/scratch/insert_to_external/external_parquet_table_q1/' OPTIONS (create_local_path 'true'); query TT @@ -378,7 +378,7 @@ statement ok CREATE EXTERNAL TABLE table_without_values(field1 BIGINT NULL, field2 BIGINT NULL) STORED AS parquet -LOCATION 'test_files/scratch/insert_to_external/external_parquet_table_q2' +LOCATION 'test_files/scratch/insert_to_external/external_parquet_table_q2/' OPTIONS (create_local_path 'true'); query TT @@ -423,7 +423,7 @@ statement ok CREATE EXTERNAL TABLE table_without_values(c1 varchar NULL) STORED AS parquet -LOCATION 'test_files/scratch/insert_to_external/external_parquet_table_q3' +LOCATION 'test_files/scratch/insert_to_external/external_parquet_table_q3/' OPTIONS (create_local_path 'true'); # verify that the sort order of the insert query is maintained into the @@ -462,7 +462,7 @@ statement ok CREATE EXTERNAL TABLE table_without_values(id BIGINT, name varchar) STORED AS parquet -LOCATION 'test_files/scratch/insert_to_external/external_parquet_table_q4' +LOCATION 'test_files/scratch/insert_to_external/external_parquet_table_q4/' OPTIONS (create_local_path 'true'); query IT @@ -505,7 +505,7 @@ statement ok CREATE EXTERNAL TABLE table_without_values(field1 BIGINT NOT NULL, field2 BIGINT NULL) STORED AS parquet -LOCATION 'test_files/scratch/insert_to_external/external_parquet_table_q5' +LOCATION 'test_files/scratch/insert_to_external/external_parquet_table_q5/' OPTIONS (create_local_path 'true'); query II @@ -555,7 +555,7 @@ CREATE EXTERNAL TABLE test_column_defaults( d text default lower('DEFAULT_TEXT'), e timestamp default now() ) STORED AS parquet -LOCATION 'test_files/scratch/insert_to_external/external_parquet_table_q6' +LOCATION 'test_files/scratch/insert_to_external/external_parquet_table_q6/' OPTIONS (create_local_path 'true'); # fill in all column values @@ -608,5 +608,5 @@ CREATE EXTERNAL TABLE test_column_defaults( a int, b int default a+1 ) STORED AS parquet -LOCATION 'test_files/scratch/insert_to_external/external_parquet_table_q7' +LOCATION 'test_files/scratch/insert_to_external/external_parquet_table_q7/' OPTIONS (create_local_path 'true'); diff --git a/docs/source/user-guide/sql/write_options.md b/docs/source/user-guide/sql/write_options.md index 941484e84efd..94adee960996 100644 --- a/docs/source/user-guide/sql/write_options.md +++ b/docs/source/user-guide/sql/write_options.md @@ -78,11 +78,9 @@ The following special options are specific to the `COPY` command. The following special options are specific to creating an external table. -| Option | Description | Default Value | -| ----------------- | ------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------ | ---------------------------------------------------------------------------- | -| SINGLE_FILE | If true, indicates that this external table is backed by a single file. INSERT INTO queries will append to this file. | false | -| CREATE_LOCAL_PATH | If true, the folder or file backing this table will be created on the local file system if it does not already exist when running INSERT INTO queries. | false | -| INSERT_MODE | Determines if INSERT INTO queries should append to existing files or append new files to an existing directory. Valid values are append_to_file, append_new_files, and error. Note that "error" will block inserting data into this table. | CSV and JSON default to append_to_file. Parquet defaults to append_new_files | +| Option | Description | Default Value | +| ----------- | --------------------------------------------------------------------------------------------------------------------- | ------------- | +| SINGLE_FILE | If true, indicates that this external table is backed by a single file. INSERT INTO queries will append to this file. | false | ### JSON Format Specific Options From efa7b3421a4b05d939e92b94554f6f7fb2164d71 Mon Sep 17 00:00:00 2001 From: Andrew Lamb Date: Thu, 14 Dec 2023 13:52:25 -0500 Subject: [PATCH 430/572] Minor: Improve error handling in sqllogictest runner (#8544) --- datafusion/sqllogictest/bin/sqllogictests.rs | 53 +++++++++++++------- 1 file changed, 34 insertions(+), 19 deletions(-) diff --git a/datafusion/sqllogictest/bin/sqllogictests.rs b/datafusion/sqllogictest/bin/sqllogictests.rs index 484677d58e79..aeb1cc4ec919 100644 --- a/datafusion/sqllogictest/bin/sqllogictests.rs +++ b/datafusion/sqllogictest/bin/sqllogictests.rs @@ -26,7 +26,7 @@ use futures::stream::StreamExt; use log::info; use sqllogictest::strict_column_validator; -use datafusion_common::{exec_err, DataFusionError, Result}; +use datafusion_common::{exec_datafusion_err, exec_err, DataFusionError, Result}; const TEST_DIRECTORY: &str = "test_files/"; const PG_COMPAT_FILE_PREFIX: &str = "pg_compat_"; @@ -84,7 +84,7 @@ async fn run_tests() -> Result<()> { // Doing so is safe because each slt file runs with its own // `SessionContext` and should not have side effects (like // modifying shared state like `/tmp/`) - let errors: Vec<_> = futures::stream::iter(read_test_files(&options)) + let errors: Vec<_> = futures::stream::iter(read_test_files(&options)?) .map(|test_file| { tokio::task::spawn(async move { println!("Running {:?}", test_file.relative_path); @@ -247,30 +247,45 @@ impl TestFile { } } -fn read_test_files<'a>(options: &'a Options) -> Box + 'a> { - Box::new( - read_dir_recursive(TEST_DIRECTORY) +fn read_test_files<'a>( + options: &'a Options, +) -> Result + 'a>> { + Ok(Box::new( + read_dir_recursive(TEST_DIRECTORY)? + .into_iter() .map(TestFile::new) .filter(|f| options.check_test_file(&f.relative_path)) .filter(|f| f.is_slt_file()) .filter(|f| f.check_tpch(options)) .filter(|f| options.check_pg_compat_file(f.path.as_path())), - ) + )) } -fn read_dir_recursive>(path: P) -> Box> { - Box::new( - std::fs::read_dir(path) - .expect("Readable directory") - .map(|path| path.expect("Readable entry").path()) - .flat_map(|path| { - if path.is_dir() { - read_dir_recursive(path) - } else { - Box::new(std::iter::once(path)) - } - }), - ) +fn read_dir_recursive>(path: P) -> Result> { + let mut dst = vec![]; + read_dir_recursive_impl(&mut dst, path.as_ref())?; + Ok(dst) +} + +/// Append all paths recursively to dst +fn read_dir_recursive_impl(dst: &mut Vec, path: &Path) -> Result<()> { + let entries = std::fs::read_dir(path) + .map_err(|e| exec_datafusion_err!("Error reading directory {path:?}: {e}"))?; + for entry in entries { + let path = entry + .map_err(|e| { + exec_datafusion_err!("Error reading entry in directory {path:?}: {e}") + })? + .path(); + + if path.is_dir() { + read_dir_recursive_impl(dst, &path)?; + } else { + dst.push(path); + } + } + + Ok(()) } /// Parsed command line options From d67c0bbecd8f32049de2c931c077a66ed640413a Mon Sep 17 00:00:00 2001 From: Mustafa Akur <106137913+mustafasrepo@users.noreply.github.com> Date: Thu, 14 Dec 2023 23:15:15 +0300 Subject: [PATCH 431/572] Remove order_bys from AggregateExec state (#8537) * Initial commit * Remove order by from aggregate exec state --- .../aggregate_statistics.rs | 12 -------- .../combine_partial_final_agg.rs | 4 --- .../enforce_distribution.rs | 4 --- .../limited_distinct_aggregation.rs | 25 +++-------------- .../core/src/physical_optimizer/test_utils.rs | 1 - .../physical_optimizer/topk_aggregation.rs | 1 - datafusion/core/src/physical_planner.rs | 5 +--- .../core/tests/fuzz_cases/aggregate_fuzz.rs | 2 -- .../physical-plan/src/aggregates/mod.rs | 28 +++---------------- datafusion/physical-plan/src/limit.rs | 1 - datafusion/proto/proto/datafusion.proto | 1 - datafusion/proto/src/generated/pbjson.rs | 18 ------------ datafusion/proto/src/generated/prost.rs | 2 -- datafusion/proto/src/physical_plan/mod.rs | 21 -------------- .../tests/cases/roundtrip_physical_plan.rs | 3 -- 15 files changed, 9 insertions(+), 119 deletions(-) diff --git a/datafusion/core/src/physical_optimizer/aggregate_statistics.rs b/datafusion/core/src/physical_optimizer/aggregate_statistics.rs index 795857b10ef5..86a8cdb7b3d4 100644 --- a/datafusion/core/src/physical_optimizer/aggregate_statistics.rs +++ b/datafusion/core/src/physical_optimizer/aggregate_statistics.rs @@ -397,7 +397,6 @@ pub(crate) mod tests { PhysicalGroupBy::default(), vec![agg.count_expr()], vec![None], - vec![None], source, Arc::clone(&schema), )?; @@ -407,7 +406,6 @@ pub(crate) mod tests { PhysicalGroupBy::default(), vec![agg.count_expr()], vec![None], - vec![None], Arc::new(partial_agg), Arc::clone(&schema), )?; @@ -429,7 +427,6 @@ pub(crate) mod tests { PhysicalGroupBy::default(), vec![agg.count_expr()], vec![None], - vec![None], source, Arc::clone(&schema), )?; @@ -439,7 +436,6 @@ pub(crate) mod tests { PhysicalGroupBy::default(), vec![agg.count_expr()], vec![None], - vec![None], Arc::new(partial_agg), Arc::clone(&schema), )?; @@ -460,7 +456,6 @@ pub(crate) mod tests { PhysicalGroupBy::default(), vec![agg.count_expr()], vec![None], - vec![None], source, Arc::clone(&schema), )?; @@ -473,7 +468,6 @@ pub(crate) mod tests { PhysicalGroupBy::default(), vec![agg.count_expr()], vec![None], - vec![None], Arc::new(coalesce), Arc::clone(&schema), )?; @@ -494,7 +488,6 @@ pub(crate) mod tests { PhysicalGroupBy::default(), vec![agg.count_expr()], vec![None], - vec![None], source, Arc::clone(&schema), )?; @@ -507,7 +500,6 @@ pub(crate) mod tests { PhysicalGroupBy::default(), vec![agg.count_expr()], vec![None], - vec![None], Arc::new(coalesce), Arc::clone(&schema), )?; @@ -539,7 +531,6 @@ pub(crate) mod tests { PhysicalGroupBy::default(), vec![agg.count_expr()], vec![None], - vec![None], filter, Arc::clone(&schema), )?; @@ -549,7 +540,6 @@ pub(crate) mod tests { PhysicalGroupBy::default(), vec![agg.count_expr()], vec![None], - vec![None], Arc::new(partial_agg), Arc::clone(&schema), )?; @@ -586,7 +576,6 @@ pub(crate) mod tests { PhysicalGroupBy::default(), vec![agg.count_expr()], vec![None], - vec![None], filter, Arc::clone(&schema), )?; @@ -596,7 +585,6 @@ pub(crate) mod tests { PhysicalGroupBy::default(), vec![agg.count_expr()], vec![None], - vec![None], Arc::new(partial_agg), Arc::clone(&schema), )?; diff --git a/datafusion/core/src/physical_optimizer/combine_partial_final_agg.rs b/datafusion/core/src/physical_optimizer/combine_partial_final_agg.rs index 0948445de20d..c50ea36b68ec 100644 --- a/datafusion/core/src/physical_optimizer/combine_partial_final_agg.rs +++ b/datafusion/core/src/physical_optimizer/combine_partial_final_agg.rs @@ -91,7 +91,6 @@ impl PhysicalOptimizerRule for CombinePartialFinalAggregate { input_agg_exec.group_by().clone(), input_agg_exec.aggr_expr().to_vec(), input_agg_exec.filter_expr().to_vec(), - input_agg_exec.order_by_expr().to_vec(), input_agg_exec.input().clone(), input_agg_exec.input_schema(), ) @@ -277,7 +276,6 @@ mod tests { group_by, aggr_expr, vec![], - vec![], input, schema, ) @@ -297,7 +295,6 @@ mod tests { group_by, aggr_expr, vec![], - vec![], input, schema, ) @@ -458,7 +455,6 @@ mod tests { final_group_by, aggr_expr, vec![], - vec![], partial_agg, schema, ) diff --git a/datafusion/core/src/physical_optimizer/enforce_distribution.rs b/datafusion/core/src/physical_optimizer/enforce_distribution.rs index 93cdbf858367..f2e04989ef66 100644 --- a/datafusion/core/src/physical_optimizer/enforce_distribution.rs +++ b/datafusion/core/src/physical_optimizer/enforce_distribution.rs @@ -521,7 +521,6 @@ fn reorder_aggregate_keys( new_partial_group_by, agg_exec.aggr_expr().to_vec(), agg_exec.filter_expr().to_vec(), - agg_exec.order_by_expr().to_vec(), agg_exec.input().clone(), agg_exec.input_schema.clone(), )?)) @@ -548,7 +547,6 @@ fn reorder_aggregate_keys( new_group_by, agg_exec.aggr_expr().to_vec(), agg_exec.filter_expr().to_vec(), - agg_exec.order_by_expr().to_vec(), partial_agg, agg_exec.input_schema(), )?); @@ -1909,14 +1907,12 @@ pub(crate) mod tests { final_grouping, vec![], vec![], - vec![], Arc::new( AggregateExec::try_new( AggregateMode::Partial, group_by, vec![], vec![], - vec![], input, schema.clone(), ) diff --git a/datafusion/core/src/physical_optimizer/limited_distinct_aggregation.rs b/datafusion/core/src/physical_optimizer/limited_distinct_aggregation.rs index 8f5dbc2e9214..540f9a6a132b 100644 --- a/datafusion/core/src/physical_optimizer/limited_distinct_aggregation.rs +++ b/datafusion/core/src/physical_optimizer/limited_distinct_aggregation.rs @@ -55,7 +55,6 @@ impl LimitedDistinctAggregation { aggr.group_by().clone(), aggr.aggr_expr().to_vec(), aggr.filter_expr().to_vec(), - aggr.order_by_expr().to_vec(), aggr.input().clone(), aggr.input_schema(), ) @@ -307,7 +306,6 @@ mod tests { build_group_by(&schema.clone(), vec!["a".to_string()]), vec![], /* aggr_expr */ vec![None], /* filter_expr */ - vec![None], /* order_by_expr */ source, /* input */ schema.clone(), /* input_schema */ )?; @@ -316,7 +314,6 @@ mod tests { build_group_by(&schema.clone(), vec!["a".to_string()]), vec![], /* aggr_expr */ vec![None], /* filter_expr */ - vec![None], /* order_by_expr */ Arc::new(partial_agg), /* input */ schema.clone(), /* input_schema */ )?; @@ -359,7 +356,6 @@ mod tests { build_group_by(&schema.clone(), vec!["a".to_string()]), vec![], /* aggr_expr */ vec![None], /* filter_expr */ - vec![None], /* order_by_expr */ source, /* input */ schema.clone(), /* input_schema */ )?; @@ -401,7 +397,6 @@ mod tests { build_group_by(&schema.clone(), vec!["a".to_string()]), vec![], /* aggr_expr */ vec![None], /* filter_expr */ - vec![None], /* order_by_expr */ source, /* input */ schema.clone(), /* input_schema */ )?; @@ -443,7 +438,6 @@ mod tests { build_group_by(&schema.clone(), vec!["a".to_string(), "b".to_string()]), vec![], /* aggr_expr */ vec![None], /* filter_expr */ - vec![None], /* order_by_expr */ source, /* input */ schema.clone(), /* input_schema */ )?; @@ -452,7 +446,6 @@ mod tests { build_group_by(&schema.clone(), vec!["a".to_string()]), vec![], /* aggr_expr */ vec![None], /* filter_expr */ - vec![None], /* order_by_expr */ Arc::new(group_by_agg), /* input */ schema.clone(), /* input_schema */ )?; @@ -495,7 +488,6 @@ mod tests { build_group_by(&schema.clone(), vec![]), vec![], /* aggr_expr */ vec![None], /* filter_expr */ - vec![None], /* order_by_expr */ source, /* input */ schema.clone(), /* input_schema */ )?; @@ -526,7 +518,6 @@ mod tests { build_group_by(&schema.clone(), vec!["a".to_string()]), vec![agg.count_expr()], /* aggr_expr */ vec![None], /* filter_expr */ - vec![None], /* order_by_expr */ source, /* input */ schema.clone(), /* input_schema */ )?; @@ -563,7 +554,6 @@ mod tests { build_group_by(&schema.clone(), vec!["a".to_string()]), vec![], /* aggr_expr */ vec![filter_expr], /* filter_expr */ - vec![None], /* order_by_expr */ source, /* input */ schema.clone(), /* input_schema */ )?; @@ -592,22 +582,15 @@ mod tests { let source = parquet_exec_with_sort(vec![sort_key]); let schema = source.schema(); - // `SELECT a FROM MemoryExec GROUP BY a ORDER BY a LIMIT 10;`, Single AggregateExec - let order_by_expr = Some(vec![PhysicalSortExpr { - expr: expressions::col("a", &schema.clone()).unwrap(), - options: SortOptions::default(), - }]); - // `SELECT a FROM MemoryExec WHERE a > 1 GROUP BY a LIMIT 10;`, Single AggregateExec // the `a > 1` filter is applied in the AggregateExec let single_agg = AggregateExec::try_new( AggregateMode::Single, build_group_by(&schema.clone(), vec!["a".to_string()]), - vec![], /* aggr_expr */ - vec![None], /* filter_expr */ - vec![order_by_expr], /* order_by_expr */ - source, /* input */ - schema.clone(), /* input_schema */ + vec![], /* aggr_expr */ + vec![None], /* filter_expr */ + source, /* input */ + schema.clone(), /* input_schema */ )?; let limit_exec = LocalLimitExec::new( Arc::new(single_agg), diff --git a/datafusion/core/src/physical_optimizer/test_utils.rs b/datafusion/core/src/physical_optimizer/test_utils.rs index 37a76eff1ee2..678dc1f373e3 100644 --- a/datafusion/core/src/physical_optimizer/test_utils.rs +++ b/datafusion/core/src/physical_optimizer/test_utils.rs @@ -339,7 +339,6 @@ pub fn aggregate_exec(input: Arc) -> Arc { PhysicalGroupBy::default(), vec![], vec![], - vec![], input, schema, ) diff --git a/datafusion/core/src/physical_optimizer/topk_aggregation.rs b/datafusion/core/src/physical_optimizer/topk_aggregation.rs index 52d34d4f8198..dd0261420304 100644 --- a/datafusion/core/src/physical_optimizer/topk_aggregation.rs +++ b/datafusion/core/src/physical_optimizer/topk_aggregation.rs @@ -73,7 +73,6 @@ impl TopKAggregation { aggr.group_by().clone(), aggr.aggr_expr().to_vec(), aggr.filter_expr().to_vec(), - aggr.order_by_expr().to_vec(), aggr.input().clone(), aggr.input_schema(), ) diff --git a/datafusion/core/src/physical_planner.rs b/datafusion/core/src/physical_planner.rs index 93f0b31e5234..e5816eb49ebb 100644 --- a/datafusion/core/src/physical_planner.rs +++ b/datafusion/core/src/physical_planner.rs @@ -795,14 +795,13 @@ impl DefaultPhysicalPlanner { }) .collect::>>()?; - let (aggregates, filters, order_bys) : (Vec<_>, Vec<_>, Vec<_>) = multiunzip(agg_filter); + let (aggregates, filters, _order_bys) : (Vec<_>, Vec<_>, Vec<_>) = multiunzip(agg_filter); let initial_aggr = Arc::new(AggregateExec::try_new( AggregateMode::Partial, groups.clone(), aggregates.clone(), filters.clone(), - order_bys, input_exec, physical_input_schema.clone(), )?); @@ -820,7 +819,6 @@ impl DefaultPhysicalPlanner { // To reflect such changes to subsequent stages, use the updated // `AggregateExpr`/`PhysicalSortExpr` objects. let updated_aggregates = initial_aggr.aggr_expr().to_vec(); - let updated_order_bys = initial_aggr.order_by_expr().to_vec(); let next_partition_mode = if can_repartition { // construct a second aggregation with 'AggregateMode::FinalPartitioned' @@ -844,7 +842,6 @@ impl DefaultPhysicalPlanner { final_grouping_set, updated_aggregates, filters, - updated_order_bys, initial_aggr, physical_input_schema.clone(), )?)) diff --git a/datafusion/core/tests/fuzz_cases/aggregate_fuzz.rs b/datafusion/core/tests/fuzz_cases/aggregate_fuzz.rs index 821f236af87b..9069dbbd5850 100644 --- a/datafusion/core/tests/fuzz_cases/aggregate_fuzz.rs +++ b/datafusion/core/tests/fuzz_cases/aggregate_fuzz.rs @@ -109,7 +109,6 @@ async fn run_aggregate_test(input1: Vec, group_by_columns: Vec<&str group_by.clone(), aggregate_expr.clone(), vec![None], - vec![None], running_source, schema.clone(), ) @@ -122,7 +121,6 @@ async fn run_aggregate_test(input1: Vec, group_by_columns: Vec<&str group_by.clone(), aggregate_expr.clone(), vec![None], - vec![None], usual_source, schema.clone(), ) diff --git a/datafusion/physical-plan/src/aggregates/mod.rs b/datafusion/physical-plan/src/aggregates/mod.rs index 2f69ed061ce1..c74c4ac0f821 100644 --- a/datafusion/physical-plan/src/aggregates/mod.rs +++ b/datafusion/physical-plan/src/aggregates/mod.rs @@ -279,8 +279,6 @@ pub struct AggregateExec { aggr_expr: Vec>, /// FILTER (WHERE clause) expression for each aggregate expression filter_expr: Vec>>, - /// (ORDER BY clause) expression for each aggregate expression - order_by_expr: Vec>, /// Set if the output of this aggregation is truncated by a upstream sort/limit clause limit: Option, /// Input plan, could be a partial aggregate or the input to the aggregate @@ -468,8 +466,6 @@ impl AggregateExec { group_by: PhysicalGroupBy, mut aggr_expr: Vec>, filter_expr: Vec>>, - // Ordering requirement of each aggregate expression - mut order_by_expr: Vec>, input: Arc, input_schema: SchemaRef, ) -> Result { @@ -487,10 +483,10 @@ impl AggregateExec { )); let original_schema = Arc::new(original_schema); // Reset ordering requirement to `None` if aggregator is not order-sensitive - order_by_expr = aggr_expr + let mut order_by_expr = aggr_expr .iter() - .zip(order_by_expr) - .map(|(aggr_expr, fn_reqs)| { + .map(|aggr_expr| { + let fn_reqs = aggr_expr.order_bys().map(|ordering| ordering.to_vec()); // If // - aggregation function is order-sensitive and // - aggregation is performing a "first stage" calculation, and @@ -558,7 +554,6 @@ impl AggregateExec { group_by, aggr_expr, filter_expr, - order_by_expr, input, original_schema, schema, @@ -602,11 +597,6 @@ impl AggregateExec { &self.filter_expr } - /// ORDER BY clause expression for each aggregate expression - pub fn order_by_expr(&self) -> &[Option] { - &self.order_by_expr - } - /// Input plan pub fn input(&self) -> &Arc { &self.input @@ -684,7 +674,7 @@ impl AggregateExec { return false; } // ensure there are no order by expressions - if self.order_by_expr().iter().any(|e| e.is_some()) { + if self.aggr_expr().iter().any(|e| e.order_bys().is_some()) { return false; } // ensure there is no output ordering; can this rule be relaxed? @@ -873,7 +863,6 @@ impl ExecutionPlan for AggregateExec { self.group_by.clone(), self.aggr_expr.clone(), self.filter_expr.clone(), - self.order_by_expr.clone(), children[0].clone(), self.input_schema.clone(), )?; @@ -1395,7 +1384,6 @@ mod tests { grouping_set.clone(), aggregates.clone(), vec![None], - vec![None], input, input_schema.clone(), )?); @@ -1474,7 +1462,6 @@ mod tests { final_grouping_set, aggregates, vec![None], - vec![None], merge, input_schema, )?); @@ -1540,7 +1527,6 @@ mod tests { grouping_set.clone(), aggregates.clone(), vec![None], - vec![None], input, input_schema.clone(), )?); @@ -1588,7 +1574,6 @@ mod tests { final_grouping_set, aggregates, vec![None], - vec![None], merge, input_schema, )?); @@ -1855,7 +1840,6 @@ mod tests { groups, aggregates, vec![None; 3], - vec![None; 3], input.clone(), input_schema.clone(), )?); @@ -1911,7 +1895,6 @@ mod tests { groups.clone(), aggregates.clone(), vec![None], - vec![None], blocking_exec, schema, )?); @@ -1950,7 +1933,6 @@ mod tests { groups, aggregates.clone(), vec![None], - vec![None], blocking_exec, schema, )?); @@ -2052,7 +2034,6 @@ mod tests { groups.clone(), aggregates.clone(), vec![None], - vec![Some(ordering_req.clone())], memory_exec, schema.clone(), )?); @@ -2068,7 +2049,6 @@ mod tests { groups, aggregates.clone(), vec![None], - vec![Some(ordering_req)], coalesce, schema, )?) as Arc; diff --git a/datafusion/physical-plan/src/limit.rs b/datafusion/physical-plan/src/limit.rs index 355561c36f35..37e8ffd76159 100644 --- a/datafusion/physical-plan/src/limit.rs +++ b/datafusion/physical-plan/src/limit.rs @@ -878,7 +878,6 @@ mod tests { build_group_by(&csv.schema().clone(), vec!["i".to_string()]), vec![], vec![None], - vec![None], csv.clone(), csv.schema().clone(), )?; diff --git a/datafusion/proto/proto/datafusion.proto b/datafusion/proto/proto/datafusion.proto index f391592dfe76..bd8053c817e7 100644 --- a/datafusion/proto/proto/datafusion.proto +++ b/datafusion/proto/proto/datafusion.proto @@ -1553,7 +1553,6 @@ message AggregateExecNode { repeated PhysicalExprNode null_expr = 8; repeated bool groups = 9; repeated MaybeFilter filter_expr = 10; - repeated MaybePhysicalSortExprs order_by_expr = 11; } message GlobalLimitExecNode { diff --git a/datafusion/proto/src/generated/pbjson.rs b/datafusion/proto/src/generated/pbjson.rs index d506b5dcce53..88310be0318a 100644 --- a/datafusion/proto/src/generated/pbjson.rs +++ b/datafusion/proto/src/generated/pbjson.rs @@ -36,9 +36,6 @@ impl serde::Serialize for AggregateExecNode { if !self.filter_expr.is_empty() { len += 1; } - if !self.order_by_expr.is_empty() { - len += 1; - } let mut struct_ser = serializer.serialize_struct("datafusion.AggregateExecNode", len)?; if !self.group_expr.is_empty() { struct_ser.serialize_field("groupExpr", &self.group_expr)?; @@ -72,9 +69,6 @@ impl serde::Serialize for AggregateExecNode { if !self.filter_expr.is_empty() { struct_ser.serialize_field("filterExpr", &self.filter_expr)?; } - if !self.order_by_expr.is_empty() { - struct_ser.serialize_field("orderByExpr", &self.order_by_expr)?; - } struct_ser.end() } } @@ -102,8 +96,6 @@ impl<'de> serde::Deserialize<'de> for AggregateExecNode { "groups", "filter_expr", "filterExpr", - "order_by_expr", - "orderByExpr", ]; #[allow(clippy::enum_variant_names)] @@ -118,7 +110,6 @@ impl<'de> serde::Deserialize<'de> for AggregateExecNode { NullExpr, Groups, FilterExpr, - OrderByExpr, } impl<'de> serde::Deserialize<'de> for GeneratedField { fn deserialize(deserializer: D) -> std::result::Result @@ -150,7 +141,6 @@ impl<'de> serde::Deserialize<'de> for AggregateExecNode { "nullExpr" | "null_expr" => Ok(GeneratedField::NullExpr), "groups" => Ok(GeneratedField::Groups), "filterExpr" | "filter_expr" => Ok(GeneratedField::FilterExpr), - "orderByExpr" | "order_by_expr" => Ok(GeneratedField::OrderByExpr), _ => Err(serde::de::Error::unknown_field(value, FIELDS)), } } @@ -180,7 +170,6 @@ impl<'de> serde::Deserialize<'de> for AggregateExecNode { let mut null_expr__ = None; let mut groups__ = None; let mut filter_expr__ = None; - let mut order_by_expr__ = None; while let Some(k) = map_.next_key()? { match k { GeneratedField::GroupExpr => { @@ -243,12 +232,6 @@ impl<'de> serde::Deserialize<'de> for AggregateExecNode { } filter_expr__ = Some(map_.next_value()?); } - GeneratedField::OrderByExpr => { - if order_by_expr__.is_some() { - return Err(serde::de::Error::duplicate_field("orderByExpr")); - } - order_by_expr__ = Some(map_.next_value()?); - } } } Ok(AggregateExecNode { @@ -262,7 +245,6 @@ impl<'de> serde::Deserialize<'de> for AggregateExecNode { null_expr: null_expr__.unwrap_or_default(), groups: groups__.unwrap_or_default(), filter_expr: filter_expr__.unwrap_or_default(), - order_by_expr: order_by_expr__.unwrap_or_default(), }) } } diff --git a/datafusion/proto/src/generated/prost.rs b/datafusion/proto/src/generated/prost.rs index 8aadc96349ca..3dfd3938615f 100644 --- a/datafusion/proto/src/generated/prost.rs +++ b/datafusion/proto/src/generated/prost.rs @@ -2193,8 +2193,6 @@ pub struct AggregateExecNode { pub groups: ::prost::alloc::vec::Vec, #[prost(message, repeated, tag = "10")] pub filter_expr: ::prost::alloc::vec::Vec, - #[prost(message, repeated, tag = "11")] - pub order_by_expr: ::prost::alloc::vec::Vec, } #[allow(clippy::derive_partial_eq_without_eq)] #[derive(Clone, PartialEq, ::prost::Message)] diff --git a/datafusion/proto/src/physical_plan/mod.rs b/datafusion/proto/src/physical_plan/mod.rs index 73091a6fced9..df01097cfa78 100644 --- a/datafusion/proto/src/physical_plan/mod.rs +++ b/datafusion/proto/src/physical_plan/mod.rs @@ -427,19 +427,6 @@ impl AsExecutionPlan for PhysicalPlanNode { .transpose() }) .collect::, _>>()?; - let physical_order_by_expr = hash_agg - .order_by_expr - .iter() - .map(|expr| { - expr.sort_expr - .iter() - .map(|e| { - parse_physical_sort_expr(e, registry, &physical_schema) - }) - .collect::>>() - .map(|exprs| (!exprs.is_empty()).then_some(exprs)) - }) - .collect::>>()?; let physical_aggr_expr: Vec> = hash_agg .aggr_expr @@ -498,7 +485,6 @@ impl AsExecutionPlan for PhysicalPlanNode { PhysicalGroupBy::new(group_expr, null_expr, groups), physical_aggr_expr, physical_filter_expr, - physical_order_by_expr, input, Arc::new(input_schema.try_into()?), )?)) @@ -1237,12 +1223,6 @@ impl AsExecutionPlan for PhysicalPlanNode { .map(|expr| expr.to_owned().try_into()) .collect::>>()?; - let order_by = exec - .order_by_expr() - .iter() - .map(|expr| expr.to_owned().try_into()) - .collect::>>()?; - let agg = exec .aggr_expr() .iter() @@ -1295,7 +1275,6 @@ impl AsExecutionPlan for PhysicalPlanNode { group_expr_name: group_names, aggr_expr: agg, filter_expr: filter, - order_by_expr: order_by, aggr_expr_name: agg_names, mode: agg_mode as i32, input: Some(Box::new(input)), diff --git a/datafusion/proto/tests/cases/roundtrip_physical_plan.rs b/datafusion/proto/tests/cases/roundtrip_physical_plan.rs index da76209dbb49..4a512413e73e 100644 --- a/datafusion/proto/tests/cases/roundtrip_physical_plan.rs +++ b/datafusion/proto/tests/cases/roundtrip_physical_plan.rs @@ -311,7 +311,6 @@ fn rountrip_aggregate() -> Result<()> { PhysicalGroupBy::new_single(groups.clone()), aggregates.clone(), vec![None], - vec![None], Arc::new(EmptyExec::new(schema.clone())), schema, )?)) @@ -379,7 +378,6 @@ fn roundtrip_aggregate_udaf() -> Result<()> { PhysicalGroupBy::new_single(groups.clone()), aggregates.clone(), vec![None], - vec![None], Arc::new(EmptyExec::new(schema.clone())), schema, )?), @@ -594,7 +592,6 @@ fn roundtrip_distinct_count() -> Result<()> { PhysicalGroupBy::new_single(groups), aggregates.clone(), vec![None], - vec![None], Arc::new(EmptyExec::new(schema.clone())), schema, )?)) From 06d3bcca5b4070e41429ddbd01c5d8155a5b6084 Mon Sep 17 00:00:00 2001 From: Georgi Krastev Date: Thu, 14 Dec 2023 21:19:23 +0100 Subject: [PATCH 432/572] Fix count(null) and count(distinct null) (#8511) Use `logical_nulls` when the array data type is `Null`. --- datafusion/physical-expr/src/aggregate/count.rs | 10 ++++++---- .../src/aggregate/count_distinct.rs | 5 +++++ .../sqllogictest/test_files/aggregate.slt | 17 ++++++++++++++++- 3 files changed, 27 insertions(+), 5 deletions(-) diff --git a/datafusion/physical-expr/src/aggregate/count.rs b/datafusion/physical-expr/src/aggregate/count.rs index 738ca4e915f7..8e9ae5cea36b 100644 --- a/datafusion/physical-expr/src/aggregate/count.rs +++ b/datafusion/physical-expr/src/aggregate/count.rs @@ -123,7 +123,7 @@ impl GroupsAccumulator for CountGroupsAccumulator { self.counts.resize(total_num_groups, 0); accumulate_indices( group_indices, - values.nulls(), // ignore values + values.logical_nulls().as_ref(), opt_filter, |group_index| { self.counts[group_index] += 1; @@ -198,16 +198,18 @@ fn null_count_for_multiple_cols(values: &[ArrayRef]) -> usize { if values.len() > 1 { let result_bool_buf: Option = values .iter() - .map(|a| a.nulls()) + .map(|a| a.logical_nulls()) .fold(None, |acc, b| match (acc, b) { (Some(acc), Some(b)) => Some(acc.bitand(b.inner())), (Some(acc), None) => Some(acc), - (None, Some(b)) => Some(b.inner().clone()), + (None, Some(b)) => Some(b.into_inner()), _ => None, }); result_bool_buf.map_or(0, |b| values[0].len() - b.count_set_bits()) } else { - values[0].null_count() + values[0] + .logical_nulls() + .map_or(0, |nulls| nulls.null_count()) } } diff --git a/datafusion/physical-expr/src/aggregate/count_distinct.rs b/datafusion/physical-expr/src/aggregate/count_distinct.rs index f5242d983d4c..c2fd32a96c4f 100644 --- a/datafusion/physical-expr/src/aggregate/count_distinct.rs +++ b/datafusion/physical-expr/src/aggregate/count_distinct.rs @@ -152,7 +152,12 @@ impl Accumulator for DistinctCountAccumulator { if values.is_empty() { return Ok(()); } + let arr = &values[0]; + if arr.data_type() == &DataType::Null { + return Ok(()); + } + (0..arr.len()).try_for_each(|index| { if !arr.is_null(index) { let scalar = ScalarValue::try_from_array(arr, index)?; diff --git a/datafusion/sqllogictest/test_files/aggregate.slt b/datafusion/sqllogictest/test_files/aggregate.slt index bcda3464f49b..78575c9dffc5 100644 --- a/datafusion/sqllogictest/test_files/aggregate.slt +++ b/datafusion/sqllogictest/test_files/aggregate.slt @@ -1492,6 +1492,12 @@ SELECT count(c1, c2) FROM test ---- 3 +# count_null +query III +SELECT count(null), count(null, null), count(distinct null) FROM test +---- +0 0 0 + # count_multi_expr_group_by query I SELECT count(c1, c2) FROM test group by c1 order by c1 @@ -1501,6 +1507,15 @@ SELECT count(c1, c2) FROM test group by c1 order by c1 2 0 +# count_null_group_by +query III +SELECT count(null), count(null, null), count(distinct null) FROM test group by c1 order by c1 +---- +0 0 0 +0 0 0 +0 0 0 +0 0 0 + # aggreggte_with_alias query II select c1, sum(c2) as `Total Salary` from test group by c1 order by c1 @@ -3241,4 +3256,4 @@ select count(*) from (select count(*) from (select 1)); query I select count(*) from (select count(*) a, count(*) b from (select 1)); ---- -1 \ No newline at end of file +1 From 5be8dbe0e5f45984b5e6480d8766373f3bbff93d Mon Sep 17 00:00:00 2001 From: Alex Huang Date: Thu, 14 Dec 2023 21:27:36 +0100 Subject: [PATCH 433/572] Minor: reduce code duplication in `date_bin_impl` (#8528) * reduce code duplication in date_bin_impl --- .../physical-expr/src/datetime_expressions.rs | 143 ++++++++++-------- 1 file changed, 78 insertions(+), 65 deletions(-) diff --git a/datafusion/physical-expr/src/datetime_expressions.rs b/datafusion/physical-expr/src/datetime_expressions.rs index bbeb2b0dce86..f6373d40d965 100644 --- a/datafusion/physical-expr/src/datetime_expressions.rs +++ b/datafusion/physical-expr/src/datetime_expressions.rs @@ -21,12 +21,6 @@ use crate::datetime_expressions; use crate::expressions::cast_column; use arrow::array::Float64Builder; use arrow::compute::cast; -use arrow::{ - array::TimestampNanosecondArray, - compute::kernels::temporal, - datatypes::TimeUnit, - temporal_conversions::{as_datetime_with_timezone, timestamp_ns_to_datetime}, -}; use arrow::{ array::{Array, ArrayRef, Float64Array, OffsetSizeTrait, PrimitiveArray}, compute::kernels::cast_utils::string_to_timestamp_nanos, @@ -36,11 +30,14 @@ use arrow::{ TimestampMillisecondType, TimestampNanosecondType, TimestampSecondType, }, }; -use arrow_array::types::ArrowTimestampType; -use arrow_array::{ - timezone::Tz, TimestampMicrosecondArray, TimestampMillisecondArray, - TimestampSecondArray, +use arrow::{ + compute::kernels::temporal, + datatypes::TimeUnit, + temporal_conversions::{as_datetime_with_timezone, timestamp_ns_to_datetime}, }; +use arrow_array::temporal_conversions::NANOSECONDS; +use arrow_array::timezone::Tz; +use arrow_array::types::ArrowTimestampType; use chrono::prelude::*; use chrono::{Duration, Months, NaiveDate}; use datafusion_common::cast::{ @@ -647,89 +644,104 @@ fn date_bin_impl( return exec_err!("DATE_BIN stride must be non-zero"); } - let f_nanos = |x: Option| x.map(|x| stride_fn(stride, x, origin)); - let f_micros = |x: Option| { - let scale = 1_000; - x.map(|x| stride_fn(stride, x * scale, origin) / scale) - }; - let f_millis = |x: Option| { - let scale = 1_000_000; - x.map(|x| stride_fn(stride, x * scale, origin) / scale) - }; - let f_secs = |x: Option| { - let scale = 1_000_000_000; - x.map(|x| stride_fn(stride, x * scale, origin) / scale) - }; + fn stride_map_fn( + origin: i64, + stride: i64, + stride_fn: fn(i64, i64, i64) -> i64, + ) -> impl Fn(Option) -> Option { + let scale = match T::UNIT { + TimeUnit::Nanosecond => 1, + TimeUnit::Microsecond => NANOSECONDS / 1_000_000, + TimeUnit::Millisecond => NANOSECONDS / 1_000, + TimeUnit::Second => NANOSECONDS, + }; + move |x: Option| x.map(|x| stride_fn(stride, x * scale, origin) / scale) + } Ok(match array { ColumnarValue::Scalar(ScalarValue::TimestampNanosecond(v, tz_opt)) => { + let apply_stride_fn = + stride_map_fn::(origin, stride, stride_fn); ColumnarValue::Scalar(ScalarValue::TimestampNanosecond( - f_nanos(*v), + apply_stride_fn(*v), tz_opt.clone(), )) } ColumnarValue::Scalar(ScalarValue::TimestampMicrosecond(v, tz_opt)) => { + let apply_stride_fn = + stride_map_fn::(origin, stride, stride_fn); ColumnarValue::Scalar(ScalarValue::TimestampMicrosecond( - f_micros(*v), + apply_stride_fn(*v), tz_opt.clone(), )) } ColumnarValue::Scalar(ScalarValue::TimestampMillisecond(v, tz_opt)) => { + let apply_stride_fn = + stride_map_fn::(origin, stride, stride_fn); ColumnarValue::Scalar(ScalarValue::TimestampMillisecond( - f_millis(*v), + apply_stride_fn(*v), tz_opt.clone(), )) } ColumnarValue::Scalar(ScalarValue::TimestampSecond(v, tz_opt)) => { + let apply_stride_fn = + stride_map_fn::(origin, stride, stride_fn); ColumnarValue::Scalar(ScalarValue::TimestampSecond( - f_secs(*v), + apply_stride_fn(*v), tz_opt.clone(), )) } - ColumnarValue::Array(array) => match array.data_type() { - DataType::Timestamp(TimeUnit::Nanosecond, tz_opt) => { - let array = as_timestamp_nanosecond_array(array)? - .iter() - .map(f_nanos) - .collect::() - .with_timezone_opt(tz_opt.clone()); - - ColumnarValue::Array(Arc::new(array)) - } - DataType::Timestamp(TimeUnit::Microsecond, tz_opt) => { - let array = as_timestamp_microsecond_array(array)? - .iter() - .map(f_micros) - .collect::() - .with_timezone_opt(tz_opt.clone()); - - ColumnarValue::Array(Arc::new(array)) - } - DataType::Timestamp(TimeUnit::Millisecond, tz_opt) => { - let array = as_timestamp_millisecond_array(array)? - .iter() - .map(f_millis) - .collect::() - .with_timezone_opt(tz_opt.clone()); - ColumnarValue::Array(Arc::new(array)) - } - DataType::Timestamp(TimeUnit::Second, tz_opt) => { - let array = as_timestamp_second_array(array)? + ColumnarValue::Array(array) => { + fn transform_array_with_stride( + origin: i64, + stride: i64, + stride_fn: fn(i64, i64, i64) -> i64, + array: &ArrayRef, + tz_opt: &Option>, + ) -> Result + where + T: ArrowTimestampType, + { + let array = as_primitive_array::(array)?; + let apply_stride_fn = stride_map_fn::(origin, stride, stride_fn); + let array = array .iter() - .map(f_secs) - .collect::() + .map(apply_stride_fn) + .collect::>() .with_timezone_opt(tz_opt.clone()); - ColumnarValue::Array(Arc::new(array)) + Ok(ColumnarValue::Array(Arc::new(array))) } - _ => { - return exec_err!( - "DATE_BIN expects source argument to be a TIMESTAMP but got {}", - array.data_type() - ) + match array.data_type() { + DataType::Timestamp(TimeUnit::Nanosecond, tz_opt) => { + transform_array_with_stride::( + origin, stride, stride_fn, array, tz_opt, + )? + } + DataType::Timestamp(TimeUnit::Microsecond, tz_opt) => { + transform_array_with_stride::( + origin, stride, stride_fn, array, tz_opt, + )? + } + DataType::Timestamp(TimeUnit::Millisecond, tz_opt) => { + transform_array_with_stride::( + origin, stride, stride_fn, array, tz_opt, + )? + } + DataType::Timestamp(TimeUnit::Second, tz_opt) => { + transform_array_with_stride::( + origin, stride, stride_fn, array, tz_opt, + )? + } + _ => { + return exec_err!( + "DATE_BIN expects source argument to be a TIMESTAMP but got {}", + array.data_type() + ) + } } - }, + } _ => { return exec_err!( "DATE_BIN expects source argument to be a TIMESTAMP scalar or array" @@ -1061,6 +1073,7 @@ mod tests { use arrow::array::{ as_primitive_array, ArrayRef, Int64Array, IntervalDayTimeArray, StringBuilder, }; + use arrow_array::TimestampNanosecondArray; use super::*; From 72e39b8bce867d1f141356a918400b185e6efe74 Mon Sep 17 00:00:00 2001 From: Simon Vandel Sillesen Date: Thu, 14 Dec 2023 23:28:25 +0300 Subject: [PATCH 434/572] Add metrics for UnnestExec (#8482) --- datafusion/core/tests/dataframe/mod.rs | 24 ++++++- datafusion/physical-plan/src/unnest.rs | 92 ++++++++++++++++++-------- 2 files changed, 86 insertions(+), 30 deletions(-) diff --git a/datafusion/core/tests/dataframe/mod.rs b/datafusion/core/tests/dataframe/mod.rs index c6b8e0e01b4f..ba661aa2445c 100644 --- a/datafusion/core/tests/dataframe/mod.rs +++ b/datafusion/core/tests/dataframe/mod.rs @@ -39,7 +39,7 @@ use datafusion::prelude::JoinType; use datafusion::prelude::{CsvReadOptions, ParquetReadOptions}; use datafusion::test_util::parquet_test_data; use datafusion::{assert_batches_eq, assert_batches_sorted_eq}; -use datafusion_common::{DataFusionError, ScalarValue, UnnestOptions}; +use datafusion_common::{assert_contains, DataFusionError, ScalarValue, UnnestOptions}; use datafusion_execution::config::SessionConfig; use datafusion_expr::expr::{GroupingSet, Sort}; use datafusion_expr::{ @@ -1408,6 +1408,28 @@ async fn unnest_with_redundant_columns() -> Result<()> { Ok(()) } +#[tokio::test] +async fn unnest_analyze_metrics() -> Result<()> { + const NUM_ROWS: usize = 5; + + let df = table_with_nested_types(NUM_ROWS).await?; + let results = df + .unnest_column("tags")? + .explain(false, true)? + .collect() + .await?; + let formatted = arrow::util::pretty::pretty_format_batches(&results) + .unwrap() + .to_string(); + assert_contains!(&formatted, "elapsed_compute="); + assert_contains!(&formatted, "input_batches=1"); + assert_contains!(&formatted, "input_rows=5"); + assert_contains!(&formatted, "output_rows=10"); + assert_contains!(&formatted, "output_batches=1"); + + Ok(()) +} + async fn create_test_table(name: &str) -> Result { let schema = Arc::new(Schema::new(vec![ Field::new("a", DataType::Utf8, false), diff --git a/datafusion/physical-plan/src/unnest.rs b/datafusion/physical-plan/src/unnest.rs index af4a81626cd7..b9e732c317af 100644 --- a/datafusion/physical-plan/src/unnest.rs +++ b/datafusion/physical-plan/src/unnest.rs @@ -17,8 +17,6 @@ //! Defines the unnest column plan for unnesting values in a column that contains a list //! type, conceptually is like joining each row with all the values in the list column. - -use std::time::Instant; use std::{any::Any, sync::Arc}; use super::DisplayAs; @@ -44,6 +42,8 @@ use async_trait::async_trait; use futures::{Stream, StreamExt}; use log::trace; +use super::metrics::{self, ExecutionPlanMetricsSet, MetricBuilder, MetricsSet}; + /// Unnest the given column by joining the row with each value in the /// nested type. /// @@ -58,6 +58,8 @@ pub struct UnnestExec { column: Column, /// Options options: UnnestOptions, + /// Execution metrics + metrics: ExecutionPlanMetricsSet, } impl UnnestExec { @@ -73,6 +75,7 @@ impl UnnestExec { schema, column, options, + metrics: Default::default(), } } } @@ -141,19 +144,58 @@ impl ExecutionPlan for UnnestExec { context: Arc, ) -> Result { let input = self.input.execute(partition, context)?; + let metrics = UnnestMetrics::new(partition, &self.metrics); Ok(Box::pin(UnnestStream { input, schema: self.schema.clone(), column: self.column.clone(), options: self.options.clone(), - num_input_batches: 0, - num_input_rows: 0, - num_output_batches: 0, - num_output_rows: 0, - unnest_time: 0, + metrics, })) } + + fn metrics(&self) -> Option { + Some(self.metrics.clone_inner()) + } +} + +#[derive(Clone, Debug)] +struct UnnestMetrics { + /// total time for column unnesting + elapsed_compute: metrics::Time, + /// Number of batches consumed + input_batches: metrics::Count, + /// Number of rows consumed + input_rows: metrics::Count, + /// Number of batches produced + output_batches: metrics::Count, + /// Number of rows produced by this operator + output_rows: metrics::Count, +} + +impl UnnestMetrics { + fn new(partition: usize, metrics: &ExecutionPlanMetricsSet) -> Self { + let elapsed_compute = MetricBuilder::new(metrics).elapsed_compute(partition); + + let input_batches = + MetricBuilder::new(metrics).counter("input_batches", partition); + + let input_rows = MetricBuilder::new(metrics).counter("input_rows", partition); + + let output_batches = + MetricBuilder::new(metrics).counter("output_batches", partition); + + let output_rows = MetricBuilder::new(metrics).output_rows(partition); + + Self { + input_batches, + input_rows, + output_batches, + output_rows, + elapsed_compute, + } + } } /// A stream that issues [RecordBatch]es with unnested column data. @@ -166,16 +208,8 @@ struct UnnestStream { column: Column, /// Options options: UnnestOptions, - /// number of input batches - num_input_batches: usize, - /// number of input rows - num_input_rows: usize, - /// number of batches produced - num_output_batches: usize, - /// number of rows produced - num_output_rows: usize, - /// total time for column unnesting, in ms - unnest_time: usize, + /// Metrics + metrics: UnnestMetrics, } impl RecordBatchStream for UnnestStream { @@ -207,15 +241,15 @@ impl UnnestStream { .poll_next_unpin(cx) .map(|maybe_batch| match maybe_batch { Some(Ok(batch)) => { - let start = Instant::now(); + let timer = self.metrics.elapsed_compute.timer(); let result = build_batch(&batch, &self.schema, &self.column, &self.options); - self.num_input_batches += 1; - self.num_input_rows += batch.num_rows(); + self.metrics.input_batches.add(1); + self.metrics.input_rows.add(batch.num_rows()); if let Ok(ref batch) = result { - self.unnest_time += start.elapsed().as_millis() as usize; - self.num_output_batches += 1; - self.num_output_rows += batch.num_rows(); + timer.done(); + self.metrics.output_batches.add(1); + self.metrics.output_rows.add(batch.num_rows()); } Some(result) @@ -223,12 +257,12 @@ impl UnnestStream { other => { trace!( "Processed {} probe-side input batches containing {} rows and \ - produced {} output batches containing {} rows in {} ms", - self.num_input_batches, - self.num_input_rows, - self.num_output_batches, - self.num_output_rows, - self.unnest_time, + produced {} output batches containing {} rows in {}", + self.metrics.input_batches, + self.metrics.input_rows, + self.metrics.output_batches, + self.metrics.output_rows, + self.metrics.elapsed_compute, ); other } From 14c99b87005cc0227bb06041ef7d8383d0c0e341 Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Thu, 14 Dec 2023 13:52:48 -0700 Subject: [PATCH 435/572] regenerate changelog (#8549) --- dev/changelog/34.0.0.md | 26 ++++++++++++++++++++++++++ 1 file changed, 26 insertions(+) diff --git a/dev/changelog/34.0.0.md b/dev/changelog/34.0.0.md index 8b8933017cfb..c5526f60531c 100644 --- a/dev/changelog/34.0.0.md +++ b/dev/changelog/34.0.0.md @@ -54,6 +54,7 @@ - feat: customize column default values for external tables [#8415](https://github.com/apache/arrow-datafusion/pull/8415) (jonahgao) - feat: Support `array_sort`(`list_sort`) [#8279](https://github.com/apache/arrow-datafusion/pull/8279) (Asura7969) - feat: support `InterleaveExecNode` in the proto [#8460](https://github.com/apache/arrow-datafusion/pull/8460) (liukun4515) +- feat: improve string statistics display in datafusion-cli `parquet_metadata` function [#8535](https://github.com/apache/arrow-datafusion/pull/8535) (asimsedhain) **Fixed bugs:** @@ -66,11 +67,15 @@ - fix: RANGE frame for corner cases with empty ORDER BY clause should be treated as constant sort [#8445](https://github.com/apache/arrow-datafusion/pull/8445) (viirya) - fix: don't unifies projection if expr is non-trival [#8454](https://github.com/apache/arrow-datafusion/pull/8454) (haohuaijin) - fix: support uppercase when parsing `Interval` [#8478](https://github.com/apache/arrow-datafusion/pull/8478) (QuenKar) +- fix: incorrect set preserve_partitioning in SortExec [#8485](https://github.com/apache/arrow-datafusion/pull/8485) (haohuaijin) +- fix: Pull stats in `IdentVisitor`/`GraphvizVisitor` only when requested [#8514](https://github.com/apache/arrow-datafusion/pull/8514) (vrongmeal) +- fix: volatile expressions should not be target of common subexpt elimination [#8520](https://github.com/apache/arrow-datafusion/pull/8520) (viirya) **Documentation updates:** - Library Guide: Add Using the DataFrame API [#8319](https://github.com/apache/arrow-datafusion/pull/8319) (Veeupup) - Minor: Add installation link to README.md [#8389](https://github.com/apache/arrow-datafusion/pull/8389) (Weijun-H) +- Prepare version 34.0.0 [#8508](https://github.com/apache/arrow-datafusion/pull/8508) (andygrove) **Merged pull requests:** @@ -245,3 +250,24 @@ - Minor: Improve comments in EnforceDistribution tests [#8474](https://github.com/apache/arrow-datafusion/pull/8474) (alamb) - fix: support uppercase when parsing `Interval` [#8478](https://github.com/apache/arrow-datafusion/pull/8478) (QuenKar) - Better Equivalence (ordering and exact equivalence) Propagation through ProjectionExec [#8484](https://github.com/apache/arrow-datafusion/pull/8484) (mustafasrepo) +- Add `today` alias for `current_date` [#8423](https://github.com/apache/arrow-datafusion/pull/8423) (smallzhongfeng) +- Minor: remove useless clone in `array_expression` [#8495](https://github.com/apache/arrow-datafusion/pull/8495) (Weijun-H) +- fix: incorrect set preserve_partitioning in SortExec [#8485](https://github.com/apache/arrow-datafusion/pull/8485) (haohuaijin) +- Explicitly mark parquet for tests in datafusion-common [#8497](https://github.com/apache/arrow-datafusion/pull/8497) (Dennis40816) +- Minor/Doc: Clarify DataFrame::write_table Documentation [#8519](https://github.com/apache/arrow-datafusion/pull/8519) (devinjdangelo) +- fix: Pull stats in `IdentVisitor`/`GraphvizVisitor` only when requested [#8514](https://github.com/apache/arrow-datafusion/pull/8514) (vrongmeal) +- Change display of RepartitionExec from SortPreservingRepartitionExec to RepartitionExec preserve_order=true [#8521](https://github.com/apache/arrow-datafusion/pull/8521) (JacobOgle) +- Fix `DataFrame::cache` errors with `Plan("Mismatch between schema and batches")` [#8510](https://github.com/apache/arrow-datafusion/pull/8510) (Asura7969) +- Minor: update pbjson_dependency [#8470](https://github.com/apache/arrow-datafusion/pull/8470) (alamb) +- Minor: Update prost-derive dependency [#8471](https://github.com/apache/arrow-datafusion/pull/8471) (alamb) +- Minor/Doc: Add DataFrame::write_table to DataFrame user guide [#8527](https://github.com/apache/arrow-datafusion/pull/8527) (devinjdangelo) +- Minor: Add repartition_file.slt end to end test for repartitioning files, and supporting tweaks [#8505](https://github.com/apache/arrow-datafusion/pull/8505) (alamb) +- Prepare version 34.0.0 [#8508](https://github.com/apache/arrow-datafusion/pull/8508) (andygrove) +- refactor: use ExprBuilder to consume substrait expr and use macro to generate error [#8515](https://github.com/apache/arrow-datafusion/pull/8515) (waynexia) +- [MINOR]: Make some slt tests deterministic [#8525](https://github.com/apache/arrow-datafusion/pull/8525) (mustafasrepo) +- fix: volatile expressions should not be target of common subexpt elimination [#8520](https://github.com/apache/arrow-datafusion/pull/8520) (viirya) +- Minor: Add LakeSoul to the list of Known Users [#8536](https://github.com/apache/arrow-datafusion/pull/8536) (xuchen-plus) +- Fix regression with Incorrect results when reading parquet files with different schemas and statistics [#8533](https://github.com/apache/arrow-datafusion/pull/8533) (alamb) +- feat: improve string statistics display in datafusion-cli `parquet_metadata` function [#8535](https://github.com/apache/arrow-datafusion/pull/8535) (asimsedhain) +- Defer file creation to write [#8539](https://github.com/apache/arrow-datafusion/pull/8539) (tustvold) +- Minor: Improve error handling in sqllogictest runner [#8544](https://github.com/apache/arrow-datafusion/pull/8544) (alamb) From 5a24ec909b0433c4b297eeae3fed0265283d7b66 Mon Sep 17 00:00:00 2001 From: Huaijin Date: Fri, 15 Dec 2023 08:21:37 +0800 Subject: [PATCH 436/572] fix: make sure CASE WHEN pick first true branch when WHEN clause is true (#8477) * fix: make case when pick first true branch when when clause is true * add more test --- datafusion/physical-expr/src/expressions/case.rs | 4 ++++ datafusion/sqllogictest/test_files/scalar.slt | 10 ++++++++++ 2 files changed, 14 insertions(+) diff --git a/datafusion/physical-expr/src/expressions/case.rs b/datafusion/physical-expr/src/expressions/case.rs index 5fcfd61d90e4..52fb85657f4e 100644 --- a/datafusion/physical-expr/src/expressions/case.rs +++ b/datafusion/physical-expr/src/expressions/case.rs @@ -145,6 +145,8 @@ impl CaseExpr { 0 => Cow::Borrowed(&when_match), _ => Cow::Owned(prep_null_mask_filter(&when_match)), }; + // Make sure we only consider rows that have not been matched yet + let when_match = and(&when_match, &remainder)?; let then_value = self.when_then_expr[i] .1 @@ -206,6 +208,8 @@ impl CaseExpr { 0 => Cow::Borrowed(when_value), _ => Cow::Owned(prep_null_mask_filter(when_value)), }; + // Make sure we only consider rows that have not been matched yet + let when_value = and(&when_value, &remainder)?; let then_value = self.when_then_expr[i] .1 diff --git a/datafusion/sqllogictest/test_files/scalar.slt b/datafusion/sqllogictest/test_files/scalar.slt index b3597c664fbb..9b30699e3fa3 100644 --- a/datafusion/sqllogictest/test_files/scalar.slt +++ b/datafusion/sqllogictest/test_files/scalar.slt @@ -1943,3 +1943,13 @@ select ; ---- true true true true true true true true true true + +query I +SELECT ALL - CASE WHEN NOT - AVG ( - 41 ) IS NULL THEN 47 WHEN NULL IS NULL THEN COUNT ( * ) END + 93 + - - 44 * 91 + CASE + 44 WHEN - - 21 * 69 - 12 THEN 58 ELSE - 3 END * + + 23 * + 84 * - - 59 +---- +-337914 + +query T +SELECT CASE 3 WHEN 1+2 THEN 'first' WHEN 1+1+1 THEN 'second' END +---- +first From b457f2b70682f0d574b841ec5e38a3d6709dd2a0 Mon Sep 17 00:00:00 2001 From: Bo Lin Date: Thu, 14 Dec 2023 19:22:25 -0500 Subject: [PATCH 437/572] Minor: make SubqueryAlias::try_new take Arc (#8542) Currently, all `#[non_exhaustive]` logical plan structs with a `try_new` constructor take `Arc` as parameter, except for `SubqueryAlias`, which takes a `LogicalPlan`. This changes `SubqueryAlias::try_new` to align with the other plan types, to improve API ergonomics. --- datafusion/expr/src/logical_plan/builder.rs | 2 +- datafusion/expr/src/logical_plan/plan.rs | 6 +++--- datafusion/proto/src/logical_plan/mod.rs | 2 +- 3 files changed, 5 insertions(+), 5 deletions(-) diff --git a/datafusion/expr/src/logical_plan/builder.rs b/datafusion/expr/src/logical_plan/builder.rs index be2c45b901fa..88310dab82a2 100644 --- a/datafusion/expr/src/logical_plan/builder.rs +++ b/datafusion/expr/src/logical_plan/builder.rs @@ -1343,7 +1343,7 @@ pub fn subquery_alias( plan: LogicalPlan, alias: impl Into, ) -> Result { - SubqueryAlias::try_new(plan, alias).map(LogicalPlan::SubqueryAlias) + SubqueryAlias::try_new(Arc::new(plan), alias).map(LogicalPlan::SubqueryAlias) } /// Create a LogicalPlanBuilder representing a scan of a table with the provided name and schema. diff --git a/datafusion/expr/src/logical_plan/plan.rs b/datafusion/expr/src/logical_plan/plan.rs index d74015bf094d..1f3711407a14 100644 --- a/datafusion/expr/src/logical_plan/plan.rs +++ b/datafusion/expr/src/logical_plan/plan.rs @@ -792,7 +792,7 @@ impl LogicalPlan { })) } LogicalPlan::SubqueryAlias(SubqueryAlias { alias, .. }) => { - SubqueryAlias::try_new(inputs[0].clone(), alias.clone()) + SubqueryAlias::try_new(Arc::new(inputs[0].clone()), alias.clone()) .map(LogicalPlan::SubqueryAlias) } LogicalPlan::Limit(Limit { skip, fetch, .. }) => { @@ -1855,7 +1855,7 @@ pub struct SubqueryAlias { impl SubqueryAlias { pub fn try_new( - plan: LogicalPlan, + plan: Arc, alias: impl Into, ) -> Result { let alias = alias.into(); @@ -1868,7 +1868,7 @@ impl SubqueryAlias { .with_functional_dependencies(func_dependencies)?, ); Ok(SubqueryAlias { - input: Arc::new(plan), + input: plan, alias, schema, }) diff --git a/datafusion/proto/src/logical_plan/mod.rs b/datafusion/proto/src/logical_plan/mod.rs index 50bca0295def..948228d87d46 100644 --- a/datafusion/proto/src/logical_plan/mod.rs +++ b/datafusion/proto/src/logical_plan/mod.rs @@ -253,7 +253,7 @@ impl AsLogicalPlan for LogicalPlanNode { Some(a) => match a { protobuf::projection_node::OptionalAlias::Alias(alias) => { Ok(LogicalPlan::SubqueryAlias(SubqueryAlias::try_new( - new_proj, + Arc::new(new_proj), alias.clone(), )?)) } From e1a9177b8a5cb7bbb9a56db4a5c0e6f842af8fac Mon Sep 17 00:00:00 2001 From: Mohammad Razeghi Date: Fri, 15 Dec 2023 01:24:54 +0100 Subject: [PATCH 438/572] Fallback on null empty value in ExprBoundaries::try_from_column (#8501) * Fallback on null empty value in ExprBoundaries::try_from_column * Add test --- datafusion/physical-expr/src/analysis.rs | 3 ++- datafusion/sqllogictest/src/test_context.rs | 21 +++++++++++++++++++ .../sqllogictest/test_files/explain.slt | 4 ++++ 3 files changed, 27 insertions(+), 1 deletion(-) diff --git a/datafusion/physical-expr/src/analysis.rs b/datafusion/physical-expr/src/analysis.rs index f43434362a19..6d36e2233cdd 100644 --- a/datafusion/physical-expr/src/analysis.rs +++ b/datafusion/physical-expr/src/analysis.rs @@ -95,7 +95,8 @@ impl ExprBoundaries { col_index: usize, ) -> Result { let field = &schema.fields()[col_index]; - let empty_field = ScalarValue::try_from(field.data_type())?; + let empty_field = + ScalarValue::try_from(field.data_type()).unwrap_or(ScalarValue::Null); let interval = Interval::try_new( col_stats .min_value diff --git a/datafusion/sqllogictest/src/test_context.rs b/datafusion/sqllogictest/src/test_context.rs index f5ab8f71aaaf..91093510afec 100644 --- a/datafusion/sqllogictest/src/test_context.rs +++ b/datafusion/sqllogictest/src/test_context.rs @@ -84,6 +84,10 @@ impl TestContext { info!("Registering table with many types"); register_table_with_many_types(test_ctx.session_ctx()).await; } + "explain.slt" => { + info!("Registering table with map"); + register_table_with_map(test_ctx.session_ctx()).await; + } "avro.slt" => { #[cfg(feature = "avro")] { @@ -268,6 +272,23 @@ pub async fn register_table_with_many_types(ctx: &SessionContext) { .unwrap(); } +pub async fn register_table_with_map(ctx: &SessionContext) { + let key = Field::new("key", DataType::Int64, false); + let value = Field::new("value", DataType::Int64, true); + let map_field = + Field::new("entries", DataType::Struct(vec![key, value].into()), false); + let fields = vec![ + Field::new("int_field", DataType::Int64, true), + Field::new("map_field", DataType::Map(map_field.into(), false), true), + ]; + let schema = Schema::new(fields); + + let memory_table = MemTable::try_new(schema.into(), vec![vec![]]).unwrap(); + + ctx.register_table("table_with_map", Arc::new(memory_table)) + .unwrap(); +} + fn table_with_many_types() -> Arc { let schema = Schema::new(vec![ Field::new("int32_col", DataType::Int32, false), diff --git a/datafusion/sqllogictest/test_files/explain.slt b/datafusion/sqllogictest/test_files/explain.slt index 4583ef319b7f..a51c3aed13ec 100644 --- a/datafusion/sqllogictest/test_files/explain.slt +++ b/datafusion/sqllogictest/test_files/explain.slt @@ -379,3 +379,7 @@ Projection: List([[1, 2, 3], [4, 5, 6]]) AS make_array(make_array(Int64(1),Int64 physical_plan ProjectionExec: expr=[[[1, 2, 3], [4, 5, 6]] as make_array(make_array(Int64(1),Int64(2),Int64(3)),make_array(Int64(4),Int64(5),Int64(6)))] --PlaceholderRowExec + +# Testing explain on a table with a map filter, registered in test_context.rs. +statement ok +explain select * from table_with_map where int_field > 0 From b276d479918400105017db1f7f46dcb67b52206d Mon Sep 17 00:00:00 2001 From: Devin D'Angelo Date: Thu, 14 Dec 2023 19:32:46 -0500 Subject: [PATCH 439/572] Add test for DataFrame::write_table (#8531) * add test for DataFrame::write_table * remove duplicate let df=... * remove println! --- .../datasource/physical_plan/parquet/mod.rs | 95 ++++++++++++++++++- 1 file changed, 92 insertions(+), 3 deletions(-) diff --git a/datafusion/core/src/datasource/physical_plan/parquet/mod.rs b/datafusion/core/src/datasource/physical_plan/parquet/mod.rs index 641b7bbb1596..847ea6505632 100644 --- a/datafusion/core/src/datasource/physical_plan/parquet/mod.rs +++ b/datafusion/core/src/datasource/physical_plan/parquet/mod.rs @@ -752,7 +752,7 @@ mod tests { use crate::datasource::file_format::options::CsvReadOptions; use crate::datasource::file_format::parquet::test_util::store_parquet; use crate::datasource::file_format::test_util::scan_format; - use crate::datasource::listing::{FileRange, PartitionedFile}; + use crate::datasource::listing::{FileRange, ListingOptions, PartitionedFile}; use crate::datasource::object_store::ObjectStoreUrl; use crate::execution::context::SessionState; use crate::physical_plan::displayable; @@ -772,8 +772,8 @@ mod tests { }; use arrow_array::Date64Array; use chrono::{TimeZone, Utc}; - use datafusion_common::ScalarValue; use datafusion_common::{assert_contains, ToDFSchema}; + use datafusion_common::{FileType, GetExt, ScalarValue}; use datafusion_expr::{col, lit, when, Expr}; use datafusion_physical_expr::create_physical_expr; use datafusion_physical_expr::execution_props::ExecutionProps; @@ -1941,6 +1941,96 @@ mod tests { Ok(schema) } + #[tokio::test] + async fn write_table_results() -> Result<()> { + // create partitioned input file and context + let tmp_dir = TempDir::new()?; + // let mut ctx = create_ctx(&tmp_dir, 4).await?; + let ctx = SessionContext::new_with_config( + SessionConfig::new().with_target_partitions(8), + ); + let schema = populate_csv_partitions(&tmp_dir, 4, ".csv")?; + // register csv file with the execution context + ctx.register_csv( + "test", + tmp_dir.path().to_str().unwrap(), + CsvReadOptions::new().schema(&schema), + ) + .await?; + + // register a local file system object store for /tmp directory + let local = Arc::new(LocalFileSystem::new_with_prefix(&tmp_dir)?); + let local_url = Url::parse("file://local").unwrap(); + ctx.runtime_env().register_object_store(&local_url, local); + + // Configure listing options + let file_format = ParquetFormat::default().with_enable_pruning(Some(true)); + let listing_options = ListingOptions::new(Arc::new(file_format)) + .with_file_extension(FileType::PARQUET.get_ext()); + + // execute a simple query and write the results to parquet + let out_dir = tmp_dir.as_ref().to_str().unwrap().to_string() + "/out"; + std::fs::create_dir(&out_dir).unwrap(); + let df = ctx.sql("SELECT c1, c2 FROM test").await?; + let schema: Schema = df.schema().into(); + // Register a listing table - this will use all files in the directory as data sources + // for the query + ctx.register_listing_table( + "my_table", + &out_dir, + listing_options, + Some(Arc::new(schema)), + None, + ) + .await + .unwrap(); + df.write_table("my_table", DataFrameWriteOptions::new()) + .await?; + + // create a new context and verify that the results were saved to a partitioned parquet file + let ctx = SessionContext::new(); + + // get write_id + let mut paths = fs::read_dir(&out_dir).unwrap(); + let path = paths.next(); + let name = path + .unwrap()? + .path() + .file_name() + .expect("Should be a file name") + .to_str() + .expect("Should be a str") + .to_owned(); + let (parsed_id, _) = name.split_once('_').expect("File should contain _ !"); + let write_id = parsed_id.to_owned(); + + // register each partition as well as the top level dir + ctx.register_parquet( + "part0", + &format!("{out_dir}/{write_id}_0.parquet"), + ParquetReadOptions::default(), + ) + .await?; + + ctx.register_parquet("allparts", &out_dir, ParquetReadOptions::default()) + .await?; + + let part0 = ctx.sql("SELECT c1, c2 FROM part0").await?.collect().await?; + let allparts = ctx + .sql("SELECT c1, c2 FROM allparts") + .await? + .collect() + .await?; + + let allparts_count: usize = allparts.iter().map(|batch| batch.num_rows()).sum(); + + assert_eq!(part0[0].schema(), allparts[0].schema()); + + assert_eq!(allparts_count, 40); + + Ok(()) + } + #[tokio::test] async fn write_parquet_results() -> Result<()> { // create partitioned input file and context @@ -1985,7 +2075,6 @@ mod tests { .to_str() .expect("Should be a str") .to_owned(); - println!("{name}"); let (parsed_id, _) = name.split_once('_').expect("File should contain _ !"); let write_id = parsed_id.to_owned(); From 28e7f60cf7d4fb87eeaf4e4c1102eb54bfb67426 Mon Sep 17 00:00:00 2001 From: Mustafa Akur <106137913+mustafasrepo@users.noreply.github.com> Date: Fri, 15 Dec 2023 15:00:10 +0300 Subject: [PATCH 440/572] Generate empty column at placeholder exec (#8553) --- datafusion/physical-plan/src/placeholder_row.rs | 7 ++++--- datafusion/sqllogictest/test_files/window.slt | 6 ++++++ 2 files changed, 10 insertions(+), 3 deletions(-) diff --git a/datafusion/physical-plan/src/placeholder_row.rs b/datafusion/physical-plan/src/placeholder_row.rs index 94f32788530b..3ab3de62f37a 100644 --- a/datafusion/physical-plan/src/placeholder_row.rs +++ b/datafusion/physical-plan/src/placeholder_row.rs @@ -27,6 +27,7 @@ use crate::{memory::MemoryStream, DisplayFormatType, ExecutionPlan, Partitioning use arrow::array::{ArrayRef, NullArray}; use arrow::datatypes::{DataType, Field, Fields, Schema, SchemaRef}; use arrow::record_batch::RecordBatch; +use arrow_array::RecordBatchOptions; use datafusion_common::{internal_err, DataFusionError, Result}; use datafusion_execution::TaskContext; @@ -59,9 +60,7 @@ impl PlaceholderRowExec { fn data(&self) -> Result> { Ok({ let n_field = self.schema.fields.len(); - // hack for https://github.com/apache/arrow-datafusion/pull/3242 - let n_field = if n_field == 0 { 1 } else { n_field }; - vec![RecordBatch::try_new( + vec![RecordBatch::try_new_with_options( Arc::new(Schema::new( (0..n_field) .map(|i| { @@ -75,6 +74,8 @@ impl PlaceholderRowExec { ret }) .collect(), + // Even if column number is empty we can generate single row. + &RecordBatchOptions::new().with_row_count(Some(1)), )?] }) } diff --git a/datafusion/sqllogictest/test_files/window.slt b/datafusion/sqllogictest/test_files/window.slt index 7b628f9b6f14..6198209aaac5 100644 --- a/datafusion/sqllogictest/test_files/window.slt +++ b/datafusion/sqllogictest/test_files/window.slt @@ -3793,3 +3793,9 @@ select a, ---- 1 1 2 1 + +query I +select rank() over (order by 1) rnk from (select 1 a union all select 2 a) x +---- +1 +1 From f54eeea08eafc1c434d67ede4f39d5c2fb14dfdb Mon Sep 17 00:00:00 2001 From: Andrew Lamb Date: Fri, 15 Dec 2023 07:18:50 -0500 Subject: [PATCH 441/572] Minor: Remove now dead SUPPORTED_STRUCT_TYPES (#8480) --- datafusion/expr/src/lib.rs | 1 - datafusion/expr/src/struct_expressions.rs | 35 ----------------------- 2 files changed, 36 deletions(-) delete mode 100644 datafusion/expr/src/struct_expressions.rs diff --git a/datafusion/expr/src/lib.rs b/datafusion/expr/src/lib.rs index 6172d17365ad..48532e13dcd7 100644 --- a/datafusion/expr/src/lib.rs +++ b/datafusion/expr/src/lib.rs @@ -49,7 +49,6 @@ pub mod field_util; pub mod function; pub mod interval_arithmetic; pub mod logical_plan; -pub mod struct_expressions; pub mod tree_node; pub mod type_coercion; pub mod utils; diff --git a/datafusion/expr/src/struct_expressions.rs b/datafusion/expr/src/struct_expressions.rs deleted file mode 100644 index bbfcac0e2396..000000000000 --- a/datafusion/expr/src/struct_expressions.rs +++ /dev/null @@ -1,35 +0,0 @@ -// Licensed to the Apache Software Foundation (ASF) under one -// or more contributor license agreements. See the NOTICE file -// distributed with this work for additional information -// regarding copyright ownership. The ASF licenses this file -// to you under the Apache License, Version 2.0 (the -// "License"); you may not use this file except in compliance -// with the License. You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, -// software distributed under the License is distributed on an -// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -// KIND, either express or implied. See the License for the -// specific language governing permissions and limitations -// under the License. - -use arrow::datatypes::DataType; - -/// Currently supported types by the struct function. -pub static SUPPORTED_STRUCT_TYPES: &[DataType] = &[ - DataType::Boolean, - DataType::UInt8, - DataType::UInt16, - DataType::UInt32, - DataType::UInt64, - DataType::Int8, - DataType::Int16, - DataType::Int32, - DataType::Int64, - DataType::Float32, - DataType::Float64, - DataType::Utf8, - DataType::LargeUtf8, -]; From 82235aeaec0eb096b762181ce323f4e39f8250a9 Mon Sep 17 00:00:00 2001 From: Mustafa Akur <106137913+mustafasrepo@users.noreply.github.com> Date: Fri, 15 Dec 2023 16:14:54 +0300 Subject: [PATCH 442/572] [MINOR]: Add getter methods to first and last value (#8555) --- .../physical-expr/src/aggregate/first_last.rs | 50 +++++++++++++++++++ 1 file changed, 50 insertions(+) diff --git a/datafusion/physical-expr/src/aggregate/first_last.rs b/datafusion/physical-expr/src/aggregate/first_last.rs index 0dc27dede8b6..5e2012bdbb67 100644 --- a/datafusion/physical-expr/src/aggregate/first_last.rs +++ b/datafusion/physical-expr/src/aggregate/first_last.rs @@ -61,6 +61,31 @@ impl FirstValue { ordering_req, } } + + /// Returns the name of the aggregate expression. + pub fn name(&self) -> &str { + &self.name + } + + /// Returns the input data type of the aggregate expression. + pub fn input_data_type(&self) -> &DataType { + &self.input_data_type + } + + /// Returns the data types of the order-by columns. + pub fn order_by_data_types(&self) -> &Vec { + &self.order_by_data_types + } + + /// Returns the expression associated with the aggregate function. + pub fn expr(&self) -> &Arc { + &self.expr + } + + /// Returns the lexical ordering requirements of the aggregate expression. + pub fn ordering_req(&self) -> &LexOrdering { + &self.ordering_req + } } impl AggregateExpr for FirstValue { @@ -285,6 +310,31 @@ impl LastValue { ordering_req, } } + + /// Returns the name of the aggregate expression. + pub fn name(&self) -> &str { + &self.name + } + + /// Returns the input data type of the aggregate expression. + pub fn input_data_type(&self) -> &DataType { + &self.input_data_type + } + + /// Returns the data types of the order-by columns. + pub fn order_by_data_types(&self) -> &Vec { + &self.order_by_data_types + } + + /// Returns the expression associated with the aggregate function. + pub fn expr(&self) -> &Arc { + &self.expr + } + + /// Returns the lexical ordering requirements of the aggregate expression. + pub fn ordering_req(&self) -> &LexOrdering { + &self.ordering_req + } } impl AggregateExpr for LastValue { From bf0073c03ace1e4212f5895c529592d9925bf28d Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Metehan=20Y=C4=B1ld=C4=B1r=C4=B1m?= <100111937+metesynnada@users.noreply.github.com> Date: Fri, 15 Dec 2023 16:10:12 +0200 Subject: [PATCH 443/572] [MINOR]: Some code changes and a new empty batch guard for SHJ (#8557) * minor changes * Fix imports --------- Co-authored-by: Mehmet Ozan Kabak --- .../src/joins/stream_join_utils.rs | 83 ++++++++++++++++++- .../src/joins/symmetric_hash_join.rs | 64 +------------- 2 files changed, 83 insertions(+), 64 deletions(-) diff --git a/datafusion/physical-plan/src/joins/stream_join_utils.rs b/datafusion/physical-plan/src/joins/stream_join_utils.rs index 5083f96b01fb..2f74bd1c4bb2 100644 --- a/datafusion/physical-plan/src/joins/stream_join_utils.rs +++ b/datafusion/physical-plan/src/joins/stream_join_utils.rs @@ -23,8 +23,9 @@ use std::sync::Arc; use std::task::{Context, Poll}; use std::usize; -use crate::handle_async_state; use crate::joins::utils::{JoinFilter, JoinHashMapType}; +use crate::metrics::{ExecutionPlanMetricsSet, MetricBuilder}; +use crate::{handle_async_state, metrics}; use arrow::compute::concat_batches; use arrow_array::{ArrowPrimitiveType, NativeAdapter, PrimitiveArray, RecordBatch}; @@ -824,6 +825,10 @@ pub trait EagerJoinStream { ) -> Result>> { match self.right_stream().next().await { Some(Ok(batch)) => { + if batch.num_rows() == 0 { + return Ok(StreamJoinStateResult::Continue); + } + self.set_state(EagerJoinStreamState::PullLeft); self.process_batch_from_right(batch) } @@ -849,6 +854,9 @@ pub trait EagerJoinStream { ) -> Result>> { match self.left_stream().next().await { Some(Ok(batch)) => { + if batch.num_rows() == 0 { + return Ok(StreamJoinStateResult::Continue); + } self.set_state(EagerJoinStreamState::PullRight); self.process_batch_from_left(batch) } @@ -874,7 +882,12 @@ pub trait EagerJoinStream { &mut self, ) -> Result>> { match self.left_stream().next().await { - Some(Ok(batch)) => self.process_batch_after_right_end(batch), + Some(Ok(batch)) => { + if batch.num_rows() == 0 { + return Ok(StreamJoinStateResult::Continue); + } + self.process_batch_after_right_end(batch) + } Some(Err(e)) => Err(e), None => { self.set_state(EagerJoinStreamState::BothExhausted { @@ -899,7 +912,12 @@ pub trait EagerJoinStream { &mut self, ) -> Result>> { match self.right_stream().next().await { - Some(Ok(batch)) => self.process_batch_after_left_end(batch), + Some(Ok(batch)) => { + if batch.num_rows() == 0 { + return Ok(StreamJoinStateResult::Continue); + } + self.process_batch_after_left_end(batch) + } Some(Err(e)) => Err(e), None => { self.set_state(EagerJoinStreamState::BothExhausted { @@ -1020,6 +1038,65 @@ pub trait EagerJoinStream { fn state(&mut self) -> EagerJoinStreamState; } +#[derive(Debug)] +pub struct StreamJoinSideMetrics { + /// Number of batches consumed by this operator + pub(crate) input_batches: metrics::Count, + /// Number of rows consumed by this operator + pub(crate) input_rows: metrics::Count, +} + +/// Metrics for HashJoinExec +#[derive(Debug)] +pub struct StreamJoinMetrics { + /// Number of left batches/rows consumed by this operator + pub(crate) left: StreamJoinSideMetrics, + /// Number of right batches/rows consumed by this operator + pub(crate) right: StreamJoinSideMetrics, + /// Memory used by sides in bytes + pub(crate) stream_memory_usage: metrics::Gauge, + /// Number of batches produced by this operator + pub(crate) output_batches: metrics::Count, + /// Number of rows produced by this operator + pub(crate) output_rows: metrics::Count, +} + +impl StreamJoinMetrics { + pub fn new(partition: usize, metrics: &ExecutionPlanMetricsSet) -> Self { + let input_batches = + MetricBuilder::new(metrics).counter("input_batches", partition); + let input_rows = MetricBuilder::new(metrics).counter("input_rows", partition); + let left = StreamJoinSideMetrics { + input_batches, + input_rows, + }; + + let input_batches = + MetricBuilder::new(metrics).counter("input_batches", partition); + let input_rows = MetricBuilder::new(metrics).counter("input_rows", partition); + let right = StreamJoinSideMetrics { + input_batches, + input_rows, + }; + + let stream_memory_usage = + MetricBuilder::new(metrics).gauge("stream_memory_usage", partition); + + let output_batches = + MetricBuilder::new(metrics).counter("output_batches", partition); + + let output_rows = MetricBuilder::new(metrics).output_rows(partition); + + Self { + left, + right, + output_batches, + stream_memory_usage, + output_rows, + } + } +} + #[cfg(test)] pub mod tests { use std::sync::Arc; diff --git a/datafusion/physical-plan/src/joins/symmetric_hash_join.rs b/datafusion/physical-plan/src/joins/symmetric_hash_join.rs index 95f15877b960..00a7f23ebae7 100644 --- a/datafusion/physical-plan/src/joins/symmetric_hash_join.rs +++ b/datafusion/physical-plan/src/joins/symmetric_hash_join.rs @@ -37,7 +37,8 @@ use crate::joins::stream_join_utils::{ calculate_filter_expr_intervals, combine_two_batches, convert_sort_expr_with_filter_schema, get_pruning_anti_indices, get_pruning_semi_indices, record_visited_indices, EagerJoinStream, - EagerJoinStreamState, PruningJoinHashMap, SortedFilterExpr, StreamJoinStateResult, + EagerJoinStreamState, PruningJoinHashMap, SortedFilterExpr, StreamJoinMetrics, + StreamJoinStateResult, }; use crate::joins::utils::{ build_batch_from_indices, build_join_schema, check_join_is_valid, @@ -47,7 +48,7 @@ use crate::joins::utils::{ use crate::{ expressions::{Column, PhysicalSortExpr}, joins::StreamJoinPartitionMode, - metrics::{self, ExecutionPlanMetricsSet, MetricBuilder, MetricsSet}, + metrics::{ExecutionPlanMetricsSet, MetricsSet}, DisplayAs, DisplayFormatType, Distribution, EquivalenceProperties, ExecutionPlan, Partitioning, RecordBatchStream, SendableRecordBatchStream, Statistics, }; @@ -184,65 +185,6 @@ pub struct SymmetricHashJoinExec { mode: StreamJoinPartitionMode, } -#[derive(Debug)] -pub struct StreamJoinSideMetrics { - /// Number of batches consumed by this operator - pub(crate) input_batches: metrics::Count, - /// Number of rows consumed by this operator - pub(crate) input_rows: metrics::Count, -} - -/// Metrics for HashJoinExec -#[derive(Debug)] -pub struct StreamJoinMetrics { - /// Number of left batches/rows consumed by this operator - pub(crate) left: StreamJoinSideMetrics, - /// Number of right batches/rows consumed by this operator - pub(crate) right: StreamJoinSideMetrics, - /// Memory used by sides in bytes - pub(crate) stream_memory_usage: metrics::Gauge, - /// Number of batches produced by this operator - pub(crate) output_batches: metrics::Count, - /// Number of rows produced by this operator - pub(crate) output_rows: metrics::Count, -} - -impl StreamJoinMetrics { - pub fn new(partition: usize, metrics: &ExecutionPlanMetricsSet) -> Self { - let input_batches = - MetricBuilder::new(metrics).counter("input_batches", partition); - let input_rows = MetricBuilder::new(metrics).counter("input_rows", partition); - let left = StreamJoinSideMetrics { - input_batches, - input_rows, - }; - - let input_batches = - MetricBuilder::new(metrics).counter("input_batches", partition); - let input_rows = MetricBuilder::new(metrics).counter("input_rows", partition); - let right = StreamJoinSideMetrics { - input_batches, - input_rows, - }; - - let stream_memory_usage = - MetricBuilder::new(metrics).gauge("stream_memory_usage", partition); - - let output_batches = - MetricBuilder::new(metrics).counter("output_batches", partition); - - let output_rows = MetricBuilder::new(metrics).output_rows(partition); - - Self { - left, - right, - output_batches, - stream_memory_usage, - output_rows, - } - } -} - impl SymmetricHashJoinExec { /// Tries to create a new [SymmetricHashJoinExec]. /// # Error From b7fde3ce7040c0569295c8b90d5d4f267296878e Mon Sep 17 00:00:00 2001 From: Trent Hauck Date: Fri, 15 Dec 2023 11:14:43 -0800 Subject: [PATCH 444/572] docs: update udf docs for udtf (#8546) * docs: update udf docs for udtf * docs: update header * style: run prettier * fix: fix stale comment * docs: expand on use cases --- datafusion-examples/examples/simple_udtf.rs | 1 + docs/source/library-user-guide/adding-udfs.md | 110 +++++++++++++++++- 2 files changed, 105 insertions(+), 6 deletions(-) diff --git a/datafusion-examples/examples/simple_udtf.rs b/datafusion-examples/examples/simple_udtf.rs index e120c5e7bf8e..f1d763ba6e41 100644 --- a/datafusion-examples/examples/simple_udtf.rs +++ b/datafusion-examples/examples/simple_udtf.rs @@ -125,6 +125,7 @@ impl TableProvider for LocalCsvTable { )?)) } } + struct LocalCsvTableFunc {} impl TableFunctionImpl for LocalCsvTableFunc { diff --git a/docs/source/library-user-guide/adding-udfs.md b/docs/source/library-user-guide/adding-udfs.md index 1e710bc321a2..11cf52eb3fcf 100644 --- a/docs/source/library-user-guide/adding-udfs.md +++ b/docs/source/library-user-guide/adding-udfs.md @@ -17,17 +17,18 @@ under the License. --> -# Adding User Defined Functions: Scalar/Window/Aggregate +# Adding User Defined Functions: Scalar/Window/Aggregate/Table Functions User Defined Functions (UDFs) are functions that can be used in the context of DataFusion execution. This page covers how to add UDFs to DataFusion. In particular, it covers how to add Scalar, Window, and Aggregate UDFs. -| UDF Type | Description | Example | -| --------- | ---------------------------------------------------------------------------------------------------------- | ------------------------------------------------------------------------------------------------------------------ | -| Scalar | A function that takes a row of data and returns a single value. | [simple_udf.rs](https://github.com/apache/arrow-datafusion/blob/main/datafusion-examples/examples/simple_udf.rs) | -| Window | A function that takes a row of data and returns a single value, but also has access to the rows around it. | [simple_udwf.rs](https://github.com/apache/arrow-datafusion/blob/main/datafusion-examples/examples/simple_udwf.rs) | -| Aggregate | A function that takes a group of rows and returns a single value. | [simple_udaf.rs](https://github.com/apache/arrow-datafusion/blob/main/datafusion-examples/examples/simple_udaf.rs) | +| UDF Type | Description | Example | +| --------- | ---------------------------------------------------------------------------------------------------------- | ------------------- | +| Scalar | A function that takes a row of data and returns a single value. | [simple_udf.rs][1] | +| Window | A function that takes a row of data and returns a single value, but also has access to the rows around it. | [simple_udwf.rs][2] | +| Aggregate | A function that takes a group of rows and returns a single value. | [simple_udaf.rs][3] | +| Table | A function that takes parameters and returns a `TableProvider` to be used in an query plan. | [simple_udtf.rs][4] | First we'll talk about adding an Scalar UDF end-to-end, then we'll talk about the differences between the different types of UDFs. @@ -432,3 +433,100 @@ Then, we can query like below: ```rust let df = ctx.sql("SELECT geo_mean(a) FROM t").await?; ``` + +## Adding a User-Defined Table Function + +A User-Defined Table Function (UDTF) is a function that takes parameters and returns a `TableProvider`. + +Because we're returning a `TableProvider`, in this example we'll use the `MemTable` data source to represent a table. This is a simple struct that holds a set of RecordBatches in memory and treats them as a table. In your case, this would be replaced with your own struct that implements `TableProvider`. + +While this is a simple example for illustrative purposes, UDTFs have a lot of potential use cases. And can be particularly useful for reading data from external sources and interactive analysis. For example, see the [example][4] for a working example that reads from a CSV file. As another example, you could use the built-in UDTF `parquet_metadata` in the CLI to read the metadata from a Parquet file. + +```console +❯ select filename, row_group_id, row_group_num_rows, row_group_bytes, stats_min, stats_max from parquet_metadata('./benchmarks/data/hits.parquet') where column_id = 17 limit 10; ++--------------------------------+--------------+--------------------+-----------------+-----------+-----------+ +| filename | row_group_id | row_group_num_rows | row_group_bytes | stats_min | stats_max | ++--------------------------------+--------------+--------------------+-----------------+-----------+-----------+ +| ./benchmarks/data/hits.parquet | 0 | 450560 | 188921521 | 0 | 73256 | +| ./benchmarks/data/hits.parquet | 1 | 612174 | 210338885 | 0 | 109827 | +| ./benchmarks/data/hits.parquet | 2 | 344064 | 161242466 | 0 | 122484 | +| ./benchmarks/data/hits.parquet | 3 | 606208 | 235549898 | 0 | 121073 | +| ./benchmarks/data/hits.parquet | 4 | 335872 | 137103898 | 0 | 108996 | +| ./benchmarks/data/hits.parquet | 5 | 311296 | 145453612 | 0 | 108996 | +| ./benchmarks/data/hits.parquet | 6 | 303104 | 138833963 | 0 | 108996 | +| ./benchmarks/data/hits.parquet | 7 | 303104 | 191140113 | 0 | 73256 | +| ./benchmarks/data/hits.parquet | 8 | 573440 | 208038598 | 0 | 95823 | +| ./benchmarks/data/hits.parquet | 9 | 344064 | 147838157 | 0 | 73256 | ++--------------------------------+--------------+--------------------+-----------------+-----------+-----------+ +``` + +### Writing the UDTF + +The simple UDTF used here takes a single `Int64` argument and returns a table with a single column with the value of the argument. To create a function in DataFusion, you need to implement the `TableFunctionImpl` trait. This trait has a single method, `call`, that takes a slice of `Expr`s and returns a `Result>`. + +In the `call` method, you parse the input `Expr`s and return a `TableProvider`. You might also want to do some validation of the input `Expr`s, e.g. checking that the number of arguments is correct. + +```rust +use datafusion::common::plan_err; +use datafusion::datasource::function::TableFunctionImpl; +// Other imports here + +/// A table function that returns a table provider with the value as a single column +#[derive(Default)] +pub struct EchoFunction {} + +impl TableFunctionImpl for EchoFunction { + fn call(&self, exprs: &[Expr]) -> Result> { + let Some(Expr::Literal(ScalarValue::Int64(Some(value)))) = exprs.get(0) else { + return plan_err!("First argument must be an integer"); + }; + + // Create the schema for the table + let schema = Arc::new(Schema::new(vec![Field::new("a", DataType::Int64, false)])); + + // Create a single RecordBatch with the value as a single column + let batch = RecordBatch::try_new( + schema.clone(), + vec![Arc::new(Int64Array::from(vec![*value]))], + )?; + + // Create a MemTable plan that returns the RecordBatch + let provider = MemTable::try_new(schema, vec![vec![batch]])?; + + Ok(Arc::new(provider)) + } +} +``` + +### Registering and Using the UDTF + +With the UDTF implemented, you can register it with the `SessionContext`: + +```rust +use datafusion::execution::context::SessionContext; + +let ctx = SessionContext::new(); + +ctx.register_udtf("echo", Arc::new(EchoFunction::default())); +``` + +And if all goes well, you can use it in your query: + +```rust +use datafusion::arrow::util::pretty; + +let df = ctx.sql("SELECT * FROM echo(1)").await?; + +let results = df.collect().await?; +pretty::print_batches(&results)?; +// +---+ +// | a | +// +---+ +// | 1 | +// +---+ +``` + +[1]: https://github.com/apache/arrow-datafusion/blob/main/datafusion-examples/examples/simple_udf.rs +[2]: https://github.com/apache/arrow-datafusion/blob/main/datafusion-examples/examples/simple_udwf.rs +[3]: https://github.com/apache/arrow-datafusion/blob/main/datafusion-examples/examples/simple_udaf.rs +[4]: https://github.com/apache/arrow-datafusion/blob/main/datafusion-examples/examples/simple_udtf.rs From b71bec0fd7d17eeab5e8002842322082cd187a25 Mon Sep 17 00:00:00 2001 From: Ruihang Xia Date: Sat, 16 Dec 2023 03:18:08 +0800 Subject: [PATCH 445/572] feat: implement Unary Expr in substrait (#8534) Signed-off-by: Ruihang Xia --- .../substrait/src/logical_plan/consumer.rs | 74 ++++----- .../substrait/src/logical_plan/producer.rs | 141 ++++++++++++------ .../tests/cases/roundtrip_logical_plan.rs | 40 +++++ 3 files changed, 169 insertions(+), 86 deletions(-) diff --git a/datafusion/substrait/src/logical_plan/consumer.rs b/datafusion/substrait/src/logical_plan/consumer.rs index f6b556fc6448..f64dc764a7ed 100644 --- a/datafusion/substrait/src/logical_plan/consumer.rs +++ b/datafusion/substrait/src/logical_plan/consumer.rs @@ -1253,7 +1253,9 @@ struct BuiltinExprBuilder { impl BuiltinExprBuilder { pub fn try_from_name(name: &str) -> Option { match name { - "not" | "like" | "ilike" | "is_null" | "is_not_null" => Some(Self { + "not" | "like" | "ilike" | "is_null" | "is_not_null" | "is_true" + | "is_false" | "is_not_true" | "is_not_false" | "is_unknown" + | "is_not_unknown" | "negative" => Some(Self { expr_name: name.to_string(), }), _ => None, @@ -1267,14 +1269,11 @@ impl BuiltinExprBuilder { extensions: &HashMap, ) -> Result> { match self.expr_name.as_str() { - "not" => Self::build_not_expr(f, input_schema, extensions).await, "like" => Self::build_like_expr(false, f, input_schema, extensions).await, "ilike" => Self::build_like_expr(true, f, input_schema, extensions).await, - "is_null" => { - Self::build_is_null_expr(false, f, input_schema, extensions).await - } - "is_not_null" => { - Self::build_is_null_expr(true, f, input_schema, extensions).await + "not" | "negative" | "is_null" | "is_not_null" | "is_true" | "is_false" + | "is_not_true" | "is_not_false" | "is_unknown" | "is_not_unknown" => { + Self::build_unary_expr(&self.expr_name, f, input_schema, extensions).await } _ => { not_impl_err!("Unsupported builtin expression: {}", self.expr_name) @@ -1282,22 +1281,39 @@ impl BuiltinExprBuilder { } } - async fn build_not_expr( + async fn build_unary_expr( + fn_name: &str, f: &ScalarFunction, input_schema: &DFSchema, extensions: &HashMap, ) -> Result> { if f.arguments.len() != 1 { - return not_impl_err!("Expect one argument for `NOT` expr"); + return substrait_err!("Expect one argument for {fn_name} expr"); } let Some(ArgType::Value(expr_substrait)) = &f.arguments[0].arg_type else { - return not_impl_err!("Invalid arguments type for `NOT` expr"); + return substrait_err!("Invalid arguments type for {fn_name} expr"); }; - let expr = from_substrait_rex(expr_substrait, input_schema, extensions) + let arg = from_substrait_rex(expr_substrait, input_schema, extensions) .await? .as_ref() .clone(); - Ok(Arc::new(Expr::Not(Box::new(expr)))) + let arg = Box::new(arg); + + let expr = match fn_name { + "not" => Expr::Not(arg), + "negative" => Expr::Negative(arg), + "is_null" => Expr::IsNull(arg), + "is_not_null" => Expr::IsNotNull(arg), + "is_true" => Expr::IsTrue(arg), + "is_false" => Expr::IsFalse(arg), + "is_not_true" => Expr::IsNotTrue(arg), + "is_not_false" => Expr::IsNotFalse(arg), + "is_unknown" => Expr::IsUnknown(arg), + "is_not_unknown" => Expr::IsNotUnknown(arg), + _ => return not_impl_err!("Unsupported builtin expression: {}", fn_name), + }; + + Ok(Arc::new(expr)) } async fn build_like_expr( @@ -1308,25 +1324,25 @@ impl BuiltinExprBuilder { ) -> Result> { let fn_name = if case_insensitive { "ILIKE" } else { "LIKE" }; if f.arguments.len() != 3 { - return not_impl_err!("Expect three arguments for `{fn_name}` expr"); + return substrait_err!("Expect three arguments for `{fn_name}` expr"); } let Some(ArgType::Value(expr_substrait)) = &f.arguments[0].arg_type else { - return not_impl_err!("Invalid arguments type for `{fn_name}` expr"); + return substrait_err!("Invalid arguments type for `{fn_name}` expr"); }; let expr = from_substrait_rex(expr_substrait, input_schema, extensions) .await? .as_ref() .clone(); let Some(ArgType::Value(pattern_substrait)) = &f.arguments[1].arg_type else { - return not_impl_err!("Invalid arguments type for `{fn_name}` expr"); + return substrait_err!("Invalid arguments type for `{fn_name}` expr"); }; let pattern = from_substrait_rex(pattern_substrait, input_schema, extensions) .await? .as_ref() .clone(); let Some(ArgType::Value(escape_char_substrait)) = &f.arguments[2].arg_type else { - return not_impl_err!("Invalid arguments type for `{fn_name}` expr"); + return substrait_err!("Invalid arguments type for `{fn_name}` expr"); }; let escape_char_expr = from_substrait_rex(escape_char_substrait, input_schema, extensions) @@ -1347,30 +1363,4 @@ impl BuiltinExprBuilder { case_insensitive, }))) } - - async fn build_is_null_expr( - is_not: bool, - f: &ScalarFunction, - input_schema: &DFSchema, - extensions: &HashMap, - ) -> Result> { - let fn_name = if is_not { "IS NOT NULL" } else { "IS NULL" }; - let arg = f.arguments.first().ok_or_else(|| { - substrait_datafusion_err!("expect one argument for `{fn_name}` expr") - })?; - match &arg.arg_type { - Some(ArgType::Value(e)) => { - let expr = from_substrait_rex(e, input_schema, extensions) - .await? - .as_ref() - .clone(); - if is_not { - Ok(Arc::new(Expr::IsNotNull(Box::new(expr)))) - } else { - Ok(Arc::new(Expr::IsNull(Box::new(expr)))) - } - } - _ => substrait_err!("Invalid arguments for `{fn_name}` expression"), - } - } } diff --git a/datafusion/substrait/src/logical_plan/producer.rs b/datafusion/substrait/src/logical_plan/producer.rs index c5f1278be6e0..81498964eb61 100644 --- a/datafusion/substrait/src/logical_plan/producer.rs +++ b/datafusion/substrait/src/logical_plan/producer.rs @@ -1083,50 +1083,76 @@ pub fn to_substrait_rex( col_ref_offset, extension_info, ), - Expr::IsNull(arg) => { - let arguments: Vec = vec![FunctionArgument { - arg_type: Some(ArgType::Value(to_substrait_rex( - arg, - schema, - col_ref_offset, - extension_info, - )?)), - }]; - - let function_name = "is_null".to_string(); - let function_anchor = _register_function(function_name, extension_info); - Ok(Expression { - rex_type: Some(RexType::ScalarFunction(ScalarFunction { - function_reference: function_anchor, - arguments, - output_type: None, - args: vec![], - options: vec![], - })), - }) - } - Expr::IsNotNull(arg) => { - let arguments: Vec = vec![FunctionArgument { - arg_type: Some(ArgType::Value(to_substrait_rex( - arg, - schema, - col_ref_offset, - extension_info, - )?)), - }]; - - let function_name = "is_not_null".to_string(); - let function_anchor = _register_function(function_name, extension_info); - Ok(Expression { - rex_type: Some(RexType::ScalarFunction(ScalarFunction { - function_reference: function_anchor, - arguments, - output_type: None, - args: vec![], - options: vec![], - })), - }) - } + Expr::Not(arg) => to_substrait_unary_scalar_fn( + "not", + arg, + schema, + col_ref_offset, + extension_info, + ), + Expr::IsNull(arg) => to_substrait_unary_scalar_fn( + "is_null", + arg, + schema, + col_ref_offset, + extension_info, + ), + Expr::IsNotNull(arg) => to_substrait_unary_scalar_fn( + "is_not_null", + arg, + schema, + col_ref_offset, + extension_info, + ), + Expr::IsTrue(arg) => to_substrait_unary_scalar_fn( + "is_true", + arg, + schema, + col_ref_offset, + extension_info, + ), + Expr::IsFalse(arg) => to_substrait_unary_scalar_fn( + "is_false", + arg, + schema, + col_ref_offset, + extension_info, + ), + Expr::IsUnknown(arg) => to_substrait_unary_scalar_fn( + "is_unknown", + arg, + schema, + col_ref_offset, + extension_info, + ), + Expr::IsNotTrue(arg) => to_substrait_unary_scalar_fn( + "is_not_true", + arg, + schema, + col_ref_offset, + extension_info, + ), + Expr::IsNotFalse(arg) => to_substrait_unary_scalar_fn( + "is_not_false", + arg, + schema, + col_ref_offset, + extension_info, + ), + Expr::IsNotUnknown(arg) => to_substrait_unary_scalar_fn( + "is_not_unknown", + arg, + schema, + col_ref_offset, + extension_info, + ), + Expr::Negative(arg) => to_substrait_unary_scalar_fn( + "negative", + arg, + schema, + col_ref_offset, + extension_info, + ), _ => { not_impl_err!("Unsupported expression: {expr:?}") } @@ -1591,6 +1617,33 @@ fn to_substrait_literal(value: &ScalarValue) -> Result { }) } +/// Util to generate substrait [RexType::ScalarFunction] with one argument +fn to_substrait_unary_scalar_fn( + fn_name: &str, + arg: &Expr, + schema: &DFSchemaRef, + col_ref_offset: usize, + extension_info: &mut ( + Vec, + HashMap, + ), +) -> Result { + let function_anchor = _register_function(fn_name.to_string(), extension_info); + let substrait_expr = to_substrait_rex(arg, schema, col_ref_offset, extension_info)?; + + Ok(Expression { + rex_type: Some(RexType::ScalarFunction(ScalarFunction { + function_reference: function_anchor, + arguments: vec![FunctionArgument { + arg_type: Some(ArgType::Value(substrait_expr)), + }], + output_type: None, + options: vec![], + ..Default::default() + })), + }) +} + fn try_to_substrait_null(v: &ScalarValue) -> Result { let default_nullability = r#type::Nullability::Nullable as i32; match v { diff --git a/datafusion/substrait/tests/cases/roundtrip_logical_plan.rs b/datafusion/substrait/tests/cases/roundtrip_logical_plan.rs index 691fba864449..91d5a9469627 100644 --- a/datafusion/substrait/tests/cases/roundtrip_logical_plan.rs +++ b/datafusion/substrait/tests/cases/roundtrip_logical_plan.rs @@ -483,6 +483,46 @@ async fn roundtrip_ilike() -> Result<()> { roundtrip("SELECT f FROM data WHERE f ILIKE 'a%b'").await } +#[tokio::test] +async fn roundtrip_not() -> Result<()> { + roundtrip("SELECT * FROM data WHERE NOT d").await +} + +#[tokio::test] +async fn roundtrip_negative() -> Result<()> { + roundtrip("SELECT * FROM data WHERE -a = 1").await +} + +#[tokio::test] +async fn roundtrip_is_true() -> Result<()> { + roundtrip("SELECT * FROM data WHERE d IS TRUE").await +} + +#[tokio::test] +async fn roundtrip_is_false() -> Result<()> { + roundtrip("SELECT * FROM data WHERE d IS FALSE").await +} + +#[tokio::test] +async fn roundtrip_is_not_true() -> Result<()> { + roundtrip("SELECT * FROM data WHERE d IS NOT TRUE").await +} + +#[tokio::test] +async fn roundtrip_is_not_false() -> Result<()> { + roundtrip("SELECT * FROM data WHERE d IS NOT FALSE").await +} + +#[tokio::test] +async fn roundtrip_is_unknown() -> Result<()> { + roundtrip("SELECT * FROM data WHERE d IS UNKNOWN").await +} + +#[tokio::test] +async fn roundtrip_is_not_unknown() -> Result<()> { + roundtrip("SELECT * FROM data WHERE d IS NOT UNKNOWN").await +} + #[tokio::test] async fn roundtrip_union() -> Result<()> { roundtrip("SELECT a, e FROM data UNION SELECT a, e FROM data").await From 0fcd077c67b07092c94acae86ffaa97dfb54789a Mon Sep 17 00:00:00 2001 From: Asura7969 <1402357969@qq.com> Date: Sat, 16 Dec 2023 20:17:32 +0800 Subject: [PATCH 446/572] Fix `compute_record_batch_statistics` wrong with `projection` (#8489) * Minor: Improve the document format of JoinHashMap * fix `compute_record_batch_statistics` wrong with `projection` * fix test * fix test --- datafusion/physical-plan/src/common.rs | 38 +++++++++++------ .../sqllogictest/test_files/groupby.slt | 21 +++++----- datafusion/sqllogictest/test_files/joins.slt | 42 +++++++++---------- 3 files changed, 57 insertions(+), 44 deletions(-) diff --git a/datafusion/physical-plan/src/common.rs b/datafusion/physical-plan/src/common.rs index 649f3a31aa7e..e83dc2525b9f 100644 --- a/datafusion/physical-plan/src/common.rs +++ b/datafusion/physical-plan/src/common.rs @@ -30,6 +30,7 @@ use crate::{ColumnStatistics, ExecutionPlan, Statistics}; use arrow::datatypes::Schema; use arrow::ipc::writer::{FileWriter, IpcWriteOptions}; use arrow::record_batch::RecordBatch; +use arrow_array::Array; use datafusion_common::stats::Precision; use datafusion_common::{plan_err, DataFusionError, Result}; use datafusion_execution::memory_pool::MemoryReservation; @@ -139,17 +140,22 @@ pub fn compute_record_batch_statistics( ) -> Statistics { let nb_rows = batches.iter().flatten().map(RecordBatch::num_rows).sum(); - let total_byte_size = batches - .iter() - .flatten() - .map(|b| b.get_array_memory_size()) - .sum(); - let projection = match projection { Some(p) => p, None => (0..schema.fields().len()).collect(), }; + let total_byte_size = batches + .iter() + .flatten() + .map(|b| { + projection + .iter() + .map(|index| b.column(*index).get_array_memory_size()) + .sum::() + }) + .sum(); + let mut column_statistics = vec![ColumnStatistics::new_unknown(); projection.len()]; for partition in batches.iter() { @@ -388,6 +394,7 @@ mod tests { datatypes::{DataType, Field, Schema}, record_batch::RecordBatch, }; + use arrow_array::UInt64Array; use datafusion_expr::Operator; use datafusion_physical_expr::expressions::{col, Column}; @@ -685,20 +692,30 @@ mod tests { let schema = Arc::new(Schema::new(vec![ Field::new("f32", DataType::Float32, false), Field::new("f64", DataType::Float64, false), + Field::new("u64", DataType::UInt64, false), ])); let batch = RecordBatch::try_new( Arc::clone(&schema), vec![ Arc::new(Float32Array::from(vec![1., 2., 3.])), Arc::new(Float64Array::from(vec![9., 8., 7.])), + Arc::new(UInt64Array::from(vec![4, 5, 6])), ], )?; + + // just select f32,f64 + let select_projection = Some(vec![0, 1]); + let byte_size = batch + .project(&select_projection.clone().unwrap()) + .unwrap() + .get_array_memory_size(); + let actual = - compute_record_batch_statistics(&[vec![batch]], &schema, Some(vec![0, 1])); + compute_record_batch_statistics(&[vec![batch]], &schema, select_projection); - let mut expected = Statistics { + let expected = Statistics { num_rows: Precision::Exact(3), - total_byte_size: Precision::Exact(464), // this might change a bit if the way we compute the size changes + total_byte_size: Precision::Exact(byte_size), column_statistics: vec![ ColumnStatistics { distinct_count: Precision::Absent, @@ -715,9 +732,6 @@ mod tests { ], }; - // Prevent test flakiness due to undefined / changing implementation details - expected.total_byte_size = actual.total_byte_size.clone(); - assert_eq!(actual, expected); Ok(()) } diff --git a/datafusion/sqllogictest/test_files/groupby.slt b/datafusion/sqllogictest/test_files/groupby.slt index b915c439059b..44d30ba0b34c 100644 --- a/datafusion/sqllogictest/test_files/groupby.slt +++ b/datafusion/sqllogictest/test_files/groupby.slt @@ -2021,14 +2021,15 @@ SortPreservingMergeExec: [col0@0 ASC NULLS LAST] ----------RepartitionExec: partitioning=Hash([col0@0, col1@1, col2@2], 4), input_partitions=4 ------------AggregateExec: mode=Partial, gby=[col0@0 as col0, col1@1 as col1, col2@2 as col2], aggr=[LAST_VALUE(r.col1)], ordering_mode=PartiallySorted([0]) --------------SortExec: expr=[col0@3 ASC NULLS LAST] -----------------CoalesceBatchesExec: target_batch_size=8192 -------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(col0@0, col0@0)] ---------------------CoalesceBatchesExec: target_batch_size=8192 -----------------------RepartitionExec: partitioning=Hash([col0@0], 4), input_partitions=1 -------------------------MemoryExec: partitions=1, partition_sizes=[3] ---------------------CoalesceBatchesExec: target_batch_size=8192 -----------------------RepartitionExec: partitioning=Hash([col0@0], 4), input_partitions=1 -------------------------MemoryExec: partitions=1, partition_sizes=[3] +----------------ProjectionExec: expr=[col0@2 as col0, col1@3 as col1, col2@4 as col2, col0@0 as col0, col1@1 as col1] +------------------CoalesceBatchesExec: target_batch_size=8192 +--------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(col0@0, col0@0)] +----------------------CoalesceBatchesExec: target_batch_size=8192 +------------------------RepartitionExec: partitioning=Hash([col0@0], 4), input_partitions=1 +--------------------------MemoryExec: partitions=1, partition_sizes=[3] +----------------------CoalesceBatchesExec: target_batch_size=8192 +------------------------RepartitionExec: partitioning=Hash([col0@0], 4), input_partitions=1 +--------------------------MemoryExec: partitions=1, partition_sizes=[3] # Columns in the table are a,b,c,d. Source is CsvExec which is ordered by # a,b,c column. Column a has cardinality 2, column b has cardinality 4. @@ -2709,9 +2710,9 @@ SortExec: expr=[sn@2 ASC NULLS LAST] --ProjectionExec: expr=[zip_code@1 as zip_code, country@2 as country, sn@0 as sn, ts@3 as ts, currency@4 as currency, LAST_VALUE(e.amount) ORDER BY [e.sn ASC NULLS LAST]@5 as last_rate] ----AggregateExec: mode=Single, gby=[sn@2 as sn, zip_code@0 as zip_code, country@1 as country, ts@3 as ts, currency@4 as currency], aggr=[LAST_VALUE(e.amount)] ------SortExec: expr=[sn@5 ASC NULLS LAST] ---------ProjectionExec: expr=[zip_code@0 as zip_code, country@1 as country, sn@2 as sn, ts@3 as ts, currency@4 as currency, sn@5 as sn, amount@8 as amount] +--------ProjectionExec: expr=[zip_code@4 as zip_code, country@5 as country, sn@6 as sn, ts@7 as ts, currency@8 as currency, sn@0 as sn, amount@3 as amount] ----------CoalesceBatchesExec: target_batch_size=8192 -------------HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(currency@4, currency@2)], filter=ts@0 >= ts@1 +------------HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(currency@2, currency@4)], filter=ts@0 >= ts@1 --------------MemoryExec: partitions=1, partition_sizes=[1] --------------MemoryExec: partitions=1, partition_sizes=[1] diff --git a/datafusion/sqllogictest/test_files/joins.slt b/datafusion/sqllogictest/test_files/joins.slt index 67e3750113da..1ad17fbb8c91 100644 --- a/datafusion/sqllogictest/test_files/joins.slt +++ b/datafusion/sqllogictest/test_files/joins.slt @@ -1569,15 +1569,13 @@ Projection: join_t1.t1_id, join_t2.t2_id, join_t1.t1_name ----TableScan: join_t1 projection=[t1_id, t1_name] ----TableScan: join_t2 projection=[t2_id] physical_plan -ProjectionExec: expr=[t1_id@0 as t1_id, t2_id@3 as t2_id, t1_name@1 as t1_name] +ProjectionExec: expr=[t1_id@1 as t1_id, t2_id@0 as t2_id, t1_name@2 as t1_name] --CoalesceBatchesExec: target_batch_size=2 -----HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(join_t1.t1_id + UInt32(11)@2, t2_id@0)] -------CoalescePartitionsExec ---------ProjectionExec: expr=[t1_id@0 as t1_id, t1_name@1 as t1_name, t1_id@0 + 11 as join_t1.t1_id + UInt32(11)] -----------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -------------MemoryExec: partitions=1, partition_sizes=[1] -------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 ---------MemoryExec: partitions=1, partition_sizes=[1] +----HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(t2_id@0, join_t1.t1_id + UInt32(11)@2)] +------MemoryExec: partitions=1, partition_sizes=[1] +------ProjectionExec: expr=[t1_id@0 as t1_id, t1_name@1 as t1_name, t1_id@0 + 11 as join_t1.t1_id + UInt32(11)] +--------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +----------MemoryExec: partitions=1, partition_sizes=[1] statement ok set datafusion.optimizer.repartition_joins = true; @@ -1595,18 +1593,18 @@ Projection: join_t1.t1_id, join_t2.t2_id, join_t1.t1_name ----TableScan: join_t1 projection=[t1_id, t1_name] ----TableScan: join_t2 projection=[t2_id] physical_plan -ProjectionExec: expr=[t1_id@0 as t1_id, t2_id@3 as t2_id, t1_name@1 as t1_name] +ProjectionExec: expr=[t1_id@1 as t1_id, t2_id@0 as t2_id, t1_name@2 as t1_name] --CoalesceBatchesExec: target_batch_size=2 -----HashJoinExec: mode=Partitioned, join_type=Inner, on=[(join_t1.t1_id + UInt32(11)@2, t2_id@0)] +----HashJoinExec: mode=Partitioned, join_type=Inner, on=[(t2_id@0, join_t1.t1_id + UInt32(11)@2)] +------CoalesceBatchesExec: target_batch_size=2 +--------RepartitionExec: partitioning=Hash([t2_id@0], 2), input_partitions=2 +----------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +------------MemoryExec: partitions=1, partition_sizes=[1] ------CoalesceBatchesExec: target_batch_size=2 --------RepartitionExec: partitioning=Hash([join_t1.t1_id + UInt32(11)@2], 2), input_partitions=2 ----------ProjectionExec: expr=[t1_id@0 as t1_id, t1_name@1 as t1_name, t1_id@0 + 11 as join_t1.t1_id + UInt32(11)] ------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 --------------MemoryExec: partitions=1, partition_sizes=[1] -------CoalesceBatchesExec: target_batch_size=2 ---------RepartitionExec: partitioning=Hash([t2_id@0], 2), input_partitions=2 -----------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -------------MemoryExec: partitions=1, partition_sizes=[1] # Right side expr key inner join @@ -2821,13 +2819,13 @@ physical_plan SortPreservingMergeExec: [t1_id@0 ASC NULLS LAST] --SortExec: expr=[t1_id@0 ASC NULLS LAST] ----CoalesceBatchesExec: target_batch_size=2 -------HashJoinExec: mode=Partitioned, join_type=LeftSemi, on=[(t1_id@0, t2_id@0)] +------HashJoinExec: mode=Partitioned, join_type=RightSemi, on=[(t2_id@0, t1_id@0)] --------CoalesceBatchesExec: target_batch_size=2 -----------RepartitionExec: partitioning=Hash([t1_id@0], 2), input_partitions=2 +----------RepartitionExec: partitioning=Hash([t2_id@0], 2), input_partitions=2 ------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 --------------MemoryExec: partitions=1, partition_sizes=[1] --------CoalesceBatchesExec: target_batch_size=2 -----------RepartitionExec: partitioning=Hash([t2_id@0], 2), input_partitions=2 +----------RepartitionExec: partitioning=Hash([t1_id@0], 2), input_partitions=2 ------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 --------------MemoryExec: partitions=1, partition_sizes=[1] @@ -2862,13 +2860,13 @@ physical_plan SortPreservingMergeExec: [t1_id@0 ASC NULLS LAST] --SortExec: expr=[t1_id@0 ASC NULLS LAST] ----CoalesceBatchesExec: target_batch_size=2 -------HashJoinExec: mode=Partitioned, join_type=LeftSemi, on=[(t1_id@0, t2_id@0)] +------HashJoinExec: mode=Partitioned, join_type=RightSemi, on=[(t2_id@0, t1_id@0)] --------CoalesceBatchesExec: target_batch_size=2 -----------RepartitionExec: partitioning=Hash([t1_id@0], 2), input_partitions=2 +----------RepartitionExec: partitioning=Hash([t2_id@0], 2), input_partitions=2 ------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 --------------MemoryExec: partitions=1, partition_sizes=[1] --------CoalesceBatchesExec: target_batch_size=2 -----------RepartitionExec: partitioning=Hash([t2_id@0], 2), input_partitions=2 +----------RepartitionExec: partitioning=Hash([t1_id@0], 2), input_partitions=2 ------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 --------------MemoryExec: partitions=1, partition_sizes=[1] @@ -2924,7 +2922,7 @@ physical_plan SortPreservingMergeExec: [t1_id@0 ASC NULLS LAST] --SortExec: expr=[t1_id@0 ASC NULLS LAST] ----CoalesceBatchesExec: target_batch_size=2 -------HashJoinExec: mode=CollectLeft, join_type=LeftSemi, on=[(t1_id@0, t2_id@0)] +------HashJoinExec: mode=CollectLeft, join_type=RightSemi, on=[(t2_id@0, t1_id@0)] --------MemoryExec: partitions=1, partition_sizes=[1] --------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 ----------MemoryExec: partitions=1, partition_sizes=[1] @@ -2960,7 +2958,7 @@ physical_plan SortPreservingMergeExec: [t1_id@0 ASC NULLS LAST] --SortExec: expr=[t1_id@0 ASC NULLS LAST] ----CoalesceBatchesExec: target_batch_size=2 -------HashJoinExec: mode=CollectLeft, join_type=LeftSemi, on=[(t1_id@0, t2_id@0)] +------HashJoinExec: mode=CollectLeft, join_type=RightSemi, on=[(t2_id@0, t1_id@0)] --------MemoryExec: partitions=1, partition_sizes=[1] --------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 ----------MemoryExec: partitions=1, partition_sizes=[1] From 1f4c14c7b942de81c518b31be9a16dfb07e5237e Mon Sep 17 00:00:00 2001 From: Jay Zhan Date: Sun, 17 Dec 2023 19:39:27 +0800 Subject: [PATCH 447/572] cleanup parquet flag (#8563) Signed-off-by: jayzhan211 --- datafusion/common/src/file_options/file_type.rs | 2 +- datafusion/common/src/file_options/mod.rs | 7 +------ 2 files changed, 2 insertions(+), 7 deletions(-) diff --git a/datafusion/common/src/file_options/file_type.rs b/datafusion/common/src/file_options/file_type.rs index b1d61b1a2567..97362bdad3cc 100644 --- a/datafusion/common/src/file_options/file_type.rs +++ b/datafusion/common/src/file_options/file_type.rs @@ -103,13 +103,13 @@ impl FromStr for FileType { } #[cfg(test)] +#[cfg(feature = "parquet")] mod tests { use crate::error::DataFusionError; use crate::file_options::FileType; use std::str::FromStr; #[test] - #[cfg(feature = "parquet")] fn from_str() { for (ext, file_type) in [ ("csv", FileType::CSV), diff --git a/datafusion/common/src/file_options/mod.rs b/datafusion/common/src/file_options/mod.rs index f0e49dd85597..1d661b17eb1c 100644 --- a/datafusion/common/src/file_options/mod.rs +++ b/datafusion/common/src/file_options/mod.rs @@ -296,10 +296,10 @@ impl Display for FileTypeWriterOptions { } #[cfg(test)] +#[cfg(feature = "parquet")] mod tests { use std::collections::HashMap; - #[cfg(feature = "parquet")] use parquet::{ basic::{Compression, Encoding, ZstdLevel}, file::properties::{EnabledStatistics, WriterVersion}, @@ -314,11 +314,9 @@ mod tests { use crate::Result; - #[cfg(feature = "parquet")] use super::{parquet_writer::ParquetWriterOptions, StatementOptions}; #[test] - #[cfg(feature = "parquet")] fn test_writeroptions_parquet_from_statement_options() -> Result<()> { let mut option_map: HashMap = HashMap::new(); option_map.insert("max_row_group_size".to_owned(), "123".to_owned()); @@ -389,7 +387,6 @@ mod tests { } #[test] - #[cfg(feature = "parquet")] fn test_writeroptions_parquet_column_specific() -> Result<()> { let mut option_map: HashMap = HashMap::new(); @@ -511,7 +508,6 @@ mod tests { #[test] // for StatementOptions - #[cfg(feature = "parquet")] fn test_writeroptions_csv_from_statement_options() -> Result<()> { let mut option_map: HashMap = HashMap::new(); option_map.insert("header".to_owned(), "true".to_owned()); @@ -540,7 +536,6 @@ mod tests { #[test] // for StatementOptions - #[cfg(feature = "parquet")] fn test_writeroptions_json_from_statement_options() -> Result<()> { let mut option_map: HashMap = HashMap::new(); option_map.insert("compression".to_owned(), "gzip".to_owned()); From b59ddf64fc77bbd37aa761c856d47ebc473ea2e2 Mon Sep 17 00:00:00 2001 From: Huaijin Date: Sun, 17 Dec 2023 19:41:06 +0800 Subject: [PATCH 448/572] Minor: move some invariants out of the loop (#8564) --- datafusion/optimizer/src/push_down_filter.rs | 19 +++++++++---------- 1 file changed, 9 insertions(+), 10 deletions(-) diff --git a/datafusion/optimizer/src/push_down_filter.rs b/datafusion/optimizer/src/push_down_filter.rs index c090fb849a82..4bea17500acc 100644 --- a/datafusion/optimizer/src/push_down_filter.rs +++ b/datafusion/optimizer/src/push_down_filter.rs @@ -559,6 +559,15 @@ fn push_down_join( let mut is_inner_join = false; let infer_predicates = if join.join_type == JoinType::Inner { is_inner_join = true; + // Only allow both side key is column. + let join_col_keys = join + .on + .iter() + .flat_map(|(l, r)| match (l.try_into_col(), r.try_into_col()) { + (Ok(l_col), Ok(r_col)) => Some((l_col, r_col)), + _ => None, + }) + .collect::>(); // TODO refine the logic, introduce EquivalenceProperties to logical plan and infer additional filters to push down // For inner joins, duplicate filters for joined columns so filters can be pushed down // to both sides. Take the following query as an example: @@ -583,16 +592,6 @@ fn push_down_join( Err(e) => return Some(Err(e)), }; - // Only allow both side key is column. - let join_col_keys = join - .on - .iter() - .flat_map(|(l, r)| match (l.try_into_col(), r.try_into_col()) { - (Ok(l_col), Ok(r_col)) => Some((l_col, r_col)), - _ => None, - }) - .collect::>(); - for col in columns.iter() { for (l, r) in join_col_keys.iter() { if col == l { From 0f83ffc448a4d7fb4297148f653e267a847d769a Mon Sep 17 00:00:00 2001 From: Ruihang Xia Date: Sun, 17 Dec 2023 19:55:29 +0800 Subject: [PATCH 449/572] feat: implement Repartition plan in substrait (#8526) * feat: implement Repartition plan in substrait Signed-off-by: Ruihang Xia * use substrait_err macro Signed-off-by: Ruihang Xia --------- Signed-off-by: Ruihang Xia --- .../substrait/src/logical_plan/consumer.rs | 96 ++++++++++++++----- .../substrait/src/logical_plan/producer.rs | 81 +++++++++++++++- .../tests/cases/roundtrip_logical_plan.rs | 36 ++++++- 3 files changed, 185 insertions(+), 28 deletions(-) diff --git a/datafusion/substrait/src/logical_plan/consumer.rs b/datafusion/substrait/src/logical_plan/consumer.rs index f64dc764a7ed..b7fee96bba1c 100644 --- a/datafusion/substrait/src/logical_plan/consumer.rs +++ b/datafusion/substrait/src/logical_plan/consumer.rs @@ -27,8 +27,8 @@ use datafusion::logical_expr::{ BuiltinScalarFunction, Case, Expr, LogicalPlan, Operator, }; use datafusion::logical_expr::{ - expr, Cast, Extension, GroupingSet, Like, LogicalPlanBuilder, WindowFrameBound, - WindowFrameUnits, + expr, Cast, Extension, GroupingSet, Like, LogicalPlanBuilder, Partitioning, + Repartition, WindowFrameBound, WindowFrameUnits, }; use datafusion::prelude::JoinType; use datafusion::sql::TableReference; @@ -38,7 +38,8 @@ use datafusion::{ prelude::{Column, SessionContext}, scalar::ScalarValue, }; -use substrait::proto::expression::{Literal, ScalarFunction}; +use substrait::proto::exchange_rel::ExchangeKind; +use substrait::proto::expression::{FieldReference, Literal, ScalarFunction}; use substrait::proto::{ aggregate_function::AggregationInvocation, expression::{ @@ -550,6 +551,45 @@ pub async fn from_substrait_rel( let plan = plan.from_template(&plan.expressions(), &inputs); Ok(LogicalPlan::Extension(Extension { node: plan })) } + Some(RelType::Exchange(exchange)) => { + let Some(input) = exchange.input.as_ref() else { + return substrait_err!("Unexpected empty input in ExchangeRel"); + }; + let input = Arc::new(from_substrait_rel(ctx, input, extensions).await?); + + let Some(exchange_kind) = &exchange.exchange_kind else { + return substrait_err!("Unexpected empty input in ExchangeRel"); + }; + + // ref: https://substrait.io/relations/physical_relations/#exchange-types + let partitioning_scheme = match exchange_kind { + ExchangeKind::ScatterByFields(scatter_fields) => { + let mut partition_columns = vec![]; + let input_schema = input.schema(); + for field_ref in &scatter_fields.fields { + let column = + from_substrait_field_reference(field_ref, input_schema)?; + partition_columns.push(column); + } + Partitioning::Hash( + partition_columns, + exchange.partition_count as usize, + ) + } + ExchangeKind::RoundRobin(_) => { + Partitioning::RoundRobinBatch(exchange.partition_count as usize) + } + ExchangeKind::SingleTarget(_) + | ExchangeKind::MultiTarget(_) + | ExchangeKind::Broadcast(_) => { + return not_impl_err!("Unsupported exchange kind: {exchange_kind:?}"); + } + }; + Ok(LogicalPlan::Repartition(Repartition { + input, + partitioning_scheme, + })) + } _ => not_impl_err!("Unsupported RelType: {:?}", rel.rel_type), } } @@ -725,27 +765,9 @@ pub async fn from_substrait_rex( negated: false, }))) } - Some(RexType::Selection(field_ref)) => match &field_ref.reference_type { - Some(DirectReference(direct)) => match &direct.reference_type.as_ref() { - Some(StructField(x)) => match &x.child.as_ref() { - Some(_) => not_impl_err!( - "Direct reference StructField with child is not supported" - ), - None => { - let column = - input_schema.field(x.field as usize).qualified_column(); - Ok(Arc::new(Expr::Column(Column { - relation: column.relation, - name: column.name, - }))) - } - }, - _ => not_impl_err!( - "Direct reference with types other than StructField is not supported" - ), - }, - _ => not_impl_err!("unsupported field ref type"), - }, + Some(RexType::Selection(field_ref)) => Ok(Arc::new( + from_substrait_field_reference(field_ref, input_schema)?, + )), Some(RexType::IfThen(if_then)) => { // Parse `ifs` // If the first element does not have a `then` part, then we can assume it's a base expression @@ -1245,6 +1267,32 @@ fn from_substrait_null(null_type: &Type) -> Result { } } +fn from_substrait_field_reference( + field_ref: &FieldReference, + input_schema: &DFSchema, +) -> Result { + match &field_ref.reference_type { + Some(DirectReference(direct)) => match &direct.reference_type.as_ref() { + Some(StructField(x)) => match &x.child.as_ref() { + Some(_) => not_impl_err!( + "Direct reference StructField with child is not supported" + ), + None => { + let column = input_schema.field(x.field as usize).qualified_column(); + Ok(Expr::Column(Column { + relation: column.relation, + name: column.name, + })) + } + }, + _ => not_impl_err!( + "Direct reference with types other than StructField is not supported" + ), + }, + _ => not_impl_err!("unsupported field ref type"), + } +} + /// Build [`Expr`] from its name and required inputs. struct BuiltinExprBuilder { expr_name: String, diff --git a/datafusion/substrait/src/logical_plan/producer.rs b/datafusion/substrait/src/logical_plan/producer.rs index 81498964eb61..50f872544298 100644 --- a/datafusion/substrait/src/logical_plan/producer.rs +++ b/datafusion/substrait/src/logical_plan/producer.rs @@ -19,7 +19,9 @@ use std::collections::HashMap; use std::ops::Deref; use std::sync::Arc; -use datafusion::logical_expr::{CrossJoin, Distinct, Like, WindowFrameUnits}; +use datafusion::logical_expr::{ + CrossJoin, Distinct, Like, Partitioning, WindowFrameUnits, +}; use datafusion::{ arrow::datatypes::{DataType, TimeUnit}, error::{DataFusionError, Result}, @@ -28,8 +30,8 @@ use datafusion::{ scalar::ScalarValue, }; -use datafusion::common::DFSchemaRef; use datafusion::common::{exec_err, internal_err, not_impl_err}; +use datafusion::common::{substrait_err, DFSchemaRef}; #[allow(unused_imports)] use datafusion::logical_expr::aggregate_function; use datafusion::logical_expr::expr::{ @@ -39,8 +41,9 @@ use datafusion::logical_expr::expr::{ use datafusion::logical_expr::{expr, Between, JoinConstraint, LogicalPlan, Operator}; use datafusion::prelude::Expr; use prost_types::Any as ProtoAny; +use substrait::proto::exchange_rel::{ExchangeKind, RoundRobin, ScatterFields}; use substrait::proto::expression::window_function::BoundsType; -use substrait::proto::CrossRel; +use substrait::proto::{CrossRel, ExchangeRel}; use substrait::{ proto::{ aggregate_function::AggregationInvocation, @@ -410,6 +413,53 @@ pub fn to_substrait_rel( rel_type: Some(RelType::Project(project_rel)), })) } + LogicalPlan::Repartition(repartition) => { + let input = + to_substrait_rel(repartition.input.as_ref(), ctx, extension_info)?; + let partition_count = match repartition.partitioning_scheme { + Partitioning::RoundRobinBatch(num) => num, + Partitioning::Hash(_, num) => num, + Partitioning::DistributeBy(_) => { + return not_impl_err!( + "Physical plan does not support DistributeBy partitioning" + ) + } + }; + // ref: https://substrait.io/relations/physical_relations/#exchange-types + let exchange_kind = match &repartition.partitioning_scheme { + Partitioning::RoundRobinBatch(_) => { + ExchangeKind::RoundRobin(RoundRobin::default()) + } + Partitioning::Hash(exprs, _) => { + let fields = exprs + .iter() + .map(|e| { + try_to_substrait_field_reference( + e, + repartition.input.schema(), + ) + }) + .collect::>>()?; + ExchangeKind::ScatterByFields(ScatterFields { fields }) + } + Partitioning::DistributeBy(_) => { + return not_impl_err!( + "Physical plan does not support DistributeBy partitioning" + ) + } + }; + let exchange_rel = ExchangeRel { + common: None, + input: Some(input), + exchange_kind: Some(exchange_kind), + advanced_extension: None, + partition_count: partition_count as i32, + targets: vec![], + }; + Ok(Box::new(Rel { + rel_type: Some(RelType::Exchange(Box::new(exchange_rel))), + })) + } LogicalPlan::Extension(extension_plan) => { let extension_bytes = ctx .state() @@ -1804,6 +1854,31 @@ fn try_to_substrait_null(v: &ScalarValue) -> Result { } } +/// Try to convert an [Expr] to a [FieldReference]. +/// Returns `Err` if the [Expr] is not a [Expr::Column]. +fn try_to_substrait_field_reference( + expr: &Expr, + schema: &DFSchemaRef, +) -> Result { + match expr { + Expr::Column(col) => { + let index = schema.index_of_column(col)?; + Ok(FieldReference { + reference_type: Some(ReferenceType::DirectReference(ReferenceSegment { + reference_type: Some(reference_segment::ReferenceType::StructField( + Box::new(reference_segment::StructField { + field: index as i32, + child: None, + }), + )), + })), + root_type: None, + }) + } + _ => substrait_err!("Expect a `Column` expr, but found {expr:?}"), + } +} + fn substrait_sort_field( expr: &Expr, schema: &DFSchemaRef, diff --git a/datafusion/substrait/tests/cases/roundtrip_logical_plan.rs b/datafusion/substrait/tests/cases/roundtrip_logical_plan.rs index 91d5a9469627..47eb5a8f73f5 100644 --- a/datafusion/substrait/tests/cases/roundtrip_logical_plan.rs +++ b/datafusion/substrait/tests/cases/roundtrip_logical_plan.rs @@ -32,7 +32,7 @@ use datafusion::execution::context::SessionState; use datafusion::execution::registry::SerializerRegistry; use datafusion::execution::runtime_env::RuntimeEnv; use datafusion::logical_expr::{ - Extension, LogicalPlan, UserDefinedLogicalNode, Volatility, + Extension, LogicalPlan, Repartition, UserDefinedLogicalNode, Volatility, }; use datafusion::optimizer::simplify_expressions::expr_simplifier::THRESHOLD_INLINE_INLIST; use datafusion::prelude::*; @@ -738,6 +738,40 @@ async fn roundtrip_aggregate_udf() -> Result<()> { roundtrip_with_ctx("select dummy_agg(a) from data", ctx).await } +#[tokio::test] +async fn roundtrip_repartition_roundrobin() -> Result<()> { + let ctx = create_context().await?; + let scan_plan = ctx.sql("SELECT * FROM data").await?.into_optimized_plan()?; + let plan = LogicalPlan::Repartition(Repartition { + input: Arc::new(scan_plan), + partitioning_scheme: Partitioning::RoundRobinBatch(8), + }); + + let proto = to_substrait_plan(&plan, &ctx)?; + let plan2 = from_substrait_plan(&ctx, &proto).await?; + let plan2 = ctx.state().optimize(&plan2)?; + + assert_eq!(format!("{plan:?}"), format!("{plan2:?}")); + Ok(()) +} + +#[tokio::test] +async fn roundtrip_repartition_hash() -> Result<()> { + let ctx = create_context().await?; + let scan_plan = ctx.sql("SELECT * FROM data").await?.into_optimized_plan()?; + let plan = LogicalPlan::Repartition(Repartition { + input: Arc::new(scan_plan), + partitioning_scheme: Partitioning::Hash(vec![col("data.a")], 8), + }); + + let proto = to_substrait_plan(&plan, &ctx)?; + let plan2 = from_substrait_plan(&ctx, &proto).await?; + let plan2 = ctx.state().optimize(&plan2)?; + + assert_eq!(format!("{plan:?}"), format!("{plan2:?}")); + Ok(()) +} + fn check_post_join_filters(rel: &Rel) -> Result<()> { // search for target_rel and field value in proto match &rel.rel_type { From 2e16c7519cb4a21d54975e56e2127039a3a6fd04 Mon Sep 17 00:00:00 2001 From: Andrew Lamb Date: Sun, 17 Dec 2023 07:12:10 -0500 Subject: [PATCH 450/572] Fix sort order aware file group parallelization (#8517) * Minor: Extract file group repartitioning and tests into `FileGroupRepartitioner` * Implement sort order aware redistribution --- datafusion/core/src/datasource/listing/mod.rs | 16 +- .../core/src/datasource/physical_plan/csv.rs | 14 +- .../datasource/physical_plan/file_groups.rs | 826 ++++++++++++++++++ .../physical_plan/file_scan_config.rs | 85 +- .../core/src/datasource/physical_plan/mod.rs | 344 +------- .../datasource/physical_plan/parquet/mod.rs | 16 +- .../enforce_distribution.rs | 61 +- .../test_files/repartition_scan.slt | 2 +- 8 files changed, 918 insertions(+), 446 deletions(-) create mode 100644 datafusion/core/src/datasource/physical_plan/file_groups.rs diff --git a/datafusion/core/src/datasource/listing/mod.rs b/datafusion/core/src/datasource/listing/mod.rs index 5e5b96f6ba8c..e7583501f9d9 100644 --- a/datafusion/core/src/datasource/listing/mod.rs +++ b/datafusion/core/src/datasource/listing/mod.rs @@ -40,7 +40,7 @@ pub type PartitionedFileStream = /// Only scan a subset of Row Groups from the Parquet file whose data "midpoint" /// lies within the [start, end) byte offsets. This option can be used to scan non-overlapping /// sections of a Parquet file in parallel. -#[derive(Debug, Clone)] +#[derive(Debug, Clone, PartialEq, Hash, Eq, PartialOrd, Ord)] pub struct FileRange { /// Range start pub start: i64, @@ -70,13 +70,12 @@ pub struct PartitionedFile { /// An optional field for user defined per object metadata pub extensions: Option>, } - impl PartitionedFile { /// Create a simple file without metadata or partition - pub fn new(path: String, size: u64) -> Self { + pub fn new(path: impl Into, size: u64) -> Self { Self { object_meta: ObjectMeta { - location: Path::from(path), + location: Path::from(path.into()), last_modified: chrono::Utc.timestamp_nanos(0), size: size as usize, e_tag: None, @@ -99,9 +98,10 @@ impl PartitionedFile { version: None, }, partition_values: vec![], - range: Some(FileRange { start, end }), + range: None, extensions: None, } + .with_range(start, end) } /// Return a file reference from the given path @@ -114,6 +114,12 @@ impl PartitionedFile { pub fn path(&self) -> &Path { &self.object_meta.location } + + /// Update the file to only scan the specified range (in bytes) + pub fn with_range(mut self, start: i64, end: i64) -> Self { + self.range = Some(FileRange { start, end }); + self + } } impl From for PartitionedFile { diff --git a/datafusion/core/src/datasource/physical_plan/csv.rs b/datafusion/core/src/datasource/physical_plan/csv.rs index 816a82543bab..0eca37da139d 100644 --- a/datafusion/core/src/datasource/physical_plan/csv.rs +++ b/datafusion/core/src/datasource/physical_plan/csv.rs @@ -23,7 +23,7 @@ use std::ops::Range; use std::sync::Arc; use std::task::Poll; -use super::FileScanConfig; +use super::{FileGroupPartitioner, FileScanConfig}; use crate::datasource::file_format::file_compression_type::FileCompressionType; use crate::datasource::listing::{FileRange, ListingTableUrl}; use crate::datasource::physical_plan::file_stream::{ @@ -177,7 +177,7 @@ impl ExecutionPlan for CsvExec { } /// Redistribute files across partitions according to their size - /// See comments on `repartition_file_groups()` for more detail. + /// See comments on [`FileGroupPartitioner`] for more detail. /// /// Return `None` if can't get repartitioned(empty/compressed file). fn repartitioned( @@ -191,11 +191,11 @@ impl ExecutionPlan for CsvExec { return Ok(None); } - let repartitioned_file_groups_option = FileScanConfig::repartition_file_groups( - self.base_config.file_groups.clone(), - target_partitions, - repartition_file_min_size, - ); + let repartitioned_file_groups_option = FileGroupPartitioner::new() + .with_target_partitions(target_partitions) + .with_preserve_order_within_groups(self.output_ordering().is_some()) + .with_repartition_file_min_size(repartition_file_min_size) + .repartition_file_groups(&self.base_config.file_groups); if let Some(repartitioned_file_groups) = repartitioned_file_groups_option { let mut new_plan = self.clone(); diff --git a/datafusion/core/src/datasource/physical_plan/file_groups.rs b/datafusion/core/src/datasource/physical_plan/file_groups.rs new file mode 100644 index 000000000000..6456bd5c7276 --- /dev/null +++ b/datafusion/core/src/datasource/physical_plan/file_groups.rs @@ -0,0 +1,826 @@ +// 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. + +//! Logic for managing groups of [`PartitionedFile`]s in DataFusion + +use crate::datasource::listing::{FileRange, PartitionedFile}; +use itertools::Itertools; +use std::cmp::min; +use std::collections::BinaryHeap; +use std::iter::repeat_with; + +/// Repartition input files into `target_partitions` partitions, if total file size exceed +/// `repartition_file_min_size` +/// +/// This partitions evenly by file byte range, and does not have any knowledge +/// of how data is laid out in specific files. The specific `FileOpener` are +/// responsible for the actual partitioning on specific data source type. (e.g. +/// the `CsvOpener` will read lines overlap with byte range as well as +/// handle boundaries to ensure all lines will be read exactly once) +/// +/// # Example +/// +/// For example, if there are two files `A` and `B` that we wish to read with 4 +/// partitions (with 4 threads) they will be divided as follows: +/// +/// ```text +/// ┌ ─ ─ ─ ─ ─ ─ ─ ─ ─ ─ ┐ +/// ┌─────────────────┐ +/// │ │ │ │ +/// │ File A │ +/// │ │ Range: 0-2MB │ │ +/// │ │ +/// │ └─────────────────┘ │ +/// ─ ─ ─ ─ ─ ─ ─ ─ ─ ─ ─ +/// ┌─────────────────┐ ┌ ─ ─ ─ ─ ─ ─ ─ ─ ─ ─ ┐ +/// │ │ ┌─────────────────┐ +/// │ │ │ │ │ │ +/// │ │ │ File A │ +/// │ │ │ │ Range 2-4MB │ │ +/// │ │ │ │ +/// │ │ │ └─────────────────┘ │ +/// │ File A (7MB) │ ────────▶ ─ ─ ─ ─ ─ ─ ─ ─ ─ ─ ─ +/// │ │ ┌ ─ ─ ─ ─ ─ ─ ─ ─ ─ ─ ┐ +/// │ │ ┌─────────────────┐ +/// │ │ │ │ │ │ +/// │ │ │ File A │ +/// │ │ │ │ Range: 4-6MB │ │ +/// │ │ │ │ +/// │ │ │ └─────────────────┘ │ +/// └─────────────────┘ ─ ─ ─ ─ ─ ─ ─ ─ ─ ─ ─ +/// ┌─────────────────┐ ┌ ─ ─ ─ ─ ─ ─ ─ ─ ─ ─ ┐ +/// │ File B (1MB) │ ┌─────────────────┐ +/// │ │ │ │ File A │ │ +/// └─────────────────┘ │ Range: 6-7MB │ +/// │ └─────────────────┘ │ +/// ┌─────────────────┐ +/// │ │ File B (1MB) │ │ +/// │ │ +/// │ └─────────────────┘ │ +/// ─ ─ ─ ─ ─ ─ ─ ─ ─ ─ ─ +/// +/// If target_partitions = 4, +/// divides into 4 groups +/// ``` +/// +/// # Maintaining Order +/// +/// Within each group files are read sequentially. Thus, if the overall order of +/// tuples must be preserved, multiple files can not be mixed in the same group. +/// +/// In this case, the code will split the largest files evenly into any +/// available empty groups, but the overall distribution may not not be as even +/// as as even as if the order did not need to be preserved. +/// +/// ```text +/// ┌ ─ ─ ─ ─ ─ ─ ─ ─ ─ ─ ┐ +/// ┌─────────────────┐ +/// │ │ │ │ +/// │ File A │ +/// │ │ Range: 0-2MB │ │ +/// │ │ +/// ┌─────────────────┐ │ └─────────────────┘ │ +/// │ │ ─ ─ ─ ─ ─ ─ ─ ─ ─ ─ ─ +/// │ │ ┌ ─ ─ ─ ─ ─ ─ ─ ─ ─ ─ ┐ +/// │ │ ┌─────────────────┐ +/// │ │ │ │ │ │ +/// │ │ │ File A │ +/// │ │ │ │ Range 2-4MB │ │ +/// │ File A (6MB) │ ────────▶ │ │ +/// │ (ordered) │ │ └─────────────────┘ │ +/// │ │ ─ ─ ─ ─ ─ ─ ─ ─ ─ ─ ─ +/// │ │ ┌ ─ ─ ─ ─ ─ ─ ─ ─ ─ ─ ┐ +/// │ │ ┌─────────────────┐ +/// │ │ │ │ │ │ +/// │ │ │ File A │ +/// │ │ │ │ Range: 4-6MB │ │ +/// └─────────────────┘ │ │ +/// ┌─────────────────┐ │ └─────────────────┘ │ +/// │ File B (1MB) │ ─ ─ ─ ─ ─ ─ ─ ─ ─ ─ ─ +/// │ (ordered) │ ┌ ─ ─ ─ ─ ─ ─ ─ ─ ─ ─ ┐ +/// └─────────────────┘ ┌─────────────────┐ +/// │ │ File B (1MB) │ │ +/// │ │ +/// │ └─────────────────┘ │ +/// ─ ─ ─ ─ ─ ─ ─ ─ ─ ─ ─ +/// +/// If target_partitions = 4, +/// divides into 4 groups +/// ``` +#[derive(Debug, Clone, Copy)] +pub struct FileGroupPartitioner { + /// how many partitions should be created + target_partitions: usize, + /// the minimum size for a file to be repartitioned. + repartition_file_min_size: usize, + /// if the order when reading the files must be preserved + preserve_order_within_groups: bool, +} + +impl Default for FileGroupPartitioner { + fn default() -> Self { + Self::new() + } +} + +impl FileGroupPartitioner { + /// Creates a new [`FileGroupPartitioner`] with default values: + /// 1. `target_partitions = 1` + /// 2. `repartition_file_min_size = 10MB` + /// 3. `preserve_order_within_groups = false` + pub fn new() -> Self { + Self { + target_partitions: 1, + repartition_file_min_size: 10 * 1024 * 1024, + preserve_order_within_groups: false, + } + } + + /// Set the target partitions + pub fn with_target_partitions(mut self, target_partitions: usize) -> Self { + self.target_partitions = target_partitions; + self + } + + /// Set the minimum size at which to repartition a file + pub fn with_repartition_file_min_size( + mut self, + repartition_file_min_size: usize, + ) -> Self { + self.repartition_file_min_size = repartition_file_min_size; + self + } + + /// Set whether the order of tuples within a file must be preserved + pub fn with_preserve_order_within_groups( + mut self, + preserve_order_within_groups: bool, + ) -> Self { + self.preserve_order_within_groups = preserve_order_within_groups; + self + } + + /// Repartition input files according to the settings on this [`FileGroupPartitioner`]. + /// + /// If no repartitioning is needed or possible, return `None`. + pub fn repartition_file_groups( + &self, + file_groups: &[Vec], + ) -> Option>> { + if file_groups.is_empty() { + return None; + } + + // Perform redistribution only in case all files should be read from beginning to end + let has_ranges = file_groups.iter().flatten().any(|f| f.range.is_some()); + if has_ranges { + return None; + } + + // special case when order must be preserved + if self.preserve_order_within_groups { + self.repartition_preserving_order(file_groups) + } else { + self.repartition_evenly_by_size(file_groups) + } + } + + /// Evenly repartition files across partitions by size, ignoring any + /// existing grouping / ordering + fn repartition_evenly_by_size( + &self, + file_groups: &[Vec], + ) -> Option>> { + let target_partitions = self.target_partitions; + let repartition_file_min_size = self.repartition_file_min_size; + let flattened_files = file_groups.iter().flatten().collect::>(); + + let total_size = flattened_files + .iter() + .map(|f| f.object_meta.size as i64) + .sum::(); + if total_size < (repartition_file_min_size as i64) || total_size == 0 { + return None; + } + + let target_partition_size = + (total_size as usize + (target_partitions) - 1) / (target_partitions); + + let current_partition_index: usize = 0; + let current_partition_size: usize = 0; + + // Partition byte range evenly for all `PartitionedFile`s + let repartitioned_files = flattened_files + .into_iter() + .scan( + (current_partition_index, current_partition_size), + |state, source_file| { + let mut produced_files = vec![]; + let mut range_start = 0; + while range_start < source_file.object_meta.size { + let range_end = min( + range_start + (target_partition_size - state.1), + source_file.object_meta.size, + ); + + let mut produced_file = source_file.clone(); + produced_file.range = Some(FileRange { + start: range_start as i64, + end: range_end as i64, + }); + produced_files.push((state.0, produced_file)); + + if state.1 + (range_end - range_start) >= target_partition_size { + state.0 += 1; + state.1 = 0; + } else { + state.1 += range_end - range_start; + } + range_start = range_end; + } + Some(produced_files) + }, + ) + .flatten() + .group_by(|(partition_idx, _)| *partition_idx) + .into_iter() + .map(|(_, group)| group.map(|(_, vals)| vals).collect_vec()) + .collect_vec(); + + Some(repartitioned_files) + } + + /// Redistribute file groups across size preserving order + fn repartition_preserving_order( + &self, + file_groups: &[Vec], + ) -> Option>> { + // Can't repartition and preserve order if there are more groups + // than partitions + if file_groups.len() >= self.target_partitions { + return None; + } + let num_new_groups = self.target_partitions - file_groups.len(); + + // If there is only a single file + if file_groups.len() == 1 && file_groups[0].len() == 1 { + return self.repartition_evenly_by_size(file_groups); + } + + // Find which files could be split (single file groups) + let mut heap: BinaryHeap<_> = file_groups + .iter() + .enumerate() + .filter_map(|(group_index, group)| { + // ignore groups that do not have exactly 1 file + if group.len() == 1 { + Some(ToRepartition { + source_index: group_index, + file_size: group[0].object_meta.size, + new_groups: vec![group_index], + }) + } else { + None + } + }) + .collect(); + + // No files can be redistributed + if heap.is_empty() { + return None; + } + + // Add new empty groups to which we will redistribute ranges of existing files + let mut file_groups: Vec<_> = file_groups + .iter() + .cloned() + .chain(repeat_with(Vec::new).take(num_new_groups)) + .collect(); + + // Divide up empty groups + for (group_index, group) in file_groups.iter().enumerate() { + if !group.is_empty() { + continue; + } + // Pick the file that has the largest ranges to read so far + let mut largest_group = heap.pop().unwrap(); + largest_group.new_groups.push(group_index); + heap.push(largest_group); + } + + // Distribute files to their newly assigned groups + while let Some(to_repartition) = heap.pop() { + let range_size = to_repartition.range_size() as i64; + let ToRepartition { + source_index, + file_size, + new_groups, + } = to_repartition; + assert_eq!(file_groups[source_index].len(), 1); + let original_file = file_groups[source_index].pop().unwrap(); + + let last_group = new_groups.len() - 1; + let mut range_start: i64 = 0; + let mut range_end: i64 = range_size; + for (i, group_index) in new_groups.into_iter().enumerate() { + let target_group = &mut file_groups[group_index]; + assert!(target_group.is_empty()); + + // adjust last range to include the entire file + if i == last_group { + range_end = file_size as i64; + } + target_group + .push(original_file.clone().with_range(range_start, range_end)); + range_start = range_end; + range_end += range_size; + } + } + + Some(file_groups) + } +} + +/// Tracks how a individual file will be repartitioned +#[derive(Debug, Clone, PartialEq, Eq)] +struct ToRepartition { + /// the index from which the original file will be taken + source_index: usize, + /// the size of the original file + file_size: usize, + /// indexes of which group(s) will this be distributed to (including `source_index`) + new_groups: Vec, +} + +impl ToRepartition { + // how big will each file range be when this file is read in its new groups? + fn range_size(&self) -> usize { + self.file_size / self.new_groups.len() + } +} + +impl PartialOrd for ToRepartition { + fn partial_cmp(&self, other: &Self) -> Option { + Some(self.cmp(other)) + } +} + +/// Order based on individual range +impl Ord for ToRepartition { + fn cmp(&self, other: &Self) -> std::cmp::Ordering { + self.range_size().cmp(&other.range_size()) + } +} + +#[cfg(test)] +mod test { + use super::*; + + /// Empty file won't get partitioned + #[test] + fn repartition_empty_file_only() { + let partitioned_file_empty = pfile("empty", 0); + let file_group = vec![vec![partitioned_file_empty.clone()]]; + + let partitioned_files = FileGroupPartitioner::new() + .with_target_partitions(4) + .with_repartition_file_min_size(0) + .repartition_file_groups(&file_group); + + assert_partitioned_files(None, partitioned_files); + } + + /// Repartition when there is a empty file in file groups + #[test] + fn repartition_empty_files() { + let pfile_a = pfile("a", 10); + let pfile_b = pfile("b", 10); + let pfile_empty = pfile("empty", 0); + + let empty_first = vec![ + vec![pfile_empty.clone()], + vec![pfile_a.clone()], + vec![pfile_b.clone()], + ]; + let empty_middle = vec![ + vec![pfile_a.clone()], + vec![pfile_empty.clone()], + vec![pfile_b.clone()], + ]; + let empty_last = vec![vec![pfile_a], vec![pfile_b], vec![pfile_empty]]; + + // Repartition file groups into x partitions + let expected_2 = vec![ + vec![pfile("a", 10).with_range(0, 10)], + vec![pfile("b", 10).with_range(0, 10)], + ]; + let expected_3 = vec![ + vec![pfile("a", 10).with_range(0, 7)], + vec![ + pfile("a", 10).with_range(7, 10), + pfile("b", 10).with_range(0, 4), + ], + vec![pfile("b", 10).with_range(4, 10)], + ]; + + let file_groups_tests = [empty_first, empty_middle, empty_last]; + + for fg in file_groups_tests { + let all_expected = [(2, expected_2.clone()), (3, expected_3.clone())]; + for (n_partition, expected) in all_expected { + let actual = FileGroupPartitioner::new() + .with_target_partitions(n_partition) + .with_repartition_file_min_size(10) + .repartition_file_groups(&fg); + + assert_partitioned_files(Some(expected), actual); + } + } + } + + #[test] + fn repartition_single_file() { + // Single file, single partition into multiple partitions + let single_partition = vec![vec![pfile("a", 123)]]; + + let actual = FileGroupPartitioner::new() + .with_target_partitions(4) + .with_repartition_file_min_size(10) + .repartition_file_groups(&single_partition); + + let expected = Some(vec![ + vec![pfile("a", 123).with_range(0, 31)], + vec![pfile("a", 123).with_range(31, 62)], + vec![pfile("a", 123).with_range(62, 93)], + vec![pfile("a", 123).with_range(93, 123)], + ]); + assert_partitioned_files(expected, actual); + } + + #[test] + fn repartition_too_much_partitions() { + // Single file, single partition into 96 partitions + let partitioned_file = pfile("a", 8); + let single_partition = vec![vec![partitioned_file]]; + + let actual = FileGroupPartitioner::new() + .with_target_partitions(96) + .with_repartition_file_min_size(5) + .repartition_file_groups(&single_partition); + + let expected = Some(vec![ + vec![pfile("a", 8).with_range(0, 1)], + vec![pfile("a", 8).with_range(1, 2)], + vec![pfile("a", 8).with_range(2, 3)], + vec![pfile("a", 8).with_range(3, 4)], + vec![pfile("a", 8).with_range(4, 5)], + vec![pfile("a", 8).with_range(5, 6)], + vec![pfile("a", 8).with_range(6, 7)], + vec![pfile("a", 8).with_range(7, 8)], + ]); + + assert_partitioned_files(expected, actual); + } + + #[test] + fn repartition_multiple_partitions() { + // Multiple files in single partition after redistribution + let source_partitions = vec![vec![pfile("a", 40)], vec![pfile("b", 60)]]; + + let actual = FileGroupPartitioner::new() + .with_target_partitions(3) + .with_repartition_file_min_size(10) + .repartition_file_groups(&source_partitions); + + let expected = Some(vec![ + vec![pfile("a", 40).with_range(0, 34)], + vec![ + pfile("a", 40).with_range(34, 40), + pfile("b", 60).with_range(0, 28), + ], + vec![pfile("b", 60).with_range(28, 60)], + ]); + assert_partitioned_files(expected, actual); + } + + #[test] + fn repartition_same_num_partitions() { + // "Rebalance" files across partitions + let source_partitions = vec![vec![pfile("a", 40)], vec![pfile("b", 60)]]; + + let actual = FileGroupPartitioner::new() + .with_target_partitions(2) + .with_repartition_file_min_size(10) + .repartition_file_groups(&source_partitions); + + let expected = Some(vec![ + vec![ + pfile("a", 40).with_range(0, 40), + pfile("b", 60).with_range(0, 10), + ], + vec![pfile("b", 60).with_range(10, 60)], + ]); + assert_partitioned_files(expected, actual); + } + + #[test] + fn repartition_no_action_ranges() { + // No action due to Some(range) in second file + let source_partitions = vec![ + vec![pfile("a", 123)], + vec![pfile("b", 144).with_range(1, 50)], + ]; + + let actual = FileGroupPartitioner::new() + .with_target_partitions(65) + .with_repartition_file_min_size(10) + .repartition_file_groups(&source_partitions); + + assert_partitioned_files(None, actual) + } + + #[test] + fn repartition_no_action_min_size() { + // No action due to target_partition_size + let single_partition = vec![vec![pfile("a", 123)]]; + + let actual = FileGroupPartitioner::new() + .with_target_partitions(65) + .with_repartition_file_min_size(500) + .repartition_file_groups(&single_partition); + + assert_partitioned_files(None, actual) + } + + #[test] + fn repartition_no_action_zero_files() { + // No action due to no files + let empty_partition = vec![]; + + let partitioner = FileGroupPartitioner::new() + .with_target_partitions(65) + .with_repartition_file_min_size(500); + + assert_partitioned_files(None, repartition_test(partitioner, empty_partition)) + } + + #[test] + fn repartition_ordered_no_action_too_few_partitions() { + // No action as there are no new groups to redistribute to + let input_partitions = vec![vec![pfile("a", 100)], vec![pfile("b", 200)]]; + + let actual = FileGroupPartitioner::new() + .with_preserve_order_within_groups(true) + .with_target_partitions(2) + .with_repartition_file_min_size(10) + .repartition_file_groups(&input_partitions); + + assert_partitioned_files(None, actual) + } + + #[test] + fn repartition_ordered_no_action_file_too_small() { + // No action as there are no new groups to redistribute to + let single_partition = vec![vec![pfile("a", 100)]]; + + let actual = FileGroupPartitioner::new() + .with_preserve_order_within_groups(true) + .with_target_partitions(2) + // file is too small to repartition + .with_repartition_file_min_size(1000) + .repartition_file_groups(&single_partition); + + assert_partitioned_files(None, actual) + } + + #[test] + fn repartition_ordered_one_large_file() { + // "Rebalance" the single large file across partitions + let source_partitions = vec![vec![pfile("a", 100)]]; + + let actual = FileGroupPartitioner::new() + .with_preserve_order_within_groups(true) + .with_target_partitions(3) + .with_repartition_file_min_size(10) + .repartition_file_groups(&source_partitions); + + let expected = Some(vec![ + vec![pfile("a", 100).with_range(0, 34)], + vec![pfile("a", 100).with_range(34, 68)], + vec![pfile("a", 100).with_range(68, 100)], + ]); + assert_partitioned_files(expected, actual); + } + + #[test] + fn repartition_ordered_one_large_one_small_file() { + // "Rebalance" the single large file across empty partitions, but can't split + // small file + let source_partitions = vec![vec![pfile("a", 100)], vec![pfile("b", 30)]]; + + let actual = FileGroupPartitioner::new() + .with_preserve_order_within_groups(true) + .with_target_partitions(4) + .with_repartition_file_min_size(10) + .repartition_file_groups(&source_partitions); + + let expected = Some(vec![ + // scan first third of "a" + vec![pfile("a", 100).with_range(0, 33)], + // only b in this group (can't do this) + vec![pfile("b", 30).with_range(0, 30)], + // second third of "a" + vec![pfile("a", 100).with_range(33, 66)], + // final third of "a" + vec![pfile("a", 100).with_range(66, 100)], + ]); + assert_partitioned_files(expected, actual); + } + + #[test] + fn repartition_ordered_two_large_files() { + // "Rebalance" two large files across empty partitions, but can't mix them + let source_partitions = vec![vec![pfile("a", 100)], vec![pfile("b", 100)]]; + + let actual = FileGroupPartitioner::new() + .with_preserve_order_within_groups(true) + .with_target_partitions(4) + .with_repartition_file_min_size(10) + .repartition_file_groups(&source_partitions); + + let expected = Some(vec![ + // scan first half of "a" + vec![pfile("a", 100).with_range(0, 50)], + // scan first half of "b" + vec![pfile("b", 100).with_range(0, 50)], + // second half of "a" + vec![pfile("a", 100).with_range(50, 100)], + // second half of "b" + vec![pfile("b", 100).with_range(50, 100)], + ]); + assert_partitioned_files(expected, actual); + } + + #[test] + fn repartition_ordered_two_large_one_small_files() { + // "Rebalance" two large files and one small file across empty partitions + let source_partitions = vec![ + vec![pfile("a", 100)], + vec![pfile("b", 100)], + vec![pfile("c", 30)], + ]; + + let partitioner = FileGroupPartitioner::new() + .with_preserve_order_within_groups(true) + .with_repartition_file_min_size(10); + + // with 4 partitions, can only split the first large file "a" + let actual = partitioner + .with_target_partitions(4) + .repartition_file_groups(&source_partitions); + + let expected = Some(vec![ + // scan first half of "a" + vec![pfile("a", 100).with_range(0, 50)], + // All of "b" + vec![pfile("b", 100).with_range(0, 100)], + // All of "c" + vec![pfile("c", 30).with_range(0, 30)], + // second half of "a" + vec![pfile("a", 100).with_range(50, 100)], + ]); + assert_partitioned_files(expected, actual); + + // With 5 partitions, we can split both "a" and "b", but they can't be intermixed + let actual = partitioner + .with_target_partitions(5) + .repartition_file_groups(&source_partitions); + + let expected = Some(vec![ + // scan first half of "a" + vec![pfile("a", 100).with_range(0, 50)], + // scan first half of "b" + vec![pfile("b", 100).with_range(0, 50)], + // All of "c" + vec![pfile("c", 30).with_range(0, 30)], + // second half of "a" + vec![pfile("a", 100).with_range(50, 100)], + // second half of "b" + vec![pfile("b", 100).with_range(50, 100)], + ]); + assert_partitioned_files(expected, actual); + } + + #[test] + fn repartition_ordered_one_large_one_small_existing_empty() { + // "Rebalance" files using existing empty partition + let source_partitions = + vec![vec![pfile("a", 100)], vec![], vec![pfile("b", 40)], vec![]]; + + let actual = FileGroupPartitioner::new() + .with_preserve_order_within_groups(true) + .with_target_partitions(5) + .with_repartition_file_min_size(10) + .repartition_file_groups(&source_partitions); + + // Of the three available groups (2 original empty and 1 new from the + // target partitions), assign two to "a" and one to "b" + let expected = Some(vec![ + // Scan of "a" across three groups + vec![pfile("a", 100).with_range(0, 33)], + vec![pfile("a", 100).with_range(33, 66)], + // scan first half of "b" + vec![pfile("b", 40).with_range(0, 20)], + // final third of "a" + vec![pfile("a", 100).with_range(66, 100)], + // second half of "b" + vec![pfile("b", 40).with_range(20, 40)], + ]); + assert_partitioned_files(expected, actual); + } + #[test] + fn repartition_ordered_existing_group_multiple_files() { + // groups with multiple files in a group can not be changed, but can divide others + let source_partitions = vec![ + // two files in an existing partition + vec![pfile("a", 100), pfile("b", 100)], + vec![pfile("c", 40)], + ]; + + let actual = FileGroupPartitioner::new() + .with_preserve_order_within_groups(true) + .with_target_partitions(3) + .with_repartition_file_min_size(10) + .repartition_file_groups(&source_partitions); + + // Of the three available groups (2 original empty and 1 new from the + // target partitions), assign two to "a" and one to "b" + let expected = Some(vec![ + // don't try and rearrange files in the existing partition + // assuming that the caller had a good reason to put them that way. + // (it is technically possible to split off ranges from the files if desired) + vec![pfile("a", 100), pfile("b", 100)], + // first half of "c" + vec![pfile("c", 40).with_range(0, 20)], + // second half of "c" + vec![pfile("c", 40).with_range(20, 40)], + ]); + assert_partitioned_files(expected, actual); + } + + /// Asserts that the two groups of `ParititonedFile` are the same + /// (PartitionedFile doesn't implement PartialEq) + fn assert_partitioned_files( + expected: Option>>, + actual: Option>>, + ) { + match (expected, actual) { + (None, None) => {} + (Some(_), None) => panic!("Expected Some, got None"), + (None, Some(_)) => panic!("Expected None, got Some"), + (Some(expected), Some(actual)) => { + let expected_string = format!("{:#?}", expected); + let actual_string = format!("{:#?}", actual); + assert_eq!(expected_string, actual_string); + } + } + } + + /// returns a partitioned file with the specified path and size + fn pfile(path: impl Into, file_size: u64) -> PartitionedFile { + PartitionedFile::new(path, file_size) + } + + /// repartition the file groups both with and without preserving order + /// asserting they return the same value and returns that value + fn repartition_test( + partitioner: FileGroupPartitioner, + file_groups: Vec>, + ) -> Option>> { + let repartitioned = partitioner.repartition_file_groups(&file_groups); + + let repartitioned_preserving_sort = partitioner + .with_preserve_order_within_groups(true) + .repartition_file_groups(&file_groups); + + assert_partitioned_files( + repartitioned.clone(), + repartitioned_preserving_sort.clone(), + ); + repartitioned + } +} diff --git a/datafusion/core/src/datasource/physical_plan/file_scan_config.rs b/datafusion/core/src/datasource/physical_plan/file_scan_config.rs index d308397ab6e2..89694ff28500 100644 --- a/datafusion/core/src/datasource/physical_plan/file_scan_config.rs +++ b/datafusion/core/src/datasource/physical_plan/file_scan_config.rs @@ -19,15 +19,11 @@ //! file sources. use std::{ - borrow::Cow, cmp::min, collections::HashMap, fmt::Debug, marker::PhantomData, - sync::Arc, vec, + borrow::Cow, collections::HashMap, fmt::Debug, marker::PhantomData, sync::Arc, vec, }; -use super::get_projected_output_ordering; -use crate::datasource::{ - listing::{FileRange, PartitionedFile}, - object_store::ObjectStoreUrl, -}; +use super::{get_projected_output_ordering, FileGroupPartitioner}; +use crate::datasource::{listing::PartitionedFile, object_store::ObjectStoreUrl}; use crate::{ error::{DataFusionError, Result}, scalar::ScalarValue, @@ -42,7 +38,6 @@ use datafusion_common::stats::Precision; use datafusion_common::{exec_err, ColumnStatistics, Statistics}; use datafusion_physical_expr::LexOrdering; -use itertools::Itertools; use log::warn; /// Convert type to a type suitable for use as a [`ListingTable`] @@ -176,79 +171,17 @@ impl FileScanConfig { }) } - /// Repartition all input files into `target_partitions` partitions, if total file size exceed - /// `repartition_file_min_size` - /// `target_partitions` and `repartition_file_min_size` directly come from configuration. - /// - /// This function only try to partition file byte range evenly, and let specific `FileOpener` to - /// do actual partition on specific data source type. (e.g. `CsvOpener` will only read lines - /// overlap with byte range but also handle boundaries to ensure all lines will be read exactly once) + #[allow(missing_docs)] + #[deprecated(since = "33.0.0", note = "Use SessionContext::new_with_config")] pub fn repartition_file_groups( file_groups: Vec>, target_partitions: usize, repartition_file_min_size: usize, ) -> Option>> { - let flattened_files = file_groups.iter().flatten().collect::>(); - - // Perform redistribution only in case all files should be read from beginning to end - let has_ranges = flattened_files.iter().any(|f| f.range.is_some()); - if has_ranges { - return None; - } - - let total_size = flattened_files - .iter() - .map(|f| f.object_meta.size as i64) - .sum::(); - if total_size < (repartition_file_min_size as i64) || total_size == 0 { - return None; - } - - let target_partition_size = - (total_size as usize + (target_partitions) - 1) / (target_partitions); - - let current_partition_index: usize = 0; - let current_partition_size: usize = 0; - - // Partition byte range evenly for all `PartitionedFile`s - let repartitioned_files = flattened_files - .into_iter() - .scan( - (current_partition_index, current_partition_size), - |state, source_file| { - let mut produced_files = vec![]; - let mut range_start = 0; - while range_start < source_file.object_meta.size { - let range_end = min( - range_start + (target_partition_size - state.1), - source_file.object_meta.size, - ); - - let mut produced_file = source_file.clone(); - produced_file.range = Some(FileRange { - start: range_start as i64, - end: range_end as i64, - }); - produced_files.push((state.0, produced_file)); - - if state.1 + (range_end - range_start) >= target_partition_size { - state.0 += 1; - state.1 = 0; - } else { - state.1 += range_end - range_start; - } - range_start = range_end; - } - Some(produced_files) - }, - ) - .flatten() - .group_by(|(partition_idx, _)| *partition_idx) - .into_iter() - .map(|(_, group)| group.map(|(_, vals)| vals).collect_vec()) - .collect_vec(); - - Some(repartitioned_files) + FileGroupPartitioner::new() + .with_target_partitions(target_partitions) + .with_repartition_file_min_size(repartition_file_min_size) + .repartition_file_groups(&file_groups) } } diff --git a/datafusion/core/src/datasource/physical_plan/mod.rs b/datafusion/core/src/datasource/physical_plan/mod.rs index 14e550eab1d5..8e4dd5400b20 100644 --- a/datafusion/core/src/datasource/physical_plan/mod.rs +++ b/datafusion/core/src/datasource/physical_plan/mod.rs @@ -20,11 +20,13 @@ mod arrow_file; mod avro; mod csv; +mod file_groups; mod file_scan_config; mod file_stream; mod json; #[cfg(feature = "parquet")] pub mod parquet; +pub use file_groups::FileGroupPartitioner; pub(crate) use self::csv::plan_to_csv; pub use self::csv::{CsvConfig, CsvExec, CsvOpener}; @@ -537,7 +539,6 @@ mod tests { }; use arrow_schema::Field; use chrono::Utc; - use datafusion_common::config::ConfigOptions; use crate::physical_plan::{DefaultDisplay, VerboseDisplay}; @@ -809,345 +810,4 @@ mod tests { extensions: None, } } - - /// Unit tests for `repartition_file_groups()` - #[cfg(feature = "parquet")] - mod repartition_file_groups_test { - use datafusion_common::Statistics; - use itertools::Itertools; - - use super::*; - - /// Empty file won't get partitioned - #[tokio::test] - async fn repartition_empty_file_only() { - let partitioned_file_empty = PartitionedFile::new("empty".to_string(), 0); - let file_group = vec![vec![partitioned_file_empty]]; - - let parquet_exec = ParquetExec::new( - FileScanConfig { - object_store_url: ObjectStoreUrl::local_filesystem(), - file_groups: file_group, - file_schema: Arc::new(Schema::empty()), - statistics: Statistics::new_unknown(&Schema::empty()), - projection: None, - limit: None, - table_partition_cols: vec![], - output_ordering: vec![], - infinite_source: false, - }, - None, - None, - ); - - let partitioned_file = repartition_with_size(&parquet_exec, 4, 0); - - assert!(partitioned_file[0][0].range.is_none()); - } - - // Repartition when there is a empty file in file groups - #[tokio::test] - async fn repartition_empty_files() { - let partitioned_file_a = PartitionedFile::new("a".to_string(), 10); - let partitioned_file_b = PartitionedFile::new("b".to_string(), 10); - let partitioned_file_empty = PartitionedFile::new("empty".to_string(), 0); - - let empty_first = vec![ - vec![partitioned_file_empty.clone()], - vec![partitioned_file_a.clone()], - vec![partitioned_file_b.clone()], - ]; - let empty_middle = vec![ - vec![partitioned_file_a.clone()], - vec![partitioned_file_empty.clone()], - vec![partitioned_file_b.clone()], - ]; - let empty_last = vec![ - vec![partitioned_file_a], - vec![partitioned_file_b], - vec![partitioned_file_empty], - ]; - - // Repartition file groups into x partitions - let expected_2 = - vec![(0, "a".to_string(), 0, 10), (1, "b".to_string(), 0, 10)]; - let expected_3 = vec![ - (0, "a".to_string(), 0, 7), - (1, "a".to_string(), 7, 10), - (1, "b".to_string(), 0, 4), - (2, "b".to_string(), 4, 10), - ]; - - //let file_groups_testset = [empty_first, empty_middle, empty_last]; - let file_groups_testset = [empty_first, empty_middle, empty_last]; - - for fg in file_groups_testset { - for (n_partition, expected) in [(2, &expected_2), (3, &expected_3)] { - let parquet_exec = ParquetExec::new( - FileScanConfig { - object_store_url: ObjectStoreUrl::local_filesystem(), - file_groups: fg.clone(), - file_schema: Arc::new(Schema::empty()), - statistics: Statistics::new_unknown(&Arc::new( - Schema::empty(), - )), - projection: None, - limit: None, - table_partition_cols: vec![], - output_ordering: vec![], - infinite_source: false, - }, - None, - None, - ); - - let actual = - repartition_with_size_to_vec(&parquet_exec, n_partition, 10); - - assert_eq!(expected, &actual); - } - } - } - - #[tokio::test] - async fn repartition_single_file() { - // Single file, single partition into multiple partitions - let partitioned_file = PartitionedFile::new("a".to_string(), 123); - let single_partition = vec![vec![partitioned_file]]; - let parquet_exec = ParquetExec::new( - FileScanConfig { - object_store_url: ObjectStoreUrl::local_filesystem(), - file_groups: single_partition, - file_schema: Arc::new(Schema::empty()), - statistics: Statistics::new_unknown(&Schema::empty()), - projection: None, - limit: None, - table_partition_cols: vec![], - output_ordering: vec![], - infinite_source: false, - }, - None, - None, - ); - - let actual = repartition_with_size_to_vec(&parquet_exec, 4, 10); - let expected = vec![ - (0, "a".to_string(), 0, 31), - (1, "a".to_string(), 31, 62), - (2, "a".to_string(), 62, 93), - (3, "a".to_string(), 93, 123), - ]; - assert_eq!(expected, actual); - } - - #[tokio::test] - async fn repartition_too_much_partitions() { - // Single file, single parittion into 96 partitions - let partitioned_file = PartitionedFile::new("a".to_string(), 8); - let single_partition = vec![vec![partitioned_file]]; - let parquet_exec = ParquetExec::new( - FileScanConfig { - object_store_url: ObjectStoreUrl::local_filesystem(), - file_groups: single_partition, - file_schema: Arc::new(Schema::empty()), - statistics: Statistics::new_unknown(&Schema::empty()), - projection: None, - limit: None, - table_partition_cols: vec![], - output_ordering: vec![], - infinite_source: false, - }, - None, - None, - ); - - let actual = repartition_with_size_to_vec(&parquet_exec, 96, 5); - let expected = vec![ - (0, "a".to_string(), 0, 1), - (1, "a".to_string(), 1, 2), - (2, "a".to_string(), 2, 3), - (3, "a".to_string(), 3, 4), - (4, "a".to_string(), 4, 5), - (5, "a".to_string(), 5, 6), - (6, "a".to_string(), 6, 7), - (7, "a".to_string(), 7, 8), - ]; - assert_eq!(expected, actual); - } - - #[tokio::test] - async fn repartition_multiple_partitions() { - // Multiple files in single partition after redistribution - let partitioned_file_1 = PartitionedFile::new("a".to_string(), 40); - let partitioned_file_2 = PartitionedFile::new("b".to_string(), 60); - let source_partitions = - vec![vec![partitioned_file_1], vec![partitioned_file_2]]; - let parquet_exec = ParquetExec::new( - FileScanConfig { - object_store_url: ObjectStoreUrl::local_filesystem(), - file_groups: source_partitions, - file_schema: Arc::new(Schema::empty()), - statistics: Statistics::new_unknown(&Schema::empty()), - projection: None, - limit: None, - table_partition_cols: vec![], - output_ordering: vec![], - infinite_source: false, - }, - None, - None, - ); - - let actual = repartition_with_size_to_vec(&parquet_exec, 3, 10); - let expected = vec![ - (0, "a".to_string(), 0, 34), - (1, "a".to_string(), 34, 40), - (1, "b".to_string(), 0, 28), - (2, "b".to_string(), 28, 60), - ]; - assert_eq!(expected, actual); - } - - #[tokio::test] - async fn repartition_same_num_partitions() { - // "Rebalance" files across partitions - let partitioned_file_1 = PartitionedFile::new("a".to_string(), 40); - let partitioned_file_2 = PartitionedFile::new("b".to_string(), 60); - let source_partitions = - vec![vec![partitioned_file_1], vec![partitioned_file_2]]; - let parquet_exec = ParquetExec::new( - FileScanConfig { - object_store_url: ObjectStoreUrl::local_filesystem(), - file_groups: source_partitions, - file_schema: Arc::new(Schema::empty()), - statistics: Statistics::new_unknown(&Schema::empty()), - projection: None, - limit: None, - table_partition_cols: vec![], - output_ordering: vec![], - infinite_source: false, - }, - None, - None, - ); - - let actual = repartition_with_size_to_vec(&parquet_exec, 2, 10); - let expected = vec![ - (0, "a".to_string(), 0, 40), - (0, "b".to_string(), 0, 10), - (1, "b".to_string(), 10, 60), - ]; - assert_eq!(expected, actual); - } - - #[tokio::test] - async fn repartition_no_action_ranges() { - // No action due to Some(range) in second file - let partitioned_file_1 = PartitionedFile::new("a".to_string(), 123); - let mut partitioned_file_2 = PartitionedFile::new("b".to_string(), 144); - partitioned_file_2.range = Some(FileRange { start: 1, end: 50 }); - - let source_partitions = - vec![vec![partitioned_file_1], vec![partitioned_file_2]]; - let parquet_exec = ParquetExec::new( - FileScanConfig { - object_store_url: ObjectStoreUrl::local_filesystem(), - file_groups: source_partitions, - file_schema: Arc::new(Schema::empty()), - statistics: Statistics::new_unknown(&Schema::empty()), - projection: None, - limit: None, - table_partition_cols: vec![], - output_ordering: vec![], - infinite_source: false, - }, - None, - None, - ); - - let actual = repartition_with_size(&parquet_exec, 65, 10); - assert_eq!(2, actual.len()); - } - - #[tokio::test] - async fn repartition_no_action_min_size() { - // No action due to target_partition_size - let partitioned_file = PartitionedFile::new("a".to_string(), 123); - let single_partition = vec![vec![partitioned_file]]; - let parquet_exec = ParquetExec::new( - FileScanConfig { - object_store_url: ObjectStoreUrl::local_filesystem(), - file_groups: single_partition, - file_schema: Arc::new(Schema::empty()), - statistics: Statistics::new_unknown(&Schema::empty()), - projection: None, - limit: None, - table_partition_cols: vec![], - output_ordering: vec![], - infinite_source: false, - }, - None, - None, - ); - - let actual = repartition_with_size(&parquet_exec, 65, 500); - assert_eq!(1, actual.len()); - } - - /// Calls `ParquetExec.repartitioned` with the specified - /// `target_partitions` and `repartition_file_min_size`, returning the - /// resulting `PartitionedFile`s - fn repartition_with_size( - parquet_exec: &ParquetExec, - target_partitions: usize, - repartition_file_min_size: usize, - ) -> Vec> { - let mut config = ConfigOptions::new(); - config.optimizer.repartition_file_min_size = repartition_file_min_size; - - parquet_exec - .repartitioned(target_partitions, &config) - .unwrap() // unwrap Result - .unwrap() // unwrap Option - .as_any() - .downcast_ref::() - .unwrap() - .base_config() - .file_groups - .clone() - } - - /// Calls `repartition_with_size` and returns a tuple for each output `PartitionedFile`: - /// - /// `(partition index, file path, start, end)` - fn repartition_with_size_to_vec( - parquet_exec: &ParquetExec, - target_partitions: usize, - repartition_file_min_size: usize, - ) -> Vec<(usize, String, i64, i64)> { - let file_groups = repartition_with_size( - parquet_exec, - target_partitions, - repartition_file_min_size, - ); - - file_groups - .iter() - .enumerate() - .flat_map(|(part_idx, files)| { - files - .iter() - .map(|f| { - ( - part_idx, - f.object_meta.location.to_string(), - f.range.as_ref().unwrap().start, - f.range.as_ref().unwrap().end, - ) - }) - .collect_vec() - }) - .collect_vec() - } - } } diff --git a/datafusion/core/src/datasource/physical_plan/parquet/mod.rs b/datafusion/core/src/datasource/physical_plan/parquet/mod.rs index 847ea6505632..2b10b05a273a 100644 --- a/datafusion/core/src/datasource/physical_plan/parquet/mod.rs +++ b/datafusion/core/src/datasource/physical_plan/parquet/mod.rs @@ -26,8 +26,8 @@ use crate::datasource::physical_plan::file_stream::{ FileOpenFuture, FileOpener, FileStream, }; use crate::datasource::physical_plan::{ - parquet::page_filter::PagePruningPredicate, DisplayAs, FileMeta, FileScanConfig, - SchemaAdapter, + parquet::page_filter::PagePruningPredicate, DisplayAs, FileGroupPartitioner, + FileMeta, FileScanConfig, SchemaAdapter, }; use crate::{ config::ConfigOptions, @@ -330,18 +330,18 @@ impl ExecutionPlan for ParquetExec { } /// Redistribute files across partitions according to their size - /// See comments on `get_file_groups_repartitioned()` for more detail. + /// See comments on [`FileGroupPartitioner`] for more detail. fn repartitioned( &self, target_partitions: usize, config: &ConfigOptions, ) -> Result>> { let repartition_file_min_size = config.optimizer.repartition_file_min_size; - let repartitioned_file_groups_option = FileScanConfig::repartition_file_groups( - self.base_config.file_groups.clone(), - target_partitions, - repartition_file_min_size, - ); + let repartitioned_file_groups_option = FileGroupPartitioner::new() + .with_target_partitions(target_partitions) + .with_repartition_file_min_size(repartition_file_min_size) + .with_preserve_order_within_groups(self.output_ordering().is_some()) + .repartition_file_groups(&self.base_config.file_groups); let mut new_plan = self.clone(); if let Some(repartitioned_file_groups) = repartitioned_file_groups_option { diff --git a/datafusion/core/src/physical_optimizer/enforce_distribution.rs b/datafusion/core/src/physical_optimizer/enforce_distribution.rs index f2e04989ef66..099759741a10 100644 --- a/datafusion/core/src/physical_optimizer/enforce_distribution.rs +++ b/datafusion/core/src/physical_optimizer/enforce_distribution.rs @@ -1761,6 +1761,7 @@ pub(crate) mod tests { parquet_exec_with_sort(vec![]) } + /// create a single parquet file that is sorted pub(crate) fn parquet_exec_with_sort( output_ordering: Vec>, ) -> Arc { @@ -1785,7 +1786,7 @@ pub(crate) mod tests { parquet_exec_multiple_sorted(vec![]) } - // Created a sorted parquet exec with multiple files + /// Created a sorted parquet exec with multiple files fn parquet_exec_multiple_sorted( output_ordering: Vec>, ) -> Arc { @@ -3858,6 +3859,56 @@ pub(crate) mod tests { Ok(()) } + #[test] + fn parallelization_multiple_files() -> Result<()> { + let schema = schema(); + let sort_key = vec![PhysicalSortExpr { + expr: col("a", &schema).unwrap(), + options: SortOptions::default(), + }]; + + let plan = filter_exec(parquet_exec_multiple_sorted(vec![sort_key])); + let plan = sort_required_exec(plan); + + // The groups must have only contiguous ranges of rows from the same file + // if any group has rows from multiple files, the data is no longer sorted destroyed + // https://github.com/apache/arrow-datafusion/issues/8451 + let expected = [ + "SortRequiredExec: [a@0 ASC]", + "FilterExec: c@2 = 0", + "ParquetExec: file_groups={3 groups: [[x:0..50], [y:0..100], [x:50..100]]}, projection=[a, b, c, d, e], output_ordering=[a@0 ASC]", ]; + let target_partitions = 3; + let repartition_size = 1; + assert_optimized!( + expected, + plan, + true, + true, + target_partitions, + true, + repartition_size + ); + + let expected = [ + "SortRequiredExec: [a@0 ASC]", + "FilterExec: c@2 = 0", + "ParquetExec: file_groups={8 groups: [[x:0..25], [y:0..25], [x:25..50], [y:25..50], [x:50..75], [y:50..75], [x:75..100], [y:75..100]]}, projection=[a, b, c, d, e], output_ordering=[a@0 ASC]", + ]; + let target_partitions = 8; + let repartition_size = 1; + assert_optimized!( + expected, + plan, + true, + true, + target_partitions, + true, + repartition_size + ); + + Ok(()) + } + #[test] /// CsvExec on compressed csv file will not be partitioned /// (Not able to decompress chunked csv file) @@ -4529,15 +4580,11 @@ pub(crate) mod tests { assert_plan_txt!(expected, physical_plan); let expected = &[ - "SortRequiredExec: [a@0 ASC]", // Since at the start of the rule ordering requirement is satisfied // EnforceDistribution rule satisfy this requirement also. - // ordering is re-satisfied by introduction of SortExec. - "SortExec: expr=[a@0 ASC]", + "SortRequiredExec: [a@0 ASC]", "FilterExec: c@2 = 0", - // ordering is lost here - "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=2", - "ParquetExec: file_groups={2 groups: [[x], [y]]}, projection=[a, b, c, d, e], output_ordering=[a@0 ASC]", + "ParquetExec: file_groups={10 groups: [[x:0..20], [y:0..20], [x:20..40], [y:20..40], [x:40..60], [y:40..60], [x:60..80], [y:60..80], [x:80..100], [y:80..100]]}, projection=[a, b, c, d, e], output_ordering=[a@0 ASC]", ]; let mut config = ConfigOptions::new(); diff --git a/datafusion/sqllogictest/test_files/repartition_scan.slt b/datafusion/sqllogictest/test_files/repartition_scan.slt index 551d6d9ed48a..5dcdbb504e76 100644 --- a/datafusion/sqllogictest/test_files/repartition_scan.slt +++ b/datafusion/sqllogictest/test_files/repartition_scan.slt @@ -118,7 +118,7 @@ physical_plan SortPreservingMergeExec: [column1@0 ASC NULLS LAST] --CoalesceBatchesExec: target_batch_size=8192 ----FilterExec: column1@0 != 42 -------ParquetExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/parquet_table/1.parquet:0..200], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/parquet_table/1.parquet:200..394, WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/parquet_table/2.parquet:0..6], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/parquet_table/2.parquet:6..206], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/parquet_table/2.parquet:206..403]]}, projection=[column1], predicate=column1@0 != 42, pruning_predicate=column1_min@0 != 42 OR 42 != column1_max@1 +------ParquetExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/parquet_table/1.parquet:0..197], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/parquet_table/2.parquet:0..201], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/parquet_table/2.parquet:201..403], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/parquet_table/1.parquet:197..394]]}, projection=[column1], output_ordering=[column1@0 ASC NULLS LAST], predicate=column1@0 != 42, pruning_predicate=column1_min@0 != 42 OR 42 != column1_max@1 # Cleanup statement ok From fc6cc48e372b0c945aa78d78207441bca2bd11bf Mon Sep 17 00:00:00 2001 From: Alex Huang Date: Sun, 17 Dec 2023 13:15:05 +0100 Subject: [PATCH 451/572] feat: support largelist in array_slice (#8561) * support largelist in array_slice * remove T trait * fix clippy --- .../physical-expr/src/array_expressions.rs | 110 ++++++++++----- datafusion/sqllogictest/test_files/array.slt | 129 ++++++++++++++++++ 2 files changed, 208 insertions(+), 31 deletions(-) diff --git a/datafusion/physical-expr/src/array_expressions.rs b/datafusion/physical-expr/src/array_expressions.rs index 7fa97dad7aa6..7ccf58af832d 100644 --- a/datafusion/physical-expr/src/array_expressions.rs +++ b/datafusion/physical-expr/src/array_expressions.rs @@ -524,11 +524,33 @@ pub fn array_except(args: &[ArrayRef]) -> Result { /// /// See test cases in `array.slt` for more details. pub fn array_slice(args: &[ArrayRef]) -> Result { - let list_array = as_list_array(&args[0])?; - let from_array = as_int64_array(&args[1])?; - let to_array = as_int64_array(&args[2])?; + let array_data_type = args[0].data_type(); + match array_data_type { + DataType::List(_) => { + let array = as_list_array(&args[0])?; + let from_array = as_int64_array(&args[1])?; + let to_array = as_int64_array(&args[2])?; + general_array_slice::(array, from_array, to_array) + } + DataType::LargeList(_) => { + let array = as_large_list_array(&args[0])?; + let from_array = as_int64_array(&args[1])?; + let to_array = as_int64_array(&args[2])?; + general_array_slice::(array, from_array, to_array) + } + _ => not_impl_err!("array_slice does not support type: {:?}", array_data_type), + } +} - let values = list_array.values(); +fn general_array_slice( + array: &GenericListArray, + from_array: &Int64Array, + to_array: &Int64Array, +) -> Result +where + i64: TryInto, +{ + let values = array.values(); let original_data = values.to_data(); let capacity = Capacities::Array(original_data.len()); @@ -539,72 +561,98 @@ pub fn array_slice(args: &[ArrayRef]) -> Result { // We have the slice syntax compatible with DuckDB v0.8.1. // The rule `adjusted_from_index` and `adjusted_to_index` follows the rule of array_slice in duckdb. - fn adjusted_from_index(index: i64, len: usize) -> Option { + fn adjusted_from_index(index: i64, len: O) -> Result> + where + i64: TryInto, + { // 0 ~ len - 1 let adjusted_zero_index = if index < 0 { - index + len as i64 + if let Ok(index) = index.try_into() { + index + len + } else { + return exec_err!("array_slice got invalid index: {}", index); + } } else { // array_slice(arr, 1, to) is the same as array_slice(arr, 0, to) - std::cmp::max(index - 1, 0) + if let Ok(index) = index.try_into() { + std::cmp::max(index - O::usize_as(1), O::usize_as(0)) + } else { + return exec_err!("array_slice got invalid index: {}", index); + } }; - if 0 <= adjusted_zero_index && adjusted_zero_index < len as i64 { - Some(adjusted_zero_index) + if O::usize_as(0) <= adjusted_zero_index && adjusted_zero_index < len { + Ok(Some(adjusted_zero_index)) } else { // Out of bounds - None + Ok(None) } } - fn adjusted_to_index(index: i64, len: usize) -> Option { + fn adjusted_to_index(index: i64, len: O) -> Result> + where + i64: TryInto, + { // 0 ~ len - 1 let adjusted_zero_index = if index < 0 { // array_slice in duckdb with negative to_index is python-like, so index itself is exclusive - index + len as i64 - 1 + if let Ok(index) = index.try_into() { + index + len - O::usize_as(1) + } else { + return exec_err!("array_slice got invalid index: {}", index); + } } else { // array_slice(arr, from, len + 1) is the same as array_slice(arr, from, len) - std::cmp::min(index - 1, len as i64 - 1) + if let Ok(index) = index.try_into() { + std::cmp::min(index - O::usize_as(1), len - O::usize_as(1)) + } else { + return exec_err!("array_slice got invalid index: {}", index); + } }; - if 0 <= adjusted_zero_index && adjusted_zero_index < len as i64 { - Some(adjusted_zero_index) + if O::usize_as(0) <= adjusted_zero_index && adjusted_zero_index < len { + Ok(Some(adjusted_zero_index)) } else { // Out of bounds - None + Ok(None) } } - let mut offsets = vec![0]; + let mut offsets = vec![O::usize_as(0)]; - for (row_index, offset_window) in list_array.offsets().windows(2).enumerate() { - let start = offset_window[0] as usize; - let end = offset_window[1] as usize; + for (row_index, offset_window) in array.offsets().windows(2).enumerate() { + let start = offset_window[0]; + let end = offset_window[1]; let len = end - start; // len 0 indicate array is null, return empty array in this row. - if len == 0 { + if len == O::usize_as(0) { offsets.push(offsets[row_index]); continue; } // If index is null, we consider it as the minimum / maximum index of the array. let from_index = if from_array.is_null(row_index) { - Some(0) + Some(O::usize_as(0)) } else { - adjusted_from_index(from_array.value(row_index), len) + adjusted_from_index::(from_array.value(row_index), len)? }; let to_index = if to_array.is_null(row_index) { - Some(len as i64 - 1) + Some(len - O::usize_as(1)) } else { - adjusted_to_index(to_array.value(row_index), len) + adjusted_to_index::(to_array.value(row_index), len)? }; if let (Some(from), Some(to)) = (from_index, to_index) { if from <= to { - assert!(start + to as usize <= end); - mutable.extend(0, start + from as usize, start + to as usize + 1); - offsets.push(offsets[row_index] + (to - from + 1) as i32); + assert!(start + to <= end); + mutable.extend( + 0, + (start + from).to_usize().unwrap(), + (start + to + O::usize_as(1)).to_usize().unwrap(), + ); + offsets.push(offsets[row_index] + (to - from + O::usize_as(1))); } else { // invalid range, return empty array offsets.push(offsets[row_index]); @@ -617,9 +665,9 @@ pub fn array_slice(args: &[ArrayRef]) -> Result { let data = mutable.freeze(); - Ok(Arc::new(ListArray::try_new( - Arc::new(Field::new("item", list_array.value_type(), true)), - OffsetBuffer::new(offsets.into()), + Ok(Arc::new(GenericListArray::::try_new( + Arc::new(Field::new("item", array.value_type(), true)), + OffsetBuffer::::new(offsets.into()), arrow_array::make_array(data), None, )?)) diff --git a/datafusion/sqllogictest/test_files/array.slt b/datafusion/sqllogictest/test_files/array.slt index 1202a2b1e99d..210739aa51da 100644 --- a/datafusion/sqllogictest/test_files/array.slt +++ b/datafusion/sqllogictest/test_files/array.slt @@ -912,128 +912,235 @@ select array_slice(make_array(1, 2, 3, 4, 5), 2, 4), array_slice(make_array('h', ---- [2, 3, 4] [h, e] +query ?? +select array_slice(arrow_cast(make_array(1, 2, 3, 4, 5), 'LargeList(Int64)'), 2, 4), array_slice(arrow_cast(make_array('h', 'e', 'l', 'l', 'o'), 'LargeList(Utf8)'), 1, 2); +---- +[2, 3, 4] [h, e] + # array_slice scalar function #2 (with positive indexes; full array) query ?? select array_slice(make_array(1, 2, 3, 4, 5), 0, 6), array_slice(make_array('h', 'e', 'l', 'l', 'o'), 0, 5); ---- [1, 2, 3, 4, 5] [h, e, l, l, o] +query ?? +select array_slice(arrow_cast(make_array(1, 2, 3, 4, 5), 'LargeList(Int64)'), 0, 6), array_slice(arrow_cast(make_array('h', 'e', 'l', 'l', 'o'), 'LargeList(Utf8)'), 0, 5); +---- +[1, 2, 3, 4, 5] [h, e, l, l, o] + # array_slice scalar function #3 (with positive indexes; first index = second index) query ?? select array_slice(make_array(1, 2, 3, 4, 5), 4, 4), array_slice(make_array('h', 'e', 'l', 'l', 'o'), 3, 3); ---- [4] [l] +query ?? +select array_slice(arrow_cast(make_array(1, 2, 3, 4, 5), 'LargeList(Int64)'), 4, 4), array_slice(arrow_cast(make_array('h', 'e', 'l', 'l', 'o'), 'LargeList(Utf8)'), 3, 3); +---- +[4] [l] + # array_slice scalar function #4 (with positive indexes; first index > second_index) query ?? select array_slice(make_array(1, 2, 3, 4, 5), 2, 1), array_slice(make_array('h', 'e', 'l', 'l', 'o'), 4, 1); ---- [] [] +query ?? +select array_slice(arrow_cast(make_array(1, 2, 3, 4, 5), 'LargeList(Int64)'), 2, 1), array_slice(arrow_cast(make_array('h', 'e', 'l', 'l', 'o'), 'LargeList(Utf8)'), 4, 1); +---- +[] [] + # array_slice scalar function #5 (with positive indexes; out of bounds) query ?? select array_slice(make_array(1, 2, 3, 4, 5), 2, 6), array_slice(make_array('h', 'e', 'l', 'l', 'o'), 3, 7); ---- [2, 3, 4, 5] [l, l, o] +query ?? +select array_slice(arrow_cast(make_array(1, 2, 3, 4, 5), 'LargeList(Int64)'), 2, 6), array_slice(arrow_cast(make_array('h', 'e', 'l', 'l', 'o'), 'LargeList(Utf8)'), 3, 7); +---- +[2, 3, 4, 5] [l, l, o] + # array_slice scalar function #6 (with positive indexes; nested array) query ? select array_slice(make_array(make_array(1, 2, 3, 4, 5), make_array(6, 7, 8, 9, 10)), 1, 1); ---- [[1, 2, 3, 4, 5]] +query ? +select array_slice(arrow_cast(make_array(make_array(1, 2, 3, 4, 5), make_array(6, 7, 8, 9, 10)), 'LargeList(List(Int64))'), 1, 1); +---- +[[1, 2, 3, 4, 5]] + # array_slice scalar function #7 (with zero and positive number) query ?? select array_slice(make_array(1, 2, 3, 4, 5), 0, 4), array_slice(make_array('h', 'e', 'l', 'l', 'o'), 0, 3); ---- [1, 2, 3, 4] [h, e, l] +query ?? +select array_slice(arrow_cast(make_array(1, 2, 3, 4, 5), 'LargeList(Int64)'), 0, 4), array_slice(arrow_cast(make_array('h', 'e', 'l', 'l', 'o'), 'LargeList(Utf8)'), 0, 3); +---- +[1, 2, 3, 4] [h, e, l] + # array_slice scalar function #8 (with NULL and positive number) query error select array_slice(make_array(1, 2, 3, 4, 5), NULL, 4), array_slice(make_array('h', 'e', 'l', 'l', 'o'), NULL, 3); +query error +select array_slice(arrow_cast(make_array(1, 2, 3, 4, 5), 'LargeList(Int64)'), NULL, 4), array_slice(arrow_cast(make_array('h', 'e', 'l', 'l', 'o'), 'LargeList(Utf8)'), NULL, 3); + # array_slice scalar function #9 (with positive number and NULL) query error select array_slice(make_array(1, 2, 3, 4, 5), 2, NULL), array_slice(make_array('h', 'e', 'l', 'l', 'o'), 3, NULL); +query error +select array_slice(arrow_cast(make_array(1, 2, 3, 4, 5), 'LargeList(Int64)'), 2, NULL), array_slice(arrow_cast(make_array('h', 'e', 'l', 'l', 'o'), 'LargeList(Utf8)'), 3, NULL); + # array_slice scalar function #10 (with zero-zero) query ?? select array_slice(make_array(1, 2, 3, 4, 5), 0, 0), array_slice(make_array('h', 'e', 'l', 'l', 'o'), 0, 0); ---- [] [] +query ?? +select array_slice(arrow_cast(make_array(1, 2, 3, 4, 5), 'LargeList(Int64)'), 0, 0), array_slice(arrow_cast(make_array('h', 'e', 'l', 'l', 'o'), 'LargeList(Utf8)'), 0, 0); +---- +[] [] + # array_slice scalar function #11 (with NULL-NULL) query error select array_slice(make_array(1, 2, 3, 4, 5), NULL), array_slice(make_array('h', 'e', 'l', 'l', 'o'), NULL); +query error +select array_slice(arrow_cast(make_array(1, 2, 3, 4, 5), 'LargeList(Int64)'), NULL), array_slice(arrow_cast(make_array('h', 'e', 'l', 'l', 'o'), 'LargeList(Utf8)'), NULL); + + # array_slice scalar function #12 (with zero and negative number) query ?? select array_slice(make_array(1, 2, 3, 4, 5), 0, -4), array_slice(make_array('h', 'e', 'l', 'l', 'o'), 0, -3); ---- [1] [h, e] +query ?? +select array_slice(arrow_cast(make_array(1, 2, 3, 4, 5), 'LargeList(Int64)'), 0, -4), array_slice(arrow_cast(make_array('h', 'e', 'l', 'l', 'o'), 'LargeList(Utf8)'), 0, -3); +---- +[1] [h, e] + # array_slice scalar function #13 (with negative number and NULL) query error select array_slice(make_array(1, 2, 3, 4, 5), -2, NULL), array_slice(make_array('h', 'e', 'l', 'l', 'o'), -3, NULL); +query error +select array_slice(arrow_cast(make_array(1, 2, 3, 4, 5), 'LargeList(Int64)'), -2, NULL), array_slice(arrow_cast(make_array('h', 'e', 'l', 'l', 'o'), 'LargeList(Utf8)'), -3, NULL); + # array_slice scalar function #14 (with NULL and negative number) query error select array_slice(make_array(1, 2, 3, 4, 5), NULL, -4), array_slice(make_array('h', 'e', 'l', 'l', 'o'), NULL, -3); +query error +select array_slice(arrow_cast(make_array(1, 2, 3, 4, 5), 'LargeList(Int64)'), NULL, -4), array_slice(arrow_cast(make_array('h', 'e', 'l', 'l', 'o'), 'LargeList(Utf8)'), NULL, -3); + # array_slice scalar function #15 (with negative indexes) query ?? select array_slice(make_array(1, 2, 3, 4, 5), -4, -1), array_slice(make_array('h', 'e', 'l', 'l', 'o'), -3, -1); ---- [2, 3, 4] [l, l] +query ?? +select array_slice(arrow_cast(make_array(1, 2, 3, 4, 5), 'LargeList(Int64)'), -4, -1), array_slice(arrow_cast(make_array('h', 'e', 'l', 'l', 'o'), 'LargeList(Utf8)'), -3, -1); +---- +[2, 3, 4] [l, l] + # array_slice scalar function #16 (with negative indexes; almost full array (only with negative indices cannot return full array)) query ?? select array_slice(make_array(1, 2, 3, 4, 5), -5, -1), array_slice(make_array('h', 'e', 'l', 'l', 'o'), -5, -1); ---- [1, 2, 3, 4] [h, e, l, l] +query ?? +select array_slice(arrow_cast(make_array(1, 2, 3, 4, 5), 'LargeList(Int64)'), -5, -1), array_slice(arrow_cast(make_array('h', 'e', 'l', 'l', 'o'), 'LargeList(Utf8)'), -5, -1); +---- +[1, 2, 3, 4] [h, e, l, l] + # array_slice scalar function #17 (with negative indexes; first index = second index) query ?? select array_slice(make_array(1, 2, 3, 4, 5), -4, -4), array_slice(make_array('h', 'e', 'l', 'l', 'o'), -3, -3); ---- [] [] +query ?? +select array_slice(arrow_cast(make_array(1, 2, 3, 4, 5), 'LargeList(Int64)'), -4, -4), array_slice(arrow_cast(make_array('h', 'e', 'l', 'l', 'o'), 'LargeList(Utf8)'), -3, -3); +---- +[] [] + # array_slice scalar function #18 (with negative indexes; first index > second_index) query ?? select array_slice(make_array(1, 2, 3, 4, 5), -4, -6), array_slice(make_array('h', 'e', 'l', 'l', 'o'), -3, -6); ---- [] [] +query ?? +select array_slice(arrow_cast(make_array(1, 2, 3, 4, 5), 'LargeList(Int64)'), -4, -6), array_slice(arrow_cast(make_array('h', 'e', 'l', 'l', 'o'), 'LargeList(Utf8)'), -3, -6); +---- +[] [] + # array_slice scalar function #19 (with negative indexes; out of bounds) query ?? select array_slice(make_array(1, 2, 3, 4, 5), -7, -2), array_slice(make_array('h', 'e', 'l', 'l', 'o'), -7, -3); ---- [] [] +query ?? +select array_slice(arrow_cast(make_array(1, 2, 3, 4, 5), 'LargeList(Int64)'), -7, -2), array_slice(arrow_cast(make_array('h', 'e', 'l', 'l', 'o'), 'LargeList(Utf8)'), -7, -3); +---- +[] [] + # array_slice scalar function #20 (with negative indexes; nested array) query ?? select array_slice(make_array(make_array(1, 2, 3, 4, 5), make_array(6, 7, 8, 9, 10)), -2, -1), array_slice(make_array(make_array(1, 2, 3), make_array(6, 7, 8)), -1, -1); ---- [[1, 2, 3, 4, 5]] [] +query ?? +select array_slice(arrow_cast(make_array(make_array(1, 2, 3, 4, 5), make_array(6, 7, 8, 9, 10)), 'LargeList(List(Int64))'), -2, -1), array_slice(arrow_cast(make_array(make_array(1, 2, 3), make_array(6, 7, 8)), 'LargeList(List(Int64))'), -1, -1); +---- +[[1, 2, 3, 4, 5]] [] + + # array_slice scalar function #21 (with first positive index and last negative index) query ?? select array_slice(make_array(1, 2, 3, 4, 5), 2, -3), array_slice(make_array('h', 'e', 'l', 'l', 'o'), 2, -2); ---- [2] [e, l] +query ?? +select array_slice(arrow_cast(make_array(1, 2, 3, 4, 5), 'LargeList(Int64)'), 2, -3), array_slice(arrow_cast(make_array('h', 'e', 'l', 'l', 'o'), 'LargeList(Utf8)'), 2, -2); +---- +[2] [e, l] + # array_slice scalar function #22 (with first negative index and last positive index) query ?? select array_slice(make_array(1, 2, 3, 4, 5), -2, 5), array_slice(make_array('h', 'e', 'l', 'l', 'o'), -3, 4); ---- [4, 5] [l, l] +query ?? +select array_slice(arrow_cast(make_array(1, 2, 3, 4, 5), 'LargeList(Int64)'), -2, 5), array_slice(arrow_cast(make_array('h', 'e', 'l', 'l', 'o'), 'LargeList(Utf8)'), -3, 4); +---- +[4, 5] [l, l] + # list_slice scalar function #23 (function alias `array_slice`) query ?? select list_slice(make_array(1, 2, 3, 4, 5), 2, 4), list_slice(make_array('h', 'e', 'l', 'l', 'o'), 1, 2); ---- [2, 3, 4] [h, e] +query ?? +select array_slice(arrow_cast(make_array(1, 2, 3, 4, 5), 'LargeList(Int64)'), 2, 4), array_slice(arrow_cast(make_array('h', 'e', 'l', 'l', 'o'), 'LargeList(Utf8)'), 1, 2); +---- +[2, 3, 4] [h, e] + # array_slice with columns query ? select array_slice(column1, column2, column3) from slices; @@ -1046,6 +1153,17 @@ select array_slice(column1, column2, column3) from slices; [41, 42, 43, 44, 45, 46] [55, 56, 57, 58, 59, 60] +query ? +select array_slice(arrow_cast(column1, 'LargeList(Int64)'), column2, column3) from slices; +---- +[] +[12, 13, 14, 15, 16] +[] +[] +[] +[41, 42, 43, 44, 45, 46] +[55, 56, 57, 58, 59, 60] + # TODO: support NULLS in output instead of `[]` # array_slice with columns and scalars query ??? @@ -1059,6 +1177,17 @@ select array_slice(make_array(1, 2, 3, 4, 5), column2, column3), array_slice(col [1, 2, 3, 4, 5] [43, 44, 45, 46] [41, 42, 43, 44, 45] [5] [, 54, 55, 56, 57, 58, 59, 60] [55] +query ??? +select array_slice(arrow_cast(make_array(1, 2, 3, 4, 5), 'LargeList(Int64)'), column2, column3), array_slice(arrow_cast(column1, 'LargeList(Int64)'), 3, column3), array_slice(arrow_cast(column1, 'LargeList(Int64)'), column2, 5) from slices; +---- +[1] [] [, 2, 3, 4, 5] +[] [13, 14, 15, 16] [12, 13, 14, 15] +[] [] [21, 22, 23, , 25] +[] [33] [] +[4, 5] [] [] +[1, 2, 3, 4, 5] [43, 44, 45, 46] [41, 42, 43, 44, 45] +[5] [, 54, 55, 56, 57, 58, 59, 60] [55] + # make_array with nulls query ??????? select make_array(make_array('a','b'), null), From b287cda40fa906dbdf035fa6a4dabe485927f42d Mon Sep 17 00:00:00 2001 From: comphead Date: Sun, 17 Dec 2023 22:55:31 -0800 Subject: [PATCH 452/572] minor: fix to support scalars (#8559) * minor: fix to support scalars * Update datafusion/sql/src/expr/function.rs Co-authored-by: Andrew Lamb --------- Co-authored-by: Mustafa Akur <106137913+mustafasrepo@users.noreply.github.com> Co-authored-by: Andrew Lamb --- datafusion/sql/src/expr/function.rs | 3 ++ datafusion/sqllogictest/test_files/window.slt | 28 +++++++++++++++++++ 2 files changed, 31 insertions(+) diff --git a/datafusion/sql/src/expr/function.rs b/datafusion/sql/src/expr/function.rs index 73de4fa43907..3934d6701c63 100644 --- a/datafusion/sql/src/expr/function.rs +++ b/datafusion/sql/src/expr/function.rs @@ -90,6 +90,9 @@ impl<'a, S: ContextProvider> SqlToRel<'a, S> { let partition_by = window .partition_by .into_iter() + // ignore window spec PARTITION BY for scalar values + // as they do not change and thus do not generate new partitions + .filter(|e| !matches!(e, sqlparser::ast::Expr::Value { .. },)) .map(|e| self.sql_expr_to_logical_expr(e, schema, planner_context)) .collect::>>()?; let mut order_by = self.order_by_to_sort_expr( diff --git a/datafusion/sqllogictest/test_files/window.slt b/datafusion/sqllogictest/test_files/window.slt index 6198209aaac5..864f7dc0a47d 100644 --- a/datafusion/sqllogictest/test_files/window.slt +++ b/datafusion/sqllogictest/test_files/window.slt @@ -3794,8 +3794,36 @@ select a, 1 1 2 1 +# support scalar value in ORDER BY query I select rank() over (order by 1) rnk from (select 1 a union all select 2 a) x ---- 1 1 + +# support scalar value in both ORDER BY and PARTITION BY, RANK function +# TODO: fix the test, some issue in RANK +#query IIIIII +#select rank() over (partition by 1 order by 1) rnk, +# rank() over (partition by a, 1 order by 1) rnk1, +# rank() over (partition by a, 1 order by a, 1) rnk2, +# rank() over (partition by 1) rnk3, +# rank() over (partition by null) rnk4, +# rank() over (partition by 1, null, a) rnk5 +#from (select 1 a union all select 2 a) x +#---- +#1 1 1 1 1 1 +#1 1 1 1 1 1 + +# support scalar value in both ORDER BY and PARTITION BY, ROW_NUMBER function +query IIIIII +select row_number() over (partition by 1 order by 1) rn, + row_number() over (partition by a, 1 order by 1) rn1, + row_number() over (partition by a, 1 order by a, 1) rn2, + row_number() over (partition by 1) rn3, + row_number() over (partition by null) rn4, + row_number() over (partition by 1, null, a) rn5 +from (select 1 a union all select 2 a) x; +---- +1 1 1 1 1 1 +2 1 1 2 2 1 \ No newline at end of file From a71a76a996a32a0f068370940ebe475ec237b4ff Mon Sep 17 00:00:00 2001 From: Eduard Karacharov <13005055+korowa@users.noreply.github.com> Date: Mon, 18 Dec 2023 11:53:26 +0200 Subject: [PATCH 453/572] refactor: `HashJoinStream` state machine (#8538) * hash join state machine * StreamJoinStateResult to StatefulStreamResult * doc comments & naming & fmt * suggestions from code review Co-authored-by: Andrew Lamb * more review comments addressed * post-merge fixes --------- Co-authored-by: Andrew Lamb --- .../physical-plan/src/joins/hash_join.rs | 431 ++++++++++++------ .../src/joins/stream_join_utils.rs | 127 ++---- .../src/joins/symmetric_hash_join.rs | 25 +- datafusion/physical-plan/src/joins/utils.rs | 83 ++++ 4 files changed, 420 insertions(+), 246 deletions(-) diff --git a/datafusion/physical-plan/src/joins/hash_join.rs b/datafusion/physical-plan/src/joins/hash_join.rs index 4846d0a5e046..13ac06ee301c 100644 --- a/datafusion/physical-plan/src/joins/hash_join.rs +++ b/datafusion/physical-plan/src/joins/hash_join.rs @@ -28,7 +28,6 @@ use crate::joins::utils::{ calculate_join_output_ordering, get_final_indices_from_bit_map, need_produce_result_in_final, JoinHashMap, JoinHashMapType, }; -use crate::DisplayAs; use crate::{ coalesce_batches::concat_batches, coalesce_partitions::CoalescePartitionsExec, @@ -38,12 +37,13 @@ use crate::{ joins::utils::{ adjust_right_output_partitioning, build_join_schema, check_join_is_valid, estimate_join_statistics, partitioned_join_output_partitioning, - BuildProbeJoinMetrics, ColumnIndex, JoinFilter, JoinOn, + BuildProbeJoinMetrics, ColumnIndex, JoinFilter, JoinOn, StatefulStreamResult, }, metrics::{ExecutionPlanMetricsSet, MetricsSet}, DisplayFormatType, Distribution, ExecutionPlan, Partitioning, PhysicalExpr, RecordBatchStream, SendableRecordBatchStream, Statistics, }; +use crate::{handle_state, DisplayAs}; use super::{ utils::{OnceAsync, OnceFut}, @@ -618,15 +618,14 @@ impl ExecutionPlan for HashJoinExec { on_right, filter: self.filter.clone(), join_type: self.join_type, - left_fut, - visited_left_side: None, right: right_stream, column_indices: self.column_indices.clone(), random_state: self.random_state.clone(), join_metrics, null_equals_null: self.null_equals_null, - is_exhausted: false, reservation, + state: HashJoinStreamState::WaitBuildSide, + build_side: BuildSide::Initial(BuildSideInitialState { left_fut }), })) } @@ -789,6 +788,104 @@ where Ok(()) } +/// Represents build-side of hash join. +enum BuildSide { + /// Indicates that build-side not collected yet + Initial(BuildSideInitialState), + /// Indicates that build-side data has been collected + Ready(BuildSideReadyState), +} + +/// Container for BuildSide::Initial related data +struct BuildSideInitialState { + /// Future for building hash table from build-side input + left_fut: OnceFut, +} + +/// Container for BuildSide::Ready related data +struct BuildSideReadyState { + /// Collected build-side data + left_data: Arc, + /// Which build-side rows have been matched while creating output. + /// For some OUTER joins, we need to know which rows have not been matched + /// to produce the correct output. + visited_left_side: BooleanBufferBuilder, +} + +impl BuildSide { + /// Tries to extract BuildSideInitialState from BuildSide enum. + /// Returns an error if state is not Initial. + fn try_as_initial_mut(&mut self) -> Result<&mut BuildSideInitialState> { + match self { + BuildSide::Initial(state) => Ok(state), + _ => internal_err!("Expected build side in initial state"), + } + } + + /// Tries to extract BuildSideReadyState from BuildSide enum. + /// Returns an error if state is not Ready. + fn try_as_ready(&self) -> Result<&BuildSideReadyState> { + match self { + BuildSide::Ready(state) => Ok(state), + _ => internal_err!("Expected build side in ready state"), + } + } + + /// Tries to extract BuildSideReadyState from BuildSide enum. + /// Returns an error if state is not Ready. + fn try_as_ready_mut(&mut self) -> Result<&mut BuildSideReadyState> { + match self { + BuildSide::Ready(state) => Ok(state), + _ => internal_err!("Expected build side in ready state"), + } + } +} + +/// Represents state of HashJoinStream +/// +/// Expected state transitions performed by HashJoinStream are: +/// +/// ```text +/// +/// WaitBuildSide +/// │ +/// ▼ +/// ┌─► FetchProbeBatch ───► ExhaustedProbeSide ───► Completed +/// │ │ +/// │ ▼ +/// └─ ProcessProbeBatch +/// +/// ``` +enum HashJoinStreamState { + /// Initial state for HashJoinStream indicating that build-side data not collected yet + WaitBuildSide, + /// Indicates that build-side has been collected, and stream is ready for fetching probe-side + FetchProbeBatch, + /// Indicates that non-empty batch has been fetched from probe-side, and is ready to be processed + ProcessProbeBatch(ProcessProbeBatchState), + /// Indicates that probe-side has been fully processed + ExhaustedProbeSide, + /// Indicates that HashJoinStream execution is completed + Completed, +} + +/// Container for HashJoinStreamState::ProcessProbeBatch related data +struct ProcessProbeBatchState { + /// Current probe-side batch + batch: RecordBatch, +} + +impl HashJoinStreamState { + /// Tries to extract ProcessProbeBatchState from HashJoinStreamState enum. + /// Returns an error if state is not ProcessProbeBatchState. + fn try_as_process_probe_batch(&self) -> Result<&ProcessProbeBatchState> { + match self { + HashJoinStreamState::ProcessProbeBatch(state) => Ok(state), + _ => internal_err!("Expected hash join stream in ProcessProbeBatch state"), + } + } +} + /// [`Stream`] for [`HashJoinExec`] that does the actual join. /// /// This stream: @@ -808,20 +905,10 @@ struct HashJoinStream { filter: Option, /// type of the join (left, right, semi, etc) join_type: JoinType, - /// future which builds hash table from left side - left_fut: OnceFut, - /// Which left (probe) side rows have been matches while creating output. - /// For some OUTER joins, we need to know which rows have not been matched - /// to produce the correct output. - visited_left_side: Option, /// right (probe) input right: SendableRecordBatchStream, /// Random state used for hashing initialization random_state: RandomState, - /// The join output is complete. For outer joins, this is used to - /// distinguish when the input stream is exhausted and when any unmatched - /// rows are output. - is_exhausted: bool, /// Metrics join_metrics: BuildProbeJoinMetrics, /// Information of index and left / right placement of columns @@ -830,6 +917,10 @@ struct HashJoinStream { null_equals_null: bool, /// Memory reservation reservation: MemoryReservation, + /// State of the stream + state: HashJoinStreamState, + /// Build side + build_side: BuildSide, } impl RecordBatchStream for HashJoinStream { @@ -1069,19 +1160,44 @@ impl HashJoinStream { &mut self, cx: &mut std::task::Context<'_>, ) -> Poll>> { + loop { + return match self.state { + HashJoinStreamState::WaitBuildSide => { + handle_state!(ready!(self.collect_build_side(cx))) + } + HashJoinStreamState::FetchProbeBatch => { + handle_state!(ready!(self.fetch_probe_batch(cx))) + } + HashJoinStreamState::ProcessProbeBatch(_) => { + handle_state!(self.process_probe_batch()) + } + HashJoinStreamState::ExhaustedProbeSide => { + handle_state!(self.process_unmatched_build_batch()) + } + HashJoinStreamState::Completed => Poll::Ready(None), + }; + } + } + + /// Collects build-side data by polling `OnceFut` future from initialized build-side + /// + /// Updates build-side to `Ready`, and state to `FetchProbeSide` + fn collect_build_side( + &mut self, + cx: &mut std::task::Context<'_>, + ) -> Poll>>> { let build_timer = self.join_metrics.build_time.timer(); // build hash table from left (build) side, if not yet done - let left_data = match ready!(self.left_fut.get(cx)) { - Ok(left_data) => left_data, - Err(e) => return Poll::Ready(Some(Err(e))), - }; + let left_data = ready!(self + .build_side + .try_as_initial_mut()? + .left_fut + .get_shared(cx))?; build_timer.done(); // Reserving memory for visited_left_side bitmap in case it hasn't been initialized yet // and join_type requires to store it - if self.visited_left_side.is_none() - && need_produce_result_in_final(self.join_type) - { + if need_produce_result_in_final(self.join_type) { // TODO: Replace `ceil` wrapper with stable `div_cell` after // https://github.com/rust-lang/rust/issues/88581 let visited_bitmap_size = bit_util::ceil(left_data.num_rows(), 8); @@ -1089,124 +1205,167 @@ impl HashJoinStream { self.join_metrics.build_mem_used.add(visited_bitmap_size); } - let visited_left_side = self.visited_left_side.get_or_insert_with(|| { + let visited_left_side = if need_produce_result_in_final(self.join_type) { let num_rows = left_data.num_rows(); - if need_produce_result_in_final(self.join_type) { - // Some join types need to track which row has be matched or unmatched: - // `left semi` join: need to use the bitmap to produce the matched row in the left side - // `left` join: need to use the bitmap to produce the unmatched row in the left side with null - // `left anti` join: need to use the bitmap to produce the unmatched row in the left side - // `full` join: need to use the bitmap to produce the unmatched row in the left side with null - let mut buffer = BooleanBufferBuilder::new(num_rows); - buffer.append_n(num_rows, false); - buffer - } else { - BooleanBufferBuilder::new(0) - } + // Some join types need to track which row has be matched or unmatched: + // `left semi` join: need to use the bitmap to produce the matched row in the left side + // `left` join: need to use the bitmap to produce the unmatched row in the left side with null + // `left anti` join: need to use the bitmap to produce the unmatched row in the left side + // `full` join: need to use the bitmap to produce the unmatched row in the left side with null + let mut buffer = BooleanBufferBuilder::new(num_rows); + buffer.append_n(num_rows, false); + buffer + } else { + BooleanBufferBuilder::new(0) + }; + + self.state = HashJoinStreamState::FetchProbeBatch; + self.build_side = BuildSide::Ready(BuildSideReadyState { + left_data, + visited_left_side, }); + + Poll::Ready(Ok(StatefulStreamResult::Continue)) + } + + /// Fetches next batch from probe-side + /// + /// If non-empty batch has been fetched, updates state to `ProcessProbeBatchState`, + /// otherwise updates state to `ExhaustedProbeSide` + fn fetch_probe_batch( + &mut self, + cx: &mut std::task::Context<'_>, + ) -> Poll>>> { + match ready!(self.right.poll_next_unpin(cx)) { + None => { + self.state = HashJoinStreamState::ExhaustedProbeSide; + } + Some(Ok(batch)) => { + self.state = + HashJoinStreamState::ProcessProbeBatch(ProcessProbeBatchState { + batch, + }); + } + Some(Err(err)) => return Poll::Ready(Err(err)), + }; + + Poll::Ready(Ok(StatefulStreamResult::Continue)) + } + + /// Joins current probe batch with build-side data and produces batch with matched output + /// + /// Updates state to `FetchProbeBatch` + fn process_probe_batch( + &mut self, + ) -> Result>> { + let state = self.state.try_as_process_probe_batch()?; + let build_side = self.build_side.try_as_ready_mut()?; + + self.join_metrics.input_batches.add(1); + self.join_metrics.input_rows.add(state.batch.num_rows()); + let timer = self.join_metrics.join_time.timer(); + let mut hashes_buffer = vec![]; - // get next right (probe) input batch - self.right - .poll_next_unpin(cx) - .map(|maybe_batch| match maybe_batch { - // one right batch in the join loop - Some(Ok(batch)) => { - self.join_metrics.input_batches.add(1); - self.join_metrics.input_rows.add(batch.num_rows()); - let timer = self.join_metrics.join_time.timer(); - - // get the matched two indices for the on condition - let left_right_indices = build_equal_condition_join_indices( - left_data.hash_map(), - left_data.batch(), - &batch, - &self.on_left, - &self.on_right, - &self.random_state, - self.null_equals_null, - &mut hashes_buffer, - self.filter.as_ref(), - JoinSide::Left, - None, - ); - - let result = match left_right_indices { - Ok((left_side, right_side)) => { - // set the left bitmap - // and only left, full, left semi, left anti need the left bitmap - if need_produce_result_in_final(self.join_type) { - left_side.iter().flatten().for_each(|x| { - visited_left_side.set_bit(x as usize, true); - }); - } - - // adjust the two side indices base on the join type - let (left_side, right_side) = adjust_indices_by_join_type( - left_side, - right_side, - batch.num_rows(), - self.join_type, - ); - - let result = build_batch_from_indices( - &self.schema, - left_data.batch(), - &batch, - &left_side, - &right_side, - &self.column_indices, - JoinSide::Left, - ); - self.join_metrics.output_batches.add(1); - self.join_metrics.output_rows.add(batch.num_rows()); - Some(result) - } - Err(err) => Some(exec_err!( - "Fail to build join indices in HashJoinExec, error:{err}" - )), - }; - timer.done(); - result - } - None => { - let timer = self.join_metrics.join_time.timer(); - if need_produce_result_in_final(self.join_type) && !self.is_exhausted - { - // use the global left bitmap to produce the left indices and right indices - let (left_side, right_side) = get_final_indices_from_bit_map( - visited_left_side, - self.join_type, - ); - let empty_right_batch = - RecordBatch::new_empty(self.right.schema()); - // use the left and right indices to produce the batch result - let result = build_batch_from_indices( - &self.schema, - left_data.batch(), - &empty_right_batch, - &left_side, - &right_side, - &self.column_indices, - JoinSide::Left, - ); - - if let Ok(ref batch) = result { - self.join_metrics.input_batches.add(1); - self.join_metrics.input_rows.add(batch.num_rows()); - - self.join_metrics.output_batches.add(1); - self.join_metrics.output_rows.add(batch.num_rows()); - } - timer.done(); - self.is_exhausted = true; - Some(result) - } else { - // end of the join loop - None - } + // get the matched two indices for the on condition + let left_right_indices = build_equal_condition_join_indices( + build_side.left_data.hash_map(), + build_side.left_data.batch(), + &state.batch, + &self.on_left, + &self.on_right, + &self.random_state, + self.null_equals_null, + &mut hashes_buffer, + self.filter.as_ref(), + JoinSide::Left, + None, + ); + + let result = match left_right_indices { + Ok((left_side, right_side)) => { + // set the left bitmap + // and only left, full, left semi, left anti need the left bitmap + if need_produce_result_in_final(self.join_type) { + left_side.iter().flatten().for_each(|x| { + build_side.visited_left_side.set_bit(x as usize, true); + }); } - Some(err) => Some(err), - }) + + // adjust the two side indices base on the join type + let (left_side, right_side) = adjust_indices_by_join_type( + left_side, + right_side, + state.batch.num_rows(), + self.join_type, + ); + + let result = build_batch_from_indices( + &self.schema, + build_side.left_data.batch(), + &state.batch, + &left_side, + &right_side, + &self.column_indices, + JoinSide::Left, + ); + self.join_metrics.output_batches.add(1); + self.join_metrics.output_rows.add(state.batch.num_rows()); + result + } + Err(err) => { + exec_err!("Fail to build join indices in HashJoinExec, error:{err}") + } + }; + timer.done(); + + self.state = HashJoinStreamState::FetchProbeBatch; + + Ok(StatefulStreamResult::Ready(Some(result?))) + } + + /// Processes unmatched build-side rows for certain join types and produces output batch + /// + /// Updates state to `Completed` + fn process_unmatched_build_batch( + &mut self, + ) -> Result>> { + let timer = self.join_metrics.join_time.timer(); + + if !need_produce_result_in_final(self.join_type) { + self.state = HashJoinStreamState::Completed; + + return Ok(StatefulStreamResult::Continue); + } + + let build_side = self.build_side.try_as_ready()?; + + // use the global left bitmap to produce the left indices and right indices + let (left_side, right_side) = + get_final_indices_from_bit_map(&build_side.visited_left_side, self.join_type); + let empty_right_batch = RecordBatch::new_empty(self.right.schema()); + // use the left and right indices to produce the batch result + let result = build_batch_from_indices( + &self.schema, + build_side.left_data.batch(), + &empty_right_batch, + &left_side, + &right_side, + &self.column_indices, + JoinSide::Left, + ); + + if let Ok(ref batch) = result { + self.join_metrics.input_batches.add(1); + self.join_metrics.input_rows.add(batch.num_rows()); + + self.join_metrics.output_batches.add(1); + self.join_metrics.output_rows.add(batch.num_rows()); + } + timer.done(); + + self.state = HashJoinStreamState::Completed; + + Ok(StatefulStreamResult::Ready(Some(result?))) } } diff --git a/datafusion/physical-plan/src/joins/stream_join_utils.rs b/datafusion/physical-plan/src/joins/stream_join_utils.rs index 2f74bd1c4bb2..64a976a1e39f 100644 --- a/datafusion/physical-plan/src/joins/stream_join_utils.rs +++ b/datafusion/physical-plan/src/joins/stream_join_utils.rs @@ -23,9 +23,9 @@ use std::sync::Arc; use std::task::{Context, Poll}; use std::usize; -use crate::joins::utils::{JoinFilter, JoinHashMapType}; +use crate::joins::utils::{JoinFilter, JoinHashMapType, StatefulStreamResult}; use crate::metrics::{ExecutionPlanMetricsSet, MetricBuilder}; -use crate::{handle_async_state, metrics}; +use crate::{handle_async_state, handle_state, metrics}; use arrow::compute::concat_batches; use arrow_array::{ArrowPrimitiveType, NativeAdapter, PrimitiveArray, RecordBatch}; @@ -624,73 +624,6 @@ pub fn record_visited_indices( } } -/// The `handle_state` macro is designed to process the result of a state-changing -/// operation, typically encountered in implementations of `EagerJoinStream`. It -/// operates on a `StreamJoinStateResult` by matching its variants and executing -/// corresponding actions. This macro is used to streamline code that deals with -/// state transitions, reducing boilerplate and improving readability. -/// -/// # Cases -/// -/// - `Ok(StreamJoinStateResult::Continue)`: Continues the loop, indicating the -/// stream join operation should proceed to the next step. -/// - `Ok(StreamJoinStateResult::Ready(result))`: Returns a `Poll::Ready` with the -/// result, either yielding a value or indicating the stream is awaiting more -/// data. -/// - `Err(e)`: Returns a `Poll::Ready` containing an error, signaling an issue -/// during the stream join operation. -/// -/// # Arguments -/// -/// * `$match_case`: An expression that evaluates to a `Result>`. -#[macro_export] -macro_rules! handle_state { - ($match_case:expr) => { - match $match_case { - Ok(StreamJoinStateResult::Continue) => continue, - Ok(StreamJoinStateResult::Ready(result)) => { - Poll::Ready(Ok(result).transpose()) - } - Err(e) => Poll::Ready(Some(Err(e))), - } - }; -} - -/// The `handle_async_state` macro adapts the `handle_state` macro for use in -/// asynchronous operations, particularly when dealing with `Poll` results within -/// async traits like `EagerJoinStream`. It polls the asynchronous state-changing -/// function using `poll_unpin` and then passes the result to `handle_state` for -/// further processing. -/// -/// # Arguments -/// -/// * `$state_func`: An async function or future that returns a -/// `Result>`. -/// * `$cx`: The context to be passed for polling, usually of type `&mut Context`. -/// -#[macro_export] -macro_rules! handle_async_state { - ($state_func:expr, $cx:expr) => { - $crate::handle_state!(ready!($state_func.poll_unpin($cx))) - }; -} - -/// Represents the result of a stateful operation on `EagerJoinStream`. -/// -/// This enumueration indicates whether the state produced a result that is -/// ready for use (`Ready`) or if the operation requires continuation (`Continue`). -/// -/// Variants: -/// - `Ready(T)`: Indicates that the operation is complete with a result of type `T`. -/// - `Continue`: Indicates that the operation is not yet complete and requires further -/// processing or more data. When this variant is returned, it typically means that the -/// current invocation of the state did not produce a final result, and the operation -/// should be invoked again later with more data and possibly with a different state. -pub enum StreamJoinStateResult { - Ready(T), - Continue, -} - /// Represents the various states of an eager join stream operation. /// /// This enum is used to track the current state of streaming during a join @@ -819,14 +752,14 @@ pub trait EagerJoinStream { /// /// # Returns /// - /// * `Result>>` - The state result after pulling the batch. + /// * `Result>>` - The state result after pulling the batch. async fn fetch_next_from_right_stream( &mut self, - ) -> Result>> { + ) -> Result>> { match self.right_stream().next().await { Some(Ok(batch)) => { if batch.num_rows() == 0 { - return Ok(StreamJoinStateResult::Continue); + return Ok(StatefulStreamResult::Continue); } self.set_state(EagerJoinStreamState::PullLeft); @@ -835,7 +768,7 @@ pub trait EagerJoinStream { Some(Err(e)) => Err(e), None => { self.set_state(EagerJoinStreamState::RightExhausted); - Ok(StreamJoinStateResult::Continue) + Ok(StatefulStreamResult::Continue) } } } @@ -848,14 +781,14 @@ pub trait EagerJoinStream { /// /// # Returns /// - /// * `Result>>` - The state result after pulling the batch. + /// * `Result>>` - The state result after pulling the batch. async fn fetch_next_from_left_stream( &mut self, - ) -> Result>> { + ) -> Result>> { match self.left_stream().next().await { Some(Ok(batch)) => { if batch.num_rows() == 0 { - return Ok(StreamJoinStateResult::Continue); + return Ok(StatefulStreamResult::Continue); } self.set_state(EagerJoinStreamState::PullRight); self.process_batch_from_left(batch) @@ -863,7 +796,7 @@ pub trait EagerJoinStream { Some(Err(e)) => Err(e), None => { self.set_state(EagerJoinStreamState::LeftExhausted); - Ok(StreamJoinStateResult::Continue) + Ok(StatefulStreamResult::Continue) } } } @@ -877,14 +810,14 @@ pub trait EagerJoinStream { /// /// # Returns /// - /// * `Result>>` - The state result after checking the exhaustion state. + /// * `Result>>` - The state result after checking the exhaustion state. async fn handle_right_stream_end( &mut self, - ) -> Result>> { + ) -> Result>> { match self.left_stream().next().await { Some(Ok(batch)) => { if batch.num_rows() == 0 { - return Ok(StreamJoinStateResult::Continue); + return Ok(StatefulStreamResult::Continue); } self.process_batch_after_right_end(batch) } @@ -893,7 +826,7 @@ pub trait EagerJoinStream { self.set_state(EagerJoinStreamState::BothExhausted { final_result: false, }); - Ok(StreamJoinStateResult::Continue) + Ok(StatefulStreamResult::Continue) } } } @@ -907,14 +840,14 @@ pub trait EagerJoinStream { /// /// # Returns /// - /// * `Result>>` - The state result after checking the exhaustion state. + /// * `Result>>` - The state result after checking the exhaustion state. async fn handle_left_stream_end( &mut self, - ) -> Result>> { + ) -> Result>> { match self.right_stream().next().await { Some(Ok(batch)) => { if batch.num_rows() == 0 { - return Ok(StreamJoinStateResult::Continue); + return Ok(StatefulStreamResult::Continue); } self.process_batch_after_left_end(batch) } @@ -923,7 +856,7 @@ pub trait EagerJoinStream { self.set_state(EagerJoinStreamState::BothExhausted { final_result: false, }); - Ok(StreamJoinStateResult::Continue) + Ok(StatefulStreamResult::Continue) } } } @@ -936,10 +869,10 @@ pub trait EagerJoinStream { /// /// # Returns /// - /// * `Result>>` - The state result after both streams are exhausted. + /// * `Result>>` - The state result after both streams are exhausted. fn prepare_for_final_results_after_exhaustion( &mut self, - ) -> Result>> { + ) -> Result>> { self.set_state(EagerJoinStreamState::BothExhausted { final_result: true }); self.process_batches_before_finalization() } @@ -952,11 +885,11 @@ pub trait EagerJoinStream { /// /// # Returns /// - /// * `Result>>` - The state result after processing the batch. + /// * `Result>>` - The state result after processing the batch. fn process_batch_from_right( &mut self, batch: RecordBatch, - ) -> Result>>; + ) -> Result>>; /// Handles a pulled batch from the left stream. /// @@ -966,11 +899,11 @@ pub trait EagerJoinStream { /// /// # Returns /// - /// * `Result>>` - The state result after processing the batch. + /// * `Result>>` - The state result after processing the batch. fn process_batch_from_left( &mut self, batch: RecordBatch, - ) -> Result>>; + ) -> Result>>; /// Handles the situation when only the left stream is exhausted. /// @@ -980,11 +913,11 @@ pub trait EagerJoinStream { /// /// # Returns /// - /// * `Result>>` - The state result after the left stream is exhausted. + /// * `Result>>` - The state result after the left stream is exhausted. fn process_batch_after_left_end( &mut self, right_batch: RecordBatch, - ) -> Result>>; + ) -> Result>>; /// Handles the situation when only the right stream is exhausted. /// @@ -994,20 +927,20 @@ pub trait EagerJoinStream { /// /// # Returns /// - /// * `Result>>` - The state result after the right stream is exhausted. + /// * `Result>>` - The state result after the right stream is exhausted. fn process_batch_after_right_end( &mut self, left_batch: RecordBatch, - ) -> Result>>; + ) -> Result>>; /// Handles the final state after both streams are exhausted. /// /// # Returns /// - /// * `Result>>` - The final state result after processing. + /// * `Result>>` - The final state result after processing. fn process_batches_before_finalization( &mut self, - ) -> Result>>; + ) -> Result>>; /// Provides mutable access to the right stream. /// diff --git a/datafusion/physical-plan/src/joins/symmetric_hash_join.rs b/datafusion/physical-plan/src/joins/symmetric_hash_join.rs index 00a7f23ebae7..b9101b57c3e5 100644 --- a/datafusion/physical-plan/src/joins/symmetric_hash_join.rs +++ b/datafusion/physical-plan/src/joins/symmetric_hash_join.rs @@ -38,12 +38,11 @@ use crate::joins::stream_join_utils::{ convert_sort_expr_with_filter_schema, get_pruning_anti_indices, get_pruning_semi_indices, record_visited_indices, EagerJoinStream, EagerJoinStreamState, PruningJoinHashMap, SortedFilterExpr, StreamJoinMetrics, - StreamJoinStateResult, }; use crate::joins::utils::{ build_batch_from_indices, build_join_schema, check_join_is_valid, partitioned_join_output_partitioning, prepare_sorted_exprs, ColumnIndex, JoinFilter, - JoinOn, + JoinOn, StatefulStreamResult, }; use crate::{ expressions::{Column, PhysicalSortExpr}, @@ -956,13 +955,13 @@ impl EagerJoinStream for SymmetricHashJoinStream { fn process_batch_from_right( &mut self, batch: RecordBatch, - ) -> Result>> { + ) -> Result>> { self.perform_join_for_given_side(batch, JoinSide::Right) .map(|maybe_batch| { if maybe_batch.is_some() { - StreamJoinStateResult::Ready(maybe_batch) + StatefulStreamResult::Ready(maybe_batch) } else { - StreamJoinStateResult::Continue + StatefulStreamResult::Continue } }) } @@ -970,13 +969,13 @@ impl EagerJoinStream for SymmetricHashJoinStream { fn process_batch_from_left( &mut self, batch: RecordBatch, - ) -> Result>> { + ) -> Result>> { self.perform_join_for_given_side(batch, JoinSide::Left) .map(|maybe_batch| { if maybe_batch.is_some() { - StreamJoinStateResult::Ready(maybe_batch) + StatefulStreamResult::Ready(maybe_batch) } else { - StreamJoinStateResult::Continue + StatefulStreamResult::Continue } }) } @@ -984,20 +983,20 @@ impl EagerJoinStream for SymmetricHashJoinStream { fn process_batch_after_left_end( &mut self, right_batch: RecordBatch, - ) -> Result>> { + ) -> Result>> { self.process_batch_from_right(right_batch) } fn process_batch_after_right_end( &mut self, left_batch: RecordBatch, - ) -> Result>> { + ) -> Result>> { self.process_batch_from_left(left_batch) } fn process_batches_before_finalization( &mut self, - ) -> Result>> { + ) -> Result>> { // Get the left side results: let left_result = build_side_determined_results( &self.left, @@ -1025,9 +1024,9 @@ impl EagerJoinStream for SymmetricHashJoinStream { // Update the metrics: self.metrics.output_batches.add(1); self.metrics.output_rows.add(batch.num_rows()); - return Ok(StreamJoinStateResult::Ready(result)); + return Ok(StatefulStreamResult::Ready(result)); } - Ok(StreamJoinStateResult::Continue) + Ok(StatefulStreamResult::Continue) } fn right_stream(&mut self) -> &mut SendableRecordBatchStream { diff --git a/datafusion/physical-plan/src/joins/utils.rs b/datafusion/physical-plan/src/joins/utils.rs index 5e01ca227cf5..eae65ce9c26b 100644 --- a/datafusion/physical-plan/src/joins/utils.rs +++ b/datafusion/physical-plan/src/joins/utils.rs @@ -849,6 +849,22 @@ impl OnceFut { ), } } + + /// Get shared reference to the result of the computation if it is ready, without consuming it + pub(crate) fn get_shared(&mut self, cx: &mut Context<'_>) -> Poll>> { + if let OnceFutState::Pending(fut) = &mut self.state { + let r = ready!(fut.poll_unpin(cx)); + self.state = OnceFutState::Ready(r); + } + + match &self.state { + OnceFutState::Pending(_) => unreachable!(), + OnceFutState::Ready(r) => Poll::Ready( + r.clone() + .map_err(|e| DataFusionError::External(Box::new(e))), + ), + } + } } /// Some type `join_type` of join need to maintain the matched indices bit map for the left side, and @@ -1277,6 +1293,73 @@ pub fn prepare_sorted_exprs( Ok((left_sorted_filter_expr, right_sorted_filter_expr, graph)) } +/// The `handle_state` macro is designed to process the result of a state-changing +/// operation, encountered e.g. in implementations of `EagerJoinStream`. It +/// operates on a `StatefulStreamResult` by matching its variants and executing +/// corresponding actions. This macro is used to streamline code that deals with +/// state transitions, reducing boilerplate and improving readability. +/// +/// # Cases +/// +/// - `Ok(StatefulStreamResult::Continue)`: Continues the loop, indicating the +/// stream join operation should proceed to the next step. +/// - `Ok(StatefulStreamResult::Ready(result))`: Returns a `Poll::Ready` with the +/// result, either yielding a value or indicating the stream is awaiting more +/// data. +/// - `Err(e)`: Returns a `Poll::Ready` containing an error, signaling an issue +/// during the stream join operation. +/// +/// # Arguments +/// +/// * `$match_case`: An expression that evaluates to a `Result>`. +#[macro_export] +macro_rules! handle_state { + ($match_case:expr) => { + match $match_case { + Ok(StatefulStreamResult::Continue) => continue, + Ok(StatefulStreamResult::Ready(result)) => { + Poll::Ready(Ok(result).transpose()) + } + Err(e) => Poll::Ready(Some(Err(e))), + } + }; +} + +/// The `handle_async_state` macro adapts the `handle_state` macro for use in +/// asynchronous operations, particularly when dealing with `Poll` results within +/// async traits like `EagerJoinStream`. It polls the asynchronous state-changing +/// function using `poll_unpin` and then passes the result to `handle_state` for +/// further processing. +/// +/// # Arguments +/// +/// * `$state_func`: An async function or future that returns a +/// `Result>`. +/// * `$cx`: The context to be passed for polling, usually of type `&mut Context`. +/// +#[macro_export] +macro_rules! handle_async_state { + ($state_func:expr, $cx:expr) => { + $crate::handle_state!(ready!($state_func.poll_unpin($cx))) + }; +} + +/// Represents the result of an operation on stateful join stream. +/// +/// This enumueration indicates whether the state produced a result that is +/// ready for use (`Ready`) or if the operation requires continuation (`Continue`). +/// +/// Variants: +/// - `Ready(T)`: Indicates that the operation is complete with a result of type `T`. +/// - `Continue`: Indicates that the operation is not yet complete and requires further +/// processing or more data. When this variant is returned, it typically means that the +/// current invocation of the state did not produce a final result, and the operation +/// should be invoked again later with more data and possibly with a different state. +pub enum StatefulStreamResult { + Ready(T), + Continue, +} + #[cfg(test)] mod tests { use std::pin::Pin; From a1e959d87a66da7060bd005b1993b824c0683a63 Mon Sep 17 00:00:00 2001 From: Raphael Taylor-Davies <1781103+tustvold@users.noreply.github.com> Date: Mon, 18 Dec 2023 10:55:49 +0000 Subject: [PATCH 454/572] Remove ListingTable and FileScanConfig Unbounded (#8540) (#8573) * Remove ListingTable and FileScanConfig Unbounded (#8540) * Fix substrait * Fix logical conflicts * Add deleted tests as ignored --------- Co-authored-by: Mustafa Akur --- datafusion-examples/examples/csv_opener.rs | 1 - datafusion-examples/examples/json_opener.rs | 1 - .../core/src/datasource/file_format/mod.rs | 1 - .../src/datasource/file_format/options.rs | 48 ++---- .../core/src/datasource/listing/table.rs | 152 ------------------ .../src/datasource/listing_table_factory.rs | 16 +- .../datasource/physical_plan/arrow_file.rs | 4 - .../core/src/datasource/physical_plan/avro.rs | 7 - .../core/src/datasource/physical_plan/csv.rs | 4 - .../physical_plan/file_scan_config.rs | 3 - .../datasource/physical_plan/file_stream.rs | 1 - .../core/src/datasource/physical_plan/json.rs | 8 - .../core/src/datasource/physical_plan/mod.rs | 4 - .../datasource/physical_plan/parquet/mod.rs | 4 - datafusion/core/src/execution/context/mod.rs | 11 +- .../combine_partial_final_agg.rs | 1 - .../enforce_distribution.rs | 5 - .../src/physical_optimizer/enforce_sorting.rs | 15 +- .../physical_optimizer/projection_pushdown.rs | 2 - .../replace_with_order_preserving_variants.rs | 92 ++++++----- .../core/src/physical_optimizer/test_utils.rs | 24 +-- datafusion/core/src/test/mod.rs | 3 - datafusion/core/src/test_util/mod.rs | 25 +-- datafusion/core/src/test_util/parquet.rs | 1 - .../core/tests/parquet/custom_reader.rs | 1 - datafusion/core/tests/parquet/page_pruning.rs | 1 - .../core/tests/parquet/schema_coercion.rs | 2 - datafusion/core/tests/sql/joins.rs | 42 ++--- .../proto/src/physical_plan/from_proto.rs | 1 - .../tests/cases/roundtrip_physical_plan.rs | 1 - .../substrait/src/physical_plan/consumer.rs | 1 - .../tests/cases/roundtrip_physical_plan.rs | 1 - 32 files changed, 102 insertions(+), 381 deletions(-) diff --git a/datafusion-examples/examples/csv_opener.rs b/datafusion-examples/examples/csv_opener.rs index 15fb07ded481..96753c8c5260 100644 --- a/datafusion-examples/examples/csv_opener.rs +++ b/datafusion-examples/examples/csv_opener.rs @@ -67,7 +67,6 @@ async fn main() -> Result<()> { limit: Some(5), table_partition_cols: vec![], output_ordering: vec![], - infinite_source: false, }; let result = diff --git a/datafusion-examples/examples/json_opener.rs b/datafusion-examples/examples/json_opener.rs index 1a3dbe57be75..ee33f969caa9 100644 --- a/datafusion-examples/examples/json_opener.rs +++ b/datafusion-examples/examples/json_opener.rs @@ -70,7 +70,6 @@ async fn main() -> Result<()> { limit: Some(5), table_partition_cols: vec![], output_ordering: vec![], - infinite_source: false, }; let result = diff --git a/datafusion/core/src/datasource/file_format/mod.rs b/datafusion/core/src/datasource/file_format/mod.rs index 7c2331548e5e..12c9fb91adb1 100644 --- a/datafusion/core/src/datasource/file_format/mod.rs +++ b/datafusion/core/src/datasource/file_format/mod.rs @@ -165,7 +165,6 @@ pub(crate) mod test_util { limit, table_partition_cols: vec![], output_ordering: vec![], - infinite_source: false, }, None, ) diff --git a/datafusion/core/src/datasource/file_format/options.rs b/datafusion/core/src/datasource/file_format/options.rs index 4c7557a4a9c0..d389137785ff 100644 --- a/datafusion/core/src/datasource/file_format/options.rs +++ b/datafusion/core/src/datasource/file_format/options.rs @@ -21,7 +21,6 @@ use std::sync::Arc; use arrow::datatypes::{DataType, Schema, SchemaRef}; use async_trait::async_trait; -use datafusion_common::{plan_err, DataFusionError}; use crate::datasource::file_format::arrow::ArrowFormat; use crate::datasource::file_format::file_compression_type::FileCompressionType; @@ -72,8 +71,6 @@ pub struct CsvReadOptions<'a> { pub table_partition_cols: Vec<(String, DataType)>, /// File compression type pub file_compression_type: FileCompressionType, - /// Flag indicating whether this file may be unbounded (as in a FIFO file). - pub infinite: bool, /// Indicates how the file is sorted pub file_sort_order: Vec>, } @@ -97,7 +94,6 @@ impl<'a> CsvReadOptions<'a> { file_extension: DEFAULT_CSV_EXTENSION, table_partition_cols: vec![], file_compression_type: FileCompressionType::UNCOMPRESSED, - infinite: false, file_sort_order: vec![], } } @@ -108,12 +104,6 @@ impl<'a> CsvReadOptions<'a> { self } - /// Configure mark_infinite setting - pub fn mark_infinite(mut self, infinite: bool) -> Self { - self.infinite = infinite; - self - } - /// Specify delimiter to use for CSV read pub fn delimiter(mut self, delimiter: u8) -> Self { self.delimiter = delimiter; @@ -324,8 +314,6 @@ pub struct AvroReadOptions<'a> { pub file_extension: &'a str, /// Partition Columns pub table_partition_cols: Vec<(String, DataType)>, - /// Flag indicating whether this file may be unbounded (as in a FIFO file). - pub infinite: bool, } impl<'a> Default for AvroReadOptions<'a> { @@ -334,7 +322,6 @@ impl<'a> Default for AvroReadOptions<'a> { schema: None, file_extension: DEFAULT_AVRO_EXTENSION, table_partition_cols: vec![], - infinite: false, } } } @@ -349,12 +336,6 @@ impl<'a> AvroReadOptions<'a> { self } - /// Configure mark_infinite setting - pub fn mark_infinite(mut self, infinite: bool) -> Self { - self.infinite = infinite; - self - } - /// Specify schema to use for AVRO read pub fn schema(mut self, schema: &'a Schema) -> Self { self.schema = Some(schema); @@ -466,21 +447,17 @@ pub trait ReadOptions<'a> { state: SessionState, table_path: ListingTableUrl, schema: Option<&'a Schema>, - infinite: bool, ) -> Result where 'a: 'async_trait, { - match (schema, infinite) { - (Some(s), _) => Ok(Arc::new(s.to_owned())), - (None, false) => Ok(self - .to_listing_options(config) - .infer_schema(&state, &table_path) - .await?), - (None, true) => { - plan_err!("Schema inference for infinite data sources is not supported.") - } + if let Some(s) = schema { + return Ok(Arc::new(s.to_owned())); } + + self.to_listing_options(config) + .infer_schema(&state, &table_path) + .await } } @@ -500,7 +477,6 @@ impl ReadOptions<'_> for CsvReadOptions<'_> { .with_target_partitions(config.target_partitions()) .with_table_partition_cols(self.table_partition_cols.clone()) .with_file_sort_order(self.file_sort_order.clone()) - .with_infinite_source(self.infinite) } async fn get_resolved_schema( @@ -509,7 +485,7 @@ impl ReadOptions<'_> for CsvReadOptions<'_> { state: SessionState, table_path: ListingTableUrl, ) -> Result { - self._get_resolved_schema(config, state, table_path, self.schema, self.infinite) + self._get_resolved_schema(config, state, table_path, self.schema) .await } } @@ -535,7 +511,7 @@ impl ReadOptions<'_> for ParquetReadOptions<'_> { state: SessionState, table_path: ListingTableUrl, ) -> Result { - self._get_resolved_schema(config, state, table_path, self.schema, false) + self._get_resolved_schema(config, state, table_path, self.schema) .await } } @@ -551,7 +527,6 @@ impl ReadOptions<'_> for NdJsonReadOptions<'_> { .with_file_extension(self.file_extension) .with_target_partitions(config.target_partitions()) .with_table_partition_cols(self.table_partition_cols.clone()) - .with_infinite_source(self.infinite) .with_file_sort_order(self.file_sort_order.clone()) } @@ -561,7 +536,7 @@ impl ReadOptions<'_> for NdJsonReadOptions<'_> { state: SessionState, table_path: ListingTableUrl, ) -> Result { - self._get_resolved_schema(config, state, table_path, self.schema, self.infinite) + self._get_resolved_schema(config, state, table_path, self.schema) .await } } @@ -575,7 +550,6 @@ impl ReadOptions<'_> for AvroReadOptions<'_> { .with_file_extension(self.file_extension) .with_target_partitions(config.target_partitions()) .with_table_partition_cols(self.table_partition_cols.clone()) - .with_infinite_source(self.infinite) } async fn get_resolved_schema( @@ -584,7 +558,7 @@ impl ReadOptions<'_> for AvroReadOptions<'_> { state: SessionState, table_path: ListingTableUrl, ) -> Result { - self._get_resolved_schema(config, state, table_path, self.schema, self.infinite) + self._get_resolved_schema(config, state, table_path, self.schema) .await } } @@ -606,7 +580,7 @@ impl ReadOptions<'_> for ArrowReadOptions<'_> { state: SessionState, table_path: ListingTableUrl, ) -> Result { - self._get_resolved_schema(config, state, table_path, self.schema, false) + self._get_resolved_schema(config, state, table_path, self.schema) .await } } diff --git a/datafusion/core/src/datasource/listing/table.rs b/datafusion/core/src/datasource/listing/table.rs index 0ce1b43fe456..4c13d9d443ca 100644 --- a/datafusion/core/src/datasource/listing/table.rs +++ b/datafusion/core/src/datasource/listing/table.rs @@ -246,11 +246,6 @@ pub struct ListingOptions { /// multiple equivalent orderings, the outer `Vec` will have a /// single element. pub file_sort_order: Vec>, - /// Infinite source means that the input is not guaranteed to end. - /// Currently, CSV, JSON, and AVRO formats are supported. - /// In order to support infinite inputs, DataFusion may adjust query - /// plans (e.g. joins) to run the given query in full pipelining mode. - pub infinite_source: bool, /// This setting when true indicates that the table is backed by a single file. /// Any inserts to the table may only append to this existing file. pub single_file: bool, @@ -274,30 +269,11 @@ impl ListingOptions { collect_stat: true, target_partitions: 1, file_sort_order: vec![], - infinite_source: false, single_file: false, file_type_write_options: None, } } - /// Set unbounded assumption on [`ListingOptions`] and returns self. - /// - /// ``` - /// use std::sync::Arc; - /// use datafusion::datasource::{listing::ListingOptions, file_format::csv::CsvFormat}; - /// use datafusion::prelude::SessionContext; - /// let ctx = SessionContext::new(); - /// let listing_options = ListingOptions::new(Arc::new( - /// CsvFormat::default() - /// )).with_infinite_source(true); - /// - /// assert_eq!(listing_options.infinite_source, true); - /// ``` - pub fn with_infinite_source(mut self, infinite_source: bool) -> Self { - self.infinite_source = infinite_source; - self - } - /// Set file extension on [`ListingOptions`] and returns self. /// /// ``` @@ -557,7 +533,6 @@ pub struct ListingTable { options: ListingOptions, definition: Option, collected_statistics: FileStatisticsCache, - infinite_source: bool, constraints: Constraints, column_defaults: HashMap, } @@ -587,7 +562,6 @@ impl ListingTable { for (part_col_name, part_col_type) in &options.table_partition_cols { builder.push(Field::new(part_col_name, part_col_type.clone(), false)); } - let infinite_source = options.infinite_source; let table = Self { table_paths: config.table_paths, @@ -596,7 +570,6 @@ impl ListingTable { options, definition: None, collected_statistics: Arc::new(DefaultFileStatisticsCache::default()), - infinite_source, constraints: Constraints::empty(), column_defaults: HashMap::new(), }; @@ -729,7 +702,6 @@ impl TableProvider for ListingTable { limit, output_ordering: self.try_create_output_ordering()?, table_partition_cols, - infinite_source: self.infinite_source, }, filters.as_ref(), ) @@ -943,7 +915,6 @@ impl ListingTable { #[cfg(test)] mod tests { use std::collections::HashMap; - use std::fs::File; use super::*; #[cfg(feature = "parquet")] @@ -955,7 +926,6 @@ mod tests { use crate::{ assert_batches_eq, datasource::file_format::avro::AvroFormat, - execution::options::ReadOptions, logical_expr::{col, lit}, test::{columns, object_store::register_test_store}, }; @@ -967,37 +937,8 @@ mod tests { use datafusion_common::{assert_contains, GetExt, ScalarValue}; use datafusion_expr::{BinaryExpr, LogicalPlanBuilder, Operator}; use datafusion_physical_expr::PhysicalSortExpr; - use rstest::*; use tempfile::TempDir; - /// It creates dummy file and checks if it can create unbounded input executors. - async fn unbounded_table_helper( - file_type: FileType, - listing_option: ListingOptions, - infinite_data: bool, - ) -> Result<()> { - let ctx = SessionContext::new(); - register_test_store( - &ctx, - &[(&format!("table/file{}", file_type.get_ext()), 100)], - ); - - let schema = Schema::new(vec![Field::new("a", DataType::Boolean, false)]); - - let table_path = ListingTableUrl::parse("test:///table/").unwrap(); - let config = ListingTableConfig::new(table_path) - .with_listing_options(listing_option) - .with_schema(Arc::new(schema)); - // Create a table - let table = ListingTable::try_new(config)?; - // Create executor from table - let source_exec = table.scan(&ctx.state(), None, &[], None).await?; - - assert_eq!(source_exec.unbounded_output(&[])?, infinite_data); - - Ok(()) - } - #[tokio::test] async fn read_single_file() -> Result<()> { let ctx = SessionContext::new(); @@ -1205,99 +1146,6 @@ mod tests { Ok(()) } - #[tokio::test] - async fn unbounded_csv_table_without_schema() -> Result<()> { - let tmp_dir = TempDir::new()?; - let file_path = tmp_dir.path().join("dummy.csv"); - File::create(file_path)?; - let ctx = SessionContext::new(); - let error = ctx - .register_csv( - "test", - tmp_dir.path().to_str().unwrap(), - CsvReadOptions::new().mark_infinite(true), - ) - .await - .unwrap_err(); - match error { - DataFusionError::Plan(_) => Ok(()), - val => Err(val), - } - } - - #[tokio::test] - async fn unbounded_json_table_without_schema() -> Result<()> { - let tmp_dir = TempDir::new()?; - let file_path = tmp_dir.path().join("dummy.json"); - File::create(file_path)?; - let ctx = SessionContext::new(); - let error = ctx - .register_json( - "test", - tmp_dir.path().to_str().unwrap(), - NdJsonReadOptions::default().mark_infinite(true), - ) - .await - .unwrap_err(); - match error { - DataFusionError::Plan(_) => Ok(()), - val => Err(val), - } - } - - #[tokio::test] - async fn unbounded_avro_table_without_schema() -> Result<()> { - let tmp_dir = TempDir::new()?; - let file_path = tmp_dir.path().join("dummy.avro"); - File::create(file_path)?; - let ctx = SessionContext::new(); - let error = ctx - .register_avro( - "test", - tmp_dir.path().to_str().unwrap(), - AvroReadOptions::default().mark_infinite(true), - ) - .await - .unwrap_err(); - match error { - DataFusionError::Plan(_) => Ok(()), - val => Err(val), - } - } - - #[rstest] - #[tokio::test] - async fn unbounded_csv_table( - #[values(true, false)] infinite_data: bool, - ) -> Result<()> { - let config = CsvReadOptions::new().mark_infinite(infinite_data); - let session_config = SessionConfig::new().with_target_partitions(1); - let listing_options = config.to_listing_options(&session_config); - unbounded_table_helper(FileType::CSV, listing_options, infinite_data).await - } - - #[rstest] - #[tokio::test] - async fn unbounded_json_table( - #[values(true, false)] infinite_data: bool, - ) -> Result<()> { - let config = NdJsonReadOptions::default().mark_infinite(infinite_data); - let session_config = SessionConfig::new().with_target_partitions(1); - let listing_options = config.to_listing_options(&session_config); - unbounded_table_helper(FileType::JSON, listing_options, infinite_data).await - } - - #[rstest] - #[tokio::test] - async fn unbounded_avro_table( - #[values(true, false)] infinite_data: bool, - ) -> Result<()> { - let config = AvroReadOptions::default().mark_infinite(infinite_data); - let session_config = SessionConfig::new().with_target_partitions(1); - let listing_options = config.to_listing_options(&session_config); - unbounded_table_helper(FileType::AVRO, listing_options, infinite_data).await - } - #[tokio::test] async fn test_assert_list_files_for_scan_grouping() -> Result<()> { // more expected partitions than files diff --git a/datafusion/core/src/datasource/listing_table_factory.rs b/datafusion/core/src/datasource/listing_table_factory.rs index a9d0c3a0099e..7c859ee988d5 100644 --- a/datafusion/core/src/datasource/listing_table_factory.rs +++ b/datafusion/core/src/datasource/listing_table_factory.rs @@ -133,21 +133,9 @@ impl TableProviderFactory for ListingTableFactory { (Some(schema), table_partition_cols) }; - // look for 'infinite' as an option - let infinite_source = cmd.unbounded; - let mut statement_options = StatementOptions::from(&cmd.options); // Extract ListingTable specific options if present or set default - let unbounded = if infinite_source { - statement_options.take_str_option("unbounded"); - infinite_source - } else { - statement_options - .take_bool_option("unbounded")? - .unwrap_or(false) - }; - let single_file = statement_options .take_bool_option("single_file")? .unwrap_or(false); @@ -159,6 +147,7 @@ impl TableProviderFactory for ListingTableFactory { } } statement_options.take_bool_option("create_local_path")?; + statement_options.take_str_option("unbounded"); let file_type = file_format.file_type(); @@ -207,8 +196,7 @@ impl TableProviderFactory for ListingTableFactory { .with_table_partition_cols(table_partition_cols) .with_file_sort_order(cmd.order_exprs.clone()) .with_single_file(single_file) - .with_write_options(file_type_writer_options) - .with_infinite_source(unbounded); + .with_write_options(file_type_writer_options); let resolved_schema = match provided_schema { None => options.infer_schema(state, &table_path).await?, diff --git a/datafusion/core/src/datasource/physical_plan/arrow_file.rs b/datafusion/core/src/datasource/physical_plan/arrow_file.rs index 30b55db28491..ae1e879d0da1 100644 --- a/datafusion/core/src/datasource/physical_plan/arrow_file.rs +++ b/datafusion/core/src/datasource/physical_plan/arrow_file.rs @@ -93,10 +93,6 @@ impl ExecutionPlan for ArrowExec { Partitioning::UnknownPartitioning(self.base_config.file_groups.len()) } - fn unbounded_output(&self, _: &[bool]) -> Result { - Ok(self.base_config().infinite_source) - } - fn output_ordering(&self) -> Option<&[PhysicalSortExpr]> { self.projected_output_ordering .first() diff --git a/datafusion/core/src/datasource/physical_plan/avro.rs b/datafusion/core/src/datasource/physical_plan/avro.rs index 885b4c5d3911..e448bf39f427 100644 --- a/datafusion/core/src/datasource/physical_plan/avro.rs +++ b/datafusion/core/src/datasource/physical_plan/avro.rs @@ -89,10 +89,6 @@ impl ExecutionPlan for AvroExec { Partitioning::UnknownPartitioning(self.base_config.file_groups.len()) } - fn unbounded_output(&self, _: &[bool]) -> Result { - Ok(self.base_config().infinite_source) - } - fn output_ordering(&self) -> Option<&[PhysicalSortExpr]> { self.projected_output_ordering .first() @@ -276,7 +272,6 @@ mod tests { limit: None, table_partition_cols: vec![], output_ordering: vec![], - infinite_source: false, }); assert_eq!(avro_exec.output_partitioning().partition_count(), 1); let mut results = avro_exec @@ -348,7 +343,6 @@ mod tests { limit: None, table_partition_cols: vec![], output_ordering: vec![], - infinite_source: false, }); assert_eq!(avro_exec.output_partitioning().partition_count(), 1); @@ -419,7 +413,6 @@ mod tests { limit: None, table_partition_cols: vec![Field::new("date", DataType::Utf8, false)], output_ordering: vec![], - infinite_source: false, }); assert_eq!(avro_exec.output_partitioning().partition_count(), 1); diff --git a/datafusion/core/src/datasource/physical_plan/csv.rs b/datafusion/core/src/datasource/physical_plan/csv.rs index 0eca37da139d..0c34d22e9fa9 100644 --- a/datafusion/core/src/datasource/physical_plan/csv.rs +++ b/datafusion/core/src/datasource/physical_plan/csv.rs @@ -146,10 +146,6 @@ impl ExecutionPlan for CsvExec { Partitioning::UnknownPartitioning(self.base_config.file_groups.len()) } - fn unbounded_output(&self, _: &[bool]) -> Result { - Ok(self.base_config().infinite_source) - } - /// See comments on `impl ExecutionPlan for ParquetExec`: output order can't be fn output_ordering(&self) -> Option<&[PhysicalSortExpr]> { self.projected_output_ordering diff --git a/datafusion/core/src/datasource/physical_plan/file_scan_config.rs b/datafusion/core/src/datasource/physical_plan/file_scan_config.rs index 89694ff28500..516755e4d293 100644 --- a/datafusion/core/src/datasource/physical_plan/file_scan_config.rs +++ b/datafusion/core/src/datasource/physical_plan/file_scan_config.rs @@ -99,8 +99,6 @@ pub struct FileScanConfig { pub table_partition_cols: Vec, /// All equivalent lexicographical orderings that describe the schema. pub output_ordering: Vec, - /// Indicates whether this plan may produce an infinite stream of records. - pub infinite_source: bool, } impl FileScanConfig { @@ -707,7 +705,6 @@ mod tests { statistics, table_partition_cols, output_ordering: vec![], - infinite_source: false, } } diff --git a/datafusion/core/src/datasource/physical_plan/file_stream.rs b/datafusion/core/src/datasource/physical_plan/file_stream.rs index a715f6e8e3cd..99fb088b66f4 100644 --- a/datafusion/core/src/datasource/physical_plan/file_stream.rs +++ b/datafusion/core/src/datasource/physical_plan/file_stream.rs @@ -667,7 +667,6 @@ mod tests { limit: self.limit, table_partition_cols: vec![], output_ordering: vec![], - infinite_source: false, }; let metrics_set = ExecutionPlanMetricsSet::new(); let file_stream = FileStream::new(&config, 0, self.opener, &metrics_set) diff --git a/datafusion/core/src/datasource/physical_plan/json.rs b/datafusion/core/src/datasource/physical_plan/json.rs index 9c3b523a652c..c74fd13e77aa 100644 --- a/datafusion/core/src/datasource/physical_plan/json.rs +++ b/datafusion/core/src/datasource/physical_plan/json.rs @@ -110,10 +110,6 @@ impl ExecutionPlan for NdJsonExec { Partitioning::UnknownPartitioning(self.base_config.file_groups.len()) } - fn unbounded_output(&self, _: &[bool]) -> Result { - Ok(self.base_config.infinite_source) - } - fn output_ordering(&self) -> Option<&[PhysicalSortExpr]> { self.projected_output_ordering .first() @@ -462,7 +458,6 @@ mod tests { limit: Some(3), table_partition_cols: vec![], output_ordering: vec![], - infinite_source: false, }, file_compression_type.to_owned(), ); @@ -541,7 +536,6 @@ mod tests { limit: Some(3), table_partition_cols: vec![], output_ordering: vec![], - infinite_source: false, }, file_compression_type.to_owned(), ); @@ -589,7 +583,6 @@ mod tests { limit: None, table_partition_cols: vec![], output_ordering: vec![], - infinite_source: false, }, file_compression_type.to_owned(), ); @@ -642,7 +635,6 @@ mod tests { limit: None, table_partition_cols: vec![], output_ordering: vec![], - infinite_source: false, }, file_compression_type.to_owned(), ); diff --git a/datafusion/core/src/datasource/physical_plan/mod.rs b/datafusion/core/src/datasource/physical_plan/mod.rs index 8e4dd5400b20..9d1c373aee7c 100644 --- a/datafusion/core/src/datasource/physical_plan/mod.rs +++ b/datafusion/core/src/datasource/physical_plan/mod.rs @@ -133,10 +133,6 @@ impl DisplayAs for FileScanConfig { write!(f, ", limit={limit}")?; } - if self.infinite_source { - write!(f, ", infinite_source=true")?; - } - if let Some(ordering) = orderings.first() { if !ordering.is_empty() { let start = if orderings.len() == 1 { diff --git a/datafusion/core/src/datasource/physical_plan/parquet/mod.rs b/datafusion/core/src/datasource/physical_plan/parquet/mod.rs index 2b10b05a273a..ade149da6991 100644 --- a/datafusion/core/src/datasource/physical_plan/parquet/mod.rs +++ b/datafusion/core/src/datasource/physical_plan/parquet/mod.rs @@ -882,7 +882,6 @@ mod tests { limit: None, table_partition_cols: vec![], output_ordering: vec![], - infinite_source: false, }, predicate, None, @@ -1539,7 +1538,6 @@ mod tests { limit: None, table_partition_cols: vec![], output_ordering: vec![], - infinite_source: false, }, None, None, @@ -1654,7 +1652,6 @@ mod tests { ), ], output_ordering: vec![], - infinite_source: false, }, None, None, @@ -1718,7 +1715,6 @@ mod tests { limit: None, table_partition_cols: vec![], output_ordering: vec![], - infinite_source: false, }, None, None, diff --git a/datafusion/core/src/execution/context/mod.rs b/datafusion/core/src/execution/context/mod.rs index 58a4f08341d6..8916fa814a4a 100644 --- a/datafusion/core/src/execution/context/mod.rs +++ b/datafusion/core/src/execution/context/mod.rs @@ -964,14 +964,9 @@ impl SessionContext { sql_definition: Option, ) -> Result<()> { let table_path = ListingTableUrl::parse(table_path)?; - let resolved_schema = match (provided_schema, options.infinite_source) { - (Some(s), _) => s, - (None, false) => options.infer_schema(&self.state(), &table_path).await?, - (None, true) => { - return plan_err!( - "Schema inference for infinite data sources is not supported." - ) - } + let resolved_schema = match provided_schema { + Some(s) => s, + None => options.infer_schema(&self.state(), &table_path).await?, }; let config = ListingTableConfig::new(table_path) .with_listing_options(options) diff --git a/datafusion/core/src/physical_optimizer/combine_partial_final_agg.rs b/datafusion/core/src/physical_optimizer/combine_partial_final_agg.rs index c50ea36b68ec..7359a6463059 100644 --- a/datafusion/core/src/physical_optimizer/combine_partial_final_agg.rs +++ b/datafusion/core/src/physical_optimizer/combine_partial_final_agg.rs @@ -257,7 +257,6 @@ mod tests { limit: None, table_partition_cols: vec![], output_ordering: vec![], - infinite_source: false, }, None, None, diff --git a/datafusion/core/src/physical_optimizer/enforce_distribution.rs b/datafusion/core/src/physical_optimizer/enforce_distribution.rs index 099759741a10..0aef126578f3 100644 --- a/datafusion/core/src/physical_optimizer/enforce_distribution.rs +++ b/datafusion/core/src/physical_optimizer/enforce_distribution.rs @@ -1775,7 +1775,6 @@ pub(crate) mod tests { limit: None, table_partition_cols: vec![], output_ordering, - infinite_source: false, }, None, None, @@ -1803,7 +1802,6 @@ pub(crate) mod tests { limit: None, table_partition_cols: vec![], output_ordering, - infinite_source: false, }, None, None, @@ -1825,7 +1823,6 @@ pub(crate) mod tests { limit: None, table_partition_cols: vec![], output_ordering, - infinite_source: false, }, false, b',', @@ -1856,7 +1853,6 @@ pub(crate) mod tests { limit: None, table_partition_cols: vec![], output_ordering, - infinite_source: false, }, false, b',', @@ -3957,7 +3953,6 @@ pub(crate) mod tests { limit: None, table_partition_cols: vec![], output_ordering: vec![], - infinite_source: false, }, false, b',', diff --git a/datafusion/core/src/physical_optimizer/enforce_sorting.rs b/datafusion/core/src/physical_optimizer/enforce_sorting.rs index 277404b301c4..c0e9b834e66f 100644 --- a/datafusion/core/src/physical_optimizer/enforce_sorting.rs +++ b/datafusion/core/src/physical_optimizer/enforce_sorting.rs @@ -2117,7 +2117,7 @@ mod tests { async fn test_with_lost_ordering_bounded() -> Result<()> { let schema = create_test_schema3()?; let sort_exprs = vec![sort_expr("a", &schema)]; - let source = csv_exec_sorted(&schema, sort_exprs, false); + let source = csv_exec_sorted(&schema, sort_exprs); let repartition_rr = repartition_exec(source); let repartition_hash = Arc::new(RepartitionExec::try_new( repartition_rr, @@ -2141,10 +2141,11 @@ mod tests { } #[tokio::test] + #[ignore] async fn test_with_lost_ordering_unbounded() -> Result<()> { let schema = create_test_schema3()?; let sort_exprs = vec![sort_expr("a", &schema)]; - let source = csv_exec_sorted(&schema, sort_exprs, true); + let source = csv_exec_sorted(&schema, sort_exprs); let repartition_rr = repartition_exec(source); let repartition_hash = Arc::new(RepartitionExec::try_new( repartition_rr, @@ -2171,10 +2172,12 @@ mod tests { } #[tokio::test] + #[ignore] async fn test_with_lost_ordering_unbounded_parallelize_off() -> Result<()> { let schema = create_test_schema3()?; let sort_exprs = vec![sort_expr("a", &schema)]; - let source = csv_exec_sorted(&schema, sort_exprs, true); + // Make source unbounded + let source = csv_exec_sorted(&schema, sort_exprs); let repartition_rr = repartition_exec(source); let repartition_hash = Arc::new(RepartitionExec::try_new( repartition_rr, @@ -2203,7 +2206,7 @@ mod tests { async fn test_do_not_pushdown_through_spm() -> Result<()> { let schema = create_test_schema3()?; let sort_exprs = vec![sort_expr("a", &schema), sort_expr("b", &schema)]; - let source = csv_exec_sorted(&schema, sort_exprs.clone(), false); + let source = csv_exec_sorted(&schema, sort_exprs.clone()); let repartition_rr = repartition_exec(source); let spm = sort_preserving_merge_exec(sort_exprs, repartition_rr); let physical_plan = sort_exec(vec![sort_expr("b", &schema)], spm); @@ -2224,7 +2227,7 @@ mod tests { async fn test_pushdown_through_spm() -> Result<()> { let schema = create_test_schema3()?; let sort_exprs = vec![sort_expr("a", &schema), sort_expr("b", &schema)]; - let source = csv_exec_sorted(&schema, sort_exprs.clone(), false); + let source = csv_exec_sorted(&schema, sort_exprs.clone()); let repartition_rr = repartition_exec(source); let spm = sort_preserving_merge_exec(sort_exprs, repartition_rr); let physical_plan = sort_exec( @@ -2252,7 +2255,7 @@ mod tests { async fn test_window_multi_layer_requirement() -> Result<()> { let schema = create_test_schema3()?; let sort_exprs = vec![sort_expr("a", &schema), sort_expr("b", &schema)]; - let source = csv_exec_sorted(&schema, vec![], false); + let source = csv_exec_sorted(&schema, vec![]); let sort = sort_exec(sort_exprs.clone(), source); let repartition = repartition_exec(sort); let repartition = spr_repartition_exec(repartition); diff --git a/datafusion/core/src/physical_optimizer/projection_pushdown.rs b/datafusion/core/src/physical_optimizer/projection_pushdown.rs index 664afbe822ff..7e1312dad23e 100644 --- a/datafusion/core/src/physical_optimizer/projection_pushdown.rs +++ b/datafusion/core/src/physical_optimizer/projection_pushdown.rs @@ -1541,7 +1541,6 @@ mod tests { limit: None, table_partition_cols: vec![], output_ordering: vec![vec![]], - infinite_source: false, }, false, 0, @@ -1568,7 +1567,6 @@ mod tests { limit: None, table_partition_cols: vec![], output_ordering: vec![vec![]], - infinite_source: false, }, false, 0, diff --git a/datafusion/core/src/physical_optimizer/replace_with_order_preserving_variants.rs b/datafusion/core/src/physical_optimizer/replace_with_order_preserving_variants.rs index af45df7d8474..41f2b39978a4 100644 --- a/datafusion/core/src/physical_optimizer/replace_with_order_preserving_variants.rs +++ b/datafusion/core/src/physical_optimizer/replace_with_order_preserving_variants.rs @@ -350,7 +350,7 @@ mod tests { async fn test_replace_multiple_input_repartition_1() -> Result<()> { let schema = create_test_schema()?; let sort_exprs = vec![sort_expr("a", &schema)]; - let source = csv_exec_sorted(&schema, sort_exprs, true); + let source = csv_exec_sorted(&schema, sort_exprs); let repartition = repartition_exec_hash(repartition_exec_round_robin(source)); let sort = sort_exec(vec![sort_expr("a", &schema)], repartition, true); @@ -362,15 +362,15 @@ mod tests { " SortExec: expr=[a@0 ASC NULLS LAST]", " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8", " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", - " CsvExec: file_groups={1 group: [[file_path]]}, projection=[a, c, d], infinite_source=true, output_ordering=[a@0 ASC NULLS LAST], has_header=true", + " CsvExec: file_groups={1 group: [[file_path]]}, projection=[a, c, d], output_ordering=[a@0 ASC NULLS LAST], has_header=true", ]; let expected_optimized = [ "SortPreservingMergeExec: [a@0 ASC NULLS LAST]", " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8, preserve_order=true, sort_exprs=a@0 ASC NULLS LAST", " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", - " CsvExec: file_groups={1 group: [[file_path]]}, projection=[a, c, d], infinite_source=true, output_ordering=[a@0 ASC NULLS LAST], has_header=true", + " CsvExec: file_groups={1 group: [[file_path]]}, projection=[a, c, d], output_ordering=[a@0 ASC NULLS LAST], has_header=true", ]; - assert_optimized!(expected_input, expected_optimized, physical_plan); + assert_optimized!(expected_input, expected_optimized, physical_plan, true); Ok(()) } @@ -378,7 +378,7 @@ mod tests { async fn test_with_inter_children_change_only() -> Result<()> { let schema = create_test_schema()?; let sort_exprs = vec![sort_expr_default("a", &schema)]; - let source = csv_exec_sorted(&schema, sort_exprs, true); + let source = csv_exec_sorted(&schema, sort_exprs); let repartition_rr = repartition_exec_round_robin(source); let repartition_hash = repartition_exec_hash(repartition_rr); let coalesce_partitions = coalesce_partitions_exec(repartition_hash); @@ -408,7 +408,7 @@ mod tests { " CoalescePartitionsExec", " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8", " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", - " CsvExec: file_groups={1 group: [[file_path]]}, projection=[a, c, d], infinite_source=true, output_ordering=[a@0 ASC], has_header=true", + " CsvExec: file_groups={1 group: [[file_path]]}, projection=[a, c, d], output_ordering=[a@0 ASC], has_header=true", ]; let expected_optimized = [ @@ -419,9 +419,9 @@ mod tests { " SortPreservingMergeExec: [a@0 ASC]", " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8, preserve_order=true, sort_exprs=a@0 ASC", " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", - " CsvExec: file_groups={1 group: [[file_path]]}, projection=[a, c, d], infinite_source=true, output_ordering=[a@0 ASC], has_header=true", + " CsvExec: file_groups={1 group: [[file_path]]}, projection=[a, c, d], output_ordering=[a@0 ASC], has_header=true", ]; - assert_optimized!(expected_input, expected_optimized, physical_plan); + assert_optimized!(expected_input, expected_optimized, physical_plan, true); Ok(()) } @@ -429,7 +429,7 @@ mod tests { async fn test_replace_multiple_input_repartition_2() -> Result<()> { let schema = create_test_schema()?; let sort_exprs = vec![sort_expr("a", &schema)]; - let source = csv_exec_sorted(&schema, sort_exprs, true); + let source = csv_exec_sorted(&schema, sort_exprs); let repartition_rr = repartition_exec_round_robin(source); let filter = filter_exec(repartition_rr); let repartition_hash = repartition_exec_hash(filter); @@ -444,16 +444,16 @@ mod tests { " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8", " FilterExec: c@1 > 3", " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", - " CsvExec: file_groups={1 group: [[file_path]]}, projection=[a, c, d], infinite_source=true, output_ordering=[a@0 ASC NULLS LAST], has_header=true", + " CsvExec: file_groups={1 group: [[file_path]]}, projection=[a, c, d], output_ordering=[a@0 ASC NULLS LAST], has_header=true", ]; let expected_optimized = [ "SortPreservingMergeExec: [a@0 ASC NULLS LAST]", " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8, preserve_order=true, sort_exprs=a@0 ASC NULLS LAST", " FilterExec: c@1 > 3", " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", - " CsvExec: file_groups={1 group: [[file_path]]}, projection=[a, c, d], infinite_source=true, output_ordering=[a@0 ASC NULLS LAST], has_header=true", + " CsvExec: file_groups={1 group: [[file_path]]}, projection=[a, c, d], output_ordering=[a@0 ASC NULLS LAST], has_header=true", ]; - assert_optimized!(expected_input, expected_optimized, physical_plan); + assert_optimized!(expected_input, expected_optimized, physical_plan, true); Ok(()) } @@ -461,7 +461,7 @@ mod tests { async fn test_replace_multiple_input_repartition_with_extra_steps() -> Result<()> { let schema = create_test_schema()?; let sort_exprs = vec![sort_expr("a", &schema)]; - let source = csv_exec_sorted(&schema, sort_exprs, true); + let source = csv_exec_sorted(&schema, sort_exprs); let repartition_rr = repartition_exec_round_robin(source); let repartition_hash = repartition_exec_hash(repartition_rr); let filter = filter_exec(repartition_hash); @@ -478,7 +478,7 @@ mod tests { " FilterExec: c@1 > 3", " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8", " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", - " CsvExec: file_groups={1 group: [[file_path]]}, projection=[a, c, d], infinite_source=true, output_ordering=[a@0 ASC NULLS LAST], has_header=true", + " CsvExec: file_groups={1 group: [[file_path]]}, projection=[a, c, d], output_ordering=[a@0 ASC NULLS LAST], has_header=true", ]; let expected_optimized = [ "SortPreservingMergeExec: [a@0 ASC NULLS LAST]", @@ -486,9 +486,9 @@ mod tests { " FilterExec: c@1 > 3", " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8, preserve_order=true, sort_exprs=a@0 ASC NULLS LAST", " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", - " CsvExec: file_groups={1 group: [[file_path]]}, projection=[a, c, d], infinite_source=true, output_ordering=[a@0 ASC NULLS LAST], has_header=true", + " CsvExec: file_groups={1 group: [[file_path]]}, projection=[a, c, d], output_ordering=[a@0 ASC NULLS LAST], has_header=true", ]; - assert_optimized!(expected_input, expected_optimized, physical_plan); + assert_optimized!(expected_input, expected_optimized, physical_plan, true); Ok(()) } @@ -496,7 +496,7 @@ mod tests { async fn test_replace_multiple_input_repartition_with_extra_steps_2() -> Result<()> { let schema = create_test_schema()?; let sort_exprs = vec![sort_expr("a", &schema)]; - let source = csv_exec_sorted(&schema, sort_exprs, true); + let source = csv_exec_sorted(&schema, sort_exprs); let repartition_rr = repartition_exec_round_robin(source); let coalesce_batches_exec_1 = coalesce_batches_exec(repartition_rr); let repartition_hash = repartition_exec_hash(coalesce_batches_exec_1); @@ -516,7 +516,7 @@ mod tests { " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8", " CoalesceBatchesExec: target_batch_size=8192", " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", - " CsvExec: file_groups={1 group: [[file_path]]}, projection=[a, c, d], infinite_source=true, output_ordering=[a@0 ASC NULLS LAST], has_header=true", + " CsvExec: file_groups={1 group: [[file_path]]}, projection=[a, c, d], output_ordering=[a@0 ASC NULLS LAST], has_header=true", ]; let expected_optimized = [ "SortPreservingMergeExec: [a@0 ASC NULLS LAST]", @@ -525,9 +525,9 @@ mod tests { " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8, preserve_order=true, sort_exprs=a@0 ASC NULLS LAST", " CoalesceBatchesExec: target_batch_size=8192", " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", - " CsvExec: file_groups={1 group: [[file_path]]}, projection=[a, c, d], infinite_source=true, output_ordering=[a@0 ASC NULLS LAST], has_header=true", + " CsvExec: file_groups={1 group: [[file_path]]}, projection=[a, c, d], output_ordering=[a@0 ASC NULLS LAST], has_header=true", ]; - assert_optimized!(expected_input, expected_optimized, physical_plan); + assert_optimized!(expected_input, expected_optimized, physical_plan, true); Ok(()) } @@ -535,7 +535,7 @@ mod tests { async fn test_not_replacing_when_no_need_to_preserve_sorting() -> Result<()> { let schema = create_test_schema()?; let sort_exprs = vec![sort_expr("a", &schema)]; - let source = csv_exec_sorted(&schema, sort_exprs, true); + let source = csv_exec_sorted(&schema, sort_exprs); let repartition_rr = repartition_exec_round_robin(source); let repartition_hash = repartition_exec_hash(repartition_rr); let filter = filter_exec(repartition_hash); @@ -550,7 +550,7 @@ mod tests { " FilterExec: c@1 > 3", " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8", " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", - " CsvExec: file_groups={1 group: [[file_path]]}, projection=[a, c, d], infinite_source=true, output_ordering=[a@0 ASC NULLS LAST], has_header=true", + " CsvExec: file_groups={1 group: [[file_path]]}, projection=[a, c, d], output_ordering=[a@0 ASC NULLS LAST], has_header=true", ]; let expected_optimized = [ "CoalescePartitionsExec", @@ -558,7 +558,7 @@ mod tests { " FilterExec: c@1 > 3", " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8", " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", - " CsvExec: file_groups={1 group: [[file_path]]}, projection=[a, c, d], infinite_source=true, output_ordering=[a@0 ASC NULLS LAST], has_header=true", + " CsvExec: file_groups={1 group: [[file_path]]}, projection=[a, c, d], output_ordering=[a@0 ASC NULLS LAST], has_header=true", ]; assert_optimized!(expected_input, expected_optimized, physical_plan); Ok(()) @@ -568,7 +568,7 @@ mod tests { async fn test_with_multiple_replacable_repartitions() -> Result<()> { let schema = create_test_schema()?; let sort_exprs = vec![sort_expr("a", &schema)]; - let source = csv_exec_sorted(&schema, sort_exprs, true); + let source = csv_exec_sorted(&schema, sort_exprs); let repartition_rr = repartition_exec_round_robin(source); let repartition_hash = repartition_exec_hash(repartition_rr); let filter = filter_exec(repartition_hash); @@ -587,7 +587,7 @@ mod tests { " FilterExec: c@1 > 3", " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8", " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", - " CsvExec: file_groups={1 group: [[file_path]]}, projection=[a, c, d], infinite_source=true, output_ordering=[a@0 ASC NULLS LAST], has_header=true", + " CsvExec: file_groups={1 group: [[file_path]]}, projection=[a, c, d], output_ordering=[a@0 ASC NULLS LAST], has_header=true", ]; let expected_optimized = [ "SortPreservingMergeExec: [a@0 ASC NULLS LAST]", @@ -596,9 +596,9 @@ mod tests { " FilterExec: c@1 > 3", " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8, preserve_order=true, sort_exprs=a@0 ASC NULLS LAST", " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", - " CsvExec: file_groups={1 group: [[file_path]]}, projection=[a, c, d], infinite_source=true, output_ordering=[a@0 ASC NULLS LAST], has_header=true", + " CsvExec: file_groups={1 group: [[file_path]]}, projection=[a, c, d], output_ordering=[a@0 ASC NULLS LAST], has_header=true", ]; - assert_optimized!(expected_input, expected_optimized, physical_plan); + assert_optimized!(expected_input, expected_optimized, physical_plan, true); Ok(()) } @@ -606,7 +606,7 @@ mod tests { async fn test_not_replace_with_different_orderings() -> Result<()> { let schema = create_test_schema()?; let sort_exprs = vec![sort_expr("a", &schema)]; - let source = csv_exec_sorted(&schema, sort_exprs, true); + let source = csv_exec_sorted(&schema, sort_exprs); let repartition_rr = repartition_exec_round_robin(source); let repartition_hash = repartition_exec_hash(repartition_rr); let sort = sort_exec( @@ -625,14 +625,14 @@ mod tests { " SortExec: expr=[c@1 ASC]", " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8", " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", - " CsvExec: file_groups={1 group: [[file_path]]}, projection=[a, c, d], infinite_source=true, output_ordering=[a@0 ASC NULLS LAST], has_header=true", + " CsvExec: file_groups={1 group: [[file_path]]}, projection=[a, c, d], output_ordering=[a@0 ASC NULLS LAST], has_header=true", ]; let expected_optimized = [ "SortPreservingMergeExec: [c@1 ASC]", " SortExec: expr=[c@1 ASC]", " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8", " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", - " CsvExec: file_groups={1 group: [[file_path]]}, projection=[a, c, d], infinite_source=true, output_ordering=[a@0 ASC NULLS LAST], has_header=true", + " CsvExec: file_groups={1 group: [[file_path]]}, projection=[a, c, d], output_ordering=[a@0 ASC NULLS LAST], has_header=true", ]; assert_optimized!(expected_input, expected_optimized, physical_plan); Ok(()) @@ -642,7 +642,7 @@ mod tests { async fn test_with_lost_ordering() -> Result<()> { let schema = create_test_schema()?; let sort_exprs = vec![sort_expr("a", &schema)]; - let source = csv_exec_sorted(&schema, sort_exprs, true); + let source = csv_exec_sorted(&schema, sort_exprs); let repartition_rr = repartition_exec_round_robin(source); let repartition_hash = repartition_exec_hash(repartition_rr); let coalesce_partitions = coalesce_partitions_exec(repartition_hash); @@ -654,15 +654,15 @@ mod tests { " CoalescePartitionsExec", " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8", " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", - " CsvExec: file_groups={1 group: [[file_path]]}, projection=[a, c, d], infinite_source=true, output_ordering=[a@0 ASC NULLS LAST], has_header=true", + " CsvExec: file_groups={1 group: [[file_path]]}, projection=[a, c, d], output_ordering=[a@0 ASC NULLS LAST], has_header=true", ]; let expected_optimized = [ "SortPreservingMergeExec: [a@0 ASC NULLS LAST]", " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8, preserve_order=true, sort_exprs=a@0 ASC NULLS LAST", " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", - " CsvExec: file_groups={1 group: [[file_path]]}, projection=[a, c, d], infinite_source=true, output_ordering=[a@0 ASC NULLS LAST], has_header=true", + " CsvExec: file_groups={1 group: [[file_path]]}, projection=[a, c, d], output_ordering=[a@0 ASC NULLS LAST], has_header=true", ]; - assert_optimized!(expected_input, expected_optimized, physical_plan); + assert_optimized!(expected_input, expected_optimized, physical_plan, true); Ok(()) } @@ -670,7 +670,7 @@ mod tests { async fn test_with_lost_and_kept_ordering() -> Result<()> { let schema = create_test_schema()?; let sort_exprs = vec![sort_expr("a", &schema)]; - let source = csv_exec_sorted(&schema, sort_exprs, true); + let source = csv_exec_sorted(&schema, sort_exprs); let repartition_rr = repartition_exec_round_robin(source); let repartition_hash = repartition_exec_hash(repartition_rr); let coalesce_partitions = coalesce_partitions_exec(repartition_hash); @@ -700,7 +700,7 @@ mod tests { " CoalescePartitionsExec", " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8", " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", - " CsvExec: file_groups={1 group: [[file_path]]}, projection=[a, c, d], infinite_source=true, output_ordering=[a@0 ASC NULLS LAST], has_header=true", + " CsvExec: file_groups={1 group: [[file_path]]}, projection=[a, c, d], output_ordering=[a@0 ASC NULLS LAST], has_header=true", ]; let expected_optimized = [ @@ -712,9 +712,9 @@ mod tests { " CoalescePartitionsExec", " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8", " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", - " CsvExec: file_groups={1 group: [[file_path]]}, projection=[a, c, d], infinite_source=true, output_ordering=[a@0 ASC NULLS LAST], has_header=true", + " CsvExec: file_groups={1 group: [[file_path]]}, projection=[a, c, d], output_ordering=[a@0 ASC NULLS LAST], has_header=true", ]; - assert_optimized!(expected_input, expected_optimized, physical_plan); + assert_optimized!(expected_input, expected_optimized, physical_plan, true); Ok(()) } @@ -723,14 +723,14 @@ mod tests { let schema = create_test_schema()?; let left_sort_exprs = vec![sort_expr("a", &schema)]; - let left_source = csv_exec_sorted(&schema, left_sort_exprs, true); + let left_source = csv_exec_sorted(&schema, left_sort_exprs); let left_repartition_rr = repartition_exec_round_robin(left_source); let left_repartition_hash = repartition_exec_hash(left_repartition_rr); let left_coalesce_partitions = Arc::new(CoalesceBatchesExec::new(left_repartition_hash, 4096)); let right_sort_exprs = vec![sort_expr("a", &schema)]; - let right_source = csv_exec_sorted(&schema, right_sort_exprs, true); + let right_source = csv_exec_sorted(&schema, right_sort_exprs); let right_repartition_rr = repartition_exec_round_robin(right_source); let right_repartition_hash = repartition_exec_hash(right_repartition_rr); let right_coalesce_partitions = @@ -756,11 +756,11 @@ mod tests { " CoalesceBatchesExec: target_batch_size=4096", " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8", " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", - " CsvExec: file_groups={1 group: [[file_path]]}, projection=[a, c, d], infinite_source=true, output_ordering=[a@0 ASC NULLS LAST], has_header=true", + " CsvExec: file_groups={1 group: [[file_path]]}, projection=[a, c, d], output_ordering=[a@0 ASC NULLS LAST], has_header=true", " CoalesceBatchesExec: target_batch_size=4096", " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8", " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", - " CsvExec: file_groups={1 group: [[file_path]]}, projection=[a, c, d], infinite_source=true, output_ordering=[a@0 ASC NULLS LAST], has_header=true", + " CsvExec: file_groups={1 group: [[file_path]]}, projection=[a, c, d], output_ordering=[a@0 ASC NULLS LAST], has_header=true", ]; let expected_optimized = [ @@ -770,11 +770,11 @@ mod tests { " CoalesceBatchesExec: target_batch_size=4096", " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8", " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", - " CsvExec: file_groups={1 group: [[file_path]]}, projection=[a, c, d], infinite_source=true, output_ordering=[a@0 ASC NULLS LAST], has_header=true", + " CsvExec: file_groups={1 group: [[file_path]]}, projection=[a, c, d], output_ordering=[a@0 ASC NULLS LAST], has_header=true", " CoalesceBatchesExec: target_batch_size=4096", " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8", " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", - " CsvExec: file_groups={1 group: [[file_path]]}, projection=[a, c, d], infinite_source=true, output_ordering=[a@0 ASC NULLS LAST], has_header=true", + " CsvExec: file_groups={1 group: [[file_path]]}, projection=[a, c, d], output_ordering=[a@0 ASC NULLS LAST], has_header=true", ]; assert_optimized!(expected_input, expected_optimized, physical_plan); Ok(()) @@ -784,7 +784,7 @@ mod tests { async fn test_with_bounded_input() -> Result<()> { let schema = create_test_schema()?; let sort_exprs = vec![sort_expr("a", &schema)]; - let source = csv_exec_sorted(&schema, sort_exprs, false); + let source = csv_exec_sorted(&schema, sort_exprs); let repartition = repartition_exec_hash(repartition_exec_round_robin(source)); let sort = sort_exec(vec![sort_expr("a", &schema)], repartition, true); @@ -931,7 +931,6 @@ mod tests { fn csv_exec_sorted( schema: &SchemaRef, sort_exprs: impl IntoIterator, - infinite_source: bool, ) -> Arc { let sort_exprs = sort_exprs.into_iter().collect(); let projection: Vec = vec![0, 2, 3]; @@ -949,7 +948,6 @@ mod tests { limit: None, table_partition_cols: vec![], output_ordering: vec![sort_exprs], - infinite_source, }, true, 0, diff --git a/datafusion/core/src/physical_optimizer/test_utils.rs b/datafusion/core/src/physical_optimizer/test_utils.rs index 678dc1f373e3..6e14cca21fed 100644 --- a/datafusion/core/src/physical_optimizer/test_utils.rs +++ b/datafusion/core/src/physical_optimizer/test_utils.rs @@ -45,6 +45,7 @@ use datafusion_expr::{AggregateFunction, WindowFrame, WindowFunction}; use datafusion_physical_expr::expressions::col; use datafusion_physical_expr::{PhysicalExpr, PhysicalSortExpr}; +use crate::datasource::stream::{StreamConfig, StreamTable}; use async_trait::async_trait; async fn register_current_csv( @@ -54,14 +55,19 @@ async fn register_current_csv( ) -> Result<()> { let testdata = crate::test_util::arrow_test_data(); let schema = crate::test_util::aggr_test_schema(); - ctx.register_csv( - table_name, - &format!("{testdata}/csv/aggregate_test_100.csv"), - CsvReadOptions::new() - .schema(&schema) - .mark_infinite(infinite), - ) - .await?; + let path = format!("{testdata}/csv/aggregate_test_100.csv"); + + match infinite { + true => { + let config = StreamConfig::new_file(schema, path.into()); + ctx.register_table(table_name, Arc::new(StreamTable::new(Arc::new(config))))?; + } + false => { + ctx.register_csv(table_name, &path, CsvReadOptions::new().schema(&schema)) + .await?; + } + } + Ok(()) } @@ -272,7 +278,6 @@ pub fn parquet_exec(schema: &SchemaRef) -> Arc { limit: None, table_partition_cols: vec![], output_ordering: vec![], - infinite_source: false, }, None, None, @@ -296,7 +301,6 @@ pub fn parquet_exec_sorted( limit: None, table_partition_cols: vec![], output_ordering: vec![sort_exprs], - infinite_source: false, }, None, None, diff --git a/datafusion/core/src/test/mod.rs b/datafusion/core/src/test/mod.rs index aad5c19044ea..8770c0c4238a 100644 --- a/datafusion/core/src/test/mod.rs +++ b/datafusion/core/src/test/mod.rs @@ -203,7 +203,6 @@ pub fn partitioned_csv_config( limit: None, table_partition_cols: vec![], output_ordering: vec![], - infinite_source: false, }) } @@ -277,7 +276,6 @@ fn make_decimal() -> RecordBatch { pub fn csv_exec_sorted( schema: &SchemaRef, sort_exprs: impl IntoIterator, - infinite_source: bool, ) -> Arc { let sort_exprs = sort_exprs.into_iter().collect(); @@ -291,7 +289,6 @@ pub fn csv_exec_sorted( limit: None, table_partition_cols: vec![], output_ordering: vec![sort_exprs], - infinite_source, }, false, 0, diff --git a/datafusion/core/src/test_util/mod.rs b/datafusion/core/src/test_util/mod.rs index c6b43de0c18d..282b0f7079ee 100644 --- a/datafusion/core/src/test_util/mod.rs +++ b/datafusion/core/src/test_util/mod.rs @@ -36,7 +36,6 @@ use crate::datasource::provider::TableProviderFactory; use crate::datasource::{empty::EmptyTable, provider_as_source, TableProvider}; use crate::error::Result; use crate::execution::context::{SessionState, TaskContext}; -use crate::execution::options::ReadOptions; use crate::logical_expr::{LogicalPlanBuilder, UNNAMED_TABLE}; use crate::physical_plan::{ DisplayAs, DisplayFormatType, ExecutionPlan, Partitioning, RecordBatchStream, @@ -58,6 +57,7 @@ use futures::Stream; pub use datafusion_common::test_util::parquet_test_data; pub use datafusion_common::test_util::{arrow_test_data, get_data_dir}; +use crate::datasource::stream::{StreamConfig, StreamTable}; pub use datafusion_common::{assert_batches_eq, assert_batches_sorted_eq}; /// Scan an empty data source, mainly used in tests @@ -342,30 +342,17 @@ impl RecordBatchStream for UnboundedStream { } /// This function creates an unbounded sorted file for testing purposes. -pub async fn register_unbounded_file_with_ordering( +pub fn register_unbounded_file_with_ordering( ctx: &SessionContext, schema: SchemaRef, file_path: &Path, table_name: &str, file_sort_order: Vec>, - with_unbounded_execution: bool, ) -> Result<()> { - // Mark infinite and provide schema: - let fifo_options = CsvReadOptions::new() - .schema(schema.as_ref()) - .mark_infinite(with_unbounded_execution); - // Get listing options: - let options_sort = fifo_options - .to_listing_options(&ctx.copied_config()) - .with_file_sort_order(file_sort_order); + let config = + StreamConfig::new_file(schema, file_path.into()).with_order(file_sort_order); + // Register table: - ctx.register_listing_table( - table_name, - file_path.as_os_str().to_str().unwrap(), - options_sort, - Some(schema), - None, - ) - .await?; + ctx.register_table(table_name, Arc::new(StreamTable::new(Arc::new(config))))?; Ok(()) } diff --git a/datafusion/core/src/test_util/parquet.rs b/datafusion/core/src/test_util/parquet.rs index f3c0d2987a46..336a6804637a 100644 --- a/datafusion/core/src/test_util/parquet.rs +++ b/datafusion/core/src/test_util/parquet.rs @@ -156,7 +156,6 @@ impl TestParquetFile { limit: None, table_partition_cols: vec![], output_ordering: vec![], - infinite_source: false, }; let df_schema = self.schema.clone().to_dfschema_ref()?; diff --git a/datafusion/core/tests/parquet/custom_reader.rs b/datafusion/core/tests/parquet/custom_reader.rs index 3752d42dbf43..e76b201e0222 100644 --- a/datafusion/core/tests/parquet/custom_reader.rs +++ b/datafusion/core/tests/parquet/custom_reader.rs @@ -85,7 +85,6 @@ async fn route_data_access_ops_to_parquet_file_reader_factory() { limit: None, table_partition_cols: vec![], output_ordering: vec![], - infinite_source: false, }, None, None, diff --git a/datafusion/core/tests/parquet/page_pruning.rs b/datafusion/core/tests/parquet/page_pruning.rs index e1e8b8e66edd..23a56bc821d4 100644 --- a/datafusion/core/tests/parquet/page_pruning.rs +++ b/datafusion/core/tests/parquet/page_pruning.rs @@ -81,7 +81,6 @@ async fn get_parquet_exec(state: &SessionState, filter: Expr) -> ParquetExec { limit: None, table_partition_cols: vec![], output_ordering: vec![], - infinite_source: false, }, Some(predicate), None, diff --git a/datafusion/core/tests/parquet/schema_coercion.rs b/datafusion/core/tests/parquet/schema_coercion.rs index 25c62f18f5ba..00f3eada496e 100644 --- a/datafusion/core/tests/parquet/schema_coercion.rs +++ b/datafusion/core/tests/parquet/schema_coercion.rs @@ -69,7 +69,6 @@ async fn multi_parquet_coercion() { limit: None, table_partition_cols: vec![], output_ordering: vec![], - infinite_source: false, }, None, None, @@ -133,7 +132,6 @@ async fn multi_parquet_coercion_projection() { limit: None, table_partition_cols: vec![], output_ordering: vec![], - infinite_source: false, }, None, None, diff --git a/datafusion/core/tests/sql/joins.rs b/datafusion/core/tests/sql/joins.rs index 528bde632355..d1f270b540b5 100644 --- a/datafusion/core/tests/sql/joins.rs +++ b/datafusion/core/tests/sql/joins.rs @@ -15,6 +15,7 @@ // specific language governing permissions and limitations // under the License. +use datafusion::datasource::stream::{StreamConfig, StreamTable}; use datafusion::test_util::register_unbounded_file_with_ordering; use super::*; @@ -105,9 +106,7 @@ async fn join_change_in_planner() -> Result<()> { &left_file_path, "left", file_sort_order.clone(), - true, - ) - .await?; + )?; let right_file_path = tmp_dir.path().join("right.csv"); File::create(right_file_path.clone()).unwrap(); register_unbounded_file_with_ordering( @@ -116,9 +115,7 @@ async fn join_change_in_planner() -> Result<()> { &right_file_path, "right", file_sort_order, - true, - ) - .await?; + )?; let sql = "SELECT t1.a1, t1.a2, t2.a1, t2.a2 FROM left as t1 FULL JOIN right as t2 ON t1.a2 = t2.a2 AND t1.a1 > t2.a1 + 3 AND t1.a1 < t2.a1 + 10"; let dataframe = ctx.sql(sql).await?; let physical_plan = dataframe.create_physical_plan().await?; @@ -160,20 +157,13 @@ async fn join_change_in_planner_without_sort() -> Result<()> { Field::new("a1", DataType::UInt32, false), Field::new("a2", DataType::UInt32, false), ])); - ctx.register_csv( - "left", - left_file_path.as_os_str().to_str().unwrap(), - CsvReadOptions::new().schema(&schema).mark_infinite(true), - ) - .await?; + let left = StreamConfig::new_file(schema.clone(), left_file_path); + ctx.register_table("left", Arc::new(StreamTable::new(Arc::new(left))))?; + let right_file_path = tmp_dir.path().join("right.csv"); File::create(right_file_path.clone())?; - ctx.register_csv( - "right", - right_file_path.as_os_str().to_str().unwrap(), - CsvReadOptions::new().schema(&schema).mark_infinite(true), - ) - .await?; + let right = StreamConfig::new_file(schema, right_file_path); + ctx.register_table("right", Arc::new(StreamTable::new(Arc::new(right))))?; let sql = "SELECT t1.a1, t1.a2, t2.a1, t2.a2 FROM left as t1 FULL JOIN right as t2 ON t1.a2 = t2.a2 AND t1.a1 > t2.a1 + 3 AND t1.a1 < t2.a1 + 10"; let dataframe = ctx.sql(sql).await?; let physical_plan = dataframe.create_physical_plan().await?; @@ -217,20 +207,12 @@ async fn join_change_in_planner_without_sort_not_allowed() -> Result<()> { Field::new("a1", DataType::UInt32, false), Field::new("a2", DataType::UInt32, false), ])); - ctx.register_csv( - "left", - left_file_path.as_os_str().to_str().unwrap(), - CsvReadOptions::new().schema(&schema).mark_infinite(true), - ) - .await?; + let left = StreamConfig::new_file(schema.clone(), left_file_path); + ctx.register_table("left", Arc::new(StreamTable::new(Arc::new(left))))?; let right_file_path = tmp_dir.path().join("right.csv"); File::create(right_file_path.clone())?; - ctx.register_csv( - "right", - right_file_path.as_os_str().to_str().unwrap(), - CsvReadOptions::new().schema(&schema).mark_infinite(true), - ) - .await?; + let right = StreamConfig::new_file(schema.clone(), right_file_path); + ctx.register_table("right", Arc::new(StreamTable::new(Arc::new(right))))?; let df = ctx.sql("SELECT t1.a1, t1.a2, t2.a1, t2.a2 FROM left as t1 FULL JOIN right as t2 ON t1.a2 = t2.a2 AND t1.a1 > t2.a1 + 3 AND t1.a1 < t2.a1 + 10").await?; match df.create_physical_plan().await { Ok(_) => panic!("Expecting error."), diff --git a/datafusion/proto/src/physical_plan/from_proto.rs b/datafusion/proto/src/physical_plan/from_proto.rs index dcebfbf2dabb..5c0ef615cacd 100644 --- a/datafusion/proto/src/physical_plan/from_proto.rs +++ b/datafusion/proto/src/physical_plan/from_proto.rs @@ -526,7 +526,6 @@ pub fn parse_protobuf_file_scan_config( limit: proto.limit.as_ref().map(|sl| sl.limit as usize), table_partition_cols, output_ordering, - infinite_source: false, }) } diff --git a/datafusion/proto/tests/cases/roundtrip_physical_plan.rs b/datafusion/proto/tests/cases/roundtrip_physical_plan.rs index 4a512413e73e..9a9827f2a090 100644 --- a/datafusion/proto/tests/cases/roundtrip_physical_plan.rs +++ b/datafusion/proto/tests/cases/roundtrip_physical_plan.rs @@ -492,7 +492,6 @@ fn roundtrip_parquet_exec_with_pruning_predicate() -> Result<()> { limit: None, table_partition_cols: vec![], output_ordering: vec![], - infinite_source: false, }; let predicate = Arc::new(BinaryExpr::new( diff --git a/datafusion/substrait/src/physical_plan/consumer.rs b/datafusion/substrait/src/physical_plan/consumer.rs index 942798173e0e..3098dc386e6a 100644 --- a/datafusion/substrait/src/physical_plan/consumer.rs +++ b/datafusion/substrait/src/physical_plan/consumer.rs @@ -112,7 +112,6 @@ pub async fn from_substrait_rel( limit: None, table_partition_cols: vec![], output_ordering: vec![], - infinite_source: false, }; if let Some(MaskExpression { select, .. }) = &read.projection { diff --git a/datafusion/substrait/tests/cases/roundtrip_physical_plan.rs b/datafusion/substrait/tests/cases/roundtrip_physical_plan.rs index b64dd2c138fc..e5af3f94cc05 100644 --- a/datafusion/substrait/tests/cases/roundtrip_physical_plan.rs +++ b/datafusion/substrait/tests/cases/roundtrip_physical_plan.rs @@ -49,7 +49,6 @@ async fn parquet_exec() -> Result<()> { limit: None, table_partition_cols: vec![], output_ordering: vec![], - infinite_source: false, }; let parquet_exec: Arc = Arc::new(ParquetExec::new(scan_config, None, None)); From d65b51a4d5fef13135b900249a4f7934b1098339 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Mon, 18 Dec 2023 14:00:39 -0500 Subject: [PATCH 455/572] Update substrait requirement from 0.20.0 to 0.21.0 (#8574) Updates the requirements on [substrait](https://github.com/substrait-io/substrait-rs) to permit the latest version. - [Release notes](https://github.com/substrait-io/substrait-rs/releases) - [Changelog](https://github.com/substrait-io/substrait-rs/blob/main/CHANGELOG.md) - [Commits](https://github.com/substrait-io/substrait-rs/compare/v0.20.0...v0.21.0) --- updated-dependencies: - dependency-name: substrait dependency-type: direct:production ... Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- datafusion/substrait/Cargo.toml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/datafusion/substrait/Cargo.toml b/datafusion/substrait/Cargo.toml index 42ebe56c298b..0a9a6e8dd12b 100644 --- a/datafusion/substrait/Cargo.toml +++ b/datafusion/substrait/Cargo.toml @@ -35,7 +35,7 @@ itertools = { workspace = true } object_store = { workspace = true } prost = "0.12" prost-types = "0.12" -substrait = "0.20.0" +substrait = "0.21.0" tokio = "1.17" [features] From ceead1cc48fd903bd877bb45e258b8ccc12e5b30 Mon Sep 17 00:00:00 2001 From: Mustafa Akur <106137913+mustafasrepo@users.noreply.github.com> Date: Mon, 18 Dec 2023 22:01:50 +0300 Subject: [PATCH 456/572] [minor]: Fix rank calculation bug when empty order by is seen (#8567) * minor: fix to support scalars * Fix empty order by rank implementation --------- Co-authored-by: comphead --- datafusion/physical-expr/src/window/rank.rs | 13 ++++++-- .../physical-expr/src/window/window_expr.rs | 2 +- datafusion/sqllogictest/test_files/window.slt | 30 +++++++++++-------- 3 files changed, 29 insertions(+), 16 deletions(-) diff --git a/datafusion/physical-expr/src/window/rank.rs b/datafusion/physical-expr/src/window/rank.rs index 9bc36728f46e..86af5b322133 100644 --- a/datafusion/physical-expr/src/window/rank.rs +++ b/datafusion/physical-expr/src/window/rank.rs @@ -141,9 +141,16 @@ impl PartitionEvaluator for RankEvaluator { // There is no argument, values are order by column values (where rank is calculated) let range_columns = values; let last_rank_data = get_row_at_idx(range_columns, row_idx)?; - let empty = self.state.last_rank_data.is_empty(); - if empty || self.state.last_rank_data != last_rank_data { - self.state.last_rank_data = last_rank_data; + let new_rank_encountered = + if let Some(state_last_rank_data) = &self.state.last_rank_data { + // if rank data changes, new rank is encountered + state_last_rank_data != &last_rank_data + } else { + // First rank seen + true + }; + if new_rank_encountered { + self.state.last_rank_data = Some(last_rank_data); self.state.last_rank_boundary += self.state.current_group_count; self.state.current_group_count = 1; self.state.n_rank += 1; diff --git a/datafusion/physical-expr/src/window/window_expr.rs b/datafusion/physical-expr/src/window/window_expr.rs index 4211a616e100..548fae75bd97 100644 --- a/datafusion/physical-expr/src/window/window_expr.rs +++ b/datafusion/physical-expr/src/window/window_expr.rs @@ -274,7 +274,7 @@ pub enum WindowFn { #[derive(Debug, Clone, Default)] pub struct RankState { /// The last values for rank as these values change, we increase n_rank - pub last_rank_data: Vec, + pub last_rank_data: Option>, /// The index where last_rank_boundary is started pub last_rank_boundary: usize, /// Keep the number of entries in current rank diff --git a/datafusion/sqllogictest/test_files/window.slt b/datafusion/sqllogictest/test_files/window.slt index 864f7dc0a47d..aa083290b4f4 100644 --- a/datafusion/sqllogictest/test_files/window.slt +++ b/datafusion/sqllogictest/test_files/window.slt @@ -3801,19 +3801,25 @@ select rank() over (order by 1) rnk from (select 1 a union all select 2 a) x 1 1 +# support scalar value in ORDER BY +query I +select dense_rank() over () rnk from (select 1 a union all select 2 a) x +---- +1 +1 + # support scalar value in both ORDER BY and PARTITION BY, RANK function -# TODO: fix the test, some issue in RANK -#query IIIIII -#select rank() over (partition by 1 order by 1) rnk, -# rank() over (partition by a, 1 order by 1) rnk1, -# rank() over (partition by a, 1 order by a, 1) rnk2, -# rank() over (partition by 1) rnk3, -# rank() over (partition by null) rnk4, -# rank() over (partition by 1, null, a) rnk5 -#from (select 1 a union all select 2 a) x -#---- -#1 1 1 1 1 1 -#1 1 1 1 1 1 +query IIIIII +select rank() over (partition by 1 order by 1) rnk, + rank() over (partition by a, 1 order by 1) rnk1, + rank() over (partition by a, 1 order by a, 1) rnk2, + rank() over (partition by 1) rnk3, + rank() over (partition by null) rnk4, + rank() over (partition by 1, null, a) rnk5 +from (select 1 a union all select 2 a) x +---- +1 1 1 1 1 1 +1 1 1 1 1 1 # support scalar value in both ORDER BY and PARTITION BY, ROW_NUMBER function query IIIIII From b5e94a688e3a66325cc6ed9b2e35b44cf6cd9ba8 Mon Sep 17 00:00:00 2001 From: Andrew Lamb Date: Mon, 18 Dec 2023 14:08:33 -0500 Subject: [PATCH 457/572] Add `LiteralGuarantee` on columns to extract conditions required for `PhysicalExpr` expressions to evaluate to true (#8437) * Introduce LiteralGurantee to find col=const * Improve comments * Improve documentation * Add more documentation and tests * refine documentation and tests * Apply suggestions from code review Co-authored-by: Nga Tran * Fix half comment * swap operators before analysis * More tests * cmt * Apply suggestions from code review Co-authored-by: Ruihang Xia * refine comments more --------- Co-authored-by: Nga Tran Co-authored-by: Ruihang Xia --- .../physical-expr/src/utils/guarantee.rs | 709 ++++++++++++++++++ .../src/{utils.rs => utils/mod.rs} | 33 +- 2 files changed, 729 insertions(+), 13 deletions(-) create mode 100644 datafusion/physical-expr/src/utils/guarantee.rs rename datafusion/physical-expr/src/{utils.rs => utils/mod.rs} (96%) diff --git a/datafusion/physical-expr/src/utils/guarantee.rs b/datafusion/physical-expr/src/utils/guarantee.rs new file mode 100644 index 000000000000..59ec255754c0 --- /dev/null +++ b/datafusion/physical-expr/src/utils/guarantee.rs @@ -0,0 +1,709 @@ +// 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. + +//! [`LiteralGuarantee`] predicate analysis to determine if a column is a +//! constant. + +use crate::utils::split_disjunction; +use crate::{split_conjunction, PhysicalExpr}; +use datafusion_common::{Column, ScalarValue}; +use datafusion_expr::Operator; +use std::collections::{HashMap, HashSet}; +use std::sync::Arc; + +/// Represents a guarantee that must be true for a boolean expression to +/// evaluate to `true`. +/// +/// The guarantee takes the form of a column and a set of literal (constant) +/// [`ScalarValue`]s. For the expression to evaluate to `true`, the column *must +/// satisfy* the guarantee(s). +/// +/// To satisfy the guarantee, depending on [`Guarantee`], the values in the +/// column must either: +/// +/// 1. be ONLY one of that set +/// 2. NOT be ANY of that set +/// +/// # Uses `LiteralGuarantee`s +/// +/// `LiteralGuarantee`s can be used to simplify filter expressions and skip data +/// files (e.g. row groups in parquet files) by proving expressions can not +/// possibly evaluate to `true`. For example, if we have a guarantee that `a` +/// must be in (`1`) for a filter to evaluate to `true`, then we can skip any +/// partition where we know that `a` never has the value of `1`. +/// +/// **Important**: If a `LiteralGuarantee` is not satisfied, the relevant +/// expression is *guaranteed* to evaluate to `false` or `null`. **However**, +/// the opposite does not hold. Even if all `LiteralGuarantee`s are satisfied, +/// that does **not** guarantee that the predicate will actually evaluate to +/// `true`: it may still evaluate to `true`, `false` or `null`. +/// +/// # Creating `LiteralGuarantee`s +/// +/// Use [`LiteralGuarantee::analyze`] to extract literal guarantees from a +/// filter predicate. +/// +/// # Details +/// A guarantee can be one of two forms: +/// +/// 1. The column must be one the values for the predicate to be `true`. If the +/// column takes on any other value, the predicate can not evaluate to `true`. +/// For example, +/// `(a = 1)`, `(a = 1 OR a = 2) or `a IN (1, 2, 3)` +/// +/// 2. The column must NOT be one of the values for the predicate to be `true`. +/// If the column can ONLY take one of these values, the predicate can not +/// evaluate to `true`. For example, +/// `(a != 1)`, `(a != 1 AND a != 2)` or `a NOT IN (1, 2, 3)` +#[derive(Debug, Clone, PartialEq)] +pub struct LiteralGuarantee { + pub column: Column, + pub guarantee: Guarantee, + pub literals: HashSet, +} + +/// What is guaranteed about the values for a [`LiteralGuarantee`]? +#[derive(Debug, Clone, PartialEq)] +pub enum Guarantee { + /// Guarantee that the expression is `true` if `column` is one of the values. If + /// `column` is not one of the values, the expression can not be `true`. + In, + /// Guarantee that the expression is `true` if `column` is not ANY of the + /// values. If `column` only takes one of these values, the expression can + /// not be `true`. + NotIn, +} + +impl LiteralGuarantee { + /// Create a new instance of the guarantee if the provided operator is + /// supported. Returns None otherwise. See [`LiteralGuarantee::analyze`] to + /// create these structures from an predicate (boolean expression). + fn try_new<'a>( + column_name: impl Into, + op: Operator, + literals: impl IntoIterator, + ) -> Option { + let guarantee = match op { + Operator::Eq => Guarantee::In, + Operator::NotEq => Guarantee::NotIn, + _ => return None, + }; + + let literals: HashSet<_> = literals.into_iter().cloned().collect(); + + Some(Self { + column: Column::from_name(column_name), + guarantee, + literals, + }) + } + + /// Return a list of [`LiteralGuarantee`]s that must be satisfied for `expr` + /// to evaluate to `true`. + /// + /// If more than one `LiteralGuarantee` is returned, they must **all** hold + /// for the expression to possibly be `true`. If any is not satisfied, the + /// expression is guaranteed to be `null` or `false`. + /// + /// # Notes: + /// 1. `expr` must be a boolean expression. + /// 2. `expr` is not simplified prior to analysis. + pub fn analyze(expr: &Arc) -> Vec { + // split conjunction: AND AND ... + split_conjunction(expr) + .into_iter() + // for an `AND` conjunction to be true, all terms individually must be true + .fold(GuaranteeBuilder::new(), |builder, expr| { + if let Some(cel) = ColOpLit::try_new(expr) { + return builder.aggregate_conjunct(cel); + } else { + // split disjunction: OR OR ... + let disjunctions = split_disjunction(expr); + + // We are trying to add a guarantee that a column must be + // in/not in a particular set of values for the expression + // to evaluate to true. + // + // A disjunction is true, if at least one of the terms is be + // true. + // + // Thus, we can infer a guarantee if all terms are of the + // form `(col literal) OR (col literal) OR ...`. + // + // For example, we can infer that `a = 1 OR a = 2 OR a = 3` + // is guaranteed to be true ONLY if a is in (`1`, `2` or `3`). + // + // However, for something like `a = 1 OR a = 2 OR a < 0` we + // **can't** guarantee that the predicate is only true if a + // is in (`1`, `2`), as it could also be true if `a` were less + // than zero. + let terms = disjunctions + .iter() + .filter_map(|expr| ColOpLit::try_new(expr)) + .collect::>(); + + if terms.is_empty() { + return builder; + } + + // if not all terms are of the form (col literal), + // can't infer any guarantees + if terms.len() != disjunctions.len() { + return builder; + } + + // if all terms are 'col literal' with the same column + // and operation we can infer any guarantees + let first_term = &terms[0]; + if terms.iter().all(|term| { + term.col.name() == first_term.col.name() + && term.op == first_term.op + }) { + builder.aggregate_multi_conjunct( + first_term.col, + first_term.op, + terms.iter().map(|term| term.lit.value()), + ) + } else { + // can't infer anything + builder + } + } + }) + .build() + } +} + +/// Combines conjuncts (aka terms `AND`ed together) into [`LiteralGuarantee`]s, +/// preserving insert order +#[derive(Debug, Default)] +struct GuaranteeBuilder<'a> { + /// List of guarantees that have been created so far + /// if we have determined a subsequent conjunct invalidates a guarantee + /// e.g. `a = foo AND a = bar` then the relevant guarantee will be None + guarantees: Vec>, + + /// Key is the (column name, operator type) + /// Value is the index into `guarantees` + map: HashMap<(&'a crate::expressions::Column, Operator), usize>, +} + +impl<'a> GuaranteeBuilder<'a> { + fn new() -> Self { + Default::default() + } + + /// Aggregate a new single `AND col literal` term to this builder + /// combining with existing guarantees if possible. + /// + /// # Examples + /// * `AND (a = 1)`: `a` is guaranteed to be 1 + /// * `AND (a != 1)`: a is guaranteed to not be 1 + fn aggregate_conjunct(self, col_op_lit: ColOpLit<'a>) -> Self { + self.aggregate_multi_conjunct( + col_op_lit.col, + col_op_lit.op, + [col_op_lit.lit.value()], + ) + } + + /// Aggregates a new single column, multi literal term to ths builder + /// combining with previously known guarantees if possible. + /// + /// # Examples + /// For the following examples, we can guarantee the expression is `true` if: + /// * `AND (a = 1 OR a = 2 OR a = 3)`: a is in (1, 2, or 3) + /// * `AND (a IN (1,2,3))`: a is in (1, 2, or 3) + /// * `AND (a != 1 OR a != 2 OR a != 3)`: a is not in (1, 2, or 3) + /// * `AND (a NOT IN (1,2,3))`: a is not in (1, 2, or 3) + fn aggregate_multi_conjunct( + mut self, + col: &'a crate::expressions::Column, + op: Operator, + new_values: impl IntoIterator, + ) -> Self { + let key = (col, op); + if let Some(index) = self.map.get(&key) { + // already have a guarantee for this column + let entry = &mut self.guarantees[*index]; + + let Some(existing) = entry else { + // determined the previous guarantee for this column has been + // invalidated, nothing to do + return self; + }; + + // Combine conjuncts if we have `a != foo AND a != bar`. `a = foo + // AND a = bar` doesn't make logical sense so we don't optimize this + // case + match existing.guarantee { + // knew that the column could not be a set of values + // + // For example, if we previously had `a != 5` and now we see + // another `AND a != 6` we know that a must not be either 5 or 6 + // for the expression to be true + Guarantee::NotIn => { + // can extend if only single literal, otherwise invalidate + let new_values: HashSet<_> = new_values.into_iter().collect(); + if new_values.len() == 1 { + existing.literals.extend(new_values.into_iter().cloned()) + } else { + // this is like (a != foo AND (a != bar OR a != baz)). + // We can't combine the (a != bar OR a != baz) part, but + // it also doesn't invalidate our knowledge that a != + // foo is required for the expression to be true + } + } + Guarantee::In => { + // for an IN guarantee, it is ok if the value is the same + // e.g. `a = foo AND a = foo` but not if the value is different + // e.g. `a = foo AND a = bar` + if new_values + .into_iter() + .all(|new_value| existing.literals.contains(new_value)) + { + // all values are already in the set + } else { + // at least one was not, so invalidate the guarantee + *entry = None; + } + } + } + } else { + // This is a new guarantee + let new_values: HashSet<_> = new_values.into_iter().collect(); + + // new_values are combined with OR, so we can only create a + // multi-column guarantee for `=` (or a single value). + // (e.g. ignore `a != foo OR a != bar`) + if op == Operator::Eq || new_values.len() == 1 { + if let Some(guarantee) = + LiteralGuarantee::try_new(col.name(), op, new_values) + { + // add it to the list of guarantees + self.guarantees.push(Some(guarantee)); + self.map.insert(key, self.guarantees.len() - 1); + } + } + } + + self + } + + /// Return all guarantees that have been created so far + fn build(self) -> Vec { + // filter out any guarantees that have been invalidated + self.guarantees.into_iter().flatten().collect() + } +} + +/// Represents a single `col literal` expression +struct ColOpLit<'a> { + col: &'a crate::expressions::Column, + op: Operator, + lit: &'a crate::expressions::Literal, +} + +impl<'a> ColOpLit<'a> { + /// Returns Some(ColEqLit) if the expression is either: + /// 1. `col literal` + /// 2. `literal col` + /// + /// Returns None otherwise + fn try_new(expr: &'a Arc) -> Option { + let binary_expr = expr + .as_any() + .downcast_ref::()?; + + let (left, op, right) = ( + binary_expr.left().as_any(), + binary_expr.op(), + binary_expr.right().as_any(), + ); + + // col literal + if let (Some(col), Some(lit)) = ( + left.downcast_ref::(), + right.downcast_ref::(), + ) { + Some(Self { col, op: *op, lit }) + } + // literal col + else if let (Some(lit), Some(col)) = ( + left.downcast_ref::(), + right.downcast_ref::(), + ) { + // Used swapped operator operator, if possible + op.swap().map(|op| Self { col, op, lit }) + } else { + None + } + } +} + +#[cfg(test)] +mod test { + use super::*; + use crate::create_physical_expr; + use crate::execution_props::ExecutionProps; + use arrow_schema::{DataType, Field, Schema, SchemaRef}; + use datafusion_common::ToDFSchema; + use datafusion_expr::expr_fn::*; + use datafusion_expr::{lit, Expr}; + use std::sync::OnceLock; + + #[test] + fn test_literal() { + // a single literal offers no guarantee + test_analyze(lit(true), vec![]) + } + + #[test] + fn test_single() { + // a = "foo" + test_analyze(col("a").eq(lit("foo")), vec![in_guarantee("a", ["foo"])]); + // "foo" = a + test_analyze(lit("foo").eq(col("a")), vec![in_guarantee("a", ["foo"])]); + // a != "foo" + test_analyze( + col("a").not_eq(lit("foo")), + vec![not_in_guarantee("a", ["foo"])], + ); + // "foo" != a + test_analyze( + lit("foo").not_eq(col("a")), + vec![not_in_guarantee("a", ["foo"])], + ); + } + + #[test] + fn test_conjunction_single_column() { + // b = 1 AND b = 2. This is impossible. Ideally this expression could be simplified to false + test_analyze(col("b").eq(lit(1)).and(col("b").eq(lit(2))), vec![]); + // b = 1 AND b != 2 . In theory, this could be simplified to `b = 1`. + test_analyze( + col("b").eq(lit(1)).and(col("b").not_eq(lit(2))), + vec![ + // can only be true of b is 1 and b is not 2 (even though it is redundant) + in_guarantee("b", [1]), + not_in_guarantee("b", [2]), + ], + ); + // b != 1 AND b = 2. In theory, this could be simplified to `b = 2`. + test_analyze( + col("b").not_eq(lit(1)).and(col("b").eq(lit(2))), + vec![ + // can only be true of b is not 1 and b is is 2 (even though it is redundant) + not_in_guarantee("b", [1]), + in_guarantee("b", [2]), + ], + ); + // b != 1 AND b != 2 + test_analyze( + col("b").not_eq(lit(1)).and(col("b").not_eq(lit(2))), + vec![not_in_guarantee("b", [1, 2])], + ); + // b != 1 AND b != 2 and b != 3 + test_analyze( + col("b") + .not_eq(lit(1)) + .and(col("b").not_eq(lit(2))) + .and(col("b").not_eq(lit(3))), + vec![not_in_guarantee("b", [1, 2, 3])], + ); + // b != 1 AND b = 2 and b != 3. Can only be true if b is 2 and b is not in (1, 3) + test_analyze( + col("b") + .not_eq(lit(1)) + .and(col("b").eq(lit(2))) + .and(col("b").not_eq(lit(3))), + vec![not_in_guarantee("b", [1, 3]), in_guarantee("b", [2])], + ); + // b != 1 AND b != 2 and b = 3 (in theory could determine b = 3) + test_analyze( + col("b") + .not_eq(lit(1)) + .and(col("b").not_eq(lit(2))) + .and(col("b").eq(lit(3))), + vec![not_in_guarantee("b", [1, 2]), in_guarantee("b", [3])], + ); + // b != 1 AND b != 2 and b > 3 (to be true, b can't be either 1 or 2 + test_analyze( + col("b") + .not_eq(lit(1)) + .and(col("b").not_eq(lit(2))) + .and(col("b").gt(lit(3))), + vec![not_in_guarantee("b", [1, 2])], + ); + } + + #[test] + fn test_conjunction_multi_column() { + // a = "foo" AND b = 1 + test_analyze( + col("a").eq(lit("foo")).and(col("b").eq(lit(1))), + vec![ + // should find both column guarantees + in_guarantee("a", ["foo"]), + in_guarantee("b", [1]), + ], + ); + // a != "foo" AND b != 1 + test_analyze( + col("a").not_eq(lit("foo")).and(col("b").not_eq(lit(1))), + // should find both column guarantees + vec![not_in_guarantee("a", ["foo"]), not_in_guarantee("b", [1])], + ); + // a = "foo" AND a = "bar" + test_analyze( + col("a").eq(lit("foo")).and(col("a").eq(lit("bar"))), + // this predicate is impossible ( can't be both foo and bar), + vec![], + ); + // a = "foo" AND b != "bar" + test_analyze( + col("a").eq(lit("foo")).and(col("a").not_eq(lit("bar"))), + vec![in_guarantee("a", ["foo"]), not_in_guarantee("a", ["bar"])], + ); + // a != "foo" AND a != "bar" + test_analyze( + col("a").not_eq(lit("foo")).and(col("a").not_eq(lit("bar"))), + // know it isn't "foo" or "bar" + vec![not_in_guarantee("a", ["foo", "bar"])], + ); + // a != "foo" AND a != "bar" and a != "baz" + test_analyze( + col("a") + .not_eq(lit("foo")) + .and(col("a").not_eq(lit("bar"))) + .and(col("a").not_eq(lit("baz"))), + // know it isn't "foo" or "bar" or "baz" + vec![not_in_guarantee("a", ["foo", "bar", "baz"])], + ); + // a = "foo" AND a = "foo" + let expr = col("a").eq(lit("foo")); + test_analyze(expr.clone().and(expr), vec![in_guarantee("a", ["foo"])]); + // b > 5 AND b = 10 (should get an b = 10 guarantee) + test_analyze( + col("b").gt(lit(5)).and(col("b").eq(lit(10))), + vec![in_guarantee("b", [10])], + ); + // b > 10 AND b = 10 (this is impossible) + test_analyze( + col("b").gt(lit(10)).and(col("b").eq(lit(10))), + vec![ + // if b isn't 10, it can not be true (though the expression actually can never be true) + in_guarantee("b", [10]), + ], + ); + // a != "foo" and (a != "bar" OR a != "baz") + test_analyze( + col("a") + .not_eq(lit("foo")) + .and(col("a").not_eq(lit("bar")).or(col("a").not_eq(lit("baz")))), + // a is not foo (we can't represent other knowledge about a) + vec![not_in_guarantee("a", ["foo"])], + ); + } + + #[test] + fn test_conjunction_and_disjunction_single_column() { + // b != 1 AND (b > 2) + test_analyze( + col("b").not_eq(lit(1)).and(col("b").gt(lit(2))), + vec![ + // for the expression to be true, b can not be one + not_in_guarantee("b", [1]), + ], + ); + + // b = 1 AND (b = 2 OR b = 3). Could be simplified to false. + test_analyze( + col("b") + .eq(lit(1)) + .and(col("b").eq(lit(2)).or(col("b").eq(lit(3)))), + vec![ + // in theory, b must be 1 and one of 2,3 for this expression to be true + // which is a logical contradiction + ], + ); + } + + #[test] + fn test_disjunction_single_column() { + // b = 1 OR b = 2 + test_analyze( + col("b").eq(lit(1)).or(col("b").eq(lit(2))), + vec![in_guarantee("b", [1, 2])], + ); + // b != 1 OR b = 2 + test_analyze(col("b").not_eq(lit(1)).or(col("b").eq(lit(2))), vec![]); + // b = 1 OR b != 2 + test_analyze(col("b").eq(lit(1)).or(col("b").not_eq(lit(2))), vec![]); + // b != 1 OR b != 2 + test_analyze(col("b").not_eq(lit(1)).or(col("b").not_eq(lit(2))), vec![]); + // b != 1 OR b != 2 OR b = 3 -- in theory could guarantee that b = 3 + test_analyze( + col("b") + .not_eq(lit(1)) + .or(col("b").not_eq(lit(2))) + .or(lit("b").eq(lit(3))), + vec![], + ); + // b = 1 OR b = 2 OR b = 3 + test_analyze( + col("b") + .eq(lit(1)) + .or(col("b").eq(lit(2))) + .or(col("b").eq(lit(3))), + vec![in_guarantee("b", [1, 2, 3])], + ); + // b = 1 OR b = 2 OR b > 3 -- can't guarantee that the expression is only true if a is in (1, 2) + test_analyze( + col("b") + .eq(lit(1)) + .or(col("b").eq(lit(2))) + .or(lit("b").eq(lit(3))), + vec![], + ); + } + + #[test] + fn test_disjunction_multi_column() { + // a = "foo" OR b = 1 + test_analyze( + col("a").eq(lit("foo")).or(col("b").eq(lit(1))), + // no can't have a single column guarantee (if a = "foo" then b != 1) etc + vec![], + ); + // a != "foo" OR b != 1 + test_analyze( + col("a").not_eq(lit("foo")).or(col("b").not_eq(lit(1))), + // No single column guarantee + vec![], + ); + // a = "foo" OR a = "bar" + test_analyze( + col("a").eq(lit("foo")).or(col("a").eq(lit("bar"))), + vec![in_guarantee("a", ["foo", "bar"])], + ); + // a = "foo" OR a = "foo" + test_analyze( + col("a").eq(lit("foo")).or(col("a").eq(lit("foo"))), + vec![in_guarantee("a", ["foo"])], + ); + // a != "foo" OR a != "bar" + test_analyze( + col("a").not_eq(lit("foo")).or(col("a").not_eq(lit("bar"))), + // can't represent knowledge about a in this case + vec![], + ); + // a = "foo" OR a = "bar" OR a = "baz" + test_analyze( + col("a") + .eq(lit("foo")) + .or(col("a").eq(lit("bar"))) + .or(col("a").eq(lit("baz"))), + vec![in_guarantee("a", ["foo", "bar", "baz"])], + ); + // (a = "foo" OR a = "bar") AND (a = "baz)" + test_analyze( + (col("a").eq(lit("foo")).or(col("a").eq(lit("bar")))) + .and(col("a").eq(lit("baz"))), + // this could potentially be represented as 2 constraints with a more + // sophisticated analysis + vec![], + ); + // (a = "foo" OR a = "bar") AND (b = 1) + test_analyze( + (col("a").eq(lit("foo")).or(col("a").eq(lit("bar")))) + .and(col("b").eq(lit(1))), + vec![in_guarantee("a", ["foo", "bar"]), in_guarantee("b", [1])], + ); + // (a = "foo" OR a = "bar") OR (b = 1) + test_analyze( + col("a") + .eq(lit("foo")) + .or(col("a").eq(lit("bar"))) + .or(col("b").eq(lit(1))), + // can't represent knowledge about a or b in this case + vec![], + ); + } + + // TODO https://github.com/apache/arrow-datafusion/issues/8436 + // a IN (...) + // b NOT IN (...) + + /// Tests that analyzing expr results in the expected guarantees + fn test_analyze(expr: Expr, expected: Vec) { + println!("Begin analyze of {expr}"); + let schema = schema(); + let physical_expr = logical2physical(&expr, &schema); + + let actual = LiteralGuarantee::analyze(&physical_expr); + assert_eq!( + expected, actual, + "expr: {expr}\ + \n\nexpected: {expected:#?}\ + \n\nactual: {actual:#?}\ + \n\nexpr: {expr:#?}\ + \n\nphysical_expr: {physical_expr:#?}" + ); + } + + /// Guarantee that the expression is true if the column is one of the specified values + fn in_guarantee<'a, I, S>(column: &str, literals: I) -> LiteralGuarantee + where + I: IntoIterator, + S: Into + 'a, + { + let literals: Vec<_> = literals.into_iter().map(|s| s.into()).collect(); + LiteralGuarantee::try_new(column, Operator::Eq, literals.iter()).unwrap() + } + + /// Guarantee that the expression is true if the column is NOT any of the specified values + fn not_in_guarantee<'a, I, S>(column: &str, literals: I) -> LiteralGuarantee + where + I: IntoIterator, + S: Into + 'a, + { + let literals: Vec<_> = literals.into_iter().map(|s| s.into()).collect(); + LiteralGuarantee::try_new(column, Operator::NotEq, literals.iter()).unwrap() + } + + /// Convert a logical expression to a physical expression (without any simplification, etc) + fn logical2physical(expr: &Expr, schema: &Schema) -> Arc { + let df_schema = schema.clone().to_dfschema().unwrap(); + let execution_props = ExecutionProps::new(); + create_physical_expr(expr, &df_schema, schema, &execution_props).unwrap() + } + + // Schema for testing + fn schema() -> SchemaRef { + SCHEMA + .get_or_init(|| { + Arc::new(Schema::new(vec![ + Field::new("a", DataType::Utf8, false), + Field::new("b", DataType::Int32, false), + ])) + }) + .clone() + } + + static SCHEMA: OnceLock = OnceLock::new(); +} diff --git a/datafusion/physical-expr/src/utils.rs b/datafusion/physical-expr/src/utils/mod.rs similarity index 96% rename from datafusion/physical-expr/src/utils.rs rename to datafusion/physical-expr/src/utils/mod.rs index 71a7ff5fb778..87ef36558b96 100644 --- a/datafusion/physical-expr/src/utils.rs +++ b/datafusion/physical-expr/src/utils/mod.rs @@ -15,6 +15,9 @@ // specific language governing permissions and limitations // under the License. +mod guarantee; +pub use guarantee::{Guarantee, LiteralGuarantee}; + use std::borrow::Borrow; use std::collections::{HashMap, HashSet}; use std::sync::Arc; @@ -41,25 +44,29 @@ use petgraph::stable_graph::StableGraph; pub fn split_conjunction( predicate: &Arc, ) -> Vec<&Arc> { - split_conjunction_impl(predicate, vec![]) + split_impl(Operator::And, predicate, vec![]) } -fn split_conjunction_impl<'a>( +/// Assume the predicate is in the form of DNF, split the predicate to a Vec of PhysicalExprs. +/// +/// For example, split "a1 = a2 OR b1 <= b2 OR c1 != c2" into ["a1 = a2", "b1 <= b2", "c1 != c2"] +pub fn split_disjunction( + predicate: &Arc, +) -> Vec<&Arc> { + split_impl(Operator::Or, predicate, vec![]) +} + +fn split_impl<'a>( + operator: Operator, predicate: &'a Arc, mut exprs: Vec<&'a Arc>, ) -> Vec<&'a Arc> { match predicate.as_any().downcast_ref::() { - Some(binary) => match binary.op() { - Operator::And => { - let exprs = split_conjunction_impl(binary.left(), exprs); - split_conjunction_impl(binary.right(), exprs) - } - _ => { - exprs.push(predicate); - exprs - } - }, - None => { + Some(binary) if binary.op() == &operator => { + let exprs = split_impl(operator, binary.left(), exprs); + split_impl(operator, binary.right(), exprs) + } + Some(_) | None => { exprs.push(predicate); exprs } From 65b997bc465fe6b9dc6692deebbd2d72da189702 Mon Sep 17 00:00:00 2001 From: Mustafa Akur <106137913+mustafasrepo@users.noreply.github.com> Date: Mon, 18 Dec 2023 22:11:56 +0300 Subject: [PATCH 458/572] [MINOR]: Parametrize sort-preservation tests to exercise all situations (unbounded/bounded sources and flag behavior) (#8575) * Re-introduce unbounded tests with new executor * Remove unnecessary test --- .../src/physical_optimizer/enforce_sorting.rs | 19 +- .../replace_with_order_preserving_variants.rs | 275 +++++++++++------- datafusion/core/src/test/mod.rs | 36 +++ 3 files changed, 208 insertions(+), 122 deletions(-) diff --git a/datafusion/core/src/physical_optimizer/enforce_sorting.rs b/datafusion/core/src/physical_optimizer/enforce_sorting.rs index c0e9b834e66f..2b650a42696b 100644 --- a/datafusion/core/src/physical_optimizer/enforce_sorting.rs +++ b/datafusion/core/src/physical_optimizer/enforce_sorting.rs @@ -769,7 +769,7 @@ mod tests { use crate::physical_plan::repartition::RepartitionExec; use crate::physical_plan::{displayable, get_plan_string, Partitioning}; use crate::prelude::{SessionConfig, SessionContext}; - use crate::test::csv_exec_sorted; + use crate::test::{csv_exec_sorted, stream_exec_ordered}; use arrow::compute::SortOptions; use arrow::datatypes::{DataType, Field, Schema, SchemaRef}; @@ -2141,11 +2141,11 @@ mod tests { } #[tokio::test] - #[ignore] async fn test_with_lost_ordering_unbounded() -> Result<()> { let schema = create_test_schema3()?; let sort_exprs = vec![sort_expr("a", &schema)]; - let source = csv_exec_sorted(&schema, sort_exprs); + // create an unbounded source + let source = stream_exec_ordered(&schema, sort_exprs); let repartition_rr = repartition_exec(source); let repartition_hash = Arc::new(RepartitionExec::try_new( repartition_rr, @@ -2159,25 +2159,24 @@ mod tests { " CoalescePartitionsExec", " RepartitionExec: partitioning=Hash([c@2], 10), input_partitions=10", " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - " CsvExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], infinite_source=true, output_ordering=[a@0 ASC], has_header=false" + " StreamingTableExec: partition_sizes=1, projection=[a, b, c, d, e], infinite_source=true, output_ordering=[a@0 ASC]", ]; let expected_optimized = [ "SortPreservingMergeExec: [a@0 ASC]", " RepartitionExec: partitioning=Hash([c@2], 10), input_partitions=10, preserve_order=true, sort_exprs=a@0 ASC", " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - " CsvExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], infinite_source=true, output_ordering=[a@0 ASC], has_header=false", + " StreamingTableExec: partition_sizes=1, projection=[a, b, c, d, e], infinite_source=true, output_ordering=[a@0 ASC]", ]; assert_optimized!(expected_input, expected_optimized, physical_plan, true); Ok(()) } #[tokio::test] - #[ignore] async fn test_with_lost_ordering_unbounded_parallelize_off() -> Result<()> { let schema = create_test_schema3()?; let sort_exprs = vec![sort_expr("a", &schema)]; - // Make source unbounded - let source = csv_exec_sorted(&schema, sort_exprs); + // create an unbounded source + let source = stream_exec_ordered(&schema, sort_exprs); let repartition_rr = repartition_exec(source); let repartition_hash = Arc::new(RepartitionExec::try_new( repartition_rr, @@ -2190,13 +2189,13 @@ mod tests { " CoalescePartitionsExec", " RepartitionExec: partitioning=Hash([c@2], 10), input_partitions=10", " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - " CsvExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], infinite_source=true, output_ordering=[a@0 ASC], has_header=false" + " StreamingTableExec: partition_sizes=1, projection=[a, b, c, d, e], infinite_source=true, output_ordering=[a@0 ASC]", ]; let expected_optimized = [ "SortPreservingMergeExec: [a@0 ASC]", " RepartitionExec: partitioning=Hash([c@2], 10), input_partitions=10, preserve_order=true, sort_exprs=a@0 ASC", " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - " CsvExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], infinite_source=true, output_ordering=[a@0 ASC], has_header=false", + " StreamingTableExec: partition_sizes=1, projection=[a, b, c, d, e], infinite_source=true, output_ordering=[a@0 ASC]", ]; assert_optimized!(expected_input, expected_optimized, physical_plan, false); Ok(()) diff --git a/datafusion/core/src/physical_optimizer/replace_with_order_preserving_variants.rs b/datafusion/core/src/physical_optimizer/replace_with_order_preserving_variants.rs index 41f2b39978a4..671891be433c 100644 --- a/datafusion/core/src/physical_optimizer/replace_with_order_preserving_variants.rs +++ b/datafusion/core/src/physical_optimizer/replace_with_order_preserving_variants.rs @@ -276,9 +276,6 @@ pub(crate) fn replace_with_order_preserving_variants( mod tests { use super::*; - use crate::datasource::file_format::file_compression_type::FileCompressionType; - use crate::datasource::listing::PartitionedFile; - use crate::datasource::physical_plan::{CsvExec, FileScanConfig}; use crate::physical_plan::coalesce_batches::CoalesceBatchesExec; use crate::physical_plan::coalesce_partitions::CoalescePartitionsExec; use crate::physical_plan::filter::FilterExec; @@ -289,14 +286,16 @@ mod tests { use crate::physical_plan::{displayable, get_plan_string, Partitioning}; use crate::prelude::SessionConfig; + use crate::test::TestStreamPartition; use arrow::compute::SortOptions; use arrow::datatypes::{DataType, Field, Schema, SchemaRef}; use datafusion_common::tree_node::TreeNode; - use datafusion_common::{Result, Statistics}; - use datafusion_execution::object_store::ObjectStoreUrl; + use datafusion_common::Result; use datafusion_expr::{JoinType, Operator}; use datafusion_physical_expr::expressions::{self, col, Column}; use datafusion_physical_expr::PhysicalSortExpr; + use datafusion_physical_plan::streaming::StreamingTableExec; + use rstest::rstest; /// Runs the `replace_with_order_preserving_variants` sub-rule and asserts the plan /// against the original and expected plans. @@ -345,12 +344,15 @@ mod tests { }; } + #[rstest] #[tokio::test] // Searches for a simple sort and a repartition just after it, the second repartition with 1 input partition should not be affected - async fn test_replace_multiple_input_repartition_1() -> Result<()> { + async fn test_replace_multiple_input_repartition_1( + #[values(false, true)] prefer_existing_sort: bool, + ) -> Result<()> { let schema = create_test_schema()?; let sort_exprs = vec![sort_expr("a", &schema)]; - let source = csv_exec_sorted(&schema, sort_exprs); + let source = stream_exec_ordered(&schema, sort_exprs); let repartition = repartition_exec_hash(repartition_exec_round_robin(source)); let sort = sort_exec(vec![sort_expr("a", &schema)], repartition, true); @@ -362,23 +364,31 @@ mod tests { " SortExec: expr=[a@0 ASC NULLS LAST]", " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8", " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", - " CsvExec: file_groups={1 group: [[file_path]]}, projection=[a, c, d], output_ordering=[a@0 ASC NULLS LAST], has_header=true", + " StreamingTableExec: partition_sizes=1, projection=[a, c, d], infinite_source=true, output_ordering=[a@0 ASC NULLS LAST]", ]; let expected_optimized = [ "SortPreservingMergeExec: [a@0 ASC NULLS LAST]", " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8, preserve_order=true, sort_exprs=a@0 ASC NULLS LAST", " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", - " CsvExec: file_groups={1 group: [[file_path]]}, projection=[a, c, d], output_ordering=[a@0 ASC NULLS LAST], has_header=true", + " StreamingTableExec: partition_sizes=1, projection=[a, c, d], infinite_source=true, output_ordering=[a@0 ASC NULLS LAST]", ]; - assert_optimized!(expected_input, expected_optimized, physical_plan, true); + assert_optimized!( + expected_input, + expected_optimized, + physical_plan, + prefer_existing_sort + ); Ok(()) } + #[rstest] #[tokio::test] - async fn test_with_inter_children_change_only() -> Result<()> { + async fn test_with_inter_children_change_only( + #[values(false, true)] prefer_existing_sort: bool, + ) -> Result<()> { let schema = create_test_schema()?; let sort_exprs = vec![sort_expr_default("a", &schema)]; - let source = csv_exec_sorted(&schema, sort_exprs); + let source = stream_exec_ordered(&schema, sort_exprs); let repartition_rr = repartition_exec_round_robin(source); let repartition_hash = repartition_exec_hash(repartition_rr); let coalesce_partitions = coalesce_partitions_exec(repartition_hash); @@ -408,7 +418,7 @@ mod tests { " CoalescePartitionsExec", " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8", " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", - " CsvExec: file_groups={1 group: [[file_path]]}, projection=[a, c, d], output_ordering=[a@0 ASC], has_header=true", + " StreamingTableExec: partition_sizes=1, projection=[a, c, d], infinite_source=true, output_ordering=[a@0 ASC]", ]; let expected_optimized = [ @@ -419,17 +429,25 @@ mod tests { " SortPreservingMergeExec: [a@0 ASC]", " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8, preserve_order=true, sort_exprs=a@0 ASC", " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", - " CsvExec: file_groups={1 group: [[file_path]]}, projection=[a, c, d], output_ordering=[a@0 ASC], has_header=true", + " StreamingTableExec: partition_sizes=1, projection=[a, c, d], infinite_source=true, output_ordering=[a@0 ASC]", ]; - assert_optimized!(expected_input, expected_optimized, physical_plan, true); + assert_optimized!( + expected_input, + expected_optimized, + physical_plan, + prefer_existing_sort + ); Ok(()) } + #[rstest] #[tokio::test] - async fn test_replace_multiple_input_repartition_2() -> Result<()> { + async fn test_replace_multiple_input_repartition_2( + #[values(false, true)] prefer_existing_sort: bool, + ) -> Result<()> { let schema = create_test_schema()?; let sort_exprs = vec![sort_expr("a", &schema)]; - let source = csv_exec_sorted(&schema, sort_exprs); + let source = stream_exec_ordered(&schema, sort_exprs); let repartition_rr = repartition_exec_round_robin(source); let filter = filter_exec(repartition_rr); let repartition_hash = repartition_exec_hash(filter); @@ -444,24 +462,32 @@ mod tests { " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8", " FilterExec: c@1 > 3", " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", - " CsvExec: file_groups={1 group: [[file_path]]}, projection=[a, c, d], output_ordering=[a@0 ASC NULLS LAST], has_header=true", + " StreamingTableExec: partition_sizes=1, projection=[a, c, d], infinite_source=true, output_ordering=[a@0 ASC NULLS LAST]", ]; let expected_optimized = [ "SortPreservingMergeExec: [a@0 ASC NULLS LAST]", " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8, preserve_order=true, sort_exprs=a@0 ASC NULLS LAST", " FilterExec: c@1 > 3", " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", - " CsvExec: file_groups={1 group: [[file_path]]}, projection=[a, c, d], output_ordering=[a@0 ASC NULLS LAST], has_header=true", + " StreamingTableExec: partition_sizes=1, projection=[a, c, d], infinite_source=true, output_ordering=[a@0 ASC NULLS LAST]", ]; - assert_optimized!(expected_input, expected_optimized, physical_plan, true); + assert_optimized!( + expected_input, + expected_optimized, + physical_plan, + prefer_existing_sort + ); Ok(()) } + #[rstest] #[tokio::test] - async fn test_replace_multiple_input_repartition_with_extra_steps() -> Result<()> { + async fn test_replace_multiple_input_repartition_with_extra_steps( + #[values(false, true)] prefer_existing_sort: bool, + ) -> Result<()> { let schema = create_test_schema()?; let sort_exprs = vec![sort_expr("a", &schema)]; - let source = csv_exec_sorted(&schema, sort_exprs); + let source = stream_exec_ordered(&schema, sort_exprs); let repartition_rr = repartition_exec_round_robin(source); let repartition_hash = repartition_exec_hash(repartition_rr); let filter = filter_exec(repartition_hash); @@ -478,7 +504,7 @@ mod tests { " FilterExec: c@1 > 3", " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8", " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", - " CsvExec: file_groups={1 group: [[file_path]]}, projection=[a, c, d], output_ordering=[a@0 ASC NULLS LAST], has_header=true", + " StreamingTableExec: partition_sizes=1, projection=[a, c, d], infinite_source=true, output_ordering=[a@0 ASC NULLS LAST]", ]; let expected_optimized = [ "SortPreservingMergeExec: [a@0 ASC NULLS LAST]", @@ -486,17 +512,25 @@ mod tests { " FilterExec: c@1 > 3", " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8, preserve_order=true, sort_exprs=a@0 ASC NULLS LAST", " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", - " CsvExec: file_groups={1 group: [[file_path]]}, projection=[a, c, d], output_ordering=[a@0 ASC NULLS LAST], has_header=true", + " StreamingTableExec: partition_sizes=1, projection=[a, c, d], infinite_source=true, output_ordering=[a@0 ASC NULLS LAST]", ]; - assert_optimized!(expected_input, expected_optimized, physical_plan, true); + assert_optimized!( + expected_input, + expected_optimized, + physical_plan, + prefer_existing_sort + ); Ok(()) } + #[rstest] #[tokio::test] - async fn test_replace_multiple_input_repartition_with_extra_steps_2() -> Result<()> { + async fn test_replace_multiple_input_repartition_with_extra_steps_2( + #[values(false, true)] prefer_existing_sort: bool, + ) -> Result<()> { let schema = create_test_schema()?; let sort_exprs = vec![sort_expr("a", &schema)]; - let source = csv_exec_sorted(&schema, sort_exprs); + let source = stream_exec_ordered(&schema, sort_exprs); let repartition_rr = repartition_exec_round_robin(source); let coalesce_batches_exec_1 = coalesce_batches_exec(repartition_rr); let repartition_hash = repartition_exec_hash(coalesce_batches_exec_1); @@ -516,7 +550,7 @@ mod tests { " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8", " CoalesceBatchesExec: target_batch_size=8192", " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", - " CsvExec: file_groups={1 group: [[file_path]]}, projection=[a, c, d], output_ordering=[a@0 ASC NULLS LAST], has_header=true", + " StreamingTableExec: partition_sizes=1, projection=[a, c, d], infinite_source=true, output_ordering=[a@0 ASC NULLS LAST]", ]; let expected_optimized = [ "SortPreservingMergeExec: [a@0 ASC NULLS LAST]", @@ -525,17 +559,25 @@ mod tests { " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8, preserve_order=true, sort_exprs=a@0 ASC NULLS LAST", " CoalesceBatchesExec: target_batch_size=8192", " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", - " CsvExec: file_groups={1 group: [[file_path]]}, projection=[a, c, d], output_ordering=[a@0 ASC NULLS LAST], has_header=true", + " StreamingTableExec: partition_sizes=1, projection=[a, c, d], infinite_source=true, output_ordering=[a@0 ASC NULLS LAST]", ]; - assert_optimized!(expected_input, expected_optimized, physical_plan, true); + assert_optimized!( + expected_input, + expected_optimized, + physical_plan, + prefer_existing_sort + ); Ok(()) } + #[rstest] #[tokio::test] - async fn test_not_replacing_when_no_need_to_preserve_sorting() -> Result<()> { + async fn test_not_replacing_when_no_need_to_preserve_sorting( + #[values(false, true)] prefer_existing_sort: bool, + ) -> Result<()> { let schema = create_test_schema()?; let sort_exprs = vec![sort_expr("a", &schema)]; - let source = csv_exec_sorted(&schema, sort_exprs); + let source = stream_exec_ordered(&schema, sort_exprs); let repartition_rr = repartition_exec_round_robin(source); let repartition_hash = repartition_exec_hash(repartition_rr); let filter = filter_exec(repartition_hash); @@ -550,7 +592,7 @@ mod tests { " FilterExec: c@1 > 3", " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8", " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", - " CsvExec: file_groups={1 group: [[file_path]]}, projection=[a, c, d], output_ordering=[a@0 ASC NULLS LAST], has_header=true", + " StreamingTableExec: partition_sizes=1, projection=[a, c, d], infinite_source=true, output_ordering=[a@0 ASC NULLS LAST]", ]; let expected_optimized = [ "CoalescePartitionsExec", @@ -558,17 +600,25 @@ mod tests { " FilterExec: c@1 > 3", " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8", " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", - " CsvExec: file_groups={1 group: [[file_path]]}, projection=[a, c, d], output_ordering=[a@0 ASC NULLS LAST], has_header=true", + " StreamingTableExec: partition_sizes=1, projection=[a, c, d], infinite_source=true, output_ordering=[a@0 ASC NULLS LAST]", ]; - assert_optimized!(expected_input, expected_optimized, physical_plan); + assert_optimized!( + expected_input, + expected_optimized, + physical_plan, + prefer_existing_sort + ); Ok(()) } + #[rstest] #[tokio::test] - async fn test_with_multiple_replacable_repartitions() -> Result<()> { + async fn test_with_multiple_replacable_repartitions( + #[values(false, true)] prefer_existing_sort: bool, + ) -> Result<()> { let schema = create_test_schema()?; let sort_exprs = vec![sort_expr("a", &schema)]; - let source = csv_exec_sorted(&schema, sort_exprs); + let source = stream_exec_ordered(&schema, sort_exprs); let repartition_rr = repartition_exec_round_robin(source); let repartition_hash = repartition_exec_hash(repartition_rr); let filter = filter_exec(repartition_hash); @@ -587,7 +637,7 @@ mod tests { " FilterExec: c@1 > 3", " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8", " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", - " CsvExec: file_groups={1 group: [[file_path]]}, projection=[a, c, d], output_ordering=[a@0 ASC NULLS LAST], has_header=true", + " StreamingTableExec: partition_sizes=1, projection=[a, c, d], infinite_source=true, output_ordering=[a@0 ASC NULLS LAST]", ]; let expected_optimized = [ "SortPreservingMergeExec: [a@0 ASC NULLS LAST]", @@ -596,17 +646,25 @@ mod tests { " FilterExec: c@1 > 3", " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8, preserve_order=true, sort_exprs=a@0 ASC NULLS LAST", " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", - " CsvExec: file_groups={1 group: [[file_path]]}, projection=[a, c, d], output_ordering=[a@0 ASC NULLS LAST], has_header=true", + " StreamingTableExec: partition_sizes=1, projection=[a, c, d], infinite_source=true, output_ordering=[a@0 ASC NULLS LAST]", ]; - assert_optimized!(expected_input, expected_optimized, physical_plan, true); + assert_optimized!( + expected_input, + expected_optimized, + physical_plan, + prefer_existing_sort + ); Ok(()) } + #[rstest] #[tokio::test] - async fn test_not_replace_with_different_orderings() -> Result<()> { + async fn test_not_replace_with_different_orderings( + #[values(false, true)] prefer_existing_sort: bool, + ) -> Result<()> { let schema = create_test_schema()?; let sort_exprs = vec![sort_expr("a", &schema)]; - let source = csv_exec_sorted(&schema, sort_exprs); + let source = stream_exec_ordered(&schema, sort_exprs); let repartition_rr = repartition_exec_round_robin(source); let repartition_hash = repartition_exec_hash(repartition_rr); let sort = sort_exec( @@ -625,24 +683,32 @@ mod tests { " SortExec: expr=[c@1 ASC]", " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8", " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", - " CsvExec: file_groups={1 group: [[file_path]]}, projection=[a, c, d], output_ordering=[a@0 ASC NULLS LAST], has_header=true", + " StreamingTableExec: partition_sizes=1, projection=[a, c, d], infinite_source=true, output_ordering=[a@0 ASC NULLS LAST]", ]; let expected_optimized = [ "SortPreservingMergeExec: [c@1 ASC]", " SortExec: expr=[c@1 ASC]", " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8", " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", - " CsvExec: file_groups={1 group: [[file_path]]}, projection=[a, c, d], output_ordering=[a@0 ASC NULLS LAST], has_header=true", + " StreamingTableExec: partition_sizes=1, projection=[a, c, d], infinite_source=true, output_ordering=[a@0 ASC NULLS LAST]", ]; - assert_optimized!(expected_input, expected_optimized, physical_plan); + assert_optimized!( + expected_input, + expected_optimized, + physical_plan, + prefer_existing_sort + ); Ok(()) } + #[rstest] #[tokio::test] - async fn test_with_lost_ordering() -> Result<()> { + async fn test_with_lost_ordering( + #[values(false, true)] prefer_existing_sort: bool, + ) -> Result<()> { let schema = create_test_schema()?; let sort_exprs = vec![sort_expr("a", &schema)]; - let source = csv_exec_sorted(&schema, sort_exprs); + let source = stream_exec_ordered(&schema, sort_exprs); let repartition_rr = repartition_exec_round_robin(source); let repartition_hash = repartition_exec_hash(repartition_rr); let coalesce_partitions = coalesce_partitions_exec(repartition_hash); @@ -654,23 +720,31 @@ mod tests { " CoalescePartitionsExec", " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8", " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", - " CsvExec: file_groups={1 group: [[file_path]]}, projection=[a, c, d], output_ordering=[a@0 ASC NULLS LAST], has_header=true", + " StreamingTableExec: partition_sizes=1, projection=[a, c, d], infinite_source=true, output_ordering=[a@0 ASC NULLS LAST]", ]; let expected_optimized = [ "SortPreservingMergeExec: [a@0 ASC NULLS LAST]", " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8, preserve_order=true, sort_exprs=a@0 ASC NULLS LAST", " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", - " CsvExec: file_groups={1 group: [[file_path]]}, projection=[a, c, d], output_ordering=[a@0 ASC NULLS LAST], has_header=true", + " StreamingTableExec: partition_sizes=1, projection=[a, c, d], infinite_source=true, output_ordering=[a@0 ASC NULLS LAST]", ]; - assert_optimized!(expected_input, expected_optimized, physical_plan, true); + assert_optimized!( + expected_input, + expected_optimized, + physical_plan, + prefer_existing_sort + ); Ok(()) } + #[rstest] #[tokio::test] - async fn test_with_lost_and_kept_ordering() -> Result<()> { + async fn test_with_lost_and_kept_ordering( + #[values(false, true)] prefer_existing_sort: bool, + ) -> Result<()> { let schema = create_test_schema()?; let sort_exprs = vec![sort_expr("a", &schema)]; - let source = csv_exec_sorted(&schema, sort_exprs); + let source = stream_exec_ordered(&schema, sort_exprs); let repartition_rr = repartition_exec_round_robin(source); let repartition_hash = repartition_exec_hash(repartition_rr); let coalesce_partitions = coalesce_partitions_exec(repartition_hash); @@ -700,7 +774,7 @@ mod tests { " CoalescePartitionsExec", " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8", " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", - " CsvExec: file_groups={1 group: [[file_path]]}, projection=[a, c, d], output_ordering=[a@0 ASC NULLS LAST], has_header=true", + " StreamingTableExec: partition_sizes=1, projection=[a, c, d], infinite_source=true, output_ordering=[a@0 ASC NULLS LAST]", ]; let expected_optimized = [ @@ -712,25 +786,33 @@ mod tests { " CoalescePartitionsExec", " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8", " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", - " CsvExec: file_groups={1 group: [[file_path]]}, projection=[a, c, d], output_ordering=[a@0 ASC NULLS LAST], has_header=true", + " StreamingTableExec: partition_sizes=1, projection=[a, c, d], infinite_source=true, output_ordering=[a@0 ASC NULLS LAST]", ]; - assert_optimized!(expected_input, expected_optimized, physical_plan, true); + assert_optimized!( + expected_input, + expected_optimized, + physical_plan, + prefer_existing_sort + ); Ok(()) } + #[rstest] #[tokio::test] - async fn test_with_multiple_child_trees() -> Result<()> { + async fn test_with_multiple_child_trees( + #[values(false, true)] prefer_existing_sort: bool, + ) -> Result<()> { let schema = create_test_schema()?; let left_sort_exprs = vec![sort_expr("a", &schema)]; - let left_source = csv_exec_sorted(&schema, left_sort_exprs); + let left_source = stream_exec_ordered(&schema, left_sort_exprs); let left_repartition_rr = repartition_exec_round_robin(left_source); let left_repartition_hash = repartition_exec_hash(left_repartition_rr); let left_coalesce_partitions = Arc::new(CoalesceBatchesExec::new(left_repartition_hash, 4096)); let right_sort_exprs = vec![sort_expr("a", &schema)]; - let right_source = csv_exec_sorted(&schema, right_sort_exprs); + let right_source = stream_exec_ordered(&schema, right_sort_exprs); let right_repartition_rr = repartition_exec_round_robin(right_source); let right_repartition_hash = repartition_exec_hash(right_repartition_rr); let right_coalesce_partitions = @@ -756,11 +838,11 @@ mod tests { " CoalesceBatchesExec: target_batch_size=4096", " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8", " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", - " CsvExec: file_groups={1 group: [[file_path]]}, projection=[a, c, d], output_ordering=[a@0 ASC NULLS LAST], has_header=true", + " StreamingTableExec: partition_sizes=1, projection=[a, c, d], infinite_source=true, output_ordering=[a@0 ASC NULLS LAST]", " CoalesceBatchesExec: target_batch_size=4096", " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8", " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", - " CsvExec: file_groups={1 group: [[file_path]]}, projection=[a, c, d], output_ordering=[a@0 ASC NULLS LAST], has_header=true", + " StreamingTableExec: partition_sizes=1, projection=[a, c, d], infinite_source=true, output_ordering=[a@0 ASC NULLS LAST]", ]; let expected_optimized = [ @@ -770,41 +852,18 @@ mod tests { " CoalesceBatchesExec: target_batch_size=4096", " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8", " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", - " CsvExec: file_groups={1 group: [[file_path]]}, projection=[a, c, d], output_ordering=[a@0 ASC NULLS LAST], has_header=true", + " StreamingTableExec: partition_sizes=1, projection=[a, c, d], infinite_source=true, output_ordering=[a@0 ASC NULLS LAST]", " CoalesceBatchesExec: target_batch_size=4096", " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8", " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", - " CsvExec: file_groups={1 group: [[file_path]]}, projection=[a, c, d], output_ordering=[a@0 ASC NULLS LAST], has_header=true", + " StreamingTableExec: partition_sizes=1, projection=[a, c, d], infinite_source=true, output_ordering=[a@0 ASC NULLS LAST]", ]; - assert_optimized!(expected_input, expected_optimized, physical_plan); - Ok(()) - } - - #[tokio::test] - async fn test_with_bounded_input() -> Result<()> { - let schema = create_test_schema()?; - let sort_exprs = vec![sort_expr("a", &schema)]; - let source = csv_exec_sorted(&schema, sort_exprs); - let repartition = repartition_exec_hash(repartition_exec_round_robin(source)); - let sort = sort_exec(vec![sort_expr("a", &schema)], repartition, true); - - let physical_plan = - sort_preserving_merge_exec(vec![sort_expr("a", &schema)], sort); - - let expected_input = [ - "SortPreservingMergeExec: [a@0 ASC NULLS LAST]", - " SortExec: expr=[a@0 ASC NULLS LAST]", - " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8", - " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", - " CsvExec: file_groups={1 group: [[file_path]]}, projection=[a, c, d], output_ordering=[a@0 ASC NULLS LAST], has_header=true", - ]; - let expected_optimized = [ - "SortPreservingMergeExec: [a@0 ASC NULLS LAST]", - " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8, preserve_order=true, sort_exprs=a@0 ASC NULLS LAST", - " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", - " CsvExec: file_groups={1 group: [[file_path]]}, projection=[a, c, d], output_ordering=[a@0 ASC NULLS LAST], has_header=true", - ]; - assert_optimized!(expected_input, expected_optimized, physical_plan, true); + assert_optimized!( + expected_input, + expected_optimized, + physical_plan, + prefer_existing_sort + ); Ok(()) } @@ -928,32 +987,24 @@ mod tests { // creates a csv exec source for the test purposes // projection and has_header parameters are given static due to testing needs - fn csv_exec_sorted( + fn stream_exec_ordered( schema: &SchemaRef, sort_exprs: impl IntoIterator, ) -> Arc { let sort_exprs = sort_exprs.into_iter().collect(); let projection: Vec = vec![0, 2, 3]; - Arc::new(CsvExec::new( - FileScanConfig { - object_store_url: ObjectStoreUrl::parse("test:///").unwrap(), - file_schema: schema.clone(), - file_groups: vec![vec![PartitionedFile::new( - "file_path".to_string(), - 100, - )]], - statistics: Statistics::new_unknown(schema), - projection: Some(projection), - limit: None, - table_partition_cols: vec![], - output_ordering: vec![sort_exprs], - }, - true, - 0, - b'"', - None, - FileCompressionType::UNCOMPRESSED, - )) + Arc::new( + StreamingTableExec::try_new( + schema.clone(), + vec![Arc::new(TestStreamPartition { + schema: schema.clone(), + }) as _], + Some(&projection), + vec![sort_exprs], + true, + ) + .unwrap(), + ) } } diff --git a/datafusion/core/src/test/mod.rs b/datafusion/core/src/test/mod.rs index 8770c0c4238a..7a63466a3906 100644 --- a/datafusion/core/src/test/mod.rs +++ b/datafusion/core/src/test/mod.rs @@ -49,6 +49,7 @@ use datafusion_physical_plan::{DisplayAs, DisplayFormatType}; use bzip2::write::BzEncoder; #[cfg(feature = "compression")] use bzip2::Compression as BzCompression; +use datafusion_physical_plan::streaming::{PartitionStream, StreamingTableExec}; #[cfg(feature = "compression")] use flate2::write::GzEncoder; #[cfg(feature = "compression")] @@ -298,6 +299,41 @@ pub fn csv_exec_sorted( )) } +// construct a stream partition for test purposes +pub(crate) struct TestStreamPartition { + pub schema: SchemaRef, +} + +impl PartitionStream for TestStreamPartition { + fn schema(&self) -> &SchemaRef { + &self.schema + } + fn execute(&self, _ctx: Arc) -> SendableRecordBatchStream { + unreachable!() + } +} + +/// Create an unbounded stream exec +pub fn stream_exec_ordered( + schema: &SchemaRef, + sort_exprs: impl IntoIterator, +) -> Arc { + let sort_exprs = sort_exprs.into_iter().collect(); + + Arc::new( + StreamingTableExec::try_new( + schema.clone(), + vec![Arc::new(TestStreamPartition { + schema: schema.clone(), + }) as _], + None, + vec![sort_exprs], + true, + ) + .unwrap(), + ) +} + /// A mock execution plan that simply returns the provided statistics #[derive(Debug, Clone)] pub struct StatisticsExec { From fc46b36a4078a7fdababfc2d3735e83caf1326f7 Mon Sep 17 00:00:00 2001 From: Andrew Lamb Date: Mon, 18 Dec 2023 14:27:32 -0500 Subject: [PATCH 459/572] Minor: Add some comments to scalar_udf example (#8576) * refine example * clippy --- datafusion-examples/examples/simple_udf.rs | 30 +++++++++++----------- 1 file changed, 15 insertions(+), 15 deletions(-) diff --git a/datafusion-examples/examples/simple_udf.rs b/datafusion-examples/examples/simple_udf.rs index dba4385b8eea..591991786515 100644 --- a/datafusion-examples/examples/simple_udf.rs +++ b/datafusion-examples/examples/simple_udf.rs @@ -29,23 +29,23 @@ use datafusion::{error::Result, physical_plan::functions::make_scalar_function}; use datafusion_common::cast::as_float64_array; use std::sync::Arc; -// create local execution context with an in-memory table +/// create local execution context with an in-memory table: +/// +/// ```text +/// +-----+-----+ +/// | a | b | +/// +-----+-----+ +/// | 2.1 | 1.0 | +/// | 3.1 | 2.0 | +/// | 4.1 | 3.0 | +/// | 5.1 | 4.0 | +/// +-----+-----+ +/// ``` fn create_context() -> Result { - use datafusion::arrow::datatypes::{Field, Schema}; - // define a schema. - let schema = Arc::new(Schema::new(vec![ - Field::new("a", DataType::Float32, false), - Field::new("b", DataType::Float64, false), - ])); - // define data. - let batch = RecordBatch::try_new( - schema, - vec![ - Arc::new(Float32Array::from(vec![2.1, 3.1, 4.1, 5.1])), - Arc::new(Float64Array::from(vec![1.0, 2.0, 3.0, 4.0])), - ], - )?; + let a: ArrayRef = Arc::new(Float32Array::from(vec![2.1, 3.1, 4.1, 5.1])); + let b: ArrayRef = Arc::new(Float64Array::from(vec![1.0, 2.0, 3.0, 4.0])); + let batch = RecordBatch::try_from_iter(vec![("a", a), ("b", b)])?; // declare a new context. In spark API, this corresponds to a new spark SQLsession let ctx = SessionContext::new(); From 1935c58f5cffe123839bae4e9d77a128351728e1 Mon Sep 17 00:00:00 2001 From: Jay Zhan Date: Tue, 19 Dec 2023 04:17:47 +0800 Subject: [PATCH 460/572] Move Coercion for MakeArray to `coerce_arguments_for_signature` and introduce another one for ArrayAppend (#8317) * Signature for array_append and make_array Signed-off-by: jayzhan211 * combine variadicequal and coerced to equal Signed-off-by: jayzhan211 * follow postgres style on array_append(null, T) Signed-off-by: jayzhan211 * update comment for ArrayAndElement Signed-off-by: jayzhan211 * remove test Signed-off-by: jayzhan211 * add more test Signed-off-by: jayzhan211 --------- Signed-off-by: jayzhan211 Co-authored-by: Andrew Lamb --- datafusion/common/src/utils.rs | 38 ++++++++++ datafusion/expr/src/built_in_function.rs | 13 ++-- datafusion/expr/src/signature.rs | 20 ++++- .../expr/src/type_coercion/functions.rs | 74 +++++++++++++++++-- .../optimizer/src/analyzer/type_coercion.rs | 34 --------- .../physical-expr/src/array_expressions.rs | 25 ++----- datafusion/sqllogictest/test_files/array.slt | 51 +++++++++---- 7 files changed, 176 insertions(+), 79 deletions(-) diff --git a/datafusion/common/src/utils.rs b/datafusion/common/src/utils.rs index fecab8835e50..2d38ca21829b 100644 --- a/datafusion/common/src/utils.rs +++ b/datafusion/common/src/utils.rs @@ -342,6 +342,8 @@ pub fn longest_consecutive_prefix>( count } +/// Array Utils + /// Wrap an array into a single element `ListArray`. /// For example `[1, 2, 3]` would be converted into `[[1, 2, 3]]` pub fn array_into_list_array(arr: ArrayRef) -> ListArray { @@ -429,6 +431,42 @@ pub fn base_type(data_type: &DataType) -> DataType { } } +/// A helper function to coerce base type in List. +/// +/// Example +/// ``` +/// use arrow::datatypes::{DataType, Field}; +/// use datafusion_common::utils::coerced_type_with_base_type_only; +/// use std::sync::Arc; +/// +/// let data_type = DataType::List(Arc::new(Field::new("item", DataType::Int32, true))); +/// let base_type = DataType::Float64; +/// let coerced_type = coerced_type_with_base_type_only(&data_type, &base_type); +/// assert_eq!(coerced_type, DataType::List(Arc::new(Field::new("item", DataType::Float64, true)))); +pub fn coerced_type_with_base_type_only( + data_type: &DataType, + base_type: &DataType, +) -> DataType { + match data_type { + DataType::List(field) => { + let data_type = match field.data_type() { + DataType::List(_) => { + coerced_type_with_base_type_only(field.data_type(), base_type) + } + _ => base_type.to_owned(), + }; + + DataType::List(Arc::new(Field::new( + field.name(), + data_type, + field.is_nullable(), + ))) + } + + _ => base_type.clone(), + } +} + /// Compute the number of dimensions in a list data type. pub fn list_ndims(data_type: &DataType) -> u64 { if let DataType::List(field) = data_type { diff --git a/datafusion/expr/src/built_in_function.rs b/datafusion/expr/src/built_in_function.rs index fd899289ac82..289704ed98f8 100644 --- a/datafusion/expr/src/built_in_function.rs +++ b/datafusion/expr/src/built_in_function.rs @@ -915,10 +915,17 @@ impl BuiltinScalarFunction { // for now, the list is small, as we do not have many built-in functions. match self { - BuiltinScalarFunction::ArrayAppend => Signature::any(2, self.volatility()), BuiltinScalarFunction::ArraySort => { Signature::variadic_any(self.volatility()) } + BuiltinScalarFunction::ArrayAppend => Signature { + type_signature: ArrayAndElement, + volatility: self.volatility(), + }, + BuiltinScalarFunction::MakeArray => { + // 0 or more arguments of arbitrary type + Signature::one_of(vec![VariadicEqual, Any(0)], self.volatility()) + } BuiltinScalarFunction::ArrayPopFront => Signature::any(1, self.volatility()), BuiltinScalarFunction::ArrayPopBack => Signature::any(1, self.volatility()), BuiltinScalarFunction::ArrayConcat => { @@ -958,10 +965,6 @@ impl BuiltinScalarFunction { BuiltinScalarFunction::ArrayIntersect => Signature::any(2, self.volatility()), BuiltinScalarFunction::ArrayUnion => Signature::any(2, self.volatility()), BuiltinScalarFunction::Cardinality => Signature::any(1, self.volatility()), - BuiltinScalarFunction::MakeArray => { - // 0 or more arguments of arbitrary type - Signature::one_of(vec![VariadicAny, Any(0)], self.volatility()) - } BuiltinScalarFunction::Range => Signature::one_of( vec![ Exact(vec![Int64]), diff --git a/datafusion/expr/src/signature.rs b/datafusion/expr/src/signature.rs index 685601523f9b..3f07c300e196 100644 --- a/datafusion/expr/src/signature.rs +++ b/datafusion/expr/src/signature.rs @@ -91,11 +91,14 @@ pub enum TypeSignature { /// DataFusion attempts to coerce all argument types to match the first argument's type /// /// # Examples - /// A function such as `array` is `VariadicEqual` + /// Given types in signature should be coericible to the same final type. + /// A function such as `make_array` is `VariadicEqual`. + /// + /// `make_array(i32, i64) -> make_array(i64, i64)` VariadicEqual, /// One or more arguments with arbitrary types VariadicAny, - /// fixed number of arguments of an arbitrary but equal type out of a list of valid types. + /// Fixed number of arguments of an arbitrary but equal type out of a list of valid types. /// /// # Examples /// 1. A function of one argument of f64 is `Uniform(1, vec![DataType::Float64])` @@ -113,6 +116,12 @@ pub enum TypeSignature { /// Function `make_array` takes 0 or more arguments with arbitrary types, its `TypeSignature` /// is `OneOf(vec![Any(0), VariadicAny])`. OneOf(Vec), + /// Specialized Signature for ArrayAppend and similar functions + /// The first argument should be List/LargeList, and the second argument should be non-list or list. + /// The second argument's list dimension should be one dimension less than the first argument's list dimension. + /// List dimension of the List/LargeList is equivalent to the number of List. + /// List dimension of the non-list is 0. + ArrayAndElement, } impl TypeSignature { @@ -136,11 +145,16 @@ impl TypeSignature { .collect::>() .join(", ")] } - TypeSignature::VariadicEqual => vec!["T, .., T".to_string()], + TypeSignature::VariadicEqual => { + vec!["CoercibleT, .., CoercibleT".to_string()] + } TypeSignature::VariadicAny => vec!["Any, .., Any".to_string()], TypeSignature::OneOf(sigs) => { sigs.iter().flat_map(|s| s.to_string_repr()).collect() } + TypeSignature::ArrayAndElement => { + vec!["ArrayAndElement(List, T)".to_string()] + } } } diff --git a/datafusion/expr/src/type_coercion/functions.rs b/datafusion/expr/src/type_coercion/functions.rs index 79b574238495..f95a30e025b4 100644 --- a/datafusion/expr/src/type_coercion/functions.rs +++ b/datafusion/expr/src/type_coercion/functions.rs @@ -21,7 +21,10 @@ use arrow::{ compute::can_cast_types, datatypes::{DataType, TimeUnit}, }; -use datafusion_common::{plan_err, DataFusionError, Result}; +use datafusion_common::utils::list_ndims; +use datafusion_common::{internal_err, plan_err, DataFusionError, Result}; + +use super::binary::comparison_coercion; /// Performs type coercion for function arguments. /// @@ -86,16 +89,66 @@ fn get_valid_types( .map(|valid_type| (0..*number).map(|_| valid_type.clone()).collect()) .collect(), TypeSignature::VariadicEqual => { - // one entry with the same len as current_types, whose type is `current_types[0]`. - vec![current_types - .iter() - .map(|_| current_types[0].clone()) - .collect()] + let new_type = current_types.iter().skip(1).try_fold( + current_types.first().unwrap().clone(), + |acc, x| { + let coerced_type = comparison_coercion(&acc, x); + if let Some(coerced_type) = coerced_type { + Ok(coerced_type) + } else { + internal_err!("Coercion from {acc:?} to {x:?} failed.") + } + }, + ); + + match new_type { + Ok(new_type) => vec![vec![new_type; current_types.len()]], + Err(e) => return Err(e), + } } TypeSignature::VariadicAny => { vec![current_types.to_vec()] } + TypeSignature::Exact(valid_types) => vec![valid_types.clone()], + TypeSignature::ArrayAndElement => { + if current_types.len() != 2 { + return Ok(vec![vec![]]); + } + + let array_type = ¤t_types[0]; + let elem_type = ¤t_types[1]; + + // We follow Postgres on `array_append(Null, T)`, which is not valid. + if array_type.eq(&DataType::Null) { + return Ok(vec![vec![]]); + } + + // We need to find the coerced base type, mainly for cases like: + // `array_append(List(null), i64)` -> `List(i64)` + let array_base_type = datafusion_common::utils::base_type(array_type); + let elem_base_type = datafusion_common::utils::base_type(elem_type); + let new_base_type = comparison_coercion(&array_base_type, &elem_base_type); + + if new_base_type.is_none() { + return internal_err!( + "Coercion from {array_base_type:?} to {elem_base_type:?} not supported." + ); + } + let new_base_type = new_base_type.unwrap(); + + let array_type = datafusion_common::utils::coerced_type_with_base_type_only( + array_type, + &new_base_type, + ); + + if let DataType::List(ref field) = array_type { + let elem_type = field.data_type(); + return Ok(vec![vec![array_type.clone(), elem_type.to_owned()]]); + } else { + return Ok(vec![vec![]]); + } + } TypeSignature::Any(number) => { if current_types.len() != *number { return plan_err!( @@ -241,6 +294,15 @@ fn coerced_from<'a>( Utf8 | LargeUtf8 => Some(type_into.clone()), Null if can_cast_types(type_from, type_into) => Some(type_into.clone()), + // Only accept list with the same number of dimensions unless the type is Null. + // List with different dimensions should be handled in TypeSignature or other places before this. + List(_) + if datafusion_common::utils::base_type(type_from).eq(&Null) + || list_ndims(type_from) == list_ndims(type_into) => + { + Some(type_into.clone()) + } + Timestamp(unit, Some(tz)) if tz.as_ref() == TIMEZONE_WILDCARD => { match type_from { Timestamp(_, Some(from_tz)) => { diff --git a/datafusion/optimizer/src/analyzer/type_coercion.rs b/datafusion/optimizer/src/analyzer/type_coercion.rs index 91611251d9dd..c5e1180b9f97 100644 --- a/datafusion/optimizer/src/analyzer/type_coercion.rs +++ b/datafusion/optimizer/src/analyzer/type_coercion.rs @@ -590,26 +590,6 @@ fn coerce_arguments_for_fun( .collect::>>()?; } - if *fun == BuiltinScalarFunction::MakeArray { - // Find the final data type for the function arguments - let current_types = expressions - .iter() - .map(|e| e.get_type(schema)) - .collect::>>()?; - - let new_type = current_types - .iter() - .skip(1) - .fold(current_types.first().unwrap().clone(), |acc, x| { - comparison_coercion(&acc, x).unwrap_or(acc) - }); - - return expressions - .iter() - .zip(current_types) - .map(|(expr, from_type)| cast_array_expr(expr, &from_type, &new_type, schema)) - .collect(); - } Ok(expressions) } @@ -618,20 +598,6 @@ fn cast_expr(expr: &Expr, to_type: &DataType, schema: &DFSchema) -> Result expr.clone().cast_to(to_type, schema) } -/// Cast array `expr` to the specified type, if possible -fn cast_array_expr( - expr: &Expr, - from_type: &DataType, - to_type: &DataType, - schema: &DFSchema, -) -> Result { - if from_type.equals_datatype(&DataType::Null) { - Ok(expr.clone()) - } else { - cast_expr(expr, to_type, schema) - } -} - /// Returns the coerced exprs for each `input_exprs`. /// Get the coerced data type from `aggregate_rule::coerce_types` and add `try_cast` if the /// data type of `input_exprs` need to be coerced. diff --git a/datafusion/physical-expr/src/array_expressions.rs b/datafusion/physical-expr/src/array_expressions.rs index 7ccf58af832d..98c9aee8940f 100644 --- a/datafusion/physical-expr/src/array_expressions.rs +++ b/datafusion/physical-expr/src/array_expressions.rs @@ -361,7 +361,8 @@ pub fn make_array(arrays: &[ArrayRef]) -> Result { match data_type { // Either an empty array or all nulls: DataType::Null => { - let array = new_null_array(&DataType::Null, arrays.len()); + let array = + new_null_array(&DataType::Null, arrays.iter().map(|a| a.len()).sum()); Ok(Arc::new(array_into_list_array(array))) } DataType::LargeList(..) => array_array::(arrays, data_type), @@ -827,10 +828,14 @@ pub fn array_append(args: &[ArrayRef]) -> Result { let list_array = as_list_array(&args[0])?; let element_array = &args[1]; - check_datatypes("array_append", &[list_array.values(), element_array])?; let res = match list_array.value_type() { DataType::List(_) => concat_internal(args)?, - DataType::Null => return make_array(&[element_array.to_owned()]), + DataType::Null => { + return make_array(&[ + list_array.values().to_owned(), + element_array.to_owned(), + ]); + } data_type => { return general_append_and_prepend( list_array, @@ -2284,18 +2289,4 @@ mod tests { expected_dim ); } - - #[test] - fn test_check_invalid_datatypes() { - let data = vec![Some(vec![Some(1), Some(2), Some(3)])]; - let list_array = - Arc::new(ListArray::from_iter_primitive::(data)) as ArrayRef; - let int64_array = Arc::new(StringArray::from(vec![Some("string")])) as ArrayRef; - - let args = [list_array.clone(), int64_array.clone()]; - - let array = array_append(&args); - - assert_eq!(array.unwrap_err().strip_backtrace(), "Error during planning: array_append received incompatible types: '[Int64, Utf8]'."); - } } diff --git a/datafusion/sqllogictest/test_files/array.slt b/datafusion/sqllogictest/test_files/array.slt index 210739aa51da..640f5064eae6 100644 --- a/datafusion/sqllogictest/test_files/array.slt +++ b/datafusion/sqllogictest/test_files/array.slt @@ -297,10 +297,8 @@ AS VALUES (make_array([28, 29, 30], [31, 32, 33], [34, 35, 36], [28, 29, 30], [31, 32, 33], [34, 35, 36], [28, 29, 30], [31, 32, 33], [34, 35, 36], [28, 29, 30]), [28, 29, 30], [37, 38, 39], 10) ; -query ? +query error select [1, true, null] ----- -[1, 1, ] query error DataFusion error: This feature is not implemented: ScalarFunctions without MakeArray are not supported: now() SELECT [now()] @@ -1253,18 +1251,43 @@ select list_sort(make_array(1, 3, null, 5, NULL, -5)), list_sort(make_array(1, 3 ## array_append (aliases: `list_append`, `array_push_back`, `list_push_back`) -# TODO: array_append with NULLs -# array_append scalar function #1 -# query ? -# select array_append(make_array(), 4); -# ---- -# [4] +# array_append with NULLs -# array_append scalar function #2 -# query ?? -# select array_append(make_array(), make_array()), array_append(make_array(), make_array(4)); -# ---- -# [[]] [[4]] +query error +select array_append(null, 1); + +query error +select array_append(null, [2, 3]); + +query error +select array_append(null, [[4]]); + +query ???? +select + array_append(make_array(), 4), + array_append(make_array(), null), + array_append(make_array(1, null, 3), 4), + array_append(make_array(null, null), 1) +; +---- +[4] [] [1, , 3, 4] [, , 1] + +# test invalid (non-null) +query error +select array_append(1, 2); + +query error +select array_append(1, [2]); + +query error +select array_append([1], [2]); + +query ?? +select + array_append(make_array(make_array(1, null, 3)), make_array(null)), + array_append(make_array(make_array(1, null, 3)), null); +---- +[[1, , 3], []] [[1, , 3], ] # array_append scalar function #3 query ??? From d220bf47f944dd019d6b1e5b2741535a3f90204f Mon Sep 17 00:00:00 2001 From: Alex Huang Date: Mon, 18 Dec 2023 21:22:34 +0100 Subject: [PATCH 461/572] support LargeList in array_positions (#8571) --- .../physical-expr/src/array_expressions.rs | 19 ++++++-- datafusion/sqllogictest/test_files/array.slt | 43 +++++++++++++++++++ 2 files changed, 58 insertions(+), 4 deletions(-) diff --git a/datafusion/physical-expr/src/array_expressions.rs b/datafusion/physical-expr/src/array_expressions.rs index 98c9aee8940f..cc4b2899fcb1 100644 --- a/datafusion/physical-expr/src/array_expressions.rs +++ b/datafusion/physical-expr/src/array_expressions.rs @@ -1289,12 +1289,23 @@ fn general_position( /// Array_positions SQL function pub fn array_positions(args: &[ArrayRef]) -> Result { - let arr = as_list_array(&args[0])?; let element = &args[1]; - check_datatypes("array_positions", &[arr.values(), element])?; - - general_positions::(arr, element) + match &args[0].data_type() { + DataType::List(_) => { + let arr = as_list_array(&args[0])?; + check_datatypes("array_positions", &[arr.values(), element])?; + general_positions::(arr, element) + } + DataType::LargeList(_) => { + let arr = as_large_list_array(&args[0])?; + check_datatypes("array_positions", &[arr.values(), element])?; + general_positions::(arr, element) + } + array_type => { + not_impl_err!("array_positions does not support type '{array_type:?}'.") + } + } } fn general_positions( diff --git a/datafusion/sqllogictest/test_files/array.slt b/datafusion/sqllogictest/test_files/array.slt index 640f5064eae6..d148f7118176 100644 --- a/datafusion/sqllogictest/test_files/array.slt +++ b/datafusion/sqllogictest/test_files/array.slt @@ -1832,18 +1832,33 @@ select array_positions(['h', 'e', 'l', 'l', 'o'], 'l'), array_positions([1, 2, 3 ---- [3, 4] [5] [1, 2, 3] +query ??? +select array_positions(arrow_cast(['h', 'e', 'l', 'l', 'o'], 'LargeList(Utf8)'), 'l'), array_positions(arrow_cast([1, 2, 3, 4, 5], 'LargeList(Int64)'), 5), array_positions(arrow_cast([1, 1, 1], 'LargeList(Int64)'), 1); +---- +[3, 4] [5] [1, 2, 3] + # array_positions scalar function #2 (element is list) query ? select array_positions(make_array([1, 2, 3], [2, 1, 3], [1, 5, 6], [2, 1, 3], [4, 5, 6]), [2, 1, 3]); ---- [2, 4] +query ? +select array_positions(arrow_cast(make_array([1, 2, 3], [2, 1, 3], [1, 5, 6], [2, 1, 3], [4, 5, 6]), 'LargeList(List(Int64))'), [2, 1, 3]); +---- +[2, 4] + # list_positions scalar function #3 (function alias `array_positions`) query ??? select list_positions(['h', 'e', 'l', 'l', 'o'], 'l'), list_positions([1, 2, 3, 4, 5], 5), list_positions([1, 1, 1], 1); ---- [3, 4] [5] [1, 2, 3] +query ??? +select list_positions(arrow_cast(['h', 'e', 'l', 'l', 'o'], 'LargeList(Utf8)'), 'l'), list_positions(arrow_cast([1, 2, 3, 4, 5], 'LargeList(Int64)'), 5), list_positions(arrow_cast([1, 1, 1], 'LargeList(Int64)'), 1); +---- +[3, 4] [5] [1, 2, 3] + # array_positions with columns #1 query ? select array_positions(column1, column2) from arrays_values_without_nulls; @@ -1853,6 +1868,14 @@ select array_positions(column1, column2) from arrays_values_without_nulls; [3] [4] +query ? +select array_positions(arrow_cast(column1, 'LargeList(Int64)'), column2) from arrays_values_without_nulls; +---- +[1] +[2] +[3] +[4] + # array_positions with columns #2 (element is list) query ? select array_positions(column1, column2) from nested_arrays; @@ -1860,6 +1883,12 @@ select array_positions(column1, column2) from nested_arrays; [3] [2, 5] +query ? +select array_positions(arrow_cast(column1, 'LargeList(List(Int64))'), column2) from nested_arrays; +---- +[3] +[2, 5] + # array_positions with columns and scalars #1 query ?? select array_positions(column1, 4), array_positions(array[1, 2, 23, 13, 33, 45], column2) from arrays_values_without_nulls; @@ -1869,6 +1898,14 @@ select array_positions(column1, 4), array_positions(array[1, 2, 23, 13, 33, 45], [] [3] [] [] +query ?? +select array_positions(arrow_cast(column1, 'LargeList(Int64)'), 4), array_positions(array[1, 2, 23, 13, 33, 45], column2) from arrays_values_without_nulls; +---- +[4] [1] +[] [] +[] [3] +[] [] + # array_positions with columns and scalars #2 (element is list) query ?? select array_positions(column1, make_array(4, 5, 6)), array_positions(make_array([1, 2, 3], [11, 12, 13], [4, 5, 6]), column2) from nested_arrays; @@ -1876,6 +1913,12 @@ select array_positions(column1, make_array(4, 5, 6)), array_positions(make_array [6] [] [1] [] +query ?? +select array_positions(arrow_cast(column1, 'LargeList(List(Int64))'), make_array(4, 5, 6)), array_positions(arrow_cast(make_array([1, 2, 3], [11, 12, 13], [4, 5, 6]), 'LargeList(List(Int64))'), column2) from nested_arrays; +---- +[6] [] +[1] [] + ## array_replace (aliases: `list_replace`) # array_replace scalar function #1 From d33ca4dd37b8b47120579b7c3e0456c1fcbcb06f Mon Sep 17 00:00:00 2001 From: Alex Huang Date: Mon, 18 Dec 2023 21:26:02 +0100 Subject: [PATCH 462/572] support LargeList in array_element (#8570) --- datafusion/expr/src/built_in_function.rs | 3 +- .../physical-expr/src/array_expressions.rs | 82 +++++++++++++------ datafusion/sqllogictest/test_files/array.slt | 72 +++++++++++++++- 3 files changed, 130 insertions(+), 27 deletions(-) diff --git a/datafusion/expr/src/built_in_function.rs b/datafusion/expr/src/built_in_function.rs index 289704ed98f8..3818e8ee5658 100644 --- a/datafusion/expr/src/built_in_function.rs +++ b/datafusion/expr/src/built_in_function.rs @@ -591,8 +591,9 @@ impl BuiltinScalarFunction { BuiltinScalarFunction::ArrayDistinct => Ok(input_expr_types[0].clone()), BuiltinScalarFunction::ArrayElement => match &input_expr_types[0] { List(field) => Ok(field.data_type().clone()), + LargeList(field) => Ok(field.data_type().clone()), _ => plan_err!( - "The {self} function can only accept list as the first argument" + "The {self} function can only accept list or largelist as the first argument" ), }, BuiltinScalarFunction::ArrayLength => Ok(UInt64), diff --git a/datafusion/physical-expr/src/array_expressions.rs b/datafusion/physical-expr/src/array_expressions.rs index cc4b2899fcb1..d39658108337 100644 --- a/datafusion/physical-expr/src/array_expressions.rs +++ b/datafusion/physical-expr/src/array_expressions.rs @@ -370,18 +370,14 @@ pub fn make_array(arrays: &[ArrayRef]) -> Result { } } -/// array_element SQL function -/// -/// There are two arguments for array_element, the first one is the array, the second one is the 1-indexed index. -/// `array_element(array, index)` -/// -/// For example: -/// > array_element(\[1, 2, 3], 2) -> 2 -pub fn array_element(args: &[ArrayRef]) -> Result { - let list_array = as_list_array(&args[0])?; - let indexes = as_int64_array(&args[1])?; - - let values = list_array.values(); +fn general_array_element( + array: &GenericListArray, + indexes: &Int64Array, +) -> Result +where + i64: TryInto, +{ + let values = array.values(); let original_data = values.to_data(); let capacity = Capacities::Array(original_data.len()); @@ -389,37 +385,47 @@ pub fn array_element(args: &[ArrayRef]) -> Result { let mut mutable = MutableArrayData::with_capacities(vec![&original_data], true, capacity); - fn adjusted_array_index(index: i64, len: usize) -> Option { + fn adjusted_array_index(index: i64, len: O) -> Result> + where + i64: TryInto, + { + let index: O = index.try_into().map_err(|_| { + DataFusionError::Execution(format!( + "array_element got invalid index: {}", + index + )) + })?; // 0 ~ len - 1 - let adjusted_zero_index = if index < 0 { - index + len as i64 + let adjusted_zero_index = if index < O::usize_as(0) { + index + len } else { - index - 1 + index - O::usize_as(1) }; - if 0 <= adjusted_zero_index && adjusted_zero_index < len as i64 { - Some(adjusted_zero_index) + if O::usize_as(0) <= adjusted_zero_index && adjusted_zero_index < len { + Ok(Some(adjusted_zero_index)) } else { // Out of bounds - None + Ok(None) } } - for (row_index, offset_window) in list_array.offsets().windows(2).enumerate() { - let start = offset_window[0] as usize; - let end = offset_window[1] as usize; + for (row_index, offset_window) in array.offsets().windows(2).enumerate() { + let start = offset_window[0]; + let end = offset_window[1]; let len = end - start; // array is null - if len == 0 { + if len == O::usize_as(0) { mutable.extend_nulls(1); continue; } - let index = adjusted_array_index(indexes.value(row_index), len); + let index = adjusted_array_index::(indexes.value(row_index), len)?; if let Some(index) = index { - mutable.extend(0, start + index as usize, start + index as usize + 1); + let start = start.as_usize() + index.as_usize(); + mutable.extend(0, start, start + 1_usize); } else { // Index out of bounds mutable.extend_nulls(1); @@ -430,6 +436,32 @@ pub fn array_element(args: &[ArrayRef]) -> Result { Ok(arrow_array::make_array(data)) } +/// array_element SQL function +/// +/// There are two arguments for array_element, the first one is the array, the second one is the 1-indexed index. +/// `array_element(array, index)` +/// +/// For example: +/// > array_element(\[1, 2, 3], 2) -> 2 +pub fn array_element(args: &[ArrayRef]) -> Result { + match &args[0].data_type() { + DataType::List(_) => { + let array = as_list_array(&args[0])?; + let indexes = as_int64_array(&args[1])?; + general_array_element::(array, indexes) + } + DataType::LargeList(_) => { + let array = as_large_list_array(&args[0])?; + let indexes = as_int64_array(&args[1])?; + general_array_element::(array, indexes) + } + _ => not_impl_err!( + "array_element does not support type: {:?}", + args[0].data_type() + ), + } +} + fn general_except( l: &GenericListArray, r: &GenericListArray, diff --git a/datafusion/sqllogictest/test_files/array.slt b/datafusion/sqllogictest/test_files/array.slt index d148f7118176..b38f73ecb8db 100644 --- a/datafusion/sqllogictest/test_files/array.slt +++ b/datafusion/sqllogictest/test_files/array.slt @@ -717,7 +717,7 @@ from arrays_values_without_nulls; ## array_element (aliases: array_extract, list_extract, list_element) # array_element error -query error DataFusion error: Error during planning: The array_element function can only accept list as the first argument +query error DataFusion error: Error during planning: The array_element function can only accept list or largelist as the first argument select array_element(1, 2); @@ -727,58 +727,106 @@ select array_element(make_array(1, 2, 3, 4, 5), 2), array_element(make_array('h' ---- 2 l +query IT +select array_element(arrow_cast(make_array(1, 2, 3, 4, 5), 'LargeList(Int64)'), 2), array_element(arrow_cast(make_array('h', 'e', 'l', 'l', 'o'), 'LargeList(Utf8)'), 3); +---- +2 l + # array_element scalar function #2 (with positive index; out of bounds) query IT select array_element(make_array(1, 2, 3, 4, 5), 7), array_element(make_array('h', 'e', 'l', 'l', 'o'), 11); ---- NULL NULL +query IT +select array_element(arrow_cast(make_array(1, 2, 3, 4, 5), 'LargeList(Int64)'), 7), array_element(arrow_cast(make_array('h', 'e', 'l', 'l', 'o'), 'LargeList(Utf8)'), 11); +---- +NULL NULL + # array_element scalar function #3 (with zero) query IT select array_element(make_array(1, 2, 3, 4, 5), 0), array_element(make_array('h', 'e', 'l', 'l', 'o'), 0); ---- NULL NULL +query IT +select array_element(arrow_cast(make_array(1, 2, 3, 4, 5), 'LargeList(Int64)'), 0), array_element(arrow_cast(make_array('h', 'e', 'l', 'l', 'o'), 'LargeList(Utf8)'), 0); +---- +NULL NULL + # array_element scalar function #4 (with NULL) query error select array_element(make_array(1, 2, 3, 4, 5), NULL), array_element(make_array('h', 'e', 'l', 'l', 'o'), NULL); +query error +select array_element(arrow_cast(make_array(1, 2, 3, 4, 5), 'LargeList(Int64)'), NULL), array_element(arrow_cast(make_array('h', 'e', 'l', 'l', 'o'), 'LargeList(Utf8)'), NULL); + # array_element scalar function #5 (with negative index) query IT select array_element(make_array(1, 2, 3, 4, 5), -2), array_element(make_array('h', 'e', 'l', 'l', 'o'), -3); ---- 4 l +query IT +select array_element(arrow_cast(make_array(1, 2, 3, 4, 5), 'LargeList(Int64)'), -2), array_element(arrow_cast(make_array('h', 'e', 'l', 'l', 'o'), 'LargeList(Utf8)'), -3); +---- +4 l + # array_element scalar function #6 (with negative index; out of bounds) query IT select array_element(make_array(1, 2, 3, 4, 5), -11), array_element(make_array('h', 'e', 'l', 'l', 'o'), -7); ---- NULL NULL +query IT +select array_element(arrow_cast(make_array(1, 2, 3, 4, 5), 'LargeList(Int64)'), -11), array_element(arrow_cast(make_array('h', 'e', 'l', 'l', 'o'), 'LargeList(Utf8)'), -7); +---- +NULL NULL + # array_element scalar function #7 (nested array) query ? select array_element(make_array(make_array(1, 2, 3, 4, 5), make_array(6, 7, 8, 9, 10)), 1); ---- [1, 2, 3, 4, 5] +query ? +select array_element(arrow_cast(make_array(make_array(1, 2, 3, 4, 5), make_array(6, 7, 8, 9, 10)), 'LargeList(List(Int64))'), 1); +---- +[1, 2, 3, 4, 5] + # array_extract scalar function #8 (function alias `array_slice`) query IT select array_extract(make_array(1, 2, 3, 4, 5), 2), array_extract(make_array('h', 'e', 'l', 'l', 'o'), 3); ---- 2 l +query IT +select array_extract(arrow_cast(make_array(1, 2, 3, 4, 5), 'LargeList(Int64)'), 2), array_extract(arrow_cast(make_array('h', 'e', 'l', 'l', 'o'), 'LargeList(Utf8)'), 3); +---- +2 l + # list_element scalar function #9 (function alias `array_slice`) query IT select list_element(make_array(1, 2, 3, 4, 5), 2), list_element(make_array('h', 'e', 'l', 'l', 'o'), 3); ---- 2 l +query IT +select list_element(arrow_cast(make_array(1, 2, 3, 4, 5), 'LargeList(Int64)'), 2), array_extract(arrow_cast(make_array('h', 'e', 'l', 'l', 'o'), 'LargeList(Utf8)'), 3); +---- +2 l + # list_extract scalar function #10 (function alias `array_slice`) query IT select list_extract(make_array(1, 2, 3, 4, 5), 2), list_extract(make_array('h', 'e', 'l', 'l', 'o'), 3); ---- 2 l +query IT +select list_extract(arrow_cast(make_array(1, 2, 3, 4, 5), 'LargeList(Int64)'), 2), array_extract(arrow_cast(make_array('h', 'e', 'l', 'l', 'o'), 'LargeList(Utf8)'), 3); +---- +2 l + # array_element with columns query I select array_element(column1, column2) from slices; @@ -791,6 +839,17 @@ NULL NULL 55 +query I +select array_element(arrow_cast(column1, 'LargeList(Int64)'), column2) from slices; +---- +NULL +12 +NULL +37 +NULL +NULL +55 + # array_element with columns and scalars query II select array_element(make_array(1, 2, 3, 4, 5), column2), array_element(column1, 3) from slices; @@ -803,6 +862,17 @@ NULL 23 NULL 43 5 NULL +query II +select array_element(arrow_cast(make_array(1, 2, 3, 4, 5), 'LargeList(Int64)'), column2), array_element(arrow_cast(column1, 'LargeList(Int64)'), 3) from slices; +---- +1 3 +2 13 +NULL 23 +2 33 +4 NULL +NULL 43 +5 NULL + ## array_pop_back (aliases: `list_pop_back`) # array_pop_back scalar function #1 From 9bc61b31ae4f67c55c03214c9b807079e4fe0f44 Mon Sep 17 00:00:00 2001 From: Mustafa Akur <106137913+mustafasrepo@users.noreply.github.com> Date: Tue, 19 Dec 2023 18:22:33 +0300 Subject: [PATCH 463/572] Increase test coverage for unbounded and bounded cases (#8581) * Re-introduce unbounded tests with new executor * Remove unnecessary test * Enhance test coverage * Review * Test passes * Change argument order * Parametrize enforce sorting test * Imports --------- Co-authored-by: Mehmet Ozan Kabak --- .../src/physical_optimizer/enforce_sorting.rs | 92 ++- .../replace_with_order_preserving_variants.rs | 714 +++++++++++++++--- datafusion/core/src/test/mod.rs | 28 +- 3 files changed, 697 insertions(+), 137 deletions(-) diff --git a/datafusion/core/src/physical_optimizer/enforce_sorting.rs b/datafusion/core/src/physical_optimizer/enforce_sorting.rs index 2b650a42696b..2ecc1e11b985 100644 --- a/datafusion/core/src/physical_optimizer/enforce_sorting.rs +++ b/datafusion/core/src/physical_optimizer/enforce_sorting.rs @@ -60,8 +60,8 @@ use crate::physical_plan::{ use datafusion_common::tree_node::{Transformed, TreeNode, VisitRecursion}; use datafusion_common::{plan_err, DataFusionError}; use datafusion_physical_expr::{PhysicalSortExpr, PhysicalSortRequirement}; - use datafusion_physical_plan::repartition::RepartitionExec; + use itertools::izip; /// This rule inspects [`SortExec`]'s in the given physical plan and removes the @@ -769,7 +769,7 @@ mod tests { use crate::physical_plan::repartition::RepartitionExec; use crate::physical_plan::{displayable, get_plan_string, Partitioning}; use crate::prelude::{SessionConfig, SessionContext}; - use crate::test::{csv_exec_sorted, stream_exec_ordered}; + use crate::test::{csv_exec_ordered, csv_exec_sorted, stream_exec_ordered}; use arrow::compute::SortOptions; use arrow::datatypes::{DataType, Field, Schema, SchemaRef}; @@ -777,6 +777,8 @@ mod tests { use datafusion_expr::JoinType; use datafusion_physical_expr::expressions::{col, Column, NotExpr}; + use rstest::rstest; + fn create_test_schema() -> Result { let nullable_column = Field::new("nullable_col", DataType::Int32, true); let non_nullable_column = Field::new("non_nullable_col", DataType::Int32, false); @@ -2140,12 +2142,19 @@ mod tests { Ok(()) } + #[rstest] #[tokio::test] - async fn test_with_lost_ordering_unbounded() -> Result<()> { + async fn test_with_lost_ordering_unbounded_bounded( + #[values(false, true)] source_unbounded: bool, + ) -> Result<()> { let schema = create_test_schema3()?; let sort_exprs = vec![sort_expr("a", &schema)]; - // create an unbounded source - let source = stream_exec_ordered(&schema, sort_exprs); + // create either bounded or unbounded source + let source = if source_unbounded { + stream_exec_ordered(&schema, sort_exprs) + } else { + csv_exec_ordered(&schema, sort_exprs) + }; let repartition_rr = repartition_exec(source); let repartition_hash = Arc::new(RepartitionExec::try_new( repartition_rr, @@ -2154,50 +2163,71 @@ mod tests { let coalesce_partitions = coalesce_partitions_exec(repartition_hash); let physical_plan = sort_exec(vec![sort_expr("a", &schema)], coalesce_partitions); - let expected_input = [ + // Expected inputs unbounded and bounded + let expected_input_unbounded = vec![ "SortExec: expr=[a@0 ASC]", " CoalescePartitionsExec", " RepartitionExec: partitioning=Hash([c@2], 10), input_partitions=10", " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", " StreamingTableExec: partition_sizes=1, projection=[a, b, c, d, e], infinite_source=true, output_ordering=[a@0 ASC]", ]; - let expected_optimized = [ + let expected_input_bounded = vec![ + "SortExec: expr=[a@0 ASC]", + " CoalescePartitionsExec", + " RepartitionExec: partitioning=Hash([c@2], 10), input_partitions=10", + " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", + " CsvExec: file_groups={1 group: [[file_path]]}, projection=[a, b, c, d, e], output_ordering=[a@0 ASC], has_header=true", + ]; + + // Expected unbounded result (same for with and without flag) + let expected_optimized_unbounded = vec![ "SortPreservingMergeExec: [a@0 ASC]", " RepartitionExec: partitioning=Hash([c@2], 10), input_partitions=10, preserve_order=true, sort_exprs=a@0 ASC", " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", " StreamingTableExec: partition_sizes=1, projection=[a, b, c, d, e], infinite_source=true, output_ordering=[a@0 ASC]", ]; - assert_optimized!(expected_input, expected_optimized, physical_plan, true); - Ok(()) - } - #[tokio::test] - async fn test_with_lost_ordering_unbounded_parallelize_off() -> Result<()> { - let schema = create_test_schema3()?; - let sort_exprs = vec![sort_expr("a", &schema)]; - // create an unbounded source - let source = stream_exec_ordered(&schema, sort_exprs); - let repartition_rr = repartition_exec(source); - let repartition_hash = Arc::new(RepartitionExec::try_new( - repartition_rr, - Partitioning::Hash(vec![col("c", &schema).unwrap()], 10), - )?) as _; - let coalesce_partitions = coalesce_partitions_exec(repartition_hash); - let physical_plan = sort_exec(vec![sort_expr("a", &schema)], coalesce_partitions); - - let expected_input = ["SortExec: expr=[a@0 ASC]", + // Expected bounded results with and without flag + let expected_optimized_bounded = vec![ + "SortExec: expr=[a@0 ASC]", " CoalescePartitionsExec", " RepartitionExec: partitioning=Hash([c@2], 10), input_partitions=10", " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - " StreamingTableExec: partition_sizes=1, projection=[a, b, c, d, e], infinite_source=true, output_ordering=[a@0 ASC]", + " CsvExec: file_groups={1 group: [[file_path]]}, projection=[a, b, c, d, e], output_ordering=[a@0 ASC], has_header=true", ]; - let expected_optimized = [ + let expected_optimized_bounded_parallelize_sort = vec![ "SortPreservingMergeExec: [a@0 ASC]", - " RepartitionExec: partitioning=Hash([c@2], 10), input_partitions=10, preserve_order=true, sort_exprs=a@0 ASC", - " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - " StreamingTableExec: partition_sizes=1, projection=[a, b, c, d, e], infinite_source=true, output_ordering=[a@0 ASC]", + " SortExec: expr=[a@0 ASC]", + " RepartitionExec: partitioning=Hash([c@2], 10), input_partitions=10", + " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", + " CsvExec: file_groups={1 group: [[file_path]]}, projection=[a, b, c, d, e], output_ordering=[a@0 ASC], has_header=true", ]; - assert_optimized!(expected_input, expected_optimized, physical_plan, false); + let (expected_input, expected_optimized, expected_optimized_sort_parallelize) = + if source_unbounded { + ( + expected_input_unbounded, + expected_optimized_unbounded.clone(), + expected_optimized_unbounded, + ) + } else { + ( + expected_input_bounded, + expected_optimized_bounded, + expected_optimized_bounded_parallelize_sort, + ) + }; + assert_optimized!( + expected_input, + expected_optimized, + physical_plan.clone(), + false + ); + assert_optimized!( + expected_input, + expected_optimized_sort_parallelize, + physical_plan, + true + ); Ok(()) } diff --git a/datafusion/core/src/physical_optimizer/replace_with_order_preserving_variants.rs b/datafusion/core/src/physical_optimizer/replace_with_order_preserving_variants.rs index 671891be433c..0ff7e9f48edc 100644 --- a/datafusion/core/src/physical_optimizer/replace_with_order_preserving_variants.rs +++ b/datafusion/core/src/physical_optimizer/replace_with_order_preserving_variants.rs @@ -276,6 +276,9 @@ pub(crate) fn replace_with_order_preserving_variants( mod tests { use super::*; + use crate::datasource::file_format::file_compression_type::FileCompressionType; + use crate::datasource::listing::PartitionedFile; + use crate::datasource::physical_plan::{CsvExec, FileScanConfig}; use crate::physical_plan::coalesce_batches::CoalesceBatchesExec; use crate::physical_plan::coalesce_partitions::CoalescePartitionsExec; use crate::physical_plan::filter::FilterExec; @@ -285,35 +288,95 @@ mod tests { use crate::physical_plan::sorts::sort_preserving_merge::SortPreservingMergeExec; use crate::physical_plan::{displayable, get_plan_string, Partitioning}; use crate::prelude::SessionConfig; - use crate::test::TestStreamPartition; + use arrow::compute::SortOptions; use arrow::datatypes::{DataType, Field, Schema, SchemaRef}; use datafusion_common::tree_node::TreeNode; - use datafusion_common::Result; + use datafusion_common::{Result, Statistics}; + use datafusion_execution::object_store::ObjectStoreUrl; use datafusion_expr::{JoinType, Operator}; use datafusion_physical_expr::expressions::{self, col, Column}; use datafusion_physical_expr::PhysicalSortExpr; use datafusion_physical_plan::streaming::StreamingTableExec; + use rstest::rstest; - /// Runs the `replace_with_order_preserving_variants` sub-rule and asserts the plan - /// against the original and expected plans. + /// Runs the `replace_with_order_preserving_variants` sub-rule and asserts + /// the plan against the original and expected plans for both bounded and + /// unbounded cases. /// - /// `$EXPECTED_PLAN_LINES`: input plan - /// `$EXPECTED_OPTIMIZED_PLAN_LINES`: optimized plan - /// `$PLAN`: the plan to optimized - /// `$ALLOW_BOUNDED`: whether to allow the plan to be optimized for bounded cases - macro_rules! assert_optimized { - ($EXPECTED_PLAN_LINES: expr, $EXPECTED_OPTIMIZED_PLAN_LINES: expr, $PLAN: expr) => { + /// # Parameters + /// + /// * `EXPECTED_UNBOUNDED_PLAN_LINES`: Expected input unbounded plan. + /// * `EXPECTED_BOUNDED_PLAN_LINES`: Expected input bounded plan. + /// * `EXPECTED_UNBOUNDED_OPTIMIZED_PLAN_LINES`: Optimized plan, which is + /// the same regardless of the value of the `prefer_existing_sort` flag. + /// * `EXPECTED_BOUNDED_OPTIMIZED_PLAN_LINES`: Optimized plan when the flag + /// `prefer_existing_sort` is `false` for bounded cases. + /// * `EXPECTED_BOUNDED_PREFER_SORT_ON_OPTIMIZED_PLAN_LINES`: Optimized plan + /// when the flag `prefer_existing_sort` is `true` for bounded cases. + /// * `$PLAN`: The plan to optimize. + /// * `$SOURCE_UNBOUNDED`: Whether the given plan contains an unbounded source. + macro_rules! assert_optimized_in_all_boundedness_situations { + ($EXPECTED_UNBOUNDED_PLAN_LINES: expr, $EXPECTED_BOUNDED_PLAN_LINES: expr, $EXPECTED_UNBOUNDED_OPTIMIZED_PLAN_LINES: expr, $EXPECTED_BOUNDED_OPTIMIZED_PLAN_LINES: expr, $EXPECTED_BOUNDED_PREFER_SORT_ON_OPTIMIZED_PLAN_LINES: expr, $PLAN: expr, $SOURCE_UNBOUNDED: expr) => { + if $SOURCE_UNBOUNDED { + assert_optimized_prefer_sort_on_off!( + $EXPECTED_UNBOUNDED_PLAN_LINES, + $EXPECTED_UNBOUNDED_OPTIMIZED_PLAN_LINES, + $EXPECTED_UNBOUNDED_OPTIMIZED_PLAN_LINES, + $PLAN + ); + } else { + assert_optimized_prefer_sort_on_off!( + $EXPECTED_BOUNDED_PLAN_LINES, + $EXPECTED_BOUNDED_OPTIMIZED_PLAN_LINES, + $EXPECTED_BOUNDED_PREFER_SORT_ON_OPTIMIZED_PLAN_LINES, + $PLAN + ); + } + }; + } + + /// Runs the `replace_with_order_preserving_variants` sub-rule and asserts + /// the plan against the original and expected plans. + /// + /// # Parameters + /// + /// * `$EXPECTED_PLAN_LINES`: Expected input plan. + /// * `EXPECTED_OPTIMIZED_PLAN_LINES`: Optimized plan when the flag + /// `prefer_existing_sort` is `false`. + /// * `EXPECTED_PREFER_SORT_ON_OPTIMIZED_PLAN_LINES`: Optimized plan when + /// the flag `prefer_existing_sort` is `true`. + /// * `$PLAN`: The plan to optimize. + macro_rules! assert_optimized_prefer_sort_on_off { + ($EXPECTED_PLAN_LINES: expr, $EXPECTED_OPTIMIZED_PLAN_LINES: expr, $EXPECTED_PREFER_SORT_ON_OPTIMIZED_PLAN_LINES: expr, $PLAN: expr) => { assert_optimized!( $EXPECTED_PLAN_LINES, $EXPECTED_OPTIMIZED_PLAN_LINES, - $PLAN, + $PLAN.clone(), false ); + assert_optimized!( + $EXPECTED_PLAN_LINES, + $EXPECTED_PREFER_SORT_ON_OPTIMIZED_PLAN_LINES, + $PLAN, + true + ); }; - ($EXPECTED_PLAN_LINES: expr, $EXPECTED_OPTIMIZED_PLAN_LINES: expr, $PLAN: expr, $ALLOW_BOUNDED: expr) => { + } + + /// Runs the `replace_with_order_preserving_variants` sub-rule and asserts + /// the plan against the original and expected plans. + /// + /// # Parameters + /// + /// * `$EXPECTED_PLAN_LINES`: Expected input plan. + /// * `$EXPECTED_OPTIMIZED_PLAN_LINES`: Expected optimized plan. + /// * `$PLAN`: The plan to optimize. + /// * `$PREFER_EXISTING_SORT`: Value of the `prefer_existing_sort` flag. + macro_rules! assert_optimized { + ($EXPECTED_PLAN_LINES: expr, $EXPECTED_OPTIMIZED_PLAN_LINES: expr, $PLAN: expr, $PREFER_EXISTING_SORT: expr) => { let physical_plan = $PLAN; let formatted = displayable(physical_plan.as_ref()).indent(true).to_string(); let actual: Vec<&str> = formatted.trim().lines().collect(); @@ -329,8 +392,7 @@ mod tests { let expected_optimized_lines: Vec<&str> = $EXPECTED_OPTIMIZED_PLAN_LINES.iter().map(|s| *s).collect(); // Run the rule top-down - // let optimized_physical_plan = physical_plan.transform_down(&replace_repartition_execs)?; - let config = SessionConfig::new().with_prefer_existing_sort($ALLOW_BOUNDED); + let config = SessionConfig::new().with_prefer_existing_sort($PREFER_EXISTING_SORT); let plan_with_pipeline_fixer = OrderPreservationContext::new(physical_plan); let parallel = plan_with_pipeline_fixer.transform_up(&|plan_with_pipeline_fixer| replace_with_order_preserving_variants(plan_with_pipeline_fixer, false, false, config.options()))?; let optimized_physical_plan = parallel.plan; @@ -348,35 +410,67 @@ mod tests { #[tokio::test] // Searches for a simple sort and a repartition just after it, the second repartition with 1 input partition should not be affected async fn test_replace_multiple_input_repartition_1( - #[values(false, true)] prefer_existing_sort: bool, + #[values(false, true)] source_unbounded: bool, ) -> Result<()> { let schema = create_test_schema()?; let sort_exprs = vec![sort_expr("a", &schema)]; - let source = stream_exec_ordered(&schema, sort_exprs); + let source = if source_unbounded { + stream_exec_ordered(&schema, sort_exprs) + } else { + csv_exec_sorted(&schema, sort_exprs) + }; let repartition = repartition_exec_hash(repartition_exec_round_robin(source)); let sort = sort_exec(vec![sort_expr("a", &schema)], repartition, true); let physical_plan = sort_preserving_merge_exec(vec![sort_expr("a", &schema)], sort); - let expected_input = [ + // Expected inputs unbounded and bounded + let expected_input_unbounded = [ "SortPreservingMergeExec: [a@0 ASC NULLS LAST]", " SortExec: expr=[a@0 ASC NULLS LAST]", " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8", " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", " StreamingTableExec: partition_sizes=1, projection=[a, c, d], infinite_source=true, output_ordering=[a@0 ASC NULLS LAST]", ]; - let expected_optimized = [ + let expected_input_bounded = [ + "SortPreservingMergeExec: [a@0 ASC NULLS LAST]", + " SortExec: expr=[a@0 ASC NULLS LAST]", + " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8", + " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", + " CsvExec: file_groups={1 group: [[file_path]]}, projection=[a, c, d], output_ordering=[a@0 ASC NULLS LAST], has_header=true", + ]; + + // Expected unbounded result (same for with and without flag) + let expected_optimized_unbounded = [ "SortPreservingMergeExec: [a@0 ASC NULLS LAST]", " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8, preserve_order=true, sort_exprs=a@0 ASC NULLS LAST", " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", " StreamingTableExec: partition_sizes=1, projection=[a, c, d], infinite_source=true, output_ordering=[a@0 ASC NULLS LAST]", ]; - assert_optimized!( - expected_input, - expected_optimized, + + // Expected bounded results with and without flag + let expected_optimized_bounded = [ + "SortPreservingMergeExec: [a@0 ASC NULLS LAST]", + " SortExec: expr=[a@0 ASC NULLS LAST]", + " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8", + " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", + " CsvExec: file_groups={1 group: [[file_path]]}, projection=[a, c, d], output_ordering=[a@0 ASC NULLS LAST], has_header=true", + ]; + let expected_optimized_bounded_sort_preserve = [ + "SortPreservingMergeExec: [a@0 ASC NULLS LAST]", + " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8, preserve_order=true, sort_exprs=a@0 ASC NULLS LAST", + " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", + " CsvExec: file_groups={1 group: [[file_path]]}, projection=[a, c, d], output_ordering=[a@0 ASC NULLS LAST], has_header=true", + ]; + assert_optimized_in_all_boundedness_situations!( + expected_input_unbounded, + expected_input_bounded, + expected_optimized_unbounded, + expected_optimized_bounded, + expected_optimized_bounded_sort_preserve, physical_plan, - prefer_existing_sort + source_unbounded ); Ok(()) } @@ -384,11 +478,15 @@ mod tests { #[rstest] #[tokio::test] async fn test_with_inter_children_change_only( - #[values(false, true)] prefer_existing_sort: bool, + #[values(false, true)] source_unbounded: bool, ) -> Result<()> { let schema = create_test_schema()?; let sort_exprs = vec![sort_expr_default("a", &schema)]; - let source = stream_exec_ordered(&schema, sort_exprs); + let source = if source_unbounded { + stream_exec_ordered(&schema, sort_exprs) + } else { + csv_exec_sorted(&schema, sort_exprs) + }; let repartition_rr = repartition_exec_round_robin(source); let repartition_hash = repartition_exec_hash(repartition_rr); let coalesce_partitions = coalesce_partitions_exec(repartition_hash); @@ -408,7 +506,8 @@ mod tests { sort2, ); - let expected_input = [ + // Expected inputs unbounded and bounded + let expected_input_unbounded = [ "SortPreservingMergeExec: [a@0 ASC]", " SortExec: expr=[a@0 ASC]", " FilterExec: c@1 > 3", @@ -420,8 +519,21 @@ mod tests { " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", " StreamingTableExec: partition_sizes=1, projection=[a, c, d], infinite_source=true, output_ordering=[a@0 ASC]", ]; + let expected_input_bounded = [ + "SortPreservingMergeExec: [a@0 ASC]", + " SortExec: expr=[a@0 ASC]", + " FilterExec: c@1 > 3", + " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8", + " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", + " SortExec: expr=[a@0 ASC]", + " CoalescePartitionsExec", + " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8", + " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", + " CsvExec: file_groups={1 group: [[file_path]]}, projection=[a, c, d], output_ordering=[a@0 ASC], has_header=true", + ]; - let expected_optimized = [ + // Expected unbounded result (same for with and without flag) + let expected_optimized_unbounded = [ "SortPreservingMergeExec: [a@0 ASC]", " FilterExec: c@1 > 3", " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8, preserve_order=true, sort_exprs=a@0 ASC", @@ -431,11 +543,38 @@ mod tests { " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", " StreamingTableExec: partition_sizes=1, projection=[a, c, d], infinite_source=true, output_ordering=[a@0 ASC]", ]; - assert_optimized!( - expected_input, - expected_optimized, + + // Expected bounded results with and without flag + let expected_optimized_bounded = [ + "SortPreservingMergeExec: [a@0 ASC]", + " SortExec: expr=[a@0 ASC]", + " FilterExec: c@1 > 3", + " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8", + " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", + " SortExec: expr=[a@0 ASC]", + " CoalescePartitionsExec", + " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8", + " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", + " CsvExec: file_groups={1 group: [[file_path]]}, projection=[a, c, d], output_ordering=[a@0 ASC], has_header=true", + ]; + let expected_optimized_bounded_sort_preserve = [ + "SortPreservingMergeExec: [a@0 ASC]", + " FilterExec: c@1 > 3", + " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8, preserve_order=true, sort_exprs=a@0 ASC", + " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", + " SortPreservingMergeExec: [a@0 ASC]", + " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8, preserve_order=true, sort_exprs=a@0 ASC", + " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", + " CsvExec: file_groups={1 group: [[file_path]]}, projection=[a, c, d], output_ordering=[a@0 ASC], has_header=true", + ]; + assert_optimized_in_all_boundedness_situations!( + expected_input_unbounded, + expected_input_bounded, + expected_optimized_unbounded, + expected_optimized_bounded, + expected_optimized_bounded_sort_preserve, physical_plan, - prefer_existing_sort + source_unbounded ); Ok(()) } @@ -443,11 +582,15 @@ mod tests { #[rstest] #[tokio::test] async fn test_replace_multiple_input_repartition_2( - #[values(false, true)] prefer_existing_sort: bool, + #[values(false, true)] source_unbounded: bool, ) -> Result<()> { let schema = create_test_schema()?; let sort_exprs = vec![sort_expr("a", &schema)]; - let source = stream_exec_ordered(&schema, sort_exprs); + let source = if source_unbounded { + stream_exec_ordered(&schema, sort_exprs) + } else { + csv_exec_sorted(&schema, sort_exprs) + }; let repartition_rr = repartition_exec_round_robin(source); let filter = filter_exec(repartition_rr); let repartition_hash = repartition_exec_hash(filter); @@ -456,7 +599,8 @@ mod tests { let physical_plan = sort_preserving_merge_exec(vec![sort_expr("a", &schema)], sort); - let expected_input = [ + // Expected inputs unbounded and bounded + let expected_input_unbounded = [ "SortPreservingMergeExec: [a@0 ASC NULLS LAST]", " SortExec: expr=[a@0 ASC NULLS LAST]", " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8", @@ -464,18 +608,48 @@ mod tests { " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", " StreamingTableExec: partition_sizes=1, projection=[a, c, d], infinite_source=true, output_ordering=[a@0 ASC NULLS LAST]", ]; - let expected_optimized = [ + let expected_input_bounded = [ + "SortPreservingMergeExec: [a@0 ASC NULLS LAST]", + " SortExec: expr=[a@0 ASC NULLS LAST]", + " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8", + " FilterExec: c@1 > 3", + " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", + " CsvExec: file_groups={1 group: [[file_path]]}, projection=[a, c, d], output_ordering=[a@0 ASC NULLS LAST], has_header=true", + ]; + + // Expected unbounded result (same for with and without flag) + let expected_optimized_unbounded = [ "SortPreservingMergeExec: [a@0 ASC NULLS LAST]", " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8, preserve_order=true, sort_exprs=a@0 ASC NULLS LAST", " FilterExec: c@1 > 3", " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", " StreamingTableExec: partition_sizes=1, projection=[a, c, d], infinite_source=true, output_ordering=[a@0 ASC NULLS LAST]", ]; - assert_optimized!( - expected_input, - expected_optimized, + + // Expected bounded results with and without flag + let expected_optimized_bounded = [ + "SortPreservingMergeExec: [a@0 ASC NULLS LAST]", + " SortExec: expr=[a@0 ASC NULLS LAST]", + " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8", + " FilterExec: c@1 > 3", + " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", + " CsvExec: file_groups={1 group: [[file_path]]}, projection=[a, c, d], output_ordering=[a@0 ASC NULLS LAST], has_header=true", + ]; + let expected_optimized_bounded_sort_preserve = [ + "SortPreservingMergeExec: [a@0 ASC NULLS LAST]", + " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8, preserve_order=true, sort_exprs=a@0 ASC NULLS LAST", + " FilterExec: c@1 > 3", + " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", + " CsvExec: file_groups={1 group: [[file_path]]}, projection=[a, c, d], output_ordering=[a@0 ASC NULLS LAST], has_header=true", + ]; + assert_optimized_in_all_boundedness_situations!( + expected_input_unbounded, + expected_input_bounded, + expected_optimized_unbounded, + expected_optimized_bounded, + expected_optimized_bounded_sort_preserve, physical_plan, - prefer_existing_sort + source_unbounded ); Ok(()) } @@ -483,11 +657,15 @@ mod tests { #[rstest] #[tokio::test] async fn test_replace_multiple_input_repartition_with_extra_steps( - #[values(false, true)] prefer_existing_sort: bool, + #[values(false, true)] source_unbounded: bool, ) -> Result<()> { let schema = create_test_schema()?; let sort_exprs = vec![sort_expr("a", &schema)]; - let source = stream_exec_ordered(&schema, sort_exprs); + let source = if source_unbounded { + stream_exec_ordered(&schema, sort_exprs) + } else { + csv_exec_sorted(&schema, sort_exprs) + }; let repartition_rr = repartition_exec_round_robin(source); let repartition_hash = repartition_exec_hash(repartition_rr); let filter = filter_exec(repartition_hash); @@ -497,7 +675,8 @@ mod tests { let physical_plan = sort_preserving_merge_exec(vec![sort_expr("a", &schema)], sort); - let expected_input = [ + // Expected inputs unbounded and bounded + let expected_input_unbounded = [ "SortPreservingMergeExec: [a@0 ASC NULLS LAST]", " SortExec: expr=[a@0 ASC NULLS LAST]", " CoalesceBatchesExec: target_batch_size=8192", @@ -506,7 +685,18 @@ mod tests { " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", " StreamingTableExec: partition_sizes=1, projection=[a, c, d], infinite_source=true, output_ordering=[a@0 ASC NULLS LAST]", ]; - let expected_optimized = [ + let expected_input_bounded = [ + "SortPreservingMergeExec: [a@0 ASC NULLS LAST]", + " SortExec: expr=[a@0 ASC NULLS LAST]", + " CoalesceBatchesExec: target_batch_size=8192", + " FilterExec: c@1 > 3", + " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8", + " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", + " CsvExec: file_groups={1 group: [[file_path]]}, projection=[a, c, d], output_ordering=[a@0 ASC NULLS LAST], has_header=true", + ]; + + // Expected unbounded result (same for with and without flag) + let expected_optimized_unbounded = [ "SortPreservingMergeExec: [a@0 ASC NULLS LAST]", " CoalesceBatchesExec: target_batch_size=8192", " FilterExec: c@1 > 3", @@ -514,11 +704,33 @@ mod tests { " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", " StreamingTableExec: partition_sizes=1, projection=[a, c, d], infinite_source=true, output_ordering=[a@0 ASC NULLS LAST]", ]; - assert_optimized!( - expected_input, - expected_optimized, + + // Expected bounded results with and without flag + let expected_optimized_bounded = [ + "SortPreservingMergeExec: [a@0 ASC NULLS LAST]", + " SortExec: expr=[a@0 ASC NULLS LAST]", + " CoalesceBatchesExec: target_batch_size=8192", + " FilterExec: c@1 > 3", + " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8", + " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", + " CsvExec: file_groups={1 group: [[file_path]]}, projection=[a, c, d], output_ordering=[a@0 ASC NULLS LAST], has_header=true", + ]; + let expected_optimized_bounded_sort_preserve = [ + "SortPreservingMergeExec: [a@0 ASC NULLS LAST]", + " CoalesceBatchesExec: target_batch_size=8192", + " FilterExec: c@1 > 3", + " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8, preserve_order=true, sort_exprs=a@0 ASC NULLS LAST", + " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", + " CsvExec: file_groups={1 group: [[file_path]]}, projection=[a, c, d], output_ordering=[a@0 ASC NULLS LAST], has_header=true", + ]; + assert_optimized_in_all_boundedness_situations!( + expected_input_unbounded, + expected_input_bounded, + expected_optimized_unbounded, + expected_optimized_bounded, + expected_optimized_bounded_sort_preserve, physical_plan, - prefer_existing_sort + source_unbounded ); Ok(()) } @@ -526,11 +738,15 @@ mod tests { #[rstest] #[tokio::test] async fn test_replace_multiple_input_repartition_with_extra_steps_2( - #[values(false, true)] prefer_existing_sort: bool, + #[values(false, true)] source_unbounded: bool, ) -> Result<()> { let schema = create_test_schema()?; let sort_exprs = vec![sort_expr("a", &schema)]; - let source = stream_exec_ordered(&schema, sort_exprs); + let source = if source_unbounded { + stream_exec_ordered(&schema, sort_exprs) + } else { + csv_exec_sorted(&schema, sort_exprs) + }; let repartition_rr = repartition_exec_round_robin(source); let coalesce_batches_exec_1 = coalesce_batches_exec(repartition_rr); let repartition_hash = repartition_exec_hash(coalesce_batches_exec_1); @@ -542,7 +758,8 @@ mod tests { let physical_plan = sort_preserving_merge_exec(vec![sort_expr("a", &schema)], sort); - let expected_input = [ + // Expected inputs unbounded and bounded + let expected_input_unbounded = [ "SortPreservingMergeExec: [a@0 ASC NULLS LAST]", " SortExec: expr=[a@0 ASC NULLS LAST]", " CoalesceBatchesExec: target_batch_size=8192", @@ -552,7 +769,19 @@ mod tests { " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", " StreamingTableExec: partition_sizes=1, projection=[a, c, d], infinite_source=true, output_ordering=[a@0 ASC NULLS LAST]", ]; - let expected_optimized = [ + let expected_input_bounded = [ + "SortPreservingMergeExec: [a@0 ASC NULLS LAST]", + " SortExec: expr=[a@0 ASC NULLS LAST]", + " CoalesceBatchesExec: target_batch_size=8192", + " FilterExec: c@1 > 3", + " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8", + " CoalesceBatchesExec: target_batch_size=8192", + " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", + " CsvExec: file_groups={1 group: [[file_path]]}, projection=[a, c, d], output_ordering=[a@0 ASC NULLS LAST], has_header=true", + ]; + + // Expected unbounded result (same for with and without flag) + let expected_optimized_unbounded = [ "SortPreservingMergeExec: [a@0 ASC NULLS LAST]", " CoalesceBatchesExec: target_batch_size=8192", " FilterExec: c@1 > 3", @@ -561,11 +790,35 @@ mod tests { " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", " StreamingTableExec: partition_sizes=1, projection=[a, c, d], infinite_source=true, output_ordering=[a@0 ASC NULLS LAST]", ]; - assert_optimized!( - expected_input, - expected_optimized, + + // Expected bounded results with and without flag + let expected_optimized_bounded = [ + "SortPreservingMergeExec: [a@0 ASC NULLS LAST]", + " SortExec: expr=[a@0 ASC NULLS LAST]", + " CoalesceBatchesExec: target_batch_size=8192", + " FilterExec: c@1 > 3", + " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8", + " CoalesceBatchesExec: target_batch_size=8192", + " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", + " CsvExec: file_groups={1 group: [[file_path]]}, projection=[a, c, d], output_ordering=[a@0 ASC NULLS LAST], has_header=true", + ]; + let expected_optimized_bounded_sort_preserve = [ + "SortPreservingMergeExec: [a@0 ASC NULLS LAST]", + " CoalesceBatchesExec: target_batch_size=8192", + " FilterExec: c@1 > 3", + " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8, preserve_order=true, sort_exprs=a@0 ASC NULLS LAST", + " CoalesceBatchesExec: target_batch_size=8192", + " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", + " CsvExec: file_groups={1 group: [[file_path]]}, projection=[a, c, d], output_ordering=[a@0 ASC NULLS LAST], has_header=true", + ]; + assert_optimized_in_all_boundedness_situations!( + expected_input_unbounded, + expected_input_bounded, + expected_optimized_unbounded, + expected_optimized_bounded, + expected_optimized_bounded_sort_preserve, physical_plan, - prefer_existing_sort + source_unbounded ); Ok(()) } @@ -573,11 +826,15 @@ mod tests { #[rstest] #[tokio::test] async fn test_not_replacing_when_no_need_to_preserve_sorting( - #[values(false, true)] prefer_existing_sort: bool, + #[values(false, true)] source_unbounded: bool, ) -> Result<()> { let schema = create_test_schema()?; let sort_exprs = vec![sort_expr("a", &schema)]; - let source = stream_exec_ordered(&schema, sort_exprs); + let source = if source_unbounded { + stream_exec_ordered(&schema, sort_exprs) + } else { + csv_exec_sorted(&schema, sort_exprs) + }; let repartition_rr = repartition_exec_round_robin(source); let repartition_hash = repartition_exec_hash(repartition_rr); let filter = filter_exec(repartition_hash); @@ -586,7 +843,8 @@ mod tests { let physical_plan: Arc = coalesce_partitions_exec(coalesce_batches_exec); - let expected_input = [ + // Expected inputs unbounded and bounded + let expected_input_unbounded = [ "CoalescePartitionsExec", " CoalesceBatchesExec: target_batch_size=8192", " FilterExec: c@1 > 3", @@ -594,7 +852,17 @@ mod tests { " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", " StreamingTableExec: partition_sizes=1, projection=[a, c, d], infinite_source=true, output_ordering=[a@0 ASC NULLS LAST]", ]; - let expected_optimized = [ + let expected_input_bounded = [ + "CoalescePartitionsExec", + " CoalesceBatchesExec: target_batch_size=8192", + " FilterExec: c@1 > 3", + " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8", + " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", + " CsvExec: file_groups={1 group: [[file_path]]}, projection=[a, c, d], output_ordering=[a@0 ASC NULLS LAST], has_header=true", + ]; + + // Expected unbounded result (same for with and without flag) + let expected_optimized_unbounded = [ "CoalescePartitionsExec", " CoalesceBatchesExec: target_batch_size=8192", " FilterExec: c@1 > 3", @@ -602,11 +870,26 @@ mod tests { " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", " StreamingTableExec: partition_sizes=1, projection=[a, c, d], infinite_source=true, output_ordering=[a@0 ASC NULLS LAST]", ]; - assert_optimized!( - expected_input, - expected_optimized, + + // Expected bounded results same with and without flag, because there is no executor with ordering requirement + let expected_optimized_bounded = [ + "CoalescePartitionsExec", + " CoalesceBatchesExec: target_batch_size=8192", + " FilterExec: c@1 > 3", + " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8", + " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", + " CsvExec: file_groups={1 group: [[file_path]]}, projection=[a, c, d], output_ordering=[a@0 ASC NULLS LAST], has_header=true", + ]; + let expected_optimized_bounded_sort_preserve = expected_optimized_bounded; + + assert_optimized_in_all_boundedness_situations!( + expected_input_unbounded, + expected_input_bounded, + expected_optimized_unbounded, + expected_optimized_bounded, + expected_optimized_bounded_sort_preserve, physical_plan, - prefer_existing_sort + source_unbounded ); Ok(()) } @@ -614,11 +897,15 @@ mod tests { #[rstest] #[tokio::test] async fn test_with_multiple_replacable_repartitions( - #[values(false, true)] prefer_existing_sort: bool, + #[values(false, true)] source_unbounded: bool, ) -> Result<()> { let schema = create_test_schema()?; let sort_exprs = vec![sort_expr("a", &schema)]; - let source = stream_exec_ordered(&schema, sort_exprs); + let source = if source_unbounded { + stream_exec_ordered(&schema, sort_exprs) + } else { + csv_exec_sorted(&schema, sort_exprs) + }; let repartition_rr = repartition_exec_round_robin(source); let repartition_hash = repartition_exec_hash(repartition_rr); let filter = filter_exec(repartition_hash); @@ -629,7 +916,8 @@ mod tests { let physical_plan = sort_preserving_merge_exec(vec![sort_expr("a", &schema)], sort); - let expected_input = [ + // Expected inputs unbounded and bounded + let expected_input_unbounded = [ "SortPreservingMergeExec: [a@0 ASC NULLS LAST]", " SortExec: expr=[a@0 ASC NULLS LAST]", " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8", @@ -639,7 +927,19 @@ mod tests { " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", " StreamingTableExec: partition_sizes=1, projection=[a, c, d], infinite_source=true, output_ordering=[a@0 ASC NULLS LAST]", ]; - let expected_optimized = [ + let expected_input_bounded = [ + "SortPreservingMergeExec: [a@0 ASC NULLS LAST]", + " SortExec: expr=[a@0 ASC NULLS LAST]", + " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8", + " CoalesceBatchesExec: target_batch_size=8192", + " FilterExec: c@1 > 3", + " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8", + " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", + " CsvExec: file_groups={1 group: [[file_path]]}, projection=[a, c, d], output_ordering=[a@0 ASC NULLS LAST], has_header=true", + ]; + + // Expected unbounded result (same for with and without flag) + let expected_optimized_unbounded = [ "SortPreservingMergeExec: [a@0 ASC NULLS LAST]", " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8, preserve_order=true, sort_exprs=a@0 ASC NULLS LAST", " CoalesceBatchesExec: target_batch_size=8192", @@ -648,11 +948,35 @@ mod tests { " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", " StreamingTableExec: partition_sizes=1, projection=[a, c, d], infinite_source=true, output_ordering=[a@0 ASC NULLS LAST]", ]; - assert_optimized!( - expected_input, - expected_optimized, + + // Expected bounded results with and without flag + let expected_optimized_bounded = [ + "SortPreservingMergeExec: [a@0 ASC NULLS LAST]", + " SortExec: expr=[a@0 ASC NULLS LAST]", + " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8", + " CoalesceBatchesExec: target_batch_size=8192", + " FilterExec: c@1 > 3", + " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8", + " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", + " CsvExec: file_groups={1 group: [[file_path]]}, projection=[a, c, d], output_ordering=[a@0 ASC NULLS LAST], has_header=true", + ]; + let expected_optimized_bounded_sort_preserve = [ + "SortPreservingMergeExec: [a@0 ASC NULLS LAST]", + " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8, preserve_order=true, sort_exprs=a@0 ASC NULLS LAST", + " CoalesceBatchesExec: target_batch_size=8192", + " FilterExec: c@1 > 3", + " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8, preserve_order=true, sort_exprs=a@0 ASC NULLS LAST", + " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", + " CsvExec: file_groups={1 group: [[file_path]]}, projection=[a, c, d], output_ordering=[a@0 ASC NULLS LAST], has_header=true", + ]; + assert_optimized_in_all_boundedness_situations!( + expected_input_unbounded, + expected_input_bounded, + expected_optimized_unbounded, + expected_optimized_bounded, + expected_optimized_bounded_sort_preserve, physical_plan, - prefer_existing_sort + source_unbounded ); Ok(()) } @@ -660,11 +984,15 @@ mod tests { #[rstest] #[tokio::test] async fn test_not_replace_with_different_orderings( - #[values(false, true)] prefer_existing_sort: bool, + #[values(false, true)] source_unbounded: bool, ) -> Result<()> { let schema = create_test_schema()?; let sort_exprs = vec![sort_expr("a", &schema)]; - let source = stream_exec_ordered(&schema, sort_exprs); + let source = if source_unbounded { + stream_exec_ordered(&schema, sort_exprs) + } else { + csv_exec_sorted(&schema, sort_exprs) + }; let repartition_rr = repartition_exec_round_robin(source); let repartition_hash = repartition_exec_hash(repartition_rr); let sort = sort_exec( @@ -678,25 +1006,49 @@ mod tests { sort, ); - let expected_input = [ + // Expected inputs unbounded and bounded + let expected_input_unbounded = [ "SortPreservingMergeExec: [c@1 ASC]", " SortExec: expr=[c@1 ASC]", " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8", " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", " StreamingTableExec: partition_sizes=1, projection=[a, c, d], infinite_source=true, output_ordering=[a@0 ASC NULLS LAST]", ]; - let expected_optimized = [ + let expected_input_bounded = [ + "SortPreservingMergeExec: [c@1 ASC]", + " SortExec: expr=[c@1 ASC]", + " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8", + " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", + " CsvExec: file_groups={1 group: [[file_path]]}, projection=[a, c, d], output_ordering=[a@0 ASC NULLS LAST], has_header=true", + ]; + + // Expected unbounded result (same for with and without flag) + let expected_optimized_unbounded = [ "SortPreservingMergeExec: [c@1 ASC]", " SortExec: expr=[c@1 ASC]", " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8", " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", " StreamingTableExec: partition_sizes=1, projection=[a, c, d], infinite_source=true, output_ordering=[a@0 ASC NULLS LAST]", ]; - assert_optimized!( - expected_input, - expected_optimized, + + // Expected bounded results same with and without flag, because ordering requirement of the executor is different than the existing ordering. + let expected_optimized_bounded = [ + "SortPreservingMergeExec: [c@1 ASC]", + " SortExec: expr=[c@1 ASC]", + " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8", + " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", + " CsvExec: file_groups={1 group: [[file_path]]}, projection=[a, c, d], output_ordering=[a@0 ASC NULLS LAST], has_header=true", + ]; + let expected_optimized_bounded_sort_preserve = expected_optimized_bounded; + + assert_optimized_in_all_boundedness_situations!( + expected_input_unbounded, + expected_input_bounded, + expected_optimized_unbounded, + expected_optimized_bounded, + expected_optimized_bounded_sort_preserve, physical_plan, - prefer_existing_sort + source_unbounded ); Ok(()) } @@ -704,35 +1056,67 @@ mod tests { #[rstest] #[tokio::test] async fn test_with_lost_ordering( - #[values(false, true)] prefer_existing_sort: bool, + #[values(false, true)] source_unbounded: bool, ) -> Result<()> { let schema = create_test_schema()?; let sort_exprs = vec![sort_expr("a", &schema)]; - let source = stream_exec_ordered(&schema, sort_exprs); + let source = if source_unbounded { + stream_exec_ordered(&schema, sort_exprs) + } else { + csv_exec_sorted(&schema, sort_exprs) + }; let repartition_rr = repartition_exec_round_robin(source); let repartition_hash = repartition_exec_hash(repartition_rr); let coalesce_partitions = coalesce_partitions_exec(repartition_hash); let physical_plan = sort_exec(vec![sort_expr("a", &schema)], coalesce_partitions, false); - let expected_input = [ + // Expected inputs unbounded and bounded + let expected_input_unbounded = [ "SortExec: expr=[a@0 ASC NULLS LAST]", " CoalescePartitionsExec", " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8", " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", " StreamingTableExec: partition_sizes=1, projection=[a, c, d], infinite_source=true, output_ordering=[a@0 ASC NULLS LAST]", ]; - let expected_optimized = [ + let expected_input_bounded = [ + "SortExec: expr=[a@0 ASC NULLS LAST]", + " CoalescePartitionsExec", + " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8", + " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", + " CsvExec: file_groups={1 group: [[file_path]]}, projection=[a, c, d], output_ordering=[a@0 ASC NULLS LAST], has_header=true", + ]; + + // Expected unbounded result (same for with and without flag) + let expected_optimized_unbounded = [ "SortPreservingMergeExec: [a@0 ASC NULLS LAST]", " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8, preserve_order=true, sort_exprs=a@0 ASC NULLS LAST", " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", " StreamingTableExec: partition_sizes=1, projection=[a, c, d], infinite_source=true, output_ordering=[a@0 ASC NULLS LAST]", ]; - assert_optimized!( - expected_input, - expected_optimized, + + // Expected bounded results with and without flag + let expected_optimized_bounded = [ + "SortExec: expr=[a@0 ASC NULLS LAST]", + " CoalescePartitionsExec", + " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8", + " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", + " CsvExec: file_groups={1 group: [[file_path]]}, projection=[a, c, d], output_ordering=[a@0 ASC NULLS LAST], has_header=true", + ]; + let expected_optimized_bounded_sort_preserve = [ + "SortPreservingMergeExec: [a@0 ASC NULLS LAST]", + " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8, preserve_order=true, sort_exprs=a@0 ASC NULLS LAST", + " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", + " CsvExec: file_groups={1 group: [[file_path]]}, projection=[a, c, d], output_ordering=[a@0 ASC NULLS LAST], has_header=true", + ]; + assert_optimized_in_all_boundedness_situations!( + expected_input_unbounded, + expected_input_bounded, + expected_optimized_unbounded, + expected_optimized_bounded, + expected_optimized_bounded_sort_preserve, physical_plan, - prefer_existing_sort + source_unbounded ); Ok(()) } @@ -740,11 +1124,15 @@ mod tests { #[rstest] #[tokio::test] async fn test_with_lost_and_kept_ordering( - #[values(false, true)] prefer_existing_sort: bool, + #[values(false, true)] source_unbounded: bool, ) -> Result<()> { let schema = create_test_schema()?; let sort_exprs = vec![sort_expr("a", &schema)]; - let source = stream_exec_ordered(&schema, sort_exprs); + let source = if source_unbounded { + stream_exec_ordered(&schema, sort_exprs) + } else { + csv_exec_sorted(&schema, sort_exprs) + }; let repartition_rr = repartition_exec_round_robin(source); let repartition_hash = repartition_exec_hash(repartition_rr); let coalesce_partitions = coalesce_partitions_exec(repartition_hash); @@ -764,7 +1152,8 @@ mod tests { sort2, ); - let expected_input = [ + // Expected inputs unbounded and bounded + let expected_input_unbounded = [ "SortPreservingMergeExec: [c@1 ASC]", " SortExec: expr=[c@1 ASC]", " FilterExec: c@1 > 3", @@ -776,8 +1165,21 @@ mod tests { " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", " StreamingTableExec: partition_sizes=1, projection=[a, c, d], infinite_source=true, output_ordering=[a@0 ASC NULLS LAST]", ]; + let expected_input_bounded = [ + "SortPreservingMergeExec: [c@1 ASC]", + " SortExec: expr=[c@1 ASC]", + " FilterExec: c@1 > 3", + " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8", + " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", + " SortExec: expr=[c@1 ASC]", + " CoalescePartitionsExec", + " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8", + " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", + " CsvExec: file_groups={1 group: [[file_path]]}, projection=[a, c, d], output_ordering=[a@0 ASC NULLS LAST], has_header=true", + ]; - let expected_optimized = [ + // Expected unbounded result (same for with and without flag) + let expected_optimized_unbounded = [ "SortPreservingMergeExec: [c@1 ASC]", " FilterExec: c@1 > 3", " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8, preserve_order=true, sort_exprs=c@1 ASC", @@ -788,11 +1190,39 @@ mod tests { " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", " StreamingTableExec: partition_sizes=1, projection=[a, c, d], infinite_source=true, output_ordering=[a@0 ASC NULLS LAST]", ]; - assert_optimized!( - expected_input, - expected_optimized, + + // Expected bounded results with and without flag + let expected_optimized_bounded = [ + "SortPreservingMergeExec: [c@1 ASC]", + " SortExec: expr=[c@1 ASC]", + " FilterExec: c@1 > 3", + " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8", + " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", + " SortExec: expr=[c@1 ASC]", + " CoalescePartitionsExec", + " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8", + " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", + " CsvExec: file_groups={1 group: [[file_path]]}, projection=[a, c, d], output_ordering=[a@0 ASC NULLS LAST], has_header=true", + ]; + let expected_optimized_bounded_sort_preserve = [ + "SortPreservingMergeExec: [c@1 ASC]", + " FilterExec: c@1 > 3", + " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8, preserve_order=true, sort_exprs=c@1 ASC", + " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", + " SortExec: expr=[c@1 ASC]", + " CoalescePartitionsExec", + " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8", + " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", + " CsvExec: file_groups={1 group: [[file_path]]}, projection=[a, c, d], output_ordering=[a@0 ASC NULLS LAST], has_header=true", + ]; + assert_optimized_in_all_boundedness_situations!( + expected_input_unbounded, + expected_input_bounded, + expected_optimized_unbounded, + expected_optimized_bounded, + expected_optimized_bounded_sort_preserve, physical_plan, - prefer_existing_sort + source_unbounded ); Ok(()) } @@ -800,19 +1230,27 @@ mod tests { #[rstest] #[tokio::test] async fn test_with_multiple_child_trees( - #[values(false, true)] prefer_existing_sort: bool, + #[values(false, true)] source_unbounded: bool, ) -> Result<()> { let schema = create_test_schema()?; let left_sort_exprs = vec![sort_expr("a", &schema)]; - let left_source = stream_exec_ordered(&schema, left_sort_exprs); + let left_source = if source_unbounded { + stream_exec_ordered(&schema, left_sort_exprs) + } else { + csv_exec_sorted(&schema, left_sort_exprs) + }; let left_repartition_rr = repartition_exec_round_robin(left_source); let left_repartition_hash = repartition_exec_hash(left_repartition_rr); let left_coalesce_partitions = Arc::new(CoalesceBatchesExec::new(left_repartition_hash, 4096)); let right_sort_exprs = vec![sort_expr("a", &schema)]; - let right_source = stream_exec_ordered(&schema, right_sort_exprs); + let right_source = if source_unbounded { + stream_exec_ordered(&schema, right_sort_exprs) + } else { + csv_exec_sorted(&schema, right_sort_exprs) + }; let right_repartition_rr = repartition_exec_round_robin(right_source); let right_repartition_hash = repartition_exec_hash(right_repartition_rr); let right_coalesce_partitions = @@ -831,7 +1269,8 @@ mod tests { sort, ); - let expected_input = [ + // Expected inputs unbounded and bounded + let expected_input_unbounded = [ "SortPreservingMergeExec: [a@0 ASC]", " SortExec: expr=[a@0 ASC]", " HashJoinExec: mode=Partitioned, join_type=Inner, on=[(c@1, c@1)]", @@ -844,8 +1283,22 @@ mod tests { " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", " StreamingTableExec: partition_sizes=1, projection=[a, c, d], infinite_source=true, output_ordering=[a@0 ASC NULLS LAST]", ]; + let expected_input_bounded = [ + "SortPreservingMergeExec: [a@0 ASC]", + " SortExec: expr=[a@0 ASC]", + " HashJoinExec: mode=Partitioned, join_type=Inner, on=[(c@1, c@1)]", + " CoalesceBatchesExec: target_batch_size=4096", + " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8", + " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", + " CsvExec: file_groups={1 group: [[file_path]]}, projection=[a, c, d], output_ordering=[a@0 ASC NULLS LAST], has_header=true", + " CoalesceBatchesExec: target_batch_size=4096", + " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8", + " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", + " CsvExec: file_groups={1 group: [[file_path]]}, projection=[a, c, d], output_ordering=[a@0 ASC NULLS LAST], has_header=true", + ]; - let expected_optimized = [ + // Expected unbounded result (same for with and without flag) + let expected_optimized_unbounded = [ "SortPreservingMergeExec: [a@0 ASC]", " SortExec: expr=[a@0 ASC]", " HashJoinExec: mode=Partitioned, join_type=Inner, on=[(c@1, c@1)]", @@ -858,11 +1311,32 @@ mod tests { " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", " StreamingTableExec: partition_sizes=1, projection=[a, c, d], infinite_source=true, output_ordering=[a@0 ASC NULLS LAST]", ]; - assert_optimized!( - expected_input, - expected_optimized, + + // Expected bounded results same with and without flag, because ordering get lost during intermediate executor anyway. Hence no need to preserve + // existing ordering. + let expected_optimized_bounded = [ + "SortPreservingMergeExec: [a@0 ASC]", + " SortExec: expr=[a@0 ASC]", + " HashJoinExec: mode=Partitioned, join_type=Inner, on=[(c@1, c@1)]", + " CoalesceBatchesExec: target_batch_size=4096", + " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8", + " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", + " CsvExec: file_groups={1 group: [[file_path]]}, projection=[a, c, d], output_ordering=[a@0 ASC NULLS LAST], has_header=true", + " CoalesceBatchesExec: target_batch_size=4096", + " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8", + " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", + " CsvExec: file_groups={1 group: [[file_path]]}, projection=[a, c, d], output_ordering=[a@0 ASC NULLS LAST], has_header=true", + ]; + let expected_optimized_bounded_sort_preserve = expected_optimized_bounded; + + assert_optimized_in_all_boundedness_situations!( + expected_input_unbounded, + expected_input_bounded, + expected_optimized_unbounded, + expected_optimized_bounded, + expected_optimized_bounded_sort_preserve, physical_plan, - prefer_existing_sort + source_unbounded ); Ok(()) } @@ -985,8 +1459,7 @@ mod tests { Ok(schema) } - // creates a csv exec source for the test purposes - // projection and has_header parameters are given static due to testing needs + // creates a stream exec source for the test purposes fn stream_exec_ordered( schema: &SchemaRef, sort_exprs: impl IntoIterator, @@ -1007,4 +1480,35 @@ mod tests { .unwrap(), ) } + + // creates a csv exec source for the test purposes + // projection and has_header parameters are given static due to testing needs + fn csv_exec_sorted( + schema: &SchemaRef, + sort_exprs: impl IntoIterator, + ) -> Arc { + let sort_exprs = sort_exprs.into_iter().collect(); + let projection: Vec = vec![0, 2, 3]; + + Arc::new(CsvExec::new( + FileScanConfig { + object_store_url: ObjectStoreUrl::parse("test:///").unwrap(), + file_schema: schema.clone(), + file_groups: vec![vec![PartitionedFile::new( + "file_path".to_string(), + 100, + )]], + statistics: Statistics::new_unknown(schema), + projection: Some(projection), + limit: None, + table_partition_cols: vec![], + output_ordering: vec![sort_exprs], + }, + true, + 0, + b'"', + None, + FileCompressionType::UNCOMPRESSED, + )) + } } diff --git a/datafusion/core/src/test/mod.rs b/datafusion/core/src/test/mod.rs index 7a63466a3906..ed5aa15e291b 100644 --- a/datafusion/core/src/test/mod.rs +++ b/datafusion/core/src/test/mod.rs @@ -43,13 +43,13 @@ use arrow::record_batch::RecordBatch; use datafusion_common::{DataFusionError, FileType, Statistics}; use datafusion_execution::{SendableRecordBatchStream, TaskContext}; use datafusion_physical_expr::{Partitioning, PhysicalSortExpr}; +use datafusion_physical_plan::streaming::{PartitionStream, StreamingTableExec}; use datafusion_physical_plan::{DisplayAs, DisplayFormatType}; #[cfg(feature = "compression")] use bzip2::write::BzEncoder; #[cfg(feature = "compression")] use bzip2::Compression as BzCompression; -use datafusion_physical_plan::streaming::{PartitionStream, StreamingTableExec}; #[cfg(feature = "compression")] use flate2::write::GzEncoder; #[cfg(feature = "compression")] @@ -334,6 +334,32 @@ pub fn stream_exec_ordered( ) } +/// Create a csv exec for tests +pub fn csv_exec_ordered( + schema: &SchemaRef, + sort_exprs: impl IntoIterator, +) -> Arc { + let sort_exprs = sort_exprs.into_iter().collect(); + + Arc::new(CsvExec::new( + FileScanConfig { + object_store_url: ObjectStoreUrl::parse("test:///").unwrap(), + file_schema: schema.clone(), + file_groups: vec![vec![PartitionedFile::new("file_path".to_string(), 100)]], + statistics: Statistics::new_unknown(schema), + projection: None, + limit: None, + table_partition_cols: vec![], + output_ordering: vec![sort_exprs], + }, + true, + 0, + b'"', + None, + FileCompressionType::UNCOMPRESSED, + )) +} + /// A mock execution plan that simply returns the provided statistics #[derive(Debug, Clone)] pub struct StatisticsExec { From f041e73b48e426a3679301d3b28c9dc4410a8d97 Mon Sep 17 00:00:00 2001 From: Trevor Hilton Date: Tue, 19 Dec 2023 14:03:50 -0500 Subject: [PATCH 464/572] Port tests in `parquet.rs` to sqllogictest (#8560) * setup parquet.slt and port parquet_query test to it * port parquet_with_sort_order_specified, but missing files * port fixed_size_binary_columns test * port window_fn_timestamp_tz test * port parquet_single_nan_schema test * port parquet_query_with_max_min test * use COPY to create tables in parquet.slt to test partitioning over multi-file data * remove unneeded optimizer setting; check type of timestamp column --- datafusion/core/tests/sql/parquet.rs | 292 ----------------- .../sqllogictest/test_files/parquet.slt | 304 ++++++++++++++++++ 2 files changed, 304 insertions(+), 292 deletions(-) create mode 100644 datafusion/sqllogictest/test_files/parquet.slt diff --git a/datafusion/core/tests/sql/parquet.rs b/datafusion/core/tests/sql/parquet.rs index 8f810a929df3..f80a28f7e4f9 100644 --- a/datafusion/core/tests/sql/parquet.rs +++ b/datafusion/core/tests/sql/parquet.rs @@ -15,207 +15,10 @@ // specific language governing permissions and limitations // under the License. -use std::{fs, path::Path}; - -use ::parquet::arrow::ArrowWriter; -use datafusion::{datasource::listing::ListingOptions, execution::options::ReadOptions}; use datafusion_common::cast::{as_list_array, as_primitive_array, as_string_array}; -use tempfile::TempDir; use super::*; -#[tokio::test] -async fn parquet_query() { - let ctx = SessionContext::new(); - register_alltypes_parquet(&ctx).await; - // NOTE that string_col is actually a binary column and does not have the UTF8 logical type - // so we need an explicit cast - let sql = "SELECT id, CAST(string_col AS varchar) FROM alltypes_plain"; - let actual = execute_to_batches(&ctx, sql).await; - let expected = [ - "+----+---------------------------+", - "| id | alltypes_plain.string_col |", - "+----+---------------------------+", - "| 4 | 0 |", - "| 5 | 1 |", - "| 6 | 0 |", - "| 7 | 1 |", - "| 2 | 0 |", - "| 3 | 1 |", - "| 0 | 0 |", - "| 1 | 1 |", - "+----+---------------------------+", - ]; - - assert_batches_eq!(expected, &actual); -} - -#[tokio::test] -/// Test that if sort order is specified in ListingOptions, the sort -/// expressions make it all the way down to the ParquetExec -async fn parquet_with_sort_order_specified() { - let parquet_read_options = ParquetReadOptions::default(); - let session_config = SessionConfig::new().with_target_partitions(2); - - // The sort order is not specified - let options_no_sort = parquet_read_options.to_listing_options(&session_config); - - // The sort order is specified (not actually correct in this case) - let file_sort_order = [col("string_col"), col("int_col")] - .into_iter() - .map(|e| { - let ascending = true; - let nulls_first = false; - e.sort(ascending, nulls_first) - }) - .collect::>(); - - let options_sort = parquet_read_options - .to_listing_options(&session_config) - .with_file_sort_order(vec![file_sort_order]); - - // This string appears in ParquetExec if the output ordering is - // specified - let expected_output_ordering = - "output_ordering=[string_col@1 ASC NULLS LAST, int_col@0 ASC NULLS LAST]"; - - // when sort not specified, should not appear in the explain plan - let num_files = 1; - assert_not_contains!( - run_query_with_options(options_no_sort, num_files).await, - expected_output_ordering - ); - - // when sort IS specified, SHOULD appear in the explain plan - let num_files = 1; - assert_contains!( - run_query_with_options(options_sort.clone(), num_files).await, - expected_output_ordering - ); - - // when sort IS specified, but there are too many files (greater - // than the number of partitions) sort should not appear - let num_files = 3; - assert_not_contains!( - run_query_with_options(options_sort, num_files).await, - expected_output_ordering - ); -} - -/// Runs a limit query against a parquet file that was registered from -/// options on num_files copies of all_types_plain.parquet -async fn run_query_with_options(options: ListingOptions, num_files: usize) -> String { - let ctx = SessionContext::new(); - - let testdata = datafusion::test_util::parquet_test_data(); - let file_path = format!("{testdata}/alltypes_plain.parquet"); - - // Create a directory of parquet files with names - // 0.parquet - // 1.parquet - let tmpdir = TempDir::new().unwrap(); - for i in 0..num_files { - let target_file = tmpdir.path().join(format!("{i}.parquet")); - println!("Copying {file_path} to {target_file:?}"); - std::fs::copy(&file_path, target_file).unwrap(); - } - - let provided_schema = None; - let sql_definition = None; - ctx.register_listing_table( - "t", - tmpdir.path().to_string_lossy(), - options.clone(), - provided_schema, - sql_definition, - ) - .await - .unwrap(); - - let batches = ctx.sql("explain select int_col, string_col from t order by string_col, int_col limit 10") - .await - .expect("planing worked") - .collect() - .await - .expect("execution worked"); - - arrow::util::pretty::pretty_format_batches(&batches) - .unwrap() - .to_string() -} - -#[tokio::test] -async fn fixed_size_binary_columns() { - let ctx = SessionContext::new(); - ctx.register_parquet( - "t0", - "tests/data/test_binary.parquet", - ParquetReadOptions::default(), - ) - .await - .unwrap(); - let sql = "SELECT ids FROM t0 ORDER BY ids"; - let dataframe = ctx.sql(sql).await.unwrap(); - let results = dataframe.collect().await.unwrap(); - for batch in results { - assert_eq!(466, batch.num_rows()); - assert_eq!(1, batch.num_columns()); - } -} - -#[tokio::test] -async fn window_fn_timestamp_tz() { - let ctx = SessionContext::new(); - ctx.register_parquet( - "t0", - "tests/data/timestamp_with_tz.parquet", - ParquetReadOptions::default(), - ) - .await - .unwrap(); - - let sql = "SELECT count, LAG(timestamp, 1) OVER (ORDER BY timestamp) FROM t0"; - let dataframe = ctx.sql(sql).await.unwrap(); - let results = dataframe.collect().await.unwrap(); - - let mut num_rows = 0; - for batch in results { - num_rows += batch.num_rows(); - assert_eq!(2, batch.num_columns()); - - let ty = batch.column(0).data_type().clone(); - assert_eq!(DataType::Int64, ty); - - let ty = batch.column(1).data_type().clone(); - assert_eq!( - DataType::Timestamp(TimeUnit::Millisecond, Some("UTC".into())), - ty - ); - } - - assert_eq!(131072, num_rows); -} - -#[tokio::test] -async fn parquet_single_nan_schema() { - let ctx = SessionContext::new(); - let testdata = datafusion::test_util::parquet_test_data(); - ctx.register_parquet( - "single_nan", - &format!("{testdata}/single_nan.parquet"), - ParquetReadOptions::default(), - ) - .await - .unwrap(); - let sql = "SELECT mycol FROM single_nan"; - let dataframe = ctx.sql(sql).await.unwrap(); - let results = dataframe.collect().await.unwrap(); - for batch in results { - assert_eq!(1, batch.num_rows()); - assert_eq!(1, batch.num_columns()); - } -} - #[tokio::test] #[ignore = "Test ignored, will be enabled as part of the nested Parquet reader"] async fn parquet_list_columns() { @@ -286,98 +89,3 @@ async fn parquet_list_columns() { assert_eq!(result.value(2), "hij"); assert_eq!(result.value(3), "xyz"); } - -#[tokio::test] -async fn parquet_query_with_max_min() { - let tmp_dir = TempDir::new().unwrap(); - let table_dir = tmp_dir.path().join("parquet_test"); - let table_path = Path::new(&table_dir); - - let fields = vec![ - Field::new("c1", DataType::Int32, true), - Field::new("c2", DataType::Utf8, true), - Field::new("c3", DataType::Int64, true), - Field::new("c4", DataType::Date32, true), - ]; - - let schema = Arc::new(Schema::new(fields.clone())); - - if let Ok(()) = fs::create_dir(table_path) { - let filename = "foo.parquet"; - let path = table_path.join(filename); - let file = fs::File::create(path).unwrap(); - let mut writer = - ArrowWriter::try_new(file.try_clone().unwrap(), schema.clone(), None) - .unwrap(); - - // create mock record batch - let c1s = Arc::new(Int32Array::from(vec![1, 2, 3])); - let c2s = Arc::new(StringArray::from(vec!["aaa", "bbb", "ccc"])); - let c3s = Arc::new(Int64Array::from(vec![100, 200, 300])); - let c4s = Arc::new(Date32Array::from(vec![Some(1), Some(2), Some(3)])); - let rec_batch = - RecordBatch::try_new(schema.clone(), vec![c1s, c2s, c3s, c4s]).unwrap(); - - writer.write(&rec_batch).unwrap(); - writer.close().unwrap(); - } - - // query parquet - let ctx = SessionContext::new(); - - ctx.register_parquet( - "foo", - &format!("{}/foo.parquet", table_dir.to_str().unwrap()), - ParquetReadOptions::default(), - ) - .await - .unwrap(); - - let sql = "SELECT max(c1) FROM foo"; - let actual = execute_to_batches(&ctx, sql).await; - let expected = [ - "+-------------+", - "| MAX(foo.c1) |", - "+-------------+", - "| 3 |", - "+-------------+", - ]; - - assert_batches_eq!(expected, &actual); - - let sql = "SELECT min(c2) FROM foo"; - let actual = execute_to_batches(&ctx, sql).await; - let expected = [ - "+-------------+", - "| MIN(foo.c2) |", - "+-------------+", - "| aaa |", - "+-------------+", - ]; - - assert_batches_eq!(expected, &actual); - - let sql = "SELECT max(c3) FROM foo"; - let actual = execute_to_batches(&ctx, sql).await; - let expected = [ - "+-------------+", - "| MAX(foo.c3) |", - "+-------------+", - "| 300 |", - "+-------------+", - ]; - - assert_batches_eq!(expected, &actual); - - let sql = "SELECT min(c4) FROM foo"; - let actual = execute_to_batches(&ctx, sql).await; - let expected = [ - "+-------------+", - "| MIN(foo.c4) |", - "+-------------+", - "| 1970-01-02 |", - "+-------------+", - ]; - - assert_batches_eq!(expected, &actual); -} diff --git a/datafusion/sqllogictest/test_files/parquet.slt b/datafusion/sqllogictest/test_files/parquet.slt new file mode 100644 index 000000000000..bbe7f33e260c --- /dev/null +++ b/datafusion/sqllogictest/test_files/parquet.slt @@ -0,0 +1,304 @@ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at + +# http://www.apache.org/licenses/LICENSE-2.0 + +# Unless required by applicable law or agreed to in writing, +# software distributed under the License is distributed on an +# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +# KIND, either express or implied. See the License for the +# specific language governing permissions and limitations +# under the License. + +# TESTS FOR PARQUET FILES + +# Set 2 partitions for deterministic output plans +statement ok +set datafusion.execution.target_partitions = 2; + +# Create a table as a data source +statement ok +CREATE TABLE src_table ( + int_col INT, + string_col TEXT, + bigint_col BIGINT, + date_col DATE +) AS VALUES +(1, 'aaa', 100, 1), +(2, 'bbb', 200, 2), +(3, 'ccc', 300, 3), +(4, 'ddd', 400, 4), +(5, 'eee', 500, 5), +(6, 'fff', 600, 6), +(7, 'ggg', 700, 7), +(8, 'hhh', 800, 8), +(9, 'iii', 900, 9); + +# Setup 2 files, i.e., as many as there are partitions: + +# File 1: +query ITID +COPY (SELECT * FROM src_table LIMIT 3) +TO 'test_files/scratch/parquet/test_table/0.parquet' +(FORMAT PARQUET, SINGLE_FILE_OUTPUT true); +---- +3 + +# File 2: +query ITID +COPY (SELECT * FROM src_table WHERE int_col > 3 LIMIT 3) +TO 'test_files/scratch/parquet/test_table/1.parquet' +(FORMAT PARQUET, SINGLE_FILE_OUTPUT true); +---- +3 + +# Create a table from generated parquet files, without ordering: +statement ok +CREATE EXTERNAL TABLE test_table ( + int_col INT, + string_col TEXT, + bigint_col BIGINT, + date_col DATE +) +STORED AS PARQUET +WITH HEADER ROW +LOCATION 'test_files/scratch/parquet/test_table'; + +# Basic query: +query ITID +SELECT * FROM test_table ORDER BY int_col; +---- +1 aaa 100 1970-01-02 +2 bbb 200 1970-01-03 +3 ccc 300 1970-01-04 +4 ddd 400 1970-01-05 +5 eee 500 1970-01-06 +6 fff 600 1970-01-07 + +# Check output plan, expect no "output_ordering" clause in the physical_plan -> ParquetExec: +query TT +EXPLAIN SELECT int_col, string_col +FROM test_table +ORDER BY string_col, int_col; +---- +logical_plan +Sort: test_table.string_col ASC NULLS LAST, test_table.int_col ASC NULLS LAST +--TableScan: test_table projection=[int_col, string_col] +physical_plan +SortPreservingMergeExec: [string_col@1 ASC NULLS LAST,int_col@0 ASC NULLS LAST] +--SortExec: expr=[string_col@1 ASC NULLS LAST,int_col@0 ASC NULLS LAST] +----ParquetExec: file_groups={2 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet/test_table/0.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet/test_table/1.parquet]]}, projection=[int_col, string_col] + +# Tear down test_table: +statement ok +DROP TABLE test_table; + +# Create test_table again, but with ordering: +statement ok +CREATE EXTERNAL TABLE test_table ( + int_col INT, + string_col TEXT, + bigint_col BIGINT, + date_col DATE +) +STORED AS PARQUET +WITH HEADER ROW +WITH ORDER (string_col ASC NULLS LAST, int_col ASC NULLS LAST) +LOCATION 'test_files/scratch/parquet/test_table'; + +# Check output plan, expect an "output_ordering" clause in the physical_plan -> ParquetExec: +query TT +EXPLAIN SELECT int_col, string_col +FROM test_table +ORDER BY string_col, int_col; +---- +logical_plan +Sort: test_table.string_col ASC NULLS LAST, test_table.int_col ASC NULLS LAST +--TableScan: test_table projection=[int_col, string_col] +physical_plan +SortPreservingMergeExec: [string_col@1 ASC NULLS LAST,int_col@0 ASC NULLS LAST] +--ParquetExec: file_groups={2 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet/test_table/0.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet/test_table/1.parquet]]}, projection=[int_col, string_col], output_ordering=[string_col@1 ASC NULLS LAST, int_col@0 ASC NULLS LAST] + +# Add another file to the directory underlying test_table +query ITID +COPY (SELECT * FROM src_table WHERE int_col > 6 LIMIT 3) +TO 'test_files/scratch/parquet/test_table/2.parquet' +(FORMAT PARQUET, SINGLE_FILE_OUTPUT true); +---- +3 + +# Check output plan again, expect no "output_ordering" clause in the physical_plan -> ParquetExec, +# due to there being more files than partitions: +query TT +EXPLAIN SELECT int_col, string_col +FROM test_table +ORDER BY string_col, int_col; +---- +logical_plan +Sort: test_table.string_col ASC NULLS LAST, test_table.int_col ASC NULLS LAST +--TableScan: test_table projection=[int_col, string_col] +physical_plan +SortPreservingMergeExec: [string_col@1 ASC NULLS LAST,int_col@0 ASC NULLS LAST] +--SortExec: expr=[string_col@1 ASC NULLS LAST,int_col@0 ASC NULLS LAST] +----ParquetExec: file_groups={2 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet/test_table/0.parquet, WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet/test_table/1.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet/test_table/2.parquet]]}, projection=[int_col, string_col] + + +# Perform queries using MIN and MAX +query I +SELECT max(int_col) FROM test_table; +---- +9 + +query T +SELECT min(string_col) FROM test_table; +---- +aaa + +query I +SELECT max(bigint_col) FROM test_table; +---- +900 + +query D +SELECT min(date_col) FROM test_table; +---- +1970-01-02 + +# Clean up +statement ok +DROP TABLE test_table; + +# Setup alltypes_plain table: +statement ok +CREATE EXTERNAL TABLE alltypes_plain ( + id INT NOT NULL, + bool_col BOOLEAN NOT NULL, + tinyint_col TINYINT NOT NULL, + smallint_col SMALLINT NOT NULL, + int_col INT NOT NULL, + bigint_col BIGINT NOT NULL, + float_col FLOAT NOT NULL, + double_col DOUBLE NOT NULL, + date_string_col BYTEA NOT NULL, + string_col VARCHAR NOT NULL, + timestamp_col TIMESTAMP NOT NULL, +) +STORED AS PARQUET +WITH HEADER ROW +LOCATION '../../parquet-testing/data/alltypes_plain.parquet' + +# Test a basic query with a CAST: +query IT +SELECT id, CAST(string_col AS varchar) FROM alltypes_plain +---- +4 0 +5 1 +6 0 +7 1 +2 0 +3 1 +0 0 +1 1 + +# Clean up +statement ok +DROP TABLE alltypes_plain; + +# Perform SELECT on table with fixed sized binary columns + +statement ok +CREATE EXTERNAL TABLE test_binary +STORED AS PARQUET +WITH HEADER ROW +LOCATION '../core/tests/data/test_binary.parquet'; + +# Check size of table: +query I +SELECT count(ids) FROM test_binary; +---- +466 + +# Do the SELECT query: +query ? +SELECT ids FROM test_binary ORDER BY ids LIMIT 10; +---- +008c7196f68089ab692e4739c5fd16b5 +00a51a7bc5ff8eb1627f8f3dc959dce8 +0166ce1d46129ad104fa4990c6057c91 +03a4893f3285b422820b4cd74c9b9786 +04999ac861e14682cd339eae2cc74359 +04b86bf8f228739fde391f850636a77d +050fb9cf722a709eb94b70b3ee7dc342 +052578a65e8e91b8526b182d40e846e8 +05408e6a403e4296526006e20cc4a45a +0592e6fb7d7169b888a4029b53abb701 + +# Clean up +statement ok +DROP TABLE test_binary; + +# Perform a query with a window function and timestamp data: + +statement ok +CREATE EXTERNAL TABLE timestamp_with_tz +STORED AS PARQUET +WITH HEADER ROW +LOCATION '../core/tests/data/timestamp_with_tz.parquet'; + +# Check size of table: +query I +SELECT COUNT(*) FROM timestamp_with_tz; +---- +131072 + +# Perform the query: +query IPT +SELECT + count, + LAG(timestamp, 1) OVER (ORDER BY timestamp), + arrow_typeof(LAG(timestamp, 1) OVER (ORDER BY timestamp)) +FROM timestamp_with_tz +LIMIT 10; +---- +0 NULL Timestamp(Millisecond, Some("UTC")) +0 2014-08-27T14:00:00Z Timestamp(Millisecond, Some("UTC")) +0 2014-08-27T14:00:00Z Timestamp(Millisecond, Some("UTC")) +4 2014-08-27T14:00:00Z Timestamp(Millisecond, Some("UTC")) +0 2014-08-27T14:00:00Z Timestamp(Millisecond, Some("UTC")) +0 2014-08-27T14:00:00Z Timestamp(Millisecond, Some("UTC")) +0 2014-08-27T14:00:00Z Timestamp(Millisecond, Some("UTC")) +14 2014-08-27T14:00:00Z Timestamp(Millisecond, Some("UTC")) +0 2014-08-27T14:00:00Z Timestamp(Millisecond, Some("UTC")) +0 2014-08-27T14:00:00Z Timestamp(Millisecond, Some("UTC")) + +# Clean up +statement ok +DROP TABLE timestamp_with_tz; + +# Test a query from the single_nan data set: +statement ok +CREATE EXTERNAL TABLE single_nan +STORED AS PARQUET +WITH HEADER ROW +LOCATION '../../parquet-testing/data/single_nan.parquet'; + +# Check table size: +query I +SELECT COUNT(*) FROM single_nan; +---- +1 + +# Query for the single NULL: +query R +SELECT mycol FROM single_nan; +---- +NULL + +# Clean up +statement ok +DROP TABLE single_nan; From b456cf78db87bd1369b79a7eec4e3764f551982d Mon Sep 17 00:00:00 2001 From: Andrew Lamb Date: Tue, 19 Dec 2023 14:56:34 -0500 Subject: [PATCH 465/572] Minor: avoid a copy in Expr::unalias (#8588) --- datafusion/expr/src/expr.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/datafusion/expr/src/expr.rs b/datafusion/expr/src/expr.rs index f0aab95b8f0d..b46e9ec8f69d 100644 --- a/datafusion/expr/src/expr.rs +++ b/datafusion/expr/src/expr.rs @@ -956,7 +956,7 @@ impl Expr { /// Remove an alias from an expression if one exists. pub fn unalias(self) -> Expr { match self { - Expr::Alias(alias) => alias.expr.as_ref().clone(), + Expr::Alias(alias) => *alias.expr, _ => self, } } From 1bcaac4835457627d881f755a87dbd140ec3388c Mon Sep 17 00:00:00 2001 From: Kun Liu Date: Wed, 20 Dec 2023 10:11:29 +0800 Subject: [PATCH 466/572] Minor: support complex expr as the arg in the ApproxPercentileCont function (#8580) * support complex lit expr for the arg * enchancement the percentile --- .../tests/dataframe/dataframe_functions.rs | 20 +++++++++ .../src/aggregate/approx_percentile_cont.rs | 45 +++++++++---------- 2 files changed, 41 insertions(+), 24 deletions(-) diff --git a/datafusion/core/tests/dataframe/dataframe_functions.rs b/datafusion/core/tests/dataframe/dataframe_functions.rs index 9677003ec226..fe56fc22ea8c 100644 --- a/datafusion/core/tests/dataframe/dataframe_functions.rs +++ b/datafusion/core/tests/dataframe/dataframe_functions.rs @@ -31,6 +31,7 @@ use datafusion::prelude::*; use datafusion::execution::context::SessionContext; use datafusion::assert_batches_eq; +use datafusion_expr::expr::Alias; use datafusion_expr::{approx_median, cast}; async fn create_test_table() -> Result { @@ -186,6 +187,25 @@ async fn test_fn_approx_percentile_cont() -> Result<()> { assert_batches_eq!(expected, &batches); + // the arg2 parameter is a complex expr, but it can be evaluated to the literal value + let alias_expr = Expr::Alias(Alias::new( + cast(lit(0.5), DataType::Float32), + None::<&str>, + "arg_2".to_string(), + )); + let expr = approx_percentile_cont(col("b"), alias_expr); + let df = create_test_table().await?; + let expected = [ + "+--------------------------------------+", + "| APPROX_PERCENTILE_CONT(test.b,arg_2) |", + "+--------------------------------------+", + "| 10 |", + "+--------------------------------------+", + ]; + let batches = df.aggregate(vec![], vec![expr]).unwrap().collect().await?; + + assert_batches_eq!(expected, &batches); + Ok(()) } diff --git a/datafusion/physical-expr/src/aggregate/approx_percentile_cont.rs b/datafusion/physical-expr/src/aggregate/approx_percentile_cont.rs index aa4749f64ae9..15c0fb3ace4d 100644 --- a/datafusion/physical-expr/src/aggregate/approx_percentile_cont.rs +++ b/datafusion/physical-expr/src/aggregate/approx_percentile_cont.rs @@ -18,7 +18,7 @@ use crate::aggregate::tdigest::TryIntoF64; use crate::aggregate::tdigest::{TDigest, DEFAULT_MAX_SIZE}; use crate::aggregate::utils::down_cast_any_ref; -use crate::expressions::{format_state_name, Literal}; +use crate::expressions::format_state_name; use crate::{AggregateExpr, PhysicalExpr}; use arrow::{ array::{ @@ -27,11 +27,13 @@ use arrow::{ }, datatypes::{DataType, Field}, }; +use arrow_array::RecordBatch; +use arrow_schema::Schema; use datafusion_common::{ downcast_value, exec_err, internal_err, not_impl_err, plan_err, DataFusionError, Result, ScalarValue, }; -use datafusion_expr::Accumulator; +use datafusion_expr::{Accumulator, ColumnarValue}; use std::{any::Any, iter, sync::Arc}; /// APPROX_PERCENTILE_CONT aggregate expression @@ -131,18 +133,22 @@ impl PartialEq for ApproxPercentileCont { } } +fn get_lit_value(expr: &Arc) -> Result { + let empty_schema = Schema::empty(); + let empty_batch = RecordBatch::new_empty(Arc::new(empty_schema)); + let result = expr.evaluate(&empty_batch)?; + match result { + ColumnarValue::Array(_) => Err(DataFusionError::Internal(format!( + "The expr {:?} can't be evaluated to scalar value", + expr + ))), + ColumnarValue::Scalar(scalar_value) => Ok(scalar_value), + } +} + fn validate_input_percentile_expr(expr: &Arc) -> Result { - // Extract the desired percentile literal - let lit = expr - .as_any() - .downcast_ref::() - .ok_or_else(|| { - DataFusionError::Internal( - "desired percentile argument must be float literal".to_string(), - ) - })? - .value(); - let percentile = match lit { + let lit = get_lit_value(expr)?; + let percentile = match &lit { ScalarValue::Float32(Some(q)) => *q as f64, ScalarValue::Float64(Some(q)) => *q, got => return not_impl_err!( @@ -161,17 +167,8 @@ fn validate_input_percentile_expr(expr: &Arc) -> Result { } fn validate_input_max_size_expr(expr: &Arc) -> Result { - // Extract the desired percentile literal - let lit = expr - .as_any() - .downcast_ref::() - .ok_or_else(|| { - DataFusionError::Internal( - "desired percentile argument must be float literal".to_string(), - ) - })? - .value(); - let max_size = match lit { + let lit = get_lit_value(expr)?; + let max_size = match &lit { ScalarValue::UInt8(Some(q)) => *q as usize, ScalarValue::UInt16(Some(q)) => *q as usize, ScalarValue::UInt32(Some(q)) => *q as usize, From 6f5230ffc77ec0151a7aa870808d2fb31e6146c7 Mon Sep 17 00:00:00 2001 From: Mustafa Akur <106137913+mustafasrepo@users.noreply.github.com> Date: Wed, 20 Dec 2023 10:58:49 +0300 Subject: [PATCH 467/572] Bugfix: Add functional dependency check and aggregate try_new schema (#8584) * Add functional dependency check and aggregate try_new schema * Update comments, make implementation idiomatic * Use constraint during stream table initialization --- datafusion/common/src/dfschema.rs | 16 ++++ datafusion/core/src/datasource/stream.rs | 3 +- datafusion/expr/src/utils.rs | 13 +-- .../physical-plan/src/aggregates/mod.rs | 92 ++++++++++++++++++- .../sqllogictest/test_files/groupby.slt | 12 +++ 5 files changed, 125 insertions(+), 11 deletions(-) diff --git a/datafusion/common/src/dfschema.rs b/datafusion/common/src/dfschema.rs index e06f947ad5e7..d6e4490cec4c 100644 --- a/datafusion/common/src/dfschema.rs +++ b/datafusion/common/src/dfschema.rs @@ -347,6 +347,22 @@ impl DFSchema { .collect() } + /// Find all fields indices having the given qualifier + pub fn fields_indices_with_qualified( + &self, + qualifier: &TableReference, + ) -> Vec { + self.fields + .iter() + .enumerate() + .filter_map(|(idx, field)| { + field + .qualifier() + .and_then(|q| q.eq(qualifier).then_some(idx)) + }) + .collect() + } + /// Find all fields match the given name pub fn fields_with_unqualified_name(&self, name: &str) -> Vec<&DFField> { self.fields diff --git a/datafusion/core/src/datasource/stream.rs b/datafusion/core/src/datasource/stream.rs index b9b45a6c7470..830cd7a07e46 100644 --- a/datafusion/core/src/datasource/stream.rs +++ b/datafusion/core/src/datasource/stream.rs @@ -64,7 +64,8 @@ impl TableProviderFactory for StreamTableFactory { .with_encoding(encoding) .with_order(cmd.order_exprs.clone()) .with_header(cmd.has_header) - .with_batch_size(state.config().batch_size()); + .with_batch_size(state.config().batch_size()) + .with_constraints(cmd.constraints.clone()); Ok(Arc::new(StreamTable(Arc::new(config)))) } diff --git a/datafusion/expr/src/utils.rs b/datafusion/expr/src/utils.rs index abdd7f5f57f6..09f4842c9e64 100644 --- a/datafusion/expr/src/utils.rs +++ b/datafusion/expr/src/utils.rs @@ -32,6 +32,7 @@ use crate::{ use arrow::datatypes::{DataType, TimeUnit}; use datafusion_common::tree_node::{TreeNode, VisitRecursion}; +use datafusion_common::utils::get_at_indices; use datafusion_common::{ internal_err, plan_datafusion_err, plan_err, Column, DFField, DFSchema, DFSchemaRef, DataFusionError, Result, ScalarValue, TableReference, @@ -425,18 +426,18 @@ pub fn expand_qualified_wildcard( wildcard_options: Option<&WildcardAdditionalOptions>, ) -> Result> { let qualifier = TableReference::from(qualifier); - let qualified_fields: Vec = schema - .fields_with_qualified(&qualifier) - .into_iter() - .cloned() - .collect(); + let qualified_indices = schema.fields_indices_with_qualified(&qualifier); + let projected_func_dependencies = schema + .functional_dependencies() + .project_functional_dependencies(&qualified_indices, qualified_indices.len()); + let qualified_fields = get_at_indices(schema.fields(), &qualified_indices)?; if qualified_fields.is_empty() { return plan_err!("Invalid qualifier {qualifier}"); } let qualified_schema = DFSchema::new_with_metadata(qualified_fields, schema.metadata().clone())? // We can use the functional dependencies as is, since it only stores indices: - .with_functional_dependencies(schema.functional_dependencies().clone())?; + .with_functional_dependencies(projected_func_dependencies)?; let excluded_columns = if let Some(WildcardAdditionalOptions { opt_exclude, opt_except, diff --git a/datafusion/physical-plan/src/aggregates/mod.rs b/datafusion/physical-plan/src/aggregates/mod.rs index c74c4ac0f821..921de96252f0 100644 --- a/datafusion/physical-plan/src/aggregates/mod.rs +++ b/datafusion/physical-plan/src/aggregates/mod.rs @@ -43,7 +43,7 @@ use datafusion_execution::TaskContext; use datafusion_expr::Accumulator; use datafusion_physical_expr::{ aggregate::is_order_sensitive, - equivalence::collapse_lex_req, + equivalence::{collapse_lex_req, ProjectionMapping}, expressions::{Column, Max, Min, UnKnownColumn}, physical_exprs_contains, reverse_order_bys, AggregateExpr, EquivalenceProperties, LexOrdering, LexRequirement, PhysicalExpr, PhysicalSortExpr, PhysicalSortRequirement, @@ -59,7 +59,6 @@ mod topk; mod topk_stream; pub use datafusion_expr::AggregateFunction; -use datafusion_physical_expr::equivalence::ProjectionMapping; pub use datafusion_physical_expr::expressions::create_aggregate_expr; /// Hash aggregate modes @@ -464,7 +463,7 @@ impl AggregateExec { pub fn try_new( mode: AggregateMode, group_by: PhysicalGroupBy, - mut aggr_expr: Vec>, + aggr_expr: Vec>, filter_expr: Vec>>, input: Arc, input_schema: SchemaRef, @@ -482,6 +481,37 @@ impl AggregateExec { group_by.expr.len(), )); let original_schema = Arc::new(original_schema); + AggregateExec::try_new_with_schema( + mode, + group_by, + aggr_expr, + filter_expr, + input, + input_schema, + schema, + original_schema, + ) + } + + /// Create a new hash aggregate execution plan with the given schema. + /// This constructor isn't part of the public API, it is used internally + /// by Datafusion to enforce schema consistency during when re-creating + /// `AggregateExec`s inside optimization rules. Schema field names of an + /// `AggregateExec` depends on the names of aggregate expressions. Since + /// a rule may re-write aggregate expressions (e.g. reverse them) during + /// initialization, field names may change inadvertently if one re-creates + /// the schema in such cases. + #[allow(clippy::too_many_arguments)] + fn try_new_with_schema( + mode: AggregateMode, + group_by: PhysicalGroupBy, + mut aggr_expr: Vec>, + filter_expr: Vec>>, + input: Arc, + input_schema: SchemaRef, + schema: SchemaRef, + original_schema: SchemaRef, + ) -> Result { // Reset ordering requirement to `None` if aggregator is not order-sensitive let mut order_by_expr = aggr_expr .iter() @@ -858,13 +888,15 @@ impl ExecutionPlan for AggregateExec { self: Arc, children: Vec>, ) -> Result> { - let mut me = AggregateExec::try_new( + let mut me = AggregateExec::try_new_with_schema( self.mode, self.group_by.clone(), self.aggr_expr.clone(), self.filter_expr.clone(), children[0].clone(), self.input_schema.clone(), + self.schema.clone(), + self.original_schema.clone(), )?; me.limit = self.limit; Ok(Arc::new(me)) @@ -2162,4 +2194,56 @@ mod tests { assert_eq!(res, common_requirement); Ok(()) } + + #[test] + fn test_agg_exec_same_schema() -> Result<()> { + let schema = Arc::new(Schema::new(vec![ + Field::new("a", DataType::Float32, true), + Field::new("b", DataType::Float32, true), + ])); + + let col_a = col("a", &schema)?; + let col_b = col("b", &schema)?; + let option_desc = SortOptions { + descending: true, + nulls_first: true, + }; + let sort_expr = vec![PhysicalSortExpr { + expr: col_b.clone(), + options: option_desc, + }]; + let sort_expr_reverse = reverse_order_bys(&sort_expr); + let groups = PhysicalGroupBy::new_single(vec![(col_a, "a".to_string())]); + + let aggregates: Vec> = vec![ + Arc::new(FirstValue::new( + col_b.clone(), + "FIRST_VALUE(b)".to_string(), + DataType::Float64, + sort_expr_reverse.clone(), + vec![DataType::Float64], + )), + Arc::new(LastValue::new( + col_b.clone(), + "LAST_VALUE(b)".to_string(), + DataType::Float64, + sort_expr.clone(), + vec![DataType::Float64], + )), + ]; + let blocking_exec = Arc::new(BlockingExec::new(Arc::clone(&schema), 1)); + let aggregate_exec = Arc::new(AggregateExec::try_new( + AggregateMode::Partial, + groups, + aggregates.clone(), + vec![None, None], + blocking_exec.clone(), + schema, + )?); + let new_agg = aggregate_exec + .clone() + .with_new_children(vec![blocking_exec])?; + assert_eq!(new_agg.schema(), aggregate_exec.schema()); + Ok(()) + } } diff --git a/datafusion/sqllogictest/test_files/groupby.slt b/datafusion/sqllogictest/test_files/groupby.slt index 44d30ba0b34c..f1b6a57287b5 100644 --- a/datafusion/sqllogictest/test_files/groupby.slt +++ b/datafusion/sqllogictest/test_files/groupby.slt @@ -4280,3 +4280,15 @@ LIMIT 5 2 0 0 3 0 0 4 0 1 + + +query ITIPTR rowsort +SELECT r.* +FROM sales_global_with_pk as l, sales_global_with_pk as r +LIMIT 5 +---- +0 GRC 0 2022-01-01T06:00:00 EUR 30 +1 FRA 1 2022-01-01T08:00:00 EUR 50 +1 FRA 3 2022-01-02T12:00:00 EUR 200 +1 TUR 2 2022-01-01T11:30:00 TRY 75 +1 TUR 4 2022-01-03T10:00:00 TRY 100 From 8d72196f957147335b3828f44153277126eb3c0f Mon Sep 17 00:00:00 2001 From: Mehmet Ozan Kabak Date: Wed, 20 Dec 2023 17:03:57 +0300 Subject: [PATCH 468/572] Remove GroupByOrderMode (#8593) --- .../physical-plan/src/aggregates/mod.rs | 28 ------------------- 1 file changed, 28 deletions(-) diff --git a/datafusion/physical-plan/src/aggregates/mod.rs b/datafusion/physical-plan/src/aggregates/mod.rs index 921de96252f0..f779322456ca 100644 --- a/datafusion/physical-plan/src/aggregates/mod.rs +++ b/datafusion/physical-plan/src/aggregates/mod.rs @@ -101,34 +101,6 @@ impl AggregateMode { } } -/// Group By expression modes -/// -/// `PartiallyOrdered` and `FullyOrdered` are used to reason about -/// when certain group by keys will never again be seen (and thus can -/// be emitted by the grouping operator). -/// -/// Specifically, each distinct combination of the relevant columns -/// are contiguous in the input, and once a new combination is seen -/// previous combinations are guaranteed never to appear again -#[derive(Debug, Clone, Copy, PartialEq, Eq)] -pub enum GroupByOrderMode { - /// The input is known to be ordered by a preset (prefix but - /// possibly reordered) of the expressions in the `GROUP BY` clause. - /// - /// For example, if the input is ordered by `a, b, c` and we group - /// by `b, a, d`, `PartiallyOrdered` means a subset of group `b, - /// a, d` defines a preset for the existing ordering, in this case - /// `a, b`. - PartiallyOrdered, - /// The input is known to be ordered by *all* the expressions in the - /// `GROUP BY` clause. - /// - /// For example, if the input is ordered by `a, b, c, d` and we group by b, a, - /// `Ordered` means that all of the of group by expressions appear - /// as a preset for the existing ordering, in this case `a, b`. - FullyOrdered, -} - /// Represents `GROUP BY` clause in the plan (including the more general GROUPING SET) /// In the case of a simple `GROUP BY a, b` clause, this will contain the expression [a, b] /// and a single group [false, false]. From b925b78fd8040f858168e439eda5042bd2a34af6 Mon Sep 17 00:00:00 2001 From: Alex Huang Date: Wed, 20 Dec 2023 18:18:56 +0100 Subject: [PATCH 469/572] replace not-impl-err (#8589) --- datafusion/physical-expr/src/array_expressions.rs | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/datafusion/physical-expr/src/array_expressions.rs b/datafusion/physical-expr/src/array_expressions.rs index d39658108337..0a7631918804 100644 --- a/datafusion/physical-expr/src/array_expressions.rs +++ b/datafusion/physical-expr/src/array_expressions.rs @@ -455,7 +455,7 @@ pub fn array_element(args: &[ArrayRef]) -> Result { let indexes = as_int64_array(&args[1])?; general_array_element::(array, indexes) } - _ => not_impl_err!( + _ => exec_err!( "array_element does not support type: {:?}", args[0].data_type() ), @@ -571,7 +571,7 @@ pub fn array_slice(args: &[ArrayRef]) -> Result { let to_array = as_int64_array(&args[2])?; general_array_slice::(array, from_array, to_array) } - _ => not_impl_err!("array_slice does not support type: {:?}", array_data_type), + _ => exec_err!("array_slice does not support type: {:?}", array_data_type), } } @@ -1335,7 +1335,7 @@ pub fn array_positions(args: &[ArrayRef]) -> Result { general_positions::(arr, element) } array_type => { - not_impl_err!("array_positions does not support type '{array_type:?}'.") + exec_err!("array_positions does not support type '{array_type:?}'.") } } } From 0e9c189a2e4f8f6304239d6cbe14f5114a6d0406 Mon Sep 17 00:00:00 2001 From: Tanmay Gujar Date: Wed, 20 Dec 2023 15:48:11 -0500 Subject: [PATCH 470/572] Substrait insubquery (#8363) * testing in subquery support for substrait producer * consumer fails with table not found * testing roundtrip check * pass in ctx to expr * basic test for Insubquery * fix: outer refs in consumer * fix: merge issues * minor fixes * fix: fmt and clippy CI errors * improve error msg in consumer * minor fixes --- .../substrait/src/logical_plan/consumer.rs | 151 +++++++++++++---- .../substrait/src/logical_plan/producer.rs | 155 ++++++++++++++---- .../tests/cases/roundtrip_logical_plan.rs | 18 ++ 3 files changed, 256 insertions(+), 68 deletions(-) diff --git a/datafusion/substrait/src/logical_plan/consumer.rs b/datafusion/substrait/src/logical_plan/consumer.rs index b7fee96bba1c..9931dd15aec8 100644 --- a/datafusion/substrait/src/logical_plan/consumer.rs +++ b/datafusion/substrait/src/logical_plan/consumer.rs @@ -28,7 +28,7 @@ use datafusion::logical_expr::{ }; use datafusion::logical_expr::{ expr, Cast, Extension, GroupingSet, Like, LogicalPlanBuilder, Partitioning, - Repartition, WindowFrameBound, WindowFrameUnits, + Repartition, Subquery, WindowFrameBound, WindowFrameUnits, }; use datafusion::prelude::JoinType; use datafusion::sql::TableReference; @@ -39,6 +39,7 @@ use datafusion::{ scalar::ScalarValue, }; use substrait::proto::exchange_rel::ExchangeKind; +use substrait::proto::expression::subquery::SubqueryType; use substrait::proto::expression::{FieldReference, Literal, ScalarFunction}; use substrait::proto::{ aggregate_function::AggregationInvocation, @@ -61,7 +62,7 @@ use substrait::proto::{ use substrait::proto::{FunctionArgument, SortField}; use datafusion::common::plan_err; -use datafusion::logical_expr::expr::{InList, Sort}; +use datafusion::logical_expr::expr::{InList, InSubquery, Sort}; use std::collections::HashMap; use std::str::FromStr; use std::sync::Arc; @@ -230,7 +231,8 @@ pub async fn from_substrait_rel( let mut exprs: Vec = vec![]; for e in &p.expressions { let x = - from_substrait_rex(e, input.clone().schema(), extensions).await?; + from_substrait_rex(ctx, e, input.clone().schema(), extensions) + .await?; // if the expression is WindowFunction, wrap in a Window relation // before returning and do not add to list of this Projection's expression list // otherwise, add expression to the Projection's expression list @@ -256,7 +258,8 @@ pub async fn from_substrait_rel( ); if let Some(condition) = filter.condition.as_ref() { let expr = - from_substrait_rex(condition, input.schema(), extensions).await?; + from_substrait_rex(ctx, condition, input.schema(), extensions) + .await?; input.filter(expr.as_ref().clone())?.build() } else { not_impl_err!("Filter without an condition is not valid") @@ -288,7 +291,8 @@ pub async fn from_substrait_rel( from_substrait_rel(ctx, input, extensions).await?, ); let sorts = - from_substrait_sorts(&sort.sorts, input.schema(), extensions).await?; + from_substrait_sorts(ctx, &sort.sorts, input.schema(), extensions) + .await?; input.sort(sorts)?.build() } else { not_impl_err!("Sort without an input is not valid") @@ -306,7 +310,8 @@ pub async fn from_substrait_rel( 1 => { for e in &agg.groupings[0].grouping_expressions { let x = - from_substrait_rex(e, input.schema(), extensions).await?; + from_substrait_rex(ctx, e, input.schema(), extensions) + .await?; group_expr.push(x.as_ref().clone()); } } @@ -315,8 +320,13 @@ pub async fn from_substrait_rel( for grouping in &agg.groupings { let mut grouping_set = vec![]; for e in &grouping.grouping_expressions { - let x = from_substrait_rex(e, input.schema(), extensions) - .await?; + let x = from_substrait_rex( + ctx, + e, + input.schema(), + extensions, + ) + .await?; grouping_set.push(x.as_ref().clone()); } grouping_sets.push(grouping_set); @@ -334,7 +344,7 @@ pub async fn from_substrait_rel( for m in &agg.measures { let filter = match &m.filter { Some(fil) => Some(Box::new( - from_substrait_rex(fil, input.schema(), extensions) + from_substrait_rex(ctx, fil, input.schema(), extensions) .await? .as_ref() .clone(), @@ -402,8 +412,8 @@ pub async fn from_substrait_rel( // Otherwise, build join with only the filter, without join keys match &join.expression.as_ref() { Some(expr) => { - let on = - from_substrait_rex(expr, &in_join_schema, extensions).await?; + let on = from_substrait_rex(ctx, expr, &in_join_schema, extensions) + .await?; // The join expression can contain both equal and non-equal ops. // As of datafusion 31.0.0, the equal and non equal join conditions are in separate fields. // So we extract each part as follows: @@ -612,14 +622,16 @@ fn from_substrait_jointype(join_type: i32) -> Result { /// Convert Substrait Sorts to DataFusion Exprs pub async fn from_substrait_sorts( + ctx: &SessionContext, substrait_sorts: &Vec, input_schema: &DFSchema, extensions: &HashMap, ) -> Result> { let mut sorts: Vec = vec![]; for s in substrait_sorts { - let expr = from_substrait_rex(s.expr.as_ref().unwrap(), input_schema, extensions) - .await?; + let expr = + from_substrait_rex(ctx, s.expr.as_ref().unwrap(), input_schema, extensions) + .await?; let asc_nullfirst = match &s.sort_kind { Some(k) => match k { Direction(d) => { @@ -660,13 +672,14 @@ pub async fn from_substrait_sorts( /// Convert Substrait Expressions to DataFusion Exprs pub async fn from_substrait_rex_vec( + ctx: &SessionContext, exprs: &Vec, input_schema: &DFSchema, extensions: &HashMap, ) -> Result> { let mut expressions: Vec = vec![]; for expr in exprs { - let expression = from_substrait_rex(expr, input_schema, extensions).await?; + let expression = from_substrait_rex(ctx, expr, input_schema, extensions).await?; expressions.push(expression.as_ref().clone()); } Ok(expressions) @@ -674,6 +687,7 @@ pub async fn from_substrait_rex_vec( /// Convert Substrait FunctionArguments to DataFusion Exprs pub async fn from_substriat_func_args( + ctx: &SessionContext, arguments: &Vec, input_schema: &DFSchema, extensions: &HashMap, @@ -682,7 +696,7 @@ pub async fn from_substriat_func_args( for arg in arguments { let arg_expr = match &arg.arg_type { Some(ArgType::Value(e)) => { - from_substrait_rex(e, input_schema, extensions).await + from_substrait_rex(ctx, e, input_schema, extensions).await } _ => { not_impl_err!("Aggregated function argument non-Value type not supported") @@ -707,7 +721,7 @@ pub async fn from_substrait_agg_func( for arg in &f.arguments { let arg_expr = match &arg.arg_type { Some(ArgType::Value(e)) => { - from_substrait_rex(e, input_schema, extensions).await + from_substrait_rex(ctx, e, input_schema, extensions).await } _ => { not_impl_err!("Aggregated function argument non-Value type not supported") @@ -745,6 +759,7 @@ pub async fn from_substrait_agg_func( /// Convert Substrait Rex to DataFusion Expr #[async_recursion] pub async fn from_substrait_rex( + ctx: &SessionContext, e: &Expression, input_schema: &DFSchema, extensions: &HashMap, @@ -755,13 +770,18 @@ pub async fn from_substrait_rex( let substrait_list = s.options.as_ref(); Ok(Arc::new(Expr::InList(InList { expr: Box::new( - from_substrait_rex(substrait_expr, input_schema, extensions) + from_substrait_rex(ctx, substrait_expr, input_schema, extensions) .await? .as_ref() .clone(), ), - list: from_substrait_rex_vec(substrait_list, input_schema, extensions) - .await?, + list: from_substrait_rex_vec( + ctx, + substrait_list, + input_schema, + extensions, + ) + .await?, negated: false, }))) } @@ -779,6 +799,7 @@ pub async fn from_substrait_rex( if if_expr.then.is_none() { expr = Some(Box::new( from_substrait_rex( + ctx, if_expr.r#if.as_ref().unwrap(), input_schema, extensions, @@ -793,6 +814,7 @@ pub async fn from_substrait_rex( when_then_expr.push(( Box::new( from_substrait_rex( + ctx, if_expr.r#if.as_ref().unwrap(), input_schema, extensions, @@ -803,6 +825,7 @@ pub async fn from_substrait_rex( ), Box::new( from_substrait_rex( + ctx, if_expr.then.as_ref().unwrap(), input_schema, extensions, @@ -816,7 +839,7 @@ pub async fn from_substrait_rex( // Parse `else` let else_expr = match &if_then.r#else { Some(e) => Some(Box::new( - from_substrait_rex(e, input_schema, extensions) + from_substrait_rex(ctx, e, input_schema, extensions) .await? .as_ref() .clone(), @@ -843,7 +866,7 @@ pub async fn from_substrait_rex( for arg in &f.arguments { let arg_expr = match &arg.arg_type { Some(ArgType::Value(e)) => { - from_substrait_rex(e, input_schema, extensions).await + from_substrait_rex(ctx, e, input_schema, extensions).await } _ => not_impl_err!( "Aggregated function argument non-Value type not supported" @@ -868,14 +891,14 @@ pub async fn from_substrait_rex( (Some(ArgType::Value(l)), Some(ArgType::Value(r))) => { Ok(Arc::new(Expr::BinaryExpr(BinaryExpr { left: Box::new( - from_substrait_rex(l, input_schema, extensions) + from_substrait_rex(ctx, l, input_schema, extensions) .await? .as_ref() .clone(), ), op, right: Box::new( - from_substrait_rex(r, input_schema, extensions) + from_substrait_rex(ctx, r, input_schema, extensions) .await? .as_ref() .clone(), @@ -888,7 +911,7 @@ pub async fn from_substrait_rex( } } ScalarFunctionType::Expr(builder) => { - builder.build(f, input_schema, extensions).await + builder.build(ctx, f, input_schema, extensions).await } } } @@ -900,6 +923,7 @@ pub async fn from_substrait_rex( Some(output_type) => Ok(Arc::new(Expr::Cast(Cast::new( Box::new( from_substrait_rex( + ctx, cast.as_ref().input.as_ref().unwrap().as_ref(), input_schema, extensions, @@ -921,7 +945,8 @@ pub async fn from_substrait_rex( ), }; let order_by = - from_substrait_sorts(&window.sorts, input_schema, extensions).await?; + from_substrait_sorts(ctx, &window.sorts, input_schema, extensions) + .await?; // Substrait does not encode WindowFrameUnits so we're using a simple logic to determine the units // If there is no `ORDER BY`, then by default, the frame counts each row from the lower up to upper boundary // If there is `ORDER BY`, then by default, each frame is a range starting from unbounded preceding to current row @@ -934,12 +959,14 @@ pub async fn from_substrait_rex( Ok(Arc::new(Expr::WindowFunction(expr::WindowFunction { fun: fun?.unwrap(), args: from_substriat_func_args( + ctx, &window.arguments, input_schema, extensions, ) .await?, partition_by: from_substrait_rex_vec( + ctx, &window.partitions, input_schema, extensions, @@ -953,6 +980,51 @@ pub async fn from_substrait_rex( }, }))) } + Some(RexType::Subquery(subquery)) => match &subquery.as_ref().subquery_type { + Some(subquery_type) => match subquery_type { + SubqueryType::InPredicate(in_predicate) => { + if in_predicate.needles.len() != 1 { + Err(DataFusionError::Substrait( + "InPredicate Subquery type must have exactly one Needle expression" + .to_string(), + )) + } else { + let needle_expr = &in_predicate.needles[0]; + let haystack_expr = &in_predicate.haystack; + if let Some(haystack_expr) = haystack_expr { + let haystack_expr = + from_substrait_rel(ctx, haystack_expr, extensions) + .await?; + let outer_refs = haystack_expr.all_out_ref_exprs(); + Ok(Arc::new(Expr::InSubquery(InSubquery { + expr: Box::new( + from_substrait_rex( + ctx, + needle_expr, + input_schema, + extensions, + ) + .await? + .as_ref() + .clone(), + ), + subquery: Subquery { + subquery: Arc::new(haystack_expr), + outer_ref_columns: outer_refs, + }, + negated: false, + }))) + } else { + substrait_err!("InPredicate Subquery type must have a Haystack expression") + } + } + } + _ => substrait_err!("Subquery type not implemented"), + }, + None => { + substrait_err!("Subquery experssion without SubqueryType is not allowed") + } + }, _ => not_impl_err!("unsupported rex_type"), } } @@ -1312,16 +1384,22 @@ impl BuiltinExprBuilder { pub async fn build( self, + ctx: &SessionContext, f: &ScalarFunction, input_schema: &DFSchema, extensions: &HashMap, ) -> Result> { match self.expr_name.as_str() { - "like" => Self::build_like_expr(false, f, input_schema, extensions).await, - "ilike" => Self::build_like_expr(true, f, input_schema, extensions).await, + "like" => { + Self::build_like_expr(ctx, false, f, input_schema, extensions).await + } + "ilike" => { + Self::build_like_expr(ctx, true, f, input_schema, extensions).await + } "not" | "negative" | "is_null" | "is_not_null" | "is_true" | "is_false" | "is_not_true" | "is_not_false" | "is_unknown" | "is_not_unknown" => { - Self::build_unary_expr(&self.expr_name, f, input_schema, extensions).await + Self::build_unary_expr(ctx, &self.expr_name, f, input_schema, extensions) + .await } _ => { not_impl_err!("Unsupported builtin expression: {}", self.expr_name) @@ -1330,6 +1408,7 @@ impl BuiltinExprBuilder { } async fn build_unary_expr( + ctx: &SessionContext, fn_name: &str, f: &ScalarFunction, input_schema: &DFSchema, @@ -1341,7 +1420,7 @@ impl BuiltinExprBuilder { let Some(ArgType::Value(expr_substrait)) = &f.arguments[0].arg_type else { return substrait_err!("Invalid arguments type for {fn_name} expr"); }; - let arg = from_substrait_rex(expr_substrait, input_schema, extensions) + let arg = from_substrait_rex(ctx, expr_substrait, input_schema, extensions) .await? .as_ref() .clone(); @@ -1365,6 +1444,7 @@ impl BuiltinExprBuilder { } async fn build_like_expr( + ctx: &SessionContext, case_insensitive: bool, f: &ScalarFunction, input_schema: &DFSchema, @@ -1378,22 +1458,23 @@ impl BuiltinExprBuilder { let Some(ArgType::Value(expr_substrait)) = &f.arguments[0].arg_type else { return substrait_err!("Invalid arguments type for `{fn_name}` expr"); }; - let expr = from_substrait_rex(expr_substrait, input_schema, extensions) + let expr = from_substrait_rex(ctx, expr_substrait, input_schema, extensions) .await? .as_ref() .clone(); let Some(ArgType::Value(pattern_substrait)) = &f.arguments[1].arg_type else { return substrait_err!("Invalid arguments type for `{fn_name}` expr"); }; - let pattern = from_substrait_rex(pattern_substrait, input_schema, extensions) - .await? - .as_ref() - .clone(); + let pattern = + from_substrait_rex(ctx, pattern_substrait, input_schema, extensions) + .await? + .as_ref() + .clone(); let Some(ArgType::Value(escape_char_substrait)) = &f.arguments[2].arg_type else { return substrait_err!("Invalid arguments type for `{fn_name}` expr"); }; let escape_char_expr = - from_substrait_rex(escape_char_substrait, input_schema, extensions) + from_substrait_rex(ctx, escape_char_substrait, input_schema, extensions) .await? .as_ref() .clone(); diff --git a/datafusion/substrait/src/logical_plan/producer.rs b/datafusion/substrait/src/logical_plan/producer.rs index 50f872544298..926883251a63 100644 --- a/datafusion/substrait/src/logical_plan/producer.rs +++ b/datafusion/substrait/src/logical_plan/producer.rs @@ -36,12 +36,13 @@ use datafusion::common::{substrait_err, DFSchemaRef}; use datafusion::logical_expr::aggregate_function; use datafusion::logical_expr::expr::{ AggregateFunctionDefinition, Alias, BinaryExpr, Case, Cast, GroupingSet, InList, - ScalarFunctionDefinition, Sort, WindowFunction, + InSubquery, ScalarFunctionDefinition, Sort, WindowFunction, }; use datafusion::logical_expr::{expr, Between, JoinConstraint, LogicalPlan, Operator}; use datafusion::prelude::Expr; use prost_types::Any as ProtoAny; use substrait::proto::exchange_rel::{ExchangeKind, RoundRobin, ScatterFields}; +use substrait::proto::expression::subquery::InPredicate; use substrait::proto::expression::window_function::BoundsType; use substrait::proto::{CrossRel, ExchangeRel}; use substrait::{ @@ -58,7 +59,8 @@ use substrait::{ window_function::bound::Kind as BoundKind, window_function::Bound, FieldReference, IfThen, Literal, MaskExpression, ReferenceSegment, RexType, - ScalarFunction, SingularOrList, WindowFunction as SubstraitWindowFunction, + ScalarFunction, SingularOrList, Subquery, + WindowFunction as SubstraitWindowFunction, }, extensions::{ self, @@ -167,7 +169,7 @@ pub fn to_substrait_rel( let expressions = p .expr .iter() - .map(|e| to_substrait_rex(e, p.input.schema(), 0, extension_info)) + .map(|e| to_substrait_rex(ctx, e, p.input.schema(), 0, extension_info)) .collect::>>()?; Ok(Box::new(Rel { rel_type: Some(RelType::Project(Box::new(ProjectRel { @@ -181,6 +183,7 @@ pub fn to_substrait_rel( LogicalPlan::Filter(filter) => { let input = to_substrait_rel(filter.input.as_ref(), ctx, extension_info)?; let filter_expr = to_substrait_rex( + ctx, &filter.predicate, filter.input.schema(), 0, @@ -214,7 +217,9 @@ pub fn to_substrait_rel( let sort_fields = sort .expr .iter() - .map(|e| substrait_sort_field(e, sort.input.schema(), extension_info)) + .map(|e| { + substrait_sort_field(ctx, e, sort.input.schema(), extension_info) + }) .collect::>>()?; Ok(Box::new(Rel { rel_type: Some(RelType::Sort(Box::new(SortRel { @@ -228,6 +233,7 @@ pub fn to_substrait_rel( LogicalPlan::Aggregate(agg) => { let input = to_substrait_rel(agg.input.as_ref(), ctx, extension_info)?; let groupings = to_substrait_groupings( + ctx, &agg.group_expr, agg.input.schema(), extension_info, @@ -235,7 +241,9 @@ pub fn to_substrait_rel( let measures = agg .aggr_expr .iter() - .map(|e| to_substrait_agg_measure(e, agg.input.schema(), extension_info)) + .map(|e| { + to_substrait_agg_measure(ctx, e, agg.input.schema(), extension_info) + }) .collect::>>()?; Ok(Box::new(Rel { @@ -283,6 +291,7 @@ pub fn to_substrait_rel( let in_join_schema = join.left.schema().join(join.right.schema())?; let join_filter = match &join.filter { Some(filter) => Some(to_substrait_rex( + ctx, filter, &Arc::new(in_join_schema), 0, @@ -299,6 +308,7 @@ pub fn to_substrait_rel( Operator::Eq }; let join_on = to_substrait_join_expr( + ctx, &join.on, eq_op, join.left.schema(), @@ -401,6 +411,7 @@ pub fn to_substrait_rel( let mut window_exprs = vec![]; for expr in &window.window_expr { window_exprs.push(to_substrait_rex( + ctx, expr, window.input.schema(), 0, @@ -500,6 +511,7 @@ pub fn to_substrait_rel( } fn to_substrait_join_expr( + ctx: &SessionContext, join_conditions: &Vec<(Expr, Expr)>, eq_op: Operator, left_schema: &DFSchemaRef, @@ -513,9 +525,10 @@ fn to_substrait_join_expr( let mut exprs: Vec = vec![]; for (left, right) in join_conditions { // Parse left - let l = to_substrait_rex(left, left_schema, 0, extension_info)?; + let l = to_substrait_rex(ctx, left, left_schema, 0, extension_info)?; // Parse right let r = to_substrait_rex( + ctx, right, right_schema, left_schema.fields().len(), // offset to return the correct index @@ -576,6 +589,7 @@ pub fn operator_to_name(op: Operator) -> &'static str { } pub fn parse_flat_grouping_exprs( + ctx: &SessionContext, exprs: &[Expr], schema: &DFSchemaRef, extension_info: &mut ( @@ -585,7 +599,7 @@ pub fn parse_flat_grouping_exprs( ) -> Result { let grouping_expressions = exprs .iter() - .map(|e| to_substrait_rex(e, schema, 0, extension_info)) + .map(|e| to_substrait_rex(ctx, e, schema, 0, extension_info)) .collect::>>()?; Ok(Grouping { grouping_expressions, @@ -593,6 +607,7 @@ pub fn parse_flat_grouping_exprs( } pub fn to_substrait_groupings( + ctx: &SessionContext, exprs: &Vec, schema: &DFSchemaRef, extension_info: &mut ( @@ -608,7 +623,9 @@ pub fn to_substrait_groupings( )), GroupingSet::GroupingSets(sets) => Ok(sets .iter() - .map(|set| parse_flat_grouping_exprs(set, schema, extension_info)) + .map(|set| { + parse_flat_grouping_exprs(ctx, set, schema, extension_info) + }) .collect::>>()?), GroupingSet::Rollup(set) => { let mut sets: Vec> = vec![vec![]]; @@ -618,17 +635,21 @@ pub fn to_substrait_groupings( Ok(sets .iter() .rev() - .map(|set| parse_flat_grouping_exprs(set, schema, extension_info)) + .map(|set| { + parse_flat_grouping_exprs(ctx, set, schema, extension_info) + }) .collect::>>()?) } }, _ => Ok(vec![parse_flat_grouping_exprs( + ctx, exprs, schema, extension_info, )?]), }, _ => Ok(vec![parse_flat_grouping_exprs( + ctx, exprs, schema, extension_info, @@ -638,6 +659,7 @@ pub fn to_substrait_groupings( #[allow(deprecated)] pub fn to_substrait_agg_measure( + ctx: &SessionContext, expr: &Expr, schema: &DFSchemaRef, extension_info: &mut ( @@ -650,13 +672,13 @@ pub fn to_substrait_agg_measure( match func_def { AggregateFunctionDefinition::BuiltIn (fun) => { let sorts = if let Some(order_by) = order_by { - order_by.iter().map(|expr| to_substrait_sort_field(expr, schema, extension_info)).collect::>>()? + order_by.iter().map(|expr| to_substrait_sort_field(ctx, expr, schema, extension_info)).collect::>>()? } else { vec![] }; let mut arguments: Vec = vec![]; for arg in args { - arguments.push(FunctionArgument { arg_type: Some(ArgType::Value(to_substrait_rex(arg, schema, 0, extension_info)?)) }); + arguments.push(FunctionArgument { arg_type: Some(ArgType::Value(to_substrait_rex(ctx, arg, schema, 0, extension_info)?)) }); } let function_anchor = _register_function(fun.to_string(), extension_info); Ok(Measure { @@ -674,20 +696,20 @@ pub fn to_substrait_agg_measure( options: vec![], }), filter: match filter { - Some(f) => Some(to_substrait_rex(f, schema, 0, extension_info)?), + Some(f) => Some(to_substrait_rex(ctx, f, schema, 0, extension_info)?), None => None } }) } AggregateFunctionDefinition::UDF(fun) => { let sorts = if let Some(order_by) = order_by { - order_by.iter().map(|expr| to_substrait_sort_field(expr, schema, extension_info)).collect::>>()? + order_by.iter().map(|expr| to_substrait_sort_field(ctx, expr, schema, extension_info)).collect::>>()? } else { vec![] }; let mut arguments: Vec = vec![]; for arg in args { - arguments.push(FunctionArgument { arg_type: Some(ArgType::Value(to_substrait_rex(arg, schema, 0, extension_info)?)) }); + arguments.push(FunctionArgument { arg_type: Some(ArgType::Value(to_substrait_rex(ctx, arg, schema, 0, extension_info)?)) }); } let function_anchor = _register_function(fun.name().to_string(), extension_info); Ok(Measure { @@ -702,7 +724,7 @@ pub fn to_substrait_agg_measure( options: vec![], }), filter: match filter { - Some(f) => Some(to_substrait_rex(f, schema, 0, extension_info)?), + Some(f) => Some(to_substrait_rex(ctx, f, schema, 0, extension_info)?), None => None } }) @@ -714,7 +736,7 @@ pub fn to_substrait_agg_measure( } Expr::Alias(Alias{expr,..})=> { - to_substrait_agg_measure(expr, schema, extension_info) + to_substrait_agg_measure(ctx, expr, schema, extension_info) } _ => internal_err!( "Expression must be compatible with aggregation. Unsupported expression: {:?}. ExpressionType: {:?}", @@ -726,6 +748,7 @@ pub fn to_substrait_agg_measure( /// Converts sort expression to corresponding substrait `SortField` fn to_substrait_sort_field( + ctx: &SessionContext, expr: &Expr, schema: &DFSchemaRef, extension_info: &mut ( @@ -743,6 +766,7 @@ fn to_substrait_sort_field( }; Ok(SortField { expr: Some(to_substrait_rex( + ctx, sort.expr.deref(), schema, 0, @@ -851,6 +875,7 @@ pub fn make_binary_op_scalar_func( /// * `extension_info` - Substrait extension info. Contains registered function information #[allow(deprecated)] pub fn to_substrait_rex( + ctx: &SessionContext, expr: &Expr, schema: &DFSchemaRef, col_ref_offset: usize, @@ -867,10 +892,10 @@ pub fn to_substrait_rex( }) => { let substrait_list = list .iter() - .map(|x| to_substrait_rex(x, schema, col_ref_offset, extension_info)) + .map(|x| to_substrait_rex(ctx, x, schema, col_ref_offset, extension_info)) .collect::>>()?; let substrait_expr = - to_substrait_rex(expr, schema, col_ref_offset, extension_info)?; + to_substrait_rex(ctx, expr, schema, col_ref_offset, extension_info)?; let substrait_or_list = Expression { rex_type: Some(RexType::SingularOrList(Box::new(SingularOrList { @@ -903,6 +928,7 @@ pub fn to_substrait_rex( for arg in &fun.args { arguments.push(FunctionArgument { arg_type: Some(ArgType::Value(to_substrait_rex( + ctx, arg, schema, col_ref_offset, @@ -937,11 +963,11 @@ pub fn to_substrait_rex( if *negated { // `expr NOT BETWEEN low AND high` can be translated into (expr < low OR high < expr) let substrait_expr = - to_substrait_rex(expr, schema, col_ref_offset, extension_info)?; + to_substrait_rex(ctx, expr, schema, col_ref_offset, extension_info)?; let substrait_low = - to_substrait_rex(low, schema, col_ref_offset, extension_info)?; + to_substrait_rex(ctx, low, schema, col_ref_offset, extension_info)?; let substrait_high = - to_substrait_rex(high, schema, col_ref_offset, extension_info)?; + to_substrait_rex(ctx, high, schema, col_ref_offset, extension_info)?; let l_expr = make_binary_op_scalar_func( &substrait_expr, @@ -965,11 +991,11 @@ pub fn to_substrait_rex( } else { // `expr BETWEEN low AND high` can be translated into (low <= expr AND expr <= high) let substrait_expr = - to_substrait_rex(expr, schema, col_ref_offset, extension_info)?; + to_substrait_rex(ctx, expr, schema, col_ref_offset, extension_info)?; let substrait_low = - to_substrait_rex(low, schema, col_ref_offset, extension_info)?; + to_substrait_rex(ctx, low, schema, col_ref_offset, extension_info)?; let substrait_high = - to_substrait_rex(high, schema, col_ref_offset, extension_info)?; + to_substrait_rex(ctx, high, schema, col_ref_offset, extension_info)?; let l_expr = make_binary_op_scalar_func( &substrait_low, @@ -997,8 +1023,8 @@ pub fn to_substrait_rex( substrait_field_ref(index + col_ref_offset) } Expr::BinaryExpr(BinaryExpr { left, op, right }) => { - let l = to_substrait_rex(left, schema, col_ref_offset, extension_info)?; - let r = to_substrait_rex(right, schema, col_ref_offset, extension_info)?; + let l = to_substrait_rex(ctx, left, schema, col_ref_offset, extension_info)?; + let r = to_substrait_rex(ctx, right, schema, col_ref_offset, extension_info)?; Ok(make_binary_op_scalar_func(&l, &r, *op, extension_info)) } @@ -1013,6 +1039,7 @@ pub fn to_substrait_rex( // Base expression exists ifs.push(IfClause { r#if: Some(to_substrait_rex( + ctx, e, schema, col_ref_offset, @@ -1025,12 +1052,14 @@ pub fn to_substrait_rex( for (r#if, then) in when_then_expr { ifs.push(IfClause { r#if: Some(to_substrait_rex( + ctx, r#if, schema, col_ref_offset, extension_info, )?), then: Some(to_substrait_rex( + ctx, then, schema, col_ref_offset, @@ -1042,6 +1071,7 @@ pub fn to_substrait_rex( // Parse outer `else` let r#else: Option> = match else_expr { Some(e) => Some(Box::new(to_substrait_rex( + ctx, e, schema, col_ref_offset, @@ -1060,6 +1090,7 @@ pub fn to_substrait_rex( substrait::proto::expression::Cast { r#type: Some(to_substrait_type(data_type)?), input: Some(Box::new(to_substrait_rex( + ctx, expr, schema, col_ref_offset, @@ -1072,7 +1103,7 @@ pub fn to_substrait_rex( } Expr::Literal(value) => to_substrait_literal(value), Expr::Alias(Alias { expr, .. }) => { - to_substrait_rex(expr, schema, col_ref_offset, extension_info) + to_substrait_rex(ctx, expr, schema, col_ref_offset, extension_info) } Expr::WindowFunction(WindowFunction { fun, @@ -1088,6 +1119,7 @@ pub fn to_substrait_rex( for arg in args { arguments.push(FunctionArgument { arg_type: Some(ArgType::Value(to_substrait_rex( + ctx, arg, schema, col_ref_offset, @@ -1098,12 +1130,12 @@ pub fn to_substrait_rex( // partition by expressions let partition_by = partition_by .iter() - .map(|e| to_substrait_rex(e, schema, col_ref_offset, extension_info)) + .map(|e| to_substrait_rex(ctx, e, schema, col_ref_offset, extension_info)) .collect::>>()?; // order by expressions let order_by = order_by .iter() - .map(|e| substrait_sort_field(e, schema, extension_info)) + .map(|e| substrait_sort_field(ctx, e, schema, extension_info)) .collect::>>()?; // window frame let bounds = to_substrait_bounds(window_frame)?; @@ -1124,6 +1156,7 @@ pub fn to_substrait_rex( escape_char, case_insensitive, }) => make_substrait_like_expr( + ctx, *case_insensitive, *negated, expr, @@ -1133,7 +1166,50 @@ pub fn to_substrait_rex( col_ref_offset, extension_info, ), + Expr::InSubquery(InSubquery { + expr, + subquery, + negated, + }) => { + let substrait_expr = + to_substrait_rex(ctx, expr, schema, col_ref_offset, extension_info)?; + + let subquery_plan = + to_substrait_rel(subquery.subquery.as_ref(), ctx, extension_info)?; + + let substrait_subquery = Expression { + rex_type: Some(RexType::Subquery(Box::new(Subquery { + subquery_type: Some( + substrait::proto::expression::subquery::SubqueryType::InPredicate( + Box::new(InPredicate { + needles: (vec![substrait_expr]), + haystack: Some(subquery_plan), + }), + ), + ), + }))), + }; + if *negated { + let function_anchor = + _register_function("not".to_string(), extension_info); + + Ok(Expression { + rex_type: Some(RexType::ScalarFunction(ScalarFunction { + function_reference: function_anchor, + arguments: vec![FunctionArgument { + arg_type: Some(ArgType::Value(substrait_subquery)), + }], + output_type: None, + args: vec![], + options: vec![], + })), + }) + } else { + Ok(substrait_subquery) + } + } Expr::Not(arg) => to_substrait_unary_scalar_fn( + ctx, "not", arg, schema, @@ -1141,6 +1217,7 @@ pub fn to_substrait_rex( extension_info, ), Expr::IsNull(arg) => to_substrait_unary_scalar_fn( + ctx, "is_null", arg, schema, @@ -1148,6 +1225,7 @@ pub fn to_substrait_rex( extension_info, ), Expr::IsNotNull(arg) => to_substrait_unary_scalar_fn( + ctx, "is_not_null", arg, schema, @@ -1155,6 +1233,7 @@ pub fn to_substrait_rex( extension_info, ), Expr::IsTrue(arg) => to_substrait_unary_scalar_fn( + ctx, "is_true", arg, schema, @@ -1162,6 +1241,7 @@ pub fn to_substrait_rex( extension_info, ), Expr::IsFalse(arg) => to_substrait_unary_scalar_fn( + ctx, "is_false", arg, schema, @@ -1169,6 +1249,7 @@ pub fn to_substrait_rex( extension_info, ), Expr::IsUnknown(arg) => to_substrait_unary_scalar_fn( + ctx, "is_unknown", arg, schema, @@ -1176,6 +1257,7 @@ pub fn to_substrait_rex( extension_info, ), Expr::IsNotTrue(arg) => to_substrait_unary_scalar_fn( + ctx, "is_not_true", arg, schema, @@ -1183,6 +1265,7 @@ pub fn to_substrait_rex( extension_info, ), Expr::IsNotFalse(arg) => to_substrait_unary_scalar_fn( + ctx, "is_not_false", arg, schema, @@ -1190,6 +1273,7 @@ pub fn to_substrait_rex( extension_info, ), Expr::IsNotUnknown(arg) => to_substrait_unary_scalar_fn( + ctx, "is_not_unknown", arg, schema, @@ -1197,6 +1281,7 @@ pub fn to_substrait_rex( extension_info, ), Expr::Negative(arg) => to_substrait_unary_scalar_fn( + ctx, "negative", arg, schema, @@ -1421,6 +1506,7 @@ fn make_substrait_window_function( #[allow(deprecated)] #[allow(clippy::too_many_arguments)] fn make_substrait_like_expr( + ctx: &SessionContext, ignore_case: bool, negated: bool, expr: &Expr, @@ -1438,8 +1524,8 @@ fn make_substrait_like_expr( } else { _register_function("like".to_string(), extension_info) }; - let expr = to_substrait_rex(expr, schema, col_ref_offset, extension_info)?; - let pattern = to_substrait_rex(pattern, schema, col_ref_offset, extension_info)?; + let expr = to_substrait_rex(ctx, expr, schema, col_ref_offset, extension_info)?; + let pattern = to_substrait_rex(ctx, pattern, schema, col_ref_offset, extension_info)?; let escape_char = to_substrait_literal(&ScalarValue::Utf8(escape_char.map(|c| c.to_string())))?; let arguments = vec![ @@ -1669,6 +1755,7 @@ fn to_substrait_literal(value: &ScalarValue) -> Result { /// Util to generate substrait [RexType::ScalarFunction] with one argument fn to_substrait_unary_scalar_fn( + ctx: &SessionContext, fn_name: &str, arg: &Expr, schema: &DFSchemaRef, @@ -1679,7 +1766,8 @@ fn to_substrait_unary_scalar_fn( ), ) -> Result { let function_anchor = _register_function(fn_name.to_string(), extension_info); - let substrait_expr = to_substrait_rex(arg, schema, col_ref_offset, extension_info)?; + let substrait_expr = + to_substrait_rex(ctx, arg, schema, col_ref_offset, extension_info)?; Ok(Expression { rex_type: Some(RexType::ScalarFunction(ScalarFunction { @@ -1880,6 +1968,7 @@ fn try_to_substrait_field_reference( } fn substrait_sort_field( + ctx: &SessionContext, expr: &Expr, schema: &DFSchemaRef, extension_info: &mut ( @@ -1893,7 +1982,7 @@ fn substrait_sort_field( asc, nulls_first, }) => { - let e = to_substrait_rex(expr, schema, 0, extension_info)?; + let e = to_substrait_rex(ctx, expr, schema, 0, extension_info)?; let d = match (asc, nulls_first) { (true, true) => SortDirection::AscNullsFirst, (true, false) => SortDirection::AscNullsLast, diff --git a/datafusion/substrait/tests/cases/roundtrip_logical_plan.rs b/datafusion/substrait/tests/cases/roundtrip_logical_plan.rs index 47eb5a8f73f5..d7327caee43d 100644 --- a/datafusion/substrait/tests/cases/roundtrip_logical_plan.rs +++ b/datafusion/substrait/tests/cases/roundtrip_logical_plan.rs @@ -394,6 +394,24 @@ async fn roundtrip_inlist_4() -> Result<()> { roundtrip("SELECT * FROM data WHERE f NOT IN ('a', 'b', 'c', 'd')").await } +#[tokio::test] +async fn roundtrip_inlist_5() -> Result<()> { + // on roundtrip there is an additional projection during TableScan which includes all column of the table, + // using assert_expected_plan here as a workaround + assert_expected_plan( + "SELECT a, f FROM data WHERE (f IN ('a', 'b', 'c') OR a in (SELECT data2.a FROM data2 WHERE f IN ('b', 'c', 'd')))", + "Filter: data.f = Utf8(\"a\") OR data.f = Utf8(\"b\") OR data.f = Utf8(\"c\") OR data.a IN ()\ + \n Subquery:\ + \n Projection: data2.a\ + \n Filter: data2.f IN ([Utf8(\"b\"), Utf8(\"c\"), Utf8(\"d\")])\ + \n TableScan: data2 projection=[a, b, c, d, e, f]\ + \n TableScan: data projection=[a, f], partial_filters=[data.f = Utf8(\"a\") OR data.f = Utf8(\"b\") OR data.f = Utf8(\"c\") OR data.a IN ()]\ + \n Subquery:\ + \n Projection: data2.a\ + \n Filter: data2.f IN ([Utf8(\"b\"), Utf8(\"c\"), Utf8(\"d\")])\ + \n TableScan: data2 projection=[a, b, c, d, e, f]").await +} + #[tokio::test] async fn roundtrip_cross_join() -> Result<()> { roundtrip("SELECT * FROM data CROSS JOIN data2").await From 448e413584226fc86e3d35a2f90725bcbdf390c9 Mon Sep 17 00:00:00 2001 From: Andrew Lamb Date: Wed, 20 Dec 2023 15:48:46 -0500 Subject: [PATCH 471/572] Minor: port last test from parquet.rs (#8587) --- datafusion/core/tests/sql/mod.rs | 1 - datafusion/core/tests/sql/parquet.rs | 91 ------------------- .../sqllogictest/test_files/parquet.slt | 17 ++++ 3 files changed, 17 insertions(+), 92 deletions(-) delete mode 100644 datafusion/core/tests/sql/parquet.rs diff --git a/datafusion/core/tests/sql/mod.rs b/datafusion/core/tests/sql/mod.rs index 94fc8015a78a..a3d5e32097c6 100644 --- a/datafusion/core/tests/sql/mod.rs +++ b/datafusion/core/tests/sql/mod.rs @@ -79,7 +79,6 @@ pub mod expr; pub mod group_by; pub mod joins; pub mod order; -pub mod parquet; pub mod parquet_schema; pub mod partitioned_csv; pub mod predicates; diff --git a/datafusion/core/tests/sql/parquet.rs b/datafusion/core/tests/sql/parquet.rs deleted file mode 100644 index f80a28f7e4f9..000000000000 --- a/datafusion/core/tests/sql/parquet.rs +++ /dev/null @@ -1,91 +0,0 @@ -// Licensed to the Apache Software Foundation (ASF) under one -// or more contributor license agreements. See the NOTICE file -// distributed with this work for additional information -// regarding copyright ownership. The ASF licenses this file -// to you under the Apache License, Version 2.0 (the -// "License"); you may not use this file except in compliance -// with the License. You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, -// software distributed under the License is distributed on an -// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -// KIND, either express or implied. See the License for the -// specific language governing permissions and limitations -// under the License. - -use datafusion_common::cast::{as_list_array, as_primitive_array, as_string_array}; - -use super::*; - -#[tokio::test] -#[ignore = "Test ignored, will be enabled as part of the nested Parquet reader"] -async fn parquet_list_columns() { - let ctx = SessionContext::new(); - let testdata = datafusion::test_util::parquet_test_data(); - ctx.register_parquet( - "list_columns", - &format!("{testdata}/list_columns.parquet"), - ParquetReadOptions::default(), - ) - .await - .unwrap(); - - let schema = Arc::new(Schema::new(vec![ - Field::new_list( - "int64_list", - Field::new("item", DataType::Int64, true), - true, - ), - Field::new_list("utf8_list", Field::new("item", DataType::Utf8, true), true), - ])); - - let sql = "SELECT int64_list, utf8_list FROM list_columns"; - let dataframe = ctx.sql(sql).await.unwrap(); - let results = dataframe.collect().await.unwrap(); - - // int64_list utf8_list - // 0 [1, 2, 3] [abc, efg, hij] - // 1 [None, 1] None - // 2 [4] [efg, None, hij, xyz] - - assert_eq!(1, results.len()); - let batch = &results[0]; - assert_eq!(3, batch.num_rows()); - assert_eq!(2, batch.num_columns()); - assert_eq!(schema, batch.schema()); - - let int_list_array = as_list_array(batch.column(0)).unwrap(); - let utf8_list_array = as_list_array(batch.column(1)).unwrap(); - - assert_eq!( - as_primitive_array::(&int_list_array.value(0)).unwrap(), - &PrimitiveArray::::from(vec![Some(1), Some(2), Some(3),]) - ); - - assert_eq!( - as_string_array(&utf8_list_array.value(0)).unwrap(), - &StringArray::from(vec![Some("abc"), Some("efg"), Some("hij"),]) - ); - - assert_eq!( - as_primitive_array::(&int_list_array.value(1)).unwrap(), - &PrimitiveArray::::from(vec![None, Some(1),]) - ); - - assert!(utf8_list_array.is_null(1)); - - assert_eq!( - as_primitive_array::(&int_list_array.value(2)).unwrap(), - &PrimitiveArray::::from(vec![Some(4),]) - ); - - let result = utf8_list_array.value(2); - let result = as_string_array(&result).unwrap(); - - assert_eq!(result.value(0), "efg"); - assert!(result.is_null(1)); - assert_eq!(result.value(2), "hij"); - assert_eq!(result.value(3), "xyz"); -} diff --git a/datafusion/sqllogictest/test_files/parquet.slt b/datafusion/sqllogictest/test_files/parquet.slt index bbe7f33e260c..6c3bd687700a 100644 --- a/datafusion/sqllogictest/test_files/parquet.slt +++ b/datafusion/sqllogictest/test_files/parquet.slt @@ -302,3 +302,20 @@ NULL # Clean up statement ok DROP TABLE single_nan; + + +statement ok +CREATE EXTERNAL TABLE list_columns +STORED AS PARQUET +WITH HEADER ROW +LOCATION '../../parquet-testing/data/list_columns.parquet'; + +query ?? +SELECT int64_list, utf8_list FROM list_columns +---- +[1, 2, 3] [abc, efg, hij] +[, 1] NULL +[4] [efg, , hij, xyz] + +statement ok +DROP TABLE list_columns; From 778779f7d72c45e7583100e5ff25c504cd48042b Mon Sep 17 00:00:00 2001 From: Andrew Lamb Date: Wed, 20 Dec 2023 15:49:36 -0500 Subject: [PATCH 472/572] Minor: consolidate map sqllogictest tests (#8550) * Minor: consolidate map sqllogictest tests * add plan --- datafusion/sqllogictest/src/test_context.rs | 2 +- .../sqllogictest/test_files/explain.slt | 4 ---- datafusion/sqllogictest/test_files/map.slt | 19 +++++++++++++++++++ 3 files changed, 20 insertions(+), 5 deletions(-) diff --git a/datafusion/sqllogictest/src/test_context.rs b/datafusion/sqllogictest/src/test_context.rs index 91093510afec..941dcb69d2f4 100644 --- a/datafusion/sqllogictest/src/test_context.rs +++ b/datafusion/sqllogictest/src/test_context.rs @@ -84,7 +84,7 @@ impl TestContext { info!("Registering table with many types"); register_table_with_many_types(test_ctx.session_ctx()).await; } - "explain.slt" => { + "map.slt" => { info!("Registering table with map"); register_table_with_map(test_ctx.session_ctx()).await; } diff --git a/datafusion/sqllogictest/test_files/explain.slt b/datafusion/sqllogictest/test_files/explain.slt index a51c3aed13ec..4583ef319b7f 100644 --- a/datafusion/sqllogictest/test_files/explain.slt +++ b/datafusion/sqllogictest/test_files/explain.slt @@ -379,7 +379,3 @@ Projection: List([[1, 2, 3], [4, 5, 6]]) AS make_array(make_array(Int64(1),Int64 physical_plan ProjectionExec: expr=[[[1, 2, 3], [4, 5, 6]] as make_array(make_array(Int64(1),Int64(2),Int64(3)),make_array(Int64(4),Int64(5),Int64(6)))] --PlaceholderRowExec - -# Testing explain on a table with a map filter, registered in test_context.rs. -statement ok -explain select * from table_with_map where int_field > 0 diff --git a/datafusion/sqllogictest/test_files/map.slt b/datafusion/sqllogictest/test_files/map.slt index c3d16fca904e..7863bf445499 100644 --- a/datafusion/sqllogictest/test_files/map.slt +++ b/datafusion/sqllogictest/test_files/map.slt @@ -44,3 +44,22 @@ DELETE 24 query T SELECT strings['not_found'] FROM data LIMIT 1; ---- + +statement ok +drop table data; + + +# Testing explain on a table with a map filter, registered in test_context.rs. +query TT +explain select * from table_with_map where int_field > 0; +---- +logical_plan +Filter: table_with_map.int_field > Int64(0) +--TableScan: table_with_map projection=[int_field, map_field] +physical_plan +CoalesceBatchesExec: target_batch_size=8192 +--FilterExec: int_field@0 > 0 +----MemoryExec: partitions=1, partition_sizes=[0] + +statement ok +drop table table_with_map; From 98a5a4eb1ea1277f5fe001e1c7602b37592452f1 Mon Sep 17 00:00:00 2001 From: Alex Huang Date: Wed, 20 Dec 2023 22:11:30 +0100 Subject: [PATCH 473/572] feat: support `LargeList` in `array_dims` (#8592) * support LargeList in array_dims * drop table * add argument check --- .../physical-expr/src/array_expressions.rs | 31 ++++++++++--- datafusion/sqllogictest/test_files/array.slt | 43 ++++++++++++++++++- 2 files changed, 67 insertions(+), 7 deletions(-) diff --git a/datafusion/physical-expr/src/array_expressions.rs b/datafusion/physical-expr/src/array_expressions.rs index 0a7631918804..bdab65cab9e3 100644 --- a/datafusion/physical-expr/src/array_expressions.rs +++ b/datafusion/physical-expr/src/array_expressions.rs @@ -1925,12 +1925,33 @@ pub fn array_length(args: &[ArrayRef]) -> Result { /// Array_dims SQL function pub fn array_dims(args: &[ArrayRef]) -> Result { - let list_array = as_list_array(&args[0])?; + if args.len() != 1 { + return exec_err!("array_dims needs one argument"); + } + + let data = match args[0].data_type() { + DataType::List(_) => { + let array = as_list_array(&args[0])?; + array + .iter() + .map(compute_array_dims) + .collect::>>()? + } + DataType::LargeList(_) => { + let array = as_large_list_array(&args[0])?; + array + .iter() + .map(compute_array_dims) + .collect::>>()? + } + _ => { + return exec_err!( + "array_dims does not support type '{:?}'", + args[0].data_type() + ); + } + }; - let data = list_array - .iter() - .map(compute_array_dims) - .collect::>>()?; let result = ListArray::from_iter_primitive::(data); Ok(Arc::new(result) as ArrayRef) diff --git a/datafusion/sqllogictest/test_files/array.slt b/datafusion/sqllogictest/test_files/array.slt index b38f73ecb8db..ca33f08de06d 100644 --- a/datafusion/sqllogictest/test_files/array.slt +++ b/datafusion/sqllogictest/test_files/array.slt @@ -67,6 +67,16 @@ AS VALUES (make_array(make_array(15, 16),make_array(NULL, 18)), make_array(16.6, 17.7, 18.8), NULL) ; +statement ok +CREATE TABLE large_arrays +AS + SELECT + arrow_cast(column1, 'LargeList(List(Int64))') AS column1, + arrow_cast(column2, 'LargeList(Float64)') AS column2, + arrow_cast(column3, 'LargeList(Utf8)') AS column3 + FROM arrays +; + statement ok CREATE TABLE slices AS VALUES @@ -2820,8 +2830,7 @@ NULL 10 ## array_dims (aliases: `list_dims`) # array dims error -# TODO this is a separate bug -query error Internal error: could not cast value to arrow_array::array::list_array::GenericListArray\. +query error Execution error: array_dims does not support type 'Int64' select array_dims(1); # array_dims scalar function @@ -2830,6 +2839,11 @@ select array_dims(make_array(1, 2, 3)), array_dims(make_array([1, 2], [3, 4])), ---- [3] [2, 2] [1, 1, 1, 2, 1] +query ??? +select array_dims(arrow_cast(make_array(1, 2, 3), 'LargeList(Int64)')), array_dims(arrow_cast(make_array([1, 2], [3, 4]), 'LargeList(List(Int64))')), array_dims(arrow_cast(make_array([[[[1], [2]]]]), 'LargeList(List(List(List(List(Int64)))))')); +---- +[3] [2, 2] [1, 1, 1, 2, 1] + # array_dims scalar function #2 query ?? select array_dims(array_repeat(array_repeat(array_repeat(2, 3), 2), 1)), array_dims(array_repeat(array_repeat(array_repeat(3, 4), 5), 2)); @@ -2842,12 +2856,22 @@ select array_dims(make_array()), array_dims(make_array(make_array())) ---- NULL [1, 0] +query ?? +select array_dims(arrow_cast(make_array(), 'LargeList(Null)')), array_dims(arrow_cast(make_array(make_array()), 'LargeList(List(Null))')) +---- +NULL [1, 0] + # list_dims scalar function #4 (function alias `array_dims`) query ??? select list_dims(make_array(1, 2, 3)), list_dims(make_array([1, 2], [3, 4])), list_dims(make_array([[[[1], [2]]]])); ---- [3] [2, 2] [1, 1, 1, 2, 1] +query ??? +select list_dims(arrow_cast(make_array(1, 2, 3), 'LargeList(Int64)')), list_dims(arrow_cast(make_array([1, 2], [3, 4]), 'LargeList(List(Int64))')), list_dims(arrow_cast(make_array([[[[1], [2]]]]), 'LargeList(List(List(List(List(Int64)))))')); +---- +[3] [2, 2] [1, 1, 1, 2, 1] + # array_dims with columns query ??? select array_dims(column1), array_dims(column2), array_dims(column3) from arrays; @@ -2860,6 +2884,18 @@ NULL [3] [4] [2, 2] NULL [1] [2, 2] [3] NULL +query ??? +select array_dims(column1), array_dims(column2), array_dims(column3) from large_arrays; +---- +[2, 2] [3] [5] +[2, 2] [3] [5] +[2, 2] [3] [5] +[2, 2] [3] [3] +NULL [3] [4] +[2, 2] NULL [1] +[2, 2] [3] NULL + + ## array_ndims (aliases: `list_ndims`) # array_ndims scalar function #1 @@ -3768,6 +3804,9 @@ drop table nested_arrays; statement ok drop table arrays; +statement ok +drop table large_arrays; + statement ok drop table slices; From bc013fc98a6c3c86cff8fe22de688cdd250b8674 Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Wed, 20 Dec 2023 15:49:42 -0700 Subject: [PATCH 474/572] Fix regression in regenerating protobuf source (#8603) * Fix regression in regenerating protobuf source * update serde code --- datafusion/proto/proto/datafusion.proto | 2 +- datafusion/proto/src/generated/pbjson.rs | 10 +++++----- datafusion/proto/src/generated/prost.rs | 4 ++-- datafusion/proto/src/logical_plan/from_proto.rs | 6 +++++- datafusion/proto/src/logical_plan/to_proto.rs | 2 +- 5 files changed, 14 insertions(+), 10 deletions(-) diff --git a/datafusion/proto/proto/datafusion.proto b/datafusion/proto/proto/datafusion.proto index bd8053c817e7..76fe449d2fa3 100644 --- a/datafusion/proto/proto/datafusion.proto +++ b/datafusion/proto/proto/datafusion.proto @@ -409,7 +409,7 @@ message LogicalExprNode { } message Wildcard { - optional string qualifier = 1; + string qualifier = 1; } message PlaceholderNode { diff --git a/datafusion/proto/src/generated/pbjson.rs b/datafusion/proto/src/generated/pbjson.rs index 88310be0318a..0671757ad427 100644 --- a/datafusion/proto/src/generated/pbjson.rs +++ b/datafusion/proto/src/generated/pbjson.rs @@ -25797,12 +25797,12 @@ impl serde::Serialize for Wildcard { { use serde::ser::SerializeStruct; let mut len = 0; - if self.qualifier.is_some() { + if !self.qualifier.is_empty() { len += 1; } let mut struct_ser = serializer.serialize_struct("datafusion.Wildcard", len)?; - if let Some(v) = self.qualifier.as_ref() { - struct_ser.serialize_field("qualifier", v)?; + if !self.qualifier.is_empty() { + struct_ser.serialize_field("qualifier", &self.qualifier)?; } struct_ser.end() } @@ -25868,12 +25868,12 @@ impl<'de> serde::Deserialize<'de> for Wildcard { if qualifier__.is_some() { return Err(serde::de::Error::duplicate_field("qualifier")); } - qualifier__ = map_.next_value()?; + qualifier__ = Some(map_.next_value()?); } } } Ok(Wildcard { - qualifier: qualifier__, + qualifier: qualifier__.unwrap_or_default(), }) } } diff --git a/datafusion/proto/src/generated/prost.rs b/datafusion/proto/src/generated/prost.rs index 3dfd3938615f..771bd715d3c5 100644 --- a/datafusion/proto/src/generated/prost.rs +++ b/datafusion/proto/src/generated/prost.rs @@ -636,8 +636,8 @@ pub mod logical_expr_node { #[allow(clippy::derive_partial_eq_without_eq)] #[derive(Clone, PartialEq, ::prost::Message)] pub struct Wildcard { - #[prost(string, optional, tag = "1")] - pub qualifier: ::core::option::Option<::prost::alloc::string::String>, + #[prost(string, tag = "1")] + pub qualifier: ::prost::alloc::string::String, } #[allow(clippy::derive_partial_eq_without_eq)] #[derive(Clone, PartialEq, ::prost::Message)] diff --git a/datafusion/proto/src/logical_plan/from_proto.rs b/datafusion/proto/src/logical_plan/from_proto.rs index 193e0947d6d9..854bfda9a861 100644 --- a/datafusion/proto/src/logical_plan/from_proto.rs +++ b/datafusion/proto/src/logical_plan/from_proto.rs @@ -1338,7 +1338,11 @@ pub fn parse_expr( in_list.negated, ))), ExprType::Wildcard(protobuf::Wildcard { qualifier }) => Ok(Expr::Wildcard { - qualifier: qualifier.clone(), + qualifier: if qualifier.is_empty() { + None + } else { + Some(qualifier.clone()) + }, }), ExprType::ScalarFunction(expr) => { let scalar_function = protobuf::ScalarFunction::try_from(expr.fun) diff --git a/datafusion/proto/src/logical_plan/to_proto.rs b/datafusion/proto/src/logical_plan/to_proto.rs index 2997d147424d..b9987ff6c727 100644 --- a/datafusion/proto/src/logical_plan/to_proto.rs +++ b/datafusion/proto/src/logical_plan/to_proto.rs @@ -1000,7 +1000,7 @@ impl TryFrom<&Expr> for protobuf::LogicalExprNode { } Expr::Wildcard { qualifier } => Self { expr_type: Some(ExprType::Wildcard(protobuf::Wildcard { - qualifier: qualifier.clone(), + qualifier: qualifier.clone().unwrap_or("".to_string()), })), }, Expr::ScalarSubquery(_) From 96c5b8afcda12f95ce6852102c5387021f907ca6 Mon Sep 17 00:00:00 2001 From: Devin D'Angelo Date: Thu, 21 Dec 2023 08:27:12 -0500 Subject: [PATCH 475/572] Remove unbounded_input from FileSinkOptions (#8605) * regen protoc * remove proto flag --- .../file_format/write/orchestration.rs | 17 ++--------------- .../core/src/datasource/listing/table.rs | 9 +-------- .../core/src/datasource/physical_plan/mod.rs | 18 ------------------ datafusion/core/src/physical_planner.rs | 1 - datafusion/proto/proto/datafusion.proto | 5 ++--- datafusion/proto/src/generated/pbjson.rs | 18 ------------------ datafusion/proto/src/generated/prost.rs | 4 +--- .../proto/src/physical_plan/from_proto.rs | 1 - datafusion/proto/src/physical_plan/to_proto.rs | 1 - .../tests/cases/roundtrip_physical_plan.rs | 1 - 10 files changed, 6 insertions(+), 69 deletions(-) diff --git a/datafusion/core/src/datasource/file_format/write/orchestration.rs b/datafusion/core/src/datasource/file_format/write/orchestration.rs index 2ae6b70ed1c5..120e27ecf669 100644 --- a/datafusion/core/src/datasource/file_format/write/orchestration.rs +++ b/datafusion/core/src/datasource/file_format/write/orchestration.rs @@ -52,7 +52,6 @@ pub(crate) async fn serialize_rb_stream_to_object_store( mut data_rx: Receiver, mut serializer: Box, mut writer: AbortableWrite>, - unbounded_input: bool, ) -> std::result::Result<(WriterType, u64), (WriterType, DataFusionError)> { let (tx, mut rx) = mpsc::channel::>>(100); @@ -71,9 +70,6 @@ pub(crate) async fn serialize_rb_stream_to_object_store( "Unknown error writing to object store".into(), ) })?; - if unbounded_input { - tokio::task::yield_now().await; - } } Err(_) => { return Err(DataFusionError::Internal( @@ -140,7 +136,6 @@ type FileWriteBundle = (Receiver, SerializerType, WriterType); pub(crate) async fn stateless_serialize_and_write_files( mut rx: Receiver, tx: tokio::sync::oneshot::Sender, - unbounded_input: bool, ) -> Result<()> { let mut row_count = 0; // tracks if any writers encountered an error triggering the need to abort @@ -153,13 +148,7 @@ pub(crate) async fn stateless_serialize_and_write_files( let mut join_set = JoinSet::new(); while let Some((data_rx, serializer, writer)) = rx.recv().await { join_set.spawn(async move { - serialize_rb_stream_to_object_store( - data_rx, - serializer, - writer, - unbounded_input, - ) - .await + serialize_rb_stream_to_object_store(data_rx, serializer, writer).await }); } let mut finished_writers = Vec::new(); @@ -241,7 +230,6 @@ pub(crate) async fn stateless_multipart_put( let single_file_output = config.single_file_output; let base_output_path = &config.table_paths[0]; - let unbounded_input = config.unbounded_input; let part_cols = if !config.table_partition_cols.is_empty() { Some(config.table_partition_cols.clone()) } else { @@ -266,8 +254,7 @@ pub(crate) async fn stateless_multipart_put( let (tx_file_bundle, rx_file_bundle) = tokio::sync::mpsc::channel(rb_buffer_size / 2); let (tx_row_cnt, rx_row_cnt) = tokio::sync::oneshot::channel(); let write_coordinater_task = tokio::spawn(async move { - stateless_serialize_and_write_files(rx_file_bundle, tx_row_cnt, unbounded_input) - .await + stateless_serialize_and_write_files(rx_file_bundle, tx_row_cnt).await }); while let Some((location, rb_stream)) = file_stream_rx.recv().await { let serializer = get_serializer(); diff --git a/datafusion/core/src/datasource/listing/table.rs b/datafusion/core/src/datasource/listing/table.rs index 4c13d9d443ca..21d43dcd56db 100644 --- a/datafusion/core/src/datasource/listing/table.rs +++ b/datafusion/core/src/datasource/listing/table.rs @@ -38,7 +38,7 @@ use crate::datasource::{ }, get_statistics_with_limit, listing::ListingTableUrl, - physical_plan::{is_plan_streaming, FileScanConfig, FileSinkConfig}, + physical_plan::{FileScanConfig, FileSinkConfig}, TableProvider, TableType, }; use crate::{ @@ -790,13 +790,6 @@ impl TableProvider for ListingTable { file_groups, output_schema: self.schema(), table_partition_cols: self.options.table_partition_cols.clone(), - // A plan can produce finite number of rows even if it has unbounded sources, like LIMIT - // queries. Thus, we can check if the plan is streaming to ensure file sink input is - // unbounded. When `unbounded_input` flag is `true` for sink, we occasionally call `yield_now` - // to consume data at the input. When `unbounded_input` flag is `false` (e.g non-streaming data), - // all of the data at the input is sink after execution finishes. See discussion for rationale: - // https://github.com/apache/arrow-datafusion/pull/7610#issuecomment-1728979918 - unbounded_input: is_plan_streaming(&input)?, single_file_output: self.options.single_file, overwrite, file_type_writer_options, diff --git a/datafusion/core/src/datasource/physical_plan/mod.rs b/datafusion/core/src/datasource/physical_plan/mod.rs index 9d1c373aee7c..4a6ebeab09e1 100644 --- a/datafusion/core/src/datasource/physical_plan/mod.rs +++ b/datafusion/core/src/datasource/physical_plan/mod.rs @@ -69,7 +69,6 @@ use arrow::{ use datafusion_common::{file_options::FileTypeWriterOptions, plan_err}; use datafusion_physical_expr::expressions::Column; use datafusion_physical_expr::PhysicalSortExpr; -use datafusion_physical_plan::ExecutionPlan; use log::debug; use object_store::path::Path; @@ -93,8 +92,6 @@ pub struct FileSinkConfig { /// regardless of input partitioning. Otherwise, each table path is assumed to be a directory /// to which each output partition is written to its own output file. pub single_file_output: bool, - /// If input is unbounded, tokio tasks need to yield to not block execution forever - pub unbounded_input: bool, /// Controls whether existing data should be overwritten by this sink pub overwrite: bool, /// Contains settings specific to writing a given FileType, e.g. parquet max_row_group_size @@ -510,21 +507,6 @@ fn get_projected_output_ordering( all_orderings } -// Get output (un)boundedness information for the given `plan`. -pub(crate) fn is_plan_streaming(plan: &Arc) -> Result { - let result = if plan.children().is_empty() { - plan.unbounded_output(&[]) - } else { - let children_unbounded_output = plan - .children() - .iter() - .map(is_plan_streaming) - .collect::>>(); - plan.unbounded_output(&children_unbounded_output?) - }; - result -} - #[cfg(test)] mod tests { use arrow_array::cast::AsArray; diff --git a/datafusion/core/src/physical_planner.rs b/datafusion/core/src/physical_planner.rs index e5816eb49ebb..31d50be10f70 100644 --- a/datafusion/core/src/physical_planner.rs +++ b/datafusion/core/src/physical_planner.rs @@ -593,7 +593,6 @@ impl DefaultPhysicalPlanner { file_groups: vec![], output_schema: Arc::new(schema), table_partition_cols: vec![], - unbounded_input: false, single_file_output: *single_file_output, overwrite: false, file_type_writer_options diff --git a/datafusion/proto/proto/datafusion.proto b/datafusion/proto/proto/datafusion.proto index 76fe449d2fa3..cc802ee95710 100644 --- a/datafusion/proto/proto/datafusion.proto +++ b/datafusion/proto/proto/datafusion.proto @@ -1201,9 +1201,8 @@ message FileSinkConfig { Schema output_schema = 4; repeated PartitionColumn table_partition_cols = 5; bool single_file_output = 7; - bool unbounded_input = 8; - bool overwrite = 9; - FileTypeWriterOptions file_type_writer_options = 10; + bool overwrite = 8; + FileTypeWriterOptions file_type_writer_options = 9; } message JsonSink { diff --git a/datafusion/proto/src/generated/pbjson.rs b/datafusion/proto/src/generated/pbjson.rs index 0671757ad427..fb3a3ad91d06 100644 --- a/datafusion/proto/src/generated/pbjson.rs +++ b/datafusion/proto/src/generated/pbjson.rs @@ -7500,9 +7500,6 @@ impl serde::Serialize for FileSinkConfig { if self.single_file_output { len += 1; } - if self.unbounded_input { - len += 1; - } if self.overwrite { len += 1; } @@ -7528,9 +7525,6 @@ impl serde::Serialize for FileSinkConfig { if self.single_file_output { struct_ser.serialize_field("singleFileOutput", &self.single_file_output)?; } - if self.unbounded_input { - struct_ser.serialize_field("unboundedInput", &self.unbounded_input)?; - } if self.overwrite { struct_ser.serialize_field("overwrite", &self.overwrite)?; } @@ -7559,8 +7553,6 @@ impl<'de> serde::Deserialize<'de> for FileSinkConfig { "tablePartitionCols", "single_file_output", "singleFileOutput", - "unbounded_input", - "unboundedInput", "overwrite", "file_type_writer_options", "fileTypeWriterOptions", @@ -7574,7 +7566,6 @@ impl<'de> serde::Deserialize<'de> for FileSinkConfig { OutputSchema, TablePartitionCols, SingleFileOutput, - UnboundedInput, Overwrite, FileTypeWriterOptions, } @@ -7604,7 +7595,6 @@ impl<'de> serde::Deserialize<'de> for FileSinkConfig { "outputSchema" | "output_schema" => Ok(GeneratedField::OutputSchema), "tablePartitionCols" | "table_partition_cols" => Ok(GeneratedField::TablePartitionCols), "singleFileOutput" | "single_file_output" => Ok(GeneratedField::SingleFileOutput), - "unboundedInput" | "unbounded_input" => Ok(GeneratedField::UnboundedInput), "overwrite" => Ok(GeneratedField::Overwrite), "fileTypeWriterOptions" | "file_type_writer_options" => Ok(GeneratedField::FileTypeWriterOptions), _ => Err(serde::de::Error::unknown_field(value, FIELDS)), @@ -7632,7 +7622,6 @@ impl<'de> serde::Deserialize<'de> for FileSinkConfig { let mut output_schema__ = None; let mut table_partition_cols__ = None; let mut single_file_output__ = None; - let mut unbounded_input__ = None; let mut overwrite__ = None; let mut file_type_writer_options__ = None; while let Some(k) = map_.next_key()? { @@ -7673,12 +7662,6 @@ impl<'de> serde::Deserialize<'de> for FileSinkConfig { } single_file_output__ = Some(map_.next_value()?); } - GeneratedField::UnboundedInput => { - if unbounded_input__.is_some() { - return Err(serde::de::Error::duplicate_field("unboundedInput")); - } - unbounded_input__ = Some(map_.next_value()?); - } GeneratedField::Overwrite => { if overwrite__.is_some() { return Err(serde::de::Error::duplicate_field("overwrite")); @@ -7700,7 +7683,6 @@ impl<'de> serde::Deserialize<'de> for FileSinkConfig { output_schema: output_schema__, table_partition_cols: table_partition_cols__.unwrap_or_default(), single_file_output: single_file_output__.unwrap_or_default(), - unbounded_input: unbounded_input__.unwrap_or_default(), overwrite: overwrite__.unwrap_or_default(), file_type_writer_options: file_type_writer_options__, }) diff --git a/datafusion/proto/src/generated/prost.rs b/datafusion/proto/src/generated/prost.rs index 771bd715d3c5..9030e90a24c8 100644 --- a/datafusion/proto/src/generated/prost.rs +++ b/datafusion/proto/src/generated/prost.rs @@ -1635,10 +1635,8 @@ pub struct FileSinkConfig { #[prost(bool, tag = "7")] pub single_file_output: bool, #[prost(bool, tag = "8")] - pub unbounded_input: bool, - #[prost(bool, tag = "9")] pub overwrite: bool, - #[prost(message, optional, tag = "10")] + #[prost(message, optional, tag = "9")] pub file_type_writer_options: ::core::option::Option, } #[allow(clippy::derive_partial_eq_without_eq)] diff --git a/datafusion/proto/src/physical_plan/from_proto.rs b/datafusion/proto/src/physical_plan/from_proto.rs index 5c0ef615cacd..65f9f139a87b 100644 --- a/datafusion/proto/src/physical_plan/from_proto.rs +++ b/datafusion/proto/src/physical_plan/from_proto.rs @@ -739,7 +739,6 @@ impl TryFrom<&protobuf::FileSinkConfig> for FileSinkConfig { output_schema: Arc::new(convert_required!(conf.output_schema)?), table_partition_cols, single_file_output: conf.single_file_output, - unbounded_input: conf.unbounded_input, overwrite: conf.overwrite, file_type_writer_options: convert_required!(conf.file_type_writer_options)?, }) diff --git a/datafusion/proto/src/physical_plan/to_proto.rs b/datafusion/proto/src/physical_plan/to_proto.rs index ea00b726b9d6..e9cdb34cf1b9 100644 --- a/datafusion/proto/src/physical_plan/to_proto.rs +++ b/datafusion/proto/src/physical_plan/to_proto.rs @@ -846,7 +846,6 @@ impl TryFrom<&FileSinkConfig> for protobuf::FileSinkConfig { output_schema: Some(conf.output_schema.as_ref().try_into()?), table_partition_cols, single_file_output: conf.single_file_output, - unbounded_input: conf.unbounded_input, overwrite: conf.overwrite, file_type_writer_options: Some(file_type_writer_options.try_into()?), }) diff --git a/datafusion/proto/tests/cases/roundtrip_physical_plan.rs b/datafusion/proto/tests/cases/roundtrip_physical_plan.rs index 9a9827f2a090..2eb04ab6cbab 100644 --- a/datafusion/proto/tests/cases/roundtrip_physical_plan.rs +++ b/datafusion/proto/tests/cases/roundtrip_physical_plan.rs @@ -733,7 +733,6 @@ fn roundtrip_json_sink() -> Result<()> { output_schema: schema.clone(), table_partition_cols: vec![("plan_type".to_string(), DataType::Utf8)], single_file_output: true, - unbounded_input: false, overwrite: true, file_type_writer_options: FileTypeWriterOptions::JSON(JsonWriterOptions::new( CompressionTypeVariant::UNCOMPRESSED, From df806bd314df9c2a8087fe1422337bce25dc8614 Mon Sep 17 00:00:00 2001 From: comphead Date: Thu, 21 Dec 2023 09:41:51 -0800 Subject: [PATCH 476/572] Add `arrow_err!` macros, optional backtrace to ArrowError (#8586) * Introducing `arrow_err!` macros --- datafusion-cli/Cargo.lock | 80 ++++++++--------- datafusion-cli/Cargo.toml | 2 +- datafusion/common/src/error.rs | 85 +++++++++++++------ datafusion/common/src/scalar.rs | 9 +- datafusion/common/src/utils.rs | 10 +-- .../avro_to_arrow/arrow_array_reader.rs | 5 +- .../src/datasource/listing_table_factory.rs | 4 +- datafusion/core/src/datasource/memory.rs | 2 +- .../physical_plan/parquet/row_filter.rs | 4 +- .../tests/user_defined/user_defined_plan.rs | 3 +- .../simplify_expressions/expr_simplifier.rs | 42 +++++---- .../physical-expr/src/aggregate/first_last.rs | 4 +- .../aggregate/groups_accumulator/adapter.rs | 7 +- .../physical-expr/src/expressions/binary.rs | 15 ++-- .../physical-expr/src/regex_expressions.rs | 6 +- .../physical-expr/src/window/lead_lag.rs | 8 +- .../src/joins/stream_join_utils.rs | 6 +- datafusion/physical-plan/src/joins/utils.rs | 14 ++- .../physical-plan/src/repartition/mod.rs | 9 +- .../src/windows/bounded_window_agg_exec.rs | 4 +- .../proto/src/logical_plan/from_proto.rs | 7 +- datafusion/sqllogictest/test_files/math.slt | 37 ++++---- 22 files changed, 191 insertions(+), 172 deletions(-) diff --git a/datafusion-cli/Cargo.lock b/datafusion-cli/Cargo.lock index 19ad6709362d..ac05ddf10a73 100644 --- a/datafusion-cli/Cargo.lock +++ b/datafusion-cli/Cargo.lock @@ -384,7 +384,7 @@ checksum = "a66537f1bb974b254c98ed142ff995236e81b9d0fe4db0575f46612cb15eb0f9" dependencies = [ "proc-macro2", "quote", - "syn 2.0.40", + "syn 2.0.41", ] [[package]] @@ -1069,12 +1069,12 @@ dependencies = [ [[package]] name = "ctor" -version = "0.2.5" +version = "0.2.6" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "37e366bff8cd32dd8754b0991fb66b279dc48f598c3a18914852a6673deef583" +checksum = "30d2b3721e861707777e3195b0158f950ae6dc4a27e4d02ff9f67e3eb3de199e" dependencies = [ "quote", - "syn 2.0.40", + "syn 2.0.41", ] [[package]] @@ -1576,7 +1576,7 @@ checksum = "53b153fd91e4b0147f4aced87be237c98248656bb01050b96bf3ee89220a8ddb" dependencies = [ "proc-macro2", "quote", - "syn 2.0.40", + "syn 2.0.41", ] [[package]] @@ -1781,9 +1781,9 @@ checksum = "9a3a5bfb195931eeb336b2a7b4d761daec841b97f947d34394601737a7bba5e4" [[package]] name = "hyper" -version = "0.14.27" +version = "0.14.28" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "ffb1cfd654a8219eaef89881fdb3bb3b1cdc5fa75ded05d6933b2b382e395468" +checksum = "bf96e135eb83a2a8ddf766e426a841d8ddd7449d5f00d34ea02b41d2f19eef80" dependencies = [ "bytes", "futures-channel", @@ -1796,7 +1796,7 @@ dependencies = [ "httpdate", "itoa", "pin-project-lite", - "socket2 0.4.10", + "socket2", "tokio", "tower-service", "tracing", @@ -2496,7 +2496,7 @@ checksum = "4359fd9c9171ec6e8c62926d6faaf553a8dc3f64e1507e76da7911b4f6a04405" dependencies = [ "proc-macro2", "quote", - "syn 2.0.40", + "syn 2.0.41", ] [[package]] @@ -2715,9 +2715,9 @@ checksum = "c08c74e62047bb2de4ff487b251e4a92e24f48745648451635cec7d591162d9f" [[package]] name = "reqwest" -version = "0.11.22" +version = "0.11.23" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "046cd98826c46c2ac8ddecae268eb5c2e58628688a5fc7a2643704a73faba95b" +checksum = "37b1ae8d9ac08420c66222fb9096fc5de435c3c48542bc5336c51892cffafb41" dependencies = [ "base64", "bytes", @@ -3020,7 +3020,7 @@ checksum = "43576ca501357b9b071ac53cdc7da8ef0cbd9493d8df094cd821777ea6e894d3" dependencies = [ "proc-macro2", "quote", - "syn 2.0.40", + "syn 2.0.41", ] [[package]] @@ -3106,16 +3106,6 @@ version = "1.1.1" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "1b6b67fb9a61334225b5b790716f609cd58395f895b3fe8b328786812a40bc3b" -[[package]] -name = "socket2" -version = "0.4.10" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "9f7916fc008ca5542385b89a3d3ce689953c143e9304a9bf8beec1de48994c0d" -dependencies = [ - "libc", - "winapi", -] - [[package]] name = "socket2" version = "0.5.5" @@ -3196,7 +3186,7 @@ dependencies = [ "proc-macro2", "quote", "rustversion", - "syn 2.0.40", + "syn 2.0.41", ] [[package]] @@ -3218,9 +3208,9 @@ dependencies = [ [[package]] name = "syn" -version = "2.0.40" +version = "2.0.41" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "13fa70a4ee923979ffb522cacce59d34421ebdea5625e1073c4326ef9d2dd42e" +checksum = "44c8b28c477cc3bf0e7966561e3460130e1255f7a1cf71931075f1c5e7a7e269" dependencies = [ "proc-macro2", "quote", @@ -3284,22 +3274,22 @@ checksum = "222a222a5bfe1bba4a77b45ec488a741b3cb8872e5e499451fd7d0129c9c7c3d" [[package]] name = "thiserror" -version = "1.0.50" +version = "1.0.51" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "f9a7210f5c9a7156bb50aa36aed4c95afb51df0df00713949448cf9e97d382d2" +checksum = "f11c217e1416d6f036b870f14e0413d480dbf28edbee1f877abaf0206af43bb7" dependencies = [ "thiserror-impl", ] [[package]] name = "thiserror-impl" -version = "1.0.50" +version = "1.0.51" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "266b2e40bc00e5a6c09c3584011e08b06f123c00362c92b975ba9843aaaa14b8" +checksum = "01742297787513b79cf8e29d1056ede1313e2420b7b3b15d0a768b4921f549df" dependencies = [ "proc-macro2", "quote", - "syn 2.0.40", + "syn 2.0.41", ] [[package]] @@ -3315,9 +3305,9 @@ dependencies = [ [[package]] name = "time" -version = "0.3.30" +version = "0.3.31" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "c4a34ab300f2dee6e562c10a046fc05e358b29f9bf92277f30c3c8d82275f6f5" +checksum = "f657ba42c3f86e7680e53c8cd3af8abbe56b5491790b46e22e19c0d57463583e" dependencies = [ "deranged", "powerfmt", @@ -3334,9 +3324,9 @@ checksum = "ef927ca75afb808a4d64dd374f00a2adf8d0fcff8e7b184af886c3c87ec4a3f3" [[package]] name = "time-macros" -version = "0.2.15" +version = "0.2.16" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "4ad70d68dba9e1f8aceda7aa6711965dfec1cac869f311a51bd08b3a2ccbce20" +checksum = "26197e33420244aeb70c3e8c78376ca46571bc4e701e4791c2cd9f57dcb3a43f" dependencies = [ "time-core", ] @@ -3378,7 +3368,7 @@ dependencies = [ "num_cpus", "parking_lot", "pin-project-lite", - "socket2 0.5.5", + "socket2", "tokio-macros", "windows-sys 0.48.0", ] @@ -3391,7 +3381,7 @@ checksum = "5b8a1e28f2deaa14e508979454cb3a223b10b938b45af148bc0986de36f1923b" dependencies = [ "proc-macro2", "quote", - "syn 2.0.40", + "syn 2.0.41", ] [[package]] @@ -3488,7 +3478,7 @@ checksum = "34704c8d6ebcbc939824180af020566b01a7c01f80641264eba0999f6c2b6be7" dependencies = [ "proc-macro2", "quote", - "syn 2.0.40", + "syn 2.0.41", ] [[package]] @@ -3533,7 +3523,7 @@ checksum = "f03ca4cb38206e2bef0700092660bb74d696f808514dae47fa1467cbfe26e96e" dependencies = [ "proc-macro2", "quote", - "syn 2.0.40", + "syn 2.0.41", ] [[package]] @@ -3687,7 +3677,7 @@ dependencies = [ "once_cell", "proc-macro2", "quote", - "syn 2.0.40", + "syn 2.0.41", "wasm-bindgen-shared", ] @@ -3721,7 +3711,7 @@ checksum = "f0eb82fcb7930ae6219a7ecfd55b217f5f0893484b7a13022ebb2b2bf20b5283" dependencies = [ "proc-macro2", "quote", - "syn 2.0.40", + "syn 2.0.41", "wasm-bindgen-backend", "wasm-bindgen-shared", ] @@ -3970,22 +3960,22 @@ dependencies = [ [[package]] name = "zerocopy" -version = "0.7.30" +version = "0.7.31" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "306dca4455518f1f31635ec308b6b3e4eb1b11758cefafc782827d0aa7acb5c7" +checksum = "1c4061bedbb353041c12f413700357bec76df2c7e2ca8e4df8bac24c6bf68e3d" dependencies = [ "zerocopy-derive", ] [[package]] name = "zerocopy-derive" -version = "0.7.30" +version = "0.7.31" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "be912bf68235a88fbefd1b73415cb218405958d1655b2ece9035a19920bdf6ba" +checksum = "b3c129550b3e6de3fd0ba67ba5c81818f9805e58b8d7fee80a3a59d2c9fc601a" dependencies = [ "proc-macro2", "quote", - "syn 2.0.40", + "syn 2.0.41", ] [[package]] diff --git a/datafusion-cli/Cargo.toml b/datafusion-cli/Cargo.toml index 1bf24808fb90..f57097683698 100644 --- a/datafusion-cli/Cargo.toml +++ b/datafusion-cli/Cargo.toml @@ -35,6 +35,7 @@ aws-config = "0.55" aws-credential-types = "0.55" clap = { version = "3", features = ["derive", "cargo"] } datafusion = { path = "../datafusion/core", version = "34.0.0", features = ["avro", "crypto_expressions", "encoding_expressions", "parquet", "regex_expressions", "unicode_expressions", "compression"] } +datafusion-common = { path = "../datafusion/common" } dirs = "4.0.0" env_logger = "0.9" mimalloc = { version = "0.1", default-features = false } @@ -49,6 +50,5 @@ url = "2.2" [dev-dependencies] assert_cmd = "2.0" ctor = "0.2.0" -datafusion-common = { path = "../datafusion/common" } predicates = "3.0" rstest = "0.17" diff --git a/datafusion/common/src/error.rs b/datafusion/common/src/error.rs index 56b52bd73f9b..515acc6d1c47 100644 --- a/datafusion/common/src/error.rs +++ b/datafusion/common/src/error.rs @@ -47,7 +47,8 @@ pub type GenericError = Box; #[derive(Debug)] pub enum DataFusionError { /// Error returned by arrow. - ArrowError(ArrowError), + /// 2nd argument is for optional backtrace + ArrowError(ArrowError, Option), /// Wraps an error from the Parquet crate #[cfg(feature = "parquet")] ParquetError(ParquetError), @@ -60,7 +61,8 @@ pub enum DataFusionError { /// Error associated to I/O operations and associated traits. IoError(io::Error), /// Error returned when SQL is syntactically incorrect. - SQL(ParserError), + /// 2nd argument is for optional backtrace + SQL(ParserError, Option), /// Error returned on a branch that we know it is possible /// but to which we still have no implementation for. /// Often, these errors are tracked in our issue tracker. @@ -223,14 +225,14 @@ impl From for DataFusionError { impl From for DataFusionError { fn from(e: ArrowError) -> Self { - DataFusionError::ArrowError(e) + DataFusionError::ArrowError(e, None) } } impl From for ArrowError { fn from(e: DataFusionError) -> Self { match e { - DataFusionError::ArrowError(e) => e, + DataFusionError::ArrowError(e, _) => e, DataFusionError::External(e) => ArrowError::ExternalError(e), other => ArrowError::ExternalError(Box::new(other)), } @@ -267,7 +269,7 @@ impl From for DataFusionError { impl From for DataFusionError { fn from(e: ParserError) -> Self { - DataFusionError::SQL(e) + DataFusionError::SQL(e, None) } } @@ -280,8 +282,9 @@ impl From for DataFusionError { impl Display for DataFusionError { fn fmt(&self, f: &mut Formatter) -> std::fmt::Result { match *self { - DataFusionError::ArrowError(ref desc) => { - write!(f, "Arrow error: {desc}") + DataFusionError::ArrowError(ref desc, ref backtrace) => { + let backtrace = backtrace.clone().unwrap_or("".to_owned()); + write!(f, "Arrow error: {desc}{backtrace}") } #[cfg(feature = "parquet")] DataFusionError::ParquetError(ref desc) => { @@ -294,8 +297,9 @@ impl Display for DataFusionError { DataFusionError::IoError(ref desc) => { write!(f, "IO error: {desc}") } - DataFusionError::SQL(ref desc) => { - write!(f, "SQL error: {desc:?}") + DataFusionError::SQL(ref desc, ref backtrace) => { + let backtrace = backtrace.clone().unwrap_or("".to_owned()); + write!(f, "SQL error: {desc:?}{backtrace}") } DataFusionError::Configuration(ref desc) => { write!(f, "Invalid or Unsupported Configuration: {desc}") @@ -339,7 +343,7 @@ impl Display for DataFusionError { impl Error for DataFusionError { fn source(&self) -> Option<&(dyn Error + 'static)> { match self { - DataFusionError::ArrowError(e) => Some(e), + DataFusionError::ArrowError(e, _) => Some(e), #[cfg(feature = "parquet")] DataFusionError::ParquetError(e) => Some(e), #[cfg(feature = "avro")] @@ -347,7 +351,7 @@ impl Error for DataFusionError { #[cfg(feature = "object_store")] DataFusionError::ObjectStore(e) => Some(e), DataFusionError::IoError(e) => Some(e), - DataFusionError::SQL(e) => Some(e), + DataFusionError::SQL(e, _) => Some(e), DataFusionError::NotImplemented(_) => None, DataFusionError::Internal(_) => None, DataFusionError::Configuration(_) => None, @@ -505,32 +509,57 @@ macro_rules! make_error { }; } -// Exposes a macro to create `DataFusionError::Plan` +// Exposes a macro to create `DataFusionError::Plan` with optional backtrace make_error!(plan_err, plan_datafusion_err, Plan); -// Exposes a macro to create `DataFusionError::Internal` +// Exposes a macro to create `DataFusionError::Internal` with optional backtrace make_error!(internal_err, internal_datafusion_err, Internal); -// Exposes a macro to create `DataFusionError::NotImplemented` +// Exposes a macro to create `DataFusionError::NotImplemented` with optional backtrace make_error!(not_impl_err, not_impl_datafusion_err, NotImplemented); -// Exposes a macro to create `DataFusionError::Execution` +// Exposes a macro to create `DataFusionError::Execution` with optional backtrace make_error!(exec_err, exec_datafusion_err, Execution); -// Exposes a macro to create `DataFusionError::Substrait` +// Exposes a macro to create `DataFusionError::Substrait` with optional backtrace make_error!(substrait_err, substrait_datafusion_err, Substrait); -// Exposes a macro to create `DataFusionError::SQL` +// Exposes a macro to create `DataFusionError::SQL` with optional backtrace +#[macro_export] +macro_rules! sql_datafusion_err { + ($ERR:expr) => { + DataFusionError::SQL($ERR, Some(DataFusionError::get_back_trace())) + }; +} + +// Exposes a macro to create `Err(DataFusionError::SQL)` with optional backtrace #[macro_export] macro_rules! sql_err { ($ERR:expr) => { - Err(DataFusionError::SQL($ERR)) + Err(datafusion_common::sql_datafusion_err!($ERR)) + }; +} + +// Exposes a macro to create `DataFusionError::ArrowError` with optional backtrace +#[macro_export] +macro_rules! arrow_datafusion_err { + ($ERR:expr) => { + DataFusionError::ArrowError($ERR, Some(DataFusionError::get_back_trace())) + }; +} + +// Exposes a macro to create `Err(DataFusionError::ArrowError)` with optional backtrace +#[macro_export] +macro_rules! arrow_err { + ($ERR:expr) => { + Err(datafusion_common::arrow_datafusion_err!($ERR)) }; } // To avoid compiler error when using macro in the same crate: // macros from the current crate cannot be referred to by absolute paths pub use exec_err as _exec_err; +pub use internal_datafusion_err as _internal_datafusion_err; pub use internal_err as _internal_err; pub use not_impl_err as _not_impl_err; pub use plan_err as _plan_err; @@ -600,9 +629,12 @@ mod test { ); do_root_test( - DataFusionError::ArrowError(ArrowError::ExternalError(Box::new( - DataFusionError::ResourcesExhausted("foo".to_string()), - ))), + DataFusionError::ArrowError( + ArrowError::ExternalError(Box::new(DataFusionError::ResourcesExhausted( + "foo".to_string(), + ))), + None, + ), DataFusionError::ResourcesExhausted("foo".to_string()), ); @@ -621,11 +653,12 @@ mod test { ); do_root_test( - DataFusionError::ArrowError(ArrowError::ExternalError(Box::new( - ArrowError::ExternalError(Box::new(DataFusionError::ResourcesExhausted( - "foo".to_string(), - ))), - ))), + DataFusionError::ArrowError( + ArrowError::ExternalError(Box::new(ArrowError::ExternalError(Box::new( + DataFusionError::ResourcesExhausted("foo".to_string()), + )))), + None, + ), DataFusionError::ResourcesExhausted("foo".to_string()), ); diff --git a/datafusion/common/src/scalar.rs b/datafusion/common/src/scalar.rs index d730fbf89b72..48878aa9bd99 100644 --- a/datafusion/common/src/scalar.rs +++ b/datafusion/common/src/scalar.rs @@ -24,6 +24,7 @@ use std::convert::{Infallible, TryInto}; use std::str::FromStr; use std::{convert::TryFrom, fmt, iter::repeat, sync::Arc}; +use crate::arrow_datafusion_err; use crate::cast::{ as_decimal128_array, as_decimal256_array, as_dictionary_array, as_fixed_size_binary_array, as_fixed_size_list_array, as_struct_array, @@ -1654,11 +1655,11 @@ impl ScalarValue { match value { Some(val) => Decimal128Array::from(vec![val; size]) .with_precision_and_scale(precision, scale) - .map_err(DataFusionError::ArrowError), + .map_err(|e| arrow_datafusion_err!(e)), None => { let mut builder = Decimal128Array::builder(size) .with_precision_and_scale(precision, scale) - .map_err(DataFusionError::ArrowError)?; + .map_err(|e| arrow_datafusion_err!(e))?; builder.append_nulls(size); Ok(builder.finish()) } @@ -1675,7 +1676,7 @@ impl ScalarValue { .take(size) .collect::() .with_precision_and_scale(precision, scale) - .map_err(DataFusionError::ArrowError) + .map_err(|e| arrow_datafusion_err!(e)) } /// Converts `Vec` where each element has type corresponding to @@ -1882,7 +1883,7 @@ impl ScalarValue { .take(size) .collect::>(); arrow::compute::concat(arrays.as_slice()) - .map_err(DataFusionError::ArrowError)? + .map_err(|e| arrow_datafusion_err!(e))? } ScalarValue::Date32(e) => { build_array_from_option!(Date32, Date32Array, e, size) diff --git a/datafusion/common/src/utils.rs b/datafusion/common/src/utils.rs index 2d38ca21829b..cfdef309a4ee 100644 --- a/datafusion/common/src/utils.rs +++ b/datafusion/common/src/utils.rs @@ -17,8 +17,8 @@ //! This module provides the bisect function, which implements binary search. -use crate::error::_internal_err; -use crate::{DataFusionError, Result, ScalarValue}; +use crate::error::{_internal_datafusion_err, _internal_err}; +use crate::{arrow_datafusion_err, DataFusionError, Result, ScalarValue}; use arrow::array::{ArrayRef, PrimitiveArray}; use arrow::buffer::OffsetBuffer; use arrow::compute; @@ -95,7 +95,7 @@ pub fn get_record_batch_at_indices( new_columns, &RecordBatchOptions::new().with_row_count(Some(indices.len())), ) - .map_err(DataFusionError::ArrowError) + .map_err(|e| arrow_datafusion_err!(e)) } /// This function compares two tuples depending on the given sort options. @@ -117,7 +117,7 @@ pub fn compare_rows( lhs.partial_cmp(rhs) } .ok_or_else(|| { - DataFusionError::Internal("Column array shouldn't be empty".to_string()) + _internal_datafusion_err!("Column array shouldn't be empty") })?, (true, true, _) => continue, }; @@ -291,7 +291,7 @@ pub fn get_arrayref_at_indices( indices, None, // None: no index check ) - .map_err(DataFusionError::ArrowError) + .map_err(|e| arrow_datafusion_err!(e)) }) .collect() } diff --git a/datafusion/core/src/datasource/avro_to_arrow/arrow_array_reader.rs b/datafusion/core/src/datasource/avro_to_arrow/arrow_array_reader.rs index 855a8d0dbf40..a16c1ae3333f 100644 --- a/datafusion/core/src/datasource/avro_to_arrow/arrow_array_reader.rs +++ b/datafusion/core/src/datasource/avro_to_arrow/arrow_array_reader.rs @@ -45,6 +45,7 @@ use arrow::array::{BinaryArray, FixedSizeBinaryArray, GenericListArray}; use arrow::datatypes::{Fields, SchemaRef}; use arrow::error::ArrowError::SchemaError; use arrow::error::Result as ArrowResult; +use datafusion_common::arrow_err; use num_traits::NumCast; use std::collections::BTreeMap; use std::io::Read; @@ -86,9 +87,9 @@ impl<'a, R: Read> AvroArrowArrayReader<'a, R> { } Ok(lookup) } - _ => Err(DataFusionError::ArrowError(SchemaError( + _ => arrow_err!(SchemaError( "expected avro schema to be a record".to_string(), - ))), + )), } } diff --git a/datafusion/core/src/datasource/listing_table_factory.rs b/datafusion/core/src/datasource/listing_table_factory.rs index 7c859ee988d5..68c97bbb7806 100644 --- a/datafusion/core/src/datasource/listing_table_factory.rs +++ b/datafusion/core/src/datasource/listing_table_factory.rs @@ -36,7 +36,7 @@ use crate::execution::context::SessionState; use arrow::datatypes::{DataType, SchemaRef}; use datafusion_common::file_options::{FileTypeWriterOptions, StatementOptions}; -use datafusion_common::{plan_err, DataFusionError, FileType}; +use datafusion_common::{arrow_datafusion_err, plan_err, DataFusionError, FileType}; use datafusion_expr::CreateExternalTable; use async_trait::async_trait; @@ -114,7 +114,7 @@ impl TableProviderFactory for ListingTableFactory { .map(|col| { schema .field_with_name(col) - .map_err(DataFusionError::ArrowError) + .map_err(|e| arrow_datafusion_err!(e)) }) .collect::>>()? .into_iter() diff --git a/datafusion/core/src/datasource/memory.rs b/datafusion/core/src/datasource/memory.rs index 7c044b29366d..7c61cc536860 100644 --- a/datafusion/core/src/datasource/memory.rs +++ b/datafusion/core/src/datasource/memory.rs @@ -423,7 +423,7 @@ mod tests { .scan(&session_ctx.state(), Some(&projection), &[], None) .await { - Err(DataFusionError::ArrowError(ArrowError::SchemaError(e))) => { + Err(DataFusionError::ArrowError(ArrowError::SchemaError(e), _)) => { assert_eq!( "\"project index 4 out of bounds, max field 3\"", format!("{e:?}") diff --git a/datafusion/core/src/datasource/physical_plan/parquet/row_filter.rs b/datafusion/core/src/datasource/physical_plan/parquet/row_filter.rs index 5fe0a0a13a73..151ab5f657b1 100644 --- a/datafusion/core/src/datasource/physical_plan/parquet/row_filter.rs +++ b/datafusion/core/src/datasource/physical_plan/parquet/row_filter.rs @@ -21,7 +21,7 @@ use arrow::error::{ArrowError, Result as ArrowResult}; use arrow::record_batch::RecordBatch; use datafusion_common::cast::as_boolean_array; use datafusion_common::tree_node::{RewriteRecursion, TreeNode, TreeNodeRewriter}; -use datafusion_common::{DataFusionError, Result, ScalarValue}; +use datafusion_common::{arrow_err, DataFusionError, Result, ScalarValue}; use datafusion_physical_expr::expressions::{Column, Literal}; use datafusion_physical_expr::utils::reassign_predicate_columns; use std::collections::BTreeSet; @@ -243,7 +243,7 @@ impl<'a> TreeNodeRewriter for FilterCandidateBuilder<'a> { } Err(e) => { // If the column is not in the table schema, should throw the error - Err(DataFusionError::ArrowError(e)) + arrow_err!(e) } }; } diff --git a/datafusion/core/tests/user_defined/user_defined_plan.rs b/datafusion/core/tests/user_defined/user_defined_plan.rs index d4a8842c0a7a..29708c4422ca 100644 --- a/datafusion/core/tests/user_defined/user_defined_plan.rs +++ b/datafusion/core/tests/user_defined/user_defined_plan.rs @@ -91,6 +91,7 @@ use datafusion::{ }; use async_trait::async_trait; +use datafusion_common::arrow_datafusion_err; use futures::{Stream, StreamExt}; /// Execute the specified sql and return the resulting record batches @@ -99,7 +100,7 @@ async fn exec_sql(ctx: &mut SessionContext, sql: &str) -> Result { let df = ctx.sql(sql).await?; let batches = df.collect().await?; pretty_format_batches(&batches) - .map_err(DataFusionError::ArrowError) + .map_err(|e| arrow_datafusion_err!(e)) .map(|d| d.to_string()) } diff --git a/datafusion/optimizer/src/simplify_expressions/expr_simplifier.rs b/datafusion/optimizer/src/simplify_expressions/expr_simplifier.rs index e2fbd5e927a1..5a300e2ff246 100644 --- a/datafusion/optimizer/src/simplify_expressions/expr_simplifier.rs +++ b/datafusion/optimizer/src/simplify_expressions/expr_simplifier.rs @@ -29,11 +29,11 @@ use crate::simplify_expressions::SimplifyInfo; use arrow::{ array::new_null_array, datatypes::{DataType, Field, Schema}, - error::ArrowError, record_batch::RecordBatch, }; use datafusion_common::{ cast::{as_large_list_array, as_list_array}, + plan_err, tree_node::{RewriteRecursion, TreeNode, TreeNodeRewriter}, }; use datafusion_common::{ @@ -792,7 +792,7 @@ impl<'a, S: SimplifyInfo> TreeNodeRewriter for Simplifier<'a, S> { op: Divide, right, }) if is_null(&right) => *right, - // A / 0 -> DivideByZero Error if A is not null and not floating + // A / 0 -> Divide by zero error if A is not null and not floating // (float / 0 -> inf | -inf | NAN) Expr::BinaryExpr(BinaryExpr { left, @@ -802,7 +802,7 @@ impl<'a, S: SimplifyInfo> TreeNodeRewriter for Simplifier<'a, S> { && !info.get_data_type(&left)?.is_floating() && is_zero(&right) => { - return Err(DataFusionError::ArrowError(ArrowError::DivideByZero)); + return plan_err!("Divide by zero"); } // @@ -832,7 +832,7 @@ impl<'a, S: SimplifyInfo> TreeNodeRewriter for Simplifier<'a, S> { { lit(0) } - // A % 0 --> DivideByZero Error (if A is not floating and not null) + // A % 0 --> Divide by zero Error (if A is not floating and not null) // A % 0 --> NAN (if A is floating and not null) Expr::BinaryExpr(BinaryExpr { left, @@ -843,9 +843,7 @@ impl<'a, S: SimplifyInfo> TreeNodeRewriter for Simplifier<'a, S> { DataType::Float32 => lit(f32::NAN), DataType::Float64 => lit(f64::NAN), _ => { - return Err(DataFusionError::ArrowError( - ArrowError::DivideByZero, - )); + return plan_err!("Divide by zero"); } } } @@ -1315,7 +1313,9 @@ mod tests { array::{ArrayRef, Int32Array}, datatypes::{DataType, Field, Schema}, }; - use datafusion_common::{assert_contains, cast::as_int32_array, DFField, ToDFSchema}; + use datafusion_common::{ + assert_contains, cast::as_int32_array, plan_datafusion_err, DFField, ToDFSchema, + }; use datafusion_expr::{interval_arithmetic::Interval, *}; use datafusion_physical_expr::{ execution_props::ExecutionProps, functions::make_scalar_function, @@ -1771,25 +1771,23 @@ mod tests { #[test] fn test_simplify_divide_zero_by_zero() { - // 0 / 0 -> DivideByZero + // 0 / 0 -> Divide by zero let expr = lit(0) / lit(0); let err = try_simplify(expr).unwrap_err(); - assert!( - matches!(err, DataFusionError::ArrowError(ArrowError::DivideByZero)), - "{err}" - ); + let _expected = plan_datafusion_err!("Divide by zero"); + + assert!(matches!(err, ref _expected), "{err}"); } #[test] - #[should_panic( - expected = "called `Result::unwrap()` on an `Err` value: ArrowError(DivideByZero)" - )] fn test_simplify_divide_by_zero() { // A / 0 -> DivideByZeroError let expr = col("c2_non_null") / lit(0); - - simplify(expr); + assert_eq!( + try_simplify(expr).unwrap_err().strip_backtrace(), + "Error during planning: Divide by zero" + ); } #[test] @@ -2209,12 +2207,12 @@ mod tests { } #[test] - #[should_panic( - expected = "called `Result::unwrap()` on an `Err` value: ArrowError(DivideByZero)" - )] fn test_simplify_modulo_by_zero_non_null() { let expr = col("c2_non_null") % lit(0); - simplify(expr); + assert_eq!( + try_simplify(expr).unwrap_err().strip_backtrace(), + "Error during planning: Divide by zero" + ); } #[test] diff --git a/datafusion/physical-expr/src/aggregate/first_last.rs b/datafusion/physical-expr/src/aggregate/first_last.rs index 5e2012bdbb67..c009881d8918 100644 --- a/datafusion/physical-expr/src/aggregate/first_last.rs +++ b/datafusion/physical-expr/src/aggregate/first_last.rs @@ -31,7 +31,7 @@ use arrow::compute::{self, lexsort_to_indices, SortColumn}; use arrow::datatypes::{DataType, Field}; use arrow_schema::SortOptions; use datafusion_common::utils::{compare_rows, get_arrayref_at_indices, get_row_at_idx}; -use datafusion_common::{DataFusionError, Result, ScalarValue}; +use datafusion_common::{arrow_datafusion_err, DataFusionError, Result, ScalarValue}; use datafusion_expr::Accumulator; /// FIRST_VALUE aggregate expression @@ -541,7 +541,7 @@ fn filter_states_according_to_is_set( ) -> Result> { states .iter() - .map(|state| compute::filter(state, flags).map_err(DataFusionError::ArrowError)) + .map(|state| compute::filter(state, flags).map_err(|e| arrow_datafusion_err!(e))) .collect::>>() } diff --git a/datafusion/physical-expr/src/aggregate/groups_accumulator/adapter.rs b/datafusion/physical-expr/src/aggregate/groups_accumulator/adapter.rs index cf980f4c3f16..c6fd17a69b39 100644 --- a/datafusion/physical-expr/src/aggregate/groups_accumulator/adapter.rs +++ b/datafusion/physical-expr/src/aggregate/groups_accumulator/adapter.rs @@ -25,7 +25,8 @@ use arrow::{ }; use arrow_array::{ArrayRef, BooleanArray, PrimitiveArray}; use datafusion_common::{ - utils::get_arrayref_at_indices, DataFusionError, Result, ScalarValue, + arrow_datafusion_err, utils::get_arrayref_at_indices, DataFusionError, Result, + ScalarValue, }; use datafusion_expr::Accumulator; @@ -372,7 +373,7 @@ fn get_filter_at_indices( ) }) .transpose() - .map_err(DataFusionError::ArrowError) + .map_err(|e| arrow_datafusion_err!(e)) } // Copied from physical-plan @@ -394,7 +395,7 @@ pub(crate) fn slice_and_maybe_filter( sliced_arrays .iter() .map(|array| { - compute::filter(array, filter_array).map_err(DataFusionError::ArrowError) + compute::filter(array, filter_array).map_err(|e| arrow_datafusion_err!(e)) }) .collect() } else { diff --git a/datafusion/physical-expr/src/expressions/binary.rs b/datafusion/physical-expr/src/expressions/binary.rs index 9c7fdd2e814b..c17081398cb8 100644 --- a/datafusion/physical-expr/src/expressions/binary.rs +++ b/datafusion/physical-expr/src/expressions/binary.rs @@ -629,8 +629,7 @@ mod tests { use arrow::datatypes::{ ArrowNumericType, Decimal128Type, Field, Int32Type, SchemaRef, }; - use arrow_schema::ArrowError; - use datafusion_common::Result; + use datafusion_common::{plan_datafusion_err, Result}; use datafusion_expr::type_coercion::binary::get_input_types; /// Performs a binary operation, applying any type coercion necessary @@ -3608,10 +3607,9 @@ mod tests { ) .unwrap_err(); - assert!( - matches!(err, DataFusionError::ArrowError(ArrowError::DivideByZero)), - "{err}" - ); + let _expected = plan_datafusion_err!("Divide by zero"); + + assert!(matches!(err, ref _expected), "{err}"); // decimal let schema = Arc::new(Schema::new(vec![ @@ -3633,10 +3631,7 @@ mod tests { ) .unwrap_err(); - assert!( - matches!(err, DataFusionError::ArrowError(ArrowError::DivideByZero)), - "{err}" - ); + assert!(matches!(err, ref _expected), "{err}"); Ok(()) } diff --git a/datafusion/physical-expr/src/regex_expressions.rs b/datafusion/physical-expr/src/regex_expressions.rs index 41cd01949595..7bafed072b61 100644 --- a/datafusion/physical-expr/src/regex_expressions.rs +++ b/datafusion/physical-expr/src/regex_expressions.rs @@ -26,7 +26,7 @@ use arrow::array::{ OffsetSizeTrait, }; use arrow::compute; -use datafusion_common::plan_err; +use datafusion_common::{arrow_datafusion_err, plan_err}; use datafusion_common::{ cast::as_generic_string_array, internal_err, DataFusionError, Result, }; @@ -58,7 +58,7 @@ pub fn regexp_match(args: &[ArrayRef]) -> Result { 2 => { let values = as_generic_string_array::(&args[0])?; let regex = as_generic_string_array::(&args[1])?; - compute::regexp_match(values, regex, None).map_err(DataFusionError::ArrowError) + compute::regexp_match(values, regex, None).map_err(|e| arrow_datafusion_err!(e)) } 3 => { let values = as_generic_string_array::(&args[0])?; @@ -69,7 +69,7 @@ pub fn regexp_match(args: &[ArrayRef]) -> Result { Some(f) if f.iter().any(|s| s == Some("g")) => { plan_err!("regexp_match() does not support the \"global\" option") }, - _ => compute::regexp_match(values, regex, flags).map_err(DataFusionError::ArrowError), + _ => compute::regexp_match(values, regex, flags).map_err(|e| arrow_datafusion_err!(e)), } } other => internal_err!( diff --git a/datafusion/physical-expr/src/window/lead_lag.rs b/datafusion/physical-expr/src/window/lead_lag.rs index d22660d41ebd..7ee736ce9caa 100644 --- a/datafusion/physical-expr/src/window/lead_lag.rs +++ b/datafusion/physical-expr/src/window/lead_lag.rs @@ -23,7 +23,7 @@ use crate::PhysicalExpr; use arrow::array::ArrayRef; use arrow::compute::cast; use arrow::datatypes::{DataType, Field}; -use datafusion_common::ScalarValue; +use datafusion_common::{arrow_datafusion_err, ScalarValue}; use datafusion_common::{internal_err, DataFusionError, Result}; use datafusion_expr::PartitionEvaluator; use std::any::Any; @@ -142,7 +142,7 @@ fn create_empty_array( .transpose()? .unwrap_or_else(|| new_null_array(data_type, size)); if array.data_type() != data_type { - cast(&array, data_type).map_err(DataFusionError::ArrowError) + cast(&array, data_type).map_err(|e| arrow_datafusion_err!(e)) } else { Ok(array) } @@ -172,10 +172,10 @@ fn shift_with_default_value( // Concatenate both arrays, add nulls after if shift > 0 else before if offset > 0 { concat(&[default_values.as_ref(), slice.as_ref()]) - .map_err(DataFusionError::ArrowError) + .map_err(|e| arrow_datafusion_err!(e)) } else { concat(&[slice.as_ref(), default_values.as_ref()]) - .map_err(DataFusionError::ArrowError) + .map_err(|e| arrow_datafusion_err!(e)) } } } diff --git a/datafusion/physical-plan/src/joins/stream_join_utils.rs b/datafusion/physical-plan/src/joins/stream_join_utils.rs index 64a976a1e39f..50b1618a35dd 100644 --- a/datafusion/physical-plan/src/joins/stream_join_utils.rs +++ b/datafusion/physical-plan/src/joins/stream_join_utils.rs @@ -33,7 +33,9 @@ use arrow_buffer::{ArrowNativeType, BooleanBufferBuilder}; use arrow_schema::{Schema, SchemaRef}; use async_trait::async_trait; use datafusion_common::tree_node::{Transformed, TreeNode}; -use datafusion_common::{DataFusionError, JoinSide, Result, ScalarValue}; +use datafusion_common::{ + arrow_datafusion_err, DataFusionError, JoinSide, Result, ScalarValue, +}; use datafusion_execution::SendableRecordBatchStream; use datafusion_expr::interval_arithmetic::Interval; use datafusion_physical_expr::expressions::Column; @@ -595,7 +597,7 @@ pub fn combine_two_batches( (Some(left_batch), Some(right_batch)) => { // If both batches are present, concatenate them: concat_batches(output_schema, &[left_batch, right_batch]) - .map_err(DataFusionError::ArrowError) + .map_err(|e| arrow_datafusion_err!(e)) .map(Some) } (None, None) => { diff --git a/datafusion/physical-plan/src/joins/utils.rs b/datafusion/physical-plan/src/joins/utils.rs index eae65ce9c26b..c902ba85f271 100644 --- a/datafusion/physical-plan/src/joins/utils.rs +++ b/datafusion/physical-plan/src/joins/utils.rs @@ -1370,7 +1370,7 @@ mod tests { use arrow::error::{ArrowError, Result as ArrowResult}; use arrow_schema::SortOptions; - use datafusion_common::ScalarValue; + use datafusion_common::{arrow_datafusion_err, arrow_err, ScalarValue}; fn check(left: &[Column], right: &[Column], on: &[(Column, Column)]) -> Result<()> { let left = left @@ -1406,9 +1406,7 @@ mod tests { #[tokio::test] async fn check_error_nesting() { let once_fut = OnceFut::<()>::new(async { - Err(DataFusionError::ArrowError(ArrowError::CsvError( - "some error".to_string(), - ))) + arrow_err!(ArrowError::CsvError("some error".to_string())) }); struct TestFut(OnceFut<()>); @@ -1432,10 +1430,10 @@ mod tests { let wrapped_err = DataFusionError::from(arrow_err_from_fut); let root_err = wrapped_err.find_root(); - assert!(matches!( - root_err, - DataFusionError::ArrowError(ArrowError::CsvError(_)) - )) + let _expected = + arrow_datafusion_err!(ArrowError::CsvError("some error".to_owned())); + + assert!(matches!(root_err, _expected)) } #[test] diff --git a/datafusion/physical-plan/src/repartition/mod.rs b/datafusion/physical-plan/src/repartition/mod.rs index 769dc5e0e197..07693f747fee 100644 --- a/datafusion/physical-plan/src/repartition/mod.rs +++ b/datafusion/physical-plan/src/repartition/mod.rs @@ -34,7 +34,7 @@ use log::trace; use parking_lot::Mutex; use tokio::task::JoinHandle; -use datafusion_common::{not_impl_err, DataFusionError, Result}; +use datafusion_common::{arrow_datafusion_err, not_impl_err, DataFusionError, Result}; use datafusion_execution::memory_pool::MemoryConsumer; use datafusion_execution::TaskContext; use datafusion_physical_expr::{EquivalenceProperties, PhysicalExpr}; @@ -200,7 +200,7 @@ impl BatchPartitioner { .iter() .map(|c| { arrow::compute::take(c.as_ref(), &indices, None) - .map_err(DataFusionError::ArrowError) + .map_err(|e| arrow_datafusion_err!(e)) }) .collect::>>()?; @@ -1414,9 +1414,8 @@ mod tests { // pull partitions for i in 0..exec.partitioning.partition_count() { let mut stream = exec.execute(i, task_ctx.clone())?; - let err = DataFusionError::ArrowError( - stream.next().await.unwrap().unwrap_err().into(), - ); + let err = + arrow_datafusion_err!(stream.next().await.unwrap().unwrap_err().into()); let err = err.find_root(); assert!( matches!(err, DataFusionError::ResourcesExhausted(_)), diff --git a/datafusion/physical-plan/src/windows/bounded_window_agg_exec.rs b/datafusion/physical-plan/src/windows/bounded_window_agg_exec.rs index 431a43bc6055..0871ec0d7ff3 100644 --- a/datafusion/physical-plan/src/windows/bounded_window_agg_exec.rs +++ b/datafusion/physical-plan/src/windows/bounded_window_agg_exec.rs @@ -51,7 +51,7 @@ use datafusion_common::utils::{ evaluate_partition_ranges, get_arrayref_at_indices, get_at_indices, get_record_batch_at_indices, get_row_at_idx, }; -use datafusion_common::{exec_err, DataFusionError, Result}; +use datafusion_common::{arrow_datafusion_err, exec_err, DataFusionError, Result}; use datafusion_execution::TaskContext; use datafusion_expr::window_state::{PartitionBatchState, WindowAggState}; use datafusion_expr::ColumnarValue; @@ -499,7 +499,7 @@ impl PartitionSearcher for LinearSearch { .iter() .map(|items| { concat(&items.iter().map(|e| e.as_ref()).collect::>()) - .map_err(DataFusionError::ArrowError) + .map_err(|e| arrow_datafusion_err!(e)) }) .collect::>>()?; // We should emit columns according to row index ordering. diff --git a/datafusion/proto/src/logical_plan/from_proto.rs b/datafusion/proto/src/logical_plan/from_proto.rs index 854bfda9a861..c582e92dc11c 100644 --- a/datafusion/proto/src/logical_plan/from_proto.rs +++ b/datafusion/proto/src/logical_plan/from_proto.rs @@ -36,8 +36,9 @@ use arrow::{ }; use datafusion::execution::registry::FunctionRegistry; use datafusion_common::{ - internal_err, plan_datafusion_err, Column, Constraint, Constraints, DFField, - DFSchema, DFSchemaRef, DataFusionError, OwnedTableReference, Result, ScalarValue, + arrow_datafusion_err, internal_err, plan_datafusion_err, Column, Constraint, + Constraints, DFField, DFSchema, DFSchemaRef, DataFusionError, OwnedTableReference, + Result, ScalarValue, }; use datafusion_expr::window_frame::{check_window_frame, regularize_window_order_by}; use datafusion_expr::{ @@ -717,7 +718,7 @@ impl TryFrom<&protobuf::ScalarValue> for ScalarValue { None, &message.version(), ) - .map_err(DataFusionError::ArrowError) + .map_err(|e| arrow_datafusion_err!(e)) .map_err(|e| e.context("Decoding ScalarValue::List Value"))?; let arr = record_batch.column(0); match value { diff --git a/datafusion/sqllogictest/test_files/math.slt b/datafusion/sqllogictest/test_files/math.slt index ee1e345f946a..0fa7ff9c2051 100644 --- a/datafusion/sqllogictest/test_files/math.slt +++ b/datafusion/sqllogictest/test_files/math.slt @@ -293,53 +293,52 @@ select c1*0, c2*0, c3*0, c4*0, c5*0, c6*0, c7*0, c8*0 from test_non_nullable_int ---- 0 0 0 0 0 0 0 0 -query error DataFusion error: Optimizer rule 'simplify_expressions' failed\ncaused by\nArrow error: Divide by zero error +query error DataFusion error: Optimizer rule 'simplify_expressions' failed\ncaused by\nError during planning: Divide by zero SELECT c1/0 FROM test_non_nullable_integer -query error DataFusion error: Optimizer rule 'simplify_expressions' failed\ncaused by\nArrow error: Divide by zero error +query error DataFusion error: Optimizer rule 'simplify_expressions' failed\ncaused by\nError during planning: Divide by zero SELECT c2/0 FROM test_non_nullable_integer -query error DataFusion error: Optimizer rule 'simplify_expressions' failed\ncaused by\nArrow error: Divide by zero error +query error DataFusion error: Optimizer rule 'simplify_expressions' failed\ncaused by\nError during planning: Divide by zero SELECT c3/0 FROM test_non_nullable_integer -query error DataFusion error: Optimizer rule 'simplify_expressions' failed\ncaused by\nArrow error: Divide by zero error +query error DataFusion error: Optimizer rule 'simplify_expressions' failed\ncaused by\nError during planning: Divide by zero SELECT c4/0 FROM test_non_nullable_integer -query error DataFusion error: Optimizer rule 'simplify_expressions' failed\ncaused by\nArrow error: Divide by zero error +query error DataFusion error: Optimizer rule 'simplify_expressions' failed\ncaused by\nError during planning: Divide by zero SELECT c5/0 FROM test_non_nullable_integer -query error DataFusion error: Optimizer rule 'simplify_expressions' failed\ncaused by\nArrow error: Divide by zero error +query error DataFusion error: Optimizer rule 'simplify_expressions' failed\ncaused by\nError during planning: Divide by zero SELECT c6/0 FROM test_non_nullable_integer -query error DataFusion error: Optimizer rule 'simplify_expressions' failed\ncaused by\nArrow error: Divide by zero error +query error DataFusion error: Optimizer rule 'simplify_expressions' failed\ncaused by\nError during planning: Divide by zero SELECT c7/0 FROM test_non_nullable_integer -query error DataFusion error: Optimizer rule 'simplify_expressions' failed\ncaused by\nArrow error: Divide by zero error +query error DataFusion error: Optimizer rule 'simplify_expressions' failed\ncaused by\nError during planning: Divide by zero SELECT c8/0 FROM test_non_nullable_integer - -query error DataFusion error: Optimizer rule 'simplify_expressions' failed\ncaused by\nArrow error: Divide by zero error +query error DataFusion error: Optimizer rule 'simplify_expressions' failed\ncaused by\nError during planning: Divide by zero SELECT c1%0 FROM test_non_nullable_integer -query error DataFusion error: Optimizer rule 'simplify_expressions' failed\ncaused by\nArrow error: Divide by zero error +query error DataFusion error: Optimizer rule 'simplify_expressions' failed\ncaused by\nError during planning: Divide by zero SELECT c2%0 FROM test_non_nullable_integer -query error DataFusion error: Optimizer rule 'simplify_expressions' failed\ncaused by\nArrow error: Divide by zero error +query error DataFusion error: Optimizer rule 'simplify_expressions' failed\ncaused by\nError during planning: Divide by zero SELECT c3%0 FROM test_non_nullable_integer -query error DataFusion error: Optimizer rule 'simplify_expressions' failed\ncaused by\nArrow error: Divide by zero error +query error DataFusion error: Optimizer rule 'simplify_expressions' failed\ncaused by\nError during planning: Divide by zero SELECT c4%0 FROM test_non_nullable_integer -query error DataFusion error: Optimizer rule 'simplify_expressions' failed\ncaused by\nArrow error: Divide by zero error +query error DataFusion error: Optimizer rule 'simplify_expressions' failed\ncaused by\nError during planning: Divide by zero SELECT c5%0 FROM test_non_nullable_integer -query error DataFusion error: Optimizer rule 'simplify_expressions' failed\ncaused by\nArrow error: Divide by zero error +query error DataFusion error: Optimizer rule 'simplify_expressions' failed\ncaused by\nError during planning: Divide by zero SELECT c6%0 FROM test_non_nullable_integer -query error DataFusion error: Optimizer rule 'simplify_expressions' failed\ncaused by\nArrow error: Divide by zero error +query error DataFusion error: Optimizer rule 'simplify_expressions' failed\ncaused by\nError during planning: Divide by zero SELECT c7%0 FROM test_non_nullable_integer -query error DataFusion error: Optimizer rule 'simplify_expressions' failed\ncaused by\nArrow error: Divide by zero error +query error DataFusion error: Optimizer rule 'simplify_expressions' failed\ncaused by\nError during planning: Divide by zero SELECT c8%0 FROM test_non_nullable_integer statement ok @@ -557,10 +556,10 @@ SELECT c1*0 FROM test_non_nullable_decimal ---- 0 -query error DataFusion error: Optimizer rule 'simplify_expressions' failed\ncaused by\nArrow error: Divide by zero error +query error DataFusion error: Optimizer rule 'simplify_expressions' failed\ncaused by\nError during planning: Divide by zero SELECT c1/0 FROM test_non_nullable_decimal -query error DataFusion error: Optimizer rule 'simplify_expressions' failed\ncaused by\nArrow error: Divide by zero error +query error DataFusion error: Optimizer rule 'simplify_expressions' failed\ncaused by\nError during planning: Divide by zero SELECT c1%0 FROM test_non_nullable_decimal statement ok From fd121d3e29404a243a3c18c67c40fa7132ed9ed2 Mon Sep 17 00:00:00 2001 From: Devin D'Angelo Date: Fri, 22 Dec 2023 02:00:25 -0500 Subject: [PATCH 477/572] Add examples of DataFrame::write* methods without S3 dependency (#8606) --- datafusion-examples/README.md | 3 +- .../examples/dataframe_output.rs | 76 +++++++++++++++++++ 2 files changed, 78 insertions(+), 1 deletion(-) create mode 100644 datafusion-examples/examples/dataframe_output.rs diff --git a/datafusion-examples/README.md b/datafusion-examples/README.md index 305422ccd0be..057cdd475273 100644 --- a/datafusion-examples/README.md +++ b/datafusion-examples/README.md @@ -47,7 +47,8 @@ cargo run --example csv_sql - [`catalog.rs`](examples/external_dependency/catalog.rs): Register the table into a custom catalog - [`custom_datasource.rs`](examples/custom_datasource.rs): Run queries against a custom datasource (TableProvider) - [`dataframe.rs`](examples/dataframe.rs): Run a query using a DataFrame against a local parquet file -- [`dataframe-to-s3.rs`](examples/external_dependency/dataframe-to-s3.rs): Run a query using a DataFrame against a parquet file from s3 +- [`dataframe-to-s3.rs`](examples/external_dependency/dataframe-to-s3.rs): Run a query using a DataFrame against a parquet file from s3 and writing back to s3 +- [`dataframe_output.rs`](examples/dataframe_output.rs): Examples of methods which write data out from a DataFrame - [`dataframe_in_memory.rs`](examples/dataframe_in_memory.rs): Run a query using a DataFrame against data in memory - [`deserialize_to_struct.rs`](examples/deserialize_to_struct.rs): Convert query results into rust structs using serde - [`expr_api.rs`](examples/expr_api.rs): Create, execute, simplify and anaylze `Expr`s diff --git a/datafusion-examples/examples/dataframe_output.rs b/datafusion-examples/examples/dataframe_output.rs new file mode 100644 index 000000000000..c773384dfcd5 --- /dev/null +++ b/datafusion-examples/examples/dataframe_output.rs @@ -0,0 +1,76 @@ +// 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 datafusion::{dataframe::DataFrameWriteOptions, prelude::*}; +use datafusion_common::{parsers::CompressionTypeVariant, DataFusionError}; + +/// This example demonstrates the various methods to write out a DataFrame to local storage. +/// See datafusion-examples/examples/external_dependency/dataframe-to-s3.rs for an example +/// using a remote object store. +#[tokio::main] +async fn main() -> Result<(), DataFusionError> { + let ctx = SessionContext::new(); + + let mut df = ctx.sql("values ('a'), ('b'), ('c')").await.unwrap(); + + // Ensure the column names and types match the target table + df = df.with_column_renamed("column1", "tablecol1").unwrap(); + + ctx.sql( + "create external table + test(tablecol1 varchar) + stored as parquet + location './datafusion-examples/test_table/'", + ) + .await? + .collect() + .await?; + + // This is equivalent to INSERT INTO test VALUES ('a'), ('b'), ('c'). + // The behavior of write_table depends on the TableProvider's implementation + // of the insert_into method. + df.clone() + .write_table("test", DataFrameWriteOptions::new()) + .await?; + + df.clone() + .write_parquet( + "./datafusion-examples/test_parquet/", + DataFrameWriteOptions::new(), + None, + ) + .await?; + + df.clone() + .write_csv( + "./datafusion-examples/test_csv/", + // DataFrameWriteOptions contains options which control how data is written + // such as compression codec + DataFrameWriteOptions::new().with_compression(CompressionTypeVariant::GZIP), + None, + ) + .await?; + + df.clone() + .write_json( + "./datafusion-examples/test_json/", + DataFrameWriteOptions::new(), + ) + .await?; + + Ok(()) +} From 0ff5305db6b03128282d31afac69fa727e1fe7c4 Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Fri, 22 Dec 2023 04:14:45 -0700 Subject: [PATCH 478/572] Implement logical plan serde for CopyTo (#8618) * Implement logical plan serde for CopyTo * add link to issue * clippy * remove debug logging --- datafusion/proto/proto/datafusion.proto | 21 + datafusion/proto/src/generated/pbjson.rs | 395 ++++++++++++++++++ datafusion/proto/src/generated/prost.rs | 43 +- datafusion/proto/src/logical_plan/mod.rs | 86 +++- .../tests/cases/roundtrip_logical_plan.rs | 68 ++- 5 files changed, 603 insertions(+), 10 deletions(-) diff --git a/datafusion/proto/proto/datafusion.proto b/datafusion/proto/proto/datafusion.proto index cc802ee95710..05f0b6434368 100644 --- a/datafusion/proto/proto/datafusion.proto +++ b/datafusion/proto/proto/datafusion.proto @@ -74,6 +74,7 @@ message LogicalPlanNode { PrepareNode prepare = 26; DropViewNode drop_view = 27; DistinctOnNode distinct_on = 28; + CopyToNode copy_to = 29; } } @@ -317,6 +318,26 @@ message DistinctOnNode { LogicalPlanNode input = 4; } +message CopyToNode { + LogicalPlanNode input = 1; + string output_url = 2; + bool single_file_output = 3; + oneof CopyOptions { + SQLOptions sql_options = 4; + FileTypeWriterOptions writer_options = 5; + } + string file_type = 6; +} + +message SQLOptions { + repeated SQLOption option = 1; +} + +message SQLOption { + string key = 1; + string value = 2; +} + message UnionNode { repeated LogicalPlanNode inputs = 1; } diff --git a/datafusion/proto/src/generated/pbjson.rs b/datafusion/proto/src/generated/pbjson.rs index fb3a3ad91d06..0fdeab0a40f6 100644 --- a/datafusion/proto/src/generated/pbjson.rs +++ b/datafusion/proto/src/generated/pbjson.rs @@ -3704,6 +3704,188 @@ impl<'de> serde::Deserialize<'de> for Constraints { deserializer.deserialize_struct("datafusion.Constraints", FIELDS, GeneratedVisitor) } } +impl serde::Serialize for CopyToNode { + #[allow(deprecated)] + fn serialize(&self, serializer: S) -> std::result::Result + where + S: serde::Serializer, + { + use serde::ser::SerializeStruct; + let mut len = 0; + if self.input.is_some() { + len += 1; + } + if !self.output_url.is_empty() { + len += 1; + } + if self.single_file_output { + len += 1; + } + if !self.file_type.is_empty() { + len += 1; + } + if self.copy_options.is_some() { + len += 1; + } + let mut struct_ser = serializer.serialize_struct("datafusion.CopyToNode", len)?; + if let Some(v) = self.input.as_ref() { + struct_ser.serialize_field("input", v)?; + } + if !self.output_url.is_empty() { + struct_ser.serialize_field("outputUrl", &self.output_url)?; + } + if self.single_file_output { + struct_ser.serialize_field("singleFileOutput", &self.single_file_output)?; + } + if !self.file_type.is_empty() { + struct_ser.serialize_field("fileType", &self.file_type)?; + } + if let Some(v) = self.copy_options.as_ref() { + match v { + copy_to_node::CopyOptions::SqlOptions(v) => { + struct_ser.serialize_field("sqlOptions", v)?; + } + copy_to_node::CopyOptions::WriterOptions(v) => { + struct_ser.serialize_field("writerOptions", v)?; + } + } + } + struct_ser.end() + } +} +impl<'de> serde::Deserialize<'de> for CopyToNode { + #[allow(deprecated)] + fn deserialize(deserializer: D) -> std::result::Result + where + D: serde::Deserializer<'de>, + { + const FIELDS: &[&str] = &[ + "input", + "output_url", + "outputUrl", + "single_file_output", + "singleFileOutput", + "file_type", + "fileType", + "sql_options", + "sqlOptions", + "writer_options", + "writerOptions", + ]; + + #[allow(clippy::enum_variant_names)] + enum GeneratedField { + Input, + OutputUrl, + SingleFileOutput, + FileType, + SqlOptions, + WriterOptions, + } + impl<'de> serde::Deserialize<'de> for GeneratedField { + fn deserialize(deserializer: D) -> std::result::Result + where + D: serde::Deserializer<'de>, + { + struct GeneratedVisitor; + + impl<'de> serde::de::Visitor<'de> for GeneratedVisitor { + type Value = GeneratedField; + + fn expecting(&self, formatter: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + write!(formatter, "expected one of: {:?}", &FIELDS) + } + + #[allow(unused_variables)] + fn visit_str(self, value: &str) -> std::result::Result + where + E: serde::de::Error, + { + match value { + "input" => Ok(GeneratedField::Input), + "outputUrl" | "output_url" => Ok(GeneratedField::OutputUrl), + "singleFileOutput" | "single_file_output" => Ok(GeneratedField::SingleFileOutput), + "fileType" | "file_type" => Ok(GeneratedField::FileType), + "sqlOptions" | "sql_options" => Ok(GeneratedField::SqlOptions), + "writerOptions" | "writer_options" => Ok(GeneratedField::WriterOptions), + _ => Err(serde::de::Error::unknown_field(value, FIELDS)), + } + } + } + deserializer.deserialize_identifier(GeneratedVisitor) + } + } + struct GeneratedVisitor; + impl<'de> serde::de::Visitor<'de> for GeneratedVisitor { + type Value = CopyToNode; + + fn expecting(&self, formatter: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + formatter.write_str("struct datafusion.CopyToNode") + } + + fn visit_map(self, mut map_: V) -> std::result::Result + where + V: serde::de::MapAccess<'de>, + { + let mut input__ = None; + let mut output_url__ = None; + let mut single_file_output__ = None; + let mut file_type__ = None; + let mut copy_options__ = None; + while let Some(k) = map_.next_key()? { + match k { + GeneratedField::Input => { + if input__.is_some() { + return Err(serde::de::Error::duplicate_field("input")); + } + input__ = map_.next_value()?; + } + GeneratedField::OutputUrl => { + if output_url__.is_some() { + return Err(serde::de::Error::duplicate_field("outputUrl")); + } + output_url__ = Some(map_.next_value()?); + } + GeneratedField::SingleFileOutput => { + if single_file_output__.is_some() { + return Err(serde::de::Error::duplicate_field("singleFileOutput")); + } + single_file_output__ = Some(map_.next_value()?); + } + GeneratedField::FileType => { + if file_type__.is_some() { + return Err(serde::de::Error::duplicate_field("fileType")); + } + file_type__ = Some(map_.next_value()?); + } + GeneratedField::SqlOptions => { + if copy_options__.is_some() { + return Err(serde::de::Error::duplicate_field("sqlOptions")); + } + copy_options__ = map_.next_value::<::std::option::Option<_>>()?.map(copy_to_node::CopyOptions::SqlOptions) +; + } + GeneratedField::WriterOptions => { + if copy_options__.is_some() { + return Err(serde::de::Error::duplicate_field("writerOptions")); + } + copy_options__ = map_.next_value::<::std::option::Option<_>>()?.map(copy_to_node::CopyOptions::WriterOptions) +; + } + } + } + Ok(CopyToNode { + input: input__, + output_url: output_url__.unwrap_or_default(), + single_file_output: single_file_output__.unwrap_or_default(), + file_type: file_type__.unwrap_or_default(), + copy_options: copy_options__, + }) + } + } + deserializer.deserialize_struct("datafusion.CopyToNode", FIELDS, GeneratedVisitor) + } +} impl serde::Serialize for CreateCatalogNode { #[allow(deprecated)] fn serialize(&self, serializer: S) -> std::result::Result @@ -13336,6 +13518,9 @@ impl serde::Serialize for LogicalPlanNode { logical_plan_node::LogicalPlanType::DistinctOn(v) => { struct_ser.serialize_field("distinctOn", v)?; } + logical_plan_node::LogicalPlanType::CopyTo(v) => { + struct_ser.serialize_field("copyTo", v)?; + } } } struct_ser.end() @@ -13387,6 +13572,8 @@ impl<'de> serde::Deserialize<'de> for LogicalPlanNode { "dropView", "distinct_on", "distinctOn", + "copy_to", + "copyTo", ]; #[allow(clippy::enum_variant_names)] @@ -13418,6 +13605,7 @@ impl<'de> serde::Deserialize<'de> for LogicalPlanNode { Prepare, DropView, DistinctOn, + CopyTo, } impl<'de> serde::Deserialize<'de> for GeneratedField { fn deserialize(deserializer: D) -> std::result::Result @@ -13466,6 +13654,7 @@ impl<'de> serde::Deserialize<'de> for LogicalPlanNode { "prepare" => Ok(GeneratedField::Prepare), "dropView" | "drop_view" => Ok(GeneratedField::DropView), "distinctOn" | "distinct_on" => Ok(GeneratedField::DistinctOn), + "copyTo" | "copy_to" => Ok(GeneratedField::CopyTo), _ => Err(serde::de::Error::unknown_field(value, FIELDS)), } } @@ -13675,6 +13864,13 @@ impl<'de> serde::Deserialize<'de> for LogicalPlanNode { return Err(serde::de::Error::duplicate_field("distinctOn")); } logical_plan_type__ = map_.next_value::<::std::option::Option<_>>()?.map(logical_plan_node::LogicalPlanType::DistinctOn) +; + } + GeneratedField::CopyTo => { + if logical_plan_type__.is_some() { + return Err(serde::de::Error::duplicate_field("copyTo")); + } + logical_plan_type__ = map_.next_value::<::std::option::Option<_>>()?.map(logical_plan_node::LogicalPlanType::CopyTo) ; } } @@ -20742,6 +20938,205 @@ impl<'de> serde::Deserialize<'de> for RollupNode { deserializer.deserialize_struct("datafusion.RollupNode", FIELDS, GeneratedVisitor) } } +impl serde::Serialize for SqlOption { + #[allow(deprecated)] + fn serialize(&self, serializer: S) -> std::result::Result + where + S: serde::Serializer, + { + use serde::ser::SerializeStruct; + let mut len = 0; + if !self.key.is_empty() { + len += 1; + } + if !self.value.is_empty() { + len += 1; + } + let mut struct_ser = serializer.serialize_struct("datafusion.SQLOption", len)?; + if !self.key.is_empty() { + struct_ser.serialize_field("key", &self.key)?; + } + if !self.value.is_empty() { + struct_ser.serialize_field("value", &self.value)?; + } + struct_ser.end() + } +} +impl<'de> serde::Deserialize<'de> for SqlOption { + #[allow(deprecated)] + fn deserialize(deserializer: D) -> std::result::Result + where + D: serde::Deserializer<'de>, + { + const FIELDS: &[&str] = &[ + "key", + "value", + ]; + + #[allow(clippy::enum_variant_names)] + enum GeneratedField { + Key, + Value, + } + impl<'de> serde::Deserialize<'de> for GeneratedField { + fn deserialize(deserializer: D) -> std::result::Result + where + D: serde::Deserializer<'de>, + { + struct GeneratedVisitor; + + impl<'de> serde::de::Visitor<'de> for GeneratedVisitor { + type Value = GeneratedField; + + fn expecting(&self, formatter: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + write!(formatter, "expected one of: {:?}", &FIELDS) + } + + #[allow(unused_variables)] + fn visit_str(self, value: &str) -> std::result::Result + where + E: serde::de::Error, + { + match value { + "key" => Ok(GeneratedField::Key), + "value" => Ok(GeneratedField::Value), + _ => Err(serde::de::Error::unknown_field(value, FIELDS)), + } + } + } + deserializer.deserialize_identifier(GeneratedVisitor) + } + } + struct GeneratedVisitor; + impl<'de> serde::de::Visitor<'de> for GeneratedVisitor { + type Value = SqlOption; + + fn expecting(&self, formatter: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + formatter.write_str("struct datafusion.SQLOption") + } + + fn visit_map(self, mut map_: V) -> std::result::Result + where + V: serde::de::MapAccess<'de>, + { + let mut key__ = None; + let mut value__ = None; + while let Some(k) = map_.next_key()? { + match k { + GeneratedField::Key => { + if key__.is_some() { + return Err(serde::de::Error::duplicate_field("key")); + } + key__ = Some(map_.next_value()?); + } + GeneratedField::Value => { + if value__.is_some() { + return Err(serde::de::Error::duplicate_field("value")); + } + value__ = Some(map_.next_value()?); + } + } + } + Ok(SqlOption { + key: key__.unwrap_or_default(), + value: value__.unwrap_or_default(), + }) + } + } + deserializer.deserialize_struct("datafusion.SQLOption", FIELDS, GeneratedVisitor) + } +} +impl serde::Serialize for SqlOptions { + #[allow(deprecated)] + fn serialize(&self, serializer: S) -> std::result::Result + where + S: serde::Serializer, + { + use serde::ser::SerializeStruct; + let mut len = 0; + if !self.option.is_empty() { + len += 1; + } + let mut struct_ser = serializer.serialize_struct("datafusion.SQLOptions", len)?; + if !self.option.is_empty() { + struct_ser.serialize_field("option", &self.option)?; + } + struct_ser.end() + } +} +impl<'de> serde::Deserialize<'de> for SqlOptions { + #[allow(deprecated)] + fn deserialize(deserializer: D) -> std::result::Result + where + D: serde::Deserializer<'de>, + { + const FIELDS: &[&str] = &[ + "option", + ]; + + #[allow(clippy::enum_variant_names)] + enum GeneratedField { + Option, + } + impl<'de> serde::Deserialize<'de> for GeneratedField { + fn deserialize(deserializer: D) -> std::result::Result + where + D: serde::Deserializer<'de>, + { + struct GeneratedVisitor; + + impl<'de> serde::de::Visitor<'de> for GeneratedVisitor { + type Value = GeneratedField; + + fn expecting(&self, formatter: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + write!(formatter, "expected one of: {:?}", &FIELDS) + } + + #[allow(unused_variables)] + fn visit_str(self, value: &str) -> std::result::Result + where + E: serde::de::Error, + { + match value { + "option" => Ok(GeneratedField::Option), + _ => Err(serde::de::Error::unknown_field(value, FIELDS)), + } + } + } + deserializer.deserialize_identifier(GeneratedVisitor) + } + } + struct GeneratedVisitor; + impl<'de> serde::de::Visitor<'de> for GeneratedVisitor { + type Value = SqlOptions; + + fn expecting(&self, formatter: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + formatter.write_str("struct datafusion.SQLOptions") + } + + fn visit_map(self, mut map_: V) -> std::result::Result + where + V: serde::de::MapAccess<'de>, + { + let mut option__ = None; + while let Some(k) = map_.next_key()? { + match k { + GeneratedField::Option => { + if option__.is_some() { + return Err(serde::de::Error::duplicate_field("option")); + } + option__ = Some(map_.next_value()?); + } + } + } + Ok(SqlOptions { + option: option__.unwrap_or_default(), + }) + } + } + deserializer.deserialize_struct("datafusion.SQLOptions", FIELDS, GeneratedVisitor) + } +} impl serde::Serialize for ScalarDictionaryValue { #[allow(deprecated)] fn serialize(&self, serializer: S) -> std::result::Result diff --git a/datafusion/proto/src/generated/prost.rs b/datafusion/proto/src/generated/prost.rs index 9030e90a24c8..e44355859d65 100644 --- a/datafusion/proto/src/generated/prost.rs +++ b/datafusion/proto/src/generated/prost.rs @@ -38,7 +38,7 @@ pub struct DfSchema { pub struct LogicalPlanNode { #[prost( oneof = "logical_plan_node::LogicalPlanType", - tags = "1, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28" + tags = "1, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29" )] pub logical_plan_type: ::core::option::Option, } @@ -101,6 +101,8 @@ pub mod logical_plan_node { DropView(super::DropViewNode), #[prost(message, tag = "28")] DistinctOn(::prost::alloc::boxed::Box), + #[prost(message, tag = "29")] + CopyTo(::prost::alloc::boxed::Box), } } #[allow(clippy::derive_partial_eq_without_eq)] @@ -502,6 +504,45 @@ pub struct DistinctOnNode { } #[allow(clippy::derive_partial_eq_without_eq)] #[derive(Clone, PartialEq, ::prost::Message)] +pub struct CopyToNode { + #[prost(message, optional, boxed, tag = "1")] + pub input: ::core::option::Option<::prost::alloc::boxed::Box>, + #[prost(string, tag = "2")] + pub output_url: ::prost::alloc::string::String, + #[prost(bool, tag = "3")] + pub single_file_output: bool, + #[prost(string, tag = "6")] + pub file_type: ::prost::alloc::string::String, + #[prost(oneof = "copy_to_node::CopyOptions", tags = "4, 5")] + pub copy_options: ::core::option::Option, +} +/// Nested message and enum types in `CopyToNode`. +pub mod copy_to_node { + #[allow(clippy::derive_partial_eq_without_eq)] + #[derive(Clone, PartialEq, ::prost::Oneof)] + pub enum CopyOptions { + #[prost(message, tag = "4")] + SqlOptions(super::SqlOptions), + #[prost(message, tag = "5")] + WriterOptions(super::FileTypeWriterOptions), + } +} +#[allow(clippy::derive_partial_eq_without_eq)] +#[derive(Clone, PartialEq, ::prost::Message)] +pub struct SqlOptions { + #[prost(message, repeated, tag = "1")] + pub option: ::prost::alloc::vec::Vec, +} +#[allow(clippy::derive_partial_eq_without_eq)] +#[derive(Clone, PartialEq, ::prost::Message)] +pub struct SqlOption { + #[prost(string, tag = "1")] + pub key: ::prost::alloc::string::String, + #[prost(string, tag = "2")] + pub value: ::prost::alloc::string::String, +} +#[allow(clippy::derive_partial_eq_without_eq)] +#[derive(Clone, PartialEq, ::prost::Message)] pub struct UnionNode { #[prost(message, repeated, tag = "1")] pub inputs: ::prost::alloc::vec::Vec, diff --git a/datafusion/proto/src/logical_plan/mod.rs b/datafusion/proto/src/logical_plan/mod.rs index 948228d87d46..e03b3ffa7b84 100644 --- a/datafusion/proto/src/logical_plan/mod.rs +++ b/datafusion/proto/src/logical_plan/mod.rs @@ -22,7 +22,9 @@ use std::sync::Arc; use crate::common::{byte_to_string, proto_error, str_to_byte}; use crate::protobuf::logical_plan_node::LogicalPlanType::CustomScan; -use crate::protobuf::{CustomTableScanNode, LogicalExprNodeCollection}; +use crate::protobuf::{ + copy_to_node, CustomTableScanNode, LogicalExprNodeCollection, SqlOption, +}; use crate::{ convert_required, protobuf::{ @@ -44,12 +46,13 @@ use datafusion::{ datasource::{provider_as_source, source_as_provider}, prelude::SessionContext, }; -use datafusion_common::plan_datafusion_err; use datafusion_common::{ - context, internal_err, not_impl_err, parsers::CompressionTypeVariant, - DataFusionError, OwnedTableReference, Result, + context, file_options::StatementOptions, internal_err, not_impl_err, + parsers::CompressionTypeVariant, plan_datafusion_err, DataFusionError, FileType, + OwnedTableReference, Result, }; use datafusion_expr::{ + dml, logical_plan::{ builder::project, Aggregate, CreateCatalog, CreateCatalogSchema, CreateExternalTable, CreateView, CrossJoin, DdlStatement, Distinct, @@ -59,6 +62,7 @@ use datafusion_expr::{ DistinctOn, DropView, Expr, LogicalPlan, LogicalPlanBuilder, }; +use datafusion_expr::dml::CopyOptions; use prost::bytes::BufMut; use prost::Message; @@ -823,6 +827,36 @@ impl AsLogicalPlan for LogicalPlanNode { schema: Arc::new(convert_required!(dropview.schema)?), }), )), + LogicalPlanType::CopyTo(copy) => { + let input: LogicalPlan = + into_logical_plan!(copy.input, ctx, extension_codec)?; + + let copy_options = match ©.copy_options { + Some(copy_to_node::CopyOptions::SqlOptions(opt)) => { + let options = opt.option.iter().map(|o| (o.key.clone(), o.value.clone())).collect(); + CopyOptions::SQLOptions(StatementOptions::from( + &options, + )) + } + Some(copy_to_node::CopyOptions::WriterOptions(_)) => { + return Err(proto_error( + "LogicalPlan serde is not yet implemented for CopyTo with WriterOptions", + )) + } + other => return Err(proto_error(format!( + "LogicalPlan serde is not yet implemented for CopyTo with CopyOptions {other:?}", + ))) + }; + Ok(datafusion_expr::LogicalPlan::Copy( + datafusion_expr::dml::CopyTo { + input: Arc::new(input), + output_url: copy.output_url.clone(), + file_format: FileType::from_str(©.file_type)?, + single_file_output: copy.single_file_output, + copy_options, + }, + )) + } } } @@ -1534,9 +1568,47 @@ impl AsLogicalPlan for LogicalPlanNode { LogicalPlan::Dml(_) => Err(proto_error( "LogicalPlan serde is not yet implemented for Dml", )), - LogicalPlan::Copy(_) => Err(proto_error( - "LogicalPlan serde is not yet implemented for Copy", - )), + LogicalPlan::Copy(dml::CopyTo { + input, + output_url, + single_file_output, + file_format, + copy_options, + }) => { + let input = protobuf::LogicalPlanNode::try_from_logical_plan( + input, + extension_codec, + )?; + + let copy_options_proto: Option = match copy_options { + CopyOptions::SQLOptions(opt) => { + let options: Vec = opt.clone().into_inner().iter().map(|(k, v)| SqlOption { + key: k.to_string(), + value: v.to_string(), + }).collect(); + Some(copy_to_node::CopyOptions::SqlOptions(protobuf::SqlOptions { + option: options + })) + } + CopyOptions::WriterOptions(_) => { + return Err(proto_error( + "LogicalPlan serde is not yet implemented for CopyTo with WriterOptions", + )) + } + }; + + Ok(protobuf::LogicalPlanNode { + logical_plan_type: Some(LogicalPlanType::CopyTo(Box::new( + protobuf::CopyToNode { + input: Some(Box::new(input)), + single_file_output: *single_file_output, + output_url: output_url.to_string(), + file_type: file_format.to_string(), + copy_options: copy_options_proto, + }, + ))), + }) + } LogicalPlan::DescribeTable(_) => Err(proto_error( "LogicalPlan serde is not yet implemented for DescribeTable", )), diff --git a/datafusion/proto/tests/cases/roundtrip_logical_plan.rs b/datafusion/proto/tests/cases/roundtrip_logical_plan.rs index 8e15b5d0d480..9798b06f4724 100644 --- a/datafusion/proto/tests/cases/roundtrip_logical_plan.rs +++ b/datafusion/proto/tests/cases/roundtrip_logical_plan.rs @@ -31,12 +31,16 @@ use datafusion::datasource::provider::TableProviderFactory; use datafusion::datasource::TableProvider; use datafusion::execution::context::SessionState; use datafusion::execution::runtime_env::{RuntimeConfig, RuntimeEnv}; +use datafusion::parquet::file::properties::WriterProperties; use datafusion::physical_plan::functions::make_scalar_function; use datafusion::prelude::{create_udf, CsvReadOptions, SessionConfig, SessionContext}; use datafusion::test_util::{TestTableFactory, TestTableProvider}; -use datafusion_common::Result; -use datafusion_common::{internal_err, not_impl_err, plan_err}; +use datafusion_common::file_options::parquet_writer::ParquetWriterOptions; +use datafusion_common::file_options::StatementOptions; +use datafusion_common::{internal_err, not_impl_err, plan_err, FileTypeWriterOptions}; use datafusion_common::{DFField, DFSchema, DFSchemaRef, DataFusionError, ScalarValue}; +use datafusion_common::{FileType, Result}; +use datafusion_expr::dml::{CopyOptions, CopyTo}; use datafusion_expr::expr::{ self, Between, BinaryExpr, Case, Cast, GroupingSet, InList, Like, ScalarFunction, Sort, @@ -301,6 +305,66 @@ async fn roundtrip_logical_plan_aggregation() -> Result<()> { Ok(()) } +#[tokio::test] +async fn roundtrip_logical_plan_copy_to_sql_options() -> Result<()> { + let ctx = SessionContext::new(); + + let input = create_csv_scan(&ctx).await?; + + let mut options = HashMap::new(); + options.insert("foo".to_string(), "bar".to_string()); + + let plan = LogicalPlan::Copy(CopyTo { + input: Arc::new(input), + output_url: "test.csv".to_string(), + file_format: FileType::CSV, + single_file_output: true, + copy_options: CopyOptions::SQLOptions(StatementOptions::from(&options)), + }); + + let bytes = logical_plan_to_bytes(&plan)?; + let logical_round_trip = logical_plan_from_bytes(&bytes, &ctx)?; + assert_eq!(format!("{plan:?}"), format!("{logical_round_trip:?}")); + + Ok(()) +} + +#[tokio::test] +#[ignore] // see https://github.com/apache/arrow-datafusion/issues/8619 +async fn roundtrip_logical_plan_copy_to_writer_options() -> Result<()> { + let ctx = SessionContext::new(); + + let input = create_csv_scan(&ctx).await?; + + let writer_properties = WriterProperties::builder() + .set_bloom_filter_enabled(true) + .set_created_by("DataFusion Test".to_string()) + .build(); + let plan = LogicalPlan::Copy(CopyTo { + input: Arc::new(input), + output_url: "test.csv".to_string(), + file_format: FileType::CSV, + single_file_output: true, + copy_options: CopyOptions::WriterOptions(Box::new( + FileTypeWriterOptions::Parquet(ParquetWriterOptions::new(writer_properties)), + )), + }); + + let bytes = logical_plan_to_bytes(&plan)?; + let logical_round_trip = logical_plan_from_bytes(&bytes, &ctx)?; + assert_eq!(format!("{plan:?}"), format!("{logical_round_trip:?}")); + + Ok(()) +} + +async fn create_csv_scan(ctx: &SessionContext) -> Result { + ctx.register_csv("t1", "tests/testdata/test.csv", CsvReadOptions::default()) + .await?; + + let input = ctx.table("t1").await?.into_optimized_plan()?; + Ok(input) +} + #[tokio::test] async fn roundtrip_logical_plan_distinct_on() -> Result<()> { let ctx = SessionContext::new(); From 55121d8e48d99178a72a5dbaa773f1fbf4a2e059 Mon Sep 17 00:00:00 2001 From: Andrew Lamb Date: Fri, 22 Dec 2023 06:15:13 -0500 Subject: [PATCH 479/572] Fix InListExpr to return the correct number of rows (#8601) * Fix InListExpr to return the correct number of rows * Reduce repetition --- .../physical-expr/src/expressions/in_list.rs | 57 +++++++++++++++++-- 1 file changed, 53 insertions(+), 4 deletions(-) diff --git a/datafusion/physical-expr/src/expressions/in_list.rs b/datafusion/physical-expr/src/expressions/in_list.rs index 625b01ec9a7e..1a1634081c38 100644 --- a/datafusion/physical-expr/src/expressions/in_list.rs +++ b/datafusion/physical-expr/src/expressions/in_list.rs @@ -349,17 +349,18 @@ impl PhysicalExpr for InListExpr { } fn evaluate(&self, batch: &RecordBatch) -> Result { + let num_rows = batch.num_rows(); let value = self.expr.evaluate(batch)?; let r = match &self.static_filter { - Some(f) => f.contains(value.into_array(1)?.as_ref(), self.negated)?, + Some(f) => f.contains(value.into_array(num_rows)?.as_ref(), self.negated)?, None => { - let value = value.into_array(batch.num_rows())?; + let value = value.into_array(num_rows)?; let found = self.list.iter().map(|expr| expr.evaluate(batch)).try_fold( - BooleanArray::new(BooleanBuffer::new_unset(batch.num_rows()), None), + BooleanArray::new(BooleanBuffer::new_unset(num_rows), None), |result, expr| -> Result { Ok(or_kleene( &result, - &eq(&value, &expr?.into_array(batch.num_rows())?)?, + &eq(&value, &expr?.into_array(num_rows)?)?, )?) }, )?; @@ -1267,4 +1268,52 @@ mod tests { Ok(()) } + + #[test] + fn in_list_no_cols() -> Result<()> { + // test logic when the in_list expression doesn't have any columns + let schema = Schema::new(vec![Field::new("a", DataType::Int32, true)]); + let a = Int32Array::from(vec![Some(1), Some(2), None]); + let batch = RecordBatch::try_new(Arc::new(schema.clone()), vec![Arc::new(a)])?; + + let list = vec![lit(ScalarValue::from(1i32)), lit(ScalarValue::from(6i32))]; + + // 1 IN (1, 6) + let expr = lit(ScalarValue::Int32(Some(1))); + in_list!( + batch, + list.clone(), + &false, + // should have three outputs, as the input batch has three rows + vec![Some(true), Some(true), Some(true)], + expr, + &schema + ); + + // 2 IN (1, 6) + let expr = lit(ScalarValue::Int32(Some(2))); + in_list!( + batch, + list.clone(), + &false, + // should have three outputs, as the input batch has three rows + vec![Some(false), Some(false), Some(false)], + expr, + &schema + ); + + // NULL IN (1, 6) + let expr = lit(ScalarValue::Int32(None)); + in_list!( + batch, + list.clone(), + &false, + // should have three outputs, as the input batch has three rows + vec![None, None, None], + expr, + &schema + ); + + Ok(()) + } } From 39e9f41a21e8e2ffac39feabd13d6aa7eda5f213 Mon Sep 17 00:00:00 2001 From: Devin D'Angelo Date: Fri, 22 Dec 2023 06:56:27 -0500 Subject: [PATCH 480/572] Remove ListingTable single_file option (#8604) * remove listingtable single_file option * prettier --------- Co-authored-by: Andrew Lamb --- datafusion/core/src/datasource/listing/table.rs | 12 +----------- .../core/src/datasource/listing_table_factory.rs | 9 ++------- docs/source/user-guide/sql/write_options.md | 15 +++------------ 3 files changed, 6 insertions(+), 30 deletions(-) diff --git a/datafusion/core/src/datasource/listing/table.rs b/datafusion/core/src/datasource/listing/table.rs index 21d43dcd56db..a7af1bf1be28 100644 --- a/datafusion/core/src/datasource/listing/table.rs +++ b/datafusion/core/src/datasource/listing/table.rs @@ -246,9 +246,6 @@ pub struct ListingOptions { /// multiple equivalent orderings, the outer `Vec` will have a /// single element. pub file_sort_order: Vec>, - /// This setting when true indicates that the table is backed by a single file. - /// Any inserts to the table may only append to this existing file. - pub single_file: bool, /// This setting holds file format specific options which should be used /// when inserting into this table. pub file_type_write_options: Option, @@ -269,7 +266,6 @@ impl ListingOptions { collect_stat: true, target_partitions: 1, file_sort_order: vec![], - single_file: false, file_type_write_options: None, } } @@ -421,12 +417,6 @@ impl ListingOptions { self } - /// Configure if this table is backed by a sigle file - pub fn with_single_file(mut self, single_file: bool) -> Self { - self.single_file = single_file; - self - } - /// Configure file format specific writing options. pub fn with_write_options( mut self, @@ -790,7 +780,7 @@ impl TableProvider for ListingTable { file_groups, output_schema: self.schema(), table_partition_cols: self.options.table_partition_cols.clone(), - single_file_output: self.options.single_file, + single_file_output: false, overwrite, file_type_writer_options, }; diff --git a/datafusion/core/src/datasource/listing_table_factory.rs b/datafusion/core/src/datasource/listing_table_factory.rs index 68c97bbb7806..e8ffece320d7 100644 --- a/datafusion/core/src/datasource/listing_table_factory.rs +++ b/datafusion/core/src/datasource/listing_table_factory.rs @@ -135,12 +135,8 @@ impl TableProviderFactory for ListingTableFactory { let mut statement_options = StatementOptions::from(&cmd.options); - // Extract ListingTable specific options if present or set default - let single_file = statement_options - .take_bool_option("single_file")? - .unwrap_or(false); - - // Backwards compatibility (#8547) + // Backwards compatibility (#8547), discard deprecated options + statement_options.take_bool_option("single_file")?; if let Some(s) = statement_options.take_str_option("insert_mode") { if !s.eq_ignore_ascii_case("append_new_files") { return plan_err!("Unknown or unsupported insert mode {s}. Only append_new_files supported"); @@ -195,7 +191,6 @@ impl TableProviderFactory for ListingTableFactory { .with_target_partitions(state.config().target_partitions()) .with_table_partition_cols(table_partition_cols) .with_file_sort_order(cmd.order_exprs.clone()) - .with_single_file(single_file) .with_write_options(file_type_writer_options); let resolved_schema = match provided_schema { diff --git a/docs/source/user-guide/sql/write_options.md b/docs/source/user-guide/sql/write_options.md index 94adee960996..470591afafff 100644 --- a/docs/source/user-guide/sql/write_options.md +++ b/docs/source/user-guide/sql/write_options.md @@ -42,12 +42,11 @@ WITH HEADER ROW DELIMITER ';' LOCATION '/test/location/my_csv_table/' OPTIONS( -CREATE_LOCAL_PATH 'true', NULL_VALUE 'NAN' ); ``` -When running `INSERT INTO my_table ...`, the options from the `CREATE TABLE` will be respected (gzip compression, special delimiter, and header row included). Note that compression, header, and delimiter settings can also be specified within the `OPTIONS` tuple list. Dedicated syntax within the SQL statement always takes precedence over arbitrary option tuples, so if both are specified the `OPTIONS` setting will be ignored. CREATE_LOCAL_PATH is a special option that indicates if DataFusion should create local file paths when writing new files if they do not already exist. This option is useful if you wish to create an external table from scratch, using only DataFusion SQL statements. Finally, NULL_VALUE is a CSV format specific option that determines how null values should be encoded within the CSV file. +When running `INSERT INTO my_table ...`, the options from the `CREATE TABLE` will be respected (gzip compression, special delimiter, and header row included). Note that compression, header, and delimiter settings can also be specified within the `OPTIONS` tuple list. Dedicated syntax within the SQL statement always takes precedence over arbitrary option tuples, so if both are specified the `OPTIONS` setting will be ignored. NULL_VALUE is a CSV format specific option that determines how null values should be encoded within the CSV file. Finally, options can be passed when running a `COPY` command. @@ -70,17 +69,9 @@ In this example, we write the entirety of `source_table` out to a folder of parq The following special options are specific to the `COPY` command. | Option | Description | Default Value | -| ------------------ | ---------------------------------------------------------------------------------------------------------------------------------------------------------------------------- | ------------- | +| ------------------ | ---------------------------------------------------------------------------------------------------------------------------------------------------------------------------- | ------------- | --- | | SINGLE_FILE_OUTPUT | If true, COPY query will write output to a single file. Otherwise, multiple files will be written to a directory in parallel. | true | -| FORMAT | Specifies the file format COPY query will write out. If single_file_output is false or the format cannot be inferred from the file extension, then FORMAT must be specified. | N/A | - -### CREATE EXTERNAL TABLE Specific Options - -The following special options are specific to creating an external table. - -| Option | Description | Default Value | -| ----------- | --------------------------------------------------------------------------------------------------------------------- | ------------- | -| SINGLE_FILE | If true, indicates that this external table is backed by a single file. INSERT INTO queries will append to this file. | false | +| FORMAT | Specifies the file format COPY query will write out. If single_file_output is false or the format cannot be inferred from the file extension, then FORMAT must be specified. | N/A | | ### JSON Format Specific Options From ef34af8877d25cd84006806b355127179e2d4c89 Mon Sep 17 00:00:00 2001 From: Alex Huang Date: Fri, 22 Dec 2023 13:36:01 +0100 Subject: [PATCH 481/572] support LargeList in array_remove (#8595) --- .../physical-expr/src/array_expressions.rs | 114 ++++++-- datafusion/sqllogictest/test_files/array.slt | 269 ++++++++++++++++++ 2 files changed, 365 insertions(+), 18 deletions(-) diff --git a/datafusion/physical-expr/src/array_expressions.rs b/datafusion/physical-expr/src/array_expressions.rs index bdab65cab9e3..4dfc157e53c7 100644 --- a/datafusion/physical-expr/src/array_expressions.rs +++ b/datafusion/physical-expr/src/array_expressions.rs @@ -100,6 +100,14 @@ fn compare_element_to_list( row_index: usize, eq: bool, ) -> Result { + if list_array_row.data_type() != element_array.data_type() { + return exec_err!( + "compare_element_to_list received incompatible types: '{:?}' and '{:?}'.", + list_array_row.data_type(), + element_array.data_type() + ); + } + let indices = UInt32Array::from(vec![row_index as u32]); let element_array_row = arrow::compute::take(element_array, &indices, None)?; @@ -126,6 +134,26 @@ fn compare_element_to_list( }) .collect::() } + DataType::LargeList(_) => { + // compare each element of the from array + let element_array_row_inner = + as_large_list_array(&element_array_row)?.value(0); + let list_array_row_inner = as_large_list_array(list_array_row)?; + + list_array_row_inner + .iter() + // compare element by element the current row of list_array + .map(|row| { + row.map(|row| { + if eq { + row.eq(&element_array_row_inner) + } else { + row.ne(&element_array_row_inner) + } + }) + }) + .collect::() + } _ => { let element_arr = Scalar::new(element_array_row); // use not_distinct so we can compare NULL @@ -1511,14 +1539,14 @@ pub fn array_remove_n(args: &[ArrayRef]) -> Result { /// [4, 5, 6, 5], 5, 20, 2 ==> [4, 20, 6, 20] (both 5s are replaced) /// ) /// ``` -fn general_replace( - list_array: &ListArray, +fn general_replace( + list_array: &GenericListArray, from_array: &ArrayRef, to_array: &ArrayRef, arr_n: Vec, ) -> Result { // Build up the offsets for the final output array - let mut offsets: Vec = vec![0]; + let mut offsets: Vec = vec![O::usize_as(0)]; let values = list_array.values(); let original_data = values.to_data(); let to_data = to_array.to_data(); @@ -1540,8 +1568,8 @@ fn general_replace( continue; } - let start = offset_window[0] as usize; - let end = offset_window[1] as usize; + let start = offset_window[0]; + let end = offset_window[1]; let list_array_row = list_array.value(row_index); @@ -1550,43 +1578,56 @@ fn general_replace( let eq_array = compare_element_to_list(&list_array_row, &from_array, row_index, true)?; - let original_idx = 0; - let replace_idx = 1; + let original_idx = O::usize_as(0); + let replace_idx = O::usize_as(1); let n = arr_n[row_index]; let mut counter = 0; // All elements are false, no need to replace, just copy original data if eq_array.false_count() == eq_array.len() { - mutable.extend(original_idx, start, end); - offsets.push(offsets[row_index] + (end - start) as i32); + mutable.extend( + original_idx.to_usize().unwrap(), + start.to_usize().unwrap(), + end.to_usize().unwrap(), + ); + offsets.push(offsets[row_index] + (end - start)); valid.append(true); continue; } for (i, to_replace) in eq_array.iter().enumerate() { + let i = O::usize_as(i); if let Some(true) = to_replace { - mutable.extend(replace_idx, row_index, row_index + 1); + mutable.extend(replace_idx.to_usize().unwrap(), row_index, row_index + 1); counter += 1; if counter == n { // copy original data for any matches past n - mutable.extend(original_idx, start + i + 1, end); + mutable.extend( + original_idx.to_usize().unwrap(), + (start + i).to_usize().unwrap() + 1, + end.to_usize().unwrap(), + ); break; } } else { // copy original data for false / null matches - mutable.extend(original_idx, start + i, start + i + 1); + mutable.extend( + original_idx.to_usize().unwrap(), + (start + i).to_usize().unwrap(), + (start + i).to_usize().unwrap() + 1, + ); } } - offsets.push(offsets[row_index] + (end - start) as i32); + offsets.push(offsets[row_index] + (end - start)); valid.append(true); } let data = mutable.freeze(); - Ok(Arc::new(ListArray::try_new( + Ok(Arc::new(GenericListArray::::try_new( Arc::new(Field::new("item", list_array.value_type(), true)), - OffsetBuffer::new(offsets.into()), + OffsetBuffer::::new(offsets.into()), arrow_array::make_array(data), Some(NullBuffer::new(valid.finish())), )?)) @@ -1595,19 +1636,56 @@ fn general_replace( pub fn array_replace(args: &[ArrayRef]) -> Result { // replace at most one occurence for each element let arr_n = vec![1; args[0].len()]; - general_replace(as_list_array(&args[0])?, &args[1], &args[2], arr_n) + let array = &args[0]; + match array.data_type() { + DataType::List(_) => { + let list_array = array.as_list::(); + general_replace::(list_array, &args[1], &args[2], arr_n) + } + DataType::LargeList(_) => { + let list_array = array.as_list::(); + general_replace::(list_array, &args[1], &args[2], arr_n) + } + array_type => exec_err!("array_replace does not support type '{array_type:?}'."), + } } pub fn array_replace_n(args: &[ArrayRef]) -> Result { // replace the specified number of occurences let arr_n = as_int64_array(&args[3])?.values().to_vec(); - general_replace(as_list_array(&args[0])?, &args[1], &args[2], arr_n) + let array = &args[0]; + match array.data_type() { + DataType::List(_) => { + let list_array = array.as_list::(); + general_replace::(list_array, &args[1], &args[2], arr_n) + } + DataType::LargeList(_) => { + let list_array = array.as_list::(); + general_replace::(list_array, &args[1], &args[2], arr_n) + } + array_type => { + exec_err!("array_replace_n does not support type '{array_type:?}'.") + } + } } pub fn array_replace_all(args: &[ArrayRef]) -> Result { // replace all occurrences (up to "i64::MAX") let arr_n = vec![i64::MAX; args[0].len()]; - general_replace(as_list_array(&args[0])?, &args[1], &args[2], arr_n) + let array = &args[0]; + match array.data_type() { + DataType::List(_) => { + let list_array = array.as_list::(); + general_replace::(list_array, &args[1], &args[2], arr_n) + } + DataType::LargeList(_) => { + let list_array = array.as_list::(); + general_replace::(list_array, &args[1], &args[2], arr_n) + } + array_type => { + exec_err!("array_replace_all does not support type '{array_type:?}'.") + } + } } macro_rules! to_string { diff --git a/datafusion/sqllogictest/test_files/array.slt b/datafusion/sqllogictest/test_files/array.slt index ca33f08de06d..283f2d67b7a0 100644 --- a/datafusion/sqllogictest/test_files/array.slt +++ b/datafusion/sqllogictest/test_files/array.slt @@ -298,6 +298,17 @@ AS VALUES (make_array(10, 11, 12, 10, 11, 12, 10, 11, 12, 10), 10, 13, 10) ; +statement ok +CREATE TABLE large_arrays_with_repeating_elements +AS + SELECT + arrow_cast(column1, 'LargeList(Int64)') AS column1, + column2, + column3, + column4 + FROM arrays_with_repeating_elements +; + statement ok CREATE TABLE nested_arrays_with_repeating_elements AS VALUES @@ -307,6 +318,17 @@ AS VALUES (make_array([28, 29, 30], [31, 32, 33], [34, 35, 36], [28, 29, 30], [31, 32, 33], [34, 35, 36], [28, 29, 30], [31, 32, 33], [34, 35, 36], [28, 29, 30]), [28, 29, 30], [37, 38, 39], 10) ; +statement ok +CREATE TABLE large_nested_arrays_with_repeating_elements +AS + SELECT + arrow_cast(column1, 'LargeList(List(Int64))') AS column1, + column2, + column3, + column4 + FROM nested_arrays_with_repeating_elements +; + query error select [1, true, null] @@ -2010,6 +2032,14 @@ select ---- [1, 3, 3, 4] [1, 0, 4, 5, 4, 6, 7] [1, 2, 3] +query ??? +select + array_replace(arrow_cast(make_array(1, 2, 3, 4), 'LargeList(Int64)'), 2, 3), + array_replace(arrow_cast(make_array(1, 4, 4, 5, 4, 6, 7), 'LargeList(Int64)'), 4, 0), + array_replace(arrow_cast(make_array(1, 2, 3), 'LargeList(Int64)'), 4, 0); +---- +[1, 3, 3, 4] [1, 0, 4, 5, 4, 6, 7] [1, 2, 3] + # array_replace scalar function #2 (element is list) query ?? select @@ -2026,6 +2056,21 @@ select ---- [[1, 2, 3], [1, 1, 1], [5, 5, 5], [4, 5, 6], [7, 8, 9]] [[1, 3, 2], [3, 1, 4], [2, 3, 4], [5, 3, 1], [1, 3, 2]] +query ?? +select + array_replace( + arrow_cast(make_array([1, 2, 3], [4, 5, 6], [5, 5, 5], [4, 5, 6], [7, 8, 9]), 'LargeList(List(Int64))'), + [4, 5, 6], + [1, 1, 1] + ), + array_replace( + arrow_cast(make_array([1, 3, 2], [2, 3, 4], [2, 3, 4], [5, 3, 1], [1, 3, 2]), 'LargeList(List(Int64))'), + [2, 3, 4], + [3, 1, 4] + ); +---- +[[1, 2, 3], [1, 1, 1], [5, 5, 5], [4, 5, 6], [7, 8, 9]] [[1, 3, 2], [3, 1, 4], [2, 3, 4], [5, 3, 1], [1, 3, 2]] + # list_replace scalar function #3 (function alias `list_replace`) query ??? select list_replace( @@ -2035,6 +2080,14 @@ select list_replace( ---- [1, 3, 3, 4] [1, 0, 4, 5, 4, 6, 7] [1, 2, 3] +query ??? +select list_replace( + arrow_cast(make_array(1, 2, 3, 4), 'LargeList(Int64)'), 2, 3), + list_replace(arrow_cast(make_array(1, 4, 4, 5, 4, 6, 7), 'LargeList(Int64)'), 4, 0), + list_replace(arrow_cast(make_array(1, 2, 3), 'LargeList(Int64)'), 4, 0); +---- +[1, 3, 3, 4] [1, 0, 4, 5, 4, 6, 7] [1, 2, 3] + # array_replace scalar function with columns #1 query ? select array_replace(column1, column2, column3) from arrays_with_repeating_elements; @@ -2044,6 +2097,14 @@ select array_replace(column1, column2, column3) from arrays_with_repeating_eleme [10, 7, 7, 8, 7, 9, 7, 8, 7, 7] [13, 11, 12, 10, 11, 12, 10, 11, 12, 10] +query ? +select array_replace(column1, column2, column3) from large_arrays_with_repeating_elements; +---- +[1, 4, 1, 3, 2, 2, 1, 3, 2, 3] +[7, 4, 5, 5, 6, 5, 5, 5, 4, 4] +[10, 7, 7, 8, 7, 9, 7, 8, 7, 7] +[13, 11, 12, 10, 11, 12, 10, 11, 12, 10] + # array_replace scalar function with columns #2 (element is list) query ? select array_replace(column1, column2, column3) from nested_arrays_with_repeating_elements; @@ -2053,6 +2114,14 @@ select array_replace(column1, column2, column3) from nested_arrays_with_repeatin [[28, 29, 30], [19, 20, 21], [19, 20, 21], [22, 23, 24], [19, 20, 21], [25, 26, 27], [19, 20, 21], [22, 23, 24], [19, 20, 21], [19, 20, 21]] [[37, 38, 39], [31, 32, 33], [34, 35, 36], [28, 29, 30], [31, 32, 33], [34, 35, 36], [28, 29, 30], [31, 32, 33], [34, 35, 36], [28, 29, 30]] +query ? +select array_replace(column1, column2, column3) from large_nested_arrays_with_repeating_elements; +---- +[[1, 2, 3], [10, 11, 12], [1, 2, 3], [7, 8, 9], [4, 5, 6], [4, 5, 6], [1, 2, 3], [7, 8, 9], [4, 5, 6], [7, 8, 9]] +[[19, 20, 21], [10, 11, 12], [13, 14, 15], [13, 14, 15], [16, 17, 18], [13, 14, 15], [13, 14, 15], [13, 14, 15], [10, 11, 12], [10, 11, 12]] +[[28, 29, 30], [19, 20, 21], [19, 20, 21], [22, 23, 24], [19, 20, 21], [25, 26, 27], [19, 20, 21], [22, 23, 24], [19, 20, 21], [19, 20, 21]] +[[37, 38, 39], [31, 32, 33], [34, 35, 36], [28, 29, 30], [31, 32, 33], [34, 35, 36], [28, 29, 30], [31, 32, 33], [34, 35, 36], [28, 29, 30]] + # array_replace scalar function with columns and scalars #1 query ??? select @@ -2066,6 +2135,18 @@ from arrays_with_repeating_elements; [1, 2, 2, 4, 5, 4, 4, 10, 7, 10, 7, 8] [7, 7, 7, 8, 7, 9, 7, 8, 7, 7] [4, 7, 7, 8, 7, 9, 7, 8, 7, 7] [1, 2, 2, 4, 5, 4, 4, 7, 7, 13, 7, 8] [10, 11, 12, 10, 11, 12, 10, 11, 12, 10] [4, 11, 12, 10, 11, 12, 10, 11, 12, 10] +query ??? +select + array_replace(arrow_cast(make_array(1, 2, 2, 4, 5, 4, 4, 7, 7, 10, 7, 8), 'LargeList(Int64)'), column2, column3), + array_replace(column1, 1, column3), + array_replace(column1, column2, 4) +from large_arrays_with_repeating_elements; +---- +[1, 4, 2, 4, 5, 4, 4, 7, 7, 10, 7, 8] [4, 2, 1, 3, 2, 2, 1, 3, 2, 3] [1, 4, 1, 3, 2, 2, 1, 3, 2, 3] +[1, 2, 2, 7, 5, 4, 4, 7, 7, 10, 7, 8] [4, 4, 5, 5, 6, 5, 5, 5, 4, 4] [4, 4, 5, 5, 6, 5, 5, 5, 4, 4] +[1, 2, 2, 4, 5, 4, 4, 10, 7, 10, 7, 8] [7, 7, 7, 8, 7, 9, 7, 8, 7, 7] [4, 7, 7, 8, 7, 9, 7, 8, 7, 7] +[1, 2, 2, 4, 5, 4, 4, 7, 7, 13, 7, 8] [10, 11, 12, 10, 11, 12, 10, 11, 12, 10] [4, 11, 12, 10, 11, 12, 10, 11, 12, 10] + # array_replace scalar function with columns and scalars #2 (element is list) query ??? select @@ -2084,6 +2165,23 @@ from nested_arrays_with_repeating_elements; [[1, 2, 3], [4, 5, 6], [4, 5, 6], [10, 11, 12], [13, 14, 15], [10, 11, 12], [10, 11, 12], [28, 29, 30], [19, 20, 21], [28, 29, 30], [19, 20, 21], [22, 23, 24]] [[19, 20, 21], [19, 20, 21], [19, 20, 21], [22, 23, 24], [19, 20, 21], [25, 26, 27], [19, 20, 21], [22, 23, 24], [19, 20, 21], [19, 20, 21]] [[11, 12, 13], [19, 20, 21], [19, 20, 21], [22, 23, 24], [19, 20, 21], [25, 26, 27], [19, 20, 21], [22, 23, 24], [19, 20, 21], [19, 20, 21]] [[1, 2, 3], [4, 5, 6], [4, 5, 6], [10, 11, 12], [13, 14, 15], [10, 11, 12], [10, 11, 12], [19, 20, 21], [19, 20, 21], [37, 38, 39], [19, 20, 21], [22, 23, 24]] [[28, 29, 30], [31, 32, 33], [34, 35, 36], [28, 29, 30], [31, 32, 33], [34, 35, 36], [28, 29, 30], [31, 32, 33], [34, 35, 36], [28, 29, 30]] [[11, 12, 13], [31, 32, 33], [34, 35, 36], [28, 29, 30], [31, 32, 33], [34, 35, 36], [28, 29, 30], [31, 32, 33], [34, 35, 36], [28, 29, 30]] +query ??? +select + array_replace( + arrow_cast(make_array( + [1, 2, 3], [4, 5, 6], [4, 5, 6], [10, 11, 12], [13, 14, 15], [10, 11, 12], [10, 11, 12], [19, 20, 21], [19, 20, 21], [28, 29, 30], [19, 20, 21], [22, 23, 24]),'LargeList(List(Int64))'), + column2, + column3 + ), + array_replace(column1, make_array(1, 2, 3), column3), + array_replace(column1, column2, make_array(11, 12, 13)) +from large_nested_arrays_with_repeating_elements; +---- +[[1, 2, 3], [10, 11, 12], [4, 5, 6], [10, 11, 12], [13, 14, 15], [10, 11, 12], [10, 11, 12], [19, 20, 21], [19, 20, 21], [28, 29, 30], [19, 20, 21], [22, 23, 24]] [[10, 11, 12], [4, 5, 6], [1, 2, 3], [7, 8, 9], [4, 5, 6], [4, 5, 6], [1, 2, 3], [7, 8, 9], [4, 5, 6], [7, 8, 9]] [[1, 2, 3], [11, 12, 13], [1, 2, 3], [7, 8, 9], [4, 5, 6], [4, 5, 6], [1, 2, 3], [7, 8, 9], [4, 5, 6], [7, 8, 9]] +[[1, 2, 3], [4, 5, 6], [4, 5, 6], [19, 20, 21], [13, 14, 15], [10, 11, 12], [10, 11, 12], [19, 20, 21], [19, 20, 21], [28, 29, 30], [19, 20, 21], [22, 23, 24]] [[10, 11, 12], [10, 11, 12], [13, 14, 15], [13, 14, 15], [16, 17, 18], [13, 14, 15], [13, 14, 15], [13, 14, 15], [10, 11, 12], [10, 11, 12]] [[11, 12, 13], [10, 11, 12], [13, 14, 15], [13, 14, 15], [16, 17, 18], [13, 14, 15], [13, 14, 15], [13, 14, 15], [10, 11, 12], [10, 11, 12]] +[[1, 2, 3], [4, 5, 6], [4, 5, 6], [10, 11, 12], [13, 14, 15], [10, 11, 12], [10, 11, 12], [28, 29, 30], [19, 20, 21], [28, 29, 30], [19, 20, 21], [22, 23, 24]] [[19, 20, 21], [19, 20, 21], [19, 20, 21], [22, 23, 24], [19, 20, 21], [25, 26, 27], [19, 20, 21], [22, 23, 24], [19, 20, 21], [19, 20, 21]] [[11, 12, 13], [19, 20, 21], [19, 20, 21], [22, 23, 24], [19, 20, 21], [25, 26, 27], [19, 20, 21], [22, 23, 24], [19, 20, 21], [19, 20, 21]] +[[1, 2, 3], [4, 5, 6], [4, 5, 6], [10, 11, 12], [13, 14, 15], [10, 11, 12], [10, 11, 12], [19, 20, 21], [19, 20, 21], [37, 38, 39], [19, 20, 21], [22, 23, 24]] [[28, 29, 30], [31, 32, 33], [34, 35, 36], [28, 29, 30], [31, 32, 33], [34, 35, 36], [28, 29, 30], [31, 32, 33], [34, 35, 36], [28, 29, 30]] [[11, 12, 13], [31, 32, 33], [34, 35, 36], [28, 29, 30], [31, 32, 33], [34, 35, 36], [28, 29, 30], [31, 32, 33], [34, 35, 36], [28, 29, 30]] + ## array_replace_n (aliases: `list_replace_n`) # array_replace_n scalar function #1 @@ -2095,6 +2193,14 @@ select ---- [1, 3, 3, 4] [1, 0, 0, 5, 4, 6, 7] [1, 2, 3] +query ??? +select + array_replace_n(arrow_cast(make_array(1, 2, 3, 4), 'LargeList(Int64)'), 2, 3, 2), + array_replace_n(arrow_cast(make_array(1, 4, 4, 5, 4, 6, 7), 'LargeList(Int64)'), 4, 0, 2), + array_replace_n(arrow_cast(make_array(1, 2, 3), 'LargeList(Int64)'), 4, 0, 3); +---- +[1, 3, 3, 4] [1, 0, 0, 5, 4, 6, 7] [1, 2, 3] + # array_replace_n scalar function #2 (element is list) query ?? select @@ -2113,6 +2219,23 @@ select ---- [[1, 2, 3], [1, 1, 1], [5, 5, 5], [1, 1, 1], [7, 8, 9]] [[1, 3, 2], [3, 1, 4], [3, 1, 4], [5, 3, 1], [1, 3, 2]] +query ?? +select + array_replace_n( + arrow_cast(make_array([1, 2, 3], [4, 5, 6], [5, 5, 5], [4, 5, 6], [7, 8, 9]), 'LargeList(List(Int64))'), + [4, 5, 6], + [1, 1, 1], + 2 + ), + array_replace_n( + arrow_cast(make_array([1, 3, 2], [2, 3, 4], [2, 3, 4], [5, 3, 1], [1, 3, 2]), 'LargeList(List(Int64))'), + [2, 3, 4], + [3, 1, 4], + 2 + ); +---- +[[1, 2, 3], [1, 1, 1], [5, 5, 5], [1, 1, 1], [7, 8, 9]] [[1, 3, 2], [3, 1, 4], [3, 1, 4], [5, 3, 1], [1, 3, 2]] + # list_replace_n scalar function #3 (function alias `array_replace_n`) query ??? select @@ -2122,6 +2245,14 @@ select ---- [1, 3, 3, 4] [1, 0, 0, 5, 4, 6, 7] [1, 2, 3] +query ??? +select + list_replace_n(arrow_cast(make_array(1, 2, 3, 4), 'LargeList(Int64)'), 2, 3, 2), + list_replace_n(arrow_cast(make_array(1, 4, 4, 5, 4, 6, 7), 'LargeList(Int64)'), 4, 0, 2), + list_replace_n(arrow_cast(make_array(1, 2, 3), 'LargeList(Int64)'), 4, 0, 3); +---- +[1, 3, 3, 4] [1, 0, 0, 5, 4, 6, 7] [1, 2, 3] + # array_replace_n scalar function with columns #1 query ? select @@ -2133,6 +2264,16 @@ from arrays_with_repeating_elements; [10, 10, 10, 8, 10, 9, 10, 8, 7, 7] [13, 11, 12, 13, 11, 12, 13, 11, 12, 13] +query ? +select + array_replace_n(column1, column2, column3, column4) +from large_arrays_with_repeating_elements; +---- +[1, 4, 1, 3, 4, 4, 1, 3, 2, 3] +[7, 7, 5, 5, 6, 5, 5, 5, 4, 4] +[10, 10, 10, 8, 10, 9, 10, 8, 7, 7] +[13, 11, 12, 13, 11, 12, 13, 11, 12, 13] + # array_replace_n scalar function with columns #2 (element is list) query ? select @@ -2144,6 +2285,17 @@ from nested_arrays_with_repeating_elements; [[28, 29, 30], [28, 29, 30], [28, 29, 30], [22, 23, 24], [28, 29, 30], [25, 26, 27], [28, 29, 30], [22, 23, 24], [19, 20, 21], [19, 20, 21]] [[37, 38, 39], [31, 32, 33], [34, 35, 36], [37, 38, 39], [31, 32, 33], [34, 35, 36], [37, 38, 39], [31, 32, 33], [34, 35, 36], [37, 38, 39]] +query ? +select + array_replace_n(column1, column2, column3, column4) +from large_nested_arrays_with_repeating_elements; +---- +[[1, 2, 3], [10, 11, 12], [1, 2, 3], [7, 8, 9], [10, 11, 12], [10, 11, 12], [1, 2, 3], [7, 8, 9], [4, 5, 6], [7, 8, 9]] +[[19, 20, 21], [19, 20, 21], [13, 14, 15], [13, 14, 15], [16, 17, 18], [13, 14, 15], [13, 14, 15], [13, 14, 15], [10, 11, 12], [10, 11, 12]] +[[28, 29, 30], [28, 29, 30], [28, 29, 30], [22, 23, 24], [28, 29, 30], [25, 26, 27], [28, 29, 30], [22, 23, 24], [19, 20, 21], [19, 20, 21]] +[[37, 38, 39], [31, 32, 33], [34, 35, 36], [37, 38, 39], [31, 32, 33], [34, 35, 36], [37, 38, 39], [31, 32, 33], [34, 35, 36], [37, 38, 39]] + + # array_replace_n scalar function with columns and scalars #1 query ???? select @@ -2158,6 +2310,19 @@ from arrays_with_repeating_elements; [1, 2, 2, 4, 5, 4, 4, 10, 10, 10, 10, 8] [7, 7, 7, 8, 7, 9, 7, 8, 7, 7] [4, 4, 4, 8, 4, 9, 4, 8, 7, 7] [10, 10, 7, 8, 7, 9, 7, 8, 7, 7] [1, 2, 2, 4, 5, 4, 4, 7, 7, 13, 7, 8] [10, 11, 12, 10, 11, 12, 10, 11, 12, 10] [4, 11, 12, 4, 11, 12, 4, 11, 12, 4] [13, 11, 12, 13, 11, 12, 10, 11, 12, 10] +query ???? +select + array_replace_n(arrow_cast(make_array(1, 2, 2, 4, 5, 4, 4, 7, 7, 10, 7, 8), 'LargeList(Int64)'), column2, column3, column4), + array_replace_n(column1, 1, column3, column4), + array_replace_n(column1, column2, 4, column4), + array_replace_n(column1, column2, column3, 2) +from large_arrays_with_repeating_elements; +---- +[1, 4, 4, 4, 5, 4, 4, 7, 7, 10, 7, 8] [4, 2, 4, 3, 2, 2, 4, 3, 2, 3] [1, 4, 1, 3, 4, 4, 1, 3, 2, 3] [1, 4, 1, 3, 4, 2, 1, 3, 2, 3] +[1, 2, 2, 7, 5, 7, 4, 7, 7, 10, 7, 8] [4, 4, 5, 5, 6, 5, 5, 5, 4, 4] [4, 4, 5, 5, 6, 5, 5, 5, 4, 4] [7, 7, 5, 5, 6, 5, 5, 5, 4, 4] +[1, 2, 2, 4, 5, 4, 4, 10, 10, 10, 10, 8] [7, 7, 7, 8, 7, 9, 7, 8, 7, 7] [4, 4, 4, 8, 4, 9, 4, 8, 7, 7] [10, 10, 7, 8, 7, 9, 7, 8, 7, 7] +[1, 2, 2, 4, 5, 4, 4, 7, 7, 13, 7, 8] [10, 11, 12, 10, 11, 12, 10, 11, 12, 10] [4, 11, 12, 4, 11, 12, 4, 11, 12, 4] [13, 11, 12, 13, 11, 12, 10, 11, 12, 10] + # array_replace_n scalar function with columns and scalars #2 (element is list) query ???? select @@ -2178,6 +2343,25 @@ from nested_arrays_with_repeating_elements; [[7, 8, 9], [2, 1, 3], [1, 5, 6], [10, 11, 12], [2, 1, 3], [7, 8, 9], [4, 5, 6]] [[19, 20, 21], [19, 20, 21], [19, 20, 21], [22, 23, 24], [19, 20, 21], [25, 26, 27], [19, 20, 21], [22, 23, 24], [19, 20, 21], [19, 20, 21]] [[11, 12, 13], [11, 12, 13], [11, 12, 13], [22, 23, 24], [11, 12, 13], [25, 26, 27], [11, 12, 13], [22, 23, 24], [19, 20, 21], [19, 20, 21]] [[28, 29, 30], [28, 29, 30], [19, 20, 21], [22, 23, 24], [19, 20, 21], [25, 26, 27], [19, 20, 21], [22, 23, 24], [19, 20, 21], [19, 20, 21]] [[7, 8, 9], [2, 1, 3], [1, 5, 6], [10, 11, 12], [2, 1, 3], [7, 8, 9], [4, 5, 6]] [[28, 29, 30], [31, 32, 33], [34, 35, 36], [28, 29, 30], [31, 32, 33], [34, 35, 36], [28, 29, 30], [31, 32, 33], [34, 35, 36], [28, 29, 30]] [[11, 12, 13], [31, 32, 33], [34, 35, 36], [11, 12, 13], [31, 32, 33], [34, 35, 36], [11, 12, 13], [31, 32, 33], [34, 35, 36], [11, 12, 13]] [[37, 38, 39], [31, 32, 33], [34, 35, 36], [37, 38, 39], [31, 32, 33], [34, 35, 36], [28, 29, 30], [31, 32, 33], [34, 35, 36], [28, 29, 30]] +query ???? +select + array_replace_n( + arrow_cast(make_array( + [7, 8, 9], [2, 1, 3], [1, 5, 6], [10, 11, 12], [2, 1, 3], [7, 8, 9], [4, 5, 6]), 'LargeList(List(Int64))'), + column2, + column3, + column4 + ), + array_replace_n(column1, make_array(1, 2, 3), column3, column4), + array_replace_n(column1, column2, make_array(11, 12, 13), column4), + array_replace_n(column1, column2, column3, 2) +from large_nested_arrays_with_repeating_elements; +---- +[[7, 8, 9], [2, 1, 3], [1, 5, 6], [10, 11, 12], [2, 1, 3], [7, 8, 9], [10, 11, 12]] [[10, 11, 12], [4, 5, 6], [10, 11, 12], [7, 8, 9], [4, 5, 6], [4, 5, 6], [10, 11, 12], [7, 8, 9], [4, 5, 6], [7, 8, 9]] [[1, 2, 3], [11, 12, 13], [1, 2, 3], [7, 8, 9], [11, 12, 13], [11, 12, 13], [1, 2, 3], [7, 8, 9], [4, 5, 6], [7, 8, 9]] [[1, 2, 3], [10, 11, 12], [1, 2, 3], [7, 8, 9], [10, 11, 12], [4, 5, 6], [1, 2, 3], [7, 8, 9], [4, 5, 6], [7, 8, 9]] +[[7, 8, 9], [2, 1, 3], [1, 5, 6], [19, 20, 21], [2, 1, 3], [7, 8, 9], [4, 5, 6]] [[10, 11, 12], [10, 11, 12], [13, 14, 15], [13, 14, 15], [16, 17, 18], [13, 14, 15], [13, 14, 15], [13, 14, 15], [10, 11, 12], [10, 11, 12]] [[11, 12, 13], [11, 12, 13], [13, 14, 15], [13, 14, 15], [16, 17, 18], [13, 14, 15], [13, 14, 15], [13, 14, 15], [10, 11, 12], [10, 11, 12]] [[19, 20, 21], [19, 20, 21], [13, 14, 15], [13, 14, 15], [16, 17, 18], [13, 14, 15], [13, 14, 15], [13, 14, 15], [10, 11, 12], [10, 11, 12]] +[[7, 8, 9], [2, 1, 3], [1, 5, 6], [10, 11, 12], [2, 1, 3], [7, 8, 9], [4, 5, 6]] [[19, 20, 21], [19, 20, 21], [19, 20, 21], [22, 23, 24], [19, 20, 21], [25, 26, 27], [19, 20, 21], [22, 23, 24], [19, 20, 21], [19, 20, 21]] [[11, 12, 13], [11, 12, 13], [11, 12, 13], [22, 23, 24], [11, 12, 13], [25, 26, 27], [11, 12, 13], [22, 23, 24], [19, 20, 21], [19, 20, 21]] [[28, 29, 30], [28, 29, 30], [19, 20, 21], [22, 23, 24], [19, 20, 21], [25, 26, 27], [19, 20, 21], [22, 23, 24], [19, 20, 21], [19, 20, 21]] +[[7, 8, 9], [2, 1, 3], [1, 5, 6], [10, 11, 12], [2, 1, 3], [7, 8, 9], [4, 5, 6]] [[28, 29, 30], [31, 32, 33], [34, 35, 36], [28, 29, 30], [31, 32, 33], [34, 35, 36], [28, 29, 30], [31, 32, 33], [34, 35, 36], [28, 29, 30]] [[11, 12, 13], [31, 32, 33], [34, 35, 36], [11, 12, 13], [31, 32, 33], [34, 35, 36], [11, 12, 13], [31, 32, 33], [34, 35, 36], [11, 12, 13]] [[37, 38, 39], [31, 32, 33], [34, 35, 36], [37, 38, 39], [31, 32, 33], [34, 35, 36], [28, 29, 30], [31, 32, 33], [34, 35, 36], [28, 29, 30]] + ## array_replace_all (aliases: `list_replace_all`) # array_replace_all scalar function #1 @@ -2189,6 +2373,14 @@ select ---- [1, 3, 3, 4] [1, 0, 0, 5, 0, 6, 7] [1, 2, 3] +query ??? +select + array_replace_all(arrow_cast(make_array(1, 2, 3, 4), 'LargeList(Int64)'), 2, 3), + array_replace_all(arrow_cast(make_array(1, 4, 4, 5, 4, 6, 7), 'LargeList(Int64)'), 4, 0), + array_replace_all(arrow_cast(make_array(1, 2, 3), 'LargeList(Int64)'), 4, 0); +---- +[1, 3, 3, 4] [1, 0, 0, 5, 0, 6, 7] [1, 2, 3] + # array_replace_all scalar function #2 (element is list) query ?? select @@ -2205,6 +2397,21 @@ select ---- [[1, 2, 3], [1, 1, 1], [5, 5, 5], [1, 1, 1], [7, 8, 9]] [[1, 3, 2], [3, 1, 4], [3, 1, 4], [5, 3, 1], [1, 3, 2]] +query ?? +select + array_replace_all( + arrow_cast(make_array([1, 2, 3], [4, 5, 6], [5, 5, 5], [4, 5, 6], [7, 8, 9]), 'LargeList(List(Int64))'), + [4, 5, 6], + [1, 1, 1] + ), + array_replace_all( + arrow_cast(make_array([1, 3, 2], [2, 3, 4], [2, 3, 4], [5, 3, 1], [1, 3, 2]), 'LargeList(List(Int64))'), + [2, 3, 4], + [3, 1, 4] + ); +---- +[[1, 2, 3], [1, 1, 1], [5, 5, 5], [1, 1, 1], [7, 8, 9]] [[1, 3, 2], [3, 1, 4], [3, 1, 4], [5, 3, 1], [1, 3, 2]] + # list_replace_all scalar function #3 (function alias `array_replace_all`) query ??? select @@ -2214,6 +2421,14 @@ select ---- [1, 3, 3, 4] [1, 0, 0, 5, 0, 6, 7] [1, 2, 3] +query ??? +select + list_replace_all(arrow_cast(make_array(1, 2, 3, 4), 'LargeList(Int64)'), 2, 3), + list_replace_all(arrow_cast(make_array(1, 4, 4, 5, 4, 6, 7), 'LargeList(Int64)'), 4, 0), + list_replace_all(arrow_cast(make_array(1, 2, 3), 'LargeList(Int64)'), 4, 0); +---- +[1, 3, 3, 4] [1, 0, 0, 5, 0, 6, 7] [1, 2, 3] + # array_replace_all scalar function with columns #1 query ? select @@ -2225,6 +2440,16 @@ from arrays_with_repeating_elements; [10, 10, 10, 8, 10, 9, 10, 8, 10, 10] [13, 11, 12, 13, 11, 12, 13, 11, 12, 13] +query ? +select + array_replace_all(column1, column2, column3) +from large_arrays_with_repeating_elements; +---- +[1, 4, 1, 3, 4, 4, 1, 3, 4, 3] +[7, 7, 5, 5, 6, 5, 5, 5, 7, 7] +[10, 10, 10, 8, 10, 9, 10, 8, 10, 10] +[13, 11, 12, 13, 11, 12, 13, 11, 12, 13] + # array_replace_all scalar function with columns #2 (element is list) query ? select @@ -2236,6 +2461,16 @@ from nested_arrays_with_repeating_elements; [[28, 29, 30], [28, 29, 30], [28, 29, 30], [22, 23, 24], [28, 29, 30], [25, 26, 27], [28, 29, 30], [22, 23, 24], [28, 29, 30], [28, 29, 30]] [[37, 38, 39], [31, 32, 33], [34, 35, 36], [37, 38, 39], [31, 32, 33], [34, 35, 36], [37, 38, 39], [31, 32, 33], [34, 35, 36], [37, 38, 39]] +query ? +select + array_replace_all(column1, column2, column3) +from large_nested_arrays_with_repeating_elements; +---- +[[1, 2, 3], [10, 11, 12], [1, 2, 3], [7, 8, 9], [10, 11, 12], [10, 11, 12], [1, 2, 3], [7, 8, 9], [10, 11, 12], [7, 8, 9]] +[[19, 20, 21], [19, 20, 21], [13, 14, 15], [13, 14, 15], [16, 17, 18], [13, 14, 15], [13, 14, 15], [13, 14, 15], [19, 20, 21], [19, 20, 21]] +[[28, 29, 30], [28, 29, 30], [28, 29, 30], [22, 23, 24], [28, 29, 30], [25, 26, 27], [28, 29, 30], [22, 23, 24], [28, 29, 30], [28, 29, 30]] +[[37, 38, 39], [31, 32, 33], [34, 35, 36], [37, 38, 39], [31, 32, 33], [34, 35, 36], [37, 38, 39], [31, 32, 33], [34, 35, 36], [37, 38, 39]] + # array_replace_all scalar function with columns and scalars #1 query ??? select @@ -2249,6 +2484,18 @@ from arrays_with_repeating_elements; [1, 2, 2, 4, 5, 4, 4, 10, 10, 10, 10, 8] [7, 7, 7, 8, 7, 9, 7, 8, 7, 7] [4, 4, 4, 8, 4, 9, 4, 8, 4, 4] [1, 2, 2, 4, 5, 4, 4, 7, 7, 13, 7, 8] [10, 11, 12, 10, 11, 12, 10, 11, 12, 10] [4, 11, 12, 4, 11, 12, 4, 11, 12, 4] +query ??? +select + array_replace_all(arrow_cast(make_array(1, 2, 2, 4, 5, 4, 4, 7, 7, 10, 7, 8), 'LargeList(Int64)'), column2, column3), + array_replace_all(column1, 1, column3), + array_replace_all(column1, column2, 4) +from large_arrays_with_repeating_elements; +---- +[1, 4, 4, 4, 5, 4, 4, 7, 7, 10, 7, 8] [4, 2, 4, 3, 2, 2, 4, 3, 2, 3] [1, 4, 1, 3, 4, 4, 1, 3, 4, 3] +[1, 2, 2, 7, 5, 7, 7, 7, 7, 10, 7, 8] [4, 4, 5, 5, 6, 5, 5, 5, 4, 4] [4, 4, 5, 5, 6, 5, 5, 5, 4, 4] +[1, 2, 2, 4, 5, 4, 4, 10, 10, 10, 10, 8] [7, 7, 7, 8, 7, 9, 7, 8, 7, 7] [4, 4, 4, 8, 4, 9, 4, 8, 4, 4] +[1, 2, 2, 4, 5, 4, 4, 7, 7, 13, 7, 8] [10, 11, 12, 10, 11, 12, 10, 11, 12, 10] [4, 11, 12, 4, 11, 12, 4, 11, 12, 4] + # array_replace_all scalar function with columns and scalars #2 (element is list) query ??? select @@ -2266,6 +2513,22 @@ from nested_arrays_with_repeating_elements; [[1, 2, 3], [4, 5, 6], [4, 5, 6], [10, 11, 12], [13, 14, 15], [10, 11, 12], [10, 11, 12], [28, 29, 30], [28, 29, 30], [28, 29, 30], [28, 29, 30], [22, 23, 24]] [[19, 20, 21], [19, 20, 21], [19, 20, 21], [22, 23, 24], [19, 20, 21], [25, 26, 27], [19, 20, 21], [22, 23, 24], [19, 20, 21], [19, 20, 21]] [[11, 12, 13], [11, 12, 13], [11, 12, 13], [22, 23, 24], [11, 12, 13], [25, 26, 27], [11, 12, 13], [22, 23, 24], [11, 12, 13], [11, 12, 13]] [[1, 2, 3], [4, 5, 6], [4, 5, 6], [10, 11, 12], [13, 14, 15], [10, 11, 12], [10, 11, 12], [19, 20, 21], [19, 20, 21], [37, 38, 39], [19, 20, 21], [22, 23, 24]] [[28, 29, 30], [31, 32, 33], [34, 35, 36], [28, 29, 30], [31, 32, 33], [34, 35, 36], [28, 29, 30], [31, 32, 33], [34, 35, 36], [28, 29, 30]] [[11, 12, 13], [31, 32, 33], [34, 35, 36], [11, 12, 13], [31, 32, 33], [34, 35, 36], [11, 12, 13], [31, 32, 33], [34, 35, 36], [11, 12, 13]] +query ??? +select + array_replace_all( + arrow_cast(make_array([1, 2, 3], [4, 5, 6], [4, 5, 6], [10, 11, 12], [13, 14, 15], [10, 11, 12], [10, 11, 12], [19, 20, 21], [19, 20, 21], [28, 29, 30], [19, 20, 21], [22, 23, 24]), 'LargeList(List(Int64))'), + column2, + column3 + ), + array_replace_all(column1, make_array(1, 2, 3), column3), + array_replace_all(column1, column2, make_array(11, 12, 13)) +from nested_arrays_with_repeating_elements; +---- +[[1, 2, 3], [10, 11, 12], [10, 11, 12], [10, 11, 12], [13, 14, 15], [10, 11, 12], [10, 11, 12], [19, 20, 21], [19, 20, 21], [28, 29, 30], [19, 20, 21], [22, 23, 24]] [[10, 11, 12], [4, 5, 6], [10, 11, 12], [7, 8, 9], [4, 5, 6], [4, 5, 6], [10, 11, 12], [7, 8, 9], [4, 5, 6], [7, 8, 9]] [[1, 2, 3], [11, 12, 13], [1, 2, 3], [7, 8, 9], [11, 12, 13], [11, 12, 13], [1, 2, 3], [7, 8, 9], [11, 12, 13], [7, 8, 9]] +[[1, 2, 3], [4, 5, 6], [4, 5, 6], [19, 20, 21], [13, 14, 15], [19, 20, 21], [19, 20, 21], [19, 20, 21], [19, 20, 21], [28, 29, 30], [19, 20, 21], [22, 23, 24]] [[10, 11, 12], [10, 11, 12], [13, 14, 15], [13, 14, 15], [16, 17, 18], [13, 14, 15], [13, 14, 15], [13, 14, 15], [10, 11, 12], [10, 11, 12]] [[11, 12, 13], [11, 12, 13], [13, 14, 15], [13, 14, 15], [16, 17, 18], [13, 14, 15], [13, 14, 15], [13, 14, 15], [11, 12, 13], [11, 12, 13]] +[[1, 2, 3], [4, 5, 6], [4, 5, 6], [10, 11, 12], [13, 14, 15], [10, 11, 12], [10, 11, 12], [28, 29, 30], [28, 29, 30], [28, 29, 30], [28, 29, 30], [22, 23, 24]] [[19, 20, 21], [19, 20, 21], [19, 20, 21], [22, 23, 24], [19, 20, 21], [25, 26, 27], [19, 20, 21], [22, 23, 24], [19, 20, 21], [19, 20, 21]] [[11, 12, 13], [11, 12, 13], [11, 12, 13], [22, 23, 24], [11, 12, 13], [25, 26, 27], [11, 12, 13], [22, 23, 24], [11, 12, 13], [11, 12, 13]] +[[1, 2, 3], [4, 5, 6], [4, 5, 6], [10, 11, 12], [13, 14, 15], [10, 11, 12], [10, 11, 12], [19, 20, 21], [19, 20, 21], [37, 38, 39], [19, 20, 21], [22, 23, 24]] [[28, 29, 30], [31, 32, 33], [34, 35, 36], [28, 29, 30], [31, 32, 33], [34, 35, 36], [28, 29, 30], [31, 32, 33], [34, 35, 36], [28, 29, 30]] [[11, 12, 13], [31, 32, 33], [34, 35, 36], [11, 12, 13], [31, 32, 33], [34, 35, 36], [11, 12, 13], [31, 32, 33], [34, 35, 36], [11, 12, 13]] + # array_replace with null handling statement ok @@ -3870,8 +4133,14 @@ drop table arrays_range; statement ok drop table arrays_with_repeating_elements; +statement ok +drop table large_arrays_with_repeating_elements; + statement ok drop table nested_arrays_with_repeating_elements; +statement ok +drop table large_nested_arrays_with_repeating_elements; + statement ok drop table flatten_table; From 0e62fa4df924f8657e43a97ca7aa8c6ca48bc08f Mon Sep 17 00:00:00 2001 From: Tomoaki Kawada Date: Fri, 22 Dec 2023 21:47:14 +0900 Subject: [PATCH 482/572] Rename `ParamValues::{LIST -> List,MAP -> Map}` (#8611) * Rename `ParamValues::{LIST -> List,MAP -> Map}` * Reformat the doc comments of `ParamValues::*` --- datafusion/common/src/param_value.rs | 22 +++++++++++----------- 1 file changed, 11 insertions(+), 11 deletions(-) diff --git a/datafusion/common/src/param_value.rs b/datafusion/common/src/param_value.rs index 253c312b66d5..1b6195c0d0bc 100644 --- a/datafusion/common/src/param_value.rs +++ b/datafusion/common/src/param_value.rs @@ -23,17 +23,17 @@ use std::collections::HashMap; /// The parameter value corresponding to the placeholder #[derive(Debug, Clone)] pub enum ParamValues { - /// for positional query parameters, like select * from test where a > $1 and b = $2 - LIST(Vec), - /// for named query parameters, like select * from test where a > $foo and b = $goo - MAP(HashMap), + /// For positional query parameters, like `SELECT * FROM test WHERE a > $1 AND b = $2` + List(Vec), + /// For named query parameters, like `SELECT * FROM test WHERE a > $foo AND b = $goo` + Map(HashMap), } impl ParamValues { /// Verify parameter list length and type pub fn verify(&self, expect: &Vec) -> Result<()> { match self { - ParamValues::LIST(list) => { + ParamValues::List(list) => { // Verify if the number of params matches the number of values if expect.len() != list.len() { return _plan_err!( @@ -57,7 +57,7 @@ impl ParamValues { } Ok(()) } - ParamValues::MAP(_) => { + ParamValues::Map(_) => { // If it is a named query, variables can be reused, // but the lengths are not necessarily equal Ok(()) @@ -71,7 +71,7 @@ impl ParamValues { data_type: &Option, ) -> Result { match self { - ParamValues::LIST(list) => { + ParamValues::List(list) => { if id.is_empty() || id == "$0" { return _plan_err!("Empty placeholder id"); } @@ -97,7 +97,7 @@ impl ParamValues { } Ok(value.clone()) } - ParamValues::MAP(map) => { + ParamValues::Map(map) => { // convert name (in format $a, $b, ..) to mapped values (a, b, ..) let name = &id[1..]; // value at the name position in param_values should be the value for the placeholder @@ -122,7 +122,7 @@ impl ParamValues { impl From> for ParamValues { fn from(value: Vec) -> Self { - Self::LIST(value) + Self::List(value) } } @@ -133,7 +133,7 @@ where fn from(value: Vec<(K, ScalarValue)>) -> Self { let value: HashMap = value.into_iter().map(|(k, v)| (k.into(), v)).collect(); - Self::MAP(value) + Self::Map(value) } } @@ -144,6 +144,6 @@ where fn from(value: HashMap) -> Self { let value: HashMap = value.into_iter().map(|(k, v)| (k.into(), v)).collect(); - Self::MAP(value) + Self::Map(value) } } From 26a488d6ae0f45b33d2566b8b97d4f82a2e80fa3 Mon Sep 17 00:00:00 2001 From: Asura7969 <1402357969@qq.com> Date: Sat, 23 Dec 2023 02:18:53 +0800 Subject: [PATCH 483/572] Support binary temporal coercion for Date64 and Timestamp types --- datafusion/expr/src/type_coercion/binary.rs | 6 ++++++ datafusion/sqllogictest/test_files/timestamps.slt | 9 +++++++++ 2 files changed, 15 insertions(+) diff --git a/datafusion/expr/src/type_coercion/binary.rs b/datafusion/expr/src/type_coercion/binary.rs index dd9449198796..1b62c1bc05c1 100644 --- a/datafusion/expr/src/type_coercion/binary.rs +++ b/datafusion/expr/src/type_coercion/binary.rs @@ -785,6 +785,12 @@ fn temporal_coercion(lhs_type: &DataType, rhs_type: &DataType) -> Option Some(Interval(MonthDayNano)), (Date64, Date32) | (Date32, Date64) => Some(Date64), + (Timestamp(_, None), Date64) | (Date64, Timestamp(_, None)) => { + Some(Timestamp(Nanosecond, None)) + } + (Timestamp(_, _tz), Date64) | (Date64, Timestamp(_, _tz)) => { + Some(Timestamp(Nanosecond, None)) + } (Timestamp(_, None), Date32) | (Date32, Timestamp(_, None)) => { Some(Timestamp(Nanosecond, None)) } diff --git a/datafusion/sqllogictest/test_files/timestamps.slt b/datafusion/sqllogictest/test_files/timestamps.slt index f956d59b1da0..2b3b4bf2e45b 100644 --- a/datafusion/sqllogictest/test_files/timestamps.slt +++ b/datafusion/sqllogictest/test_files/timestamps.slt @@ -1888,3 +1888,12 @@ true true true true true true #SELECT to_timestamp(-62125747200), to_timestamp(1926632005177), -62125747200::timestamp, 1926632005177::timestamp, cast(-62125747200 as timestamp), cast(1926632005177 as timestamp) #---- #0001-04-25T00:00:00 +63022-07-16T12:59:37 0001-04-25T00:00:00 +63022-07-16T12:59:37 0001-04-25T00:00:00 +63022-07-16T12:59:37 + +########## +## Test binary temporal coercion for Date and Timestamp +########## + +query B +select arrow_cast(now(), 'Date64') < arrow_cast('2022-02-02 02:02:02', 'Timestamp(Nanosecond, None)'); +---- +false From ba46434f839d612be01ee0d00e0d826475ce5f10 Mon Sep 17 00:00:00 2001 From: Asura7969 <1402357969@qq.com> Date: Sat, 23 Dec 2023 03:28:47 +0800 Subject: [PATCH 484/572] Add new configuration item `listing_table_ignore_subdirectory` (#8565) * init * test * add config * rename * doc * fix doc * add sqllogictests & rename * fmt & fix test * clippy * test read partition table * simplify testing * simplify testing --- datafusion/common/src/config.rs | 5 +++ .../core/src/datasource/listing/helpers.rs | 4 +- datafusion/core/src/datasource/listing/url.rs | 26 ++++++++++--- .../core/src/execution/context/parquet.rs | 9 ++++- .../test_files/information_schema.slt | 2 + .../sqllogictest/test_files/parquet.slt | 38 ++++++++++++++++++- docs/source/user-guide/configs.md | 1 + 7 files changed, 75 insertions(+), 10 deletions(-) diff --git a/datafusion/common/src/config.rs b/datafusion/common/src/config.rs index 03fb5ea320a0..dedce74ff40d 100644 --- a/datafusion/common/src/config.rs +++ b/datafusion/common/src/config.rs @@ -273,6 +273,11 @@ config_namespace! { /// memory consumption pub max_buffered_batches_per_output_file: usize, default = 2 + /// When scanning file paths, whether to ignore subdirectory files, + /// ignored by default (true), when reading a partitioned table, + /// `listing_table_ignore_subdirectory` is always equal to false, even if set to true + pub listing_table_ignore_subdirectory: bool, default = true + } } diff --git a/datafusion/core/src/datasource/listing/helpers.rs b/datafusion/core/src/datasource/listing/helpers.rs index be74afa1f4d6..68de55e1a410 100644 --- a/datafusion/core/src/datasource/listing/helpers.rs +++ b/datafusion/core/src/datasource/listing/helpers.rs @@ -375,10 +375,10 @@ pub async fn pruned_partition_list<'a>( store.list(Some(&partition.path)).try_collect().await? } }; - let files = files.into_iter().filter(move |o| { let extension_match = o.location.as_ref().ends_with(file_extension); - let glob_match = table_path.contains(&o.location); + // here need to scan subdirectories(`listing_table_ignore_subdirectory` = false) + let glob_match = table_path.contains(&o.location, false); extension_match && glob_match }); diff --git a/datafusion/core/src/datasource/listing/url.rs b/datafusion/core/src/datasource/listing/url.rs index 3ca7864f7f9e..766dee7de901 100644 --- a/datafusion/core/src/datasource/listing/url.rs +++ b/datafusion/core/src/datasource/listing/url.rs @@ -20,6 +20,7 @@ use std::fs; use crate::datasource::object_store::ObjectStoreUrl; use crate::execution::context::SessionState; use datafusion_common::{DataFusionError, Result}; +use datafusion_optimizer::OptimizerConfig; use futures::stream::BoxStream; use futures::{StreamExt, TryStreamExt}; use glob::Pattern; @@ -184,14 +185,27 @@ impl ListingTableUrl { } /// Returns `true` if `path` matches this [`ListingTableUrl`] - pub fn contains(&self, path: &Path) -> bool { + pub fn contains(&self, path: &Path, ignore_subdirectory: bool) -> bool { match self.strip_prefix(path) { Some(mut segments) => match &self.glob { Some(glob) => { - let stripped = segments.join("/"); - glob.matches(&stripped) + if ignore_subdirectory { + segments + .next() + .map_or(false, |file_name| glob.matches(file_name)) + } else { + let stripped = segments.join("/"); + glob.matches(&stripped) + } + } + None => { + if ignore_subdirectory { + let has_subdirectory = segments.collect::>().len() > 1; + !has_subdirectory + } else { + true + } } - None => true, }, None => false, } @@ -223,6 +237,8 @@ impl ListingTableUrl { store: &'a dyn ObjectStore, file_extension: &'a str, ) -> Result>> { + let exec_options = &ctx.options().execution; + let ignore_subdirectory = exec_options.listing_table_ignore_subdirectory; // If the prefix is a file, use a head request, otherwise list let list = match self.is_collection() { true => match ctx.runtime_env().cache_manager.get_list_files_cache() { @@ -246,7 +262,7 @@ impl ListingTableUrl { .try_filter(move |meta| { let path = &meta.location; let extension_match = path.as_ref().ends_with(file_extension); - let glob_match = self.contains(path); + let glob_match = self.contains(path, ignore_subdirectory); futures::future::ready(extension_match && glob_match) }) .map_err(DataFusionError::ObjectStore) diff --git a/datafusion/core/src/execution/context/parquet.rs b/datafusion/core/src/execution/context/parquet.rs index 5d649d3e6df8..7825d9b88297 100644 --- a/datafusion/core/src/execution/context/parquet.rs +++ b/datafusion/core/src/execution/context/parquet.rs @@ -80,6 +80,7 @@ mod tests { use crate::dataframe::DataFrameWriteOptions; use crate::parquet::basic::Compression; use crate::test_util::parquet_test_data; + use datafusion_execution::config::SessionConfig; use tempfile::tempdir; use super::*; @@ -103,8 +104,12 @@ mod tests { #[tokio::test] async fn read_with_glob_path_issue_2465() -> Result<()> { - let ctx = SessionContext::new(); - + let config = + SessionConfig::from_string_hash_map(std::collections::HashMap::from([( + "datafusion.execution.listing_table_ignore_subdirectory".to_owned(), + "false".to_owned(), + )]))?; + let ctx = SessionContext::new_with_config(config); let df = ctx .read_parquet( // it was reported that when a path contains // (two consecutive separator) no files were found diff --git a/datafusion/sqllogictest/test_files/information_schema.slt b/datafusion/sqllogictest/test_files/information_schema.slt index 5c6bf6e2dac1..36876beb1447 100644 --- a/datafusion/sqllogictest/test_files/information_schema.slt +++ b/datafusion/sqllogictest/test_files/information_schema.slt @@ -150,6 +150,7 @@ datafusion.execution.aggregate.scalar_update_factor 10 datafusion.execution.batch_size 8192 datafusion.execution.coalesce_batches true datafusion.execution.collect_statistics false +datafusion.execution.listing_table_ignore_subdirectory true datafusion.execution.max_buffered_batches_per_output_file 2 datafusion.execution.meta_fetch_concurrency 32 datafusion.execution.minimum_parallel_output_files 4 @@ -224,6 +225,7 @@ datafusion.execution.aggregate.scalar_update_factor 10 Specifies the threshold f datafusion.execution.batch_size 8192 Default batch size while creating new batches, it's especially useful for buffer-in-memory batches since creating tiny batches would result in too much metadata memory consumption datafusion.execution.coalesce_batches true When set to true, record batches will be examined between each operator and small batches will be coalesced into larger batches. This is helpful when there are highly selective filters or joins that could produce tiny output batches. The target batch size is determined by the configuration setting datafusion.execution.collect_statistics false Should DataFusion collect statistics after listing files +datafusion.execution.listing_table_ignore_subdirectory true When scanning file paths, whether to ignore subdirectory files, ignored by default (true), when reading a partitioned table, `listing_table_ignore_subdirectory` is always equal to false, even if set to true datafusion.execution.max_buffered_batches_per_output_file 2 This is the maximum number of RecordBatches buffered for each output file being worked. Higher values can potentially give faster write performance at the cost of higher peak memory consumption datafusion.execution.meta_fetch_concurrency 32 Number of files to read in parallel when inferring schema and statistics datafusion.execution.minimum_parallel_output_files 4 Guarantees a minimum level of output files running in parallel. RecordBatches will be distributed in round robin fashion to each parallel writer. Each writer is closed and a new file opened once soft_max_rows_per_output_file is reached. diff --git a/datafusion/sqllogictest/test_files/parquet.slt b/datafusion/sqllogictest/test_files/parquet.slt index 6c3bd687700a..0f26c14f0017 100644 --- a/datafusion/sqllogictest/test_files/parquet.slt +++ b/datafusion/sqllogictest/test_files/parquet.slt @@ -276,6 +276,39 @@ LIMIT 10; 0 2014-08-27T14:00:00Z Timestamp(Millisecond, Some("UTC")) 0 2014-08-27T14:00:00Z Timestamp(Millisecond, Some("UTC")) +# Test config listing_table_ignore_subdirectory: + +query ITID +COPY (SELECT * FROM src_table WHERE int_col > 6 LIMIT 3) +TO 'test_files/scratch/parquet/test_table/subdir/3.parquet' +(FORMAT PARQUET, SINGLE_FILE_OUTPUT true); +---- +3 + +statement ok +CREATE EXTERNAL TABLE listing_table +STORED AS PARQUET +WITH HEADER ROW +LOCATION 'test_files/scratch/parquet/test_table/*.parquet'; + +statement ok +set datafusion.execution.listing_table_ignore_subdirectory = true; + +# scan file: 0.parquet 1.parquet 2.parquet +query I +select count(*) from listing_table; +---- +9 + +statement ok +set datafusion.execution.listing_table_ignore_subdirectory = false; + +# scan file: 0.parquet 1.parquet 2.parquet 3.parquet +query I +select count(*) from listing_table; +---- +12 + # Clean up statement ok DROP TABLE timestamp_with_tz; @@ -303,7 +336,6 @@ NULL statement ok DROP TABLE single_nan; - statement ok CREATE EXTERNAL TABLE list_columns STORED AS PARQUET @@ -319,3 +351,7 @@ SELECT int64_list, utf8_list FROM list_columns statement ok DROP TABLE list_columns; + +# Clean up +statement ok +DROP TABLE listing_table; diff --git a/docs/source/user-guide/configs.md b/docs/source/user-guide/configs.md index 6fb5cc4ca870..1f7fa7760b94 100644 --- a/docs/source/user-guide/configs.md +++ b/docs/source/user-guide/configs.md @@ -82,6 +82,7 @@ Environment variables are read during `SessionConfig` initialisation so they mus | datafusion.execution.minimum_parallel_output_files | 4 | Guarantees a minimum level of output files running in parallel. RecordBatches will be distributed in round robin fashion to each parallel writer. Each writer is closed and a new file opened once soft_max_rows_per_output_file is reached. | | datafusion.execution.soft_max_rows_per_output_file | 50000000 | Target number of rows in output files when writing multiple. This is a soft max, so it can be exceeded slightly. There also will be one file smaller than the limit if the total number of rows written is not roughly divisible by the soft max | | datafusion.execution.max_buffered_batches_per_output_file | 2 | This is the maximum number of RecordBatches buffered for each output file being worked. Higher values can potentially give faster write performance at the cost of higher peak memory consumption | +| datafusion.execution.listing_table_ignore_subdirectory | true | When scanning file paths, whether to ignore subdirectory files, ignored by default (true), when reading a partitioned table, `listing_table_ignore_subdirectory` is always equal to false, even if set to true | | datafusion.optimizer.enable_distinct_aggregation_soft_limit | true | When set to true, the optimizer will push a limit operation into grouped aggregations which have no aggregate expressions, as a soft limit, emitting groups once the limit is reached, before all rows in the group are read. | | datafusion.optimizer.enable_round_robin_repartition | true | When set to true, the physical plan optimizer will try to add round robin repartitioning to increase parallelism to leverage more CPU cores | | datafusion.optimizer.enable_topk_aggregation | true | When set to true, the optimizer will attempt to perform limit operations during aggregations, if possible | From e4674929a1d17b2c2a80b8588fe61664606d9d63 Mon Sep 17 00:00:00 2001 From: Tomoaki Kawada Date: Sat, 23 Dec 2023 05:59:41 +0900 Subject: [PATCH 485/572] Optimize the parameter types of `ParamValues`'s methods (#8613) * Take `&str` instead of `&String` in `ParamValue::get_placeholders_with_values` * Take `Option<&DataType>` instead of `&Option` in `ParamValue::get_placeholders_with_values` * Take `&[_]` instead of `&Vec<_>` in `ParamValues::verify` --------- Co-authored-by: Andrew Lamb --- datafusion/common/src/param_value.rs | 10 +++++----- datafusion/expr/src/logical_plan/plan.rs | 4 ++-- 2 files changed, 7 insertions(+), 7 deletions(-) diff --git a/datafusion/common/src/param_value.rs b/datafusion/common/src/param_value.rs index 1b6195c0d0bc..004c1371d1ae 100644 --- a/datafusion/common/src/param_value.rs +++ b/datafusion/common/src/param_value.rs @@ -31,7 +31,7 @@ pub enum ParamValues { impl ParamValues { /// Verify parameter list length and type - pub fn verify(&self, expect: &Vec) -> Result<()> { + pub fn verify(&self, expect: &[DataType]) -> Result<()> { match self { ParamValues::List(list) => { // Verify if the number of params matches the number of values @@ -67,8 +67,8 @@ impl ParamValues { pub fn get_placeholders_with_values( &self, - id: &String, - data_type: &Option, + id: &str, + data_type: Option<&DataType>, ) -> Result { match self { ParamValues::List(list) => { @@ -88,7 +88,7 @@ impl ParamValues { )) })?; // check if the data type of the value matches the data type of the placeholder - if Some(value.data_type()) != *data_type { + if Some(&value.data_type()) != data_type { return _internal_err!( "Placeholder value type mismatch: expected {:?}, got {:?}", data_type, @@ -107,7 +107,7 @@ impl ParamValues { )) })?; // check if the data type of the value matches the data type of the placeholder - if Some(value.data_type()) != *data_type { + if Some(&value.data_type()) != data_type { return _internal_err!( "Placeholder value type mismatch: expected {:?}, got {:?}", data_type, diff --git a/datafusion/expr/src/logical_plan/plan.rs b/datafusion/expr/src/logical_plan/plan.rs index 1f3711407a14..50f4a6b76e18 100644 --- a/datafusion/expr/src/logical_plan/plan.rs +++ b/datafusion/expr/src/logical_plan/plan.rs @@ -1250,8 +1250,8 @@ impl LogicalPlan { expr.transform(&|expr| { match &expr { Expr::Placeholder(Placeholder { id, data_type }) => { - let value = - param_values.get_placeholders_with_values(id, data_type)?; + let value = param_values + .get_placeholders_with_values(id, data_type.as_ref())?; // Replace the placeholder with the value Ok(Transformed::Yes(Expr::Literal(value))) } From 03c2ef46f2d88fb015ee305ab67df6d930b780e2 Mon Sep 17 00:00:00 2001 From: Tomoaki Kawada Date: Sat, 23 Dec 2023 06:20:05 +0900 Subject: [PATCH 486/572] Don't panic on zero placeholder in `ParamValues::get_placeholders_with_values` (#8615) It correctly rejected `$0` but not the other ones that are parsed equally (e.g., `$000`). Co-authored-by: Andrew Lamb --- datafusion/common/src/param_value.rs | 17 ++++++++++------- datafusion/expr/src/logical_plan/plan.rs | 13 +++++++++++++ 2 files changed, 23 insertions(+), 7 deletions(-) diff --git a/datafusion/common/src/param_value.rs b/datafusion/common/src/param_value.rs index 004c1371d1ae..3fe2ba99ab83 100644 --- a/datafusion/common/src/param_value.rs +++ b/datafusion/common/src/param_value.rs @@ -72,17 +72,20 @@ impl ParamValues { ) -> Result { match self { ParamValues::List(list) => { - if id.is_empty() || id == "$0" { + if id.is_empty() { return _plan_err!("Empty placeholder id"); } // convert id (in format $1, $2, ..) to idx (0, 1, ..) - let idx = id[1..].parse::().map_err(|e| { - DataFusionError::Internal(format!( - "Failed to parse placeholder id: {e}" - )) - })? - 1; + let idx = id[1..] + .parse::() + .map_err(|e| { + DataFusionError::Internal(format!( + "Failed to parse placeholder id: {e}" + )) + })? + .checked_sub(1); // value at the idx-th position in param_values should be the value for the placeholder - let value = list.get(idx).ok_or_else(|| { + let value = idx.and_then(|idx| list.get(idx)).ok_or_else(|| { DataFusionError::Internal(format!( "No value found for placeholder with id {id}" )) diff --git a/datafusion/expr/src/logical_plan/plan.rs b/datafusion/expr/src/logical_plan/plan.rs index 50f4a6b76e18..9b0f441ef902 100644 --- a/datafusion/expr/src/logical_plan/plan.rs +++ b/datafusion/expr/src/logical_plan/plan.rs @@ -3099,6 +3099,19 @@ digraph { .build() .unwrap(); + plan.replace_params_with_values(¶m_values.clone().into()) + .expect_err("unexpectedly succeeded to replace an invalid placeholder"); + + // test $00 placeholder + let schema = Schema::new(vec![Field::new("id", DataType::Int32, false)]); + + let plan = table_scan(TableReference::none(), &schema, None) + .unwrap() + .filter(col("id").eq(placeholder("$00"))) + .unwrap() + .build() + .unwrap(); + plan.replace_params_with_values(¶m_values.into()) .expect_err("unexpectedly succeeded to replace an invalid placeholder"); } From df2e1e2587340c513743b965f9aef301c4a2a859 Mon Sep 17 00:00:00 2001 From: Marvin Lanhenke <62298609+marvinlanhenke@users.noreply.github.com> Date: Sat, 23 Dec 2023 13:09:50 +0100 Subject: [PATCH 487/572] Fix #8507: Non-null sub-field on nullable struct-field has wrong nullity (#8623) * added test * added guard clause * rename schema fields * clippy --------- Co-authored-by: mlanhenke --- datafusion/expr/src/expr_schema.rs | 32 ++++++++++++++++++++++++++++-- 1 file changed, 30 insertions(+), 2 deletions(-) diff --git a/datafusion/expr/src/expr_schema.rs b/datafusion/expr/src/expr_schema.rs index e5b0185d90e0..ba21d09f0619 100644 --- a/datafusion/expr/src/expr_schema.rs +++ b/datafusion/expr/src/expr_schema.rs @@ -277,6 +277,13 @@ impl ExprSchemable for Expr { "Wildcard expressions are not valid in a logical query plan" ), Expr::GetIndexedField(GetIndexedField { expr, field }) => { + // If schema is nested, check if parent is nullable + // if it is, return early + if let Expr::Column(col) = expr.as_ref() { + if input_schema.nullable(col)? { + return Ok(true); + } + } field_for_index(expr, field, input_schema).map(|x| x.is_nullable()) } Expr::GroupingSet(_) => { @@ -411,8 +418,8 @@ pub fn cast_subquery(subquery: Subquery, cast_to_type: &DataType) -> Result {{ @@ -548,6 +555,27 @@ mod tests { assert_eq!(&meta, expr.to_field(&schema).unwrap().metadata()); } + #[test] + fn test_nested_schema_nullability() { + let fields = DFField::new( + Some(TableReference::Bare { + table: "table_name".into(), + }), + "parent", + DataType::Struct(Fields::from(vec![Field::new( + "child", + DataType::Int64, + false, + )])), + true, + ); + + let schema = DFSchema::new_with_metadata(vec![fields], HashMap::new()).unwrap(); + + let expr = col("parent").field("child"); + assert!(expr.nullable(&schema).unwrap()); + } + #[derive(Debug)] struct MockExprSchema { nullable: bool, From 8524d58e303b65597eeebc41c75025a6f0822793 Mon Sep 17 00:00:00 2001 From: Andrew Lamb Date: Sat, 23 Dec 2023 07:10:56 -0500 Subject: [PATCH 488/572] Implement `contained` API in PruningPredicate (#8440) * Implement `contains` API in PruningPredicate * Apply suggestions from code review Co-authored-by: Nga Tran * Add comment to len(), fix fmt * rename BoolVecBuilder::append* to BoolVecBuilder::combine* --------- Co-authored-by: Nga Tran --- .../physical_plan/parquet/page_filter.rs | 11 +- .../physical_plan/parquet/row_groups.rs | 9 + .../core/src/physical_optimizer/pruning.rs | 1073 +++++++++++++---- 3 files changed, 857 insertions(+), 236 deletions(-) diff --git a/datafusion/core/src/datasource/physical_plan/parquet/page_filter.rs b/datafusion/core/src/datasource/physical_plan/parquet/page_filter.rs index 42bfef35996e..f6310c49bcd6 100644 --- a/datafusion/core/src/datasource/physical_plan/parquet/page_filter.rs +++ b/datafusion/core/src/datasource/physical_plan/parquet/page_filter.rs @@ -23,7 +23,7 @@ use arrow::array::{ }; use arrow::datatypes::DataType; use arrow::{array::ArrayRef, datatypes::SchemaRef, error::ArrowError}; -use datafusion_common::{DataFusionError, Result}; +use datafusion_common::{DataFusionError, Result, ScalarValue}; use datafusion_physical_expr::expressions::Column; use datafusion_physical_expr::{split_conjunction, PhysicalExpr}; use log::{debug, trace}; @@ -37,6 +37,7 @@ use parquet::{ }, format::PageLocation, }; +use std::collections::HashSet; use std::sync::Arc; use crate::datasource::physical_plan::parquet::parquet_to_arrow_decimal_type; @@ -554,4 +555,12 @@ impl<'a> PruningStatistics for PagesPruningStatistics<'a> { ))), } } + + fn contained( + &self, + _column: &datafusion_common::Column, + _values: &HashSet, + ) -> Option { + None + } } diff --git a/datafusion/core/src/datasource/physical_plan/parquet/row_groups.rs b/datafusion/core/src/datasource/physical_plan/parquet/row_groups.rs index 7c3f7d9384ab..09e4907c9437 100644 --- a/datafusion/core/src/datasource/physical_plan/parquet/row_groups.rs +++ b/datafusion/core/src/datasource/physical_plan/parquet/row_groups.rs @@ -16,6 +16,7 @@ // under the License. use arrow::{array::ArrayRef, datatypes::Schema}; +use arrow_array::BooleanArray; use arrow_schema::FieldRef; use datafusion_common::tree_node::{TreeNode, VisitRecursion}; use datafusion_common::{Column, DataFusionError, Result, ScalarValue}; @@ -340,6 +341,14 @@ impl<'a> PruningStatistics for RowGroupPruningStatistics<'a> { let scalar = ScalarValue::UInt64(Some(c.statistics()?.null_count())); scalar.to_array().ok() } + + fn contained( + &self, + _column: &Column, + _values: &HashSet, + ) -> Option { + None + } } #[cfg(test)] diff --git a/datafusion/core/src/physical_optimizer/pruning.rs b/datafusion/core/src/physical_optimizer/pruning.rs index b2ba7596db8d..79e084d7b7f1 100644 --- a/datafusion/core/src/physical_optimizer/pruning.rs +++ b/datafusion/core/src/physical_optimizer/pruning.rs @@ -35,12 +35,13 @@ use arrow::{ datatypes::{DataType, Field, Schema, SchemaRef}, record_batch::RecordBatch, }; -use datafusion_common::{downcast_value, plan_datafusion_err, ScalarValue}; +use arrow_array::cast::AsArray; use datafusion_common::{ internal_err, plan_err, tree_node::{Transformed, TreeNode}, }; -use datafusion_physical_expr::utils::collect_columns; +use datafusion_common::{plan_datafusion_err, ScalarValue}; +use datafusion_physical_expr::utils::{collect_columns, Guarantee, LiteralGuarantee}; use datafusion_physical_expr::{expressions as phys_expr, PhysicalExprRef}; use log::trace; @@ -93,6 +94,30 @@ pub trait PruningStatistics { /// /// Note: the returned array must contain [`Self::num_containers`] rows fn null_counts(&self, column: &Column) -> Option; + + /// Returns an array where each row represents information known about + /// the `values` contained in a column. + /// + /// This API is designed to be used along with [`LiteralGuarantee`] to prove + /// that predicates can not possibly evaluate to `true` and thus prune + /// containers. For example, Parquet Bloom Filters can prove that values are + /// not present. + /// + /// The returned array has one row for each container, with the following + /// meanings: + /// * `true` if the values in `column` ONLY contain values from `values` + /// * `false` if the values in `column` are NOT ANY of `values` + /// * `null` if the neither of the above holds or is unknown. + /// + /// If these statistics can not determine column membership for any + /// container, return `None` (the default). + /// + /// Note: the returned array must contain [`Self::num_containers`] rows + fn contained( + &self, + column: &Column, + values: &HashSet, + ) -> Option; } /// Evaluates filter expressions on statistics such as min/max values and null @@ -142,12 +167,17 @@ pub trait PruningStatistics { pub struct PruningPredicate { /// The input schema against which the predicate will be evaluated schema: SchemaRef, - /// Actual pruning predicate (rewritten in terms of column min/max statistics) + /// A min/max pruning predicate (rewritten in terms of column min/max + /// values, which are supplied by statistics) predicate_expr: Arc, - /// The statistics required to evaluate this predicate - required_columns: RequiredStatColumns, - /// Original physical predicate from which this predicate expr is derived (required for serialization) + /// Description of which statistics are required to evaluate `predicate_expr` + required_columns: RequiredColumns, + /// Original physical predicate from which this predicate expr is derived + /// (required for serialization) orig_expr: Arc, + /// [`LiteralGuarantee`]s that are used to try and prove a predicate can not + /// possibly evaluate to `true`. + literal_guarantees: Vec, } impl PruningPredicate { @@ -172,14 +202,18 @@ impl PruningPredicate { /// `(column_min / 2) <= 4 && 4 <= (column_max / 2))` pub fn try_new(expr: Arc, schema: SchemaRef) -> Result { // build predicate expression once - let mut required_columns = RequiredStatColumns::new(); + let mut required_columns = RequiredColumns::new(); let predicate_expr = build_predicate_expression(&expr, schema.as_ref(), &mut required_columns); + + let literal_guarantees = LiteralGuarantee::analyze(&expr); + Ok(Self { schema, predicate_expr, required_columns, orig_expr: expr, + literal_guarantees, }) } @@ -198,40 +232,47 @@ impl PruningPredicate { /// /// [`ExprSimplifier`]: crate::optimizer::simplify_expressions::ExprSimplifier pub fn prune(&self, statistics: &S) -> Result> { + let mut builder = BoolVecBuilder::new(statistics.num_containers()); + + // Try to prove the predicate can't be true for the containers based on + // literal guarantees + for literal_guarantee in &self.literal_guarantees { + let LiteralGuarantee { + column, + guarantee, + literals, + } = literal_guarantee; + if let Some(results) = statistics.contained(column, literals) { + match guarantee { + // `In` means the values in the column must be one of the + // values in the set for the predicate to evaluate to true. + // If `contained` returns false, that means the column is + // not any of the values so we can prune the container + Guarantee::In => builder.combine_array(&results), + // `NotIn` means the values in the column must must not be + // any of the values in the set for the predicate to + // evaluate to true. If contained returns true, it means the + // column is only in the set of values so we can prune the + // container + Guarantee::NotIn => { + builder.combine_array(&arrow::compute::not(&results)?) + } + } + } + } + + // Next, try to prove the predicate can't be true for the containers based + // on min/max values + // build a RecordBatch that contains the min/max values in the - // appropriate statistics columns + // appropriate statistics columns for the min/max predicate let statistics_batch = build_statistics_record_batch(statistics, &self.required_columns)?; - // Evaluate the pruning predicate on that record batch. - // - // Use true when the result of evaluating a predicate - // expression on a row group is null (aka `None`). Null can - // arise when the statistics are unknown or some calculation - // in the predicate means we don't know for sure if the row - // group can be filtered out or not. To maintain correctness - // the row group must be kept and thus `true` is returned. - match self.predicate_expr.evaluate(&statistics_batch)? { - ColumnarValue::Array(array) => { - let predicate_array = downcast_value!(array, BooleanArray); + // Evaluate the pruning predicate on that record batch and append any results to the builder + builder.combine_value(self.predicate_expr.evaluate(&statistics_batch)?); - Ok(predicate_array - .into_iter() - .map(|x| x.unwrap_or(true)) // None -> true per comments above - .collect::>()) - } - // result was a column - ColumnarValue::Scalar(ScalarValue::Boolean(v)) => { - let v = v.unwrap_or(true); // None -> true per comments above - Ok(vec![v; statistics.num_containers()]) - } - other => { - internal_err!( - "Unexpected result of pruning predicate evaluation. Expected Boolean array \ - or scalar but got {other:?}" - ) - } - } + Ok(builder.build()) } /// Return a reference to the input schema @@ -254,9 +295,91 @@ impl PruningPredicate { is_always_true(&self.predicate_expr) } - pub(crate) fn required_columns(&self) -> &RequiredStatColumns { + pub(crate) fn required_columns(&self) -> &RequiredColumns { &self.required_columns } + + /// Names of the columns that are known to be / not be in a set + /// of literals (constants). These are the columns the that may be passed to + /// [`PruningStatistics::contained`] during pruning. + /// + /// This is useful to avoid fetching statistics for columns that will not be + /// used in the predicate. For example, it can be used to avoid reading + /// uneeded bloom filters (a non trivial operation). + pub fn literal_columns(&self) -> Vec { + let mut seen = HashSet::new(); + self.literal_guarantees + .iter() + .map(|e| &e.column.name) + // avoid duplicates + .filter(|name| seen.insert(*name)) + .map(|s| s.to_string()) + .collect() + } +} + +/// Builds the return `Vec` for [`PruningPredicate::prune`]. +#[derive(Debug)] +struct BoolVecBuilder { + /// One element per container. Each element is + /// * `true`: if the container has row that may pass the predicate + /// * `false`: if the container has rows that DEFINITELY DO NOT pass the predicate + inner: Vec, +} + +impl BoolVecBuilder { + /// Create a new `BoolVecBuilder` with `num_containers` elements + fn new(num_containers: usize) -> Self { + Self { + // assume by default all containers may pass the predicate + inner: vec![true; num_containers], + } + } + + /// Combines result `array` for a conjunct (e.g. `AND` clause) of a + /// predicate into the currently in progress array. + /// + /// Each `array` element is: + /// * `true`: container has row that may pass the predicate + /// * `false`: all container rows DEFINITELY DO NOT pass the predicate + /// * `null`: container may or may not have rows that pass the predicate + fn combine_array(&mut self, array: &BooleanArray) { + assert_eq!(array.len(), self.inner.len()); + for (cur, new) in self.inner.iter_mut().zip(array.iter()) { + // `false` for this conjunct means we know for sure no rows could + // pass the predicate and thus we set the corresponding container + // location to false. + if let Some(false) = new { + *cur = false; + } + } + } + + /// Combines the results in the [`ColumnarValue`] to the currently in + /// progress array, following the same rules as [`Self::combine_array`]. + /// + /// # Panics + /// If `value` is not boolean + fn combine_value(&mut self, value: ColumnarValue) { + match value { + ColumnarValue::Array(array) => { + self.combine_array(array.as_boolean()); + } + ColumnarValue::Scalar(ScalarValue::Boolean(Some(false))) => { + // False means all containers can not pass the predicate + self.inner = vec![false; self.inner.len()]; + } + _ => { + // Null or true means the rows in container may pass this + // conjunct so we can't prune any containers based on that + } + } + } + + /// Convert this builder into a Vec of bools + fn build(self) -> Vec { + self.inner + } } fn is_always_true(expr: &Arc) -> bool { @@ -276,21 +399,21 @@ fn is_always_true(expr: &Arc) -> bool { /// Handles creating references to the min/max statistics /// for columns as well as recording which statistics are needed #[derive(Debug, Default, Clone)] -pub(crate) struct RequiredStatColumns { +pub(crate) struct RequiredColumns { /// The statistics required to evaluate this predicate: /// * The unqualified column in the input schema /// * Statistics type (e.g. Min or Max or Null_Count) /// * The field the statistics value should be placed in for - /// pruning predicate evaluation + /// pruning predicate evaluation (e.g. `min_value` or `max_value`) columns: Vec<(phys_expr::Column, StatisticsType, Field)>, } -impl RequiredStatColumns { +impl RequiredColumns { fn new() -> Self { Self::default() } - /// Returns number of unique columns. + /// Returns number of unique columns pub(crate) fn n_columns(&self) -> usize { self.iter() .map(|(c, _s, _f)| c) @@ -344,11 +467,10 @@ impl RequiredStatColumns { // only add statistics column if not previously added if need_to_insert { - let stat_field = Field::new( - stat_column.name(), - field.data_type().clone(), - field.is_nullable(), - ); + // may be null if statistics are not present + let nullable = true; + let stat_field = + Field::new(stat_column.name(), field.data_type().clone(), nullable); self.columns.push((column.clone(), stat_type, stat_field)); } rewrite_column_expr(column_expr.clone(), column, &stat_column) @@ -391,7 +513,7 @@ impl RequiredStatColumns { } } -impl From> for RequiredStatColumns { +impl From> for RequiredColumns { fn from(columns: Vec<(phys_expr::Column, StatisticsType, Field)>) -> Self { Self { columns } } @@ -424,7 +546,7 @@ impl From> for RequiredStatColum /// ``` fn build_statistics_record_batch( statistics: &S, - required_columns: &RequiredStatColumns, + required_columns: &RequiredColumns, ) -> Result { let mut fields = Vec::::new(); let mut arrays = Vec::::new(); @@ -480,7 +602,7 @@ struct PruningExpressionBuilder<'a> { op: Operator, scalar_expr: Arc, field: &'a Field, - required_columns: &'a mut RequiredStatColumns, + required_columns: &'a mut RequiredColumns, } impl<'a> PruningExpressionBuilder<'a> { @@ -489,7 +611,7 @@ impl<'a> PruningExpressionBuilder<'a> { right: &'a Arc, op: Operator, schema: &'a Schema, - required_columns: &'a mut RequiredStatColumns, + required_columns: &'a mut RequiredColumns, ) -> Result { // find column name; input could be a more complicated expression let left_columns = collect_columns(left); @@ -704,7 +826,7 @@ fn reverse_operator(op: Operator) -> Result { fn build_single_column_expr( column: &phys_expr::Column, schema: &Schema, - required_columns: &mut RequiredStatColumns, + required_columns: &mut RequiredColumns, is_not: bool, // if true, treat as !col ) -> Option> { let field = schema.field_with_name(column.name()).ok()?; @@ -745,7 +867,7 @@ fn build_single_column_expr( fn build_is_null_column_expr( expr: &Arc, schema: &Schema, - required_columns: &mut RequiredStatColumns, + required_columns: &mut RequiredColumns, ) -> Option> { if let Some(col) = expr.as_any().downcast_ref::() { let field = schema.field_with_name(col.name()).ok()?; @@ -775,7 +897,7 @@ fn build_is_null_column_expr( fn build_predicate_expression( expr: &Arc, schema: &Schema, - required_columns: &mut RequiredStatColumns, + required_columns: &mut RequiredColumns, ) -> Arc { // Returned for unsupported expressions. Such expressions are // converted to TRUE. @@ -984,7 +1106,7 @@ mod tests { use std::collections::HashMap; use std::ops::{Not, Rem}; - #[derive(Debug)] + #[derive(Debug, Default)] /// Mock statistic provider for tests /// /// Each row represents the statistics for a "container" (which @@ -993,95 +1115,142 @@ mod tests { /// /// Note All `ArrayRefs` must be the same size. struct ContainerStats { - min: ArrayRef, - max: ArrayRef, + min: Option, + max: Option, /// Optional values null_counts: Option, + /// Optional known values (e.g. mimic a bloom filter) + /// (value, contained) + /// If present, all BooleanArrays must be the same size as min/max + contained: Vec<(HashSet, BooleanArray)>, } impl ContainerStats { + fn new() -> Self { + Default::default() + } fn new_decimal128( min: impl IntoIterator>, max: impl IntoIterator>, precision: u8, scale: i8, ) -> Self { - Self { - min: Arc::new( + Self::new() + .with_min(Arc::new( min.into_iter() .collect::() .with_precision_and_scale(precision, scale) .unwrap(), - ), - max: Arc::new( + )) + .with_max(Arc::new( max.into_iter() .collect::() .with_precision_and_scale(precision, scale) .unwrap(), - ), - null_counts: None, - } + )) } fn new_i64( min: impl IntoIterator>, max: impl IntoIterator>, ) -> Self { - Self { - min: Arc::new(min.into_iter().collect::()), - max: Arc::new(max.into_iter().collect::()), - null_counts: None, - } + Self::new() + .with_min(Arc::new(min.into_iter().collect::())) + .with_max(Arc::new(max.into_iter().collect::())) } fn new_i32( min: impl IntoIterator>, max: impl IntoIterator>, ) -> Self { - Self { - min: Arc::new(min.into_iter().collect::()), - max: Arc::new(max.into_iter().collect::()), - null_counts: None, - } + Self::new() + .with_min(Arc::new(min.into_iter().collect::())) + .with_max(Arc::new(max.into_iter().collect::())) } fn new_utf8<'a>( min: impl IntoIterator>, max: impl IntoIterator>, ) -> Self { - Self { - min: Arc::new(min.into_iter().collect::()), - max: Arc::new(max.into_iter().collect::()), - null_counts: None, - } + Self::new() + .with_min(Arc::new(min.into_iter().collect::())) + .with_max(Arc::new(max.into_iter().collect::())) } fn new_bool( min: impl IntoIterator>, max: impl IntoIterator>, ) -> Self { - Self { - min: Arc::new(min.into_iter().collect::()), - max: Arc::new(max.into_iter().collect::()), - null_counts: None, - } + Self::new() + .with_min(Arc::new(min.into_iter().collect::())) + .with_max(Arc::new(max.into_iter().collect::())) } fn min(&self) -> Option { - Some(self.min.clone()) + self.min.clone() } fn max(&self) -> Option { - Some(self.max.clone()) + self.max.clone() } fn null_counts(&self) -> Option { self.null_counts.clone() } + /// return an iterator over all arrays in this statistics + fn arrays(&self) -> Vec { + let contained_arrays = self + .contained + .iter() + .map(|(_values, contained)| Arc::new(contained.clone()) as ArrayRef); + + [ + self.min.as_ref().cloned(), + self.max.as_ref().cloned(), + self.null_counts.as_ref().cloned(), + ] + .into_iter() + .flatten() + .chain(contained_arrays) + .collect() + } + + /// Returns the number of containers represented by this statistics This + /// picks the length of the first array as all arrays must have the same + /// length (which is verified by `assert_invariants`). fn len(&self) -> usize { - assert_eq!(self.min.len(), self.max.len()); - self.min.len() + // pick the first non zero length + self.arrays().iter().map(|a| a.len()).next().unwrap_or(0) + } + + /// Ensure that the lengths of all arrays are consistent + fn assert_invariants(&self) { + let mut prev_len = None; + + for len in self.arrays().iter().map(|a| a.len()) { + // Get a length, if we don't already have one + match prev_len { + None => { + prev_len = Some(len); + } + Some(prev_len) => { + assert_eq!(prev_len, len); + } + } + } + } + + /// Add min values + fn with_min(mut self, min: ArrayRef) -> Self { + self.min = Some(min); + self + } + + /// Add max values + fn with_max(mut self, max: ArrayRef) -> Self { + self.max = Some(max); + self } /// Add null counts. There must be the same number of null counts as @@ -1090,14 +1259,36 @@ mod tests { mut self, counts: impl IntoIterator>, ) -> Self { - // take stats out and update them let null_counts: ArrayRef = Arc::new(counts.into_iter().collect::()); - assert_eq!(null_counts.len(), self.len()); + self.assert_invariants(); self.null_counts = Some(null_counts); self } + + /// Add contained information. + pub fn with_contained( + mut self, + values: impl IntoIterator, + contained: impl IntoIterator>, + ) -> Self { + let contained: BooleanArray = contained.into_iter().collect(); + let values: HashSet<_> = values.into_iter().collect(); + + self.contained.push((values, contained)); + self.assert_invariants(); + self + } + + /// get any contained information for the specified values + fn contained(&self, find_values: &HashSet) -> Option { + // find the one with the matching values + self.contained + .iter() + .find(|(values, _contained)| values == find_values) + .map(|(_values, contained)| contained.clone()) + } } #[derive(Debug, Default)] @@ -1135,13 +1326,34 @@ mod tests { let container_stats = self .stats .remove(&col) - .expect("Can not find stats for column") + .unwrap_or_default() .with_null_counts(counts); // put stats back in self.stats.insert(col, container_stats); self } + + /// Add contained information for the specified columm. + fn with_contained( + mut self, + name: impl Into, + values: impl IntoIterator, + contained: impl IntoIterator>, + ) -> Self { + let col = Column::from_name(name.into()); + + // take stats out and update them + let container_stats = self + .stats + .remove(&col) + .unwrap_or_default() + .with_contained(values, contained); + + // put stats back in + self.stats.insert(col, container_stats); + self + } } impl PruningStatistics for TestStatistics { @@ -1173,6 +1385,16 @@ mod tests { .map(|container_stats| container_stats.null_counts()) .unwrap_or(None) } + + fn contained( + &self, + column: &Column, + values: &HashSet, + ) -> Option { + self.stats + .get(column) + .and_then(|container_stats| container_stats.contained(values)) + } } /// Returns the specified min/max container values @@ -1198,12 +1420,20 @@ mod tests { fn null_counts(&self, _column: &Column) -> Option { None } + + fn contained( + &self, + _column: &Column, + _values: &HashSet, + ) -> Option { + None + } } #[test] fn test_build_statistics_record_batch() { // Request a record batch with of s1_min, s2_max, s3_max, s3_min - let required_columns = RequiredStatColumns::from(vec![ + let required_columns = RequiredColumns::from(vec![ // min of original column s1, named s1_min ( phys_expr::Column::new("s1", 1), @@ -1275,7 +1505,7 @@ mod tests { // which is what Parquet does // Request a record batch with of s1_min as a timestamp - let required_columns = RequiredStatColumns::from(vec![( + let required_columns = RequiredColumns::from(vec![( phys_expr::Column::new("s3", 3), StatisticsType::Min, Field::new( @@ -1307,7 +1537,7 @@ mod tests { #[test] fn test_build_statistics_no_required_stats() { - let required_columns = RequiredStatColumns::new(); + let required_columns = RequiredColumns::new(); let statistics = OneContainerStats { min_values: Some(Arc::new(Int64Array::from(vec![Some(10)]))), @@ -1325,7 +1555,7 @@ mod tests { // Test requesting a Utf8 column when the stats return some other type // Request a record batch with of s1_min as a timestamp - let required_columns = RequiredStatColumns::from(vec![( + let required_columns = RequiredColumns::from(vec![( phys_expr::Column::new("s3", 3), StatisticsType::Min, Field::new("s1_min", DataType::Utf8, true), @@ -1354,7 +1584,7 @@ mod tests { #[test] fn test_build_statistics_inconsistent_length() { // return an inconsistent length to the actual statistics arrays - let required_columns = RequiredStatColumns::from(vec![( + let required_columns = RequiredColumns::from(vec![( phys_expr::Column::new("s1", 3), StatisticsType::Min, Field::new("s1_min", DataType::Int64, true), @@ -1385,20 +1615,14 @@ mod tests { // test column on the left let expr = col("c1").eq(lit(1)); - let predicate_expr = test_build_predicate_expression( - &expr, - &schema, - &mut RequiredStatColumns::new(), - ); + let predicate_expr = + test_build_predicate_expression(&expr, &schema, &mut RequiredColumns::new()); assert_eq!(predicate_expr.to_string(), expected_expr); // test column on the right let expr = lit(1).eq(col("c1")); - let predicate_expr = test_build_predicate_expression( - &expr, - &schema, - &mut RequiredStatColumns::new(), - ); + let predicate_expr = + test_build_predicate_expression(&expr, &schema, &mut RequiredColumns::new()); assert_eq!(predicate_expr.to_string(), expected_expr); Ok(()) @@ -1411,20 +1635,14 @@ mod tests { // test column on the left let expr = col("c1").not_eq(lit(1)); - let predicate_expr = test_build_predicate_expression( - &expr, - &schema, - &mut RequiredStatColumns::new(), - ); + let predicate_expr = + test_build_predicate_expression(&expr, &schema, &mut RequiredColumns::new()); assert_eq!(predicate_expr.to_string(), expected_expr); // test column on the right let expr = lit(1).not_eq(col("c1")); - let predicate_expr = test_build_predicate_expression( - &expr, - &schema, - &mut RequiredStatColumns::new(), - ); + let predicate_expr = + test_build_predicate_expression(&expr, &schema, &mut RequiredColumns::new()); assert_eq!(predicate_expr.to_string(), expected_expr); Ok(()) @@ -1437,20 +1655,14 @@ mod tests { // test column on the left let expr = col("c1").gt(lit(1)); - let predicate_expr = test_build_predicate_expression( - &expr, - &schema, - &mut RequiredStatColumns::new(), - ); + let predicate_expr = + test_build_predicate_expression(&expr, &schema, &mut RequiredColumns::new()); assert_eq!(predicate_expr.to_string(), expected_expr); // test column on the right let expr = lit(1).lt(col("c1")); - let predicate_expr = test_build_predicate_expression( - &expr, - &schema, - &mut RequiredStatColumns::new(), - ); + let predicate_expr = + test_build_predicate_expression(&expr, &schema, &mut RequiredColumns::new()); assert_eq!(predicate_expr.to_string(), expected_expr); Ok(()) @@ -1463,19 +1675,13 @@ mod tests { // test column on the left let expr = col("c1").gt_eq(lit(1)); - let predicate_expr = test_build_predicate_expression( - &expr, - &schema, - &mut RequiredStatColumns::new(), - ); + let predicate_expr = + test_build_predicate_expression(&expr, &schema, &mut RequiredColumns::new()); assert_eq!(predicate_expr.to_string(), expected_expr); // test column on the right let expr = lit(1).lt_eq(col("c1")); - let predicate_expr = test_build_predicate_expression( - &expr, - &schema, - &mut RequiredStatColumns::new(), - ); + let predicate_expr = + test_build_predicate_expression(&expr, &schema, &mut RequiredColumns::new()); assert_eq!(predicate_expr.to_string(), expected_expr); Ok(()) @@ -1488,20 +1694,14 @@ mod tests { // test column on the left let expr = col("c1").lt(lit(1)); - let predicate_expr = test_build_predicate_expression( - &expr, - &schema, - &mut RequiredStatColumns::new(), - ); + let predicate_expr = + test_build_predicate_expression(&expr, &schema, &mut RequiredColumns::new()); assert_eq!(predicate_expr.to_string(), expected_expr); // test column on the right let expr = lit(1).gt(col("c1")); - let predicate_expr = test_build_predicate_expression( - &expr, - &schema, - &mut RequiredStatColumns::new(), - ); + let predicate_expr = + test_build_predicate_expression(&expr, &schema, &mut RequiredColumns::new()); assert_eq!(predicate_expr.to_string(), expected_expr); Ok(()) @@ -1514,19 +1714,13 @@ mod tests { // test column on the left let expr = col("c1").lt_eq(lit(1)); - let predicate_expr = test_build_predicate_expression( - &expr, - &schema, - &mut RequiredStatColumns::new(), - ); + let predicate_expr = + test_build_predicate_expression(&expr, &schema, &mut RequiredColumns::new()); assert_eq!(predicate_expr.to_string(), expected_expr); // test column on the right let expr = lit(1).gt_eq(col("c1")); - let predicate_expr = test_build_predicate_expression( - &expr, - &schema, - &mut RequiredStatColumns::new(), - ); + let predicate_expr = + test_build_predicate_expression(&expr, &schema, &mut RequiredColumns::new()); assert_eq!(predicate_expr.to_string(), expected_expr); Ok(()) @@ -1542,11 +1736,8 @@ mod tests { // test AND operator joining supported c1 < 1 expression and unsupported c2 > c3 expression let expr = col("c1").lt(lit(1)).and(col("c2").lt(col("c3"))); let expected_expr = "c1_min@0 < 1"; - let predicate_expr = test_build_predicate_expression( - &expr, - &schema, - &mut RequiredStatColumns::new(), - ); + let predicate_expr = + test_build_predicate_expression(&expr, &schema, &mut RequiredColumns::new()); assert_eq!(predicate_expr.to_string(), expected_expr); Ok(()) @@ -1561,11 +1752,8 @@ mod tests { // test OR operator joining supported c1 < 1 expression and unsupported c2 % 2 = 0 expression let expr = col("c1").lt(lit(1)).or(col("c2").rem(lit(2)).eq(lit(0))); let expected_expr = "true"; - let predicate_expr = test_build_predicate_expression( - &expr, - &schema, - &mut RequiredStatColumns::new(), - ); + let predicate_expr = + test_build_predicate_expression(&expr, &schema, &mut RequiredColumns::new()); assert_eq!(predicate_expr.to_string(), expected_expr); Ok(()) @@ -1577,11 +1765,8 @@ mod tests { let expected_expr = "true"; let expr = col("c1").not(); - let predicate_expr = test_build_predicate_expression( - &expr, - &schema, - &mut RequiredStatColumns::new(), - ); + let predicate_expr = + test_build_predicate_expression(&expr, &schema, &mut RequiredColumns::new()); assert_eq!(predicate_expr.to_string(), expected_expr); Ok(()) @@ -1593,11 +1778,8 @@ mod tests { let expected_expr = "NOT c1_min@0 AND c1_max@1"; let expr = col("c1").not(); - let predicate_expr = test_build_predicate_expression( - &expr, - &schema, - &mut RequiredStatColumns::new(), - ); + let predicate_expr = + test_build_predicate_expression(&expr, &schema, &mut RequiredColumns::new()); assert_eq!(predicate_expr.to_string(), expected_expr); Ok(()) @@ -1609,11 +1791,8 @@ mod tests { let expected_expr = "c1_min@0 OR c1_max@1"; let expr = col("c1"); - let predicate_expr = test_build_predicate_expression( - &expr, - &schema, - &mut RequiredStatColumns::new(), - ); + let predicate_expr = + test_build_predicate_expression(&expr, &schema, &mut RequiredColumns::new()); assert_eq!(predicate_expr.to_string(), expected_expr); Ok(()) @@ -1627,11 +1806,8 @@ mod tests { // DF doesn't support arithmetic on boolean columns so // this predicate will error when evaluated let expr = col("c1").lt(lit(true)); - let predicate_expr = test_build_predicate_expression( - &expr, - &schema, - &mut RequiredStatColumns::new(), - ); + let predicate_expr = + test_build_predicate_expression(&expr, &schema, &mut RequiredColumns::new()); assert_eq!(predicate_expr.to_string(), expected_expr); Ok(()) @@ -1643,7 +1819,7 @@ mod tests { Field::new("c1", DataType::Int32, false), Field::new("c2", DataType::Int32, false), ]); - let mut required_columns = RequiredStatColumns::new(); + let mut required_columns = RequiredColumns::new(); // c1 < 1 and (c2 = 2 or c2 = 3) let expr = col("c1") .lt(lit(1)) @@ -1659,7 +1835,7 @@ mod tests { ( phys_expr::Column::new("c1", 0), StatisticsType::Min, - c1_min_field + c1_min_field.with_nullable(true) // could be nullable if stats are not present ) ); // c2 = 2 should add c2_min and c2_max @@ -1669,7 +1845,7 @@ mod tests { ( phys_expr::Column::new("c2", 1), StatisticsType::Min, - c2_min_field + c2_min_field.with_nullable(true) // could be nullable if stats are not present ) ); let c2_max_field = Field::new("c2_max", DataType::Int32, false); @@ -1678,7 +1854,7 @@ mod tests { ( phys_expr::Column::new("c2", 1), StatisticsType::Max, - c2_max_field + c2_max_field.with_nullable(true) // could be nullable if stats are not present ) ); // c2 = 3 shouldn't add any new statistics fields @@ -1700,11 +1876,8 @@ mod tests { false, )); let expected_expr = "c1_min@0 <= 1 AND 1 <= c1_max@1 OR c1_min@0 <= 2 AND 2 <= c1_max@1 OR c1_min@0 <= 3 AND 3 <= c1_max@1"; - let predicate_expr = test_build_predicate_expression( - &expr, - &schema, - &mut RequiredStatColumns::new(), - ); + let predicate_expr = + test_build_predicate_expression(&expr, &schema, &mut RequiredColumns::new()); assert_eq!(predicate_expr.to_string(), expected_expr); Ok(()) @@ -1719,11 +1892,8 @@ mod tests { // test c1 in() let expr = Expr::InList(InList::new(Box::new(col("c1")), vec![], false)); let expected_expr = "true"; - let predicate_expr = test_build_predicate_expression( - &expr, - &schema, - &mut RequiredStatColumns::new(), - ); + let predicate_expr = + test_build_predicate_expression(&expr, &schema, &mut RequiredColumns::new()); assert_eq!(predicate_expr.to_string(), expected_expr); Ok(()) @@ -1744,11 +1914,8 @@ mod tests { let expected_expr = "(c1_min@0 != 1 OR 1 != c1_max@1) \ AND (c1_min@0 != 2 OR 2 != c1_max@1) \ AND (c1_min@0 != 3 OR 3 != c1_max@1)"; - let predicate_expr = test_build_predicate_expression( - &expr, - &schema, - &mut RequiredStatColumns::new(), - ); + let predicate_expr = + test_build_predicate_expression(&expr, &schema, &mut RequiredColumns::new()); assert_eq!(predicate_expr.to_string(), expected_expr); Ok(()) @@ -1762,20 +1929,14 @@ mod tests { // test column on the left let expr = cast(col("c1"), DataType::Int64).eq(lit(ScalarValue::Int64(Some(1)))); - let predicate_expr = test_build_predicate_expression( - &expr, - &schema, - &mut RequiredStatColumns::new(), - ); + let predicate_expr = + test_build_predicate_expression(&expr, &schema, &mut RequiredColumns::new()); assert_eq!(predicate_expr.to_string(), expected_expr); // test column on the right let expr = lit(ScalarValue::Int64(Some(1))).eq(cast(col("c1"), DataType::Int64)); - let predicate_expr = test_build_predicate_expression( - &expr, - &schema, - &mut RequiredStatColumns::new(), - ); + let predicate_expr = + test_build_predicate_expression(&expr, &schema, &mut RequiredColumns::new()); assert_eq!(predicate_expr.to_string(), expected_expr); let expected_expr = "TRY_CAST(c1_max@0 AS Int64) > 1"; @@ -1783,21 +1944,15 @@ mod tests { // test column on the left let expr = try_cast(col("c1"), DataType::Int64).gt(lit(ScalarValue::Int64(Some(1)))); - let predicate_expr = test_build_predicate_expression( - &expr, - &schema, - &mut RequiredStatColumns::new(), - ); + let predicate_expr = + test_build_predicate_expression(&expr, &schema, &mut RequiredColumns::new()); assert_eq!(predicate_expr.to_string(), expected_expr); // test column on the right let expr = lit(ScalarValue::Int64(Some(1))).lt(try_cast(col("c1"), DataType::Int64)); - let predicate_expr = test_build_predicate_expression( - &expr, - &schema, - &mut RequiredStatColumns::new(), - ); + let predicate_expr = + test_build_predicate_expression(&expr, &schema, &mut RequiredColumns::new()); assert_eq!(predicate_expr.to_string(), expected_expr); Ok(()) @@ -1817,11 +1972,8 @@ mod tests { false, )); let expected_expr = "CAST(c1_min@0 AS Int64) <= 1 AND 1 <= CAST(c1_max@1 AS Int64) OR CAST(c1_min@0 AS Int64) <= 2 AND 2 <= CAST(c1_max@1 AS Int64) OR CAST(c1_min@0 AS Int64) <= 3 AND 3 <= CAST(c1_max@1 AS Int64)"; - let predicate_expr = test_build_predicate_expression( - &expr, - &schema, - &mut RequiredStatColumns::new(), - ); + let predicate_expr = + test_build_predicate_expression(&expr, &schema, &mut RequiredColumns::new()); assert_eq!(predicate_expr.to_string(), expected_expr); let expr = Expr::InList(InList::new( @@ -1837,11 +1989,8 @@ mod tests { "(CAST(c1_min@0 AS Int64) != 1 OR 1 != CAST(c1_max@1 AS Int64)) \ AND (CAST(c1_min@0 AS Int64) != 2 OR 2 != CAST(c1_max@1 AS Int64)) \ AND (CAST(c1_min@0 AS Int64) != 3 OR 3 != CAST(c1_max@1 AS Int64))"; - let predicate_expr = test_build_predicate_expression( - &expr, - &schema, - &mut RequiredStatColumns::new(), - ); + let predicate_expr = + test_build_predicate_expression(&expr, &schema, &mut RequiredColumns::new()); assert_eq!(predicate_expr.to_string(), expected_expr); Ok(()) @@ -2484,10 +2633,464 @@ mod tests { // TODO: add other negative test for other case and op } + #[test] + fn prune_with_contained_one_column() { + let schema = Arc::new(Schema::new(vec![Field::new("s1", DataType::Utf8, true)])); + + // Model having information like a bloom filter for s1 + let statistics = TestStatistics::new() + .with_contained( + "s1", + [ScalarValue::from("foo")], + [ + // container 0 known to only contain "foo"", + Some(true), + // container 1 known to not contain "foo" + Some(false), + // container 2 unknown about "foo" + None, + // container 3 known to only contain "foo" + Some(true), + // container 4 known to not contain "foo" + Some(false), + // container 5 unknown about "foo" + None, + // container 6 known to only contain "foo" + Some(true), + // container 7 known to not contain "foo" + Some(false), + // container 8 unknown about "foo" + None, + ], + ) + .with_contained( + "s1", + [ScalarValue::from("bar")], + [ + // containers 0,1,2 known to only contain "bar" + Some(true), + Some(true), + Some(true), + // container 3,4,5 known to not contain "bar" + Some(false), + Some(false), + Some(false), + // container 6,7,8 unknown about "bar" + None, + None, + None, + ], + ) + .with_contained( + // the way the tests are setup, this data is + // consulted if the "foo" and "bar" are being checked at the same time + "s1", + [ScalarValue::from("foo"), ScalarValue::from("bar")], + [ + // container 0,1,2 unknown about ("foo, "bar") + None, + None, + None, + // container 3,4,5 known to contain only either "foo" and "bar" + Some(true), + Some(true), + Some(true), + // container 6,7,8 known to contain neither "foo" and "bar" + Some(false), + Some(false), + Some(false), + ], + ); + + // s1 = 'foo' + prune_with_expr( + col("s1").eq(lit("foo")), + &schema, + &statistics, + // rule out containers ('false) where we know foo is not present + vec![true, false, true, true, false, true, true, false, true], + ); + + // s1 = 'bar' + prune_with_expr( + col("s1").eq(lit("bar")), + &schema, + &statistics, + // rule out containers where we know bar is not present + vec![true, true, true, false, false, false, true, true, true], + ); + + // s1 = 'baz' (unknown value) + prune_with_expr( + col("s1").eq(lit("baz")), + &schema, + &statistics, + // can't rule out anything + vec![true, true, true, true, true, true, true, true, true], + ); + + // s1 = 'foo' AND s1 = 'bar' + prune_with_expr( + col("s1").eq(lit("foo")).and(col("s1").eq(lit("bar"))), + &schema, + &statistics, + // logically this predicate can't possibly be true (the column can't + // take on both values) but we could rule it out if the stats tell + // us that both values are not present + vec![true, true, true, true, true, true, true, true, true], + ); + + // s1 = 'foo' OR s1 = 'bar' + prune_with_expr( + col("s1").eq(lit("foo")).or(col("s1").eq(lit("bar"))), + &schema, + &statistics, + // can rule out containers that we know contain neither foo nor bar + vec![true, true, true, true, true, true, false, false, false], + ); + + // s1 = 'foo' OR s1 = 'baz' + prune_with_expr( + col("s1").eq(lit("foo")).or(col("s1").eq(lit("baz"))), + &schema, + &statistics, + // can't rule out anything container + vec![true, true, true, true, true, true, true, true, true], + ); + + // s1 = 'foo' OR s1 = 'bar' OR s1 = 'baz' + prune_with_expr( + col("s1") + .eq(lit("foo")) + .or(col("s1").eq(lit("bar"))) + .or(col("s1").eq(lit("baz"))), + &schema, + &statistics, + // can rule out any containers based on knowledge of s1 and `foo`, + // `bar` and (`foo`, `bar`) + vec![true, true, true, true, true, true, true, true, true], + ); + + // s1 != foo + prune_with_expr( + col("s1").not_eq(lit("foo")), + &schema, + &statistics, + // rule out containers we know for sure only contain foo + vec![false, true, true, false, true, true, false, true, true], + ); + + // s1 != bar + prune_with_expr( + col("s1").not_eq(lit("bar")), + &schema, + &statistics, + // rule out when we know for sure s1 has the value bar + vec![false, false, false, true, true, true, true, true, true], + ); + + // s1 != foo AND s1 != bar + prune_with_expr( + col("s1") + .not_eq(lit("foo")) + .and(col("s1").not_eq(lit("bar"))), + &schema, + &statistics, + // can rule out any container where we know s1 does not have either 'foo' or 'bar' + vec![true, true, true, false, false, false, true, true, true], + ); + + // s1 != foo AND s1 != bar AND s1 != baz + prune_with_expr( + col("s1") + .not_eq(lit("foo")) + .and(col("s1").not_eq(lit("bar"))) + .and(col("s1").not_eq(lit("baz"))), + &schema, + &statistics, + // can't rule out any container based on knowledge of s1,s2 + vec![true, true, true, true, true, true, true, true, true], + ); + + // s1 != foo OR s1 != bar + prune_with_expr( + col("s1") + .not_eq(lit("foo")) + .or(col("s1").not_eq(lit("bar"))), + &schema, + &statistics, + // cant' rule out anything based on contains information + vec![true, true, true, true, true, true, true, true, true], + ); + + // s1 != foo OR s1 != bar OR s1 != baz + prune_with_expr( + col("s1") + .not_eq(lit("foo")) + .or(col("s1").not_eq(lit("bar"))) + .or(col("s1").not_eq(lit("baz"))), + &schema, + &statistics, + // cant' rule out anything based on contains information + vec![true, true, true, true, true, true, true, true, true], + ); + } + + #[test] + fn prune_with_contained_two_columns() { + let schema = Arc::new(Schema::new(vec![ + Field::new("s1", DataType::Utf8, true), + Field::new("s2", DataType::Utf8, true), + ])); + + // Model having information like bloom filters for s1 and s2 + let statistics = TestStatistics::new() + .with_contained( + "s1", + [ScalarValue::from("foo")], + [ + // container 0, s1 known to only contain "foo"", + Some(true), + // container 1, s1 known to not contain "foo" + Some(false), + // container 2, s1 unknown about "foo" + None, + // container 3, s1 known to only contain "foo" + Some(true), + // container 4, s1 known to not contain "foo" + Some(false), + // container 5, s1 unknown about "foo" + None, + // container 6, s1 known to only contain "foo" + Some(true), + // container 7, s1 known to not contain "foo" + Some(false), + // container 8, s1 unknown about "foo" + None, + ], + ) + .with_contained( + "s2", // for column s2 + [ScalarValue::from("bar")], + [ + // containers 0,1,2 s2 known to only contain "bar" + Some(true), + Some(true), + Some(true), + // container 3,4,5 s2 known to not contain "bar" + Some(false), + Some(false), + Some(false), + // container 6,7,8 s2 unknown about "bar" + None, + None, + None, + ], + ); + + // s1 = 'foo' + prune_with_expr( + col("s1").eq(lit("foo")), + &schema, + &statistics, + // rule out containers where we know s1 is not present + vec![true, false, true, true, false, true, true, false, true], + ); + + // s1 = 'foo' OR s2 = 'bar' + let expr = col("s1").eq(lit("foo")).or(col("s2").eq(lit("bar"))); + prune_with_expr( + expr, + &schema, + &statistics, + // can't rule out any container (would need to prove that s1 != foo AND s2 != bar) + vec![true, true, true, true, true, true, true, true, true], + ); + + // s1 = 'foo' AND s2 != 'bar' + prune_with_expr( + col("s1").eq(lit("foo")).and(col("s2").not_eq(lit("bar"))), + &schema, + &statistics, + // can only rule out container where we know either: + // 1. s1 doesn't have the value 'foo` or + // 2. s2 has only the value of 'bar' + vec![false, false, false, true, false, true, true, false, true], + ); + + // s1 != 'foo' AND s2 != 'bar' + prune_with_expr( + col("s1") + .not_eq(lit("foo")) + .and(col("s2").not_eq(lit("bar"))), + &schema, + &statistics, + // Can rule out any container where we know either + // 1. s1 has only the value 'foo' + // 2. s2 has only the value 'bar' + vec![false, false, false, false, true, true, false, true, true], + ); + + // s1 != 'foo' AND (s2 = 'bar' OR s2 = 'baz') + prune_with_expr( + col("s1") + .not_eq(lit("foo")) + .and(col("s2").eq(lit("bar")).or(col("s2").eq(lit("baz")))), + &schema, + &statistics, + // Can rule out any container where we know s1 has only the value + // 'foo'. Can't use knowledge of s2 and bar to rule out anything + vec![false, true, true, false, true, true, false, true, true], + ); + + // s1 like '%foo%bar%' + prune_with_expr( + col("s1").like(lit("foo%bar%")), + &schema, + &statistics, + // cant rule out anything with information we know + vec![true, true, true, true, true, true, true, true, true], + ); + + // s1 like '%foo%bar%' AND s2 = 'bar' + prune_with_expr( + col("s1") + .like(lit("foo%bar%")) + .and(col("s2").eq(lit("bar"))), + &schema, + &statistics, + // can rule out any container where we know s2 does not have the value 'bar' + vec![true, true, true, false, false, false, true, true, true], + ); + + // s1 like '%foo%bar%' OR s2 = 'bar' + prune_with_expr( + col("s1").like(lit("foo%bar%")).or(col("s2").eq(lit("bar"))), + &schema, + &statistics, + // can't rule out anything (we would have to prove that both the + // like and the equality must be false) + vec![true, true, true, true, true, true, true, true, true], + ); + } + + #[test] + fn prune_with_range_and_contained() { + // Setup mimics range information for i, a bloom filter for s + let schema = Arc::new(Schema::new(vec![ + Field::new("i", DataType::Int32, true), + Field::new("s", DataType::Utf8, true), + ])); + + let statistics = TestStatistics::new() + .with( + "i", + ContainerStats::new_i32( + // Container 0, 3, 6: [-5 to 5] + // Container 1, 4, 7: [10 to 20] + // Container 2, 5, 9: unknown + vec![ + Some(-5), + Some(10), + None, + Some(-5), + Some(10), + None, + Some(-5), + Some(10), + None, + ], // min + vec![ + Some(5), + Some(20), + None, + Some(5), + Some(20), + None, + Some(5), + Some(20), + None, + ], // max + ), + ) + // Add contained information about the s and "foo" + .with_contained( + "s", + [ScalarValue::from("foo")], + [ + // container 0,1,2 known to only contain "foo" + Some(true), + Some(true), + Some(true), + // container 3,4,5 known to not contain "foo" + Some(false), + Some(false), + Some(false), + // container 6,7,8 unknown about "foo" + None, + None, + None, + ], + ); + + // i = 0 and s = 'foo' + prune_with_expr( + col("i").eq(lit(0)).and(col("s").eq(lit("foo"))), + &schema, + &statistics, + // Can rule out container where we know that either: + // 1. 0 is outside the min/max range of i + // 1. s does not contain foo + // (range is false, and contained is false) + vec![true, false, true, false, false, false, true, false, true], + ); + + // i = 0 and s != 'foo' + prune_with_expr( + col("i").eq(lit(0)).and(col("s").not_eq(lit("foo"))), + &schema, + &statistics, + // Can rule out containers where either: + // 1. 0 is outside the min/max range of i + // 2. s only contains foo + vec![false, false, false, true, false, true, true, false, true], + ); + + // i = 0 OR s = 'foo' + prune_with_expr( + col("i").eq(lit(0)).or(col("s").eq(lit("foo"))), + &schema, + &statistics, + // in theory could rule out containers if we had min/max values for + // s as well. But in this case we don't so we can't rule out anything + vec![true, true, true, true, true, true, true, true, true], + ); + } + + /// prunes the specified expr with the specified schema and statistics, and + /// ensures it returns expected. + /// + /// `expected` is a vector of bools, where true means the row group should + /// be kept, and false means it should be pruned. + /// + // TODO refactor other tests to use this to reduce boiler plate + fn prune_with_expr( + expr: Expr, + schema: &SchemaRef, + statistics: &TestStatistics, + expected: Vec, + ) { + println!("Pruning with expr: {}", expr); + let expr = logical2physical(&expr, schema); + let p = PruningPredicate::try_new(expr, schema.clone()).unwrap(); + let result = p.prune(statistics).unwrap(); + assert_eq!(result, expected); + } + fn test_build_predicate_expression( expr: &Expr, schema: &Schema, - required_columns: &mut RequiredStatColumns, + required_columns: &mut RequiredColumns, ) -> Arc { let expr = logical2physical(expr, schema); build_predicate_expression(&expr, schema, required_columns) From bf43bb2eed304369c078637bc84d1b842c24b399 Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Sat, 23 Dec 2023 09:25:07 -0700 Subject: [PATCH 489/572] Add partial serde support for ParquetWriterOptions (#8627) * Add serde support for ParquetWriterOptions * save progress * test passes * Improve test * Refactor and add link to follow on issue * remove duplicate code * clippy * Regen * remove comments from proto file * change proto types from i32 to u32 pre feedback on PR * change to u64 --- datafusion/proto/proto/datafusion.proto | 15 + datafusion/proto/src/generated/pbjson.rs | 321 ++++++++++++++++++ datafusion/proto/src/generated/prost.rs | 28 +- datafusion/proto/src/logical_plan/mod.rs | 146 ++++++-- .../proto/src/physical_plan/from_proto.rs | 7 + .../tests/cases/roundtrip_logical_plan.rs | 41 ++- 6 files changed, 524 insertions(+), 34 deletions(-) diff --git a/datafusion/proto/proto/datafusion.proto b/datafusion/proto/proto/datafusion.proto index 05f0b6434368..d02fc8e91b41 100644 --- a/datafusion/proto/proto/datafusion.proto +++ b/datafusion/proto/proto/datafusion.proto @@ -1206,6 +1206,7 @@ message PartitionColumn { message FileTypeWriterOptions { oneof FileType { JsonWriterOptions json_options = 1; + ParquetWriterOptions parquet_options = 2; } } @@ -1213,6 +1214,20 @@ message JsonWriterOptions { CompressionTypeVariant compression = 1; } +message ParquetWriterOptions { + WriterProperties writer_properties = 1; +} + +message WriterProperties { + uint64 data_page_size_limit = 1; + uint64 dictionary_page_size_limit = 2; + uint64 data_page_row_count_limit = 3; + uint64 write_batch_size = 4; + uint64 max_row_group_size = 5; + string writer_version = 6; + string created_by = 7; +} + message FileSinkConfig { reserved 6; // writer_mode diff --git a/datafusion/proto/src/generated/pbjson.rs b/datafusion/proto/src/generated/pbjson.rs index 0fdeab0a40f6..f860b1f1e6a0 100644 --- a/datafusion/proto/src/generated/pbjson.rs +++ b/datafusion/proto/src/generated/pbjson.rs @@ -7890,6 +7890,9 @@ impl serde::Serialize for FileTypeWriterOptions { file_type_writer_options::FileType::JsonOptions(v) => { struct_ser.serialize_field("jsonOptions", v)?; } + file_type_writer_options::FileType::ParquetOptions(v) => { + struct_ser.serialize_field("parquetOptions", v)?; + } } } struct_ser.end() @@ -7904,11 +7907,14 @@ impl<'de> serde::Deserialize<'de> for FileTypeWriterOptions { const FIELDS: &[&str] = &[ "json_options", "jsonOptions", + "parquet_options", + "parquetOptions", ]; #[allow(clippy::enum_variant_names)] enum GeneratedField { JsonOptions, + ParquetOptions, } impl<'de> serde::Deserialize<'de> for GeneratedField { fn deserialize(deserializer: D) -> std::result::Result @@ -7931,6 +7937,7 @@ impl<'de> serde::Deserialize<'de> for FileTypeWriterOptions { { match value { "jsonOptions" | "json_options" => Ok(GeneratedField::JsonOptions), + "parquetOptions" | "parquet_options" => Ok(GeneratedField::ParquetOptions), _ => Err(serde::de::Error::unknown_field(value, FIELDS)), } } @@ -7958,6 +7965,13 @@ impl<'de> serde::Deserialize<'de> for FileTypeWriterOptions { return Err(serde::de::Error::duplicate_field("jsonOptions")); } file_type__ = map_.next_value::<::std::option::Option<_>>()?.map(file_type_writer_options::FileType::JsonOptions) +; + } + GeneratedField::ParquetOptions => { + if file_type__.is_some() { + return Err(serde::de::Error::duplicate_field("parquetOptions")); + } + file_type__ = map_.next_value::<::std::option::Option<_>>()?.map(file_type_writer_options::FileType::ParquetOptions) ; } } @@ -15171,6 +15185,98 @@ impl<'de> serde::Deserialize<'de> for ParquetScanExecNode { deserializer.deserialize_struct("datafusion.ParquetScanExecNode", FIELDS, GeneratedVisitor) } } +impl serde::Serialize for ParquetWriterOptions { + #[allow(deprecated)] + fn serialize(&self, serializer: S) -> std::result::Result + where + S: serde::Serializer, + { + use serde::ser::SerializeStruct; + let mut len = 0; + if self.writer_properties.is_some() { + len += 1; + } + let mut struct_ser = serializer.serialize_struct("datafusion.ParquetWriterOptions", len)?; + if let Some(v) = self.writer_properties.as_ref() { + struct_ser.serialize_field("writerProperties", v)?; + } + struct_ser.end() + } +} +impl<'de> serde::Deserialize<'de> for ParquetWriterOptions { + #[allow(deprecated)] + fn deserialize(deserializer: D) -> std::result::Result + where + D: serde::Deserializer<'de>, + { + const FIELDS: &[&str] = &[ + "writer_properties", + "writerProperties", + ]; + + #[allow(clippy::enum_variant_names)] + enum GeneratedField { + WriterProperties, + } + impl<'de> serde::Deserialize<'de> for GeneratedField { + fn deserialize(deserializer: D) -> std::result::Result + where + D: serde::Deserializer<'de>, + { + struct GeneratedVisitor; + + impl<'de> serde::de::Visitor<'de> for GeneratedVisitor { + type Value = GeneratedField; + + fn expecting(&self, formatter: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + write!(formatter, "expected one of: {:?}", &FIELDS) + } + + #[allow(unused_variables)] + fn visit_str(self, value: &str) -> std::result::Result + where + E: serde::de::Error, + { + match value { + "writerProperties" | "writer_properties" => Ok(GeneratedField::WriterProperties), + _ => Err(serde::de::Error::unknown_field(value, FIELDS)), + } + } + } + deserializer.deserialize_identifier(GeneratedVisitor) + } + } + struct GeneratedVisitor; + impl<'de> serde::de::Visitor<'de> for GeneratedVisitor { + type Value = ParquetWriterOptions; + + fn expecting(&self, formatter: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + formatter.write_str("struct datafusion.ParquetWriterOptions") + } + + fn visit_map(self, mut map_: V) -> std::result::Result + where + V: serde::de::MapAccess<'de>, + { + let mut writer_properties__ = None; + while let Some(k) = map_.next_key()? { + match k { + GeneratedField::WriterProperties => { + if writer_properties__.is_some() { + return Err(serde::de::Error::duplicate_field("writerProperties")); + } + writer_properties__ = map_.next_value()?; + } + } + } + Ok(ParquetWriterOptions { + writer_properties: writer_properties__, + }) + } + } + deserializer.deserialize_struct("datafusion.ParquetWriterOptions", FIELDS, GeneratedVisitor) + } +} impl serde::Serialize for PartialTableReference { #[allow(deprecated)] fn serialize(&self, serializer: S) -> std::result::Result @@ -27144,3 +27250,218 @@ impl<'de> serde::Deserialize<'de> for WindowNode { deserializer.deserialize_struct("datafusion.WindowNode", FIELDS, GeneratedVisitor) } } +impl serde::Serialize for WriterProperties { + #[allow(deprecated)] + fn serialize(&self, serializer: S) -> std::result::Result + where + S: serde::Serializer, + { + use serde::ser::SerializeStruct; + let mut len = 0; + if self.data_page_size_limit != 0 { + len += 1; + } + if self.dictionary_page_size_limit != 0 { + len += 1; + } + if self.data_page_row_count_limit != 0 { + len += 1; + } + if self.write_batch_size != 0 { + len += 1; + } + if self.max_row_group_size != 0 { + len += 1; + } + if !self.writer_version.is_empty() { + len += 1; + } + if !self.created_by.is_empty() { + len += 1; + } + let mut struct_ser = serializer.serialize_struct("datafusion.WriterProperties", len)?; + if self.data_page_size_limit != 0 { + #[allow(clippy::needless_borrow)] + struct_ser.serialize_field("dataPageSizeLimit", ToString::to_string(&self.data_page_size_limit).as_str())?; + } + if self.dictionary_page_size_limit != 0 { + #[allow(clippy::needless_borrow)] + struct_ser.serialize_field("dictionaryPageSizeLimit", ToString::to_string(&self.dictionary_page_size_limit).as_str())?; + } + if self.data_page_row_count_limit != 0 { + #[allow(clippy::needless_borrow)] + struct_ser.serialize_field("dataPageRowCountLimit", ToString::to_string(&self.data_page_row_count_limit).as_str())?; + } + if self.write_batch_size != 0 { + #[allow(clippy::needless_borrow)] + struct_ser.serialize_field("writeBatchSize", ToString::to_string(&self.write_batch_size).as_str())?; + } + if self.max_row_group_size != 0 { + #[allow(clippy::needless_borrow)] + struct_ser.serialize_field("maxRowGroupSize", ToString::to_string(&self.max_row_group_size).as_str())?; + } + if !self.writer_version.is_empty() { + struct_ser.serialize_field("writerVersion", &self.writer_version)?; + } + if !self.created_by.is_empty() { + struct_ser.serialize_field("createdBy", &self.created_by)?; + } + struct_ser.end() + } +} +impl<'de> serde::Deserialize<'de> for WriterProperties { + #[allow(deprecated)] + fn deserialize(deserializer: D) -> std::result::Result + where + D: serde::Deserializer<'de>, + { + const FIELDS: &[&str] = &[ + "data_page_size_limit", + "dataPageSizeLimit", + "dictionary_page_size_limit", + "dictionaryPageSizeLimit", + "data_page_row_count_limit", + "dataPageRowCountLimit", + "write_batch_size", + "writeBatchSize", + "max_row_group_size", + "maxRowGroupSize", + "writer_version", + "writerVersion", + "created_by", + "createdBy", + ]; + + #[allow(clippy::enum_variant_names)] + enum GeneratedField { + DataPageSizeLimit, + DictionaryPageSizeLimit, + DataPageRowCountLimit, + WriteBatchSize, + MaxRowGroupSize, + WriterVersion, + CreatedBy, + } + impl<'de> serde::Deserialize<'de> for GeneratedField { + fn deserialize(deserializer: D) -> std::result::Result + where + D: serde::Deserializer<'de>, + { + struct GeneratedVisitor; + + impl<'de> serde::de::Visitor<'de> for GeneratedVisitor { + type Value = GeneratedField; + + fn expecting(&self, formatter: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + write!(formatter, "expected one of: {:?}", &FIELDS) + } + + #[allow(unused_variables)] + fn visit_str(self, value: &str) -> std::result::Result + where + E: serde::de::Error, + { + match value { + "dataPageSizeLimit" | "data_page_size_limit" => Ok(GeneratedField::DataPageSizeLimit), + "dictionaryPageSizeLimit" | "dictionary_page_size_limit" => Ok(GeneratedField::DictionaryPageSizeLimit), + "dataPageRowCountLimit" | "data_page_row_count_limit" => Ok(GeneratedField::DataPageRowCountLimit), + "writeBatchSize" | "write_batch_size" => Ok(GeneratedField::WriteBatchSize), + "maxRowGroupSize" | "max_row_group_size" => Ok(GeneratedField::MaxRowGroupSize), + "writerVersion" | "writer_version" => Ok(GeneratedField::WriterVersion), + "createdBy" | "created_by" => Ok(GeneratedField::CreatedBy), + _ => Err(serde::de::Error::unknown_field(value, FIELDS)), + } + } + } + deserializer.deserialize_identifier(GeneratedVisitor) + } + } + struct GeneratedVisitor; + impl<'de> serde::de::Visitor<'de> for GeneratedVisitor { + type Value = WriterProperties; + + fn expecting(&self, formatter: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + formatter.write_str("struct datafusion.WriterProperties") + } + + fn visit_map(self, mut map_: V) -> std::result::Result + where + V: serde::de::MapAccess<'de>, + { + let mut data_page_size_limit__ = None; + let mut dictionary_page_size_limit__ = None; + let mut data_page_row_count_limit__ = None; + let mut write_batch_size__ = None; + let mut max_row_group_size__ = None; + let mut writer_version__ = None; + let mut created_by__ = None; + while let Some(k) = map_.next_key()? { + match k { + GeneratedField::DataPageSizeLimit => { + if data_page_size_limit__.is_some() { + return Err(serde::de::Error::duplicate_field("dataPageSizeLimit")); + } + data_page_size_limit__ = + Some(map_.next_value::<::pbjson::private::NumberDeserialize<_>>()?.0) + ; + } + GeneratedField::DictionaryPageSizeLimit => { + if dictionary_page_size_limit__.is_some() { + return Err(serde::de::Error::duplicate_field("dictionaryPageSizeLimit")); + } + dictionary_page_size_limit__ = + Some(map_.next_value::<::pbjson::private::NumberDeserialize<_>>()?.0) + ; + } + GeneratedField::DataPageRowCountLimit => { + if data_page_row_count_limit__.is_some() { + return Err(serde::de::Error::duplicate_field("dataPageRowCountLimit")); + } + data_page_row_count_limit__ = + Some(map_.next_value::<::pbjson::private::NumberDeserialize<_>>()?.0) + ; + } + GeneratedField::WriteBatchSize => { + if write_batch_size__.is_some() { + return Err(serde::de::Error::duplicate_field("writeBatchSize")); + } + write_batch_size__ = + Some(map_.next_value::<::pbjson::private::NumberDeserialize<_>>()?.0) + ; + } + GeneratedField::MaxRowGroupSize => { + if max_row_group_size__.is_some() { + return Err(serde::de::Error::duplicate_field("maxRowGroupSize")); + } + max_row_group_size__ = + Some(map_.next_value::<::pbjson::private::NumberDeserialize<_>>()?.0) + ; + } + GeneratedField::WriterVersion => { + if writer_version__.is_some() { + return Err(serde::de::Error::duplicate_field("writerVersion")); + } + writer_version__ = Some(map_.next_value()?); + } + GeneratedField::CreatedBy => { + if created_by__.is_some() { + return Err(serde::de::Error::duplicate_field("createdBy")); + } + created_by__ = Some(map_.next_value()?); + } + } + } + Ok(WriterProperties { + data_page_size_limit: data_page_size_limit__.unwrap_or_default(), + dictionary_page_size_limit: dictionary_page_size_limit__.unwrap_or_default(), + data_page_row_count_limit: data_page_row_count_limit__.unwrap_or_default(), + write_batch_size: write_batch_size__.unwrap_or_default(), + max_row_group_size: max_row_group_size__.unwrap_or_default(), + writer_version: writer_version__.unwrap_or_default(), + created_by: created_by__.unwrap_or_default(), + }) + } + } + deserializer.deserialize_struct("datafusion.WriterProperties", FIELDS, GeneratedVisitor) + } +} diff --git a/datafusion/proto/src/generated/prost.rs b/datafusion/proto/src/generated/prost.rs index e44355859d65..459d5a965cd3 100644 --- a/datafusion/proto/src/generated/prost.rs +++ b/datafusion/proto/src/generated/prost.rs @@ -1642,7 +1642,7 @@ pub struct PartitionColumn { #[allow(clippy::derive_partial_eq_without_eq)] #[derive(Clone, PartialEq, ::prost::Message)] pub struct FileTypeWriterOptions { - #[prost(oneof = "file_type_writer_options::FileType", tags = "1")] + #[prost(oneof = "file_type_writer_options::FileType", tags = "1, 2")] pub file_type: ::core::option::Option, } /// Nested message and enum types in `FileTypeWriterOptions`. @@ -1652,6 +1652,8 @@ pub mod file_type_writer_options { pub enum FileType { #[prost(message, tag = "1")] JsonOptions(super::JsonWriterOptions), + #[prost(message, tag = "2")] + ParquetOptions(super::ParquetWriterOptions), } } #[allow(clippy::derive_partial_eq_without_eq)] @@ -1662,6 +1664,30 @@ pub struct JsonWriterOptions { } #[allow(clippy::derive_partial_eq_without_eq)] #[derive(Clone, PartialEq, ::prost::Message)] +pub struct ParquetWriterOptions { + #[prost(message, optional, tag = "1")] + pub writer_properties: ::core::option::Option, +} +#[allow(clippy::derive_partial_eq_without_eq)] +#[derive(Clone, PartialEq, ::prost::Message)] +pub struct WriterProperties { + #[prost(uint64, tag = "1")] + pub data_page_size_limit: u64, + #[prost(uint64, tag = "2")] + pub dictionary_page_size_limit: u64, + #[prost(uint64, tag = "3")] + pub data_page_row_count_limit: u64, + #[prost(uint64, tag = "4")] + pub write_batch_size: u64, + #[prost(uint64, tag = "5")] + pub max_row_group_size: u64, + #[prost(string, tag = "6")] + pub writer_version: ::prost::alloc::string::String, + #[prost(string, tag = "7")] + pub created_by: ::prost::alloc::string::String, +} +#[allow(clippy::derive_partial_eq_without_eq)] +#[derive(Clone, PartialEq, ::prost::Message)] pub struct FileSinkConfig { #[prost(string, tag = "1")] pub object_store_url: ::prost::alloc::string::String, diff --git a/datafusion/proto/src/logical_plan/mod.rs b/datafusion/proto/src/logical_plan/mod.rs index e03b3ffa7b84..d137a41fa19b 100644 --- a/datafusion/proto/src/logical_plan/mod.rs +++ b/datafusion/proto/src/logical_plan/mod.rs @@ -23,7 +23,8 @@ use std::sync::Arc; use crate::common::{byte_to_string, proto_error, str_to_byte}; use crate::protobuf::logical_plan_node::LogicalPlanType::CustomScan; use crate::protobuf::{ - copy_to_node, CustomTableScanNode, LogicalExprNodeCollection, SqlOption, + copy_to_node, file_type_writer_options, CustomTableScanNode, + LogicalExprNodeCollection, SqlOption, }; use crate::{ convert_required, @@ -49,7 +50,7 @@ use datafusion::{ use datafusion_common::{ context, file_options::StatementOptions, internal_err, not_impl_err, parsers::CompressionTypeVariant, plan_datafusion_err, DataFusionError, FileType, - OwnedTableReference, Result, + FileTypeWriterOptions, OwnedTableReference, Result, }; use datafusion_expr::{ dml, @@ -62,6 +63,8 @@ use datafusion_expr::{ DistinctOn, DropView, Expr, LogicalPlan, LogicalPlanBuilder, }; +use datafusion::parquet::file::properties::{WriterProperties, WriterVersion}; +use datafusion_common::file_options::parquet_writer::ParquetWriterOptions; use datafusion_expr::dml::CopyOptions; use prost::bytes::BufMut; use prost::Message; @@ -833,19 +836,48 @@ impl AsLogicalPlan for LogicalPlanNode { let copy_options = match ©.copy_options { Some(copy_to_node::CopyOptions::SqlOptions(opt)) => { - let options = opt.option.iter().map(|o| (o.key.clone(), o.value.clone())).collect(); - CopyOptions::SQLOptions(StatementOptions::from( - &options, - )) + let options = opt + .option + .iter() + .map(|o| (o.key.clone(), o.value.clone())) + .collect(); + CopyOptions::SQLOptions(StatementOptions::from(&options)) } - Some(copy_to_node::CopyOptions::WriterOptions(_)) => { - return Err(proto_error( - "LogicalPlan serde is not yet implemented for CopyTo with WriterOptions", - )) + Some(copy_to_node::CopyOptions::WriterOptions(opt)) => { + match &opt.file_type { + Some(ft) => match ft { + file_type_writer_options::FileType::ParquetOptions( + writer_options, + ) => { + let writer_properties = + match &writer_options.writer_properties { + Some(serialized_writer_options) => { + writer_properties_from_proto( + serialized_writer_options, + )? + } + _ => WriterProperties::default(), + }; + CopyOptions::WriterOptions(Box::new( + FileTypeWriterOptions::Parquet( + ParquetWriterOptions::new(writer_properties), + ), + )) + } + _ => { + return Err(proto_error( + "WriterOptions unsupported file_type", + )) + } + }, + None => { + return Err(proto_error( + "WriterOptions missing file_type", + )) + } + } } - other => return Err(proto_error(format!( - "LogicalPlan serde is not yet implemented for CopyTo with CopyOptions {other:?}", - ))) + None => return Err(proto_error("CopyTo missing CopyOptions")), }; Ok(datafusion_expr::LogicalPlan::Copy( datafusion_expr::dml::CopyTo { @@ -1580,22 +1612,48 @@ impl AsLogicalPlan for LogicalPlanNode { extension_codec, )?; - let copy_options_proto: Option = match copy_options { - CopyOptions::SQLOptions(opt) => { - let options: Vec = opt.clone().into_inner().iter().map(|(k, v)| SqlOption { - key: k.to_string(), - value: v.to_string(), - }).collect(); - Some(copy_to_node::CopyOptions::SqlOptions(protobuf::SqlOptions { - option: options - })) - } - CopyOptions::WriterOptions(_) => { - return Err(proto_error( - "LogicalPlan serde is not yet implemented for CopyTo with WriterOptions", - )) - } - }; + let copy_options_proto: Option = + match copy_options { + CopyOptions::SQLOptions(opt) => { + let options: Vec = opt + .clone() + .into_inner() + .iter() + .map(|(k, v)| SqlOption { + key: k.to_string(), + value: v.to_string(), + }) + .collect(); + Some(copy_to_node::CopyOptions::SqlOptions( + protobuf::SqlOptions { option: options }, + )) + } + CopyOptions::WriterOptions(opt) => { + match opt.as_ref() { + FileTypeWriterOptions::Parquet(parquet_opts) => { + let parquet_writer_options = + protobuf::ParquetWriterOptions { + writer_properties: Some( + writer_properties_to_proto( + &parquet_opts.writer_options, + ), + ), + }; + let parquet_options = file_type_writer_options::FileType::ParquetOptions(parquet_writer_options); + Some(copy_to_node::CopyOptions::WriterOptions( + protobuf::FileTypeWriterOptions { + file_type: Some(parquet_options), + }, + )) + } + _ => { + return Err(proto_error( + "Unsupported FileTypeWriterOptions in CopyTo", + )) + } + } + } + }; Ok(protobuf::LogicalPlanNode { logical_plan_type: Some(LogicalPlanType::CopyTo(Box::new( @@ -1615,3 +1673,33 @@ impl AsLogicalPlan for LogicalPlanNode { } } } + +pub(crate) fn writer_properties_to_proto( + props: &WriterProperties, +) -> protobuf::WriterProperties { + protobuf::WriterProperties { + data_page_size_limit: props.data_page_size_limit() as u64, + dictionary_page_size_limit: props.dictionary_page_size_limit() as u64, + data_page_row_count_limit: props.data_page_row_count_limit() as u64, + write_batch_size: props.write_batch_size() as u64, + max_row_group_size: props.max_row_group_size() as u64, + writer_version: format!("{:?}", props.writer_version()), + created_by: props.created_by().to_string(), + } +} + +pub(crate) fn writer_properties_from_proto( + props: &protobuf::WriterProperties, +) -> Result { + let writer_version = WriterVersion::from_str(&props.writer_version) + .map_err(|e| proto_error(e.to_string()))?; + Ok(WriterProperties::builder() + .set_created_by(props.created_by.clone()) + .set_writer_version(writer_version) + .set_dictionary_page_size_limit(props.dictionary_page_size_limit as usize) + .set_data_page_row_count_limit(props.data_page_row_count_limit as usize) + .set_data_page_size_limit(props.data_page_size_limit as usize) + .set_write_batch_size(props.write_batch_size as usize) + .set_max_row_group_size(props.max_row_group_size as usize) + .build()) +} diff --git a/datafusion/proto/src/physical_plan/from_proto.rs b/datafusion/proto/src/physical_plan/from_proto.rs index 65f9f139a87b..824eb60a5715 100644 --- a/datafusion/proto/src/physical_plan/from_proto.rs +++ b/datafusion/proto/src/physical_plan/from_proto.rs @@ -40,6 +40,7 @@ use datafusion::physical_plan::{ functions, ColumnStatistics, Partitioning, PhysicalExpr, Statistics, WindowExpr, }; use datafusion_common::file_options::json_writer::JsonWriterOptions; +use datafusion_common::file_options::parquet_writer::ParquetWriterOptions; use datafusion_common::parsers::CompressionTypeVariant; use datafusion_common::stats::Precision; use datafusion_common::{ @@ -52,6 +53,7 @@ use crate::logical_plan; use crate::protobuf; use crate::protobuf::physical_expr_node::ExprType; +use crate::logical_plan::writer_properties_from_proto; use chrono::{TimeZone, Utc}; use object_store::path::Path; use object_store::ObjectMeta; @@ -769,6 +771,11 @@ impl TryFrom<&protobuf::FileTypeWriterOptions> for FileTypeWriterOptions { protobuf::file_type_writer_options::FileType::JsonOptions(opts) => Ok( Self::JSON(JsonWriterOptions::new(opts.compression().into())), ), + protobuf::file_type_writer_options::FileType::ParquetOptions(opt) => { + let props = opt.writer_properties.clone().unwrap_or_default(); + let writer_properties = writer_properties_from_proto(&props)?; + Ok(Self::Parquet(ParquetWriterOptions::new(writer_properties))) + } } } } diff --git a/datafusion/proto/tests/cases/roundtrip_logical_plan.rs b/datafusion/proto/tests/cases/roundtrip_logical_plan.rs index 9798b06f4724..3eeae01a643e 100644 --- a/datafusion/proto/tests/cases/roundtrip_logical_plan.rs +++ b/datafusion/proto/tests/cases/roundtrip_logical_plan.rs @@ -31,7 +31,7 @@ use datafusion::datasource::provider::TableProviderFactory; use datafusion::datasource::TableProvider; use datafusion::execution::context::SessionState; use datafusion::execution::runtime_env::{RuntimeConfig, RuntimeEnv}; -use datafusion::parquet::file::properties::WriterProperties; +use datafusion::parquet::file::properties::{WriterProperties, WriterVersion}; use datafusion::physical_plan::functions::make_scalar_function; use datafusion::prelude::{create_udf, CsvReadOptions, SessionConfig, SessionContext}; use datafusion::test_util::{TestTableFactory, TestTableProvider}; @@ -330,7 +330,6 @@ async fn roundtrip_logical_plan_copy_to_sql_options() -> Result<()> { } #[tokio::test] -#[ignore] // see https://github.com/apache/arrow-datafusion/issues/8619 async fn roundtrip_logical_plan_copy_to_writer_options() -> Result<()> { let ctx = SessionContext::new(); @@ -339,11 +338,17 @@ async fn roundtrip_logical_plan_copy_to_writer_options() -> Result<()> { let writer_properties = WriterProperties::builder() .set_bloom_filter_enabled(true) .set_created_by("DataFusion Test".to_string()) + .set_writer_version(WriterVersion::PARQUET_2_0) + .set_write_batch_size(111) + .set_data_page_size_limit(222) + .set_data_page_row_count_limit(333) + .set_dictionary_page_size_limit(444) + .set_max_row_group_size(555) .build(); let plan = LogicalPlan::Copy(CopyTo { input: Arc::new(input), - output_url: "test.csv".to_string(), - file_format: FileType::CSV, + output_url: "test.parquet".to_string(), + file_format: FileType::PARQUET, single_file_output: true, copy_options: CopyOptions::WriterOptions(Box::new( FileTypeWriterOptions::Parquet(ParquetWriterOptions::new(writer_properties)), @@ -354,6 +359,34 @@ async fn roundtrip_logical_plan_copy_to_writer_options() -> Result<()> { let logical_round_trip = logical_plan_from_bytes(&bytes, &ctx)?; assert_eq!(format!("{plan:?}"), format!("{logical_round_trip:?}")); + match logical_round_trip { + LogicalPlan::Copy(copy_to) => { + assert_eq!("test.parquet", copy_to.output_url); + assert_eq!(FileType::PARQUET, copy_to.file_format); + assert!(copy_to.single_file_output); + match ©_to.copy_options { + CopyOptions::WriterOptions(y) => match y.as_ref() { + FileTypeWriterOptions::Parquet(p) => { + let props = &p.writer_options; + assert_eq!("DataFusion Test", props.created_by()); + assert_eq!( + "PARQUET_2_0", + format!("{:?}", props.writer_version()) + ); + assert_eq!(111, props.write_batch_size()); + assert_eq!(222, props.data_page_size_limit()); + assert_eq!(333, props.data_page_row_count_limit()); + assert_eq!(444, props.dictionary_page_size_limit()); + assert_eq!(555, props.max_row_group_size()); + } + _ => panic!(), + }, + _ => panic!(), + } + } + _ => panic!(), + } + Ok(()) } From 7443f30fc020cca05af74e22d2b5f42ebfe9604e Mon Sep 17 00:00:00 2001 From: Alex Huang Date: Sat, 23 Dec 2023 19:39:03 +0100 Subject: [PATCH 490/572] add arguments length check (#8622) --- .../physical-expr/src/array_expressions.rs | 110 +++++++++++++++++- 1 file changed, 107 insertions(+), 3 deletions(-) diff --git a/datafusion/physical-expr/src/array_expressions.rs b/datafusion/physical-expr/src/array_expressions.rs index 4dfc157e53c7..3ee99d7e8e55 100644 --- a/datafusion/physical-expr/src/array_expressions.rs +++ b/datafusion/physical-expr/src/array_expressions.rs @@ -472,6 +472,10 @@ where /// For example: /// > array_element(\[1, 2, 3], 2) -> 2 pub fn array_element(args: &[ArrayRef]) -> Result { + if args.len() != 2 { + return exec_err!("array_element needs two arguments"); + } + match &args[0].data_type() { DataType::List(_) => { let array = as_list_array(&args[0])?; @@ -585,6 +589,10 @@ pub fn array_except(args: &[ArrayRef]) -> Result { /// /// See test cases in `array.slt` for more details. pub fn array_slice(args: &[ArrayRef]) -> Result { + if args.len() != 3 { + return exec_err!("array_slice needs three arguments"); + } + let array_data_type = args[0].data_type(); match array_data_type { DataType::List(_) => { @@ -736,6 +744,10 @@ where /// array_pop_back SQL function pub fn array_pop_back(args: &[ArrayRef]) -> Result { + if args.len() != 1 { + return exec_err!("array_pop_back needs one argument"); + } + let list_array = as_list_array(&args[0])?; let from_array = Int64Array::from(vec![1; list_array.len()]); let to_array = Int64Array::from( @@ -885,6 +897,10 @@ pub fn array_pop_front(args: &[ArrayRef]) -> Result { /// Array_append SQL function pub fn array_append(args: &[ArrayRef]) -> Result { + if args.len() != 2 { + return exec_err!("array_append expects two arguments"); + } + let list_array = as_list_array(&args[0])?; let element_array = &args[1]; @@ -911,6 +927,10 @@ pub fn array_append(args: &[ArrayRef]) -> Result { /// Array_sort SQL function pub fn array_sort(args: &[ArrayRef]) -> Result { + if args.is_empty() || args.len() > 3 { + return exec_err!("array_sort expects one to three arguments"); + } + let sort_option = match args.len() { 1 => None, 2 => { @@ -990,6 +1010,10 @@ fn order_nulls_first(modifier: &str) -> Result { /// Array_prepend SQL function pub fn array_prepend(args: &[ArrayRef]) -> Result { + if args.len() != 2 { + return exec_err!("array_prepend expects two arguments"); + } + let list_array = as_list_array(&args[1])?; let element_array = &args[0]; @@ -1110,6 +1134,10 @@ fn concat_internal(args: &[ArrayRef]) -> Result { /// Array_concat/Array_cat SQL function pub fn array_concat(args: &[ArrayRef]) -> Result { + if args.is_empty() { + return exec_err!("array_concat expects at least one arguments"); + } + let mut new_args = vec![]; for arg in args { let ndim = list_ndims(arg.data_type()); @@ -1126,6 +1154,10 @@ pub fn array_concat(args: &[ArrayRef]) -> Result { /// Array_empty SQL function pub fn array_empty(args: &[ArrayRef]) -> Result { + if args.len() != 1 { + return exec_err!("array_empty expects one argument"); + } + if as_null_array(&args[0]).is_ok() { // Make sure to return Boolean type. return Ok(Arc::new(BooleanArray::new_null(args[0].len()))); @@ -1150,6 +1182,10 @@ fn array_empty_dispatch(array: &ArrayRef) -> Result Result { + if args.len() != 2 { + return exec_err!("array_repeat expects two arguments"); + } + let element = &args[0]; let count_array = as_int64_array(&args[1])?; @@ -1285,6 +1321,10 @@ fn general_list_repeat( /// Array_position SQL function pub fn array_position(args: &[ArrayRef]) -> Result { + if args.len() < 2 || args.len() > 3 { + return exec_err!("array_position expects two or three arguments"); + } + let list_array = as_list_array(&args[0])?; let element_array = &args[1]; @@ -1349,6 +1389,10 @@ fn general_position( /// Array_positions SQL function pub fn array_positions(args: &[ArrayRef]) -> Result { + if args.len() != 2 { + return exec_err!("array_positions expects two arguments"); + } + let element = &args[1]; match &args[0].data_type() { @@ -1508,16 +1552,28 @@ fn array_remove_internal( } pub fn array_remove_all(args: &[ArrayRef]) -> Result { + if args.len() != 2 { + return exec_err!("array_remove_all expects two arguments"); + } + let arr_n = vec![i64::MAX; args[0].len()]; array_remove_internal(&args[0], &args[1], arr_n) } pub fn array_remove(args: &[ArrayRef]) -> Result { + if args.len() != 2 { + return exec_err!("array_remove expects two arguments"); + } + let arr_n = vec![1; args[0].len()]; array_remove_internal(&args[0], &args[1], arr_n) } pub fn array_remove_n(args: &[ArrayRef]) -> Result { + if args.len() != 3 { + return exec_err!("array_remove_n expects three arguments"); + } + let arr_n = as_int64_array(&args[2])?.values().to_vec(); array_remove_internal(&args[0], &args[1], arr_n) } @@ -1634,6 +1690,10 @@ fn general_replace( } pub fn array_replace(args: &[ArrayRef]) -> Result { + if args.len() != 3 { + return exec_err!("array_replace expects three arguments"); + } + // replace at most one occurence for each element let arr_n = vec![1; args[0].len()]; let array = &args[0]; @@ -1651,6 +1711,10 @@ pub fn array_replace(args: &[ArrayRef]) -> Result { } pub fn array_replace_n(args: &[ArrayRef]) -> Result { + if args.len() != 4 { + return exec_err!("array_replace_n expects four arguments"); + } + // replace the specified number of occurences let arr_n = as_int64_array(&args[3])?.values().to_vec(); let array = &args[0]; @@ -1670,6 +1734,10 @@ pub fn array_replace_n(args: &[ArrayRef]) -> Result { } pub fn array_replace_all(args: &[ArrayRef]) -> Result { + if args.len() != 3 { + return exec_err!("array_replace_all expects three arguments"); + } + // replace all occurrences (up to "i64::MAX") let arr_n = vec![i64::MAX; args[0].len()]; let array = &args[0]; @@ -1760,7 +1828,7 @@ fn union_generic_lists( /// Array_union SQL function pub fn array_union(args: &[ArrayRef]) -> Result { if args.len() != 2 { - return exec_err!("array_union needs two arguments"); + return exec_err!("array_union needs 2 arguments"); } let array1 = &args[0]; let array2 = &args[1]; @@ -1802,6 +1870,10 @@ pub fn array_union(args: &[ArrayRef]) -> Result { /// Array_to_string SQL function pub fn array_to_string(args: &[ArrayRef]) -> Result { + if args.len() < 2 || args.len() > 3 { + return exec_err!("array_to_string expects two or three arguments"); + } + let arr = &args[0]; let delimiters = as_string_array(&args[1])?; @@ -1911,6 +1983,10 @@ pub fn array_to_string(args: &[ArrayRef]) -> Result { /// Cardinality SQL function pub fn cardinality(args: &[ArrayRef]) -> Result { + if args.len() != 1 { + return exec_err!("cardinality expects one argument"); + } + let list_array = as_list_array(&args[0])?.clone(); let result = list_array @@ -1967,6 +2043,10 @@ fn flatten_internal( /// Flatten SQL function pub fn flatten(args: &[ArrayRef]) -> Result { + if args.len() != 1 { + return exec_err!("flatten expects one argument"); + } + let flattened_array = flatten_internal(&args[0], None)?; Ok(Arc::new(flattened_array) as ArrayRef) } @@ -1991,6 +2071,10 @@ fn array_length_dispatch(array: &[ArrayRef]) -> Result Result { + if args.len() != 1 && args.len() != 2 { + return exec_err!("array_length expects one or two arguments"); + } + match &args[0].data_type() { DataType::List(_) => array_length_dispatch::(args), DataType::LargeList(_) => array_length_dispatch::(args), @@ -2037,6 +2121,10 @@ pub fn array_dims(args: &[ArrayRef]) -> Result { /// Array_ndims SQL function pub fn array_ndims(args: &[ArrayRef]) -> Result { + if args.len() != 1 { + return exec_err!("array_ndims needs one argument"); + } + if let Some(list_array) = args[0].as_list_opt::() { let ndims = datafusion_common::utils::list_ndims(list_array.data_type()); @@ -2127,6 +2215,10 @@ fn general_array_has_dispatch( /// Array_has SQL function pub fn array_has(args: &[ArrayRef]) -> Result { + if args.len() != 2 { + return exec_err!("array_has needs two arguments"); + } + let array_type = args[0].data_type(); match array_type { @@ -2142,6 +2234,10 @@ pub fn array_has(args: &[ArrayRef]) -> Result { /// Array_has_any SQL function pub fn array_has_any(args: &[ArrayRef]) -> Result { + if args.len() != 2 { + return exec_err!("array_has_any needs two arguments"); + } + let array_type = args[0].data_type(); match array_type { @@ -2157,6 +2253,10 @@ pub fn array_has_any(args: &[ArrayRef]) -> Result { /// Array_has_all SQL function pub fn array_has_all(args: &[ArrayRef]) -> Result { + if args.len() != 2 { + return exec_err!("array_has_all needs two arguments"); + } + let array_type = args[0].data_type(); match array_type { @@ -2261,7 +2361,9 @@ pub fn string_to_array(args: &[ArrayRef]) -> Result Result { - assert_eq!(args.len(), 2); + if args.len() != 2 { + return exec_err!("array_intersect needs two arguments"); + } let first_array = &args[0]; let second_array = &args[1]; @@ -2364,7 +2466,9 @@ pub fn general_array_distinct( /// array_distinct SQL function /// example: from list [1, 3, 2, 3, 1, 2, 4] to [1, 2, 3, 4] pub fn array_distinct(args: &[ArrayRef]) -> Result { - assert_eq!(args.len(), 1); + if args.len() != 1 { + return exec_err!("array_distinct needs one argument"); + } // handle null if args[0].data_type() == &DataType::Null { From 69e5382aaac8dff6b163de68abc8a46f8780791a Mon Sep 17 00:00:00 2001 From: Andrew Lamb Date: Sat, 23 Dec 2023 13:41:59 -0500 Subject: [PATCH 491/572] Improve DataFrame functional tests (#8630) --- datafusion/core/src/dataframe/mod.rs | 220 ++++++++++----------------- 1 file changed, 82 insertions(+), 138 deletions(-) diff --git a/datafusion/core/src/dataframe/mod.rs b/datafusion/core/src/dataframe/mod.rs index 4b8a9c5b7d79..2ae4a7c21a9c 100644 --- a/datafusion/core/src/dataframe/mod.rs +++ b/datafusion/core/src/dataframe/mod.rs @@ -1356,15 +1356,30 @@ mod tests { use arrow::array::{self, Int32Array}; use arrow::datatypes::DataType; - use datafusion_common::{Constraint, Constraints, ScalarValue}; + use datafusion_common::{Constraint, Constraints}; use datafusion_expr::{ avg, cast, count, count_distinct, create_udf, expr, lit, max, min, sum, - BinaryExpr, BuiltInWindowFunction, Operator, ScalarFunctionImplementation, - Volatility, WindowFrame, WindowFunction, + BuiltInWindowFunction, ScalarFunctionImplementation, Volatility, WindowFrame, + WindowFunction, }; use datafusion_physical_expr::expressions::Column; use datafusion_physical_plan::get_plan_string; + // Get string representation of the plan + async fn assert_physical_plan(df: &DataFrame, expected: Vec<&str>) { + let physical_plan = df + .clone() + .create_physical_plan() + .await + .expect("Error creating physical plan"); + + let actual = get_plan_string(&physical_plan); + assert_eq!( + expected, actual, + "\n**Optimized Plan Mismatch\n\nexpected:\n\n{expected:#?}\nactual:\n\n{actual:#?}\n\n" + ); + } + pub fn table_with_constraints() -> Arc { let dual_schema = Arc::new(Schema::new(vec![ Field::new("id", DataType::Int32, false), @@ -1587,47 +1602,36 @@ mod tests { let config = SessionConfig::new().with_target_partitions(1); let ctx = SessionContext::new_with_config(config); - let table1 = table_with_constraints(); - let df = ctx.read_table(table1)?; - let col_id = Expr::Column(datafusion_common::Column { - relation: None, - name: "id".to_string(), - }); - let col_name = Expr::Column(datafusion_common::Column { - relation: None, - name: "name".to_string(), - }); + let df = ctx.read_table(table_with_constraints())?; - // group by contains id column - let group_expr = vec![col_id.clone()]; + // GROUP BY id + let group_expr = vec![col("id")]; let aggr_expr = vec![]; let df = df.aggregate(group_expr, aggr_expr)?; - // expr list contains id, name - let expr_list = vec![col_id, col_name]; - let df = df.select(expr_list)?; - let physical_plan = df.clone().create_physical_plan().await?; - let expected = vec![ - "AggregateExec: mode=Single, gby=[id@0 as id, name@1 as name], aggr=[]", - " MemoryExec: partitions=1, partition_sizes=[1]", - ]; - // Get string representation of the plan - let actual = get_plan_string(&physical_plan); - assert_eq!( - expected, actual, - "\n**Optimized Plan Mismatch\n\nexpected:\n\n{expected:#?}\nactual:\n\n{actual:#?}\n\n" - ); - // Since id and name are functionally dependant, we can use name among expression - // even if it is not part of the group by expression. - let df_results = collect(physical_plan, ctx.task_ctx()).await?; + // Since id and name are functionally dependant, we can use name among + // expression even if it is not part of the group by expression and can + // select "name" column even though it wasn't explicitly grouped + let df = df.select(vec![col("id"), col("name")])?; + assert_physical_plan( + &df, + vec![ + "AggregateExec: mode=Single, gby=[id@0 as id, name@1 as name], aggr=[]", + " MemoryExec: partitions=1, partition_sizes=[1]", + ], + ) + .await; + + let df_results = df.collect().await?; #[rustfmt::skip] - assert_batches_sorted_eq!( - ["+----+------+", + assert_batches_sorted_eq!([ + "+----+------+", "| id | name |", "+----+------+", "| 1 | a |", - "+----+------+",], + "+----+------+" + ], &df_results ); @@ -1640,57 +1644,31 @@ mod tests { let config = SessionConfig::new().with_target_partitions(1); let ctx = SessionContext::new_with_config(config); - let table1 = table_with_constraints(); - let df = ctx.read_table(table1)?; - let col_id = Expr::Column(datafusion_common::Column { - relation: None, - name: "id".to_string(), - }); - let col_name = Expr::Column(datafusion_common::Column { - relation: None, - name: "name".to_string(), - }); + let df = ctx.read_table(table_with_constraints())?; - // group by contains id column - let group_expr = vec![col_id.clone()]; + // GROUP BY id + let group_expr = vec![col("id")]; let aggr_expr = vec![]; let df = df.aggregate(group_expr, aggr_expr)?; - let condition1 = Expr::BinaryExpr(BinaryExpr::new( - Box::new(col_id.clone()), - Operator::Eq, - Box::new(Expr::Literal(ScalarValue::Int32(Some(1)))), - )); - let condition2 = Expr::BinaryExpr(BinaryExpr::new( - Box::new(col_name), - Operator::Eq, - Box::new(Expr::Literal(ScalarValue::Utf8(Some("a".to_string())))), - )); - // Predicate refers to id, and name fields - let predicate = Expr::BinaryExpr(BinaryExpr::new( - Box::new(condition1), - Operator::And, - Box::new(condition2), - )); + // Predicate refers to id, and name fields: + // id = 1 AND name = 'a' + let predicate = col("id").eq(lit(1i32)).and(col("name").eq(lit("a"))); let df = df.filter(predicate)?; - let physical_plan = df.clone().create_physical_plan().await?; - - let expected = vec![ + assert_physical_plan( + &df, + vec![ "CoalesceBatchesExec: target_batch_size=8192", " FilterExec: id@0 = 1 AND name@1 = a", " AggregateExec: mode=Single, gby=[id@0 as id, name@1 as name], aggr=[]", " MemoryExec: partitions=1, partition_sizes=[1]", - ]; - // Get string representation of the plan - let actual = get_plan_string(&physical_plan); - assert_eq!( - expected, actual, - "\n**Optimized Plan Mismatch\n\nexpected:\n\n{expected:#?}\nactual:\n\n{actual:#?}\n\n" - ); + ], + ) + .await; // Since id and name are functionally dependant, we can use name among expression // even if it is not part of the group by expression. - let df_results = collect(physical_plan, ctx.task_ctx()).await?; + let df_results = df.collect().await?; #[rustfmt::skip] assert_batches_sorted_eq!( @@ -1711,53 +1689,35 @@ mod tests { let config = SessionConfig::new().with_target_partitions(1); let ctx = SessionContext::new_with_config(config); - let table1 = table_with_constraints(); - let df = ctx.read_table(table1)?; - let col_id = Expr::Column(datafusion_common::Column { - relation: None, - name: "id".to_string(), - }); - let col_name = Expr::Column(datafusion_common::Column { - relation: None, - name: "name".to_string(), - }); + let df = ctx.read_table(table_with_constraints())?; - // group by contains id column - let group_expr = vec![col_id.clone()]; + // GROUP BY id + let group_expr = vec![col("id")]; let aggr_expr = vec![]; // group by id, let df = df.aggregate(group_expr, aggr_expr)?; - let condition1 = Expr::BinaryExpr(BinaryExpr::new( - Box::new(col_id.clone()), - Operator::Eq, - Box::new(Expr::Literal(ScalarValue::Int32(Some(1)))), - )); // Predicate refers to id field - let predicate = condition1; - // id=0 + // id = 1 + let predicate = col("id").eq(lit(1i32)); let df = df.filter(predicate)?; // Select expression refers to id, and name columns. // id, name - let df = df.select(vec![col_id.clone(), col_name.clone()])?; - let physical_plan = df.clone().create_physical_plan().await?; - - let expected = vec![ + let df = df.select(vec![col("id"), col("name")])?; + assert_physical_plan( + &df, + vec![ "CoalesceBatchesExec: target_batch_size=8192", " FilterExec: id@0 = 1", " AggregateExec: mode=Single, gby=[id@0 as id, name@1 as name], aggr=[]", " MemoryExec: partitions=1, partition_sizes=[1]", - ]; - // Get string representation of the plan - let actual = get_plan_string(&physical_plan); - assert_eq!( - expected, actual, - "\n**Optimized Plan Mismatch\n\nexpected:\n\n{expected:#?}\nactual:\n\n{actual:#?}\n\n" - ); + ], + ) + .await; // Since id and name are functionally dependant, we can use name among expression // even if it is not part of the group by expression. - let df_results = collect(physical_plan, ctx.task_ctx()).await?; + let df_results = df.collect().await?; #[rustfmt::skip] assert_batches_sorted_eq!( @@ -1778,51 +1738,35 @@ mod tests { let config = SessionConfig::new().with_target_partitions(1); let ctx = SessionContext::new_with_config(config); - let table1 = table_with_constraints(); - let df = ctx.read_table(table1)?; - let col_id = Expr::Column(datafusion_common::Column { - relation: None, - name: "id".to_string(), - }); + let df = ctx.read_table(table_with_constraints())?; - // group by contains id column - let group_expr = vec![col_id.clone()]; + // GROUP BY id + let group_expr = vec![col("id")]; let aggr_expr = vec![]; - // group by id, let df = df.aggregate(group_expr, aggr_expr)?; - let condition1 = Expr::BinaryExpr(BinaryExpr::new( - Box::new(col_id.clone()), - Operator::Eq, - Box::new(Expr::Literal(ScalarValue::Int32(Some(1)))), - )); // Predicate refers to id field - let predicate = condition1; - // id=1 + // id = 1 + let predicate = col("id").eq(lit(1i32)); let df = df.filter(predicate)?; // Select expression refers to id column. // id - let df = df.select(vec![col_id.clone()])?; - let physical_plan = df.clone().create_physical_plan().await?; + let df = df.select(vec![col("id")])?; // In this case aggregate shouldn't be expanded, since these // columns are not used. - let expected = vec![ - "CoalesceBatchesExec: target_batch_size=8192", - " FilterExec: id@0 = 1", - " AggregateExec: mode=Single, gby=[id@0 as id], aggr=[]", - " MemoryExec: partitions=1, partition_sizes=[1]", - ]; - // Get string representation of the plan - let actual = get_plan_string(&physical_plan); - assert_eq!( - expected, actual, - "\n**Optimized Plan Mismatch\n\nexpected:\n\n{expected:#?}\nactual:\n\n{actual:#?}\n\n" - ); + assert_physical_plan( + &df, + vec![ + "CoalesceBatchesExec: target_batch_size=8192", + " FilterExec: id@0 = 1", + " AggregateExec: mode=Single, gby=[id@0 as id], aggr=[]", + " MemoryExec: partitions=1, partition_sizes=[1]", + ], + ) + .await; - // Since id and name are functionally dependant, we can use name among expression - // even if it is not part of the group by expression. - let df_results = collect(physical_plan, ctx.task_ctx()).await?; + let df_results = df.collect().await?; #[rustfmt::skip] assert_batches_sorted_eq!( From 72af0ffdf00247e5383adcdbe3dada7ca85d9172 Mon Sep 17 00:00:00 2001 From: Liang-Chi Hsieh Date: Sun, 24 Dec 2023 00:17:05 -0800 Subject: [PATCH 492/572] Improve regexp_match performance by avoiding cloning Regex (#8631) * Improve regexp_match performance by avoiding cloning Regex * Update datafusion/physical-expr/src/regex_expressions.rs Co-authored-by: Andrew Lamb * Removing clone of Regex in regexp_replace --------- Co-authored-by: Andrew Lamb --- .../physical-expr/src/regex_expressions.rs | 96 +++++++++++++++++-- 1 file changed, 87 insertions(+), 9 deletions(-) diff --git a/datafusion/physical-expr/src/regex_expressions.rs b/datafusion/physical-expr/src/regex_expressions.rs index 7bafed072b61..b778fd86c24b 100644 --- a/datafusion/physical-expr/src/regex_expressions.rs +++ b/datafusion/physical-expr/src/regex_expressions.rs @@ -25,7 +25,8 @@ use arrow::array::{ new_null_array, Array, ArrayDataBuilder, ArrayRef, BufferBuilder, GenericStringArray, OffsetSizeTrait, }; -use arrow::compute; +use arrow_array::builder::{GenericStringBuilder, ListBuilder}; +use arrow_schema::ArrowError; use datafusion_common::{arrow_datafusion_err, plan_err}; use datafusion_common::{ cast::as_generic_string_array, internal_err, DataFusionError, Result, @@ -58,7 +59,7 @@ pub fn regexp_match(args: &[ArrayRef]) -> Result { 2 => { let values = as_generic_string_array::(&args[0])?; let regex = as_generic_string_array::(&args[1])?; - compute::regexp_match(values, regex, None).map_err(|e| arrow_datafusion_err!(e)) + _regexp_match(values, regex, None).map_err(|e| arrow_datafusion_err!(e)) } 3 => { let values = as_generic_string_array::(&args[0])?; @@ -69,7 +70,7 @@ pub fn regexp_match(args: &[ArrayRef]) -> Result { Some(f) if f.iter().any(|s| s == Some("g")) => { plan_err!("regexp_match() does not support the \"global\" option") }, - _ => compute::regexp_match(values, regex, flags).map_err(|e| arrow_datafusion_err!(e)), + _ => _regexp_match(values, regex, flags).map_err(|e| arrow_datafusion_err!(e)), } } other => internal_err!( @@ -78,6 +79,83 @@ pub fn regexp_match(args: &[ArrayRef]) -> Result { } } +/// TODO: Remove this once it is included in arrow-rs new release. +/// +fn _regexp_match( + array: &GenericStringArray, + regex_array: &GenericStringArray, + flags_array: Option<&GenericStringArray>, +) -> std::result::Result { + let mut patterns: std::collections::HashMap = + std::collections::HashMap::new(); + let builder: GenericStringBuilder = + GenericStringBuilder::with_capacity(0, 0); + let mut list_builder = ListBuilder::new(builder); + + let complete_pattern = match flags_array { + Some(flags) => Box::new(regex_array.iter().zip(flags.iter()).map( + |(pattern, flags)| { + pattern.map(|pattern| match flags { + Some(value) => format!("(?{value}){pattern}"), + None => pattern.to_string(), + }) + }, + )) as Box>>, + None => Box::new( + regex_array + .iter() + .map(|pattern| pattern.map(|pattern| pattern.to_string())), + ), + }; + + array + .iter() + .zip(complete_pattern) + .map(|(value, pattern)| { + match (value, pattern) { + // Required for Postgres compatibility: + // SELECT regexp_match('foobarbequebaz', ''); = {""} + (Some(_), Some(pattern)) if pattern == *"" => { + list_builder.values().append_value(""); + list_builder.append(true); + } + (Some(value), Some(pattern)) => { + let existing_pattern = patterns.get(&pattern); + let re = match existing_pattern { + Some(re) => re, + None => { + let re = Regex::new(pattern.as_str()).map_err(|e| { + ArrowError::ComputeError(format!( + "Regular expression did not compile: {e:?}" + )) + })?; + patterns.insert(pattern.clone(), re); + patterns.get(&pattern).unwrap() + } + }; + match re.captures(value) { + Some(caps) => { + let mut iter = caps.iter(); + if caps.len() > 1 { + iter.next(); + } + for m in iter.flatten() { + list_builder.values().append_value(m.as_str()); + } + + list_builder.append(true); + } + None => list_builder.append(false), + } + } + _ => list_builder.append(false), + } + Ok(()) + }) + .collect::, ArrowError>>()?; + Ok(Arc::new(list_builder.finish())) +} + /// replace POSIX capture groups (like \1) with Rust Regex group (like ${1}) /// used by regexp_replace fn regex_replace_posix_groups(replacement: &str) -> String { @@ -116,12 +194,12 @@ pub fn regexp_replace(args: &[ArrayRef]) -> Result // if patterns hashmap already has regexp then use else else create and return let re = match patterns.get(pattern) { - Some(re) => Ok(re.clone()), + Some(re) => Ok(re), None => { match Regex::new(pattern) { Ok(re) => { - patterns.insert(pattern.to_string(), re.clone()); - Ok(re) + patterns.insert(pattern.to_string(), re); + Ok(patterns.get(pattern).unwrap()) }, Err(err) => Err(DataFusionError::External(Box::new(err))), } @@ -162,12 +240,12 @@ pub fn regexp_replace(args: &[ArrayRef]) -> Result // if patterns hashmap already has regexp then use else else create and return let re = match patterns.get(&pattern) { - Some(re) => Ok(re.clone()), + Some(re) => Ok(re), None => { match Regex::new(pattern.as_str()) { Ok(re) => { - patterns.insert(pattern, re.clone()); - Ok(re) + patterns.insert(pattern.clone(), re); + Ok(patterns.get(&pattern).unwrap()) }, Err(err) => Err(DataFusionError::External(Box::new(err))), } From 6b433a839948c406a41128186e81572ec1fff689 Mon Sep 17 00:00:00 2001 From: Andrew Lamb Date: Sun, 24 Dec 2023 07:37:38 -0500 Subject: [PATCH 493/572] Minor: improve `listing_table_ignore_subdirectory` config documentation (#8634) * Minor: improve `listing_table_ignore_subdirectory` config documentation * update slt --- datafusion/common/src/config.rs | 8 ++++---- datafusion/sqllogictest/test_files/information_schema.slt | 2 +- docs/source/user-guide/configs.md | 2 +- 3 files changed, 6 insertions(+), 6 deletions(-) diff --git a/datafusion/common/src/config.rs b/datafusion/common/src/config.rs index dedce74ff40d..5b1325ec06ee 100644 --- a/datafusion/common/src/config.rs +++ b/datafusion/common/src/config.rs @@ -273,11 +273,11 @@ config_namespace! { /// memory consumption pub max_buffered_batches_per_output_file: usize, default = 2 - /// When scanning file paths, whether to ignore subdirectory files, - /// ignored by default (true), when reading a partitioned table, - /// `listing_table_ignore_subdirectory` is always equal to false, even if set to true + /// Should sub directories be ignored when scanning directories for data + /// files. Defaults to true (ignores subdirectories), consistent with + /// Hive. Note that this setting does not affect reading partitioned + /// tables (e.g. `/table/year=2021/month=01/data.parquet`). pub listing_table_ignore_subdirectory: bool, default = true - } } diff --git a/datafusion/sqllogictest/test_files/information_schema.slt b/datafusion/sqllogictest/test_files/information_schema.slt index 36876beb1447..1b5ad86546a3 100644 --- a/datafusion/sqllogictest/test_files/information_schema.slt +++ b/datafusion/sqllogictest/test_files/information_schema.slt @@ -225,7 +225,7 @@ datafusion.execution.aggregate.scalar_update_factor 10 Specifies the threshold f datafusion.execution.batch_size 8192 Default batch size while creating new batches, it's especially useful for buffer-in-memory batches since creating tiny batches would result in too much metadata memory consumption datafusion.execution.coalesce_batches true When set to true, record batches will be examined between each operator and small batches will be coalesced into larger batches. This is helpful when there are highly selective filters or joins that could produce tiny output batches. The target batch size is determined by the configuration setting datafusion.execution.collect_statistics false Should DataFusion collect statistics after listing files -datafusion.execution.listing_table_ignore_subdirectory true When scanning file paths, whether to ignore subdirectory files, ignored by default (true), when reading a partitioned table, `listing_table_ignore_subdirectory` is always equal to false, even if set to true +datafusion.execution.listing_table_ignore_subdirectory true Should sub directories be ignored when scanning directories for data files. Defaults to true (ignores subdirectories), consistent with Hive. Note that this setting does not affect reading partitioned tables (e.g. `/table/year=2021/month=01/data.parquet`). datafusion.execution.max_buffered_batches_per_output_file 2 This is the maximum number of RecordBatches buffered for each output file being worked. Higher values can potentially give faster write performance at the cost of higher peak memory consumption datafusion.execution.meta_fetch_concurrency 32 Number of files to read in parallel when inferring schema and statistics datafusion.execution.minimum_parallel_output_files 4 Guarantees a minimum level of output files running in parallel. RecordBatches will be distributed in round robin fashion to each parallel writer. Each writer is closed and a new file opened once soft_max_rows_per_output_file is reached. diff --git a/docs/source/user-guide/configs.md b/docs/source/user-guide/configs.md index 1f7fa7760b94..0a5c221c5034 100644 --- a/docs/source/user-guide/configs.md +++ b/docs/source/user-guide/configs.md @@ -82,7 +82,7 @@ Environment variables are read during `SessionConfig` initialisation so they mus | datafusion.execution.minimum_parallel_output_files | 4 | Guarantees a minimum level of output files running in parallel. RecordBatches will be distributed in round robin fashion to each parallel writer. Each writer is closed and a new file opened once soft_max_rows_per_output_file is reached. | | datafusion.execution.soft_max_rows_per_output_file | 50000000 | Target number of rows in output files when writing multiple. This is a soft max, so it can be exceeded slightly. There also will be one file smaller than the limit if the total number of rows written is not roughly divisible by the soft max | | datafusion.execution.max_buffered_batches_per_output_file | 2 | This is the maximum number of RecordBatches buffered for each output file being worked. Higher values can potentially give faster write performance at the cost of higher peak memory consumption | -| datafusion.execution.listing_table_ignore_subdirectory | true | When scanning file paths, whether to ignore subdirectory files, ignored by default (true), when reading a partitioned table, `listing_table_ignore_subdirectory` is always equal to false, even if set to true | +| datafusion.execution.listing_table_ignore_subdirectory | true | Should sub directories be ignored when scanning directories for data files. Defaults to true (ignores subdirectories), consistent with Hive. Note that this setting does not affect reading partitioned tables (e.g. `/table/year=2021/month=01/data.parquet`). | | datafusion.optimizer.enable_distinct_aggregation_soft_limit | true | When set to true, the optimizer will push a limit operation into grouped aggregations which have no aggregate expressions, as a soft limit, emitting groups once the limit is reached, before all rows in the group are read. | | datafusion.optimizer.enable_round_robin_repartition | true | When set to true, the physical plan optimizer will try to add round robin repartitioning to increase parallelism to leverage more CPU cores | | datafusion.optimizer.enable_topk_aggregation | true | When set to true, the optimizer will attempt to perform limit operations during aggregations, if possible | From d5704f75fc28f88632518ef9a808c9cda38dc162 Mon Sep 17 00:00:00 2001 From: Devin D'Angelo Date: Sun, 24 Dec 2023 07:46:26 -0500 Subject: [PATCH 494/572] Support Writing Arrow files (#8608) * write arrow files * update datafusion-cli lock * fix toml formatting * Update insert_to_external.slt Co-authored-by: Andrew Lamb * add ticket tracking arrow options * default to lz4 compression * update datafusion-cli lock * cargo update --------- Co-authored-by: Andrew Lamb --- Cargo.toml | 28 +-- datafusion-cli/Cargo.lock | 56 ++--- datafusion/core/Cargo.toml | 1 + .../core/src/datasource/file_format/arrow.rs | 207 +++++++++++++++++- .../src/datasource/file_format/parquet.rs | 34 +-- .../src/datasource/file_format/write/mod.rs | 33 ++- datafusion/sqllogictest/test_files/copy.slt | 56 +++++ .../test_files/insert_to_external.slt | 39 ++++ 8 files changed, 368 insertions(+), 86 deletions(-) diff --git a/Cargo.toml b/Cargo.toml index 023dc6c6fc4f..a698fbf471f9 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -17,24 +17,7 @@ [workspace] exclude = ["datafusion-cli"] -members = [ - "datafusion/common", - "datafusion/core", - "datafusion/expr", - "datafusion/execution", - "datafusion/optimizer", - "datafusion/physical-expr", - "datafusion/physical-plan", - "datafusion/proto", - "datafusion/proto/gen", - "datafusion/sql", - "datafusion/sqllogictest", - "datafusion/substrait", - "datafusion/wasmtest", - "datafusion-examples", - "docs", - "test-utils", - "benchmarks", +members = ["datafusion/common", "datafusion/core", "datafusion/expr", "datafusion/execution", "datafusion/optimizer", "datafusion/physical-expr", "datafusion/physical-plan", "datafusion/proto", "datafusion/proto/gen", "datafusion/sql", "datafusion/sqllogictest", "datafusion/substrait", "datafusion/wasmtest", "datafusion-examples", "docs", "test-utils", "benchmarks", ] resolver = "2" @@ -53,24 +36,26 @@ arrow = { version = "49.0.0", features = ["prettyprint"] } arrow-array = { version = "49.0.0", default-features = false, features = ["chrono-tz"] } arrow-buffer = { version = "49.0.0", default-features = false } arrow-flight = { version = "49.0.0", features = ["flight-sql-experimental"] } +arrow-ipc = { version = "49.0.0", default-features = false, features=["lz4"] } arrow-ord = { version = "49.0.0", default-features = false } arrow-schema = { version = "49.0.0", default-features = false } async-trait = "0.1.73" bigdecimal = "0.4.1" bytes = "1.4" +chrono = { version = "0.4.31", default-features = false } ctor = "0.2.0" +dashmap = "5.4.0" datafusion = { path = "datafusion/core", version = "34.0.0" } datafusion-common = { path = "datafusion/common", version = "34.0.0" } +datafusion-execution = { path = "datafusion/execution", version = "34.0.0" } datafusion-expr = { path = "datafusion/expr", version = "34.0.0" } -datafusion-sql = { path = "datafusion/sql", version = "34.0.0" } datafusion-optimizer = { path = "datafusion/optimizer", version = "34.0.0" } datafusion-physical-expr = { path = "datafusion/physical-expr", version = "34.0.0" } datafusion-physical-plan = { path = "datafusion/physical-plan", version = "34.0.0" } -datafusion-execution = { path = "datafusion/execution", version = "34.0.0" } datafusion-proto = { path = "datafusion/proto", version = "34.0.0" } +datafusion-sql = { path = "datafusion/sql", version = "34.0.0" } datafusion-sqllogictest = { path = "datafusion/sqllogictest", version = "34.0.0" } datafusion-substrait = { path = "datafusion/substrait", version = "34.0.0" } -dashmap = "5.4.0" doc-comment = "0.3" env_logger = "0.10" futures = "0.3" @@ -88,7 +73,6 @@ serde_json = "1" sqlparser = { version = "0.40.0", features = ["visitor"] } tempfile = "3" thiserror = "1.0.44" -chrono = { version = "0.4.31", default-features = false } url = "2.2" [profile.release] diff --git a/datafusion-cli/Cargo.lock b/datafusion-cli/Cargo.lock index ac05ddf10a73..9f75013c86dc 100644 --- a/datafusion-cli/Cargo.lock +++ b/datafusion-cli/Cargo.lock @@ -255,6 +255,7 @@ dependencies = [ "arrow-data", "arrow-schema", "flatbuffers", + "lz4_flex", ] [[package]] @@ -378,13 +379,13 @@ dependencies = [ [[package]] name = "async-trait" -version = "0.1.74" +version = "0.1.75" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "a66537f1bb974b254c98ed142ff995236e81b9d0fe4db0575f46612cb15eb0f9" +checksum = "fdf6721fb0140e4f897002dd086c06f6c27775df19cfe1fccb21181a48fd2c98" dependencies = [ "proc-macro2", "quote", - "syn 2.0.41", + "syn 2.0.42", ] [[package]] @@ -1074,7 +1075,7 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "30d2b3721e861707777e3195b0158f950ae6dc4a27e4d02ff9f67e3eb3de199e" dependencies = [ "quote", - "syn 2.0.41", + "syn 2.0.42", ] [[package]] @@ -1104,6 +1105,7 @@ dependencies = [ "apache-avro", "arrow", "arrow-array", + "arrow-ipc", "arrow-schema", "async-compression", "async-trait", @@ -1576,7 +1578,7 @@ checksum = "53b153fd91e4b0147f4aced87be237c98248656bb01050b96bf3ee89220a8ddb" dependencies = [ "proc-macro2", "quote", - "syn 2.0.41", + "syn 2.0.42", ] [[package]] @@ -2496,7 +2498,7 @@ checksum = "4359fd9c9171ec6e8c62926d6faaf553a8dc3f64e1507e76da7911b4f6a04405" dependencies = [ "proc-macro2", "quote", - "syn 2.0.41", + "syn 2.0.42", ] [[package]] @@ -2513,9 +2515,9 @@ checksum = "8b870d8c151b6f2fb93e84a13146138f05d02ed11c7e7c54f8826aaaf7c9f184" [[package]] name = "pkg-config" -version = "0.3.27" +version = "0.3.28" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "26072860ba924cbfa98ea39c8c19b4dd6a4a25423dbdf219c1eca91aa0cf6964" +checksum = "69d3587f8a9e599cc7ec2c00e331f71c4e69a5f9a4b8a6efd5b07466b9736f9a" [[package]] name = "powerfmt" @@ -2586,9 +2588,9 @@ dependencies = [ [[package]] name = "proc-macro2" -version = "1.0.70" +version = "1.0.71" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "39278fbbf5fb4f646ce651690877f89d1c5811a3d4acb27700c1cb3cdb78fd3b" +checksum = "75cb1540fadbd5b8fbccc4dddad2734eba435053f725621c070711a14bb5f4b8" dependencies = [ "unicode-ident", ] @@ -3020,7 +3022,7 @@ checksum = "43576ca501357b9b071ac53cdc7da8ef0cbd9493d8df094cd821777ea6e894d3" dependencies = [ "proc-macro2", "quote", - "syn 2.0.41", + "syn 2.0.42", ] [[package]] @@ -3186,7 +3188,7 @@ dependencies = [ "proc-macro2", "quote", "rustversion", - "syn 2.0.41", + "syn 2.0.42", ] [[package]] @@ -3208,9 +3210,9 @@ dependencies = [ [[package]] name = "syn" -version = "2.0.41" +version = "2.0.42" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "44c8b28c477cc3bf0e7966561e3460130e1255f7a1cf71931075f1c5e7a7e269" +checksum = "5b7d0a2c048d661a1a59fcd7355baa232f7ed34e0ee4df2eef3c1c1c0d3852d8" dependencies = [ "proc-macro2", "quote", @@ -3289,7 +3291,7 @@ checksum = "01742297787513b79cf8e29d1056ede1313e2420b7b3b15d0a768b4921f549df" dependencies = [ "proc-macro2", "quote", - "syn 2.0.41", + "syn 2.0.42", ] [[package]] @@ -3357,9 +3359,9 @@ checksum = "1f3ccbac311fea05f86f61904b462b55fb3df8837a366dfc601a0161d0532f20" [[package]] name = "tokio" -version = "1.35.0" +version = "1.35.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "841d45b238a16291a4e1584e61820b8ae57d696cc5015c459c229ccc6990cc1c" +checksum = "c89b4efa943be685f629b149f53829423f8f5531ea21249408e8e2f8671ec104" dependencies = [ "backtrace", "bytes", @@ -3381,7 +3383,7 @@ checksum = "5b8a1e28f2deaa14e508979454cb3a223b10b938b45af148bc0986de36f1923b" dependencies = [ "proc-macro2", "quote", - "syn 2.0.41", + "syn 2.0.42", ] [[package]] @@ -3478,7 +3480,7 @@ checksum = "34704c8d6ebcbc939824180af020566b01a7c01f80641264eba0999f6c2b6be7" dependencies = [ "proc-macro2", "quote", - "syn 2.0.41", + "syn 2.0.42", ] [[package]] @@ -3523,7 +3525,7 @@ checksum = "f03ca4cb38206e2bef0700092660bb74d696f808514dae47fa1467cbfe26e96e" dependencies = [ "proc-macro2", "quote", - "syn 2.0.41", + "syn 2.0.42", ] [[package]] @@ -3677,7 +3679,7 @@ dependencies = [ "once_cell", "proc-macro2", "quote", - "syn 2.0.41", + "syn 2.0.42", "wasm-bindgen-shared", ] @@ -3711,7 +3713,7 @@ checksum = "f0eb82fcb7930ae6219a7ecfd55b217f5f0893484b7a13022ebb2b2bf20b5283" dependencies = [ "proc-macro2", "quote", - "syn 2.0.41", + "syn 2.0.42", "wasm-bindgen-backend", "wasm-bindgen-shared", ] @@ -3960,22 +3962,22 @@ dependencies = [ [[package]] name = "zerocopy" -version = "0.7.31" +version = "0.7.32" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "1c4061bedbb353041c12f413700357bec76df2c7e2ca8e4df8bac24c6bf68e3d" +checksum = "74d4d3961e53fa4c9a25a8637fc2bfaf2595b3d3ae34875568a5cf64787716be" dependencies = [ "zerocopy-derive", ] [[package]] name = "zerocopy-derive" -version = "0.7.31" +version = "0.7.32" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "b3c129550b3e6de3fd0ba67ba5c81818f9805e58b8d7fee80a3a59d2c9fc601a" +checksum = "9ce1b18ccd8e73a9321186f97e46f9f04b778851177567b1975109d26a08d2a6" dependencies = [ "proc-macro2", "quote", - "syn 2.0.41", + "syn 2.0.42", ] [[package]] diff --git a/datafusion/core/Cargo.toml b/datafusion/core/Cargo.toml index 0ee83e756745..9de6a7f7d6a0 100644 --- a/datafusion/core/Cargo.toml +++ b/datafusion/core/Cargo.toml @@ -55,6 +55,7 @@ ahash = { version = "0.8", default-features = false, features = ["runtime-rng"] apache-avro = { version = "0.16", optional = true } arrow = { workspace = true } arrow-array = { workspace = true } +arrow-ipc = { workspace = true } arrow-schema = { workspace = true } async-compression = { version = "0.4.0", features = ["bzip2", "gzip", "xz", "zstd", "futures-io", "tokio"], optional = true } async-trait = { workspace = true } diff --git a/datafusion/core/src/datasource/file_format/arrow.rs b/datafusion/core/src/datasource/file_format/arrow.rs index 07c96bdae1b4..7d393d9129dd 100644 --- a/datafusion/core/src/datasource/file_format/arrow.rs +++ b/datafusion/core/src/datasource/file_format/arrow.rs @@ -21,10 +21,13 @@ use std::any::Any; use std::borrow::Cow; +use std::fmt::{self, Debug}; use std::sync::Arc; use crate::datasource::file_format::FileFormat; -use crate::datasource::physical_plan::{ArrowExec, FileScanConfig}; +use crate::datasource::physical_plan::{ + ArrowExec, FileGroupDisplay, FileScanConfig, FileSinkConfig, +}; use crate::error::Result; use crate::execution::context::SessionState; use crate::physical_plan::ExecutionPlan; @@ -32,16 +35,28 @@ use crate::physical_plan::ExecutionPlan; use arrow::ipc::convert::fb_to_schema; use arrow::ipc::reader::FileReader; use arrow::ipc::root_as_message; +use arrow_ipc::writer::IpcWriteOptions; +use arrow_ipc::CompressionType; use arrow_schema::{ArrowError, Schema, SchemaRef}; use bytes::Bytes; -use datafusion_common::{FileType, Statistics}; -use datafusion_physical_expr::PhysicalExpr; +use datafusion_common::{not_impl_err, DataFusionError, FileType, Statistics}; +use datafusion_execution::{SendableRecordBatchStream, TaskContext}; +use datafusion_physical_expr::{PhysicalExpr, PhysicalSortRequirement}; +use crate::physical_plan::{DisplayAs, DisplayFormatType}; use async_trait::async_trait; +use datafusion_physical_plan::insert::{DataSink, FileSinkExec}; +use datafusion_physical_plan::metrics::MetricsSet; use futures::stream::BoxStream; use futures::StreamExt; use object_store::{GetResultPayload, ObjectMeta, ObjectStore}; +use tokio::io::AsyncWriteExt; +use tokio::task::JoinSet; + +use super::file_compression_type::FileCompressionType; +use super::write::demux::start_demuxer_task; +use super::write::{create_writer, SharedBuffer}; /// Arrow `FileFormat` implementation. #[derive(Default, Debug)] @@ -97,11 +112,197 @@ impl FileFormat for ArrowFormat { Ok(Arc::new(exec)) } + async fn create_writer_physical_plan( + &self, + input: Arc, + _state: &SessionState, + conf: FileSinkConfig, + order_requirements: Option>, + ) -> Result> { + if conf.overwrite { + return not_impl_err!("Overwrites are not implemented yet for Arrow format"); + } + + let sink_schema = conf.output_schema().clone(); + let sink = Arc::new(ArrowFileSink::new(conf)); + + Ok(Arc::new(FileSinkExec::new( + input, + sink, + sink_schema, + order_requirements, + )) as _) + } + fn file_type(&self) -> FileType { FileType::ARROW } } +/// Implements [`DataSink`] for writing to arrow_ipc files +struct ArrowFileSink { + config: FileSinkConfig, +} + +impl ArrowFileSink { + fn new(config: FileSinkConfig) -> Self { + Self { config } + } + + /// Converts table schema to writer schema, which may differ in the case + /// of hive style partitioning where some columns are removed from the + /// underlying files. + fn get_writer_schema(&self) -> Arc { + if !self.config.table_partition_cols.is_empty() { + let schema = self.config.output_schema(); + let partition_names: Vec<_> = self + .config + .table_partition_cols + .iter() + .map(|(s, _)| s) + .collect(); + Arc::new(Schema::new( + schema + .fields() + .iter() + .filter(|f| !partition_names.contains(&f.name())) + .map(|f| (**f).clone()) + .collect::>(), + )) + } else { + self.config.output_schema().clone() + } + } +} + +impl Debug for ArrowFileSink { + fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result { + f.debug_struct("ArrowFileSink").finish() + } +} + +impl DisplayAs for ArrowFileSink { + fn fmt_as(&self, t: DisplayFormatType, f: &mut fmt::Formatter<'_>) -> fmt::Result { + match t { + DisplayFormatType::Default | DisplayFormatType::Verbose => { + write!(f, "ArrowFileSink(file_groups=",)?; + FileGroupDisplay(&self.config.file_groups).fmt_as(t, f)?; + write!(f, ")") + } + } + } +} + +#[async_trait] +impl DataSink for ArrowFileSink { + fn as_any(&self) -> &dyn Any { + self + } + + fn metrics(&self) -> Option { + None + } + + async fn write_all( + &self, + data: SendableRecordBatchStream, + context: &Arc, + ) -> Result { + // No props are supported yet, but can be by updating FileTypeWriterOptions + // to populate this struct and use those options to initialize the arrow_ipc::writer::FileWriter + // https://github.com/apache/arrow-datafusion/issues/8635 + let _arrow_props = self.config.file_type_writer_options.try_into_arrow()?; + + let object_store = context + .runtime_env() + .object_store(&self.config.object_store_url)?; + + let part_col = if !self.config.table_partition_cols.is_empty() { + Some(self.config.table_partition_cols.clone()) + } else { + None + }; + + let (demux_task, mut file_stream_rx) = start_demuxer_task( + data, + context, + part_col, + self.config.table_paths[0].clone(), + "arrow".into(), + self.config.single_file_output, + ); + + let mut file_write_tasks: JoinSet> = + JoinSet::new(); + + let ipc_options = + IpcWriteOptions::try_new(64, false, arrow_ipc::MetadataVersion::V5)? + .try_with_compression(Some(CompressionType::LZ4_FRAME))?; + while let Some((path, mut rx)) = file_stream_rx.recv().await { + let shared_buffer = SharedBuffer::new(1048576); + let mut arrow_writer = arrow_ipc::writer::FileWriter::try_new_with_options( + shared_buffer.clone(), + &self.get_writer_schema(), + ipc_options.clone(), + )?; + let mut object_store_writer = create_writer( + FileCompressionType::UNCOMPRESSED, + &path, + object_store.clone(), + ) + .await?; + file_write_tasks.spawn(async move { + let mut row_count = 0; + while let Some(batch) = rx.recv().await { + row_count += batch.num_rows(); + arrow_writer.write(&batch)?; + let mut buff_to_flush = shared_buffer.buffer.try_lock().unwrap(); + if buff_to_flush.len() > 1024000 { + object_store_writer + .write_all(buff_to_flush.as_slice()) + .await?; + buff_to_flush.clear(); + } + } + arrow_writer.finish()?; + let final_buff = shared_buffer.buffer.try_lock().unwrap(); + + object_store_writer.write_all(final_buff.as_slice()).await?; + object_store_writer.shutdown().await?; + Ok(row_count) + }); + } + + let mut row_count = 0; + while let Some(result) = file_write_tasks.join_next().await { + match result { + Ok(r) => { + row_count += r?; + } + Err(e) => { + if e.is_panic() { + std::panic::resume_unwind(e.into_panic()); + } else { + unreachable!(); + } + } + } + } + + match demux_task.await { + Ok(r) => r?, + Err(e) => { + if e.is_panic() { + std::panic::resume_unwind(e.into_panic()); + } else { + unreachable!(); + } + } + } + Ok(row_count as u64) + } +} + const ARROW_MAGIC: [u8; 6] = [b'A', b'R', b'R', b'O', b'W', b'1']; const CONTINUATION_MARKER: [u8; 4] = [0xff; 4]; diff --git a/datafusion/core/src/datasource/file_format/parquet.rs b/datafusion/core/src/datasource/file_format/parquet.rs index 9db320fb9da4..0c813b6ccbf0 100644 --- a/datafusion/core/src/datasource/file_format/parquet.rs +++ b/datafusion/core/src/datasource/file_format/parquet.rs @@ -29,7 +29,6 @@ use parquet::file::writer::SerializedFileWriter; use std::any::Any; use std::fmt; use std::fmt::Debug; -use std::io::Write; use std::sync::Arc; use tokio::io::{AsyncWrite, AsyncWriteExt}; use tokio::sync::mpsc::{self, Receiver, Sender}; @@ -56,7 +55,7 @@ use parquet::file::properties::WriterProperties; use parquet::file::statistics::Statistics as ParquetStatistics; use super::write::demux::start_demuxer_task; -use super::write::{create_writer, AbortableWrite}; +use super::write::{create_writer, AbortableWrite, SharedBuffer}; use super::{FileFormat, FileScanConfig}; use crate::arrow::array::{ BooleanArray, Float32Array, Float64Array, Int32Array, Int64Array, @@ -1101,37 +1100,6 @@ async fn output_single_parquet_file_parallelized( Ok(row_count) } -/// A buffer with interior mutability shared by the SerializedFileWriter and -/// ObjectStore writer -#[derive(Clone)] -struct SharedBuffer { - /// The inner buffer for reading and writing - /// - /// The lock is used to obtain internal mutability, so no worry about the - /// lock contention. - buffer: Arc>>, -} - -impl SharedBuffer { - pub fn new(capacity: usize) -> Self { - Self { - buffer: Arc::new(futures::lock::Mutex::new(Vec::with_capacity(capacity))), - } - } -} - -impl Write for SharedBuffer { - fn write(&mut self, buf: &[u8]) -> std::io::Result { - let mut buffer = self.buffer.try_lock().unwrap(); - Write::write(&mut *buffer, buf) - } - - fn flush(&mut self) -> std::io::Result<()> { - let mut buffer = self.buffer.try_lock().unwrap(); - Write::flush(&mut *buffer) - } -} - #[cfg(test)] pub(crate) mod test_util { use super::*; diff --git a/datafusion/core/src/datasource/file_format/write/mod.rs b/datafusion/core/src/datasource/file_format/write/mod.rs index cfcdbd8c464e..68fe81ce91fa 100644 --- a/datafusion/core/src/datasource/file_format/write/mod.rs +++ b/datafusion/core/src/datasource/file_format/write/mod.rs @@ -18,7 +18,7 @@ //! Module containing helper methods/traits related to enabling //! write support for the various file formats -use std::io::Error; +use std::io::{Error, Write}; use std::pin::Pin; use std::sync::Arc; use std::task::{Context, Poll}; @@ -43,6 +43,37 @@ use tokio::io::AsyncWrite; pub(crate) mod demux; pub(crate) mod orchestration; +/// A buffer with interior mutability shared by the SerializedFileWriter and +/// ObjectStore writer +#[derive(Clone)] +pub(crate) struct SharedBuffer { + /// The inner buffer for reading and writing + /// + /// The lock is used to obtain internal mutability, so no worry about the + /// lock contention. + pub(crate) buffer: Arc>>, +} + +impl SharedBuffer { + pub fn new(capacity: usize) -> Self { + Self { + buffer: Arc::new(futures::lock::Mutex::new(Vec::with_capacity(capacity))), + } + } +} + +impl Write for SharedBuffer { + fn write(&mut self, buf: &[u8]) -> std::io::Result { + let mut buffer = self.buffer.try_lock().unwrap(); + Write::write(&mut *buffer, buf) + } + + fn flush(&mut self) -> std::io::Result<()> { + let mut buffer = self.buffer.try_lock().unwrap(); + Write::flush(&mut *buffer) + } +} + /// Stores data needed during abortion of MultiPart writers #[derive(Clone)] pub(crate) struct MultiPart { diff --git a/datafusion/sqllogictest/test_files/copy.slt b/datafusion/sqllogictest/test_files/copy.slt index 02ab33083315..89b23917884c 100644 --- a/datafusion/sqllogictest/test_files/copy.slt +++ b/datafusion/sqllogictest/test_files/copy.slt @@ -230,6 +230,62 @@ select * from validate_csv_with_options; 1;Foo 2;Bar +# Copy from table to single arrow file +query IT +COPY source_table to 'test_files/scratch/copy/table.arrow'; +---- +2 + +# Validate single csv output +statement ok +CREATE EXTERNAL TABLE validate_arrow_file +STORED AS arrow +LOCATION 'test_files/scratch/copy/table.arrow'; + +query IT +select * from validate_arrow_file; +---- +1 Foo +2 Bar + +# Copy from dict encoded values to single arrow file +query T? +COPY (values +('c', arrow_cast('foo', 'Dictionary(Int32, Utf8)')), ('d', arrow_cast('bar', 'Dictionary(Int32, Utf8)'))) +to 'test_files/scratch/copy/table_dict.arrow'; +---- +2 + +# Validate single csv output +statement ok +CREATE EXTERNAL TABLE validate_arrow_file_dict +STORED AS arrow +LOCATION 'test_files/scratch/copy/table_dict.arrow'; + +query T? +select * from validate_arrow_file_dict; +---- +c foo +d bar + + +# Copy from table to folder of json +query IT +COPY source_table to 'test_files/scratch/copy/table_arrow' (format arrow, single_file_output false); +---- +2 + +# Validate json output +statement ok +CREATE EXTERNAL TABLE validate_arrow STORED AS arrow LOCATION 'test_files/scratch/copy/table_arrow'; + +query IT +select * from validate_arrow; +---- +1 Foo +2 Bar + + # Error cases: # Copy from table with options diff --git a/datafusion/sqllogictest/test_files/insert_to_external.slt b/datafusion/sqllogictest/test_files/insert_to_external.slt index cdaf0bb64339..e73778ad44e5 100644 --- a/datafusion/sqllogictest/test_files/insert_to_external.slt +++ b/datafusion/sqllogictest/test_files/insert_to_external.slt @@ -76,6 +76,45 @@ select * from dictionary_encoded_parquet_partitioned order by (a); a foo b bar +statement ok +CREATE EXTERNAL TABLE dictionary_encoded_arrow_partitioned( + a varchar, + b varchar, +) +STORED AS arrow +LOCATION 'test_files/scratch/insert_to_external/arrow_dict_partitioned/' +PARTITIONED BY (b) +OPTIONS( +create_local_path 'true', +insert_mode 'append_new_files', +); + +query TT +insert into dictionary_encoded_arrow_partitioned +select * from dictionary_encoded_values +---- +2 + +statement ok +CREATE EXTERNAL TABLE dictionary_encoded_arrow_test_readback( + a varchar, +) +STORED AS arrow +LOCATION 'test_files/scratch/insert_to_external/arrow_dict_partitioned/b=bar/' +OPTIONS( +create_local_path 'true', +insert_mode 'append_new_files', +); + +query T +select * from dictionary_encoded_arrow_test_readback; +---- +b + +# https://github.com/apache/arrow-datafusion/issues/7816 +query error DataFusion error: Arrow error: Schema error: project index 1 out of bounds, max field 1 +select * from dictionary_encoded_arrow_partitioned order by (a); + # test_insert_into statement ok From 3698693fab040dfb077edaf763b6935e9f42ea06 Mon Sep 17 00:00:00 2001 From: Mustafa Akur <106137913+mustafasrepo@users.noreply.github.com> Date: Mon, 25 Dec 2023 10:43:52 +0300 Subject: [PATCH 495/572] Filter pushdown into cross join (#8626) * Initial commit * Simplifications * Review * Review Part 2 * More idiomatic Rust --------- Co-authored-by: Mehmet Ozan Kabak --- .../optimizer/src/eliminate_cross_join.rs | 128 ++++++++++-------- datafusion/optimizer/src/push_down_filter.rs | 89 ++++++++---- datafusion/sqllogictest/test_files/joins.slt | 17 +++ 3 files changed, 152 insertions(+), 82 deletions(-) diff --git a/datafusion/optimizer/src/eliminate_cross_join.rs b/datafusion/optimizer/src/eliminate_cross_join.rs index cf9a59d6b892..7c866950a622 100644 --- a/datafusion/optimizer/src/eliminate_cross_join.rs +++ b/datafusion/optimizer/src/eliminate_cross_join.rs @@ -20,6 +20,7 @@ use std::collections::HashSet; use std::sync::Arc; use crate::{utils, OptimizerConfig, OptimizerRule}; + use datafusion_common::{plan_err, DataFusionError, Result}; use datafusion_expr::expr::{BinaryExpr, Expr}; use datafusion_expr::logical_plan::{ @@ -47,81 +48,93 @@ impl EliminateCrossJoin { /// For above queries, the join predicate is available in filters and they are moved to /// join nodes appropriately /// This fix helps to improve the performance of TPCH Q19. issue#78 -/// impl OptimizerRule for EliminateCrossJoin { fn try_optimize( &self, plan: &LogicalPlan, config: &dyn OptimizerConfig, ) -> Result> { - match plan { + let mut possible_join_keys: Vec<(Expr, Expr)> = vec![]; + let mut all_inputs: Vec = vec![]; + let parent_predicate = match plan { LogicalPlan::Filter(filter) => { - let input = filter.input.as_ref().clone(); - - let mut possible_join_keys: Vec<(Expr, Expr)> = vec![]; - let mut all_inputs: Vec = vec![]; - let did_flat_successfully = match &input { + let input = filter.input.as_ref(); + match input { LogicalPlan::Join(Join { join_type: JoinType::Inner, .. }) - | LogicalPlan::CrossJoin(_) => try_flatten_join_inputs( - &input, - &mut possible_join_keys, - &mut all_inputs, - )?, + | LogicalPlan::CrossJoin(_) => { + if !try_flatten_join_inputs( + input, + &mut possible_join_keys, + &mut all_inputs, + )? { + return Ok(None); + } + extract_possible_join_keys( + &filter.predicate, + &mut possible_join_keys, + )?; + Some(&filter.predicate) + } _ => { return utils::optimize_children(self, plan, config); } - }; - - if !did_flat_successfully { + } + } + LogicalPlan::Join(Join { + join_type: JoinType::Inner, + .. + }) => { + if !try_flatten_join_inputs( + plan, + &mut possible_join_keys, + &mut all_inputs, + )? { return Ok(None); } + None + } + _ => return utils::optimize_children(self, plan, config), + }; - let predicate = &filter.predicate; - // join keys are handled locally - let mut all_join_keys: HashSet<(Expr, Expr)> = HashSet::new(); - - extract_possible_join_keys(predicate, &mut possible_join_keys)?; + // Join keys are handled locally: + let mut all_join_keys = HashSet::<(Expr, Expr)>::new(); + let mut left = all_inputs.remove(0); + while !all_inputs.is_empty() { + left = find_inner_join( + &left, + &mut all_inputs, + &mut possible_join_keys, + &mut all_join_keys, + )?; + } - let mut left = all_inputs.remove(0); - while !all_inputs.is_empty() { - left = find_inner_join( - &left, - &mut all_inputs, - &mut possible_join_keys, - &mut all_join_keys, - )?; - } + left = utils::optimize_children(self, &left, config)?.unwrap_or(left); - left = utils::optimize_children(self, &left, config)?.unwrap_or(left); + if plan.schema() != left.schema() { + left = LogicalPlan::Projection(Projection::new_from_schema( + Arc::new(left), + plan.schema().clone(), + )); + } - if plan.schema() != left.schema() { - left = LogicalPlan::Projection(Projection::new_from_schema( - Arc::new(left.clone()), - plan.schema().clone(), - )); - } + let Some(predicate) = parent_predicate else { + return Ok(Some(left)); + }; - // if there are no join keys then do nothing. - if all_join_keys.is_empty() { - Ok(Some(LogicalPlan::Filter(Filter::try_new( - predicate.clone(), - Arc::new(left), - )?))) - } else { - // remove join expressions from filter - match remove_join_expressions(predicate, &all_join_keys)? { - Some(filter_expr) => Ok(Some(LogicalPlan::Filter( - Filter::try_new(filter_expr, Arc::new(left))?, - ))), - _ => Ok(Some(left)), - } - } + // If there are no join keys then do nothing: + if all_join_keys.is_empty() { + Filter::try_new(predicate.clone(), Arc::new(left)) + .map(|f| Some(LogicalPlan::Filter(f))) + } else { + // Remove join expressions from filter: + match remove_join_expressions(predicate, &all_join_keys)? { + Some(filter_expr) => Filter::try_new(filter_expr, Arc::new(left)) + .map(|f| Some(LogicalPlan::Filter(f))), + _ => Ok(Some(left)), } - - _ => utils::optimize_children(self, plan, config), } } @@ -325,17 +338,16 @@ fn remove_join_expressions( #[cfg(test)] mod tests { + use super::*; + use crate::optimizer::OptimizerContext; + use crate::test::*; + use datafusion_expr::{ binary_expr, col, lit, logical_plan::builder::LogicalPlanBuilder, Operator::{And, Or}, }; - use crate::optimizer::OptimizerContext; - use crate::test::*; - - use super::*; - fn assert_optimized_plan_eq(plan: &LogicalPlan, expected: Vec<&str>) { let rule = EliminateCrossJoin::new(); let optimized_plan = rule diff --git a/datafusion/optimizer/src/push_down_filter.rs b/datafusion/optimizer/src/push_down_filter.rs index 4bea17500acc..4eed39a08941 100644 --- a/datafusion/optimizer/src/push_down_filter.rs +++ b/datafusion/optimizer/src/push_down_filter.rs @@ -15,25 +15,29 @@ //! [`PushDownFilter`] Moves filters so they are applied as early as possible in //! the plan. +use std::collections::{HashMap, HashSet}; +use std::sync::Arc; + use crate::optimizer::ApplyOrder; use crate::{OptimizerConfig, OptimizerRule}; + use datafusion_common::tree_node::{Transformed, TreeNode, VisitRecursion}; use datafusion_common::{ - internal_err, plan_datafusion_err, Column, DFSchema, DataFusionError, Result, + internal_err, plan_datafusion_err, Column, DFSchema, DFSchemaRef, DataFusionError, + JoinConstraint, Result, }; use datafusion_expr::expr::Alias; +use datafusion_expr::expr_rewriter::replace_col; +use datafusion_expr::logical_plan::{ + CrossJoin, Join, JoinType, LogicalPlan, TableScan, Union, +}; use datafusion_expr::utils::{conjunction, split_conjunction, split_conjunction_owned}; -use datafusion_expr::Volatility; use datafusion_expr::{ - and, - expr_rewriter::replace_col, - logical_plan::{CrossJoin, Join, JoinType, LogicalPlan, TableScan, Union}, - or, BinaryExpr, Expr, Filter, Operator, ScalarFunctionDefinition, - TableProviderFilterPushDown, + and, build_join_schema, or, BinaryExpr, Expr, Filter, LogicalPlanBuilder, Operator, + ScalarFunctionDefinition, TableProviderFilterPushDown, Volatility, }; + use itertools::Itertools; -use std::collections::{HashMap, HashSet}; -use std::sync::Arc; /// Optimizer rule for pushing (moving) filter expressions down in a plan so /// they are applied as early as possible. @@ -848,17 +852,23 @@ impl OptimizerRule for PushDownFilter { None => return Ok(None), } } - LogicalPlan::CrossJoin(CrossJoin { left, right, .. }) => { + LogicalPlan::CrossJoin(cross_join) => { let predicates = split_conjunction_owned(filter.predicate.clone()); - push_down_all_join( + let join = convert_cross_join_to_inner_join(cross_join.clone())?; + let join_plan = LogicalPlan::Join(join); + let inputs = join_plan.inputs(); + let left = inputs[0]; + let right = inputs[1]; + let plan = push_down_all_join( predicates, vec![], - &filter.input, + &join_plan, left, right, vec![], - false, - )? + true, + )?; + convert_to_cross_join_if_beneficial(plan)? } LogicalPlan::TableScan(scan) => { let filter_predicates = split_conjunction(&filter.predicate); @@ -955,6 +965,36 @@ impl PushDownFilter { } } +/// Convert cross join to join by pushing down filter predicate to the join condition +fn convert_cross_join_to_inner_join(cross_join: CrossJoin) -> Result { + let CrossJoin { left, right, .. } = cross_join; + let join_schema = build_join_schema(left.schema(), right.schema(), &JoinType::Inner)?; + // predicate is given + Ok(Join { + left, + right, + join_type: JoinType::Inner, + join_constraint: JoinConstraint::On, + on: vec![], + filter: None, + schema: DFSchemaRef::new(join_schema), + null_equals_null: true, + }) +} + +/// Converts the inner join with empty equality predicate and empty filter condition to the cross join +fn convert_to_cross_join_if_beneficial(plan: LogicalPlan) -> Result { + if let LogicalPlan::Join(join) = &plan { + // Can be converted back to cross join + if join.on.is_empty() && join.filter.is_none() { + return LogicalPlanBuilder::from(join.left.as_ref().clone()) + .cross_join(join.right.as_ref().clone())? + .build(); + } + } + Ok(plan) +} + /// replaces columns by its name on the projection. pub fn replace_cols_by_name( e: Expr, @@ -1026,13 +1066,16 @@ fn contain(e: &Expr, check_map: &HashMap) -> bool { #[cfg(test)] mod tests { + use std::fmt::{Debug, Formatter}; + use std::sync::Arc; + use super::*; use crate::optimizer::Optimizer; use crate::rewrite_disjunctive_predicate::RewriteDisjunctivePredicate; use crate::test::*; use crate::OptimizerContext; + use arrow::datatypes::{DataType, Field, Schema, SchemaRef}; - use async_trait::async_trait; use datafusion_common::{DFSchema, DFSchemaRef}; use datafusion_expr::logical_plan::table_scan; use datafusion_expr::{ @@ -1040,8 +1083,8 @@ mod tests { BinaryExpr, Expr, Extension, LogicalPlanBuilder, Operator, TableSource, TableType, UserDefinedLogicalNodeCore, }; - use std::fmt::{Debug, Formatter}; - use std::sync::Arc; + + use async_trait::async_trait; fn assert_optimized_plan_eq(plan: &LogicalPlan, expected: &str) -> Result<()> { crate::test::assert_optimized_plan_eq( @@ -2665,14 +2708,12 @@ Projection: a, b .cross_join(right)? .filter(filter)? .build()?; - let expected = "\ - Filter: test.a = d AND test.b > UInt32(1) OR test.b = e AND test.c < UInt32(10)\ - \n CrossJoin:\ - \n Projection: test.a, test.b, test.c\ - \n TableScan: test, full_filters=[test.b > UInt32(1) OR test.c < UInt32(10)]\ - \n Projection: test1.a AS d, test1.a AS e\ - \n TableScan: test1"; + Inner Join: Filter: test.a = d AND test.b > UInt32(1) OR test.b = e AND test.c < UInt32(10)\ + \n Projection: test.a, test.b, test.c\ + \n TableScan: test, full_filters=[test.b > UInt32(1) OR test.c < UInt32(10)]\ + \n Projection: test1.a AS d, test1.a AS e\ + \n TableScan: test1"; assert_optimized_plan_eq_with_rewrite_predicate(&plan, expected)?; // Originally global state which can help to avoid duplicate Filters been generated and pushed down. diff --git a/datafusion/sqllogictest/test_files/joins.slt b/datafusion/sqllogictest/test_files/joins.slt index 1ad17fbb8c91..eee213811f44 100644 --- a/datafusion/sqllogictest/test_files/joins.slt +++ b/datafusion/sqllogictest/test_files/joins.slt @@ -3466,6 +3466,23 @@ SortPreservingMergeExec: [a@0 ASC] ----------------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 ------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, b], output_ordering=[a@0 ASC, b@1 ASC NULLS LAST], has_header=true +query TT +EXPLAIN SELECT * +FROM annotated_data as l, annotated_data as r +WHERE l.a > r.a +---- +logical_plan +Inner Join: Filter: l.a > r.a +--SubqueryAlias: l +----TableScan: annotated_data projection=[a0, a, b, c, d] +--SubqueryAlias: r +----TableScan: annotated_data projection=[a0, a, b, c, d] +physical_plan +NestedLoopJoinExec: join_type=Inner, filter=a@0 > a@1 +--RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +----CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a0, a, b, c, d], output_ordering=[a@1 ASC, b@2 ASC NULLS LAST, c@3 ASC NULLS LAST], has_header=true +--CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a0, a, b, c, d], output_ordering=[a@1 ASC, b@2 ASC NULLS LAST, c@3 ASC NULLS LAST], has_header=true + #### # Config teardown #### From 18c75669e18929ca095c47af4ebf285b14d2c814 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Metehan=20Y=C4=B1ld=C4=B1r=C4=B1m?= <100111937+metesynnada@users.noreply.github.com> Date: Mon, 25 Dec 2023 23:12:51 +0300 Subject: [PATCH 496/572] [MINOR] Remove duplicate test utility and move one utility function for better organization (#8652) * Code rearrange * Update stream_join_utils.rs --- .../src/joins/stream_join_utils.rs | 156 +++++++++++------- .../src/joins/symmetric_hash_join.rs | 11 +- datafusion/physical-plan/src/joins/utils.rs | 90 +--------- 3 files changed, 104 insertions(+), 153 deletions(-) diff --git a/datafusion/physical-plan/src/joins/stream_join_utils.rs b/datafusion/physical-plan/src/joins/stream_join_utils.rs index 50b1618a35dd..9a4c98927683 100644 --- a/datafusion/physical-plan/src/joins/stream_join_utils.rs +++ b/datafusion/physical-plan/src/joins/stream_join_utils.rs @@ -25,23 +25,25 @@ use std::usize; use crate::joins::utils::{JoinFilter, JoinHashMapType, StatefulStreamResult}; use crate::metrics::{ExecutionPlanMetricsSet, MetricBuilder}; -use crate::{handle_async_state, handle_state, metrics}; +use crate::{handle_async_state, handle_state, metrics, ExecutionPlan}; use arrow::compute::concat_batches; use arrow_array::{ArrowPrimitiveType, NativeAdapter, PrimitiveArray, RecordBatch}; use arrow_buffer::{ArrowNativeType, BooleanBufferBuilder}; use arrow_schema::{Schema, SchemaRef}; -use async_trait::async_trait; use datafusion_common::tree_node::{Transformed, TreeNode}; use datafusion_common::{ - arrow_datafusion_err, DataFusionError, JoinSide, Result, ScalarValue, + arrow_datafusion_err, plan_datafusion_err, DataFusionError, JoinSide, Result, + ScalarValue, }; use datafusion_execution::SendableRecordBatchStream; use datafusion_expr::interval_arithmetic::Interval; use datafusion_physical_expr::expressions::Column; +use datafusion_physical_expr::intervals::cp_solver::ExprIntervalGraph; use datafusion_physical_expr::utils::collect_columns; use datafusion_physical_expr::{PhysicalExpr, PhysicalSortExpr}; +use async_trait::async_trait; use futures::{ready, FutureExt, StreamExt}; use hashbrown::raw::RawTable; use hashbrown::HashSet; @@ -175,7 +177,7 @@ impl PruningJoinHashMap { prune_length: usize, deleting_offset: u64, shrink_factor: usize, - ) -> Result<()> { + ) { // Remove elements from the list based on the pruning length. self.next.drain(0..prune_length); @@ -198,11 +200,10 @@ impl PruningJoinHashMap { // Shrink the map if necessary. self.shrink_if_necessary(shrink_factor); - Ok(()) } } -pub fn check_filter_expr_contains_sort_information( +fn check_filter_expr_contains_sort_information( expr: &Arc, reference: &Arc, ) -> bool { @@ -227,7 +228,7 @@ pub fn map_origin_col_to_filter_col( side: &JoinSide, ) -> Result> { let filter_schema = filter.schema(); - let mut col_to_col_map: HashMap = HashMap::new(); + let mut col_to_col_map = HashMap::::new(); for (filter_schema_index, index) in filter.column_indices().iter().enumerate() { if index.side.eq(side) { // Get the main field from column index: @@ -581,7 +582,7 @@ where // get the semi index (0..prune_length) .filter_map(|idx| (bitmap.get_bit(idx)).then_some(T::Native::from_usize(idx))) - .collect::>() + .collect() } pub fn combine_two_batches( @@ -763,7 +764,6 @@ pub trait EagerJoinStream { if batch.num_rows() == 0 { return Ok(StatefulStreamResult::Continue); } - self.set_state(EagerJoinStreamState::PullLeft); self.process_batch_from_right(batch) } @@ -1032,6 +1032,91 @@ impl StreamJoinMetrics { } } +/// Updates sorted filter expressions with corresponding node indices from the +/// expression interval graph. +/// +/// This function iterates through the provided sorted filter expressions, +/// gathers the corresponding node indices from the expression interval graph, +/// and then updates the sorted expressions with these indices. It ensures +/// that these sorted expressions are aligned with the structure of the graph. +fn update_sorted_exprs_with_node_indices( + graph: &mut ExprIntervalGraph, + sorted_exprs: &mut [SortedFilterExpr], +) { + // Extract filter expressions from the sorted expressions: + let filter_exprs = sorted_exprs + .iter() + .map(|expr| expr.filter_expr().clone()) + .collect::>(); + + // Gather corresponding node indices for the extracted filter expressions from the graph: + let child_node_indices = graph.gather_node_indices(&filter_exprs); + + // Iterate through the sorted expressions and the gathered node indices: + for (sorted_expr, (_, index)) in sorted_exprs.iter_mut().zip(child_node_indices) { + // Update each sorted expression with the corresponding node index: + sorted_expr.set_node_index(index); + } +} + +/// Prepares and sorts expressions based on a given filter, left and right execution plans, and sort expressions. +/// +/// # Arguments +/// +/// * `filter` - The join filter to base the sorting on. +/// * `left` - The left execution plan. +/// * `right` - The right execution plan. +/// * `left_sort_exprs` - The expressions to sort on the left side. +/// * `right_sort_exprs` - The expressions to sort on the right side. +/// +/// # Returns +/// +/// * A tuple consisting of the sorted filter expression for the left and right sides, and an expression interval graph. +pub fn prepare_sorted_exprs( + filter: &JoinFilter, + left: &Arc, + right: &Arc, + left_sort_exprs: &[PhysicalSortExpr], + right_sort_exprs: &[PhysicalSortExpr], +) -> Result<(SortedFilterExpr, SortedFilterExpr, ExprIntervalGraph)> { + // Build the filter order for the left side + let err = || plan_datafusion_err!("Filter does not include the child order"); + + let left_temp_sorted_filter_expr = build_filter_input_order( + JoinSide::Left, + filter, + &left.schema(), + &left_sort_exprs[0], + )? + .ok_or_else(err)?; + + // Build the filter order for the right side + let right_temp_sorted_filter_expr = build_filter_input_order( + JoinSide::Right, + filter, + &right.schema(), + &right_sort_exprs[0], + )? + .ok_or_else(err)?; + + // Collect the sorted expressions + let mut sorted_exprs = + vec![left_temp_sorted_filter_expr, right_temp_sorted_filter_expr]; + + // Build the expression interval graph + let mut graph = + ExprIntervalGraph::try_new(filter.expression().clone(), filter.schema())?; + + // Update sorted expressions with node indices + update_sorted_exprs_with_node_indices(&mut graph, &mut sorted_exprs); + + // Swap and remove to get the final sorted filter expressions + let right_sorted_filter_expr = sorted_exprs.swap_remove(1); + let left_sorted_filter_expr = sorted_exprs.swap_remove(0); + + Ok((left_sorted_filter_expr, right_sorted_filter_expr, graph)) +} + #[cfg(test)] pub mod tests { use std::sync::Arc; @@ -1043,62 +1128,15 @@ pub mod tests { }; use crate::{ expressions::{Column, PhysicalSortExpr}, + joins::test_utils::complicated_filter, joins::utils::{ColumnIndex, JoinFilter}, }; use arrow::compute::SortOptions; use arrow::datatypes::{DataType, Field, Schema}; - use datafusion_common::{JoinSide, ScalarValue}; + use datafusion_common::JoinSide; use datafusion_expr::Operator; - use datafusion_physical_expr::expressions::{binary, cast, col, lit}; - - /// Filter expr for a + b > c + 10 AND a + b < c + 100 - pub(crate) fn complicated_filter( - filter_schema: &Schema, - ) -> Result> { - let left_expr = binary( - cast( - binary( - col("0", filter_schema)?, - Operator::Plus, - col("1", filter_schema)?, - filter_schema, - )?, - filter_schema, - DataType::Int64, - )?, - Operator::Gt, - binary( - cast(col("2", filter_schema)?, filter_schema, DataType::Int64)?, - Operator::Plus, - lit(ScalarValue::Int64(Some(10))), - filter_schema, - )?, - filter_schema, - )?; - - let right_expr = binary( - cast( - binary( - col("0", filter_schema)?, - Operator::Plus, - col("1", filter_schema)?, - filter_schema, - )?, - filter_schema, - DataType::Int64, - )?, - Operator::Lt, - binary( - cast(col("2", filter_schema)?, filter_schema, DataType::Int64)?, - Operator::Plus, - lit(ScalarValue::Int64(Some(100))), - filter_schema, - )?, - filter_schema, - )?; - binary(left_expr, Operator::And, right_expr, filter_schema) - } + use datafusion_physical_expr::expressions::{binary, cast, col}; #[test] fn test_column_exchange() -> Result<()> { diff --git a/datafusion/physical-plan/src/joins/symmetric_hash_join.rs b/datafusion/physical-plan/src/joins/symmetric_hash_join.rs index b9101b57c3e5..f071a7f6015a 100644 --- a/datafusion/physical-plan/src/joins/symmetric_hash_join.rs +++ b/datafusion/physical-plan/src/joins/symmetric_hash_join.rs @@ -36,13 +36,14 @@ use crate::joins::hash_join::{build_equal_condition_join_indices, update_hash}; use crate::joins::stream_join_utils::{ calculate_filter_expr_intervals, combine_two_batches, convert_sort_expr_with_filter_schema, get_pruning_anti_indices, - get_pruning_semi_indices, record_visited_indices, EagerJoinStream, - EagerJoinStreamState, PruningJoinHashMap, SortedFilterExpr, StreamJoinMetrics, + get_pruning_semi_indices, prepare_sorted_exprs, record_visited_indices, + EagerJoinStream, EagerJoinStreamState, PruningJoinHashMap, SortedFilterExpr, + StreamJoinMetrics, }; use crate::joins::utils::{ build_batch_from_indices, build_join_schema, check_join_is_valid, - partitioned_join_output_partitioning, prepare_sorted_exprs, ColumnIndex, JoinFilter, - JoinOn, StatefulStreamResult, + partitioned_join_output_partitioning, ColumnIndex, JoinFilter, JoinOn, + StatefulStreamResult, }; use crate::{ expressions::{Column, PhysicalSortExpr}, @@ -936,7 +937,7 @@ impl OneSideHashJoiner { prune_length, self.deleted_offset as u64, HASHMAP_SHRINK_SCALE_FACTOR, - )?; + ); // Remove pruned rows from the visited rows set: for row in self.deleted_offset..(self.deleted_offset + prune_length) { self.visited_rows.remove(&row); diff --git a/datafusion/physical-plan/src/joins/utils.rs b/datafusion/physical-plan/src/joins/utils.rs index c902ba85f271..ac805b50e6a5 100644 --- a/datafusion/physical-plan/src/joins/utils.rs +++ b/datafusion/physical-plan/src/joins/utils.rs @@ -25,7 +25,6 @@ use std::sync::Arc; use std::task::{Context, Poll}; use std::usize; -use crate::joins::stream_join_utils::{build_filter_input_order, SortedFilterExpr}; use crate::metrics::{self, ExecutionPlanMetricsSet, MetricBuilder}; use crate::{ColumnStatistics, ExecutionPlan, Partitioning, Statistics}; @@ -39,13 +38,11 @@ use arrow::record_batch::{RecordBatch, RecordBatchOptions}; use datafusion_common::cast::as_boolean_array; use datafusion_common::stats::Precision; use datafusion_common::{ - plan_datafusion_err, plan_err, DataFusionError, JoinSide, JoinType, Result, - SharedResult, + plan_err, DataFusionError, JoinSide, JoinType, Result, SharedResult, }; use datafusion_expr::interval_arithmetic::Interval; use datafusion_physical_expr::equivalence::add_offset_to_expr; use datafusion_physical_expr::expressions::Column; -use datafusion_physical_expr::intervals::cp_solver::ExprIntervalGraph; use datafusion_physical_expr::utils::merge_vectors; use datafusion_physical_expr::{ LexOrdering, LexOrderingRef, PhysicalExpr, PhysicalSortExpr, @@ -1208,91 +1205,6 @@ impl BuildProbeJoinMetrics { } } -/// Updates sorted filter expressions with corresponding node indices from the -/// expression interval graph. -/// -/// This function iterates through the provided sorted filter expressions, -/// gathers the corresponding node indices from the expression interval graph, -/// and then updates the sorted expressions with these indices. It ensures -/// that these sorted expressions are aligned with the structure of the graph. -fn update_sorted_exprs_with_node_indices( - graph: &mut ExprIntervalGraph, - sorted_exprs: &mut [SortedFilterExpr], -) { - // Extract filter expressions from the sorted expressions: - let filter_exprs = sorted_exprs - .iter() - .map(|expr| expr.filter_expr().clone()) - .collect::>(); - - // Gather corresponding node indices for the extracted filter expressions from the graph: - let child_node_indices = graph.gather_node_indices(&filter_exprs); - - // Iterate through the sorted expressions and the gathered node indices: - for (sorted_expr, (_, index)) in sorted_exprs.iter_mut().zip(child_node_indices) { - // Update each sorted expression with the corresponding node index: - sorted_expr.set_node_index(index); - } -} - -/// Prepares and sorts expressions based on a given filter, left and right execution plans, and sort expressions. -/// -/// # Arguments -/// -/// * `filter` - The join filter to base the sorting on. -/// * `left` - The left execution plan. -/// * `right` - The right execution plan. -/// * `left_sort_exprs` - The expressions to sort on the left side. -/// * `right_sort_exprs` - The expressions to sort on the right side. -/// -/// # Returns -/// -/// * A tuple consisting of the sorted filter expression for the left and right sides, and an expression interval graph. -pub fn prepare_sorted_exprs( - filter: &JoinFilter, - left: &Arc, - right: &Arc, - left_sort_exprs: &[PhysicalSortExpr], - right_sort_exprs: &[PhysicalSortExpr], -) -> Result<(SortedFilterExpr, SortedFilterExpr, ExprIntervalGraph)> { - // Build the filter order for the left side - let err = || plan_datafusion_err!("Filter does not include the child order"); - - let left_temp_sorted_filter_expr = build_filter_input_order( - JoinSide::Left, - filter, - &left.schema(), - &left_sort_exprs[0], - )? - .ok_or_else(err)?; - - // Build the filter order for the right side - let right_temp_sorted_filter_expr = build_filter_input_order( - JoinSide::Right, - filter, - &right.schema(), - &right_sort_exprs[0], - )? - .ok_or_else(err)?; - - // Collect the sorted expressions - let mut sorted_exprs = - vec![left_temp_sorted_filter_expr, right_temp_sorted_filter_expr]; - - // Build the expression interval graph - let mut graph = - ExprIntervalGraph::try_new(filter.expression().clone(), filter.schema())?; - - // Update sorted expressions with node indices - update_sorted_exprs_with_node_indices(&mut graph, &mut sorted_exprs); - - // Swap and remove to get the final sorted filter expressions - let right_sorted_filter_expr = sorted_exprs.swap_remove(1); - let left_sorted_filter_expr = sorted_exprs.swap_remove(0); - - Ok((left_sorted_filter_expr, right_sorted_filter_expr, graph)) -} - /// The `handle_state` macro is designed to process the result of a state-changing /// operation, encountered e.g. in implementations of `EagerJoinStream`. It /// operates on a `StatefulStreamResult` by matching its variants and executing From ec8fd44594cada9cb0189f56ddf586ec48175ce0 Mon Sep 17 00:00:00 2001 From: Mustafa Akur <106137913+mustafasrepo@users.noreply.github.com> Date: Tue, 26 Dec 2023 01:01:10 +0300 Subject: [PATCH 497/572] [MINOR]: Add new test for filter pushdown into cross join (#8648) * Initial commit * Minor changes * Simplifications * Update UDF example * Address review --------- Co-authored-by: Mehmet Ozan Kabak --- .../optimizer/src/eliminate_cross_join.rs | 1 + datafusion/optimizer/src/push_down_filter.rs | 12 +++- datafusion/sqllogictest/src/test_context.rs | 61 ++++++++++++++----- datafusion/sqllogictest/test_files/joins.slt | 22 +++++++ 4 files changed, 78 insertions(+), 18 deletions(-) diff --git a/datafusion/optimizer/src/eliminate_cross_join.rs b/datafusion/optimizer/src/eliminate_cross_join.rs index 7c866950a622..d9e96a9f2543 100644 --- a/datafusion/optimizer/src/eliminate_cross_join.rs +++ b/datafusion/optimizer/src/eliminate_cross_join.rs @@ -45,6 +45,7 @@ impl EliminateCrossJoin { /// 'select ... from a, b where (a.x = b.y and b.xx = 100) or (a.x = b.y and b.xx = 200);' /// 'select ... from a, b, c where (a.x = b.y and b.xx = 100 and a.z = c.z) /// or (a.x = b.y and b.xx = 200 and a.z=c.z);' +/// 'select ... from a, b where a.x > b.y' /// For above queries, the join predicate is available in filters and they are moved to /// join nodes appropriately /// This fix helps to improve the performance of TPCH Q19. issue#78 diff --git a/datafusion/optimizer/src/push_down_filter.rs b/datafusion/optimizer/src/push_down_filter.rs index 4eed39a08941..9d277d18d2f7 100644 --- a/datafusion/optimizer/src/push_down_filter.rs +++ b/datafusion/optimizer/src/push_down_filter.rs @@ -965,11 +965,11 @@ impl PushDownFilter { } } -/// Convert cross join to join by pushing down filter predicate to the join condition +/// Converts the given cross join to an inner join with an empty equality +/// predicate and an empty filter condition. fn convert_cross_join_to_inner_join(cross_join: CrossJoin) -> Result { let CrossJoin { left, right, .. } = cross_join; let join_schema = build_join_schema(left.schema(), right.schema(), &JoinType::Inner)?; - // predicate is given Ok(Join { left, right, @@ -982,7 +982,8 @@ fn convert_cross_join_to_inner_join(cross_join: CrossJoin) -> Result { }) } -/// Converts the inner join with empty equality predicate and empty filter condition to the cross join +/// Converts the given inner join with an empty equality predicate and an +/// empty filter condition to a cross join. fn convert_to_cross_join_if_beneficial(plan: LogicalPlan) -> Result { if let LogicalPlan::Join(join) = &plan { // Can be converted back to cross join @@ -991,6 +992,11 @@ fn convert_to_cross_join_if_beneficial(plan: LogicalPlan) -> Result .cross_join(join.right.as_ref().clone())? .build(); } + } else if let LogicalPlan::Filter(filter) = &plan { + let new_input = + convert_to_cross_join_if_beneficial(filter.input.as_ref().clone())?; + return Filter::try_new(filter.predicate.clone(), Arc::new(new_input)) + .map(LogicalPlan::Filter); } Ok(plan) } diff --git a/datafusion/sqllogictest/src/test_context.rs b/datafusion/sqllogictest/src/test_context.rs index 941dcb69d2f4..a5ce7ccb9fe0 100644 --- a/datafusion/sqllogictest/src/test_context.rs +++ b/datafusion/sqllogictest/src/test_context.rs @@ -15,31 +15,33 @@ // specific language governing permissions and limitations // under the License. -use async_trait::async_trait; +use std::collections::HashMap; +use std::fs::File; +use std::io::Write; +use std::path::Path; +use std::sync::Arc; + +use arrow::array::{ + ArrayRef, BinaryArray, Float64Array, Int32Array, LargeBinaryArray, LargeStringArray, + StringArray, TimestampNanosecondArray, +}; +use arrow::datatypes::{DataType, Field, Schema, SchemaRef, TimeUnit}; +use arrow::record_batch::RecordBatch; use datafusion::execution::context::SessionState; -use datafusion::logical_expr::Expr; +use datafusion::logical_expr::{create_udf, Expr, ScalarUDF, Volatility}; +use datafusion::physical_expr::functions::make_scalar_function; use datafusion::physical_plan::ExecutionPlan; use datafusion::prelude::SessionConfig; use datafusion::{ - arrow::{ - array::{ - BinaryArray, Float64Array, Int32Array, LargeBinaryArray, LargeStringArray, - StringArray, TimestampNanosecondArray, - }, - datatypes::{DataType, Field, Schema, SchemaRef, TimeUnit}, - record_batch::RecordBatch, - }, catalog::{schema::MemorySchemaProvider, CatalogProvider, MemoryCatalogProvider}, datasource::{MemTable, TableProvider, TableType}, prelude::{CsvReadOptions, SessionContext}, }; +use datafusion_common::cast::as_float64_array; use datafusion_common::DataFusionError; + +use async_trait::async_trait; use log::info; -use std::collections::HashMap; -use std::fs::File; -use std::io::Write; -use std::path::Path; -use std::sync::Arc; use tempfile::TempDir; /// Context for running tests @@ -102,6 +104,8 @@ impl TestContext { } "joins.slt" => { info!("Registering partition table tables"); + let example_udf = create_example_udf(); + test_ctx.ctx.register_udf(example_udf); register_partition_table(&mut test_ctx).await; } "metadata.slt" => { @@ -348,3 +352,30 @@ pub async fn register_metadata_tables(ctx: &SessionContext) { ctx.register_batch("table_with_metadata", batch).unwrap(); } + +/// Create a UDF function named "example". See the `sample_udf.rs` example +/// file for an explanation of the API. +fn create_example_udf() -> ScalarUDF { + let adder = make_scalar_function(|args: &[ArrayRef]| { + let lhs = as_float64_array(&args[0]).expect("cast failed"); + let rhs = as_float64_array(&args[1]).expect("cast failed"); + let array = lhs + .iter() + .zip(rhs.iter()) + .map(|(lhs, rhs)| match (lhs, rhs) { + (Some(lhs), Some(rhs)) => Some(lhs + rhs), + _ => None, + }) + .collect::(); + Ok(Arc::new(array) as ArrayRef) + }); + create_udf( + "example", + // Expects two f64 values: + vec![DataType::Float64, DataType::Float64], + // Returns an f64 value: + Arc::new(DataType::Float64), + Volatility::Immutable, + adder, + ) +} diff --git a/datafusion/sqllogictest/test_files/joins.slt b/datafusion/sqllogictest/test_files/joins.slt index eee213811f44..9a349f600091 100644 --- a/datafusion/sqllogictest/test_files/joins.slt +++ b/datafusion/sqllogictest/test_files/joins.slt @@ -3483,6 +3483,28 @@ NestedLoopJoinExec: join_type=Inner, filter=a@0 > a@1 ----CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a0, a, b, c, d], output_ordering=[a@1 ASC, b@2 ASC NULLS LAST, c@3 ASC NULLS LAST], has_header=true --CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a0, a, b, c, d], output_ordering=[a@1 ASC, b@2 ASC NULLS LAST, c@3 ASC NULLS LAST], has_header=true +# Currently datafusion cannot pushdown filter conditions with scalar UDF into +# cross join. +query TT +EXPLAIN SELECT * +FROM annotated_data as t1, annotated_data as t2 +WHERE EXAMPLE(t1.a, t2.a) > 3 +---- +logical_plan +Filter: example(CAST(t1.a AS Float64), CAST(t2.a AS Float64)) > Float64(3) +--CrossJoin: +----SubqueryAlias: t1 +------TableScan: annotated_data projection=[a0, a, b, c, d] +----SubqueryAlias: t2 +------TableScan: annotated_data projection=[a0, a, b, c, d] +physical_plan +CoalesceBatchesExec: target_batch_size=2 +--FilterExec: example(CAST(a@1 AS Float64), CAST(a@6 AS Float64)) > 3 +----CrossJoinExec +------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a0, a, b, c, d], output_ordering=[a@1 ASC, b@2 ASC NULLS LAST, c@3 ASC NULLS LAST], has_header=true +------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +--------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a0, a, b, c, d], output_ordering=[a@1 ASC, b@2 ASC NULLS LAST, c@3 ASC NULLS LAST], has_header=true + #### # Config teardown #### From e10d3e2a0267c70bf36373c6811906e5b9b47703 Mon Sep 17 00:00:00 2001 From: Andrew Lamb Date: Tue, 26 Dec 2023 06:53:07 -0500 Subject: [PATCH 498/572] Rewrite bloom filters to use `contains` API (#8442) --- .../datasource/physical_plan/parquet/mod.rs | 1 + .../physical_plan/parquet/row_groups.rs | 245 +++++++----------- 2 files changed, 91 insertions(+), 155 deletions(-) diff --git a/datafusion/core/src/datasource/physical_plan/parquet/mod.rs b/datafusion/core/src/datasource/physical_plan/parquet/mod.rs index ade149da6991..76a6cc297b0e 100644 --- a/datafusion/core/src/datasource/physical_plan/parquet/mod.rs +++ b/datafusion/core/src/datasource/physical_plan/parquet/mod.rs @@ -522,6 +522,7 @@ impl FileOpener for ParquetOpener { if enable_bloom_filter && !row_groups.is_empty() { if let Some(predicate) = predicate { row_groups = row_groups::prune_row_groups_by_bloom_filters( + &file_schema, &mut builder, &row_groups, file_metadata.row_groups(), diff --git a/datafusion/core/src/datasource/physical_plan/parquet/row_groups.rs b/datafusion/core/src/datasource/physical_plan/parquet/row_groups.rs index 09e4907c9437..8a1abb7d965f 100644 --- a/datafusion/core/src/datasource/physical_plan/parquet/row_groups.rs +++ b/datafusion/core/src/datasource/physical_plan/parquet/row_groups.rs @@ -18,8 +18,7 @@ use arrow::{array::ArrayRef, datatypes::Schema}; use arrow_array::BooleanArray; use arrow_schema::FieldRef; -use datafusion_common::tree_node::{TreeNode, VisitRecursion}; -use datafusion_common::{Column, DataFusionError, Result, ScalarValue}; +use datafusion_common::{Column, ScalarValue}; use parquet::file::metadata::ColumnChunkMetaData; use parquet::schema::types::SchemaDescriptor; use parquet::{ @@ -27,19 +26,13 @@ use parquet::{ bloom_filter::Sbbf, file::metadata::RowGroupMetaData, }; -use std::{ - collections::{HashMap, HashSet}, - sync::Arc, -}; +use std::collections::{HashMap, HashSet}; use crate::datasource::listing::FileRange; use crate::datasource::physical_plan::parquet::statistics::{ max_statistics, min_statistics, parquet_column, }; -use crate::logical_expr::Operator; -use crate::physical_expr::expressions as phys_expr; use crate::physical_optimizer::pruning::{PruningPredicate, PruningStatistics}; -use crate::physical_plan::PhysicalExpr; use super::ParquetFileMetrics; @@ -118,188 +111,129 @@ pub(crate) fn prune_row_groups_by_statistics( pub(crate) async fn prune_row_groups_by_bloom_filters< T: AsyncFileReader + Send + 'static, >( + arrow_schema: &Schema, builder: &mut ParquetRecordBatchStreamBuilder, row_groups: &[usize], groups: &[RowGroupMetaData], predicate: &PruningPredicate, metrics: &ParquetFileMetrics, ) -> Vec { - let bf_predicates = match BloomFilterPruningPredicate::try_new(predicate.orig_expr()) - { - Ok(predicates) => predicates, - Err(_) => { - return row_groups.to_vec(); - } - }; let mut filtered = Vec::with_capacity(groups.len()); for idx in row_groups { - let rg_metadata = &groups[*idx]; - // get all columns bloom filter - let mut column_sbbf = - HashMap::with_capacity(bf_predicates.required_columns.len()); - for column_name in bf_predicates.required_columns.iter() { - let column_idx = match rg_metadata - .columns() - .iter() - .enumerate() - .find(|(_, column)| column.column_path().string().eq(column_name)) - { - Some((column_idx, _)) => column_idx, - None => continue, + // get all columns in the predicate that we could use a bloom filter with + let literal_columns = predicate.literal_columns(); + let mut column_sbbf = HashMap::with_capacity(literal_columns.len()); + + for column_name in literal_columns { + let Some((column_idx, _field)) = + parquet_column(builder.parquet_schema(), arrow_schema, &column_name) + else { + continue; }; + let bf = match builder .get_row_group_column_bloom_filter(*idx, column_idx) .await { - Ok(bf) => match bf { - Some(bf) => bf, - None => { - continue; - } - }, + Ok(Some(bf)) => bf, + Ok(None) => continue, // no bloom filter for this column Err(e) => { - log::error!("Error evaluating row group predicate values when using BloomFilterPruningPredicate {e}"); + log::debug!("Ignoring error reading bloom filter: {e}"); metrics.predicate_evaluation_errors.add(1); continue; } }; - column_sbbf.insert(column_name.to_owned(), bf); + column_sbbf.insert(column_name.to_string(), bf); } - if bf_predicates.prune(&column_sbbf) { + + let stats = BloomFilterStatistics { column_sbbf }; + + // Can this group be pruned? + let prune_group = match predicate.prune(&stats) { + Ok(values) => !values[0], + Err(e) => { + log::debug!("Error evaluating row group predicate on bloom filter: {e}"); + metrics.predicate_evaluation_errors.add(1); + false + } + }; + + if prune_group { metrics.row_groups_pruned.add(1); - continue; + } else { + filtered.push(*idx); } - filtered.push(*idx); } filtered } -struct BloomFilterPruningPredicate { - /// Actual pruning predicate - predicate_expr: Option, - /// The statistics required to evaluate this predicate - required_columns: Vec, +/// Implements `PruningStatistics` for Parquet Split Block Bloom Filters (SBBF) +struct BloomFilterStatistics { + /// Maps column name to the parquet bloom filter + column_sbbf: HashMap, } -impl BloomFilterPruningPredicate { - fn try_new(expr: &Arc) -> Result { - let binary_expr = expr.as_any().downcast_ref::(); - match binary_expr { - Some(binary_expr) => { - let columns = Self::get_predicate_columns(expr); - Ok(Self { - predicate_expr: Some(binary_expr.clone()), - required_columns: columns.into_iter().collect(), - }) - } - None => Err(DataFusionError::Execution( - "BloomFilterPruningPredicate only support binary expr".to_string(), - )), - } +impl PruningStatistics for BloomFilterStatistics { + fn min_values(&self, _column: &Column) -> Option { + None } - fn prune(&self, column_sbbf: &HashMap) -> bool { - Self::prune_expr_with_bloom_filter(self.predicate_expr.as_ref(), column_sbbf) + fn max_values(&self, _column: &Column) -> Option { + None } - /// Return true if the `expr` can be proved not `true` - /// based on the bloom filter. - /// - /// We only checked `BinaryExpr` but it also support `InList`, - /// Because of the `optimizer` will convert `InList` to `BinaryExpr`. - fn prune_expr_with_bloom_filter( - expr: Option<&phys_expr::BinaryExpr>, - column_sbbf: &HashMap, - ) -> bool { - let Some(expr) = expr else { - // unsupported predicate - return false; - }; - match expr.op() { - Operator::And | Operator::Or => { - let left = Self::prune_expr_with_bloom_filter( - expr.left().as_any().downcast_ref::(), - column_sbbf, - ); - let right = Self::prune_expr_with_bloom_filter( - expr.right() - .as_any() - .downcast_ref::(), - column_sbbf, - ); - match expr.op() { - Operator::And => left || right, - Operator::Or => left && right, - _ => false, - } - } - Operator::Eq => { - if let Some((col, val)) = Self::check_expr_is_col_equal_const(expr) { - if let Some(sbbf) = column_sbbf.get(col.name()) { - match val { - ScalarValue::Utf8(Some(v)) => !sbbf.check(&v.as_str()), - ScalarValue::Boolean(Some(v)) => !sbbf.check(&v), - ScalarValue::Float64(Some(v)) => !sbbf.check(&v), - ScalarValue::Float32(Some(v)) => !sbbf.check(&v), - ScalarValue::Int64(Some(v)) => !sbbf.check(&v), - ScalarValue::Int32(Some(v)) => !sbbf.check(&v), - ScalarValue::Int16(Some(v)) => !sbbf.check(&v), - ScalarValue::Int8(Some(v)) => !sbbf.check(&v), - _ => false, - } - } else { - false - } - } else { - false - } - } - _ => false, - } + fn num_containers(&self) -> usize { + 1 } - fn get_predicate_columns(expr: &Arc) -> HashSet { - let mut columns = HashSet::new(); - expr.apply(&mut |expr| { - if let Some(binary_expr) = - expr.as_any().downcast_ref::() - { - if let Some((column, _)) = - Self::check_expr_is_col_equal_const(binary_expr) - { - columns.insert(column.name().to_string()); - } - } - Ok(VisitRecursion::Continue) - }) - // no way to fail as only Ok(VisitRecursion::Continue) is returned - .unwrap(); - - columns + fn null_counts(&self, _column: &Column) -> Option { + None } - fn check_expr_is_col_equal_const( - exr: &phys_expr::BinaryExpr, - ) -> Option<(phys_expr::Column, ScalarValue)> { - if Operator::Eq.ne(exr.op()) { - return None; - } + /// Use bloom filters to determine if we are sure this column can not + /// possibly contain `values` + /// + /// The `contained` API returns false if the bloom filters knows that *ALL* + /// of the values in a column are not present. + fn contained( + &self, + column: &Column, + values: &HashSet, + ) -> Option { + let sbbf = self.column_sbbf.get(column.name.as_str())?; - let left_any = exr.left().as_any(); - let right_any = exr.right().as_any(); - if let (Some(col), Some(liter)) = ( - left_any.downcast_ref::(), - right_any.downcast_ref::(), - ) { - return Some((col.clone(), liter.value().clone())); - } - if let (Some(liter), Some(col)) = ( - left_any.downcast_ref::(), - right_any.downcast_ref::(), - ) { - return Some((col.clone(), liter.value().clone())); - } - None + // Bloom filters are probabilistic data structures that can return false + // positives (i.e. it might return true even if the value is not + // present) however, the bloom filter will return `false` if the value is + // definitely not present. + + let known_not_present = values + .iter() + .map(|value| match value { + ScalarValue::Utf8(Some(v)) => sbbf.check(&v.as_str()), + ScalarValue::Boolean(Some(v)) => sbbf.check(v), + ScalarValue::Float64(Some(v)) => sbbf.check(v), + ScalarValue::Float32(Some(v)) => sbbf.check(v), + ScalarValue::Int64(Some(v)) => sbbf.check(v), + ScalarValue::Int32(Some(v)) => sbbf.check(v), + ScalarValue::Int16(Some(v)) => sbbf.check(v), + ScalarValue::Int8(Some(v)) => sbbf.check(v), + _ => true, + }) + // The row group doesn't contain any of the values if + // all the checks are false + .all(|v| !v); + + let contains = if known_not_present { + Some(false) + } else { + // Given the bloom filter is probabilistic, we can't be sure that + // the row group actually contains the values. Return `None` to + // indicate this uncertainty + None + }; + + Some(BooleanArray::from(vec![contains])) } } @@ -1367,6 +1301,7 @@ mod tests { let metadata = builder.metadata().clone(); let pruned_row_group = prune_row_groups_by_bloom_filters( + pruning_predicate.schema(), &mut builder, row_groups, metadata.row_groups(), From 4e4d0508587096551c9a34439703f765fd96edaa Mon Sep 17 00:00:00 2001 From: tushushu <33303747+tushushu@users.noreply.github.com> Date: Tue, 26 Dec 2023 19:54:27 +0800 Subject: [PATCH 499/572] Split equivalence code into smaller modules. (#8649) * refactor * refactor * fix imports * fix ordering * private func as pub * private as pub * fix import * fix mod func * fix add_equal_conditions_test * fix project_equivalence_properties_test * fix test_ordering_satisfy * fix test_ordering_satisfy_with_equivalence2 * fix other ordering tests * fix join_equivalence_properties * fix test_expr_consists_of_constants * fix test_bridge_groups * fix test_remove_redundant_entries_eq_group * fix proj tests * test_remove_redundant_entries_oeq_class * test_schema_normalize_expr_with_equivalence * test_normalize_ordering_equivalence_classes * test_get_indices_of_matching_sort_exprs_with_order_eq * test_contains_any * test_update_ordering * test_find_longest_permutation_random * test_find_longest_permutation * test_get_meet_ordering * test_get_finer * test_normalize_sort_reqs * test_schema_normalize_sort_requirement_with_equivalence * expose func and struct * remove unused export --- datafusion/physical-expr/src/equivalence.rs | 5327 ----------------- .../physical-expr/src/equivalence/class.rs | 598 ++ .../physical-expr/src/equivalence/mod.rs | 533 ++ .../physical-expr/src/equivalence/ordering.rs | 1159 ++++ .../src/equivalence/projection.rs | 1153 ++++ .../src/equivalence/properties.rs | 2062 +++++++ 6 files changed, 5505 insertions(+), 5327 deletions(-) delete mode 100644 datafusion/physical-expr/src/equivalence.rs create mode 100644 datafusion/physical-expr/src/equivalence/class.rs create mode 100644 datafusion/physical-expr/src/equivalence/mod.rs create mode 100644 datafusion/physical-expr/src/equivalence/ordering.rs create mode 100644 datafusion/physical-expr/src/equivalence/projection.rs create mode 100644 datafusion/physical-expr/src/equivalence/properties.rs diff --git a/datafusion/physical-expr/src/equivalence.rs b/datafusion/physical-expr/src/equivalence.rs deleted file mode 100644 index defd7b5786a3..000000000000 --- a/datafusion/physical-expr/src/equivalence.rs +++ /dev/null @@ -1,5327 +0,0 @@ -// Licensed to the Apache Software Foundation (ASF) under one -// or more contributor license agreements. See the NOTICE file -// distributed with this work for additional information -// regarding copyright ownership. The ASF licenses this file -// to you under the Apache License, Version 2.0 (the -// "License"); you may not use this file except in compliance -// with the License. You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, -// software distributed under the License is distributed on an -// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -// KIND, either express or implied. See the License for the -// specific language governing permissions and limitations -// under the License. - -use std::collections::{HashMap, HashSet}; -use std::hash::{Hash, Hasher}; -use std::sync::Arc; - -use crate::expressions::{Column, Literal}; -use crate::physical_expr::deduplicate_physical_exprs; -use crate::sort_properties::{ExprOrdering, SortProperties}; -use crate::{ - physical_exprs_bag_equal, physical_exprs_contains, LexOrdering, LexOrderingRef, - LexRequirement, LexRequirementRef, PhysicalExpr, PhysicalSortExpr, - PhysicalSortRequirement, -}; - -use arrow::datatypes::SchemaRef; -use arrow_schema::SortOptions; -use datafusion_common::tree_node::{Transformed, TreeNode}; -use datafusion_common::{JoinSide, JoinType, Result}; - -use indexmap::IndexSet; -use itertools::Itertools; - -/// An `EquivalenceClass` is a set of [`Arc`]s that are known -/// to have the same value for all tuples in a relation. These are generated by -/// equality predicates (e.g. `a = b`), typically equi-join conditions and -/// equality conditions in filters. -/// -/// Two `EquivalenceClass`es are equal if they contains the same expressions in -/// without any ordering. -#[derive(Debug, Clone)] -pub struct EquivalenceClass { - /// The expressions in this equivalence class. The order doesn't - /// matter for equivalence purposes - /// - /// TODO: use a HashSet for this instead of a Vec - exprs: Vec>, -} - -impl PartialEq for EquivalenceClass { - /// Returns true if other is equal in the sense - /// of bags (multi-sets), disregarding their orderings. - fn eq(&self, other: &Self) -> bool { - physical_exprs_bag_equal(&self.exprs, &other.exprs) - } -} - -impl EquivalenceClass { - /// Create a new empty equivalence class - pub fn new_empty() -> Self { - Self { exprs: vec![] } - } - - // Create a new equivalence class from a pre-existing `Vec` - pub fn new(mut exprs: Vec>) -> Self { - deduplicate_physical_exprs(&mut exprs); - Self { exprs } - } - - /// Return the inner vector of expressions - pub fn into_vec(self) -> Vec> { - self.exprs - } - - /// Return the "canonical" expression for this class (the first element) - /// if any - fn canonical_expr(&self) -> Option> { - self.exprs.first().cloned() - } - - /// Insert the expression into this class, meaning it is known to be equal to - /// all other expressions in this class - pub fn push(&mut self, expr: Arc) { - if !self.contains(&expr) { - self.exprs.push(expr); - } - } - - /// Inserts all the expressions from other into this class - pub fn extend(&mut self, other: Self) { - for expr in other.exprs { - // use push so entries are deduplicated - self.push(expr); - } - } - - /// Returns true if this equivalence class contains t expression - pub fn contains(&self, expr: &Arc) -> bool { - physical_exprs_contains(&self.exprs, expr) - } - - /// Returns true if this equivalence class has any entries in common with `other` - pub fn contains_any(&self, other: &Self) -> bool { - self.exprs.iter().any(|e| other.contains(e)) - } - - /// return the number of items in this class - pub fn len(&self) -> usize { - self.exprs.len() - } - - /// return true if this class is empty - pub fn is_empty(&self) -> bool { - self.exprs.is_empty() - } - - /// Iterate over all elements in this class, in some arbitrary order - pub fn iter(&self) -> impl Iterator> { - self.exprs.iter() - } - - /// Return a new equivalence class that have the specified offset added to - /// each expression (used when schemas are appended such as in joins) - pub fn with_offset(&self, offset: usize) -> Self { - let new_exprs = self - .exprs - .iter() - .cloned() - .map(|e| add_offset_to_expr(e, offset)) - .collect(); - Self::new(new_exprs) - } -} - -/// Stores the mapping between source expressions and target expressions for a -/// projection. -#[derive(Debug, Clone)] -pub struct ProjectionMapping { - /// Mapping between source expressions and target expressions. - /// Vector indices correspond to the indices after projection. - map: Vec<(Arc, Arc)>, -} - -impl ProjectionMapping { - /// Constructs the mapping between a projection's input and output - /// expressions. - /// - /// For example, given the input projection expressions (`a + b`, `c + d`) - /// and an output schema with two columns `"c + d"` and `"a + b"`, the - /// projection mapping would be: - /// - /// ```text - /// [0]: (c + d, col("c + d")) - /// [1]: (a + b, col("a + b")) - /// ``` - /// - /// where `col("c + d")` means the column named `"c + d"`. - pub fn try_new( - expr: &[(Arc, String)], - input_schema: &SchemaRef, - ) -> Result { - // Construct a map from the input expressions to the output expression of the projection: - expr.iter() - .enumerate() - .map(|(expr_idx, (expression, name))| { - let target_expr = Arc::new(Column::new(name, expr_idx)) as _; - expression - .clone() - .transform_down(&|e| match e.as_any().downcast_ref::() { - Some(col) => { - // Sometimes, an expression and its name in the input_schema - // doesn't match. This can cause problems, so we make sure - // that the expression name matches with the name in `input_schema`. - // Conceptually, `source_expr` and `expression` should be the same. - let idx = col.index(); - let matching_input_field = input_schema.field(idx); - let matching_input_column = - Column::new(matching_input_field.name(), idx); - Ok(Transformed::Yes(Arc::new(matching_input_column))) - } - None => Ok(Transformed::No(e)), - }) - .map(|source_expr| (source_expr, target_expr)) - }) - .collect::>>() - .map(|map| Self { map }) - } - - /// Iterate over pairs of (source, target) expressions - pub fn iter( - &self, - ) -> impl Iterator, Arc)> + '_ { - self.map.iter() - } - - /// This function returns the target expression for a given source expression. - /// - /// # Arguments - /// - /// * `expr` - Source physical expression. - /// - /// # Returns - /// - /// An `Option` containing the target for the given source expression, - /// where a `None` value means that `expr` is not inside the mapping. - pub fn target_expr( - &self, - expr: &Arc, - ) -> Option> { - self.map - .iter() - .find(|(source, _)| source.eq(expr)) - .map(|(_, target)| target.clone()) - } -} - -/// An `EquivalenceGroup` is a collection of `EquivalenceClass`es where each -/// class represents a distinct equivalence class in a relation. -#[derive(Debug, Clone)] -pub struct EquivalenceGroup { - classes: Vec, -} - -impl EquivalenceGroup { - /// Creates an empty equivalence group. - fn empty() -> Self { - Self { classes: vec![] } - } - - /// Creates an equivalence group from the given equivalence classes. - fn new(classes: Vec) -> Self { - let mut result = Self { classes }; - result.remove_redundant_entries(); - result - } - - /// Returns how many equivalence classes there are in this group. - fn len(&self) -> usize { - self.classes.len() - } - - /// Checks whether this equivalence group is empty. - pub fn is_empty(&self) -> bool { - self.len() == 0 - } - - /// Returns an iterator over the equivalence classes in this group. - pub fn iter(&self) -> impl Iterator { - self.classes.iter() - } - - /// Adds the equality `left` = `right` to this equivalence group. - /// New equality conditions often arise after steps like `Filter(a = b)`, - /// `Alias(a, a as b)` etc. - fn add_equal_conditions( - &mut self, - left: &Arc, - right: &Arc, - ) { - let mut first_class = None; - let mut second_class = None; - for (idx, cls) in self.classes.iter().enumerate() { - if cls.contains(left) { - first_class = Some(idx); - } - if cls.contains(right) { - second_class = Some(idx); - } - } - match (first_class, second_class) { - (Some(mut first_idx), Some(mut second_idx)) => { - // If the given left and right sides belong to different classes, - // we should unify/bridge these classes. - if first_idx != second_idx { - // By convention, make sure `second_idx` is larger than `first_idx`. - if first_idx > second_idx { - (first_idx, second_idx) = (second_idx, first_idx); - } - // Remove the class at `second_idx` and merge its values with - // the class at `first_idx`. The convention above makes sure - // that `first_idx` is still valid after removing `second_idx`. - let other_class = self.classes.swap_remove(second_idx); - self.classes[first_idx].extend(other_class); - } - } - (Some(group_idx), None) => { - // Right side is new, extend left side's class: - self.classes[group_idx].push(right.clone()); - } - (None, Some(group_idx)) => { - // Left side is new, extend right side's class: - self.classes[group_idx].push(left.clone()); - } - (None, None) => { - // None of the expressions is among existing classes. - // Create a new equivalence class and extend the group. - self.classes - .push(EquivalenceClass::new(vec![left.clone(), right.clone()])); - } - } - } - - /// Removes redundant entries from this group. - fn remove_redundant_entries(&mut self) { - // Remove duplicate entries from each equivalence class: - self.classes.retain_mut(|cls| { - // Keep groups that have at least two entries as singleton class is - // meaningless (i.e. it contains no non-trivial information): - cls.len() > 1 - }); - // Unify/bridge groups that have common expressions: - self.bridge_classes() - } - - /// This utility function unifies/bridges classes that have common expressions. - /// For example, assume that we have [`EquivalenceClass`]es `[a, b]` and `[b, c]`. - /// Since both classes contain `b`, columns `a`, `b` and `c` are actually all - /// equal and belong to one class. This utility converts merges such classes. - fn bridge_classes(&mut self) { - let mut idx = 0; - while idx < self.classes.len() { - let mut next_idx = idx + 1; - let start_size = self.classes[idx].len(); - while next_idx < self.classes.len() { - if self.classes[idx].contains_any(&self.classes[next_idx]) { - let extension = self.classes.swap_remove(next_idx); - self.classes[idx].extend(extension); - } else { - next_idx += 1; - } - } - if self.classes[idx].len() > start_size { - continue; - } - idx += 1; - } - } - - /// Extends this equivalence group with the `other` equivalence group. - fn extend(&mut self, other: Self) { - self.classes.extend(other.classes); - self.remove_redundant_entries(); - } - - /// Normalizes the given physical expression according to this group. - /// The expression is replaced with the first expression in the equivalence - /// class it matches with (if any). - pub fn normalize_expr(&self, expr: Arc) -> Arc { - expr.clone() - .transform(&|expr| { - for cls in self.iter() { - if cls.contains(&expr) { - return Ok(Transformed::Yes(cls.canonical_expr().unwrap())); - } - } - Ok(Transformed::No(expr)) - }) - .unwrap_or(expr) - } - - /// Normalizes the given sort expression according to this group. - /// The underlying physical expression is replaced with the first expression - /// in the equivalence class it matches with (if any). If the underlying - /// expression does not belong to any equivalence class in this group, returns - /// the sort expression as is. - pub fn normalize_sort_expr( - &self, - mut sort_expr: PhysicalSortExpr, - ) -> PhysicalSortExpr { - sort_expr.expr = self.normalize_expr(sort_expr.expr); - sort_expr - } - - /// Normalizes the given sort requirement according to this group. - /// The underlying physical expression is replaced with the first expression - /// in the equivalence class it matches with (if any). If the underlying - /// expression does not belong to any equivalence class in this group, returns - /// the given sort requirement as is. - pub fn normalize_sort_requirement( - &self, - mut sort_requirement: PhysicalSortRequirement, - ) -> PhysicalSortRequirement { - sort_requirement.expr = self.normalize_expr(sort_requirement.expr); - sort_requirement - } - - /// This function applies the `normalize_expr` function for all expressions - /// in `exprs` and returns the corresponding normalized physical expressions. - pub fn normalize_exprs( - &self, - exprs: impl IntoIterator>, - ) -> Vec> { - exprs - .into_iter() - .map(|expr| self.normalize_expr(expr)) - .collect() - } - - /// This function applies the `normalize_sort_expr` function for all sort - /// expressions in `sort_exprs` and returns the corresponding normalized - /// sort expressions. - pub fn normalize_sort_exprs(&self, sort_exprs: LexOrderingRef) -> LexOrdering { - // Convert sort expressions to sort requirements: - let sort_reqs = PhysicalSortRequirement::from_sort_exprs(sort_exprs.iter()); - // Normalize the requirements: - let normalized_sort_reqs = self.normalize_sort_requirements(&sort_reqs); - // Convert sort requirements back to sort expressions: - PhysicalSortRequirement::to_sort_exprs(normalized_sort_reqs) - } - - /// This function applies the `normalize_sort_requirement` function for all - /// requirements in `sort_reqs` and returns the corresponding normalized - /// sort requirements. - pub fn normalize_sort_requirements( - &self, - sort_reqs: LexRequirementRef, - ) -> LexRequirement { - collapse_lex_req( - sort_reqs - .iter() - .map(|sort_req| self.normalize_sort_requirement(sort_req.clone())) - .collect(), - ) - } - - /// Projects `expr` according to the given projection mapping. - /// If the resulting expression is invalid after projection, returns `None`. - fn project_expr( - &self, - mapping: &ProjectionMapping, - expr: &Arc, - ) -> Option> { - // First, we try to project expressions with an exact match. If we are - // unable to do this, we consult equivalence classes. - if let Some(target) = mapping.target_expr(expr) { - // If we match the source, we can project directly: - return Some(target); - } else { - // If the given expression is not inside the mapping, try to project - // expressions considering the equivalence classes. - for (source, target) in mapping.iter() { - // If we match an equivalent expression to `source`, then we can - // project. For example, if we have the mapping `(a as a1, a + c)` - // and the equivalence class `(a, b)`, expression `b` projects to `a1`. - if self - .get_equivalence_class(source) - .map_or(false, |group| group.contains(expr)) - { - return Some(target.clone()); - } - } - } - // Project a non-leaf expression by projecting its children. - let children = expr.children(); - if children.is_empty() { - // Leaf expression should be inside mapping. - return None; - } - children - .into_iter() - .map(|child| self.project_expr(mapping, &child)) - .collect::>>() - .map(|children| expr.clone().with_new_children(children).unwrap()) - } - - /// Projects this equivalence group according to the given projection mapping. - pub fn project(&self, mapping: &ProjectionMapping) -> Self { - let projected_classes = self.iter().filter_map(|cls| { - let new_class = cls - .iter() - .filter_map(|expr| self.project_expr(mapping, expr)) - .collect::>(); - (new_class.len() > 1).then_some(EquivalenceClass::new(new_class)) - }); - // TODO: Convert the algorithm below to a version that uses `HashMap`. - // once `Arc` can be stored in `HashMap`. - // See issue: https://github.com/apache/arrow-datafusion/issues/8027 - let mut new_classes = vec![]; - for (source, target) in mapping.iter() { - if new_classes.is_empty() { - new_classes.push((source, vec![target.clone()])); - } - if let Some((_, values)) = - new_classes.iter_mut().find(|(key, _)| key.eq(source)) - { - if !physical_exprs_contains(values, target) { - values.push(target.clone()); - } - } - } - // Only add equivalence classes with at least two members as singleton - // equivalence classes are meaningless. - let new_classes = new_classes - .into_iter() - .filter_map(|(_, values)| (values.len() > 1).then_some(values)) - .map(EquivalenceClass::new); - - let classes = projected_classes.chain(new_classes).collect(); - Self::new(classes) - } - - /// Returns the equivalence class containing `expr`. If no equivalence class - /// contains `expr`, returns `None`. - fn get_equivalence_class( - &self, - expr: &Arc, - ) -> Option<&EquivalenceClass> { - self.iter().find(|cls| cls.contains(expr)) - } - - /// Combine equivalence groups of the given join children. - pub fn join( - &self, - right_equivalences: &Self, - join_type: &JoinType, - left_size: usize, - on: &[(Column, Column)], - ) -> Self { - match join_type { - JoinType::Inner | JoinType::Left | JoinType::Full | JoinType::Right => { - let mut result = Self::new( - self.iter() - .cloned() - .chain( - right_equivalences - .iter() - .map(|cls| cls.with_offset(left_size)), - ) - .collect(), - ); - // In we have an inner join, expressions in the "on" condition - // are equal in the resulting table. - if join_type == &JoinType::Inner { - for (lhs, rhs) in on.iter() { - let index = rhs.index() + left_size; - let new_lhs = Arc::new(lhs.clone()) as _; - let new_rhs = Arc::new(Column::new(rhs.name(), index)) as _; - result.add_equal_conditions(&new_lhs, &new_rhs); - } - } - result - } - JoinType::LeftSemi | JoinType::LeftAnti => self.clone(), - JoinType::RightSemi | JoinType::RightAnti => right_equivalences.clone(), - } - } -} - -/// This function constructs a duplicate-free `LexOrderingReq` by filtering out -/// duplicate entries that have same physical expression inside. For example, -/// `vec![a Some(ASC), a Some(DESC)]` collapses to `vec![a Some(ASC)]`. -pub fn collapse_lex_req(input: LexRequirement) -> LexRequirement { - let mut output = Vec::::new(); - for item in input { - if !output.iter().any(|req| req.expr.eq(&item.expr)) { - output.push(item); - } - } - output -} - -/// This function constructs a duplicate-free `LexOrdering` by filtering out -/// duplicate entries that have same physical expression inside. For example, -/// `vec![a ASC, a DESC]` collapses to `vec![a ASC]`. -pub fn collapse_lex_ordering(input: LexOrdering) -> LexOrdering { - let mut output = Vec::::new(); - for item in input { - if !output.iter().any(|req| req.expr.eq(&item.expr)) { - output.push(item); - } - } - output -} - -/// An `OrderingEquivalenceClass` object keeps track of different alternative -/// orderings than can describe a schema. For example, consider the following table: -/// -/// ```text -/// |a|b|c|d| -/// |1|4|3|1| -/// |2|3|3|2| -/// |3|1|2|2| -/// |3|2|1|3| -/// ``` -/// -/// Here, both `vec![a ASC, b ASC]` and `vec![c DESC, d ASC]` describe the table -/// ordering. In this case, we say that these orderings are equivalent. -#[derive(Debug, Clone, Eq, PartialEq, Hash)] -pub struct OrderingEquivalenceClass { - orderings: Vec, -} - -impl OrderingEquivalenceClass { - /// Creates new empty ordering equivalence class. - fn empty() -> Self { - Self { orderings: vec![] } - } - - /// Clears (empties) this ordering equivalence class. - pub fn clear(&mut self) { - self.orderings.clear(); - } - - /// Creates new ordering equivalence class from the given orderings. - pub fn new(orderings: Vec) -> Self { - let mut result = Self { orderings }; - result.remove_redundant_entries(); - result - } - - /// Checks whether `ordering` is a member of this equivalence class. - pub fn contains(&self, ordering: &LexOrdering) -> bool { - self.orderings.contains(ordering) - } - - /// Adds `ordering` to this equivalence class. - #[allow(dead_code)] - fn push(&mut self, ordering: LexOrdering) { - self.orderings.push(ordering); - // Make sure that there are no redundant orderings: - self.remove_redundant_entries(); - } - - /// Checks whether this ordering equivalence class is empty. - pub fn is_empty(&self) -> bool { - self.len() == 0 - } - - /// Returns an iterator over the equivalent orderings in this class. - pub fn iter(&self) -> impl Iterator { - self.orderings.iter() - } - - /// Returns how many equivalent orderings there are in this class. - pub fn len(&self) -> usize { - self.orderings.len() - } - - /// Extend this ordering equivalence class with the `other` class. - pub fn extend(&mut self, other: Self) { - self.orderings.extend(other.orderings); - // Make sure that there are no redundant orderings: - self.remove_redundant_entries(); - } - - /// Adds new orderings into this ordering equivalence class. - pub fn add_new_orderings( - &mut self, - orderings: impl IntoIterator, - ) { - self.orderings.extend(orderings); - // Make sure that there are no redundant orderings: - self.remove_redundant_entries(); - } - - /// Removes redundant orderings from this equivalence class. For instance, - /// if we already have the ordering `[a ASC, b ASC, c DESC]`, then there is - /// no need to keep ordering `[a ASC, b ASC]` in the state. - fn remove_redundant_entries(&mut self) { - let mut work = true; - while work { - work = false; - let mut idx = 0; - while idx < self.orderings.len() { - let mut ordering_idx = idx + 1; - let mut removal = self.orderings[idx].is_empty(); - while ordering_idx < self.orderings.len() { - work |= resolve_overlap(&mut self.orderings, idx, ordering_idx); - if self.orderings[idx].is_empty() { - removal = true; - break; - } - work |= resolve_overlap(&mut self.orderings, ordering_idx, idx); - if self.orderings[ordering_idx].is_empty() { - self.orderings.swap_remove(ordering_idx); - } else { - ordering_idx += 1; - } - } - if removal { - self.orderings.swap_remove(idx); - } else { - idx += 1; - } - } - } - } - - /// Returns the concatenation of all the orderings. This enables merge - /// operations to preserve all equivalent orderings simultaneously. - pub fn output_ordering(&self) -> Option { - let output_ordering = self.orderings.iter().flatten().cloned().collect(); - let output_ordering = collapse_lex_ordering(output_ordering); - (!output_ordering.is_empty()).then_some(output_ordering) - } - - // Append orderings in `other` to all existing orderings in this equivalence - // class. - pub fn join_suffix(mut self, other: &Self) -> Self { - let n_ordering = self.orderings.len(); - // Replicate entries before cross product - let n_cross = std::cmp::max(n_ordering, other.len() * n_ordering); - self.orderings = self - .orderings - .iter() - .cloned() - .cycle() - .take(n_cross) - .collect(); - // Suffix orderings of other to the current orderings. - for (outer_idx, ordering) in other.iter().enumerate() { - for idx in 0..n_ordering { - // Calculate cross product index - let idx = outer_idx * n_ordering + idx; - self.orderings[idx].extend(ordering.iter().cloned()); - } - } - self - } - - /// Adds `offset` value to the index of each expression inside this - /// ordering equivalence class. - pub fn add_offset(&mut self, offset: usize) { - for ordering in self.orderings.iter_mut() { - for sort_expr in ordering { - sort_expr.expr = add_offset_to_expr(sort_expr.expr.clone(), offset); - } - } - } - - /// Gets sort options associated with this expression if it is a leading - /// ordering expression. Otherwise, returns `None`. - fn get_options(&self, expr: &Arc) -> Option { - for ordering in self.iter() { - let leading_ordering = &ordering[0]; - if leading_ordering.expr.eq(expr) { - return Some(leading_ordering.options); - } - } - None - } -} - -/// Adds the `offset` value to `Column` indices inside `expr`. This function is -/// generally used during the update of the right table schema in join operations. -pub fn add_offset_to_expr( - expr: Arc, - offset: usize, -) -> Arc { - expr.transform_down(&|e| match e.as_any().downcast_ref::() { - Some(col) => Ok(Transformed::Yes(Arc::new(Column::new( - col.name(), - offset + col.index(), - )))), - None => Ok(Transformed::No(e)), - }) - .unwrap() - // Note that we can safely unwrap here since our transform always returns - // an `Ok` value. -} - -/// Trims `orderings[idx]` if some suffix of it overlaps with a prefix of -/// `orderings[pre_idx]`. Returns `true` if there is any overlap, `false` otherwise. -fn resolve_overlap(orderings: &mut [LexOrdering], idx: usize, pre_idx: usize) -> bool { - let length = orderings[idx].len(); - let other_length = orderings[pre_idx].len(); - for overlap in 1..=length.min(other_length) { - if orderings[idx][length - overlap..] == orderings[pre_idx][..overlap] { - orderings[idx].truncate(length - overlap); - return true; - } - } - false -} - -/// A `EquivalenceProperties` object stores useful information related to a schema. -/// Currently, it keeps track of: -/// - Equivalent expressions, e.g expressions that have same value. -/// - Valid sort expressions (orderings) for the schema. -/// - Constants expressions (e.g expressions that are known to have constant values). -/// -/// Consider table below: -/// -/// ```text -/// ┌-------┐ -/// | a | b | -/// |---|---| -/// | 1 | 9 | -/// | 2 | 8 | -/// | 3 | 7 | -/// | 5 | 5 | -/// └---┴---┘ -/// ``` -/// -/// where both `a ASC` and `b DESC` can describe the table ordering. With -/// `EquivalenceProperties`, we can keep track of these different valid sort -/// expressions and treat `a ASC` and `b DESC` on an equal footing. -/// -/// Similarly, consider the table below: -/// -/// ```text -/// ┌-------┐ -/// | a | b | -/// |---|---| -/// | 1 | 1 | -/// | 2 | 2 | -/// | 3 | 3 | -/// | 5 | 5 | -/// └---┴---┘ -/// ``` -/// -/// where columns `a` and `b` always have the same value. We keep track of such -/// equivalences inside this object. With this information, we can optimize -/// things like partitioning. For example, if the partition requirement is -/// `Hash(a)` and output partitioning is `Hash(b)`, then we can deduce that -/// the existing partitioning satisfies the requirement. -#[derive(Debug, Clone)] -pub struct EquivalenceProperties { - /// Collection of equivalence classes that store expressions with the same - /// value. - eq_group: EquivalenceGroup, - /// Equivalent sort expressions for this table. - oeq_class: OrderingEquivalenceClass, - /// Expressions whose values are constant throughout the table. - /// TODO: We do not need to track constants separately, they can be tracked - /// inside `eq_groups` as `Literal` expressions. - constants: Vec>, - /// Schema associated with this object. - schema: SchemaRef, -} - -impl EquivalenceProperties { - /// Creates an empty `EquivalenceProperties` object. - pub fn new(schema: SchemaRef) -> Self { - Self { - eq_group: EquivalenceGroup::empty(), - oeq_class: OrderingEquivalenceClass::empty(), - constants: vec![], - schema, - } - } - - /// Creates a new `EquivalenceProperties` object with the given orderings. - pub fn new_with_orderings(schema: SchemaRef, orderings: &[LexOrdering]) -> Self { - Self { - eq_group: EquivalenceGroup::empty(), - oeq_class: OrderingEquivalenceClass::new(orderings.to_vec()), - constants: vec![], - schema, - } - } - - /// Returns the associated schema. - pub fn schema(&self) -> &SchemaRef { - &self.schema - } - - /// Returns a reference to the ordering equivalence class within. - pub fn oeq_class(&self) -> &OrderingEquivalenceClass { - &self.oeq_class - } - - /// Returns a reference to the equivalence group within. - pub fn eq_group(&self) -> &EquivalenceGroup { - &self.eq_group - } - - /// Returns a reference to the constant expressions - pub fn constants(&self) -> &[Arc] { - &self.constants - } - - /// Returns the normalized version of the ordering equivalence class within. - /// Normalization removes constants and duplicates as well as standardizing - /// expressions according to the equivalence group within. - pub fn normalized_oeq_class(&self) -> OrderingEquivalenceClass { - OrderingEquivalenceClass::new( - self.oeq_class - .iter() - .map(|ordering| self.normalize_sort_exprs(ordering)) - .collect(), - ) - } - - /// Extends this `EquivalenceProperties` with the `other` object. - pub fn extend(mut self, other: Self) -> Self { - self.eq_group.extend(other.eq_group); - self.oeq_class.extend(other.oeq_class); - self.add_constants(other.constants) - } - - /// Clears (empties) the ordering equivalence class within this object. - /// Call this method when existing orderings are invalidated. - pub fn clear_orderings(&mut self) { - self.oeq_class.clear(); - } - - /// Extends this `EquivalenceProperties` by adding the orderings inside the - /// ordering equivalence class `other`. - pub fn add_ordering_equivalence_class(&mut self, other: OrderingEquivalenceClass) { - self.oeq_class.extend(other); - } - - /// Adds new orderings into the existing ordering equivalence class. - pub fn add_new_orderings( - &mut self, - orderings: impl IntoIterator, - ) { - self.oeq_class.add_new_orderings(orderings); - } - - /// Incorporates the given equivalence group to into the existing - /// equivalence group within. - pub fn add_equivalence_group(&mut self, other_eq_group: EquivalenceGroup) { - self.eq_group.extend(other_eq_group); - } - - /// Adds a new equality condition into the existing equivalence group. - /// If the given equality defines a new equivalence class, adds this new - /// equivalence class to the equivalence group. - pub fn add_equal_conditions( - &mut self, - left: &Arc, - right: &Arc, - ) { - self.eq_group.add_equal_conditions(left, right); - } - - /// Track/register physical expressions with constant values. - pub fn add_constants( - mut self, - constants: impl IntoIterator>, - ) -> Self { - for expr in self.eq_group.normalize_exprs(constants) { - if !physical_exprs_contains(&self.constants, &expr) { - self.constants.push(expr); - } - } - self - } - - /// Updates the ordering equivalence group within assuming that the table - /// is re-sorted according to the argument `sort_exprs`. Note that constants - /// and equivalence classes are unchanged as they are unaffected by a re-sort. - pub fn with_reorder(mut self, sort_exprs: Vec) -> Self { - // TODO: In some cases, existing ordering equivalences may still be valid add this analysis. - self.oeq_class = OrderingEquivalenceClass::new(vec![sort_exprs]); - self - } - - /// Normalizes the given sort expressions (i.e. `sort_exprs`) using the - /// equivalence group and the ordering equivalence class within. - /// - /// Assume that `self.eq_group` states column `a` and `b` are aliases. - /// Also assume that `self.oeq_class` states orderings `d ASC` and `a ASC, c ASC` - /// are equivalent (in the sense that both describe the ordering of the table). - /// If the `sort_exprs` argument were `vec![b ASC, c ASC, a ASC]`, then this - /// function would return `vec![a ASC, c ASC]`. Internally, it would first - /// normalize to `vec![a ASC, c ASC, a ASC]` and end up with the final result - /// after deduplication. - fn normalize_sort_exprs(&self, sort_exprs: LexOrderingRef) -> LexOrdering { - // Convert sort expressions to sort requirements: - let sort_reqs = PhysicalSortRequirement::from_sort_exprs(sort_exprs.iter()); - // Normalize the requirements: - let normalized_sort_reqs = self.normalize_sort_requirements(&sort_reqs); - // Convert sort requirements back to sort expressions: - PhysicalSortRequirement::to_sort_exprs(normalized_sort_reqs) - } - - /// Normalizes the given sort requirements (i.e. `sort_reqs`) using the - /// equivalence group and the ordering equivalence class within. It works by: - /// - Removing expressions that have a constant value from the given requirement. - /// - Replacing sections that belong to some equivalence class in the equivalence - /// group with the first entry in the matching equivalence class. - /// - /// Assume that `self.eq_group` states column `a` and `b` are aliases. - /// Also assume that `self.oeq_class` states orderings `d ASC` and `a ASC, c ASC` - /// are equivalent (in the sense that both describe the ordering of the table). - /// If the `sort_reqs` argument were `vec![b ASC, c ASC, a ASC]`, then this - /// function would return `vec![a ASC, c ASC]`. Internally, it would first - /// normalize to `vec![a ASC, c ASC, a ASC]` and end up with the final result - /// after deduplication. - fn normalize_sort_requirements( - &self, - sort_reqs: LexRequirementRef, - ) -> LexRequirement { - let normalized_sort_reqs = self.eq_group.normalize_sort_requirements(sort_reqs); - let constants_normalized = self.eq_group.normalize_exprs(self.constants.clone()); - // Prune redundant sections in the requirement: - collapse_lex_req( - normalized_sort_reqs - .iter() - .filter(|&order| { - !physical_exprs_contains(&constants_normalized, &order.expr) - }) - .cloned() - .collect(), - ) - } - - /// Checks whether the given ordering is satisfied by any of the existing - /// orderings. - pub fn ordering_satisfy(&self, given: LexOrderingRef) -> bool { - // Convert the given sort expressions to sort requirements: - let sort_requirements = PhysicalSortRequirement::from_sort_exprs(given.iter()); - self.ordering_satisfy_requirement(&sort_requirements) - } - - /// Checks whether the given sort requirements are satisfied by any of the - /// existing orderings. - pub fn ordering_satisfy_requirement(&self, reqs: LexRequirementRef) -> bool { - let mut eq_properties = self.clone(); - // First, standardize the given requirement: - let normalized_reqs = eq_properties.normalize_sort_requirements(reqs); - for normalized_req in normalized_reqs { - // Check whether given ordering is satisfied - if !eq_properties.ordering_satisfy_single(&normalized_req) { - return false; - } - // Treat satisfied keys as constants in subsequent iterations. We - // can do this because the "next" key only matters in a lexicographical - // ordering when the keys to its left have the same values. - // - // Note that these expressions are not properly "constants". This is just - // an implementation strategy confined to this function. - // - // For example, assume that the requirement is `[a ASC, (b + c) ASC]`, - // and existing equivalent orderings are `[a ASC, b ASC]` and `[c ASC]`. - // From the analysis above, we know that `[a ASC]` is satisfied. Then, - // we add column `a` as constant to the algorithm state. This enables us - // to deduce that `(b + c) ASC` is satisfied, given `a` is constant. - eq_properties = - eq_properties.add_constants(std::iter::once(normalized_req.expr)); - } - true - } - - /// Determines whether the ordering specified by the given sort requirement - /// is satisfied based on the orderings within, equivalence classes, and - /// constant expressions. - /// - /// # Arguments - /// - /// - `req`: A reference to a `PhysicalSortRequirement` for which the ordering - /// satisfaction check will be done. - /// - /// # Returns - /// - /// Returns `true` if the specified ordering is satisfied, `false` otherwise. - fn ordering_satisfy_single(&self, req: &PhysicalSortRequirement) -> bool { - let expr_ordering = self.get_expr_ordering(req.expr.clone()); - let ExprOrdering { expr, state, .. } = expr_ordering; - match state { - SortProperties::Ordered(options) => { - let sort_expr = PhysicalSortExpr { expr, options }; - sort_expr.satisfy(req, self.schema()) - } - // Singleton expressions satisfies any ordering. - SortProperties::Singleton => true, - SortProperties::Unordered => false, - } - } - - /// Checks whether the `given`` sort requirements are equal or more specific - /// than the `reference` sort requirements. - pub fn requirements_compatible( - &self, - given: LexRequirementRef, - reference: LexRequirementRef, - ) -> bool { - let normalized_given = self.normalize_sort_requirements(given); - let normalized_reference = self.normalize_sort_requirements(reference); - - (normalized_reference.len() <= normalized_given.len()) - && normalized_reference - .into_iter() - .zip(normalized_given) - .all(|(reference, given)| given.compatible(&reference)) - } - - /// Returns the finer ordering among the orderings `lhs` and `rhs`, breaking - /// any ties by choosing `lhs`. - /// - /// The finer ordering is the ordering that satisfies both of the orderings. - /// If the orderings are incomparable, returns `None`. - /// - /// For example, the finer ordering among `[a ASC]` and `[a ASC, b ASC]` is - /// the latter. - pub fn get_finer_ordering( - &self, - lhs: LexOrderingRef, - rhs: LexOrderingRef, - ) -> Option { - // Convert the given sort expressions to sort requirements: - let lhs = PhysicalSortRequirement::from_sort_exprs(lhs); - let rhs = PhysicalSortRequirement::from_sort_exprs(rhs); - let finer = self.get_finer_requirement(&lhs, &rhs); - // Convert the chosen sort requirements back to sort expressions: - finer.map(PhysicalSortRequirement::to_sort_exprs) - } - - /// Returns the finer ordering among the requirements `lhs` and `rhs`, - /// breaking any ties by choosing `lhs`. - /// - /// The finer requirements are the ones that satisfy both of the given - /// requirements. If the requirements are incomparable, returns `None`. - /// - /// For example, the finer requirements among `[a ASC]` and `[a ASC, b ASC]` - /// is the latter. - pub fn get_finer_requirement( - &self, - req1: LexRequirementRef, - req2: LexRequirementRef, - ) -> Option { - let mut lhs = self.normalize_sort_requirements(req1); - let mut rhs = self.normalize_sort_requirements(req2); - lhs.iter_mut() - .zip(rhs.iter_mut()) - .all(|(lhs, rhs)| { - lhs.expr.eq(&rhs.expr) - && match (lhs.options, rhs.options) { - (Some(lhs_opt), Some(rhs_opt)) => lhs_opt == rhs_opt, - (Some(options), None) => { - rhs.options = Some(options); - true - } - (None, Some(options)) => { - lhs.options = Some(options); - true - } - (None, None) => true, - } - }) - .then_some(if lhs.len() >= rhs.len() { lhs } else { rhs }) - } - - /// Calculates the "meet" of the given orderings (`lhs` and `rhs`). - /// The meet of a set of orderings is the finest ordering that is satisfied - /// by all the orderings in that set. For details, see: - /// - /// - /// - /// If there is no ordering that satisfies both `lhs` and `rhs`, returns - /// `None`. As an example, the meet of orderings `[a ASC]` and `[a ASC, b ASC]` - /// is `[a ASC]`. - pub fn get_meet_ordering( - &self, - lhs: LexOrderingRef, - rhs: LexOrderingRef, - ) -> Option { - let lhs = self.normalize_sort_exprs(lhs); - let rhs = self.normalize_sort_exprs(rhs); - let mut meet = vec![]; - for (lhs, rhs) in lhs.into_iter().zip(rhs.into_iter()) { - if lhs.eq(&rhs) { - meet.push(lhs); - } else { - break; - } - } - (!meet.is_empty()).then_some(meet) - } - - /// Projects argument `expr` according to `projection_mapping`, taking - /// equivalences into account. - /// - /// For example, assume that columns `a` and `c` are always equal, and that - /// `projection_mapping` encodes following mapping: - /// - /// ```text - /// a -> a1 - /// b -> b1 - /// ``` - /// - /// Then, this function projects `a + b` to `Some(a1 + b1)`, `c + b` to - /// `Some(a1 + b1)` and `d` to `None`, meaning that it cannot be projected. - pub fn project_expr( - &self, - expr: &Arc, - projection_mapping: &ProjectionMapping, - ) -> Option> { - self.eq_group.project_expr(projection_mapping, expr) - } - - /// Constructs a dependency map based on existing orderings referred to in - /// the projection. - /// - /// This function analyzes the orderings in the normalized order-equivalence - /// class and builds a dependency map. The dependency map captures relationships - /// between expressions within the orderings, helping to identify dependencies - /// and construct valid projected orderings during projection operations. - /// - /// # Parameters - /// - /// - `mapping`: A reference to the `ProjectionMapping` that defines the - /// relationship between source and target expressions. - /// - /// # Returns - /// - /// A [`DependencyMap`] representing the dependency map, where each - /// [`DependencyNode`] contains dependencies for the key [`PhysicalSortExpr`]. - /// - /// # Example - /// - /// Assume we have two equivalent orderings: `[a ASC, b ASC]` and `[a ASC, c ASC]`, - /// and the projection mapping is `[a -> a_new, b -> b_new, b + c -> b + c]`. - /// Then, the dependency map will be: - /// - /// ```text - /// a ASC: Node {Some(a_new ASC), HashSet{}} - /// b ASC: Node {Some(b_new ASC), HashSet{a ASC}} - /// c ASC: Node {None, HashSet{a ASC}} - /// ``` - fn construct_dependency_map(&self, mapping: &ProjectionMapping) -> DependencyMap { - let mut dependency_map = HashMap::new(); - for ordering in self.normalized_oeq_class().iter() { - for (idx, sort_expr) in ordering.iter().enumerate() { - let target_sort_expr = - self.project_expr(&sort_expr.expr, mapping).map(|expr| { - PhysicalSortExpr { - expr, - options: sort_expr.options, - } - }); - let is_projected = target_sort_expr.is_some(); - if is_projected - || mapping - .iter() - .any(|(source, _)| expr_refers(source, &sort_expr.expr)) - { - // Previous ordering is a dependency. Note that there is no, - // dependency for a leading ordering (i.e. the first sort - // expression). - let dependency = idx.checked_sub(1).map(|a| &ordering[a]); - // Add sort expressions that can be projected or referred to - // by any of the projection expressions to the dependency map: - dependency_map - .entry(sort_expr.clone()) - .or_insert_with(|| DependencyNode { - target_sort_expr: target_sort_expr.clone(), - dependencies: HashSet::new(), - }) - .insert_dependency(dependency); - } - if !is_projected { - // If we can not project, stop constructing the dependency - // map as remaining dependencies will be invalid after projection. - break; - } - } - } - dependency_map - } - - /// Returns a new `ProjectionMapping` where source expressions are normalized. - /// - /// This normalization ensures that source expressions are transformed into a - /// consistent representation. This is beneficial for algorithms that rely on - /// exact equalities, as it allows for more precise and reliable comparisons. - /// - /// # Parameters - /// - /// - `mapping`: A reference to the original `ProjectionMapping` to be normalized. - /// - /// # Returns - /// - /// A new `ProjectionMapping` with normalized source expressions. - fn normalized_mapping(&self, mapping: &ProjectionMapping) -> ProjectionMapping { - // Construct the mapping where source expressions are normalized. In this way - // In the algorithms below we can work on exact equalities - ProjectionMapping { - map: mapping - .iter() - .map(|(source, target)| { - let normalized_source = self.eq_group.normalize_expr(source.clone()); - (normalized_source, target.clone()) - }) - .collect(), - } - } - - /// Computes projected orderings based on a given projection mapping. - /// - /// This function takes a `ProjectionMapping` and computes the possible - /// orderings for the projected expressions. It considers dependencies - /// between expressions and generates valid orderings according to the - /// specified sort properties. - /// - /// # Parameters - /// - /// - `mapping`: A reference to the `ProjectionMapping` that defines the - /// relationship between source and target expressions. - /// - /// # Returns - /// - /// A vector of `LexOrdering` containing all valid orderings after projection. - fn projected_orderings(&self, mapping: &ProjectionMapping) -> Vec { - let mapping = self.normalized_mapping(mapping); - - // Get dependency map for existing orderings: - let dependency_map = self.construct_dependency_map(&mapping); - - let orderings = mapping.iter().flat_map(|(source, target)| { - referred_dependencies(&dependency_map, source) - .into_iter() - .filter_map(|relevant_deps| { - if let SortProperties::Ordered(options) = - get_expr_ordering(source, &relevant_deps) - { - Some((options, relevant_deps)) - } else { - // Do not consider unordered cases - None - } - }) - .flat_map(|(options, relevant_deps)| { - let sort_expr = PhysicalSortExpr { - expr: target.clone(), - options, - }; - // Generate dependent orderings (i.e. prefixes for `sort_expr`): - let mut dependency_orderings = - generate_dependency_orderings(&relevant_deps, &dependency_map); - // Append `sort_expr` to the dependent orderings: - for ordering in dependency_orderings.iter_mut() { - ordering.push(sort_expr.clone()); - } - dependency_orderings - }) - }); - - // Add valid projected orderings. For example, if existing ordering is - // `a + b` and projection is `[a -> a_new, b -> b_new]`, we need to - // preserve `a_new + b_new` as ordered. Please note that `a_new` and - // `b_new` themselves need not be ordered. Such dependencies cannot be - // deduced via the pass above. - let projected_orderings = dependency_map.iter().flat_map(|(sort_expr, node)| { - let mut prefixes = construct_prefix_orderings(sort_expr, &dependency_map); - if prefixes.is_empty() { - // If prefix is empty, there is no dependency. Insert - // empty ordering: - prefixes = vec![vec![]]; - } - // Append current ordering on top its dependencies: - for ordering in prefixes.iter_mut() { - if let Some(target) = &node.target_sort_expr { - ordering.push(target.clone()) - } - } - prefixes - }); - - // Simplify each ordering by removing redundant sections: - orderings - .chain(projected_orderings) - .map(collapse_lex_ordering) - .collect() - } - - /// Projects constants based on the provided `ProjectionMapping`. - /// - /// This function takes a `ProjectionMapping` and identifies/projects - /// constants based on the existing constants and the mapping. It ensures - /// that constants are appropriately propagated through the projection. - /// - /// # Arguments - /// - /// - `mapping`: A reference to a `ProjectionMapping` representing the - /// mapping of source expressions to target expressions in the projection. - /// - /// # Returns - /// - /// Returns a `Vec>` containing the projected constants. - fn projected_constants( - &self, - mapping: &ProjectionMapping, - ) -> Vec> { - // First, project existing constants. For example, assume that `a + b` - // is known to be constant. If the projection were `a as a_new`, `b as b_new`, - // then we would project constant `a + b` as `a_new + b_new`. - let mut projected_constants = self - .constants - .iter() - .flat_map(|expr| self.eq_group.project_expr(mapping, expr)) - .collect::>(); - // Add projection expressions that are known to be constant: - for (source, target) in mapping.iter() { - if self.is_expr_constant(source) - && !physical_exprs_contains(&projected_constants, target) - { - projected_constants.push(target.clone()); - } - } - projected_constants - } - - /// Projects the equivalences within according to `projection_mapping` - /// and `output_schema`. - pub fn project( - &self, - projection_mapping: &ProjectionMapping, - output_schema: SchemaRef, - ) -> Self { - let projected_constants = self.projected_constants(projection_mapping); - let projected_eq_group = self.eq_group.project(projection_mapping); - let projected_orderings = self.projected_orderings(projection_mapping); - Self { - eq_group: projected_eq_group, - oeq_class: OrderingEquivalenceClass::new(projected_orderings), - constants: projected_constants, - schema: output_schema, - } - } - - /// Returns the longest (potentially partial) permutation satisfying the - /// existing ordering. For example, if we have the equivalent orderings - /// `[a ASC, b ASC]` and `[c DESC]`, with `exprs` containing `[c, b, a, d]`, - /// then this function returns `([a ASC, b ASC, c DESC], [2, 1, 0])`. - /// This means that the specification `[a ASC, b ASC, c DESC]` is satisfied - /// by the existing ordering, and `[a, b, c]` resides at indices: `2, 1, 0` - /// inside the argument `exprs` (respectively). For the mathematical - /// definition of "partial permutation", see: - /// - /// - pub fn find_longest_permutation( - &self, - exprs: &[Arc], - ) -> (LexOrdering, Vec) { - let mut eq_properties = self.clone(); - let mut result = vec![]; - // The algorithm is as follows: - // - Iterate over all the expressions and insert ordered expressions - // into the result. - // - Treat inserted expressions as constants (i.e. add them as constants - // to the state). - // - Continue the above procedure until no expression is inserted; i.e. - // the algorithm reaches a fixed point. - // This algorithm should reach a fixed point in at most `exprs.len()` - // iterations. - let mut search_indices = (0..exprs.len()).collect::>(); - for _idx in 0..exprs.len() { - // Get ordered expressions with their indices. - let ordered_exprs = search_indices - .iter() - .flat_map(|&idx| { - let ExprOrdering { expr, state, .. } = - eq_properties.get_expr_ordering(exprs[idx].clone()); - if let SortProperties::Ordered(options) = state { - Some((PhysicalSortExpr { expr, options }, idx)) - } else { - None - } - }) - .collect::>(); - // We reached a fixed point, exit. - if ordered_exprs.is_empty() { - break; - } - // Remove indices that have an ordering from `search_indices`, and - // treat ordered expressions as constants in subsequent iterations. - // We can do this because the "next" key only matters in a lexicographical - // ordering when the keys to its left have the same values. - // - // Note that these expressions are not properly "constants". This is just - // an implementation strategy confined to this function. - for (PhysicalSortExpr { expr, .. }, idx) in &ordered_exprs { - eq_properties = - eq_properties.add_constants(std::iter::once(expr.clone())); - search_indices.remove(idx); - } - // Add new ordered section to the state. - result.extend(ordered_exprs); - } - result.into_iter().unzip() - } - - /// This function determines whether the provided expression is constant - /// based on the known constants. - /// - /// # Arguments - /// - /// - `expr`: A reference to a `Arc` representing the - /// expression to be checked. - /// - /// # Returns - /// - /// Returns `true` if the expression is constant according to equivalence - /// group, `false` otherwise. - fn is_expr_constant(&self, expr: &Arc) -> bool { - // As an example, assume that we know columns `a` and `b` are constant. - // Then, `a`, `b` and `a + b` will all return `true` whereas `c` will - // return `false`. - let normalized_constants = self.eq_group.normalize_exprs(self.constants.to_vec()); - let normalized_expr = self.eq_group.normalize_expr(expr.clone()); - is_constant_recurse(&normalized_constants, &normalized_expr) - } - - /// Retrieves the ordering information for a given physical expression. - /// - /// This function constructs an `ExprOrdering` object for the provided - /// expression, which encapsulates information about the expression's - /// ordering, including its [`SortProperties`]. - /// - /// # Arguments - /// - /// - `expr`: An `Arc` representing the physical expression - /// for which ordering information is sought. - /// - /// # Returns - /// - /// Returns an `ExprOrdering` object containing the ordering information for - /// the given expression. - pub fn get_expr_ordering(&self, expr: Arc) -> ExprOrdering { - ExprOrdering::new(expr.clone()) - .transform_up(&|expr| Ok(update_ordering(expr, self))) - // Guaranteed to always return `Ok`. - .unwrap() - } -} - -/// This function determines whether the provided expression is constant -/// based on the known constants. -/// -/// # Arguments -/// -/// - `constants`: A `&[Arc]` containing expressions known to -/// be a constant. -/// - `expr`: A reference to a `Arc` representing the expression -/// to check. -/// -/// # Returns -/// -/// Returns `true` if the expression is constant according to equivalence -/// group, `false` otherwise. -fn is_constant_recurse( - constants: &[Arc], - expr: &Arc, -) -> bool { - if physical_exprs_contains(constants, expr) { - return true; - } - let children = expr.children(); - !children.is_empty() && children.iter().all(|c| is_constant_recurse(constants, c)) -} - -/// This function examines whether a referring expression directly refers to a -/// given referred expression or if any of its children in the expression tree -/// refer to the specified expression. -/// -/// # Parameters -/// -/// - `referring_expr`: A reference to the referring expression (`Arc`). -/// - `referred_expr`: A reference to the referred expression (`Arc`) -/// -/// # Returns -/// -/// A boolean value indicating whether `referring_expr` refers (needs it to evaluate its result) -/// `referred_expr` or not. -fn expr_refers( - referring_expr: &Arc, - referred_expr: &Arc, -) -> bool { - referring_expr.eq(referred_expr) - || referring_expr - .children() - .iter() - .any(|child| expr_refers(child, referred_expr)) -} - -/// Wrapper struct for `Arc` to use them as keys in a hash map. -#[derive(Debug, Clone)] -struct ExprWrapper(Arc); - -impl PartialEq for ExprWrapper { - fn eq(&self, other: &Self) -> bool { - self.0.eq(&other.0) - } -} - -impl Eq for ExprWrapper {} - -impl Hash for ExprWrapper { - fn hash(&self, state: &mut H) { - self.0.hash(state); - } -} - -/// This function analyzes the dependency map to collect referred dependencies for -/// a given source expression. -/// -/// # Parameters -/// -/// - `dependency_map`: A reference to the `DependencyMap` where each -/// `PhysicalSortExpr` is associated with a `DependencyNode`. -/// - `source`: A reference to the source expression (`Arc`) -/// for which relevant dependencies need to be identified. -/// -/// # Returns -/// -/// A `Vec` containing the dependencies for the given source -/// expression. These dependencies are expressions that are referred to by -/// the source expression based on the provided dependency map. -fn referred_dependencies( - dependency_map: &DependencyMap, - source: &Arc, -) -> Vec { - // Associate `PhysicalExpr`s with `PhysicalSortExpr`s that contain them: - let mut expr_to_sort_exprs = HashMap::::new(); - for sort_expr in dependency_map - .keys() - .filter(|sort_expr| expr_refers(source, &sort_expr.expr)) - { - let key = ExprWrapper(sort_expr.expr.clone()); - expr_to_sort_exprs - .entry(key) - .or_default() - .insert(sort_expr.clone()); - } - - // Generate all valid dependencies for the source. For example, if the source - // is `a + b` and the map is `[a -> (a ASC, a DESC), b -> (b ASC)]`, we get - // `vec![HashSet(a ASC, b ASC), HashSet(a DESC, b ASC)]`. - expr_to_sort_exprs - .values() - .multi_cartesian_product() - .map(|referred_deps| referred_deps.into_iter().cloned().collect()) - .collect() -} - -/// This function recursively analyzes the dependencies of the given sort -/// expression within the given dependency map to construct lexicographical -/// orderings that include the sort expression and its dependencies. -/// -/// # Parameters -/// -/// - `referred_sort_expr`: A reference to the sort expression (`PhysicalSortExpr`) -/// for which lexicographical orderings satisfying its dependencies are to be -/// constructed. -/// - `dependency_map`: A reference to the `DependencyMap` that contains -/// dependencies for different `PhysicalSortExpr`s. -/// -/// # Returns -/// -/// A vector of lexicographical orderings (`Vec`) based on the given -/// sort expression and its dependencies. -fn construct_orderings( - referred_sort_expr: &PhysicalSortExpr, - dependency_map: &DependencyMap, -) -> Vec { - // We are sure that `referred_sort_expr` is inside `dependency_map`. - let node = &dependency_map[referred_sort_expr]; - // Since we work on intermediate nodes, we are sure `val.target_sort_expr` - // exists. - let target_sort_expr = node.target_sort_expr.clone().unwrap(); - if node.dependencies.is_empty() { - vec![vec![target_sort_expr]] - } else { - node.dependencies - .iter() - .flat_map(|dep| { - let mut orderings = construct_orderings(dep, dependency_map); - for ordering in orderings.iter_mut() { - ordering.push(target_sort_expr.clone()) - } - orderings - }) - .collect() - } -} - -/// This function retrieves the dependencies of the given relevant sort expression -/// from the given dependency map. It then constructs prefix orderings by recursively -/// analyzing the dependencies and include them in the orderings. -/// -/// # Parameters -/// -/// - `relevant_sort_expr`: A reference to the relevant sort expression -/// (`PhysicalSortExpr`) for which prefix orderings are to be constructed. -/// - `dependency_map`: A reference to the `DependencyMap` containing dependencies. -/// -/// # Returns -/// -/// A vector of prefix orderings (`Vec`) based on the given relevant -/// sort expression and its dependencies. -fn construct_prefix_orderings( - relevant_sort_expr: &PhysicalSortExpr, - dependency_map: &DependencyMap, -) -> Vec { - dependency_map[relevant_sort_expr] - .dependencies - .iter() - .flat_map(|dep| construct_orderings(dep, dependency_map)) - .collect() -} - -/// Given a set of relevant dependencies (`relevant_deps`) and a map of dependencies -/// (`dependency_map`), this function generates all possible prefix orderings -/// based on the given dependencies. -/// -/// # Parameters -/// -/// * `dependencies` - A reference to the dependencies. -/// * `dependency_map` - A reference to the map of dependencies for expressions. -/// -/// # Returns -/// -/// A vector of lexical orderings (`Vec`) representing all valid orderings -/// based on the given dependencies. -fn generate_dependency_orderings( - dependencies: &Dependencies, - dependency_map: &DependencyMap, -) -> Vec { - // Construct all the valid prefix orderings for each expression appearing - // in the projection: - let relevant_prefixes = dependencies - .iter() - .flat_map(|dep| { - let prefixes = construct_prefix_orderings(dep, dependency_map); - (!prefixes.is_empty()).then_some(prefixes) - }) - .collect::>(); - - // No dependency, dependent is a leading ordering. - if relevant_prefixes.is_empty() { - // Return an empty ordering: - return vec![vec![]]; - } - - // Generate all possible orderings where dependencies are satisfied for the - // current projection expression. For example, if expression is `a + b ASC`, - // and the dependency for `a ASC` is `[c ASC]`, the dependency for `b ASC` - // is `[d DESC]`, then we generate `[c ASC, d DESC, a + b ASC]` and - // `[d DESC, c ASC, a + b ASC]`. - relevant_prefixes - .into_iter() - .multi_cartesian_product() - .flat_map(|prefix_orderings| { - prefix_orderings - .iter() - .permutations(prefix_orderings.len()) - .map(|prefixes| prefixes.into_iter().flatten().cloned().collect()) - .collect::>() - }) - .collect() -} - -/// This function examines the given expression and the sort expressions it -/// refers to determine the ordering properties of the expression. -/// -/// # Parameters -/// -/// - `expr`: A reference to the source expression (`Arc`) for -/// which ordering properties need to be determined. -/// - `dependencies`: A reference to `Dependencies`, containing sort expressions -/// referred to by `expr`. -/// -/// # Returns -/// -/// A `SortProperties` indicating the ordering information of the given expression. -fn get_expr_ordering( - expr: &Arc, - dependencies: &Dependencies, -) -> SortProperties { - if let Some(column_order) = dependencies.iter().find(|&order| expr.eq(&order.expr)) { - // If exact match is found, return its ordering. - SortProperties::Ordered(column_order.options) - } else { - // Find orderings of its children - let child_states = expr - .children() - .iter() - .map(|child| get_expr_ordering(child, dependencies)) - .collect::>(); - // Calculate expression ordering using ordering of its children. - expr.get_ordering(&child_states) - } -} - -/// Represents a node in the dependency map used to construct projected orderings. -/// -/// A `DependencyNode` contains information about a particular sort expression, -/// including its target sort expression and a set of dependencies on other sort -/// expressions. -/// -/// # Fields -/// -/// - `target_sort_expr`: An optional `PhysicalSortExpr` representing the target -/// sort expression associated with the node. It is `None` if the sort expression -/// cannot be projected. -/// - `dependencies`: A [`Dependencies`] containing dependencies on other sort -/// expressions that are referred to by the target sort expression. -#[derive(Debug, Clone, PartialEq, Eq)] -struct DependencyNode { - target_sort_expr: Option, - dependencies: Dependencies, -} - -impl DependencyNode { - // Insert dependency to the state (if exists). - fn insert_dependency(&mut self, dependency: Option<&PhysicalSortExpr>) { - if let Some(dep) = dependency { - self.dependencies.insert(dep.clone()); - } - } -} - -type DependencyMap = HashMap; -type Dependencies = HashSet; - -/// Calculate ordering equivalence properties for the given join operation. -pub fn join_equivalence_properties( - left: EquivalenceProperties, - right: EquivalenceProperties, - join_type: &JoinType, - join_schema: SchemaRef, - maintains_input_order: &[bool], - probe_side: Option, - on: &[(Column, Column)], -) -> EquivalenceProperties { - let left_size = left.schema.fields.len(); - let mut result = EquivalenceProperties::new(join_schema); - result.add_equivalence_group(left.eq_group().join( - right.eq_group(), - join_type, - left_size, - on, - )); - - let left_oeq_class = left.oeq_class; - let mut right_oeq_class = right.oeq_class; - match maintains_input_order { - [true, false] => { - // In this special case, right side ordering can be prefixed with - // the left side ordering. - if let (Some(JoinSide::Left), JoinType::Inner) = (probe_side, join_type) { - updated_right_ordering_equivalence_class( - &mut right_oeq_class, - join_type, - left_size, - ); - - // Right side ordering equivalence properties should be prepended - // with those of the left side while constructing output ordering - // equivalence properties since stream side is the left side. - // - // For example, if the right side ordering equivalences contain - // `b ASC`, and the left side ordering equivalences contain `a ASC`, - // then we should add `a ASC, b ASC` to the ordering equivalences - // of the join output. - let out_oeq_class = left_oeq_class.join_suffix(&right_oeq_class); - result.add_ordering_equivalence_class(out_oeq_class); - } else { - result.add_ordering_equivalence_class(left_oeq_class); - } - } - [false, true] => { - updated_right_ordering_equivalence_class( - &mut right_oeq_class, - join_type, - left_size, - ); - // In this special case, left side ordering can be prefixed with - // the right side ordering. - if let (Some(JoinSide::Right), JoinType::Inner) = (probe_side, join_type) { - // Left side ordering equivalence properties should be prepended - // with those of the right side while constructing output ordering - // equivalence properties since stream side is the right side. - // - // For example, if the left side ordering equivalences contain - // `a ASC`, and the right side ordering equivalences contain `b ASC`, - // then we should add `b ASC, a ASC` to the ordering equivalences - // of the join output. - let out_oeq_class = right_oeq_class.join_suffix(&left_oeq_class); - result.add_ordering_equivalence_class(out_oeq_class); - } else { - result.add_ordering_equivalence_class(right_oeq_class); - } - } - [false, false] => {} - [true, true] => unreachable!("Cannot maintain ordering of both sides"), - _ => unreachable!("Join operators can not have more than two children"), - } - result -} - -/// In the context of a join, update the right side `OrderingEquivalenceClass` -/// so that they point to valid indices in the join output schema. -/// -/// To do so, we increment column indices by the size of the left table when -/// join schema consists of a combination of the left and right schemas. This -/// is the case for `Inner`, `Left`, `Full` and `Right` joins. For other cases, -/// indices do not change. -fn updated_right_ordering_equivalence_class( - right_oeq_class: &mut OrderingEquivalenceClass, - join_type: &JoinType, - left_size: usize, -) { - if matches!( - join_type, - JoinType::Inner | JoinType::Left | JoinType::Full | JoinType::Right - ) { - right_oeq_class.add_offset(left_size); - } -} - -/// Calculates the [`SortProperties`] of a given [`ExprOrdering`] node. -/// The node can either be a leaf node, or an intermediate node: -/// - If it is a leaf node, we directly find the order of the node by looking -/// at the given sort expression and equivalence properties if it is a `Column` -/// leaf, or we mark it as unordered. In the case of a `Literal` leaf, we mark -/// it as singleton so that it can cooperate with all ordered columns. -/// - If it is an intermediate node, the children states matter. Each `PhysicalExpr` -/// and operator has its own rules on how to propagate the children orderings. -/// However, before we engage in recursion, we check whether this intermediate -/// node directly matches with the sort expression. If there is a match, the -/// sort expression emerges at that node immediately, discarding the recursive -/// result coming from its children. -fn update_ordering( - mut node: ExprOrdering, - eq_properties: &EquivalenceProperties, -) -> Transformed { - // We have a Column, which is one of the two possible leaf node types: - let normalized_expr = eq_properties.eq_group.normalize_expr(node.expr.clone()); - if eq_properties.is_expr_constant(&normalized_expr) { - node.state = SortProperties::Singleton; - } else if let Some(options) = eq_properties - .normalized_oeq_class() - .get_options(&normalized_expr) - { - node.state = SortProperties::Ordered(options); - } else if !node.expr.children().is_empty() { - // We have an intermediate (non-leaf) node, account for its children: - node.state = node.expr.get_ordering(&node.children_state()); - } else if node.expr.as_any().is::() { - // We have a Literal, which is the other possible leaf node type: - node.state = node.expr.get_ordering(&[]); - } else { - return Transformed::No(node); - } - Transformed::Yes(node) -} - -#[cfg(test)] -mod tests { - use std::ops::Not; - use std::sync::Arc; - - use super::*; - use crate::execution_props::ExecutionProps; - use crate::expressions::{col, lit, BinaryExpr, Column, Literal}; - use crate::functions::create_physical_expr; - - use arrow::compute::{lexsort_to_indices, SortColumn}; - use arrow::datatypes::{DataType, Field, Schema}; - use arrow_array::{ArrayRef, Float64Array, RecordBatch, UInt32Array}; - use arrow_schema::{Fields, SortOptions, TimeUnit}; - use datafusion_common::{plan_datafusion_err, DataFusionError, Result, ScalarValue}; - use datafusion_expr::{BuiltinScalarFunction, Operator}; - - use itertools::{izip, Itertools}; - use rand::rngs::StdRng; - use rand::seq::SliceRandom; - use rand::{Rng, SeedableRng}; - - fn output_schema( - mapping: &ProjectionMapping, - input_schema: &Arc, - ) -> Result { - // Calculate output schema - let fields: Result> = mapping - .iter() - .map(|(source, target)| { - let name = target - .as_any() - .downcast_ref::() - .ok_or_else(|| plan_datafusion_err!("Expects to have column"))? - .name(); - let field = Field::new( - name, - source.data_type(input_schema)?, - source.nullable(input_schema)?, - ); - - Ok(field) - }) - .collect(); - - let output_schema = Arc::new(Schema::new_with_metadata( - fields?, - input_schema.metadata().clone(), - )); - - Ok(output_schema) - } - - // Generate a schema which consists of 8 columns (a, b, c, d, e, f, g, h) - fn create_test_schema() -> Result { - let a = Field::new("a", DataType::Int32, true); - let b = Field::new("b", DataType::Int32, true); - let c = Field::new("c", DataType::Int32, true); - let d = Field::new("d", DataType::Int32, true); - let e = Field::new("e", DataType::Int32, true); - let f = Field::new("f", DataType::Int32, true); - let g = Field::new("g", DataType::Int32, true); - let h = Field::new("h", DataType::Int32, true); - let schema = Arc::new(Schema::new(vec![a, b, c, d, e, f, g, h])); - - Ok(schema) - } - - /// Construct a schema with following properties - /// Schema satisfies following orderings: - /// [a ASC], [d ASC, b ASC], [e DESC, f ASC, g ASC] - /// and - /// Column [a=c] (e.g they are aliases). - fn create_test_params() -> Result<(SchemaRef, EquivalenceProperties)> { - let test_schema = create_test_schema()?; - let col_a = &col("a", &test_schema)?; - let col_b = &col("b", &test_schema)?; - let col_c = &col("c", &test_schema)?; - let col_d = &col("d", &test_schema)?; - let col_e = &col("e", &test_schema)?; - let col_f = &col("f", &test_schema)?; - let col_g = &col("g", &test_schema)?; - let mut eq_properties = EquivalenceProperties::new(test_schema.clone()); - eq_properties.add_equal_conditions(col_a, col_c); - - let option_asc = SortOptions { - descending: false, - nulls_first: false, - }; - let option_desc = SortOptions { - descending: true, - nulls_first: true, - }; - let orderings = vec![ - // [a ASC] - vec![(col_a, option_asc)], - // [d ASC, b ASC] - vec![(col_d, option_asc), (col_b, option_asc)], - // [e DESC, f ASC, g ASC] - vec![ - (col_e, option_desc), - (col_f, option_asc), - (col_g, option_asc), - ], - ]; - let orderings = convert_to_orderings(&orderings); - eq_properties.add_new_orderings(orderings); - Ok((test_schema, eq_properties)) - } - - // Generate a schema which consists of 6 columns (a, b, c, d, e, f) - fn create_test_schema_2() -> Result { - let a = Field::new("a", DataType::Float64, true); - let b = Field::new("b", DataType::Float64, true); - let c = Field::new("c", DataType::Float64, true); - let d = Field::new("d", DataType::Float64, true); - let e = Field::new("e", DataType::Float64, true); - let f = Field::new("f", DataType::Float64, true); - let schema = Arc::new(Schema::new(vec![a, b, c, d, e, f])); - - Ok(schema) - } - - /// Construct a schema with random ordering - /// among column a, b, c, d - /// where - /// Column [a=f] (e.g they are aliases). - /// Column e is constant. - fn create_random_schema(seed: u64) -> Result<(SchemaRef, EquivalenceProperties)> { - let test_schema = create_test_schema_2()?; - let col_a = &col("a", &test_schema)?; - let col_b = &col("b", &test_schema)?; - let col_c = &col("c", &test_schema)?; - let col_d = &col("d", &test_schema)?; - let col_e = &col("e", &test_schema)?; - let col_f = &col("f", &test_schema)?; - let col_exprs = [col_a, col_b, col_c, col_d, col_e, col_f]; - - let mut eq_properties = EquivalenceProperties::new(test_schema.clone()); - // Define a and f are aliases - eq_properties.add_equal_conditions(col_a, col_f); - // Column e has constant value. - eq_properties = eq_properties.add_constants([col_e.clone()]); - - // Randomly order columns for sorting - let mut rng = StdRng::seed_from_u64(seed); - let mut remaining_exprs = col_exprs[0..4].to_vec(); // only a, b, c, d are sorted - - let options_asc = SortOptions { - descending: false, - nulls_first: false, - }; - - while !remaining_exprs.is_empty() { - let n_sort_expr = rng.gen_range(0..remaining_exprs.len() + 1); - remaining_exprs.shuffle(&mut rng); - - let ordering = remaining_exprs - .drain(0..n_sort_expr) - .map(|expr| PhysicalSortExpr { - expr: expr.clone(), - options: options_asc, - }) - .collect(); - - eq_properties.add_new_orderings([ordering]); - } - - Ok((test_schema, eq_properties)) - } - - // Convert each tuple to PhysicalSortRequirement - fn convert_to_sort_reqs( - in_data: &[(&Arc, Option)], - ) -> Vec { - in_data - .iter() - .map(|(expr, options)| { - PhysicalSortRequirement::new((*expr).clone(), *options) - }) - .collect() - } - - // Convert each tuple to PhysicalSortExpr - fn convert_to_sort_exprs( - in_data: &[(&Arc, SortOptions)], - ) -> Vec { - in_data - .iter() - .map(|(expr, options)| PhysicalSortExpr { - expr: (*expr).clone(), - options: *options, - }) - .collect() - } - - // Convert each inner tuple to PhysicalSortExpr - fn convert_to_orderings( - orderings: &[Vec<(&Arc, SortOptions)>], - ) -> Vec> { - orderings - .iter() - .map(|sort_exprs| convert_to_sort_exprs(sort_exprs)) - .collect() - } - - // Convert each tuple to PhysicalSortExpr - fn convert_to_sort_exprs_owned( - in_data: &[(Arc, SortOptions)], - ) -> Vec { - in_data - .iter() - .map(|(expr, options)| PhysicalSortExpr { - expr: (*expr).clone(), - options: *options, - }) - .collect() - } - - // Convert each inner tuple to PhysicalSortExpr - fn convert_to_orderings_owned( - orderings: &[Vec<(Arc, SortOptions)>], - ) -> Vec> { - orderings - .iter() - .map(|sort_exprs| convert_to_sort_exprs_owned(sort_exprs)) - .collect() - } - - // Apply projection to the input_data, return projected equivalence properties and record batch - fn apply_projection( - proj_exprs: Vec<(Arc, String)>, - input_data: &RecordBatch, - input_eq_properties: &EquivalenceProperties, - ) -> Result<(RecordBatch, EquivalenceProperties)> { - let input_schema = input_data.schema(); - let projection_mapping = ProjectionMapping::try_new(&proj_exprs, &input_schema)?; - - let output_schema = output_schema(&projection_mapping, &input_schema)?; - let num_rows = input_data.num_rows(); - // Apply projection to the input record batch. - let projected_values = projection_mapping - .iter() - .map(|(source, _target)| source.evaluate(input_data)?.into_array(num_rows)) - .collect::>>()?; - let projected_batch = if projected_values.is_empty() { - RecordBatch::new_empty(output_schema.clone()) - } else { - RecordBatch::try_new(output_schema.clone(), projected_values)? - }; - - let projected_eq = - input_eq_properties.project(&projection_mapping, output_schema); - Ok((projected_batch, projected_eq)) - } - - #[test] - fn add_equal_conditions_test() -> Result<()> { - let schema = Arc::new(Schema::new(vec![ - Field::new("a", DataType::Int64, true), - Field::new("b", DataType::Int64, true), - Field::new("c", DataType::Int64, true), - Field::new("x", DataType::Int64, true), - Field::new("y", DataType::Int64, true), - ])); - - let mut eq_properties = EquivalenceProperties::new(schema); - let col_a_expr = Arc::new(Column::new("a", 0)) as Arc; - let col_b_expr = Arc::new(Column::new("b", 1)) as Arc; - let col_c_expr = Arc::new(Column::new("c", 2)) as Arc; - let col_x_expr = Arc::new(Column::new("x", 3)) as Arc; - let col_y_expr = Arc::new(Column::new("y", 4)) as Arc; - - // a and b are aliases - eq_properties.add_equal_conditions(&col_a_expr, &col_b_expr); - assert_eq!(eq_properties.eq_group().len(), 1); - - // This new entry is redundant, size shouldn't increase - eq_properties.add_equal_conditions(&col_b_expr, &col_a_expr); - assert_eq!(eq_properties.eq_group().len(), 1); - let eq_groups = &eq_properties.eq_group().classes[0]; - assert_eq!(eq_groups.len(), 2); - assert!(eq_groups.contains(&col_a_expr)); - assert!(eq_groups.contains(&col_b_expr)); - - // b and c are aliases. Exising equivalence class should expand, - // however there shouldn't be any new equivalence class - eq_properties.add_equal_conditions(&col_b_expr, &col_c_expr); - assert_eq!(eq_properties.eq_group().len(), 1); - let eq_groups = &eq_properties.eq_group().classes[0]; - assert_eq!(eq_groups.len(), 3); - assert!(eq_groups.contains(&col_a_expr)); - assert!(eq_groups.contains(&col_b_expr)); - assert!(eq_groups.contains(&col_c_expr)); - - // This is a new set of equality. Hence equivalent class count should be 2. - eq_properties.add_equal_conditions(&col_x_expr, &col_y_expr); - assert_eq!(eq_properties.eq_group().len(), 2); - - // This equality bridges distinct equality sets. - // Hence equivalent class count should decrease from 2 to 1. - eq_properties.add_equal_conditions(&col_x_expr, &col_a_expr); - assert_eq!(eq_properties.eq_group().len(), 1); - let eq_groups = &eq_properties.eq_group().classes[0]; - assert_eq!(eq_groups.len(), 5); - assert!(eq_groups.contains(&col_a_expr)); - assert!(eq_groups.contains(&col_b_expr)); - assert!(eq_groups.contains(&col_c_expr)); - assert!(eq_groups.contains(&col_x_expr)); - assert!(eq_groups.contains(&col_y_expr)); - - Ok(()) - } - - #[test] - fn project_equivalence_properties_test() -> Result<()> { - let input_schema = Arc::new(Schema::new(vec![ - Field::new("a", DataType::Int64, true), - Field::new("b", DataType::Int64, true), - Field::new("c", DataType::Int64, true), - ])); - - let input_properties = EquivalenceProperties::new(input_schema.clone()); - let col_a = col("a", &input_schema)?; - - // a as a1, a as a2, a as a3, a as a3 - let proj_exprs = vec![ - (col_a.clone(), "a1".to_string()), - (col_a.clone(), "a2".to_string()), - (col_a.clone(), "a3".to_string()), - (col_a.clone(), "a4".to_string()), - ]; - let projection_mapping = ProjectionMapping::try_new(&proj_exprs, &input_schema)?; - - let out_schema = output_schema(&projection_mapping, &input_schema)?; - // a as a1, a as a2, a as a3, a as a3 - let proj_exprs = vec![ - (col_a.clone(), "a1".to_string()), - (col_a.clone(), "a2".to_string()), - (col_a.clone(), "a3".to_string()), - (col_a.clone(), "a4".to_string()), - ]; - let projection_mapping = ProjectionMapping::try_new(&proj_exprs, &input_schema)?; - - // a as a1, a as a2, a as a3, a as a3 - let col_a1 = &col("a1", &out_schema)?; - let col_a2 = &col("a2", &out_schema)?; - let col_a3 = &col("a3", &out_schema)?; - let col_a4 = &col("a4", &out_schema)?; - let out_properties = input_properties.project(&projection_mapping, out_schema); - - // At the output a1=a2=a3=a4 - assert_eq!(out_properties.eq_group().len(), 1); - let eq_class = &out_properties.eq_group().classes[0]; - assert_eq!(eq_class.len(), 4); - assert!(eq_class.contains(col_a1)); - assert!(eq_class.contains(col_a2)); - assert!(eq_class.contains(col_a3)); - assert!(eq_class.contains(col_a4)); - - Ok(()) - } - - #[test] - fn test_ordering_satisfy() -> Result<()> { - let input_schema = Arc::new(Schema::new(vec![ - Field::new("a", DataType::Int64, true), - Field::new("b", DataType::Int64, true), - ])); - let crude = vec![PhysicalSortExpr { - expr: Arc::new(Column::new("a", 0)), - options: SortOptions::default(), - }]; - let finer = vec![ - PhysicalSortExpr { - expr: Arc::new(Column::new("a", 0)), - options: SortOptions::default(), - }, - PhysicalSortExpr { - expr: Arc::new(Column::new("b", 1)), - options: SortOptions::default(), - }, - ]; - // finer ordering satisfies, crude ordering should return true - let mut eq_properties_finer = EquivalenceProperties::new(input_schema.clone()); - eq_properties_finer.oeq_class.push(finer.clone()); - assert!(eq_properties_finer.ordering_satisfy(&crude)); - - // Crude ordering doesn't satisfy finer ordering. should return false - let mut eq_properties_crude = EquivalenceProperties::new(input_schema.clone()); - eq_properties_crude.oeq_class.push(crude.clone()); - assert!(!eq_properties_crude.ordering_satisfy(&finer)); - Ok(()) - } - - #[test] - fn test_ordering_satisfy_with_equivalence() -> Result<()> { - // Schema satisfies following orderings: - // [a ASC], [d ASC, b ASC], [e DESC, f ASC, g ASC] - // and - // Column [a=c] (e.g they are aliases). - let (test_schema, eq_properties) = create_test_params()?; - let col_a = &col("a", &test_schema)?; - let col_b = &col("b", &test_schema)?; - let col_c = &col("c", &test_schema)?; - let col_d = &col("d", &test_schema)?; - let col_e = &col("e", &test_schema)?; - let col_f = &col("f", &test_schema)?; - let col_g = &col("g", &test_schema)?; - let option_asc = SortOptions { - descending: false, - nulls_first: false, - }; - let option_desc = SortOptions { - descending: true, - nulls_first: true, - }; - let table_data_with_properties = - generate_table_for_eq_properties(&eq_properties, 625, 5)?; - - // First element in the tuple stores vector of requirement, second element is the expected return value for ordering_satisfy function - let requirements = vec![ - // `a ASC NULLS LAST`, expects `ordering_satisfy` to be `true`, since existing ordering `a ASC NULLS LAST, b ASC NULLS LAST` satisfies it - (vec![(col_a, option_asc)], true), - (vec![(col_a, option_desc)], false), - // Test whether equivalence works as expected - (vec![(col_c, option_asc)], true), - (vec![(col_c, option_desc)], false), - // Test whether ordering equivalence works as expected - (vec![(col_d, option_asc)], true), - (vec![(col_d, option_asc), (col_b, option_asc)], true), - (vec![(col_d, option_desc), (col_b, option_asc)], false), - ( - vec![ - (col_e, option_desc), - (col_f, option_asc), - (col_g, option_asc), - ], - true, - ), - (vec![(col_e, option_desc), (col_f, option_asc)], true), - (vec![(col_e, option_asc), (col_f, option_asc)], false), - (vec![(col_e, option_desc), (col_b, option_asc)], false), - (vec![(col_e, option_asc), (col_b, option_asc)], false), - ( - vec![ - (col_d, option_asc), - (col_b, option_asc), - (col_d, option_asc), - (col_b, option_asc), - ], - true, - ), - ( - vec![ - (col_d, option_asc), - (col_b, option_asc), - (col_e, option_desc), - (col_f, option_asc), - ], - true, - ), - ( - vec![ - (col_d, option_asc), - (col_b, option_asc), - (col_e, option_desc), - (col_b, option_asc), - ], - true, - ), - ( - vec![ - (col_d, option_asc), - (col_b, option_asc), - (col_d, option_desc), - (col_b, option_asc), - ], - true, - ), - ( - vec![ - (col_d, option_asc), - (col_b, option_asc), - (col_e, option_asc), - (col_f, option_asc), - ], - false, - ), - ( - vec![ - (col_d, option_asc), - (col_b, option_asc), - (col_e, option_asc), - (col_b, option_asc), - ], - false, - ), - (vec![(col_d, option_asc), (col_e, option_desc)], true), - ( - vec![ - (col_d, option_asc), - (col_c, option_asc), - (col_b, option_asc), - ], - true, - ), - ( - vec![ - (col_d, option_asc), - (col_e, option_desc), - (col_f, option_asc), - (col_b, option_asc), - ], - true, - ), - ( - vec![ - (col_d, option_asc), - (col_e, option_desc), - (col_c, option_asc), - (col_b, option_asc), - ], - true, - ), - ( - vec![ - (col_d, option_asc), - (col_e, option_desc), - (col_b, option_asc), - (col_f, option_asc), - ], - true, - ), - ]; - - for (cols, expected) in requirements { - let err_msg = format!("Error in test case:{cols:?}"); - let required = cols - .into_iter() - .map(|(expr, options)| PhysicalSortExpr { - expr: expr.clone(), - options, - }) - .collect::>(); - - // Check expected result with experimental result. - assert_eq!( - is_table_same_after_sort( - required.clone(), - table_data_with_properties.clone() - )?, - expected - ); - assert_eq!( - eq_properties.ordering_satisfy(&required), - expected, - "{err_msg}" - ); - } - Ok(()) - } - - #[test] - fn test_ordering_satisfy_with_equivalence2() -> Result<()> { - let test_schema = create_test_schema()?; - let col_a = &col("a", &test_schema)?; - let col_b = &col("b", &test_schema)?; - let col_c = &col("c", &test_schema)?; - let col_d = &col("d", &test_schema)?; - let col_e = &col("e", &test_schema)?; - let col_f = &col("f", &test_schema)?; - let floor_a = &create_physical_expr( - &BuiltinScalarFunction::Floor, - &[col("a", &test_schema)?], - &test_schema, - &ExecutionProps::default(), - )?; - let floor_f = &create_physical_expr( - &BuiltinScalarFunction::Floor, - &[col("f", &test_schema)?], - &test_schema, - &ExecutionProps::default(), - )?; - let exp_a = &create_physical_expr( - &BuiltinScalarFunction::Exp, - &[col("a", &test_schema)?], - &test_schema, - &ExecutionProps::default(), - )?; - let a_plus_b = Arc::new(BinaryExpr::new( - col_a.clone(), - Operator::Plus, - col_b.clone(), - )) as Arc; - let options = SortOptions { - descending: false, - nulls_first: false, - }; - - let test_cases = vec![ - // ------------ TEST CASE 1 ------------ - ( - // orderings - vec![ - // [a ASC, d ASC, b ASC] - vec![(col_a, options), (col_d, options), (col_b, options)], - // [c ASC] - vec![(col_c, options)], - ], - // equivalence classes - vec![vec![col_a, col_f]], - // constants - vec![col_e], - // requirement [a ASC, b ASC], requirement is not satisfied. - vec![(col_a, options), (col_b, options)], - // expected: requirement is not satisfied. - false, - ), - // ------------ TEST CASE 2 ------------ - ( - // orderings - vec![ - // [a ASC, c ASC, b ASC] - vec![(col_a, options), (col_c, options), (col_b, options)], - // [d ASC] - vec![(col_d, options)], - ], - // equivalence classes - vec![vec![col_a, col_f]], - // constants - vec![col_e], - // requirement [floor(a) ASC], - vec![(floor_a, options)], - // expected: requirement is satisfied. - true, - ), - // ------------ TEST CASE 2.1 ------------ - ( - // orderings - vec![ - // [a ASC, c ASC, b ASC] - vec![(col_a, options), (col_c, options), (col_b, options)], - // [d ASC] - vec![(col_d, options)], - ], - // equivalence classes - vec![vec![col_a, col_f]], - // constants - vec![col_e], - // requirement [floor(f) ASC], (Please note that a=f) - vec![(floor_f, options)], - // expected: requirement is satisfied. - true, - ), - // ------------ TEST CASE 3 ------------ - ( - // orderings - vec![ - // [a ASC, c ASC, b ASC] - vec![(col_a, options), (col_c, options), (col_b, options)], - // [d ASC] - vec![(col_d, options)], - ], - // equivalence classes - vec![vec![col_a, col_f]], - // constants - vec![col_e], - // requirement [a ASC, c ASC, a+b ASC], - vec![(col_a, options), (col_c, options), (&a_plus_b, options)], - // expected: requirement is satisfied. - true, - ), - // ------------ TEST CASE 4 ------------ - ( - // orderings - vec![ - // [a ASC, b ASC, c ASC, d ASC] - vec![ - (col_a, options), - (col_b, options), - (col_c, options), - (col_d, options), - ], - ], - // equivalence classes - vec![vec![col_a, col_f]], - // constants - vec![col_e], - // requirement [floor(a) ASC, a+b ASC], - vec![(floor_a, options), (&a_plus_b, options)], - // expected: requirement is satisfied. - false, - ), - // ------------ TEST CASE 5 ------------ - ( - // orderings - vec![ - // [a ASC, b ASC, c ASC, d ASC] - vec![ - (col_a, options), - (col_b, options), - (col_c, options), - (col_d, options), - ], - ], - // equivalence classes - vec![vec![col_a, col_f]], - // constants - vec![col_e], - // requirement [exp(a) ASC, a+b ASC], - vec![(exp_a, options), (&a_plus_b, options)], - // expected: requirement is not satisfied. - // TODO: If we know that exp function is 1-to-1 function. - // we could have deduced that above requirement is satisfied. - false, - ), - // ------------ TEST CASE 6 ------------ - ( - // orderings - vec![ - // [a ASC, d ASC, b ASC] - vec![(col_a, options), (col_d, options), (col_b, options)], - // [c ASC] - vec![(col_c, options)], - ], - // equivalence classes - vec![vec![col_a, col_f]], - // constants - vec![col_e], - // requirement [a ASC, d ASC, floor(a) ASC], - vec![(col_a, options), (col_d, options), (floor_a, options)], - // expected: requirement is satisfied. - true, - ), - // ------------ TEST CASE 7 ------------ - ( - // orderings - vec![ - // [a ASC, c ASC, b ASC] - vec![(col_a, options), (col_c, options), (col_b, options)], - // [d ASC] - vec![(col_d, options)], - ], - // equivalence classes - vec![vec![col_a, col_f]], - // constants - vec![col_e], - // requirement [a ASC, floor(a) ASC, a + b ASC], - vec![(col_a, options), (floor_a, options), (&a_plus_b, options)], - // expected: requirement is not satisfied. - false, - ), - // ------------ TEST CASE 8 ------------ - ( - // orderings - vec![ - // [a ASC, b ASC, c ASC] - vec![(col_a, options), (col_b, options), (col_c, options)], - // [d ASC] - vec![(col_d, options)], - ], - // equivalence classes - vec![vec![col_a, col_f]], - // constants - vec![col_e], - // requirement [a ASC, c ASC, floor(a) ASC, a + b ASC], - vec![ - (col_a, options), - (col_c, options), - (&floor_a, options), - (&a_plus_b, options), - ], - // expected: requirement is not satisfied. - false, - ), - // ------------ TEST CASE 9 ------------ - ( - // orderings - vec![ - // [a ASC, b ASC, c ASC, d ASC] - vec![ - (col_a, options), - (col_b, options), - (col_c, options), - (col_d, options), - ], - ], - // equivalence classes - vec![vec![col_a, col_f]], - // constants - vec![col_e], - // requirement [a ASC, b ASC, c ASC, floor(a) ASC], - vec![ - (col_a, options), - (col_b, options), - (&col_c, options), - (&floor_a, options), - ], - // expected: requirement is satisfied. - true, - ), - // ------------ TEST CASE 10 ------------ - ( - // orderings - vec![ - // [d ASC, b ASC] - vec![(col_d, options), (col_b, options)], - // [c ASC, a ASC] - vec![(col_c, options), (col_a, options)], - ], - // equivalence classes - vec![vec![col_a, col_f]], - // constants - vec![col_e], - // requirement [c ASC, d ASC, a + b ASC], - vec![(col_c, options), (col_d, options), (&a_plus_b, options)], - // expected: requirement is satisfied. - true, - ), - ]; - - for (orderings, eq_group, constants, reqs, expected) in test_cases { - let err_msg = - format!("error in test orderings: {orderings:?}, eq_group: {eq_group:?}, constants: {constants:?}, reqs: {reqs:?}, expected: {expected:?}"); - let mut eq_properties = EquivalenceProperties::new(test_schema.clone()); - let orderings = convert_to_orderings(&orderings); - eq_properties.add_new_orderings(orderings); - let eq_group = eq_group - .into_iter() - .map(|eq_class| { - let eq_classes = eq_class.into_iter().cloned().collect::>(); - EquivalenceClass::new(eq_classes) - }) - .collect::>(); - let eq_group = EquivalenceGroup::new(eq_group); - eq_properties.add_equivalence_group(eq_group); - - let constants = constants.into_iter().cloned(); - eq_properties = eq_properties.add_constants(constants); - - let reqs = convert_to_sort_exprs(&reqs); - assert_eq!( - eq_properties.ordering_satisfy(&reqs), - expected, - "{}", - err_msg - ); - } - - Ok(()) - } - - #[test] - fn test_ordering_satisfy_with_equivalence_random() -> Result<()> { - const N_RANDOM_SCHEMA: usize = 5; - const N_ELEMENTS: usize = 125; - const N_DISTINCT: usize = 5; - const SORT_OPTIONS: SortOptions = SortOptions { - descending: false, - nulls_first: false, - }; - - for seed in 0..N_RANDOM_SCHEMA { - // Create a random schema with random properties - let (test_schema, eq_properties) = create_random_schema(seed as u64)?; - // Generate a data that satisfies properties given - let table_data_with_properties = - generate_table_for_eq_properties(&eq_properties, N_ELEMENTS, N_DISTINCT)?; - let col_exprs = vec![ - col("a", &test_schema)?, - col("b", &test_schema)?, - col("c", &test_schema)?, - col("d", &test_schema)?, - col("e", &test_schema)?, - col("f", &test_schema)?, - ]; - - for n_req in 0..=col_exprs.len() { - for exprs in col_exprs.iter().combinations(n_req) { - let requirement = exprs - .into_iter() - .map(|expr| PhysicalSortExpr { - expr: expr.clone(), - options: SORT_OPTIONS, - }) - .collect::>(); - let expected = is_table_same_after_sort( - requirement.clone(), - table_data_with_properties.clone(), - )?; - let err_msg = format!( - "Error in test case requirement:{:?}, expected: {:?}, eq_properties.oeq_class: {:?}, eq_properties.eq_group: {:?}, eq_properties.constants: {:?}", - requirement, expected, eq_properties.oeq_class, eq_properties.eq_group, eq_properties.constants - ); - // Check whether ordering_satisfy API result and - // experimental result matches. - assert_eq!( - eq_properties.ordering_satisfy(&requirement), - expected, - "{}", - err_msg - ); - } - } - } - - Ok(()) - } - - #[test] - fn test_ordering_satisfy_with_equivalence_complex_random() -> Result<()> { - const N_RANDOM_SCHEMA: usize = 100; - const N_ELEMENTS: usize = 125; - const N_DISTINCT: usize = 5; - const SORT_OPTIONS: SortOptions = SortOptions { - descending: false, - nulls_first: false, - }; - - for seed in 0..N_RANDOM_SCHEMA { - // Create a random schema with random properties - let (test_schema, eq_properties) = create_random_schema(seed as u64)?; - // Generate a data that satisfies properties given - let table_data_with_properties = - generate_table_for_eq_properties(&eq_properties, N_ELEMENTS, N_DISTINCT)?; - - let floor_a = create_physical_expr( - &BuiltinScalarFunction::Floor, - &[col("a", &test_schema)?], - &test_schema, - &ExecutionProps::default(), - )?; - let a_plus_b = Arc::new(BinaryExpr::new( - col("a", &test_schema)?, - Operator::Plus, - col("b", &test_schema)?, - )) as Arc; - let exprs = vec![ - col("a", &test_schema)?, - col("b", &test_schema)?, - col("c", &test_schema)?, - col("d", &test_schema)?, - col("e", &test_schema)?, - col("f", &test_schema)?, - floor_a, - a_plus_b, - ]; - - for n_req in 0..=exprs.len() { - for exprs in exprs.iter().combinations(n_req) { - let requirement = exprs - .into_iter() - .map(|expr| PhysicalSortExpr { - expr: expr.clone(), - options: SORT_OPTIONS, - }) - .collect::>(); - let expected = is_table_same_after_sort( - requirement.clone(), - table_data_with_properties.clone(), - )?; - let err_msg = format!( - "Error in test case requirement:{:?}, expected: {:?}, eq_properties.oeq_class: {:?}, eq_properties.eq_group: {:?}, eq_properties.constants: {:?}", - requirement, expected, eq_properties.oeq_class, eq_properties.eq_group, eq_properties.constants - ); - // Check whether ordering_satisfy API result and - // experimental result matches. - - assert_eq!( - eq_properties.ordering_satisfy(&requirement), - (expected | false), - "{}", - err_msg - ); - } - } - } - - Ok(()) - } - - #[test] - fn test_ordering_satisfy_different_lengths() -> Result<()> { - let test_schema = create_test_schema()?; - let col_a = &col("a", &test_schema)?; - let col_b = &col("b", &test_schema)?; - let col_c = &col("c", &test_schema)?; - let col_d = &col("d", &test_schema)?; - let col_e = &col("e", &test_schema)?; - let col_f = &col("f", &test_schema)?; - let options = SortOptions { - descending: false, - nulls_first: false, - }; - // a=c (e.g they are aliases). - let mut eq_properties = EquivalenceProperties::new(test_schema); - eq_properties.add_equal_conditions(col_a, col_c); - - let orderings = vec![ - vec![(col_a, options)], - vec![(col_e, options)], - vec![(col_d, options), (col_f, options)], - ]; - let orderings = convert_to_orderings(&orderings); - - // Column [a ASC], [e ASC], [d ASC, f ASC] are all valid orderings for the schema. - eq_properties.add_new_orderings(orderings); - - // First entry in the tuple is required ordering, second entry is the expected flag - // that indicates whether this required ordering is satisfied. - // ([a ASC], true) indicate a ASC requirement is already satisfied by existing orderings. - let test_cases = vec![ - // [c ASC, a ASC, e ASC], expected represents this requirement is satisfied - ( - vec![(col_c, options), (col_a, options), (col_e, options)], - true, - ), - (vec![(col_c, options), (col_b, options)], false), - (vec![(col_c, options), (col_d, options)], true), - ( - vec![(col_d, options), (col_f, options), (col_b, options)], - false, - ), - (vec![(col_d, options), (col_f, options)], true), - ]; - - for (reqs, expected) in test_cases { - let err_msg = - format!("error in test reqs: {:?}, expected: {:?}", reqs, expected,); - let reqs = convert_to_sort_exprs(&reqs); - assert_eq!( - eq_properties.ordering_satisfy(&reqs), - expected, - "{}", - err_msg - ); - } - - Ok(()) - } - - #[test] - fn test_bridge_groups() -> Result<()> { - // First entry in the tuple is argument, second entry is the bridged result - let test_cases = vec![ - // ------- TEST CASE 1 -----------// - ( - vec![vec![1, 2, 3], vec![2, 4, 5], vec![11, 12, 9], vec![7, 6, 5]], - // Expected is compared with set equality. Order of the specific results may change. - vec![vec![1, 2, 3, 4, 5, 6, 7], vec![9, 11, 12]], - ), - // ------- TEST CASE 2 -----------// - ( - vec![vec![1, 2, 3], vec![3, 4, 5], vec![9, 8, 7], vec![7, 6, 5]], - // Expected - vec![vec![1, 2, 3, 4, 5, 6, 7, 8, 9]], - ), - ]; - for (entries, expected) in test_cases { - let entries = entries - .into_iter() - .map(|entry| entry.into_iter().map(lit).collect::>()) - .map(EquivalenceClass::new) - .collect::>(); - let expected = expected - .into_iter() - .map(|entry| entry.into_iter().map(lit).collect::>()) - .map(EquivalenceClass::new) - .collect::>(); - let mut eq_groups = EquivalenceGroup::new(entries.clone()); - eq_groups.bridge_classes(); - let eq_groups = eq_groups.classes; - let err_msg = format!( - "error in test entries: {:?}, expected: {:?}, actual:{:?}", - entries, expected, eq_groups - ); - assert_eq!(eq_groups.len(), expected.len(), "{}", err_msg); - for idx in 0..eq_groups.len() { - assert_eq!(&eq_groups[idx], &expected[idx], "{}", err_msg); - } - } - Ok(()) - } - - #[test] - fn test_remove_redundant_entries_eq_group() -> Result<()> { - let entries = vec![ - EquivalenceClass::new(vec![lit(1), lit(1), lit(2)]), - // This group is meaningless should be removed - EquivalenceClass::new(vec![lit(3), lit(3)]), - EquivalenceClass::new(vec![lit(4), lit(5), lit(6)]), - ]; - // Given equivalences classes are not in succinct form. - // Expected form is the most plain representation that is functionally same. - let expected = vec![ - EquivalenceClass::new(vec![lit(1), lit(2)]), - EquivalenceClass::new(vec![lit(4), lit(5), lit(6)]), - ]; - let mut eq_groups = EquivalenceGroup::new(entries); - eq_groups.remove_redundant_entries(); - - let eq_groups = eq_groups.classes; - assert_eq!(eq_groups.len(), expected.len()); - assert_eq!(eq_groups.len(), 2); - - assert_eq!(eq_groups[0], expected[0]); - assert_eq!(eq_groups[1], expected[1]); - Ok(()) - } - - #[test] - fn test_remove_redundant_entries_oeq_class() -> Result<()> { - let schema = create_test_schema()?; - let col_a = &col("a", &schema)?; - let col_b = &col("b", &schema)?; - let col_c = &col("c", &schema)?; - let col_d = &col("d", &schema)?; - let col_e = &col("e", &schema)?; - - let option_asc = SortOptions { - descending: false, - nulls_first: false, - }; - let option_desc = SortOptions { - descending: true, - nulls_first: true, - }; - - // First entry in the tuple is the given orderings for the table - // Second entry is the simplest version of the given orderings that is functionally equivalent. - let test_cases = vec![ - // ------- TEST CASE 1 --------- - ( - // ORDERINGS GIVEN - vec![ - // [a ASC, b ASC] - vec![(col_a, option_asc), (col_b, option_asc)], - ], - // EXPECTED orderings that is succinct. - vec![ - // [a ASC, b ASC] - vec![(col_a, option_asc), (col_b, option_asc)], - ], - ), - // ------- TEST CASE 2 --------- - ( - // ORDERINGS GIVEN - vec![ - // [a ASC, b ASC] - vec![(col_a, option_asc), (col_b, option_asc)], - // [a ASC, b ASC, c ASC] - vec![ - (col_a, option_asc), - (col_b, option_asc), - (col_c, option_asc), - ], - ], - // EXPECTED orderings that is succinct. - vec![ - // [a ASC, b ASC, c ASC] - vec![ - (col_a, option_asc), - (col_b, option_asc), - (col_c, option_asc), - ], - ], - ), - // ------- TEST CASE 3 --------- - ( - // ORDERINGS GIVEN - vec![ - // [a ASC, b DESC] - vec![(col_a, option_asc), (col_b, option_desc)], - // [a ASC] - vec![(col_a, option_asc)], - // [a ASC, c ASC] - vec![(col_a, option_asc), (col_c, option_asc)], - ], - // EXPECTED orderings that is succinct. - vec![ - // [a ASC, b DESC] - vec![(col_a, option_asc), (col_b, option_desc)], - // [a ASC, c ASC] - vec![(col_a, option_asc), (col_c, option_asc)], - ], - ), - // ------- TEST CASE 4 --------- - ( - // ORDERINGS GIVEN - vec![ - // [a ASC, b ASC] - vec![(col_a, option_asc), (col_b, option_asc)], - // [a ASC, b ASC, c ASC] - vec![ - (col_a, option_asc), - (col_b, option_asc), - (col_c, option_asc), - ], - // [a ASC] - vec![(col_a, option_asc)], - ], - // EXPECTED orderings that is succinct. - vec![ - // [a ASC, b ASC, c ASC] - vec![ - (col_a, option_asc), - (col_b, option_asc), - (col_c, option_asc), - ], - ], - ), - // ------- TEST CASE 5 --------- - // Empty ordering - ( - vec![vec![]], - // No ordering in the state (empty ordering is ignored). - vec![], - ), - // ------- TEST CASE 6 --------- - ( - // ORDERINGS GIVEN - vec![ - // [a ASC, b ASC] - vec![(col_a, option_asc), (col_b, option_asc)], - // [b ASC] - vec![(col_b, option_asc)], - ], - // EXPECTED orderings that is succinct. - vec![ - // [a ASC] - vec![(col_a, option_asc)], - // [b ASC] - vec![(col_b, option_asc)], - ], - ), - // ------- TEST CASE 7 --------- - // b, a - // c, a - // d, b, c - ( - // ORDERINGS GIVEN - vec![ - // [b ASC, a ASC] - vec![(col_b, option_asc), (col_a, option_asc)], - // [c ASC, a ASC] - vec![(col_c, option_asc), (col_a, option_asc)], - // [d ASC, b ASC, c ASC] - vec![ - (col_d, option_asc), - (col_b, option_asc), - (col_c, option_asc), - ], - ], - // EXPECTED orderings that is succinct. - vec![ - // [b ASC, a ASC] - vec![(col_b, option_asc), (col_a, option_asc)], - // [c ASC, a ASC] - vec![(col_c, option_asc), (col_a, option_asc)], - // [d ASC] - vec![(col_d, option_asc)], - ], - ), - // ------- TEST CASE 8 --------- - // b, e - // c, a - // d, b, e, c, a - ( - // ORDERINGS GIVEN - vec![ - // [b ASC, e ASC] - vec![(col_b, option_asc), (col_e, option_asc)], - // [c ASC, a ASC] - vec![(col_c, option_asc), (col_a, option_asc)], - // [d ASC, b ASC, e ASC, c ASC, a ASC] - vec![ - (col_d, option_asc), - (col_b, option_asc), - (col_e, option_asc), - (col_c, option_asc), - (col_a, option_asc), - ], - ], - // EXPECTED orderings that is succinct. - vec![ - // [b ASC, e ASC] - vec![(col_b, option_asc), (col_e, option_asc)], - // [c ASC, a ASC] - vec![(col_c, option_asc), (col_a, option_asc)], - // [d ASC] - vec![(col_d, option_asc)], - ], - ), - // ------- TEST CASE 9 --------- - // b - // a, b, c - // d, a, b - ( - // ORDERINGS GIVEN - vec![ - // [b ASC] - vec![(col_b, option_asc)], - // [a ASC, b ASC, c ASC] - vec![ - (col_a, option_asc), - (col_b, option_asc), - (col_c, option_asc), - ], - // [d ASC, a ASC, b ASC] - vec![ - (col_d, option_asc), - (col_a, option_asc), - (col_b, option_asc), - ], - ], - // EXPECTED orderings that is succinct. - vec![ - // [b ASC] - vec![(col_b, option_asc)], - // [a ASC, b ASC, c ASC] - vec![ - (col_a, option_asc), - (col_b, option_asc), - (col_c, option_asc), - ], - // [d ASC] - vec![(col_d, option_asc)], - ], - ), - ]; - for (orderings, expected) in test_cases { - let orderings = convert_to_orderings(&orderings); - let expected = convert_to_orderings(&expected); - let actual = OrderingEquivalenceClass::new(orderings.clone()); - let actual = actual.orderings; - let err_msg = format!( - "orderings: {:?}, expected: {:?}, actual :{:?}", - orderings, expected, actual - ); - assert_eq!(actual.len(), expected.len(), "{}", err_msg); - for elem in actual { - assert!(expected.contains(&elem), "{}", err_msg); - } - } - - Ok(()) - } - - #[test] - fn test_get_updated_right_ordering_equivalence_properties() -> Result<()> { - let join_type = JoinType::Inner; - // Join right child schema - let child_fields: Fields = ["x", "y", "z", "w"] - .into_iter() - .map(|name| Field::new(name, DataType::Int32, true)) - .collect(); - let child_schema = Schema::new(child_fields); - let col_x = &col("x", &child_schema)?; - let col_y = &col("y", &child_schema)?; - let col_z = &col("z", &child_schema)?; - let col_w = &col("w", &child_schema)?; - let option_asc = SortOptions { - descending: false, - nulls_first: false, - }; - // [x ASC, y ASC], [z ASC, w ASC] - let orderings = vec![ - vec![(col_x, option_asc), (col_y, option_asc)], - vec![(col_z, option_asc), (col_w, option_asc)], - ]; - let orderings = convert_to_orderings(&orderings); - // Right child ordering equivalences - let mut right_oeq_class = OrderingEquivalenceClass::new(orderings); - - let left_columns_len = 4; - - let fields: Fields = ["a", "b", "c", "d", "x", "y", "z", "w"] - .into_iter() - .map(|name| Field::new(name, DataType::Int32, true)) - .collect(); - - // Join Schema - let schema = Schema::new(fields); - let col_a = &col("a", &schema)?; - let col_d = &col("d", &schema)?; - let col_x = &col("x", &schema)?; - let col_y = &col("y", &schema)?; - let col_z = &col("z", &schema)?; - let col_w = &col("w", &schema)?; - - let mut join_eq_properties = EquivalenceProperties::new(Arc::new(schema)); - // a=x and d=w - join_eq_properties.add_equal_conditions(col_a, col_x); - join_eq_properties.add_equal_conditions(col_d, col_w); - - updated_right_ordering_equivalence_class( - &mut right_oeq_class, - &join_type, - left_columns_len, - ); - join_eq_properties.add_ordering_equivalence_class(right_oeq_class); - let result = join_eq_properties.oeq_class().clone(); - - // [x ASC, y ASC], [z ASC, w ASC] - let orderings = vec![ - vec![(col_x, option_asc), (col_y, option_asc)], - vec![(col_z, option_asc), (col_w, option_asc)], - ]; - let orderings = convert_to_orderings(&orderings); - let expected = OrderingEquivalenceClass::new(orderings); - - assert_eq!(result, expected); - - Ok(()) - } - - /// Checks if the table (RecordBatch) remains unchanged when sorted according to the provided `required_ordering`. - /// - /// The function works by adding a unique column of ascending integers to the original table. This column ensures - /// that rows that are otherwise indistinguishable (e.g., if they have the same values in all other columns) can - /// still be differentiated. When sorting the extended table, the unique column acts as a tie-breaker to produce - /// deterministic sorting results. - /// - /// If the table remains the same after sorting with the added unique column, it indicates that the table was - /// already sorted according to `required_ordering` to begin with. - fn is_table_same_after_sort( - mut required_ordering: Vec, - batch: RecordBatch, - ) -> Result { - // Clone the original schema and columns - let original_schema = batch.schema(); - let mut columns = batch.columns().to_vec(); - - // Create a new unique column - let n_row = batch.num_rows(); - let vals: Vec = (0..n_row).collect::>(); - let vals: Vec = vals.into_iter().map(|val| val as f64).collect(); - let unique_col = Arc::new(Float64Array::from_iter_values(vals)) as ArrayRef; - columns.push(unique_col.clone()); - - // Create a new schema with the added unique column - let unique_col_name = "unique"; - let unique_field = - Arc::new(Field::new(unique_col_name, DataType::Float64, false)); - let fields: Vec<_> = original_schema - .fields() - .iter() - .cloned() - .chain(std::iter::once(unique_field)) - .collect(); - let schema = Arc::new(Schema::new(fields)); - - // Create a new batch with the added column - let new_batch = RecordBatch::try_new(schema.clone(), columns)?; - - // Add the unique column to the required ordering to ensure deterministic results - required_ordering.push(PhysicalSortExpr { - expr: Arc::new(Column::new(unique_col_name, original_schema.fields().len())), - options: Default::default(), - }); - - // Convert the required ordering to a list of SortColumn - let sort_columns = required_ordering - .iter() - .map(|order_expr| { - let expr_result = order_expr.expr.evaluate(&new_batch)?; - let values = expr_result.into_array(new_batch.num_rows())?; - Ok(SortColumn { - values, - options: Some(order_expr.options), - }) - }) - .collect::>>()?; - - // Check if the indices after sorting match the initial ordering - let sorted_indices = lexsort_to_indices(&sort_columns, None)?; - let original_indices = UInt32Array::from_iter_values(0..n_row as u32); - - Ok(sorted_indices == original_indices) - } - - // If we already generated a random result for one of the - // expressions in the equivalence classes. For other expressions in the same - // equivalence class use same result. This util gets already calculated result, when available. - fn get_representative_arr( - eq_group: &EquivalenceClass, - existing_vec: &[Option], - schema: SchemaRef, - ) -> Option { - for expr in eq_group.iter() { - let col = expr.as_any().downcast_ref::().unwrap(); - let (idx, _field) = schema.column_with_name(col.name()).unwrap(); - if let Some(res) = &existing_vec[idx] { - return Some(res.clone()); - } - } - None - } - - // Generate a table that satisfies the given equivalence properties; i.e. - // equivalences, ordering equivalences, and constants. - fn generate_table_for_eq_properties( - eq_properties: &EquivalenceProperties, - n_elem: usize, - n_distinct: usize, - ) -> Result { - let mut rng = StdRng::seed_from_u64(23); - - let schema = eq_properties.schema(); - let mut schema_vec = vec![None; schema.fields.len()]; - - // Utility closure to generate random array - let mut generate_random_array = |num_elems: usize, max_val: usize| -> ArrayRef { - let values: Vec = (0..num_elems) - .map(|_| rng.gen_range(0..max_val) as f64 / 2.0) - .collect(); - Arc::new(Float64Array::from_iter_values(values)) - }; - - // Fill constant columns - for constant in &eq_properties.constants { - let col = constant.as_any().downcast_ref::().unwrap(); - let (idx, _field) = schema.column_with_name(col.name()).unwrap(); - let arr = Arc::new(Float64Array::from_iter_values(vec![0 as f64; n_elem])) - as ArrayRef; - schema_vec[idx] = Some(arr); - } - - // Fill columns based on ordering equivalences - for ordering in eq_properties.oeq_class.iter() { - let (sort_columns, indices): (Vec<_>, Vec<_>) = ordering - .iter() - .map(|PhysicalSortExpr { expr, options }| { - let col = expr.as_any().downcast_ref::().unwrap(); - let (idx, _field) = schema.column_with_name(col.name()).unwrap(); - let arr = generate_random_array(n_elem, n_distinct); - ( - SortColumn { - values: arr, - options: Some(*options), - }, - idx, - ) - }) - .unzip(); - - let sort_arrs = arrow::compute::lexsort(&sort_columns, None)?; - for (idx, arr) in izip!(indices, sort_arrs) { - schema_vec[idx] = Some(arr); - } - } - - // Fill columns based on equivalence groups - for eq_group in eq_properties.eq_group.iter() { - let representative_array = - get_representative_arr(eq_group, &schema_vec, schema.clone()) - .unwrap_or_else(|| generate_random_array(n_elem, n_distinct)); - - for expr in eq_group.iter() { - let col = expr.as_any().downcast_ref::().unwrap(); - let (idx, _field) = schema.column_with_name(col.name()).unwrap(); - schema_vec[idx] = Some(representative_array.clone()); - } - } - - let res: Vec<_> = schema_vec - .into_iter() - .zip(schema.fields.iter()) - .map(|(elem, field)| { - ( - field.name(), - // Generate random values for columns that do not occur in any of the groups (equivalence, ordering equivalence, constants) - elem.unwrap_or_else(|| generate_random_array(n_elem, n_distinct)), - ) - }) - .collect(); - - Ok(RecordBatch::try_from_iter(res)?) - } - - #[test] - fn test_schema_normalize_expr_with_equivalence() -> Result<()> { - let col_a = &Column::new("a", 0); - let col_b = &Column::new("b", 1); - let col_c = &Column::new("c", 2); - // Assume that column a and c are aliases. - let (_test_schema, eq_properties) = create_test_params()?; - - let col_a_expr = Arc::new(col_a.clone()) as Arc; - let col_b_expr = Arc::new(col_b.clone()) as Arc; - let col_c_expr = Arc::new(col_c.clone()) as Arc; - // Test cases for equivalence normalization, - // First entry in the tuple is argument, second entry is expected result after normalization. - let expressions = vec![ - // Normalized version of the column a and c should go to a - // (by convention all the expressions inside equivalence class are mapped to the first entry - // in this case a is the first entry in the equivalence class.) - (&col_a_expr, &col_a_expr), - (&col_c_expr, &col_a_expr), - // Cannot normalize column b - (&col_b_expr, &col_b_expr), - ]; - let eq_group = eq_properties.eq_group(); - for (expr, expected_eq) in expressions { - assert!( - expected_eq.eq(&eq_group.normalize_expr(expr.clone())), - "error in test: expr: {expr:?}" - ); - } - - Ok(()) - } - - #[test] - fn test_schema_normalize_sort_requirement_with_equivalence() -> Result<()> { - let option1 = SortOptions { - descending: false, - nulls_first: false, - }; - // Assume that column a and c are aliases. - let (test_schema, eq_properties) = create_test_params()?; - let col_a = &col("a", &test_schema)?; - let col_c = &col("c", &test_schema)?; - let col_d = &col("d", &test_schema)?; - - // Test cases for equivalence normalization - // First entry in the tuple is PhysicalSortRequirement, second entry in the tuple is - // expected PhysicalSortRequirement after normalization. - let test_cases = vec![ - (vec![(col_a, Some(option1))], vec![(col_a, Some(option1))]), - // In the normalized version column c should be replace with column a - (vec![(col_c, Some(option1))], vec![(col_a, Some(option1))]), - (vec![(col_c, None)], vec![(col_a, None)]), - (vec![(col_d, Some(option1))], vec![(col_d, Some(option1))]), - ]; - for (reqs, expected) in test_cases.into_iter() { - let reqs = convert_to_sort_reqs(&reqs); - let expected = convert_to_sort_reqs(&expected); - - let normalized = eq_properties.normalize_sort_requirements(&reqs); - assert!( - expected.eq(&normalized), - "error in test: reqs: {reqs:?}, expected: {expected:?}, normalized: {normalized:?}" - ); - } - - Ok(()) - } - - #[test] - fn test_normalize_sort_reqs() -> Result<()> { - // Schema satisfies following properties - // a=c - // and following orderings are valid - // [a ASC], [d ASC, b ASC], [e DESC, f ASC, g ASC] - let (test_schema, eq_properties) = create_test_params()?; - let col_a = &col("a", &test_schema)?; - let col_b = &col("b", &test_schema)?; - let col_c = &col("c", &test_schema)?; - let col_d = &col("d", &test_schema)?; - let col_e = &col("e", &test_schema)?; - let col_f = &col("f", &test_schema)?; - let option_asc = SortOptions { - descending: false, - nulls_first: false, - }; - let option_desc = SortOptions { - descending: true, - nulls_first: true, - }; - // First element in the tuple stores vector of requirement, second element is the expected return value for ordering_satisfy function - let requirements = vec![ - ( - vec![(col_a, Some(option_asc))], - vec![(col_a, Some(option_asc))], - ), - ( - vec![(col_a, Some(option_desc))], - vec![(col_a, Some(option_desc))], - ), - (vec![(col_a, None)], vec![(col_a, None)]), - // Test whether equivalence works as expected - ( - vec![(col_c, Some(option_asc))], - vec![(col_a, Some(option_asc))], - ), - (vec![(col_c, None)], vec![(col_a, None)]), - // Test whether ordering equivalence works as expected - ( - vec![(col_d, Some(option_asc)), (col_b, Some(option_asc))], - vec![(col_d, Some(option_asc)), (col_b, Some(option_asc))], - ), - ( - vec![(col_d, None), (col_b, None)], - vec![(col_d, None), (col_b, None)], - ), - ( - vec![(col_e, Some(option_desc)), (col_f, Some(option_asc))], - vec![(col_e, Some(option_desc)), (col_f, Some(option_asc))], - ), - // We should be able to normalize in compatible requirements also (not exactly equal) - ( - vec![(col_e, Some(option_desc)), (col_f, None)], - vec![(col_e, Some(option_desc)), (col_f, None)], - ), - ( - vec![(col_e, None), (col_f, None)], - vec![(col_e, None), (col_f, None)], - ), - ]; - - for (reqs, expected_normalized) in requirements.into_iter() { - let req = convert_to_sort_reqs(&reqs); - let expected_normalized = convert_to_sort_reqs(&expected_normalized); - - assert_eq!( - eq_properties.normalize_sort_requirements(&req), - expected_normalized - ); - } - - Ok(()) - } - - #[test] - fn test_get_finer() -> Result<()> { - let schema = create_test_schema()?; - let col_a = &col("a", &schema)?; - let col_b = &col("b", &schema)?; - let col_c = &col("c", &schema)?; - let eq_properties = EquivalenceProperties::new(schema); - let option_asc = SortOptions { - descending: false, - nulls_first: false, - }; - let option_desc = SortOptions { - descending: true, - nulls_first: true, - }; - // First entry, and second entry are the physical sort requirement that are argument for get_finer_requirement. - // Third entry is the expected result. - let tests_cases = vec![ - // Get finer requirement between [a Some(ASC)] and [a None, b Some(ASC)] - // result should be [a Some(ASC), b Some(ASC)] - ( - vec![(col_a, Some(option_asc))], - vec![(col_a, None), (col_b, Some(option_asc))], - Some(vec![(col_a, Some(option_asc)), (col_b, Some(option_asc))]), - ), - // Get finer requirement between [a Some(ASC), b Some(ASC), c Some(ASC)] and [a Some(ASC), b Some(ASC)] - // result should be [a Some(ASC), b Some(ASC), c Some(ASC)] - ( - vec![ - (col_a, Some(option_asc)), - (col_b, Some(option_asc)), - (col_c, Some(option_asc)), - ], - vec![(col_a, Some(option_asc)), (col_b, Some(option_asc))], - Some(vec![ - (col_a, Some(option_asc)), - (col_b, Some(option_asc)), - (col_c, Some(option_asc)), - ]), - ), - // Get finer requirement between [a Some(ASC), b Some(ASC)] and [a Some(ASC), b Some(DESC)] - // result should be None - ( - vec![(col_a, Some(option_asc)), (col_b, Some(option_asc))], - vec![(col_a, Some(option_asc)), (col_b, Some(option_desc))], - None, - ), - ]; - for (lhs, rhs, expected) in tests_cases { - let lhs = convert_to_sort_reqs(&lhs); - let rhs = convert_to_sort_reqs(&rhs); - let expected = expected.map(|expected| convert_to_sort_reqs(&expected)); - let finer = eq_properties.get_finer_requirement(&lhs, &rhs); - assert_eq!(finer, expected) - } - - Ok(()) - } - - #[test] - fn test_get_meet_ordering() -> Result<()> { - let schema = create_test_schema()?; - let col_a = &col("a", &schema)?; - let col_b = &col("b", &schema)?; - let eq_properties = EquivalenceProperties::new(schema); - let option_asc = SortOptions { - descending: false, - nulls_first: false, - }; - let option_desc = SortOptions { - descending: true, - nulls_first: true, - }; - let tests_cases = vec![ - // Get meet ordering between [a ASC] and [a ASC, b ASC] - // result should be [a ASC] - ( - vec![(col_a, option_asc)], - vec![(col_a, option_asc), (col_b, option_asc)], - Some(vec![(col_a, option_asc)]), - ), - // Get meet ordering between [a ASC] and [a DESC] - // result should be None. - (vec![(col_a, option_asc)], vec![(col_a, option_desc)], None), - // Get meet ordering between [a ASC, b ASC] and [a ASC, b DESC] - // result should be [a ASC]. - ( - vec![(col_a, option_asc), (col_b, option_asc)], - vec![(col_a, option_asc), (col_b, option_desc)], - Some(vec![(col_a, option_asc)]), - ), - ]; - for (lhs, rhs, expected) in tests_cases { - let lhs = convert_to_sort_exprs(&lhs); - let rhs = convert_to_sort_exprs(&rhs); - let expected = expected.map(|expected| convert_to_sort_exprs(&expected)); - let finer = eq_properties.get_meet_ordering(&lhs, &rhs); - assert_eq!(finer, expected) - } - - Ok(()) - } - - #[test] - fn test_find_longest_permutation() -> Result<()> { - // Schema satisfies following orderings: - // [a ASC], [d ASC, b ASC], [e DESC, f ASC, g ASC] - // and - // Column [a=c] (e.g they are aliases). - // At below we add [d ASC, h DESC] also, for test purposes - let (test_schema, mut eq_properties) = create_test_params()?; - let col_a = &col("a", &test_schema)?; - let col_b = &col("b", &test_schema)?; - let col_c = &col("c", &test_schema)?; - let col_d = &col("d", &test_schema)?; - let col_e = &col("e", &test_schema)?; - let col_h = &col("h", &test_schema)?; - // a + d - let a_plus_d = Arc::new(BinaryExpr::new( - col_a.clone(), - Operator::Plus, - col_d.clone(), - )) as Arc; - - let option_asc = SortOptions { - descending: false, - nulls_first: false, - }; - let option_desc = SortOptions { - descending: true, - nulls_first: true, - }; - // [d ASC, h ASC] also satisfies schema. - eq_properties.add_new_orderings([vec![ - PhysicalSortExpr { - expr: col_d.clone(), - options: option_asc, - }, - PhysicalSortExpr { - expr: col_h.clone(), - options: option_desc, - }, - ]]); - let test_cases = vec![ - // TEST CASE 1 - (vec![col_a], vec![(col_a, option_asc)]), - // TEST CASE 2 - (vec![col_c], vec![(col_c, option_asc)]), - // TEST CASE 3 - ( - vec![col_d, col_e, col_b], - vec![ - (col_d, option_asc), - (col_e, option_desc), - (col_b, option_asc), - ], - ), - // TEST CASE 4 - (vec![col_b], vec![]), - // TEST CASE 5 - (vec![col_d], vec![(col_d, option_asc)]), - // TEST CASE 5 - (vec![&a_plus_d], vec![(&a_plus_d, option_asc)]), - // TEST CASE 6 - ( - vec![col_b, col_d], - vec![(col_d, option_asc), (col_b, option_asc)], - ), - // TEST CASE 6 - ( - vec![col_c, col_e], - vec![(col_c, option_asc), (col_e, option_desc)], - ), - ]; - for (exprs, expected) in test_cases { - let exprs = exprs.into_iter().cloned().collect::>(); - let expected = convert_to_sort_exprs(&expected); - let (actual, _) = eq_properties.find_longest_permutation(&exprs); - assert_eq!(actual, expected); - } - - Ok(()) - } - - #[test] - fn test_find_longest_permutation_random() -> Result<()> { - const N_RANDOM_SCHEMA: usize = 100; - const N_ELEMENTS: usize = 125; - const N_DISTINCT: usize = 5; - - for seed in 0..N_RANDOM_SCHEMA { - // Create a random schema with random properties - let (test_schema, eq_properties) = create_random_schema(seed as u64)?; - // Generate a data that satisfies properties given - let table_data_with_properties = - generate_table_for_eq_properties(&eq_properties, N_ELEMENTS, N_DISTINCT)?; - - let floor_a = create_physical_expr( - &BuiltinScalarFunction::Floor, - &[col("a", &test_schema)?], - &test_schema, - &ExecutionProps::default(), - )?; - let a_plus_b = Arc::new(BinaryExpr::new( - col("a", &test_schema)?, - Operator::Plus, - col("b", &test_schema)?, - )) as Arc; - let exprs = vec![ - col("a", &test_schema)?, - col("b", &test_schema)?, - col("c", &test_schema)?, - col("d", &test_schema)?, - col("e", &test_schema)?, - col("f", &test_schema)?, - floor_a, - a_plus_b, - ]; - - for n_req in 0..=exprs.len() { - for exprs in exprs.iter().combinations(n_req) { - let exprs = exprs.into_iter().cloned().collect::>(); - let (ordering, indices) = - eq_properties.find_longest_permutation(&exprs); - // Make sure that find_longest_permutation return values are consistent - let ordering2 = indices - .iter() - .zip(ordering.iter()) - .map(|(&idx, sort_expr)| PhysicalSortExpr { - expr: exprs[idx].clone(), - options: sort_expr.options, - }) - .collect::>(); - assert_eq!( - ordering, ordering2, - "indices and lexicographical ordering do not match" - ); - - let err_msg = format!( - "Error in test case ordering:{:?}, eq_properties.oeq_class: {:?}, eq_properties.eq_group: {:?}, eq_properties.constants: {:?}", - ordering, eq_properties.oeq_class, eq_properties.eq_group, eq_properties.constants - ); - assert_eq!(ordering.len(), indices.len(), "{}", err_msg); - // Since ordered section satisfies schema, we expect - // that result will be same after sort (e.g sort was unnecessary). - assert!( - is_table_same_after_sort( - ordering.clone(), - table_data_with_properties.clone(), - )?, - "{}", - err_msg - ); - } - } - } - - Ok(()) - } - - #[test] - fn test_update_ordering() -> Result<()> { - let schema = Schema::new(vec![ - Field::new("a", DataType::Int32, true), - Field::new("b", DataType::Int32, true), - Field::new("c", DataType::Int32, true), - Field::new("d", DataType::Int32, true), - ]); - - let mut eq_properties = EquivalenceProperties::new(Arc::new(schema.clone())); - let col_a = &col("a", &schema)?; - let col_b = &col("b", &schema)?; - let col_c = &col("c", &schema)?; - let col_d = &col("d", &schema)?; - let option_asc = SortOptions { - descending: false, - nulls_first: false, - }; - // b=a (e.g they are aliases) - eq_properties.add_equal_conditions(col_b, col_a); - // [b ASC], [d ASC] - eq_properties.add_new_orderings(vec![ - vec![PhysicalSortExpr { - expr: col_b.clone(), - options: option_asc, - }], - vec![PhysicalSortExpr { - expr: col_d.clone(), - options: option_asc, - }], - ]); - - let test_cases = vec![ - // d + b - ( - Arc::new(BinaryExpr::new( - col_d.clone(), - Operator::Plus, - col_b.clone(), - )) as Arc, - SortProperties::Ordered(option_asc), - ), - // b - (col_b.clone(), SortProperties::Ordered(option_asc)), - // a - (col_a.clone(), SortProperties::Ordered(option_asc)), - // a + c - ( - Arc::new(BinaryExpr::new( - col_a.clone(), - Operator::Plus, - col_c.clone(), - )), - SortProperties::Unordered, - ), - ]; - for (expr, expected) in test_cases { - let leading_orderings = eq_properties - .oeq_class() - .iter() - .flat_map(|ordering| ordering.first().cloned()) - .collect::>(); - let expr_ordering = eq_properties.get_expr_ordering(expr.clone()); - let err_msg = format!( - "expr:{:?}, expected: {:?}, actual: {:?}, leading_orderings: {leading_orderings:?}", - expr, expected, expr_ordering.state - ); - assert_eq!(expr_ordering.state, expected, "{}", err_msg); - } - - Ok(()) - } - - #[test] - fn test_contains_any() { - let lit_true = Arc::new(Literal::new(ScalarValue::Boolean(Some(true)))) - as Arc; - let lit_false = Arc::new(Literal::new(ScalarValue::Boolean(Some(false)))) - as Arc; - let lit2 = - Arc::new(Literal::new(ScalarValue::Int32(Some(2)))) as Arc; - let lit1 = - Arc::new(Literal::new(ScalarValue::Int32(Some(1)))) as Arc; - let col_b_expr = Arc::new(Column::new("b", 1)) as Arc; - - let cls1 = EquivalenceClass::new(vec![lit_true.clone(), lit_false.clone()]); - let cls2 = EquivalenceClass::new(vec![lit_true.clone(), col_b_expr.clone()]); - let cls3 = EquivalenceClass::new(vec![lit2.clone(), lit1.clone()]); - - // lit_true is common - assert!(cls1.contains_any(&cls2)); - // there is no common entry - assert!(!cls1.contains_any(&cls3)); - assert!(!cls2.contains_any(&cls3)); - } - - #[test] - fn test_get_indices_of_matching_sort_exprs_with_order_eq() -> Result<()> { - let sort_options = SortOptions::default(); - let sort_options_not = SortOptions::default().not(); - - let schema = Schema::new(vec![ - Field::new("a", DataType::Int32, true), - Field::new("b", DataType::Int32, true), - ]); - let col_a = &col("a", &schema)?; - let col_b = &col("b", &schema)?; - let required_columns = [col_b.clone(), col_a.clone()]; - let mut eq_properties = EquivalenceProperties::new(Arc::new(schema)); - eq_properties.add_new_orderings([vec![ - PhysicalSortExpr { - expr: Arc::new(Column::new("b", 1)), - options: sort_options_not, - }, - PhysicalSortExpr { - expr: Arc::new(Column::new("a", 0)), - options: sort_options, - }, - ]]); - let (result, idxs) = eq_properties.find_longest_permutation(&required_columns); - assert_eq!(idxs, vec![0, 1]); - assert_eq!( - result, - vec![ - PhysicalSortExpr { - expr: col_b.clone(), - options: sort_options_not - }, - PhysicalSortExpr { - expr: col_a.clone(), - options: sort_options - } - ] - ); - - let schema = Schema::new(vec![ - Field::new("a", DataType::Int32, true), - Field::new("b", DataType::Int32, true), - Field::new("c", DataType::Int32, true), - ]); - let col_a = &col("a", &schema)?; - let col_b = &col("b", &schema)?; - let required_columns = [col_b.clone(), col_a.clone()]; - let mut eq_properties = EquivalenceProperties::new(Arc::new(schema)); - eq_properties.add_new_orderings([ - vec![PhysicalSortExpr { - expr: Arc::new(Column::new("c", 2)), - options: sort_options, - }], - vec![ - PhysicalSortExpr { - expr: Arc::new(Column::new("b", 1)), - options: sort_options_not, - }, - PhysicalSortExpr { - expr: Arc::new(Column::new("a", 0)), - options: sort_options, - }, - ], - ]); - let (result, idxs) = eq_properties.find_longest_permutation(&required_columns); - assert_eq!(idxs, vec![0, 1]); - assert_eq!( - result, - vec![ - PhysicalSortExpr { - expr: col_b.clone(), - options: sort_options_not - }, - PhysicalSortExpr { - expr: col_a.clone(), - options: sort_options - } - ] - ); - - let required_columns = [ - Arc::new(Column::new("b", 1)) as _, - Arc::new(Column::new("a", 0)) as _, - ]; - let schema = Schema::new(vec![ - Field::new("a", DataType::Int32, true), - Field::new("b", DataType::Int32, true), - Field::new("c", DataType::Int32, true), - ]); - let mut eq_properties = EquivalenceProperties::new(Arc::new(schema)); - - // not satisfied orders - eq_properties.add_new_orderings([vec![ - PhysicalSortExpr { - expr: Arc::new(Column::new("b", 1)), - options: sort_options_not, - }, - PhysicalSortExpr { - expr: Arc::new(Column::new("c", 2)), - options: sort_options, - }, - PhysicalSortExpr { - expr: Arc::new(Column::new("a", 0)), - options: sort_options, - }, - ]]); - let (_, idxs) = eq_properties.find_longest_permutation(&required_columns); - assert_eq!(idxs, vec![0]); - - Ok(()) - } - - #[test] - fn test_normalize_ordering_equivalence_classes() -> Result<()> { - let sort_options = SortOptions::default(); - - let schema = Schema::new(vec![ - Field::new("a", DataType::Int32, true), - Field::new("b", DataType::Int32, true), - Field::new("c", DataType::Int32, true), - ]); - let col_a_expr = col("a", &schema)?; - let col_b_expr = col("b", &schema)?; - let col_c_expr = col("c", &schema)?; - let mut eq_properties = EquivalenceProperties::new(Arc::new(schema.clone())); - - eq_properties.add_equal_conditions(&col_a_expr, &col_c_expr); - let others = vec![ - vec![PhysicalSortExpr { - expr: col_b_expr.clone(), - options: sort_options, - }], - vec![PhysicalSortExpr { - expr: col_c_expr.clone(), - options: sort_options, - }], - ]; - eq_properties.add_new_orderings(others); - - let mut expected_eqs = EquivalenceProperties::new(Arc::new(schema)); - expected_eqs.add_new_orderings([ - vec![PhysicalSortExpr { - expr: col_b_expr.clone(), - options: sort_options, - }], - vec![PhysicalSortExpr { - expr: col_c_expr.clone(), - options: sort_options, - }], - ]); - - let oeq_class = eq_properties.oeq_class().clone(); - let expected = expected_eqs.oeq_class(); - assert!(oeq_class.eq(expected)); - - Ok(()) - } - - #[test] - fn project_orderings() -> Result<()> { - let schema = Arc::new(Schema::new(vec![ - Field::new("a", DataType::Int32, true), - Field::new("b", DataType::Int32, true), - Field::new("c", DataType::Int32, true), - Field::new("d", DataType::Int32, true), - Field::new("e", DataType::Int32, true), - Field::new("ts", DataType::Timestamp(TimeUnit::Nanosecond, None), true), - ])); - let col_a = &col("a", &schema)?; - let col_b = &col("b", &schema)?; - let col_c = &col("c", &schema)?; - let col_d = &col("d", &schema)?; - let col_e = &col("e", &schema)?; - let col_ts = &col("ts", &schema)?; - let interval = Arc::new(Literal::new(ScalarValue::IntervalDayTime(Some(2)))) - as Arc; - let date_bin_func = &create_physical_expr( - &BuiltinScalarFunction::DateBin, - &[interval, col_ts.clone()], - &schema, - &ExecutionProps::default(), - )?; - let a_plus_b = Arc::new(BinaryExpr::new( - col_a.clone(), - Operator::Plus, - col_b.clone(), - )) as Arc; - let b_plus_d = Arc::new(BinaryExpr::new( - col_b.clone(), - Operator::Plus, - col_d.clone(), - )) as Arc; - let b_plus_e = Arc::new(BinaryExpr::new( - col_b.clone(), - Operator::Plus, - col_e.clone(), - )) as Arc; - let c_plus_d = Arc::new(BinaryExpr::new( - col_c.clone(), - Operator::Plus, - col_d.clone(), - )) as Arc; - - let option_asc = SortOptions { - descending: false, - nulls_first: false, - }; - let option_desc = SortOptions { - descending: true, - nulls_first: true, - }; - - let test_cases = vec![ - // ---------- TEST CASE 1 ------------ - ( - // orderings - vec![ - // [b ASC] - vec![(col_b, option_asc)], - ], - // projection exprs - vec![(col_b, "b_new".to_string()), (col_a, "a_new".to_string())], - // expected - vec![ - // [b_new ASC] - vec![("b_new", option_asc)], - ], - ), - // ---------- TEST CASE 2 ------------ - ( - // orderings - vec![ - // empty ordering - ], - // projection exprs - vec![(col_c, "c_new".to_string()), (col_b, "b_new".to_string())], - // expected - vec![ - // no ordering at the output - ], - ), - // ---------- TEST CASE 3 ------------ - ( - // orderings - vec![ - // [ts ASC] - vec![(col_ts, option_asc)], - ], - // projection exprs - vec![ - (col_b, "b_new".to_string()), - (col_a, "a_new".to_string()), - (col_ts, "ts_new".to_string()), - (date_bin_func, "date_bin_res".to_string()), - ], - // expected - vec![ - // [date_bin_res ASC] - vec![("date_bin_res", option_asc)], - // [ts_new ASC] - vec![("ts_new", option_asc)], - ], - ), - // ---------- TEST CASE 4 ------------ - ( - // orderings - vec![ - // [a ASC, ts ASC] - vec![(col_a, option_asc), (col_ts, option_asc)], - // [b ASC, ts ASC] - vec![(col_b, option_asc), (col_ts, option_asc)], - ], - // projection exprs - vec![ - (col_b, "b_new".to_string()), - (col_a, "a_new".to_string()), - (col_ts, "ts_new".to_string()), - (date_bin_func, "date_bin_res".to_string()), - ], - // expected - vec![ - // [a_new ASC, ts_new ASC] - vec![("a_new", option_asc), ("ts_new", option_asc)], - // [a_new ASC, date_bin_res ASC] - vec![("a_new", option_asc), ("date_bin_res", option_asc)], - // [b_new ASC, ts_new ASC] - vec![("b_new", option_asc), ("ts_new", option_asc)], - // [b_new ASC, date_bin_res ASC] - vec![("b_new", option_asc), ("date_bin_res", option_asc)], - ], - ), - // ---------- TEST CASE 5 ------------ - ( - // orderings - vec![ - // [a + b ASC] - vec![(&a_plus_b, option_asc)], - ], - // projection exprs - vec![ - (col_b, "b_new".to_string()), - (col_a, "a_new".to_string()), - (&a_plus_b, "a+b".to_string()), - ], - // expected - vec![ - // [a + b ASC] - vec![("a+b", option_asc)], - ], - ), - // ---------- TEST CASE 6 ------------ - ( - // orderings - vec![ - // [a + b ASC, c ASC] - vec![(&a_plus_b, option_asc), (&col_c, option_asc)], - ], - // projection exprs - vec![ - (col_b, "b_new".to_string()), - (col_a, "a_new".to_string()), - (col_c, "c_new".to_string()), - (&a_plus_b, "a+b".to_string()), - ], - // expected - vec![ - // [a + b ASC, c_new ASC] - vec![("a+b", option_asc), ("c_new", option_asc)], - ], - ), - // ------- TEST CASE 7 ---------- - ( - vec![ - // [a ASC, b ASC, c ASC] - vec![(col_a, option_asc), (col_b, option_asc)], - // [a ASC, d ASC] - vec![(col_a, option_asc), (col_d, option_asc)], - ], - // b as b_new, a as a_new, d as d_new b+d - vec![ - (col_b, "b_new".to_string()), - (col_a, "a_new".to_string()), - (col_d, "d_new".to_string()), - (&b_plus_d, "b+d".to_string()), - ], - // expected - vec![ - // [a_new ASC, b_new ASC] - vec![("a_new", option_asc), ("b_new", option_asc)], - // [a_new ASC, d_new ASC] - vec![("a_new", option_asc), ("d_new", option_asc)], - // [a_new ASC, b+d ASC] - vec![("a_new", option_asc), ("b+d", option_asc)], - ], - ), - // ------- TEST CASE 8 ---------- - ( - // orderings - vec![ - // [b+d ASC] - vec![(&b_plus_d, option_asc)], - ], - // proj exprs - vec![ - (col_b, "b_new".to_string()), - (col_a, "a_new".to_string()), - (col_d, "d_new".to_string()), - (&b_plus_d, "b+d".to_string()), - ], - // expected - vec![ - // [b+d ASC] - vec![("b+d", option_asc)], - ], - ), - // ------- TEST CASE 9 ---------- - ( - // orderings - vec![ - // [a ASC, d ASC, b ASC] - vec![ - (col_a, option_asc), - (col_d, option_asc), - (col_b, option_asc), - ], - // [c ASC] - vec![(col_c, option_asc)], - ], - // proj exprs - vec![ - (col_b, "b_new".to_string()), - (col_a, "a_new".to_string()), - (col_d, "d_new".to_string()), - (col_c, "c_new".to_string()), - ], - // expected - vec![ - // [a_new ASC, d_new ASC, b_new ASC] - vec![ - ("a_new", option_asc), - ("d_new", option_asc), - ("b_new", option_asc), - ], - // [c_new ASC], - vec![("c_new", option_asc)], - ], - ), - // ------- TEST CASE 10 ---------- - ( - vec![ - // [a ASC, b ASC, c ASC] - vec![ - (col_a, option_asc), - (col_b, option_asc), - (col_c, option_asc), - ], - // [a ASC, d ASC] - vec![(col_a, option_asc), (col_d, option_asc)], - ], - // proj exprs - vec![ - (col_b, "b_new".to_string()), - (col_a, "a_new".to_string()), - (col_c, "c_new".to_string()), - (&c_plus_d, "c+d".to_string()), - ], - // expected - vec![ - // [a_new ASC, b_new ASC, c_new ASC] - vec![ - ("a_new", option_asc), - ("b_new", option_asc), - ("c_new", option_asc), - ], - // [a_new ASC, b_new ASC, c+d ASC] - vec![ - ("a_new", option_asc), - ("b_new", option_asc), - ("c+d", option_asc), - ], - ], - ), - // ------- TEST CASE 11 ---------- - ( - // orderings - vec![ - // [a ASC, b ASC] - vec![(col_a, option_asc), (col_b, option_asc)], - // [a ASC, d ASC] - vec![(col_a, option_asc), (col_d, option_asc)], - ], - // proj exprs - vec![ - (col_b, "b_new".to_string()), - (col_a, "a_new".to_string()), - (&b_plus_d, "b+d".to_string()), - ], - // expected - vec![ - // [a_new ASC, b_new ASC] - vec![("a_new", option_asc), ("b_new", option_asc)], - // [a_new ASC, b + d ASC] - vec![("a_new", option_asc), ("b+d", option_asc)], - ], - ), - // ------- TEST CASE 12 ---------- - ( - // orderings - vec![ - // [a ASC, b ASC, c ASC] - vec![ - (col_a, option_asc), - (col_b, option_asc), - (col_c, option_asc), - ], - ], - // proj exprs - vec![(col_c, "c_new".to_string()), (col_a, "a_new".to_string())], - // expected - vec![ - // [a_new ASC] - vec![("a_new", option_asc)], - ], - ), - // ------- TEST CASE 13 ---------- - ( - // orderings - vec![ - // [a ASC, b ASC, c ASC] - vec![ - (col_a, option_asc), - (col_b, option_asc), - (col_c, option_asc), - ], - // [a ASC, a + b ASC, c ASC] - vec![ - (col_a, option_asc), - (&a_plus_b, option_asc), - (col_c, option_asc), - ], - ], - // proj exprs - vec![ - (col_c, "c_new".to_string()), - (col_b, "b_new".to_string()), - (col_a, "a_new".to_string()), - (&a_plus_b, "a+b".to_string()), - ], - // expected - vec![ - // [a_new ASC, b_new ASC, c_new ASC] - vec![ - ("a_new", option_asc), - ("b_new", option_asc), - ("c_new", option_asc), - ], - // [a_new ASC, a+b ASC, c_new ASC] - vec![ - ("a_new", option_asc), - ("a+b", option_asc), - ("c_new", option_asc), - ], - ], - ), - // ------- TEST CASE 14 ---------- - ( - // orderings - vec![ - // [a ASC, b ASC] - vec![(col_a, option_asc), (col_b, option_asc)], - // [c ASC, b ASC] - vec![(col_c, option_asc), (col_b, option_asc)], - // [d ASC, e ASC] - vec![(col_d, option_asc), (col_e, option_asc)], - ], - // proj exprs - vec![ - (col_c, "c_new".to_string()), - (col_d, "d_new".to_string()), - (col_a, "a_new".to_string()), - (&b_plus_e, "b+e".to_string()), - ], - // expected - vec![ - // [a_new ASC, d_new ASC, b+e ASC] - vec![ - ("a_new", option_asc), - ("d_new", option_asc), - ("b+e", option_asc), - ], - // [d_new ASC, a_new ASC, b+e ASC] - vec![ - ("d_new", option_asc), - ("a_new", option_asc), - ("b+e", option_asc), - ], - // [c_new ASC, d_new ASC, b+e ASC] - vec![ - ("c_new", option_asc), - ("d_new", option_asc), - ("b+e", option_asc), - ], - // [d_new ASC, c_new ASC, b+e ASC] - vec![ - ("d_new", option_asc), - ("c_new", option_asc), - ("b+e", option_asc), - ], - ], - ), - // ------- TEST CASE 15 ---------- - ( - // orderings - vec![ - // [a ASC, c ASC, b ASC] - vec![ - (col_a, option_asc), - (col_c, option_asc), - (&col_b, option_asc), - ], - ], - // proj exprs - vec![ - (col_c, "c_new".to_string()), - (col_a, "a_new".to_string()), - (&a_plus_b, "a+b".to_string()), - ], - // expected - vec![ - // [a_new ASC, d_new ASC, b+e ASC] - vec![ - ("a_new", option_asc), - ("c_new", option_asc), - ("a+b", option_asc), - ], - ], - ), - // ------- TEST CASE 16 ---------- - ( - // orderings - vec![ - // [a ASC, b ASC] - vec![(col_a, option_asc), (col_b, option_asc)], - // [c ASC, b DESC] - vec![(col_c, option_asc), (col_b, option_desc)], - // [e ASC] - vec![(col_e, option_asc)], - ], - // proj exprs - vec![ - (col_c, "c_new".to_string()), - (col_a, "a_new".to_string()), - (col_b, "b_new".to_string()), - (&b_plus_e, "b+e".to_string()), - ], - // expected - vec![ - // [a_new ASC, b_new ASC] - vec![("a_new", option_asc), ("b_new", option_asc)], - // [a_new ASC, b_new ASC] - vec![("a_new", option_asc), ("b+e", option_asc)], - // [c_new ASC, b_new DESC] - vec![("c_new", option_asc), ("b_new", option_desc)], - ], - ), - ]; - - for (idx, (orderings, proj_exprs, expected)) in test_cases.into_iter().enumerate() - { - let mut eq_properties = EquivalenceProperties::new(schema.clone()); - - let orderings = convert_to_orderings(&orderings); - eq_properties.add_new_orderings(orderings); - - let proj_exprs = proj_exprs - .into_iter() - .map(|(expr, name)| (expr.clone(), name)) - .collect::>(); - let projection_mapping = ProjectionMapping::try_new(&proj_exprs, &schema)?; - let output_schema = output_schema(&projection_mapping, &schema)?; - - let expected = expected - .into_iter() - .map(|ordering| { - ordering - .into_iter() - .map(|(name, options)| { - (col(name, &output_schema).unwrap(), options) - }) - .collect::>() - }) - .collect::>(); - let expected = convert_to_orderings_owned(&expected); - - let projected_eq = eq_properties.project(&projection_mapping, output_schema); - let orderings = projected_eq.oeq_class(); - - let err_msg = format!( - "test_idx: {:?}, actual: {:?}, expected: {:?}, projection_mapping: {:?}", - idx, orderings.orderings, expected, projection_mapping - ); - - assert_eq!(orderings.len(), expected.len(), "{}", err_msg); - for expected_ordering in &expected { - assert!(orderings.contains(expected_ordering), "{}", err_msg) - } - } - - Ok(()) - } - - #[test] - fn project_orderings2() -> Result<()> { - let schema = Arc::new(Schema::new(vec![ - Field::new("a", DataType::Int32, true), - Field::new("b", DataType::Int32, true), - Field::new("c", DataType::Int32, true), - Field::new("d", DataType::Int32, true), - Field::new("ts", DataType::Timestamp(TimeUnit::Nanosecond, None), true), - ])); - let col_a = &col("a", &schema)?; - let col_b = &col("b", &schema)?; - let col_c = &col("c", &schema)?; - let col_ts = &col("ts", &schema)?; - let a_plus_b = Arc::new(BinaryExpr::new( - col_a.clone(), - Operator::Plus, - col_b.clone(), - )) as Arc; - let interval = Arc::new(Literal::new(ScalarValue::IntervalDayTime(Some(2)))) - as Arc; - let date_bin_ts = &create_physical_expr( - &BuiltinScalarFunction::DateBin, - &[interval, col_ts.clone()], - &schema, - &ExecutionProps::default(), - )?; - - let round_c = &create_physical_expr( - &BuiltinScalarFunction::Round, - &[col_c.clone()], - &schema, - &ExecutionProps::default(), - )?; - - let option_asc = SortOptions { - descending: false, - nulls_first: false, - }; - - let proj_exprs = vec![ - (col_b, "b_new".to_string()), - (col_a, "a_new".to_string()), - (col_c, "c_new".to_string()), - (date_bin_ts, "date_bin_res".to_string()), - (round_c, "round_c_res".to_string()), - ]; - let proj_exprs = proj_exprs - .into_iter() - .map(|(expr, name)| (expr.clone(), name)) - .collect::>(); - let projection_mapping = ProjectionMapping::try_new(&proj_exprs, &schema)?; - let output_schema = output_schema(&projection_mapping, &schema)?; - - let col_a_new = &col("a_new", &output_schema)?; - let col_b_new = &col("b_new", &output_schema)?; - let col_c_new = &col("c_new", &output_schema)?; - let col_date_bin_res = &col("date_bin_res", &output_schema)?; - let col_round_c_res = &col("round_c_res", &output_schema)?; - let a_new_plus_b_new = Arc::new(BinaryExpr::new( - col_a_new.clone(), - Operator::Plus, - col_b_new.clone(), - )) as Arc; - - let test_cases = vec![ - // ---------- TEST CASE 1 ------------ - ( - // orderings - vec![ - // [a ASC] - vec![(col_a, option_asc)], - ], - // expected - vec![ - // [b_new ASC] - vec![(col_a_new, option_asc)], - ], - ), - // ---------- TEST CASE 2 ------------ - ( - // orderings - vec![ - // [a+b ASC] - vec![(&a_plus_b, option_asc)], - ], - // expected - vec![ - // [b_new ASC] - vec![(&a_new_plus_b_new, option_asc)], - ], - ), - // ---------- TEST CASE 3 ------------ - ( - // orderings - vec![ - // [a ASC, ts ASC] - vec![(col_a, option_asc), (col_ts, option_asc)], - ], - // expected - vec![ - // [a_new ASC, date_bin_res ASC] - vec![(col_a_new, option_asc), (col_date_bin_res, option_asc)], - ], - ), - // ---------- TEST CASE 4 ------------ - ( - // orderings - vec![ - // [a ASC, ts ASC, b ASC] - vec![ - (col_a, option_asc), - (col_ts, option_asc), - (col_b, option_asc), - ], - ], - // expected - vec![ - // [a_new ASC, date_bin_res ASC] - // Please note that result is not [a_new ASC, date_bin_res ASC, b_new ASC] - // because, datebin_res may not be 1-1 function. Hence without introducing ts - // dependency we cannot guarantee any ordering after date_bin_res column. - vec![(col_a_new, option_asc), (col_date_bin_res, option_asc)], - ], - ), - // ---------- TEST CASE 5 ------------ - ( - // orderings - vec![ - // [a ASC, c ASC] - vec![(col_a, option_asc), (col_c, option_asc)], - ], - // expected - vec![ - // [a_new ASC, round_c_res ASC, c_new ASC] - vec![(col_a_new, option_asc), (col_round_c_res, option_asc)], - // [a_new ASC, c_new ASC] - vec![(col_a_new, option_asc), (col_c_new, option_asc)], - ], - ), - // ---------- TEST CASE 6 ------------ - ( - // orderings - vec![ - // [c ASC, b ASC] - vec![(col_c, option_asc), (col_b, option_asc)], - ], - // expected - vec![ - // [round_c_res ASC] - vec![(col_round_c_res, option_asc)], - // [c_new ASC, b_new ASC] - vec![(col_c_new, option_asc), (col_b_new, option_asc)], - ], - ), - // ---------- TEST CASE 7 ------------ - ( - // orderings - vec![ - // [a+b ASC, c ASC] - vec![(&a_plus_b, option_asc), (col_c, option_asc)], - ], - // expected - vec![ - // [a+b ASC, round(c) ASC, c_new ASC] - vec![ - (&a_new_plus_b_new, option_asc), - (&col_round_c_res, option_asc), - ], - // [a+b ASC, c_new ASC] - vec![(&a_new_plus_b_new, option_asc), (col_c_new, option_asc)], - ], - ), - ]; - - for (idx, (orderings, expected)) in test_cases.iter().enumerate() { - let mut eq_properties = EquivalenceProperties::new(schema.clone()); - - let orderings = convert_to_orderings(orderings); - eq_properties.add_new_orderings(orderings); - - let expected = convert_to_orderings(expected); - - let projected_eq = - eq_properties.project(&projection_mapping, output_schema.clone()); - let orderings = projected_eq.oeq_class(); - - let err_msg = format!( - "test idx: {:?}, actual: {:?}, expected: {:?}, projection_mapping: {:?}", - idx, orderings.orderings, expected, projection_mapping - ); - - assert_eq!(orderings.len(), expected.len(), "{}", err_msg); - for expected_ordering in &expected { - assert!(orderings.contains(expected_ordering), "{}", err_msg) - } - } - Ok(()) - } - - #[test] - fn project_orderings3() -> Result<()> { - let schema = Arc::new(Schema::new(vec![ - Field::new("a", DataType::Int32, true), - Field::new("b", DataType::Int32, true), - Field::new("c", DataType::Int32, true), - Field::new("d", DataType::Int32, true), - Field::new("e", DataType::Int32, true), - Field::new("f", DataType::Int32, true), - ])); - let col_a = &col("a", &schema)?; - let col_b = &col("b", &schema)?; - let col_c = &col("c", &schema)?; - let col_d = &col("d", &schema)?; - let col_e = &col("e", &schema)?; - let col_f = &col("f", &schema)?; - let a_plus_b = Arc::new(BinaryExpr::new( - col_a.clone(), - Operator::Plus, - col_b.clone(), - )) as Arc; - - let option_asc = SortOptions { - descending: false, - nulls_first: false, - }; - - let proj_exprs = vec![ - (col_c, "c_new".to_string()), - (col_d, "d_new".to_string()), - (&a_plus_b, "a+b".to_string()), - ]; - let proj_exprs = proj_exprs - .into_iter() - .map(|(expr, name)| (expr.clone(), name)) - .collect::>(); - let projection_mapping = ProjectionMapping::try_new(&proj_exprs, &schema)?; - let output_schema = output_schema(&projection_mapping, &schema)?; - - let col_a_plus_b_new = &col("a+b", &output_schema)?; - let col_c_new = &col("c_new", &output_schema)?; - let col_d_new = &col("d_new", &output_schema)?; - - let test_cases = vec![ - // ---------- TEST CASE 1 ------------ - ( - // orderings - vec![ - // [d ASC, b ASC] - vec![(col_d, option_asc), (col_b, option_asc)], - // [c ASC, a ASC] - vec![(col_c, option_asc), (col_a, option_asc)], - ], - // equal conditions - vec![], - // expected - vec![ - // [d_new ASC, c_new ASC, a+b ASC] - vec![ - (col_d_new, option_asc), - (col_c_new, option_asc), - (col_a_plus_b_new, option_asc), - ], - // [c_new ASC, d_new ASC, a+b ASC] - vec![ - (col_c_new, option_asc), - (col_d_new, option_asc), - (col_a_plus_b_new, option_asc), - ], - ], - ), - // ---------- TEST CASE 2 ------------ - ( - // orderings - vec![ - // [d ASC, b ASC] - vec![(col_d, option_asc), (col_b, option_asc)], - // [c ASC, e ASC], Please note that a=e - vec![(col_c, option_asc), (col_e, option_asc)], - ], - // equal conditions - vec![(col_e, col_a)], - // expected - vec![ - // [d_new ASC, c_new ASC, a+b ASC] - vec![ - (col_d_new, option_asc), - (col_c_new, option_asc), - (col_a_plus_b_new, option_asc), - ], - // [c_new ASC, d_new ASC, a+b ASC] - vec![ - (col_c_new, option_asc), - (col_d_new, option_asc), - (col_a_plus_b_new, option_asc), - ], - ], - ), - // ---------- TEST CASE 3 ------------ - ( - // orderings - vec![ - // [d ASC, b ASC] - vec![(col_d, option_asc), (col_b, option_asc)], - // [c ASC, e ASC], Please note that a=f - vec![(col_c, option_asc), (col_e, option_asc)], - ], - // equal conditions - vec![(col_a, col_f)], - // expected - vec![ - // [d_new ASC] - vec![(col_d_new, option_asc)], - // [c_new ASC] - vec![(col_c_new, option_asc)], - ], - ), - ]; - for (orderings, equal_columns, expected) in test_cases { - let mut eq_properties = EquivalenceProperties::new(schema.clone()); - for (lhs, rhs) in equal_columns { - eq_properties.add_equal_conditions(lhs, rhs); - } - - let orderings = convert_to_orderings(&orderings); - eq_properties.add_new_orderings(orderings); - - let expected = convert_to_orderings(&expected); - - let projected_eq = - eq_properties.project(&projection_mapping, output_schema.clone()); - let orderings = projected_eq.oeq_class(); - - let err_msg = format!( - "actual: {:?}, expected: {:?}, projection_mapping: {:?}", - orderings.orderings, expected, projection_mapping - ); - - assert_eq!(orderings.len(), expected.len(), "{}", err_msg); - for expected_ordering in &expected { - assert!(orderings.contains(expected_ordering), "{}", err_msg) - } - } - - Ok(()) - } - - #[test] - fn project_orderings_random() -> Result<()> { - const N_RANDOM_SCHEMA: usize = 20; - const N_ELEMENTS: usize = 125; - const N_DISTINCT: usize = 5; - - for seed in 0..N_RANDOM_SCHEMA { - // Create a random schema with random properties - let (test_schema, eq_properties) = create_random_schema(seed as u64)?; - // Generate a data that satisfies properties given - let table_data_with_properties = - generate_table_for_eq_properties(&eq_properties, N_ELEMENTS, N_DISTINCT)?; - // Floor(a) - let floor_a = create_physical_expr( - &BuiltinScalarFunction::Floor, - &[col("a", &test_schema)?], - &test_schema, - &ExecutionProps::default(), - )?; - // a + b - let a_plus_b = Arc::new(BinaryExpr::new( - col("a", &test_schema)?, - Operator::Plus, - col("b", &test_schema)?, - )) as Arc; - let proj_exprs = vec![ - (col("a", &test_schema)?, "a_new"), - (col("b", &test_schema)?, "b_new"), - (col("c", &test_schema)?, "c_new"), - (col("d", &test_schema)?, "d_new"), - (col("e", &test_schema)?, "e_new"), - (col("f", &test_schema)?, "f_new"), - (floor_a, "floor(a)"), - (a_plus_b, "a+b"), - ]; - - for n_req in 0..=proj_exprs.len() { - for proj_exprs in proj_exprs.iter().combinations(n_req) { - let proj_exprs = proj_exprs - .into_iter() - .map(|(expr, name)| (expr.clone(), name.to_string())) - .collect::>(); - let (projected_batch, projected_eq) = apply_projection( - proj_exprs.clone(), - &table_data_with_properties, - &eq_properties, - )?; - - // Make sure each ordering after projection is valid. - for ordering in projected_eq.oeq_class().iter() { - let err_msg = format!( - "Error in test case ordering:{:?}, eq_properties.oeq_class: {:?}, eq_properties.eq_group: {:?}, eq_properties.constants: {:?}, proj_exprs: {:?}", - ordering, eq_properties.oeq_class, eq_properties.eq_group, eq_properties.constants, proj_exprs - ); - // Since ordered section satisfies schema, we expect - // that result will be same after sort (e.g sort was unnecessary). - assert!( - is_table_same_after_sort( - ordering.clone(), - projected_batch.clone(), - )?, - "{}", - err_msg - ); - } - } - } - } - - Ok(()) - } - - #[test] - fn ordering_satisfy_after_projection_random() -> Result<()> { - const N_RANDOM_SCHEMA: usize = 20; - const N_ELEMENTS: usize = 125; - const N_DISTINCT: usize = 5; - const SORT_OPTIONS: SortOptions = SortOptions { - descending: false, - nulls_first: false, - }; - - for seed in 0..N_RANDOM_SCHEMA { - // Create a random schema with random properties - let (test_schema, eq_properties) = create_random_schema(seed as u64)?; - // Generate a data that satisfies properties given - let table_data_with_properties = - generate_table_for_eq_properties(&eq_properties, N_ELEMENTS, N_DISTINCT)?; - // Floor(a) - let floor_a = create_physical_expr( - &BuiltinScalarFunction::Floor, - &[col("a", &test_schema)?], - &test_schema, - &ExecutionProps::default(), - )?; - // a + b - let a_plus_b = Arc::new(BinaryExpr::new( - col("a", &test_schema)?, - Operator::Plus, - col("b", &test_schema)?, - )) as Arc; - let proj_exprs = vec![ - (col("a", &test_schema)?, "a_new"), - (col("b", &test_schema)?, "b_new"), - (col("c", &test_schema)?, "c_new"), - (col("d", &test_schema)?, "d_new"), - (col("e", &test_schema)?, "e_new"), - (col("f", &test_schema)?, "f_new"), - (floor_a, "floor(a)"), - (a_plus_b, "a+b"), - ]; - - for n_req in 0..=proj_exprs.len() { - for proj_exprs in proj_exprs.iter().combinations(n_req) { - let proj_exprs = proj_exprs - .into_iter() - .map(|(expr, name)| (expr.clone(), name.to_string())) - .collect::>(); - let (projected_batch, projected_eq) = apply_projection( - proj_exprs.clone(), - &table_data_with_properties, - &eq_properties, - )?; - - let projection_mapping = - ProjectionMapping::try_new(&proj_exprs, &test_schema)?; - - let projected_exprs = projection_mapping - .iter() - .map(|(_source, target)| target.clone()) - .collect::>(); - - for n_req in 0..=projected_exprs.len() { - for exprs in projected_exprs.iter().combinations(n_req) { - let requirement = exprs - .into_iter() - .map(|expr| PhysicalSortExpr { - expr: expr.clone(), - options: SORT_OPTIONS, - }) - .collect::>(); - let expected = is_table_same_after_sort( - requirement.clone(), - projected_batch.clone(), - )?; - let err_msg = format!( - "Error in test case requirement:{:?}, expected: {:?}, eq_properties.oeq_class: {:?}, eq_properties.eq_group: {:?}, eq_properties.constants: {:?}, projected_eq.oeq_class: {:?}, projected_eq.eq_group: {:?}, projected_eq.constants: {:?}, projection_mapping: {:?}", - requirement, expected, eq_properties.oeq_class, eq_properties.eq_group, eq_properties.constants, projected_eq.oeq_class, projected_eq.eq_group, projected_eq.constants, projection_mapping - ); - // Check whether ordering_satisfy API result and - // experimental result matches. - assert_eq!( - projected_eq.ordering_satisfy(&requirement), - expected, - "{}", - err_msg - ); - } - } - } - } - } - - Ok(()) - } - - #[test] - fn test_expr_consists_of_constants() -> Result<()> { - let schema = Arc::new(Schema::new(vec![ - Field::new("a", DataType::Int32, true), - Field::new("b", DataType::Int32, true), - Field::new("c", DataType::Int32, true), - Field::new("d", DataType::Int32, true), - Field::new("ts", DataType::Timestamp(TimeUnit::Nanosecond, None), true), - ])); - let col_a = col("a", &schema)?; - let col_b = col("b", &schema)?; - let col_d = col("d", &schema)?; - let b_plus_d = Arc::new(BinaryExpr::new( - col_b.clone(), - Operator::Plus, - col_d.clone(), - )) as Arc; - - let constants = vec![col_a.clone(), col_b.clone()]; - let expr = b_plus_d.clone(); - assert!(!is_constant_recurse(&constants, &expr)); - - let constants = vec![col_a.clone(), col_b.clone(), col_d.clone()]; - let expr = b_plus_d.clone(); - assert!(is_constant_recurse(&constants, &expr)); - Ok(()) - } - - #[test] - fn test_join_equivalence_properties() -> Result<()> { - let schema = create_test_schema()?; - let col_a = &col("a", &schema)?; - let col_b = &col("b", &schema)?; - let col_c = &col("c", &schema)?; - let offset = schema.fields.len(); - let col_a2 = &add_offset_to_expr(col_a.clone(), offset); - let col_b2 = &add_offset_to_expr(col_b.clone(), offset); - let option_asc = SortOptions { - descending: false, - nulls_first: false, - }; - let test_cases = vec![ - // ------- TEST CASE 1 -------- - // [a ASC], [b ASC] - ( - // [a ASC], [b ASC] - vec![vec![(col_a, option_asc)], vec![(col_b, option_asc)]], - // [a ASC], [b ASC] - vec![vec![(col_a, option_asc)], vec![(col_b, option_asc)]], - // expected [a ASC, a2 ASC], [a ASC, b2 ASC], [b ASC, a2 ASC], [b ASC, b2 ASC] - vec![ - vec![(col_a, option_asc), (col_a2, option_asc)], - vec![(col_a, option_asc), (col_b2, option_asc)], - vec![(col_b, option_asc), (col_a2, option_asc)], - vec![(col_b, option_asc), (col_b2, option_asc)], - ], - ), - // ------- TEST CASE 2 -------- - // [a ASC], [b ASC] - ( - // [a ASC], [b ASC], [c ASC] - vec![ - vec![(col_a, option_asc)], - vec![(col_b, option_asc)], - vec![(col_c, option_asc)], - ], - // [a ASC], [b ASC] - vec![vec![(col_a, option_asc)], vec![(col_b, option_asc)]], - // expected [a ASC, a2 ASC], [a ASC, b2 ASC], [b ASC, a2 ASC], [b ASC, b2 ASC], [c ASC, a2 ASC], [c ASC, b2 ASC] - vec![ - vec![(col_a, option_asc), (col_a2, option_asc)], - vec![(col_a, option_asc), (col_b2, option_asc)], - vec![(col_b, option_asc), (col_a2, option_asc)], - vec![(col_b, option_asc), (col_b2, option_asc)], - vec![(col_c, option_asc), (col_a2, option_asc)], - vec![(col_c, option_asc), (col_b2, option_asc)], - ], - ), - ]; - for (left_orderings, right_orderings, expected) in test_cases { - let mut left_eq_properties = EquivalenceProperties::new(schema.clone()); - let mut right_eq_properties = EquivalenceProperties::new(schema.clone()); - let left_orderings = convert_to_orderings(&left_orderings); - let right_orderings = convert_to_orderings(&right_orderings); - let expected = convert_to_orderings(&expected); - left_eq_properties.add_new_orderings(left_orderings); - right_eq_properties.add_new_orderings(right_orderings); - let join_eq = join_equivalence_properties( - left_eq_properties, - right_eq_properties, - &JoinType::Inner, - Arc::new(Schema::empty()), - &[true, false], - Some(JoinSide::Left), - &[], - ); - let orderings = &join_eq.oeq_class.orderings; - let err_msg = format!("expected: {:?}, actual:{:?}", expected, orderings); - assert_eq!( - join_eq.oeq_class.orderings.len(), - expected.len(), - "{}", - err_msg - ); - for ordering in orderings { - assert!( - expected.contains(ordering), - "{}, ordering: {:?}", - err_msg, - ordering - ); - } - } - Ok(()) - } -} diff --git a/datafusion/physical-expr/src/equivalence/class.rs b/datafusion/physical-expr/src/equivalence/class.rs new file mode 100644 index 000000000000..f0bd1740d5d2 --- /dev/null +++ b/datafusion/physical-expr/src/equivalence/class.rs @@ -0,0 +1,598 @@ +// 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 super::{add_offset_to_expr, collapse_lex_req, ProjectionMapping}; +use crate::{ + expressions::Column, physical_expr::deduplicate_physical_exprs, + physical_exprs_bag_equal, physical_exprs_contains, LexOrdering, LexOrderingRef, + LexRequirement, LexRequirementRef, PhysicalExpr, PhysicalSortExpr, + PhysicalSortRequirement, +}; +use datafusion_common::tree_node::TreeNode; +use datafusion_common::{tree_node::Transformed, JoinType}; +use std::sync::Arc; + +/// An `EquivalenceClass` is a set of [`Arc`]s that are known +/// to have the same value for all tuples in a relation. These are generated by +/// equality predicates (e.g. `a = b`), typically equi-join conditions and +/// equality conditions in filters. +/// +/// Two `EquivalenceClass`es are equal if they contains the same expressions in +/// without any ordering. +#[derive(Debug, Clone)] +pub struct EquivalenceClass { + /// The expressions in this equivalence class. The order doesn't + /// matter for equivalence purposes + /// + /// TODO: use a HashSet for this instead of a Vec + exprs: Vec>, +} + +impl PartialEq for EquivalenceClass { + /// Returns true if other is equal in the sense + /// of bags (multi-sets), disregarding their orderings. + fn eq(&self, other: &Self) -> bool { + physical_exprs_bag_equal(&self.exprs, &other.exprs) + } +} + +impl EquivalenceClass { + /// Create a new empty equivalence class + pub fn new_empty() -> Self { + Self { exprs: vec![] } + } + + // Create a new equivalence class from a pre-existing `Vec` + pub fn new(mut exprs: Vec>) -> Self { + deduplicate_physical_exprs(&mut exprs); + Self { exprs } + } + + /// Return the inner vector of expressions + pub fn into_vec(self) -> Vec> { + self.exprs + } + + /// Return the "canonical" expression for this class (the first element) + /// if any + fn canonical_expr(&self) -> Option> { + self.exprs.first().cloned() + } + + /// Insert the expression into this class, meaning it is known to be equal to + /// all other expressions in this class + pub fn push(&mut self, expr: Arc) { + if !self.contains(&expr) { + self.exprs.push(expr); + } + } + + /// Inserts all the expressions from other into this class + pub fn extend(&mut self, other: Self) { + for expr in other.exprs { + // use push so entries are deduplicated + self.push(expr); + } + } + + /// Returns true if this equivalence class contains t expression + pub fn contains(&self, expr: &Arc) -> bool { + physical_exprs_contains(&self.exprs, expr) + } + + /// Returns true if this equivalence class has any entries in common with `other` + pub fn contains_any(&self, other: &Self) -> bool { + self.exprs.iter().any(|e| other.contains(e)) + } + + /// return the number of items in this class + pub fn len(&self) -> usize { + self.exprs.len() + } + + /// return true if this class is empty + pub fn is_empty(&self) -> bool { + self.exprs.is_empty() + } + + /// Iterate over all elements in this class, in some arbitrary order + pub fn iter(&self) -> impl Iterator> { + self.exprs.iter() + } + + /// Return a new equivalence class that have the specified offset added to + /// each expression (used when schemas are appended such as in joins) + pub fn with_offset(&self, offset: usize) -> Self { + let new_exprs = self + .exprs + .iter() + .cloned() + .map(|e| add_offset_to_expr(e, offset)) + .collect(); + Self::new(new_exprs) + } +} + +/// An `EquivalenceGroup` is a collection of `EquivalenceClass`es where each +/// class represents a distinct equivalence class in a relation. +#[derive(Debug, Clone)] +pub struct EquivalenceGroup { + pub classes: Vec, +} + +impl EquivalenceGroup { + /// Creates an empty equivalence group. + pub fn empty() -> Self { + Self { classes: vec![] } + } + + /// Creates an equivalence group from the given equivalence classes. + pub fn new(classes: Vec) -> Self { + let mut result = Self { classes }; + result.remove_redundant_entries(); + result + } + + /// Returns how many equivalence classes there are in this group. + pub fn len(&self) -> usize { + self.classes.len() + } + + /// Checks whether this equivalence group is empty. + pub fn is_empty(&self) -> bool { + self.len() == 0 + } + + /// Returns an iterator over the equivalence classes in this group. + pub fn iter(&self) -> impl Iterator { + self.classes.iter() + } + + /// Adds the equality `left` = `right` to this equivalence group. + /// New equality conditions often arise after steps like `Filter(a = b)`, + /// `Alias(a, a as b)` etc. + pub fn add_equal_conditions( + &mut self, + left: &Arc, + right: &Arc, + ) { + let mut first_class = None; + let mut second_class = None; + for (idx, cls) in self.classes.iter().enumerate() { + if cls.contains(left) { + first_class = Some(idx); + } + if cls.contains(right) { + second_class = Some(idx); + } + } + match (first_class, second_class) { + (Some(mut first_idx), Some(mut second_idx)) => { + // If the given left and right sides belong to different classes, + // we should unify/bridge these classes. + if first_idx != second_idx { + // By convention, make sure `second_idx` is larger than `first_idx`. + if first_idx > second_idx { + (first_idx, second_idx) = (second_idx, first_idx); + } + // Remove the class at `second_idx` and merge its values with + // the class at `first_idx`. The convention above makes sure + // that `first_idx` is still valid after removing `second_idx`. + let other_class = self.classes.swap_remove(second_idx); + self.classes[first_idx].extend(other_class); + } + } + (Some(group_idx), None) => { + // Right side is new, extend left side's class: + self.classes[group_idx].push(right.clone()); + } + (None, Some(group_idx)) => { + // Left side is new, extend right side's class: + self.classes[group_idx].push(left.clone()); + } + (None, None) => { + // None of the expressions is among existing classes. + // Create a new equivalence class and extend the group. + self.classes + .push(EquivalenceClass::new(vec![left.clone(), right.clone()])); + } + } + } + + /// Removes redundant entries from this group. + fn remove_redundant_entries(&mut self) { + // Remove duplicate entries from each equivalence class: + self.classes.retain_mut(|cls| { + // Keep groups that have at least two entries as singleton class is + // meaningless (i.e. it contains no non-trivial information): + cls.len() > 1 + }); + // Unify/bridge groups that have common expressions: + self.bridge_classes() + } + + /// This utility function unifies/bridges classes that have common expressions. + /// For example, assume that we have [`EquivalenceClass`]es `[a, b]` and `[b, c]`. + /// Since both classes contain `b`, columns `a`, `b` and `c` are actually all + /// equal and belong to one class. This utility converts merges such classes. + fn bridge_classes(&mut self) { + let mut idx = 0; + while idx < self.classes.len() { + let mut next_idx = idx + 1; + let start_size = self.classes[idx].len(); + while next_idx < self.classes.len() { + if self.classes[idx].contains_any(&self.classes[next_idx]) { + let extension = self.classes.swap_remove(next_idx); + self.classes[idx].extend(extension); + } else { + next_idx += 1; + } + } + if self.classes[idx].len() > start_size { + continue; + } + idx += 1; + } + } + + /// Extends this equivalence group with the `other` equivalence group. + pub fn extend(&mut self, other: Self) { + self.classes.extend(other.classes); + self.remove_redundant_entries(); + } + + /// Normalizes the given physical expression according to this group. + /// The expression is replaced with the first expression in the equivalence + /// class it matches with (if any). + pub fn normalize_expr(&self, expr: Arc) -> Arc { + expr.clone() + .transform(&|expr| { + for cls in self.iter() { + if cls.contains(&expr) { + return Ok(Transformed::Yes(cls.canonical_expr().unwrap())); + } + } + Ok(Transformed::No(expr)) + }) + .unwrap_or(expr) + } + + /// Normalizes the given sort expression according to this group. + /// The underlying physical expression is replaced with the first expression + /// in the equivalence class it matches with (if any). If the underlying + /// expression does not belong to any equivalence class in this group, returns + /// the sort expression as is. + pub fn normalize_sort_expr( + &self, + mut sort_expr: PhysicalSortExpr, + ) -> PhysicalSortExpr { + sort_expr.expr = self.normalize_expr(sort_expr.expr); + sort_expr + } + + /// Normalizes the given sort requirement according to this group. + /// The underlying physical expression is replaced with the first expression + /// in the equivalence class it matches with (if any). If the underlying + /// expression does not belong to any equivalence class in this group, returns + /// the given sort requirement as is. + pub fn normalize_sort_requirement( + &self, + mut sort_requirement: PhysicalSortRequirement, + ) -> PhysicalSortRequirement { + sort_requirement.expr = self.normalize_expr(sort_requirement.expr); + sort_requirement + } + + /// This function applies the `normalize_expr` function for all expressions + /// in `exprs` and returns the corresponding normalized physical expressions. + pub fn normalize_exprs( + &self, + exprs: impl IntoIterator>, + ) -> Vec> { + exprs + .into_iter() + .map(|expr| self.normalize_expr(expr)) + .collect() + } + + /// This function applies the `normalize_sort_expr` function for all sort + /// expressions in `sort_exprs` and returns the corresponding normalized + /// sort expressions. + pub fn normalize_sort_exprs(&self, sort_exprs: LexOrderingRef) -> LexOrdering { + // Convert sort expressions to sort requirements: + let sort_reqs = PhysicalSortRequirement::from_sort_exprs(sort_exprs.iter()); + // Normalize the requirements: + let normalized_sort_reqs = self.normalize_sort_requirements(&sort_reqs); + // Convert sort requirements back to sort expressions: + PhysicalSortRequirement::to_sort_exprs(normalized_sort_reqs) + } + + /// This function applies the `normalize_sort_requirement` function for all + /// requirements in `sort_reqs` and returns the corresponding normalized + /// sort requirements. + pub fn normalize_sort_requirements( + &self, + sort_reqs: LexRequirementRef, + ) -> LexRequirement { + collapse_lex_req( + sort_reqs + .iter() + .map(|sort_req| self.normalize_sort_requirement(sort_req.clone())) + .collect(), + ) + } + + /// Projects `expr` according to the given projection mapping. + /// If the resulting expression is invalid after projection, returns `None`. + pub fn project_expr( + &self, + mapping: &ProjectionMapping, + expr: &Arc, + ) -> Option> { + // First, we try to project expressions with an exact match. If we are + // unable to do this, we consult equivalence classes. + if let Some(target) = mapping.target_expr(expr) { + // If we match the source, we can project directly: + return Some(target); + } else { + // If the given expression is not inside the mapping, try to project + // expressions considering the equivalence classes. + for (source, target) in mapping.iter() { + // If we match an equivalent expression to `source`, then we can + // project. For example, if we have the mapping `(a as a1, a + c)` + // and the equivalence class `(a, b)`, expression `b` projects to `a1`. + if self + .get_equivalence_class(source) + .map_or(false, |group| group.contains(expr)) + { + return Some(target.clone()); + } + } + } + // Project a non-leaf expression by projecting its children. + let children = expr.children(); + if children.is_empty() { + // Leaf expression should be inside mapping. + return None; + } + children + .into_iter() + .map(|child| self.project_expr(mapping, &child)) + .collect::>>() + .map(|children| expr.clone().with_new_children(children).unwrap()) + } + + /// Projects this equivalence group according to the given projection mapping. + pub fn project(&self, mapping: &ProjectionMapping) -> Self { + let projected_classes = self.iter().filter_map(|cls| { + let new_class = cls + .iter() + .filter_map(|expr| self.project_expr(mapping, expr)) + .collect::>(); + (new_class.len() > 1).then_some(EquivalenceClass::new(new_class)) + }); + // TODO: Convert the algorithm below to a version that uses `HashMap`. + // once `Arc` can be stored in `HashMap`. + // See issue: https://github.com/apache/arrow-datafusion/issues/8027 + let mut new_classes = vec![]; + for (source, target) in mapping.iter() { + if new_classes.is_empty() { + new_classes.push((source, vec![target.clone()])); + } + if let Some((_, values)) = + new_classes.iter_mut().find(|(key, _)| key.eq(source)) + { + if !physical_exprs_contains(values, target) { + values.push(target.clone()); + } + } + } + // Only add equivalence classes with at least two members as singleton + // equivalence classes are meaningless. + let new_classes = new_classes + .into_iter() + .filter_map(|(_, values)| (values.len() > 1).then_some(values)) + .map(EquivalenceClass::new); + + let classes = projected_classes.chain(new_classes).collect(); + Self::new(classes) + } + + /// Returns the equivalence class containing `expr`. If no equivalence class + /// contains `expr`, returns `None`. + fn get_equivalence_class( + &self, + expr: &Arc, + ) -> Option<&EquivalenceClass> { + self.iter().find(|cls| cls.contains(expr)) + } + + /// Combine equivalence groups of the given join children. + pub fn join( + &self, + right_equivalences: &Self, + join_type: &JoinType, + left_size: usize, + on: &[(Column, Column)], + ) -> Self { + match join_type { + JoinType::Inner | JoinType::Left | JoinType::Full | JoinType::Right => { + let mut result = Self::new( + self.iter() + .cloned() + .chain( + right_equivalences + .iter() + .map(|cls| cls.with_offset(left_size)), + ) + .collect(), + ); + // In we have an inner join, expressions in the "on" condition + // are equal in the resulting table. + if join_type == &JoinType::Inner { + for (lhs, rhs) in on.iter() { + let index = rhs.index() + left_size; + let new_lhs = Arc::new(lhs.clone()) as _; + let new_rhs = Arc::new(Column::new(rhs.name(), index)) as _; + result.add_equal_conditions(&new_lhs, &new_rhs); + } + } + result + } + JoinType::LeftSemi | JoinType::LeftAnti => self.clone(), + JoinType::RightSemi | JoinType::RightAnti => right_equivalences.clone(), + } + } +} + +#[cfg(test)] +mod tests { + use super::*; + use crate::equivalence::tests::create_test_params; + use crate::equivalence::{EquivalenceClass, EquivalenceGroup}; + use crate::expressions::lit; + use crate::expressions::Column; + use crate::expressions::Literal; + use datafusion_common::Result; + use datafusion_common::ScalarValue; + use std::sync::Arc; + + #[test] + fn test_bridge_groups() -> Result<()> { + // First entry in the tuple is argument, second entry is the bridged result + let test_cases = vec![ + // ------- TEST CASE 1 -----------// + ( + vec![vec![1, 2, 3], vec![2, 4, 5], vec![11, 12, 9], vec![7, 6, 5]], + // Expected is compared with set equality. Order of the specific results may change. + vec![vec![1, 2, 3, 4, 5, 6, 7], vec![9, 11, 12]], + ), + // ------- TEST CASE 2 -----------// + ( + vec![vec![1, 2, 3], vec![3, 4, 5], vec![9, 8, 7], vec![7, 6, 5]], + // Expected + vec![vec![1, 2, 3, 4, 5, 6, 7, 8, 9]], + ), + ]; + for (entries, expected) in test_cases { + let entries = entries + .into_iter() + .map(|entry| entry.into_iter().map(lit).collect::>()) + .map(EquivalenceClass::new) + .collect::>(); + let expected = expected + .into_iter() + .map(|entry| entry.into_iter().map(lit).collect::>()) + .map(EquivalenceClass::new) + .collect::>(); + let mut eq_groups = EquivalenceGroup::new(entries.clone()); + eq_groups.bridge_classes(); + let eq_groups = eq_groups.classes; + let err_msg = format!( + "error in test entries: {:?}, expected: {:?}, actual:{:?}", + entries, expected, eq_groups + ); + assert_eq!(eq_groups.len(), expected.len(), "{}", err_msg); + for idx in 0..eq_groups.len() { + assert_eq!(&eq_groups[idx], &expected[idx], "{}", err_msg); + } + } + Ok(()) + } + + #[test] + fn test_remove_redundant_entries_eq_group() -> Result<()> { + let entries = vec![ + EquivalenceClass::new(vec![lit(1), lit(1), lit(2)]), + // This group is meaningless should be removed + EquivalenceClass::new(vec![lit(3), lit(3)]), + EquivalenceClass::new(vec![lit(4), lit(5), lit(6)]), + ]; + // Given equivalences classes are not in succinct form. + // Expected form is the most plain representation that is functionally same. + let expected = vec![ + EquivalenceClass::new(vec![lit(1), lit(2)]), + EquivalenceClass::new(vec![lit(4), lit(5), lit(6)]), + ]; + let mut eq_groups = EquivalenceGroup::new(entries); + eq_groups.remove_redundant_entries(); + + let eq_groups = eq_groups.classes; + assert_eq!(eq_groups.len(), expected.len()); + assert_eq!(eq_groups.len(), 2); + + assert_eq!(eq_groups[0], expected[0]); + assert_eq!(eq_groups[1], expected[1]); + Ok(()) + } + + #[test] + fn test_schema_normalize_expr_with_equivalence() -> Result<()> { + let col_a = &Column::new("a", 0); + let col_b = &Column::new("b", 1); + let col_c = &Column::new("c", 2); + // Assume that column a and c are aliases. + let (_test_schema, eq_properties) = create_test_params()?; + + let col_a_expr = Arc::new(col_a.clone()) as Arc; + let col_b_expr = Arc::new(col_b.clone()) as Arc; + let col_c_expr = Arc::new(col_c.clone()) as Arc; + // Test cases for equivalence normalization, + // First entry in the tuple is argument, second entry is expected result after normalization. + let expressions = vec![ + // Normalized version of the column a and c should go to a + // (by convention all the expressions inside equivalence class are mapped to the first entry + // in this case a is the first entry in the equivalence class.) + (&col_a_expr, &col_a_expr), + (&col_c_expr, &col_a_expr), + // Cannot normalize column b + (&col_b_expr, &col_b_expr), + ]; + let eq_group = eq_properties.eq_group(); + for (expr, expected_eq) in expressions { + assert!( + expected_eq.eq(&eq_group.normalize_expr(expr.clone())), + "error in test: expr: {expr:?}" + ); + } + + Ok(()) + } + + #[test] + fn test_contains_any() { + let lit_true = Arc::new(Literal::new(ScalarValue::Boolean(Some(true)))) + as Arc; + let lit_false = Arc::new(Literal::new(ScalarValue::Boolean(Some(false)))) + as Arc; + let lit2 = + Arc::new(Literal::new(ScalarValue::Int32(Some(2)))) as Arc; + let lit1 = + Arc::new(Literal::new(ScalarValue::Int32(Some(1)))) as Arc; + let col_b_expr = Arc::new(Column::new("b", 1)) as Arc; + + let cls1 = EquivalenceClass::new(vec![lit_true.clone(), lit_false.clone()]); + let cls2 = EquivalenceClass::new(vec![lit_true.clone(), col_b_expr.clone()]); + let cls3 = EquivalenceClass::new(vec![lit2.clone(), lit1.clone()]); + + // lit_true is common + assert!(cls1.contains_any(&cls2)); + // there is no common entry + assert!(!cls1.contains_any(&cls3)); + assert!(!cls2.contains_any(&cls3)); + } +} diff --git a/datafusion/physical-expr/src/equivalence/mod.rs b/datafusion/physical-expr/src/equivalence/mod.rs new file mode 100644 index 000000000000..387dce2cdc8b --- /dev/null +++ b/datafusion/physical-expr/src/equivalence/mod.rs @@ -0,0 +1,533 @@ +// 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. + +mod class; +mod ordering; +mod projection; +mod properties; +use crate::expressions::Column; +use crate::{LexRequirement, PhysicalExpr, PhysicalSortRequirement}; +pub use class::{EquivalenceClass, EquivalenceGroup}; +use datafusion_common::tree_node::{Transformed, TreeNode}; +pub use ordering::OrderingEquivalenceClass; +pub use projection::ProjectionMapping; +pub use properties::{join_equivalence_properties, EquivalenceProperties}; +use std::sync::Arc; + +/// This function constructs a duplicate-free `LexOrderingReq` by filtering out +/// duplicate entries that have same physical expression inside. For example, +/// `vec![a Some(ASC), a Some(DESC)]` collapses to `vec![a Some(ASC)]`. +pub fn collapse_lex_req(input: LexRequirement) -> LexRequirement { + let mut output = Vec::::new(); + for item in input { + if !output.iter().any(|req| req.expr.eq(&item.expr)) { + output.push(item); + } + } + output +} + +/// Adds the `offset` value to `Column` indices inside `expr`. This function is +/// generally used during the update of the right table schema in join operations. +pub fn add_offset_to_expr( + expr: Arc, + offset: usize, +) -> Arc { + expr.transform_down(&|e| match e.as_any().downcast_ref::() { + Some(col) => Ok(Transformed::Yes(Arc::new(Column::new( + col.name(), + offset + col.index(), + )))), + None => Ok(Transformed::No(e)), + }) + .unwrap() + // Note that we can safely unwrap here since our transform always returns + // an `Ok` value. +} + +#[cfg(test)] +mod tests { + use super::*; + use crate::expressions::{col, Column}; + use crate::PhysicalSortExpr; + use arrow::compute::{lexsort_to_indices, SortColumn}; + use arrow::datatypes::{DataType, Field, Schema}; + use arrow_array::{ArrayRef, Float64Array, RecordBatch, UInt32Array}; + use arrow_schema::{SchemaRef, SortOptions}; + use datafusion_common::{plan_datafusion_err, DataFusionError, Result}; + use itertools::izip; + use rand::rngs::StdRng; + use rand::seq::SliceRandom; + use rand::{Rng, SeedableRng}; + use std::sync::Arc; + + pub fn output_schema( + mapping: &ProjectionMapping, + input_schema: &Arc, + ) -> Result { + // Calculate output schema + let fields: Result> = mapping + .iter() + .map(|(source, target)| { + let name = target + .as_any() + .downcast_ref::() + .ok_or_else(|| plan_datafusion_err!("Expects to have column"))? + .name(); + let field = Field::new( + name, + source.data_type(input_schema)?, + source.nullable(input_schema)?, + ); + + Ok(field) + }) + .collect(); + + let output_schema = Arc::new(Schema::new_with_metadata( + fields?, + input_schema.metadata().clone(), + )); + + Ok(output_schema) + } + + // Generate a schema which consists of 8 columns (a, b, c, d, e, f, g, h) + pub fn create_test_schema() -> Result { + let a = Field::new("a", DataType::Int32, true); + let b = Field::new("b", DataType::Int32, true); + let c = Field::new("c", DataType::Int32, true); + let d = Field::new("d", DataType::Int32, true); + let e = Field::new("e", DataType::Int32, true); + let f = Field::new("f", DataType::Int32, true); + let g = Field::new("g", DataType::Int32, true); + let h = Field::new("h", DataType::Int32, true); + let schema = Arc::new(Schema::new(vec![a, b, c, d, e, f, g, h])); + + Ok(schema) + } + + /// Construct a schema with following properties + /// Schema satisfies following orderings: + /// [a ASC], [d ASC, b ASC], [e DESC, f ASC, g ASC] + /// and + /// Column [a=c] (e.g they are aliases). + pub fn create_test_params() -> Result<(SchemaRef, EquivalenceProperties)> { + let test_schema = create_test_schema()?; + let col_a = &col("a", &test_schema)?; + let col_b = &col("b", &test_schema)?; + let col_c = &col("c", &test_schema)?; + let col_d = &col("d", &test_schema)?; + let col_e = &col("e", &test_schema)?; + let col_f = &col("f", &test_schema)?; + let col_g = &col("g", &test_schema)?; + let mut eq_properties = EquivalenceProperties::new(test_schema.clone()); + eq_properties.add_equal_conditions(col_a, col_c); + + let option_asc = SortOptions { + descending: false, + nulls_first: false, + }; + let option_desc = SortOptions { + descending: true, + nulls_first: true, + }; + let orderings = vec![ + // [a ASC] + vec![(col_a, option_asc)], + // [d ASC, b ASC] + vec![(col_d, option_asc), (col_b, option_asc)], + // [e DESC, f ASC, g ASC] + vec![ + (col_e, option_desc), + (col_f, option_asc), + (col_g, option_asc), + ], + ]; + let orderings = convert_to_orderings(&orderings); + eq_properties.add_new_orderings(orderings); + Ok((test_schema, eq_properties)) + } + + // Generate a schema which consists of 6 columns (a, b, c, d, e, f) + fn create_test_schema_2() -> Result { + let a = Field::new("a", DataType::Float64, true); + let b = Field::new("b", DataType::Float64, true); + let c = Field::new("c", DataType::Float64, true); + let d = Field::new("d", DataType::Float64, true); + let e = Field::new("e", DataType::Float64, true); + let f = Field::new("f", DataType::Float64, true); + let schema = Arc::new(Schema::new(vec![a, b, c, d, e, f])); + + Ok(schema) + } + + /// Construct a schema with random ordering + /// among column a, b, c, d + /// where + /// Column [a=f] (e.g they are aliases). + /// Column e is constant. + pub fn create_random_schema(seed: u64) -> Result<(SchemaRef, EquivalenceProperties)> { + let test_schema = create_test_schema_2()?; + let col_a = &col("a", &test_schema)?; + let col_b = &col("b", &test_schema)?; + let col_c = &col("c", &test_schema)?; + let col_d = &col("d", &test_schema)?; + let col_e = &col("e", &test_schema)?; + let col_f = &col("f", &test_schema)?; + let col_exprs = [col_a, col_b, col_c, col_d, col_e, col_f]; + + let mut eq_properties = EquivalenceProperties::new(test_schema.clone()); + // Define a and f are aliases + eq_properties.add_equal_conditions(col_a, col_f); + // Column e has constant value. + eq_properties = eq_properties.add_constants([col_e.clone()]); + + // Randomly order columns for sorting + let mut rng = StdRng::seed_from_u64(seed); + let mut remaining_exprs = col_exprs[0..4].to_vec(); // only a, b, c, d are sorted + + let options_asc = SortOptions { + descending: false, + nulls_first: false, + }; + + while !remaining_exprs.is_empty() { + let n_sort_expr = rng.gen_range(0..remaining_exprs.len() + 1); + remaining_exprs.shuffle(&mut rng); + + let ordering = remaining_exprs + .drain(0..n_sort_expr) + .map(|expr| PhysicalSortExpr { + expr: expr.clone(), + options: options_asc, + }) + .collect(); + + eq_properties.add_new_orderings([ordering]); + } + + Ok((test_schema, eq_properties)) + } + + // Convert each tuple to PhysicalSortRequirement + pub fn convert_to_sort_reqs( + in_data: &[(&Arc, Option)], + ) -> Vec { + in_data + .iter() + .map(|(expr, options)| { + PhysicalSortRequirement::new((*expr).clone(), *options) + }) + .collect() + } + + // Convert each tuple to PhysicalSortExpr + pub fn convert_to_sort_exprs( + in_data: &[(&Arc, SortOptions)], + ) -> Vec { + in_data + .iter() + .map(|(expr, options)| PhysicalSortExpr { + expr: (*expr).clone(), + options: *options, + }) + .collect() + } + + // Convert each inner tuple to PhysicalSortExpr + pub fn convert_to_orderings( + orderings: &[Vec<(&Arc, SortOptions)>], + ) -> Vec> { + orderings + .iter() + .map(|sort_exprs| convert_to_sort_exprs(sort_exprs)) + .collect() + } + + // Convert each tuple to PhysicalSortExpr + pub fn convert_to_sort_exprs_owned( + in_data: &[(Arc, SortOptions)], + ) -> Vec { + in_data + .iter() + .map(|(expr, options)| PhysicalSortExpr { + expr: (*expr).clone(), + options: *options, + }) + .collect() + } + + // Convert each inner tuple to PhysicalSortExpr + pub fn convert_to_orderings_owned( + orderings: &[Vec<(Arc, SortOptions)>], + ) -> Vec> { + orderings + .iter() + .map(|sort_exprs| convert_to_sort_exprs_owned(sort_exprs)) + .collect() + } + + // Apply projection to the input_data, return projected equivalence properties and record batch + pub fn apply_projection( + proj_exprs: Vec<(Arc, String)>, + input_data: &RecordBatch, + input_eq_properties: &EquivalenceProperties, + ) -> Result<(RecordBatch, EquivalenceProperties)> { + let input_schema = input_data.schema(); + let projection_mapping = ProjectionMapping::try_new(&proj_exprs, &input_schema)?; + + let output_schema = output_schema(&projection_mapping, &input_schema)?; + let num_rows = input_data.num_rows(); + // Apply projection to the input record batch. + let projected_values = projection_mapping + .iter() + .map(|(source, _target)| source.evaluate(input_data)?.into_array(num_rows)) + .collect::>>()?; + let projected_batch = if projected_values.is_empty() { + RecordBatch::new_empty(output_schema.clone()) + } else { + RecordBatch::try_new(output_schema.clone(), projected_values)? + }; + + let projected_eq = + input_eq_properties.project(&projection_mapping, output_schema); + Ok((projected_batch, projected_eq)) + } + + #[test] + fn add_equal_conditions_test() -> Result<()> { + let schema = Arc::new(Schema::new(vec![ + Field::new("a", DataType::Int64, true), + Field::new("b", DataType::Int64, true), + Field::new("c", DataType::Int64, true), + Field::new("x", DataType::Int64, true), + Field::new("y", DataType::Int64, true), + ])); + + let mut eq_properties = EquivalenceProperties::new(schema); + let col_a_expr = Arc::new(Column::new("a", 0)) as Arc; + let col_b_expr = Arc::new(Column::new("b", 1)) as Arc; + let col_c_expr = Arc::new(Column::new("c", 2)) as Arc; + let col_x_expr = Arc::new(Column::new("x", 3)) as Arc; + let col_y_expr = Arc::new(Column::new("y", 4)) as Arc; + + // a and b are aliases + eq_properties.add_equal_conditions(&col_a_expr, &col_b_expr); + assert_eq!(eq_properties.eq_group().len(), 1); + + // This new entry is redundant, size shouldn't increase + eq_properties.add_equal_conditions(&col_b_expr, &col_a_expr); + assert_eq!(eq_properties.eq_group().len(), 1); + let eq_groups = &eq_properties.eq_group().classes[0]; + assert_eq!(eq_groups.len(), 2); + assert!(eq_groups.contains(&col_a_expr)); + assert!(eq_groups.contains(&col_b_expr)); + + // b and c are aliases. Exising equivalence class should expand, + // however there shouldn't be any new equivalence class + eq_properties.add_equal_conditions(&col_b_expr, &col_c_expr); + assert_eq!(eq_properties.eq_group().len(), 1); + let eq_groups = &eq_properties.eq_group().classes[0]; + assert_eq!(eq_groups.len(), 3); + assert!(eq_groups.contains(&col_a_expr)); + assert!(eq_groups.contains(&col_b_expr)); + assert!(eq_groups.contains(&col_c_expr)); + + // This is a new set of equality. Hence equivalent class count should be 2. + eq_properties.add_equal_conditions(&col_x_expr, &col_y_expr); + assert_eq!(eq_properties.eq_group().len(), 2); + + // This equality bridges distinct equality sets. + // Hence equivalent class count should decrease from 2 to 1. + eq_properties.add_equal_conditions(&col_x_expr, &col_a_expr); + assert_eq!(eq_properties.eq_group().len(), 1); + let eq_groups = &eq_properties.eq_group().classes[0]; + assert_eq!(eq_groups.len(), 5); + assert!(eq_groups.contains(&col_a_expr)); + assert!(eq_groups.contains(&col_b_expr)); + assert!(eq_groups.contains(&col_c_expr)); + assert!(eq_groups.contains(&col_x_expr)); + assert!(eq_groups.contains(&col_y_expr)); + + Ok(()) + } + + /// Checks if the table (RecordBatch) remains unchanged when sorted according to the provided `required_ordering`. + /// + /// The function works by adding a unique column of ascending integers to the original table. This column ensures + /// that rows that are otherwise indistinguishable (e.g., if they have the same values in all other columns) can + /// still be differentiated. When sorting the extended table, the unique column acts as a tie-breaker to produce + /// deterministic sorting results. + /// + /// If the table remains the same after sorting with the added unique column, it indicates that the table was + /// already sorted according to `required_ordering` to begin with. + pub fn is_table_same_after_sort( + mut required_ordering: Vec, + batch: RecordBatch, + ) -> Result { + // Clone the original schema and columns + let original_schema = batch.schema(); + let mut columns = batch.columns().to_vec(); + + // Create a new unique column + let n_row = batch.num_rows(); + let vals: Vec = (0..n_row).collect::>(); + let vals: Vec = vals.into_iter().map(|val| val as f64).collect(); + let unique_col = Arc::new(Float64Array::from_iter_values(vals)) as ArrayRef; + columns.push(unique_col.clone()); + + // Create a new schema with the added unique column + let unique_col_name = "unique"; + let unique_field = + Arc::new(Field::new(unique_col_name, DataType::Float64, false)); + let fields: Vec<_> = original_schema + .fields() + .iter() + .cloned() + .chain(std::iter::once(unique_field)) + .collect(); + let schema = Arc::new(Schema::new(fields)); + + // Create a new batch with the added column + let new_batch = RecordBatch::try_new(schema.clone(), columns)?; + + // Add the unique column to the required ordering to ensure deterministic results + required_ordering.push(PhysicalSortExpr { + expr: Arc::new(Column::new(unique_col_name, original_schema.fields().len())), + options: Default::default(), + }); + + // Convert the required ordering to a list of SortColumn + let sort_columns = required_ordering + .iter() + .map(|order_expr| { + let expr_result = order_expr.expr.evaluate(&new_batch)?; + let values = expr_result.into_array(new_batch.num_rows())?; + Ok(SortColumn { + values, + options: Some(order_expr.options), + }) + }) + .collect::>>()?; + + // Check if the indices after sorting match the initial ordering + let sorted_indices = lexsort_to_indices(&sort_columns, None)?; + let original_indices = UInt32Array::from_iter_values(0..n_row as u32); + + Ok(sorted_indices == original_indices) + } + + // If we already generated a random result for one of the + // expressions in the equivalence classes. For other expressions in the same + // equivalence class use same result. This util gets already calculated result, when available. + fn get_representative_arr( + eq_group: &EquivalenceClass, + existing_vec: &[Option], + schema: SchemaRef, + ) -> Option { + for expr in eq_group.iter() { + let col = expr.as_any().downcast_ref::().unwrap(); + let (idx, _field) = schema.column_with_name(col.name()).unwrap(); + if let Some(res) = &existing_vec[idx] { + return Some(res.clone()); + } + } + None + } + + // Generate a table that satisfies the given equivalence properties; i.e. + // equivalences, ordering equivalences, and constants. + pub fn generate_table_for_eq_properties( + eq_properties: &EquivalenceProperties, + n_elem: usize, + n_distinct: usize, + ) -> Result { + let mut rng = StdRng::seed_from_u64(23); + + let schema = eq_properties.schema(); + let mut schema_vec = vec![None; schema.fields.len()]; + + // Utility closure to generate random array + let mut generate_random_array = |num_elems: usize, max_val: usize| -> ArrayRef { + let values: Vec = (0..num_elems) + .map(|_| rng.gen_range(0..max_val) as f64 / 2.0) + .collect(); + Arc::new(Float64Array::from_iter_values(values)) + }; + + // Fill constant columns + for constant in &eq_properties.constants { + let col = constant.as_any().downcast_ref::().unwrap(); + let (idx, _field) = schema.column_with_name(col.name()).unwrap(); + let arr = Arc::new(Float64Array::from_iter_values(vec![0 as f64; n_elem])) + as ArrayRef; + schema_vec[idx] = Some(arr); + } + + // Fill columns based on ordering equivalences + for ordering in eq_properties.oeq_class.iter() { + let (sort_columns, indices): (Vec<_>, Vec<_>) = ordering + .iter() + .map(|PhysicalSortExpr { expr, options }| { + let col = expr.as_any().downcast_ref::().unwrap(); + let (idx, _field) = schema.column_with_name(col.name()).unwrap(); + let arr = generate_random_array(n_elem, n_distinct); + ( + SortColumn { + values: arr, + options: Some(*options), + }, + idx, + ) + }) + .unzip(); + + let sort_arrs = arrow::compute::lexsort(&sort_columns, None)?; + for (idx, arr) in izip!(indices, sort_arrs) { + schema_vec[idx] = Some(arr); + } + } + + // Fill columns based on equivalence groups + for eq_group in eq_properties.eq_group.iter() { + let representative_array = + get_representative_arr(eq_group, &schema_vec, schema.clone()) + .unwrap_or_else(|| generate_random_array(n_elem, n_distinct)); + + for expr in eq_group.iter() { + let col = expr.as_any().downcast_ref::().unwrap(); + let (idx, _field) = schema.column_with_name(col.name()).unwrap(); + schema_vec[idx] = Some(representative_array.clone()); + } + } + + let res: Vec<_> = schema_vec + .into_iter() + .zip(schema.fields.iter()) + .map(|(elem, field)| { + ( + field.name(), + // Generate random values for columns that do not occur in any of the groups (equivalence, ordering equivalence, constants) + elem.unwrap_or_else(|| generate_random_array(n_elem, n_distinct)), + ) + }) + .collect(); + + Ok(RecordBatch::try_from_iter(res)?) + } +} diff --git a/datafusion/physical-expr/src/equivalence/ordering.rs b/datafusion/physical-expr/src/equivalence/ordering.rs new file mode 100644 index 000000000000..1a414592ce4c --- /dev/null +++ b/datafusion/physical-expr/src/equivalence/ordering.rs @@ -0,0 +1,1159 @@ +// 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_schema::SortOptions; +use std::hash::Hash; +use std::sync::Arc; + +use crate::equivalence::add_offset_to_expr; +use crate::{LexOrdering, PhysicalExpr, PhysicalSortExpr}; + +/// An `OrderingEquivalenceClass` object keeps track of different alternative +/// orderings than can describe a schema. For example, consider the following table: +/// +/// ```text +/// |a|b|c|d| +/// |1|4|3|1| +/// |2|3|3|2| +/// |3|1|2|2| +/// |3|2|1|3| +/// ``` +/// +/// Here, both `vec![a ASC, b ASC]` and `vec![c DESC, d ASC]` describe the table +/// ordering. In this case, we say that these orderings are equivalent. +#[derive(Debug, Clone, Eq, PartialEq, Hash)] +pub struct OrderingEquivalenceClass { + pub orderings: Vec, +} + +impl OrderingEquivalenceClass { + /// Creates new empty ordering equivalence class. + pub fn empty() -> Self { + Self { orderings: vec![] } + } + + /// Clears (empties) this ordering equivalence class. + pub fn clear(&mut self) { + self.orderings.clear(); + } + + /// Creates new ordering equivalence class from the given orderings. + pub fn new(orderings: Vec) -> Self { + let mut result = Self { orderings }; + result.remove_redundant_entries(); + result + } + + /// Checks whether `ordering` is a member of this equivalence class. + pub fn contains(&self, ordering: &LexOrdering) -> bool { + self.orderings.contains(ordering) + } + + /// Adds `ordering` to this equivalence class. + #[allow(dead_code)] + fn push(&mut self, ordering: LexOrdering) { + self.orderings.push(ordering); + // Make sure that there are no redundant orderings: + self.remove_redundant_entries(); + } + + /// Checks whether this ordering equivalence class is empty. + pub fn is_empty(&self) -> bool { + self.len() == 0 + } + + /// Returns an iterator over the equivalent orderings in this class. + pub fn iter(&self) -> impl Iterator { + self.orderings.iter() + } + + /// Returns how many equivalent orderings there are in this class. + pub fn len(&self) -> usize { + self.orderings.len() + } + + /// Extend this ordering equivalence class with the `other` class. + pub fn extend(&mut self, other: Self) { + self.orderings.extend(other.orderings); + // Make sure that there are no redundant orderings: + self.remove_redundant_entries(); + } + + /// Adds new orderings into this ordering equivalence class. + pub fn add_new_orderings( + &mut self, + orderings: impl IntoIterator, + ) { + self.orderings.extend(orderings); + // Make sure that there are no redundant orderings: + self.remove_redundant_entries(); + } + + /// Removes redundant orderings from this equivalence class. For instance, + /// if we already have the ordering `[a ASC, b ASC, c DESC]`, then there is + /// no need to keep ordering `[a ASC, b ASC]` in the state. + fn remove_redundant_entries(&mut self) { + let mut work = true; + while work { + work = false; + let mut idx = 0; + while idx < self.orderings.len() { + let mut ordering_idx = idx + 1; + let mut removal = self.orderings[idx].is_empty(); + while ordering_idx < self.orderings.len() { + work |= resolve_overlap(&mut self.orderings, idx, ordering_idx); + if self.orderings[idx].is_empty() { + removal = true; + break; + } + work |= resolve_overlap(&mut self.orderings, ordering_idx, idx); + if self.orderings[ordering_idx].is_empty() { + self.orderings.swap_remove(ordering_idx); + } else { + ordering_idx += 1; + } + } + if removal { + self.orderings.swap_remove(idx); + } else { + idx += 1; + } + } + } + } + + /// Returns the concatenation of all the orderings. This enables merge + /// operations to preserve all equivalent orderings simultaneously. + pub fn output_ordering(&self) -> Option { + let output_ordering = self.orderings.iter().flatten().cloned().collect(); + let output_ordering = collapse_lex_ordering(output_ordering); + (!output_ordering.is_empty()).then_some(output_ordering) + } + + // Append orderings in `other` to all existing orderings in this equivalence + // class. + pub fn join_suffix(mut self, other: &Self) -> Self { + let n_ordering = self.orderings.len(); + // Replicate entries before cross product + let n_cross = std::cmp::max(n_ordering, other.len() * n_ordering); + self.orderings = self + .orderings + .iter() + .cloned() + .cycle() + .take(n_cross) + .collect(); + // Suffix orderings of other to the current orderings. + for (outer_idx, ordering) in other.iter().enumerate() { + for idx in 0..n_ordering { + // Calculate cross product index + let idx = outer_idx * n_ordering + idx; + self.orderings[idx].extend(ordering.iter().cloned()); + } + } + self + } + + /// Adds `offset` value to the index of each expression inside this + /// ordering equivalence class. + pub fn add_offset(&mut self, offset: usize) { + for ordering in self.orderings.iter_mut() { + for sort_expr in ordering { + sort_expr.expr = add_offset_to_expr(sort_expr.expr.clone(), offset); + } + } + } + + /// Gets sort options associated with this expression if it is a leading + /// ordering expression. Otherwise, returns `None`. + pub fn get_options(&self, expr: &Arc) -> Option { + for ordering in self.iter() { + let leading_ordering = &ordering[0]; + if leading_ordering.expr.eq(expr) { + return Some(leading_ordering.options); + } + } + None + } +} + +/// This function constructs a duplicate-free `LexOrdering` by filtering out +/// duplicate entries that have same physical expression inside. For example, +/// `vec![a ASC, a DESC]` collapses to `vec![a ASC]`. +pub fn collapse_lex_ordering(input: LexOrdering) -> LexOrdering { + let mut output = Vec::::new(); + for item in input { + if !output.iter().any(|req| req.expr.eq(&item.expr)) { + output.push(item); + } + } + output +} + +/// Trims `orderings[idx]` if some suffix of it overlaps with a prefix of +/// `orderings[pre_idx]`. Returns `true` if there is any overlap, `false` otherwise. +fn resolve_overlap(orderings: &mut [LexOrdering], idx: usize, pre_idx: usize) -> bool { + let length = orderings[idx].len(); + let other_length = orderings[pre_idx].len(); + for overlap in 1..=length.min(other_length) { + if orderings[idx][length - overlap..] == orderings[pre_idx][..overlap] { + orderings[idx].truncate(length - overlap); + return true; + } + } + false +} + +#[cfg(test)] +mod tests { + use crate::equivalence::tests::{ + convert_to_orderings, convert_to_sort_exprs, create_random_schema, + create_test_params, generate_table_for_eq_properties, is_table_same_after_sort, + }; + use crate::equivalence::{tests::create_test_schema, EquivalenceProperties}; + use crate::equivalence::{ + EquivalenceClass, EquivalenceGroup, OrderingEquivalenceClass, + }; + use crate::execution_props::ExecutionProps; + use crate::expressions::Column; + use crate::expressions::{col, BinaryExpr}; + use crate::functions::create_physical_expr; + use crate::{PhysicalExpr, PhysicalSortExpr}; + use arrow::datatypes::{DataType, Field, Schema}; + use arrow_schema::SortOptions; + use datafusion_common::Result; + use datafusion_expr::{BuiltinScalarFunction, Operator}; + use itertools::Itertools; + use std::sync::Arc; + + #[test] + fn test_ordering_satisfy() -> Result<()> { + let input_schema = Arc::new(Schema::new(vec![ + Field::new("a", DataType::Int64, true), + Field::new("b", DataType::Int64, true), + ])); + let crude = vec![PhysicalSortExpr { + expr: Arc::new(Column::new("a", 0)), + options: SortOptions::default(), + }]; + let finer = vec![ + PhysicalSortExpr { + expr: Arc::new(Column::new("a", 0)), + options: SortOptions::default(), + }, + PhysicalSortExpr { + expr: Arc::new(Column::new("b", 1)), + options: SortOptions::default(), + }, + ]; + // finer ordering satisfies, crude ordering should return true + let mut eq_properties_finer = EquivalenceProperties::new(input_schema.clone()); + eq_properties_finer.oeq_class.push(finer.clone()); + assert!(eq_properties_finer.ordering_satisfy(&crude)); + + // Crude ordering doesn't satisfy finer ordering. should return false + let mut eq_properties_crude = EquivalenceProperties::new(input_schema.clone()); + eq_properties_crude.oeq_class.push(crude.clone()); + assert!(!eq_properties_crude.ordering_satisfy(&finer)); + Ok(()) + } + + #[test] + fn test_ordering_satisfy_with_equivalence2() -> Result<()> { + let test_schema = create_test_schema()?; + let col_a = &col("a", &test_schema)?; + let col_b = &col("b", &test_schema)?; + let col_c = &col("c", &test_schema)?; + let col_d = &col("d", &test_schema)?; + let col_e = &col("e", &test_schema)?; + let col_f = &col("f", &test_schema)?; + let floor_a = &create_physical_expr( + &BuiltinScalarFunction::Floor, + &[col("a", &test_schema)?], + &test_schema, + &ExecutionProps::default(), + )?; + let floor_f = &create_physical_expr( + &BuiltinScalarFunction::Floor, + &[col("f", &test_schema)?], + &test_schema, + &ExecutionProps::default(), + )?; + let exp_a = &create_physical_expr( + &BuiltinScalarFunction::Exp, + &[col("a", &test_schema)?], + &test_schema, + &ExecutionProps::default(), + )?; + let a_plus_b = Arc::new(BinaryExpr::new( + col_a.clone(), + Operator::Plus, + col_b.clone(), + )) as Arc; + let options = SortOptions { + descending: false, + nulls_first: false, + }; + + let test_cases = vec![ + // ------------ TEST CASE 1 ------------ + ( + // orderings + vec![ + // [a ASC, d ASC, b ASC] + vec![(col_a, options), (col_d, options), (col_b, options)], + // [c ASC] + vec![(col_c, options)], + ], + // equivalence classes + vec![vec![col_a, col_f]], + // constants + vec![col_e], + // requirement [a ASC, b ASC], requirement is not satisfied. + vec![(col_a, options), (col_b, options)], + // expected: requirement is not satisfied. + false, + ), + // ------------ TEST CASE 2 ------------ + ( + // orderings + vec![ + // [a ASC, c ASC, b ASC] + vec![(col_a, options), (col_c, options), (col_b, options)], + // [d ASC] + vec![(col_d, options)], + ], + // equivalence classes + vec![vec![col_a, col_f]], + // constants + vec![col_e], + // requirement [floor(a) ASC], + vec![(floor_a, options)], + // expected: requirement is satisfied. + true, + ), + // ------------ TEST CASE 2.1 ------------ + ( + // orderings + vec![ + // [a ASC, c ASC, b ASC] + vec![(col_a, options), (col_c, options), (col_b, options)], + // [d ASC] + vec![(col_d, options)], + ], + // equivalence classes + vec![vec![col_a, col_f]], + // constants + vec![col_e], + // requirement [floor(f) ASC], (Please note that a=f) + vec![(floor_f, options)], + // expected: requirement is satisfied. + true, + ), + // ------------ TEST CASE 3 ------------ + ( + // orderings + vec![ + // [a ASC, c ASC, b ASC] + vec![(col_a, options), (col_c, options), (col_b, options)], + // [d ASC] + vec![(col_d, options)], + ], + // equivalence classes + vec![vec![col_a, col_f]], + // constants + vec![col_e], + // requirement [a ASC, c ASC, a+b ASC], + vec![(col_a, options), (col_c, options), (&a_plus_b, options)], + // expected: requirement is satisfied. + true, + ), + // ------------ TEST CASE 4 ------------ + ( + // orderings + vec![ + // [a ASC, b ASC, c ASC, d ASC] + vec![ + (col_a, options), + (col_b, options), + (col_c, options), + (col_d, options), + ], + ], + // equivalence classes + vec![vec![col_a, col_f]], + // constants + vec![col_e], + // requirement [floor(a) ASC, a+b ASC], + vec![(floor_a, options), (&a_plus_b, options)], + // expected: requirement is satisfied. + false, + ), + // ------------ TEST CASE 5 ------------ + ( + // orderings + vec![ + // [a ASC, b ASC, c ASC, d ASC] + vec![ + (col_a, options), + (col_b, options), + (col_c, options), + (col_d, options), + ], + ], + // equivalence classes + vec![vec![col_a, col_f]], + // constants + vec![col_e], + // requirement [exp(a) ASC, a+b ASC], + vec![(exp_a, options), (&a_plus_b, options)], + // expected: requirement is not satisfied. + // TODO: If we know that exp function is 1-to-1 function. + // we could have deduced that above requirement is satisfied. + false, + ), + // ------------ TEST CASE 6 ------------ + ( + // orderings + vec![ + // [a ASC, d ASC, b ASC] + vec![(col_a, options), (col_d, options), (col_b, options)], + // [c ASC] + vec![(col_c, options)], + ], + // equivalence classes + vec![vec![col_a, col_f]], + // constants + vec![col_e], + // requirement [a ASC, d ASC, floor(a) ASC], + vec![(col_a, options), (col_d, options), (floor_a, options)], + // expected: requirement is satisfied. + true, + ), + // ------------ TEST CASE 7 ------------ + ( + // orderings + vec![ + // [a ASC, c ASC, b ASC] + vec![(col_a, options), (col_c, options), (col_b, options)], + // [d ASC] + vec![(col_d, options)], + ], + // equivalence classes + vec![vec![col_a, col_f]], + // constants + vec![col_e], + // requirement [a ASC, floor(a) ASC, a + b ASC], + vec![(col_a, options), (floor_a, options), (&a_plus_b, options)], + // expected: requirement is not satisfied. + false, + ), + // ------------ TEST CASE 8 ------------ + ( + // orderings + vec![ + // [a ASC, b ASC, c ASC] + vec![(col_a, options), (col_b, options), (col_c, options)], + // [d ASC] + vec![(col_d, options)], + ], + // equivalence classes + vec![vec![col_a, col_f]], + // constants + vec![col_e], + // requirement [a ASC, c ASC, floor(a) ASC, a + b ASC], + vec![ + (col_a, options), + (col_c, options), + (&floor_a, options), + (&a_plus_b, options), + ], + // expected: requirement is not satisfied. + false, + ), + // ------------ TEST CASE 9 ------------ + ( + // orderings + vec![ + // [a ASC, b ASC, c ASC, d ASC] + vec![ + (col_a, options), + (col_b, options), + (col_c, options), + (col_d, options), + ], + ], + // equivalence classes + vec![vec![col_a, col_f]], + // constants + vec![col_e], + // requirement [a ASC, b ASC, c ASC, floor(a) ASC], + vec![ + (col_a, options), + (col_b, options), + (&col_c, options), + (&floor_a, options), + ], + // expected: requirement is satisfied. + true, + ), + // ------------ TEST CASE 10 ------------ + ( + // orderings + vec![ + // [d ASC, b ASC] + vec![(col_d, options), (col_b, options)], + // [c ASC, a ASC] + vec![(col_c, options), (col_a, options)], + ], + // equivalence classes + vec![vec![col_a, col_f]], + // constants + vec![col_e], + // requirement [c ASC, d ASC, a + b ASC], + vec![(col_c, options), (col_d, options), (&a_plus_b, options)], + // expected: requirement is satisfied. + true, + ), + ]; + + for (orderings, eq_group, constants, reqs, expected) in test_cases { + let err_msg = + format!("error in test orderings: {orderings:?}, eq_group: {eq_group:?}, constants: {constants:?}, reqs: {reqs:?}, expected: {expected:?}"); + let mut eq_properties = EquivalenceProperties::new(test_schema.clone()); + let orderings = convert_to_orderings(&orderings); + eq_properties.add_new_orderings(orderings); + let eq_group = eq_group + .into_iter() + .map(|eq_class| { + let eq_classes = eq_class.into_iter().cloned().collect::>(); + EquivalenceClass::new(eq_classes) + }) + .collect::>(); + let eq_group = EquivalenceGroup::new(eq_group); + eq_properties.add_equivalence_group(eq_group); + + let constants = constants.into_iter().cloned(); + eq_properties = eq_properties.add_constants(constants); + + let reqs = convert_to_sort_exprs(&reqs); + assert_eq!( + eq_properties.ordering_satisfy(&reqs), + expected, + "{}", + err_msg + ); + } + + Ok(()) + } + + #[test] + fn test_ordering_satisfy_with_equivalence() -> Result<()> { + // Schema satisfies following orderings: + // [a ASC], [d ASC, b ASC], [e DESC, f ASC, g ASC] + // and + // Column [a=c] (e.g they are aliases). + let (test_schema, eq_properties) = create_test_params()?; + let col_a = &col("a", &test_schema)?; + let col_b = &col("b", &test_schema)?; + let col_c = &col("c", &test_schema)?; + let col_d = &col("d", &test_schema)?; + let col_e = &col("e", &test_schema)?; + let col_f = &col("f", &test_schema)?; + let col_g = &col("g", &test_schema)?; + let option_asc = SortOptions { + descending: false, + nulls_first: false, + }; + let option_desc = SortOptions { + descending: true, + nulls_first: true, + }; + let table_data_with_properties = + generate_table_for_eq_properties(&eq_properties, 625, 5)?; + + // First element in the tuple stores vector of requirement, second element is the expected return value for ordering_satisfy function + let requirements = vec![ + // `a ASC NULLS LAST`, expects `ordering_satisfy` to be `true`, since existing ordering `a ASC NULLS LAST, b ASC NULLS LAST` satisfies it + (vec![(col_a, option_asc)], true), + (vec![(col_a, option_desc)], false), + // Test whether equivalence works as expected + (vec![(col_c, option_asc)], true), + (vec![(col_c, option_desc)], false), + // Test whether ordering equivalence works as expected + (vec![(col_d, option_asc)], true), + (vec![(col_d, option_asc), (col_b, option_asc)], true), + (vec![(col_d, option_desc), (col_b, option_asc)], false), + ( + vec![ + (col_e, option_desc), + (col_f, option_asc), + (col_g, option_asc), + ], + true, + ), + (vec![(col_e, option_desc), (col_f, option_asc)], true), + (vec![(col_e, option_asc), (col_f, option_asc)], false), + (vec![(col_e, option_desc), (col_b, option_asc)], false), + (vec![(col_e, option_asc), (col_b, option_asc)], false), + ( + vec![ + (col_d, option_asc), + (col_b, option_asc), + (col_d, option_asc), + (col_b, option_asc), + ], + true, + ), + ( + vec![ + (col_d, option_asc), + (col_b, option_asc), + (col_e, option_desc), + (col_f, option_asc), + ], + true, + ), + ( + vec![ + (col_d, option_asc), + (col_b, option_asc), + (col_e, option_desc), + (col_b, option_asc), + ], + true, + ), + ( + vec![ + (col_d, option_asc), + (col_b, option_asc), + (col_d, option_desc), + (col_b, option_asc), + ], + true, + ), + ( + vec![ + (col_d, option_asc), + (col_b, option_asc), + (col_e, option_asc), + (col_f, option_asc), + ], + false, + ), + ( + vec![ + (col_d, option_asc), + (col_b, option_asc), + (col_e, option_asc), + (col_b, option_asc), + ], + false, + ), + (vec![(col_d, option_asc), (col_e, option_desc)], true), + ( + vec![ + (col_d, option_asc), + (col_c, option_asc), + (col_b, option_asc), + ], + true, + ), + ( + vec![ + (col_d, option_asc), + (col_e, option_desc), + (col_f, option_asc), + (col_b, option_asc), + ], + true, + ), + ( + vec![ + (col_d, option_asc), + (col_e, option_desc), + (col_c, option_asc), + (col_b, option_asc), + ], + true, + ), + ( + vec![ + (col_d, option_asc), + (col_e, option_desc), + (col_b, option_asc), + (col_f, option_asc), + ], + true, + ), + ]; + + for (cols, expected) in requirements { + let err_msg = format!("Error in test case:{cols:?}"); + let required = cols + .into_iter() + .map(|(expr, options)| PhysicalSortExpr { + expr: expr.clone(), + options, + }) + .collect::>(); + + // Check expected result with experimental result. + assert_eq!( + is_table_same_after_sort( + required.clone(), + table_data_with_properties.clone() + )?, + expected + ); + assert_eq!( + eq_properties.ordering_satisfy(&required), + expected, + "{err_msg}" + ); + } + Ok(()) + } + + #[test] + fn test_ordering_satisfy_with_equivalence_random() -> Result<()> { + const N_RANDOM_SCHEMA: usize = 5; + const N_ELEMENTS: usize = 125; + const N_DISTINCT: usize = 5; + const SORT_OPTIONS: SortOptions = SortOptions { + descending: false, + nulls_first: false, + }; + + for seed in 0..N_RANDOM_SCHEMA { + // Create a random schema with random properties + let (test_schema, eq_properties) = create_random_schema(seed as u64)?; + // Generate a data that satisfies properties given + let table_data_with_properties = + generate_table_for_eq_properties(&eq_properties, N_ELEMENTS, N_DISTINCT)?; + let col_exprs = vec![ + col("a", &test_schema)?, + col("b", &test_schema)?, + col("c", &test_schema)?, + col("d", &test_schema)?, + col("e", &test_schema)?, + col("f", &test_schema)?, + ]; + + for n_req in 0..=col_exprs.len() { + for exprs in col_exprs.iter().combinations(n_req) { + let requirement = exprs + .into_iter() + .map(|expr| PhysicalSortExpr { + expr: expr.clone(), + options: SORT_OPTIONS, + }) + .collect::>(); + let expected = is_table_same_after_sort( + requirement.clone(), + table_data_with_properties.clone(), + )?; + let err_msg = format!( + "Error in test case requirement:{:?}, expected: {:?}, eq_properties.oeq_class: {:?}, eq_properties.eq_group: {:?}, eq_properties.constants: {:?}", + requirement, expected, eq_properties.oeq_class, eq_properties.eq_group, eq_properties.constants + ); + // Check whether ordering_satisfy API result and + // experimental result matches. + assert_eq!( + eq_properties.ordering_satisfy(&requirement), + expected, + "{}", + err_msg + ); + } + } + } + + Ok(()) + } + + #[test] + fn test_ordering_satisfy_with_equivalence_complex_random() -> Result<()> { + const N_RANDOM_SCHEMA: usize = 100; + const N_ELEMENTS: usize = 125; + const N_DISTINCT: usize = 5; + const SORT_OPTIONS: SortOptions = SortOptions { + descending: false, + nulls_first: false, + }; + + for seed in 0..N_RANDOM_SCHEMA { + // Create a random schema with random properties + let (test_schema, eq_properties) = create_random_schema(seed as u64)?; + // Generate a data that satisfies properties given + let table_data_with_properties = + generate_table_for_eq_properties(&eq_properties, N_ELEMENTS, N_DISTINCT)?; + + let floor_a = create_physical_expr( + &BuiltinScalarFunction::Floor, + &[col("a", &test_schema)?], + &test_schema, + &ExecutionProps::default(), + )?; + let a_plus_b = Arc::new(BinaryExpr::new( + col("a", &test_schema)?, + Operator::Plus, + col("b", &test_schema)?, + )) as Arc; + let exprs = vec![ + col("a", &test_schema)?, + col("b", &test_schema)?, + col("c", &test_schema)?, + col("d", &test_schema)?, + col("e", &test_schema)?, + col("f", &test_schema)?, + floor_a, + a_plus_b, + ]; + + for n_req in 0..=exprs.len() { + for exprs in exprs.iter().combinations(n_req) { + let requirement = exprs + .into_iter() + .map(|expr| PhysicalSortExpr { + expr: expr.clone(), + options: SORT_OPTIONS, + }) + .collect::>(); + let expected = is_table_same_after_sort( + requirement.clone(), + table_data_with_properties.clone(), + )?; + let err_msg = format!( + "Error in test case requirement:{:?}, expected: {:?}, eq_properties.oeq_class: {:?}, eq_properties.eq_group: {:?}, eq_properties.constants: {:?}", + requirement, expected, eq_properties.oeq_class, eq_properties.eq_group, eq_properties.constants + ); + // Check whether ordering_satisfy API result and + // experimental result matches. + + assert_eq!( + eq_properties.ordering_satisfy(&requirement), + (expected | false), + "{}", + err_msg + ); + } + } + } + + Ok(()) + } + + #[test] + fn test_ordering_satisfy_different_lengths() -> Result<()> { + let test_schema = create_test_schema()?; + let col_a = &col("a", &test_schema)?; + let col_b = &col("b", &test_schema)?; + let col_c = &col("c", &test_schema)?; + let col_d = &col("d", &test_schema)?; + let col_e = &col("e", &test_schema)?; + let col_f = &col("f", &test_schema)?; + let options = SortOptions { + descending: false, + nulls_first: false, + }; + // a=c (e.g they are aliases). + let mut eq_properties = EquivalenceProperties::new(test_schema); + eq_properties.add_equal_conditions(col_a, col_c); + + let orderings = vec![ + vec![(col_a, options)], + vec![(col_e, options)], + vec![(col_d, options), (col_f, options)], + ]; + let orderings = convert_to_orderings(&orderings); + + // Column [a ASC], [e ASC], [d ASC, f ASC] are all valid orderings for the schema. + eq_properties.add_new_orderings(orderings); + + // First entry in the tuple is required ordering, second entry is the expected flag + // that indicates whether this required ordering is satisfied. + // ([a ASC], true) indicate a ASC requirement is already satisfied by existing orderings. + let test_cases = vec![ + // [c ASC, a ASC, e ASC], expected represents this requirement is satisfied + ( + vec![(col_c, options), (col_a, options), (col_e, options)], + true, + ), + (vec![(col_c, options), (col_b, options)], false), + (vec![(col_c, options), (col_d, options)], true), + ( + vec![(col_d, options), (col_f, options), (col_b, options)], + false, + ), + (vec![(col_d, options), (col_f, options)], true), + ]; + + for (reqs, expected) in test_cases { + let err_msg = + format!("error in test reqs: {:?}, expected: {:?}", reqs, expected,); + let reqs = convert_to_sort_exprs(&reqs); + assert_eq!( + eq_properties.ordering_satisfy(&reqs), + expected, + "{}", + err_msg + ); + } + + Ok(()) + } + + #[test] + fn test_remove_redundant_entries_oeq_class() -> Result<()> { + let schema = create_test_schema()?; + let col_a = &col("a", &schema)?; + let col_b = &col("b", &schema)?; + let col_c = &col("c", &schema)?; + let col_d = &col("d", &schema)?; + let col_e = &col("e", &schema)?; + + let option_asc = SortOptions { + descending: false, + nulls_first: false, + }; + let option_desc = SortOptions { + descending: true, + nulls_first: true, + }; + + // First entry in the tuple is the given orderings for the table + // Second entry is the simplest version of the given orderings that is functionally equivalent. + let test_cases = vec![ + // ------- TEST CASE 1 --------- + ( + // ORDERINGS GIVEN + vec![ + // [a ASC, b ASC] + vec![(col_a, option_asc), (col_b, option_asc)], + ], + // EXPECTED orderings that is succinct. + vec![ + // [a ASC, b ASC] + vec![(col_a, option_asc), (col_b, option_asc)], + ], + ), + // ------- TEST CASE 2 --------- + ( + // ORDERINGS GIVEN + vec![ + // [a ASC, b ASC] + vec![(col_a, option_asc), (col_b, option_asc)], + // [a ASC, b ASC, c ASC] + vec![ + (col_a, option_asc), + (col_b, option_asc), + (col_c, option_asc), + ], + ], + // EXPECTED orderings that is succinct. + vec![ + // [a ASC, b ASC, c ASC] + vec![ + (col_a, option_asc), + (col_b, option_asc), + (col_c, option_asc), + ], + ], + ), + // ------- TEST CASE 3 --------- + ( + // ORDERINGS GIVEN + vec![ + // [a ASC, b DESC] + vec![(col_a, option_asc), (col_b, option_desc)], + // [a ASC] + vec![(col_a, option_asc)], + // [a ASC, c ASC] + vec![(col_a, option_asc), (col_c, option_asc)], + ], + // EXPECTED orderings that is succinct. + vec![ + // [a ASC, b DESC] + vec![(col_a, option_asc), (col_b, option_desc)], + // [a ASC, c ASC] + vec![(col_a, option_asc), (col_c, option_asc)], + ], + ), + // ------- TEST CASE 4 --------- + ( + // ORDERINGS GIVEN + vec![ + // [a ASC, b ASC] + vec![(col_a, option_asc), (col_b, option_asc)], + // [a ASC, b ASC, c ASC] + vec![ + (col_a, option_asc), + (col_b, option_asc), + (col_c, option_asc), + ], + // [a ASC] + vec![(col_a, option_asc)], + ], + // EXPECTED orderings that is succinct. + vec![ + // [a ASC, b ASC, c ASC] + vec![ + (col_a, option_asc), + (col_b, option_asc), + (col_c, option_asc), + ], + ], + ), + // ------- TEST CASE 5 --------- + // Empty ordering + ( + vec![vec![]], + // No ordering in the state (empty ordering is ignored). + vec![], + ), + // ------- TEST CASE 6 --------- + ( + // ORDERINGS GIVEN + vec![ + // [a ASC, b ASC] + vec![(col_a, option_asc), (col_b, option_asc)], + // [b ASC] + vec![(col_b, option_asc)], + ], + // EXPECTED orderings that is succinct. + vec![ + // [a ASC] + vec![(col_a, option_asc)], + // [b ASC] + vec![(col_b, option_asc)], + ], + ), + // ------- TEST CASE 7 --------- + // b, a + // c, a + // d, b, c + ( + // ORDERINGS GIVEN + vec![ + // [b ASC, a ASC] + vec![(col_b, option_asc), (col_a, option_asc)], + // [c ASC, a ASC] + vec![(col_c, option_asc), (col_a, option_asc)], + // [d ASC, b ASC, c ASC] + vec![ + (col_d, option_asc), + (col_b, option_asc), + (col_c, option_asc), + ], + ], + // EXPECTED orderings that is succinct. + vec![ + // [b ASC, a ASC] + vec![(col_b, option_asc), (col_a, option_asc)], + // [c ASC, a ASC] + vec![(col_c, option_asc), (col_a, option_asc)], + // [d ASC] + vec![(col_d, option_asc)], + ], + ), + // ------- TEST CASE 8 --------- + // b, e + // c, a + // d, b, e, c, a + ( + // ORDERINGS GIVEN + vec![ + // [b ASC, e ASC] + vec![(col_b, option_asc), (col_e, option_asc)], + // [c ASC, a ASC] + vec![(col_c, option_asc), (col_a, option_asc)], + // [d ASC, b ASC, e ASC, c ASC, a ASC] + vec![ + (col_d, option_asc), + (col_b, option_asc), + (col_e, option_asc), + (col_c, option_asc), + (col_a, option_asc), + ], + ], + // EXPECTED orderings that is succinct. + vec![ + // [b ASC, e ASC] + vec![(col_b, option_asc), (col_e, option_asc)], + // [c ASC, a ASC] + vec![(col_c, option_asc), (col_a, option_asc)], + // [d ASC] + vec![(col_d, option_asc)], + ], + ), + // ------- TEST CASE 9 --------- + // b + // a, b, c + // d, a, b + ( + // ORDERINGS GIVEN + vec![ + // [b ASC] + vec![(col_b, option_asc)], + // [a ASC, b ASC, c ASC] + vec![ + (col_a, option_asc), + (col_b, option_asc), + (col_c, option_asc), + ], + // [d ASC, a ASC, b ASC] + vec![ + (col_d, option_asc), + (col_a, option_asc), + (col_b, option_asc), + ], + ], + // EXPECTED orderings that is succinct. + vec![ + // [b ASC] + vec![(col_b, option_asc)], + // [a ASC, b ASC, c ASC] + vec![ + (col_a, option_asc), + (col_b, option_asc), + (col_c, option_asc), + ], + // [d ASC] + vec![(col_d, option_asc)], + ], + ), + ]; + for (orderings, expected) in test_cases { + let orderings = convert_to_orderings(&orderings); + let expected = convert_to_orderings(&expected); + let actual = OrderingEquivalenceClass::new(orderings.clone()); + let actual = actual.orderings; + let err_msg = format!( + "orderings: {:?}, expected: {:?}, actual :{:?}", + orderings, expected, actual + ); + assert_eq!(actual.len(), expected.len(), "{}", err_msg); + for elem in actual { + assert!(expected.contains(&elem), "{}", err_msg); + } + } + + Ok(()) + } +} diff --git a/datafusion/physical-expr/src/equivalence/projection.rs b/datafusion/physical-expr/src/equivalence/projection.rs new file mode 100644 index 000000000000..0f92b2c2f431 --- /dev/null +++ b/datafusion/physical-expr/src/equivalence/projection.rs @@ -0,0 +1,1153 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +use std::sync::Arc; + +use crate::expressions::Column; +use crate::PhysicalExpr; + +use arrow::datatypes::SchemaRef; +use datafusion_common::tree_node::{Transformed, TreeNode}; +use datafusion_common::Result; + +/// Stores the mapping between source expressions and target expressions for a +/// projection. +#[derive(Debug, Clone)] +pub struct ProjectionMapping { + /// Mapping between source expressions and target expressions. + /// Vector indices correspond to the indices after projection. + pub map: Vec<(Arc, Arc)>, +} + +impl ProjectionMapping { + /// Constructs the mapping between a projection's input and output + /// expressions. + /// + /// For example, given the input projection expressions (`a + b`, `c + d`) + /// and an output schema with two columns `"c + d"` and `"a + b"`, the + /// projection mapping would be: + /// + /// ```text + /// [0]: (c + d, col("c + d")) + /// [1]: (a + b, col("a + b")) + /// ``` + /// + /// where `col("c + d")` means the column named `"c + d"`. + pub fn try_new( + expr: &[(Arc, String)], + input_schema: &SchemaRef, + ) -> Result { + // Construct a map from the input expressions to the output expression of the projection: + expr.iter() + .enumerate() + .map(|(expr_idx, (expression, name))| { + let target_expr = Arc::new(Column::new(name, expr_idx)) as _; + expression + .clone() + .transform_down(&|e| match e.as_any().downcast_ref::() { + Some(col) => { + // Sometimes, an expression and its name in the input_schema + // doesn't match. This can cause problems, so we make sure + // that the expression name matches with the name in `input_schema`. + // Conceptually, `source_expr` and `expression` should be the same. + let idx = col.index(); + let matching_input_field = input_schema.field(idx); + let matching_input_column = + Column::new(matching_input_field.name(), idx); + Ok(Transformed::Yes(Arc::new(matching_input_column))) + } + None => Ok(Transformed::No(e)), + }) + .map(|source_expr| (source_expr, target_expr)) + }) + .collect::>>() + .map(|map| Self { map }) + } + + /// Iterate over pairs of (source, target) expressions + pub fn iter( + &self, + ) -> impl Iterator, Arc)> + '_ { + self.map.iter() + } + + /// This function returns the target expression for a given source expression. + /// + /// # Arguments + /// + /// * `expr` - Source physical expression. + /// + /// # Returns + /// + /// An `Option` containing the target for the given source expression, + /// where a `None` value means that `expr` is not inside the mapping. + pub fn target_expr( + &self, + expr: &Arc, + ) -> Option> { + self.map + .iter() + .find(|(source, _)| source.eq(expr)) + .map(|(_, target)| target.clone()) + } +} + +#[cfg(test)] +mod tests { + use super::*; + use crate::equivalence::tests::{ + apply_projection, convert_to_orderings, convert_to_orderings_owned, + create_random_schema, generate_table_for_eq_properties, is_table_same_after_sort, + output_schema, + }; + use crate::equivalence::EquivalenceProperties; + use crate::execution_props::ExecutionProps; + use crate::expressions::{col, BinaryExpr, Literal}; + use crate::functions::create_physical_expr; + use crate::PhysicalSortExpr; + use arrow::datatypes::{DataType, Field, Schema}; + use arrow_schema::{SortOptions, TimeUnit}; + use datafusion_common::{Result, ScalarValue}; + use datafusion_expr::{BuiltinScalarFunction, Operator}; + use itertools::Itertools; + use std::sync::Arc; + + #[test] + fn project_orderings() -> Result<()> { + let schema = Arc::new(Schema::new(vec![ + Field::new("a", DataType::Int32, true), + Field::new("b", DataType::Int32, true), + Field::new("c", DataType::Int32, true), + Field::new("d", DataType::Int32, true), + Field::new("e", DataType::Int32, true), + Field::new("ts", DataType::Timestamp(TimeUnit::Nanosecond, None), true), + ])); + let col_a = &col("a", &schema)?; + let col_b = &col("b", &schema)?; + let col_c = &col("c", &schema)?; + let col_d = &col("d", &schema)?; + let col_e = &col("e", &schema)?; + let col_ts = &col("ts", &schema)?; + let interval = Arc::new(Literal::new(ScalarValue::IntervalDayTime(Some(2)))) + as Arc; + let date_bin_func = &create_physical_expr( + &BuiltinScalarFunction::DateBin, + &[interval, col_ts.clone()], + &schema, + &ExecutionProps::default(), + )?; + let a_plus_b = Arc::new(BinaryExpr::new( + col_a.clone(), + Operator::Plus, + col_b.clone(), + )) as Arc; + let b_plus_d = Arc::new(BinaryExpr::new( + col_b.clone(), + Operator::Plus, + col_d.clone(), + )) as Arc; + let b_plus_e = Arc::new(BinaryExpr::new( + col_b.clone(), + Operator::Plus, + col_e.clone(), + )) as Arc; + let c_plus_d = Arc::new(BinaryExpr::new( + col_c.clone(), + Operator::Plus, + col_d.clone(), + )) as Arc; + + let option_asc = SortOptions { + descending: false, + nulls_first: false, + }; + let option_desc = SortOptions { + descending: true, + nulls_first: true, + }; + + let test_cases = vec![ + // ---------- TEST CASE 1 ------------ + ( + // orderings + vec![ + // [b ASC] + vec![(col_b, option_asc)], + ], + // projection exprs + vec![(col_b, "b_new".to_string()), (col_a, "a_new".to_string())], + // expected + vec![ + // [b_new ASC] + vec![("b_new", option_asc)], + ], + ), + // ---------- TEST CASE 2 ------------ + ( + // orderings + vec![ + // empty ordering + ], + // projection exprs + vec![(col_c, "c_new".to_string()), (col_b, "b_new".to_string())], + // expected + vec![ + // no ordering at the output + ], + ), + // ---------- TEST CASE 3 ------------ + ( + // orderings + vec![ + // [ts ASC] + vec![(col_ts, option_asc)], + ], + // projection exprs + vec![ + (col_b, "b_new".to_string()), + (col_a, "a_new".to_string()), + (col_ts, "ts_new".to_string()), + (date_bin_func, "date_bin_res".to_string()), + ], + // expected + vec![ + // [date_bin_res ASC] + vec![("date_bin_res", option_asc)], + // [ts_new ASC] + vec![("ts_new", option_asc)], + ], + ), + // ---------- TEST CASE 4 ------------ + ( + // orderings + vec![ + // [a ASC, ts ASC] + vec![(col_a, option_asc), (col_ts, option_asc)], + // [b ASC, ts ASC] + vec![(col_b, option_asc), (col_ts, option_asc)], + ], + // projection exprs + vec![ + (col_b, "b_new".to_string()), + (col_a, "a_new".to_string()), + (col_ts, "ts_new".to_string()), + (date_bin_func, "date_bin_res".to_string()), + ], + // expected + vec![ + // [a_new ASC, ts_new ASC] + vec![("a_new", option_asc), ("ts_new", option_asc)], + // [a_new ASC, date_bin_res ASC] + vec![("a_new", option_asc), ("date_bin_res", option_asc)], + // [b_new ASC, ts_new ASC] + vec![("b_new", option_asc), ("ts_new", option_asc)], + // [b_new ASC, date_bin_res ASC] + vec![("b_new", option_asc), ("date_bin_res", option_asc)], + ], + ), + // ---------- TEST CASE 5 ------------ + ( + // orderings + vec![ + // [a + b ASC] + vec![(&a_plus_b, option_asc)], + ], + // projection exprs + vec![ + (col_b, "b_new".to_string()), + (col_a, "a_new".to_string()), + (&a_plus_b, "a+b".to_string()), + ], + // expected + vec![ + // [a + b ASC] + vec![("a+b", option_asc)], + ], + ), + // ---------- TEST CASE 6 ------------ + ( + // orderings + vec![ + // [a + b ASC, c ASC] + vec![(&a_plus_b, option_asc), (&col_c, option_asc)], + ], + // projection exprs + vec![ + (col_b, "b_new".to_string()), + (col_a, "a_new".to_string()), + (col_c, "c_new".to_string()), + (&a_plus_b, "a+b".to_string()), + ], + // expected + vec![ + // [a + b ASC, c_new ASC] + vec![("a+b", option_asc), ("c_new", option_asc)], + ], + ), + // ------- TEST CASE 7 ---------- + ( + vec![ + // [a ASC, b ASC, c ASC] + vec![(col_a, option_asc), (col_b, option_asc)], + // [a ASC, d ASC] + vec![(col_a, option_asc), (col_d, option_asc)], + ], + // b as b_new, a as a_new, d as d_new b+d + vec![ + (col_b, "b_new".to_string()), + (col_a, "a_new".to_string()), + (col_d, "d_new".to_string()), + (&b_plus_d, "b+d".to_string()), + ], + // expected + vec![ + // [a_new ASC, b_new ASC] + vec![("a_new", option_asc), ("b_new", option_asc)], + // [a_new ASC, d_new ASC] + vec![("a_new", option_asc), ("d_new", option_asc)], + // [a_new ASC, b+d ASC] + vec![("a_new", option_asc), ("b+d", option_asc)], + ], + ), + // ------- TEST CASE 8 ---------- + ( + // orderings + vec![ + // [b+d ASC] + vec![(&b_plus_d, option_asc)], + ], + // proj exprs + vec![ + (col_b, "b_new".to_string()), + (col_a, "a_new".to_string()), + (col_d, "d_new".to_string()), + (&b_plus_d, "b+d".to_string()), + ], + // expected + vec![ + // [b+d ASC] + vec![("b+d", option_asc)], + ], + ), + // ------- TEST CASE 9 ---------- + ( + // orderings + vec![ + // [a ASC, d ASC, b ASC] + vec![ + (col_a, option_asc), + (col_d, option_asc), + (col_b, option_asc), + ], + // [c ASC] + vec![(col_c, option_asc)], + ], + // proj exprs + vec![ + (col_b, "b_new".to_string()), + (col_a, "a_new".to_string()), + (col_d, "d_new".to_string()), + (col_c, "c_new".to_string()), + ], + // expected + vec![ + // [a_new ASC, d_new ASC, b_new ASC] + vec![ + ("a_new", option_asc), + ("d_new", option_asc), + ("b_new", option_asc), + ], + // [c_new ASC], + vec![("c_new", option_asc)], + ], + ), + // ------- TEST CASE 10 ---------- + ( + vec![ + // [a ASC, b ASC, c ASC] + vec![ + (col_a, option_asc), + (col_b, option_asc), + (col_c, option_asc), + ], + // [a ASC, d ASC] + vec![(col_a, option_asc), (col_d, option_asc)], + ], + // proj exprs + vec![ + (col_b, "b_new".to_string()), + (col_a, "a_new".to_string()), + (col_c, "c_new".to_string()), + (&c_plus_d, "c+d".to_string()), + ], + // expected + vec![ + // [a_new ASC, b_new ASC, c_new ASC] + vec![ + ("a_new", option_asc), + ("b_new", option_asc), + ("c_new", option_asc), + ], + // [a_new ASC, b_new ASC, c+d ASC] + vec![ + ("a_new", option_asc), + ("b_new", option_asc), + ("c+d", option_asc), + ], + ], + ), + // ------- TEST CASE 11 ---------- + ( + // orderings + vec![ + // [a ASC, b ASC] + vec![(col_a, option_asc), (col_b, option_asc)], + // [a ASC, d ASC] + vec![(col_a, option_asc), (col_d, option_asc)], + ], + // proj exprs + vec![ + (col_b, "b_new".to_string()), + (col_a, "a_new".to_string()), + (&b_plus_d, "b+d".to_string()), + ], + // expected + vec![ + // [a_new ASC, b_new ASC] + vec![("a_new", option_asc), ("b_new", option_asc)], + // [a_new ASC, b + d ASC] + vec![("a_new", option_asc), ("b+d", option_asc)], + ], + ), + // ------- TEST CASE 12 ---------- + ( + // orderings + vec![ + // [a ASC, b ASC, c ASC] + vec![ + (col_a, option_asc), + (col_b, option_asc), + (col_c, option_asc), + ], + ], + // proj exprs + vec![(col_c, "c_new".to_string()), (col_a, "a_new".to_string())], + // expected + vec![ + // [a_new ASC] + vec![("a_new", option_asc)], + ], + ), + // ------- TEST CASE 13 ---------- + ( + // orderings + vec![ + // [a ASC, b ASC, c ASC] + vec![ + (col_a, option_asc), + (col_b, option_asc), + (col_c, option_asc), + ], + // [a ASC, a + b ASC, c ASC] + vec![ + (col_a, option_asc), + (&a_plus_b, option_asc), + (col_c, option_asc), + ], + ], + // proj exprs + vec![ + (col_c, "c_new".to_string()), + (col_b, "b_new".to_string()), + (col_a, "a_new".to_string()), + (&a_plus_b, "a+b".to_string()), + ], + // expected + vec![ + // [a_new ASC, b_new ASC, c_new ASC] + vec![ + ("a_new", option_asc), + ("b_new", option_asc), + ("c_new", option_asc), + ], + // [a_new ASC, a+b ASC, c_new ASC] + vec![ + ("a_new", option_asc), + ("a+b", option_asc), + ("c_new", option_asc), + ], + ], + ), + // ------- TEST CASE 14 ---------- + ( + // orderings + vec![ + // [a ASC, b ASC] + vec![(col_a, option_asc), (col_b, option_asc)], + // [c ASC, b ASC] + vec![(col_c, option_asc), (col_b, option_asc)], + // [d ASC, e ASC] + vec![(col_d, option_asc), (col_e, option_asc)], + ], + // proj exprs + vec![ + (col_c, "c_new".to_string()), + (col_d, "d_new".to_string()), + (col_a, "a_new".to_string()), + (&b_plus_e, "b+e".to_string()), + ], + // expected + vec![ + // [a_new ASC, d_new ASC, b+e ASC] + vec![ + ("a_new", option_asc), + ("d_new", option_asc), + ("b+e", option_asc), + ], + // [d_new ASC, a_new ASC, b+e ASC] + vec![ + ("d_new", option_asc), + ("a_new", option_asc), + ("b+e", option_asc), + ], + // [c_new ASC, d_new ASC, b+e ASC] + vec![ + ("c_new", option_asc), + ("d_new", option_asc), + ("b+e", option_asc), + ], + // [d_new ASC, c_new ASC, b+e ASC] + vec![ + ("d_new", option_asc), + ("c_new", option_asc), + ("b+e", option_asc), + ], + ], + ), + // ------- TEST CASE 15 ---------- + ( + // orderings + vec![ + // [a ASC, c ASC, b ASC] + vec![ + (col_a, option_asc), + (col_c, option_asc), + (&col_b, option_asc), + ], + ], + // proj exprs + vec![ + (col_c, "c_new".to_string()), + (col_a, "a_new".to_string()), + (&a_plus_b, "a+b".to_string()), + ], + // expected + vec![ + // [a_new ASC, d_new ASC, b+e ASC] + vec![ + ("a_new", option_asc), + ("c_new", option_asc), + ("a+b", option_asc), + ], + ], + ), + // ------- TEST CASE 16 ---------- + ( + // orderings + vec![ + // [a ASC, b ASC] + vec![(col_a, option_asc), (col_b, option_asc)], + // [c ASC, b DESC] + vec![(col_c, option_asc), (col_b, option_desc)], + // [e ASC] + vec![(col_e, option_asc)], + ], + // proj exprs + vec![ + (col_c, "c_new".to_string()), + (col_a, "a_new".to_string()), + (col_b, "b_new".to_string()), + (&b_plus_e, "b+e".to_string()), + ], + // expected + vec![ + // [a_new ASC, b_new ASC] + vec![("a_new", option_asc), ("b_new", option_asc)], + // [a_new ASC, b_new ASC] + vec![("a_new", option_asc), ("b+e", option_asc)], + // [c_new ASC, b_new DESC] + vec![("c_new", option_asc), ("b_new", option_desc)], + ], + ), + ]; + + for (idx, (orderings, proj_exprs, expected)) in test_cases.into_iter().enumerate() + { + let mut eq_properties = EquivalenceProperties::new(schema.clone()); + + let orderings = convert_to_orderings(&orderings); + eq_properties.add_new_orderings(orderings); + + let proj_exprs = proj_exprs + .into_iter() + .map(|(expr, name)| (expr.clone(), name)) + .collect::>(); + let projection_mapping = ProjectionMapping::try_new(&proj_exprs, &schema)?; + let output_schema = output_schema(&projection_mapping, &schema)?; + + let expected = expected + .into_iter() + .map(|ordering| { + ordering + .into_iter() + .map(|(name, options)| { + (col(name, &output_schema).unwrap(), options) + }) + .collect::>() + }) + .collect::>(); + let expected = convert_to_orderings_owned(&expected); + + let projected_eq = eq_properties.project(&projection_mapping, output_schema); + let orderings = projected_eq.oeq_class(); + + let err_msg = format!( + "test_idx: {:?}, actual: {:?}, expected: {:?}, projection_mapping: {:?}", + idx, orderings.orderings, expected, projection_mapping + ); + + assert_eq!(orderings.len(), expected.len(), "{}", err_msg); + for expected_ordering in &expected { + assert!(orderings.contains(expected_ordering), "{}", err_msg) + } + } + + Ok(()) + } + + #[test] + fn project_orderings2() -> Result<()> { + let schema = Arc::new(Schema::new(vec![ + Field::new("a", DataType::Int32, true), + Field::new("b", DataType::Int32, true), + Field::new("c", DataType::Int32, true), + Field::new("d", DataType::Int32, true), + Field::new("ts", DataType::Timestamp(TimeUnit::Nanosecond, None), true), + ])); + let col_a = &col("a", &schema)?; + let col_b = &col("b", &schema)?; + let col_c = &col("c", &schema)?; + let col_ts = &col("ts", &schema)?; + let a_plus_b = Arc::new(BinaryExpr::new( + col_a.clone(), + Operator::Plus, + col_b.clone(), + )) as Arc; + let interval = Arc::new(Literal::new(ScalarValue::IntervalDayTime(Some(2)))) + as Arc; + let date_bin_ts = &create_physical_expr( + &BuiltinScalarFunction::DateBin, + &[interval, col_ts.clone()], + &schema, + &ExecutionProps::default(), + )?; + + let round_c = &create_physical_expr( + &BuiltinScalarFunction::Round, + &[col_c.clone()], + &schema, + &ExecutionProps::default(), + )?; + + let option_asc = SortOptions { + descending: false, + nulls_first: false, + }; + + let proj_exprs = vec![ + (col_b, "b_new".to_string()), + (col_a, "a_new".to_string()), + (col_c, "c_new".to_string()), + (date_bin_ts, "date_bin_res".to_string()), + (round_c, "round_c_res".to_string()), + ]; + let proj_exprs = proj_exprs + .into_iter() + .map(|(expr, name)| (expr.clone(), name)) + .collect::>(); + let projection_mapping = ProjectionMapping::try_new(&proj_exprs, &schema)?; + let output_schema = output_schema(&projection_mapping, &schema)?; + + let col_a_new = &col("a_new", &output_schema)?; + let col_b_new = &col("b_new", &output_schema)?; + let col_c_new = &col("c_new", &output_schema)?; + let col_date_bin_res = &col("date_bin_res", &output_schema)?; + let col_round_c_res = &col("round_c_res", &output_schema)?; + let a_new_plus_b_new = Arc::new(BinaryExpr::new( + col_a_new.clone(), + Operator::Plus, + col_b_new.clone(), + )) as Arc; + + let test_cases = vec![ + // ---------- TEST CASE 1 ------------ + ( + // orderings + vec![ + // [a ASC] + vec![(col_a, option_asc)], + ], + // expected + vec![ + // [b_new ASC] + vec![(col_a_new, option_asc)], + ], + ), + // ---------- TEST CASE 2 ------------ + ( + // orderings + vec![ + // [a+b ASC] + vec![(&a_plus_b, option_asc)], + ], + // expected + vec![ + // [b_new ASC] + vec![(&a_new_plus_b_new, option_asc)], + ], + ), + // ---------- TEST CASE 3 ------------ + ( + // orderings + vec![ + // [a ASC, ts ASC] + vec![(col_a, option_asc), (col_ts, option_asc)], + ], + // expected + vec![ + // [a_new ASC, date_bin_res ASC] + vec![(col_a_new, option_asc), (col_date_bin_res, option_asc)], + ], + ), + // ---------- TEST CASE 4 ------------ + ( + // orderings + vec![ + // [a ASC, ts ASC, b ASC] + vec![ + (col_a, option_asc), + (col_ts, option_asc), + (col_b, option_asc), + ], + ], + // expected + vec![ + // [a_new ASC, date_bin_res ASC] + // Please note that result is not [a_new ASC, date_bin_res ASC, b_new ASC] + // because, datebin_res may not be 1-1 function. Hence without introducing ts + // dependency we cannot guarantee any ordering after date_bin_res column. + vec![(col_a_new, option_asc), (col_date_bin_res, option_asc)], + ], + ), + // ---------- TEST CASE 5 ------------ + ( + // orderings + vec![ + // [a ASC, c ASC] + vec![(col_a, option_asc), (col_c, option_asc)], + ], + // expected + vec![ + // [a_new ASC, round_c_res ASC, c_new ASC] + vec![(col_a_new, option_asc), (col_round_c_res, option_asc)], + // [a_new ASC, c_new ASC] + vec![(col_a_new, option_asc), (col_c_new, option_asc)], + ], + ), + // ---------- TEST CASE 6 ------------ + ( + // orderings + vec![ + // [c ASC, b ASC] + vec![(col_c, option_asc), (col_b, option_asc)], + ], + // expected + vec![ + // [round_c_res ASC] + vec![(col_round_c_res, option_asc)], + // [c_new ASC, b_new ASC] + vec![(col_c_new, option_asc), (col_b_new, option_asc)], + ], + ), + // ---------- TEST CASE 7 ------------ + ( + // orderings + vec![ + // [a+b ASC, c ASC] + vec![(&a_plus_b, option_asc), (col_c, option_asc)], + ], + // expected + vec![ + // [a+b ASC, round(c) ASC, c_new ASC] + vec![ + (&a_new_plus_b_new, option_asc), + (&col_round_c_res, option_asc), + ], + // [a+b ASC, c_new ASC] + vec![(&a_new_plus_b_new, option_asc), (col_c_new, option_asc)], + ], + ), + ]; + + for (idx, (orderings, expected)) in test_cases.iter().enumerate() { + let mut eq_properties = EquivalenceProperties::new(schema.clone()); + + let orderings = convert_to_orderings(orderings); + eq_properties.add_new_orderings(orderings); + + let expected = convert_to_orderings(expected); + + let projected_eq = + eq_properties.project(&projection_mapping, output_schema.clone()); + let orderings = projected_eq.oeq_class(); + + let err_msg = format!( + "test idx: {:?}, actual: {:?}, expected: {:?}, projection_mapping: {:?}", + idx, orderings.orderings, expected, projection_mapping + ); + + assert_eq!(orderings.len(), expected.len(), "{}", err_msg); + for expected_ordering in &expected { + assert!(orderings.contains(expected_ordering), "{}", err_msg) + } + } + Ok(()) + } + + #[test] + fn project_orderings3() -> Result<()> { + let schema = Arc::new(Schema::new(vec![ + Field::new("a", DataType::Int32, true), + Field::new("b", DataType::Int32, true), + Field::new("c", DataType::Int32, true), + Field::new("d", DataType::Int32, true), + Field::new("e", DataType::Int32, true), + Field::new("f", DataType::Int32, true), + ])); + let col_a = &col("a", &schema)?; + let col_b = &col("b", &schema)?; + let col_c = &col("c", &schema)?; + let col_d = &col("d", &schema)?; + let col_e = &col("e", &schema)?; + let col_f = &col("f", &schema)?; + let a_plus_b = Arc::new(BinaryExpr::new( + col_a.clone(), + Operator::Plus, + col_b.clone(), + )) as Arc; + + let option_asc = SortOptions { + descending: false, + nulls_first: false, + }; + + let proj_exprs = vec![ + (col_c, "c_new".to_string()), + (col_d, "d_new".to_string()), + (&a_plus_b, "a+b".to_string()), + ]; + let proj_exprs = proj_exprs + .into_iter() + .map(|(expr, name)| (expr.clone(), name)) + .collect::>(); + let projection_mapping = ProjectionMapping::try_new(&proj_exprs, &schema)?; + let output_schema = output_schema(&projection_mapping, &schema)?; + + let col_a_plus_b_new = &col("a+b", &output_schema)?; + let col_c_new = &col("c_new", &output_schema)?; + let col_d_new = &col("d_new", &output_schema)?; + + let test_cases = vec![ + // ---------- TEST CASE 1 ------------ + ( + // orderings + vec![ + // [d ASC, b ASC] + vec![(col_d, option_asc), (col_b, option_asc)], + // [c ASC, a ASC] + vec![(col_c, option_asc), (col_a, option_asc)], + ], + // equal conditions + vec![], + // expected + vec![ + // [d_new ASC, c_new ASC, a+b ASC] + vec![ + (col_d_new, option_asc), + (col_c_new, option_asc), + (col_a_plus_b_new, option_asc), + ], + // [c_new ASC, d_new ASC, a+b ASC] + vec![ + (col_c_new, option_asc), + (col_d_new, option_asc), + (col_a_plus_b_new, option_asc), + ], + ], + ), + // ---------- TEST CASE 2 ------------ + ( + // orderings + vec![ + // [d ASC, b ASC] + vec![(col_d, option_asc), (col_b, option_asc)], + // [c ASC, e ASC], Please note that a=e + vec![(col_c, option_asc), (col_e, option_asc)], + ], + // equal conditions + vec![(col_e, col_a)], + // expected + vec![ + // [d_new ASC, c_new ASC, a+b ASC] + vec![ + (col_d_new, option_asc), + (col_c_new, option_asc), + (col_a_plus_b_new, option_asc), + ], + // [c_new ASC, d_new ASC, a+b ASC] + vec![ + (col_c_new, option_asc), + (col_d_new, option_asc), + (col_a_plus_b_new, option_asc), + ], + ], + ), + // ---------- TEST CASE 3 ------------ + ( + // orderings + vec![ + // [d ASC, b ASC] + vec![(col_d, option_asc), (col_b, option_asc)], + // [c ASC, e ASC], Please note that a=f + vec![(col_c, option_asc), (col_e, option_asc)], + ], + // equal conditions + vec![(col_a, col_f)], + // expected + vec![ + // [d_new ASC] + vec![(col_d_new, option_asc)], + // [c_new ASC] + vec![(col_c_new, option_asc)], + ], + ), + ]; + for (orderings, equal_columns, expected) in test_cases { + let mut eq_properties = EquivalenceProperties::new(schema.clone()); + for (lhs, rhs) in equal_columns { + eq_properties.add_equal_conditions(lhs, rhs); + } + + let orderings = convert_to_orderings(&orderings); + eq_properties.add_new_orderings(orderings); + + let expected = convert_to_orderings(&expected); + + let projected_eq = + eq_properties.project(&projection_mapping, output_schema.clone()); + let orderings = projected_eq.oeq_class(); + + let err_msg = format!( + "actual: {:?}, expected: {:?}, projection_mapping: {:?}", + orderings.orderings, expected, projection_mapping + ); + + assert_eq!(orderings.len(), expected.len(), "{}", err_msg); + for expected_ordering in &expected { + assert!(orderings.contains(expected_ordering), "{}", err_msg) + } + } + + Ok(()) + } + + #[test] + fn project_orderings_random() -> Result<()> { + const N_RANDOM_SCHEMA: usize = 20; + const N_ELEMENTS: usize = 125; + const N_DISTINCT: usize = 5; + + for seed in 0..N_RANDOM_SCHEMA { + // Create a random schema with random properties + let (test_schema, eq_properties) = create_random_schema(seed as u64)?; + // Generate a data that satisfies properties given + let table_data_with_properties = + generate_table_for_eq_properties(&eq_properties, N_ELEMENTS, N_DISTINCT)?; + // Floor(a) + let floor_a = create_physical_expr( + &BuiltinScalarFunction::Floor, + &[col("a", &test_schema)?], + &test_schema, + &ExecutionProps::default(), + )?; + // a + b + let a_plus_b = Arc::new(BinaryExpr::new( + col("a", &test_schema)?, + Operator::Plus, + col("b", &test_schema)?, + )) as Arc; + let proj_exprs = vec![ + (col("a", &test_schema)?, "a_new"), + (col("b", &test_schema)?, "b_new"), + (col("c", &test_schema)?, "c_new"), + (col("d", &test_schema)?, "d_new"), + (col("e", &test_schema)?, "e_new"), + (col("f", &test_schema)?, "f_new"), + (floor_a, "floor(a)"), + (a_plus_b, "a+b"), + ]; + + for n_req in 0..=proj_exprs.len() { + for proj_exprs in proj_exprs.iter().combinations(n_req) { + let proj_exprs = proj_exprs + .into_iter() + .map(|(expr, name)| (expr.clone(), name.to_string())) + .collect::>(); + let (projected_batch, projected_eq) = apply_projection( + proj_exprs.clone(), + &table_data_with_properties, + &eq_properties, + )?; + + // Make sure each ordering after projection is valid. + for ordering in projected_eq.oeq_class().iter() { + let err_msg = format!( + "Error in test case ordering:{:?}, eq_properties.oeq_class: {:?}, eq_properties.eq_group: {:?}, eq_properties.constants: {:?}, proj_exprs: {:?}", + ordering, eq_properties.oeq_class, eq_properties.eq_group, eq_properties.constants, proj_exprs + ); + // Since ordered section satisfies schema, we expect + // that result will be same after sort (e.g sort was unnecessary). + assert!( + is_table_same_after_sort( + ordering.clone(), + projected_batch.clone(), + )?, + "{}", + err_msg + ); + } + } + } + } + + Ok(()) + } + + #[test] + fn ordering_satisfy_after_projection_random() -> Result<()> { + const N_RANDOM_SCHEMA: usize = 20; + const N_ELEMENTS: usize = 125; + const N_DISTINCT: usize = 5; + const SORT_OPTIONS: SortOptions = SortOptions { + descending: false, + nulls_first: false, + }; + + for seed in 0..N_RANDOM_SCHEMA { + // Create a random schema with random properties + let (test_schema, eq_properties) = create_random_schema(seed as u64)?; + // Generate a data that satisfies properties given + let table_data_with_properties = + generate_table_for_eq_properties(&eq_properties, N_ELEMENTS, N_DISTINCT)?; + // Floor(a) + let floor_a = create_physical_expr( + &BuiltinScalarFunction::Floor, + &[col("a", &test_schema)?], + &test_schema, + &ExecutionProps::default(), + )?; + // a + b + let a_plus_b = Arc::new(BinaryExpr::new( + col("a", &test_schema)?, + Operator::Plus, + col("b", &test_schema)?, + )) as Arc; + let proj_exprs = vec![ + (col("a", &test_schema)?, "a_new"), + (col("b", &test_schema)?, "b_new"), + (col("c", &test_schema)?, "c_new"), + (col("d", &test_schema)?, "d_new"), + (col("e", &test_schema)?, "e_new"), + (col("f", &test_schema)?, "f_new"), + (floor_a, "floor(a)"), + (a_plus_b, "a+b"), + ]; + + for n_req in 0..=proj_exprs.len() { + for proj_exprs in proj_exprs.iter().combinations(n_req) { + let proj_exprs = proj_exprs + .into_iter() + .map(|(expr, name)| (expr.clone(), name.to_string())) + .collect::>(); + let (projected_batch, projected_eq) = apply_projection( + proj_exprs.clone(), + &table_data_with_properties, + &eq_properties, + )?; + + let projection_mapping = + ProjectionMapping::try_new(&proj_exprs, &test_schema)?; + + let projected_exprs = projection_mapping + .iter() + .map(|(_source, target)| target.clone()) + .collect::>(); + + for n_req in 0..=projected_exprs.len() { + for exprs in projected_exprs.iter().combinations(n_req) { + let requirement = exprs + .into_iter() + .map(|expr| PhysicalSortExpr { + expr: expr.clone(), + options: SORT_OPTIONS, + }) + .collect::>(); + let expected = is_table_same_after_sort( + requirement.clone(), + projected_batch.clone(), + )?; + let err_msg = format!( + "Error in test case requirement:{:?}, expected: {:?}, eq_properties.oeq_class: {:?}, eq_properties.eq_group: {:?}, eq_properties.constants: {:?}, projected_eq.oeq_class: {:?}, projected_eq.eq_group: {:?}, projected_eq.constants: {:?}, projection_mapping: {:?}", + requirement, expected, eq_properties.oeq_class, eq_properties.eq_group, eq_properties.constants, projected_eq.oeq_class, projected_eq.eq_group, projected_eq.constants, projection_mapping + ); + // Check whether ordering_satisfy API result and + // experimental result matches. + assert_eq!( + projected_eq.ordering_satisfy(&requirement), + expected, + "{}", + err_msg + ); + } + } + } + } + } + + Ok(()) + } +} diff --git a/datafusion/physical-expr/src/equivalence/properties.rs b/datafusion/physical-expr/src/equivalence/properties.rs new file mode 100644 index 000000000000..31c1cf61193a --- /dev/null +++ b/datafusion/physical-expr/src/equivalence/properties.rs @@ -0,0 +1,2062 @@ +// 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 crate::expressions::Column; +use arrow_schema::SchemaRef; +use datafusion_common::{JoinSide, JoinType}; +use indexmap::IndexSet; +use itertools::Itertools; +use std::collections::{HashMap, HashSet}; +use std::hash::{Hash, Hasher}; +use std::sync::Arc; + +use crate::equivalence::{ + collapse_lex_req, EquivalenceGroup, OrderingEquivalenceClass, ProjectionMapping, +}; + +use crate::expressions::Literal; +use crate::sort_properties::{ExprOrdering, SortProperties}; +use crate::{ + physical_exprs_contains, LexOrdering, LexOrderingRef, LexRequirement, + LexRequirementRef, PhysicalExpr, PhysicalSortExpr, PhysicalSortRequirement, +}; +use datafusion_common::tree_node::{Transformed, TreeNode}; + +use super::ordering::collapse_lex_ordering; + +/// A `EquivalenceProperties` object stores useful information related to a schema. +/// Currently, it keeps track of: +/// - Equivalent expressions, e.g expressions that have same value. +/// - Valid sort expressions (orderings) for the schema. +/// - Constants expressions (e.g expressions that are known to have constant values). +/// +/// Consider table below: +/// +/// ```text +/// ┌-------┐ +/// | a | b | +/// |---|---| +/// | 1 | 9 | +/// | 2 | 8 | +/// | 3 | 7 | +/// | 5 | 5 | +/// └---┴---┘ +/// ``` +/// +/// where both `a ASC` and `b DESC` can describe the table ordering. With +/// `EquivalenceProperties`, we can keep track of these different valid sort +/// expressions and treat `a ASC` and `b DESC` on an equal footing. +/// +/// Similarly, consider the table below: +/// +/// ```text +/// ┌-------┐ +/// | a | b | +/// |---|---| +/// | 1 | 1 | +/// | 2 | 2 | +/// | 3 | 3 | +/// | 5 | 5 | +/// └---┴---┘ +/// ``` +/// +/// where columns `a` and `b` always have the same value. We keep track of such +/// equivalences inside this object. With this information, we can optimize +/// things like partitioning. For example, if the partition requirement is +/// `Hash(a)` and output partitioning is `Hash(b)`, then we can deduce that +/// the existing partitioning satisfies the requirement. +#[derive(Debug, Clone)] +pub struct EquivalenceProperties { + /// Collection of equivalence classes that store expressions with the same + /// value. + pub eq_group: EquivalenceGroup, + /// Equivalent sort expressions for this table. + pub oeq_class: OrderingEquivalenceClass, + /// Expressions whose values are constant throughout the table. + /// TODO: We do not need to track constants separately, they can be tracked + /// inside `eq_groups` as `Literal` expressions. + pub constants: Vec>, + /// Schema associated with this object. + schema: SchemaRef, +} + +impl EquivalenceProperties { + /// Creates an empty `EquivalenceProperties` object. + pub fn new(schema: SchemaRef) -> Self { + Self { + eq_group: EquivalenceGroup::empty(), + oeq_class: OrderingEquivalenceClass::empty(), + constants: vec![], + schema, + } + } + + /// Creates a new `EquivalenceProperties` object with the given orderings. + pub fn new_with_orderings(schema: SchemaRef, orderings: &[LexOrdering]) -> Self { + Self { + eq_group: EquivalenceGroup::empty(), + oeq_class: OrderingEquivalenceClass::new(orderings.to_vec()), + constants: vec![], + schema, + } + } + + /// Returns the associated schema. + pub fn schema(&self) -> &SchemaRef { + &self.schema + } + + /// Returns a reference to the ordering equivalence class within. + pub fn oeq_class(&self) -> &OrderingEquivalenceClass { + &self.oeq_class + } + + /// Returns a reference to the equivalence group within. + pub fn eq_group(&self) -> &EquivalenceGroup { + &self.eq_group + } + + /// Returns a reference to the constant expressions + pub fn constants(&self) -> &[Arc] { + &self.constants + } + + /// Returns the normalized version of the ordering equivalence class within. + /// Normalization removes constants and duplicates as well as standardizing + /// expressions according to the equivalence group within. + pub fn normalized_oeq_class(&self) -> OrderingEquivalenceClass { + OrderingEquivalenceClass::new( + self.oeq_class + .iter() + .map(|ordering| self.normalize_sort_exprs(ordering)) + .collect(), + ) + } + + /// Extends this `EquivalenceProperties` with the `other` object. + pub fn extend(mut self, other: Self) -> Self { + self.eq_group.extend(other.eq_group); + self.oeq_class.extend(other.oeq_class); + self.add_constants(other.constants) + } + + /// Clears (empties) the ordering equivalence class within this object. + /// Call this method when existing orderings are invalidated. + pub fn clear_orderings(&mut self) { + self.oeq_class.clear(); + } + + /// Extends this `EquivalenceProperties` by adding the orderings inside the + /// ordering equivalence class `other`. + pub fn add_ordering_equivalence_class(&mut self, other: OrderingEquivalenceClass) { + self.oeq_class.extend(other); + } + + /// Adds new orderings into the existing ordering equivalence class. + pub fn add_new_orderings( + &mut self, + orderings: impl IntoIterator, + ) { + self.oeq_class.add_new_orderings(orderings); + } + + /// Incorporates the given equivalence group to into the existing + /// equivalence group within. + pub fn add_equivalence_group(&mut self, other_eq_group: EquivalenceGroup) { + self.eq_group.extend(other_eq_group); + } + + /// Adds a new equality condition into the existing equivalence group. + /// If the given equality defines a new equivalence class, adds this new + /// equivalence class to the equivalence group. + pub fn add_equal_conditions( + &mut self, + left: &Arc, + right: &Arc, + ) { + self.eq_group.add_equal_conditions(left, right); + } + + /// Track/register physical expressions with constant values. + pub fn add_constants( + mut self, + constants: impl IntoIterator>, + ) -> Self { + for expr in self.eq_group.normalize_exprs(constants) { + if !physical_exprs_contains(&self.constants, &expr) { + self.constants.push(expr); + } + } + self + } + + /// Updates the ordering equivalence group within assuming that the table + /// is re-sorted according to the argument `sort_exprs`. Note that constants + /// and equivalence classes are unchanged as they are unaffected by a re-sort. + pub fn with_reorder(mut self, sort_exprs: Vec) -> Self { + // TODO: In some cases, existing ordering equivalences may still be valid add this analysis. + self.oeq_class = OrderingEquivalenceClass::new(vec![sort_exprs]); + self + } + + /// Normalizes the given sort expressions (i.e. `sort_exprs`) using the + /// equivalence group and the ordering equivalence class within. + /// + /// Assume that `self.eq_group` states column `a` and `b` are aliases. + /// Also assume that `self.oeq_class` states orderings `d ASC` and `a ASC, c ASC` + /// are equivalent (in the sense that both describe the ordering of the table). + /// If the `sort_exprs` argument were `vec![b ASC, c ASC, a ASC]`, then this + /// function would return `vec![a ASC, c ASC]`. Internally, it would first + /// normalize to `vec![a ASC, c ASC, a ASC]` and end up with the final result + /// after deduplication. + fn normalize_sort_exprs(&self, sort_exprs: LexOrderingRef) -> LexOrdering { + // Convert sort expressions to sort requirements: + let sort_reqs = PhysicalSortRequirement::from_sort_exprs(sort_exprs.iter()); + // Normalize the requirements: + let normalized_sort_reqs = self.normalize_sort_requirements(&sort_reqs); + // Convert sort requirements back to sort expressions: + PhysicalSortRequirement::to_sort_exprs(normalized_sort_reqs) + } + + /// Normalizes the given sort requirements (i.e. `sort_reqs`) using the + /// equivalence group and the ordering equivalence class within. It works by: + /// - Removing expressions that have a constant value from the given requirement. + /// - Replacing sections that belong to some equivalence class in the equivalence + /// group with the first entry in the matching equivalence class. + /// + /// Assume that `self.eq_group` states column `a` and `b` are aliases. + /// Also assume that `self.oeq_class` states orderings `d ASC` and `a ASC, c ASC` + /// are equivalent (in the sense that both describe the ordering of the table). + /// If the `sort_reqs` argument were `vec![b ASC, c ASC, a ASC]`, then this + /// function would return `vec![a ASC, c ASC]`. Internally, it would first + /// normalize to `vec![a ASC, c ASC, a ASC]` and end up with the final result + /// after deduplication. + fn normalize_sort_requirements( + &self, + sort_reqs: LexRequirementRef, + ) -> LexRequirement { + let normalized_sort_reqs = self.eq_group.normalize_sort_requirements(sort_reqs); + let constants_normalized = self.eq_group.normalize_exprs(self.constants.clone()); + // Prune redundant sections in the requirement: + collapse_lex_req( + normalized_sort_reqs + .iter() + .filter(|&order| { + !physical_exprs_contains(&constants_normalized, &order.expr) + }) + .cloned() + .collect(), + ) + } + + /// Checks whether the given ordering is satisfied by any of the existing + /// orderings. + pub fn ordering_satisfy(&self, given: LexOrderingRef) -> bool { + // Convert the given sort expressions to sort requirements: + let sort_requirements = PhysicalSortRequirement::from_sort_exprs(given.iter()); + self.ordering_satisfy_requirement(&sort_requirements) + } + + /// Checks whether the given sort requirements are satisfied by any of the + /// existing orderings. + pub fn ordering_satisfy_requirement(&self, reqs: LexRequirementRef) -> bool { + let mut eq_properties = self.clone(); + // First, standardize the given requirement: + let normalized_reqs = eq_properties.normalize_sort_requirements(reqs); + for normalized_req in normalized_reqs { + // Check whether given ordering is satisfied + if !eq_properties.ordering_satisfy_single(&normalized_req) { + return false; + } + // Treat satisfied keys as constants in subsequent iterations. We + // can do this because the "next" key only matters in a lexicographical + // ordering when the keys to its left have the same values. + // + // Note that these expressions are not properly "constants". This is just + // an implementation strategy confined to this function. + // + // For example, assume that the requirement is `[a ASC, (b + c) ASC]`, + // and existing equivalent orderings are `[a ASC, b ASC]` and `[c ASC]`. + // From the analysis above, we know that `[a ASC]` is satisfied. Then, + // we add column `a` as constant to the algorithm state. This enables us + // to deduce that `(b + c) ASC` is satisfied, given `a` is constant. + eq_properties = + eq_properties.add_constants(std::iter::once(normalized_req.expr)); + } + true + } + + /// Determines whether the ordering specified by the given sort requirement + /// is satisfied based on the orderings within, equivalence classes, and + /// constant expressions. + /// + /// # Arguments + /// + /// - `req`: A reference to a `PhysicalSortRequirement` for which the ordering + /// satisfaction check will be done. + /// + /// # Returns + /// + /// Returns `true` if the specified ordering is satisfied, `false` otherwise. + fn ordering_satisfy_single(&self, req: &PhysicalSortRequirement) -> bool { + let expr_ordering = self.get_expr_ordering(req.expr.clone()); + let ExprOrdering { expr, state, .. } = expr_ordering; + match state { + SortProperties::Ordered(options) => { + let sort_expr = PhysicalSortExpr { expr, options }; + sort_expr.satisfy(req, self.schema()) + } + // Singleton expressions satisfies any ordering. + SortProperties::Singleton => true, + SortProperties::Unordered => false, + } + } + + /// Checks whether the `given`` sort requirements are equal or more specific + /// than the `reference` sort requirements. + pub fn requirements_compatible( + &self, + given: LexRequirementRef, + reference: LexRequirementRef, + ) -> bool { + let normalized_given = self.normalize_sort_requirements(given); + let normalized_reference = self.normalize_sort_requirements(reference); + + (normalized_reference.len() <= normalized_given.len()) + && normalized_reference + .into_iter() + .zip(normalized_given) + .all(|(reference, given)| given.compatible(&reference)) + } + + /// Returns the finer ordering among the orderings `lhs` and `rhs`, breaking + /// any ties by choosing `lhs`. + /// + /// The finer ordering is the ordering that satisfies both of the orderings. + /// If the orderings are incomparable, returns `None`. + /// + /// For example, the finer ordering among `[a ASC]` and `[a ASC, b ASC]` is + /// the latter. + pub fn get_finer_ordering( + &self, + lhs: LexOrderingRef, + rhs: LexOrderingRef, + ) -> Option { + // Convert the given sort expressions to sort requirements: + let lhs = PhysicalSortRequirement::from_sort_exprs(lhs); + let rhs = PhysicalSortRequirement::from_sort_exprs(rhs); + let finer = self.get_finer_requirement(&lhs, &rhs); + // Convert the chosen sort requirements back to sort expressions: + finer.map(PhysicalSortRequirement::to_sort_exprs) + } + + /// Returns the finer ordering among the requirements `lhs` and `rhs`, + /// breaking any ties by choosing `lhs`. + /// + /// The finer requirements are the ones that satisfy both of the given + /// requirements. If the requirements are incomparable, returns `None`. + /// + /// For example, the finer requirements among `[a ASC]` and `[a ASC, b ASC]` + /// is the latter. + pub fn get_finer_requirement( + &self, + req1: LexRequirementRef, + req2: LexRequirementRef, + ) -> Option { + let mut lhs = self.normalize_sort_requirements(req1); + let mut rhs = self.normalize_sort_requirements(req2); + lhs.iter_mut() + .zip(rhs.iter_mut()) + .all(|(lhs, rhs)| { + lhs.expr.eq(&rhs.expr) + && match (lhs.options, rhs.options) { + (Some(lhs_opt), Some(rhs_opt)) => lhs_opt == rhs_opt, + (Some(options), None) => { + rhs.options = Some(options); + true + } + (None, Some(options)) => { + lhs.options = Some(options); + true + } + (None, None) => true, + } + }) + .then_some(if lhs.len() >= rhs.len() { lhs } else { rhs }) + } + + /// Calculates the "meet" of the given orderings (`lhs` and `rhs`). + /// The meet of a set of orderings is the finest ordering that is satisfied + /// by all the orderings in that set. For details, see: + /// + /// + /// + /// If there is no ordering that satisfies both `lhs` and `rhs`, returns + /// `None`. As an example, the meet of orderings `[a ASC]` and `[a ASC, b ASC]` + /// is `[a ASC]`. + pub fn get_meet_ordering( + &self, + lhs: LexOrderingRef, + rhs: LexOrderingRef, + ) -> Option { + let lhs = self.normalize_sort_exprs(lhs); + let rhs = self.normalize_sort_exprs(rhs); + let mut meet = vec![]; + for (lhs, rhs) in lhs.into_iter().zip(rhs.into_iter()) { + if lhs.eq(&rhs) { + meet.push(lhs); + } else { + break; + } + } + (!meet.is_empty()).then_some(meet) + } + + /// Projects argument `expr` according to `projection_mapping`, taking + /// equivalences into account. + /// + /// For example, assume that columns `a` and `c` are always equal, and that + /// `projection_mapping` encodes following mapping: + /// + /// ```text + /// a -> a1 + /// b -> b1 + /// ``` + /// + /// Then, this function projects `a + b` to `Some(a1 + b1)`, `c + b` to + /// `Some(a1 + b1)` and `d` to `None`, meaning that it cannot be projected. + pub fn project_expr( + &self, + expr: &Arc, + projection_mapping: &ProjectionMapping, + ) -> Option> { + self.eq_group.project_expr(projection_mapping, expr) + } + + /// Constructs a dependency map based on existing orderings referred to in + /// the projection. + /// + /// This function analyzes the orderings in the normalized order-equivalence + /// class and builds a dependency map. The dependency map captures relationships + /// between expressions within the orderings, helping to identify dependencies + /// and construct valid projected orderings during projection operations. + /// + /// # Parameters + /// + /// - `mapping`: A reference to the `ProjectionMapping` that defines the + /// relationship between source and target expressions. + /// + /// # Returns + /// + /// A [`DependencyMap`] representing the dependency map, where each + /// [`DependencyNode`] contains dependencies for the key [`PhysicalSortExpr`]. + /// + /// # Example + /// + /// Assume we have two equivalent orderings: `[a ASC, b ASC]` and `[a ASC, c ASC]`, + /// and the projection mapping is `[a -> a_new, b -> b_new, b + c -> b + c]`. + /// Then, the dependency map will be: + /// + /// ```text + /// a ASC: Node {Some(a_new ASC), HashSet{}} + /// b ASC: Node {Some(b_new ASC), HashSet{a ASC}} + /// c ASC: Node {None, HashSet{a ASC}} + /// ``` + fn construct_dependency_map(&self, mapping: &ProjectionMapping) -> DependencyMap { + let mut dependency_map = HashMap::new(); + for ordering in self.normalized_oeq_class().iter() { + for (idx, sort_expr) in ordering.iter().enumerate() { + let target_sort_expr = + self.project_expr(&sort_expr.expr, mapping).map(|expr| { + PhysicalSortExpr { + expr, + options: sort_expr.options, + } + }); + let is_projected = target_sort_expr.is_some(); + if is_projected + || mapping + .iter() + .any(|(source, _)| expr_refers(source, &sort_expr.expr)) + { + // Previous ordering is a dependency. Note that there is no, + // dependency for a leading ordering (i.e. the first sort + // expression). + let dependency = idx.checked_sub(1).map(|a| &ordering[a]); + // Add sort expressions that can be projected or referred to + // by any of the projection expressions to the dependency map: + dependency_map + .entry(sort_expr.clone()) + .or_insert_with(|| DependencyNode { + target_sort_expr: target_sort_expr.clone(), + dependencies: HashSet::new(), + }) + .insert_dependency(dependency); + } + if !is_projected { + // If we can not project, stop constructing the dependency + // map as remaining dependencies will be invalid after projection. + break; + } + } + } + dependency_map + } + + /// Returns a new `ProjectionMapping` where source expressions are normalized. + /// + /// This normalization ensures that source expressions are transformed into a + /// consistent representation. This is beneficial for algorithms that rely on + /// exact equalities, as it allows for more precise and reliable comparisons. + /// + /// # Parameters + /// + /// - `mapping`: A reference to the original `ProjectionMapping` to be normalized. + /// + /// # Returns + /// + /// A new `ProjectionMapping` with normalized source expressions. + fn normalized_mapping(&self, mapping: &ProjectionMapping) -> ProjectionMapping { + // Construct the mapping where source expressions are normalized. In this way + // In the algorithms below we can work on exact equalities + ProjectionMapping { + map: mapping + .iter() + .map(|(source, target)| { + let normalized_source = self.eq_group.normalize_expr(source.clone()); + (normalized_source, target.clone()) + }) + .collect(), + } + } + + /// Computes projected orderings based on a given projection mapping. + /// + /// This function takes a `ProjectionMapping` and computes the possible + /// orderings for the projected expressions. It considers dependencies + /// between expressions and generates valid orderings according to the + /// specified sort properties. + /// + /// # Parameters + /// + /// - `mapping`: A reference to the `ProjectionMapping` that defines the + /// relationship between source and target expressions. + /// + /// # Returns + /// + /// A vector of `LexOrdering` containing all valid orderings after projection. + fn projected_orderings(&self, mapping: &ProjectionMapping) -> Vec { + let mapping = self.normalized_mapping(mapping); + + // Get dependency map for existing orderings: + let dependency_map = self.construct_dependency_map(&mapping); + + let orderings = mapping.iter().flat_map(|(source, target)| { + referred_dependencies(&dependency_map, source) + .into_iter() + .filter_map(|relevant_deps| { + if let SortProperties::Ordered(options) = + get_expr_ordering(source, &relevant_deps) + { + Some((options, relevant_deps)) + } else { + // Do not consider unordered cases + None + } + }) + .flat_map(|(options, relevant_deps)| { + let sort_expr = PhysicalSortExpr { + expr: target.clone(), + options, + }; + // Generate dependent orderings (i.e. prefixes for `sort_expr`): + let mut dependency_orderings = + generate_dependency_orderings(&relevant_deps, &dependency_map); + // Append `sort_expr` to the dependent orderings: + for ordering in dependency_orderings.iter_mut() { + ordering.push(sort_expr.clone()); + } + dependency_orderings + }) + }); + + // Add valid projected orderings. For example, if existing ordering is + // `a + b` and projection is `[a -> a_new, b -> b_new]`, we need to + // preserve `a_new + b_new` as ordered. Please note that `a_new` and + // `b_new` themselves need not be ordered. Such dependencies cannot be + // deduced via the pass above. + let projected_orderings = dependency_map.iter().flat_map(|(sort_expr, node)| { + let mut prefixes = construct_prefix_orderings(sort_expr, &dependency_map); + if prefixes.is_empty() { + // If prefix is empty, there is no dependency. Insert + // empty ordering: + prefixes = vec![vec![]]; + } + // Append current ordering on top its dependencies: + for ordering in prefixes.iter_mut() { + if let Some(target) = &node.target_sort_expr { + ordering.push(target.clone()) + } + } + prefixes + }); + + // Simplify each ordering by removing redundant sections: + orderings + .chain(projected_orderings) + .map(collapse_lex_ordering) + .collect() + } + + /// Projects constants based on the provided `ProjectionMapping`. + /// + /// This function takes a `ProjectionMapping` and identifies/projects + /// constants based on the existing constants and the mapping. It ensures + /// that constants are appropriately propagated through the projection. + /// + /// # Arguments + /// + /// - `mapping`: A reference to a `ProjectionMapping` representing the + /// mapping of source expressions to target expressions in the projection. + /// + /// # Returns + /// + /// Returns a `Vec>` containing the projected constants. + fn projected_constants( + &self, + mapping: &ProjectionMapping, + ) -> Vec> { + // First, project existing constants. For example, assume that `a + b` + // is known to be constant. If the projection were `a as a_new`, `b as b_new`, + // then we would project constant `a + b` as `a_new + b_new`. + let mut projected_constants = self + .constants + .iter() + .flat_map(|expr| self.eq_group.project_expr(mapping, expr)) + .collect::>(); + // Add projection expressions that are known to be constant: + for (source, target) in mapping.iter() { + if self.is_expr_constant(source) + && !physical_exprs_contains(&projected_constants, target) + { + projected_constants.push(target.clone()); + } + } + projected_constants + } + + /// Projects the equivalences within according to `projection_mapping` + /// and `output_schema`. + pub fn project( + &self, + projection_mapping: &ProjectionMapping, + output_schema: SchemaRef, + ) -> Self { + let projected_constants = self.projected_constants(projection_mapping); + let projected_eq_group = self.eq_group.project(projection_mapping); + let projected_orderings = self.projected_orderings(projection_mapping); + Self { + eq_group: projected_eq_group, + oeq_class: OrderingEquivalenceClass::new(projected_orderings), + constants: projected_constants, + schema: output_schema, + } + } + + /// Returns the longest (potentially partial) permutation satisfying the + /// existing ordering. For example, if we have the equivalent orderings + /// `[a ASC, b ASC]` and `[c DESC]`, with `exprs` containing `[c, b, a, d]`, + /// then this function returns `([a ASC, b ASC, c DESC], [2, 1, 0])`. + /// This means that the specification `[a ASC, b ASC, c DESC]` is satisfied + /// by the existing ordering, and `[a, b, c]` resides at indices: `2, 1, 0` + /// inside the argument `exprs` (respectively). For the mathematical + /// definition of "partial permutation", see: + /// + /// + pub fn find_longest_permutation( + &self, + exprs: &[Arc], + ) -> (LexOrdering, Vec) { + let mut eq_properties = self.clone(); + let mut result = vec![]; + // The algorithm is as follows: + // - Iterate over all the expressions and insert ordered expressions + // into the result. + // - Treat inserted expressions as constants (i.e. add them as constants + // to the state). + // - Continue the above procedure until no expression is inserted; i.e. + // the algorithm reaches a fixed point. + // This algorithm should reach a fixed point in at most `exprs.len()` + // iterations. + let mut search_indices = (0..exprs.len()).collect::>(); + for _idx in 0..exprs.len() { + // Get ordered expressions with their indices. + let ordered_exprs = search_indices + .iter() + .flat_map(|&idx| { + let ExprOrdering { expr, state, .. } = + eq_properties.get_expr_ordering(exprs[idx].clone()); + if let SortProperties::Ordered(options) = state { + Some((PhysicalSortExpr { expr, options }, idx)) + } else { + None + } + }) + .collect::>(); + // We reached a fixed point, exit. + if ordered_exprs.is_empty() { + break; + } + // Remove indices that have an ordering from `search_indices`, and + // treat ordered expressions as constants in subsequent iterations. + // We can do this because the "next" key only matters in a lexicographical + // ordering when the keys to its left have the same values. + // + // Note that these expressions are not properly "constants". This is just + // an implementation strategy confined to this function. + for (PhysicalSortExpr { expr, .. }, idx) in &ordered_exprs { + eq_properties = + eq_properties.add_constants(std::iter::once(expr.clone())); + search_indices.remove(idx); + } + // Add new ordered section to the state. + result.extend(ordered_exprs); + } + result.into_iter().unzip() + } + + /// This function determines whether the provided expression is constant + /// based on the known constants. + /// + /// # Arguments + /// + /// - `expr`: A reference to a `Arc` representing the + /// expression to be checked. + /// + /// # Returns + /// + /// Returns `true` if the expression is constant according to equivalence + /// group, `false` otherwise. + fn is_expr_constant(&self, expr: &Arc) -> bool { + // As an example, assume that we know columns `a` and `b` are constant. + // Then, `a`, `b` and `a + b` will all return `true` whereas `c` will + // return `false`. + let normalized_constants = self.eq_group.normalize_exprs(self.constants.to_vec()); + let normalized_expr = self.eq_group.normalize_expr(expr.clone()); + is_constant_recurse(&normalized_constants, &normalized_expr) + } + + /// Retrieves the ordering information for a given physical expression. + /// + /// This function constructs an `ExprOrdering` object for the provided + /// expression, which encapsulates information about the expression's + /// ordering, including its [`SortProperties`]. + /// + /// # Arguments + /// + /// - `expr`: An `Arc` representing the physical expression + /// for which ordering information is sought. + /// + /// # Returns + /// + /// Returns an `ExprOrdering` object containing the ordering information for + /// the given expression. + pub fn get_expr_ordering(&self, expr: Arc) -> ExprOrdering { + ExprOrdering::new(expr.clone()) + .transform_up(&|expr| Ok(update_ordering(expr, self))) + // Guaranteed to always return `Ok`. + .unwrap() + } +} + +/// Calculates the [`SortProperties`] of a given [`ExprOrdering`] node. +/// The node can either be a leaf node, or an intermediate node: +/// - If it is a leaf node, we directly find the order of the node by looking +/// at the given sort expression and equivalence properties if it is a `Column` +/// leaf, or we mark it as unordered. In the case of a `Literal` leaf, we mark +/// it as singleton so that it can cooperate with all ordered columns. +/// - If it is an intermediate node, the children states matter. Each `PhysicalExpr` +/// and operator has its own rules on how to propagate the children orderings. +/// However, before we engage in recursion, we check whether this intermediate +/// node directly matches with the sort expression. If there is a match, the +/// sort expression emerges at that node immediately, discarding the recursive +/// result coming from its children. +fn update_ordering( + mut node: ExprOrdering, + eq_properties: &EquivalenceProperties, +) -> Transformed { + // We have a Column, which is one of the two possible leaf node types: + let normalized_expr = eq_properties.eq_group.normalize_expr(node.expr.clone()); + if eq_properties.is_expr_constant(&normalized_expr) { + node.state = SortProperties::Singleton; + } else if let Some(options) = eq_properties + .normalized_oeq_class() + .get_options(&normalized_expr) + { + node.state = SortProperties::Ordered(options); + } else if !node.expr.children().is_empty() { + // We have an intermediate (non-leaf) node, account for its children: + node.state = node.expr.get_ordering(&node.children_state()); + } else if node.expr.as_any().is::() { + // We have a Literal, which is the other possible leaf node type: + node.state = node.expr.get_ordering(&[]); + } else { + return Transformed::No(node); + } + Transformed::Yes(node) +} + +/// This function determines whether the provided expression is constant +/// based on the known constants. +/// +/// # Arguments +/// +/// - `constants`: A `&[Arc]` containing expressions known to +/// be a constant. +/// - `expr`: A reference to a `Arc` representing the expression +/// to check. +/// +/// # Returns +/// +/// Returns `true` if the expression is constant according to equivalence +/// group, `false` otherwise. +fn is_constant_recurse( + constants: &[Arc], + expr: &Arc, +) -> bool { + if physical_exprs_contains(constants, expr) { + return true; + } + let children = expr.children(); + !children.is_empty() && children.iter().all(|c| is_constant_recurse(constants, c)) +} + +/// This function examines whether a referring expression directly refers to a +/// given referred expression or if any of its children in the expression tree +/// refer to the specified expression. +/// +/// # Parameters +/// +/// - `referring_expr`: A reference to the referring expression (`Arc`). +/// - `referred_expr`: A reference to the referred expression (`Arc`) +/// +/// # Returns +/// +/// A boolean value indicating whether `referring_expr` refers (needs it to evaluate its result) +/// `referred_expr` or not. +fn expr_refers( + referring_expr: &Arc, + referred_expr: &Arc, +) -> bool { + referring_expr.eq(referred_expr) + || referring_expr + .children() + .iter() + .any(|child| expr_refers(child, referred_expr)) +} + +/// This function analyzes the dependency map to collect referred dependencies for +/// a given source expression. +/// +/// # Parameters +/// +/// - `dependency_map`: A reference to the `DependencyMap` where each +/// `PhysicalSortExpr` is associated with a `DependencyNode`. +/// - `source`: A reference to the source expression (`Arc`) +/// for which relevant dependencies need to be identified. +/// +/// # Returns +/// +/// A `Vec` containing the dependencies for the given source +/// expression. These dependencies are expressions that are referred to by +/// the source expression based on the provided dependency map. +fn referred_dependencies( + dependency_map: &DependencyMap, + source: &Arc, +) -> Vec { + // Associate `PhysicalExpr`s with `PhysicalSortExpr`s that contain them: + let mut expr_to_sort_exprs = HashMap::::new(); + for sort_expr in dependency_map + .keys() + .filter(|sort_expr| expr_refers(source, &sort_expr.expr)) + { + let key = ExprWrapper(sort_expr.expr.clone()); + expr_to_sort_exprs + .entry(key) + .or_default() + .insert(sort_expr.clone()); + } + + // Generate all valid dependencies for the source. For example, if the source + // is `a + b` and the map is `[a -> (a ASC, a DESC), b -> (b ASC)]`, we get + // `vec![HashSet(a ASC, b ASC), HashSet(a DESC, b ASC)]`. + expr_to_sort_exprs + .values() + .multi_cartesian_product() + .map(|referred_deps| referred_deps.into_iter().cloned().collect()) + .collect() +} + +/// This function retrieves the dependencies of the given relevant sort expression +/// from the given dependency map. It then constructs prefix orderings by recursively +/// analyzing the dependencies and include them in the orderings. +/// +/// # Parameters +/// +/// - `relevant_sort_expr`: A reference to the relevant sort expression +/// (`PhysicalSortExpr`) for which prefix orderings are to be constructed. +/// - `dependency_map`: A reference to the `DependencyMap` containing dependencies. +/// +/// # Returns +/// +/// A vector of prefix orderings (`Vec`) based on the given relevant +/// sort expression and its dependencies. +fn construct_prefix_orderings( + relevant_sort_expr: &PhysicalSortExpr, + dependency_map: &DependencyMap, +) -> Vec { + dependency_map[relevant_sort_expr] + .dependencies + .iter() + .flat_map(|dep| construct_orderings(dep, dependency_map)) + .collect() +} + +/// Given a set of relevant dependencies (`relevant_deps`) and a map of dependencies +/// (`dependency_map`), this function generates all possible prefix orderings +/// based on the given dependencies. +/// +/// # Parameters +/// +/// * `dependencies` - A reference to the dependencies. +/// * `dependency_map` - A reference to the map of dependencies for expressions. +/// +/// # Returns +/// +/// A vector of lexical orderings (`Vec`) representing all valid orderings +/// based on the given dependencies. +fn generate_dependency_orderings( + dependencies: &Dependencies, + dependency_map: &DependencyMap, +) -> Vec { + // Construct all the valid prefix orderings for each expression appearing + // in the projection: + let relevant_prefixes = dependencies + .iter() + .flat_map(|dep| { + let prefixes = construct_prefix_orderings(dep, dependency_map); + (!prefixes.is_empty()).then_some(prefixes) + }) + .collect::>(); + + // No dependency, dependent is a leading ordering. + if relevant_prefixes.is_empty() { + // Return an empty ordering: + return vec![vec![]]; + } + + // Generate all possible orderings where dependencies are satisfied for the + // current projection expression. For example, if expression is `a + b ASC`, + // and the dependency for `a ASC` is `[c ASC]`, the dependency for `b ASC` + // is `[d DESC]`, then we generate `[c ASC, d DESC, a + b ASC]` and + // `[d DESC, c ASC, a + b ASC]`. + relevant_prefixes + .into_iter() + .multi_cartesian_product() + .flat_map(|prefix_orderings| { + prefix_orderings + .iter() + .permutations(prefix_orderings.len()) + .map(|prefixes| prefixes.into_iter().flatten().cloned().collect()) + .collect::>() + }) + .collect() +} + +/// This function examines the given expression and the sort expressions it +/// refers to determine the ordering properties of the expression. +/// +/// # Parameters +/// +/// - `expr`: A reference to the source expression (`Arc`) for +/// which ordering properties need to be determined. +/// - `dependencies`: A reference to `Dependencies`, containing sort expressions +/// referred to by `expr`. +/// +/// # Returns +/// +/// A `SortProperties` indicating the ordering information of the given expression. +fn get_expr_ordering( + expr: &Arc, + dependencies: &Dependencies, +) -> SortProperties { + if let Some(column_order) = dependencies.iter().find(|&order| expr.eq(&order.expr)) { + // If exact match is found, return its ordering. + SortProperties::Ordered(column_order.options) + } else { + // Find orderings of its children + let child_states = expr + .children() + .iter() + .map(|child| get_expr_ordering(child, dependencies)) + .collect::>(); + // Calculate expression ordering using ordering of its children. + expr.get_ordering(&child_states) + } +} + +/// Represents a node in the dependency map used to construct projected orderings. +/// +/// A `DependencyNode` contains information about a particular sort expression, +/// including its target sort expression and a set of dependencies on other sort +/// expressions. +/// +/// # Fields +/// +/// - `target_sort_expr`: An optional `PhysicalSortExpr` representing the target +/// sort expression associated with the node. It is `None` if the sort expression +/// cannot be projected. +/// - `dependencies`: A [`Dependencies`] containing dependencies on other sort +/// expressions that are referred to by the target sort expression. +#[derive(Debug, Clone, PartialEq, Eq)] +struct DependencyNode { + target_sort_expr: Option, + dependencies: Dependencies, +} + +impl DependencyNode { + // Insert dependency to the state (if exists). + fn insert_dependency(&mut self, dependency: Option<&PhysicalSortExpr>) { + if let Some(dep) = dependency { + self.dependencies.insert(dep.clone()); + } + } +} + +type DependencyMap = HashMap; +type Dependencies = HashSet; + +/// This function recursively analyzes the dependencies of the given sort +/// expression within the given dependency map to construct lexicographical +/// orderings that include the sort expression and its dependencies. +/// +/// # Parameters +/// +/// - `referred_sort_expr`: A reference to the sort expression (`PhysicalSortExpr`) +/// for which lexicographical orderings satisfying its dependencies are to be +/// constructed. +/// - `dependency_map`: A reference to the `DependencyMap` that contains +/// dependencies for different `PhysicalSortExpr`s. +/// +/// # Returns +/// +/// A vector of lexicographical orderings (`Vec`) based on the given +/// sort expression and its dependencies. +fn construct_orderings( + referred_sort_expr: &PhysicalSortExpr, + dependency_map: &DependencyMap, +) -> Vec { + // We are sure that `referred_sort_expr` is inside `dependency_map`. + let node = &dependency_map[referred_sort_expr]; + // Since we work on intermediate nodes, we are sure `val.target_sort_expr` + // exists. + let target_sort_expr = node.target_sort_expr.clone().unwrap(); + if node.dependencies.is_empty() { + vec![vec![target_sort_expr]] + } else { + node.dependencies + .iter() + .flat_map(|dep| { + let mut orderings = construct_orderings(dep, dependency_map); + for ordering in orderings.iter_mut() { + ordering.push(target_sort_expr.clone()) + } + orderings + }) + .collect() + } +} + +/// Calculate ordering equivalence properties for the given join operation. +pub fn join_equivalence_properties( + left: EquivalenceProperties, + right: EquivalenceProperties, + join_type: &JoinType, + join_schema: SchemaRef, + maintains_input_order: &[bool], + probe_side: Option, + on: &[(Column, Column)], +) -> EquivalenceProperties { + let left_size = left.schema.fields.len(); + let mut result = EquivalenceProperties::new(join_schema); + result.add_equivalence_group(left.eq_group().join( + right.eq_group(), + join_type, + left_size, + on, + )); + + let left_oeq_class = left.oeq_class; + let mut right_oeq_class = right.oeq_class; + match maintains_input_order { + [true, false] => { + // In this special case, right side ordering can be prefixed with + // the left side ordering. + if let (Some(JoinSide::Left), JoinType::Inner) = (probe_side, join_type) { + updated_right_ordering_equivalence_class( + &mut right_oeq_class, + join_type, + left_size, + ); + + // Right side ordering equivalence properties should be prepended + // with those of the left side while constructing output ordering + // equivalence properties since stream side is the left side. + // + // For example, if the right side ordering equivalences contain + // `b ASC`, and the left side ordering equivalences contain `a ASC`, + // then we should add `a ASC, b ASC` to the ordering equivalences + // of the join output. + let out_oeq_class = left_oeq_class.join_suffix(&right_oeq_class); + result.add_ordering_equivalence_class(out_oeq_class); + } else { + result.add_ordering_equivalence_class(left_oeq_class); + } + } + [false, true] => { + updated_right_ordering_equivalence_class( + &mut right_oeq_class, + join_type, + left_size, + ); + // In this special case, left side ordering can be prefixed with + // the right side ordering. + if let (Some(JoinSide::Right), JoinType::Inner) = (probe_side, join_type) { + // Left side ordering equivalence properties should be prepended + // with those of the right side while constructing output ordering + // equivalence properties since stream side is the right side. + // + // For example, if the left side ordering equivalences contain + // `a ASC`, and the right side ordering equivalences contain `b ASC`, + // then we should add `b ASC, a ASC` to the ordering equivalences + // of the join output. + let out_oeq_class = right_oeq_class.join_suffix(&left_oeq_class); + result.add_ordering_equivalence_class(out_oeq_class); + } else { + result.add_ordering_equivalence_class(right_oeq_class); + } + } + [false, false] => {} + [true, true] => unreachable!("Cannot maintain ordering of both sides"), + _ => unreachable!("Join operators can not have more than two children"), + } + result +} + +/// In the context of a join, update the right side `OrderingEquivalenceClass` +/// so that they point to valid indices in the join output schema. +/// +/// To do so, we increment column indices by the size of the left table when +/// join schema consists of a combination of the left and right schemas. This +/// is the case for `Inner`, `Left`, `Full` and `Right` joins. For other cases, +/// indices do not change. +fn updated_right_ordering_equivalence_class( + right_oeq_class: &mut OrderingEquivalenceClass, + join_type: &JoinType, + left_size: usize, +) { + if matches!( + join_type, + JoinType::Inner | JoinType::Left | JoinType::Full | JoinType::Right + ) { + right_oeq_class.add_offset(left_size); + } +} + +/// Wrapper struct for `Arc` to use them as keys in a hash map. +#[derive(Debug, Clone)] +struct ExprWrapper(Arc); + +impl PartialEq for ExprWrapper { + fn eq(&self, other: &Self) -> bool { + self.0.eq(&other.0) + } +} + +impl Eq for ExprWrapper {} + +impl Hash for ExprWrapper { + fn hash(&self, state: &mut H) { + self.0.hash(state); + } +} + +#[cfg(test)] +mod tests { + use std::ops::Not; + use std::sync::Arc; + + use super::*; + use crate::equivalence::add_offset_to_expr; + use crate::equivalence::tests::{ + convert_to_orderings, convert_to_sort_exprs, convert_to_sort_reqs, + create_random_schema, create_test_params, create_test_schema, + generate_table_for_eq_properties, is_table_same_after_sort, output_schema, + }; + use crate::execution_props::ExecutionProps; + use crate::expressions::{col, BinaryExpr, Column}; + use crate::functions::create_physical_expr; + use crate::PhysicalSortExpr; + use arrow::datatypes::{DataType, Field, Schema}; + use arrow_schema::{Fields, SortOptions, TimeUnit}; + use datafusion_common::Result; + use datafusion_expr::{BuiltinScalarFunction, Operator}; + use itertools::Itertools; + + #[test] + fn project_equivalence_properties_test() -> Result<()> { + let input_schema = Arc::new(Schema::new(vec![ + Field::new("a", DataType::Int64, true), + Field::new("b", DataType::Int64, true), + Field::new("c", DataType::Int64, true), + ])); + + let input_properties = EquivalenceProperties::new(input_schema.clone()); + let col_a = col("a", &input_schema)?; + + // a as a1, a as a2, a as a3, a as a3 + let proj_exprs = vec![ + (col_a.clone(), "a1".to_string()), + (col_a.clone(), "a2".to_string()), + (col_a.clone(), "a3".to_string()), + (col_a.clone(), "a4".to_string()), + ]; + let projection_mapping = ProjectionMapping::try_new(&proj_exprs, &input_schema)?; + + let out_schema = output_schema(&projection_mapping, &input_schema)?; + // a as a1, a as a2, a as a3, a as a3 + let proj_exprs = vec![ + (col_a.clone(), "a1".to_string()), + (col_a.clone(), "a2".to_string()), + (col_a.clone(), "a3".to_string()), + (col_a.clone(), "a4".to_string()), + ]; + let projection_mapping = ProjectionMapping::try_new(&proj_exprs, &input_schema)?; + + // a as a1, a as a2, a as a3, a as a3 + let col_a1 = &col("a1", &out_schema)?; + let col_a2 = &col("a2", &out_schema)?; + let col_a3 = &col("a3", &out_schema)?; + let col_a4 = &col("a4", &out_schema)?; + let out_properties = input_properties.project(&projection_mapping, out_schema); + + // At the output a1=a2=a3=a4 + assert_eq!(out_properties.eq_group().len(), 1); + let eq_class = &out_properties.eq_group().classes[0]; + assert_eq!(eq_class.len(), 4); + assert!(eq_class.contains(col_a1)); + assert!(eq_class.contains(col_a2)); + assert!(eq_class.contains(col_a3)); + assert!(eq_class.contains(col_a4)); + + Ok(()) + } + + #[test] + fn test_join_equivalence_properties() -> Result<()> { + let schema = create_test_schema()?; + let col_a = &col("a", &schema)?; + let col_b = &col("b", &schema)?; + let col_c = &col("c", &schema)?; + let offset = schema.fields.len(); + let col_a2 = &add_offset_to_expr(col_a.clone(), offset); + let col_b2 = &add_offset_to_expr(col_b.clone(), offset); + let option_asc = SortOptions { + descending: false, + nulls_first: false, + }; + let test_cases = vec![ + // ------- TEST CASE 1 -------- + // [a ASC], [b ASC] + ( + // [a ASC], [b ASC] + vec![vec![(col_a, option_asc)], vec![(col_b, option_asc)]], + // [a ASC], [b ASC] + vec![vec![(col_a, option_asc)], vec![(col_b, option_asc)]], + // expected [a ASC, a2 ASC], [a ASC, b2 ASC], [b ASC, a2 ASC], [b ASC, b2 ASC] + vec![ + vec![(col_a, option_asc), (col_a2, option_asc)], + vec![(col_a, option_asc), (col_b2, option_asc)], + vec![(col_b, option_asc), (col_a2, option_asc)], + vec![(col_b, option_asc), (col_b2, option_asc)], + ], + ), + // ------- TEST CASE 2 -------- + // [a ASC], [b ASC] + ( + // [a ASC], [b ASC], [c ASC] + vec![ + vec![(col_a, option_asc)], + vec![(col_b, option_asc)], + vec![(col_c, option_asc)], + ], + // [a ASC], [b ASC] + vec![vec![(col_a, option_asc)], vec![(col_b, option_asc)]], + // expected [a ASC, a2 ASC], [a ASC, b2 ASC], [b ASC, a2 ASC], [b ASC, b2 ASC], [c ASC, a2 ASC], [c ASC, b2 ASC] + vec![ + vec![(col_a, option_asc), (col_a2, option_asc)], + vec![(col_a, option_asc), (col_b2, option_asc)], + vec![(col_b, option_asc), (col_a2, option_asc)], + vec![(col_b, option_asc), (col_b2, option_asc)], + vec![(col_c, option_asc), (col_a2, option_asc)], + vec![(col_c, option_asc), (col_b2, option_asc)], + ], + ), + ]; + for (left_orderings, right_orderings, expected) in test_cases { + let mut left_eq_properties = EquivalenceProperties::new(schema.clone()); + let mut right_eq_properties = EquivalenceProperties::new(schema.clone()); + let left_orderings = convert_to_orderings(&left_orderings); + let right_orderings = convert_to_orderings(&right_orderings); + let expected = convert_to_orderings(&expected); + left_eq_properties.add_new_orderings(left_orderings); + right_eq_properties.add_new_orderings(right_orderings); + let join_eq = join_equivalence_properties( + left_eq_properties, + right_eq_properties, + &JoinType::Inner, + Arc::new(Schema::empty()), + &[true, false], + Some(JoinSide::Left), + &[], + ); + let orderings = &join_eq.oeq_class.orderings; + let err_msg = format!("expected: {:?}, actual:{:?}", expected, orderings); + assert_eq!( + join_eq.oeq_class.orderings.len(), + expected.len(), + "{}", + err_msg + ); + for ordering in orderings { + assert!( + expected.contains(ordering), + "{}, ordering: {:?}", + err_msg, + ordering + ); + } + } + Ok(()) + } + + #[test] + fn test_expr_consists_of_constants() -> Result<()> { + let schema = Arc::new(Schema::new(vec![ + Field::new("a", DataType::Int32, true), + Field::new("b", DataType::Int32, true), + Field::new("c", DataType::Int32, true), + Field::new("d", DataType::Int32, true), + Field::new("ts", DataType::Timestamp(TimeUnit::Nanosecond, None), true), + ])); + let col_a = col("a", &schema)?; + let col_b = col("b", &schema)?; + let col_d = col("d", &schema)?; + let b_plus_d = Arc::new(BinaryExpr::new( + col_b.clone(), + Operator::Plus, + col_d.clone(), + )) as Arc; + + let constants = vec![col_a.clone(), col_b.clone()]; + let expr = b_plus_d.clone(); + assert!(!is_constant_recurse(&constants, &expr)); + + let constants = vec![col_a.clone(), col_b.clone(), col_d.clone()]; + let expr = b_plus_d.clone(); + assert!(is_constant_recurse(&constants, &expr)); + Ok(()) + } + + #[test] + fn test_get_updated_right_ordering_equivalence_properties() -> Result<()> { + let join_type = JoinType::Inner; + // Join right child schema + let child_fields: Fields = ["x", "y", "z", "w"] + .into_iter() + .map(|name| Field::new(name, DataType::Int32, true)) + .collect(); + let child_schema = Schema::new(child_fields); + let col_x = &col("x", &child_schema)?; + let col_y = &col("y", &child_schema)?; + let col_z = &col("z", &child_schema)?; + let col_w = &col("w", &child_schema)?; + let option_asc = SortOptions { + descending: false, + nulls_first: false, + }; + // [x ASC, y ASC], [z ASC, w ASC] + let orderings = vec![ + vec![(col_x, option_asc), (col_y, option_asc)], + vec![(col_z, option_asc), (col_w, option_asc)], + ]; + let orderings = convert_to_orderings(&orderings); + // Right child ordering equivalences + let mut right_oeq_class = OrderingEquivalenceClass::new(orderings); + + let left_columns_len = 4; + + let fields: Fields = ["a", "b", "c", "d", "x", "y", "z", "w"] + .into_iter() + .map(|name| Field::new(name, DataType::Int32, true)) + .collect(); + + // Join Schema + let schema = Schema::new(fields); + let col_a = &col("a", &schema)?; + let col_d = &col("d", &schema)?; + let col_x = &col("x", &schema)?; + let col_y = &col("y", &schema)?; + let col_z = &col("z", &schema)?; + let col_w = &col("w", &schema)?; + + let mut join_eq_properties = EquivalenceProperties::new(Arc::new(schema)); + // a=x and d=w + join_eq_properties.add_equal_conditions(col_a, col_x); + join_eq_properties.add_equal_conditions(col_d, col_w); + + updated_right_ordering_equivalence_class( + &mut right_oeq_class, + &join_type, + left_columns_len, + ); + join_eq_properties.add_ordering_equivalence_class(right_oeq_class); + let result = join_eq_properties.oeq_class().clone(); + + // [x ASC, y ASC], [z ASC, w ASC] + let orderings = vec![ + vec![(col_x, option_asc), (col_y, option_asc)], + vec![(col_z, option_asc), (col_w, option_asc)], + ]; + let orderings = convert_to_orderings(&orderings); + let expected = OrderingEquivalenceClass::new(orderings); + + assert_eq!(result, expected); + + Ok(()) + } + + #[test] + fn test_normalize_ordering_equivalence_classes() -> Result<()> { + let sort_options = SortOptions::default(); + + let schema = Schema::new(vec![ + Field::new("a", DataType::Int32, true), + Field::new("b", DataType::Int32, true), + Field::new("c", DataType::Int32, true), + ]); + let col_a_expr = col("a", &schema)?; + let col_b_expr = col("b", &schema)?; + let col_c_expr = col("c", &schema)?; + let mut eq_properties = EquivalenceProperties::new(Arc::new(schema.clone())); + + eq_properties.add_equal_conditions(&col_a_expr, &col_c_expr); + let others = vec![ + vec![PhysicalSortExpr { + expr: col_b_expr.clone(), + options: sort_options, + }], + vec![PhysicalSortExpr { + expr: col_c_expr.clone(), + options: sort_options, + }], + ]; + eq_properties.add_new_orderings(others); + + let mut expected_eqs = EquivalenceProperties::new(Arc::new(schema)); + expected_eqs.add_new_orderings([ + vec![PhysicalSortExpr { + expr: col_b_expr.clone(), + options: sort_options, + }], + vec![PhysicalSortExpr { + expr: col_c_expr.clone(), + options: sort_options, + }], + ]); + + let oeq_class = eq_properties.oeq_class().clone(); + let expected = expected_eqs.oeq_class(); + assert!(oeq_class.eq(expected)); + + Ok(()) + } + + #[test] + fn test_get_indices_of_matching_sort_exprs_with_order_eq() -> Result<()> { + let sort_options = SortOptions::default(); + let sort_options_not = SortOptions::default().not(); + + let schema = Schema::new(vec![ + Field::new("a", DataType::Int32, true), + Field::new("b", DataType::Int32, true), + ]); + let col_a = &col("a", &schema)?; + let col_b = &col("b", &schema)?; + let required_columns = [col_b.clone(), col_a.clone()]; + let mut eq_properties = EquivalenceProperties::new(Arc::new(schema)); + eq_properties.add_new_orderings([vec![ + PhysicalSortExpr { + expr: Arc::new(Column::new("b", 1)), + options: sort_options_not, + }, + PhysicalSortExpr { + expr: Arc::new(Column::new("a", 0)), + options: sort_options, + }, + ]]); + let (result, idxs) = eq_properties.find_longest_permutation(&required_columns); + assert_eq!(idxs, vec![0, 1]); + assert_eq!( + result, + vec![ + PhysicalSortExpr { + expr: col_b.clone(), + options: sort_options_not + }, + PhysicalSortExpr { + expr: col_a.clone(), + options: sort_options + } + ] + ); + + let schema = Schema::new(vec![ + Field::new("a", DataType::Int32, true), + Field::new("b", DataType::Int32, true), + Field::new("c", DataType::Int32, true), + ]); + let col_a = &col("a", &schema)?; + let col_b = &col("b", &schema)?; + let required_columns = [col_b.clone(), col_a.clone()]; + let mut eq_properties = EquivalenceProperties::new(Arc::new(schema)); + eq_properties.add_new_orderings([ + vec![PhysicalSortExpr { + expr: Arc::new(Column::new("c", 2)), + options: sort_options, + }], + vec![ + PhysicalSortExpr { + expr: Arc::new(Column::new("b", 1)), + options: sort_options_not, + }, + PhysicalSortExpr { + expr: Arc::new(Column::new("a", 0)), + options: sort_options, + }, + ], + ]); + let (result, idxs) = eq_properties.find_longest_permutation(&required_columns); + assert_eq!(idxs, vec![0, 1]); + assert_eq!( + result, + vec![ + PhysicalSortExpr { + expr: col_b.clone(), + options: sort_options_not + }, + PhysicalSortExpr { + expr: col_a.clone(), + options: sort_options + } + ] + ); + + let required_columns = [ + Arc::new(Column::new("b", 1)) as _, + Arc::new(Column::new("a", 0)) as _, + ]; + let schema = Schema::new(vec![ + Field::new("a", DataType::Int32, true), + Field::new("b", DataType::Int32, true), + Field::new("c", DataType::Int32, true), + ]); + let mut eq_properties = EquivalenceProperties::new(Arc::new(schema)); + + // not satisfied orders + eq_properties.add_new_orderings([vec![ + PhysicalSortExpr { + expr: Arc::new(Column::new("b", 1)), + options: sort_options_not, + }, + PhysicalSortExpr { + expr: Arc::new(Column::new("c", 2)), + options: sort_options, + }, + PhysicalSortExpr { + expr: Arc::new(Column::new("a", 0)), + options: sort_options, + }, + ]]); + let (_, idxs) = eq_properties.find_longest_permutation(&required_columns); + assert_eq!(idxs, vec![0]); + + Ok(()) + } + + #[test] + fn test_update_ordering() -> Result<()> { + let schema = Schema::new(vec![ + Field::new("a", DataType::Int32, true), + Field::new("b", DataType::Int32, true), + Field::new("c", DataType::Int32, true), + Field::new("d", DataType::Int32, true), + ]); + + let mut eq_properties = EquivalenceProperties::new(Arc::new(schema.clone())); + let col_a = &col("a", &schema)?; + let col_b = &col("b", &schema)?; + let col_c = &col("c", &schema)?; + let col_d = &col("d", &schema)?; + let option_asc = SortOptions { + descending: false, + nulls_first: false, + }; + // b=a (e.g they are aliases) + eq_properties.add_equal_conditions(col_b, col_a); + // [b ASC], [d ASC] + eq_properties.add_new_orderings(vec![ + vec![PhysicalSortExpr { + expr: col_b.clone(), + options: option_asc, + }], + vec![PhysicalSortExpr { + expr: col_d.clone(), + options: option_asc, + }], + ]); + + let test_cases = vec![ + // d + b + ( + Arc::new(BinaryExpr::new( + col_d.clone(), + Operator::Plus, + col_b.clone(), + )) as Arc, + SortProperties::Ordered(option_asc), + ), + // b + (col_b.clone(), SortProperties::Ordered(option_asc)), + // a + (col_a.clone(), SortProperties::Ordered(option_asc)), + // a + c + ( + Arc::new(BinaryExpr::new( + col_a.clone(), + Operator::Plus, + col_c.clone(), + )), + SortProperties::Unordered, + ), + ]; + for (expr, expected) in test_cases { + let leading_orderings = eq_properties + .oeq_class() + .iter() + .flat_map(|ordering| ordering.first().cloned()) + .collect::>(); + let expr_ordering = eq_properties.get_expr_ordering(expr.clone()); + let err_msg = format!( + "expr:{:?}, expected: {:?}, actual: {:?}, leading_orderings: {leading_orderings:?}", + expr, expected, expr_ordering.state + ); + assert_eq!(expr_ordering.state, expected, "{}", err_msg); + } + + Ok(()) + } + + #[test] + fn test_find_longest_permutation_random() -> Result<()> { + const N_RANDOM_SCHEMA: usize = 100; + const N_ELEMENTS: usize = 125; + const N_DISTINCT: usize = 5; + + for seed in 0..N_RANDOM_SCHEMA { + // Create a random schema with random properties + let (test_schema, eq_properties) = create_random_schema(seed as u64)?; + // Generate a data that satisfies properties given + let table_data_with_properties = + generate_table_for_eq_properties(&eq_properties, N_ELEMENTS, N_DISTINCT)?; + + let floor_a = create_physical_expr( + &BuiltinScalarFunction::Floor, + &[col("a", &test_schema)?], + &test_schema, + &ExecutionProps::default(), + )?; + let a_plus_b = Arc::new(BinaryExpr::new( + col("a", &test_schema)?, + Operator::Plus, + col("b", &test_schema)?, + )) as Arc; + let exprs = vec![ + col("a", &test_schema)?, + col("b", &test_schema)?, + col("c", &test_schema)?, + col("d", &test_schema)?, + col("e", &test_schema)?, + col("f", &test_schema)?, + floor_a, + a_plus_b, + ]; + + for n_req in 0..=exprs.len() { + for exprs in exprs.iter().combinations(n_req) { + let exprs = exprs.into_iter().cloned().collect::>(); + let (ordering, indices) = + eq_properties.find_longest_permutation(&exprs); + // Make sure that find_longest_permutation return values are consistent + let ordering2 = indices + .iter() + .zip(ordering.iter()) + .map(|(&idx, sort_expr)| PhysicalSortExpr { + expr: exprs[idx].clone(), + options: sort_expr.options, + }) + .collect::>(); + assert_eq!( + ordering, ordering2, + "indices and lexicographical ordering do not match" + ); + + let err_msg = format!( + "Error in test case ordering:{:?}, eq_properties.oeq_class: {:?}, eq_properties.eq_group: {:?}, eq_properties.constants: {:?}", + ordering, eq_properties.oeq_class, eq_properties.eq_group, eq_properties.constants + ); + assert_eq!(ordering.len(), indices.len(), "{}", err_msg); + // Since ordered section satisfies schema, we expect + // that result will be same after sort (e.g sort was unnecessary). + assert!( + is_table_same_after_sort( + ordering.clone(), + table_data_with_properties.clone(), + )?, + "{}", + err_msg + ); + } + } + } + + Ok(()) + } + #[test] + fn test_find_longest_permutation() -> Result<()> { + // Schema satisfies following orderings: + // [a ASC], [d ASC, b ASC], [e DESC, f ASC, g ASC] + // and + // Column [a=c] (e.g they are aliases). + // At below we add [d ASC, h DESC] also, for test purposes + let (test_schema, mut eq_properties) = create_test_params()?; + let col_a = &col("a", &test_schema)?; + let col_b = &col("b", &test_schema)?; + let col_c = &col("c", &test_schema)?; + let col_d = &col("d", &test_schema)?; + let col_e = &col("e", &test_schema)?; + let col_h = &col("h", &test_schema)?; + // a + d + let a_plus_d = Arc::new(BinaryExpr::new( + col_a.clone(), + Operator::Plus, + col_d.clone(), + )) as Arc; + + let option_asc = SortOptions { + descending: false, + nulls_first: false, + }; + let option_desc = SortOptions { + descending: true, + nulls_first: true, + }; + // [d ASC, h ASC] also satisfies schema. + eq_properties.add_new_orderings([vec![ + PhysicalSortExpr { + expr: col_d.clone(), + options: option_asc, + }, + PhysicalSortExpr { + expr: col_h.clone(), + options: option_desc, + }, + ]]); + let test_cases = vec![ + // TEST CASE 1 + (vec![col_a], vec![(col_a, option_asc)]), + // TEST CASE 2 + (vec![col_c], vec![(col_c, option_asc)]), + // TEST CASE 3 + ( + vec![col_d, col_e, col_b], + vec![ + (col_d, option_asc), + (col_e, option_desc), + (col_b, option_asc), + ], + ), + // TEST CASE 4 + (vec![col_b], vec![]), + // TEST CASE 5 + (vec![col_d], vec![(col_d, option_asc)]), + // TEST CASE 5 + (vec![&a_plus_d], vec![(&a_plus_d, option_asc)]), + // TEST CASE 6 + ( + vec![col_b, col_d], + vec![(col_d, option_asc), (col_b, option_asc)], + ), + // TEST CASE 6 + ( + vec![col_c, col_e], + vec![(col_c, option_asc), (col_e, option_desc)], + ), + ]; + for (exprs, expected) in test_cases { + let exprs = exprs.into_iter().cloned().collect::>(); + let expected = convert_to_sort_exprs(&expected); + let (actual, _) = eq_properties.find_longest_permutation(&exprs); + assert_eq!(actual, expected); + } + + Ok(()) + } + #[test] + fn test_get_meet_ordering() -> Result<()> { + let schema = create_test_schema()?; + let col_a = &col("a", &schema)?; + let col_b = &col("b", &schema)?; + let eq_properties = EquivalenceProperties::new(schema); + let option_asc = SortOptions { + descending: false, + nulls_first: false, + }; + let option_desc = SortOptions { + descending: true, + nulls_first: true, + }; + let tests_cases = vec![ + // Get meet ordering between [a ASC] and [a ASC, b ASC] + // result should be [a ASC] + ( + vec![(col_a, option_asc)], + vec![(col_a, option_asc), (col_b, option_asc)], + Some(vec![(col_a, option_asc)]), + ), + // Get meet ordering between [a ASC] and [a DESC] + // result should be None. + (vec![(col_a, option_asc)], vec![(col_a, option_desc)], None), + // Get meet ordering between [a ASC, b ASC] and [a ASC, b DESC] + // result should be [a ASC]. + ( + vec![(col_a, option_asc), (col_b, option_asc)], + vec![(col_a, option_asc), (col_b, option_desc)], + Some(vec![(col_a, option_asc)]), + ), + ]; + for (lhs, rhs, expected) in tests_cases { + let lhs = convert_to_sort_exprs(&lhs); + let rhs = convert_to_sort_exprs(&rhs); + let expected = expected.map(|expected| convert_to_sort_exprs(&expected)); + let finer = eq_properties.get_meet_ordering(&lhs, &rhs); + assert_eq!(finer, expected) + } + + Ok(()) + } + + #[test] + fn test_get_finer() -> Result<()> { + let schema = create_test_schema()?; + let col_a = &col("a", &schema)?; + let col_b = &col("b", &schema)?; + let col_c = &col("c", &schema)?; + let eq_properties = EquivalenceProperties::new(schema); + let option_asc = SortOptions { + descending: false, + nulls_first: false, + }; + let option_desc = SortOptions { + descending: true, + nulls_first: true, + }; + // First entry, and second entry are the physical sort requirement that are argument for get_finer_requirement. + // Third entry is the expected result. + let tests_cases = vec![ + // Get finer requirement between [a Some(ASC)] and [a None, b Some(ASC)] + // result should be [a Some(ASC), b Some(ASC)] + ( + vec![(col_a, Some(option_asc))], + vec![(col_a, None), (col_b, Some(option_asc))], + Some(vec![(col_a, Some(option_asc)), (col_b, Some(option_asc))]), + ), + // Get finer requirement between [a Some(ASC), b Some(ASC), c Some(ASC)] and [a Some(ASC), b Some(ASC)] + // result should be [a Some(ASC), b Some(ASC), c Some(ASC)] + ( + vec![ + (col_a, Some(option_asc)), + (col_b, Some(option_asc)), + (col_c, Some(option_asc)), + ], + vec![(col_a, Some(option_asc)), (col_b, Some(option_asc))], + Some(vec![ + (col_a, Some(option_asc)), + (col_b, Some(option_asc)), + (col_c, Some(option_asc)), + ]), + ), + // Get finer requirement between [a Some(ASC), b Some(ASC)] and [a Some(ASC), b Some(DESC)] + // result should be None + ( + vec![(col_a, Some(option_asc)), (col_b, Some(option_asc))], + vec![(col_a, Some(option_asc)), (col_b, Some(option_desc))], + None, + ), + ]; + for (lhs, rhs, expected) in tests_cases { + let lhs = convert_to_sort_reqs(&lhs); + let rhs = convert_to_sort_reqs(&rhs); + let expected = expected.map(|expected| convert_to_sort_reqs(&expected)); + let finer = eq_properties.get_finer_requirement(&lhs, &rhs); + assert_eq!(finer, expected) + } + + Ok(()) + } + + #[test] + fn test_normalize_sort_reqs() -> Result<()> { + // Schema satisfies following properties + // a=c + // and following orderings are valid + // [a ASC], [d ASC, b ASC], [e DESC, f ASC, g ASC] + let (test_schema, eq_properties) = create_test_params()?; + let col_a = &col("a", &test_schema)?; + let col_b = &col("b", &test_schema)?; + let col_c = &col("c", &test_schema)?; + let col_d = &col("d", &test_schema)?; + let col_e = &col("e", &test_schema)?; + let col_f = &col("f", &test_schema)?; + let option_asc = SortOptions { + descending: false, + nulls_first: false, + }; + let option_desc = SortOptions { + descending: true, + nulls_first: true, + }; + // First element in the tuple stores vector of requirement, second element is the expected return value for ordering_satisfy function + let requirements = vec![ + ( + vec![(col_a, Some(option_asc))], + vec![(col_a, Some(option_asc))], + ), + ( + vec![(col_a, Some(option_desc))], + vec![(col_a, Some(option_desc))], + ), + (vec![(col_a, None)], vec![(col_a, None)]), + // Test whether equivalence works as expected + ( + vec![(col_c, Some(option_asc))], + vec![(col_a, Some(option_asc))], + ), + (vec![(col_c, None)], vec![(col_a, None)]), + // Test whether ordering equivalence works as expected + ( + vec![(col_d, Some(option_asc)), (col_b, Some(option_asc))], + vec![(col_d, Some(option_asc)), (col_b, Some(option_asc))], + ), + ( + vec![(col_d, None), (col_b, None)], + vec![(col_d, None), (col_b, None)], + ), + ( + vec![(col_e, Some(option_desc)), (col_f, Some(option_asc))], + vec![(col_e, Some(option_desc)), (col_f, Some(option_asc))], + ), + // We should be able to normalize in compatible requirements also (not exactly equal) + ( + vec![(col_e, Some(option_desc)), (col_f, None)], + vec![(col_e, Some(option_desc)), (col_f, None)], + ), + ( + vec![(col_e, None), (col_f, None)], + vec![(col_e, None), (col_f, None)], + ), + ]; + + for (reqs, expected_normalized) in requirements.into_iter() { + let req = convert_to_sort_reqs(&reqs); + let expected_normalized = convert_to_sort_reqs(&expected_normalized); + + assert_eq!( + eq_properties.normalize_sort_requirements(&req), + expected_normalized + ); + } + + Ok(()) + } + + #[test] + fn test_schema_normalize_sort_requirement_with_equivalence() -> Result<()> { + let option1 = SortOptions { + descending: false, + nulls_first: false, + }; + // Assume that column a and c are aliases. + let (test_schema, eq_properties) = create_test_params()?; + let col_a = &col("a", &test_schema)?; + let col_c = &col("c", &test_schema)?; + let col_d = &col("d", &test_schema)?; + + // Test cases for equivalence normalization + // First entry in the tuple is PhysicalSortRequirement, second entry in the tuple is + // expected PhysicalSortRequirement after normalization. + let test_cases = vec![ + (vec![(col_a, Some(option1))], vec![(col_a, Some(option1))]), + // In the normalized version column c should be replace with column a + (vec![(col_c, Some(option1))], vec![(col_a, Some(option1))]), + (vec![(col_c, None)], vec![(col_a, None)]), + (vec![(col_d, Some(option1))], vec![(col_d, Some(option1))]), + ]; + for (reqs, expected) in test_cases.into_iter() { + let reqs = convert_to_sort_reqs(&reqs); + let expected = convert_to_sort_reqs(&expected); + + let normalized = eq_properties.normalize_sort_requirements(&reqs); + assert!( + expected.eq(&normalized), + "error in test: reqs: {reqs:?}, expected: {expected:?}, normalized: {normalized:?}" + ); + } + + Ok(()) + } +} From 78832f11a45dd47e5490583c2f0e90aef20b073f Mon Sep 17 00:00:00 2001 From: Andrew Lamb Date: Tue, 26 Dec 2023 06:59:40 -0500 Subject: [PATCH 500/572] Move parquet_schema.rs from sql to parquet tests (#8644) --- datafusion/core/tests/parquet/mod.rs | 1 + .../parquet_schema.rs => parquet/schema.rs} | 17 +++++++++++++++-- datafusion/core/tests/sql/mod.rs | 1 - 3 files changed, 16 insertions(+), 3 deletions(-) rename datafusion/core/tests/{sql/parquet_schema.rs => parquet/schema.rs} (95%) diff --git a/datafusion/core/tests/parquet/mod.rs b/datafusion/core/tests/parquet/mod.rs index 3f003c077d6a..943f7fdbf4ac 100644 --- a/datafusion/core/tests/parquet/mod.rs +++ b/datafusion/core/tests/parquet/mod.rs @@ -44,6 +44,7 @@ mod file_statistics; mod filter_pushdown; mod page_pruning; mod row_group_pruning; +mod schema; mod schema_coercion; #[cfg(test)] diff --git a/datafusion/core/tests/sql/parquet_schema.rs b/datafusion/core/tests/parquet/schema.rs similarity index 95% rename from datafusion/core/tests/sql/parquet_schema.rs rename to datafusion/core/tests/parquet/schema.rs index bc1578da2c58..30d4e1193022 100644 --- a/datafusion/core/tests/sql/parquet_schema.rs +++ b/datafusion/core/tests/parquet/schema.rs @@ -22,6 +22,7 @@ use ::parquet::arrow::ArrowWriter; use tempfile::TempDir; use super::*; +use datafusion_common::assert_batches_sorted_eq; #[tokio::test] async fn schema_merge_ignores_metadata_by_default() { @@ -90,7 +91,13 @@ async fn schema_merge_ignores_metadata_by_default() { .await .unwrap(); - let actual = execute_to_batches(&ctx, "SELECT * from t").await; + let actual = ctx + .sql("SELECT * from t") + .await + .unwrap() + .collect() + .await + .unwrap(); assert_batches_sorted_eq!(expected, &actual); assert_no_metadata(&actual); } @@ -151,7 +158,13 @@ async fn schema_merge_can_preserve_metadata() { .await .unwrap(); - let actual = execute_to_batches(&ctx, "SELECT * from t").await; + let actual = ctx + .sql("SELECT * from t") + .await + .unwrap() + .collect() + .await + .unwrap(); assert_batches_sorted_eq!(expected, &actual); assert_metadata(&actual, &expected_metadata); } diff --git a/datafusion/core/tests/sql/mod.rs b/datafusion/core/tests/sql/mod.rs index a3d5e32097c6..849d85dec6bf 100644 --- a/datafusion/core/tests/sql/mod.rs +++ b/datafusion/core/tests/sql/mod.rs @@ -79,7 +79,6 @@ pub mod expr; pub mod group_by; pub mod joins; pub mod order; -pub mod parquet_schema; pub mod partitioned_csv; pub mod predicates; pub mod references; From 26a8000fe2343e6a187dcd6e4e8fc037d55e213f Mon Sep 17 00:00:00 2001 From: Andrew Lamb Date: Tue, 26 Dec 2023 07:04:43 -0500 Subject: [PATCH 501/572] Fix group by aliased expression in LogicalPLanBuilder::aggregate (#8629) --- datafusion/core/src/dataframe/mod.rs | 36 ++++++++++++- datafusion/expr/src/logical_plan/builder.rs | 58 ++++++++++++++------- 2 files changed, 73 insertions(+), 21 deletions(-) diff --git a/datafusion/core/src/dataframe/mod.rs b/datafusion/core/src/dataframe/mod.rs index 2ae4a7c21a9c..3c3bcd497b7f 100644 --- a/datafusion/core/src/dataframe/mod.rs +++ b/datafusion/core/src/dataframe/mod.rs @@ -1769,8 +1769,8 @@ mod tests { let df_results = df.collect().await?; #[rustfmt::skip] - assert_batches_sorted_eq!( - [ "+----+", + assert_batches_sorted_eq!([ + "+----+", "| id |", "+----+", "| 1 |", @@ -1781,6 +1781,38 @@ mod tests { Ok(()) } + #[tokio::test] + async fn test_aggregate_alias() -> Result<()> { + let df = test_table().await?; + + let df = df + // GROUP BY `c2 + 1` + .aggregate(vec![col("c2") + lit(1)], vec![])? + // SELECT `c2 + 1` as c2 + .select(vec![(col("c2") + lit(1)).alias("c2")])? + // GROUP BY c2 as "c2" (alias in expr is not supported by SQL) + .aggregate(vec![col("c2").alias("c2")], vec![])?; + + let df_results = df.collect().await?; + + #[rustfmt::skip] + assert_batches_sorted_eq!([ + "+----+", + "| c2 |", + "+----+", + "| 2 |", + "| 3 |", + "| 4 |", + "| 5 |", + "| 6 |", + "+----+", + ], + &df_results + ); + + Ok(()) + } + #[tokio::test] async fn test_distinct() -> Result<()> { let t = test_table().await?; diff --git a/datafusion/expr/src/logical_plan/builder.rs b/datafusion/expr/src/logical_plan/builder.rs index 88310dab82a2..549c25f89bae 100644 --- a/datafusion/expr/src/logical_plan/builder.rs +++ b/datafusion/expr/src/logical_plan/builder.rs @@ -904,27 +904,11 @@ impl LogicalPlanBuilder { group_expr: impl IntoIterator>, aggr_expr: impl IntoIterator>, ) -> Result { - let mut group_expr = normalize_cols(group_expr, &self.plan)?; + let group_expr = normalize_cols(group_expr, &self.plan)?; let aggr_expr = normalize_cols(aggr_expr, &self.plan)?; - // Rewrite groupby exprs according to functional dependencies - let group_by_expr_names = group_expr - .iter() - .map(|group_by_expr| group_by_expr.display_name()) - .collect::>>()?; - let schema = self.plan.schema(); - if let Some(target_indices) = - get_target_functional_dependencies(schema, &group_by_expr_names) - { - for idx in target_indices { - let field = schema.field(idx); - let expr = - Expr::Column(Column::new(field.qualifier().cloned(), field.name())); - if !group_expr.contains(&expr) { - group_expr.push(expr); - } - } - } + let group_expr = + add_group_by_exprs_from_dependencies(group_expr, self.plan.schema())?; Aggregate::try_new(Arc::new(self.plan), group_expr, aggr_expr) .map(LogicalPlan::Aggregate) .map(Self::from) @@ -1189,6 +1173,42 @@ pub fn build_join_schema( schema.with_functional_dependencies(func_dependencies) } +/// Add additional "synthetic" group by expressions based on functional +/// dependencies. +/// +/// For example, if we are grouping on `[c1]`, and we know from +/// functional dependencies that column `c1` determines `c2`, this function +/// adds `c2` to the group by list. +/// +/// This allows MySQL style selects like +/// `SELECT col FROM t WHERE pk = 5` if col is unique +fn add_group_by_exprs_from_dependencies( + mut group_expr: Vec, + schema: &DFSchemaRef, +) -> Result> { + // Names of the fields produced by the GROUP BY exprs for example, `GROUP BY + // c1 + 1` produces an output field named `"c1 + 1"` + let mut group_by_field_names = group_expr + .iter() + .map(|e| e.display_name()) + .collect::>>()?; + + if let Some(target_indices) = + get_target_functional_dependencies(schema, &group_by_field_names) + { + for idx in target_indices { + let field = schema.field(idx); + let expr = + Expr::Column(Column::new(field.qualifier().cloned(), field.name())); + let expr_name = expr.display_name()?; + if !group_by_field_names.contains(&expr_name) { + group_by_field_names.push(expr_name); + group_expr.push(expr); + } + } + } + Ok(group_expr) +} /// Errors if one or more expressions have equal names. pub(crate) fn validate_unique_names<'a>( node_name: &str, From 58b0a2bfd4ec9b671fd60b8992b111fc8acd4889 Mon Sep 17 00:00:00 2001 From: Alex Huang Date: Wed, 27 Dec 2023 13:51:14 +0100 Subject: [PATCH 502/572] Refactor `array_union` and `array_intersect` functions to one general function (#8516) * Refactor array_union and array_intersect functions * fix cli * fix ci * add tests for null * modify the return type * update tests * fix clippy * fix clippy * add tests for largelist * fix clippy * Add field parameter to generic_set_lists() function * Add large array drop statements * fix clippy --- datafusion/expr/src/built_in_function.rs | 13 +- .../physical-expr/src/array_expressions.rs | 283 +++++++++-------- datafusion/sqllogictest/test_files/array.slt | 294 +++++++++++++++++- 3 files changed, 446 insertions(+), 144 deletions(-) diff --git a/datafusion/expr/src/built_in_function.rs b/datafusion/expr/src/built_in_function.rs index 3818e8ee5658..c454a9781eda 100644 --- a/datafusion/expr/src/built_in_function.rs +++ b/datafusion/expr/src/built_in_function.rs @@ -618,7 +618,18 @@ impl BuiltinScalarFunction { BuiltinScalarFunction::ArrayReplaceAll => Ok(input_expr_types[0].clone()), BuiltinScalarFunction::ArraySlice => Ok(input_expr_types[0].clone()), BuiltinScalarFunction::ArrayToString => Ok(Utf8), - BuiltinScalarFunction::ArrayUnion | BuiltinScalarFunction::ArrayIntersect => { + BuiltinScalarFunction::ArrayIntersect => { + match (input_expr_types[0].clone(), input_expr_types[1].clone()) { + (DataType::Null, DataType::Null) | (DataType::Null, _) => { + Ok(DataType::Null) + } + (_, DataType::Null) => { + Ok(List(Arc::new(Field::new("item", Null, true)))) + } + (dt, _) => Ok(dt), + } + } + BuiltinScalarFunction::ArrayUnion => { match (input_expr_types[0].clone(), input_expr_types[1].clone()) { (DataType::Null, dt) => Ok(dt), (dt, DataType::Null) => Ok(dt), diff --git a/datafusion/physical-expr/src/array_expressions.rs b/datafusion/physical-expr/src/array_expressions.rs index 3ee99d7e8e55..274d1db4eb0d 100644 --- a/datafusion/physical-expr/src/array_expressions.rs +++ b/datafusion/physical-expr/src/array_expressions.rs @@ -19,6 +19,7 @@ use std::any::type_name; use std::collections::HashSet; +use std::fmt::{Display, Formatter}; use std::sync::Arc; use arrow::array::*; @@ -1777,97 +1778,173 @@ macro_rules! to_string { }}; } -fn union_generic_lists( +#[derive(Debug, PartialEq)] +enum SetOp { + Union, + Intersect, +} + +impl Display for SetOp { + fn fmt(&self, f: &mut Formatter<'_>) -> std::fmt::Result { + match self { + SetOp::Union => write!(f, "array_union"), + SetOp::Intersect => write!(f, "array_intersect"), + } + } +} + +fn generic_set_lists( l: &GenericListArray, r: &GenericListArray, - field: &FieldRef, -) -> Result> { - let converter = RowConverter::new(vec![SortField::new(l.value_type())])?; + field: Arc, + set_op: SetOp, +) -> Result { + if matches!(l.value_type(), DataType::Null) { + let field = Arc::new(Field::new("item", r.value_type(), true)); + return general_array_distinct::(r, &field); + } else if matches!(r.value_type(), DataType::Null) { + let field = Arc::new(Field::new("item", l.value_type(), true)); + return general_array_distinct::(l, &field); + } - let nulls = NullBuffer::union(l.nulls(), r.nulls()); - let l_values = l.values().clone(); - let r_values = r.values().clone(); - let l_values = converter.convert_columns(&[l_values])?; - let r_values = converter.convert_columns(&[r_values])?; + if l.value_type() != r.value_type() { + return internal_err!("{set_op:?} is not implemented for '{l:?}' and '{r:?}'"); + } - // Might be worth adding an upstream OffsetBufferBuilder - let mut offsets = Vec::::with_capacity(l.len() + 1); - offsets.push(OffsetSize::usize_as(0)); - let mut rows = Vec::with_capacity(l_values.num_rows() + r_values.num_rows()); - let mut dedup = HashSet::new(); - for (l_w, r_w) in l.offsets().windows(2).zip(r.offsets().windows(2)) { - let l_slice = l_w[0].as_usize()..l_w[1].as_usize(); - let r_slice = r_w[0].as_usize()..r_w[1].as_usize(); - for i in l_slice { - let left_row = l_values.row(i); - if dedup.insert(left_row) { - rows.push(left_row); - } - } - for i in r_slice { - let right_row = r_values.row(i); - if dedup.insert(right_row) { - rows.push(right_row); + let dt = l.value_type(); + + let mut offsets = vec![OffsetSize::usize_as(0)]; + let mut new_arrays = vec![]; + + let converter = RowConverter::new(vec![SortField::new(dt)])?; + for (first_arr, second_arr) in l.iter().zip(r.iter()) { + if let (Some(first_arr), Some(second_arr)) = (first_arr, second_arr) { + let l_values = converter.convert_columns(&[first_arr])?; + let r_values = converter.convert_columns(&[second_arr])?; + + let l_iter = l_values.iter().sorted().dedup(); + let values_set: HashSet<_> = l_iter.clone().collect(); + let mut rows = if set_op == SetOp::Union { + l_iter.collect::>() + } else { + vec![] + }; + for r_val in r_values.iter().sorted().dedup() { + match set_op { + SetOp::Union => { + if !values_set.contains(&r_val) { + rows.push(r_val); + } + } + SetOp::Intersect => { + if values_set.contains(&r_val) { + rows.push(r_val); + } + } + } } + + let last_offset = match offsets.last().copied() { + Some(offset) => offset, + None => return internal_err!("offsets should not be empty"), + }; + offsets.push(last_offset + OffsetSize::usize_as(rows.len())); + let arrays = converter.convert_rows(rows)?; + let array = match arrays.first() { + Some(array) => array.clone(), + None => { + return internal_err!("{set_op}: failed to get array from rows"); + } + }; + new_arrays.push(array); } - offsets.push(OffsetSize::usize_as(rows.len())); - dedup.clear(); } - let values = converter.convert_rows(rows)?; let offsets = OffsetBuffer::new(offsets.into()); - let result = values[0].clone(); - Ok(GenericListArray::::new( - field.clone(), - offsets, - result, - nulls, - )) + let new_arrays_ref = new_arrays.iter().map(|v| v.as_ref()).collect::>(); + let values = compute::concat(&new_arrays_ref)?; + let arr = GenericListArray::::try_new(field, offsets, values, None)?; + Ok(Arc::new(arr)) } -/// Array_union SQL function -pub fn array_union(args: &[ArrayRef]) -> Result { - if args.len() != 2 { - return exec_err!("array_union needs 2 arguments"); - } - let array1 = &args[0]; - let array2 = &args[1]; +fn general_set_op( + array1: &ArrayRef, + array2: &ArrayRef, + set_op: SetOp, +) -> Result { + match (array1.data_type(), array2.data_type()) { + (DataType::Null, DataType::List(field)) => { + if set_op == SetOp::Intersect { + return Ok(new_empty_array(&DataType::Null)); + } + let array = as_list_array(&array2)?; + general_array_distinct::(array, field) + } - fn union_arrays( - array1: &ArrayRef, - array2: &ArrayRef, - l_field_ref: &Arc, - r_field_ref: &Arc, - ) -> Result { - match (l_field_ref.data_type(), r_field_ref.data_type()) { - (DataType::Null, _) => Ok(array2.clone()), - (_, DataType::Null) => Ok(array1.clone()), - (_, _) => { - let list1 = array1.as_list::(); - let list2 = array2.as_list::(); - let result = union_generic_lists::(list1, list2, l_field_ref)?; - Ok(Arc::new(result)) + (DataType::List(field), DataType::Null) => { + if set_op == SetOp::Intersect { + return make_array(&[]); } + let array = as_list_array(&array1)?; + general_array_distinct::(array, field) } - } + (DataType::Null, DataType::LargeList(field)) => { + if set_op == SetOp::Intersect { + return Ok(new_empty_array(&DataType::Null)); + } + let array = as_large_list_array(&array2)?; + general_array_distinct::(array, field) + } + (DataType::LargeList(field), DataType::Null) => { + if set_op == SetOp::Intersect { + return make_array(&[]); + } + let array = as_large_list_array(&array1)?; + general_array_distinct::(array, field) + } + (DataType::Null, DataType::Null) => Ok(new_empty_array(&DataType::Null)), - match (array1.data_type(), array2.data_type()) { - (DataType::Null, _) => Ok(array2.clone()), - (_, DataType::Null) => Ok(array1.clone()), - (DataType::List(l_field_ref), DataType::List(r_field_ref)) => { - union_arrays::(array1, array2, l_field_ref, r_field_ref) + (DataType::List(field), DataType::List(_)) => { + let array1 = as_list_array(&array1)?; + let array2 = as_list_array(&array2)?; + generic_set_lists::(array1, array2, field.clone(), set_op) } - (DataType::LargeList(l_field_ref), DataType::LargeList(r_field_ref)) => { - union_arrays::(array1, array2, l_field_ref, r_field_ref) + (DataType::LargeList(field), DataType::LargeList(_)) => { + let array1 = as_large_list_array(&array1)?; + let array2 = as_large_list_array(&array2)?; + generic_set_lists::(array1, array2, field.clone(), set_op) } - _ => { + (data_type1, data_type2) => { internal_err!( - "array_union only support list with offsets of type int32 and int64" + "{set_op} does not support types '{data_type1:?}' and '{data_type2:?}'" ) } } } +/// Array_union SQL function +pub fn array_union(args: &[ArrayRef]) -> Result { + if args.len() != 2 { + return exec_err!("array_union needs two arguments"); + } + let array1 = &args[0]; + let array2 = &args[1]; + + general_set_op(array1, array2, SetOp::Union) +} + +/// array_intersect SQL function +pub fn array_intersect(args: &[ArrayRef]) -> Result { + if args.len() != 2 { + return exec_err!("array_intersect needs two arguments"); + } + + let array1 = &args[0]; + let array2 = &args[1]; + + general_set_op(array1, array2, SetOp::Intersect) +} + /// Array_to_string SQL function pub fn array_to_string(args: &[ArrayRef]) -> Result { if args.len() < 2 || args.len() > 3 { @@ -2228,7 +2305,7 @@ pub fn array_has(args: &[ArrayRef]) -> Result { DataType::LargeList(_) => { general_array_has_dispatch::(&args[0], &args[1], ComparisonType::Single) } - _ => internal_err!("array_has does not support type '{array_type:?}'."), + _ => exec_err!("array_has does not support type '{array_type:?}'."), } } @@ -2359,74 +2436,6 @@ pub fn string_to_array(args: &[ArrayRef]) -> Result Result { - if args.len() != 2 { - return exec_err!("array_intersect needs two arguments"); - } - - let first_array = &args[0]; - let second_array = &args[1]; - - match (first_array.data_type(), second_array.data_type()) { - (DataType::Null, _) => Ok(second_array.clone()), - (_, DataType::Null) => Ok(first_array.clone()), - _ => { - let first_array = as_list_array(&first_array)?; - let second_array = as_list_array(&second_array)?; - - if first_array.value_type() != second_array.value_type() { - return internal_err!("array_intersect is not implemented for '{first_array:?}' and '{second_array:?}'"); - } - - let dt = first_array.value_type(); - - let mut offsets = vec![0]; - let mut new_arrays = vec![]; - - let converter = RowConverter::new(vec![SortField::new(dt.clone())])?; - for (first_arr, second_arr) in first_array.iter().zip(second_array.iter()) { - if let (Some(first_arr), Some(second_arr)) = (first_arr, second_arr) { - let l_values = converter.convert_columns(&[first_arr])?; - let r_values = converter.convert_columns(&[second_arr])?; - - let values_set: HashSet<_> = l_values.iter().collect(); - let mut rows = Vec::with_capacity(r_values.num_rows()); - for r_val in r_values.iter().sorted().dedup() { - if values_set.contains(&r_val) { - rows.push(r_val); - } - } - - let last_offset: i32 = match offsets.last().copied() { - Some(offset) => offset, - None => return internal_err!("offsets should not be empty"), - }; - offsets.push(last_offset + rows.len() as i32); - let arrays = converter.convert_rows(rows)?; - let array = match arrays.first() { - Some(array) => array.clone(), - None => { - return internal_err!( - "array_intersect: failed to get array from rows" - ) - } - }; - new_arrays.push(array); - } - } - - let field = Arc::new(Field::new("item", dt, true)); - let offsets = OffsetBuffer::new(offsets.into()); - let new_arrays_ref = - new_arrays.iter().map(|v| v.as_ref()).collect::>(); - let values = compute::concat(&new_arrays_ref)?; - let arr = Arc::new(ListArray::try_new(field, offsets, values, None)?); - Ok(arr) - } - } -} - pub fn general_array_distinct( array: &GenericListArray, field: &FieldRef, diff --git a/datafusion/sqllogictest/test_files/array.slt b/datafusion/sqllogictest/test_files/array.slt index 283f2d67b7a0..4c4adbabfda5 100644 --- a/datafusion/sqllogictest/test_files/array.slt +++ b/datafusion/sqllogictest/test_files/array.slt @@ -231,6 +231,19 @@ AS VALUES (make_array(11, 22), make_array(11), make_array(11,22,33), make_array(11,33), make_array(11,33,55), make_array(22,44,66,88,11,33)) ; +statement ok +CREATE TABLE large_array_intersect_table_1D +AS + SELECT + arrow_cast(column1, 'LargeList(Int64)') as column1, + arrow_cast(column2, 'LargeList(Int64)') as column2, + arrow_cast(column3, 'LargeList(Int64)') as column3, + arrow_cast(column4, 'LargeList(Int64)') as column4, + arrow_cast(column5, 'LargeList(Int64)') as column5, + arrow_cast(column6, 'LargeList(Int64)') as column6 +FROM array_intersect_table_1D +; + statement ok CREATE TABLE array_intersect_table_1D_Float AS VALUES @@ -238,6 +251,19 @@ AS VALUES (make_array(3.0, 4.0, 5.0), make_array(2.0), make_array(1.0,2.0,3.0,4.0), make_array(2.0,5.0), make_array(2.22, 1.11), make_array(1.11, 3.33)) ; +statement ok +CREATE TABLE large_array_intersect_table_1D_Float +AS + SELECT + arrow_cast(column1, 'LargeList(Float64)') as column1, + arrow_cast(column2, 'LargeList(Float64)') as column2, + arrow_cast(column3, 'LargeList(Float64)') as column3, + arrow_cast(column4, 'LargeList(Float64)') as column4, + arrow_cast(column5, 'LargeList(Float64)') as column5, + arrow_cast(column6, 'LargeList(Float64)') as column6 +FROM array_intersect_table_1D_Float +; + statement ok CREATE TABLE array_intersect_table_1D_Boolean AS VALUES @@ -245,6 +271,19 @@ AS VALUES (make_array(false, false, false), make_array(false), make_array(true, false, true), make_array(true, true), make_array(true, true), make_array(false,false,true)) ; +statement ok +CREATE TABLE large_array_intersect_table_1D_Boolean +AS + SELECT + arrow_cast(column1, 'LargeList(Boolean)') as column1, + arrow_cast(column2, 'LargeList(Boolean)') as column2, + arrow_cast(column3, 'LargeList(Boolean)') as column3, + arrow_cast(column4, 'LargeList(Boolean)') as column4, + arrow_cast(column5, 'LargeList(Boolean)') as column5, + arrow_cast(column6, 'LargeList(Boolean)') as column6 +FROM array_intersect_table_1D_Boolean +; + statement ok CREATE TABLE array_intersect_table_1D_UTF8 AS VALUES @@ -252,6 +291,19 @@ AS VALUES (make_array('a', 'bc', 'def'), make_array('defg'), make_array('datafusion', 'rust', 'arrow'), make_array('datafusion', 'rust', 'arrow', 'python'), make_array('rust', 'arrow'), make_array('datafusion', 'rust', 'arrow')) ; +statement ok +CREATE TABLE large_array_intersect_table_1D_UTF8 +AS + SELECT + arrow_cast(column1, 'LargeList(Utf8)') as column1, + arrow_cast(column2, 'LargeList(Utf8)') as column2, + arrow_cast(column3, 'LargeList(Utf8)') as column3, + arrow_cast(column4, 'LargeList(Utf8)') as column4, + arrow_cast(column5, 'LargeList(Utf8)') as column5, + arrow_cast(column6, 'LargeList(Utf8)') as column6 +FROM array_intersect_table_1D_UTF8 +; + statement ok CREATE TABLE array_intersect_table_2D AS VALUES @@ -259,6 +311,17 @@ AS VALUES (make_array([3,4], [5]), make_array([3,4]), make_array([1,2,3,4], [5,6,7], [8,9,10]), make_array([1,2,3], [5,6,7], [8,9,10])) ; +statement ok +CREATE TABLE large_array_intersect_table_2D +AS + SELECT + arrow_cast(column1, 'LargeList(List(Int64))') as column1, + arrow_cast(column2, 'LargeList(List(Int64))') as column2, + arrow_cast(column3, 'LargeList(List(Int64))') as column3, + arrow_cast(column4, 'LargeList(List(Int64))') as column4 +FROM array_intersect_table_2D +; + statement ok CREATE TABLE array_intersect_table_2D_float AS VALUES @@ -266,6 +329,15 @@ AS VALUES (make_array([1.0, 2.0, 3.0], [1.1, 2.2], [3.3]), make_array([1.0], [1.1, 2.2], [3.3])) ; +statement ok +CREATE TABLE large_array_intersect_table_2D_Float +AS + SELECT + arrow_cast(column1, 'LargeList(List(Float64))') as column1, + arrow_cast(column2, 'LargeList(List(Float64))') as column2 +FROM array_intersect_table_2D_Float +; + statement ok CREATE TABLE array_intersect_table_3D AS VALUES @@ -273,6 +345,15 @@ AS VALUES (make_array([[1,2]]), make_array([[1,2]])) ; +statement ok +CREATE TABLE large_array_intersect_table_3D +AS + SELECT + arrow_cast(column1, 'LargeList(List(List(Int64)))') as column1, + arrow_cast(column2, 'LargeList(List(List(Int64)))') as column2 +FROM array_intersect_table_3D +; + statement ok CREATE TABLE arrays_values_without_nulls AS VALUES @@ -2589,24 +2670,44 @@ select array_union([1, 2, 3, 4], [5, 6, 3, 4]); ---- [1, 2, 3, 4, 5, 6] +query ? +select array_union(arrow_cast([1, 2, 3, 4], 'LargeList(Int64)'), arrow_cast([5, 6, 3, 4], 'LargeList(Int64)')); +---- +[1, 2, 3, 4, 5, 6] + # array_union scalar function #2 query ? select array_union([1, 2, 3, 4], [5, 6, 7, 8]); ---- [1, 2, 3, 4, 5, 6, 7, 8] +query ? +select array_union(arrow_cast([1, 2, 3, 4], 'LargeList(Int64)'), arrow_cast([5, 6, 7, 8], 'LargeList(Int64)')); +---- +[1, 2, 3, 4, 5, 6, 7, 8] + # array_union scalar function #3 query ? select array_union([1,2,3], []); ---- [1, 2, 3] +query ? +select array_union(arrow_cast([1,2,3], 'LargeList(Int64)'), arrow_cast([], 'LargeList(Null)')); +---- +[1, 2, 3] + # array_union scalar function #4 query ? select array_union([1, 2, 3, 4], [5, 4]); ---- [1, 2, 3, 4, 5] +query ? +select array_union(arrow_cast([1, 2, 3, 4], 'LargeList(Int64)'), arrow_cast([5, 4], 'LargeList(Int64)')); +---- +[1, 2, 3, 4, 5] + # array_union scalar function #5 statement ok CREATE TABLE arrays_with_repeating_elements_for_union @@ -2623,6 +2724,13 @@ select array_union(column1, column2) from arrays_with_repeating_elements_for_uni [2, 3] [3, 4] +query ? +select array_union(arrow_cast(column1, 'LargeList(Int64)'), arrow_cast(column2, 'LargeList(Int64)')) from arrays_with_repeating_elements_for_union; +---- +[1, 2] +[2, 3] +[3, 4] + statement ok drop table arrays_with_repeating_elements_for_union; @@ -2632,24 +2740,44 @@ select array_union([], []); ---- [] +query ? +select array_union(arrow_cast([], 'LargeList(Null)'), arrow_cast([], 'LargeList(Null)')); +---- +[] + # array_union scalar function #7 query ? select array_union([[null]], []); ---- [[]] +query ? +select array_union(arrow_cast([[null]], 'LargeList(List(Null))'), arrow_cast([], 'LargeList(Null)')); +---- +[[]] + # array_union scalar function #8 query ? select array_union([null], [null]); ---- [] +query ? +select array_union(arrow_cast([[null]], 'LargeList(List(Null))'), arrow_cast([[null]], 'LargeList(List(Null))')); +---- +[[]] + # array_union scalar function #9 query ? select array_union(null, []); ---- [] +query ? +select array_union(null, arrow_cast([], 'LargeList(Null)')); +---- +[] + # array_union scalar function #10 query ? select array_union(null, null); @@ -2658,21 +2786,47 @@ NULL # array_union scalar function #11 query ? -select array_union([1.2, 3.0], [1.2, 3.0, 5.7]); +select array_union([1, 1, 2, 2, 3, 3], null); ---- -[1.2, 3.0, 5.7] +[1, 2, 3] -# array_union scalar function #12 query ? -select array_union(['hello'], ['hello','datafusion']); +select array_union(arrow_cast([1, 1, 2, 2, 3, 3], 'LargeList(Int64)'), null); ---- -[hello, datafusion] +[1, 2, 3] +# array_union scalar function #12 +query ? +select array_union(null, [1, 1, 2, 2, 3, 3]); +---- +[1, 2, 3] +query ? +select array_union(null, arrow_cast([1, 1, 2, 2, 3, 3], 'LargeList(Int64)')); +---- +[1, 2, 3] +# array_union scalar function #13 +query ? +select array_union([1.2, 3.0], [1.2, 3.0, 5.7]); +---- +[1.2, 3.0, 5.7] +query ? +select array_union(arrow_cast([1.2, 3.0], 'LargeList(Float64)'), arrow_cast([1.2, 3.0, 5.7], 'LargeList(Float64)')); +---- +[1.2, 3.0, 5.7] +# array_union scalar function #14 +query ? +select array_union(['hello'], ['hello','datafusion']); +---- +[hello, datafusion] +query ? +select array_union(arrow_cast(['hello'], 'LargeList(Utf8)'), arrow_cast(['hello','datafusion'], 'LargeList(Utf8)')); +---- +[hello, datafusion] # list_to_string scalar function #4 (function alias `array_to_string`) @@ -3536,6 +3690,15 @@ from array_intersect_table_1D; [1] [1, 3] [1, 3] [11] [11, 33] [11, 33] +query ??? +select array_intersect(column1, column2), + array_intersect(column3, column4), + array_intersect(column5, column6) +from large_array_intersect_table_1D; +---- +[1] [1, 3] [1, 3] +[11] [11, 33] [11, 33] + query ??? select array_intersect(column1, column2), array_intersect(column3, column4), @@ -3554,6 +3717,15 @@ from array_intersect_table_1D_Boolean; [] [false, true] [false] [false] [true] [true] +query ??? +select array_intersect(column1, column2), + array_intersect(column3, column4), + array_intersect(column5, column6) +from large_array_intersect_table_1D_Boolean; +---- +[] [false, true] [false] +[false] [true] [true] + query ??? select array_intersect(column1, column2), array_intersect(column3, column4), @@ -3563,6 +3735,15 @@ from array_intersect_table_1D_UTF8; [bc] [arrow, rust] [] [] [arrow, datafusion, rust] [arrow, rust] +query ??? +select array_intersect(column1, column2), + array_intersect(column3, column4), + array_intersect(column5, column6) +from large_array_intersect_table_1D_UTF8; +---- +[bc] [arrow, rust] [] +[] [arrow, datafusion, rust] [arrow, rust] + query ?? select array_intersect(column1, column2), array_intersect(column3, column4) @@ -3571,6 +3752,15 @@ from array_intersect_table_2D; [] [[4, 5], [6, 7]] [[3, 4]] [[5, 6, 7], [8, 9, 10]] +query ?? +select array_intersect(column1, column2), + array_intersect(column3, column4) +from large_array_intersect_table_2D; +---- +[] [[4, 5], [6, 7]] +[[3, 4]] [[5, 6, 7], [8, 9, 10]] + + query ? select array_intersect(column1, column2) from array_intersect_table_2D_float; @@ -3578,6 +3768,13 @@ from array_intersect_table_2D_float; [[1.1, 2.2], [3.3]] [[1.1, 2.2], [3.3]] +query ? +select array_intersect(column1, column2) +from large_array_intersect_table_2D_float; +---- +[[1.1, 2.2], [3.3]] +[[1.1, 2.2], [3.3]] + query ? select array_intersect(column1, column2) from array_intersect_table_3D; @@ -3585,6 +3782,13 @@ from array_intersect_table_3D; [] [[[1, 2]]] +query ? +select array_intersect(column1, column2) +from large_array_intersect_table_3D; +---- +[] +[[[1, 2]]] + query ?????? SELECT array_intersect(make_array(1,2,3), make_array(2,3,4)), array_intersect(make_array(1,3,5), make_array(2,4,6)), @@ -3596,21 +3800,67 @@ SELECT array_intersect(make_array(1,2,3), make_array(2,3,4)), ---- [2, 3] [] [aa, cc] [true] [2.2, 3.3] [[2, 2], [3, 3]] +query ?????? +SELECT array_intersect(arrow_cast(make_array(1,2,3), 'LargeList(Int64)'), arrow_cast(make_array(2,3,4), 'LargeList(Int64)')), + array_intersect(arrow_cast(make_array(1,3,5), 'LargeList(Int64)'), arrow_cast(make_array(2,4,6), 'LargeList(Int64)')), + array_intersect(arrow_cast(make_array('aa','bb','cc'), 'LargeList(Utf8)'), arrow_cast(make_array('cc','aa','dd'), 'LargeList(Utf8)')), + array_intersect(arrow_cast(make_array(true, false), 'LargeList(Boolean)'), arrow_cast(make_array(true), 'LargeList(Boolean)')), + array_intersect(arrow_cast(make_array(1.1, 2.2, 3.3), 'LargeList(Float64)'), arrow_cast(make_array(2.2, 3.3, 4.4), 'LargeList(Float64)')), + array_intersect(arrow_cast(make_array([1, 1], [2, 2], [3, 3]), 'LargeList(List(Int64))'), arrow_cast(make_array([2, 2], [3, 3], [4, 4]), 'LargeList(List(Int64))')) +; +---- +[2, 3] [] [aa, cc] [true] [2.2, 3.3] [[2, 2], [3, 3]] + query ? select array_intersect([], []); ---- [] +query ? +select array_intersect(arrow_cast([], 'LargeList(Null)'), arrow_cast([], 'LargeList(Null)')); +---- +[] + +query ? +select array_intersect([1, 1, 2, 2, 3, 3], null); +---- +[] + +query ? +select array_intersect(arrow_cast([1, 1, 2, 2, 3, 3], 'LargeList(Int64)'), null); +---- +[] + +query ? +select array_intersect(null, [1, 1, 2, 2, 3, 3]); +---- +NULL + +query ? +select array_intersect(null, arrow_cast([1, 1, 2, 2, 3, 3], 'LargeList(Int64)')); +---- +NULL + query ? select array_intersect([], null); ---- [] query ? -select array_intersect(null, []); +select array_intersect(arrow_cast([], 'LargeList(Null)'), null); ---- [] +query ? +select array_intersect(null, []); +---- +NULL + +query ? +select array_intersect(null, arrow_cast([], 'LargeList(Null)')); +---- +NULL + query ? select array_intersect(null, null); ---- @@ -3627,6 +3877,17 @@ SELECT list_intersect(make_array(1,2,3), make_array(2,3,4)), ---- [2, 3] [] [aa, cc] [true] [2.2, 3.3] [[2, 2], [3, 3]] +query ?????? +SELECT list_intersect(arrow_cast(make_array(1,2,3), 'LargeList(Int64)'), arrow_cast(make_array(2,3,4), 'LargeList(Int64)')), + list_intersect(arrow_cast(make_array(1,3,5), 'LargeList(Int64)'), arrow_cast(make_array(2,4,6), 'LargeList(Int64)')), + list_intersect(arrow_cast(make_array('aa','bb','cc'), 'LargeList(Utf8)'), arrow_cast(make_array('cc','aa','dd'), 'LargeList(Utf8)')), + list_intersect(arrow_cast(make_array(true, false), 'LargeList(Boolean)'), arrow_cast(make_array(true), 'LargeList(Boolean)')), + list_intersect(arrow_cast(make_array(1.1, 2.2, 3.3), 'LargeList(Float64)'), arrow_cast(make_array(2.2, 3.3, 4.4), 'LargeList(Float64)')), + list_intersect(arrow_cast(make_array([1, 1], [2, 2], [3, 3]), 'LargeList(List(Int64))'), arrow_cast(make_array([2, 2], [3, 3], [4, 4]), 'LargeList(List(Int64))')) +; +---- +[2, 3] [] [aa, cc] [true] [2.2, 3.3] [[2, 2], [3, 3]] + query BBBB select list_has_all(make_array(1,2,3), make_array(4,5,6)), list_has_all(make_array(1,2,3), make_array(1,2)), @@ -4106,24 +4367,45 @@ drop table array_has_table_3D; statement ok drop table array_intersect_table_1D; +statement ok +drop table large_array_intersect_table_1D; + statement ok drop table array_intersect_table_1D_Float; +statement ok +drop table large_array_intersect_table_1D_Float; + statement ok drop table array_intersect_table_1D_Boolean; +statement ok +drop table large_array_intersect_table_1D_Boolean; + statement ok drop table array_intersect_table_1D_UTF8; +statement ok +drop table large_array_intersect_table_1D_UTF8; + statement ok drop table array_intersect_table_2D; +statement ok +drop table large_array_intersect_table_2D; + statement ok drop table array_intersect_table_2D_float; +statement ok +drop table large_array_intersect_table_2D_float; + statement ok drop table array_intersect_table_3D; +statement ok +drop table large_array_intersect_table_3D; + statement ok drop table arrays_values_without_nulls; From bb99d2a97df3c654ee8c1d5520ffd15ef5612193 Mon Sep 17 00:00:00 2001 From: Chih Wang Date: Wed, 27 Dec 2023 22:56:14 +0800 Subject: [PATCH 503/572] Avoid extra clone in datafusion-proto::physical_plan (#8650) --- datafusion/proto/src/physical_plan/mod.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/datafusion/proto/src/physical_plan/mod.rs b/datafusion/proto/src/physical_plan/mod.rs index df01097cfa78..24ede3fcaf62 100644 --- a/datafusion/proto/src/physical_plan/mod.rs +++ b/datafusion/proto/src/physical_plan/mod.rs @@ -486,7 +486,7 @@ impl AsExecutionPlan for PhysicalPlanNode { physical_aggr_expr, physical_filter_expr, input, - Arc::new(input_schema.try_into()?), + physical_schema, )?)) } PhysicalPlanType::HashJoin(hashjoin) => { From 28ca6d1ad9692d0f159ed1f1f45a20c0998a47ea Mon Sep 17 00:00:00 2001 From: Andrew Lamb Date: Wed, 27 Dec 2023 10:08:39 -0500 Subject: [PATCH 504/572] Minor: name some constant values in arrow writer, parquet writer (#8642) * Minor: name some constant values in arrow writer * Add constants to parquet.rs, update doc comments * fix --- .../core/src/datasource/file_format/arrow.rs | 13 ++++++++++--- .../core/src/datasource/file_format/avro.rs | 2 +- .../core/src/datasource/file_format/csv.rs | 2 +- .../core/src/datasource/file_format/json.rs | 2 +- .../src/datasource/file_format/parquet.rs | 19 +++++++++++++++---- 5 files changed, 28 insertions(+), 10 deletions(-) diff --git a/datafusion/core/src/datasource/file_format/arrow.rs b/datafusion/core/src/datasource/file_format/arrow.rs index 7d393d9129dd..650f8c844eda 100644 --- a/datafusion/core/src/datasource/file_format/arrow.rs +++ b/datafusion/core/src/datasource/file_format/arrow.rs @@ -15,7 +15,7 @@ // specific language governing permissions and limitations // under the License. -//! Apache Arrow format abstractions +//! [`ArrowFormat`]: Apache Arrow [`FileFormat`] abstractions //! //! Works with files following the [Arrow IPC format](https://arrow.apache.org/docs/format/Columnar.html#ipc-file-format) @@ -58,6 +58,13 @@ use super::file_compression_type::FileCompressionType; use super::write::demux::start_demuxer_task; use super::write::{create_writer, SharedBuffer}; +/// Initial writing buffer size. Note this is just a size hint for efficiency. It +/// will grow beyond the set value if needed. +const INITIAL_BUFFER_BYTES: usize = 1048576; + +/// If the buffered Arrow data exceeds this size, it is flushed to object store +const BUFFER_FLUSH_BYTES: usize = 1024000; + /// Arrow `FileFormat` implementation. #[derive(Default, Debug)] pub struct ArrowFormat; @@ -239,7 +246,7 @@ impl DataSink for ArrowFileSink { IpcWriteOptions::try_new(64, false, arrow_ipc::MetadataVersion::V5)? .try_with_compression(Some(CompressionType::LZ4_FRAME))?; while let Some((path, mut rx)) = file_stream_rx.recv().await { - let shared_buffer = SharedBuffer::new(1048576); + let shared_buffer = SharedBuffer::new(INITIAL_BUFFER_BYTES); let mut arrow_writer = arrow_ipc::writer::FileWriter::try_new_with_options( shared_buffer.clone(), &self.get_writer_schema(), @@ -257,7 +264,7 @@ impl DataSink for ArrowFileSink { row_count += batch.num_rows(); arrow_writer.write(&batch)?; let mut buff_to_flush = shared_buffer.buffer.try_lock().unwrap(); - if buff_to_flush.len() > 1024000 { + if buff_to_flush.len() > BUFFER_FLUSH_BYTES { object_store_writer .write_all(buff_to_flush.as_slice()) .await?; diff --git a/datafusion/core/src/datasource/file_format/avro.rs b/datafusion/core/src/datasource/file_format/avro.rs index a24a28ad6fdd..6d424bf0b28f 100644 --- a/datafusion/core/src/datasource/file_format/avro.rs +++ b/datafusion/core/src/datasource/file_format/avro.rs @@ -15,7 +15,7 @@ // specific language governing permissions and limitations // under the License. -//! Apache Avro format abstractions +//! [`AvroFormat`] Apache Avro [`FileFormat`] abstractions use std::any::Any; use std::sync::Arc; diff --git a/datafusion/core/src/datasource/file_format/csv.rs b/datafusion/core/src/datasource/file_format/csv.rs index df6689af6b73..4033bcd3b557 100644 --- a/datafusion/core/src/datasource/file_format/csv.rs +++ b/datafusion/core/src/datasource/file_format/csv.rs @@ -15,7 +15,7 @@ // specific language governing permissions and limitations // under the License. -//! CSV format abstractions +//! [`CsvFormat`], Comma Separated Value (CSV) [`FileFormat`] abstractions use std::any::Any; use std::collections::HashSet; diff --git a/datafusion/core/src/datasource/file_format/json.rs b/datafusion/core/src/datasource/file_format/json.rs index 9893a1db45de..fcb1d5f8e527 100644 --- a/datafusion/core/src/datasource/file_format/json.rs +++ b/datafusion/core/src/datasource/file_format/json.rs @@ -15,7 +15,7 @@ // specific language governing permissions and limitations // under the License. -//! Line delimited JSON format abstractions +//! [`JsonFormat`]: Line delimited JSON [`FileFormat`] abstractions use std::any::Any; use std::fmt; diff --git a/datafusion/core/src/datasource/file_format/parquet.rs b/datafusion/core/src/datasource/file_format/parquet.rs index 0c813b6ccbf0..7044acccd6dc 100644 --- a/datafusion/core/src/datasource/file_format/parquet.rs +++ b/datafusion/core/src/datasource/file_format/parquet.rs @@ -15,7 +15,7 @@ // specific language governing permissions and limitations // under the License. -//! Parquet format abstractions +//! [`ParquetFormat`]: Parquet [`FileFormat`] abstractions use arrow_array::RecordBatch; use async_trait::async_trait; @@ -75,6 +75,17 @@ use crate::physical_plan::{ Statistics, }; +/// Size of the buffer for [`AsyncArrowWriter`]. +const PARQUET_WRITER_BUFFER_SIZE: usize = 10485760; + +/// Initial writing buffer size. Note this is just a size hint for efficiency. It +/// will grow beyond the set value if needed. +const INITIAL_BUFFER_BYTES: usize = 1048576; + +/// When writing parquet files in parallel, if the buffered Parquet data exceeds +/// this size, it is flushed to object store +const BUFFER_FLUSH_BYTES: usize = 1024000; + /// The Apache Parquet `FileFormat` implementation /// /// Note it is recommended these are instead configured on the [`ConfigOptions`] @@ -680,7 +691,7 @@ impl ParquetSink { let writer = AsyncArrowWriter::try_new( multipart_writer, self.get_writer_schema(), - 10485760, + PARQUET_WRITER_BUFFER_SIZE, Some(parquet_props), )?; Ok(writer) @@ -1004,7 +1015,7 @@ async fn concatenate_parallel_row_groups( writer_props: Arc, mut object_store_writer: AbortableWrite>, ) -> Result { - let merged_buff = SharedBuffer::new(1048576); + let merged_buff = SharedBuffer::new(INITIAL_BUFFER_BYTES); let schema_desc = arrow_to_parquet_schema(schema.as_ref())?; let mut parquet_writer = SerializedFileWriter::new( @@ -1025,7 +1036,7 @@ async fn concatenate_parallel_row_groups( for chunk in serialized_columns { chunk.append_to_row_group(&mut rg_out)?; let mut buff_to_flush = merged_buff.buffer.try_lock().unwrap(); - if buff_to_flush.len() > 1024000 { + if buff_to_flush.len() > BUFFER_FLUSH_BYTES { object_store_writer .write_all(buff_to_flush.as_slice()) .await?; From 6403222c1eda8ed3438fe2555229319b92bfa056 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Berkay=20=C5=9Eahin?= <124376117+berkaysynnada@users.noreply.github.com> Date: Wed, 27 Dec 2023 23:18:27 +0300 Subject: [PATCH 505/572] TreeNode Refactor Part 2 (#8653) * Refactor TreeNode's * Update utils.rs * Final review * Remove unnecessary clones, more idiomatic Rust --------- Co-authored-by: Mehmet Ozan Kabak --- .../enforce_distribution.rs | 767 ++++++++---------- .../src/physical_optimizer/enforce_sorting.rs | 554 +++++++------ .../physical_optimizer/output_requirements.rs | 4 + .../physical_optimizer/pipeline_checker.rs | 32 +- .../replace_with_order_preserving_variants.rs | 292 +++---- .../src/physical_optimizer/sort_pushdown.rs | 138 ++-- .../core/src/physical_optimizer/utils.rs | 69 +- .../physical-expr/src/sort_properties.rs | 10 +- datafusion/physical-plan/src/union.rs | 20 +- 9 files changed, 872 insertions(+), 1014 deletions(-) diff --git a/datafusion/core/src/physical_optimizer/enforce_distribution.rs b/datafusion/core/src/physical_optimizer/enforce_distribution.rs index 0aef126578f3..d5a086227323 100644 --- a/datafusion/core/src/physical_optimizer/enforce_distribution.rs +++ b/datafusion/core/src/physical_optimizer/enforce_distribution.rs @@ -25,11 +25,11 @@ use std::fmt; use std::fmt::Formatter; use std::sync::Arc; +use super::output_requirements::OutputRequirementExec; use crate::config::ConfigOptions; use crate::error::Result; use crate::physical_optimizer::utils::{ - add_sort_above, get_children_exectrees, is_coalesce_partitions, is_repartition, - is_sort_preserving_merge, ExecTree, + is_coalesce_partitions, is_repartition, is_sort_preserving_merge, }; use crate::physical_optimizer::PhysicalOptimizerRule; use crate::physical_plan::aggregates::{AggregateExec, AggregateMode, PhysicalGroupBy}; @@ -52,8 +52,10 @@ use datafusion_expr::logical_plan::JoinType; use datafusion_physical_expr::expressions::{Column, NoOp}; use datafusion_physical_expr::utils::map_columns_before_projection; use datafusion_physical_expr::{ - physical_exprs_equal, EquivalenceProperties, PhysicalExpr, + physical_exprs_equal, EquivalenceProperties, LexRequirementRef, PhysicalExpr, + PhysicalSortRequirement, }; +use datafusion_physical_plan::sorts::sort::SortExec; use datafusion_physical_plan::windows::{get_best_fitting_window, BoundedWindowAggExec}; use datafusion_physical_plan::{get_plan_string, unbounded_output}; @@ -268,11 +270,12 @@ impl PhysicalOptimizerRule for EnforceDistribution { /// 5) For other types of operators, by default, pushdown the parent requirements to children. /// fn adjust_input_keys_ordering( - requirements: PlanWithKeyRequirements, + mut requirements: PlanWithKeyRequirements, ) -> Result> { let parent_required = requirements.required_key_ordering.clone(); let plan_any = requirements.plan.as_any(); - let transformed = if let Some(HashJoinExec { + + if let Some(HashJoinExec { left, right, on, @@ -287,7 +290,7 @@ fn adjust_input_keys_ordering( PartitionMode::Partitioned => { let join_constructor = |new_conditions: (Vec<(Column, Column)>, Vec)| { - Ok(Arc::new(HashJoinExec::try_new( + HashJoinExec::try_new( left.clone(), right.clone(), new_conditions.0, @@ -295,15 +298,17 @@ fn adjust_input_keys_ordering( join_type, PartitionMode::Partitioned, *null_equals_null, - )?) as Arc) + ) + .map(|e| Arc::new(e) as _) }; - Some(reorder_partitioned_join_keys( + reorder_partitioned_join_keys( requirements.plan.clone(), &parent_required, on, vec![], &join_constructor, - )?) + ) + .map(Transformed::Yes) } PartitionMode::CollectLeft => { let new_right_request = match join_type { @@ -321,15 +326,15 @@ fn adjust_input_keys_ordering( }; // Push down requirements to the right side - Some(PlanWithKeyRequirements { - plan: requirements.plan.clone(), - required_key_ordering: vec![], - request_key_ordering: vec![None, new_right_request], - }) + requirements.children[1].required_key_ordering = + new_right_request.unwrap_or(vec![]); + Ok(Transformed::Yes(requirements)) } PartitionMode::Auto => { // Can not satisfy, clear the current requirements and generate new empty requirements - Some(PlanWithKeyRequirements::new(requirements.plan.clone())) + Ok(Transformed::Yes(PlanWithKeyRequirements::new( + requirements.plan, + ))) } } } else if let Some(CrossJoinExec { left, .. }) = @@ -337,14 +342,9 @@ fn adjust_input_keys_ordering( { let left_columns_len = left.schema().fields().len(); // Push down requirements to the right side - Some(PlanWithKeyRequirements { - plan: requirements.plan.clone(), - required_key_ordering: vec![], - request_key_ordering: vec![ - None, - shift_right_required(&parent_required, left_columns_len), - ], - }) + requirements.children[1].required_key_ordering = + shift_right_required(&parent_required, left_columns_len).unwrap_or_default(); + Ok(Transformed::Yes(requirements)) } else if let Some(SortMergeJoinExec { left, right, @@ -357,35 +357,40 @@ fn adjust_input_keys_ordering( { let join_constructor = |new_conditions: (Vec<(Column, Column)>, Vec)| { - Ok(Arc::new(SortMergeJoinExec::try_new( + SortMergeJoinExec::try_new( left.clone(), right.clone(), new_conditions.0, *join_type, new_conditions.1, *null_equals_null, - )?) as Arc) + ) + .map(|e| Arc::new(e) as _) }; - Some(reorder_partitioned_join_keys( + reorder_partitioned_join_keys( requirements.plan.clone(), &parent_required, on, sort_options.clone(), &join_constructor, - )?) + ) + .map(Transformed::Yes) } else if let Some(aggregate_exec) = plan_any.downcast_ref::() { if !parent_required.is_empty() { match aggregate_exec.mode() { - AggregateMode::FinalPartitioned => Some(reorder_aggregate_keys( + AggregateMode::FinalPartitioned => reorder_aggregate_keys( requirements.plan.clone(), &parent_required, aggregate_exec, - )?), - _ => Some(PlanWithKeyRequirements::new(requirements.plan.clone())), + ) + .map(Transformed::Yes), + _ => Ok(Transformed::Yes(PlanWithKeyRequirements::new( + requirements.plan, + ))), } } else { // Keep everything unchanged - None + Ok(Transformed::No(requirements)) } } else if let Some(proj) = plan_any.downcast_ref::() { let expr = proj.expr(); @@ -394,34 +399,28 @@ fn adjust_input_keys_ordering( // Construct a mapping from new name to the the orginal Column let new_required = map_columns_before_projection(&parent_required, expr); if new_required.len() == parent_required.len() { - Some(PlanWithKeyRequirements { - plan: requirements.plan.clone(), - required_key_ordering: vec![], - request_key_ordering: vec![Some(new_required.clone())], - }) + requirements.children[0].required_key_ordering = new_required; + Ok(Transformed::Yes(requirements)) } else { // Can not satisfy, clear the current requirements and generate new empty requirements - Some(PlanWithKeyRequirements::new(requirements.plan.clone())) + Ok(Transformed::Yes(PlanWithKeyRequirements::new( + requirements.plan, + ))) } } else if plan_any.downcast_ref::().is_some() || plan_any.downcast_ref::().is_some() || plan_any.downcast_ref::().is_some() { - Some(PlanWithKeyRequirements::new(requirements.plan.clone())) + Ok(Transformed::Yes(PlanWithKeyRequirements::new( + requirements.plan, + ))) } else { // By default, push down the parent requirements to children - let children_len = requirements.plan.children().len(); - Some(PlanWithKeyRequirements { - plan: requirements.plan.clone(), - required_key_ordering: vec![], - request_key_ordering: vec![Some(parent_required.clone()); children_len], - }) - }; - Ok(if let Some(transformed) = transformed { - Transformed::Yes(transformed) - } else { - Transformed::No(requirements) - }) + requirements.children.iter_mut().for_each(|child| { + child.required_key_ordering = parent_required.clone(); + }); + Ok(Transformed::Yes(requirements)) + } } fn reorder_partitioned_join_keys( @@ -452,28 +451,24 @@ where for idx in 0..sort_options.len() { new_sort_options.push(sort_options[new_positions[idx]]) } - - Ok(PlanWithKeyRequirements { - plan: join_constructor((new_join_on, new_sort_options))?, - required_key_ordering: vec![], - request_key_ordering: vec![Some(left_keys), Some(right_keys)], - }) + let mut requirement_tree = PlanWithKeyRequirements::new(join_constructor(( + new_join_on, + new_sort_options, + ))?); + requirement_tree.children[0].required_key_ordering = left_keys; + requirement_tree.children[1].required_key_ordering = right_keys; + Ok(requirement_tree) } else { - Ok(PlanWithKeyRequirements { - plan: join_plan, - required_key_ordering: vec![], - request_key_ordering: vec![Some(left_keys), Some(right_keys)], - }) + let mut requirement_tree = PlanWithKeyRequirements::new(join_plan); + requirement_tree.children[0].required_key_ordering = left_keys; + requirement_tree.children[1].required_key_ordering = right_keys; + Ok(requirement_tree) } } else { - Ok(PlanWithKeyRequirements { - plan: join_plan, - required_key_ordering: vec![], - request_key_ordering: vec![ - Some(join_key_pairs.left_keys), - Some(join_key_pairs.right_keys), - ], - }) + let mut requirement_tree = PlanWithKeyRequirements::new(join_plan); + requirement_tree.children[0].required_key_ordering = join_key_pairs.left_keys; + requirement_tree.children[1].required_key_ordering = join_key_pairs.right_keys; + Ok(requirement_tree) } } @@ -868,59 +863,24 @@ fn new_join_conditions( .collect() } -/// Updates `dist_onward` such that, to keep track of -/// `input` in the `exec_tree`. -/// -/// # Arguments -/// -/// * `input`: Current execution plan -/// * `dist_onward`: It keeps track of executors starting from a distribution -/// changing operator (e.g Repartition, SortPreservingMergeExec, etc.) -/// until child of `input` (`input` should have single child). -/// * `input_idx`: index of the `input`, for its parent. -/// -fn update_distribution_onward( - input: Arc, - dist_onward: &mut Option, - input_idx: usize, -) { - // Update the onward tree if there is an active branch - if let Some(exec_tree) = dist_onward { - // When we add a new operator to change distribution - // we add RepartitionExec, SortPreservingMergeExec, CoalescePartitionsExec - // in this case, we need to update exec tree idx such that exec tree is now child of these - // operators (change the 0, since all of the operators have single child). - exec_tree.idx = 0; - *exec_tree = ExecTree::new(input, input_idx, vec![exec_tree.clone()]); - } else { - *dist_onward = Some(ExecTree::new(input, input_idx, vec![])); - } -} - /// Adds RoundRobin repartition operator to the plan increase parallelism. /// /// # Arguments /// -/// * `input`: Current execution plan +/// * `input`: Current node. /// * `n_target`: desired target partition number, if partition number of the /// current executor is less than this value. Partition number will be increased. -/// * `dist_onward`: It keeps track of executors starting from a distribution -/// changing operator (e.g Repartition, SortPreservingMergeExec, etc.) -/// until `input` plan. -/// * `input_idx`: index of the `input`, for its parent. /// /// # Returns /// -/// A [Result] object that contains new execution plan, where desired partition number -/// is achieved by adding RoundRobin Repartition. +/// A [`Result`] object that contains new execution plan where the desired +/// partition number is achieved by adding a RoundRobin repartition. fn add_roundrobin_on_top( - input: Arc, + input: DistributionContext, n_target: usize, - dist_onward: &mut Option, - input_idx: usize, -) -> Result> { - // Adding repartition is helpful - if input.output_partitioning().partition_count() < n_target { +) -> Result { + // Adding repartition is helpful: + if input.plan.output_partitioning().partition_count() < n_target { // When there is an existing ordering, we preserve ordering // during repartition. This will be un-done in the future // If any of the following conditions is true @@ -928,13 +888,16 @@ fn add_roundrobin_on_top( // - Usage of order preserving variants is not desirable // (determined by flag `config.optimizer.prefer_existing_sort`) let partitioning = Partitioning::RoundRobinBatch(n_target); - let repartition = - RepartitionExec::try_new(input, partitioning)?.with_preserve_order(); + let repartition = RepartitionExec::try_new(input.plan.clone(), partitioning)? + .with_preserve_order(); - // update distribution onward with new operator - let new_plan = Arc::new(repartition) as Arc; - update_distribution_onward(new_plan.clone(), dist_onward, input_idx); - Ok(new_plan) + let new_plan = Arc::new(repartition) as _; + + Ok(DistributionContext { + plan: new_plan, + distribution_connection: true, + children_nodes: vec![input], + }) } else { // Partition is not helpful, we already have desired number of partitions. Ok(input) @@ -948,46 +911,38 @@ fn add_roundrobin_on_top( /// /// # Arguments /// -/// * `input`: Current execution plan +/// * `input`: Current node. /// * `hash_exprs`: Stores Physical Exprs that are used during hashing. /// * `n_target`: desired target partition number, if partition number of the /// current executor is less than this value. Partition number will be increased. -/// * `dist_onward`: It keeps track of executors starting from a distribution -/// changing operator (e.g Repartition, SortPreservingMergeExec, etc.) -/// until `input` plan. -/// * `input_idx`: index of the `input`, for its parent. /// /// # Returns /// -/// A [`Result`] object that contains new execution plan, where desired distribution is -/// satisfied by adding Hash Repartition. +/// A [`Result`] object that contains new execution plan where the desired +/// distribution is satisfied by adding a Hash repartition. fn add_hash_on_top( - input: Arc, + mut input: DistributionContext, hash_exprs: Vec>, - // Repartition(Hash) will have `n_target` partitions at the output. n_target: usize, - // Stores executors starting from Repartition(RoundRobin) until - // current executor. When Repartition(Hash) is added, `dist_onward` - // is updated such that it stores connection from Repartition(RoundRobin) - // until Repartition(Hash). - dist_onward: &mut Option, - input_idx: usize, repartition_beneficial_stats: bool, -) -> Result> { - if n_target == input.output_partitioning().partition_count() && n_target == 1 { - // In this case adding a hash repartition is unnecessary as the hash - // requirement is implicitly satisfied. +) -> Result { + let partition_count = input.plan.output_partitioning().partition_count(); + // Early return if hash repartition is unnecessary + if n_target == partition_count && n_target == 1 { return Ok(input); } + let satisfied = input + .plan .output_partitioning() .satisfy(Distribution::HashPartitioned(hash_exprs.clone()), || { - input.equivalence_properties() + input.plan.equivalence_properties() }); + // Add hash repartitioning when: // - The hash distribution requirement is not satisfied, or // - We can increase parallelism by adding hash partitioning. - if !satisfied || n_target > input.output_partitioning().partition_count() { + if !satisfied || n_target > input.plan.output_partitioning().partition_count() { // When there is an existing ordering, we preserve ordering during // repartition. This will be rolled back in the future if any of the // following conditions is true: @@ -995,75 +950,66 @@ fn add_hash_on_top( // requirements. // - Usage of order preserving variants is not desirable (per the flag // `config.optimizer.prefer_existing_sort`). - let mut new_plan = if repartition_beneficial_stats { + if repartition_beneficial_stats { // Since hashing benefits from partitioning, add a round-robin repartition // before it: - add_roundrobin_on_top(input, n_target, dist_onward, 0)? - } else { - input - }; + input = add_roundrobin_on_top(input, n_target)?; + } + let partitioning = Partitioning::Hash(hash_exprs, n_target); - let repartition = RepartitionExec::try_new(new_plan, partitioning)? - // preserve any ordering if possible + let repartition = RepartitionExec::try_new(input.plan.clone(), partitioning)? .with_preserve_order(); - new_plan = Arc::new(repartition) as _; - // update distribution onward with new operator - update_distribution_onward(new_plan.clone(), dist_onward, input_idx); - Ok(new_plan) - } else { - Ok(input) + input.children_nodes = vec![input.clone()]; + input.distribution_connection = true; + input.plan = Arc::new(repartition) as _; } + + Ok(input) } -/// Adds a `SortPreservingMergeExec` operator on top of input executor: -/// - to satisfy single distribution requirement. +/// Adds a [`SortPreservingMergeExec`] operator on top of input executor +/// to satisfy single distribution requirement. /// /// # Arguments /// -/// * `input`: Current execution plan -/// * `dist_onward`: It keeps track of executors starting from a distribution -/// changing operator (e.g Repartition, SortPreservingMergeExec, etc.) -/// until `input` plan. -/// * `input_idx`: index of the `input`, for its parent. +/// * `input`: Current node. /// /// # Returns /// -/// New execution plan, where desired single -/// distribution is satisfied by adding `SortPreservingMergeExec`. -fn add_spm_on_top( - input: Arc, - dist_onward: &mut Option, - input_idx: usize, -) -> Arc { +/// Updated node with an execution plan, where desired single +/// distribution is satisfied by adding [`SortPreservingMergeExec`]. +fn add_spm_on_top(input: DistributionContext) -> DistributionContext { // Add SortPreservingMerge only when partition count is larger than 1. - if input.output_partitioning().partition_count() > 1 { + if input.plan.output_partitioning().partition_count() > 1 { // When there is an existing ordering, we preserve ordering - // during decreasıng partıtıons. This will be un-done in the future - // If any of the following conditions is true + // when decreasing partitions. This will be un-done in the future + // if any of the following conditions is true // - Preserving ordering is not helpful in terms of satisfying ordering requirements // - Usage of order preserving variants is not desirable - // (determined by flag `config.optimizer.prefer_existing_sort`) - let should_preserve_ordering = input.output_ordering().is_some(); - let new_plan: Arc = if should_preserve_ordering { - let existing_ordering = input.output_ordering().unwrap_or(&[]); + // (determined by flag `config.optimizer.bounded_order_preserving_variants`) + let should_preserve_ordering = input.plan.output_ordering().is_some(); + + let new_plan = if should_preserve_ordering { Arc::new(SortPreservingMergeExec::new( - existing_ordering.to_vec(), - input, + input.plan.output_ordering().unwrap_or(&[]).to_vec(), + input.plan.clone(), )) as _ } else { - Arc::new(CoalescePartitionsExec::new(input)) as _ + Arc::new(CoalescePartitionsExec::new(input.plan.clone())) as _ }; - // update repartition onward with new operator - update_distribution_onward(new_plan.clone(), dist_onward, input_idx); - new_plan + DistributionContext { + plan: new_plan, + distribution_connection: true, + children_nodes: vec![input], + } } else { input } } -/// Updates the physical plan inside `distribution_context` so that distribution +/// Updates the physical plan inside [`DistributionContext`] so that distribution /// changing operators are removed from the top. If they are necessary, they will /// be added in subsequent stages. /// @@ -1081,48 +1027,23 @@ fn add_spm_on_top( /// "ParquetExec: file_groups={2 groups: \[\[x], \[y]]}, projection=\[a, b, c, d, e], output_ordering=\[a@0 ASC]", /// ``` fn remove_dist_changing_operators( - distribution_context: DistributionContext, + mut distribution_context: DistributionContext, ) -> Result { - let DistributionContext { - mut plan, - mut distribution_onwards, - } = distribution_context; - - // Remove any distribution changing operators at the beginning: - // Note that they will be re-inserted later on if necessary or helpful. - while is_repartition(&plan) - || is_coalesce_partitions(&plan) - || is_sort_preserving_merge(&plan) + while is_repartition(&distribution_context.plan) + || is_coalesce_partitions(&distribution_context.plan) + || is_sort_preserving_merge(&distribution_context.plan) { - // All of above operators have a single child. When we remove the top - // operator, we take the first child. - plan = plan.children().swap_remove(0); - distribution_onwards = - get_children_exectrees(plan.children().len(), &distribution_onwards[0]); + // All of above operators have a single child. First child is only child. + let child = distribution_context.children_nodes.swap_remove(0); + // Remove any distribution changing operators at the beginning: + // Note that they will be re-inserted later on if necessary or helpful. + distribution_context = child; } - // Create a plan with the updated children: - Ok(DistributionContext { - plan, - distribution_onwards, - }) + Ok(distribution_context) } -/// Updates the physical plan `input` by using `dist_onward` replace order preserving operator variants -/// with their corresponding operators that do not preserve order. It is a wrapper for `replace_order_preserving_variants_helper` -fn replace_order_preserving_variants( - input: &mut Arc, - dist_onward: &mut Option, -) -> Result<()> { - if let Some(dist_onward) = dist_onward { - *input = replace_order_preserving_variants_helper(dist_onward)?; - } - *dist_onward = None; - Ok(()) -} - -/// Updates the physical plan inside `ExecTree` if preserving ordering while changing partitioning -/// is not helpful or desirable. +/// Updates the [`DistributionContext`] if preserving ordering while changing partitioning is not helpful or desirable. /// /// Assume that following plan is given: /// ```text @@ -1132,7 +1053,7 @@ fn replace_order_preserving_variants( /// " ParquetExec: file_groups={2 groups: \[\[x], \[y]]}, projection=\[a, b, c, d, e], output_ordering=\[a@0 ASC]", /// ``` /// -/// This function converts plan above (inside `ExecTree`) to the following: +/// This function converts plan above to the following: /// /// ```text /// "CoalescePartitionsExec" @@ -1140,30 +1061,75 @@ fn replace_order_preserving_variants( /// " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=2", /// " ParquetExec: file_groups={2 groups: \[\[x], \[y]]}, projection=\[a, b, c, d, e], output_ordering=\[a@0 ASC]", /// ``` -fn replace_order_preserving_variants_helper( - exec_tree: &ExecTree, -) -> Result> { - let mut updated_children = exec_tree.plan.children(); - for child in &exec_tree.children { - updated_children[child.idx] = replace_order_preserving_variants_helper(child)?; - } - if is_sort_preserving_merge(&exec_tree.plan) { - return Ok(Arc::new(CoalescePartitionsExec::new( - updated_children.swap_remove(0), - ))); - } - if let Some(repartition) = exec_tree.plan.as_any().downcast_ref::() { +fn replace_order_preserving_variants( + mut context: DistributionContext, +) -> Result { + let mut updated_children = context + .children_nodes + .iter() + .map(|child| { + if child.distribution_connection { + replace_order_preserving_variants(child.clone()) + } else { + Ok(child.clone()) + } + }) + .collect::>>()?; + + if is_sort_preserving_merge(&context.plan) { + let child = updated_children.swap_remove(0); + context.plan = Arc::new(CoalescePartitionsExec::new(child.plan.clone())); + context.children_nodes = vec![child]; + return Ok(context); + } else if let Some(repartition) = + context.plan.as_any().downcast_ref::() + { if repartition.preserve_order() { - return Ok(Arc::new( - // new RepartitionExec don't preserve order - RepartitionExec::try_new( - updated_children.swap_remove(0), - repartition.partitioning().clone(), - )?, - )); + let child = updated_children.swap_remove(0); + context.plan = Arc::new(RepartitionExec::try_new( + child.plan.clone(), + repartition.partitioning().clone(), + )?); + context.children_nodes = vec![child]; + return Ok(context); + } + } + + context.plan = context + .plan + .clone() + .with_new_children(updated_children.into_iter().map(|c| c.plan).collect())?; + Ok(context) +} + +/// This utility function adds a [`SortExec`] above an operator according to the +/// given ordering requirements while preserving the original partitioning. +fn add_sort_preserving_partitions( + node: DistributionContext, + sort_requirement: LexRequirementRef, + fetch: Option, +) -> DistributionContext { + // If the ordering requirement is already satisfied, do not add a sort. + if !node + .plan + .equivalence_properties() + .ordering_satisfy_requirement(sort_requirement) + { + let sort_expr = PhysicalSortRequirement::to_sort_exprs(sort_requirement.to_vec()); + let new_sort = SortExec::new(sort_expr, node.plan.clone()).with_fetch(fetch); + + DistributionContext { + plan: Arc::new(if node.plan.output_partitioning().partition_count() > 1 { + new_sort.with_preserve_partitioning(true) + } else { + new_sort + }), + distribution_connection: false, + children_nodes: vec![node], } + } else { + node } - exec_tree.plan.clone().with_new_children(updated_children) } /// This function checks whether we need to add additional data exchange @@ -1174,6 +1140,12 @@ fn ensure_distribution( dist_context: DistributionContext, config: &ConfigOptions, ) -> Result> { + let dist_context = dist_context.update_children()?; + + if dist_context.plan.children().is_empty() { + return Ok(Transformed::No(dist_context)); + } + let target_partitions = config.execution.target_partitions; // When `false`, round robin repartition will not be added to increase parallelism let enable_round_robin = config.optimizer.enable_round_robin_repartition; @@ -1186,14 +1158,11 @@ fn ensure_distribution( let order_preserving_variants_desirable = is_unbounded || config.optimizer.prefer_existing_sort; - if dist_context.plan.children().is_empty() { - return Ok(Transformed::No(dist_context)); - } - // Remove unnecessary repartition from the physical plan if any let DistributionContext { mut plan, - mut distribution_onwards, + distribution_connection, + children_nodes, } = remove_dist_changing_operators(dist_context)?; if let Some(exec) = plan.as_any().downcast_ref::() { @@ -1213,33 +1182,23 @@ fn ensure_distribution( plan = updated_window; } }; - let n_children = plan.children().len(); + // This loop iterates over all the children to: // - Increase parallelism for every child if it is beneficial. // - Satisfy the distribution requirements of every child, if it is not // already satisfied. // We store the updated children in `new_children`. - let new_children = izip!( - plan.children().into_iter(), + let children_nodes = izip!( + children_nodes.into_iter(), plan.required_input_distribution().iter(), plan.required_input_ordering().iter(), - distribution_onwards.iter_mut(), plan.benefits_from_input_partitioning(), - plan.maintains_input_order(), - 0..n_children + plan.maintains_input_order() ) .map( - |( - mut child, - requirement, - required_input_ordering, - dist_onward, - would_benefit, - maintains, - child_idx, - )| { + |(mut child, requirement, required_input_ordering, would_benefit, maintains)| { // Don't need to apply when the returned row count is not greater than 1: - let num_rows = child.statistics()?.num_rows; + let num_rows = child.plan.statistics()?.num_rows; let repartition_beneficial_stats = if num_rows.is_exact().unwrap_or(false) { num_rows .get_value() @@ -1248,45 +1207,39 @@ fn ensure_distribution( } else { true }; + if enable_round_robin // Operator benefits from partitioning (e.g. filter): && (would_benefit && repartition_beneficial_stats) // Unless partitioning doesn't increase the partition count, it is not beneficial: - && child.output_partitioning().partition_count() < target_partitions + && child.plan.output_partitioning().partition_count() < target_partitions { // When `repartition_file_scans` is set, attempt to increase // parallelism at the source. if repartition_file_scans { if let Some(new_child) = - child.repartitioned(target_partitions, config)? + child.plan.repartitioned(target_partitions, config)? { - child = new_child; + child.plan = new_child; } } // Increase parallelism by adding round-robin repartitioning // on top of the operator. Note that we only do this if the // partition count is not already equal to the desired partition // count. - child = add_roundrobin_on_top( - child, - target_partitions, - dist_onward, - child_idx, - )?; + child = add_roundrobin_on_top(child, target_partitions)?; } // Satisfy the distribution requirement if it is unmet. match requirement { Distribution::SinglePartition => { - child = add_spm_on_top(child, dist_onward, child_idx); + child = add_spm_on_top(child); } Distribution::HashPartitioned(exprs) => { child = add_hash_on_top( child, exprs.to_vec(), target_partitions, - dist_onward, - child_idx, repartition_beneficial_stats, )?; } @@ -1299,31 +1252,38 @@ fn ensure_distribution( // - Ordering requirement cannot be satisfied by preserving ordering through repartitions, or // - using order preserving variant is not desirable. let ordering_satisfied = child + .plan .equivalence_properties() .ordering_satisfy_requirement(required_input_ordering); - if !ordering_satisfied || !order_preserving_variants_desirable { - replace_order_preserving_variants(&mut child, dist_onward)?; + if (!ordering_satisfied || !order_preserving_variants_desirable) + && child.distribution_connection + { + child = replace_order_preserving_variants(child)?; // If ordering requirements were satisfied before repartitioning, // make sure ordering requirements are still satisfied after. if ordering_satisfied { // Make sure to satisfy ordering requirement: - add_sort_above(&mut child, required_input_ordering, None); + child = add_sort_preserving_partitions( + child, + required_input_ordering, + None, + ); } } // Stop tracking distribution changing operators - *dist_onward = None; + child.distribution_connection = false; } else { // no ordering requirement match requirement { // Operator requires specific distribution. Distribution::SinglePartition | Distribution::HashPartitioned(_) => { // Since there is no ordering requirement, preserving ordering is pointless - replace_order_preserving_variants(&mut child, dist_onward)?; + child = replace_order_preserving_variants(child)?; } Distribution::UnspecifiedDistribution => { // Since ordering is lost, trying to preserve ordering is pointless - if !maintains { - replace_order_preserving_variants(&mut child, dist_onward)?; + if !maintains || plan.as_any().is::() { + child = replace_order_preserving_variants(child)?; } } } @@ -1334,7 +1294,9 @@ fn ensure_distribution( .collect::>>()?; let new_distribution_context = DistributionContext { - plan: if plan.as_any().is::() && can_interleave(&new_children) { + plan: if plan.as_any().is::() + && can_interleave(children_nodes.iter().map(|c| c.plan.clone())) + { // Add a special case for [`UnionExec`] since we want to "bubble up" // hash-partitioned data. So instead of // @@ -1358,120 +1320,91 @@ fn ensure_distribution( // - Agg: // Repartition (hash): // Data - Arc::new(InterleaveExec::try_new(new_children)?) + Arc::new(InterleaveExec::try_new( + children_nodes.iter().map(|c| c.plan.clone()).collect(), + )?) } else { - plan.with_new_children(new_children)? + plan.with_new_children( + children_nodes.iter().map(|c| c.plan.clone()).collect(), + )? }, - distribution_onwards, + distribution_connection, + children_nodes, }; + Ok(Transformed::Yes(new_distribution_context)) } -/// A struct to keep track of distribution changing executors +/// A struct to keep track of distribution changing operators /// (`RepartitionExec`, `SortPreservingMergeExec`, `CoalescePartitionsExec`), /// and their associated parents inside `plan`. Using this information, /// we can optimize distribution of the plan if/when necessary. #[derive(Debug, Clone)] struct DistributionContext { plan: Arc, - /// Keep track of associations for each child of the plan. If `None`, - /// there is no distribution changing operator in its descendants. - distribution_onwards: Vec>, + /// Indicates whether this plan is connected to a distribution-changing + /// operator. + distribution_connection: bool, + children_nodes: Vec, } impl DistributionContext { - /// Creates an empty context. + /// Creates a tree according to the plan with empty states. fn new(plan: Arc) -> Self { - let length = plan.children().len(); - DistributionContext { + let children = plan.children(); + Self { plan, - distribution_onwards: vec![None; length], + distribution_connection: false, + children_nodes: children.into_iter().map(Self::new).collect(), } } - /// Constructs a new context from children contexts. - fn new_from_children_nodes( - children_nodes: Vec, - parent_plan: Arc, - ) -> Result { - let children_plans = children_nodes - .iter() - .map(|item| item.plan.clone()) - .collect(); - let distribution_onwards = children_nodes - .into_iter() - .enumerate() - .map(|(idx, context)| { - let DistributionContext { - plan, - // The `distribution_onwards` tree keeps track of operators - // that change distribution, or preserves the existing - // distribution (starting from an operator that change distribution). - distribution_onwards, - } = context; - if plan.children().is_empty() { - // Plan has no children, there is nothing to propagate. - None - } else if distribution_onwards[0].is_none() { - if let Some(repartition) = - plan.as_any().downcast_ref::() - { - match repartition.partitioning() { - Partitioning::RoundRobinBatch(_) - | Partitioning::Hash(_, _) => { - // Start tracking operators starting from this repartition (either roundrobin or hash): - return Some(ExecTree::new(plan, idx, vec![])); - } - _ => {} - } - } else if plan.as_any().is::() - || plan.as_any().is::() - { - // Start tracking operators starting from this sort preserving merge: - return Some(ExecTree::new(plan, idx, vec![])); - } - None - } else { - // Propagate children distribution tracking to the above - let new_distribution_onwards = izip!( - plan.required_input_distribution().iter(), - distribution_onwards.into_iter() - ) - .flat_map(|(required_dist, distribution_onwards)| { - if let Some(distribution_onwards) = distribution_onwards { - // Operator can safely propagate the distribution above. - // This is similar to maintaining order in the EnforceSorting rule. - if let Distribution::UnspecifiedDistribution = required_dist { - return Some(distribution_onwards); - } - } - None - }) - .collect::>(); - // Either: - // - None of the children has a connection to an operator that modifies distribution, or - // - The current operator requires distribution at its input so doesn't propagate it above. - if new_distribution_onwards.is_empty() { - None - } else { - Some(ExecTree::new(plan, idx, new_distribution_onwards)) - } + fn update_children(mut self) -> Result { + for child_context in self.children_nodes.iter_mut() { + child_context.distribution_connection = match child_context.plan.as_any() { + plan_any if plan_any.is::() => matches!( + plan_any + .downcast_ref::() + .unwrap() + .partitioning(), + Partitioning::RoundRobinBatch(_) | Partitioning::Hash(_, _) + ), + plan_any + if plan_any.is::() + || plan_any.is::() => + { + true } - }) - .collect(); - Ok(DistributionContext { - plan: with_new_children_if_necessary(parent_plan, children_plans)?.into(), - distribution_onwards, - }) - } + _ => { + child_context.plan.children().is_empty() + || child_context.children_nodes[0].distribution_connection + || child_context + .plan + .required_input_distribution() + .iter() + .zip(child_context.children_nodes.iter()) + .any(|(required_dist, child_context)| { + child_context.distribution_connection + && matches!( + required_dist, + Distribution::UnspecifiedDistribution + ) + }) + } + }; + } - /// Computes distribution tracking contexts for every child of the plan. - fn children(&self) -> Vec { - self.plan - .children() - .into_iter() - .map(DistributionContext::new) - .collect() + let children_plans = self + .children_nodes + .iter() + .map(|context| context.plan.clone()) + .collect::>(); + + Ok(Self { + plan: with_new_children_if_necessary(self.plan, children_plans)?.into(), + distribution_connection: false, + children_nodes: self.children_nodes, + }) } } @@ -1480,8 +1413,8 @@ impl TreeNode for DistributionContext { where F: FnMut(&Self) -> Result, { - for child in self.children() { - match op(&child)? { + for child in &self.children_nodes { + match op(child)? { VisitRecursion::Continue => {} VisitRecursion::Skip => return Ok(VisitRecursion::Continue), VisitRecursion::Stop => return Ok(VisitRecursion::Stop), @@ -1490,20 +1423,23 @@ impl TreeNode for DistributionContext { Ok(VisitRecursion::Continue) } - fn map_children(self, transform: F) -> Result + fn map_children(mut self, transform: F) -> Result where F: FnMut(Self) -> Result, { - let children = self.children(); - if children.is_empty() { - Ok(self) - } else { - let children_nodes = children + if !self.children_nodes.is_empty() { + self.children_nodes = self + .children_nodes .into_iter() .map(transform) - .collect::>>()?; - DistributionContext::new_from_children_nodes(children_nodes, self.plan) + .collect::>()?; + self.plan = with_new_children_if_necessary( + self.plan, + self.children_nodes.iter().map(|c| c.plan.clone()).collect(), + )? + .into(); } + Ok(self) } } @@ -1512,11 +1448,11 @@ impl fmt::Display for DistributionContext { fn fmt(&self, f: &mut Formatter<'_>) -> fmt::Result { let plan_string = get_plan_string(&self.plan); write!(f, "plan: {:?}", plan_string)?; - for (idx, child) in self.distribution_onwards.iter().enumerate() { - if let Some(child) = child { - write!(f, "idx:{:?}, exec_tree:{}", idx, child)?; - } - } + write!( + f, + "distribution_connection:{}", + self.distribution_connection, + )?; write!(f, "") } } @@ -1532,37 +1468,18 @@ struct PlanWithKeyRequirements { plan: Arc, /// Parent required key ordering required_key_ordering: Vec>, - /// The request key ordering to children - request_key_ordering: Vec>>>, + children: Vec, } impl PlanWithKeyRequirements { fn new(plan: Arc) -> Self { - let children_len = plan.children().len(); - PlanWithKeyRequirements { + let children = plan.children(); + Self { plan, required_key_ordering: vec![], - request_key_ordering: vec![None; children_len], + children: children.into_iter().map(Self::new).collect(), } } - - fn children(&self) -> Vec { - let plan_children = self.plan.children(); - assert_eq!(plan_children.len(), self.request_key_ordering.len()); - plan_children - .into_iter() - .zip(self.request_key_ordering.clone()) - .map(|(child, required)| { - let from_parent = required.unwrap_or_default(); - let length = child.children().len(); - PlanWithKeyRequirements { - plan: child, - required_key_ordering: from_parent, - request_key_ordering: vec![None; length], - } - }) - .collect() - } } impl TreeNode for PlanWithKeyRequirements { @@ -1570,9 +1487,8 @@ impl TreeNode for PlanWithKeyRequirements { where F: FnMut(&Self) -> Result, { - let children = self.children(); - for child in children { - match op(&child)? { + for child in &self.children { + match op(child)? { VisitRecursion::Continue => {} VisitRecursion::Skip => return Ok(VisitRecursion::Continue), VisitRecursion::Stop => return Ok(VisitRecursion::Stop), @@ -1582,28 +1498,23 @@ impl TreeNode for PlanWithKeyRequirements { Ok(VisitRecursion::Continue) } - fn map_children(self, transform: F) -> Result + fn map_children(mut self, transform: F) -> Result where F: FnMut(Self) -> Result, { - let children = self.children(); - if !children.is_empty() { - let new_children: Result> = - children.into_iter().map(transform).collect(); - - let children_plans = new_children? + if !self.children.is_empty() { + self.children = self + .children .into_iter() - .map(|child| child.plan) - .collect::>(); - let new_plan = with_new_children_if_necessary(self.plan, children_plans)?; - Ok(PlanWithKeyRequirements { - plan: new_plan.into(), - required_key_ordering: self.required_key_ordering, - request_key_ordering: self.request_key_ordering, - }) - } else { - Ok(self) + .map(transform) + .collect::>()?; + self.plan = with_new_children_if_necessary( + self.plan, + self.children.iter().map(|c| c.plan.clone()).collect(), + )? + .into(); } + Ok(self) } } diff --git a/datafusion/core/src/physical_optimizer/enforce_sorting.rs b/datafusion/core/src/physical_optimizer/enforce_sorting.rs index 2ecc1e11b985..77d04a61c59e 100644 --- a/datafusion/core/src/physical_optimizer/enforce_sorting.rs +++ b/datafusion/core/src/physical_optimizer/enforce_sorting.rs @@ -44,7 +44,7 @@ use crate::physical_optimizer::replace_with_order_preserving_variants::{ use crate::physical_optimizer::sort_pushdown::{pushdown_sorts, SortPushDown}; use crate::physical_optimizer::utils::{ add_sort_above, is_coalesce_partitions, is_limit, is_repartition, is_sort, - is_sort_preserving_merge, is_union, is_window, ExecTree, + is_sort_preserving_merge, is_union, is_window, }; use crate::physical_optimizer::PhysicalOptimizerRule; use crate::physical_plan::coalesce_partitions::CoalescePartitionsExec; @@ -81,78 +81,66 @@ impl EnforceSorting { #[derive(Debug, Clone)] struct PlanWithCorrespondingSort { plan: Arc, - // For every child, keep a subtree of `ExecutionPlan`s starting from the - // child until the `SortExec`(s) -- could be multiple for n-ary plans like - // Union -- that determine the output ordering of the child. If the child - // has no connection to any sort, simply store None (and not a subtree). - sort_onwards: Vec>, + // For every child, track `ExecutionPlan`s starting from the child until + // the `SortExec`(s). If the child has no connection to any sort, it simply + // stores false. + sort_connection: bool, + children_nodes: Vec, } impl PlanWithCorrespondingSort { fn new(plan: Arc) -> Self { - let length = plan.children().len(); - PlanWithCorrespondingSort { + let children = plan.children(); + Self { plan, - sort_onwards: vec![None; length], + sort_connection: false, + children_nodes: children.into_iter().map(Self::new).collect(), } } - fn new_from_children_nodes( - children_nodes: Vec, + fn update_children( parent_plan: Arc, + mut children_nodes: Vec, ) -> Result { - let children_plans = children_nodes - .iter() - .map(|item| item.plan.clone()) - .collect::>(); - let sort_onwards = children_nodes - .into_iter() - .enumerate() - .map(|(idx, item)| { - let plan = &item.plan; - // Leaves of `sort_onwards` are `SortExec` operators, which impose - // an ordering. This tree collects all the intermediate executors - // that maintain this ordering. If we just saw a order imposing - // operator, we reset the tree and start accumulating. - if is_sort(plan) { - return Some(ExecTree::new(item.plan, idx, vec![])); - } else if is_limit(plan) { - // There is no sort linkage for this path, it starts at a limit. - return None; - } + for node in children_nodes.iter_mut() { + let plan = &node.plan; + // Leaves of `sort_onwards` are `SortExec` operators, which impose + // an ordering. This tree collects all the intermediate executors + // that maintain this ordering. If we just saw a order imposing + // operator, we reset the tree and start accumulating. + node.sort_connection = if is_sort(plan) { + // Initiate connection + true + } else if is_limit(plan) { + // There is no sort linkage for this path, it starts at a limit. + false + } else { let is_spm = is_sort_preserving_merge(plan); let required_orderings = plan.required_input_ordering(); let flags = plan.maintains_input_order(); - let children = izip!(flags, item.sort_onwards, required_orderings) - .filter_map(|(maintains, element, required_ordering)| { - if (required_ordering.is_none() && maintains) || is_spm { - element - } else { - None - } - }) - .collect::>(); - if !children.is_empty() { - // Add parent node to the tree if there is at least one - // child with a subtree: - Some(ExecTree::new(item.plan, idx, children)) - } else { - // There is no sort linkage for this child, do nothing. - None - } - }) - .collect(); + // Add parent node to the tree if there is at least one + // child with a sort connection: + izip!(flags, required_orderings).any(|(maintains, required_ordering)| { + let propagates_ordering = + (maintains && required_ordering.is_none()) || is_spm; + let connected_to_sort = + node.children_nodes.iter().any(|item| item.sort_connection); + propagates_ordering && connected_to_sort + }) + } + } + let children_plans = children_nodes + .iter() + .map(|item| item.plan.clone()) + .collect::>(); let plan = with_new_children_if_necessary(parent_plan, children_plans)?.into(); - Ok(PlanWithCorrespondingSort { plan, sort_onwards }) - } - fn children(&self) -> Vec { - self.plan - .children() - .into_iter() - .map(PlanWithCorrespondingSort::new) - .collect() + Ok(Self { + plan, + sort_connection: false, + children_nodes, + }) } } @@ -161,9 +149,8 @@ impl TreeNode for PlanWithCorrespondingSort { where F: FnMut(&Self) -> Result, { - let children = self.children(); - for child in children { - match op(&child)? { + for child in &self.children_nodes { + match op(child)? { VisitRecursion::Continue => {} VisitRecursion::Skip => return Ok(VisitRecursion::Continue), VisitRecursion::Stop => return Ok(VisitRecursion::Stop), @@ -173,102 +160,79 @@ impl TreeNode for PlanWithCorrespondingSort { Ok(VisitRecursion::Continue) } - fn map_children(self, transform: F) -> Result + fn map_children(mut self, transform: F) -> Result where F: FnMut(Self) -> Result, { - let children = self.children(); - if children.is_empty() { - Ok(self) - } else { - let children_nodes = children + if !self.children_nodes.is_empty() { + self.children_nodes = self + .children_nodes .into_iter() .map(transform) - .collect::>>()?; - PlanWithCorrespondingSort::new_from_children_nodes(children_nodes, self.plan) + .collect::>()?; + self.plan = with_new_children_if_necessary( + self.plan, + self.children_nodes.iter().map(|c| c.plan.clone()).collect(), + )? + .into(); } + Ok(self) } } -/// This object is used within the [EnforceSorting] rule to track the closest +/// This object is used within the [`EnforceSorting`] rule to track the closest /// [`CoalescePartitionsExec`] descendant(s) for every child of a plan. #[derive(Debug, Clone)] struct PlanWithCorrespondingCoalescePartitions { plan: Arc, - // For every child, keep a subtree of `ExecutionPlan`s starting from the - // child until the `CoalescePartitionsExec`(s) -- could be multiple for - // n-ary plans like Union -- that affect the output partitioning of the - // child. If the child has no connection to any `CoalescePartitionsExec`, - // simply store None (and not a subtree). - coalesce_onwards: Vec>, + // Stores whether the plan is a `CoalescePartitionsExec` or it is connected to + // a `CoalescePartitionsExec` via its children. + coalesce_connection: bool, + children_nodes: Vec, } impl PlanWithCorrespondingCoalescePartitions { + /// Creates an empty tree with empty connections. fn new(plan: Arc) -> Self { - let length = plan.children().len(); - PlanWithCorrespondingCoalescePartitions { + let children = plan.children(); + Self { plan, - coalesce_onwards: vec![None; length], + coalesce_connection: false, + children_nodes: children.into_iter().map(Self::new).collect(), } } - fn new_from_children_nodes( - children_nodes: Vec, - parent_plan: Arc, - ) -> Result { - let children_plans = children_nodes + fn update_children(mut self) -> Result { + self.coalesce_connection = if self.plan.children().is_empty() { + // Plan has no children, it cannot be a `CoalescePartitionsExec`. + false + } else if is_coalesce_partitions(&self.plan) { + // Initiate a connection + true + } else { + self.children_nodes + .iter() + .enumerate() + .map(|(idx, node)| { + // Only consider operators that don't require a + // single partition, and connected to any coalesce + node.coalesce_connection + && !matches!( + self.plan.required_input_distribution()[idx], + Distribution::SinglePartition + ) + // If all children are None. There is nothing to track, set connection false. + }) + .any(|c| c) + }; + + let children_plans = self + .children_nodes .iter() .map(|item| item.plan.clone()) .collect(); - let coalesce_onwards = children_nodes - .into_iter() - .enumerate() - .map(|(idx, item)| { - // Leaves of the `coalesce_onwards` tree are `CoalescePartitionsExec` - // operators. This tree collects all the intermediate executors that - // maintain a single partition. If we just saw a `CoalescePartitionsExec` - // operator, we reset the tree and start accumulating. - let plan = item.plan; - if plan.children().is_empty() { - // Plan has no children, there is nothing to propagate. - None - } else if is_coalesce_partitions(&plan) { - Some(ExecTree::new(plan, idx, vec![])) - } else { - let children = item - .coalesce_onwards - .into_iter() - .flatten() - .filter(|item| { - // Only consider operators that don't require a - // single partition. - !matches!( - plan.required_input_distribution()[item.idx], - Distribution::SinglePartition - ) - }) - .collect::>(); - if children.is_empty() { - None - } else { - Some(ExecTree::new(plan, idx, children)) - } - } - }) - .collect(); - let plan = with_new_children_if_necessary(parent_plan, children_plans)?.into(); - Ok(PlanWithCorrespondingCoalescePartitions { - plan, - coalesce_onwards, - }) - } - - fn children(&self) -> Vec { - self.plan - .children() - .into_iter() - .map(PlanWithCorrespondingCoalescePartitions::new) - .collect() + self.plan = with_new_children_if_necessary(self.plan, children_plans)?.into(); + Ok(self) } } @@ -277,9 +241,8 @@ impl TreeNode for PlanWithCorrespondingCoalescePartitions { where F: FnMut(&Self) -> Result, { - let children = self.children(); - for child in children { - match op(&child)? { + for child in &self.children_nodes { + match op(child)? { VisitRecursion::Continue => {} VisitRecursion::Skip => return Ok(VisitRecursion::Continue), VisitRecursion::Stop => return Ok(VisitRecursion::Stop), @@ -289,23 +252,23 @@ impl TreeNode for PlanWithCorrespondingCoalescePartitions { Ok(VisitRecursion::Continue) } - fn map_children(self, transform: F) -> Result + fn map_children(mut self, transform: F) -> Result where F: FnMut(Self) -> Result, { - let children = self.children(); - if children.is_empty() { - Ok(self) - } else { - let children_nodes = children + if !self.children_nodes.is_empty() { + self.children_nodes = self + .children_nodes .into_iter() .map(transform) - .collect::>>()?; - PlanWithCorrespondingCoalescePartitions::new_from_children_nodes( - children_nodes, + .collect::>()?; + self.plan = with_new_children_if_necessary( self.plan, - ) + self.children_nodes.iter().map(|c| c.plan.clone()).collect(), + )? + .into(); } + Ok(self) } } @@ -332,6 +295,7 @@ impl PhysicalOptimizerRule for EnforceSorting { } else { adjusted.plan }; + let plan_with_pipeline_fixer = OrderPreservationContext::new(new_plan); let updated_plan = plan_with_pipeline_fixer.transform_up(&|plan_with_pipeline_fixer| { @@ -345,7 +309,8 @@ impl PhysicalOptimizerRule for EnforceSorting { // Execute a top-down traversal to exploit sort push-down opportunities // missed by the bottom-up traversal: - let sort_pushdown = SortPushDown::init(updated_plan.plan); + let mut sort_pushdown = SortPushDown::new(updated_plan.plan); + sort_pushdown.assign_initial_requirements(); let adjusted = sort_pushdown.transform_down(&pushdown_sorts)?; Ok(adjusted.plan) } @@ -376,16 +341,21 @@ impl PhysicalOptimizerRule for EnforceSorting { fn parallelize_sorts( requirements: PlanWithCorrespondingCoalescePartitions, ) -> Result> { - let plan = requirements.plan; - let mut coalesce_onwards = requirements.coalesce_onwards; - if plan.children().is_empty() || coalesce_onwards[0].is_none() { + let PlanWithCorrespondingCoalescePartitions { + mut plan, + coalesce_connection, + mut children_nodes, + } = requirements.update_children()?; + + if plan.children().is_empty() || !children_nodes[0].coalesce_connection { // We only take an action when the plan is either a SortExec, a // SortPreservingMergeExec or a CoalescePartitionsExec, and they // all have a single child. Therefore, if the first child is `None`, // we can return immediately. return Ok(Transformed::No(PlanWithCorrespondingCoalescePartitions { plan, - coalesce_onwards, + coalesce_connection, + children_nodes, })); } else if (is_sort(&plan) || is_sort_preserving_merge(&plan)) && plan.output_partitioning().partition_count() <= 1 @@ -395,34 +365,30 @@ fn parallelize_sorts( // executors don't require single partition), then we can replace // the CoalescePartitionsExec + Sort cascade with a SortExec + // SortPreservingMergeExec cascade to parallelize sorting. - let mut prev_layer = plan.clone(); - update_child_to_remove_coalesce(&mut prev_layer, &mut coalesce_onwards[0])?; let (sort_exprs, fetch) = get_sort_exprs(&plan)?; - add_sort_above( - &mut prev_layer, - &PhysicalSortRequirement::from_sort_exprs(sort_exprs), - fetch, - ); - let spm = SortPreservingMergeExec::new(sort_exprs.to_vec(), prev_layer) - .with_fetch(fetch); - return Ok(Transformed::Yes(PlanWithCorrespondingCoalescePartitions { - plan: Arc::new(spm), - coalesce_onwards: vec![None], - })); + let sort_reqs = PhysicalSortRequirement::from_sort_exprs(sort_exprs); + let sort_exprs = sort_exprs.to_vec(); + update_child_to_remove_coalesce(&mut plan, &mut children_nodes[0])?; + add_sort_above(&mut plan, &sort_reqs, fetch); + let spm = SortPreservingMergeExec::new(sort_exprs, plan).with_fetch(fetch); + + return Ok(Transformed::Yes( + PlanWithCorrespondingCoalescePartitions::new(Arc::new(spm)), + )); } else if is_coalesce_partitions(&plan) { // There is an unnecessary `CoalescePartitionsExec` in the plan. - let mut prev_layer = plan.clone(); - update_child_to_remove_coalesce(&mut prev_layer, &mut coalesce_onwards[0])?; - let new_plan = plan.with_new_children(vec![prev_layer])?; - return Ok(Transformed::Yes(PlanWithCorrespondingCoalescePartitions { - plan: new_plan, - coalesce_onwards: vec![None], - })); + update_child_to_remove_coalesce(&mut plan, &mut children_nodes[0])?; + + let new_plan = Arc::new(CoalescePartitionsExec::new(plan)) as _; + return Ok(Transformed::Yes( + PlanWithCorrespondingCoalescePartitions::new(new_plan), + )); } Ok(Transformed::Yes(PlanWithCorrespondingCoalescePartitions { plan, - coalesce_onwards, + coalesce_connection, + children_nodes, })) } @@ -431,91 +397,102 @@ fn parallelize_sorts( fn ensure_sorting( requirements: PlanWithCorrespondingSort, ) -> Result> { + let requirements = PlanWithCorrespondingSort::update_children( + requirements.plan, + requirements.children_nodes, + )?; + // Perform naive analysis at the beginning -- remove already-satisfied sorts: if requirements.plan.children().is_empty() { return Ok(Transformed::No(requirements)); } - let plan = requirements.plan; - let mut children = plan.children(); - let mut sort_onwards = requirements.sort_onwards; - if let Some(result) = analyze_immediate_sort_removal(&plan, &sort_onwards) { + if let Some(result) = analyze_immediate_sort_removal(&requirements) { return Ok(Transformed::Yes(result)); } - for (idx, (child, sort_onwards, required_ordering)) in izip!( - children.iter_mut(), - sort_onwards.iter_mut(), - plan.required_input_ordering() - ) - .enumerate() + + let plan = requirements.plan; + let mut children_nodes = requirements.children_nodes; + + for (idx, (child_node, required_ordering)) in + izip!(children_nodes.iter_mut(), plan.required_input_ordering()).enumerate() { - let physical_ordering = child.output_ordering(); + let mut child_plan = child_node.plan.clone(); + let physical_ordering = child_plan.output_ordering(); match (required_ordering, physical_ordering) { (Some(required_ordering), Some(_)) => { - if !child + if !child_plan .equivalence_properties() .ordering_satisfy_requirement(&required_ordering) { // Make sure we preserve the ordering requirements: - update_child_to_remove_unnecessary_sort(child, sort_onwards, &plan)?; - add_sort_above(child, &required_ordering, None); - if is_sort(child) { - *sort_onwards = Some(ExecTree::new(child.clone(), idx, vec![])); - } else { - *sort_onwards = None; + update_child_to_remove_unnecessary_sort(idx, child_node, &plan)?; + add_sort_above(&mut child_plan, &required_ordering, None); + if is_sort(&child_plan) { + *child_node = PlanWithCorrespondingSort::update_children( + child_plan, + vec![child_node.clone()], + )?; + child_node.sort_connection = true; } } } (Some(required), None) => { // Ordering requirement is not met, we should add a `SortExec` to the plan. - add_sort_above(child, &required, None); - *sort_onwards = Some(ExecTree::new(child.clone(), idx, vec![])); + add_sort_above(&mut child_plan, &required, None); + *child_node = PlanWithCorrespondingSort::update_children( + child_plan, + vec![child_node.clone()], + )?; + child_node.sort_connection = true; } (None, Some(_)) => { // We have a `SortExec` whose effect may be neutralized by // another order-imposing operator. Remove this sort. if !plan.maintains_input_order()[idx] || is_union(&plan) { - update_child_to_remove_unnecessary_sort(child, sort_onwards, &plan)?; + update_child_to_remove_unnecessary_sort(idx, child_node, &plan)?; } } (None, None) => { - update_child_to_remove_unnecessary_sort(child, sort_onwards, &plan)?; + update_child_to_remove_unnecessary_sort(idx, child_node, &plan)?; } } } // For window expressions, we can remove some sorts when we can // calculate the result in reverse: - if is_window(&plan) { - if let Some(tree) = &mut sort_onwards[0] { - if let Some(result) = analyze_window_sort_removal(tree, &plan)? { - return Ok(Transformed::Yes(result)); - } + if is_window(&plan) && children_nodes[0].sort_connection { + if let Some(result) = analyze_window_sort_removal(&mut children_nodes[0], &plan)? + { + return Ok(Transformed::Yes(result)); } } else if is_sort_preserving_merge(&plan) - && children[0].output_partitioning().partition_count() <= 1 + && children_nodes[0] + .plan + .output_partitioning() + .partition_count() + <= 1 { // This SortPreservingMergeExec is unnecessary, input already has a // single partition. - sort_onwards.truncate(1); - return Ok(Transformed::Yes(PlanWithCorrespondingSort { - plan: children.swap_remove(0), - sort_onwards, - })); + let child_node = children_nodes.swap_remove(0); + return Ok(Transformed::Yes(child_node)); } - Ok(Transformed::Yes(PlanWithCorrespondingSort { - plan: plan.with_new_children(children)?, - sort_onwards, - })) + Ok(Transformed::Yes( + PlanWithCorrespondingSort::update_children(plan, children_nodes)?, + )) } /// Analyzes a given [`SortExec`] (`plan`) to determine whether its input /// already has a finer ordering than it enforces. fn analyze_immediate_sort_removal( - plan: &Arc, - sort_onwards: &[Option], + node: &PlanWithCorrespondingSort, ) -> Option { + let PlanWithCorrespondingSort { + plan, + children_nodes, + .. + } = node; if let Some(sort_exec) = plan.as_any().downcast_ref::() { let sort_input = sort_exec.input().clone(); - // If this sort is unnecessary, we should remove it: if sort_input .equivalence_properties() @@ -533,20 +510,33 @@ fn analyze_immediate_sort_removal( sort_exec.expr().to_vec(), sort_input, )); - let new_tree = ExecTree::new( - new_plan.clone(), - 0, - sort_onwards.iter().flat_map(|e| e.clone()).collect(), - ); PlanWithCorrespondingSort { plan: new_plan, - sort_onwards: vec![Some(new_tree)], + // SortPreservingMergeExec has single child. + sort_connection: false, + children_nodes: children_nodes + .iter() + .cloned() + .map(|mut node| { + node.sort_connection = false; + node + }) + .collect(), } } else { // Remove the sort: PlanWithCorrespondingSort { plan: sort_input, - sort_onwards: sort_onwards.to_vec(), + sort_connection: false, + children_nodes: children_nodes[0] + .children_nodes + .iter() + .cloned() + .map(|mut node| { + node.sort_connection = false; + node + }) + .collect(), } }, ); @@ -558,15 +548,15 @@ fn analyze_immediate_sort_removal( /// Analyzes a [`WindowAggExec`] or a [`BoundedWindowAggExec`] to determine /// whether it may allow removing a sort. fn analyze_window_sort_removal( - sort_tree: &mut ExecTree, + sort_tree: &mut PlanWithCorrespondingSort, window_exec: &Arc, ) -> Result> { let requires_single_partition = matches!( - window_exec.required_input_distribution()[sort_tree.idx], + window_exec.required_input_distribution()[0], Distribution::SinglePartition ); - let mut window_child = - remove_corresponding_sort_from_sub_plan(sort_tree, requires_single_partition)?; + remove_corresponding_sort_from_sub_plan(sort_tree, requires_single_partition)?; + let mut window_child = sort_tree.plan.clone(); let (window_expr, new_window) = if let Some(exec) = window_exec.as_any().downcast_ref::() { ( @@ -628,9 +618,9 @@ fn analyze_window_sort_removal( /// Updates child to remove the unnecessary [`CoalescePartitionsExec`] below it. fn update_child_to_remove_coalesce( child: &mut Arc, - coalesce_onwards: &mut Option, + coalesce_onwards: &mut PlanWithCorrespondingCoalescePartitions, ) -> Result<()> { - if let Some(coalesce_onwards) = coalesce_onwards { + if coalesce_onwards.coalesce_connection { *child = remove_corresponding_coalesce_in_sub_plan(coalesce_onwards, child)?; } Ok(()) @@ -638,10 +628,10 @@ fn update_child_to_remove_coalesce( /// Removes the [`CoalescePartitionsExec`] from the plan in `coalesce_onwards`. fn remove_corresponding_coalesce_in_sub_plan( - coalesce_onwards: &mut ExecTree, + coalesce_onwards: &mut PlanWithCorrespondingCoalescePartitions, parent: &Arc, ) -> Result> { - Ok(if is_coalesce_partitions(&coalesce_onwards.plan) { + if is_coalesce_partitions(&coalesce_onwards.plan) { // We can safely use the 0th index since we have a `CoalescePartitionsExec`. let mut new_plan = coalesce_onwards.plan.children()[0].clone(); while new_plan.output_partitioning() == parent.output_partitioning() @@ -650,89 +640,113 @@ fn remove_corresponding_coalesce_in_sub_plan( { new_plan = new_plan.children().swap_remove(0) } - new_plan + Ok(new_plan) } else { let plan = coalesce_onwards.plan.clone(); let mut children = plan.children(); - for item in &mut coalesce_onwards.children { - children[item.idx] = remove_corresponding_coalesce_in_sub_plan(item, &plan)?; + for (idx, node) in coalesce_onwards.children_nodes.iter_mut().enumerate() { + if node.coalesce_connection { + children[idx] = remove_corresponding_coalesce_in_sub_plan(node, &plan)?; + } } - plan.with_new_children(children)? - }) + plan.with_new_children(children) + } } /// Updates child to remove the unnecessary sort below it. fn update_child_to_remove_unnecessary_sort( - child: &mut Arc, - sort_onwards: &mut Option, + child_idx: usize, + sort_onwards: &mut PlanWithCorrespondingSort, parent: &Arc, ) -> Result<()> { - if let Some(sort_onwards) = sort_onwards { + if sort_onwards.sort_connection { let requires_single_partition = matches!( - parent.required_input_distribution()[sort_onwards.idx], + parent.required_input_distribution()[child_idx], Distribution::SinglePartition ); - *child = remove_corresponding_sort_from_sub_plan( - sort_onwards, - requires_single_partition, - )?; + remove_corresponding_sort_from_sub_plan(sort_onwards, requires_single_partition)?; } - *sort_onwards = None; + sort_onwards.sort_connection = false; Ok(()) } /// Removes the sort from the plan in `sort_onwards`. fn remove_corresponding_sort_from_sub_plan( - sort_onwards: &mut ExecTree, + sort_onwards: &mut PlanWithCorrespondingSort, requires_single_partition: bool, -) -> Result> { +) -> Result<()> { // A `SortExec` is always at the bottom of the tree. - let mut updated_plan = if is_sort(&sort_onwards.plan) { - sort_onwards.plan.children().swap_remove(0) + if is_sort(&sort_onwards.plan) { + *sort_onwards = sort_onwards.children_nodes.swap_remove(0); } else { - let plan = &sort_onwards.plan; - let mut children = plan.children(); - for item in &mut sort_onwards.children { - let requires_single_partition = matches!( - plan.required_input_distribution()[item.idx], - Distribution::SinglePartition - ); - children[item.idx] = - remove_corresponding_sort_from_sub_plan(item, requires_single_partition)?; + let PlanWithCorrespondingSort { + plan, + sort_connection: _, + children_nodes, + } = sort_onwards; + let mut any_connection = false; + for (child_idx, child_node) in children_nodes.iter_mut().enumerate() { + if child_node.sort_connection { + any_connection = true; + let requires_single_partition = matches!( + plan.required_input_distribution()[child_idx], + Distribution::SinglePartition + ); + remove_corresponding_sort_from_sub_plan( + child_node, + requires_single_partition, + )?; + } } + if any_connection || children_nodes.is_empty() { + *sort_onwards = PlanWithCorrespondingSort::update_children( + plan.clone(), + children_nodes.clone(), + )?; + } + let PlanWithCorrespondingSort { + plan, + children_nodes, + .. + } = sort_onwards; // Replace with variants that do not preserve order. if is_sort_preserving_merge(plan) { - children.swap_remove(0) + children_nodes.swap_remove(0); + *plan = plan.children().swap_remove(0); } else if let Some(repartition) = plan.as_any().downcast_ref::() { - Arc::new( - // By default, RepartitionExec does not preserve order - RepartitionExec::try_new( - children.swap_remove(0), - repartition.partitioning().clone(), - )?, - ) - } else { - plan.clone().with_new_children(children)? + *plan = Arc::new(RepartitionExec::try_new( + children_nodes[0].plan.clone(), + repartition.output_partitioning(), + )?) as _; } }; // Deleting a merging sort may invalidate distribution requirements. // Ensure that we stay compliant with such requirements: if requires_single_partition - && updated_plan.output_partitioning().partition_count() > 1 + && sort_onwards.plan.output_partitioning().partition_count() > 1 { // If there is existing ordering, to preserve ordering use SortPreservingMergeExec // instead of CoalescePartitionsExec. - if let Some(ordering) = updated_plan.output_ordering() { - updated_plan = Arc::new(SortPreservingMergeExec::new( + if let Some(ordering) = sort_onwards.plan.output_ordering() { + let plan = Arc::new(SortPreservingMergeExec::new( ordering.to_vec(), - updated_plan, - )); + sort_onwards.plan.clone(), + )) as _; + *sort_onwards = PlanWithCorrespondingSort::update_children( + plan, + vec![sort_onwards.clone()], + )?; } else { - updated_plan = Arc::new(CoalescePartitionsExec::new(updated_plan)); + let plan = + Arc::new(CoalescePartitionsExec::new(sort_onwards.plan.clone())) as _; + *sort_onwards = PlanWithCorrespondingSort::update_children( + plan, + vec![sort_onwards.clone()], + )?; } } - Ok(updated_plan) + Ok(()) } /// Converts an [ExecutionPlan] trait object to a [PhysicalSortExpr] slice when possible. diff --git a/datafusion/core/src/physical_optimizer/output_requirements.rs b/datafusion/core/src/physical_optimizer/output_requirements.rs index f8bf3bb965e8..4d03840d3dd3 100644 --- a/datafusion/core/src/physical_optimizer/output_requirements.rs +++ b/datafusion/core/src/physical_optimizer/output_requirements.rs @@ -147,6 +147,10 @@ impl ExecutionPlan for OutputRequirementExec { self.input.output_ordering() } + fn maintains_input_order(&self) -> Vec { + vec![true] + } + fn children(&self) -> Vec> { vec![self.input.clone()] } diff --git a/datafusion/core/src/physical_optimizer/pipeline_checker.rs b/datafusion/core/src/physical_optimizer/pipeline_checker.rs index d59248aadf05..9e9f647d073f 100644 --- a/datafusion/core/src/physical_optimizer/pipeline_checker.rs +++ b/datafusion/core/src/physical_optimizer/pipeline_checker.rs @@ -24,13 +24,13 @@ use std::sync::Arc; use crate::config::ConfigOptions; use crate::error::Result; use crate::physical_optimizer::PhysicalOptimizerRule; -use crate::physical_plan::joins::SymmetricHashJoinExec; use crate::physical_plan::{with_new_children_if_necessary, ExecutionPlan}; use datafusion_common::config::OptimizerOptions; use datafusion_common::tree_node::{Transformed, TreeNode, VisitRecursion}; use datafusion_common::{plan_err, DataFusionError}; use datafusion_physical_expr::intervals::utils::{check_support, is_datatype_supported}; +use datafusion_physical_plan::joins::SymmetricHashJoinExec; /// The PipelineChecker rule rejects non-runnable query plans that use /// pipeline-breaking operators on infinite input(s). @@ -70,14 +70,14 @@ impl PhysicalOptimizerRule for PipelineChecker { pub struct PipelineStatePropagator { pub(crate) plan: Arc, pub(crate) unbounded: bool, - pub(crate) children: Vec, + pub(crate) children: Vec, } impl PipelineStatePropagator { /// Constructs a new, default pipelining state. pub fn new(plan: Arc) -> Self { let children = plan.children(); - PipelineStatePropagator { + Self { plan, unbounded: false, children: children.into_iter().map(Self::new).collect(), @@ -86,10 +86,7 @@ impl PipelineStatePropagator { /// Returns the children unboundedness information. pub fn children_unbounded(&self) -> Vec { - self.children - .iter() - .map(|c| c.unbounded) - .collect::>() + self.children.iter().map(|c| c.unbounded).collect() } } @@ -109,26 +106,23 @@ impl TreeNode for PipelineStatePropagator { Ok(VisitRecursion::Continue) } - fn map_children(self, transform: F) -> Result + fn map_children(mut self, transform: F) -> Result where F: FnMut(Self) -> Result, { if !self.children.is_empty() { - let new_children = self + self.children = self .children .into_iter() .map(transform) - .collect::>>()?; - let children_plans = new_children.iter().map(|c| c.plan.clone()).collect(); - - Ok(PipelineStatePropagator { - plan: with_new_children_if_necessary(self.plan, children_plans)?.into(), - unbounded: self.unbounded, - children: new_children, - }) - } else { - Ok(self) + .collect::>()?; + self.plan = with_new_children_if_necessary( + self.plan, + self.children.iter().map(|c| c.plan.clone()).collect(), + )? + .into(); } + Ok(self) } } diff --git a/datafusion/core/src/physical_optimizer/replace_with_order_preserving_variants.rs b/datafusion/core/src/physical_optimizer/replace_with_order_preserving_variants.rs index 0ff7e9f48edc..91f3d2abc6ff 100644 --- a/datafusion/core/src/physical_optimizer/replace_with_order_preserving_variants.rs +++ b/datafusion/core/src/physical_optimizer/replace_with_order_preserving_variants.rs @@ -21,14 +21,13 @@ use std::sync::Arc; +use super::utils::is_repartition; use crate::error::Result; -use crate::physical_optimizer::utils::{is_coalesce_partitions, is_sort, ExecTree}; +use crate::physical_optimizer::utils::{is_coalesce_partitions, is_sort}; use crate::physical_plan::repartition::RepartitionExec; use crate::physical_plan::sorts::sort_preserving_merge::SortPreservingMergeExec; use crate::physical_plan::{with_new_children_if_necessary, ExecutionPlan}; -use super::utils::is_repartition; - use datafusion_common::config::ConfigOptions; use datafusion_common::tree_node::{Transformed, TreeNode, VisitRecursion}; use datafusion_physical_plan::unbounded_output; @@ -40,80 +39,67 @@ use datafusion_physical_plan::unbounded_output; #[derive(Debug, Clone)] pub(crate) struct OrderPreservationContext { pub(crate) plan: Arc, - ordering_onwards: Vec>, + ordering_connection: bool, + children_nodes: Vec, } impl OrderPreservationContext { - /// Creates a "default" order-preservation context. + /// Creates an empty context tree. Each node has `false` connections. pub fn new(plan: Arc) -> Self { - let length = plan.children().len(); - OrderPreservationContext { + let children = plan.children(); + Self { plan, - ordering_onwards: vec![None; length], + ordering_connection: false, + children_nodes: children.into_iter().map(Self::new).collect(), } } /// Creates a new order-preservation context from those of children nodes. - pub fn new_from_children_nodes( - children_nodes: Vec, - parent_plan: Arc, - ) -> Result { - let children_plans = children_nodes - .iter() - .map(|item| item.plan.clone()) - .collect(); - let ordering_onwards = children_nodes - .into_iter() - .enumerate() - .map(|(idx, item)| { - // `ordering_onwards` tree keeps track of executors that maintain - // ordering, (or that can maintain ordering with the replacement of - // its variant) - let plan = item.plan; - let children = plan.children(); - let ordering_onwards = item.ordering_onwards; - if children.is_empty() { - // Plan has no children, there is nothing to propagate. - None - } else if ordering_onwards[0].is_none() - && ((is_repartition(&plan) && !plan.maintains_input_order()[0]) - || (is_coalesce_partitions(&plan) - && children[0].output_ordering().is_some())) - { - Some(ExecTree::new(plan, idx, vec![])) - } else { - let children = ordering_onwards - .into_iter() - .flatten() - .filter(|item| { - // Only consider operators that maintains ordering - plan.maintains_input_order()[item.idx] - || is_coalesce_partitions(&plan) - || is_repartition(&plan) - }) - .collect::>(); - if children.is_empty() { - None - } else { - Some(ExecTree::new(plan, idx, children)) - } - } - }) - .collect(); - let plan = with_new_children_if_necessary(parent_plan, children_plans)?.into(); - Ok(OrderPreservationContext { - plan, - ordering_onwards, - }) - } + pub fn update_children(mut self) -> Result { + for node in self.children_nodes.iter_mut() { + let plan = node.plan.clone(); + let children = plan.children(); + let maintains_input_order = plan.maintains_input_order(); + let inspect_child = |idx| { + maintains_input_order[idx] + || is_coalesce_partitions(&plan) + || is_repartition(&plan) + }; + + // We cut the path towards nodes that do not maintain ordering. + for (idx, c) in node.children_nodes.iter_mut().enumerate() { + c.ordering_connection &= inspect_child(idx); + } + + node.ordering_connection = if children.is_empty() { + false + } else if !node.children_nodes[0].ordering_connection + && ((is_repartition(&plan) && !maintains_input_order[0]) + || (is_coalesce_partitions(&plan) + && children[0].output_ordering().is_some())) + { + // We either have a RepartitionExec or a CoalescePartitionsExec + // and they lose their input ordering, so initiate connection: + true + } else { + // Maintain connection if there is a child with a connection, + // and operator can possibly maintain that connection (either + // in its current form or when we replace it with the corresponding + // order preserving operator). + node.children_nodes + .iter() + .enumerate() + .any(|(idx, c)| c.ordering_connection && inspect_child(idx)) + } + } - /// Computes order-preservation contexts for every child of the plan. - pub fn children(&self) -> Vec { - self.plan - .children() - .into_iter() - .map(OrderPreservationContext::new) - .collect() + self.plan = with_new_children_if_necessary( + self.plan, + self.children_nodes.iter().map(|c| c.plan.clone()).collect(), + )? + .into(); + self.ordering_connection = false; + Ok(self) } } @@ -122,8 +108,8 @@ impl TreeNode for OrderPreservationContext { where F: FnMut(&Self) -> Result, { - for child in self.children() { - match op(&child)? { + for child in &self.children_nodes { + match op(child)? { VisitRecursion::Continue => {} VisitRecursion::Skip => return Ok(VisitRecursion::Continue), VisitRecursion::Stop => return Ok(VisitRecursion::Stop), @@ -132,68 +118,88 @@ impl TreeNode for OrderPreservationContext { Ok(VisitRecursion::Continue) } - fn map_children(self, transform: F) -> Result + fn map_children(mut self, transform: F) -> Result where F: FnMut(Self) -> Result, { - let children = self.children(); - if children.is_empty() { - Ok(self) - } else { - let children_nodes = children + if !self.children_nodes.is_empty() { + self.children_nodes = self + .children_nodes .into_iter() .map(transform) - .collect::>>()?; - OrderPreservationContext::new_from_children_nodes(children_nodes, self.plan) + .collect::>()?; + self.plan = with_new_children_if_necessary( + self.plan, + self.children_nodes.iter().map(|c| c.plan.clone()).collect(), + )? + .into(); } + Ok(self) } } -/// Calculates the updated plan by replacing executors that lose ordering -/// inside the `ExecTree` with their order-preserving variants. This will +/// Calculates the updated plan by replacing operators that lose ordering +/// inside `sort_input` with their order-preserving variants. This will /// generate an alternative plan, which will be accepted or rejected later on /// depending on whether it helps us remove a `SortExec`. fn get_updated_plan( - exec_tree: &ExecTree, + mut sort_input: OrderPreservationContext, // Flag indicating that it is desirable to replace `RepartitionExec`s with // `SortPreservingRepartitionExec`s: is_spr_better: bool, // Flag indicating that it is desirable to replace `CoalescePartitionsExec`s // with `SortPreservingMergeExec`s: is_spm_better: bool, -) -> Result> { - let plan = exec_tree.plan.clone(); +) -> Result { + let updated_children = sort_input + .children_nodes + .clone() + .into_iter() + .map(|item| { + // Update children and their descendants in the given tree if the connection is open: + if item.ordering_connection { + get_updated_plan(item, is_spr_better, is_spm_better) + } else { + Ok(item) + } + }) + .collect::>>()?; - let mut children = plan.children(); - // Update children and their descendants in the given tree: - for item in &exec_tree.children { - children[item.idx] = get_updated_plan(item, is_spr_better, is_spm_better)?; - } - // Construct the plan with updated children: - let mut plan = plan.with_new_children(children)?; + sort_input.plan = sort_input + .plan + .with_new_children(updated_children.iter().map(|c| c.plan.clone()).collect())?; + sort_input.ordering_connection = false; + sort_input.children_nodes = updated_children; // When a `RepartitionExec` doesn't preserve ordering, replace it with - // a `SortPreservingRepartitionExec` if appropriate: - if is_repartition(&plan) && !plan.maintains_input_order()[0] && is_spr_better { - let child = plan.children().swap_remove(0); - let repartition = RepartitionExec::try_new(child, plan.output_partitioning())? - .with_preserve_order(); - plan = Arc::new(repartition) as _ - } - // When the input of a `CoalescePartitionsExec` has an ordering, replace it - // with a `SortPreservingMergeExec` if appropriate: - let mut children = plan.children(); - if is_coalesce_partitions(&plan) - && children[0].output_ordering().is_some() - && is_spm_better + // a sort-preserving variant if appropriate: + if is_repartition(&sort_input.plan) + && !sort_input.plan.maintains_input_order()[0] + && is_spr_better { - let child = children.swap_remove(0); - plan = Arc::new(SortPreservingMergeExec::new( - child.output_ordering().unwrap_or(&[]).to_vec(), - child, - )) as _ + let child = sort_input.plan.children().swap_remove(0); + let repartition = + RepartitionExec::try_new(child, sort_input.plan.output_partitioning())? + .with_preserve_order(); + sort_input.plan = Arc::new(repartition) as _; + sort_input.children_nodes[0].ordering_connection = true; + } else if is_coalesce_partitions(&sort_input.plan) && is_spm_better { + // When the input of a `CoalescePartitionsExec` has an ordering, replace it + // with a `SortPreservingMergeExec` if appropriate: + if let Some(ordering) = sort_input.children_nodes[0] + .plan + .output_ordering() + .map(|o| o.to_vec()) + { + // Now we can mutate `new_node.children_nodes` safely + let child = sort_input.children_nodes.clone().swap_remove(0); + sort_input.plan = + Arc::new(SortPreservingMergeExec::new(ordering, child.plan)) as _; + sort_input.children_nodes[0].ordering_connection = true; + } } - Ok(plan) + + Ok(sort_input) } /// The `replace_with_order_preserving_variants` optimizer sub-rule tries to @@ -211,11 +217,11 @@ fn get_updated_plan( /// /// The algorithm flow is simply like this: /// 1. Visit nodes of the physical plan bottom-up and look for `SortExec` nodes. -/// 1_1. During the traversal, build an `ExecTree` to keep track of operators -/// that maintain ordering (or can maintain ordering when replaced by an -/// order-preserving variant) until a `SortExec` is found. +/// 1_1. During the traversal, keep track of operators that maintain ordering +/// (or can maintain ordering when replaced by an order-preserving variant) until +/// a `SortExec` is found. /// 2. When a `SortExec` is found, update the child of the `SortExec` by replacing -/// operators that do not preserve ordering in the `ExecTree` with their order +/// operators that do not preserve ordering in the tree with their order /// preserving variants. /// 3. Check if the `SortExec` is still necessary in the updated plan by comparing /// its input ordering with the output ordering it imposes. We do this because @@ -239,37 +245,41 @@ pub(crate) fn replace_with_order_preserving_variants( is_spm_better: bool, config: &ConfigOptions, ) -> Result> { - let plan = &requirements.plan; - let ordering_onwards = &requirements.ordering_onwards; - if is_sort(plan) { - let exec_tree = if let Some(exec_tree) = &ordering_onwards[0] { - exec_tree - } else { - return Ok(Transformed::No(requirements)); - }; - // For unbounded cases, replace with the order-preserving variant in - // any case, as doing so helps fix the pipeline. - // Also do the replacement if opted-in via config options. - let use_order_preserving_variant = - config.optimizer.prefer_existing_sort || unbounded_output(plan); - let updated_sort_input = get_updated_plan( - exec_tree, - is_spr_better || use_order_preserving_variant, - is_spm_better || use_order_preserving_variant, - )?; - // If this sort is unnecessary, we should remove it and update the plan: - if updated_sort_input - .equivalence_properties() - .ordering_satisfy(plan.output_ordering().unwrap_or(&[])) - { - return Ok(Transformed::Yes(OrderPreservationContext { - plan: updated_sort_input, - ordering_onwards: vec![None], - })); - } + let mut requirements = requirements.update_children()?; + if !(is_sort(&requirements.plan) + && requirements.children_nodes[0].ordering_connection) + { + return Ok(Transformed::No(requirements)); } - Ok(Transformed::No(requirements)) + // For unbounded cases, replace with the order-preserving variant in + // any case, as doing so helps fix the pipeline. + // Also do the replacement if opted-in via config options. + let use_order_preserving_variant = + config.optimizer.prefer_existing_sort || unbounded_output(&requirements.plan); + + let mut updated_sort_input = get_updated_plan( + requirements.children_nodes.clone().swap_remove(0), + is_spr_better || use_order_preserving_variant, + is_spm_better || use_order_preserving_variant, + )?; + + // If this sort is unnecessary, we should remove it and update the plan: + if updated_sort_input + .plan + .equivalence_properties() + .ordering_satisfy(requirements.plan.output_ordering().unwrap_or(&[])) + { + for child in updated_sort_input.children_nodes.iter_mut() { + child.ordering_connection = false; + } + Ok(Transformed::Yes(updated_sort_input)) + } else { + for child in requirements.children_nodes.iter_mut() { + child.ordering_connection = false; + } + Ok(Transformed::Yes(requirements)) + } } #[cfg(test)] diff --git a/datafusion/core/src/physical_optimizer/sort_pushdown.rs b/datafusion/core/src/physical_optimizer/sort_pushdown.rs index b9502d92ac12..b0013863010a 100644 --- a/datafusion/core/src/physical_optimizer/sort_pushdown.rs +++ b/datafusion/core/src/physical_optimizer/sort_pushdown.rs @@ -36,8 +36,6 @@ use datafusion_physical_expr::{ LexRequirementRef, PhysicalSortExpr, PhysicalSortRequirement, }; -use itertools::izip; - /// This is a "data class" we use within the [`EnforceSorting`] rule to push /// down [`SortExec`] in the plan. In some cases, we can reduce the total /// computational cost by pushing down `SortExec`s through some executors. @@ -49,35 +47,26 @@ pub(crate) struct SortPushDown { pub plan: Arc, /// Parent required sort ordering required_ordering: Option>, - /// The adjusted request sort ordering to children. - /// By default they are the same as the plan's required input ordering, but can be adjusted based on parent required sort ordering properties. - adjusted_request_ordering: Vec>>, + children_nodes: Vec, } impl SortPushDown { - pub fn init(plan: Arc) -> Self { - let request_ordering = plan.required_input_ordering(); - SortPushDown { + /// Creates an empty tree with empty `required_ordering`'s. + pub fn new(plan: Arc) -> Self { + let children = plan.children(); + Self { plan, required_ordering: None, - adjusted_request_ordering: request_ordering, + children_nodes: children.into_iter().map(Self::new).collect(), } } - pub fn children(&self) -> Vec { - izip!( - self.plan.children().into_iter(), - self.adjusted_request_ordering.clone().into_iter(), - ) - .map(|(child, from_parent)| { - let child_request_ordering = child.required_input_ordering(); - SortPushDown { - plan: child, - required_ordering: from_parent, - adjusted_request_ordering: child_request_ordering, - } - }) - .collect() + /// Assigns the ordering requirement of the root node to the its children. + pub fn assign_initial_requirements(&mut self) { + let reqs = self.plan.required_input_ordering(); + for (child, requirement) in self.children_nodes.iter_mut().zip(reqs) { + child.required_ordering = requirement; + } } } @@ -86,9 +75,8 @@ impl TreeNode for SortPushDown { where F: FnMut(&Self) -> Result, { - let children = self.children(); - for child in children { - match op(&child)? { + for child in &self.children_nodes { + match op(child)? { VisitRecursion::Continue => {} VisitRecursion::Skip => return Ok(VisitRecursion::Continue), VisitRecursion::Stop => return Ok(VisitRecursion::Stop), @@ -97,64 +85,64 @@ impl TreeNode for SortPushDown { Ok(VisitRecursion::Continue) } - fn map_children(mut self, transform: F) -> Result where F: FnMut(Self) -> Result, { - let children = self.children(); - if !children.is_empty() { - let children_plans = children + if !self.children_nodes.is_empty() { + self.children_nodes = self + .children_nodes .into_iter() .map(transform) - .map(|r| r.map(|s| s.plan)) - .collect::>>()?; - - match with_new_children_if_necessary(self.plan, children_plans)? { - Transformed::Yes(plan) | Transformed::No(plan) => { - self.plan = plan; - } - } - }; + .collect::>()?; + self.plan = with_new_children_if_necessary( + self.plan, + self.children_nodes.iter().map(|c| c.plan.clone()).collect(), + )? + .into(); + } Ok(self) } } pub(crate) fn pushdown_sorts( - requirements: SortPushDown, + mut requirements: SortPushDown, ) -> Result> { let plan = &requirements.plan; let parent_required = requirements.required_ordering.as_deref().unwrap_or(&[]); + if let Some(sort_exec) = plan.as_any().downcast_ref::() { - let new_plan = if !plan + if !plan .equivalence_properties() .ordering_satisfy_requirement(parent_required) { // If the current plan is a SortExec, modify it to satisfy parent requirements: let mut new_plan = sort_exec.input().clone(); add_sort_above(&mut new_plan, parent_required, sort_exec.fetch()); - new_plan - } else { - requirements.plan + requirements.plan = new_plan; }; - let required_ordering = new_plan + + let required_ordering = requirements + .plan .output_ordering() .map(PhysicalSortRequirement::from_sort_exprs) .unwrap_or_default(); // Since new_plan is a SortExec, we can safely get the 0th index. - let child = new_plan.children().swap_remove(0); + let mut child = requirements.children_nodes.swap_remove(0); if let Some(adjusted) = - pushdown_requirement_to_children(&child, &required_ordering)? + pushdown_requirement_to_children(&child.plan, &required_ordering)? { + for (c, o) in child.children_nodes.iter_mut().zip(adjusted) { + c.required_ordering = o; + } // Can push down requirements - Ok(Transformed::Yes(SortPushDown { - plan: child, - required_ordering: None, - adjusted_request_ordering: adjusted, - })) + child.required_ordering = None; + Ok(Transformed::Yes(child)) } else { // Can not push down requirements - Ok(Transformed::Yes(SortPushDown::init(new_plan))) + let mut empty_node = SortPushDown::new(requirements.plan); + empty_node.assign_initial_requirements(); + Ok(Transformed::Yes(empty_node)) } } else { // Executors other than SortExec @@ -163,23 +151,27 @@ pub(crate) fn pushdown_sorts( .ordering_satisfy_requirement(parent_required) { // Satisfies parent requirements, immediately return. - return Ok(Transformed::Yes(SortPushDown { - required_ordering: None, - ..requirements - })); + let reqs = requirements.plan.required_input_ordering(); + for (child, order) in requirements.children_nodes.iter_mut().zip(reqs) { + child.required_ordering = order; + } + return Ok(Transformed::Yes(requirements)); } // Can not satisfy the parent requirements, check whether the requirements can be pushed down: if let Some(adjusted) = pushdown_requirement_to_children(plan, parent_required)? { - Ok(Transformed::Yes(SortPushDown { - plan: requirements.plan, - required_ordering: None, - adjusted_request_ordering: adjusted, - })) + for (c, o) in requirements.children_nodes.iter_mut().zip(adjusted) { + c.required_ordering = o; + } + requirements.required_ordering = None; + Ok(Transformed::Yes(requirements)) } else { // Can not push down requirements, add new SortExec: let mut new_plan = requirements.plan; add_sort_above(&mut new_plan, parent_required, None); - Ok(Transformed::Yes(SortPushDown::init(new_plan))) + let mut new_empty = SortPushDown::new(new_plan); + new_empty.assign_initial_requirements(); + // Can not push down requirements + Ok(Transformed::Yes(new_empty)) } } } @@ -297,10 +289,11 @@ fn pushdown_requirement_to_children( // TODO: Add support for Projection push down } -/// Determine the children requirements -/// If the children requirements are more specific, do not push down the parent requirements -/// If the the parent requirements are more specific, push down the parent requirements -/// If they are not compatible, need to add Sort. +/// Determine children requirements: +/// - If children requirements are more specific, do not push down parent +/// requirements. +/// - If parent requirements are more specific, push down parent requirements. +/// - If they are not compatible, need to add a sort. fn determine_children_requirement( parent_required: LexRequirementRef, request_child: LexRequirementRef, @@ -310,18 +303,15 @@ fn determine_children_requirement( .equivalence_properties() .requirements_compatible(request_child, parent_required) { - // request child requirements are more specific, no need to push down the parent requirements + // Child requirements are more specific, no need to push down. RequirementsCompatibility::Satisfy } else if child_plan .equivalence_properties() .requirements_compatible(parent_required, request_child) { - // parent requirements are more specific, adjust the request child requirements and push down the new requirements - let adjusted = if parent_required.is_empty() { - None - } else { - Some(parent_required.to_vec()) - }; + // Parent requirements are more specific, adjust child's requirements + // and push down the new requirements: + let adjusted = (!parent_required.is_empty()).then(|| parent_required.to_vec()); RequirementsCompatibility::Compatible(adjusted) } else { RequirementsCompatibility::NonCompatible diff --git a/datafusion/core/src/physical_optimizer/utils.rs b/datafusion/core/src/physical_optimizer/utils.rs index fccc1db0d359..f8063e969422 100644 --- a/datafusion/core/src/physical_optimizer/utils.rs +++ b/datafusion/core/src/physical_optimizer/utils.rs @@ -17,83 +17,18 @@ //! Collection of utility functions that are leveraged by the query optimizer rules -use std::fmt; -use std::fmt::Formatter; use std::sync::Arc; use crate::physical_plan::coalesce_partitions::CoalescePartitionsExec; -use crate::physical_plan::limit::{GlobalLimitExec, LocalLimitExec}; use crate::physical_plan::repartition::RepartitionExec; use crate::physical_plan::sorts::sort::SortExec; use crate::physical_plan::sorts::sort_preserving_merge::SortPreservingMergeExec; use crate::physical_plan::union::UnionExec; use crate::physical_plan::windows::{BoundedWindowAggExec, WindowAggExec}; -use crate::physical_plan::{get_plan_string, ExecutionPlan}; +use crate::physical_plan::ExecutionPlan; use datafusion_physical_expr::{LexRequirementRef, PhysicalSortRequirement}; - -/// This object implements a tree that we use while keeping track of paths -/// leading to [`SortExec`]s. -#[derive(Debug, Clone)] -pub(crate) struct ExecTree { - /// The `ExecutionPlan` associated with this node - pub plan: Arc, - /// Child index of the plan in its parent - pub idx: usize, - /// Children of the plan that would need updating if we remove leaf executors - pub children: Vec, -} - -impl fmt::Display for ExecTree { - fn fmt(&self, f: &mut Formatter<'_>) -> fmt::Result { - let plan_string = get_plan_string(&self.plan); - write!(f, "\nidx: {:?}", self.idx)?; - write!(f, "\nplan: {:?}", plan_string)?; - for child in self.children.iter() { - write!(f, "\nexec_tree:{}", child)?; - } - writeln!(f) - } -} - -impl ExecTree { - /// Create new Exec tree - pub fn new( - plan: Arc, - idx: usize, - children: Vec, - ) -> Self { - ExecTree { - plan, - idx, - children, - } - } -} - -/// Get `ExecTree` for each child of the plan if they are tracked. -/// # Arguments -/// -/// * `n_children` - Children count of the plan of interest -/// * `onward` - Contains `Some(ExecTree)` of the plan tracked. -/// - Contains `None` is plan is not tracked. -/// -/// # Returns -/// -/// A `Vec>` that contains tracking information of each child. -/// If a child is `None`, it is not tracked. If `Some(ExecTree)` child is tracked also. -pub(crate) fn get_children_exectrees( - n_children: usize, - onward: &Option, -) -> Vec> { - let mut children_onward = vec![None; n_children]; - if let Some(exec_tree) = &onward { - for child in &exec_tree.children { - children_onward[child.idx] = Some(child.clone()); - } - } - children_onward -} +use datafusion_physical_plan::limit::{GlobalLimitExec, LocalLimitExec}; /// This utility function adds a `SortExec` above an operator according to the /// given ordering requirements while preserving the original partitioning. diff --git a/datafusion/physical-expr/src/sort_properties.rs b/datafusion/physical-expr/src/sort_properties.rs index f51374461776..91238e5b04b4 100644 --- a/datafusion/physical-expr/src/sort_properties.rs +++ b/datafusion/physical-expr/src/sort_properties.rs @@ -151,7 +151,7 @@ impl Neg for SortProperties { pub struct ExprOrdering { pub expr: Arc, pub state: SortProperties, - pub children: Vec, + pub children: Vec, } impl ExprOrdering { @@ -191,15 +191,13 @@ impl TreeNode for ExprOrdering { where F: FnMut(Self) -> Result, { - if self.children.is_empty() { - Ok(self) - } else { + if !self.children.is_empty() { self.children = self .children .into_iter() .map(transform) - .collect::>>()?; - Ok(self) + .collect::>()?; } + Ok(self) } } diff --git a/datafusion/physical-plan/src/union.rs b/datafusion/physical-plan/src/union.rs index 14ef9c2ec27b..d01ea5507449 100644 --- a/datafusion/physical-plan/src/union.rs +++ b/datafusion/physical-plan/src/union.rs @@ -21,6 +21,7 @@ //! The Union operator combines multiple inputs with the same schema +use std::borrow::Borrow; use std::pin::Pin; use std::task::{Context, Poll}; use std::{any::Any, sync::Arc}; @@ -336,7 +337,7 @@ impl InterleaveExec { pub fn try_new(inputs: Vec>) -> Result { let schema = union_schema(&inputs); - if !can_interleave(&inputs) { + if !can_interleave(inputs.iter()) { return internal_err!( "Not all InterleaveExec children have a consistent hash partitioning" ); @@ -474,17 +475,18 @@ impl ExecutionPlan for InterleaveExec { /// It might be too strict here in the case that the input partition specs are compatible but not exactly the same. /// For example one input partition has the partition spec Hash('a','b','c') and /// other has the partition spec Hash('a'), It is safe to derive the out partition with the spec Hash('a','b','c'). -pub fn can_interleave(inputs: &[Arc]) -> bool { - if inputs.is_empty() { +pub fn can_interleave>>( + mut inputs: impl Iterator, +) -> bool { + let Some(first) = inputs.next() else { return false; - } + }; - let first_input_partition = inputs[0].output_partitioning(); - matches!(first_input_partition, Partitioning::Hash(_, _)) + let reference = first.borrow().output_partitioning(); + matches!(reference, Partitioning::Hash(_, _)) && inputs - .iter() - .map(|plan| plan.output_partitioning()) - .all(|partition| partition == first_input_partition) + .map(|plan| plan.borrow().output_partitioning()) + .all(|partition| partition == reference) } fn union_schema(inputs: &[Arc]) -> SchemaRef { From 1737d49185e9e37c15aa432342604ee559a1069d Mon Sep 17 00:00:00 2001 From: yi wang <48236141+my-vegetable-has-exploded@users.noreply.github.com> Date: Thu, 28 Dec 2023 20:12:49 +0800 Subject: [PATCH 506/572] feat: support inlist in LiteralGurantee for pruning (#8654) * support inlist in LiteralGuarantee for pruning. * add more tests * rm useless notes * Apply suggestions from code review Co-authored-by: Huaijin * add tests in row_groups * Apply suggestions from code review Co-authored-by: Ruihang Xia Co-authored-by: Andrew Lamb * update comment & add more tests --------- Co-authored-by: Huaijin Co-authored-by: Ruihang Xia Co-authored-by: Andrew Lamb --- .../physical_plan/parquet/row_groups.rs | 121 +-------- .../physical-expr/src/utils/guarantee.rs | 257 ++++++++++++++---- 2 files changed, 216 insertions(+), 162 deletions(-) diff --git a/datafusion/core/src/datasource/physical_plan/parquet/row_groups.rs b/datafusion/core/src/datasource/physical_plan/parquet/row_groups.rs index 8a1abb7d965f..5d18eac7d9fb 100644 --- a/datafusion/core/src/datasource/physical_plan/parquet/row_groups.rs +++ b/datafusion/core/src/datasource/physical_plan/parquet/row_groups.rs @@ -293,15 +293,10 @@ mod tests { use arrow::datatypes::DataType::Decimal128; use arrow::datatypes::Schema; use arrow::datatypes::{DataType, Field}; - use datafusion_common::{config::ConfigOptions, TableReference, ToDFSchema}; - use datafusion_common::{DataFusionError, Result}; - use datafusion_expr::{ - builder::LogicalTableSource, cast, col, lit, AggregateUDF, Expr, ScalarUDF, - TableSource, WindowUDF, - }; + use datafusion_common::{Result, ToDFSchema}; + use datafusion_expr::{cast, col, lit, Expr}; use datafusion_physical_expr::execution_props::ExecutionProps; use datafusion_physical_expr::{create_physical_expr, PhysicalExpr}; - use datafusion_sql::planner::ContextProvider; use parquet::arrow::arrow_to_parquet_schema; use parquet::arrow::async_reader::ParquetObjectReader; use parquet::basic::LogicalType; @@ -1105,13 +1100,18 @@ mod tests { let data = bytes::Bytes::from(std::fs::read(path).unwrap()); // generate pruning predicate - let schema = Schema::new(vec![ - Field::new("String", DataType::Utf8, false), - Field::new("String3", DataType::Utf8, false), - ]); - let sql = - "SELECT * FROM tbl WHERE \"String\" IN ('Hello_Not_Exists', 'Hello_Not_Exists2')"; - let expr = sql_to_physical_plan(sql).unwrap(); + let schema = Schema::new(vec![Field::new("String", DataType::Utf8, false)]); + + let expr = col(r#""String""#).in_list( + vec![ + lit("Hello_Not_Exists"), + lit("Hello_Not_Exists2"), + lit("Hello_Not_Exists3"), + lit("Hello_Not_Exist4"), + ], + false, + ); + let expr = logical2physical(&expr, &schema); let pruning_predicate = PruningPredicate::try_new(expr, Arc::new(schema)).unwrap(); @@ -1312,97 +1312,4 @@ mod tests { Ok(pruned_row_group) } - - fn sql_to_physical_plan(sql: &str) -> Result> { - use datafusion_optimizer::{ - analyzer::Analyzer, optimizer::Optimizer, OptimizerConfig, OptimizerContext, - }; - use datafusion_sql::{ - planner::SqlToRel, - sqlparser::{ast::Statement, parser::Parser}, - }; - use sqlparser::dialect::GenericDialect; - - // parse the SQL - let dialect = GenericDialect {}; // or AnsiDialect, or your own dialect ... - let ast: Vec = Parser::parse_sql(&dialect, sql).unwrap(); - let statement = &ast[0]; - - // create a logical query plan - let schema_provider = TestSchemaProvider::new(); - let sql_to_rel = SqlToRel::new(&schema_provider); - let plan = sql_to_rel.sql_statement_to_plan(statement.clone()).unwrap(); - - // hard code the return value of now() - let config = OptimizerContext::new().with_skip_failing_rules(false); - let analyzer = Analyzer::new(); - let optimizer = Optimizer::new(); - // analyze and optimize the logical plan - let plan = analyzer.execute_and_check(&plan, config.options(), |_, _| {})?; - let plan = optimizer.optimize(&plan, &config, |_, _| {})?; - // convert the logical plan into a physical plan - let exprs = plan.expressions(); - let expr = &exprs[0]; - let df_schema = plan.schema().as_ref().to_owned(); - let tb_schema: Schema = df_schema.clone().into(); - let execution_props = ExecutionProps::new(); - create_physical_expr(expr, &df_schema, &tb_schema, &execution_props) - } - - struct TestSchemaProvider { - options: ConfigOptions, - tables: HashMap>, - } - - impl TestSchemaProvider { - pub fn new() -> Self { - let mut tables = HashMap::new(); - tables.insert( - "tbl".to_string(), - create_table_source(vec![Field::new( - "String".to_string(), - DataType::Utf8, - false, - )]), - ); - - Self { - options: Default::default(), - tables, - } - } - } - - impl ContextProvider for TestSchemaProvider { - fn get_table_source(&self, name: TableReference) -> Result> { - match self.tables.get(name.table()) { - Some(table) => Ok(table.clone()), - _ => datafusion_common::plan_err!("Table not found: {}", name.table()), - } - } - - fn get_function_meta(&self, _name: &str) -> Option> { - None - } - - fn get_aggregate_meta(&self, _name: &str) -> Option> { - None - } - - fn get_variable_type(&self, _variable_names: &[String]) -> Option { - None - } - - fn options(&self) -> &ConfigOptions { - &self.options - } - - fn get_window_meta(&self, _name: &str) -> Option> { - None - } - } - - fn create_table_source(fields: Vec) -> Arc { - Arc::new(LogicalTableSource::new(Arc::new(Schema::new(fields)))) - } } diff --git a/datafusion/physical-expr/src/utils/guarantee.rs b/datafusion/physical-expr/src/utils/guarantee.rs index 59ec255754c0..0aee2af67fdd 100644 --- a/datafusion/physical-expr/src/utils/guarantee.rs +++ b/datafusion/physical-expr/src/utils/guarantee.rs @@ -77,7 +77,7 @@ pub struct LiteralGuarantee { } /// What is guaranteed about the values for a [`LiteralGuarantee`]? -#[derive(Debug, Clone, PartialEq)] +#[derive(Debug, Clone, Copy, PartialEq, Eq, Hash)] pub enum Guarantee { /// Guarantee that the expression is `true` if `column` is one of the values. If /// `column` is not one of the values, the expression can not be `true`. @@ -94,15 +94,9 @@ impl LiteralGuarantee { /// create these structures from an predicate (boolean expression). fn try_new<'a>( column_name: impl Into, - op: Operator, + guarantee: Guarantee, literals: impl IntoIterator, ) -> Option { - let guarantee = match op { - Operator::Eq => Guarantee::In, - Operator::NotEq => Guarantee::NotIn, - _ => return None, - }; - let literals: HashSet<_> = literals.into_iter().cloned().collect(); Some(Self { @@ -120,7 +114,7 @@ impl LiteralGuarantee { /// expression is guaranteed to be `null` or `false`. /// /// # Notes: - /// 1. `expr` must be a boolean expression. + /// 1. `expr` must be a boolean expression or inlist expression. /// 2. `expr` is not simplified prior to analysis. pub fn analyze(expr: &Arc) -> Vec { // split conjunction: AND AND ... @@ -130,6 +124,39 @@ impl LiteralGuarantee { .fold(GuaranteeBuilder::new(), |builder, expr| { if let Some(cel) = ColOpLit::try_new(expr) { return builder.aggregate_conjunct(cel); + } else if let Some(inlist) = expr + .as_any() + .downcast_ref::() + { + // Only support single-column inlist currently, multi-column inlist is not supported + let col = inlist + .expr() + .as_any() + .downcast_ref::(); + let Some(col) = col else { + return builder; + }; + + let literals = inlist + .list() + .iter() + .map(|e| e.as_any().downcast_ref::()) + .collect::>>(); + let Some(literals) = literals else { + return builder; + }; + + let guarantee = if inlist.negated() { + Guarantee::NotIn + } else { + Guarantee::In + }; + + builder.aggregate_multi_conjunct( + col, + guarantee, + literals.iter().map(|e| e.value()), + ) } else { // split disjunction: OR OR ... let disjunctions = split_disjunction(expr); @@ -168,14 +195,21 @@ impl LiteralGuarantee { // if all terms are 'col literal' with the same column // and operation we can infer any guarantees + // + // For those like (a != foo AND (a != bar OR a != baz)). + // We can't combine the (a != bar OR a != baz) part, but + // it also doesn't invalidate our knowledge that a != + // foo is required for the expression to be true. + // So we can only create a multi value guarantee for `=` + // (or a single value). (e.g. ignore `a != foo OR a != bar`) let first_term = &terms[0]; if terms.iter().all(|term| { term.col.name() == first_term.col.name() - && term.op == first_term.op + && term.guarantee == Guarantee::In }) { builder.aggregate_multi_conjunct( first_term.col, - first_term.op, + Guarantee::In, terms.iter().map(|term| term.lit.value()), ) } else { @@ -197,9 +231,9 @@ struct GuaranteeBuilder<'a> { /// e.g. `a = foo AND a = bar` then the relevant guarantee will be None guarantees: Vec>, - /// Key is the (column name, operator type) + /// Key is the (column name, guarantee type) /// Value is the index into `guarantees` - map: HashMap<(&'a crate::expressions::Column, Operator), usize>, + map: HashMap<(&'a crate::expressions::Column, Guarantee), usize>, } impl<'a> GuaranteeBuilder<'a> { @@ -216,7 +250,7 @@ impl<'a> GuaranteeBuilder<'a> { fn aggregate_conjunct(self, col_op_lit: ColOpLit<'a>) -> Self { self.aggregate_multi_conjunct( col_op_lit.col, - col_op_lit.op, + col_op_lit.guarantee, [col_op_lit.lit.value()], ) } @@ -233,10 +267,10 @@ impl<'a> GuaranteeBuilder<'a> { fn aggregate_multi_conjunct( mut self, col: &'a crate::expressions::Column, - op: Operator, + guarantee: Guarantee, new_values: impl IntoIterator, ) -> Self { - let key = (col, op); + let key = (col, guarantee); if let Some(index) = self.map.get(&key) { // already have a guarantee for this column let entry = &mut self.guarantees[*index]; @@ -257,26 +291,20 @@ impl<'a> GuaranteeBuilder<'a> { // another `AND a != 6` we know that a must not be either 5 or 6 // for the expression to be true Guarantee::NotIn => { - // can extend if only single literal, otherwise invalidate let new_values: HashSet<_> = new_values.into_iter().collect(); - if new_values.len() == 1 { - existing.literals.extend(new_values.into_iter().cloned()) - } else { - // this is like (a != foo AND (a != bar OR a != baz)). - // We can't combine the (a != bar OR a != baz) part, but - // it also doesn't invalidate our knowledge that a != - // foo is required for the expression to be true - } + existing.literals.extend(new_values.into_iter().cloned()); } Guarantee::In => { - // for an IN guarantee, it is ok if the value is the same - // e.g. `a = foo AND a = foo` but not if the value is different - // e.g. `a = foo AND a = bar` - if new_values + let intersection = new_values .into_iter() - .all(|new_value| existing.literals.contains(new_value)) - { - // all values are already in the set + .filter(|new_value| existing.literals.contains(*new_value)) + .collect::>(); + // for an In guarantee, if the intersection is not empty, we can extend the guarantee + // e.g. `a IN (1,2,3) AND a IN (2,3,4)` is `a IN (2,3)` + // otherwise, we invalidate the guarantee + // e.g. `a IN (1,2,3) AND a IN (4,5,6)` is `a IN ()`, which is invalid + if !intersection.is_empty() { + existing.literals = intersection.into_iter().cloned().collect(); } else { // at least one was not, so invalidate the guarantee *entry = None; @@ -287,17 +315,12 @@ impl<'a> GuaranteeBuilder<'a> { // This is a new guarantee let new_values: HashSet<_> = new_values.into_iter().collect(); - // new_values are combined with OR, so we can only create a - // multi-column guarantee for `=` (or a single value). - // (e.g. ignore `a != foo OR a != bar`) - if op == Operator::Eq || new_values.len() == 1 { - if let Some(guarantee) = - LiteralGuarantee::try_new(col.name(), op, new_values) - { - // add it to the list of guarantees - self.guarantees.push(Some(guarantee)); - self.map.insert(key, self.guarantees.len() - 1); - } + if let Some(guarantee) = + LiteralGuarantee::try_new(col.name(), guarantee, new_values) + { + // add it to the list of guarantees + self.guarantees.push(Some(guarantee)); + self.map.insert(key, self.guarantees.len() - 1); } } @@ -311,10 +334,10 @@ impl<'a> GuaranteeBuilder<'a> { } } -/// Represents a single `col literal` expression +/// Represents a single `col [not]in literal` expression struct ColOpLit<'a> { col: &'a crate::expressions::Column, - op: Operator, + guarantee: Guarantee, lit: &'a crate::expressions::Literal, } @@ -322,7 +345,7 @@ impl<'a> ColOpLit<'a> { /// Returns Some(ColEqLit) if the expression is either: /// 1. `col literal` /// 2. `literal col` - /// + /// 3. operator is `=` or `!=` /// Returns None otherwise fn try_new(expr: &'a Arc) -> Option { let binary_expr = expr @@ -334,21 +357,32 @@ impl<'a> ColOpLit<'a> { binary_expr.op(), binary_expr.right().as_any(), ); - + let guarantee = match op { + Operator::Eq => Guarantee::In, + Operator::NotEq => Guarantee::NotIn, + _ => return None, + }; // col literal if let (Some(col), Some(lit)) = ( left.downcast_ref::(), right.downcast_ref::(), ) { - Some(Self { col, op: *op, lit }) + Some(Self { + col, + guarantee, + lit, + }) } // literal col else if let (Some(lit), Some(col)) = ( left.downcast_ref::(), right.downcast_ref::(), ) { - // Used swapped operator operator, if possible - op.swap().map(|op| Self { col, op, lit }) + Some(Self { + col, + guarantee, + lit, + }) } else { None } @@ -645,9 +679,122 @@ mod test { ); } - // TODO https://github.com/apache/arrow-datafusion/issues/8436 - // a IN (...) - // b NOT IN (...) + #[test] + fn test_single_inlist() { + // b IN (1, 2, 3) + test_analyze( + col("b").in_list(vec![lit(1), lit(2), lit(3)], false), + vec![in_guarantee("b", [1, 2, 3])], + ); + // b NOT IN (1, 2, 3) + test_analyze( + col("b").in_list(vec![lit(1), lit(2), lit(3)], true), + vec![not_in_guarantee("b", [1, 2, 3])], + ); + } + + #[test] + fn test_inlist_conjunction() { + // b IN (1, 2, 3) AND b IN (2, 3, 4) + test_analyze( + col("b") + .in_list(vec![lit(1), lit(2), lit(3)], false) + .and(col("b").in_list(vec![lit(2), lit(3), lit(4)], false)), + vec![in_guarantee("b", [2, 3])], + ); + // b NOT IN (1, 2, 3) AND b IN (2, 3, 4) + test_analyze( + col("b") + .in_list(vec![lit(1), lit(2), lit(3)], true) + .and(col("b").in_list(vec![lit(2), lit(3), lit(4)], false)), + vec![ + not_in_guarantee("b", [1, 2, 3]), + in_guarantee("b", [2, 3, 4]), + ], + ); + // b NOT IN (1, 2, 3) AND b NOT IN (2, 3, 4) + test_analyze( + col("b") + .in_list(vec![lit(1), lit(2), lit(3)], true) + .and(col("b").in_list(vec![lit(2), lit(3), lit(4)], true)), + vec![not_in_guarantee("b", [1, 2, 3, 4])], + ); + // b IN (1, 2, 3) AND b = 4 + test_analyze( + col("b") + .in_list(vec![lit(1), lit(2), lit(3)], false) + .and(col("b").eq(lit(4))), + vec![], + ); + // b IN (1, 2, 3) AND b = 2 + test_analyze( + col("b") + .in_list(vec![lit(1), lit(2), lit(3)], false) + .and(col("b").eq(lit(2))), + vec![in_guarantee("b", [2])], + ); + // b IN (1, 2, 3) AND b != 2 + test_analyze( + col("b") + .in_list(vec![lit(1), lit(2), lit(3)], false) + .and(col("b").not_eq(lit(2))), + vec![in_guarantee("b", [1, 2, 3]), not_in_guarantee("b", [2])], + ); + // b NOT IN (1, 2, 3) AND b != 4 + test_analyze( + col("b") + .in_list(vec![lit(1), lit(2), lit(3)], true) + .and(col("b").not_eq(lit(4))), + vec![not_in_guarantee("b", [1, 2, 3, 4])], + ); + // b NOT IN (1, 2, 3) AND b != 2 + test_analyze( + col("b") + .in_list(vec![lit(1), lit(2), lit(3)], true) + .and(col("b").not_eq(lit(2))), + vec![not_in_guarantee("b", [1, 2, 3])], + ); + } + + #[test] + fn test_inlist_with_disjunction() { + // b IN (1, 2, 3) AND (b = 3 OR b = 4) + test_analyze( + col("b") + .in_list(vec![lit(1), lit(2), lit(3)], false) + .and(col("b").eq(lit(3)).or(col("b").eq(lit(4)))), + vec![in_guarantee("b", [3])], + ); + // b IN (1, 2, 3) AND (b = 4 OR b = 5) + test_analyze( + col("b") + .in_list(vec![lit(1), lit(2), lit(3)], false) + .and(col("b").eq(lit(4)).or(col("b").eq(lit(5)))), + vec![], + ); + // b NOT IN (1, 2, 3) AND (b = 3 OR b = 4) + test_analyze( + col("b") + .in_list(vec![lit(1), lit(2), lit(3)], true) + .and(col("b").eq(lit(3)).or(col("b").eq(lit(4)))), + vec![not_in_guarantee("b", [1, 2, 3]), in_guarantee("b", [3, 4])], + ); + // b IN (1, 2, 3) OR b = 2 + // TODO this should be in_guarantee("b", [1, 2, 3]) but currently we don't support to anylize this kind of disjunction. Only `ColOpLit OR ColOpLit` is supported. + test_analyze( + col("b") + .in_list(vec![lit(1), lit(2), lit(3)], false) + .or(col("b").eq(lit(2))), + vec![], + ); + // b IN (1, 2, 3) OR b != 3 + test_analyze( + col("b") + .in_list(vec![lit(1), lit(2), lit(3)], false) + .or(col("b").not_eq(lit(3))), + vec![], + ); + } /// Tests that analyzing expr results in the expected guarantees fn test_analyze(expr: Expr, expected: Vec) { @@ -673,7 +820,7 @@ mod test { S: Into + 'a, { let literals: Vec<_> = literals.into_iter().map(|s| s.into()).collect(); - LiteralGuarantee::try_new(column, Operator::Eq, literals.iter()).unwrap() + LiteralGuarantee::try_new(column, Guarantee::In, literals.iter()).unwrap() } /// Guarantee that the expression is true if the column is NOT any of the specified values @@ -683,7 +830,7 @@ mod test { S: Into + 'a, { let literals: Vec<_> = literals.into_iter().map(|s| s.into()).collect(); - LiteralGuarantee::try_new(column, Operator::NotEq, literals.iter()).unwrap() + LiteralGuarantee::try_new(column, Guarantee::NotIn, literals.iter()).unwrap() } /// Convert a logical expression to a physical expression (without any simplification, etc) From fba5cc0b9062297e38cbe388d7f1b13debe8ba92 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Berkay=20=C5=9Eahin?= <124376117+berkaysynnada@users.noreply.github.com> Date: Thu, 28 Dec 2023 15:27:21 +0300 Subject: [PATCH 507/572] Streaming CLI support (#8651) * Streaming CLI support * Update Cargo.toml * Remove duplications * Clean up * Stream test will be added * Update print_format.rs * Address feedback * Final fix --------- Co-authored-by: Mehmet Ozan Kabak --- Cargo.toml | 2 +- datafusion-cli/Cargo.lock | 1 + datafusion-cli/Cargo.toml | 1 + datafusion-cli/src/exec.rs | 66 +++-- datafusion-cli/src/main.rs | 19 +- datafusion-cli/src/print_format.rs | 278 +++++++++++------- datafusion-cli/src/print_options.rs | 74 ++++- .../core/src/datasource/physical_plan/mod.rs | 15 + 8 files changed, 295 insertions(+), 161 deletions(-) diff --git a/Cargo.toml b/Cargo.toml index a698fbf471f9..4ee29ea6298c 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -36,7 +36,7 @@ arrow = { version = "49.0.0", features = ["prettyprint"] } arrow-array = { version = "49.0.0", default-features = false, features = ["chrono-tz"] } arrow-buffer = { version = "49.0.0", default-features = false } arrow-flight = { version = "49.0.0", features = ["flight-sql-experimental"] } -arrow-ipc = { version = "49.0.0", default-features = false, features=["lz4"] } +arrow-ipc = { version = "49.0.0", default-features = false, features = ["lz4"] } arrow-ord = { version = "49.0.0", default-features = false } arrow-schema = { version = "49.0.0", default-features = false } async-trait = "0.1.73" diff --git a/datafusion-cli/Cargo.lock b/datafusion-cli/Cargo.lock index 9f75013c86dc..8e9bbd8a0dfd 100644 --- a/datafusion-cli/Cargo.lock +++ b/datafusion-cli/Cargo.lock @@ -1160,6 +1160,7 @@ dependencies = [ "datafusion-common", "dirs", "env_logger", + "futures", "mimalloc", "object_store", "parking_lot", diff --git a/datafusion-cli/Cargo.toml b/datafusion-cli/Cargo.toml index f57097683698..e1ddba4cad1a 100644 --- a/datafusion-cli/Cargo.toml +++ b/datafusion-cli/Cargo.toml @@ -38,6 +38,7 @@ datafusion = { path = "../datafusion/core", version = "34.0.0", features = ["avr datafusion-common = { path = "../datafusion/common" } dirs = "4.0.0" env_logger = "0.9" +futures = "0.3" mimalloc = { version = "0.1", default-features = false } object_store = { version = "0.8.0", features = ["aws", "gcp"] } parking_lot = { version = "0.12" } diff --git a/datafusion-cli/src/exec.rs b/datafusion-cli/src/exec.rs index 8af534cd1375..ba9aa2e69aa6 100644 --- a/datafusion-cli/src/exec.rs +++ b/datafusion-cli/src/exec.rs @@ -17,6 +17,12 @@ //! Execution functions +use std::io::prelude::*; +use std::io::BufReader; +use std::time::Instant; +use std::{fs::File, sync::Arc}; + +use crate::print_format::PrintFormat; use crate::{ command::{Command, OutputFormat}, helper::{unescape_input, CliHelper}, @@ -26,21 +32,19 @@ use crate::{ }, print_options::{MaxRows, PrintOptions}, }; -use datafusion::common::plan_datafusion_err; + +use datafusion::common::{exec_datafusion_err, plan_datafusion_err}; +use datafusion::datasource::listing::ListingTableUrl; +use datafusion::datasource::physical_plan::is_plan_streaming; +use datafusion::error::{DataFusionError, Result}; +use datafusion::logical_expr::{CreateExternalTable, DdlStatement, LogicalPlan}; +use datafusion::physical_plan::{collect, execute_stream}; +use datafusion::prelude::SessionContext; use datafusion::sql::{parser::DFParser, sqlparser::dialect::dialect_from_str}; -use datafusion::{ - datasource::listing::ListingTableUrl, - error::{DataFusionError, Result}, - logical_expr::{CreateExternalTable, DdlStatement}, -}; -use datafusion::{logical_expr::LogicalPlan, prelude::SessionContext}; + use object_store::ObjectStore; use rustyline::error::ReadlineError; use rustyline::Editor; -use std::io::prelude::*; -use std::io::BufReader; -use std::time::Instant; -use std::{fs::File, sync::Arc}; use url::Url; /// run and execute SQL statements and commands, against a context with the given print options @@ -125,8 +129,6 @@ pub async fn exec_from_repl( ))); rl.load_history(".history").ok(); - let mut print_options = print_options.clone(); - loop { match rl.readline("❯ ") { Ok(line) if line.starts_with('\\') => { @@ -138,9 +140,7 @@ pub async fn exec_from_repl( Command::OutputFormat(subcommand) => { if let Some(subcommand) = subcommand { if let Ok(command) = subcommand.parse::() { - if let Err(e) = - command.execute(&mut print_options).await - { + if let Err(e) = command.execute(print_options).await { eprintln!("{e}") } } else { @@ -154,7 +154,7 @@ pub async fn exec_from_repl( } } _ => { - if let Err(e) = cmd.execute(ctx, &mut print_options).await { + if let Err(e) = cmd.execute(ctx, print_options).await { eprintln!("{e}") } } @@ -165,7 +165,7 @@ pub async fn exec_from_repl( } Ok(line) => { rl.add_history_entry(line.trim_end())?; - match exec_and_print(ctx, &print_options, line).await { + match exec_and_print(ctx, print_options, line).await { Ok(_) => {} Err(err) => eprintln!("{err}"), } @@ -198,7 +198,6 @@ async fn exec_and_print( sql: String, ) -> Result<()> { let now = Instant::now(); - let sql = unescape_input(&sql)?; let task_ctx = ctx.task_ctx(); let dialect = &task_ctx.session_config().options().sql_parser.dialect; @@ -227,18 +226,24 @@ async fn exec_and_print( if let LogicalPlan::Ddl(DdlStatement::CreateExternalTable(cmd)) = &mut plan { create_external_table(ctx, cmd).await?; } + let df = ctx.execute_logical_plan(plan).await?; - let results = df.collect().await?; + let physical_plan = df.create_physical_plan().await?; - let print_options = if should_ignore_maxrows { - PrintOptions { - maxrows: MaxRows::Unlimited, - ..print_options.clone() - } + if is_plan_streaming(&physical_plan)? { + let stream = execute_stream(physical_plan, task_ctx.clone())?; + print_options.print_stream(stream, now).await?; } else { - print_options.clone() - }; - print_options.print_batches(&results, now)?; + let mut print_options = print_options.clone(); + if should_ignore_maxrows { + print_options.maxrows = MaxRows::Unlimited; + } + if print_options.format == PrintFormat::Automatic { + print_options.format = PrintFormat::Table; + } + let results = collect(physical_plan, task_ctx.clone()).await?; + print_options.print_batches(&results, now)?; + } } Ok(()) @@ -272,10 +277,7 @@ async fn create_external_table( .object_store_registry .get_store(url) .map_err(|_| { - DataFusionError::Execution(format!( - "Unsupported object store scheme: {}", - scheme - )) + exec_datafusion_err!("Unsupported object store scheme: {}", scheme) })? } }; diff --git a/datafusion-cli/src/main.rs b/datafusion-cli/src/main.rs index 8b74a797b57b..563d172f2c95 100644 --- a/datafusion-cli/src/main.rs +++ b/datafusion-cli/src/main.rs @@ -15,7 +15,12 @@ // specific language governing permissions and limitations // under the License. -use clap::Parser; +use std::collections::HashMap; +use std::env; +use std::path::Path; +use std::str::FromStr; +use std::sync::{Arc, OnceLock}; + use datafusion::error::{DataFusionError, Result}; use datafusion::execution::context::SessionConfig; use datafusion::execution::memory_pool::{FairSpillPool, GreedyMemoryPool}; @@ -29,12 +34,9 @@ use datafusion_cli::{ print_options::{MaxRows, PrintOptions}, DATAFUSION_CLI_VERSION, }; + +use clap::Parser; use mimalloc::MiMalloc; -use std::collections::HashMap; -use std::env; -use std::path::Path; -use std::str::FromStr; -use std::sync::{Arc, OnceLock}; #[global_allocator] static GLOBAL: MiMalloc = MiMalloc; @@ -111,7 +113,7 @@ struct Args { )] rc: Option>, - #[clap(long, arg_enum, default_value_t = PrintFormat::Table)] + #[clap(long, arg_enum, default_value_t = PrintFormat::Automatic)] format: PrintFormat, #[clap( @@ -331,9 +333,8 @@ fn extract_memory_pool_size(size: &str) -> Result { #[cfg(test)] mod tests { - use datafusion::assert_batches_eq; - use super::*; + use datafusion::assert_batches_eq; fn assert_conversion(input: &str, expected: Result) { let result = extract_memory_pool_size(input); diff --git a/datafusion-cli/src/print_format.rs b/datafusion-cli/src/print_format.rs index 0738bf6f9b47..ea418562495d 100644 --- a/datafusion-cli/src/print_format.rs +++ b/datafusion-cli/src/print_format.rs @@ -16,23 +16,27 @@ // under the License. //! Print format variants + +use std::str::FromStr; + use crate::print_options::MaxRows; + use arrow::csv::writer::WriterBuilder; use arrow::json::{ArrayWriter, LineDelimitedWriter}; +use arrow::record_batch::RecordBatch; use arrow::util::pretty::pretty_format_batches_with_options; -use datafusion::arrow::record_batch::RecordBatch; use datafusion::common::format::DEFAULT_FORMAT_OPTIONS; -use datafusion::error::{DataFusionError, Result}; -use std::str::FromStr; +use datafusion::error::Result; /// Allow records to be printed in different formats -#[derive(Debug, PartialEq, Eq, clap::ArgEnum, Clone)] +#[derive(Debug, PartialEq, Eq, clap::ArgEnum, Clone, Copy)] pub enum PrintFormat { Csv, Tsv, Table, Json, NdJson, + Automatic, } impl FromStr for PrintFormat { @@ -44,31 +48,44 @@ impl FromStr for PrintFormat { } macro_rules! batches_to_json { - ($WRITER: ident, $batches: expr) => {{ - let mut bytes = vec![]; + ($WRITER: ident, $writer: expr, $batches: expr) => {{ { - let mut writer = $WRITER::new(&mut bytes); - $batches.iter().try_for_each(|batch| writer.write(batch))?; - writer.finish()?; + if !$batches.is_empty() { + let mut json_writer = $WRITER::new(&mut *$writer); + for batch in $batches { + json_writer.write(batch)?; + } + json_writer.finish()?; + json_finish!($WRITER, $writer); + } } - String::from_utf8(bytes).map_err(|e| DataFusionError::External(Box::new(e)))? + Ok(()) as Result<()> }}; } -fn print_batches_with_sep(batches: &[RecordBatch], delimiter: u8) -> Result { - let mut bytes = vec![]; - { - let builder = WriterBuilder::new() - .with_header(true) - .with_delimiter(delimiter); - let mut writer = builder.build(&mut bytes); - for batch in batches { - writer.write(batch)?; - } +macro_rules! json_finish { + (ArrayWriter, $writer: expr) => {{ + writeln!($writer)?; + }}; + (LineDelimitedWriter, $writer: expr) => {{}}; +} + +fn print_batches_with_sep( + writer: &mut W, + batches: &[RecordBatch], + delimiter: u8, + with_header: bool, +) -> Result<()> { + let builder = WriterBuilder::new() + .with_header(with_header) + .with_delimiter(delimiter); + let mut csv_writer = builder.build(writer); + + for batch in batches { + csv_writer.write(batch)?; } - let formatted = - String::from_utf8(bytes).map_err(|e| DataFusionError::External(Box::new(e)))?; - Ok(formatted) + + Ok(()) } fn keep_only_maxrows(s: &str, maxrows: usize) -> String { @@ -88,97 +105,118 @@ fn keep_only_maxrows(s: &str, maxrows: usize) -> String { result.join("\n") } -fn format_batches_with_maxrows( +fn format_batches_with_maxrows( + writer: &mut W, batches: &[RecordBatch], maxrows: MaxRows, -) -> Result { +) -> Result<()> { match maxrows { MaxRows::Limited(maxrows) => { - // Only format enough batches for maxrows + // Filter batches to meet the maxrows condition let mut filtered_batches = Vec::new(); - let mut batches = batches; - let row_count: usize = batches.iter().map(|b| b.num_rows()).sum(); - if row_count > maxrows { - let mut accumulated_rows = 0; - - for batch in batches { + let mut row_count: usize = 0; + let mut over_limit = false; + for batch in batches { + if row_count + batch.num_rows() > maxrows { + // If adding this batch exceeds maxrows, slice the batch + let limit = maxrows - row_count; + let sliced_batch = batch.slice(0, limit); + filtered_batches.push(sliced_batch); + over_limit = true; + break; + } else { filtered_batches.push(batch.clone()); - if accumulated_rows + batch.num_rows() > maxrows { - break; - } - accumulated_rows += batch.num_rows(); + row_count += batch.num_rows(); } - - batches = &filtered_batches; } - let mut formatted = format!( - "{}", - pretty_format_batches_with_options(batches, &DEFAULT_FORMAT_OPTIONS)?, - ); - - if row_count > maxrows { - formatted = keep_only_maxrows(&formatted, maxrows); + let formatted = pretty_format_batches_with_options( + &filtered_batches, + &DEFAULT_FORMAT_OPTIONS, + )?; + if over_limit { + let mut formatted_str = format!("{}", formatted); + formatted_str = keep_only_maxrows(&formatted_str, maxrows); + writeln!(writer, "{}", formatted_str)?; + } else { + writeln!(writer, "{}", formatted)?; } - - Ok(formatted) } MaxRows::Unlimited => { - // maxrows not specified, print all rows - Ok(format!( - "{}", - pretty_format_batches_with_options(batches, &DEFAULT_FORMAT_OPTIONS)?, - )) + let formatted = + pretty_format_batches_with_options(batches, &DEFAULT_FORMAT_OPTIONS)?; + writeln!(writer, "{}", formatted)?; } } + + Ok(()) } impl PrintFormat { - /// print the batches to stdout using the specified format - /// `maxrows` option is only used for `Table` format: - /// If `maxrows` is Some(n), then at most n rows will be displayed - /// If `maxrows` is None, then every row will be displayed - pub fn print_batches(&self, batches: &[RecordBatch], maxrows: MaxRows) -> Result<()> { - if batches.is_empty() { + /// Print the batches to a writer using the specified format + pub fn print_batches( + &self, + writer: &mut W, + batches: &[RecordBatch], + maxrows: MaxRows, + with_header: bool, + ) -> Result<()> { + if batches.is_empty() || batches[0].num_rows() == 0 { return Ok(()); } match self { - Self::Csv => println!("{}", print_batches_with_sep(batches, b',')?), - Self::Tsv => println!("{}", print_batches_with_sep(batches, b'\t')?), + Self::Csv | Self::Automatic => { + print_batches_with_sep(writer, batches, b',', with_header) + } + Self::Tsv => print_batches_with_sep(writer, batches, b'\t', with_header), Self::Table => { if maxrows == MaxRows::Limited(0) { return Ok(()); } - println!("{}", format_batches_with_maxrows(batches, maxrows)?,) - } - Self::Json => println!("{}", batches_to_json!(ArrayWriter, batches)), - Self::NdJson => { - println!("{}", batches_to_json!(LineDelimitedWriter, batches)) + format_batches_with_maxrows(writer, batches, maxrows) } + Self::Json => batches_to_json!(ArrayWriter, writer, batches), + Self::NdJson => batches_to_json!(LineDelimitedWriter, writer, batches), } - Ok(()) } } #[cfg(test)] mod tests { + use std::io::{Cursor, Read, Write}; + use std::sync::Arc; + use super::*; + use arrow::array::Int32Array; use arrow::datatypes::{DataType, Field, Schema}; - use std::sync::Arc; + use datafusion::error::Result; + + fn run_test(batches: &[RecordBatch], test_fn: F) -> Result + where + F: Fn(&mut Cursor>, &[RecordBatch]) -> Result<()>, + { + let mut buffer = Cursor::new(Vec::new()); + test_fn(&mut buffer, batches)?; + buffer.set_position(0); + let mut contents = String::new(); + buffer.read_to_string(&mut contents)?; + Ok(contents) + } #[test] - fn test_print_batches_with_sep() { - let batches = vec![]; - assert_eq!("", print_batches_with_sep(&batches, b',').unwrap()); + fn test_print_batches_with_sep() -> Result<()> { + let contents = run_test(&[], |buffer, batches| { + print_batches_with_sep(buffer, batches, b',', true) + })?; + assert_eq!(contents, ""); let schema = Arc::new(Schema::new(vec![ Field::new("a", DataType::Int32, false), Field::new("b", DataType::Int32, false), Field::new("c", DataType::Int32, false), ])); - let batch = RecordBatch::try_new( schema, vec![ @@ -186,29 +224,33 @@ mod tests { Arc::new(Int32Array::from(vec![4, 5, 6])), Arc::new(Int32Array::from(vec![7, 8, 9])), ], - ) - .unwrap(); + )?; - let batches = vec![batch]; - let r = print_batches_with_sep(&batches, b',').unwrap(); - assert_eq!("a,b,c\n1,4,7\n2,5,8\n3,6,9\n", r); + let contents = run_test(&[batch], |buffer, batches| { + print_batches_with_sep(buffer, batches, b',', true) + })?; + assert_eq!(contents, "a,b,c\n1,4,7\n2,5,8\n3,6,9\n"); + + Ok(()) } #[test] fn test_print_batches_to_json_empty() -> Result<()> { - let batches = vec![]; - let r = batches_to_json!(ArrayWriter, &batches); - assert_eq!("", r); + let contents = run_test(&[], |buffer, batches| { + batches_to_json!(ArrayWriter, buffer, batches) + })?; + assert_eq!(contents, ""); - let r = batches_to_json!(LineDelimitedWriter, &batches); - assert_eq!("", r); + let contents = run_test(&[], |buffer, batches| { + batches_to_json!(LineDelimitedWriter, buffer, batches) + })?; + assert_eq!(contents, ""); let schema = Arc::new(Schema::new(vec![ Field::new("a", DataType::Int32, false), Field::new("b", DataType::Int32, false), Field::new("c", DataType::Int32, false), ])); - let batch = RecordBatch::try_new( schema, vec![ @@ -216,25 +258,29 @@ mod tests { Arc::new(Int32Array::from(vec![4, 5, 6])), Arc::new(Int32Array::from(vec![7, 8, 9])), ], - ) - .unwrap(); - + )?; let batches = vec![batch]; - let r = batches_to_json!(ArrayWriter, &batches); - assert_eq!("[{\"a\":1,\"b\":4,\"c\":7},{\"a\":2,\"b\":5,\"c\":8},{\"a\":3,\"b\":6,\"c\":9}]", r); - let r = batches_to_json!(LineDelimitedWriter, &batches); - assert_eq!("{\"a\":1,\"b\":4,\"c\":7}\n{\"a\":2,\"b\":5,\"c\":8}\n{\"a\":3,\"b\":6,\"c\":9}\n", r); + let contents = run_test(&batches, |buffer, batches| { + batches_to_json!(ArrayWriter, buffer, batches) + })?; + assert_eq!(contents, "[{\"a\":1,\"b\":4,\"c\":7},{\"a\":2,\"b\":5,\"c\":8},{\"a\":3,\"b\":6,\"c\":9}]\n"); + + let contents = run_test(&batches, |buffer, batches| { + batches_to_json!(LineDelimitedWriter, buffer, batches) + })?; + assert_eq!(contents, "{\"a\":1,\"b\":4,\"c\":7}\n{\"a\":2,\"b\":5,\"c\":8}\n{\"a\":3,\"b\":6,\"c\":9}\n"); + Ok(()) } #[test] fn test_format_batches_with_maxrows() -> Result<()> { let schema = Arc::new(Schema::new(vec![Field::new("a", DataType::Int32, false)])); - - let batch = - RecordBatch::try_new(schema, vec![Arc::new(Int32Array::from(vec![1, 2, 3]))]) - .unwrap(); + let batch = RecordBatch::try_new( + schema, + vec![Arc::new(Int32Array::from(vec![1, 2, 3]))], + )?; #[rustfmt::skip] let all_rows_expected = [ @@ -244,7 +290,7 @@ mod tests { "| 1 |", "| 2 |", "| 3 |", - "+---+", + "+---+\n", ].join("\n"); #[rustfmt::skip] @@ -256,7 +302,7 @@ mod tests { "| . |", "| . |", "| . |", - "+---+", + "+---+\n", ].join("\n"); #[rustfmt::skip] @@ -272,26 +318,36 @@ mod tests { "| . |", "| . |", "| . |", - "+---+", + "+---+\n", ].join("\n"); - let no_limit = format_batches_with_maxrows(&[batch.clone()], MaxRows::Unlimited)?; - assert_eq!(all_rows_expected, no_limit); - - let maxrows_less_than_actual = - format_batches_with_maxrows(&[batch.clone()], MaxRows::Limited(1))?; - assert_eq!(one_row_expected, maxrows_less_than_actual); - let maxrows_more_than_actual = - format_batches_with_maxrows(&[batch.clone()], MaxRows::Limited(5))?; - assert_eq!(all_rows_expected, maxrows_more_than_actual); - let maxrows_equals_actual = - format_batches_with_maxrows(&[batch.clone()], MaxRows::Limited(3))?; - assert_eq!(all_rows_expected, maxrows_equals_actual); - let multi_batches = format_batches_with_maxrows( + let no_limit = run_test(&[batch.clone()], |buffer, batches| { + format_batches_with_maxrows(buffer, batches, MaxRows::Unlimited) + })?; + assert_eq!(no_limit, all_rows_expected); + + let maxrows_less_than_actual = run_test(&[batch.clone()], |buffer, batches| { + format_batches_with_maxrows(buffer, batches, MaxRows::Limited(1)) + })?; + assert_eq!(maxrows_less_than_actual, one_row_expected); + + let maxrows_more_than_actual = run_test(&[batch.clone()], |buffer, batches| { + format_batches_with_maxrows(buffer, batches, MaxRows::Limited(5)) + })?; + assert_eq!(maxrows_more_than_actual, all_rows_expected); + + let maxrows_equals_actual = run_test(&[batch.clone()], |buffer, batches| { + format_batches_with_maxrows(buffer, batches, MaxRows::Limited(3)) + })?; + assert_eq!(maxrows_equals_actual, all_rows_expected); + + let multi_batches = run_test( &[batch.clone(), batch.clone(), batch.clone()], - MaxRows::Limited(5), + |buffer, batches| { + format_batches_with_maxrows(buffer, batches, MaxRows::Limited(5)) + }, )?; - assert_eq!(multi_batches_expected, multi_batches); + assert_eq!(multi_batches, multi_batches_expected); Ok(()) } diff --git a/datafusion-cli/src/print_options.rs b/datafusion-cli/src/print_options.rs index 0a6c8d4c36fc..b8594352b585 100644 --- a/datafusion-cli/src/print_options.rs +++ b/datafusion-cli/src/print_options.rs @@ -15,13 +15,21 @@ // specific language governing permissions and limitations // under the License. -use crate::print_format::PrintFormat; -use datafusion::arrow::record_batch::RecordBatch; -use datafusion::error::Result; use std::fmt::{Display, Formatter}; +use std::io::Write; +use std::pin::Pin; use std::str::FromStr; use std::time::Instant; +use crate::print_format::PrintFormat; + +use arrow::record_batch::RecordBatch; +use datafusion::common::DataFusionError; +use datafusion::error::Result; +use datafusion::physical_plan::RecordBatchStream; + +use futures::StreamExt; + #[derive(Debug, Clone, PartialEq, Copy)] pub enum MaxRows { /// show all rows in the output @@ -85,20 +93,70 @@ fn get_timing_info_str( } impl PrintOptions { - /// print the batches to stdout using the specified format + /// Print the batches to stdout using the specified format pub fn print_batches( &self, batches: &[RecordBatch], query_start_time: Instant, ) -> Result<()> { + let stdout = std::io::stdout(); + let mut writer = stdout.lock(); + + self.format + .print_batches(&mut writer, batches, self.maxrows, true)?; + let row_count: usize = batches.iter().map(|b| b.num_rows()).sum(); - // Elapsed time should not count time for printing batches - let timing_info = get_timing_info_str(row_count, self.maxrows, query_start_time); + let timing_info = get_timing_info_str( + row_count, + if self.format == PrintFormat::Table { + self.maxrows + } else { + MaxRows::Unlimited + }, + query_start_time, + ); + + if !self.quiet { + writeln!(writer, "{timing_info}")?; + } + + Ok(()) + } + + /// Print the stream to stdout using the specified format + pub async fn print_stream( + &self, + mut stream: Pin>, + query_start_time: Instant, + ) -> Result<()> { + if self.format == PrintFormat::Table { + return Err(DataFusionError::External( + "PrintFormat::Table is not implemented".to_string().into(), + )); + }; + + let stdout = std::io::stdout(); + let mut writer = stdout.lock(); + + let mut row_count = 0_usize; + let mut with_header = true; + + while let Some(Ok(batch)) = stream.next().await { + row_count += batch.num_rows(); + self.format.print_batches( + &mut writer, + &[batch], + MaxRows::Unlimited, + with_header, + )?; + with_header = false; + } - self.format.print_batches(batches, self.maxrows)?; + let timing_info = + get_timing_info_str(row_count, MaxRows::Unlimited, query_start_time); if !self.quiet { - println!("{timing_info}"); + writeln!(writer, "{timing_info}")?; } Ok(()) diff --git a/datafusion/core/src/datasource/physical_plan/mod.rs b/datafusion/core/src/datasource/physical_plan/mod.rs index 4a6ebeab09e1..5583991355c6 100644 --- a/datafusion/core/src/datasource/physical_plan/mod.rs +++ b/datafusion/core/src/datasource/physical_plan/mod.rs @@ -69,6 +69,7 @@ use arrow::{ use datafusion_common::{file_options::FileTypeWriterOptions, plan_err}; use datafusion_physical_expr::expressions::Column; use datafusion_physical_expr::PhysicalSortExpr; +use datafusion_physical_plan::ExecutionPlan; use log::debug; use object_store::path::Path; @@ -507,6 +508,20 @@ fn get_projected_output_ordering( all_orderings } +/// Get output (un)boundedness information for the given `plan`. +pub fn is_plan_streaming(plan: &Arc) -> Result { + if plan.children().is_empty() { + plan.unbounded_output(&[]) + } else { + let children_unbounded_output = plan + .children() + .iter() + .map(is_plan_streaming) + .collect::>>(); + plan.unbounded_output(&children_unbounded_output?) + } +} + #[cfg(test)] mod tests { use arrow_array::cast::AsArray; From f39c040ace0b34b0775827907aa01d6bb71cbb14 Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Thu, 28 Dec 2023 11:38:16 -0700 Subject: [PATCH 508/572] Add serde support for CSV FileTypeWriterOptions (#8641) --- datafusion/proto/proto/datafusion.proto | 18 ++ datafusion/proto/src/generated/pbjson.rs | 213 ++++++++++++++++++ datafusion/proto/src/generated/prost.rs | 29 ++- datafusion/proto/src/logical_plan/mod.rs | 74 ++++++ .../proto/src/physical_plan/from_proto.rs | 12 +- .../tests/cases/roundtrip_logical_plan.rs | 64 +++++- 6 files changed, 406 insertions(+), 4 deletions(-) diff --git a/datafusion/proto/proto/datafusion.proto b/datafusion/proto/proto/datafusion.proto index d02fc8e91b41..59b82efcbb43 100644 --- a/datafusion/proto/proto/datafusion.proto +++ b/datafusion/proto/proto/datafusion.proto @@ -1207,6 +1207,7 @@ message FileTypeWriterOptions { oneof FileType { JsonWriterOptions json_options = 1; ParquetWriterOptions parquet_options = 2; + CsvWriterOptions csv_options = 3; } } @@ -1218,6 +1219,23 @@ message ParquetWriterOptions { WriterProperties writer_properties = 1; } +message CsvWriterOptions { + // Optional column delimiter. Defaults to `b','` + string delimiter = 1; + // Whether to write column names as file headers. Defaults to `true` + bool has_header = 2; + // Optional date format for date arrays + string date_format = 3; + // Optional datetime format for datetime arrays + string datetime_format = 4; + // Optional timestamp format for timestamp arrays + string timestamp_format = 5; + // Optional time format for time arrays + string time_format = 6; + // Optional value to represent null + string null_value = 7; +} + message WriterProperties { uint64 data_page_size_limit = 1; uint64 dictionary_page_size_limit = 2; diff --git a/datafusion/proto/src/generated/pbjson.rs b/datafusion/proto/src/generated/pbjson.rs index f860b1f1e6a0..956244ffdbc2 100644 --- a/datafusion/proto/src/generated/pbjson.rs +++ b/datafusion/proto/src/generated/pbjson.rs @@ -5151,6 +5151,205 @@ impl<'de> serde::Deserialize<'de> for CsvScanExecNode { deserializer.deserialize_struct("datafusion.CsvScanExecNode", FIELDS, GeneratedVisitor) } } +impl serde::Serialize for CsvWriterOptions { + #[allow(deprecated)] + fn serialize(&self, serializer: S) -> std::result::Result + where + S: serde::Serializer, + { + use serde::ser::SerializeStruct; + let mut len = 0; + if !self.delimiter.is_empty() { + len += 1; + } + if self.has_header { + len += 1; + } + if !self.date_format.is_empty() { + len += 1; + } + if !self.datetime_format.is_empty() { + len += 1; + } + if !self.timestamp_format.is_empty() { + len += 1; + } + if !self.time_format.is_empty() { + len += 1; + } + if !self.null_value.is_empty() { + len += 1; + } + let mut struct_ser = serializer.serialize_struct("datafusion.CsvWriterOptions", len)?; + if !self.delimiter.is_empty() { + struct_ser.serialize_field("delimiter", &self.delimiter)?; + } + if self.has_header { + struct_ser.serialize_field("hasHeader", &self.has_header)?; + } + if !self.date_format.is_empty() { + struct_ser.serialize_field("dateFormat", &self.date_format)?; + } + if !self.datetime_format.is_empty() { + struct_ser.serialize_field("datetimeFormat", &self.datetime_format)?; + } + if !self.timestamp_format.is_empty() { + struct_ser.serialize_field("timestampFormat", &self.timestamp_format)?; + } + if !self.time_format.is_empty() { + struct_ser.serialize_field("timeFormat", &self.time_format)?; + } + if !self.null_value.is_empty() { + struct_ser.serialize_field("nullValue", &self.null_value)?; + } + struct_ser.end() + } +} +impl<'de> serde::Deserialize<'de> for CsvWriterOptions { + #[allow(deprecated)] + fn deserialize(deserializer: D) -> std::result::Result + where + D: serde::Deserializer<'de>, + { + const FIELDS: &[&str] = &[ + "delimiter", + "has_header", + "hasHeader", + "date_format", + "dateFormat", + "datetime_format", + "datetimeFormat", + "timestamp_format", + "timestampFormat", + "time_format", + "timeFormat", + "null_value", + "nullValue", + ]; + + #[allow(clippy::enum_variant_names)] + enum GeneratedField { + Delimiter, + HasHeader, + DateFormat, + DatetimeFormat, + TimestampFormat, + TimeFormat, + NullValue, + } + impl<'de> serde::Deserialize<'de> for GeneratedField { + fn deserialize(deserializer: D) -> std::result::Result + where + D: serde::Deserializer<'de>, + { + struct GeneratedVisitor; + + impl<'de> serde::de::Visitor<'de> for GeneratedVisitor { + type Value = GeneratedField; + + fn expecting(&self, formatter: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + write!(formatter, "expected one of: {:?}", &FIELDS) + } + + #[allow(unused_variables)] + fn visit_str(self, value: &str) -> std::result::Result + where + E: serde::de::Error, + { + match value { + "delimiter" => Ok(GeneratedField::Delimiter), + "hasHeader" | "has_header" => Ok(GeneratedField::HasHeader), + "dateFormat" | "date_format" => Ok(GeneratedField::DateFormat), + "datetimeFormat" | "datetime_format" => Ok(GeneratedField::DatetimeFormat), + "timestampFormat" | "timestamp_format" => Ok(GeneratedField::TimestampFormat), + "timeFormat" | "time_format" => Ok(GeneratedField::TimeFormat), + "nullValue" | "null_value" => Ok(GeneratedField::NullValue), + _ => Err(serde::de::Error::unknown_field(value, FIELDS)), + } + } + } + deserializer.deserialize_identifier(GeneratedVisitor) + } + } + struct GeneratedVisitor; + impl<'de> serde::de::Visitor<'de> for GeneratedVisitor { + type Value = CsvWriterOptions; + + fn expecting(&self, formatter: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + formatter.write_str("struct datafusion.CsvWriterOptions") + } + + fn visit_map(self, mut map_: V) -> std::result::Result + where + V: serde::de::MapAccess<'de>, + { + let mut delimiter__ = None; + let mut has_header__ = None; + let mut date_format__ = None; + let mut datetime_format__ = None; + let mut timestamp_format__ = None; + let mut time_format__ = None; + let mut null_value__ = None; + while let Some(k) = map_.next_key()? { + match k { + GeneratedField::Delimiter => { + if delimiter__.is_some() { + return Err(serde::de::Error::duplicate_field("delimiter")); + } + delimiter__ = Some(map_.next_value()?); + } + GeneratedField::HasHeader => { + if has_header__.is_some() { + return Err(serde::de::Error::duplicate_field("hasHeader")); + } + has_header__ = Some(map_.next_value()?); + } + GeneratedField::DateFormat => { + if date_format__.is_some() { + return Err(serde::de::Error::duplicate_field("dateFormat")); + } + date_format__ = Some(map_.next_value()?); + } + GeneratedField::DatetimeFormat => { + if datetime_format__.is_some() { + return Err(serde::de::Error::duplicate_field("datetimeFormat")); + } + datetime_format__ = Some(map_.next_value()?); + } + GeneratedField::TimestampFormat => { + if timestamp_format__.is_some() { + return Err(serde::de::Error::duplicate_field("timestampFormat")); + } + timestamp_format__ = Some(map_.next_value()?); + } + GeneratedField::TimeFormat => { + if time_format__.is_some() { + return Err(serde::de::Error::duplicate_field("timeFormat")); + } + time_format__ = Some(map_.next_value()?); + } + GeneratedField::NullValue => { + if null_value__.is_some() { + return Err(serde::de::Error::duplicate_field("nullValue")); + } + null_value__ = Some(map_.next_value()?); + } + } + } + Ok(CsvWriterOptions { + delimiter: delimiter__.unwrap_or_default(), + has_header: has_header__.unwrap_or_default(), + date_format: date_format__.unwrap_or_default(), + datetime_format: datetime_format__.unwrap_or_default(), + timestamp_format: timestamp_format__.unwrap_or_default(), + time_format: time_format__.unwrap_or_default(), + null_value: null_value__.unwrap_or_default(), + }) + } + } + deserializer.deserialize_struct("datafusion.CsvWriterOptions", FIELDS, GeneratedVisitor) + } +} impl serde::Serialize for CubeNode { #[allow(deprecated)] fn serialize(&self, serializer: S) -> std::result::Result @@ -7893,6 +8092,9 @@ impl serde::Serialize for FileTypeWriterOptions { file_type_writer_options::FileType::ParquetOptions(v) => { struct_ser.serialize_field("parquetOptions", v)?; } + file_type_writer_options::FileType::CsvOptions(v) => { + struct_ser.serialize_field("csvOptions", v)?; + } } } struct_ser.end() @@ -7909,12 +8111,15 @@ impl<'de> serde::Deserialize<'de> for FileTypeWriterOptions { "jsonOptions", "parquet_options", "parquetOptions", + "csv_options", + "csvOptions", ]; #[allow(clippy::enum_variant_names)] enum GeneratedField { JsonOptions, ParquetOptions, + CsvOptions, } impl<'de> serde::Deserialize<'de> for GeneratedField { fn deserialize(deserializer: D) -> std::result::Result @@ -7938,6 +8143,7 @@ impl<'de> serde::Deserialize<'de> for FileTypeWriterOptions { match value { "jsonOptions" | "json_options" => Ok(GeneratedField::JsonOptions), "parquetOptions" | "parquet_options" => Ok(GeneratedField::ParquetOptions), + "csvOptions" | "csv_options" => Ok(GeneratedField::CsvOptions), _ => Err(serde::de::Error::unknown_field(value, FIELDS)), } } @@ -7972,6 +8178,13 @@ impl<'de> serde::Deserialize<'de> for FileTypeWriterOptions { return Err(serde::de::Error::duplicate_field("parquetOptions")); } file_type__ = map_.next_value::<::std::option::Option<_>>()?.map(file_type_writer_options::FileType::ParquetOptions) +; + } + GeneratedField::CsvOptions => { + if file_type__.is_some() { + return Err(serde::de::Error::duplicate_field("csvOptions")); + } + file_type__ = map_.next_value::<::std::option::Option<_>>()?.map(file_type_writer_options::FileType::CsvOptions) ; } } diff --git a/datafusion/proto/src/generated/prost.rs b/datafusion/proto/src/generated/prost.rs index 459d5a965cd3..32e892e663ef 100644 --- a/datafusion/proto/src/generated/prost.rs +++ b/datafusion/proto/src/generated/prost.rs @@ -1642,7 +1642,7 @@ pub struct PartitionColumn { #[allow(clippy::derive_partial_eq_without_eq)] #[derive(Clone, PartialEq, ::prost::Message)] pub struct FileTypeWriterOptions { - #[prost(oneof = "file_type_writer_options::FileType", tags = "1, 2")] + #[prost(oneof = "file_type_writer_options::FileType", tags = "1, 2, 3")] pub file_type: ::core::option::Option, } /// Nested message and enum types in `FileTypeWriterOptions`. @@ -1654,6 +1654,8 @@ pub mod file_type_writer_options { JsonOptions(super::JsonWriterOptions), #[prost(message, tag = "2")] ParquetOptions(super::ParquetWriterOptions), + #[prost(message, tag = "3")] + CsvOptions(super::CsvWriterOptions), } } #[allow(clippy::derive_partial_eq_without_eq)] @@ -1670,6 +1672,31 @@ pub struct ParquetWriterOptions { } #[allow(clippy::derive_partial_eq_without_eq)] #[derive(Clone, PartialEq, ::prost::Message)] +pub struct CsvWriterOptions { + /// Optional column delimiter. Defaults to `b','` + #[prost(string, tag = "1")] + pub delimiter: ::prost::alloc::string::String, + /// Whether to write column names as file headers. Defaults to `true` + #[prost(bool, tag = "2")] + pub has_header: bool, + /// Optional date format for date arrays + #[prost(string, tag = "3")] + pub date_format: ::prost::alloc::string::String, + /// Optional datetime format for datetime arrays + #[prost(string, tag = "4")] + pub datetime_format: ::prost::alloc::string::String, + /// Optional timestamp format for timestamp arrays + #[prost(string, tag = "5")] + pub timestamp_format: ::prost::alloc::string::String, + /// Optional time format for time arrays + #[prost(string, tag = "6")] + pub time_format: ::prost::alloc::string::String, + /// Optional value to represent null + #[prost(string, tag = "7")] + pub null_value: ::prost::alloc::string::String, +} +#[allow(clippy::derive_partial_eq_without_eq)] +#[derive(Clone, PartialEq, ::prost::Message)] pub struct WriterProperties { #[prost(uint64, tag = "1")] pub data_page_size_limit: u64, diff --git a/datafusion/proto/src/logical_plan/mod.rs b/datafusion/proto/src/logical_plan/mod.rs index d137a41fa19b..e997bcde426e 100644 --- a/datafusion/proto/src/logical_plan/mod.rs +++ b/datafusion/proto/src/logical_plan/mod.rs @@ -15,6 +15,7 @@ // specific language governing permissions and limitations // under the License. +use arrow::csv::WriterBuilder; use std::collections::HashMap; use std::fmt::Debug; use std::str::FromStr; @@ -64,6 +65,7 @@ use datafusion_expr::{ }; use datafusion::parquet::file::properties::{WriterProperties, WriterVersion}; +use datafusion_common::file_options::csv_writer::CsvWriterOptions; use datafusion_common::file_options::parquet_writer::ParquetWriterOptions; use datafusion_expr::dml::CopyOptions; use prost::bytes::BufMut; @@ -846,6 +848,20 @@ impl AsLogicalPlan for LogicalPlanNode { Some(copy_to_node::CopyOptions::WriterOptions(opt)) => { match &opt.file_type { Some(ft) => match ft { + file_type_writer_options::FileType::CsvOptions( + writer_options, + ) => { + let writer_builder = + csv_writer_options_from_proto(writer_options)?; + CopyOptions::WriterOptions(Box::new( + FileTypeWriterOptions::CSV( + CsvWriterOptions::new( + writer_builder, + CompressionTypeVariant::UNCOMPRESSED, + ), + ), + )) + } file_type_writer_options::FileType::ParquetOptions( writer_options, ) => { @@ -1630,6 +1646,40 @@ impl AsLogicalPlan for LogicalPlanNode { } CopyOptions::WriterOptions(opt) => { match opt.as_ref() { + FileTypeWriterOptions::CSV(csv_opts) => { + let csv_options = &csv_opts.writer_options; + let csv_writer_options = protobuf::CsvWriterOptions { + delimiter: (csv_options.delimiter() as char) + .to_string(), + has_header: csv_options.header(), + date_format: csv_options + .date_format() + .unwrap_or("") + .to_owned(), + datetime_format: csv_options + .datetime_format() + .unwrap_or("") + .to_owned(), + timestamp_format: csv_options + .timestamp_format() + .unwrap_or("") + .to_owned(), + time_format: csv_options + .time_format() + .unwrap_or("") + .to_owned(), + null_value: csv_options.null().to_owned(), + }; + let csv_options = + file_type_writer_options::FileType::CsvOptions( + csv_writer_options, + ); + Some(copy_to_node::CopyOptions::WriterOptions( + protobuf::FileTypeWriterOptions { + file_type: Some(csv_options), + }, + )) + } FileTypeWriterOptions::Parquet(parquet_opts) => { let parquet_writer_options = protobuf::ParquetWriterOptions { @@ -1674,6 +1724,30 @@ impl AsLogicalPlan for LogicalPlanNode { } } +pub(crate) fn csv_writer_options_from_proto( + writer_options: &protobuf::CsvWriterOptions, +) -> Result { + let mut builder = WriterBuilder::new(); + if !writer_options.delimiter.is_empty() { + if let Some(delimiter) = writer_options.delimiter.chars().next() { + if delimiter.is_ascii() { + builder = builder.with_delimiter(delimiter as u8); + } else { + return Err(proto_error("CSV Delimiter is not ASCII")); + } + } else { + return Err(proto_error("Error parsing CSV Delimiter")); + } + } + Ok(builder + .with_header(writer_options.has_header) + .with_date_format(writer_options.date_format.clone()) + .with_datetime_format(writer_options.datetime_format.clone()) + .with_timestamp_format(writer_options.timestamp_format.clone()) + .with_time_format(writer_options.time_format.clone()) + .with_null(writer_options.null_value.clone())) +} + pub(crate) fn writer_properties_to_proto( props: &WriterProperties, ) -> protobuf::WriterProperties { diff --git a/datafusion/proto/src/physical_plan/from_proto.rs b/datafusion/proto/src/physical_plan/from_proto.rs index 824eb60a5715..6f1e811510c6 100644 --- a/datafusion/proto/src/physical_plan/from_proto.rs +++ b/datafusion/proto/src/physical_plan/from_proto.rs @@ -39,6 +39,7 @@ use datafusion::physical_plan::windows::create_window_expr; use datafusion::physical_plan::{ functions, ColumnStatistics, Partitioning, PhysicalExpr, Statistics, WindowExpr, }; +use datafusion_common::file_options::csv_writer::CsvWriterOptions; use datafusion_common::file_options::json_writer::JsonWriterOptions; use datafusion_common::file_options::parquet_writer::ParquetWriterOptions; use datafusion_common::parsers::CompressionTypeVariant; @@ -53,7 +54,7 @@ use crate::logical_plan; use crate::protobuf; use crate::protobuf::physical_expr_node::ExprType; -use crate::logical_plan::writer_properties_from_proto; +use crate::logical_plan::{csv_writer_options_from_proto, writer_properties_from_proto}; use chrono::{TimeZone, Utc}; use object_store::path::Path; use object_store::ObjectMeta; @@ -766,11 +767,18 @@ impl TryFrom<&protobuf::FileTypeWriterOptions> for FileTypeWriterOptions { let file_type = value .file_type .as_ref() - .ok_or_else(|| proto_error("Missing required field in protobuf"))?; + .ok_or_else(|| proto_error("Missing required file_type field in protobuf"))?; match file_type { protobuf::file_type_writer_options::FileType::JsonOptions(opts) => Ok( Self::JSON(JsonWriterOptions::new(opts.compression().into())), ), + protobuf::file_type_writer_options::FileType::CsvOptions(opt) => { + let write_options = csv_writer_options_from_proto(opt)?; + Ok(Self::CSV(CsvWriterOptions::new( + write_options, + CompressionTypeVariant::UNCOMPRESSED, + ))) + } protobuf::file_type_writer_options::FileType::ParquetOptions(opt) => { let props = opt.writer_properties.clone().unwrap_or_default(); let writer_properties = writer_properties_from_proto(&props)?; diff --git a/datafusion/proto/tests/cases/roundtrip_logical_plan.rs b/datafusion/proto/tests/cases/roundtrip_logical_plan.rs index 3eeae01a643e..2d7d85abda96 100644 --- a/datafusion/proto/tests/cases/roundtrip_logical_plan.rs +++ b/datafusion/proto/tests/cases/roundtrip_logical_plan.rs @@ -20,6 +20,7 @@ use std::fmt::{self, Debug, Formatter}; use std::sync::Arc; use arrow::array::{ArrayRef, FixedSizeListArray}; +use arrow::csv::WriterBuilder; use arrow::datatypes::{ DataType, Field, Fields, Int32Type, IntervalDayTimeType, IntervalMonthDayNanoType, IntervalUnit, Schema, SchemaRef, TimeUnit, UnionFields, UnionMode, @@ -35,8 +36,10 @@ use datafusion::parquet::file::properties::{WriterProperties, WriterVersion}; use datafusion::physical_plan::functions::make_scalar_function; use datafusion::prelude::{create_udf, CsvReadOptions, SessionConfig, SessionContext}; use datafusion::test_util::{TestTableFactory, TestTableProvider}; +use datafusion_common::file_options::csv_writer::CsvWriterOptions; use datafusion_common::file_options::parquet_writer::ParquetWriterOptions; use datafusion_common::file_options::StatementOptions; +use datafusion_common::parsers::CompressionTypeVariant; use datafusion_common::{internal_err, not_impl_err, plan_err, FileTypeWriterOptions}; use datafusion_common::{DFField, DFSchema, DFSchemaRef, DataFusionError, ScalarValue}; use datafusion_common::{FileType, Result}; @@ -386,10 +389,69 @@ async fn roundtrip_logical_plan_copy_to_writer_options() -> Result<()> { } _ => panic!(), } - Ok(()) } +#[tokio::test] +async fn roundtrip_logical_plan_copy_to_csv() -> Result<()> { + let ctx = SessionContext::new(); + + let input = create_csv_scan(&ctx).await?; + + let writer_properties = WriterBuilder::new() + .with_delimiter(b'*') + .with_date_format("dd/MM/yyyy".to_string()) + .with_datetime_format("dd/MM/yyyy HH:mm:ss".to_string()) + .with_timestamp_format("HH:mm:ss.SSSSSS".to_string()) + .with_time_format("HH:mm:ss".to_string()) + .with_null("NIL".to_string()); + + let plan = LogicalPlan::Copy(CopyTo { + input: Arc::new(input), + output_url: "test.csv".to_string(), + file_format: FileType::CSV, + single_file_output: true, + copy_options: CopyOptions::WriterOptions(Box::new(FileTypeWriterOptions::CSV( + CsvWriterOptions::new( + writer_properties, + CompressionTypeVariant::UNCOMPRESSED, + ), + ))), + }); + + let bytes = logical_plan_to_bytes(&plan)?; + let logical_round_trip = logical_plan_from_bytes(&bytes, &ctx)?; + assert_eq!(format!("{plan:?}"), format!("{logical_round_trip:?}")); + + match logical_round_trip { + LogicalPlan::Copy(copy_to) => { + assert_eq!("test.csv", copy_to.output_url); + assert_eq!(FileType::CSV, copy_to.file_format); + assert!(copy_to.single_file_output); + match ©_to.copy_options { + CopyOptions::WriterOptions(y) => match y.as_ref() { + FileTypeWriterOptions::CSV(p) => { + let props = &p.writer_options; + assert_eq!(b'*', props.delimiter()); + assert_eq!("dd/MM/yyyy", props.date_format().unwrap()); + assert_eq!( + "dd/MM/yyyy HH:mm:ss", + props.datetime_format().unwrap() + ); + assert_eq!("HH:mm:ss.SSSSSS", props.timestamp_format().unwrap()); + assert_eq!("HH:mm:ss", props.time_format().unwrap()); + assert_eq!("NIL", props.null()); + } + _ => panic!(), + }, + _ => panic!(), + } + } + _ => panic!(), + } + + Ok(()) +} async fn create_csv_scan(ctx: &SessionContext) -> Result { ctx.register_csv("t1", "tests/testdata/test.csv", CsvReadOptions::default()) .await?; From b2cbc7809ee0656099169307a73aadff23ab1030 Mon Sep 17 00:00:00 2001 From: Andrew Lamb Date: Thu, 28 Dec 2023 15:07:32 -0500 Subject: [PATCH 509/572] Add trait based ScalarUDF API (#8578) * Introduce new trait based ScalarUDF API * change name to `Self::new_from_impl` * Improve documentation, add link to advanced_udf.rs in the user guide * typo * Improve docs for aliases * Apply suggestions from code review Co-authored-by: Liang-Chi Hsieh * improve docs --------- Co-authored-by: Liang-Chi Hsieh --- datafusion-examples/README.md | 3 +- datafusion-examples/examples/advanced_udf.rs | 243 ++++++++++++++++++ datafusion-examples/examples/simple_udf.rs | 6 + datafusion/expr/src/expr.rs | 55 ++-- datafusion/expr/src/expr_fn.rs | 85 +++++- datafusion/expr/src/lib.rs | 2 +- datafusion/expr/src/udf.rs | 169 +++++++++++- .../optimizer/src/analyzer/type_coercion.rs | 64 ++--- docs/source/library-user-guide/adding-udfs.md | 9 +- 9 files changed, 562 insertions(+), 74 deletions(-) create mode 100644 datafusion-examples/examples/advanced_udf.rs diff --git a/datafusion-examples/README.md b/datafusion-examples/README.md index 057cdd475273..1296c74ea277 100644 --- a/datafusion-examples/README.md +++ b/datafusion-examples/README.md @@ -59,8 +59,9 @@ cargo run --example csv_sql - [`query-aws-s3.rs`](examples/external_dependency/query-aws-s3.rs): Configure `object_store` and run a query against files stored in AWS S3 - [`query-http-csv.rs`](examples/query-http-csv.rs): Configure `object_store` and run a query against files vi HTTP - [`rewrite_expr.rs`](examples/rewrite_expr.rs): Define and invoke a custom Query Optimizer pass +- [`simple_udf.rs`](examples/simple_udf.rs): Define and invoke a User Defined Scalar Function (UDF) +- [`advanced_udf.rs`](examples/advanced_udf.rs): Define and invoke a more complicated User Defined Scalar Function (UDF) - [`simple_udaf.rs`](examples/simple_udaf.rs): Define and invoke a User Defined Aggregate Function (UDAF) -- [`simple_udf.rs`](examples/simple_udf.rs): Define and invoke a User Defined (scalar) Function (UDF) - [`simple_udfw.rs`](examples/simple_udwf.rs): Define and invoke a User Defined Window Function (UDWF) ## Distributed diff --git a/datafusion-examples/examples/advanced_udf.rs b/datafusion-examples/examples/advanced_udf.rs new file mode 100644 index 000000000000..6ebf88a0b671 --- /dev/null +++ b/datafusion-examples/examples/advanced_udf.rs @@ -0,0 +1,243 @@ +// 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 datafusion::{ + arrow::{ + array::{ArrayRef, Float32Array, Float64Array}, + datatypes::DataType, + record_batch::RecordBatch, + }, + logical_expr::Volatility, +}; +use std::any::Any; + +use arrow::array::{new_null_array, Array, AsArray}; +use arrow::compute; +use arrow::datatypes::Float64Type; +use datafusion::error::Result; +use datafusion::prelude::*; +use datafusion_common::{internal_err, ScalarValue}; +use datafusion_expr::{ColumnarValue, ScalarUDF, ScalarUDFImpl, Signature}; +use std::sync::Arc; + +/// This example shows how to use the full ScalarUDFImpl API to implement a user +/// defined function. As in the `simple_udf.rs` example, this struct implements +/// a function that takes two arguments and returns the first argument raised to +/// the power of the second argument `a^b`. +/// +/// To do so, we must implement the `ScalarUDFImpl` trait. +struct PowUdf { + signature: Signature, + aliases: Vec, +} + +impl PowUdf { + /// Create a new instance of the `PowUdf` struct + fn new() -> Self { + Self { + signature: Signature::exact( + // this function will always take two arguments of type f64 + vec![DataType::Float64, DataType::Float64], + // this function is deterministic and will always return the same + // result for the same input + Volatility::Immutable, + ), + // we will also add an alias of "my_pow" + aliases: vec!["my_pow".to_string()], + } + } +} + +impl ScalarUDFImpl for PowUdf { + /// We implement as_any so that we can downcast the ScalarUDFImpl trait object + fn as_any(&self) -> &dyn Any { + self + } + + /// Return the name of this function + fn name(&self) -> &str { + "pow" + } + + /// Return the "signature" of this function -- namely what types of arguments it will take + fn signature(&self) -> &Signature { + &self.signature + } + + /// What is the type of value that will be returned by this function? In + /// this case it will always be a constant value, but it could also be a + /// function of the input types. + fn return_type(&self, _arg_types: &[DataType]) -> Result { + Ok(DataType::Float64) + } + + /// This is the function that actually calculates the results. + /// + /// This is the same way that functions built into DataFusion are invoked, + /// which permits important special cases when one or both of the arguments + /// are single values (constants). For example `pow(a, 2)` + /// + /// However, it also means the implementation is more complex than when + /// using `create_udf`. + fn invoke(&self, args: &[ColumnarValue]) -> Result { + // DataFusion has arranged for the correct inputs to be passed to this + // function, but we check again to make sure + assert_eq!(args.len(), 2); + let (base, exp) = (&args[0], &args[1]); + assert_eq!(base.data_type(), DataType::Float64); + assert_eq!(exp.data_type(), DataType::Float64); + + match (base, exp) { + // For demonstration purposes we also implement the scalar / scalar + // case here, but it is not typically required for high performance. + // + // For performance it is most important to optimize cases where at + // least one argument is an array. If all arguments are constants, + // the DataFusion expression simplification logic will often invoke + // this path once during planning, and simply use the result during + // execution. + ( + ColumnarValue::Scalar(ScalarValue::Float64(base)), + ColumnarValue::Scalar(ScalarValue::Float64(exp)), + ) => { + // compute the output. Note DataFusion treats `None` as NULL. + let res = match (base, exp) { + (Some(base), Some(exp)) => Some(base.powf(*exp)), + // one or both arguments were NULL + _ => None, + }; + Ok(ColumnarValue::Scalar(ScalarValue::from(res))) + } + // special case if the exponent is a constant + ( + ColumnarValue::Array(base_array), + ColumnarValue::Scalar(ScalarValue::Float64(exp)), + ) => { + let result_array = match exp { + // a ^ null = null + None => new_null_array(base_array.data_type(), base_array.len()), + // a ^ exp + Some(exp) => { + // DataFusion has ensured both arguments are Float64: + let base_array = base_array.as_primitive::(); + // calculate the result for every row. The `unary` + // kernel creates very fast "vectorized" code and + // handles things like null values for us. + let res: Float64Array = + compute::unary(base_array, |base| base.powf(*exp)); + Arc::new(res) + } + }; + Ok(ColumnarValue::Array(result_array)) + } + + // special case if the base is a constant (note this code is quite + // similar to the previous case, so we omit comments) + ( + ColumnarValue::Scalar(ScalarValue::Float64(base)), + ColumnarValue::Array(exp_array), + ) => { + let res = match base { + None => new_null_array(exp_array.data_type(), exp_array.len()), + Some(base) => { + let exp_array = exp_array.as_primitive::(); + let res: Float64Array = + compute::unary(exp_array, |exp| base.powf(exp)); + Arc::new(res) + } + }; + Ok(ColumnarValue::Array(res)) + } + // Both arguments are arrays so we have to perform the calculation for every row + (ColumnarValue::Array(base_array), ColumnarValue::Array(exp_array)) => { + let res: Float64Array = compute::binary( + base_array.as_primitive::(), + exp_array.as_primitive::(), + |base, exp| base.powf(exp), + )?; + Ok(ColumnarValue::Array(Arc::new(res))) + } + // if the types were not float, it is a bug in DataFusion + _ => { + use datafusion_common::DataFusionError; + internal_err!("Invalid argument types to pow function") + } + } + } + + /// We will also add an alias of "my_pow" + fn aliases(&self) -> &[String] { + &self.aliases + } +} + +/// In this example we register `PowUdf` as a user defined function +/// and invoke it via the DataFrame API and SQL +#[tokio::main] +async fn main() -> Result<()> { + let ctx = create_context()?; + + // create the UDF + let pow = ScalarUDF::from(PowUdf::new()); + + // register the UDF with the context so it can be invoked by name and from SQL + ctx.register_udf(pow.clone()); + + // get a DataFrame from the context for scanning the "t" table + let df = ctx.table("t").await?; + + // Call pow(a, 10) using the DataFrame API + let df = df.select(vec![pow.call(vec![col("a"), lit(10i32)])])?; + + // note that the second argument is passed as an i32, not f64. DataFusion + // automatically coerces the types to match the UDF's defined signature. + + // print the results + df.show().await?; + + // You can also invoke both pow(2, 10) and its alias my_pow(a, b) using SQL + let sql_df = ctx.sql("SELECT pow(2, 10), my_pow(a, b) FROM t").await?; + sql_df.show().await?; + + Ok(()) +} + +/// create local execution context with an in-memory table: +/// +/// ```text +/// +-----+-----+ +/// | a | b | +/// +-----+-----+ +/// | 2.1 | 1.0 | +/// | 3.1 | 2.0 | +/// | 4.1 | 3.0 | +/// | 5.1 | 4.0 | +/// +-----+-----+ +/// ``` +fn create_context() -> Result { + // define data. + let a: ArrayRef = Arc::new(Float32Array::from(vec![2.1, 3.1, 4.1, 5.1])); + let b: ArrayRef = Arc::new(Float64Array::from(vec![1.0, 2.0, 3.0, 4.0])); + let batch = RecordBatch::try_from_iter(vec![("a", a), ("b", b)])?; + + // declare a new context. In Spark API, this corresponds to a new SparkSession + let ctx = SessionContext::new(); + + // declare a table in memory. In Spark API, this corresponds to createDataFrame(...). + ctx.register_batch("t", batch)?; + Ok(ctx) +} diff --git a/datafusion-examples/examples/simple_udf.rs b/datafusion-examples/examples/simple_udf.rs index 591991786515..39e1e13ce39a 100644 --- a/datafusion-examples/examples/simple_udf.rs +++ b/datafusion-examples/examples/simple_udf.rs @@ -140,5 +140,11 @@ async fn main() -> Result<()> { // print the results df.show().await?; + // Given that `pow` is registered in the context, we can also use it in SQL: + let sql_df = ctx.sql("SELECT pow(a, b) FROM t").await?; + + // print the results + sql_df.show().await?; + Ok(()) } diff --git a/datafusion/expr/src/expr.rs b/datafusion/expr/src/expr.rs index b46e9ec8f69d..0ec19bcadbf6 100644 --- a/datafusion/expr/src/expr.rs +++ b/datafusion/expr/src/expr.rs @@ -1724,13 +1724,13 @@ mod test { use crate::expr::Cast; use crate::expr_fn::col; use crate::{ - case, lit, BuiltinScalarFunction, ColumnarValue, Expr, ReturnTypeFunction, - ScalarFunctionDefinition, ScalarFunctionImplementation, ScalarUDF, Signature, - Volatility, + case, lit, BuiltinScalarFunction, ColumnarValue, Expr, ScalarFunctionDefinition, + ScalarUDF, ScalarUDFImpl, Signature, Volatility, }; use arrow::datatypes::DataType; use datafusion_common::Column; use datafusion_common::{Result, ScalarValue}; + use std::any::Any; use std::sync::Arc; #[test] @@ -1848,24 +1848,41 @@ mod test { ); // UDF - let return_type: ReturnTypeFunction = - Arc::new(move |_| Ok(Arc::new(DataType::Utf8))); - let fun: ScalarFunctionImplementation = - Arc::new(move |_| Ok(ColumnarValue::Scalar(ScalarValue::new_utf8("a")))); - let udf = Arc::new(ScalarUDF::new( - "TestScalarUDF", - &Signature::uniform(1, vec![DataType::Float32], Volatility::Stable), - &return_type, - &fun, - )); + struct TestScalarUDF { + signature: Signature, + } + impl ScalarUDFImpl for TestScalarUDF { + fn as_any(&self) -> &dyn Any { + self + } + fn name(&self) -> &str { + "TestScalarUDF" + } + + fn signature(&self) -> &Signature { + &self.signature + } + + fn return_type(&self, _arg_types: &[DataType]) -> Result { + Ok(DataType::Utf8) + } + + fn invoke(&self, _args: &[ColumnarValue]) -> Result { + Ok(ColumnarValue::Scalar(ScalarValue::from("a"))) + } + } + let udf = Arc::new(ScalarUDF::from(TestScalarUDF { + signature: Signature::uniform(1, vec![DataType::Float32], Volatility::Stable), + })); assert!(!ScalarFunctionDefinition::UDF(udf).is_volatile().unwrap()); - let udf = Arc::new(ScalarUDF::new( - "TestScalarUDF", - &Signature::uniform(1, vec![DataType::Float32], Volatility::Volatile), - &return_type, - &fun, - )); + let udf = Arc::new(ScalarUDF::from(TestScalarUDF { + signature: Signature::uniform( + 1, + vec![DataType::Float32], + Volatility::Volatile, + ), + })); assert!(ScalarFunctionDefinition::UDF(udf).is_volatile().unwrap()); // Unresolved function diff --git a/datafusion/expr/src/expr_fn.rs b/datafusion/expr/src/expr_fn.rs index cedf1d845137..eed41d97ccba 100644 --- a/datafusion/expr/src/expr_fn.rs +++ b/datafusion/expr/src/expr_fn.rs @@ -22,15 +22,16 @@ use crate::expr::{ Placeholder, ScalarFunction, TryCast, }; use crate::function::PartitionEvaluatorFactory; -use crate::WindowUDF; use crate::{ aggregate_function, built_in_function, conditional_expressions::CaseBuilder, logical_plan::Subquery, AccumulatorFactoryFunction, AggregateUDF, BuiltinScalarFunction, Expr, LogicalPlan, Operator, ReturnTypeFunction, ScalarFunctionImplementation, ScalarUDF, Signature, StateTypeFunction, Volatility, }; +use crate::{ColumnarValue, ScalarUDFImpl, WindowUDF}; use arrow::datatypes::DataType; use datafusion_common::{Column, Result}; +use std::any::Any; use std::ops::Not; use std::sync::Arc; @@ -944,11 +945,18 @@ pub fn when(when: Expr, then: Expr) -> CaseBuilder { CaseBuilder::new(None, vec![when], vec![then], None) } -/// Creates a new UDF with a specific signature and specific return type. -/// This is a helper function to create a new UDF. -/// The function `create_udf` returns a subset of all possible `ScalarFunction`: -/// * the UDF has a fixed return type -/// * the UDF has a fixed signature (e.g. [f64, f64]) +/// Convenience method to create a new user defined scalar function (UDF) with a +/// specific signature and specific return type. +/// +/// Note this function does not expose all available features of [`ScalarUDF`], +/// such as +/// +/// * computing return types based on input types +/// * multiple [`Signature`]s +/// * aliases +/// +/// See [`ScalarUDF`] for details and examples on how to use the full +/// functionality. pub fn create_udf( name: &str, input_types: Vec, @@ -956,13 +964,66 @@ pub fn create_udf( volatility: Volatility, fun: ScalarFunctionImplementation, ) -> ScalarUDF { - let return_type: ReturnTypeFunction = Arc::new(move |_| Ok(return_type.clone())); - ScalarUDF::new( + let return_type = Arc::try_unwrap(return_type).unwrap_or_else(|t| t.as_ref().clone()); + ScalarUDF::from(SimpleScalarUDF::new( name, - &Signature::exact(input_types, volatility), - &return_type, - &fun, - ) + input_types, + return_type, + volatility, + fun, + )) +} + +/// Implements [`ScalarUDFImpl`] for functions that have a single signature and +/// return type. +pub struct SimpleScalarUDF { + name: String, + signature: Signature, + return_type: DataType, + fun: ScalarFunctionImplementation, +} + +impl SimpleScalarUDF { + /// Create a new `SimpleScalarUDF` from a name, input types, return type and + /// implementation. Implementing [`ScalarUDFImpl`] allows more flexibility + pub fn new( + name: impl Into, + input_types: Vec, + return_type: DataType, + volatility: Volatility, + fun: ScalarFunctionImplementation, + ) -> Self { + let name = name.into(); + let signature = Signature::exact(input_types, volatility); + Self { + name, + signature, + return_type, + fun, + } + } +} + +impl ScalarUDFImpl for SimpleScalarUDF { + fn as_any(&self) -> &dyn Any { + self + } + + fn name(&self) -> &str { + &self.name + } + + fn signature(&self) -> &Signature { + &self.signature + } + + fn return_type(&self, _arg_types: &[DataType]) -> Result { + Ok(self.return_type.clone()) + } + + fn invoke(&self, args: &[ColumnarValue]) -> Result { + (self.fun)(args) + } } /// Creates a new UDAF with a specific signature, state type and return type. diff --git a/datafusion/expr/src/lib.rs b/datafusion/expr/src/lib.rs index 48532e13dcd7..bf8e9e2954f4 100644 --- a/datafusion/expr/src/lib.rs +++ b/datafusion/expr/src/lib.rs @@ -80,7 +80,7 @@ pub use signature::{ }; pub use table_source::{TableProviderFilterPushDown, TableSource, TableType}; pub use udaf::AggregateUDF; -pub use udf::ScalarUDF; +pub use udf::{ScalarUDF, ScalarUDFImpl}; pub use udwf::WindowUDF; pub use window_frame::{WindowFrame, WindowFrameBound, WindowFrameUnits}; pub use window_function::{BuiltInWindowFunction, WindowFunction}; diff --git a/datafusion/expr/src/udf.rs b/datafusion/expr/src/udf.rs index 3a18ca2d25e8..2ec80a4a9ea1 100644 --- a/datafusion/expr/src/udf.rs +++ b/datafusion/expr/src/udf.rs @@ -17,9 +17,12 @@ //! [`ScalarUDF`]: Scalar User Defined Functions -use crate::{Expr, ReturnTypeFunction, ScalarFunctionImplementation, Signature}; +use crate::{ + ColumnarValue, Expr, ReturnTypeFunction, ScalarFunctionImplementation, Signature, +}; use arrow::datatypes::DataType; use datafusion_common::Result; +use std::any::Any; use std::fmt; use std::fmt::Debug; use std::fmt::Formatter; @@ -27,11 +30,19 @@ use std::sync::Arc; /// Logical representation of a Scalar User Defined Function. /// -/// A scalar function produces a single row output for each row of input. +/// A scalar function produces a single row output for each row of input. This +/// struct contains the information DataFusion needs to plan and invoke +/// functions you supply such name, type signature, return type, and actual +/// implementation. /// -/// This struct contains the information DataFusion needs to plan and invoke -/// functions such name, type signature, return type, and actual implementation. /// +/// 1. For simple (less performant) use cases, use [`create_udf`] and [`simple_udf.rs`]. +/// +/// 2. For advanced use cases, use [`ScalarUDFImpl`] and [`advanced_udf.rs`]. +/// +/// [`create_udf`]: crate::expr_fn::create_udf +/// [`simple_udf.rs`]: https://github.com/apache/arrow-datafusion/blob/main/datafusion-examples/examples/simple_udf.rs +/// [`advanced_udf.rs`]: https://github.com/apache/arrow-datafusion/blob/main/datafusion-examples/examples/advanced_udf.rs #[derive(Clone)] pub struct ScalarUDF { /// The name of the function @@ -79,7 +90,11 @@ impl std::hash::Hash for ScalarUDF { } impl ScalarUDF { - /// Create a new ScalarUDF + /// Create a new ScalarUDF from low level details. + /// + /// See [`ScalarUDFImpl`] for a more convenient way to create a + /// `ScalarUDF` using trait objects + #[deprecated(since = "34.0.0", note = "please implement ScalarUDFImpl instead")] pub fn new( name: &str, signature: &Signature, @@ -95,6 +110,34 @@ impl ScalarUDF { } } + /// Create a new `ScalarUDF` from a `[ScalarUDFImpl]` trait object + /// + /// Note this is the same as using the `From` impl (`ScalarUDF::from`) + pub fn new_from_impl(fun: F) -> ScalarUDF + where + F: ScalarUDFImpl + Send + Sync + 'static, + { + // TODO change the internal implementation to use the trait object + let arc_fun = Arc::new(fun); + let captured_self = arc_fun.clone(); + let return_type: ReturnTypeFunction = Arc::new(move |arg_types| { + let return_type = captured_self.return_type(arg_types)?; + Ok(Arc::new(return_type)) + }); + + let captured_self = arc_fun.clone(); + let func: ScalarFunctionImplementation = + Arc::new(move |args| captured_self.invoke(args)); + + Self { + name: arc_fun.name().to_string(), + signature: arc_fun.signature().clone(), + return_type: return_type.clone(), + fun: func, + aliases: arc_fun.aliases().to_vec(), + } + } + /// Adds additional names that can be used to invoke this function, in addition to `name` pub fn with_aliases( mut self, @@ -105,7 +148,9 @@ impl ScalarUDF { self } - /// creates a logical expression with a call of the UDF + /// Returns a [`Expr`] logical expression to call this UDF with specified + /// arguments. + /// /// This utility allows using the UDF without requiring access to the registry. pub fn call(&self, args: Vec) -> Expr { Expr::ScalarFunction(crate::expr::ScalarFunction::new_udf( @@ -124,22 +169,126 @@ impl ScalarUDF { &self.aliases } - /// Returns this function's signature (what input types are accepted) + /// Returns this function's [`Signature`] (what input types are accepted) pub fn signature(&self) -> &Signature { &self.signature } - /// Return the type of the function given its input types + /// The datatype this function returns given the input argument input types pub fn return_type(&self, args: &[DataType]) -> Result { // Old API returns an Arc of the datatype for some reason let res = (self.return_type)(args)?; Ok(res.as_ref().clone()) } - /// Return the actual implementation + /// Return an [`Arc`] to the function implementation pub fn fun(&self) -> ScalarFunctionImplementation { self.fun.clone() } +} - // TODO maybe add an invoke() method that runs the actual function? +impl From for ScalarUDF +where + F: ScalarUDFImpl + Send + Sync + 'static, +{ + fn from(fun: F) -> Self { + Self::new_from_impl(fun) + } +} + +/// Trait for implementing [`ScalarUDF`]. +/// +/// This trait exposes the full API for implementing user defined functions and +/// can be used to implement any function. +/// +/// See [`advanced_udf.rs`] for a full example with complete implementation and +/// [`ScalarUDF`] for other available options. +/// +/// +/// [`advanced_udf.rs`]: https://github.com/apache/arrow-datafusion/blob/main/datafusion-examples/examples/advanced_udf.rs +/// # Basic Example +/// ``` +/// # use std::any::Any; +/// # use arrow::datatypes::DataType; +/// # use datafusion_common::{DataFusionError, plan_err, Result}; +/// # use datafusion_expr::{col, ColumnarValue, Signature, Volatility}; +/// # use datafusion_expr::{ScalarUDFImpl, ScalarUDF}; +/// struct AddOne { +/// signature: Signature +/// }; +/// +/// impl AddOne { +/// fn new() -> Self { +/// Self { +/// signature: Signature::uniform(1, vec![DataType::Int32], Volatility::Immutable) +/// } +/// } +/// } +/// +/// /// Implement the ScalarUDFImpl trait for AddOne +/// impl ScalarUDFImpl for AddOne { +/// fn as_any(&self) -> &dyn Any { self } +/// fn name(&self) -> &str { "add_one" } +/// fn signature(&self) -> &Signature { &self.signature } +/// fn return_type(&self, args: &[DataType]) -> Result { +/// if !matches!(args.get(0), Some(&DataType::Int32)) { +/// return plan_err!("add_one only accepts Int32 arguments"); +/// } +/// Ok(DataType::Int32) +/// } +/// // The actual implementation would add one to the argument +/// fn invoke(&self, args: &[ColumnarValue]) -> Result { unimplemented!() } +/// } +/// +/// // Create a new ScalarUDF from the implementation +/// let add_one = ScalarUDF::from(AddOne::new()); +/// +/// // Call the function `add_one(col)` +/// let expr = add_one.call(vec![col("a")]); +/// ``` +pub trait ScalarUDFImpl { + /// Returns this object as an [`Any`] trait object + fn as_any(&self) -> &dyn Any; + + /// Returns this function's name + fn name(&self) -> &str; + + /// Returns the function's [`Signature`] for information about what input + /// types are accepted and the function's Volatility. + fn signature(&self) -> &Signature; + + /// What [`DataType`] will be returned by this function, given the types of + /// the arguments + fn return_type(&self, arg_types: &[DataType]) -> Result; + + /// Invoke the function on `args`, returning the appropriate result + /// + /// The function will be invoked passed with the slice of [`ColumnarValue`] + /// (either scalar or array). + /// + /// # Zero Argument Functions + /// If the function has zero parameters (e.g. `now()`) it will be passed a + /// single element slice which is a a null array to indicate the batch's row + /// count (so the function can know the resulting array size). + /// + /// # Performance + /// + /// For the best performance, the implementations of `invoke` should handle + /// the common case when one or more of their arguments are constant values + /// (aka [`ColumnarValue::Scalar`]). Calling [`ColumnarValue::into_array`] + /// and treating all arguments as arrays will work, but will be slower. + fn invoke(&self, args: &[ColumnarValue]) -> Result; + + /// Returns any aliases (alternate names) for this function. + /// + /// Aliases can be used to invoke the same function using different names. + /// For example in some databases `now()` and `current_timestamp()` are + /// aliases for the same function. This behavior can be obtained by + /// returning `current_timestamp` as an alias for the `now` function. + /// + /// Note: `aliases` should only include names other than [`Self::name`]. + /// Defaults to `[]` (no aliases) + fn aliases(&self) -> &[String] { + &[] + } } diff --git a/datafusion/optimizer/src/analyzer/type_coercion.rs b/datafusion/optimizer/src/analyzer/type_coercion.rs index c5e1180b9f97..b6298f5b552f 100644 --- a/datafusion/optimizer/src/analyzer/type_coercion.rs +++ b/datafusion/optimizer/src/analyzer/type_coercion.rs @@ -738,7 +738,8 @@ fn coerce_case_expression(case: Case, schema: &DFSchemaRef) -> Result { #[cfg(test)] mod test { - use std::sync::Arc; + use std::any::Any; + use std::sync::{Arc, OnceLock}; use arrow::array::{FixedSizeListArray, Int32Array}; use arrow::datatypes::{DataType, TimeUnit}; @@ -750,13 +751,13 @@ mod test { use datafusion_expr::{ cast, col, concat, concat_ws, create_udaf, is_true, AccumulatorFactoryFunction, AggregateFunction, AggregateUDF, BinaryExpr, BuiltinScalarFunction, Case, - ColumnarValue, ExprSchemable, Filter, Operator, StateTypeFunction, Subquery, + ColumnarValue, ExprSchemable, Filter, Operator, ScalarUDFImpl, StateTypeFunction, + Subquery, }; use datafusion_expr::{ lit, logical_plan::{EmptyRelation, Projection}, - Expr, LogicalPlan, ReturnTypeFunction, ScalarFunctionImplementation, ScalarUDF, - Signature, Volatility, + Expr, LogicalPlan, ReturnTypeFunction, ScalarUDF, Signature, Volatility, }; use datafusion_physical_expr::expressions::AvgAccumulator; @@ -808,22 +809,36 @@ mod test { assert_analyzed_plan_eq(Arc::new(TypeCoercion::new()), &plan, expected) } + static TEST_SIGNATURE: OnceLock = OnceLock::new(); + + struct TestScalarUDF {} + impl ScalarUDFImpl for TestScalarUDF { + fn as_any(&self) -> &dyn Any { + self + } + + fn name(&self) -> &str { + "TestScalarUDF" + } + fn signature(&self) -> &Signature { + TEST_SIGNATURE.get_or_init(|| { + Signature::uniform(1, vec![DataType::Float32], Volatility::Stable) + }) + } + fn return_type(&self, _args: &[DataType]) -> Result { + Ok(DataType::Utf8) + } + + fn invoke(&self, _args: &[ColumnarValue]) -> Result { + Ok(ColumnarValue::Scalar(ScalarValue::from("a"))) + } + } + #[test] fn scalar_udf() -> Result<()> { let empty = empty(); - let return_type: ReturnTypeFunction = - Arc::new(move |_| Ok(Arc::new(DataType::Utf8))); - let fun: ScalarFunctionImplementation = - Arc::new(move |_| Ok(ColumnarValue::Scalar(ScalarValue::new_utf8("a")))); - let udf = Expr::ScalarFunction(expr::ScalarFunction::new_udf( - Arc::new(ScalarUDF::new( - "TestScalarUDF", - &Signature::uniform(1, vec![DataType::Float32], Volatility::Stable), - &return_type, - &fun, - )), - vec![lit(123_i32)], - )); + + let udf = ScalarUDF::from(TestScalarUDF {}).call(vec![lit(123_i32)]); let plan = LogicalPlan::Projection(Projection::try_new(vec![udf], empty)?); let expected = "Projection: TestScalarUDF(CAST(Int32(123) AS Float32))\n EmptyRelation"; @@ -833,24 +848,13 @@ mod test { #[test] fn scalar_udf_invalid_input() -> Result<()> { let empty = empty(); - let return_type: ReturnTypeFunction = - Arc::new(move |_| Ok(Arc::new(DataType::Utf8))); - let fun: ScalarFunctionImplementation = Arc::new(move |_| unimplemented!()); - let udf = Expr::ScalarFunction(expr::ScalarFunction::new_udf( - Arc::new(ScalarUDF::new( - "TestScalarUDF", - &Signature::uniform(1, vec![DataType::Int32], Volatility::Stable), - &return_type, - &fun, - )), - vec![lit("Apple")], - )); + let udf = ScalarUDF::from(TestScalarUDF {}).call(vec![lit("Apple")]); let plan = LogicalPlan::Projection(Projection::try_new(vec![udf], empty)?); let err = assert_analyzed_plan_eq(Arc::new(TypeCoercion::new()), &plan, "") .err() .unwrap(); assert_eq!( - "type_coercion\ncaused by\nError during planning: Coercion from [Utf8] to the signature Uniform(1, [Int32]) failed.", + "type_coercion\ncaused by\nError during planning: Coercion from [Utf8] to the signature Uniform(1, [Float32]) failed.", err.strip_backtrace() ); Ok(()) diff --git a/docs/source/library-user-guide/adding-udfs.md b/docs/source/library-user-guide/adding-udfs.md index 11cf52eb3fcf..c51e4de3236c 100644 --- a/docs/source/library-user-guide/adding-udfs.md +++ b/docs/source/library-user-guide/adding-udfs.md @@ -76,7 +76,9 @@ The challenge however is that DataFusion doesn't know about this function. We ne ### Registering a Scalar UDF -To register a Scalar UDF, you need to wrap the function implementation in a `ScalarUDF` struct and then register it with the `SessionContext`. DataFusion provides the `create_udf` and `make_scalar_function` helper functions to make this easier. +To register a Scalar UDF, you need to wrap the function implementation in a [`ScalarUDF`] struct and then register it with the `SessionContext`. +DataFusion provides the [`create_udf`] and helper functions to make this easier. +There is a lower level API with more functionality but is more complex, that is documented in [`advanced_udf.rs`]. ```rust use datafusion::logical_expr::{Volatility, create_udf}; @@ -93,6 +95,11 @@ let udf = create_udf( ); ``` +[`scalarudf`]: https://docs.rs/datafusion/latest/datafusion/logical_expr/struct.ScalarUDF.html +[`create_udf`]: https://docs.rs/datafusion/latest/datafusion/logical_expr/fn.create_udf.html +[`make_scalar_function`]: https://docs.rs/datafusion/latest/datafusion/physical_expr/functions/fn.make_scalar_function.html +[`advanced_udf.rs`]: https://github.com/apache/arrow-datafusion/blob/main/datafusion-examples/examples/advanced_udf.rs + A few things to note: - The first argument is the name of the function. This is the name that will be used in SQL queries. From 06ed3dd1ac01b1bd6a70b93b56cb72cb40777690 Mon Sep 17 00:00:00 2001 From: Mustafa Akur <106137913+mustafasrepo@users.noreply.github.com> Date: Thu, 28 Dec 2023 23:34:40 +0300 Subject: [PATCH 510/572] Handle ordering of first last aggregation inside aggregator (#8662) * Initial commit * Update tests in distinct_on * Update group by joins slt * Remove unused code * Minor changes * Minor changes * Simplifications * Update comments * Review * Fix clippy --------- Co-authored-by: Mehmet Ozan Kabak --- datafusion-cli/src/functions.rs | 2 +- datafusion/common/src/error.rs | 1 - .../physical_optimizer/projection_pushdown.rs | 4 + .../src/simplify_expressions/guarantees.rs | 4 + .../physical-expr/src/aggregate/first_last.rs | 131 +++-- datafusion/physical-expr/src/aggregate/mod.rs | 30 +- .../physical-expr/src/aggregate/utils.rs | 18 +- .../physical-expr/src/array_expressions.rs | 2 +- .../physical-plan/src/aggregates/mod.rs | 461 ++++++++---------- .../src/engines/datafusion_engine/mod.rs | 1 - .../sqllogictest/test_files/distinct_on.slt | 9 +- .../sqllogictest/test_files/groupby.slt | 82 ++-- datafusion/sqllogictest/test_files/joins.slt | 4 +- 13 files changed, 373 insertions(+), 376 deletions(-) diff --git a/datafusion-cli/src/functions.rs b/datafusion-cli/src/functions.rs index f8d9ed238be4..5390fa9f2271 100644 --- a/datafusion-cli/src/functions.rs +++ b/datafusion-cli/src/functions.rs @@ -297,7 +297,7 @@ pub struct ParquetMetadataFunc {} impl TableFunctionImpl for ParquetMetadataFunc { fn call(&self, exprs: &[Expr]) -> Result> { - let filename = match exprs.get(0) { + let filename = match exprs.first() { Some(Expr::Literal(ScalarValue::Utf8(Some(s)))) => s, // single quote: parquet_metadata('x.parquet') Some(Expr::Column(Column { name, .. })) => name, // double quote: parquet_metadata("x.parquet") _ => { diff --git a/datafusion/common/src/error.rs b/datafusion/common/src/error.rs index 515acc6d1c47..e58faaa15096 100644 --- a/datafusion/common/src/error.rs +++ b/datafusion/common/src/error.rs @@ -558,7 +558,6 @@ macro_rules! arrow_err { // To avoid compiler error when using macro in the same crate: // macros from the current crate cannot be referred to by absolute paths -pub use exec_err as _exec_err; pub use internal_datafusion_err as _internal_datafusion_err; pub use internal_err as _internal_err; pub use not_impl_err as _not_impl_err; diff --git a/datafusion/core/src/physical_optimizer/projection_pushdown.rs b/datafusion/core/src/physical_optimizer/projection_pushdown.rs index 7e1312dad23e..d237a3e8607e 100644 --- a/datafusion/core/src/physical_optimizer/projection_pushdown.rs +++ b/datafusion/core/src/physical_optimizer/projection_pushdown.rs @@ -990,6 +990,10 @@ fn update_join_on( proj_right_exprs: &[(Column, String)], hash_join_on: &[(Column, Column)], ) -> Option> { + // TODO: Clippy wants the "map" call removed, but doing so generates + // a compilation error. Remove the clippy directive once this + // issue is fixed. + #[allow(clippy::map_identity)] let (left_idx, right_idx): (Vec<_>, Vec<_>) = hash_join_on .iter() .map(|(left, right)| (left, right)) diff --git a/datafusion/optimizer/src/simplify_expressions/guarantees.rs b/datafusion/optimizer/src/simplify_expressions/guarantees.rs index 860dc326b9b0..aa7bb4f78a93 100644 --- a/datafusion/optimizer/src/simplify_expressions/guarantees.rs +++ b/datafusion/optimizer/src/simplify_expressions/guarantees.rs @@ -47,6 +47,10 @@ impl<'a> GuaranteeRewriter<'a> { guarantees: impl IntoIterator, ) -> Self { Self { + // TODO: Clippy wants the "map" call removed, but doing so generates + // a compilation error. Remove the clippy directive once this + // issue is fixed. + #[allow(clippy::map_identity)] guarantees: guarantees.into_iter().map(|(k, v)| (k, v)).collect(), } } diff --git a/datafusion/physical-expr/src/aggregate/first_last.rs b/datafusion/physical-expr/src/aggregate/first_last.rs index c009881d8918..c7032e601cf8 100644 --- a/datafusion/physical-expr/src/aggregate/first_last.rs +++ b/datafusion/physical-expr/src/aggregate/first_last.rs @@ -20,7 +20,7 @@ use std::any::Any; use std::sync::Arc; -use crate::aggregate::utils::{down_cast_any_ref, ordering_fields}; +use crate::aggregate::utils::{down_cast_any_ref, get_sort_options, ordering_fields}; use crate::expressions::format_state_name; use crate::{ reverse_order_bys, AggregateExpr, LexOrdering, PhysicalExpr, PhysicalSortExpr, @@ -29,9 +29,10 @@ use crate::{ use arrow::array::{Array, ArrayRef, AsArray, BooleanArray}; use arrow::compute::{self, lexsort_to_indices, SortColumn}; use arrow::datatypes::{DataType, Field}; -use arrow_schema::SortOptions; use datafusion_common::utils::{compare_rows, get_arrayref_at_indices, get_row_at_idx}; -use datafusion_common::{arrow_datafusion_err, DataFusionError, Result, ScalarValue}; +use datafusion_common::{ + arrow_datafusion_err, internal_err, DataFusionError, Result, ScalarValue, +}; use datafusion_expr::Accumulator; /// FIRST_VALUE aggregate expression @@ -211,10 +212,45 @@ impl FirstValueAccumulator { } // Updates state with the values in the given row. - fn update_with_new_row(&mut self, row: &[ScalarValue]) { - self.first = row[0].clone(); - self.orderings = row[1..].to_vec(); - self.is_set = true; + fn update_with_new_row(&mut self, row: &[ScalarValue]) -> Result<()> { + let [value, orderings @ ..] = row else { + return internal_err!("Empty row in FIRST_VALUE"); + }; + // Update when there is no entry in the state, or we have an "earlier" + // entry according to sort requirements. + if !self.is_set + || compare_rows( + &self.orderings, + orderings, + &get_sort_options(&self.ordering_req), + )? + .is_gt() + { + self.first = value.clone(); + self.orderings = orderings.to_vec(); + self.is_set = true; + } + Ok(()) + } + + fn get_first_idx(&self, values: &[ArrayRef]) -> Result> { + let [value, ordering_values @ ..] = values else { + return internal_err!("Empty row in FIRST_VALUE"); + }; + if self.ordering_req.is_empty() { + // Get first entry according to receive order (0th index) + return Ok((!value.is_empty()).then_some(0)); + } + let sort_columns = ordering_values + .iter() + .zip(self.ordering_req.iter()) + .map(|(values, req)| SortColumn { + values: values.clone(), + options: Some(req.options), + }) + .collect::>(); + let indices = lexsort_to_indices(&sort_columns, Some(1))?; + Ok((!indices.is_empty()).then_some(indices.value(0) as _)) } } @@ -227,11 +263,9 @@ impl Accumulator for FirstValueAccumulator { } fn update_batch(&mut self, values: &[ArrayRef]) -> Result<()> { - // If we have seen first value, we shouldn't update it - if !values[0].is_empty() && !self.is_set { - let row = get_row_at_idx(values, 0)?; - // Update with first value in the array. - self.update_with_new_row(&row); + if let Some(first_idx) = self.get_first_idx(values)? { + let row = get_row_at_idx(values, first_idx)?; + self.update_with_new_row(&row)?; } Ok(()) } @@ -265,7 +299,7 @@ impl Accumulator for FirstValueAccumulator { // Update with first value in the state. Note that we should exclude the // is_set flag from the state. Otherwise, we will end up with a state // containing two is_set flags. - self.update_with_new_row(&first_row[0..is_set_idx]); + self.update_with_new_row(&first_row[0..is_set_idx])?; } } Ok(()) @@ -459,10 +493,50 @@ impl LastValueAccumulator { } // Updates state with the values in the given row. - fn update_with_new_row(&mut self, row: &[ScalarValue]) { - self.last = row[0].clone(); - self.orderings = row[1..].to_vec(); - self.is_set = true; + fn update_with_new_row(&mut self, row: &[ScalarValue]) -> Result<()> { + let [value, orderings @ ..] = row else { + return internal_err!("Empty row in LAST_VALUE"); + }; + // Update when there is no entry in the state, or we have a "later" + // entry (either according to sort requirements or the order of execution). + if !self.is_set + || self.orderings.is_empty() + || compare_rows( + &self.orderings, + orderings, + &get_sort_options(&self.ordering_req), + )? + .is_lt() + { + self.last = value.clone(); + self.orderings = orderings.to_vec(); + self.is_set = true; + } + Ok(()) + } + + fn get_last_idx(&self, values: &[ArrayRef]) -> Result> { + let [value, ordering_values @ ..] = values else { + return internal_err!("Empty row in LAST_VALUE"); + }; + if self.ordering_req.is_empty() { + // Get last entry according to the order of data: + return Ok((!value.is_empty()).then_some(value.len() - 1)); + } + let sort_columns = ordering_values + .iter() + .zip(self.ordering_req.iter()) + .map(|(values, req)| { + // Take the reverse ordering requirement. This enables us to + // use "fetch = 1" to get the last value. + SortColumn { + values: values.clone(), + options: Some(!req.options), + } + }) + .collect::>(); + let indices = lexsort_to_indices(&sort_columns, Some(1))?; + Ok((!indices.is_empty()).then_some(indices.value(0) as _)) } } @@ -475,10 +549,9 @@ impl Accumulator for LastValueAccumulator { } fn update_batch(&mut self, values: &[ArrayRef]) -> Result<()> { - if !values[0].is_empty() { - let row = get_row_at_idx(values, values[0].len() - 1)?; - // Update with last value in the array. - self.update_with_new_row(&row); + if let Some(last_idx) = self.get_last_idx(values)? { + let row = get_row_at_idx(values, last_idx)?; + self.update_with_new_row(&row)?; } Ok(()) } @@ -515,7 +588,7 @@ impl Accumulator for LastValueAccumulator { // Update with last value in the state. Note that we should exclude the // is_set flag from the state. Otherwise, we will end up with a state // containing two is_set flags. - self.update_with_new_row(&last_row[0..is_set_idx]); + self.update_with_new_row(&last_row[0..is_set_idx])?; } } Ok(()) @@ -559,26 +632,18 @@ fn convert_to_sort_cols( .collect::>() } -/// Selects the sort option attribute from all the given `PhysicalSortExpr`s. -fn get_sort_options(ordering_req: &[PhysicalSortExpr]) -> Vec { - ordering_req - .iter() - .map(|item| item.options) - .collect::>() -} - #[cfg(test)] mod tests { + use std::sync::Arc; + use crate::aggregate::first_last::{FirstValueAccumulator, LastValueAccumulator}; + use arrow::compute::concat; use arrow_array::{ArrayRef, Int64Array}; use arrow_schema::DataType; use datafusion_common::{Result, ScalarValue}; use datafusion_expr::Accumulator; - use arrow::compute::concat; - use std::sync::Arc; - #[test] fn test_first_last_value_value() -> Result<()> { let mut first_accumulator = diff --git a/datafusion/physical-expr/src/aggregate/mod.rs b/datafusion/physical-expr/src/aggregate/mod.rs index 329bb1e6415e..5bd1fca385b1 100644 --- a/datafusion/physical-expr/src/aggregate/mod.rs +++ b/datafusion/physical-expr/src/aggregate/mod.rs @@ -15,16 +15,20 @@ // specific language governing permissions and limitations // under the License. -use crate::expressions::{FirstValue, LastValue, OrderSensitiveArrayAgg}; -use crate::{PhysicalExpr, PhysicalSortExpr}; -use arrow::datatypes::Field; -use datafusion_common::{not_impl_err, DataFusionError, Result}; -use datafusion_expr::Accumulator; use std::any::Any; use std::fmt::Debug; use std::sync::Arc; use self::groups_accumulator::GroupsAccumulator; +use crate::expressions::OrderSensitiveArrayAgg; +use crate::{PhysicalExpr, PhysicalSortExpr}; + +use arrow::datatypes::Field; +use datafusion_common::{not_impl_err, DataFusionError, Result}; +use datafusion_expr::Accumulator; + +mod hyperloglog; +mod tdigest; pub(crate) mod approx_distinct; pub(crate) mod approx_median; @@ -46,19 +50,18 @@ pub(crate) mod median; pub(crate) mod string_agg; #[macro_use] pub(crate) mod min_max; -pub mod build_in; pub(crate) mod groups_accumulator; -mod hyperloglog; -pub mod moving_min_max; pub(crate) mod regr; pub(crate) mod stats; pub(crate) mod stddev; pub(crate) mod sum; pub(crate) mod sum_distinct; -mod tdigest; -pub mod utils; pub(crate) mod variance; +pub mod build_in; +pub mod moving_min_max; +pub mod utils; + /// An aggregate expression that: /// * knows its resulting field /// * knows how to create its accumulator @@ -134,10 +137,7 @@ pub trait AggregateExpr: Send + Sync + Debug + PartialEq { /// Checks whether the given aggregate expression is order-sensitive. /// For instance, a `SUM` aggregation doesn't depend on the order of its inputs. -/// However, a `FirstValue` depends on the input ordering (if the order changes, -/// the first value in the list would change). +/// However, an `ARRAY_AGG` with `ORDER BY` depends on the input ordering. pub fn is_order_sensitive(aggr_expr: &Arc) -> bool { - aggr_expr.as_any().is::() - || aggr_expr.as_any().is::() - || aggr_expr.as_any().is::() + aggr_expr.as_any().is::() } diff --git a/datafusion/physical-expr/src/aggregate/utils.rs b/datafusion/physical-expr/src/aggregate/utils.rs index e5421ef5ab7e..9777158da133 100644 --- a/datafusion/physical-expr/src/aggregate/utils.rs +++ b/datafusion/physical-expr/src/aggregate/utils.rs @@ -17,20 +17,21 @@ //! Utilities used in aggregates +use std::any::Any; +use std::sync::Arc; + use crate::{AggregateExpr, PhysicalSortExpr}; -use arrow::array::ArrayRef; + +use arrow::array::{ArrayRef, ArrowNativeTypeOp}; use arrow_array::cast::AsArray; use arrow_array::types::{ Decimal128Type, DecimalType, TimestampMicrosecondType, TimestampMillisecondType, TimestampNanosecondType, TimestampSecondType, }; -use arrow_array::ArrowNativeTypeOp; use arrow_buffer::ArrowNativeType; -use arrow_schema::{DataType, Field}; +use arrow_schema::{DataType, Field, SortOptions}; use datafusion_common::{exec_err, DataFusionError, Result}; use datafusion_expr::Accumulator; -use std::any::Any; -use std::sync::Arc; /// Convert scalar values from an accumulator into arrays. pub fn get_accum_scalar_values_as_arrays( @@ -40,7 +41,7 @@ pub fn get_accum_scalar_values_as_arrays( .state()? .iter() .map(|s| s.to_array_of_size(1)) - .collect::>>() + .collect() } /// Computes averages for `Decimal128`/`Decimal256` values, checking for overflow @@ -205,3 +206,8 @@ pub(crate) fn ordering_fields( }) .collect() } + +/// Selects the sort option attribute from all the given `PhysicalSortExpr`s. +pub fn get_sort_options(ordering_req: &[PhysicalSortExpr]) -> Vec { + ordering_req.iter().map(|item| item.options).collect() +} diff --git a/datafusion/physical-expr/src/array_expressions.rs b/datafusion/physical-expr/src/array_expressions.rs index 274d1db4eb0d..7a986810bad2 100644 --- a/datafusion/physical-expr/src/array_expressions.rs +++ b/datafusion/physical-expr/src/array_expressions.rs @@ -2453,7 +2453,7 @@ pub fn general_array_distinct( let last_offset: OffsetSize = offsets.last().copied().unwrap(); offsets.push(last_offset + OffsetSize::usize_as(rows.len())); let arrays = converter.convert_rows(rows)?; - let array = match arrays.get(0) { + let array = match arrays.first() { Some(array) => array.clone(), None => { return internal_err!("array_distinct: failed to get array from rows") diff --git a/datafusion/physical-plan/src/aggregates/mod.rs b/datafusion/physical-plan/src/aggregates/mod.rs index f779322456ca..f5bb4fe59b5d 100644 --- a/datafusion/physical-plan/src/aggregates/mod.rs +++ b/datafusion/physical-plan/src/aggregates/mod.rs @@ -27,7 +27,7 @@ use crate::aggregates::{ }; use crate::metrics::{ExecutionPlanMetricsSet, MetricsSet}; -use crate::windows::{get_ordered_partition_by_indices, get_window_mode}; +use crate::windows::get_ordered_partition_by_indices; use crate::{ DisplayFormatType, Distribution, ExecutionPlan, InputOrderMode, Partitioning, SendableRecordBatchStream, Statistics, @@ -45,11 +45,11 @@ use datafusion_physical_expr::{ aggregate::is_order_sensitive, equivalence::{collapse_lex_req, ProjectionMapping}, expressions::{Column, Max, Min, UnKnownColumn}, - physical_exprs_contains, reverse_order_bys, AggregateExpr, EquivalenceProperties, - LexOrdering, LexRequirement, PhysicalExpr, PhysicalSortExpr, PhysicalSortRequirement, + physical_exprs_contains, AggregateExpr, EquivalenceProperties, LexOrdering, + LexRequirement, PhysicalExpr, PhysicalSortExpr, PhysicalSortRequirement, }; -use itertools::{izip, Itertools}; +use itertools::Itertools; mod group_values; mod no_grouping; @@ -277,159 +277,6 @@ pub struct AggregateExec { output_ordering: Option, } -/// This function returns the ordering requirement of the first non-reversible -/// order-sensitive aggregate function such as ARRAY_AGG. This requirement serves -/// as the initial requirement while calculating the finest requirement among all -/// aggregate functions. If this function returns `None`, it means there is no -/// hard ordering requirement for the aggregate functions (in terms of direction). -/// Then, we can generate two alternative requirements with opposite directions. -fn get_init_req( - aggr_expr: &[Arc], - order_by_expr: &[Option], -) -> Option { - for (aggr_expr, fn_reqs) in aggr_expr.iter().zip(order_by_expr.iter()) { - // If the aggregation function is a non-reversible order-sensitive function - // and there is a hard requirement, choose first such requirement: - if is_order_sensitive(aggr_expr) - && aggr_expr.reverse_expr().is_none() - && fn_reqs.is_some() - { - return fn_reqs.clone(); - } - } - None -} - -/// This function gets the finest ordering requirement among all the aggregation -/// functions. If requirements are conflicting, (i.e. we can not compute the -/// aggregations in a single [`AggregateExec`]), the function returns an error. -fn get_finest_requirement( - aggr_expr: &mut [Arc], - order_by_expr: &mut [Option], - eq_properties: &EquivalenceProperties, -) -> Result> { - // First, we check if all the requirements are satisfied by the existing - // ordering. If so, we return `None` to indicate this. - let mut all_satisfied = true; - for (aggr_expr, fn_req) in aggr_expr.iter_mut().zip(order_by_expr.iter_mut()) { - if eq_properties.ordering_satisfy(fn_req.as_deref().unwrap_or(&[])) { - continue; - } - if let Some(reverse) = aggr_expr.reverse_expr() { - let reverse_req = fn_req.as_ref().map(|item| reverse_order_bys(item)); - if eq_properties.ordering_satisfy(reverse_req.as_deref().unwrap_or(&[])) { - // We need to update `aggr_expr` with its reverse since only its - // reverse requirement is compatible with the existing requirements: - *aggr_expr = reverse; - *fn_req = reverse_req; - continue; - } - } - // Requirement is not satisfied: - all_satisfied = false; - } - if all_satisfied { - // All of the requirements are already satisfied. - return Ok(None); - } - let mut finest_req = get_init_req(aggr_expr, order_by_expr); - for (aggr_expr, fn_req) in aggr_expr.iter_mut().zip(order_by_expr.iter_mut()) { - let Some(fn_req) = fn_req else { - continue; - }; - - if let Some(finest_req) = &mut finest_req { - if let Some(finer) = eq_properties.get_finer_ordering(finest_req, fn_req) { - *finest_req = finer; - continue; - } - // If an aggregate function is reversible, analyze whether its reverse - // direction is compatible with existing requirements: - if let Some(reverse) = aggr_expr.reverse_expr() { - let fn_req_reverse = reverse_order_bys(fn_req); - if let Some(finer) = - eq_properties.get_finer_ordering(finest_req, &fn_req_reverse) - { - // We need to update `aggr_expr` with its reverse, since only its - // reverse requirement is compatible with existing requirements: - *aggr_expr = reverse; - *finest_req = finer; - *fn_req = fn_req_reverse; - continue; - } - } - // If neither of the requirements satisfy the other, this means - // requirements are conflicting. Currently, we do not support - // conflicting requirements. - return not_impl_err!( - "Conflicting ordering requirements in aggregate functions is not supported" - ); - } else { - finest_req = Some(fn_req.clone()); - } - } - Ok(finest_req) -} - -/// Calculates search_mode for the aggregation -fn get_aggregate_search_mode( - group_by: &PhysicalGroupBy, - input: &Arc, - aggr_expr: &mut [Arc], - order_by_expr: &mut [Option], - ordering_req: &mut Vec, -) -> InputOrderMode { - let groupby_exprs = group_by - .expr - .iter() - .map(|(item, _)| item.clone()) - .collect::>(); - let mut input_order_mode = InputOrderMode::Linear; - if !group_by.is_single() || groupby_exprs.is_empty() { - return input_order_mode; - } - - if let Some((should_reverse, mode)) = - get_window_mode(&groupby_exprs, ordering_req, input) - { - let all_reversible = aggr_expr - .iter() - .all(|expr| !is_order_sensitive(expr) || expr.reverse_expr().is_some()); - if should_reverse && all_reversible { - izip!(aggr_expr.iter_mut(), order_by_expr.iter_mut()).for_each( - |(aggr, order_by)| { - if let Some(reverse) = aggr.reverse_expr() { - *aggr = reverse; - } else { - unreachable!(); - } - *order_by = order_by.as_ref().map(|ob| reverse_order_bys(ob)); - }, - ); - *ordering_req = reverse_order_bys(ordering_req); - } - input_order_mode = mode; - } - input_order_mode -} - -/// Check whether group by expression contains all of the expression inside `requirement` -// As an example Group By (c,b,a) contains all of the expressions in the `requirement`: (a ASC, b DESC) -fn group_by_contains_all_requirements( - group_by: &PhysicalGroupBy, - requirement: &LexOrdering, -) -> bool { - let physical_exprs = group_by.input_exprs(); - // When we have multiple groups (grouping set) - // since group by may be calculated on the subset of the group_by.expr() - // it is not guaranteed to have all of the requirements among group by expressions. - // Hence do the analysis: whether group by contains all requirements in the single group case. - group_by.is_single() - && requirement - .iter() - .all(|req| physical_exprs_contains(&physical_exprs, &req.expr)) -} - impl AggregateExec { /// Create a new hash aggregate execution plan pub fn try_new( @@ -477,50 +324,14 @@ impl AggregateExec { fn try_new_with_schema( mode: AggregateMode, group_by: PhysicalGroupBy, - mut aggr_expr: Vec>, + aggr_expr: Vec>, filter_expr: Vec>>, input: Arc, input_schema: SchemaRef, schema: SchemaRef, original_schema: SchemaRef, ) -> Result { - // Reset ordering requirement to `None` if aggregator is not order-sensitive - let mut order_by_expr = aggr_expr - .iter() - .map(|aggr_expr| { - let fn_reqs = aggr_expr.order_bys().map(|ordering| ordering.to_vec()); - // If - // - aggregation function is order-sensitive and - // - aggregation is performing a "first stage" calculation, and - // - at least one of the aggregate function requirement is not inside group by expression - // keep the ordering requirement as is; otherwise ignore the ordering requirement. - // In non-first stage modes, we accumulate data (using `merge_batch`) - // from different partitions (i.e. merge partial results). During - // this merge, we consider the ordering of each partial result. - // Hence, we do not need to use the ordering requirement in such - // modes as long as partial results are generated with the - // correct ordering. - fn_reqs.filter(|req| { - is_order_sensitive(aggr_expr) - && mode.is_first_stage() - && !group_by_contains_all_requirements(&group_by, req) - }) - }) - .collect::>(); - let requirement = get_finest_requirement( - &mut aggr_expr, - &mut order_by_expr, - &input.equivalence_properties(), - )?; - let mut ordering_req = requirement.unwrap_or(vec![]); - let input_order_mode = get_aggregate_search_mode( - &group_by, - &input, - &mut aggr_expr, - &mut order_by_expr, - &mut ordering_req, - ); - + let input_eq_properties = input.equivalence_properties(); // Get GROUP BY expressions: let groupby_exprs = group_by.input_exprs(); // If existing ordering satisfies a prefix of the GROUP BY expressions, @@ -528,17 +339,31 @@ impl AggregateExec { // work more efficiently. let indices = get_ordered_partition_by_indices(&groupby_exprs, &input); let mut new_requirement = indices - .into_iter() - .map(|idx| PhysicalSortRequirement { + .iter() + .map(|&idx| PhysicalSortRequirement { expr: groupby_exprs[idx].clone(), options: None, }) .collect::>(); - // Postfix ordering requirement of the aggregation to the requirement. - let req = PhysicalSortRequirement::from_sort_exprs(&ordering_req); + + let req = get_aggregate_exprs_requirement( + &aggr_expr, + &group_by, + &input_eq_properties, + &mode, + )?; new_requirement.extend(req); new_requirement = collapse_lex_req(new_requirement); + let input_order_mode = + if indices.len() == groupby_exprs.len() && !indices.is_empty() { + InputOrderMode::Sorted + } else if !indices.is_empty() { + InputOrderMode::PartiallySorted(indices) + } else { + InputOrderMode::Linear + }; + // construct a map from the input expression to the output expression of the Aggregation group by let projection_mapping = ProjectionMapping::try_new(&group_by.expr, &input.schema())?; @@ -546,9 +371,8 @@ impl AggregateExec { let required_input_ordering = (!new_requirement.is_empty()).then_some(new_requirement); - let aggregate_eqs = input - .equivalence_properties() - .project(&projection_mapping, schema.clone()); + let aggregate_eqs = + input_eq_properties.project(&projection_mapping, schema.clone()); let output_ordering = aggregate_eqs.oeq_class().output_ordering(); Ok(AggregateExec { @@ -998,6 +822,121 @@ fn group_schema(schema: &Schema, group_count: usize) -> SchemaRef { Arc::new(Schema::new(group_fields)) } +/// Determines the lexical ordering requirement for an aggregate expression. +/// +/// # Parameters +/// +/// - `aggr_expr`: A reference to an `Arc` representing the +/// aggregate expression. +/// - `group_by`: A reference to a `PhysicalGroupBy` instance representing the +/// physical GROUP BY expression. +/// - `agg_mode`: A reference to an `AggregateMode` instance representing the +/// mode of aggregation. +/// +/// # Returns +/// +/// A `LexOrdering` instance indicating the lexical ordering requirement for +/// the aggregate expression. +fn get_aggregate_expr_req( + aggr_expr: &Arc, + group_by: &PhysicalGroupBy, + agg_mode: &AggregateMode, +) -> LexOrdering { + // If the aggregation function is not order sensitive, or the aggregation + // is performing a "second stage" calculation, or all aggregate function + // requirements are inside the GROUP BY expression, then ignore the ordering + // requirement. + if !is_order_sensitive(aggr_expr) || !agg_mode.is_first_stage() { + return vec![]; + } + + let mut req = aggr_expr.order_bys().unwrap_or_default().to_vec(); + + // In non-first stage modes, we accumulate data (using `merge_batch`) from + // different partitions (i.e. merge partial results). During this merge, we + // consider the ordering of each partial result. Hence, we do not need to + // use the ordering requirement in such modes as long as partial results are + // generated with the correct ordering. + if group_by.is_single() { + // Remove all orderings that occur in the group by. These requirements + // will definitely be satisfied -- Each group by expression will have + // distinct values per group, hence all requirements are satisfied. + let physical_exprs = group_by.input_exprs(); + req.retain(|sort_expr| { + !physical_exprs_contains(&physical_exprs, &sort_expr.expr) + }); + } + req +} + +/// Computes the finer ordering for between given existing ordering requirement +/// of aggregate expression. +/// +/// # Parameters +/// +/// * `existing_req` - The existing lexical ordering that needs refinement. +/// * `aggr_expr` - A reference to an aggregate expression trait object. +/// * `group_by` - Information about the physical grouping (e.g group by expression). +/// * `eq_properties` - Equivalence properties relevant to the computation. +/// * `agg_mode` - The mode of aggregation (e.g., Partial, Final, etc.). +/// +/// # Returns +/// +/// An `Option` representing the computed finer lexical ordering, +/// or `None` if there is no finer ordering; e.g. the existing requirement and +/// the aggregator requirement is incompatible. +fn finer_ordering( + existing_req: &LexOrdering, + aggr_expr: &Arc, + group_by: &PhysicalGroupBy, + eq_properties: &EquivalenceProperties, + agg_mode: &AggregateMode, +) -> Option { + let aggr_req = get_aggregate_expr_req(aggr_expr, group_by, agg_mode); + eq_properties.get_finer_ordering(existing_req, &aggr_req) +} + +/// Get the common requirement that satisfies all the aggregate expressions. +/// +/// # Parameters +/// +/// - `aggr_exprs`: A slice of `Arc` containing all the +/// aggregate expressions. +/// - `group_by`: A reference to a `PhysicalGroupBy` instance representing the +/// physical GROUP BY expression. +/// - `eq_properties`: A reference to an `EquivalenceProperties` instance +/// representing equivalence properties for ordering. +/// - `agg_mode`: A reference to an `AggregateMode` instance representing the +/// mode of aggregation. +/// +/// # Returns +/// +/// A `LexRequirement` instance, which is the requirement that satisfies all the +/// aggregate requirements. Returns an error in case of conflicting requirements. +fn get_aggregate_exprs_requirement( + aggr_exprs: &[Arc], + group_by: &PhysicalGroupBy, + eq_properties: &EquivalenceProperties, + agg_mode: &AggregateMode, +) -> Result { + let mut requirement = vec![]; + for aggr_expr in aggr_exprs.iter() { + if let Some(finer_ordering) = + finer_ordering(&requirement, aggr_expr, group_by, eq_properties, agg_mode) + { + requirement = finer_ordering; + } else { + // If neither of the requirements satisfy the other, this means + // requirements are conflicting. Currently, we do not support + // conflicting requirements. + return not_impl_err!( + "Conflicting ordering requirements in aggregate functions is not supported" + ); + } + } + Ok(PhysicalSortRequirement::from_sort_exprs(&requirement)) +} + /// returns physical expressions for arguments to evaluate against a batch /// The expressions are different depending on `mode`: /// * Partial: AggregateExpr::expressions @@ -1013,33 +952,27 @@ fn aggregate_expressions( | AggregateMode::SinglePartitioned => Ok(aggr_expr .iter() .map(|agg| { - let mut result = agg.expressions().clone(); - // In partial mode, append ordering requirements to expressions' results. - // Ordering requirements are used by subsequent executors to satisfy the required - // ordering for `AggregateMode::FinalPartitioned`/`AggregateMode::Final` modes. - if matches!(mode, AggregateMode::Partial) { - if let Some(ordering_req) = agg.order_bys() { - let ordering_exprs = ordering_req - .iter() - .map(|item| item.expr.clone()) - .collect::>(); - result.extend(ordering_exprs); - } + let mut result = agg.expressions(); + // Append ordering requirements to expressions' results. This + // way order sensitive aggregators can satisfy requirement + // themselves. + if let Some(ordering_req) = agg.order_bys() { + result.extend(ordering_req.iter().map(|item| item.expr.clone())); } result }) .collect()), - // in this mode, we build the merge expressions of the aggregation + // In this mode, we build the merge expressions of the aggregation. AggregateMode::Final | AggregateMode::FinalPartitioned => { let mut col_idx_base = col_idx_base; - Ok(aggr_expr + aggr_expr .iter() .map(|agg| { let exprs = merge_expressions(col_idx_base, agg)?; col_idx_base += exprs.len(); Ok(exprs) }) - .collect::>>()?) + .collect() } } } @@ -1052,14 +985,13 @@ fn merge_expressions( index_base: usize, expr: &Arc, ) -> Result>> { - Ok(expr - .state_fields()? - .iter() - .enumerate() - .map(|(idx, f)| { - Arc::new(Column::new(f.name(), index_base + idx)) as Arc - }) - .collect::>()) + expr.state_fields().map(|fields| { + fields + .iter() + .enumerate() + .map(|(idx, f)| Arc::new(Column::new(f.name(), index_base + idx)) as _) + .collect() + }) } pub(crate) type AccumulatorItem = Box; @@ -1070,7 +1002,7 @@ fn create_accumulators( aggr_expr .iter() .map(|expr| expr.create_accumulator()) - .collect::>>() + .collect() } /// returns a vector of ArrayRefs, where each entry corresponds to either the @@ -1081,8 +1013,8 @@ fn finalize_aggregation( ) -> Result> { match mode { AggregateMode::Partial => { - // build the vector of states - let a = accumulators + // Build the vector of states + accumulators .iter() .map(|accumulator| { accumulator.state().and_then(|e| { @@ -1091,18 +1023,18 @@ fn finalize_aggregation( .collect::>>() }) }) - .collect::>>()?; - Ok(a.iter().flatten().cloned().collect::>()) + .flatten_ok() + .collect() } AggregateMode::Final | AggregateMode::FinalPartitioned | AggregateMode::Single | AggregateMode::SinglePartitioned => { - // merge the state to the final value + // Merge the state to the final value accumulators .iter() .map(|accumulator| accumulator.evaluate().and_then(|v| v.to_array())) - .collect::>>() + .collect() } } } @@ -1125,9 +1057,7 @@ pub(crate) fn evaluate_many( expr: &[Vec>], batch: &RecordBatch, ) -> Result>> { - expr.iter() - .map(|expr| evaluate(expr, batch)) - .collect::>>() + expr.iter().map(|expr| evaluate(expr, batch)).collect() } fn evaluate_optional( @@ -1143,7 +1073,7 @@ fn evaluate_optional( }) .transpose() }) - .collect::>>() + .collect() } /// Evaluate a group by expression against a `RecordBatch` @@ -1204,9 +1134,7 @@ mod tests { use std::task::{Context, Poll}; use super::*; - use crate::aggregates::{ - get_finest_requirement, AggregateExec, AggregateMode, PhysicalGroupBy, - }; + use crate::aggregates::{AggregateExec, AggregateMode, PhysicalGroupBy}; use crate::coalesce_batches::CoalesceBatchesExec; use crate::coalesce_partitions::CoalescePartitionsExec; use crate::common; @@ -1228,15 +1156,16 @@ mod tests { Result, ScalarValue, }; use datafusion_execution::config::SessionConfig; + use datafusion_execution::memory_pool::FairSpillPool; use datafusion_execution::runtime_env::{RuntimeConfig, RuntimeEnv}; use datafusion_physical_expr::expressions::{ - lit, ApproxDistinct, Count, FirstValue, LastValue, Median, + lit, ApproxDistinct, Count, FirstValue, LastValue, Median, OrderSensitiveArrayAgg, }; use datafusion_physical_expr::{ - AggregateExpr, EquivalenceProperties, PhysicalExpr, PhysicalSortExpr, + reverse_order_bys, AggregateExpr, EquivalenceProperties, PhysicalExpr, + PhysicalSortExpr, }; - use datafusion_execution::memory_pool::FairSpillPool; use futures::{FutureExt, Stream}; // Generate a schema which consists of 5 columns (a, b, c, d, e) @@ -2093,11 +2022,6 @@ mod tests { descending: false, nulls_first: false, }; - // This is the reverse requirement of options1 - let options2 = SortOptions { - descending: true, - nulls_first: true, - }; let col_a = &col("a", &test_schema)?; let col_b = &col("b", &test_schema)?; let col_c = &col("c", &test_schema)?; @@ -2106,7 +2030,7 @@ mod tests { eq_properties.add_equal_conditions(col_a, col_b); // Aggregate requirements are // [None], [a ASC], [a ASC, b ASC, c ASC], [a ASC, b ASC] respectively - let mut order_by_exprs = vec![ + let order_by_exprs = vec![ None, Some(vec![PhysicalSortExpr { expr: col_a.clone(), @@ -2136,14 +2060,8 @@ mod tests { options: options1, }, ]), - // Since aggregate expression is reversible (FirstValue), we should be able to resolve below - // contradictory requirement by reversing it. - Some(vec![PhysicalSortExpr { - expr: col_b.clone(), - options: options2, - }]), ]; - let common_requirement = Some(vec![ + let common_requirement = vec![ PhysicalSortExpr { expr: col_a.clone(), options: options1, @@ -2152,17 +2070,28 @@ mod tests { expr: col_c.clone(), options: options1, }, - ]); - let aggr_expr = Arc::new(FirstValue::new( - col_a.clone(), - "first1", - DataType::Int32, - vec![], - vec![], - )) as _; - let mut aggr_exprs = vec![aggr_expr; order_by_exprs.len()]; - let res = - get_finest_requirement(&mut aggr_exprs, &mut order_by_exprs, &eq_properties)?; + ]; + let aggr_exprs = order_by_exprs + .into_iter() + .map(|order_by_expr| { + Arc::new(OrderSensitiveArrayAgg::new( + col_a.clone(), + "array_agg", + DataType::Int32, + false, + vec![], + order_by_expr.unwrap_or_default(), + )) as _ + }) + .collect::>(); + let group_by = PhysicalGroupBy::new_single(vec![]); + let res = get_aggregate_exprs_requirement( + &aggr_exprs, + &group_by, + &eq_properties, + &AggregateMode::Partial, + )?; + let res = PhysicalSortRequirement::to_sort_exprs(res); assert_eq!(res, common_requirement); Ok(()) } diff --git a/datafusion/sqllogictest/src/engines/datafusion_engine/mod.rs b/datafusion/sqllogictest/src/engines/datafusion_engine/mod.rs index 663bbdd5a3c7..8e2bbbfe4f69 100644 --- a/datafusion/sqllogictest/src/engines/datafusion_engine/mod.rs +++ b/datafusion/sqllogictest/src/engines/datafusion_engine/mod.rs @@ -21,5 +21,4 @@ mod normalize; mod runner; pub use error::*; -pub use normalize::*; pub use runner::*; diff --git a/datafusion/sqllogictest/test_files/distinct_on.slt b/datafusion/sqllogictest/test_files/distinct_on.slt index 9a7117b69b99..3f609e254839 100644 --- a/datafusion/sqllogictest/test_files/distinct_on.slt +++ b/datafusion/sqllogictest/test_files/distinct_on.slt @@ -78,7 +78,7 @@ c 4 query I SELECT DISTINCT ON (c1) c2 FROM aggregate_test_100 ORDER BY c1, c3; ---- -5 +4 4 2 1 @@ -100,10 +100,9 @@ ProjectionExec: expr=[FIRST_VALUE(aggregate_test_100.c3) ORDER BY [aggregate_tes ------AggregateExec: mode=FinalPartitioned, gby=[c1@0 as c1], aggr=[FIRST_VALUE(aggregate_test_100.c3), FIRST_VALUE(aggregate_test_100.c2)] --------CoalesceBatchesExec: target_batch_size=8192 ----------RepartitionExec: partitioning=Hash([c1@0], 4), input_partitions=4 -------------AggregateExec: mode=Partial, gby=[c1@0 as c1], aggr=[FIRST_VALUE(aggregate_test_100.c3), FIRST_VALUE(aggregate_test_100.c2)], ordering_mode=Sorted ---------------SortExec: expr=[c1@0 ASC NULLS LAST,c3@2 ASC NULLS LAST] -----------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1, c2, c3], has_header=true +------------AggregateExec: mode=Partial, gby=[c1@0 as c1], aggr=[FIRST_VALUE(aggregate_test_100.c3), FIRST_VALUE(aggregate_test_100.c2)] +--------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +----------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1, c2, c3], has_header=true # ON expressions are not a sub-set of the ORDER BY expressions query error SELECT DISTINCT ON expressions must match initial ORDER BY expressions diff --git a/datafusion/sqllogictest/test_files/groupby.slt b/datafusion/sqllogictest/test_files/groupby.slt index f1b6a57287b5..bbf21e135fe4 100644 --- a/datafusion/sqllogictest/test_files/groupby.slt +++ b/datafusion/sqllogictest/test_files/groupby.slt @@ -2019,17 +2019,16 @@ SortPreservingMergeExec: [col0@0 ASC NULLS LAST] ------AggregateExec: mode=FinalPartitioned, gby=[col0@0 as col0, col1@1 as col1, col2@2 as col2], aggr=[LAST_VALUE(r.col1)] --------CoalesceBatchesExec: target_batch_size=8192 ----------RepartitionExec: partitioning=Hash([col0@0, col1@1, col2@2], 4), input_partitions=4 -------------AggregateExec: mode=Partial, gby=[col0@0 as col0, col1@1 as col1, col2@2 as col2], aggr=[LAST_VALUE(r.col1)], ordering_mode=PartiallySorted([0]) ---------------SortExec: expr=[col0@3 ASC NULLS LAST] -----------------ProjectionExec: expr=[col0@2 as col0, col1@3 as col1, col2@4 as col2, col0@0 as col0, col1@1 as col1] -------------------CoalesceBatchesExec: target_batch_size=8192 ---------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(col0@0, col0@0)] -----------------------CoalesceBatchesExec: target_batch_size=8192 -------------------------RepartitionExec: partitioning=Hash([col0@0], 4), input_partitions=1 ---------------------------MemoryExec: partitions=1, partition_sizes=[3] -----------------------CoalesceBatchesExec: target_batch_size=8192 -------------------------RepartitionExec: partitioning=Hash([col0@0], 4), input_partitions=1 ---------------------------MemoryExec: partitions=1, partition_sizes=[3] +------------AggregateExec: mode=Partial, gby=[col0@0 as col0, col1@1 as col1, col2@2 as col2], aggr=[LAST_VALUE(r.col1)] +--------------ProjectionExec: expr=[col0@2 as col0, col1@3 as col1, col2@4 as col2, col0@0 as col0, col1@1 as col1] +----------------CoalesceBatchesExec: target_batch_size=8192 +------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(col0@0, col0@0)] +--------------------CoalesceBatchesExec: target_batch_size=8192 +----------------------RepartitionExec: partitioning=Hash([col0@0], 4), input_partitions=1 +------------------------MemoryExec: partitions=1, partition_sizes=[3] +--------------------CoalesceBatchesExec: target_batch_size=8192 +----------------------RepartitionExec: partitioning=Hash([col0@0], 4), input_partitions=1 +------------------------MemoryExec: partitions=1, partition_sizes=[3] # Columns in the table are a,b,c,d. Source is CsvExec which is ordered by # a,b,c column. Column a has cardinality 2, column b has cardinality 4. @@ -2209,7 +2208,7 @@ ProjectionExec: expr=[a@0 as a, b@1 as b, LAST_VALUE(annotated_data_infinite2.c) ----StreamingTableExec: partition_sizes=1, projection=[a, b, c], infinite_source=true, output_ordering=[a@0 ASC NULLS LAST, b@1 ASC NULLS LAST, c@2 ASC NULLS LAST] query III -SELECT a, b, LAST_VALUE(c ORDER BY a DESC) as last_c +SELECT a, b, LAST_VALUE(c ORDER BY a DESC, c ASC) as last_c FROM annotated_data_infinite2 GROUP BY a, b ---- @@ -2509,7 +2508,7 @@ Projection: sales_global.country, ARRAY_AGG(sales_global.amount) ORDER BY [sales ----TableScan: sales_global projection=[country, amount] physical_plan ProjectionExec: expr=[country@0 as country, ARRAY_AGG(sales_global.amount) ORDER BY [sales_global.amount DESC NULLS FIRST]@1 as amounts, FIRST_VALUE(sales_global.amount) ORDER BY [sales_global.amount ASC NULLS LAST]@2 as fv1, LAST_VALUE(sales_global.amount) ORDER BY [sales_global.amount DESC NULLS FIRST]@3 as fv2] ---AggregateExec: mode=Single, gby=[country@0 as country], aggr=[ARRAY_AGG(sales_global.amount), LAST_VALUE(sales_global.amount), LAST_VALUE(sales_global.amount)] +--AggregateExec: mode=Single, gby=[country@0 as country], aggr=[ARRAY_AGG(sales_global.amount), FIRST_VALUE(sales_global.amount), LAST_VALUE(sales_global.amount)] ----SortExec: expr=[amount@1 DESC] ------MemoryExec: partitions=1, partition_sizes=[1] @@ -2540,7 +2539,7 @@ Projection: sales_global.country, ARRAY_AGG(sales_global.amount) ORDER BY [sales ----TableScan: sales_global projection=[country, amount] physical_plan ProjectionExec: expr=[country@0 as country, ARRAY_AGG(sales_global.amount) ORDER BY [sales_global.amount ASC NULLS LAST]@1 as amounts, FIRST_VALUE(sales_global.amount) ORDER BY [sales_global.amount ASC NULLS LAST]@2 as fv1, LAST_VALUE(sales_global.amount) ORDER BY [sales_global.amount DESC NULLS FIRST]@3 as fv2] ---AggregateExec: mode=Single, gby=[country@0 as country], aggr=[ARRAY_AGG(sales_global.amount), FIRST_VALUE(sales_global.amount), FIRST_VALUE(sales_global.amount)] +--AggregateExec: mode=Single, gby=[country@0 as country], aggr=[ARRAY_AGG(sales_global.amount), FIRST_VALUE(sales_global.amount), LAST_VALUE(sales_global.amount)] ----SortExec: expr=[amount@1 ASC NULLS LAST] ------MemoryExec: partitions=1, partition_sizes=[1] @@ -2572,7 +2571,7 @@ Projection: sales_global.country, FIRST_VALUE(sales_global.amount) ORDER BY [sal ----TableScan: sales_global projection=[country, amount] physical_plan ProjectionExec: expr=[country@0 as country, FIRST_VALUE(sales_global.amount) ORDER BY [sales_global.amount ASC NULLS LAST]@1 as fv1, LAST_VALUE(sales_global.amount) ORDER BY [sales_global.amount DESC NULLS FIRST]@2 as fv2, ARRAY_AGG(sales_global.amount) ORDER BY [sales_global.amount ASC NULLS LAST]@3 as amounts] ---AggregateExec: mode=Single, gby=[country@0 as country], aggr=[FIRST_VALUE(sales_global.amount), FIRST_VALUE(sales_global.amount), ARRAY_AGG(sales_global.amount)] +--AggregateExec: mode=Single, gby=[country@0 as country], aggr=[FIRST_VALUE(sales_global.amount), LAST_VALUE(sales_global.amount), ARRAY_AGG(sales_global.amount)] ----SortExec: expr=[amount@1 ASC NULLS LAST] ------MemoryExec: partitions=1, partition_sizes=[1] @@ -2637,9 +2636,8 @@ Projection: sales_global.country, FIRST_VALUE(sales_global.amount) ORDER BY [sal ------TableScan: sales_global projection=[country, ts, amount] physical_plan ProjectionExec: expr=[country@0 as country, FIRST_VALUE(sales_global.amount) ORDER BY [sales_global.ts DESC NULLS FIRST]@1 as fv1, LAST_VALUE(sales_global.amount) ORDER BY [sales_global.ts DESC NULLS FIRST]@2 as lv1, SUM(sales_global.amount) ORDER BY [sales_global.ts DESC NULLS FIRST]@3 as sum1] ---AggregateExec: mode=Single, gby=[country@0 as country], aggr=[LAST_VALUE(sales_global.amount), FIRST_VALUE(sales_global.amount), SUM(sales_global.amount)] -----SortExec: expr=[ts@1 ASC NULLS LAST] -------MemoryExec: partitions=1, partition_sizes=[1] +--AggregateExec: mode=Single, gby=[country@0 as country], aggr=[FIRST_VALUE(sales_global.amount), LAST_VALUE(sales_global.amount), SUM(sales_global.amount)] +----MemoryExec: partitions=1, partition_sizes=[1] query TRRR rowsort SELECT country, FIRST_VALUE(amount ORDER BY ts DESC) as fv1, @@ -2672,8 +2670,7 @@ Projection: sales_global.country, FIRST_VALUE(sales_global.amount) ORDER BY [sal physical_plan ProjectionExec: expr=[country@0 as country, FIRST_VALUE(sales_global.amount) ORDER BY [sales_global.ts DESC NULLS FIRST]@1 as fv1, LAST_VALUE(sales_global.amount) ORDER BY [sales_global.ts DESC NULLS FIRST]@2 as lv1, SUM(sales_global.amount) ORDER BY [sales_global.ts DESC NULLS FIRST]@3 as sum1] --AggregateExec: mode=Single, gby=[country@0 as country], aggr=[FIRST_VALUE(sales_global.amount), LAST_VALUE(sales_global.amount), SUM(sales_global.amount)] -----SortExec: expr=[ts@1 DESC] -------MemoryExec: partitions=1, partition_sizes=[1] +----MemoryExec: partitions=1, partition_sizes=[1] query TRRR rowsort SELECT country, FIRST_VALUE(amount ORDER BY ts DESC) as fv1, @@ -2709,12 +2706,11 @@ physical_plan SortExec: expr=[sn@2 ASC NULLS LAST] --ProjectionExec: expr=[zip_code@1 as zip_code, country@2 as country, sn@0 as sn, ts@3 as ts, currency@4 as currency, LAST_VALUE(e.amount) ORDER BY [e.sn ASC NULLS LAST]@5 as last_rate] ----AggregateExec: mode=Single, gby=[sn@2 as sn, zip_code@0 as zip_code, country@1 as country, ts@3 as ts, currency@4 as currency], aggr=[LAST_VALUE(e.amount)] -------SortExec: expr=[sn@5 ASC NULLS LAST] ---------ProjectionExec: expr=[zip_code@4 as zip_code, country@5 as country, sn@6 as sn, ts@7 as ts, currency@8 as currency, sn@0 as sn, amount@3 as amount] -----------CoalesceBatchesExec: target_batch_size=8192 -------------HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(currency@2, currency@4)], filter=ts@0 >= ts@1 ---------------MemoryExec: partitions=1, partition_sizes=[1] ---------------MemoryExec: partitions=1, partition_sizes=[1] +------ProjectionExec: expr=[zip_code@4 as zip_code, country@5 as country, sn@6 as sn, ts@7 as ts, currency@8 as currency, sn@0 as sn, amount@3 as amount] +--------CoalesceBatchesExec: target_batch_size=8192 +----------HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(currency@2, currency@4)], filter=ts@0 >= ts@1 +------------MemoryExec: partitions=1, partition_sizes=[1] +------------MemoryExec: partitions=1, partition_sizes=[1] query ITIPTR rowsort SELECT s.zip_code, s.country, s.sn, s.ts, s.currency, LAST_VALUE(e.amount ORDER BY e.sn) AS last_rate @@ -2759,8 +2755,7 @@ SortPreservingMergeExec: [country@0 ASC NULLS LAST] ----------RepartitionExec: partitioning=Hash([country@0], 8), input_partitions=8 ------------RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1 --------------AggregateExec: mode=Partial, gby=[country@0 as country], aggr=[FIRST_VALUE(sales_global.amount), LAST_VALUE(sales_global.amount)] -----------------SortExec: expr=[ts@1 ASC NULLS LAST] -------------------MemoryExec: partitions=1, partition_sizes=[1] +----------------MemoryExec: partitions=1, partition_sizes=[1] query TRR SELECT country, FIRST_VALUE(amount ORDER BY ts ASC) AS fv1, @@ -2791,13 +2786,12 @@ physical_plan SortPreservingMergeExec: [country@0 ASC NULLS LAST] --SortExec: expr=[country@0 ASC NULLS LAST] ----ProjectionExec: expr=[country@0 as country, FIRST_VALUE(sales_global.amount) ORDER BY [sales_global.ts ASC NULLS LAST]@1 as fv1, LAST_VALUE(sales_global.amount) ORDER BY [sales_global.ts DESC NULLS FIRST]@2 as fv2] -------AggregateExec: mode=FinalPartitioned, gby=[country@0 as country], aggr=[FIRST_VALUE(sales_global.amount), FIRST_VALUE(sales_global.amount)] +------AggregateExec: mode=FinalPartitioned, gby=[country@0 as country], aggr=[FIRST_VALUE(sales_global.amount), LAST_VALUE(sales_global.amount)] --------CoalesceBatchesExec: target_batch_size=8192 ----------RepartitionExec: partitioning=Hash([country@0], 8), input_partitions=8 ------------RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1 ---------------AggregateExec: mode=Partial, gby=[country@0 as country], aggr=[FIRST_VALUE(sales_global.amount), FIRST_VALUE(sales_global.amount)] -----------------SortExec: expr=[ts@1 ASC NULLS LAST] -------------------MemoryExec: partitions=1, partition_sizes=[1] +--------------AggregateExec: mode=Partial, gby=[country@0 as country], aggr=[FIRST_VALUE(sales_global.amount), LAST_VALUE(sales_global.amount)] +----------------MemoryExec: partitions=1, partition_sizes=[1] query TRR SELECT country, FIRST_VALUE(amount ORDER BY ts ASC) AS fv1, @@ -2831,16 +2825,15 @@ ProjectionExec: expr=[FIRST_VALUE(sales_global.amount) ORDER BY [sales_global.ts --AggregateExec: mode=Final, gby=[], aggr=[FIRST_VALUE(sales_global.amount), LAST_VALUE(sales_global.amount)] ----CoalescePartitionsExec ------AggregateExec: mode=Partial, gby=[], aggr=[FIRST_VALUE(sales_global.amount), LAST_VALUE(sales_global.amount)] ---------SortExec: expr=[ts@0 ASC NULLS LAST] -----------RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1 -------------MemoryExec: partitions=1, partition_sizes=[1] +--------RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1 +----------MemoryExec: partitions=1, partition_sizes=[1] query RR SELECT FIRST_VALUE(amount ORDER BY ts ASC) AS fv1, LAST_VALUE(amount ORDER BY ts ASC) AS fv2 FROM sales_global ---- -30 80 +30 100 # Conversion in between FIRST_VALUE and LAST_VALUE to resolve # contradictory requirements should work in multi partitions. @@ -2855,12 +2848,11 @@ Projection: FIRST_VALUE(sales_global.amount) ORDER BY [sales_global.ts ASC NULLS ----TableScan: sales_global projection=[ts, amount] physical_plan ProjectionExec: expr=[FIRST_VALUE(sales_global.amount) ORDER BY [sales_global.ts ASC NULLS LAST]@0 as fv1, LAST_VALUE(sales_global.amount) ORDER BY [sales_global.ts DESC NULLS FIRST]@1 as fv2] ---AggregateExec: mode=Final, gby=[], aggr=[FIRST_VALUE(sales_global.amount), FIRST_VALUE(sales_global.amount)] +--AggregateExec: mode=Final, gby=[], aggr=[FIRST_VALUE(sales_global.amount), LAST_VALUE(sales_global.amount)] ----CoalescePartitionsExec -------AggregateExec: mode=Partial, gby=[], aggr=[FIRST_VALUE(sales_global.amount), FIRST_VALUE(sales_global.amount)] ---------SortExec: expr=[ts@0 ASC NULLS LAST] -----------RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1 -------------MemoryExec: partitions=1, partition_sizes=[1] +------AggregateExec: mode=Partial, gby=[], aggr=[FIRST_VALUE(sales_global.amount), LAST_VALUE(sales_global.amount)] +--------RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1 +----------MemoryExec: partitions=1, partition_sizes=[1] query RR SELECT FIRST_VALUE(amount ORDER BY ts ASC) AS fv1, @@ -2993,10 +2985,10 @@ physical_plan SortPreservingMergeExec: [country@0 ASC NULLS LAST] --SortExec: expr=[country@0 ASC NULLS LAST] ----ProjectionExec: expr=[country@0 as country, ARRAY_AGG(sales_global.amount) ORDER BY [sales_global.amount DESC NULLS FIRST]@1 as amounts, FIRST_VALUE(sales_global.amount) ORDER BY [sales_global.amount ASC NULLS LAST]@2 as fv1, LAST_VALUE(sales_global.amount) ORDER BY [sales_global.amount DESC NULLS FIRST]@3 as fv2] -------AggregateExec: mode=FinalPartitioned, gby=[country@0 as country], aggr=[ARRAY_AGG(sales_global.amount), LAST_VALUE(sales_global.amount), LAST_VALUE(sales_global.amount)] +------AggregateExec: mode=FinalPartitioned, gby=[country@0 as country], aggr=[ARRAY_AGG(sales_global.amount), FIRST_VALUE(sales_global.amount), LAST_VALUE(sales_global.amount)] --------CoalesceBatchesExec: target_batch_size=4 ----------RepartitionExec: partitioning=Hash([country@0], 8), input_partitions=8 -------------AggregateExec: mode=Partial, gby=[country@0 as country], aggr=[ARRAY_AGG(sales_global.amount), LAST_VALUE(sales_global.amount), LAST_VALUE(sales_global.amount)] +------------AggregateExec: mode=Partial, gby=[country@0 as country], aggr=[ARRAY_AGG(sales_global.amount), FIRST_VALUE(sales_global.amount), LAST_VALUE(sales_global.amount)] --------------SortExec: expr=[amount@1 DESC] ----------------RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1 ------------------MemoryExec: partitions=1, partition_sizes=[1] @@ -3639,10 +3631,10 @@ Projection: FIRST_VALUE(multiple_ordered_table.a) ORDER BY [multiple_ordered_tab ----TableScan: multiple_ordered_table projection=[a, c, d] physical_plan ProjectionExec: expr=[FIRST_VALUE(multiple_ordered_table.a) ORDER BY [multiple_ordered_table.a ASC NULLS LAST]@1 as first_a, LAST_VALUE(multiple_ordered_table.c) ORDER BY [multiple_ordered_table.c DESC NULLS FIRST]@2 as last_c] ---AggregateExec: mode=FinalPartitioned, gby=[d@0 as d], aggr=[FIRST_VALUE(multiple_ordered_table.a), FIRST_VALUE(multiple_ordered_table.c)] +--AggregateExec: mode=FinalPartitioned, gby=[d@0 as d], aggr=[FIRST_VALUE(multiple_ordered_table.a), LAST_VALUE(multiple_ordered_table.c)] ----CoalesceBatchesExec: target_batch_size=2 ------RepartitionExec: partitioning=Hash([d@0], 8), input_partitions=8 ---------AggregateExec: mode=Partial, gby=[d@2 as d], aggr=[FIRST_VALUE(multiple_ordered_table.a), FIRST_VALUE(multiple_ordered_table.c)] +--------AggregateExec: mode=Partial, gby=[d@2 as d], aggr=[FIRST_VALUE(multiple_ordered_table.a), LAST_VALUE(multiple_ordered_table.c)] ----------RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1 ------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, c, d], output_orderings=[[a@0 ASC NULLS LAST], [c@1 ASC NULLS LAST]], has_header=true diff --git a/datafusion/sqllogictest/test_files/joins.slt b/datafusion/sqllogictest/test_files/joins.slt index 9a349f600091..a7146a5a91c4 100644 --- a/datafusion/sqllogictest/test_files/joins.slt +++ b/datafusion/sqllogictest/test_files/joins.slt @@ -3454,7 +3454,7 @@ SortPreservingMergeExec: [a@0 ASC] ------AggregateExec: mode=FinalPartitioned, gby=[a@0 as a, b@1 as b, c@2 as c], aggr=[LAST_VALUE(r.b)] --------CoalesceBatchesExec: target_batch_size=2 ----------RepartitionExec: partitioning=Hash([a@0, b@1, c@2], 2), input_partitions=2 -------------AggregateExec: mode=Partial, gby=[a@0 as a, b@1 as b, c@2 as c], aggr=[LAST_VALUE(r.b)], ordering_mode=PartiallySorted([0]) +------------AggregateExec: mode=Partial, gby=[a@0 as a, b@1 as b, c@2 as c], aggr=[LAST_VALUE(r.b)] --------------CoalesceBatchesExec: target_batch_size=2 ----------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(a@0, a@0)] ------------------CoalesceBatchesExec: target_batch_size=2 @@ -3462,7 +3462,7 @@ SortPreservingMergeExec: [a@0 ASC] ----------------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 ------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, b, c], output_ordering=[a@0 ASC, b@1 ASC NULLS LAST, c@2 ASC NULLS LAST], has_header=true ------------------CoalesceBatchesExec: target_batch_size=2 ---------------------RepartitionExec: partitioning=Hash([a@0], 2), input_partitions=2, preserve_order=true, sort_exprs=a@0 ASC,b@1 ASC NULLS LAST +--------------------RepartitionExec: partitioning=Hash([a@0], 2), input_partitions=2 ----------------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 ------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, b], output_ordering=[a@0 ASC, b@1 ASC NULLS LAST], has_header=true From 8284371cb5dbeb5d0b1d50c420affb9be86b1599 Mon Sep 17 00:00:00 2001 From: Alex Huang Date: Thu, 28 Dec 2023 22:08:09 +0100 Subject: [PATCH 511/572] feat: support 'LargeList' in `array_pop_front` and `array_pop_back` (#8569) * support largelist in pop back * support largelist in pop front * add function comment * use execution error * use execution error * spilit the general code --- .../physical-expr/src/array_expressions.rs | 90 ++++++++++++++----- datafusion/sqllogictest/test_files/array.slt | 75 ++++++++++++++++ 2 files changed, 141 insertions(+), 24 deletions(-) diff --git a/datafusion/physical-expr/src/array_expressions.rs b/datafusion/physical-expr/src/array_expressions.rs index 7a986810bad2..250250630eff 100644 --- a/datafusion/physical-expr/src/array_expressions.rs +++ b/datafusion/physical-expr/src/array_expressions.rs @@ -743,22 +743,78 @@ where )?)) } -/// array_pop_back SQL function -pub fn array_pop_back(args: &[ArrayRef]) -> Result { - if args.len() != 1 { - return exec_err!("array_pop_back needs one argument"); - } +fn general_pop_front_list( + array: &GenericListArray, +) -> Result +where + i64: TryInto, +{ + let from_array = Int64Array::from(vec![2; array.len()]); + let to_array = Int64Array::from( + array + .iter() + .map(|arr| arr.map_or(0, |arr| arr.len() as i64)) + .collect::>(), + ); + general_array_slice::(array, &from_array, &to_array) +} - let list_array = as_list_array(&args[0])?; - let from_array = Int64Array::from(vec![1; list_array.len()]); +fn general_pop_back_list( + array: &GenericListArray, +) -> Result +where + i64: TryInto, +{ + let from_array = Int64Array::from(vec![1; array.len()]); let to_array = Int64Array::from( - list_array + array .iter() .map(|arr| arr.map_or(0, |arr| arr.len() as i64 - 1)) .collect::>(), ); - let args = vec![args[0].clone(), Arc::new(from_array), Arc::new(to_array)]; - array_slice(args.as_slice()) + general_array_slice::(array, &from_array, &to_array) +} + +/// array_pop_front SQL function +pub fn array_pop_front(args: &[ArrayRef]) -> Result { + let array_data_type = args[0].data_type(); + match array_data_type { + DataType::List(_) => { + let array = as_list_array(&args[0])?; + general_pop_front_list::(array) + } + DataType::LargeList(_) => { + let array = as_large_list_array(&args[0])?; + general_pop_front_list::(array) + } + _ => exec_err!( + "array_pop_front does not support type: {:?}", + array_data_type + ), + } +} + +/// array_pop_back SQL function +pub fn array_pop_back(args: &[ArrayRef]) -> Result { + if args.len() != 1 { + return exec_err!("array_pop_back needs one argument"); + } + + let array_data_type = args[0].data_type(); + match array_data_type { + DataType::List(_) => { + let array = as_list_array(&args[0])?; + general_pop_back_list::(array) + } + DataType::LargeList(_) => { + let array = as_large_list_array(&args[0])?; + general_pop_back_list::(array) + } + _ => exec_err!( + "array_pop_back does not support type: {:?}", + array_data_type + ), + } } /// Appends or prepends elements to a ListArray. @@ -882,20 +938,6 @@ pub fn gen_range(args: &[ArrayRef]) -> Result { Ok(arr) } -/// array_pop_front SQL function -pub fn array_pop_front(args: &[ArrayRef]) -> Result { - let list_array = as_list_array(&args[0])?; - let from_array = Int64Array::from(vec![2; list_array.len()]); - let to_array = Int64Array::from( - list_array - .iter() - .map(|arr| arr.map_or(0, |arr| arr.len() as i64)) - .collect::>(), - ); - let args = vec![args[0].clone(), Arc::new(from_array), Arc::new(to_array)]; - array_slice(args.as_slice()) -} - /// Array_append SQL function pub fn array_append(args: &[ArrayRef]) -> Result { if args.len() != 2 { diff --git a/datafusion/sqllogictest/test_files/array.slt b/datafusion/sqllogictest/test_files/array.slt index 4c4adbabfda5..b8d89edb49b1 100644 --- a/datafusion/sqllogictest/test_files/array.slt +++ b/datafusion/sqllogictest/test_files/array.slt @@ -994,18 +994,33 @@ select array_pop_back(make_array(1, 2, 3, 4, 5)), array_pop_back(make_array('h', ---- [1, 2, 3, 4] [h, e, l, l] +query ?? +select array_pop_back(arrow_cast(make_array(1, 2, 3, 4, 5), 'LargeList(Int64)')), array_pop_back(arrow_cast(make_array('h', 'e', 'l', 'l', 'o'), 'LargeList(Utf8)')); +---- +[1, 2, 3, 4] [h, e, l, l] + # array_pop_back scalar function #2 (after array_pop_back, array is empty) query ? select array_pop_back(make_array(1)); ---- [] +query ? +select array_pop_back(arrow_cast(make_array(1), 'LargeList(Int64)')); +---- +[] + # array_pop_back scalar function #3 (array_pop_back the empty array) query ? select array_pop_back(array_pop_back(make_array(1))); ---- [] +query ? +select array_pop_back(array_pop_back(arrow_cast(make_array(1), 'LargeList(Int64)'))); +---- +[] + # array_pop_back scalar function #4 (array_pop_back the arrays which have NULL) query ?? select array_pop_back(make_array(1, 2, 3, 4, NULL)), array_pop_back(make_array(NULL, 'e', 'l', NULL, 'o')); @@ -1018,24 +1033,44 @@ select array_pop_back(make_array(make_array(1, 2, 3), make_array(2, 9, 1), make_ ---- [[1, 2, 3], [2, 9, 1], [7, 8, 9], [1, 2, 3], [1, 7, 4]] +query ? +select array_pop_back(arrow_cast(make_array(make_array(1, 2, 3), make_array(2, 9, 1), make_array(7, 8, 9), make_array(1, 2, 3), make_array(1, 7, 4), make_array(4, 5, 6)), 'LargeList(List(Int64))')); +---- +[[1, 2, 3], [2, 9, 1], [7, 8, 9], [1, 2, 3], [1, 7, 4]] + # array_pop_back scalar function #6 (array_pop_back the nested arrays with NULL) query ? select array_pop_back(make_array(make_array(1, 2, 3), make_array(2, 9, 1), make_array(7, 8, 9), make_array(1, 2, 3), make_array(1, 7, 4), NULL)); ---- [[1, 2, 3], [2, 9, 1], [7, 8, 9], [1, 2, 3], [1, 7, 4]] +query ? +select array_pop_back(arrow_cast(make_array(make_array(1, 2, 3), make_array(2, 9, 1), make_array(7, 8, 9), make_array(1, 2, 3), make_array(1, 7, 4), NULL), 'LargeList(List(Int64))')); +---- +[[1, 2, 3], [2, 9, 1], [7, 8, 9], [1, 2, 3], [1, 7, 4]] + # array_pop_back scalar function #7 (array_pop_back the nested arrays with NULL) query ? select array_pop_back(make_array(make_array(1, 2, 3), make_array(2, 9, 1), make_array(7, 8, 9), NULL, make_array(1, 7, 4))); ---- [[1, 2, 3], [2, 9, 1], [7, 8, 9], ] +query ? +select array_pop_back(arrow_cast(make_array(make_array(1, 2, 3), make_array(2, 9, 1), make_array(7, 8, 9), NULL, make_array(1, 7, 4)), 'LargeList(List(Int64))')); +---- +[[1, 2, 3], [2, 9, 1], [7, 8, 9], ] + # array_pop_back scalar function #8 (after array_pop_back, nested array is empty) query ? select array_pop_back(make_array(make_array(1, 2, 3))); ---- [] +query ? +select array_pop_back(arrow_cast(make_array(make_array(1, 2, 3)), 'LargeList(List(Int64))')); +---- +[] + # array_pop_back with columns query ? select array_pop_back(column1) from arrayspop; @@ -1047,6 +1082,16 @@ select array_pop_back(column1) from arrayspop; [] [, 10, 11] +query ? +select array_pop_back(arrow_cast(column1, 'LargeList(Int64)')) from arrayspop; +---- +[1, 2] +[3, 4, 5] +[6, 7, 8, ] +[, ] +[] +[, 10, 11] + ## array_pop_front (aliases: `list_pop_front`) # array_pop_front scalar function #1 @@ -1055,36 +1100,66 @@ select array_pop_front(make_array(1, 2, 3, 4, 5)), array_pop_front(make_array('h ---- [2, 3, 4, 5] [e, l, l, o] +query ?? +select array_pop_front(arrow_cast(make_array(1, 2, 3, 4, 5), 'LargeList(Int64)')), array_pop_front(arrow_cast(make_array('h', 'e', 'l', 'l', 'o'), 'LargeList(Utf8)')); +---- +[2, 3, 4, 5] [e, l, l, o] + # array_pop_front scalar function #2 (after array_pop_front, array is empty) query ? select array_pop_front(make_array(1)); ---- [] +query ? +select array_pop_front(arrow_cast(make_array(1), 'LargeList(Int64)')); +---- +[] + # array_pop_front scalar function #3 (array_pop_front the empty array) query ? select array_pop_front(array_pop_front(make_array(1))); ---- [] +query ? +select array_pop_front(array_pop_front(arrow_cast(make_array(1), 'LargeList(Int64)'))); +---- +[] + # array_pop_front scalar function #5 (array_pop_front the nested arrays) query ? select array_pop_front(make_array(make_array(1, 2, 3), make_array(2, 9, 1), make_array(7, 8, 9), make_array(1, 2, 3), make_array(1, 7, 4), make_array(4, 5, 6))); ---- [[2, 9, 1], [7, 8, 9], [1, 2, 3], [1, 7, 4], [4, 5, 6]] +query ? +select array_pop_front(arrow_cast(make_array(make_array(1, 2, 3), make_array(2, 9, 1), make_array(7, 8, 9), make_array(1, 2, 3), make_array(1, 7, 4), make_array(4, 5, 6)), 'LargeList(List(Int64))')); +---- +[[2, 9, 1], [7, 8, 9], [1, 2, 3], [1, 7, 4], [4, 5, 6]] + # array_pop_front scalar function #6 (array_pop_front the nested arrays with NULL) query ? select array_pop_front(make_array(NULL, make_array(1, 2, 3), make_array(2, 9, 1), make_array(7, 8, 9), make_array(1, 2, 3), make_array(1, 7, 4))); ---- [[1, 2, 3], [2, 9, 1], [7, 8, 9], [1, 2, 3], [1, 7, 4]] +query ? +select array_pop_front(arrow_cast(make_array(NULL, make_array(1, 2, 3), make_array(2, 9, 1), make_array(7, 8, 9), make_array(1, 2, 3), make_array(1, 7, 4)), 'LargeList(List(Int64))')); +---- +[[1, 2, 3], [2, 9, 1], [7, 8, 9], [1, 2, 3], [1, 7, 4]] + # array_pop_front scalar function #8 (after array_pop_front, nested array is empty) query ? select array_pop_front(make_array(make_array(1, 2, 3))); ---- [] +query ? +select array_pop_front(arrow_cast(make_array(make_array(1, 2, 3)), 'LargeList(List(Int64))')); +---- +[] + ## array_slice (aliases: list_slice) # array_slice scalar function #1 (with positive indexes) From 673f0e17ace7e7a08474c26be50038cf0e251477 Mon Sep 17 00:00:00 2001 From: Ruixiang Tan Date: Fri, 29 Dec 2023 19:27:39 +0800 Subject: [PATCH 512/572] chore: rename ceresdb to apache horaedb (#8674) --- docs/source/user-guide/introduction.md | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/docs/source/user-guide/introduction.md b/docs/source/user-guide/introduction.md index 6c1e54c2b701..b737c3bab266 100644 --- a/docs/source/user-guide/introduction.md +++ b/docs/source/user-guide/introduction.md @@ -75,7 +75,7 @@ latency). Here are some example systems built using DataFusion: -- Specialized Analytical Database systems such as [CeresDB] and more general Apache Spark like system such a [Ballista]. +- Specialized Analytical Database systems such as [HoraeDB] and more general Apache Spark like system such a [Ballista]. - New query language engines such as [prql-query] and accelerators such as [VegaFusion] - Research platform for new Database Systems, such as [Flock] - SQL support to another library, such as [dask sql] @@ -96,7 +96,6 @@ Here are some active projects using DataFusion: - [Arroyo](https://github.com/ArroyoSystems/arroyo) Distributed stream processing engine in Rust - [Ballista](https://github.com/apache/arrow-ballista) Distributed SQL Query Engine -- [CeresDB](https://github.com/CeresDB/ceresdb) Distributed Time-Series Database - [CnosDB](https://github.com/cnosdb/cnosdb) Open Source Distributed Time Series Database - [Cube Store](https://github.com/cube-js/cube.js/tree/master/rust) - [Dask SQL](https://github.com/dask-contrib/dask-sql) Distributed SQL query engine in Python @@ -104,6 +103,7 @@ Here are some active projects using DataFusion: - [delta-rs](https://github.com/delta-io/delta-rs) Native Rust implementation of Delta Lake - [GreptimeDB](https://github.com/GreptimeTeam/greptimedb) Open Source & Cloud Native Distributed Time Series Database - [GlareDB](https://github.com/GlareDB/glaredb) Fast SQL database for querying and analyzing distributed data. +- [HoraeDB](https://github.com/apache/incubator-horaedb) Distributed Time-Series Database - [InfluxDB IOx](https://github.com/influxdata/influxdb_iox) Time Series Database - [Kamu](https://github.com/kamu-data/kamu-cli/) Planet-scale streaming data pipeline - [LakeSoul](https://github.com/lakesoul-io/LakeSoul) Open source LakeHouse framework with native IO in Rust. @@ -128,7 +128,6 @@ Here are some less active projects that used DataFusion: [ballista]: https://github.com/apache/arrow-ballista [blaze]: https://github.com/blaze-init/blaze -[ceresdb]: https://github.com/CeresDB/ceresdb [cloudfuse buzz]: https://github.com/cloudfuse-io/buzz-rust [cnosdb]: https://github.com/cnosdb/cnosdb [cube store]: https://github.com/cube-js/cube.js/tree/master/rust @@ -138,6 +137,7 @@ Here are some less active projects that used DataFusion: [flock]: https://github.com/flock-lab/flock [kamu]: https://github.com/kamu-data/kamu-cli [greptime db]: https://github.com/GreptimeTeam/greptimedb +[horaedb]: https://github.com/apache/incubator-horaedb [influxdb iox]: https://github.com/influxdata/influxdb_iox [parseable]: https://github.com/parseablehq/parseable [prql-query]: https://github.com/prql/prql-query From d515c68da6e9795271c54a2f4b7853ca25cc90da Mon Sep 17 00:00:00 2001 From: Alex Huang Date: Fri, 29 Dec 2023 12:44:07 +0100 Subject: [PATCH 513/572] clean code (#8671) --- datafusion/proto/src/logical_plan/mod.rs | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/datafusion/proto/src/logical_plan/mod.rs b/datafusion/proto/src/logical_plan/mod.rs index e997bcde426e..dbed0252d051 100644 --- a/datafusion/proto/src/logical_plan/mod.rs +++ b/datafusion/proto/src/logical_plan/mod.rs @@ -1765,8 +1765,8 @@ pub(crate) fn writer_properties_to_proto( pub(crate) fn writer_properties_from_proto( props: &protobuf::WriterProperties, ) -> Result { - let writer_version = WriterVersion::from_str(&props.writer_version) - .map_err(|e| proto_error(e.to_string()))?; + let writer_version = + WriterVersion::from_str(&props.writer_version).map_err(proto_error)?; Ok(WriterProperties::builder() .set_created_by(props.created_by.clone()) .set_writer_version(writer_version) From 8ced56e418a50456cc8193547683bfcceb063f0d Mon Sep 17 00:00:00 2001 From: Eduard Karacharov <13005055+korowa@users.noreply.github.com> Date: Fri, 29 Dec 2023 14:37:25 +0200 Subject: [PATCH 514/572] remove tz with modified offset from tests (#8677) --- datafusion/sqllogictest/test_files/timestamps.slt | 3 --- 1 file changed, 3 deletions(-) diff --git a/datafusion/sqllogictest/test_files/timestamps.slt b/datafusion/sqllogictest/test_files/timestamps.slt index 2b3b4bf2e45b..c84e46c965fa 100644 --- a/datafusion/sqllogictest/test_files/timestamps.slt +++ b/datafusion/sqllogictest/test_files/timestamps.slt @@ -1730,14 +1730,11 @@ SELECT TIMESTAMPTZ '2022-01-01 01:10:00 AEST' query P rowsort SELECT TIMESTAMPTZ '2022-01-01 01:10:00 Australia/Sydney' as ts_geo UNION ALL -SELECT TIMESTAMPTZ '2022-01-01 01:10:00 Antarctica/Vostok' as ts_geo - UNION ALL SELECT TIMESTAMPTZ '2022-01-01 01:10:00 Africa/Johannesburg' as ts_geo UNION ALL SELECT TIMESTAMPTZ '2022-01-01 01:10:00 America/Los_Angeles' as ts_geo ---- 2021-12-31T14:10:00Z -2021-12-31T19:10:00Z 2021-12-31T23:10:00Z 2022-01-01T09:10:00Z From b85a39739e754576723ff4b1691c518a86335769 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Metehan=20Y=C4=B1ld=C4=B1r=C4=B1m?= <100111937+metesynnada@users.noreply.github.com> Date: Fri, 29 Dec 2023 15:51:02 +0300 Subject: [PATCH 515/572] Make the BatchSerializer behind Arc to avoid unnecessary struct creation (#8666) * Make the BatchSerializer behind Arc * Commenting * Review * Incorporate review suggestions * Use old names --------- Co-authored-by: Mehmet Ozan Kabak --- .../core/src/datasource/file_format/csv.rs | 69 +++++++---------- .../core/src/datasource/file_format/json.rs | 77 ++++++++----------- .../src/datasource/file_format/write/mod.rs | 16 +--- .../file_format/write/orchestration.rs | 74 ++++++++---------- .../datasource/physical_plan/file_stream.rs | 12 ++- 5 files changed, 98 insertions(+), 150 deletions(-) diff --git a/datafusion/core/src/datasource/file_format/csv.rs b/datafusion/core/src/datasource/file_format/csv.rs index 4033bcd3b557..d4e63904bdd4 100644 --- a/datafusion/core/src/datasource/file_format/csv.rs +++ b/datafusion/core/src/datasource/file_format/csv.rs @@ -19,21 +19,9 @@ use std::any::Any; use std::collections::HashSet; -use std::fmt; -use std::fmt::Debug; +use std::fmt::{self, Debug}; use std::sync::Arc; -use arrow_array::RecordBatch; -use datafusion_common::{exec_err, not_impl_err, DataFusionError, FileType}; -use datafusion_execution::TaskContext; -use datafusion_physical_expr::{PhysicalExpr, PhysicalSortRequirement}; - -use bytes::{Buf, Bytes}; -use datafusion_physical_plan::metrics::MetricsSet; -use futures::stream::BoxStream; -use futures::{pin_mut, Stream, StreamExt, TryStreamExt}; -use object_store::{delimited::newline_delimited_stream, ObjectMeta, ObjectStore}; - use super::write::orchestration::stateless_multipart_put; use super::{FileFormat, DEFAULT_SCHEMA_INFER_MAX_RECORD}; use crate::datasource::file_format::file_compression_type::FileCompressionType; @@ -47,11 +35,20 @@ use crate::physical_plan::insert::{DataSink, FileSinkExec}; use crate::physical_plan::{DisplayAs, DisplayFormatType, Statistics}; use crate::physical_plan::{ExecutionPlan, SendableRecordBatchStream}; +use arrow::array::RecordBatch; use arrow::csv::WriterBuilder; use arrow::datatypes::{DataType, Field, Fields, Schema}; use arrow::{self, datatypes::SchemaRef}; +use datafusion_common::{exec_err, not_impl_err, DataFusionError, FileType}; +use datafusion_execution::TaskContext; +use datafusion_physical_expr::{PhysicalExpr, PhysicalSortRequirement}; +use datafusion_physical_plan::metrics::MetricsSet; use async_trait::async_trait; +use bytes::{Buf, Bytes}; +use futures::stream::BoxStream; +use futures::{pin_mut, Stream, StreamExt, TryStreamExt}; +use object_store::{delimited::newline_delimited_stream, ObjectMeta, ObjectStore}; /// Character Separated Value `FileFormat` implementation. #[derive(Debug)] @@ -400,8 +397,6 @@ impl Default for CsvSerializer { pub struct CsvSerializer { // CSV writer builder builder: WriterBuilder, - // Inner buffer for avoiding reallocation - buffer: Vec, // Flag to indicate whether there will be a header header: bool, } @@ -412,7 +407,6 @@ impl CsvSerializer { Self { builder: WriterBuilder::new(), header: true, - buffer: Vec::with_capacity(4096), } } @@ -431,21 +425,14 @@ impl CsvSerializer { #[async_trait] impl BatchSerializer for CsvSerializer { - async fn serialize(&mut self, batch: RecordBatch) -> Result { + async fn serialize(&self, batch: RecordBatch, initial: bool) -> Result { + let mut buffer = Vec::with_capacity(4096); let builder = self.builder.clone(); - let mut writer = builder.with_header(self.header).build(&mut self.buffer); + let header = self.header && initial; + let mut writer = builder.with_header(header).build(&mut buffer); writer.write(&batch)?; drop(writer); - self.header = false; - Ok(Bytes::from(self.buffer.drain(..).collect::>())) - } - - fn duplicate(&mut self) -> Result> { - let new_self = CsvSerializer::new() - .with_builder(self.builder.clone()) - .with_header(self.header); - self.header = false; - Ok(Box::new(new_self)) + Ok(Bytes::from(buffer)) } } @@ -488,13 +475,11 @@ impl CsvSink { let builder_clone = builder.clone(); let options_clone = writer_options.clone(); let get_serializer = move || { - let inner_clone = builder_clone.clone(); - let serializer: Box = Box::new( + Arc::new( CsvSerializer::new() - .with_builder(inner_clone) + .with_builder(builder_clone.clone()) .with_header(options_clone.writer_options.header()), - ); - serializer + ) as _ }; stateless_multipart_put( @@ -541,15 +526,15 @@ mod tests { use crate::physical_plan::collect; use crate::prelude::{CsvReadOptions, SessionConfig, SessionContext}; use crate::test_util::arrow_test_data; + use arrow::compute::concat_batches; - use bytes::Bytes; - use chrono::DateTime; use datafusion_common::cast::as_string_array; - use datafusion_common::internal_err; use datafusion_common::stats::Precision; - use datafusion_common::FileType; - use datafusion_common::GetExt; + use datafusion_common::{internal_err, FileType, GetExt}; use datafusion_expr::{col, lit}; + + use bytes::Bytes; + use chrono::DateTime; use futures::StreamExt; use object_store::local::LocalFileSystem; use object_store::path::Path; @@ -836,8 +821,8 @@ mod tests { .collect() .await?; let batch = concat_batches(&batches[0].schema(), &batches)?; - let mut serializer = CsvSerializer::new(); - let bytes = serializer.serialize(batch).await?; + let serializer = CsvSerializer::new(); + let bytes = serializer.serialize(batch, true).await?; assert_eq!( "c2,c3\n2,1\n5,-40\n1,29\n1,-85\n5,-82\n4,-111\n3,104\n3,13\n1,38\n4,-38\n", String::from_utf8(bytes.into()).unwrap() @@ -860,8 +845,8 @@ mod tests { .collect() .await?; let batch = concat_batches(&batches[0].schema(), &batches)?; - let mut serializer = CsvSerializer::new().with_header(false); - let bytes = serializer.serialize(batch).await?; + let serializer = CsvSerializer::new().with_header(false); + let bytes = serializer.serialize(batch, true).await?; assert_eq!( "2,1\n5,-40\n1,29\n1,-85\n5,-82\n4,-111\n3,104\n3,13\n1,38\n4,-38\n", String::from_utf8(bytes.into()).unwrap() diff --git a/datafusion/core/src/datasource/file_format/json.rs b/datafusion/core/src/datasource/file_format/json.rs index fcb1d5f8e527..3d437bc5fe68 100644 --- a/datafusion/core/src/datasource/file_format/json.rs +++ b/datafusion/core/src/datasource/file_format/json.rs @@ -23,40 +23,34 @@ use std::fmt::Debug; use std::io::BufReader; use std::sync::Arc; -use super::{FileFormat, FileScanConfig}; -use arrow::datatypes::Schema; -use arrow::datatypes::SchemaRef; -use arrow::json; -use arrow::json::reader::infer_json_schema_from_iterator; -use arrow::json::reader::ValueIter; -use arrow_array::RecordBatch; -use async_trait::async_trait; -use bytes::Buf; - -use bytes::Bytes; -use datafusion_physical_expr::PhysicalExpr; -use datafusion_physical_expr::PhysicalSortRequirement; -use datafusion_physical_plan::ExecutionPlan; -use object_store::{GetResultPayload, ObjectMeta, ObjectStore}; - -use crate::datasource::physical_plan::FileGroupDisplay; -use crate::physical_plan::insert::DataSink; -use crate::physical_plan::insert::FileSinkExec; -use crate::physical_plan::SendableRecordBatchStream; -use crate::physical_plan::{DisplayAs, DisplayFormatType, Statistics}; - use super::write::orchestration::stateless_multipart_put; - +use super::{FileFormat, FileScanConfig}; use crate::datasource::file_format::file_compression_type::FileCompressionType; use crate::datasource::file_format::write::BatchSerializer; use crate::datasource::file_format::DEFAULT_SCHEMA_INFER_MAX_RECORD; +use crate::datasource::physical_plan::FileGroupDisplay; use crate::datasource::physical_plan::{FileSinkConfig, NdJsonExec}; use crate::error::Result; use crate::execution::context::SessionState; +use crate::physical_plan::insert::{DataSink, FileSinkExec}; +use crate::physical_plan::{ + DisplayAs, DisplayFormatType, SendableRecordBatchStream, Statistics, +}; +use arrow::datatypes::Schema; +use arrow::datatypes::SchemaRef; +use arrow::json; +use arrow::json::reader::{infer_json_schema_from_iterator, ValueIter}; +use arrow_array::RecordBatch; use datafusion_common::{not_impl_err, DataFusionError, FileType}; use datafusion_execution::TaskContext; +use datafusion_physical_expr::{PhysicalExpr, PhysicalSortRequirement}; use datafusion_physical_plan::metrics::MetricsSet; +use datafusion_physical_plan::ExecutionPlan; + +use async_trait::async_trait; +use bytes::{Buf, Bytes}; +use object_store::{GetResultPayload, ObjectMeta, ObjectStore}; /// New line delimited JSON `FileFormat` implementation. #[derive(Debug)] @@ -201,31 +195,22 @@ impl Default for JsonSerializer { } /// Define a struct for serializing Json records to a stream -pub struct JsonSerializer { - // Inner buffer for avoiding reallocation - buffer: Vec, -} +pub struct JsonSerializer {} impl JsonSerializer { /// Constructor for the JsonSerializer object pub fn new() -> Self { - Self { - buffer: Vec::with_capacity(4096), - } + Self {} } } #[async_trait] impl BatchSerializer for JsonSerializer { - async fn serialize(&mut self, batch: RecordBatch) -> Result { - let mut writer = json::LineDelimitedWriter::new(&mut self.buffer); + async fn serialize(&self, batch: RecordBatch, _initial: bool) -> Result { + let mut buffer = Vec::with_capacity(4096); + let mut writer = json::LineDelimitedWriter::new(&mut buffer); writer.write(&batch)?; - //drop(writer); - Ok(Bytes::from(self.buffer.drain(..).collect::>())) - } - - fn duplicate(&mut self) -> Result> { - Ok(Box::new(JsonSerializer::new())) + Ok(Bytes::from(buffer)) } } @@ -272,10 +257,7 @@ impl JsonSink { let writer_options = self.config.file_type_writer_options.try_into_json()?; let compression = &writer_options.compression; - let get_serializer = move || { - let serializer: Box = Box::new(JsonSerializer::new()); - serializer - }; + let get_serializer = move || Arc::new(JsonSerializer::new()) as _; stateless_multipart_put( data, @@ -312,16 +294,17 @@ impl DataSink for JsonSink { #[cfg(test)] mod tests { use super::super::test_util::scan_format; - use datafusion_common::cast::as_int64_array; - use datafusion_common::stats::Precision; - use futures::StreamExt; - use object_store::local::LocalFileSystem; - use super::*; use crate::physical_plan::collect; use crate::prelude::{SessionConfig, SessionContext}; use crate::test::object_store::local_unpartitioned_file; + use datafusion_common::cast::as_int64_array; + use datafusion_common::stats::Precision; + + use futures::StreamExt; + use object_store::local::LocalFileSystem; + #[tokio::test] async fn read_small_batches() -> Result<()> { let config = SessionConfig::new().with_batch_size(2); diff --git a/datafusion/core/src/datasource/file_format/write/mod.rs b/datafusion/core/src/datasource/file_format/write/mod.rs index 68fe81ce91fa..c481f2accf19 100644 --- a/datafusion/core/src/datasource/file_format/write/mod.rs +++ b/datafusion/core/src/datasource/file_format/write/mod.rs @@ -24,20 +24,16 @@ use std::sync::Arc; use std::task::{Context, Poll}; use crate::datasource::file_format::file_compression_type::FileCompressionType; - use crate::error::Result; use arrow_array::RecordBatch; - use datafusion_common::DataFusionError; use async_trait::async_trait; use bytes::Bytes; - use futures::future::BoxFuture; use object_store::path::Path; use object_store::{MultipartId, ObjectStore}; - use tokio::io::AsyncWrite; pub(crate) mod demux; @@ -149,15 +145,11 @@ impl AsyncWrite for AbortableWrite { /// A trait that defines the methods required for a RecordBatch serializer. #[async_trait] -pub trait BatchSerializer: Unpin + Send { +pub trait BatchSerializer: Sync + Send { /// Asynchronously serializes a `RecordBatch` and returns the serialized bytes. - async fn serialize(&mut self, batch: RecordBatch) -> Result; - /// Duplicates self to support serializing multiple batches in parallel on multiple cores - fn duplicate(&mut self) -> Result> { - Err(DataFusionError::NotImplemented( - "Parallel serialization is not implemented for this file type".into(), - )) - } + /// Parameter `initial` signals whether the given batch is the first batch. + /// This distinction is important for certain serializers (like CSV). + async fn serialize(&self, batch: RecordBatch, initial: bool) -> Result; } /// Returns an [`AbortableWrite`] which writes to the given object store location diff --git a/datafusion/core/src/datasource/file_format/write/orchestration.rs b/datafusion/core/src/datasource/file_format/write/orchestration.rs index 120e27ecf669..9b820a15b280 100644 --- a/datafusion/core/src/datasource/file_format/write/orchestration.rs +++ b/datafusion/core/src/datasource/file_format/write/orchestration.rs @@ -21,28 +21,25 @@ use std::sync::Arc; +use super::demux::start_demuxer_task; +use super::{create_writer, AbortableWrite, BatchSerializer}; use crate::datasource::file_format::file_compression_type::FileCompressionType; use crate::datasource::physical_plan::FileSinkConfig; use crate::error::Result; use crate::physical_plan::SendableRecordBatchStream; use arrow_array::RecordBatch; - -use datafusion_common::DataFusionError; - -use bytes::Bytes; +use datafusion_common::{internal_datafusion_err, internal_err, DataFusionError}; use datafusion_execution::TaskContext; +use bytes::Bytes; use tokio::io::{AsyncWrite, AsyncWriteExt}; use tokio::sync::mpsc::{self, Receiver}; use tokio::task::{JoinHandle, JoinSet}; use tokio::try_join; -use super::demux::start_demuxer_task; -use super::{create_writer, AbortableWrite, BatchSerializer}; - type WriterType = AbortableWrite>; -type SerializerType = Box; +type SerializerType = Arc; /// Serializes a single data stream in parallel and writes to an ObjectStore /// concurrently. Data order is preserved. In the event of an error, @@ -50,33 +47,28 @@ type SerializerType = Box; /// so that the caller may handle aborting failed writes. pub(crate) async fn serialize_rb_stream_to_object_store( mut data_rx: Receiver, - mut serializer: Box, + serializer: Arc, mut writer: AbortableWrite>, ) -> std::result::Result<(WriterType, u64), (WriterType, DataFusionError)> { let (tx, mut rx) = mpsc::channel::>>(100); - let serialize_task = tokio::spawn(async move { + // Some serializers (like CSV) handle the first batch differently than + // subsequent batches, so we track that here. + let mut initial = true; while let Some(batch) = data_rx.recv().await { - match serializer.duplicate() { - Ok(mut serializer_clone) => { - let handle = tokio::spawn(async move { - let num_rows = batch.num_rows(); - let bytes = serializer_clone.serialize(batch).await?; - Ok((num_rows, bytes)) - }); - tx.send(handle).await.map_err(|_| { - DataFusionError::Internal( - "Unknown error writing to object store".into(), - ) - })?; - } - Err(_) => { - return Err(DataFusionError::Internal( - "Unknown error writing to object store".into(), - )) - } + let serializer_clone = serializer.clone(); + let handle = tokio::spawn(async move { + let num_rows = batch.num_rows(); + let bytes = serializer_clone.serialize(batch, initial).await?; + Ok((num_rows, bytes)) + }); + if initial { + initial = false; } + tx.send(handle).await.map_err(|_| { + internal_datafusion_err!("Unknown error writing to object store") + })?; } Ok(()) }); @@ -120,7 +112,7 @@ pub(crate) async fn serialize_rb_stream_to_object_store( Err(_) => { return Err(( writer, - DataFusionError::Internal("Unknown error writing to object store".into()), + internal_datafusion_err!("Unknown error writing to object store"), )) } }; @@ -171,9 +163,9 @@ pub(crate) async fn stateless_serialize_and_write_files( // this thread, so we cannot clean it up (hence any_abort_errors is true) any_errors = true; any_abort_errors = true; - triggering_error = Some(DataFusionError::Internal(format!( + triggering_error = Some(internal_datafusion_err!( "Unexpected join error while serializing file {e}" - ))); + )); } } } @@ -190,24 +182,24 @@ pub(crate) async fn stateless_serialize_and_write_files( false => { writer.shutdown() .await - .map_err(|_| DataFusionError::Internal("Error encountered while finalizing writes! Partial results may have been written to ObjectStore!".into()))?; + .map_err(|_| internal_datafusion_err!("Error encountered while finalizing writes! Partial results may have been written to ObjectStore!"))?; } } } if any_errors { match any_abort_errors{ - true => return Err(DataFusionError::Internal("Error encountered during writing to ObjectStore and failed to abort all writers. Partial result may have been written.".into())), + true => return internal_err!("Error encountered during writing to ObjectStore and failed to abort all writers. Partial result may have been written."), false => match triggering_error { Some(e) => return Err(e), - None => return Err(DataFusionError::Internal("Unknown Error encountered during writing to ObjectStore. All writers succesfully aborted.".into())) + None => return internal_err!("Unknown Error encountered during writing to ObjectStore. All writers succesfully aborted.") } } } tx.send(row_count).map_err(|_| { - DataFusionError::Internal( - "Error encountered while sending row count back to file sink!".into(), + internal_datafusion_err!( + "Error encountered while sending row count back to file sink!" ) })?; Ok(()) @@ -220,7 +212,7 @@ pub(crate) async fn stateless_multipart_put( data: SendableRecordBatchStream, context: &Arc, file_extension: String, - get_serializer: Box Box + Send>, + get_serializer: Box Arc + Send>, config: &FileSinkConfig, compression: FileCompressionType, ) -> Result { @@ -264,8 +256,8 @@ pub(crate) async fn stateless_multipart_put( .send((rb_stream, serializer, writer)) .await .map_err(|_| { - DataFusionError::Internal( - "Writer receive file bundle channel closed unexpectedly!".into(), + internal_datafusion_err!( + "Writer receive file bundle channel closed unexpectedly!" ) })?; } @@ -288,9 +280,7 @@ pub(crate) async fn stateless_multipart_put( } let total_count = rx_row_cnt.await.map_err(|_| { - DataFusionError::Internal( - "Did not receieve row count from write coordinater".into(), - ) + internal_datafusion_err!("Did not receieve row count from write coordinater") })?; Ok(total_count) diff --git a/datafusion/core/src/datasource/physical_plan/file_stream.rs b/datafusion/core/src/datasource/physical_plan/file_stream.rs index 99fb088b66f4..bb4c8313642c 100644 --- a/datafusion/core/src/datasource/physical_plan/file_stream.rs +++ b/datafusion/core/src/datasource/physical_plan/file_stream.rs @@ -518,10 +518,8 @@ impl RecordBatchStream for FileStream { #[cfg(test)] mod tests { - use arrow_schema::Schema; - use datafusion_common::internal_err; - use datafusion_common::DataFusionError; - use datafusion_common::Statistics; + use std::sync::atomic::{AtomicUsize, Ordering}; + use std::sync::Arc; use super::*; use crate::datasource::file_format::write::BatchSerializer; @@ -534,8 +532,8 @@ mod tests { test::{make_partition, object_store::register_test_store}, }; - use std::sync::atomic::{AtomicUsize, Ordering}; - use std::sync::Arc; + use arrow_schema::Schema; + use datafusion_common::{internal_err, DataFusionError, Statistics}; use async_trait::async_trait; use bytes::Bytes; @@ -993,7 +991,7 @@ mod tests { #[async_trait] impl BatchSerializer for TestSerializer { - async fn serialize(&mut self, _batch: RecordBatch) -> Result { + async fn serialize(&self, _batch: RecordBatch, _initial: bool) -> Result { Ok(self.bytes.clone()) } } From 7fc663c2e40be2928778102386bbf76962dd2cdc Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Fri, 29 Dec 2023 16:53:31 -0700 Subject: [PATCH 516/572] Implement serde for CSV and Parquet FileSinkExec (#8646) * Add serde for Csv and Parquet sink * Add tests * parquet test passes * save progress * add compression type to csv serde * remove hard-coded compression from CSV serde --- .../core/src/datasource/file_format/csv.rs | 11 +- .../src/datasource/file_format/parquet.rs | 9 +- datafusion/proto/proto/datafusion.proto | 40 +- datafusion/proto/src/generated/pbjson.rs | 517 ++++++++++++++++++ datafusion/proto/src/generated/prost.rs | 59 +- datafusion/proto/src/logical_plan/mod.rs | 43 +- .../proto/src/physical_plan/from_proto.rs | 38 +- datafusion/proto/src/physical_plan/mod.rs | 91 +++ .../proto/src/physical_plan/to_proto.rs | 46 +- .../tests/cases/roundtrip_physical_plan.rs | 125 ++++- 10 files changed, 922 insertions(+), 57 deletions(-) diff --git a/datafusion/core/src/datasource/file_format/csv.rs b/datafusion/core/src/datasource/file_format/csv.rs index d4e63904bdd4..7a0af3ff0809 100644 --- a/datafusion/core/src/datasource/file_format/csv.rs +++ b/datafusion/core/src/datasource/file_format/csv.rs @@ -437,7 +437,7 @@ impl BatchSerializer for CsvSerializer { } /// Implements [`DataSink`] for writing to a CSV file. -struct CsvSink { +pub struct CsvSink { /// Config options for writing data config: FileSinkConfig, } @@ -461,9 +461,16 @@ impl DisplayAs for CsvSink { } impl CsvSink { - fn new(config: FileSinkConfig) -> Self { + /// Create from config. + pub fn new(config: FileSinkConfig) -> Self { Self { config } } + + /// Retrieve the inner [`FileSinkConfig`]. + pub fn config(&self) -> &FileSinkConfig { + &self.config + } + async fn multipartput_all( &self, data: SendableRecordBatchStream, diff --git a/datafusion/core/src/datasource/file_format/parquet.rs b/datafusion/core/src/datasource/file_format/parquet.rs index 7044acccd6dc..9729bfa163af 100644 --- a/datafusion/core/src/datasource/file_format/parquet.rs +++ b/datafusion/core/src/datasource/file_format/parquet.rs @@ -621,7 +621,7 @@ async fn fetch_statistics( } /// Implements [`DataSink`] for writing to a parquet file. -struct ParquetSink { +pub struct ParquetSink { /// Config options for writing data config: FileSinkConfig, } @@ -645,10 +645,15 @@ impl DisplayAs for ParquetSink { } impl ParquetSink { - fn new(config: FileSinkConfig) -> Self { + /// Create from config. + pub fn new(config: FileSinkConfig) -> Self { Self { config } } + /// Retrieve the inner [`FileSinkConfig`]. + pub fn config(&self) -> &FileSinkConfig { + &self.config + } /// Converts table schema to writer schema, which may differ in the case /// of hive style partitioning where some columns are removed from the /// underlying files. diff --git a/datafusion/proto/proto/datafusion.proto b/datafusion/proto/proto/datafusion.proto index 59b82efcbb43..d5f8397aa30c 100644 --- a/datafusion/proto/proto/datafusion.proto +++ b/datafusion/proto/proto/datafusion.proto @@ -1187,6 +1187,8 @@ message PhysicalPlanNode { SymmetricHashJoinExecNode symmetric_hash_join = 25; InterleaveExecNode interleave = 26; PlaceholderRowExecNode placeholder_row = 27; + CsvSinkExecNode csv_sink = 28; + ParquetSinkExecNode parquet_sink = 29; } } @@ -1220,20 +1222,22 @@ message ParquetWriterOptions { } message CsvWriterOptions { + // Compression type + CompressionTypeVariant compression = 1; // Optional column delimiter. Defaults to `b','` - string delimiter = 1; + string delimiter = 2; // Whether to write column names as file headers. Defaults to `true` - bool has_header = 2; + bool has_header = 3; // Optional date format for date arrays - string date_format = 3; + string date_format = 4; // Optional datetime format for datetime arrays - string datetime_format = 4; + string datetime_format = 5; // Optional timestamp format for timestamp arrays - string timestamp_format = 5; + string timestamp_format = 6; // Optional time format for time arrays - string time_format = 6; + string time_format = 7; // Optional value to represent null - string null_value = 7; + string null_value = 8; } message WriterProperties { @@ -1270,6 +1274,28 @@ message JsonSinkExecNode { PhysicalSortExprNodeCollection sort_order = 4; } +message CsvSink { + FileSinkConfig config = 1; +} + +message CsvSinkExecNode { + PhysicalPlanNode input = 1; + CsvSink sink = 2; + Schema sink_schema = 3; + PhysicalSortExprNodeCollection sort_order = 4; +} + +message ParquetSink { + FileSinkConfig config = 1; +} + +message ParquetSinkExecNode { + PhysicalPlanNode input = 1; + ParquetSink sink = 2; + Schema sink_schema = 3; + PhysicalSortExprNodeCollection sort_order = 4; +} + message PhysicalExtensionNode { bytes node = 1; repeated PhysicalPlanNode inputs = 2; diff --git a/datafusion/proto/src/generated/pbjson.rs b/datafusion/proto/src/generated/pbjson.rs index 956244ffdbc2..12e834d75adf 100644 --- a/datafusion/proto/src/generated/pbjson.rs +++ b/datafusion/proto/src/generated/pbjson.rs @@ -5151,6 +5151,241 @@ impl<'de> serde::Deserialize<'de> for CsvScanExecNode { deserializer.deserialize_struct("datafusion.CsvScanExecNode", FIELDS, GeneratedVisitor) } } +impl serde::Serialize for CsvSink { + #[allow(deprecated)] + fn serialize(&self, serializer: S) -> std::result::Result + where + S: serde::Serializer, + { + use serde::ser::SerializeStruct; + let mut len = 0; + if self.config.is_some() { + len += 1; + } + let mut struct_ser = serializer.serialize_struct("datafusion.CsvSink", len)?; + if let Some(v) = self.config.as_ref() { + struct_ser.serialize_field("config", v)?; + } + struct_ser.end() + } +} +impl<'de> serde::Deserialize<'de> for CsvSink { + #[allow(deprecated)] + fn deserialize(deserializer: D) -> std::result::Result + where + D: serde::Deserializer<'de>, + { + const FIELDS: &[&str] = &[ + "config", + ]; + + #[allow(clippy::enum_variant_names)] + enum GeneratedField { + Config, + } + impl<'de> serde::Deserialize<'de> for GeneratedField { + fn deserialize(deserializer: D) -> std::result::Result + where + D: serde::Deserializer<'de>, + { + struct GeneratedVisitor; + + impl<'de> serde::de::Visitor<'de> for GeneratedVisitor { + type Value = GeneratedField; + + fn expecting(&self, formatter: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + write!(formatter, "expected one of: {:?}", &FIELDS) + } + + #[allow(unused_variables)] + fn visit_str(self, value: &str) -> std::result::Result + where + E: serde::de::Error, + { + match value { + "config" => Ok(GeneratedField::Config), + _ => Err(serde::de::Error::unknown_field(value, FIELDS)), + } + } + } + deserializer.deserialize_identifier(GeneratedVisitor) + } + } + struct GeneratedVisitor; + impl<'de> serde::de::Visitor<'de> for GeneratedVisitor { + type Value = CsvSink; + + fn expecting(&self, formatter: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + formatter.write_str("struct datafusion.CsvSink") + } + + fn visit_map(self, mut map_: V) -> std::result::Result + where + V: serde::de::MapAccess<'de>, + { + let mut config__ = None; + while let Some(k) = map_.next_key()? { + match k { + GeneratedField::Config => { + if config__.is_some() { + return Err(serde::de::Error::duplicate_field("config")); + } + config__ = map_.next_value()?; + } + } + } + Ok(CsvSink { + config: config__, + }) + } + } + deserializer.deserialize_struct("datafusion.CsvSink", FIELDS, GeneratedVisitor) + } +} +impl serde::Serialize for CsvSinkExecNode { + #[allow(deprecated)] + fn serialize(&self, serializer: S) -> std::result::Result + where + S: serde::Serializer, + { + use serde::ser::SerializeStruct; + let mut len = 0; + if self.input.is_some() { + len += 1; + } + if self.sink.is_some() { + len += 1; + } + if self.sink_schema.is_some() { + len += 1; + } + if self.sort_order.is_some() { + len += 1; + } + let mut struct_ser = serializer.serialize_struct("datafusion.CsvSinkExecNode", len)?; + if let Some(v) = self.input.as_ref() { + struct_ser.serialize_field("input", v)?; + } + if let Some(v) = self.sink.as_ref() { + struct_ser.serialize_field("sink", v)?; + } + if let Some(v) = self.sink_schema.as_ref() { + struct_ser.serialize_field("sinkSchema", v)?; + } + if let Some(v) = self.sort_order.as_ref() { + struct_ser.serialize_field("sortOrder", v)?; + } + struct_ser.end() + } +} +impl<'de> serde::Deserialize<'de> for CsvSinkExecNode { + #[allow(deprecated)] + fn deserialize(deserializer: D) -> std::result::Result + where + D: serde::Deserializer<'de>, + { + const FIELDS: &[&str] = &[ + "input", + "sink", + "sink_schema", + "sinkSchema", + "sort_order", + "sortOrder", + ]; + + #[allow(clippy::enum_variant_names)] + enum GeneratedField { + Input, + Sink, + SinkSchema, + SortOrder, + } + impl<'de> serde::Deserialize<'de> for GeneratedField { + fn deserialize(deserializer: D) -> std::result::Result + where + D: serde::Deserializer<'de>, + { + struct GeneratedVisitor; + + impl<'de> serde::de::Visitor<'de> for GeneratedVisitor { + type Value = GeneratedField; + + fn expecting(&self, formatter: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + write!(formatter, "expected one of: {:?}", &FIELDS) + } + + #[allow(unused_variables)] + fn visit_str(self, value: &str) -> std::result::Result + where + E: serde::de::Error, + { + match value { + "input" => Ok(GeneratedField::Input), + "sink" => Ok(GeneratedField::Sink), + "sinkSchema" | "sink_schema" => Ok(GeneratedField::SinkSchema), + "sortOrder" | "sort_order" => Ok(GeneratedField::SortOrder), + _ => Err(serde::de::Error::unknown_field(value, FIELDS)), + } + } + } + deserializer.deserialize_identifier(GeneratedVisitor) + } + } + struct GeneratedVisitor; + impl<'de> serde::de::Visitor<'de> for GeneratedVisitor { + type Value = CsvSinkExecNode; + + fn expecting(&self, formatter: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + formatter.write_str("struct datafusion.CsvSinkExecNode") + } + + fn visit_map(self, mut map_: V) -> std::result::Result + where + V: serde::de::MapAccess<'de>, + { + let mut input__ = None; + let mut sink__ = None; + let mut sink_schema__ = None; + let mut sort_order__ = None; + while let Some(k) = map_.next_key()? { + match k { + GeneratedField::Input => { + if input__.is_some() { + return Err(serde::de::Error::duplicate_field("input")); + } + input__ = map_.next_value()?; + } + GeneratedField::Sink => { + if sink__.is_some() { + return Err(serde::de::Error::duplicate_field("sink")); + } + sink__ = map_.next_value()?; + } + GeneratedField::SinkSchema => { + if sink_schema__.is_some() { + return Err(serde::de::Error::duplicate_field("sinkSchema")); + } + sink_schema__ = map_.next_value()?; + } + GeneratedField::SortOrder => { + if sort_order__.is_some() { + return Err(serde::de::Error::duplicate_field("sortOrder")); + } + sort_order__ = map_.next_value()?; + } + } + } + Ok(CsvSinkExecNode { + input: input__, + sink: sink__, + sink_schema: sink_schema__, + sort_order: sort_order__, + }) + } + } + deserializer.deserialize_struct("datafusion.CsvSinkExecNode", FIELDS, GeneratedVisitor) + } +} impl serde::Serialize for CsvWriterOptions { #[allow(deprecated)] fn serialize(&self, serializer: S) -> std::result::Result @@ -5159,6 +5394,9 @@ impl serde::Serialize for CsvWriterOptions { { use serde::ser::SerializeStruct; let mut len = 0; + if self.compression != 0 { + len += 1; + } if !self.delimiter.is_empty() { len += 1; } @@ -5181,6 +5419,11 @@ impl serde::Serialize for CsvWriterOptions { len += 1; } let mut struct_ser = serializer.serialize_struct("datafusion.CsvWriterOptions", len)?; + if self.compression != 0 { + let v = CompressionTypeVariant::try_from(self.compression) + .map_err(|_| serde::ser::Error::custom(format!("Invalid variant {}", self.compression)))?; + struct_ser.serialize_field("compression", &v)?; + } if !self.delimiter.is_empty() { struct_ser.serialize_field("delimiter", &self.delimiter)?; } @@ -5212,6 +5455,7 @@ impl<'de> serde::Deserialize<'de> for CsvWriterOptions { D: serde::Deserializer<'de>, { const FIELDS: &[&str] = &[ + "compression", "delimiter", "has_header", "hasHeader", @@ -5229,6 +5473,7 @@ impl<'de> serde::Deserialize<'de> for CsvWriterOptions { #[allow(clippy::enum_variant_names)] enum GeneratedField { + Compression, Delimiter, HasHeader, DateFormat, @@ -5257,6 +5502,7 @@ impl<'de> serde::Deserialize<'de> for CsvWriterOptions { E: serde::de::Error, { match value { + "compression" => Ok(GeneratedField::Compression), "delimiter" => Ok(GeneratedField::Delimiter), "hasHeader" | "has_header" => Ok(GeneratedField::HasHeader), "dateFormat" | "date_format" => Ok(GeneratedField::DateFormat), @@ -5283,6 +5529,7 @@ impl<'de> serde::Deserialize<'de> for CsvWriterOptions { where V: serde::de::MapAccess<'de>, { + let mut compression__ = None; let mut delimiter__ = None; let mut has_header__ = None; let mut date_format__ = None; @@ -5292,6 +5539,12 @@ impl<'de> serde::Deserialize<'de> for CsvWriterOptions { let mut null_value__ = None; while let Some(k) = map_.next_key()? { match k { + GeneratedField::Compression => { + if compression__.is_some() { + return Err(serde::de::Error::duplicate_field("compression")); + } + compression__ = Some(map_.next_value::()? as i32); + } GeneratedField::Delimiter => { if delimiter__.is_some() { return Err(serde::de::Error::duplicate_field("delimiter")); @@ -5337,6 +5590,7 @@ impl<'de> serde::Deserialize<'de> for CsvWriterOptions { } } Ok(CsvWriterOptions { + compression: compression__.unwrap_or_default(), delimiter: delimiter__.unwrap_or_default(), has_header: has_header__.unwrap_or_default(), date_format: date_format__.unwrap_or_default(), @@ -15398,6 +15652,241 @@ impl<'de> serde::Deserialize<'de> for ParquetScanExecNode { deserializer.deserialize_struct("datafusion.ParquetScanExecNode", FIELDS, GeneratedVisitor) } } +impl serde::Serialize for ParquetSink { + #[allow(deprecated)] + fn serialize(&self, serializer: S) -> std::result::Result + where + S: serde::Serializer, + { + use serde::ser::SerializeStruct; + let mut len = 0; + if self.config.is_some() { + len += 1; + } + let mut struct_ser = serializer.serialize_struct("datafusion.ParquetSink", len)?; + if let Some(v) = self.config.as_ref() { + struct_ser.serialize_field("config", v)?; + } + struct_ser.end() + } +} +impl<'de> serde::Deserialize<'de> for ParquetSink { + #[allow(deprecated)] + fn deserialize(deserializer: D) -> std::result::Result + where + D: serde::Deserializer<'de>, + { + const FIELDS: &[&str] = &[ + "config", + ]; + + #[allow(clippy::enum_variant_names)] + enum GeneratedField { + Config, + } + impl<'de> serde::Deserialize<'de> for GeneratedField { + fn deserialize(deserializer: D) -> std::result::Result + where + D: serde::Deserializer<'de>, + { + struct GeneratedVisitor; + + impl<'de> serde::de::Visitor<'de> for GeneratedVisitor { + type Value = GeneratedField; + + fn expecting(&self, formatter: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + write!(formatter, "expected one of: {:?}", &FIELDS) + } + + #[allow(unused_variables)] + fn visit_str(self, value: &str) -> std::result::Result + where + E: serde::de::Error, + { + match value { + "config" => Ok(GeneratedField::Config), + _ => Err(serde::de::Error::unknown_field(value, FIELDS)), + } + } + } + deserializer.deserialize_identifier(GeneratedVisitor) + } + } + struct GeneratedVisitor; + impl<'de> serde::de::Visitor<'de> for GeneratedVisitor { + type Value = ParquetSink; + + fn expecting(&self, formatter: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + formatter.write_str("struct datafusion.ParquetSink") + } + + fn visit_map(self, mut map_: V) -> std::result::Result + where + V: serde::de::MapAccess<'de>, + { + let mut config__ = None; + while let Some(k) = map_.next_key()? { + match k { + GeneratedField::Config => { + if config__.is_some() { + return Err(serde::de::Error::duplicate_field("config")); + } + config__ = map_.next_value()?; + } + } + } + Ok(ParquetSink { + config: config__, + }) + } + } + deserializer.deserialize_struct("datafusion.ParquetSink", FIELDS, GeneratedVisitor) + } +} +impl serde::Serialize for ParquetSinkExecNode { + #[allow(deprecated)] + fn serialize(&self, serializer: S) -> std::result::Result + where + S: serde::Serializer, + { + use serde::ser::SerializeStruct; + let mut len = 0; + if self.input.is_some() { + len += 1; + } + if self.sink.is_some() { + len += 1; + } + if self.sink_schema.is_some() { + len += 1; + } + if self.sort_order.is_some() { + len += 1; + } + let mut struct_ser = serializer.serialize_struct("datafusion.ParquetSinkExecNode", len)?; + if let Some(v) = self.input.as_ref() { + struct_ser.serialize_field("input", v)?; + } + if let Some(v) = self.sink.as_ref() { + struct_ser.serialize_field("sink", v)?; + } + if let Some(v) = self.sink_schema.as_ref() { + struct_ser.serialize_field("sinkSchema", v)?; + } + if let Some(v) = self.sort_order.as_ref() { + struct_ser.serialize_field("sortOrder", v)?; + } + struct_ser.end() + } +} +impl<'de> serde::Deserialize<'de> for ParquetSinkExecNode { + #[allow(deprecated)] + fn deserialize(deserializer: D) -> std::result::Result + where + D: serde::Deserializer<'de>, + { + const FIELDS: &[&str] = &[ + "input", + "sink", + "sink_schema", + "sinkSchema", + "sort_order", + "sortOrder", + ]; + + #[allow(clippy::enum_variant_names)] + enum GeneratedField { + Input, + Sink, + SinkSchema, + SortOrder, + } + impl<'de> serde::Deserialize<'de> for GeneratedField { + fn deserialize(deserializer: D) -> std::result::Result + where + D: serde::Deserializer<'de>, + { + struct GeneratedVisitor; + + impl<'de> serde::de::Visitor<'de> for GeneratedVisitor { + type Value = GeneratedField; + + fn expecting(&self, formatter: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + write!(formatter, "expected one of: {:?}", &FIELDS) + } + + #[allow(unused_variables)] + fn visit_str(self, value: &str) -> std::result::Result + where + E: serde::de::Error, + { + match value { + "input" => Ok(GeneratedField::Input), + "sink" => Ok(GeneratedField::Sink), + "sinkSchema" | "sink_schema" => Ok(GeneratedField::SinkSchema), + "sortOrder" | "sort_order" => Ok(GeneratedField::SortOrder), + _ => Err(serde::de::Error::unknown_field(value, FIELDS)), + } + } + } + deserializer.deserialize_identifier(GeneratedVisitor) + } + } + struct GeneratedVisitor; + impl<'de> serde::de::Visitor<'de> for GeneratedVisitor { + type Value = ParquetSinkExecNode; + + fn expecting(&self, formatter: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + formatter.write_str("struct datafusion.ParquetSinkExecNode") + } + + fn visit_map(self, mut map_: V) -> std::result::Result + where + V: serde::de::MapAccess<'de>, + { + let mut input__ = None; + let mut sink__ = None; + let mut sink_schema__ = None; + let mut sort_order__ = None; + while let Some(k) = map_.next_key()? { + match k { + GeneratedField::Input => { + if input__.is_some() { + return Err(serde::de::Error::duplicate_field("input")); + } + input__ = map_.next_value()?; + } + GeneratedField::Sink => { + if sink__.is_some() { + return Err(serde::de::Error::duplicate_field("sink")); + } + sink__ = map_.next_value()?; + } + GeneratedField::SinkSchema => { + if sink_schema__.is_some() { + return Err(serde::de::Error::duplicate_field("sinkSchema")); + } + sink_schema__ = map_.next_value()?; + } + GeneratedField::SortOrder => { + if sort_order__.is_some() { + return Err(serde::de::Error::duplicate_field("sortOrder")); + } + sort_order__ = map_.next_value()?; + } + } + } + Ok(ParquetSinkExecNode { + input: input__, + sink: sink__, + sink_schema: sink_schema__, + sort_order: sort_order__, + }) + } + } + deserializer.deserialize_struct("datafusion.ParquetSinkExecNode", FIELDS, GeneratedVisitor) + } +} impl serde::Serialize for ParquetWriterOptions { #[allow(deprecated)] fn serialize(&self, serializer: S) -> std::result::Result @@ -18484,6 +18973,12 @@ impl serde::Serialize for PhysicalPlanNode { physical_plan_node::PhysicalPlanType::PlaceholderRow(v) => { struct_ser.serialize_field("placeholderRow", v)?; } + physical_plan_node::PhysicalPlanType::CsvSink(v) => { + struct_ser.serialize_field("csvSink", v)?; + } + physical_plan_node::PhysicalPlanType::ParquetSink(v) => { + struct_ser.serialize_field("parquetSink", v)?; + } } } struct_ser.end() @@ -18535,6 +19030,10 @@ impl<'de> serde::Deserialize<'de> for PhysicalPlanNode { "interleave", "placeholder_row", "placeholderRow", + "csv_sink", + "csvSink", + "parquet_sink", + "parquetSink", ]; #[allow(clippy::enum_variant_names)] @@ -18565,6 +19064,8 @@ impl<'de> serde::Deserialize<'de> for PhysicalPlanNode { SymmetricHashJoin, Interleave, PlaceholderRow, + CsvSink, + ParquetSink, } impl<'de> serde::Deserialize<'de> for GeneratedField { fn deserialize(deserializer: D) -> std::result::Result @@ -18612,6 +19113,8 @@ impl<'de> serde::Deserialize<'de> for PhysicalPlanNode { "symmetricHashJoin" | "symmetric_hash_join" => Ok(GeneratedField::SymmetricHashJoin), "interleave" => Ok(GeneratedField::Interleave), "placeholderRow" | "placeholder_row" => Ok(GeneratedField::PlaceholderRow), + "csvSink" | "csv_sink" => Ok(GeneratedField::CsvSink), + "parquetSink" | "parquet_sink" => Ok(GeneratedField::ParquetSink), _ => Err(serde::de::Error::unknown_field(value, FIELDS)), } } @@ -18814,6 +19317,20 @@ impl<'de> serde::Deserialize<'de> for PhysicalPlanNode { return Err(serde::de::Error::duplicate_field("placeholderRow")); } physical_plan_type__ = map_.next_value::<::std::option::Option<_>>()?.map(physical_plan_node::PhysicalPlanType::PlaceholderRow) +; + } + GeneratedField::CsvSink => { + if physical_plan_type__.is_some() { + return Err(serde::de::Error::duplicate_field("csvSink")); + } + physical_plan_type__ = map_.next_value::<::std::option::Option<_>>()?.map(physical_plan_node::PhysicalPlanType::CsvSink) +; + } + GeneratedField::ParquetSink => { + if physical_plan_type__.is_some() { + return Err(serde::de::Error::duplicate_field("parquetSink")); + } + physical_plan_type__ = map_.next_value::<::std::option::Option<_>>()?.map(physical_plan_node::PhysicalPlanType::ParquetSink) ; } } diff --git a/datafusion/proto/src/generated/prost.rs b/datafusion/proto/src/generated/prost.rs index 32e892e663ef..4ee0b70325ca 100644 --- a/datafusion/proto/src/generated/prost.rs +++ b/datafusion/proto/src/generated/prost.rs @@ -1566,7 +1566,7 @@ pub mod owned_table_reference { pub struct PhysicalPlanNode { #[prost( oneof = "physical_plan_node::PhysicalPlanType", - tags = "1, 2, 3, 4, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27" + tags = "1, 2, 3, 4, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29" )] pub physical_plan_type: ::core::option::Option, } @@ -1629,6 +1629,10 @@ pub mod physical_plan_node { Interleave(super::InterleaveExecNode), #[prost(message, tag = "27")] PlaceholderRow(super::PlaceholderRowExecNode), + #[prost(message, tag = "28")] + CsvSink(::prost::alloc::boxed::Box), + #[prost(message, tag = "29")] + ParquetSink(::prost::alloc::boxed::Box), } } #[allow(clippy::derive_partial_eq_without_eq)] @@ -1673,26 +1677,29 @@ pub struct ParquetWriterOptions { #[allow(clippy::derive_partial_eq_without_eq)] #[derive(Clone, PartialEq, ::prost::Message)] pub struct CsvWriterOptions { + /// Compression type + #[prost(enumeration = "CompressionTypeVariant", tag = "1")] + pub compression: i32, /// Optional column delimiter. Defaults to `b','` - #[prost(string, tag = "1")] + #[prost(string, tag = "2")] pub delimiter: ::prost::alloc::string::String, /// Whether to write column names as file headers. Defaults to `true` - #[prost(bool, tag = "2")] + #[prost(bool, tag = "3")] pub has_header: bool, /// Optional date format for date arrays - #[prost(string, tag = "3")] + #[prost(string, tag = "4")] pub date_format: ::prost::alloc::string::String, /// Optional datetime format for datetime arrays - #[prost(string, tag = "4")] + #[prost(string, tag = "5")] pub datetime_format: ::prost::alloc::string::String, /// Optional timestamp format for timestamp arrays - #[prost(string, tag = "5")] + #[prost(string, tag = "6")] pub timestamp_format: ::prost::alloc::string::String, /// Optional time format for time arrays - #[prost(string, tag = "6")] + #[prost(string, tag = "7")] pub time_format: ::prost::alloc::string::String, /// Optional value to represent null - #[prost(string, tag = "7")] + #[prost(string, tag = "8")] pub null_value: ::prost::alloc::string::String, } #[allow(clippy::derive_partial_eq_without_eq)] @@ -1753,6 +1760,42 @@ pub struct JsonSinkExecNode { } #[allow(clippy::derive_partial_eq_without_eq)] #[derive(Clone, PartialEq, ::prost::Message)] +pub struct CsvSink { + #[prost(message, optional, tag = "1")] + pub config: ::core::option::Option, +} +#[allow(clippy::derive_partial_eq_without_eq)] +#[derive(Clone, PartialEq, ::prost::Message)] +pub struct CsvSinkExecNode { + #[prost(message, optional, boxed, tag = "1")] + pub input: ::core::option::Option<::prost::alloc::boxed::Box>, + #[prost(message, optional, tag = "2")] + pub sink: ::core::option::Option, + #[prost(message, optional, tag = "3")] + pub sink_schema: ::core::option::Option, + #[prost(message, optional, tag = "4")] + pub sort_order: ::core::option::Option, +} +#[allow(clippy::derive_partial_eq_without_eq)] +#[derive(Clone, PartialEq, ::prost::Message)] +pub struct ParquetSink { + #[prost(message, optional, tag = "1")] + pub config: ::core::option::Option, +} +#[allow(clippy::derive_partial_eq_without_eq)] +#[derive(Clone, PartialEq, ::prost::Message)] +pub struct ParquetSinkExecNode { + #[prost(message, optional, boxed, tag = "1")] + pub input: ::core::option::Option<::prost::alloc::boxed::Box>, + #[prost(message, optional, tag = "2")] + pub sink: ::core::option::Option, + #[prost(message, optional, tag = "3")] + pub sink_schema: ::core::option::Option, + #[prost(message, optional, tag = "4")] + pub sort_order: ::core::option::Option, +} +#[allow(clippy::derive_partial_eq_without_eq)] +#[derive(Clone, PartialEq, ::prost::Message)] pub struct PhysicalExtensionNode { #[prost(bytes = "vec", tag = "1")] pub node: ::prost::alloc::vec::Vec, diff --git a/datafusion/proto/src/logical_plan/mod.rs b/datafusion/proto/src/logical_plan/mod.rs index dbed0252d051..5ee88c3d5328 100644 --- a/datafusion/proto/src/logical_plan/mod.rs +++ b/datafusion/proto/src/logical_plan/mod.rs @@ -1648,28 +1648,10 @@ impl AsLogicalPlan for LogicalPlanNode { match opt.as_ref() { FileTypeWriterOptions::CSV(csv_opts) => { let csv_options = &csv_opts.writer_options; - let csv_writer_options = protobuf::CsvWriterOptions { - delimiter: (csv_options.delimiter() as char) - .to_string(), - has_header: csv_options.header(), - date_format: csv_options - .date_format() - .unwrap_or("") - .to_owned(), - datetime_format: csv_options - .datetime_format() - .unwrap_or("") - .to_owned(), - timestamp_format: csv_options - .timestamp_format() - .unwrap_or("") - .to_owned(), - time_format: csv_options - .time_format() - .unwrap_or("") - .to_owned(), - null_value: csv_options.null().to_owned(), - }; + let csv_writer_options = csv_writer_options_to_proto( + csv_options, + (&csv_opts.compression).into(), + ); let csv_options = file_type_writer_options::FileType::CsvOptions( csv_writer_options, @@ -1724,6 +1706,23 @@ impl AsLogicalPlan for LogicalPlanNode { } } +pub(crate) fn csv_writer_options_to_proto( + csv_options: &WriterBuilder, + compression: &CompressionTypeVariant, +) -> protobuf::CsvWriterOptions { + let compression: protobuf::CompressionTypeVariant = compression.into(); + protobuf::CsvWriterOptions { + compression: compression.into(), + delimiter: (csv_options.delimiter() as char).to_string(), + has_header: csv_options.header(), + date_format: csv_options.date_format().unwrap_or("").to_owned(), + datetime_format: csv_options.datetime_format().unwrap_or("").to_owned(), + timestamp_format: csv_options.timestamp_format().unwrap_or("").to_owned(), + time_format: csv_options.time_format().unwrap_or("").to_owned(), + null_value: csv_options.null().to_owned(), + } +} + pub(crate) fn csv_writer_options_from_proto( writer_options: &protobuf::CsvWriterOptions, ) -> Result { diff --git a/datafusion/proto/src/physical_plan/from_proto.rs b/datafusion/proto/src/physical_plan/from_proto.rs index 6f1e811510c6..8ad6d679df4d 100644 --- a/datafusion/proto/src/physical_plan/from_proto.rs +++ b/datafusion/proto/src/physical_plan/from_proto.rs @@ -22,7 +22,10 @@ use std::sync::Arc; use arrow::compute::SortOptions; use datafusion::arrow::datatypes::Schema; +use datafusion::datasource::file_format::csv::CsvSink; use datafusion::datasource::file_format::json::JsonSink; +#[cfg(feature = "parquet")] +use datafusion::datasource::file_format::parquet::ParquetSink; use datafusion::datasource::listing::{FileRange, ListingTableUrl, PartitionedFile}; use datafusion::datasource::object_store::ObjectStoreUrl; use datafusion::datasource::physical_plan::{FileScanConfig, FileSinkConfig}; @@ -713,6 +716,23 @@ impl TryFrom<&protobuf::JsonSink> for JsonSink { } } +#[cfg(feature = "parquet")] +impl TryFrom<&protobuf::ParquetSink> for ParquetSink { + type Error = DataFusionError; + + fn try_from(value: &protobuf::ParquetSink) -> Result { + Ok(Self::new(convert_required!(value.config)?)) + } +} + +impl TryFrom<&protobuf::CsvSink> for CsvSink { + type Error = DataFusionError; + + fn try_from(value: &protobuf::CsvSink) -> Result { + Ok(Self::new(convert_required!(value.config)?)) + } +} + impl TryFrom<&protobuf::FileSinkConfig> for FileSinkConfig { type Error = DataFusionError; @@ -768,16 +788,16 @@ impl TryFrom<&protobuf::FileTypeWriterOptions> for FileTypeWriterOptions { .file_type .as_ref() .ok_or_else(|| proto_error("Missing required file_type field in protobuf"))?; + match file_type { - protobuf::file_type_writer_options::FileType::JsonOptions(opts) => Ok( - Self::JSON(JsonWriterOptions::new(opts.compression().into())), - ), - protobuf::file_type_writer_options::FileType::CsvOptions(opt) => { - let write_options = csv_writer_options_from_proto(opt)?; - Ok(Self::CSV(CsvWriterOptions::new( - write_options, - CompressionTypeVariant::UNCOMPRESSED, - ))) + protobuf::file_type_writer_options::FileType::JsonOptions(opts) => { + let compression: CompressionTypeVariant = opts.compression().into(); + Ok(Self::JSON(JsonWriterOptions::new(compression))) + } + protobuf::file_type_writer_options::FileType::CsvOptions(opts) => { + let write_options = csv_writer_options_from_proto(opts)?; + let compression: CompressionTypeVariant = opts.compression().into(); + Ok(Self::CSV(CsvWriterOptions::new(write_options, compression))) } protobuf::file_type_writer_options::FileType::ParquetOptions(opt) => { let props = opt.writer_properties.clone().unwrap_or_default(); diff --git a/datafusion/proto/src/physical_plan/mod.rs b/datafusion/proto/src/physical_plan/mod.rs index 24ede3fcaf62..95becb3fe4b3 100644 --- a/datafusion/proto/src/physical_plan/mod.rs +++ b/datafusion/proto/src/physical_plan/mod.rs @@ -21,9 +21,12 @@ use std::sync::Arc; use datafusion::arrow::compute::SortOptions; use datafusion::arrow::datatypes::SchemaRef; +use datafusion::datasource::file_format::csv::CsvSink; use datafusion::datasource::file_format::file_compression_type::FileCompressionType; use datafusion::datasource::file_format::json::JsonSink; #[cfg(feature = "parquet")] +use datafusion::datasource::file_format::parquet::ParquetSink; +#[cfg(feature = "parquet")] use datafusion::datasource::physical_plan::ParquetExec; use datafusion::datasource::physical_plan::{AvroExec, CsvExec}; use datafusion::execution::runtime_env::RuntimeEnv; @@ -921,6 +924,68 @@ impl AsExecutionPlan for PhysicalPlanNode { sort_order, ))) } + PhysicalPlanType::CsvSink(sink) => { + let input = + into_physical_plan(&sink.input, registry, runtime, extension_codec)?; + + let data_sink: CsvSink = sink + .sink + .as_ref() + .ok_or_else(|| proto_error("Missing required field in protobuf"))? + .try_into()?; + let sink_schema = convert_required!(sink.sink_schema)?; + let sort_order = sink + .sort_order + .as_ref() + .map(|collection| { + collection + .physical_sort_expr_nodes + .iter() + .map(|proto| { + parse_physical_sort_expr(proto, registry, &sink_schema) + .map(Into::into) + }) + .collect::>>() + }) + .transpose()?; + Ok(Arc::new(FileSinkExec::new( + input, + Arc::new(data_sink), + Arc::new(sink_schema), + sort_order, + ))) + } + PhysicalPlanType::ParquetSink(sink) => { + let input = + into_physical_plan(&sink.input, registry, runtime, extension_codec)?; + + let data_sink: ParquetSink = sink + .sink + .as_ref() + .ok_or_else(|| proto_error("Missing required field in protobuf"))? + .try_into()?; + let sink_schema = convert_required!(sink.sink_schema)?; + let sort_order = sink + .sort_order + .as_ref() + .map(|collection| { + collection + .physical_sort_expr_nodes + .iter() + .map(|proto| { + parse_physical_sort_expr(proto, registry, &sink_schema) + .map(Into::into) + }) + .collect::>>() + }) + .transpose()?; + Ok(Arc::new(FileSinkExec::new( + input, + Arc::new(data_sink), + Arc::new(sink_schema), + sort_order, + ))) + } } } @@ -1678,6 +1743,32 @@ impl AsExecutionPlan for PhysicalPlanNode { }); } + if let Some(sink) = exec.sink().as_any().downcast_ref::() { + return Ok(protobuf::PhysicalPlanNode { + physical_plan_type: Some(PhysicalPlanType::CsvSink(Box::new( + protobuf::CsvSinkExecNode { + input: Some(Box::new(input)), + sink: Some(sink.try_into()?), + sink_schema: Some(exec.schema().as_ref().try_into()?), + sort_order, + }, + ))), + }); + } + + if let Some(sink) = exec.sink().as_any().downcast_ref::() { + return Ok(protobuf::PhysicalPlanNode { + physical_plan_type: Some(PhysicalPlanType::ParquetSink(Box::new( + protobuf::ParquetSinkExecNode { + input: Some(Box::new(input)), + sink: Some(sink.try_into()?), + sink_schema: Some(exec.schema().as_ref().try_into()?), + sort_order, + }, + ))), + }); + } + // If unknown DataSink then let extension handle it } diff --git a/datafusion/proto/src/physical_plan/to_proto.rs b/datafusion/proto/src/physical_plan/to_proto.rs index e9cdb34cf1b9..f4e3f9e4dca7 100644 --- a/datafusion/proto/src/physical_plan/to_proto.rs +++ b/datafusion/proto/src/physical_plan/to_proto.rs @@ -28,7 +28,12 @@ use crate::protobuf::{ ScalarValue, }; +#[cfg(feature = "parquet")] +use datafusion::datasource::file_format::parquet::ParquetSink; + +use crate::logical_plan::{csv_writer_options_to_proto, writer_properties_to_proto}; use datafusion::datasource::{ + file_format::csv::CsvSink, file_format::json::JsonSink, listing::{FileRange, PartitionedFile}, physical_plan::FileScanConfig, @@ -814,6 +819,27 @@ impl TryFrom<&JsonSink> for protobuf::JsonSink { } } +impl TryFrom<&CsvSink> for protobuf::CsvSink { + type Error = DataFusionError; + + fn try_from(value: &CsvSink) -> Result { + Ok(Self { + config: Some(value.config().try_into()?), + }) + } +} + +#[cfg(feature = "parquet")] +impl TryFrom<&ParquetSink> for protobuf::ParquetSink { + type Error = DataFusionError; + + fn try_from(value: &ParquetSink) -> Result { + Ok(Self { + config: Some(value.config().try_into()?), + }) + } +} + impl TryFrom<&FileSinkConfig> for protobuf::FileSinkConfig { type Error = DataFusionError; @@ -870,13 +896,21 @@ impl TryFrom<&FileTypeWriterOptions> for protobuf::FileTypeWriterOptions { fn try_from(opts: &FileTypeWriterOptions) -> Result { let file_type = match opts { #[cfg(feature = "parquet")] - FileTypeWriterOptions::Parquet(ParquetWriterOptions { - writer_options: _, - }) => return not_impl_err!("Parquet file sink protobuf serialization"), + FileTypeWriterOptions::Parquet(ParquetWriterOptions { writer_options }) => { + protobuf::file_type_writer_options::FileType::ParquetOptions( + protobuf::ParquetWriterOptions { + writer_properties: Some(writer_properties_to_proto( + writer_options, + )), + }, + ) + } FileTypeWriterOptions::CSV(CsvWriterOptions { - writer_options: _, - compression: _, - }) => return not_impl_err!("CSV file sink protobuf serialization"), + writer_options, + compression, + }) => protobuf::file_type_writer_options::FileType::CsvOptions( + csv_writer_options_to_proto(writer_options, compression), + ), FileTypeWriterOptions::JSON(JsonWriterOptions { compression }) => { let compression: protobuf::CompressionTypeVariant = compression.into(); protobuf::file_type_writer_options::FileType::JsonOptions( diff --git a/datafusion/proto/tests/cases/roundtrip_physical_plan.rs b/datafusion/proto/tests/cases/roundtrip_physical_plan.rs index 2eb04ab6cbab..27ac5d122f83 100644 --- a/datafusion/proto/tests/cases/roundtrip_physical_plan.rs +++ b/datafusion/proto/tests/cases/roundtrip_physical_plan.rs @@ -15,13 +15,16 @@ // specific language governing permissions and limitations // under the License. +use arrow::csv::WriterBuilder; use std::ops::Deref; use std::sync::Arc; use datafusion::arrow::array::ArrayRef; use datafusion::arrow::compute::kernels::sort::SortOptions; use datafusion::arrow::datatypes::{DataType, Field, Fields, IntervalUnit, Schema}; +use datafusion::datasource::file_format::csv::CsvSink; use datafusion::datasource::file_format::json::JsonSink; +use datafusion::datasource::file_format::parquet::ParquetSink; use datafusion::datasource::listing::{ListingTableUrl, PartitionedFile}; use datafusion::datasource::object_store::ObjectStoreUrl; use datafusion::datasource::physical_plan::{ @@ -31,6 +34,7 @@ use datafusion::execution::context::ExecutionProps; use datafusion::logical_expr::{ create_udf, BuiltinScalarFunction, JoinType, Operator, Volatility, }; +use datafusion::parquet::file::properties::WriterProperties; use datafusion::physical_expr::window::SlidingAggregateWindowExpr; use datafusion::physical_expr::{PhysicalSortRequirement, ScalarFunctionExpr}; use datafusion::physical_plan::aggregates::{ @@ -62,7 +66,9 @@ use datafusion::physical_plan::{ }; use datafusion::prelude::SessionContext; use datafusion::scalar::ScalarValue; +use datafusion_common::file_options::csv_writer::CsvWriterOptions; use datafusion_common::file_options::json_writer::JsonWriterOptions; +use datafusion_common::file_options::parquet_writer::ParquetWriterOptions; use datafusion_common::parsers::CompressionTypeVariant; use datafusion_common::stats::Precision; use datafusion_common::{FileTypeWriterOptions, Result}; @@ -73,7 +79,23 @@ use datafusion_expr::{ use datafusion_proto::physical_plan::{AsExecutionPlan, DefaultPhysicalExtensionCodec}; use datafusion_proto::protobuf; +/// Perform a serde roundtrip and assert that the string representation of the before and after plans +/// are identical. Note that this often isn't sufficient to guarantee that no information is +/// lost during serde because the string representation of a plan often only shows a subset of state. fn roundtrip_test(exec_plan: Arc) -> Result<()> { + let _ = roundtrip_test_and_return(exec_plan); + Ok(()) +} + +/// Perform a serde roundtrip and assert that the string representation of the before and after plans +/// are identical. Note that this often isn't sufficient to guarantee that no information is +/// lost during serde because the string representation of a plan often only shows a subset of state. +/// +/// This version of the roundtrip_test method returns the final plan after serde so that it can be inspected +/// farther in tests. +fn roundtrip_test_and_return( + exec_plan: Arc, +) -> Result> { let ctx = SessionContext::new(); let codec = DefaultPhysicalExtensionCodec {}; let proto: protobuf::PhysicalPlanNode = @@ -84,9 +106,15 @@ fn roundtrip_test(exec_plan: Arc) -> Result<()> { .try_into_physical_plan(&ctx, runtime.deref(), &codec) .expect("from proto"); assert_eq!(format!("{exec_plan:?}"), format!("{result_exec_plan:?}")); - Ok(()) + Ok(result_exec_plan) } +/// Perform a serde roundtrip and assert that the string representation of the before and after plans +/// are identical. Note that this often isn't sufficient to guarantee that no information is +/// lost during serde because the string representation of a plan often only shows a subset of state. +/// +/// This version of the roundtrip_test function accepts a SessionContext, which is required when +/// performing serde on some plans. fn roundtrip_test_with_context( exec_plan: Arc, ctx: SessionContext, @@ -755,6 +783,101 @@ fn roundtrip_json_sink() -> Result<()> { ))) } +#[test] +fn roundtrip_csv_sink() -> Result<()> { + let field_a = Field::new("plan_type", DataType::Utf8, false); + let field_b = Field::new("plan", DataType::Utf8, false); + let schema = Arc::new(Schema::new(vec![field_a, field_b])); + let input = Arc::new(PlaceholderRowExec::new(schema.clone())); + + let file_sink_config = FileSinkConfig { + object_store_url: ObjectStoreUrl::local_filesystem(), + file_groups: vec![PartitionedFile::new("/tmp".to_string(), 1)], + table_paths: vec![ListingTableUrl::parse("file:///")?], + output_schema: schema.clone(), + table_partition_cols: vec![("plan_type".to_string(), DataType::Utf8)], + single_file_output: true, + overwrite: true, + file_type_writer_options: FileTypeWriterOptions::CSV(CsvWriterOptions::new( + WriterBuilder::default(), + CompressionTypeVariant::ZSTD, + )), + }; + let data_sink = Arc::new(CsvSink::new(file_sink_config)); + let sort_order = vec![PhysicalSortRequirement::new( + Arc::new(Column::new("plan_type", 0)), + Some(SortOptions { + descending: true, + nulls_first: false, + }), + )]; + + let roundtrip_plan = roundtrip_test_and_return(Arc::new(FileSinkExec::new( + input, + data_sink, + schema.clone(), + Some(sort_order), + ))) + .unwrap(); + + let roundtrip_plan = roundtrip_plan + .as_any() + .downcast_ref::() + .unwrap(); + let csv_sink = roundtrip_plan + .sink() + .as_any() + .downcast_ref::() + .unwrap(); + assert_eq!( + CompressionTypeVariant::ZSTD, + csv_sink + .config() + .file_type_writer_options + .try_into_csv() + .unwrap() + .compression + ); + + Ok(()) +} + +#[test] +fn roundtrip_parquet_sink() -> Result<()> { + let field_a = Field::new("plan_type", DataType::Utf8, false); + let field_b = Field::new("plan", DataType::Utf8, false); + let schema = Arc::new(Schema::new(vec![field_a, field_b])); + let input = Arc::new(PlaceholderRowExec::new(schema.clone())); + + let file_sink_config = FileSinkConfig { + object_store_url: ObjectStoreUrl::local_filesystem(), + file_groups: vec![PartitionedFile::new("/tmp".to_string(), 1)], + table_paths: vec![ListingTableUrl::parse("file:///")?], + output_schema: schema.clone(), + table_partition_cols: vec![("plan_type".to_string(), DataType::Utf8)], + single_file_output: true, + overwrite: true, + file_type_writer_options: FileTypeWriterOptions::Parquet( + ParquetWriterOptions::new(WriterProperties::default()), + ), + }; + let data_sink = Arc::new(ParquetSink::new(file_sink_config)); + let sort_order = vec![PhysicalSortRequirement::new( + Arc::new(Column::new("plan_type", 0)), + Some(SortOptions { + descending: true, + nulls_first: false, + }), + )]; + + roundtrip_test(Arc::new(FileSinkExec::new( + input, + data_sink, + schema.clone(), + Some(sort_order), + ))) +} + #[test] fn roundtrip_sym_hash_join() -> Result<()> { let field_a = Field::new("col", DataType::Int64, false); From 7f440e18f22ac9b6a6b72ca305fd04704de325fd Mon Sep 17 00:00:00 2001 From: Yang Jiang Date: Sat, 30 Dec 2023 08:33:32 +0800 Subject: [PATCH 517/572] [pruning] Add shortcut when all units have been pruned (#8675) --- datafusion/core/src/physical_optimizer/pruning.rs | 10 ++++++++++ 1 file changed, 10 insertions(+) diff --git a/datafusion/core/src/physical_optimizer/pruning.rs b/datafusion/core/src/physical_optimizer/pruning.rs index 79e084d7b7f1..fecbffdbb041 100644 --- a/datafusion/core/src/physical_optimizer/pruning.rs +++ b/datafusion/core/src/physical_optimizer/pruning.rs @@ -258,6 +258,11 @@ impl PruningPredicate { builder.combine_array(&arrow::compute::not(&results)?) } } + // if all containers are pruned (has rows that DEFINITELY DO NOT pass the predicate) + // can return early without evaluating the rest of predicates. + if builder.check_all_pruned() { + return Ok(builder.build()); + } } } @@ -380,6 +385,11 @@ impl BoolVecBuilder { fn build(self) -> Vec { self.inner } + + /// Check all containers has rows that DEFINITELY DO NOT pass the predicate + fn check_all_pruned(&self) -> bool { + self.inner.iter().all(|&x| !x) + } } fn is_always_true(expr: &Arc) -> bool { From bb98dfed08d8c2b94ab668a064b206d8b84b51b0 Mon Sep 17 00:00:00 2001 From: Mustafa Akur <106137913+mustafasrepo@users.noreply.github.com> Date: Sat, 30 Dec 2023 03:48:36 +0300 Subject: [PATCH 518/572] Change first/last implementation to prevent redundant comparisons when data is already sorted (#8678) * Change fist last implementation to prevent redundant computations * Remove redundant checks * Review --------- Co-authored-by: Mehmet Ozan Kabak --- .../physical-expr/src/aggregate/first_last.rs | 259 +++++++++++------- .../physical-plan/src/aggregates/mod.rs | 77 +++++- .../sqllogictest/test_files/groupby.slt | 14 +- 3 files changed, 234 insertions(+), 116 deletions(-) diff --git a/datafusion/physical-expr/src/aggregate/first_last.rs b/datafusion/physical-expr/src/aggregate/first_last.rs index c7032e601cf8..4afa8d0dd5ec 100644 --- a/datafusion/physical-expr/src/aggregate/first_last.rs +++ b/datafusion/physical-expr/src/aggregate/first_last.rs @@ -36,13 +36,14 @@ use datafusion_common::{ use datafusion_expr::Accumulator; /// FIRST_VALUE aggregate expression -#[derive(Debug)] +#[derive(Debug, Clone)] pub struct FirstValue { name: String, input_data_type: DataType, order_by_data_types: Vec, expr: Arc, ordering_req: LexOrdering, + requirement_satisfied: bool, } impl FirstValue { @@ -54,12 +55,14 @@ impl FirstValue { ordering_req: LexOrdering, order_by_data_types: Vec, ) -> Self { + let requirement_satisfied = ordering_req.is_empty(); Self { name: name.into(), input_data_type, order_by_data_types, expr, ordering_req, + requirement_satisfied, } } @@ -87,6 +90,33 @@ impl FirstValue { pub fn ordering_req(&self) -> &LexOrdering { &self.ordering_req } + + pub fn with_requirement_satisfied(mut self, requirement_satisfied: bool) -> Self { + self.requirement_satisfied = requirement_satisfied; + self + } + + pub fn convert_to_last(self) -> LastValue { + let name = if self.name.starts_with("FIRST") { + format!("LAST{}", &self.name[5..]) + } else { + format!("LAST_VALUE({})", self.expr) + }; + let FirstValue { + expr, + input_data_type, + ordering_req, + order_by_data_types, + .. + } = self; + LastValue::new( + expr, + name, + input_data_type, + reverse_order_bys(&ordering_req), + order_by_data_types, + ) + } } impl AggregateExpr for FirstValue { @@ -100,11 +130,14 @@ impl AggregateExpr for FirstValue { } fn create_accumulator(&self) -> Result> { - Ok(Box::new(FirstValueAccumulator::try_new( + FirstValueAccumulator::try_new( &self.input_data_type, &self.order_by_data_types, self.ordering_req.clone(), - )?)) + ) + .map(|acc| { + Box::new(acc.with_requirement_satisfied(self.requirement_satisfied)) as _ + }) } fn state_fields(&self) -> Result> { @@ -130,11 +163,7 @@ impl AggregateExpr for FirstValue { } fn order_bys(&self) -> Option<&[PhysicalSortExpr]> { - if self.ordering_req.is_empty() { - None - } else { - Some(&self.ordering_req) - } + (!self.ordering_req.is_empty()).then_some(&self.ordering_req) } fn name(&self) -> &str { @@ -142,26 +171,18 @@ impl AggregateExpr for FirstValue { } fn reverse_expr(&self) -> Option> { - let name = if self.name.starts_with("FIRST") { - format!("LAST{}", &self.name[5..]) - } else { - format!("LAST_VALUE({})", self.expr) - }; - Some(Arc::new(LastValue::new( - self.expr.clone(), - name, - self.input_data_type.clone(), - reverse_order_bys(&self.ordering_req), - self.order_by_data_types.clone(), - ))) + Some(Arc::new(self.clone().convert_to_last())) } fn create_sliding_accumulator(&self) -> Result> { - Ok(Box::new(FirstValueAccumulator::try_new( + FirstValueAccumulator::try_new( &self.input_data_type, &self.order_by_data_types, self.ordering_req.clone(), - )?)) + ) + .map(|acc| { + Box::new(acc.with_requirement_satisfied(self.requirement_satisfied)) as _ + }) } } @@ -190,6 +211,8 @@ struct FirstValueAccumulator { orderings: Vec, // Stores the applicable ordering requirement. ordering_req: LexOrdering, + // Stores whether incoming data already satisfies the ordering requirement. + requirement_satisfied: bool, } impl FirstValueAccumulator { @@ -203,42 +226,29 @@ impl FirstValueAccumulator { .iter() .map(ScalarValue::try_from) .collect::>>()?; - ScalarValue::try_from(data_type).map(|value| Self { - first: value, + let requirement_satisfied = ordering_req.is_empty(); + ScalarValue::try_from(data_type).map(|first| Self { + first, is_set: false, orderings, ordering_req, + requirement_satisfied, }) } // Updates state with the values in the given row. - fn update_with_new_row(&mut self, row: &[ScalarValue]) -> Result<()> { - let [value, orderings @ ..] = row else { - return internal_err!("Empty row in FIRST_VALUE"); - }; - // Update when there is no entry in the state, or we have an "earlier" - // entry according to sort requirements. - if !self.is_set - || compare_rows( - &self.orderings, - orderings, - &get_sort_options(&self.ordering_req), - )? - .is_gt() - { - self.first = value.clone(); - self.orderings = orderings.to_vec(); - self.is_set = true; - } - Ok(()) + fn update_with_new_row(&mut self, row: &[ScalarValue]) { + self.first = row[0].clone(); + self.orderings = row[1..].to_vec(); + self.is_set = true; } fn get_first_idx(&self, values: &[ArrayRef]) -> Result> { let [value, ordering_values @ ..] = values else { return internal_err!("Empty row in FIRST_VALUE"); }; - if self.ordering_req.is_empty() { - // Get first entry according to receive order (0th index) + if self.requirement_satisfied { + // Get first entry according to the pre-existing ordering (0th index): return Ok((!value.is_empty()).then_some(0)); } let sort_columns = ordering_values @@ -252,6 +262,11 @@ impl FirstValueAccumulator { let indices = lexsort_to_indices(&sort_columns, Some(1))?; Ok((!indices.is_empty()).then_some(indices.value(0) as _)) } + + fn with_requirement_satisfied(mut self, requirement_satisfied: bool) -> Self { + self.requirement_satisfied = requirement_satisfied; + self + } } impl Accumulator for FirstValueAccumulator { @@ -263,9 +278,25 @@ impl Accumulator for FirstValueAccumulator { } fn update_batch(&mut self, values: &[ArrayRef]) -> Result<()> { - if let Some(first_idx) = self.get_first_idx(values)? { - let row = get_row_at_idx(values, first_idx)?; - self.update_with_new_row(&row)?; + if !self.is_set { + if let Some(first_idx) = self.get_first_idx(values)? { + let row = get_row_at_idx(values, first_idx)?; + self.update_with_new_row(&row); + } + } else if !self.requirement_satisfied { + if let Some(first_idx) = self.get_first_idx(values)? { + let row = get_row_at_idx(values, first_idx)?; + let orderings = &row[1..]; + if compare_rows( + &self.orderings, + orderings, + &get_sort_options(&self.ordering_req), + )? + .is_gt() + { + self.update_with_new_row(&row); + } + } } Ok(()) } @@ -294,12 +325,12 @@ impl Accumulator for FirstValueAccumulator { let sort_options = get_sort_options(&self.ordering_req); // Either there is no existing value, or there is an earlier version in new data. if !self.is_set - || compare_rows(first_ordering, &self.orderings, &sort_options)?.is_lt() + || compare_rows(&self.orderings, first_ordering, &sort_options)?.is_gt() { // Update with first value in the state. Note that we should exclude the // is_set flag from the state. Otherwise, we will end up with a state // containing two is_set flags. - self.update_with_new_row(&first_row[0..is_set_idx])?; + self.update_with_new_row(&first_row[0..is_set_idx]); } } Ok(()) @@ -318,13 +349,14 @@ impl Accumulator for FirstValueAccumulator { } /// LAST_VALUE aggregate expression -#[derive(Debug)] +#[derive(Debug, Clone)] pub struct LastValue { name: String, input_data_type: DataType, order_by_data_types: Vec, expr: Arc, ordering_req: LexOrdering, + requirement_satisfied: bool, } impl LastValue { @@ -336,12 +368,14 @@ impl LastValue { ordering_req: LexOrdering, order_by_data_types: Vec, ) -> Self { + let requirement_satisfied = ordering_req.is_empty(); Self { name: name.into(), input_data_type, order_by_data_types, expr, ordering_req, + requirement_satisfied, } } @@ -369,6 +403,33 @@ impl LastValue { pub fn ordering_req(&self) -> &LexOrdering { &self.ordering_req } + + pub fn with_requirement_satisfied(mut self, requirement_satisfied: bool) -> Self { + self.requirement_satisfied = requirement_satisfied; + self + } + + pub fn convert_to_first(self) -> FirstValue { + let name = if self.name.starts_with("LAST") { + format!("FIRST{}", &self.name[4..]) + } else { + format!("FIRST_VALUE({})", self.expr) + }; + let LastValue { + expr, + input_data_type, + ordering_req, + order_by_data_types, + .. + } = self; + FirstValue::new( + expr, + name, + input_data_type, + reverse_order_bys(&ordering_req), + order_by_data_types, + ) + } } impl AggregateExpr for LastValue { @@ -382,11 +443,14 @@ impl AggregateExpr for LastValue { } fn create_accumulator(&self) -> Result> { - Ok(Box::new(LastValueAccumulator::try_new( + LastValueAccumulator::try_new( &self.input_data_type, &self.order_by_data_types, self.ordering_req.clone(), - )?)) + ) + .map(|acc| { + Box::new(acc.with_requirement_satisfied(self.requirement_satisfied)) as _ + }) } fn state_fields(&self) -> Result> { @@ -412,11 +476,7 @@ impl AggregateExpr for LastValue { } fn order_bys(&self) -> Option<&[PhysicalSortExpr]> { - if self.ordering_req.is_empty() { - None - } else { - Some(&self.ordering_req) - } + (!self.ordering_req.is_empty()).then_some(&self.ordering_req) } fn name(&self) -> &str { @@ -424,26 +484,18 @@ impl AggregateExpr for LastValue { } fn reverse_expr(&self) -> Option> { - let name = if self.name.starts_with("LAST") { - format!("FIRST{}", &self.name[4..]) - } else { - format!("FIRST_VALUE({})", self.expr) - }; - Some(Arc::new(FirstValue::new( - self.expr.clone(), - name, - self.input_data_type.clone(), - reverse_order_bys(&self.ordering_req), - self.order_by_data_types.clone(), - ))) + Some(Arc::new(self.clone().convert_to_first())) } fn create_sliding_accumulator(&self) -> Result> { - Ok(Box::new(LastValueAccumulator::try_new( + LastValueAccumulator::try_new( &self.input_data_type, &self.order_by_data_types, self.ordering_req.clone(), - )?)) + ) + .map(|acc| { + Box::new(acc.with_requirement_satisfied(self.requirement_satisfied)) as _ + }) } } @@ -471,6 +523,8 @@ struct LastValueAccumulator { orderings: Vec, // Stores the applicable ordering requirement. ordering_req: LexOrdering, + // Stores whether incoming data already satisfies the ordering requirement. + requirement_satisfied: bool, } impl LastValueAccumulator { @@ -484,42 +538,28 @@ impl LastValueAccumulator { .iter() .map(ScalarValue::try_from) .collect::>>()?; - Ok(Self { - last: ScalarValue::try_from(data_type)?, + let requirement_satisfied = ordering_req.is_empty(); + ScalarValue::try_from(data_type).map(|last| Self { + last, is_set: false, orderings, ordering_req, + requirement_satisfied, }) } // Updates state with the values in the given row. - fn update_with_new_row(&mut self, row: &[ScalarValue]) -> Result<()> { - let [value, orderings @ ..] = row else { - return internal_err!("Empty row in LAST_VALUE"); - }; - // Update when there is no entry in the state, or we have a "later" - // entry (either according to sort requirements or the order of execution). - if !self.is_set - || self.orderings.is_empty() - || compare_rows( - &self.orderings, - orderings, - &get_sort_options(&self.ordering_req), - )? - .is_lt() - { - self.last = value.clone(); - self.orderings = orderings.to_vec(); - self.is_set = true; - } - Ok(()) + fn update_with_new_row(&mut self, row: &[ScalarValue]) { + self.last = row[0].clone(); + self.orderings = row[1..].to_vec(); + self.is_set = true; } fn get_last_idx(&self, values: &[ArrayRef]) -> Result> { let [value, ordering_values @ ..] = values else { return internal_err!("Empty row in LAST_VALUE"); }; - if self.ordering_req.is_empty() { + if self.requirement_satisfied { // Get last entry according to the order of data: return Ok((!value.is_empty()).then_some(value.len() - 1)); } @@ -538,6 +578,11 @@ impl LastValueAccumulator { let indices = lexsort_to_indices(&sort_columns, Some(1))?; Ok((!indices.is_empty()).then_some(indices.value(0) as _)) } + + fn with_requirement_satisfied(mut self, requirement_satisfied: bool) -> Self { + self.requirement_satisfied = requirement_satisfied; + self + } } impl Accumulator for LastValueAccumulator { @@ -549,10 +594,26 @@ impl Accumulator for LastValueAccumulator { } fn update_batch(&mut self, values: &[ArrayRef]) -> Result<()> { - if let Some(last_idx) = self.get_last_idx(values)? { + if !self.is_set || self.requirement_satisfied { + if let Some(last_idx) = self.get_last_idx(values)? { + let row = get_row_at_idx(values, last_idx)?; + self.update_with_new_row(&row); + } + } else if let Some(last_idx) = self.get_last_idx(values)? { let row = get_row_at_idx(values, last_idx)?; - self.update_with_new_row(&row)?; + let orderings = &row[1..]; + // Update when there is a more recent entry + if compare_rows( + &self.orderings, + orderings, + &get_sort_options(&self.ordering_req), + )? + .is_lt() + { + self.update_with_new_row(&row); + } } + Ok(()) } @@ -583,12 +644,12 @@ impl Accumulator for LastValueAccumulator { // Either there is no existing value, or there is a newer (latest) // version in the new data: if !self.is_set - || compare_rows(last_ordering, &self.orderings, &sort_options)?.is_gt() + || compare_rows(&self.orderings, last_ordering, &sort_options)?.is_lt() { // Update with last value in the state. Note that we should exclude the // is_set flag from the state. Otherwise, we will end up with a state // containing two is_set flags. - self.update_with_new_row(&last_row[0..is_set_idx])?; + self.update_with_new_row(&last_row[0..is_set_idx]); } } Ok(()) diff --git a/datafusion/physical-plan/src/aggregates/mod.rs b/datafusion/physical-plan/src/aggregates/mod.rs index f5bb4fe59b5d..a38044de02e3 100644 --- a/datafusion/physical-plan/src/aggregates/mod.rs +++ b/datafusion/physical-plan/src/aggregates/mod.rs @@ -44,9 +44,9 @@ use datafusion_expr::Accumulator; use datafusion_physical_expr::{ aggregate::is_order_sensitive, equivalence::{collapse_lex_req, ProjectionMapping}, - expressions::{Column, Max, Min, UnKnownColumn}, - physical_exprs_contains, AggregateExpr, EquivalenceProperties, LexOrdering, - LexRequirement, PhysicalExpr, PhysicalSortExpr, PhysicalSortRequirement, + expressions::{Column, FirstValue, LastValue, Max, Min, UnKnownColumn}, + physical_exprs_contains, reverse_order_bys, AggregateExpr, EquivalenceProperties, + LexOrdering, LexRequirement, PhysicalExpr, PhysicalSortExpr, PhysicalSortRequirement, }; use itertools::Itertools; @@ -324,7 +324,7 @@ impl AggregateExec { fn try_new_with_schema( mode: AggregateMode, group_by: PhysicalGroupBy, - aggr_expr: Vec>, + mut aggr_expr: Vec>, filter_expr: Vec>>, input: Arc, input_schema: SchemaRef, @@ -347,7 +347,8 @@ impl AggregateExec { .collect::>(); let req = get_aggregate_exprs_requirement( - &aggr_expr, + &new_requirement, + &mut aggr_expr, &group_by, &input_eq_properties, &mode, @@ -896,6 +897,11 @@ fn finer_ordering( eq_properties.get_finer_ordering(existing_req, &aggr_req) } +/// Concatenates the given slices. +fn concat_slices(lhs: &[T], rhs: &[T]) -> Vec { + [lhs, rhs].concat() +} + /// Get the common requirement that satisfies all the aggregate expressions. /// /// # Parameters @@ -914,14 +920,64 @@ fn finer_ordering( /// A `LexRequirement` instance, which is the requirement that satisfies all the /// aggregate requirements. Returns an error in case of conflicting requirements. fn get_aggregate_exprs_requirement( - aggr_exprs: &[Arc], + prefix_requirement: &[PhysicalSortRequirement], + aggr_exprs: &mut [Arc], group_by: &PhysicalGroupBy, eq_properties: &EquivalenceProperties, agg_mode: &AggregateMode, ) -> Result { let mut requirement = vec![]; - for aggr_expr in aggr_exprs.iter() { - if let Some(finer_ordering) = + for aggr_expr in aggr_exprs.iter_mut() { + let aggr_req = aggr_expr.order_bys().unwrap_or(&[]); + let reverse_aggr_req = reverse_order_bys(aggr_req); + let aggr_req = PhysicalSortRequirement::from_sort_exprs(aggr_req); + let reverse_aggr_req = + PhysicalSortRequirement::from_sort_exprs(&reverse_aggr_req); + if let Some(first_value) = aggr_expr.as_any().downcast_ref::() { + let mut first_value = first_value.clone(); + if eq_properties.ordering_satisfy_requirement(&concat_slices( + prefix_requirement, + &aggr_req, + )) { + first_value = first_value.with_requirement_satisfied(true); + *aggr_expr = Arc::new(first_value) as _; + } else if eq_properties.ordering_satisfy_requirement(&concat_slices( + prefix_requirement, + &reverse_aggr_req, + )) { + // Converting to LAST_VALUE enables more efficient execution + // given the existing ordering: + let mut last_value = first_value.convert_to_last(); + last_value = last_value.with_requirement_satisfied(true); + *aggr_expr = Arc::new(last_value) as _; + } else { + // Requirement is not satisfied with existing ordering. + first_value = first_value.with_requirement_satisfied(false); + *aggr_expr = Arc::new(first_value) as _; + } + } else if let Some(last_value) = aggr_expr.as_any().downcast_ref::() { + let mut last_value = last_value.clone(); + if eq_properties.ordering_satisfy_requirement(&concat_slices( + prefix_requirement, + &aggr_req, + )) { + last_value = last_value.with_requirement_satisfied(true); + *aggr_expr = Arc::new(last_value) as _; + } else if eq_properties.ordering_satisfy_requirement(&concat_slices( + prefix_requirement, + &reverse_aggr_req, + )) { + // Converting to FIRST_VALUE enables more efficient execution + // given the existing ordering: + let mut first_value = last_value.convert_to_first(); + first_value = first_value.with_requirement_satisfied(true); + *aggr_expr = Arc::new(first_value) as _; + } else { + // Requirement is not satisfied with existing ordering. + last_value = last_value.with_requirement_satisfied(false); + *aggr_expr = Arc::new(last_value) as _; + } + } else if let Some(finer_ordering) = finer_ordering(&requirement, aggr_expr, group_by, eq_properties, agg_mode) { requirement = finer_ordering; @@ -2071,7 +2127,7 @@ mod tests { options: options1, }, ]; - let aggr_exprs = order_by_exprs + let mut aggr_exprs = order_by_exprs .into_iter() .map(|order_by_expr| { Arc::new(OrderSensitiveArrayAgg::new( @@ -2086,7 +2142,8 @@ mod tests { .collect::>(); let group_by = PhysicalGroupBy::new_single(vec![]); let res = get_aggregate_exprs_requirement( - &aggr_exprs, + &[], + &mut aggr_exprs, &group_by, &eq_properties, &AggregateMode::Partial, diff --git a/datafusion/sqllogictest/test_files/groupby.slt b/datafusion/sqllogictest/test_files/groupby.slt index bbf21e135fe4..b09ff79e88d5 100644 --- a/datafusion/sqllogictest/test_files/groupby.slt +++ b/datafusion/sqllogictest/test_files/groupby.slt @@ -2508,7 +2508,7 @@ Projection: sales_global.country, ARRAY_AGG(sales_global.amount) ORDER BY [sales ----TableScan: sales_global projection=[country, amount] physical_plan ProjectionExec: expr=[country@0 as country, ARRAY_AGG(sales_global.amount) ORDER BY [sales_global.amount DESC NULLS FIRST]@1 as amounts, FIRST_VALUE(sales_global.amount) ORDER BY [sales_global.amount ASC NULLS LAST]@2 as fv1, LAST_VALUE(sales_global.amount) ORDER BY [sales_global.amount DESC NULLS FIRST]@3 as fv2] ---AggregateExec: mode=Single, gby=[country@0 as country], aggr=[ARRAY_AGG(sales_global.amount), FIRST_VALUE(sales_global.amount), LAST_VALUE(sales_global.amount)] +--AggregateExec: mode=Single, gby=[country@0 as country], aggr=[ARRAY_AGG(sales_global.amount), LAST_VALUE(sales_global.amount), LAST_VALUE(sales_global.amount)] ----SortExec: expr=[amount@1 DESC] ------MemoryExec: partitions=1, partition_sizes=[1] @@ -2539,7 +2539,7 @@ Projection: sales_global.country, ARRAY_AGG(sales_global.amount) ORDER BY [sales ----TableScan: sales_global projection=[country, amount] physical_plan ProjectionExec: expr=[country@0 as country, ARRAY_AGG(sales_global.amount) ORDER BY [sales_global.amount ASC NULLS LAST]@1 as amounts, FIRST_VALUE(sales_global.amount) ORDER BY [sales_global.amount ASC NULLS LAST]@2 as fv1, LAST_VALUE(sales_global.amount) ORDER BY [sales_global.amount DESC NULLS FIRST]@3 as fv2] ---AggregateExec: mode=Single, gby=[country@0 as country], aggr=[ARRAY_AGG(sales_global.amount), FIRST_VALUE(sales_global.amount), LAST_VALUE(sales_global.amount)] +--AggregateExec: mode=Single, gby=[country@0 as country], aggr=[ARRAY_AGG(sales_global.amount), FIRST_VALUE(sales_global.amount), FIRST_VALUE(sales_global.amount)] ----SortExec: expr=[amount@1 ASC NULLS LAST] ------MemoryExec: partitions=1, partition_sizes=[1] @@ -2571,7 +2571,7 @@ Projection: sales_global.country, FIRST_VALUE(sales_global.amount) ORDER BY [sal ----TableScan: sales_global projection=[country, amount] physical_plan ProjectionExec: expr=[country@0 as country, FIRST_VALUE(sales_global.amount) ORDER BY [sales_global.amount ASC NULLS LAST]@1 as fv1, LAST_VALUE(sales_global.amount) ORDER BY [sales_global.amount DESC NULLS FIRST]@2 as fv2, ARRAY_AGG(sales_global.amount) ORDER BY [sales_global.amount ASC NULLS LAST]@3 as amounts] ---AggregateExec: mode=Single, gby=[country@0 as country], aggr=[FIRST_VALUE(sales_global.amount), LAST_VALUE(sales_global.amount), ARRAY_AGG(sales_global.amount)] +--AggregateExec: mode=Single, gby=[country@0 as country], aggr=[FIRST_VALUE(sales_global.amount), FIRST_VALUE(sales_global.amount), ARRAY_AGG(sales_global.amount)] ----SortExec: expr=[amount@1 ASC NULLS LAST] ------MemoryExec: partitions=1, partition_sizes=[1] @@ -2636,7 +2636,7 @@ Projection: sales_global.country, FIRST_VALUE(sales_global.amount) ORDER BY [sal ------TableScan: sales_global projection=[country, ts, amount] physical_plan ProjectionExec: expr=[country@0 as country, FIRST_VALUE(sales_global.amount) ORDER BY [sales_global.ts DESC NULLS FIRST]@1 as fv1, LAST_VALUE(sales_global.amount) ORDER BY [sales_global.ts DESC NULLS FIRST]@2 as lv1, SUM(sales_global.amount) ORDER BY [sales_global.ts DESC NULLS FIRST]@3 as sum1] ---AggregateExec: mode=Single, gby=[country@0 as country], aggr=[FIRST_VALUE(sales_global.amount), LAST_VALUE(sales_global.amount), SUM(sales_global.amount)] +--AggregateExec: mode=Single, gby=[country@0 as country], aggr=[LAST_VALUE(sales_global.amount), FIRST_VALUE(sales_global.amount), SUM(sales_global.amount)] ----MemoryExec: partitions=1, partition_sizes=[1] query TRRR rowsort @@ -2988,7 +2988,7 @@ SortPreservingMergeExec: [country@0 ASC NULLS LAST] ------AggregateExec: mode=FinalPartitioned, gby=[country@0 as country], aggr=[ARRAY_AGG(sales_global.amount), FIRST_VALUE(sales_global.amount), LAST_VALUE(sales_global.amount)] --------CoalesceBatchesExec: target_batch_size=4 ----------RepartitionExec: partitioning=Hash([country@0], 8), input_partitions=8 -------------AggregateExec: mode=Partial, gby=[country@0 as country], aggr=[ARRAY_AGG(sales_global.amount), FIRST_VALUE(sales_global.amount), LAST_VALUE(sales_global.amount)] +------------AggregateExec: mode=Partial, gby=[country@0 as country], aggr=[ARRAY_AGG(sales_global.amount), LAST_VALUE(sales_global.amount), LAST_VALUE(sales_global.amount)] --------------SortExec: expr=[amount@1 DESC] ----------------RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1 ------------------MemoryExec: partitions=1, partition_sizes=[1] @@ -3631,10 +3631,10 @@ Projection: FIRST_VALUE(multiple_ordered_table.a) ORDER BY [multiple_ordered_tab ----TableScan: multiple_ordered_table projection=[a, c, d] physical_plan ProjectionExec: expr=[FIRST_VALUE(multiple_ordered_table.a) ORDER BY [multiple_ordered_table.a ASC NULLS LAST]@1 as first_a, LAST_VALUE(multiple_ordered_table.c) ORDER BY [multiple_ordered_table.c DESC NULLS FIRST]@2 as last_c] ---AggregateExec: mode=FinalPartitioned, gby=[d@0 as d], aggr=[FIRST_VALUE(multiple_ordered_table.a), LAST_VALUE(multiple_ordered_table.c)] +--AggregateExec: mode=FinalPartitioned, gby=[d@0 as d], aggr=[FIRST_VALUE(multiple_ordered_table.a), FIRST_VALUE(multiple_ordered_table.c)] ----CoalesceBatchesExec: target_batch_size=2 ------RepartitionExec: partitioning=Hash([d@0], 8), input_partitions=8 ---------AggregateExec: mode=Partial, gby=[d@2 as d], aggr=[FIRST_VALUE(multiple_ordered_table.a), LAST_VALUE(multiple_ordered_table.c)] +--------AggregateExec: mode=Partial, gby=[d@2 as d], aggr=[FIRST_VALUE(multiple_ordered_table.a), FIRST_VALUE(multiple_ordered_table.c)] ----------RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1 ------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, c, d], output_orderings=[[a@0 ASC NULLS LAST], [c@1 ASC NULLS LAST]], has_header=true From cc3042a6343457036770267f921bb3b6e726956c Mon Sep 17 00:00:00 2001 From: comphead Date: Fri, 29 Dec 2023 22:47:46 -0800 Subject: [PATCH 519/572] minor: remove unused conversion (#8684) Fixes clippy error in main --- datafusion/proto/src/logical_plan/mod.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/datafusion/proto/src/logical_plan/mod.rs b/datafusion/proto/src/logical_plan/mod.rs index 5ee88c3d5328..e8a38784481b 100644 --- a/datafusion/proto/src/logical_plan/mod.rs +++ b/datafusion/proto/src/logical_plan/mod.rs @@ -1650,7 +1650,7 @@ impl AsLogicalPlan for LogicalPlanNode { let csv_options = &csv_opts.writer_options; let csv_writer_options = csv_writer_options_to_proto( csv_options, - (&csv_opts.compression).into(), + &csv_opts.compression, ); let csv_options = file_type_writer_options::FileType::CsvOptions( From 00a679a0533f1f878db43c2a9cdcaa2e92ab859e Mon Sep 17 00:00:00 2001 From: Eduard Karacharov <13005055+korowa@users.noreply.github.com> Date: Sat, 30 Dec 2023 16:08:59 +0200 Subject: [PATCH 520/572] refactor: modified `JoinHashMap` build order for `HashJoinStream` (#8658) * maintaining fifo hashmap in hash join * extended HashJoinExec docstring on build phase * testcases for randomly ordered build side input * trigger ci --- .../physical-plan/src/joins/hash_join.rs | 316 ++++++++++++------ .../src/joins/symmetric_hash_join.rs | 2 + datafusion/physical-plan/src/joins/utils.rs | 78 ++++- 3 files changed, 300 insertions(+), 96 deletions(-) diff --git a/datafusion/physical-plan/src/joins/hash_join.rs b/datafusion/physical-plan/src/joins/hash_join.rs index 13ac06ee301c..374a0ad50700 100644 --- a/datafusion/physical-plan/src/joins/hash_join.rs +++ b/datafusion/physical-plan/src/joins/hash_join.rs @@ -29,7 +29,6 @@ use crate::joins::utils::{ need_produce_result_in_final, JoinHashMap, JoinHashMapType, }; use crate::{ - coalesce_batches::concat_batches, coalesce_partitions::CoalescePartitionsExec, expressions::Column, expressions::PhysicalSortExpr, @@ -52,10 +51,10 @@ use super::{ use arrow::array::{ Array, ArrayRef, BooleanArray, BooleanBufferBuilder, PrimitiveArray, UInt32Array, - UInt32BufferBuilder, UInt64Array, UInt64BufferBuilder, + UInt64Array, }; use arrow::compute::kernels::cmp::{eq, not_distinct}; -use arrow::compute::{and, take, FilterBuilder}; +use arrow::compute::{and, concat_batches, take, FilterBuilder}; use arrow::datatypes::{Schema, SchemaRef}; use arrow::record_batch::RecordBatch; use arrow::util::bit_util; @@ -156,8 +155,48 @@ impl JoinLeftData { /// /// Execution proceeds in 2 stages: /// -/// 1. the **build phase** where a hash table is created from the tuples of the -/// build side. +/// 1. the **build phase** creates a hash table from the tuples of the build side, +/// and single concatenated batch containing data from all fetched record batches. +/// Resulting hash table stores hashed join-key fields for each row as a key, and +/// indices of corresponding rows in concatenated batch. +/// +/// Hash join uses LIFO data structure as a hash table, and in order to retain +/// original build-side input order while obtaining data during probe phase, hash +/// table is updated by iterating batch sequence in reverse order -- it allows to +/// keep rows with smaller indices "on the top" of hash table, and still maintain +/// correct indexing for concatenated build-side data batch. +/// +/// Example of build phase for 3 record batches: +/// +/// +/// ```text +/// +/// Original build-side data Inserting build-side values into hashmap Concatenated build-side batch +/// ┌───────────────────────────┐ +/// hasmap.insert(row-hash, row-idx + offset) │ idx │ +/// ┌───────┐ │ ┌───────┐ │ +/// │ Row 1 │ 1) update_hash for batch 3 with offset 0 │ │ Row 6 │ 0 │ +/// Batch 1 │ │ - hashmap.insert(Row 7, idx 1) │ Batch 3 │ │ │ +/// │ Row 2 │ - hashmap.insert(Row 6, idx 0) │ │ Row 7 │ 1 │ +/// └───────┘ │ └───────┘ │ +/// │ │ +/// ┌───────┐ │ ┌───────┐ │ +/// │ Row 3 │ 2) update_hash for batch 2 with offset 2 │ │ Row 3 │ 2 │ +/// │ │ - hashmap.insert(Row 5, idx 4) │ │ │ │ +/// Batch 2 │ Row 4 │ - hashmap.insert(Row 4, idx 3) │ Batch 2 │ Row 4 │ 3 │ +/// │ │ - hashmap.insert(Row 3, idx 2) │ │ │ │ +/// │ Row 5 │ │ │ Row 5 │ 4 │ +/// └───────┘ │ └───────┘ │ +/// │ │ +/// ┌───────┐ │ ┌───────┐ │ +/// │ Row 6 │ 3) update_hash for batch 1 with offset 5 │ │ Row 1 │ 5 │ +/// Batch 3 │ │ - hashmap.insert(Row 2, idx 5) │ Batch 1 │ │ │ +/// │ Row 7 │ - hashmap.insert(Row 1, idx 6) │ │ Row 2 │ 6 │ +/// └───────┘ │ └───────┘ │ +/// │ │ +/// └───────────────────────────┘ +/// +/// ``` /// /// 2. the **probe phase** where the tuples of the probe side are streamed /// through, checking for matches of the join keys in the hash table. @@ -715,7 +754,10 @@ async fn collect_left_input( let mut hashmap = JoinHashMap::with_capacity(num_rows); let mut hashes_buffer = Vec::new(); let mut offset = 0; - for batch in batches.iter() { + + // Updating hashmap starting from the last batch + let batches_iter = batches.iter().rev(); + for batch in batches_iter.clone() { hashes_buffer.clear(); hashes_buffer.resize(batch.num_rows(), 0); update_hash( @@ -726,19 +768,25 @@ async fn collect_left_input( &random_state, &mut hashes_buffer, 0, + true, )?; offset += batch.num_rows(); } // Merge all batches into a single batch, so we // can directly index into the arrays - let single_batch = concat_batches(&schema, &batches, num_rows)?; + let single_batch = concat_batches(&schema, batches_iter)?; let data = JoinLeftData::new(hashmap, single_batch, reservation); Ok(data) } -/// Updates `hash` with new entries from [RecordBatch] evaluated against the expressions `on`, -/// assuming that the [RecordBatch] corresponds to the `index`th +/// Updates `hash_map` with new entries from `batch` evaluated against the expressions `on` +/// using `offset` as a start value for `batch` row indices. +/// +/// `fifo_hashmap` sets the order of iteration over `batch` rows while updating hashmap, +/// which allows to keep either first (if set to true) or last (if set to false) row index +/// as a chain head for rows with equal hash values. +#[allow(clippy::too_many_arguments)] pub fn update_hash( on: &[Column], batch: &RecordBatch, @@ -747,6 +795,7 @@ pub fn update_hash( random_state: &RandomState, hashes_buffer: &mut Vec, deleted_offset: usize, + fifo_hashmap: bool, ) -> Result<()> where T: JoinHashMapType, @@ -763,28 +812,18 @@ where // For usual JoinHashmap, the implementation is void. hash_map.extend_zero(batch.num_rows()); - // insert hashes to key of the hashmap - let (mut_map, mut_list) = hash_map.get_mut(); - for (row, hash_value) in hash_values.iter().enumerate() { - let item = mut_map.get_mut(*hash_value, |(hash, _)| *hash_value == *hash); - if let Some((_, index)) = item { - // Already exists: add index to next array - let prev_index = *index; - // Store new value inside hashmap - *index = (row + offset + 1) as u64; - // Update chained Vec at row + offset with previous value - mut_list[row + offset - deleted_offset] = prev_index; - } else { - mut_map.insert( - *hash_value, - // store the value + 1 as 0 value reserved for end of list - (*hash_value, (row + offset + 1) as u64), - |(hash, _)| *hash, - ); - // chained list at (row + offset) is already initialized with 0 - // meaning end of list - } + // Updating JoinHashMap from hash values iterator + let hash_values_iter = hash_values + .iter() + .enumerate() + .map(|(i, val)| (i + offset, val)); + + if fifo_hashmap { + hash_map.update_from_iter(hash_values_iter.rev(), deleted_offset); + } else { + hash_map.update_from_iter(hash_values_iter, deleted_offset); } + Ok(()) } @@ -987,6 +1026,7 @@ pub fn build_equal_condition_join_indices( filter: Option<&JoinFilter>, build_side: JoinSide, deleted_offset: Option, + fifo_hashmap: bool, ) -> Result<(UInt64Array, UInt32Array)> { let keys_values = probe_on .iter() @@ -1002,10 +1042,9 @@ pub fn build_equal_condition_join_indices( hashes_buffer.clear(); hashes_buffer.resize(probe_batch.num_rows(), 0); let hash_values = create_hashes(&keys_values, random_state, hashes_buffer)?; - // Using a buffer builder to avoid slower normal builder - let mut build_indices = UInt64BufferBuilder::new(0); - let mut probe_indices = UInt32BufferBuilder::new(0); - // The chained list algorithm generates build indices for each probe row in a reversed sequence as such: + + // In case build-side input has not been inverted while JoinHashMap creation, the chained list algorithm + // will return build indices for each probe row in a reverse order as such: // Build Indices: [5, 4, 3] // Probe Indices: [1, 1, 1] // @@ -1034,44 +1073,17 @@ pub fn build_equal_condition_join_indices( // (5,1) // // With this approach, the lexicographic order on both the probe side and the build side is preserved. - let hash_map = build_hashmap.get_map(); - let next_chain = build_hashmap.get_list(); - for (row, hash_value) in hash_values.iter().enumerate().rev() { - // Get the hash and find it in the build index - - // For every item on the build and probe we check if it matches - // This possibly contains rows with hash collisions, - // So we have to check here whether rows are equal or not - if let Some((_, index)) = - hash_map.get(*hash_value, |(hash, _)| *hash_value == *hash) - { - let mut i = *index - 1; - loop { - let build_row_value = if let Some(offset) = deleted_offset { - // This arguments means that we prune the next index way before here. - if i < offset as u64 { - // End of the list due to pruning - break; - } - i - offset as u64 - } else { - i - }; - build_indices.append(build_row_value); - probe_indices.append(row as u32); - // Follow the chain to get the next index value - let next = next_chain[build_row_value as usize]; - if next == 0 { - // end of list - break; - } - i = next - 1; - } - } - } - // Reversing both sets of indices - build_indices.as_slice_mut().reverse(); - probe_indices.as_slice_mut().reverse(); + let (mut probe_indices, mut build_indices) = if fifo_hashmap { + build_hashmap.get_matched_indices(hash_values.iter().enumerate(), deleted_offset) + } else { + let (mut matched_probe, mut matched_build) = build_hashmap + .get_matched_indices(hash_values.iter().enumerate().rev(), deleted_offset); + + matched_probe.as_slice_mut().reverse(); + matched_build.as_slice_mut().reverse(); + + (matched_probe, matched_build) + }; let left: UInt64Array = PrimitiveArray::new(build_indices.finish().into(), None); let right: UInt32Array = PrimitiveArray::new(probe_indices.finish().into(), None); @@ -1279,6 +1291,7 @@ impl HashJoinStream { self.filter.as_ref(), JoinSide::Left, None, + true, ); let result = match left_right_indices { @@ -1393,7 +1406,9 @@ mod tests { use arrow::array::{ArrayRef, Date32Array, Int32Array, UInt32Builder, UInt64Builder}; use arrow::datatypes::{DataType, Field, Schema}; - use datafusion_common::{assert_batches_sorted_eq, assert_contains, ScalarValue}; + use datafusion_common::{ + assert_batches_eq, assert_batches_sorted_eq, assert_contains, ScalarValue, + }; use datafusion_execution::config::SessionConfig; use datafusion_execution::runtime_env::{RuntimeConfig, RuntimeEnv}; use datafusion_expr::Operator; @@ -1558,7 +1573,9 @@ mod tests { "| 3 | 5 | 9 | 20 | 5 | 80 |", "+----+----+----+----+----+----+", ]; - assert_batches_sorted_eq!(expected, &batches); + + // Inner join output is expected to preserve both inputs order + assert_batches_eq!(expected, &batches); Ok(()) } @@ -1640,7 +1657,48 @@ mod tests { "+----+----+----+----+----+----+", ]; - assert_batches_sorted_eq!(expected, &batches); + // Inner join output is expected to preserve both inputs order + assert_batches_eq!(expected, &batches); + + Ok(()) + } + + #[tokio::test] + async fn join_inner_one_randomly_ordered() -> Result<()> { + let task_ctx = Arc::new(TaskContext::default()); + let left = build_table( + ("a1", &vec![0, 3, 2, 1]), + ("b1", &vec![4, 5, 5, 4]), + ("c1", &vec![6, 9, 8, 7]), + ); + let right = build_table( + ("a2", &vec![20, 30, 10]), + ("b2", &vec![5, 6, 4]), + ("c2", &vec![80, 90, 70]), + ); + let on = vec![( + Column::new_with_schema("b1", &left.schema())?, + Column::new_with_schema("b2", &right.schema())?, + )]; + + let (columns, batches) = + join_collect(left, right, on, &JoinType::Inner, false, task_ctx).await?; + + assert_eq!(columns, vec!["a1", "b1", "c1", "a2", "b2", "c2"]); + + let expected = [ + "+----+----+----+----+----+----+", + "| a1 | b1 | c1 | a2 | b2 | c2 |", + "+----+----+----+----+----+----+", + "| 3 | 5 | 9 | 20 | 5 | 80 |", + "| 2 | 5 | 8 | 20 | 5 | 80 |", + "| 0 | 4 | 6 | 10 | 4 | 70 |", + "| 1 | 4 | 7 | 10 | 4 | 70 |", + "+----+----+----+----+----+----+", + ]; + + // Inner join output is expected to preserve both inputs order + assert_batches_eq!(expected, &batches); Ok(()) } @@ -1686,7 +1744,8 @@ mod tests { "+----+----+----+----+----+----+", ]; - assert_batches_sorted_eq!(expected, &batches); + // Inner join output is expected to preserve both inputs order + assert_batches_eq!(expected, &batches); Ok(()) } @@ -1740,7 +1799,58 @@ mod tests { "+----+----+----+----+----+----+", ]; - assert_batches_sorted_eq!(expected, &batches); + // Inner join output is expected to preserve both inputs order + assert_batches_eq!(expected, &batches); + + Ok(()) + } + + #[tokio::test] + async fn join_inner_one_two_parts_left_randomly_ordered() -> Result<()> { + let task_ctx = Arc::new(TaskContext::default()); + let batch1 = build_table_i32( + ("a1", &vec![0, 3]), + ("b1", &vec![4, 5]), + ("c1", &vec![6, 9]), + ); + let batch2 = build_table_i32( + ("a1", &vec![2, 1]), + ("b1", &vec![5, 4]), + ("c1", &vec![8, 7]), + ); + let schema = batch1.schema(); + + let left = Arc::new( + MemoryExec::try_new(&[vec![batch1], vec![batch2]], schema, None).unwrap(), + ); + let right = build_table( + ("a2", &vec![20, 30, 10]), + ("b2", &vec![5, 6, 4]), + ("c2", &vec![80, 90, 70]), + ); + let on = vec![( + Column::new_with_schema("b1", &left.schema())?, + Column::new_with_schema("b2", &right.schema())?, + )]; + + let (columns, batches) = + join_collect(left, right, on, &JoinType::Inner, false, task_ctx).await?; + + assert_eq!(columns, vec!["a1", "b1", "c1", "a2", "b2", "c2"]); + + let expected = [ + "+----+----+----+----+----+----+", + "| a1 | b1 | c1 | a2 | b2 | c2 |", + "+----+----+----+----+----+----+", + "| 3 | 5 | 9 | 20 | 5 | 80 |", + "| 2 | 5 | 8 | 20 | 5 | 80 |", + "| 0 | 4 | 6 | 10 | 4 | 70 |", + "| 1 | 4 | 7 | 10 | 4 | 70 |", + "+----+----+----+----+----+----+", + ]; + + // Inner join output is expected to preserve both inputs order + assert_batches_eq!(expected, &batches); Ok(()) } @@ -1789,7 +1899,9 @@ mod tests { "| 1 | 4 | 7 | 10 | 4 | 70 |", "+----+----+----+----+----+----+", ]; - assert_batches_sorted_eq!(expected, &batches); + + // Inner join output is expected to preserve both inputs order + assert_batches_eq!(expected, &batches); // second part let stream = join.execute(1, task_ctx.clone())?; @@ -1804,7 +1916,8 @@ mod tests { "+----+----+----+----+----+----+", ]; - assert_batches_sorted_eq!(expected, &batches); + // Inner join output is expected to preserve both inputs order + assert_batches_eq!(expected, &batches); Ok(()) } @@ -2228,12 +2341,14 @@ mod tests { "+----+----+-----+", "| a2 | b2 | c2 |", "+----+----+-----+", - "| 10 | 10 | 100 |", - "| 12 | 10 | 40 |", "| 8 | 8 | 20 |", + "| 12 | 10 | 40 |", + "| 10 | 10 | 100 |", "+----+----+-----+", ]; - assert_batches_sorted_eq!(expected, &batches); + + // RightSemi join output is expected to preserve right input order + assert_batches_eq!(expected, &batches); Ok(()) } @@ -2288,12 +2403,14 @@ mod tests { "+----+----+-----+", "| a2 | b2 | c2 |", "+----+----+-----+", - "| 10 | 10 | 100 |", - "| 12 | 10 | 40 |", "| 8 | 8 | 20 |", + "| 12 | 10 | 40 |", + "| 10 | 10 | 100 |", "+----+----+-----+", ]; - assert_batches_sorted_eq!(expected, &batches); + + // RightSemi join output is expected to preserve right input order + assert_batches_eq!(expected, &batches); // left_table right semi join right_table on left_table.b1 = right_table.b2 on left_table.a1!=9 let filter_expression = Arc::new(BinaryExpr::new( @@ -2314,11 +2431,13 @@ mod tests { "+----+----+-----+", "| a2 | b2 | c2 |", "+----+----+-----+", - "| 10 | 10 | 100 |", "| 12 | 10 | 40 |", + "| 10 | 10 | 100 |", "+----+----+-----+", ]; - assert_batches_sorted_eq!(expected, &batches); + + // RightSemi join output is expected to preserve right input order + assert_batches_eq!(expected, &batches); Ok(()) } @@ -2471,12 +2590,14 @@ mod tests { "+----+----+-----+", "| a2 | b2 | c2 |", "+----+----+-----+", + "| 6 | 6 | 60 |", "| 2 | 2 | 80 |", "| 4 | 4 | 120 |", - "| 6 | 6 | 60 |", "+----+----+-----+", ]; - assert_batches_sorted_eq!(expected, &batches); + + // RightAnti join output is expected to preserve right input order + assert_batches_eq!(expected, &batches); Ok(()) } @@ -2529,14 +2650,16 @@ mod tests { "+----+----+-----+", "| a2 | b2 | c2 |", "+----+----+-----+", - "| 10 | 10 | 100 |", "| 12 | 10 | 40 |", + "| 6 | 6 | 60 |", "| 2 | 2 | 80 |", + "| 10 | 10 | 100 |", "| 4 | 4 | 120 |", - "| 6 | 6 | 60 |", "+----+----+-----+", ]; - assert_batches_sorted_eq!(expected, &batches); + + // RightAnti join output is expected to preserve right input order + assert_batches_eq!(expected, &batches); // left_table right anti join right_table on left_table.b1 = right_table.b2 and right_table.b2!=8 let column_indices = vec![ColumnIndex { @@ -2565,13 +2688,15 @@ mod tests { "+----+----+-----+", "| a2 | b2 | c2 |", "+----+----+-----+", + "| 8 | 8 | 20 |", + "| 6 | 6 | 60 |", "| 2 | 2 | 80 |", "| 4 | 4 | 120 |", - "| 6 | 6 | 60 |", - "| 8 | 8 | 20 |", "+----+----+-----+", ]; - assert_batches_sorted_eq!(expected, &batches); + + // RightAnti join output is expected to preserve right input order + assert_batches_eq!(expected, &batches); Ok(()) } @@ -2734,6 +2859,7 @@ mod tests { None, JoinSide::Left, None, + false, )?; let mut left_ids = UInt64Builder::with_capacity(0); diff --git a/datafusion/physical-plan/src/joins/symmetric_hash_join.rs b/datafusion/physical-plan/src/joins/symmetric_hash_join.rs index f071a7f6015a..2d38c2bd16c3 100644 --- a/datafusion/physical-plan/src/joins/symmetric_hash_join.rs +++ b/datafusion/physical-plan/src/joins/symmetric_hash_join.rs @@ -771,6 +771,7 @@ pub(crate) fn join_with_probe_batch( filter, build_hash_joiner.build_side, Some(build_hash_joiner.deleted_offset), + false, )?; if need_to_produce_result_in_final(build_hash_joiner.build_side, join_type) { record_visited_indices( @@ -883,6 +884,7 @@ impl OneSideHashJoiner { random_state, &mut self.hashes_buffer, self.deleted_offset, + false, )?; Ok(()) } diff --git a/datafusion/physical-plan/src/joins/utils.rs b/datafusion/physical-plan/src/joins/utils.rs index ac805b50e6a5..1e3cf5abb477 100644 --- a/datafusion/physical-plan/src/joins/utils.rs +++ b/datafusion/physical-plan/src/joins/utils.rs @@ -30,7 +30,7 @@ use crate::{ColumnStatistics, ExecutionPlan, Partitioning, Statistics}; use arrow::array::{ downcast_array, new_null_array, Array, BooleanBufferBuilder, UInt32Array, - UInt32Builder, UInt64Array, + UInt32BufferBuilder, UInt32Builder, UInt64Array, UInt64BufferBuilder, }; use arrow::compute; use arrow::datatypes::{Field, Schema, SchemaBuilder}; @@ -148,6 +148,82 @@ pub trait JoinHashMapType { fn get_map(&self) -> &RawTable<(u64, u64)>; /// Returns a reference to the next. fn get_list(&self) -> &Self::NextType; + + /// Updates hashmap from iterator of row indices & row hashes pairs. + fn update_from_iter<'a>( + &mut self, + iter: impl Iterator, + deleted_offset: usize, + ) { + let (mut_map, mut_list) = self.get_mut(); + for (row, hash_value) in iter { + let item = mut_map.get_mut(*hash_value, |(hash, _)| *hash_value == *hash); + if let Some((_, index)) = item { + // Already exists: add index to next array + let prev_index = *index; + // Store new value inside hashmap + *index = (row + 1) as u64; + // Update chained Vec at `row` with previous value + mut_list[row - deleted_offset] = prev_index; + } else { + mut_map.insert( + *hash_value, + // store the value + 1 as 0 value reserved for end of list + (*hash_value, (row + 1) as u64), + |(hash, _)| *hash, + ); + // chained list at `row` is already initialized with 0 + // meaning end of list + } + } + } + + /// Returns all pairs of row indices matched by hash. + /// + /// This method only compares hashes, so additional further check for actual values + /// equality may be required. + fn get_matched_indices<'a>( + &self, + iter: impl Iterator, + deleted_offset: Option, + ) -> (UInt32BufferBuilder, UInt64BufferBuilder) { + let mut input_indices = UInt32BufferBuilder::new(0); + let mut match_indices = UInt64BufferBuilder::new(0); + + let hash_map = self.get_map(); + let next_chain = self.get_list(); + for (row_idx, hash_value) in iter { + // Get the hash and find it in the index + if let Some((_, index)) = + hash_map.get(*hash_value, |(hash, _)| *hash_value == *hash) + { + let mut i = *index - 1; + loop { + let match_row_idx = if let Some(offset) = deleted_offset { + // This arguments means that we prune the next index way before here. + if i < offset as u64 { + // End of the list due to pruning + break; + } + i - offset as u64 + } else { + i + }; + match_indices.append(match_row_idx); + input_indices.append(row_idx as u32); + // Follow the chain to get the next index value + let next = next_chain[match_row_idx as usize]; + if next == 0 { + // end of list + break; + } + i = next - 1; + } + } + } + + (input_indices, match_indices) + } } /// Implementation of `JoinHashMapType` for `JoinHashMap`. From 545275bff316507226c68cb9d5a0739a0d90f32e Mon Sep 17 00:00:00 2001 From: Matthew Turner Date: Sat, 30 Dec 2023 09:12:26 -0500 Subject: [PATCH 521/572] Start setting up tpch planning benchmarks (#8665) * Start setting up tpch planning benchmarks * Add remaining tpch queries * Fix bench function * Clippy --- datafusion/core/benches/sql_planner.rs | 156 +++++++++++++++++++++++++ 1 file changed, 156 insertions(+) diff --git a/datafusion/core/benches/sql_planner.rs b/datafusion/core/benches/sql_planner.rs index 7a41b6bec6f5..1754129a768f 100644 --- a/datafusion/core/benches/sql_planner.rs +++ b/datafusion/core/benches/sql_planner.rs @@ -60,6 +60,104 @@ pub fn create_table_provider(column_prefix: &str, num_columns: usize) -> Arc [(String, Schema); 8] { + let lineitem_schema = Schema::new(vec![ + Field::new("l_orderkey", DataType::Int64, false), + Field::new("l_partkey", DataType::Int64, false), + Field::new("l_suppkey", DataType::Int64, false), + Field::new("l_linenumber", DataType::Int32, false), + Field::new("l_quantity", DataType::Decimal128(15, 2), false), + Field::new("l_extendedprice", DataType::Decimal128(15, 2), false), + Field::new("l_discount", DataType::Decimal128(15, 2), false), + Field::new("l_tax", DataType::Decimal128(15, 2), false), + Field::new("l_returnflag", DataType::Utf8, false), + Field::new("l_linestatus", DataType::Utf8, false), + Field::new("l_shipdate", DataType::Date32, false), + Field::new("l_commitdate", DataType::Date32, false), + Field::new("l_receiptdate", DataType::Date32, false), + Field::new("l_shipinstruct", DataType::Utf8, false), + Field::new("l_shipmode", DataType::Utf8, false), + Field::new("l_comment", DataType::Utf8, false), + ]); + + let orders_schema = Schema::new(vec![ + Field::new("o_orderkey", DataType::Int64, false), + Field::new("o_custkey", DataType::Int64, false), + Field::new("o_orderstatus", DataType::Utf8, false), + Field::new("o_totalprice", DataType::Decimal128(15, 2), false), + Field::new("o_orderdate", DataType::Date32, false), + Field::new("o_orderpriority", DataType::Utf8, false), + Field::new("o_clerk", DataType::Utf8, false), + Field::new("o_shippriority", DataType::Int32, false), + Field::new("o_comment", DataType::Utf8, false), + ]); + + let part_schema = Schema::new(vec![ + Field::new("p_partkey", DataType::Int64, false), + Field::new("p_name", DataType::Utf8, false), + Field::new("p_mfgr", DataType::Utf8, false), + Field::new("p_brand", DataType::Utf8, false), + Field::new("p_type", DataType::Utf8, false), + Field::new("p_size", DataType::Int32, false), + Field::new("p_container", DataType::Utf8, false), + Field::new("p_retailprice", DataType::Decimal128(15, 2), false), + Field::new("p_comment", DataType::Utf8, false), + ]); + + let supplier_schema = Schema::new(vec![ + Field::new("s_suppkey", DataType::Int64, false), + Field::new("s_name", DataType::Utf8, false), + Field::new("s_address", DataType::Utf8, false), + Field::new("s_nationkey", DataType::Int64, false), + Field::new("s_phone", DataType::Utf8, false), + Field::new("s_acctbal", DataType::Decimal128(15, 2), false), + Field::new("s_comment", DataType::Utf8, false), + ]); + + let partsupp_schema = Schema::new(vec![ + Field::new("ps_partkey", DataType::Int64, false), + Field::new("ps_suppkey", DataType::Int64, false), + Field::new("ps_availqty", DataType::Int32, false), + Field::new("ps_supplycost", DataType::Decimal128(15, 2), false), + Field::new("ps_comment", DataType::Utf8, false), + ]); + + let customer_schema = Schema::new(vec![ + Field::new("c_custkey", DataType::Int64, false), + Field::new("c_name", DataType::Utf8, false), + Field::new("c_address", DataType::Utf8, false), + Field::new("c_nationkey", DataType::Int64, false), + Field::new("c_phone", DataType::Utf8, false), + Field::new("c_acctbal", DataType::Decimal128(15, 2), false), + Field::new("c_mktsegment", DataType::Utf8, false), + Field::new("c_comment", DataType::Utf8, false), + ]); + + let nation_schema = Schema::new(vec![ + Field::new("n_nationkey", DataType::Int64, false), + Field::new("n_name", DataType::Utf8, false), + Field::new("n_regionkey", DataType::Int64, false), + Field::new("n_comment", DataType::Utf8, false), + ]); + + let region_schema = Schema::new(vec![ + Field::new("r_regionkey", DataType::Int64, false), + Field::new("r_name", DataType::Utf8, false), + Field::new("r_comment", DataType::Utf8, false), + ]); + + [ + ("lineitem".to_string(), lineitem_schema), + ("orders".to_string(), orders_schema), + ("part".to_string(), part_schema), + ("supplier".to_string(), supplier_schema), + ("partsupp".to_string(), partsupp_schema), + ("customer".to_string(), customer_schema), + ("nation".to_string(), nation_schema), + ("region".to_string(), region_schema), + ] +} + fn create_context() -> SessionContext { let ctx = SessionContext::new(); ctx.register_table("t1", create_table_provider("a", 200)) @@ -68,6 +166,16 @@ fn create_context() -> SessionContext { .unwrap(); ctx.register_table("t700", create_table_provider("c", 700)) .unwrap(); + + let tpch_schemas = create_tpch_schemas(); + tpch_schemas.iter().for_each(|(name, schema)| { + ctx.register_table( + name, + Arc::new(MemTable::try_new(Arc::new(schema.clone()), vec![]).unwrap()), + ) + .unwrap(); + }); + ctx } @@ -115,6 +223,54 @@ fn criterion_benchmark(c: &mut Criterion) { ) }) }); + + let q1_sql = std::fs::read_to_string("../../benchmarks/queries/q1.sql").unwrap(); + let q2_sql = std::fs::read_to_string("../../benchmarks/queries/q2.sql").unwrap(); + let q3_sql = std::fs::read_to_string("../../benchmarks/queries/q3.sql").unwrap(); + let q4_sql = std::fs::read_to_string("../../benchmarks/queries/q4.sql").unwrap(); + let q5_sql = std::fs::read_to_string("../../benchmarks/queries/q5.sql").unwrap(); + let q6_sql = std::fs::read_to_string("../../benchmarks/queries/q6.sql").unwrap(); + let q7_sql = std::fs::read_to_string("../../benchmarks/queries/q7.sql").unwrap(); + let q8_sql = std::fs::read_to_string("../../benchmarks/queries/q8.sql").unwrap(); + let q9_sql = std::fs::read_to_string("../../benchmarks/queries/q9.sql").unwrap(); + let q10_sql = std::fs::read_to_string("../../benchmarks/queries/q10.sql").unwrap(); + let q11_sql = std::fs::read_to_string("../../benchmarks/queries/q11.sql").unwrap(); + let q12_sql = std::fs::read_to_string("../../benchmarks/queries/q12.sql").unwrap(); + let q13_sql = std::fs::read_to_string("../../benchmarks/queries/q13.sql").unwrap(); + let q14_sql = std::fs::read_to_string("../../benchmarks/queries/q14.sql").unwrap(); + // let q15_sql = std::fs::read_to_string("../../benchmarks/queries/q15.sql").unwrap(); + let q16_sql = std::fs::read_to_string("../../benchmarks/queries/q16.sql").unwrap(); + let q17_sql = std::fs::read_to_string("../../benchmarks/queries/q17.sql").unwrap(); + let q18_sql = std::fs::read_to_string("../../benchmarks/queries/q18.sql").unwrap(); + let q19_sql = std::fs::read_to_string("../../benchmarks/queries/q19.sql").unwrap(); + let q20_sql = std::fs::read_to_string("../../benchmarks/queries/q20.sql").unwrap(); + let q21_sql = std::fs::read_to_string("../../benchmarks/queries/q21.sql").unwrap(); + let q22_sql = std::fs::read_to_string("../../benchmarks/queries/q22.sql").unwrap(); + + c.bench_function("physical_plan_tpch", |b| { + b.iter(|| physical_plan(&ctx, &q1_sql)); + b.iter(|| physical_plan(&ctx, &q2_sql)); + b.iter(|| physical_plan(&ctx, &q3_sql)); + b.iter(|| physical_plan(&ctx, &q4_sql)); + b.iter(|| physical_plan(&ctx, &q5_sql)); + b.iter(|| physical_plan(&ctx, &q6_sql)); + b.iter(|| physical_plan(&ctx, &q7_sql)); + b.iter(|| physical_plan(&ctx, &q8_sql)); + b.iter(|| physical_plan(&ctx, &q9_sql)); + b.iter(|| physical_plan(&ctx, &q10_sql)); + b.iter(|| physical_plan(&ctx, &q11_sql)); + b.iter(|| physical_plan(&ctx, &q12_sql)); + b.iter(|| physical_plan(&ctx, &q13_sql)); + b.iter(|| physical_plan(&ctx, &q14_sql)); + // b.iter(|| physical_plan(&ctx, &q15_sql)); + b.iter(|| physical_plan(&ctx, &q16_sql)); + b.iter(|| physical_plan(&ctx, &q17_sql)); + b.iter(|| physical_plan(&ctx, &q18_sql)); + b.iter(|| physical_plan(&ctx, &q19_sql)); + b.iter(|| physical_plan(&ctx, &q20_sql)); + b.iter(|| physical_plan(&ctx, &q21_sql)); + b.iter(|| physical_plan(&ctx, &q22_sql)); + }); } criterion_group!(benches, criterion_benchmark); From 848f6c395afef790880112f809b1443949d4bb0b Mon Sep 17 00:00:00 2001 From: Devin D'Angelo Date: Sun, 31 Dec 2023 07:34:54 -0500 Subject: [PATCH 522/572] update doc (#8686) --- datafusion/core/src/datasource/provider.rs | 6 +++++- 1 file changed, 5 insertions(+), 1 deletion(-) diff --git a/datafusion/core/src/datasource/provider.rs b/datafusion/core/src/datasource/provider.rs index 275523405a09..c1cee849fe5c 100644 --- a/datafusion/core/src/datasource/provider.rs +++ b/datafusion/core/src/datasource/provider.rs @@ -141,7 +141,11 @@ pub trait TableProvider: Sync + Send { /// (though it may return more). Like Projection Pushdown and Filter /// Pushdown, DataFusion pushes `LIMIT`s as far down in the plan as /// possible, called "Limit Pushdown" as some sources can use this - /// information to improve their performance. + /// information to improve their performance. Note that if there are any + /// Inexact filters pushed down, the LIMIT cannot be pushed down. This is + /// because inexact filters do not guarentee that every filtered row is + /// removed, so applying the limit could lead to too few rows being available + /// to return as a final result. async fn scan( &self, state: &SessionState, From 03bd9b462e9068476e704f0056a3761bd9dce3f0 Mon Sep 17 00:00:00 2001 From: Marvin Lanhenke <62298609+marvinlanhenke@users.noreply.github.com> Date: Sun, 31 Dec 2023 13:52:04 +0100 Subject: [PATCH 523/572] Closes #8502: Parallel NDJSON file reading (#8659) * added basic test * added `fn repartitioned` * added basic version of FileOpener * refactor: extract calculate_range * refactor: handle GetResultPayload::Stream * refactor: extract common functions to mod.rs * refactor: use common functions * added docs * added test * clippy * fix: test_chunked_json * fix: sqllogictest * delete imports * update docs --- .../core/src/datasource/file_format/json.rs | 106 ++++++++++++++++- .../core/src/datasource/physical_plan/csv.rs | 98 +++------------- .../core/src/datasource/physical_plan/json.rs | 105 +++++++++++++---- .../core/src/datasource/physical_plan/mod.rs | 107 +++++++++++++++++- datafusion/core/tests/data/empty.json | 0 .../test_files/repartition_scan.slt | 8 +- 6 files changed, 305 insertions(+), 119 deletions(-) create mode 100644 datafusion/core/tests/data/empty.json diff --git a/datafusion/core/src/datasource/file_format/json.rs b/datafusion/core/src/datasource/file_format/json.rs index 3d437bc5fe68..8c02955ad363 100644 --- a/datafusion/core/src/datasource/file_format/json.rs +++ b/datafusion/core/src/datasource/file_format/json.rs @@ -294,16 +294,20 @@ impl DataSink for JsonSink { #[cfg(test)] mod tests { use super::super::test_util::scan_format; - use super::*; - use crate::physical_plan::collect; - use crate::prelude::{SessionConfig, SessionContext}; - use crate::test::object_store::local_unpartitioned_file; - + use arrow::util::pretty; use datafusion_common::cast::as_int64_array; use datafusion_common::stats::Precision; - + use datafusion_common::{assert_batches_eq, internal_err}; use futures::StreamExt; use object_store::local::LocalFileSystem; + use regex::Regex; + use rstest::rstest; + + use super::*; + use crate::execution::options::NdJsonReadOptions; + use crate::physical_plan::collect; + use crate::prelude::{SessionConfig, SessionContext}; + use crate::test::object_store::local_unpartitioned_file; #[tokio::test] async fn read_small_batches() -> Result<()> { @@ -424,4 +428,94 @@ mod tests { .collect::>(); assert_eq!(vec!["a: Int64", "b: Float64", "c: Boolean"], fields); } + + async fn count_num_partitions(ctx: &SessionContext, query: &str) -> Result { + let result = ctx + .sql(&format!("EXPLAIN {query}")) + .await? + .collect() + .await?; + + let plan = format!("{}", &pretty::pretty_format_batches(&result)?); + + let re = Regex::new(r"file_groups=\{(\d+) group").unwrap(); + + if let Some(captures) = re.captures(&plan) { + if let Some(match_) = captures.get(1) { + let count = match_.as_str().parse::().unwrap(); + return Ok(count); + } + } + + internal_err!("Query contains no Exec: file_groups") + } + + #[rstest(n_partitions, case(1), case(2), case(3), case(4))] + #[tokio::test] + async fn it_can_read_ndjson_in_parallel(n_partitions: usize) -> Result<()> { + let config = SessionConfig::new() + .with_repartition_file_scans(true) + .with_repartition_file_min_size(0) + .with_target_partitions(n_partitions); + + let ctx = SessionContext::new_with_config(config); + + let table_path = "tests/data/1.json"; + let options = NdJsonReadOptions::default(); + + ctx.register_json("json_parallel", table_path, options) + .await?; + + let query = "SELECT SUM(a) FROM json_parallel;"; + + let result = ctx.sql(query).await?.collect().await?; + let actual_partitions = count_num_partitions(&ctx, query).await?; + + #[rustfmt::skip] + let expected = [ + "+----------------------+", + "| SUM(json_parallel.a) |", + "+----------------------+", + "| -7 |", + "+----------------------+" + ]; + + assert_batches_eq!(expected, &result); + assert_eq!(n_partitions, actual_partitions); + + Ok(()) + } + + #[rstest(n_partitions, case(1), case(2), case(3), case(4))] + #[tokio::test] + async fn it_can_read_empty_ndjson_in_parallel(n_partitions: usize) -> Result<()> { + let config = SessionConfig::new() + .with_repartition_file_scans(true) + .with_repartition_file_min_size(0) + .with_target_partitions(n_partitions); + + let ctx = SessionContext::new_with_config(config); + + let table_path = "tests/data/empty.json"; + let options = NdJsonReadOptions::default(); + + ctx.register_json("json_parallel_empty", table_path, options) + .await?; + + let query = "SELECT * FROM json_parallel_empty WHERE random() > 0.5;"; + + let result = ctx.sql(query).await?.collect().await?; + let actual_partitions = count_num_partitions(&ctx, query).await?; + + #[rustfmt::skip] + let expected = [ + "++", + "++", + ]; + + assert_batches_eq!(expected, &result); + assert_eq!(1, actual_partitions); + + Ok(()) + } } diff --git a/datafusion/core/src/datasource/physical_plan/csv.rs b/datafusion/core/src/datasource/physical_plan/csv.rs index 0c34d22e9fa9..b28bc7d56688 100644 --- a/datafusion/core/src/datasource/physical_plan/csv.rs +++ b/datafusion/core/src/datasource/physical_plan/csv.rs @@ -19,11 +19,10 @@ use std::any::Any; use std::io::{Read, Seek, SeekFrom}; -use std::ops::Range; use std::sync::Arc; use std::task::Poll; -use super::{FileGroupPartitioner, FileScanConfig}; +use super::{calculate_range, FileGroupPartitioner, FileScanConfig, RangeCalculation}; use crate::datasource::file_format::file_compression_type::FileCompressionType; use crate::datasource::listing::{FileRange, ListingTableUrl}; use crate::datasource::physical_plan::file_stream::{ @@ -318,47 +317,6 @@ impl CsvOpener { } } -/// Returns the offset of the first newline in the object store range [start, end), or the end offset if no newline is found. -async fn find_first_newline( - object_store: &Arc, - location: &object_store::path::Path, - start_byte: usize, - end_byte: usize, -) -> Result { - let options = GetOptions { - range: Some(Range { - start: start_byte, - end: end_byte, - }), - ..Default::default() - }; - - let r = object_store.get_opts(location, options).await?; - let mut input = r.into_stream(); - - let mut buffered = Bytes::new(); - let mut index = 0; - - loop { - if buffered.is_empty() { - match input.next().await { - Some(Ok(b)) => buffered = b, - Some(Err(e)) => return Err(e.into()), - None => return Ok(index), - }; - } - - for byte in &buffered { - if *byte == b'\n' { - return Ok(index); - } - index += 1; - } - - buffered.advance(buffered.len()); - } -} - impl FileOpener for CsvOpener { /// Open a partitioned CSV file. /// @@ -408,44 +366,20 @@ impl FileOpener for CsvOpener { ); } + let store = self.config.object_store.clone(); + Ok(Box::pin(async move { - let file_size = file_meta.object_meta.size; // Current partition contains bytes [start_byte, end_byte) (might contain incomplete lines at boundaries) - let range = match file_meta.range { - None => None, - Some(FileRange { start, end }) => { - let (start, end) = (start as usize, end as usize); - // Partition byte range is [start, end), the boundary might be in the middle of - // some line. Need to find out the exact line boundaries. - let start_delta = if start != 0 { - find_first_newline( - &config.object_store, - file_meta.location(), - start - 1, - file_size, - ) - .await? - } else { - 0 - }; - let end_delta = if end != file_size { - find_first_newline( - &config.object_store, - file_meta.location(), - end - 1, - file_size, - ) - .await? - } else { - 0 - }; - let range = start + start_delta..end + end_delta; - if range.start == range.end { - return Ok( - futures::stream::poll_fn(move |_| Poll::Ready(None)).boxed() - ); - } - Some(range) + + let calculated_range = calculate_range(&file_meta, &store).await?; + + let range = match calculated_range { + RangeCalculation::Range(None) => None, + RangeCalculation::Range(Some(range)) => Some(range), + RangeCalculation::TerminateEarly => { + return Ok( + futures::stream::poll_fn(move |_| Poll::Ready(None)).boxed() + ) } }; @@ -453,10 +387,8 @@ impl FileOpener for CsvOpener { range, ..Default::default() }; - let result = config - .object_store - .get_opts(file_meta.location(), options) - .await?; + + let result = store.get_opts(file_meta.location(), options).await?; match result.payload { GetResultPayload::File(mut file, _) => { diff --git a/datafusion/core/src/datasource/physical_plan/json.rs b/datafusion/core/src/datasource/physical_plan/json.rs index c74fd13e77aa..529632dab85a 100644 --- a/datafusion/core/src/datasource/physical_plan/json.rs +++ b/datafusion/core/src/datasource/physical_plan/json.rs @@ -18,11 +18,11 @@ //! Execution plan for reading line-delimited JSON files use std::any::Any; -use std::io::BufReader; +use std::io::{BufReader, Read, Seek, SeekFrom}; use std::sync::Arc; use std::task::Poll; -use super::FileScanConfig; +use super::{calculate_range, FileGroupPartitioner, FileScanConfig, RangeCalculation}; use crate::datasource::file_format::file_compression_type::FileCompressionType; use crate::datasource::listing::ListingTableUrl; use crate::datasource::physical_plan::file_stream::{ @@ -43,8 +43,8 @@ use datafusion_execution::TaskContext; use datafusion_physical_expr::{EquivalenceProperties, LexOrdering}; use bytes::{Buf, Bytes}; -use futures::{ready, stream, StreamExt, TryStreamExt}; -use object_store; +use futures::{ready, StreamExt, TryStreamExt}; +use object_store::{self, GetOptions}; use object_store::{GetResultPayload, ObjectStore}; use tokio::io::AsyncWriteExt; use tokio::task::JoinSet; @@ -134,6 +134,30 @@ impl ExecutionPlan for NdJsonExec { Ok(self) } + fn repartitioned( + &self, + target_partitions: usize, + config: &datafusion_common::config::ConfigOptions, + ) -> Result>> { + let repartition_file_min_size = config.optimizer.repartition_file_min_size; + let preserve_order_within_groups = self.output_ordering().is_some(); + let file_groups = &self.base_config.file_groups; + + let repartitioned_file_groups_option = FileGroupPartitioner::new() + .with_target_partitions(target_partitions) + .with_preserve_order_within_groups(preserve_order_within_groups) + .with_repartition_file_min_size(repartition_file_min_size) + .repartition_file_groups(file_groups); + + if let Some(repartitioned_file_groups) = repartitioned_file_groups_option { + let mut new_plan = self.clone(); + new_plan.base_config.file_groups = repartitioned_file_groups; + return Ok(Some(Arc::new(new_plan))); + } + + Ok(None) + } + fn execute( &self, partition: usize, @@ -193,54 +217,89 @@ impl JsonOpener { } impl FileOpener for JsonOpener { + /// Open a partitioned NDJSON file. + /// + /// If `file_meta.range` is `None`, the entire file is opened. + /// Else `file_meta.range` is `Some(FileRange{start, end})`, which corresponds to the byte range [start, end) within the file. + /// + /// Note: `start` or `end` might be in the middle of some lines. In such cases, the following rules + /// are applied to determine which lines to read: + /// 1. The first line of the partition is the line in which the index of the first character >= `start`. + /// 2. The last line of the partition is the line in which the byte at position `end - 1` resides. + /// + /// See [`CsvOpener`](super::CsvOpener) for an example. fn open(&self, file_meta: FileMeta) -> Result { let store = self.object_store.clone(); let schema = self.projected_schema.clone(); let batch_size = self.batch_size; - let file_compression_type = self.file_compression_type.to_owned(); + Ok(Box::pin(async move { - let r = store.get(file_meta.location()).await?; - match r.payload { - GetResultPayload::File(file, _) => { - let bytes = file_compression_type.convert_read(file)?; + let calculated_range = calculate_range(&file_meta, &store).await?; + + let range = match calculated_range { + RangeCalculation::Range(None) => None, + RangeCalculation::Range(Some(range)) => Some(range), + RangeCalculation::TerminateEarly => { + return Ok( + futures::stream::poll_fn(move |_| Poll::Ready(None)).boxed() + ) + } + }; + + let options = GetOptions { + range, + ..Default::default() + }; + + let result = store.get_opts(file_meta.location(), options).await?; + + match result.payload { + GetResultPayload::File(mut file, _) => { + let bytes = match file_meta.range { + None => file_compression_type.convert_read(file)?, + Some(_) => { + file.seek(SeekFrom::Start(result.range.start as _))?; + let limit = result.range.end - result.range.start; + file_compression_type.convert_read(file.take(limit as u64))? + } + }; + let reader = ReaderBuilder::new(schema) .with_batch_size(batch_size) .build(BufReader::new(bytes))?; + Ok(futures::stream::iter(reader).boxed()) } GetResultPayload::Stream(s) => { + let s = s.map_err(DataFusionError::from); + let mut decoder = ReaderBuilder::new(schema) .with_batch_size(batch_size) .build_decoder()?; - - let s = s.map_err(DataFusionError::from); let mut input = file_compression_type.convert_stream(s.boxed())?.fuse(); - let mut buffered = Bytes::new(); + let mut buffer = Bytes::new(); - let s = stream::poll_fn(move |cx| { + let s = futures::stream::poll_fn(move |cx| { loop { - if buffered.is_empty() { - buffered = match ready!(input.poll_next_unpin(cx)) { - Some(Ok(b)) => b, + if buffer.is_empty() { + match ready!(input.poll_next_unpin(cx)) { + Some(Ok(b)) => buffer = b, Some(Err(e)) => { return Poll::Ready(Some(Err(e.into()))) } - None => break, + None => {} }; } - let read = buffered.len(); - let decoded = match decoder.decode(buffered.as_ref()) { + let decoded = match decoder.decode(buffer.as_ref()) { + Ok(0) => break, Ok(decoded) => decoded, Err(e) => return Poll::Ready(Some(Err(e))), }; - buffered.advance(decoded); - if decoded != read { - break; - } + buffer.advance(decoded); } Poll::Ready(decoder.flush().transpose()) diff --git a/datafusion/core/src/datasource/physical_plan/mod.rs b/datafusion/core/src/datasource/physical_plan/mod.rs index 5583991355c6..d7be017a1868 100644 --- a/datafusion/core/src/datasource/physical_plan/mod.rs +++ b/datafusion/core/src/datasource/physical_plan/mod.rs @@ -27,6 +27,7 @@ mod json; #[cfg(feature = "parquet")] pub mod parquet; pub use file_groups::FileGroupPartitioner; +use futures::StreamExt; pub(crate) use self::csv::plan_to_csv; pub use self::csv::{CsvConfig, CsvExec, CsvOpener}; @@ -45,6 +46,7 @@ pub use json::{JsonOpener, NdJsonExec}; use std::{ fmt::{Debug, Formatter, Result as FmtResult}, + ops::Range, sync::Arc, vec, }; @@ -72,8 +74,8 @@ use datafusion_physical_expr::PhysicalSortExpr; use datafusion_physical_plan::ExecutionPlan; use log::debug; -use object_store::path::Path; use object_store::ObjectMeta; +use object_store::{path::Path, GetOptions, ObjectStore}; /// The base configurations to provide when creating a physical plan for /// writing to any given file format. @@ -522,6 +524,109 @@ pub fn is_plan_streaming(plan: &Arc) -> Result { } } +/// Represents the possible outcomes of a range calculation. +/// +/// This enum is used to encapsulate the result of calculating the range of +/// bytes to read from an object (like a file) in an object store. +/// +/// Variants: +/// - `Range(Option>)`: +/// Represents a range of bytes to be read. It contains an `Option` wrapping a +/// `Range`. `None` signifies that the entire object should be read, +/// while `Some(range)` specifies the exact byte range to read. +/// - `TerminateEarly`: +/// Indicates that the range calculation determined no further action is +/// necessary, possibly because the calculated range is empty or invalid. +enum RangeCalculation { + Range(Option>), + TerminateEarly, +} + +/// Calculates an appropriate byte range for reading from an object based on the +/// provided metadata. +/// +/// This asynchronous function examines the `FileMeta` of an object in an object store +/// and determines the range of bytes to be read. The range calculation may adjust +/// the start and end points to align with meaningful data boundaries (like newlines). +/// +/// Returns a `Result` wrapping a `RangeCalculation`, which is either a calculated byte range or an indication to terminate early. +/// +/// Returns an `Error` if any part of the range calculation fails, such as issues in reading from the object store or invalid range boundaries. +async fn calculate_range( + file_meta: &FileMeta, + store: &Arc, +) -> Result { + let location = file_meta.location(); + let file_size = file_meta.object_meta.size; + + match file_meta.range { + None => Ok(RangeCalculation::Range(None)), + Some(FileRange { start, end }) => { + let (start, end) = (start as usize, end as usize); + + let start_delta = if start != 0 { + find_first_newline(store, location, start - 1, file_size).await? + } else { + 0 + }; + + let end_delta = if end != file_size { + find_first_newline(store, location, end - 1, file_size).await? + } else { + 0 + }; + + let range = start + start_delta..end + end_delta; + + if range.start == range.end { + return Ok(RangeCalculation::TerminateEarly); + } + + Ok(RangeCalculation::Range(Some(range))) + } + } +} + +/// Asynchronously finds the position of the first newline character in a specified byte range +/// within an object, such as a file, in an object store. +/// +/// This function scans the contents of the object starting from the specified `start` position +/// up to the `end` position, looking for the first occurrence of a newline (`'\n'`) character. +/// It returns the position of the first newline relative to the start of the range. +/// +/// Returns a `Result` wrapping a `usize` that represents the position of the first newline character found within the specified range. If no newline is found, it returns the length of the scanned data, effectively indicating the end of the range. +/// +/// The function returns an `Error` if any issues arise while reading from the object store or processing the data stream. +/// +async fn find_first_newline( + object_store: &Arc, + location: &Path, + start: usize, + end: usize, +) -> Result { + let range = Some(Range { start, end }); + + let options = GetOptions { + range, + ..Default::default() + }; + + let result = object_store.get_opts(location, options).await?; + let mut result_stream = result.into_stream(); + + let mut index = 0; + + while let Some(chunk) = result_stream.next().await.transpose()? { + if let Some(position) = chunk.iter().position(|&byte| byte == b'\n') { + return Ok(index + position); + } + + index += chunk.len(); + } + + Ok(index) +} + #[cfg(test)] mod tests { use arrow_array::cast::AsArray; diff --git a/datafusion/core/tests/data/empty.json b/datafusion/core/tests/data/empty.json new file mode 100644 index 000000000000..e69de29bb2d1 diff --git a/datafusion/sqllogictest/test_files/repartition_scan.slt b/datafusion/sqllogictest/test_files/repartition_scan.slt index 5dcdbb504e76..3cb42c2206ad 100644 --- a/datafusion/sqllogictest/test_files/repartition_scan.slt +++ b/datafusion/sqllogictest/test_files/repartition_scan.slt @@ -198,9 +198,7 @@ select * from json_table; 4 5 -## In the future it would be cool to see the file read as "4" groups with even sizes (offsets) -## but for now it is just one group -## https://github.com/apache/arrow-datafusion/issues/8502 +## Expect to see the scan read the file as "4" groups with even sizes (offsets) query TT EXPLAIN SELECT column1 FROM json_table WHERE column1 <> 42; ---- @@ -210,9 +208,7 @@ Filter: json_table.column1 != Int32(42) physical_plan CoalesceBatchesExec: target_batch_size=8192 --FilterExec: column1@0 != 42 -----RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -------JsonExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/json_table/1.json]]}, projection=[column1] - +----JsonExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/json_table/1.json:0..18], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/json_table/1.json:18..36], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/json_table/1.json:36..54], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/json_table/1.json:54..70]]}, projection=[column1] # Cleanup statement ok From f0af5eb949e2c5fa9f66eb6f6a9fcdf8f7389c9d Mon Sep 17 00:00:00 2001 From: Jay Zhan Date: Sun, 31 Dec 2023 21:50:52 +0800 Subject: [PATCH 524/572] init draft (#8625) Signed-off-by: jayzhan211 --- datafusion/expr/src/built_in_function.rs | 5 +- datafusion/expr/src/signature.rs | 7 ++ .../expr/src/type_coercion/functions.rs | 89 +++++++++++-------- datafusion/sqllogictest/test_files/array.slt | 62 ++++++++++--- 4 files changed, 115 insertions(+), 48 deletions(-) diff --git a/datafusion/expr/src/built_in_function.rs b/datafusion/expr/src/built_in_function.rs index c454a9781eda..e642dae06e4f 100644 --- a/datafusion/expr/src/built_in_function.rs +++ b/datafusion/expr/src/built_in_function.rs @@ -960,7 +960,10 @@ impl BuiltinScalarFunction { Signature::variadic_any(self.volatility()) } BuiltinScalarFunction::ArrayPositions => Signature::any(2, self.volatility()), - BuiltinScalarFunction::ArrayPrepend => Signature::any(2, self.volatility()), + BuiltinScalarFunction::ArrayPrepend => Signature { + type_signature: ElementAndArray, + volatility: self.volatility(), + }, BuiltinScalarFunction::ArrayRepeat => Signature::any(2, self.volatility()), BuiltinScalarFunction::ArrayRemove => Signature::any(2, self.volatility()), BuiltinScalarFunction::ArrayRemoveN => Signature::any(3, self.volatility()), diff --git a/datafusion/expr/src/signature.rs b/datafusion/expr/src/signature.rs index 3f07c300e196..729131bd95e1 100644 --- a/datafusion/expr/src/signature.rs +++ b/datafusion/expr/src/signature.rs @@ -122,6 +122,10 @@ pub enum TypeSignature { /// List dimension of the List/LargeList is equivalent to the number of List. /// List dimension of the non-list is 0. ArrayAndElement, + /// Specialized Signature for ArrayPrepend and similar functions + /// The first argument should be non-list or list, and the second argument should be List/LargeList. + /// The first argument's list dimension should be one dimension less than the second argument's list dimension. + ElementAndArray, } impl TypeSignature { @@ -155,6 +159,9 @@ impl TypeSignature { TypeSignature::ArrayAndElement => { vec!["ArrayAndElement(List, T)".to_string()] } + TypeSignature::ElementAndArray => { + vec!["ElementAndArray(T, List)".to_string()] + } } } diff --git a/datafusion/expr/src/type_coercion/functions.rs b/datafusion/expr/src/type_coercion/functions.rs index f95a30e025b4..fa47c92762bf 100644 --- a/datafusion/expr/src/type_coercion/functions.rs +++ b/datafusion/expr/src/type_coercion/functions.rs @@ -79,6 +79,55 @@ fn get_valid_types( signature: &TypeSignature, current_types: &[DataType], ) -> Result>> { + fn array_append_or_prepend_valid_types( + current_types: &[DataType], + is_append: bool, + ) -> Result>> { + if current_types.len() != 2 { + return Ok(vec![vec![]]); + } + + let (array_type, elem_type) = if is_append { + (¤t_types[0], ¤t_types[1]) + } else { + (¤t_types[1], ¤t_types[0]) + }; + + // We follow Postgres on `array_append(Null, T)`, which is not valid. + if array_type.eq(&DataType::Null) { + return Ok(vec![vec![]]); + } + + // We need to find the coerced base type, mainly for cases like: + // `array_append(List(null), i64)` -> `List(i64)` + let array_base_type = datafusion_common::utils::base_type(array_type); + let elem_base_type = datafusion_common::utils::base_type(elem_type); + let new_base_type = comparison_coercion(&array_base_type, &elem_base_type); + + if new_base_type.is_none() { + return internal_err!( + "Coercion from {array_base_type:?} to {elem_base_type:?} not supported." + ); + } + let new_base_type = new_base_type.unwrap(); + + let array_type = datafusion_common::utils::coerced_type_with_base_type_only( + array_type, + &new_base_type, + ); + + if let DataType::List(ref field) = array_type { + let elem_type = field.data_type(); + if is_append { + Ok(vec![vec![array_type.clone(), elem_type.to_owned()]]) + } else { + Ok(vec![vec![elem_type.to_owned(), array_type.clone()]]) + } + } else { + Ok(vec![vec![]]) + } + } + let valid_types = match signature { TypeSignature::Variadic(valid_types) => valid_types .iter() @@ -112,42 +161,10 @@ fn get_valid_types( TypeSignature::Exact(valid_types) => vec![valid_types.clone()], TypeSignature::ArrayAndElement => { - if current_types.len() != 2 { - return Ok(vec![vec![]]); - } - - let array_type = ¤t_types[0]; - let elem_type = ¤t_types[1]; - - // We follow Postgres on `array_append(Null, T)`, which is not valid. - if array_type.eq(&DataType::Null) { - return Ok(vec![vec![]]); - } - - // We need to find the coerced base type, mainly for cases like: - // `array_append(List(null), i64)` -> `List(i64)` - let array_base_type = datafusion_common::utils::base_type(array_type); - let elem_base_type = datafusion_common::utils::base_type(elem_type); - let new_base_type = comparison_coercion(&array_base_type, &elem_base_type); - - if new_base_type.is_none() { - return internal_err!( - "Coercion from {array_base_type:?} to {elem_base_type:?} not supported." - ); - } - let new_base_type = new_base_type.unwrap(); - - let array_type = datafusion_common::utils::coerced_type_with_base_type_only( - array_type, - &new_base_type, - ); - - if let DataType::List(ref field) = array_type { - let elem_type = field.data_type(); - return Ok(vec![vec![array_type.clone(), elem_type.to_owned()]]); - } else { - return Ok(vec![vec![]]); - } + return array_append_or_prepend_valid_types(current_types, true) + } + TypeSignature::ElementAndArray => { + return array_append_or_prepend_valid_types(current_types, false) } TypeSignature::Any(number) => { if current_types.len() != *number { diff --git a/datafusion/sqllogictest/test_files/array.slt b/datafusion/sqllogictest/test_files/array.slt index b8d89edb49b1..6dab3b3084a9 100644 --- a/datafusion/sqllogictest/test_files/array.slt +++ b/datafusion/sqllogictest/test_files/array.slt @@ -1618,18 +1618,58 @@ select array_append(column1, make_array(1, 11, 111)), array_append(make_array(ma ## array_prepend (aliases: `list_prepend`, `array_push_front`, `list_push_front`) -# TODO: array_prepend with NULLs -# array_prepend scalar function #1 -# query ? -# select array_prepend(4, make_array()); -# ---- -# [4] +# array_prepend with NULLs + +# DuckDB: [4] +# ClickHouse: Null +# Since they dont have the same result, we just follow Postgres, return error +query error +select array_prepend(4, NULL); + +query ? +select array_prepend(4, []); +---- +[4] + +query ? +select array_prepend(4, [null]); +---- +[4, ] + +# DuckDB: [null] +# ClickHouse: [null] +query ? +select array_prepend(null, []); +---- +[] + +query ? +select array_prepend(null, [1]); +---- +[, 1] + +query ? +select array_prepend(null, [[1,2,3]]); +---- +[, [1, 2, 3]] + +# DuckDB: [[]] +# ClickHouse: [[]] +# TODO: We may also return [[]] +query error +select array_prepend([], []); + +# DuckDB: [null] +# ClickHouse: [null] +# TODO: We may also return [null] +query error +select array_prepend(null, null); + +query ? +select array_append([], null); +---- +[] -# array_prepend scalar function #2 -# query ?? -# select array_prepend(make_array(), make_array()), array_prepend(make_array(4), make_array()); -# ---- -# [[]] [[4]] # array_prepend scalar function #3 query ??? From bf3bd9259aa0e93ccc2c79a606207add30d004a4 Mon Sep 17 00:00:00 2001 From: Liang-Chi Hsieh Date: Mon, 1 Jan 2024 00:22:18 -0800 Subject: [PATCH 525/572] Cleanup TreeNode implementations (#8672) * Refactor TreeNode and cleanup some implementations * More * More * Fix clippy * avoid cloning in `TreeNode.children_nodes()` implementations where possible using `Cow` * Remove more unnecessary apply_children * Fix clippy * Remove --------- Co-authored-by: Peter Toth --- datafusion/common/src/tree_node.rs | 33 ++++--- .../enforce_distribution.rs | 32 ++----- .../src/physical_optimizer/enforce_sorting.rs | 33 ++----- .../physical_optimizer/pipeline_checker.rs | 18 +--- .../replace_with_order_preserving_variants.rs | 17 +--- .../src/physical_optimizer/sort_pushdown.rs | 19 +--- datafusion/expr/src/tree_node/expr.rs | 93 ++++++++----------- datafusion/expr/src/tree_node/plan.rs | 20 +--- .../physical-expr/src/sort_properties.rs | 19 +--- datafusion/physical-expr/src/utils/mod.rs | 17 +--- 10 files changed, 97 insertions(+), 204 deletions(-) diff --git a/datafusion/common/src/tree_node.rs b/datafusion/common/src/tree_node.rs index 5da9636ffe18..5f11c8cc1d11 100644 --- a/datafusion/common/src/tree_node.rs +++ b/datafusion/common/src/tree_node.rs @@ -18,6 +18,7 @@ //! This module provides common traits for visiting or rewriting tree //! data structures easily. +use std::borrow::Cow; use std::sync::Arc; use crate::Result; @@ -32,7 +33,10 @@ use crate::Result; /// [`PhysicalExpr`]: https://docs.rs/datafusion/latest/datafusion/physical_plan/trait.PhysicalExpr.html /// [`LogicalPlan`]: https://docs.rs/datafusion-expr/latest/datafusion_expr/logical_plan/enum.LogicalPlan.html /// [`Expr`]: https://docs.rs/datafusion-expr/latest/datafusion_expr/expr/enum.Expr.html -pub trait TreeNode: Sized { +pub trait TreeNode: Sized + Clone { + /// Returns all children of the TreeNode + fn children_nodes(&self) -> Vec>; + /// Use preorder to iterate the node on the tree so that we can /// stop fast for some cases. /// @@ -211,7 +215,17 @@ pub trait TreeNode: Sized { /// Apply the closure `F` to the node's children fn apply_children(&self, op: &mut F) -> Result where - F: FnMut(&Self) -> Result; + F: FnMut(&Self) -> Result, + { + for child in self.children_nodes() { + match op(&child)? { + VisitRecursion::Continue => {} + VisitRecursion::Skip => return Ok(VisitRecursion::Continue), + VisitRecursion::Stop => return Ok(VisitRecursion::Stop), + } + } + Ok(VisitRecursion::Continue) + } /// Apply transform `F` to the node's children, the transform `F` might have a direction(Preorder or Postorder) fn map_children(self, transform: F) -> Result @@ -342,19 +356,8 @@ pub trait DynTreeNode { /// Blanket implementation for Arc for any tye that implements /// [`DynTreeNode`] (such as [`Arc`]) impl TreeNode for Arc { - fn apply_children(&self, op: &mut F) -> Result - where - F: FnMut(&Self) -> Result, - { - for child in self.arc_children() { - match op(&child)? { - VisitRecursion::Continue => {} - VisitRecursion::Skip => return Ok(VisitRecursion::Continue), - VisitRecursion::Stop => return Ok(VisitRecursion::Stop), - } - } - - Ok(VisitRecursion::Continue) + fn children_nodes(&self) -> Vec> { + self.arc_children().into_iter().map(Cow::Owned).collect() } fn map_children(self, transform: F) -> Result diff --git a/datafusion/core/src/physical_optimizer/enforce_distribution.rs b/datafusion/core/src/physical_optimizer/enforce_distribution.rs index d5a086227323..bf5aa7d02272 100644 --- a/datafusion/core/src/physical_optimizer/enforce_distribution.rs +++ b/datafusion/core/src/physical_optimizer/enforce_distribution.rs @@ -21,6 +21,7 @@ //! according to the configuration), this rule increases partition counts in //! the physical plan. +use std::borrow::Cow; use std::fmt; use std::fmt::Formatter; use std::sync::Arc; @@ -47,7 +48,7 @@ use crate::physical_plan::{ }; use arrow::compute::SortOptions; -use datafusion_common::tree_node::{Transformed, TreeNode, VisitRecursion}; +use datafusion_common::tree_node::{Transformed, TreeNode}; use datafusion_expr::logical_plan::JoinType; use datafusion_physical_expr::expressions::{Column, NoOp}; use datafusion_physical_expr::utils::map_columns_before_projection; @@ -1409,18 +1410,8 @@ impl DistributionContext { } impl TreeNode for DistributionContext { - fn apply_children(&self, op: &mut F) -> Result - where - F: FnMut(&Self) -> Result, - { - for child in &self.children_nodes { - match op(child)? { - VisitRecursion::Continue => {} - VisitRecursion::Skip => return Ok(VisitRecursion::Continue), - VisitRecursion::Stop => return Ok(VisitRecursion::Stop), - } - } - Ok(VisitRecursion::Continue) + fn children_nodes(&self) -> Vec> { + self.children_nodes.iter().map(Cow::Borrowed).collect() } fn map_children(mut self, transform: F) -> Result @@ -1483,19 +1474,8 @@ impl PlanWithKeyRequirements { } impl TreeNode for PlanWithKeyRequirements { - fn apply_children(&self, op: &mut F) -> Result - where - F: FnMut(&Self) -> Result, - { - for child in &self.children { - match op(child)? { - VisitRecursion::Continue => {} - VisitRecursion::Skip => return Ok(VisitRecursion::Continue), - VisitRecursion::Stop => return Ok(VisitRecursion::Stop), - } - } - - Ok(VisitRecursion::Continue) + fn children_nodes(&self) -> Vec> { + self.children.iter().map(Cow::Borrowed).collect() } fn map_children(mut self, transform: F) -> Result diff --git a/datafusion/core/src/physical_optimizer/enforce_sorting.rs b/datafusion/core/src/physical_optimizer/enforce_sorting.rs index 77d04a61c59e..f609ddea66cf 100644 --- a/datafusion/core/src/physical_optimizer/enforce_sorting.rs +++ b/datafusion/core/src/physical_optimizer/enforce_sorting.rs @@ -34,6 +34,7 @@ //! in the physical plan. The first sort is unnecessary since its result is overwritten //! by another [`SortExec`]. Therefore, this rule removes it from the physical plan. +use std::borrow::Cow; use std::sync::Arc; use crate::config::ConfigOptions; @@ -57,7 +58,7 @@ use crate::physical_plan::{ with_new_children_if_necessary, Distribution, ExecutionPlan, InputOrderMode, }; -use datafusion_common::tree_node::{Transformed, TreeNode, VisitRecursion}; +use datafusion_common::tree_node::{Transformed, TreeNode}; use datafusion_common::{plan_err, DataFusionError}; use datafusion_physical_expr::{PhysicalSortExpr, PhysicalSortRequirement}; use datafusion_physical_plan::repartition::RepartitionExec; @@ -145,19 +146,8 @@ impl PlanWithCorrespondingSort { } impl TreeNode for PlanWithCorrespondingSort { - fn apply_children(&self, op: &mut F) -> Result - where - F: FnMut(&Self) -> Result, - { - for child in &self.children_nodes { - match op(child)? { - VisitRecursion::Continue => {} - VisitRecursion::Skip => return Ok(VisitRecursion::Continue), - VisitRecursion::Stop => return Ok(VisitRecursion::Stop), - } - } - - Ok(VisitRecursion::Continue) + fn children_nodes(&self) -> Vec> { + self.children_nodes.iter().map(Cow::Borrowed).collect() } fn map_children(mut self, transform: F) -> Result @@ -237,19 +227,8 @@ impl PlanWithCorrespondingCoalescePartitions { } impl TreeNode for PlanWithCorrespondingCoalescePartitions { - fn apply_children(&self, op: &mut F) -> Result - where - F: FnMut(&Self) -> Result, - { - for child in &self.children_nodes { - match op(child)? { - VisitRecursion::Continue => {} - VisitRecursion::Skip => return Ok(VisitRecursion::Continue), - VisitRecursion::Stop => return Ok(VisitRecursion::Stop), - } - } - - Ok(VisitRecursion::Continue) + fn children_nodes(&self) -> Vec> { + self.children_nodes.iter().map(Cow::Borrowed).collect() } fn map_children(mut self, transform: F) -> Result diff --git a/datafusion/core/src/physical_optimizer/pipeline_checker.rs b/datafusion/core/src/physical_optimizer/pipeline_checker.rs index 9e9f647d073f..e281d0e7c23e 100644 --- a/datafusion/core/src/physical_optimizer/pipeline_checker.rs +++ b/datafusion/core/src/physical_optimizer/pipeline_checker.rs @@ -19,6 +19,7 @@ //! infinite sources, if there are any. It will reject non-runnable query plans //! that use pipeline-breaking operators on infinite input(s). +use std::borrow::Cow; use std::sync::Arc; use crate::config::ConfigOptions; @@ -27,7 +28,7 @@ use crate::physical_optimizer::PhysicalOptimizerRule; use crate::physical_plan::{with_new_children_if_necessary, ExecutionPlan}; use datafusion_common::config::OptimizerOptions; -use datafusion_common::tree_node::{Transformed, TreeNode, VisitRecursion}; +use datafusion_common::tree_node::{Transformed, TreeNode}; use datafusion_common::{plan_err, DataFusionError}; use datafusion_physical_expr::intervals::utils::{check_support, is_datatype_supported}; use datafusion_physical_plan::joins::SymmetricHashJoinExec; @@ -91,19 +92,8 @@ impl PipelineStatePropagator { } impl TreeNode for PipelineStatePropagator { - fn apply_children(&self, op: &mut F) -> Result - where - F: FnMut(&Self) -> Result, - { - for child in &self.children { - match op(child)? { - VisitRecursion::Continue => {} - VisitRecursion::Skip => return Ok(VisitRecursion::Continue), - VisitRecursion::Stop => return Ok(VisitRecursion::Stop), - } - } - - Ok(VisitRecursion::Continue) + fn children_nodes(&self) -> Vec> { + self.children.iter().map(Cow::Borrowed).collect() } fn map_children(mut self, transform: F) -> Result diff --git a/datafusion/core/src/physical_optimizer/replace_with_order_preserving_variants.rs b/datafusion/core/src/physical_optimizer/replace_with_order_preserving_variants.rs index 91f3d2abc6ff..e49b358608aa 100644 --- a/datafusion/core/src/physical_optimizer/replace_with_order_preserving_variants.rs +++ b/datafusion/core/src/physical_optimizer/replace_with_order_preserving_variants.rs @@ -19,6 +19,7 @@ //! order-preserving variants when it is helpful; either in terms of //! performance or to accommodate unbounded streams by fixing the pipeline. +use std::borrow::Cow; use std::sync::Arc; use super::utils::is_repartition; @@ -29,7 +30,7 @@ use crate::physical_plan::sorts::sort_preserving_merge::SortPreservingMergeExec; use crate::physical_plan::{with_new_children_if_necessary, ExecutionPlan}; use datafusion_common::config::ConfigOptions; -use datafusion_common::tree_node::{Transformed, TreeNode, VisitRecursion}; +use datafusion_common::tree_node::{Transformed, TreeNode}; use datafusion_physical_plan::unbounded_output; /// For a given `plan`, this object carries the information one needs from its @@ -104,18 +105,8 @@ impl OrderPreservationContext { } impl TreeNode for OrderPreservationContext { - fn apply_children(&self, op: &mut F) -> Result - where - F: FnMut(&Self) -> Result, - { - for child in &self.children_nodes { - match op(child)? { - VisitRecursion::Continue => {} - VisitRecursion::Skip => return Ok(VisitRecursion::Continue), - VisitRecursion::Stop => return Ok(VisitRecursion::Stop), - } - } - Ok(VisitRecursion::Continue) + fn children_nodes(&self) -> Vec> { + self.children_nodes.iter().map(Cow::Borrowed).collect() } fn map_children(mut self, transform: F) -> Result diff --git a/datafusion/core/src/physical_optimizer/sort_pushdown.rs b/datafusion/core/src/physical_optimizer/sort_pushdown.rs index b0013863010a..97ca47baf05f 100644 --- a/datafusion/core/src/physical_optimizer/sort_pushdown.rs +++ b/datafusion/core/src/physical_optimizer/sort_pushdown.rs @@ -15,6 +15,7 @@ // specific language governing permissions and limitations // under the License. +use std::borrow::Cow; use std::sync::Arc; use crate::physical_optimizer::utils::{ @@ -28,7 +29,7 @@ use crate::physical_plan::repartition::RepartitionExec; use crate::physical_plan::sorts::sort::SortExec; use crate::physical_plan::{with_new_children_if_necessary, ExecutionPlan}; -use datafusion_common::tree_node::{Transformed, TreeNode, VisitRecursion}; +use datafusion_common::tree_node::{Transformed, TreeNode}; use datafusion_common::{plan_err, DataFusionError, JoinSide, Result}; use datafusion_expr::JoinType; use datafusion_physical_expr::expressions::Column; @@ -71,20 +72,10 @@ impl SortPushDown { } impl TreeNode for SortPushDown { - fn apply_children(&self, op: &mut F) -> Result - where - F: FnMut(&Self) -> Result, - { - for child in &self.children_nodes { - match op(child)? { - VisitRecursion::Continue => {} - VisitRecursion::Skip => return Ok(VisitRecursion::Continue), - VisitRecursion::Stop => return Ok(VisitRecursion::Stop), - } - } - - Ok(VisitRecursion::Continue) + fn children_nodes(&self) -> Vec> { + self.children_nodes.iter().map(Cow::Borrowed).collect() } + fn map_children(mut self, transform: F) -> Result where F: FnMut(Self) -> Result, diff --git a/datafusion/expr/src/tree_node/expr.rs b/datafusion/expr/src/tree_node/expr.rs index 1098842716b9..56388be58b8a 100644 --- a/datafusion/expr/src/tree_node/expr.rs +++ b/datafusion/expr/src/tree_node/expr.rs @@ -23,17 +23,15 @@ use crate::expr::{ ScalarFunction, ScalarFunctionDefinition, Sort, TryCast, WindowFunction, }; use crate::{Expr, GetFieldAccess}; +use std::borrow::Cow; -use datafusion_common::tree_node::{TreeNode, VisitRecursion}; +use datafusion_common::tree_node::TreeNode; use datafusion_common::{internal_err, DataFusionError, Result}; impl TreeNode for Expr { - fn apply_children(&self, op: &mut F) -> Result - where - F: FnMut(&Self) -> Result, - { - let children = match self { - Expr::Alias(Alias{expr,..}) + fn children_nodes(&self) -> Vec> { + match self { + Expr::Alias(Alias { expr, .. }) | Expr::Not(expr) | Expr::IsNotNull(expr) | Expr::IsTrue(expr) @@ -47,28 +45,26 @@ impl TreeNode for Expr { | Expr::Cast(Cast { expr, .. }) | Expr::TryCast(TryCast { expr, .. }) | Expr::Sort(Sort { expr, .. }) - | Expr::InSubquery(InSubquery{ expr, .. }) => vec![expr.as_ref().clone()], + | Expr::InSubquery(InSubquery { expr, .. }) => vec![Cow::Borrowed(expr)], Expr::GetIndexedField(GetIndexedField { expr, field }) => { - let expr = expr.as_ref().clone(); + let expr = Cow::Borrowed(expr.as_ref()); match field { - GetFieldAccess::ListIndex {key} => { - vec![key.as_ref().clone(), expr] - }, - GetFieldAccess::ListRange {start, stop} => { - vec![start.as_ref().clone(), stop.as_ref().clone(), expr] + GetFieldAccess::ListIndex { key } => { + vec![Cow::Borrowed(key.as_ref()), expr] } - GetFieldAccess::NamedStructField {name: _name} => { + GetFieldAccess::ListRange { start, stop } => { + vec![Cow::Borrowed(start), Cow::Borrowed(stop), expr] + } + GetFieldAccess::NamedStructField { name: _name } => { vec![expr] } } } Expr::GroupingSet(GroupingSet::Rollup(exprs)) - | Expr::GroupingSet(GroupingSet::Cube(exprs)) => exprs.clone(), - Expr::ScalarFunction (ScalarFunction{ args, .. } ) => { - args.clone() - } + | Expr::GroupingSet(GroupingSet::Cube(exprs)) => exprs.iter().map(Cow::Borrowed).collect(), + Expr::ScalarFunction(ScalarFunction { args, .. }) => args.iter().map(Cow::Borrowed).collect(), Expr::GroupingSet(GroupingSet::GroupingSets(lists_of_exprs)) => { - lists_of_exprs.clone().into_iter().flatten().collect() + lists_of_exprs.iter().flatten().map(Cow::Borrowed).collect() } Expr::Column(_) // Treat OuterReferenceColumn as a leaf expression @@ -77,45 +73,49 @@ impl TreeNode for Expr { | Expr::Literal(_) | Expr::Exists { .. } | Expr::ScalarSubquery(_) - | Expr::Wildcard {..} - | Expr::Placeholder (_) => vec![], + | Expr::Wildcard { .. } + | Expr::Placeholder(_) => vec![], Expr::BinaryExpr(BinaryExpr { left, right, .. }) => { - vec![left.as_ref().clone(), right.as_ref().clone()] + vec![Cow::Borrowed(left), Cow::Borrowed(right)] } Expr::Like(Like { expr, pattern, .. }) | Expr::SimilarTo(Like { expr, pattern, .. }) => { - vec![expr.as_ref().clone(), pattern.as_ref().clone()] + vec![Cow::Borrowed(expr), Cow::Borrowed(pattern)] } Expr::Between(Between { expr, low, high, .. }) => vec![ - expr.as_ref().clone(), - low.as_ref().clone(), - high.as_ref().clone(), + Cow::Borrowed(expr), + Cow::Borrowed(low), + Cow::Borrowed(high), ], Expr::Case(case) => { let mut expr_vec = vec![]; if let Some(expr) = case.expr.as_ref() { - expr_vec.push(expr.as_ref().clone()); + expr_vec.push(Cow::Borrowed(expr.as_ref())); }; for (when, then) in case.when_then_expr.iter() { - expr_vec.push(when.as_ref().clone()); - expr_vec.push(then.as_ref().clone()); + expr_vec.push(Cow::Borrowed(when)); + expr_vec.push(Cow::Borrowed(then)); } if let Some(else_expr) = case.else_expr.as_ref() { - expr_vec.push(else_expr.as_ref().clone()); + expr_vec.push(Cow::Borrowed(else_expr)); } expr_vec } - Expr::AggregateFunction(AggregateFunction { args, filter, order_by, .. }) - => { - let mut expr_vec = args.clone(); + Expr::AggregateFunction(AggregateFunction { + args, + filter, + order_by, + .. + }) => { + let mut expr_vec: Vec<_> = args.iter().map(Cow::Borrowed).collect(); if let Some(f) = filter { - expr_vec.push(f.as_ref().clone()); + expr_vec.push(Cow::Borrowed(f)); } if let Some(o) = order_by { - expr_vec.extend(o.clone()); + expr_vec.extend(o.iter().map(Cow::Borrowed).collect::>()); } expr_vec @@ -126,28 +126,17 @@ impl TreeNode for Expr { order_by, .. }) => { - let mut expr_vec = args.clone(); - expr_vec.extend(partition_by.clone()); - expr_vec.extend(order_by.clone()); + let mut expr_vec: Vec<_> = args.iter().map(Cow::Borrowed).collect(); + expr_vec.extend(partition_by.iter().map(Cow::Borrowed).collect::>()); + expr_vec.extend(order_by.iter().map(Cow::Borrowed).collect::>()); expr_vec } Expr::InList(InList { expr, list, .. }) => { - let mut expr_vec = vec![]; - expr_vec.push(expr.as_ref().clone()); - expr_vec.extend(list.clone()); + let mut expr_vec = vec![Cow::Borrowed(expr.as_ref())]; + expr_vec.extend(list.iter().map(Cow::Borrowed).collect::>()); expr_vec } - }; - - for child in children.iter() { - match op(child)? { - VisitRecursion::Continue => {} - VisitRecursion::Skip => return Ok(VisitRecursion::Continue), - VisitRecursion::Stop => return Ok(VisitRecursion::Stop), - } } - - Ok(VisitRecursion::Continue) } fn map_children(self, transform: F) -> Result diff --git a/datafusion/expr/src/tree_node/plan.rs b/datafusion/expr/src/tree_node/plan.rs index c7621bc17833..217116530d4a 100644 --- a/datafusion/expr/src/tree_node/plan.rs +++ b/datafusion/expr/src/tree_node/plan.rs @@ -20,8 +20,13 @@ use crate::LogicalPlan; use datafusion_common::tree_node::{TreeNodeVisitor, VisitRecursion}; use datafusion_common::{tree_node::TreeNode, Result}; +use std::borrow::Cow; impl TreeNode for LogicalPlan { + fn children_nodes(&self) -> Vec> { + self.inputs().into_iter().map(Cow::Borrowed).collect() + } + fn apply(&self, op: &mut F) -> Result where F: FnMut(&Self) -> Result, @@ -91,21 +96,6 @@ impl TreeNode for LogicalPlan { visitor.post_visit(self) } - fn apply_children(&self, op: &mut F) -> Result - where - F: FnMut(&Self) -> Result, - { - for child in self.inputs() { - match op(child)? { - VisitRecursion::Continue => {} - VisitRecursion::Skip => return Ok(VisitRecursion::Continue), - VisitRecursion::Stop => return Ok(VisitRecursion::Stop), - } - } - - Ok(VisitRecursion::Continue) - } - fn map_children(self, transform: F) -> Result where F: FnMut(Self) -> Result, diff --git a/datafusion/physical-expr/src/sort_properties.rs b/datafusion/physical-expr/src/sort_properties.rs index 91238e5b04b4..0205f85dced4 100644 --- a/datafusion/physical-expr/src/sort_properties.rs +++ b/datafusion/physical-expr/src/sort_properties.rs @@ -15,12 +15,13 @@ // specific language governing permissions and limitations // under the License. +use std::borrow::Cow; use std::{ops::Neg, sync::Arc}; use arrow_schema::SortOptions; use crate::PhysicalExpr; -use datafusion_common::tree_node::{TreeNode, VisitRecursion}; +use datafusion_common::tree_node::TreeNode; use datafusion_common::Result; /// To propagate [`SortOptions`] across the [`PhysicalExpr`], it is insufficient @@ -147,7 +148,7 @@ impl Neg for SortProperties { /// It encapsulates the orderings (`state`) associated with the expression (`expr`), and /// orderings of the children expressions (`children_states`). The [`ExprOrdering`] of a parent /// expression is determined based on the [`ExprOrdering`] states of its children expressions. -#[derive(Debug)] +#[derive(Debug, Clone)] pub struct ExprOrdering { pub expr: Arc, pub state: SortProperties, @@ -173,18 +174,8 @@ impl ExprOrdering { } impl TreeNode for ExprOrdering { - fn apply_children(&self, op: &mut F) -> Result - where - F: FnMut(&Self) -> Result, - { - for child in &self.children { - match op(child)? { - VisitRecursion::Continue => {} - VisitRecursion::Skip => return Ok(VisitRecursion::Continue), - VisitRecursion::Stop => return Ok(VisitRecursion::Stop), - } - } - Ok(VisitRecursion::Continue) + fn children_nodes(&self) -> Vec> { + self.children.iter().map(Cow::Borrowed).collect() } fn map_children(mut self, transform: F) -> Result diff --git a/datafusion/physical-expr/src/utils/mod.rs b/datafusion/physical-expr/src/utils/mod.rs index 87ef36558b96..64a62dc7820d 100644 --- a/datafusion/physical-expr/src/utils/mod.rs +++ b/datafusion/physical-expr/src/utils/mod.rs @@ -18,7 +18,7 @@ mod guarantee; pub use guarantee::{Guarantee, LiteralGuarantee}; -use std::borrow::Borrow; +use std::borrow::{Borrow, Cow}; use std::collections::{HashMap, HashSet}; use std::sync::Arc; @@ -154,19 +154,8 @@ impl ExprTreeNode { } impl TreeNode for ExprTreeNode { - fn apply_children(&self, op: &mut F) -> Result - where - F: FnMut(&Self) -> Result, - { - for child in self.children() { - match op(child)? { - VisitRecursion::Continue => {} - VisitRecursion::Skip => return Ok(VisitRecursion::Continue), - VisitRecursion::Stop => return Ok(VisitRecursion::Stop), - } - } - - Ok(VisitRecursion::Continue) + fn children_nodes(&self) -> Vec> { + self.children().iter().map(Cow::Borrowed).collect() } fn map_children(mut self, transform: F) -> Result From 8ae7ddc7f9008db39ad86fe0983026a2ac210a5b Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Mon, 1 Jan 2024 07:13:35 -0500 Subject: [PATCH 526/572] Update sqlparser requirement from 0.40.0 to 0.41.0 (#8647) * Update sqlparser requirement from 0.40.0 to 0.41.0 Updates the requirements on [sqlparser](https://github.com/sqlparser-rs/sqlparser-rs) to permit the latest version. - [Changelog](https://github.com/sqlparser-rs/sqlparser-rs/blob/main/CHANGELOG.md) - [Commits](https://github.com/sqlparser-rs/sqlparser-rs/compare/v0.40.0...v0.40.0) --- updated-dependencies: - dependency-name: sqlparser dependency-type: direct:production ... Signed-off-by: dependabot[bot] * error on unsupported syntax * Update datafusion-cli dependencies * fix test --------- Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> Co-authored-by: Andrew Lamb --- Cargo.toml | 2 +- datafusion-cli/Cargo.lock | 82 +++++++++---------- datafusion/sql/src/statement.rs | 6 ++ .../test_files/repartition_scan.slt | 6 +- 4 files changed, 51 insertions(+), 45 deletions(-) diff --git a/Cargo.toml b/Cargo.toml index 4ee29ea6298c..a87923b6a1a0 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -70,7 +70,7 @@ parquet = { version = "49.0.0", default-features = false, features = ["arrow", " rand = "0.8" rstest = "0.18.0" serde_json = "1" -sqlparser = { version = "0.40.0", features = ["visitor"] } +sqlparser = { version = "0.41.0", features = ["visitor"] } tempfile = "3" thiserror = "1.0.44" url = "2.2" diff --git a/datafusion-cli/Cargo.lock b/datafusion-cli/Cargo.lock index 8e9bbd8a0dfd..e85e8b1a9edb 100644 --- a/datafusion-cli/Cargo.lock +++ b/datafusion-cli/Cargo.lock @@ -385,7 +385,7 @@ checksum = "fdf6721fb0140e4f897002dd086c06f6c27775df19cfe1fccb21181a48fd2c98" dependencies = [ "proc-macro2", "quote", - "syn 2.0.42", + "syn 2.0.43", ] [[package]] @@ -1075,7 +1075,7 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "30d2b3721e861707777e3195b0158f950ae6dc4a27e4d02ff9f67e3eb3de199e" dependencies = [ "quote", - "syn 2.0.42", + "syn 2.0.43", ] [[package]] @@ -1525,9 +1525,9 @@ dependencies = [ [[package]] name = "futures" -version = "0.3.29" +version = "0.3.30" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "da0290714b38af9b4a7b094b8a37086d1b4e61f2df9122c3cad2577669145335" +checksum = "645c6916888f6cb6350d2550b80fb63e734897a8498abe35cfb732b6487804b0" dependencies = [ "futures-channel", "futures-core", @@ -1540,9 +1540,9 @@ dependencies = [ [[package]] name = "futures-channel" -version = "0.3.29" +version = "0.3.30" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "ff4dd66668b557604244583e3e1e1eada8c5c2e96a6d0d6653ede395b78bbacb" +checksum = "eac8f7d7865dcb88bd4373ab671c8cf4508703796caa2b1985a9ca867b3fcb78" dependencies = [ "futures-core", "futures-sink", @@ -1550,15 +1550,15 @@ dependencies = [ [[package]] name = "futures-core" -version = "0.3.29" +version = "0.3.30" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "eb1d22c66e66d9d72e1758f0bd7d4fd0bee04cad842ee34587d68c07e45d088c" +checksum = "dfc6580bb841c5a68e9ef15c77ccc837b40a7504914d52e47b8b0e9bbda25a1d" [[package]] name = "futures-executor" -version = "0.3.29" +version = "0.3.30" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "0f4fb8693db0cf099eadcca0efe2a5a22e4550f98ed16aba6c48700da29597bc" +checksum = "a576fc72ae164fca6b9db127eaa9a9dda0d61316034f33a0a0d4eda41f02b01d" dependencies = [ "futures-core", "futures-task", @@ -1567,32 +1567,32 @@ dependencies = [ [[package]] name = "futures-io" -version = "0.3.29" +version = "0.3.30" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "8bf34a163b5c4c52d0478a4d757da8fb65cabef42ba90515efee0f6f9fa45aaa" +checksum = "a44623e20b9681a318efdd71c299b6b222ed6f231972bfe2f224ebad6311f0c1" [[package]] name = "futures-macro" -version = "0.3.29" +version = "0.3.30" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "53b153fd91e4b0147f4aced87be237c98248656bb01050b96bf3ee89220a8ddb" +checksum = "87750cf4b7a4c0625b1529e4c543c2182106e4dedc60a2a6455e00d212c489ac" dependencies = [ "proc-macro2", "quote", - "syn 2.0.42", + "syn 2.0.43", ] [[package]] name = "futures-sink" -version = "0.3.29" +version = "0.3.30" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "e36d3378ee38c2a36ad710c5d30c2911d752cb941c00c72dbabfb786a7970817" +checksum = "9fb8e00e87438d937621c1c6269e53f536c14d3fbd6a042bb24879e57d474fb5" [[package]] name = "futures-task" -version = "0.3.29" +version = "0.3.30" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "efd193069b0ddadc69c46389b740bbccdd97203899b48d09c5f7969591d6bae2" +checksum = "38d84fa142264698cdce1a9f9172cf383a0c82de1bddcf3092901442c4097004" [[package]] name = "futures-timer" @@ -1602,9 +1602,9 @@ checksum = "e64b03909df88034c26dc1547e8970b91f98bdb65165d6a4e9110d94263dbb2c" [[package]] name = "futures-util" -version = "0.3.29" +version = "0.3.30" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "a19526d624e703a3179b3d322efec918b6246ea0fa51d41124525f00f1cc8104" +checksum = "3d6401deb83407ab3da39eba7e33987a73c3df0c82b4bb5813ee871c19c41d48" dependencies = [ "futures-channel", "futures-core", @@ -2286,9 +2286,9 @@ dependencies = [ [[package]] name = "object" -version = "0.32.1" +version = "0.32.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "9cf5f9dd3933bd50a9e1f149ec995f39ae2c496d31fd772c1fd45ebc27e902b0" +checksum = "a6a622008b6e321afc04970976f62ee297fdbaa6f95318ca343e3eebb9648441" dependencies = [ "memchr", ] @@ -2499,7 +2499,7 @@ checksum = "4359fd9c9171ec6e8c62926d6faaf553a8dc3f64e1507e76da7911b4f6a04405" dependencies = [ "proc-macro2", "quote", - "syn 2.0.42", + "syn 2.0.43", ] [[package]] @@ -3023,7 +3023,7 @@ checksum = "43576ca501357b9b071ac53cdc7da8ef0cbd9493d8df094cd821777ea6e894d3" dependencies = [ "proc-macro2", "quote", - "syn 2.0.42", + "syn 2.0.43", ] [[package]] @@ -3133,9 +3133,9 @@ checksum = "6980e8d7511241f8acf4aebddbb1ff938df5eebe98691418c4468d0b72a96a67" [[package]] name = "sqlparser" -version = "0.40.0" +version = "0.41.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "7c80afe31cdb649e56c0d9bb5503be9166600d68a852c38dd445636d126858e5" +checksum = "5cc2c25a6c66789625ef164b4c7d2e548d627902280c13710d33da8222169964" dependencies = [ "log", "sqlparser_derive", @@ -3189,7 +3189,7 @@ dependencies = [ "proc-macro2", "quote", "rustversion", - "syn 2.0.42", + "syn 2.0.43", ] [[package]] @@ -3211,9 +3211,9 @@ dependencies = [ [[package]] name = "syn" -version = "2.0.42" +version = "2.0.43" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "5b7d0a2c048d661a1a59fcd7355baa232f7ed34e0ee4df2eef3c1c1c0d3852d8" +checksum = "ee659fb5f3d355364e1f3e5bc10fb82068efbf824a1e9d1c9504244a6469ad53" dependencies = [ "proc-macro2", "quote", @@ -3277,22 +3277,22 @@ checksum = "222a222a5bfe1bba4a77b45ec488a741b3cb8872e5e499451fd7d0129c9c7c3d" [[package]] name = "thiserror" -version = "1.0.51" +version = "1.0.52" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "f11c217e1416d6f036b870f14e0413d480dbf28edbee1f877abaf0206af43bb7" +checksum = "83a48fd946b02c0a526b2e9481c8e2a17755e47039164a86c4070446e3a4614d" dependencies = [ "thiserror-impl", ] [[package]] name = "thiserror-impl" -version = "1.0.51" +version = "1.0.52" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "01742297787513b79cf8e29d1056ede1313e2420b7b3b15d0a768b4921f549df" +checksum = "e7fbe9b594d6568a6a1443250a7e67d80b74e1e96f6d1715e1e21cc1888291d3" dependencies = [ "proc-macro2", "quote", - "syn 2.0.42", + "syn 2.0.43", ] [[package]] @@ -3384,7 +3384,7 @@ checksum = "5b8a1e28f2deaa14e508979454cb3a223b10b938b45af148bc0986de36f1923b" dependencies = [ "proc-macro2", "quote", - "syn 2.0.42", + "syn 2.0.43", ] [[package]] @@ -3481,7 +3481,7 @@ checksum = "34704c8d6ebcbc939824180af020566b01a7c01f80641264eba0999f6c2b6be7" dependencies = [ "proc-macro2", "quote", - "syn 2.0.42", + "syn 2.0.43", ] [[package]] @@ -3526,7 +3526,7 @@ checksum = "f03ca4cb38206e2bef0700092660bb74d696f808514dae47fa1467cbfe26e96e" dependencies = [ "proc-macro2", "quote", - "syn 2.0.42", + "syn 2.0.43", ] [[package]] @@ -3680,7 +3680,7 @@ dependencies = [ "once_cell", "proc-macro2", "quote", - "syn 2.0.42", + "syn 2.0.43", "wasm-bindgen-shared", ] @@ -3714,7 +3714,7 @@ checksum = "f0eb82fcb7930ae6219a7ecfd55b217f5f0893484b7a13022ebb2b2bf20b5283" dependencies = [ "proc-macro2", "quote", - "syn 2.0.42", + "syn 2.0.43", "wasm-bindgen-backend", "wasm-bindgen-shared", ] @@ -3978,7 +3978,7 @@ checksum = "9ce1b18ccd8e73a9321186f97e46f9f04b778851177567b1975109d26a08d2a6" dependencies = [ "proc-macro2", "quote", - "syn 2.0.42", + "syn 2.0.43", ] [[package]] diff --git a/datafusion/sql/src/statement.rs b/datafusion/sql/src/statement.rs index 12083554f093..a365d23f435c 100644 --- a/datafusion/sql/src/statement.rs +++ b/datafusion/sql/src/statement.rs @@ -513,7 +513,13 @@ impl<'a, S: ContextProvider> SqlToRel<'a, S> { Statement::StartTransaction { modes, begin: false, + modifier, } => { + if let Some(modifier) = modifier { + return not_impl_err!( + "Transaction modifier not supported: {modifier}" + ); + } let isolation_level: ast::TransactionIsolationLevel = modes .iter() .filter_map(|m: &ast::TransactionMode| match m { diff --git a/datafusion/sqllogictest/test_files/repartition_scan.slt b/datafusion/sqllogictest/test_files/repartition_scan.slt index 3cb42c2206ad..02eccd7c5d06 100644 --- a/datafusion/sqllogictest/test_files/repartition_scan.slt +++ b/datafusion/sqllogictest/test_files/repartition_scan.slt @@ -185,12 +185,12 @@ COPY (VALUES (1), (2), (3), (4), (5)) TO 'test_files/scratch/repartition_scan/j (FORMAT json, SINGLE_FILE_OUTPUT true); statement ok -CREATE EXTERNAL TABLE json_table(column1 int) +CREATE EXTERNAL TABLE json_table (column1 int) STORED AS json LOCATION 'test_files/scratch/repartition_scan/json_table/'; query I -select * from json_table; +select * from "json_table"; ---- 1 2 @@ -200,7 +200,7 @@ select * from json_table; ## Expect to see the scan read the file as "4" groups with even sizes (offsets) query TT -EXPLAIN SELECT column1 FROM json_table WHERE column1 <> 42; +EXPLAIN SELECT column1 FROM "json_table" WHERE column1 <> 42; ---- logical_plan Filter: json_table.column1 != Int32(42) From 4dcfd7dd81153cfc70e5772f70519b7257e31932 Mon Sep 17 00:00:00 2001 From: Jeffrey <22608443+Jefffrey@users.noreply.github.com> Date: Mon, 1 Jan 2024 23:25:37 +1100 Subject: [PATCH 527/572] Update scalar functions doc for extract/datepart (#8682) --- docs/source/user-guide/sql/scalar_functions.md | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/docs/source/user-guide/sql/scalar_functions.md b/docs/source/user-guide/sql/scalar_functions.md index ad4c6ed083bf..629a5f6ecb88 100644 --- a/docs/source/user-guide/sql/scalar_functions.md +++ b/docs/source/user-guide/sql/scalar_functions.md @@ -1410,6 +1410,7 @@ date_part(part, expression) The following date parts are supported: - year + - quarter _(emits value in inclusive range [1, 4] based on which quartile of the year the date is in)_ - month - week _(week of the year)_ - day _(day of the month)_ @@ -1421,6 +1422,7 @@ date_part(part, expression) - nanosecond - dow _(day of the week)_ - doy _(day of the year)_ + - epoch _(seconds since Unix epoch)_ - **expression**: Time expression to operate on. Can be a constant, column, or function. @@ -1448,6 +1450,7 @@ extract(field FROM source) The following date fields are supported: - year + - quarter _(emits value in inclusive range [1, 4] based on which quartile of the year the date is in)_ - month - week _(week of the year)_ - day _(day of the month)_ @@ -1459,6 +1462,7 @@ extract(field FROM source) - nanosecond - dow _(day of the week)_ - doy _(day of the year)_ + - epoch _(seconds since Unix epoch)_ - **source**: Source time expression to operate on. Can be a constant, column, or function. From 77c2180cf6cb83a3e0aa6356b7017a2ed663d4f1 Mon Sep 17 00:00:00 2001 From: Jeffrey <22608443+Jefffrey@users.noreply.github.com> Date: Tue, 2 Jan 2024 04:30:20 +1100 Subject: [PATCH 528/572] Remove DescribeTableStmt in parser in favour of existing functionality from sqlparser-rs (#8703) --- datafusion/core/src/execution/context/mod.rs | 3 --- datafusion/sql/src/parser.rs | 22 -------------------- datafusion/sql/src/statement.rs | 15 +++++++------ 3 files changed, 7 insertions(+), 33 deletions(-) diff --git a/datafusion/core/src/execution/context/mod.rs b/datafusion/core/src/execution/context/mod.rs index 8916fa814a4a..c51f2d132aad 100644 --- a/datafusion/core/src/execution/context/mod.rs +++ b/datafusion/core/src/execution/context/mod.rs @@ -1621,9 +1621,6 @@ impl SessionState { .0 .insert(ObjectName(vec![Ident::from(table.name.as_str())])); } - DFStatement::DescribeTableStmt(table) => { - visitor.insert(&table.table_name) - } DFStatement::CopyTo(CopyToStatement { source, target: _, diff --git a/datafusion/sql/src/parser.rs b/datafusion/sql/src/parser.rs index 9c104ff18a9b..dbd72ec5eb7a 100644 --- a/datafusion/sql/src/parser.rs +++ b/datafusion/sql/src/parser.rs @@ -213,13 +213,6 @@ impl fmt::Display for CreateExternalTable { } } -/// DataFusion extension DDL for `DESCRIBE TABLE` -#[derive(Debug, Clone, PartialEq, Eq)] -pub struct DescribeTableStmt { - /// Table name - pub table_name: ObjectName, -} - /// DataFusion SQL Statement. /// /// This can either be a [`Statement`] from [`sqlparser`] from a @@ -233,8 +226,6 @@ pub enum Statement { Statement(Box), /// Extension: `CREATE EXTERNAL TABLE` CreateExternalTable(CreateExternalTable), - /// Extension: `DESCRIBE TABLE` - DescribeTableStmt(DescribeTableStmt), /// Extension: `COPY TO` CopyTo(CopyToStatement), /// EXPLAIN for extensions @@ -246,7 +237,6 @@ impl fmt::Display for Statement { match self { Statement::Statement(stmt) => write!(f, "{stmt}"), Statement::CreateExternalTable(stmt) => write!(f, "{stmt}"), - Statement::DescribeTableStmt(_) => write!(f, "DESCRIBE TABLE ..."), Statement::CopyTo(stmt) => write!(f, "{stmt}"), Statement::Explain(stmt) => write!(f, "{stmt}"), } @@ -345,10 +335,6 @@ impl<'a> DFParser<'a> { self.parser.next_token(); // COPY self.parse_copy() } - Keyword::DESCRIBE => { - self.parser.next_token(); // DESCRIBE - self.parse_describe() - } Keyword::EXPLAIN => { // (TODO parse all supported statements) self.parser.next_token(); // EXPLAIN @@ -371,14 +357,6 @@ impl<'a> DFParser<'a> { } } - /// Parse a SQL `DESCRIBE` statement - pub fn parse_describe(&mut self) -> Result { - let table_name = self.parser.parse_object_name()?; - Ok(Statement::DescribeTableStmt(DescribeTableStmt { - table_name, - })) - } - /// Parse a SQL `COPY TO` statement pub fn parse_copy(&mut self) -> Result { // parse as a query diff --git a/datafusion/sql/src/statement.rs b/datafusion/sql/src/statement.rs index a365d23f435c..b96553ffbf86 100644 --- a/datafusion/sql/src/statement.rs +++ b/datafusion/sql/src/statement.rs @@ -19,8 +19,8 @@ use std::collections::{BTreeMap, HashMap, HashSet}; use std::sync::Arc; use crate::parser::{ - CopyToSource, CopyToStatement, CreateExternalTable, DFParser, DescribeTableStmt, - ExplainStatement, LexOrdering, Statement as DFStatement, + CopyToSource, CopyToStatement, CreateExternalTable, DFParser, ExplainStatement, + LexOrdering, Statement as DFStatement, }; use crate::planner::{ object_name_to_qualifier, ContextProvider, PlannerContext, SqlToRel, @@ -136,7 +136,6 @@ impl<'a, S: ContextProvider> SqlToRel<'a, S> { match statement { DFStatement::CreateExternalTable(s) => self.external_table_to_plan(s), DFStatement::Statement(s) => self.sql_statement_to_plan(*s), - DFStatement::DescribeTableStmt(s) => self.describe_table_to_plan(s), DFStatement::CopyTo(s) => self.copy_to_plan(s), DFStatement::Explain(ExplainStatement { verbose, @@ -170,6 +169,10 @@ impl<'a, S: ContextProvider> SqlToRel<'a, S> { ) -> Result { let sql = Some(statement.to_string()); match statement { + Statement::ExplainTable { + describe_alias: true, // only parse 'DESCRIBE table_name' and not 'EXPLAIN table_name' + table_name, + } => self.describe_table_to_plan(table_name), Statement::Explain { verbose, statement, @@ -635,11 +638,7 @@ impl<'a, S: ContextProvider> SqlToRel<'a, S> { } } - fn describe_table_to_plan( - &self, - statement: DescribeTableStmt, - ) -> Result { - let DescribeTableStmt { table_name } = statement; + fn describe_table_to_plan(&self, table_name: ObjectName) -> Result { let table_ref = self.object_name_to_table_reference(table_name)?; let table_source = self.context_provider.get_table_source(table_ref)?; From e82707ec5a912dc5f23e9fe89bea5f49ec64688f Mon Sep 17 00:00:00 2001 From: Ashim Sedhain <38435962+asimsedhain@users.noreply.github.com> Date: Mon, 1 Jan 2024 11:44:27 -0600 Subject: [PATCH 529/572] feat: simplify null in list (#8691) GH-8688 --- .../simplify_expressions/expr_simplifier.rs | 20 +++++++++++++++++++ 1 file changed, 20 insertions(+) diff --git a/datafusion/optimizer/src/simplify_expressions/expr_simplifier.rs b/datafusion/optimizer/src/simplify_expressions/expr_simplifier.rs index 5a300e2ff246..7d09aec7e748 100644 --- a/datafusion/optimizer/src/simplify_expressions/expr_simplifier.rs +++ b/datafusion/optimizer/src/simplify_expressions/expr_simplifier.rs @@ -481,6 +481,14 @@ impl<'a, S: SimplifyInfo> TreeNodeRewriter for Simplifier<'a, S> { lit(negated) } + // null in (x, y, z) --> null + // null not in (x, y, z) --> null + Expr::InList(InList { + expr, + list: _, + negated: _, + }) if is_null(&expr) => lit_bool_null(), + // expr IN ((subquery)) -> expr IN (subquery), see ##5529 Expr::InList(InList { expr, @@ -3096,6 +3104,18 @@ mod tests { assert_eq!(simplify(in_list(col("c1"), vec![], false)), lit(false)); assert_eq!(simplify(in_list(col("c1"), vec![], true)), lit(true)); + // null in (...) --> null + assert_eq!( + simplify(in_list(lit_bool_null(), vec![col("c1"), lit(1)], false)), + lit_bool_null() + ); + + // null not in (...) --> null + assert_eq!( + simplify(in_list(lit_bool_null(), vec![col("c1"), lit(1)], true)), + lit_bool_null() + ); + assert_eq!( simplify(in_list(col("c1"), vec![lit(1)], false)), col("c1").eq(lit(1)) From d2b3d1c7538b9fb7ab9cfc0c4c6a238b0dcd91e6 Mon Sep 17 00:00:00 2001 From: Edmondo Porcu Date: Mon, 1 Jan 2024 14:09:41 -0500 Subject: [PATCH 530/572] Rename `expr::window_function::WindowFunction` to `WindowFunctionDefinition`, make structure consistent with ScalarFunction (#8382) * Refactoring WindowFunction into coherent structure with AggregateFunction * One more cargo fmt --------- Co-authored-by: Andrew Lamb --- datafusion/core/src/dataframe/mod.rs | 6 +- .../core/src/physical_optimizer/test_utils.rs | 4 +- datafusion/core/tests/dataframe/mod.rs | 4 +- .../core/tests/fuzz_cases/window_fuzz.rs | 46 +- .../expr/src/built_in_window_function.rs | 207 ++++++++ datafusion/expr/src/expr.rs | 291 ++++++++++- datafusion/expr/src/lib.rs | 6 +- datafusion/expr/src/udwf.rs | 2 +- datafusion/expr/src/utils.rs | 22 +- datafusion/expr/src/window_function.rs | 483 ------------------ .../src/analyzer/count_wildcard_rule.rs | 10 +- .../optimizer/src/analyzer/type_coercion.rs | 8 +- .../optimizer/src/push_down_projection.rs | 6 +- datafusion/physical-plan/src/windows/mod.rs | 28 +- .../proto/src/logical_plan/from_proto.rs | 8 +- datafusion/proto/src/logical_plan/to_proto.rs | 10 +- .../proto/src/physical_plan/from_proto.rs | 10 +- .../tests/cases/roundtrip_logical_plan.rs | 20 +- datafusion/sql/src/expr/function.rs | 19 +- .../substrait/src/logical_plan/consumer.rs | 4 +- 20 files changed, 613 insertions(+), 581 deletions(-) create mode 100644 datafusion/expr/src/built_in_window_function.rs delete mode 100644 datafusion/expr/src/window_function.rs diff --git a/datafusion/core/src/dataframe/mod.rs b/datafusion/core/src/dataframe/mod.rs index 3c3bcd497b7f..5a8c706e32cd 100644 --- a/datafusion/core/src/dataframe/mod.rs +++ b/datafusion/core/src/dataframe/mod.rs @@ -1360,7 +1360,7 @@ mod tests { use datafusion_expr::{ avg, cast, count, count_distinct, create_udf, expr, lit, max, min, sum, BuiltInWindowFunction, ScalarFunctionImplementation, Volatility, WindowFrame, - WindowFunction, + WindowFunctionDefinition, }; use datafusion_physical_expr::expressions::Column; use datafusion_physical_plan::get_plan_string; @@ -1525,7 +1525,9 @@ mod tests { // build plan using Table API let t = test_table().await?; let first_row = Expr::WindowFunction(expr::WindowFunction::new( - WindowFunction::BuiltInWindowFunction(BuiltInWindowFunction::FirstValue), + WindowFunctionDefinition::BuiltInWindowFunction( + BuiltInWindowFunction::FirstValue, + ), vec![col("aggregate_test_100.c1")], vec![col("aggregate_test_100.c2")], vec![], diff --git a/datafusion/core/src/physical_optimizer/test_utils.rs b/datafusion/core/src/physical_optimizer/test_utils.rs index 6e14cca21fed..debafefe39ab 100644 --- a/datafusion/core/src/physical_optimizer/test_utils.rs +++ b/datafusion/core/src/physical_optimizer/test_utils.rs @@ -41,7 +41,7 @@ use crate::prelude::{CsvReadOptions, SessionContext}; use arrow_schema::{Schema, SchemaRef, SortOptions}; use datafusion_common::{JoinType, Statistics}; use datafusion_execution::object_store::ObjectStoreUrl; -use datafusion_expr::{AggregateFunction, WindowFrame, WindowFunction}; +use datafusion_expr::{AggregateFunction, WindowFrame, WindowFunctionDefinition}; use datafusion_physical_expr::expressions::col; use datafusion_physical_expr::{PhysicalExpr, PhysicalSortExpr}; @@ -234,7 +234,7 @@ pub fn bounded_window_exec( Arc::new( crate::physical_plan::windows::BoundedWindowAggExec::try_new( vec![create_window_expr( - &WindowFunction::AggregateFunction(AggregateFunction::Count), + &WindowFunctionDefinition::AggregateFunction(AggregateFunction::Count), "count".to_owned(), &[col(col_name, &schema).unwrap()], &[], diff --git a/datafusion/core/tests/dataframe/mod.rs b/datafusion/core/tests/dataframe/mod.rs index ba661aa2445c..cca23ac6847c 100644 --- a/datafusion/core/tests/dataframe/mod.rs +++ b/datafusion/core/tests/dataframe/mod.rs @@ -45,7 +45,7 @@ use datafusion_expr::expr::{GroupingSet, Sort}; use datafusion_expr::{ array_agg, avg, col, count, exists, expr, in_subquery, lit, max, out_ref_col, scalar_subquery, sum, wildcard, AggregateFunction, Expr, ExprSchemable, WindowFrame, - WindowFrameBound, WindowFrameUnits, WindowFunction, + WindowFrameBound, WindowFrameUnits, WindowFunctionDefinition, }; use datafusion_physical_expr::var_provider::{VarProvider, VarType}; @@ -170,7 +170,7 @@ async fn test_count_wildcard_on_window() -> Result<()> { .table("t1") .await? .select(vec![Expr::WindowFunction(expr::WindowFunction::new( - WindowFunction::AggregateFunction(AggregateFunction::Count), + WindowFunctionDefinition::AggregateFunction(AggregateFunction::Count), vec![wildcard()], vec![], vec![Expr::Sort(Sort::new(Box::new(col("a")), false, true))], diff --git a/datafusion/core/tests/fuzz_cases/window_fuzz.rs b/datafusion/core/tests/fuzz_cases/window_fuzz.rs index 44ff71d02392..3037b4857a3b 100644 --- a/datafusion/core/tests/fuzz_cases/window_fuzz.rs +++ b/datafusion/core/tests/fuzz_cases/window_fuzz.rs @@ -33,7 +33,7 @@ use datafusion_common::{Result, ScalarValue}; use datafusion_expr::type_coercion::aggregates::coerce_types; use datafusion_expr::{ AggregateFunction, BuiltInWindowFunction, WindowFrame, WindowFrameBound, - WindowFrameUnits, WindowFunction, + WindowFrameUnits, WindowFunctionDefinition, }; use datafusion_physical_expr::expressions::{cast, col, lit}; use datafusion_physical_expr::{PhysicalExpr, PhysicalSortExpr}; @@ -143,7 +143,7 @@ fn get_random_function( schema: &SchemaRef, rng: &mut StdRng, is_linear: bool, -) -> (WindowFunction, Vec>, String) { +) -> (WindowFunctionDefinition, Vec>, String) { let mut args = if is_linear { // In linear test for the test version with WindowAggExec we use insert SortExecs to the plan to be able to generate // same result with BoundedWindowAggExec which doesn't use any SortExec. To make result @@ -159,28 +159,28 @@ fn get_random_function( window_fn_map.insert( "sum", ( - WindowFunction::AggregateFunction(AggregateFunction::Sum), + WindowFunctionDefinition::AggregateFunction(AggregateFunction::Sum), vec![], ), ); window_fn_map.insert( "count", ( - WindowFunction::AggregateFunction(AggregateFunction::Count), + WindowFunctionDefinition::AggregateFunction(AggregateFunction::Count), vec![], ), ); window_fn_map.insert( "min", ( - WindowFunction::AggregateFunction(AggregateFunction::Min), + WindowFunctionDefinition::AggregateFunction(AggregateFunction::Min), vec![], ), ); window_fn_map.insert( "max", ( - WindowFunction::AggregateFunction(AggregateFunction::Max), + WindowFunctionDefinition::AggregateFunction(AggregateFunction::Max), vec![], ), ); @@ -191,28 +191,36 @@ fn get_random_function( window_fn_map.insert( "row_number", ( - WindowFunction::BuiltInWindowFunction(BuiltInWindowFunction::RowNumber), + WindowFunctionDefinition::BuiltInWindowFunction( + BuiltInWindowFunction::RowNumber, + ), vec![], ), ); window_fn_map.insert( "rank", ( - WindowFunction::BuiltInWindowFunction(BuiltInWindowFunction::Rank), + WindowFunctionDefinition::BuiltInWindowFunction( + BuiltInWindowFunction::Rank, + ), vec![], ), ); window_fn_map.insert( "dense_rank", ( - WindowFunction::BuiltInWindowFunction(BuiltInWindowFunction::DenseRank), + WindowFunctionDefinition::BuiltInWindowFunction( + BuiltInWindowFunction::DenseRank, + ), vec![], ), ); window_fn_map.insert( "lead", ( - WindowFunction::BuiltInWindowFunction(BuiltInWindowFunction::Lead), + WindowFunctionDefinition::BuiltInWindowFunction( + BuiltInWindowFunction::Lead, + ), vec![ lit(ScalarValue::Int64(Some(rng.gen_range(1..10)))), lit(ScalarValue::Int64(Some(rng.gen_range(1..1000)))), @@ -222,7 +230,9 @@ fn get_random_function( window_fn_map.insert( "lag", ( - WindowFunction::BuiltInWindowFunction(BuiltInWindowFunction::Lag), + WindowFunctionDefinition::BuiltInWindowFunction( + BuiltInWindowFunction::Lag, + ), vec![ lit(ScalarValue::Int64(Some(rng.gen_range(1..10)))), lit(ScalarValue::Int64(Some(rng.gen_range(1..1000)))), @@ -233,21 +243,27 @@ fn get_random_function( window_fn_map.insert( "first_value", ( - WindowFunction::BuiltInWindowFunction(BuiltInWindowFunction::FirstValue), + WindowFunctionDefinition::BuiltInWindowFunction( + BuiltInWindowFunction::FirstValue, + ), vec![], ), ); window_fn_map.insert( "last_value", ( - WindowFunction::BuiltInWindowFunction(BuiltInWindowFunction::LastValue), + WindowFunctionDefinition::BuiltInWindowFunction( + BuiltInWindowFunction::LastValue, + ), vec![], ), ); window_fn_map.insert( "nth_value", ( - WindowFunction::BuiltInWindowFunction(BuiltInWindowFunction::NthValue), + WindowFunctionDefinition::BuiltInWindowFunction( + BuiltInWindowFunction::NthValue, + ), vec![lit(ScalarValue::Int64(Some(rng.gen_range(1..10))))], ), ); @@ -255,7 +271,7 @@ fn get_random_function( let rand_fn_idx = rng.gen_range(0..window_fn_map.len()); let fn_name = window_fn_map.keys().collect::>()[rand_fn_idx]; let (window_fn, new_args) = window_fn_map.values().collect::>()[rand_fn_idx]; - if let WindowFunction::AggregateFunction(f) = window_fn { + if let WindowFunctionDefinition::AggregateFunction(f) = window_fn { let a = args[0].clone(); let dt = a.data_type(schema.as_ref()).unwrap(); let sig = f.signature(); diff --git a/datafusion/expr/src/built_in_window_function.rs b/datafusion/expr/src/built_in_window_function.rs new file mode 100644 index 000000000000..a03e3d2d24a9 --- /dev/null +++ b/datafusion/expr/src/built_in_window_function.rs @@ -0,0 +1,207 @@ +// 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. + +//! Built-in functions module contains all the built-in functions definitions. + +use std::fmt; +use std::str::FromStr; + +use crate::type_coercion::functions::data_types; +use crate::utils; +use crate::{Signature, TypeSignature, Volatility}; +use datafusion_common::{plan_datafusion_err, plan_err, DataFusionError, Result}; + +use arrow::datatypes::DataType; + +use strum_macros::EnumIter; + +impl fmt::Display for BuiltInWindowFunction { + fn fmt(&self, f: &mut fmt::Formatter) -> fmt::Result { + write!(f, "{}", self.name()) + } +} + +/// A [window function] built in to DataFusion +/// +/// [window function]: https://en.wikipedia.org/wiki/Window_function_(SQL) +#[derive(Debug, Clone, PartialEq, Eq, Hash, EnumIter)] +pub enum BuiltInWindowFunction { + /// number of the current row within its partition, counting from 1 + RowNumber, + /// rank of the current row with gaps; same as row_number of its first peer + Rank, + /// rank of the current row without gaps; this function counts peer groups + DenseRank, + /// relative rank of the current row: (rank - 1) / (total rows - 1) + PercentRank, + /// relative rank of the current row: (number of rows preceding or peer with current row) / (total rows) + CumeDist, + /// integer ranging from 1 to the argument value, dividing the partition as equally as possible + Ntile, + /// returns value evaluated at the row that is offset rows before the current row within the partition; + /// if there is no such row, instead return default (which must be of the same type as value). + /// Both offset and default are evaluated with respect to the current row. + /// If omitted, offset defaults to 1 and default to null + Lag, + /// returns value evaluated at the row that is offset rows after the current row within the partition; + /// if there is no such row, instead return default (which must be of the same type as value). + /// Both offset and default are evaluated with respect to the current row. + /// If omitted, offset defaults to 1 and default to null + Lead, + /// returns value evaluated at the row that is the first row of the window frame + FirstValue, + /// returns value evaluated at the row that is the last row of the window frame + LastValue, + /// returns value evaluated at the row that is the nth row of the window frame (counting from 1); null if no such row + NthValue, +} + +impl BuiltInWindowFunction { + fn name(&self) -> &str { + use BuiltInWindowFunction::*; + match self { + RowNumber => "ROW_NUMBER", + Rank => "RANK", + DenseRank => "DENSE_RANK", + PercentRank => "PERCENT_RANK", + CumeDist => "CUME_DIST", + Ntile => "NTILE", + Lag => "LAG", + Lead => "LEAD", + FirstValue => "FIRST_VALUE", + LastValue => "LAST_VALUE", + NthValue => "NTH_VALUE", + } + } +} + +impl FromStr for BuiltInWindowFunction { + type Err = DataFusionError; + fn from_str(name: &str) -> Result { + Ok(match name.to_uppercase().as_str() { + "ROW_NUMBER" => BuiltInWindowFunction::RowNumber, + "RANK" => BuiltInWindowFunction::Rank, + "DENSE_RANK" => BuiltInWindowFunction::DenseRank, + "PERCENT_RANK" => BuiltInWindowFunction::PercentRank, + "CUME_DIST" => BuiltInWindowFunction::CumeDist, + "NTILE" => BuiltInWindowFunction::Ntile, + "LAG" => BuiltInWindowFunction::Lag, + "LEAD" => BuiltInWindowFunction::Lead, + "FIRST_VALUE" => BuiltInWindowFunction::FirstValue, + "LAST_VALUE" => BuiltInWindowFunction::LastValue, + "NTH_VALUE" => BuiltInWindowFunction::NthValue, + _ => return plan_err!("There is no built-in window function named {name}"), + }) + } +} + +/// Returns the datatype of the built-in window function +impl BuiltInWindowFunction { + pub fn return_type(&self, input_expr_types: &[DataType]) -> Result { + // Note that this function *must* return the same type that the respective physical expression returns + // or the execution panics. + + // verify that this is a valid set of data types for this function + data_types(input_expr_types, &self.signature()) + // original errors are all related to wrong function signature + // aggregate them for better error message + .map_err(|_| { + plan_datafusion_err!( + "{}", + utils::generate_signature_error_msg( + &format!("{self}"), + self.signature(), + input_expr_types, + ) + ) + })?; + + match self { + BuiltInWindowFunction::RowNumber + | BuiltInWindowFunction::Rank + | BuiltInWindowFunction::DenseRank => Ok(DataType::UInt64), + BuiltInWindowFunction::PercentRank | BuiltInWindowFunction::CumeDist => { + Ok(DataType::Float64) + } + BuiltInWindowFunction::Ntile => Ok(DataType::UInt64), + BuiltInWindowFunction::Lag + | BuiltInWindowFunction::Lead + | BuiltInWindowFunction::FirstValue + | BuiltInWindowFunction::LastValue + | BuiltInWindowFunction::NthValue => Ok(input_expr_types[0].clone()), + } + } + + /// the signatures supported by the built-in window function `fun`. + pub fn signature(&self) -> Signature { + // note: the physical expression must accept the type returned by this function or the execution panics. + match self { + BuiltInWindowFunction::RowNumber + | BuiltInWindowFunction::Rank + | BuiltInWindowFunction::DenseRank + | BuiltInWindowFunction::PercentRank + | BuiltInWindowFunction::CumeDist => Signature::any(0, Volatility::Immutable), + BuiltInWindowFunction::Lag | BuiltInWindowFunction::Lead => { + Signature::one_of( + vec![ + TypeSignature::Any(1), + TypeSignature::Any(2), + TypeSignature::Any(3), + ], + Volatility::Immutable, + ) + } + BuiltInWindowFunction::FirstValue | BuiltInWindowFunction::LastValue => { + Signature::any(1, Volatility::Immutable) + } + BuiltInWindowFunction::Ntile => Signature::uniform( + 1, + vec![ + DataType::UInt64, + DataType::UInt32, + DataType::UInt16, + DataType::UInt8, + DataType::Int64, + DataType::Int32, + DataType::Int16, + DataType::Int8, + ], + Volatility::Immutable, + ), + BuiltInWindowFunction::NthValue => Signature::any(2, Volatility::Immutable), + } + } +} + +#[cfg(test)] +mod tests { + use super::*; + use strum::IntoEnumIterator; + #[test] + // Test for BuiltInWindowFunction's Display and from_str() implementations. + // For each variant in BuiltInWindowFunction, it converts the variant to a string + // and then back to a variant. The test asserts that the original variant and + // the reconstructed variant are the same. This assertion is also necessary for + // function suggestion. See https://github.com/apache/arrow-datafusion/issues/8082 + fn test_display_and_from_str() { + for func_original in BuiltInWindowFunction::iter() { + let func_name = func_original.to_string(); + let func_from_str = BuiltInWindowFunction::from_str(&func_name).unwrap(); + assert_eq!(func_from_str, func_original); + } + } +} diff --git a/datafusion/expr/src/expr.rs b/datafusion/expr/src/expr.rs index 0ec19bcadbf6..ebf4d3143c12 100644 --- a/datafusion/expr/src/expr.rs +++ b/datafusion/expr/src/expr.rs @@ -19,13 +19,13 @@ use crate::expr_fn::binary_expr; use crate::logical_plan::Subquery; -use crate::udaf; use crate::utils::{expr_to_columns, find_out_reference_exprs}; use crate::window_frame; -use crate::window_function; + use crate::Operator; use crate::{aggregate_function, ExprSchemable}; use crate::{built_in_function, BuiltinScalarFunction}; +use crate::{built_in_window_function, udaf}; use arrow::datatypes::DataType; use datafusion_common::tree_node::{Transformed, TreeNode}; use datafusion_common::{internal_err, DFSchema, OwnedTableReference}; @@ -34,8 +34,11 @@ use std::collections::HashSet; use std::fmt; use std::fmt::{Display, Formatter, Write}; use std::hash::{BuildHasher, Hash, Hasher}; +use std::str::FromStr; use std::sync::Arc; +use crate::Signature; + /// `Expr` is a central struct of DataFusion's query API, and /// represent logical expressions such as `A + 1`, or `CAST(c1 AS /// int)`. @@ -566,11 +569,64 @@ impl AggregateFunction { } } +/// WindowFunction +#[derive(Debug, Clone, PartialEq, Eq, Hash)] +/// Defines which implementation of an aggregate function DataFusion should call. +pub enum WindowFunctionDefinition { + /// A built in aggregate function that leverages an aggregate function + AggregateFunction(aggregate_function::AggregateFunction), + /// A a built-in window function + BuiltInWindowFunction(built_in_window_function::BuiltInWindowFunction), + /// A user defined aggregate function + AggregateUDF(Arc), + /// A user defined aggregate function + WindowUDF(Arc), +} + +impl WindowFunctionDefinition { + /// Returns the datatype of the window function + pub fn return_type(&self, input_expr_types: &[DataType]) -> Result { + match self { + WindowFunctionDefinition::AggregateFunction(fun) => { + fun.return_type(input_expr_types) + } + WindowFunctionDefinition::BuiltInWindowFunction(fun) => { + fun.return_type(input_expr_types) + } + WindowFunctionDefinition::AggregateUDF(fun) => { + fun.return_type(input_expr_types) + } + WindowFunctionDefinition::WindowUDF(fun) => fun.return_type(input_expr_types), + } + } + + /// the signatures supported by the function `fun`. + pub fn signature(&self) -> Signature { + match self { + WindowFunctionDefinition::AggregateFunction(fun) => fun.signature(), + WindowFunctionDefinition::BuiltInWindowFunction(fun) => fun.signature(), + WindowFunctionDefinition::AggregateUDF(fun) => fun.signature().clone(), + WindowFunctionDefinition::WindowUDF(fun) => fun.signature().clone(), + } + } +} + +impl fmt::Display for WindowFunctionDefinition { + fn fmt(&self, f: &mut fmt::Formatter) -> fmt::Result { + match self { + WindowFunctionDefinition::AggregateFunction(fun) => fun.fmt(f), + WindowFunctionDefinition::BuiltInWindowFunction(fun) => fun.fmt(f), + WindowFunctionDefinition::AggregateUDF(fun) => std::fmt::Debug::fmt(fun, f), + WindowFunctionDefinition::WindowUDF(fun) => fun.fmt(f), + } + } +} + /// Window function #[derive(Clone, PartialEq, Eq, Hash, Debug)] pub struct WindowFunction { /// Name of the function - pub fun: window_function::WindowFunction, + pub fun: WindowFunctionDefinition, /// List of expressions to feed to the functions as arguments pub args: Vec, /// List of partition by expressions @@ -584,7 +640,7 @@ pub struct WindowFunction { impl WindowFunction { /// Create a new Window expression pub fn new( - fun: window_function::WindowFunction, + fun: WindowFunctionDefinition, args: Vec, partition_by: Vec, order_by: Vec, @@ -600,6 +656,50 @@ impl WindowFunction { } } +/// Find DataFusion's built-in window function by name. +pub fn find_df_window_func(name: &str) -> Option { + let name = name.to_lowercase(); + // Code paths for window functions leveraging ordinary aggregators and + // built-in window functions are quite different, and the same function + // may have different implementations for these cases. If the sought + // function is not found among built-in window functions, we search for + // it among aggregate functions. + if let Ok(built_in_function) = + built_in_window_function::BuiltInWindowFunction::from_str(name.as_str()) + { + Some(WindowFunctionDefinition::BuiltInWindowFunction( + built_in_function, + )) + } else if let Ok(aggregate) = + aggregate_function::AggregateFunction::from_str(name.as_str()) + { + Some(WindowFunctionDefinition::AggregateFunction(aggregate)) + } else { + None + } +} + +/// Returns the datatype of the window function +#[deprecated( + since = "27.0.0", + note = "please use `WindowFunction::return_type` instead" +)] +pub fn return_type( + fun: &WindowFunctionDefinition, + input_expr_types: &[DataType], +) -> Result { + fun.return_type(input_expr_types) +} + +/// the signatures supported by the function `fun`. +#[deprecated( + since = "27.0.0", + note = "please use `WindowFunction::signature` instead" +)] +pub fn signature(fun: &WindowFunctionDefinition) -> Signature { + fun.signature() +} + // Exists expression. #[derive(Clone, PartialEq, Eq, Hash, Debug)] pub struct Exists { @@ -1890,4 +1990,187 @@ mod test { .is_volatile() .expect_err("Shouldn't determine volatility of unresolved function"); } + + use super::*; + + #[test] + fn test_count_return_type() -> Result<()> { + let fun = find_df_window_func("count").unwrap(); + let observed = fun.return_type(&[DataType::Utf8])?; + assert_eq!(DataType::Int64, observed); + + let observed = fun.return_type(&[DataType::UInt64])?; + assert_eq!(DataType::Int64, observed); + + Ok(()) + } + + #[test] + fn test_first_value_return_type() -> Result<()> { + let fun = find_df_window_func("first_value").unwrap(); + let observed = fun.return_type(&[DataType::Utf8])?; + assert_eq!(DataType::Utf8, observed); + + let observed = fun.return_type(&[DataType::UInt64])?; + assert_eq!(DataType::UInt64, observed); + + Ok(()) + } + + #[test] + fn test_last_value_return_type() -> Result<()> { + let fun = find_df_window_func("last_value").unwrap(); + let observed = fun.return_type(&[DataType::Utf8])?; + assert_eq!(DataType::Utf8, observed); + + let observed = fun.return_type(&[DataType::Float64])?; + assert_eq!(DataType::Float64, observed); + + Ok(()) + } + + #[test] + fn test_lead_return_type() -> Result<()> { + let fun = find_df_window_func("lead").unwrap(); + let observed = fun.return_type(&[DataType::Utf8])?; + assert_eq!(DataType::Utf8, observed); + + let observed = fun.return_type(&[DataType::Float64])?; + assert_eq!(DataType::Float64, observed); + + Ok(()) + } + + #[test] + fn test_lag_return_type() -> Result<()> { + let fun = find_df_window_func("lag").unwrap(); + let observed = fun.return_type(&[DataType::Utf8])?; + assert_eq!(DataType::Utf8, observed); + + let observed = fun.return_type(&[DataType::Float64])?; + assert_eq!(DataType::Float64, observed); + + Ok(()) + } + + #[test] + fn test_nth_value_return_type() -> Result<()> { + let fun = find_df_window_func("nth_value").unwrap(); + let observed = fun.return_type(&[DataType::Utf8, DataType::UInt64])?; + assert_eq!(DataType::Utf8, observed); + + let observed = fun.return_type(&[DataType::Float64, DataType::UInt64])?; + assert_eq!(DataType::Float64, observed); + + Ok(()) + } + + #[test] + fn test_percent_rank_return_type() -> Result<()> { + let fun = find_df_window_func("percent_rank").unwrap(); + let observed = fun.return_type(&[])?; + assert_eq!(DataType::Float64, observed); + + Ok(()) + } + + #[test] + fn test_cume_dist_return_type() -> Result<()> { + let fun = find_df_window_func("cume_dist").unwrap(); + let observed = fun.return_type(&[])?; + assert_eq!(DataType::Float64, observed); + + Ok(()) + } + + #[test] + fn test_ntile_return_type() -> Result<()> { + let fun = find_df_window_func("ntile").unwrap(); + let observed = fun.return_type(&[DataType::Int16])?; + assert_eq!(DataType::UInt64, observed); + + Ok(()) + } + + #[test] + fn test_window_function_case_insensitive() -> Result<()> { + let names = vec![ + "row_number", + "rank", + "dense_rank", + "percent_rank", + "cume_dist", + "ntile", + "lag", + "lead", + "first_value", + "last_value", + "nth_value", + "min", + "max", + "count", + "avg", + "sum", + ]; + for name in names { + let fun = find_df_window_func(name).unwrap(); + let fun2 = find_df_window_func(name.to_uppercase().as_str()).unwrap(); + assert_eq!(fun, fun2); + assert_eq!(fun.to_string(), name.to_uppercase()); + } + Ok(()) + } + + #[test] + fn test_find_df_window_function() { + assert_eq!( + find_df_window_func("max"), + Some(WindowFunctionDefinition::AggregateFunction( + aggregate_function::AggregateFunction::Max + )) + ); + assert_eq!( + find_df_window_func("min"), + Some(WindowFunctionDefinition::AggregateFunction( + aggregate_function::AggregateFunction::Min + )) + ); + assert_eq!( + find_df_window_func("avg"), + Some(WindowFunctionDefinition::AggregateFunction( + aggregate_function::AggregateFunction::Avg + )) + ); + assert_eq!( + find_df_window_func("cume_dist"), + Some(WindowFunctionDefinition::BuiltInWindowFunction( + built_in_window_function::BuiltInWindowFunction::CumeDist + )) + ); + assert_eq!( + find_df_window_func("first_value"), + Some(WindowFunctionDefinition::BuiltInWindowFunction( + built_in_window_function::BuiltInWindowFunction::FirstValue + )) + ); + assert_eq!( + find_df_window_func("LAST_value"), + Some(WindowFunctionDefinition::BuiltInWindowFunction( + built_in_window_function::BuiltInWindowFunction::LastValue + )) + ); + assert_eq!( + find_df_window_func("LAG"), + Some(WindowFunctionDefinition::BuiltInWindowFunction( + built_in_window_function::BuiltInWindowFunction::Lag + )) + ); + assert_eq!( + find_df_window_func("LEAD"), + Some(WindowFunctionDefinition::BuiltInWindowFunction( + built_in_window_function::BuiltInWindowFunction::Lead + )) + ); + assert_eq!(find_df_window_func("not_exist"), None) + } } diff --git a/datafusion/expr/src/lib.rs b/datafusion/expr/src/lib.rs index bf8e9e2954f4..ab213a19a352 100644 --- a/datafusion/expr/src/lib.rs +++ b/datafusion/expr/src/lib.rs @@ -27,6 +27,7 @@ mod accumulator; mod built_in_function; +mod built_in_window_function; mod columnar_value; mod literal; mod nullif; @@ -53,16 +54,16 @@ pub mod tree_node; pub mod type_coercion; pub mod utils; pub mod window_frame; -pub mod window_function; pub mod window_state; pub use accumulator::Accumulator; pub use aggregate_function::AggregateFunction; pub use built_in_function::BuiltinScalarFunction; +pub use built_in_window_function::BuiltInWindowFunction; pub use columnar_value::ColumnarValue; pub use expr::{ Between, BinaryExpr, Case, Cast, Expr, GetFieldAccess, GetIndexedField, GroupingSet, - Like, ScalarFunctionDefinition, TryCast, + Like, ScalarFunctionDefinition, TryCast, WindowFunctionDefinition, }; pub use expr_fn::*; pub use expr_schema::ExprSchemable; @@ -83,7 +84,6 @@ pub use udaf::AggregateUDF; pub use udf::{ScalarUDF, ScalarUDFImpl}; pub use udwf::WindowUDF; pub use window_frame::{WindowFrame, WindowFrameBound, WindowFrameUnits}; -pub use window_function::{BuiltInWindowFunction, WindowFunction}; #[cfg(test)] #[ctor::ctor] diff --git a/datafusion/expr/src/udwf.rs b/datafusion/expr/src/udwf.rs index c233ee84b32d..a97a68341f5c 100644 --- a/datafusion/expr/src/udwf.rs +++ b/datafusion/expr/src/udwf.rs @@ -107,7 +107,7 @@ impl WindowUDF { order_by: Vec, window_frame: WindowFrame, ) -> Expr { - let fun = crate::WindowFunction::WindowUDF(Arc::new(self.clone())); + let fun = crate::WindowFunctionDefinition::WindowUDF(Arc::new(self.clone())); Expr::WindowFunction(crate::expr::WindowFunction { fun, diff --git a/datafusion/expr/src/utils.rs b/datafusion/expr/src/utils.rs index 09f4842c9e64..e3ecdf154e61 100644 --- a/datafusion/expr/src/utils.rs +++ b/datafusion/expr/src/utils.rs @@ -1234,7 +1234,7 @@ mod tests { use super::*; use crate::{ col, cube, expr, expr_vec_fmt, grouping_set, lit, rollup, AggregateFunction, - WindowFrame, WindowFunction, + WindowFrame, WindowFunctionDefinition, }; #[test] @@ -1248,28 +1248,28 @@ mod tests { #[test] fn test_group_window_expr_by_sort_keys_empty_window() -> Result<()> { let max1 = Expr::WindowFunction(expr::WindowFunction::new( - WindowFunction::AggregateFunction(AggregateFunction::Max), + WindowFunctionDefinition::AggregateFunction(AggregateFunction::Max), vec![col("name")], vec![], vec![], WindowFrame::new(false), )); let max2 = Expr::WindowFunction(expr::WindowFunction::new( - WindowFunction::AggregateFunction(AggregateFunction::Max), + WindowFunctionDefinition::AggregateFunction(AggregateFunction::Max), vec![col("name")], vec![], vec![], WindowFrame::new(false), )); let min3 = Expr::WindowFunction(expr::WindowFunction::new( - WindowFunction::AggregateFunction(AggregateFunction::Min), + WindowFunctionDefinition::AggregateFunction(AggregateFunction::Min), vec![col("name")], vec![], vec![], WindowFrame::new(false), )); let sum4 = Expr::WindowFunction(expr::WindowFunction::new( - WindowFunction::AggregateFunction(AggregateFunction::Sum), + WindowFunctionDefinition::AggregateFunction(AggregateFunction::Sum), vec![col("age")], vec![], vec![], @@ -1291,28 +1291,28 @@ mod tests { let created_at_desc = Expr::Sort(expr::Sort::new(Box::new(col("created_at")), false, true)); let max1 = Expr::WindowFunction(expr::WindowFunction::new( - WindowFunction::AggregateFunction(AggregateFunction::Max), + WindowFunctionDefinition::AggregateFunction(AggregateFunction::Max), vec![col("name")], vec![], vec![age_asc.clone(), name_desc.clone()], WindowFrame::new(true), )); let max2 = Expr::WindowFunction(expr::WindowFunction::new( - WindowFunction::AggregateFunction(AggregateFunction::Max), + WindowFunctionDefinition::AggregateFunction(AggregateFunction::Max), vec![col("name")], vec![], vec![], WindowFrame::new(false), )); let min3 = Expr::WindowFunction(expr::WindowFunction::new( - WindowFunction::AggregateFunction(AggregateFunction::Min), + WindowFunctionDefinition::AggregateFunction(AggregateFunction::Min), vec![col("name")], vec![], vec![age_asc.clone(), name_desc.clone()], WindowFrame::new(true), )); let sum4 = Expr::WindowFunction(expr::WindowFunction::new( - WindowFunction::AggregateFunction(AggregateFunction::Sum), + WindowFunctionDefinition::AggregateFunction(AggregateFunction::Sum), vec![col("age")], vec![], vec![name_desc.clone(), age_asc.clone(), created_at_desc.clone()], @@ -1343,7 +1343,7 @@ mod tests { fn test_find_sort_exprs() -> Result<()> { let exprs = &[ Expr::WindowFunction(expr::WindowFunction::new( - WindowFunction::AggregateFunction(AggregateFunction::Max), + WindowFunctionDefinition::AggregateFunction(AggregateFunction::Max), vec![col("name")], vec![], vec![ @@ -1353,7 +1353,7 @@ mod tests { WindowFrame::new(true), )), Expr::WindowFunction(expr::WindowFunction::new( - WindowFunction::AggregateFunction(AggregateFunction::Sum), + WindowFunctionDefinition::AggregateFunction(AggregateFunction::Sum), vec![col("age")], vec![], vec![ diff --git a/datafusion/expr/src/window_function.rs b/datafusion/expr/src/window_function.rs deleted file mode 100644 index 610f1ecaeae9..000000000000 --- a/datafusion/expr/src/window_function.rs +++ /dev/null @@ -1,483 +0,0 @@ -// Licensed to the Apache Software Foundation (ASF) under one -// or more contributor license agreements. See the NOTICE file -// distributed with this work for additional information -// regarding copyright ownership. The ASF licenses this file -// to you under the Apache License, Version 2.0 (the -// "License"); you may not use this file except in compliance -// with the License. You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, -// software distributed under the License is distributed on an -// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -// KIND, either express or implied. See the License for the -// specific language governing permissions and limitations -// under the License. - -//! Window functions provide the ability to perform calculations across -//! sets of rows that are related to the current query row. -//! -//! see also - -use crate::aggregate_function::AggregateFunction; -use crate::type_coercion::functions::data_types; -use crate::utils; -use crate::{AggregateUDF, Signature, TypeSignature, Volatility, WindowUDF}; -use arrow::datatypes::DataType; -use datafusion_common::{plan_datafusion_err, plan_err, DataFusionError, Result}; -use std::sync::Arc; -use std::{fmt, str::FromStr}; -use strum_macros::EnumIter; - -/// WindowFunction -#[derive(Debug, Clone, PartialEq, Eq, Hash)] -pub enum WindowFunction { - /// A built in aggregate function that leverages an aggregate function - AggregateFunction(AggregateFunction), - /// A a built-in window function - BuiltInWindowFunction(BuiltInWindowFunction), - /// A user defined aggregate function - AggregateUDF(Arc), - /// A user defined aggregate function - WindowUDF(Arc), -} - -/// Find DataFusion's built-in window function by name. -pub fn find_df_window_func(name: &str) -> Option { - let name = name.to_lowercase(); - // Code paths for window functions leveraging ordinary aggregators and - // built-in window functions are quite different, and the same function - // may have different implementations for these cases. If the sought - // function is not found among built-in window functions, we search for - // it among aggregate functions. - if let Ok(built_in_function) = BuiltInWindowFunction::from_str(name.as_str()) { - Some(WindowFunction::BuiltInWindowFunction(built_in_function)) - } else if let Ok(aggregate) = AggregateFunction::from_str(name.as_str()) { - Some(WindowFunction::AggregateFunction(aggregate)) - } else { - None - } -} - -impl fmt::Display for BuiltInWindowFunction { - fn fmt(&self, f: &mut fmt::Formatter) -> fmt::Result { - write!(f, "{}", self.name()) - } -} - -impl fmt::Display for WindowFunction { - fn fmt(&self, f: &mut fmt::Formatter) -> fmt::Result { - match self { - WindowFunction::AggregateFunction(fun) => fun.fmt(f), - WindowFunction::BuiltInWindowFunction(fun) => fun.fmt(f), - WindowFunction::AggregateUDF(fun) => std::fmt::Debug::fmt(fun, f), - WindowFunction::WindowUDF(fun) => fun.fmt(f), - } - } -} - -/// A [window function] built in to DataFusion -/// -/// [window function]: https://en.wikipedia.org/wiki/Window_function_(SQL) -#[derive(Debug, Clone, PartialEq, Eq, Hash, EnumIter)] -pub enum BuiltInWindowFunction { - /// number of the current row within its partition, counting from 1 - RowNumber, - /// rank of the current row with gaps; same as row_number of its first peer - Rank, - /// rank of the current row without gaps; this function counts peer groups - DenseRank, - /// relative rank of the current row: (rank - 1) / (total rows - 1) - PercentRank, - /// relative rank of the current row: (number of rows preceding or peer with current row) / (total rows) - CumeDist, - /// integer ranging from 1 to the argument value, dividing the partition as equally as possible - Ntile, - /// returns value evaluated at the row that is offset rows before the current row within the partition; - /// if there is no such row, instead return default (which must be of the same type as value). - /// Both offset and default are evaluated with respect to the current row. - /// If omitted, offset defaults to 1 and default to null - Lag, - /// returns value evaluated at the row that is offset rows after the current row within the partition; - /// if there is no such row, instead return default (which must be of the same type as value). - /// Both offset and default are evaluated with respect to the current row. - /// If omitted, offset defaults to 1 and default to null - Lead, - /// returns value evaluated at the row that is the first row of the window frame - FirstValue, - /// returns value evaluated at the row that is the last row of the window frame - LastValue, - /// returns value evaluated at the row that is the nth row of the window frame (counting from 1); null if no such row - NthValue, -} - -impl BuiltInWindowFunction { - fn name(&self) -> &str { - use BuiltInWindowFunction::*; - match self { - RowNumber => "ROW_NUMBER", - Rank => "RANK", - DenseRank => "DENSE_RANK", - PercentRank => "PERCENT_RANK", - CumeDist => "CUME_DIST", - Ntile => "NTILE", - Lag => "LAG", - Lead => "LEAD", - FirstValue => "FIRST_VALUE", - LastValue => "LAST_VALUE", - NthValue => "NTH_VALUE", - } - } -} - -impl FromStr for BuiltInWindowFunction { - type Err = DataFusionError; - fn from_str(name: &str) -> Result { - Ok(match name.to_uppercase().as_str() { - "ROW_NUMBER" => BuiltInWindowFunction::RowNumber, - "RANK" => BuiltInWindowFunction::Rank, - "DENSE_RANK" => BuiltInWindowFunction::DenseRank, - "PERCENT_RANK" => BuiltInWindowFunction::PercentRank, - "CUME_DIST" => BuiltInWindowFunction::CumeDist, - "NTILE" => BuiltInWindowFunction::Ntile, - "LAG" => BuiltInWindowFunction::Lag, - "LEAD" => BuiltInWindowFunction::Lead, - "FIRST_VALUE" => BuiltInWindowFunction::FirstValue, - "LAST_VALUE" => BuiltInWindowFunction::LastValue, - "NTH_VALUE" => BuiltInWindowFunction::NthValue, - _ => return plan_err!("There is no built-in window function named {name}"), - }) - } -} - -/// Returns the datatype of the window function -#[deprecated( - since = "27.0.0", - note = "please use `WindowFunction::return_type` instead" -)] -pub fn return_type( - fun: &WindowFunction, - input_expr_types: &[DataType], -) -> Result { - fun.return_type(input_expr_types) -} - -impl WindowFunction { - /// Returns the datatype of the window function - pub fn return_type(&self, input_expr_types: &[DataType]) -> Result { - match self { - WindowFunction::AggregateFunction(fun) => fun.return_type(input_expr_types), - WindowFunction::BuiltInWindowFunction(fun) => { - fun.return_type(input_expr_types) - } - WindowFunction::AggregateUDF(fun) => fun.return_type(input_expr_types), - WindowFunction::WindowUDF(fun) => fun.return_type(input_expr_types), - } - } -} - -/// Returns the datatype of the built-in window function -impl BuiltInWindowFunction { - pub fn return_type(&self, input_expr_types: &[DataType]) -> Result { - // Note that this function *must* return the same type that the respective physical expression returns - // or the execution panics. - - // verify that this is a valid set of data types for this function - data_types(input_expr_types, &self.signature()) - // original errors are all related to wrong function signature - // aggregate them for better error message - .map_err(|_| { - plan_datafusion_err!( - "{}", - utils::generate_signature_error_msg( - &format!("{self}"), - self.signature(), - input_expr_types, - ) - ) - })?; - - match self { - BuiltInWindowFunction::RowNumber - | BuiltInWindowFunction::Rank - | BuiltInWindowFunction::DenseRank => Ok(DataType::UInt64), - BuiltInWindowFunction::PercentRank | BuiltInWindowFunction::CumeDist => { - Ok(DataType::Float64) - } - BuiltInWindowFunction::Ntile => Ok(DataType::UInt64), - BuiltInWindowFunction::Lag - | BuiltInWindowFunction::Lead - | BuiltInWindowFunction::FirstValue - | BuiltInWindowFunction::LastValue - | BuiltInWindowFunction::NthValue => Ok(input_expr_types[0].clone()), - } - } -} - -/// the signatures supported by the function `fun`. -#[deprecated( - since = "27.0.0", - note = "please use `WindowFunction::signature` instead" -)] -pub fn signature(fun: &WindowFunction) -> Signature { - fun.signature() -} - -impl WindowFunction { - /// the signatures supported by the function `fun`. - pub fn signature(&self) -> Signature { - match self { - WindowFunction::AggregateFunction(fun) => fun.signature(), - WindowFunction::BuiltInWindowFunction(fun) => fun.signature(), - WindowFunction::AggregateUDF(fun) => fun.signature().clone(), - WindowFunction::WindowUDF(fun) => fun.signature().clone(), - } - } -} - -/// the signatures supported by the built-in window function `fun`. -#[deprecated( - since = "27.0.0", - note = "please use `BuiltInWindowFunction::signature` instead" -)] -pub fn signature_for_built_in(fun: &BuiltInWindowFunction) -> Signature { - fun.signature() -} - -impl BuiltInWindowFunction { - /// the signatures supported by the built-in window function `fun`. - pub fn signature(&self) -> Signature { - // note: the physical expression must accept the type returned by this function or the execution panics. - match self { - BuiltInWindowFunction::RowNumber - | BuiltInWindowFunction::Rank - | BuiltInWindowFunction::DenseRank - | BuiltInWindowFunction::PercentRank - | BuiltInWindowFunction::CumeDist => Signature::any(0, Volatility::Immutable), - BuiltInWindowFunction::Lag | BuiltInWindowFunction::Lead => { - Signature::one_of( - vec![ - TypeSignature::Any(1), - TypeSignature::Any(2), - TypeSignature::Any(3), - ], - Volatility::Immutable, - ) - } - BuiltInWindowFunction::FirstValue | BuiltInWindowFunction::LastValue => { - Signature::any(1, Volatility::Immutable) - } - BuiltInWindowFunction::Ntile => Signature::uniform( - 1, - vec![ - DataType::UInt64, - DataType::UInt32, - DataType::UInt16, - DataType::UInt8, - DataType::Int64, - DataType::Int32, - DataType::Int16, - DataType::Int8, - ], - Volatility::Immutable, - ), - BuiltInWindowFunction::NthValue => Signature::any(2, Volatility::Immutable), - } - } -} - -#[cfg(test)] -mod tests { - use super::*; - use strum::IntoEnumIterator; - - #[test] - fn test_count_return_type() -> Result<()> { - let fun = find_df_window_func("count").unwrap(); - let observed = fun.return_type(&[DataType::Utf8])?; - assert_eq!(DataType::Int64, observed); - - let observed = fun.return_type(&[DataType::UInt64])?; - assert_eq!(DataType::Int64, observed); - - Ok(()) - } - - #[test] - fn test_first_value_return_type() -> Result<()> { - let fun = find_df_window_func("first_value").unwrap(); - let observed = fun.return_type(&[DataType::Utf8])?; - assert_eq!(DataType::Utf8, observed); - - let observed = fun.return_type(&[DataType::UInt64])?; - assert_eq!(DataType::UInt64, observed); - - Ok(()) - } - - #[test] - fn test_last_value_return_type() -> Result<()> { - let fun = find_df_window_func("last_value").unwrap(); - let observed = fun.return_type(&[DataType::Utf8])?; - assert_eq!(DataType::Utf8, observed); - - let observed = fun.return_type(&[DataType::Float64])?; - assert_eq!(DataType::Float64, observed); - - Ok(()) - } - - #[test] - fn test_lead_return_type() -> Result<()> { - let fun = find_df_window_func("lead").unwrap(); - let observed = fun.return_type(&[DataType::Utf8])?; - assert_eq!(DataType::Utf8, observed); - - let observed = fun.return_type(&[DataType::Float64])?; - assert_eq!(DataType::Float64, observed); - - Ok(()) - } - - #[test] - fn test_lag_return_type() -> Result<()> { - let fun = find_df_window_func("lag").unwrap(); - let observed = fun.return_type(&[DataType::Utf8])?; - assert_eq!(DataType::Utf8, observed); - - let observed = fun.return_type(&[DataType::Float64])?; - assert_eq!(DataType::Float64, observed); - - Ok(()) - } - - #[test] - fn test_nth_value_return_type() -> Result<()> { - let fun = find_df_window_func("nth_value").unwrap(); - let observed = fun.return_type(&[DataType::Utf8, DataType::UInt64])?; - assert_eq!(DataType::Utf8, observed); - - let observed = fun.return_type(&[DataType::Float64, DataType::UInt64])?; - assert_eq!(DataType::Float64, observed); - - Ok(()) - } - - #[test] - fn test_percent_rank_return_type() -> Result<()> { - let fun = find_df_window_func("percent_rank").unwrap(); - let observed = fun.return_type(&[])?; - assert_eq!(DataType::Float64, observed); - - Ok(()) - } - - #[test] - fn test_cume_dist_return_type() -> Result<()> { - let fun = find_df_window_func("cume_dist").unwrap(); - let observed = fun.return_type(&[])?; - assert_eq!(DataType::Float64, observed); - - Ok(()) - } - - #[test] - fn test_ntile_return_type() -> Result<()> { - let fun = find_df_window_func("ntile").unwrap(); - let observed = fun.return_type(&[DataType::Int16])?; - assert_eq!(DataType::UInt64, observed); - - Ok(()) - } - - #[test] - fn test_window_function_case_insensitive() -> Result<()> { - let names = vec![ - "row_number", - "rank", - "dense_rank", - "percent_rank", - "cume_dist", - "ntile", - "lag", - "lead", - "first_value", - "last_value", - "nth_value", - "min", - "max", - "count", - "avg", - "sum", - ]; - for name in names { - let fun = find_df_window_func(name).unwrap(); - let fun2 = find_df_window_func(name.to_uppercase().as_str()).unwrap(); - assert_eq!(fun, fun2); - assert_eq!(fun.to_string(), name.to_uppercase()); - } - Ok(()) - } - - #[test] - fn test_find_df_window_function() { - assert_eq!( - find_df_window_func("max"), - Some(WindowFunction::AggregateFunction(AggregateFunction::Max)) - ); - assert_eq!( - find_df_window_func("min"), - Some(WindowFunction::AggregateFunction(AggregateFunction::Min)) - ); - assert_eq!( - find_df_window_func("avg"), - Some(WindowFunction::AggregateFunction(AggregateFunction::Avg)) - ); - assert_eq!( - find_df_window_func("cume_dist"), - Some(WindowFunction::BuiltInWindowFunction( - BuiltInWindowFunction::CumeDist - )) - ); - assert_eq!( - find_df_window_func("first_value"), - Some(WindowFunction::BuiltInWindowFunction( - BuiltInWindowFunction::FirstValue - )) - ); - assert_eq!( - find_df_window_func("LAST_value"), - Some(WindowFunction::BuiltInWindowFunction( - BuiltInWindowFunction::LastValue - )) - ); - assert_eq!( - find_df_window_func("LAG"), - Some(WindowFunction::BuiltInWindowFunction( - BuiltInWindowFunction::Lag - )) - ); - assert_eq!( - find_df_window_func("LEAD"), - Some(WindowFunction::BuiltInWindowFunction( - BuiltInWindowFunction::Lead - )) - ); - assert_eq!(find_df_window_func("not_exist"), None) - } - - #[test] - // Test for BuiltInWindowFunction's Display and from_str() implementations. - // For each variant in BuiltInWindowFunction, it converts the variant to a string - // and then back to a variant. The test asserts that the original variant and - // the reconstructed variant are the same. This assertion is also necessary for - // function suggestion. See https://github.com/apache/arrow-datafusion/issues/8082 - fn test_display_and_from_str() { - for func_original in BuiltInWindowFunction::iter() { - let func_name = func_original.to_string(); - let func_from_str = BuiltInWindowFunction::from_str(&func_name).unwrap(); - assert_eq!(func_from_str, func_original); - } - } -} diff --git a/datafusion/optimizer/src/analyzer/count_wildcard_rule.rs b/datafusion/optimizer/src/analyzer/count_wildcard_rule.rs index fd84bb80160b..953716713e41 100644 --- a/datafusion/optimizer/src/analyzer/count_wildcard_rule.rs +++ b/datafusion/optimizer/src/analyzer/count_wildcard_rule.rs @@ -24,7 +24,7 @@ use datafusion_expr::expr_rewriter::rewrite_preserving_name; use datafusion_expr::utils::COUNT_STAR_EXPANSION; use datafusion_expr::Expr::ScalarSubquery; use datafusion_expr::{ - aggregate_function, expr, lit, window_function, Aggregate, Expr, Filter, LogicalPlan, + aggregate_function, expr, lit, Aggregate, Expr, Filter, LogicalPlan, LogicalPlanBuilder, Projection, Sort, Subquery, }; use std::sync::Arc; @@ -121,7 +121,7 @@ impl TreeNodeRewriter for CountWildcardRewriter { let new_expr = match old_expr.clone() { Expr::WindowFunction(expr::WindowFunction { fun: - window_function::WindowFunction::AggregateFunction( + expr::WindowFunctionDefinition::AggregateFunction( aggregate_function::AggregateFunction::Count, ), args, @@ -131,7 +131,7 @@ impl TreeNodeRewriter for CountWildcardRewriter { }) if args.len() == 1 => match args[0] { Expr::Wildcard { qualifier: None } => { Expr::WindowFunction(expr::WindowFunction { - fun: window_function::WindowFunction::AggregateFunction( + fun: expr::WindowFunctionDefinition::AggregateFunction( aggregate_function::AggregateFunction::Count, ), args: vec![lit(COUNT_STAR_EXPANSION)], @@ -229,7 +229,7 @@ mod tests { use datafusion_expr::{ col, count, exists, expr, in_subquery, lit, logical_plan::LogicalPlanBuilder, max, out_ref_col, scalar_subquery, wildcard, AggregateFunction, Expr, - WindowFrame, WindowFrameBound, WindowFrameUnits, WindowFunction, + WindowFrame, WindowFrameBound, WindowFrameUnits, WindowFunctionDefinition, }; fn assert_plan_eq(plan: &LogicalPlan, expected: &str) -> Result<()> { @@ -342,7 +342,7 @@ mod tests { let plan = LogicalPlanBuilder::from(table_scan) .window(vec![Expr::WindowFunction(expr::WindowFunction::new( - WindowFunction::AggregateFunction(AggregateFunction::Count), + WindowFunctionDefinition::AggregateFunction(AggregateFunction::Count), vec![wildcard()], vec![], vec![Expr::Sort(Sort::new(Box::new(col("a")), false, true))], diff --git a/datafusion/optimizer/src/analyzer/type_coercion.rs b/datafusion/optimizer/src/analyzer/type_coercion.rs index b6298f5b552f..4d54dad99670 100644 --- a/datafusion/optimizer/src/analyzer/type_coercion.rs +++ b/datafusion/optimizer/src/analyzer/type_coercion.rs @@ -45,9 +45,9 @@ use datafusion_expr::type_coercion::{is_datetime, is_utf8_or_large_utf8}; use datafusion_expr::utils::merge_schema; use datafusion_expr::{ is_false, is_not_false, is_not_true, is_not_unknown, is_true, is_unknown, - type_coercion, window_function, AggregateFunction, BuiltinScalarFunction, Expr, - ExprSchemable, LogicalPlan, Operator, Projection, ScalarFunctionDefinition, - Signature, WindowFrame, WindowFrameBound, WindowFrameUnits, + type_coercion, AggregateFunction, BuiltinScalarFunction, Expr, ExprSchemable, + LogicalPlan, Operator, Projection, ScalarFunctionDefinition, Signature, WindowFrame, + WindowFrameBound, WindowFrameUnits, }; use crate::analyzer::AnalyzerRule; @@ -390,7 +390,7 @@ impl TreeNodeRewriter for TypeCoercionRewriter { coerce_window_frame(window_frame, &self.schema, &order_by)?; let args = match &fun { - window_function::WindowFunction::AggregateFunction(fun) => { + expr::WindowFunctionDefinition::AggregateFunction(fun) => { coerce_agg_exprs_for_signature( fun, &args, diff --git a/datafusion/optimizer/src/push_down_projection.rs b/datafusion/optimizer/src/push_down_projection.rs index 10cc1879aeeb..4ee4f7e417a6 100644 --- a/datafusion/optimizer/src/push_down_projection.rs +++ b/datafusion/optimizer/src/push_down_projection.rs @@ -37,7 +37,7 @@ mod tests { }; use datafusion_expr::{ col, count, lit, max, min, AggregateFunction, Expr, LogicalPlan, Projection, - WindowFrame, WindowFunction, + WindowFrame, WindowFunctionDefinition, }; #[test] @@ -582,7 +582,7 @@ mod tests { let table_scan = test_table_scan()?; let max1 = Expr::WindowFunction(expr::WindowFunction::new( - WindowFunction::AggregateFunction(AggregateFunction::Max), + WindowFunctionDefinition::AggregateFunction(AggregateFunction::Max), vec![col("test.a")], vec![col("test.b")], vec![], @@ -590,7 +590,7 @@ mod tests { )); let max2 = Expr::WindowFunction(expr::WindowFunction::new( - WindowFunction::AggregateFunction(AggregateFunction::Max), + WindowFunctionDefinition::AggregateFunction(AggregateFunction::Max), vec![col("test.b")], vec![], vec![], diff --git a/datafusion/physical-plan/src/windows/mod.rs b/datafusion/physical-plan/src/windows/mod.rs index 3187e6b0fbd3..fec168fabf48 100644 --- a/datafusion/physical-plan/src/windows/mod.rs +++ b/datafusion/physical-plan/src/windows/mod.rs @@ -34,8 +34,8 @@ use arrow::datatypes::Schema; use arrow_schema::{DataType, Field, SchemaRef}; use datafusion_common::{exec_err, DataFusionError, Result, ScalarValue}; use datafusion_expr::{ - window_function::{BuiltInWindowFunction, WindowFunction}, - PartitionEvaluator, WindowFrame, WindowUDF, + BuiltInWindowFunction, PartitionEvaluator, WindowFrame, WindowFunctionDefinition, + WindowUDF, }; use datafusion_physical_expr::equivalence::collapse_lex_req; use datafusion_physical_expr::{ @@ -56,7 +56,7 @@ pub use datafusion_physical_expr::window::{ /// Create a physical expression for window function pub fn create_window_expr( - fun: &WindowFunction, + fun: &WindowFunctionDefinition, name: String, args: &[Arc], partition_by: &[Arc], @@ -65,7 +65,7 @@ pub fn create_window_expr( input_schema: &Schema, ) -> Result> { Ok(match fun { - WindowFunction::AggregateFunction(fun) => { + WindowFunctionDefinition::AggregateFunction(fun) => { let aggregate = aggregates::create_aggregate_expr( fun, false, @@ -81,13 +81,15 @@ pub fn create_window_expr( aggregate, ) } - WindowFunction::BuiltInWindowFunction(fun) => Arc::new(BuiltInWindowExpr::new( - create_built_in_window_expr(fun, args, input_schema, name)?, - partition_by, - order_by, - window_frame, - )), - WindowFunction::AggregateUDF(fun) => { + WindowFunctionDefinition::BuiltInWindowFunction(fun) => { + Arc::new(BuiltInWindowExpr::new( + create_built_in_window_expr(fun, args, input_schema, name)?, + partition_by, + order_by, + window_frame, + )) + } + WindowFunctionDefinition::AggregateUDF(fun) => { let aggregate = udaf::create_aggregate_expr(fun.as_ref(), args, input_schema, name)?; window_expr_from_aggregate_expr( @@ -97,7 +99,7 @@ pub fn create_window_expr( aggregate, ) } - WindowFunction::WindowUDF(fun) => Arc::new(BuiltInWindowExpr::new( + WindowFunctionDefinition::WindowUDF(fun) => Arc::new(BuiltInWindowExpr::new( create_udwf_window_expr(fun, args, input_schema, name)?, partition_by, order_by, @@ -647,7 +649,7 @@ mod tests { let refs = blocking_exec.refs(); let window_agg_exec = Arc::new(WindowAggExec::try_new( vec![create_window_expr( - &WindowFunction::AggregateFunction(AggregateFunction::Count), + &WindowFunctionDefinition::AggregateFunction(AggregateFunction::Count), "count".to_owned(), &[col("a", &schema)?], &[], diff --git a/datafusion/proto/src/logical_plan/from_proto.rs b/datafusion/proto/src/logical_plan/from_proto.rs index c582e92dc11c..36c5b44f00b9 100644 --- a/datafusion/proto/src/logical_plan/from_proto.rs +++ b/datafusion/proto/src/logical_plan/from_proto.rs @@ -1112,7 +1112,7 @@ pub fn parse_expr( let aggr_function = parse_i32_to_aggregate_function(i)?; Ok(Expr::WindowFunction(WindowFunction::new( - datafusion_expr::window_function::WindowFunction::AggregateFunction( + datafusion_expr::expr::WindowFunctionDefinition::AggregateFunction( aggr_function, ), vec![parse_required_expr(expr.expr.as_deref(), registry, "expr")?], @@ -1131,7 +1131,7 @@ pub fn parse_expr( .unwrap_or_else(Vec::new); Ok(Expr::WindowFunction(WindowFunction::new( - datafusion_expr::window_function::WindowFunction::BuiltInWindowFunction( + datafusion_expr::expr::WindowFunctionDefinition::BuiltInWindowFunction( built_in_function, ), args, @@ -1146,7 +1146,7 @@ pub fn parse_expr( .map(|e| vec![e]) .unwrap_or_else(Vec::new); Ok(Expr::WindowFunction(WindowFunction::new( - datafusion_expr::window_function::WindowFunction::AggregateUDF( + datafusion_expr::expr::WindowFunctionDefinition::AggregateUDF( udaf_function, ), args, @@ -1161,7 +1161,7 @@ pub fn parse_expr( .map(|e| vec![e]) .unwrap_or_else(Vec::new); Ok(Expr::WindowFunction(WindowFunction::new( - datafusion_expr::window_function::WindowFunction::WindowUDF( + datafusion_expr::expr::WindowFunctionDefinition::WindowUDF( udwf_function, ), args, diff --git a/datafusion/proto/src/logical_plan/to_proto.rs b/datafusion/proto/src/logical_plan/to_proto.rs index b9987ff6c727..a162b2389cd1 100644 --- a/datafusion/proto/src/logical_plan/to_proto.rs +++ b/datafusion/proto/src/logical_plan/to_proto.rs @@ -51,7 +51,7 @@ use datafusion_expr::expr::{ use datafusion_expr::{ logical_plan::PlanType, logical_plan::StringifiedPlan, AggregateFunction, BuiltInWindowFunction, BuiltinScalarFunction, Expr, JoinConstraint, JoinType, - TryCast, WindowFrame, WindowFrameBound, WindowFrameUnits, WindowFunction, + TryCast, WindowFrame, WindowFrameBound, WindowFrameUnits, WindowFunctionDefinition, }; #[derive(Debug)] @@ -605,22 +605,22 @@ impl TryFrom<&Expr> for protobuf::LogicalExprNode { ref window_frame, }) => { let window_function = match fun { - WindowFunction::AggregateFunction(fun) => { + WindowFunctionDefinition::AggregateFunction(fun) => { protobuf::window_expr_node::WindowFunction::AggrFunction( protobuf::AggregateFunction::from(fun).into(), ) } - WindowFunction::BuiltInWindowFunction(fun) => { + WindowFunctionDefinition::BuiltInWindowFunction(fun) => { protobuf::window_expr_node::WindowFunction::BuiltInFunction( protobuf::BuiltInWindowFunction::from(fun).into(), ) } - WindowFunction::AggregateUDF(aggr_udf) => { + WindowFunctionDefinition::AggregateUDF(aggr_udf) => { protobuf::window_expr_node::WindowFunction::Udaf( aggr_udf.name().to_string(), ) } - WindowFunction::WindowUDF(window_udf) => { + WindowFunctionDefinition::WindowUDF(window_udf) => { protobuf::window_expr_node::WindowFunction::Udwf( window_udf.name().to_string(), ) diff --git a/datafusion/proto/src/physical_plan/from_proto.rs b/datafusion/proto/src/physical_plan/from_proto.rs index 8ad6d679df4d..23ab813ca739 100644 --- a/datafusion/proto/src/physical_plan/from_proto.rs +++ b/datafusion/proto/src/physical_plan/from_proto.rs @@ -31,7 +31,7 @@ use datafusion::datasource::object_store::ObjectStoreUrl; use datafusion::datasource::physical_plan::{FileScanConfig, FileSinkConfig}; use datafusion::execution::context::ExecutionProps; use datafusion::execution::FunctionRegistry; -use datafusion::logical_expr::window_function::WindowFunction; +use datafusion::logical_expr::WindowFunctionDefinition; use datafusion::physical_expr::{PhysicalSortExpr, ScalarFunctionExpr}; use datafusion::physical_plan::expressions::{ in_list, BinaryExpr, CaseExpr, CastExpr, Column, IsNotNullExpr, IsNullExpr, LikeExpr, @@ -414,7 +414,9 @@ fn parse_required_physical_expr( }) } -impl TryFrom<&protobuf::physical_window_expr_node::WindowFunction> for WindowFunction { +impl TryFrom<&protobuf::physical_window_expr_node::WindowFunction> + for WindowFunctionDefinition +{ type Error = DataFusionError; fn try_from( @@ -428,7 +430,7 @@ impl TryFrom<&protobuf::physical_window_expr_node::WindowFunction> for WindowFun )) })?; - Ok(WindowFunction::AggregateFunction(f.into())) + Ok(WindowFunctionDefinition::AggregateFunction(f.into())) } protobuf::physical_window_expr_node::WindowFunction::BuiltInFunction(n) => { let f = protobuf::BuiltInWindowFunction::try_from(*n).map_err(|_| { @@ -437,7 +439,7 @@ impl TryFrom<&protobuf::physical_window_expr_node::WindowFunction> for WindowFun )) })?; - Ok(WindowFunction::BuiltInWindowFunction(f.into())) + Ok(WindowFunctionDefinition::BuiltInWindowFunction(f.into())) } } } diff --git a/datafusion/proto/tests/cases/roundtrip_logical_plan.rs b/datafusion/proto/tests/cases/roundtrip_logical_plan.rs index 2d7d85abda96..dea99f91e392 100644 --- a/datafusion/proto/tests/cases/roundtrip_logical_plan.rs +++ b/datafusion/proto/tests/cases/roundtrip_logical_plan.rs @@ -53,7 +53,7 @@ use datafusion_expr::{ col, create_udaf, lit, Accumulator, AggregateFunction, BuiltinScalarFunction::{Sqrt, Substr}, Expr, LogicalPlan, Operator, PartitionEvaluator, Signature, TryCast, Volatility, - WindowFrame, WindowFrameBound, WindowFrameUnits, WindowFunction, WindowUDF, + WindowFrame, WindowFrameBound, WindowFrameUnits, WindowFunctionDefinition, WindowUDF, }; use datafusion_proto::bytes::{ logical_plan_from_bytes, logical_plan_from_bytes_with_extension_codec, @@ -1663,8 +1663,8 @@ fn roundtrip_window() { // 1. without window_frame let test_expr1 = Expr::WindowFunction(expr::WindowFunction::new( - WindowFunction::BuiltInWindowFunction( - datafusion_expr::window_function::BuiltInWindowFunction::Rank, + WindowFunctionDefinition::BuiltInWindowFunction( + datafusion_expr::BuiltInWindowFunction::Rank, ), vec![], vec![col("col1")], @@ -1674,8 +1674,8 @@ fn roundtrip_window() { // 2. with default window_frame let test_expr2 = Expr::WindowFunction(expr::WindowFunction::new( - WindowFunction::BuiltInWindowFunction( - datafusion_expr::window_function::BuiltInWindowFunction::Rank, + WindowFunctionDefinition::BuiltInWindowFunction( + datafusion_expr::BuiltInWindowFunction::Rank, ), vec![], vec![col("col1")], @@ -1691,8 +1691,8 @@ fn roundtrip_window() { }; let test_expr3 = Expr::WindowFunction(expr::WindowFunction::new( - WindowFunction::BuiltInWindowFunction( - datafusion_expr::window_function::BuiltInWindowFunction::Rank, + WindowFunctionDefinition::BuiltInWindowFunction( + datafusion_expr::BuiltInWindowFunction::Rank, ), vec![], vec![col("col1")], @@ -1708,7 +1708,7 @@ fn roundtrip_window() { }; let test_expr4 = Expr::WindowFunction(expr::WindowFunction::new( - WindowFunction::AggregateFunction(AggregateFunction::Max), + WindowFunctionDefinition::AggregateFunction(AggregateFunction::Max), vec![col("col1")], vec![col("col1")], vec![col("col2")], @@ -1759,7 +1759,7 @@ fn roundtrip_window() { ); let test_expr5 = Expr::WindowFunction(expr::WindowFunction::new( - WindowFunction::AggregateUDF(Arc::new(dummy_agg.clone())), + WindowFunctionDefinition::AggregateUDF(Arc::new(dummy_agg.clone())), vec![col("col1")], vec![col("col1")], vec![col("col2")], @@ -1808,7 +1808,7 @@ fn roundtrip_window() { ); let test_expr6 = Expr::WindowFunction(expr::WindowFunction::new( - WindowFunction::WindowUDF(Arc::new(dummy_window_udf.clone())), + WindowFunctionDefinition::WindowUDF(Arc::new(dummy_window_udf.clone())), vec![col("col1")], vec![col("col1")], vec![col("col2")], diff --git a/datafusion/sql/src/expr/function.rs b/datafusion/sql/src/expr/function.rs index 3934d6701c63..395f10b6f783 100644 --- a/datafusion/sql/src/expr/function.rs +++ b/datafusion/sql/src/expr/function.rs @@ -23,8 +23,8 @@ use datafusion_expr::expr::ScalarFunction; use datafusion_expr::function::suggest_valid_function; use datafusion_expr::window_frame::{check_window_frame, regularize_window_order_by}; use datafusion_expr::{ - expr, window_function, AggregateFunction, BuiltinScalarFunction, Expr, WindowFrame, - WindowFunction, + expr, AggregateFunction, BuiltinScalarFunction, Expr, WindowFrame, + WindowFunctionDefinition, }; use sqlparser::ast::{ Expr as SQLExpr, Function as SQLFunction, FunctionArg, FunctionArgExpr, WindowType, @@ -121,12 +121,12 @@ impl<'a, S: ContextProvider> SqlToRel<'a, S> { if let Ok(fun) = self.find_window_func(&name) { let expr = match fun { - WindowFunction::AggregateFunction(aggregate_fun) => { + WindowFunctionDefinition::AggregateFunction(aggregate_fun) => { let args = self.function_args_to_expr(args, schema, planner_context)?; Expr::WindowFunction(expr::WindowFunction::new( - WindowFunction::AggregateFunction(aggregate_fun), + WindowFunctionDefinition::AggregateFunction(aggregate_fun), args, partition_by, order_by, @@ -191,19 +191,22 @@ impl<'a, S: ContextProvider> SqlToRel<'a, S> { Ok(Expr::ScalarFunction(ScalarFunction::new(fun, args))) } - pub(super) fn find_window_func(&self, name: &str) -> Result { - window_function::find_df_window_func(name) + pub(super) fn find_window_func( + &self, + name: &str, + ) -> Result { + expr::find_df_window_func(name) // next check user defined aggregates .or_else(|| { self.context_provider .get_aggregate_meta(name) - .map(WindowFunction::AggregateUDF) + .map(WindowFunctionDefinition::AggregateUDF) }) // next check user defined window functions .or_else(|| { self.context_provider .get_window_meta(name) - .map(WindowFunction::WindowUDF) + .map(WindowFunctionDefinition::WindowUDF) }) .ok_or_else(|| { plan_datafusion_err!("There is no window function named {name}") diff --git a/datafusion/substrait/src/logical_plan/consumer.rs b/datafusion/substrait/src/logical_plan/consumer.rs index 9931dd15aec8..a4ec3e7722a2 100644 --- a/datafusion/substrait/src/logical_plan/consumer.rs +++ b/datafusion/substrait/src/logical_plan/consumer.rs @@ -23,8 +23,8 @@ use datafusion::common::{ use datafusion::execution::FunctionRegistry; use datafusion::logical_expr::{ - aggregate_function, window_function::find_df_window_func, BinaryExpr, - BuiltinScalarFunction, Case, Expr, LogicalPlan, Operator, + aggregate_function, expr::find_df_window_func, BinaryExpr, BuiltinScalarFunction, + Case, Expr, LogicalPlan, Operator, }; use datafusion::logical_expr::{ expr, Cast, Extension, GroupingSet, Like, LogicalPlanBuilder, Partitioning, From bf0a39a791e7cd0e965abb8c87950cc4101149f7 Mon Sep 17 00:00:00 2001 From: Liang-Chi Hsieh Date: Tue, 2 Jan 2024 00:28:36 -0800 Subject: [PATCH 531/572] Deprecate duplicate function `LogicalPlan::with_new_inputs` (#8707) * Remove duplicate function with_new_inputs * Make it as deprecated function --- datafusion/expr/src/logical_plan/builder.rs | 2 +- datafusion/expr/src/logical_plan/plan.rs | 47 ++----------------- datafusion/expr/src/tree_node/plan.rs | 2 +- .../optimizer/src/eliminate_outer_join.rs | 3 +- .../optimizer/src/optimize_projections.rs | 3 +- datafusion/optimizer/src/optimizer.rs | 2 +- datafusion/optimizer/src/push_down_filter.rs | 28 +++++++---- datafusion/optimizer/src/push_down_limit.rs | 23 +++++---- datafusion/optimizer/src/utils.rs | 2 +- 9 files changed, 45 insertions(+), 67 deletions(-) diff --git a/datafusion/expr/src/logical_plan/builder.rs b/datafusion/expr/src/logical_plan/builder.rs index 549c25f89bae..cfc052cfc14c 100644 --- a/datafusion/expr/src/logical_plan/builder.rs +++ b/datafusion/expr/src/logical_plan/builder.rs @@ -445,7 +445,7 @@ impl LogicalPlanBuilder { ) }) .collect::>>()?; - curr_plan.with_new_inputs(&new_inputs) + curr_plan.with_new_exprs(curr_plan.expressions(), &new_inputs) } } } diff --git a/datafusion/expr/src/logical_plan/plan.rs b/datafusion/expr/src/logical_plan/plan.rs index 9b0f441ef902..c0c520c4e211 100644 --- a/datafusion/expr/src/logical_plan/plan.rs +++ b/datafusion/expr/src/logical_plan/plan.rs @@ -541,35 +541,9 @@ impl LogicalPlan { } /// Returns a copy of this `LogicalPlan` with the new inputs + #[deprecated(since = "35.0.0", note = "please use `with_new_exprs` instead")] pub fn with_new_inputs(&self, inputs: &[LogicalPlan]) -> Result { - // with_new_inputs use original expression, - // so we don't need to recompute Schema. - match &self { - LogicalPlan::Projection(projection) => { - // Schema of the projection may change - // when its input changes. Hence we should use - // `try_new` method instead of `try_new_with_schema`. - Projection::try_new(projection.expr.to_vec(), Arc::new(inputs[0].clone())) - .map(LogicalPlan::Projection) - } - LogicalPlan::Window(Window { window_expr, .. }) => Ok(LogicalPlan::Window( - Window::try_new(window_expr.to_vec(), Arc::new(inputs[0].clone()))?, - )), - LogicalPlan::Aggregate(Aggregate { - group_expr, - aggr_expr, - .. - }) => Aggregate::try_new( - // Schema of the aggregate may change - // when its input changes. Hence we should use - // `try_new` method instead of `try_new_with_schema`. - Arc::new(inputs[0].clone()), - group_expr.to_vec(), - aggr_expr.to_vec(), - ) - .map(LogicalPlan::Aggregate), - _ => self.with_new_exprs(self.expressions(), inputs), - } + self.with_new_exprs(self.expressions(), inputs) } /// Returns a new `LogicalPlan` based on `self` with inputs and @@ -591,10 +565,6 @@ impl LogicalPlan { /// // create new plan using rewritten_exprs in same position /// let new_plan = plan.new_with_exprs(rewritten_exprs, new_inputs); /// ``` - /// - /// Note: sometimes [`Self::with_new_exprs`] will use schema of - /// original plan, it will not change the scheam. Such as - /// `Projection/Aggregate/Window` pub fn with_new_exprs( &self, mut expr: Vec, @@ -706,17 +676,10 @@ impl LogicalPlan { })) } }, - LogicalPlan::Window(Window { - window_expr, - schema, - .. - }) => { + LogicalPlan::Window(Window { window_expr, .. }) => { assert_eq!(window_expr.len(), expr.len()); - Ok(LogicalPlan::Window(Window { - input: Arc::new(inputs[0].clone()), - window_expr: expr, - schema: schema.clone(), - })) + Window::try_new(expr, Arc::new(inputs[0].clone())) + .map(LogicalPlan::Window) } LogicalPlan::Aggregate(Aggregate { group_expr, .. }) => { // group exprs are the first expressions diff --git a/datafusion/expr/src/tree_node/plan.rs b/datafusion/expr/src/tree_node/plan.rs index 217116530d4a..208a8b57d7b0 100644 --- a/datafusion/expr/src/tree_node/plan.rs +++ b/datafusion/expr/src/tree_node/plan.rs @@ -113,7 +113,7 @@ impl TreeNode for LogicalPlan { .zip(new_children.iter()) .any(|(c1, c2)| c1 != &c2) { - self.with_new_inputs(new_children.as_slice()) + self.with_new_exprs(self.expressions(), new_children.as_slice()) } else { Ok(self) } diff --git a/datafusion/optimizer/src/eliminate_outer_join.rs b/datafusion/optimizer/src/eliminate_outer_join.rs index e4d57f0209a4..53c4b3702b1e 100644 --- a/datafusion/optimizer/src/eliminate_outer_join.rs +++ b/datafusion/optimizer/src/eliminate_outer_join.rs @@ -106,7 +106,8 @@ impl OptimizerRule for EliminateOuterJoin { schema: join.schema.clone(), null_equals_null: join.null_equals_null, }); - let new_plan = plan.with_new_inputs(&[new_join])?; + let new_plan = + plan.with_new_exprs(plan.expressions(), &[new_join])?; Ok(Some(new_plan)) } _ => Ok(None), diff --git a/datafusion/optimizer/src/optimize_projections.rs b/datafusion/optimizer/src/optimize_projections.rs index 7ae9f7edf5e5..891a909a3378 100644 --- a/datafusion/optimizer/src/optimize_projections.rs +++ b/datafusion/optimizer/src/optimize_projections.rs @@ -373,7 +373,8 @@ fn optimize_projections( // `old_child` during construction: .map(|(new_input, old_child)| new_input.unwrap_or_else(|| old_child.clone())) .collect::>(); - plan.with_new_inputs(&new_inputs).map(Some) + plan.with_new_exprs(plan.expressions(), &new_inputs) + .map(Some) } } diff --git a/datafusion/optimizer/src/optimizer.rs b/datafusion/optimizer/src/optimizer.rs index 0dc34cb809eb..2cb59d511ccf 100644 --- a/datafusion/optimizer/src/optimizer.rs +++ b/datafusion/optimizer/src/optimizer.rs @@ -382,7 +382,7 @@ impl Optimizer { }) .collect::>(); - Ok(Some(plan.with_new_inputs(&new_inputs)?)) + Ok(Some(plan.with_new_exprs(plan.expressions(), &new_inputs)?)) } /// Use a rule to optimize the whole plan. diff --git a/datafusion/optimizer/src/push_down_filter.rs b/datafusion/optimizer/src/push_down_filter.rs index 9d277d18d2f7..4eb925ac0629 100644 --- a/datafusion/optimizer/src/push_down_filter.rs +++ b/datafusion/optimizer/src/push_down_filter.rs @@ -691,9 +691,11 @@ impl OptimizerRule for PushDownFilter { | LogicalPlan::Distinct(_) | LogicalPlan::Sort(_) => { // commutable - let new_filter = - plan.with_new_inputs(&[child_plan.inputs()[0].clone()])?; - child_plan.with_new_inputs(&[new_filter])? + let new_filter = plan.with_new_exprs( + plan.expressions(), + &[child_plan.inputs()[0].clone()], + )?; + child_plan.with_new_exprs(child_plan.expressions(), &[new_filter])? } LogicalPlan::SubqueryAlias(subquery_alias) => { let mut replace_map = HashMap::new(); @@ -716,7 +718,7 @@ impl OptimizerRule for PushDownFilter { new_predicate, subquery_alias.input.clone(), )?); - child_plan.with_new_inputs(&[new_filter])? + child_plan.with_new_exprs(child_plan.expressions(), &[new_filter])? } LogicalPlan::Projection(projection) => { // A projection is filter-commutable if it do not contain volatile predicates or contain volatile @@ -760,10 +762,15 @@ impl OptimizerRule for PushDownFilter { )?); match conjunction(keep_predicates) { - None => child_plan.with_new_inputs(&[new_filter])?, + None => child_plan.with_new_exprs( + child_plan.expressions(), + &[new_filter], + )?, Some(keep_predicate) => { - let child_plan = - child_plan.with_new_inputs(&[new_filter])?; + let child_plan = child_plan.with_new_exprs( + child_plan.expressions(), + &[new_filter], + )?; LogicalPlan::Filter(Filter::try_new( keep_predicate, Arc::new(child_plan), @@ -837,7 +844,9 @@ impl OptimizerRule for PushDownFilter { )?), None => (*agg.input).clone(), }; - let new_agg = filter.input.with_new_inputs(&vec![child])?; + let new_agg = filter + .input + .with_new_exprs(filter.input.expressions(), &vec![child])?; match conjunction(keep_predicates) { Some(predicate) => LogicalPlan::Filter(Filter::try_new( predicate, @@ -942,7 +951,8 @@ impl OptimizerRule for PushDownFilter { None => extension_plan.node.inputs().into_iter().cloned().collect(), }; // extension with new inputs. - let new_extension = child_plan.with_new_inputs(&new_children)?; + let new_extension = + child_plan.with_new_exprs(child_plan.expressions(), &new_children)?; match conjunction(keep_predicates) { Some(predicate) => LogicalPlan::Filter(Filter::try_new( diff --git a/datafusion/optimizer/src/push_down_limit.rs b/datafusion/optimizer/src/push_down_limit.rs index 6703a1d787a7..c2f35a790616 100644 --- a/datafusion/optimizer/src/push_down_limit.rs +++ b/datafusion/optimizer/src/push_down_limit.rs @@ -126,7 +126,7 @@ impl OptimizerRule for PushDownLimit { fetch: scan.fetch.map(|x| min(x, limit)).or(Some(limit)), projected_schema: scan.projected_schema.clone(), }); - Some(plan.with_new_inputs(&[new_input])?) + Some(plan.with_new_exprs(plan.expressions(), &[new_input])?) } } LogicalPlan::Union(union) => { @@ -145,7 +145,7 @@ impl OptimizerRule for PushDownLimit { inputs: new_inputs, schema: union.schema.clone(), }); - Some(plan.with_new_inputs(&[union])?) + Some(plan.with_new_exprs(plan.expressions(), &[union])?) } LogicalPlan::CrossJoin(cross_join) => { @@ -166,15 +166,16 @@ impl OptimizerRule for PushDownLimit { right: Arc::new(new_right), schema: plan.schema().clone(), }); - Some(plan.with_new_inputs(&[new_cross_join])?) + Some(plan.with_new_exprs(plan.expressions(), &[new_cross_join])?) } LogicalPlan::Join(join) => { let new_join = push_down_join(join, fetch + skip); match new_join { - Some(new_join) => { - Some(plan.with_new_inputs(&[LogicalPlan::Join(new_join)])?) - } + Some(new_join) => Some(plan.with_new_exprs( + plan.expressions(), + &[LogicalPlan::Join(new_join)], + )?), None => None, } } @@ -192,14 +193,16 @@ impl OptimizerRule for PushDownLimit { input: Arc::new((*sort.input).clone()), fetch: new_fetch, }); - Some(plan.with_new_inputs(&[new_sort])?) + Some(plan.with_new_exprs(plan.expressions(), &[new_sort])?) } } LogicalPlan::Projection(_) | LogicalPlan::SubqueryAlias(_) => { // commute - let new_limit = - plan.with_new_inputs(&[child_plan.inputs()[0].clone()])?; - Some(child_plan.with_new_inputs(&[new_limit])?) + let new_limit = plan.with_new_exprs( + plan.expressions(), + &[child_plan.inputs()[0].clone()], + )?; + Some(child_plan.with_new_exprs(child_plan.expressions(), &[new_limit])?) } _ => None, }; diff --git a/datafusion/optimizer/src/utils.rs b/datafusion/optimizer/src/utils.rs index 48f72ee7a0f8..44f2404afade 100644 --- a/datafusion/optimizer/src/utils.rs +++ b/datafusion/optimizer/src/utils.rs @@ -46,7 +46,7 @@ pub fn optimize_children( new_inputs.push(new_input.unwrap_or_else(|| input.clone())) } if plan_is_changed { - Ok(Some(plan.with_new_inputs(&new_inputs)?)) + Ok(Some(plan.with_new_exprs(plan.expressions(), &new_inputs)?)) } else { Ok(None) } From f4233a92761e9144b8747e66b95bf0b3f82464b8 Mon Sep 17 00:00:00 2001 From: Andrew Lamb Date: Tue, 2 Jan 2024 10:36:43 -0500 Subject: [PATCH 532/572] Minor: refactor bloom filter tests to reduce duplication (#8435) --- .../physical_plan/parquet/row_groups.rs | 343 ++++++++---------- 1 file changed, 153 insertions(+), 190 deletions(-) diff --git a/datafusion/core/src/datasource/physical_plan/parquet/row_groups.rs b/datafusion/core/src/datasource/physical_plan/parquet/row_groups.rs index 5d18eac7d9fb..24c65423dd4c 100644 --- a/datafusion/core/src/datasource/physical_plan/parquet/row_groups.rs +++ b/datafusion/core/src/datasource/physical_plan/parquet/row_groups.rs @@ -1013,82 +1013,28 @@ mod tests { create_physical_expr(expr, &df_schema, schema, &execution_props).unwrap() } - // Note the values in the `String` column are: - // ❯ select * from './parquet-testing/data/data_index_bloom_encoding_stats.parquet'; - // +-----------+ - // | String | - // +-----------+ - // | Hello | - // | This is | - // | a | - // | test | - // | How | - // | are you | - // | doing | - // | today | - // | the quick | - // | brown fox | - // | jumps | - // | over | - // | the lazy | - // | dog | - // +-----------+ #[tokio::test] async fn test_row_group_bloom_filter_pruning_predicate_simple_expr() { - // load parquet file - let testdata = datafusion_common::test_util::parquet_test_data(); - let file_name = "data_index_bloom_encoding_stats.parquet"; - let path = format!("{testdata}/{file_name}"); - let data = bytes::Bytes::from(std::fs::read(path).unwrap()); - - // generate pruning predicate `(String = "Hello_Not_exists")` - let schema = Schema::new(vec![Field::new("String", DataType::Utf8, false)]); - let expr = col(r#""String""#).eq(lit("Hello_Not_Exists")); - let expr = logical2physical(&expr, &schema); - let pruning_predicate = - PruningPredicate::try_new(expr, Arc::new(schema)).unwrap(); - - let row_groups = vec![0]; - let pruned_row_groups = test_row_group_bloom_filter_pruning_predicate( - file_name, - data, - &pruning_predicate, - &row_groups, - ) - .await - .unwrap(); - assert!(pruned_row_groups.is_empty()); + BloomFilterTest::new_data_index_bloom_encoding_stats() + .with_expect_all_pruned() + // generate pruning predicate `(String = "Hello_Not_exists")` + .run(col(r#""String""#).eq(lit("Hello_Not_Exists"))) + .await } #[tokio::test] async fn test_row_group_bloom_filter_pruning_predicate_mutiple_expr() { - // load parquet file - let testdata = datafusion_common::test_util::parquet_test_data(); - let file_name = "data_index_bloom_encoding_stats.parquet"; - let path = format!("{testdata}/{file_name}"); - let data = bytes::Bytes::from(std::fs::read(path).unwrap()); - - // generate pruning predicate `(String = "Hello_Not_exists" OR String = "Hello_Not_exists2")` - let schema = Schema::new(vec![Field::new("String", DataType::Utf8, false)]); - let expr = lit("1").eq(lit("1")).and( - col(r#""String""#) - .eq(lit("Hello_Not_Exists")) - .or(col(r#""String""#).eq(lit("Hello_Not_Exists2"))), - ); - let expr = logical2physical(&expr, &schema); - let pruning_predicate = - PruningPredicate::try_new(expr, Arc::new(schema)).unwrap(); - - let row_groups = vec![0]; - let pruned_row_groups = test_row_group_bloom_filter_pruning_predicate( - file_name, - data, - &pruning_predicate, - &row_groups, - ) - .await - .unwrap(); - assert!(pruned_row_groups.is_empty()); + BloomFilterTest::new_data_index_bloom_encoding_stats() + .with_expect_all_pruned() + // generate pruning predicate `(String = "Hello_Not_exists" OR String = "Hello_Not_exists2")` + .run( + lit("1").eq(lit("1")).and( + col(r#""String""#) + .eq(lit("Hello_Not_Exists")) + .or(col(r#""String""#).eq(lit("Hello_Not_Exists2"))), + ), + ) + .await } #[tokio::test] @@ -1129,144 +1075,161 @@ mod tests { #[tokio::test] async fn test_row_group_bloom_filter_pruning_predicate_with_exists_value() { - // load parquet file - let testdata = datafusion_common::test_util::parquet_test_data(); - let file_name = "data_index_bloom_encoding_stats.parquet"; - let path = format!("{testdata}/{file_name}"); - let data = bytes::Bytes::from(std::fs::read(path).unwrap()); - - // generate pruning predicate `(String = "Hello")` - let schema = Schema::new(vec![Field::new("String", DataType::Utf8, false)]); - let expr = col(r#""String""#).eq(lit("Hello")); - let expr = logical2physical(&expr, &schema); - let pruning_predicate = - PruningPredicate::try_new(expr, Arc::new(schema)).unwrap(); - - let row_groups = vec![0]; - let pruned_row_groups = test_row_group_bloom_filter_pruning_predicate( - file_name, - data, - &pruning_predicate, - &row_groups, - ) - .await - .unwrap(); - assert_eq!(pruned_row_groups, row_groups); + BloomFilterTest::new_data_index_bloom_encoding_stats() + .with_expect_none_pruned() + // generate pruning predicate `(String = "Hello")` + .run(col(r#""String""#).eq(lit("Hello"))) + .await } #[tokio::test] async fn test_row_group_bloom_filter_pruning_predicate_with_exists_2_values() { - // load parquet file - let testdata = datafusion_common::test_util::parquet_test_data(); - let file_name = "data_index_bloom_encoding_stats.parquet"; - let path = format!("{testdata}/{file_name}"); - let data = bytes::Bytes::from(std::fs::read(path).unwrap()); - - // generate pruning predicate `(String = "Hello") OR (String = "the quick")` - let schema = Schema::new(vec![Field::new("String", DataType::Utf8, false)]); - let expr = col(r#""String""#) - .eq(lit("Hello")) - .or(col(r#""String""#).eq(lit("the quick"))); - let expr = logical2physical(&expr, &schema); - let pruning_predicate = - PruningPredicate::try_new(expr, Arc::new(schema)).unwrap(); - - let row_groups = vec![0]; - let pruned_row_groups = test_row_group_bloom_filter_pruning_predicate( - file_name, - data, - &pruning_predicate, - &row_groups, - ) - .await - .unwrap(); - assert_eq!(pruned_row_groups, row_groups); + BloomFilterTest::new_data_index_bloom_encoding_stats() + .with_expect_none_pruned() + // generate pruning predicate `(String = "Hello") OR (String = "the quick")` + .run( + col(r#""String""#) + .eq(lit("Hello")) + .or(col(r#""String""#).eq(lit("the quick"))), + ) + .await } #[tokio::test] async fn test_row_group_bloom_filter_pruning_predicate_with_exists_3_values() { - // load parquet file - let testdata = datafusion_common::test_util::parquet_test_data(); - let file_name = "data_index_bloom_encoding_stats.parquet"; - let path = format!("{testdata}/{file_name}"); - let data = bytes::Bytes::from(std::fs::read(path).unwrap()); - - // generate pruning predicate `(String = "Hello") OR (String = "the quick") OR (String = "are you")` - let schema = Schema::new(vec![Field::new("String", DataType::Utf8, false)]); - let expr = col(r#""String""#) - .eq(lit("Hello")) - .or(col(r#""String""#).eq(lit("the quick"))) - .or(col(r#""String""#).eq(lit("are you"))); - let expr = logical2physical(&expr, &schema); - let pruning_predicate = - PruningPredicate::try_new(expr, Arc::new(schema)).unwrap(); - - let row_groups = vec![0]; - let pruned_row_groups = test_row_group_bloom_filter_pruning_predicate( - file_name, - data, - &pruning_predicate, - &row_groups, - ) - .await - .unwrap(); - assert_eq!(pruned_row_groups, row_groups); + BloomFilterTest::new_data_index_bloom_encoding_stats() + .with_expect_none_pruned() + // generate pruning predicate `(String = "Hello") OR (String = "the quick") OR (String = "are you")` + .run( + col(r#""String""#) + .eq(lit("Hello")) + .or(col(r#""String""#).eq(lit("the quick"))) + .or(col(r#""String""#).eq(lit("are you"))), + ) + .await } #[tokio::test] async fn test_row_group_bloom_filter_pruning_predicate_with_or_not_eq() { - // load parquet file - let testdata = datafusion_common::test_util::parquet_test_data(); - let file_name = "data_index_bloom_encoding_stats.parquet"; - let path = format!("{testdata}/{file_name}"); - let data = bytes::Bytes::from(std::fs::read(path).unwrap()); - - // generate pruning predicate `(String = "foo") OR (String != "bar")` - let schema = Schema::new(vec![Field::new("String", DataType::Utf8, false)]); - let expr = col(r#""String""#) - .not_eq(lit("foo")) - .or(col(r#""String""#).not_eq(lit("bar"))); - let expr = logical2physical(&expr, &schema); - let pruning_predicate = - PruningPredicate::try_new(expr, Arc::new(schema)).unwrap(); - - let row_groups = vec![0]; - let pruned_row_groups = test_row_group_bloom_filter_pruning_predicate( - file_name, - data, - &pruning_predicate, - &row_groups, - ) - .await - .unwrap(); - assert_eq!(pruned_row_groups, row_groups); + BloomFilterTest::new_data_index_bloom_encoding_stats() + .with_expect_none_pruned() + // generate pruning predicate `(String = "foo") OR (String != "bar")` + .run( + col(r#""String""#) + .not_eq(lit("foo")) + .or(col(r#""String""#).not_eq(lit("bar"))), + ) + .await } #[tokio::test] async fn test_row_group_bloom_filter_pruning_predicate_without_bloom_filter() { - // load parquet file - let testdata = datafusion_common::test_util::parquet_test_data(); - let file_name = "alltypes_plain.parquet"; - let path = format!("{testdata}/{file_name}"); - let data = bytes::Bytes::from(std::fs::read(path).unwrap()); - // generate pruning predicate on a column without a bloom filter - let schema = Schema::new(vec![Field::new("string_col", DataType::Utf8, false)]); - let expr = col(r#""string_col""#).eq(lit("0")); - let expr = logical2physical(&expr, &schema); - let pruning_predicate = - PruningPredicate::try_new(expr, Arc::new(schema)).unwrap(); + BloomFilterTest::new_all_types() + .with_expect_none_pruned() + .run(col(r#""string_col""#).eq(lit("0"))) + .await + } - let row_groups = vec![0]; - let pruned_row_groups = test_row_group_bloom_filter_pruning_predicate( - file_name, - data, - &pruning_predicate, - &row_groups, - ) - .await - .unwrap(); - assert_eq!(pruned_row_groups, row_groups); + struct BloomFilterTest { + file_name: String, + schema: Schema, + // which row groups should be attempted to prune + row_groups: Vec, + // which row groups are expected to be left after pruning. Must be set + // otherwise will panic on run() + post_pruning_row_groups: Option>, + } + + impl BloomFilterTest { + /// Return a test for data_index_bloom_encoding_stats.parquet + /// Note the values in the `String` column are: + /// ```sql + /// ❯ select * from './parquet-testing/data/data_index_bloom_encoding_stats.parquet'; + /// +-----------+ + /// | String | + /// +-----------+ + /// | Hello | + /// | This is | + /// | a | + /// | test | + /// | How | + /// | are you | + /// | doing | + /// | today | + /// | the quick | + /// | brown fox | + /// | jumps | + /// | over | + /// | the lazy | + /// | dog | + /// +-----------+ + /// ``` + fn new_data_index_bloom_encoding_stats() -> Self { + Self { + file_name: String::from("data_index_bloom_encoding_stats.parquet"), + schema: Schema::new(vec![Field::new("String", DataType::Utf8, false)]), + row_groups: vec![0], + post_pruning_row_groups: None, + } + } + + // Return a test for alltypes_plain.parquet + fn new_all_types() -> Self { + Self { + file_name: String::from("alltypes_plain.parquet"), + schema: Schema::new(vec![Field::new( + "string_col", + DataType::Utf8, + false, + )]), + row_groups: vec![0], + post_pruning_row_groups: None, + } + } + + /// Expect all row groups to be pruned + pub fn with_expect_all_pruned(mut self) -> Self { + self.post_pruning_row_groups = Some(vec![]); + self + } + + /// Expect all row groups not to be pruned + pub fn with_expect_none_pruned(mut self) -> Self { + self.post_pruning_row_groups = Some(self.row_groups.clone()); + self + } + + /// Prune this file using the specified expression and check that the expected row groups are left + async fn run(self, expr: Expr) { + let Self { + file_name, + schema, + row_groups, + post_pruning_row_groups, + } = self; + + let post_pruning_row_groups = + post_pruning_row_groups.expect("post_pruning_row_groups must be set"); + + let testdata = datafusion_common::test_util::parquet_test_data(); + let path = format!("{testdata}/{file_name}"); + let data = bytes::Bytes::from(std::fs::read(path).unwrap()); + + let expr = logical2physical(&expr, &schema); + let pruning_predicate = + PruningPredicate::try_new(expr, Arc::new(schema)).unwrap(); + + let pruned_row_groups = test_row_group_bloom_filter_pruning_predicate( + &file_name, + data, + &pruning_predicate, + &row_groups, + ) + .await + .unwrap(); + assert_eq!(pruned_row_groups, post_pruning_row_groups); + } } async fn test_row_group_bloom_filter_pruning_predicate( From 82656af2c79246f28b8519210be42de6e5a82e54 Mon Sep 17 00:00:00 2001 From: Alex Huang Date: Tue, 2 Jan 2024 23:48:29 +0800 Subject: [PATCH 533/572] clean up code (#8715) --- .../core/src/datasource/file_format/write/demux.rs | 4 ++-- .../datasource/physical_plan/parquet/page_filter.rs | 2 +- datafusion/core/src/execution/context/mod.rs | 10 ++++++---- .../core/src/physical_optimizer/sort_pushdown.rs | 4 +--- datafusion/core/src/physical_planner.rs | 5 +++-- datafusion/substrait/src/logical_plan/producer.rs | 2 +- 6 files changed, 14 insertions(+), 13 deletions(-) diff --git a/datafusion/core/src/datasource/file_format/write/demux.rs b/datafusion/core/src/datasource/file_format/write/demux.rs index fa4ed8437015..dbfeb67eaeb9 100644 --- a/datafusion/core/src/datasource/file_format/write/demux.rs +++ b/datafusion/core/src/datasource/file_format/write/demux.rs @@ -383,7 +383,7 @@ fn compute_take_arrays( fn remove_partition_by_columns( parted_batch: &RecordBatch, - partition_by: &Vec<(String, DataType)>, + partition_by: &[(String, DataType)], ) -> Result { let end_idx = parted_batch.num_columns() - partition_by.len(); let non_part_cols = &parted_batch.columns()[..end_idx]; @@ -405,7 +405,7 @@ fn remove_partition_by_columns( } fn compute_hive_style_file_path( - part_key: &Vec, + part_key: &[String], partition_by: &[(String, DataType)], write_id: &str, file_extension: &str, diff --git a/datafusion/core/src/datasource/physical_plan/parquet/page_filter.rs b/datafusion/core/src/datasource/physical_plan/parquet/page_filter.rs index f6310c49bcd6..a0637f379610 100644 --- a/datafusion/core/src/datasource/physical_plan/parquet/page_filter.rs +++ b/datafusion/core/src/datasource/physical_plan/parquet/page_filter.rs @@ -372,7 +372,7 @@ fn prune_pages_in_one_row_group( } fn create_row_count_in_each_page( - location: &Vec, + location: &[PageLocation], num_rows: usize, ) -> Vec { let mut vec = Vec::with_capacity(location.len()); diff --git a/datafusion/core/src/execution/context/mod.rs b/datafusion/core/src/execution/context/mod.rs index c51f2d132aad..d6b7f046f3e3 100644 --- a/datafusion/core/src/execution/context/mod.rs +++ b/datafusion/core/src/execution/context/mod.rs @@ -1719,7 +1719,7 @@ impl SessionState { let mut stringified_plans = e.stringified_plans.clone(); // analyze & capture output of each rule - let analyzed_plan = match self.analyzer.execute_and_check( + let analyzer_result = self.analyzer.execute_and_check( e.plan.as_ref(), self.options(), |analyzed_plan, analyzer| { @@ -1727,7 +1727,8 @@ impl SessionState { let plan_type = PlanType::AnalyzedLogicalPlan { analyzer_name }; stringified_plans.push(analyzed_plan.to_stringified(plan_type)); }, - ) { + ); + let analyzed_plan = match analyzer_result { Ok(plan) => plan, Err(DataFusionError::Context(analyzer_name, err)) => { let plan_type = PlanType::AnalyzedLogicalPlan { analyzer_name }; @@ -1750,7 +1751,7 @@ impl SessionState { .push(analyzed_plan.to_stringified(PlanType::FinalAnalyzedLogicalPlan)); // optimize the child plan, capturing the output of each optimizer - let (plan, logical_optimization_succeeded) = match self.optimizer.optimize( + let optimized_plan = self.optimizer.optimize( &analyzed_plan, self, |optimized_plan, optimizer| { @@ -1758,7 +1759,8 @@ impl SessionState { let plan_type = PlanType::OptimizedLogicalPlan { optimizer_name }; stringified_plans.push(optimized_plan.to_stringified(plan_type)); }, - ) { + ); + let (plan, logical_optimization_succeeded) = match optimized_plan { Ok(plan) => (Arc::new(plan), true), Err(DataFusionError::Context(optimizer_name, err)) => { let plan_type = PlanType::OptimizedLogicalPlan { optimizer_name }; diff --git a/datafusion/core/src/physical_optimizer/sort_pushdown.rs b/datafusion/core/src/physical_optimizer/sort_pushdown.rs index 97ca47baf05f..f0a8c8cfd3cb 100644 --- a/datafusion/core/src/physical_optimizer/sort_pushdown.rs +++ b/datafusion/core/src/physical_optimizer/sort_pushdown.rs @@ -405,9 +405,7 @@ fn shift_right_required( let new_right_required: Vec = parent_required .iter() .filter_map(|r| { - let Some(col) = r.expr.as_any().downcast_ref::() else { - return None; - }; + let col = r.expr.as_any().downcast_ref::()?; if col.index() < left_columns_len { return None; diff --git a/datafusion/core/src/physical_planner.rs b/datafusion/core/src/physical_planner.rs index 31d50be10f70..d696c55a8c13 100644 --- a/datafusion/core/src/physical_planner.rs +++ b/datafusion/core/src/physical_planner.rs @@ -1879,7 +1879,7 @@ impl DefaultPhysicalPlanner { ); } - match self.optimize_internal( + let optimized_plan = self.optimize_internal( input, session_state, |plan, optimizer| { @@ -1891,7 +1891,8 @@ impl DefaultPhysicalPlanner { .to_stringified(e.verbose, plan_type), ); }, - ) { + ); + match optimized_plan { Ok(input) => { // This plan will includes statistics if show_statistics is on stringified_plans.push( diff --git a/datafusion/substrait/src/logical_plan/producer.rs b/datafusion/substrait/src/logical_plan/producer.rs index 926883251a63..ab0e8c860858 100644 --- a/datafusion/substrait/src/logical_plan/producer.rs +++ b/datafusion/substrait/src/logical_plan/producer.rs @@ -608,7 +608,7 @@ pub fn parse_flat_grouping_exprs( pub fn to_substrait_groupings( ctx: &SessionContext, - exprs: &Vec, + exprs: &[Expr], schema: &DFSchemaRef, extension_info: &mut ( Vec, From 94aff5555874f023c934cd6c3a52dd956a773342 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Berkay=20=C5=9Eahin?= <124376117+berkaysynnada@users.noreply.github.com> Date: Tue, 2 Jan 2024 18:52:12 +0300 Subject: [PATCH 534/572] Update analyze.rs (#8717) --- datafusion/physical-plan/src/analyze.rs | 8 ++++++-- 1 file changed, 6 insertions(+), 2 deletions(-) diff --git a/datafusion/physical-plan/src/analyze.rs b/datafusion/physical-plan/src/analyze.rs index ded37983bb21..4f1578e220dd 100644 --- a/datafusion/physical-plan/src/analyze.rs +++ b/datafusion/physical-plan/src/analyze.rs @@ -115,8 +115,12 @@ impl ExecutionPlan for AnalyzeExec { /// Specifies whether this plan generates an infinite stream of records. /// If the plan does not support pipelining, but its input(s) are /// infinite, returns an error to indicate this. - fn unbounded_output(&self, _children: &[bool]) -> Result { - internal_err!("Optimization not supported for ANALYZE") + fn unbounded_output(&self, children: &[bool]) -> Result { + if children[0] { + internal_err!("Streaming execution of AnalyzeExec is not possible") + } else { + Ok(false) + } } /// Get the output partitioning of this plan From d4b96a80c86d216613ecbec24d4908bb31ed4c7e Mon Sep 17 00:00:00 2001 From: Alex Huang Date: Tue, 2 Jan 2024 23:57:26 +0800 Subject: [PATCH 535/572] support LargeList in array_position (#8714) --- .../physical-expr/src/array_expressions.rs | 14 ++-- datafusion/sqllogictest/test_files/array.slt | 71 +++++++++++++++++++ 2 files changed, 81 insertions(+), 4 deletions(-) diff --git a/datafusion/physical-expr/src/array_expressions.rs b/datafusion/physical-expr/src/array_expressions.rs index 250250630eff..9b93782237f8 100644 --- a/datafusion/physical-expr/src/array_expressions.rs +++ b/datafusion/physical-expr/src/array_expressions.rs @@ -1367,8 +1367,14 @@ pub fn array_position(args: &[ArrayRef]) -> Result { if args.len() < 2 || args.len() > 3 { return exec_err!("array_position expects two or three arguments"); } - - let list_array = as_list_array(&args[0])?; + match &args[0].data_type() { + DataType::List(_) => general_position_dispatch::(args), + DataType::LargeList(_) => general_position_dispatch::(args), + array_type => exec_err!("array_position does not support type '{array_type:?}'."), + } +} +fn general_position_dispatch(args: &[ArrayRef]) -> Result { + let list_array = as_generic_list_array::(&args[0])?; let element_array = &args[1]; check_datatypes("array_position", &[list_array.values(), element_array])?; @@ -1395,10 +1401,10 @@ pub fn array_position(args: &[ArrayRef]) -> Result { } } - general_position::(list_array, element_array, arr_from) + generic_position::(list_array, element_array, arr_from) } -fn general_position( +fn generic_position( list_array: &GenericListArray, element_array: &ArrayRef, arr_from: Vec, // 0-indexed diff --git a/datafusion/sqllogictest/test_files/array.slt b/datafusion/sqllogictest/test_files/array.slt index 6dab3b3084a9..4205f64c19d0 100644 --- a/datafusion/sqllogictest/test_files/array.slt +++ b/datafusion/sqllogictest/test_files/array.slt @@ -363,6 +363,17 @@ AS VALUES (make_array(31, 32, 33, 34, 35, 26, 37, 38, 39, 40), 34, 4, 'ok', [8,9]) ; +statement ok +CREATE TABLE large_arrays_values_without_nulls +AS SELECT + arrow_cast(column1, 'LargeList(Int64)') AS column1, + column2, + column3, + column4, + arrow_cast(column5, 'LargeList(Int64)') AS column5 +FROM arrays_values_without_nulls +; + statement ok CREATE TABLE arrays_range AS VALUES @@ -2054,12 +2065,22 @@ select array_position(['h', 'e', 'l', 'l', 'o'], 'l'), array_position([1, 2, 3, ---- 3 5 1 +query III +select array_position(arrow_cast(['h', 'e', 'l', 'l', 'o'], 'LargeList(Utf8)'), 'l'), array_position(arrow_cast([1, 2, 3, 4, 5], 'LargeList(Int64)'), 5), array_position(arrow_cast([1, 1, 1], 'LargeList(Int64)'), 1); +---- +3 5 1 + # array_position scalar function #2 (with optional argument) query III select array_position(['h', 'e', 'l', 'l', 'o'], 'l', 4), array_position([1, 2, 5, 4, 5], 5, 4), array_position([1, 1, 1], 1, 2); ---- 4 5 2 +query III +select array_position(arrow_cast(['h', 'e', 'l', 'l', 'o'], 'LargeList(Utf8)'), 'l', 4), array_position(arrow_cast([1, 2, 3, 4, 5], 'LargeList(Int64)'), 5, 4), array_position(arrow_cast([1, 1, 1], 'LargeList(Int64)'), 1, 2); +---- +4 5 2 + # array_position scalar function #3 (element is list) query II select array_position(make_array([1, 2, 3], [4, 5, 6], [5, 5, 5], [4, 5, 6], [7, 8, 9]), [4, 5, 6]), array_position(make_array([1, 3, 2], [2, 3, 4], [2, 3, 4], [5, 3, 1], [1, 3, 2]), [2, 3, 4]); @@ -2072,24 +2093,44 @@ select array_position(make_array([1, 2, 3], [4, 5, 6], [5, 5, 5], [4, 5, 6], [7, ---- 4 3 +query II +select array_position(arrow_cast(make_array([1, 2, 3], [4, 5, 6], [5, 5, 5], [4, 5, 6], [7, 8, 9]), 'LargeList(List(Int64))'), [4, 5, 6]), array_position(arrow_cast(make_array([1, 3, 2], [2, 3, 4], [2, 3, 4], [5, 3, 1], [1, 3, 2]), 'LargeList(List(Int64))'), [2, 3, 4]); +---- +2 2 + # list_position scalar function #5 (function alias `array_position`) query III select list_position(['h', 'e', 'l', 'l', 'o'], 'l'), list_position([1, 2, 3, 4, 5], 5), list_position([1, 1, 1], 1); ---- 3 5 1 +query III +select list_position(arrow_cast(['h', 'e', 'l', 'l', 'o'], 'LargeList(Utf8)'), 'l'), list_position(arrow_cast([1, 2, 3, 4, 5], 'LargeList(Int64)'), 5), list_position(arrow_cast([1, 1, 1], 'LargeList(Int64)'), 1); +---- +3 5 1 + # array_indexof scalar function #6 (function alias `array_position`) query III select array_indexof(['h', 'e', 'l', 'l', 'o'], 'l'), array_indexof([1, 2, 3, 4, 5], 5), array_indexof([1, 1, 1], 1); ---- 3 5 1 +query III +select array_indexof(arrow_cast(['h', 'e', 'l', 'l', 'o'], 'LargeList(Utf8)'), 'l'), array_indexof(arrow_cast([1, 2, 3, 4, 5], 'LargeList(Int64)'), 5), array_indexof(arrow_cast([1, 1, 1], 'LargeList(Int64)'), 1); +---- +3 5 1 + # list_indexof scalar function #7 (function alias `array_position`) query III select list_indexof(['h', 'e', 'l', 'l', 'o'], 'l'), list_indexof([1, 2, 3, 4, 5], 5), list_indexof([1, 1, 1], 1); ---- 3 5 1 +query III +select list_indexof(arrow_cast(['h', 'e', 'l', 'l', 'o'], 'LargeList(Utf8)'), 'l'), list_indexof(arrow_cast([1, 2, 3, 4, 5], 'LargeList(Int64)'), 5), list_indexof(arrow_cast([1, 1, 1], 'LargeList(Int64)'), 1); +---- +3 5 1 + # array_position with columns #1 query II select array_position(column1, column2), array_position(column1, column2, column3) from arrays_values_without_nulls; @@ -2099,6 +2140,14 @@ select array_position(column1, column2), array_position(column1, column2, column 3 3 4 4 +query II +select array_position(column1, column2), array_position(column1, column2, column3) from large_arrays_values_without_nulls; +---- +1 1 +2 2 +3 3 +4 4 + # array_position with columns #2 (element is list) query II select array_position(column1, column2), array_position(column1, column2, column3) from nested_arrays; @@ -2106,6 +2155,13 @@ select array_position(column1, column2), array_position(column1, column2, column 3 3 2 5 +#TODO: add this test when #8305 is fixed +#query II +#select array_position(column1, column2), array_position(column1, column2, column3) from nested_arrays; +#---- +#3 3 +#2 5 + # array_position with columns and scalars #1 query III select array_position(make_array(1, 2, 3, 4, 5), column2), array_position(column1, 3), array_position(column1, 3, 5) from arrays_values_without_nulls; @@ -2115,6 +2171,14 @@ NULL NULL NULL NULL NULL NULL NULL NULL NULL +query III +select array_position(arrow_cast(make_array(1, 2, 3, 4, 5), 'LargeList(Int64)'), column2), array_position(column1, 3), array_position(column1, 3, 5) from large_arrays_values_without_nulls; +---- +1 3 NULL +NULL NULL NULL +NULL NULL NULL +NULL NULL NULL + # array_position with columns and scalars #2 (element is list) query III select array_position(make_array([1, 2, 3], [4, 5, 6], [11, 12, 13]), column2), array_position(column1, make_array(4, 5, 6)), array_position(column1, make_array(1, 2, 3), 2) from nested_arrays; @@ -2122,6 +2186,13 @@ select array_position(make_array([1, 2, 3], [4, 5, 6], [11, 12, 13]), column2), NULL 6 4 NULL 1 NULL +#TODO: add this test when #8305 is fixed +#query III +#select array_position(arrow_cast(make_array([1, 2, 3], [4, 5, 6], [11, 12, 13]), 'LargeList(List(Int64))'), column2), array_position(column1, make_array(4, 5, 6)), array_position(column1, make_array(1, 2, 3), 2) from large_nested_arrays; +#---- +#NULL 6 4 +#NULL 1 NULL + ## array_positions (aliases: `list_positions`) # array_positions scalar function #1 From 96cede202a8a554051001143e8345883992c3f74 Mon Sep 17 00:00:00 2001 From: Alex Huang Date: Tue, 2 Jan 2024 23:58:18 +0800 Subject: [PATCH 536/572] support LargeList in array_ndims (#8716) --- datafusion/common/src/utils.rs | 9 +-- .../physical-expr/src/array_expressions.rs | 24 ++++++-- datafusion/sqllogictest/test_files/array.slt | 57 ++++++++++++++++++- 3 files changed, 77 insertions(+), 13 deletions(-) diff --git a/datafusion/common/src/utils.rs b/datafusion/common/src/utils.rs index cfdef309a4ee..49a00b24d10e 100644 --- a/datafusion/common/src/utils.rs +++ b/datafusion/common/src/utils.rs @@ -469,10 +469,11 @@ pub fn coerced_type_with_base_type_only( /// Compute the number of dimensions in a list data type. pub fn list_ndims(data_type: &DataType) -> u64 { - if let DataType::List(field) = data_type { - 1 + list_ndims(field.data_type()) - } else { - 0 + match data_type { + DataType::List(field) | DataType::LargeList(field) => { + 1 + list_ndims(field.data_type()) + } + _ => 0, } } diff --git a/datafusion/physical-expr/src/array_expressions.rs b/datafusion/physical-expr/src/array_expressions.rs index 9b93782237f8..92ba7a4d1dcd 100644 --- a/datafusion/physical-expr/src/array_expressions.rs +++ b/datafusion/physical-expr/src/array_expressions.rs @@ -2250,11 +2250,13 @@ pub fn array_ndims(args: &[ArrayRef]) -> Result { return exec_err!("array_ndims needs one argument"); } - if let Some(list_array) = args[0].as_list_opt::() { - let ndims = datafusion_common::utils::list_ndims(list_array.data_type()); + fn general_list_ndims( + array: &GenericListArray, + ) -> Result { + let mut data = Vec::new(); + let ndims = datafusion_common::utils::list_ndims(array.data_type()); - let mut data = vec![]; - for arr in list_array.iter() { + for arr in array.iter() { if arr.is_some() { data.push(Some(ndims)) } else { @@ -2263,8 +2265,18 @@ pub fn array_ndims(args: &[ArrayRef]) -> Result { } Ok(Arc::new(UInt64Array::from(data)) as ArrayRef) - } else { - Ok(Arc::new(UInt64Array::from(vec![0; args[0].len()])) as ArrayRef) + } + + match args[0].data_type() { + DataType::List(_) => { + let array = as_list_array(&args[0])?; + general_list_ndims::(array) + } + DataType::LargeList(_) => { + let array = as_large_list_array(&args[0])?; + general_list_ndims::(array) + } + _ => Ok(Arc::new(UInt64Array::from(vec![0; args[0].len()])) as ArrayRef), } } diff --git a/datafusion/sqllogictest/test_files/array.slt b/datafusion/sqllogictest/test_files/array.slt index 4205f64c19d0..2f8e3c805f73 100644 --- a/datafusion/sqllogictest/test_files/array.slt +++ b/datafusion/sqllogictest/test_files/array.slt @@ -3504,7 +3504,7 @@ NULL [3] [4] # array_ndims scalar function #1 query III -select +select array_ndims(1), array_ndims(null), array_ndims([2, 3]); @@ -3520,8 +3520,17 @@ AS VALUES (3, [6], [[9]], [[[[[10]]]]]) ; +statement ok +CREATE TABLE large_array_ndims_table +AS SELECT + column1, + arrow_cast(column2, 'LargeList(Int64)') as column2, + arrow_cast(column3, 'LargeList(List(Int64))') as column3, + arrow_cast(column4, 'LargeList(List(List(List(List(Int64)))))') as column4 +FROM array_ndims_table; + query IIII -select +select array_ndims(column1), array_ndims(column2), array_ndims(column3), @@ -3533,9 +3542,25 @@ from array_ndims_table; 0 1 2 5 0 1 2 5 +query IIII +select + array_ndims(column1), + array_ndims(column2), + array_ndims(column3), + array_ndims(column4) +from large_array_ndims_table; +---- +0 1 2 5 +0 1 2 5 +0 1 2 5 +0 1 2 5 + statement ok drop table array_ndims_table; +statement ok +drop table large_array_ndims_table + query I select array_ndims(arrow_cast([null], 'List(List(List(Int64)))')); ---- @@ -3553,14 +3578,29 @@ select array_ndims(make_array()), array_ndims(make_array(make_array())) ---- 1 2 +query II +select array_ndims(arrow_cast(make_array(), 'LargeList(Null)')), array_ndims(arrow_cast(make_array(make_array()), 'LargeList(List(Null))')) +---- +1 2 + # list_ndims scalar function #4 (function alias `array_ndims`) query III select list_ndims(make_array(1, 2, 3)), list_ndims(make_array([1, 2], [3, 4])), list_ndims(make_array([[[[1], [2]]]])); ---- 1 2 5 +query III +select list_ndims(arrow_cast(make_array(1, 2, 3), 'LargeList(Int64)')), list_ndims(arrow_cast(make_array([1, 2], [3, 4]), 'LargeList(List(Int64))')), list_ndims(arrow_cast(make_array([[[[1], [2]]]]), 'LargeList(List(List(List(List(Int64)))))')); +---- +1 2 5 + query II -select array_ndims(make_array()), array_ndims(make_array(make_array())) +select list_ndims(make_array()), list_ndims(make_array(make_array())) +---- +1 2 + +query II +select list_ndims(arrow_cast(make_array(), 'LargeList(Null)')), list_ndims(arrow_cast(make_array(make_array()), 'LargeList(List(Null))')) ---- 1 2 @@ -3576,6 +3616,17 @@ NULL 1 1 2 NULL 1 2 1 NULL +query III +select array_ndims(column1), array_ndims(column2), array_ndims(column3) from large_arrays; +---- +2 1 1 +2 1 1 +2 1 1 +2 1 1 +NULL 1 1 +2 NULL 1 +2 1 NULL + ## array_has/array_has_all/array_has_any query BBBBBBBBBBBB From c1fe3dd8f95ab75511c3295e87782373ad060877 Mon Sep 17 00:00:00 2001 From: Ashim Sedhain <38435962+asimsedhain@users.noreply.github.com> Date: Tue, 2 Jan 2024 09:59:15 -0600 Subject: [PATCH 537/572] feat: remove filters with null constants (#8700) --- datafusion/optimizer/src/eliminate_filter.rs | 33 +++++++++++++++----- 1 file changed, 25 insertions(+), 8 deletions(-) diff --git a/datafusion/optimizer/src/eliminate_filter.rs b/datafusion/optimizer/src/eliminate_filter.rs index c97906a81adf..fea14342ca77 100644 --- a/datafusion/optimizer/src/eliminate_filter.rs +++ b/datafusion/optimizer/src/eliminate_filter.rs @@ -15,7 +15,7 @@ // specific language governing permissions and limitations // under the License. -//! Optimizer rule to replace `where false` on a plan with an empty relation. +//! Optimizer rule to replace `where false or null` on a plan with an empty relation. //! This saves time in planning and executing the query. //! Note that this rule should be applied after simplify expressions optimizer rule. use crate::optimizer::ApplyOrder; @@ -27,7 +27,7 @@ use datafusion_expr::{ use crate::{OptimizerConfig, OptimizerRule}; -/// Optimization rule that eliminate the scalar value (true/false) filter with an [LogicalPlan::EmptyRelation] +/// Optimization rule that eliminate the scalar value (true/false/null) filter with an [LogicalPlan::EmptyRelation] #[derive(Default)] pub struct EliminateFilter; @@ -46,20 +46,22 @@ impl OptimizerRule for EliminateFilter { ) -> Result> { match plan { LogicalPlan::Filter(Filter { - predicate: Expr::Literal(ScalarValue::Boolean(Some(v))), + predicate: Expr::Literal(ScalarValue::Boolean(v)), input, .. }) => { match *v { // input also can be filter, apply again - true => Ok(Some( + Some(true) => Ok(Some( self.try_optimize(input, _config)? .unwrap_or_else(|| input.as_ref().clone()), )), - false => Ok(Some(LogicalPlan::EmptyRelation(EmptyRelation { - produce_one_row: false, - schema: input.schema().clone(), - }))), + Some(false) | None => { + Ok(Some(LogicalPlan::EmptyRelation(EmptyRelation { + produce_one_row: false, + schema: input.schema().clone(), + }))) + } } } _ => Ok(None), @@ -105,6 +107,21 @@ mod tests { assert_optimized_plan_equal(&plan, expected) } + #[test] + fn filter_null() -> Result<()> { + let filter_expr = Expr::Literal(ScalarValue::Boolean(None)); + + let table_scan = test_table_scan().unwrap(); + let plan = LogicalPlanBuilder::from(table_scan) + .aggregate(vec![col("a")], vec![sum(col("b"))])? + .filter(filter_expr)? + .build()?; + + // No aggregate / scan / limit + let expected = "EmptyRelation"; + assert_optimized_plan_equal(&plan, expected) + } + #[test] fn filter_false_nested() -> Result<()> { let filter_expr = Expr::Literal(ScalarValue::Boolean(Some(false))); From 67baf10249b26b4983d3cc3145817903dad8dcd4 Mon Sep 17 00:00:00 2001 From: Alex Huang Date: Wed, 3 Jan 2024 06:37:20 +0800 Subject: [PATCH 538/572] support `LargeList` in `array_prepend` and `array_append` (#8679) * support largelist * fix cast error * fix cast * add tests * fix conflict * s TODO comment for future tests add TODO comment for future tests --------- Co-authored-by: hwj --- datafusion/common/src/utils.rs | 23 ++- .../expr/src/type_coercion/functions.rs | 24 +-- .../physical-expr/src/array_expressions.rs | 144 +++++++------- datafusion/sqllogictest/test_files/array.slt | 184 +++++++++++++++++- 4 files changed, 284 insertions(+), 91 deletions(-) diff --git a/datafusion/common/src/utils.rs b/datafusion/common/src/utils.rs index 49a00b24d10e..0a61fce15482 100644 --- a/datafusion/common/src/utils.rs +++ b/datafusion/common/src/utils.rs @@ -424,10 +424,11 @@ pub fn arrays_into_list_array( /// assert_eq!(base_type(&data_type), DataType::Int32); /// ``` pub fn base_type(data_type: &DataType) -> DataType { - if let DataType::List(field) = data_type { - base_type(field.data_type()) - } else { - data_type.to_owned() + match data_type { + DataType::List(field) | DataType::LargeList(field) => { + base_type(field.data_type()) + } + _ => data_type.to_owned(), } } @@ -462,6 +463,20 @@ pub fn coerced_type_with_base_type_only( field.is_nullable(), ))) } + DataType::LargeList(field) => { + let data_type = match field.data_type() { + DataType::LargeList(_) => { + coerced_type_with_base_type_only(field.data_type(), base_type) + } + _ => base_type.to_owned(), + }; + + DataType::LargeList(Arc::new(Field::new( + field.name(), + data_type, + field.is_nullable(), + ))) + } _ => base_type.clone(), } diff --git a/datafusion/expr/src/type_coercion/functions.rs b/datafusion/expr/src/type_coercion/functions.rs index fa47c92762bf..63908d539bd0 100644 --- a/datafusion/expr/src/type_coercion/functions.rs +++ b/datafusion/expr/src/type_coercion/functions.rs @@ -116,18 +116,18 @@ fn get_valid_types( &new_base_type, ); - if let DataType::List(ref field) = array_type { - let elem_type = field.data_type(); - if is_append { - Ok(vec![vec![array_type.clone(), elem_type.to_owned()]]) - } else { - Ok(vec![vec![elem_type.to_owned(), array_type.clone()]]) + match array_type { + DataType::List(ref field) | DataType::LargeList(ref field) => { + let elem_type = field.data_type(); + if is_append { + Ok(vec![vec![array_type.clone(), elem_type.to_owned()]]) + } else { + Ok(vec![vec![elem_type.to_owned(), array_type.clone()]]) + } } - } else { - Ok(vec![vec![]]) + _ => Ok(vec![vec![]]), } } - let valid_types = match signature { TypeSignature::Variadic(valid_types) => valid_types .iter() @@ -311,9 +311,9 @@ fn coerced_from<'a>( Utf8 | LargeUtf8 => Some(type_into.clone()), Null if can_cast_types(type_from, type_into) => Some(type_into.clone()), - // Only accept list with the same number of dimensions unless the type is Null. - // List with different dimensions should be handled in TypeSignature or other places before this. - List(_) + // Only accept list and largelist with the same number of dimensions unless the type is Null. + // List or LargeList with different dimensions should be handled in TypeSignature or other places before this. + List(_) | LargeList(_) if datafusion_common::utils::base_type(type_from).eq(&Null) || list_ndims(type_from) == list_ndims(type_into) => { diff --git a/datafusion/physical-expr/src/array_expressions.rs b/datafusion/physical-expr/src/array_expressions.rs index 92ba7a4d1dcd..aad021610fcb 100644 --- a/datafusion/physical-expr/src/array_expressions.rs +++ b/datafusion/physical-expr/src/array_expressions.rs @@ -52,22 +52,6 @@ macro_rules! downcast_arg { }}; } -/// Downcasts multiple arguments into a single concrete type -/// $ARGS: &[ArrayRef] -/// $ARRAY_TYPE: type to downcast to -/// -/// $returns a Vec<$ARRAY_TYPE> -macro_rules! downcast_vec { - ($ARGS:expr, $ARRAY_TYPE:ident) => {{ - $ARGS - .iter() - .map(|e| match e.as_any().downcast_ref::<$ARRAY_TYPE>() { - Some(array) => Ok(array), - _ => internal_err!("failed to downcast"), - }) - }}; -} - /// Computes a BooleanArray indicating equality or inequality between elements in a list array and a specified element array. /// /// # Arguments @@ -832,17 +816,20 @@ pub fn array_pop_back(args: &[ArrayRef]) -> Result { /// /// # Examples /// -/// general_append_and_prepend( +/// generic_append_and_prepend( /// [1, 2, 3], 4, append => [1, 2, 3, 4] /// 5, [6, 7, 8], prepend => [5, 6, 7, 8] /// ) -fn general_append_and_prepend( - list_array: &ListArray, +fn generic_append_and_prepend( + list_array: &GenericListArray, element_array: &ArrayRef, data_type: &DataType, is_append: bool, -) -> Result { - let mut offsets = vec![0]; +) -> Result +where + i64: TryInto, +{ + let mut offsets = vec![O::usize_as(0)]; let values = list_array.values(); let original_data = values.to_data(); let element_data = element_array.to_data(); @@ -858,8 +845,8 @@ fn general_append_and_prepend( let element_index = 1; for (row_index, offset_window) in list_array.offsets().windows(2).enumerate() { - let start = offset_window[0] as usize; - let end = offset_window[1] as usize; + let start = offset_window[0].to_usize().unwrap(); + let end = offset_window[1].to_usize().unwrap(); if is_append { mutable.extend(values_index, start, end); mutable.extend(element_index, row_index, row_index + 1); @@ -867,12 +854,12 @@ fn general_append_and_prepend( mutable.extend(element_index, row_index, row_index + 1); mutable.extend(values_index, start, end); } - offsets.push(offsets[row_index] + (end - start + 1) as i32); + offsets.push(offsets[row_index] + O::usize_as(end - start + 1)); } let data = mutable.freeze(); - Ok(Arc::new(ListArray::try_new( + Ok(Arc::new(GenericListArray::::try_new( Arc::new(Field::new("item", data_type.to_owned(), true)), OffsetBuffer::new(offsets.into()), arrow_array::make_array(data), @@ -938,36 +925,6 @@ pub fn gen_range(args: &[ArrayRef]) -> Result { Ok(arr) } -/// Array_append SQL function -pub fn array_append(args: &[ArrayRef]) -> Result { - if args.len() != 2 { - return exec_err!("array_append expects two arguments"); - } - - let list_array = as_list_array(&args[0])?; - let element_array = &args[1]; - - let res = match list_array.value_type() { - DataType::List(_) => concat_internal(args)?, - DataType::Null => { - return make_array(&[ - list_array.values().to_owned(), - element_array.to_owned(), - ]); - } - data_type => { - return general_append_and_prepend( - list_array, - element_array, - &data_type, - true, - ); - } - }; - - Ok(res) -} - /// Array_sort SQL function pub fn array_sort(args: &[ArrayRef]) -> Result { if args.is_empty() || args.len() > 3 { @@ -1051,25 +1008,40 @@ fn order_nulls_first(modifier: &str) -> Result { } } -/// Array_prepend SQL function -pub fn array_prepend(args: &[ArrayRef]) -> Result { - if args.len() != 2 { - return exec_err!("array_prepend expects two arguments"); - } - - let list_array = as_list_array(&args[1])?; - let element_array = &args[0]; +fn general_append_and_prepend( + args: &[ArrayRef], + is_append: bool, +) -> Result +where + i64: TryInto, +{ + let (list_array, element_array) = if is_append { + let list_array = as_generic_list_array::(&args[0])?; + let element_array = &args[1]; + check_datatypes("array_append", &[element_array, list_array.values()])?; + (list_array, element_array) + } else { + let list_array = as_generic_list_array::(&args[1])?; + let element_array = &args[0]; + check_datatypes("array_prepend", &[list_array.values(), element_array])?; + (list_array, element_array) + }; - check_datatypes("array_prepend", &[element_array, list_array.values()])?; let res = match list_array.value_type() { - DataType::List(_) => concat_internal(args)?, - DataType::Null => return make_array(&[element_array.to_owned()]), + DataType::List(_) => concat_internal::(args)?, + DataType::LargeList(_) => concat_internal::(args)?, + DataType::Null => { + return make_array(&[ + list_array.values().to_owned(), + element_array.to_owned(), + ]); + } data_type => { - return general_append_and_prepend( + return generic_append_and_prepend::( list_array, element_array, &data_type, - false, + is_append, ); } }; @@ -1077,6 +1049,30 @@ pub fn array_prepend(args: &[ArrayRef]) -> Result { Ok(res) } +/// Array_append SQL function +pub fn array_append(args: &[ArrayRef]) -> Result { + if args.len() != 2 { + return exec_err!("array_append expects two arguments"); + } + + match args[0].data_type() { + DataType::LargeList(_) => general_append_and_prepend::(args, true), + _ => general_append_and_prepend::(args, true), + } +} + +/// Array_prepend SQL function +pub fn array_prepend(args: &[ArrayRef]) -> Result { + if args.len() != 2 { + return exec_err!("array_prepend expects two arguments"); + } + + match args[1].data_type() { + DataType::LargeList(_) => general_append_and_prepend::(args, false), + _ => general_append_and_prepend::(args, false), + } +} + fn align_array_dimensions(args: Vec) -> Result> { let args_ndim = args .iter() @@ -1114,11 +1110,13 @@ fn align_array_dimensions(args: Vec) -> Result> { } // Concatenate arrays on the same row. -fn concat_internal(args: &[ArrayRef]) -> Result { +fn concat_internal(args: &[ArrayRef]) -> Result { let args = align_array_dimensions(args.to_vec())?; - let list_arrays = - downcast_vec!(args, ListArray).collect::>>()?; + let list_arrays = args + .iter() + .map(|arg| as_generic_list_array::(arg)) + .collect::>>()?; // Assume number of rows is the same for all arrays let row_count = list_arrays[0].len(); @@ -1165,7 +1163,7 @@ fn concat_internal(args: &[ArrayRef]) -> Result { .map(|a| a.as_ref()) .collect::>(); - let list_arr = ListArray::new( + let list_arr = GenericListArray::::new( Arc::new(Field::new("item", data_type, true)), OffsetBuffer::from_lengths(array_lengths), Arc::new(compute::concat(elements.as_slice())?), @@ -1192,7 +1190,7 @@ pub fn array_concat(args: &[ArrayRef]) -> Result { } } - concat_internal(new_args.as_slice()) + concat_internal::(new_args.as_slice()) } /// Array_empty SQL function diff --git a/datafusion/sqllogictest/test_files/array.slt b/datafusion/sqllogictest/test_files/array.slt index 2f8e3c805f73..a3b2c8cdf1e9 100644 --- a/datafusion/sqllogictest/test_files/array.slt +++ b/datafusion/sqllogictest/test_files/array.slt @@ -107,6 +107,19 @@ AS VALUES (make_array(make_array(4, 5, 6), make_array(10, 11, 12), make_array(4, 9, 8), make_array(7, 8, 9), make_array(10, 11, 12), make_array(1, 8, 7)), make_array(10, 11, 12), 3, make_array([[11, 12, 13], [14, 15, 16]], [[17, 18, 19], [20, 21, 22]]), make_array(121, 131, 141)) ; +# TODO: add this when #8305 is fixed +# statement ok +# CREATE TABLE large_nested_arrays +# AS +# SELECT +# arrow_cast(column1, 'LargeList(LargeList(Int64))') AS column1, +# arrow_cast(column2, 'LargeList(Int64)') AS column2, +# column3, +# arrow_cast(column4, 'LargeList(LargeList(List(Int64)))') AS column4, +# arrow_cast(column5, 'LargeList(Int64)') AS column5 +# FROM nested_arrays +# ; + statement ok CREATE TABLE arrays_values AS VALUES @@ -120,6 +133,17 @@ AS VALUES (make_array(61, 62, 63, 64, 65, 66, 67, 68, 69, 70), 66, 7, NULL) ; +statement ok +CREATE TABLE large_arrays_values +AS SELECT + arrow_cast(column1, 'LargeList(Int64)') AS column1, + column2, + column3, + column4 +FROM arrays_values +; + + statement ok CREATE TABLE arrays_values_v2 AS VALUES @@ -131,6 +155,17 @@ AS VALUES (NULL, NULL, NULL, NULL) ; +# TODO: add this when #8305 is fixed +# statement ok +# CREATE TABLE large_arrays_values_v2 +# AS SELECT +# arrow_cast(column1, 'LargeList(Int64)') AS column1, +# arrow_cast(column2, 'LargeList(Int64)') AS column2, +# column3, +# arrow_cast(column4, 'LargeList(LargeList(Int64))') AS column4 +# FROM arrays_values_v2 +# ; + statement ok CREATE TABLE flatten_table AS VALUES @@ -1532,7 +1567,7 @@ query error select array_append(null, [[4]]); query ???? -select +select array_append(make_array(), 4), array_append(make_array(), null), array_append(make_array(1, null, 3), 4), @@ -1541,6 +1576,17 @@ select ---- [4] [] [1, , 3, 4] [, , 1] +# TODO: add this when #8305 is fixed +# query ???? +# select +# array_append(arrow_cast(make_array(), 'LargeList(Null)'), 4), +# array_append(make_array(), null), +# array_append(make_array(1, null, 3), 4), +# array_append(make_array(null, null), 1) +# ; +# ---- +# [4] [] [1, , 3, 4] [, , 1] + # test invalid (non-null) query error select array_append(1, 2); @@ -1552,42 +1598,76 @@ query error select array_append([1], [2]); query ?? -select +select array_append(make_array(make_array(1, null, 3)), make_array(null)), array_append(make_array(make_array(1, null, 3)), null); ---- [[1, , 3], []] [[1, , 3], ] +# TODO: add this when #8305 is fixed +# query ?? +# select +# array_append(arrow_cast(make_array(make_array(1, null, 3), 'LargeList(LargeList(Int64))')), arrow_cast(make_array(null), 'LargeList(Int64)')), +# array_append(arrow_cast(make_array(make_array(1, null, 3), 'LargeList(LargeList(Int64))')), null); +# ---- +# [[1, , 3], []] [[1, , 3], ] + # array_append scalar function #3 query ??? select array_append(make_array(1, 2, 3), 4), array_append(make_array(1.0, 2.0, 3.0), 4.0), array_append(make_array('h', 'e', 'l', 'l'), 'o'); ---- [1, 2, 3, 4] [1.0, 2.0, 3.0, 4.0] [h, e, l, l, o] +query ??? +select array_append(arrow_cast(make_array(1, 2, 3), 'LargeList(Int64)'), 4), array_append(arrow_cast(make_array(1.0, 2.0, 3.0), 'LargeList(Float64)'), 4.0), array_append(make_array('h', 'e', 'l', 'l'), 'o'); +---- +[1, 2, 3, 4] [1.0, 2.0, 3.0, 4.0] [h, e, l, l, o] + # array_append scalar function #4 (element is list) query ??? select array_append(make_array([1], [2], [3]), make_array(4)), array_append(make_array([1.0], [2.0], [3.0]), make_array(4.0)), array_append(make_array(['h'], ['e'], ['l'], ['l']), make_array('o')); ---- [[1], [2], [3], [4]] [[1.0], [2.0], [3.0], [4.0]] [[h], [e], [l], [l], [o]] +# TODO: add this when #8305 is fixed +# query ??? +# select array_append(arrow_cast(make_array([1], [2], [3]), 'LargeList(LargeList(Int64))'), arrow_cast(make_array(4), 'LargeList(Int64)')), array_append(arrow_cast(make_array([1.0], [2.0], [3.0]), 'LargeList(LargeList(Float64))'), arrow_cast(make_array(4.0), 'LargeList(Float64)')), array_append(arrow_cast(make_array(['h'], ['e'], ['l'], ['l']), 'LargeList(LargeList(Utf8))'), arrow_cast(make_array('o'), 'LargeList(Utf8)')); +# ---- +# [[1], [2], [3], [4]] [[1.0], [2.0], [3.0], [4.0]] [[h], [e], [l], [l], [o]] + # list_append scalar function #5 (function alias `array_append`) query ??? select list_append(make_array(1, 2, 3), 4), list_append(make_array(1.0, 2.0, 3.0), 4.0), list_append(make_array('h', 'e', 'l', 'l'), 'o'); ---- [1, 2, 3, 4] [1.0, 2.0, 3.0, 4.0] [h, e, l, l, o] +query ??? +select list_append(arrow_cast(make_array(1, 2, 3), 'LargeList(Int64)'), 4), list_append(arrow_cast(make_array(1.0, 2.0, 3.0), 'LargeList(Float64)'), 4.0), list_append(make_array('h', 'e', 'l', 'l'), 'o'); +---- +[1, 2, 3, 4] [1.0, 2.0, 3.0, 4.0] [h, e, l, l, o] + # array_push_back scalar function #6 (function alias `array_append`) query ??? select array_push_back(make_array(1, 2, 3), 4), array_push_back(make_array(1.0, 2.0, 3.0), 4.0), array_push_back(make_array('h', 'e', 'l', 'l'), 'o'); ---- [1, 2, 3, 4] [1.0, 2.0, 3.0, 4.0] [h, e, l, l, o] +query ??? +select array_push_back(arrow_cast(make_array(1, 2, 3), 'LargeList(Int64)'), 4), array_push_back(arrow_cast(make_array(1.0, 2.0, 3.0), 'LargeList(Float64)'), 4.0), array_push_back(make_array('h', 'e', 'l', 'l'), 'o'); +---- +[1, 2, 3, 4] [1.0, 2.0, 3.0, 4.0] [h, e, l, l, o] + # list_push_back scalar function #7 (function alias `array_append`) query ??? select list_push_back(make_array(1, 2, 3), 4), list_push_back(make_array(1.0, 2.0, 3.0), 4.0), list_push_back(make_array('h', 'e', 'l', 'l'), 'o'); ---- [1, 2, 3, 4] [1.0, 2.0, 3.0, 4.0] [h, e, l, l, o] +query ??? +select list_push_back(arrow_cast(make_array(1, 2, 3), 'LargeList(Int64)'), 4), list_push_back(arrow_cast(make_array(1.0, 2.0, 3.0), 'LargeList(Float64)'), 4.0), list_push_back(make_array('h', 'e', 'l', 'l'), 'o'); +---- +[1, 2, 3, 4] [1.0, 2.0, 3.0, 4.0] [h, e, l, l, o] + # array_append with columns #1 query ? select array_append(column1, column2) from arrays_values; @@ -1601,6 +1681,18 @@ select array_append(column1, column2) from arrays_values; [51, 52, , 54, 55, 56, 57, 58, 59, 60, 55] [61, 62, 63, 64, 65, 66, 67, 68, 69, 70, 66] +query ? +select array_append(column1, column2) from large_arrays_values; +---- +[, 2, 3, 4, 5, 6, 7, 8, 9, 10, 1] +[11, 12, 13, 14, 15, 16, 17, 18, , 20, 12] +[21, 22, 23, , 25, 26, 27, 28, 29, 30, 23] +[31, 32, 33, 34, 35, , 37, 38, 39, 40, 34] +[44] +[41, 42, 43, 44, 45, 46, 47, 48, 49, 50, ] +[51, 52, , 54, 55, 56, 57, 58, 59, 60, 55] +[61, 62, 63, 64, 65, 66, 67, 68, 69, 70, 66] + # array_append with columns #2 (element is list) query ? select array_append(column1, column2) from nested_arrays; @@ -1608,6 +1700,13 @@ select array_append(column1, column2) from nested_arrays; [[1, 2, 3], [2, 9, 1], [7, 8, 9], [1, 2, 3], [1, 7, 4], [4, 5, 6], [7, 8, 9]] [[4, 5, 6], [10, 11, 12], [4, 9, 8], [7, 8, 9], [10, 11, 12], [1, 8, 7], [10, 11, 12]] +# TODO: add this when #8305 is fixed +# query ? +# select array_append(column1, column2) from large_nested_arrays; +# ---- +# [[1, 2, 3], [2, 9, 1], [7, 8, 9], [1, 2, 3], [1, 7, 4], [4, 5, 6], [7, 8, 9]] +# [[4, 5, 6], [10, 11, 12], [4, 9, 8], [7, 8, 9], [10, 11, 12], [1, 8, 7], [10, 11, 12]] + # array_append with columns and scalars #1 query ?? select array_append(column2, 100.1), array_append(column3, '.') from arrays; @@ -1620,6 +1719,17 @@ select array_append(column2, 100.1), array_append(column3, '.') from arrays; [100.1] [,, .] [16.6, 17.7, 18.8, 100.1] [.] +query ?? +select array_append(column2, 100.1), array_append(column3, '.') from large_arrays; +---- +[1.1, 2.2, 3.3, 100.1] [L, o, r, e, m, .] +[, 5.5, 6.6, 100.1] [i, p, , u, m, .] +[7.7, 8.8, 9.9, 100.1] [d, , l, o, r, .] +[10.1, , 12.2, 100.1] [s, i, t, .] +[13.3, 14.4, 15.5, 100.1] [a, m, e, t, .] +[100.1] [,, .] +[16.6, 17.7, 18.8, 100.1] [.] + # array_append with columns and scalars #2 query ?? select array_append(column1, make_array(1, 11, 111)), array_append(make_array(make_array(1, 2, 3), make_array(11, 12, 13)), column2) from nested_arrays; @@ -1627,6 +1737,13 @@ select array_append(column1, make_array(1, 11, 111)), array_append(make_array(ma [[1, 2, 3], [2, 9, 1], [7, 8, 9], [1, 2, 3], [1, 7, 4], [4, 5, 6], [1, 11, 111]] [[1, 2, 3], [11, 12, 13], [7, 8, 9]] [[4, 5, 6], [10, 11, 12], [4, 9, 8], [7, 8, 9], [10, 11, 12], [1, 8, 7], [1, 11, 111]] [[1, 2, 3], [11, 12, 13], [10, 11, 12]] +# TODO: add this when #8305 is fixed +# query ?? +# select array_append(column1, arrow_cast(make_array(1, 11, 111), 'LargeList(Int64)')), array_append(arrow_cast(make_array(make_array(1, 2, 3), make_array(11, 12, 13)), 'LargeList(LargeList(Int64))'), column2) from large_nested_arrays; +# ---- +# [[1, 2, 3], [2, 9, 1], [7, 8, 9], [1, 2, 3], [1, 7, 4], [4, 5, 6], [1, 11, 111]] [[1, 2, 3], [11, 12, 13], [7, 8, 9]] +# [[4, 5, 6], [10, 11, 12], [4, 9, 8], [7, 8, 9], [10, 11, 12], [1, 8, 7], [1, 11, 111]] [[1, 2, 3], [11, 12, 13], [10, 11, 12]] + ## array_prepend (aliases: `list_prepend`, `array_push_front`, `list_push_front`) # array_prepend with NULLs @@ -1688,30 +1805,56 @@ select array_prepend(1, make_array(2, 3, 4)), array_prepend(1.0, make_array(2.0, ---- [1, 2, 3, 4] [1.0, 2.0, 3.0, 4.0] [h, e, l, l, o] +query ??? +select array_prepend(1, arrow_cast(make_array(2, 3, 4), 'LargeList(Int64)')), array_prepend(1.0, arrow_cast(make_array(2.0, 3.0, 4.0), 'LargeList(Float64)')), array_prepend('h', arrow_cast(make_array('e', 'l', 'l', 'o'), 'LargeList(Utf8)')); +---- +[1, 2, 3, 4] [1.0, 2.0, 3.0, 4.0] [h, e, l, l, o] + # array_prepend scalar function #4 (element is list) query ??? select array_prepend(make_array(1), make_array(make_array(2), make_array(3), make_array(4))), array_prepend(make_array(1.0), make_array([2.0], [3.0], [4.0])), array_prepend(make_array('h'), make_array(['e'], ['l'], ['l'], ['o'])); ---- [[1], [2], [3], [4]] [[1.0], [2.0], [3.0], [4.0]] [[h], [e], [l], [l], [o]] +# TODO: add this when #8305 is fixed +# query ??? +# select array_prepend(arrow_cast(make_array(1), 'LargeList(Int64)'), arrow_cast(make_array(make_array(2), make_array(3), make_array(4)), 'LargeList(LargeList(Int64))')), array_prepend(arrow_cast(make_array(1.0), 'LargeList(Float64)'), arrow_cast(make_array([2.0], [3.0], [4.0]), 'LargeList(LargeList(Float64))')), array_prepend(arrow_cast(make_array('h'), 'LargeList(Utf8)'), arrow_cast(make_array(['e'], ['l'], ['l'], ['o']), 'LargeList(LargeList(Utf8))'')); +# ---- +# [[1], [2], [3], [4]] [[1.0], [2.0], [3.0], [4.0]] [[h], [e], [l], [l], [o]] + # list_prepend scalar function #5 (function alias `array_prepend`) query ??? select list_prepend(1, make_array(2, 3, 4)), list_prepend(1.0, make_array(2.0, 3.0, 4.0)), list_prepend('h', make_array('e', 'l', 'l', 'o')); ---- [1, 2, 3, 4] [1.0, 2.0, 3.0, 4.0] [h, e, l, l, o] +query ??? +select list_prepend(1, arrow_cast(make_array(2, 3, 4), 'LargeList(Int64)')), list_prepend(1.0, arrow_cast(make_array(2.0, 3.0, 4.0), 'LargeList(Float64)')), list_prepend('h', arrow_cast(make_array('e', 'l', 'l', 'o'), 'LargeList(Utf8)')); +---- +[1, 2, 3, 4] [1.0, 2.0, 3.0, 4.0] [h, e, l, l, o] + # array_push_front scalar function #6 (function alias `array_prepend`) query ??? select array_push_front(1, make_array(2, 3, 4)), array_push_front(1.0, make_array(2.0, 3.0, 4.0)), array_push_front('h', make_array('e', 'l', 'l', 'o')); ---- [1, 2, 3, 4] [1.0, 2.0, 3.0, 4.0] [h, e, l, l, o] +query ??? +select array_push_front(1, arrow_cast(make_array(2, 3, 4), 'LargeList(Int64)')), array_push_front(1.0, arrow_cast(make_array(2.0, 3.0, 4.0), 'LargeList(Float64)')), array_push_front('h', arrow_cast(make_array('e', 'l', 'l', 'o'), 'LargeList(Utf8)')); +---- +[1, 2, 3, 4] [1.0, 2.0, 3.0, 4.0] [h, e, l, l, o] + # list_push_front scalar function #7 (function alias `array_prepend`) query ??? select list_push_front(1, make_array(2, 3, 4)), list_push_front(1.0, make_array(2.0, 3.0, 4.0)), list_push_front('h', make_array('e', 'l', 'l', 'o')); ---- [1, 2, 3, 4] [1.0, 2.0, 3.0, 4.0] [h, e, l, l, o] +query ??? +select list_push_front(1, arrow_cast(make_array(2, 3, 4), 'LargeList(Int64)')), list_push_front(1.0, arrow_cast(make_array(2.0, 3.0, 4.0), 'LargeList(Float64)')), list_push_front('h', arrow_cast(make_array('e', 'l', 'l', 'o'), 'LargeList(Utf8)')); +---- +[1, 2, 3, 4] [1.0, 2.0, 3.0, 4.0] [h, e, l, l, o] + # array_prepend with columns #1 query ? select array_prepend(column2, column1) from arrays_values; @@ -1725,6 +1868,18 @@ select array_prepend(column2, column1) from arrays_values; [55, 51, 52, , 54, 55, 56, 57, 58, 59, 60] [66, 61, 62, 63, 64, 65, 66, 67, 68, 69, 70] +query ? +select array_prepend(column2, column1) from large_arrays_values; +---- +[1, , 2, 3, 4, 5, 6, 7, 8, 9, 10] +[12, 11, 12, 13, 14, 15, 16, 17, 18, , 20] +[23, 21, 22, 23, , 25, 26, 27, 28, 29, 30] +[34, 31, 32, 33, 34, 35, , 37, 38, 39, 40] +[44] +[, 41, 42, 43, 44, 45, 46, 47, 48, 49, 50] +[55, 51, 52, , 54, 55, 56, 57, 58, 59, 60] +[66, 61, 62, 63, 64, 65, 66, 67, 68, 69, 70] + # array_prepend with columns #2 (element is list) query ? select array_prepend(column2, column1) from nested_arrays; @@ -1732,6 +1887,13 @@ select array_prepend(column2, column1) from nested_arrays; [[7, 8, 9], [1, 2, 3], [2, 9, 1], [7, 8, 9], [1, 2, 3], [1, 7, 4], [4, 5, 6]] [[10, 11, 12], [4, 5, 6], [10, 11, 12], [4, 9, 8], [7, 8, 9], [10, 11, 12], [1, 8, 7]] +# TODO: add this when #8305 is fixed +# query ? +# select array_prepend(column2, column1) from large_nested_arrays; +# ---- +# [[7, 8, 9], [1, 2, 3], [2, 9, 1], [7, 8, 9], [1, 2, 3], [1, 7, 4], [4, 5, 6]] +# [[10, 11, 12], [4, 5, 6], [10, 11, 12], [4, 9, 8], [7, 8, 9], [10, 11, 12], [1, 8, 7]] + # array_prepend with columns and scalars #1 query ?? select array_prepend(100.1, column2), array_prepend('.', column3) from arrays; @@ -1744,6 +1906,17 @@ select array_prepend(100.1, column2), array_prepend('.', column3) from arrays; [100.1] [., ,] [100.1, 16.6, 17.7, 18.8] [.] +query ?? +select array_prepend(100.1, column2), array_prepend('.', column3) from large_arrays; +---- +[100.1, 1.1, 2.2, 3.3] [., L, o, r, e, m] +[100.1, , 5.5, 6.6] [., i, p, , u, m] +[100.1, 7.7, 8.8, 9.9] [., d, , l, o, r] +[100.1, 10.1, , 12.2] [., s, i, t] +[100.1, 13.3, 14.4, 15.5] [., a, m, e, t] +[100.1] [., ,] +[100.1, 16.6, 17.7, 18.8] [.] + # array_prepend with columns and scalars #2 (element is list) query ?? select array_prepend(make_array(1, 11, 111), column1), array_prepend(column2, make_array(make_array(1, 2, 3), make_array(11, 12, 13))) from nested_arrays; @@ -1751,6 +1924,13 @@ select array_prepend(make_array(1, 11, 111), column1), array_prepend(column2, ma [[1, 11, 111], [1, 2, 3], [2, 9, 1], [7, 8, 9], [1, 2, 3], [1, 7, 4], [4, 5, 6]] [[7, 8, 9], [1, 2, 3], [11, 12, 13]] [[1, 11, 111], [4, 5, 6], [10, 11, 12], [4, 9, 8], [7, 8, 9], [10, 11, 12], [1, 8, 7]] [[10, 11, 12], [1, 2, 3], [11, 12, 13]] +# TODO: add this when #8305 is fixed +# query ?? +# select array_prepend(arrow_cast(make_array(1, 11, 111), 'LargeList(Int64)'), column1), array_prepend(column2, arrow_cast(make_array(make_array(1, 2, 3), make_array(11, 12, 13)), 'LargeList(LargeList(Int64))')) from large_nested_arrays; +# ---- +# [[1, 11, 111], [1, 2, 3], [2, 9, 1], [7, 8, 9], [1, 2, 3], [1, 7, 4], [4, 5, 6]] [[7, 8, 9], [1, 2, 3], [11, 12, 13]] +# [[1, 11, 111], [4, 5, 6], [10, 11, 12], [4, 9, 8], [7, 8, 9], [10, 11, 12], [1, 8, 7]] [[10, 11, 12], [1, 2, 3], [11, 12, 13]] + ## array_repeat (aliases: `list_repeat`) # array_repeat scalar function #1 From 9a6cc889a40e4740bfc859557a9ca9c8d043891e Mon Sep 17 00:00:00 2001 From: Jeffrey <22608443+Jefffrey@users.noreply.github.com> Date: Wed, 3 Jan 2024 10:17:26 +1100 Subject: [PATCH 539/572] Support for `extract(epoch from date)` for Date32 and Date64 (#8695) --- datafusion/core/tests/sql/expr.rs | 34 ++++++++++++++ .../physical-expr/src/datetime_expressions.rs | 44 ++++++++++--------- 2 files changed, 58 insertions(+), 20 deletions(-) diff --git a/datafusion/core/tests/sql/expr.rs b/datafusion/core/tests/sql/expr.rs index 7d41ad4a881c..8ac0e3e5ef19 100644 --- a/datafusion/core/tests/sql/expr.rs +++ b/datafusion/core/tests/sql/expr.rs @@ -741,6 +741,7 @@ async fn test_extract_date_part() -> Result<()> { #[tokio::test] async fn test_extract_epoch() -> Result<()> { + // timestamp test_expression!( "extract(epoch from '1870-01-01T07:29:10.256'::timestamp)", "-3155646649.744" @@ -754,6 +755,39 @@ async fn test_extract_epoch() -> Result<()> { "946684800.0" ); test_expression!("extract(epoch from NULL::timestamp)", "NULL"); + // date + test_expression!( + "extract(epoch from arrow_cast('1970-01-01', 'Date32'))", + "0.0" + ); + test_expression!( + "extract(epoch from arrow_cast('1970-01-02', 'Date32'))", + "86400.0" + ); + test_expression!( + "extract(epoch from arrow_cast('1970-01-11', 'Date32'))", + "864000.0" + ); + test_expression!( + "extract(epoch from arrow_cast('1969-12-31', 'Date32'))", + "-86400.0" + ); + test_expression!( + "extract(epoch from arrow_cast('1970-01-01', 'Date64'))", + "0.0" + ); + test_expression!( + "extract(epoch from arrow_cast('1970-01-02', 'Date64'))", + "86400.0" + ); + test_expression!( + "extract(epoch from arrow_cast('1970-01-11', 'Date64'))", + "864000.0" + ); + test_expression!( + "extract(epoch from arrow_cast('1969-12-31', 'Date64'))", + "-86400.0" + ); Ok(()) } diff --git a/datafusion/physical-expr/src/datetime_expressions.rs b/datafusion/physical-expr/src/datetime_expressions.rs index f6373d40d965..589bbc8a952b 100644 --- a/datafusion/physical-expr/src/datetime_expressions.rs +++ b/datafusion/physical-expr/src/datetime_expressions.rs @@ -19,7 +19,6 @@ use crate::datetime_expressions; use crate::expressions::cast_column; -use arrow::array::Float64Builder; use arrow::compute::cast; use arrow::{ array::{Array, ArrayRef, Float64Array, OffsetSizeTrait, PrimitiveArray}, @@ -887,28 +886,33 @@ where T: ArrowTemporalType + ArrowNumericType, i64: From, { - let mut b = Float64Builder::with_capacity(array.len()); - match array.data_type() { + let b = match array.data_type() { DataType::Timestamp(tu, _) => { - for i in 0..array.len() { - if array.is_null(i) { - b.append_null(); - } else { - let scale = match tu { - TimeUnit::Second => 1, - TimeUnit::Millisecond => 1_000, - TimeUnit::Microsecond => 1_000_000, - TimeUnit::Nanosecond => 1_000_000_000, - }; - - let n: i64 = array.value(i).into(); - b.append_value(n as f64 / scale as f64); - } - } + let scale = match tu { + TimeUnit::Second => 1, + TimeUnit::Millisecond => 1_000, + TimeUnit::Microsecond => 1_000_000, + TimeUnit::Nanosecond => 1_000_000_000, + } as f64; + array.unary(|n| { + let n: i64 = n.into(); + n as f64 / scale + }) } + DataType::Date32 => { + let seconds_in_a_day = 86400_f64; + array.unary(|n| { + let n: i64 = n.into(); + n as f64 * seconds_in_a_day + }) + } + DataType::Date64 => array.unary(|n| { + let n: i64 = n.into(); + n as f64 / 1_000_f64 + }), _ => return internal_err!("Can not convert {:?} to epoch", array.data_type()), - } - Ok(b.finish()) + }; + Ok(b) } /// to_timestammp() SQL function implementation From 6b1e9c6a3ae95b7065e902d99d9fde66f0f8e054 Mon Sep 17 00:00:00 2001 From: junxiangMu <63799833+guojidan@users.noreply.github.com> Date: Wed, 3 Jan 2024 20:24:58 +0800 Subject: [PATCH 540/572] Implement trait based API for defining WindowUDF (#8719) * Implement trait based API for defining WindowUDF * add test case & docs * fix docs * rename WindowUDFImpl function --- datafusion-examples/README.md | 1 + datafusion-examples/examples/advanced_udwf.rs | 230 ++++++++++++++++++ .../user_defined_window_functions.rs | 64 +++-- datafusion/expr/src/expr_fn.rs | 67 ++++- datafusion/expr/src/lib.rs | 2 +- datafusion/expr/src/udwf.rs | 116 ++++++++- .../tests/cases/roundtrip_logical_plan.rs | 55 +++-- docs/source/library-user-guide/adding-udfs.md | 7 +- 8 files changed, 498 insertions(+), 44 deletions(-) create mode 100644 datafusion-examples/examples/advanced_udwf.rs diff --git a/datafusion-examples/README.md b/datafusion-examples/README.md index 1296c74ea277..aae451add9e7 100644 --- a/datafusion-examples/README.md +++ b/datafusion-examples/README.md @@ -63,6 +63,7 @@ cargo run --example csv_sql - [`advanced_udf.rs`](examples/advanced_udf.rs): Define and invoke a more complicated User Defined Scalar Function (UDF) - [`simple_udaf.rs`](examples/simple_udaf.rs): Define and invoke a User Defined Aggregate Function (UDAF) - [`simple_udfw.rs`](examples/simple_udwf.rs): Define and invoke a User Defined Window Function (UDWF) +- [`advanced_udwf.rs`](examples/advanced_udwf.rs): Define and invoke a more complicated User Defined Window Function (UDWF) ## Distributed diff --git a/datafusion-examples/examples/advanced_udwf.rs b/datafusion-examples/examples/advanced_udwf.rs new file mode 100644 index 000000000000..91869d80a41a --- /dev/null +++ b/datafusion-examples/examples/advanced_udwf.rs @@ -0,0 +1,230 @@ +// 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 datafusion::{arrow::datatypes::DataType, logical_expr::Volatility}; +use std::any::Any; + +use arrow::{ + array::{ArrayRef, AsArray, Float64Array}, + datatypes::Float64Type, +}; +use datafusion::error::Result; +use datafusion::prelude::*; +use datafusion_common::ScalarValue; +use datafusion_expr::{ + PartitionEvaluator, Signature, WindowFrame, WindowUDF, WindowUDFImpl, +}; + +/// This example shows how to use the full WindowUDFImpl API to implement a user +/// defined window function. As in the `simple_udwf.rs` example, this struct implements +/// a function `partition_evaluator` that returns the `MyPartitionEvaluator` instance. +/// +/// To do so, we must implement the `WindowUDFImpl` trait. +struct SmoothItUdf { + signature: Signature, +} + +impl SmoothItUdf { + /// Create a new instance of the SmoothItUdf struct + fn new() -> Self { + Self { + signature: Signature::exact( + // this function will always take one arguments of type f64 + vec![DataType::Float64], + // this function is deterministic and will always return the same + // result for the same input + Volatility::Immutable, + ), + } + } +} + +impl WindowUDFImpl for SmoothItUdf { + /// We implement as_any so that we can downcast the WindowUDFImpl trait object + fn as_any(&self) -> &dyn Any { + self + } + + /// Return the name of this function + fn name(&self) -> &str { + "smooth_it" + } + + /// Return the "signature" of this function -- namely that types of arguments it will take + fn signature(&self) -> &Signature { + &self.signature + } + + /// What is the type of value that will be returned by this function. + fn return_type(&self, _arg_types: &[DataType]) -> Result { + Ok(DataType::Float64) + } + + /// Create a `PartitionEvalutor` to evaluate this function on a new + /// partition. + fn partition_evaluator(&self) -> Result> { + Ok(Box::new(MyPartitionEvaluator::new())) + } +} + +/// This implements the lowest level evaluation for a window function +/// +/// It handles calculating the value of the window function for each +/// distinct values of `PARTITION BY` (each car type in our example) +#[derive(Clone, Debug)] +struct MyPartitionEvaluator {} + +impl MyPartitionEvaluator { + fn new() -> Self { + Self {} + } +} + +/// Different evaluation methods are called depending on the various +/// settings of WindowUDF. This example uses the simplest and most +/// general, `evaluate`. See `PartitionEvaluator` for the other more +/// advanced uses. +impl PartitionEvaluator for MyPartitionEvaluator { + /// Tell DataFusion the window function varies based on the value + /// of the window frame. + fn uses_window_frame(&self) -> bool { + true + } + + /// This function is called once per input row. + /// + /// `range`specifies which indexes of `values` should be + /// considered for the calculation. + /// + /// Note this is the SLOWEST, but simplest, way to evaluate a + /// window function. It is much faster to implement + /// evaluate_all or evaluate_all_with_rank, if possible + fn evaluate( + &mut self, + values: &[ArrayRef], + range: &std::ops::Range, + ) -> Result { + // Again, the input argument is an array of floating + // point numbers to calculate a moving average + let arr: &Float64Array = values[0].as_ref().as_primitive::(); + + let range_len = range.end - range.start; + + // our smoothing function will average all the values in the + let output = if range_len > 0 { + let sum: f64 = arr.values().iter().skip(range.start).take(range_len).sum(); + Some(sum / range_len as f64) + } else { + None + }; + + Ok(ScalarValue::Float64(output)) + } +} + +// create local execution context with `cars.csv` registered as a table named `cars` +async fn create_context() -> Result { + // declare a new context. In spark API, this corresponds to a new spark SQL session + let ctx = SessionContext::new(); + + // declare a table in memory. In spark API, this corresponds to createDataFrame(...). + println!("pwd: {}", std::env::current_dir().unwrap().display()); + let csv_path = "../../datafusion/core/tests/data/cars.csv".to_string(); + let read_options = CsvReadOptions::default().has_header(true); + + ctx.register_csv("cars", &csv_path, read_options).await?; + Ok(ctx) +} + +#[tokio::main] +async fn main() -> Result<()> { + let ctx = create_context().await?; + let smooth_it = WindowUDF::from(SmoothItUdf::new()); + ctx.register_udwf(smooth_it.clone()); + + // Use SQL to run the new window function + let df = ctx.sql("SELECT * from cars").await?; + // print the results + df.show().await?; + + // Use SQL to run the new window function: + // + // `PARTITION BY car`:each distinct value of car (red, and green) + // should be treated as a separate partition (and will result in + // creating a new `PartitionEvaluator`) + // + // `ORDER BY time`: within each partition ('green' or 'red') the + // rows will be be ordered by the value in the `time` column + // + // `evaluate_inside_range` is invoked with a window defined by the + // SQL. In this case: + // + // The first invocation will be passed row 0, the first row in the + // partition. + // + // The second invocation will be passed rows 0 and 1, the first + // two rows in the partition. + // + // etc. + let df = ctx + .sql( + "SELECT \ + car, \ + speed, \ + smooth_it(speed) OVER (PARTITION BY car ORDER BY time) AS smooth_speed,\ + time \ + from cars \ + ORDER BY \ + car", + ) + .await?; + // print the results + df.show().await?; + + // this time, call the new widow function with an explicit + // window so evaluate will be invoked with each window. + // + // `ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING`: each invocation + // sees at most 3 rows: the row before, the current row, and the 1 + // row afterward. + let df = ctx.sql( + "SELECT \ + car, \ + speed, \ + smooth_it(speed) OVER (PARTITION BY car ORDER BY time ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING) AS smooth_speed,\ + time \ + from cars \ + ORDER BY \ + car", + ).await?; + // print the results + df.show().await?; + + // Now, run the function using the DataFrame API: + let window_expr = smooth_it.call( + vec![col("speed")], // smooth_it(speed) + vec![col("car")], // PARTITION BY car + vec![col("time").sort(true, true)], // ORDER BY time ASC + WindowFrame::new(false), + ); + let df = ctx.table("cars").await?.window(vec![window_expr])?; + + // print the results + df.show().await?; + + Ok(()) +} diff --git a/datafusion/core/tests/user_defined/user_defined_window_functions.rs b/datafusion/core/tests/user_defined/user_defined_window_functions.rs index 5f9939157217..3040fbafe81a 100644 --- a/datafusion/core/tests/user_defined/user_defined_window_functions.rs +++ b/datafusion/core/tests/user_defined/user_defined_window_functions.rs @@ -19,6 +19,7 @@ //! user defined window functions use std::{ + any::Any, ops::Range, sync::{ atomic::{AtomicUsize, Ordering}, @@ -32,8 +33,7 @@ use arrow_schema::DataType; use datafusion::{assert_batches_eq, prelude::SessionContext}; use datafusion_common::{Result, ScalarValue}; use datafusion_expr::{ - function::PartitionEvaluatorFactory, PartitionEvaluator, ReturnTypeFunction, - Signature, Volatility, WindowUDF, + PartitionEvaluator, Signature, Volatility, WindowUDF, WindowUDFImpl, }; /// A query with a window function evaluated over the entire partition @@ -471,24 +471,48 @@ impl OddCounter { } fn register(ctx: &mut SessionContext, test_state: Arc) { - let name = "odd_counter"; - let volatility = Volatility::Immutable; - - let signature = Signature::exact(vec![DataType::Int64], volatility); - - let return_type = Arc::new(DataType::Int64); - let return_type: ReturnTypeFunction = - Arc::new(move |_| Ok(Arc::clone(&return_type))); - - let partition_evaluator_factory: PartitionEvaluatorFactory = - Arc::new(move || Ok(Box::new(OddCounter::new(Arc::clone(&test_state))))); - - ctx.register_udwf(WindowUDF::new( - name, - &signature, - &return_type, - &partition_evaluator_factory, - )) + struct SimpleWindowUDF { + signature: Signature, + return_type: DataType, + test_state: Arc, + } + + impl SimpleWindowUDF { + fn new(test_state: Arc) -> Self { + let signature = + Signature::exact(vec![DataType::Float64], Volatility::Immutable); + let return_type = DataType::Int64; + Self { + signature, + return_type, + test_state, + } + } + } + + impl WindowUDFImpl for SimpleWindowUDF { + fn as_any(&self) -> &dyn Any { + self + } + + fn name(&self) -> &str { + "odd_counter" + } + + fn signature(&self) -> &Signature { + &self.signature + } + + fn return_type(&self, _arg_types: &[DataType]) -> Result { + Ok(self.return_type.clone()) + } + + fn partition_evaluator(&self) -> Result> { + Ok(Box::new(OddCounter::new(Arc::clone(&self.test_state)))) + } + } + + ctx.register_udwf(WindowUDF::from(SimpleWindowUDF::new(test_state))) } } diff --git a/datafusion/expr/src/expr_fn.rs b/datafusion/expr/src/expr_fn.rs index eed41d97ccba..f76fb17b38bb 100644 --- a/datafusion/expr/src/expr_fn.rs +++ b/datafusion/expr/src/expr_fn.rs @@ -28,7 +28,7 @@ use crate::{ BuiltinScalarFunction, Expr, LogicalPlan, Operator, ReturnTypeFunction, ScalarFunctionImplementation, ScalarUDF, Signature, StateTypeFunction, Volatility, }; -use crate::{ColumnarValue, ScalarUDFImpl, WindowUDF}; +use crate::{ColumnarValue, ScalarUDFImpl, WindowUDF, WindowUDFImpl}; use arrow::datatypes::DataType; use datafusion_common::{Column, Result}; use std::any::Any; @@ -1059,13 +1059,66 @@ pub fn create_udwf( volatility: Volatility, partition_evaluator_factory: PartitionEvaluatorFactory, ) -> WindowUDF { - let return_type: ReturnTypeFunction = Arc::new(move |_| Ok(return_type.clone())); - WindowUDF::new( + let return_type = Arc::try_unwrap(return_type).unwrap_or_else(|t| t.as_ref().clone()); + WindowUDF::from(SimpleWindowUDF::new( name, - &Signature::exact(vec![input_type], volatility), - &return_type, - &partition_evaluator_factory, - ) + input_type, + return_type, + volatility, + partition_evaluator_factory, + )) +} + +/// Implements [`WindowUDFImpl`] for functions that have a single signature and +/// return type. +pub struct SimpleWindowUDF { + name: String, + signature: Signature, + return_type: DataType, + partition_evaluator_factory: PartitionEvaluatorFactory, +} + +impl SimpleWindowUDF { + /// Create a new `SimpleWindowUDF` from a name, input types, return type and + /// implementation. Implementing [`WindowUDFImpl`] allows more flexibility + pub fn new( + name: impl Into, + input_type: DataType, + return_type: DataType, + volatility: Volatility, + partition_evaluator_factory: PartitionEvaluatorFactory, + ) -> Self { + let name = name.into(); + let signature = Signature::exact([input_type].to_vec(), volatility); + Self { + name, + signature, + return_type, + partition_evaluator_factory, + } + } +} + +impl WindowUDFImpl for SimpleWindowUDF { + fn as_any(&self) -> &dyn Any { + self + } + + fn name(&self) -> &str { + &self.name + } + + fn signature(&self) -> &Signature { + &self.signature + } + + fn return_type(&self, _arg_types: &[DataType]) -> Result { + Ok(self.return_type.clone()) + } + + fn partition_evaluator(&self) -> Result> { + (self.partition_evaluator_factory)() + } } /// Calls a named built in function diff --git a/datafusion/expr/src/lib.rs b/datafusion/expr/src/lib.rs index ab213a19a352..077681d21725 100644 --- a/datafusion/expr/src/lib.rs +++ b/datafusion/expr/src/lib.rs @@ -82,7 +82,7 @@ pub use signature::{ pub use table_source::{TableProviderFilterPushDown, TableSource, TableType}; pub use udaf::AggregateUDF; pub use udf::{ScalarUDF, ScalarUDFImpl}; -pub use udwf::WindowUDF; +pub use udwf::{WindowUDF, WindowUDFImpl}; pub use window_frame::{WindowFrame, WindowFrameBound, WindowFrameUnits}; #[cfg(test)] diff --git a/datafusion/expr/src/udwf.rs b/datafusion/expr/src/udwf.rs index a97a68341f5c..800386bfc77b 100644 --- a/datafusion/expr/src/udwf.rs +++ b/datafusion/expr/src/udwf.rs @@ -24,6 +24,7 @@ use crate::{ use arrow::datatypes::DataType; use datafusion_common::Result; use std::{ + any::Any, fmt::{self, Debug, Display, Formatter}, sync::Arc, }; @@ -80,7 +81,11 @@ impl std::hash::Hash for WindowUDF { } impl WindowUDF { - /// Create a new WindowUDF + /// Create a new WindowUDF from low level details. + /// + /// See [`WindowUDFImpl`] for a more convenient way to create a + /// `WindowUDF` using trait objects + #[deprecated(since = "34.0.0", note = "please implement ScalarUDFImpl instead")] pub fn new( name: &str, signature: &Signature, @@ -95,6 +100,32 @@ impl WindowUDF { } } + /// Create a new `WindowUDF` from a `[WindowUDFImpl]` trait object + /// + /// Note this is the same as using the `From` impl (`WindowUDF::from`) + pub fn new_from_impl(fun: F) -> WindowUDF + where + F: WindowUDFImpl + Send + Sync + 'static, + { + let arc_fun = Arc::new(fun); + let captured_self = arc_fun.clone(); + let return_type: ReturnTypeFunction = Arc::new(move |arg_types| { + let return_type = captured_self.return_type(arg_types)?; + Ok(Arc::new(return_type)) + }); + + let captured_self = arc_fun.clone(); + let partition_evaluator_factory: PartitionEvaluatorFactory = + Arc::new(move || captured_self.partition_evaluator()); + + Self { + name: arc_fun.name().to_string(), + signature: arc_fun.signature().clone(), + return_type: return_type.clone(), + partition_evaluator_factory, + } + } + /// creates a [`Expr`] that calls the window function given /// the `partition_by`, `order_by`, and `window_frame` definition /// @@ -140,3 +171,86 @@ impl WindowUDF { (self.partition_evaluator_factory)() } } + +impl From for WindowUDF +where + F: WindowUDFImpl + Send + Sync + 'static, +{ + fn from(fun: F) -> Self { + Self::new_from_impl(fun) + } +} + +/// Trait for implementing [`WindowUDF`]. +/// +/// This trait exposes the full API for implementing user defined window functions and +/// can be used to implement any function. +/// +/// See [`advanced_udwf.rs`] for a full example with complete implementation and +/// [`WindowUDF`] for other available options. +/// +/// +/// [`advanced_udwf.rs`]: https://github.com/apache/arrow-datafusion/blob/main/datafusion-examples/examples/advanced_udwf.rs +/// # Basic Example +/// ``` +/// # use std::any::Any; +/// # use arrow::datatypes::DataType; +/// # use datafusion_common::{DataFusionError, plan_err, Result}; +/// # use datafusion_expr::{col, Signature, Volatility, PartitionEvaluator, WindowFrame}; +/// # use datafusion_expr::{WindowUDFImpl, WindowUDF}; +/// struct SmoothIt { +/// signature: Signature +/// }; +/// +/// impl SmoothIt { +/// fn new() -> Self { +/// Self { +/// signature: Signature::uniform(1, vec![DataType::Int32], Volatility::Immutable) +/// } +/// } +/// } +/// +/// /// Implement the WindowUDFImpl trait for AddOne +/// impl WindowUDFImpl for SmoothIt { +/// fn as_any(&self) -> &dyn Any { self } +/// fn name(&self) -> &str { "smooth_it" } +/// fn signature(&self) -> &Signature { &self.signature } +/// fn return_type(&self, args: &[DataType]) -> Result { +/// if !matches!(args.get(0), Some(&DataType::Int32)) { +/// return plan_err!("smooth_it only accepts Int32 arguments"); +/// } +/// Ok(DataType::Int32) +/// } +/// // The actual implementation would add one to the argument +/// fn partition_evaluator(&self) -> Result> { unimplemented!() } +/// } +/// +/// // Create a new ScalarUDF from the implementation +/// let smooth_it = WindowUDF::from(SmoothIt::new()); +/// +/// // Call the function `add_one(col)` +/// let expr = smooth_it.call( +/// vec![col("speed")], // smooth_it(speed) +/// vec![col("car")], // PARTITION BY car +/// vec![col("time").sort(true, true)], // ORDER BY time ASC +/// WindowFrame::new(false), +/// ); +/// ``` +pub trait WindowUDFImpl { + /// Returns this object as an [`Any`] trait object + fn as_any(&self) -> &dyn Any; + + /// Returns this function's name + fn name(&self) -> &str; + + /// Returns the function's [`Signature`] for information about what input + /// types are accepted and the function's Volatility. + fn signature(&self) -> &Signature; + + /// What [`DataType`] will be returned by this function, given the types of + /// the arguments + fn return_type(&self, arg_types: &[DataType]) -> Result; + + /// Invoke the function, returning the [`PartitionEvaluator`] instance + fn partition_evaluator(&self) -> Result>; +} diff --git a/datafusion/proto/tests/cases/roundtrip_logical_plan.rs b/datafusion/proto/tests/cases/roundtrip_logical_plan.rs index dea99f91e392..402781e17e6f 100644 --- a/datafusion/proto/tests/cases/roundtrip_logical_plan.rs +++ b/datafusion/proto/tests/cases/roundtrip_logical_plan.rs @@ -15,6 +15,7 @@ // specific language governing permissions and limitations // under the License. +use std::any::Any; use std::collections::HashMap; use std::fmt::{self, Debug, Formatter}; use std::sync::Arc; @@ -54,6 +55,7 @@ use datafusion_expr::{ BuiltinScalarFunction::{Sqrt, Substr}, Expr, LogicalPlan, Operator, PartitionEvaluator, Signature, TryCast, Volatility, WindowFrame, WindowFrameBound, WindowFrameUnits, WindowFunctionDefinition, WindowUDF, + WindowUDFImpl, }; use datafusion_proto::bytes::{ logical_plan_from_bytes, logical_plan_from_bytes_with_extension_codec, @@ -1785,27 +1787,52 @@ fn roundtrip_window() { } } - fn return_type(arg_types: &[DataType]) -> Result> { - if arg_types.len() != 1 { - return plan_err!( - "dummy_udwf expects 1 argument, got {}: {:?}", - arg_types.len(), - arg_types - ); + struct SimpleWindowUDF { + signature: Signature, + } + + impl SimpleWindowUDF { + fn new() -> Self { + let signature = + Signature::exact(vec![DataType::Float64], Volatility::Immutable); + Self { signature } + } + } + + impl WindowUDFImpl for SimpleWindowUDF { + fn as_any(&self) -> &dyn Any { + self + } + + fn name(&self) -> &str { + "dummy_udwf" + } + + fn signature(&self) -> &Signature { + &self.signature + } + + fn return_type(&self, arg_types: &[DataType]) -> Result { + if arg_types.len() != 1 { + return plan_err!( + "dummy_udwf expects 1 argument, got {}: {:?}", + arg_types.len(), + arg_types + ); + } + Ok(arg_types[0].clone()) + } + + fn partition_evaluator(&self) -> Result> { + make_partition_evaluator() } - Ok(Arc::new(arg_types[0].clone())) } fn make_partition_evaluator() -> Result> { Ok(Box::new(DummyWindow {})) } - let dummy_window_udf = WindowUDF::new( - "dummy_udwf", - &Signature::exact(vec![DataType::Float64], Volatility::Immutable), - &(Arc::new(return_type) as _), - &(Arc::new(make_partition_evaluator) as _), - ); + let dummy_window_udf = WindowUDF::from(SimpleWindowUDF::new()); let test_expr6 = Expr::WindowFunction(expr::WindowFunction::new( WindowFunctionDefinition::WindowUDF(Arc::new(dummy_window_udf.clone())), diff --git a/docs/source/library-user-guide/adding-udfs.md b/docs/source/library-user-guide/adding-udfs.md index c51e4de3236c..1f687f978f30 100644 --- a/docs/source/library-user-guide/adding-udfs.md +++ b/docs/source/library-user-guide/adding-udfs.md @@ -201,7 +201,8 @@ fn make_partition_evaluator() -> Result> { ### Registering a Window UDF -To register a Window UDF, you need to wrap the function implementation in a `WindowUDF` struct and then register it with the `SessionContext`. DataFusion provides the `create_udwf` helper functions to make this easier. +To register a Window UDF, you need to wrap the function implementation in a [`WindowUDF`] struct and then register it with the `SessionContext`. DataFusion provides the [`create_udwf`] helper functions to make this easier. +There is a lower level API with more functionality but is more complex, that is documented in [`advanced_udwf.rs`]. ```rust use datafusion::logical_expr::{Volatility, create_udwf}; @@ -218,6 +219,10 @@ let smooth_it = create_udwf( ); ``` +[`windowudf`]: https://docs.rs/datafusion/latest/datafusion/logical_expr/struct.WindowUDF.html +[`create_udwf`]: https://docs.rs/datafusion/latest/datafusion/logical_expr/fn.create_udwf.html +[`advanced_udwf.rs`]: https://github.com/apache/arrow-datafusion/blob/main/datafusion-examples/examples/advanced_udwf.rs + The `create_udwf` has five arguments to check: - The first argument is the name of the function. This is the name that will be used in SQL queries. From 1179a76567892b259c88f08243ee01f05c4c3d5c Mon Sep 17 00:00:00 2001 From: Jay Zhan Date: Thu, 4 Jan 2024 01:50:46 +0800 Subject: [PATCH 541/572] Minor: Introduce utils::hash for StructArray (#8552) * hash struct Signed-off-by: jayzhan211 * fmt Signed-off-by: jayzhan211 * row-wise hash Signed-off-by: jayzhan211 * fmt Signed-off-by: jayzhan211 * create hashes once Signed-off-by: jayzhan211 * cleanup Signed-off-by: jayzhan211 * add comment Signed-off-by: jayzhan211 --------- Signed-off-by: jayzhan211 --- datafusion/common/src/hash_utils.rs | 92 ++++++++++++++++++++++++++++- 1 file changed, 89 insertions(+), 3 deletions(-) diff --git a/datafusion/common/src/hash_utils.rs b/datafusion/common/src/hash_utils.rs index 9198461e00bf..5c36f41a6e42 100644 --- a/datafusion/common/src/hash_utils.rs +++ b/datafusion/common/src/hash_utils.rs @@ -27,7 +27,8 @@ use arrow::{downcast_dictionary_array, downcast_primitive_array}; use arrow_buffer::i256; use crate::cast::{ - as_boolean_array, as_generic_binary_array, as_primitive_array, as_string_array, + as_boolean_array, as_generic_binary_array, as_large_list_array, as_list_array, + as_primitive_array, as_string_array, as_struct_array, }; use crate::error::{DataFusionError, Result, _internal_err}; @@ -207,6 +208,35 @@ fn hash_dictionary( Ok(()) } +fn hash_struct_array( + array: &StructArray, + random_state: &RandomState, + hashes_buffer: &mut [u64], +) -> Result<()> { + let nulls = array.nulls(); + let num_columns = array.num_columns(); + + // Skip null columns + let valid_indices: Vec = if let Some(nulls) = nulls { + nulls.valid_indices().collect() + } else { + (0..num_columns).collect() + }; + + // Create hashes for each row that combines the hashes over all the column at that row. + // array.len() is the number of rows. + let mut values_hashes = vec![0u64; array.len()]; + create_hashes(array.columns(), random_state, &mut values_hashes)?; + + // Skip the null columns, nulls should get hash value 0. + for i in valid_indices { + let hash = &mut hashes_buffer[i]; + *hash = combine_hashes(*hash, values_hashes[i]); + } + + Ok(()) +} + fn hash_list_array( array: &GenericListArray, random_state: &RandomState, @@ -327,12 +357,16 @@ pub fn create_hashes<'a>( array => hash_dictionary(array, random_state, hashes_buffer, rehash)?, _ => unreachable!() } + DataType::Struct(_) => { + let array = as_struct_array(array)?; + hash_struct_array(array, random_state, hashes_buffer)?; + } DataType::List(_) => { - let array = as_list_array(array); + let array = as_list_array(array)?; hash_list_array(array, random_state, hashes_buffer)?; } DataType::LargeList(_) => { - let array = as_large_list_array(array); + let array = as_large_list_array(array)?; hash_list_array(array, random_state, hashes_buffer)?; } _ => { @@ -515,6 +549,58 @@ mod tests { assert_eq!(hashes[2], hashes[3]); } + #[test] + // Tests actual values of hashes, which are different if forcing collisions + #[cfg(not(feature = "force_hash_collisions"))] + fn create_hashes_for_struct_arrays() { + use arrow_buffer::Buffer; + + let boolarr = Arc::new(BooleanArray::from(vec![ + false, false, true, true, true, true, + ])); + let i32arr = Arc::new(Int32Array::from(vec![10, 10, 20, 20, 30, 31])); + + let struct_array = StructArray::from(( + vec![ + ( + Arc::new(Field::new("bool", DataType::Boolean, false)), + boolarr.clone() as ArrayRef, + ), + ( + Arc::new(Field::new("i32", DataType::Int32, false)), + i32arr.clone() as ArrayRef, + ), + ( + Arc::new(Field::new("i32", DataType::Int32, false)), + i32arr.clone() as ArrayRef, + ), + ( + Arc::new(Field::new("bool", DataType::Boolean, false)), + boolarr.clone() as ArrayRef, + ), + ], + Buffer::from(&[0b001011]), + )); + + assert!(struct_array.is_valid(0)); + assert!(struct_array.is_valid(1)); + assert!(struct_array.is_null(2)); + assert!(struct_array.is_valid(3)); + assert!(struct_array.is_null(4)); + assert!(struct_array.is_null(5)); + + let array = Arc::new(struct_array) as ArrayRef; + + let random_state = RandomState::with_seeds(0, 0, 0, 0); + let mut hashes = vec![0; array.len()]; + create_hashes(&[array], &random_state, &mut hashes).unwrap(); + assert_eq!(hashes[0], hashes[1]); + // same value but the third row ( hashes[2] ) is null + assert_ne!(hashes[2], hashes[3]); + // different values but both are null + assert_eq!(hashes[4], hashes[5]); + } + #[test] // Tests actual values of hashes, which are different if forcing collisions #[cfg(not(feature = "force_hash_collisions"))] From 93da699c0e9a6d60c075c252dcf537112b06996a Mon Sep 17 00:00:00 2001 From: comphead Date: Wed, 3 Jan 2024 13:58:11 -0800 Subject: [PATCH 542/572] [CI] Improve windows machine CI test time (#8730) * Test WIN64 CI * Test WIN64 CI * Test WIN64 CI * Test WIN64 CI * Adding incremental compilation * Adding codegen units * Try without opt-level * set opt level only for win machines * set opt level only for win machines. remove incremental compile * update comments --- .github/workflows/rust.yml | 24 +++++++++++++++++++++--- 1 file changed, 21 insertions(+), 3 deletions(-) diff --git a/.github/workflows/rust.yml b/.github/workflows/rust.yml index a541091e3a2b..622521a6fbc7 100644 --- a/.github/workflows/rust.yml +++ b/.github/workflows/rust.yml @@ -99,6 +99,14 @@ jobs: rust-version: stable - name: Run tests (excluding doctests) run: cargo test --lib --tests --bins --features avro,json,backtrace + env: + # do not produce debug symbols to keep memory usage down + # hardcoding other profile params to avoid profile override values + # More on Cargo profiles https://doc.rust-lang.org/cargo/reference/profiles.html?profile-settings#profile-settings + RUSTFLAGS: "-C debuginfo=0 -C opt-level=0 -C incremental=false -C codegen-units=256" + RUST_BACKTRACE: "1" + # avoid rust stack overflows on tpc-ds tests + RUST_MINSTACK: "3000000" - name: Verify Working Directory Clean run: git diff --exit-code @@ -290,6 +298,7 @@ jobs: # with a OS-dependent path. - name: Setup Rust toolchain run: | + rustup update stable rustup toolchain install stable rustup default stable rustup component add rustfmt @@ -302,9 +311,13 @@ jobs: cargo test --lib --tests --bins --all-features env: # do not produce debug symbols to keep memory usage down - RUSTFLAGS: "-C debuginfo=0" + # use higher optimization level to overcome Windows rust slowness for tpc-ds + # and speed builds: https://github.com/apache/arrow-datafusion/issues/8696 + # Cargo profile docs https://doc.rust-lang.org/cargo/reference/profiles.html?profile-settings#profile-settings + RUSTFLAGS: "-C debuginfo=0 -C opt-level=1 -C target-feature=+crt-static -C incremental=false -C codegen-units=256" RUST_BACKTRACE: "1" - + # avoid rust stack overflows on tpc-ds tests + RUST_MINSTACK: "3000000" macos: name: cargo test (mac) runs-on: macos-latest @@ -327,6 +340,7 @@ jobs: # with a OS-dependent path. - name: Setup Rust toolchain run: | + rustup update stable rustup toolchain install stable rustup default stable rustup component add rustfmt @@ -338,8 +352,12 @@ jobs: cargo test --lib --tests --bins --all-features env: # do not produce debug symbols to keep memory usage down - RUSTFLAGS: "-C debuginfo=0" + # hardcoding other profile params to avoid profile override values + # More on Cargo profiles https://doc.rust-lang.org/cargo/reference/profiles.html?profile-settings#profile-settings + RUSTFLAGS: "-C debuginfo=0 -C opt-level=0 -C incremental=false -C codegen-units=256" RUST_BACKTRACE: "1" + # avoid rust stack overflows on tpc-ds tests + RUST_MINSTACK: "3000000" test-datafusion-pyarrow: name: cargo test pyarrow (amd64) From ad4b7b7cfd4a2f93bbef3c2bff8a6ce65db24b53 Mon Sep 17 00:00:00 2001 From: yi wang <48236141+my-vegetable-has-exploded@users.noreply.github.com> Date: Thu, 4 Jan 2024 06:01:13 +0800 Subject: [PATCH 543/572] fix guarantees in allways_true of PruningPredicate (#8732) * fix: check guarantees in allways_true * Add test for allways_true * refine comment --------- Co-authored-by: Andrew Lamb --- .../datasource/physical_plan/parquet/mod.rs | 78 ++++++++++++------- .../core/src/physical_optimizer/pruning.rs | 7 +- 2 files changed, 53 insertions(+), 32 deletions(-) diff --git a/datafusion/core/src/datasource/physical_plan/parquet/mod.rs b/datafusion/core/src/datasource/physical_plan/parquet/mod.rs index 76a6cc297b0e..9d81d8d083c2 100644 --- a/datafusion/core/src/datasource/physical_plan/parquet/mod.rs +++ b/datafusion/core/src/datasource/physical_plan/parquet/mod.rs @@ -1768,8 +1768,9 @@ mod tests { ); } - #[tokio::test] - async fn parquet_exec_metrics() { + /// Returns a string array with contents: + /// "[Foo, null, bar, bar, bar, bar, zzz]" + fn string_batch() -> RecordBatch { let c1: ArrayRef = Arc::new(StringArray::from(vec![ Some("Foo"), None, @@ -1781,9 +1782,15 @@ mod tests { ])); // batch1: c1(string) - let batch1 = create_batch(vec![("c1", c1.clone())]); + create_batch(vec![("c1", c1.clone())]) + } + + #[tokio::test] + async fn parquet_exec_metrics() { + // batch1: c1(string) + let batch1 = string_batch(); - // on + // c1 != 'bar' let filter = col("c1").not_eq(lit("bar")); // read/write them files: @@ -1812,20 +1819,10 @@ mod tests { #[tokio::test] async fn parquet_exec_display() { - let c1: ArrayRef = Arc::new(StringArray::from(vec![ - Some("Foo"), - None, - Some("bar"), - Some("bar"), - Some("bar"), - Some("bar"), - Some("zzz"), - ])); - // batch1: c1(string) - let batch1 = create_batch(vec![("c1", c1.clone())]); + let batch1 = string_batch(); - // on + // c1 != 'bar' let filter = col("c1").not_eq(lit("bar")); let rt = RoundTrip::new() @@ -1854,21 +1851,15 @@ mod tests { } #[tokio::test] - async fn parquet_exec_skip_empty_pruning() { - let c1: ArrayRef = Arc::new(StringArray::from(vec![ - Some("Foo"), - None, - Some("bar"), - Some("bar"), - Some("bar"), - Some("bar"), - Some("zzz"), - ])); - + async fn parquet_exec_has_no_pruning_predicate_if_can_not_prune() { // batch1: c1(string) - let batch1 = create_batch(vec![("c1", c1.clone())]); + let batch1 = string_batch(); - // filter is too complicated for pruning + // filter is too complicated for pruning (PruningPredicate code does not + // handle case expressions), so the pruning predicate will always be + // "true" + + // WHEN c1 != bar THEN true ELSE false END let filter = when(col("c1").not_eq(lit("bar")), lit(true)) .otherwise(lit(false)) .unwrap(); @@ -1879,7 +1870,7 @@ mod tests { .round_trip(vec![batch1]) .await; - // Should not contain a pruning predicate + // Should not contain a pruning predicate (since nothing can be pruned) let pruning_predicate = &rt.parquet_exec.pruning_predicate; assert!( pruning_predicate.is_none(), @@ -1892,6 +1883,33 @@ mod tests { assert_eq!(predicate.unwrap().to_string(), filter_phys.to_string()); } + #[tokio::test] + async fn parquet_exec_has_pruning_predicate_for_guarantees() { + // batch1: c1(string) + let batch1 = string_batch(); + + // part of the filter is too complicated for pruning (PruningPredicate code does not + // handle case expressions), but part (c1 = 'foo') can be used for bloom filtering, so + // should still have the pruning predicate. + + // c1 = 'foo' AND (WHEN c1 != bar THEN true ELSE false END) + let filter = col("c1").eq(lit("foo")).and( + when(col("c1").not_eq(lit("bar")), lit(true)) + .otherwise(lit(false)) + .unwrap(), + ); + + let rt = RoundTrip::new() + .with_predicate(filter.clone()) + .with_pushdown_predicate() + .round_trip(vec![batch1]) + .await; + + // Should have a pruning predicate + let pruning_predicate = &rt.parquet_exec.pruning_predicate; + assert!(pruning_predicate.is_some()); + } + /// returns the sum of all the metrics with the specified name /// the returned set. /// diff --git a/datafusion/core/src/physical_optimizer/pruning.rs b/datafusion/core/src/physical_optimizer/pruning.rs index fecbffdbb041..06cfc7282468 100644 --- a/datafusion/core/src/physical_optimizer/pruning.rs +++ b/datafusion/core/src/physical_optimizer/pruning.rs @@ -295,9 +295,12 @@ impl PruningPredicate { &self.predicate_expr } - /// Returns true if this pruning predicate is "always true" (aka will not prune anything) + /// Returns true if this pruning predicate can not prune anything. + /// + /// This happens if the predicate is a literal `true` and + /// literal_guarantees is empty. pub fn allways_true(&self) -> bool { - is_always_true(&self.predicate_expr) + is_always_true(&self.predicate_expr) && self.literal_guarantees.is_empty() } pub(crate) fn required_columns(&self) -> &RequiredColumns { From 881d03f72cddec7e1cd659ef0c748760c6177b1c Mon Sep 17 00:00:00 2001 From: Yang Jiang Date: Thu, 4 Jan 2024 06:02:38 +0800 Subject: [PATCH 544/572] [Minor] Avoid mem copy in generate window exprs (#8718) --- datafusion/expr/src/logical_plan/builder.rs | 4 +-- datafusion/expr/src/utils.rs | 30 ++++++++++----------- 2 files changed, 17 insertions(+), 17 deletions(-) diff --git a/datafusion/expr/src/logical_plan/builder.rs b/datafusion/expr/src/logical_plan/builder.rs index cfc052cfc14c..a684f3e97485 100644 --- a/datafusion/expr/src/logical_plan/builder.rs +++ b/datafusion/expr/src/logical_plan/builder.rs @@ -292,7 +292,7 @@ impl LogicalPlanBuilder { window_exprs: Vec, ) -> Result { let mut plan = input; - let mut groups = group_window_expr_by_sort_keys(&window_exprs)?; + let mut groups = group_window_expr_by_sort_keys(window_exprs)?; // To align with the behavior of PostgreSQL, we want the sort_keys sorted as same rule as PostgreSQL that first // we compare the sort key themselves and if one window's sort keys are a prefix of another // put the window with more sort keys first. so more deeply sorted plans gets nested further down as children. @@ -314,7 +314,7 @@ impl LogicalPlanBuilder { key_b.len().cmp(&key_a.len()) }); for (_, exprs) in groups { - let window_exprs = exprs.into_iter().cloned().collect::>(); + let window_exprs = exprs.into_iter().collect::>(); // Partition and sorting is done at physical level, see the EnforceDistribution // and EnforceSorting rules. plan = LogicalPlanBuilder::from(plan) diff --git a/datafusion/expr/src/utils.rs b/datafusion/expr/src/utils.rs index e3ecdf154e61..914b354d2950 100644 --- a/datafusion/expr/src/utils.rs +++ b/datafusion/expr/src/utils.rs @@ -575,14 +575,14 @@ pub fn compare_sort_expr( /// group a slice of window expression expr by their order by expressions pub fn group_window_expr_by_sort_keys( - window_expr: &[Expr], -) -> Result)>> { + window_expr: Vec, +) -> Result)>> { let mut result = vec![]; - window_expr.iter().try_for_each(|expr| match expr { - Expr::WindowFunction(WindowFunction{ partition_by, order_by, .. }) => { + window_expr.into_iter().try_for_each(|expr| match &expr { + Expr::WindowFunction( WindowFunction{ partition_by, order_by, .. }) => { let sort_key = generate_sort_key(partition_by, order_by)?; if let Some((_, values)) = result.iter_mut().find( - |group: &&mut (WindowSortKey, Vec<&Expr>)| matches!(group, (key, _) if *key == sort_key), + |group: &&mut (WindowSortKey, Vec)| matches!(group, (key, _) if *key == sort_key), ) { values.push(expr); } else { @@ -1239,8 +1239,8 @@ mod tests { #[test] fn test_group_window_expr_by_sort_keys_empty_case() -> Result<()> { - let result = group_window_expr_by_sort_keys(&[])?; - let expected: Vec<(WindowSortKey, Vec<&Expr>)> = vec![]; + let result = group_window_expr_by_sort_keys(vec![])?; + let expected: Vec<(WindowSortKey, Vec)> = vec![]; assert_eq!(expected, result); Ok(()) } @@ -1276,10 +1276,10 @@ mod tests { WindowFrame::new(false), )); let exprs = &[max1.clone(), max2.clone(), min3.clone(), sum4.clone()]; - let result = group_window_expr_by_sort_keys(exprs)?; + let result = group_window_expr_by_sort_keys(exprs.to_vec())?; let key = vec![]; - let expected: Vec<(WindowSortKey, Vec<&Expr>)> = - vec![(key, vec![&max1, &max2, &min3, &sum4])]; + let expected: Vec<(WindowSortKey, Vec)> = + vec![(key, vec![max1, max2, min3, sum4])]; assert_eq!(expected, result); Ok(()) } @@ -1320,7 +1320,7 @@ mod tests { )); // FIXME use as_ref let exprs = &[max1.clone(), max2.clone(), min3.clone(), sum4.clone()]; - let result = group_window_expr_by_sort_keys(exprs)?; + let result = group_window_expr_by_sort_keys(exprs.to_vec())?; let key1 = vec![(age_asc.clone(), false), (name_desc.clone(), false)]; let key2 = vec![]; @@ -1330,10 +1330,10 @@ mod tests { (created_at_desc, false), ]; - let expected: Vec<(WindowSortKey, Vec<&Expr>)> = vec![ - (key1, vec![&max1, &min3]), - (key2, vec![&max2]), - (key3, vec![&sum4]), + let expected: Vec<(WindowSortKey, Vec)> = vec![ + (key1, vec![max1, min3]), + (key2, vec![max2]), + (key3, vec![sum4]), ]; assert_eq!(expected, result); Ok(()) From ca260d99f17ef667b7f06d2da4a67255d27c94a9 Mon Sep 17 00:00:00 2001 From: Alex Huang Date: Thu, 4 Jan 2024 06:14:23 +0800 Subject: [PATCH 545/572] support LargeList in array_repeat (#8725) --- .../physical-expr/src/array_expressions.rs | 16 +++++--- datafusion/sqllogictest/test_files/array.slt | 37 +++++++++++++++++++ 2 files changed, 47 insertions(+), 6 deletions(-) diff --git a/datafusion/physical-expr/src/array_expressions.rs b/datafusion/physical-expr/src/array_expressions.rs index aad021610fcb..15330af640ae 100644 --- a/datafusion/physical-expr/src/array_expressions.rs +++ b/datafusion/physical-expr/src/array_expressions.rs @@ -1233,7 +1233,11 @@ pub fn array_repeat(args: &[ArrayRef]) -> Result { match element.data_type() { DataType::List(_) => { let list_array = as_list_array(element)?; - general_list_repeat(list_array, count_array) + general_list_repeat::(list_array, count_array) + } + DataType::LargeList(_) => { + let list_array = as_large_list_array(element)?; + general_list_repeat::(list_array, count_array) } _ => general_repeat(element, count_array), } @@ -1302,8 +1306,8 @@ fn general_repeat(array: &ArrayRef, count_array: &Int64Array) -> Result [[[1, 2, 3], [1, 2, 3]], [], [[6]]] /// ) /// ``` -fn general_list_repeat( - list_array: &ListArray, +fn general_list_repeat( + list_array: &GenericListArray, count_array: &Int64Array, ) -> Result { let data_type = list_array.data_type(); @@ -1335,9 +1339,9 @@ fn general_list_repeat( let data = mutable.freeze(); let repeated_array = arrow_array::make_array(data); - let list_arr = ListArray::try_new( + let list_arr = GenericListArray::::try_new( Arc::new(Field::new("item", value_type.clone(), true)), - OffsetBuffer::from_lengths(vec![original_data.len(); count]), + OffsetBuffer::::from_lengths(vec![original_data.len(); count]), repeated_array, None, )?; @@ -1354,7 +1358,7 @@ fn general_list_repeat( Ok(Arc::new(ListArray::try_new( Arc::new(Field::new("item", data_type.to_owned(), true)), - OffsetBuffer::from_lengths(lengths), + OffsetBuffer::::from_lengths(lengths), values, None, )?)) diff --git a/datafusion/sqllogictest/test_files/array.slt b/datafusion/sqllogictest/test_files/array.slt index a3b2c8cdf1e9..7cee615a5729 100644 --- a/datafusion/sqllogictest/test_files/array.slt +++ b/datafusion/sqllogictest/test_files/array.slt @@ -1957,6 +1957,15 @@ select ---- [[1], [1], [1], [1], [1]] [[1.1, 2.2, 3.3], [1.1, 2.2, 3.3], [1.1, 2.2, 3.3]] [[, ], [, ], [, ]] [[[1, 2], [3, 4]], [[1, 2], [3, 4]]] +query ???? +select + array_repeat(arrow_cast([1], 'LargeList(Int64)'), 5), + array_repeat(arrow_cast([1.1, 2.2, 3.3], 'LargeList(Float64)'), 3), + array_repeat(arrow_cast([null, null], 'LargeList(Null)'), 3), + array_repeat(arrow_cast([[1, 2], [3, 4]], 'LargeList(List(Int64))'), 2); +---- +[[1], [1], [1], [1], [1]] [[1.1, 2.2, 3.3], [1.1, 2.2, 3.3], [1.1, 2.2, 3.3]] [[, ], [, ], [, ]] [[[1, 2], [3, 4]], [[1, 2], [3, 4]]] + # array_repeat with columns #1 statement ok @@ -1967,6 +1976,16 @@ AS VALUES (3, 2, 2.2, 'rust', make_array(7)), (0, 3, 3.3, 'datafusion', make_array(8, 9)); +statement ok +CREATE TABLE large_array_repeat_table +AS SELECT + column1, + column2, + column3, + column4, + arrow_cast(column5, 'LargeList(Int64)') as column5 +FROM array_repeat_table; + query ?????? select array_repeat(column2, column1), @@ -1982,9 +2001,27 @@ from array_repeat_table; [2, 2, 2] [2.2, 2.2, 2.2] [rust, rust, rust] [[7], [7], [7]] [2, 2, 2] [[1], [1], [1]] [] [] [] [] [3, 3, 3] [] +query ?????? +select + array_repeat(column2, column1), + array_repeat(column3, column1), + array_repeat(column4, column1), + array_repeat(column5, column1), + array_repeat(column2, 3), + array_repeat(make_array(1), column1) +from large_array_repeat_table; +---- +[1] [1.1] [a] [[4, 5, 6]] [1, 1, 1] [[1]] +[, ] [, ] [, ] [, ] [, , ] [[1], [1]] +[2, 2, 2] [2.2, 2.2, 2.2] [rust, rust, rust] [[7], [7], [7]] [2, 2, 2] [[1], [1], [1]] +[] [] [] [] [3, 3, 3] [] + statement ok drop table array_repeat_table; +statement ok +drop table large_array_repeat_table; + ## array_concat (aliases: `array_cat`, `list_concat`, `list_cat`) # array_concat error From e6b9f527d3a1823887b32a8d3dfca85ea21b204c Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Berkay=20=C5=9Eahin?= <124376117+berkaysynnada@users.noreply.github.com> Date: Thu, 4 Jan 2024 09:18:04 +0300 Subject: [PATCH 546/572] ctrl+c termination (#8739) --- datafusion-cli/Cargo.lock | 10 ++++++++++ datafusion-cli/Cargo.toml | 2 +- datafusion-cli/src/exec.rs | 13 ++++++++++--- 3 files changed, 21 insertions(+), 4 deletions(-) diff --git a/datafusion-cli/Cargo.lock b/datafusion-cli/Cargo.lock index e85e8b1a9edb..252b00ca0adc 100644 --- a/datafusion-cli/Cargo.lock +++ b/datafusion-cli/Cargo.lock @@ -3060,6 +3060,15 @@ dependencies = [ "digest", ] +[[package]] +name = "signal-hook-registry" +version = "1.4.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "d8229b473baa5980ac72ef434c4415e70c4b5e71b423043adb4ba059f89c99a1" +dependencies = [ + "libc", +] + [[package]] name = "siphasher" version = "0.3.11" @@ -3371,6 +3380,7 @@ dependencies = [ "num_cpus", "parking_lot", "pin-project-lite", + "signal-hook-registry", "socket2", "tokio-macros", "windows-sys 0.48.0", diff --git a/datafusion-cli/Cargo.toml b/datafusion-cli/Cargo.toml index e1ddba4cad1a..eab7c8e0d1f8 100644 --- a/datafusion-cli/Cargo.toml +++ b/datafusion-cli/Cargo.toml @@ -45,7 +45,7 @@ parking_lot = { version = "0.12" } parquet = { version = "49.0.0", default-features = false } regex = "1.8" rustyline = "11.0" -tokio = { version = "1.24", features = ["macros", "rt", "rt-multi-thread", "sync", "parking_lot"] } +tokio = { version = "1.24", features = ["macros", "rt", "rt-multi-thread", "sync", "parking_lot", "signal"] } url = "2.2" [dev-dependencies] diff --git a/datafusion-cli/src/exec.rs b/datafusion-cli/src/exec.rs index ba9aa2e69aa6..2320a8c314cf 100644 --- a/datafusion-cli/src/exec.rs +++ b/datafusion-cli/src/exec.rs @@ -45,6 +45,7 @@ use datafusion::sql::{parser::DFParser, sqlparser::dialect::dialect_from_str}; use object_store::ObjectStore; use rustyline::error::ReadlineError; use rustyline::Editor; +use tokio::signal; use url::Url; /// run and execute SQL statements and commands, against a context with the given print options @@ -165,9 +166,15 @@ pub async fn exec_from_repl( } Ok(line) => { rl.add_history_entry(line.trim_end())?; - match exec_and_print(ctx, print_options, line).await { - Ok(_) => {} - Err(err) => eprintln!("{err}"), + tokio::select! { + res = exec_and_print(ctx, print_options, line) => match res { + Ok(_) => {} + Err(err) => eprintln!("{err}"), + }, + _ = signal::ctrl_c() => { + println!("^C"); + continue + }, } // dialect might have changed rl.helper_mut().unwrap().set_dialect( From 819d3577872a082f2aea7a68ae83d68534049662 Mon Sep 17 00:00:00 2001 From: Mustafa Akur <106137913+mustafasrepo@users.noreply.github.com> Date: Thu, 4 Jan 2024 09:39:02 +0300 Subject: [PATCH 547/572] Add support for functional dependency for ROW_NUMBER window function. (#8737) * Add primary key support for row_number window function * Add comments, minor changes * Add new test * Review --------- Co-authored-by: Mehmet Ozan Kabak --- datafusion/expr/src/logical_plan/plan.rs | 59 ++++++++++++++++--- datafusion/sqllogictest/test_files/window.slt | 40 ++++++++++++- 2 files changed, 91 insertions(+), 8 deletions(-) diff --git a/datafusion/expr/src/logical_plan/plan.rs b/datafusion/expr/src/logical_plan/plan.rs index c0c520c4e211..93a38fb40df5 100644 --- a/datafusion/expr/src/logical_plan/plan.rs +++ b/datafusion/expr/src/logical_plan/plan.rs @@ -25,7 +25,9 @@ use std::sync::Arc; use super::dml::CopyTo; use super::DdlStatement; use crate::dml::CopyOptions; -use crate::expr::{Alias, Exists, InSubquery, Placeholder, Sort as SortExpr}; +use crate::expr::{ + Alias, Exists, InSubquery, Placeholder, Sort as SortExpr, WindowFunction, +}; use crate::expr_rewriter::{create_col_from_scalar_expr, normalize_cols}; use crate::logical_plan::display::{GraphvizVisitor, IndentVisitor}; use crate::logical_plan::extension::UserDefinedLogicalNode; @@ -36,9 +38,9 @@ use crate::utils::{ split_conjunction, }; use crate::{ - build_join_schema, expr_vec_fmt, BinaryExpr, CreateMemoryTable, CreateView, Expr, - ExprSchemable, LogicalPlanBuilder, Operator, TableProviderFilterPushDown, - TableSource, + build_join_schema, expr_vec_fmt, BinaryExpr, BuiltInWindowFunction, + CreateMemoryTable, CreateView, Expr, ExprSchemable, LogicalPlanBuilder, Operator, + TableProviderFilterPushDown, TableSource, WindowFunctionDefinition, }; use arrow::datatypes::{DataType, Field, Schema, SchemaRef}; @@ -48,9 +50,10 @@ use datafusion_common::tree_node::{ }; use datafusion_common::{ aggregate_functional_dependencies, internal_err, plan_err, Column, Constraints, - DFField, DFSchema, DFSchemaRef, DataFusionError, Dependency, FunctionalDependencies, - OwnedTableReference, ParamValues, Result, UnnestOptions, + DFField, DFSchema, DFSchemaRef, DataFusionError, Dependency, FunctionalDependence, + FunctionalDependencies, OwnedTableReference, ParamValues, Result, UnnestOptions, }; + // backwards compatibility pub use datafusion_common::display::{PlanType, StringifiedPlan, ToStringifiedPlan}; pub use datafusion_common::{JoinConstraint, JoinType}; @@ -1967,7 +1970,9 @@ pub struct Window { impl Window { /// Create a new window operator. pub fn try_new(window_expr: Vec, input: Arc) -> Result { - let mut window_fields: Vec = input.schema().fields().clone(); + let fields = input.schema().fields(); + let input_len = fields.len(); + let mut window_fields = fields.clone(); window_fields.extend_from_slice(&exprlist_to_fields(window_expr.iter(), &input)?); let metadata = input.schema().metadata().clone(); @@ -1976,6 +1981,46 @@ impl Window { input.schema().functional_dependencies().clone(); window_func_dependencies.extend_target_indices(window_fields.len()); + // Since we know that ROW_NUMBER outputs will be unique (i.e. it consists + // of consecutive numbers per partition), we can represent this fact with + // functional dependencies. + let mut new_dependencies = window_expr + .iter() + .enumerate() + .filter_map(|(idx, expr)| { + if let Expr::WindowFunction(WindowFunction { + // Function is ROW_NUMBER + fun: + WindowFunctionDefinition::BuiltInWindowFunction( + BuiltInWindowFunction::RowNumber, + ), + partition_by, + .. + }) = expr + { + // When there is no PARTITION BY, row number will be unique + // across the entire table. + if partition_by.is_empty() { + return Some(idx + input_len); + } + } + None + }) + .map(|idx| { + FunctionalDependence::new(vec![idx], vec![], false) + .with_mode(Dependency::Single) + }) + .collect::>(); + + if !new_dependencies.is_empty() { + for dependence in new_dependencies.iter_mut() { + dependence.target_indices = (0..window_fields.len()).collect(); + } + // Add the dependency introduced because of ROW_NUMBER window function to the functional dependency + let new_deps = FunctionalDependencies::new(new_dependencies); + window_func_dependencies.extend(new_deps); + } + Ok(Window { input, window_expr, diff --git a/datafusion/sqllogictest/test_files/window.slt b/datafusion/sqllogictest/test_files/window.slt index aa083290b4f4..7d6d59201396 100644 --- a/datafusion/sqllogictest/test_files/window.slt +++ b/datafusion/sqllogictest/test_files/window.slt @@ -3832,4 +3832,42 @@ select row_number() over (partition by 1 order by 1) rn, from (select 1 a union all select 2 a) x; ---- 1 1 1 1 1 1 -2 1 1 2 2 1 \ No newline at end of file +2 1 1 2 2 1 + +# when partition by expression is empty row number result will be unique. +query TII +SELECT * +FROM (SELECT c1, c2, ROW_NUMBER() OVER() as rn + FROM aggregate_test_100 + LIMIT 5) +GROUP BY rn +ORDER BY rn; +---- +c 2 1 +d 5 2 +b 1 3 +a 1 4 +b 5 5 + +# when partition by expression is constant row number result will be unique. +query TII +SELECT * +FROM (SELECT c1, c2, ROW_NUMBER() OVER(PARTITION BY 3) as rn + FROM aggregate_test_100 + LIMIT 5) +GROUP BY rn +ORDER BY rn; +---- +c 2 1 +d 5 2 +b 1 3 +a 1 4 +b 5 5 + +statement error DataFusion error: Error during planning: Projection references non-aggregate values: Expression aggregate_test_100.c1 could not be resolved from available columns: rn +SELECT * +FROM (SELECT c1, c2, ROW_NUMBER() OVER(PARTITION BY c1) as rn + FROM aggregate_test_100 + LIMIT 5) +GROUP BY rn +ORDER BY rn; From e5036d0e760b637724e8ac59c32924f126311d39 Mon Sep 17 00:00:00 2001 From: Andrew Lamb Date: Thu, 4 Jan 2024 17:20:46 -0500 Subject: [PATCH 548/572] Minor: reduce code duplication in PruningPredicate test (#8441) --- .../core/src/physical_optimizer/pruning.rs | 612 +++++++++--------- 1 file changed, 310 insertions(+), 302 deletions(-) diff --git a/datafusion/core/src/physical_optimizer/pruning.rs b/datafusion/core/src/physical_optimizer/pruning.rs index 06cfc7282468..1b68553a896f 100644 --- a/datafusion/core/src/physical_optimizer/pruning.rs +++ b/datafusion/core/src/physical_optimizer/pruning.rs @@ -2017,54 +2017,52 @@ mod tests { DataType::Decimal128(9, 2), true, )])); - // s1 > 5 - let expr = col("s1").gt(lit(ScalarValue::Decimal128(Some(500), 9, 2))); - let expr = logical2physical(&expr, &schema); - // If the data is written by spark, the physical data type is INT32 in the parquet - // So we use the INT32 type of statistic. - let statistics = TestStatistics::new().with( - "s1", - ContainerStats::new_i32( - vec![Some(0), Some(4), None, Some(3)], // min - vec![Some(5), Some(6), Some(4), None], // max + + prune_with_expr( + // s1 > 5 + col("s1").gt(lit(ScalarValue::Decimal128(Some(500), 9, 2))), + &schema, + // If the data is written by spark, the physical data type is INT32 in the parquet + // So we use the INT32 type of statistic. + &TestStatistics::new().with( + "s1", + ContainerStats::new_i32( + vec![Some(0), Some(4), None, Some(3)], // min + vec![Some(5), Some(6), Some(4), None], // max + ), ), + &[false, true, false, true], ); - let p = PruningPredicate::try_new(expr, schema.clone()).unwrap(); - let result = p.prune(&statistics).unwrap(); - let expected = vec![false, true, false, true]; - assert_eq!(result, expected); - // with cast column to other type - let expr = cast(col("s1"), DataType::Decimal128(14, 3)) - .gt(lit(ScalarValue::Decimal128(Some(5000), 14, 3))); - let expr = logical2physical(&expr, &schema); - let statistics = TestStatistics::new().with( - "s1", - ContainerStats::new_i32( - vec![Some(0), Some(4), None, Some(3)], // min - vec![Some(5), Some(6), Some(4), None], // max + prune_with_expr( + // with cast column to other type + cast(col("s1"), DataType::Decimal128(14, 3)) + .gt(lit(ScalarValue::Decimal128(Some(5000), 14, 3))), + &schema, + &TestStatistics::new().with( + "s1", + ContainerStats::new_i32( + vec![Some(0), Some(4), None, Some(3)], // min + vec![Some(5), Some(6), Some(4), None], // max + ), ), + &[false, true, false, true], ); - let p = PruningPredicate::try_new(expr, schema.clone()).unwrap(); - let result = p.prune(&statistics).unwrap(); - let expected = vec![false, true, false, true]; - assert_eq!(result, expected); - // with try cast column to other type - let expr = try_cast(col("s1"), DataType::Decimal128(14, 3)) - .gt(lit(ScalarValue::Decimal128(Some(5000), 14, 3))); - let expr = logical2physical(&expr, &schema); - let statistics = TestStatistics::new().with( - "s1", - ContainerStats::new_i32( - vec![Some(0), Some(4), None, Some(3)], // min - vec![Some(5), Some(6), Some(4), None], // max + prune_with_expr( + // with try cast column to other type + try_cast(col("s1"), DataType::Decimal128(14, 3)) + .gt(lit(ScalarValue::Decimal128(Some(5000), 14, 3))), + &schema, + &TestStatistics::new().with( + "s1", + ContainerStats::new_i32( + vec![Some(0), Some(4), None, Some(3)], // min + vec![Some(5), Some(6), Some(4), None], // max + ), ), + &[false, true, false, true], ); - let p = PruningPredicate::try_new(expr, schema).unwrap(); - let result = p.prune(&statistics).unwrap(); - let expected = vec![false, true, false, true]; - assert_eq!(result, expected); // decimal(18,2) let schema = Arc::new(Schema::new(vec![Field::new( @@ -2072,22 +2070,21 @@ mod tests { DataType::Decimal128(18, 2), true, )])); - // s1 > 5 - let expr = col("s1").gt(lit(ScalarValue::Decimal128(Some(500), 18, 2))); - let expr = logical2physical(&expr, &schema); - // If the data is written by spark, the physical data type is INT64 in the parquet - // So we use the INT32 type of statistic. - let statistics = TestStatistics::new().with( - "s1", - ContainerStats::new_i64( - vec![Some(0), Some(4), None, Some(3)], // min - vec![Some(5), Some(6), Some(4), None], // max + prune_with_expr( + // s1 > 5 + col("s1").gt(lit(ScalarValue::Decimal128(Some(500), 18, 2))), + &schema, + // If the data is written by spark, the physical data type is INT64 in the parquet + // So we use the INT32 type of statistic. + &TestStatistics::new().with( + "s1", + ContainerStats::new_i64( + vec![Some(0), Some(4), None, Some(3)], // min + vec![Some(5), Some(6), Some(4), None], // max + ), ), + &[false, true, false, true], ); - let p = PruningPredicate::try_new(expr, schema).unwrap(); - let result = p.prune(&statistics).unwrap(); - let expected = vec![false, true, false, true]; - assert_eq!(result, expected); // decimal(23,2) let schema = Arc::new(Schema::new(vec![Field::new( @@ -2095,22 +2092,22 @@ mod tests { DataType::Decimal128(23, 2), true, )])); - // s1 > 5 - let expr = col("s1").gt(lit(ScalarValue::Decimal128(Some(500), 23, 2))); - let expr = logical2physical(&expr, &schema); - let statistics = TestStatistics::new().with( - "s1", - ContainerStats::new_decimal128( - vec![Some(0), Some(400), None, Some(300)], // min - vec![Some(500), Some(600), Some(400), None], // max - 23, - 2, + + prune_with_expr( + // s1 > 5 + col("s1").gt(lit(ScalarValue::Decimal128(Some(500), 23, 2))), + &schema, + &TestStatistics::new().with( + "s1", + ContainerStats::new_decimal128( + vec![Some(0), Some(400), None, Some(300)], // min + vec![Some(500), Some(600), Some(400), None], // max + 23, + 2, + ), ), + &[false, true, false, true], ); - let p = PruningPredicate::try_new(expr, schema).unwrap(); - let result = p.prune(&statistics).unwrap(); - let expected = vec![false, true, false, true]; - assert_eq!(result, expected); } #[test] @@ -2120,10 +2117,6 @@ mod tests { Field::new("s2", DataType::Int32, true), ])); - // Prune using s2 > 5 - let expr = col("s2").gt(lit(5)); - let expr = logical2physical(&expr, &schema); - let statistics = TestStatistics::new().with( "s2", ContainerStats::new_i32( @@ -2131,53 +2124,50 @@ mod tests { vec![Some(5), Some(6), None, None], // max ), ); + prune_with_expr( + // Prune using s2 > 5 + col("s2").gt(lit(5)), + &schema, + &statistics, + // s2 [0, 5] ==> no rows should pass + // s2 [4, 6] ==> some rows could pass + // No stats for s2 ==> some rows could pass + // s2 [3, None] (null max) ==> some rows could pass + &[false, true, true, true], + ); - // s2 [0, 5] ==> no rows should pass - // s2 [4, 6] ==> some rows could pass - // No stats for s2 ==> some rows could pass - // s2 [3, None] (null max) ==> some rows could pass - - let p = PruningPredicate::try_new(expr, schema.clone()).unwrap(); - let result = p.prune(&statistics).unwrap(); - let expected = vec![false, true, true, true]; - assert_eq!(result, expected); - - // filter with cast - let expr = cast(col("s2"), DataType::Int64).gt(lit(ScalarValue::Int64(Some(5)))); - let expr = logical2physical(&expr, &schema); - let p = PruningPredicate::try_new(expr, schema).unwrap(); - let result = p.prune(&statistics).unwrap(); - let expected = vec![false, true, true, true]; - assert_eq!(result, expected); + prune_with_expr( + // filter with cast + cast(col("s2"), DataType::Int64).gt(lit(ScalarValue::Int64(Some(5)))), + &schema, + &statistics, + &[false, true, true, true], + ); } #[test] fn prune_not_eq_data() { let schema = Arc::new(Schema::new(vec![Field::new("s1", DataType::Utf8, true)])); - // Prune using s2 != 'M' - let expr = col("s1").not_eq(lit("M")); - let expr = logical2physical(&expr, &schema); - - let statistics = TestStatistics::new().with( - "s1", - ContainerStats::new_utf8( - vec![Some("A"), Some("A"), Some("N"), Some("M"), None, Some("A")], // min - vec![Some("Z"), Some("L"), Some("Z"), Some("M"), None, None], // max + prune_with_expr( + // Prune using s2 != 'M' + col("s1").not_eq(lit("M")), + &schema, + &TestStatistics::new().with( + "s1", + ContainerStats::new_utf8( + vec![Some("A"), Some("A"), Some("N"), Some("M"), None, Some("A")], // min + vec![Some("Z"), Some("L"), Some("Z"), Some("M"), None, None], // max + ), ), + // s1 [A, Z] ==> might have values that pass predicate + // s1 [A, L] ==> all rows pass the predicate + // s1 [N, Z] ==> all rows pass the predicate + // s1 [M, M] ==> all rows do not pass the predicate + // No stats for s2 ==> some rows could pass + // s2 [3, None] (null max) ==> some rows could pass + &[true, true, true, false, true, true], ); - - // s1 [A, Z] ==> might have values that pass predicate - // s1 [A, L] ==> all rows pass the predicate - // s1 [N, Z] ==> all rows pass the predicate - // s1 [M, M] ==> all rows do not pass the predicate - // No stats for s2 ==> some rows could pass - // s2 [3, None] (null max) ==> some rows could pass - - let p = PruningPredicate::try_new(expr, schema).unwrap(); - let result = p.prune(&statistics).unwrap(); - let expected = vec![true, true, true, false, true, true]; - assert_eq!(result, expected); } /// Creates setup for boolean chunk pruning @@ -2216,69 +2206,75 @@ mod tests { fn prune_bool_const_expr() { let (schema, statistics, _, _) = bool_setup(); - // true - let expr = lit(true); - let expr = logical2physical(&expr, &schema); - let p = PruningPredicate::try_new(expr, schema.clone()).unwrap(); - let result = p.prune(&statistics).unwrap(); - assert_eq!(result, vec![true, true, true, true, true]); + prune_with_expr( + // true + lit(true), + &schema, + &statistics, + &[true, true, true, true, true], + ); - // false - // constant literals that do NOT refer to any columns are currently not evaluated at all, hence the result is - // "all true" - let expr = lit(false); - let expr = logical2physical(&expr, &schema); - let p = PruningPredicate::try_new(expr, schema).unwrap(); - let result = p.prune(&statistics).unwrap(); - assert_eq!(result, vec![true, true, true, true, true]); + prune_with_expr( + // false + // constant literals that do NOT refer to any columns are currently not evaluated at all, hence the result is + // "all true" + lit(false), + &schema, + &statistics, + &[true, true, true, true, true], + ); } #[test] fn prune_bool_column() { let (schema, statistics, expected_true, _) = bool_setup(); - // b1 - let expr = col("b1"); - let expr = logical2physical(&expr, &schema); - let p = PruningPredicate::try_new(expr, schema).unwrap(); - let result = p.prune(&statistics).unwrap(); - assert_eq!(result, expected_true); + prune_with_expr( + // b1 + col("b1"), + &schema, + &statistics, + &expected_true, + ); } #[test] fn prune_bool_not_column() { let (schema, statistics, _, expected_false) = bool_setup(); - // !b1 - let expr = col("b1").not(); - let expr = logical2physical(&expr, &schema); - let p = PruningPredicate::try_new(expr, schema).unwrap(); - let result = p.prune(&statistics).unwrap(); - assert_eq!(result, expected_false); + prune_with_expr( + // !b1 + col("b1").not(), + &schema, + &statistics, + &expected_false, + ); } #[test] fn prune_bool_column_eq_true() { let (schema, statistics, expected_true, _) = bool_setup(); - // b1 = true - let expr = col("b1").eq(lit(true)); - let expr = logical2physical(&expr, &schema); - let p = PruningPredicate::try_new(expr, schema).unwrap(); - let result = p.prune(&statistics).unwrap(); - assert_eq!(result, expected_true); + prune_with_expr( + // b1 = true + col("b1").eq(lit(true)), + &schema, + &statistics, + &expected_true, + ); } #[test] fn prune_bool_not_column_eq_true() { let (schema, statistics, _, expected_false) = bool_setup(); - // !b1 = true - let expr = col("b1").not().eq(lit(true)); - let expr = logical2physical(&expr, &schema); - let p = PruningPredicate::try_new(expr, schema).unwrap(); - let result = p.prune(&statistics).unwrap(); - assert_eq!(result, expected_false); + prune_with_expr( + // !b1 = true + col("b1").not().eq(lit(true)), + &schema, + &statistics, + &expected_false, + ); } /// Creates a setup for chunk pruning, modeling a int32 column "i" @@ -2313,21 +2309,18 @@ mod tests { // i [-11, -1] ==> no rows can pass (not keep) // i [NULL, NULL] ==> unknown (must keep) // i [1, NULL] ==> unknown (must keep) - let expected_ret = vec![true, true, false, true, true]; + let expected_ret = &[true, true, false, true, true]; // i > 0 - let expr = col("i").gt(lit(0)); - let expr = logical2physical(&expr, &schema); - let p = PruningPredicate::try_new(expr, schema.clone()).unwrap(); - let result = p.prune(&statistics).unwrap(); - assert_eq!(result, expected_ret); + prune_with_expr(col("i").gt(lit(0)), &schema, &statistics, expected_ret); // -i < 0 - let expr = Expr::Negative(Box::new(col("i"))).lt(lit(0)); - let expr = logical2physical(&expr, &schema); - let p = PruningPredicate::try_new(expr, schema).unwrap(); - let result = p.prune(&statistics).unwrap(); - assert_eq!(result, expected_ret); + prune_with_expr( + Expr::Negative(Box::new(col("i"))).lt(lit(0)), + &schema, + &statistics, + expected_ret, + ); } #[test] @@ -2340,21 +2333,23 @@ mod tests { // i [-11, -1] ==> all rows must pass (must keep) // i [NULL, NULL] ==> unknown (must keep) // i [1, NULL] ==> no rows can pass (not keep) - let expected_ret = vec![true, false, true, true, false]; + let expected_ret = &[true, false, true, true, false]; - // i <= 0 - let expr = col("i").lt_eq(lit(0)); - let expr = logical2physical(&expr, &schema); - let p = PruningPredicate::try_new(expr, schema.clone()).unwrap(); - let result = p.prune(&statistics).unwrap(); - assert_eq!(result, expected_ret); + prune_with_expr( + // i <= 0 + col("i").lt_eq(lit(0)), + &schema, + &statistics, + expected_ret, + ); - // -i >= 0 - let expr = Expr::Negative(Box::new(col("i"))).gt_eq(lit(0)); - let expr = logical2physical(&expr, &schema); - let p = PruningPredicate::try_new(expr, schema).unwrap(); - let result = p.prune(&statistics).unwrap(); - assert_eq!(result, expected_ret); + prune_with_expr( + // -i >= 0 + Expr::Negative(Box::new(col("i"))).gt_eq(lit(0)), + &schema, + &statistics, + expected_ret, + ); } #[test] @@ -2367,37 +2362,39 @@ mod tests { // i [-11, -1] ==> no rows could pass in theory (conservatively keep) // i [NULL, NULL] ==> unknown (must keep) // i [1, NULL] ==> no rows can pass (conservatively keep) - let expected_ret = vec![true, true, true, true, true]; + let expected_ret = &[true, true, true, true, true]; - // cast(i as utf8) <= 0 - let expr = cast(col("i"), DataType::Utf8).lt_eq(lit("0")); - let expr = logical2physical(&expr, &schema); - let p = PruningPredicate::try_new(expr, schema.clone()).unwrap(); - let result = p.prune(&statistics).unwrap(); - assert_eq!(result, expected_ret); + prune_with_expr( + // cast(i as utf8) <= 0 + cast(col("i"), DataType::Utf8).lt_eq(lit("0")), + &schema, + &statistics, + expected_ret, + ); - // try_cast(i as utf8) <= 0 - let expr = try_cast(col("i"), DataType::Utf8).lt_eq(lit("0")); - let expr = logical2physical(&expr, &schema); - let p = PruningPredicate::try_new(expr, schema.clone()).unwrap(); - let result = p.prune(&statistics).unwrap(); - assert_eq!(result, expected_ret); + prune_with_expr( + // try_cast(i as utf8) <= 0 + try_cast(col("i"), DataType::Utf8).lt_eq(lit("0")), + &schema, + &statistics, + expected_ret, + ); - // cast(-i as utf8) >= 0 - let expr = - cast(Expr::Negative(Box::new(col("i"))), DataType::Utf8).gt_eq(lit("0")); - let expr = logical2physical(&expr, &schema); - let p = PruningPredicate::try_new(expr, schema.clone()).unwrap(); - let result = p.prune(&statistics).unwrap(); - assert_eq!(result, expected_ret); + prune_with_expr( + // cast(-i as utf8) >= 0 + cast(Expr::Negative(Box::new(col("i"))), DataType::Utf8).gt_eq(lit("0")), + &schema, + &statistics, + expected_ret, + ); - // try_cast(-i as utf8) >= 0 - let expr = - try_cast(Expr::Negative(Box::new(col("i"))), DataType::Utf8).gt_eq(lit("0")); - let expr = logical2physical(&expr, &schema); - let p = PruningPredicate::try_new(expr, schema).unwrap(); - let result = p.prune(&statistics).unwrap(); - assert_eq!(result, expected_ret); + prune_with_expr( + // try_cast(-i as utf8) >= 0 + try_cast(Expr::Negative(Box::new(col("i"))), DataType::Utf8).gt_eq(lit("0")), + &schema, + &statistics, + expected_ret, + ); } #[test] @@ -2410,14 +2407,15 @@ mod tests { // i [-11, -1] ==> no rows can pass (not keep) // i [NULL, NULL] ==> unknown (must keep) // i [1, NULL] ==> no rows can pass (not keep) - let expected_ret = vec![true, false, false, true, false]; + let expected_ret = &[true, false, false, true, false]; - // i = 0 - let expr = col("i").eq(lit(0)); - let expr = logical2physical(&expr, &schema); - let p = PruningPredicate::try_new(expr, schema).unwrap(); - let result = p.prune(&statistics).unwrap(); - assert_eq!(result, expected_ret); + prune_with_expr( + // i = 0 + col("i").eq(lit(0)), + &schema, + &statistics, + expected_ret, + ); } #[test] @@ -2430,19 +2428,21 @@ mod tests { // i [-11, -1] ==> no rows can pass (not keep) // i [NULL, NULL] ==> unknown (must keep) // i [1, NULL] ==> no rows can pass (not keep) - let expected_ret = vec![true, false, false, true, false]; + let expected_ret = &[true, false, false, true, false]; - let expr = cast(col("i"), DataType::Int64).eq(lit(0i64)); - let expr = logical2physical(&expr, &schema); - let p = PruningPredicate::try_new(expr, schema.clone()).unwrap(); - let result = p.prune(&statistics).unwrap(); - assert_eq!(result, expected_ret); + prune_with_expr( + cast(col("i"), DataType::Int64).eq(lit(0i64)), + &schema, + &statistics, + expected_ret, + ); - let expr = try_cast(col("i"), DataType::Int64).eq(lit(0i64)); - let expr = logical2physical(&expr, &schema); - let p = PruningPredicate::try_new(expr, schema).unwrap(); - let result = p.prune(&statistics).unwrap(); - assert_eq!(result, expected_ret); + prune_with_expr( + try_cast(col("i"), DataType::Int64).eq(lit(0i64)), + &schema, + &statistics, + expected_ret, + ); } #[test] @@ -2458,13 +2458,14 @@ mod tests { // i [-11, -1] ==> no rows can pass (could keep) // i [NULL, NULL] ==> unknown (keep) // i [1, NULL] ==> no rows can pass (could keep) - let expected_ret = vec![true, true, true, true, true]; + let expected_ret = &[true, true, true, true, true]; - let expr = cast(col("i"), DataType::Utf8).eq(lit("0")); - let expr = logical2physical(&expr, &schema); - let p = PruningPredicate::try_new(expr, schema).unwrap(); - let result = p.prune(&statistics).unwrap(); - assert_eq!(result, expected_ret); + prune_with_expr( + cast(col("i"), DataType::Utf8).eq(lit("0")), + &schema, + &statistics, + expected_ret, + ); } #[test] @@ -2477,21 +2478,23 @@ mod tests { // i [-11, -1] ==> no rows can pass (not keep) // i [NULL, NULL] ==> unknown (must keep) // i [1, NULL] ==> all rows must pass (must keep) - let expected_ret = vec![true, true, false, true, true]; + let expected_ret = &[true, true, false, true, true]; - // i > -1 - let expr = col("i").gt(lit(-1)); - let expr = logical2physical(&expr, &schema); - let p = PruningPredicate::try_new(expr, schema.clone()).unwrap(); - let result = p.prune(&statistics).unwrap(); - assert_eq!(result, expected_ret); + prune_with_expr( + // i > -1 + col("i").gt(lit(-1)), + &schema, + &statistics, + expected_ret, + ); - // -i < 1 - let expr = Expr::Negative(Box::new(col("i"))).lt(lit(1)); - let expr = logical2physical(&expr, &schema); - let p = PruningPredicate::try_new(expr, schema).unwrap(); - let result = p.prune(&statistics).unwrap(); - assert_eq!(result, expected_ret); + prune_with_expr( + // -i < 1 + Expr::Negative(Box::new(col("i"))).lt(lit(1)), + &schema, + &statistics, + expected_ret, + ); } #[test] @@ -2500,14 +2503,15 @@ mod tests { // Expression "i IS NULL" when there are no null statistics, // should all be kept - let expected_ret = vec![true, true, true, true, true]; + let expected_ret = &[true, true, true, true, true]; - // i IS NULL, no null statistics - let expr = col("i").is_null(); - let expr = logical2physical(&expr, &schema); - let p = PruningPredicate::try_new(expr, schema.clone()).unwrap(); - let result = p.prune(&statistics).unwrap(); - assert_eq!(result, expected_ret); + prune_with_expr( + // i IS NULL, no null statistics + col("i").is_null(), + &schema, + &statistics, + expected_ret, + ); // provide null counts for each column let statistics = statistics.with_null_counts( @@ -2521,51 +2525,55 @@ mod tests { ], ); - let expected_ret = vec![false, true, true, true, false]; + let expected_ret = &[false, true, true, true, false]; - // i IS NULL, with actual null statistcs - let expr = col("i").is_null(); - let expr = logical2physical(&expr, &schema); - let p = PruningPredicate::try_new(expr, schema).unwrap(); - let result = p.prune(&statistics).unwrap(); - assert_eq!(result, expected_ret); + prune_with_expr( + // i IS NULL, with actual null statistcs + col("i").is_null(), + &schema, + &statistics, + expected_ret, + ); } #[test] fn prune_cast_column_scalar() { // The data type of column i is INT32 let (schema, statistics) = int32_setup(); - let expected_ret = vec![true, true, false, true, true]; + let expected_ret = &[true, true, false, true, true]; - // i > int64(0) - let expr = col("i").gt(cast(lit(ScalarValue::Int64(Some(0))), DataType::Int32)); - let expr = logical2physical(&expr, &schema); - let p = PruningPredicate::try_new(expr, schema.clone()).unwrap(); - let result = p.prune(&statistics).unwrap(); - assert_eq!(result, expected_ret); + prune_with_expr( + // i > int64(0) + col("i").gt(cast(lit(ScalarValue::Int64(Some(0))), DataType::Int32)), + &schema, + &statistics, + expected_ret, + ); - // cast(i as int64) > int64(0) - let expr = cast(col("i"), DataType::Int64).gt(lit(ScalarValue::Int64(Some(0)))); - let expr = logical2physical(&expr, &schema); - let p = PruningPredicate::try_new(expr, schema.clone()).unwrap(); - let result = p.prune(&statistics).unwrap(); - assert_eq!(result, expected_ret); + prune_with_expr( + // cast(i as int64) > int64(0) + cast(col("i"), DataType::Int64).gt(lit(ScalarValue::Int64(Some(0)))), + &schema, + &statistics, + expected_ret, + ); - // try_cast(i as int64) > int64(0) - let expr = - try_cast(col("i"), DataType::Int64).gt(lit(ScalarValue::Int64(Some(0)))); - let expr = logical2physical(&expr, &schema); - let p = PruningPredicate::try_new(expr, schema.clone()).unwrap(); - let result = p.prune(&statistics).unwrap(); - assert_eq!(result, expected_ret); + prune_with_expr( + // try_cast(i as int64) > int64(0) + try_cast(col("i"), DataType::Int64).gt(lit(ScalarValue::Int64(Some(0)))), + &schema, + &statistics, + expected_ret, + ); - // `-cast(i as int64) < 0` convert to `cast(i as int64) > -0` - let expr = Expr::Negative(Box::new(cast(col("i"), DataType::Int64))) - .lt(lit(ScalarValue::Int64(Some(0)))); - let expr = logical2physical(&expr, &schema); - let p = PruningPredicate::try_new(expr, schema).unwrap(); - let result = p.prune(&statistics).unwrap(); - assert_eq!(result, expected_ret); + prune_with_expr( + // `-cast(i as int64) < 0` convert to `cast(i as int64) > -0` + Expr::Negative(Box::new(cast(col("i"), DataType::Int64))) + .lt(lit(ScalarValue::Int64(Some(0)))), + &schema, + &statistics, + expected_ret, + ); } #[test] @@ -2721,7 +2729,7 @@ mod tests { &schema, &statistics, // rule out containers ('false) where we know foo is not present - vec![true, false, true, true, false, true, true, false, true], + &[true, false, true, true, false, true, true, false, true], ); // s1 = 'bar' @@ -2730,7 +2738,7 @@ mod tests { &schema, &statistics, // rule out containers where we know bar is not present - vec![true, true, true, false, false, false, true, true, true], + &[true, true, true, false, false, false, true, true, true], ); // s1 = 'baz' (unknown value) @@ -2739,7 +2747,7 @@ mod tests { &schema, &statistics, // can't rule out anything - vec![true, true, true, true, true, true, true, true, true], + &[true, true, true, true, true, true, true, true, true], ); // s1 = 'foo' AND s1 = 'bar' @@ -2750,7 +2758,7 @@ mod tests { // logically this predicate can't possibly be true (the column can't // take on both values) but we could rule it out if the stats tell // us that both values are not present - vec![true, true, true, true, true, true, true, true, true], + &[true, true, true, true, true, true, true, true, true], ); // s1 = 'foo' OR s1 = 'bar' @@ -2759,7 +2767,7 @@ mod tests { &schema, &statistics, // can rule out containers that we know contain neither foo nor bar - vec![true, true, true, true, true, true, false, false, false], + &[true, true, true, true, true, true, false, false, false], ); // s1 = 'foo' OR s1 = 'baz' @@ -2768,7 +2776,7 @@ mod tests { &schema, &statistics, // can't rule out anything container - vec![true, true, true, true, true, true, true, true, true], + &[true, true, true, true, true, true, true, true, true], ); // s1 = 'foo' OR s1 = 'bar' OR s1 = 'baz' @@ -2781,7 +2789,7 @@ mod tests { &statistics, // can rule out any containers based on knowledge of s1 and `foo`, // `bar` and (`foo`, `bar`) - vec![true, true, true, true, true, true, true, true, true], + &[true, true, true, true, true, true, true, true, true], ); // s1 != foo @@ -2790,7 +2798,7 @@ mod tests { &schema, &statistics, // rule out containers we know for sure only contain foo - vec![false, true, true, false, true, true, false, true, true], + &[false, true, true, false, true, true, false, true, true], ); // s1 != bar @@ -2799,7 +2807,7 @@ mod tests { &schema, &statistics, // rule out when we know for sure s1 has the value bar - vec![false, false, false, true, true, true, true, true, true], + &[false, false, false, true, true, true, true, true, true], ); // s1 != foo AND s1 != bar @@ -2810,7 +2818,7 @@ mod tests { &schema, &statistics, // can rule out any container where we know s1 does not have either 'foo' or 'bar' - vec![true, true, true, false, false, false, true, true, true], + &[true, true, true, false, false, false, true, true, true], ); // s1 != foo AND s1 != bar AND s1 != baz @@ -2822,7 +2830,7 @@ mod tests { &schema, &statistics, // can't rule out any container based on knowledge of s1,s2 - vec![true, true, true, true, true, true, true, true, true], + &[true, true, true, true, true, true, true, true, true], ); // s1 != foo OR s1 != bar @@ -2833,7 +2841,7 @@ mod tests { &schema, &statistics, // cant' rule out anything based on contains information - vec![true, true, true, true, true, true, true, true, true], + &[true, true, true, true, true, true, true, true, true], ); // s1 != foo OR s1 != bar OR s1 != baz @@ -2845,7 +2853,7 @@ mod tests { &schema, &statistics, // cant' rule out anything based on contains information - vec![true, true, true, true, true, true, true, true, true], + &[true, true, true, true, true, true, true, true, true], ); } @@ -2907,7 +2915,7 @@ mod tests { &schema, &statistics, // rule out containers where we know s1 is not present - vec![true, false, true, true, false, true, true, false, true], + &[true, false, true, true, false, true, true, false, true], ); // s1 = 'foo' OR s2 = 'bar' @@ -2917,7 +2925,7 @@ mod tests { &schema, &statistics, // can't rule out any container (would need to prove that s1 != foo AND s2 != bar) - vec![true, true, true, true, true, true, true, true, true], + &[true, true, true, true, true, true, true, true, true], ); // s1 = 'foo' AND s2 != 'bar' @@ -2928,7 +2936,7 @@ mod tests { // can only rule out container where we know either: // 1. s1 doesn't have the value 'foo` or // 2. s2 has only the value of 'bar' - vec![false, false, false, true, false, true, true, false, true], + &[false, false, false, true, false, true, true, false, true], ); // s1 != 'foo' AND s2 != 'bar' @@ -2941,7 +2949,7 @@ mod tests { // Can rule out any container where we know either // 1. s1 has only the value 'foo' // 2. s2 has only the value 'bar' - vec![false, false, false, false, true, true, false, true, true], + &[false, false, false, false, true, true, false, true, true], ); // s1 != 'foo' AND (s2 = 'bar' OR s2 = 'baz') @@ -2953,7 +2961,7 @@ mod tests { &statistics, // Can rule out any container where we know s1 has only the value // 'foo'. Can't use knowledge of s2 and bar to rule out anything - vec![false, true, true, false, true, true, false, true, true], + &[false, true, true, false, true, true, false, true, true], ); // s1 like '%foo%bar%' @@ -2962,7 +2970,7 @@ mod tests { &schema, &statistics, // cant rule out anything with information we know - vec![true, true, true, true, true, true, true, true, true], + &[true, true, true, true, true, true, true, true, true], ); // s1 like '%foo%bar%' AND s2 = 'bar' @@ -2973,7 +2981,7 @@ mod tests { &schema, &statistics, // can rule out any container where we know s2 does not have the value 'bar' - vec![true, true, true, false, false, false, true, true, true], + &[true, true, true, false, false, false, true, true, true], ); // s1 like '%foo%bar%' OR s2 = 'bar' @@ -2983,7 +2991,7 @@ mod tests { &statistics, // can't rule out anything (we would have to prove that both the // like and the equality must be false) - vec![true, true, true, true, true, true, true, true, true], + &[true, true, true, true, true, true, true, true, true], ); } @@ -3055,7 +3063,7 @@ mod tests { // 1. 0 is outside the min/max range of i // 1. s does not contain foo // (range is false, and contained is false) - vec![true, false, true, false, false, false, true, false, true], + &[true, false, true, false, false, false, true, false, true], ); // i = 0 and s != 'foo' @@ -3066,7 +3074,7 @@ mod tests { // Can rule out containers where either: // 1. 0 is outside the min/max range of i // 2. s only contains foo - vec![false, false, false, true, false, true, true, false, true], + &[false, false, false, true, false, true, true, false, true], ); // i = 0 OR s = 'foo' @@ -3076,7 +3084,7 @@ mod tests { &statistics, // in theory could rule out containers if we had min/max values for // s as well. But in this case we don't so we can't rule out anything - vec![true, true, true, true, true, true, true, true, true], + &[true, true, true, true, true, true, true, true, true], ); } @@ -3091,7 +3099,7 @@ mod tests { expr: Expr, schema: &SchemaRef, statistics: &TestStatistics, - expected: Vec, + expected: &[bool], ) { println!("Pruning with expr: {}", expr); let expr = logical2physical(&expr, schema); From 561d941854e96bed2de3c7a4c6de9afab622a08b Mon Sep 17 00:00:00 2001 From: Eduard Karacharov <13005055+korowa@users.noreply.github.com> Date: Fri, 5 Jan 2024 11:32:17 +0200 Subject: [PATCH 549/572] feat: native types in `DistinctCountAccumulator` for primitive types (#8721) * DistinctCountGroupsAccumulator * test coverage * clippy warnings * count distinct for primitive types * revert hashset to std * fixed accumulator size estimation --- .../src/aggregate/count_distinct.rs | 298 +++++++++++++++++- .../src/aggregate/sum_distinct.rs | 22 +- .../physical-expr/src/aggregate/utils.rs | 20 +- 3 files changed, 311 insertions(+), 29 deletions(-) diff --git a/datafusion/physical-expr/src/aggregate/count_distinct.rs b/datafusion/physical-expr/src/aggregate/count_distinct.rs index c2fd32a96c4f..f7c13948b2dc 100644 --- a/datafusion/physical-expr/src/aggregate/count_distinct.rs +++ b/datafusion/physical-expr/src/aggregate/count_distinct.rs @@ -15,21 +15,32 @@ // specific language governing permissions and limitations // under the License. -use arrow::datatypes::{DataType, Field}; +use arrow::datatypes::{DataType, Field, TimeUnit}; +use arrow_array::types::{ + ArrowPrimitiveType, Date32Type, Date64Type, Decimal128Type, Decimal256Type, + Float16Type, Float32Type, Float64Type, Int16Type, Int32Type, Int64Type, Int8Type, + Time32MillisecondType, Time32SecondType, Time64MicrosecondType, Time64NanosecondType, + TimestampMicrosecondType, TimestampMillisecondType, TimestampNanosecondType, + TimestampSecondType, UInt16Type, UInt32Type, UInt64Type, UInt8Type, +}; +use arrow_array::PrimitiveArray; use std::any::Any; +use std::cmp::Eq; use std::fmt::Debug; +use std::hash::Hash; use std::sync::Arc; use ahash::RandomState; use arrow::array::{Array, ArrayRef}; use std::collections::HashSet; -use crate::aggregate::utils::down_cast_any_ref; +use crate::aggregate::utils::{down_cast_any_ref, Hashable}; use crate::expressions::format_state_name; use crate::{AggregateExpr, PhysicalExpr}; -use datafusion_common::Result; -use datafusion_common::ScalarValue; +use datafusion_common::cast::{as_list_array, as_primitive_array}; +use datafusion_common::utils::array_into_list_array; +use datafusion_common::{Result, ScalarValue}; use datafusion_expr::Accumulator; type DistinctScalarValues = ScalarValue; @@ -60,6 +71,18 @@ impl DistinctCount { } } +macro_rules! native_distinct_count_accumulator { + ($TYPE:ident) => {{ + Ok(Box::new(NativeDistinctCountAccumulator::<$TYPE>::new())) + }}; +} + +macro_rules! float_distinct_count_accumulator { + ($TYPE:ident) => {{ + Ok(Box::new(FloatDistinctCountAccumulator::<$TYPE>::new())) + }}; +} + impl AggregateExpr for DistinctCount { /// Return a reference to Any that can be used for downcasting fn as_any(&self) -> &dyn Any { @@ -83,10 +106,57 @@ impl AggregateExpr for DistinctCount { } fn create_accumulator(&self) -> Result> { - Ok(Box::new(DistinctCountAccumulator { - values: HashSet::default(), - state_data_type: self.state_data_type.clone(), - })) + use DataType::*; + use TimeUnit::*; + + match &self.state_data_type { + Int8 => native_distinct_count_accumulator!(Int8Type), + Int16 => native_distinct_count_accumulator!(Int16Type), + Int32 => native_distinct_count_accumulator!(Int32Type), + Int64 => native_distinct_count_accumulator!(Int64Type), + UInt8 => native_distinct_count_accumulator!(UInt8Type), + UInt16 => native_distinct_count_accumulator!(UInt16Type), + UInt32 => native_distinct_count_accumulator!(UInt32Type), + UInt64 => native_distinct_count_accumulator!(UInt64Type), + Decimal128(_, _) => native_distinct_count_accumulator!(Decimal128Type), + Decimal256(_, _) => native_distinct_count_accumulator!(Decimal256Type), + + Date32 => native_distinct_count_accumulator!(Date32Type), + Date64 => native_distinct_count_accumulator!(Date64Type), + Time32(Millisecond) => { + native_distinct_count_accumulator!(Time32MillisecondType) + } + Time32(Second) => { + native_distinct_count_accumulator!(Time32SecondType) + } + Time64(Microsecond) => { + native_distinct_count_accumulator!(Time64MicrosecondType) + } + Time64(Nanosecond) => { + native_distinct_count_accumulator!(Time64NanosecondType) + } + Timestamp(Microsecond, _) => { + native_distinct_count_accumulator!(TimestampMicrosecondType) + } + Timestamp(Millisecond, _) => { + native_distinct_count_accumulator!(TimestampMillisecondType) + } + Timestamp(Nanosecond, _) => { + native_distinct_count_accumulator!(TimestampNanosecondType) + } + Timestamp(Second, _) => { + native_distinct_count_accumulator!(TimestampSecondType) + } + + Float16 => float_distinct_count_accumulator!(Float16Type), + Float32 => float_distinct_count_accumulator!(Float32Type), + Float64 => float_distinct_count_accumulator!(Float64Type), + + _ => Ok(Box::new(DistinctCountAccumulator { + values: HashSet::default(), + state_data_type: self.state_data_type.clone(), + })), + } } fn name(&self) -> &str { @@ -192,6 +262,182 @@ impl Accumulator for DistinctCountAccumulator { } } +#[derive(Debug)] +struct NativeDistinctCountAccumulator +where + T: ArrowPrimitiveType + Send, + T::Native: Eq + Hash, +{ + values: HashSet, +} + +impl NativeDistinctCountAccumulator +where + T: ArrowPrimitiveType + Send, + T::Native: Eq + Hash, +{ + fn new() -> Self { + Self { + values: HashSet::default(), + } + } +} + +impl Accumulator for NativeDistinctCountAccumulator +where + T: ArrowPrimitiveType + Send + Debug, + T::Native: Eq + Hash, +{ + fn state(&self) -> Result> { + let arr = Arc::new(PrimitiveArray::::from_iter_values( + self.values.iter().cloned(), + )) as ArrayRef; + let list = Arc::new(array_into_list_array(arr)) as ArrayRef; + Ok(vec![ScalarValue::List(list)]) + } + + fn update_batch(&mut self, values: &[ArrayRef]) -> Result<()> { + if values.is_empty() { + return Ok(()); + } + + let arr = as_primitive_array::(&values[0])?; + arr.iter().for_each(|value| { + if let Some(value) = value { + self.values.insert(value); + } + }); + + Ok(()) + } + + fn merge_batch(&mut self, states: &[ArrayRef]) -> Result<()> { + if states.is_empty() { + return Ok(()); + } + assert_eq!( + states.len(), + 1, + "count_distinct states must be single array" + ); + + let arr = as_list_array(&states[0])?; + arr.iter().try_for_each(|maybe_list| { + if let Some(list) = maybe_list { + let list = as_primitive_array::(&list)?; + self.values.extend(list.values()) + }; + Ok(()) + }) + } + + fn evaluate(&self) -> Result { + Ok(ScalarValue::Int64(Some(self.values.len() as i64))) + } + + fn size(&self) -> usize { + let estimated_buckets = (self.values.len().checked_mul(8).unwrap_or(usize::MAX) + / 7) + .next_power_of_two(); + + // Size of accumulator + // + size of entry * number of buckets + // + 1 byte for each bucket + // + fixed size of HashSet + std::mem::size_of_val(self) + + std::mem::size_of::() * estimated_buckets + + estimated_buckets + + std::mem::size_of_val(&self.values) + } +} + +#[derive(Debug)] +struct FloatDistinctCountAccumulator +where + T: ArrowPrimitiveType + Send, +{ + values: HashSet, RandomState>, +} + +impl FloatDistinctCountAccumulator +where + T: ArrowPrimitiveType + Send, +{ + fn new() -> Self { + Self { + values: HashSet::default(), + } + } +} + +impl Accumulator for FloatDistinctCountAccumulator +where + T: ArrowPrimitiveType + Send + Debug, +{ + fn state(&self) -> Result> { + let arr = Arc::new(PrimitiveArray::::from_iter_values( + self.values.iter().map(|v| v.0), + )) as ArrayRef; + let list = Arc::new(array_into_list_array(arr)) as ArrayRef; + Ok(vec![ScalarValue::List(list)]) + } + + fn update_batch(&mut self, values: &[ArrayRef]) -> Result<()> { + if values.is_empty() { + return Ok(()); + } + + let arr = as_primitive_array::(&values[0])?; + arr.iter().for_each(|value| { + if let Some(value) = value { + self.values.insert(Hashable(value)); + } + }); + + Ok(()) + } + + fn merge_batch(&mut self, states: &[ArrayRef]) -> Result<()> { + if states.is_empty() { + return Ok(()); + } + assert_eq!( + states.len(), + 1, + "count_distinct states must be single array" + ); + + let arr = as_list_array(&states[0])?; + arr.iter().try_for_each(|maybe_list| { + if let Some(list) = maybe_list { + let list = as_primitive_array::(&list)?; + self.values + .extend(list.values().iter().map(|v| Hashable(*v))); + }; + Ok(()) + }) + } + + fn evaluate(&self) -> Result { + Ok(ScalarValue::Int64(Some(self.values.len() as i64))) + } + + fn size(&self) -> usize { + let estimated_buckets = (self.values.len().checked_mul(8).unwrap_or(usize::MAX) + / 7) + .next_power_of_two(); + + // Size of accumulator + // + size of entry * number of buckets + // + 1 byte for each bucket + // + fixed size of HashSet + std::mem::size_of_val(self) + + std::mem::size_of::() * estimated_buckets + + estimated_buckets + + std::mem::size_of_val(&self.values) + } +} + #[cfg(test)] mod tests { use crate::expressions::NoOp; @@ -206,6 +452,8 @@ mod tests { Float32Type, Float64Type, Int16Type, Int32Type, Int64Type, Int8Type, UInt16Type, UInt32Type, UInt64Type, UInt8Type, }; + use arrow_array::Decimal256Array; + use arrow_buffer::i256; use datafusion_common::cast::{as_boolean_array, as_list_array, as_primitive_array}; use datafusion_common::internal_err; use datafusion_common::DataFusionError; @@ -367,6 +615,35 @@ mod tests { }}; } + macro_rules! test_count_distinct_update_batch_bigint { + ($ARRAY_TYPE:ident, $DATA_TYPE:ident, $PRIM_TYPE:ty) => {{ + let values: Vec> = vec![ + Some(i256::from(1)), + Some(i256::from(1)), + None, + Some(i256::from(3)), + Some(i256::from(2)), + None, + Some(i256::from(2)), + Some(i256::from(3)), + Some(i256::from(1)), + ]; + + let arrays = vec![Arc::new($ARRAY_TYPE::from(values)) as ArrayRef]; + + let (states, result) = run_update_batch(&arrays)?; + + let mut state_vec = state_to_vec_primitive!(&states[0], $DATA_TYPE); + state_vec.sort(); + + assert_eq!(states.len(), 1); + assert_eq!(state_vec, vec![i256::from(1), i256::from(2), i256::from(3)]); + assert_eq!(result, ScalarValue::Int64(Some(3))); + + Ok(()) + }}; + } + #[test] fn count_distinct_update_batch_i8() -> Result<()> { test_count_distinct_update_batch_numeric!(Int8Array, Int8Type, i8) @@ -417,6 +694,11 @@ mod tests { test_count_distinct_update_batch_floating_point!(Float64Array, Float64Type, f64) } + #[test] + fn count_distinct_update_batch_i256() -> Result<()> { + test_count_distinct_update_batch_bigint!(Decimal256Array, Decimal256Type, i256) + } + #[test] fn count_distinct_update_batch_boolean() -> Result<()> { let get_count = |data: BooleanArray| -> Result<(Vec, i64)> { diff --git a/datafusion/physical-expr/src/aggregate/sum_distinct.rs b/datafusion/physical-expr/src/aggregate/sum_distinct.rs index 0cf4a90ab8cc..6dbb39224629 100644 --- a/datafusion/physical-expr/src/aggregate/sum_distinct.rs +++ b/datafusion/physical-expr/src/aggregate/sum_distinct.rs @@ -25,11 +25,11 @@ use arrow::array::{Array, ArrayRef}; use arrow_array::cast::AsArray; use arrow_array::types::*; use arrow_array::{ArrowNativeTypeOp, ArrowPrimitiveType}; -use arrow_buffer::{ArrowNativeType, ToByteSlice}; +use arrow_buffer::ArrowNativeType; use std::collections::HashSet; use crate::aggregate::sum::downcast_sum; -use crate::aggregate::utils::down_cast_any_ref; +use crate::aggregate::utils::{down_cast_any_ref, Hashable}; use crate::{AggregateExpr, PhysicalExpr}; use datafusion_common::{not_impl_err, DataFusionError, Result, ScalarValue}; use datafusion_expr::type_coercion::aggregates::sum_return_type; @@ -119,24 +119,6 @@ impl PartialEq for DistinctSum { } } -/// A wrapper around a type to provide hash for floats -#[derive(Copy, Clone)] -struct Hashable(T); - -impl std::hash::Hash for Hashable { - fn hash(&self, state: &mut H) { - self.0.to_byte_slice().hash(state) - } -} - -impl PartialEq for Hashable { - fn eq(&self, other: &Self) -> bool { - self.0.is_eq(other.0) - } -} - -impl Eq for Hashable {} - struct DistinctSumAccumulator { values: HashSet, RandomState>, data_type: DataType, diff --git a/datafusion/physical-expr/src/aggregate/utils.rs b/datafusion/physical-expr/src/aggregate/utils.rs index 9777158da133..d73c46a0f687 100644 --- a/datafusion/physical-expr/src/aggregate/utils.rs +++ b/datafusion/physical-expr/src/aggregate/utils.rs @@ -28,7 +28,7 @@ use arrow_array::types::{ Decimal128Type, DecimalType, TimestampMicrosecondType, TimestampMillisecondType, TimestampNanosecondType, TimestampSecondType, }; -use arrow_buffer::ArrowNativeType; +use arrow_buffer::{ArrowNativeType, ToByteSlice}; use arrow_schema::{DataType, Field, SortOptions}; use datafusion_common::{exec_err, DataFusionError, Result}; use datafusion_expr::Accumulator; @@ -211,3 +211,21 @@ pub(crate) fn ordering_fields( pub fn get_sort_options(ordering_req: &[PhysicalSortExpr]) -> Vec { ordering_req.iter().map(|item| item.options).collect() } + +/// A wrapper around a type to provide hash for floats +#[derive(Copy, Clone, Debug)] +pub(crate) struct Hashable(pub T); + +impl std::hash::Hash for Hashable { + fn hash(&self, state: &mut H) { + self.0.to_byte_slice().hash(state) + } +} + +impl PartialEq for Hashable { + fn eq(&self, other: &Self) -> bool { + self.0.is_eq(other.0) + } +} + +impl Eq for Hashable {} From 05e3d456ea5d8bf537b68787f67c8aa0af9e448a Mon Sep 17 00:00:00 2001 From: Mustafa Akur <106137913+mustafasrepo@users.noreply.github.com> Date: Fri, 5 Jan 2024 15:20:40 +0300 Subject: [PATCH 550/572] Add test case for unnecessary hash when target is 1 (#8757) --- .../enforce_distribution.rs | 3 +- datafusion/sqllogictest/test_files/join.slt | 32 +++++++++++++++++++ 2 files changed, 33 insertions(+), 2 deletions(-) diff --git a/datafusion/core/src/physical_optimizer/enforce_distribution.rs b/datafusion/core/src/physical_optimizer/enforce_distribution.rs index bf5aa7d02272..1c86c4c3205a 100644 --- a/datafusion/core/src/physical_optimizer/enforce_distribution.rs +++ b/datafusion/core/src/physical_optimizer/enforce_distribution.rs @@ -927,9 +927,8 @@ fn add_hash_on_top( n_target: usize, repartition_beneficial_stats: bool, ) -> Result { - let partition_count = input.plan.output_partitioning().partition_count(); // Early return if hash repartition is unnecessary - if n_target == partition_count && n_target == 1 { + if n_target == 1 { return Ok(input); } diff --git a/datafusion/sqllogictest/test_files/join.slt b/datafusion/sqllogictest/test_files/join.slt index c9dd7ca604ad..ca9b918ff3ee 100644 --- a/datafusion/sqllogictest/test_files/join.slt +++ b/datafusion/sqllogictest/test_files/join.slt @@ -626,6 +626,38 @@ Alice 100 Alice 1 Alice 50 Alice 2 Alice 100 Alice 2 +statement ok +set datafusion.execution.target_partitions = 1; + +statement ok +set datafusion.optimizer.repartition_joins = true; + +# make sure when target partition is 1, hash repartition is not added +# to the final plan. +query TT +EXPLAIN SELECT * +FROM t1, +t1 as t2 +WHERE t1.a=t2.a; +---- +logical_plan +Inner Join: t1.a = t2.a +--TableScan: t1 projection=[a, b] +--SubqueryAlias: t2 +----TableScan: t1 projection=[a, b] +physical_plan +CoalesceBatchesExec: target_batch_size=8192 +--HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(a@0, a@0)] +----MemoryExec: partitions=1, partition_sizes=[1] +----MemoryExec: partitions=1, partition_sizes=[1] + +# Reset the configs to old values +statement ok +set datafusion.execution.target_partitions = 4; + +statement ok +set datafusion.optimizer.repartition_joins = false; + statement ok DROP TABLE t1; From 4173070be5df2959bdb85abd57e097062ab128d4 Mon Sep 17 00:00:00 2001 From: Andrew Lamb Date: Fri, 5 Jan 2024 10:29:24 -0500 Subject: [PATCH 551/572] Minor: Improve `PruningPredicate` docstrings more (#8748) --- .../core/src/physical_optimizer/pruning.rs | 77 ++++++++++++------- 1 file changed, 48 insertions(+), 29 deletions(-) diff --git a/datafusion/core/src/physical_optimizer/pruning.rs b/datafusion/core/src/physical_optimizer/pruning.rs index 1b68553a896f..0cbbaf2bf6cd 100644 --- a/datafusion/core/src/physical_optimizer/pruning.rs +++ b/datafusion/core/src/physical_optimizer/pruning.rs @@ -45,12 +45,23 @@ use datafusion_physical_expr::utils::{collect_columns, Guarantee, LiteralGuarant use datafusion_physical_expr::{expressions as phys_expr, PhysicalExprRef}; use log::trace; -/// Interface to pass statistics (min/max/nulls) information to [`PruningPredicate`]. +/// A source of runtime statistical information to [`PruningPredicate`]s. /// -/// Returns statistics for containers / files as Arrow [`ArrayRef`], so the -/// evaluation happens once on a single `RecordBatch`, amortizing the overhead -/// of evaluating of the predicate. This is important when pruning 1000s of -/// containers which often happens in analytic systems. +/// # Supported Information +/// +/// 1. Minimum and maximum values for columns +/// +/// 2. Null counts for columns +/// +/// 3. Whether the values in a column are contained in a set of literals +/// +/// # Vectorized Interface +/// +/// Information for containers / files are returned as Arrow [`ArrayRef`], so +/// the evaluation happens once on a single `RecordBatch`, which amortizes the +/// overhead of evaluating the predicate. This is important when pruning 1000s +/// of containers which often happens in analytic systems that have 1000s of +/// potential files to consider. /// /// For example, for the following three files with a single column `a`: /// ```text @@ -83,8 +94,11 @@ pub trait PruningStatistics { /// Note: the returned array must contain [`Self::num_containers`] rows fn max_values(&self, column: &Column) -> Option; - /// Return the number of containers (e.g. row groups) being - /// pruned with these statistics (the number of rows in each returned array) + /// Return the number of containers (e.g. Row Groups) being pruned with + /// these statistics. + /// + /// This value corresponds to the size of the [`ArrayRef`] returned by + /// [`Self::min_values`], [`Self::max_values`], and [`Self::null_counts`]. fn num_containers(&self) -> usize; /// Return the number of null values for the named column as an @@ -95,13 +109,11 @@ pub trait PruningStatistics { /// Note: the returned array must contain [`Self::num_containers`] rows fn null_counts(&self, column: &Column) -> Option; - /// Returns an array where each row represents information known about - /// the `values` contained in a column. + /// Returns [`BooleanArray`] where each row represents information known + /// about specific literal `values` in a column. /// - /// This API is designed to be used along with [`LiteralGuarantee`] to prove - /// that predicates can not possibly evaluate to `true` and thus prune - /// containers. For example, Parquet Bloom Filters can prove that values are - /// not present. + /// For example, Parquet Bloom Filters implement this API to communicate + /// that `values` are known not to be present in a Row Group. /// /// The returned array has one row for each container, with the following /// meanings: @@ -120,28 +132,34 @@ pub trait PruningStatistics { ) -> Option; } -/// Evaluates filter expressions on statistics such as min/max values and null -/// counts, attempting to prove a "container" (e.g. Parquet Row Group) can be -/// skipped without reading the actual data, potentially leading to significant -/// performance improvements. +/// Used to prove that arbitrary predicates (boolean expression) can not +/// possibly evaluate to `true` given information about a column provided by +/// [`PruningStatistics`]. +/// +/// `PruningPredicate` analyzes filter expressions using statistics such as +/// min/max values and null counts, attempting to prove a "container" (e.g. +/// Parquet Row Group) can be skipped without reading the actual data, +/// potentially leading to significant performance improvements. +/// +/// For example, `PruningPredicate`s are used to prune Parquet Row Groups based +/// on the min/max values found in the Parquet metadata. If the +/// `PruningPredicate` can prove that the filter can never evaluate to `true` +/// for any row in the Row Group, the entire Row Group is skipped during query +/// execution. /// -/// For example, [`PruningPredicate`]s are used to prune Parquet Row Groups -/// based on the min/max values found in the Parquet metadata. If the -/// `PruningPredicate` can guarantee that no rows in the Row Group match the -/// filter, the entire Row Group is skipped during query execution. +/// The `PruningPredicate` API is designed to be general, so it can used for +/// pruning other types of containers (e.g. files) based on statistics that may +/// be known from external catalogs (e.g. Delta Lake) or other sources. /// -/// The `PruningPredicate` API is general, allowing it to be used for pruning -/// other types of containers (e.g. files) based on statistics that may be -/// known from external catalogs (e.g. Delta Lake) or other sources. Thus it -/// supports: +/// It currently supports: /// -/// 1. Arbitrary expressions expressions (including user defined functions) +/// 1. Arbitrary expressions (including user defined functions) /// /// 2. Vectorized evaluation (provide more than one set of statistics at a time) /// so it is suitable for pruning 1000s of containers. /// -/// 3. Anything that implements the [`PruningStatistics`] trait, not just -/// Parquet metadata. +/// 3. Any source of information that implements the [`PruningStatistics`] trait +/// (not just Parquet metadata). /// /// # Example /// @@ -154,7 +172,8 @@ pub trait PruningStatistics { /// C: {x_min = 5, x_max = 8} /// ``` /// -/// Applying the `PruningPredicate` will concludes that `A` can be pruned: +/// `PruningPredicate` will conclude that the rows in container `A` can never +/// be true (as the maximum value is only `4`), so it can be pruned: /// /// ```text /// A: false (no rows could possibly match x = 5) From af20d2d874c359031dfb4f6828a9a55891ffae06 Mon Sep 17 00:00:00 2001 From: Alex Huang Date: Sat, 6 Jan 2024 00:55:26 +0800 Subject: [PATCH 552/572] feat: support `LargeList` in `cardinality` (#8726) * support LargeList in cardinality --- .../physical-expr/src/array_expressions.rs | 21 ++++++++++++--- datafusion/sqllogictest/test_files/array.slt | 26 +++++++++++++++++++ 2 files changed, 44 insertions(+), 3 deletions(-) diff --git a/datafusion/physical-expr/src/array_expressions.rs b/datafusion/physical-expr/src/array_expressions.rs index 15330af640ae..78c490d5dba8 100644 --- a/datafusion/physical-expr/src/array_expressions.rs +++ b/datafusion/physical-expr/src/array_expressions.rs @@ -2114,16 +2114,31 @@ pub fn cardinality(args: &[ArrayRef]) -> Result { return exec_err!("cardinality expects one argument"); } - let list_array = as_list_array(&args[0])?.clone(); + match &args[0].data_type() { + DataType::List(_) => { + let list_array = as_list_array(&args[0])?; + generic_list_cardinality::(list_array) + } + DataType::LargeList(_) => { + let list_array = as_large_list_array(&args[0])?; + generic_list_cardinality::(list_array) + } + other => { + exec_err!("cardinality does not support type '{:?}'", other) + } + } +} - let result = list_array +fn generic_list_cardinality( + array: &GenericListArray, +) -> Result { + let result = array .iter() .map(|arr| match compute_array_dims(arr)? { Some(vector) => Ok(Some(vector.iter().map(|x| x.unwrap()).product::())), None => Ok(None), }) .collect::>()?; - Ok(Arc::new(result) as ArrayRef) } diff --git a/datafusion/sqllogictest/test_files/array.slt b/datafusion/sqllogictest/test_files/array.slt index 7cee615a5729..774d67b4fde2 100644 --- a/datafusion/sqllogictest/test_files/array.slt +++ b/datafusion/sqllogictest/test_files/array.slt @@ -3320,18 +3320,33 @@ select cardinality(make_array(1, 2, 3, 4, 5)), cardinality([1, 3, 5]), cardinali ---- 5 3 5 +query III +select cardinality(arrow_cast(make_array(1, 2, 3, 4, 5), 'LargeList(Int64)')), cardinality(arrow_cast([1, 3, 5], 'LargeList(Int64)')), cardinality(arrow_cast(make_array('h', 'e', 'l', 'l', 'o'), 'LargeList(Utf8)')); +---- +5 3 5 + # cardinality scalar function #2 query II select cardinality(make_array([1, 2], [3, 4], [5, 6])), cardinality(array_repeat(array_repeat(array_repeat(3, 3), 2), 3)); ---- 6 18 +query I +select cardinality(arrow_cast(make_array([1, 2], [3, 4], [5, 6]), 'LargeList(List(Int64))')); +---- +6 + # cardinality scalar function #3 query II select cardinality(make_array()), cardinality(make_array(make_array())) ---- NULL 0 +query II +select cardinality(arrow_cast(make_array(), 'LargeList(Null)')), cardinality(arrow_cast(make_array(make_array()), 'LargeList(List(Null))')) +---- +NULL 0 + # cardinality with columns query III select cardinality(column1), cardinality(column2), cardinality(column3) from arrays; @@ -3344,6 +3359,17 @@ NULL 3 4 4 NULL 1 4 3 NULL +query III +select cardinality(column1), cardinality(column2), cardinality(column3) from large_arrays; +---- +4 3 5 +4 3 5 +4 3 5 +4 3 3 +NULL 3 4 +4 NULL 1 +4 3 NULL + ## array_remove (aliases: `list_remove`) # array_remove scalar function #1 From 93b4a4cc42d87497b53a300c9ab587fb6e88ce78 Mon Sep 17 00:00:00 2001 From: Andrew Lamb Date: Fri, 5 Jan 2024 13:39:51 -0500 Subject: [PATCH 553/572] Add reproducer for #8738 (#8750) --- .../sqllogictest/test_files/dictionary.slt | 203 ++++++++++++++++++ 1 file changed, 203 insertions(+) create mode 100644 datafusion/sqllogictest/test_files/dictionary.slt diff --git a/datafusion/sqllogictest/test_files/dictionary.slt b/datafusion/sqllogictest/test_files/dictionary.slt new file mode 100644 index 000000000000..d4ad46711b9f --- /dev/null +++ b/datafusion/sqllogictest/test_files/dictionary.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. + +# Tests for querying on dictionary encoded data + +# Note: These tables model data as is common for timeseries, such as in InfluxDB IOx +# There are three types of columns: +# 1. tag columns, which are string dictionaries, often with low cardinality +# 2. field columns, which are typed, +# 3. a `time` columns, which is a nanosecond timestamp + +# It is common to group and filter on the "tag" columns (and thus on dictionary +# encoded values) + +# Table m1 with a tag column `tag_id` 4 fields `f1` - `f4`, and `time` + +statement ok +CREATE VIEW m1 AS +SELECT + arrow_cast(column1, 'Dictionary(Int32, Utf8)') as tag_id, + arrow_cast(column2, 'Float64') as f1, + arrow_cast(column3, 'Utf8') as f2, + arrow_cast(column4, 'Utf8') as f3, + arrow_cast(column5, 'Float64') as f4, + arrow_cast(column6, 'Timestamp(Nanosecond, None)') as time +FROM ( + VALUES + -- equivalent to the following line protocol data + -- m1,tag_id=1000 f1=32,f2="foo",f3="True",f4=1.0 1703030400000000000 + -- m1,tag_id=1000 f1=32,f2="foo",f3="True",f4=2.0 1703031000000000000 + -- m1,tag_id=1000 f1=32,f2="foo",f3="True",f4=3.0 1703031600000000000 + -- m1,tag_id=1000 f1=32,f2="foo",f3="True",f4=4.0 1703032200000000000 + -- m1,tag_id=1000 f1=32,f2="foo",f3="True",f4=5.0 1703032800000000000 + -- m1,tag_id=1000 f1=32,f2="foo",f3="True",f4=6.0 1703033400000000000 + -- m1,tag_id=1000 f1=32,f2="foo",f3="True",f4=7.0 1703034000000000000 + -- m1,tag_id=1000 f1=32,f2="foo",f3="True",f4=8.0 1703034600000000000 + -- m1,tag_id=1000 f1=32,f2="foo",f3="True",f4=9.0 1703035200000000000 + -- m1,tag_id=1000 f1=32,f2="foo",f3="True",f4=10.0 1703035800000000000 + ('1000', 32, 'foo', 'True', 1.0, 1703030400000000000), + ('1000', 32, 'foo', 'True', 2.0, 1703031000000000000), + ('1000', 32, 'foo', 'True', 3.0, 1703031600000000000), + ('1000', 32, 'foo', 'True', 4.0, 1703032200000000000), + ('1000', 32, 'foo', 'True', 5.0, 1703032800000000000), + ('1000', 32, 'foo', 'True', 6.0, 1703033400000000000), + ('1000', 32, 'foo', 'True', 7.0, 1703034000000000000), + ('1000', 32, 'foo', 'True', 8.0, 1703034600000000000), + ('1000', 32, 'foo', 'True', 9.0, 1703035200000000000), + ('1000', 32, 'foo', 'True', 10.0, 1703035800000000000) +); + +query ?RTTRP +SELECT * FROM m1; +---- +1000 32 foo True 1 2023-12-20T00:00:00 +1000 32 foo True 2 2023-12-20T00:10:00 +1000 32 foo True 3 2023-12-20T00:20:00 +1000 32 foo True 4 2023-12-20T00:30:00 +1000 32 foo True 5 2023-12-20T00:40:00 +1000 32 foo True 6 2023-12-20T00:50:00 +1000 32 foo True 7 2023-12-20T01:00:00 +1000 32 foo True 8 2023-12-20T01:10:00 +1000 32 foo True 9 2023-12-20T01:20:00 +1000 32 foo True 10 2023-12-20T01:30:00 + +# Note that te type of the tag column is `Dictionary(Int32, Utf8)` +query TTT +DESCRIBE m1; +---- +tag_id Dictionary(Int32, Utf8) YES +f1 Float64 YES +f2 Utf8 YES +f3 Utf8 YES +f4 Float64 YES +time Timestamp(Nanosecond, None) YES + + +# Table m2 with a tag columns `tag_id` and `type`, a field column `f5`, and `time` +statement ok +CREATE VIEW m2 AS +SELECT + arrow_cast(column1, 'Dictionary(Int32, Utf8)') as type, + arrow_cast(column2, 'Dictionary(Int32, Utf8)') as tag_id, + arrow_cast(column3, 'Float64') as f5, + arrow_cast(column4, 'Timestamp(Nanosecond, None)') as time +FROM ( + VALUES + -- equivalent to the following line protocol data + -- m2,type=active,tag_id=1000 f5=100 1701648000000000000 + -- m2,type=active,tag_id=1000 f5=200 1701648600000000000 + -- m2,type=active,tag_id=1000 f5=300 1701649200000000000 + -- m2,type=active,tag_id=1000 f5=400 1701649800000000000 + -- m2,type=active,tag_id=1000 f5=500 1701650400000000000 + -- m2,type=active,tag_id=1000 f5=600 1701651000000000000 + -- m2,type=passive,tag_id=2000 f5=700 1701651600000000000 + -- m2,type=passive,tag_id=1000 f5=800 1701652200000000000 + -- m2,type=passive,tag_id=1000 f5=900 1701652800000000000 + -- m2,type=passive,tag_id=1000 f5=1000 1701653400000000000 + ('active', '1000', 100, 1701648000000000000), + ('active', '1000', 200, 1701648600000000000), + ('active', '1000', 300, 1701649200000000000), + ('active', '1000', 400, 1701649800000000000), + ('active', '1000', 500, 1701650400000000000), + ('active', '1000', 600, 1701651000000000000), + ('passive', '1000', 700, 1701651600000000000), + ('passive', '1000', 800, 1701652200000000000), + ('passive', '1000', 900, 1701652800000000000), + ('passive', '1000', 1000, 1701653400000000000) +); + +query ??RP +SELECT * FROM m2; +---- +active 1000 100 2023-12-04T00:00:00 +active 1000 200 2023-12-04T00:10:00 +active 1000 300 2023-12-04T00:20:00 +active 1000 400 2023-12-04T00:30:00 +active 1000 500 2023-12-04T00:40:00 +active 1000 600 2023-12-04T00:50:00 +passive 1000 700 2023-12-04T01:00:00 +passive 1000 800 2023-12-04T01:10:00 +passive 1000 900 2023-12-04T01:20:00 +passive 1000 1000 2023-12-04T01:30:00 + +query TTT +DESCRIBE m2; +---- +type Dictionary(Int32, Utf8) YES +tag_id Dictionary(Int32, Utf8) YES +f5 Float64 YES +time Timestamp(Nanosecond, None) YES + +query I +select count(*) from m1 where tag_id = '1000' and time < '2024-01-03T14:46:35+01:00'; +---- +10 + +query RRR +select min(f5), max(f5), avg(f5) from m2 where tag_id = '1000' and time < '2024-01-03T14:46:35+01:00' group by type; +---- +100 600 350 +700 1000 850 + +query IRRRP +select count(*), min(f5), max(f5), avg(f5), date_bin('30 minutes', time) as "time" +from m2 where tag_id = '1000' and time < '2024-01-03T14:46:35+01:00' +group by date_bin('30 minutes', time) +order by date_bin('30 minutes', time) DESC +---- +1 1000 1000 1000 2023-12-04T01:30:00 +3 700 900 800 2023-12-04T01:00:00 +3 400 600 500 2023-12-04T00:30:00 +3 100 300 200 2023-12-04T00:00:00 + + + +# Reproducer for https://github.com/apache/arrow-datafusion/issues/8738 +# This query should work correctly +query error DataFusion error: External error: Arrow error: Invalid argument error: RowConverter column schema mismatch, expected Utf8 got Dictionary\(Int32, Utf8\) +SELECT + "data"."timestamp" as "time", + "data"."tag_id", + "data"."field", + "data"."value" +FROM ( + ( + SELECT "m2"."time" as "timestamp", "m2"."tag_id", 'active_power' as "field", "m2"."f5" as "value" + FROM "m2" + WHERE "m2"."time" >= '2023-12-05T14:46:35+01:00' AND "m2"."time" < '2024-01-03T14:46:35+01:00' + AND "m2"."f5" IS NOT NULL + AND "m2"."type" IN ('active') + AND "m2"."tag_id" IN ('1000') + ) UNION ( + SELECT "m1"."time" as "timestamp", "m1"."tag_id", 'f1' as "field", "m1"."f1" as "value" + FROM "m1" + WHERE "m1"."time" >= '2023-12-05T14:46:35+01:00' AND "m1"."time" < '2024-01-03T14:46:35+01:00' + AND "m1"."f1" IS NOT NULL + AND "m1"."tag_id" IN ('1000') + ) UNION ( + SELECT "m1"."time" as "timestamp", "m1"."tag_id", 'f2' as "field", "m1"."f2" as "value" + FROM "m1" + WHERE "m1"."time" >= '2023-12-05T14:46:35+01:00' AND "m1"."time" < '2024-01-03T14:46:35+01:00' + AND "m1"."f2" IS NOT NULL + AND "m1"."tag_id" IN ('1000') + ) +) as "data" +ORDER BY + "time", + "data"."tag_id" +; From 6e7529734ccb7ddc86129b8c6b490d51ec7054d9 Mon Sep 17 00:00:00 2001 From: Matthew Turner Date: Fri, 5 Jan 2024 15:36:41 -0500 Subject: [PATCH 554/572] Update checking columns in schema merge (#8765) --- datafusion/common/src/dfschema.rs | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/datafusion/common/src/dfschema.rs b/datafusion/common/src/dfschema.rs index d6e4490cec4c..8772c4c3e1ff 100644 --- a/datafusion/common/src/dfschema.rs +++ b/datafusion/common/src/dfschema.rs @@ -230,9 +230,9 @@ impl DFSchema { for field in other_schema.fields() { // skip duplicate columns let duplicated_field = match field.qualifier() { - Some(q) => self.field_with_name(Some(q), field.name()).is_ok(), + Some(q) => self.has_column_with_qualified_name(q, field.name()), // for unqualified columns, check as unqualified name - None => self.field_with_unqualified_name(field.name()).is_ok(), + None => self.has_column_with_unqualified_name(field.name()), }; if !duplicated_field { self.fields.push(field.clone()); From 0208755751d5a389a55e9bf505d3b42c85ebedba Mon Sep 17 00:00:00 2001 From: Andrew Lamb Date: Fri, 5 Jan 2024 15:37:51 -0500 Subject: [PATCH 555/572] Minor: Add documentation about stream cancellation (#8747) --- datafusion/core/src/dataframe/mod.rs | 11 ++++++++++ datafusion/physical-plan/src/lib.rs | 33 ++++++++++++++++++++++++++-- 2 files changed, 42 insertions(+), 2 deletions(-) diff --git a/datafusion/core/src/dataframe/mod.rs b/datafusion/core/src/dataframe/mod.rs index 5a8c706e32cd..33e198d6d588 100644 --- a/datafusion/core/src/dataframe/mod.rs +++ b/datafusion/core/src/dataframe/mod.rs @@ -802,6 +802,7 @@ impl DataFrame { /// Executes this DataFrame and returns a stream over a single partition /// + /// # Example /// ``` /// # use datafusion::prelude::*; /// # use datafusion::error::Result; @@ -813,6 +814,11 @@ impl DataFrame { /// # Ok(()) /// # } /// ``` + /// + /// # Aborting Execution + /// + /// Dropping the stream will abort the execution of the query, and free up + /// any allocated resources pub async fn execute_stream(self) -> Result { let task_ctx = Arc::new(self.task_ctx()); let plan = self.create_physical_plan().await?; @@ -841,6 +847,7 @@ impl DataFrame { /// Executes this DataFrame and returns one stream per partition. /// + /// # Example /// ``` /// # use datafusion::prelude::*; /// # use datafusion::error::Result; @@ -852,6 +859,10 @@ impl DataFrame { /// # Ok(()) /// # } /// ``` + /// # Aborting Execution + /// + /// Dropping the stream will abort the execution of the query, and free up + /// any allocated resources pub async fn execute_stream_partitioned( self, ) -> Result> { diff --git a/datafusion/physical-plan/src/lib.rs b/datafusion/physical-plan/src/lib.rs index 6c9e97e03cb7..1dd1392b9d86 100644 --- a/datafusion/physical-plan/src/lib.rs +++ b/datafusion/physical-plan/src/lib.rs @@ -288,6 +288,24 @@ pub trait ExecutionPlan: Debug + DisplayAs + Send + Sync { /// [`TryStreamExt`]: futures::stream::TryStreamExt /// [`RecordBatchStreamAdapter`]: crate::stream::RecordBatchStreamAdapter /// + /// # Cancellation / Aborting Execution + /// + /// The [`Stream`] that is returned must ensure that any allocated resources + /// are freed when the stream itself is dropped. This is particularly + /// important for [`spawn`]ed tasks or threads. Unless care is taken to + /// "abort" such tasks, they may continue to consume resources even after + /// the plan is dropped, generating intermediate results that are never + /// used. + /// + /// See [`AbortOnDropSingle`], [`AbortOnDropMany`] and + /// [`RecordBatchReceiverStreamBuilder`] for structures to help ensure all + /// background tasks are cancelled. + /// + /// [`spawn`]: tokio::task::spawn + /// [`AbortOnDropSingle`]: crate::common::AbortOnDropSingle + /// [`AbortOnDropMany`]: crate::common::AbortOnDropMany + /// [`RecordBatchReceiverStreamBuilder`]: crate::stream::RecordBatchReceiverStreamBuilder + /// /// # Implementation Examples /// /// While `async` `Stream`s have a non trivial learning curve, the @@ -491,7 +509,12 @@ pub async fn collect( common::collect(stream).await } -/// Execute the [ExecutionPlan] and return a single stream of results +/// Execute the [ExecutionPlan] and return a single stream of results. +/// +/// # Aborting Execution +/// +/// Dropping the stream will abort the execution of the query, and free up +/// any allocated resources pub fn execute_stream( plan: Arc, context: Arc, @@ -549,7 +572,13 @@ pub async fn collect_partitioned( Ok(batches) } -/// Execute the [ExecutionPlan] and return a vec with one stream per output partition +/// Execute the [ExecutionPlan] and return a vec with one stream per output +/// partition +/// +/// # Aborting Execution +/// +/// Dropping the stream will abort the execution of the query, and free up +/// any allocated resources pub fn execute_stream_partitioned( plan: Arc, context: Arc, From 29f23ebffb8a95f6663d23a3a2a0fb8e87765f7b Mon Sep 17 00:00:00 2001 From: Liang-Chi Hsieh Date: Fri, 5 Jan 2024 12:38:56 -0800 Subject: [PATCH 556/572] Move `repartition_file_scans` out of `enable_round_robin` check in `EnforceDistribution` rule (#8731) * Cleanup * More * Restore add_roundrobin_on_top * Restore test files * More * Restore * More * More * Make test stable * For review * Add test --- .../enforce_distribution.rs | 42 ++++++++----------- .../sqllogictest/test_files/arrow_typeof.slt | 2 +- .../test_files/repartition_scan.slt | 24 ++++++++++- .../sqllogictest/test_files/timestamps.slt | 4 +- .../sqllogictest/test_files/tpch/q2.slt.part | 2 +- datafusion/sqllogictest/test_files/window.slt | 2 +- 6 files changed, 45 insertions(+), 31 deletions(-) diff --git a/datafusion/core/src/physical_optimizer/enforce_distribution.rs b/datafusion/core/src/physical_optimizer/enforce_distribution.rs index 1c86c4c3205a..bf3f9ef0f3e6 100644 --- a/datafusion/core/src/physical_optimizer/enforce_distribution.rs +++ b/datafusion/core/src/physical_optimizer/enforce_distribution.rs @@ -1197,32 +1197,33 @@ fn ensure_distribution( ) .map( |(mut child, requirement, required_input_ordering, would_benefit, maintains)| { - // Don't need to apply when the returned row count is not greater than 1: + // Don't need to apply when the returned row count is not greater than batch size let num_rows = child.plan.statistics()?.num_rows; let repartition_beneficial_stats = if num_rows.is_exact().unwrap_or(false) { num_rows .get_value() .map(|value| value > &batch_size) - .unwrap_or(true) + .unwrap() // safe to unwrap since is_exact() is true } else { true }; + // When `repartition_file_scans` is set, attempt to increase + // parallelism at the source. + if repartition_file_scans && repartition_beneficial_stats { + if let Some(new_child) = + child.plan.repartitioned(target_partitions, config)? + { + child.plan = new_child; + } + } + if enable_round_robin // Operator benefits from partitioning (e.g. filter): && (would_benefit && repartition_beneficial_stats) // Unless partitioning doesn't increase the partition count, it is not beneficial: && child.plan.output_partitioning().partition_count() < target_partitions { - // When `repartition_file_scans` is set, attempt to increase - // parallelism at the source. - if repartition_file_scans { - if let Some(new_child) = - child.plan.repartitioned(target_partitions, config)? - { - child.plan = new_child; - } - } // Increase parallelism by adding round-robin repartitioning // on top of the operator. Note that we only do this if the // partition count is not already equal to the desired partition @@ -1361,17 +1362,10 @@ impl DistributionContext { fn update_children(mut self) -> Result { for child_context in self.children_nodes.iter_mut() { - child_context.distribution_connection = match child_context.plan.as_any() { - plan_any if plan_any.is::() => matches!( - plan_any - .downcast_ref::() - .unwrap() - .partitioning(), - Partitioning::RoundRobinBatch(_) | Partitioning::Hash(_, _) - ), - plan_any - if plan_any.is::() - || plan_any.is::() => + child_context.distribution_connection = match &child_context.plan { + plan if is_repartition(plan) + || is_coalesce_partitions(plan) + || is_sort_preserving_merge(plan) => { true } @@ -3870,14 +3864,14 @@ pub(crate) mod tests { "RepartitionExec: partitioning=Hash([a@0], 2), input_partitions=2", "AggregateExec: mode=Partial, gby=[a@0 as a], aggr=[]", // Plan already has two partitions - "ParquetExec: file_groups={2 groups: [[x], [y]]}, projection=[a, b, c, d, e]", + "ParquetExec: file_groups={2 groups: [[x:0..100], [y:0..100]]}, projection=[a, b, c, d, e]", ]; let expected_csv = [ "AggregateExec: mode=FinalPartitioned, gby=[a@0 as a], aggr=[]", "RepartitionExec: partitioning=Hash([a@0], 2), input_partitions=2", "AggregateExec: mode=Partial, gby=[a@0 as a], aggr=[]", // Plan already has two partitions - "CsvExec: file_groups={2 groups: [[x], [y]]}, projection=[a, b, c, d, e], has_header=false", + "CsvExec: file_groups={2 groups: [[x:0..100], [y:0..100]]}, projection=[a, b, c, d, e], has_header=false", ]; assert_optimized!(expected_parquet, plan_parquet, true, false, 2, true, 10); diff --git a/datafusion/sqllogictest/test_files/arrow_typeof.slt b/datafusion/sqllogictest/test_files/arrow_typeof.slt index 3fad4d0f61b9..6a623e6c92f9 100644 --- a/datafusion/sqllogictest/test_files/arrow_typeof.slt +++ b/datafusion/sqllogictest/test_files/arrow_typeof.slt @@ -375,4 +375,4 @@ select arrow_cast(make_array(1, 2, 3), 'LargeList(Int64)'); query T select arrow_typeof(arrow_cast(make_array(1, 2, 3), 'LargeList(Int64)')); ---- -LargeList(Field { name: "item", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }) \ No newline at end of file +LargeList(Field { name: "item", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }) diff --git a/datafusion/sqllogictest/test_files/repartition_scan.slt b/datafusion/sqllogictest/test_files/repartition_scan.slt index 02eccd7c5d06..9d4951c7ecac 100644 --- a/datafusion/sqllogictest/test_files/repartition_scan.slt +++ b/datafusion/sqllogictest/test_files/repartition_scan.slt @@ -63,6 +63,26 @@ CoalesceBatchesExec: target_batch_size=8192 --FilterExec: column1@0 != 42 ----ParquetExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/parquet_table/2.parquet:0..101], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/parquet_table/2.parquet:101..202], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/parquet_table/2.parquet:202..303], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/parquet_table/2.parquet:303..403]]}, projection=[column1], predicate=column1@0 != 42, pruning_predicate=column1_min@0 != 42 OR 42 != column1_max@1 +# disable round robin repartitioning +statement ok +set datafusion.optimizer.enable_round_robin_repartition = false; + +## Expect to see the scan read the file as "4" groups with even sizes (offsets) again +query TT +EXPLAIN SELECT column1 FROM parquet_table WHERE column1 <> 42; +---- +logical_plan +Filter: parquet_table.column1 != Int32(42) +--TableScan: parquet_table projection=[column1], partial_filters=[parquet_table.column1 != Int32(42)] +physical_plan +CoalesceBatchesExec: target_batch_size=8192 +--FilterExec: column1@0 != 42 +----ParquetExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/parquet_table/2.parquet:0..101], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/parquet_table/2.parquet:101..202], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/parquet_table/2.parquet:202..303], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/parquet_table/2.parquet:303..403]]}, projection=[column1], predicate=column1@0 != 42, pruning_predicate=column1_min@0 != 42 OR 42 != column1_max@1 + +# enable round robin repartitioning again +statement ok +set datafusion.optimizer.enable_round_robin_repartition = true; + # create a second parquet file statement ok COPY (VALUES (100), (200)) TO 'test_files/scratch/repartition_scan/parquet_table/1.parquet' @@ -147,7 +167,7 @@ WITH HEADER ROW LOCATION 'test_files/scratch/repartition_scan/csv_table/'; query I -select * from csv_table; +select * from csv_table ORDER BY column1; ---- 1 2 @@ -190,7 +210,7 @@ STORED AS json LOCATION 'test_files/scratch/repartition_scan/json_table/'; query I -select * from "json_table"; +select * from "json_table" ORDER BY column1; ---- 1 2 diff --git a/datafusion/sqllogictest/test_files/timestamps.slt b/datafusion/sqllogictest/test_files/timestamps.slt index c84e46c965fa..8b0f50cedf05 100644 --- a/datafusion/sqllogictest/test_files/timestamps.slt +++ b/datafusion/sqllogictest/test_files/timestamps.slt @@ -1862,7 +1862,7 @@ SELECT to_timestamp(null) is null as c1, ---- true true true true true true true true true true true true true -# verify timestamp output types +# verify timestamp output types query TTT SELECT arrow_typeof(to_timestamp(1)), arrow_typeof(to_timestamp(null)), arrow_typeof(to_timestamp('2023-01-10 12:34:56.000')) ---- @@ -1880,7 +1880,7 @@ SELECT arrow_typeof(to_timestamp(1)) = arrow_typeof(1::timestamp) as c1, true true true true true true # known issues. currently overflows (expects default precision to be microsecond instead of nanoseconds. Work pending) -#verify extreme values +#verify extreme values #query PPPPPPPP #SELECT to_timestamp(-62125747200), to_timestamp(1926632005177), -62125747200::timestamp, 1926632005177::timestamp, cast(-62125747200 as timestamp), cast(1926632005177 as timestamp) #---- diff --git a/datafusion/sqllogictest/test_files/tpch/q2.slt.part b/datafusion/sqllogictest/test_files/tpch/q2.slt.part index ed439348d22d..ed950db190bb 100644 --- a/datafusion/sqllogictest/test_files/tpch/q2.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/q2.slt.part @@ -238,7 +238,7 @@ order by p_partkey limit 10; ---- -9828.21 Supplier#000000647 UNITED KINGDOM 13120 Manufacturer#5 x5U7MBZmwfG9 33-258-202-4782 s the slyly even ideas poach fluffily +9828.21 Supplier#000000647 UNITED KINGDOM 13120 Manufacturer#5 x5U7MBZmwfG9 33-258-202-4782 s the slyly even ideas poach fluffily 9508.37 Supplier#000000070 FRANCE 3563 Manufacturer#1 INWNH2w,OOWgNDq0BRCcBwOMQc6PdFDc4 16-821-608-1166 ests sleep quickly express ideas. ironic ideas haggle about the final T 9508.37 Supplier#000000070 FRANCE 17268 Manufacturer#4 INWNH2w,OOWgNDq0BRCcBwOMQc6PdFDc4 16-821-608-1166 ests sleep quickly express ideas. ironic ideas haggle about the final T 9453.01 Supplier#000000802 ROMANIA 10021 Manufacturer#5 ,6HYXb4uaHITmtMBj4Ak57Pd 29-342-882-6463 gular frets. permanently special multipliers believe blithely alongs diff --git a/datafusion/sqllogictest/test_files/window.slt b/datafusion/sqllogictest/test_files/window.slt index 7d6d59201396..100c2143837a 100644 --- a/datafusion/sqllogictest/test_files/window.slt +++ b/datafusion/sqllogictest/test_files/window.slt @@ -3794,7 +3794,7 @@ select a, 1 1 2 1 -# support scalar value in ORDER BY +# support scalar value in ORDER BY query I select rank() over (order by 1) rnk from (select 1 a union all select 2 a) x ---- From 98f02ffec5e51861185be298ff1981eef5f04958 Mon Sep 17 00:00:00 2001 From: junxiangMu <63799833+guojidan@users.noreply.github.com> Date: Sat, 6 Jan 2024 05:06:31 +0800 Subject: [PATCH 557/572] Clean internal implementation of WindowUDF (#8746) * Clean internal implementation of WindowUDF * fix doc --- datafusion-examples/examples/advanced_udwf.rs | 1 + .../user_defined_window_functions.rs | 1 + datafusion/expr/src/expr_fn.rs | 12 ++ datafusion/expr/src/udwf.rs | 138 +++++++++++------- .../tests/cases/roundtrip_logical_plan.rs | 1 + 5 files changed, 103 insertions(+), 50 deletions(-) diff --git a/datafusion-examples/examples/advanced_udwf.rs b/datafusion-examples/examples/advanced_udwf.rs index 91869d80a41a..f46031434fc9 100644 --- a/datafusion-examples/examples/advanced_udwf.rs +++ b/datafusion-examples/examples/advanced_udwf.rs @@ -34,6 +34,7 @@ use datafusion_expr::{ /// a function `partition_evaluator` that returns the `MyPartitionEvaluator` instance. /// /// To do so, we must implement the `WindowUDFImpl` trait. +#[derive(Debug, Clone)] struct SmoothItUdf { signature: Signature, } diff --git a/datafusion/core/tests/user_defined/user_defined_window_functions.rs b/datafusion/core/tests/user_defined/user_defined_window_functions.rs index 3040fbafe81a..54eab4315a97 100644 --- a/datafusion/core/tests/user_defined/user_defined_window_functions.rs +++ b/datafusion/core/tests/user_defined/user_defined_window_functions.rs @@ -471,6 +471,7 @@ impl OddCounter { } fn register(ctx: &mut SessionContext, test_state: Arc) { + #[derive(Debug, Clone)] struct SimpleWindowUDF { signature: Signature, return_type: DataType, diff --git a/datafusion/expr/src/expr_fn.rs b/datafusion/expr/src/expr_fn.rs index f76fb17b38bb..7b3f65248586 100644 --- a/datafusion/expr/src/expr_fn.rs +++ b/datafusion/expr/src/expr_fn.rs @@ -32,6 +32,7 @@ use crate::{ColumnarValue, ScalarUDFImpl, WindowUDF, WindowUDFImpl}; use arrow::datatypes::DataType; use datafusion_common::{Column, Result}; use std::any::Any; +use std::fmt::Debug; use std::ops::Not; use std::sync::Arc; @@ -1078,6 +1079,17 @@ pub struct SimpleWindowUDF { partition_evaluator_factory: PartitionEvaluatorFactory, } +impl Debug for SimpleWindowUDF { + fn fmt(&self, f: &mut std::fmt::Formatter) -> std::fmt::Result { + f.debug_struct("WindowUDF") + .field("name", &self.name) + .field("signature", &self.signature) + .field("return_type", &"") + .field("partition_evaluator_factory", &"") + .finish() + } +} + impl SimpleWindowUDF { /// Create a new `SimpleWindowUDF` from a name, input types, return type and /// implementation. Implementing [`WindowUDFImpl`] allows more flexibility diff --git a/datafusion/expr/src/udwf.rs b/datafusion/expr/src/udwf.rs index 800386bfc77b..239a5e24cbf2 100644 --- a/datafusion/expr/src/udwf.rs +++ b/datafusion/expr/src/udwf.rs @@ -34,40 +34,33 @@ use std::{ /// /// See the documetnation on [`PartitionEvaluator`] for more details /// +/// 1. For simple (less performant) use cases, use [`create_udwf`] and [`simple_udwf.rs`]. +/// +/// 2. For advanced use cases, use [`WindowUDFImpl`] and [`advanced_udf.rs`]. +/// +/// # API Note +/// This is a separate struct from `WindowUDFImpl` to maintain backwards +/// compatibility with the older API. +/// /// [`PartitionEvaluator`]: crate::PartitionEvaluator -#[derive(Clone)] +/// [`create_udwf`]: crate::expr_fn::create_udwf +/// [`simple_udwf.rs`]: https://github.com/apache/arrow-datafusion/blob/main/datafusion-examples/examples/simple_udwf.rs +/// [`advanced_udwf.rs`]: https://github.com/apache/arrow-datafusion/blob/main/datafusion-examples/examples/advanced_udwf.rs +#[derive(Debug, Clone)] pub struct WindowUDF { - /// name - name: String, - /// signature - signature: Signature, - /// Return type - return_type: ReturnTypeFunction, - /// Return the partition evaluator - partition_evaluator_factory: PartitionEvaluatorFactory, -} - -impl Debug for WindowUDF { - fn fmt(&self, f: &mut Formatter) -> fmt::Result { - f.debug_struct("WindowUDF") - .field("name", &self.name) - .field("signature", &self.signature) - .field("return_type", &"") - .field("partition_evaluator_factory", &"") - .finish_non_exhaustive() - } + inner: Arc, } /// Defines how the WindowUDF is shown to users impl Display for WindowUDF { fn fmt(&self, f: &mut Formatter) -> fmt::Result { - write!(f, "{}", self.name) + write!(f, "{}", self.name()) } } impl PartialEq for WindowUDF { fn eq(&self, other: &Self) -> bool { - self.name == other.name && self.signature == other.signature + self.name() == other.name() && self.signature() == other.signature() } } @@ -75,8 +68,8 @@ impl Eq for WindowUDF {} impl std::hash::Hash for WindowUDF { fn hash(&self, state: &mut H) { - self.name.hash(state); - self.signature.hash(state); + self.name().hash(state); + self.signature().hash(state); } } @@ -92,12 +85,12 @@ impl WindowUDF { return_type: &ReturnTypeFunction, partition_evaluator_factory: &PartitionEvaluatorFactory, ) -> Self { - Self { - name: name.to_string(), + Self::new_from_impl(WindowUDFLegacyWrapper { + name: name.to_owned(), signature: signature.clone(), return_type: return_type.clone(), partition_evaluator_factory: partition_evaluator_factory.clone(), - } + }) } /// Create a new `WindowUDF` from a `[WindowUDFImpl]` trait object @@ -105,27 +98,18 @@ impl WindowUDF { /// Note this is the same as using the `From` impl (`WindowUDF::from`) pub fn new_from_impl(fun: F) -> WindowUDF where - F: WindowUDFImpl + Send + Sync + 'static, + F: WindowUDFImpl + 'static, { - let arc_fun = Arc::new(fun); - let captured_self = arc_fun.clone(); - let return_type: ReturnTypeFunction = Arc::new(move |arg_types| { - let return_type = captured_self.return_type(arg_types)?; - Ok(Arc::new(return_type)) - }); - - let captured_self = arc_fun.clone(); - let partition_evaluator_factory: PartitionEvaluatorFactory = - Arc::new(move || captured_self.partition_evaluator()); - Self { - name: arc_fun.name().to_string(), - signature: arc_fun.signature().clone(), - return_type: return_type.clone(), - partition_evaluator_factory, + inner: Arc::new(fun), } } + /// Return the underlying [`WindowUDFImpl`] trait object for this function + pub fn inner(&self) -> Arc { + self.inner.clone() + } + /// creates a [`Expr`] that calls the window function given /// the `partition_by`, `order_by`, and `window_frame` definition /// @@ -150,25 +134,29 @@ impl WindowUDF { } /// Returns this function's name + /// + /// See [`WindowUDFImpl::name`] for more details. pub fn name(&self) -> &str { - &self.name + self.inner.name() } /// Returns this function's signature (what input types are accepted) + /// + /// See [`WindowUDFImpl::signature`] for more details. pub fn signature(&self) -> &Signature { - &self.signature + self.inner.signature() } /// Return the type of the function given its input types + /// + /// See [`WindowUDFImpl::return_type`] for more details. pub fn return_type(&self, args: &[DataType]) -> Result { - // Old API returns an Arc of the datatype for some reason - let res = (self.return_type)(args)?; - Ok(res.as_ref().clone()) + self.inner.return_type(args) } /// Return a `PartitionEvaluator` for evaluating this window function pub fn partition_evaluator_factory(&self) -> Result> { - (self.partition_evaluator_factory)() + self.inner.partition_evaluator() } } @@ -198,6 +186,7 @@ where /// # use datafusion_common::{DataFusionError, plan_err, Result}; /// # use datafusion_expr::{col, Signature, Volatility, PartitionEvaluator, WindowFrame}; /// # use datafusion_expr::{WindowUDFImpl, WindowUDF}; +/// #[derive(Debug, Clone)] /// struct SmoothIt { /// signature: Signature /// }; @@ -236,7 +225,7 @@ where /// WindowFrame::new(false), /// ); /// ``` -pub trait WindowUDFImpl { +pub trait WindowUDFImpl: Debug + Send + Sync { /// Returns this object as an [`Any`] trait object fn as_any(&self) -> &dyn Any; @@ -254,3 +243,52 @@ pub trait WindowUDFImpl { /// Invoke the function, returning the [`PartitionEvaluator`] instance fn partition_evaluator(&self) -> Result>; } + +/// Implementation of [`WindowUDFImpl`] that wraps the function style pointers +/// of the older API (see +/// for more details) +pub struct WindowUDFLegacyWrapper { + /// name + name: String, + /// signature + signature: Signature, + /// Return type + return_type: ReturnTypeFunction, + /// Return the partition evaluator + partition_evaluator_factory: PartitionEvaluatorFactory, +} + +impl Debug for WindowUDFLegacyWrapper { + fn fmt(&self, f: &mut Formatter) -> fmt::Result { + f.debug_struct("WindowUDF") + .field("name", &self.name) + .field("signature", &self.signature) + .field("return_type", &"") + .field("partition_evaluator_factory", &"") + .finish_non_exhaustive() + } +} + +impl WindowUDFImpl for WindowUDFLegacyWrapper { + fn as_any(&self) -> &dyn Any { + self + } + + fn name(&self) -> &str { + &self.name + } + + fn signature(&self) -> &Signature { + &self.signature + } + + fn return_type(&self, arg_types: &[DataType]) -> Result { + // Old API returns an Arc of the datatype for some reason + let res = (self.return_type)(arg_types)?; + Ok(res.as_ref().clone()) + } + + fn partition_evaluator(&self) -> Result> { + (self.partition_evaluator_factory)() + } +} diff --git a/datafusion/proto/tests/cases/roundtrip_logical_plan.rs b/datafusion/proto/tests/cases/roundtrip_logical_plan.rs index 402781e17e6f..03daf535f201 100644 --- a/datafusion/proto/tests/cases/roundtrip_logical_plan.rs +++ b/datafusion/proto/tests/cases/roundtrip_logical_plan.rs @@ -1787,6 +1787,7 @@ fn roundtrip_window() { } } + #[derive(Debug, Clone)] struct SimpleWindowUDF { signature: Signature, } From b2e8848406b0884fbc631e130c3e25e35550a1a1 Mon Sep 17 00:00:00 2001 From: Alex Huang Date: Sat, 6 Jan 2024 05:09:24 +0800 Subject: [PATCH 558/572] feat: support `largelist` in `array_to_string` (#8729) * support largelist in array_to_string * reduce code duplication --- .../physical-expr/src/array_expressions.rs | 94 +++++++++++++------ datafusion/sqllogictest/test_files/array.slt | 61 ++++++++++++ 2 files changed, 128 insertions(+), 27 deletions(-) diff --git a/datafusion/physical-expr/src/array_expressions.rs b/datafusion/physical-expr/src/array_expressions.rs index 78c490d5dba8..cb4ad3ed63fe 100644 --- a/datafusion/physical-expr/src/array_expressions.rs +++ b/datafusion/physical-expr/src/array_expressions.rs @@ -2022,8 +2022,21 @@ pub fn array_to_string(args: &[ArrayRef]) -> Result { ) -> Result<&mut String> { match arr.data_type() { DataType::List(..) => { - let list_array = downcast_arg!(arr, ListArray); + let list_array = as_list_array(&arr)?; + for i in 0..list_array.len() { + compute_array_to_string( + arg, + list_array.value(i), + delimiter.clone(), + null_string.clone(), + with_null_string, + )?; + } + Ok(arg) + } + DataType::LargeList(..) => { + let list_array = as_large_list_array(&arr)?; for i in 0..list_array.len() { compute_array_to_string( arg, @@ -2055,35 +2068,61 @@ pub fn array_to_string(args: &[ArrayRef]) -> Result { } } - let mut arg = String::from(""); - let mut res: Vec> = Vec::new(); - - match arr.data_type() { - DataType::List(_) | DataType::LargeList(_) | DataType::FixedSizeList(_, _) => { - let list_array = arr.as_list::(); - for (arr, &delimiter) in list_array.iter().zip(delimiters.iter()) { - if let (Some(arr), Some(delimiter)) = (arr, delimiter) { - arg = String::from(""); - let s = compute_array_to_string( - &mut arg, - arr, - delimiter.to_string(), - null_string.clone(), - with_null_string, - )? - .clone(); - - if let Some(s) = s.strip_suffix(delimiter) { - res.push(Some(s.to_string())); - } else { - res.push(Some(s)); - } + fn generate_string_array( + list_arr: &GenericListArray, + delimiters: Vec>, + null_string: String, + with_null_string: bool, + ) -> Result { + let mut res: Vec> = Vec::new(); + for (arr, &delimiter) in list_arr.iter().zip(delimiters.iter()) { + if let (Some(arr), Some(delimiter)) = (arr, delimiter) { + let mut arg = String::from(""); + let s = compute_array_to_string( + &mut arg, + arr, + delimiter.to_string(), + null_string.clone(), + with_null_string, + )? + .clone(); + + if let Some(s) = s.strip_suffix(delimiter) { + res.push(Some(s.to_string())); } else { - res.push(None); + res.push(Some(s)); } + } else { + res.push(None); } } + + Ok(StringArray::from(res)) + } + + let arr_type = arr.data_type(); + let string_arr = match arr_type { + DataType::List(_) | DataType::FixedSizeList(_, _) => { + let list_array = as_list_array(&arr)?; + generate_string_array::( + list_array, + delimiters, + null_string, + with_null_string, + )? + } + DataType::LargeList(_) => { + let list_array = as_large_list_array(&arr)?; + generate_string_array::( + list_array, + delimiters, + null_string, + with_null_string, + )? + } _ => { + let mut arg = String::from(""); + let mut res: Vec> = Vec::new(); // delimiter length is 1 assert_eq!(delimiters.len(), 1); let delimiter = delimiters[0].unwrap(); @@ -2102,10 +2141,11 @@ pub fn array_to_string(args: &[ArrayRef]) -> Result { } else { res.push(Some(s)); } + StringArray::from(res) } - } + }; - Ok(Arc::new(StringArray::from(res))) + Ok(Arc::new(string_arr)) } /// Cardinality SQL function diff --git a/datafusion/sqllogictest/test_files/array.slt b/datafusion/sqllogictest/test_files/array.slt index 774d67b4fde2..083c4ff31b8f 100644 --- a/datafusion/sqllogictest/test_files/array.slt +++ b/datafusion/sqllogictest/test_files/array.slt @@ -3238,30 +3238,55 @@ select list_to_string(['h', 'e', 'l', 'l', 'o'], ','), list_to_string([1, 2, 3, ---- h,e,l,l,o 1-2-3-4-5 1|2|3 +query TTT +select list_to_string(arrow_cast(['h', 'e', 'l', 'l', 'o'], 'LargeList(Utf8)'), ','), list_to_string(arrow_cast([1, 2, 3, 4, 5], 'LargeList(Int64)'), '-'), list_to_string(arrow_cast([1.0, 2.0, 3.0], 'LargeList(Float64)'), '|'); +---- +h,e,l,l,o 1-2-3-4-5 1|2|3 + # array_join scalar function #5 (function alias `array_to_string`) query TTT select array_join(['h', 'e', 'l', 'l', 'o'], ','), array_join([1, 2, 3, 4, 5], '-'), array_join([1.0, 2.0, 3.0], '|'); ---- h,e,l,l,o 1-2-3-4-5 1|2|3 +query TTT +select array_join(arrow_cast(['h', 'e', 'l', 'l', 'o'], 'LargeList(Utf8)'), ','), array_join(arrow_cast([1, 2, 3, 4, 5], 'LargeList(Int64)'), '-'), array_join(arrow_cast([1.0, 2.0, 3.0], 'LargeList(Float64)'), '|'); +---- +h,e,l,l,o 1-2-3-4-5 1|2|3 + # list_join scalar function #6 (function alias `list_join`) query TTT select list_join(['h', 'e', 'l', 'l', 'o'], ','), list_join([1, 2, 3, 4, 5], '-'), list_join([1.0, 2.0, 3.0], '|'); ---- h,e,l,l,o 1-2-3-4-5 1|2|3 +query TTT +select list_join(arrow_cast(['h', 'e', 'l', 'l', 'o'], 'LargeList(Utf8)'), ','), list_join(arrow_cast([1, 2, 3, 4, 5], 'LargeList(Int64)'), '-'), list_join(arrow_cast([1.0, 2.0, 3.0], 'LargeList(Float64)'), '|'); +---- +h,e,l,l,o 1-2-3-4-5 1|2|3 + # array_to_string scalar function with nulls #1 query TTT select array_to_string(make_array('h', NULL, 'l', NULL, 'o'), ','), array_to_string(make_array(1, NULL, 3, NULL, 5), '-'), array_to_string(make_array(NULL, 2.0, 3.0), '|'); ---- h,l,o 1-3-5 2|3 +query TTT +select array_to_string(arrow_cast(['h', 'e', 'l', 'l', 'o'], 'LargeList(Utf8)'), ','), array_to_string(arrow_cast([1, 2, 3, 4, 5], 'LargeList(Int64)'), '-'), array_to_string(arrow_cast([1.0, 2.0, 3.0], 'LargeList(Float64)'), '|'); +---- +h,e,l,l,o 1-2-3-4-5 1|2|3 + # array_to_string scalar function with nulls #2 query TTT select array_to_string(make_array('h', NULL, NULL, NULL, 'o'), ',', '-'), array_to_string(make_array(NULL, 2, NULL, 4, 5), '-', 'nil'), array_to_string(make_array(1.0, NULL, 3.0), '|', '0'); ---- h,-,-,-,o nil-2-nil-4-5 1|0|3 +query TTT +select array_to_string(arrow_cast(make_array('h', NULL, NULL, NULL, 'o'), 'LargeList(Utf8)'), ',', '-'), array_to_string(arrow_cast(make_array(NULL, 2, NULL, 4, 5), 'LargeList(Int64)'), '-', 'nil'), array_to_string(arrow_cast(make_array(1.0, NULL, 3.0), 'LargeList(Float64)'), '|', '0'); +---- +h,-,-,-,o nil-2-nil-4-5 1|0|3 + # array_to_string with columns #1 # For reference @@ -3288,6 +3313,18 @@ NULL 51^52^54^55^56^57^58^59^60 NULL +query T +select array_to_string(column1, column4) from large_arrays_values; +---- +2,3,4,5,6,7,8,9,10 +11.12.13.14.15.16.17.18.20 +21-22-23-25-26-27-28-29-30 +31ok32ok33ok34ok35ok37ok38ok39ok40 +NULL +41$42$43$44$45$46$47$48$49$50 +51^52^54^55^56^57^58^59^60 +NULL + query TT select array_to_string(column1, '_'), array_to_string(make_array(1,2,3), '/') from arrays_values; ---- @@ -3300,6 +3337,18 @@ NULL 1/2/3 51_52_54_55_56_57_58_59_60 1/2/3 61_62_63_64_65_66_67_68_69_70 1/2/3 +query TT +select array_to_string(column1, '_'), array_to_string(make_array(1,2,3), '/') from large_arrays_values; +---- +2_3_4_5_6_7_8_9_10 1/2/3 +11_12_13_14_15_16_17_18_20 1/2/3 +21_22_23_25_26_27_28_29_30 1/2/3 +31_32_33_34_35_37_38_39_40 1/2/3 +NULL 1/2/3 +41_42_43_44_45_46_47_48_49_50 1/2/3 +51_52_54_55_56_57_58_59_60 1/2/3 +61_62_63_64_65_66_67_68_69_70 1/2/3 + query TT select array_to_string(column1, '_', '*'), array_to_string(make_array(make_array(1,2,3)), '.') from arrays_values; ---- @@ -3312,6 +3361,18 @@ NULL 1.2.3 51_52_*_54_55_56_57_58_59_60 1.2.3 61_62_63_64_65_66_67_68_69_70 1.2.3 +query TT +select array_to_string(column1, '_', '*'), array_to_string(make_array(make_array(1,2,3)), '.') from large_arrays_values; +---- +*_2_3_4_5_6_7_8_9_10 1.2.3 +11_12_13_14_15_16_17_18_*_20 1.2.3 +21_22_23_*_25_26_27_28_29_30 1.2.3 +31_32_33_34_35_*_37_38_39_40 1.2.3 +NULL 1.2.3 +41_42_43_44_45_46_47_48_49_50 1.2.3 +51_52_*_54_55_56_57_58_59_60 1.2.3 +61_62_63_64_65_66_67_68_69_70 1.2.3 + ## cardinality # cardinality scalar function From 821db545be48b39d8522310bf08bc4b0bb28d4bf Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Metehan=20Y=C4=B1ld=C4=B1r=C4=B1m?= <100111937+metesynnada@users.noreply.github.com> Date: Sat, 6 Jan 2024 00:17:04 +0300 Subject: [PATCH 559/572] Error handling. (#8761) --- datafusion-cli/src/print_options.rs | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/datafusion-cli/src/print_options.rs b/datafusion-cli/src/print_options.rs index b8594352b585..b382eb34f62c 100644 --- a/datafusion-cli/src/print_options.rs +++ b/datafusion-cli/src/print_options.rs @@ -141,7 +141,8 @@ impl PrintOptions { let mut row_count = 0_usize; let mut with_header = true; - while let Some(Ok(batch)) = stream.next().await { + while let Some(maybe_batch) = stream.next().await { + let batch = maybe_batch?; row_count += batch.num_rows(); self.format.print_batches( &mut writer, From 4e4059a68455fbc14f04902c76acbcd258b7f2ef Mon Sep 17 00:00:00 2001 From: Jay Zhan Date: Sat, 6 Jan 2024 06:14:55 +0800 Subject: [PATCH 560/572] Convert Binary Operator `StringConcat` to Function for `array_concat`, `array_append` and `array_prepend` (#8636) * reuse function for string concat Signed-off-by: jayzhan211 * remove casting in string concat Signed-off-by: jayzhan211 * add test Signed-off-by: jayzhan211 * operator to function rewrite Signed-off-by: jayzhan211 * fix explain Signed-off-by: jayzhan211 * add more test Signed-off-by: jayzhan211 * add column cases Signed-off-by: jayzhan211 * cleanup Signed-off-by: jayzhan211 * presever name Signed-off-by: jayzhan211 * Update datafusion/optimizer/src/analyzer/rewrite_expr.rs Co-authored-by: Andrew Lamb * rename Signed-off-by: jayzhan211 --------- Signed-off-by: jayzhan211 Co-authored-by: Andrew Lamb --- datafusion/expr/src/type_coercion/binary.rs | 2 - datafusion/optimizer/src/analyzer/mod.rs | 6 + .../optimizer/src/analyzer/rewrite_expr.rs | 321 ++++++++++++++++++ .../physical-expr/src/expressions/binary.rs | 11 +- datafusion/sql/src/expr/mod.rs | 2 + datafusion/sqllogictest/test_files/array.slt | 39 +++ .../sqllogictest/test_files/explain.slt | 1 + 7 files changed, 371 insertions(+), 11 deletions(-) create mode 100644 datafusion/optimizer/src/analyzer/rewrite_expr.rs diff --git a/datafusion/expr/src/type_coercion/binary.rs b/datafusion/expr/src/type_coercion/binary.rs index 1b62c1bc05c1..6bacc1870079 100644 --- a/datafusion/expr/src/type_coercion/binary.rs +++ b/datafusion/expr/src/type_coercion/binary.rs @@ -667,8 +667,6 @@ fn string_concat_coercion(lhs_type: &DataType, rhs_type: &DataType) -> Option { string_concat_internal_coercion(from_type, &LargeUtf8) } - // TODO: cast between array elements (#6558) - (List(_), from_type) | (from_type, List(_)) => Some(from_type.to_owned()), _ => None, }) } diff --git a/datafusion/optimizer/src/analyzer/mod.rs b/datafusion/optimizer/src/analyzer/mod.rs index 14d5ddf47378..9d47299a5616 100644 --- a/datafusion/optimizer/src/analyzer/mod.rs +++ b/datafusion/optimizer/src/analyzer/mod.rs @@ -17,6 +17,7 @@ pub mod count_wildcard_rule; pub mod inline_table_scan; +pub mod rewrite_expr; pub mod subquery; pub mod type_coercion; @@ -37,6 +38,8 @@ use log::debug; use std::sync::Arc; use std::time::Instant; +use self::rewrite_expr::OperatorToFunction; + /// [`AnalyzerRule`]s transform [`LogicalPlan`]s in some way to make /// the plan valid prior to the rest of the DataFusion optimization process. /// @@ -72,6 +75,9 @@ impl Analyzer { pub fn new() -> Self { let rules: Vec> = vec![ Arc::new(InlineTableScan::new()), + // OperatorToFunction should be run before TypeCoercion, since it rewrite based on the argument types (List or Scalar), + // and TypeCoercion may cast the argument types from Scalar to List. + Arc::new(OperatorToFunction::new()), Arc::new(TypeCoercion::new()), Arc::new(CountWildcardRule::new()), ]; diff --git a/datafusion/optimizer/src/analyzer/rewrite_expr.rs b/datafusion/optimizer/src/analyzer/rewrite_expr.rs new file mode 100644 index 000000000000..8f1c844ed062 --- /dev/null +++ b/datafusion/optimizer/src/analyzer/rewrite_expr.rs @@ -0,0 +1,321 @@ +// 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. + +//! Analyzer rule for to replace operators with function calls (e.g `||` to array_concat`) + +use std::sync::Arc; + +use datafusion_common::config::ConfigOptions; +use datafusion_common::tree_node::TreeNodeRewriter; +use datafusion_common::utils::list_ndims; +use datafusion_common::DFSchema; +use datafusion_common::DFSchemaRef; +use datafusion_common::Result; +use datafusion_expr::expr::ScalarFunction; +use datafusion_expr::expr_rewriter::rewrite_preserving_name; +use datafusion_expr::utils::merge_schema; +use datafusion_expr::BuiltinScalarFunction; +use datafusion_expr::Operator; +use datafusion_expr::ScalarFunctionDefinition; +use datafusion_expr::{BinaryExpr, Expr, LogicalPlan}; + +use super::AnalyzerRule; + +#[derive(Default)] +pub struct OperatorToFunction {} + +impl OperatorToFunction { + pub fn new() -> Self { + Self {} + } +} + +impl AnalyzerRule for OperatorToFunction { + fn name(&self) -> &str { + "operator_to_function" + } + + fn analyze(&self, plan: LogicalPlan, _: &ConfigOptions) -> Result { + analyze_internal(&plan) + } +} + +fn analyze_internal(plan: &LogicalPlan) -> Result { + // optimize child plans first + let new_inputs = plan + .inputs() + .iter() + .map(|p| analyze_internal(p)) + .collect::>>()?; + + // get schema representing all available input fields. This is used for data type + // resolution only, so order does not matter here + let mut schema = merge_schema(new_inputs.iter().collect()); + + if let LogicalPlan::TableScan(ts) = plan { + let source_schema = + DFSchema::try_from_qualified_schema(&ts.table_name, &ts.source.schema())?; + schema.merge(&source_schema); + } + + let mut expr_rewrite = OperatorToFunctionRewriter { + schema: Arc::new(schema), + }; + + let new_expr = plan + .expressions() + .into_iter() + .map(|expr| { + // ensure names don't change: + // https://github.com/apache/arrow-datafusion/issues/3555 + rewrite_preserving_name(expr, &mut expr_rewrite) + }) + .collect::>>()?; + + plan.with_new_exprs(new_expr, &new_inputs) +} + +pub(crate) struct OperatorToFunctionRewriter { + pub(crate) schema: DFSchemaRef, +} + +impl TreeNodeRewriter for OperatorToFunctionRewriter { + type N = Expr; + + fn mutate(&mut self, expr: Expr) -> Result { + match expr { + Expr::BinaryExpr(BinaryExpr { + ref left, + op, + ref right, + }) => { + if let Some(fun) = rewrite_array_concat_operator_to_func_for_column( + left.as_ref(), + op, + right.as_ref(), + self.schema.as_ref(), + )? + .or_else(|| { + rewrite_array_concat_operator_to_func( + left.as_ref(), + op, + right.as_ref(), + ) + }) { + // Convert &Box -> Expr + let left = (**left).clone(); + let right = (**right).clone(); + return Ok(Expr::ScalarFunction(ScalarFunction { + func_def: ScalarFunctionDefinition::BuiltIn(fun), + args: vec![left, right], + })); + } + + Ok(expr) + } + _ => Ok(expr), + } + } +} + +/// Summary of the logic below: +/// +/// 1) array || array -> array concat +/// +/// 2) array || scalar -> array append +/// +/// 3) scalar || array -> array prepend +/// +/// 4) (arry concat, array append, array prepend) || array -> array concat +/// +/// 5) (arry concat, array append, array prepend) || scalar -> array append +fn rewrite_array_concat_operator_to_func( + left: &Expr, + op: Operator, + right: &Expr, +) -> Option { + // Convert `Array StringConcat Array` to ScalarFunction::ArrayConcat + + if op != Operator::StringConcat { + return None; + } + + match (left, right) { + // Chain concat operator (a || b) || array, + // (arry concat, array append, array prepend) || array -> array concat + ( + Expr::ScalarFunction(ScalarFunction { + func_def: + ScalarFunctionDefinition::BuiltIn(BuiltinScalarFunction::ArrayConcat), + args: _left_args, + }), + Expr::ScalarFunction(ScalarFunction { + func_def: + ScalarFunctionDefinition::BuiltIn(BuiltinScalarFunction::MakeArray), + args: _right_args, + }), + ) + | ( + Expr::ScalarFunction(ScalarFunction { + func_def: + ScalarFunctionDefinition::BuiltIn(BuiltinScalarFunction::ArrayAppend), + args: _left_args, + }), + Expr::ScalarFunction(ScalarFunction { + func_def: + ScalarFunctionDefinition::BuiltIn(BuiltinScalarFunction::MakeArray), + args: _right_args, + }), + ) + | ( + Expr::ScalarFunction(ScalarFunction { + func_def: + ScalarFunctionDefinition::BuiltIn(BuiltinScalarFunction::ArrayPrepend), + args: _left_args, + }), + Expr::ScalarFunction(ScalarFunction { + func_def: + ScalarFunctionDefinition::BuiltIn(BuiltinScalarFunction::MakeArray), + args: _right_args, + }), + ) => Some(BuiltinScalarFunction::ArrayConcat), + // Chain concat operator (a || b) || scalar, + // (arry concat, array append, array prepend) || scalar -> array append + ( + Expr::ScalarFunction(ScalarFunction { + func_def: + ScalarFunctionDefinition::BuiltIn(BuiltinScalarFunction::ArrayConcat), + args: _left_args, + }), + _scalar, + ) + | ( + Expr::ScalarFunction(ScalarFunction { + func_def: + ScalarFunctionDefinition::BuiltIn(BuiltinScalarFunction::ArrayAppend), + args: _left_args, + }), + _scalar, + ) + | ( + Expr::ScalarFunction(ScalarFunction { + func_def: + ScalarFunctionDefinition::BuiltIn(BuiltinScalarFunction::ArrayPrepend), + args: _left_args, + }), + _scalar, + ) => Some(BuiltinScalarFunction::ArrayAppend), + // array || array -> array concat + ( + Expr::ScalarFunction(ScalarFunction { + func_def: + ScalarFunctionDefinition::BuiltIn(BuiltinScalarFunction::MakeArray), + args: _left_args, + }), + Expr::ScalarFunction(ScalarFunction { + func_def: + ScalarFunctionDefinition::BuiltIn(BuiltinScalarFunction::MakeArray), + args: _right_args, + }), + ) => Some(BuiltinScalarFunction::ArrayConcat), + // array || scalar -> array append + ( + Expr::ScalarFunction(ScalarFunction { + func_def: + ScalarFunctionDefinition::BuiltIn(BuiltinScalarFunction::MakeArray), + args: _left_args, + }), + _right_scalar, + ) => Some(BuiltinScalarFunction::ArrayAppend), + // scalar || array -> array prepend + ( + _left_scalar, + Expr::ScalarFunction(ScalarFunction { + func_def: + ScalarFunctionDefinition::BuiltIn(BuiltinScalarFunction::MakeArray), + args: _right_args, + }), + ) => Some(BuiltinScalarFunction::ArrayPrepend), + + _ => None, + } +} + +/// Summary of the logic below: +/// +/// 1) (arry concat, array append, array prepend) || column -> (array append, array concat) +/// +/// 2) column1 || column2 -> (array prepend, array append, array concat) +fn rewrite_array_concat_operator_to_func_for_column( + left: &Expr, + op: Operator, + right: &Expr, + schema: &DFSchema, +) -> Result> { + if op != Operator::StringConcat { + return Ok(None); + } + + match (left, right) { + // Column cases: + // 1) array_prepend/append/concat || column + ( + Expr::ScalarFunction(ScalarFunction { + func_def: + ScalarFunctionDefinition::BuiltIn(BuiltinScalarFunction::ArrayPrepend), + args: _left_args, + }), + Expr::Column(c), + ) + | ( + Expr::ScalarFunction(ScalarFunction { + func_def: + ScalarFunctionDefinition::BuiltIn(BuiltinScalarFunction::ArrayAppend), + args: _left_args, + }), + Expr::Column(c), + ) + | ( + Expr::ScalarFunction(ScalarFunction { + func_def: + ScalarFunctionDefinition::BuiltIn(BuiltinScalarFunction::ArrayConcat), + args: _left_args, + }), + Expr::Column(c), + ) => { + let d = schema.field_from_column(c)?.data_type(); + let ndim = list_ndims(d); + match ndim { + 0 => Ok(Some(BuiltinScalarFunction::ArrayAppend)), + _ => Ok(Some(BuiltinScalarFunction::ArrayConcat)), + } + } + // 2) select column1 || column2 + (Expr::Column(c1), Expr::Column(c2)) => { + let d1 = schema.field_from_column(c1)?.data_type(); + let d2 = schema.field_from_column(c2)?.data_type(); + let ndim1 = list_ndims(d1); + let ndim2 = list_ndims(d2); + match (ndim1, ndim2) { + (0, _) => Ok(Some(BuiltinScalarFunction::ArrayPrepend)), + (_, 0) => Ok(Some(BuiltinScalarFunction::ArrayAppend)), + _ => Ok(Some(BuiltinScalarFunction::ArrayConcat)), + } + } + _ => Ok(None), + } +} diff --git a/datafusion/physical-expr/src/expressions/binary.rs b/datafusion/physical-expr/src/expressions/binary.rs index c17081398cb8..8c4078dbce8c 100644 --- a/datafusion/physical-expr/src/expressions/binary.rs +++ b/datafusion/physical-expr/src/expressions/binary.rs @@ -20,9 +20,7 @@ mod kernels; use std::hash::{Hash, Hasher}; use std::{any::Any, sync::Arc}; -use crate::array_expressions::{ - array_append, array_concat, array_has_all, array_prepend, -}; +use crate::array_expressions::array_has_all; use crate::expressions::datum::{apply, apply_cmp}; use crate::intervals::cp_solver::{propagate_arithmetic, propagate_comparison}; use crate::physical_expr::down_cast_any_ref; @@ -598,12 +596,7 @@ impl BinaryExpr { BitwiseXor => bitwise_xor_dyn(left, right), BitwiseShiftRight => bitwise_shift_right_dyn(left, right), BitwiseShiftLeft => bitwise_shift_left_dyn(left, right), - StringConcat => match (left_data_type, right_data_type) { - (DataType::List(_), DataType::List(_)) => array_concat(&[left, right]), - (DataType::List(_), _) => array_append(&[left, right]), - (_, DataType::List(_)) => array_prepend(&[left, right]), - _ => binary_string_array_op!(left, right, concat_elements), - }, + StringConcat => binary_string_array_op!(left, right, concat_elements), AtArrow => array_has_all(&[left, right]), ArrowAt => array_has_all(&[right, left]), } diff --git a/datafusion/sql/src/expr/mod.rs b/datafusion/sql/src/expr/mod.rs index 27351e10eb34..9fded63af3fc 100644 --- a/datafusion/sql/src/expr/mod.rs +++ b/datafusion/sql/src/expr/mod.rs @@ -98,11 +98,13 @@ impl<'a, S: ContextProvider> SqlToRel<'a, S> { StackEntry::Operator(op) => { let right = eval_stack.pop().unwrap(); let left = eval_stack.pop().unwrap(); + let expr = Expr::BinaryExpr(BinaryExpr::new( Box::new(left), op, Box::new(right), )); + eval_stack.push(expr); } } diff --git a/datafusion/sqllogictest/test_files/array.slt b/datafusion/sqllogictest/test_files/array.slt index 083c4ff31b8f..d864091a8588 100644 --- a/datafusion/sqllogictest/test_files/array.slt +++ b/datafusion/sqllogictest/test_files/array.slt @@ -4617,6 +4617,45 @@ select 1 || make_array(2, 3, 4), 1.0 || make_array(2.0, 3.0, 4.0), 'h' || make_a ---- [1, 2, 3, 4] [1.0, 2.0, 3.0, 4.0] [h, e, l, l, o] +# array concatenate operator with scalars #4 (mixed) +query ? +select 0 || [1,2,3] || 4 || [5] || [6,7]; +---- +[0, 1, 2, 3, 4, 5, 6, 7] + +# array concatenate operator with nd-list #5 (mixed) +query ? +select 0 || [1,2,3] || [[4,5]] || [[6,7,8]] || [9,10]; +---- +[[0, 1, 2, 3], [4, 5], [6, 7, 8], [9, 10]] + +# array concatenate operator non-valid cases +## concat 2D with scalar is not valid +query error +select 0 || [1,2,3] || [[4,5]] || [[6,7,8]] || [9,10] || 11; + +## concat scalar with 2D is not valid +query error +select 0 || [[1,2,3]]; + +# array concatenate operator with column + +statement ok +CREATE TABLE array_concat_operator_table +AS VALUES + (0, [1, 2, 2, 3], 4, [5, 6, 5]), + (-1, [4, 5, 6], 7, [8, 1, 1]) +; + +query ? +select column1 || column2 || column3 || column4 from array_concat_operator_table; +---- +[0, 1, 2, 2, 3, 4, 5, 6, 5] +[-1, 4, 5, 6, 7, 8, 1, 1] + +statement ok +drop table array_concat_operator_table; + ## array containment operator # array containment operator with scalars #1 (at arrow) diff --git a/datafusion/sqllogictest/test_files/explain.slt b/datafusion/sqllogictest/test_files/explain.slt index 4583ef319b7f..2a39e3138869 100644 --- a/datafusion/sqllogictest/test_files/explain.slt +++ b/datafusion/sqllogictest/test_files/explain.slt @@ -180,6 +180,7 @@ initial_logical_plan Projection: simple_explain_test.a, simple_explain_test.b, simple_explain_test.c --TableScan: simple_explain_test logical_plan after inline_table_scan SAME TEXT AS ABOVE +logical_plan after operator_to_function SAME TEXT AS ABOVE logical_plan after type_coercion SAME TEXT AS ABOVE logical_plan after count_wildcard_rule SAME TEXT AS ABOVE analyzed_logical_plan SAME TEXT AS ABOVE From 73162744431317f19abd562484372b113ffc2846 Mon Sep 17 00:00:00 2001 From: Jay Zhan Date: Sat, 6 Jan 2024 22:34:57 +0800 Subject: [PATCH 561/572] Minor: Fix incorrect indices for hashing struct (#8775) * fix bug Signed-off-by: jayzhan211 * fmt Signed-off-by: jayzhan211 * add rowsort Signed-off-by: jayzhan211 --------- Signed-off-by: jayzhan211 --- datafusion/common/src/hash_utils.rs | 46 +++++++++++++++---- .../sqllogictest/test_files/dictionary.slt | 2 +- 2 files changed, 39 insertions(+), 9 deletions(-) diff --git a/datafusion/common/src/hash_utils.rs b/datafusion/common/src/hash_utils.rs index 5c36f41a6e42..8dcc00ca1c29 100644 --- a/datafusion/common/src/hash_utils.rs +++ b/datafusion/common/src/hash_utils.rs @@ -214,22 +214,19 @@ fn hash_struct_array( hashes_buffer: &mut [u64], ) -> Result<()> { let nulls = array.nulls(); - let num_columns = array.num_columns(); + let row_len = array.len(); - // Skip null columns - let valid_indices: Vec = if let Some(nulls) = nulls { + let valid_row_indices: Vec = if let Some(nulls) = nulls { nulls.valid_indices().collect() } else { - (0..num_columns).collect() + (0..row_len).collect() }; // Create hashes for each row that combines the hashes over all the column at that row. - // array.len() is the number of rows. - let mut values_hashes = vec![0u64; array.len()]; + let mut values_hashes = vec![0u64; row_len]; create_hashes(array.columns(), random_state, &mut values_hashes)?; - // Skip the null columns, nulls should get hash value 0. - for i in valid_indices { + for i in valid_row_indices { let hash = &mut hashes_buffer[i]; *hash = combine_hashes(*hash, values_hashes[i]); } @@ -601,6 +598,39 @@ mod tests { assert_eq!(hashes[4], hashes[5]); } + #[test] + // Tests actual values of hashes, which are different if forcing collisions + #[cfg(not(feature = "force_hash_collisions"))] + fn create_hashes_for_struct_arrays_more_column_than_row() { + let struct_array = StructArray::from(vec![ + ( + Arc::new(Field::new("bool", DataType::Boolean, false)), + Arc::new(BooleanArray::from(vec![false, false])) as ArrayRef, + ), + ( + Arc::new(Field::new("i32-1", DataType::Int32, false)), + Arc::new(Int32Array::from(vec![10, 10])) as ArrayRef, + ), + ( + Arc::new(Field::new("i32-2", DataType::Int32, false)), + Arc::new(Int32Array::from(vec![10, 10])) as ArrayRef, + ), + ( + Arc::new(Field::new("i32-3", DataType::Int32, false)), + Arc::new(Int32Array::from(vec![10, 10])) as ArrayRef, + ), + ]); + + assert!(struct_array.is_valid(0)); + assert!(struct_array.is_valid(1)); + + let array = Arc::new(struct_array) as ArrayRef; + let random_state = RandomState::with_seeds(0, 0, 0, 0); + let mut hashes = vec![0; array.len()]; + create_hashes(&[array], &random_state, &mut hashes).unwrap(); + assert_eq!(hashes[0], hashes[1]); + } + #[test] // Tests actual values of hashes, which are different if forcing collisions #[cfg(not(feature = "force_hash_collisions"))] diff --git a/datafusion/sqllogictest/test_files/dictionary.slt b/datafusion/sqllogictest/test_files/dictionary.slt index d4ad46711b9f..b7f375dd6c4f 100644 --- a/datafusion/sqllogictest/test_files/dictionary.slt +++ b/datafusion/sqllogictest/test_files/dictionary.slt @@ -148,7 +148,7 @@ select count(*) from m1 where tag_id = '1000' and time < '2024-01-03T14:46:35+01 ---- 10 -query RRR +query RRR rowsort select min(f5), max(f5), avg(f5) from m2 where tag_id = '1000' and time < '2024-01-03T14:46:35+01:00' group by type; ---- 100 600 350 From 4dd09f3f5d0a3228bd7a0c684c19885c5ee58c5f Mon Sep 17 00:00:00 2001 From: Andrew Lamb Date: Sat, 6 Jan 2024 09:46:14 -0500 Subject: [PATCH 562/572] Minor: Improve library docs to mention TreeNode, ExprSimplifier, PruningPredicate and cp_solver (#8749) * Minor: Improve library docs to mention TreeNode, ExprSimplifier, PruningPredicate and cp_solver * fix link --- datafusion/core/src/lib.rs | 27 ++++++++++++++------------- 1 file changed, 14 insertions(+), 13 deletions(-) diff --git a/datafusion/core/src/lib.rs b/datafusion/core/src/lib.rs index b3ebbc6e3637..8fc724a22443 100644 --- a/datafusion/core/src/lib.rs +++ b/datafusion/core/src/lib.rs @@ -285,37 +285,38 @@ //! //! ### Logical Plans //! Logical planning yields [`LogicalPlan`] nodes and [`Expr`] -//! expressions which are [`Schema`] aware and represent statements +//! representing expressions which are [`Schema`] aware and represent statements //! independent of how they are physically executed. //! A [`LogicalPlan`] is a Directed Acyclic Graph (DAG) of other //! [`LogicalPlan`]s, each potentially containing embedded [`Expr`]s. //! -//! Examples of working with and executing `Expr`s can be found in the +//! [`Expr`]s can be rewritten using the [`TreeNode`] API and simplified using +//! [`ExprSimplifier`]. Examples of working with and executing `Expr`s can be found in the //! [`expr_api`.rs] example //! +//! [`TreeNode`]: datafusion_common::tree_node::TreeNode +//! [`ExprSimplifier`]: crate::optimizer::simplify_expressions::ExprSimplifier //! [`expr_api`.rs]: https://github.com/apache/arrow-datafusion/blob/main/datafusion-examples/examples/expr_api.rs //! //! ### Physical Plans //! //! An [`ExecutionPlan`] (sometimes referred to as a "physical plan") //! is a plan that can be executed against data. It a DAG of other -//! [`ExecutionPlan`]s each potentially containing expressions of the -//! following types: +//! [`ExecutionPlan`]s each potentially containing expressions that implement the +//! [`PhysicalExpr`] trait. //! -//! 1. [`PhysicalExpr`]: Scalar functions -//! -//! 2. [`AggregateExpr`]: Aggregate functions -//! -//! 2. [`WindowExpr`]: Window functions -//! -//! Compared to a [`LogicalPlan`], an [`ExecutionPlan`] has concrete +//! Compared to a [`LogicalPlan`], an [`ExecutionPlan`] has additional concrete //! information about how to perform calculations (e.g. hash vs merge //! join), and how data flows during execution (e.g. partitioning and //! sortedness). //! +//! [cp_solver] performs range propagation analysis on [`PhysicalExpr`]s and +//! [`PruningPredicate`] can prove certain boolean [`PhysicalExpr`]s used for +//! filtering can never be `true` using additional statistical information. +//! +//! [cp_solver]: crate::physical_expr::intervals::cp_solver +//! [`PruningPredicate`]: crate::physical_optimizer::pruning::PruningPredicate //! [`PhysicalExpr`]: crate::physical_plan::PhysicalExpr -//! [`AggregateExpr`]: crate::physical_plan::AggregateExpr -//! [`WindowExpr`]: crate::physical_plan::WindowExpr //! //! ## Execution //! From d6c891e195ed631a9228140caffde282ef3ce6ac Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Sat, 6 Jan 2024 09:19:55 -0700 Subject: [PATCH 563/572] [MINOR] Add logo source files (#8762) * Add logo source files * add another file --- docs/logos/DataFUSION-Logo-Dark.svg | 1 + docs/logos/DataFUSION-Logo-Dark@2x.png | Bin 0 -> 20134 bytes docs/logos/DataFUSION-Logo-Dark@4x.png | Bin 0 -> 45176 bytes docs/logos/DataFUSION-Logo-Light.svg | 1 + docs/logos/DataFUSION-Logo-Light@2x.png | Bin 0 -> 19102 bytes docs/logos/DataFUSION-Logo-Light@4x.png | Bin 0 -> 41968 bytes ...sion-LogoAndColorPaletteExploration_v01.pdf | Bin 0 -> 875036 bytes 7 files changed, 2 insertions(+) create mode 100644 docs/logos/DataFUSION-Logo-Dark.svg create mode 100644 docs/logos/DataFUSION-Logo-Dark@2x.png create mode 100644 docs/logos/DataFUSION-Logo-Dark@4x.png create mode 100644 docs/logos/DataFUSION-Logo-Light.svg create mode 100644 docs/logos/DataFUSION-Logo-Light@2x.png create mode 100644 docs/logos/DataFUSION-Logo-Light@4x.png create mode 100644 docs/logos/DataFusion-LogoAndColorPaletteExploration_v01.pdf diff --git a/docs/logos/DataFUSION-Logo-Dark.svg b/docs/logos/DataFUSION-Logo-Dark.svg new file mode 100644 index 000000000000..e16f244430e6 --- /dev/null +++ b/docs/logos/DataFUSION-Logo-Dark.svg @@ -0,0 +1 @@ +DataFUSION-Logo-Dark \ No newline at end of file diff --git a/docs/logos/DataFUSION-Logo-Dark@2x.png b/docs/logos/DataFUSION-Logo-Dark@2x.png new file mode 100644 index 0000000000000000000000000000000000000000..cc60f12a0e4f5a3bba66a01f377adda7a03c8113 GIT binary patch literal 20134 zcmXtgby$?&^Y+p$DP0RoNq5(hOQ%XIE!`<40xn2{)Y2(}AV{}#NF&{iNO!*vpYQwo zgUbuAefFF)bDx=e?wN-OEe$0+Y)Whp2!sbyme&D+kP|^5Bvnjw;C~_$F&Kb`|3$A&ev zsB8%2)q2JX8I~*m2e1rp;eN6#JI0H(i(@ek6U1Ee*Ed6Y(|VrMxGQ;*l%G?5er&k`^~c#~aC?WqY@GkSC-1ora@`itoF_*N#*{6ybr> zib=n;?Xce_9Z&A^;kI~v2Zs`j3vnZnqs-Xv-aEC#=1=0bGcn})DpnY9`cP#ds?8Pd zqsEph=Y(XZ(Vw2Bn}rgAo>3RZ_;e~`g73E&O{ju*pdNJX2vC-&iaJeU3{9bzc9n_99`NU91l)^LIhWy&scxbMu%eV3Uhz=J;0VRY0zi_ z)w2IhM%Fv3wb%zKA9kK(eBL!-`v<}$Xj7aO1X_-cC@N;?t$rVT4{p}uS0 z7jI4$6I>v7(0o=It4@GbBTzB5I3?F?)&%F8Yl0h2J7D))^U6)GF@GJbm7@PMtcl|! z1$d%f)c&8`99zvv-COZBNB_E&ty3%VabgWjuw>yMo=ZO5Z0=5;*WtNsxSjb{1o2w*aBo<(|&`Vx!Q#tz@NL|%x$&Eb_t{K zT$ZKJ7_77d{+RSGNtD9YIZS;vaesui)gG+01=L`Df^H}&Uo>1J?_tccOR@As^ru>p zYgqa9NIG2TIWeASJ!=RupL)E9O7*JBD?n^(-7gp|X$*w9_#=2EFkH$j=>VV%^1f8- zfs?x-HAnvQFH3aY zkbSDmbc2Gy&fFG`F@KC<(l*O9;dBw`zQS~Ycb$B=BXI%iy}POa6y?eY6Wcv2hY9ok z9ySjgifl+?$~IVI_Zzq|SKVc#d{jAvutla+dR5|f+S(0G&twxo;oYy9m@T?aR~e-L zwM6&HiH5Ebaob3i?(|Ii<3Y;_E6iW6M!#{vgBNi}+w(;L4@mlT z-X;%(?DaLx_O-YY##$|%G1Z|X0QZpxpPLiaKl9zjDs?>LS!e-{>sxii934t`dIR$N z!^tSgw0F4of<2QVEc@qyepNVd1l}{K8uGMhoJ73GsGKHTgI9jcu{ohOj+QY<9C4Gn z6<>!}%uNVY*4&9sC6U%lBU$av#bythrQi7xtq^E`1pKTVkRjX@eJHUpZ`sedlR;?v#l(eDOV^s}i-JNN^6ZK@ffOz3j(^1vc<$9`?)*?qfIK zESHKddqOBCFWfW|_G`r8(y%%g3%F0BXB{!1sYs`AIxMYRz;GmME;mGg3*w|c_e*H@ z#eB~_@YGcnhy*+BBYqCAVXj$~*%s;+d`Ry91No=`z*gudAB|f7F3sVe!?RuN3;n1Momw z`1Ej7JO7=CT64zvfG`(T5%<_rKp`Es=Vddi`AsS-q|aWF26Z#dJ=#jgTU8=m!^P*e z*#D*?rhqMNlW7i|fumsBT&J%IG^b~r;2nQ%?;qPdy@nPj1NVNiT*a~DiRtHw6ijdz zG@PEfSai?nvyfil2}y!FM<)*mh(lpiWV_WaS*O*nrf;7z%O~(>82fAfumT})T5y;M zinY2}w=K;s3ERwa5@3i9g06ynAJIKeJ=fy?ruJnU@G$+6+mwBqs{wz2t);FMKM=&t zKEsUMege*Up{wjROAu)Jwdv7^!oH?Rx8TTqYBv)SO7NLHBo)_fD*7@|Kp@PL*tcQ` z$3NB}QDas3sun#jEdVdP)k4pTU`N`Xa z;r=FrBZQyaaj5$Z(JqNniHTAPTW~R`2b2Z!!Molc#{{zh=RTE)^F04;9||CWk~O^k zoK|$twDVB4kx{i-Faf~!h&W7B#FjeZ0Tn)HCW+2t8ooLPM5bi39nQ|ty-^8hnHi=z zUu+SYfj;0VW^CukR*cSA$%1^3$W(qyvbBFdc0LP4$Qx!ch+WWebh?apM2mr2Ec?vI zJpd~hL<43Zbe>e`(Z!X%AO_+f*YoLU1~TVbEQaq4q8-PsmpRcEnQ>D5sJx;Fb08Do z4%&9IQ><^7ogyzYcHP#c9P3&5Hcq_r}DYO7^K()$M z8Gx=OSt)!R*PB=%wEj^9kmbh$TUXB7kh|7rA7JPfk!>L*9LM+LWM&FY(SJ=ak|#vS zZL{b+7EW%u^8Rp4?o$9KC zDtR6Jz;J({g-fb!R1PP?D;9EgPTaxrEroLLfCo;IV8at+%o;5QfK}vI z_nv_2jI=k-W++sM>D$}pNQ1AeR1it2F42Rvi`PPw2=CG`E#D}R9$1Y`HO>YCCB(SJA5Yf8QK3SF%RavmFNi#nP7ppOP=|bBa~%nPUt-B&ymOT; zb+D+g2n4tp6XLc~ragw4&&xf5-`;c1$#6hCogBy6eMeX9iOum^2ol$xpSovdbsFcc zZiNE7NA=`=ew{S#Ua;1)1wR-GgX zwu4(q-6ICY-s2KL9#dZZ&)QmWU=O{luaaDu&0KXdHyQ+Y1~8%Fp`PaBK=HC-Qr5P2 ziCeCP#*qru0Rb4YA49J~Qeq>U-6aWZO4jQbF=%tv8B6gYqf1^0pdyuL;s8H$9pPvc zW$Cg5Xkd&^Q;ZH$uo9I}z}pYKOoO_Xf9fj)S9=|GJ7ZD02MsQbs&k2m%tVHsz?!7) z^ZasqSzSSAJL~0eag2`cg%p!quMh7otLL=#U-EciK6^i<1Pt_;Fe6zv?4KQ{LNT`k zrv*Nal(JR3FfoOw?4M3Lh6;bOKF;1iN4#FWa05O(fp*56jbGnhrPC2y^iT>{8cJ{Xp;p z*cwjk9WfZt3Lo=JE2$C?`iAZtUhxL6GniUw(D8i+;8O}Fv*ma+7kWNQvWk^Q81XWU z2grw_b^DJAzj*VQhioNA1epo6D70 z8iQt52Ad(i(Lz3me;eL0JE0NHP-wuyT>6rEYKZN3*3XuR=32TygrY!btOKv_3ajTN zN~Oc|@uC8WNaYIR57|rcSDxK|sEJJZ zxrAtAX=rHf^Ra8P*RDCgnZ!7faj%;IBAR}B1qLYVN~C3g#OnnsYT7=6>v;84-Z&1r9gDbX9I*QYxoD zdqk1|MbtJUmV#Dmh{G?0CHhJYu!`xI%S6QCua~uv!VhAflFfAam3AlncZ3mYrH_ja$8iI>s4R6SO*GXF@AV;IGbo9!1wLCwEuR?oD3{< zjb!@0w<0;BkB3gMF@PyP)v(Z{%NvG1F>gQkMm|0`g-=w-W9|!K2mL$>EPiMn-ZA$I z9_etCT&zCT&La#|&g?KvTZ?1$Cgs<+xWW_r8ieXqX7NMqIZxYxe}|khRfzqBlEYcZ z+f`nRlP2I7{FJeP>gWjH6`5uyeR^66U4A4zgQXi%x^4vhRzFc5LZh`iOUJqAkPNw z8D$(40TMRbXP1XoB?=%bm5`OK3DNm6V}ZMHjY0TGS|pA#sG3BPhm>&-`AER7*u+Q@ zK=Vkm$WM!>%F@Ta*O62`p(qZ__IfK<0eoArGG>1f%Us-p3@>nObo%ReWG8!-8{a)6 ze}##%XZ6>nTVH?H&d&sAh}{6>Zsq&D|0eq+RazET$# zxvyFl*o&VUq~ejA3RE?61o%eAcz1o;b_S?ejoyz}oMKHLH2e zdusEOY(A5-2y3YkLE#2!1ST^d0)LMF=04!PWPtJ+R5~WbK`x+A2M-7Dzg|Pma7>+kUp3f4eUZxXA%Bt=!1shLzuP#s z2+}74D6YZ_$MxpgS9txldnCtjQyc?1oX{wS`!sMPg-Bi;#8Y!Zo=bFJuahCjUI>P^LopmiDiv6R~+d_8GKc+yTmjs2@pS+JBya}JrL8L2m zCnAlg3;mT!Y^S~A?EbUJEcI#;`F3u8(tIvwL$!8N;%b^zt0nq)wm`cbPGN5m7Hk#N z46CAzgaN=I$5PTDQZ0JE-u3iJu2$YD{#kvl;%W_*tvTc|Jp_#G!oeKKh>cE38sFsp zJ3Hh0{@PtTjZz8h)0XAh0e%u0zYonYv2#xI1*I&eO^zia6WEO38WPCxU zuB|gjvm@rmIEpNR>U^u4U1_xW?|G&I=5>&E{rrAkM&6`BYT}vNu$|wKBEi^(@ExsE zJtst+F4iYlA76YIDI-`86Pyf={<{9~fgjW)cQjroVMsQ}>$}s_@0|ukK7bOr&01(-ZpI3UuziXV2A_?)Oyt2mCh}{A zSX*$e(GL=)T%9g+qwfu3Lq(R=IL`gN>Zm^fCZx4F*B}|I|EvsYJeir~rQ%27QCOQ< z6w6RY_%LF_%ozY!#67q4>ygUCIlkA3j^G+c3g!p>B7ioCiKs9kbVrb)9Iv7v#@_|! zG$W&FnUZO_J6#yoc%@9eD-FjMnB3)ioj~+`r*BCWiXN|C-y;g*Ac{{HI)Yp87T!_( zey{Ez$sP0nRPc>LU8n`q8%+^~0NXzBw$jud*dQ+I-gxuqF8#3g_|SIqpxI7+TpuBt zhsfiXU#t4DqxpMQL!+h=s@(Hu%S0WOVc&(5dGjreIq`zJg!5^vKG(C|>z~tSqES{a z5Us@WtFPBvWDVeCCz$c@>dib{c`_Zk_R5mnBjoK#Tbf!MpMQSSZ(LmY8kU*wr772j~WWh9EA* z#+D!g>3i0sD5>Gb8O$JBF9#K334`!l!skP1&QQ?XTHH=oyu*}d-bMiv4I&i20MkA9 z+|7PWTkgc53w-oLZ1-k6E&teRklFZt#s+cjJ8G?1$4j%)w3Cq-q{5k5E7~MUKp2p1 zm3?&Wtk6V{+2B$i(*$FI@HZddA+Lwb|IxAHSX*;7?~mLSv3-cT4O_lt3A%T0`}CFY za=?>~4I=$;qGVKCCU^moxZlsdy*pgEN7{Zw28wLvwG8mN)(&+ymPqIsHITr@H*VN& zvIqDM=7P%!My~YD`@?rdKn$82YnB4{&aFfb@pKQrHtAr}kNe$&>}^4-u@COwXQzY? zC2mmzPui}gY_fx#uV;PspB(|A;&W9rt7C9ACL)=5+yb)Pt@Nm1`FoU`f%awRVrza< z<%BCK3nSUsal_Q@F}}@^z@vt1{7)&x(|{L9^3W6Zpwn4S?tjy5e$L0vfd@@Z^|t9w zfZMrozb-7fCZ*5B`Aw3sGGpzsrTZ#2LJ$ISfmu#7Eaj20r5U|nFBt;FJVu82aJgv; zY1g;j(}9WX6>D$|GopIn|6cqF|EqKmh}%xL^Uv)A!a^XcU4V1qwS+~2#@+kEWJ6!* zNy_7zr2p=cw=~MiE^v6H6-S!kkQax71hk?mnRxdZqolwsJ|l=mj>|KaN#}mW`Y^Dd z39SS+DQ5EWt%l0MG<&j{2w|*8hO_sw_5gpONgO7_cSOeHechwW<|SRr-rMNZ=8hpq zdd>k%U@twErRcD$*FD!S6c5VRu#4F~4+JMP25r$l5BLHjw$td4qhHCNz5};hIW`V^ zlz(!<=g!!4udJV^1U><3=XF_}ZCBV(XBYdh?Mi33Itl;uXG*>|MkmF@&sP53_1Cr) zey8umAdv}MbW>plfePvM?tWHU^p%12?;gZ>;51A8_MhGTJI7P24m$HeIp}lm@{51P ze&x;)=}lW|G$I5=>hAs5OT5$AG=D*UslZY(#Y zC`y>G|91KIVc<~uI4u@5=IU2lvg@4!|8c%aN6M1_#iF4gPVGc_j+OT;b3_iy8^eTZ z+~)OAOBzL9PY1w}z?L*bbL9*WO{4`N2-}Pwx8ip4AA3b1Nwn`nBN%mY)Eede_L-zh zfeiLi;d0H=!`qb)TZ`BG5^)&dym`6)cs(Cvx975z#sT>`BbJdFvFM%Md>Ww?bmM$C zC0$Anu~-|uEU!qFE}I>8=W_RvDe;?ZwY(~X6hMJXHC ziXBWsCCW6O`R=ejujZXM?wCWe+kqUlNMG5E#P6Kn{%hH$(+e2lif2B3;8(LgsMoJ^%zf z#f)4_xm-!&x2au&D|d9L>9T?+*cWN#&+-+^(L^NM$`|+mjDoy&j@cr_4 z!{#a!`nZucR|w@=?Xjugk(Q3NGRh9XgNYRhpabB%cJ78-lqKp5x?^Y-OERV7-)0et zYdGb`t5xjwQln4&uiTK{kFf}9DkHlD?b8LG1_37X1s&uXDx3-B2|*nT1gP7~xv+7) zg;Cv^u__T*12yBmZ&gak$FXVi?haF18hbfa0yikW)f51#zz(c z3wh4WlO9PYhT;5zoMtfB8lxb znEe=H3i3oZUaq&+Ijr+JIcgm)-v6D1rIC}Xoa5cH?3@{0so8|ib)q2fVPKtmWMcbq z&+ z??jd)MO%Hs!j9c98vJU!A>Lm3Q$sP+(og+#f&|`iIkzachJ1arLdQSzh*X{h6Z|JL zd~!OuDj5p6R=$Vl!0NKulB5l^l}*7fs&*ZNw61YSMuX#`07FWqtYx6(yPp2vEC3&8 zs0xk>PHCO+*gHfS_{kIj0FGb)j`l=5?3!{MC(W-%IUHf(10ozg?Z)EkU(NjWFt&0WJSyW(2M&fjfF(M1Hv8Lq&|KA7H>wzFQmH2 zm$4>8)JsG0?2D~(;0xO7r7A_`+HqOg6U{;FfuGD&2=8`WZ7QZk2Bx+^-Eb739Pr9@ zzm%{jhs<{I;l{c)eag)~o8-_?gX=G|iATih zMwF4n)wQ=&uDC50Ae2;BX``>RZmwUDeA5WCjEZ{8F(K425K`fRHfYy; zd@5E|M{(PJ8+5CC#JhfGzfKG~f7KS@ofQDxwp!ZR^XVcdOxQJtn=_GqRKw59km%Ho8xXETOq0QzsZ_OX(D8aiu*M8`g z_>vXWAPKebrdv?w;?RFJ$40spxzyvxo7)AI)_v*wVdOWyH>*V^+oRk*qlLdQf6e_+ zQ-->|bY7$;X|GsLcQOcHB|MACc04heGk6XzI&^u$e!aRh>#>!XV4v#)eksD2Y*=5 zTm0|52lXy$j(Fh73{c$$i?)P3RsWd5ds`dR&j;G>R;>K*He<^jPG1FX)=i*^y6?RA zkK}XO@x)<-@-6-di0Nar)kSUpXj=cCN|LVA@`G}Ury>r_8+ekU=@!JWc5~YwlEK_LzaDN{M<7X*;QCNd+%PY@+?*JKA!*jK&1$7q*c$P4)$qm0 z95T1ILc4y}Deazf2OS3#oU#O+8nwkft{mQh3G|)oD3+eh`7xG2Mkv9ynSgZ;Rq9Me z3y^fh6k6LNwww|-;M!H#fPXJ?+>tC)aqtxu(YvX;3- z-h41cfemoOZ)aA%L|u=!9In~4`g6B5%kjl^AUrDj znVI|F06I3vhp|RI6HPQ)e<(6eLYcF7!qJZ#LOd$YR@NYE*ekm{XYu6&u>9?4rbx#E zZ9>=q+7Z-`q@4cG58K0+CTOCK=5+)K53*paw|g#LSS<#BOy!EmKP=GiUaV64*CU=D z75Qk&iVl}h_!q61uoDRj8gf_a7 z)4Ba(e6nrcJmWL?2@49g3UUDN4{Zl!-FuGJo#~zD+%zGiRCiIue!R?;*a@xeTHYZE zs|OZFkt2Vftc=69n~9odT$^KR4-eb%!@=<_Pv`1m+mkfrXFfh24YNdkV=BzXmN?!@V{e{GU`%!q(5WH&7o!no|^+^OJs*-#~8xC?UzkO z!Kep-XHhBV8}Vk}Qm_4*!tzEz!SU7wLWy*vuOti@aO3)vE+D1lA;~P8CLA9^ExcY* z%<1bG6VD03m{!&KC}S}%yP?i}l=JGW-oo6ztSAS_Pi{nx93G`j7G5CZXS^deIR$7_ zVq9y-5ms>os~@9^Lztc4PN~%$vqGkc-Z@F9YL!zD0wJ0fk7+?2sA}-zh%-N*gt$>@ zXKA+Eir!nf^0^6mV|{Ba)>AFk<)z$v|3X;7cfv7zLDh{hLAb>InJc2QORb>(AJn~0 z_Uy7S6;i-5NzaP(7QcNS5C?DrE)!lt7i%3vMOz!kU-_}OiDky z4iy3QP|j=Y1-grJwL}%Gyz=?HEg|Ej>qh!zH{TP5_ebObg!Zg+F_1Bt90WjHk_pV} zN!)WfgMNfb@V)?8?StUdZzg8og;Ok7>lmATLKg?O_qS<9iBTe|n zWPmR=79|TvjXtB|6;An|R-J(ChYnN(DDtB2jo#x335{>Mt>vQe<`9 zT~szcs%RqbVF|-9lH^^NHfK2oz*`gqL>Eg+8-;3hwj~3Y+OaA^Qo@O@)93{TjI8*e zyLp3KIN6<9k1nt(oo=+_d$OuO3Qm*R?+%1^b>{_5cpHS1#$-az(yYkQ2YIMbd{EBA zTR$Lrwn@HN^G?mwmgjwshJ${ejNduDbyH~ znTMl`uG6rC>j1`58A`&^cX6QyCjC+jVjlmO5@!U=7&*l}zUwS9xK1i2Nhq)f>}%Y- ze~pU)f|QR)mA%-0}=rvB!`Ego0=8sV=JJmKMx zmuR<>L9yg(EZa6h2aaZlKJomdaD|sT{z8rjkRXu-GBmMV(qr&Ec9~o&udSKO7p!q| ztT7vf+TA3E_PwbKnek@ED1jIjrv{0&iSnb$>Mc{7uw6NiD1;b(=Wr9s1$0M7u>Ioc z79#>DgAM3QG$^y@Vg5SoHZnCMAEWRxG*w>pI>{Nf z^M0XyFVVlTF)kBnhOM3xjj0k1|7O*xjZl>`Pi^Alb?U4R?2>v6HN$o~oM2pO!`c1~ zl)smi3q7(5V?~Eq;YzSgh!-`jtGdMOfx6UmkVbx*(eFn@C4h7lgL?x&llD*%9z|xxtgWu%83DfE>VcyJmm+^i?gr4K zF@${4^ff#kbVl((tjSW*9QJ(WWhW{tSqMaF7Y8mH#+_`Bu5;BZHeQNj9>4Ua{aKgt ze4rg95PnvCMsQv{`TgBO7a~SyN?&rWW6g4MXS=b;?N;x_TX(|4%fzVSm|4EBq&sO2 znEf9nIbUa_nuaO^@UW%c)!bz`VRcMzD(gggJ!S;|FK($$`AikQ@$0!5w6 z{2t9KA6>1NaxBZfb4i4fJs$(>=bgI}(t)HDS{?3_r9RDV4us z0O3yjix7=(vW(eQ8k-l?o26$a3#3WnIlYY>hS{@#bT$`9)1v4W^ixN0>bP&xo%#v4 z#5&hWp1*Ic;qLrm=HD+yv|ABn1N-+?nNT9rtm@+T7}}p-;+}ARCR_&z|nbr4EGyDNNfLUAY z=)1YYVR*_m$cu<~`H{b$Zd(r+O{^l%aArQzy3Mi5Q)Hs$s!4xY7wWbfg7`A9FGi@j zO+}$8X?4@7Xgsvf#=O@#u@D%_x;>8h^t2+;$cU(0QJngGB=5Xvf7WXNJ6? z5g(OexA%q@_e`N#epUae??ud zyZIWTrJTqX-sw9{e+V5nu@n}x`7reYpj~%&lrqg10LM8{Re=F*uk_AppgqN#i5-%# z)UWSx!fcXBmK=h(E!iw3`Z8Y=N04OeJVN_kt9-@QL7ZFrw*odGREY)U%k=&Mm}Iz? zY`MSA`5`YIaHGW`KJkq1ERsFv0EHAJobOMMo`W_$Q**F8HF z-(;axM%ofzWuJ;Oyq-#tS`9dZnEfY3URxG5YL<^hmQ>L+;f;(1g`YL%X`B}m?XjI= z28I_%?P#W1YJC7V)70mj3Dgy*Iv`+n^Is^Sd^8gl{Yw|Y)JBDBc%CjUpsQ6rf8+k1+n%?0Wy@Hxiy{KPz#;1^)LyjJxJ=xVz^TLc-A40) z%jl!|I7cKlQFsGZ#OU)xIb!SvfIJt+?b4gT6LnqO(Qr<56||&aoX!?90L5a-ZyG(F z8jPO#Rj@=Vvaw@8Im5t+T4`8AnN_Q ze#NUXXr;i^Lz3xkzF`&DulI25N!M2Y75%Dmg<0LOe)3^)^Nefp7f;cK&;R5wY3!>qJo$JtAJ* zD|4aERjz1eK;C9u>2abrh_qeS?i!$qh7}g{TvcdRZxHF40(FOb+i$6ecWTa&4VC9} zQUx%Fn#JqD{K}ykA|JB)55}&a`R&@i_)M}!Lb-S`{8MwK|MAM`Ma_ccFLnECU;8Z| zx--Xr=2#UQ4uF0yY=JkHYw3B3c_WN(2$)Mr8y|;?NBDE;)7~!C1ZxXlJqjhNQ~AF? z=4(9>&Mr*ZLj-2Ud?Xa&9 ziUGQO2bw>Ul;ES~!x=lcdgWREBGx*(6%4f%W4p&MLQ@ zE)kf(wkUf5COGPDG9hlj*a&uVp+wy!OWfdYJ6F2yV`Ixf)hD!6F z(x*kD4fA~~iSx-?(>9>A{^p9GtafR>fIu)hibtk%k6oC*j(D_Z;rNrZ2f&>o1`6l+ZZKJXT$iP$1(_ccj4ln7`q8%e^>v ze4YeY0QF{dp9P3Oj8fh@QC8%%a&7b;IOgMpBb(Q#M8Y{T)P3w6-_E%0? z^Q4l*j8HY+r7p$`6s)ALD+oje#p(IiMh~6sM}l?IPIms(V@ff;nGv)T)*O{oY#ZXr zoItmkYk}&#{X=UaFiMQqY8dk24{Z{{2WOL@=akuulOL02WpHW>c9z zSDbCltDitkFwhW=54L0R2_|Cx?os>SA$#=h1JKI znM++Sne7OZ*woIv{OTpcHgY&+f$smJNk$ZM5^<2T|p_(DGo3go(4R+X^rcF1)Sc%0Z+8i%@56U2tTm&~tXv79}tDsi4K#vNpO0fAB*s zJV2||-)LURLy=AEi=W(Q!U7cos9mDJBoXEEY+`p{e4!kGif4nc4JWH^vqi7Y$K>qglDoD-H8)G=w5IRTzoEa1-dKqz@&CYl*G_|+4b!_J*z!iOw!1q$SHGdVL>W=`n0p92}~_)k*Szucrm9}2K9)4nzw+~-=hXC;v$3;L)O z>eC1dvd-ENdx5C}@@W*I*oD6@b^9`nIUu}QR7;}sZasgY0rx9CBw|s31_)Kvv5X5+ zvd_(afSs|pTMW#njasJRgjb{XI?qPgvFKmHvy+D~%2`?IfL><`8Uj%Pr=x1rFC$FS z9K=+?vd3UuE-$W%v-GT6=!9+19DhE=~ja-hVw&(D-&vLjW%$MS-k+TVGj|Z$he( zR00EmNf)&*XyKRxd;x%lb$B=gD48jYuQzt>SEJIfLiz>s=mx5<3;{S5 z!beNoP6Ge3ZeH^KRnG*~DbWOA=@w@koYr7Un?Ay;DYByS~ z;U|*dO9Edu7K)L={puHu2|Jf54-R~2g&T^r`E$OwaONB><%H)2&vK?*fa})H(K)Jg zeMAJj*vZ{(VI~=)W1&<6+B{*j(6Ew79}`ib5m%8^@~HGR+S>7>LFkLSIcR7fg!uhh zE3qGU3?AOR(`E-6u>6+qS0k(NRsc>Vp#IX|#z*se6IL>Ul8g$qF!ZjNb-IJ2e;26U z+F;;a*7K0lrtjTy(VZXwL`tEW&t*lCkCO8z)1r!gK07v62T91)(0bHj_WRhubf}{U zOp=OXbh7qox8+;+j98~1Lc4WCeEuc|k%isBk`)xW8%MKoGAu*CuHCu;il)`~F)3e1 z`mDEP6d@9!DV;u(LVF}T%)TrxW@r{MN!BI~B(9x@0N zV()>D);z`$`Zr%`FHm!f2!*p6~kH8DGrc&-n z*lL$A2fY7+2ck=FGNs-oUfbVI9?soEO6lFRP4q@aDDQEy?^n6I+Wh}9dh2yh?up6$LZy;`{*mive+lDd1QvTrS3N0}I_9mKJ>SEdzbwalpfk2=u856zhV1mLu$|p1 z75R?JbLB+$ZKwYhG!gf1VgqPE%F`f$UK#!UUF^b6@3D>4bp`Cau1+?9`# zTSHMvC-eRw`ypj3=x^J->gB=zT70jdTEj@qV;+H!U6zDF@sb-cZ_{ROc1ck^ z`$-#m$dxC)*0{w%bsnGsex3VhL~I5X#c0laosL58O>>jdGcTn=qIYl8cHw$A{W$1s z^7wFfHPK(pO^|ttoG9U*aJb7L7G(91z3h8|<$u1{_BrS#vwPs?|3>YW&q&EY&EGp` ztG|6`W)xOb&`G2Hxl>M^kZ&qh12?ZqW=+Qffu6}c{at|KKD1lhKXWQ&7U8TA-w5S6 zwNgwq!kDI#Aobb6u&-@dykwaS%I98-!k92Fm(xs*J)%@Mjsu3quZ>?H-)Axo#0zZQ z{?kz#SRQA3=`3L?SmN5G$Dx_yD)HE|d!%(QWs?#%sa>rk#?!@yE#OqA-;jM09eZ?F^q3TC=5l zsaX@x@C7UU-lT{8{pya2nssPhMY7pDOR!0lTFbHa%4y^Kv!(M9w9fXIm9ko<^hEJJ zJ}dJf=2emd;Y|j>Xegw)=Cj|o7RL>&Jb&KOFbIL*H-I{jYv+DB54couLtA-nD)6%Z z9hH?r#Sq|N51@EL_IXa}sM31c(xdh_>c8Idt*X&gN9_I4`5Z8JNOzg-@9uiJ(;|xv z8yBNhd8)9Vg#u0b6uZM$TDou8aO!1}*sZNzzo$WY{>5GXyQ7!-K&${9>!swBl2CNQ z0x*b!rJXi=bjKp?oyc+>8+eI7&s%0?E-QL7i61~v5kNPCOmG(9@nSctYe$pQoYz1v z?BZ{Tggo$K30n!ap($uE6-*Rg4pYETk0ba}k>|JWml)A9sQ8?^%*@cG$#82uvQK#n zCd9{8i6^gjfKvZyC$-#h1A+D%DI*XV*nI!fx$V)QjFxU5u2}4s`Pv!-_!|vpNDWSZ z<@vTNQ_pY?-VR1&H~=qd%i=3+q6JR_<9G*6a5cu|VvLnUcu~wGgAVQrNxLLY%b?QC z9nEj;&exq8M_!xLjyM!?E5*}Hcl+QM8-Adbzi()R?ryFhZy&KHnr41K&wE_=0W1s* z1Qxk9H`NDFOV+@&rUcq<$q|oP8lSwDoKvy#mVO|dwzfbq2hBc&f0p{H6FrCCu-62F ztr_=?VhBt^)5}`FBSiB^mH%H4R~`>l`^86;T{H38$ymmWE!o$>SY~4EM0$}WQCY)l zsW)3SlQ0I!Si(%Cddbp|M#N-DvQ+jYOSV#ms8oLUj?d@!_ndj|x#ynq-20sK{eD?O zmueN?FQY5JG}Rb5IBHufXJ>1Ql1fRgZQ&j=$8q~Q{Qrqwd<3j!e zfHy`IJBMv``>Wz^SvhqQ5Of#In>Jc_t=3(?x^{N~+$9}KoxAFtNN%W>>!1 zlU!Dy6+yXD9MvUuFrLrlb#PfrPSDw+G;%C3zD3~!?X%Q#g7lPZX6?qg=7Xme1!?>Q zmTJUPGrQjsb1h-93OR-N;u<@!1bb#>u=AFe&1fSZjqe8%zorTZnafa&+>G{omQyHz44X4 z-cMPjWRoebb)|>Z6gRXY4zo_MzKPvmyf?3WH_S`79)%=^fz4OKX(M~zS%liDU->dK z6VwEsM^Ghw-z9x_cV*c-zYv*DG2Qp2k02L}w5Odac>#zkhG5$h`Zzd=IGUxB1kC51 z9B0*}+O$vDNKlOZ#8FL2Bp-T3_#mp1u1mu*-T>noDCRmGrP3?)Jcsv|w2tF1Ms2C@ zs2CCn8{3C*qJ> zX|$zvz_f=0N`_LMQ=_ra&y0RiY5YF3ceRl2l{(3fS*(m;{YQ%KsXeGv0~! z4xE}QfSn6%(i}tttC2E2tse7s*jO1`v2IJ$>@915Zz&|iC*jElOmW;TErh8m962D` zXR3{N0nSTxCVvrQ!A^CCB+hOqYcvmaXSpOkk^*-Y?Hr)CpO^EZeh-hL-fPN&IMlu^V@>q=zwU^Fg zG@r{Iv3lbDXna1*nCSvG-kA`lx-r>5$2-=YFlK7JHcU{yniwgVJ7m)H=KnTA2=-Xm zP*&;{gH^uy=%xsz24HL|55KpWJ=1BqY(zIj*J7FS@JB(W}eB_y82_<(fQo`_xF>uaX zsf;7ZUZYQq2}y(asAW<<&=)CH*@C}atCt9$Dpi49;_rR2xU$d4S;OM1FosV#T9r z(Fg}qh}M$_n}`q&@kLJIEz5MNL!IB-E3{EW{6iD4CxM+XlIF!m{k+U2GM^jRpb~8F-f(z9 zUiuaFZ-Atk_vXfQG$62q7TbUUTM3~$if2ZClFJio);2Id>P#;+P)NM!yyKJ7$j^Vl z02Dg|R+vrg?0{G;@n0>lXWq}qiQo)%n z{PS!t^?il#C1nz8#QGgDT0i(!T`CW@g|>W?L$2ZFP65kf;`HGT#+vqwB`5sU$qM20 z5UH!L0fx$K)kHFHZn>FH<4IJ~a8x^oBZhOpMZD+hIKh3yA_LeKc-SqYcVP3a6FZ(Z zj<1@z7`;bt{01>Pn2D|e*2MGR915urLpeK;y2|hvhWBX;L9evUjZbfL;emPUxui@q zK5|8v6JzQhX>kjMn=DcDyY=(%4?k>zkX%kM3R@PNtgw4e|2y8O0T7 zY&>runm>&=8Hs#}6#g4=20}mive;~4+ZD*I18(cT4^+yI?hp`*c<=I{ zBYR%DN6q`cZjxT9B5AOIi6?9UdRvk0hQ^5f)S9v*Ebsrg7y1v}WAElui5rTE^%}Mc zIT6uvfmGR#niQy?f{e)AR1ky z0QXGNR4;#J_n*mzi`h#DKE!}ngZ#|U>u

xThpJ{@XGqVFwspF`u-1bu5j~)W-_Ynt9txI``wwhEhL^y| z_k5yAlIFZ5hSiyk-{fq}&1KXRV)By!jUzxW01c;ifc!{!8-{Uw5fVSvMIwFNGsn(e zZ0)XkFsObE$I~?hYSds)f?7r9K;Z(^)Qnc=XD){DEKa@C>h$DS5E&e@D2-)g@5p6# z=58_RBZq>DyS?su;qIjCbw5eWA+V76mUW++8H_MWNEKZmftBP3pli9u79T`lD5+&2 zUh@)*_n~HxI6qw98sq?yrst4xHbF zPde}|OMq_@J}CiFNHTx`%yyGMxerDb-ZM+5zL81+SkBq{+r|4NHG!`P(Rf&(?%qER zVIgZK_Kteng3$h-hzuIX{ho4?xRo;(qd};vFS{Sk1Odx{`jkjm@S37PqQ#dHY0M{1 zarDeY<+o-cV}&7F`&Hlz0k1&MpTTE`3jBe-Q}jP0&_E7h5p&=e#^nG+OYRP0ww?!i zrE^?_Kmrl&fX^Cz+7|&jhLTNX%nh8!nre*N;n)ya_|1+IFf18RH_Gl|jrMzW7HE-l QKzI&v%+|@K7IP}?e|Xv(bN~PV literal 0 HcmV?d00001 diff --git a/docs/logos/DataFUSION-Logo-Dark@4x.png b/docs/logos/DataFUSION-Logo-Dark@4x.png new file mode 100644 index 0000000000000000000000000000000000000000..0503c216ac8480ad6ad45abc57dba291c7962f71 GIT binary patch literal 45176 zcmYg&cR1Dm8~4XfGQK3)qhn;28KILA$H=kEE;D;%?;??T>`hS|q3mNf%rY{QW0Y|s zd&UXR`|x|N=lR3s()FJA`yQ|Rb>E+&k$2Uo&$FC|Acz{FuA&D)WV{eWoO+HN{EKW( zJUjSB>8^g?3xb3-2tP!=0gtsIhzmlf+&1vbS(yrbZnG3Jvp- zthUEnCAKi|{mFJ@MdXc>?uNjp+~?m<*>m;yY(>3hN}759n(yEXrCp#(d-YwbyR6w< zu!kbasvkkA-Ag~ z40T5R>)w<#_Tz%`aroKMGGpSw3rW|XHe2gEsr|GghB}_dPNeT#!)^Q-`H{|zBLTN5 zV52?98*cYJHk!9(!>=1pnhY%qi!T-yt^3Q*-*;SnRB`}DK>`CeldgY4^S>`nM^)In zaXr7S$?hMYz#A8nPd@hO1KyD-PTwgR&HwH8CixsCFA)TVUGm%LH*Wi}sMlj^8@?df zz}6&|7FwvwV||Ef#m*Ma%-gpFC0%z3v2-ab8~Jg2H2HT{cgFjoJlrQm_41Z;*9fwP zcmv)nQnX7Upe1@kJhSJ|9i{a1E7Cwnx=)J=C6=6g@QZ%*lPONoVbCLqREF zTFJS>dB>>UM)puA)=b&A$BIg&W+8tzl1X;sX&+%?g>R&>J z3>YF^SuYYfVd#Q}7vl8QB9wvs6}TU3Zma3s2Fp>=HG7)X@+5C9EMF6n`j6?anL#+=aMWN-^MpI)N784@8oV!ci;NE;N%4AY(NaEDy^Lx`SD`p$MDFH zV8l$yPIy)EQhhjWXvtfB8GhuwUNW42*$Cej9-~n7lR7Ns@5+ha+;yLE!*qfy9TNK^ zQQLY>mW_$xcgurIHi#A~igd16_d7&9fCGE)6G#P`@}2|MycpiDgjEmzAL6!;UaSl=;t?>%&9R7C<6)eJ zi@Su_rhEuNx5lCsI{C=rGQB^yekGi=+6r~e?5a6F`S1d3O)TpQL>*|(R8O?~_URbxar5{Gqg z(`YBSgH`5tYiE?NmE)@V(C0bKenTQ*s}ZI`8!~M4C_I}iS>eF=MZq&3>mU}aS%ss! z#DF2k!HV?{uWh2RILr&XF4?G=Mk;T`psud}i6jD9#pCMM9xhGANBbyf`272>?>0|y z`@vexAFe}IMl08S%)3@q=?6A9$-BT_5RzD_=e!HE7R~*%F|&&IUmpN*j5l|tyM4-5 zt|N!u6@RyWyDrTzEV!YW5VbM{BY58uex;rmjFtHGY+=(wX5Zuk;nZnZVvbMQZm@H6 z&wHrdTwwBx?7z;V<4KiYvU%lK!JIG_o|Lyl#$@#f<2CU94?YavmR0hen%;ZwOa)o# zE3R|)yr%GIlL#(r$7gd8)E|C10wwiR8h>kMwovEBgku`enyP22{$`kk7*>x|br}xY zrS~FG81;bxjL($g%l_tOS{K0+A+IU*1jwdZhDoMgl&6hg42;!2eTL3Q&IRn_IYwol{A$hkrrYM@O$mBcE&djd8SE=Q@Ovvw9hTKT+?e+Mv5 z;)Fkw8;cn)dSN11bY#eSUhTDiP%MwvJQylIpXftIGuy z=dtz6;-brc!91u=S>V3DYunp5d9;&^jIQ3m_lQ(!cn;&{?J3(eT=P4D_8YJHnkM;K z5+uhGgRGkFpEsW3(-paI4^#)%zF8_jjI04_s<-pOT=gKXOF5vzkpNTR9p`wRzKUqG z1}Mfl_ON(Pee(qp6Mp%jPu{rj@zQ3Zd1E8Vy-suJ=2zK&(zVi*l8orHPy-Y})|Vb< zF2Zaj_&r`{d>l3cllF6M93{TTw|Cjpplg*U5C5EzjIhb>Io|9Czh(2z*Ryy~TW=XL zXzMvW&24VN?NMTAANi43QRY)waqsn{?gSWqz;r45bdWKDc?F&kC~A1E%^Z-VvJrzz zKtalpACSVLzvcSN{9*Cf>srnpkp=C?CcmIRphabPif2g-7EEqvX_1CR5LPe zXG}m2EcKk$=6gN4s-Fl*ix54sc{kYx^6rn8%|+w+x?)Ja62T?duYw~bc-7;qnZIWd z5P<`oQjZ-_xqY)@4#8P;G(imwW$_4&0CWGqKbwixKuqqyLD{xe#_^g5$lIjPB!Ovg z?{mDYHcuHu5iA#PG0a~KjBOEkpY?n9xytXN25zn*q!f>%gE$< zSKBh`KlG(edKR7k9}ZrAW$ZB!Ttya};PL%9U&CGLA5!t~6-U%rTm zS>l|;DRTWnryieZ+;03jNz&V6QFl=g*IMs@?H8@ z@5cjOS_q%N0IqSbd$s;MaEW}idvddngEr2bz-2J>Aa7?E$a^2e<3D=ZFkM|zD-4)N z??5wcc4N7Ew1>kqm$2>vcZ##XxcP{E+uLT%nD>R<8CMo6`5HY>p4HASdIEFH!=I21 zK6K4j%^we-cM!~eX+lT~O0EKR+D50@aJJ6^tw9vXI$zmjI8%^HLwAuF2C`uwtJXx6qK=_v?1G(PDFu zBOJ{RJjLX9uYX+aY>6PeS&Y2Pa)?3mKqy#!+b=?R^?mRv9z-0EFj>Oz3}vv3_mcbh zHpiE8QVp41$4=1e|D8>w)(6%mxIuz$Cjh+-U^9L7KjJH15}lzKzKSc(BA90~#=!Nf zc3$}Sl^ou2HuGZS8Mf;KXZcS4bLl$0Nw` zKMmr^jZzwCEiTzlGS1HlM(fXGO^ylxn810^z1%Sx=XWMY_&MI2&x#!)V-z~2%d^yb zpBUsnlkq63vZPhC5Va zh{5+31VwVCkji7oWdH!2L1zM2h|+UX+xlY^INN#)%HrRKnl(MVOeYEAfWczi8<>HC@!u z0NLN5Qqpz%UO`U#Pub(Wf+N^($vKcGHxuoM_kBZi(;PTVlS7_PN&YeGH%_Y%l0Bj1`}^%^H@fCdei zOxV=**eRhB(n6c<-LgD*6x-EMsc*Jaw`O_&=Q9hsGg^zD=6rGkOo)4RDFN;Yx1M>R@ZlP2j*~IRP&>dhTKPXD|6F$MC}!KpR`;_7qBMuN)YLb2f4@ zK`fsP{Rh^#{e`ab5vIl=t4#mSzULnI_I6us|N2?P*Ojq80h5ATlzf@TTC-s}lnchJV5uJw5my%^l$$MadTFy>vh^OL|V}gSp z$%qj@t8bsz7bY1G+!h$v?#mmAx;k#IE!g60X{^eLKukj#}c?=%Gy z$4P6-cdo^B(!p07UXXP;QM4ujzhe%L;crQTbP~NBB=76RwvkxP}@o=v>!| z`>Pu!`$ z6=tpcN(P7kPganiF?cTqu2j_FgW9Kx zO+}Cj^#hmj^P=f~@o!A%;VumW72A_E)L-%`A?To4f0y*#`*7Ufeo{NKm|$|?Sw3pL zsyob^v-Y-klPjo*hph(0>hM@A^%Yac^H_6K3Dd*1FW`Z;8_g)0qG8}kCa(dOs$@PtQ%l?nyX1k=&<1Z+3yH8v8 zRU^)kiiI#}mP4tB+=Ej-XkQMNvI9;N9{;>74;tjoIhQt<_)G@Ukj}y5%a?a2PCOR} z{)rMpSfC5St%fLYH19(z0Awqquf!^hBhSJ8R+3E#Cxm{J*E9BSV*8j%YsJbk2I#AC z+}>GCPyyd`X#jC|SMbJj1yV1N`Tu)9{CZ{ zFd><6s=tzUJP%VUSsBzRur#;3kF-D&z=J9(s+%y*U^t zH@ijwNzxSBkO&{lzNg{T`UhP2!L%q2-x3W*g~8g+;|al_9V3}GneG1zqz+8yNZ@7m zPLoAqto#Et1YtlrihX8?qU6+S2D^$KOjj(JiXiN*3ar+Oyg_nmx_4*-pvnLq1qNJ6Qj$47PiqxNG)`8|na<)010>b7nsjB$yW+ z5!QDUvxy-V0Nr1k-P^Ny8bw2ur!lFkeUY5nEi9(N<<^$PAaKVK#1zP4 zzYi{m9#G*ZxE6uhtwk*KN}yreUSH;d@3NHg_4Y)v0nbm!H!pVLf<-8$qSG{hBX115 z4As>YU7w4f(uBhJuB5yLuv~v?9bMmFUV_^3F8VjBVocgP9i(iAjtLXNxQ8AWd8fj@ zYMHzMLDKO4j%=xJnIDfesypL4lQRVA0~TT~&4pIR?>Dn~VGDr;fO|PqgSdsSB$@>A zGH*NZQ$o`X@43xH0z|%O^9@&2w1RM6x;K%RAx1!{^o2I;yjyOrUdbjmg+l~a7w*z; z8@-xFKQMmzr#+G#cLtn^wJCnbe76=}cgo%@Pn&;26;|FbP#)+dBjd5Ox9~MT%21ma zLUMxR&@(6hKBkD5zwvVuH#+;w@nzh>GY*gE2*`X^>FA*1Ede}H^2&G-o>IVgF5UHJ zwHJq>6y+IZ5n=k#z4?Q3e(_j*Os+ORP?bmS)!a-0e`BSgIRYELEy18V;8kI>osG+#-l+%6b zvw00j0M!S))uW=CbPnxNilcV{()PA13Z=a_2-XwBLtn0r5qHq1r89RrC2f28I{I$z^uk33jCl99-V;W$p{U|NR-7;|??k zuY8RDhrh+N+HUxW*!$W&cT=Y{SGCU1N!P9S=qSQmucTaPH&nAwKfE{T{h~QZa#qFj zW(5X0ZS~hk$PMppU0NGjeh!;#aDFvpqD} zGEia>Y=SnQ3$W-u9LhV!smS^K81>uB~MYfYveaAF&-V_i!sJbPBqMyxI^+{tj4N{aUI7C&x zftBmiJ-{Qe4VPbP=)#eTQc6)a^MbVQ3#ub>nNDk^DuPvNF!BgyU*A&l$%zv}5-IhF zV2Xc47Y)phyIV06_Yh-}TbsEzBjTSG11%>crIIMBrQobLyy35()a#P9w%i_NaR2== zuD3zKp2_fILT@g@y_)e~&VIprfB+kbix0$N1jE?Q@yg6k2R%czj`-x(+M7qis~i;h zgAEXA04H1Sa#+K2fg)C#)L|>@u=@uU8MG}^77V8a`>?0VM$?Cu*YYUCTyehC+XDn zJ|DO}GDS-qCR8ZDMIrf`#D`{p8j`N#dO3gZgM~_M$Y2MNJBSj5RiR2q8Q((>aYe&7 z%b(@?$fhKKK1}W<=Sbhu`n-NsvR#Y(orBK0kS!B+#qR6n{lcp8t)!v>9&6areMgbV zOYMN6$T6e9=`$KL%KmvZVuId@0dBdaNU8R5^gj!o;TJ@3d(4a~paHP2$M}feJjIKq z7*63@9GG7q>D%>>eL=DOtQTPNk~xx-pw5-`#5YrGWW2V^t7i6@oLDF=)aBx0{TGgp zZ-5FjD0e=rfa|{x3M8vnxfvgeN!!81(8An!TXeXXwN4oc3gMq2+&aLZKTivX8K=V7 z>opfeT@Qk00xOc`Acqd|0o~|fNY15EJKHqtNXd=3M+lLi=fwGXtka3O=YBQrCM8ay zeexVH6Cfl!`sGQB>_;n$`nSqCACgLLfowSDfiV9!4Z0C!4{ zrXIno1yDdn%c&0P1D1)Kn9kwTTKrav-Er>E9$jadO znMxqYKooOcy*i`5xbjtox;W;nH=9cCre2vt^M|c@ zQg|!q6In7scYvY>@kfZ)oN=B@d~w83$)c&x?w0-@xq%Vafn;sk+J}YbfW-UH-B5Is zO{W$n*X13&a}b;wkxA(ZxCUTUg90}>o@2 zwP?StJeG9!&HGdomGqMkfk!h({-KNOg=tne@2g8GGIe@qP8NQ*w3aF9dOhHBG_OTh zY#wCZvTu9oSUlDm8!Kykj6MQY7Ev=hAq}OKxm72F|9>~f3Jf-LdQIV)u1{-!ngyH; zWJExxpJ)|Dx~8;X;?Q~O=_;KoLMN1c<==IxMClO;Uhz|Lv9iCOlE`(wVLo3O7oa|0 zpW0nolF^LZEu3HNkDAl%@Pf-wUz+OfIByl`+JSgkZ=8$Okkc$@H@e0P-Tjm9W^Bc? zlw^ulKnIAk&+BVHzAdHe@^`KH5-1iR)z;7<1ezkEy_ExN9~RxE9Pi+#|6-J?IRx3r zh(}nYDx?cGh}40O1&>j$+f5DS*k`?2-@lTh(ouVqu<+&oG%n#Z?q8npWu17*qByEz z?k0$2bqyrRr=5;kivc{s8HM;RlMtHzg+$f%gEPNJ(>{0xeD;*&Jx_eeXITS)1v~1Q z40T9Xe~KH`CXcG8&&k?Rbr{TJ;Hw1Vnl8@bvn6fheecqgRY_EO(N0Zk6 z`ncabwZyB`(|G1T2PHq9PY7OnqFdm6_Rh+wq99BAaYg(!C$ks#l736e(0pc*qL4;d z0O-j0sxqXHSZ^#7eiegz3M@HIu;&ni^w|}!yfG*rCbwCd)AS1++R_k8o1eV;w8A*N z>4+`g(EYcmE^qYmg*5DRD%I}@6#>vfQCcW{SY|_O^h- zE8jg{{*?Qf{Vl7^6-o&P=iXrTjRWM57qQHwSu{u{Mf^NpAzyeoNLzxB`z9Sr-afF( z{3TunGTn#uQ5Je**L8J-_L7^IkYS#M!Z)@iqL}>p4AtUc!Efu0`xW%-eCJDI{E6-* zAU$g*xV9cRx@cj1U4Y7p-Q3}>d<6KP8={W~CTADZL{yf0f?W&FV?TkHNbf~Q1h=|P z`ZaFGp1ZufhOvS4@jmwN_kumSw{1&nj`MiJ@Af-Cd8-ZD{#9 za;ufq*4SX0|G9I@8TtPmHHjO3BS+xo9Nq(@-u7l$1cbsr(zta=LG|~uVQK0MF=J)yyG>BWV(@9raT6RA;+pqSPU9s1?HL##y zNhY`O;il!$av7Rm7sZOOwr!iO3ADNZ!&|Z%oUL%}%<{YaanxQ&?V~GZyrxMObBa zXGG#2me$ht=20PiD-z%@@ohSlunzBkEzHc@dKIYTaVhtio4pq8F4uf4Q9F8W6`~g7 zD;ktB`^baP_)M-gSK#LKo2f{ML7Z&c^_ogc}{DgVN{$ek)$u{5l z@qL@GtyK#@0=;~-VeFg76g5<-C@>l>v4S|nuBtw_V-?2awv**9>{+rcg>d=vKFWvF zJ8&u?z@UQZy#AwLU9XFhUyf0eYPNS@-sSlpx1_gghOFq$h;mIBWir8<2ex~<6y%;e zEeEpz_o@+~GqL4T^Qj71^(ECx47LknSL$-Kp&;R6RAq-Y>D@>{iQipV%eDyRVzgDM0# zWEjW*E~HJ3Zz^ny+8t{NOgxnMQhUq7eIHXqM4cbz3Th36QCq^MjlqV?!pk_N`&?2< zk4-ks=(=MTexP;)YNcJbj+2I4bj$K61}_rQQZNDfO}Ce9)=s7i3>to;qG|61neUWf zS-kU(Q~PRUb{^|~STGtNK3`r$YjRm#<9_mMH6S<7z>Ve6GBgg%f(qcSJvJ|$OE+M( zb(TvJTnJCiWhN2p7OisITXK?llee~%hCssQ2J6svb08_WAv{B9q%-Nv-4QG6RrYr9 zd^+aIq$~>Cy@UKzzwtb|eLeQ%!=nnv1_I={t4t_LB%iOy2S=>R@cI@7+r% zTA~UozSjzPD+u~2Ld9FDk*i7q^1zBi8mWc7gyykda8I5`a4t$K_ESBE&bi$ekQMc! zr}hwDx*`57eqL&RFM;jR=5dk?YH#TsH5cufzJCHp9#aY4sRmocCo?C`{|&H5QVh&q zd`po0X;PilLBV14YzO9MgF6>9Qh_|_&!I(VDb7U1CNIULyf-Irgrn`FN zN5g8kjm_6@Un6rGmJc9pH37o&KAt@InxF1H*x_RoiJGh8`^J@!vZ~pg7a>K*JZyX^ zk5iO+OQ3eIriRkcNm_u@NQ$yRAOCO5d`gKA#5-+0;Nr3sn z&7oflKk8BOpq6IaU;41RaJ~eMfLv~$>PEr%7X~~aa?iU?EIb8PW;jq>Os?trW?S0v zi-^v#*%y+Y+g-#6M3$(|EuHeXvYc~+s(`c6;;TG&Qncg9q%)i^phsbEvL;F*mzm;kjR4 zL^RqADZSJq)al*LHvHs#CA8h9{p7ZBwhzeJ#&6hU>LP|6-gak1Xe|6v?Uav=#f)u@ zZ^RV^6C)NqOrB=$2ZeI#NbPmOkGSm#zEXbKjsrj zdi7}!Qw6Wu4)+K3erAc+dQ9g)+6Q*d0L%;blH;0P)^xjZXh(1as?5CqUnbI1Q&!HI zkiU|mD%*^KFIOoRrMq}4^>^RhKe*UeFw-Qeg-)yq?b%KtFIu8OKYEQ**&fBV4SiynSOvb z8Y*>KiDU*~DRKU4xCq@CNW{TRsDwbB+kBNkB;eLGY69Be1wf>_2roDV2JTt3K<(n1 zDpm8qa+nMxodyaN1wv>3<*l@V5B;q&(550BD%kfQ^LB5rc~eE3dG+#1X7DmyL|}QG z3KwMo(V6@i&6H6;pETt)@HXCa(HX+AU+};StzDs0`sqoElHo}tMP4~Is&*DoIPPU} z{i>;vr;#k5i^nHx*y-jPrV_kCxGP9;^)9K4{&}Yjmi9SYI_MW!J-bS$aAV81+0*Cy zDUsX^hy=l0tn)w z3HBh|RMD7^M}ju#Y@O;o!^j}O_)4$qK#oxwgow}826%JO0w`k#f~pZ#`$mKk@T(>a z>L3p%MM2AF(Ym`wq$zhsNEbyjmFF4;Q!w-33_N@w1gzbh%fI>IcYjA0v;J7fO5*gg zVfhlansw__9L&{%u}Z#gP4cuyv!Xyvm&_e`NnOy%p;|EU-*dPp@E)(aGmO`SDgqSY z>1Wo!fz|6CCiC5eQGdy$c62lpKol3{7ZzDR>8U5~X<_`t0b1@-C0=!Al{*sLBy_f6 z{m2iMq0uKz9F-F~bW`up57RUdw-okg3{%6)L63mgqqn=^HhR4gSuWBkYy@um>``7{ zGSt-2`zY>u3-E6BDO*1x#k6H+3WG1DqC!BD1RupZJuG@#I>uAalelyJNU>_24|)AG za9EL)){dj{L>kK8EQIr3Vr@KGLmbMB+cg9^w3Jg7Ek&%XzFej>=#d&^dF0I{%aynz zu6yZMC&=l3H&Rquw4ehBy-DyZ7|8K$NBD<8(FPTo?WqWTpo%iteV7G&z6b`-S6fG& zT2Sm!`CbCCZ1Q|Q>s|A`>ozRvTZ$wFNhPN!N4nT{me2PDD2SoSz=_?kX z?aXk1q8+pu5?&%)R5pN7?zSzD!^3f+eVTT~bLj<&5fyhRjDV*}gO_1rq%ZG-rp zLRmZT?X1Ak9&k4KgPv1VZ<@I%s=+qd_-0Xr|-fQJIq_Z9=)7f-q5}nG}Ul^#~7=Y{A%^W=*D=QQ-lOYWHrBdS!mA3?LR6OCIt1U@_Mq zIH04-#f;qv4qxX*hMpbvhO&s8`$zG$z5BIlvn6j4HD+^42qyK`dlIp5dN__ybO!># za+p^C)Upg=RI{FPOLwzd!3Zt^p%uRo>Qe}=lFO}W8iO^@hudZysnmkkwo5o87D#)^*w+wy)tj0fQG?dPQ$3!ASyHvE6{f}PTSTLLVSrZBd; zeIu~&O&pVdod>V4hx(91^1_TQcOq%nKY{?-^(t7zjeJnpR6%x523!6fre3AqA9(_= zILsPC%NM>h>;tq7z+<1C65^`+H4LA7NE!X!ntx@b43i)E@}g>c+ZT5$0s3eT(@H*;GKLm#gIFIn)(%y1rgpQ~rPhA&F zsq=DSiebAk>Eie*={W(7@HUA&5Hc@q$ajAb@sClxe7}~}w(S=9=56SrD`#h6UD@vz z)MJA;3~d1Ni_l6v@r~_V1cxN=oz_VRS-Ls%{iwt8=;6}$lZ9KyOgmj6zbc9{xuZ-m z=l61xqxbE-eR&jz4)?DR_ds?^Jtje+Q}$&NE^WUe20lBGre~TtY;Q-K+TQQ!_=6M| zUsRfmy7s$bY$e#h{ldOq{^^`pEzMcv9jR#n{B$TG0j3j!Jsxgyx%Uj?n7l>ZG9a!{)3;)wNXFex z8)5q+?)DALM|4Fe>mkA3+HsS;S+-cH4WQR<{W)d&>eWB zhQMjot7UfPAfq2*nmL*(9@yk>RppwXYw9)9B={`fpJ}+ZJ6zGcN^jv9_fZEpJ7XrE zO-s2aBwQzE?N=NF^7sY-a(2In5d{in-|2NKf17+x~mK#B?AD8X2Y@kf~{2-79RcUCHdH1$5BTCWjKc(QxT-#Bz` z;3O}^aXh4@8+Tk9rJJM^MSnEp+gf;Qdxh#~c!zc@SB62Ihje=T+i7@~%qYugG^(S$ z6t?)n7+e45tq0?I@oFu6N87)slchTYo2)ZaS0CRkiLyNGm)UvIcBq11BbX!@ip6s2GBx`$Q$_*dIr z`Al)O9I>^|gPI>_$;e+1O@C39?MUgY=Y6X;#cF+iKk2faybc2tB_J$q`^%`o^$t<} zDN^EBKkui>OsBzux1ChkORv@U3a;44N&Dpmx3o;DSc){Lr^zJ#8#uw%?`+Q;ACJRY z2%4e6vX-NTY9Ia+(Vd}Eao^H$^{J!1KGmo0j29pY8f74gxdV6z_|t}kA54xPqVtP{ z_Gi)q@TC67>tA-J#$UFrmu2>aJR{Y1k56f#xpc&)lfFaUn}bZAQdn0=d5nJh-@U@) zay-c+YmL$vLLuy(vo&sfX#R>zGF5NqQ!- z1Hh+j_wL`2Jum)&CW9JQ8;~$1n&~sgVQ`bWz>nbXB~@&+-a3~)NXGu|qui=30*zI# zs%SU>E#mV~UV)jTq;*uDVUPC{_<-ZP%eRe96gLGZQ;oAx2w@f0*{_zxzn%9!S@mse z3rTD{*dO!AES!_O$)Ehb>HtB7zGkl83?f-FIr{Izp-bWOZ%GjO;7Wk!^`4>lE1P%V zWi|?Pe7+x4Lw~A%vyD#UEt!3Q&KLmwCE}mDY_WQ8-e1bOsMdKQC6`pE4ZUW@VH^}R zqUE@nY1-NHwrVBXbjUdS8v3qTt;NgsRR5E%TUFfo7QJZB%7tmuNn~Wbm$-lF=z|5T z#kLPmd!K;PW)zBMJ#Rk`aLp%e8`~-Sb%A~ZYJUhp73t-rnYp~EF(E#rJFEz68hk8y z)HGS<6+kbK#AsfqT&DZ=GVP&=Wv2)N6y5N`_|WXhxpf0xQj=TnPG*MiAm=S?5EqHCF z?(_PP43!St>kDZz?zI!!qWY)3;Z@1AhT3W|aC>dx6l-Q*oe8{(*;vl<3c(vrAOaty z4&3@X^>``x*1iV@X>bOg~ zVudu1SK0)4Qo_xo9&d1j`c)cTHTz0`iqjr_HEL~UUfu2Pvlr-gIX9EktGx?p9<(*h z);j?0mi!4sH7<95WSw}so!xeHGh2SeXzf>jf6x5O@65H?RJ;Ay@y7#$;Z^&KSi=-v zGE_AUjNlyiALDGqg?~|gyRU-{JT6&|h9-GZRFZBK5{D6w2q!k9z?Z<22g!EEh|C5t zOv~|W%Slo|T)%nd`aE?tp>f52^=ocX{Ff}Gc}lJCJYVZBgJtqg({LNT3i)hnI#X{N zVqGY({Ott+p>r^M1&`ayk68C8RBh?&*5Ir`#h{x&%Js$rvapXirHAk>jnnGg^bC_` zd5d~E_yC<#;}AY{K(4c0^MafHABJJTuOrdi_Jh`&2HyAMXo19kaqAV+5?fuZdsp=8 zB=O(Cb$|Bv!1xj`*yC(GTfD?&r(`pCcv~u0f)9Lrv;fCEl2;`kRW$1$m+A0CaE9 zw}DOtxlIXgAi<3_f3c(;C0@zu`-?9^x2^zBU>dr95>f}Yf7ydL03q-S6Y;H-K+6gQ zm+YVSMNY;)DgOqn)a^kmWnUL2Wl#bluK835O4~W;ox=FnmuNuQE~TKmSHFsG#>XxR zIax9M332cgdepwI<2-9nc9ZTFD3&{Ht40cDO*k(%)wg{KbfjJZey;m6V#eH5*YqKm z>}c|iY@qf-Wnd1$iaG_Qz2|uI1t(v`Fag$8Q>F7Wy5RQHPwUGNwrWKm@n&tNetj+A zs_5?r%Amb;YGBx2=vnO^Y*b`iQ9p7tsrncABo0}^INe}eCe*YNUM<-Aa* zks0IsFr%^wrp<;u&1nFyR==b@>ecS(s`<1we171#=hn-%oi{>Mus3@tBDi>f-vq$N z3rW{20V!>$kbX(_ao>JeCnd9n1z40QpA!n}^vbPcNDW0DX-#bX{DVrMDu6&O)t2c} znykC2(kXRsGP+iO^N-Pefq!c=$AKsAHI7!<>EC3+1F7ZvukZYGJ3g2Rv^>}&p>Z)D zdI3Uyk&CuF_BhJ_SYcMoBmZjn9k8Fz!(@=2*BSV{Sv9p4V!m5CM#3UCW@i6WR>zFj z#~O2Vr3*c9oP+ENyFatd(5`QrHhKVB;*anr4-Lzl-s&!Gcz#N2AG5 zP{g!8Jt&R6D|pCN!-@z_b0S$qnFPKbH8fX9#ark<2RAOlCSIvYyiw*q$0G(1bJ=H7HMbSmHz#5){Oc?z_*89E@4I|U(@-Wn)$&?xT zf({7wqFK!YMNWrf{D~K0Y_Sd(2~2>-qIAG5;U#xc6#NTj+_%~8dzbf&<&abjbLo#~ z()Fl35abU*%6oT(6{f-^TMHr!h`h`i0mh?7?Hp)TycSPJOP!>gGXC_Ctwlw1Ai-o$ zBCxy@Sr2NbEQbt3<6!ItH{ygaeA52pxW5)v=64WEE5Vn6j@LV`4^P zI%VXPWlqBqkMFM$p`S=_uKf$CEqm#u3&$u?Gv?C9;=%fWYJ%pGiC!8t`NBDeP)lA#%kpjmpRZTgsE3sX!VqOJl zgxDhxUWO8WCATyZNe{^h*6wNj$@Va4Y_kFGGc~3(-i&D_zCI{K_$RYExuxtieW?TW zQ3Gv75A#3as;I^_{tKs)iNE(#rxi5E@`Wc56)?xw$g!0=hxWvf^t^FhS4a1-qFn*# z$W2DP6n<+_Tkj;$Wk`_>I4@x1SYxvVvyHU8Dut-QuYwT%J6bVAKI>rn9U63xb%6!{L=i0zs-s+> z4eHwCuGJeG+awMdBR{ChY42!ET2_68dp4Ri{Vo?FFI%$_ihC% zpS^OWuBul%z_xc*3HPG2VEcQJ@3}=~6wLRbG1GvuwX0Q z%a~E?uH{Um?@cvsAH#2T?Cs(NKu3C!c{u&? zjYcHqw0sKN@GueytbF+m@dxB0DPoDusgDiDXQC=@?!$-td!j_8GPKH9hutgxqxb#B zR>DSrb`Ls;^6@dWC(xF9xkB<|jNQFLyk;XP1Z)H@_ctaPt+UCq&YXp7fMD_vfcDd7arMVg>K=RTW9PE+|kf6$I2A5?rlCNp)J zv7v2V{lN?gE|poGhYKK^n%0j#Eb7vWGg!lov?(acihi3$$)Jm*^>hLL`JvV9ksrG4 zIEq1W*GGRE_lwOJ%I=!>kZ~E3qKXX!{tr!885ZUDw3jXkQF1AjZloJQI#pP@L20C= zLqd=RNs&g7?pQh`1*D`~1VOsH-n0DP{|BziFP?p#b7t+MO`=MCCpNbD$#O3Lu3B2^M9D~56@WC=f22^KyeIl0lET3&5+EF}Vo!>MAKDto zgRf&n?M^4>7V9#L2^xt0TZlKw$N990>@!$Q{}EYMfc@Q(2k76B7b;n`8^uBd67)pJ zF67wN4dU)|aqhj_k}QI!HTqr*zcs|A&wCxz`xN@*uI`8jOUy0^AlE=b4pBGVZxg9r$wr=>RtyZ3;94+z4zkrKo` zn}djSr(JdH83(!`tcG|dr*ezcW*>|*efrtFMRIyyfpg8T<#6jtkl04OJ9g}Ri92p+1AGv7eEUGM2sxJ10KszvEJt-d> z5-Ti#QcsayoMZhh+h?b28Zpkt_RC;$l)&en>m3AP=#8Zz=(GdV$sh}{c^#XCerJ~# z^<#-SY~`lZ|H>IjvMRjXilH9R?8$h33h92o}w;b zKH?8ThKWp_YpEeo0~CN7c%m#RQo02QzRldPso<(s_LEB%YX&x_0(<${I&SSCTq z{Bshnv}@!_bnCrZM4w-D)v*;sspfL$h}Mw$& zNS-b2Zj$*Odqi0IxD?qvp#cZLZVX$*2X zD@bB33UkV4i#MI(qTlJ|MM%c=#7xgsLhE73&4lAS!IpZhGu({F$YI_2x=)YQ^Hu0{ zGrx2TW~D8Fmr@oYK!DOi^Ne6y(c+%>=2s%5wL%~Ze;RgLNS*A313bofPp}KXYbYn39*AdwLvqmOA8V#cV_U=%P z;ds^Y%JCLnW;K5kdv_Qk#zTg4^I!nL6UcIr{q7!vpT>niyFd!(w+qMfUg#>Gkhy*G=7m-PN?{6-?$UKFQ+v$muIL~L76chV2G8SvO;fyn~;GIk~L z4NV#J6tHJ4ekbZTF*QS{fbVyN6vkZts7Po@D4V*5hk_9%XOg zs2%phTJ+iIXOjoqy<jAD1^tMyK5mKKoC@7#+*Gk(3R|tN8o7XrbwpPz;IKJjHc<9p#j};Qsd|vXx z^7x*2ZrpdhC4TqUKFNXmXFpzSTqkVRI%QNDwxQBRcXV;nYq3x*D)8anq-V-$@8O6!aJd+xW>!U6K2hDrP^Eps?L^e+KDx z*JTjiT>rdNQo>cRm3thKi+%h_NKnIo&XWzbmz8qN$7hAPNuos5l9q8$x9pcJxWEjI zw{sP(;uo`{npl47q06`(OY_UU{gNaj8e4iG^aewdedbf1{JhHRVvd)RObTz_&~-;yq?-J4?e=HwdbH1^(a$Uy#JQIIS*=jC*cH7UhU_uE z#q94KMs{9a>3!2ffv8gmq&OcI*pmB6w=3TgtbbV;vwp;7pm;t|+Hn6dF*i`sLf-$nRZ zbWct4VXDT`biywUuul5-9K$atDWjVGJx11{QG`#Hp1nU7aOiV2d4vQ;=R_~bM8m3- z7Ub(ZM#thQ`s40I(Qu=7{IJ%odIRoA>GDeafdb8QDnC^^UH>J1iYjMMecz4>(;}|f z?Q7~6xoCJuJ2Wx3;M?*0W3Xst2iYFx`c3tIn(*bWF>@&=C)_c@O?FO?8WNOANfHEDn|bj@;F9o~KmaCPD%Mn0SHTv-Yji2TEe&k498?8}Iw4 zcMEnCuZ2cjO4P-S;blFgq-zq$phypziRDl5^5HVhPSuz?lF&MAQj>mv zwE;`nwn(s=5J2$)S4Mwy3}Msjn_3B-jLggCv6kuD<#8dLvhO5HD_|aP_Kv_hEeKqK<>0eGZXA)MG+_6}oAWL{SlN5g6z3jnc_%-C~hW zIKjh6?X~Orbm3sATcE|09P#K$y9yn}13BZ|1ws|3;J#_m zG=tvKr2#a=TR~~#lgSF1<^30?NMs_%oiz@{NR3EwgA{4nY@QTJ3@Zq8P;;tI>-yw=`!-c z*5#=hxYi@|8u~+8Q(`LhQa9&*w4e|d+FJ>^552W}NI!P#Yvz+IFZR}k3As#UNT?vJ zYh}&Q=>?XF+Tyz_1DIaOg=O>;g6;e!&dK@M!N%~IKWM#1IyfPTGooo?pXW&(U`hf# zQtpdPufQsI-WI{b{7>vMRxaFSK4=gt^P>2D;2i9ISVJfeoP+=vxM!LNS{m^n1;ZvhH|MkldBJD7Z!(y(e;z*qp|XYTAfrGxWvPeXoq2 z;EqvjI?5gf6X7y+o>WfEf%L2R_&qxmQE*fe0E~~meV7SBqQ{?mxMGuml_YWKaZ-)C zWiR`DL&jCiLz21$6AHgUX{1Ov`NF5@Km8O{kb_e=F6~gjKN9CGWU%aHjc|qiqvA#Rp2?G3mT68J`oR~`)u1f_XgH0j#CNbgYcOXMcmf=APMdT}m&Z6f z&C9>E&Z`N_X1XuAv;H!|@##^p6mij3np&yF>2UQMK29a#G;G+FB2Yxq4% ztb)2d%1h&KC3d4}hhK%2g)mzV0W445@cGlRK(q-83ivDtOMf@|-A{*Fm}SQ9dXVwi zWoETb%DUXb%x9^PzENJPSyy1Gz*EEuP2_hBLK0-2bYa9!7M%Ky;$$hx$t}C*vrUVc zLQmU(uZ6paX-$(M*1?Q1j?}J$@abaL2(8rFH||$oeiS0UUZq zi$!1Rfu~QY8s7*BC=n~!>FnN6RM&se9lc|1W6R-}x;S}$(Is~ zin;~AO$emm&=Wo;$WYh_(aM#IFTeQG5Id!54BR#>bUFJu(%<5X%C=T3ta`#-p5T5! zgIhKiI$DU`NH3YHsVt{NDJ5rTxqw#zs&Q=kif=I^b>GVPLgD4Hz9v4&GD<}J50JLO zFjEfGTOeS70|;YCg$9(QFHKE)=f(X(Ez@|yle<6g>=i5|>^6G&gN$QB$tU)l1tv;s z5JSGt%Z6cP;gcYo^mlJOHA9|K{vf+s4^J@Ka@=$7^AYfF;UdOBxr!zyC`A-?QC?k` zaF1JAHguKt0mv#S0_olPo>HYq6--}L`u~vNqoKvPJ6_cG#7+<8)OH!gp$BVL5fDC^ zp8|{a@OwQ`#IP@?4?WPXjGpb<^ArgR5{rblk^#dz(?zx87)z-B#Y+Wcz}xNjT^^+> zhd1gXZ7%;q`7uK-EIA=6=k&o^=|5-eu}OfvZ-j3^*su`xE5_IULWnnY@_94yL_=@c zdS&S|dhCxFEkR3w3I-sDjF|482f-Xy8krS@1nXH_V#dWDN&N!0fQE2Is~O{`nq@7X zQo{a|{+!BYJZAPewuZTAs&QxrwUkjYOxYdYEn9P>)h4jh(3D|*SLLwgKK`CVFaB}@ zJ8$B6DuC3+OT2dy>9w5fC9?860IlOO3O48Ro02=NIEyrXRTiVOn&pOPe~a1@_4_Yc zXljH6_4-;KPXSpEMZkuw7BdA^0Sa|}M_)!365g;O0b*aq6RJ760&NtUX;~adE;(r` z;f*(UdyX&+?31n%#>NC!pnb;c9&l&~ zHP6<=jAiUz%U9JUV86>8L$uhF;*Iz_DE3_`^V1tY4vKe`RL*vTP@GfwNXN=&G+L}F zKWZ-J4)eb)M614ekEkl7o#m$_BYCAExZ`WNn+7hs1AfF>8U*%&TInL-EpdM8L`iHa ziAZY;)blKtx_;osp(F7CD<oJ>??D}X zd%)^cs()euNh*~tlqE_;f(n*!iu;Pd^y4HU!B4JKz*Mk}P;AY!ou+MOpJq|$stha| z4#OLB?=~fT>@us;>xv){dzaK6qq6!|jAO3nWFVMCqn$sGN5LepHHoKmA*=!Uc86mVP4_~sAu0(I8% zJMiEzUhRlWkA?)-hcwbCmgRaJ)5w!1qj`uQsqR8f`5_DQZj0-OtxxO~<0{uGyIXSd z!d-&n0#KGk>}v?R+Y#TVqUN4-B+NhQ;=ia%7*lShg$|&7FgR?EKWcb;;UD*&Mf)8_>>i_hec7*r#XXOxLGX~Yl$^W)vU=O5@Tr!7GGuK5x5t}fgjWSz@ zzlm-xk8}Y(op-qNS_bdK3g6MKcXZZ20=9w!FS4&QmEXymmhF7NiZV*c3Q%i*ssiA%&PY_3uNpW4d8RbE9AXivG zX8zCd-1^fYtlvO#TpMf24C*OCJ}O!W#O{%uep8viN~u8?k?S6tPSMe9mE zL`z1eH3a{CZ$kUmNPsgLL<(Oq&0xf6qQLwVy+C4#weo8GfEpupS!24e7gcRs>49R$ zTXP&pn;bZ_*o!4OugtAAcj_%RTM3>|eAq>jfryWWX|)9)+9t$!-HJKwsO+azlAX&$ z_!Ij6yd<@0FRX?Tz9|18H65YYPp}P+>vD;boFV(t{o0LQTq)Ii5GDX=L)aKNR%WxO zQXQ^9+c*WbegFa7Zf&11V=N|+G-(3>hHt}*dBg`w#So3;Abw}BY%gG^CNM7DxYzAaR z39X^+GD-WG{uU2!?mBo$scc|mUgKH9_ zv9lBhy^4`k3&)21&@#rbTV8E$yrX;Q1o_JZJ(<_hl9w{0W~LrBGA~`5(65UMo|!MQ zSa*8kKvtd{`>|iz&!wRb22vmafleFz7nT9O4i@jn5Y{DwU*Adnccl2#Apvv2_H#p# z>W7~yCA#jOhLKXk@!&TDP1|gnKj!(#q4_Cf0633HLAqLIVF9h>0T;c#BTnQ-gJ?ht zPP4%#7><`MtUR&QZ}7HMN0)L4ohcnf&|g=5Fh5~1+?PTdKCEO4xcDC|$n>2~okyV# zY1MM*x$PVL(t5`O@I@IMcb3ey)UOWbH&rp#Pg0p;F_#L&H4?)scNOg(Aw$Xal>NNI zMe#&grI-dnJhfyl|GmTX1IPq9vp$dY`Jp3V5|x^nREBi#XRVkkt{LQ@K?1beYua_O zF+*bs`qM__Xp}^hQbqp+kuXs9ujUr-7hOm{NH(PX;PXmX_*)XDFgT_T4Rf;mr!=ma zzW}upe7Z+?Pn~zZd!QpHRo8J*Lb3VHIAF9t07j|Qy16$>x}>*~33a7-M9|dJeMYUB zR!JmW>?-LL4>D!=Y_^W@-OykLz+ehLW-te8SKNKfNE<>h59|Qdqy}Hq{+5deE5tjq z{|66k2#JK$-#J6qpD@MUG&MPxd3GAWlK6y?@*`%kAJS^>1tNLukzIfut|`qmw+k>+Y0WzOFr^ThBdyjU6zP=PJKR-NQ0VM+3`8_Sk1ND zPg!5mySmORV)L8RDo`=c!U!~}7t+nWmxX@Ky=aO<)&cL6^Dpry z>;H(_pOX2Sex8ZtHNQu1C`WT+l<$ud-&N?^@x#jCH-EvY`LyHm0-2>C0$E@{z&(D) zjCpX5sf+hbI`&#c_U~vm>S*Mtyo!n|;MUEJW0x&|Zqw2aexq6f{%+4VOb&atuB87` z?DwNzcSDs@=HodFE7d4!ty|#`HrM|2Hi7odjTfC+wXt5NTB`S>9vZ(kd(ItBoKzBp z%-D*%F%$pw`Y;~mkc zxbXS83;V+o?GyBXrr^`LmVN7~?q&>RSu;$@pD(D$9d4*=FJ3yX)Qk_E6&tJ?buR|V zM${dh!>k03b<^QTn{>gccI6_!i$Yu_xDOkI%ZY+_&AlKyt|)2@@Prjzp8$T_4YyM+ z!uFRFg_2nO39DwI4U`#uu>AxATbJFIT&j7gmW#jqQgNW4nwrlJ?upJ7AP-d<9FMx~AG`kekB z9gEA&D531eo8U%4 zaPiyd$-90@GKS-pvNMy|Q%oHm?-GDKIxOM8NXFp$v7vm-`gb2(GEw_CJf-DkqiV!3 z2<#Qz#vJ&YN?Lb&2(%@Ob=X=oFJ*|=@|e@ypn@0+v$!SCthEfqMb>^9hG2c5jNB3k zI}uFvP`q(Wx~FgZr5DBl4>xRMe&CCj{R+6=!$ zAxvCE3PT{}f#qqJ)CrSm-o{R*_h#%tg|i0sgr8DW3Hxu^!^x@FpVekRKBRF7?bc_C z38M|;*lt63$u0j{!PJ`kd|g9Ln7DSgyLQa3c3Xu_DaGayx#sx&_OXCMG&f+G#jTB! z&D(y7$OyfSp3<5j?3i#jdE!Bqb{T{a{jg+tMLgGZnjn>eGxLIF9c1&`?C-uRLxVG6J>tvp)v* zlx4cek_@1AJ}*cNbAm@{%PDbfb>?-BBfAY*d|GJ%1ZA5w6p(T@Ep}qYb!d1$_+Uw4 z0oXsxgR#%o0y|)p$up`NWDg0T;@Bl`*Q8JXXlAg>R|zDS*e)>C9{hIO8i_+uN_KXm z9KWgFM6gj;a9IN|A_s5g9ibo61n&jxlpUYIr=qdT3V^KScdtejoa`uz6-NzIHVgTW zWi@_~j2iiva5n_nSLs@!&izW{u|QHz1*#c%x7>7k8wR^&KhO`AK{$b1$3g2bH^;nDx=)WctrDPx6^?eq!%dT%)S- zv;U?_xa-D$sv?SdIGxaF{JUO*E0HZ;MO)cncxqLIFTgsj{AqPwoyoy7Y|p^SV9E-f zqL|ue;)-Li|2nv|^n>`3LoalLW7{IfE?-Q80aHF1Y{gYRYImOsfaR@^v$^8@c$ z>kAUaV>JDZ3AgXtsWzlA0#y0l&J83iYO#Y+<8iEegg*>yI8z2{R<_T^9asbBf`{n^n5ViV9-$@Xs~Yy=Q_hMJ6rp_- zFgiSTKylGwJ-#v6Keh|ApXdBAd4%8NhX7^NwbE<8ZYg(B!dx20!DFyZEWxn>;+!YL zeCXS4q0$;A;~1W#S@a9#9wHU@cB2kA+MB^X^^aXXqgk8N+_Pv&At|dH zn(zaaZn1m9pRO+-6lOb$NgUIU=wyXwo$rv<(9JQ$qc89cdQ0Jc$TmIm!*n-q);q|= zSD&azP_kcID6bfJJGzfwqBn0`PyG~%8Q1fOS?*AG;&t=uO@U-iJ_9y0s7LlDGjL8v zN@!&7z@y~OKbK#&XRY8tFy(_ih$FUPT?y*`AYfH!z4sX5_R^!ITXe-6>nsvUR7q;s z-x?bK9mS6m*F5E;g67AzZ1o@9BVKFhUSr13qSv_RKrljY_?~CF1&*Mgj9%oWp*_~R zu;~TK$Pprg;B|;eC9UUKc6ltx0N^k)N@lG>_F6=^B~-I(k$B-qNf9~CPyT@UV^^G; zv|_cA)x`eKy@$x=j$>kxDt4LCAx1o!cxm z4)@lM{IhtVVemz23VK{q2A}fcJ0FLyOQdQK_ZimR0H+epJ-u5&x zu&&yK0ft>;);_$6$%j}^6b~dXIDHe#`GjDe?7PJ)Dwa~$c0?%f_ylCdnYbF0^FG&p z8H^W`R^{V-51~qK`*Zouj+;?~>`6z(dHwqk>1M%xsVny4AF)R-%u})^!X7=a%)-GX zM*Tfy;`~GAWClteEeTpA`3)`qH6-Ls!IvSr*kQ^i_qx%MBY!LXPstyPwXr6GM&$7Y z=}1=2dwzeUf*x)kK^D}Tgmr+`2U9s%cqs#dOp9GIiJIXK5Ostuqa~2}g&yMxX_?ay z15)#&t{szQ7p@J*)$gjhB+E+yJvM_TJ$~wS&|1XA#&=|I3`s|BT8RB?LbIr{x8^CH zp#CxWt>CmPi)FKV>kkD|6Tq+#j!EZJI5fi>v(0RDl0j4@1%C@;yp$exe4H9&;Qs_n zAoUKnf~As9oa-+-E7VyRre41Sg=KlcUO-OX50j@(4oIO#VSrXF5*1>W&ZOp#EL-81 zrVs7{+DABk_tsbz^htS;V!2Uv0#5)*(PUfJNSzh3Q?8*)-81L=*Vg-@%x}GeOP0!u znzEEnv33i`&YUqR@;qILF{wBKNQDyr(gn=m?B>aiDYO(`BH_UVrQ=f$Xgg{Ir+-gM z)W-^Jpi7ef5=#0${i+=~cbd^!v&v05tNHUM-1R4J#&V}{dwv9)Nx}-f*l!3opkwIQ z!vWH19sUB}cWiSeHB}rkQE4-Hhx(>sBZ3>Vu>rO>zdCAtp}|3rycFx9ayM2GVWBiZyc4xlNyhY}~M$OlLGo`<8iY{sHGvDJR_C)}yMQ2*2 zvt|kO08rDdh?<NodF;JKnKCjKu>A8{ys$q41OO9^j)qt)H_te!&0}~sHIQpTrUs;9CMAr| zzBJar?IljdGse!zj0gR_$lK#Ypb6rEm3BlbG#^YVt=?1?ph?)8h)E1vJu)Z%4`Zh& zfSD{LjMVvp{Seblpe*+?M1i;yAuLJ_F^*|(PW?4nPf~6z{_@b?PSqkMck7L(E&R9N zqOb*QIMCxJ?}yRkccD1M>~M;JQJJR_1IHk1w(=p!+?N)POgbGlq#84Y3L<|5q-kK^ zr8tS#qAX6Uk=bnFrU4sF+34nacWm)`5q#X`+}3}C_u?uq(S5~lD#(6L)gz~F#tOKb z%yNBZlGcm&sDmigvc{mPu|ttz4#rp7ykJYc|4FIn2*E2jzT7Ryz=DvFz?9QDE@re_ z5P7W!9+Q(XSK+ibe$=(w+|hp{|V|f4^k8bm>g(l6EM^; z2e^pHA(U#ljcXkY`{hz+3Vg8h@VDQOe_EY|kX=blw-3Z?t#t#GlSl)t>A$>|y@8}rSItF-xlNi|7{{%XhMaG1RJ_a?|EB`V9Wg_QqF)8Tr9ZE9~XbA_$N>TL_g#gQs zFK=4h2%AuyDg@uUiC1cyXon3bD_*P{p9kM8l=->+-M>5LwU(nWGJb^a+7&1y=U+!L zP=YLgb(%ZL=Q&RHTxx&I>S(?aB3mm>(*fZ`^u}J-uYmX z?jE3C!gv@^F?Pv@q@%Pk`mI1n+-K)X!~(&5q&C zNeL=UkpRC3b7QA`3kP!8&WcEwm65!uaGV_F`m6VKL#!CJ2#r#a<9<-@lx@WQ?<;bl zm1R#@6t9~{aR}oAQA5Id)g#qk6(FCu?KOL?BGs>OC;X~yWA-I~t9Q69wSj+&9XfmDmcK4ZYetpV&DJnt%dM*|A5yCBD!Mp(CC5yG+R7w^>Lq?KIF* z>daB<3VSHCHmGZq7kfdfWj9#41gB4P;R$^lLSUD8yj5!T7Ol8bNl{Zyg}T?D<8-Mkp;p8QvkCzB30o=&(JJ7s#~A`T1g!w`0^NOwhxM6@wz)w`6YmK1;-Z* zG*)IJ_KQk-7nI7W-V}Z>o=`DsoOxc>1-!z?6Y7x$Z$Sz||0W4)hk)^i??%`yC3v}n zk^jH%9E9lQSpcaL#H~a==V-X>C?FW@U*mTOz{aS)uXynWt~5+q7@2JI%SDIWa=!f4P*wq>JoGPar}lGyhqA2kZgoPG3m%+*v#C! zSQSmg`@_^gB{0Iyqsli3!*~nwB*0iCs%o(&8DtZP?*|JT zy?%mstJw=^D>Rzo1?EMUCHAx7^&3G`w=q2izLo+dZeh;zs~R!{>gm`o6X)Cl)fa&| z@%)IapI92aOEk6`Z3TeVXjVvVR7!<14#F*g93A}1V0(z>{lKM;@@>+o35^rkQAe$x zn<)9B{i}4yO@zb@3_8@9h6fTkA0bl4VTHshoTD7xsBRUq+``;x8qbveRdOgYfx-x) zsR_unPPBi%mNi4smQx==I{D+-oy{$JQo<%7P#iY<5=xt&BH3rc<$Bn`?Ao0g2WXp} zV=-HfRU6{yaZIx+Hp-S*aYe8cw~xusAJ~zrNEBXnT*RXG_7A7xN;MyA%?+Z1PX$T& zU0|fWdA^`R<}B8>1YIcoY;r{ zkR!KYRz zl#j$TD}74t;rGZ5gv^shPf)qXd9@y%T46KN?urazo-p{HgV@Tom&!eSWO4xh`3fQZ ztD*a}7>cS0TBjWyGn4lLv2oy*YnJ$e7IY>-Bja29z*>3csJL2lo{32Lf)qmVtKEFJ z^is0v{Ao{#b1_c6hX{p2!%y2L1tf0BO3*_~&h*MMh4}^nwdi7Km}2rYc$GH)x714? zE>zWV?SVUvL7251nh2Or5~K{ChpZVi>R1Cvh}XO-OBl{0OJ;SJ3^wt`Y5y9M2^9~r ziX{2$g1E|lmij|?G^lq^Tz70Ip#ZAb+&miY`6aNk{__$UK4D-`@M$kR`Ii>pKt5Fs z<8L+q1plDcll1B>9SeU>I*tZp^K>(7|17_NXrdUelQ!%nUp*U3l2}XQzes7``^x6a z`}!PP?f(GDvh_vH3J=N3g1;C$S|tu=wgThlbi!+O0|J{a7! ztl~-cYE)Z`k1Zb@z~tx(0HPZ;w}^Uv4uX^$q%|U0A0TWAV?{tGvhsPR^}sKL%y*)? zkxm;}XxEx;3Xpy<$V&01?$uJl1&kqbSfK)U_ap1G`>AhyanIO=L;k1~cxu(a1el#S76hHHKBNJ-y%?gM5v&xSbJuBa6E#TRQAJSs1m# z3LZT6Xm981Au|8t*a$Yg6mx!gY~uX7@#OvEA!t0Mf*L4(UhOto4x(5AdYso1ztjed z({gLR6a;tv3>B5zW1IlJhNcfw57SsF+63OQAIGMG(zGhGe(iqp-dpFo)$%u4!p1Dp z+@3Gn3kd1Ohdf13-*rtk2&ZEeI&gb>j8NpKZqK}wMOB0(3mlRdx-2 z46N9p{Ot&g(&~#h&kNk>wQ;`p`6kp>E@y3NtF}vxe7DG_KF`YOFzlWlf#+PJ$3hZu zcH!?U*DAorb~TWrk(Oc=N$#t!Ema4Cw`ZP^7)v^DTQ2=_Q+$c1evq1_ahhe*`l>ODk=6ZSP!Z5M zfBX-53T#SqVsYOCnN;AKOjpQmhubqn4yCt1$8H4vyjWXEPm2Kwr%GT(V07rUwaQKV z9Gg(XoRZ1@p-O1UjCQPqhA{FAMu3;wndJYJT7Ofp5@Da2by2v7bfHZ2M@j513H@k_ z%S7+8f}{i77}k9gw2S^#7S3MTY$)kkabNCw<`+LAS+c+nNwAjvteL-cq96YRooy3w zpj{fZB7)2-`qqDn*KnuimrDgH@1q56tU;9;Nj9{%XC8_=P2LdC$DL;-rwp2xtz!dZz_z_(Wp#EQ?J zg@f_Eu*~IxzGevHr=KsTV8S27d<0UrQdDvE@lXz35?;N`(hdU;p*FQ&UYNLE@@Hd9 zYeEGrwbIs%nzcv}2nZJ}re}Tit!M9IA)(uo{ua`d1}&JRv(CwL=U*A;zF`^GQs5MR z7y5xyupmJ;OMq0*T-gJs9k%UaIV;t_wZeQ~L66+#5Sk*RsNP{)IiwVHz%Y0$&6{$n zISAiSoJTCS!`>N_mkJk?y3tR5Dd0V?+p1^w_P+sG$D4}BB>z_htKW`TW#RID@9i;E z^0>@bOP5?Xf$m)HWvwDLqGPCnwJ9oa0%sEKx^j&Nd|4ttAObz^<|L9N3;-$U^ZB>S5DwCUoRTXI1gCRRm z2h@VEb_@#u;X>imon<~|eW;=|R?EvzCI`!c#XP6`()h!nuHqyCV-ie38diLrUYOf- z9Y4{rooo`|;rgn!?Y429nYsSZ1~3LMVR;p-g2=lF&x#~ZrfKz^?iI}|KlFxQ+qRyo zABAMpzv}(IQtfB)1)m3nPv3a_T^$pc;mVJXz=$JWvM6PxlvA3vN%otLv&sat2w{AQ z6%ErMzK-~pR*j*<`ab`(h5XfHNK7jj0xK5EIRV>`Gg}``-&_tHy?xndK`c2XX_2v^beT?E*V}223FGk~|3NEwocjP7aM5tDTkpHzuli+^)b< z^h%J~n^-lj1F=m`ZMmpUs|H=(o0&6uXEh82#YL3-T@KI9q3j5US@%|sMI?*uqA1?H zFjD(4bID!t|ffpBzt23`Ojqe0s9Gn3_z z447kOWoc48(kwA~=~dN2%^wTHM(I)b71oFFhp|L9X^|=r^rtI~t~A+;PJmB`t?q_y z&!fl3w9C8?tPI1{!hXVls>ptSzinADpKP2GJk1*TyJW5R>|T>}t39&y-`N{|13=&q zztP%4(BnOUr2W zpBbr>zy3X}hbw z?DO8Ez3y)<0p#%jNu2-2;?~BS+m&5a$*X@2)<_P@&Y6Sg=K8X)^gB$#AnF9I!&lFM|+;-ka;+?8JAg~LLm%G{Cck_qD;#u1B_VuxpHv-Iw;?U z{kj&o-ZK0PoLl20pnN#&{q2cA$Mv=G?F=;I_WO{Z((R^y(_XaCQOi%n{5G(F+(~0` z`?D4!Gj7i!_DH+&!qr@Gaos$ShOpV@>*50BVQfz<`C1#i^G9wPe=OfxzgN0^WVGnx z@50g#sU9rrk!>Anf94W4_908KrQ?Dv{+X$irSgX2JoXR8eYg{{=>@Vi5W#266mA_- z!coGeMMoOvi_7fCzt>u2CU1WWE}mUQXgE(iFI)IrrXyafqk}(<*Xn-b>3x3M`pWq6%}K`fwV^swg@|*B#;m8t=TkmuMZRZWM4LfbLaCT4`qDkn@H3I! zL`@~$_FA9xrx}CD5DoK_$8`~G#S@-_$2Uz|ihOA!L!*yXWct=dZztCIMc9g&H`mAC zzZYpT%E(!GrEV9F2#dXcdkbiaYqQs{BuyFS>v7+4ohi2-Ib9DJ-^e@q+$_@q;dHD2 zwehrTT{O*(lGnmS$pY`Am7~`BThW`ou#Dl18%K$10&w-vpbmBYg31r=kEa62XMG+{ z4z9458VDAQ%qndD!_@s;cYpu?n)>c|s=xUEw@pGRe2k*vmJpShEsZPtW5mU!?CaXD zy^4HPQrC#=%(BULxlzeZ_8!T#$-21Yd+zo7eIMUHJbK7`ocHUz)_INR>rI)!?498? z?h73W^smR)CvVYfmcy{G-qo`gX{f#3{@8zIY(RY&<*|=4`?{8y7(w65J5Y<0DQiMU zNk?F(2|qo=*dD>!D$#=*7u58~ap_nzL3-S+Of)t&i;Ivp;Kn#kNZEWtP_U z3+WfC6EK%npCR=zJtF2&#OrSsHdSNs)UHMR5}Id6}=2g&Sb^-$;M-DNlN@`Lz85!Gpi}XVPAv@%B&J;+-}* z9_ZIrl;$27Y{H@Ow<+z2>TsUWm`Yd}NNiw>ZWdqYzby#*&+l#q3$RVEVI22Ks~yD} zjk$!gnq`Lm3?74eag$(pb|j36+4t^M2_*2=9gA$oYpm8xO@^*t^S_AZRErJ0TCzbC z=bm5Sqr7b{^W&TDjRPls)Si}VHUY^66dkDeTpW8+=5ObMnT*!}Lvp2#B!ur8bFgt& zmEHyt>6EmCclFByg*cL0Gh>Na#OJprt2n$aLeikdHfw@hqgM9f!l&NCGFxdy>edYq z!fm?qJF<-93_hxCB$yWa-}Sx?l?@{k8Q*1lprQ;ay>?4q7H2pI-RvG~`Vv~k;E;6S zXeJ;GCkLqK*ABvUmH>0_#I+pakcS_G4qOhYgh>;=q8}azFr}|HRV;ex5kN+j=OXB| z7(TPqj?H~Cqj<*Zo)9=4-4+vUYQ^{``vs>l(EoS35?wrUhIR5EB!>buIepDVT2KCU^S=m-fa~dW~}&2O&3^T3V`a13d}= z#lC7~r(jqvANde}F)JS}AxSv%+-K~d=nDIa2}UIe9||XS>?;5fei|-=Ey}tqq(iq? zW59Wrb$$$iypRm~L=4OK*!<5fMrJwp*M-0LyI6_l|I>g>u5sNkFKxqB1kLwOCLC}b zP1ptPh&h<&zW?>5Cck8cdx(DRPu{zRmt1n-SkJ;$>6r%K|CA?VV%Hsurpb6IP!r(-)-s1}|+x_EA`!$d!D*{G$ zq`cspZ)IgPfkl($8V?0H2{%AiX-g1S$Wol$uqWoIG4Q3ZEO@jEERT3Q0t4sAaczGB zD08PiTPMx>c1pLoi3)IKm?iXg6}3BDk?ZicG-87K)n!e1-(n|Nr55hm=+J*vYzMZSY9L*En zuB{)o4enGb3N4Rc@xL-O4*m#M!pt=k=0BFo2zwACJIKa%wyQn$foo0xrq;||G4#Ze z$dTV}6_(xw7m%pP2kJ;4_`Pe!55Avc(dC_fv>L#?`RIAJPmf#M_~1=U2*7)v2Wnp>r|a%M)Y>M6~Ks|ZKiGbrcBkXjT;IDxahlD}2X^iZ&VEkz`a^yUE^91H*4%dnP(%OqQlUcS|x zzctH^w{2>UY^ZFUZpOf{^w^?m`>(gG4EwjZsBEq~$BkFB`dEf22z8k|>}f5R>w0j? zB{2zkFbUd`>n{RVdj^l9i^u;?5at?pX1fnVUIo9_Ewk<=ZsvmkwKlPkHs*H%VnIP~ z0KzsR(sL}Bg{V~a=c&FXmW5z{G=JNA@AEMHa(&)qY7GyWwp7MVJb)A_o+Q2v*fTQW z@@`mjv^N<#IHP=Uxc@;spgfGbVnW@-r`qF`T(o5wU-@V2#Y4pJE_(bmQ0rTuKL4i9 z?u5>Kjl`YQCR9c4a>a;AprlcPl7wvT==B11r_rJ>@#;>O-JAs$dJYIjQQe0bc1N2? zpFK6Re>tY_78|L*Yj_(mxEVLF2E>3i1SAfj_osE8qXgK)PgD^7+AQ*8A3t2nTnUKH zP(3{WOOhC};j#kann$&n(@VUz`f80I;@8)=NpT{b_ZO)>Y*q<5~TlLQ5PV>@P z8^+>Bf~m!$X>=QXA9C0G?>HT@Rttw5)s+m3?d4d*t&^nJbZh$amkQow5o`*=Ufz*l zg8qQV&R#QnsUx1V7J=0I=;mFxJhNLe&f4qqgO?Bw3=_3;y-lS{~`XoaJXoMVqyL!{8FnbARQ3f=@Gj z7M(AO@{t5Uh${I&k6tmaj;zi#=IYnSSSiMNp8_*-EbbRydzQUC^RP{gJ?@4+$>m+DD&eaR7l6=|nrY}C zraG#I`q_5oy6cZUZf4AaS!u%}I>xkmeXQfUyLF%$gKbRQ>jvWaNtlULO3as<(kxX& z(75PB26KQQiL&>VQ-xWif|rgt98&7yOFRW}HW&`4M8t+SP+3_8a|<8YFVn`zCG4e6 zZ_^u$Re>xsxGx<9Il4a8J5PqFl=Z}iYm8sK9J&Y$fWFb>Cg!UZK&r3`m%@1S-O8Xj zM$>=<>>@qPZ4laAH94=W%M0_O{asoKJ=9io);hE?1F4+{4aNiif5Hhg>yr0%RMJIg zn>mH4%^f%8;ftwvegWTIv-wc|o$0gng(BM^XDAKV~e8~C1>K=yo~IuoRylxaZDHPI%2G*5&H-a~Ge7Xn*J zfjxj!KtG-z_WZTI8{?H#vH#D~8CTW*`%5UzE?ZCu>vsgaqK1LN4nc5_EZBYk2Bs8` z?gXYO-l)%}pq|4EG7*lH2@_kSje$&$H)lw1W=!P;uTcrw|G|ZTL>58*w~}$@nw$Zz z;;;|xfdgC+vH*opYWN{bP#0njZ&#zPM7y}0`s;sbVWp4sdE>SSs(#9&Aoh#?g#-l? zk0F06j_Xk0h1YX31Lu}&W*05;t0K_dy%p?mba})E15#b zRW6w`1}2M#%wnuDwRAe!7&V8m$GC}5TpWn3BTag9ZCJsrC<&6PPYU%FO5YPyQe!h( z$sYkxC75mC1txuE@8g zf|osg&x{{;2Zz%#g02nt=ZlaNJ<5XZ2kH?hD%uh#kJ?@qo46R*hVO);u31 zu=2pRgZ?qnEII*8gLOp$P1~wW*LD-@*KI-7b;N)0%%e-?|b;j?#V60 zfK8GvBd6N0!qlvGj)-#_7ZU<{olHP?6`lg92bHK)n;;EtZor3NkpgZ2)M{+Aq}JYS zC*Z1MPn*=DMw2>d5d;Y?BL;FN4Bo(oTe+A0s?B|8n60Ylyc`bvau@ddD6XoBK7v~aLZ-c12t_miQ(Qw%Z*x@lP08-nMIKZq?ZPhi5Z&n`Y~B@QXEr86FD$x}w~^Id`aMn8T3l&jL< zu71+(fgrY3xVox=o-vmGl1nvKvS)ElbVzl+BM=GR#DBTVT&PQrcs7Ll_^aYRKj}{C z8Kajo>08c0Yzcl8-EK(po6NSt;&k1oTfwpa3M%Bb1QVW?vgrcCl0FGYAzv-oCwyWj zgn(B536M}6wmo*;0_EzBsa42ME9k|NF@{fk z{F*O#n=t;XjL@Sy=)un)dxP0h+uCk5H_`(g4~!PE`z$cp1E$ z!2p!2^?9s+nL!Mx$F@uX?t;+L38dXyp6}^2MIA7`B7WypK$nb&&171PPz2w7o~k_B z&v992Y=Zfpy5~~&O(MjRKQ-9^Bd?q>f-9wvw&X$k^Nih75CG*BM%YM8!EtNxH7cX004`=M2#GCeo))_Y`T)pJBdVd&&4XJEWOBs&gi1?Ans{-PNU@@P1 z0I}LZ;*4qYissB1;CKSi4rd5?(+1bxjn^;aEi(yd54}Hp1<2rlvID4XWJpIyB|nVA zp>VM}1;D#(*@zW0L{Nv@%4~*eSA~5rV9KzL3e)Ez3XO}(S&t;vtysy29FHsfA^o%_ znzvCy|7T`RH2>Qh%pLELV#7P@a~dZ1eiRp%)EYJPz1!Ie)i@`ksBUo)I6M91i z7-d@cI|BE6wb+F0Dp8a77bD{QuIN3)?VCOX$9^NZaw<&uJju6hzqKJZc+)t zGwITnI29NKWI(RP)ZDJDhs6~fI3qm(cTn|F01}{)8K>1>AoHm%l|9j@YjoRkbmr?eU;!5{$Q^y2z!ATDF> ze*c~cO>&NxS(=jdKe2L5>Dm-Y`9XgVyJM$h8z=ZNgmugwwEBC%T~KZ*tpFw@f?iJR zI^)sG|Gjz>QN>`RNu6->9mz+zPl=A_g5CEf%PSx9RI4bWotqY9d}oz z_o%IuJx$b+dK70pW2P)=9zxjv?a$1 zg-67RDH+#_1O4uCk|_!-b4o#G-vnR3=oxFlvCrwgOXO$-G^X|kbZ>h?Ork@k4kRjw zV~n6S3L-)XEXl?7rl`YtU*0SE2n@ylQ~gl>`Py9%{wj3#tmoIY^jY6@0g(UbBauuS zQb<-aV|h$oFsg!`f%dntX+Zm^+ytJ0;JIgPm;3hBIK=Xp_7`A%ic!}|N2!395V86?*~ zD!0KQ51bl4%<;lbwkOb@vXUgaB~`itoFu4?qcnwt`%)uP=|R&;In;#YEOsCKH9=H1 zzmF%cUO1NT3NuVcBb7m;GDn_xG&wQOSLJ1mhJtX_hPX@^wcEv09HG}w-+!gzN*m8r zE?s~0@$Q+>JjQ3wEvFbY??@QX=3vadxDGh&O^hXkdf6jM&4dq+{zx-Y+o}H7)ZQDI zDk!(6QKM7L@=(DJEVM)xy|i;gFB3Re$<#FaiM3?%S(O~}4e2B9eAmXGKfJeic+@$E{Tt3xml7sIq~d;^n)>MLgn(tF7t%z&n*o`aodhzJeKXgsG5sp2+V{A~uSc za~+duo?1$+MJRViJ1uImRWA{3h6s-eF4pQx!-G1Vch$<0idbZhzi3*!qN~#GPwiCv za*EU*X01I&9>qRHxyxYmbQ&r5^LBp2KyT>@E6h%}s(F;&mvjEX1Cz~q`Ic^`gR^jK zFmm&^OUyy>VGm;h1;qM68Bblv9BH}M%ObJ54cRm3ml~aWqWJFB-`BWRno=ChW4dD$ zx>{N;)D!E zIDh2CO5mOq_6U=mJp7a87iFD-jSO!B3!kvfIW8coWrn?g2kL%qjAmu`D56Su;jiSc&>y~dQr0smM3@beUJNk1MvZ8Y} zYOzkR7#l#~Q((;~-d5n(IoS}`g`A4sX=**Zc{v)Z(C#eIu?veH?UGh0q z;<0b2sjcOyqlchs!xe>+xA32X`<>PO__q!zwcR5dXlAVt`GKY{UP|?Ui#B0(%4X;o zvz-d(Q6f-P4gmbUZcUkFdbD=UFpxgS_)7g(57Y4-cJKq3Zr%N4$Ej3y4` zpLPuK&oACCE8f1t`+Soqs5}=vTBl4f#Qv(wV~JNGixIKOqG}c}^mf8FUduB(S;+Ti zd@;#1MU3gpk4hi@wLmkNoR0gO*@EK!Pv~6=W;I@&rTZTGuk>JgX387x;8T!5X&^5+ zRkwBR4CG&bzH7P?&|$ZCFm(!AJx%rv@ah#s-v^&2&D93_7kSaRQqaYqUg_odWK^}x zU@cYkC9(nyao~N1y=N|bIn!|CcNex^YvXVz9Fkz69|xYC=Ges9)nH z+0D*zVM0_>O?gv(ffb4TR>e-z%P6j*6#4W2_afNpJK?dVHyTtS2+E-}fdSm}@Z@L} zc!2GDk>593^$;UXEO@5{vgel&ZB;evS8x%ULXkNfn)~&2YJ}4nvV8y)86GsSn01R5IKk#^v8o%|AC&0ClxMVKOhCLXFU)wk3 ze`(U~RjZ#0+tWcbte_OngGa#bllK7s637b7w?F>gnH6=uRPR04PgkJ?Q3~x2l39xe zFZHj>HS2C{+Z#d>CX|;Z0>Oa!_Qp?QviYKoJ1ed+4BBNHQsJ$Sm+c_CKDTFYXyy_# z9=pB<@_AD{6f$-g7`NGAGc`S@SfGIC+GMKlfuNCGf2a)PDwaZyXx;3O+Yo4VLGR@h zZ(DcxK~p)NWUJp{fLyY{8l(5}L~NuU$w973izE8CAG!mg PK@j|owtA78<&*ybF5cb+ literal 0 HcmV?d00001 diff --git a/docs/logos/DataFUSION-Logo-Light.svg b/docs/logos/DataFUSION-Logo-Light.svg new file mode 100644 index 000000000000..b3bef2193dde --- /dev/null +++ b/docs/logos/DataFUSION-Logo-Light.svg @@ -0,0 +1 @@ +DataFUSION-Logo-Light \ No newline at end of file diff --git a/docs/logos/DataFUSION-Logo-Light@2x.png b/docs/logos/DataFUSION-Logo-Light@2x.png new file mode 100644 index 0000000000000000000000000000000000000000..8992213b0e6072414aaddb574b2336cc6beca66b GIT binary patch literal 19102 zcmXtgbyQT}_cjtD($Xy;14x5(DIp*o(v6gK=Kv~_(lvBQN_Tg6cS%bQ3@}6UyL`Uy z`v=Rl;NEl2es(>3pM4EgQIf%aN&XTE2?<+HR!R*C2{|4K>6!eC=fH2QlqvOqe=r?2%6o+kVgYeY=U|m8~Q) zw|>XBFDOGZbV)DxP_f4GUdy}?;q|9_#!QBTB`C(wck%k$u@qh+3JV6boXrqfhYOX1 zk?XZl$(L|vB<2_2>A%U!r8V2um!P6|3dYnynhl_A=l$xu2CD|MtIGGj8GptxU(3#2 zL|G14%ok+r-HMElt*%l{$(z6`k5C01Y8j?Ho^cKeQZvxd(M`Dsdw6)DL!u{xk>I0;MA;$j~~Onf&%}B&L(QKcD{nd z1=7!8%*;zcfpx!u5Im9+2!N+|ZEEeD=CPTq75pHm=j2$Yh({0|ot}5;;d3M(2*DK3 zUmFPZ@C$#RNqxtbjrDFma@rpi`_Ol`zwv=1v5_F}REhq@Kc&tmvHKo0YJ2oMuBFCu zx_6j<#70d@%6p{ueKK#?g}z_Fel4ZPBB?T^sl>Gq{R|50EE-{w=9m3i5Fi+0?r!OmA*(M)DNjThEAVCzSpCY_|s-?Q38W+l!rV2TniJn*jj{Dgy! zPIOB>OCjCZn9TvIk){#w@7(sM9GQAJ-}NJt31 zFIGG!v{awuEqxHz#+xip?lHdp_mVMH>0H-&M5<$zE-=PDt0WM93^j~bgSpl-bOym;qELtBupP#<4rkO1q|V}`L8g6 zgYL^`q4FH@F<;f+UO-X#@;&Qg6~pY$@f62mKHqi#{&@I_MGe2V`5jtAUc^PK&-96o~0HOZ%NIyLc1;%Ohewo_YiL z5*X{hWs;;5>Q_s@#N{jpCq~iV6A@EJvC3-yZb@pl5mcw^w1^vC#0h!D#)%tP$_>1UwPhj9Py% zDvkDP1};->i;~o&dR)Lb@zSDne?20MAuhJ7_6(^~X#H-nNt}Dy$x~q(llQO;87V1| zslE3Yy`ppjg-#^$hsQ&EN=O^@1*>LK0F#G#q?5wcMtd9^nMX2eikB`3(LC!Pk z`o0A`HKUTdje!{{LqtI@qce?Ho?$)pH_tV?ZKQW!C zo|TGKSk=c0Au|vL|E$RGC-Hyhb%vahMo%{-EQu2@#zv*l56DLN^C!9$le26JYpHB{ zoVfEuQUp2xtL2MtRsY`nQ^z%*+=dkP?KR{->S+#Hm~@^M?(Q*ri=chN$H&E{Re@&* zb3@wiFlmjo^Y2kRaK<3w50HF!_#rjCh(B+pb{_-mQZrfGE!`5936^0q9W%1 z4npxV{^Uu}#nLYXieqGDe4kjV!VcWKgEEw*q{wsL@7F!~{S5CipYo*HZ5w)1NxiZk z$A$a>2Y?LVZB8-)UXRUsH+V5@Oe#%mfNl4C2T11w@`T_Fa+F%ZHqmO5T3j7y)KAXE zKpGj(qQ12&o=HjL>pZOuD3l67`xO!8yj)v*SDKk(qGs;g@l-Cgll9WGUAmiRyAs~N zuJu9!G9^eR>GagIT(u?vMPrV)+NXD9{oG4(j{tTF(e*rG2E4?Y4)SgvX_}`oct2Dm zx2;xF`nk-vDFbQ7>B79d`M}T(D;4;PdBx##Z8e7m4K6vc@xlVh4V=e^ASv?M zz4v5lE6XJH^S??X2G>h_%=cAtugndse-_us$HnvJC$9|((a87sfXlM6FeBT0MsEgl zs39Z`lWPHY9TYkIn(KrKnrRy!(>I@8^QO_-8)AhmcWs|zUq&dU&t3@7wNB2zCsP4- zsGY}Rk#0_C;GQ8tg6>h`s%Bj%uG#z^n($ z|1OtYi~GK1z>3ork+b?KCvmw_N#Q~08~q)H7*!RQ1=XavYV)YJO&FQn!0-8#g%`<) zzWT=7xPiYwfPH>7FH8nROkLY%#!4{;6A?-^PJv2jO_W0M%ee`Z;% zFGcxn9!llSRsd&=%Za=EL+wRf!69zKQI za66_gO3zrAJJR%j22XiI*H)5@v7=eAUvkAkxZa357=qdF&fdv^r(j0nH@J}$Yo&*`BGY3ZENZ6vo31IbFU)&Ks*47y@pmb|oMe|ZW}0XIb_ zT=fN0;kv|&Pi_6XoEWlbn4Y>agl)@Ylpq}+Xf#I+qI$ptkZ1n9U%_qQd<{_AUA|~O-+IyY5skZORHn@UV z2YjjOFVgUJ#R@UdA*#*t3b>~jpq7H>H8cZW1y~~~Q$K`clu&)8ZM2HQs3UV@Gbf9v zb}@w9i^s$?;Qv9cuHo3C%s{+VM)9|i-J2=TkQOB67vWWSXA5)dXi0*B@4y^~3|h!L zq~~-({Ur`pzm5}E8oF8 z9%?>|GhbpPd%Q6)olC~%%yzo=5VR}(Tvc0!Y~JBp5H7C};+r(e8>=(UEGC6^XaKMq#6~Tgb?=IH_db8v zu!yudd{bolV3d><+vApjY78i%RB*uxE9WqENUs4O)s0&?I``}f^MS|>A12Xr)*>|v()&wl) z&~Qk8By((D@r!Mqk{n-a9YeV$o__~5GByM#RE*;IW)&I<{-V(lq0dGYWo=wcDWVhS6= zT^RikUp`1)Hlq?2BHMaG`Y~OH@4fpfCb8SyodLO---9*8H*7yy148)%V4E?1OD3e< z?H_gz%TRVy7-fEr?7dtuPQ00ReveXZZ{mWFk3VnDDB@1V6wG8wom^V8h~XtL3^)AMK@@MBEmNF;ED?PaK{zBw;_rhjgMEF$05)_L)&{_|JPTxw%z zVer#35;rcU?$$vb>&IrFUQTf)vrpz`HY2@^r=#2CP#mz9A*ZByv)(S{<23y0i9(;L zR$R2QeB;ic*+DImR2bi8x_-ba{HrKavHJ_fGSjDV zrl-fYP^aZF>`@#0OW4P^z8mlR%zHEaE~7A}MP3{9^$%L-&6>i0!p%e9%saee=0(sd zFB+$b*@_bd>=%zwvjED~_YRob=p zk}AcV6L>%Coq`1;pJ-o1#w^CkV)4f98MPhMtJ`cCo+ z|Awgj<3{t~KjD@I)Pf0v=NJNcb>M%5UNfv$Shn$+~O! zHy7gW+~VeSk*-o_9Ofri=m9U{33iV5_Nl9OM~M&l9_tDW9eiaHGcbdh>_Uc{*N?I? z5QZvGpc;+%R_=+eh&~H`N1>m{xLp|RtV#}oT1GZfg7RdZO!&@mq{U+5G8>w2HJ@dQ z4$+nRDYqZ^6b@(-8UHCb`uuCg=$t}0q1!o5!oKeyDXT)_iJuJSYAf&>M~EF4Z0Xim zJchUZbFjamp49JcR|MIq8qbpnw0{Y10+Zg9xoD*u@834$_uGA%Io%504j#dK79^7C{@{ydH3CqUNmtNIfLbp%5)A zUSQ;4HsnjYC9hy-e4|sxZ{8ND)RbVjZmz7Y`M9?{JxQwWk^(MWYOyz2qfcqGSQJg^ zmFDjYY@XOWt0&6gWR3)^Ek+TCQ`8uJu>ZcczHArSC1=b}whd_Oo!`6Mw7$O(JH1VY za%fW_EC1N~uxqAJU9Q8~+JjlXy!u;5y?hn#r<_wXfDr34It~}(p(OEP^^*IKeIg-c zv6=8jD=(wN`cwMAb(8j7#SYT|F_f<5X;tq|0%PQDtLsQAIY|qw9tv;ndZJ3*s6U*{ zN`RtbW@vX&A04}on_RyMdcD;rzlBW11{IRo+n3j~_V_LSmi*g#kqUh%f83e%c<6t0 zr13!B24RxOG|TWTud4=TnsE1A6gC+`-x9H*4`uy!Z#eo~svtj=QbYHL*Jd4Z$=Jqx zp7MVE$jkyG_{zmpDc!{TpZ-Y*vkNa;agY0=`>+Ek7dswk?hBmxK5GW1`xxDGYc^?JeJHaI#WwI6z9KVOWnM zmXP^yA%las`mz10qz)5uCi(+C#**@ksXK|vd0P)>sLxreyy=Lpf9kdpp5siQAL^hv z3B*K|L@uk;L~jeffXjat2W%hoPo8Z;b~?;8B?zlPja>nh<(f7 zh{nckFN-I7q^6742`pW&F0b3wC>kt!fuqLnt`-qTcQO9gK8X+3g(g+MUPSr)1LwVG zo^n8k@XzPZZ@5U$xS6QYkD3#StXMGWaVkUao1Z4I`3G(qzk4mQAIzTx+eE1p-J>kn5kkt2X5}ZrkoKR2%NAI{$a%8j ziwZ9n`-ldSM+arnH{NulMWg&V?wM}RbN&>@Roy>b;huRdRWi1k5ec~Lb!qo}K^p?y zF2l6%&Q13=lDAo=n)y7u;HS{+2)pP9m{%$WIJN5we1iIP?0Ekl=?fcO)>eD26NNCr z&KgUg)e-dJn_8|v-GNT+M%Xpo<}^8VxonzmWbC-P`$1zzw;U68im{*j{se+xH~gyy z-G0KaLX(ZV=BS5#Sa#Bz$M!d;Zgd2E;~TS&g2oTJhB_(ZbP5&`0SE_W84j+_spK`w z6^uAenj4%ZyFN}P6?=yzHx#KlJXKMJ42CEIE#I`NsU5YP?OT))xwp9j`bsHwu^wST zZNfU;bqeKQ<_aGEW$Ir&#QDS8f2?->O_yDA%kZUSh^+8=7p{*T=LM)L2>^C$_4ne{n&sn6 z`}uzRsfWYcz`+^n(>362k$wDx&rr@IqSv~rAQi)B4yG24*cy zOYiypV8!CMOAolLfk&@QxLJ-%vHtLlsm;L76NrE=SQ`rg6Uw_gmd#l3(0rePF{tN? zPv3pick^cni77j|+PGCiJ}ULv^W&y-Y0Z)I_`5ObP@I$+H+py|^^`u6I}IL$UZe%NKQ@U{ z{e&!yz8(I&iWKD$WdWe0qao8wsan&2yOqA`zc0Qnnxe=_;yGTlyLi0MWN-b#cdE-} zW`~UY#{Sn=Ch0rks-3Af@)ox3{EKOdxM~ zD4TuWH7kAJNrfyPnu{WBexXxQm$;+&WPfD(^&gP??`oABlf%d5*Oc$v2R0qQmR`mK zh=zQ;Zg%>i<=|HTymDW&w9Gu70-3Nw%;hFGPbJxr7u{9)Qwc|EeU1NkOdNv`~QTxp?1& zW@38Hd6~q{_ZQgUd4QtzC@-J9j%$)Tq31SnV$Pnrj3?@Tvcbhp;?`Jzw#-p@dE_!F zBvLaBTxc5FHQ9aplg*tYUflBQwESV(8sLI{De(>WHE_+4C|FQi=^-;q;GpYskIZ?5 z0ST!4EpFP3CUr$q$79`}z#*B7;xD_jO%r7ifV}=V1B(Qn5o9TW^x^DWJonXgWd*>Y ze`K(s)s$3nCfEAB!{l;~9J(u#w_hllNK#V*WBKp{WY_yOYVmhh+OozP-zP1KXZ2((w+gVtE{v3gk zo*vX>o^GRQGe8}E-Xupnz+P?TMGgY%|62Lj^L6}KURPU(7USUAj>L{vcX(#~@pB%h zVfW)P42vYXC+gY5@oncAy!=~liKMz|cAbFx1J7a?2m9HA_2`!b&V5O+Ue8yEty^;0 z%WP-dke7AVEyYV^iL1tb&!6RMbXiCvVS}}D0d13{W(2aVP*W3nC}@jbe;o0#H9aDk zET>oKIAc+AINrAsR5+fQ$mLUXY0K0W z>l0ouXpon95{Iwb-$^|p+AyU`NXq9iLi~Qau*Xaku=C!}6x$7El$dcv012^>%l7cr zRK~b2CbNCTI}y324k$8QQF*ea=%cJdsqN_167B<`!(E5+f=J(RO1&E8{FSyF2A8Og zj`e*8foS325tRGZpFe(@>Hp6H$XwR-V9>ftt|KO$ZRP;o-VE33@|<;Z$?M&&-rjLI zLVf(LQg6o!{pp5abPv-hH0zoJ2euvsfLMzW19cuJepPc;ncK%WwSzqGF6^WMX=wNy z?9CH#@t^(8eDvt|kHaOYo;(zXXI7ry^mz%IU=E`wQuy7uNrVCQ1c@UpT>c;BdPZd5 zC4Zj!-a_zU1m^v48erMtwrdOeqK%>GVmIw;x2ZJu2!Hd9g|W3%-`#V6o7L3}M2kIl zPdlgFt5ZA1{R=mYbiC&9V|?sBZ$)@7Q%?-CYb3p!|561UqpD zd?NMT1f`aDFwVgGo4^kRqLQg-M7L&ZYJDc8`<1musMi-f;&J{G2Gqq zz--S76P~z4M6-Rs6~lhIzM)o3bCmtnsa8^Aj;~qf{EzmrvBTTPcz2tTr8Mc)dQKhp ze?C+1)tfXe<`9xYY^36(z}62C-4%DQBBB^Yko4o}8NdIJDF5?4lE`D<^=Fx4cf#vd zCYl~3aIS18L4}MBYsiQ|XPHM?d22iuTyh=2k^RX|_2co!CWa36LL8*Rsx zKh|UR8{V#oUlWhQjUy<0kLYe86_qWa9b}`kVRfKJB#8FfC>-9Vt>fbMLsUUDZ~H0Y zgpbCOuq#;Mi7V7KPI~y=i~l{@{jt+MZe`EpxH7Rtyzd54U}*lQuf1aK>LYY;=;#21 zDD*F&vcV^7FtP=Sqbl-B+?b`V4bH5}@cUlOW{%w!g!t)qq*arH%KxcCMu2_W%4z@D z8tgZ~6FrrlJnLn=TK&XsAC3QEeZy^E`2(YT@U%{a;(SsswwQb1K6t1pr@&ukknV!7 zH6BpDkJ8ug_Y)p7vhpJN{{@KmUYULuXcx5f1_A6OFgKw@WtCot-@g9|`2sFF5 zvGdt$#!a}7^JGEECQp z8pa2bUne<^pj1^RwXFj2pspDRB?iguA@<591>GfE!6$jBx7pC!t9h37r~cXF!gHzF z8`+Er7@U95nP=Ga@BUO}9cVwEsDd2S86SehtdM`Gwp3Dj#o0)@U(CD{)b^u^Vw2+O zRfS5LM6ZwHNn$ZDTc4C1-beTE;&HAx1Gp+{_C?w#%i%L^2(=6(4mK#k#tE7R%TqET zOlTO#_)|b=i0#3_5mPN!v$1~ePF3U+J65xK9}eI18*}Xb($rrM+GBlLNpVBXbsF;; z*h^e`AswkBP0Eb$ZLb)fK0~%ydQAqmL#&e3wXbwO(MfaEe)7n2rjOc&({0z+h2~1l zhAr~j`D23TA{V6qO63(j;Hg$CbkTS8bBwK4!^?_;xWxvD$O65$k%^J*QhBe726p>I z*TT3CDFJvo+uGj(iec$%OMEpV$7_-0;ElY1{wZ~>7;+~kWUf@>*80&bs5;M-Sq`^m z6whHoMO~l36yZX4#4hxd-Z3kEs^wQ@vYn%2kIxf*4(1w{0Di@m2?9>+O>1T*h?l3Q+`1d;AeN}nDXfdA$!K*r!KYo=va%Oc#5 z%5g)o`^Cw)1u?l4!;%Q49csugh}0Iv=dTnsn`9UGVc4(|tB8BjQ1nq?#QmKWr{p?- z@_p=_WFpmrl4u9kkA5a?_#og3P7&kak*`sE-~-{s$ia0q8O)^RP&Rwv+)zQC?44sM zPr!T@5Nsc3h~An|vyPKlI7qGkVskL^Xx*SKE+0nMfQqs%q3t0jI4-0ttz5xf-_@8n zaQc&svq8^)WhA|L!b)9NT}K9+65=*1){%DA&nuc=Grk+hw=9}6+ghU}R7vZw>!A(f znl^BR&81MT=mvaF&~|YxB)A4G7}dyJ@^@1`9X=uG(=r{*jhYKLQBpMKYGl`g&#n+V zX&b%O12fXWx$7%LAqjm-H*RMEH7~2^qvz|MGK+h$-ODBUr?)+RC2#1OSB+K4jF5cR zs<;SN#5G^hpdVPzz#sV;rbs~_HlLyaVO17yKGruiqPPys2yz4}Ws{J?@-l9qcEC)@ zG2Rc~A@6p}SLCIOsej!*nz2q>Z1PHhqhKzjq;Gd`{;;WoEzQ_SmvFKhArHK}(a_U! zy-XbE5R08?G8)CUnp}5ENn1=JcnH5!`k^FyrW+@XNH|`+P=T|PCg7R!BfrZDu5Ke@ zk`A&&Zjl2MxU2a<;_e91y^FIRZrQ0xnp*yL%oUjcmTAX+Wo@sB5Hyc=%gnIf8hd1b4kW7wfc- z*dlG^>jC$dWPV9x{F+4-@wU0*8uL!~mZb}8HdOY$j7u~WAO6jl0`aZfMB*@aw?EO> z;K=V)m6z(h1JZitwU~iDLr1wwES7SJ7tQhfN*59MfyJ z1o*D3XrBRpRskrU;s$P-zD7F>Ak|ElQdGXuePJQ(3Hx~Xef4*;UAa9)ZKyO)(>N2m z1H{yuK1p(i9!P>6S*zD;Db4&AnNOk)TKDG|rY9|3~&RSjJ{xZR5aO<8rFf zjc(-BbIr-%OS~mdwi3bi5JsgBu7cJ($3%eBxzgsW-3*D2;0T*2M$K#SF+OHx^HD|{ z9g4?=^z!gM-(rG4(&CuVd>SU7I+nrn>`i$o19?MdNqUjUU&N>_Hopz>)Wt2zh3% z(GJL-YZbB*>#ewJoV$?x9OU^kKGxQEQOF)V z(q+d$#$gBJPxdfRZ6nb5VJbXpJST2$&uKdahE&NDR29d$Q2G4%57J9c$Zy-fp$jzp zhpy3e^b5jC zrU)pTtu%rA6B6ldT$3fzY`mC)<50n8r(;u6(XiX zg3asyo%7%o!E0FJ&&4^(6eHzH{>omvA7SHe?;pOTMq6~tDgfs+`7wUj4DbRc6g$#C zuC9H3V9F(F8KCCTPn;tWEj=YwWnLjH4*p+2}5CQ4m9z;86mTB0(Cr1#bd9YzP}Hp1tMmg!ZX^rnbIvMyA{Jgt@vI)oMX_ z*j+2duDcJI&MJPk>M%(X-G1toK;?bY15z0SQcTLmYDvp#CST~kQ)IbsQ*6r*-wwi> zMENA&Pi5fU3jGfL=YSpnbkcsUq8;&>{I+^EW_FM|DO%>Zp0nyo&3PKCH|SVDIwt#a zV31n{mmg5VGb)5I_K?x* zgU#C7tPKmXmhBbhV{f#VcfQX^E@oh4V1Zdwn#`9C*)LVj z;J^c>YP#^{NPA_OVOlPSS4xX^M9c1SXZAwu!5dABa5@jo-~7f&3q~U1xjW(XXrKJi z4ET+aY`vKZfZ~oXjMbq$W3O=l246f?eTTu=lldw-FWXfF_Plzj}ZXJPWKn=Ryp`$pSr#&;zEDC2@DX_VGEAgn-P9P&H-MR z8VJ`mSaxM<-Y@i+;-JOBeG$j1k^5YG%UK+ADCg?#1(}QodUO)rwpXIpyc1Lc%z#f4Gh(e+V$$?BV%{hK;kt7A=7HL zT|^J~<{GpY*0EXlhxuTw=J2_veX-$C`aHsOYdoEurmZuuQYE73U_O?yuHiU3vCb-Wbezxr4}Mi`QLkO? z(Om2oCKtPbs|#HJ&~&KY;3kKam=znsx6bJU-o!K;d7Rv5TvjMwQ9Fa$q#)2j7`Hlw z0&}gRKbORjuTc7_wd103T-8pGJK@!xpYbz-pIE2n@liy--=47uI zA?Kw6<`aZnYce1az6>ua`}8F{$$9Tm)DZXK#&mT%c>bx)uzT{G!m40F-2UopUQT&o zxb?mA=}{|xa!7TH^e%{FMtHx4RNdQj3j=$kQzgqaPEpn?gjld#J;WfwU`2sPYANYL zO=$`$eoyc3mS+#!^1q@vJFE9&@%i^*>U=h_&Qh=7){i6j(>nY<60nOO-i%-_liQe4 z4XIIofZN*(^_Q{J*%nes66&tNU(TM$jUQ2uDg_kt`~79~JVy8by0I>HeQdPZUHUtc^4>euy{qA<4rM-_UxegGzofnG=dg;hii4%)(~-$hmA zp919YMpJpEI9)tD%_wt7)?tVT(UpK)6333jH66O=`W_1Z8 z2Y#kr_2FMSea>WlJ*DKYX~Eg=_t{h5H_FKCv0Se7z~Fx}FXC99^?trZJRFVRk2nK( z8r|F*)dMPCZlKbQe4#i|scKzz4eMJlz$*-Lo)z0|r56?vLcswpY0M9* zIuxdy^6JZ&@vpP1DpzIsujo0Zm|CqzrN}E^;tk~HWbYINdSmaTKP#Z+LXNd2yUBdMIg^;n5o~MS5<0Wv3%hb z5Mdx_iOVPsIFT@~+Z%y9)M3+71rNP6f!PArHo!++&xDCm-mD6Dx_)2qWNchVJ-=D? zR!c(8qp|g2?ktIzLu}-~jjZ%P7nN*{E5oRuIEhTP4yx}fF|24C5$rpeJc@ND`QrxH zXkyL%t)fG#V3^IYcf;NzCN9z$|6RwR1;_y#DbD^ETrU&6Bt||%uo0axxilP5Z4cfwa@4mKjhgK`nKmY{M2t#!5PF& zP)%2kW!0rN6HldxU9auhHcp$6Xp92B>yQA}lW|p{JSjvOx^≫uSMcw~$CUKE7o& zc60k-R+m_zNHC4sv<>I}ervA2=stG7W!1=b4fBmK?6Yk5 z^4MFI8}m<%WrwZ<=K4nJEiNJE-<%|tqZ^sKPCNu98M`!^eteoEOBBxx%Z5GKciqD6 za)0tQZE-dXpU5Z}xM;&T6@x>~RoE21Tln1`r38(ezl?uiUUmc1oaZ0GR8-X^{)!H@ zJ`1*j%v0Gdo$U*}v0XM*UuxZF?aL-mxRxz8qDCt+F5kLQD-v5?6 zNqZg))cf~@YWCS1t|Y;<$3cz~jVW@R%Uag2g7qYhlnW2i(}XR!emQ*RZ&d14NuK*` zPmXMa^8x5_&Hrrqxlt;y1Ly_M#l%LPC+)KgB;N0Zwbhe(D_hUIC6n}0=hxe-nfd6~ zL}838e_DI|_3-%1*F~W*9EC01l(OO`Wv$8HIsJ`Bo8mRWY#gBGos_jtnT%1nHlDh{ zPihxP?lo`^L?Z;5Px4~By#*}jF!t-MySO~HnZAq}C4tKE*R_J$#XiVm`C%O4A6dq~ z3+$Ju7M?P#BOiNAnJUXt^%)nf8M0RXVL+ySVlWX@F{Dlj2i* z)0>cSfIZ}}S_*GJ0EoEDU1x(J^Pg{CW$?-UkXiihCOc=*eNP>ON01`kapzwESu1Em z8+NtuOAbX!QQ_q`ar)>;PogO2h*4p*$$E&NU*}xq>YH-yt-o>BT=luIMU;waEh91? z68=yHH8b`@?*xhr-3>0jM#-HwZ7CI=F`CK9d5?5|eh!H$j4RJlkC#!IDG7YGczyE< z)DKyjIKtrqzJ8pD_d^{9k34SK#IJwlXePOgMq`~z*94@rL|D1E0=J2ryaVRP&&Gj- zI-N3C>MCtgc(;8+;V+)-dQR=$EWRTMFpAnrt8229%j4oDOMqj+C8cUrFa{k7^7)wx zue9x6a?u`8{V^E5eFZXc^N1^Xr~M$d>$b5~@_^^^P2HW31=1ee5ZUL+?$A1>djR5> zg9n$XX<>l0TYY?lQv6{4cdJKdPCxsPbnOd$?H+Iak0O3#GVA5XH9ez-Qjd0pMOByL zjo!d@6lyWs(O$;iQ7Hm_vc~M;{fhj^LSi-x7hwTD1KcURn-r@ z9~g)A;oeCt1}oO9{rXd<{M=mPx@qre+gHyVRL5hHtSizsl}H6U?q~{T2;OvJEuvoG zgK(MVZTPz`{sIHetFAi;jNQ%GY+8j|9^N1QzkV+k`r;(u{1KfuZxHW__7v@H3qKf< z`Z(N$7<9aHszS>?k9B*~{bdf0o_Xj0h-iOIf81U7yFWn?3x`9&4s!g{Zro`un*KUNS%gicXte8zTyaba=?#x4s%bv$4){`7?T@@BZ_8 zSy_DP^eVq42JNOCyB#^a&%kzUm^rkTQ@GqZ7-wyWkC@6O`>?+>9a)JHv zu6oHSYKcB5GwYp0x1Wo;o)}l;lcRWZt1y3{pgg3$+rBJxWagLo0XEqKz2qX_9ShUO zic@wnzU}%FJ77|^_9d`adefu*aycB`@G;}aqug*9C>c3<(#%F4ud6na$I9P4mLJ)G z%?uDv*G#w2`@94dFX(wYQU$w)7`l=!#MA5$$4%tg>U-lU^D2Jc`$ePu%{*YX|5E}6 zix-@;(^!x)_XB|fXEO21V}RFtp{xfuQDevYF`=>*`gGe+nJa z(fFB3-kTDU+&(uwZuYii%;@LP_#>OU82=*y$BP7#j>@3=yl4A9)pT#~;YA>>q;Fnx8BUXOFKgdPm|FNUk@cI-+LSb)kD|TM~h;4Y=nW-y~h~~;*rNKm%6 z!d31!h;Bbli!xOxyhG+49XX%C-hC{018GUK!r21_9(X>1jsrVVA#$I4+om7u%N-tX zlCHMv{jE0^TE`!eftioN_uaWQhk>ya1WYP8!x;JXruV7t{v({7^c~~0CvIb0wAXfj zjz4Xtl0fDw94^c41S~4|ZqTUj(}0p|TCo!r_lWTKxxBP_+|2x1U_u2H1^%JR50lsR zi8$@y?tZcEf${%*tFYs1?WgI}WwrX(S1A6LlUc;}kwz&MGW0?^?m(dT7qyss47-n> zzqUl}LxCD-i#)?!|AC<8uV`F+QdOA=B(ChS4;P_0$JtRlHzoz$^hejnN`K6K!7 zp(43ccbOyB!hdBPb`P_ex6WFrGZwD$MRuBxe?S|yjs7KMCwH3<;ST_hjAtE4OqF?t z?*A0+DL32=Irfe`T=?}GELI-W#6#!&U>KKuItavb1z5_%w%+63-E*Hzn>;7BWRp*^ zYb(Gwj73nLuK-Bn{i;O)as!6dDQ4vEtUXx@N@d5s5g zh;kVW1q}zl6RoSxVz0ZdZ_@28M>b1NFjrQ&YjJKN;DcvZWB!-F#SyFGSLrl;nV9*Q zBZtE5l}E}wUkgRL%hx{bbD@1&+z#-1vF}`zLUvANKq&AB)5<;j%q~|nhU+FhF=177 z|M@vW=(vRUcp#(3kgYME^=*blNJLlMDruA(y_jcxY2^2(ql|_qfr;;cQJ+Ea`1- zvGT@Jpjb&~l*(441g^*322P@1vEDDk5EZ&n!sgVZTT8kOZZBfT7tt#t_#ozR@;>cBFSvO&kVzxV47fm( zx+CB!FL3eWPm_xG#bFT}3%o94W>@5p6ZUZC@XeV8bE73Za@Ud%9kLFt6Y0HFAv0MLJ*gp56Fvs!|2Lz+#6Ip*$!C(>$4$@l zzM6s8NeRZPD9A;+j@R1pehwO{Jx%_LgGPdVz+?yZ62M&hCAMOO%g9&dJPu)bIozku z7kY5fN4noMv{ESbnEor)GpZ+LJA5^*?v)@2e?h>FaIxo569&4h_~KuZP@TbrxA>zt z*A|vq=-6UW$}ou?FyY%g2B70icY>K*-s<^dIgY&pOEMT+tZj~S)FZ+_oBN>YnHuiX z3iZd;3&Mh(tJx=Y3Oq*WRJA7Qh248flIC^60@w*#@YHJ6010yr;8G!^!LDl)c~^1) z+cM4;H!ob&Vc5sM{+N7PvW4<3N3w&&Mi|EG|CMn5@l5A&9H$*|=QK@rh|c=4{Fuz| zA6e04ex##Q5wTpmEeXx$N5fUuwo(|H&Cg`$I#x#^LynyM*1}GHtjo_e!pO2*?)%;S z{rNoJpU3DdXoIevSO&@ID8yoe5a z9>BAIUjoqXuT7TD%0BM+@NFO!MZoxXmZ{7>%p9sRc&c`P@TgJ((Y5AB54CH>kC>df zvCxZni_#M<(cn$BynV{3a8*jDr0@q5sD9g(r)e;9NDe>n*-o?2yg3RZ=~xqxc!+oG zWZ!W^V59|4P;S8_Qsstcg2x&Qe0;}A)T%CIJ?vfIKxMSDB~yRCmBzgXjUO(!m;`L& zTN+IT%vomf0~1-mb=7bw_)wOG?Q zi%((mLtZkGN)17}*NpUWsr{!t3uSSnfOMax`caLm72L(wp_C8>_tsW0gU7506DL-; z$cGE!{0;WY=TaWn@1TzVvfa-L;1#kdK*K_b=@{OhB8yZu3uV6_5UmqI6TJ)(L zhacYNLh20}TwdvBgVq}0R$;u)oJ&$sNItEm4+zVa^AyQhu|#XhwWr#>xINA?UCAF%E)#-|KNi5}AD7F(4V&$u@OcHnd+FvNfdBO6yX1-d9|zn7!a zvMI*lXGE2P042+Jq$j-xB2nUk-t&DgW+BF7-j^vy>`zp;Rp0nT9!O}Q}8Ywd2_1@9w$au z_vw%GUztUMB@(y0@U{)aPlMm1U;3YF! zG|2AhhADgvvMhCMX(i57pf*|GM%C)vn>+RUpSst=q0+#)B$)8@yvdH3v>0>~wTkb(wmK^20>_El5VToasS& zy5cc5A?g4MhvkK;mLrkxYxSF8fEVdS&mZ}yN10cZ_||#*ihe(6(a2c6^}ik1#le;c zPc8D?j=>|6YmvS;J)FOxZA-`?&|4S^=XkKegcEG(8F)KQA04?hw>gzh5juCd>Y54k z@F`p2opK4ZKIMQbfcYSVMQDQkd?F;nNVwVQkpH4{_FkJk6WzXHEYx(7qXp8P@Ue;E zk(qp4zS86DwE+7y}%rTcvFMm-=PFQO^w~DTqc&6lER$#2m ziMXCv$}aAAki^JO_y)N*Koz_;d#_%lEy{In;}63EVqGR8m!lZeSwIEBR5OB;EX9RZ zkE+HXMd+~aLk|L64?@U)m>o+Uh!H!3CYPN8`|Fxa0qLQd@2BV68Z$I!fGpf&2kLD| z{dNx*BZCWQ+QCW9P_+%~KD4ULsa*6$XGxXt9WAxHgK0p_&3QtOuP7_PVuPzQBqdHi ztvseEW)7~Z)kG~oo@Yq@HHaV89T8#(YjQMU|Asou5veQs#*>pqk8O*}R*JQrL|&7W z4QeBLZ=zv`ChmXvh$(X-J=Q`~9)}Yco*Z#RL*b-J`4<`beOVj`=i&rwXk>dtkCj;8 zz`AM_O-RhEu&wb)vt6`W`6G8_vHz7tYz3I7IPq=t$Aklf{WTUq^Xj3}tS7Wj2YO5?6{?_)F$xtL}R zWY#g6tAWnu38R#6J$QS-M;#aD!O?FGzVF4ugC!|E$n}6Kp?CS=5m6>wP}T(M0P#S67*R= z5BS$wBA!g%!J+;)NxrF>jpW3bB6fi7`Ik@+&W%2Rnes3LpDR|$*Ia)CvNxt(qnt3gGl!*ra~p&CMHYsq{=0|g z&SUl`R-+Q3NAZO*>XD>&$38-Dy3&PUk!HpIqyXZ{zd^~0+p-hc3s77;bv zuZjMPG*Q%`CwOIu3BM6CVi10tP|(4lo2WzJUh%kyZ_~;^#Yba3_2S{w1Ngy&Q$KQ$S_k89G>B zCnpv8bi5yf|2;3kg^Ith=G>ycue%+44?BXn{_yB%9P$0SRH7Mi+PMsQigj6DYZ0Sl z$lzhu)EcxgL3`Sl_fC#J$%|sV-yr@S;lP}?LFT-U8te6o)dzK!U@FJW37rjXv zLgOy&5;(&y9CIb_R3W+OuQzAaZTP{k{HuiQV@`dB`XT>4RErof*w&gV&yM1&gXV2~ zj%MlzIdC8-Y>`ZZs-9jiH@cc&l+y;y%<%`ZwqBgsdpc@^-AaX~W6@^WQhG(JwB51^+fTV8vXefXPElSMOkS z-rxARg$u?Jv83}1CKWOE^6iQ~uzxOjw`MpAsR{`Su*p#Mh=2i1R``e|r#_#3hG@2# zaZTmA;rBfHR5(%Lceme9%(qZs<@s^FXWIrNO=a=Fo4(0;|H$4Tc&e%RYZs+*S{AVJeuckd!;;?{?ID?p8b&WVC$zSkpE24m!6z zQA$b_dN-nXTu@}2m=df@d=m)5TKYCn<0e~9U+$L=tieppx3&j%cL!2^^R)A*zu3@s zH?_K$G0;K1*5F5mv1iZmuye}Zb}XVyFm^x8RA@J-b%pL!daWswXkA@rRg0+CgoBOv@fjOV zKiE`U@5>NC8@+akEB7%APnM79yWOS0ys&m)N_1#Gl694UyOZKYu&vLn#(uqas@9@s zyHnc|aSMLX_{%myArQVG3C6~s;bThjM zyIOm)O=i@U*=z^k>Esj|Oi*AJGHerF$8HX~GksD8)0yt#BQ{;^w1Zl!Jn(wu-*2%_ zX(4M~UbK##+RpTN;OCf=ty%vCBif}6YnDtn_=sjYbfhXHGFxkgfH@n6Y@I^J zq)zu%=sq&>SrvzlhLeb|$I|q*Uhsm`jr{L)gdMj2ktnNq4v+Fp##x_bvw zSdBFZ+_aIX1jYdo%L_bi^~79DGE4KN&=OTQ^2r-w-n1=eF+$Vl)$?cmg#Pr1$MsKY zd$8715;;@OQekgcP7WUtGS!!RE4XMH$w9=C)d|M@nDXVXFVP^F$7cv*492{!29OO2 zB|43={_`Giju11nq4vziADS9BRLzMOso;Wu=%ibl6SWC;qYki$=u{|M{qB?tk@o3!?T|qQZK-h|tT+X)I@N+y`)Iz6` zU*KtmJ3R@QoNtYf7?nud3$L@72nYtW2ew| z*zP}7Z?2#A-AN%#TKf!Er0-5x{Bird-JK75U0M!A-TctH+Ajn)1ZGP6U-}IjDy=3j zZF4U^q};%S1?^Rhd7*2ABa_yqVvUn^1fju@Z7AgwdP95@@TMVDl9w5!jjWME} z{!)QdK~<5ZNoyu5QqV)q4sVYX6PSSmPxt)}-fg#%0rMa(C4t+yLq2^?BwZnhxA$(^ z)bQqY?w@>39mIp+#5g3^5?bKIv}TQw89Xc4jhMkkuUhMzqIT8l)^4sks7l@y_bL4S zLUruV1ngJ}VlwdTSW_YnAbF|;rPpU@FsxS2HpJNTZnGZ8a-$S8+J03Y$ zBGxC=v;DMZ1#4eFaXM^GUD$$AU572?W=)TVQ_P+if6m!hx287oDDRdgGiZROZi5B% zpR3o=o=#XvX}W=FVT5EF-7BUp944`|uY?=)Dec&#W?QD@p+}!~ai85xD-jz*b<9_S zeM%x$@WBKfW^j_*jay>=tXfTab9%Lil)Y@>4xFC|cbY8lW98`2o@t-NoIcCbzX!J;jNTv!_OKHrH_t2YK}$auBVmS)CaehVxN8%7YH z#56pT07hY#un_-B0y||Kx5M^yz^;77l!j!teKW3e{Obt4DdU)5I$qdc00w=PsoCz% z&pnxu9auZ%B-QBNGOv3ZA6rM5Y&~MMjVz=)e#-?fDwuk{$uiVM)$YhkS%u_lUhOYD z;boZgOUD8#q>PVn__Am&g7(;K0N0yQDgiq=`j`s68%~?vDTw06Ktt5P(ZMew=5&po zcK6eQqRLnP?PluOs6f$0twBrT3B8s{i>1n_Qq`F5-L!gqYbyzim%}GyA4!TvqV>UX zTT>eSE;Yq#pNF0BUaJ$ zJ&dtSQw1Lusg~_>my>#GansJmAr*JVu}F`}#A6lx7g%fWAk`?b=R9N=Z`mmg$NkG? zZs2v-gv_Kksfg~?w)6HA+{{Xrz>e~$XrxJZ3#TXjpdvMx#YFlXE!>#<>pi~M4FIE` z(aoPx%8<*ytL+UjpJBn?q#V#&vP8m`e>&ce^)Gy6Jl5~rI*ObqvMJt+K;LYEK^jaq zq{Z7F*6|>D|HU;zuv%am$BftV1;!H zNR%mEBh4xO`>r6-qR>s7d)l6yG3??{46rGA1NDIh!xxOP*Zsgixg{Wph!_3rt^mQi z<>9?$C9ConR+I3+6@hady|EEf=<{2)<`j9dg3K8naFdTN>4$!trI49?&o;j5sNm)5 zZYdH8rH8xF-6=t7NB~*> znVpp`cIDq=C2vvh0CB?9fN2db*_QtjvT0Z^zXYwFPc-qi<+(AAv37}aX5O~-Fiz{j zqf~eP|Ak!A4%@$`czmM!hn;q-7GD8e0e3J4hO+E%)2`rOSjLX|_4SUqkPH_AN7BVzD~%V^{TgS2McPY63Ov=b#gV08+YgeYZt%jxQzZ(E_A>`;7>{*KxD$g z<6bj9EFEx3Rx(JkUGc}Fq`8Tl*#~UGMHdJx6@WPy&-_q;XrsFr$+;^+qz&L-UlVuV zlD^ygZUwL6ILOdIC@1V-t!_zxEu=BCtKCb_7LVJCt-uYFbZh+b`ZzO!gGF`h=E9^$ zGlb=F?KohtG;q4&;)Gb!$WI$rOu=#9cBiR_UnKgk0%aobS|us=t|McQn4RvljSZ1tL#9T$kR<3ef*0z@z@ zX_M+N-~uP@|28&)_~u<{cj%-+38+~06I9PORRMN$DuKW?^ z1`brKEI`x%$OgaN;fgAzW#d9w7m!tay^$odI-}1D4*Uzb01qFD5anyd74h$RITg}j z9EVt>CkUdAi>ykz|lIo)97E}_I270U+=+wVyw2i-l0 z$A|DA3S`J2#gL<)k%hO%1U?<5ymGo5uflLVa*fa7sqXo8x#~@!$2fmiK z{`8cE5%G*P%`xNa7%Of8(M=s~;0ZZMHnwJ4_r}Q$$ydN@hVZa8e+0^NBLb;EUC=>o$x`2lCm1W}V`0`?${i$SoTFDAnAtqc9 zFV-V+lu?fnhy6K?XKwO6j-BQ(V)3lHF-c>X%9Tehmt*-GlM=B9+U`%=%( z4bS<6GG1P=r{hDnK!_HV^e}o;HIMTr2bMM_Nrx6ayt}5eS!Do))GoHR*-pC;U6ISV zVJrKhKhEI16&~wiN4KWZ?>7iR$q@iF@APbYlY9;O7Z;*5m&J1dTA2O~m>{&uaS~bh zL^Y`omBl9Hh8{>FF0MYsQw^LyB81%Jz{X=-A0&Mq?N@+XtSb9DB4v$0iyq%aV$&VL zg4KRc77S@!S>1+A4n&&eDcy49#e$^4fw1svGU$H#t&@IjKs>HRzvk2IS1TSfSh%0p zo|pfuB!$Sjt?2(KfX~p#ZtU7SQyfSk76_}11*_i!OD{|jS~F|_FZ6YJGq$j#T1!Dh zJgk9dD=qNu0Kh7;K(YYb@yvCh&G(rG{YH_MQ%$*f(cHu9^k_rJ_)l7R&!iRv>PnXl zCY!IuEA4MRxyeMVG{`+qFsh9Qh0TLNH45M-MbEeLzc)@-oG8d&|UC&dj+H$cjx-iRG8yC?8U(kWV+EcmDWwTP}Q%j26%tuZ)I@6)}Y;=%mC z4ghgMm@?zbDP()DY7`7I)`IjpLZvVrNsrJiIYIsBc!bW?d%j3E97q)q0)$ibJu>x1 zvy5*PWr7p|xN`EBKk+p{gy<+oPy{LjVlD>Ye*fSy)VvmT#vCk}dUhMHER{$)oc2Dc ztOYc8BlSF_!Vk$7U&QkVAU+weafFkA#GYU@t&M7Ks6I8$=-R5k2yw67Lv2tk+21=o zVt6E98D2!OjUj*%rZxfNe#6ZDmK{zDbLo=_Kx2yeM zTYh~dhKzwi?${Tu#=G(vD4xQM0KuwZr2CoHw`IKSL>^7oTS0qjtk=`00a*_4oSkV6 zJBRs0P&Eahf*%$ce#R;KEf<#hkF{$aVV3Dso7`zaIZcMHw3DPE7qOep{jCAaAWS=0 z`eG{c3N{(I4=$`zbQGR#+4Rpr%VPhBAqT$YzXE%5HeOzN2ZS%hPJ7zr{J0C0z4CA2 zBeo|?o_H|JWH&rRS2VHh3MV#wv9)-icw2S^q8${ETeAKUa0suw(_L@Jy^WI(yCd!T z!;kj>RE1b$EQLZQ4b^w%iW*vDF5+Aj40OpdI>)gb#BNNzxTO|AxM=a zJrjVxwS;>zte8;dh|-t4+1^Z7xiFSjhaC`uUEx=fz4YA-Q2=p!M@M!`#Prp1 zAvj3Pt6F3DJF^O&4ClDh^byF4RaPkivL}aEvGya=p2y*rD_1Q7sD}c><%g0|Rra5i$)l!}D0pSLcZS}KNkR!(j!7ubZbRh5g zCZ#keTVLXLLpCEMqkR{G6rNiO$=Jga>W^d|?MP*|v*GszX%;R6{?y$`AI1*uwaShg znq#LJ{RtuK*F2qKIu^e&?SaJFohdy|-DtSfBxtsFoaL*5vI~yevOR6BSh5NAqz%X) zK`|~j66AQoa?1h-auEo~XJmM7u;MNO0@P5M)8katCxaW(qbPEnK{!ma-Wo9bhEGtY zWYm>4Q`Er!SmU1zkZ=#u5xw4PubYr&eVq$xe@YHht;><$@Zc*M_D>hwYv9c1G7|2!_`pzGx~ob)XTO(+SSQEQsM@DnC~ zT_VZ5Fz`8$6tV>IkzR;Ww(+r7g_mq{#P>(rq<%ah-OWvD!4*_ylC!DO=pqcMj;XrW zLghqbk?+lg_%c$iu1@u)0O*}NBq znf&9{n&8@f-=5-H%4aUQRQPWyEScY~RG`RKlMebJNRsERgbM#78jm{(>hOPez=FqM z`2%64YrxN&D7Z@?NaTUQZNrk^4^^YtrY0THzwLAVGnGfMMKUc5Kt4@@CiT@*7tCEK z)tws+YOUAiYP8nDLOPm z@Gb}(DCtwC-1Vdh#1h^@NmL*W)q{~NkH*N4^kejzFFKQA+Pcj?un`k+kO`)`<3Wyz zmZ*T5&B;kXSqdlYsQ-6RkiywZi0Q=jZS|&<)$xy1Od&h86-4~3M%vxV<|l8#7Jx-W z8M$^jQ@keu>QW+PFCR@h3v20bRe?`ycP9&{v9BTh5CzeM?0uqe;vcb+-#^xHg$3~$ z_+f1-eZA`;@G0+IaI-8I1&d$9i?|Q66(WqgiJ>y-=jgRP+bS2Akzf?w>}R}mVUL`2q)wSm%7vFO#G0o zL011onl0^($kXE+xH4kZ&qlB8-ybd4>4@T0g7UWs+0P_KT-59B&0`Q~p??Ij9ScP_ z(`MtHMgb`iHS%zJK0>YOwnJmk91(~i3e;pK*EH^ye(ij%+g=)a26)_#x2>#j5OP}9 zxb4<|k8exD6X?6`LrTKmCT&n&6efUrqa|e;5(S9xpLncz;$(vq@*;OHL5`D%5>>cCa74T+YQpr6#y#LKq`lkjSdTJ7$DYgNd^tbg4t!qX`4;u&2`V6?$mmMXK;}=K3Rmxk+LjN5 z|GP_Dn&x>@{Ye6F`^-R)eGcM4sLPt7nd22|=s zd@u!zOM4Q8CcXCb-7`_hLQ;^(OnKQ>4NwCh5D>3)PGtKow|kI=Z~Z-un5ykHZ-V%1 z-ZDRy)e}@(X+}(r?9I~I`+=sAIw8B!`WN%tw^vs?$}}i2!3G-W#8?SXTmj})ZMZBx z11giR$@S-J(I}qx%?M&jg z6)WLwQgAVGEg6*FIhE2}d14UZ22$5sHu|S(C&t-dBE8r$f3iAXEDQg zIe_yF-gkTCt&useSs?EYQxR^F3i+g}bcZyQE~R!vp-3WRh@W@0A0=kJl|)a7nCqPq^W z(R-pY@;H?066qVijwxTw><*=U5zA*l0zJ(@7XaoC)z5Tde6Q6 z2{^^Sw*W*1EcJGl8F>!vLn@*gch^)lhMqtD$1ub0k1bj9_3b;^1Esf2&0G)qFI|BU zDBk8R>j5vtmc^ep9D$YFPj+=Ym{;GtJb~~P1RgWsqQx78l^SndXbFP!zWNEMyKJ&E z>qo1Cd~)H*QA-P7Db8J;+Rd0rc1JA%f1xwSy(_fINVb!VU!Kxu^!tVf**Syia;%`o zES9<#O~`6vMc=*i^j4aN3o*gb&zO>bMU-}*O*{VQl-^~da|$*Z%~rGc*`JtxdcJ9t zd=iX{_#i!{Cz$#96S)h(r z=40Xc6A!0+@}`L!gP_X)HuJ{eJ-?cZ?Ps@sH4ef48QXtUL3N#cO{g}pJdi5QO@b3E zL|P8_=fJ!d2YXu`$YEVwmsKL1W_Mx(Yz>jjUE)DA#R+TIGtNN4cV*pq$y=dPO(|y| z4T`hQ5|v+E%NzW1-a3PvGeEso09RYyclV&Z?kc?X+eiqWzA7#eQ~DH)N-4zR;04QQ zlKa{(hqveK;I1r{SsHog?t{9zjw@e^jlOy}H3H9Z42)i{;fu{EzXO7IB}(|aM0 zp;&#;HJI42=i%ZytmndVa$lD4J~3+<_et|k!0JIdk~MzAKY>W{I+GqgVo=j|_Ljw( z1C7KraSqtB$3u6Pb7>opR=5UvlR1`i(>=KbaH-*6AA}XyMN-m*+m4nVzg%Zl$M9^L zcG(d9or)QY9j$)uNP$8>$mJ@E+Lua}zsryfleVXIa2Q1D!zGtZ$o@O)yclM%NPs%= zzl{;=(k)Z%$WS-X1<4a<7q*;N%}BS_yHp!oEG`pdQ=7Cg^Ih>yw4?XP~ErO!*$&|41wXeea>zZa``tDNs46hBsE>A_=i_s~!@;#6li zcO$&-i=pBU4wI%drR*1N1%cbmfsFO zUMEnmZ!zIUGqZj?c66s%UvkWuxYvc%nF=OW*ocmbe{CCEp4wZ?lkpG-F_=(16UlbS`g}2>G zsgYx;?!m4G>Bj=0>{)Zx{Zkf=ckQMj+OQbx3k9g`hwi}dH(VTK1!<^`Ta#4ZP1u`F zLnV?wCK3OBbdxQ9oP*L01s)^O1YGuVzTGlYwOE|$5eokPSh>8}MSa*r)jnmQBq$iu zBE&{l?fXgdiV6WGNYTKZ<(Lor_BGJLEi*PB`yCA{hOL2TBNvSJ6CF=+SR*aTvD;I# zG$;QtJPy3gOn;E3Fp}-$jUB3!XCTFBLG=Q?Zdn77%U|B&cXjQ$Co4KW=s!%0wtqCt z1tgFjwmDjK%5eBOHI;2xiV*A>@9uQLgPDi-J{wa`dAGweuyld7ITp6#>J z{e--SQivywL;{+!XF}WNTa$@x4^x9ctvFvGa~3Q0ezK}$#<`fL!TdMGp8r6wZ28|S zBG1@aCcOhwv~AOT@WNQRE*tetLa1=o0$Zt_RCjF!&gi{bTmGnb*Vs7VGUNrUVQl=M z=!E@6`@CDM40LRFmJCcN<3zT4i6L!L4Xnw@iAYM+Q;aj&Nn><88rSP=v!?bG6a~WQ zd6uv(_b|tIj`+-E8q8+)sFRY)Yi#Qii&y19K7;L6w6}eZIDo@z7mtFbe-G}I0@lJ!<**`j?YD*Cjc|FcljYC%NK72L(0Dt-6Yykg2yVcK{H*#CxXI;_nI5XV$PL z&^7=kcn21?)t0HrzxQBz@5hwbyJt7UVlg2}<(c3u(Xg)$1~xDFx+1w~_%xCAe3yf)|3rLa&`YYI+q8a0jXj8=+^=bF9m?O9Ih4VkX0!7kOW?r7Q3})pRjm znF{5gtKQALqQmuX5UpxJv)5$wk1Y~H7^<>h*yA2CQ$#9^a#3~QObpfYz`P1l1Kkv` z-dEzipxL$u`y){kZgCA>XIT-VU{1plYE#>j+I7VPrq_; zN3WRY`xjYr(z#T98g!9nq00pjR>tw>lwAzIr{|S>RR`yvdz5bOeXi2$uDOeB3)lZHwON-N-iS2GvbYU8`b23)^z#_h*>cH{E0m}Yf` z_lKQ@KRwW$7CDSZW-5FV!nlp$=O{MCZ3u2K0V5_;s}xEc1lfu&BBkO@;7H>fjzTlv+Ig z=^ue|Ip+DPUb0`XXhNoj8-;&qv(=zkX1iKf_iOyg_vlP&txA*b{^7?*@Vjlhb+(7U zi^8eHp8bQKtZ652!eg~{ys{D#*b@{V&)YZNqaQkv?BU(d&@OkX?br{fjNP zJozU7^LuQ#Ev08|+lhV;;~ZuM>edyE7OMTwx_hADqdoPm@EDQrOWAQSa)*XzfWau;jw zw-jmiJBw4Y*V%|Z;V3wP7pa1KpuP`RamsfK5aA#vHW1Pq@bg&qY@OnM2kxkt@tAb;7H~XaQowr!#Ifh95mGKSM7IV)Fp5#TEb+)SO04n0lwoU(@`LDe$~EG0Qp}zgLZ9_Y`pZKFB0zznqDHA3q5Qh+`ng zBnbO=YRmf>Fw*J`V|>oZbl&o>xPxzQF(_a(1Vn? z)RkqRy+l2r{}OJ#g?pXo&CC-e1I@z7Od5@Wy7=*1YME+JACcf;)X-^a7shz3iI<=> zK2uRt&+Q^hnj&7wM|~b>Mu>RsU9eY%RQ>*?81_2Z_rLr~5E$S$$Kw7=_DVw=ieQ~f zj{G0(F{`!hEHSv_A)8E0rK3^e&za#NALGsOc$RvORdl`gT;|<%wppaV6>nJPKb-Zk zXKgWYKQR`yw4AM9ex~{t*hwjULm{mh<~nUTpU>Co8@J_ntCO)0pT^BfM&sQktW|wS z-c4F~=wWzXb3Igr;z1)G{A+UO;?%iXoe5ti7TqU-X|mo1rv8lPpWV;{pYYi1Q%Kt^ z`30U#;Y+Rq56{Gsj`a?kCL`HN`q5>yTe&jzt z-)U!p$JEXVr+xkD>fsIS6YaM6CZI zS}c?7snqlw);CRe7x2TP$S$MO(!D3s;f=h3pW-~SIa}R4rWwN9HP}!7_RDiNw&eaG zXuW)8fIZxn`Ej6HuF9H`Sjv+o)1xi#IjzwEx=ltyfmcmJG5irTO6oq3MGJ};d8LDM z-c9UM%PIT|{IUaV(~DDS(QMPr0s*vI-OocGgt?~^`7KP>-~z^Q1W%&+=k67SzQND3l`@x1#K`k zxCN|X-TBtoc=l(-a5yXbBq3$CgU;cJk)Xc?hKDqZxnPl}fgP)7Y$Oh&`LOp9Ubb>X z(^=d$8%rVe*@kuU=qX0wpjH;?A@t`d=PKt)tGXhof~@HWqdTrf({*SS^{UCUk!`n0 zf6CX6UA9IC-tQ~#4(DtHGR%pDb^BlP@%)3Q93OLvt5Sa$B{uY$$Vl9qTz5_0y>evB z8r(X6fQOOZ4WJyja*n-`tONNXU^%Cn@m}rr)rhQ=80{9>~P)Y zqT@>F0$5x5;AmoGBvxidUYVpjlhM;YbY?RzVV(RNFi1J@W)*@gO?WXBNQ_a&XyW_y zFS$GG2v7gjUPHYKal=L)8GFAi7Z zH)H}h2j__r2i-i(e(~@M!yXed+goV1e5 z?Ei8;;|&;UyB5}h+cmTgg-JMquOJ%x z;~6f{t`|p#7g?bTFH2XtJZl!VM$cFM^gDI|3>fkvE@mAdN+!T;@-a_Glkkc$qau`J znYpY4xdihl7uEW$Gg95w0uQeL7(>^U+Ad_zmmKY1WVmu+q+cfERlH3$OP2>>zz%H4 zhRLE0HLjH&FKA%6_JxO#ZGC_dT)Ul{}rg^MO9z<#;2BU#FYQ!9|Yp94SF zHKDeKCTUW|6NO+;hR>1+YY>0jhBiYEXU|M7MglLswjEtB-{eS-Q^KSqXXLt2ygtm9 zcK^yhVjZ;|K7Y*9*mpva7oAk-d=HUC~oo5~HM-O8=ozNj|OAewR|<+>)Dk?32ena>z>5}psR-Z8`Ejqzzdv<`BW zLF_pU1XExxwsh@k9$1l3|4!Jj|9MaK_t}X1<~ZZU->P5blNJ@vo^t+rBAWlT?(|$E z0$%X-VVrU1%UWAmimfJJ9v~V#+=tuE?|yLHNtdO^BWelD9Le7=zbEb4MJM)LJt%vl zX&_T+9Jyejj1Pa`lzdiy`(=*O_gD1{=3L=4u){oQ z(+|pUnK3=i!lwG1l{?((vipao(t848Uc(+w2&N(DyyZiaUX9_#H^qB7wt3d$Wm6tGBNN?en>d7C zHQ>-~hyh4Q0B&Y!K>^4hwX{HgWNiBS;l}{3L+foZN`zprfP52SJr2`NX^fp>UQD;> zTM3<@(I9V1lW)F!&9M90>@qb*Pia-2qDH?~eV8-;TakBeKPJo6Ve2@7qTW{mYwuQF ztXQ0<+18sfC4UyO=vfNifV<=$%uGswgow3}jm{;g(j$$s;ED3QZPDAfoir~-q|D^9 z!a4`yVtsqPE%@(!kLmOI(opx`)3WE&v*XUzgiDrNlLS>E$P>mX2TvP{hpNdS5Iu_n zEmaUN)P@jekotarw3ouYY<607WTHP^ZyUT~BId2THm}Bg7)eR!S?>DbR?EdbJ1v5w z@soq_){~VBr{8C%-MCH}=ptGU zJ=>B4XBQUxoEfQb$kdq0Hw5D*SO5$#nVDhM-Wq|*l1IGVA)%t@eo9->iJZR1CTzm) z#Pt4E_)CqDzV*&GL5lL|p3tCjMzeG7w#Gu$n>r;tpdp=S3^H@MOz?&bkhA@o!0|PODRKG8eJKOhIY@~#C~_uNSEsDxSrWb zhe#jC!q4YB6;JerG3Tr4z36GXObdbW>*9&VUk_*)B66<&B)|f_Lk92kfyvyX`1i{k zpE`?AQZI5#({>M2hhdNx2N#Q_nQBw2SGMOxDXNV_e~fAmzzcfJ{NvUN_1CkZi(L}6 z&?XUfR8{mzu`JtaUGF#8-`&(@oaK3=WRpY~nHt|r5c6BviqFy<`BP6y0)o2lPC8)@ zuOu9o-N9n|@DQAsS?}7^Qx3FzzxHcouB!KaZk=bm7`Ck%+500^#^nJ6pn{<*h-%~Z zPYE_cRXO3l7Ac8$9!O!_*<3U^37zWcbP>8=H_)nt6!)95z3`GU^}Xm+4D^9*q8ZYf zuN|G&*>FmDGSM+*0B^9o@XAh+9hq3?Jw5a;Q&otuk#UR$3^h#n>j8qtd9{E;ER!Iq z2C?*dQ-lP(by>=~l46FTD5(5&9^|V3QSVlNwS%g!iKyE>%6<+BnN(-~$&-vwm$NY0 z(^5OB{*5CyZaTSyUcYaz543w~daLH~!e_N|uq&y3i0CnW8MT4?6wh3KorhT;%)9nI8$33qO(CUUg*9KhXc}A+IG| z-CM#o%`qaF-OitGxL_BP|LHU_WaVOfdGhIl#cGFwAD=E<*)d9r0Rg6bwex3Phavf{ zhv!c+!0s%nc{LYB2o0D6YJj_6))%K1TUn8tiko-xxObi9-l%KUo5#i(nin%k(qx<3 z7kiX9Es!Cy>d8G{T4i>g?`ViN+%~wE)1X<-YGJe=H2b$Jv@z~gBZ8Q=GLuTdMTmGe za-{094rb(Z{Nw(WBI?LH~hA|N0OOzFTRM9c;Xi+imHgTtufZa_ez=&Aw)8-z8)9(@1Ny0JWc30W4HMB;j)3zip$c@BhY3)9GqKtI zH$M|yiD@2NzLvqQx=xXnnI0)we~m`XW#@>X?f!)99ba%!cn{HzU>6?|X}-3jnY_QH zNfGdSuL=XmC$pRva9AsPW6;s#-&+!t9A`popr7hS2Y$53lLHg7P zv}R%)RU4X9rp^DWYE)-AUF=Oj1UpxVSj%H9ceD$(LV=b)^@lw_h_BsJ8=zNf3I-zl&P%8v zrI15iMfG&)jyph;+bzi_T6KX--*$H)=yj8ZmDY#ZFS*rZ@M96Uw0nB+_HK9O-zRNm z?WM@aX`7dv!j~ggN3RLvCfl|?NBJb4R6IFb;`JVg|H1oiLzSNM!F;Q_+Wg~Ifi-IW zOscIf(5=QduR`tY_9#j%1KPgLMv<~3f1G5SqW-qFT8!%sn8?2B`Eqj28WUeca;V+& zf_Ook`daxfs=L0SekT@pWJ09<-&mjEo%ryWNIx!*z5YlVv@K~OvXp+)N9S>nFWpQ3 zNLKtAe0ejRQUOB_*OtNY*M6EF97<@LMDwqNG(4*ZydVQ9tF@eQ1>OR(5`jJ2YQs4l zJ-w~Ti-Xyt+0qA|%040wo;hQ>#qA3?^a?R`BKJB4M#?J2kdKR{GQ%sUz<0AFdi&G2 zW}_&H<#K2$s{Tmo7WT1~sy}{(x*#ctXKR^ae!YmRLjyuwG-zaqZAG7yE}S^X9S5tX zg_Nu;jsWEuB8sOEQQ_sbp>^z2`i|V!rE%IRu>ooBhVVt0`O7rx3=$o&!-=Wz6Vqse zA&s=^gj*r9Xl&0&yC)|*lC_0+c2F^YtuX>Fy%f;{HW)fz9shDQIqQIY|F^=h+r;Tz zZyU@Y3p*$(ujvH+khjKS4dI*8V+QqBP1a!6%0fJEk>ov}{#lwgZ` zxFlGuD6LaW19UqJ9po^Gy#iPnRqs0|Up?yEhQFc)Sz23uEXT6T3IAIXNimH<8;z=W zkJ@UBdEy3h2LG?`xLTYgj`Yi8#jK$(jQ?r2=6uV^s2yi25yy80qXV zcFD27YL&kZva%!L1(L3y^V<+Axk+tY*V5aRXY=KZFsqQQ*+A9q-_cSwa-+xWWGY`g zew6(zB#$hIThZ$yfg!d$lP}4Ou^g@kBpO%q$Jk%4%4Y{zH|W*O)z?5a`|7?_cayV( z(tRmDQ75#aTAf(jZc_#NeDx>nmRY@td)ie< zhf~6&$*hZ^?GHz^j>pp`UkBxN{+manT-mVI)xlHL{7TPLW#rQ;&-TmK!TElzzvsD` zhAsCS$P!agq%9iv*0v2TSFo5*?A(GyjNQkFftfhJhh}E1gx*(NATnA zkw`+09@Ir%$)}YFx+X6c$;%Z+;lCq)^~E2B5n0x>{@C(ealRD7?uuEx*x9=V%MZmLq*JyoCdPgZ1BF?%HrtX&$;yNUG>k1T=&qh~}KS;)LV(_s*z5T|unf8caV z-Rz=pa*$ECjf9^>g+}>9`_41K&X?7?9-`IRNYw`LZ*r(tMJWxVzEMD|SQ87{X^9K` zx&0pJ`$&c;0rS=e8@B!rlI6YF#~ClK$*Vee3ZNMflYO>nSuSu}M9L}^p0vSaW2PH< zpIQ~1pzcx%S@5bhYN+Iloz0}8&LB*XnC;bM&{it@&gA0N(O!MnaVQ*n9X9p4wR4LF zgdE%7xiPQl2`}4|p#cnx_jixRp*3e#6NyGG6BBy-lJK4sn71J%A=^6JBM_$*!#}Aa z7pplDue$B!bT2hOvb)zn0V;yynkD^KU*3xnr1wRl3Z7Cvd<9=mrA_3{QVh0rH0G3` zV9J*AZ+J$bWnIEsNAM7QFbyt#D5(vx%dMSwJ^%5*2MM_o{VV(^Dbo-4D4h)TBEJEn)J5gO{H|P>_g?*D zgfjS*v2kU~X>c$LAQNF2wxlfUv-YgH+t@4MicYSHY95TnGhZZS0!PvwWZgHN5Hu$^OMYBFfw|5P4Q%hnjN znF8qzQz*4ns@Yavg4z8D7A+-UywS9@Y6BXbslp)jrWYW6ZEHf$W`wCEXG@F={Rf9;U#TgrY{UdS740y60T5hHuYQ!l%<66QTt!mb(t zcLGbh8kp5Dc62H>KN|A166@YwVC=n`A0~Dw)8N>-M)o_LvR<7lsYI=zr%n?mb!jD2 zVgSU$6AAdD2ifht^@Nj?H$r?q4pzJmm1OAk={|^G=2&e=xnP^F>JlVjxYSiEuqwJq z1}EUT`I;x1t+T_v7hh*NPy9KMl|nERVM5GTHt^Bc^Xi9y9%K~tUCRZ!UZweR!BCX3 z(xdzWyU6QE&3;a({Z;&l=H^*6OYn-tS-G7|X>>Ivz8w6=9QZ%LZ2`$_-Z*B?RTtgA zLzg&uUozQib9;Np^71sXT(pq4#|2{cxWDa#!?$`F@loxwXjzl)-4;6w^kS_o-BVJv zzf1R{;2UlKS?SX%?X*!Za$|xm{~t}?9Z&W9{(p=Jp^|JWd&|yFQMR`&vyho_>`fUZ zE62#*dn@BuM}!U`%F0TPWF1-O;Nbl3yi zP(@&^w=4vE* zF=|e0F#j`%?J}B1+#6mw`|i$B=53rV7wIsjkpbiV_@dOCh~f<)P6DwyZ}!<>(m7gG z%cgdW8XMMKy>pt4pOqfl2N{`VM&j3%Z*-S?q*5i7c)xRjGn6=9@@Nu%i86{u;g>ej zetYQ0ME$QI8SrPhNHLk3i%TQPM(<#9NonCIbmt5i{%8l^)21-F0GQujvR&^Jfb~Xr zU(<{+0%HptC3`j0dDh!st`)@HEdAe#{(Y8=2r%*8U%_L;4d0#d*qpA*1ns`X2aT6Y zh&_uxi8p(?Q z4ie$um5_HD~MA0zv%q;Rav!3qK&w9j}F({!6Ok3 zA^WR9vVY|e4*@RrNN}!yCtl%c(lG}gGy)%t1f~#JWi}DYg-PnCA(%R7H@IR+7JkT* z8Ma4@k4;#yKFvhb0c-!XRZmwMdVLP1$dgdwe`!F+UXa^xPWhuq{Mw*007a~#GWn21 z>cP~$_mv-8cAS0}{0H+<19?xibF=;2A@2Ow&KYkuHaqp;e7AlClg$dS!mzH$)y34E zQx(ajo#BJnJ(XM|h>95?>+6d?sntE-riCALPa%^`Zu>e*TQbL!F?~xo;n1}s1a91? zl7C{tc=y%@V7TwfW-Q&RQtWykDQVPLg3}k;AH3i|_%iBYjfv%hmm+GbxgEE{VEFX` zubs1b(NI&P$&xa`jvJD#T_-OyelN*Tey>7Bu9e@tSi(>^_qJ=}5ILCM0LaR|sYoKo z4Y8R+qVAK}a6{$e5gS4AgNKJSk}-w6`gc^3Q6TG;IDQfJAQpXag=--&EK$4WAd=+5 zJFK~`-!Ls8N_C%hL8>XA1cw_Z;oJ0=kKhxEWuYjKl()Am&r z*+*OW(e)u*V6GK{Ul*Jn=$p z?jqXHj1|u@_>&dl2R3ga1aGbh+|*n|#xOq~?Ac#o8B0pqOF-d$z3wE^&k*90h`j^_ zCBLy1I})dxD>#btU;Bu?Y2%vPW|||O(pI;aP3}_AiqBqs3X`A3-m$4B`E~!2F-&Mp z_dER5&ppFzB7ypU>v(*gKHxdY)H7N(Eb=x=zvKNW3X9*tTlS|@mHoI<{#fm)i#UC- zqP9I=k%IfXh+uEmV4RDHCf=EmxiTD z1r2o6Byz+?jTBP1kULe7fQOS@Py!gD&4jRzXb9SH$`}+hPXanxZ2iF>nz;ofYXn*R z3U-Py6uvM|P^qmQLl#^oNlFSm@*LN7&>~Yv(`d(>vi=V{z}O5@NKaDX;2?WIh#N!- zR2$I4CcGy;_iEEEeRw*Qmw6=nDr4Rqj6M}Aw5Qgt<_SgfJ1HnhpNfObm6!mdJxLbF zIL~Rqj0?9hqkpWR875f3&rQ3MNjI)tfL*iuhSt{Ra}by(3=YQFVE6Dl{#@y&cK^$M zjuhSvgRezo0i$oV$DZu12R%t^9_os-GJDg^r0cdn3$AEYpV-794^|i2E+Hf`*Fu$7EmNL(DNFPgEjUIq!Q%bUpUDoi|EP zMCF8u{w>5>T>b9zo7*!ZG29VfbU5UCc!-Fv$cyLvz4N|L2SP!X!cUf*OvUZ3`HoFZ zg-iTevwZgKwq>Vp`egsfzR6_ZBDQlaL>V&9Ree(1I&0kT9kv#@G$@n$Ron5^jEF<) z>erQyed{GBw;8I&l3<&y%tQ);T5QdK@w<1$G>0f4oAeTHfcAs#>&Fr~o*0w>cW4%S z*EQvxscgyerp^e>N&BCa8ujrUD{!+3YJ%6d9cg#jYm|*7{WTbbi=E4bd36t*d7yf! zvX*D6R5kMTPoA9Wu3sN^L4Eafi@KY-o#I}Z8BD<`M)+N~)g}u8u8WEy66}aXEc?=P*}*u5fCElNbh|iQ zfQy4fgY!dX!8i}Yjk@?4u#;f>ffek-2ZN&obGj9;@NI>aJVXdWars9=M`BgAZ`N|B zi3D~^_?e!Hh)TeA;1chh)TnK*z8)Y#iIlSx^x(7Oa9)D=ud3hfh>J2GY})Cjy2VDa zvMcJRJK(#mV>X2U{83+yKECkEM{B;!>=9D_M$1v1Z$H={G?uV{!FPZ&b7cJ2XdTy; z3z41rq_R+`ejD*zR`NV(I`R}RYW+t4IvtFP%7j6WgC*guY)JJ_nP$PoY`Ith8hTP{ zcn~Wsy8c%Q>5*n^Te!z$VPsMJ{Yq??uN9y9m)sxq(A!lTyUFI0sD;2JyLT8qhw#F) z#(<=a8-!!;r3@`dtIotV5MBO-?vxP=yFAQYuqoNc#5`KOjzKMMhN>b3|I1({Un7P%;%N$oLC-;i*>kXQ-9_&glOzYxeJg@&Pi`+d*e>xyH;+a>SP5(O>y0fb^ zl2A7}l7CG>yWQJjn?qKNnPJx+;E&6NTMKYZ86e_F0)JW;xjbVs+*Y2pNBbQ*ix`u7 zp**Y_25K~DxV|au&PT(ilIZh=zbyODAj-k4~K2`)z%eOUB=df`oEMd(|u zC8Bg~sleUf+^e?X$liccXZUH>^1%%eu>}5%IScPJ3%H7t{XM3SeWS=WG1f3!S?S+M z`;HO|(9*PdarAm+4O;Eoy&45tWCJU)G!Tj7?MVL$zx*%H_o%qJO>_15-uRkHNii`d zylr#0U($EAS#bB93LbjIlekZox(R-DV4J*%DbKb5e_dxLTN=TDfE z@->18n&;xLLd6n2-ljB1b4%y836UbhU~R1$AwIB-?)6)*JK#yui8U$c`q_X$)vz3s z*VRwAd01)oHInZEDhH@hHh&EnO+yzcSUrOBOAzhThZn~X@*sN2!vLCaM=5Rd`e-Ek zw>$o8ikm|zKoSn>zyvwrbhF(HMlbIrhckxUmYaR{U`yf+b+p-~AD|jyaZ%lS{sq5hh58vhcwE$3??pG@`AUlAQ#Dyua!E@FbyZo1LB zKzZCH>akKOYyIMamCIUIcQQSlgsE7VkwK!DXEtTiY$BTKeKMqWkRuDc zshv0HybIV(7}ziK%z5zPxEJ8ECNJm-8l6>~8g54}sK>ki<<2Nk52pR9wx-DI{d*K) zLjKF#+`&*~*Zi{Etx$yWGf6I0+zs^=40<$}RCD{Li-y1ZDdS~Ak8{LxCjPMg;X2D# z(W*ijL$?Z~*ZD~%LF7KQ?ds5BGIFF+!han-nY+YUsG4Sp{yaCR(sKX0-19Yi%OUr}fgUGI z$zgqIY_ny%(VV~OZP6^fD~4_UvbvKVQRRhvRXr~UkQW7bJ?%tKu} zCjoa+rSE{5PRfw&h~XtsashS&m#lma));Z^;*4Axk*ppz*i#=gbyC{|U>ui0e*O zb&}ngYer$iToxX135@Fc3z>96F@{0Rfi^Z1)15~%T`;19B*s1_ISOfJ(0F+r@$~)%XN`or&nG7QvT_|6=N0i zFNkLpDxvK^kDljQP6&?SZc^IxH_HUhhbG|`hlFFfQXf7p9dFyxThIZ}$vu=X7)wMr zn+b1De^1ASbx)R9py)*0SDEQyJ924rFt0Sm+;;C{Fhez%LuYp_>7%*gT7}DInE?iu zvIDvpkpzbSmem7|{0eg&uje}Csb>`uMD3sUq7ag-a|UI9d^bi{vv!UMTJia*aiLbf zz5nW7huO!3wckYn1_XEWc&;?7V+Zx|friZ&wq#84D5;ao2fA|*FjOs@89SioD&<>q z;w}5XH4?O=^v&zoI1RHeH=b+A5Cz=1_<+sijq*yIuCDw|B5qscMigS5LD_@}xjp`O z#mrBqQ(Qw-aVPlpv>5U0{;Fpa<+F4@tJ}@&D|yp>crW~hUj@xxUd60dhu|Re{wICQ zaWaE%F=c|$Q=2~&D~@+*c~(vWIB|6Tw)7ta)(K&fiyPOn0Yze>AZed6 zFpzYuy^=}#A~D6KLQP+1&o>AEbW@?Q^%i<(DGXcc&(@Li&G7DSm0=oO!~ZcspLI~v zS2@MrFnJ1aH{E9^E;NXR&~_VvVb(l(>};f`xkHv}{v#9e-uQE#%19!r)#JI|=GokE zW<8cC9I>_wywCVU$)3nk)d|6s!2lt=V$3x#I0^#UHW|wOTh@dvLFxXoc^h%vYV-rQ=|wyoylcN~c(H4C%L^!gL=SW@R#IdnRr4Ti9iQ zl_+)Wa7V5D?WuMB3CBAHq(yh#UtTl3jn`cmn`V@Oh4US&ZbioY5l9%=2BC)}eG?)e zjBowN(cK23&C+>zMGiO!XHB?3p~n}Gq~@~sG<>-K9El|jl`*8((}od-tx2447Y`1G zqIHJ$N{(P+r<}sAB(2!r@M#d}*y~|&4-OMJz&O2{^5nS?rP4EAZU+6ZLfq;)B&U14 zty1#aPxZ2(^IY=1`moNMxWB|bAHN2KwEM{Dzy2fB0X4mXNIpnU`l;-W!e|y$GeK+( z;S+P=2GC(H3O=GA{jG*r@#y*EF@fo4u9PK(3Ao`RxH{#Bl3=G(Ml%vC`1e-9^+t5H zHE2B&uI?KA&-m-3pX!EH+*ejcoTveSm~qt zWZ$th)D=MlnK3B@f0IMQx88aAN3rYepO4RT+oyf2&}Ryffgx3I$E`}YGAwq;1Fi(z zdheDWwnYJfJmEIY!s($kgDP=6Au0^|kw*EJ6V+LRAv^*;ke=zxt?YV52HctY1*c@9 z5r{|TS<-bIkfu5*w}@3tSC+iZ8fZ_JN0LE~h6%QeQ0Bb2Fw(7A3dffK^-27k<^?&| zZ9c;-q_v3=L$diC5K?-2LhFeaA$m>qx3hEDN}HkiTzAPB>YVx&z@6lmEO2a`j-X#% zKl?vOLxLqn&@V3zyWvEDiJmejJYp|;!X=qgBNj^JAOeP&Hq4hT+fPPzXjzSnD=CDJ zlV+MtEIc7Vvn|WyIgGr@L$oD1kvsHuE0%@47;=-zxodT1iJ_3|iS3%bHbbD~r-ZH6 z&olJYLq!)%pfBGl4I|<A?6)sd{4XmBR-Nnl#N|JaP@1AZnIE;WM0PIr$ZxzOpqGoo)FG5)pL80Z|gn zCn9i(P}PE{@~5KDVTv00aYMY47T1D_g}wQm5oL{-W~)_h$S>;(?uAYdauu;zrQ#Vn zAj4?Obu?H9wj+I2QKe%QT91`uiwhl&(2Nu0`z91U-&j%_3Lny7yz|~I!}Z-?639gI z-Z{(|i+<9GgN~lTGskkXNo9leYC6O%NV5bh+S>dRz%By8Ux?^X$-!JD!lN zEu(X`HgYKaJEbYO?0?%<0DDF=w?v;Sr&W(W)cFhg8X*Q={H+g9qPH?OeylcHs(4)u zzVny#Ll@=FXz+_GGf5~pdt5-#VR&~246(Li76o(Ssh(B}uI|+Y<#yg^;L;!^&nrG+ z^0$yu1BEb7?d_a+;q0Z4!j?AWB-po7XnDa5eDI`U<0aCpK^H;tEZ{5ZQhFx}L~gx(iCnsJb?Q!*f+8(`F#xzTgB^vU8BRUlDN%NydKMp*$GagEP%yuQ~ls8OVAP z{{@g%%4uNW0ZoFYv$N?cjLyuA3E91rCT~5LZrifZUPODup{Wy}}jX?_;>MO+i4J94R1h7?^qjxzc zNb;%LhOzVOv8su&@>jle+sIs@6Te^9#-BPj);4j>quh$SojZ*cHR`OP#a>#)UXf~) zvmW~&L{0C#ks=^jM4i*gIc%~?r&q(R_#^X7(}Vg#OLTMdQ1hvPi-~bbo#E5;;pjr8 zyf{VYYk7Lu6P;ABj9~yGDj^9S~M&p^FS%5TeBwm}_NJ1;^=K*L`L&t-exbbSVXC z%v1*zWWbnjkF2j4o#P$h6DQ#+?I$sI7BB7*)fhYudV0zzv~EpD(5$~rs%X?^bXGH- zYniKM{E$qRd~c^GwB+bEGGRXYdcG&r6uV;)Q3`^LR-jVv8Hghaw?uv!=LuMIhd~MA zgv{UHh|Fs4%=WyVEf7DTbNl@~43EGgnX%1&oD3-^mbYfpHd#l5Nrb552NFVWLz00X z``?u$U4H*NQzhtyDA>$HXURHa&iHnt!V(X1^gL9Yg>QB7Rmfs^6_%_gZZi7vOAX0S zk3n1;3=L+pyF@ts2>;wl^GO))chAjD{#r8k07taMvG1>L18;fDB$RS%sFbLs8r6I* zU|gw=Wd37LpsH~tge8MT&a)SkG=w(I#pDpxIP;h-?|*raj@4+W09MX?^~qc=SGvWi zl}!e8_-iQZThHr0qGT}cKz zwj-@teZFj09p1lW^+*Ko5HslFeu;0w{#cIp{XgR~D|4|d=uONT`WkMQruwccbt!RcSV3KLiGfQOhu(g=MOD(G( zOgDK9+QlN=k@n#s{yZWgV;3v=C2D$F3Q}6!I?*uT5&(%c{cc!Pg<>{x?#{;%6gKaJ zAVW9MGrIxmy30J{Mx|RxwQX&k7qz{fpsLb^Ze#76F+dRMV(t+VVytu1ne!8E#&v1u zdvKOxo2!+&A=w6#aR*nf42oPCBx^|gmqztG!l6bW zm|T73WQa@gux-g~ckM}ri#$bsX$%fZbQRpRyW7A#bM%rzKUN#4L~K#h85jbflt{yWK#Uw-`ACxE=)G1b1MPA2UfB+0Dw z=O$l7{l=YS?sT8Ai&$x9wDh#^W&3|pZyYEADHLNT+Qf?u+IRMYA)hOEOxS+)jYkoi zawPR5MW?_{JH%L7D!voIZD-hO{(m>=HpEh@QIuBKn(7a*W3O_1J>8Z(YjiVGg4nUa z(r^z8dZ^U^P%N0u(T5N#h8Q~%-(ge>35X$t*3&PJRX@YPOU{p$M4A11qqlhXf5Dhc z1=!N{?^M3UDY6;ivd5N|JrVhU4Fsf*$^xL7)16Fs2ow{fKzGw;tX4}eipH!4uoeFP zi&Vzz?~wJgBhA14<+sU0YAi)wAMQTc(s;|wQ}37`htjds6G?^e^K*5Hhh0hcx!{Nk z+>K?Kpkqj(Uv{^J!EXS45G02d{uF41?!RFRGrJ`J)*iTL(f+3ULV%1T$e=-zkaU%A z97S_p80bkf4KJfiuphn-DRm0yRzd9k7R%_R^`ZZ1 zg-~OOJh&C!lQ$Oj7~aj>6ycstFalg<|7D413O;7DElfW9 zU6sO3-Ei7$#O#{P6 z7Lb+)RKzP1-68_v9OSxqs#X8bk5;}15jO?tW9%H&?bw6ocQlPC^{UfZPuy-diV|DR z4__H<=@cW5O1%^iZ^_o_1P)liXdn+9m2cvW=Ln6L8VVOR19zLx?O?i;bO-$A7&cIS zZ)pkBc$rsTv`5_irN+np$CzqczsXNnhnE_+;zz9xA&^eaAAi=9`gd5rX7obcm!8YC zLhH-RL}(KhURP1_hO%nOeaNP~p3Ib}xOk5#BW8u9kedFvhJU#U39!1SpOsWDK|cFe zifOf1-sw|-FVC3ktlZdh^@H)d_kDcUzpc|IE0*F)iCU@bwUoUt;SghIDEvm6vXSz6 z1}KB*grID!Sc1k z;O5XOK5zd?&E7im@M_Tt^3t_}gi*lD{?TVMpU?rKw7-^kKs`YE%(?fv)GY_cCM)8P zg1s7FGguOHUmRg+Pz=)6)ttOeWHHdfx+GPw8c+YlIoOM6#k3Kp4g!MMRFUi;Nnt9C zr1zZY-r6LZ3{yq&w7(0TobCw-;rbGA4~t$16t&pj*+ND$RAbTrO|D;2vt-GG0RB-V zhoHbCo?Hk;E15KwL~6yB5IcJvY&egMp*F7zaeTgApmL z=fO?Wm3s{jvafz5nG;+^e1y_<>NdW1XhttH>v)c$rXO#=F(HrC`JSX0$(@Kw@2_tX z#d@zXwnPKwmgHC}`{#||Peoedgfy@2CdV8IWA-P{2A&KK&>QscI0{&{IVF7t#nlrP@-jEBYyuW1rvSHxxg(7Whg8{_f0G)5VKh z{u7rN$1psyKR|@!kk7sYAy98D^vrz3N5Q$O@r&Gud()HWQRaY!^uB$rAJtb2^`kQgOF*hm93}w(VJbKeu2aSxj zR2p$7lNv2|f8o^oL02|Vs23@Xj2|3J7$naUfQ3F7D|f-{3?={--v@Wnzqayb(ZW;~djtb^3NmyUzi9FVPb(ex?oXliwrPs-_U|qeb+R}CMzMzXEJSW(cwlAr$ zWRN`^`(~oPNN@eU2iNq~!9lIYP9@(=HxV&YXkTOGjTtd!ojr!@u!j5mH8x7 z3W#(^>vWlelR_9_MJcKiR4aos##+|?KwY2R z}Hl_3cL9pmqqqw%PdP+xvLi>%gTkdfSTeT#cJVg0VUg3!4$&Ak{g@ zTa5UB<%mjb*dK-v^v{^*G9L_xyR#-NwQ%;hpK4)y!rLkH6VDF$@z%&v;0%@F1&a6*jk-rhjX2F*fOene#a}( zgJlEV=`R3=h3?v0T!zpld`>~90nO;I(YWt%+oME@OaP)8%sRrS)g?%HN6?l1vta`@wF90JmU&3-2i|lS?7( zRW&(M4hFY!{aSnMhkIjcEif(|0@k=Hq@Kbi70OVFtdF(WCxHW300oZEw3dz?x1J$s*XiZt~;ejXqXNp zHP(}gG6Dk~rdMBi4$`8V7!*dha?WhQoRS>E&*;9F5$3ig`nNB9C1mw1>_~oTY60#* zn^@4^LXS*1&A7uZxZE}RlTe9GuC{HKlw{{{DkdO#zF^}vpzb4ADT}?#M-x)@g()Cx z4bGFOSY^EHMC>pwuk)<$T{qSOU&%$ytEFu?jVb-RACJ=Y3)QW%iKE)Ro6D4>*UZ(B zY4B8J9;I1vn+ZHykfjFq)q7*ruTt)yzH$>Rw{p*%2QKo~#qr~uXRbW{2yrf~c;YSS zS8=?jZgW1c^lffuNz>iA($|StelrSIuO`bh4-T5IHRt=pw}6m>k&6VsxID_b9GHm< z(7sHdfVn44RMKGKkw$5aypS40spDlm&^m?LMGoOYK~9A|c-66#e?CR%huEloWMZt{ z5~#fU?$p$<(|V<-GYS?198-DIQuoR;P~KxyCi_O8 z(PT4``+4UH2mHo$iF;VZLtHi-YmcFM$~~!rv_t2E&3Wr~e$f=)7hX|+{<25AAkO3(;lvot*gCfg~ay)yuj!KAU3cLqBHo~;_JDYUQVQ*4PZ1wD&iqxVdlk<(SLX5z zRYxZ=o1-suG#cWz&6|iX$8UD!pSPwjtq-$=VmZ!!x6u0Dqp&de4Y*2sy&z=h>+b%l&h)SsQ?(Al1SO6zmg8W zQ`WtONl%qshom>k$qK~}sL7z7G#VYmB5<20(1U7UW%MoxKdXcFNgTE)Bw3g3M|eiu z_aLSIrG{cIH^-Yz==ex<<)@mR* zP+{3(Nw5H&8GiD~2herJ?#^*+_=?Am#(I=^28^u_BhRhBC?YVB9uFP zf9HJ3=4{-gj6sqHKm4^-TYfjuhw;M}G`S5Y3xj-DMw27E+e zT470L-8Um!hAsg5jtD_}rDY`9@r`m}VllU%pcU%jYHXShull1b zqfZ3$Z>mRCu!KJjNH{jCR zRjszlbW)QKlsq2&o3!W9+C|g7v_GscZA>xM_`Q$N5C9NlK|&lrd%4`W*53@C$xg>8ZNB|z^|=p?G}Z(?lFvB`j&DR`1TOXf3ki{KYB3n*BnhVJOeCtzm<7<?*EH$yJenKQIUzs4xi0`Xe5_wb6AgpuE86n~3OfIq-6B)* zrlL(aQ7^JFhgl&C09=vuy{Ngf*UqCpPG=c})PW6VuZhYhP+xoW4p@ zD3Vc5S6V<9+V>-pO!?F2pLgYZTdeNQ)l05>Lia+fd?O2Vqf%5luF&&@i-8j54?(H} zSy2+Dn-eX({@>VtqyDDNxI+kw4Hm+5_Cet;^QO>uua{A)06viglsk+=9*7B4FS28Q zaS^o8ByV4;3M6y&`9=0EjjpUytnv3B*uW1{J4f7%-tR~wBl)E{t$W!v_+s0TXPgxUg=%Tfl z7jm(xY@BIY?kln{+8u z<3#*88aC+SBpV1?QqvAHd$+KP!uEXkwkzjd*;N?Lw9Z|ZxMc$o`)l|eI6 zt&gSWkqu^!O~3G^8nAZPfXz_uj5lEEnkPQjDb1g~sp-v zuLz(SkSV3^KVkBfR|5axRNuHs(?0a$!728GY=%=Ow6?CK>tCDRi#su(5Tw;1Eb{w$ zcNC02(sK`lkVrPp=tVt)I4}nX?RfrBQjPT4J%gjAdJ7<7H+o77gD$~!L+S11-~`v1yfs%cvg{+tZYY z{E{0osWm7lgW6uj2S|IR(aSX_5PNkMTE3>6aOFOViArGQ3yB?UtoV8Dqh~nwnEc-ZC6!3id)?h(PJDfpu`sC zRpN=$G0A%?nDzqXNvW2T-wNEzsINz)S72^_P;ch?!}NuXT*$lXOIjGrv#DM-_!--U z@{vi!H*c*y-lvr>-wQ^w4hKk$6S5aD4?vlx`l=@J`*i+7?gav2CKLBzsjqSVlQ?<* zwzt5#M6?ah!9yW)5|c_*+0xZuSHhO>f(Dd%^x(A6l=qm zlg>1$@4dUkECld|Th42x&~muCrD#hK{?$l!{e$Fm?>?|^r{MwFw<1d3rw)QoWbYYs zGZlZ*Sw(`j)c@)oAqnrag6M*;D*p{8@A3ci1{2@(28&;sr&I{d{%UlIi0{lnf1B( z;ril93gnQHbpFT-DFr;}uf80>??3T%0)3Dp5VE$dxl-jCK_-`WP}v)ORTNdV9I0vJ z-iLE2+7W~gm7Hv9eCAh3n+s0i@o>IN%L~%|@Uk}Jl-3j_04cwg)hR5$fXLzZmm7|v z+!}JJ@|mj<^3=glEh!rT-O2vu`N%Nv{p(W6RF@WYmE2JxCZ3JAtnRj$w%yEa+tsgI zMes6zs7DFFdRFcFtM~qDoE)G#O$n8)L#z$&POZ|0nD|;I1m$bi4%gxYGpN{rB755^MVa_*>(_cKObmwue~Y znn-cX!xqUirO(IYS9Oase83NSN!`w=sp6%At6bRyj4vCfaD7OlNpwn%{UmRO%@+k0 zeM~J+R07+VF6n2IijSZ3i@Y@V9ejq z@7QmZIw@+@uaTw>t8yk*|27lMIo)VIN8|ah!7>=76ws$VI`<+-W08sW|9Sy-IFM2v zBv%LZcf*sEK!_pvz_3@txye#Ph?m;omjB^K^E^@)+!{OFuYCSw)eQZx^6GHjGvtkW zO#^8C2q$;j4Tdr@DB=SHQ}w6e7U{cS5p>eK;k*UFC;*!g1@BC1T*99$oep!HAIOaG zqSl|dulunF-yw@hVb@WDMEMqzy{Lhp0u)A#N-aX(q+Z zU0PoJ(9#r6`S`;hx+jG{^-}D_Avc*#;o4x2TCiS?HxU!PDkGF!r5YKJz}=Z`BWe7g zB7Az4WB3wXLdV7|F^SZgZ`PY>9(GXJFM2eA(>wuPlKGWS2em!IcLXr34KpLAc+`$^ zSTHvH{6i`B#8Zz$>D=$D=brgn42<-f;dh7*^EdyBb3raW?h3$d=NBv1O(3^^7IT> zr`}bXN0y*ekqQ{DZ?d7nv~1O9kL4ZK>=ja7Tgu~RLbhVYM1GfYl^C|UX z3x7*l{__$v9fu9)&iGr|pi9IIb4I(mt>4ffOmw-s;uYV#afMGi)TN7V&a2p1@(ezTI>uXiGlJ%^uraf%qn`S`{BHJZ{@@NlVA0CM{f_@ zV)%K+A%W*R!z8I{YFxrm)_LTLK}}GedkH1+_oXDd;mlE%8}e$QZgOPcn(e{m?Yl3FOD-IBg=w^UyZck6h5f zeD!0il)m!sEq$-(OWo~D1qWDZX~u%`#h>TnwNC#1MdSp}Wn|!62rP%C)aNwx1Ib%_o7*onN<{y}i2@|M8w@3{(U~QA z!DifSa@+rR?gWv)@6`~oV~PHJeA+xwh(%v{86#h=E!2eGC^7ZCKWd{ns6^7b9vB5i zAiAARJMrVuotfHs@9rzo*-_d88>2T0G%tw7H{sau%;0QCv3s@cJr)&x_gA&A-b?<} z|0`k{C8+VBipw-e{&-*2dCSoT+*8=r=Nt8s_{5c5QJB@WQ&KMoht(QjpE_*4Y9@h0 zF|R6<%C?nTMbfd+-nUwjVJ!0-=f37$x^G|an}c@7`zYJJK*Q=gYs*EE`Kk-lJh4yC z6kPa9+bpg3seSe2bNLI z^G(}RXW>kl-B)3%vgU-feKXCu2orkApuBhZb0cu=S*PZg(qw{KQz~X`zjiNf8z{U# z4zpP&PySGau2Dk;`sfr&RvkziUrJy7I=Hp@(Ve`V_I5R8R`*2M0(88`rqG=X24)MF zcup+x!T!jWi_3+bya_KSlndBH5M{Q|Ji!G7Z4y_S4s8#Qaq?vJuWshOF!!svi$lhb z^Cl(x>)wYhxd!Q*5|-!1wf?;)3B2FP1)2pn!s)VQZ`H{_d8>_{Mo(wRKi0V?HfKU_ zZ8Ak6j0?ivQDu2xEy7$=uP^fo>7dm%2<83PIacp{Jw8`Mt}b{h|7a7h4=N7JhKI3; z{THEF`K2n{a(ZSMhX_x&ZoM`?kfOr^t(>{sQhtbesP(0H27OE4IYJw%3H6TmxbUOI z$z`tLmR5-$p``)eU$v`}!kjX+qDL4&ESG+{9WKwDP}e=uz>CcaVGd+>Vo!JXDAUKj zL`n#U(;k~`X|<w5TI(WyhRo6kl0j$M4dwNoR_`ziT)!z^fwt}2 zx)HkhUg~{gniOwAtfj36N63|r;5psfs>oWM-r`U8HhqOIvN0#i_q~KmI{#oYmPOwqNxfXA`qx+3$-PVP(0S*N9HZ=2f~ZR7!%2kyM~Wcj$bJsmp6}6 zcm+nocDfLptvUW`)2xukLzUpYus`6C{HoI5g$Y%C{JEit#$%1Pj5AW+p#}LU_~y1% z(M3eYDQ1Hz!x^5AcTm!;Ix5E){F(C2MU0(%_|uLzw<+sACxoL#AIee`C{}SVTzrpZ zJuJ{axtdK(CLM=oeSOw%U@~-8g#Q2y?%zFeOA>^OandnRH{=;>wTV9T4bhh%T4l*c z>;s6k379JmKewrKe0CWP4q&RE*5kZ&$;?ZpR|E3GkgrEmd)PMRl~=*Ns?`O(hme%2 z#(BJrP0srk&>Pn@(mjtCr5|0dvMwf(>i8~!1|6ae&I^2Q21%5>Yl;&32i$qZ~Ap^vvkDF!k!tYd?v4ZL|j& zfxV3om--%Q&+g0902&FOa3FIpkAD`RZiO1xhwd^rr%o$ae-|OX@-c)0`>yS;$L)** zzrDr@kmAhWV{!xy&5loegnV(q-iWhBy>r*iv8#72U7=f2&MtukQ@Gvc7#0 z_U)kzbnv!`NVP4rB#8E#d;h0<-|XR+`LYvLnmRP$LluhfwTS7x`=l~#k7~NX)g)zg8tcE1-!XV;)3onJpKg`aDxJ8~k4FQg>P+T!pR4`Pu z>P(otjs~+QJVMhx??KoI^)DBaZ#a7eyc&bX!TGoxnc&&viz4bd&U*p3Y& zbV)d!2kIQ3b>kq{*{d1Vd|A3i+f>8aX_)#PWHrICySmB>N2QB68~kc4?NPAl$@lQn zw{r?tHn8dwtKhQK(yyK8Db|3Jshwe%?>*SWvbYRPu|9yWeJ&Edmkxh_de+o33z1Gl zC0a-a=IK?pRKQb9)@`9Dke-mn}|JJ80zxRdqZMcGsSS*;fg7g z!D_$lyUOqtuF#fWEf#RQJ@Fs*a%FN9Kjl(G#Qx#WmlT5!>Q9>y_+L0un9i;`x@{Ax zVO&zL9R<6a+D@xJ0NqKmzp7I7=MoIkP>%ccNyTj#OI_TMRUG>j(gv3-;U3m4)Y>XF#|1h zChMDFmDm>UIZspH>Vo2UdHRpqHt6OcE)hPr>Z(Lk5s{FvMemt<3(H8~R9=a4o`2s$ z;7*#QC$gW*Vlui^OC$O|<2}Le8=eNx;74m^5=0T7L5yNH7>RyGOH*-X{M25gs_v?~ z@u9+13yGulsZd69^<0W}+k#cCHr$;XdJ9Fc8sL`*m1p!)G5PP%b{f-m+B=Ug$f8 z9iGgq|G%29JRa)q`+pEEQay_4p)6&2ELqB$FoiPF*s~2rB0JC6VvO)aMVpy0cF7V# zQ8C6E(%8nDFqSeylZ+yS)bGyB_cwpe>wfM%_uO;NJ@?-8KIbUtj=9c^)2Vb~TxNyY zzT;0Z86KnQJx;~RzYjQg-n#lM{#&{Mi*JaZFU?;@FAuDyD%xYt(oF~du{J4G1>JGLcuD~V`4S7i z6>+QbKpKF{#rJCV(;K}J4Fc6sEcwGekhXnB>vZ?)Zp#(E&oIKo`*3jm@%eXSeL&~s z)E6jztolX`Q<{G4CIOlOVSRAZk+{$sU#N1Dw)Q(apOhLVUmxO$A0NV5I^w2=Tb7;H zs|LP2xvL|IP66BQyvp~EaSWo9K$SGfzEHfeB@}qXyzf-$v#$%k>&iA8?ktU@e)(Cp zc^Fae(~tRDe*noFyA#ChU9wD&7|F}rbi(W$3yav%oVnRR{N)Jl5R3_8=nUrQLhcl9 zkySb+xDQC2l#<+yv{RDsJ-|Ch1y7Vy7C#^>w=Aub#Lf31=N)s)3$f|2 zV>zr8>3`)GKGj6tNnJ_TCe`A1&#^vt@tXwqRE)D6-Rc@oiy%7y8bZqt{c02_%3K#W zo;{Q)X?oDSa^7^Xgg~|VJ+_fZpvvTAP>Uxs!&7)$gqJ!%%pM4%x5hdhQ9~tLRxgU4 zh%j*YhUm3BVp!@^6;zpXZn-19*?&GSOyRn7{iBn0Hdv)`>Hg?&xl{LQ77eg zQ}Sk?);zVBdxJ1LPFi<_c3%hZG0-?RZEWZhJWiH2X{GYhK!mVBk-H)K*E|{=Qu3qg z=>~SM+`RNNM8%TUouqbi0%hTiNZD0US{i@$&QFS>(RoRfRnHglXu54>eUR0E%3yPV zcfa7&qg4r`39jME8SF>i}d4~ zEAifna_UOyg7WkHW2v1OyE-7kP8`St^t7|Iu#LaglV^TFjx@+B(^Dm)hD(o5rgdxf zI*B4#AkZ4#4GadD)WpQlrzF#~;Gn?M;Nr5Q?oJ?}vkcL5O8BEp?0q)26M|l+`^Cr} zF{@kn_pwGfGMRr`-uBHgI~`qDj3f|>3Jx#`^1mik`)A^eW1Y{-!6%dpqLC6=iZC!` zgz_mo_C89LL~Ww2B+m=S76RkK9+(1){*&tPW-^S;cV?m;xIl|i7L<L!;I;dqX5g!%OWGHG@_Yu{*F(vgw=3${?c61Hh;3i=~9Qiw1(KQ(!%Hev8 z*Cr;XZl3b8?)&cgqPzNWH$5?i&1b$Sf2vOjEoAiOl?5;V(|*$_pSXDjCej19eK6I8 z6UVaajKPysG;ah^?46D{6krqaxzvr{Pg+(?2Yyf#4OL+UN5NioFih{=YU;x8Ym^Yd z+uSNHXdGYfep$yi_Z&OGREiiuW7cyj6ncE8r>oMWmz6XazT0z>%CmzF@dV&PRkv^3 zHZN{YTu`wW8ckq0C+F_}y8H7Iz(k74q`T=LM%H{m<-sG+v8ewEUVFB0Ok0-rG5Gfs zzg5hz7Jzm(N@9fXl&#s(!&WO%D&P7T0MR{Pg!-AopYa#ue_NlU4L7iuULCKtSZ7lH z9bd{wF~}J)d~fEejL(fZtZIAp4gYy~V0o;`$HL*9(ZjLETv7ZMQM@7DVhKvse6dEK ztn$Sd(LTI44Q!t9+>((XL3|jP#gP}X{+7nc`azTxgU%BZi>cw@xKUV1Sqsm#E%ju!-5<6V=3G4HcVsUa_5%4iN| z5h7MqZEU?1Vu)RuaaCdJtjUPoPeQb-0Si@?PMd#4^E=$f7@kkK-n`R%fHPI%lJ($h z#*)jOBHW63o-OM3o@bG3GTYy?mkjslr|h_jfkAt|qsqE-Y1EJ#6x{Vi-d*=O%X8fu z=;Qda2PX0pk!6HE@%I=P_5BNfq|B+Vu&S8^+xUr10ICXgygV5#%QUs!&AaPn5lHk4 zwHUSOPGe<3f=O^+2+(5Qr_(~J=6k=8(L&mos)1zLb=^nh&#tosGlzP(TEEmaAH8I&9MHb@l=ve=eM5@) zl3=!JoU9kVPG{>q5An(awqc%el^cB=E+!OYnch^W>ebY_BB*}!9sfhJ8EeHjbWs4! zEqzvJ|HVB7AM`G#jxVfkpzXkI0oB`-5|8y$q_?_|rBWIOfaoA(@vX=7gk7>$Rgv8! zAB}ioAiXR+T~j z7zCC4=>;!893U9fA663e;qKML?h=%F|MIDCcMdqz2&&=t4k6`iv)-*?WUAP%@o z1-_(G%!#0+kY|ArUj?3X@()0vBmTQ0rE8~U&_KC^OOlV?zjoB(_g9k~{xpS`4GPF~ zy>Pw{+zfa1{|u2oc2?9|6{VVi>nAv>#*VOknBe_X8=xPH&SO5(3;vXpd1$+o^KdJ1 z#;S<*MdOOs?9Jxtt4xieJ9&(?R>xKwzk0f`QZEmFR;6U&|iO@_x=V-(N0E z?zajD+v0n7nEP`o-3Uh5A5RJ0%gFqsJRY%JL^JsdB62_+kPD$V8YgGJJtP7vSortg zp^&E0{bz0!Q{*IFqV!*%Re_ZOFC#`7);@G-XA2*91bkd!TWJ;9`h)`Fft;ixJ+g%~ z&?a>&QQFjQ!L;nj~a~QJb|^{sP`Med#8;tXg9S?$L`TO@=UgnuNxJRa%mNO ze%K#tCxkY@`t7PyIK2&tR+%fhsT2PNjXybjkjFP%1~#~;Qn#HTp(SnRq5x~E zb`f|kz$uy6FH$q)D$F}5n{s!Ms9-n8(}2TzoZsfun+ICd)Drd=mJ&BI(ZT_t za!e1B@2VpiZ!pxh&OF4xh~!5OJO2MS6Gz|KF>F=nyp6++X?`zbU-q(2MIQDPjcrUp zd22h%q9RMzSbDqBTrqp;qO}qxUM73(#CV%fuYJ#k>u8x=$UsM(QXoAopxGf93!YR) zUHRbGI0h?!vAUF6rO|v4lT_B&qhkD&P%W-~L@<$}m7%MoEg#aveP z7Apn5^|e>Kv-Xcy6YGlFoj|8oFSklj!PF)K+>A>~#Tq^B_T^>M73i`8Xk^Woo%P`$ z{P^Bi(LNChFf_0C-<&=4<%qxRWeE@A*jv_>1rg&?zcdOCcWqDP8G`~vO7yhw=8TXD z_Gac<2w&h1ULjzU=SE2|gxiRma*OcPjZlG|XINc}E3Qi+KmFm~x=3ZXDk(1tII)w{ zLQsIFi+V<9X1myZ`ds93W&o{JD*npc!r7a;&?S)6*+c)UgDR}wZrToQU96JH5y1+2m^{cbFq<31yvCVE%#90>Yl12M_%bARP47d z{k9{XYih-XiVZA^&N1$w)WW3Gw!`eySHkwV3m!Q4xZ~cxYx1?)jTk(L$LAhu0z!3G zBNbb(s(|}*V05l1dNoNH6Z!gW?L4wAemsxD*bka%)~F-H1{6}KP2iSa+4cTnU;q6% z<_5*jDl9Sd$12N!U=NPZbIfj+9^LGai3%FOL_A!Ihf94x@qeaB^Ax=G66=(%YADZ) zs}diAs?i*^Y(rZAQ*39!OXPUcv9FmHI9KCLzUQU?<7%l6fqXbjbN+QFhDQ#o$1B86 zWdp{}s3aaS$x0xwjUW`=b9rOm74u`U<|-Op1%12$mR*g=+}mFeQo3A$EPsKjx#?m@2iOmtJ6zvRrt7KSZ=}i&+Vn*0-&Wh{XQw&g zze{kKab6|*m+4B@>>1crHtzt}g@D%_imJWSDb=T8E+p#$ZUEOVl1BUGGn*T>fd3SD zAQ%bBaSnt1T6Ug$&3-(rzVc;7i*+ia+petRng97(RF_?KbAKCa5@fm|U?*3FB=ioB zWVE2TY|2vn{FF>OtAbSi3z@!_vE!UrSHgp;Ia>HXOu@;ZPODqT2NEP4z_4shurdFA z#ujHsAfeI&zy}xAS5~Fo9Ep7m4jPIe_d-YZ#Dg$N%Rp?(g7=q@`hLDs|08TD)vHAF zZhCi4;pIflyStxRcZG#jf!H%X?^o89dtw~n8zfpnynkCKai1LX-2NW!W;*ex-mPFA zHDOOagb_OU7N7f$KNY@fNE(|X(gID{eVL5KOi4VdVkn-V(*IzKRGC;Cx1q1QiQhjj zPSn1b2LwaI0TwEjv~<5vwY$21qm6cAMx0VDjGyz|d@c{O&Gr4hyZ}Jyv);>ANP{)H zD>;Kzz8EC$;gXbS(W1wt{W5To;}cLLU6xC^XxH24wY;0G*Qw6AHSp_vq|wQg!bzdd z{Ba2f4*VW+0n&pzCR?jPmW-BbiNZ`8Mnv}U{(JP2fxxog)l=_rMKZCv+>S*q*hrEK z;`x1a@IRQtqmmRmE8g4Xsrb)(*h^cJarJ%|MW zb1$w780$rT6sotnYW3{S@Xhm55QN|e1oSYTsTOp=PuRJh7|phqGB3ELtqalhdg~Q~ zZhaul0k`n($n^-zWc4z*`&TjWZ5T6&q_^fTv(*676 zbp>kBpGv5N*OdR^9`l`#`O@~8@~0^6I(mIFZYa+BtR?^Lea8HhwnpPmtr!i71(pkL zdT-6OTO(g14Oa3R;gWmhc!(nZ0iQY6AR4iZ@>`%mp;WeX=1W==pq@kG{m6k7>Dcef$+U;6SwP3d-lpr@t=3j{KEWGfHvpBYv& zO}y$@1$|3wmjUYPrp25+keww{ix9oH1)&ST@Syt|avyH`S@O2)&f@t?WC&gTjO}^< zV5(C*D{hpo+f-A+Cx5wgpNEKYcQ~Y8+CGq78nZ4Odq<5!zz~sM^xlZ9)#RH(*QL13 z8bHK+1rJ=7 zy(g%tdf+FVBo}V7q$wp%h9!lXJZM$UTp}a$PToCZwtNkrb-W=Yhu-iWQ29%hL))Au zVeza^vmD7_c?&ij%?n*&+jzAL`w^F4Nj&)0WOC7ZnPs}fhZf+lwr9u=-|V-oj=u|; zbEe;NM!$Hk41vD>=&USc&RMIL27Nh_x#MCJazj8C(vas67o?4FwN51LzlY_C{d1Uo z6Vo$s+EXV3EGTQ@6))t>VcZJflyk)&(~}wtJ0QhrHg9x}dR+89iC}%vhcu+wT?L;` zl*#kkR=^V=jjL>8gPjr^jv`g|{ahr{^EP)v(;bRH5n6o2tdUFuvt|26x24FKo^XH# znA#@3hIB+Y{4DV?$K+g1r5oW(WZbItTd|`LYmF!m3}0(P$T;@GLPlK|*CQKmAP_e9 z6%@L{wx`PswyqDeAfhartB|Wp(O!uvyt0h|-lHZEB5HE}g=Rkg=8rQw)Rv;D;{jSh N2ot1n*aCh+;cwXoX>fm&vrlOP8Xw8 zUe2k2n{@ed|CW!MG~|>)Wid~vNza~a9nN4rjKRvdpv=rRNQ`uh!fA%U6X32C{4?l# zBl&w}qQYMel>e_U+d%tpx{Q-2WoDbj3Obgg621FYR|6V~#I>4$TShE^C(3w*Hr6bK zvn8NwG8U+=Zh}XB`61+BEDaOR;YZJVz!XI(eYWx$1GHmMF{7f)XB697%FRWsb zE1()4O#n0jpj`t)2}U0MmzW}t{SGexniPLLkZaF?x@?oWvH;)M-*ao-enDoQd}mgec_w;D<0cW@<% zMZv0M%6+&WOOi~-YAFzD@?@Te<=R*vhDLpo7#X9D7>P3sF3V8hOJO|yQxK;oyGZKF zE{dgB_FNcw=^%Roc2-`HgrH-*8i*gH9O9O9xmJ#_5`rpu{BkIN(UW~u-OCtIxEJ#g z{_MX*qT7$`mCyEhdqJ+uWK?y$ATut<+9kB)p$%kS zWt#9Y!D=onD2(f}qD!N?x=|1(0zsyl10k1MoiF>NX6I3^Hp+MO@_qjp3DXyR(N zD8%NYsL2v`tIYvaj0R+=1r@7MRLT~b(1cA9O!*X|9HtPZRZ?A2hoXE_nAD>z2rYcX zD5#(R$k2N_WzZg|Dc=};OsJU6;-k{AJnZ4BNmOAi6r+B&2UUTa0h3IO+GOH@O)d_E z6=IW1ElCFy;y^$vMPnLCI;JTQl!{TE6iw&EZs_^a})8Qc~@3kTBtnP)Tpp zE%1lqhA@R;(YPiOj|;L49cQK37yuf82`Z80Atpsok&Pc0%AKyu{}c$8l;yv&3@HSS zYNBjZ7m4^;T))Rp`{m9MSCh=x|L9MF;Q1<|IKwjGL{^Ov9*0Ln(I$r=V~uOQ7M<27 z3!!0GM(0N3CR*U~VEQQ*KibVymJiKGz)cS&}M+&1=<*FdjDnWG3zirLYK0Ft#M@NGQ!_t)2z*;{}2VA1Mgh{4$8tNJBQU zopz=S+?c>4^$P5Ity(Oz6HtI>ht&$TEamnXcwsj;sY+SR28k0(I`pQTh9y8X3W_ua z3j~e075I`$5$#qOWd^=9o=p04ilk3swP<;EKIJnY$p}A`a6uwtRI8S1d+SLD(DRrffC^ zX)h2&G_aZw8Q5BufD`9K0@!1BL@<}t7Si!G(NIQdi77Z?*=?ss8GIpaAXcDD~v803& z42nRE+GDi}BXSpoagr`=3L>qXcmfRs!Y(FD7Z=5iGkQ`Z5bMnepE3y;fO80jPHH4P zJ|jWstY4tAI}9w)@Do@S}YYra+cyw3DSf@VUNdo zny5nJkK}Z?9)no9l*JuZ29?43h~)-p*Wp}Lqan*E{jTB;dE3Qv?y&Ji6fJg1;cKKoA1&F zOk<7f1)iWAl?q{fAb3G276|5dtJz0kP9-f#NM(qV9}KI-o&avhC|E2>IzmLaA+9)# zJ4GsP%!C31$Lj(@6pG2|06)m1_*R}eN3o)~(@-Gj74S4BjVX(uLe%3mLvg=SjV3Hn zQHU~z)hTTPO2oNN4kUs3Udk@zILsNP$(uqY5lu>+u={-s#d0JJQQRt07YIT^GbbT3 za(I@I+i7(sd`Yp?%kxpnbQt4H@Q9FbJA4)nZi2l`RwK7ddAu;4)B7DbWfg?PF)k$t z(l$ty(kcW6f)2kRD)l%Zvqfe~$}u}-^F{m|iCiG`Vkn<-W#Xx@(`kp;HoYMQd|+0B zF|gu-pdTo}ct*m)p@c`HhcKt`yny9K5kWko_L1?lM(>EKLuysn;9^_ZF0LtpN~Bqh z&mu6x0Zz)s)+X>oT$_YKNhrzpa9mcc)QFk6OgiS^^8;#SVT3Av6l`3(Jx-?;LRBEc zhrQ;gHewRPVjUa`@?A(E>(8oTLl}vtfwwFX>N#$noh@=he4{NT#%PQyO36Km42x4B zC@?WBO_b#zeF7K^KYG!6i5cx4+`(t0N|`2=6;qC=6^27tO5sk&yj-?5Cv!U-Ml1;A zj)flxI`ne5IKmtmPZ&aC4D3~z7vjc*hHQ>N#aXq01#@vFm`pevO|mWNRL^3hQ-jQG&%~J6P@z?Ia@s z0m&jnUPx#Y=y;qWK{%MiHA*5QQkaY;Wx^lK*p=K=L{Ae?ik3RnbVR_2Oe7;DEU=WY z_}sMF?czt&B%QOFc`^dQO@aW2OA&E>qA+DP0H4X3;ACY<7wv~sv<-96YPHy#Hf6L5 z8F1l4&Ttmts;qH;kS~;JY=VSeuZUPN+@~gkm{q{R99gSE7lVs#g$o2)3>F!OLW~MX z7Uhy|2Tu@}U}_gEamtl(4Ud%*<{+NOl@o{3SxX#7vT27QnpT<}d`hbKL^uvx3?V3# zDXy}BaLU|rrN~Zak|rw6XG>GIfKndfd(?1N9~GvOs5uPK=~PKl8N?EHMvQi?((3T!;NSOpQ3fa3d;EOP*l8b}Q1XY66GFXzc7vt|pW z5(FJ?yGs^y#_eiUsJD9Lc2htmli9^wl|jR(2o+H@IEkRmhH7GR6xD;62tnhQxl3_5 z?9*atJPyWZlLSPFSgY_RWe(ELgW~}v#^+&Tt}N&_p=Kr{i59gUs^X@-)=-XP3PCP0 zoJ?soLJlP)BjJ!P;q`b?g0b?&JeX$D20w@GO2HwGH>ynYoEA2e^an%~Ux1?-L`))h zVamW}mPjCio$x4aDxr|nszr8bjulPYaZ@O7Gw2}Nn$D&yz5wtuL&i`lXhe9TxR=Lx zlv*^6=!gv4z;;CuUQr%$F^}BslyiN$h=$8XNx#iP!3jBH^czBap4FeFq(OB|s0$%M z7VdH8{FEqzb479B9Hz7awZ$CBsQJOT&~H>113_FM+&MNcVpT~PUQWy|HHd;<3Kq*1 zNP;lOG)Y(zmzc!1NPuVHYGN>3%S}*XwSg07QyFbh>xCnHtr*3Py0|pU=NKKN&Q77a z5M>Qh&alD4k!u2Z*UsdMV*v(FAO9Tp{7cV3X8>Ia06*%fU{L)+%y^QyyI4PzVT{iO!kANuJCvvtvJn84N!~^OMK3>FlA!X4%&|-m=rrLR!cz`$*weD;9BYgPPtIY4&@Rl zk-}>fS%L7SQe0zk>!l3-FxzI-DI=I58PbBNsv=?Ym^qlq?-Iuqe2ULfc-SV)gQN(B zfu%_W1SyAz$z-#LHK^jJAf3^lb$eBbjJ`lH8x@k23}|O>MrSwj5N{4wXQW&jwaPGq z$d6@sY6IlQYye*NXvCp3@%7q-mZeWwEZiuDJK$)@N5|z#hebyg2)cM`hU?3yFqtff zE7NR`BmxB?Ba&bnAw3sWVCj?!c0>t|%qkBm9E{Rz$AZSVR;@GgQc8YW5z{Bta-BQR7R9@Yg{UXoe>GS$#gIpOn7jGk8O{J2(#L)2_sRvNr6#3y%ks$91ZKTy$OOE z)}$8K@IxYl&F(0SkOB4)lu?;ADizPJ0 zkrX0wAbPL?BY~hZBv*t@W-JUTQD4qh+$ku+i?SdPY_f=9l`<>DOwI`7U?_tGlBY8S z2|}hC17#1|@Jw19hr(v93dPNCwwh&EhLaX9$#5CB(GuaaV#QTKQlHW)Nu`Oc)l*tbV3t8T9TJUa#XQVJ#>F@-69t1Pl?3LsLS;v7%IpPc zql3W;O2bf&&CLltA`jmog7p-WCF!`H_8C%o5aELzmVi;EO1p?4mq!t_N~A+_m{IAa z^}e){)WjujTQI~g@Gz4In9z(OnN4ItSVtPO#CS%PB+QLOBhs7)b|pOqOrhb*AY0bX z@PmqkCvEYA1v973HhqTFJ75oFNGs(Q4dN{jlo;F|byf`P&@3l@S%f2R%)5P56bM3G zuY<%sG$DjlBCW+>k-Mx?S%z;R2|i3(xNMyWk?W)(aS_-Q4dZN`KNqFJkgR$G$7=J_ z@~GOINg&QxT5XGAGzf2C3~&_1QUZ&LB?f~T6&T&R0Fp(5Y@Ib{rVU{n6Gn<7%#aR) z!D3Kz#byUf8?=gI%5>HwPl_N%BpA-1StXv%r1XYFLXEoZ#&|+#L?pt9iHrzwb=E0S zYK(4{EX$AcO0CF=^Dsd`M&#^#R>Y?D21LH3(J8lM%A6MV zMp8BqII|~UE!Qq5^Dz-sgmf!VO)@S{nr(ps4~ZbK6*(!rkql}B+?*ig)_Nr_ZYHSD z$&@m_+$zt&c9;zVvQA~r4m)K6F3bT2s*fKsgq@s-HsT5@1(~n}DTY}PqlIN8A-+(g zv)hE$90{9JYCeraoSaoi@T2Uc%SFe zm_Q2RYg)Nlu0m{ag#oAyr#WECCZwi7KqS%G9aqM(3nRRswplR-aF*o}IT1cjF9LC5 ztDUCQG?yJ>Kx71j3V3E09Z*1~Y&4*b1mwwpCd1e@YAz*G#LVeO!)+LSUVO$|{6Oxkw+>xX3`nDAUU`7%8jK)l2wnl1+#e_1hgL3*9ALbLN z87&b8fi$Vql|~aGb;bjfbBIMoL;<97x8#ivRKe26uY=?wurEZ zixEz{1S8=vBa#xh5OO*_R=eIp%0wi9pT}uWX1Pc%YauK;i`SL0Sox_$FvxI}TDe2R zOGP5Aq>!Z%;R=9Jxw`m}%VN-5;|H{$jLJ(SI5 z1nzf^Cu0c^LSc)D!j7QO#v)Cb2xfMxDYe{WE=n6AFo9BLlhqLr1&Mg>D*Loxu5z6| zj7`fKzf)qy-9n#>=T5s|k3hwWI&6&D!ijOS@$SL-Bv{sck_~ggg;{7&>G-S z1q4Bv&23kQ;!=^7je*C6;9&v=M1<2%@p3XIZZ}2GMDlmJIda~T5?P(}nCi#jVYii|-naIj4FAZCwKTrtfxXlNUf6>!5ksSyqb z9BL9%S(pNB{9K{Z9l#w3s+#jmWtgC}xOq6L!29X*WkqnYbI3hgC|wU2l?Fy(lF2+AqkA%R|e~45#3A zTo%jFfUT%BJ2@CoLLoz8gj6&Ho8r+pnaRrSe3Z`85Li;+^NN&EjsyI6lFOH=CGh}@ z%4v+;h%fGUu(B+%(d5(F?baA8ECWo*0(I!J7YK?a7KbexAmgMZEr%R2tHNy4T4^HA zm6{p3*&(uKg}xMc$SAWK-8nH=5YN{mL;)<|iZhHz1(8V?J1qjiz@knuXWZ7LSdkT( z#6F)%5SDT?b{{LiXu|^5@d?v zWIV=V+$JmI$w)>mQ<#Gm$_>MsK!lXYl#nZ)0gAK4X)1zJi{p-zKJH}F2+(;65bd`l zQ*mRiIOU*9s5gflY_mlp^O-p`5}+IzmNlh!NferN60$;+R1qZIxK!iR*{ptHSejL8 zL#YfqVYfj@oaJ$&34|NvF}C8?!y=)q1Skx(%EqRYj7=DJ#q6{v%YmU_0ON}!2%JvZ zbh@ZID@agMn+*#>LXXS`^qgO=wHZliju53hTsH`Q7Z!zshRrTGs4^uiHondw!(*O+ zT^yIwh?>g-;xS}4X1Ph)!XlDdwp|H)4V%rv!;zpXnNkF`3OFoQriGyph~pPwb8RCo zZ-0UK8T&HRJC0Bg;V>xe9)rUUREPo71N+@>5k0&YDUxr`*z=K`s7dQ}%F;49%c%*NRbiJmLMprpOI89=R*W$d zv_kJeGHgnrgh7#s091*FI&()xq! zlo{lt&{l?sTiGHPk6qNuEFYVvLF_)@`hzkA^+)}pY)bC*=c}Zp;46Z9>5QGnb^x1O z%7c0Ai(lCnD=ArrU!sS~!Pm6-in~REdQg@Y zUmO%_yx47F8W&sq3o|GXylfh5wkTY6DNCuCf>fHMRFj0beA3!AQ@iqMaXMY4Dw<%h zs25jBtCLB>S1CdegwMx4 zLSl(THATl_Aj>VE8(Cyq$^EO^OL3Y=;XXMXixhH>|2#piXCW)MlDM=UdA<>VZyu0+ z!R2uI3nDW_}#oNA1EW5BOHKBf4^J`I$ z{o4yu9NjhAU#lDXch{y!rsV!55*HTlvc;^Kk?# z$1pxr&ck_L1oA-;&kuI&Rn33Dk8}`^=5u5PWXdDF(8kpZnQ3a;}u1rxaQoge3?|0!1g0y1XSIxdKy9))SO|C63uaDLH*M#D~ zv3eyL&fNd_>Rpv!{9j$Y;+kH$B`@sntG95^%T5neS{RIGk`y?^<0n8myO9Pd(mmOK zP~#Gw@-hOPFsM{w2|JM>JS1*N;WU+}P%X#E1YQ`z@74GtI#wEsCV-Z}qu%WAlDeYO z)ztKbqeImoVgCFu&lg->Q-@{pbuB~={})9UP$1ykLuDG|Y=hKtF2qA%VSe8g3orGM z-&MuZAV}1PpbGF0d@fWCUMWG(S4~C=(V#sz08o+khY5yTaegQj zd~Z^6VFp(Y@Aszx`lDhA0vrZ8zio>Pcd7XIo8|{82ZG}=0F-G9<6@YC9d>bfFH|a} z^`F*97dpIbL>C|l&R6`OLGp^e|F)1!;1R+TBYoFErTPN%|Bp~ zcCBLRl>@#w`D;${0tYN0ZOt8hW3oN|Oppf6yW#)XO<$nVWSFJ~l1;bSo zYKVwhfCtxk^Wc925+w`r49W>Xi6AAwtRXH+Mt@zx0x3w0|0!4?CF@dSScv}quz>Th z$g7k8A#nl!Ra{(Uwf+s|1rPq8QzN{R?OtPf@i$i^dCT*n-C2u_i;`unp>gq7g+-}x z@;8$e+`P?ME0qYhWM%)$xcIv(5nx%=Qec$Y2mb)$KN`UhUeF`8&g>Na-)fS-yTkxN zi+pJ7qC--fLR+P#UkznOX=JI!rsrS9M(F^qyBG!tYw4#NgE;`?J&ntRXgd zC3Z@UvGEtf7^VGRchI<~P{2j43S;~q>yy&%|7M;Eh!J=S(7UJQx3>JUU_AGUG|AEEj zNqdEdBjBlOUR=~NHFA|tP(yi9YB$yx7$x%NZzeDD;fo8Y7~DKGxNt2>#VF0DuOT+B z!d8vJ!ToFKC~baydL?;`)IKw#WM%&A(D-}X8HK=jtprA?rSK0h{-c3$@W}mwJ^`1t zE48om+4+AR8l?lMp|MfwGhcV=1jNNJfCC7SS{3FhMoqhS(@kN7o%R^2jLo zC*WCptpY{~V{3?uzwDlrCII}+U;%s*9q>tj{mHLoxG1&lYY2?e<~4Rr_$9#nn*jsB zL01K0;vk@r2SzROT1ywEhQKI|Yu6YUe^HT?_*K`PA}M$#;E$_HkOxL>if5FrObwAy z+Pucd5d1ZfQ7W>oKWOBY2|xvaM&32y*P_SoaLKl-Av8)?w&u_%>A!~Bq*UfzchD%< z89=9;heju==D&@M5|MM=0Rv>~=<+6ptE!NgQJd1aN^@Rnh>X9iPX1zQ zMybbr{XyfRIssW4wad~d4K&pd8h=@xlqQGNP@R;@yz34c1@8nH8RyS8^Zs|-6G4fg zP-A^knrK)J_rY>t(c;oGbWi2$SUGKn^fQ?k#Re1{@sF&K4{8R$V z|MH$H>Ai+q>MA|{*ESW{K>yALxU}y|-%52QEJXjH;v%cAy7&*i*bWg@h=7#7xCyUo zP$2(+;wmg$Ecg$CLYM~yg47OB)FykqM1t2U{UInw3SsR4h2S4aaZxZ_b@3mR6mS6) zmu$G}T7A@}s84N{6s1v#>lzfb>49`@1_iG+8Je}}B>}Y=6x`ZmXmbCtDXualE*AU; zZ3<3pGBj&bZbEIA6t&9E{3oD!Q7)8T{0HR(|34X|xQ?F@h{11{2lGEa`#<=U0C<)c zAKL%VsNbt1^)-I_b`^yhJ}W5E6xXE#2T}tsq;vewAfq(9?;l`X@hrah2&wQQKya+> zf9OP8Y1(FuaZ!3C?r$b9N?#CJiv-4Bd`Hh;M#kS=mEBZ@4)ED#MWk#vIj3hiXMrX@7`2_Bs>?>-WPXa4;C&d$9nxGgW z7A^c>d$H6kl~*?eeOCg{FoJ_$+`dNQSg8a4*T4VMNsV~;54-+hSB*GOqssrF>mPR2 zhyyjM{13Wn7rPqv?3JM~Zy0CF@TjkL%bS!O%Ds$zR1hp(eaCs4V|~&z@{a zEE#3WIBX5!OZ4V&!3}d?c0Qe)Z3Mp_o2V|d&etv6JKqSU8F4Jr8{t(5g+f5A0xILe zTy6!o3|M{OTVJ-A#BpD~PX69C?n`1(29Gj{GEROhX10WmQGZZfizOIc%t!bM+}BN7 z$%Qxw#094tKr$MSUk(YoLQvOx**e@uV1Gcxh|!?WvSN=$0{7yos6Pgnpd!g6;1I^d z=rRsJKcMPrwi!>vlC&33fC{nEWuo&}d43iNfbm#^ z$%vtt$*7^{`||2t*f41HRo9D!$)SX{pbzz?k8S*`sLHv?b~+$_~TFO98>#$ zJm%{`yLl@&B0s%axn#u;GndAoPHzugv2o`1do#*^IN^r`L~bV zFRoWMn0ET1@dqFIiXL5k;=s+{-}1};ZX<8}Mkh&)!9JhA;b5?8!$y~U;HO#boVuZ4 zyZsSky@M^~x6WG8&lCKm{i*N2j_=VbSlu+~XV$4#j?ow&CJ(;0V%57?&+nfsZ#(g?;=}Pn_S~bP z`0uWoH>26qtzRI6H%#6$=8R)LZ$z8j&UuYLdhcYbpRo~mynOBHbIYKgMmBj;c5Eqk zvSiLl>OkYgUn%b#GxzZHDJ!hL%#7YA9_`mry?s3P%O_O7>9?J@ak^Kzq^*MQ-k3S1 zU;2sko%StLFEly0)maf+@Zj=k>w3J@qcXIQ*TT59cZUw==GO1kf5u|2_8qe&_;T}+ z?lYlBzI%pqz-4*x?pu1?F=E%@_oW>d-nK9`VJo#<_VC6Xue7=4E$P~EEq|(PU8l|J zH+7@#RI9@2_jVl|)@0bJ4~{F=xNd&-7n`m9nTGdtZOMJkJ*d$mEf!Dz*7+Nix`=M6vPTi8W9(K3;ydo(4e*LHu4^(T)`x-uf##|>Yef76T-n>)8=ez*<1w^0vxeY)|UE}J_wml>YzYwsxS5D22D%kgGyoeMX+x9vW7YAOBm;6d@T z?^IGhi@$n$>>WQmdg64Tapj|9c`fNq-`sz^s_gA=CeEwZZ8hy;ed})1uifX<-rU%( z(jd~$9^*utFYW2#Lara}FMIcFYoWY*ONV{C>$jj*N_%|q-fNxbRBBZHxAYuxTh**H zzpb1#Yu-2?z3ubeJ7J`W@utV_=FNjoOdi;({TIXbq&Ci9`u4nmx17rkEMM4S>xY;ESdZTja&^WQ9Cir|iR8(C99c`l@Ws+89(vB) zHPbnb?%x*+ESlJ3h-UC@8*N`Zd(?Sy;@OvPSiXAK+|z9)zi<#+`vknkaF^-V{q#f8 zb!%F#*4S@vw%9tt_vHq($I6E?V%tz%)1wMm?>;S6_osVqy5rzY$D6EX-^b-PUX0q3 zsQ>1t>fZZasGHJPKF%FQNBLXK*%Z8U^pqLmbq_WemA>WXT#sAOMr-fe{KHRgr|+Q1M-CSmSctW36Z%y1)hyPiW(s_0?j`MFwOfmW1@nY}7Gf&bD z1{|MXLB0RXZS!8~s0oj@x*p~8ZL4%9!y9#W+$|hB;Moy->W#m>-fZP_!%j@Tas9M< zjhlXhqhCBXzQL=a)pvHQcU0MYlzDN(JEpfT$3m7PT{E1go6PIf@{xuuMt_{Ntc{LQ zruu!{_vzEUnm)NVslvNJoi~qrrs3#UkWi!KINPwd-nhfs`R<=uz>8Y$J-K4t+E1p8 zYuC6l@A>T$Ti*Xn#o8Ok*cX_pMnAg$l@4D(k8}2gwzoN=cw&vZ#g=!U*5;;u$e+2= znXW3kciNmCCtl6=y_*$&Nv_R=ws4-=#clTT+MCyo?$+~L$s^FjH|u=Zw#~t1C%fNu z@OH)9BOAqfW#Vd?iX)=znF%7PB}KQMUy zYfbpRJvnrvzxmBGM$aW%?f(G@uRk!)($tm=WuHnv4sgpaZ-qiN9-jwom zd~iPYSHJ0gWKBNws*sq@(xs;*PC#$8LHhw9+)C+cKDrz=||IR z>uea#)PLr{sGaD$cgiU4!ax6cJsIexUneO@kv9kJX(LAJMV7WcH0O z+%;oacmBKlWpi5i?^-GzH}_p+&CbsE4Uc{h9w3{vPsO@pRVzzphWfKJziI|O+v)v@x0|fr+78@wr1!>mDx2*dcmL|q3l%rwq>-dJ;b3#Nl{`hv%^chM+PvGh4}K2bIBCW3+ori?i(5_VSl6+6%{RYxo5&F^ zeXo7P=)yQKhLrNh*ehS!>B+IMc*<|*syoTJ^E_d`cVuK8}z zk8`$qJFnR?y8cJEyjbzcqDKZDf#aK2SRnn&_jVr8dCjbjyCO>=k)?Wnx$Nc*)=n)N z1ok@CO~Wb$Gv68f>J5Q|b8eo|EBWl<7aBH^+SYXJ^~#zus#}N0XW!SL(`&PatE5$P zPg6T2?dIk>rzHd1qZ7^?96x0K%pHq&J^Oa`8*Ms87k>WXi5`zHJiD~nnkJvSHBIC@ z*ja0%J_qLBPW6@9nblnSy|%}LT@R59Iy~@}@E+5R$Dg%7)}6gcvi|VWk)D0;KogTIf(_;M_aZZ+_&oG!SGA#cZ`4kgO?h8(X@Zt?xxr~3!6W5 za@nc2!ecXe&$sj+>h}4m;|;&PXTlp!b#_&k?sF^;u-2n(WJ9HWM)I1k6?`>#(=DGJ z#(!?`(!mzyj(4o_9=qq@khxvZ@ju;{JZOv$zsvJg$GbL7&2;(9emHBKXPn$ly7bde zJKZ__oo+jb?C}2MkK6;3nPwfTUmdymHB#JV*~7CQYkJqIz2#fa*4sROgZrkTuiZOw zaGylmN4M!bfg6t)eIJcb2BVeDgf};-`K%et7F`Yxdsu@S_cUJ&BmxaM#V-7Yv^C#o?o&&d&_qI&A~|277vW!;hKF@rAwTkB|k< zg+BrQ&o_Xy@>C!8^Do@mdDwSD zTePwseN_|_y?$irpk60`{NBAx{30hb^T32i?z_o5&oQ}S?z6mG$G=}4;17Z~&t5qx z@ahlKU-%h%{^_dyKR5maI=$&*@Af+k>)+q-!^Dri;vM~XPxm#VK`q)pu<5C)w0w)@ zl_TLpOHbr*#p7ez9qC=Q@R!$@Zy}tL^{4l?RV;f^W|-3Eg=04!* z-KQqsbl|BSn@oFm+fRjG?0&}^vN7a5`c0|wO*%Dx`HySnt6NskjrIO_`@HMCd*{$6 zJGWAtYV~3AgW!i_e`Q2R&jT!b z=OatU&f@iGtU2~%y}EA&ewj0^?ycv39NUY&F@C=@f_<@g!cW^L!L7S!>h;`LZMJ?w zuE@@4uzT#dRclucWIz1+hnqG%nUT};zgZyKH;uE1u?BYc8r-wa;hPq%?$>|kOA9yI zd+wO1+p1}C$C(Eoso36a{u{H;&YSmC7gl)u(!GyMANstWXTq=L7WOFrkRe}=`fO3( znT_QQdgopmJYN1zy_ZHzAGKl-|E0}4WpDkkr|v23mR$e_QLLtwhVR?9HxuFYNBvYwO(|JI)@%+n)J2FkJcQ$H8tBMjTk^D6f9} zhn+|Co}B~QeuA!=Hhs?g`ai24yR#eLJ#N_EXO6rqX@6i( z_s(Pc9~jwb^w1BJRnOf2HFUpq(L=j8Jn_LDW51Y3rMf(QqQ{qGUmFx~&zyMdRD%W2 zww*p;ayR}wZS}rec3OfzHDCVSQ)8ZM`*Qc4eYf}r#RHE{ET>Cu6&dp&Sg`tpIw zR@lfjlMiLQ?$qWtv&3674lg`%-*?szpWN_z6>Hk9lBa(>G|@D%YS^opvaeS@b!_4D z?rPCH6FwrRoui-by+0XSNxuKh9bG>Ebo{4Y;~?9R2Wz9l=( z#$wML-?q!avsB&t=B*#y5#pUzZe=sB?b0pbr(YG>&c3>&8{EF(^shIK{~#>=deeKZ zmanra){eTh+khkQFN}Wr%bw7P&5b+n;McEz>gYbZ@o@Flb|+fiVNN{c()Dq-SaqkS zA!A?uRNEaptsS};n!MFw*1Q+*h%8vr^jKzi;K<0YpBpEsJF?lgBYQid+d^lM_rvQs zYZ{z>9wfd)>Ojb8$=5xxl z*%yCS>Z2+e1Ef zNAT_8`)5BwwrW{9VDPO|C&r`t@Sf%)myPcC!Do}&srI`mXmskJy~nE+^D<5PeYbDG zk{^G)`Bc@1SbE+GD2&wyQkLp|Bn(+CEbC1^#KPG)?!n!f@zcT3WeVX&a{E3!+(w|_DSq)$ie?zQ2&!|NWIwQc-U-H!4)Y3Jo$ z>h%2ax=)_n+OFS8;;2gEeb2kRBObI4Yu4Cnnzel|`v9)d&3#uly5}Ryw=H_c^dbLe zS;(mz*;0D6*_PM5ADwbJe|h<)9~%0$-#LE7LzW|N&3@^}4-CE2+nZM{>fPnV>P^87 zH_ZAVlx;oBTvsCKbL`XJhaUf6_g9A$H?S7$d~BPr+06YDZ$+oOmu!5m;)~mse|oC2 zD}}CmRmC~l@Pi-btY(=%I2QcnT$4FHc5T~TQRi&8-LJp(_Od35s9*a9pPu`*_rv#0>h&PUZNpz*2q(s!Iku?J zcyHb8lP`tl?ESTA-OU}xmTh0*-oEN|`efJV1LL0A;Ql!<@_BA`<-^3zek1AoH+*gS zJhGx}#{M=H2jBQAee&rZ%I90o+O`00;2v{Sa_CTx7VhJF`tNS{fd+e^*UG8PecvwV zKc++E^&8IqcF5Q&BRPo7b{-Yqwes5`b0kO3-2UXyUzZ5VJMdeaT_jRG`1*@awF|)X z@sBF+sq5-=?>*yfcH+#* zv0DcB{9g1+kAqds#~(VpYF+2u3%ZxR_+hipU8|MDcOuIjYg??XdI`I4iFAwb*V&o% zr$RgCNCQN^OK@M+hY|~ zv(jz8vs78Oet77djv4d@XItyTdne!j?%tI@+&E{(z1@}#Te(POUyi=l#fH2mAN=+B z; z&i11rH`%E1bJN){Jda*AN?)#EYPI+2U?LJt8hDUz%+VWR6(@XR>ujqF2 z;giRsiCs-k%ASABaQ5zBJI|T7_W47a$j=++jC$8`r@u69jh=n#gyW7EbaiLl^NP^3Cnjpwtbdah(lJD>XWjxMd1j1<*x zaz%^Ltz7>n@9hnDzULADvpbu%`uYB!qrKkdKM=X)^eh4A zSJRi8ic=@+f6#mKsp`h}ilhTwuLR%UU4Pq~H@y3%@C+Lox7~5e)<>K6Gq}!;Vvp#y z{p)cLkAC^tVV~98x8j$b6~~53K_IPs--#n1OZ@eO-;pzP8>TP%Cb6VyxVE>ueT(l8 zrr+H4|5$tLur`~n(YLrmad&qJgy3$$-QC@SQ(TG^mliGV?owK`xEFV6ahKv8`n=C8 zd++Z%=ey3eGk@HPs=FLU5$?p6nY4K7AqR`~n4ygy;5zclQcR4K-d)L4u73WG z!P00`t}$$g-%5$n(SacaD~UYsFOU)noq-do24?qwV!^Br>i2QAVdvK@eV}yE8xfGT zNg&5Y(p@nwp%-=UDED-|#AR{}we`y>05t(JmGfzS4CpXdvu6!wwhj_mMiEHL$iX@{ zQ>adZBE{cEZ&eM`-!Nx&1#3#|zjusIjUB-&%$<9~EbtND%ZobGfgqetQHiD;nG-_eTnq z3~g}CI^hAwWY=Zs8DzP@xjE*Gxcc8Aiu#n!S+|h-zPxJfpJMJh z_b`Q{B;*vd-3wi=towsD_KR4V((N4zj~~TnCDj?V@7eo3pObTR`$RV63=o5_Gs(Mz zFB7LjpjD^y7#S+dD4Y!8Bs`^y=`|ISYG|#zsJ)M)RYoS1o-N&*SIj}-zy2p zg6CJN6?z)O((ZFpZRCRUyIhdIVW<^n-w0IA)wrhzJU)gJkI|1(T@{FR?KfH}0{Nkm ztqXDVUV7GkH-92=f2R?sUF$-tKIC!0aRr?u-!Zc?j{V}-yBl`;rn*u+7Ul>3Wz^cK zrP?_i=1jYqASaSyXv;LixB-qkGAZ*Tz76)LBgClwd@S>n>dd90BGE)NG?rf1%Sj3q zBO~)HnCy4Jc^)}@`fqk$5gi*jsS1rhOB`UehWJ5!xuW*g9mV9B-=d|p3ev1lfj+3a zth8jlOQV0s6&B2JMdMW*C-dg>BuRF(-=3RzIr$4#zA+T-n@;=(4?Eds{^RaNzcr{u zbpcFG#b}Ad&-%5Dj}jtLiY+J+1|s31j>}l}G^2G}58XyfTG~|chDiv{$qmEp%lfKvUYqY;r@QxQ&>+aWm zaB*yT9}~V4$8dk?lOSrk9U)rkB_u!D5G8)%%0qH0%q2Og`Z>IvN1Rbkrg$gs5`QlV zr_fO=U1?7(48@w7Vfxa%4`M5hfqeVqboKON-tKH?3@hm%Aj*Ody24Y#jwV7^4cg+- z!{Dhss|Adu zZo-$&)H83-5GABFqHqRIE=^Gl6=Bm-SxlE6YV<+!t9OrM8Desjk4E}-^f3=_7{Bdu5U>@g zu%;3jJE}defzPU36lp_=0m~bN9lce~=L5otLIAat?bfn4Hekd)oKW<73o@+_9BNxf z?7*O9@yBg7iGU@3g_^cR>97GoF3DUmC3QS#5%IGg%M9}s{7#uVvyLUQfsX|XSscCA zW)3&qcphjkbrr({YVq86ZINr=oqosI-0db7Xgq=B;vKxJAer~tGn_*rq0x^ec38cuPNUp?huc1 z8q(vU6+clF4EX>R_Cy{<%{K56Cvgu)znB7#HMUg#E%x(Uqidm{Sk!EbdqeE+d_=l^ z+}|tP`qJ&CQ3e`1YfQW#u)H_&9XMoeJBxTR$Mp3s=xH$FF$k4Y9 zIPcj@u0wugE7h^UZ54yWM6hZQ^9_x?9)4)(mmR2RWlSM$__U&br7cr6QlCq>WrTC&m7d%B|Lwvvy#=hYtnAx-#OOa3BsV*R7S6x0(| zbR3yt_IMF6=eACEHPm#>qQC81fPFKH`kkD6?7891v#K${1xN3%r&2#9Y z`k`!U)wu?cfSYkV0R?+^N4W9!sXD(kof@6kIU&ub@90fQ3DZ{gh?G!(eENZR6ox-S zdw3my*I(llJyX7Y*jK@BAwoqgE7JHT_aoQr&|$uXsq&F-vdTB#IW4@H9=M^h$|-LU zNW%-(qUqld1aZ9-V@4LR0KwM3p`zr~M`EzcO4?CE%o1ChiTD&S+D&vCaBLBH#%{I& zP7X}sr&Lo1Vi9CiEA|H8WAltMQWI82qt5uQJ$e1Ia`>}-?zo?psr{4b z#FNQ#*P09+?px4S%+odfv2X+xHr4Va-)Co?HDpq`9!`G7PEF}cvd!UeZWM*)shdp0#SK;Sc@)U@?S7*=qhAQ+&cX%0fD{4upmkhlI`)WS%S49L(RZ6 zSFpo2se0_&XO#c-m??$PyadJUo1sOP+^QFUX$&#L_wS+chmvrLXqH|w#y6q>4SOOZFnm2cxRuYiU=CXxtWsM5Dz=o|ch)jwStc%rvJrA2v` zRvd1l2GL!AX%L4RvNsw_3TP0Kh{UAiaHfdAQmTm;$ObUYC*>zEpLtd5FtLT%t$UwK z`>iBAU50dYQs?Fy{+c|VnPjNhMI#FJt26cU^dH^)J?|&-^-}ohY1p4Ca!O0YEN)Q- zFJV?oxIRe0+&h+a6=jZO&2aPSL!0y`!pm`auQ~6p5_KzN_FuQHFp|8=i7q?_#xh8z zVEsdDAx!}?Y7TIpy&|RG&Iq{GwaJdy2XrBS#C$3D0odIyi!{zPBr?@%Pf|>YB{wye znGA~Xm^Ai)=u@nGCUG)SCy?0Nh&O^?Xxo=^A{CJQx&N? zyNPN5T&0vsqCKBbS9>xJRvj_0t6d3Va?2S;jOR`AK97v44sE{JjbyUl_jwGI_z5}7 zY5?;;y?*;3w>vS{mwTXLpsb5+6f3;WzLq!Rx!#IBAdvkOZm5jpf^5SF%=FuC1)s4W=VF?zG119fLxiRwnmP9ba`l z(j~vUsilaki8IlqO2M9j9*lqw<$FtEav$95R+E$MuLW~;5(N0<SXa1DiDpNN^H_3%41zlcEHC znyZ=?NmaF#YPsqmMfOAhXjAlVx#T%HmujZX5T(i!(T&JTn!S&TJ`B}Jit0@E ztz><*QnD~}R-62o!OE~2i2Jrm&J?YkaUT?%YLF2JV~!GH!4bpy*z-S6ZIqu6k3^7C z^0dC|ZMSY{Y!yn+MZD#VZ%$w^8mghN;Xg zC6u2Bf!xhz$xxri*|TAqv)RI*e}eh2zBIAvBee|RqeOR+jw-~wn-J84fuF*WP1VAg zMwHGd`fI{8Sv%kh>%M@bpm+yPx4S@41Hu8(bm&&SW_TEFh_HS|vp)_G2mXTb*a(r6 zdeAxS&%3u1U;x1g$eM$T#^@!KEJGrm;0M(?l?PA#(TeQ~{*AY;VQ zH0}<);s{R@Rv`2Lm7;6*MlJppY)%vIA+4_`gTV}s>qd_7_iS z(Mu0TsP{WogH4A`eWWh_`;0M_=D?qk#FtnT*ag_-cwMj??1ArgOoAl@^f)IuJ4Gx= z&$(?QgK%gDVLYNV1eF^Oq%Nx+$5K~mZU%<=0CzWpbIYHq3=mLoh{L&n$>TRRh~v0} zRwJ!qHRgjh2_HySX%_AxyM3y5iN8YLL~#=rm4W*+QlGeHPkRptDe!`OjaiQy$?Qq5 zOxoDKerug-?HSDb>VX}$#A+bXsWiSrSoYrJjhN0zbp7mQXoE^8#v*mquxrVFsFSFj zZ9od)(%2<>4fYzSizfB^+eYDZW%oeEk!W^Xz!>!`7yn`z&_i-RDq4K48#yq<8Q#b< ztj|m}p)W5?+`lYI{L5gPbj?OphL>^VY$+&C#!$T%Z>+CrBH@oG-O&UCJ)sZY-Urg5Db0J%E{YK>=m$(DLQgu z8c(7_O4nIRG$X(V`&twmRZ)(~7MQ))=}^|qApMnwv-#%Yh7uRZ$B`5- z)0KE*EPY7&^6`O5CW)pk`2I`2$fUh1Wt^9aK>;_j3d8WD@RO*O!_4JphT(P?UuhqWdf6v;%QmP&{}jD7;DEPNbVe$3h+Ng~M`zhNLUWJbSbMI0c* zN32O)Ga)xC@Iza~pCW}w*T3UJUv@~1MkU2l;(~IfKVt*) zeRa|C^itkS%sq+uEmjxInT(w)FG;?BBK07J-0<-=?Y@AL8}WnSU{xODGhSO0eOpt_ z&fM1o{Ri?L_#n8e%7Sl?Nax$k18_GBLG=U_>7gAR`jv>PXtCZdgT=V}Cyv9+67>!& z{w>JClkOogho=-Y^#F(VG&{#ofcc(U?-p|0L@Vj^Db5JlR$Exai`ZsNUxfs${>2Zx zvSOXFCvTLvhqlWh!RkG^3sUWUUy<~D{@|+H{IL4%TLmEnUY}fCe()ZG^V+7d=(Vb1%`$r7m>ciifZJSpRD#IIF~_B zDK81}BTTbl64^}ubW_U^$H*g$pYjg+9f?_Ag4?Iv%ljpK6bCSXC`LkEs+9RHyuqx= z;}~TSgDtE^79Ha@bbdPJ39rm4u7_g!$sL}r2StMm5uYfWQk8rYRb({_g#e)~p%H;U&IS#jO{QtU%K)?h zpr@Kz#|{Q-=#Xby^a#_n97c{DV2)!et9mz6LmQ~yI!a-%^r)8_-{Wgx0YW^jGrzz$ z3Y~F6 zMonpVMX@R6sj_?GHvO1Oa8}tZVxG@CAG%Yt?uWZl=Zps$tEyVGzdAYn{Nf+vbTiA> zaltkVo3w2|xh=+bdg4DCQtYD?Ym2X*LP|7a=5itW-WpBRN|QHZl+0d-yn=7=rY*LbRuXajsrAq|P-u=gw~TTj!XI4+E%@cN|w0<`9~ zuIF_>;}^D5MspUl_=LaQm8&vJD!ya$RaY(=ozdL6hsx z;bK$i8D4{j%73B1Wi8&8@D8tW^i(x3qixM+6jQjB@aF#ghDXg?1-bHF?r~KrSjD^& z&1LUh{vNL3&@X1+>NkT%a>+Bj;mpi8!e+sZWD9Ypq~0Vfut-s0>5*2x;t%sMz|Xm2 zn{wIXfa2t}0Eshb?@BjP`!ila@3~g;dD@I^Y3-=i@2yExIw}ISovz4DXB0gO{k$uw-wJDzn6++EiMp{Q)JIU`Kf_V5&0;kU7Nj{ zZ?5^~F>taaRE#2ij{Q(FC5fe+tc+4RW9aFlW$M$3)@uJkb>cVZadTjYhu@}8>X|RM zF#E==nX$S(mUAIl`wJXr7cX%uErCgq%?VX;o;GVvQg|OjaqpuV-I#ml-m(XwVGrrO zS~=w^rK@TI+~hJABCD%^`NR})B}3-owrerhK0J>hG1t#2zNv!4N|K03dB(%qI;lve zp^e)9ZcB@@+Cv(Buw`<$2>pTWu#7*SIrg`P_MH`lo`!oCULy$NodTqi5szxzKM~Oj z`-ya`7ZnKr`lY5oSwcsE49DS4rD%G(_`Q)pkE2DMkBg+r*#DS=6zXRsUb|mR2U8(n zH*m_4;|ERj>fS)&FGq8fmiWF62^*Merr@hYhd0$}!Wlvs zAG?|OxxuS_^c6*>q7Bu&FRsRwk3(zyp3Y5!p(@db+n(|+9eWdUuC&2It_D51^2?h?C`Avt(<5=i(ac8*s9fa7#U|314uo`;xOsQ< zTlUfCsmX1d-!Il0PYRU2doDGGvpNU`L1PP+hW)!3w4TA_{Qe|xnQ(2-Q`|O!Bn?ke zzkH?NHmniXubLY=tBY%s5gPEVnXcbE5klN&+RD}|KBpBf`77J|e1d6DjDLCeST^xI zadAFB(6EOMZSKjjl!fpl>hL8x&Ps@8GD3!!)QR@I`}dS#b4qt(L}=>mfHpGRuZ{6H zd=%sySCDnvaIP*~G?BihaClReJIQu+WFN{N_Ku#y=~DEg_WcpEn~2bjt>~Q|w%obu zF=UZM`{nhY?PqYG;8|c2DlqD6ux+}SMfG4&THl+iO8*qS6y-Z#bio718H0oy8_w&3 zOk$aK)(D?Mp4nXLV<8x<^Qx#j`f99lSTDZ!fMB{ev~<3aM)3+_IugXBruvR)>Gg(h zIWfKV{N+7|K4OLex>*ZU@#NI31=mz6RlALA$ge&-Tr8Ftr zbwl+_jaRI5t~*N<9seiJL(B4_@V-ohscRS^Oo8*{6$AsA_E4zjSQLU!3^ELXsdHYD z!Dj5{j2e38klV^GC9$2kA#n+)UxZ9C*Q<$Zrd1#LtwUQ}ajW#qcnlUu^CKL3$mhii z2xN@p-Ey?W9X&KZHFWaPx^d*8Sd(+3&(>ZqDzue)*@}H~NAu%BCD#9n)zAWKCl*On z_A4&HD7+odXg$);ZV|pxR|VcAefRY6R#3LHAO*OeOtyh_&*O>?RS(&c%#oeT^c?5PqLv4)>rn&OcE4(^m3$*_Q~EKLmZW@od*!VaOlLA-jk`xA zTM}u$2pP3CDZM6zV3>%V{vMWR_PZY7;-z{tpfMq?tAy&X^3;t(Z+HV6O*2sy6Zg0Tu8e)s zq4h#|^E3Lbm}|Am+lBXTsm3uU72(Y(ZBW;fa^mx&-1V~uSk$su`nO{8{$juDx8U}HF^A`ObY56WxBIDin zNY$Aj^iPYxZVks(-0P}m01x0J6s^6Hjz+;CW~G@ zUJGN3{{1t{eMMlB_dp==Dg4CMJIcmN%K*;=X~g{dEU1>78j798@7f>iLY|C5?0@!U zc6%MMFxXobjs>5J;GGa3jeyaJkOSi{e$ExM$|Yt2Yf2=_gyhjWH$4feIad<-3B`7} zJLY94iB+amFJFe;hNTq%4mt?{&tM|iPb-IZ0OO;G*p%OX)g z*Vyb1SH>&ABzd9Hn8XplL$QiFJPo%aeQxwwBWN)#fwfSTcTg(94z-}*MpIs+h4BH# zL8IV;)~S(Vl4dT|>%kRJwckMd|6} z%JG%(w|O(lYwQhQxkNl>FMFk2anJ#>ycCSCV6p8TI*bf3olw7FR6}??#VQ)uq6J-i zXj78>Q);m)tS~Ht?;?WhJFfoNj<390Np;$l!hnsAAKM$Q`ZLsTF~4&ih~`~YOkW`8 zS4DPA=zvmCR&yA-%%>=2L&9j+gta)D<2h8yLmG7*1-pfc^=b#_RdxV|S5btAW9Wvg zI{&R+|40iHUc0!>+ANK4jIGOLXb#&BQ)AX3(}$1HAJb_#JUmqLtisV$pMFilcyFr< z85!VFSU6@D`4pdfbf~}ORmw)re6K+}f5dRQQWuvS|4ILa!TRedN6%UD zpD$-RSDW-1+ekOXB?o#Brjs49CT*gSt1 z9qtr8zrOjiwq33%HMjk=rie__Ju&XXURbT>B;@fZxXrl9@`9n%1l$&_Xcyc4s%L7! z1CP~2?!hf*7vg&I$b2VGic$h(nzF`=%9KUb2x0?kZ9jD2^rY&D6g%Zib)0IJnvI~6 zEf{JC^|P%_Pxt=ZaB6r&4<)^KLq>7WB!$T`qIs>wGu8>K#^+l4FusYiWI!GuvKm!e zj4$V{22KOJrA`HETk`w78~1p3GV!;#$a0tw-0V~0kExo9!&`(e35~XHTseHs0z)T# zQj1V_F@1VqMtqGzbYvczeq%KqRW#PZ$p|f6U$r{bEvBYl&D7+xsnTiMx1rI|=Y`)2BMrZ^!z~ zYXF*7sr=+lq36PD)`RRw$%>wkiahG>gRZn3i3>BoD1!@=y0lXIeGwDp;?*$i+GD08 zC-3ZYn87}WW3~S9X9`sBlmxpTK9KY4Xi@*<|Ke?Ek-qJb7~}v3@)=RJ6?_Dr+!iGX z1hnorT_y|(ZCYYO^8tmkdit_<4Mi8mQ10cX)P86tx|mpFCtiQ`Z=Z!y_H-B!ZW*AV?BX(ChW9FD_Tzr)<46rVHiL@M0wBN|IS^JS2*1d{%K12P(Bq{^% z+PCQ0(YC@Jk(3KFR@VW3(q-3=E{|)UpMnB|L@+weS9o)#uy?qe=4XD&_1*gX9xJdt z5!8g6UFLdPMtQ6&L(o%QM%u!3H)b3-1cw}c>km}J?H|z9fV5%|R~dt|y1BI*CYvwy zq`5Q!p*hAzj0{-_14V;-KM6dm2P}A~tHZL&nN7plSwl+Bs!`_xd@czHIPPHddXqSv)}Y^*(e1bsj*6BVoOM(t)115AdmXgZr&}%lx@V39Vka!|Tfv(ijVd zqVypBhI?}|7LGqU9B(Mi=mqiuK!}IPD zGz(fS)8_N(-_Qh7Le?T<$nIvoYIP%G>kiT0y+@1q25q(q-4g`;MUxh(`kFt==T^1H z2+`%DA_@)DhIy&Osi^Jk#CtFEoJJkvx{ad_uIM}`9$j3H{E`W^EUw{rW%-XFX}zO`h|EF?nQOwc8_kt7 zo^N62lK4BA6!(hS_V2fGVfuL+CXU4EV$R>xdD8DeNhn z5ER3r=T^D=i{C6rV!B&!9x&QR^*&d(bq{OW0ST=?>5t-E3m%M8Xq6A)hTD0A(5NuxTuh z((9=TmP+@Leqv%p2P4Vx%1cSmOsAPJxS{mCcdRM_QT3JgkyxQce4tB8QLaQ%3n{B} z;+cBy;6bnb;Q(!#`Cjz9GvY8nsjfN9hRI)z(#kAaoN$%)fzE>x#p1w`A$SW4iYft@ zx0--4s;;x9;`@@2Go5$*UW&sCo_$h&Y=u$)u3s)yL{v!LkCFl9N!SsvroYyb&W2bm z!jv{khehvT$abuWY~$}$ip=ptBXviyH}J#={CbJawfRj2bx70nN0$CARA-3wj-ZlkI@tDy#36HM4S_7QB?B@qkQe&_B^B_9>RKk3Cup+$_z)ss>Zq ziA^yMf}QA)^etIy^V1nEN7~hp#1Y^El2jSrJ9z5JT2%S%6-S3xKN@ME!XOZl!ca!y zbmLjrSeq(1GvJ!4FDA{yAscwi3ggkD4Q7Doo#`dM@qO1K=2Iz0vdPuFr{dOsQsQ`& zoVAby1)}T64_f5leTE6%~aVrim`&^ z4>YKZ!f)hxSg^gvm7D_I_(JkZoxWF-1n)D*lqH-Rglxmg0%DMW{rr)H?M&Zd*bAe1I_`2FT!`O z?t}g?Kh&jSH7y_Em$PCeyVSkr$T;4&O47g7Si7N*a<{=VK^=s`PDoRIpE~5W*Mia( z(qs|F=HfQjMPyBV?j3R4K_?u_ik1_owoBN%&d?ODl`MpX=F<01w#R30l#;>-kw5y{ z(GZ^N@iF6PIH5QP&>lbkbcUk5)#1Giwn?gC1ung2HxD$pa{l1iAmj#vO$>Eo0dE-Z zH;sSr$5yRedWx~(9Z?ojL`zR~iNvZ)cu;%-fZDqs&qg5BhJ3qXvphlv8@mUs3P#5! zV%OU{Jw%^Lz9eByixI*nXMwiZjS7qeT@QX+lQS1O6r2~2C9?;Z}$2K_~(x2Z?ZthVUb>1Ezq zz?o*{e!$-Ul3cT<5IlZFu)F$r@6haW3TFR^-*WQnWMZriF(P*P?$}J){xg)n2LLte zDxc`*N4@n9!KWQ1wwmu)Ut*Pe4ZmaRyF7xMyB}D+xY4}>g#^4VD>6GEUcPOuUpzsV z`XHgt`$?yLYjCSe3na?I^@NMjPHq&-tD{Q%9T&DJC)KN0?g7AWeCfvTTpIcxJx4m* z^jV6Xe0%a99-IQUh1a_2!_>5m+wbmXkM7-_tVn)my^C>FvWi)}e*AUXF@7F860;a3 z@OjPc9rG`%d3D!&tsMW>Ps6)ro)C~wc6sXY)9ri4kl+XQLT3A&Q_<%G(nSq293K_E zJCywU{EwiulGW@qb}l1SYd2d<=EobYqR(7*L`1q>77?Y-SJMsWgi_*+OIY9^zu-vc$j(b7USUoVA5T%BbD@8 zn^bNHD2PA#z^7DU=L;pFjlO)VI}(xQS%~7Dqg)2$KtX?l%_ppW*WXz4sTqCd>zjO1 zCF%DyzgPE+q2S+KAD4{&pWwE?=wE+B*u-t!Au5(`63&h;&Q5>0X1uR3HwR}o^;iCz zB?ad{Xk`B&ed)7{sY`&tyuiODtPP(c0RHK_Gg_`jn1FN(Pi*F;hbWTQ^+|4g+3YV{2M5 z32Rn>6${Wn0>Y;*Cc-bMBEzW5%cv^pD#)!R0%27V<^j^1J82ovAkzw%GON;Q%LAx6 zTwQI=Eu}eGENoO|xR~Ymjr2_|t?7hl1SBbij9ooA%~cq=XqD}pY)y<PlSUOS&CJVRnk?CnUfpLrz{LIS2Ut!Hdf`~5_6&x5VJAX&`=Wr z(Tj-b$b%duJtVXYwd_Ee+OgEbrNJ^L{Hz8L4kxe#yDm4Ak*$ZBB8#lDgM*o_n=`wHrU{k4 zsk^={vJgaxMp07V0>WjYBuOo(;4UQ&1ljQN^E&J3NN^|u9GoS%Ew#m@c=SBD92sd$ z8Ej~bg_LzQd90b_WqH&z9gyt>oN2^(I2gFpXj#o&xD3Sv%wEre>Z&j>*tu#5^8gH9AAGb))itj7g1-j zl(E-jR}fJGDZ1)&i_-@D7yQG&oa2uh>9d2KEG)e(E!ZV3JzuRsO-B5cWC*0-{ENkB z_)lKqU$=j|-T%O&{$}ZM19<<_dvpwyg{54TL}}^im5oJdEk#A-1ZbIg8Tc%Pb%Yev zc+^E~jdZQWxP%PZWF>W^+0dQMD~LiEb2dX}n2 zx=buWh9VMd;>-q`4%U1YHj+$Kwi5aROhQ~H&MKVxl4792XcZN7 z4cJ61gcY5Td1zGu&bn4gYSJ_`%*slF%!X>3a*h%lU_m3x~`@^JBV74 zjtk^w?w~5~YAI()&7vs{QnNPWMAlN$mX@~?c9PWNR@Q)Us8d2rWrSs9j0gWL?jY_cA7mK+vzQi5#s zj8;-=ZtM=sOxi4ru11nfrnW#q4-W+$Hjp*DFpZA9o4KwFhZD1ogRzq)lQILcgFX$9 zk`uF@v63r?Ap~RtF*MQ9;dIjDrQ_o^HTbU#=8rl2J^tXoz?H9{#J?DgN~VsM3}&yu zO>4JTkfsIWKhT-V5;E+XkXL>t1>kRf<3BCwPpSV^5P{3iz^ySj(jp9ZkM5<)wTFYL8_yrL zo4+DiO`Y8TFhb)(zq=VPon>!3FqB{68`Dub@=G zKXw>@(5!#u;N$prBi%eK|1|U;-tdpTNWQ zCqEAlm*Kw^_eZ1uBgOIj|5V&xiwpa|6vqkR{GVu!>rcD>s{r8Tq2S^Hyqdy)m*amg z#q+AOS6(g0pV{a?1Mn|@;o-2}mNZOO)0@ zF!D}-Qqu%9_<$k7nELL`bFhT*!Pcj1iGc?*^R*$ABchFW`0ve=sUyBog@*Iu5tFqB zFnE9tQ?n)n+#dUGdo*3fH1$&hEgVcRjXkoYfNF}wd(hL~A2O*N_Bo`N;`Qp8vn*ZP z*kJe$`kEHaO`xtY(KKKZi*2O zZ-!&wmP+D2B2o-8*L1s8Ze*892nh$nbDPnMd&5>thRu58*uVX*$8AnJXAYw`869gQ zA;4dYlvxL-KNaP;*W-EArF7KeB>A?V>_`f9V=984i@Mh_Sw8+5WE$y`lDhzKV^^X|vI}~by z{DMzis=mm(XFy+Q`aM*kSTR8*Z1*HOTjuQV967m^@5txRxbI!FgopfnlE?-)vxh4!h=c^(Wgv_azlMV=Dgrl_n`Z8#&>uVTad_YLADK) zTfwhk_=FeGUxS^u4qN3}8<7ec6PHR!OKiA2RG8Il(>Oc4MP^!+wFIN&vrK^nIr%=x zGe2zFZG8p$Gm}_^^nJB=qOeq%R(hDzVr2S!Ioa6SkPVzZ&R>TdRyZXuAC0>Ttk9mQ zwXAnet!|Ssd~2%B{Y>Be{z?;hipZ`6LqTfvfp~`8eT}Oq5Wg*7mIYIM7fqSY$E&{%cI>Unh9l_i{bJ25XGSf+{;~Z!Ti(8fRdb|-EonVF9FsA+06p1 z!~7s&5Wm-_frXXlw(7QsHp{Nh)og1B*A7#b_@61x_7@B6g?K|KPo@dHbcC=mR>LVj zu#bNyuP@&bL9 z$Ge~o@(8ZW*Je0K3%0q%3%I4}ZL-L=8$9GhYzx=V$g*}Iixu!h5(f@$Pnch! z#EjkEfR=yC(YVcBqyN(9S$!E&?!TNv9Yt zi{L6({X(WmWLx)1tnqi-JF>A9%#mn@-PLcY)qT56@$)irM-d}un1*W7r3X_kc;XYv z);y!eW#3VdVcXSD@4HE5yBZz>xKlZ1N|hjXY*wSlk-GkTi%_c6tj{NrZ>w_B7BDd} zpE}{gdvrCelvua&fooWoeNrn0C6^3tgjO^)%7hl)qKqWwGh9_)4R_r5`j2^?VGx>J zfdcsUkp0@MAej{-g%ixgxMQV{g*G(fD0Y{9LR4#bzxyx&wo8z84e1dJm7*|#lNVS2_Z}!!Ky`)o) zqP>0Fe3cuL5iYnd$KQ8QslTh3zXmENQjityCHKR~3y-FfCpi4{H^tmB?&W-@ojdPV z*!&#+T_hOr-1Yu;yA#cvG5OP$-|k0SE_FKW_%g02;c3Zw>6-}yvbdz7gO9yjGh%?E zhZ1HF-O7=7hE@mD!<-^4m}EHS??o~?1S4Dp9X(h%T+DW$s|YBKl;`4Fm}?WAi|fgi&MR=43LMi zJ{`-^6wO4ntLJBgA=<8nt7@1ap5s@Hw&BR3`edhfHLsa`g@2!1@H%4Jat=KtaB9fLHB)->%(SEAB(rSYb1 zqtdo*+qP}nwzJZ6Qg8l$&_2P0Trz{kzNI)(g zleKI!P#AlV=15FvTy-$D#E?F1I5Bke<8$PXRw)uuP%NUXDvNGGzpDOKBZo8JTIN`V zN&F!CxJ3RHj58|t_aRKBm7ih(Ie($|DaF!BM3h}X-r{ftx!;Ai1)pmyNnJ)|7obTp z>Y?~~nzq{ZI&Jdc?MasLh5zoIZL8Qr%#+>8x{2O9@#& zbt%f)kqeL5|KSwOi5e&CzIz=jBMOdF%~YwnwmH-2oDEgppslsg;iX=jhQ_fUBtyWE z4WblKut1$0lvG_p)J{2ZS*H#~#Wb&SY~Fx;Jzu;zC^90qyu5vPI6Y+f&_*|-(QBB{;);;FnRVP<|UfXQS zR(?yr#L5qLpyu6LX;E0tJ9qx|yIoB`zoqKB)7i<`IGe%Q$apBcllk!#$U;YVsM@iE zGa~W=9l~M{`9`$l2Wd@8vM8iNljzULUkW8=g=Jy0@5gxJvgPd{JY{8O&p=c{n1y09 zyA-0P3Z=4ZIZC0cC5oy~>m26(3Z{)543Fz~8&?^3o=0Zp5-yASO(r7hJ1eIFc_wMP z#N%1)V_`?lGTmjB`SZmyjll+zxw6`fdave}`i+Ug#aWe8=} z3uK!p$}WQE@*#Pprc6>2-V%$3c%eGlj0-UGKZOGep^q8Rhl$WVF(&DE^$wBph+2tx zCUYXwE13H~H?thkwt7?W;Ataz@OpH3cQV^fITn`0mXS!VI+F&Ygx8f{ut|AR__ZpA zhJ_e%DyW8U?1=|1Z)uF6Qb3|dK_CWzGg}2UtjsLED@o%|M#c@E;f5J@hjIWW4lv;U z;OJ(V=-PxQ>>=fKr<=28dOsUf{1toc^bDV|kP{hHg1;8;~(Bbu9Z?6bxbcJFG; z)!m4KexuCJ?#LQ(G^Y6?TB>&?Y;`|Ah@zvab2hnRT=_Czk5%0Yq|E-L_RbNJuoXN% z;#!Ud4vux&2Yt5UukwGf@~eU18cV2yXh-hvbB4kH=@6k*y8^5Zj)mOwqs(Co56$aK zXJ@W{NUH1YlCjnOXy|I|w>#xw;7aP99_|s{O*VcJUKB&rUqXVV8$QADeL(!^JL;d@ z@rlY1P=-PuQBmdm710ZIR(02db4I{frNCsbB0MZj7Kd09q5XbXq$=6iyP+%|rncXb z@G?W?o40F&W(w-&w*I~oIs9{T_DD0a0s)Xr9%549U*73K#v)TQeY1G`YGp5_jKQ}s zP(^qLV=#*peHAr=eBl0{b{-3DPVCkF_+BNO==opK_md_Aa13Z4Kt4Qwsv0PBP?L3hrVv@uK36W(i~ z!a4;QCE~@3c5C)gLN%eI`LlY0VyG^!jxQNW^1bvdvMz?l&A`dXtZXRdz0CP&a4iTCGzDH|tfzVbn_em#TS3D9 z2fj#&m-y{~0#Zf*Qc?|YrD<<`=2@K{>&D;I{!m}}k!1DYY&^_!&)S3SG6s-28%K!1 zJXcP{lL591FfNOy1G^yUz*UOTMe(lH`rzE1ce2j_MO&JIy51`MZlqOc5< zWApS3toeWxdp!!YA{C+(+j;bE1#%OG$Z4|tEZ)W5X~{JSf13ghi=M~oR}aaYR`Nz5 z?RKU0PWKSbzBY>JGeHXEVv4IhYpF~x+u1Sa6hYr^A~Qe#5~QPdCeF3dMrvn@tZRoP zRl_7ZRWZtw9<6+nV~Jb`Hj#KhNGcL)!+o*^K?*WywtSJmzRN9G-fp6>)l6 zusQP5np_!lnKy)SJ6>96L*C_0Qy;+8JWKQ9LQ_KoTKK-OxHGgzRz|XxJkv%dj z?T;oD1lXUmq|hgOlLI%VfbY~aTBHXHL(lWQvvnm8K#p(V%~yZ3_fD3|d!=&=q&dvS zA)yF70742=1*BM3XCCg>8kEIOCSq$FXj5aNlKIDi`5ZNePC@sk>-+PN_19^BGdDI+ zd8NbFm*GLSUX~vYRY<*=fMRM0t&O#pz~!9mcu*rMzJRmr0~a|sQx_CYx!rbtk8iK} z`YtwpI*Zy5%xdPbd-^N>GDx(CAoy;UfO)<$G7~I@JAi~Z9VN|Q0J#SYMTLQnY{Iy~ z<{UJVTWvjg;2h4y2}!v1)UMppA6O||vbeW_{_410aU-ky1xZbpoBN&7_#tb}&)x7b z>kpi-!`41ukP>zQrH}|bzb?!2Nl)%IS>?9)Xz?K0suJj<5yJvoUd)3kA)rJ|WHg^r z23?>cr%Qw%;d{M-A9F1&jT~Fb8H~-_!O!QEiPhbJ`zN$dAhIm&85Klu7(U*aD_8EH zeh~QkKr%@{GSW(HKcKizo(;*&e;@$k*Z_wmXp@LYv}p*mmiiYjVP|K1+c&L!F?sDi`SO?qXib>_kBixeNniv;t%LkCJP{)^8^g;_ zX8MCXV1-FpfCLC^kp8}CT%p#VXWr>u)_8RWuMkTPAjM%~Eax9W)o=rBnVDCDJmAFc z$4;T0{p7JV*y69~eJ?%4b$5A}b(L|&aphWlgk582&RgUGs2)mM%0GBu<$B!FZ|WGa zJ^Bt$lE~Ci)iQpqKtfWUhnb4#7^#<*5G@8fnFNC*cgUg8-_v;*`Plk=$YQQ4+NGfy z<2gmeD2IiOUdOqZKbZ|fs@2R8t|Dueb}}&h>8Z^ZXJhxV&q#ZzcZr){}06;3efAFz_>qk*2Xn--)#1M26fB`v)>rijkl8 zwi+=fV@Rs5!bw{cOrN9Ix3@9*WJF=3l+?}DfGTT~pP+uOU}JV3{4mZ*f_7r*Yi8J+ zMBOY)Gq;j-fJrKi^zx*FBbWHNLxUsgNmEU(ttB-Bz+HjMhE=e-5z|7&C;O7~d9`v> zY;>1zn85406XS*yy=e%QfFtnqhh1>3X~$&{)HTvrEv#GE5ajW}-{${=;Fe&Ar_z5^vS{XI~9p49PPD>N9# zhioQ)8~s^33tYHy1*o0pz%Ki5Crc#UiYp<2%K&ogvDUh(!>3)#=x7aKG-uzC6q~qFB*6!M^CyMUbzMVhmx{{bQVWe;(%v8&w|9@k=wK<1@g7B-fs{vxTV1THWkI z5voKy1M!RX=3zNrLvqE{33<`w~?xSkN@g~8huSM{6WeeA39{mlD4A;W3<5f<~;sbYI zYD5`=E6|Fs^OUSgp`RG-BkpU(9afB6z@Z68fr?WIk|qO9FYH6lpK0W5n_Xs6A~r#U zNQ>Z=zfQ-Xm&NIglh(YaWHAp2??il8f7^<~580+wRYQ+#v4{}gl!tM%B`t!Luw1)4 zj0kQwT|cA+9;F`=ddgs0~0S}=)wb&p5U96<71n68|f|A^$Pj?XZVJ|1lwJN zOUBt;9fD}^Abt_sF`Fsq9bN{`SF7fYXR(nc@nO_aq)8olDK#x8@ODap$cj&j2gD79 zS>Fw#16o0R!H)Y?n*Z_VHOhX?Sj|zeLjGp4hAU>y5O7f$Q%&fxPL%U)MOySD@a;bCa>1 z`5&F0?S9c<{bOQd{v40sYr%y1Grl_J(%Mrt4z$P`Y>deB_Je-;GquOZ7c-(Jci&>x znVJ~$Q#7n_aFTNkmE)82nK%iw0Yi1)8_%ptYE z)%Y`ev{rw0K}b^~zH0AJd@X1gwg$VxytejUwa|LQBR$EPk8jLyQUA@A+$+x{-gv2P zkgTNBCQSiJKOD*+Zg#K^dlCC2?D;Vnjt2b?7WD638UMYB=Rf)DzjMm}z+&Hvxc(K3 zvHl;i*f$9McPvKxjl~%0|4S^!_^()u;U9=j?w^e7zsF*X|2+PGkHr}OFR0i*Z{**{ z|KC$FCT3=~|4GHZxgW#7La~HIcZ;HMoig%kcwPLP)CjD^)P4FPiOqvo{iS9};Sk9v9$kzDAVQtYd>j!X%`yr(=13c0lu zLc7|38JP<#1J2j8=R))T%vJ5%PjNljPqjW}xPPRxJ1n7NyZk`Cx5hLR&sqK)?~a|h zIWM?4A(a172?|IShLaB6ZCL0hsB^!4ya~L4;>BwJ=;3Og;(0&c^SQw2dFaimJ+F#8 z5|Q`$;tOJbtjQgIldd;eXW(`;(0#|P{)3Z(J|WfZajTj0d-x^`YYI0NFc~~q2k&e! zG&FfZhOvEP7=@Oa(PD4*3F9&)iOR&|YGUDQ4OMR<6uKxys&l|yT(~l;d~V8txUZx! z(r23u#;phOh8DEE-t0&SWy9E+@5yRCAd;k9nil;Tke1)vu%yUXl7cy`Au~a^*npTY ztL<{$Gj#a=WkuZ4Ew`*Af-x%VE2ec80L{LBofF38zT=MvG@paYH?NCuI@5FWD*#@Y zFb$p$IlLBhBII`zDz__EQeGxQJ9+iITe5Eih;-$$PJa? zhhsjP*BT%eeM073JP`E1@K(J>7QIRZvAxXeVY=%<7t6i~2DR}|Uy7Vz)jMYy=~ufV z&yZRDch4AJVQAOT?@--;!=IU?c8Nvz>HN30k+lXfG6t`1I5GzCOhAKagEp2$IBY?A z28m#&y|4$p+VF;MJloKmXF=}qAGfX8C7z}cRt1M>quft^ctt5+<2&j4b&Wf=Mqdu% zunOm+4+yw`O2?;jCKSPJ2(Ou9ZG>&j3b-V)pE#|G>o=@b7@B6^~y(ZhT`pFy#K(HXR++zQvf-}OAceT>569L; zk$;-YK%HlAY;G7!3MrKN68#v;Q=1QSLqJ5RRnQ={LYChZ>NgIE2)QY^>5M_8_|?jv zC1Y8_jSO807a1^^wwapEauvTV{)$L!%JeW>tmgC4RoZF(>Y4LdohJq=m8UF%zI5pJ`K8|)cRcbNLk@#+Lv#y`rop)93=FK*Co!G6(=6;Yw z$)((u=;&njxP7j5Szo_q3}MV+gyr)z3Fv@WU{14(v4($sv|tb+Tn+}&C=}4pU4YfT zLxt5b<4bQFEL-Kdq45Y~^BBYsoxwcKYjhw~Hqfla)d>Hk5$lDS!-tutZo*)hTWR#r zdaI%92;nsS)6lfIj4N^uhmkMdAqI9PlTpa|W`o^qlaXHAB|}?^vO6d7y-Mji-F&wh z^?+rmho&6cDYQjUG27x&dTg^FBlP;z2M}4PY$&l-8}Xc9P`1b$UeNL!UlGwLO0~(7 z(nX#@r&d7R#e#28#$!3GP$ZP}cR`NO0$WlWR%;kEReTDJXB@OYs9|un?#*T}W7uFh zo9PmLbCE!N?*$&`=y><}+D682V|n*QcrM{JdIa~WMX0V)C~07T4rSlbiLjusloVj_ zr$9N6ClhPTFi=hj^cNx0cxry{t|%CQ=28mo6x_WcBs~1kmSQhYa#gS2^!H+5d;OcB z+WD%gLoRD@*m$mxRK=L1aX(uubox%;XsZO>$dNpFDyBS$)*d5aWg9YWL{3Y>9^-W$ z?&ic{U?&F2VOa1$Vc@oI>?IZyW*q$qHeLy726VWFGnskZ+TfJHGVNV*MuTx^cDmTu^(>nL^i0W_7tkh)v|r~rG3AYBQ6{B(59iVpNB9Y zlqYoAe^=cpv!Bo4-z1ydxJg?KR{9V0v zQ75sZ952_ql5)fHR;HJck;+>Ul=$1zXhsHy66rM}Wx8CCAt)|P!!9vGn|KYGt^S^( zdwf5cl5IjxbHk=|?&6OaA=?DNe6B+Y19-&LshH6lkvzMzPxasq!t4|kGT-zj&Sh=Z za4eWa?aMs>3oe;_QnGa6g6+aoqDrgVR;MLa;1HtDYO1s=5v$2!Pq&M0=W*ZqREorK zK2v)wQG(i1o8PMclzE-GyE2mfV@4`HdZC1Bbp^pTR{KFHJiLnT#gmy-C@(f@5^ER_ zaz*UBQ;lBFAx&fAt7mA`By4xIqIyE32_+kox+&xAitb5DyOz6tl{&}EML?&g@d|42 z1$cJ(Qdifc(q8n^eAMbA#xKhEs?E9wVjy)%Npl~`H-Ws~V$;T`K_%&5U>%eoT~gIs zY6qG-52~9DtQ5ct%+`hNGkZ1c4`!8Zb%W+q&dcRgy3U%zxt+3fOVuN!h)n@wS})dZ zZ9JTk4E)hTzm<6neD=S(nXNwbQg$xp!sf77;GS?JG5ua*@&XewHRb!T@ zfzyZw^)I~Wpa+-VYPJ;&^@WkS19!P)YtJ5xmcPPH&;5Ozx#~)te^kj zGYIx^KSw9bY0xJW<_%*wF%SElgVSHY4KZgCGzi_Wp!-k`@-ST>5RXw-4@L=EF_Tir zHv}7WzZh!B*S#p_??q9A7Y2*MpkM<)G7+AS^oi7K+}MbDFJdgRhTD@gKTW1#^NCMJ1*p#6k57Mhh#8NloJ)eMR$W<7XIu^A@%nC~ zIGQBld@6Q&^L~gk(Ao93E*N}Cx1$LS7>MroTL_3nrT=*&7%C|NWs&JB;*qI6;+Y?) zuAF2T4ipQ*sc%0jnJw9kIuxj|6YBvm1|YS?KDxeayOF2 zwbrW6ZcQj*=2T?2*BPoUZVG;pHW@CM?~-hr_#hCpZVZ1@wr`I-xG$({w1G(tui{YH zu9CqYTry<>7fOGbDuB3bVdxQ=o>6-qy*M1v;dT3z0dsf9*Zs*ywZ-_koAY}yUQDXI zB8|4!#qvS2-WEm2-3Kt4V@npk2asDXgj-G}U3;DW**LWm331+u){=i1%mP%6i0xt$ z6F7}3m`g4IMzC_>)=z(1tdACmDAcA;bQ^l0Jlit}4Q=S=Zr|lbO5{cg6^1Uo5V5(m zw{b*Cg?XWlkAR#4ok&Lp0*B66W(PcWQLX7P)tZXQoRY=-m~`K`1bdDFevb900Y~4Y zl5AAzxdf)~PZ|JiBfda6l!85etrdfl1zt6NHc*q9O88=qjsdc7S~efLFDD1Q#pN#~CPwJ|`S2GUkt9pqA5n ze05{3zhj~>{NO&lA7GgFCgau%ts05I(+dOmVLyFPBl>%$Y85F1RNEzrzv*uYU@`r^93%lJ06z9{h_CDaM)UJu~S2zI9*My=c_vTJJ7(l-Xx2h z-?9dVsChi~o+F{_U3#%;K89gwGi;yUCkx8J&=nNWD13o^f-t?5t|jn60Dx+s&(iG@ zywSgilqzRG(^f=TiCBtnXDp1XN>TJ0P7j0px0Y>A=aX41PrdPM*y-un!|Gj3JRbZv z>{BX(&Gle#>{^HA@{7XNFl|XFJu&SIW^>u$Jrsimp%_o6OSYiww9j*f*RrN8-7)=X zWQ-cqr=zJiIR7F+l}STvmwA=_u*vOI4daGh4}p^+@Y?A@lz=2}MLB9egtgV*>+Kbp zq~^{)TV?6qxWYXceVkcP_aA_}CRq1{7=jo(Ist49Wb`UXh_dN=hu|dv(e_HHMkvul+4BZhK z6Ei+pVL2@q)TS4;nSzK)Mt9ftsh@}!Ve-yQp(TD9`;Unqujg*uVETEaUr89NcF}05GjjDo9KOwYzDp7KJ;QVH#aP(b%j)Ls>YRC7LffxF zNg($%Z9KX9{3SK`6KwhCXzljP!rXrC+AgvaTN0mdkrn1pxPx_-gE5AVwNRGR-q+vSBA7M_nr$%JUQbXjOTfztf$Q}wk1jV4B9yki8=#tY2zu~gm?JbRRs;kdj zIm`y(S$#;Z?8=&w&9n(_Ow`tlywcz5_7asyu1aLQ>99d9qL?pObA*4f|0C>;^7W%V zf7kUh%hmh2%?A8pCqMUgmS@`@x^i%A#wSLY>2EN5;E^m`s$W@m#H%gJTLSmckPO2M zBBt=uv{(n&{U>8ZbD~wxmC`odys+>L!EEsCzG(sxT)*|3479AyJ31aqKJan9cdl0* zdSuO0jrBXrJjs3k=NXx>L_?T{AkMzP^#Qk9pmpE-6)g{B#j8Df`S#k6iZ856#I|Xu zTV@Zqui%Y3xXQAKz2_^oqjL%beZovk(Iu+%^QPK~LJ;wlv5hq4W+tH~#aWzociUvi z%Z%EPpSeGC(Pm}6(K~ep@{Xav0s$1q(I$C|KTA)b5Nk^^e*J>7%HFb{(%Wnr&G~Y| z>w0?Qn7ZSb8oRpSj0oP&v|(aD-%urG*{Kb=YpZbBGTMab1?u4}(nym)b8fa%wca$| z&=^T7w?lJ>@vX5ZJd$Y@v|^DRRIEC5x5Jbuo=g*$qZuMT=k}rgn8Z=ysc<{AI==}@ zNCVoOtF=(Q%t+9%$DNGjT!czzKDUu|O4mHLNXWbuUww4iz@2oCvHy-#vAr!EcMz|1 z?^Z~wK43QH&~LBRM0Ll~sWP7Xq!p?8(#=v9i_~mZVJ@kZgwP*5;R)5kxr!P!I*PM! zkdq7F7^|2#@=cG?c<_Jd#U3s$HGQuE!-d$|CQW_dBdO6YHhk#UD8Ylep6i_5UVEpp zA=PaZ8ENdf1gcSuDxUPsuDCGvjDd`-hVu5Ho`;jy+hO8oCL@pWgN*VuapNzlCmBtUn9SRa6)DW()4ZZhD&0GPm;gi(Mfp0 z=f^XPgvj3(C|wl30HcwmYxm0O?LN&LW(s^5U&M`67p%O(yBpO#0egMU9u6u+x@Rvv zYvj0=wASFq=QB&wtLh6beI;AmrDNCI7IHt_Y1mW6Gi5V1S$Y_r3tSjw@rv9Suy0L( z+k`8mBUrr9H;NBe3ykknr;gOS(=ycgsde`4Uw;Ck#5MpSx%`0|Ig?G|%jt0(l4viB zhUO}zZ1@9{)^)3Ef4jQ-4-@MqUt-x`F|SjZ?lP~L$ZRfiAI)IO;evlO_D$$_%kIC< zH`p^U$=~H=VdgPCbAAvKT#Ed?K~UX|>Lr}r_u~D!+kZfJ{8Y4{=jY>L+2J zWTu)3d2X&z)oy0YI?hrWV~^sBB)2bj#@V=r?h;yG9muhbR@p5u$zY1Z5{Ijw9r*YX z^A%CmNb0n36(t`(STWHNQ(>HCCoed`0Z=@o($l(Oby&nuZvc%{lw0~8JQcaZ6EEv+r*E4 zcV(!O*h+tCHBo%63tm4S1KR)Y5hD^A-+3EDgJng$^;So?qL9{be!I}Ukj%Ajqz#MD zqrp()%sR8SQBT8)380}W|LFt+k^ahoRLj*X+CaLBcSZ@RG_c24&VVlQ7hFH|qjoax z5hrpD6N~?%Gqg*kqlI{p3ptvEqN+GKjiu)@nD}?+EESgu8CiaghFI&AGQJ}%W&+8{ zQ3qt`2yPYuq7aLiw!~{eKo&*loS*$6MuRjREBMt>H`0XDitn#xUvS8q`7sqdeOvM* z)E4!inEZPs15WL!^snTqblB+m_GRO&Eq9U!mAEl}qSl7ihZH*LMx1i!0JS9|`&EJw zL#m(AO_fsM{7;Qd#L0o5@H#(!0z1h57j^gl5UbSx=?=;Lr#1I4;?RFzkY-?H{MQn+ zM%cj)n4%HEd9rJ2`*53V*Yq;q0Tm0fl+!j}pPS?i_1i>KOIKNB?BUpCadx$Kwn$wJE#cYiAQ zBTm;B?{AZjaTN~J8<5lc0+6z9(G&a;Wq`!^IAp+A{Xt8>FL*_!H{*SbVWkoLqwrG> zXpzW|LHV1(+EA_m+vMA-J@P&EKE*!9K+-@{z*)eZXkAawHdi#k@c7=TU@h`TP*T_Ce|mW|g38~f&9~z74+-c$EgpP^fAcZ_Lx%jm^dBeu!-%8* zPwR(9S)Eaxo`G4Dj#>R%;nJjIRsS})G#TmDnP?d_nP}C&JuFQo*6$6SCNrD*f8knb zvM{SNu`+yXgY=rLwCewxW2MQ;_^rPE)5~IIR{!5ctA8j~jPwlZ%(Tp!|KeKxbFlyZ zn569t4DG%(M6&!>c7;t{*591zuH?2|M*M)Pne4N{}EIDU)fv#9Pj^;Q_-{h>#T69 zqterplatfqb@&AWP5?y6>jU(ow+oc>zxc5KV>;sGr1|e>9VP|_rvJoXbj<$?2J=wT zw7?od*yM_H7762V7r`YDZSuJPQ&6%OJrQ3zokSq~8D7z1yT43=!_f$s`LZ(*vb98` zkVEE!dHhYa%H8yHn)|*qB_F$i2`Yem7nM#7SyBFX$y;(Wc;MRM3|tb2J1LfxOu`Ri zXQS@a7b9!u%Oun-lNjn-lA9Py8eHn6kheFmPw5FQ%~{PvEB@Mx@xGOpUmsxn0fQ^z zOH)KeJR4FWLnMNk97`#wcU2YP0e_i56Q^=}2ZoAe?z7Xi+nRE_zpH9Tv*z^-x&1Tr zztfdz2cr*-tEgJeO7+R^#~AJ+8?#x`O1#1D3eixpu+9dbWRXLZm&LnY#^!O+b51)atmgw0{J7+O0AaOcGO4 zz*>qq6612a?fN%d1P?fhD`m$IZrN@b-gvmd@j!WL8rM*OTU9J7@n2BwV&b2yZXAyQ zL$~&y_L=H;juZhBDOVNyrTFQTsv?i9a`py2vA?M;No;tas?mI_ba}7o-2KvMSAAuT zl5LYtc&k7$j38t&04p2w4f@Ydd)k6XC(Gij3GUWMpKY%cGvhyM_|dBxF1#*)5>{g_ z3!S)?pF@^gJ$49TH9TuxQ6YkUPvuMpVGFEZc$hq-S1st}G+QEV>{Yot#k9j2X+C*t zy&ou|r5MuAKlb7KM?jyS9QIshMRHnKa@+WF!MLqWPvlGu1o~m4!L+taS$kfCprIK| zF9gB+*M!gMVWZ7`a@1N58cw6vK?_9i%@5RxBGe)r>hU~Jbn`gZ=oyYy|AzzkA>YdtYzMHD-iVoM*sY?vNh)$DwFmop6pIrlt4-{Z(5|^Zev51a^HW&flBDNlEiMAaWj~Xm*l3cRhj6ek!SH)`MVlSl0550Zyk} zYcej4yCo+V5)&0-j~k8eAQm1?bqMg9VLnIKmrF64?O!JuMDlbf$e2X7 z(M%AW&l2K77-~%*CnSo8B?QFdlDk&~w+C>*MMGU_jMbf!8^#Ga4FK<%5p(w(g7>Z)s3q#A}l#si4o}~+76)U3WV(%iajO_8xZgR0%OvS3-pXl zIb86|!v}9;eu>YW5LRi4_izz%IXaXf!ctx~*e|cuQq)yT3q~DfFOpw4YQ$}@g~Avj zDMi#DLez(xSo5bpocLRZKtg7gM1DVzGR`1Z4HyRK3F`SBLWh?fqxj8Zt*+w@KG9*g zJ7ju@b@_f&rT?Jvo6g;KOp7ble=GUZWGQWsx7e(W+jhfT=(YR0PKr|(fn(>>;L$-i zJGmF@7a^JDe)q$Rl(!W*I#hkG|HvurP|AK~A0(RL<`_Y`R zeRPMz7mrn3DoY;=XI;NhFiWf{1GI~&0t?hg+)Dq#4(n`V>a$_Zw4z}fEZ0(e76dQE z60Q~u(mGsH=m-~QwT*|#ZCW%6z%BbCLCCm)WvV{M$s1=-R}9f9sI}{7Iw%% zhmaa>wu$PnK;MM(Q{+wLP@Lt;!Oc!=_m;iOPZ+XC1~?Gwx5uh2FD2Wy=bLSU`w@VyW->H*uBB{(vvnY?=ZklE3hf0tzW2hye9U1~j{L|wL+5UQ&cd_#bH zpQF$9O-v)3H_|z(FHC9 zoiB_~Uk&#lXBIy$j!r8i5=C-vsx{RUx4FKXmv!iQ;|M@#NrXUw3hJOf0K(t-S zFn$=gAts?oVNawHZ=pdfe!9SFD990kE6KBYty_Qt>Nn^)Mu z0CN_6?ajT6wL7k>JMOD9lCAB`q|MDF6X|GHo;%&6zt_IDU6}d(L{qR6Qx0`7gzc?(|t8zlW%(neHm|?A+gN_z3{;4N1wOSp+*l zIqAvBNcNmHx4e@`2OAUR?P|E>Sfie12cKwBHz${hf#%?=9IuN9)mkiGIxc>j)kjN~ z*GvVJG`jT9qxfCe_Rn=icj62p_!X^=T^b46@gOgb<)XysdlUx==mJSNsl)W9fXr1< z))L_a)xiMw>7_RNp=dbEwH4%ckgmM#U!`2!b~+vLG+^5Lo*?<2>{`2L>eq>oT$nEl z7U23_`ztlD1B5buKQfuRnmcm9j$(bnm8=>f55H zEho{q*R%f>5n+CbIboD?gAuG(uV^}1a9A;eJ<rKIwycN7*S zDe34aG4WwYZ`WX@j{be!J1BeZ1Fu_p41>*pEGz=B!J-leP_V{9;5=c>vf=j3-W9g8 zU%^eX+3jli6Df$uU6+cVob+?Vbdh%H)~sQiTvF)9tq_YN4t=_SO7D0Q!VV>E{yj2H zKC<0EQZ9gP*1LyyJs3Fp%^vfCQ=B=x*HF{)k1c#80U}Z8Vt8)v1f4BilVx&<<#q7V zrP^vwu}IUzdbj8PaizLQ^t-<_b(GcSctjQpA>f_DVPz80YP6m_?s^=v07v7kV8?)n zZm1}BwkM0vDjzs;h#JnPlp7wjNUh9LjPCmSxafm(v8ke#konm9K-n;^JHp9Hxcf)g z;mRt!luiwZQ1RtLOd@PgE=WxwP>CBG{124RBwFr0{u!+l)p(6r)rZ)!S! z* znHVnWc`N;fLrtT#?Kh#o1pc+twra+3coVkzIS2@=R4RW}HdGJbf_`f+6aB*phgA zsmO$N0Rd9X0}mskTtybur%vGb3okPIt_%#CAHxzwySv&G(B-Poq{XPNS3ne_7jufT#C{pS)f+!^>l_P!vaI6XTX9d9M9&nfd zQ()tJUa6{UR^4RNcb41X%5b4LZ)zIG*A6YKc|^Q7fKMlgqih<%5j!pDmeB@$(RHZho-`s!{d8MS-Pi*rO(D_;9>*8bAxS7+xS zc0&iZ48E>W5i-GeOKR-Zo4w9f=OhrlDHIDb3-orkQ%0%INu^yIV;6t^(Qj|PB-5c_ zS|-&QUzc7*=G`NLg+9#zkJK!W>NEqeaVk9t^O(%=eviBT&X0{=#Pn_p)aN6a365djd*@W=G(1(4D`q2kMaz&KNoM?la7tA67MM;DddI^7T0N z;y|!G3qtw}hYO89YSt9y0AYpT{f~5%x8q)01j>V{y?0D(=USyls>f4tO;qOjyQTf9tY`sC+;l|abYs7zL#KK}Iq=4-B&!{Fn|?6dW;r^}vC zuRzZql$wM}yQRHGzKFdH54ZxsRZmb+h|P0H654%|_-KY1b=GnmGnxQEJ!mw0re zZwVGb(I7(~TAnbWDRC<(GTtz=L+`FsMzH__hrN|YzdM&V<|X%a*>>3~{-iC=vZZwY zMK3fbp_L=N;3TzX@cFL-tN>>x{*Rwl2#2wm6`6?@qvn;M# zgy>tL^JH{`v)k8`&@8EGz@zjGRio;wH=$|j{k8d;I#F=KO2qymE+c>A;W6J00l~I4ghQQsEqV9pXx6`A+;pnf-!2u7bPyWS2q7 z(d>OgFAw!-`rvJQFUAXzENu{@dF+$TkQHGV85}vP_wNX4$G9j*;SQ6IuiyC<)@5|F zCEA`{kN5Ua5T=#0Qb_vKS>2)Ssz{3A=l>EFBC*=Q+P7<`>NS5OCeFoXrsBaLhJ4!UoH)Qk)%-{4t|4;V*? z@9#F}-5MqGi2kxG7BKy!Snn31=uSjw)t=1BST+7rS8g;yWmt?TYbG{lJ1gtd;RMSb zKKZs*NA;EO0P)ooGOFwr_7*-?W`|)RY$Z!@7pD2rcH9^J>e8UewbzfpWM+@Ssb>$S zAyQ7{+)pvfmM*?8!fk3V+)mUy!S+(<&z#(w&#c@t6^$0A$dEpnupq?5%8D!}ujptd z#S*E?RAJb_Mu;XebnChhMqXMZOFs@K07tYJJ2D(VlgeZ-?1gVsaCDc(HXE8eSn9K> zJW>ECBo{DTFGTgl|1}FA*du46XbSZNiX#g?(FYYDr1Fc~-wxTwIN}!GaqcPsFy_OY zgLYj>NdL$7fAIDu@Kkl*|9Fz5A|+JjgplF7T&^+GH6=q)2$|<0^IREH$UH?MWS-|S z6+-6BWJt(7lOg=hzPJ0-Gk%`W=l^~E&v~7*&)Ivez1RD_*4}5jwwBMAI7NkTs})Ul zM5f~SiKd!R-|=$P_aANwJGi`=(_L{hDo3$wG2l7}^bZQ_KAP0?E@Ussc+UyJg8iS2 zRE)i9UhxVT`voSblOHcnn9PEICf4n-Ri2dGMDS~T$X`e#k( zbL<;+8F>Rc9SI#f9a?BAGw_mIR7Oz(^+Ofp!=Opn{OJKCzm_O|JQ)OJV z++dEpzi0*yO&a`{C6Ef{3&hYpoM00`zy0} zq3HnsvVf14`>(-6Q_o?ri$h;&bWAJ_Y*YMaN!1T;k^BoFmd7@P|U z=ZbSlQE_NcdndVWnpezD+6GSm_zq+M)s3^3#et>sZptL$k9& z&Zbo}FgLeyq!lxV=5Qs1IONfWKB7RAc0v>R*+C8l%! zK{i|)T_t*^SEyPr`8LSr)t0ec{RCT?SKgRv0m2r@l_2?uO`GR zx@H?IYYQD4Gf16{3&sh7hDiNuoIw1hfrYg>jxGCyf*}X02JA7_+|;o# zGO&Xp!^+$q#34T|AGQU^E!T9Mp)Si$3q=^@E?#Xh=tHF?)DC*GUTqUYZ3j$`);6-^ zhh~3||Q3@6q6ce0CFahSj%R!yZdP zuAd)M|Nk6uK;OS<3=I;NIwnwP!=RZI0m>u3Zwug&?b<=*awh-V^9+Y7YbT{MSx2nN5}wD4AU4cF%y%fT6t!bAkX2W$a(03X&aa0HGq?P7QXmIEs=$U%Ex z!`KBWwq0(ffUm=-;YI96r=Go6-6&%Vwn7&sTg`8WRQ@+l>AI3(2JxbCjVzDiC!?dU=REa z_LyV94dw{sVa_lqgpZ3`oSO$F0hd6bc(_m!++19|zn$?x$N#WFp81E;|5Wy0X#cQ| zix;e*UM^lj2JF%Qk^=3Yb^W(BK$c5zilR`WeCSOA4zUVx_b<8lFL3`wjsIDjSAti9 z2l^o&CjGhRmEaa5WB_Hn5?s8Pd}va2v?SOA?{mR^Mgfq+=8ekNsPoelPbwYolWa#QA^g<)88x&i~hV!NeLSYC)aYLkA`GHJ)pLqDY)pOQKO)`C18@EUQibp7eF7wj1L%j3`@V~21r9s#$uQhV+F7P zX#a1BG4=y($YySAjhGZOK4I%c^8t-iC}2CZK{#NdXtn@ufCbcrf)GO~R2M*l#tw~% zU;#G&z7@4-5K!=WY9vJq@1w(rn zjU2%(1~H4qFUkp}U`6NgKra{oQ{W66all@T)!+zRD|y8MCUGccz&X0FaHFjU&j3)q zIF~pKy>W`ekdn|bNco@*i9jLvM1S0+{bUGO4&ngI8aPHX33(qnhAhRR{FU`T9{=oP zT)Yr}NKqKnG9MD$ePeBqTATLVA`X`}Igo&#_}Da1Gwed6f81m*qk z5mYm@LTJR1fFU$G$Z~O*m>Bd-0u3$*6cUP2uwYPO;s#*CdI&ng^gt-K0YZQq!z1Y9 zz$%RNpHonVi4DjCF5o+H$YX$Cuz@o%F1>%>;`w{&=Imd7bwFUY)dc445z_R@3^w&NH;fH!7 zdX#~*AtczCk%aoFD2M=VF>X;zA4b;%SX9$;(yyY!E& z{oanh(?UTn3xR#mGd?<=f8pZSwlP+6iE|-Q=qTlbSU@kLl8~)Dm^jC@i1pCFPq7~R zYYnJ=h&QN@q0lRMU`59R7^T1paAW)haS1&gi%N=0@^Ooz_~3lpVC6wUD-!Ja;Bat; z`s3%H7Z6m=3jq1<}hkOrd zOF|JOfz<}%1GGXz2!sdyTmi)(8Vw&@R1zS7i(xjXCWv!5R6nN}j0>U$y`UrVw<{vV z61v|*@e4&M7$qQIL1QgM7NiHXf#-NoTu{xBU3^dwa6z`9wJ_I3Fz$m<8H`m>El`|d zxg})4a0@XC)d04i?83Q!vio~(F(cQHyB?@sEQ|kLiivaRiU9%n!7d;4cUZs%$jbl; zoEsus0@(3KZZU21K-WEh5aJjb!$36g{OIujA!Y-FV(i6Ohq-ouXyHXb+URi(Tk>yH zOte7lK~W2Q&I9hhAiJUckG_UQC5mPqY6(O(+DaZMQXy<;uHjI0Lpqo-=+|Bf9iy*l z5ISsM!p7GBIywMLd7%52AKXG5V{FIP`&$Yj`4P9^P6ERz7#kof(S7U(Y6uPJwa_T| z3wjWr=n)2rFFsK)@5^>MhU>-d|pc{vPKax%fxif8*t!wf{*Q z1>NV0LVfVZT@WS(=b&f*mJjupA3YR(MFyh|4-7(u9@zmZFrJ}v(a(9%)!@f-7$y&l z!QlNGIK$jGazQQp@tOgyKcEyyL9L^oF)?Q#7o!cz{>aF`IsX&xUzh@xqpyHJo-4uC z23$A53K&2?N&c`5!iu&WvF$oV7V82O*}0=qHKe7}zttc^c&{|(*o@-m!8*-#wu^z|{pR0dg4ju-0HwfC@)2nF z04x8TkLht>5Bz|6hYi}ocm-Pue1P>3xRd%B$>?5yzHbppiW&$EqzlD6y01Wy3h<$iA>>dwG)n&gBUBFR zM8DU7QY@1I+22_HOEmry{vX=VogrimG^TQj0$ZRI>KVXNOk7|`8&C#TG#}70upG!@ zti@XOZ&MI!e-%qmjgU2<6>J3jnnIuQpnTx(Kk@_^0^C6QKcW?Tj2R;_9AenO#`!;S z^Y>-HYJ;c%E7pI%JBL_A#|!2j8XYfSJO|eUX#kQ(FF^0FzFuKi!|?V; z3$c9uG5_cJua^JJ{p;HPR1R5=8S6kh7@hwn#l+fwqX8_y%42ZMFHA^$-yp+KZ_L-%ap-W2>04E%!XuYLsf%Rfl|@snEUUxdREKYs<*)Xr8d)W%Vc z98$UPy8BjqQHp1(PxA6NVV!{&1A&j0-KmmXG{hTsY`B>;)JTMfBi;A>^XWc|=tY0T zr4Y1`A2=&SpYBoi3V-t@O=MNhA;~567~$;mF?xvB9e7Xye>h?cWNoBQC|iFiyY!#VXwee(b66 zoSxCJ+I+hIk=~6k@$RiM{0Mhsn!&rxJA!^SNn39{6E|D^2%aG_JtY>LmQ`GZ85xZ) zMaehmgk+*_uO3`kn>=W_SMN%F)XwB_^o#}TYo1fn?R%%izKA}!ezGwkB$JHHX}Gsd zyRbyr;`!`uausjQUE6mD(_W04R0|odXQq9Uv(?MH4OsCD5+n_%0t&XYnVb^cX?)Lo z{NhLH`LUtaA|hx{h%ndj&XvV)GphApVF-gE$s9HKjSTUlvKygCbeSzyGYmNNMAYwSJfJaVmK7;hxYhC(C%gUAF1NP%bd!@=K&u2q_ zjki83vY=p7KRV^J$`KZj^lX%&Ipz^>m*1+?{=-L)nb|T`<#VcqGimh~E1NbdOLnGP zUuG;jvz?kc(>r{@Fygy`UU-~i9&2tuu@_ByqaqEZXGSdUSU0?y5P56ed$m$;yka*j z^br#s!}M?kP6XAt2lqQVj$5`i#)KiV>pV&6`RxZDH>q)VvZyZHY1uy)U%IGIPj>CW zX7}ecN5R8wk+`+P7{`_aga0k3vp0@%EPKIl+yd`hlGH z##h@#Xg(WgQj>obp*%e`qWE(B`kL%J1?Dk=`)X>hcooyzud~_Rsy>*lyVY)|D!7qX znB!VT@HP3V!u{E7_l-iF7k2L(*~(sQj_2lme4Ht=6n!@#sStG#7^X8?tv?+{pbAad zS@kS5a=64l!&2|W`2M`5w15%AlvejkuKj|sHIASY7naTw_mD6-z>& z=*MKJ`dLqX-=E}hoo`K%3Tn@iF?Ek;So6S1@`RjvXHSYRHz`ex(~|^Tj-ZOUOR9(N zm6|vBG^)Y4-JgevU+fEPn>WEBqLA$IJ=sd|VDX`448PrPlGw( zm*b9)p5>+E+)RmywzVjsq55hnT4#LTK38fmgPNmJwzh9W2}x(oiMnDCRMX?QB6u5mkqqAj+i0^->DiPD)@8_#6^qip<~Dv4<-K0#iO0a` zp+kJ8e2%u~{>t&KAtHoBePbHIEPtb%SK`z&9N9q$j`{4s=eC+YGQuLPUiDei=O%|# z)rT4HhG|ffxwuMP?zdJVFmt%0+35Y@S=hsiCC}?(UYur}BJQ~B>v3lO?(@LmHCq!3 zb~2YyQ4?z$*~OEI@OHHsA9BNFo}iM}wE1>nMjhp+*QhBDNm=$kF=#$|Z8q{mSk0n$ zN_$4Euy;%UcOl*rC6@2WX?mpH;_kvq_AWJwG=ExLYCb&4OY7(sr=RDEQ0^*t+F6!N zKbaFP%`i%)v92LZW`F;A^B$}xWgEskyA5m8aXYUU;_zK?%Jb24D*hNp;{>V=nfrVL zDlSY3`nR0xf>YT&hN&nD*T>WHrdQ4HfAQa@yR&**gv+Mr)y(H>95wptgGvI!ax!;r zehxjXIa%5;cs!SVleb1_nR85lTDR4c=nOpf5ySl#W2KZwx4sW6QM(T{OXLg?ND3rf z`5ve6W@xQ+V=-XWIFrnDJ0nE>HEqKs1-ouR+vBo1nNOU_lcPVZ6^(~$C+AfS8M>^n z$<*$W$V#j~DH(kHMJRYnd;Zf0-!H>y$yaHv%uYK}tJbHAq*lz1DoAI&c3ZF z@6_Veoabuy;QQN8zm#=GGf*3*{OS4 znxC4o(kdJ12o!=8igD-ql{TF2CCr9sKX(-ue~e_!aH135JNv6wx9Q9OYs+hw`79c6 zWMU)8a8IgJ6@Jmx`>t%>8H0BNkCS+7jW1_$_Xxp`{YmX_dY+6H;9+j7;Ju;Xd zOb@nayD07zBxJ>CFrpUhN$({qJrPP7Ta-aU7o{@qr$SX~(sSwJm5gV_Cs>ph3oe8m zF&>yG$LVeuBWAaEjh1>IrDt35R_4~p^~6)!?~!cHrqbV2NX9B)2xp`4PsH4adh@cT z)?E*3$LkLt(~oLiMDYk}U&wNWv06DzF6B7$^$u(~skX6EM6JrD zxHLYYW2-jbPqVDd-O*9(TsLfe+^AOkR+$msEN=JJM($=qS(f}8CRM7F9VuaWLo0cY z@xNrdYctkmB)OiT&~nr4IHI8CsF*?Cvo%jj$T;wUt$c(~j(@l_+UDciv>JGyR6C7@ zl!vlT0lSN&{Asf5H^Ng%&z$ZagR5Q(U)`pgmU@7b$g~qfk29<}ZDHXwK6Y~yPuF}l zhAgDQBr!Z2cbKgT2Z9P)s4aeR>$F!yZ36IT9Uq%xr;bDyfl#g8H%Oo-pI|mZI zi96jSPL}Ux_NV+@ahQc=>z9)-q0BGn1EJGe?l0s^8eJ9nl}2$*#4ML`wCto8BT&JXvt3oc!#;Sp1>X zjm-F3txl@eQ^qd5Jt6K**A-qDB}kCU6NMh14b_U*Y;ps`9QlqYVGPF@c&w$;s*G!DlGT&AjU@W-^6mE_asJ^#myl zq^IA{#ijJ&@Y~n)X%F~5&OmbTOsC`E87+dQOc5{E&^pE^%v$`Zd=(W_Cd;V8D*Y3~iOeO20{jxG6`nuJ66=+k+fuDv7Sz z$D8HZ^usEOIb*l-tVfKN>eiF4*BSRU9@HKzoym1<^^_hc`XVbF1Zy09e}4Wx<1^3u zM#y(GU6dnz1c<$6ZvPGUz`nhBJgxRf677}tv%Uu+@K3$%*L64CM5Y(a9f!lpsjas> zv$hvPGni-R&gFOJCQMyPPmidq6?>l%s7Z12OBbVS#u7fif3Ca(skU|`y`E7-_(?0K z&EabtQ(sb^`o>&1RK0QBhJ8z8jlB|YAnMloa<~Oku}MR#!Q$3-V!r52PlZQ*!***A z?M-O;@&d+tpIRjs{ASY?_6B1Bc=xETd!f4i8I~GFGhSaS(!SiLmVRB$D&6>}1j{{qAe@G&|Db#uZ8QZ?yUFIvj6kM%S6ZSUzwZ7Ab+nE@mo` zl2>`g2*}FLSQsu2b(uQN#!KxY(%)QSJC|l43O|J~BD-N;-QQ1Syc3Twbly^GFIKbP zn|n~T^SNj|EfqFho8ohmCwIvZ`O0ELOtC3~lqAQn>*F~%2W#b5i>AuUzVG8^p}7vS z#V#EPcdC#J3rn+Eet5dQ*SaB6JD4G&InePgv_WHaajU#3>RMx8stNL2puMN?i{b(b zngH2zGWGC`0d*tR2GACA%qJY~tJ&N2`xPII=o-+1KlwesmAkA$lSk>-SK_7F zlv|#ZF=G>{+B3NBThHQlx1uA7UGJ6DjPs;V*E^qJ`nFdC_Z(TW^<1>N!Tm7@U#ZB{ z=hag?72$IiljC3XB)~iCX*i5tm+h|wP~NQ4GyQ%pJWA*oGiw|>(OIEsOD6|eMW=7! zUL^Oe%#LXLG@TIfl67}(>r$rpHr!yackb(nn(gv=Ar8EfNii~u+jH~>j!$F{5aM4? zvn0hmEYup&E<8gpn!=y0ZNzVWV5FkIKT2HrHIqg)pYQh8-P$J>^N&)6&R?1k(ohz^ zK^_r29ZVKdu<(YAs!R!au(P7VN_Opi%+sE;-0-=>ds_-~s+MahgdLAj7Za95a~tXz z9Oc))o7p(cKAUOV>7ygTonK6I=kK{e?lG3o>%OV{R(p?oqV$kg#hk(2uTP6U67V36 zI$09fk^CA=+!qx)Mwma{BX~AsDE2mx;D!4%-AxN5@iPI$wZ%X$-)BepoU(A&oubvP zUzYBF;mDZsh9h*x%bPZz}9<0QoFaZ^5!dfz&GyKYUj-Z zb`U7xF0f2ZZs9rk0poj>kpSiIb=`9RO%uv^Y8f>!T|D*hYPYg@CC zFCOoT$UP4{Xd}LNKqVfDi{usV2)XZ?m9(B;N)xNGyx7zfQ)U#!D<#b)66v%TkE{RP zFJD-=BhI78?+LPSDJiK>YJ1&;s@g3XpXz)8*KDBfnqSDBWApBHA#q%a{#Jv0q(S29 zx=sDW#YI7iZawcZJ@UUM)&{x^v&|*sXQHl6g~r`3(#@cm`{nyE`FSryij&4`=tiwa=6JxAmQZIk?Ny_o)f z9+&0b-~;)-V;{6xmS>{AEfiG{l4QHoie-;%eoQz-expz7NB{v__27syjWg%_M7L+3 zpS*6d_R$VBTtYGzv#RR8Er^S*geExSh*e9-*P5tQLox^?le}eo&|s-E(Au>4v0?4Q zV4s8NqR&d)>@5l+_3B8Cl}6}oM?U=B)rF^Auvj8~r%^BVeaEBfo6c3jJjNyg!aGEA z4l=&CP;4hJhvfU>FuJ>XdkUUZBpzjy3C0!4F|2*qHcY|Ld-(gc=o@cFe2da12q{?1 zP$SEXw7w|3*5pvlt29exw;PkFKV?b@-5NgJ8xS{CrmyQ2+1m054>#MT9)-K)oE~gN zLb(|jB&E%1D>|2Cv7jL={58$PrnNS;RV_V&?gTq;vKN!(0Bl=>ow}*(xCfHOQzA=k zYnlN!NmmthO|3iFHr4l-NfMXMY8`F0QxBW&b(70JIYix+c;W)~Z!hcM$x}Y?F;;=I zn18Gyi@I0DxGPL#Pu?+VhPWE#A}udvB!Ky+)q$MM7hy(LcM82JIH0X zjliWBrJ?@*5&u1pr+IJLq#91j%~H3gb#%~3@mLpLS$s&NUSwf%eSkaZc;6N8!GX7` z%Vk+7V0GU2?TSQC>_3~@G(U{AysotI>HP=6-1@rs9$jU2E1X?|y6YW@&y)&{Cz8I( zi|HS0WMCy@*OzI?ilMn=aKxoE1c&*esge}?g&VL{)UF9$q54r{*RxyNSHn1r37j(- zQB=Wt7Z6hF-X{?3ZX@g$Zi}MYN>ZM64Ut;w3qp{`tjwP=p)PrwxW4X6zX(| zn`zlAUG;ZB)p|w{syh6O_%bXQ&r|H-kU)V!-(S3qPSPwYbCS4ZSXSfxV$BBO*%PW@!}@c zbfzm0duzpgKJ|_LH7D`eoOtiSKj* znN5a#SU!C_ZB{y*oI$|6<*XRgCb%B<#+BNrE-if)*Z$$y`+&n+{$+4+2OKHX32JfqjG zPuz7Cu}6s1RwB%ZSd8o0Y565ubE|dr$5RSb%I@A(ZOt$#_jhp_$cj@&80m|D&9azx zmP$2NZ_h3*RvKq~>i7DAPJ%_}I~?3y6*BHKa(xTjjX6Qhe9w(%hEFU@Xh<17Hyq-w z40BtF@jW`STG*7CbaFkw3rA~}vy;LDVNqW%cJAC{U&BON6=~FiDsx6st3#jibm>(% zVxLSl8}d7KjrNub_rl7iNQ&9`&pjR5N6d_PC)hSCc;}Q!7r(|CXR3J~&1cVT+vAnH zYgD4L=U!5~W09h=?JRqF=-b-RW}xB2fMeS@c)4yi!ME}g$loC}$8)T1=*o9I{t!*$ z{r>0b2v3&r!&_jCSI9aRNc?WjG($e)~V_{@K~M+x0Ls- zey)qmQ~Kj>1uyZ=SdNQbOyw4PoEsGKI@u_EX?yxH;VIuL!Ip=)&d(m-C-e66V)ht% z6h$yZZL-i$J%ex);8zgzWY^&FA|dA?3@OZbf2-?3iF(QMJuPJM-C$)0o$Ex)N8S*? zaa%*=F5GT$5}A(3tlq&%fQ^_7FmDW9b<^tTWHCC*tRhN()Xi@5`_NfZd%m}%R|LwV z@%#@h+rnJyd)PGWq+JZJh^;4TN*^C6562bpI=m%X|4DHzRT7ucq7{W}Q68P9!ur}; zCyn_`hX+CI=1$O>m$M^z@zW(N|nfpRL^8@S%k%Y7>k#%Mh7+)~A|wRpyHO5($?4AK1_ zaq$)n=E;Y2%9eMfJM_QD_waZJz-URJqDN3fUAC+a9Z!}XS_89Lqb z)?3O1YY(^RG`Mpqx zzDZ!pWG8gCCWY!khoWs{D_fabk~iGQ%&0x4OrBm@;(H5iwa#^p&rj!yZCqvM_&a#I=KcJ(AW;-Vh%9~$3{tZ{T|Z8Ft5ckwH`sqw4W z+V%IEr(St9OJn8&fL?kI=GpmCUua!gJ3W?PDnY7L?Vm0O7{vnMCHPz1lg z6=OsI0Q+M*CYDbDl4KjOWIZ&s7c;RLwdxCRWQ; zO`Y|>b%oOS>T!RE(wjo+urIQug61DRBX8?`>HBnk$->2**f_*ep=jNFtE{_&^AuGP z*PIub?&m6;xIC9`+eQxK^NT+9gA94oNLwaHJL?C+DWu)4)C2s>BVmkmZoS=X&v`c- zHPi>5WqtT47p9J*qoja`tc)$hGHv;UQbSL4_1~Qsu|fi z7K<*r%uPTib3^um(@2NsGx!?ptqz^5NJq*RY{>Gk+sR^Bz!m=I z;y^{uGb*ZfMJ!u!Sz>Bs4|vMw^jcQs>= zZMxl&l|8ktS73zyL^76|j*0IIZJUNuu!$yxn*>9~i{8SFNJ_Vf(5G6bjQ6tCaFe=| zEUmbiyU(e_X1y8gQkxCkc)(U&ZBO>>k;!6D;mKh#L#Fhluybv$Q!`7C+BNu%MafSU zPM=mJ!+E4>^O2(0{bV&Ki#Ei;=57RSGwmpUl|(7cG!3Cn%-uI#p)m2Tdm#l#QI75Q8Hn^|0~Ulwqk^Nk=;hdHz7tL1IbzMu_un@6*DxC| zFq`GPew*Fd_E621qLuCI#E^dV*h?*;JB;?aJFY_KP7ejYqc<5T;+!|pOnkz4dMHxn z?Wxx}EVSA=?6lfD;ogE96DDu-mnmc#3P*asqi{DQGs@aBKC?4$>ocBR3`~?QaVz_9 zkG{|)t(v1UD&2o6kqG5bxRb=KET5fHC@@671o6aWoURnNBov%8Or6GV6fc@r99$&O-~WhyT|eQ2^lms_g#*v?jJd& zD?AZ)iCOohaM0r@NflCJOLBc$t5hZ)?aOUqn(0kmf~R$lC_QS6GY^8Ml`U{wjl!|i zun9Faw2N-J=XGrAV{_UL4t-hGKJUvmuNDZ_BpZ@ckb3Bic5`rNi>k4C_c&3U`C(hTEKHSW;Sxjt9xGcbjgk zzZO5}w&@iZE<6EQTrQOa>D6JylS%g&rAgD9=HUYYyU z#03{4zW1)aEWrsiGtP6hsFEu_Zr!waq-Jui%KGm8nA)LdGh7ze9W-va4SzWA7}fYy z(ZPh&i_bZ#swk>!I{`H{E891~pE6M!xeTD6{BdG3pcDAYb2GJc$lOcw5I0DCE zq@MovS+>blB6}I%H_w_42a}$Je+ z>6%Z~r=7n~wod1*Hx`V_a={!4?7OXR_?{Me#Y@PJ9K@qLaiVZ?M%h;|<6dS{)=b1s zo~E)$Rg!B}dZv-jr5UY@MaSo7#2(SbDCDEgNVhF#=uV~kQMTl&Y?n2kms##=r+$AB zN21|W>U*(ymVs-L`Pk$w(zJf<)Sc7msxQj5h3_)0Z#omZH94J+Ty$3%=Cn>CORn%w zye`qreNyw-eHHib+iRg=&J2m8HXmFRM4XkV9A->q`y(jSbe6``8*+p*PPkI}2t5?l zi2t?`6fBtXcG20&sJ-Zgzx7;lm1sn2agtTwRgyOS2Z|*cFoKC2?b_YX1A=P5E8k4v z3yb5}+umcE&EHipdpGi)I8c%?KS;A$o5F#Yk~~t|a+|bUzv_A|YaDak>O~7Fjls0g zkcAt{A1P&-l|O!wx;OY$w&~r9x8d^*OC8CfNtWe~9{#*-T`|GWsi_Knftux%RGZ5l z4*ku?jQitxW#3(otDC=DJ7}nyh6!(wrhD2 z!Hc}kS3QogMTnVTm*i9hvb&B=y-s#-Zyr5kRTU(5N{BXCGdGy8p>TF~(X4YRqfK9* z_&{1F&wA}T*}H_0F#equo|7wf?18-GT~{yM57x}wViOK?6NGuK zb`6zH<;8ONz4O?flkcy^ch*r_>CVHqqSd*zXz2TjkzRr8vENs}z?U}P?iz&^#z$GG zFcaALJiR=bk1Iu2-YP7Sr0ek3n!VdU zGBUSX{VvbQidZLz-nxOhMsoj-5<^wTkmVKYMuR-0ExFMM%ff1`>()^fSLa;wgy>I` z?Ux(m?{IiJ9WWRMRd0AM`BB)=GRf>Hegb6({3#uQ%8ge78}coa%lzw&Pf-)otud&z z?JBXRrl2R!!k6!X>PeZZv%4hA9PS%PCQ;}2 z6sL3Zu}wwgqvOsC`)&c&NFt~9Tg;QI{tAgJsc}_OH2wyOCo~aCenztFGA9putio^He4I2(iPGq%rhrmf@&6Zv1OO0J6DFQw4EVOFLm?wNsvCk@gwU261MCwHqDC#|fHmpRR0D1QEJ zR5@wJ=ojSyCY*QyYN~eRCQB%>!-0%=|8N7()vVe_K1?TSEv3eSv|SGE&+JpouXqdA zbDZB;tnr_nJjEa;Ia1{%r_KFRnLMETVpL*vid5P)GYbl~FMg?h9YvkXsFsNrrW0Em z%6WTtOdT2&g0$Ygi<24;s@aj|hyF{BP5;tY5o-VTi5EN34}>)$eaja*H*~+6Uwjvw zRjSGYe=~^xKIBtdlX8JEJ6_wl6S>}mnlg=__b$AUyuvUffK32S4pb7 zR_k8|>Z?XqIu=hA5aBkahn)VJEw;5q%b<5tvdxaDz@|sm;D&2F%{yaz2ECbCDaE2q zb%t&aIE&Y*iHjX8yX}Rw1t*TAay9GMyw~?vc>S2}d7zBp^W5yPQM&PD1(k*8j(V~# zAF=|Ysf#)@$gj9fB!>(chv(nW|F{`&{TX4)EG28*$1Az8vvVgd-`i5Uo;~|+?#ODc z!YI%Fci0o_i*)|B7mI`Uj6}UATwr(Z5eC3UUMk(8;V-W(cX>i~!R~R|iQXx@(Yv-; zH-?VRSwuxT$_CKpcMrTdizJ~S_Tm+uEZN^=46&b8JGeceC>55dKl7W?4k$p*Ey3z_eaw6hZGrLmyy0K1f zsb`Ofrq|-|<($-3;au|qt;<&32O(PB4m;xu@A5JoW#3W1_VyljQY3YlS(?v#{Oz%x z$v`3rp5!=O@Z(tJ=huOuNTDR%CRBE3sorSJggdcXJVDEZ@d($D(TOKincIq+=GX9) z50dj#ZL(9+4~`$pu8vUdbf46@v~DMHW&XA@^?|7^v(wl2TQb)SZHj$+37MYqvh!36EFlAVFmJ%>hmuiZIpIJS8!EL9J zI7zXS{aT1avw!6-VT5_--Ai%lPAB!EwZC?64nJ-)-ktL}I~ipbWENEIR~r|=F}nqS zowek4`n_vm0M}01A)c+YXC`e+=P6%Rvax&!kBsIKy8Vq^zg{&b=$I3ZExvpvPmz7P z;LV42$&ptsJoSQoO)||-DrF_!lpBaB==!F`O)X$6%rL*S+jG@CM}+#>@okEL(^8K$ zlH)(Emc+2(6O`OFd5g4&=?D1e)_p26}yE@P`kBUeu(O`1nFD~{WRa3jX3F5(u3K|Gqc>} zwq=9TSIU=QC)r*H)vXJ@JCkgMCni%>rm>ozlqw>_d_LnIOHm!(4H|PB^HT7v^L?-xSFJ5HlQ>uQmbwUgQ^oTGOteXqk0nMy*Fo$M92~>SR6bk$;s?f8fMNv z>p&Q7RnN(jtyE&`;NE)VeT$U2!R1}yb}cTmWO&!i&6FG8l;4`%b1Xe{ zk9S;l`LLwMr^DXC-o)gW%8rPgf&O85OUt0^!N~nD)rG<<52x*4e5EU2niISnRB_Y? z@B6oL2};`uoC1`pB5MBHJFZ>bXBMkR3}dh4{GJ~4(yqDdk-|-=Q^1O`~6_gH{i1R1LOg#)5 zp4PpK@huO2-*-f7ZOBqbL=rhv`v>jh>Y@ z-qGIA-Q5Yi@1G3_h&e>u>QY?l=nYjhGL;-LnoFxra?A23l)2+k~ z#g|&27Kf*$R;pE1^_M3Mwe2!s%7b`ona5yHT#% z7B!NBkG-iBw2U*ZejbS_cU49{+>TqAn30g5B%z9SO=64I2%}1tDipWa_G{7#xTmg| z$Jr!380moEyIEVPWntU;GN6mn+SB#&U{l1`QL6eu2bohJwz_9#u+7q{!$UC9bud>H<6{}i&h zpz|BiaATJIz{fm++8jQvqG}P%+2_af5A&Vv&)VK!P5lt(w`*Hnr?@>$Y*2JR@DpCQ z>F$H0YaSg}$GA^=U%VSG$4R_-#{XS)&tNa5AcF5?WYF;exkQ|iw(x}hSB9z;H?J(` zzbtXFC0<4KM1PMhpQqR_44MO!-3Ja@=8{jWX3vd_-3L*2Xt0tLHhJ;PaCL~T_Ph_nxojrEK9 zqMc3peBO@Y6tAmYT*TC(g#nuu@}pt~WjgciBU>Gp7f3FrROXGZST2X(?&jlW&OA@? zLdVZL=~8*xt2-ji9}04~OI@Na#FYBvHE#FL&#ei)x)&=pZT2i-dg{BYY^ZaAPXTXh zWlg4b0OA%{EW`CRs!@2jSvJFmY;GdKK5$I#l6 zI(zF~bD=gyy$6m4WrC$K$5%~FCWS2s$(*V`liZ;1m=m0CJBxhVA8}xO|3aKYg+%)5 z1*apvp8i_rz419VRDCFOmMN;`zMJ|V8H**b8SFjA(wmv`;P5t;unU5oaQSMlB7=AK^*h)~==5v%iQBSpct$L*O@hI=AcU`TH;s^p^lfu&3 zCRu^$*D3kSZ77b3a>qb*C;c?K7=~+kRprxDz1**-uh#om1lDS25k(5$*|}|PlvFdl zPs4b0CwF6TiX+p0!GoU4zUbmuyoG&wP*g1cx0D1cRRLQoXuSP2K%MDkU)1s@Zjt*` zIoWpd7asNv1+%sOZFTqEhbM1Uz_rfg!$NJ1G>*BrS6s!{zEa2;--kz-vt6-c!JOKX zF{R_ixh5k1esWUE5G96W~AJBq+LX|iE@P|#iax=mVo+i3g)?@V>BoKn@Q z@39#vV;g<-JA-S^vgH;9TlvnHcX!^}Ne0VZzqXOKeJsA@p-w~room)+J5cq;AYh7f5;>+@m#7IDGFrMe1{{ zBCf~B;0~Uav62(v#X2`LG_PGRAy9gmN#$H)=|Dc*zjb@Xn#3u;C~oK)Sz| z{{8L*Jkh1#V>bt>>2^_d+RhL%#6E?K@yki6-dR%0_AugwqA~aK2s!yXr-V^_zSvlI zsORDBQhbVE%K*9M)Et+Doa|MxD&~jLY16ei4+3*UHV0rS2a~XvLnSK7I|q)p&E!R# zEmnhlT6&3J%S9e)8u@NjQ{92GmasiVO14gyg}GGlRL^1!&1tNoZBG;8ER)z2uZmt} znMA%Zb(G#)Y|p>vmcpYwv#2+RDrzC{KZJFfly)#n&2nkEaI=_{<<~P2rzSSmDW?`5 zhzW_lIb8Br6CL$+v!)JM<06NmiRqhfZFOsF@E#=&rxzNmK)!lt<|?|PhleWM8q_&x{d?% zEx|cWOm3OyVccY#|uoPN6Un|H5`I$sQ~JpVdZ&enS~nz$md&wkjLgx=rhYWHf(Q# zp0U`4e6jKY^V%n=$6sg~Y3BeIJ7?k1`gh5f4fToNFTTsZaWFy5a?ANBn;84nG6s5P z87+J2n?=PQDFeBah?tGRs)&`o3SgZOua2~A*pZHdvP?lkw+Az??T+N%JrF~}-SX;8 zpPE+enZu{rM5u80EDz9de2i5@h}|AWz5w*%9erEQoyD%=TWkGQ510DDPd;K}e%c?! z$2unoaZAnFze;Jb-%Svszn!MUe!tM1dqdZh`P0@EB39J{jvmA&UpbWXpl%7H&_0QY z-5V)>W@=Brr>#%AVr)+Pak-=9j!VKU*%jh)%)n>1&!A)7v&2;EI5NyWh6=F=(J_-X znb-TU5#MbO!P7s_H#eWCnc%gLg-_<&60WU&jJf7hf&qmGE7hK;ML4+S0*ogD_I+k~~I?F9X$GfHA7^|=(%sx$scPIk) zJl2ud#tE@I-w2IyFipY(D(tE4DrMFl5%I}`7ut^p%ENXum6l|NR ziagR%icd7uvd{B$=wIs`#*vN$+88Q(XqRGh&XyXR-Ev0Rfe0UO9fO6~C&8~AVsy-j zo{KtEGax&ImEngfVg|5g`Fmric&`kfzSURs$~I9IyE|0=$~gszIMEU-_QoqB*80jG z9P0&KzmAJNG!XMWT4bV40*=@o0SCM0h#7tZwZx~Rm7B+*uTPdK2!QvP7pt{=kCu(u zoj|@_`B3!GDTT#&4K=mL%XKy9t8BvQ0tIrq*3|A>>6V|bcZmOcThMfF#1~pb;P9QX z%9nNtHWuJ9mVc2)-4z3vA;G8kRFtYi6%FK9(@Fvvz}w@XSce1}GT)tl&Dc@)`_>RT z#35oASSLtHTYZQi_cW(?f0B}Dn*`Q%9Qu5J0voyf5%xsWRDN}903Ug*6yPl**tcFe zIn$@A&N~nhvM$v4ybCQf`b2|C-W3sI)(2qEPtw z^5ecc3fv#BWtOXlprIqIoS%>9=%CN@)bfi3a*hu_^6d3 zdeU+?;_-oy6t_EqdAlb>CGJmRqW8zqp=&*$yUSgbuk1ob=Gq5LSTF3&-$xM1->G;7 zznR__+_emsKlRMw(`+(AqO`f_!4N*_r_(^ryO5Bu@ILSALR-=$O-<4zV}17Z{bBIy zV+A2~4bTxHdg8wft@-y(7a53^{_-c|l%#JZ?2I22g7hDTu#x{V4^+Hx%gA{)fK&G; zaB&tPG1op#$2cXVd=V?+2QlmAPx|`sA2)ikZ+JDNOsAY#y3tR{+U-Xr{UFE;qM75~xAwt5&$KqSN!^L-OV--*I4QZF8%-Ej*t^9O* zsO+__CG+RqQDlTq!LBfm0(>AugzOAMgSH3D9@~YbFE5N#z!NblS4UuBQ{1c{trBYf zk&%n_Xc*<&qp;wm_i4Y|grFejMCEhe95VWBi4N1W=R8{YSQ`C#iAy}z5wn0=EB0$} z>AR!(54L-=ZrUeHf=-S2B3&0KyuLF0cfXo~_+!1M=8siQwMT|avP}RS_&)32I40=t zHVG=arXu`u_N#0EoNFow`D=|!`?NrZ*o2TUEi3Vggc$L?mY?|j&HxB_O(uMy6ZCLx z5EOo~!X$s)s%iAftMiW4)dfy~OBXxSuWt<(+&^8T5{1GdUTstR$wE!t zzt_0*&qg}VE(AsI4dp&GwWVJ9Yo}?zztHsIV2*`8($lMs7s(Y~9jU^nAr@YkD4

p zb%=0pw}(p~EVifqywIBV^Uet9kw=D4I8zBQ9to>xX9)GuDW;`6C4~3`5$u&iRQA*> z1IO%+f`Sxv89&X{6#eFsvp^dI(3d;I@K=^`c!*PiirOCoKi}zvJlP&5#Edd?uk>N! zzuoLbMz|!bLWh`|u|0we(bVN!mQ_c8yW3ZBZ?&HgJi;!yiA;L@|H_zU507U#1T(kv(blCG)r?!gSV{LI+Eb<9vRof1l>XNCuIP0-U$<*X9hI4W|M7x(=n zE&T6>=IkrLzK2^x=xA+o#!m-Rn7064XWE2S&-4wcSJ!*;p6?4QLO1%#?yYoJgq&(S zy6mEk`o(r)xS<&xvGxHLb*d(0A1aV>mMPFnQ%~u`K79O-AJDNse_pB=oCI>LX5nvU z8^f+nkb{2Q8%(?Akp|AIl2hq<5B_VeDfiW}kpZ?#(9u)$#P2Do55J)%Km3Of6Z22Y z5E$STO2Nh$EE=46>zkEc@Y_EC?S8DoYw^|slQsq)NC=o z=(@HQ5_GC<;vcJ+m2NROe7iUG>J;tOH*$X3j}l(m#x2LC8{6{lAM0sV$3{T^ zO~AtT0Q|QJ%OhQqinxW2jB6rR(p8_Px@b!Xd9%`;^P5!&d9^=^4!4Y>BHRj0>fQ`C zxu1~sV|`)he*wLm<&xuyHwH@{Z47`O?Ew4i6eFTtvnT++@%eTsGFsD+ab@{K?yGayjsZT?eJBr+@-lxqS>Q3htk*QSWZ>BS@!Z?i zv5K(05m2Z{3QhE@5D9;6ap=k}P!b0e`orQNGVRk&r{>ty*yfT~7nny2Z>wrzf9R@u z@}0UVwO>Oi2Kc?es~{xr3>Vzew?u#2i+=Qf zhA^Sut&QU1Iw)D+OSt9tcgLwoTcd=CwSL$$Rb%mOWnKPNl_3A;<<`o(dy|v|b!){V zU1Q!gmyDdbKT3|bjuPJZW-+lqZARE9VIctTK3Qxlym}x8e2$csWt&1ptn{Q^Ha5Ne zR>MvF*J54vW#Bn+i^GWUL1OGT^Y5UcK+TqK4}zbYItqTDpeKC)4iffl8|cN~WSoqj zR{G&DMYRRDCH#V0TO;U*#g4Lj%9`|FOdYwu>sktLPSBIT`v{BvyS@<{JS`}EG{KI$ zygg7Ed?3OkJEw?gt{FywO@z;INWd}Xk<$BsmW$pTA;nJ7a&K?yUGRPr>yj0aBUz#FCd>81qtpU)Bjqc)GquBUghv8wD78{a(^@w2+7dl$$k(yb# z(T{y&XaN20o@{4r^bnJsB2>I%x+-L;J@2-uz2L#_5De&Xdd|gCL!D!`I(Juy4L1)_ z66_=BaQ9T{L(6c{gU!C;d&}MB&+K9r(k>OC^c|3U8v_BINy}g9LcG#)^M71wskk@a zR`PiMeZg%*N6{mvj9%rER_AZ_fgY@XEPJ;35fZV{Q~6}QJO8G2G%!D{C}h4h@5V+i zIQURqLsB)C+?wO1{{rCnYoNzNEaRBByCbMb2jDGCeV}JOn3$g}lY|VrgpxH*%eY)$ z@Z$gUK_dSyp(cK>1Mp$B4;*QprWL$H$6ezTKK#F@1_szOM=dgatO%avCSF(aQ-46iT~LLs+2?%(pz@VAw?#f zF0{6v&cAE^V?$7TWW?tH9w5RZhKFrU!NPn-F7e zh+lS1-2@Lkkn*dxg!nh}-G$f2sIlLSkfQ&guFv~r{R8s#_BcIxI&cn)n|1j}S4(yQ zwQU~6h0nKV+*a1V`FW`=`?9IM@Yj`2(7hcYHb&Q0_IPs$6}3GCd%oIUaCNP<_~!lq z{Dos0A8+i=zdcQR`^{PhIMg-6Db_U?TwdrXy(8k}-e`fu{5XP(xwg^`4%wO{#*OlF zu1~RYf7_iPrRv)&p0~rJ{?>yC`>%0Q%s-|nasSZQ=U+ATax>}?NtfQiqkeQs=+GlT zV>m=uzLw~UtFcgfJP#t-nPb%o)7GrX&a_5hx9v9s`@c>vHqeTBF5 zbt%7m0KW!aJL;#^A<)w?FJ^sG7A@Nsh zAk%7?l+P;;{y|8KzjdgjKs|HhLa!7Z=b1r9A55a(0DF~X=*qvXYAw9cfyulzUY&N^ z3+M_!+rI&LD&kZ{E%r*OB^$l4Cg4{C5fd?LU^AvVWUz%)YzSl5@u?E`9yyHXXAy36Gx^WMA78qtk8j267;i z<#>Ts>D6HpY+}f(0~s>WvL1^&VQcG9BTbqZl zVVb6lUo}mMzwQmggI!`u)~TA0+m(>B<;{85HimJ@d((_;Rc*mfGql9-X6Q*jj*%n2 zo1%pOO~p<5X|Ep>>y~j#XZbl-0&|KSGIsgaAnC0Q@E(BgjGXTVKU(Olx?>q9Cs+ZZ zTk0o8Y2Q=g6^+POOYP9-TmA4Cwh2Pq<_J4o--&p>(u;m`tZ5S1q=W?9BsSVL%>_Fr z`0(99bnsSx`MveNqWdc!0pH(_4m*%=kt@T5^mo9%^ut5H+4@j^Z?(JZk^UX%>E<9N zdUJx9JjBQbup;$}TSm$JX~ zL21aBrRHvr0-v$oUGUq=`@Ab#y`Z}reN|5vdm%4ntrd5xQl9+{k#zs^g`Qe=s;w^Y z%tGJp4CFqZp+;Ql13&n?f*JX(Rg8{)hf2Cu9(Ct$ipHYn7kWC_BQAg8lN7!1&Xl~+ z*2VrjiGBTl7vJUG^r+a7wSMG_^*-2Z`wSrm@XiG;2{G3_N5}%|BzCE<;%Of)>Dz8p z($)D+_={0i#_y|x;4q62nQR_~Bp<0rCFg+*cQqGbokqtP+H}d?>F+;+oD?Z;s@w+x0+D!X@f=j zbDP8YbBoRVv_?UDH24Bjf7VT7SHfk>G(2XmIX^u`mpdwNuYat%w>1KKpl?n5#ZaGc#W{+NIaKh`t6k8i+aKYtPEgjmlM zqsS#9=9;=no-DN&-}cT?@?COTo@E06W_J?vcD@H3q;0Euw)O$>`cx~x?@qG{S3VMg z_l1qH%?W0tdMj4AA*}Q~h~h zrspW)m35k4bf|e(=b5X4ulHgj)NP2DvUkvjA4xgis#?K-7N(Y4Cd(e1#|s{9kCui3 zpEt@n0(t3`qT<(vDuTtloGT+he*(G|a;za2?DS{fS!js5zTS~`e|xYZY-bSk!Y#!l z1!nIK)02Oc)fI>O)ol2QuDaMG$0r%vvu}=9hyO6cNxPwHF1)L41;3OxS3Rysxc}V* zC+Xp-2AjFjU2u1~zC!OvfG=($k!V}+?y_P{4eie!QVT?gcP5QS$H^0NIsZ`#%v9jgxN)v z5q=da)~_RG8$Q6^0$X_9Atq-WYcOd?Ds=AoLN)AsnT4^4$f??Q6%R%9oF5M)49LE; zI?X(di(Vh9dNIj}|9zC6bpJqx$vR!6Ri7@g>enX_iQ}B4s}>2e;Lp99#xL6(npFag zQ#B=BS?J7sXq&Ehb7a6~n1-Pdu%xH|;(@{+U2Hb>1D>Yk(>%JsuYt#|4j0{994LIU zF#+gEDI*`K%a?W$E#0GFm%G#)@b)M^+#w~TIHsWCM{;a}LyU}G`IvoseIWnddT+%8 zkAhihouOuL^dVnZMsaU|`UCtcIn6&uN_PTSW9lw>C}SmkxA_75_T11UaLS3v_9@&O zhlrH4)CPKTFip;KD`+JyIkPA*LuaX@{N~;`Da9@0lx+<{gVs7r@60!+|El1|e%Fr* z|EH=Z>xNrOFWwVU)7N{^K_?4*l26a5@TjT_oHBHxUx`fzyk(+&iWut@3&2Z#l(*(F zQqrlOi}b6=ITs6b@Tr~!vxzX#yQ84Twh71!ra4P6s*aBG9a zBtr}Isfd|-HIRF>*b08;Rq+u3{-*3rB7%<%T*Ap>13!>ewB8STBIBq0G)W2l#x_>? z{F8}T^mUU>Ssp^Z26+92e~t@3pRebiFW2%emT6$G8kx8|R`S%`U-s}s$0IrwOtgfb zecL`wN^>gcQ0E+_#HV2Zdx=ZdzR!B0Y>t1h)RXr6iIGr#tS1z$e=K@v9)hMGX;^rl zii7Z|xiGhpkJt1=UVgyFelO$Y-?B+Kr8{GoaARBHZAD$?&9x8Z&jCGIc&K4k*rnu5 zSySFEWm{peN6CShNAYjXgOv}AtvSDo88N>ucVyo?HqttcdVxH<&KzD;nCFM+;V2g)&MkA zOiQ{xMUA=QQ4zBIdIscpiG@1Qk@6gRa{lQGljJkdP^Swl0>GtZ+oOos^$|jh?jtU8 zPgY%VzQ&>Lj#oym_hw!98>tBIQcaD2t-i%E&mnEe*kw*>z`LO1{YrcWzm=}fN01>NDLLCa$EmPS^ULiLE@)*K6SX>kk8&$p>O88}rV~RW|Layw z&9R=CXBC29Z})>9TP5UzH3=6!C@6a-#%YuA>DG-mjsT>^+Ie}TV^q0Se6*nx`h32<^p;OS$OL>^ zxJv|&`np7ed-eDds|*ok5g|hZd0S?nw^a2-*R{1dH+)JQ;0MV0ZUrItc!`1cEOLpx zjI2AO{L+w(NlwM)C?jj>Bkq;H1#(};%Dt{>%zu1gV&eYT64ZTKW0UL}eEPnUkp7od z(DG@6%{Z9BCa(c}s0Fk>z~|v!BO7r6coMIcQnbwP6RH;0REt@jt*vrS;*j^?OE=Q?^t;2sjYoKxzatAPRD zC3?O!=d!Lb<7e|2D99>ChHniaUI1Pvc~?l!RJVcdX_^Xu^~iYTUKy{_E8&5xqr|vA zbkwC*(2Gm6jD(-}hrmz$S{`bxANp*)tL%4w;C)OS;@Ld8(67R!ZH5wihIDpIu}9gcbDfeiOkbX2*U3ds^gGIpv&krM`*2nPa zXY+Mzy9gd`6;?jCjFjHr>??cdQv$lp#3LQe3n+7K&5)*T*#ilzX6_JY{GVkj;(r+w% zD82J_Q_y5rR#zxn%I>cX;G&KV0@k6PMLE(lu%205igmIg)GI~Ddt{_kkD-A#-H?A* z-dq^&RdJAh9T$11rI(*BGI8h2Jm$WHl(IUE3=U)|A1$_a1+M$J=c=<^GDBoy~;>mJD!!A%O zrlIQCK)?IvxmdS~kY<;_fW9nyW|=0W+SNRSOVh~M5wT0Ac;#2MUBrm(={oS<1S91X z(8EqCJZ`POEXXQF#<{e_oRjss_eU!=wT7O`V0AO-9(r8SoN`R8 zb)x*Sc`)a<{c-3^pR57nSGEc4k{bBh5G8qMlA2?alQQl7^5EknOORFkrNYG$QX#wgqsGcs*+4S0u|Q)Qijhi?v--ILd*{;X(BxxGCKi}B3S z5f{s~T!$Q+>;!7bF+)z#HWpsf)+b)t8!dmbKZOWg=_`M7s3RAjF0n~oH4Elb*PxD! zJi^vU#dAYb=4C@e-mkj))XVE#8MjW9n2cj>En&4EAE|E6zYKi7mp%n4^I#Gl?wmo! z>`#GT?@uB_oU_&GjyYbHrW+IW9ufa9r;MES1?ZnZo+I!b5}guU&em}8!|mamo3?34 z_@`9?*9+*2JuxKsKn9PuOW_G-VQH{ahDy0u;tRf-1zmO(Az#;Ce%m@#o#WLoV1QqR zc=XgV%M3QrHA~73%-O$K;L;D~IFt!?@hxQoD9*3q5zOPL$mQmncZ4N-fPga|{&8j+% zvZeC&>Hsp*uLV4kkwZB&a43EQhwRsJ&^7>roGNPhsj0qhZ-$yOV z?6@%32yB2S@fjO9UVUI*6Qj%~frjl)lst6H5RoT3dg+Og1#w96@qmxdJ_7#lp2fx3 zXAlvq{RK~!yRshc4I)DT?T`Cov;N(okqPmrXhng!mQKL4?2bXhR=dkXcSjMawrMuV zGTYr^R&=*mWDQvJBsF=h7yfjiCFd6@J>gp=H~Gqm5}$jv&?wjyH)EGa`PnO@?6j>h zMv`TMnrIm(M5~)JejR{^{!Poyx$YXnyggJ=^379(=!a96ix3szmRIKi92R*XfrYzgD?^sL(|+3;Eq>}%0~n-h zWNb~6Vs<7lk$aP*Ov@BG!>=V5ecj;DPv==U`y?XT*qU*DYp~)upbr6USb1l)2NHa! zVpaq+*`b!4yE|SMwcMF@YooXDz5~G3wNYl4kec{|f}eifJ44O$h;T6nqgA1o&mq zTBX$7)&8;v%RQwJmOoU!+8M2i^eV6g7fVdS#hO6i0Pu5Lf{$}+-_?3FY{Ze4Sa7Hz z=53FHUN}{VM9T~;!Y)O{oi8zw>%-uuyVID2(-l6=Z=h6q<}hiNvGSnx4+VFZy7GQM zkWllEjSakmS$dhOvGA&O3Lo!NQA>O(O!6PA6lh>w06m5c+!MMdrI#G7Hg|7{sM(8s zxQLBWQsPQqWr$Nk$U8L(SpNCOrZo{QcYTr&>o*E${&^N+TL=xaN|5mfa$JT_M+coQ z(knl22x`w(8k)BOO>!W|q@<8?ywb)~5YBJr})pMJJh|L)U9+vt&rOF3F$Al!gnIa_J{_@8^;G<5`zik#_J zVdA}Vbc|yf9S7*yIJ>Com0L#4I5X5y4L6tkoexj(}%-JfLVn@4eP zcK{ByjX_>G#E?j<7?-&#ssnF}+HmHXR^q;xS8ACc$LeEPc=y2Se~z zwsFMkol$(m{tU0mrK&+WB!om$7v$wICH~r2b>x-3q1?x(i?k{mpogqddbUT&N4k~F zs;#lgM;1}l3+r^ni;ceAhjwvA*y%jIGB7i7Q$$Gwc)Z-J>F9JTYYBk&i9InD!PG4kV2HLmeC8kYN+-5@f`DTh_JZw)C4eVP&{i!-a1O znOVBlvge!q$h2ck4bCQn2WtfBzf99&ztcCQ{j6)sxM7{fC%aS}h@unnLfupmd}d@K zT(i)~rLK%?XKt}PnMV@pbp^&v(PlJ8kKFCs(R@K zG=tB`L~6V8AM09kf45FRBW+XAIL{nD|7?*@J6jg80qv2ceP4dZC1-$a)A+2#j+{I8 z$;wb4fSrC7tMW+C!JI8OygOcPY&FZcbW*&vUammT$KEO*K%o0+L z7ufi(n+eX@x4OE-qvs&ECb5ZrJ-PH^jZ63I=v60r z7V><7jRp1!>;>@DHiHN4iKykPgRobdqu?i}^VG`!wzN!qT4xbYmS|O9x4GOeYjpC7 zjs$4!YM5P$&oXv{?yip^-uM?-*u!Ne&TFVHJ~UC_9zC)AWQ|{UzA0!pF|kNK9TjYz zLPq)L>5#AU4D4T9@A^)bnw#C)CZ=6R&9IB%!K?j6w}#0f-wYEUeWT!q|I0j59&|ci zLvhIHIlEKj_-bP^;W)e$?I`eacvc#Uo)&z$WUct+Rw{9WzW3wcU-T$ z=ly|10JDsv-|UU!A{=rS$gQa*?275>x|Z^LIzjd?2V=+(|7>;Uu|6;tnV992Vq=|h z0b*r@61O>o4K@#g9@s?iP``!=cq2aP$k@QN%c;5hBFGDm5|`#T@NiCfb=LN9{=@B| z^!u(^P_Sc;mhaQIb)3xiOuCgd*aI2r?WYYU{&awbR48zLP|c;aB>dD1`Z^~Hdt$Jjcrj@!C^pd3>;5#-JT$!r}Jy10l2U!3vK2-i7QceM_=r?jO_8CmvY;E$5x%#+Eb_w*&r*%HCE|ijCYRXUInymYP=7x9yuj^K# z6Sl`HUTzOp1pz)h^iWT$I$f@--5SM4E%!i!yt8!ZpR4sv7eLP)sxjHtNpQ4X4vVvC zIAF7a4_=*QX3X$Q0$FoW7e*TVf3|tNLp34e%t$Id)#D4chf1EEEORM`YfYW26S$Q5 zzTyYR^K>NO;n-izjlG{Y>pPE)6hQx1=N{^rRi8KN8r*6Ya&rhCveHw0Yq>k`rbC8L z^=OC%fd9gMHF3y?8hnvQ4NLg6PDdXoahdyLsAOO9eg+X|*Z=t2@(^mWRAJ+O0p83EW!0IB8mIGit7Z30<|42(IIGg7neHt#*E@Kr~ zXBauAkB|qtmWmsi#`51)b%nRqKR{o*1NW1v13BsV80!o@Yk!8Gz1UrNb50QbjkYoA zM-#v+yQ8Jg08dtRAQgaZll%gY4EQ^Z2;CPJ-tx^B1ND-9bEz-ovbsI>w~gVdNT0Hi zZkN`8_opa{P8p}frQm?va&or!Kly~1=xJ{B_rv(7-yVoiX{QSfe2>1KzBkRzbpVfb zi1BgO;i~6HbNG~%p7dMBmZV?Zv*-kungf%!LY^*uK)*fFaG(Gt!!MR-@M9w>&#l2G zE`H3uYU)b88o2NKbh)9{t*#^4Ww_*nnaZ$J4L0jyfrCCWFrbH8TKQ^k`BNDu_LpNL zvD9OrmD=RQB#(laZlYd$v8LB~3LmQ7UpVtY9Knrn`j2D~5gXrZ~;sjZ{#h?se9RWsW$S5Ne)*;U6z z7VGw*tx4?ydsONk+LnSE>m}vL@a&) z2QPmtd2AQMVvp3=?0=hUntT90Ta>kEtCCxJVrp#m=~-ZpwmRQ3gN)b{!9sinO4;Yl zn%X}tO#^#UYJsi`{=_2TKt69Yy!&rUZO7LYD#9nN2;b{VykcxfyS+YwNIuXv*6Rle z399Cj`(HK$4gYIz9R9k=t^H$}N;pzMGF{UJuXd-39{EiiW*~oiV+xl$)0lDf%U;8W zKlfUuzHGAVeJW(y?kM<~PtS(?v^<1M!L4%08HMHvQvBu^HDN0-+j(42XqVL?yxKaV zOUr?5P9R^e^@AVo3W?DNv+QECkQBexiwm;>>wc(ZfzM1_0>Dv)F2HBmriig_DZ9`n z#>eal!2uoh$~jZ{%%eia92x@q$cFf4SydJxJZ5L47K-VWgL z@&|B;eF7E#b)JL&Y^q_p)YPKQ5p=L+niRJ&R{7Y}k@bsb1{33z;*w5Ol)?*R4dE2n zV~>=a;hUx8eqQF&9CFkfhZN8gDs-G*Pt5jcNQGyM4D26($MES`V5fpzak$Xb=wE28 z_iJh}#|9S6Dh0oC$YBw-8F&ofjdLzm*f^(>l6$15m3>`d18a;+_Q_E(Hc`bZzmfoW zZyNYSTgyH+w0CWcV2`BSRyjRVDIaMss zj+j>DF|@pMYMMD-4IklE(@X6VRH9=Rm2f=IfB+oCIMh>0U99 zd`Hi!w2dKr2Md9v1%0zTN~r|28v}SB%7uvvbIQuz92&^QoRJ^`C5)wi%)mYqGH^$@V6%hWac@* zorlYfE$;c|#w{_scz=dk`0u^OzEA6H=H3K2$TkLkx!RNe+9E87{cDR#`QL-OzR%k{ zj&%-~yDkJL9WHSArv9pg-H9?lyFfC|S4a?GP3dkmA7Pbo%7E8P+ZaK;w#r$>HU+=x zKv|3O8EYwSodEApbD?`PggEmADr5twWAg+q+A>W|HIH*L-Lnno(}g-Zz{5PJnw$mT za;$ZloM;zwi|kTfsZ~tRu!%_tb`dh_&|lXWNWYd;;E*BTp3jpCPZwx_HiU)x z6^OTP88qsXfra>=wWhW|O)Z_L#s)T^<+7bx9%f%vgV`1m-q=O3(EahMS3sXc9?y{r z&kgmAV;vs{aAlc&wmR+HRLk}SuE9)!Uu_NL-`*H5c_tCWT?Rb)YmbVH+>$a1*GH?9 zj||O?XG@Lsr$$PdPmM|18Y_FQY)-o3kd}v@nW&IY8@09m1wQd;fk*wc{+~0qAh!Y) z?Ny=^_opge?M*;J4vjq0`Fdmf(Sm^Knqz|k*MmPT)liQ$jEX}QDch^WrCMhyBDTj% zg96t%{#a>fx>%{L0sL~IQ%)|~orK1j$E#j{Sz(he);V;GhMYS;3=iIsvJmct#zvp1 zj^_t@dx0YBrZ)2WoxAB@s1D?ezsZH?$l69e0oZePfyNwsjKskfIhhhT*KIE7h4n@ zxKn{kJX>NQKCN@e=j%-D5umq@R`_(^A_oI_8u-Z~8~KO1rrtJ(&ogvZ+;-2g!GLFG ze%;|U{P!M{cQj7`xzzwKo5&@9nt1|EH#mivAM}mBrzHKa{rZpppOeY)sjKqk-rCwsLG!1MV)t+Gl2*I$o~9E3x`2JMP4QR{tG zcb7UV?yhu0pI9eJ0lifQ=&0gjz&oFs1k9sFJ~5C1dZ@>zc@$^>7g13zpa$IvX0b=j zEIR@A1HfR=>W9p$ma*)c)`^^(wyC_w9yvD2Jx4EcYdBS|IYyyR&%?NYI@}iKKeSF4 zK61#RVUAg7m}?f3aAM$-&o`QT0<%WV5=!dH5|3n+fg`Nak|#b59H=o&oMjTgA2I0Z z*#fO{SIR8f5}~6290#2k$c6qnXzb@Dbk30$ow7Sw`N9m;qaX0kftlvhbrm=5GnDM3 zz_~sJ?9Gt^77O^y*qt$aq*Y8w^{NE;v!$ly{TX61;I)H%YC6=TVt{}>DFl2t<8-;X z&8e-Y9?S~hft+CbEDr{7bSluZSw1Zx@7O>={Ck_j{JO;?`SjSFT~Xz0s|1sHvDVOd zy27nK)KiO(wDd~9nvm_+P)bjXb*!)JJT}l{nd|*UcQ=Qsf^0G}^gTT4*KumX4ZjMN zex$<{`sBFG3nRVq>oT8uI?us*G}VR2f%99MdfNUZJOt1nu`V?x#l6VFo~;QQ{xCNW z{;?-$wJNaL9wQSK$Wn0X=x~n#o4F%_h3-g@5l%p-pDq3;H>wS&E6(v!;A~UJ$nknp z*Ox6G=U9i!_h|qhK1a@U&f${IO?2!RbA6Y(19Z=)qg4L2$8Y?y1L&$T?sbcsiC7T zR^N4>EjQK!K7;2`6AKRH#5}*Ty6R$`f%Y$OvBrLEw6v+{xm!y>|Fy@U`ZZM%J~cM& zY@ShdzQ9798EEC7fZqFSxuN09YD2^6l7I^Mo|LUo#LN9@Qi4m#1N&5sbbsJC#Xzp% ztN0L zHxgVb0m7qXm3rogsa_=|8^H5)hm4YbuIJ&utkwf-DPXz{%nFyLmgLp6)HoG<=z*M) zbs)zlZI2;eS;e@wZY4DjcwY9-C^*P60T1;kNg04YFSAY)6V2nO@Xe8`r#?A3>teZ~ z-Zx)Mx5=wBtrB9gTf+u>41gX0`phh@PG22?zupvLA_1QU19UwvuvShL;N7+CDz}PR z;!ses97=MAOGU}qn#6`X6x5v4B|iCLg^AuA$o<_qRr1&&g@&0YVXu7}Vx|-DQ#-TF z0@Dya_yl-gpNdvx9F!oZG(O!_>?vOkvuq}IAwroqbqLl+BQj|!FIY*F0&N+i% zR+c3zXE6uloWY!l40NNBNNMkLYW|Cv)w*-7 zdCb0YAUw@$NoZ*$(FyYT82G$6Xxrs?%5UampC1O|X#Rq*aHC!YkK zZ7*-L*0) zsGfOCIk_~>y0Y={-Yw?kxSm;pnwXWKdm1Ct+;YHLa?Ge;u8o5qp5~M-z9t9EJ$=zN2*YV zl$LY~JHbPp4^B3oOqx0Mk!gO-%4^D*j77rSn!hm?v5DC8Z`yudpP&>)jS@=I(XCE- zyZV9}R<*wW>K`6T~aA<*g(W8>e$qCYT~ZmdzicDddebbLyfeexW41U(utnRSj9(_2>@WIy;a$t6q?6mW?G-UUxH*KxG_A$^TYcd( z?%BPq=kDxPQ>bw}7t~;hf z;(8e|Fhwg^8|ye3)(Pr2CV>aW=vNT%*?_Br&=jxCbHCxh`cUTu#LB8pd$_GJi=YE^ zUh}#}+8Q(}=^I)ldF`E`Xz?lal(zfwXDhF(_eZ8Eg&QOE!cDEXadBGSHZvq|TpAKL z#I%CujZyMB@59D@voBf?!tWx4&2q-_B>U>}q@W_88GwC%^e&*g-W@foIIAP=r|fsi zKM#yEuOSvWBjuG#R`L17ll-cnrArpIsl@o_idTn;r{|y6>;<2(KdKcqK+OrF;4#+5 zc%@ODf*u&-mL_zZitrTW^74zeQ;RP;jx4<<9gk{64TwQOTYF2t=>M&L-~6+NpJF<4 zL0nI|7}0hVY)mwrUmc}h3>vz{QA>|%b&Pbu|G4z4&;-3OG{LLb9HN~6;1n^p=lY(6 zZBq8~IQweUrj|!+{nw+8{+m&|O1w5fEpXf``_cQj`dq>&rmu}E+8w`=4=lamUkgks zIs=pBLc~NW3r-R*dha)UJubQOuWwjqK6T%#|7CfYRWQ|G`YGt(%P}Ljb>(&2sn9t2 za@0sI4V&m?0X?TYVo(X=4z+q^j8(KULcNgkDO8w8AxqdKoS2E-5jW5)qIz~YVigfF z|Mhn%Z_llqx2r$rkx8>|DKD-UcOqs98HIg~=*T5=&l~oIjO_ZPN5WkhZ#Wh;bY6{^ zIL+XlX$iYh9&_9tMm*PF##};n)K0HRd(?fAX->KSLDgTz1*iVr+@1e#roMt7W}cK? z$oko=zpko(3j=0_OhQ6-zWdjh)SdM|o&V#;-)0!o-DYZ-?l=s?bjOjy#9_L-`(XO$ zb~KyrW;kNHbC^E*;Nau?XMBIWACLRjJ6_lIysk?|*^tVbc(U8}aqv7JdDJST;%&y! zi0x+MKF#?K>_LM3^IW)PG{jkG?M&;zwo{1E;7LT#xYiYH{i@6*N$5nH~0oUN?SUWRSA#`@ke8B@=#C7?I&w zoV?fGXd20!F$ijpB7Uu`nyKAUtb?_l9Khjl(19=Dkvj}PfrAwK`W$ip-E9I}WO;JJ z@aN%&J3h7rM-EH03*!~FJ!iA>uuXs)mg3^nU>rrt6+~k+hEhssrPUsjz17S0*EIs& z3pmU957u_?I9wg-IS(Cl3P=*%Y*n2)WhyA2xV+?I3%CHlHi$wV(IpP#9y_}o$OmbJ zyF(WWU(0*DZ)xZ{+oX5}xPt$ah|W%*VkhUZ2;5l4Ke{-y+vZt``!xw@W|P41&r2X* zOvUU82}3!f2l+c0O^0RfejNBoqX6G)vKa?R&N zH6I}|7+Wh$)}MSakxKv6xMZE0SKCqj=y9O)(7U^kJyz)`E0BmOrKQYZYe;Fn>r3lS z9YgWWuMG3zB5O~k(vj6#M$`js`)^~3b#YR0{h0@BnaLOffY}ur$${O4nx|?^Q)Ie4 z`{V62uI^=%+WS$djpEfq+S$UtdmWvg&0in4nKfA8-&FLU&pXrk=!GSUErp6#D^44_ zE4g-wE~mS4+sl;brKsO4j&gjX_n^l?iI3A~)mwlX;6n3N1tUQFdK(04)d{?IL~;a| zooXWRvsgc~>2+s&_Y{rozzN|a-%)Y0M(kBQNVt9ecub$DgVGR#ZXhHzia~3&BF+rn zy;IDmBx4_G&aBE%Lsx{3yf=OfZq=)XKGu$qywyTZK#Z4CBMcI<%EL7nPFzTR-tB9~ zG_PX4{OANG!v_X(1kkyr=wVCjg~@PRirYjCg2g5RHq zqst=PNO_#txMP0=Ndp+Hq!_RySWc3DO>yJMu zIVqEs#zM7|ST{u{c?v_L0dEhgf( zSH-FjlxMs?IwQYZ7h^K5>S;w^PgEU*wphMF%%-Re^sVyJZWAV=&}ci`365zR4q|0x2mX+@l~ZZRAFmz zs$LU61$_27rxT2XkR@-T;^V;Q#o)p_#h0M=un>dKVRdQ1rmKqM$ie**Q?#K>JQn+q z_+ksVRNrz@Jo!MF@K>M>r6ld)XrB$X8vozH#?NrY)r&!ARfWDkC0NpvZ8@BsCABm?No9cs% zCujzR<=H$qs##FA<+x4j(SDQ>{i7xx^GQr^m~0>JL7P5itU-*v&L~)|23c3!hT=a( zf$}kq5)o%u?;^5I_=CYpSHNa&J1PyB>v1X8hZ| zz=^OrV}SZNn0)E+i;E{?{WQZNJ`OrCnBz+lH2X&4!(*SzQ6JPvS-`5uNS346?=^zo zf=tVWGxg(+CVRhJK+XfGU#OHy?`d$#^@a}FP5R_K#!zTK`GMH59!q48xM3&HRL;G_ zM|=1qrkWs)iMQ7C%ths19%}MzM@zI}<4HPJv-RKU1}$k7efT5>KQ*>89~MLY_>5{e z_pbJvY1J|{cvDa%nETlCTV?=L0W^KUt*RnFO;6xC-mCYvVJrQ*c_5Y*0}66)*eZJl z&NZ_ms?IPKEYxSlfmFMx;}VS7@p02{i;5aT!iNPg_@|5VIK^34VwVQBVMV7Z!Vl;& z%uIIfnSRb@Wj9VXG5XZhc9JCI4B4yMHr);4)7~I0Xnr`D4dqZ>VcMkizwSuq8_-O_ z?m~`tR({Tooaohu&hPdiZp_@{tSZ2}YF742X8twriN~Y4$<^DKZftOKt};20#H=X$ z7Y|bf&}W$AN}?doc^$OgejedB;ku-W>c%6b_g*K+z!c=h1+tLeys9u z`C3noKD&KFnvC82_X!(3oe@tptPB=lod{Ci1CHTLaVt_Fl90%YWncXhRWNYX( z((>yf!q3M79AtXdUkfC8jh)g514Gi!-A(0lG#&bS?G)OIvmqMO1k-TaP$IA$mBrtV zAG!KLtziwv%9D??$8m`yIMDaS);jNW1FWY}4v*8|LJiN-J z{dq~ix^Y{$J8)xumRu&Q+_(SRs(MqY>cletqyoP^EZ6P&In$;5fx*Z2&(PW`*%0o)EW8>!^vfAc%DtD^orN%CQ)pz2-&mam*@LMqu2?5Qc!Td18-d^!9 znBQ$@zOk}NF7aGYZn>1(YQ=AxG0^bl;AClv_4D9!gZaiiKbl)jzS%iPb&t6I_3#pO z94a(~x!|HI!gca(PL4lcO3nbUz)DJMxni@J##VL_X6Gb%CT<@11+jFOzvsI+$w#+` zXvj|3$hgL0jSAmLalgX(D#t`MS3f2gaTu8GtKIH-Ewc5+Eh#PG-$?o|4+pJfrb0&h zLR=VqfkpnEIr?F0kM@f9>@+aNrS34Fg@oMZu|;FLRRrbfMs_85EGm!8 z0&Ivqn5#60hKf__B>LE8v6CiRa*bOhmR+1&L!3SqGnpz%pD|&tW|l?x(etZJms; zqGOl8qJWe+4*ZMzU)ru2(!_N8phb16<0j42jMnCkjawClYAQd0@G7p#l8-wS!;+>v z6c`w_k6&UeZt;h&?=>WCc}t7(JjXbFvoAaNU|ynA0?3|dNS9GrT;v!FDpz6%OD
hu?fmu0}}U8N-zs-j)5>YW-dwjb|k; zOrlmYYbw-%ca-VUG0J-$!~J0|B1+HYt&jjueRew{#>?Sc;pDdW&oW&vpRtOlY(sVU zsV_w7y%DvoNQQM z{T2Jl56fLvIMuB&Tfw*7QiKc2u|qghf0Mi35@ATQz^jDM(al+&*fh-iZ}2ifshg)9 z?`h^Zr_$G4)IejDsnil+d|0eA)y6;VHg(wozb*Kg`KUlt^3HBwz&e{y5%%Fb_D&vN zkVJIP-Dl<%weR3hD)AazubDIS`g}}OrfdF&%I~pMZ^uWax!d1>j-ECdtN*Ky9)YL~ z)veb&Pfu^$?etkV1HDo(d{RxCE}HzFhrT?X_cT=b+e;foLT1eeR*Up0w~SG7-u?C} zNdl|E)E+)7Pgx@C!WRyG6@-b&)A+|bi5Eca6;BOU4VT{u7raERWS3drz3gMVD3aUB zH1HKDEHlBF4?mA@M2M+Yb%DhHqun)sb%ld?Z>3)OjtlR+=05LB&3HGlj(jOX zB~igV7qoeGmTsDQW=Q7%AQt##XJM7<|S zuDX4!veuSUu0o$Ona&T^A)V6T9IUQlo9(mv&(t8V?v)zQVDh_aW3IAjc*E-VXRS>$ z9rgZ}On0T0l>i<*9`!Lmb49nQpOgJ0tiO;;I&&iA_Oilo(68Uu~N>(HZ&H`t{dZVwPiQ_vMH*^GfJ!n57Om+6}?=366kA%Ce zj7;(G8_d3AaIpEjsVBTUCW9k+=z9?$^jsF;XhZyLl>MeooqnW6*nZQrlGi~*@+9RK zkp!On79dv6-Qu5c7T+Uv5dU&JgD^dwIaw=>pgz12)}SVF!%bgphY zo$T-1R+CbqRRy+xrf61-7A*$vVk+kDE6fzU4#;`kv3}unlo&1)C;73<0paXkcA`C6 zb1JiT%O54fi4B6t3da{Khot@cw$q!cr}v_aI~{1jWvqLI(&+_RXR#2rmDyBN+%bJG z1#`B^zg+7px>l?11+msCY%j0wg-;006aV06Mg8+%Mq2c779!m>)*sCx@VrJG?r$kF zzK=~7ZFjj^HtZm#-&}{B0jW08>V-kXZVv6_sn0z92Z#hTToDk%-v_$Xz%AeDfDg8$ zGYnRku3NIhiL7(GlP0bSk43l5hk5FJ-ItFYMvrDv6`$=ptdL@8ybe4TV=M$gLhZReyTfU6&s8+ukO(WN6y`puLx`Ms7pd z5sylu9-zMWHlf#mpFBgXlU~Ky9Nj;#C~qmPssTdJSA7e3(WcL9hfZ^z1)Rlv`` z4PK7}#k!g7x8_UiQ`jNZm#=IOg_GUt?8r7=8s$4kbRru=E5Py-E4e#bbriPVQ;ROJ zbGtk1V4(&i7&v$-$&y2py8B2E8t@L+qRhkB{*H2MY1F_AQ*9;8ewT}-;xhVQyY`!Z zhhe#yk`Kt~ZXY@mggo+roX9GGApnmpz~A38L`V2G&Bf(TfZ?n&cH^mg^6~?StmJ)J z~8MV>jS^hiocF`a|;?-kf`OsffErO(`BZAVAZ6_Ny(bUN*%g~ z0q4%mywH5JCZmq{?!J5ZkFZ~>RbkI&3hOz}>`V&qWrerx4*o6YP*0>xpmn$R`6XEA z)0+h6nBHN(=BtW(t}Kfa7Afln2NCT)`}AzOsp(qIIKqTsWlhCV&JvdD)n!O9@E5$z zd1JeBxl=^wdO0l%wR6<5ueLh!Q|(|wq1@~{2L&KXdo|e3nTX0wvX*UMGs@K=oFB8J zTTp>3iu|*EnTqs?cgi#|(tqIndwW|7S%g2=9}o}1&*RVDm&@Ke=x!dDgd;-YgcQGd zKXomGLdSEOWA{Dcp}N#TV{w_O$I92i+G;M;c}m$XIR`4pEUNm>ib)7`Mimv}ne8tl zno;M;>Uuf4URT33x$1wAH*X?zl0d$!`g#1Xv|p!n1Q4y%ltO-C$k&qM>{^c;vEI^j zy5HD5XzE30jAVg_SkK^we-v7H$T}lAe>si*GhL8J2yE)?KL4EZf9^De;enf}O}$r~ zzPcAbetFeGeG}6Jk8rc4eZ~#(JV66OrSBFf!r4%1IjxR=drEvjCfD2^OEST*`)+I} zFR#GbQKop&;2FhFVmN;KI%`W8SQHdg-DvYg1J7mtf$II`6Cs?L-rCpc^t}bogFcVj zbb6)>)W`%`+=FKX9pjtlz`UMky`NdXRb$sadl(!u|C%uuO3Wc`k3Mc#RQU!Sba*eu z;X$+n`u}`O2>v}Cqv9AU)-GL{%5z$+Hv&am@TeQ0V6~#DMR>YS&WZB$t?x~L z0d(tpq!$o*Lybh3b{;3A4y#cE&Sw&dA_GPIw*UPJWeHevMIuIuBp+Si1Lw(+YT4oe zf2U~YpLe8K;*oC0i@glP@zmlTaCjm>8JWKW_mo%56?fi7{4os>!NQ<7h@KT*JsS5tcocS+zq6vp~)BkrBW{*M^oU25>h~V(YQ_FYeZezft!XKl8zC z;N6ZaVb$;FMhN_)a49@F?^40sfaPGiF=88VU6RLta@UfLs1=5vt2yzq#6LD`t5UDc z{Mv{2%J|~aAkj=e9R@BC!2^-CYWKC0YR4j2EKdIhhA#na`#X+^Z)fWX(5SQLasW#_ z%}XO&@_psAA?$;VGzXTIQEh&>K;QBvaV>NH4*vj0c0@xKFg8;`uZYscY;~X_Y6Zbj zrZp5t?iFW8l+v-uzhi3?qZRWa;6h}FK8<5}7;%9TsKQkc4Ihz)pateZoJxk{Uw34G zLmEB*BL5K!NJ^-1e8Op{pXqHo1+3&LpMvkUt#meK^SiqU4p0UD$w7Tm=Jl?kTdgZjHCPn45BKd2MDBzH4>uyMx%3>Ff!lbFs zg3kwT=`MVV{S;;u7hNM3`p;0}**MQ>UF{VH%mf*US~0xZ&YEYBJK)lOK|z{@0!=MC z-|ye7vEVN8Se8Ykgh`nBP zvZ}l1i)jAVR{JVW!P)g|_5ygRy?x716ra6aVh8`vrj!a$HU|?G%2sNTGH``l{*mU& z;qZ5kvAXIoyx@>dSMCrGS_O`&QjuP985^X4N=NnOR>f;_)+GFi#gEmDTGlDd&()xw zQh!@mC&$_xm2opRfY~0_R%4m+U@Gtg&SWfx0Kec;Kz2cS%4s<|8;afo$}kQ_mK^iJrL3|%2n)*9Z75(eRnLt;06?j~NWPnJ2__a9mLvkB~-Xhx1} z`t-c}j<-GIk^h7rsYnrFAY<(BIg{IK4YKy+veJ9Q9SKQ0$NG(6v>WIJDU05nP}C|1 z^}3pisxsLoP9@cJttd2S=bev~EK%sFO^=@`FP2%Bg&L4FYsN5$+V&1HHfsCUZ&Ud%n23q1@r9f7Sqg6S85j0qVODi)nUR;DJ+f#hN@wNEbK($WRIru zTx6tT8U(=TnK|NqOol_fNE8@mqjW}Os`_rq;x+OABwdF3Xk4!TfK!?u9mv@O0Q7*X z4eb>!XTTP}6*CE=$^bGfT8DPJp1mL4%&xB{Z4f)@CeCa$qr`fA!?A;bI##(cPti7m z2^3n);IJ)(<8FMPb7^v!Fcq5=Xe{^Isfr+~DVn9Tvoy+*a7*xQqo@SlQ1;J+92Ns-BcvbE zSfZ@cO!SW3A9bG^ODmD}n}%qB0v*?oohzlI+N{9rK>d5@99g4eKIMh%XZdzY9`4#W z93!N)Mc^N@e+_kj6LolKa#%87!rQ>^t6x`@#le0&p!~lUK>`q&UfB~u#aDS)=g!;8 z>EFj{X_ecVJ`Qk}$Z$CEcI8r0Df*v~3 z>}6^|;*R2Qjp%>ndVIKLXjzaH4}}cYNN}nPn=GrRnxwfFybm3d-b#B~BTF&Ln`F9J ztxw8GQy1%y`V4Jg-2NIz;F15f=K9M}ZC7&UQCIQ>yG{nhwYwjd+H%p4(qrU&N*pj= z{|gzPgNwqDQgzI-!DP5{*jvP`;4FQ?`ydf#zIM@rZ@O0E;6reh4waAwr=v2yDl_=K z=DhQc^Mvb;3eACE&!cxi=BYK4f`9w40eNtv4L$ z8Gj8)iBzl24P0yy5*=yDbRP2pbsJ^7VTy7?A?Us=tNl5yjD(}7Rlxx-a(eiI8~uV%DA=m=V}u|yPyyn|ECYFbaM%%xs_ z*fk8uU<6q&yq^++7}Ae!HEwIUGkp(9E0$eMs~_{Gqt+a1iUoG=(vw-Q=7blVw}nP4 zn8++IuMo{^#rA83X#tzVI#2X|KR4wcz8qqV2AG){1WVqb_|TgrYZ6q3ME6VMJ7rzp+|2WWNXC_8tRK4h{w&kSzWJ);wYptK zQa!*I;fav0ZzeeLp4sbO zD-Dvuvv+SRb{A^87^n@rWh7`REy!xRER18YF%?Pk76Po>GHxfkHWZ(;r_K+jNzo)v z)MIqv(bql-!)fjNR-`u!`5?)5MB7zt>Q?4i$PA5jh{4btgd(qj;XSi*$PieD^Gkcm zuWV@LTVcbfR<9F?)@SLe4)hm|@Aq;^!^bMiiQW%mK40{b;+yz-N2Y6o#bUV28h8K9 zGi*i2^BLxItxvKMpB3m5s*gGIbyNHtfkes5$p*_1>3Wc0`CwcsdS`fF+kcm1?(=iO z+|#=LWQxsfEN#Au-nf~5(Y!}|SS^zAk&l6-G(WJ(>LJgngH1qUi?aeswwrA}pJoah zyOUjA1xMxFz~2y{7QIGOA>pfMKUYWwWu@e}e*r?`v?2|n(x-79lt>ZG-~Z0aVXqYy zcN(TTkF+3WOTtv7<$|BLRZW!o=#Li-)HlXqqu)SNvaR9k@);CWb*#e1ri*ML(b@7@8h(XtoaGO%ls)PX1s&&J#jD7e2r( zx8v=n(G^#|gVrbBlQUoZ;Ku6?hr1UaKq9Y+o88rcLXr>KMUvU^hlD3&Pw@6a9GrC+F=Y+FDZue2x$WLr;eT=U|B zv9af&45>yAcbY~Q041vzioUp#Z>YZ(k`94k)5WRA^&+z3&G5nC3B`#O zbQ;X%#J1htsE3toIU7Ds=P5l*mO?yy`C2x(X0TKrXE$=U-3hnk@0=JQrTjn-#1IK#0&_K5Rv zL>}Qy(Y_!pq{C_1dY?vn?W~cnY)3iXG12st*UhsYoun(st8rD)P}IL&PIj=meh`U% zC)9rgL+Hm*8B7nx@u(j7s-0EeUTWSwh*&Y$fov2OnxoPZA2K9pxGu+ zsoj~Uck_K*F}zizTzDrk*FBW$#eWd<#h9)M_62QJ|EcPJ&~;Gd9D~KQl2@@_nF{P} zHkVKnT1|5n>}LBn|C&PGM0;s8g(D;OV~d$;{b^i(~j zpPA?1PHq<-t^3A?70F>)xXG5E?vsc)MN9O9&cG{Vl-iG9Zw#h!t#q_ETztICE>?CE z4?Ed|)|8S*J@#Ekm+r_;+nmusS?S_!^9BAqCc4EJe~3T^M1xZ{u7sPnY#GYXX4T1HcT3T1DDl- z4{EK9;HX@Uh8Mb6RCo^ESE`MAo?y}u9Ez8BDqQm1_-`_eN>3v2?8RO1_NFOA>-Z6G zLJlc4iLtRkO9UydPk&h<*q>>m)0O{_o}9mLjwsect$bft@V6F6=&1$XamdxHanZWl zOf!5s9Ur*eIpcQ-urcdVq`+-#??tmP7d{7P0OYe4VPq>ot zNC)p7t*ZR!p#8YW(s<8vHAu{cguPnWp@4+R7+e9)Q~HgfTDlgRpf3N+F@TagRj@f2 zl>Ex)KD90bNEu;*nejW^DSqI@JDd{DZ}8!(uj-%FOQrDqPDYBU?@JVxWa8Qi(6F?+ zkS-_IFxDvrNcS@)I<1wR6!Sw}74@o#`c*MtbsiFK6r_f!*!yHMB+RhX+YD%I`EWSAwD$xPCD zJt{OZKI4sb`nt)3%ZYeDt+YJ7_1R?`k1FUe|dHSG-@lM>_N~iR=qcjZ7)+Q4evP5Du z0CO~=aRB`5&_h04H;2h&$qei~VQ@$?h)O-A$#Y6MjejHbEd8h$Fkh-$O2g|hFz_MT zphqT@YrL%_yBert+S6BV8n!=losYV<778khEv9LaUvUw)VkRpgYh{FO_;MYbQoKGU z%?}!%vO7^&?bjFW8JilLHX-0UH5ZT`=amdfU%X+TL;Q8=DUySuRIO4&cOD6gwg8)f z^VOPG;(&#j-1U_=e^Gpm$pq@z%mr0XnE%1T|CBxw^dx1Qej2+kc?BmTp6#uW%{lgg zRca6B5R!T^0EbopQLVek6s1qfzMm$28I^lL1c!f{clQR7u&OtHbz#Zp3gJ=7D`|DE zK?{~4#5RMFXbo&?&A8ybI0(cN1&4{t*4#vC6LS3Hr+Lsh3eJJ_Xz!VF-#I8yPi>SX z-GWvEWG7Vvo48pWykp|4?8-$uNzX|g;dz<3x`fStJWXtC({f1th~^`B5^8eJuHBQ6 zU7WQ*D&WHA>`3hTM)Vc40gsoRe!>$5=(#5u7ILo#W7_&W4y(wI*8nfJ&jPJIO;>vy z;}6ZMgBG;9+C^9OhQ4?BP~-J6d$@lY?w~~XC&|ty8G6wMQVjdR%d#|;KDyCI&NJ(& z3x8c-KvZ01!pUJM(DJ7O1wA*Xy^$XWt09eUqm(JPq)7segGf{mm$8>&H&+Fd$>;G+ zl$`9L?)$qG{wwd#YFs81XTm>ul)02Me26vTiUkUHb-#l(FJ*|p+i>zu{F$F)Jb-S8 zYx^;Rhm*qRj*A#4?E4~uxj(h}`4%BNe_ezBq{$?X=x#RMXkkvdW$iS?vSxu%L|bSM z+feEi%L#B;xy0w3{`=ERJqA%g1F@f*V<80h(G#$iS*pxP2i^PRV9cc@C=V_s1b`E( zxo|g?$U@NoDs!C?RHCZ)o0Aog_=Si=uSQ6)Rxk=P6BssnJ<>E2dq#O%1!~J(t!#t zdPG#OE>O* z8}Zq+1p9I|x2K1wxWm|X3R4-rh4Ug+yf-6gFu&mheNuzHJrtb?#eF$EKi*2*8$%(l ze2DV?QyDbUlpES`rE*D=54P4sxbshL7J&mdDI{>*-(HzoMaU08=l1uA*|K-=+riNx za)Z!>kom*$iC&(1UWFkRG8P&aK?#MG^IyB^1;OmkX|A{HYniP)**CZA(d{DRM?_aj zN;2D{J#NjyyRCp+l$=zz53ezgPV@-IEPMyNFD+N!AAZW=CEe30*X*h}4O38+t+r*g zJ<(yYXWL}?=L}#9G*%i}fqW%3CuIekB-Mo^rzo8(NYU6*A15eRQUrs5N6W?eenc-@ ze=1`I;~JVKqlKG6H!ToelzCAdO+U&HSjcA4Wk07o7*pvlgu~Be_fe+IZY$~qA*+J*d0_tU$!B{Cgqv!ZU>>ZT?uxS3HfrXj?p6&@Cc=T zj!3?xRoCyTX`D!YA z5)xlck>N#k2=6Z}l;mUBV>HT_$u=Gh+tIUs&F?|ZPqCMF{!0SWlWM_sol|GJBuVaL z%Jio8*hq>F^Y4}#Nma4A!>B|(DuIxBbi4>**}tnslaJJJi+PrG#_C(B#!9s{q_SP? zKsUxBAE>dIeD(b-)E;TDQe{LC>W)Pf&Jol~roj%Ccxwl_#ucBqbLbdBv|%{W;PXWp zqRwPbSG8A)nI}2TGs|*l%=BDG-7Rbd9rS)12(W(RJXS(aRPX1Wq+3_6hb2_yk6O>* z2X8mVf-&u|plE53Ay`7oNE&ahBd%-%)Y(CbV%kX!TJ*0>pcZt{k;--_9E#Z~E_rB6 zDvxWk%Qa0mqbQm?j$QJvN=BeQdzlD*gj!pG*13w)uO0KgA$tQqDba9S-_KLN+YL&b zU&LOh;YdZw`B<^^bq`uoD*9X_Dhp$v*Qz(8U@yvKe?<)FgC4I3ayjP*#f;6|8_6?o} zm(@qjQ$d^HTZv7>bZVqumsHs`9%WRA4mQ7r}GGjiYqbe zTPzN6?mjCsFb$~yMj21}@^zCv=;_UNXUEm}>6T|TW3}RgeKqVH0|!19e2vdi<6e~s z5`0z7Fx+#=K&oLaT}wLP?s>|jjD^03PoPCh0N5qbbaj2d{=gJ{)t{sv)M!=-{zB=8o8IyRXzT=*^dLx8G3`w79c#r(u8qZRW#P6=~$UUk=W^4jX^JBiKe zT$C#dl^fg$Ft27`c~UJEBk`HtO?X&=A>{lw8bx>TOs{;|iC&P;wP96b}?nTH<5-zMJ4Qd1kh19nT?H?Vk zWu_~-xQ?2@pTRA0IIY;LafT1I!&Pi1qcdxH87G~28B~5bud2h;94E(7Bz;M?ch4u- zJwH51q5wV(b_i`S8TorA*K`?z@{Cf#A1A!EW9(~FB28z00ltenbf~o8b3cl<+O|Bn znYBfBTm8^7&u92_#KQwkiCiqnj*?wG^_gwM`+2!Sh#R1^O3(NX@h~k&!45j=CYUYv z9^1(-%x<=JvQwSQ%SwNfOv}SzmdLklr6B)hxH=|_G3Q9mA!}FnaXpS+_3{QALH8oN z|2vj|u~`pZRoT@~R^CPk%qFF>ns5E!ex6A=9a5S7vqsY#8>LZS$6-s#@1W{BmcCAy z;0Wf!I-XpKkBh1CYdeoHXxr77SRa{4xGTgN9KZ8y+?r3Lw%JB~LcPqJ8 z5-faPe6dvC9XVZDYNV4>;UfZBf8nT`D@1$MWjE>jTjaL<*}-?K>EZtE6v7F*H;schLkf!g{~ZWYqty8?KE|nrc`os<$+CEWNDD|aje_n*F4*I#j?_P%Mklg zDjm}*KynhnnBz)8ca0l`IT3G2|@$Z3otfeP2=XyDAz{vg|ivu=>h*xU8WEirBE2oDJZssH-{BjbH}?z8Y@TJc&BPq zXOr_>{Z2CpPU9b2GVD4vG<-TWq$AZqonl!nj?|!P^vK0HTcYGqEFUYnBJ?tmnFxGz zU-sNF9u-4~pB#0Dx?&=1rD4j#k@^QEWeRe#S8Re-t+<)hX8{hX0zn~731Zsj*bO9D zk~_GpzRuakyqDSjoEE2N{Tr^mJx3_dQ#*n@fJnP6@77mnfrJh4H)A~|1hGc={IQ~; zavt7&^rbQhqrrJ?aEY}L*H2Z16_?I4!FA~OLjVLhDgX_DE)hDHa--Cv^oBJmdF{uA z2VNCuceaE_1Lx}ZJFjHZnWxhu*1+3O9;$PwfI`?E$whvefxyT17Q8<hcy@k=!z@1AA}(4W5(%6@vlo zqRg!qT#zn#O5P~UECUaP*?~&5$!T&vEJ{LV9(t=E$<=D2@!j9{9~};W4?0LnQS;$P z!gl7b#l%$!p4pkcnt|_ldt1_ejtNJpC?;TE*6xBkM>kXQEl+iZ&1U~wA7!-Zhjr{h zB6Q4K^lo!1wtAk64{8uh`&t{gh9Zs5RNVrWdT(4#YdB%Dzu($X;zl%v5gj?&c8_O( zN)9Mghf5RP2zc1TK>w^~-_xV8G(eAeJ*Jeoj49sd%DS-uDY^Obi%i1}s#M75`dUoJ zV^w2TwzUf9A*#~c;&vjiO5~Gi-;N^dUfVUZje6a_$)5??uPW0HB>!mV-UY-l-~yVi z(!A+w(*LMKr6~ni^)+G@@teP}iXc1Rh}oD;&AD38q@Oj?-3fg&7+t=&S(p7TtvSk3 zT7&?+qdRNIcb%qfb-EGQ-BvmueE!x2Y~y_kG;$+A&z7 z{}`E8mTMSH&O}qDBd6vq%mIAKh@#_#eabOg50-y}e8OR9{OTRzBO%OMF)q^Pi%$hb zagDJt?CQb*d8V_W&^oLQRIYG4+vy3wk_I&n2kR7+Gbu`qzv#C!LL4`NwtMJb+uK-9 zlYUuvpwJgvq`zdhX*a0T`L`g^-o7;~)c!);d4+E;%OA8%N_dZwFHl&>-&$-R4r&Ad zhbpyuWrBmx6vwf}l_o1NxRJCdn@?apP6d&kBQb~0Wqw{m-NxcEg@qoAS=S9h&vn|6 zh10%{*5;3DfuBT5__%>P7+$%XN`pt?xq7}`LZ0mqY=3}5<;^V zt~Oyl^aW)6pJ)<^HgmT3I5l1&m8pd(1;##@aX|fCvw{C*LixqVTO!;iX$4-y2%`sgIL(%g=39dm`u&|- z<{%GZs}q1RP`;V1W=DJ29pP^6Lt*|tdt9V^mn!)hqn9t|i-?aS3&$GPz(;H5s(k3z zz^1*=RVN~xmSbFBw-+k7?Go5PosZOwt5k9)&AEfh@B8j;j<&+n?Ij=bcqTU}8^bHul>85&k!$<95}+9dJ0 z^FMMg4xDvKs{=yWlSd)KiC|y#S~a50VD;u7VWKT6hWVi`5+)dj?~E~RE4$P(m#!() zCUIVyQXTcY+Z*zagER^-6z}@^y|-m1t-+b|qslE!?^&{vEgR z8gWZY92AbvMftK!tNdk5Wta2qDRjvD4qNV>bhs8RCYduPD?Q(x;wXD^$sX~V9jSq{ zL(SQr`HSn3E2BvOjG^D1UhL@b6gB>l<4Et;%I^>b>Er64%B*nLI?xhofZG<9bQ8OD z0K9uv=<}4|PqjanleeaJABu=EuKEzAuU>q+E=G_4<*bt>%mnS_!Z(Pqn9wIOENMz= zfBj*7=j%$2RZfABNyD68{>#b3xH zCbPMtu%xRaWAaZvDR8zu?oZyUekpr=3$;1r)p81k66}AcugUOSD)+t_WCvE2FrKIt z`rUl9>6Q_}@hK$yrCK^5GW^aum_CCrengp~eqHo>r=2h5(C^LF--4JpfwS&Y1)x(=7@_xveWG!LV|=HP z+z@=TS4fY1#=%VDlrWjg^@h?qZ(m*(M^D~vKfR}-XY~wIOmUJeIrKm6-b1z9wd;f_5V5 zo1e@yU(3BaCp$cUm(Y%u*ZH%G+RJ$GN;Ghll64Z@64L5O)FOnHs z+_$CNKi@Jqdsx@Ex1U`+CJiRu$wiE2evGLz*L{#mt7HkfPljaByue#Pj3qLip z1m(@XS(LOM>`8bN_Q{dQ!?MBaQuK;-$`eW%MDSmv2a7YkT_|Y6mshk;d>#i=z@e1l z(cAM&&;yz+{IitGa&n(BPUug;So>hED*&s5Z&5LV0S!ZFBMl3YM(kPz!w+U(dsi^* zfW7mG)BmM;4O4URC*{{J)s%uf#vzznkL0n^FWHArL}w#jh&U63p0Rt>u8UpOTCXsg zN8xtTmA@0l&WF*5+0`QG-i(kKgXy7|g#-KoWMl0!S_`7w&I8#Va?!kf zw#!tJm`+Oeu>9Av=Bzmo431o*{oCjHa2J(BuQ(9QJN}?8^J+OvNrL|t0DBX5M5q{M zrXD1bLb7nCR_c1~kN8gpkN!%3q9_;S1p`npt#iNjTs~HNa>E|-6V8}eNxnFbR3x=B zRaF&N*GR(EZb&iZ6Q8f8(3r=Cf0Xl<6YWv#QKO*P6FAA+6rG9TRh@l@tJU+&Mp6DL zfqufOrE2DGwN%E~qt)C0=DXrM#3Xh4~?8&bpPY3&v{|@6a zp~2DuqmB0frRQlU1GZfs-q948cF{JuCwnI^N^57@Pfz#w?6XOi@euC5-SN!%O^3GZ z3udIP!C`d#n+|LqkDpWr=HyHf5M~OH5`%soz;&Zv*<|KiIpAo^&F7{der$DNn!uOR z2w4*oCZ083Bc7=^DU5!OvZ~?AKfdpX&EzNXx{JuGJR3ErKQhyepzN)yNonZvZ^@iJ zS5^M^oLT*9{&le*W0F-6io7>!Cwty&Nq%_gkOzM zis_cT{q!VDF&wS&u%mpO__6+nfrRFr`VvDZ-I5f*9|MQDAhhVew)`9ALhhMX$UjuDv3m*@Y6IZf?kFw(P!3|FNzXK|*=So4pU(iE{BNE`J62&rwF z)4zMj#K?U)F63CjK0RZ{nwl@?>}?G48Z3sy_AE}%c#)lK9sY5 zf`i*uf!vZXphwcI>=S8m!V0WAtzTTJib+t3`tzj_D=|CZ%t`BmOHV4>o$3SEJ zn$9Z!sbipzQ~-xnjMM`qGu32X?^R`F?kYG$gM^jx{DR&46cewk8FX#W7CG;xIkpP+Qn)0@}zpB}(7d z3bXfAS;YsMtb(JdTr>U`A8AOj1_?f3Pr+rJ&E?eYPcw@R0{o z=KipvzO%*L=F^$XqEnE!KNoYFPZs#%6EJoY@cz5zHBm5MM;ZpkIDzy!R#8EnaA_xM zI^s;rCjV!a!~Q?>Ji)JN9@m_+yHZm4+A!kV!6YNayoUn(2K!JWZM|fbSb*T+EJ~ivjq<{{=W@A9$f{;KR<; znYm}$%=|N5dgd=3pL?QZEC1RY5;#U@!%&DNNh zu`Ly|3?qEho|KC_1u;L;vMJkgI%>WHa8D;C?w2CJ;#k2U=m!Y_n?s~f;KhKp<`B)8 zcs9+#od7Nd{9d{-E@rQf2$27-g#r(8q9!LA$53ASL8Qw}L*&zyTG-b!nOJUj7)fvFVGgV&2u~Nj@1LHJGDRID$P!ClUzzg(LaPJ?+$>OFHok#?{Gal%p zl%=*bhl#@IXY%5NcZZWa_MXa&-)yW&A!HdQcvz!?gWZ(U5;iAUsq2$W*y<29;wOmh zPvA$7m2AwBibp@2=F-ghzdk?;SREt>13wbJD`CW*XaK&c#aTeZC+tx!nt z&jcR6;6#_60d!;m!0OK9*}|S9^O+IAk^5?Lf_|J7V+8YL(xeO4hUsw!vV!tGMMX_tE6b!RY~I(Fj_hf&3-_l)!hIc|wXLSZ^`k(K zE7X~3sr)WqchR@|;Uzx+^B++kjOJ6p)_Je$iZKbz0|aWusu{4*;qJObL#q|Peb zQI(G$%ocQ-v{@yq<9z(4jE6bWFp)o}$Vor7)U=HuoX;=diT(@f>OVRz{Y=A1Gs&s( z+W>dgdeI)6{TR;^H4T0?#b<;0!W{v>vak6uv9Ha@-&F~iM!*GU=CyHi-^;W}Y+lbj z23mGsCZrq3IB=7UgD@!slmj)71iTsgXp#X3oSI}le{nFDPBTh4X**+dplvxgbI<>w zibFUC9Afs;peC6|Q+(FhoKXC8K0WVfS|~IDZ@f3jMjQaI@lzpUgSpB(PzhL|=3);4 zzMX0^^Y#G#Ep=mE=9^RPS36U`0R0R*l#7_#;~d=j04-u?G%s^!B$Kr#e~ypbmjZ1qp~M`F(c;g5jy~1$8T&E@)V$AhtODbk5Rgxl zIfcjSoU+rYjDnxj9LA0m8))v)1iA-p&M%WP18@wVZkhys+k8G@gc`ltj&@$_#&{g6 z@X;n2#^1aTd7>$5Jy7OXnftX26STN}pz%*ZoDNi6vUz_S%rX354?CHibD&`$&1;_< zL)4gUFt#HVInF4@1@0)wG3N6TM~ZarsXDt5=-T{4bC1)sIQIn1=g)Z__?%1H8D~OO z4Tx{%HTnMm?|D4U0ltNna-;>=0dnrwydeF*OB~LB7TC-aH4}TH;ZT3hiHlDb#NW>r zGD=ToGfF^?Wc{4aEc>;XQGPNf5bsa%=$oUIaKOhohnk$yU6qirqsYoW(&aY;d@L|) zpi`j9_m%0KV@*cZ3GiKes?1!YCb!I_F6rIV6n7u$GE4S=hc(yZ(KL%T+mmRw+9$+s zjn^x8loir#b!H*ZRo}t=a!=<0<^o-IsuK##=U4VM89CdNRnmP;UiCiEAb_9p%wB#^ zMN8h2pnSF^=zu*rE$N?`^vwUwXB7XV6M$IoXb0-_9N@pJ0Uwo}YPgg$Ej{&6O-)*vb1V1koo=%og4bmHCg-2~5{S^*nqSk4is5#ZY}#&LSQae^KXat8Hl zhR-pt8|}(@xE&eae9nn-EEQ3YWjw;6oQ*Vp&j9YbBcaBaWURE!QCiee4=rf9lM=8w zfcHI+5+k-p$q`#)l<4&Va-d-_9Y5EK_g?HkIBo&_F#;^#mQdq=Dn-1XlVZVad1ovMnOj?_g!QvhC701rG)irSMB zBac3avVc~{f%;EAR*_$%z=gq+f!gd_vAv3aq>t1j;gqI4d|FdP+KRnId#V~>Dfj( zJ>@{dB^~PcOmkn(kvgw(U!Ea8o6ash(Po#Lq+;HIno9-0g*sAD(~gyZQ-BUVo1$ZW zYPnRPkMe)c7Y&{+=XISeWS0M&=5fH^()Jb91k)rX=1@h2?kgxzfLRHfV?4rvD!c3y zU<%N1yhAm>E-jmIqM;@k2CyDW%?SI|4)nL>c7n%Z1Lo`Q5H)Ofh#0T~{$}XMd91V{ zJ}x)I-YnL~zc`i=!jI)N=z)|8I|BZ}I7SLJjZwl6r1XSiIV){|Hp*{(*(zS9)Ykw&VcSnFkXhCRHp+uDQ0Vw5~1&- zg`7%-j-H71E3*{Dj{cIn_CIE z9 zr?X1T>yl?PIrXO?=0}q}{JxBb-5H}qu63t=UTjBvUT8x)7zQxj=KW=pO2|CYig^E? zLouJnm7DuLaNdSKyoaVN?e$U}*1^z6@?UO6J8bviytW6J(W@PJkNJjV>#5S{`^)vv z*V_YlZ}XXT!?=(%*M$1IGeGb^kW%A}Ag^~v7;zijL~rwXMMDogWTl1Rygkf_S?ebH zEY_nx7a=3*vpw(gzai8`ip6p7_=`XhOP7=Zv^GGI|%=>)tmZxw;$zlI8F@Rmr$cO zdI{e1^(cGe5F57A3VXNG9RJcdi1j%fr^OqGaXzL|e1LJ76uQwx^x7Nd!S_aZ@WXK) zYIl$W-5uh<*1O2y_b}g9+EVP+I^mzTM=;(y6Zl}0%)E9*O8|I|I+`R$o6p4jG@r#$ z2pQ)6Ak!El`BW)n9>~~;Q#BK1=4&%YnMO$=#u0KTc&3nDFot~@7iAhtr<UU3 z+BCsN>`$;#c7_Q4dh^}AiF45dU$Qewjo2Qf1Rsqtq5n~_F~7id0DR4Q zC(>b~6YFdmp(mVa(=*S2hdiF-5>2CwB-1cCY;TwpwAqDoS#1N@H$VyA9im0<3~*sv z{k)X@aW29nVJC0&ki6HsFwVOpc>fbM3vC*uMek3r;m1IK?t^^VQ}O9LKoe|BxtKk< zfNU6}0Iny6ZcFgN=JyinNuASmB(}r`|9i+v9d_Zk>le~`=T-<+W1cEaa`2A64 zyjdIVjEhL7i7fGsB!fHO#0=OT7UI`>dCBX&v{2xI12_ASj+?#kPg^6fPlh3+oB8vN z9&XZ!Dk~T0DPN$!lY!O<1^DMNUyZRhbQ0WGTTq`C>XCNqZ6vRiW=6n9CnsXLiRPs# zfxple!K~I=39f4$6yNzaTEJLd-2L?qf~QHsjNj}h`hq(5S#Ci(thQpE^_>)-%?`5H zMlZ=y+SE6?nv~b8T`0#B88h`1_;66?-aEsTkfTW^ z{7AvU>`GYBW8iPiYf7hTA?HxeNU)Vk`W}++dOO;2wFUWUx(4=Yu0HkMQXTG#v4GFi&ns;xr=4Lcs6kfzo`i?kH}7%&do}^} zZL&1(v8p8Q(M}&Oz>M)G!25fXOn@UIo@qiv-5TX2FSilgw8g2<)kV<95^=O;CnfOm zR7u?99l#y>E>e(5LJ0leJSFvL5*M@$H2Y#F`m4M=_Mxsa;l)M=%2D5o_covHI#LVh zJ2EcP(9euAfc)6%r-hr}BQjf${4iCKW<6JjvY&6E`t6MiDTh*O!qx!V!#IWyIZ{yK z_sl)Ft*9>xwdl`lZG=EWFV@2(VZi|Q$8HbOBSG%F>pL*tHri32ml_dvn$o1_^4!Gd zi}ehjg*uM^OeNv-d=>8VLN)I5Y&ptqxq;*|Q;D)$YsNcowv&A}fv+$Q1AggcL~M4^ zgV)=M?u(6S@26_xU-oAN+~^mC-O!Y$yjyBSI&FcP1bpDP*i3L=w7kwwu6O7^8Vl)2RY8N?V^m~jO@RCWUaXNIZ1LwAG=D0qL+gyF}TWN0eJylV{ zqq%z2r_pTa8+9qt{a}m-;>b)`Zo_`lRi#)@eNTL>EslS@-9zv@oM0yFYT;g6-ON-Kh z+pTt_fZU3EF`gZGYq37@^`3+ha{~PQj+BSllyFj(+9^KM6=^n$b!m2U)k&|%Gea#@ zg)t9Tno*xM6==r}VYs8d2OD}Y$uz&K1-U0BMQx8DoL5>Bt!FDDA1I51Emqsp945;W zZMXWU(ML)FeQS^!vC@?McD5$!v8*7_qLu3S`wz6u#b#FE%?4tK#Y_d-VX2YeqApH( zrYTCcnyJyQn%Wawr@8G1PJ>)q^#`Fi}9r6%OZ$%5E>6WKAA zGgWEtn#k^#hj|gtwp$oMTV2fX`392HbT#gira0-zN=^Lp`5!S>YqhB#*Ba3dn(v8E z4Lu^XaU@r?*2RjPsfE8$<%iu;Wry5U=Y(6R^TI4k;o)%p6&d{s?O2&1xgoeYcwuvsxl_yxO06rPZ47;<^ zPVm|s5~9`ycu9IG4`=q4yJI;21)wKK(j#t6WF%P1b7St0r3c+o7DJy^;e0JXe%kK< zownYN_n6Fyf7n9{y4;NSyWD{DyWGYOzdcn=a9(I8d8kT~pKtY%1N1!vubn>gyI&9= zhSPnnk7b43QxwKO>E-)b08EZHN%;8nc3P0Wn-;uKpY~2s9Cv?M7=C?35P3tA6?d-^ z>Gel1C(3HB4(qnv$4OZ4q6E&>!QakR0L;w<^;sPISXZ6&20YK*-EJmqr6+@_swTQD zbEXsPA$faDK$)r_f7O+v-syfMJXICPJesdbwlfY; z!zBd??_)pS{!LRH<9s9&3#aOk?tSV0ca~b?Uaq#pzR;D1Sj<<)y)cc?qRR=+clGVK zm?NM!S32=-b9KpYCyPQY<%NN_N7DmtjAzF@EJitBD~SJaX{8b4sjW_W+0PHW+QkUE z+|3NVG9rq(*-iDi+KP3((jyMKf$@BFULuOH+XOkSuE2kk7eqgt$PTxVWQN}w;09mo zq557L62&~MC5Jq!N^$>txs@K!%TIU!B*$e{ex&6}ZHnzMKk!yN-tnR$)Azcrir_BI z#yihd(*3sjcu4@?zAe|oZDz}$Rx_o^FXcJ0_obQ9mbxo6ZT zTIqqaHMmc@3gib}75r1b(B*GcNyO8QHh#i#10zuT18uv~Lib&1Be}1%V!kXkrP(bs zq}b>xVxLds1zC<~huzhd)0{LFWXHuOlIv_W(oS6rv(l8pUP$v39uJD+A5~&}E;UpA zuP-)XJ(t@kKFUJqqtVQ0i(YQ%^^wf*d$Rn9`|9rzPgXn9zI2HFEL-VbmRtRNB&fki zCAgO~H^O4FF!nycgC{*4pTAp}Zr6p;@BirK2V3nA(PNkDQ|uOM0q->?+f7%-zX4op zy;$$}a6H@X=3*zx*Dywh>H8@m>wtrlB}nV}D(pwi_te)jmB^34U)qi3#ykPu($TDg zW^0l^EH#t8RAq>_6S>fb%950qGYurK)oxalS!a!A2H#j{L^-HR;nu3}KvTEl-4)-V zHnQBrCo?}%-x$rDQI+;-NErWInwRo+p^+9enVW1iQ%(FjS&Xq662@963X+~om1EwI zW+XnN2ijbmECRZ>h2YiA3BKJy@%nR=A9P)j7xz%}17^L^jc_!L<0D22Ltj+kzh0CU zCBI$i$fi!!(0qn66Q8$Jg73AHLM#T^0oOao&VMa5COa*(CfKZZVtx0e*}11`fpDQV z=EWe_z1<#6C+?5&&^vR~&7vC{4E88>hqqE%JV@JoeE-RpKi}ZtP3-_v8l`6@b{WB#5+|H#c4bn zX{{`RKhu_^z1IG~e{2yaK8pMD;9M^|_~Uk;AWc(+{oE&X{=HB5?e~7Z=bzm)pDVgz z`0If@=(Cgnn?F$wkIrv&G9vqU;n%yUo)?F>zL%w$A=i4DURS$l?$_pOlkG+DZ&xv{ zuP(|mQvFV4V)lAB^6P9Z?3JP*?tULT^zVAC$HkHKsJkuvXv;+BSHF!4V_z>e<2;A6 zq90FW$G((hM?GvHy8j8nb)hQl)w$`K2&+b6_{(~_|AUn_qVG~O32;lAJ*W*^OvkpZr9Y z7xS=(9eBMk$>|SPko~!d;w0;0Qs8rf@B2T%bfEEr= z)`GyKu4Ea+RcZ7i zU1`dzg&LCMXddi&Zi@4tm2h{URoGBzUgG1)oY;rj@6hKGQQ%*y-01uBRVg;}jp)yr z2-hp@kWYUp3(&!*axr(MJ?-;Mb?V!>Y7Ee4X>W%3;kV^^FslZ3#2s%N%X4k?fLGIv zME?f5|FyD|Prr2$-T!DIdS3dG`t^@as^it=_O!43)Ni+Vaqq5dbQ7cI8&MzCWw8(B zMKP9&@9~evb7CG1Z~||R3PSEL)?r+7(mbw}qJ04W(!=Bh&}V?REN6h`(p4nA7|)Hk zTaW#Gxu5A|F`s+#y_L~-Hv_V0r#qFvfRRm zoUO*X8rrcw|D9x{{gTq4n_X$&G!+p~O~YK29{4v^720mP7XPhZ5NFxLinysQPI|uG zi*ef?#Rm=-N4{o-y}2}1PW2hjMc8&Rg8v#81zXNlz}|p*cAsw`xNmingAH9oe|d4t zGfidE$Bk|gVy>R$H&>5yn5|F!Fk71jbUeygk{A1ax)$YL%?y2BO$~ao)JE|e%7}Z^ z%?iEN#|^&T!w$XEN(+C|#)-C7)S%r`{oh}4dU^{|j`X;DFwQ|4%3E6er+b>w$rkoWCs599N-#RjOu zWyugi3?l>n4s?>`mA@LTqg>npZ>tQUacg0TFkZK0|&BTz~8`a*4to8Br5uXjgez>bIR?9(th_@2aauew^>5 zhOZBE5V~gEw^4D-?TL)Id(%HK_G4L5_hh*-FPB@Gp~_N}-E1Y!VYP+qx!QtvS!u$% zuC+3fjYAxwewdS%k@nRB_$t>^b$0RAC@p>!c;2l(QlO!i;I%zK^xYk#hHZ8;qvWL+ zr@_p`*M?ppZ7e_WMGM*Kk|e|9+CpRGlRmM>U*qXv7K>F_$6>&OqOdnt7He_QhcXdu zZ=4C;86gI3bi*CztHYj6W_`OnT^aN8U`$8_8a`p7F!|X+E7fPR82V&b5PH3jGQQ3isP+`g3w3m(gZ7Q134rY?sBW09bvaSAf#@0Gr|GJMr?MYy%rlIZRcy^-zW;g z?#xsszh7+OM-^iHZx7}tIp_yy5vv^-=gIF0Z)d9G-Yqq!I?UB2yp=PK|PhJH@UOe@ZPFemX%mmvBbsKZ3VAS+>`j}>Pa;v?q3T+Y?N ztXEnPp8!6+P?aa!$}5uJk5?pl%(fGvJNQA*q*;*<7i&_!thONB41LHz;|LXQl8_@; zyVAb4vplZOR3Y4qBb?NYE`sM$6YA4QR_M)9afHQ)IO0yP(D%t=bBgD1e!|y`M8_-B zEmY)68`4b$^zU3n@|&@YpxZ5I=igh%KDT7W@ozUl?FwSQ{aHx}2K6Ll?u?TY=bNIe zm)od8>+SsH-Trja?vN0>(28|lYA5*V2Sw=R7E-`k6XK(~IMzy@8*v|Co#Ua5kJ}z* zB`kK*{8d#{$DQH)%&j3Yd98!sWiX$$ZH3wIj3&G2`_mkk0WO**GP%pZ_o^#d9_sI8 z$EhERBqjK-Wk|=1m57hOk7owmTWm&s-5O*kZV!QLgp1hdBLweBIrx2bcALrD4lko5b}4We9HD{QITmpD_vWQ_%fUo334d>_IQTZ-*aV2fNPQV8{O%I znFfM~^n3Kfxkjjsd7ld8%jdNgM(}0_KcPQ8?nN&r^z~LJm$)^=Ma?uPzuf?yc@yN` zMk~Q%rvv4*)kg)nOAMTEGQZae`)y~0hZ)Ncf6~eF_(N3|{bD39;a!&?%t~9HXk#2= zBPM}S&rWl>)t?0q-y6zg&y***9R7F;}1PcB=>FzSTpEoNgldWFnnz z3=|}K8Ae#i(&EHd%?z)*ed569%CeZ3>K_r$HoI}&vI^Lj4pHF4jR7hQ;HyIe`RkQ_ zzU!^2n#9+0ZD|g3&1s+3yRcp>ZImD|FRy0m=#jgUT=7Z|D@IeEYByb-X5T9ce$Y<$ zy0+Sl@z|1!`4e^Igj65v8_K#g@7*z0>P9y?V5uGNvebrknF3zAPv~{GneKXf8+h}9 z!W7pgvhSVcM!er!KhE!h; zTJSQ!N5FBuvptl+GOGI>MHb@Yc54nn-ywi)^|Inj5=PQu3*3Ib8R@Xx#fvkaU05BY z$IUgQ+OIYtK24R!zg}pLvDqEPhZy?lF~$i#Zl#wKJJ-Sqo^KT=$V##Af$sjiT#s>> zDTloRHUCgkh4?C~!h9(q`QB)xgjp^&2oqL21c{TSX*NBgklS56znijx@E5a{P=}>P zl=D^}GkT_(>@6)zez`p;K(2P+y_BUP4_awX*LwK=x2p-?{-`B5U7e|>NAwgB9KNG` zu8d?tKN^Qv*aI02x+kF}Z1&>)r)r_r+M4jE=CK~g*@)F1O7Lc{2x%D2;m^0S!pE~? zA9m8cuXb|-ANTNl?^NMFUhLqyS;#67_7yar+ui(t`-=_ufVDPi2#7V@G{Q%153*7g zx?mqy2QmIzk__h7P!?;kog1&KA^HLz?YK}0wcYC{1Z;H^{pRXW4vS4__vKcS&t?xF zzS2n#k`x5roh(JzFL%&`mOH3^+d~56?sx`oV~B^C>%@C+O7qM06QwQsK^}6x4)I?8 zJ@)ZfUi3p{S@OHhK1K}S$@t;i@W*|^Knr~z9lqAdik+`Veo+)hK2nz@yq;^N`Yp7v zV>WuSC@XEO7S`|STI#B z>YIK@Ox+w5VwO9IzPf7YhlTp&ck_)&cB}n3UxSndUj-s>sTu2L=*98(x3U;U zd1?FlSYFoB5Dz}nhI7(3AiwA(pjH%O(R2qb(!<&Ul85pAW*E;aUFsK+bTx?g+N#7C zvyCy&RFz?m+hPsXXSJ0crXLca*2e?{CGfmJpMKft<0CdY zxzNoXE_{1fh}#(8ByRLm!yC+f+t82U+Ff(4!_m((Uey()tFUQ?Hl6HXO5h35UD z_4X9|h2|8y*%q|dS~od-f1I7RJ4}gGltw%QT^c9P=_1rK?M&cuY<98ZJKDRoV z0e5sEe}+x4S~EPM zhlGHy1*ny7O5jRADPU!g9HgJ%pf;5S)ytCNvgLtv>{2HqVx|G@wAhUI)eo>z%hQ%y-VcJ_&VUm@uB*lKQo*ZYCq_g$IJe0Z)>pPqi`DCSw6=Rg< zt!TJOQA3Jq`3tPfK#Wcunv;U z*oVvYh;M7{_~@-+0d;#kovt6?C4+iT+V1D3EYzjgkB9@V_452~^a}kS$ciK0Z1f@m z%a~j%uB$-2?WB5N9v1uEn668-UTLTJnS1d!yZNarO{|c` zdUgcR=6=)7a2I)bti7xv+IF@F?>Ey)gJ~OZNv5&v5~DJ|V%1zD!@P}nz1vX;i`vm<{a6BcGFK0kNOzs2V(+?^_YPAwu_Fnu0bZ=sY*$_ z)`^GT)#Y-$mTI`c>T=BI=@zV$rUCh3xtjszIZw3SpG}+Z$Vk;Sa)PwgR3}Xt{mV=x z*GpH%^Bm8izR!qvKi`0jxjB(RwOy`Bk6y0fhfWqToCY`;D{ThZ*3gkpo$JU*8!EWoLX|LB0mj-&$2@63r`}bF z>E0VHnQ622=r1$%DfapiQuNL^n`KlMHts45>-1w{hO{*4o#F@F9{7b2{g4o;Dnr|< zi;|u$)WB_58&R&CZRF_fZZ34KDFZ6c=Q>W8WriDCi|K~eT*6W zxUDRxUmVS#&Gc~-=KI*utAm2Ht3X=`Y$GY45bkeLj*NUTCMMe~)$@{f`wB8Q+Olvn zWfa%t8iwyyM;>Xqnx8tJlWe`*O!8lCC;1x&*|6<@E?Z<<(>n#Y0|#w^#gtzLtHD zeyyrZ_-Is#at^192acxlo0huCG0J+B%c>+xxTPrVHfr*!jA|}rwgc@wQWX14UJ>_f ztsn2dr%cZ=PtDSZE~r9jk<#LyeHLJj{$6sUm*pE1|vb%_qL@5o4aT@Q9BpiD@^p zV9}TH;GPVB_jAHPzjOJ~5x)z={Vp;6oqwZ7IREDL!3yGM{rr3hGUj11Cgk2=Cfatc zffKUQ#e?f=s9w5iipzYHFiKG^3hv9I+t$*tPs@p@+j+2*Ke?e{7jT{)=aU^iofG(Z zUJ!?ST%!7Zxe)W={W;H&dEB|442-;% z<}v|lSy7%BJDQj7*2W<`t|h>3G!oP9w9!a!8VTsf^|<7_-&3Nmr+7G=e{=s1+IAwFPh zh)vuaXOraRj=_d#OtRn=E5EjpZ{PdNWubJNYQ82+>PLlyZBWmP9qh=3 z4qBjoM1bFwif9`n%s6FLs_k$_(!+^7!pHF(@|Wy{kgHIK59i>H4(DioJ{NIbUoKLD zT>fN+cwGYHzTjqc|6GK<%_W?_+uwQd0hi_*>A{;LV%o-N7JI%!m^j^lb5T}79cJ1v z-qLccy{eAkA*~>O?#)72{lLUrtt6A)f?TRifio@KJXJs!B0W4{!2dxc|1GgmX zl=UG2dZB}xINi*E_2;8KvQmQ|^CEoy%83uWLh$`|Ey2<5YG#tJWhW>2PQNJXv95s- zexwr>{IizXr0Ehsf!%mn&V=90!X#W`C&gVz^6`Rr*u8A!Aa2%Hka#Eib7jbg>yoV4N5(Nm;--`hm$wj|WX=5GfpW2TOE&#&F(Lk{ zI3@BDH!l1F+{g8t`*wUOa|aTl@|2wRr~l`063#>ZOYi&sQLq zzr8+}o9J^zR)TvkDMGyH5v1H3&Bnh{Rq%ZV3I&dp4EiHp68thIH0)fgiwh+5<7dcM zD=Wy;>wiH$-MtHOfBFdW;eiF@=G8wSzd<078-H8`{_`nhZ=JB^~F6@!IlK5HI z#0k{3=i)~yL;;;SEW0uy_WlnN&hk4Bd5s+%b|KOE>$yO?_mF7&50He9_K-lEw-DQV zmXK#49=ESugZ%#6Z;+=q{xZh`B1-W6eWVa$H`j;rT9M#_G)*b*WhHyo31LyoA_V-h#Zmc>`j7>jvc6jq8w`m#;uBT{!-9U)%lCY={YQxH?c0#ow{Ak5 z9zTY7zIXxRegpFO+I5Kaox2c=D_0;F&i@8^2!1d0y)A?j?Q>I7Nd7$2k_lVvD$m^* z&d*vK$RJF&b0Y?e(XSc>@CP~Q$m^7FPY@HgbKY;PA#d+mKt4Tu0C9f$6mlQj@9&Ej zA&p^x@sP*WfoEft+v!`2Wh{Cgc*B%bOs-gWtV@ zAUxeJkVD=7QdA?|SNafM>K-w&F`M|jjD@%*#wJ~6Cc-WfBNHyb{6cwzd9^ z7UFudk`VTwQ;_sp)5r*vR#UzW<&)oy6tF*!eCIn=^N^3eQ?PgQh?Lv-goHnx?Q9`V z)^8yW&tF2G-m-vLfakOZnECpS1?0ian~=YL{}b}y>UGGsCodr=_n<2!$oN;%LV}yF zo)w^L;D^ZTGvhn**bX%u?1OAX?4LQvk=l8*6{*b3wWRMh$PG8Aiv*S-dRIFTRnmV+P;Ily?-CX_)o~+zyA(-Zt(ze>#ysO zS9c#keC-|1``Nv}kPQvGqpl!%uJy6v=i12LGfiYSRR!^jyqx~6iBEW1!zDe=Lc*>= zJw48ae)$XuvbTr$zO#e8eQ+Q0*M*CadzUYRc-)12u(F2S{OfOk7uO)3)-TV+xPH3U zCWy7&9OR_!$i=*ErC6{f6H+!MJmhRAEpVU^@w~r?@;(ondK>2MdLiP&CrF5`4X9;H z$SqLIxBj>U>hL~@`yI%wD}O=mf^omT_XzU%_C1L8gC~##SGV8Ep}rQQg|x3z4Wh`U zuF}l)q5NEREyH)RhUGb4O>-${rryJZcwTU`c@D9;ZwdMM#0v5N#P`;pS3q4{hdc+h zc5lT+Zjgf`6;U1YVwNB={X_GV-=$T*BIT26MJAo4z43v#9gvWbI@ zqR`F?u2WMU>s<|>YnO|`KFCKC?|&zWtl9CD2dVz4H(-A8S1FOv*T^A3*EnHe532Au z`;kIkq^^k?I8}?Vn`&4g$=!4fJu1RQIH!@nOoo` z#4?^!;}I69ptwI_9?=&a-q=E3-n{{G`7Y$q&3lkwd*_RcaOnMV9Qor|ab~2dn(e#L zB}`iG%OWoIWZ@M}B4`#i>{dD~;BuFU__iyP`>8_6f0IY&Sg}$FcS4*4FMN9O7V;Ee z=Y#72->hsQF7G~__x$8^KEfmXJS;fzk`S5tv_*)0r>da4ZVYF$mV1P-%>f~9SCW}$ zl;so{#&a{5M)F0HmV(r#qVIv3RPv+b=;+I?A3y%)@x~tF`N19X=E)n#jVsq7eji*f zVq=k(ECk8AR>1w-DdswB>I*n4{l$fIog$d|TJB9puk@y)X4;mVA_wmhbN z5gY%Ql#*tV;1hB&%-QF>v)xxvBVQ|YzMtXp@C6D+2 zIQq)&wytgM-h*vV8PcR_n=}nK%yAsYF*7q;w#CfMW@cvDVkS$LS+%x5cM23T|Pt2o}}Zi z45BfYSq)9!;3_J=gqM|l4k{}C;?3h1|9pD=_6O;Yo_>)1BKa?QuhRei_Q~`AX(%cA zh}_X~akCenvOCWLdqiCnkC=~jNf<4ME7VHu5cIBbij=dn(S?(Yvhur7&FA4ISs&#- zfAnF++mwGo%JM!dd7Jdny(^bKc=*e$zg6Z|f0Lh+|9N>%-rtFJEnjjVu&a{{^dn^- z^of{Van3RUdUd?T>y2BP96%pddzBoFOVcITlXa5CBb3)&Sje}Cs>*++-+%N!Z|~gu zp!!Y52elb_f6ILS;_vsaT=<~i)uX?_s*67AM7I4r&m^Xr=Xel@m<0{1>2SZE!L`c= z2af;&bFk4%+F9#CTc*hw+9C9#We)iI1h)BS+vqU(V1Ar&xHLfUY<7`+G9k?+;UXL& zX0u-=FjZsoR8My~US=xu?G-p+tza>b?S?KzNekw-Ei z^-wIp*rsUtyHmU>>r8Kpa%!MSy*SH}O;6)jdk6AYc)ckTWb&;pP{S8PQ0S#izMy1p zu^;1?^>>D3{k=BcEi&ql)&yYdyr5A&)>SN< z7^szw4wkR?4rPgZdNMY6%;)n&)X$2+o-)xuZ{akX`CvvUNZXw21MNvWiEbGi=9bc% z?d#;yoeA(m_bR6JH*4=|M9ak6mf>%AMCg3=BBOY&kN#qDfS zM{0I%;){0I(HWAV_FpBV#MJqIY8Jer-l^a@8vLyb-SM>ytgq( zPqwWNaowtEd1R}1CS(-$M6CU*(e0jvkWtW^xHsX`(!h2(xn_GApRED-Pcp>j0 z>HeF=QZjc;&7Mp)h&L?;PZD3!+ z=16*49`?4p{yZXPB0?$_!Lg1la*2?cjx|KOPX^?wlvo*3(V-Eeu*;)hvZCwC=59GWJ)|8O}{YiDM0$@0K## zOf$@4>x7`rC!O95Z+Fl8b!0dYt7Sepp}{YKWgacm-3rLy1!oqX(63@++%gU!Wat|9 znR=)9)O{lds^O`9#n{xotcSL>K*+ExaqD(PU2waM-VilX5XS~CWS^k}a3 zu5GON2YqkJRVnlB`Q3@C7ZIJH$GF6%hjlD)h6>+Z1|_X?a=(!B zv%dFAz!hhEf+?T5dvbH2F3Baqb|h*|9-FwlsGdy%aHe^GwIfA8T=nbLbp74H7K<6! z?wPj(O&J>F1MGdLTRjn4(;Bhvx( z;ABWeZ4F59#m-f1_SRtQU5^A)rkiMaq8)6#J?%0oy#qcT&wNR6Z_hI3I=3rncTF*s(EBzM3l$$J!4_@z(i2lKjf1$;nuN=YX?&e zckBz0m-cl+nWPJzqwYaec*T4YkVDjCBb$`yX%*L@tK4f%Nn2x?7lKN3LrmW_=uvS= z+6l-DuYwK>tNE;mx~nIu^2mdF)RP5(q#M}i)$wmsipfFuF=Sahtx)CZ*Xm-`g-wy=(cjhktRKaT`ci5j<29R(UQ zpc~FCh}yUTSMQTy3L|P_Lwt)x^UEpi;Y1&>J#OEhHWvF2D-9n~;45vD;3vj`jt3qQ zts|=M91od0N9{^debCD7iSG8Sz1tN|{(FxlJhL*`VFR&!XSL=|*obM08HN3Eldvyp zS{=dEb>cXr6&dwDsJu^F)hMNSGbuM-7*yPbhmrOFTs_0 zrKo}f5u#vg0g~la;@jgEcK6ACkI1PLc1_U=FFH36jqfdNwoi&JjVK6J0U0vazS{KI zJW_bYA!E8f7bu2eNaEZVfzaQ{M z{4xf_Izg*G9H-PAt9yn{jVyAa51@Y?p5mB;J>Qvbxuf8fe`D-xx$Rsb7rU0wX&w9t~{W?RfJTehQn2O z*8Xhk+xHgXP|VOh;FogXfem7XW2XIKa21(#s$!x25-waZgh?Gk7oR_x24}w0b6H^p z6Xq4un}AqtaEs8DemSWlq~?(9(r$_XTy$ZuJ@0{A#HM>>9E4ASDfY=>8GboD+a+pG zajrJJ^hhzK`yx{L)-?K!Tg;)v&As!9I_(|_1MJQ0T=#uEb(v^&*$FCd}8&$fE!eM$zxzQJhnYlK{{Qpj@48Au>L`%X=4T7pNt zGGtz&KS82rw@;2O3288uF%7Zx)X+6_tOdrcL=9zbWOH zZ(?c}z!oU7M|MqDEZPv6X;5 z*yLdQ(-66boy>)koXtF0Mi%E1~e^=KXZ)+5E$ z#7qoqSVL$G>-k+F!@z26yK^$O!|Ok>0gqd}Zoh_!vCSb|j4U-+$a_JEb zmln}8a50l$>}4%C1l6RLxOLbNH1*BeWlXR~Mymf~mrVR~mx%pymqPyE zJ?_x|?y-AaQYyF`@%H>QB<;udN?zAHKtG2!+FyAWYOe1Om0h&-RD6Fh+H}{sg3Orb zRbJ&(JpD4XOsYAN^NH^?EX=7E4d5xP$R}xf;aV=c`n!?bk-*!4h75hb!|IF}$q=^$ zRp64K^8;#hQB;ko49iee_9aBRV~N{ppP^RyR|xrG5wXO#j7oFNqLTgV!1Gm9mUFK8 z-rhvTPp;LDSAGep>R=9A9bBijIM(qcd&`Kly*W&pca2#e+GMnblr*SYLaI*W-icvg z{tVDHaXkz0iuh9J2DadE9hv6ZBmuE35JnAb2B0~}ezi~#)pd5AYKhIK8bp3n(VFBD zb-c2#LSNgLAxR!7xxqMwO7_Wkc3S&4mn|y&8>?_#pqZL$YRKotT>Q_5Ms-?um(n2+F}}&Xta2 z01FF(8ZI@U<`E-i1~+OV;1c~w{sZ7{49LHzfx|ho5}_dj4ee8r0NkN=M2rmFnT^GK zXJJyK8hq;nF6EqY5?uMtHXwOtO<*?}awuwe9?(JZgIZY8i3!v6`wo9BtYcCDA3~3q z2gE^5r!b^qfFo*Zdt5_q^)7cLdF5=3cBt*~_7FVnol3}!Nog$+5xz9Eiq1V8Yr5$N zVll8zDBYWbydK4-ea0%d^Hl)gTi+(R{%9WZa&Nx=mTkW7*489|!?TT#{$t{D4#b$c zIZp16yYry@-*&s^PPW)AAa9$4K#uG!G(BFYz4;=s9!N8Yc+IMNdC@%!%keGYbFBTX zH_YADKN>qLe=rD2zcUF6zjDvFJoc><%9SJF)Y`YV{xt*5z3vlHTD(#wDx{=CB1%$& zcLSWZGhKeewceT()zR<)84&_t7cHbFHV5RWf`|&19oIm!LyC^H{e`-xdsEF%!*T{T zr0t*fD5!vLK^2^+u$56cp(!9omj#vB5|0>_WnV$2>IUk5-d{rIYeqnMw#nw_VL7?p zwF-aZm;>GS0D3RBi7h-)lUn0iazbA-PmEmpJHRU>>YjKu;F$mqKXL%x`(P7SY#at+ zZJmmCNhnDBCZWy;=-Rl6NeC*5ja~_+A}A-d`ZtLczIAjSz}ML!DFu{R6B^YCxKU%z zXw)p|bW3oBE)lliU>f{XP0#%zyn-k`)A2ha044?1{{=5_zXlCHwb2;wY$Oce zgPR=7;5Xaj&3B!fxKir^u1r77EcZ!9hQi7oeoV_D#1*8vfP&oQS;yuC*2#%l2MG5( zGTpz7D~^k)jR&L6x1BT14{T#~w`IKSuN9okPfeYb_YPNZCA#6ZhqmF42aZ)@gHuEY zc{Z5f7?5wj>4bcT2vaHUEWWls4J|qa_%Hy(s#li41wiwxz%mqq(py`D&5zXst#=qz zkIy+LP=(HELY9_O`@M=;@}+`a{FOma32<2D5B_yzx^tOSKFg}PoqGTLUzkPDE`%g3 zj7vi9I9!EfJJ&(4wx>#dHVx-q^lr4J>@I@dD1}A8grw-E-vHec(ZKTk8x2pL^X0d8 z$FnYIgs(r_8?C+(5tEzTo6L^w8TiY@I_5JC6Lq5IQvDJ_m17x^Vqbv01Y#xoa29l5 z+*y4KSN7mv3)s~2QNV*AE~3)+#u{!PO;rEtTWWq_pR4+HZ>IRxu@>7NQBq;vb$D(R zi2Hz)(BN2uW=FR$Eyp$zDh6;@KtqA)$6FrkF2HkR1}5ed@E1-AvD!9|Nb#(pGTqCN zSBJB$uN1tZtHz#+Ye6xkHV9~zh@RgaHSxGWtk)kdz+c%08?IP7i@psmcBH!I+tTa{ zP0vnD9Kg?Z@&hsoWOuUVwQI3Gb!V#nmSwW`hF_NGE7di1Y8DQ~6LqO9{GO978aC1Y z*Ru&p@rYolelhq>REf;<13cjXh&NH3M}3jNb!zg^-#CJ~0V;kIg+(hij;ugEee%P%RJyblrUiB5Iwc z5A@R9*ZR!2jLANjYJTWh?Z^P|Bzt$R{i%BcRTMXou`v^>E~z$ewSKhWwqqSp6427&ekH9vYG6@MZTvp30+}CF!VA2ch`fLtR~Fs|mAo4Dsn!3jV3J%66V{`-oemYu)rtZ&yo*gZ=J&9{LOs|&505GUL zuA?H)tZW|ew`k`wzR)((dUvb8=CZlB_J(tbQL{Zk&QdVzzIV+59)Av#<(!4RJeX{| zYaFP#B;%K!Tcs6zHUfV2kKJ)xqkC0|*`5Wz1m^yxa}}H#l;WC#5`Z(rr0SiC)(6(! zx?6EM36_8jbzkGKakN4}Ev)k3bwl?W@$f!x>DbXAxiLm(Y-bO>A*k z0?T@DB!N#1R76b2p`I8x%y(M={%XjexE|khtS7X@l(_O3;In*-txtD*OTV{|m0ya> z5P3ck6SO;xOm!?G(tI1}9QQgZ&%S_8Kb(d>*L7B2w~e;Gc1vmC1fB=9Ozg3VO^vGQ zkP{u3=2tSHAr+x6FxI^`84k)r8+3a7Vb?p+}jzez7-HbQbS_c zTgPJkLya)wym_eL(wUizdABVbji`|oK5@sJpaPNaT!#XW*vcT_8+{UR*5MNL4ZvH) zE^)$7aaa)@4H4JTV1P~n1!c&5_a-!JXAblvs$(*aEhD<{mar#k6TinJ#ud2%eob)T zkqDCJ*+3R1a^jtZ&9*Jzb9B8e*FEd_nv*RSDKJ{_X-Y zWr0=x1G(V#Un3GG;%EVxZXK%rN!L~Jy|K6UicVPmy}q;j;?8LOZNUGw?Jb})wkl^x=|6QDG~@!XFjPswaUJF48&zeZEO5`DPVW_=SXAbwSN3|H0Z{edlNr zT5zyJsw#W_<6rOvPcGPJNyxZf(0MRhaos4${KDLs_qB7S;lbG!ofOfrnA7y)M+1-- z-~O+iHy+dDT0Kk654Sr@zf%#jK9LjC{qPDA6B7-43gq>viN!uK0-v+d_D0T3{me34elx14Ktoz4+AXCt9IoT?0xB9f zrf1+2HP`fCo$PkbN35J)kA_eW_~tebps!tGWKl?qt~s&L zaefUQ;+9bwLK*>oe~At|*7Mkjet0nj6Y5)|)wq_i*|xFfn`&O|WzRCH!MTXY(D4f| zn7aypNYp}*urt5k1vG|#9rE_rz-BrmjE=npLUv;9cTCNIpPESE_uC}IdmEW>vW2WY zT4}oPTtQ?+)O6&|DzwBnl<0HLf%`>VWK6+E`93K{p0Y~{ZvMBn(dLI|dVZ&C9hPn$uDNcVXnbH9 zZF;aX)bhj&=&iU4Q|Va8!%|wSV+oeLKMi>%+nn|>r|S8~)3EGwt~o;CGN<8JT=ByX zmBQNO6BW6|*k62c0CfMa#GIc#SVuhk*gac)FR+fSIs-g`Qvxg24V2zG(KE?sR$gaN z0cdnJvHmzw&sl=4IGk&~7gN&^zikT!kBv-1NR6)!tC*Omf>0aUM3o**fs%H|>h74j zYcKDP!;+S%wfA{-uP*FPf?vB=$TjX2E-bi)%?&Oi^NuFr1xkL+t^TIx|CDp9Uq>Vy zc1S{QR&g`FG<28UIGk=xdS@oWkM&HvUqS1T57xg}X5?N^Xs&<)U+Y~(WV+_z`GCI4 zaLxfbX0qjuQP6m64WIuVtMK067Ex(8{ZHn!7FTLCUZ@pxGqBeNkNQDQ~q{bM4S5XZUxk9P_ z`NYg+pXgYa6D=DT6w{mCtH`&G>ALIgnbrq!1+~q!gnDbAgue`|VG2SLR34xy9{5+A zANxdrhLsa)_BY6t_6=%nNXMYX3Y>&4+SMo}}6Eli`Ud9#P z*c)hg{tnOp@AWizTt`QlXIr1C`%CV=x3IbIZJaJQfGhU-hI{tK=Eu%uK!PV&rvtH(+mf#_F$2Y1yA>1$kd>4b|VY&0;b-+wv}eUf=wOZK5p` zh;^J#N^EqkG(A)c^MB}Ve)Ji;Gn|VmEo4u53w0n|A$v} z{jXcX!e4(kQjo4yWL8*)tB9${O($9wCalJk2Q-i(hX~MIn{7!aMkf8erF$~0Ah#b( zSKjhV+A>e{6gco*4Z#1j`!@--2Q%Q8z70}hPfhzWx%?skW7iFRO%M0S>#hgZ8eat0 z+tR%&pv?V=x+gM5@@FA&du8If@ZoIzox_=q6yQ7XiM~T|9TD`0?Z16<`T}xto3_91 zXOEPI@|#C>5$muf2AtQR2m-`&?H$ij>+7%#TlaqsGWER)Qy*4B@=kY{UGcph(TRnM zKHUPesEGmrbX-YDg{wRUxG5mP)#*DcZ`fso8r2Z!foq-6e5@5PT`Q=}-Qlty9MhG* zxaJ$~+xkm?axWor!a4@x0Pr~WrS@mR4MZ+*o+<-Maue`AGHPVv{(J4(K!Jq;jtl82 zsE{7uM!+BbZtI)|@_Z6_pJ1D7e-TxYuxDG{lP5-j@C?YCh@4R6T}7tbXJ9Y)Ceg{J zQPc}Dui~<067=?H6<@GMDfny|o!IYR_Oope{>DB}s`joB8X|IHW1>&Y-V!|70pMjo z$D#P8#5&J<)$Q%E%x`r)Z@<`}z5FOF#gxa5Y=GY}r90Cd&rSf|i~(2{Rk9Gr8g!|9 zwfX+heDfXO8uYbu1(UwPuD!U)tN!_LjnZ-?;XpNgtvBY7X&(%&EIOQSzIQa&`Y@`(RGnEFwBv0iBjKHWGE@~Iy6rRiz9zK+cK+`A0UjT?yV>Vd*b%AWKu4Z{`JXRzr%PQ%}R4{$*?;4!g= zNoYRo?ahz(#_C`EXP?17Heeb&qSj>RTJ8OS1keCFGC0xG+%2It0hsW>yMeEcs(DoB z8a&N5(QwT&RDWfIT5^tG`Qnq&_T=-fIZ&o;w)F+Q?!`IBI<_gQra<;*Tb^0Q8m^f~ zYc5+SDt`1!nxDSkVzZ)p9?LGGRE3n(j?*n(&+j|D-k6ryYU-=}Ma3*Tx7A;J#j}pi z^-35m#u3=#u#!Ut=7h30R(or(;m${sphq8y@hSgQGjhKOtU+@A_lP~@R6tq<^>;sY z&Y{cSo4QBen;4{bMpVtI0b1f*sd{J{%DHGB&N=sfn?DdT53YF>0-j4msy;IdM^E(( z;yWW99aoW>!jks%quKI1p0&nTzwL1QeOh6UOUy#VO#IGcD}xkK;;MpxH+L;|B+Vky z&IwBIe&p3~sfX+M!hnj@5>QYR`*e~IfmpInx7n+4F8{ri!v!9> zQDB_0EkH}EaOG|VrOB`6GmchJ34ij^G61@5?QglJ<&~b_8L5BjT!Q9UCfe_B4dY)O z%?Vo%W(2K=i_C_DWqj@tpa=KnJ6_u6YwyJLgoXgXfd?CuTCarG6jlfr0VM(or_aEYL4=F!HhyEE`4 z=O(@VaDxTf8X>1y`{@~GKJu=NRP)_Bq3|n(u=0kozvlKhHv2OPtKo@zg;Z%BYk6qw zF8;|q-~3{Gr0TZ0ujrCbBx? zl`#TO`(k&v>He{fMTlxxgpi5`;2;s?mOu-w(=88G{HjZTSQ0%rnLMWy;MRmb)-x&Z zO+AzEfFlO{k8>57xj)*cL-5y55uq{y`0uES)*cfxI>PHTkb420C+6jS zKZnoyJ}76P_U2H9QeMdg_a-tQz(Yjb!la(snDkQs{~~%m`)G|+6Hs!Qo(+0KOhan& zNnyEr(@hWTGvHT;GwqKOwY!|N9U1Gif-48}u%f*MOzkSY{HByqe#topPLFOPa*ySd z=7@v}*`I65-X1S`WE(C1>BK<6#|<3z!5k=Qf41`K!F<{Et--Quj-}?L-)-0qAfE=J zCI(}URsLucSA6?yyLb7-%}igyxqmio&t zZV#6HBxj|5wbfs8>0lE4Feu_eEt8N8si5$}!5la_B*)d9nQ^UW7Cb0!rDCEwE<0x6 z3F0Qc@YpEqd1vigN%X;cXW?+;W(wS|MHB=L*lM>HU!|QzyqUrmeeYQ!RXOGf1yV}s zx&FEbf9b8d`&TI;?Muf*DLXlyXkT*mq|9Og-e*Rz)ni7(+F%cyj7t!Ks;s#8u zM-KRcDMYeklL2u{ySQ7k#9Z4VF4wtJt&G0%BI%;WDX6!fL%dg{=H#tRwrvL|ghNBOPzQ z>}yW`d=Z;<&Md6>DX>Z|-{@?3Fo4YZdT$0_?HBX0M+@jo#|r3KPzHMC7B$?B=-|a+ zEeY(BP~pehT{A%gxjm%E*N3%;(x4VqUExys2s4ck;O>?e8vfWZbon;!40>tUn zh@QoYn>cKr3{z?ut@uU8%>Pi5luL4IS!OfF>pCnyM%npN>NxePJAF zdeVo?yY%|%*B_WhYm)x35#Rx7L(2Yi$!%><`OmD%yZ_);-u%!xU;FIOBX*xxiEprr zU}=EA1LBs$cyDEM|JbL|e&0p69W0e5?@m`f+nw)72gZXqHnUkF9h(r=v2bxMi+rkL zlKx|*;QrhvB2KM1$nGrY`QaQc2f$W(NZT_M((*ZBy`Vd0;0r=3HqpOqrsuIBJ+<*dG83uet7~qVshKYV>?wLF_aam_34*fXsU1mZ{2+(~l z6}3e_j(xjAslRR+fhKRTYJQ!AUYC9`VN!yZ6l2ITg1GhXZ;_;+`<^beq9Fvk4}Stv&Nmv#>7= z;EG2{tOw@z*&3_p$7xK?xnre(7EAPQSKuq6Qc9g;7LmL)RQ;2Au>6O;nYLH14NB7r zyXvy6v*u<<320pn3-1SfbwG`+2&ixs9yzW|F#vij?rwY>vGAtiHeRn!NvL#*am8La zspE8;$p-ix<5SSv9UIvE?Md*!0pUw51#_wime8+S86i z=z?W-!dF(k&x52K6kwTfv0dHz{=<1p}-fhP>KY0_MQjJM9Ku z`=sO==Q1+waH`|^-YDqF-aI77vB_)`b-~h!RY@0irW#*(H=AFZN3*^&3}#*2o^5=o z8mLav4b;5xZ^86tmTkkH4bjuM}=;@%SV-anUck3T+~ zZ%ln}WApcyFnQ)_Oom6vroOWbivMHjn)q{ZOpo1VS1P)KeS5#*J^GQE=a*8v$gC$7VR{%s+3{CkVo@jlU$ z*+55lCFGifeh#X*$LnI{hRn2&n5@CHA%}K#}$1o>uP@Gm2wGw6}3HXz&FQ?==!i8 z0mLP$z^|pY`E_(y)Z953-{N*21A6(`(!Uxtbq_~Po&86^{9C6$_e>*=w~fOs_ifYY z@&Q=#c@eAd7yk;q+BJ=MrQ_v(EoY>CY3?h#u`}J4G=tCidV^bjHwJh<_a>^?GTwOG zGSztdDAC_SO>PE!1|w?d96LMgk%r7nl3UIIdn7%aux3~wDWX{*?guHJo{dG5-tJqvk zfBRkQ7%C^I;*nfZ8YH4)k>i%0*`QI_?NX2%>;T5^%t4ZmR#7>!&f-hKO~)_#uWP{$Xve!Rp4A{OW7VT zzo!wt{#HtU@yYH`-MzyVe37iP^aq`=;_}WI=mp?&6Lrli|FGfPPmSo>qxHIb0R^)B zHvp%9vvtq^+sffZHbJigEAa|^u4^Z_PTW% z_CnDMd9pQ4s`9G31a1YP*0YH!3MlBPxSmBl+v4z!&3J4?i-rc(*k->HQyx%}JC3(| z79(4NzJQ)f+g*g_>@PMv2}s-D1Qo~v-3Tmw5SD*g)eTOu&ogVR10DC5kZE6yw7mFA zgnN5Y%g#9Om~DO*(~^h$Z=INlAy1Of=VdCQe&xcg_+K>)Yd%Q~{`%Q!v zn+TabNh$idAM*O**-rOD%rdYU17bPR&wGEW;l6Vjoa~ojO95?!5O>zy8$+gly*=5Q z3}~AsFTgb(5x(3B94-Z|)jESpIa}?UM05d%pppGYCbchk8-q zZc{ViUKJDS*A6cqt+R>yOT@~6hE1_cDE0ErwtEw>!t*o8>>q4Hb-zftg}>+qYEupt z;bqgPx941&;L^B}f{Ysg-@Qen#w=uVSdD6OEhl&sR}?o32O|dIxKlzwC+=hVGz^4O zMyj>TXdn-uIl?AkH^AS62O>)S{xY#DW*L%)&7<0=by)k~+7KpTPe@CL*Z{w{w@Itp z>}nJmic2xJ z;F9-tg|s}@iCNGcHFBBOIaJ2Z90A~ORH;vbuXU{<(sxI|kHz$wYpbO4OTf9hfi6w? zl#u`YGecKvmUji2vprV+lWnTh#~Cwd0XC&g97Rm8^sn91;e1IM2>j_|#S z%RbhU!M4eYn|o8$clK9MxdQN;bBmPnYx|S9w4IUm2d3WgA62}}ZXwfDD%Thex>0Z%MK=Xo~Cb#4)&$SnrvAFee&1^g`{tQ2&G zg>STrs$T9dl|6H;x4sT)0NrS%6M_a}yHg6w20Q@&y;V5+&WdY|8==Jk88j;j z;Co0;s&%fQGj}H8&+N0L0w9;mUBK&@5?66F*OoMmN&Rk~ocHr45YLO$nm6`2TEp%P zwZt-uco{PSn%qQy9~-ft6AK9*13ufS!c>X6D}G&I6~`UI7HVb8yb+YDgF&) zop%YG=9+8E^KBq2ml&m&CU7~ocIP2w`^%UT>jWm*HidY-zlh9qh=`?o(}+YpwnCQ_ zSH02G@^FPy_0*^A><;RL-HEl008Z9>6y(|fkZ%zyR}jz(1iA@Sx@rji(xc+DcGjr1 ztCYH*dK=Te7-)R?uY>X0tHI6o%!m?RHHyjpF7?*=zpi#wX2iExl$eoAcWk0c>;TR= zH|k$FR~u8qY8pId;BqW8&;k*y=o`}*^rcTD5Jatl;fRgX=}+h+6@Y^(I>fw4%spJC zH~Un0=-~#qG`7X=J+=&Me%lp}CwfRG2& zC+fF5W}wgahFWiI_168Q>utX=L9P9+IraMA=JADh9LtCd8Nc{@DYM`!+i27EgPF!( z{i4=a$7(txq~;OSGtlgg+#mmG>u)IvX=aRm&A23H?wdWaa`Q};YJ>rhkcV-qlrseZbOkJaie2Ho)(5MPqJ&s5TRS_kr zQOv8oVj4of^lg%Bd}36wb)xB^aiIQ}-PxvJ9UJXQZW*!OtDs^7S}y%yoz&n{a~J^~ zm*J5CIijY4<2DBHnhJGnk}48)j{yJDw7Uf4{tP~IYY6gW37z%%X#2~r`&u7=FKD>` zV?QY6(|%O>Ys(TA?pG5*jtxwOTTTc2)J$|lPi*xi>ha3(bv_xs+%f}tuIaD4YMg|> z_3F3+hYE*wsIV=56PxK&QQ8~|YO7PjV}=3W=azL8>?~B?3Mh$i>kJ|8wc;UlkNS0!tG0(M;@CCFA_d2uTXaS#P>2JHaN+|kbq&?%K(Ty*6DoHLM=pa_#2*hFOq6Z-}?V7W)D*h0euE_IerdCjv(B}A=5 zih!BJ4g(X&uAx)Gu^hH95bT zJ15Qn&mLD1Y9b13XARn`rDo&|W8uDj-zzO($NfC!culH(ggQf$ewnoyhQe+P{3t$wiP zhPJonmVYC04Wo0!wTUja%{M;`C$4uHP^HIaOw*tHRBEEXc0kHP#FaxcL1`z$E1}dS z)_vFpD=zChi!N#fg+GR5*gEN8{j1i@>wg#2JpDAPr6X-)h-7Pj`>(r0ZFhi}ewg6r zfSlSIP_W=TOKq7O+>9SwqPFb!+dvK)>FAJ=L5LU`q~G`WBS8(KY;O_taE4ZT4VwGw zcdo>GHUk#4Ki!h-m~BluR>F$y3ym)gJ;hgMU@!loYiH>nLlB^w*nunA{dblmSdEZ#dK6?P2d{=sa` zuj-!SAGL!WFJ?()SEs<~ANSR~{CjWpiw_4HlE2&@YD$XAsNjPIM46mlab3bJJ?C5} zR{=bh>Dxe*cqPaRuar;+-~jP>n@WmVDA>>zlj=}WVe*l>JJyAcweRbxR&TS?U%x{oGY!5_h%dLL^Y_IKOMY@ zkb;NzZF1VZYut9{GOhez9FwA9SAM^O%J@)0%lgbS+wkaU0-SE?Z@Q!7SD*Kb@#Why zjn6HkRoAyi>TXy@%C8(wc038J(rVuW9`D^YjS)7}uzmv_88vfQzuACTv~*9!^a8$L z%4i9R>5Xw23lf*|aC_sJ)ZKpUYu60B+SJ{4XBCt6nVMVri%L*=RmLtmuN2h$usx2= z^2>QxpNiII-$152))5(AU<{rOVqy(R$?hz+aEV=ii&XLQYkgl`-m$fJ*)1W~?#_T; z9xXxBB1%F{T#KuVsZo_)5x&YW(0cc1232va?->JfiF2YOw}s?{>ca)(8vq|}?M&iR z#XZQ!1Muo^SINykCDt?RyJ{{kqF;YF((&YPqtN7!mhr{k09NRSt-jVL>d}tms%Mw~ zhI(`5!=R(`wouh!1uWAv+;oRr`tmb;!Q&5)R*2wJ9TR^nqqT)MuoXu$?P(j# z;#(447U01$zY48EbKM)HW@A^=beBkF$0r$xJm`dg{{v+3UX6SPk|rJ!QN;El|M;2 zd0&r!-+VKTO8?S*baH@N={dGu*1tHwVlGSf{8m?gk*%|xH``YG)2oVxHt`d`l-6R_EtBt;7ZZ+s~?Ra z@-NIIa{e_9O8&conDH+K1@OYqmmd$drF_%V^7?|DQ+{)kS93q*+Sebnre6Qok%$1@ zSw^HDu7Z-ndP2Khf-berw?DJ3cDxP)dHcT(R`32iJi{+xV2-!CCu7#0h0}vx$-6_| z_}MlIbpl|NbFt;VeF^+Z-B){ScLtv3UPfiur$H|qGc74cQ}r+R$EqJilq}@I3=Y7; zmV4{8k}LZYC}8|Gm&L^V&lln8|5}2-`ACF$^RbCvc)>o^de1o7c#qfq{KJ8c*Ppl+ zP`REpNSa1ae8tdJbltO5d?z5Sycaj3YEQP<>{DxEzaayEtmn`Z_tj%sE;*{CG>0~c z<-T=FrLnj5rm?f*&L*?@?nqn4Cqpf7Kj&Ay`qwlp`%4YC{IYiuk{sB?R@#@L1xKr} z9IuQH3aI#$sFv957PY^Y^9#;xFf+g08m_(OSw?0>)l8fZ&?kNcy*;YtGa?!`F0g^j z_H8sjwJ()ivy1_HWDc5gre)z{YBtfiMyige*rbR`KzFQ=i}wM~t?O;ND;G4~H1vUP z?M@+H94^yqRKkXz#mv%+>(r8)Tf+^v03N!o;#b|>9&69gjKY%n&Dq}!wB}#)tdZL- zqfNIpg8FO59?;|cNze=LD)`m0miGTSI?KN{vu|(H&a|DGcD$Y1)ZK-awiH_2-Q5Bq zA@1%8L`Z-Dad!`pkYIt}uJ!rJInRIcg!7J1NbcO(d&%`p_F4<{;jm`FLk00VvQFAL zHSu41;!0_<}bTJ|{v%KH=q=f^m)&M$0!g0FaUau`wgXg^37u36yu^8<~yr-jv* z?WFAAC}j_RKxAGys)4*bx#IxmYgn~UDYeNLrUoJP8xDL!KwNgsMt=Rf4*mF#(e5{A zB0~)?HtqPXH8%-_sd;hUOv`@$-&qFta9;M;0FAKkMV77V+LJ>3^7~#|)}`&3ZdZ}Z zo)067?rpp9pmiIfY)uDFSyBShJiLl$I#lLgUS`p`F;U~|Ee8p1CwHIVc*E2?nSfGAl~APYCF)V65_ zEXC8^d^@5*ZYTsWL{VIbj8}9yE{UFF1zTo?GQ`!d5$4CtbzT{B7q!HG{pFU%KA1V_I{aR_kEnE zF%G7Qs1G4z-Il%M{jM8cwe3N*O(@zj*KDZf%~&k*QA^GbHYXex);v05L8bn_WKl@IL z$~m*B>U=vVYkcTsmEIWPmEBvCS3KEwgG)Yz8Jy3P0&UDk2G-oT>NN+cVcX3_?fKb+ z4I8dRB*1%JyV|*1Bb=((qWd2Mqd;NWo-SNUlYrX9tl%Do6iZ2Dbi*LCY$(Kxo zgg?i6imz{5$t`PUa{H17^zN`fW>nVtdR|uZXiZ!5aDJ%d(YmfNamNSE+w&4zKloUL zbql(5+YHJ2>_yfexPbXvPEhHF6_)>bl+dy1K$XqJG;NGEJ2%}t>XIH`I@RCy$Vt!r z&4kN26=SCh8)k4yj4^Y~4|VdEVQEcI7T&6DGj=9i}?`HeRseXzHS zN=Dm^l2j{4=bfJH!{n`MnJp_SXzHd3lD2L`l`qC@V%dqUbjGwyZ~&OR?IQyAgGBI$ z5Dpa6)a@U}P<01m=!Wn>%VQ6({?@z}R{C|8t5|nnDi>6pNl_UjH8Kp#Z~$~bi@!1I z?kKymP6s%Zcem`=+HPdg-!18nPPu!4f;C%`EiWH zUUB0Ah^qL%RD|3Y|C*sQKLld=({*^xz2)hor1G+tk$u|B$i22;##BH`U;ha&NV>Z1 zW%Bp@n6_X~$u&J9=~qViz3&9o_m9qqO0FOH&<)Era%)k-%@g#dx!+J^3x=l7|W9}K5x(bG68S?XO5>?wqx<`-m;{R(?rat z2&4PUGzoh!iYWg!fdhXIBY?{eP{FDT-@Ir*7e|LX-*}kC*Io4NzqibQ_cKaFnv+@f zr=3*%hXGys2N#g}12pr=cP32b?U~+=C)~Q{C-C{#z8eHQJ)yz8{jn?iQk)9*_SdMQ z?X{<`?yd;<`s7%5^ZQReI_BdDu5Hl-PL1esl`BpLayLLHta_;JzF|iRD>dC@j zOZu9sD|=bm^*X?YVI&gGluPHkc zXCw``w~VoU7y@UkD*%bRwvP8}rp}~IYggKa6`r%_A$RVLP$5eeY}tx|RJE*TGzJ9K z*EG^Q&nN-Une4~W5U23j*tO?0!vbCM>MEx8ne2#QeE_PERatO?~;J2$cEwlOGHFHbo(>J0My6et5&@ zaUASpkObNEP+`k9Vv~*C@L*yHnDyVKZqug;eAlKImbah-zFO7-UaV-^p2gx(+V~Ls zH6%arIIQ65uhU9E^@bB#YNjNga?vu*1o*W#f|8c_Nohm;t`A!IF+?M6xL7DVr|A5$ z1zz`MipAgb!K&A6ZON;4XwHfQTea;&H~nj#GrT^^WXMRBk1G=Y{%(JgDflwSm4BL~ z9M1f%T-3EbpH|e|2@bc$Pw8n*8hq2mhys(h@58t3I$9Dpwe?Tt^enKK+kC}MtNhbJ zsrti3uRAv`Y`;4rgS~Na>MswsXPq2qEj(w$)Lb7IG~b+*wBB1*0$#40U6IB-XZofcTe@O~XDyl$h4U6t?W&WB*dJ%J);)Ac)DqL426V}epTpQ1;ZRr2go_4abTJEms!Raf8rkrJ6?b}ZySir#ui@58-*2dy)^^O-(xMJ#fJ1Yk!tR2?u>0=x= z25VV1b*9F24`|cU%iDILYnL>wN%M;4#6?4A(v}-lw&SLPr-os98f@W($PoOUw;PhA zr!+qs<=5Yh%Br5M8|z~{0?1u+!3qxhIDMUB@(uzNRMbEy-E@;WHeK+FgCGI+Ekfgd z3{i0FZbZ?RA6vf}d+w4MUA1IFS1uc|rBNj!C8`D{?7DC*pGG+BZ5JK6Vx}}M8ASL6 zHLr7Kn27hY5$GX99jz)33xO3k*N)Vvw9$t^z(zCFt2Yy?nJlMvaQ7Ym?OoM$0;=l(XGIB4-ByD(U@O>Ad)x)T~riSPJW+#`Oo{=`**|x$8 zJ_e|mNewh(e7HSf!-=h4G$FFKyoi!@4=5+1>3FkjMV9ROXyA1#y5jJR#K#F5Wjlxi z{0xor)p35=`T3#F7YBYy*Nz|C7&QQs<}~1p z&lBwK4`DWYcZ@~e9-%_loS5>RF&cU=z+vqL`2#y+EYhltShH#-)-3NL$!fm{6MOXfJNJVu=BxtpF6JZ6F@^@OXpuz=!%dGyyECsWYg=BgTHrYm6SgGA zB&DlXZ27Dj^7dey2>lj`*}WNAJ}m<#tr&r6>y{WJSm1ekBe>Ro=2?;tVG=rKb2Xc8 zT*E;C*R|?IwI7VrX`jOk+Q%RjwI3uQmh70S{UD9@A;e&On`H6+H9;kQ9>X^8xUdzg z22{>k%%7$Ppb36XDBRceVouija6#StC?aot8tiMlJ>J`P zHzH?NhotPvseXLgvYuQ$cX)1lu;KQSqT~6R8B)0F0B26B8y^OxEl>BynY7KAPprBq zfK3k#yyK@~4?;bIpC$yV!|xV0M$pYmmadF-7olNZgDzOnAu{aD%8PyA^dCfRna5p} zmTR+vk+!YInc>1n;^ zVl-Tj#n`MhJtS+>La1JkvD5S*He-Ac_0H4X{x~QECGI+@u+hVFXp+{4(*rHfRjBMs z24cl+Paif(&V=3TDoXufsI&G)Sc*>9;H%Cu%ikX*|U4anqQjF{vTGHCKoN@V7FFRS9tiUCvQmv+8`WZwCH z0FZL)pD?56bC5w_F&xeh=y*9LD)`Gu&iE^;gq2UKaiz`ak57$=n@c{9;oA;kv2#k) z@W{_Bx-!)H@|2|c(f1}q^2u2>rjS^ldZGT^tzY$+{Cis_c;T|LJz>`je79w+e{Ucp zUJ*3hJ>eu~{k4M<UE!cbTGt^--QtbwG>#sZX!RC;@HxU*ox%D@IBkb-#?wlcPV zxr-W7{f33yx^6-hO-TUnwD7bG*z{{hl+d*Mlf!_*>A{v4Tb{%7dvwxXkPMG$FzC)G zoxJ8Gf*q{#>x(*Qc1Q}T^K>`9TU52a_R!PLEezM*-?qoz-;b_d&;Z}ZcqnB-*ZlHB z5DQqfz$z#DfiK5JZI2f;?Df2J7h|IRRbK6Q0c&8&|_p12S0hrjH$4 z7vb=Jh%fsXW^z6TNbq$}SMElPITjs|LK~~<>Z+bpF{{GlPxN=a_A}}(NBcXUFUU}t zk-^T_eopbZc@?p0%`TwF_^fKf+L;vYE59-!F1a|bY+ zsWFbPTehI`BeJg76MgNE=hd)xhqYCEemZJDz-2F(3Ds*3CU)DyA+K3T%?lQ4%cz=J z?ieJzbM->wr@z$Wv92@8k11c zn^T`1rB%h>+;PG&|C*stmXx&*C-}u@owUqfyX)^B_4BHpc>4j#?YR$6kqVz2`xYR= zzl`94AHDeIWkbjNVQAtnjFNlDxkdMmObIF<>S-0vTHjthVI&m1+#4Z-w(X3rs2r5) zVdtD11ibm4Ty*z{9+P|~GF1BvSN`Y^8>8s;zh*cCUuQXeU#3X7T|cCJ)eg=c71!O; zVscJJbnrZQ#oM!ak1l>cBL`IMSlZq$$*XTID{Jr150~HSYkhEnQ+EA?i;;3;YZP6N zD1LGhmi72Ie|OWnuK_0QTZqm55@gc%MhM_7FSLBa-jeXY3qjL7jnt32!fF4Oxz2f?-T>atrK9%NfzSN1(Mrue~v29!0g1ST%)A(`7Q zWch&)R`{LI+m=|=I{{u{kommx#@^$L^C#~?*Y$3g39X_a>)gGSJuht z-j-*}7D~HM)O3HUzxm05i`e{egxLN$NP>O|Q;54G6y%(lR5hW&7tY%`#06Jx&xDQ# zSaSAqcf4HcwiQ#lV}TcZ@MG$?T#&pS2cl@l#X?NSICxwHxa%gCo*rm<`=hEe^`rxv z_S>Ye@YbG}NnEls5Hm{HyB!a{Y0HbQ`Why|zD|I|H=oU_cMN*w(|j^D(|@-Sv@y>oJWwV}xdG2N~2=Hw7^kW2A@*oHV0{W-ZuJ zW!qx}(2|?nGByl-H7e+wk*KCrW!?_uYT3r6j04gPhNq_ZsWc?zC<(`|6cD{){ z??o)GJ&*ZYZA62wm{wx5qZWMml8XwMG1Hr(7E0aj7z6(~#O~f3#kH?EVRV2MDX7JxgRDZ|Agf?Xh0iTi3#qHcTjWgH8f8mPZD(OY*lBJX&i z!xf%abmjc+rPp3rG7}q20@xcKAm@yoQ}uSuji?Cqx7-ei+a692cD|aGp|d8WsLWLd zp>fMkY@5?SvsNvLx-Zjo_F>KWjt5dOtE#_09rORFuKmrt30E3WGD=Mx&@(lz`p%ja zQ9C23{(DALaoJDH{>@9zJ2TQ<`y?!Cc(!OjR?Jv<&|YZK--5Q3%ds3)`5|U^bL#f^ zbvv?j#f~bE=)1CZV;r*+W-~{WpgdsV(~~txch7G*v8ex9V2J)RgGK#k99k30xhY{$ z$<=XD)!mUHWKrqMd&e8!Jvh5LitGNiz#94(#-mrvEh%AP&Urf`>EwQZ3|}8(lcx>j zn$doE!m^#xd>ETo%=og{-p3#6DHE~cvV&23cUh0E9M=-71~C=qZLEezdtNg5u-Dz9 z0hGLIM&vCRv1JQ(9&W`W>{<3nq*K;@wqGu34{KF|h;e{9ug8~+u}fmPwBf4e z^qL1(yXr-iFFIhk)5^B^RSP(M&j+hm^}tJJOt7?R6Ea7~sd=nG=bYVllA1Rxq=pR} zv0;0J1d4Gnaeb6Snb4CPoZT%62jeutrx_OS(+rdQ>2UU*htRZQ$CfS{u|*3yQpKW` z1{fcJzKqBT1Yh6SfYG$JFGg_Y3c4`C{Kdz1kE5GFvrOyYnC zK~&?03tqH0hHZ-RGiFu+e!Z-RWo$TL`7`p?=V3YM)shugxf|2RJ6;;}um|_*Vb4_` zhc;tmwk=rbfJG~{HEJR?tT;&Rdm}{Pmk9=aC%_^{txUlB7+<&*U^5QSPpt1sEhL-~J(rYfx#`4m#8Ckh%L6&TKaP`Y}Ohwd$E($Ag z*-<+Uuz0F!ShG=EH~pl}RWAWB@1(Rw?R3Dpm&@56=XP%fX{cCkAs+Ve-0>3I z_WWe<{wNK(?W2MhESSQHL1@CfmQp`&Vz&GEfLs3VhPxqg{i7IbKZ+=u;+IU|oDDCr zB{0;P78qzx-5aAI*PXbA@L==PkTn11hq3zCA49EKyFnamBOn-i0QUjL$(MOkt(5!(Tt;6sQf`6m{?x17ksUL}P~PGaM{h26P6 zBIqAebmn{d>l0Vp`1Tba39|0PwJ+Pzb!#pnV8zM7x_F(>Mg-8bO-~OuYNFOhG^C<= zBe^7ZFVpFZd6KFw+`W{^L)J}y$P1$g{<7o}}G$d~R-aC*0bSkQ_W*Ay7W zz7NW=X;X(YSN-VbeTTU!5B$B5=aX_&>XMb#v1k?2R~`M5tq~Ssa|D#L<|5P`&P#Ji+7lh*@@roMT<)e{ zIJoMQsFuBb%CJd<3Fvx&ql2`ZDGjH|FQjBfq~!7~7Y8$~Czb1Yop&6)(3cY`V)eY0 zi}3d&5<IOcsFo6kEIxa9UNXc0+Qfl@BWXOh}fnRhpy4FUhkbk0F z#laMtzcE4uZ;$qfwnzI+Uk>YuN8#10*7me{U28%_)%J8crmYrjY}AOHSgIyepPrS+ zcp}CLGsB$;3p#L8XbAAsD{QzG(-_aUJ%pC+SbUBeNOd!6Lj95%(SBHa{$+{?`!K=e zEP9xTsErfza{+zD+b5fMN&07OT==|`i(7Uw2?s$6A?63Qv)a!0^Cm?0nu7$`@={@| zc63=(+mR5{eyN+zmbY{2;_LJJw%41E7)Qpm)@CeTua0o(>tkZoiibs9jpeYIE-Rjt zV>9RU_==qXr~4ozGAxgDi~DgEH@WTUe?(0{UMr zGG`~$gtfS$u$I$d5#TdffV|T&ZCAS;VzW2KY1pWZ*t9W1My!m`kx@6Xae0K&wGrU- zjVn45CseIZ*8DN{wi6mRd^FVFBwP4(p~rgIFJR3_0xi2}(Cv`KHDjc9PHVB%vHPK} zkFuG&Ay)VPM9=Wnc#m}6OYNGqV@ejigqGDY26oYC8i}0hcIa%mj~svLcfHb6;N^OnSiq2dYW@?F6t=c-GHEf zdz3}kiRJyn?}X>g^yVoQuOTS!uJ!a2GJV442T>CwXVHqUSu|5yR-A0iq>9rtVdy6W zbwW^BM=V^lcB3L@31ikFq(>~wj!`KjWkyYIoi$2WA){XI9FieC!tVS9buVheB%v-F z2DwpdFLl}gD_F4Bzuq6m0`^8Iu!C59-wF=6*F3b&s1{!|uOn4$y2)+3enQKZ7uUSx zpa8;JV$rOH-o9w#lP2Z3>>$7C+NO=vvgW44*L@7sx|a%Ca#GqRwYY-u7@Kc-c)Y{@ z_y-{dc{@miEV|K^D_$ye-OKNZn!2$L5hK&w$IlM<%((=N3EQAJ>K@Xq_y!D9=0UMV z+E=ga>B~}bC8_R#-a0!A^Jrp_TDk6(3~vW})jJ{n(9R^McWHzS-Uv}q+miy#&XmNr z8}0ROO;SivOIPlk8D6?H!oW>f=`G$Na;9gHm>O20v)5xjygWjOFL-F+DH9vMHO`al zO^cmdQ$q7ffG=Ab;cy~WOvSv53SABe`?g|!z4d==K*z26&^245@QU3K3AsBV&}@b! z)+s%vaL!C<+YSje+hK0+PKZv5T1gG#T2$t&2j8+iO~QScr-@br6ztk4hrTo2E!!RE z_UuexJ9fil)QV3e2q*_2-XR^z)@Ok#yJfj50_$-|G6bGg$;hKpW{$0!^wihONuM5; zv`(wFJikWEa}IQunK;Oo9&T6WhjEc%CnS)q_}Tb?j$G_gFbkbZ8QeIa0m`_N6mb{w znw?54S=RS(q59ad_lv~KkvhUQywr883KG}w6 zK(*=_G>q#6V3%wNZ0;MXa}G<}f_hcAQ8L`B?U5Ea2F&nLjaTDW+lDPt8%@E|)ye3J zTx=WeCaxNCr?-akOoJTCu<$hX-roL(sD=ai8fMV{J*SLEX zO}@UK;#mu|cG`f*Sg_)1*S#FZO3bf=IzgAUhw{cs#oiANP|7S^VEUAj)IMkGM$B5- zh$T0dz8~f*zD0zVn2*6W+|cq5Q>@;@JzDgMdPbGhN+S>PP=-Za@kqtZGqzs(x}Psx zcF@`bGD?wOg3X#$k}H>Dtn1)`ABLp3obV8$Afm>WZ~D26B{v5=SNJGD!)e9W>IJmD!#1L-rrm0mG!sV1_qjKG9A@AV5FJ)Y}8)d z;Cn$A>w$l$uWQaGA}x7Yhxc-2KcfxE?bdE!AtL>IwWUN?c7WCS#`go>tHWcf}W-s0~fgD;vg4CXt0Qz z(6}?j5g+!x+Mf~_Hp4XP$|w`N5ES)>Jp!WF*oQRB#C5je{syP4uVvEON1d=q`5xH- z$UCG!x}++OOD0AIbR2ltjH?N%5m_s4E`D>Y->?xH8rz-ewQo=G<$IG{@uyjK--jrf z{-0Hb^xsPy<>y)M@a`0Yu^wVl=6t=vN$WtjT}a8AH22`b)*jM`hFmt|=FxX1X^aIo zvU=KxsaSRkL|egr+xl3KVA{imO}NDjpV=%oNynIS!8kQysQY}c=W2zpaI(8!g`p*iIFmug2sNNdu9gbM}B%hp7;*brrYX^*7GN}hKG~h?@=%!2-PWQZ<;>;HF zCOUX@M;-yKepX+KKig1>|Lc7^^u)dUrAMAzNO3< ztK15p>Oy8}gH||LFhDh=h>7ZiKHAVLomAJP>+LJ>%f+CGMJ^dPYs8kJp;iS`_(qQ9 zKjd}7?zE>T|D=F{Q)gsq=CWPVvoY2yKbYdjGzN({uWNp_X6q{28}E@teELbZT1(eU z#HC6uH(AB%&-W>;f^n0aJ7pc@1ysE)!>sNkQgicfGA!=7S0pYLfe3#aX}tUyO&@v5 zET&B4?#o^tlG7h~ME z;K4P{7!Y|O6A$b)%815cGfKp<*RV+XLJmn=#35J`QE1maK$revbshgyQ9kwif+F(w zH*O}!-Maercx5U2H)1E{WyR>@I8(*rr&@=*^Q=SEoC!-eVLL2w>`w^f^WGk=!(i-{D}D4{ z*&LlCm~TMf{m)uE6z3p7`FT+5z#qjq$P;PrfIk)GF;5j&$$z~45*&BoTH(=`FB*QR z$b+3^z?nDnB4J)gCq_)#6biS=VNhvDB?8$fgd&KRka+v0c;-eX3=_Bu?=)P-fDG5M zUB;`3j^T4HP14KNm7>3DDh7Tlu8{n2>q$-A)oX=uX{m(ofgt-OF@L-zWH!oHykg0u zk<&4*?*{t}a<)?AW>foT>R|NQ1PEc~QDdj=TtQjSPi2(@zgE|(el95+IR4}%_{iVa z3*(-=MjUzk1{ZhfUPIhZryj<=h;R9!s)GFo3OIO0FO+1>I5@~TFAHN+bl1p5no1GJ zQ$(ja6R`yMBMN6Rk3yf%p88ARh7Dn|2~hBw6{!17CF#pEBaUrIRo_;O-gPD1t1 zm3fHkm}cS=i^vS0GFsFNE+J>xMni1*#hPWmDrC1QY;O2iTXkv;{Fw!q;CP! z!*S+tDW*g=!JdG@d7ji)P*2nrB7UkbL7mM_@A~1v?UY{%lgqC66J+@VSjr6rrsli? zmi*Jy5G2cPunx(GotS|^KZGp_RU^qGPa9f?FEs&F*I~qo_{LW4=_gMcjy`-?dAzui z_j6(Ozz_ExHN^dOJ~QszrTn;;Da0e^?=;6fdxboPfZHDq4vpgc7Ioj;NN@kNlL_$| zq%5_{-y<5F#gMo&xd52sN>VER#KT98$5Y-Rek`w6oXjbae*gMC@yN~ljd3rMFh>&7 z=||4rY>K=71Ql0OWBUovVZKO0D;_8W@^Zb1pFO6e6a1Qo*PQ- zMA@&G)$-5Qw`=4npWw9n^a! zy`<;67jHpv@6vHcv-3IMy-p*>U4GIS_j~MGXK$3nJ$Md@yBm)>a`y%F``UW$=>eg+ z)FqdJXRTt!oLwpoTKm~1m9S<&He7)h2Xkvs6Bo;YBfl26I(|+mksQ09M2P$2N=@9g zJB@LPsgxrRUgP5azTXjd?MYW$LMG=(d^+#wz1QU9FH&fyz%gHyawXX_uHK%AlSB1c z`e_`p@G*s=dqHOU-h$9$H}k57e~3?^97`z>{ghQGJa+qjUEITGfTQm-Dc>jObC3Lb zDKGBFpI^lNeWNPwb^<)^&r8)uUc5p5ia;9P8-_-EBUZg@)+P~-Tlr+2dKkpud9o>z zc% zM^bXQN7Hk;-#t%+efQV(k|U2_L65wTJrIx#aW@{d#65b8KK3Y;dhAt!^jLnA^)v`S{*c8Fmk#y$fDVn0Ij-f@&sr%BemN#p z&Lux$lNC?t-N8J9V7>rES$bSltNP_#7X5h4=Z+?(FpfM;z{K6W*BW>IZeuLgA>*E8 zaO1AVqmTY|tNpv1&yhz{ixofSw)p-oZ;{`Ya23UKE+%>20?Ue;iRE5}u(6-3%OMDm-sCvx_qRqz%^eV2f1wWKy!h!Zz{7?5^wtfFL9dj%-pK&a!jQ)FRtK?!um*PS@!hDTH@V@C~x{HhwJ!r|r zBS!6=IRSNNvPaJc4+y0-y%IA(Aoa9Td7iXZi2Bm|qMo0hr_oM4c!fN2KOPvfhsGlp zZM)<%+MGGnNu&%>?x~9$eYfd@;hcj2p;?7^IjJy397wSDJqCxCv4PX-@JFJ>Q4tq`8hs_(hkfzv*taGe=L z!yfpku@^Y!qP1>}Gb!6s-9w{R5mv2raYf27ktPXL6X}Lmc!KOcfu?-ig^}N?Y#qLp zSKD(fyMlT>zmj$pOtHUer#jw3sFnm4PnD$CNb6nt0nmtE(lzH2GNuiL;#o7MIIKhG zjA|)m3L!UB+9$8*8#2|=dFli-RdE+gu|6wo8oZR4%m4G{E5eT#?lv8H`VM(KrCjuT zO7-9$<%p5X)r7G-r7-oKPO7z}Up+?l*;IrnJvM(@-TZRdj;vcB<*>q55lOE%@x>}X zks}>#!t=)7fvIz^nvi2xYa!0R3hHD(+<4S>+;ZW-QCm04W6+5$dXqq<)*}a{#(JUHRmm1Oaw%+UA`+{;3r48#HG$+;GRhdg zyiG@*$t_`DX>1>Oz@X@I91Nd&c?A*3fU7()xFtqX-2&%Y@KN(FPt}^vKNf#T%Qz~V3%>qjRrMEqg6qNrSfuVpRQ8eYpq}~^$AG) z;9W3Wa1)B?zk|d`UvRjR6hWWlt*Cz}T`AEvdK7wW)G|oe^zv{AK?-Ww4K1FvlR+W- zV4v5l8?u=-T&;e%tGBl&nL+D&2*+FRH^UuwGRu2Uzs;l_%Pj5r=|u|cctQr^`;-#) z(f8%uCsM0>el2KI{Lzl_KI!idqTEJX&zOqczUd=@mYm4yB?qQqD@Z3yxP@5dAU9hq z7H9P+CQwvyv>HkCCv=fK4*^)$-B)?k-~PIqf8^q|lDOxIT}R$zz)qIeu>S^wWVcuZ z{cD?KsAf`)&zn>LUoKg3RZ#<_e%`@@2MnA#rG)fiXjtDY9&k1EN^E7~VQZ5>s;lW1 z$WysuZ8k~d%WuahU$%kS*Wm=|Bf7+1)2~#QX*7ei4&wkYY!@N+LNxT(IWqZQi*)|y zSq67pPb_vAhKNS9zT0TlkhH`76)JIWw!BZ6IXKXtBONw2N)?9ofnjZfL}D!OS9-96 zCM#Z}A862tV9%$F6}1TT80CglRy`Kr*J=o`UJNzN7UT-?PM;WRYA~ZSy!S zbHTp;jZs3!iW8Wz?nSokO>`^vrp2Dcu|AF4D(h8ihhal2oYAN|vVI%V?n8;4-RZHP6n-~*Np|V55@^~xZ&9y}vx;&u7mWOq0(6~_q z8`X9me+8c-S=`2e%hwGd_k{%v~?6d{SC@Gc2yTv24N=Oslb3 zGX_Ru*wO^$^@O-S_XS=Aar$8`m_Boo$Irrp~uB`mT2o3#EC%KWLu!4(cb&Jjke)R_0MritV!QN>$%r znY3G(#iOZTpwWt31hO?AjZ@ym;P}_bbje#*k19{lJD4#nl@v@^6+N4NxoyqGWh|O- zwI4^jd%lE(n!}j@n@(U}Kw5pzC9b{U=gwc4HepjZ$H=Lyd3T%=C*sLDpHn zyzxQ6$f)OI8z18vGfxLhRQ&w7sDHx70(dpV9E%j2Y!qW(*<_To5i_?X8s?B!!$jcj z3`_7S%8`DVrBXfy5!Ii=sMe2ROy{;ARU0v&i>Cb|;iz58op$uncgA`A&5<77nvY9d z_cLH?ep2g#6IV6wz*k1>Jjje&Dvfx%Nvi=mX4OZ*%~|keD{eyLvJ+mmpaUe%Dq0`! zkKm!-W(DT0!{6F{_@+5KsXnZRypNh%lNMdL`UwXg8O!MarI7T><5KiFbhIM30{zyb zXO)k+gqUeJ132rTw{3>Gyic<{<(3~^x8_C^986IupXYnWKFo^!yOZqRsEZ16$r+U{ zIk{v!mJi1)0~Du01T?C88+1lJb69I7D-|Y;Uaf%G_1x+a8#ZrslFi!-G1yUc^Q!i0(ptx*#y zZ`DC<|1wTNdV4^Q88Ve1Zi zNUxf}?-z8PDRZ&4kH|Zp&&VLJr`5#bStTrc&jl|#IIN-Z;+sBqb$aLfX-ZV z!W!ls=&EUl02j7Ps9r6n!7hZpm^0E^)@}68H7mMd(b`$G<3ZHy`Hwp{4K!;` z4oKefz-tcYckGW~8dq$-)vH8;G>(yNjF#s_;2MCr#-a@T?x(_m+p{XJ&@ zm2J2Q&9kbOM^i(Uw?cg-S0mE08~+nVxBtHxD*HcEH2%RD3h;4~Li#!<@_d?L372$D zZ>EHWS3<&)t5Id$laCW<;Fo#!@b)+f@@blY_}^8w^4m0>zhUjHoL5yOMC1imqw1Qc zUxKKHeC^$mjx7 z`xxj|3$Ag=0Q$dkLc4Bs^}ZKcuxhA%KCh^HxDwl=c`f+uf*zjvX$;@~!G~$wb>Zta ztia59MeFma{U=T=&#w{vI~*E z+FP@+=PYOuSq@tE&x@Lt)NLmTI6H*MUf092=7-y!uj@LK4(yQ3JsTt=DreaF)BcGm`Y^TCZMjs356)d0!bup#oceT0?; zBP8!&1k?H@j00}j5v3bOaO%FJGyT6|6yU$JWLj*E%Ql@X%zl8!+wqY>yDmh{s<}OR z-3m@$wREQMxS{!*c5vpZ8Tf8q-SYHHhyXhnLDU@hfJM8`mYffVv!Jb=sXGpM!9NpB z&ev%=H>&G+wWROL+VjxxF~0?G8M@MU48ZgqGcaR8(H6g?1to7-Q58EbZ1uhilKYJmboE24e=IwjXZTlnO(l6o8 z^3UVInss|e;kE}_eAsj4+Z2tmWrgNX4OP7gOUsgX-H_%lAqwT|II?}y)bxJC)DSBZHoRfcHL(`SFVu~O5Yf@yWK4Eg8HnvW!Pd@r^ z%^&@Ez_P9>K01_pZc$!*ZNu33a>owK-S;5NcO9V2H9g?vj*S>&I8w)|9-JJNwLOaQ z;?hq~BK*pyVwr_+5yd|Ca@sd`i>x_M`!x{Bw|9c)`!g z{e4c?n(P)gyd46jpZQ=XHtm?vrR&;`b4nEv8e7W zScu*8x~V;R$JUj(s%m?&t^_<^R<|c<$+_nTfC;Cf!!6nW4zSsuJ(z~+zS6sa?v!&= zeM#r`T@CS{{gC{R9(dKB9bSIe!(q!!>)N!_ApZ`K5dVa69ebnL=Kt%_>x5=~^ux;M z~w5qOnvHt%#dds&q({}CqeV=FDGWYnjopxp_Q#UHKg;Lzz zHGzIMT%>H5Qqx|cR~{4j#Or5t+l=X#(QQyu=PVBa-GMq@B8nP^Oz*W zS>}WHJl2udh6yn{!{}!RQ@8}Dii2>->CnAl=<|)Nc#{{^vTE=~a;+8-1lW%`@2SBYk7DUBWCqoWRE!hY(N9W3WdS z5jxth=2d&-%*ve)3iuxDWVKe{ zP;xMPqo~N0cZD~tGl*!Xrl!`nTvv0v${_?6sFi`WrgqoLTgmbIYtesi3!9F$_yWTO zENo|}{GoXo8v}fd?Ox>5_C!E4rt!%x8MVSAqeI+sdU0?D@b)ky#xf0mqI;WnLHoM& z(pDe(iDiP5ZyY5hZgp3M*eAI~2V>L((-=6e!|?kDquA%m?-6$tO=ah|dhySFQX$?j zfQ@uY$QdpfCHHWGka;4<=bk9xQGNv`X>WoMz21ws@0WAno_TuF!7%94e&|hAW8#ma zVJS%vgJedDew^@y~&( zq;F*W{NI*d-z#bozt`4h z{c_L`d+eJhq^yA)F~LaqkES*6T40e`wbD~|XPBD!^)x5_yLn;ScYWCB|Jm(@KCp|) zxhCML2cx)H!w50QJjuXVr%8De?C|eI?1w+9>%)H7=*;@nsUT%oC9IN-9#ZCBPj$kd zD>dxE5|?0;qZ1B9z;6e0?r-%LUS94fyJVWEt={e<#BaPWy1dt4^t)*YdPm)m`uz+m z=0}i~?{4>%K3274{IoZS3U|$Opu2;>4@RnLr)ha!Ef4EZFw3?F5uy55sh3P6kPz!=`F+7`d093;&*L%760b8jpUuz^F8hR6bL(6V6Q&!@pGu629H(g@D&& zAs602ZmxZRgq^Ig$e*`r8l4hKu1`+Mw*oI+e3SOeR)7BWz#5I<&@%9QQWF3 zPL`;UfSQ!Q(^nj_+FAI!SHmL*_e}SygPDb%=bR^}Z*=8dcFM`wHYqjRDkddaM2M%x z(ee;=YvP$1dhCA;1JnS9KzYB+<@cD-#kTS*>X&7|DH_WzEOo$dZub$xcL&H(d;O$n zRek>1^)C25;}|~DGJ%WS?k~Bq*q-v!LTlFAtCd1=|9fZ6#izDupt}0@P|A7$VY}@(zgdtPZV|8XT+4KulKs4*H(K7p#z-ai>Sog z|F4);deb+{EwhP`&-LvY=j8RV-_LU6zE?GsTs@rPl^tmtc$O(zhIN{n;h5q>Y@>|S zfP`Ia8m@jm%a8qbj2`w+O>@>cukQLK#XWUH%K7!q-1`S3 z(5D;SrPo&8LZ1YbuRF{Wuj?1viD8;%bokmkL_|PN$nwmiVh!VvhfAF$H@oq1KfFW7 z{Pa;@FZ2iJSj{3|%r-tdKS~byVgEzw1;BlpIC%{Apb@@bhc^E5hYoRRP? zHRa}4w4|H=8o@;W+t3FCovt(74GWs|@sW0l)Qu(0i}l=I{CM_)+{u@O++Xwn@8Yz2jn})_NyiUI`!A?ByQcw!FMk-&dyv_d2I8yoO zU=ST<7_JVt&tp>dr?5#qgw!AE3!eTL=;chC1Xr~2q4?HDFXYw^SZAxKD#|un4fGqI zXP!YtDH_txEx*fo^w$oXeY`?P0DpcM+^10gs_1uBWA??PWe&*){7m((?8yv2<46Aj zpY>_IrokqL#T*RhTrmzopY0Doo;qgW32s?c{GVG~hO`5c$b~%pZt(*u^>pWr!nf4e zW|L6zcLxiu$ZBG~>!`T%jl3!SwolK(3(G>jS?q*F`jlLRU&$_a&A=YJ71Sc&@A=Mo zLefru{$+Jb)Yo0;TmPpI^YojwL0s%>YUa1oyt31WL6?#k6n0vt{CTAQBlH-hngcq(^Obn>aaPt`A3Gm%Ji){tx4@H1; zX6Tuw@#^rE&eSv7rpT`qyoCQO)@7Xme-pddUlsO&81t3xCHyI<*|O~qu=`7|^Dm7u z;=g@a`Rwa9$b)}~x#>Ty^dKKj)aGBA7UW;v8bF6HzAn8ctw}q(^g8E~s-@uKC?n~c z_lT%}svEJPlfr^qqny|?+r8zXhZC42>o_shHpR?0P2kfl)3E5>fs*SW%SCMtkYdK^ zIajv3QOO>_(%XII4|aM>?wW?nLo6ee4`nq;-;a~RzX|r+Rxjki#@nLHgV?yA`;pJS zUu;PF*&#xNpQz|1UOB6LqX+v!(*U_-A8TiCbP|)S6V-8+$%-fX_S`E=?fEzM`VgSU z8QCZLhC0hEC1-C08@Ag=jW-XV!|dZFHx2!TH#WPAt}VYUyJr@$QRW$8wfc4CwT<3j z&ZOt9bW}Z3^74Mrw?MDy+KO-MUgclWye_K7GIv@r=0~l{uuQ56T>hja&MsexdpJrQa9vY zCnowQ!x$mmJWb6Url+5&&wucLx+|ambDEa$tqSPFYB%ip?j$|`B|7#3x8Uag4QQA! z#~iJ2={+=bmY490RFLwMT#$W6AxJypmJtegWlw(Sq9oovlrqbWQ_LJiP3Acf9b|k? z)OV_;gm0yS@UP4hr7>SN*+jFLnBu5z?{9P#-QVggePsAho&+@ix&Bq|^|kle z7yjDkuv{uq*-BUGHOmYtKCsZ*9?-pP{<0yg^=k3ifC0h{BIL8Jal|v1mPb2YYa4Pa z84!~Q6KxYCW1Vy8G@AsS2$%-p2OiT+c=olnqwI+X@S}5%Ss1KakCIVppC=~&J?n51 z6SvV_a!>s-=T=Y;cSZE{K0)aPc@y&K;f$bSYXtv7_qO1|5H04b0aDbz3&&p*G`T6FQC7x}<4iI3C1&AT#5kNj%wHSDQvid&>=&Ofv8y5#o>ZqBbQ zmC-*8;G!?AyoEj48Y9LI^0R*#XXpI3H%dxTw?pr@BP0INS@rC{hDp)?nxw}5OI@FL zeyNL>USE~?{YzxT57uc0+zT>>WdaM>1NBV*I`8u8+niqvgC*B(Vswo59rA8nLD)Yu zFQHKmz?_S33U2Q9g8b86a9LfKeD)peF?j8WA6NSzcRR2V-;R?Le>U}z;y2pMuABOc zZrCRvq1#=lkCxiwt}7c7el`YYTJ@9i_(h@LjnLyRd!$sjV~$+loI%GrrchCbW9S!P ztui$oc~@ku1;4(=WLzGmq+W4?TmiEE3*b}X0U52xIYTSn=t4wHvy#6(oFZo)Ofn(5 z_R9OZ=90@xuOK&-4Y?Or-xl5Sfjs1p5OWt=3NM3Webp^xLX5+NxRv&z%lg)gGfS;0 zKPc)VzFuriJG%?mW`B~9*jWDJ-)*qxKkW@wCwNt~LZI`B3$F_9$m&zBuDk=B(v5s( zm+-3(CTUrttc)M~D5<|_UctgQyU=0#gQ(Eu_pnfDbJ>G-ROGjv_^97JY7Sz181-Dl zO+Pb2i2i<~z3{eq5dOqG1bVKHeq+15M!HX!+<89>w$&-a~dD{ujQt!-*k;xSM@E~zgtBmkN?f4+uChB0SD$1px zmi)0P;2tkgD-LHX!Vbld@SVZT+nc?G*W5}vJfN$g*d^2~;E{Rm;4Bs~B{g`C#;Xz% zJyLcl$kwTQLsg+`UFA2o`l~~XBcufDoDjJ)$w=Lsz(udVhu>dphg{nq!A95s*Bhts z(JP%LR}3HEp-1|bjv!m__FZB>pf&4*zDH8ukwvFZsv)9!!i~%qy7{WSi$X> z`h`!?Bs9$s;!R`NDBC0tW*rqE_dcLQw|dI1t#=n*UwIF>za9PTaGHl&=_jPU1nbg+ zeEQYayRvJmZ%c2fUqbF~e!xU+juMmlm{~w8QqI}Mq)csd$`AUsgzq;xvwvA?jyY@S z%f4fsK&1H9e6p&i`dJ$??)y1m$&*j|=C_V{eERy^{NGky<(}K>f?VC`uDH9{Rryfd z3cXs9eD5DsiPz7ZsA;7EB_-c63y<9C&AB~A3;(_wcH^J(tmj`FMd+xP)rl9%B7XnJ zd}HDL6Ey?o5S2Y}O&30JP8B~;*2VlZhJE~h7hmRHbjUcBYdxq3>)nXQ<|#rpVCQ_> zG%?3MN5}+q5~J^i-tEREe*LyO@x1O0^1&cG{nF|O*fYZjDrtAHGRZ3=6(0v@xXXD} z#z}OvrakA}_AokptG_C8rw0irvLjLInoAk#vm;1*ShjqF}MLQJu{H31Ei`tI(GlofI^jvdZ zxav(#h+R#obZHqBpAL`yw92F$Em2?x6F>+03a_}-gv|Y+>Tts_G14+kOZTX_h{Gvr zvO_{iKb!;_H-dQNnIoq7WQ=0V45x5+gcj$K39*Jza>DZK@+<4_E3Rz~KyIj86V7Vt z|CM)xZ3_uWzWqs@NEP2&VE(IVx; zlr1F}6!itaF117N?DioZn+6Cmj&Ww8ZGxDy)KPp#-(GaZIY-U2N$9zTQT&U&F-)Yc z6BeRutGKuJuIh0>Da7whvI|z;6G9J08WEeLg8c0vV#M|U;?Y7|-mlvq%I}(iKUt^A zNnRDJ{9p|J8Wq_6}sVO>Z z9jB&wK`!;E=*5S#)lrJZqTl9g$}Y<4ihnaqk?1}0FvUwfx6ua+vClIh$GWEa+&T^y!_ zeK*BT{Z-ana8=m~dnjqHxDAcJ{>>;i@n%4Q&DeOGe|5R#@7n%N*OYT%^?m6ro0L*C zP?L8p`@z}&hm(2#Ys)kP;ZbqW#!+bKMt9cL#aEeUmpe+YZ1$rbZ;aqxbdclDzCwh4 zYY`EWU1C;&XO@t3I0=v5>Mwp~o+uA@%c^7CDq@!A9U>Ae;V%{uIo+qgqjF-(wClrNz-Cm7j398NPU4`wK-yOX%6^}dP+W6ZcqgN(%Mhhj`-V3AG<=-Kt_ zqg4sR+{E*SX;l6n`!$W9wz+iUG(1+;lz498O~y^rB=m(>gH2!Rhle8)@BW7m3A=T& z+0+A=rsY(J&UY)2F{}NB*A{yV?re;L967_x19kb(JV8%$%yY_YaxQFp5FcilAtYPI z;bC40Hr^saMXkKgy0YGzcWu23dc!f#DltycvNpO=4-A927oh$CzmikkbEGsY(3+*U z#W%(5#BVm=!6J_}O+u@Lm}DNuy|7G>67_A6JBO3xO#3{&*d}2W2KUg>zb?DDKTJxt zi@Bv+AK)QtZ%TgGHK+YN&x`q{r#kH4vYO0a?K6y`{SjK)dKWsxzaSvF)J&*DM#;B| z(FtxTHVv?4f_a=6V-*Qu`fh6E?hq+4pyr|6GII9G0s|IMvk|5VOw`^WcExexUi`1%9)91nS{s}~$E*YZ!6 z=`g1pm9RHde0R5}^rm0MCtBxO=xIUL74sx1)jH3BTj!|7E(H^;B`!(%D)WJ~IqrtO zGxd^RODOZH35DzL3vcfB!Be~nHr^%URypK6gk3AZD|#v)zQe|RE9U22HcfL&c7`xv z+O~o#^K}^)*WQ)g2YIrGdsD)N3F4-^)i`nff3J1Ipxz<~8N7|V2>q2Kq7+{#n zZf5$`p_+n+J0dpJp{T{#C9LADUiecHJ@J=uTJ$-GjF9P8Gb??1cJ-l(lxtCw^8za@ zl1oFc4lJ+mHCTXORI9~YiitU^(_`1 zm$W73lv-zk?1GMSOY!Nz_cGSHszY{1s-7*s&HeRo3?8~Wgm|oKO8!ybo_=m`pz@Jd z#m6|*e2j61o^GGxmhTKyKQaUUJ{W~Q(6%N2B&$pLP9{tT`Kl`6XrV!93eFpoFmhdD zW}bNf_uL_(<#`nxgn62ry8I4yd+}BAMbiW=dux!GtZFX!d84!99_X*f>u)NbF21aM zxYvVv>J&2y4@U5@yB`qu{R>>o;VdTFGKG9%9D_f$fvoA3ld}AB7Q`XpLZmMtH*{@f zk3DmYV#f?V!LI?Chzambw584=Yi;st8U>%XYHECHVy&PO ztj_whL8|cUi4e`(qDyPY7Dm9tEF)oFeCU z5fXo}O6cVux0*V<3tYTg!6@DtK|Y@nytpvUfA*_!v>@bvcX`~?B{tqB#>NH~1ZagWq3mDk1W zoL>};dACoNShz1+!n)Hn4#}**ryWQMX@44pEvFkC=HV1JX$|RvTjSV>y^)Ih%WqSEGW2Ktw%=cJ)i}n? zw1{h~cSqR;2O?&cZ5Gt_7%tL2gN^mcaVaNiLT<39Ow;&yyRweF(ha}4-d%igv%m22 zdQags(0IZX= zH^Rt}w?VEcnhJh)i1}qsF~8h7&4(BViLu@2i0@k=55Ave#{YEC2fOQ5@~hW+;P=)$ zN-w#C@3X`u9_h#hZW%6hYY=w77ihnTS8#Q`8xuM!$USS55Yqe_4$`OORQTky0?QOS zdT+Sm-cm>ISyf&9f9!L_ETBn+vbL(*D?RuK%Pck1DWzollFo(0qOqaxOa@M%Z7I*xe)8D<^{eU#>ue)nNFnN2!>M#Du&G-Eg*Wy`%I-PmaG57M9_t8X`qcsKbMq`E z^YeyKaHMBpd`fJZWg7Z?@m`r8qPV{K`l5mL?52+)_M_Y9MS6tkRP zWm7aVcP2Q+LLf%1FX!?1e}4HN0m*7A1739+VGa=KMRFZZwV zS^MME#N8osta*}}3VOZ_qHGFp30z0ln**F1zQEAj4gA zgcO%dfHI318Fm??#IwZ3T9xE%s{|8c94))O`yu2JoK5Ej= z7%kf*A*COPFcI4ykoUJfl;2i1<(ya4r~PCFxz4WFa&D*A6C;FZM<>!8ncxwTsb_9yuw~C1;fz#ms^|5i`Ry*MPUk zxfRAKWY}hZ=`~4h%1`r+$yc@q5z&r0M%Br3EzcssCRsr(S*FOz%Ep2V%KC)w_Xo@F z9E?{zUFj~n<57`|0(uU~DQ6>G@|tR|mQUCkfZo?MWt`D8LxPdT&Rk$%}P#iaX` zwS?6k{BwD8-Wl-u9=hg98HZ!YFzXa5`d}RP_+Sk6#5zk!v&``;6mKyRud3qyV-=G# zKY{)U&T|CcA;CJ0%iiiQy1Ct-ebF>o8FsoVQ1a=eX2SPSIs) z10>ch;}dp=tDi5wE%S!uZsD&#-jsY>N;9!Y~7opq#dm{cNnJ`Wh?Jt zcNGokXIqOd|8tTO_5EgVWte}p>Futpjw@{`zp~nkdhS*NM$&Sr9u1f3)^N#g6&Gy+ z8f2Bx$^uLEb^B9{oI}8~CoA;oPa7Qi=gs=pM@xLp-@hke7jKO5^EW42@#}-aqP;O@ z#?c}V2aY|#zf@QMal2*s(_X8{x5nq271YALDO$R1zDZ!0HM9b~gB>q3&_`?Ja)SsF zzc+?Rbn6I^lkJ*@eUJCv(( z9Wr`ha9>L+V3xfh__NiwWl#48t5Qsp9Ef4|ZOiWb+ZKbk0lPa!OIqtf-d$+PK0CvR z|60mRI_H<-bB-1og?plA?DC)>Yh{p=x;4a1G>p;`48w#dc~kn&y~wBkP;#<=u?=A& zJu+(E?l>W8dz7AKlh&Ysmqx2wGta+4-Tl`)^sRrX8ee>Ow8AA@)B@uAAojUoq&nO# zq2vM&dww{L2(!(SXGp^HT{>Z577G{AA)RH{`+cYWvEBSf(A30M@i1v8!nAlev^86qpRS$1?cM9 zAS-i(mhj!YAnh0D6fM^=fr~yItaxf0hsT@7P_c8hxj$=PL$8d)~FKZnNPL)?l%=ajWx!Z$~2Ub~Cf?)~~Zl0-*J=U{O>;161 zdy|;>z>0wG)=xJ)!$EjAHL<^V_WnTGnDWE_`E$PH+PY`3nlX>Ed@ zvpzL6PA0C_W^I<|Hsrib-G#8=mR=+vLWDk^mLSW znMFKW;d73bUcOeoLWGJ0Sr=_`Qt4mD+SebqYw81QY?e)l&9SN|#V);oakN(d@^qtZ z(7VK=dRLfLc92(&R$AZxeb1Ysj^L1yGu$#voKu31woIa9L7t5@PgFdzi;3w+nmXFy zG$D3#2!2ogD*b}8G3I+icjetR{UwZMfa3 zR^ow(Ut$;~$Lx;~W0pG0g7;|MGjzir9QGj}nTD$#?+oI@52pAPHd#%zWtxz%)KU4c zpBi^zi1Pg0eqYY*z#_fE1oDt^hLPov3Q%?_t72=Y{FY&&;(>7z`e36w=cZW%eHPF$ z%7c3*Zcb1WfRC3s6|djerL_dWJ~93!8q_W$W$yICLpFM$&n;p~(f%YmSKWrXaX8J) z^Qbsg7BM#7JdFz1wPjvdZcDpp8YvHRDtO4v5mu(Ewe=kf91;+Sy5Z z%s1+W)SpyM>AxB$@kurrw{reX8InwjVE;+eqdjt`40NB!~V3yg`Z7oZEn7H|- ztlt4=rPw5x9KW7L0(FQyoQ0=4<<(h+@rs8|kQrQB7E0NXcU#q(bICXge{LFw$2#Wl zc}I%^`q8qG1F}b^@>SXIHVG4An#5-=zRv#LJXZeH1+>#GW0!l?T+GpO!%N?4W9zP% zSGFY*Kz64D(1mxfd%J`17d8pG%mrBbaF&qlTj1b7Z#J|A_tJH%83<6ne4Bz&u|I`K zvCFVYb~&Zkq2{8t#;^%)HM!(ujmL1S7!`gs8+E+E!Gg7dIf0&w!aRe|)V_sWT_37?;a*^4JH zW)U)UwWsiMKl#a5{lr^e%?rZ*vpY~0642F9En-IY-Z(k#KtxKkijdD76O|7S$I2e8 zzfb%{-5U8_V39{S49?jdU?px1P~tYb%R@Xhz+6s_g(~aY(r_#pMpO!mI?>Qwec}_{^tHWs_!Z3_}u|JFpw@BC!yP}q`Ct{?j zTFS1egjr_~hfq)4vy=>9ERVxG3*v&p{=4Y4$!CKmy2N?+-!xMi56 zm)PeS#rjV09UJq{&DBG0$eN2n48V&6GG?VmMTc&VV!}1=VfQzN@SxtBnmjr_{%Dy? z^lI=K))`#d$tsWiWv8z7<62$g+AuC&)>3xQF5^)HAS=4HT#R`N6FXa*^y^%G?Dytr z_>0qZ0XQzy;(l84kD{8)>mYMiIsxn2rRezWA?U;H{)!O5;ZHqkdPQKlu6An>8?oF8 z4{^>i;D4;vH=Tf<^T;t-#xYowS%Qc)DY&rRc>!#7jFmnmC=Sk=i#XBJk^i&J=X>OY z^dl{)B%sFUZ}%17^DpzLp0%botE0GNU3bw9pN@e749EGr+t~GSv;K`wO9lC#lH*ac zDn4%1HQ42B)Mg*@$x3I@<>j}z7cF9ZibFxn2mFQmyu>AY6!=1i91(xI&OjeZaTy0g zWp_5-6kRn8A)eb66sSYXB!FJvoNNhejY?eV!UtriYoVp%bgO;xi?RNlLl@i!kW%PS za)H+K@W7V^UL`gENXJFF6g;?D%+5DXF|(K6SKd&yKz~&bkV2UCo!#kU0)=Y&yTDH{`iSOUJXH&}8HFj>Xnj1XoT6XZL@;NQurrw1bi zmtC_(pkA^r>bsNA$lFtX+vu-&?vge#%riBxgK=ttRm?56&2u4k2|3I8_k2Q3)FdzJ z+kX7ruMa1xQv(YP0*AVuu|LVlv4G#Ti14w-{)+qFIehX;XWC_LOXAP=S#-Qj&P7OC zEAKA8Lr3}*TsY8Vw5y zgaosgS>#c-H13b1pY08m-YkYi(K|pvP<9Aa7{jRNXw3@==?E754z+-P`TUxxCYx zam6{0Ne?V=$tTOr?WSo``tAfa_Gq=X{&b_R*{5Ydd@6jVX`da~ zW^;m*vpU2tG>99i+Y^*hc}H>h;yYOA^84c3W)ULBE5~O2wOiBV0{U!_)}oD4Ub%m% zvDu|&!yHOVo?!|VzCVF@;?ht{KW^64{$Xh9-JhZ5t2&T(4Ab1oj~fjy|JzXe`tu5{ z$~6NG+wV>|r)@~RvOZ9ibf|8uSAQVH%bJU?f7%c>{I9vO|MMoV_RBJj;FVQo*e3HI z?Tr`SaxZaN!TH-8)vEMTOX_Hg$lA%)f24VMHY7Ww+w(?e8zDv=&T@)&M@VsNUASi^ zaNIpgHtcALM*tpGU;~_G8YjlsXE+5W5k7i<1QyIukE~PW_Z%`*v_}(MM^2?{mR(^O zK}PQk6yLW@qZ3>TZsk^g)dSNMAFDx@4Psq@*ZZ$E> zp&%6;Ei$oR0AsjRY?yVP1NAI4HM$oX>)nbPj8DTx7-wLQED}VxX$lz)*f{%Sg@dz7 zsX1OXz4Y@62OMKul1ow@ZJK~Sa!Uz-z3DK&vXw7j$`nz>Gepvoy{l$fWh6D+gU@je~1 z5_k~Pqo$VFX7NcrRq(mC`dYVwkYSvxc)Hw~^4n@R^p0D}BYxaydgECW@+Myv{-S&j z_)Z}tc=STLR|~vZ$wMA0c~xcw7iH1C?p&Q|V~)H;JVB&}e6>7CEeF|+3Eqc#vP45z z#icJi8gh{%*sF@>MvJ`l<<>Cag<+bS>C+2XK0T}Yc#Y3GTH`W3iwu-SM$HAsq|&Bs zeCg2Da-B+intir9+CGbn^lMNV$G|&1%Z)8|U324>h*NYhMJxDgzp?vtox|E6g@u@g zU=LS2^Bx;U@?-woqEY|%u&(>#HlJ&p!{w}xz!E(RT)|RLMf~1qDabCB>BlRiN^ndW zcDbO+DCU-e*Gt_PsD5meu!~Idf{H_FZM92VOSP+nc#E6|-=8AH?v7SJ*#LFCJBo`k zOwv+zhq)Q{*#>lAp^gE3m~WMnGl4G07$?aIW)ZK@Ji{+BiWuo85h>n0fqLTAk@K8t zYX0F2HhOOm9wHN_elM+w{r7HP>1_|FQR5^tBUoE*6}K|jGe9K~G>zJ#K-5o(Z z+~}GIk@0)|S$EB&kb4$URk(kVg|y8RQ>?T2 zM7N3w@h>pSgZ2Jtol6hsiDkwq_%o2Tvi)nV{f_14*M=#2){&k^awyU9fkk}QADc|{ z=QSqEtt91J#Hh$)9jPF&KnK|n@zgzE6=@g4BThB!s{dJQYWriU_WIWW?46|T;kdoxpG&dgU8|wpFYN<<(N!%JLy)SJ}JZG6HdwR4) zt32JPt#vO5h~5Q0?R5R`XKo?(`RXXA6rFG|4t=ygTKUAI<&%!r8{54LLY8fg0|{Oa zKGoOIdNP-9&XvX>@U*{Tn?Co?UG zaHqDpEwI|y>L7_1DujMGbXppyKw^^;w>zhl^61r zy0bX#&x88+|DTn`b;&8YKu^$4RU^lt5CXknL7nR03ge z^PmSJcJB5FCD|Y%ryb5Q3%7=eFSZ6NLW~pe@Zh;Tr&>sGDtQRIlwawP*OQ&f2JrU+ zP^;|X;PqFZmRn_+=Ro!*FcItB6<76dpjTJk!tWTzNWr{S3UXAD53qA!Nyzdp3W&iO zpdK|o)iI9-x>y}y12t%$XB9c*tWrN%51_%2)pr@^4MSNMjicEYP2;(@9TIG!eU4FR zS8ywAbIbylnvb!8I@}(~yJ?&(xMh*RpIK(%&up`pc)vzKKHg~V4DL0$J55dT>-i+3 z7#41vDZb-UAVG~`Vhv+Je?*YGM+@}wy%|>F)&x2N=s3i$As4vk;4vTd=xnbNoxC?z z{$Ll>qZ{yZa8L8eI_O376gA5me6DXE@xnWghyk1#vonN$ZWK{doH8N)NZ-_aFhxuP ztR3Q#GvE#x69U$x0B|@nu-x2cRo2rEXNAb%oM7`T9|1f%1@vsDOG(J}X{c3yZF5+IwD@1$vg$Oi3RmEg;4)6MjPlRR0$M=F#W@s|0$=d?Ek!;3U<~;L zWRMt}9FuHcWMhxkgbiPIn?HQn7q%Mbu~`l+t2#JK!Kz{)9U5%L&NSlb&NM3A3Ud0< z;@@+l+CW`#efr?DO|J)h>rEY>w)kA13YX_n01lrcXISTOiN{L}?5ExO4*6@yHJ6H7 z{^!1+@zV~-TRVJ#ca=*sD%h196VybjjGXu12h9V2?l<;$7g-4J0uT3bhtKoG1 zsqnsCNkIR(&!o8(72z&9HuXrytT2SWv8I`{#M=M{OsxXsNNz3*d(mYUZH?pSV4QdSn#gBJEOIF8I5wok3WLVHEk)Atj{)J})&+ z5)*cZtHU-2D(<=@2}iY@chOpBD9Zj({7x5ltxmU&usKrbMltgz6Vy*ZbRsmA}-75A}ohb;;-zyTj1Ch7m}JbMEhFD+>b}F7{O0(jL%q z(4gMafwvZV7i*a=6%X-wg9iJ&PKEjxs70SQIpj|p^-UgaJ!_>0|J*8O6n@@ldVRXa zq1q&s&rFj=x6D(JkiC)8$F@0C`Z4etkEVw0(a;e`3j)eVkd>VB>R77;6}i_3d$8D^ z`Sa!n$OEgGl;cnc@QVY)%zD_fZ~Lg}51dj)rB6=Cc1y5v9>95CkZlhpv9a63i09r# zHqI#FR;>3{Kl!*`!#P>26Z#i;1kYMCy&XBf@DdAd>Gx+kX;iC^dbZc)(4 zpdVNtm$F0JvF=?LV_7O9vH@lWo1vHU;v|Pgptk!UVr_}t9 zfnR^Jgex>nLhhY{ob_o}*yvlKS2$GEqTo7OAqgT5NuAgiO zI7b^?Qt-#SOfEG^5D_N8?w(~P1~5tA$&!F$m>{M&lx)l&do9B!n_R9}M=v+c;uH2o z#JIzGN~Ujxf%(hKtN&x4$+XNPGyNJCU@}ysPlZYN`v=tdM;#k|q8CuT!CH|LGE5>| zLhznGn;e~JoPs>^stDP>1vbJi<5gH^89BxoROD)J{-wQ|6;MCg$o zSNz8ohjO~g!2!lZyXD*phm@LU786r!Qcjs|o>dIqzv4v4L;uYua#qP!Z`lLOG%CTT zV?e!nHp-)GZm>!_+Dwv0z(gSPNd*Y=1S4&K5F5VmChx{(Z^;ekJR=|E65w^L!UGAX zXrT-JQ1cpc1-#ZRpNx{@n5Pu)i}1;2kZ&xqdLm%MioID@o?*5IzBwjD?~DtoKo5di zr9ll7=uE4W0kNxVu)dXsdgC0ccxwXp!Z1roxBPwY7CF_cBxeUS-P4}EBt zz@K`Q)iFRfk%4748u&<2V2MvST4q-RhKjS#mA^RB(jYD=v&1-ti~gwNk-zM=i(QKx z>h3t=(V+yF=~nY_M@x+jC+qbu?J`QSvL)kp#mm$ypf}51asgpu5PttaL`VtF5w^}U zOHP1}2Nr9X8+}!w^2WT2(~Ow2+r6-4ur`IOy#>FSL{-rNH5+-X~Z1?7c43i%H@CthAU(4O;5C1aN zzV@#2DBy?uaYxv4vd&|C+!EFwuX8z8DW-6#tMK+xXYrLoF(K{arhqfol6h@!vNGvt znU45lokRX>m4@_536<+TkhDR1%uROorGEr8d`v(`Ej`v#OTB7rvSFg+rfnV-VzGvLXoVEt18&*qwE2pM(>Is3~RtJ-~(b}x*u>h&{d>ko{_iK z1$}TRs*d+BP>N62=x~pYg#tMbb|~Y+?dn>hMP7?FPqH%i2k?>d#?%YOvEmTdJTA>L zO-!@Ox#j!P8pM{UrraRnLOpU8%rb+GUGFWwvpo!d3|9=VMe5(?U$V`TbNsp*(#d)a&lH?bJjX1u>*@vW#X4^Axnk24I%;>I zW~oA91=!> zRfLQCYnem-W9MbRbCFLyn5Aa87aM9^i!HC*OD(V6`Z}&zf=^j`mv?`?uQ=AH<`T^_ z!m6eB*awDjW{z!6SZ$lbruj9PJo_9valH$AZI+k*(>Nvi?CLv6_`wt5|g&H@l0j&$1%_ zt!+#`r+JxjZ>v8q{?jIv^53S09w*Q&hlY*(wA=XJv&1EsXGvMRqww&h?)+;z;}r=` zIX(~M(BdOK3prDt^6Ta>H1^{Lhjh9~g*eBG9;j-QFT8<1{dY~l({Edm5x)bizy zwjyaRD*0edKfIH#(f?orheJ!(G1tp>SSPe=F`uvykQ z_;U+j*?^u^{b{A8!@JP-_E5?NpO1$CK35I0d$wb~I&OEk;NeDZ^6w6?-oQ^91Dnn7 zEov?*=)rO{ku9p6Jy}9!XIDTFYryBTViBtLawU=lI_50he(&U-{$&)QM$^5NDc1M;^|Q zOMPk%`Ol5o*8kmSvXAtr9McRbdvk~svoS!9b?aKYY+5z~WV1}Cf?oM?joa|oHjnF? zrx#hK0RPTnq75QMs7FI959UX=DtP`5@?d8a9<@1!E!dl9R(tfd^{1fLy;^*lTTMv) z1bX1}N-g)xs*roK#KRlKxP+bI@>_cYCD*K@l}`f-bk^rJHulS=kpE>%C^%kaqg^2D zI%i>_E(t2aEyKk4RkVU*9UB<{_3YQO$^(FNPFMJp(-i^5uVX-5N=%Yd#VmCwglLzl zk!_oAqI#74s(_A%aRSYD${B@THKR13VjItujjCYG3~Kt--vzRxvF4L{BLPJXs99 zHEDee{q#Ug%LA;|=u&Vhm%AXhH@b4J9jOU9AN8y%pNdjpoT^IM90bQ~oDgf3)svla zUZrsgmpI>)c7B?l__b9;PCNn};Fkg4ngon9i-|kc^Kbzj2kDfP^9-YT_x6W!uiC`0 zCr67c)X`E~w{M}T!KLCNy+F$x(%|ody3&!#of7E(&)$2#$8}wYqQAZT@Pg#teEz;T z$-T*>Fp+EnI%l6QO14N)9V`;NC>CHa0EQ?74hDryRI?n*juXeW6xk{kRY-}wiliu! z>Xaz<-WBKFeDA+_Z-`dWL^>5c7REA`L#_|06) znRgyreH6~zffqV&e|ycGzuW)Qxqp9l(-QCEJ4+wFvVWv-a@+E*FV^+99N)C6>*$6j z^T#$nG4Scyj_<$o(`kRX`^VG%eD|}{z71!2_NBd#J#lW&io&J$Rt#O(`_!{QxBqc)8o;kmJ*{qB?1{|x6KeFAvt z$u0Rut{q&l;_TjmjuYE^XMwKz;E9brGcN7Rw_SU0p!@Q^-i2qjW@mi-;(}Yhe6#%r zUmaZa%!$ptvkv@v&V+-{PyfeHUtV-4;MfiJ=jsc)o__Z9wkMuG477Ce-Q`19K6&yv zpx32Xck*(s<a!`1FcxUd~VaSdFM9uv>ac*Kz#P{yrxfIeDt;> z>$@hM-qt_!tM^9=UmaL6a%sF@9R z*ZaS{?`IGG&5<`3d1tovK6HHRiq6At4bJ}LxBd6-e{Sg=dw%-pgu^d)+_CScGyZYQ zntT6d=hOFo>p1A#mv#?jPHf4~e*ZTM8u$HM%Y-wV7EihKPUo~EZ*@NK<+|?aCpP5g zo!q#p^W@eaK6Pr_P}|`*+V0r<(+B_R%|Yvrzxd5O6Z;RH*_mH_Vr$<5upjUH{I!7@ zpS`qv?!kZ0I!E7FsGZxjXy2t1AQy$1fT1+tg5~IZs^C5YKRBBC)AM<<$$Vv+Frbp_Zssf>vpXTn`+2f2 zdmfkYtUt5a?~CQ?ilfSZXL{)o3*h^F0CB0=BU;_ne0ZYDo9Fw!v)0$2*R7uJV!a-< zp*^E0M3~`mya>_k_jb$6ayn#*5*G)joSsUwxQ%01`j`7x5n_)C}L;yiWt z{$$b*32^f&axv~ETVaCBn>pRT%3EIycpOE*_rUjQHyJCfrfa=BKg`WIKH~2BR6b%F ziTM7!`0ihDMe7ChdAarrzeXbb1ikLBwy|<=DIL%#{LX^&reCK9b5^a=p6GO4edp)i zkUCeLD9^V^OPftEdD}jf&vw4ujjyY+l}j1T&a4bUoi6snsET;&EOVc;4((llYeAiy zcmpl2n~kt4pJ|7STz|a#RK;cEDu=g$`7B6+kN1>!(RDz}wL^TzssPM+Tio~+{kYz0 zbHy3(u&F)X?l{xi-QugT)E&V>;mZHe!l0|O>}~UiuS>clAC#F%--=(SA2!K9ki{h4p!T@F>j-5O_*O`eoOB zbt);izKcra@!wxA-eiJJ3aO6 zk!Jbm#M$et-WC?A~7)UDRyr1UjtSvn_ zA9~-o!4`u6aIFG z?c>X4`K=-8(exFbHE(UL$7$~orr*`^>-i?8JA#|uN@Iu5k>cUUWL)X|_Q$cN=zM&4 z7ySL=_pipsZ0HHGn`4^$_VIRUdKqDozuj(fX7xg`;vLv%rYa1D)zSeleZKV(eUZIF zcntWJD%fSR`sc~NUQDpg-Ln^}8#}Z8rZxlcFnV{wJ6#yp{O_nh)(4V>ijeUV`||UB z@f?46$s&vK97pcESS^G#y(To;T^6JfT7l!5-D1x;^e~)qw>g;&msT^he|;Q!ujbqwio=`SFPI5Q;J2GXuKHYY0{3*X; zkOjO9{b_d>NJm@V7s0FTl>84==+NZO?L$+LlLV={x7+ zZBK0lI_o_STk*#agY|i?CKP_F@?vb>9Oy=O-1)$+mP6(F69BJI3vt`7y6YTTzRI7% zQ*$=w*!MRrEH+L`c}yQ2&+S zvi^pNr@!sJZDC`h3rbD_erMA?$HAfgmcen`?WfSuKf4bX1$(-Q_7=GF(@Fg&zuTJ56qUH8x%EGz+DnVR{H~czSK0svm7(pM1LN z9~F7z>E`DN3&N~;=T+&O`rhA?ZofPizijj0pYE+QUjDjUX!R3ZcYMr24>iH{CJ+*E zc3SW0OFAQ*74W(F{CfDD&gzDEoWVEbR1|Et0k?!WKh2?ijki7a-+4aF{XQ3oz1Qb& zdHq)owfqjmTbUBdLhbfsUH5e#zCB;n559QI1KhV;xFflLE)UZZd~F8uU5|G`d>!@N z9xQJeEQPusSyVT3=XET(E&F`cVm*sb2Ol*f7Im+!e_R6yM;bq#BwOR|+EeAczS3Af zUptS!Q6B5zsktix8+WtS>oLt5+jkdfua$zUUsv^+Q+Cs(Uw8fJHvUhuBV{h_sr*s= z5DZtir(KU**B~qAPkh@iNPvcq+DL$twO`wd2O%Po{+s8^DJkyS72s>R{%dz5x1hi# z+`r?(Bb)$mdH*=Ytt)XaxY}ZT+bq!kUM%l7KXs0!1%Gx%;7`iAf7?#lYIB9zCN!Mm z{hdL61$chf2RubTcE1&h+E{$f*K+a#9L?T#5@~rM2|DX|yRYZsK3mh=-L^c8G+(P+ z*4Hk%z7)2z;{oelZx+??Q#@Ta8H;J#`nBifNVhqdh^~`dZEgh*Q{7X~j|pk@stgVW zo#!8zEFN85zHzx>Qy zmXm`S*}hgk4K^MqSK@Cb9^XmRgQGqlrk5VJ-g;?swn`b@6u*5BFH2uf{bo67kM07l zuJirdZ>AUg08n3ruhX7e^t8v=dwidAm$BZXl2QSmlUv z&3-v5cibn>|MB*d65wlXQE)M{*4pg*eip7eykzF@=yX)}-uSfqe!Njt@aWPWzSjL} z_@J8hh*+dpb@kjE-tBGKH^i>d&nwvW2Ec>BKU~`~v-^TL0~kwA--QeOWS}KiE=3Kp+7cP z6%4+O9^#)PL&Z;kTkrO7r7CqaxJfEIeRZDaVlAsUF3HI6JU+}gmEq`Su)JXN=Ip3( z2RGB~4}agNiVH=^-E3yLJDAhc?%}S;-&u)t|7pfreW}mw>b5*`&zD%$v;Ojs1_4yq7()??UbLMj~ zyquuh*YIKg=qdI-Z_x9sX&5U0%GY+Mub0x_`DG{5LLqqS{^j-lB7C{A)uOrjRIvN) zA7L)|`pwQb_SMAYu!jDJ<6_bK0 z+|>q@HYJvek-}9ckKH(_C6$9-H2l$l-mExOwgO#D4I(0enV%n|>FnI&hT;cb7KlOeg*}%4eS7N^2XLx&Cn6ln%b5nbZTLbuNldEE{Ml9)(wlvXB|8NR_r6W6>rg!z zCrh~#mH)1K3sHB!5Oib|t?s+M#&L1*mmM?VjZUj*caRdCLFP=gGmWq_i(yC?qmeS~ zlXhWsTT1@5y1y3?_>O?4pLkWBr8n92iDGmfc(v_Y#q}c%3TQ5g)=s;zPuLyXc>e8` z-E_9G-r5LT4cs=|)YEGoed!IX!mfnSN)!MvV=9h6OpaK*!LD{q4M{*MPW7ljd=(h! z2!2La-3U%*yIk?F5^MyozZ)1`IAn1)n(0*W2X}{Y9N2sr8K0&mY~>0(mrA5-2&{8x zjll10^>iKTk}4Lp*YRJx$Ps?FOKrLX3<{ndp7Z){{nwUto$X{m3Na|++0YS#BHC7WrY?iJzmp2E<;v+lX&HCczAbCD}whYIB!*RMTk{tx}i z=m%$~iKhOf!kv)m7;c)j5_u}j$X#2Xg7S0vL`6+@3H=OP3(W%C`%;LiE^ z(4_)cCb%w^!w7W9JG+Cu)M%9Nlt_K2q2<*8=B*4{gI32@i^k0 zSW+p5D$Nuq(RjBt8*G89l^nA|1%5K-RO?ZwWj$MPU2KOVjSui~B%Pli?z3E0&`(W> zi14PmZxr!SChRmTW=Z-!FB3(6MVZrzJ}>W`etcNIVs}y6eCn-P`qstF141)@C zS%?YtXKIfGjpR-m+M)jY1=`-?u}RBvFtyJe_j`F|adTjuUude#J2U67W=00JqHQ!< ze`6{BqQ%N?SO>)-f!|*xAF1R5C-yXZqHGmMZYxq*9}JkWn4M=%t-a&;Dd zLM$Y;`NtxOAMF#_vP{_Mf?OACFs-%^jdHQ%cLO}|aFSG-PbNx{M$~QYBRb2aHFxwy z&Rl%eDgIQxW-bZ597n3A45ZZh2|g`WYV%OD(ZWY(94TWdk-3aBWfMTOy_Snlmv>Y? zY|~$ON_5ECbmqbN%PZPjeDK8JGTh~wjbanyypB5}$2VR3U+e9i@2K`yRI)@s)6ANqw^MU!e7IcZhx4H>u0W$0;Al_isEHK^+TGN*!eCGShdjj5bT zRs9_q-rW>+zfdj4N(XfkhQJc?l?rxOp%uczSD00ZA)Ve8hzpv=)vM3hT#7-3Y9+9jECTg_4J4kpWWjXhGuuXL; z1S|@=+WnKTv{F*C9EI-f-V0N|HafgwuJBGv%z#C2n2H4&!qK#MjEmYRHXlv0G~38J zq?3uKuQZpqR?|t>&2`UNIoHkJKnF7io4#$7PmQ$qi%UL`uGLdr$h3TdR^30HK78r&D)Px5f8 z6?Y+7Z8J)6mNg{J->6ugCKKv)4of+ftaYRrx!C-bGMYVHQ~JdX$rFa#$Ge3bw5koYzTL0VRa#P^OKwEfY^E>(9a*Uiwab zL)(F^imy&_4wP|x4a0zs;hwJ=_FO;+b13VS6&4>Z9rN1u-*v-Jyy(L+Uvk ziezV_#j}ZLOuleCB_OXSuOWQ4e850+PXoNR5AR>6T`XM?kb_~04d5XyOaD)4I+|!C z^B&tSQKn*Tk$-I&oq@dwwNA_DByqErPTYETPj(}cv7vl-64ljZwUUxXH%0*{>`gBV zH_+@&r#$G%?MCV zjB(HW??6+VWjeVlVDE|22qQiWu@-gPU@`uMAnu#=}NwdGUK zaRNZ$e#n~96-mU%g4!E1>V8OpXy(G7(Kp~*;%CgTy~e4dMshPLel6qc0`Sz(Rm8+z zu6L+HVJCN*AxCT*5KiMd#<8y*NBlDL-eE+4sRMI}>0EIhm#A;AGsOk_ArOdqNZ1QUf2SiLA6keUbCf6<^-{INXoRj^L~H~|AT6oP>mbVrSsqp!rK$CVd&KkNj=aw!8IHQqWk6y zF~wDthIu~=D4ofk?tpt9xUUjiHB^KoH;J~bZfLR)LHi6@Hjlz-yB8*3i`z8fjv22a zW&oFAsk3SJM?FBf()3SCyt3O-K}x*jrg_0)_kQLjmAGXP9KaR zG=iiz*8<(V1jSaH#O6}|@Jw_^wLvY6RQfe;8j)HtdGm`UBI0(JMhQIPa|}WBd+%ET zYW#^!F8t0wX(OvYx;iUM**E!_2lMp+Haxmo{yq?&!-0Lbxwo6nw2uVCNU7vMCJh5P&;>t0{f_S9c=W#;bFZvp#qcim zR!+KK&pyMYYKU{cF`VH<_N!h~{DuD%T04FzlyVZ-tp*H5>|lNrvrVRl)YUZgfue<8 ze3_2tRvCC0s?VC0S8gmZjYMcrL(%Jy-UKUQGO$dTBcJ_nj=c0pk}DmnCQ`WX)>-&k zkkG6kA2h%V2Md&`?eHlQTuy5RrHf{ySX2GOzsYw&t(@6@p&JSO8H6nTXN84~PL~{HHRV1pg*o2c%5&JjP z?!laYa(v)4*VahF#+?n04Y^>SNM!s7{JV~f zOOhcML_*__+m$&Zdy8u5e)DN$$Q4M0&m@DE;>vki)#{?y3$vy~tSw~ixA?5%=}xaX z9$y_!)w(@iQ`Ut_A5}0;yLuhw*8h38`W_6Itz~rN&H=wu%-DyxxceixZ@8P zTO0b65IPuIpF85@K67n958pzqDe>r4%)+H%EA99* zxKmz0M6Y6Aa_5)NX7$~t4Gms**Gy>S6=Y<|ff`9bcL#*TtX!sMoZ_a@{p`OYk3W}$ zr^lKf5nPi2>*5be06%X9`x=cuYdI90e{z@u{u>UH7xjQ>V%S5?;8|5eJ)(dp_d=?o+hC z!5^m|2*_;hXr^DDcO37`@WmaO*TrpEAbbAmFnTASXIfE``#!G4`@98tuR{~+r)Fhm zZ!|1OwH_WOw6{e2GwR~TN*+ToIlrmq@Zk^hB)>*+P;k<;QdLaH_G7={fM<;FpO&va_KXVq|k` zg2EPW@1)C>xLDZ=_6!P!T*P{IJq*VY6Pe>yqtqkG36IvKn=A{gEd6_cA02aM6$ zGyY_g94UI1z+Q;1mF_h@oF@3^jFFKOYYxTaf>9bsNOa|oI*>X!ERe^sc|)P+_Uxj4 z>|8729_A1jkYyJ+-Ec;(5D+0lZoYs9kbxqvKHau`F0PpJRt!WNnDGF{R{6Fg7~fGU z#C??4nvF1kkz8|iE zTYTx%LGYUa2H*^Mb7;SjCcrqRf&sJX?|k|@Vn*F|^7WC4a+phCQnIKx38^(97DV#U zztFrV1I(bnnCOEAF1hj$f|Ji;i*-&jf{ zUK1U*I4hgnNZ9~j2o_XskWpx)9`Y0pGL^#g54Gd&%-(V$I^4L@-gPijB0bd|o*qdz zAouqU343}wM0t!4RB|20kl06qd(9d+7{#G>AdFb*ySEthwy})l zr6m@XFz6vPk0{ZYU7Cc%XoLX-V(Hxy=#32yEwZD4RojRFpV7SF8iNWk4Kzoc4*v4M z9zE2j^<3`~o9aJ!0%jG@Y2+5JzhcKh&pDwgcv#4GXz(E4(XqQmKKlC2j*xAKJ;D+w zLO0&YDD0lOR~$R)5|zQ^s=wx`CDL;< zBTtqRU`qpz&+J?3$)yt)LecrGPy^vi+ z;|A$^wAw{T7Is2Nz+F*^Km*cCNU=mZ(lKTh^@F#9-OxG$h&H6pgo=Z?BSK+G@%Y3B z$r}E+w8W99IQP{G>qS3j3U&q?)CuDbO~y~i-tR1B(?CYi2apF* zlp0(Hf68_R<3q7z&3{~^6|N}&F^(eZ1nn{*Ucaj#>T%>-PB{dTcA9zl<_Yfd=0btQ zSAu3W{`}(;zlHp_D`@T{Ux;Y0N#0OGh`6E$IvYAB6bu4Q^(88L0$O8z(2Zm_E()5V z+iFIo$Gi18gh`o^>@!rcMLsS$fMXOFM62O>vcb1$_m9K`UHD9ID-Oqnm1#m|^>~6v zFso>k91%-DG@=wWk3*@#Des@i3OFs8e^Dd4zX6~yC9XxB{i!yhWD&d$35n1Xu;@%^ z<(6dEdn=HRD0ol}#(M}nbvE(96m@I2DwheN1L3#Czew&CX$oPxf#5^v;_Sf@=!-ZF zLvkvz2s{BHxLCA_ynW^@BVr7ZAZoJ6hBK?ITC=45>qH9hv_g_8z@%Y=TcxUk&6A)L z*6>&>P0Dda&k7!`HVM=wzJieCFts@E1fx+0(wALI{a8@3{Gycgsv^u_I+>k*$hGj< zl=QC$G_bbh9?^XnmT+;GQCDJD`uUrP9dH=}wY!Ho)`&@#FiG%bHe|{SH%mip0$Ipp zf9mIXVk@afaDNyzG@+2LqWJ`hvy3kABh~bpuImbufWf)?&=3nDNCT5P4?MhAJDU!5t9`>ZSl>~c*$AEG}%rwQVu~GeQ z6DoyYUvlJ-nzodnli~-7fWx5R6MhBz%xQ53AFsBUBOnP}fT<7tkmL-VGLMZb0ridu z_}~@#se>79RL7NBlUf-O8+q>%@k}u8_~VU2uxOh&N%R*O+tBL~Yt$gKztVb< zTSQ9~5d_M={yf-ZM1+lYwMGQH{v|#XiG9)9_t#S7F=pd=8L-DU4#$y4Pwfb)uPY=z z>^>v^ZIu{2#g2cn->(`0QulDk31YCc3ISrfN{qUpu~bVDOTU5`?CR>Pj}mI0FOQTQ zdK%rUCZvE1p0J4oR8X)eQpV-yXZD~*5t%5)acUcKl;<2+B5g9RRhQ>Xb_^We1QJVn zw+SO7!wR6(4kUr=wtT{*eCn94i&=tzwdVn4>44mhQdV>kjTzFBnKy#&F9>T9S^z!&`KhNO;MHR(}VQ&N?&b=Ybd<9Oz2~lQk zXB&fLIKI;enz$yh&O5mW2Zy;VwTIipzfG078+`XTqynYXZ}MVkObC|l?X~_U#p=QO z#{fy<|@TT!aE}z`kQ(^nxiohlLFzPnk$l-?k#KBJo)l0@}?@tVSzOWQyxZ4Mw zXrb_|@p5A#?I(_F*qKL5)4E~q9T2jWHKCs{TBajJKj6c5GAP|(*g5Vc66r#}y{#c! zkUUh<6=2kfoXf_NxkU6)Tg)Tc5#D9}@6(AfQ0ZV)_NG|SP%n@*_A>_~KxIM^3UL9) z_)^=%Gzlpewft)FP$)WglOzp>eeu7K=_5bv66srL zWV6t6Y+|TL`)p||RjFrAFD=mqLn?m)11xKZFjSUsmsd$M>aoC-;VM@%U)f)LNL{}tkRA9M7OMn+G=8`EZ>Dy;iKWIqr{`a=dNQ622BIkbkAwnAR>9;f;gr7+k-&hoJE$km?j|pP7oWYx9EjVuggP& zH)h!jI-(++7Qui`nWa&SKs?RzULKk0O8PQFCu0hl&SY9+P!qGv9=%UeGfu!Pj|dY}BgM1R%PAf5C^d~&Ff04&;6Bq_;(m!T{;FTkO!d}4}mSEPQgmmPqUh#|+!ExCPEXy~du_3xtSzjI`4rCIn!U6l0SxD!Ko zI)=_V@OM~$&o5oHKD?i3WV0iUw8bzH_88a2{|dk-2S#Y*d?v}E*}~YFb>p3*uPu|R zLP`=YYPQU4nc_oBzeRq~N-dKI8(MBjO@tnL>;tsmvFK8inbUEOIvzZdky34jv7~9( zUKwhC4EZ7gd-6y}^ueLH=-f$|=LQeK8Ur%Om1Tq-)2!HUust*sT&hhalC57P6oj!G zoM|yvlB5ly2;@InYYiRPNG8frU?hfrEEV^yRjHwaQ0T!2d@CeKsh{EGUEx)PghJM! z-;@qg>H0ol#edQY6ww<4K-lY%z>{!Vd}m$dSCm&`&&gHs@F_xqf&t+Ug`xOeFW@t( zsqkECW^z1qxtuXG2vhnmu8PR#-#{s#8%HnQuFEXBfe16YvG-NWj}6uWwPRoc>Efo{;!~Ah;%X98@vBf<+6v zsl(Ahd7p&}Bg9{i3&qKAfOp%0#sJLs*pWOh})v&CB%4)qqhVdfHZNx z_fC@-GMb!_MKg8-{#JB1Cv`7KzucN=jtGB@iJAFrf<)sW0Uq4j5czmyhv?Wd8DEaJ zDmW#u{i4oQ1i)v_3EAFk0_-`bl3`)~1B?301!gbCaF8Yw4oR0r5Imzw;f44dGL9mK zNf+8Yc>qO$#0ql)h2$T0EihIpNHPL5VLVi7OH^3X98njDucsitVv za4k^QpCGWnBZfOsEU$kpNTOFmLCx7L26y4-n1!TY2pac+00n={bi`nwb`dyK=^iD9 zxb*&Olhi-kBrO67GF9Wh#F?_lApRdP|cnQl`SAt@fRbBPjhC zt=y5ioRJ;vC|ctcx7jZC#2?y&Zd579+X`Qi5a&#y>LF>LNQn|m0MC5vqQITBB(_H# z^^QCk6jQPA%z0sJ(<2WdlaIk-?eaus--2=8JylsXbr$PVSPjQ&7n+A~v*vpMUV)~- zyTRvJLTg3^9j3cWz=Det|8~|B6p^EiLln)2(@6OVl1&$?jaLLK_7tMkLCj6TQ^S^e zZHbkYOx+jCY8pMeEv#O<@w_~Gfs$baA5N8f?m}!pj;@+6w+?(o{}Y+Qwo_XaWgN1R z1WEE1RLdjYY|PCMyof3K7(fY@IYgAED5y|;{JBd^lw3G|{{YLKO|2my%uMbcSAdjU z3zp=!!Xrwu-+iXW9%>}R1C|?Omb9(aEVsMM#WEAAr|9{PcH&Cz9ARuFQ7cwtS*Ox= zoGjkP+rO%wi81*RhBcmNn@x?`#d`GnsE8BE!v*n8ednq438tj`dE>P)n z(wXC6x{ipp9P>1Daoz9gsvys#57Rw2PR6K2LwMkTA1>^N?I)sAR$Ggr-Xxwx0F4;- zu}#F)lc(h2pLm3x5D3yndtf6^CW2_8*_0cxm5dFcQ?J@l;o*QoH&wB8U!Sv$1=b_5 z51?%Ioj4}AwjQpK=u^HRPxC-<(Ob}g}pPQc3?|E_X}AqmhEs>*|dx;sg+>=4XF{TZuTzc!X^I$DGJ?-R|q zj>>C_23dzfmmKFCpYqS7z*d2f2nr12mp7jK5<7N}u`CD1c2V=5@Qi0Z8;a2WvpLy> z1`Qe}BI<8qTsVqkc<9~(uB!0$|4iqBP{uZON zM0QuZ8@9{|b>*4G8*P@aEljUZGl=8~DJ0mhQO(!_6~mJm6P+uz81ZCL!`50DmVQIO z2~NFE4%0WkN4b-fC7?6BO76FjeqcBi|D#C~SMryVW>1_rD4t74rp36g3Y15m!0R1P z`$EJX6bOcBwy*1yrDYBudVcuV2%ougHX5b+9475^V9a<9LAk%Vw4lfdk2&8&gX31B zHwIOBB*k}~0R(#ex8+9YUjVU!JKWpho zU~MzmbjUn7Oe&IPnpn+h$jeeoaQC8Y)M^oI+=_@oWT^iXV-!a*|2e>!b&gZ&ggW3( zx&;C6jeG_T*iE(^6{#LrYZ7~F3wa=wvwEAdT?Ca8&Maw(h4E zVGZbnZv$SZ(_bk6!AA+tXR~w#QBj&H96>o8oX4(RSr{NQebm*~iWF^)3X|ijA9agE z&{qpN$Ph`m~Aa{OCWp?vl#KUCO$&pnL^f#9_a{Gp@C@C;oqEVK|ql zYoQ046)ixJ!$Jeor4~6f`G}w+ro>&~gJrRx9jvbBYvr{4=2U>`VQh2IKpFd3ZXd{C zTNd0!m0Q(;X|_LXnqAcB5EHTDeI-^a*##7tEdvpT@3b5@JMYE=k(*a#u}}q5IQYdz zmORx+n3O+x8?7ptMFT%_i&Bpp>;?x1DdqXPT4;I9G7AVq7N=X>h}01X;+uQoeKVbr z5+I8nBj?2*BTW0IL{+lzbR2ax>htsB)Kp{!kM7}OLrCeNFMr!%0jm>JWR*q4Rc$#| zVei&(&ll*MOAaSf_^VxmNlyDpQknC!<{$%RN^U_AF;C7*uO2|DNFAR7lI0f2?W6?< z#Z`XJ{lI6D`v?zcIL2m=Li)W;ygKv~4Hwew`FC)>`{0<9lOP(jHp}!d)IXSVClajD z3XQb#;t_{q6^_QR(&}}=@J}$R0OAZdcU&RfNi}6rfQC6U0=&;YmMPD8IJ{TM0+m31 z3kYn`pqLCUP#MQf8y`ap6hef02=!LjdgxOKIeX4l=`mggHhVD8&rgd)b&}?Wi;uBL zv>A<1RFeG;dL~GlpE}6)Uj@Xk`@Gv>2kIO7B9mZWpJPg8cIdx%#1@)Rm4g6CC-#4W zY1I2@;t}M7YB%%`F(;+tlHG$02%aOw7Yo$Y`Iuh8hO#sH&R)UK^}cd7l$aTz65I}; z0qCB>AN4p*-DYTA$#tYY|nc_{E_}_n$2db5bK4)SpOQluMhTI zJ~ee@{Id?p7MKgsWXsG4$=t{Ch!IfT;!}HEj@kDz@!m^p|Dk}?Msq!vdCU0~6c1?X za*eMhw&D?G$>k(v^P9u9xl%TO6QHt?Ns;tpxGsq>F%{b}-dBBPzl+Jfmpjuyst?0P zLdu8E=P*Z0k)B)$hVsW0MQ~7OEkXyuw`K=oE~@@%k(p&2vvM@yJh;pEMd}wfQAhhH z&R={t#RA=rK5?G7yNa+j#0};>PCsZungqFu*T1HV|ugq)$ zEryuh<8w;P?k)Qs%m$a^O{q+c#yxg&Uh^dFRi+cANS%~yxgFETmT1Yw`X&Fx{1BM> z%$X6ZR8FJ9(rSe+fyQ3o^3X6v?M^>FBL;owS+V_kRW|&t*pU2e3)e_$ zh!jLTO$51{oH#96`n_Z@zZOrF(zKV6*%frbwk1C0#D`*Ha#K zll(zgDY;Jm*7iK>O4fx9z3H0j{9UDAPKL^+QZU8Hef3pbna%FwU#z#G?^Gu#`9}(M zCMEQL{xDx+e%ZUnbcY69ZCN>yc*Yy(vC zGBzO3oW{=uY}MCuPbZq2EWX<~ld0{CLvI{o?|T}jy{(-o@ke&z6+T4V3`6?JGe`uH zpM|#*&b;_Qhn*n2wN&rCO-w+~+-PveMdp#n&VjCxkw%;4^mt63Gg+Fu_Y_ZmShmT8 z3VsZVt%MyFESGO6^EF`GJzR`B>CHNr^d^9~+XRDYJhr44gTcZn#WTpn>fYdhQHoAO z&=~QI^o~|kD&)Q+6#44QQW>ZSys9uMi3#N6QhK8ab6CcSG{@FOZY<6m*#>*g%NB!X}AWqM=)amR~aB*gDymkqq3@WYQiUD?AaRTIx~7bQ~pvyF@m`^ft!tAGE@ z{|(nbPLqq0K;<1lr=)NAvptbniISBa30_LNs4j?OZ@iPjImX$V4hlHc?F_f7X_1UV$O?dcvOhgQUJOk5BQ=QbSU<Fa&pkI|L2xF2&sm z?(SNk6lhCucX#(v+#Q0uYw;ol3KS?5>BoD{z2n^T=lhefpSh+ydyTPIGIpj|u#a~j zik<>+hrBvw(&b$-F(-TE=nFfbG}%WoMyS%zPiRG!>@`9zxRUR+;P_MecN}k=hWCTw zRcjA~Wmij%oC$xU-+6iGm8+`U|B00ddc@BEaa9ZzJJ>qFJBx~U+^j3g(pwK%C;Krl zNS3-XiBuj~Vr}-#+kb+JUo|uSrytd3$x-~irBu+6d3g~RJYusYrZ|KvJPxG3d97T- zoK}}!qjgq>61rCzb zbO~5&oTQ$m`sqy#T-?L(S4DbE2TV;+t)AeJm_+Z$^v!q^-Qw z7TM*ZxoAfDp0$I&N2SLI7|&QvHd7a+=yq zEi=c_;ORjLVi8yeGg1NThU2nH3M--z8nslW7_=jK?PE6j>4^Y$!mx(SQ<}ZDq9Tc!8u0%`T(Z>W=w@C z#@$rY@{YHIX>k=hqM^nQ**$A%LHnlRt*+mDwRY=JRe|GMe%^Bn!uNL!9;J7TLrs+^ z6{-UlW#?+034?``ET&EYb>q&zE7PJQ_#+x0R9CDmg&P^#Ye);|UBs2%{##%^6(+Ic z;Y)C5NMb-$#H(`=ieT?Ml>m|}zynI?$y`MCt=v7Pd`jiB*T@IUO35x?nCoB2R#&aa z8&eByOgb@BFQ*g_mWu{?PVmsr!WmD$DLT2ogp-CAELP77Yp%pIyV71LYg9+E5-ZlEP;tGqh-C= zY;_*XkvJAgLOaTeScF&bqsgPFR6dn!L@9Oca-7&0s&P&{m0B1r>jTOdmXkU*BwLOB zgP*s#8Wq0Eof`CK4Y*s9V=LPZoMBBY%BqdgQ&;v+3FH~$-o)9o2j;_k;>r^%#u{u| z@F+e5I@XwzoQuSIFB9CH%h)!!_Vz%7)y^S8%=&K)Dmyc&LgK=%*19-acRU_D3D{ov zAG4Vjakn&xDv<>f$#CBn$W^Dy7IQU7pmM4ajJY+~=bxL)M$-W4Jm|1&-E7`_d)ZpL z{^!cw$_a~)n}&<#KNn(RoT|2N_TCOO0{r~E0umDcM-P2~eZHPbZj|8%L@#aylUVBm zmunoxO4TnBE~VP$!$Fi{Pn&#$irr$qYfW}~mU1gh%E@o4oqMnbMbQOzsSk&m1Mi5K zX}<1apLl1Gw>0DPAqr5zAZDHAcp1Q)=lwx#bEc4AWxFdxb!EU%jEf~?5cGGQ#m79Q z7hOqp?wm4zt}~szQxTmIsC3*}nGd7f?#^N2efx2q6h_GTfNgaDVogfV@q``mT0{Z| z|A8LLEki}r&+|$o27Y4S20E?}f`F*9MebcY13$}bdoGz(M}spP7$v02Jeppvy1HLx z4CW}}THHOFgv1bM+p!fa^F6#mM=%!>d!9(t@Vmz&^GY=$rH5-Zt&XqIO-baArT%Ey zsnaR6^f!aDG;Z&jl_Ib09Ngj=BLDV}Y7=y(U8NxdbxUHOqM^Msh*M=^B)hshNMvKBGi+%fV{$LO%)j&x zdgZWJ*sTFVAK>?&z;j^=ra6j?A#8jv7>xCnOpGr-w;1z7MXoeDiK92f+6^wx@uDnX zc2@k>MPvF_{ zg;!Qo{5mX|z*6>7kx4XX!7@Fs$2BG(5AbupH@*-rOS|283_}x(i68yteJZKCVB1H= zRRYm?_5c(sD!#PtD?BZP4>pnQmEL0BmoWSi5qi0^8>u`-C>j|fQhI|v@@M_skyD#0 zk<>W19%pZu_7Od!7Z5MqY8jytX~lde+5-!G*cHap;oIf(2xcwr11R!aY&ff@DoO9;dPqu8jB zjHAD$<6HcY&15<0PTio|P5nJtR{#j1;xEhDG;8Ge=GJ|g)|r>zUUOY1oYy3H`DAmY z>Ip+&x|Eo$_dD+RGUzhml!I~HYEkp)pwoM|vJ5AMhJQgj!~3L>ABE84Cz11SN$st@ z>}YTYVUBg9Jje4UztkVkGNaS$veX8ckCda0!c6CPgi<#F>0THCA`8B?Enis{=@u)v9G zNd6S%9A)0JAjYKpI&SIZ*aJ`@<9t`mRYP4LJ<7!PeNsw@zu| zB2PINS?RsxCvF&yQbAw?CtQZA>6Vm1cbRKUAtW$8Px4Hj4{cPG6ktZv7yXAQ9H;Y zA$pg6F3F!P9zJ}===~GIhOQUmF|Nr(uqHWgS+fMww#|jxj+5$T%n=)o|H)E*0B85g zp4_hGO&G`OK0*8GJ8$UDO{}9<3xS}iFI^GM-cJGk7-wM{Ugon>_nMS_^K|&JOj!>} z6SgQ<&@YUNS^Qao8}SfVT6&zb4@%;9J|APc?TsP))u-P7!qrWa9$sm-Z|FY_-Y>?5 zAsYz`4iRf~&>;n-D(ER8) zrM`PAZZ)HtM*wG?SKomrsRI^-Z0K=pts^b=}-)O*$cSM7cpbXPfVZ1jD}#p1EETM%WoTJDePgeU;B*N=(e zu!li4(gYz!FhEF}tSi3ucm>J4EndQg9C=rfAJ-C*$L9cnG}=GHtHWWPTfnS;G*0Jwn{C4H2Bf7~~=k6S2$e z3EE&bYRzp>%HIO|<)SWSB|0IZ_~SR`zSB(#iB3kO)}~G347sW9`QM~6?6bC}*j{u~ zi0@~3@e9p9o#*Sv+ttaT^vrmEJT>jz457I760GdN{CGlY*@+Tra*O=bUS-Noy1>Vo z@@<8lxM#tx;d?>gEyp%me}+&pEX0oLanlhxrt(V!R^uA20&W*)%i~!sT|p+wqw^{g zIw>UF@>{fH82g=5jJxWslv7$xcwQ(7o&j;TBU3{8uv`pc+zSJ!^m4ZDPWa_)qfDDp zyH#gCc8)4^{9N3+C-_EhW8dRqqeeN=!uT--@egS^4(UX2D_2??<=q6kJGcE%S7(_% z)%Nt3OHu;U$$BJrn|z|gN4}>cT}tkWC;wxyjaUnx&JvMhh3hVjGqBAK;p6q>@9CPL z5)KSj?V;<$L+aAM?r&xKIoTW8zo-<&{U<$I=hAM-?C)(BvlQIp6s~OSygFLGWT)R$ zLOBG#V?Vcq{Zszu=otBI6j+&nCwoWm?^ik!&d?MJ`p1oeuuME|oChrP&+0DWzo73M zIB^un`JU6%YWBY^<&_?Y3<;d_aqToqBh>sEESX1unk!Hn%=M?r4E{fYwL+*;)(_}% zz~1}`wEtTPj>|%S_$5y5cZ>wPlf5^KwPNVq{tr^vG)8QBz<&wss;@?uV*-dtrqEsl z{#Ihg8*R1iw>}~m4w%bR&%xk*zGT1pJ`SCzpR~;@+hYZxba&ybebhfwt~!$h~hBm`=|k~cjbw2PVeJzZ~i{eJQmAg z5e&^*R_2d3GK4OP@?+dFk}~ocgZjpC4D+2MUfm{C1Op=-$B9)T7bZ31nqx_8KJ(UY zPdbNPXh5(lKjJ#(!uEa+`XOOP8LAC2liKE?d!q#F61z9*d}+RX42>TUt~|eq;&VLt z*;Voa79L*d{h*iZq41fcLc#8C?`{_=DeLyFxepjOS~6t`3o(f(n#TA&!Pt=x!C5sX zVm46;dKXilvK}*3Y}O{xS;?PPBR4Ru^R6rXb^%wGT{uew4j{9{AC2n|4A3{6A9Y}I z4AgeMwxU+B&s7vpb~jpkPHV7lN?m~<_YKuU!fEJ`=PAA<1r~M(Mj-1< zI-_ZPT1*wC+oC03aP7z()_yF^ey+a!&36eqZ45i*OlDfmFzUNX-t^*gghTbY76^uT zAbiMPz`(6UQ|f!pQCTid=3t&W2q7RmlT{u#X6xWx!pn-v-DnBdl(X-q$LnR0Vfs0< z=bC7lLK7LNbfb&bJT}gV!kdZX6@v3xS%v-WSR=#zpRB-a5j^Naykx#*ct^+usonJy ziQNgeizxd;N$YGIv-}7^@i#LyXJ-n}j8@>BR>^f<3pePjjbTX!9l?hZ2hFon72r#8P2+t<8+ou?nj!?B_$3(Y8)~W!_xLHLe^yEMEt(7uW4byY;c5Z;y-uo* z9tAY;DKz6ijwb!M36GjFmFr-V8Ca@>483qgm5zfEBA+zV6_)TJEa>14Dk%=5UBD?e z(Cm@Q6%OW+(7TX?`4h~=hk*Gd;Xe*;nUT1E)Q-jrFajQ8FobZ%O4A&L@kGjE1g8#z zr2*s@2>iaMw2_@WbYofpa$@A5^u6Dz_at<49zHD?UTj9lBjY^LG;$_pz*5DuZoMM? z22>6n2?o9!G4Hm@CLzowc9=1EGRK2w3l$t00CX@`c9|rkEnxVfJ(jl++P6hfkLDU9 z<~%T6k}67F>T$d&J0v>0zs)UFyh4{%{*3HtjI_wGZ4ijS8w~b{rf~y;!P@;XO4T*{ z1I<~ehbsO_DOY))Q5Q-H_C$#Z6?-Q>U`6EP``fS$fKhh_rbW_-v&dkc(LC90(H;8 z$aB)MvtHcOCqre>#xqhT*H5ImmzYY&m?7PO&kyYpI|u*b5mplP%S?^O+mK*x4~`f= z!~m28Xb>xeedIDTEEMLZF?XfdJ7~%lRX}^G|-NM&cX>*q@ob)m;SoU?3)Twzu5Io!% z=-MN5ohrqR*IqJc(3V8@qzd}+p0Rf|O$xrsx{IvOC{^8in;CjeLSHETC^_wuMp}wo zqN(^$VsriQ8C6)g_dJ0eVjLV=W5a)Eq(vh~?|UUI=S}-SAHr27_JCHWk1;#!vj1ca z;1wtX;q|k^Eg+y&pf)e(IB~V4Gj4Yin0e{^FhFh{IV|1UMa~ZV)qsOOA%XN0ZOCPm z&5N)WqHavOeky2<~>lYYI66|W*@=s26)V+dvYFbT3OABVXS%V&TbQ4WO{QA z2g%-DrdvY+yQuu=5yUi4d?NX=d$dFyTmpj>+BtJkegjYFi#xL5U_s6X$6`!9YDkm9 z&=NtvxS|>)Q{f!VlwHdrDM1b~Cw0$%cysW>Ch`%>s}otT3D&cdp&-o=>uW#OM`IT@ z9%sHDNk8OY%y4b-hxzR+oQj0>lsgW_+#=un2>XctE= z91YDnI=Lwy+x1muhg(rXKB+DeD)ZA_tlU1Ms^K1@I07UWZ+4bQt-xYcS9lP@r?1aU$U>+_%`rnmWF@+bl5bE&-JH#kTVnF)WKzS{3)ZEPSo zv5zyK_~T4^Obvj)-1`Ga`*R_(dl<0WkVP|uCV{B}zz<$H42*%2P2qSw;In9!oD(39 zOUA7av+sK1Ocg6EA5mT9Q&$bd7kZ8BqRPWG2qMi|gfh6FDa<3K3U(!<<@6_h%RU=m zjr_vY;DB2!?8Z+dQXg>&OeBw7Mhs%nlPLkp{#|A7jUWWUG~=c?6bjLo$An0sHmD<3 z)*C3ylM5K{Mluk%$8ZEX2#BN-Q5{r~HDu+SG#{lYgUc-|#Tgi|q~7lAl%xEof|F|1 zCpyK4SDy(9g+yIQfa0lKCgxHmPZvRkqK?7-yc>ytvk^Zkljl$OAMRP7}##igD50R3%J$4Y_dH2LW?DW!6#oc)*_ua4L5xPsSAd zX2+~@s=A7^l$lGZO7rf==${FY;u@6B`YQBoyQ<&2@KL^lvsOBghAt%)T{)fzchY9d zr+mg!38#~3H}WW??C|B$wjQ5ShO1IkgM-8_JxFTEIkAzZ@QAOimyAK&3GXl zyDoEcR^lWNwn1g=rwlG4>(%1C%2mn$a*x?>rOuBIDzbJmjispqX0<9tASI*|pV2Xv zcCoZ+G7Y1E$?75S*gpoxS^)6ay`k99&p{y=GN?Xf5sn7I%7sCljFS@Gl0hA6uGc5IEUOLnI>Y7G9V`y@dRb*XTw zBDoLys^c3zcXFeNUW4qkg(?Ddxh8fksy z;i>8%%tlOD68od=O0{%1Mcz=d7PC98S%D?jP%6hL&G61CoTi-6iJsaxHARadq+Wq$ zhm_@T2BayX76y4S*=b-&ZT=tw!k)I5gpr^S@+0rS4xOekKK1X?%H}K1%i?zW&)|4@ z+@Thg1ye|XNe;n)w3x_CQ>ma|^2IeUKUmBqqiMs2RiX^1myj*Syw@X5ha2Pp|9hYe zqk5u~PIH^0cd}6-aY$Go8kk3s^oZ+$%oZ@N8LS1tjMl>LW`m{Ap0eCUBGEu08<^jp zj1ZZvl|66w-``v=?m9qFM$Gx2zp<#53kj>2(-OEu0se?fRoW@#UYtDeITwMlJ6(+BZTzy{ZPh-7lx9$IpheP6W>ome;@ezzDUtHG zykGwhfg<-BNGK07OcSXqk|&C9xt0Wh>zEXT@8+b!y7!t~m3_o%`pW|1ue71qNNC6+ zftd@xgx-Vuv#(-qVv&}$s()cofm5norJo|Cgn?_?k1Vs?&5sV*7(mMrJFPk-sK_6T z-vB5!CFcp|1e}SbC=08qTO%GW^J{hhR8j#*aC`eCxsy}Q`Ut`FK1m%S!HcT)6{Zm> zt6`-vQ|nLxt)Ej_=Uk0ZZIhCD;xJ}bjX!YMV^QzaaP;M1^D;u1-4PslfJ zlHW7os(DosBL@ReaNcq{W^mN6+2v4g#)H2l>eK8P6iINZgI{5S7!hXWT^Nt}pSLnM zv4lZy2N|A%j$WuL7Cs5L+@+l6t^on`^?HvD2eFD*K8he5UnB}SLqHgc&x%o-*WdAj zqyg}FUj!Z`xVOO?Obwc`-i@!@WNw>7B*W( zr7;(zIHfZLpy4W}*frG|!zTu&MfcFob|i<#lr$7Tdz}d4{Ob;JX5zQSLqpg-E%d_& zb%mS2k)%e1wbu*9KO@rHf54!r*SKrI7>>u3C%eDu!kbA?#<#NdRI1+nPC2?_{>rWx z!Lt2H{bnSu~=`;+7^L)T|C9DF%{E&g8GI&_>5vI!RJQ5CB`^fj?4GUW$knNYh58q z^#-8>%4nM2USZN)){Car(R=q>!+N06`-{n^V0e)8cx_%{=)H!9rnE87rr0e{1OT86 zw~$vx@9o(n{77F2u^^{pDEt(`+o3}D5aE-Rj)%=+iYU+`c%p$m7XtcOiU0`-Z@>kf z4Z1le0|`yZ^R38xol8*B+64j&qm2-e7hTK>`W2WtHHE=Thlej~p}{s5@c%;VkcJDs z;fq3#11XJpX*s)2HnG19zEd2i8q2=ePsPf@Cz51?L$~$#E17n9hNCwXsvFFl$u+sm z3GLBpAt|C9e%hpCG9r5_qB9Lm;?x)vQ|gkgxZEbN@GQ1yVFI8FenccHC zGf6G{XlJeel_b%f(-$B>ZjMeZ^fx?pv8cq|LuYk&Ua&Nh$wqfW3V6t=jC+DluKX+B%r%HqxKYOs!&J9ak(NNApio&H;cA_u$>Rb}lD8C+)p2N|?rl24J=nytQ0m zm0m27NYdg9wRx3uIlWoG$|jJ4wy*+Tcv;h!w$SwB?7%jtj3Zg{*6XNmaS;xtRwhlBPgM!@q4|tgqcmKt|+h<-XA^qnI8@CNxlwQ z2ZQ3l&XXGOUgs(7^F3+-t}+H%vJTu6jS|Trf)r8M^(vTaY*Kn*TH_Lx$$<>EU{GQ} zRzv*L5}}VwP&>f*8s&4sA+{wfMKqL9k|kN*I}H<0CwL@v+l!LKvOnBdzOyP}F_Wp`Oi*lI!FH!!R=#3ng^*AN^A%5?L z{C|S&Fd29sqw)UVwURe`wxs`2q&3N&_-~hm`zH^EOR2s8<@SHY_%A^B7BKcLV9~J< z0r)M90;+!6|G4=tpS2EeJ7e|cjpZ%Ce{IwMFQ7GwAaBVFGaU5Q>mMopsBACnA{x;i z#PcDHv?*_Vu%RR|^MVcH=<85ldBHY?*UAf^Yvl#nuKDT}CSKGAfi{J2c{XQ?H0m!X zve0w7pv@8s~_DACA>ocgaYct>;+E!lS%HsbHmp4&=5nGsfp{i`M=38Du*qTQO zrZh`!f3tn-tYOv4`$@n>bn*60y9n5thqnI4#VD1A%Ssd_-C%vD=&NSydyC1`3yL)Z zTTxcceqZeQ(B`}(TbRr7SV--&95nt|R)S3zkJV08VUA`~IFx;JxQL%Q*sAsSe+E5q z@-@=M4DDcCVcU%jl)tADO)eS799Y95iM{X?I-Ytt_;YTqt8}KIW0b z_T&QX&C`%18iqBzv=V-??Qxv2ory-A(8RHyiK;-B0a#lQA(7x?DIu4<0!C7o?C_bR7DG=c%T!)8J=vGZ-CIDaBK_@%pW3`>)Mz^jkc zUh*=AEiGq!nQN|8Uk2D1DgLdWbL+Pyom zVYOu*IWKRokGrNB13H7E1KZXQ6>Lsn#8Skq{N-o8o0xKSFn^3KyA*TPVd+sK;}F2X zAZ6=tkB+HV&dW5QgG!}JiYD{9CDUra^TDi%AvEPuG>@onHrnsNtjWAo$*c(i?zWrk zgJ;~bi`hmTsx9eVb=i}77mBjujNKaY>BHLoXqGN}bIR!k=2P;Ai^gS^ju%x_tE6B2 z2l76 zM`pazhX={xO0U`pKLV!hqIIZV0t z3SPl>;yC{>B% ztX^}`j#4#J1M{wX0}YH7HLDU~-4J2-M3|iQu9Tk8Cy}vkI@v#Q4=J zfWpnSh_Ag=n$FMq8vN?qpi$1`gjJ-leLu?InGiJ*vr%=K8lw}xJeFZsk8X9e;oM(y zi`QJb7-147GjgD_u0ZCAik}A2^!&LGN?-!!>y#f}e{()NED2&B4r@G%YYQ9V2l#8G z;fs()vl2eS+r%SOYD;s`r*OdKvIz<6XU~ye^qMZ#hz*#)0V>v4rVX(s8AK zxm&*T>;SG$2h_z1Alxt+K&to& zNqbiVT7vr$&R@%+>OZc$!XmNZbLwrXNngP68{=isr@AVXFn?UNyWWa#9V4z00PE3= zc-ne}7lz%4yErN}EaT7o8+Ln8isLRypYZwUF`Qq*>y2lAfP;*rhxD7(W--K81Y7N$ z)6^|h-69}leTQc5fv9{KmrUa~Z37nBcCbPz7Qp{e!A^z`CcOFJSym{!L34!>f(7Vn zvIyx%A(K5S8;CmT8JS&Hp42|*V3JWWa-k1hXh$@L)7b)3ObTsgoVFSAS3xXJpw$eNkXLMPa3`58Anf7CO&9JIhQUb6e(faKhPx2d%4r9nJYJmZieK!TN%+Cb0V{rVz^X|dd zy8ayv;Y2c9=~n4sgkR^3h7Kit64743R>!ESrkEG?W7cyLX5>5MqX_kNZqM?$QXf-U zE`z{j&bvHnK8I&*J^3nILu?68s_68x!S8st7ax2NAW2gn{BMX=H22Svcmi9sI*%um z30L1em5ZG3al>wAs~ak-g73R1bMdT2uznLavXw!?e~B}m0y(FTdQ;TheI9KGGmC|i z%+Jilbfz5w6a-5w)@=F0ru_KZk>rap&25(X8j|8PgcqKlhoZXaV^OrgAk|x zYB;_yH)WGj;8{l}@YJ8;^%|tNDx#s8l0q|oPvfHWJOs>S`ba;7$GWiC=Ajy$)y9_j zV`NG)B(dO56i0U}8Y~YR6l=0E?DuP0y9Fl6VtewHHNG!hiyr^0^RgA^@=5c%1XQq3 zF~8TGB)8DQMTXYUWXvAqgg5?5&)4~#yf{|=Pl5F9fgTPg%OoD7bRfu=A|ue=b0z$v zw3lw)j(arG0`rt2=!-os2@u(rjfu}KZzpW{QP?gqC?^XwvqXasN=rnqlttm}l@tr< zOU#3}>hV0Q{_L@Glnd0Oo$U?vn^>*l(Ky|gtlmoh)qvts6N9Cl z)iu9h>(Cfn3sfaXR$&06F;b@@{Sau%;{Mj_w=~41%RZ?28QngMi*YntR>GG`bXc=< z%$Lj#>PSb)9XDHOWf;W>J&dl1Z zoePvB!$$9lKq{xvG!Qsr<85Md2b#06>Zo}p4{Nq7>mCY5tf^q6;;$YPrPt;19J37S z-b~U@15^K!A&c5LiOk7}RT9q|boLn_Y%?}CkFjVTACC^PKL&^<%t7IEDQitt2=Vb9rCGIBB4JeHWDrDTkWa99XXVmKct>Mcj%qsBtu6 zLEvYJ-6xq}5v+usWx?3TvwF!xA&~v3Vc3rt>+HQgKZd_Gs_9+z*e5B8Xx$=D<10W> zu|55a*9EzP&SrDQ7JYnik3xTIJzqV$OK7Ds+IT+Nddb~!Hi2}*55x1Fi+fn z5@GLjdM|T#UA%%j*tC7_Q)W|Nv8$iHL@D1ACv=8ykVjDo1(yHv>=`ln_K%Pj`xn*g zrlKhc+u;&!2U4n_V>QoyDxBw47CpfEZ1JZny{el!T_Q*8AWs0zS9-TS{vtpnYlv&E z6;jrE;Ek0}))jXgeCMzr_ilr9@W5c1l+QSl_9-mpKAwYL!TGWdIsBe_e|pLjBidL!~b9oWV7OA{$PR z`+`j`#v(yAv>%MWvmYLDQSBEW!b+;%fcwF*yBsM$XuMw*!xpx<8MG#Kt1^WTKhpw? zsN&M&ZNVjG>5g(f$)4;euYl<3d8#c5k$9e5`53YlLn_*~h?m7j$cZ1BFt?z2}o?#_H2E2?^S=MnJ5ibnXU1i1Gor)Wc1|g|^Mu$N_|^v(n&_ zlK0ROpkjMRx;h*g1`H`lAk{8-ex2R@gYz~Rsb)Vy52oNc^XhP^a&-cSG*D6*yH?4a zcv!YztsqOR_bh$hpy8i`G%1$;CkwXNr0Fjj&rvU-()fW^9P5DS3tMz(4?#><<}%Ce zm#O0OF2*MhsdJH&20Q=A@Aj~}H6?B=>y@{&gonJ=UT!!jlFO-}g_yU>VW?e~Dy9u5 z>vsX`enAIq_#YvO)oBf5LF(JUzgE4}F99uj=4C!2Jf3ZdZa!YiJkp#ky-Y+!ib`xY zWN}7TB0Xi<5zBqu8%3-Bw-b5$f7&AC+08k{m%8Df3FMs^2;D2U^k`S;zF-*P!M)Tko%((4G_KX1G;1)a*8QnL{?*A= z0gekF!}w0UvN5(P<(FDQf-G5Epf zW%85-VZJ1GrGb(|Z@is-(%UAw_B~laW?;UhQGed^0}^I(>mwI6_9dd>!)Q_NYCyrwm*N--Dl`$}0J z%B*y$NVyR}hc7(Yw z(rx=W9&(yPog^)TmhTC-_>@DqpY)b+Dg%1;;!tKgJL|xZtbQ}>WcPscwv>G;3Hy&s zl+vR$#l(J5_-@B%Qm~Y7wdzsC7(5D_Mw`%|^k&MDa6>Ehj2|eK)=GSkZoAMN4T0~q zg}#10aPFydDlkUph`@w^?~+8Oi;th=Hu_o!dt26qCdLkt&H4)4!dx|{urJj~8Nc8n zjY$6TTwv9gwTPhX8qO#?`Ra`l9z!M^6Gx#8#&jrBX%8QnT@ms-1UR@w(k%^m1BFcM zTPS^}VYuIV)*ie=)p|lBBCO?CtY<}Nht~bjuwza@wiv(tjcC02CG6PhG1-B9fXj%Z zR7&{a`k^2QfUaryz54^o_tzmD0r9z0z}M5dQrjD*clG?SqYH+{wE#A2N3j+}E6N1S zfg8<_*z}XOh9OsJhMi53Y9@d{tM9Z{3%;MH}4rxGQnr{gw-24av{2 zK$A4_g_12177jNvdo~YJZ*#+R-+?V>O)~et>YLiOb!*+9iq$lQCy-iIluHjLq_y9Q z$rDh0)rcuDn!h6S3tOouKu+C52?ccV#|jPsQ6Q2Ugu7WR#muP6>GL@F3cCm_YnNqrm(0B!O=sd)?8I3FG^vm@j8NoX3qR@9?UhWV0N6fA3Qat#x1Dj-ym=QBS zEt1r$6ij{7#)g|nQf*1hk>Sx`wn%*3S8TXkAX-?F;0VD;N2DMimIpVDq9;{e8tk|W zLtnv}JhnQCHM!-&>@f|gII$=r6X>c4igWS3$;>a5DMxNXxT}UAjPJoXOfKb8y)R}J z@HD(fK;Y!2qo5$j$L3*cy{Zqoe2}(BpD?k3|J<~O+cBz^-w5=#W}>y+n|~3zkEJ=4 zShQ?hD>Xi$CqIeuIG>WD3=;rWquPjsJp)n~QV+%m#$j5T&@A{qp9oCaJku=r1}Q^W z0~Xs`dtBAm-6~k}HuMl)L&3fVZ56l9S;tb%`#!YI_G*)R4dn*ZkK!y}0hs7XK*puQ z9+XGd5f?CosJ12*O6Ekau%px4n2~d$Iv^_mw(DfXWdVa$H=`>Lt~g*W?vOOKSr{&B zCYyUpTojdK*2Z}UCHdkWodpYAdcAA8P(9Z7N0jb;dK`l;M`#9ZJ<~urs z()wKNxv|27+A#{hAQTb1Za7t`(N%iIaM(gdvX%lTR+4_)i_xub;y4%MfYe(_{>q4p z?`?PTz(Y#svZJ_=F9|v^?tszi`V%S`xX$ZJ;MVM8l-{ms>Z^sMa$+l(NRM3mC3Tu@ zv16#S5$pb$(s(O1G1#Ul2lX??*@LU(EH`!;3cbZ?v7Y8b35MU=WO=DfkROMyre+tm zCwW@X08Z@CaH?Cw6Tar|yHlt}mt+BQ(?9PJQWYXb#BwwC^rz|laYRItrE&0NL)=WV zdLgzRSUhhC3oFktoc>|*HlK2(UhR80xkNcxO99se~J276{4ZZX(E2~gY-?BR`B z2Uk5`EqMU=Hk1r*8bd7<1BSzLL44?$06A7*!ID%OqFv}ff+y`e78>djT*cg{kWtnd z4p<%k>q1x9C7SJ5oh;|j&creVPeVo%?^>5endEHg2oSq+XN8UtqX@m5Wv7U88#%n} zTvFHdJ^QgB9Cof!ePZNPE)(zI%0m-p<+ykG5B7|*R8IomQPP}!GP$2bl=KVHBBX8J z#W?)=Y7&GS_o+B2FDN%Yu44DjDc+LF03kxoolP-rc!9k(oTMsd+o(8UJ{K~wF~bOS zupGlUbJ|KyT$F@QwTwDX9T@?d5T*Y!I2g_fF^1uzWKCYJyN$N$5g_aQlO_d}~ByBW4{ZE|ols;Y6&)pSY}e4dgV=~cf0WY6{q zTm>3R3b20{8E+GX4Npd(`lX?W7-7u_A*;xXUkML7ol2VnINnl3>^FL&m~w|;m7=ZF zmk)@DLZWstZW#Fp0-u=6x#O{?GfH!#(7t76p50x*Uj@2815%jEppg~0KfvXQQuwUm3Q zsHh$y50`18Mrh4V?@xH6S{OF-Z60LQpRQ5K(ls25#7B^U&E!lkKq)8H*!leF-L|b# zrPe5xl7K(*!Ad6uFCd5CxYz_h5%TI!_vx8`e7CTaL}y;^J~kGOmh zOC;}B6NYsw&9-=XZC$x|x~_SPbdv9ZHos!6WXF{p_N!5NcvkJu>mcZ|MTROMv={~< zUhnc7u9Gs&U`B}wHrV-^sAT0C@TWzlc4BBc;$tnxmyc>}>_ko0ZS&AHI9CxQ@ijHJ ze(H4tCBmI*NTzMPFRH+MSj{mDBbH>=%+oOU88(B(-OgV*&ID*lC=z;;rC+$^Egq#E zDbUzypU@fqT9OBI()-nce)}nckmkEnsU%AIk7A3${=lq|;*3T$yFW}XPyC^{va(#V z1h3~*{2VCv%}`da-aUjVmc?y2^)kHI?)bF*59FqG;@4<$^NcmA@` z5UQ+!nUnb|elTw?X}mB?^fgppEqXD_9mv+k-WEK^|-JL%pEV5;A|C+RJ!K^NJmV|gD7$;ZJQSMH9@Pm;ed z39V^Z1FF#fLZ`!>G`Y#~UW!J>aBVt5?rEy&+)eB_Z$asPkufOZ${U(6z$T21016Q- zpU7%G=UEB0AC+hcMM2EA^ZdcXqRvLwoPZb#rP_b^ za9!P@+)Awi4`N5yJhFpVM({_4qhu@xbtGMELXu}ngE%KoYTu(!)=J-VQERT*_?84k zJmva{l5eg*cM|c?7sH|Z3CMeo5%6aBggF8&?;D;8fx zW)LHe-cYwDHb&rrUU>91K7G`{;__JB!$CQVX{@%=jSia%C@$#6Q0NuC$f!j&6yA_> zXfX8?vGv*mN28RLM)a3RtWS+lbeTP&pIjU(};E4)T%E+dK4l_rEDx*c+ z+i`LENOsV)sTJ{nC5T0+KL}!YEJ7(GLnJ`lbeA73ZqS>FPkeCecKK2XqG9t2f$^=Z zUz(MMawg%{@Z9tEE+o_JwKN_7)_Ce63)D|Jh`du$5f?T zM=Nn>G5DTPggBTAUy=yMPpf(&*Z1k~%*%{z(LCwW3^`IyN5KVc?~;e*AFp<2kcHSlk@#fouxZ4P$64vUzv8wOx!O; zv%?`a&{%X>r`Q2K?4YnvH51X8p&koBqc{p1FAW?6m!`lSTIH6)UtdL4p{gvJYV$l( zU<}z9LjqYL21nz>058vN=Hgvk+F9ZGzev<#P}N{-8~{0ZXAWai^l~~qLT=N}8L=S_ z5wT%u?9wGF0d{Ab0y8%4`{0#-_`{;CsFd#&<2JT%Iw(yp$S(`M2S!7ccEvGxr|SXI z&=4>z7H~l&%II{i0Ye?H-c}0OPeCW|ygSe-#5KHpY=eQYj3i zXv`g^1YGc&@{M8kE(&ux=oaFEAa;pX0$?#Mj&WW*A!D=GHf(OGqZG!Aj(X7Z1W6DsJ_N*A3C3) zB`ALxwWdumsE^N4AZ1aY!WRlFU?9yRT_Ec7M|!g(Xe1kaf6yVhjPYG$M?3)|jzGFXDp1Vq!=O2916tUPz}Hw}uDt$jxPY zd`!9E-ZbDJg)ORjJ_-D6YGM+!hlYVllV*w8S`9*v;}aUg3N}NN#&QXu%>@2DY9e|% z&S0;j>&WLSDIblZ?i6BB7==Mlut*xPn1MI&9sG$IsC9HBNQ_}pVf2FOMQGwv+7`!s z1`Sm#rssu~hNwX}#M#Vtz(Bc0ikp&===_qo2%zaY+r*=w8^JUg>!3O0vbkz_jK^V- z&%iuUZ93|MC6w|Pf1|^zbN#QP>7^Kqp0x9f$GXsroDH z%AAR;=Q2h>ixIRPwy-HYsWFQh%|qQ`1fv)3!6?MI3dT%S3iWF?Jy3vw57hM35evnV zQ@}V0!x9$Rdgz*>R#Q!8L!#4*R8UQ(xbdKXLdwClSk%{LGEt@N(gf}O;q3^Ur3VsG zYz(jJ#7KC)0JmB#eiV$};snDGhCty6orhRdt|H4Q0@xj84KsYGv<%2*P10Z#vdCNt zv|wbxp<@K-C@DgVa910c)K$7E$;tXetNAz?|NQb-qls46-Ej-%0VvbDU zmSJ3!U`;7M^48_2hIS6WA z0U8oxFKD>@`qSgef9mwOi9hf58}-9hwcdg8CX=j~&Xmg^M)sTd#bI9#YhC#F;Nbut<-s^YwBVF{_K{Y zW9o?3z~~2cp!0JGO7^+bDmSQ$QZ1286g;TmwFWoJRsR4s@+(#F8+8$`w%2HiDxAOK z`ts`4w?xq^y)DcnQjAA{sX`p`lR2#)9!I8S2Ed=l5%^zB@S})Kp7g&#WBnn}y3Za2 z80;m`2nh`5(h&Gfi=PfQ;4%UW?i9_^rT_B({34+KToI^B0FxFv5;WOgf#WE38Q_4n zziPhymL?5CN#NBeX%yP~qxhMtp~8k58ZfL}_*Y5bE&z}Nql#d;>g+Et;1gCNy>*vq zf$ph!)N;4_0Q&PTTZEeC9io^7LglVds{f~E5+|%+8cf8=(hUl=#6se}DcJN|dz1b!xE`iXy>wJ-nsPF&N}_+m3$%vBKk#)%4YR3c~`GWg#= z6%{V~0)#GhzSq3=$ihy&@n@ZWfQ>nwt~X#+EbPP3!OmJpgZ=r3C%!IZxz$>z)&Of* zA6U3rPp}HCbcUIcyvjWI0}E%>b`?xK?69|oVy9Go{Xh?yQoCA~2GFQiy4B7wO_xRz z3x}mvC#IKTr`34%h98ASA8H3J-Vl0}ZFN9kqjT()8(ie|=HIoP7TEMFp=fVy%k`0QVjfONo)hv;+tZ<)s9TD zR69S5797`o@UEtuk7Is%tKP1nQ-|W_|1V`jWP+cB3*SvWZF~7Yy!t_IxREbd@{cxMge>_qTz<7$vpy}CPbQI=M0iq} z=Xic`XAZhq2$NHa^?%%U?b=A`TRi^H*jwufMsPP^2 zO!2&0={y%3-By{1u#j%GN_d(8Z%k{dYQuj_RW)6<@z2J@|Dt>Bs)3oh@X(rJ`(Ku@ z=Paakm6LudoXmEPJo5`JnS-ljHOTT>0pL5Nu#(ZI%A zL4+C*@SLw^SSsFy`}NWh#ISBEuHw2bo~r+p zj-S%;Q#uYM9gR=v_%lsMSlTAY0uYaiVmPtVQMFtfj7QA>qu{~Z;EMu9hOH`&jyvDZ zua}#Ki}%b;OR*GHm&{noLuZ<*tGNc|2mh3(pYrrmo(>^T%}-DK(-Z%tvM~{RBIi@K zeag09F55mG=}$-cQ@DK!xBp7v_UQ+I`oaG+KloD|eu~52A`T5j_G63Hw`R>3hqU$o zlOi;)A3j6?v#HiQ?+o8`mBGX3Ab$~f3r~6s#GavnJqMfQiQ3OZNsnuKdxn9 zwh<^Xb=5I7*TT#xCSn-#x#7bUwq;{Z+Fv&M_h@f|xoYT^t*Scw7b%U;;hDe&if-$e zjR#9KaLjGpu@wd122d!`w}Q#{T+PO`dcSP=`a8q7bU>qJ!gy2zpZifTYo?B`E~t)$ zxim2CgK0Uo?U<_JU?MGiSkZFvQ7fPiC<*OXjovl>U8k1+@pi~(0O?~!MH~Zv8Wo>L z#ZNFQK*D~?w$A|4r*QieZvTzK&HfZ_p8=$QRRHNz9Da(!UnUNnPjUF^$NqZ{iG9k( zPucj(WTX2j8$bQvPvQ6}96yC)k8m`VPvQ9KNB_h8=udI_DNg@w;xyy^;Q{3ukDX6} zIv`=xkFwh@U5^8&@E=KC`X68RKP{h#i3vsUA$rUJrqYWbiUu;`Y@ zFZ~Kjf>!AM-~WA2?3LQDKPzRL$bKCpNE?W1Rp`>`W57nY{X6n8vB~iyZKhga3uC(R*zWw-r`o|o&e#b)wu-Cb=VsjbUsJHTR->6H zK9?$uKR-3>zfHsblh*fduy~o&4RP?#5z^sBF`Uw}#H&UGg{1^d3EM6jRi9G!-z#PRM_m+g z2BS~nPlKEld_xaob?hM}e2>@tMJ~!ot5k1;NPYWR0$=KIIC1ae8X8s_Z`J>!3E)dg zT-3G8rJ8qkcj&dsfT0=Igneo?I;DB^S9N zD1}lPX2*R4>Zw_+A2zBHz;BH|yv@7APD~%0QXl@@=*w=k?vIC0*?X*ka=VC|@n-Cd z5%8G+9PQ^aRM-)e;7|AT^Zq^=N(p}EB#;EjI=%<+Z&%ayaaEsiwnMJPBYrQGe zb@Wfc@+nyUD+J5ugIK>ouoxkWMYlDb1dDDOKVPu?+E9_H{Zjtag~z&yKMctK|FtXKmA88D#pmzMwrUO_dbg znN?Yc{?pL@d9wQ$mwO`HXw|(|JE0|-zx1|eteF%V!p}05BAtG4>$^2@>-%4?heT$# zo*U(Zc<+DrM)_xi@>fWnYbY&sYn_{+Q|4pGa@HclmlH-Rcm$HkO!Ja<|{8 zAGV-&wf>Y$vf_*f^5qab2e9^O6o(JgGyZ4#`WU{FK!OS1AZaezBhk{4!?HA)>GS7m z$CC>2UJ;-c#0W1GF{SD3-sMi>*sJ)J6fR%`r8pVY{w4RG7){uf31lKoy1AJou? zOs(tr;N=f{r6+KfO6(h|VcgepDe{F?R-j~W1)Z$R6<0~@z@K8YRLe1WUn@WytZ}&l zMLR1&Eqt)R4b^g}vv4ibpCDB+u?j+!>);iL2tX(b157DoNZ{C5EgZf-cu`4{5W1o% zyh;dO%9QJv@pbG_CEJXxJ*=sj>>sHf zV>bS73vG+x~wf)g6AfX>#1j0ejUOInvJ+nioO zMLSq!rnIt&+a3D<0mf{91U9S6?#h!qpHDCeo)9uUexo^DIq2h{RjNNhITz~h^%AKb z^iu%qwf>YSzJ7nV52-3|FLV^!S1)#(U|(Po7&h9(S@o%oO7PvD`?o|u12>f-F1lZr z8h*&TmuhpWTAHRhmSP#UrZ7R4(iGb@TvxYr$93tL0He0WC8ee!8z)pQj;1$SrSIgY z96rg)o2_UeNPwGUm7jvHUDxQX=Z34%QQS=gK1Tg?tvGqGCC;H*fs-Wq1O zhTdoNM!1^RXY_iwn%ZadTDY3hXY^{Mnh_7w6!glps!z|IfMS7VwGxQxs2O2LYz^Ct zsU&om+*!nGkraEpT9>q8pq02LjA|u|OF0GpPgjFKsErlVKmBBt=D$q z>-3-1&U3;|7HieA_jLvlQGIx5fK5HU;k6#TR%wRAhY`!r`_F9nd3-45FyfvO^L-mmZCC-g7X zDnD0im6lg$bwq9+@fH8+d~bT}>-1#3efLMH)m{SkvIxC{mJ3z=1C#slx>(-sQxDIT zv1)uI?a;4(DR%0Ox;F%3YK`(Mc%DOGM%0}Dz4Lmg*3qC*hnO2MN73u-(Sq2^DyE{9?Kcd7bsTIh>^487E`>Tl*~oL;%x?lj(huO$0*e7M*yVbl+G z9xSn8+3}aJ3mD5kXej>;tF-<2=Ra5*|K?@TMq~0{PG!x|9ekVCRM8XE{j)p$=KLq7 zSssScyyEW_#j2vJDMi&aS9J`_GTxirvxk1?Fw+w?fA9Q~ZfJhG``!e<4>Py#zkOP& zW~DR@xYF_;cjzyBhLStKdrJPXiGI^~Xmr3}*!3Pe2dygjmBY-cx6MDmI3TIW64Pfe z*znt}1JGf=dkAk{r&IxB>FmW~pknFVi%Mh?;(8(#B7Pk)ele>+lMlwg~KNn8x z)iSIYv5KVD_!mYU{4SP!{u_D?SUn6I#ZDBTb`QTsNn}K8m{FpcQC2ac__CsXSkWd{ zv{SpM(C_I8J)frEij|}VMOU-L~B^uhu6X`Hwc@Eq&r|7tc`9dw?lRRC6Kov#D39RjYWb!ACraW_79gbu;?l_7KJpzB? z;Q{zd2zQVYk2SntQukVkX3J}Ptv|d(;}5UZ#N8uqhdYKo9Ib*WM~Q{GriDEw!)%q> z9j}%AgKdyKl%Q=n>Vrg@lxx*yqTIlU-d~B9_XG;5Ey$TP#XXz>gY=*)_(3b{Lo33c z-M|%`Z8e(dmRDkDn|nN&s1mg;z>SRw0wd#=m@o_=S8X?IrSE$@u{Q`h5PJ`eQmc|s zd!<{jqu@tN4%5f1B&h*r)ey=6J131}29}|((XQfn=_}U`nyzXoP0<|_-`O)ZPkwqj1vO)aP?^SYWi6btdh`>T`4SE`_9h}(E&OLug?TcMP)rTQPG zc9Gw}BlI$AXKJ2T0SRyxcvgOtVx!i8HFxYayET!$KH8Ei4!>>qE0a9YE>>^bVaPeO0$}ZRC}Y0dV(Fd66tQIkvJy#CKE-_*-bP{cz~&0ee2ff2yagn zUnf`I(d#yLh zuZ?a8dTt1x`rTfo1sG;4rl}>yk{5`bD0Z8utl9}W=ax7S`ig^fTt^*Xgk}6*bq%EN zAj9ieEKn_YP$61zvjOyN)HkINt7`VEjCwKK==wwL+be-TXqL*svhSl00qM1B-AlCD zbAf$aw@@ez>T&khmmFvnCPS6xas9YzhMoLL3Qsm|} zy;bPeYGMWR2Q%PnFJD>G^h)(kH33Vof?M$G-Ck&avcmq4E6QFE>JPHtIX{s@xqQox! z84)W+_|PyNyI;S=FB>s(I9?#p3ujT?qiT0WP_9{&>U&^%a;TEzFR(TZ)O=6u zy^)H3gROmTcH;%E?Z(4a{jrekh{|AKrT%P4%e{ZK=Sa0!Y*f5<^{Fn|x!NEi zJ@Bo#cK}N!*MM0BTl=aLi%zIieFN@Hbs9|(oznW%l!4l2-)gKwnKq&OhyyquS=l3o z3xrwx>{^vnqxF>Hle4ur+_OrUu`ujo^j5@ClaDZAZ9vvTYvZ#?0n>7wKjkksA5}OTOLq{WHGOk3Qmml)T?-p z=&4A&K7{fJmf8@NeR(v5-y>CzT4jua+P^YFMywZDkFSq=OnQ@i=YNno|rd1G~i_~mI zYpSt`ZKp6wPzO}>9~|_?C3z4(@b@+%6{EWu13F`|n}LV}3u~z1VZ4OGB?E;pf(E@c zu%|||Jb>@1LR-ktp**-q^FX!6*@94^&BJ_uVw7)?r~(%Oh}W%`->DMP6tPUedg5N? zJD6@XE8PKV?kyF_u6VDAfHBp=(Ri9FrV{7gLn7Ot+hJd)i{}T2UuP13Xzv(}Z(!v# z0tNMsQX|Nvdc83~{!*80BclBF{k1=(pd#8=JpqwEoi*57F%geD%I(i3@J~fFGj4&Q zh0x^NN<({;?^ttSrv)^N!wFi36=ubAqmqLL(;ifHCGqe*k!yhfZ@ptwfDJo%ELbDN z8t>bYev>; zO5Z-?(a-Ai$G;p?Bw|L8@;-eSgaHJS=SMIGOs|J?VlgHmp;-FE+k-_@g{h^(Q6|5y zV!hNne}-j{YnA@&<>Q`!^K;|R&1$6=;o$E35yZplUtSGb;=|;_p!uU-m1SQ@Fw!`(X%+|gIyiw3U`R)tL=3kFOzRy$C&1~UUAexs`Jz3M_Nj-M3$AjIu_w>nw}*PE zT5J3KF)UvN2G(0mril1tF0O{QA4{6qY{>wDJk*bM?Q zPbKz!f7lqAXqfZbohpm#hLm96IZ6%W`3(tCC%!z?U8~xk>c&s*HY@*#E!zQ>s_Mt{o7S6*KaW2oCJk zAhp0ZCEuSn*azTC4XzmAGvmju2Q7%m39>;|*_^6kSA3vB10wz0d%y!HoIEGbu<#OK z+ax;AUV?kr30PB@sHdIy^VzE>+ND2`My0x(hB$!?oG7)inB^!20RHK_R05a*yM%wk zoqca~6HV}F5};E&l!(wXf@a-4VRGea2+({2fcZ`9FxNzN&p5oTdX6@FgKikKIn5>eY{h(#c@g2WAZ#xmJblbt}W zdV~8JB&TJm8y~74LTVg9moVWC*0#~&`N1X<5$~_hr|K6qSAIJ^R#)iyS{9A#cXwDwVX?(rk;90?N%-Uo}DX9EZSEao!rY! z#<)t(r{5a8?etc$^E|g|jdcq-WBfexQkyy-naDjV+Yhmwjoa>{eRg#HHGPszo%IJQ z*IT?%8bLD*5 zJetlv8+S{~=?Q&uF0(&>KewKpIxS?q?(FK>_So}MxmbF?$3L^awEe7)Ojt{2H|yzodFpL$Wp8Q0e*3yowe5xW zqn>V+x74}UMb+=IoZD`n0ov>-&py|c%3?LUPy+g@dn?n)c`sA558v1fE_Wx^vs=q! z=lOZpYy(}kuC1|EJKMOOTRy8?FKQ2CH`MKFeF7%5YR!%zNv^GP>!^-+TFt!9SV_*B zXO&$=y&2ofEtHlg3!`&aErep>hg<4}s*Fr14_~$n`g?A*u+0BjKF{y4>e>1HJ?l)n z)V6?iFO<2})%?7cTYFmJ{hY5XFI8UlZ;6urfT6#GjLD%urOdWJfSX_d>h#JnR`Lug zGe zK8;LdjLVZdhRiFga+SYR=gu<5Mf#wSdpW<$R$r`zrN_64r$Y8|Rs{sE-rM$Zr4hh$ zir?OL(g(YGn=7f>{X< zf)E$2QPEWqR?b1TGcu9Cs!Tj-_seTJrSNrKCl*+~UoPZ;J7(wSG7a}ON~Xt9$Hvh3U!W zO!nD2e$CJArV0V4!UDK#poA&rE6Abiu&CZ9VQOPD*{A1x+g>W49;fXq?L10@!_C@R zsP8Wu`Q-eA8%GlN)!WIAOr%ftDsv!VR^41?_wqGvv{X~7w{dAd!lKD*ind|w^>R+< zurPC2TPoztD{XFd<32eu5zGr_RemhwbEciKk_Trba8l~#**pQwE!;O}GcTRWTc%b& zYot$}FUFyZojJdQW`4f0uN5Yf_I3LB@p%fG-Arbzx!q?bEoX9%Q=1lyw(-oQVE*i4 zQ;2W?uyb>w7uKIvE>OVV=N^pb^YrONxgD-InLa+bt~oEY-u^4Ru?atWSM2%j+|Mm-wI4xZ zm&eSE+O1uui}%}E{?$$2uP<+AQ&8pp45YyBWJbN}-ldD(6C?Oal|ha{!P|PPyg13I zC9ltg%jLC$Y~4P7e417tZco$tO3plPxRpkAfoM(}%JgAstXZOsE*r{s18& zWZ+O(8$?#nH%)`^6h_Ai%_{gwo%xVAV#cr@JD1h>Ny3IqfB8K!py8j4EkN@3V~oCR zft7dtSp}>Ti!Ay8oZO476*gB|IRYIk!JM)3K^#&eE7eEf3d2}rnuL?Za2IRg(PVwd zjHe?&sdBLDfRh7ig{Zn3a4l6wOr&F$%920ALtl6>CZ!L2?NhfONA+NSA(Rsqsb|h( zaRQL0<}b~Uu^}UgzlNB!q%j^_jyiKde=+pqG-&Yn&sEs;n@yjb?bHkFQ{D+!`j48d zFx+$Yj*H8`PX&3t+OWFceCYe~s( zR5K4xId3+<8Q4~6DQ$P%05RMO`|2D>uA6K`fNP4rty?QJ9+`R8)4_!S&616Uz2mQG z92ZzhV6B^1f5$l)!Zkrl>2EqGL%1f~>hn1?iBn@PbIA9-eGNPmr9fq44JXl^w%NW!s!(a2AjpmmJ*PUB3_QT!o_Hj~{r4?eRiiPt6g?zmWt5TbA-yFtS z^WzKGj$Pi}&Kw=CgSNPQ+vfxAUQVQs_KRoeYl{!(FFMC@V^XnSyAOxTZZlQeXdig3 z2xPu0Bd~uqcxOcur0G5=srF-=5vdW{I-IDq~Xl6DuMmM9V^QZ06 z5Dl(x)9%tv(*JN`d`nf+^=>C~5cf}=oBZAF#5D~5X>|Qt=62fGSkFG+fz-9e zCjb-f7V4a*{KLXkHEhF>#J(oqo$Kq}C$N<4^Sx#6vLVZv3@R zmc-M!3N$-A9-r8wH+{?k%5Ki{}gK{A?Zx@Z<|Hr_h3Q&{gsNSGcj8f$|t*?VS& z%$jwvk=@Uptwnf)?_RVZp>zkAOQ~CfCm%Pr6dytJZcmmpZN_*BAF+ zUM1rk{yDL}6zW;N_2MtrZ)x;iGjE{%Lg=5WEj)X-_X5?PA){mSpqtygZMSpt4~6{l z%=pA9@Z#mS%-g}L*~>3SUqF4mUZJJ5q)vmJGbX@PJ6Xs*p0vQ(Qy$->6}Z*Dx!;&5 zjP9(r#%?>g$3`m^!>}%B&8HdM+qo!YAJ_C?`EdUQjRPA4vCPCOj9Xc`XVv2>z-pNLWtY(LC*^5zZ%zkIdNMgW)JhGp zJ)=uM#`lBoN(UD4NjhI^YPIHl#DNcZOj6qq z`3GfrO19v6j~4rou9ZC8l^Da#8YO+)*zM%z5584|rS#hMM=k3Yf-;#ef3a@BlNoIl z^3L&Q)Y$m8dAv)G^3vnmO4wvw^?0(+8CxtS%+P-oF;RwYZOm8R8vg+wud&ygze1kb3 z&&ZHhzrB$ai86Vu?1iOeH|;z$pCpw)0{7~uv=Mwfb!~4_?shx59Fv%(>B*PrhgpGKtHvHg_rx zC+Aky?JqexI{*W@$=bzAV;QXR=J;(g2yQM`D&>sU_8ujf<;`I?$z)38Hzd z%Kq&`w{U;cea6!kQ{)>Z>+>fD5ke-mg3snLYCJ9^&w=o#)9L!dQ`?tqp&oCwH`0O9 z8H-T2XJEYEb^(Ex6Cl=(y1`e*W9R9UQ~M;qo%_1}vI{eRI}e2{5zF`IprV>eJFP75 zH>H>;AuBVre3F~3E|iz&FLyF&XMPL&o20j zOT>c~8q(1hP!8VpQt7<$0^aLF{$-+&cN^9G?Zxvc2%O|)O!a%~^x*o1OPJll*S>uh zW;#oaeA${YGkY^LRlB}*G>4veG#Y&bdVjAl1?)V2l}}zePZ{G}PfFCU-p)UzPjc;9 z>)TFgZ3TSeX>hKV)i6B2m)R9qFAs~aE4iEM<(R~1em~^agU)&D*M)wReO|95b6d6S zc|_$a+R@js!t_qY0lFMs%{Qd^hvMLtFBz)nf13G+le`&7#t`HyG#oZ76tET%k}H>D zD(EBYN+ld~SvC9 z`Rj4_=^T7y7}+Mqf}DAwwISWCV4 z_Llr-RKrh-Vz2C}QS3>48=KR9OU|GIhQJjy~ zPw-XQyz{cH)|vrJD|5fUaGcGg8XMW0&2evOO?6V4nYYPe_GYV=%b%x5H=*J7n^Kv) z%=z#R*pO*{%FMWfM`v2u$BnO}>4V#oQ!rtT7cu+Y_Q*u5rFAAo=e5G|=zQ*YV|0G+ z+#8*_e@u#-tKvB zXKS=bUx_Zc*VIDW+;s0xHdgZI8Si*~q5ZVHcyqYt=B{#U&(Zl^-_Mm)uo&wciv)BK z)#zwuFWqMJ%@w5EF;ZxGbY}m`9i4x!KKD#g+0&48(}PJzhrk9BZ*}D%J>T5lUf*%s ztIp=~d8*L3GS2d?uUAU$s&IKySV^yxqf^T!XR}IW>N-nZf!nrupK9RuCTHT6OM}fS z(`fHYL`E{UT1Kv_EL^d;DKyOaL5J#X${Sa6JY98sYt$sl7TQ}c2 zr+t8CAFSh2Qi}jZ=McfRopUvAt_XC~a3|8Q@j#cqOtz=S8_hVNt?qqleEMPh_$$yQ zX0CI~+HS57P-|@L>}ubP2WnhiDTR!65CMwR$GLtfyg1`^ed#a`s8V<=Z}tT`sTr5~ zen2bA#zyNBpu{4$@aUHZT zcXo(CXL_k>+!&qIX)+#Yb91{dP-|aV8Y2lKgY|)yTepsS5f5}fKRfo#>h8q>ZI3lG zg$U3I3xh zMk(BE9>>n~OQF3q@w#wU41kVINMJYSyZe2CPI*u32k}6cM;E}HIUGlzvpwiND$I^d ztl8_Mh+w-Jo$Ichx#U1ElhYBPR&n?Bi~-GmeJzA$dbO_XUgymKC|xk{=eqf=ZNH8k zS+PL(W?R+1K-XXIvgtlRuV;Xhi;G@6cCKG`l3#Z!-h3w(Xz^}ssV~s|ubbPM04VTS z2y9k4o4Rn~f$olVi%$;~Cl=`0uG;7ebY$XmV_~D#Z>~GaSyNx{19WZX?DqUV9Sih) z?O`?m3UmpBy}L_y&ierUYA81cCo}Ot_Ya<4E^^DWaX^7CH=Wc`-?_dW&P`tQ16ok- zj=n9&13g%uXTHgl?<>0uXz5F5BfMzVTHW(&1A)#(fR0SG zorAHZncME$9e`bFMS~T$_O|4FEhJmJ9?WgTA1e!!1iwcO{ z)achSs{laP+fn)3{@QRE(6#COd<4Ct>B7Mj1A5b0+z#88#?0y5(&oxGf?enpg{NxL z-kp2_poMM}Xsfz5!+|ayrNT4aJk3l@E)&p^34QH!cQ-WGvFo<^a7~&<=@t6!~I%?bz1c(&)0X^>wb(=j+s}T52DT#eD?| z6u^HOoyd&m@Ov*eZa&m4yD+Ze*C^Y-*XO2HpR)8mU+XRV;AYdLuQAl$>~tJ;^~#QK zy)`+yesG>Y>hra;w6nE6){cDjZI~V9W_NDR?8dj=vF`TQSG!YvzV198F4||w*ssq1 zl{zvpGre|wgWu2?%SD=hcRDE7uw2~dAraLW8TGnq4VXyi*Nm?+&oXcY<-FSI&pqf zIh!8cUyH|BtF>}>q^}=MPviGg&zfG*GFt~n8FZAj9wF39-@Ua*Z>Dm=*VX3Ca?|$$ zo7}csbvMUfNCJUtc9=zn?HHueuH7%HAmRo)ODek7=cDsyx8Q5uRZd2yuV=dW*CG0( z+0o?Aor!j`@4v0CG+tpm#dU%gKwj8$>QmqfUUX5SPlI@TT=s#EEmT(?X23@Kl5abQ z3t`FE`l3y(945J!M4pZ$a979lDT*bK8bMFzH4wNmP-Et6f1qCqx5(I zN9)HD(8}cGSOAo)|2{y?XSDvC5uiwYKG#cmu3?~G3QtO3pk)2`0cwq*_5TpqW8t|P zm!$zfHr<%I?li_7aOb1>B>pqi3`yL1IR2qd$|exXUqpM8yEb7 z(Ol=$+007;tLUNF69~Q4)%-;84^W?lvbdohiQFCGK);D9e7eDqm2Gokg8o5f2)PQO z_$D1TYUk!wAM{PC)Pwnr(?j?ns^VoG5@y|vok?z##o#Xg^`J9w zG$h&Q@tXa69rq@9soDfpm&=}S>GpdsCyqmdY>WXI)SCoPrpRCwvR2Z-L#+8A?7zS90fNnn;(g1M1AtW<9pds9@Q_G zKL=Tn1>P@qF%+`Q59rEXWQCXbN~dRutJ~cnoV0)z;EVN%>upDu_=jT&XeW;cer5c4 zhIK0R@kI<1ydNQvG@H_twk%xZ4RV;rP)AH8?Wyt|DyigcV^Qju@l{{q?eV^tac` zPE-R*meTN2Y4K<@>?jY-=Q1bjYo`qlHz5W+)9zL9ri9|qr(gE?XdZ6~a8=xkva z_If|wid`vxUnXY(pX=QZ#o~RNU*c7wTfEZy0LN1=+>Mk@cl=mCJu5J-mywBg`}yIW zil7(bu@7p#K|>1f5B9Sz`g9fvi*FR`QF{~1`ihKTS9Hj@7k2hxb<@YMq|px8^oiZ$ z>x;@0qDwq7`)_j(l4!Fr677`TviS*?1g?#~5-l7d87$G-ukQ$<;UnOTr7SoI#V>ziN>fO4vqa#KZ7NjbGac3X==;)AQ=5eaWX&hoPN8@8ix7^&{HOZ-$WQ50lw4eS*AR@hh z|1zkbp)A37N29#A!#mIeuJ|*dT2fXQ03&{Rqe$0Vd7X-tNETC7myp!KBPt@NRGOAn;sYjasB zEyZ5T>h~5>?6oKQ{c9sx$nE!SO2dQJ`J~uELMsbff)dgYPT}1;X;|g^VkQd}St*o; zn`Y9Gcb%%ZVHk~G*}tG?d%_K=W-2QLDX>^ztB|zXBg<2FE+i$-6sokM?p&x>qA_!P zsm$0_(38#OrV*8!c&P~(^ZuQg>Dt(tPH5?+oe~qN1aWa(C~WV!^iutvLZ}Cx^lFj%NRQI2s+A>J&2(*kDzPuD2C2} zMRESnqVr0-F?0rgC7e4loiTAF(2c|nn%}bt;(#b7kD7a*X?>){UAnW{i36WP5)o4l z{a33`5=p0Id_pDmwq%x;0@_m^r}NZl;b%r?iNi6I$TFRvul6(RG7kk)&-h@?jCg}j zXsTP+l)?T*4i9MLob32$#E`gCJra{H(Iqlty2BjXlV0vp+y%V|g66mfScc5C0mmc| zcB}i&jnz-t@lDD)gIKzP>`fXOjSVcl5@zGAs8lFXd1!{QqLV`9UD_1{p>BFlNQc30 z76<<_mflG>rmD&^nd|)W6Jv7Vz_uYeOcrfK(3nAlmh7HJv08IZSY*#yjA)gJg4Lc} znkOH$Zkjte$i{gUj6zH+7DZb@rUwPt07lv`<~u6b+ZRmFvSkcsWvWz{F;ZZcJ#agRb{>qBE@yFk=7|fcS*uALpQY|Ke_b(JnRMx4WJc%%Z=H@p z+a<9NJQrgn70{GzcNkh|6vwBvU{krBHwX_`dK z@tG*ho3La!YYF8a%gD!v5ew>;-R$WMyN0=LDw{n+RV+Ea)!HGss}|T`5r`$dYcb|` z)q!#bF(Z~Mq(2!E?|4KW5mOShZ#AEr`}!kBlYOOdI&6tIs<#;G{8JKiE>=l(QL(%C zhdqkiujrA}dOgx&n<QvU&KJjM;)adljjW@29;9PsX~0CS@E+WGH_3A!JCd zygnv5Pn~wSb6Dt~{C?V|dB-v`j?WMx(J8|&SzBaUWFyd0_UFk+ilI}sEHXBN;w&sz znoemgG9!tZd!0ol)t^orXDS-HsqI1{N;NUtDKRt) z526|O78wMgwN&=QZo<&??i{)SwMcl)&A7J6&~s!&pxF)2F`5`_PkYJm95Oo@`z|G% zMAjZMX6?GrA`@5VI&T=w1RiwY46I)!Q*S+4??J@XNraFH+(Wd{wFs^g8$m6p zSLdilp4`?+RaXv^5up___q>!lE31x&bCR#memq>c%8YSi9>v2!id9F%*>|ZnkB5`? zP<0g#$9br(R#}Zai-)rxk?JHKP8codX*`?^w{#y5$Hj~|4b_l{hqIA~#6>)u;kvAO z9S>KPp`FDJxjWV=$BRY1i5DxFk6Fx~9l>9JaU$#$e8NzEcsNjvVh zPc#X%E+SQ!2HMJ`MF~W^zt)|KT8B>P$uJk{t|G%+$TF+BBpEslg)^37UWdY&tc!=C zaGE^uGJ&p+Lg9on=4mLL{Jv7axTHeiRF4R`PB{*RGxCUV77Ayk7;0myCwe@~2v0QP zii3o~$%y^9I?EFq#S_x{%myK6&I?cIY~yH5B9Wn7^jamwu<2Qn1Sg*8iAfybl+P;# z;sKrqVj&|d9N5l+#?@(@W^BgQDZ)Xu04*yurZDEA3XQm5wYungb;>}l@0I=cQ#LgCPQPc*zZ3^Sg?jB3Igz%y9LcD9< z=%F;iF4NZDw-VkJ1-qF>*xjVW4=uKBuWKwM+GwoK>BF8=QLtOsB-m}KH_1J;8~|3X z$Jf_UDfOxkc0-BF3qjmj*d)TPwQ$g*;gwr7mS zs!s57+qU4L{Zqlkv-tzjLsv}e@h!}5h#xvRxn9tsQoE9_fW1;9@gBQ|V*1q=+?kq) zOSBj-b>+r{_fgKz;vI9>50e$qNk+Yu*8AF&k8ljc>4fPCZ(00Fru`?A+8froH|5RU zn_7Z4;%J5>QZ?&~ei`nP@>a>jrMoo29TZ6TT-mH7XQcg>vp0BFNzTZ+#46=kRFadP zCvDO*OP?!~x@cNKZ{^o#TKCOK5jjCq8`musKH{?DGxyYOF&v?wQx>%b#9h)_u1Kfs zIa8W+iMJfPqL*u{f*Gt!+6V|QqUnr!XmFa~GMdgPmQGjs%Y$e-BZF6q=h1XVL&Lgw z8%<|=Ug=phooVEwXVG+St=4kUbVi#GLriuRO=lEKr}u((A5CWzOJ|rz(N#2^(Z0FG zaOdnMn$FZ(?252NXXr=p%l(;0P1 z$*1IHG@a4Ys_^iX4agv;y`5+} zU6(}S%`_ra7rYb7-{}kQ?%tnTs2H?|mb)aPJqGMB97Ela%Ki7#()_~ATWDH;s-)E} zJgrw}qi%wAy*k;LHF}_=(fOUulbK>+lu^f|l$$m{!Csd*onN95>+k%p!%Y z=E+I}&}BI@D|sh!rLff{%3)4q=vT)wwL2n%ljqBsibxQhS4x?|q|4lyc|(;8-MRLc zXbDObx|8%r6qmWPQ!UGRnLCbN#D(r`7-Njf+}ZU{c2?%DDocP-p*!1VH+`8qnMxBP zHwF1JcNJM_Vi$5!=1w**Txw)J(&iEwZe~4;vlz)OTLizhNGsiBXt|KtqX^fs5DHBz2MiDph$(Dl?;NBi6>I)CR9fh zEnRANKMn;Q!qs}N-Fv66dg%?d>ZLc-s++Q|mD$#jp3}(3u&p5WvYo%W9oI3nN-Gd~ zrrc7p9kCOYojQqO*M%mzI*DN~18cPjiq*Oa)83HL4-rJgi{2aqVFR z!;Nd#2?8+NDQ_Yl2f8E>&9cT&Ri0uey`f%e1FjB<%e}Gm=a+0p8(1MtXFu}M zNrgMZN_ifTkG8^{t4evE#3iA|ow@PSC_Y5MVN7e>SvMwA;m*3=t88FxqE$SBHu#u} zjx?p+wj0ctbZT=nCKtpcHDgkSGpUH7T&PEC3vU|4)H$?a$e7iq*o<){U9uI# zMAyYsHJs>)Rw;%PJ(e}UHHx9Q9^aNXV<9&RwOV^ZdBIk&_k(*9kmCN628UQe1IPS6h#aMKjhNv)I2&!+=9|dau{kgb!^AcFdi;I~IlE zotQ*9E-Y_~A3C|PQ>;7`d`vUJn&O4}hb~v45+P0X@${97=Q)GbH@Sr1If?sP zf{4UiQtnj6sTCvW8nqqmS$ zMgNp_{-usvFMK&kmkb=Mw3%CELz@JyRYFZysKe~!?YmYP3%{k#fV?N9`8;WrTvt(P zR<2d@QE65_nwI!fX{v&nmOi~!mzGwS+>eSBo(n~nS{1QE(*|P%0?cO&HS{aOL8-%`Y?H2(LyRU?B>iz zEyrOFlaF5Qk?k9U=^3A)Ik;e&64;(0fco~#i%SK_WFFGmRFupR8ppX}wVmHtbQ=vM z6Y6bpIU>0AJ@0?8A4q2GsZy1H%uQGnQQm|#>S^VR-6%i{ep)uF zuj=O}Hxysj&)j*@I_g=juAlkPyvPlOs5^6S3aQ*s5_SD#aznYV>o;s#)z4OZrAb{s zbDtH%m_#?e1BO-os;ZcI;WS)T)z4uhS5;jSwNx$=CHWUrBLPn!UPy zV%I5D^^>#nOOyvNt?DQ51#Mo}PZr1A)%DY~QWb#fupXaHkteLj=a>RDt*9n^ixVs9 zx_&a;(tTY&^Y9dAfwu}zv8wB5?o^AE zWSsUc?m=ADPr)(wb^V4-tNL*ns9e?cvmce_q^_SK<&>-X+2CWQb^Tmx<5G3~B;_&8CF&GqtQ<`O7ZC%(|T%%nv%+>BT?7Sd_%2dZ*X7N&nha- zo4S4x)2e=&%Ed0~`k8M+t7EI3)%DZ!18r45SM~U$`KCoZKI!;b^QwOCeXf+ce&)gO z6;Bm+b^YXWr|ZekZe2ffyOur+jnle*MpIewsxg#e4(j@u53aezM+!?&|s(D27~aMeZrC>iY2uTV2&plWpEbUB43#W4{;Re2f)=RO$$Jj^C~rPu4{uz4~sw{q}D3DzgxHy)YXQkMyzB)85q)++1q! zAGjEA2?m!|i!+_2)#;&hwqrgMDF=s=WqLCi?VIa(PtYP#xNj{?M+0|fyD%OlcvK?Hw8^WZPGcTY?k2Gi_6JtjY1M$fSbPD04#||2l*dmQW<$ z9egZ=rlzM8lY6=Fm0S zBd12zTYKnn4dd-?opdu4kM*X*^evgO(QEQH7ol$nIj`K3x1&+z?e6_aH+_|A9~>+d z_QN}~@wa=Ur=tUNfjiRcSwy8xZfs7C&AxuU8$S+(a%xA1b6sPX*K_yu3w{K{qurG+ zh}fDNW?vL~xG(ksseSsqMEA|aX-lyD!~{1n#qnDhZx_1w&(=eS?B<1T{)^M}aruiL z8XTFJY9Ec$xl2q%3p7@=cOgM%FEKT;k|J-H`L+*FBTz<`yOmpx+WT@UeUsNC#$ z@5562NpW|(|8#8ivHfgv`*SgQzHBCy-^hfw)Rqm$>cm{Y%N`x0>WYPBYj6~ z@m@#iWG3^=6EVDvDO+mtinM*W8^)`ID(HJ(Q#o~wSE=%=w9>%jz$Mtoc2uR8KB$s=xOcaw^3YUAcoCDjOa9q<9m1;w&1|f^PV#ropQiCy zy3(I=ZkMUj5q8VQ zE#ZdUJ(%MVrOG$=_&!!f&Nr%4cyA&^>426LC2#w>6y|x*x|*hKV|#e3E<9nLG*%r- z**4(}8+X(gLH_9*WVVa9>$x36>k?tEZ}h8=FB09I?||&HVU8VjXCGiRpN;UJ%|)Vg zEVBw$C%R9EV)>cD>$r}?Lf5+(zi7QGB%Zuqro-bFp~u;Vd}ELR@8|nYl{aN#(P{yuNJ}trIJc4~>vEHf z&9C(g#m<&irWA9pP#5BNJeO~h`Ki;VG@V!P({)~V zGkCI%+s!9N9gu4CMfuk^W2&7W6lx@Ys61;W7&xsR4+nNPB?~`9|-I3)y z9#+Vjz7nmZZuffmCULaVRY-KNzt?toa6Wc6JTV5yx_vmI8DwP<`9TVACQp&S-E;!0*SxaghB{0K zVR82piKFT4l+86)B@$MDuD9B6oPD4_hr}LZDl)V>d93chhK@8lRVu{>XB9&hy*W9t zc9^c<82zjzh$JiXtcg-k-^AnL(kn=7%Xa(R*FcH<6m&sWh>w%_P#RaObYEzAH%0&SdypNZ;>tupM+5If$}tmEQ2Si^C*jr8)_tsfZ@4QHf1LI#!tk^yXY2 zud7T!{(iMQ1#kE%c-&dUx~P+&%Zb!&J_+=psnO+7NS%U%kwwiE^-}o`R{}p4Rv4MtIN?tY8A21f|-Cg2a7`=jtRR9~1l}94*d^@Ss!K z$tb-}o-T74b#|qjn0zzbWgi#0bL38Cm#^)Gtv>L`plLu^7pW%3H@Bh=NlLL>s+L0M zs}bT?qZ+5ljFqukgwr8@HTWiEr!u7EAM3Ul#z^k`iZ#&{)HP9=2Voz`SWNj-1+n8( z`PIX6E~+c-eTu!NlD4dT-)KQ(r892XRl!U9a!BEpUAFMjs1M2Ut;96n9t&f(Bh<$G<< zBHErslzvg`*~$A8Xwg&M+B3d)f#^(jI}q)UZU-VqJ!CHy=w1wW(|sonXv@a}Q$4Ej z5}TpQCPvfPMIb!6dWe4p0>R+UX&@9W?FGVJW2b>|>0~$%8o4+Ngcr8JVaQFFfzF$? zgFxswb{q%~&R+&vC&pT_mFg=WqH3SQpIq|imBoqfim6SttZweyRU#m;e_ML@P&s?W zCaqukDbylfIgFx2F}!t6&yOOs*1*X)`Kr_(8P0@f`j@l)#lqO~#11_#Ae|bTy1N@5 zI?RtmR@d?)q2uku?pi9Ihztz}M5hUD$x_$C3i&JE7!bsu-W)hMKNy!xCK+JKHV6iUnJDw68r9 zyrJ6^W(c4IUxN4N>J8?kl0bWVtTYKE>jD&=J6Z- z^xUuV1$_$rDxajMc(M7np`jeLPAySFBJ-Y_qLYyt?@#r&1P696_h%}GcFb~*YADz|KTybrq12JtT7J z;q&u5oTc>>HZz>S(ms$^V}yFoukQITw(bJS@b*fE%)=o;f%A9p9StwfDXv_p_JPTn zR7aqbVB>Wf*56!TYV7sMSp4LCVhAqPeTk##_R09n(n^6YU23c=Go8#|PUXmC9dLzTh|E93OiKf+rE46&QE} z?$sX%P2C-UMO_8j^Zldv_6%PTDHdoQyGaGQlZBN)cYdW9=$?PQ8Hik@=K|gP$6L64 z9~^ce5Ijj-;=VO<%jOjM+4Vpm)V>o6M7Bd~xNkAg{qFrnAbPcQqHBOWB|4TmR(=?~ z%3PCYk0;@^u@LCq*xf?2OJL#q%3IRn3~pW_w{+~`i&fG;OeFC@7p;OCvz#N*TbCSa#H}P2>Z|BL)fki4% zlrfXfc5%x(p3>hS(z-1D?_K>(+7}$R`A6_`P=>S`0=RS6TCfCCWbs}NO?Bekh%`84PkBPf;g$ zO~tXng&gK&QF$J@eVwfS(YAQ9qN^h-TC_+i-6Ru87Y9$F6hR&}&6XP&s-RrwR=J~_ zSk+DCKGK^vlwLG0O4a^MWfG=(a{jEWN%X3o9L4D#g-g)mO+_iAj5)m;(rR$}7LD4Y zXMExo!&<#grbVOn)N|yD`=aiQuVN*?5A1g4nH*YB-7tzbxma2HbXR;4hz#iOu8dhz znZ0CrtB;PO50IKf+}sURW?$%}7Pct^RFbCgq zYFF9Bq$+%K>jUr?>EYFmSlj(zWPRW+cY4*iUKpH21w(Z+c(dN2aMSjD-(#R7GSP!0 zlLy^-0i0Zy;lJ8V6t&rTI!K})E#zWn#ZBa#n@}D~@S**+6Xe>F|H8@Sa+U|+JO&lkAht93d1aonL>rHXF zqe3cL>|V5q$;!5VxOmp%5R;Q5y|ZacOb%a;%#vlwM^odmx4`7tWTJbcvyZ5QxltO+ zV-u5!Hv8T{c+8RECRa*#OJjQVII3lxWaw z>=rZ2C+)*33TQT3eG3O8L#i7XpZ9H=EZEyQda{oF%VlbuJG9lrjD(Z<6Z}w?P++tV4mX6w?JbTIUOO_Ym~w`TYn`c%h29&KKl)hofDoj;_gNc-@SpI_|p2 zHgjTqAk}epwyx}kM>ZB_wH`$4$3yX%^SMC;`KRDdQ6%jQVOtwZlJ(u3N|y5P4-NLQ zU1ap5MJg~9ce%=N;WSLGY%M|EPF05;1ULF)v9qnsG5VQ;8{MIy!T4+fDVQm4oFAnU z-S6THyij8SS!6yhCn-<9oHwACR=%;r$KJ9#5)prgJH=GT#^qtP31$l(enZ`i_PZn0 zNJKHYt>TBbiAUwM)K=iBZU3%=^f8@FmKGNJNJn-?;&1n()3A%AQez7X%W6R3=J`fs z_~GFJ6Fo#DQ*MVMBjIo$6?~jdkyqWVsg8l299Ad`6lgLw9L_{{T(e|A?&>v=_kt6!5FmhWJfF+I$h5v^DEi@RL9`pOzdoFbA~PV{(}a` zVh|2JIfACAF_=>($yDYhQe*RbxkPdO{b;h3T3c1)Y9GV+dFDod zXm870ensUld=Jw+e`Uz0dUaax5bD-MnSR;nT{YusWx)H+sbM1|Eyp95alXts)BvN0 z&=$dn-eDRSfK{7EtkPh7X7ey!S!qIsfqb$nu&}HndrI`6m-YLJ_{qb{FhQ#Me)8<~ z;Y9buDP8G4f))q))Yw^x`c|&Rk3%b4N?gFn{duRBKD#vZsgiw_<*J(O!(qZ)3PXe< zGC9?%xkiw4s;_f~9-R>$x!;pG>gg(iu3}Xpp}2 z+>IuV`ZCKB%L%u9B@C;Qr&m#?yCG@2)L2rg*9>iisR&k`Z=nz=XAOnN3-i?{h`TaO zNNSLJLmgg#AD+%9C&LH*gvKVu!N*T%+bG+xixp~3?qAGhDV@Wg9lp9mR*VevwxPg5 zhTf9uNUmq8Xi0VK-w%^l=>hUZ@Bu#Xk1I(<>KrYv58&7R3c~^R1HA}I_4Ea5PH$}dz-6da_ihurceM{h>+S)3&y4IsIxm5@a-1vwQk;!42PVlo`PpTaFE zm_z4N9j|u>33kseV4E_7LB3(Ugj?7&_Ai~cvGP3iRk(?EzK3`O;^#PB32qRf&j8|) zhet<~`R(a+^^$CE^M>YBX13h+V4`^2PHtY91WrHnC5ku4V?@}l^cW1SQ=8$CJ5}rL zrrH&e8-Yj43d<&9CCG{9qgIbX2z^%YNu|#+#;dVB8sB{(eMX2Lb&z*RpH%1Hu1NOU z&?o$pr-q_V4k@^?yvq%K@MCWW2(l0Kl2gfN|9*&|eLh3c9>FAiWs#o2ePnM?d4Q=ukxy)M9eCiW{R{bL#5rgY)0!-j5}Y+TTyn zVB@NN0>9?mEK_o&i94d`Y&BSLO7AMe) zEDU+$43&~;u=+V72)1)VR1zW*x3e%nrMG?9aORk%$cqfl1RTl})p}w$2EX@F znrl?iVccVp4u^Ebd4OZIjAKWyvehWtTqa!eHu71B?8-*&<2+TCx!W}32MTW+!6IkN ztchQ4WsgC&9CTj^$=%OX10wfvJWCYFV7km<*qzDU``cDTr46bWLw*B?48qX{B99cO zN!yWO`Y6)aNt!ZL%C49KGSET3IAmXt-cX+0I;3{gNHLxqCp(nl${2z^fp<6KM3i++ zQ+^2R!w}h(ZO^MNC1nQ!wj>!UacKtK(=jD*F&^rPvGadbeyZ~9{Na%u@>eM+LiMDw z*{Q_5^V=N=-l=4Mrz`15JHCJsdv3I{CFZKjX?YR1Dc0(n=9}W)7{yfGmY9ykVpANf zkT}^*-j5~sFS~H89!!t{b|6~Rb2L?650<57JB_HJ$}3pdkSa9?j;ThRhCW8P8~qJakrff#Pq}U~+}@K`nIciE zDX*nz9Vku{Dww!4PSMf=@=+ zgbGu+l^mhaondN-CiFBG*6`Y)ZiPL-8lbgIQlufZ-OwEGF(d_%20$tlK`FvdYIY<;lG4{a%t!n)18 z?uWfmk^P7o~Xi$fQCyp|S3pi#5{d{TeOOtndL;LhCX>nnO+&s22 z`Fb?9H+i>vwOYsqCQGBShv?+kROancEZTldw7$`|>yy*uDf^q`)6_^I+jhQ09#tsO zACAWEPX}q+ z7D=NEQ!PO{YBG%RU2;=oee!N+aDSrXY;1#cIk%VJp19l@FU>#B=N}J>b7WK-^W^RB zB<(Wk{F`~&4oy2lJ2E*KitLU}rHA$-+4M=AdIXRe!wASzGFxPrGaZBqywj4kf|KZ)2uh7>wQ~G$*jqz{1=($TgB_U-h(I{M*v&0SOOW=P zc8WZuZr z=h+1JuF*l}I%25M~n11+40eik}2#QVY4%ZC0U+U zmzyA(igFXdE!NT0CRx>!O?qGKXnW`l=;fI5B%|aB+ImS2T&svaX6>p~$qgmZVsE!Y z=dZH=n6Yb>l`n>=e|^Qfp5|03rMz`etLmY~OU-s4_1UXLxmWm{!y^AG#SsdT{9_XS zRn^{A{ehNXpfl9oYvNy3$vCLVJ@`cY$NB?ak58=tQWeyYeaPA#Ux#pf!44!jVJVKx?AozJ1foA#OI3%ujcwp!IbR5sf-C(L>1jeu8W> zZ+qgi`K|?>2cWh8UzZSFRi5k(!U~sD$q`XQf}|G8mY^HIF#hz^%qxud9JcQlChHZZ zX3uu$7bfc!))KVz3j=cM{lboBQ{&n2YZxz+7Or98=-w!-Jsa%{&YZ*I@sp*YA@Y>v zL37tI_`>O&!}|Zel_wG0+QVaJWm* z3`Q{84cd1T6B;j-97K|CVnpT`_y7Sxs5NpBk!qdnd@ym7EQL37WvjK~jd7S$j|sO& zV~Pcn*{??w)2GgvkjEFDw^w^U_U?K#k6dvn>g|OyzOqds9_rav>>c@@zLfOpc`YB{9$k|owlhbwj@B{LXufGkDJmZTmo zc2k$$SYb_Fvtu~Xfg|SV2=BijFi%mByJ`c~$UrQX%@Gi4$)2r*R%(y-d2-e`fAwh>Ug!Mv#y|w&%p%%;KSs2a9h?C7 zD-INisbG7GEd1mgA)mRaWcX;aWWWzX8k=ul7jN^mal)&*f5|!v=v@U)&Zlu2vRS~o zSUC%Da>Ld!uaY#NLinhRGJJADNjmjRUk##w0e?LS>{9?)YKyB-h zx{_we^HzH`zFEn()Y!&xpUB&En0lAd(-0abo=?#uU!3aa6z2rWiz>%oW&Xo_%5aiK znx!sZ^cU(aWw#dPa5ub2PeXE4?jzBD>wc>2&$cIGSHRl$MfGe$b-|iLzq5o-rHTZ- zN((Bl(RfHH-W_@$DV89hV)KTHm{-aQNJ&n0 zI<(6-iSCy;hQ&$ih6agTb&#E;;nK(l(M>JSaWzwP zCHh_27EH*VU_(~Q9T3FdhBh&R>>N!ZV(Y|Ftz(rap6=y^Vq0^J%?a%R*%L=^3%uVtAQV&MT!n%?#axblN!Y;B&B*%tuf2G)Xd_*IMZM zX~Xd|TO_v>3h|l6H}T5QsHX}U+|($MC8fZ~jM32eJ~A|f{7lls_9m_;sE(qXy{f8{ z#!G}h|6Y4YNGeWb54qEs*Ld2iGlKJdDG#bBW_g_IUW~5@&g%IsqbjddiCneiJbRqN zle3Ji-99)v-qVaFOOu$!XSSxtj2JT&YGog4^|+rpd#FeIw*x8_DDF*txw4l)j0!D0 zCAr>@GJr{qFUoKyrS6Icr|MEoWne`bDio+VEDS++&M*%J<_%Sr$Xi)O-KU-`U1Kx- zruzLrYOE_TfgMiy#co3!<{WbBH-#el)l0bH zDdCpw_es#pR?YO@$?b03XAoPb$*&VG~`$;jsavpx=fDli1w0tH#YJzd+{=RkHAA9#UI6&>2 z6}3wBxxyA`ry|=#lC7P&?*$)JEXm&XD{jaHEpiGc+CM5N(U1z>;j@8jVtael_YRlgwXs5iFXxnWAo6iDF`fsD7p8 zK9PMTt#$M3BPcgrh`-&4XLW9V%I;NXUy=5pH^arZ)GD8&xQMe?yY^Pz*UtzStF330 zUQFy~-dN8-FJ|^L8SB~6$|R9ZeSreqaA{NhuzD2~>sa&dPYksDJqLc+CS)G*4%bHgZISj7ikytK)1PhgCQ z<>T>c!)R|mA;YssA4=B#H-MKP|XGc1gjJfoeBE$A(dRdEpdol^s$f zZs-{%q*y1mPkSsa=KNJzncA-uZfc>9nmx)HwOWB`ORaT7L}!m;Q&MX1z~rO8N+>y% zRRx4Ar89j)^&|S0y0uUvr|JthCrrg!|&4 zePW1xF%hL-v;>u^W8>TFBkZf`^7Cf-=jn>&C=Nbztq0}L_-Z`Gy`FN}ss&_k;J&)a z!8o)~t~G^?}WCx zk+$kJnvEGXt80{U7<>#3?J+(I{IYtv=>b%H8*e$ zhZeVcXqr;k@RYK2&_~p^OX1Bj-fbuOnqvdU@5^7LxeQc?+*nGl0}~R}-jqZ{EMrhd z`5%3mHzhXVggtgQMu!2-Td9Wyt*BRhl|LxgO507T_ibE`>3g#{v9?>tU$h>Lt*pK2 zoIH$&3SCtxhJ$ay}GdS4qQ8+O@m?0a)Zgo?LM-is0+5}VTGfCq|}mNdp$~hv4tq-+^F!ezKyAg(yaYw zR8!yc2MSXJM4HN1s(^}uiuB$R6a^Il5$TZ7d+#L>5D}Fw(n~1Pq=b%?P^Fj9dq)T@ zv_OE=hyU+c_r-mG-<-A1*|X=&nzhU9`OKcVoTX2j-Uu51dt;`aE4Ni@zMv2oB8++Wyv})9=#B-!(I}jr+0lVINZ* z{=t5gNMqjq7%mmWSC7HA; z$i8`H{{D{(ke$4=ul%Gx_YYo>=0{^GM{QxY2MZ5s!W}&KVtlj9HjCmSsiwX$&HSgc zaQkCa6==UfynF4Q884194ip(x``EJK_d0Jy@5fr;V)O(d=HDHi*8lpw|D@a<(Gu)8 z=Hb2fPAfP#vrw~GBdGO-N%V&<;eh7;ziXFTnTYvP^M(6h{d0m`cW_vB(_1R};<1RX;oY`vw~d&eH_%qVBK=xxat;f`wl8v+Hx?B8(K=u@O1Bqy zvU{%s#3LwlUgEj87Tok(7Qw$X8MU^?~#- zBP;v3`O!XCM8EG7{NcQ?(gS*l4`;uP!oY5{oeT^2liUDX$?YGCu8Z;*F?>dfe4!pI z-IQG|FmjOrck`Q=RptsuOx?=k%ujM`8lcZ!y)0eycIUQf{_R~{UHWv7`x=_>i(7{< z|I>!g6`dX$(qT}$YA>lujTD|m&%7d&H!N*)8&aziG+O0mIU!+z)S5dJT5;<0?PCGMGaEl?-;{eGku)^xjsZEOCFu1Gb0d zHoi2v7cf6jo;@)5&pLJJex4RPzv;hj2QJC_(R?<37e1T1_fb(E5dofeoBw40Vt9Uj z_fCjiOSrTiLp~TA3-22Y zx{k+8kNA6N=vkF-dE9S$i{Oa7^ZBWyeo1$v_j0-;#FmD>Bv&o%vvSA{`#E->8}*k@ z?jdq@-}_1{9=3Vt#aEiP_u1btOnY3@=`eQku$ZNQhJNZ%Lh7eny7bAZ0KJ}s#qsc$ z&tLrR&`&D-cs;|JzKhw(ZqCk#KmJ}KvPZ({oP{(PgxlBXYAuUpq{R>b+Yfvo?#9Qk+SvLmEO<1YV= zw|zXZ!v#T4nXY>b)qX>a{V}<_2VwV&v^p-Wr0D&Xks9S~^5FUk1}+$BO|kUDpcPI zko2)NmC5R1Uhy4#v7*;{pk2ThI8E0mzgLACNd zZHQTJH;dR>#JBi#-IxIFF{AxB+SqX^`s>4eVpWQXHdSSa{5#>Z+>X~}AG^hkv%yt= zS7l2MdG!#w6NRQ^Y65r*`Yns4Rph(dt7t+N@_CKR8n9xhV8=7ZyVMZ`h8n2ixOz z<5)yS`u_Dy!$;howc_0Ue0^|Fz1@=s%QNpP-!;twW&ZPIo$)ge^mA%!y${zNn6wL$ z?9X;~gMz_sO~C993qpU#t20sh-ONRor)>!#%QuoUA-PK84wxB;+eSrprswg$&nF9i z%q@G4dI@X=bxGkHHN8Q8!EfdR;+vo7LbC$zWV8krH~Haq_t*AUEiaE(FAi3BpEouD z6QtE)!keuxB_>u9M|$Yd*h1WgHyv&u7!R85XK66_9YpgDPNB@_oy$L6VlK96+)Kb@J)i}M@ zhsVO#ZC+;ffBpRQv6k^w_LdE`Y{bOcosqr|?)+x*e*|C8-@dVN-koen`WJ_c)e_ih zum!t_)b9UO=Nq;Ky_zWT)~=@_!hOMyzKy;q)V(-U{I+l`jHkyhke|jl925by63Km< zFD)kyHd%O?J^zjPOH3_P%M(`snCH|$@*EPjNwWVpx7>bNGm#4VYSf0#X?hUUr%`acdH_HexAbWo7T+yzdXA@=HFp7F@#Bvxh zdVCNpOe%t&%kGg0LIfN3wNvaDXya0h z8RjR4YUf=mp|CFm$w^j_nl}}sLfi>jlYx59z-8vI)uqdO5atLYWoAs)- zpE?N^fnM|F`A4Whw$QpGES=5j~1i-_?wTQmDf75yi~+2LHuU_*iI(YQ7T zMimjmHK}Qy)6M@{qTzm@!hI>c50av-tK0h*`**+PxB=msL`uTc02@G>7+s1-B<-(F=^g`)w^bJaEVe$FTn+hO`hTY>VyL8ypt}>PQ%4r)N2ddAsY z(+^Y;hM~jnes#`v4tAsmZ96+_i{uL#9UrUh+A5{a(Kr?W@#ZW;`<3l+fs>b>PIgwc zR?`Nv&P(%@juv#XRFaCPDe(;38Wcy6F|Nil_DX8dC^Obd&5jE++rD?-j>9l3Xxd2= zB1V_sl9P?szPgz1Y^v5W%fOc&I1-U|aD&$%MeD+x-Zm&Ra6u6(Ivps zja0rTrL`-68aw1}xDy*|z0tt~-@2YP6Nuc&XwoFuQ$L?Ikkc$t{*{k-`&< z9FcbvR;lH~1Usd(NAJs>9dTDuDcXsS4xf5kse_#!hFJ1F86R83LGkYu#cI@LtHITF zM3$g}mFWq_=A*1T!%_-Z!=FOk=j9Hrp{|ynGlt(c&kpqnj3c8@rN=;#n+}c-Q_(I2 zQ>tO6JY_A(zi++~>aFd+nB^&J=6mhEpkvKixByTMv#8;Z)aFx8&3~$Py#v;a&p2|b zThYA)jJC#NrBwbYj=ttjYON?U6wY(OW!)%VxA7XwA|c=idK~*Os%&@B}7((F_Pt=W1q1$p(&m8ZmB|3^U!O; zo{v(JFUB@L0x(O*zmlwW@;u@2iM_floI3qhcev~)_OH$imo&W$ZE)YKyT6>pLu*_Q-K{y!+|OxZ5nIJNWKc zDwD{P!*W2m_e>$9bn~k2)rik3KqTRvc(Wq{Dv1fkqA9^_M|P)b7ZJhFu#-9BrgBbU z%q&@S{-5xbekzLXcKdPWvJzcpvaNRQ&l_BFpxc5t2E>ivUJfy~XnNnkXqJt?{$Utr z!XGidRQ8ZKBqViK=|rh`+D8{s46JMc5opY=LycmAtL#xWTZdgiM4JL3`7=c;tfU&$ zHMlSaHzGoWR93AwDj~wAIS^~`F*h!B!)kaC1^gNd4baAk>F&)n3HJ#XW?SO&-mtwuLvczwN;`ecDO|cUP`r&R0djM#CZi=)?a!7z zaN~H<;axE&&$OlUzNx+xIesI4M@iX&iRhI*Pnpjm02Cwd55>&nFP1eYUUf+^Ar8Iv z>)fn2lL;x>s2=CELBV#shh%x2zDak_<4*c{%nvwXRN8EV+~(`p0Q_Ox2V%}RF!WB-O)3FxYkn+zUYLxI{&OveWmYX*Zm&>d!6Bb zM-2V5_iF)vGH_bH!8J>hKkI)Zgz)cV?8=z`b1UjVyt56<=761ih%|Q*BT_g;qm+HP zI+d!GZ}j|v$%;B$+v|i$uBN2|g%o2R$IjDh`x?(o>|n&yijZEn4TMjgG$aaT?W{W^ zf1_ZoM=#(LYn<<3AvfCXBT$l{d6hGXzcEZbkGHuWMtV9%`WUPK*H*>9 zBG!uG2|nv=zpKHyxeNID)mzq zl#*Sy?_4}ojC=r=f?G|D@?F`90ixCkAi)kpE*DrA#~6#~=8XxXxkkyY%H-F?aSA--Q!1`sb@XwC9-`bd(i1ev_NQeUJgq&&@Dz zW!HZ;sqyO4sN6cXm<>L(saN4xd;d4(y$QO~05){~F8Z9&jLnKBr1H$OnXePPeZvrC zjC*Dt`&`s@?;)lZi37PAuXVg$vB2BaT$V}0Q)px*kne22dIEwT31v}PU$eg+Y%ZfC z>)+?>3Bv!fBvs0|;``2%&<5!pMW3=Xw|l=Ff5-CWEjm|QopWW*Uw8lZFZ*%aYso1e z^FKSb1ZT5*svUwYmG^U)Y}K)!K&dg6oZAs}`G&R$!}Rhjqtywf-9b7`xwdsXRH0z4 zjqx)l8cy385EAlqahxGzujrt~@M+riMw9C@P@<2_j|;&m~$U||IvVvZUrFPTOl zXmOl*5N6-Kry{9!{NnFmdcpK(MvgTGWSr4ZaiVl_(E%-LO*zhgd0?U89vh;k2uTeRb=CLlud5Z z!3yeRQAiqf^%NslmiJ|(gxpy-EPCi($@Ie0TrcE13^V^7+MhNynKjqyRD>2a!PlPu zBrgfO2HE(n!i&m^#LYv{I0#3zMA2BI(5o}*Ve1$GxLvmZ9oWig%g!?k zrSqkA`x)Np*t%=uS5L*67JeAxxk&*`w_tSb(5$c+gH#b1>T)uQKweRoiW!?XBsVyR zSGn)!FGQW-S20B*PWuNK>zyx~dSx_AtAUQFY$dT3O&w--p^(XE=5?*M0oPy>OJc+! zr$yU6lPF@$tzdw(qCgEkW3xy4gr=MGb-vY&fSXgds|NV{PHchF6xozTh?DY3urpMb zAt-&U#xQeOmjb{%`K-4rgJUo7Q!$`2tt{ZXFXidfASURRNR7WZ9O?GbGjcP|$#*Uv z79^S06$ejqxgV%}l^z$ZTcsMrsp}BY1qs%G6KTq_heu?rPo$@~Pd%vg>zw@o%R?tW z<5VRRbJYTtGIYLF*Qo0o{F;Wq%m}9bXqbQo@M6$2TRNQB|17rq8T?fD=+#?JgPjq8MY+;=?d%s1DIgyDEK)On`nuBtg$b!d`MU2Lr(A}2KbMuQ z*=X_i(xt1f7=Zw?{*FoNCbd}G#skBUpMew1E5PIhRFus={`jR7f3Z=($aD-_7F>w9{K;DEiW>!qBwe) zTPmv)WYT?4r9OKc$vZnGoew-tNiQ2J@hc0}HTRX~T*x8)E^vw246tleG*UK)V};vu zd&^5#hzmq0BO!RF?e8o0ZTfTZc?$q*HN@m6b#a2!UtFi-uS(N-&{#ql->(Ewb^zQ5 z?bXme;xTGKB=roC!%)p2%0$6uNFg{h;G5r2wyT;bE@~`x0^^h%S7Wr@`!)Guo*KfV zq#va6rEvekYWsYOAg?Wn_sVrsv-==H^4+iI)@`489Qw@S>~YITjhoN&*xB5Sw&jZ= zptq38Wz8?!UwpTOt010gb+F20Ldf6PxGUReqK`Dm==)qbM;mu6KHs1#JM&ED6Ebnc zk-G}#8m+$2W71Zltcm!BGZh z>Kt}cXblP3mX0<)_hxDt7+ZLjVuUct=gMjU^=;5Kr=v69!*mnCvLK}NZ}x#6k!pS^ zMdrRyCT&rrr7{7m5}0AT+>m8Ubau$g&NnTq-_Yzrx#rgP-s-yIk>xg&(0w5_!5Jem zR61AC26Ia^+YfjaS*=qJ5-`b%C6|;nesxvh4h$Td&@3NeOC^p*VTuF6`lNPS_I#!q z$(Qvj(_jgrr>9KsR(6{;WHfU;CM#H{W|RplVte|*DZuL5vtH@^T-y{?SAXG!8N)G9 zey?aaMk_#FG8*aZV&syVaoLQLsLgBZQ7qbfA)S&a(JXW-`M@b7NM9JRGIQ(V3lq}B0_ zq%SdR=_pE0DjbpL(`MOP*b%}{PHb?V2i?hR41BYvAqW?{&Tt1hsf$r;G)8ELRfAcZW zQX!kPzwos%bAZbz;C>>1A)7*sK|Q^!j01o+92OL!X=p|_dj0f2Y_} zs-+e6pXmO9_Rxi|@Ts&UK8MGoid&ICy9-Y|miWbqd3|Bd6rIhXJ%tAwt$jB$d8O=f z%~nEdI=Od{&Iqc!Y!??(qAwl+kYY9jup`>OFU!6C z*-u`XsMDV=Bdjw2oC3-ofbWNh90y2V*5Fo|X3Oci+&i9z(BtT8N@{}HF>RB0c1h0O z6G5>+spe4Rp9>^MNX-R>@?kHHu?l$B#!j^3$`of8?*)~ou*A@%EmEpAMYWDpo0&^f z#}FhDed-ifaS3rKqAR9JF-^1(1}Gq815(46xZX0-EpDmHmU3Q)_=PW&fj;RhxH{Z! z>3m#N%LsSaGW6j4*Ws{q5ANvkH5>UaP50(hF!t&4q(ktMZHdKtxT1SC3AD}^iu{%f zjb)u{N0m)Apze%RL(F}5dKdyws#YDEJg#tz?sx2LyQX3g)o4TlvtE>fnbxVv;>x}} zG+2yIAK(A2Vokad!{6X<`+2agQ%?tGW8d4rqGaa&A^g_)%x_ToSLh%>_ZQ59{832B zr&{(DO|8wHSrJf{&2KOw2@|a_g9hHV4>cQP@wvsc?t8`~Xby zzLXIx9jtIEWg|{t92T9dqe?dc?^s+%JR$h@3Ch(hRzzMlK?lsDBdR{jJNUKrh*u&9 zR1Ye$lRx{ha&^SiLffB#uHw?)Cm-rg&IDpS1X1QBqmc>9grzrIzx^O0M{lgbR;`mU zlSqH}-=%9OMAvk_9J1VTaD^QYfH#^jbCJpg=t(<1RQvm-3J3Rv8!z++AJJyw0J0xg z=s;Bf?oI3UHm84pg%Y!JHWE9)pFztq{y0Mo_1*$cm!E3Kwl}@4EECmTqu5rbq=D^F zmfABaVhnX1w#+^@Q&qfp=ggXtW(78I0@=$zr4-He`LoboT*0J+Eyl^g^d^8moo!W; zt%35$VtHth5LJ*B& zUBj8zl6}CAs(r)#eJaaLgA0dFc$9TX@Hh*-Da^ z#ph;X6W_RO1l_ry%g^dH`2KTugu4E$xbhNQdcB}MwtEv_@SyQ8!0B^V`;L6e6kuSVH-`}2p_C#`+dBo*1L4G|j$nYiEoYh`UUaDz$pN$ZH%DF#;! z=POenfr1>^jT4j0i zhij#FgA*N%NcxUVOtNGkg>v6X$hPTr%}{G^dD#^l;2V0@MN)ey`yf4GZe=z}zFlxm zfvpaZ-@uS(d`}J2fmP;!e=S;7l*MzwTpd|9vL<4;dj{Bqqc+-aZSzY=eA%%bekn?fsNptJs%j<;zmud8@N4-SbcH)`uMYN)I7e^OhEU~j zHj}bIro+V1Lzwstz03gHYDNX-Is^-I+Zp*?4ZiKh#z+F!XRn zBgKCJ4^@1Rl+0H9$BBJzWMu>sFWS4Y(3Mh2ZeWO~9s=j-d+nh%?eEH!gRScdFBn?! zj}C0C!!F&XKlu^}4ly*dbuE+qY0;O)rxKhLM)+NM*wwRdB_ zJ!0m95i7oe-BC7o6-Ta`P6b-?r)apywZ$P2=2`Wd`L#E6LuJJ!ALPe@j zTn)N^hzKiG1$1ku(;7VR*JHq&c1iz(H+KA8#Px6;5vRbZ&GPOp}J9KPt6&?cUyKl+^k($#@CmnCl1w@4PFDt*yZR5w&TA0 zUbN;xw;BF1Co1nNere{QYxTmJ9zCq3HyWZii5L^f?%w)*Mh%}^JSMguwtn$?211SR zIrzI3=068;uenQ(G96H|ShrBLv0mm#w7H#f%l*ZC;N;y+b`^_LV(B&Pzz=UR6REN& z{pJow{hCQTflVEz<`o>}p=v{0p+$a$`EmtoMK?fm6X{{DIilj6pL~*%beviKx~px; zx^C0T^@l_;c-h&F969^?wqIoN%&+axR#UNq!g3-;-ZR1@C;YH2XUH_7!>81uyP>T%Feo*=_z) z&ieC{Jib--Xxe*{)>zxXCXWU7K8JesmnPuwV4OxS_o_zCAou}Vbd$b_08tM-^ixwZiTN+#3)HIEK&HArM6f*x9Ol` zBDg`q!>mxACdw;Fh-92yDru5+weduXCb)8lwSiWj9(_g+2Ee2s%w)D^v|<7GP$D|) zH5DeYV|M5Ov(uvt>G#g3n>7g&v}aLD>*rIV_II&EWQ{XVWk>8}1N&7%|k+?ml?f9Ccb;tPhd3*$<}cjL5W z589oYo|}MXOGjxFY3hD))3);!*7WZ`&&*y~{Y?WpM#<`XA5O#s9esD@Sf)&ASuNU! zWL;`crMu3iDQ-Pivu!$NKiVxzpj`3e1hT_i7sCy_!y7H@L84nsO0XTRvmEYPWcCw& zN%T0=`uza1lxQ&R?BXEq=;BJemOO5@nuAxd4t+vw8oHuQlpsuy6nuxa9j{95hTveG zurtb#P^i9=vSX0o&@LY?4TS7^@Pkhy`iQsvF{4e#Eg-1rb}_(Ylbsbf?t4x>7a(^kwc|<`8R0 zIL1^h+}F|`(&WJcx>7FBav=lgG511;BjUJGfaHn?HVn*8s0g))GGQW;bFYe`kr8EJ zLy zwPx8WvfpXf!o+=ObI$GjEZ%Rwr1D2Hqa{D)#E%=6PkUnF&Y&fdx5IC+pD#pwrn;nD~eII0*Hwcl-5Mu$26e&6GOKWJdDSs`CsVo-4_wxD%1rtRWQ7z7lUkmX7g*c5a_NMUqE>V0h48PnFEJE7? zIAsh@D)*+279kFqv0L2@RD`SY*A2uO>@(*RYVLVa;i|8e|a-cf-|tK?}Fvh@k4W^EdW^m2O$sh-+4 z9mTWak&%_|m{qgc3t1H)cQ;HCc53?FqxH8U-GrKLwNxu#7#WP}gbbB!VLnJl;*^B! zH!jON*j8EHMpu1xq?cAcVzU({(mGm6iS6qje)A5-@ktGPq#Ej$4trsL9-e(9slK`& zSXzdQsZfIJ7Wni!M-{_Al_~eypuOEOt_9n{{E`xRw2@Im$rja(kM|c?7Eed?JwhEj z^F}R5b$&S#$U_>i0slgg#8W*tD#o#WqX>rC2Fkf>XV3W}1i^Zl=)}N1S-rdMU&D3H zTMGXLlk1L5^~Ae-?e)~z`HDyVL1&HjHUrm>xLIO(g;i{*Epd?J=WPm)Dk2vLzGl2H zIiy1>f@vJ#L;gj)1(kOOK6)rt-RL>3O@BIiqnOb1u0qd942Xb~peSr3(?y}x+2V1l z-GZ?k*Y3$Z*)ZsFmf;AmkcQGigpMCmFMVJrY%#EL;M-lJ&I1gZzDpU3)UC@M4OIX? z5)Bfj(z|Y~RBHTfZ8cGTkN8j$3<~S?csWl&ozLne!+{A2$=^qv;{0M%*2aLwi+aL1 zPDuSMZ)xGr^it^?j}Pv`%5OgJn=NZ9DRUg=k>_~4RJ-`*P7G7JzU>nGK;HE8TklnL zl!h*Ew;$EcjW0aZHFzncA?uVi+Rk_}cVh5+_ak4d+A8-euO~*!_U6gellq{?x<^&@ z(_FkicO>;(|0_NrRst9jMyZu)>k_8}KPC%`}5tB8;RRe1%)|BVMx9*g8QSj1vfXn9vwH{ zqX&{f`0@hnt3c4~GNVG~SuCM@@ZHWr957w2+@Bgqy4a!SoQUFjlbvF%tn}ik;?bJ# z^qx?;s4`&(g`dGYT- zEGwjFB6iGVrw^YpJ)N|z9VTtr94}~3-9DH&eE&F@EZ2WW=?00PnK@27qRWbH0zv^~3aj=((n`qZ-o1hKh(CDZQ?bg6Z2{`<~VD zE+}MKM}lKCGEc?G-^0n=>T;N_CRfR-`IYbL@2tT5)w~~Uc9Y%jV~}hpsugdD34-ow zQUBE{cH&6#cPjZ<+?sGBc)K%}3H^G$TPI8VW7Jpa!(g4?FJQbtw+eZ{GwrA5R@6JgS#zUWXK-@_#3dDg&Z`G>_8C~s8~rRvCg zkfp~T6Hes$lva=uocw;c50LaP-PY_rcMod*><&-$X&?4!?~g0au=fn=JkE`P61)6qJ`Q<9O{>1c<| z-+kjUxY6IfW_I6aC@M7h z?uWawikx)57pf`oHaH*21afx2pH&JEF`A;X9kqW}ubx=);4MQ52ti-FCupCx6H46- z3c__)Nf@+FvaB$x=8qmHD$Ob{jKhAB$@_@LWYE;q`Ce$~OUnCm8)~mgEf1&Ab=#8~ z=JbrT&d!5sFYd+Pim~sJ)$*jG5@`=(1&30*cI2|sF$Yr%u2Mm$Gs$ihb+Ev6;zxeH zBtlFs`nP@mU!jXgdM&L>DePN&5`3MO0@|Gy2{yQ3zJ@_MT(i0;*~ahrw5eZExEKK?YT$b;QsI2iYP0rTzfL3csH%Jq zx-wV;L?$n=7yeyoZI(bI=8Z2$THSspu{J+1Q?B;eJXp49AwT;=%Zpzi<=L!|(KuYR ze$r2(8h+-}5v!%k9Q&p#pM# zPW=3P-)L#ttm0RHZ>M$~P@;fDbOMiL_s-1arr(My5}IjS;FqbxnVJOS`BU0}{_a?O z8yQVDA@8()NE2K)B{x+#K%gpj%3`h7sdZzQg95N#tOv=FUPpGfkJhUOBNMM3!4D61 z8$*b=J3%51k>ubk)WH&cC2)LvjVvT#EYoO_cY?#5v)DwZ7~- zut7iQB_2p>g#T;2v}z21k{A3|g!k5Zvr9&5AdRuuasD_Lmx96ky+hFzVu3OgpFuia z@*8Hp2zndEwj%zZ-Niq7iL$gCHoUbGdQ`jX1IFU+%Hce&BToWB;7b<<+KMfyDWyzk zxz^MA9ZvN0d@vrWU1MS`l-kXZ?Y(s3_N8O^aB-E0@jQgJ!5s4 zp9y6)msv&KfBUEyGhg6;-eD&1cTQYWyIe9$Y;R`s63wXZ_1k=Ar%ZN|RrHfjI>foU znbB+UWcb!N7okB^e~OW4=HRN^P((;7|1x^6=w&HVOW_=NE;l#$0X_TuB!!3%lKe$^U9Tis3x;W7wWQccdgdO9r!?chZ&oEx<-Bs;r z&v)l4VRCzz8bep`-K3WoCr_B{pwMxdQ>K-`nS3ap9TV! zKV@vNv?1z;jBWm5d^B&j+3huhk9GXAbII_}!%?!!pQFynvoYp1nV|ATIZNfh&8B6! zvR=VQC-zM)&mNix=g3={_azbLHG923N|Wneh$a7XJ4q?mkLIm{TS1N5Ht3D$ws~n5 z3VkJn(!Q#1>njH17u?SE>`!LFGLn;Wi%fQQHZnmQb;}!)M($z(Z&7T+KL?b^o8F(+ z95**^Ex{&eTy`WlxyH*K@SUag&_B%l2h;0odC zc21TsO=+E`&2BY$&9xL~Xm)T&_jN4kP#}5V-nOQy54W$zT z$9#@m5t7G5BFH~(aY@~7F@NBo3DMTS5BIJE=ng-ObC7AStu`d!!uCrMeyA|tOCl+> zXPr03CbnwuaI%4A-zkeq2v|S2&#WrvLbXrYr~J?T*TD!qM_QI0nP-+AwiKlqC#7nW za(5dFZc{JyE2o}a_xj};tBITMK$1Ia7P`HZvaAhYZA%7aHvGw52?6r+eu6+8y!m`5 z<~6sYW?YHM&PmVRPa2-L$JYArK_DZOzfd#a&x?Q8nwHz*0Od{vP!#Z-A*=y~>2ut? zY>VR$bJs~E_xeS0I+%TVO+ZyWTn0#p0!jF!u2?Q$hz&EBKi>ngIg40_NgNQno$wXe zTVGsMZ%K|scA?fVM*$Z~5MBTa?Em#jrbhBBu>{_X#0=f~j-~3H$7=>MTfIC#&dgT) z(~4LzAfEN|k&n%btT<)JjaQ?PFQg?H1`oVuFc+RR%#yCb-9cpj*Ma;ZVjqdErW}+gOB&dQq z?b?L4nJyTcfv7wP{rN@OaF^436KUb&NVf+46m$Z1Ufsm_r+=nKA53~7kMG7u- zvR$4H?_O5hwwBj^YlTtuU~1j_{de%$oZbaQcX>8HPQW@ZO=s)J^{x2F{UqQWd^X?i z^0}qid^>dpPA?v_m94Vp^4fq7@Em+L*Y>+NKTewFI(7e@x3HKJ$@jx*2cONOyL@hO zHs4OqdTU{llZwbc?!!NEZ&F~BNwhed|7@W8&koJxZ2o#5)nE58n<|MWzj%9jq?>4d z+@V?DzHfJjgx~eKhuMtR#D6ZYam4LC)}80kJ!W(Im}j%2r~UWaWzvP&oZfh=|F6vk zt#{O`epvkZ@@)P){mK8m+3;LziDnDm{=al@>eK9!ol=1RJI>8!3-!?-7UlJRWZJcA z)HIr7n9Z83I=yi=c7Mzd$jW^)IjV(w=on`6pNTd7c9+>~E$9X-y1>-mERr9?Z2mg& zw*T%jn@1N^mBraSGOaTFbzug_Fq^Guy>{bl9@#LP>CHL4sEZ!MY<{x7ravyuW@|-8 zEJ}zq`{swAtUOC1@Hl33c~wdNVR1H(t~Lqzeqnsevtb|pyY!}Y+w;h0^T-cNW{2^{ z&TK5IxZ`ztWQm_v`~t?I3kHn&1s0}v9J6WBw#wZGP;+hJ_H5`K|FJ>Vi0NhzRMme* z|IUvqlR2K*w1YB#E)8XFWOH-xdGUAmr~PB~x5qJ?pM1M8fUTdbYBr}AY8#e1C9KUx zKl<;&t5&Y%t*HXZ`({m>saAfy@Z)tq^pDfDKQZCH{_#h4^>Um%dGWIk zF5LL~#))@dIdSsrtMNVk`>$_&_4S8eo_PJl$@iW+`{macZr}X+_UE77`11dmJ9mbMHMhdtvtE*;o7SJ^g`x`_$}Nk@mCae*NK%JF_S2 z8)y36>{nmj{^<5+`r%tQZ`}E?(!HSnKk?3*#7$Vt&LxN?%eJAm$T>2 zzIFc9S984Z;myxK_>f1CeDut@-<%$vIdlGrTW7EQ_V#=3!q$b)`kTKPqCfleFD`Z;e*NkT&vmc-@nm=F z>W!EE$xnZJ=DFA2x_i2R_U?`S&%bc# z;@4m4Q+c@b^Uq#>1{J`I)OHcpicbCq*c;oEZOFzBv$)~?a@4S-i%_mOUyVKWKFI~O({@YiEb9c_% z@%FPb&tKa9`qJsEx9-fzuhxIe*W|);@BQ{q=iYqlboSR?-+uDa&))gevqb%wFJJL5 zT>iuF-q3X4*p8aZ7oT6w3$J+bcb{KAe>=SJ_N5E)^iQ?6Z~pPpg|G7GU3~xJ>t~*S z_xYP=U)%nryYS>kKfQGJvy0bWf9~TC?`~}x$ZMCcU48$}6DOZN`OTB3FK&JQ?$i3| zrI&0sp8G6+^n4d?ojZ5-<2OF}r2FMd@BDUgAo@pJ)K71pT^z_aTW_6z|J=K;oZJ5N z!>7CO;a5%^@s4=x($D@l{Ji_{-9NoM{hrVAuP@xX_4!NZ-aL8x&CNrKU%!3+(uF%Q zUAT4T^bMO|_~Q13x97XB?a>?0UrWD#^TsQpbU(dtt6Y93{p`|b=g(ew_tty!uY~l@ zYhPb1f4=*x%NMR(x%J|?SKmDO)zsfEF5quJ_(Yui+duqzoqhe| zPoC&*y>sKG{+ajt7hl}^MR(gh_3G)X{@hz%y}U;J(d7#}vkw=){?niQpZ;+7r)OR~ z{lpdiaj}2)$xG)>oP6%JU*7fIm8b5!-@o$ex!3%)FJC#^U2(s7rMvRp&mymV;^O%=5qf@S`(B_|@g9VV*d-GEmOvCud%K`HRaol;x>^bncV$ zU%i;$y8G#svv0os)^9Fdc>4Bd8sRTqF^2T-z0`emF8pr$cW28BSFY^*E&HIOK0EuW_#=0m7DKhUYyVy&tDbE@ad(C z&*$40uc!0ZFFsqIy1M)o$WOofhqp!f$;pf7-u=@XE8NCFK0G^oaOb6SA3byRr7rbv zXMc0+>gTJcq<-|JuP@qf{S*1mgZXTOv`?V`CZJb=+bN$>q-42ttU(^-b zWjS`1JpDd>bnW8%-HQ=9^=9|c$!DDX;>5|7Urqh>^B=!^^WzJ*o_OMvMBUHd)=_z- zzxeFuuX`~_ICz!Lemgma+fTgt z?wJ>#dGFU3e)-e4?_Roo_4BYg8c0r@oFbZf<4b36|LIp3KK$y<7mdWd_vsU7x?f)2 z9`uiX2ru^^{`u@@m!5j&g;&4&{MPx~U)`2O+rEBzg_Cdm0itJ@iN2PAIA6M7zFc2+ zzAgWaCq6y<_G=$JbLskfFP94!ubqEw{rG(Ri%U2EXFtF2r)ST+_|mnr7oYv( zZ%>{1;;UCb(;A02pL_e|b7#k!=GL9F!>hj!lKi*c{^JvuPQUYuw-(p-nXlg%DQ`Sw zi%{-ddh^%eoA|<87oYia=P%s)%MZ@oIeGfxrLD_Xzj)^C&wu){yYQ-i&0YA#AD)LVw|D+#;l;Aq}KSoYI z_&=U6ao?Z6_pj{okDsl6{z$&E+g$E{`R{MV?wbYfbH(<_0!Ozmi!5;LyMKf%u>ZN- z4+|W|itWt;N4zgTOULn*-R5%WEO5ZN+#?Gd+KTPU0>`m0KT8jA_Ya)~?qx0qXMw|A zu}v&+jQg^t4IJO@A0`Xj^NJma1&&}}CRpIOPvudvz<%cPAhq#>u)tC6%Q6c*5Se^L zEU@pnJVIuPgQ-wed|XaJ;ATXjxz%b2$VSxUUs^u-?%%7I*+M`N&vczjL`i z7P$Wvd$8Wo4<=%-$GH1P#sUYL%RRHeA+6Yh^^P_cc%b^&A+x~2=W;JBaF{FhV7;S! z<#1w!_r79>#{&1bVh3b_L*JJN?HxUYSmAxH*rBk%{jAvjS>R~)gkd0!r84?fHud~myeAQpI-J$PjM@-TbwVfNsI+x>m8z{Bjp zW7wC6*@F+W2lum0<(*%Bees#6(;X`7-`;xW!mICp`qt(0_K^SImqDg6j2Sn3pye#U)|A`=yKA}MG}YY=yLKQxjtl(#BMv6&~?d+WSHGN(MLCI54qRt`ZV}KH2tvM z=QxOMvVP+9nr_(kp&Nqla&~TZD&(}C`c(9)=u+oJ()ZhW@F|zU>E)0(NO*?MMW;`_ zH4W_ZWVbyy-zE3d?76qT{_^%`AI+XTckb-Xo8Nr;>(9Tw0qMdr_sO|&`LgQ^U+U*y zaw(O%e0jcnrD#%N7$EY&=hA7i8gc37%cq%X-ksi4iVrdL!AItux}D|Ay-0lA4y?BR z%oVLU>*u@q`suSGN@ZLBI;6x;6Wh>t^Yx2KJ0446$K&9;5C#odBw<{?e#nLW_H(At zCgfe#&us_om+$wteldwleA2dw8Q`oCHJA`w?n^3KC)O{aOc05L0csJ3 zpgmz9O4{zS)=wPkHL(>ERyg$AUC=58Elf$#mT&bnQM}c4K4-0SPR?GlG8kAOXlb-9 zBB(pHGN~1kxXbdL^4l6vEPh|>*KxeN+1U?Id~-q@AH?r{Nc!)n}x;bizZYWl27!$Yu`O zm|p)=Z_S;Nlx>#0>-1i6*XH%}c?9>m@#WXAeo&X{U}rkH?XGwFj@NmOoasK=aR^BZ z=EVz|X%Z8_C8d-2bDe{zXr^L({VeGW&veGKANm~nQZ#$9fkc{49>3Qy^^$-&ZwIgS zcV1#Dc%A9q{>qC3wH!`tx_PFWw_=815T9v#b;3Gb9TLdKJd)o!u(@YP#ctbK7e~}F(FfBm z^Gs9oJdl*p8J3J2lFp*nJtdK+L#QDa4bO|A1H3C18+6bm|6I;PgedjfB?kw)b_n+v zCAEu_@YHJpqSFl$|C(8dI_gO)81?VS8)c5Wc0IeuhH`fg>`ll%aDlE7=|0hgr8(=EN;qWpJn9$=R~Fbz1zx}#z?t-TJIc7usZ3rM<;bqW)+ zWix^?yd<$l>6oz=LBG?6Nx|5Qim z&VM~^|BWv9t3wm_pXhDBob||i6ZEHgT~^NNmC`Z>SV-DSFR9}E;L*1}6^T@cQc_p^ z6v0&yNJo>-6*j!$6pN~JUz~PtB#;uN6X!;Po;8*fk$ljbB8?bzNem1S{hB%YmEWI&MF*v0dM?y~wz1Wb^^ys+@PlR|7?mqL`0 zC86ZDGVGlspq~#=qCwJ8l6}6p(r%e)p8sCI)jQkLb1wVw^UJ?GKf!)0Bv<$5(5okI zZ8(o^;yhM?GP?j;;UYB}`WzM+6cG=n5Aj78UwR6_e9UvoOzA+(f``pXW82OqTRKj-L ze(|>R=vn%HLAT4lPJ#lHiXR=^p(BbY%lSEoEot;rm#ck;YdFWDG5xuM}hGXiouXkje zNkOU5bB2->^8-H^kCJw~N41{6L&sU{A;Ty6Im5^fnaeSWN!|1P5O$8UmPyLre9;}J zQ&;BBH^j1CmL0ewix&?@J&Mf3b6a>Y(e`N&QZ9 z2^qXDk3j;g%1|A3X|U35=7(s7KnHn{(345vH;AZ#O)nc|peT}#{?7e4=r&KVO>Q@+ z8T$)Nw@x92N30+_v}_uLPN#I<-6-~0h>bfV(|9*QDg$BuMscu|;qmIz(wcp>c(wMh zMi^xnbGLKd&Bwj+j;p(UVDpftEW=%y$xT^U>&zq(xwC-lmxjX?HOcimzgpiIWPyy8 z`6BI2vHp6rdRe5Stlg7fU5#6s+8)2&@1b^6_T29KFl87m?Ss}r(t4O{OsmIP5>7VM z&aJb`<+?@7;+46!*fMaj;x}m|y;L;39^(tv4r`C0y`$`|{+jz{)3;sTE}NKO_jzyL zf+-7ZetLDs$O7JY`tS&Bb3^SY-pj{Vl5bQ^SC8f%uiiZ1DLZO+p4q7BZTf?acMT5n zcRx6hnn!){>>sWjcFOz|GQwR#>^uB1S)8XWcA?W1?_L!??lJZodb3W*wgWhy! zn#$rA4kFE^%%;ZOOBakX+$1h!pQYK<3arxHoZ_c7s!4uY|K7y@iUx<8EBI2U`N<(Y zWuOGv!_wvb27Z#dBkM>yS&{t9ZUaAwN}9M%5dZE<6}GZpgX}U%#>ngfkpUevA7!=@ zYcYY`5y}eKv1x*aG3kaYdU2{FPcVoW9vlOyKUl@cbuCz{d)M7cEYE>Dhol4fZI zZl0K+GB;enV36Uc12~Rpl`3*2#(u!G9Obx|o!9y7qvTkn3F2^xFq%N!zX+Sw*);h!oD?0Ttk@%t&l(iDG#h~h#aoF}6Y}`Pj*&|S$ zDc?4r38I3-LSOPa9=;7|f+%O{I{LXQZA8TtLP1*pN#VR70e*Ai9kuPC8O)j{ixGI=)h?`LM4$6n|0Cm+-TD@3CER} znAD3)Hhx{Ba-0_(z(EGHJ~)_@WRO4@umDQwbWK84a0hDn36NSysN2>ud%`kON$n?I z1`$Cd{Rwdgf=~LI7+%+KO5I>*Q7lXlF6p~I5i&hR7;6w0%2q2SyA~bLy%l@0QfF`S z1fjD9MY8M6uOGN2h|YGp(1}9pbn33x?I{6~O21BW)9ft`q2Qk>gk2`y6UBGBFckjM)@_iqq~41r$*3<;gVp%3LF>e%ka1_}T-S*W_)NV8FG`S{u1l0Xu8p9JtJ4?< zw=H3UgC?N2m$uZNgDGWRpB9am{U61>*@oHIO_R=vM3ZK!4U&n!PPN7Wcc!UUS&5j6 zR(S|n7srsNI8r2?#Ek4?FCj8Yv=vd5bVmml^I-L3NgAy* zqJGfzAugR9^AycG`>#_?vA?K>B9Wtw2?QLW8e|}7!dXgo;erweNYeDoJ^31yzy~dv ziahnOUt*7sTHoo>AA@| z(iTcHh+3tp{;aMM*&Du>WYJY=vCZ|B#-m1@KoSEt`NWoq`PYc1cz=Rum5oFqQz}Tu z>SSg2VS*d=bzR#Q&D0Od#NBywKXU)hrhBMAu%D#g(@%mx*Fg=^skJxtM58=P$(AWw zq`m1L(%iM0Iu{8+T>1$jZH|r)XI_UWds8j~=>%Dqt}YcRuQH{3Om%5oxzw1pw!#tV zm88&nxPm<#9g{dp`w=XrEh1vrGUB)|3h$A4?IhhKhKq%)2Q*2X8&VUBheBdZ%So`j zBv&a=HKi1r9>6blRU%ZD3wT338fvMikl&n z2Qq^saK+@lk_@bc_}@#tpE-#;DY-f$x;!O2+gtQIt~i9H=$_GRo%V-#ev%w>=2PKnjT0m{PmqHp-8bqmX<;lyVwK3w znlv#d3rRCeO*bpJv>n!Eo`r2`Td5;bD2ZQuwLgh4M~rC)yF1AqSx9%V}iC~*)>bke|KQ;5A9-1CmJcGCuiQZ5Qlk3WiGdUl*#<)~J54%RhwwJX7O4%AFiwl910l(6Nik69FiPZU zzr=R>QbKj}CCvW|qLW0_?fW_p7- zU4~6(gVDtyJw{XN%-GfXN%xK(1yLGdsJ1=(AiG31J42X}A|ORXmbB~vZ+ERoq;281 z>7}ZMSx@d3&KvDHX&jJ53(poZ!lIok*|{>|i6M(shZHMO>N#v0^q}L|TXacxotQ&r zgpp`BBk7M~0TRc^VBAv4=5QF@&ENSH|K+vxm|)a1NGQblIHbZ1Js*V}E8Q(o;x2Q>wF=zksFmI5ti=sbM%u zsw)6Djt&~PO17QidUL4!`pvpMAjTrgjEpc+5GO1K<8uys)`CkXG=I?wjXtaC@m*bI zB^w1+pX`h{$PN-qNKA+v29+R5xj9NA={^O+I0ch_*h5aLiQev$>0-DpJl23FGML#V zH3y|?7_j1;Sto=xMX4hs$t@nh<)ckRB#Ae2s_Un_KwFc<%$6u6s?w3hnwds4Etgf+ zkrMZ4#@yU8g^*%{lzfls2aQG`D}LaJlhcD9$X~bt-xax^sFbj@xAU3J52hFQOpNQG z46$xH$*wv$(jd))DM3#n2Fi$*vGbkI14HVr(MZnXSrA*LY%*EvW&W9-3nA8->F(B_N=%3wXVQZujM;Qe(DiMBq+#QN z=l(|Oh$Yb-$h|8~dDrOBpQMzUNy^SVUh1A)W$sJk3l6%bnX#;NMyu+TwVYl%FV1(lD)(9{safcGk`JE5Maejl zge753+m~Ub-AzHVUYe*KbgYOdX`tIYSFxi*#I>7kTWnIgmt@e(vLUGsBC{;aMVxi` zQC$mWHz`Ou{IS{*GKL&U*jO#BA9*!{#c*^skd!9LJ5&0^tWAN_P{pMZD#b1uZUlU+ z$Z1i|oVQrJgg6eAFiPA)V%~tC)poPTIDus`2_TV97pgYok-+*CDVoxp3Z8{floT2m-11`ag5#k|#KjH!ESNt6P7e8?uwaA_37iH#H(BaYB97bF`p3m$HH+ zotzn%J-3E-&f#0`%ginw{f$TKs`f6)HoCW^^$~#4Rj%PSpI(5~AwmDe`ystRd!;*# z)JN@;_Fv3p$}JZk8OnkDUYH`GA#2K==dyObMFfyI8X-#G47u(GDQXf``mfwBY6Kkn z68n05HQa)?N|%jq*FCBHQYCmS%Ws>@yWu7Yym6rIcx! zUQ!0N4vlNP1J@i6ok;Xxzc%@i8ah(YJJnB9x7${V&+?gaIu3-81I=P)bwl?5ZH^|zw*PSWr z%eU@Ji4wWkG##00cDPn)$fGP8Ilfm6|B2B~k*v)6=81Kpwe`C*Nwl;J>*HP9$$fCm zW-i{CV6c&kH%9p9@%C-hzRX@bQ=buzcAa4(B+RN=KkH_nSQb3F2)d(js7!mC$o!V# zD0`w{374sJtg|ltK}SFOs%1bT86>es5LObir%;Nxgd+>F$%}g5;c~PzcuSly!4$C% zn%~c33+ZP#7iI+Cqz)HcUwT^$;}KQW<*r+t3NeQ=j2%UeP07u1qme|i^lLm&*Z;Og zLTF8jZf*g6DSmowL{%GRvQ$@ebk$QrP0v8EDVTL51xG@fgHp_l z;wl;VE2+JhG^QSCb*%bN8`f9bx-)Z3*DDceZM<|*kEcSnwiah-=C?!r>LuSL$|p#q z9@IiT^5~+?Dxke<-eg@`TogN0BbJft%ujAZ#?274E4~(!Nd1f22JuDcDRlIBG6T-* zq<5Hdk7XKho?}|ot-?K!P@c4Yqa2j(?KsNE=$8KPrHf7&kcA}P9>;{E`XyH55GO#PW>dR6)7?FYX29z5)rb%H0DFB^n!#Y{ zs)b}#>E~$F5-z0@b>4h50Udjmr3Kq~wSKNe$8})zWr=9&AJdGoQE8YY6zTem&CsC4 zY@oQq!4bOL^}opNC6BgmfxSq$LkolBW~RdtG%l?kimoZ1-jbDNW62oO-{5?Ygutcm zOM~pIBY=}2c`Y@`%#{JBQcs*K*8qZ;D3eEkWumGt9_l zBx%=F;85)c*@e2hwEfBElQKEmRF9>CZ?Uo+Iu~xvgi(|RlK2PwS{|-=29ymk`T#}(59HG zdOxJ4$^IKmY7q6PLJf7jIw}HO)uTY0r0hCBC4={t<(qGzU9F`p=Ne06CmnVog8ifcm*}SeFJ?cT{g*XNxwvbHRNhsns4OW(gSjHvgWyWfUSuTkP z90E9Pb?8hm>k8J@TV%+~NN}@E@RS@xPtuQXsxY;Ldatf=FOrU&qf8KqO6^d^vd#hK zx2DWFB{vy{LC83hgvjL%QG=aFlwwHLwAZbrtJcB`EbCVSXwQuKdurhvu=@2tvUsflyHYh*}Xz?(`wjcS7O6TG6gsF z9x|MC@CvFz^+4g>l42tR1^qkSZ=kG<`LF)h$V_mO_pryEk>Q zCqO_6V&|JUDZ1Us5X6R^6nHBVXS2XNaJ@vouX>5YOL0XjqxceAUV9iOO;LOox)XRg zHN;aycZ{gRddD+2ke7KM4;y7sZ0&q%Ap#WL(c6jLdV30vY}-<6-rJe z_CZE5cWx*7-V+4Yjc1$SspMpTbvpGrJ60eF=reFm>05M6O5cDJQ(7DO0lG}2%*dwA zo#m?Xo*eZZF>>lB3CLDPO&*B&nag~QXkw?Ph*owAl4!*ny0LwlaPgpV^u$nmvnnTj zJnM34eW%QQ%a>$4)adN`mnbwL=w;T8n;h;s$(oO6ooM2OPZ2F0nDy~aNXR|K@y5$$ zWAf>yyJllv5fCBh$KFE5ATLl?8pvDZL}2lMiWiQg!~rLTEaX;vYVL(7MLB14YDsxU z;Aaf+p_1T)*U54E<1*!9qr{brHC^Jqw}T?_Kms!wSH@7CS(z;9Rsk(R3p@i5gpApU zJK56KAjw3+JFp@KKS?6BOnN8-8j=+ROc6OWQu0&DYPU~GIsmgEO)^@hNW?#ri5%IT zY_md;X`>AdXjUOQ$oV0ZRSa55MA!v|sA;kQ6e6t4z_94j$`!nVm=K{$%#KDvKqGJ! zlNmp;SF0rYRA*(-$?AY>O7SX@Ry|nsQT&POGcg+cyW}MIDB^j;tVGn4W!x!PXSye^vd(# zUTG&xLF=eYt3;mgB@uSa>D~?ij(&)8DR+6vZ7Oh3WH9Ei`W8) zw&&E-P)0ohyl5@MSarvKlDD0VEzH&;$C%i;7Kcih8T+7{l1OzJYE4R_XuO-hMBlNr z3jCYdDhr_@>_z1telj$o0O|ynzPIA+S;aT$EactRS9Yn!)?ui`XlWLd-=h4IYN46& zDH7*N_zSp}&E|~+JQK#V>^L%$h{+FBbQov{(~dx?$jAcX2xlJ{8oF+X?qS`SD$U~I zR~*cSN^M0RclEJQBTC z=Azo-Slu~uS`^JdDzS`}lC5GNxSvongA6OA67`@!ks<<5n2H=&y$IMB(oVc3jfA}d zSIq#Ev?*jx4(v;*B&4X8>wvSI)&yQPVWCdvAufXeopy4Qf%KoKmVZzOyQ zj5?XgGl?4UgBfJF#>c^MhvP5Tao|g7+tR$L@sjyJCt|tC{OU;4mSmF%9}rl5bzt-) z+h<3p!4WE-f%HWJ_AsyC*Af$C;a=?t0W@n+)+fmd)j9x}kAx($U5#&83i1K88F>g@ z3nhE2d+DXS=b4T({*fLlZb)Orm4t%K07`CUuqGn!r0!@qRXpN;)JYXaU z3^W)LMfKMPPKVS!R(_^0lfX(+@5mK7lSm4TK>{5KK(iF^_z_DT(iR!JiRB`_%} zsi;sUts&ZpF^`|e@f7RLP{>aqw^%>vs!$@wl9Uw?5?Ntl76Z6fBm)#HT>-fw4OS+t zYuH3&PfQSW5dCSkv@XOHDq1u%i1bdfB}>hhw3Jl|LuSi?}gKun72SqB*1D(vCa3BMly}!>c;8a_1C41_3M$IV(Of z-;cv%Vqc<$n_!2Pe-3sv_@%mgFjdHa!@20HyU#?2c#qT%PEcR9j7}!g;DV*(C9+xu z8SzMX3ATEXart?0jY=gN2)$%-ImKWAyJ^6Kq?p0~R3JX_mh1vaRx9DLD^ufj-m-Z^ zyE4fTOggSo`Et9R3DZgxfLIVKQIK{`BBqG}ZuwdnGQy15)eF>3LVKz9KM6w}ZZM=x zMOF(k>2z_Fu6{@-bs$;G5KnTwV=(xGkgwOEgGsvXXkSRqg^SrYl=whY}TC z>=sleG11B}nol56Qni*;-AEPd0c3xbR+sr{M67zV* z3{HzeApNG=1&~NP(FAqNc2#c6FpwsvFIklsG~#9)WW<=sb0vv^krtB5y+slu5Khz+ zk>Ab=2Yi84oLAr`n6j_t5y{7R8M*Z+z9DuyS6b182!arc(lu)VA*occo2$_1IaC5t zeML=s1#b|bw6U#1y;k2lxzEQ$6Xs}X0jlICB3M?RNm@NcD z{Fr2Ey~!@vGLT9@ZTj5wl0uK?HS61L{W96I_&1^`e-uf8Vjs)hy}ua#%&<7mUU!%U$%kIRbKEaAM;q5_HNBKfIp z#9a5JUL}V7u;E9 zY_&lgMTAnk$Z&B(yNw-Q`0-Go3hB2p8w`SAys#-4)aBa*1w=rnh%y5}IT8`22~~-{ zV=0G@JQFfns=3XrjyFm&3X9d!^GM`qr?XxB%oICrXe0ya#^h83HQp|M6q!5%QQ8`t zt_$>oZ{8JP*h@nv0?`&cap2&p-em+_4cyAbC;|%;h&IRAv zq6$SEDbC(PIyfKzp-T_BQxWf@ISM3&rLF`EgK)SpV9|g~RGZnKNaV&0z*Ma=!iuSK zUv*BjNm6eNl)w7pqSZ^`T@aV(C)_r`WsB)B*F%+9B;o_1YUZX{zX=m8n!t)JnV5ho z_an}>MCpJfff?FVE4Cp3F!aqd8z|r)0D`N&3UiOYoO+W;@(rh(luYz;aaysAZ4&l6 zYh2*kYq0ndNdnodsOnGxIYE~|N0QDgJHE`fxFpO@lx+!n6pB0v|C}c=gq$i3GoqaO z>U5%pAp`x+VWFZF=wW0?JCtOiQVT~^nomhPr3N5$r5b_FgyxDO2H_2F0UsDe#uT!r zVrmpXlaz_k(}5UfCxNQjP_LG%8X(J?Bw93M8!^z1Qf&vj_f39c^!NDYsD7(@KS>Lu zC+OI*vu6~Z5cX?lmJudJXn)?(f5+@1%q>lhTq()68X|T zg8AAi$a~(lS}#%tovBd-XJRlk_EO^yi6Wy&2_1^S4Dj7lFvbnmEVU4Yv^6SQyXv$< z@uL!W0f$kil_Dw+f^-fBpiZ|9aT3y|uNo%zpw;c#yt$$R_M5Kan*NJ)=}8+5UDFKh zQq{Isoor_W@qUT@3C6fx_>{;2td1ZAAY^ zi5<{QRt@ejSSkcuZ}OJb1te)M7~HO^=7VKE1>kB-tzEFV$)p?Wm_UEh8IWF|%nIfn zmNqO=gmU|-ihNVRKM1lRI;CGo53N^SqF@jh>jFAN1L4=LY^Y;CFS!z^^y_ys zxeoz`f5+tZM{ROfbb75akSXR}0Bjs+7>WvgHMu)7%WL_Bx0*qS$o@f#m$E`g1r5Se z)>fM7=?%iY(TsAOv;)bw&2aCXZadTr-MyH+ z@P}3zMJZRCTtbi10&o64-0QUWanX$%UX6o|`fT zIVhfGVdrL3saIXHsy_v66AR zqMBB;aX6xjR^NkW5ULzQt9BS3RY``kdIDYaAE_a9Od?`PqwP=ihNN7^L(m38K+<9H zkQ5y4DD+wcN?)u%6%vAga3sNpY?GqUS%+Kl1-G2N$MAs-=+z8LP8|(5u#!xLlUBG* zZZJ=yu(HeSEq&T3h=g`zua(r)6wA3RmKNFBX@Mvk%1DbeS@_3L-REABjGrD+qdpdr!2x)M|xB9>s`Ta+CVPguKZXQ7gC?INlz91Y1JlR03^R=DZzY(dhm(f&5 z0|`iIejc@<&jU#DmM;-oZxjkH;l~>N=}h;wgNoCyk;st2!`8PiE#FU2h~q5D+!6=s zmki>8Mo3nOV>FauWsCjXsU$dHTxV?FXo?P(win+(r64Q7H$B_>v{1=E457LrZ7_H> zNcFN55J)6)#r8FZbhF*4*(xk5;8vxv>`OCm+nmJVwR&hI8N8KZ;P058wr3=z`7`U6 z5-x}V&84*AkTF9`g{n;Aza)A^O?W>}8-9@Wnrbo$kTwI3ATt+T`&8-5Tu?W*srWUc z)d>+fqr|J7UAw+kI(ODYr!6pMe@XQ=614K6B#j9RgtGdYK?&M~d>R4j77}9}Rz?oZul7R&|!s#YVAfAq{5n;knxG`8z z79jzsEDjH|44kFpN3+(3$`fFLnH^eMyc!|FNbtT?@2ZI`OYbh$v!xPg?K&bECaWO< zmUKLtmBcWal2s^Y6Q3YEcC`UWla5@};M&zmQHzy`@r5x91Q`o-D~&G52B}($Lx6Q- zF3v!w3*uBQVWN<3Ld|KtoLv?8C{Hx=2;QQY;6!CZD4`{nrN4l@7ywhCRUu`6iAX(N z9?h68EpW?fweA*l#sHtdQ|dJjHNy>BgeuqUzU%-I>09fM(e;FSH0m`#Xr;L1c=9Ca zkzC-rc6(Fe1Pv?W>eVdakJtjhvgsSFT}x4j^8}-TBzSaxZ`D|a>dNQpg-M_TzbbkO zELtcWLQ}{F2-K?jlDVP{#E3JMNz3jR`V^qHkjsOvV>Dy|;mHSosCo#|eZZb*89fl- zwDY95nHK^L3QLPyucc>YfMGy*2kE$q8|(T}uqA7HQ6{+TEc@Ctq^hjZhjTD24E$GO ztcWU6v%D;Mqcey?Oz%>SZGJ)$77#$uz+%t~33E9rCc133Y*J#Gv?1P`t4F(lPdggK zaF4b|>6i!Gta`yvKz&@ey-iyqGl;9#8i!Ox7S9QO1zWy?fv6346gt&>lbE-y`plYS z>dj!Lkt`GEt}@hWlKyamw>%O2X+v?dsh>P;qZ6$Gr-FwQbzK)cbr>}t@7a$v?+<#n9w za`{F})|F!A(K{=V-C4NDvajlbr5qZ_dT2H=wciNwv>|KiwwT$I7kvJs8u?J++M^%TaAeuB!R2C%AljQfiHNOCAO zZwXEh6D0V!xxMM8Xo$}EUna;5DxKm3pa^4}tqrI4-m}O9UXca1n$K&6#O>zmIgD0> zJIKX3vQ>ta=5s^s zt^fh4RzxHNiz0Kacg-;;Q(7Mk6#k@{d3(v@4VIeN-lMydQUS?2-APtn(3?~<)V`V} zPAUuuEObe%DVpF(Bx-`eqO1~B*9)=7NWi_Nx!`55*rL_nOZXm;C!8#kBxKnmd<^e6 zwrtQP+oR-hv;#nzzJ@~QxlplKsscyB<3&J2}WUC>d0O8t!WN~YDX z_61K5(HdWg#A$3;q4~8r5~*3xdqAx&Ed*c)o!&08iJe-ZuyTkFATW+Q0F3K2TJeI; zkI;~sR3?UEZcln`I;Pa6$tlqbFk1B@P>OzHVL)vl%-ms8gj$-hvb<9yYuhwMgxVbO z;~>0e>ti7?o=7iv0F;2N&nN&zOSV0DqSj@4Vr7DXB?OB%5}Tb^-qpjhkMp##6k;PV z&89{&o9N481DS6YnzA2S-)5S+OP8pvn@JaeF9yQ^lb}Qc10fF;SJSR-olk+8PgXo3 zXwczir#wC|kZ}sW6q+5&__3sTLZKDVBw)iiP?voL+yaiAqze3G7P%1#keNh5fN?7z z`2x^wO)yetCVT+Gjj=}$LN^?IC|%hNXWW^EX(g^qDq4gRL>=0IS|c6G9!Sl@u?Uiw zsFDU`=~<{=XzYMIgHA%=l^}&k0@%lRyy$aiwhRCeGXC*anUL_-zzi%639_N=)-C-& z^QtRIklGD~dLp%C8w8_BLlUI0ZUuvR!v{~jBljUw*I9|&FP13+WC;sSvx8)Iye@8B zXW}GHGjW7q3qzAj!3s}lfk3GaeUHf?h`V&AK3K3sR_8XASagOqv=dkd>(D}wpf8V` zKTCwd&{a3!&I>0aQ6W(!pX63cD9xbj%C;Lzy^T>*<%|Eh5z)(N;f zhc&UjLR|tmii6qq3jkz5o4=Yoa3975LhZ8wP1@IC&)UQ*WqGz_n(QyjL`X)es@#9F1 zbQaC#ywK0POBSA~H289n;-HAt?o9Z)aLx3nPcR!zih*RpeOS@JD9!+DF&I2Kd~J!V z)W${}K`!-)kCsux zw9G}1i7&~m6%hYhe#|;9&moe0O$Vn=1%uNg9gnQ(=6|DAb60RID<9sEdmO$o; zrK`-0Bgxr`g#G9+wN7(>lDk)_uI}p$U>eu%%+6{V9hLIY`m({B#AOgUloQ%{S*HTV zf{d}=08xECFJ|j&5eNOD-$)gSF;Tf}M3^^NQZWmx*0_l&^ z)QfLB2)K7Vs(COaJe`ObPt}-Dj z!}0bch|?y3qUhA+k@1MofKH5`y;1=*lx*Y{6Ye{MrT{i5St%hbOj*!e^A;DmO@ zudPr8bWt#V@yAd5c+75IP97aJ;5dyX`j;Jc0btpsvh^>K)^j2?0*p5ahp^5MWpfMdND$=hp zx3dh8QUm5t;d3NzY06SPB(MV!l{W5`V_7dTiWIS?Es1!I=KQ`6IKtrRcUJoE8ccQ3 zY*Om14+|_v3TZU#os6@DruUKTG^@(B4sUBz)}9?B78^DNZ<$P!Qj4$>ky=`E^&c`0=*Le@`zkWp1%v@w7PssL~Df;Nnpr$#*o>nLK4F znj;9=M#q#XEdwgl4AL$M=Xx{7r>JBqQV|4RckVYFAF`+@_u_zM>%=ih<)qcoUB^&O z{e~UIZAg5ADhoiNkXbj7OiX>~8J56IPcF5m%qj9xPh4um<8Imp6Hqnz*p7IE1g@#QxTOBum$~fvW ze#qJ60xXlvfGj@4tq@J3XEXV0$WKUc6B4v;VVvFyKzeNU+Ien`rga&{R`BL*L&aUAYF&n- zU7V5L*;7shFaycFK#|a|)>r$I9**QSCu=8?+Y!YfMoF`?Qp!y0ERg3Y#|qh` z0Yx$-qYU-cz|~3+(iL>D!1)@k5PqE$r&MJ*V7n|S#+SgF4%{C$7 zhW^AQjz+hYT^VG>u>{P zSOJkK)@^at)Gk@K1Y-i4@zEX$fEH#~$QH(7N;Rg@P%P&LtBp+cYnF0IpCCyNU{M&8 z6ASi%zG8l`ggw~FgC}jN>GaDe=55wNBV91ti>S`7m1Ve`r%u2c4abdIR`A`#?VWjY zC*IVXzE5TtaN=%iK%}%gvX3kwv<`ZGYM6&)99n|C0Ya}Y5}iQkFVd5=VGSqN7aJOy zQ&Eae&&G*uJAYz|YT^t`QLPEngi2pLF@ikJy3m;G4)}u&=j7(T?tnY1rYLx3m=;(a z*6z?#olTvy^ma)U3!FsDUDdr)h}oGVf0gP~B)gg|Gp|z#_2X#mr03CHXogj2@iUM; zNfxU#g$n}LtRxuIFju8GIa}NE`O;d?cHrl+ObtgwnurIp$Xi?aWGZUuz?B|QEO}2h zHbvcv?P90xq=rD3IP2_AL_#M9Aj-Z4P7F2?guF#7wTCv@kntp zm8g+ob@Ez$Ze~)+pw&p@NjN4kcnv-Q&2H!Nr*Sn9PqGzkNYOx3@m`9@eWU8>o|D-m ztbBac49PI$EL+YS27uLe5^tKBrB0Vb&sA2f>lG5(s9RAH-b-&%c;Y<F5+k5bm_C!@s+Wa&P;UJw2M%1M56cren#bc*iu{|Ux{d9(}X#cWq=wn9vj-2d* zppIzhtfh`#R)-AA8Hn;YC9EwGR9$b0sUQ#69fSFSe51c7iXW2)IiDC!ZEFh+2EPVw zy>*Sod+MYyqbwB&8taZ&fLQUyAEZsxLJRAtX;hdnv;!F^;OK2bsD278twEL;jgAAE zCI;GOG;T1A{a%AGUn8YK^gq>+G7P4h<_00TxVI`e^SV(SL|r+(0Ic0kg7rDCGSPcy zfg2M4@%{rUNrQn?AoMwF(@kQ1eNS(-U-X8R#a?&KZPT@YLNh;9>(DwklQ2s}lr@5{ zaEJjhR2Uato z$#|iwqxBd<7SV5^usFG$fT_}(2k){SsV|o={3D>M395UwC%2oeK78U6LZGfL#$Pz|?Eat>{H^# zs`w^ADIV2mP-$qkQ>i*DuK{y3Rh5ry6Pg9nUd_Hn5R9yL5Kxh+$UU0EkxyOqJ*#4} zY!9kkQk$BF3&d;1y{$%oL8?YeA1Ss-eJy4u0o<&zY;pt*k&C7`8VmPQKSVsZN=J&W zs6ygCY_i5MAOMve>;OVlMHr3|1VFUJwSq5+kX{qTiSy@iIR+yE0~s~a)Wk~T>E%qk zKDCeWBT%IsX0`bWGVq$TvE%+UfnD>7L6*(gZe3_%HGf+E2DcvFe>It|#guTyzDB$r9ZvmF3@IzhyRnOi}SBLcnUJpb4~!CIycM+4V*T5R~vA7yg2Ly44a)sw*sJAt^g`5Uz>$t*zPorxa>2uy7wxFSib|BJgSy73e>U3+@9RjDwLbif1Z#%g^`jdebrKw+EtA z_EtZGM#H|*_ONT|3Ur$TFpHX9fY_^&MN+##+#0?Na+`z^%5IXOdR48jkitSmZ82F& zB0u0w;KWtt{# z;|!q>r*$!`A6ujHpg=^!t!EdR7p-#)wxaKvdBL4;>8>>OW2I?@N#qs1r=K==f)&tG z_X-$%Gm&d1yCZf`S&jGz70J{E`$=neZ@YQbLmQkSHTD#c6f>;Mr%xq&1P{zWSxX=rkR#)@R6xGDOtc`nOfA{966&Boo+h5BSc4)`kcG=G4gkPv9 zg=?TeBK1u*rGNFL^w<;uv{F{3Ay6wmac2N@N3m2OnKxO-cMwKEFDy;Ad8?2H zJg)kC@UPIg42*mW65;gFzGwvG@ zwzEXqf!4kDHLl2Dj01$SrQ%`rgF47=Cqpzqv2&yr)40_4qh$<^FHLi?Ev9HlSM0PN zz7wa#`ZepL1GaGW6K4x97HTjXqxjLpi19d5&uRIjy*2fTRX-+G6O@tO+Vb+TLLnu9 zWb}Fr%|HS=*<^RDr97a?<1!Toq~?*&gJJfQ2?`Ih^&B$*0?f|AShw53p=nnB@POH} zWB6*i8kGvPY9?28=u8_0hEmF+wee$9cBW?&SdX6B$A0Z-iN*qbCG07v4KdAppc zHKO7ivL>)n^HpT?cJbHdO0!mjmh{&THJg*xg4kibBQBArY1UbPf=bF*A|Aw=1gd}RYL;VhJqndSZ%QZUABG$eGLJEN`&1-=(MH3)2`e);373U*Sx(wMOkz_ zwAxN^BY~Th>P^!5q#u7DlK zOI%Mx#rTOv$Ta!@&dk&jL%Af7QLN2)6Um5ns*6i?eNoXy9yqQQPG%L#G+c@pfi;8K z0AwhZ9DpeERZk|h0n{0p1eK;>XKL4RvNyGzUL@JB_j*luAY;Jwa$u*_c%t>_pcw-% zXCj`{JOJb3tQoQiZo2R~1N}3;n4p>9-U%v-nK6^JX*HMyv<$6QaU;7LyaR10aA_i9 z23u^P0?E+Oz3r+L(Du+SGTM==yVemcr%nZ&-6nF#8@9XL_MaMKd>FDxk=M;jf?tC? zXlKRC>kPR{bJNPHXrTmj7Fc@|ec%Q*oI=;s;*y*#z^i;s_yNL)rY;oBj!qKN z5;&QlHd(}#NJ+xu;t5{Hx&T(F3rLD(@EZo?U|n%gdhBB?KmImk7e>i8ZWe1b53HQ8 zo(+Te5IxRv(6z? zR2`~}Aw|}@hlH<1r=OlW;%8S+NV1SqAYY!y^|s|&Pzqt4=)%U3+Is0?|@0R zntx+HPCpoH@X=sExe;XtgUo^Fl$xM12r|3d^R#$d5k{+8^Wk>wrCJ^D3zf4L6Q8gG z9VG^~iRg`Ep2~_5C?fD($oT4A^DM6rIV#C$V5xgDxG^OSB#(f*MdN^ka&A!1T(II1 zvqqe==8(Cz^rV_4db{ldT^H&W70ZsryIF9oVt&)L3mzHMJISR<<~9Hba)y6eLK(<1 z(0QBS9q?#&hC19&l8{E=tY!~%V^9FriPjCRb)q#p3yerBH14V6&QqH~H`Zv-%#M2| zOj=nUjwTjtAs|1wT^KKv`}D=qE~kjNgAzcO1CZ^*#DLc*Cd~D9nssA+f@a0!T_5h6 z<-9iQ2|L~5P~Zz2cKw(Yuwtf1CJEpct-|sYjrKB86l92r-nBdm48u&kic*={9*xU6 zfWI-$sk2r!15H0P9PSVthPV(zix40*0CXk0(W3q6BAp?BXpuVCBsK{omf((C%^8e$ zB(DXZQG1WXC*g1DR>LF()$RayYRKLc(g~vVy_zCg*(^s}M;Z}kN2<|OUX!#Z^Nkru zz2A^VwJw|{c8eDD6i)-y@BL)nr8BTgv><>zlbagy1j(95WrAq^oUaqD9h>zbuZzmP z<#M|hP6ZE+0fk)P0$83iF%>)$mi$=}5vT{oF-%kwRPjnh&J^8anwTIT#*^hpwXx}s z(SZ7k*}aw^JKoCaA1W9-cjMS*RW?tQ=QbvmGPhc#A)vswBa%rO=*3-)e(8 z^mZ__eD&B;asyVSW&&10D_yZ7Mx5xyQm6sdBGZmmn5~42+!^4;z=fGj?%4`Wb#&6h zlIpNbU!ci1S)Lun810jKKRF(qM*zNnFm=uk8)gRP3=t&y54mo7jUqKD9&Z@{k7aVH zCuk-%ZIWt?Ue=6^sS(k#qO&|?*lT)jw0WbT+L4Nr6IhP_M1_$~oH4 zS>sqXzyb1Eg0*Ww28~DM}!V0hGwDTJC>s@F-5*4djq>nMh108U>~O{Oi=TS zPp!XPvOAbcV~?lO*>?Aa^^Uly=&t8a5UZr)@kmyQcG`G!X@+wMS7N4tUN;UF=}_BY zx*2CW5SXf6KiijNN@YZ?b$&U2!a8*o0vI{e+`K@AtTe6;U|mMs$#6!zh^>Gf=yRo^ zGas&Cn&!>W`FvNbWgxI^zS2mW2PhMO6H9ySwyh&%vJ&N?RR3AA9+}qavVu3#k_{JJ z4};XY14>3oK_D<|MkozWoxw8#zTqndi8dk3;dswidj{MNOK-`wG=rw#wKKfE35<3cAA_-nXPh>|=;frp)=7-k+|8v+2oIYO=faz`_m4eZ*bwTeeFNSZ*{#U0X?O+-@Oda3A$-?1U~n|% z!57v5ya7hESJCR)fZ)vw??DbVCaIUmfQt_=MeP#hNe|>wwVe9S!mt51d?qr`w?xs;8LP{XbD)X_C+>{{ z+nWXwdt(j2uu$z0qceUv&x!UMPk|8Wd zyE1`aF5Mi26p*IQ;30OJHXvyNV6ljniVeJt4?bEc9otrA7Jg?c(+!fJsS#h1vPt*n zWT{d}KYHz1XI^gimnlZeyQODGU3_8#_5QEBH6lS zdfmsCDTLUjoTEw+x+UUaCzTqvUDYX({)2Cg55#KZz-M6YgKVl$VkN9@Olp(Gkmzfme21`+A)nsBhU+qVr`(pX%gWmnTIi7b-lkAjeE91Ud8!HUEO*YcXI6If?yv9$+{$+=7~(GW@S zsW%~zv>iDty@@L+3m)8{9#(3u26CuaKQen}^*SLoOYeZ*VvTlr;Ugp6&FzYUdT$M7 z#W_dZ+(uhQl#67@P{G9w5{D3TV=dE-GrbOQ43RV$D-p=a68P2_5Cu8X79|Nt=DMpDhP8x6X&wxdLQd2_p^9`O$(Rj#S3p>z#E%B)DnSfukjcs~PMl zG=v%T7oM)wcwta~n%x({qa%+jR233EPuF^_C$z7DszAVVT)|UBNQn6|a!kvaVI@-M zJG7ah9+QaREgv-2AU#RK2Br+mxh>+UT24`DrAKVZzJ@%r8o-8&Z1I`drwSN<1`oji0MV%!~dNrL#(%nrew6k^su8HV=_QlEyNXdOgWqrJldediPlC z0c#gXXD(nCja^#uexhk+m89Nu5kdsD46-A?VrDT|VzxW@<-Q{*IatPK(}asLJ}^M_ z$|Abz;M~~SwY1nv?ICa4qA6K~>;@t-jXi6LWg>1pc0u!l<)iLg^yVJgX{cB&0l@HE zA9GU#Q8Q4xNYfnW7D$Ms2^!28hia{$JmRUOc*Z19PbkV7U7~lIE<%0S+@v%0nJ6RZ zF5bFSqKkC;NpiI)+CFA&9Sp%Z*5XOCvYY`Gjf%M4>wwe^t(_A33U1EU?WwQ1Mug3Y zS>un< z1TAESb)!t{>0=zCMK$p-S94ag%vD-hKM7I{sH6c#qLmqT_#cvMO-Oq1(FmJ^R$S{c zNHVr|r8}n|S1V1>z1Fc9Bp}wl7QU-Dnq{WbC1oJNY@q2$RTmTCN-~j*L%h_V0t6;a zwsM6vwz_ATpSBt1>~b`g>nt;hE|Q_@4p>@4?S;opO+Xh&s7%37Zs28j{~SuK-Ge}U zSe~R*wAM<>+|ShZ>>M@CpU7B2uu&x4lpc!XLUs5Uy0O)ukgY#h0kyZzu2fi9i)A+> z6hPUQH$A*-+8#}etT+{;5N*#}3OYsnv5i`bjX5*QK4+ajGcB^f(QAK&_*)2N;XQqIZJ!9C)w+kBy$DR91RFw{2>qjG`T0Fqzp zsXnT(hUi$QWricrVYu*Wi?&hEr4dl`O$HNG*4$fLXu6lp8_e)fou$~8R#7sL#NfxJ zvF7qkvA}R74Va8c&1}fcFJGMrj(7By0h(LSmGReL6t=^TR_H(mh~)eF@eO2)GD~#0ed&w@}4@UYDtdRwj7bANo%Eho^cmfLhDKhkA7+yvmMp2Lon&kr~tok zb62HkqUGX{S{sK3;hmeKvi8L!FAR%Rs!Po%qR?NPUYg6(WU0Lv3|2fpzM-Vj&XNnN zGwZ-vmtY;DLS(o!M=XFGtMMM&wFTKn@#NE;HjXW_=U>P}ZH^Jdabodj=E2zvv}GV_re#sbaF zQ41PMPYw@_hN@xM(Y7$wu71^-E1@}nBwBm9b3UxfK3Au73sX=>%~e8aRtp?N!-);K zHS~(0Q-HBhTkFkg-nki!%}sAKRC$BnPA=T6)-5Me zmZ9KBJviuz_H|B4L-vhfrX?bjWQV@0eN*!Zq)$BFP;!#IU;$i4p z4fXueN8>73CYlx@*S*j6vxZWcj>HhBi_t@Wn2O`7Fh{-$Iq8jwmBhL#NaAGA7qRJ)$eAFyH%<)1g|mwHGh4Nc29ltN4KvW0+HFrdL#6)k+uDf-Nle5hY7NkcoAeQsqF*en@?@5N+; z`v?>WZNGz>###=0UYDyuN}DR2uUK94j40F7(E;QGZL@CdW^HHBteY zI*K%hVwERhO^T!Cv*BX+GtXw}m!{qM8#WYfd81+)%8*3)V5msRG9>9O2cL{gM6qH^ z=_JfqLz2vgKzXtJjxA}vf2$LO=2V2YmqCOQ4O&`j+&yknEpp11Ib5m(l6sP>i(=BU zQ(tNS&VHcb$Mb_C;MrGD5rtN%qpGGOxMppNTRIpkW+Jub!>CLyR-_u|D1P+f4k@IX z3G(9vT7Y*I3OF_-S4PriB@@M|zI%|gIYGMLjYT8?6K1B4Gm) zl_Pw)%(_}n*DRr5jdH(1FEgxl-8YXh#SKwD>ofqjX*-XhG_Q=(Mwu%XoC9K4_Ck}jkKUns? zu`=FSu{_GMdqOOJfg`__Iqhpfq9>3jFdxfVyhQy~hgrej4n-!W2Jca{(kniE3w`1P z%}YRZP$?;T-Xl9WRNQl!NV6#22$wB>kik-K%kfyVvdC-FVKtWu0k?_!%&|{xB%=!{ z)$d!U=2VP@*(cio!9bhElDfxc5FJ@bgt_M;6JjuQcGi`)xPLS{ZOPJdSFMJ~p}dE6 z#B|M2ymhU8_UAT5vA0;4j*gDT#pYr7@gi*n(5sqwy-}cCBx0zK24?POt;s!z4GID$&3G=Z-s|soAN-D8JiAA z;Iwqe$e9QgnTzZ1wRA{|{4B~uYm=7gazb{XLqbJ(K%~eu`D*plcR+`v0tBwns0uS> zep)!B8A)PgMC5T+4fmN1i5AiW8CQXEyKG3rU>{KJwiP&BoBO^4H01XB03;4IRd-@T z>N89;8RscfuE`M%$+xtyf?8E}f0q^wS%<_T`8XGT_-@FM_V#Ago1M)845{UP9yc0IlmsS0nMHXiPG+a z!%=8G=-6-wL^94+cgoSrY1sh@$*{Hc5**|*3{`FRJ0U{qQnVdMD$P=90z3gBb;~e) z*s>4aaCibj>bcoCaI7}fRKWoVd3}Gj48}O0P5uHr$l_1WJV*9!4~Sq&)KLq4i*rPl z)6eN_>#1n6gXf{={{eRk^;qZNjS*+!l;Q(|8~x<*%yJ}4^B{>}RA#juXi<(Fhd!u_ zm=$y?fUg6|(M(3ZC9@m%cxge7GLgs_vK4c|o1Hk0W<)&@O(#0r{NAjIif+5_w zl&3mT96Kg0#Ia{|VmOw&8C)RVC(sGu=uahaewA5!aC;&+QtNDi0=zrPoV(vW037v> zSHNAv=p{1^z)|0dp@Z&g%%vvq#BcO(uFV~tG-cg7;2Sk^sF9`#)cFNxHjzdk8$Z&| zbM8ToKIx6+mT~lq0bdbEW=cg4mBj>EJ;2rPdqypj(QkiXTSqL>Lo${LL?VcS@|>oQ z5pfCC{M4y%I#7H8*QhZaP^BaHw3f5M6RuH4 zARQ@R8FE3e9cYccxp9Iu_DmX9W5?)>YV^C&$)K8nFD46&k?I@%M zqQ= z;oVo_tUjR8M=>qn9K$PhcIGo`E>IwpDO>S7K21HJQwU5W3AXg8c%M9H3U|1C!M+$- zGQMXFxzA>_jW79LNjj5Z?iewr5xks`#K_~;c-y$7HA)^~VFe34G_8!l8BBg~5z88U zg2CC16TxuUfMWf4%V=grwS`mCm^Ut`BU&m!m|uNZWS@QRnT& zflYUm0@Xa;7#poo_i_~0qan)DyFA;d4dObl;8{kGPby|It(&MkCD)zkOsL3l8MzHg zK88Mg(5_9(ACel=O@1X&9`&wMP;+iCpui}i_%4*fy7Uo@6w%?Ss6ZW*i0>LDWK7k& z7{}qp^tDMs%>}ilk`TIaOHaL~Z9bVtrip{>V9Z@Csop5+x-RbRI<}n&#cR);>PE-3 zbf(kO7kf5%aodL=)y-KDC8t%(M z1;v+aM%KBcB$jM%j~5C&y|N*gJdHBZ|a&y`as6I-{DMMZ?ZWAfVfKR7CKp)q%C-ntFV0-0>u z8vECZP#9%9T>)*n$+*z$kuaC$cC|MRUh9KCX;OdC;G64^6d`juAWJQf$JEx6gW2m z88h1Ym3kONGa-vUi$^@C(#v+BAQF&q^khrJFo#;Mc!D6Bk!?V!FX#Ddg7LsWR8S_S zL!L^D%SqZ+(R!A$S00DZRmfEwj&cFpwAFjYnqOhmtehNkzhZb0)1BuKc_@zGn zP|^29prZo)qQ-TgA95gwCW5_mXb4qgcn?36iJAcP2c@+1#6Ij8wb-|w(Rt`#G65`P zqrdb-v}B?@*FwElPt;3~PUb=){iRc~=NSO|EDk1hUR38|w*8vXdKl^2?cZ`^(vJ4l zs59Cz0>&ei)=*_b_Y^*TckN(Fo_HbC3FfDL89dMhUz>_1^oa_HdOC;CD{W6>?BNAb zzSiNs(?5!C@&O0b4)+c*$c(ZLx>eAzkmug-*Z@`Y5=D%z|J&S+Judq95e13aMMc}E zV*dph0cl}RL^N-Q&0c_cK{zy}-aD2FMK9!|q`%aH4m(Dj0pBrdeI>;MblDZ0CuN=wzBKPlO3FJa%7*W=A2jS+%ZbVGn@r3L=d@(s1; z;$W68kvSEU6W&lpR3k0a^@bkZ(D!ao&=CHa%R4{74P_!)QA9So%9tM6hT7iYUdi=s zO7};!p=@ZTq&UoF?oVJtSx{eweo7k3uN=6BGD$ooN!O|V$`Na*eWZm?EBU@qVe<#7 zA>zK|+@pXhOF84+BRsTuq+5el&awL6Bjf|6X&UgT8{l-S`zzeuKre7|O@K4e5FDYR zm6^lYsSh;tegA;!K43pZCkLLP_WY<*F?m(|+TaX(CJoK7XLN)a$^wuo9crWB%86wt z6G7`{b8)=1AVZl@D7r{%uKdc0W2je*JRIa7 zXKJ1(hO%(^2Qm*mh@nhq5KoYYHQ$#pst;lwYL3H>W0u-c(O2T%AyrHUj+jV$U= z7)1wqp-hmQOT6`^U>pwcLQNC`d3Nl7rWlz*XH(HS5=VYUR5cbqU#;aLy zCQPSg?Zn7@rNl*_`0vEk2^j&WcWnhA0yGbe=9 zUS3*OA!X$1K$=KEY|8ORRH2M`g9NW*ESk~>RH00eXH4P_u9ucnC==+5HVX?Yq6HPo zi1tqe@S{tIG@L>`7c4Wvk6gs%50pY#sMczz+vhW+P$qOa=yYad9Ud8l`V!P6fYlN4 zzCje~i?eZJfuDPTLU%P5SgukQDagnZpHO}z2etCYx#Fe{ctS;DGrSChr9;)--_r@@ zB9NFNAek#H-~=c1sD-)!4%+;oPHaLC*G42cBz}Ks(S&Y77_GL=_0l5~$|zFxYsrzC)>#Kd=b>a0y(`z>6-6<^e_M@y&n)FBQf4(jyVdXaK?l z7CID=@c~4rNe)!qV#FL%jVN{)Lit*vLE0Rg!_ZO)X8By0goLD z3+zB2^spJ7qIrjvIPXj7@}t2UHx){%_rM<1z)JK4+lcTqRc9ai#CEnxh)PdtRZWLx zX@@vrjsMr72~=;KTD2`i{u4y4BT z>9ORP4^%j^4oM&ko@E)z}sx}P&sP$LWE9w#2fcb<@f?Kto4s-&cC zH;5>N^J>n!s#)saGzBd~-FF>$f{2-+eV%AZ%ejvev>=br4uR&p=8YV*rwOXtVwi|H zD5NV9{{bdQxhEYE6zmNy)zYv8(JgW5akj;6BTKS2a#MD0a_ z!8R3E_kkhE=HY?P3Ly)$zdIuYdnTP>-WZ($f_f-+ChMY8z)Q;y)Eg$2QOvXU@qzl7 z$gZ&M^2vLq2bu}Poc0z8*7e=+Q||orRw~q*Ht0YyW-c$j047^IMjG&tK@;W4snp`^ z!GRj>9DLbNNPfJ>4OD*&Hp5ElwucSWSYD}G&Fnc!oVce3vhfD!lBqRbhm!3*K?D7B zSg=8oWgLrq-GLdX5sz~=r;Wtsr9}qHL?r7`Sc*%J80eD%#cHbHh7!>qfPpdr{sH(- z^5G3HuxHX4s;$w97U=UBRNxE+AYWQopk6T zqI52~oCDymGyYmqpu!5u4mdQ9v4;~>#1N^CMoV+)Z~_%Ej8qbF7`>Jp%n1{yWd=P! z+wQ00C{L6?1q7qQSZE~jUq?uw7PN_8r|8ApJw~7+f(cy#^dmz#=H4R$6%YhUvKLAc zE-fIiW6>EAjm-%T=<&;fppo{omzE8vFv&%2>5(ZR&WR1E7tDRpBRSQhngbe8T~$(F zOxYmI$IsF+bh(6<>n9^R3OMZ<#~#;D&5$D^opFo~ z2tITDdL|9muVd8P)trqwyBfkll!fhC=b^t*CEG8K(W(NV&ZdUiQv!sEO3Qid{mff8 zLOg`L=g|A?vM_h5tjW;dg`F(AJL0oVZ$sSJG@HxQ-F9}U=ZEfYhk3yPnVV%4m#4cO zCh@q8LXb_lSK8*^GLnuNyi>X1`gx;mV23al9&*d|1HAXZNMqIXgTj{UCnK=0wBi(+ z+==VwQ;&KnwsiG(_FTW7MZ@*$7@f~SNWt{u5SIk>0w$>PGKr9 z4(-U&f_4G~$|#dko~eK=G6e@dpfWFsh5_62`RUCW57@~AwRAukWrWM-C?M2FXE>nx zx(C5Z?ElV}s+xO619mM=jJb}@iB;!!!trk=56C-zB6DFu+pZ#(aboBBo1t2G zR1t?89Uy%Eekak3=F)k0V)^yHQv(H*k$D>%nJL`(nM5I)MSv3t_OW|2NvsHSxiLEt zs`pT4G@yTEOLh@cXkJcQrCL8rG&Oc#ack1~Qaz&s46t`MCl+AusTveu@9msOfSp%- znl8_nQ9BIaE+}J_PlYQ~C|?Z`P%Be5@7z*Wdt^tmEvP%7rP9Rjgn~15~n3Sxy0|DyHHH11uYta*-1S&^Je! z1K1*$)<6KY#j>FSGNqa{3_#z`QC+kY;$4FPR5}{2s{=I3vGD-`sJw+mC!4l*j{vB= zg#Cu(4ZS+=3;UW`fi|du>;3 zZ{dG(?`FP{sCw)_HyVUtNLP5$j{N5qMUT&)`wae*No9wWUun31u0=wrMnxku*gu)f zDx#@h^Njs-lcnm+70}L!`lpwJ5)%%~-`^SfCncO&600tj+aUjx%}X&Q2ZHp8fBI4x z0;l_%OAGu{szp+o=yQ?EXt7f1O3eU)3oEbcvDtxSbv(nIc+ZM(nI~p4NyWM8P8A;~~yOoniL@IB)18DYzJ&iH$Vh}iGlY!(Q^R`@tN3h>1E z^CPs?d~8J0v%q%`h(DcdNl=DL-mz8^oe6*PY!vbVj&!M}6X8#8n2n2V5U8HLw1B^| zZUkG7gVc}z5_rF?q_`J%_u^LE z<>GD^cX@oj_s;ugH|OkT&TO*DB$-T7U*%Bjbmb>V(&xAD%Hb`TtC2q-;TE-u;w|_~ zGv@wGiik5;s$Bg>KE>YC^2Vp=#IF;u85H2p(bxjzT)ouw@w^U<##1enB{%95_dtL$*gU7YZX0r>B#O%r zdPDBla7n-Mq;wb@uB`tCE&oBlE)Zj-YMy0a+$UXOjX$R z4C-Si9DG<#83YDCM6NuU;dwOn`qe|HICb@l_TIpzn@)GFm<-3Gz|`-592duP59^Ub zGjG+-D4T9EjUVDx_c@z2V8vaIkma8b(K6|_Yqo&Bizhq0QaQyXi0vQ{9o4YJi!g{z zD+A%%ZZkkg+B577EI;5~fU!L2)8KN0Y58;Y|D)fV1LU3h9(g-a9!aI7CBJ1yQn zZ$p3@^ts*Y`nd*7R2RQ-)F!e9IEF~?DSO%>+1GrFx@)&`Gy%N7SEEhgH6FSd_h7PT z38bT{%Tw}ycER;?Dd7=wtjl9VNfwxR6K->oac@izNZ(_%7)RoY6$ryW+NfSS4ir6m zN}bw?`R-5_f?=X$sA7Ow=lH1mHwR}F_2DT1{B3D2@o~ed#a@6YYHCfFbMb8>xT=Ir z*>}{R&&V?#zA;BpmB&(Dlok{9?DIhr;z6kxjx5UtLRAS6ls@2RGTLY^kd4w$v8fC| z(ov#wmH2X(=c_x9vK$^>Rx)k;-OE>_U+DWCrs=o*BJJ{vd4y8x!l(90?^4&jW3)@| z8LuBQso^D3%LAj960836+s99uj%l^_&c5}n2tqYFSjvWRdeJE1FqkBXI9$J(YGVK$FO9k$LtgmN$I0QXQa%b?p3CMpI&d}C z(6jEtcV_=OYvfK+a&eGr0nP~aF7=(8i?S^_gV?i|Ju_wd>pj^RONvw$XA1xxk#lo&6um39IYp}Y z@491~Btf3ffUS(~(+d4Y@~O!XX4NSVI1!i%&`0;tlQNf{>?v9<&#pr9@2B zhS{-4tU9VljZ>|>`@~ioMeuKFshWpW`tQNeCoWTT8ispy1%+gZc(A6osP&*v5ORA^Zk2sE{j5>7$yAhd>S738I+uWO#6RDj z9Iix-@u(4yr4QAKk)vqlwcK_Er+A1+nSJ}bG9xw%ojD1u%0Acv@{9R%qrtprws2Ml z4wgTxz>(NrVB$1xH~hBYe9XU;7+rFkeU2bU4P+2Pm!%KDyg=jOcl53iT40qh1`|Atesm zR=*c)!GB)?4F!u+W3b9x*JIT?L(HV8D#@Va9~@FqBWEu`<&%pQYS2ppz8MHq#kprp z%8eQk{&4pPV*IPLaxW-&nyB1+E%Lln>DRomnoY;nu1|vJ7g>D;C47h${z2bHZsbxj zSQh^?D-f$U<^msg>R%QvSVE7WQUmi?q5X2~mGqYi+dZP7WPT&gl>wq*?npPLt3lJIBxU`NIIwhc9o>9&nE9+pF2@FTlL z4ok%bj*BDB2+zx=daWz@taB8yh52o;Emic+v2bD9KfNOc)&OBBdNLT`;JSs5G=!GO z#qtut%)FK1HX~d<0LRCSy_@RpBwoh*D*XU;#-i{i#~I}sE&ig^w4rA|$;b6XMN1k< z&-3-El8_tM15yPI%suJQA4ni6`E;U;98T2)wqaR+G>7>g94H=QdsZNCzO`kbd_+hF zbXmC+AT>8VQDFyMGMili$2vRWrpKs@#h%lBTwwb~Ar|;y`Fq+i2cYPMUmfi?^?GF3&>U zo;qCu0FN?5*Nb;+DLUr(w>Occ=!2fpd@ms}D1rZC$dTf&meXDpbsTtGhDWN7DRd;V zJpOE@?EG+m?Mq`n&{9D*z$;}^-VBrCmUpf!&S{1)Pi&FO0b4h;H?#ODEVM8VMgGS} z(dH7={C9!-y!?l}O=epa#z4vN;G;Z0HSHVaw&VRPYD%_&15^hnpCpxtL(cCGITQkt zpsrZ~V>^aJcbn-R9V;46C#IBYm_Nre=Ki6^E#Cs)3j+`!1XNZszi{_@$g?=H%rrD8y2eGuX~2ry{?Gg@2Ws_i3RW)N4*ElIejXFkNL*Cs1oX46_rXN90ct8FE z#%OG*0SB0F@|`j_c5{{0Cj2ov@t}$^oVEoGL5^B`#;(=m{gsMt4j>s_g$va^%_d~{ z+y-6)PK;Z{I^1W2%B)|~JZV#jl(#{!SpD-_lmTz8{*9Xt_Efmah%$;@muZ4^Nt1Cc zX$y5S(J<0jJQO~8<5!Yj7b_b8B4^l?LvV`}2RLeWq z0Ksooe-3?dNh?9$Ivs+RPZT+Z4`33Ybn=w;9R8{${x`Lxwye);gy4yO{Wu^Y%!J@c{ zPJUL(LwNWn=_KAqTL}p(llV_P* ziV)PqL-p#+CT3>-Ifyeq@+5kuD1F8~@rZBuC*#Lk(qVEn%64$#KwZ1x3iYu+)NNzv z_!dJhsGyrhu~5N84BUdYyIP{)54q0l_fXBX|H`kR>!yV>_bA->!ec~wt?&3KBMt$y zT|NPmt|KAyJ|Cj5M6dLVj33D95o8io(3bW4jZ+U7vHl!($nh0w5n6dI-INr(ER;qn2#?JjL390wOoMZ?9aI*0wx`h{%#? zqt*$<8|-Pqa_=Y0(;0~}C%M$4mRkXFQ+3DscH+z_Au|T@k3|Xuxy@7Wgqc zOLn1HBfPR^9ctR@WU^QVu{TEHQU_jJC)B`)_lGAAv^9=oNYlvN4OQc^!wS=IdEs3S zqmCdL+fnzL4NpiVqtkn_+vb>G@omSJ>VQ*{b$!Sl$Szb!v7iO%+_hAm%2n;UW(HGX zO#A1&BGJ?W3pokD9^f*+@8 zaZ^mCwD7mm-#uL>jj`pZFpt!WtH-IZ|LS`R9W%*_AWTIH0XLo~grE}ROoj}$a zl!m;56{4n1%ZR*eSpk=_XXbG4B$W;C0LbZzI{yzh-mZH5l2$`1Vta8;3}eduUSH;?s^ePmo4iae5J9fKbjK zA>D0m>7Q4QstB&Wm*(glZZO&K#Z}5}v47QC=-7`R-JUV(v5rgrMpzasylhXiS;129 z4Nl>EbCXJgK&0@Oc?yP61?_H{Fi{10?1UO?O_a`@hQZU5+yeivFvFsDH_btLIGR}a z;ih0&pL}QGs$H87J=URM-q>nb8`}FLTCN8@p3J;my0H;%ENk200b<8|k~sIL;`Iu1 zkA&ot_R$jk>r*|uNq4uOZf+B8c!@@Xf6H3*`pkP-Rg=!w7#2n=7Dwodm+ld#wZA|{ z@-uak=mY`q=bx!$S{Kc!fT^fqCA>2gSc;N6^0mcH*_;6*1>c>8KJY-h!jMZYqN!DaakI%Yy9U@O7}^B3Ax)h!tMoO zBt!i$>q+rIqqbKpL(Ogd8nsoKW~J0PdzF+pN~0vcO{Yx9xqtm~Z>c#DwOw2F^Ka5D zkGQpGJ&(}_({O4dpTzgS%ys8=5~-KGZ+tV|*~=Wd%lk#GK$V-pP0zTDM2v9wNwy^0 zSza-9Y~UmW1LipZKNU!c5F|X?acZNjv_)4Y$ml-olecG?y14Zz^_?8Yu{RAWURm%d z=w(z`JUhQ&S}lK6u^I2J2(|HGT5=zP}(qYos~YRlUFd?;dfP77E>X3i6yE0I|PGuw9Ud=;i{i#HV6FiZpjvba_ii_ zmL<&l1)A<3d1UMj^F z5)({ysfHf?ZfF&^C@>2Q@@fp`%VdUOV?qsSbiaP-`Pt2X&L@2|fbQ_UF;09#l~UR2 zfcEQN>1Ko0UTa25LHQ!&PP({BokP+F%Q>#S8$_$Okzo=sZzoksWL`vFxo|F2`f}aK z36=`ZTIca}j>CvZK`qH~rp~LD>ci!fPmbYg5{x%`%o&JVt_Ee>-Ql7iDb zhJ3-KaKXf~#k^QlAMqYEmx$3!rNp*~>f#KsomrSH$bTkTvPLu#9&|g%+|rpZtHCmu zuJp;2ij>9sP>2aGW=3hvEM#Wo(68ju`nqAGtBJD>Kn)k!xX@LEvWZNho^$Nv#Z|#z zrG~|BDe+wqwMbJ8_^4zRcc(fqg$L@u-gYZ;ycI-TOnGq_zpb(H^pQEM%Y7<#Iz;lH z?WmIoDBO{#oe*{eynMMU(eF%iKiX|C(@}nx6=~enWorKjE1(`GT33*?Ts`26pUeIF zvew6^7Ux z^uGc-f4Nn3aJ+um9=6Ds8;L9s=G7!cG9QRbJ#jp*oDnm4-hIvf(ihBWa$=B2M3Hz> zI<08he|%{QNo(EHPQNc(TdR)65bP|`87rY1dCazDh1xokgx9Gbi7$lj2LB8huARdN z^^lE=^R@*?9V4mojg0S6FsE^x>`cLM+)I_)5^;_9j?ABmzekvqI zm32F|_Z`=JcU_nav%B>Ja?N;It9^xN(V*S$uuK}kj4y~2V;(<$Nff_gq65EDl#{xNArT$sb(jHDhs_q zo5-NeM(O|v2XlYrx?p8Jh#WD9Juw=oK91xy!ySKVXiCBolo`y#3c_d7TMgnK>oYZ( zr#}X1SKy?VCTMKyGdkf;!r)>dRMMa~eCeg40$}vu+^U}mF#LVg578CQg8eh3zmbFV zMejR=_yPXwG$ZC#6Gj+%Uc;XRJrW8i$yUz7pU{ph0QYWDGRtEH)kRK3inC)Am&4Js zKWbRBh?&L)2-)95Gg1Hk76`}5XF;1Q9u16Oc8)VeGA3h*4MP1n@lpKmBBg7)Z^=cA z4Z;jUr8Cr(W(xrgHp1hoE6A_fwY63~`n^#&+^%M^V zCR28~*nkR}Vq?LW6jt996<4Ow>gUR>9pD}z75tQ2>6kKUjxluQjDW=G(Kwc&`ON7H zxN>K})cnWF(7Y$<5~JBI*H+Nq)O3r9VI*(#+c;{2D0DGv|ATu6*ar=Im@iRm)04ma zTH@?qpp<+2kzCEDg=uQ%?-SCHw?fH#L>M;HiI@b|G53{n);PUt83`FJpCI%$i*M*_ za~O-Y;)TDKVr(XqIikncg7$b{Or<<-q#Z7QbBaQqqkcyytBw`uwhTYmFaK4Vg(NS+ zZZv4s2-#v-0Ak7>yIxObocG=bXe~$md-I!ztW$h`2gtVAEJw zqBvN`PTa!C4~nTmna3yaS$_E;&Kfb*90nsoKlz)OKRnKs0p4SD?l{!bFds03YpSN8 zp!pT)3N2}%v{qsa)+OI|Qco81D<5I@m4@sa;)IK?;J?!W{7c(?wueNwB^U_~SBn$I zjo0^?FR~?Z$6=FAHVjAAy~E6iznx$oyQk~IoID*&X;rqmW4pTK5X5Nw|FC+i@GvA` zWsSzZ$73gaD+%es33k13kjUMb-I&}8*ae;(<54fjh(m3i_E}oTO%8y`**9idmSy%w zvA>{WT>8s~X+# z%6e^Xr1uZkHJO`j(`;_A0C%rLaj9+C__*(6J+w;Pn zx=UE7DDh;!zDz6SMYJ&H#3XfZ72gde%f;;S9aIA9(iUB0-2(m>^mkbo=2 z@$^r$i18NEcB)#{a==uDp{_@Y!@9YKwJc_U>s=n^W{DGfTv4L_@SYud$9oL97OPe} zlwVXmDOeOBOrIX_oy}FPdPoZ29H~~a=#eqnmUXYd`L7g5GL6K*tp}4g9zlVSdW!kB zmb!%J<&toW~Qt#9+YH^LltHj>Zw~~2S zm6oaR=9fd4x&s#l?1PWV~?)5 z`YNK`>Ef8jx1%OuBlkPH&J94VFXG}zc!O-$v7Mej)bZN8{Ij;uSh6F{+u((;giBy# ziSZX`I-CwJDGF(nL+FJR$nx7N6ErO zNAV8N$Y~gpI}=>ELP_+QJJ3vqi%*0lZNuN)_>nh zdYYL7BUSO(HGq@7KC~~M%{baJ8Q!uG&EHsSrMGV0C4GmaEMDFF5hkpOZSg6%8LhZb z`N;I~ir5R9H_`|3c-+Et&R9eHh-bEF($nb;e(~^plzVJZV=$&%Rb0b-dI%3*tTx!5 zME5MJqxNlw8K|naxMHn_vj)6Oe`r>*)gAhE&KhYGPi6N^hxGl(*0mySe{$XFz6Pa2 zz8EUNy~JDBR!ZZOGVPw`#x+T+-B+4htCPn3={9g8vU`OXO(g$QD&Roz!R`VxWpg_M zuB}z&GGVD_RfVB~?vKQhit)9Z)tM(CZ?yQqEyj_iC>+MWTdtiw_jSkxYSz^4yB$`~ zk7j&ICTTM3y9P3fUTk)QYs30dt7tzu*`9C72l?*iWYU8&d!o1JlN5zdF~&3V z2J-}gZfhX_-R{UM9e=crf|*2Q0RpkCgnxEUVBr=Pj|rZn;~=SxO=VMdIbh_Tv(pj{ zMu=s+*nz5v8RLA2vm}HL488qtAn$?ungUfmR3q(4!*qza7{uZClk0G9kxlxz@dFwU zE^9xr9(51RDBlSk;fB1`AAl)9<4lC~Gd~W-f!Cg&Sezgc#;iD-w)r)2#pcv#X-m{8 zE)katIn*#Yk0GY@a}A0OayCpa(~&>j1(i#s869Nt^wVOCY_%5euPpXSP|Rmd>uJv6 z)NBGrq}7SQ&y<~4uh_|sYC?tVn=oI5twJ*DSs~4cASS|H@?yS)R#q*if(rV9On5cAU z$drtq6LO^Nr$uRC6bTVlW}X@*@q?edB#MppSmiVd@uS8Y)CJ#hTJ2$aDLq+Z9RKztqd|Pa21v(F}pZ(cyzy8w5yL)hC z-)r4QsNFtmk3{~}%?w5}Wlb->>uOhZL4XKvWZ7X(Hx|4}jb_~+fgwqwn}aGGK=?(D zh}j@-9pB!1tS>JNGAuLwtsk7}pwE5lw{ z=uz9w>j?Qz5T=B&D`L6vnbu)Q$(Agd!!X_NNY{rH2p?#wj6jP*FDy-uCuj6GjVtF^ z!gF9DV@6q$eIr-Xu~Og;Ut{A>UD7BkmRsdE>291*a0>$Qs&GI8RPy-sO$cm`4A$rX z)Zkh}8O4Gmf;HcSPuC8A*lE*e#X0#J2@s{rKcf__(f&co#h3n2xMvg4U)JV)`E~g} zy19zT2C`32WybVC5{a543X$*0{DeutyTlXHZ(8#-kz)7!VYSNfMleMltq6Ni9Ovbh zv8?fDT zD}PgI11ts@u_zMfJ8RX43ezEWCd=7P1|*I2H1GdusB%i9;MAv{f=^T})lC93Ri?|t zY3u8sv#)r&phH@}+)rxg;oVW%*zNai^@sjrJsg!cd;k=|a|Wf@8>{MK{?M%ah-Dxr zAed2*-sRr5a0U9HSk6x@iz&JFil@>!%#(7B5X?!I*xX-N@2s_>!ji#iip}Pd|Ab1x zeFKHl&8o`iZ0i0QlM&Gw*V!M3<8H)xOrT}|4MQEzUNiZAmH0O}!fZv>2~vDgKd<0- zB}z(XLvo+)a(hGI(u!2Eu2J6~0>xR+(DU{yZ5V7JKT)(V$8H!N##i%96Cs&O>h0#u zCIb5k%IWh?-LuBBt4ekA>U-^@Z>n3nP*cSPs7w%z`l|1|U8H=q0$e_1iuY>U`|a>` zN+e?GjTO|z^sa_KSc_88rNk5WSoO2%Fpm?V;$`)*u78||PV)TF370RvMs;(ECbtR- zgB?BpD`OnyLbE=htA zUf)nZ+K+k+hQ%Qgv7J1bI9TWhe0}J}j41gpD8(#lc z;}QZfjqI%nU0H`*Ek)p=iRr?c)q>$qqp8@fsry}-44>!in-{J!Lt_+gOL@v%6WP5d0PFG*5I326mzNk%8xcy5j9&g@MFD`+_Y|J71keRDWj<2c1&cq8@=4l zACeMoEgSXGND(3VzNeDFf^)KjxEc2_8tNWg4-dJGk4e$BaaW>KjS9$kAgznZqGfMWVtu%UNQ6O_55U@N#ms9pmr!DujJ#pVK zL$Qm4|5Ba~xX#b87MHO&5hbt*B&y4m6iXwM^+*E}k!2fQ%MA(nq(JM+P{bXpQJc{W zw(lP_6mdgzQ1%uEE=@G6)LOJN28l{+*3inS4(K-SnOPw;PZSjcibq6ZD-;(%8gy5t}iqF-Z1cKryyqjA*89}4I)lD zBC)VPSP5n=IAXJZd))@E>c*0lBr-8Asou!;vF7eaNA+NvDm47|-S5U&Q>+g;evoloSr4%*Uk!w`g=~nVh4`gp1hxV<|YZ;y-96tD?SdP-lgjU-Ps1qFzP-KPrbY`BJ42I!io;Yf$@sDQ+mg zzEB(DNiw)n_9C$Lt_ZI>a$`9_ew>QuWUznwyve-vOt_wO#G+*#Uv@F3mKrp$o}YyeD!U(scR6@ND-7}3-j#yYw&fv%6jqlS2tCN zMBB<^s=STGXPyZ!o?^vQMpzi{$Ye9CE{;@N$kJ4|+8=@`go!?X^-7>*>1rY}O8NPo6XY z(l!83n$;~arO_Am4gA&IBNV!GRZ~>*rexjfFm>fG`o}AF-%-rCEqNbLENd`*9UApO zjuc=yB-JOKg@5vyztlPgDB9kN+_S)%Q8njaJH?29s8#S53>rvTar{NetdakCq-$gMabAvSkIJuxkHCdyKE% z#E-+Z+&lQzmZ_;m<`9gTBQ;y(s{{IMOFPjVVs8>|n(;JN|90Ebuh7ZoY0kGuIhv}- z_3jaH4C3049iEzh9SdANujzh!K#wV&g$9KxGN-7@2G8}UBSoI*kf1l;(qckPP#YG| zkctfM!-%Ln*uaL+bv)EjDPihMU?$n=N{Osu*{e;-*zf7?RapGp?IqQS)W?7t_$iH% z?8Wxd&8=R#rMx4XIS;)`)=R=4wNNjYiV6k8oW5Uiq)J^P(Rl&=EQOmSN|B0bwPoO)xOXud z%t6T*V^$EVQ2h2cd^{0>dY0gF-AlmzG6GaR+$WBS(|Fw(Fu8ctLEU)4HDah?F#kT< zohWYo3D1br>SRXoPeXaElLFhGzf5SJtXXOt4qY`$R4Xj7{nHA`Qn}MOODr>3XlGZ> z^t+0-g4+=;T+^Lg_<>HqW2LqN`qt(xIk*}U*)||h6ijW7+B`kd@;+}9D)le|7V!vN zBQGhrACZt1^bLTc4)tq|`#Fda*sv6*E1bB5q=oCAYA#3wk6pjh)6(QgK67ZhdDS{DWX|VgQIzd z{(E`MVlatc5eKQNp3XXap&MxAY9A&rU)uaL z{@O~5hZCwVnNHjuC#zBm-+vK65*KnTh-g5ydUy%iVB1%hekdU);V?!wOK9v>?Vl^~ zQQ?E&4>+s;${9rdVphx4?NdO0(4b|KE=e;6TV?J6{&A&-x{f{=uyj3=)Q8pmF?LU( zM~a71q+i38X-fvRhwt;B#{-Nbln1m?n8z=h>uR^Df|rH`d~Pt9XnX#Qc#J5Ryvmx* z=z7ZAzT6VVmhfP^yV!M$IC-TQ&YsX-8!~GsL8CVcHXEdhY^bO?SoDElYh_HwK8*opkJMi*1(h5 z%A;x{z8!DvmR$HLa-BT^@;Exxy%A@O$9jW-&21;hlk2gg*b!nAJM(uHceERuLOja_hFM zc}vrVvaEw-IS0JuXFl@5+&b~pXK9}p#c|NtTLL{c&BWq4k=80W=frzujE#LF&7D%l z?%#gBp22vW_M^rLmns6Hs6b{77&%|*g?bW(aW{}AjBW|7|HBJAsX)HT1>}&z4y}!J z!vR&dVx8+gzG`Y36C$vkvRI{)eQzV?p$)!YzaTQDm9*ewW&aV+REBQbR>F|0>Icor z&w*v&zu-*CGCndi%seEkHhic8tLLS}iib`6;uo<)h5WRVRRIe}O-4)Hn$?qbQ@hZF zZ>EC@N(4_H?-i=RtqCu&M%_`&&pBhd#EY#w#(LQy}EFgh^6~6$dS1{Jm%ktTss1FfICA3YK*P*s68|08-M-gieXQOBz_OIs=u$Pmj_lfr@=LVl4vQji9 zL1x${sK{0L$(LHgBT;X90c!hf9z9s5zkdssrVW!Obvqs>Ni|4dQFGvXQNNQ@IRze` zwpkYCUicR5^sqs|r`rkOen*i&Zc-jn=!2WNn?`_T1YAp;O7Q3eQ6K0;7c>UtAb0t4 zz?d;UKUFgpr!JhfDmt3`R zkQ}KMtL&grcJ5olXngU%x_J4jgk+ze|le04ZjfaG5=`&CrG4v!4oY$8#fl0_GqMLvF83rvE95iL&#ynfK zAK@YRtPF>;pT~=3_Bj3; z&`~fb_SX<8H>U>zW+nVDz}l4Jy~1ZYE&y372`;867@}J}I`m z_giAaNQ249j~WJXsFvK_wIgnXE+3X4-!%i2}HuJ@$zT&!Bpn&MtR>%b;H{v zXwqJoT*bCr+9V9)=fEkv@SU0{T>;dGeHr}T!V(eBiow9uK+;0d_f4M}%MXvSLV!5@|mKoOU%eSq72%d%Z_mm5KU%2%jb{@FK3e^O^T+?Of8rlJUDIlI5 zX_GPuAT==0@P(A7gUo0ZpR)ck*wPLFxu2@#TTq}aob%9JHke4U9pDKx+4~*tra`$( zeS(Zlk@Sybzg*1plhsHCRuo|PR1lP0IW#N=pkC~7W7h=2`mCTTA_lgvl!qDXEDZuNXU7=Ymh~ z;pp|Pdpn5FS#5DYoK2CdmKnrWqg@`eNoJ2zQ+t=}qDN~7N2(0*tb54VIr%b^+b_AxB3 z@hZz!ozFkpULiV^Oo=dHw6Y^t&zcbp%|li>&O_NiZ_RH~upyVg>Wp~~osl{k;(8~e z8u$9(Xo)6f%3@CLS4(Au7RV$W?UGnw6Kw?1bN^&3{Gysjz79Y3LI^v#L`;YVaim+v zEl6a0*=d$KmZ!Y0Mq{;Tvoi#iO9%(Tx@*5tq z@rrqW6XlLJ70?Y*Z$YtNbU(_L;r}@wI_QU9jGMF|AreF@A9ZcV%iQGia=dVeZ1Xp^}M}izYlZb zWC{g12pM0TJ9Ycr94vtaU;60pjA{Zt4(7aayPgU^9)$ef2Gat|KZuw}ZomQe7pGH3 zU2m@oTH=y{Z#yMS#6DPJK9U{Rx#IA|j2}v&Tg>|J)y5&`-~b|d>KMGo1hHsaI~#`%VKF$8Z0;C#8Vv zaju%4fV+zpr_I+r>Ri98{SGdp_p$IPrGO;A0qr0Tu;bY(C*V2FX~X~RV0<&V^{H*? zWgq9`{(|iVvB&f^{KGe}r9anW|JF*M=dp*#vGaHCmC&!u$C|h61*_bTHLbvnt^p_4 zd?9d3N?CpWrwRZ8|A9}i;&&IXD-JW_kn{xr;LcGcEzdkavY4>JMNQ6Gl zPB;C5!*NQlTbtcauPr@vUbdONxq(*_hd7%7SI?Hupr>&VEn@dx%=ZZ!qXeE6cq>IXktb)b`d=! z#2$OZ{u=)5eR@J9jmoq@p^r3T#vZ**(3!t=*f z^ukL8#euRfK*t-|7bs)Lip@kx$~?Mf2qb)#^)F}qII}pTv})9ghhS!Nk9w6I6PtF@R-Sr_;>pF=7>T)2Z3KZkVBMHHL16wrXB7Ph=EE_*g`w6rHEG#t`& zAf=O5^%YaC_6a^$Rl;{?ydg;{D>h`DKI9_n@O&upQhH_(IOdg~U(rcslZANY$(e61 z%c@a9s8v;d%y~Ou3w*z@aI$bgla!{bVw=OnfOkQ`_|{4zDeckh=-EmCmXbhsVAAQ& zxQKh#X%EFF#qx#H&w%%w_iwW>hziGx4(}+$_zUJ|#BkgT!%c2ZaWMhrMqTD1c z096}m5=mi`h;tPxLNQ-#Mp5*sk_ zBj7AJf*UM(>pO+($J9xGJvDEK5o)mzTy~6pl**vDhk98LDWI6eSl@H1e^Vd4ac`4d z$lP0%rZIN+Keog}J23snmi}>CCO}tRM_LzP<#TF~K05L3)5ViX)JTomNI#_Wn+X%{ zgGNM$jH56kSG`gMf33Dq1)b9b*UIH^{7$=~`H#gW`~)8V@%Y~L1I^H*!2jCU%&W%t zp?q_rLh%L7{}aHd!6!W}3Z`OrA7ad|hrTIii#yjKYKT*{Lg#3Cf8M{US>>oy3_MKaYb81Rs4*6M85%E?c}dBckwmoC5pxJfJ9liauS6AIIcbCED-Ky2f#ye=>JIswUI@-nw< zuFt1(B3QhNfQu3cg3FE|3EZg>inYG|#P0Y9Absw4PY26c+qJ&Bl25#%s3S23dZR@E z^VDm2IQRGkO>XP0=I^`yCwrfKc}?@`b2b9+eA?`<#QaxXJokX(s7>Ib=HIJuV5$1l zl_c*6@&6UB`Tq$QsXBsd&S3|{66k7!2#m6D)y)LS*y=WHxWTh`+zE-qefhGYCHwN-unCX4Q8vZJhI6J22rB zs3$6Ho@)M|dP8MsdoVu-LHD#MKWwDE2;A0hYKD)q<#$Mybf-~XJ(tdm=co(^H-9cO zGZ8x$O9s1o?f;3!r+knkr1>|0(;6v}EXn!2(FpvH$h7wV`@v8|!|7^vy`8>x02Us0 zHNdR;n^HB;{CL#~H4e&M%nMKrkH0k+ufS`qH;*q$Z*TBuNe31R50kYWo1IdHLi{~X z-HolQMV2MI%@g^ax@LTm4giP4|MVOwuM0S8v188PNy^~Ca~+&N=P#tI?-1Q^>NbV9 zP(9@k%X`t-wcZdbh?xse>e%qV03E&?`(L;z43fvsZ9^MQ*0SiSY+~#<8y-WbupMTzF zl6;)ap$6O?{4&bKs5^YiWZLX}y6c$gW`=%CdmAYET!(I?UYr0nAD4GHM(<_sT!Lfo zGfwYoF);e15zfd8LSMfNeFp00X4$-bSbwM9+2hv3ru_)gAM98f_gI`N7xh?@t-M9$Dooe%!`<8Z&-(C0s@Uw=*#$ zZ%1IGxBH2jZoj*>xJI0wx08jQp0-Zkhl`jRqffKI=-i31=k0nz>po;dzxx@46S$>x zmaBgAiHP%Y>SXvDi1B8Hi)7(w?&j`lVdC%~@Z^58e|{d}1txi%VG4K| zk3(1^rp^w)GdjQs41Bx1IC#H^iECO5u)Ex+ta%^oF%nwud~&%Nj(F)=e{;QwG1`f7 za@yE5+^x8&c+g5C@m`2A0tE&J`aV3Lz2CoYpH{4$K7ebO-k-4p^}B|bm;#R`?$1xb z0XiFqx!b1e{W;nI)19HfnTO1pzzJ~8ML_E(HkLSUI zusV|)Q&vwics6_aso>X#%uU0d3MGP;#nqW#i#L7^=M%=%7$9zB45ntiHI$iOy$U}^ zxp;d5@3trIKrb~U7e)p>0BUI8hrxrh;fjulnwsumF0IVKuI{d`k8w*cz}?AIj2t5v zysRW+H}VD9*)S+b>oruI_m3P1NYa z`TqctKyANs{g*E`zq!A=PP2CU@cx@bf6n1e<=^zpZ$I2#Km6|Qh6^M2nbNfV`u_Fx zZT?8Ezr4MY*^Bk}_yW`u zCYwm}Umk8=?dNzjecO@*f}W-(SDOy>72J z(|Gdz!_^Ph8)M_i>Fb;82YmDH>yzKS$5y{S`Kp}O zo&0XT;wqhTC3icydUf-F9r4}m^ z*ViYnFyANdK-9M!#k=cQ+;b;?{BR37b@KM=fiv*-PbawI&DGuOKi*=N9)ykIZm&;X z-M{@2wD+g?Ctz!@ugSNZU5ZWi# ze|>d(^+tXLb=|zWy2S{BT18nI~Py%5&y|HsR3-$7#T1?7+=)5YDZd!nu{HZLozqOi-qoz$nC)R!cw z@$LUg4**Xrx8HpG@ZtK~AKza8#J6z@HL168C-o&i+uMu^uc@~gPwGpKc$I?==GLT@`v&|)!iRCVe!?|5Be3vSA7Hd40QLG z`-eZz4_s)kTad`(cm3wi*Y98bslIRi^yBU<)Ohpm&3yUtldrj3Ufo`Q^COVM5exUt zD3`rEC61 zpEY08dJew)dwIs$70{YM+?!Xe$27j5pSS$(dCPC<}X*+{=d5Ua^Xv<())$m zF;4L`?&4WIj~DTB>L)YV$xVJ5r!diOaysp%v*~=gm@ZHIQ*&xhhf{azPsh`68c(Ow z)6?DQS$BGVdU1NW>vv{ncf-!@{BGQZUEEE()7>sjXn58!K6P}B4;O1vO|t#8M}5P` z*6`6ore=u4G@K5*;cPe`E{01insL?*&N=T!7hH6cJ9RsE2I{$Rmsm#Qt#{u0(FY&> z>zL6KoiQ3qFh?gczo98g}6f z6MqpdV;@blafnXd?(*{T0>?RyGaNe{r#L1Y5l6r=UV0o3$AH7)FgW_lo)=#5=llYj z@dDfR;bn|6n-3u443 zj_09T;$wfzaYuLTiJ#(z&m_OgOIaBij1@LPz8U)1PyK0+ZP=gn=a|7u+}jw7V=&Hm z=$fMm&=s-E&8gWLFivyM**2HFZ-+DaGvII*6lOt27NlZ94i=2TV*6X{a0`BG!Bwp+ z7uP$Jvynq?FyN=-KZDDM&&RkpLUqJLM;>x!+~-0UWZu0{3m7;KCp_E>MUk?i(AlXR zBt4QKFXWHV^kv_HN{t*c3OSq{UXD?YkdG+Gl#kQ-*v-dTC!2)F`Rcf69#8x%DV#{5 zw}XcepA3!K!%~p1kiz4q2~y}y1QghY!KIT%zdS9*Z5Z9+hR2+p?`wIAH*w^dU|`EL z$8j6|FfDI<%<1XQ{m4_TogEZ%j*Z(Ma%yqQ$Is)vlDlD?zqWA!lbT;X2PJ*;;Vt3l z{dez^$8q!foA2-FUTW}ZG( zCb~LpJ9=ZWF5_k}n3sOoglQXsIeg>8S2XWG`34C;cG7be<9By=S8uLgZ@$kTZ46eS z@gb(e73c~G$zznm=flY{$RX_SOt>e&ryP0z3!B-ApaMKi$BtNa2OPv>k|X9L$T7;{ z<#72JF7>{vjg7o=y?<- zHwh1&MCt+v^2mcoPe`scKt&!QA7cmnI8F&{;sLyKFZVi^>rds1z*7J=hlyVPOW=WX z0MQ-ZcLFdC0H_{7b%5+Ku?PFT06aMhyADH{fLQ{tiwA}oAfQa>!KW{PZ_Y56r+9b- z?iqo99Hfke-0Q*3F949vFw3U^OcB6o1b}jW=pgBg@4@shfUnLl{incN5x8pv{=&Wh z7BjAgXt)4AJHx`90oAuSBxZp0^pKkCR} zctSS|>!ZuR#O5(IkJILnX}&AJJEu!O;UB_l*LKV zC3}(X7r>+ZNxF^fv>g;r1>`gz)nSIFN0sAuzz7Jt>-(${BHVQM1Jxvkg=&*qfTPq$v z@8g6(-;@IVN(=}}ULR2%p>d3j0}Q6)Ka0aPjzJx^apa%Q?0*_uAQ5?W=-sf`E#3U9 z-d?-Lp?7SxuRHl)iwy*Iv^b06Ky7B#;%l1!HRot%r?QW+#(ZCjA_^mlqa=_{MZQre zQ7k3FEGwkai+XV(f-8xxoycg4FAA_E z!YIU&7_*&FQ!l#7MV4o0f*dH*DAkf&qhu4=hQlWDCIXHkE(y6a!5OC=Bpqd4l6I7L zN#arFiPRIhmn7f0pr4%}hAB(HAX=Ok{f7u@`hmRJaou7zOxw zfqH~^yKM-Oi3~gc z2i)+J{2Xt-d+rVQzLtOLR$_F&{Fn1_u{h4><7_^5^C1YaOaC)da!EXUZ$G}ZyRx^7 z8TN9v8x^Zku{G^xvurWV7V~VJjaOqnl3~kt+0KrdD$3`(&S+Cg+LQ z{TR(KiOt3ZW5izxC2{%PyQSY@VYT}ySIcwB`)Q-wh9N>E$a%emz;chME6=$o zjV^eIE50AZ5pAm5X*5xO>3i#(Y3eY~y6WRJ8rRfl9PNs;M$;ei7aZ~yBws=D6r{a( zDVxnE2SFzuf*`(u;uoM#;4=FHtQh~3$v=pbtGKzcqic3`WoK76`@53ItN6Uc>y`ap z^nCH2P9W}>Jz&WPb}CT#OdNoO$Cn-5VZvpsc*F*Q()5ar0E`mEOxwho=RDV(^@ytvOsaiIlqqeXG0P2x^F6_;8%vPE(BZeGmCWjAwiJR)^moU3P3OkD!j&Jf_ys&A*;Lb!ZdbemxHzPq_n! z$72YNjU!m8**_sd<1yP{*HHK$JKDCx#nC-|%A*EHb#%v`*iY>>{8V`LqkjHwm5Bf9 z8!lmKiFSB2CgvGHvO9z%1xU^?^3mv}-7AHO6%Z+g4Wg;GT{?*;Q-EEl2t z%*c~Y1n|M&ljp~tbnW1irX_D4Ze)w}?~DKgj^5rz!?Y8~OC))9(=P}tD(8PNe&)a59+M_AD^NNTC}D8*+rta?yq zYsIHqj;g_1pUqu671*bS)Yj6eIS#{?X7?(7rSY|BvA91~YlCM~1;ty6CJKFfA15p67 zRTr&7N}hAk$LZo@Qx_lUT^ts=xCKFuAjMOV;e}x+lmqESklh(1dJkfApfgh04it7N z)J0b)scJ`x0zHwMJSnM@y#Xr1dXa)iKct?t9VnNRRHjT$5;@7^B8^KN=q!jeEP5U& zs$o&+W)s^Cjb?CE;E|uX&?MY!ClNH~5><1V!IF{a8vv$GFq4-s9pa`aJoQx2(@wne z=YpXw#L(7DB)E}iaKfn}hhuwzRXjo`=qd`rni3C9p*#|On&J-6XL*#u`uaHrF~uRK zV@O9QA&Dg{v4kd$bAVzBQrsWYe4H+h-Tvru6yyGQN=UZD4);QZvT&D~l7WQt3$}D2Vs8+}6hz!n*g@2F zkad)FNzNs~okUy_a7DcBRfk$}Sg8%&lQHXBaoDR4d$v^=m+La`3K^iCW>}U1*$l;O z#$ZeIwZ~lx%+)ElRj`(ux!VF~N-sk8!=&buNy2*zyNKqFz4MPbuBR*|J z5g&6hpkAu?Gj7bVaWnE=e^}1ihEREV>W@DTI>sh+jGx^&pWQg0-8i4!IG^1(pWQe= zs{#1$?1rD+IG^1(pWQg>_^)JK&vxTH<7nUT?57{y$6WleH{2a_bjO{xx@2}d71z_= z{j_&M6(`i*5p_Ce@)dW~*{p+Ibh9n;a^;)qXYjX2Lph5uyQzw!%IAE;?5^q-KC4(> z;PiCmyV}jEjOQyK*2Nq^+3$@jT?U^vAH%|{wfAflo6A^0{G5C_E$lC1gXvagn6sS5 zc|Ol`3Ef=SWA?GL1@07dUXt+73lx`~X|k7Tm?3vb$&WL#-gOH_?IL65f{}+A1&>*0 zD~b8*wEWA3_GU{Oeo4MhY4%Bt+beQ=Rc^1$B?;msHDsaBbuz<6rAq>3t<GI{LrR9Xa^zvS$= zKH920X}fi}a&-G&MI&e(WAg}&1G}r6#cXT{TrGN+$8LSJ6jP1y9?t?$Hi7Pd-hW74 z-_Ly`BRgefw{x#A*7m2B4Qg+NGE3u6vP5+&bCfbj&50syo7ALiQjnwCEL9n$n)WDd ziAsf#szM0s5(ugIfmA(^q`Z{6W98L)A?}h_mk0_2O5mGQ__{fVua(7jFpqCp3t^qe*HxK(KdBnR zoXWS(uXm%hlF+^#IqBm*QzdNiPHvf)am>lW+{Zpnl7(V0P~&G0LTGp!jyU(DSh;Q1 zF0p16SJEh{b-1Y3vaHs=uvVN4u!%n|x#tE+bhUXO^>Z173rTxD&pyPRWWSzD2JDnY zMO90sJ6uTRU=fwwfeI=D=({5esF-~L6*-m_R8}=qx!(cCtI`QzW>lcJ9mG;#O%v`6C9-A@+6%6=>Ye-uZnU#$xK(JLvajo_*M{t1LX( zzSC6(9<%V)yV<;x+yup8zq0Qvoc3bj=~gD5rIBZ8=2;qgTJC$lxA$}_AN~O^{?h!D z{P+zwes0ZjejAB#?VpvH(Mer7wpjGK2 znr4~SHPzm>X4}+ZYsIyDLQ{RVdfFY*P#cEh$Br4*bk7VEx_Ij7o<600T-MdU*1o#B zXmstas~vXYvP-4F4pkFq)I~nzJZuyXYwIh}G#YxyF()U>RX+wvPY=;Z1z#3ONR|6(5?lp;$4^(A!}8J{L`m?l*%;tmdb=r7nRwuXiwC+$#nvcD{@2Q z=xXse8ow?^8d!I8SQb_|cuJ~Qx^-1G(_ssNKP_)?;Ea!&zGhJYw=x~}#|5LcL(v+! zLfwJ1nxE|JJ|a)(dA{zh6?^=2P?rwr3M*4v70|``>lOyJr6Fx;KwAcJ9WJYqEo9xo zK&ES|EbU=SOV~OR%g^2?S*?AH)f_tY{NSiZ!=iGb^q&wC^=R}{r#*Wcg2$czas07qk;A8-h(k7WDZ69%W9=Os@`e}} z+R0S8srOFmqNVKh6nU#IznCfZ7uDd++^(DPLvel zm~-q}eK}GMGBGTT!SAot%~GBUqJ8G^>>H*+Z_l&GJz2Z4JHB{hs@=Hm%aIDerNVEi z;9Dy5Mzwt?y*P3wj`LjOayK{O2)WXw?`CI#$ptSj=iVE;x$I?}T?6iKL$zzDEkEa?+~uHL;G-FX^h7YWUpFtPOD{j?qTKNy1gX^w;OSEu%K+Xv zt}du6n?2{E(pSMr!L?w)j>l9AO`0e)E1Dg1_DAuarpD$!C%}A8fccyN^EmvICk=LDF~2{4}%U_K|nd`^J*^a(J2+y-aH zXZbbCHJ?*plJThNrSWDF1n^ItIX`e*9CfHtY28ZG*ZO`d&0wkR@Adqw+IN*czgE?E zGy6NsrI+T`yfY&-bD)@T!*0P4$GQ3Lo+&1_m^Sv3F)mr-DtN40M30?4Y3~YobCVw|m8OIBFefUXMGWCUH!3516>8@3&Ps0MJIvK#4SmD-N9NL$g?r0$8- zm71coyWX$NL$O;s>=(jwNj+=YnVUKk17h5^xn}c!YHDJN{7b_kS=%fQ^lxppZEdz`?dr_IRqkMaVDGiJncw%^TONkItvOg%NwL_FVwaCg zv8)ZAC#POFBcP=!R7EE?m7xRbP}edJKOZBtn&MV)sH@OW8wow;RjVkgdlCbUm9%>T z0CfwIcL)I;q@5P&bP74Ih2J6Q+?IbJpKfJVIG>g0r4U@xPf??13R)9Tmz}^dJAQ?R zo{y6hD$;JCV^p%gERqTW0Eb)?++~hPXa_o8t)(R2OkMl}24Aj*Q~+w^__^mj+1{-ar8*D@xo@zRKwo+P*^PR_;jOEeOWNYXzPi~G#$j>Aig zgmlltOYXZ$)0Ys~)#YaEtRw9(@!gzCyi6!w zAO7Gm>Z4oy5sp3{`RN|}vifP%xza~>+-b)j|L5Lt-JIp|Zc!dPV&9g<%H}UkS-9c3 z-Cgp^a*tENC=FRkzo^r>UzC}(hh%!#>kLcRYqm9X%H6W}x*iTfTUr`s*K2aVb~&)3 zv|U*?T`3);&KLbw>Ru~#pOw=tw^^xLtmJTpeRq|8XO(4Fm1Rej+)brhbW&OOP$>Z~ zb=QRi(7vT@oxc8?rq?W0p4edJHK=%@8bj_wl&VEz5vaG5AI zcTCa0NRkY;yra|I53!f*`xu}*4Ue@_dB!#Cjw;@4du;MXTb1(7+*@Vz_+m6$s?D3h zL2dMI-DahE#`^kzy8;z$Y`-1s6dKppXZ#oSc8Nn%ZyZ+1qID6cMcP=EG`7zfODj(+ z-A^l{w^Z>}c2Wv!$tN5xc<{(6&za-@`KO!r*Z;WL-Cn)=GbVTS z=hAs0sdDEzO>W95a(+&a>vL+{Sx$?KIVH~JY`8vW!kv{YxK5Jbyp+XbNxWW4;hjqX zyi+NB*Q(`pez=-f_jEb0#}Wv2{$Tfb0%5M;<>yM0{am{%RqooCkjzCfI#m@TJg+Xs zTsATnjqK{Gk%wwVRwW}37L4qUsTjGA+A?i>RqMM{hvQk9y$8t{AH4j{{oU6OH+S!E z?!JHVVnN-@FS_5nMYcUxVVyZyV#rEj#K>BRJaV+{Ty) z4jndr8n$MFis*ae0NFM}h+BX=H}wwG1BHyQe=7g};V-u&DD?l-T~mA@sotly!2)aDJ&dh&%fziORT#`CMzgC6~|dH*~7cXImh z{`}_E`9Ym^9<OG z9cPgH@Ku7reI~HqN#2+I5APc$=W)WA=OpyK8HNosHfA1!bg^_j;L2I1ChQ8{#?s1w z+komZ3zO@?F$3>|r3Y_~VB_F_=~Xa8V9it;2b{r40BgtWdpfKzq;ZQ&F(hhmeIX%M ze!@eb9>(@<0H+KXKJ`b8183mf9M(>PT}hc36DAiUVby}((>VslAM^cfUSSVSl;g$y z#*ue44nm5wG;KkMxDN>yn}Ul&!GZUr9f`CWS@IpLXDBQ1l3WL@KPF(}0&QOLNx+Rl z1=u-w72L--07(XLct{J(EOgy1eI?O@0wMo^5IxWi7>SF$gE++*pf1u0B%_}qD1UNT zE`#&?7;tSG6SiRRprRhcIY0?^6BZvIfG)QKYc+sA@veM85Q3Q`Tbs*|If6*!U>(>O zh5`Z)5W*8w@gAc|Ljk|*K_ehn2u|F_jEt2f0ofjG4kBp6jM$zSJqXqXxf)^yki-~x z##0;TY_Y>YLn2`~3DY>39(y3|tR4>zKpEK67GPlmAg~>=E*ReA=<0#|6~=xelp*&6hR74_NWKnE0M?3y0Y?D_!A!Xc zRDiE>3(kW{T*M-od3kHy}uSi`|RK!zB5T-o@C!L0izZ+cRe0atNQWMrxa&I+fB zlG)^ufZ(tRIMWbB;0VSI41Fi5g^>aAAd;={s4xt@mlqB40bVq4?`{rmNIqcqM?o>| zns+ZC^d0sM8=)F%(Ud}JHV0eSqdJYBaj|e-5GLd5HBbQ zas#U}YZ~F=8yZkG}oabZQ{Nk6JKm5-s`LF?h6{$Od zzZelLf;2%g0>cw~0vxEbAhJnw*t^p-) z#$k(@!BqfFkagHN2EYp(ERb9v=pppL4asZ)Vj!J1P%*F-r#x6(PN#&OFu52j-y!fF z98-rx+mN&;ZYjqp1+iN}!}R@4d})Z)gdQ;FEvFeu4Ypd?fNO5C>kQ>Aw1bT^STJxb zY;d_5m^ztsdXYFJNJh7Z#S#N3fraB_z(oKAyuP6$)NjDcC+ap_e?is*`5|-z59J3S zI=zbHUIyZVEQ$`(ggKq$A}Dg0A1qZL2u`q6al+#udh%*ea6piNlWgftg*YP(CdLmu znqOw5NW)5d88IG}(DQ)F0BhJ_G+^0S9p{pwLArougE?W@qfBX#EddLRyMbHcnINb@ zd zEJK%^xD7EUxG5&iSr$%{n-P8ngLezqnV1JW33P;40H_va&N<>$yeyvLxf_V30U0Ad z_w&cHxZs3BgwT$t@GY*1X+pt*Msj}ciGf!0`isEKxS`yx{xWI5Kuud}EO<3g*Zd(~ z3(6`!WXakMNSw<)F8~1KCWxZMtqCF|1!8o)g^-O>-*?boj6|~F^71!N_}sg^j@W(t9eWSxgC}GE6YPT)Kr4p4id#nZlTZ(}E~rcZw|GZy z=NYGf#Yn6ZWit}ZLaqbbP0)JNqd_YK`d}x}gAGwLrUyKm9M1rufa3umVUOYVJ^2q7-jaRd zl_usR3?Mc;jPUKVUUV3zqs$jT5IKG+GF zE!0bAlAWLQ41%FygEk2+(#CoWnW89=hs#T0HN;{CK2%sb^a5es&|HbOKmcGsxQ2!S z;Fy6XTaXX90wJ=@7bpYk0K$sYhRLIB-W-^2$Xrq*>rp$PyUc!pyuAl9b_^ngx-gJ@ z@G9gsB0oV)&~ir(6?$A|p!kEXal$(T>SnGk9_r;Ka9e7k;13{b+RnV}ZEDQCyP=^b z$v~0-$3}_9rJ}op8$ql%BRfve*fYtx(4Z$TI{@JGWg+0??o-+S;9xStpmH!UO9{g? z6DW3ARS+2T2`)J19553j@B4f{&4=KY)3B!qY?&=BPoO)nUXTlB9a5ZJ6KEk4)W~ug z_CWHM;@%9&VgfBDiUJKZF_r1(!eaD&T0$cwtS!X^o;N~7C4F%_ivS2mNBId!h$-IC zj>PGOV#6yjY{z-w(CD4x(CC1`t9jYr6-VM_1wOPK_3JYYCpbUq!J13VR{1B!W?3DWKB zkC(5czAOXrv=xB&;vtX+WGQLnp7v0X1W*jb5;x{*y7IpAKp_;j^CW{AU>)$vp47fr z44Q#0m;D?jfCApU$69%UeY%Kq`7BXB=xR6X(H zYSVcqIvQ=`y=N!MvQ@?CI!#`Tu3(4r=B7E$aJj~bqjD>jKm)3vg^@8ed+y=M`z<|f zw9JrGXMW=$RSk_k7#0TPS6oWooa@auhxD3x8}0R;kYJ<%8xpo~Lq5N!JOQegO#USg z#r?OmU=@Z7>65J_2H48^!l6Cwlz`PF7NBmw|qq2LiRF9OPA2 zoGCq?$BmPOrza#8vo&U!M#ds-IbQ>X4W*^^ZY85VL5AU*1AfzhP$auA=p9UApw*&a z0osBcZPTE|9t>>;4rw(*GF)S)l6nf-z(-gHN)U*I0rOlS$`A4m4onaN5%`JHZw59b zTx>VW1kf(YJVT-M)*-z$w%7_ckq)EbPAq1Bj-h1c#7Kxx~$kd zBa9&3B*KWU%8il`bjVP3r9%dfFj~N1wZTzlC153$FzI+phg&nBmkU|TGy1Fpb-qdn z8tR%5HFz^>yg8VllAs27iF*pvh<)q{xGUjnNW)lx$;%$IT6P<@2pEMU@x}8%YSGWL zxNFI`f+xYOF@qrhVMJ1mGw8IZ@oMsPDd%z2Acd1GfhHNv(7hZPxQz=1X4X&$Dj5cX z!BJ3+Sp%za6gQ(!o@TH~E^ZCM=&h$=6Lj29+~Eqr;dVK24Gf(~K0&!@c4gzvmb+-g z=G@dkQQYJ`UGorHmO7upn>5_|0y8U=D!fnbPYy2#zXG_VJ#YY$u3;dG#>X5YZIK*e zKda+4yk=y=Boj)EPjhsQ%ncni-02F~izRR(`^Dyod4_7efEn;4TkGu2uZB&*}f;Xd^JG*nDOZ5`4T++?8LL7ys^mG^CNnnlnLAA+%ahjxf#%WsNv$eH>F;jqhBV(qfI7Dr%V5F!c z;|F`27$j84U}fqslY}y90|`e0yu@JA-i{cnz?s^9G&`_4R3}zPnE9({DNAg$I z_AY)QVUpn5jDykUGM>?6t5Sya^pB+nRMs{iqAllel1M2sJ!Pn)dm&(&==T@}V^?NA zoG7uJXEUN)+6IkFD!!BW2M01=KM zPQaGv%Rnk80#`nfR-nk;Xzv4+Hqx~d_mpy6Fanrpf)T)YnQR%T-5wS?EQSg)3AA}& zG_}Ws(>{;{Yd?z#AyF9k^Dq>?U3+QBrBXnDj|nghrx16krFMtG0+E@<8RS4cV^2pJ zOWRST23)(PAgaZUi4!pQllEqTQh^-@1^`b?0>~q(oMH8Cw1=&J5q(B$K>92-F}3qg zPj2lyJg`XOa4ajU9FvMDiZioGQ^oOF(PQ?G)^w~qs;$kS{IP0F)t+3nE0rVo!0ukV zkqd87ZfC_-l-Ih+o|MyDu{q^0SFAES*;{;8Ip!6+Rz7{j@l{AbQ*x+?f?5FSScELP zbOb|7^63DGT7v4(iCXxoz>B5q*5Mkp=vR>*3O=YnktPgLaV84Rl*p6*3ClY-wR?nxnc z6)n|L092@z(j!!qmeNI3kXKWuQ88jlH&WqcO25))wDdH6R;BKhXgH-P)**CFeRCar zr<7I|vvR;6~UU_+&`SCNWMybuktlFA05BQ0CDj1-+&>>nzKQ<+Xw z3~18?qe4iPDMv-0Ds#{ryt*(VsYq94gHl1V%8aFAY@1dv6{4%GX)3r^S>$w(aLcl% zVvKA1pbAbd21OOp+%iY1faqe(R3X&L3R(iMn?}?+T3gv-OL(_3{!KLL=GeVl{x;)%>bC4YkW! zPiO7=mgw<%S(4wpUd`Iwu4$s>wpXN8d;b-6mcRi;+{3E^YwP2@g+Xw2K6h@368NY?z3R8|5iYX86@hG-d! z0VRVm=n##RU3ThOB|x7c?vM298Ehk64*1B!mrKFf(+oY>e&627uaF>In?ucEduS7P)glqCOySC0}k z7g#t?^z3jZ@^?=nyFZF0%v%s&;d;yf{;N9>K!rV zjKQGt2$mmaTcV@28Vp|iagF=bbE)6usWo}EUPJ3v8;eenL1HN&d?~3Ead^BG;4ie& zDfR;SY67Es#M7J1Om}LDyL)M#y`Sd}`4A5t0 z*I_&Q>S}-LH>kI^9!ZNu=owM>QL|dJtK~Y>yR4?aNQ0Kko2&YAU|5{wSkyIlMhgDM zrea7tuW0V2{Z$oN?VnTATYkG&&8<%!$HXj6=~R+TW6KAq=OqkO@3V4MJw@|#4$NKI z1T=@SfscfIIF!lzE3IuEK?Jy8B$Fa@km8hDnqa|#^AH{A;)^L>dNSnTAv33YrLHl}`-UZzfRJzo>mu`v>`G@%dghuC&L$=|GMFzr^}s;H zKaBZjX%A7IlZk`Xm9yaTUVLSs9r37*!U-l8;Bie6Hqt}~Xo{)!ODB?eJ3U>cJ~ZwF z*brq(>Q~4$n%X0XW3tS817b0)nV@BlmMsqzdWfyrWH<_Ef#E01y&hQ&4t>RsB z_qTS6r_$P!WeuXW>mBE5b*06SmQF+EuHLotP+NOmB3x*+7d>56oQij_WKaqineh64 z&vv||o=7wbNXPA?R8zFYp{vvb4Fldi(YLihGRpG%5^k_oQ|JS*VP07^TtW_srN=@6 z>u>awG3Jpakb+^XkJmHb(p%K}QT964?zRJ60~<{OEO|sdD_B-OZ;=v123aL3a1Ts* zx0T^Sc>^@STMPr=bfby5XDAlsI4zdbs5iqz{V$Z=R_!HvJ3QlE)HVbU6RS^fv;i-6 zG5BUH_vyZA?@uh*BeJ;S77R;l7XD)mxDu6Ev#R|?6_?vTre>1Oms8BP9-Cs= zWm*cGU(TTx4)qdf0aGud7E~=N&;qZfCM`5;iqyilsgNrnUhfZu1L`fN2i@AL8kW>M z{V<|AgoDUhqykGRc)9yi*Dg22vbOu0n8y?pqS4E-;y4zTA~9WDfB1rNwA%=sc63Z2 zk(xRqL$P~sLE0!&O`S!RoB#H^?(etM&1#}sb#L@UKVI3!OT9rBGITw8u&&A{ngr8} zM+z6|YZ#?A;DG(l+JKfNB>@?L`5E^E5@FpdOUV^!6$Y(?g-ETuv6m>^fpG~ESkV^= z9+;pObW{qH5@=k7k`UM-h$k>^5D%WXO>~Hk9k9*l^%pN-5mb@7%E)+~AT`NYR*+TZ zSc%a&)~*N)KV^#r8j>N0ENP9t8E5)I(t5?H&G1tvo#Jv;$jsjZwK|TDsx;$ka5*dP z^YX~9LzILm@)vxKF(It5$J${Nt=TNiCZa%wVsORC3W?watbGW9VZoec5CRr5Mych` z0*j7M!Yu{{GBG$QNlve%)Ew;@VrkRWQu7d$K1j;- zkjgKP7+z0BlR4KCn_#f!jEZ}5cbs)c6j{T(Nt5?UYK~9REcC_9P&}QLka7%L({r0@88bkN~r#%moj16z-QBC6Hc+#J{RM~_Hy(_%OR#=k0yiY4Zc|7)G z;|+}H#|%1U+R}k>^jS5+)B3QV1O`A6bud_SU`m&oJ_t-`hP?zkGq!9w`ShjG9_U$o zvbA(feV0TT!*s=|vHW0=h6Lgb*a-cKO)_HIFryJX1k6Y>RpiKWrwr)_w4gP)5HbOU zRLi9SKhj^ku<{*ZL{dH^PeWdBjY~N6dWY%z z)SFblOTE|IW2!lYo|~H6sOhbFRdbaT&r^%7_@<&4#eZ9Lr}=nIzYPgc?SWcu6z|9Z z9j{T(|I2qTe|!D*`s)4lYl_8{7@XXWW@O=AJ$>9PG;7J1OMu7QiUEWxrYac%K7qR=qHS;C|064$&h0Bg_t62IyS*3cLw z4vbAM7&zXpan0GIevA9C*7l%lT=PlkK8zzby2Q0*Q4g6ZszjX2R;_F6Gs7AfgA{bB zYpFOddhEnr%qM0!)6T_dvC=va9$q*TxO`D=61~~3% z;HQo&0Y*FsJi!hiX-E1^$vZ@Ek&@Sz)=CCv4>p&)1`~49Nsubgabsh`D9fTjFQWXL zYhHuI9P1VAKIPzoq-F?FLFR;g(Q7aW#%J6AGqkc#RQ0-UTEirhB%KHkgyrs=)-aQW zg-V#o0svuj_ZrK@DIy~xUK0l0y+)Eu%(`g}(?3$2%f!I8sJquNb3}v^S%2UMb@v)3 zEJ>)QXHGH&@1lE+n?!*ZFk-4?4{xw$8 zTNxAufoIuRr0!p1B?NPbx32DABMC}| z21*E9Gmg6P3{$MQg6y3Vr3*@{e~nP7G>ZwA*r;`%C<)j+1DfDCH?ZOQI?Ab*4UkVy zD;L<$$LPK_j69QU=(=4E^OCSO*eaNUDJK@Hb~VH;@>DF^BpF*(w;IFvQ_I|!(TwuC zRkIqrmUu8Hy4LM@pji!r#c5?!&1wj*923C2lMF@OtcLs0i4_*o#pOj)&1yJ0M$@Y% zH8kRaZH+YIGC4;#sd;jH6!_1VPvkwZYqn{7HJDt$7BtG4!K?rpSQ*`JSxiPbGdP#D zSz|5(Gv-##3}W__;S=b@(@C;S3xF0ZDJjkk+NmKqE4?@dERzr0P7N?C05cjIFf46A2gOAiuzuUr~WZ;hYq(m2ksdyGRP+28wYf45EhJLXy{ zR1(@z(zN{dmG!wH{DK}+H|78nAKV`IoLm%yY5C(fDJos}_B%fgxJs#0!~;4$$O0GNI< zbAe9stX{4FnGTa*mG$K968Z-;U^$mk3Y36Aq@fdmyYB}h5QlP!sF3MX%aWrM+HTPY zhBK9bCRiTJK<9#{g70Wpq6srHiEdE^RiNES^N8eU*|~Ho`8D95PrVz>Tx*r)B*{D$pvTgmzo03Xm#QOIcT7(qsX1lJC5448!#Y1fZ4LJF<&q@mMMyjFDo(y`hCF zSDeN9Qtg2l34nUv7={_@2_a%eCnn3a3fl_kk(a@(hq)*&pBAahFjOnXeP0-+RivEF zeaT%ySz^wK`ooag6-l99$~|dclnePLOzIOR9b&r*uo9@Hh%kj^(%K-f=uRqShmHI~ zdzzKVsa3IJz5_iKQeBEI6HAF>Vu%D*SSlHOrxZmNnB$qK52h(5uvSfFTK1UUJF%)7 zlbTCOW?DSyRlTEAcKXY(Gmp}s1?#n`xbI9D1B6_AP=L*y6 zi)A@#{bf=tnY_&sH?)8{Zq@QRD})H_3oOGqtjc1`f*58aLRn;~l~PQb`McaxrRsM1 zds(8pd{*jti>lmnazPL*=}{_x^VRrl=9NdLv@Pp~bL!FxvstL!_kv-$gh>!BG|-+- zv{as4a)1X(*@awTnTJJ9b1Bh4&mP;oMphxumC&gHN#;6`mzVlTxpq1!ED@2F(9~5^ z7%_rN;RMI7fMY4sPV6{{BFBOXcHI<)D?|ky5+S4>Q@J9%yg4oCd~+tOmKyR*&KYD& z2C|4%;d!PAB_z&pIAi8=^<@2`xRS!(n~3Q zGFj2*uzo$aW+}c;g^0czrlWfnlv$SSrx;-4Ra`|>x9<`|FiNvW=?FlGE{T?ymtch^ z<~9O+VL|H5Go@o8w;te=eNUMQuqsVibFTtU7n=}cy2Q7zYHYxT7OO$-f*`dy7|+Qf zRHUEMH-X!dqI20~fi%aOcceMY^}ZDhA1S%nc;v|9sim2N&5eJtJiVdEblCwSi8U(b z6l;FpH$v*wunRDY<`8+6P7{Lr=`-ZpVm~h1Q;6R}wo$b7ij|%gR1R4wgvueIMd^Vd zUWO>{2k^RS-4li$0>*`j{!Rs>^xH5D?{ma6?CVPxU+&PsS{0)4QC*@@cG=fMwz#N4 z5LHz1b2AXRn|#i+Cn?~Tnun0bN~ss1QRq#6(QrhR>Yl0YK&dW-n;%^yKjRxl>&nLBl zNj=HO+>?Bqa_1O&UXodW^%&?!Dd%IeG60uJc0r07ru)|)uimNtavljV&Sd$rgGn4B z51kB&3HYq+#%RCweC1s?Lb(XTzI6;kE+VTqj#61_-#SLJN$9Jg4ayA5m1N-@xfg3c z&`?)apTQyW1+A8RM)0#i=t+;YhR5;AMj17?8v~T?)D2WEmW!Eh39&*>x zVhfmP*mvThNFG;ByC~Pij!+tWVH&4Bix0hsn|0H$G)HrXF?yoY->DTDB{d$hYFFmB zEQ?vBSCG<0(=6o#oW_+zoTS>*EIds?$$>eR*6qbD6Fq5v25+RVdDVc0TACQAQ@NtK z0gGWaf@CZ}+PYQym5FsMB=a>Phm=&fhZyo+VGT#H$`EQ2O5&t%UdAwHlvMa!J$2lJ z9&4pG4mU(PWV-IYg9_(W_&WXWG(8{ue2Ho6FJ~mo&=;05XEh*ZGtJnU6@^ANxb5k7 z8P|Z@4#}CpB4MVF`>PmpLF4Wy0mutrpUeNW)dB00>6!II`@!zD2`2mst@QMJhQ0pL z`@FmZ_5F2=71`iMHn@cM%&HEjpS~^fYuaFE<%lt4!e1xGLLBI{$(LbWNQTs~lG3c1 z5v8;EZPDDQ2GSP3RP(T)bXXKkaFVi5rf7`RE=BSO7T{$NN=w~LsMpO(Dx>v#iIx>* zHcC-SyHb3iGs<)3b}3Z!d%N$F;t~^B>fWAaRicm56D4dVpM7rd3avqES1p^sGVY4~ zGIPV&kxmyo8x(}j!^-9W7H+wc#Xu391Ru?fZ5_)^hLzPLuws}SXA=o4w9-sdY$-r? z#HCBSOF3oVv=@SV+FNsE4AHGy0pm@`<%)R-gOKG^1|{-E@s$RKnE3-;Nob`eWvXf& zd^b1jdyue?JN5ll>k&TD>^qJyT&GcKWRNhJ19eImneQ0%SeH2w`BZv|Flj{mHLVsR zBdbA%q0u>nj_Wka9oO{iD1^tLex5?o}n7 zdBCj}A9-qYV~;YIbpfDx1)6O<^0MjX9cg)W&yGwB&7wz{ZdA#nnb7KLOPWqwRhwoq zS4E%le$e%z^5$umo60*)RjVpnQWd(&+qYRJYrZHK*D9<)m*dKOqgl->^B7eNtneaL zMlAEKRq3(B*>t_K%mL>DXPHYjdt_ves%okg)~h>WWIo+)gprxLs{B^Cf2kE$Bt)yg zToERvDqRsabs0M?s#?|XihNVW^s*dOW%!EBY*z6r0#+3PR3cdw64au7QHM|g0#&`R zh7zg(q5>k#LW&BJwVUQuSGN(bb}FsCunROKH}9RZ5Pk zAFI?ORf9I`RILKGMg3CcZ;PI$3g%Y2U9($-(hEz~UZqVoyHzM&*hAhrmX89XSKRrJ~`{)`=ZN9M_1LEvp(Oh?VMD9-B_ZS2vkw&Y))8}qRs{f zRlm9zChF4HvlXLNIeRvFZP z`?bZfS)X6oD^+=aW$G-Q0hEEX*-k*2S5;dAZMLmj8|X7yEfVxu^?;Q@SqGw3hUjL~ z2W7`rw^7D%^(N|!=7`i;YG!J?y`J6L?0;aDDzCx3$d#KxlSl2MXp@=paMZ+FyG9hb z*N&Curm5XBYUkAs9ld#Le^7hxSKg$VbJX4^&1)*(Rhwnio-f7vl+#RcP33cIv0UY` zQw&=<`xLJ)F2c5`s2z(+;uKG0mQdPl*^+D8>sgCM?Jun*s`9oj#qjK!t!1`y<|=Wo ze8Ekypj^ocYA7#s0Vm3L-GnsC^Q{n(a++rdRXf_7uuHk=6=+i){(|yUTtKrkgo-Md zF=ic$(CiGMLK!sBt)n0mMpi)-ErhKiFcdsj!5bRl*RdaMRiO?Y(JDs{HqlB-`J$=g zlrL)ysg8TmN>&{<(^l5%pc}39)!{nFoYB%d^;xz4T7?;D6}S#kYOB(9jFY}k9Ui6C zdKG5XG!N8KTC-81j_cC)h&n8+ZP%!y%Cvo?4oK4`mO8F&X`sn5aLTNsqUN+|D9740 z4N59PPuZjD&^~3z(h-6!JD3VEtj%dEGO^g-R6JwLXs4nfi=|J+Pb%AB9evrfEY_i$ z$}XvbK9xyR#f-KLqAK9DSW!!4sxr5#c-DoDRt3Q-yRM3#RVHB-e%mxCtLR;2k5+-d z#e%KEg`0M7eR*v)FY(FB&aR@Gn`U|y6s@fMI%!|?1}l=tNFj`N0e6~22J(GYxDy3g z-NzZ}YY5OHt&(&xc`K40ga%yShc$`ni>o^J`UUD*_WoHP1btTBUV(CMuyO`l zTSeWXLBC_G)dLZ4h*HPfu+4PqK6wiDD;${kSF815_?4g5%^ld6H&s)s8-7H#n{JSu zR4X^;KnDjBN{EdH5L9!+)achRrfUAHDp>Q$b$f<#d)08Iyi~hiOCC!7Q9)^>puSkPUwGuIwC4NUXBt?vGnhZiy$>2BmbxYAy|(DF&>-+~ zRXq>$a}G>Up+{O1M!RmUKubun4$zGq_Y`5IY-tenJB+?j9-S)vD5AC}2=?tV>(sUl z^%OSS7Coh|(oXc9bXACXJ5IXNJGEJpg9H`IqIooxNx~kI#D==wyi^mS>LQ94r?4(r zpVYf4sdqy>&{9DUxGtg3CrLRm1_Majy1M#)Qr%hjzo&W1wX68^edK<169A(4R22`; zcA?QofF+N_qYF*lh)Ju}kbq~g_uNFCngxMWKLf$}V%tw`Z*?mKO}?#`3Oq}^f{dUT zU8AJ}XFyDaqr2gRw!016I%BReDT}?3RKEjvkA~)UYM7OS74ttfrUo5;85U zu;&i+Sm0S=HI`MkjYS?9w8;1M(lK4r2}=ynoK2PKK%)h@t8dWiFr>l*P^)`k2#Ve< z;WXb5+(1ebH1`eWP%6@~@g-v&^|r%ni`@z9#=5+Xty>SY$pLuZAfvNk92$&7%EJ3x z7Tzs*5cB87W6X*}Zr@g@#0j*os-IUEUsrG4s^P8&U)O@y6Ihp%*9*~7A@qXQaJ=IHbE)N)`osY;BfHBq8SEvph~P3lmhQPG$ZuZlX0 zbZ!#267Fh~D1e|gni_0#!z#d{_WePuQvoBgm;b+eEd<$EtP;X-C-=G6!t;79R5hw2Ggi41!Nc;mC|skYQw3}C|Nh-Gu(mV*G8%_GN2H| zJH|%wYa_as(S?(xrVpL09qx7^+eRAi`87#)sOW`w5llU6|IUfYf`A|Icth))rHoCS=X1=#9h~@ z)_bE>>soCweVcld>Q|}vdV5HjL8uu~yrrI0#hIFGQy80CTE!C;btvB2q%y^$6}4*a zUQ_%b;ZO`vKVyY!-SxyX$c_bmy^%82?VdoXjxo*xUIlPy^aKh6^PESV0o{-d1XNET zCa|$`uob^dPVOtsfdUo|kYHkDrH!%C9f(m0kscI4O}2H_-GKz-@*a-S7Nv<=-5qFT zCcSMm1sdu2r?nn%go>PQ3N%h>1cPBO=G?wcbq8YXH1%T^!pFXBbO-VwdEExaoyAsn zAWssZn}@apf2ccb9KqG_tMizLsmJ=^eQr&?FnxXkj zoSMKqNYx&Q^~gZPcoM@XhDLiJmbqpi&oF`=lunhZK@ejWX}%qx{n7rfI|Wfgplt%` z3~eQimO%`8r;Qi0FHU56lIj`6`*^Sf-e+R*d(|^YqTr?YHD}m2dIr_I=dAm1Y;ihL(1V-~Pyuuvl7{u4>Q{)tMV6pc8 zwCBCR)r^o}8Qw`+BOGcOBvtmv;Cptz92y;wjvXLlQ^44nEDLtjZW;8+2FN%LR;v~n z3N8AWQhUF6F(j!4<>TZVC^~oFZ+@}coxXbY;mvRF-(T_E#Nv-DV||F2{+^5S)pe^q zz%s2h5lkfzNer|XFs9eC>84Hc449uj{Xt*>_Ob;F=-K&`Hd`MhiZ(JFbg1ic5hNX!KwRvuteQ40!C>1j&YM}Mq5nfYvZGTATAkQ8q+ z8*;7*?;TSQ0t>K^qw~pc7;=EyFz*U?F)6~^vg9hONYL6QOpUD{0fGjA7BMbE)=B-0 z-^*nIQJAmLdqjYE(nO-wEFln263bKHv zVXN+*0jWl$%*Y6w0TUrTc}2AUYT0$o_03nDHA! z`37iXB*8L~YGjV;AZ>>wB9m4|iFabiumCJpolaq@6uz?5o+Z2q1S2DDiNmNQQGkz( zJhlV^sd)|Cdg3hBBo<~TP%#N`>TVHq-V@t<#(P2T<{)r^9E?O{>^Ek9vJ!G^q;Imw zZ6@{@MlwW3%F_P}jy=gpuxRWqD>@CsI+)ze&?77L^Yc&u!x@qj8L7@t8a5_o;4FAM zb}43x#!6_ZbVB1zMxn_-e8UJFhU7lIJ@!uKc6HH_fqvNS5#pSsHa1w4EmR_=Tny}l zk@}~321VoFf#8)h#)>DLJzyp$6|uNh4ANmF5@S{|a{6s?Q?`2%ls(K@RypD@wC%g<*AT7D8nk1UyFq@3F3UP;L=J!#}jlF$7kmA#b6{l{KU&<4lfe zuo7>K@ol*eARAbLg>f4xt%3b>n2s>?nYCaGI>z;m4Ethd$ysc4j0FaB!?J=y`5=`sSmT>QiBavZaMCA>+{K}L% zF%Rss%xNE(OT-v-?pap8WPCmw8#8;&Q}EKIfvq~y_aqC>GU6=A$2dQS#qXs*bg7Br z^NUL68i~uXdqIAg9bIGDpt%G@42_xxEV|1T;ltrPRk-RY(XQkKd_NR zu2$tniG9bD80RpgeMA{-7cF}R-#^lcyl8gGfc!1phQcqBYp-JPGN;E;{tu0Shi8dO58EA zqI9$+8`p^gGt#Gn-H)@Fs=^Hkp$5je8JT*I=bI$@fej?whK;yVA8SvmD47i>n^A2+ zOhD=O($yP#8}koYY=u)W4~!E|YZ&e`=H_Co@lV!fbFK`V(@`1aD>B4C@&vEH7utfj zmJQSQ(|C0W{qISz_1MPbWMS;Akza2dFYi-89M&*)jaj?mVPEYe1XB4p2I7J+3Sdp5rB zrNH|*l!8&r@?ahyge7~5;4IH_ON8Sz^x;Xgs{??tydL)IBu|={3(4pwK-XNTXOcu} z$OD#a!)|euGB%A8vniu=oCkIlVLFesDPxT+u@{czEV~ZM8M5XKg||7Kk}Mh&LXinH zN4eCqOb4!dffTv}GFBrr)N(n3s7V=+QJ#ZHVv$n_f{*=Ph|3rp&9dpzrc;cbOvHfz9i;H*7$p;iG5PF!NoSN5%WC5H0=XQu%s8e6 zAkj1R2`D!gXo!-Sk0)VKm?I!X>YThNgN#T+WBFM;EhDV`=uGfEoHBYW(Tj-g~H!NdkXgq}b>=@ydkKMKSYIuTq&#pT#GpO0WtJ2q^ zxx8!^DvQVbPSTa5#oWnc^=!WFsE85+K(=I7IY@1}G#7)|vS=%ifD{79WD#-LE<<`N zHpq%J(2=V_@&n)zR|9Iuc+?rur9@2jA?tG*g+8kuCYz&9J-8_&Cg}UuK7hJWD^0cn zBLQ7N&1HQ}!t&5>1MTnHm0+bRQ}krzO#D}`qRwayNuO0u4SUbhyM$MviahA)WxHK* zsd(A|#1zw9rrNckmYdcP=Bs&62C$$H=FwW|-x?++K)0yNGqywS&!7v_AWqL#pK=CK zZ-c7&0)o>$v*B%M?+JNO@8v-d#Ereh)w48ldB*MgSnp0?hdiZFgv55$+~bk8BL*r% zBa=Hdub?w0-ABAEronDJlQ1zNuoY@!ZMNlI%CwKrwirRA*`Jt{!;~8>KJJ;IEO7+PJQPWtL;zhTM)Jvl6d0VnbN#gSfLrX?zA#&_%qL!hVk3`K29MZ805k5&s+&w2=$G*B?}{k#^;o{h+$7?Z{& zqqimBJ+T^SFJi)bNhGUb0`Okyd>lC8to%?z4Zb7YQmiR4rZXCl#WPlnox8ylz?p)E zJ~Vxjl5qy(0|pJBXhlkq#72s#w$T^wW2LuZ&pIQ6Msc2sqgV(dn+Q3L1i3nNR+EAZ z3)VFeCEZ@$aO7N7Da~CWQ(u&I%sQ*=X>?259D7qm8W~1vJ2f~?6c>^~5&VtBuR*qF z-gl}V7HR9%VEqSqV<&YyXgep?(x9Kalm?i{^lK;$T5(9D4r$cPjc;fj++dNVQ>Z)| z*U(+)zLea^l)3M<{zFEJ8KL!)GQc&}C{Lp@%c0OcJWw^yN|>khfe>h;!GTyppfNbMFtj1tH#So+B8`cfR0rX`1 zj00dpuBMd)x@wLE@Ijo;%GyGMx@-Y<#%8?>Wv210M^2(+fe*&UG|fGn8*wm!=`om9 zn~ju*zP3Dx8s1}si76&mgVx(}53iw=flI<72do*~)Tkd&pExdRqB&yMKA z#oS1{k+$UVK3HSA$#4fs|JoWzz9I;I>{)E8ZB+DGcQ2+$I@bbZEuCw9kS$SJF+H8{ zph_@UTs?k#3fWP$%;s3faB&vAV3@)n)&3jST*r(__BCfs z#a;pp5c#_a%2N4ZnUU=6s5GNt_NnivcxV0dy!P5(umX zOJv%To{V$w+ z2D{hZ8}T~I*1<}Tw&4Jh#^18LiM8p{tsP0}wSzQBW+^kKFApI{efE@dvUDaPYccuD z-l7w5V9QMKBTNqSt>}22_8{x8m zO`pJ~k&%o^H6pvCvyO<0#X_hVvp9)NnWH+>v*bg;%}jwO(*R3g1id^-$PG`Dhyc&* zWSZCm!%gV=;^I&yV{sTLVn|6-J)|e|MD&>A zBz;p}HK%0ilRgRDnIyHET0nH5IzE8msbW0UD>L{Xh$|0!Lh9dCQK{aDKHl#B&boNVb?POf}zt4q3{h%4PGQB zk6sL1Ici5ib8{x4a z>Wo)e^I2(y`aW5q^aB=a)D4{)`8AP zWs-2*7oJH0kZ0Cw!v?vPaMH*Uc13xs4J72wG-9(O8EY_al#zvJfK9RgPaw+ErW(i6 zc9E$wJwqy=}B48yu| zG6P;)!T2N*pLmjI4ways+M4F*v2zkC!zhwrqbzdZqg&wv*x-zu)s{WSVCv+WWKMm4 zkW%0C7l^82^nj0XwEb5Tk&$dKS1-mSpGzQbvJ~!lun) zytYV#M8)I*xGnUyDaLF^`WNYGaFHYjRJL$Pfi2TA2pbu;nRi}gQ+G^O5J}3;g2fN9 zw98{&Brzqjiv(?A%Dyl2KFv1El~cemiGha5%AG)}ivrqlH4(Y;O7OYSa;sV|1$pLx z*rj)a_o2@uO5hLgQ+r6{?U)Qhs^D7Hjuv?p20e&vrSzn!{53=zEZU1vOKI|r^7**` zkG*%@wI#W(^qzq4u>Tm48mObVXQuy&W!WuT4GU^PwqO_q`mjuu-DD9IML}=x`HdO5 z_Fge#Yadn@Ey-4Qlg+a;R$e1w-o_jQ5;M7_wtdUBU28{jL0R~y>6smQ)DNF3P;CCt zSDX-<72fw+#cK5u7MWcY?Yz%aaXfwYY*63A1SmU?YQ99eK`@)CL639~S2eaf}w+YnG=G9QP1d2(uN>6m~H=owxy zoow@wmJ)P0Pbg!DTZtofj}%9RSF@JpY@1{lFT6K?RqcK%GnLYRCL z;Si>6z(;Inf52J9RkR{C!b%K?k9St|GjJqq3;}x*HkW{5+3isQ3bXEX0ZkJ&%ZbI= zZnXi)6SnLC0=hT*fFXKgAp!y^*q4A}3N|XBp|-XzpsIq+49KitZUew;YqkSWESUcQ zHd{T2fLYr*A_4mr9GV+87kr_Bx_jeE1TQdBA>$$bLbPZ}S5I zhaS(eV%bBkB4Xk#8oO%~g1T4>e?S-H^g5i)zRab%Yv41Fgd(HMGH))ZxEf!Ss{Ln}?ln1(K#JJovk z?(FijYva$%?cR+-Yc4o+7j09@p-CxZrq@2FklYUKQrjeXXuAr@_Rz3(=jlVg*EY=` z+QmY#Aheu?f>wU#Q%~CcvfU6MG4FrK52ZEL59~ zUNKurP{4af2SS72LRV;=LqEEbA7-Hn!uQfGuzdqrknJe5W4u`Vb zL4}0chTRS=m)q)|?GP4M*aV~>HOCU6oK7_yvFJHYSE(|;nikPV$yDymb%+Mh9(91tjTP5JnK#t85~;aO)zFp$SdV(lkqnt9z`s{7$x*-r1HagMa|2Uw>K4EPgL{UXS-s4y2A^GZxh7=Enfd<>M zo3*cl5pS?Bk5@&~BcA@uE`g9Lso}o4yLE)m*7f1SdEyJY<}5<=jb4}!U-awe`NdBr zuHWJVf;#cl5;j+!!D{a4!D6>@IH{jg>f40t(j@~woxotT~ebwt+ zqs-E&Tp#4c`#W>iCbPsgPPFpiC<~)(NB`73&6x-XeF!u24J^U-a!n-)A*W= zcQ6-se1JDwLVW)FSMk*?d(if+ud8aAcJvvL1^eapAVfX8iDuJDUI3a*aAE95T>9yD zF>QXn%&*rguLtJ+e%>v9+n_$-?wga`ELs~cVeXsXqmT+W`rPaRYMFqMqr&i4`=b-= zX(fiUgqIrbu~_?gU>k^06x`Q?oIJ(rT9>^R~mnw9on%}f z^iVdt64lu@z|XWU+>loAG}ELcAd)sk8#*Xir^yNsYazWl2w4=>o#^Uos z*;Mok!Eo-pPCL7BS`BHk;=nUC2Bkutu_>#NlJaR5Kb2D>#2B9>!s!i-iapJnhn`J- zr-gZ(W8{24#`>`I?bJkGfOxd(bqLo?#aJy)P<*21H7gM~erMCiC$G?j!@XuL-(tEm;cC*Arkya83c70*r zsVH|sTEPo<7oMa_a;ZD0%-#b+f)})58zTYl(vY~Z?Ns&ENT`t7DaVriTF$~()Js?| zf080>@7z6HePn<;%t-n5+37hVNoKZ%48-QGfq)S^pR_#l{_QiW3)oZ05?=A+b*BHh z`R?4p0Vif>T7m}MOrI{nCa`d}UU_$67WQ+JS9M#}v`nlM8NAO>{keJL(PlY7R)~f# zJyuZlJQfqYix8SamUf_owJyWQ`06D%N<>>P!&Gn#@&>v-e5%N%IN&qZF5i`6)N{h$l1MI?^ z@EN6-iqOTN!1fX+0@WFk$_XZ4@Q7hPN!gkhm4rr{&v8E_Wh6T8NVKT!Ak1Wl8fpGa zR5D6j5Clk~WwHS26>$(2ma1%rO}EQlY@PI(PfKzn2Y22J8B#5I6+#gmkDgJMKkXjs z)5(ce(U-7oRI-zFT>wuaj~})lp!;AQ?3yE8?bX6QZDO1AEE{3Gm32SeK1kS&=i|qX ze0_7b9oWA3XhxRr7cJ2`1)@RQ&7f!)*O?fN>oS?6iQY#1!NSi46smZ?)R_+Fyih8R zId1=FhLh-wG634|(bS#q?bj(GA4?cWiWj_i6HAxfx3r`SEZo%DFN`#@A^jea<|%TQ zknh(sZbf+>RJqM>h%C)#DoFO)5~W6bb)8pPadNb@dWu=kcBMooYy5v&Afg$IT6=?O(Q2_>Fi6G5|$~_f+8ssUwL`sDHXU ze|T6>bv$hGWs)&WueTDiHd|Dw8sJdAoj3onE~&CvFcO{>05z`5#$Eg zN-Id) z{_ZOK`X^t&(%dpA^(4JG(T8@if!LNPbU76FbB({*n`ka!YQu3qDVs}$+reg@%{zO$ z?d3p{ncnYYpG7LXR2(6gEoM~|nTQ+}dGKULzOdnv$3Y7lNp}0rY`B0xsn(9e=7jS# zL#<6$V^vu?Nt_DU)ukM}SnWT&mJO!w5=f+39ea?kdZMtD&5+5y_Q}-@cKD)9M}sqU zPa0F0Asz0O0uBF{JQy`8a7NJK+vGb`Z#_!`7i2Se5;gxAPxcielb`H$<&0-v(P}Cx zQP>V}0AKqOmgXaWCed-fx@SQ?bg0rfmW5JFUCi9MfNlvs87MIE}k6WJUgVT9z+t#D505g zh0jve;AlZVHkBDd&k8?10rTPGapwfA+J~`jkFb_fS&~ERWVVVwY4Nx72)jgD%$;s~ z{-i8Voz<&ysdL9{5b!ZTr(!a^;lLe4lRcdJC^|?;%Jx2lr0kVfm^cC=Yq_6lInumaM zrDerjG4~m^bsdmO1sYoUa+y~DDf4ntYyfeP$PtZhFab!7)Aclwt!>H#0lhHqI!(DX zbW_msw%bIu1#cP{Fy3^7#vbLZ;;m{fH>)e5#H0tdVQ6+&5YCca%K%YyA(-_6p3wgw z88LHTb5v zppe2x7LG<-I0Qt^1)8KRmHb}mQif-Lq?rwa)Wk1TVV*f8BnLp*PGgrQ>!@!6brBAz z^!tX=P4v57y*O!*G$WC3=u((YQwJ0*_SCQx{|FDBA$qHNqRv%OBI?;gUx39H(%^d+ z-3oap;XXxBQV&5OScqVlrLu-l1j=Zq#o_Zp-cwC45H{4$1C3IKW+5>z@_=%PH{$DJ zAP7gMV6#}hD6IMeh~J!Gfhv-{uVO9tGT4f>(8Fhw?j$Z9=2#r`M1U9J%b6wLtKW!> z0fsu@$@ZG*M)+aZ`^o~{rj9YH-1VQ=z7g#7K;EMM@W+E>vLo+#0wvsP-~4!wcL1T( zoKkY|{CDk}A4U5HRIbu~m(q20Z>=bWs+Hsd7FzPgT3wen4`1NeuDY>S=EF3}OcPnx zcZ?OuEcVr8+eS|3##kLy{akn~1-YzfwaBwm{DIJt$+Wjx0xX)50I+Ch+E}f#iqOTJ zbUNh5#%h^!O3%~(6;{mj@i0q4e`3@Y&$n*>pH!*a3Aj0IrMAPj^e?bjKj^VG*6I)PSpTAi>YtxD|Ca5Px&COw zpX^oO!h}{pGAq@AM5@db-HMk*pse_W5apSi{;r^rMjL*gVvBTE(sNY=GilHVtFuib z2IW?&spjoE*Dfkz@EoUE314%&>>FX~K>$3gk2 zxhAS^WpMiiNK6;6cWO6}8*NfWyyHTgQx8eB;|v#X2e~?A8z_etnkF);UtM*rLD~Z} zb4^ZVODDXflI>kjzmnIrv zXGOE_!2(zSbRYO2*XbHwXA-e_Q&%f#$+J|hR2>}&gL4&vScZ~j*b8BXdEl;XRmf#1 z3trXbZc(BojfFI&YJdLo<47(8Ytv5D1jApEedhuMqKJF+pv-dERyK=g2aVx+)#gE) ziItvi6HS16Dri{Cx+e>$c`bBwynMLSexyp*SC2Y;r{B=M>WCgtsmtEB?b?Wu+8(T0 z7p!1JqQ&OaS6-&+Ho^slEykcugBBZp0+pPv4o-}nrfON}c zg4aP(0X^LST0o`0TE6({vOG7^hM;5)C{vo$L+OQ@7VpP;x&yzdbTZgEACwM}m9(u= zQ{A5TM<{xIG(Y|oShF9g?6vBTA|rrJxJp(3urL&tGj!Ybc6bIc*}iYa?h?XM=A*cb zcu{+sD{`6(j_m`@FhtSY699(_VxFH=*A=(Lb)mSzo?;{Y0`;~};2z~85 zzR!aoZBu|@==R7H`>jAmWw&xk$f{&D^+JRWgRpQ80%&H*zt6bB3}8#p<5RwYmQp|o zO8tp0i=rFOWS8iOOVlQ4L@S7D9~uLXc#EEu?0aSCsdm#+u`!fh!BwcCwXQswwB>Uu>+qY zT?9ouiFbP0tWhtR#oTV`u;&_A8j7+_CszOol~X^bio+!_i0Wb7n9{M(LhD7?yOBH_ zCI*t`1ZkX?4R{x7s)d;!;2LRPd7}wLi*BuXQ`Nv{v+oYufWQ6I)3!y_@t*0OB*q!x z(|LizIk4l;XBJgdNy9OfxHW3-J9&>e3C+ouJgJpX8u2=z`369L{Q}qgNK6bKliWfV zF^om{r_WH@V{xqt0-|dFV+zy$ckq+ag`B0O9k{nh)WI8kp4n4d zdaZuDT1|9MWF_*s_uDzKJ#-Vy$h9CAzg;(|boJt{BLEK$Xyc@#bVCmS`at-O(MdF5&K8Zk$f>R5?BI$E*dmuz!{I_lW_ z=scUc3aZT?Q8l}>4%!4Iq6@B?B3-V2Hg^y1gg0?D2*Zq59TWFTkNop}u#}qY!=dA}5Ryc1P5|Z*1)=!_ndR{jtz#|I&Ese*r&T)Jn<~A=X~=)=xKY!vVh3oQ1hq{q&|JJPwboa5mB4 zr}Hig-53a=)zSBUx^Ra;A=t5d4{mxZ)th%vvCHy`7B?MIJMk&o^IQrkHeNb;Z`;MS zyf64ygO@IfJtfr(oRnqnrQ2TlX{d^j+4*wx(m#B^x0l{TOU*zf^2^>!hcv8-DQj@2 zIeO{Pfr+CD%=mRiCtb8)Cu-I^j~z}fQRs>0Y~<*%+aCX^5r9oc9o9v?3((S7 z$ep9k>*&!t7aa{O>Cn*(kYTHz4HuONip=R5P$zEPbWPL)FTf^hJvRP2BQV=~VPLUV zcb!kFt|7^sOe9lq*SQ^iEgWXouwLACW~yjGv2r5+UfuOZSBwA_9qzdNseof#=!6+aj=FJwPNgjUBX_twmtL z>eY4tF~8pXa}ijmaBVQ=&(a;TEdsO1fTV=lukb!FgF!SS{hhV;tDdg=ylcrMU-IPzz)`|)Eb{NMkJdmOO_@uv1QU!DnWJBs{S-t0}f*qCSxLEV8L>@^S zIGgl+fu-iF^RRR{4~$`<_{%Ht1#!anzA650ce?+I;_pX}cC7w><>gjPj?%)$?lBa?bzhF!|QxE$(^AHvsGX=1u4#B zpX3HHr4IEWO5`&pxg8Lt?F7N5q9iwvE~Rz`p8n*u4M}cJE|WPr{6AMra_c6^^$e6T z_^NBRJC})W=lSmz`jGAh@guq2F<80BBi)_raAr0-rwbX@HQg;3oN!O(z?aDzeW$xO zKVQ<_S#nm=JwB2ck=Atg7E%<_-MlMGbA(1VjYdp&+dY4}=x-csyrsu)_a?+1r<5wQVuUsSo$*j+<4{O@o4pr3nz<@?u`%nNu^Ls} z72Iu$S?`QbNHjV`G?@2U?@TO6GvFxN|6|r00HgAEA1;osAA7uT|Nr41v~9mq=Rpx8 zy79{*HXg!eg*#a8?zNdGuwbeaJ0+&n@nSKUd%G=(Z$l2fZ9Np`?S<-ikNnD#~ zIl9E}vm#C?ya5r?9@=MQ3}}lC3at3!@*RKJ%VVc*>2y|c-iyt_hSOWH z^v!8@>8d(;Lp(=5)k6aTA9RfK^fRnI2dXnRC)85HO0|(>)f8(ferMWKl1o`C1Jbqi zFs%t^lhyC7CsgOggcF(D0A<+;gRnWE_AH+&VFrwU7q2`c|b2R3}C`oNL*cZ7ZZL0?{zogQrgbFMjdIfqU%teEQ8odE{A3!`PurV{m%u(xg6W!b) zZd;Gukqc_wqXKjKRg>LO3GRY#n|s*}t+0gISHX|g&={g*1u zExQ#k$z`S+XpJJ>U`x5+CsKBTj62We62hZWj(H^~3sqLZ^K-{V+f|{9h-?G(CZ;SH zU2LqaYWwz}T5RL$53Fl;$(c(+JGSTLAq3KeKhP8^dQeK|zD4T=a(;Lm5VSzKMS<;s zAgsM9L6&SXs48O?7*}})rCBm^B3U{?5M^GWQ}4>iCyTZRQC19XGW;q|K!mXpT>9ss#4i43HrhZeyU!+;GEADVc#CSr{>lH-2si0ygIrU{u8SNy*>pU$S+lpH-wi{_Tz2L<7R;kfA+#^> zdBqPw|6on7z(Th;y|kBh4WHbdE#Z=*eT%B_VWE|<4-2)mJvGoc^jyK;F}jMkm&^ev zM?F|}r`$Frk@5en&NG~*-P~xrHam}NrV%CB26;`Hg4F!sxRBj7-^RjMER=GnL%UL9 z^*4=RZF#n@x{L~{x~I->JAl0#DEjGZmhc5dNw$BId&D;oR45Io{@@dlCv&`Tr64@F zjr~%F^y`@@mzX62j*UhFa)PPL^G&A|F#t+Sx&u6zn-RI7w)2{^S)>mv6+a>x*1Sg0 z?b^xo%^Zu_?P1#%6y)fG*`TIk*2T&qa}E=|nr3ZdOrZF1++c-b53~I-i-%skpl0{& zG3i{iZ8Wh(!O(IWf<8f#rxY44`Ka1{23qdfZ&)fw?CFI$Z+c@*vKeRYtOKs z12eE77InM1mKcp=#x}`}CaENocpVUE2@9w4CJm>y`;6jALmWw}9c)~3xGpA~jPBtP zs=f7ccZrMtzOlCKqmghNttAz)SZ#IsHz+ z*n9W@RhqVve0m=xMeW2}urJt@ueB(Tyk4TG8qX>@D~xS~X1dgTe#~>guLTC8RG-(# z$g~L@I|!IbgW9x)=qYxHIXZd8J|SnpSDZQ& zgkDE2Z#tpMp$qsVfDyIq+MA9f79)H0Fo>Uesc4MwQZ3NV^3dkiiSo(a00r>7Yc82b z*mNGxqYTpVO*fh~54*(opnJa#@gvLwFuYNUvZ(@J%lzo`;QCa2Bt`bI{}{H2LxGi; z-po6(y?8nkvU%m4n&6 z@z72kzFdQSiQ*y!>rpme-)<6cHklG=7T=jXz07Xkgq~;KOu{yEn;;p~=oKb1gh%WF z#18VwwhN*c@j!4Tc7x5rqQr?IMXxJCva_yD1bFSOZWs1I*0EBGNOJPS_CVQBpJd%x zXi01NosB6|#y!Z}nHfnrZg9#H0q7#TDbUKgNjesbq#U-HzWU8xYkrFRv+G7pSrL8& zLL(|SakHm122Udm*+5M8MnjLu%*#+BVntWuV{<}GUzgad&NG-V?h$%33)#Sm?~Ncl z)4+L1LJPDMY-mBw0sIvDG?d(e9WGFhP~fzfNd1#rdtSuw3GmJ)?saYaf^yyvC_pVT zgNtubY@4=8U?aezYeaNW0NBwgM}u)y;qP=>JwYUl^*d(;|qSIMt&~0<+=n#qo zzm-f}RzA>Xn8Jd^{q@P#X40@Xv4uo8x*X!>gklfj(uWuRJKd)l03`Uxg`p%gUH2+Q zDfSRv1*f#4W72a3`?5w^-K678oV}C&xS$wDf*#? zFGyk*Xr^s~C-yMaDUpD{fuC_#Jg^Jf!l*my1HgdcAjNqhAQb{e=<_Cn29C+Kk2@Pg z9IUoNrY~>8SNp_rPHfdx@DE=yViQV-Q6VIUa6+jJkgCW|g|J?n6{u(dCdo?AY_bBP z9;Dq8>Os(Z@jNipvaZujDu3rA;1fB71-30Jy_e2t$0QXg@apc;B6g!b>#26LEi8!@ z`nfgj*n(Qyr)Svky;KC?k0jB|?gWO9kQY)G9x{tX;DrXJEwXGLQMKK>qu?K9F5|#&M{7pKOZCUqOIv=jbY$(ugeKg>_ zE!UV&EvT4MgsMf7Wc$1hxA+0MQ^qBG$gSy0;8pZ|*J!SVoL1CX_#s=BM!cvh{ zsWC$>z&Tt1kU!ejjahDYiVk;*Z2IoA3ggpLRHr8mrjqs3Z5 z)ii$|KmbD(~vj7N(oGEsP54M9IVD3Z;cDcwFF8T|>;? zOgjK7oUJ~xxymeU886QNAn>)QtwY#Pow%K!fLjdHG9LYbms)8rJ(~4jE`i2 zf_iLFEw?u^_mSnzSk$978_wR@RfGhtzZ!qSi%fCkO zJF{(vjq0Mc0l2N1wN(2fYI)x8B#R+wO^9dOX6r@K2?#>9Ad$%%_ud1Ic%BL2z)|1X zdr^)B7D`HE;6ATrpkl|-DJdRIKCB2}8u?nqwkdN19~XGCLA*r(ypo2_Gn$vtlCUGl z3B-WuTn#qQA{ufn>wiA_iH zoezhFJ-el}9ojqv7uW?@UAH(q z9{#xIDc6_)g4ct<1IGhaz7ShLMuB9A?OU%TR3Sn^+#bYH2yWn+1-opsuO9H0gdj;G z_dG#`&0xO$5M$YGs1pydb-*HqhDFCm7hE)!tx1Em|ipA;+O6q2ZM0lq_P3goRtBcc9DVN3{8r zY5`5DDQF=WZB8#CUBd`9QJCM16B^S#+G-%(FC^q=Si5GAfX8_*O|%+LKJb)x!bZqF zEz(TQF>O`V2$e+#H>l&@^WiD|#r>g9aAg59Vit`HXgT~&Y0G8R?pK}Je^fgZ`lW5>l+}Rzk@O0DFy?sbvKH*LSG&l<2M1ws61;2+adO2+86)TdzPA z66M6CGgPx`TFM!L>pLi0lGm(AWl)eomZQEVz$K}n@bWo*&Mu~w1KVoRrkv|;Ny4+t z+=@0}Yu-Whl*kwKk<1F!e0J)!9-y~<>6@I|sqY-vknzVF0+SPJ?ts3D_(mPb?t-BLLYt0DJBede}U;DZlkFFu`GL7oi6YTP>2DP>+ykCi-L1*-2_^Vh^hhl(HDrj zh3_;=4e=ANh=Ki7S&;`}#;xFmIB8c9#D4OxV2ZeiRy@XI{RK3~cKHTW$h!DfU`g0M z0zl>MrW5cjVXF!lnXuihNSp1h82~%?rW>$6VVe$^q3w3RLXTFX5wJ_a?%dF)t=S65 zsoUrU&{eRP0lc-fy8#3i>~{c?1s@>b(cZcc0nQdYkN}AbUQB@2ZM~rznHQX?0O|`4 zS-=Et9lI4;csqpwlo81wjAd|>f2aEtE;BgU0nr&8^?(fB`TGH2 z8p4JEQjMX zyWn{&#X$s1eR4_UK-vBzZ1 zB!(81ZANlwbP0LQ&?*yhqoL*IZE7`i?1ap0==!;Hxp%M7E&&`njn+JJ>{eQH(Xs1k zm(z|tQ)_NKv|WX?dT8I;rsqQ&Sjg_jPO`N^5IfR#m4k*Lp1a@$RW5y5D(Rv2ciAjz7Cr z9d8${_{u2kaj|&W1_WDpt>*lQAAGHW%ELJ*7U5JvPtt3wr8XtdY|0#n%u(%G>#^vY zh>hM}6Ir@lRX3hxEe^*!SqseZiPl2(=k1$W3*L|Sx|Yk;7ZX0qT6T}0cCFk$X2!1Y zA1BrdAc&K5#UR8vzQZ!awX$L&;(A+Q6(2YJ@=V?`{H9H~($?MZh0}>f(>f)7IgOS= zg_ATiw9?HYO7j{ZBI3F1Lmesvnw#a_=9Q@fWm1P!343Q&K4IY35x&Ti2yJ-$LUAjc z2A)J*QPkIQD(rfa{;k{0!rFMMheora&w=Zydn;Q`c%1| zBP*>rGhH?L0y6Vg&naK810*8!1G`7+v7cw|ZvQKx8;Agst= zt4w61#@X_o4 z{QR{N)5~@kNI5bfiOpgouz^VtC`(fHb)6$cIAq8qhW!6^mJ!EgRnLzDCGPzs#qn38 ziO|5{*TZ=b5)gwCOi48^7wljIDqa}_EM$ig3TIiCZoH9o^~VR=Z6fPy31^8fcG<-~ z{65R#TW2Uu49ces2vbhy8IGeyl|lj-T!CpGtw=~B;#QDzY-PM2pbbR7fSk0}r{%Z# z)iNnJ7hFfIH~x9IZO~iM#X*swo^e*MrzMn05figxrFQh$Sa)F)tOmPmR##2f^|I4v1b5TM$D0oPH}} zaDy!Q0rp&<)?8Z>ZRXmZMH?@yIYSS)CjDAC4*qcQ9*GYMJ zWx#VX*>nD^JXKjnI~;6guc5zM7B(GRNy0Gvxrb>{S=F|i?g8KZtEQ~1#G(Z-T(39C zGggsXQoerYwh{kRGAP6)W)n?e0@=sDXzLXB}E_DOe@$gHw*!({Ho`N%U$IU@GnEea}i z?1Biov?Fg8S04^?=QJp}KGtP;*xjmbu&|v1-5?#k*&8Hl2Zp6Lw36fYQrfJx<78dY zp5B0vfBr1tMo>jW30D%#$Cu8$14g*tWSlH%*Y{h=MAqT{W;(Djv4k*d>` z)=#qwwBjeOl*X6@vRN;2mQ*MfiEL>&Yu9GB!IDMYlJ`In>Yi=S4-T8yEG9os_?jIY zWo84S>&?O`-3y%!Z1b%A#C4}4A1x#WOv(8Y*REYj=_?EO22i)S2LL|9PoX?oZ(h-` zl@wQ=qOk_3Z`_0=A}YLIb}Zbzho#AnN>&%g=r+#JJ74&D*gV%e2z&7MLGCR)yoa#q zhj$r_$M&17HYj}B)r#GdVl{!=DHSYfn3};Fud_QC_3a`F&Oumg!LJAlc74+Ax(`my zx_^A=Cw<^+v5kr0F}?Wz{psKRWKUUo)<2#@@)uH+qL)fT!Lav>nw9@}{UaQj9R}m1 z#HVfk76J^Ve=WQ}#w}g-hDGx~yrbCuZ-Pqm=|%5&HRFr^ucf8m@@NKL`oNx%HoqMI%EbSuO1v;o?s%xv#D;tk0*qye{u~B->H$=gfGpfcj zCB8_#WZL5z$W&$Lg;uH8Jlp}ZX=N)dZ=_x`^Mf~STLGbT)O=k> z_?1@iSGHJ;lWl=Rs=Yz{)iy6lhMA=sf*=MJ*xJ0apt1Ab+gnh9B{rbrvEgp9c`bUd zG(H0eZ;koA5rkb=F$H0t^gaPsce<*^UJuq5YXEjSycVe&@ z;%^L@2e4Ehs;p4(xl= z;$Hy+{*NOs6_wm}mM>~0u9h!qC9ak)XeDkfU(iYnmhaX|3 zqP>fXiXRCo)|Y1;K=aRE1r;seYKQBCyS=k5949;oD%!0=%JR@0?7)hH=HTX9Z{!p~ zd?HJjDk>Ocx1b{$@98OiB&Yb!gSng{KATyeS98~>gGV*RAN*VoEkyu^Htf~dfxAwy z2L4!bFYir>|IgSu6-Ra=W9s5>G$>y-ElS&njJ5Qs4qEsc8Q`)J8MEC_q;3}OAL4l0 zs*E)YKq=PSJ$v8dquFe>^9OOU&}sKpT+B8eF+7^1Y&-0+5f|f0$lcnL45=#yaWP>} z)huqi*Kv!4uj*nnUWGgJg|F>YKv{R!^AjkXwt4?W0P@a^8w7SonoxZmteRzX{= zO+k}f;GnxU+F}izO1dd6s#1s9R$Hv)c_p%a_1I{Op^c|PTD3*j^A2#dP$+D*#Y8cn zu1s68y+&!VI>CKl!qPn(o;1ejpmISIptGayHO9O}sC$bLkno@}M*EISn_NWQHEN7) z7s(fyo}DDqSC91f8$Ws@;QJ-QVqe|LSoccUOJfZZc&EC*&v(Sdo-apcij&j1m}#Lh z@6!C$A9Oj8Q)bv3E54H@b))ko5@SE`@pdv}f4JxS189wXG)4Xun4cdkHb!-%YmlVI z5cRR}pOUE4f^9HJ6!8RgP8M&7GK*$x+9YhBvHcEY&M6^6KnP=8>L)FZJ*rayvfH+r z963m)5x^c{!=v8ny-ZE*Dh_54gm2)A9$wg`!fGxD>1TPtBnfHKsbb%uW_u0J0Z`S^ z|1?X&i)F#ijoN8mjiqx5qaj4ZWOv- z73BOp|7?;-FyjI(>^HQKJizb`TiabSf2qJz!)`!_WW~e3-qgSc1TTDwa!^d3=JizO z8XLC#?U)~8y)2x*xb2u+stEhr*Cv1Y<~#)CT;2lgX)?qIdHMSK{s8oK#}HK7Q}?f* zC`r}|Jag)s&q&;YsUXjm1z29Eg5AQQ$wFLry)&cCD@$Gxp^@p_%uKuxO6KN;0G-}U zYJQ?tWb#GtKuI^JfprWw+Z>HcPqf|&ZW%Ax_OL2mmheR0qJ6tl&+mi!I10ej$Gg0- z^_?Q}TLr^!<-kV>--dpr&OOqoaH9sor=`YTb)RO|<+H4_qrp0&^Vo(@+h>?%P5SY) zx@KBXUT0Zy&;iZJfDXiBuuN{MZ%$)ga(@eOQ=y8MiA?v$y|UMJ6@DJpX1IP>J?_#d|Vqw zDVVNce|Y0wZ))5tb>=VNGN&KMJcxsK0j0;px?L@RDbQ!z&L9?tQOsTTTM=t<1@9&$ zhs()kMkyo;9S^8eGdEOK27{D@1V3-8_@)jhZ&QS9jyA>7oa5brBg(WMN^@F3 zgilRMZ$8u69i-4_?$*`SZ#$CCQ|H0}bRu<_kCx644v$_FFWknR zcxfQY$Yj4Ky6mA5P&j~>*OUxSQ!==`gm?p2-X;euY-aioweTDrc^ zc5~?8bo8`WHU%W5v`klP{UtA-9#7ITdmIYegY5)xP{ED}y$-%KL%}i$&PdUAUS8Ehd zZgCrzlWEM9!%i+}adCQ8x@l**C;XPU8yf_u=!E7|WImLxxRuKZ2b0LMFTpZE$(q@O z6U8huI5*m*^WoY3n2s$rD#Wt(&^dsmvU^Xu{2@6q1Gr?BTtEz`hAa%&VqvG7C}Vq8oNrmkzQi zlf_R5g6(t?r2@JO6%OZLAMAx$DQ9_F=jC6%7m&QO)1r~M6pCwFK3qB7C~ebSiDo9Q zO)}IwFl%*E%|lKs5;}9WYa$tHt^uwR49V4%BoZD{mYUJj&bId|D3$aklFx2wE7|IS z$yS0e^j34<9UNaKp;@M$0VG7|8B`BsC^QntKW0GCp12jZKTr#im_q9WajRYy-NcU;%e6ZZh1pt-j6$n<~r&9zwgJ z*j?-$94_5D3B}Cr5!#a_IHmh`A7I-7u)t1e)823YKA4H7Ix7@#)FHU7)Db8_ID3|D zdc!+0xlf@0)8TYgMRZkAfE|e2w_*FMt4@JiW70Va+1+^Y09K;i>O@C|#4_1r>GG(MTmpAS8)7(DpMuGi6H> z(`b*G+w`|{zNpsJ0eg$x<#Jh)0${Sb9*^zRS@k{Q69o~ePzS0In~+PZ7$6Pk)1cCV^R?_xXe}LNs@EC zaa#iqk$Fb|054b)od(NGxE%_-hx0p$t)>jYzo*yK#Gd26k0jJCGm+u}E2xplW~z8< zCP~ySx=})8rbCrg>fW);Gl~iq0xrn}u)ODjBR{Bph9WC+B!H1Vz zlNwU#zgFAC7~YpePJo?Nq7!-`ktsL|T^pxDeldYIeS4o==B8Z|u&r|h>{Mdn$Dt4) z9Dlk?{V$2UFcGhH=SDkVz)7}-Yk5sECH2EoJXIcwIS}(k7k^;(HIh$Fue9Z&oY8g6 zYU$ih5ipsKRXbdf5(vgIx!|&%#5M~1#C9I=^!CXK5QmfjoAcw?j@I?8VZ_fHj{2Zg9WgydGRz8}s^pTN3!jyu$`1<9pv}ZdQZdREbKg-y3eU1iqM@ z1UvOTRj9{{s@c{$+sJmFAsnmo)7+cJX1-cTyR|>;ub^6Mo!>_ z{B!lHMWNX?4!NG|*gEsjdQvk*N2ZW$3L$|ldt1f{jiDz`y;siNbC&YK;1bDoOK zHaow_ZFUI**g+?;NX$2s_6a7u3n>7|cw^oIVoXi)guvCY$$il3tw~o&q+ajjP}pL# zwpcR|=Oz_XyqStmU!53Y$>zYqER)MtG->%`QYeGV#FZiCH*(A^5z(V78M0BfXBB74 z-XycmsY0-3z|1$P(imBjXE{yjJHR3s%WV$Mquut%++9pJa&c>}kLHQ1lzpo3sPiV) zOk*d-N;zoeCaIqz<+#mgqQ&ms`H~Mcr@F(Q7&56MbSdL0|7z3h6#R2Z+9I5H;eo8_ zUfK9#pCg73)TIZO6f?XL2-AB?nl{7Euw5Se5UKKbWWNu!lb#Z zZ`-v5;ZJ1=5|qux$}Y5WMn@9lip{g`R4=DJOfUUwI;65M5Q5u(*p!!#WICOC^!oPSlw| zUr;l>l4y=@ftfk}oJxAO)tL)aUmjOL){u67fFyua9`|KihJ_=(DDH=_^{oIPWw#WM zM?Yr^5$qDSTayaAU)M!49|{kQ5bX%}`xWmoY8Dwc>wNtx+!8)fqS1Kvs?INYRkvqg zy$fuT^V8D!DRAL>KA4EdoS+9&_<*&a-E>-Xi|K znaXE(ctIl6o6hTFC7ftxItu}}ft;{6kZ*TWs}x#q z0AwIaVCR2q0Gtj2?78mYea9l%)6kVVo>o4B6|@(g%IlbKbnhyn6)3`%CouVsy+}eF zC>NR6a`ov^&ut4babmZ7dqaXiOwJJK61vPivC*D6?Xx=M=7P{hDy9VzC>iOPxv;ganR z{uob4!^las!PeBnPkQp}n&Kv=zwI$l^#|V&mWD26YQzY5q}#Q;va_os+fi?SZ*@%d zo-;iDePH=tf^!SuMW&S=dwLutI>}+g=jmBKIoq4_EDk%L76PVIQY7izjUXd0s!$8m z$2*R{OP`3U4iW~>=ZKvp9$3Pbh8EMH1;uW=OzM&l{;=&3yPj(COC<(cO(7rtlxr4l zMv$b*Vr60OU;9u@d@_*Xle&iy{|*A-Ye6Cc;%OYw5J9r5&H{#z@@k&VN?^^NsB@KI2#5KJQ#LeG*i|f9p2@^ zEJS6v=JNQiPoX|!ILNL*h69yZOc_Xo00k5H1(9h`?P^UuKnFrzQdBqQExgR6MWe^@ zlKKxSQ1wo9Adx^-11({@Tfe(IJHkO31J)~+`Oc?^_@6Q^#Q%_v-B}nlw!9icGB%Kl z96E`;BpBH49FOyobP;%s#m<<PMCh|@84sH@aoiKfOLi>;e)VsPKsh6kBVdE88#B=8w`O+@PVZWl~^3G6l z8H<6jEFIc#cLO}F$&U&Ij5a+g80k9?sw;yM=5y8*aYcm3F09OE`Z`N(FMQgiDEOEYYe{iGrPk79oSAERGtTa{OuAnsYq>QpwzXgz7wlU2 z-LLnhejK-rwFDh^oTZYz-M7|y_`3Url6u%TLur1y>xL44*t7#W;NIqMOu<`|umTaH zL75naV2T0+Vr$6)lp+|y0LKV6G@v@R7Wf8*1iKyZB&)>_u#~O45a2Dr*$5Ds;HLzr z%^T+?0Cj>#6u>^gnOZSJTURVVkAfo?;7h>;3^1tRN`AeAQ$?3_g=GcT_IDFU^91jT z4)O{SyItylFWWle0jCz+_<(hbp}-26+lLe@;x0rc6RQ_Pnft&t61oqDKx%~)#$aqk zChmi}72p_y!xba>1tYfcX&kl2xeMwuhRQ2=bRTlBAk=uC704RH{{V>HWf4|fZOlQe z*xZ=MSTVl)e8`Fwj(L>;Hx8+p6}7z0@J!6}njZ>i>6l`gVC!vSDqyukUTcMWhlE%} zjqmbl0dyYnasjx$X7mF7eVY#qIQo!I3=sS~0l5PBcS*~&3t;9sL;Jv*7Y)q^+pKEn zVF*dtyZs@gaYOgSHVYiOFhUMF^l#i*=g|JKO;v~Pk&x*Q9Vj759{O0eIrey-nV7%( zW)nd5^4Gx?YG&mZRSG!t~=8uG=wG^=mDQL2rMFzTazTxH3GOH$|&k_==UcqQut~ zYjvQW#54%ew`yv;Qs3j}YiTfEd95lw?su(4et!FuYXvjDqqP9~w?CPn>!)|U)=%U2 zS}U*dv+oMGVfL(L-!K6~!8pw0w`Jya6|AM{xQ^DE_Q#EVJjbeF$X0fUq`iZgIOFG6 zJ77sO0H#S4$4k5}#Pd}W;bxU<-`w^( z!l&>0)Zskw&0TYLK?6w#-G^V`^}DHolZmDTnN8CJ%S27grqM5-4w^Ah)^1H@^-BxH z;(}9BqMH#zKb1541wX$;B1i5hQ}chlayYQu@8_9%dL9L1MT2-7wR5qu17g&O0~KJA zH79~7b8^i@w9OOPD~3uFJp*3zBLyku60kCq**E8z?=IfLJ8%5+(}zLt1k~9gXB~69qU1e=0sGtPI>67YJO*Sz5dEq+2izcIX>>?&Dwq3 z`25@J6E0iByYFS4*UMA@^h<*2m)nCEx3u~A=oDgYQMNxvJ$555{d7CQH$V4}|9Zjs z?|FBh->f=_elzuvVzE@ABWIm-qVW|j+WeBu;uCHe2Eb~@YD^ZFumr+c!kY{C@y@%w zXE3RXQ8vc^mowqwIS8rHZ{`ruzL@-wlK|5!)gH)BQSAXynMI38#F%icr*xg{7rZm) z9n&UhGF*s`sQZ^EQJ4kAKyz&cMaG1Ehyk4UKT} z*5DNLQ;@bb{|ai)IirUc90788ShJiZdroqg%O_|}ypJr+cN>xM2j6)fK-5I_h!+V3 z<1Yj8deTl7y%nK1!)sLF%8P&;iM@aO70AVbH?a?={?PC->u_x<3y53+bDa-}Sy6K# z$h!8Es?aF611t0Zybh#RQz!FkqOfrJT} zlq9D8D?Sh&g6?(FwEO#7UyIxYu*-JxPT4QKQHpxU?tAPajrd)~P}H4j>~gJ}!)Znn zsoEsX?H=ZmifV*_r-EtnLxyLg^;%M7;?cHZq1g?;;eA@@!n-r#B|_Fp0xob`hSn-!)+Fn!yZ z6-)qkK#0F+n4iH6uM_*W;oD^qEdRZ#g3k~ZWO&5w!VW%6*dIRjgWhwa*rEPk((Jo- znthjems~Q4hon>NpsYi>LNCtj#lLIz{a%`VB%`ZKFP!T1sua;_5Golow`Wc$mEc$) zI;q|dQrK!)v{#;nGs$=^`s@^N)wtn3HYbF0E=8dv8FcuRj1XTO2}C8%TEAy3LZJYO}5b0eW@ziYn4UTTjzfc;H^4s)n)uZuzyRz~~ef1~q^ne|6@iw@zL(@z~TU9lw6l9boUJ7pt5&?$6JK|L7R<;y*#!EiL<$5j>pxx2>adrPvh^UD!vU`iOy_vIX z@+3jcdG%ck5b4t?2tcNYIV2#XPUQ+S?F2-j2cl|RL*heiz$a02p>2vbR#h9TR~z}# z^g<){N-s3oJoq-9{}3++KgZUX+BQ{Xoy7|=HmXIe&t_6O&Vbj0);C8UOMq2WTXTib zAK(4bb6gHZZmyn~)Vl09Ac zlaCg`*Kd}KaDSj&Q+CY)l`Jg8Pyt2D#Wt72=KMv<8r5$IFPG{=&sRO=litSEM& zbAzl6?x60%98XwuX`)vmKbuVl@91e>cmaXhBtR@#J^0?Yoh0s1zzz;k3TN7=waKW; zUQi)8ylgp?i9PZmk%^qT=o_Q6{k}H%7;0<6w^rz~* zX0YC3PEKB0W&f?sIhULkz$V+H0w};8Q|T}#l3oPlQs5P}RW&8S<;vu{F*>{3gpS4e&rHXtt zLdsy_q`9Gr%Arl9ozndRfKk=HiV(Ha*Q%fMl(|81`=Qh{;{zD%)%?|ln}dv_UtLMb z))c&_8FGGgA&U*eCKmVw)r^GAs??Hk<<=4Y05)w^P_Y>!5CoVz97=lU*&%OkSG!PA zn<*YBI4Uz1B|E9CzaF%Zcz=?8yx)%*RiZR=w6EbjYfzL*l0+M@&2@xwFFGKHq%*wW z_VIf9`X~Pa;(A^AZrcc^xZXt(6_n3I6^^ofv;8bjRk#bQhx|9JoEE8l0v2gOf$jA; zhvFEw#ZIeRy|wk)B`#$cYpbM8PpZCvKu*Zrzvx;Pu+SzC`|ikcd)u{JNI*aJ||$R{M~Hvm{LFT~mS)XIJ-%FSv3F$3d#>zhRa+du_;d zXzUIj)E(5rsUzF!J8mErB_E0mHi{B~D;=FwNp^V>bB0cVZ}~qrCKE1q93oo)VC3HX zyw3McwuIo|aXrwN0Q9Ph3BPl@btaN3+TrKssloav>6M4tufzL7__=zfsR7j8zjtcJ zs<8lv>6_2j(rn3V%3r*G@K&5meoR2(e!kShAyxL%B-NrY@7OvnAr^ZQk|NAqTs~q+ zLBjJU5qc*mm-8I%vfCyL-0daa%h8`*X6x`fVnw@=MVcz zc!yzs+J1|Bdkdd3Y=~ht#Eo?`slxso=4RaE*ZKXSt++#@cM@Fr>Jm%sV;ci(;ckKg>&_y73! zUpVyZ@Bho+e{=_b@$LWl+vOG}?cOnI_wu_h=&!%C$Ni7re)o&NyMK8dP)z{r8?pMF~=5IbaKZE~= z-+cd%zxSE{<91g5*-!qPKmGIi&7b}1=kurj@1K7Ai|>9l|MK6y{Lx>2|I1(h?C0PA z!*BlhCvX0@z?p1Tq_%)kzGRAkL(0F0(ma$bHG*t~Lf)QyL?(6|O}^Zp>c9WjfBeal ztjg^J+L3?KKmSMjpZ}Nr@gHAudihT;|J(nbUViSs{Hr_9_V#AhOHB!|4{qq?FLtBC zgZ;(sieENrd&OUU@m@1O?N5L9`Typ#YGzst@Y}^7fAbn&ss7W;|EyfK*=*Zw?LVjc z(fH7`VBm7lf}xD-HJtz9m-8Uzghh%ltHYtZky9*mIKKT-`K8`%R`(K@X<21`h8&=KU;DX^KmqIB+By|H#i#9Jn z_m;Ao(B+gvdTT}E|vG5}mVvX$@*F~aQ|oqhBQ=oAt)I)%$+#01#uCc6JgXn8Z* z?GP6L>Go4fNJ)%&jF8r*6|agYnAx~#ZCfs)3N zjtK>!p9|;&%mwmw-NhK6F2Q)|K5gzr0p>98XOJ0i8Xz2Za=OT(kBH^dxok{XkCD7rzvKmzAo3TMW2p-G) z`#DrAwD@xh;yG3}gRas5CDOWN){e*NI}DV#3Up92O0rxwag}A)h~CxQVgK9jQUq#D z?R>jTWRP*0F27J4^Yq=B@B|009*WZTLl(*tTt{=opwt}%jJGK5SDPA%%hJ>-T3Y>G zXYI|I6+vZJLB0GpB&$x5)!DQat|+;3I{C@mwVpRg+oSoH zc0>1^VN_O9FwxtoQ5kusL1!_c0wXGKm-)u*7u1{mCNmkm6fY(AM9h17&0m4xcr>*z zh8{%59CV}#7f>Xj#TM9g7MoT)5M>}AVD>}qBgYT{RWb$7(Jq z$`(5yl`ZyJ8{xfAbTX7Y-{s+`1sg>;-!Zn;ULTtm1>n;jH$lYC(G)*uEZS&@!pfJ)h8~!Vv zOs%TW%is!)%MWZFs0WNzofVB76##O2cpXE0dO%8lvaM%#xX#lX?+)+X7EDE2YPcG7 zEH!AqXj3GoF)m*`;zSd50G;Y)JM%RaHzr>p$Lzm2HoO*kd1r~)W^}jx*UnQx3ZrU- zG{J2|Qr{i@`*btgh?rJt-KowrpDPDvWc~WuZ`|Ph3l{f$OAg`alL8UkMPC z6c^7{P~&T~GE=oV?98;-Vk}(YkVaQe(!ukAke;1}RXODpp^=6#rHTpUlyIYsaDjQu zZO;f~7rKo2X!g2`NV;JGZ%ssn{upcrmtEETp%X8NVkE6d5KzEcs1Y+a>Nbk4z?@_b zv@W6d)4XZ_sfuIvNi&?(_XX5#dB9cDl@$ZH#vo}_GoZAt0jGMN8KEhM%Gl|5pk6hk z=Nm?w3(+seg&@rE3(haxj5fwu4Yb)K9eOHT_CU$_^9>*I=nAkzU>=Tm_Ff=UEQvR5JMNR9XSbWgm>^ z5A{eEyQX>d$AqPerUd;}NG7VNf$p2?T8f~gGYCnzh7~s38a0cM>BzLz1m_5o+#5a* z_=v!a0BwNHGtEd1Vsu2-Df(UWqbFohENGurDGoqSbXh(2?0Y=B5@7+8RB% z2k-`+mhj;PQTwdGlH4c=nrRM;4aO4Im+&6SUbw+;`Fx3k8LK9eYsiuB%KYQ7{!4Ts!clk zF5k9k5JZ)MW?EAG(Adi31|RkXaXBADmWxK-@C+i^P6F$|29a$Gi+u(oiWi z;+76(n;DN^t71eEu^XGxQJNc+tuebj$PRi&jq&zUJKH(Xv+k;#oooU|K-?zMH6l#1 z$3UAN_#5DB3ETpR={WkVY1{d`WS^L2bAfO$69Hh>CAfKl@;>z|rN!8{n$&f`ML;S& z+$$Wsii%Jc8HID6ZcK{^F_53c%spdi2;l&3oeqfe$1ZzZGRssc=Q-?a(6Va_a1=>~ zeUeCd2VlcovJXwJf#>cNWjng}1N}YnUU{}gim?e!3Nv6sCNyK|yAUoqa0v=|8wB%# zeU}^XPtfOS)@J&i90TvqF__0Vc(N#T06s7BS|W)R-m3cnVh)w~*yYr*++I(Yw7E$* zUjOEeF_^x&TzQG}`hyumkZt=@hE~ z4GOI!Cu*f3pTzJWl(fy#p2PyBgT|=dUastu@@o~$PI1kPQl&eSDBwifNT<7tzCB?@w+ z?b+7P%>l&5bGc8q9z;;WcCb#;`9d=%v1@e63Jj2E-0t}}aDRF71$=n|$2Y$+^J%65 zR-}U`p{*Rcs}9Ht8w8yTZaPea{hp~pV6jfg8QU}=V5YvS16nM^R_4uix@a{6F0WnK zfzep%gDUv>IGrjUWbd;DwH!=6>sR9YvO5I5paV7eb>t!p66t<-vXFi{dqah}1QzVP zwD`axbf{lMEpb)C=@&`avlnV%jJYr{Rr7$Kt_TqQIv8V1)PMun+t|CXTEQ0 zJ%}H}vu5@eLfu43(DD!Gl~a#uqj*!rdW_kMrf2@ff<~VyQW77E~vL*MWP=di3s8O7}6+C}L1L zY%T(>w#)SRSG>nEb*I8Xwn9xPyM{jUQ7Og1_G0cp=Ocrb7IH!DVUGj;px}2ocob1B zKYm4*j=3AQbc&01VQXz4>VS3ep)E8n-ghP>x^G7~ZMN9#7|m@uM(n@=0Kcd5JmlZz z>#UbM{YWoKj>&#N7<-?Mo(B$QK~mbi3tdt);Xoo}sm()c|HY{TF5zcZpx^h#tYob> zRw9f|^TpLet+6K*gM_KhU~Fe=?Gwuh?!E-Q=6%4o$Wtj#ES*isx4*jPM9ic=>I=oP zlX)EukQc7);b1%;B+)m$5{scABPHoiYm7UQ+?^P!hx#I53i~vOO8j)~w$W&B`9bps z^(%494KxC=XCy+a{%#>Z|GXaVivA`CgANkt}rcIO1OL9Ti!Sk5zCDbJn>#fc3d;)OxT9iRB)J*e<~bTafXRNVQ+R z%1tDhZ?~7k4fr_nm>_0pnIHuK-|O!x@`0j@OiF#^>>j@D04l1Tki!>|YbpQWwx2Ga zQ&~w;&*YIj%wn&`>a$l5QkLgTL3HZb+yNLe;r`k=AS7AR64CM(jT)F*4WXS$^vBO;CNo^w=q{r`jtE#A+yqt;wQe9j; zggON}d`aWRW0(&Pts4yGz!b5* zxE$i}8%{N6-*89>hgK@_(p4qy!28#n+C@(H&PP7pgQ=LwY-J7Vf`HNrZQvLlz zSmbdf@mh-B!dc6h=k(KyzW#t3=rLQnrcJ} zd=BUTZu+L|P3IiJmd+{a$dBu|F}I)5yQ#a8>f`Ueq8I<< z@gp@YAenR)X}wYHL|z~q@Xy-jXT8^17)5k&@ho(NOR^?Aubl4cl*F_8$KiQtSjaIs zuI)#*4{t_%o`rvsg+aTy7F)r>ee(;$kx7hv&z7j|VV}l+iD`L+$`Ejw)=t=5(H(H$2d*o;up9LeL4o zB)5y44&cN?f-=WE=0`;ZDM?M@J~F&18bt@K_{k5KctFgA5luXNBEA9dV&~!YYD6Z@ zPU8jf9F4}XEm}x&-~--QavRp!{CyOXwv1|$iC_d9;7L2Upe$oT^X%r@@DYl+xR*4! zg1j9A%Iz^7jedFQ?If>qIh&0%xb38%?p*Ic)cJ*CZByC%grI=%w-R855r};ZZ==iU zex@+WVguGFNG-w|*AWms`c$=M>&*DCgtUjmyN9Id!idk)m7$4A9(^V=8qOmIVk|iG z1rrWGIsoX73)0JOM7nW{@z`jN9_k|LuENoZ9pM53HXJ`VL297rDzpXyG9Yy zF-WZKY<5}73f*BG3=nwum|8Tf6wP@qB%i~!Cvm(S?ou*>wBqAJzU|(5_d^*S+CYV; z(0SNg7y(Cz5HoN?iN^x7ORqN@h;S;GV`|bdvQAMZ#)~)O7aEdLhgU&lBRoW_u7Ie1 zpVPuiQlvnXq_J$VMUsR=95K1F zeO1mbX%L-ZfKR)oO$HiIe>mxk2W;S%kKdT|Y;;J;Wo*JdO zVSDOB;z1DmBs{3jGo*fjG}4q#aw*G{hh%y9JRr;H*OR0HBg`{n#6hw12K1Pl0NMw! z=P@Oi^wOv+JM_k%`T~uAddKt7z0ljg9f2rmePDf-gXET~Sj<)LyrgYd^-Rw_~LKQ)%$>$6{Qh-Uc z>w!dg5Rj_Sry_@#?V!LmRbjUM4=}EFU1mCcytD(oQ1`Sgf%>n-c{EMbF%g;FL1~n_ zq#y<1Lzdq@CCE{4oxCR0kmX7qkd`ReN3a=CG;sIi7vlBU2o!?}Lz|P<9OjmhAp6@{ zwxcihL~a%Pm#Kizyg`i%Hm|PqHTC8b-kMA9zoVI{qml@xqbU5@)A7xd4FleCmd5&9 zGQL0qI@kc!+D~T}KT3B{%{qc;q|X>A9KNt0}6svyCp%8P9RQi+#DlF(RaU?sfxj?UZb98VQv|myQz+ z7VI%9JEFt{9nOuvp0)o2J^SX3#V(fvI}q_6rvX)Ss5LEXguvP;oDfMcLU2t=EfPZ; zw~`&SV-H;JF>B;_YkU-;{nXfD6NfZbbqsF6}2n`(%|9H;ZZnlrLa= z3dt_%JUnW~3D!&-B+55NX{#D|dOC?vY$UGF;m0LKOXPW|YEs0{#q>Roc^&bai1+n) zy$%$Yi~?HjqrB6B44mI?y9B+Ufa@_XykshCQ_v zEqxvby36U)T?Hg2^MK1H8IG52G_IP3*U_67nMIHaGCg~ff)E^72f-`~FwR4aao!SS z=ei?tYNXO5GuIux3@JP{5xZUheGV0hRjyI?0ctKWzzsUJ;;pb*mW0f8)a=YN+*6*c@WKsce!W%f_i;pSv#ox5pw<#ocu zNiSqJjf+NJ;s&!nl->2JLHg6kh^{YcUZP6qMN&taV+w3(XjF>{WfAHGok(~>D;b5Q zW7Hr$$0##{aPV0~rc(4daYq@S#s`o(V?eK217Ll_o=SE_7aGXFQFOR#(^T_?r6b!) zR0m}XZ8z8Xb!u~Ubxda-AME|0rSx|Uz*NN&?OwC8QU1|uZuI@tbj7XA61|wFi_5P3<jh+f21-CNOp{eq{~|u=+`a-A0leeHBeVL#Xp@L0CD@vuH=6FI zy1QmS;l1@Ze~MO_u8u_{0M1Mx0f3Ny;Hwse(cz14Et+1^x7b(tTaxq^QYh%L-Y)ud zAd!?FO-jv`37-8Y%`k9t7Jnv+oDU4E-ib+uE9SMYU zD?Pw>F-*Idm?M?8K1`i%U_?oUaS$$X*D1I13U&DFyd3QaC!PH5L9z#g?VS@y=Ad>) z+P2)ONikL60kMAxsd)KywhS_7`QS`hr73*Vl2@EbXWzx*HI z=dVS${)dnM^uPbXv|TY?Sph-) zfDr_>=O1^lFhMuqs)npT+dIem#=oye*4Mi%_Me}4_P~H z0y7QJ<83HTbtM=87~$v@0G|AWN|(L9HHvb?d!qa)fEns&B=s>)$kd!ot~p zPmjW?WoHw86A6XJ=hQfI5bB5=-7o+dnH!6v4*0TyTluODygj1OEMV?6W3;nM-0arNuy?m zd+Vt{c%B?kd=zg?mQ#8@nrBfj^~K~#N`+bSN+lM71A47_D(;{{us9BzQhHR@XEx5z zBHF{1G-2JK#|5^F9?H+H9`zWXoICbA z_G;UJyXpByF1)>*QSNTUXehca)X$riAxz%-fOk`__BbC79$s`OSfXBuD~Mox5GSld zQ9$Ih;WgnduH$OAxW{2uoO0{YskJg$F(MRlp|2Va;_)W@D!^>&86#Qg6{Z1r70ZX;3p0q=OphaV z78CVDJWAhQ^Vr;QjLB~CO zrir~_TYc&2xi@{}`qqd$pY@*V7?sjXiQ6+fr~>zRg$U%Cy*J4d-q%*ezN1aQdaMf# zt`Glq_nwyxm*mG+iSSlory2^`Yh9PUE=q)OV3|bMi~~`B?@o0RzD}z{^kBfSJ3Sok{B;?YLJg2+81RQ5NRQZtY2aZG#^ z1(w7Gr2>Rf-rSOZD!H$5dpuQ4oDV}DZ00M@9-maJQB=!o6rXt+DHSPPK%4{0aay<{ z;y)5Wc9JT}K0T;UAaKOSwX2TF581K@i%k8?q*l|W$ zuqeo~V&3Z$UwQ#B0Bha1tWhC@vR?Z`@2YxkSY|z4Re&}`v}Lm0sM_|*FUJ9Z5O2*A zMnGL9}m7iP<}WKHa+_7LW1 zR9fQ(jwR0-<)iAiRt5EtSg+{%4pirbph3CHbWZ63`Y>YzsCkqEmWr*2ziqj*Pg`yb%rAzv8y9@!ab-p&V zjW8HTrt)C0*R*aj4xE92`lhD=$RbnBV*}z@&d@lxvMdW3eOufD$IFj!NnGx{!4ey{6Sn?9tO9IA*&% zYKRfPZ0HG^el&tqdo}WdZi8psO3oS7q6pk>-K6uVjFS zGf%NrwdqA=c(ZBl9=^ZO(Ivj#WUb527u+tr1^nCv{0`Jrhsq%U3TF@%C)7kE41c{E z<&ir;l$hIfhp);>tXi_&Xe*jTrf=*^CEa~#a0SPqQto>0*GAFa479bvXdC9p*nCY? zo)8HjPbNhY3nNJh0BzSYi6%k$0oLF(OmnbSNC|IH!fZs749$K?CJpd<{ZeVs>zATU zhy=*}Oq@g}ZYHptK@9_hqCAJT#hbv1tfhy{j1mAa6!6PKMB$;NO4s3M4-#ylAA|{m z^vW8A7gQ*@D@Q=!w?FI#(cNDF^kXEiObr=CKU>AC#7>em8aqkpMYnm)NYh8^5HUlX z#-zVxMTp7<%ZMuhMnmJ?POq)0+b*&Nw8o|JWque^6^K9OD3^RA%oGKC3os&uaXs$y zY?lSUh6zkV%q!#lLW4lF=Wr4sL}96l;d{srB(~>s0zeyDjcERbQP%q9#6%wL$*Q56 z3H8V+LoLW!OQc$+ zUUX#t5hz)&cGf_a+UI_a6mW3V8&e<&W_?vFzjb;PE%k%qTTFeimsb;%>Q}XR-yhhA zjXL8|tL#eAoId!~Pzll3oQ?<;4UrX^b*B-qBvFJcE7Ef~3RyI+n6lbf)TOUh;BLuV zw+$*bq@i{jc$gP9?DL1CF~V>PJi>R6qhf?DsC1JAm%=x;GNVB<4;oa>TBM{_d$@C> z+el&$L{V=c5t_aMk0!+p7jp{H#(kuk3 zwKQz--#QoiBJ2;lN)1#hFn?RI^wBmpYlEuRtq%DKN31F* z_>vyLEsNM%n5Gtv2Rv#*u_i(JBUyk`oQwd2Y0Y)->)0PDaKCQBDW_yBMowTBB?%Ha z6hydY2ytJ~^~+uP2yGIiz%;C#7tCzQQuC0nb^Ee{5|Hpfc)68!qyD%yRb&Et-%;c` z0A}w3sREZob!yvKeBTR`Bzv`en2`?scsgheKE@DeyZO8H2tbb>Y>HLj=$|KIou>2; zjEi2-Pw6FurS}bwpb*Q@vxoK-DDp9<(Id!gCZ}{5xz;|PD6|s^Hjy5%2LyV!PaXts zK+F(RX~C3$jU3wb^Y#G8t5#FRIJKj zW5Z2h%v_vF#FAjabuVQQ-O(utN?IiOdNr$RV)y+J5^mo?Izj!jS0K?KfLOK8%}gcj zGIb+5LbSD{arxDN_oe>M*Kv&~R%1!N?AX`~$kDkvk(HM;k}Kb4r$GnY_PW&$GE7!q z_wh|W4j3){wZAZ?xmamulwp+|ThRuL5LsQ8@YW8bng9L@zGT~Sxt{^e*S=A^^_j4$ z0Xe$}P#@`Ci4+fQjYZ{-?~ai;wixJpan&5AD8~LEv7#A_t^?y%I3esu02(L^$cYK7 zg}m~N%q#dpWL`<-ew;^%lEKs+3lLS?&L2pIY#{ilEuzJ%L+MV_UUO(Z%}(?k(1jEP zloQpYPStB}_7ek&^NhOpTBKmua-TmXRurThJ07H0WQFhhlK4-LF4PI4M72jIWXcZ8 zZS)kD%e6JysWU!k0INhjjpekFOqSnrld?_(Ge152SeEc$$F&nWFj*&S?^9Jjpb(+P zS*q!!wG%=y6p-Y3m0g8X@DjCE?qAvXtVt{;ysJh!gM~;I9JQEks*#Cc<1>()n`$kx zw#06H#eJ+-HyCe^BerGD%YaPmk!lqdr<50uc&X3>{q6K%S45A8kwdQ|x}PQD-J&84 z4{v;cGH@(Uu7N|CH6{ERBAg`U`12z1daaNKu<|6WNNLJ^&3gZ> zFFibBwgL_}w5}X(N8ZNCZ)WDW&W#|R$o!;cX{Bi$7svV416TlKaKu!VqVVx--`S)A zeKmMF^CDwH%ZRT;ggh&gvg=tL%8gjvsFC3JxHi?j3~!FNxd!^p(gojZ<=g=ne=_*3=7n#?&oy?bc41htH7 zVwZmOPdOu1TqZ<_HMZQ4%ir!Pa6>^=-@L?m#!1690mr?^ameQK_-C^Ib}b3K#pYC(sr-zc=7C~UoAd#uD_;mE;(zVLGd5@*CAIlFS`f`Z zpIXjoWKDp>{!$77+GS7(igI%#PGnuP0H5ixugG9vHJG2xLR;2)DsuoG%kn`gJW4iG zD=_rKCgJRLZ{;r_%Hom`Szd$e+MB#jgb-6W>*IZ+prJ6ST9M44E4-_ys~|BWl8uA~ zI0|rBXb{S4N1c(S;n)e?lNhq9L5W#tR@901^%TQ^kk)AuU>#>ol>$xBU^rD-asdR- z>gI%GFM!px?BlMxFT6$wc5Gc5A|n%C5Z>8i0b4RP3GQy(IgYzJ%54Oxs0>M=TM@&0 zIZ{0`Bqxr)k4K9rp302z6)zWeheGbw&p9?^N(?nA^C52?_0Y2N5$(1WdwcZ^aiI}prq?6KnG8-4EM_yJd| zkf7c=)m~cHa(;XZ?7_lGj{{WR>VaVgJ=UY1Iny0%?ofT{0B*J-1{T8Wcwp6Ms*HHI zB9uZ21umh|V=CjVM+ZdjK9ko_yy;ZoY%NqI4`=XKfh0aCTrfL?(uX>0>OO7(zjI4$ z0Z?wqHwY;u=c0IL*^D@*MLDWpZE{{`^1|1dZaL|UGNYJt6!HE|iF-0bmp7481#oTT zRMjdR@IK>exaN4ELckJsypNfX$Ynz0YqKk*60WsDbfuXFj?>Y!iL%SUW^nZ|-6&On z=?4G;qId(+{P78!g|viJ9FE876@RG?giSF^7fg~x(!SW-$So&@%r8037Ey{U%K8(N znTEh7us?{`$R41mRRk;7tWwK4AslE%ejR3lDC?Yhrr~++FUm>=v+j(WrjjZ{X6we6+6&2D4>Qw?S&Uu) z#sB&*Lj9;Y0z%`aB@RXTw&wPT6i>dbe<_HizGa4`gb?}o`ot!+sdcy+ax=T>lRUb1 zyvSHFAbP_;>*Jg=i#!@q@LnAHamc)-(%Xl{780 zkV2jV7qQDoO{wXG4p%%s*ziFO!4+ZMT9XXf8xqy6kP7_OrJnA+H$7<9uci1${SYaY z#MP5+=n4dg62bS}vEl@}25gp!)l}gKFI(waYd69~1ET8D#1ir~)cYsYwJgHQ1@BXZ zc61F8tfe+hhrij|IXN;`ozNSsK@5WpCvQ%WpjsK=j7zoF8|WBi@#{%5kPA|UdcA$8 z0MTPB%z1@H4$c}*Yhn?Vaw$6DdIM(%>I>D#M|g&nM)d1?C&+i~Y8Umw)K?LB1?6_7 zPOaYkEiD{LO!chQY5sX5GR<&kqKuO0cJQ`RMJk6%mEYg*J}|(<_$P64h5%bnJk@}u=$2rKtm2$NVHBlsb{ulD@@}G zr$MTN)@(G^WO^`I;PqR>4nsUD|J2Fyjk&+k{Kl(_=ee|kUavHjLgnWPtN}>qN`szz zyHiXrIh+zYtHJap4fjrlB3Wx*YXsW&!OK}6dCpmhi`c=<(>c6u!XzvHIs$N@`Ur5h zk*x~>i>H~X)iUQQc0syTN$#zl57k#%IQ~3@o2*ahi+Z^8)Q8L9Nxev$3)TBvht%hNt0=fA| z)7pYH7YC44zm0uyGzfH&`hBSxtT?{{k+*-p5}=5nP!WPms_akC+YdL|sdOzP)?KL{ z9)exY%^K`1QSH^>;gCnXV73va+@D`EYeg7W5aqIdtuw&a&(UPqN{(-BWv?Y`iBLN5 ztg#Luz$MmGu@v-jjYRDJ++zV<25@jd76G0&AYU_Q1vQ0TSgq*okI%@>#k4Vr@qj~Ky{q*kVK@{+kQ(zZritVZ!&nkB(<4)qFK*@=T z7W8wtRPvFt9BO}`ML2eZZ=2FHmuX7>SA(l^YwF_xaXsZZT7}q!jj~+U_EW4g&vC8U#Phh8T!f)MDul_vu8cM9zK8PcPibt4VhMjD{1 z0_;o?(iIb!I-3tzfr+wymhy`paD-oDV-c<(7EbCP7W><|8=yq31~_>%1XY0yXWw6? zB+d!Ot8}uBuF>-(j(|f)2IR&hl<|21eX$yB?seyz4y{Eaa19!f6yNG3V*ag1!r40o zkP!yVdoRM&E~A#`JHt8(nU16&fi$RO39$`i%`K5l=V-jC+wp0cAZvB-nvuu8&p^Ib zFLC#m!8Ow(T=_Pel40~<+X?CaexeHECAT?nEOMAl{XUV%N#2!E zc{D}rav18c@EUcvk}P(c!B`I`C!#=4k9FcFgUGe@5kTez1O)5<7+&FxzGSnK(4=lBpNT!4KAgptl5vfXuJ3&`MY*V za)`e4_+{s2Ll8g9iJkd3$Jwx}K?GfLEyhSTQ0${&1l}Z=&v4q z%d_1=XECO{?2x-kwRd|nqb{V}(e>aZ$SGFF0i>w8d1hyNotpTlrcN$QJKKnhQP4Zv zgL&yd1?HxxQA8;qz@8Iad%(iF*f)d;h6;yz3hvyjS#%{Pz4a>4uKUxqZ+h#|;$Lf- z<9_nAUwc7bpzuHNA>X(DGyTPR(JxkS{f!w_ZB`$;g_*Cgr!!&W3hFP`(5As(Bqg(M>nQP?cgcm9NQC|KgVM+fLv47^Jkvxq zQ`{~7OzpcBb;P%xt-UzSHR+UH42M7=9g(`qda3XJDcxtzzatw;_PSI03r<1k#hN^@ zhD;A6U+H0-hdH#8uj|Rov5#MrHXW}rws?mtUx!@D$*;Kqb@3iPcJ^Mz<4l7h;A>Px z=lR&i=PO!z^!%l;UQyu^qgGilFRe*}NknFgwQ?M(#uJK@tN8N~(eJsx0GtmaY=w&$ z$XC@I1>?9BulpFv-x|dt(!r6CBplbn2k+1`KUQNNp2pg9v%Rf>4a8J#RZv44EJ7db zA>jW+^szj8n@8CCtJg821BQ^Noi0Tg4l{zpG9=&a0mzrbD zPeasWo2VDm-4hPpy6$qjxg*6~7_+F`<41p6^ zmE;8|$Q0?}mfSgobCZQ55=g#swUu;uT_~C zl-ST@EuuXP7LEi99kv(}k{$LDQgkB=LGYSIrOJJKTHGI2FBsnG<;(#L=nQ31kx6J~ z_qnTKYSI-{fp-sLCVI$;B@f5ukpchdrrcx-*{kpCVbY>ktIBLp7gf^X6gbVwMR>_l z4y4ncWL2*|W2uzMzZw*Dk#P^Nl&RWGV!zL()NG57| z#ouJQUkzU4S6`f`5y@nd+z^%L;2kc98HJnr8P22WG17w!Qyf4Z5sS$I$7wo$LU0j;I!nKst*HI0-eJetDkO2R%Avc(3^_moH z6sc7Q8rCrhnKJWEQqqU_JM^gxne(y=xF(Az;I^=*%ogw6D;H`(>t=__iUX!VOP}?= z9Wo}bxPn5ULV4OALh8xP)M%Gy_~Q2EPiZ^)L8|l7IuS4v(fq+o?Pi5^dfl>X3N;x( zl(8g;H}MuI?I3zV|qP z+HM_Muk16u^`aWg)5RWHB6y4J7LpXIR^u2fa1*DK84~vGRtll+(cgRIrlA|(6=Z4S z{`NRW3IJS)Px;Iw4zr-CMf9bEM0s5(z=rG*{k0P_Aj}XXWX=|!tC*{I^aGkw6CwDZ z2Xw@TEK)lKRj2XZ<9SID>EtDCh~2&)2j61pFcVZs76a8z0M6ELd$2tc3&BqQ;o0_$ z*0a}@Sw&qExCRX#GI+iy)gah-ss@T=2a*vNu>u1RRRXexV)Ea5% zk65P>ESkaU5xi1O`}jPrwl6nf=AGjh%X9WxbDZw=6RvLLRq}!>@>f}FZc_2R-33%` zp1OC{7gxE2??H*e2XRvEjOd0Y>*#Xn9dhs37!r%(^3~U1tyRb*?hbW#*cqO7#fT3w zWkdb^t(U}MW5$XYR8bRC@sPC}L>TzH9+8bl)9w}0zIQ@6P<;S$A?pIDWEWFaCVp1b z0cFkKfg&Rmki!KNi4CCyYY}KKd9Io%okqP?SRwFT#yl&E19?s;cZi5uk$*+uL3gco z+Tl?kg1b(S6}?`s<42+6;EfvgYNq!(`Y8KEiAv{i*xzPrm2ko?JP)Mift*`x$Semv z9xi`V+kibpg{(Mz@*R@T38zcC@0FMz8ArAI;c1&rUeh@f-Zwoip-StSQ3ZV%q+fd} zD0W6r?7D_AW|YxTLs!A|i#Yw7hXrtbD{U`_1`U8qit6q)b>iu%)z1iifk)05y|R^# zqDNEtVY;1}JDG|G+1!LP)H_*zt#&Iz*2J8%Y&PUV@#(sb=;^z2pa^=gz_M8jlN8aiIe_F|!2AEaOBB0spwzCc7W69N#;%Bx#KDLk1OjhU#?tyB zYDQ6Sr6x9jB}=OS5Pi!`()Mr=8y(jVwx^@U_LQ9@g7MTM85b31%vT1%TaSO;$swoh zuZ@t+NR&WjJ={aW@n%DV&<%VP1T3{$S#i-sdQ4W}T0><8mJLs|Q%k3LKAm{jbQ*IX z#ylAIROjw7)8oKU+#WMMz~!X}Asju%bEa*JJoOHlo1}P1Qk-JxwBz*i?W(vYx)F|o ze6RIQ_CM)?&es^9P~&3@LQnF-cOJ`K!e0=whal!!?f_h7I*mUw@Q6F)V%dLwf1x)sY&2q3rr#Rw4dR8amEeI%l8aktoE(E+I)#oO>VO zLu(sg&VxSUmoQPK7*t1l9Js+Lle*u`9gaf=Yf=cB_HeoY5_WwamwB0Qnd4h+iXaUt z(TA zNvLFQdI3`BR!yc?*N#=s3`S5?SCMI~c)kiR>+$NNSOV0)0*Ga+y~ktpAs}(i76bQI z_M#yUjfYIiD?Wv*;UbTr5hI!g zgoR$filKlj@s0j41BnJ(K9f^*&rOm8K2$tPbdu=En4kgpJ?4ooH5xq;%hd9(Dlr_7 zV%AR$=TXQL_M2VhwIj+EHWl>vol8_#^J%P<`}JkW5eEJOeXoP>ty zSsrx`o3Pozloqi)s=TtjI9Q2d!I-f%R`A5rlm3~gKzeqeg3sL&yS{G1dKHEaBbwJA zJt97n6yUALs8%921dV)NKn3m}baLLlB@?WpvL|O}de_y!hs31Zbiw&d+Gi>k6SunO z<0P|ZNc@=RglL(sP|wtuv*;YWnT=p&D%hv;-UH{4(w^M@m&WT3wv*(!CE80DB*ECy zm^9B>ciO)nk3r<9e1vcokNMe|=d`v21jQ!w_ur2>>+4xgA?S+6Px{fD-8wC{!^%8GQvp)H-+OrTUnls|hyjgwA}{L%YpXUfO8+|IDs?K0l5FPQ zP2U?%F_5F}uSOv^QUj8{`4C@ztE?gi<-%)1838cB67uhP)<-v_Zom$F)iWO1W?o55 z>+}Z*N3Tsm?){sz>en7OoA&vM?ZHvg#&#&)b?Og})J1e4L}&>xQehHGg0bRyAR?<< zC}kPQ+mL82_7|ouC~=DwSqAJiX?8X7aXNplp*~5M11r*M@N0umtLI=}+8_Za z;s`O~6v1;xRpoq^yt>(^r-`EH^URXT6dK?*6(%gj?_!6GU3U5W)|#P}_?Re6G^rh;yTAjH z6ZW1w8^aq_j_GH`cXjLF6F>T8WUG%fT&{HI)R3&QygVk-G#bgKLZPak@?K?=SWdJZ zJerDmT?Z48r{{Wcrq@)*V>Wq>eYDA&SD<*DaW?>O0W*~}D;P~x8oqu$^HXxDx&s;( z4+S!|ETq*WV)Zo+0e1gnD%f(l!uo52%%FN25}ie%ik$F;`OxcI<%)DM)kX=?BkKg@ z$RIC*1ve^_0uSMUvZRk?W0YaGYs@hr+9@^`ggm5pHkPDb_kzkam}t3e2&EVZk#cOs$GTe}1cPwTHE9Lp?mmMBwya z8~Uar<#1mcTNDPRq=jf#5j+vaecYSvsStLDmm!S4{s`x>Zo#F`D~%%AL5|!(s8uxr zBj)GlQ$A&POAaXi#c!aIsuS!Z2afC-e>7h9SeM z9KpoRVA)37zg;Q1_Xa$i9>0sX#w*IJ=L0BRuQ+1v+-GGMLG|ng(j4v8ki*ri?ONnS zaBqL>*wmAAt*ycpL0dGuuud6gFPfA zUiOe{5(O+@McuqLp!NAU>3wZ1P;EuboEx~!{5|llL-q|a%UuIc#6AdM`0)xGWrQ#- zto3Mp+|SJty0ldxl@<0(8rTt$Eedbor4;ES`$PK%qQm z6ki{0w7vx`@c)(q-ocFwDRjyepfT)Lrfw)Pb0b_@wRqRnDw5jHSJl=98p)Uyy~9+* zWG}MS?7gVY5R9_&V?`oOaLuSNdMPy(@bYA|6+tdr%jBlUzGhco6}Q|U$h0blt*zV` zuZ9P1kWwsaQ0t*RjmYVt)QgVB3NBxNtMNm<2|!1E6q%n!elH0W5PPVrqf8j7=F>ML zVmq47t=DDU8O4-w)MCzxEd>I0h#4}!utWcOV_4AN0K;-c`zlWFZL()lQupkKezw~> z03X^@HNigzfB8MZHj)W{lTi5YJlKKD;H_8zx)p$|Hps(?UR*{|MXXX z{Nw-g@t6Pkzx;21`=>ws{lEOvAO6e#`-lJKZ~x(+{_yi3|M~y>!yo_8Km1RB_YZ&i zpa12L|NWnH((k|VU;qA}|Nd|P^!I=FhjofyUEKY;fGfK?X1QAMlb^?2${mmbYjtN!{Y-qK*Dl$ z!e`8+qhk1wGuQUv!yf&%#|oXJx>i=OMUR7gS&l)y#5oTR+ z#i?9X;CLTZkR|0?AXX*7!^j;pa>toF>NY4}vzkb@03q=-9Tfl4Y&Mo%kULaMzTBAyo4Dm^nIPC-~Z_ zt*h&|;(NEOk=Nm!15pK0A?m!gugi@n+dd8e$qrR)n|o7UweHN=$ft3}}RL6xn-1;`$LOh4(7(uyAQ|{y{gvB2i%eCioj#|vi@k^?zJg@Iu zTUX%}1r&Izn_OI7N6)GaBJLzNZ}>4}r`@qa6YEyQt)~#4!?s;(EOmXHUO~1L zz=Bu49^cA!$FGP*W}X2S=oPd8B{UTr5$bSe8q#Q=^E90o%hv6Y<)rKeSk}{{5Q5R8 zVWd+M3wa&|$?0)I#yHUp>EFm!=1ixs6X_mNT4_m3t{v01=u1`=bssjrfmS0t0JK^F z|9g+iO#tJUq}e{x*IvMB2c>l6!I<;zx)G&CMRBc>E)c%*p=&%V@f^5GjfnsL+QTy$ zuGMv5Y@2!O-;SfeGT3i`B+SuB&}+>~c1I(>uRXpS#&4ZKWEJbRE)wi=0H1?$SB-b* zpe~9)#N&zN8c-h0pVDEdgY>g+Neh})ZzC`#1KG{g=(pGHsm>2{*_awvv*Q2iUDyEa zdYhf1QT0+juUaqx2cD_7D&RYD9o`@Wy^>LXi3SOhN`QF1Rw8RDu*U&M(J5FsQz)1{B^l)DL7~V^Ql_S9lCSZwG zHlNHkW^vl6DiGnEBVS_EO9Q+rVE%XF|YP@E26KGN+J+HsU;?WV>hC}i&$wt@?7XXUM(a!SEp zkk@SaHkgwFj*tfjD&-~}(buWVu*bbtLwXcFhN!L$O|`Hm-GFmwas(#c!IojKEHgrX z)T`)$%LF}65_0yS2)KGoPju^?K!-NdKD$HnRB+Vukk4Tb2gP1Vx-)+OjX*$x_pQ~i zH`fa)?IKtuks51aB?-OuJdzw&52S;FgpOHw(T^wFAzlO=@YegHIhoWDF`jGoXk^k< zMCF2^$wjOB1ahFi7X~tI>&dT~cfA&863W(b z=Snv6b09~%AIS`7p`amK5}E|Q3H5?~u7-#3wg8-Yy;s7lVQT2(&%Y!;@0Y3}BE+&& zD|51HH7KFxEuu4QtU|aT%E%4)UP7S9B*9e9hh63NafGaKTlwzb-z+OCMDq^A3^cR6Oiy z$OfSVT+<+&9`*BE%y}s4105v~TT3(s*7#X38E)fxhOBQ$j@VNtUck5S@ft^KF(K7L zSY-nPY`yO|E;p4M*TrHDjEfSoD;u)PXR64dma)Q~4+EEy&e;O27mqBxT9hN7uNi^Q z3;CahQ|!jtRrIF-&5(a2=Eu{H`b`CctFv)k8zD#!U}sJ?Jlzc7`sq6H;qdbm6D#iS z==^yy4K)BFFjt@PrfO7Q$ZxkeKD;%o2>GbHTJqi~x^i8ENveXB_u6iDErTf6WlgAZ zGvJMuS3|BYhDNMQ6<6%nw_4Bi)7b=|)|J}z+vkTnn;h%cOWBR>$E z6&e5*--v2cBc=fv8ycE$@fFLWTq|>RgCAfr>Fd2qHjLdjVZ*bSDsGI#Qr} zZ#biakWu!faWEgX{3bLXxabG29+Z=)#XE-{Mrh=N+e$DS`Su`BgBqu4Zb2j?}La0Z9NG{ zwF7wosHX7xL2;&j_cp)9B{4A_7{HgJ;sN#7ra-*-49vV#CMeaU9`OVCmfzPsvqh zhCaj-uSW0XHh1P@B=-1Ns|Xr+U1`wR3()%mwnh?^u$Fm8rI+2!1BxM5iXamF*2>efn0&jAJ<%g%EU-?nwR&Tf zfr)*d+jO~XN0q{U@uJwLC=~QK)G3f^u0$NDj3Uj1&mL@F4EQ>n^m=_tCDRxyp%p;l zr|7Iq>G(Rfw+(_>YZ<0NUl}(wO&7oeJ|RI@*_B)yJqT!>9K^ixx}^}pLFElC{ZgW( z0a&67>H}ycXN}VnB8zNbtredpvzwyXxIhKi3qE61A;1)@2#y|mZth2b!&KV4E2y`d zqf`aSTl_zs6~R=t=E)rlJq*^h-n1V)n9y%OB5y{cMTq3ik8bWZlS`$LF&P?+^0aaOfqrxrUY#Dv5)jJ7?sy z-x`2ry;y!<+5p{yoTLVD$wt$#v=UHb{&`?lW#gNiiC(~#(Y>x3562O(KJ-vSH`l>?z^1y@R7k8cg< z5hJ`D?rY-}q)Q_KeaHNdgRpRtO)4hk+pFPu_`}T`Tf9*|zja1>79%IQvnO>}wx?&{ zC%9heI=;5WR--)}jl8OCJ8?o_&rJE+2HqFGb^3YS+Rkk%O4^Dj?}P#Q&?aEjvirum zp86$?;5TW2CrKz+@D!whk<}#79@sG}Q{Z_XlRSPB2uo}us+!XP-NvQE7pyjHtO^SK zWK*3EeKkg24RXOxLvCF#W}ltA!mV1y02N18fIRzq1EzToE&Q!<<2{;^x15`I^K$CR zI>jKM)U%epL5lQg$ZtHw6_5;OlK$aacM_F{_r{CR5cI8K;1Kw_WnUUmL<6eQlrLky zThh&!}T(^$&ZozUnY`~%}u_MKB|7$--##Q<>E2%Ij_ zGqOkQS;!7N4OyUy90is;l!iXPg%LsGh2Z20seEqKTLZb_7lr9-<3w91v56$DCx)C1 zZ7hfT2{u*LwuhSR6=!{>B%elLbu^rD*+XJT5w>XYZH=SVy(0W48$&Sj}b|`v$Vx=zjyM%kFGQgklq> zU?u1dirAJ0)>Tz(G*nUb^M7@N*fQnt{bnCZX@+>g9ZLcbv_OKY3RG*j>_l#l%g>}rG`inxnC+|uI0J;EpPT}s6Cp> zk|At>1lkB0Pa6tTb`G~*9ESY5e;BLNDDQ%XVTu51yLy}0M)Psw8E9Q{DYXtA-eVCU zu<2Zb@S?uYBI#(%_4!%{6T2nWH)H(11_(Y7n63(Wr=!$ATn;9nW|`8MtXf zlO!krn1`~g2|Ybq8_Bd{7Q?M2>j4O_0;?2o>>+u_Dfs7|YEi^wz29ZVZsr2N33OzH zP5>|UgrDFW`qprlKqr&A`O*j;9p#!^>O}88z93!5AyTnMH68Q6%7vx z=5!?13+e6)5Z2D1jF}8Auypl0>zy+}2}gb%=^v!F+IqPxGqTLGd1SfpfUdfT-YC#h zghxB{ew_!(8@=M0;l&icK5{$Yq_>!mmK@jd!#7x2+qFSsokzWMUm67;T>hGguT=9k zwQz-2AdKcI{qe>H#kwJRb~U_a$11%d2aNiQ+S%S|U?b)fkhIt*V&KfN8=K#5BA8Z# znGAUIyw~axK#oR_0G4LGTFQp7al{dD1?16@uX!xV?bWD~Xay5pgAAMrm!suOK(ILz z!r+l>$uDqkjRtkQ$L1}TBfQ1&`J|0N*8S9Lr$#2AGQkKMbO(TwBDJpe*>HegYS{LS z`Up!ujdI1;%UzA_D7~SKwk{5dB={1QDn764{yXWe2<1GVvAjUyDNnh@KVfoIeXx3= zJh_*`w84ia{9V5Y!OIUaXrPO43qmvGC>>Eqa@UJoA@dI@L25$2nO}P3F$|>`?lo?x z1buUyYpa*=IGfg%W!uf6Tn0ScBe zwnMDLtlqy)qvRV@01WYkgbu2^$Ea zB7W7j_|>x|eCtI;g)p@wU05+XQ>d>;68?%UCj?D>YN;uqh_V9i<5fpNKf>|5)-NTs z=%rGote)v%kY~hxrfF$x%?gwYb_lF8+a8l1lKtorRVZZYj?Xi>df_gm4nqTRs!EQc zHy#J(7d2klLN9m=)c<{!I^Jk|9ECO)rnT{aY`r0+bUu^}cb!RAKpwIt@51X4~~1M4ReH zl-EV^YmZ-c1fzstGt#c#9A}s~wr~Z8A<8eR!Vr5 zx)p0%0fHN}6>Z<{p43KTza>jkK?emOq=Njj8?kfg;aSMTM5jSEFEYgk?5mU}+98 zvK-?PJm=ozOc%sAj?-d#h^7J@&I-7bpc8GWjRX{!ePZ=)jP~kG{oMLqZ=Dp~ zI=iA<;=1P8q4|MA);6?LpBe$UX+_hL*@_&mUfWxTYyq~qPs2b-QI9W2Im7T-djx*( z5GOdOeNsR`QcK?M5ht(UUPIQ9U#6*&p2mwYdqJ@IL6<~iuIrp5%r z;tcHD-G38NJw)8{KIH__^hIC(>UzFi2FhgK$+t9|U$!QlJ3D3tw;Yg%(9Y~Qs@?Z{ zj}*)2#tiV@J6C=^4x$-E^2ix8+n49m0F)@febeLKaJChS_5PIQzxD#&tETOGnTKcE zV8o79F`)^d?IzNYRMh2fN^DoyU8P5Pp(L7ILVmEjMjK-nZw}M1PqX?e{+6vxC@TG+ z+?yWRRD(hg)7w62$5gA6|XmJ=TeY#eeOQ6U+p9>vf?D_uX+2#X3n^gFlR%IL`2B zxV2o$Pwn}7m( z;kQ~=?6AO$lj|Eoao?1%$|k*3Q?pYVzyT!rT=!uJ#r)di>)MDOL3_bGm>RCVIR zZWvfvRw+%7m?`RP=+l6MNaznt(wQRmlnG)-O3LIVN#-iG-)on8&!_c+%RWmk-#H{;Jl;=>YNIQx1$a%OsWb4<$80X&= zLBM4?B+&fLF48H*OkSuR+ld&g!XWiqsk5xtq+u+MiUqI9nr(w#o+2>UU;w=P_2Z=U zXEF>2yTETc#TXFa-__Q3HEIj-psOd^jqg2JbIs8F9Mm5f zZf&dW66=8gRvMiBv3b|B@z!OR`st(}!FX&)SahjWa+vmnjE1L27)@X&!OJz}4hW;6 zRn{Cs-%vEj#|wpf?1DH2x)!^EEkceX_h}&lZv+YveL_oNIMxW8R!4c?Ot-alSB*zi zC!`N?C%f8biTK!3Q_fJsX@rkOtF12|3iPtkY^(`uc~^H3=p7+T0Wzc{r{+Gx9xvI# z9vfGB^*(p65E)3fw}_2Dwi+bdTqh_2KMu&vQ+b?qBAQG z<5mcJP4m^NVtcB+O`fY<{-&|QJ>BKbWV~D#n{z!Zi3~}clGZ+gg1nneOX&nqK!WFb z;`)oB=uPdBiF2@tbq+W#6)DE86V!dcz9?Jj5(^5JJX~5m5hjjI^#`6gCBo?x#jmoF zMNK&ZKZnK(9RlM)5lpGPvTddWM+{GL>6DWD2x0cXQV(VsHwc&bHovZZWK|l%2Ffxt z$h|ml3~2d=b=P&=;|a&?b?n9vR@Ug6`MT+!1D##6JNB6BMKVEmsow`>2S`u(Jl7W&Is_c`z7j_?%#L>rr$y&M&yF30Lrk zNKR9h=*$)Oq2=QmHTANra8IlAO|&;S<$|U4P`A=LD%uW@ZUb8PV!#jbq|DnW<1=rs zm$=O1(F^oIx0wbDstLL<+PkhZBXh8ulh4|eX7~(bU?0pe7_}rc08^5!9IsWeWGt??>VSdFt;tNjVv>s>Yh?Ln&ZwKuDAQw2E>s0yC2#?tD@X&8B)`qg{z-Jj_(ez_1-M%e z@i52y${JfInaK`_6su!E`|eh7Uc#>e*OZr*=h3#{_O&*rBn`;m8>X&(JX=xD#AD-3 zWVJrv-zr5L4C?8~8=&ZrLR`URq2LR*3a0equ`ur>P>-ELf?K6K6O%Gn0ZLsgXXJX# zlYj|eMqg2i@LUwbP;BHisqR+qc!&SIk#eMTLCTq(sRx3wERXXBPi3tiZE$M+atI0% ze1}Lt?9Ht;(@AR9A%^s{msxH0h$0#x1d5uu5S`9(*2-~iM4>M67${tn!bVQx?_49j zJqRm6;RYaU3hi<#j?Cj+%*bJaKe#kzw1e4nsju90day-_hcl>)V*T|vZv_VDD&Ay?sMyHxeiY+i z(GeWw3j$5!m2XY_48|_j*$^;Tz1VmSOgdQw=9U~ozPUUAteDRkv|K_f0|OSbaV1HC z-zAw33K0Zi0CT_s_7R;j@!HucwN#7Ow1VW955DanR-#x#0=;Ux4sC>(EWE|FJgrgv zg6k(Zo|`q;d6Q2Y!Ou4(Ox8a|bxiU9WC7EFB}LJt+w**)z1d_>PjNcNR7<)&_lj-V%hV_^OKJC0#;FyK;e zUaLZ#z7Pl2L&ZIEC;zy6A#O{irka5+ns(a@;r`@ul(1R%voo1+9CR)X$dl|5#ZIz^ zdCn4Q^DNxU2RBmpnp9jy?S+H^y$XZh9t7Ck9YeG*Ri{U%3lR)2h01HkFITbWGtM@6 zIL}wo=-!}>8wh5+3A)7_$}X%vY&@vpqxh%S5Of4Nx~NZKU@|MXD?tl-^dKeN9+us? zfjp;KK;!;sKCEy?@wc9B{aWDWWU3;9VAbU`#A$0M+f<5&1(Wb@EpU%y1sn``4XfAi zD(FQ@3(PnuTv>2n)K(AvWsf*W9x621!wHria;1BUGv~%dzfQyshfZv|>D8<<<5bad zy~m@TUZS3k%^wX}cPns|o*uUbl}KQ{I9||!VBQt2fbKKT;XKgTyLR5XC2wuO)>FPi z3B3>ioZDfDm7V!dB_&H6SWN@Gs~(S4FhW2<}ND*2Z>UYiK==4 zd4MF?AMFiZO76~z0uO{<$CG`>!3)#}`$@p$jxG~EOK8IWq*dS{WLbCXw8 z0$A_lldA_n28OXAa8a{kS}TRhrwuVwZjIT%ror9rK@_uk^nM1j*DY01FR~$g=T{N% zzSW*&aZImM0sjjbUcW9(VTGAT-+NJR8Bq;o3EFE+QHe2Vfb;x1{2@lDvSOC(9Mvc7 zm&iP%7!Y0~fwiyx==-ERDNfq=uOhL!e$`nXRu)PUnvRy#`Ooz`c}NTeOFpQjm}9R!6&V$0mqeBH5ry8m0;W#qtZYiiAn?eK9L_H0M_sZQc?t2Q7W5y z&aSWWkm3Z=e>kqpm-U0=%RoIumGgt3JtI^~_Zp#>2}V>u&2$t6lmz`)<7suA-WkY> zG3;!|@zg1i9t|n*j5NJgmkN?VtL<_RJGth@UchR^FwOzPsBBKd2X?09>NKye(=LL6 zZ@=|xMUEmeBwNm4qEu~VIMPhi#Z!hzwGU-&&~IZew(BwxR`I9mMC=Ce`oxZ24D<#- zQ0W`ohN!l6tW~+*Yhx}M0%w4DQD>RVc+e@2u*MshFyiJ~6B!NuTdx;(vr3nIWJ1;| zbYE_^HArZD-^fgD+P%ck3D%Y2bs@il%bt4EnTzU0Q_T_hMaE+ILI7RAvn$=5B(rnM2jbYd=7e$StcXnxN<09#-K z(RIhLDOVkUl(b>f`eo}KVB^^d-XRexQ^{7&P}`eFJd6A;Oi1oMS-6fLAeyc1QxMxm zEL(v6qgl)?5YoV-R6UCN06gg)?jH>UtPCoZifHyviA5TbZ>O;Q<|oMm?I9eyB1ZNwnx|m4!OD{COcQRrBs+J!OEew3B-_HA!OH%zp?Cy2*2lO zxcjUnu+uv4mpCL8ehm>Z^#X=0Tsdk3efZcVVd)WFpQy5=KGXr%Vzh<6QT01=dpil3 z%g?bE^z&eKnQf{Kk60*;mXkZc?U6=OSTu;a=p};WeFNxih`50%5gT4Wh_=mRDT1TU z1`UzT3IVE$yO2~!1fm2`--nVSnM-opb-(6P`9GiMe^wi z{z6Jyn`qTFV63b=dp7G^!2uY}dY#Fb!lQ=12WyYX615+}G(aC&qg_gs#4B(+`imD% zxzDK_zVan`ps*EU&Cxv+!)FA**!Sa@AjpJ-EFcrM%cQpM{)iM24grXoidelG`AN5- z^BiIaC034_1_MOIor?K|^7uE^ZJ}o4^!aDb^Jw;kMD8_Ze4}4oW>n5!Dvwvv!Hpze zvIVybulSt{xD3^MT{OfusSN0HyI$BSLE?P99p@us06vu)c87z?X_{h^6C;#aB9zRf zn&qZ`hEp(Hgvb1Xg{LuXuKB*wF*c+cMFrn<;rPmjyc{M7OWaA|b(4QCx0>M+X;Uk=4Z+fH2@)--Ur@d~_q??A}4zmVlj^Nu;vhoWzh z(p~S}AzSS>n>kD(N4;>(+f0CLYc@fW1hi3r@kwW(EC1lC3whbd_=qBk@)I>bhnsY6 z_dq{mo6$w{*w{~U7Ikm4Xdui5HUf#TDttq_gJeauL2mAIxZ~3MvY=`-e`sC3N!UR#v<8`a02?t_k_{5z~R!1`Wv12~if?|FHnEH_< z?8%-H1@f}2H%m@UQ3%77FM#HON{15Z)UaeXn1Y1-^ywlNopc6zhH*=pc|FdGH)I!dL*fU85><`rwZ_m|WdA2S=9jWC^H-uq*ED@5x%|th|P<&dM8g zwb4@K@B?q@&5a2^cDU`^6Uafy;bVQiw=NZOc=_b0u&-fUkeH!32XwVMH5M5VhdMGd zhpUJn9^%w>B}ibhH={s$F6jo`tXo4A+@b{0b~i8RL^jXo5J3B|Au;0CKm0B+1Q@AY z&+e3k4ph$C5|K4@$D*L)q!Lz=6A8T$b~Y(NGL=PI5*zUm5+rbdDfQ`|CxC~hh>O$x z%`C+HoXsbQk1d`X`NjtGgFMfU9fN$hMMwn$xdK775xnAWdugjmueJ{}c8B?N@Ztv{ z#_l2Ln*_tPd$14Il&F86h{ck-S%U0T4^p{P!s8v|jHE?Tqd{L!Hq?A{OBy5}{d z$H=wv`Rha!hS-0e)Z_qj;IXrMfj~N?pC&woRnqO?Q(GOHQn}Z*o+q6ogN}KGG~ghc ze2hJqMwFL%QW`9x+5k-l6JJdj;h@u)MoHP7BSYoH6)3LYqaRpL#E8g3E;AVXX8V*Dzu=9{36tjjupg ztnjwv@`OxQHC|i3ty}GYPMB?h3p}KR_8i}I<9dZ((*&BY!>Rv*Okr+Hy}%V>AXO?! zD>XX@*?0$4vRS#@#h@4`B=C!ltngewak|DR$Ow4!Nr%+0i zC3g_0o2G)n_kBs@r^mM0%rv~-N%n`HfFg3aw%9uL#HZrVt2lEwu(LFhvhrK*N03Uk z?n@6p)IAH0t1tR{+iktO~4+7vq4R}%C5I5;DPfcINP$pS5xI0 z3VvsVFUl$XQ&wJcFRBYNcf zqZlFBn;Uzs6-{0$vqw>z6pu32cPTVQkx@qC(6wHzOqC8;sZQNK*>P>`@2L+bXj6M%DEYydsKI zN_(=004Q^&aIQ|hLF9wB#yZ}WQCn|y*PQT1AEJdH2^xN53~JC9Ox5T-Vz?Jf@s%=E z2nnar!O5s00Zy2YknE}C1HXZ48Iqm=A*?1nsws8j4e*_3QZGkd#P!G~5e_E`D)>gI zC)G*IO5ha;0$SHEDSV8s%~BIjFOg5a&0JP#40Y`O6*{&{@7-wliK0@jpK(4TS=Yt z*NL)xunD*xgTk?ilO;M9Y68O3s+%yIutXjI<4m$&?Nj0Kp3CX2kb)qpN2?{u`NBn|wrY#~@+ve75JH>0D z^jYo<3OUta=hS(@!wjq#j_!MiQw?**5hypsU)Kyw6&{x72vVn@!%9S2i-tW4bH56({S05hnX^@PvfM67G6{f1vfQv^bFMMU4Yy9K8U3Pwoz-jXub>0ZlXiC+jdWIQZVPWaW+P?dDs z(7x5s&^x7pnZ*VtXJAe9b+o{ho35VeL{PmtRX36Xu~8putj_5W`ZODbGadE?83z=jBS?!23OPis@FS2j4X9?Dqq5wrQ`u@K($G> zI!a<(G60S&fk+R!D}nAn;afuR{h*l6pf-;!FGEa(;wUi@W}L#R%|jlPEJu#xc8=1T zlEIi&5v#Gcy3b%%*&P0nrDcOB~$mnV) z;;M$kE+{i=B2BN<_IvQ$wzT9lSq z2Cw6ROkb%eu8Ck$r8h9Ifnqz29BX#MDEmH>&`^ZwRG=&}`7=mk&^0|Rvwm^Bd7Vj!*1=?Z$2n&h#hj>QeKgC| z;FtjikFW_?`WuKfwhLt7;LO3cAfSh#-2rRKtt@!@3yse@fH3V|7N7N|YN@4OPkBfbcMDbR1 z1i-k1EYYF)Pt0VJepq+->JO~-h<*VtIJ+T`m_cv|)$Kk~SIh%B{2sx4ZDQ1exFXe- z7YkK*CPERlfH^L)WqUAWKb%QO3uzBi zz4Y3;o+Ru5OG^{uI$M#0Ygv>I#7AdI3IteGd5OfGvG_P~FPS18hP)Gjq}r>kl|eU~ z9wg3?#Fw{m60{@NXTZsgdu9tfUD;kRUYKWbu33SLlct0nYV(1D1Zx6%+g;$fjeQ2< z%g=J4O$x$s6A#W2iY9(6;MR+5j7?}haG@syL;>eiLJVI}e^8b&c+y-NRT@(+QdKvS z)LuyDp71qldNG#i0AE0$zlCtYU;jl&9p<|zs<8i6`mZ~`Mc)q1P8q2qe0*6ged`2= z>~XMN_x(ZCg7@coTwy@uyN@PNC=7%S`U{WDBTIYPI;!YD0SoU(>apxc9aP{>|RR$$zQnx>5|O8x>sQO-<>{NDKUhVmB+G zC%Oit^U5zE5o~k{m|lbQEf6OILq#RtrH>-+Iu$ zi5t#f7i32}Tr+$RmqtFn+>q8C-ji%A*K6eirV3aWT{+$_^&1NO#P0iLwG0Vki#MnU zv3(z+kavjGs6oLU?o$O>4Y4s zDNZmDru=#*L6UNL$>9Xit_DoRY`CA~XWu>}1pqF(iqp6t(wc}(u82voS0-+*uttS~ z8O-qgkS(g2D?#uEjs+>PgtD3>yCQNPf7?*TAKF09Q&()&7FJmYgPVi&6Q@+Hf z?-&BoNN@^a;o3(7LdfT&qm}kmOP|wpy_Y|yqC1A~>ql1u9}v%~t>0m4UOg%0r zgA3XslI8a+F-i~(T9+j?pXa&`tr)4sn9zE@gI;N!Y900|+>HE^i>0agJ5ERdMz+kA zO(-T&@`B6>z=hhF4H%PaB-vG#7EeI~5jqWP{;=VNHlvZW3FK&W#k5?)^&Xqbv-@IerA4%IVqe$fjKL=a=?epUY%lWU3 zGM+9O0ZAeRF+<2*l+>IuSOJ31)VAf8@9W#AywaY zG8O!*Uux1iRd6J_XX7+<&Te9%Z2-(n8h`;pet@FWye7KwR*EHb(z4Wds$Zk?4N{?1XryC6n`*L)hX zUk`8Z`OFa7{rhBL7+}au&M9qtNkk-{(z9-n*I?Z`9#wUEnYOv8{-hT+-SlER>v1eK zt{y^>SvaT4fVa2y&o99LX1ww5v``Ma&O)oHLb66hyCUnq;%mjxQ@~qODVnh)0*MkM z;%7NICY@UKs2SChog=n$v-LwnD4bCx!L@fLSBkDSkVG2@u2(}v+ZQEn11}C|WpOPK zjM;KWxcNC;i_GBKt&ya?%@qhlAajoNNj#{6(LxhbvAA{r@xZd+@E8NrW*a^&*`uwOl8eZ$cuyxNL(98J)ymoHunl&33HW8>3I+J#D=V8ibGOjMs z^uIOWe*PWDwoqF$7#*@%it8#WS<^0ic;A%Sh-e~-X|K*33Cg73DV^&%$QJXib7KhF z#t@*)1fEQ_P{8I#H0Xjupa@+*i}aYH(MzhcD1$3lgp=}yW&C#T2Ix(z0mqK&j+H$Y zg_pX&s`uwQ!FW|hwkh=*CTn>N8;%smjgKeG4iy*mFOOgcl2AG<6RSgwN24T{wI3Ay z{?;Sg-MY&gPJF)6c*?*FlP2 z*Wd@qwI=X}TrTJheV318pWnUa)HgY^u(Kd?UF zWU+BbT)dWhU8p$@c;VFpLQQ(W<{c98C*l1}qOy0O5MDRRB9$k3kiWZ8JFMrL=c4wUe;g;$&gnnNP7obf}| z)R28dp%YrgkL7C*@tY=V)^zzYV0r6x?$UPywpQaFqC$&+gjo1mmB%krkojm&c9*hq zeedzhPQJ?ZK30$t4bHdYY&_K(nv}1-7VCmGuunvrxdpc*FK;lhJrv@%Uf`Ow-lW3p z623Fx+e z(qm+t31(RhwOnC%}?EHMk znr)^*weEHfaEHabiF#sdW~ncvP*<=S9>Oi`@-RunPDi-m+H}f>97l@n5V;WM`12vr z@42}Eb}#$C!=hwN<68;X26P7jN%(u%+kiFa*>JnCR|vVg*DkkhyA=0x&Gvm$x20lvD(o~D@6CN@KM(N7sB;Ky_`v9=DKCs z=^-IM1Qd@kKV3`Pb~Zzfb}xZD@xVnzgo>y1BH1$uFgkj_bZxFJa|(A{YwZ!~i1o_1hg>*#6l=t1_~`XC>tkkU7a9)a}h^$qe! z?pIa*KY=VRJaV#|Sr2qwfNECPiiu~b7qFw@eO7E>>YpRQ6SfZMTqjNRjY%sL)LkFSlIBnvqfHy0wA z3bAND01mSVKzt5wAL`7xlkw~2KHa;dOt5=A%wJjXr;^sDGws!{^irIJ$u|Zs(?X^K z#$_mAEam5GhfV_DkfJ>XX77EDEuyc^9S`r^;?hmFk1>-{A3Q^aCX9$EB<`xmTaRbZ zOhx6_?VI2AqCO{Tz7hNB8*^vtg&VsT>ToCH2{gK)DEMQU44~UyRzDl9Bg68}ULnHG zP4Uqj=T4BQG!>>sQq#yVQ1b(}6xG^|)k6rhKYS#?1!N+Jl3@0V_YO37r3Xs5Z>tBJ z=kyRR-!LnP)_4Z*IQ1FxOwM6z1$h#JmMpcvfiVl7RolZZt1rY%5FbVmB^D9XIr_u4 z1r%T?(609$Il+w0Mne|o+STxR?Or2KR{%ah0?IW-Qa`au17h30>v8_HjWzJ6c+T|J z3zjHPx3*%!3E&PIT41x4n#LPcWG0>@Gpy=c{w`*F?SAj!nLxC|69Yrl$9KoU#a-tj zg0L$90cUZD15I6_uRW~ZX&Zq>>>kr^y$EUwK5N?v!;1~wZ}Df1u2 z7&d@x&fa@GuW9w(kn_IpZ;pc!iEJ0_NlKOj1xx_P)^B^TR}lGN-RL25ZAhNIhO>$~ zBU+I_EGAhW<(W${L~K)v^5>}KO-G(018bfoclX{S_pS%?fVtXGKj!Ok9;q&PZMB~} zn(9ZQsiLBJituZ0ga8hFNxkcnCH+}qamY4$+^$It0yacQbQ9Wcs0bfmoYm69yvypb z9%J7c*yq;3K0c4DNz3Xr#F=SYK{9%sodPF4MA@t!vu}7+a)WEUsFQUh;^q*2by63w(~{qmYKnYBfh395abN!`!n+C3Ce?Z^c@ z#LX)k{S|B<_(JjYJ9FMi=2chElE|Noi?aKuvzk_B)`PG2-eY5v)|gYN|Chag`gJcQ z!p=y9p>b5LnIRj88qas?^a!ip-)qu4dby7sA)|A5n${S*q(L^vDtN*CC$qAy()~2b zGx82ek+`LqJEe-K*xZD&HHWFEbM5J5jW==SES(La8{km4(f59iuhfdG$hqNwD{Q$?KdIQ6%T2=Fg2c)N z3bD$qb}+2r>f$CgU4V;H}5M?i`U*_uJRC zXu_2FRO~=?Kjvc8{bZd0=>p4!#~g;2693lxBJr<0ik3&BD?8P>JMp;bH0C~x=`h^M z?%iXi$6DNWkGXr)rw4HyJ;rmU?OnWjou+Q&As+G&*UXW$+7W)@HbI03A0mRht^&J! zr)8QnpepR)=f#m1mkFrWa-$7 z&1){aXO!gE4MMYPA{*dw@QR}c2EzL|dVrx)HGr6sv;nTpkSUtf z(RF_@+GuxN+7DD~Rq7(~JLS_wF1@K#B%sKQBrLZf-z|grVM8|LQ5qv+$N2{!)ef1a5ccs`Opj+Y!|Vi z7XS%CxB1#1{$D$ymCG5ZgAfstuq}vmULHjKBReuC3cI28ZE%H8W<`@e_Vt+bLDj)m zQ$uKmmd5bR@uyzD?k?dShEpqdTnLL?VGp+&6&`@Nv0_8p&U~p$CKUrNoP|i#2J3#9 z67*OKzZ5uZWJ6@=fJ1d6hYO{zc zC!5)C9r~y*u&Ur$eboD#^y=3hH)o!oI3K+*ZJerdotk>1Xc5smA|C-ghmOEyCSWdW zr!1tpg+G=-5JMr}bf!x5wFP{jms$K0>JW;eoxVvkpj*?`cZ>emQ*8y_+U$$cyqyht z)v-K1a}c^51*92DsbIu`3Y@jQ#-&xJ3RMy?^Cj1|Ndz1|&Ow?Bj1hR#w+7U~9+iTx zZJ__z(@BRgZe|Nbv3{?5Y;29uC(!TyYREJ>k~A!#s(yZpv?nWP)Eb+9-Ym&*z%CHKBWQN;D9RYz)Z1PSuh%Xfw(9}%Hay>2Gqge;R9!){ zvx|{x`Tw(acB|6lSbDw!-@|t}_CZv@)x z`>UP1rx+-js;-?LL{tnZ_@H&>{Ec~%b{qTEC6V9S z0m}C$_LrpPfdaB0Tl0i^{m4KQ`2*8z;cYHZ;!)lTr=Flq;`s}0630Co=EZ>Y19@Sn z=lb!jc@!P+Wadkt6LCxX^m#knqd3^7wi=Rw^3H-xw=Zx85g(FwL}Us^{d22%O= zh#m_UyHS;(mIEM3gSNnl&06isk^#iDVM>}DZpQ)SznBd)4m%ssdw7-&3i&h8`}=t1 zoo32jE?pcG#!?ywAvvVJsREAj1q08vO8j(H=H5QXfa&F3@u_WvkZQenQuMDWUbiPi z6oK`ehLy>?-iG*n2Z#r!l&SjTTlMfv*hucXf$Nv-GifbweRe#shMKJ2U`H2?OPJFl zys?qA+25UWJx)D_8Ezg2GO>Ym1}87@>Ahk9;yUhz?NeihODke#odA1pc>(AlMTTpm zv$pueJ!>y8;pr7A28|BR%%1X+nHdoQXiH8V!GbCi4L~XOr@D>w?^J(3H@0?UWuF}8 z80FaVfc?#;n8pdBI#E;!3D>K7HjiFK#kdr*37}`lYB-!sm9{C|W^G1<60@6!EvrEE zo(a@Et^HN8by41NIWaV(L`_N6pk~ii!Pf1o0+Nld$lw2)kiC?%Thad8Uz$OLpo*(h zNd>3G#wQ1s9{$}U2qK}-n_mqVYY=tVPF81qe2WCt{v}T$HhW%wtKvhoX-cNNI)$f^ z>7O|qXGm%q+7&TZ;bI2gP&FMb<2reX5Jtuo3+)u09WbJ($vQ9sVMopH32}`*?1!RO z;mKb1b_etK{W*K^uqZWj!88^sJ=0E)HfBW5E{_@9v`p2LC_W%6!Uw-!w|M=67KmYmv z{q*zy`sqLY;UE6;pa1FS|NG}S>DS-*?|=OBAAk3kKmOsTeTrXP-1EAEUtu`@SN^?+ z*#0rR_$z+Y7mUZbgG|Iek?~~j^{4o(Vqj0kQB5A$(2hwWe04v!r z(&pL=YrOlpIpY2Kn4y&h_KkSWwYnv1sFv;=7HK!NQUKyH6_lJ+q zSA?DHG}_%8oi(62T@EOQs)~Fp0hmqNeH4J*e=XLJJA3*_R21Sd88MGXH zKDxiUR*Yc%I~n z^7^QMlT)ha<%$>8&5^hO?VA9c91WO9f-*G>jcK3I4nksRoDBj~1c*zfh$SCmJ|TSD z+i@0mhT(=Y!{k5@$Ub1BnFBUTijEGYJ<&nG^ezg#oD;zV<0hi1Iz_~QK7Od_Im*Eb z0W?LBb{xowqJ7{+Mh_Qz(0_vZa}LyL7%p>tM`c=IZVG`HQ4jm6)j9&8k~(&(PtZ;SQ2|Xc7bjr=$`< z`~-#@dS4wMrvbe|r-G=rNHnQ6doQPhJPHj>Bv0u*PZTwzv92K= z0|Vq=DbWCV2Px|=K7;$@H(q`nOjKDh+dcd7D#)>>{J*q{H4ITF3Rr$oT6JQ9`)Y6* zq2{M3Q+x)NTtsUQO1COhWXZ&!$K-~Gs`RNH4w1^!Qx(Y}74J;;J_@ZNUU86sKxoNL z@6VjG1ZJ@PSF%sTiV4x7;+c6sO6c^EG(uE>L^rU-q3<3evOpN0YBc6ojRo^+*cLzp z@#N6SEa|+Xxc&Vb@xO91#jnpqmMP~xc4tizsD3it5^dbSy)@id=6rpILmT-L@pgHA z{JodZ1J%ACvB{o(W2$UF5!cid1~8gR_je%}%3n4xv%=^A_DS(l2_)}XyMM>cGcMs;Z-#MRQwyV}Y2*>?~ za!KSVr`~WMe>>LTFSYgh?BStMcg~@qc>gwodUz7|GZ{(9H$pE!rvgZJQ?foa6()XQ ze)7du6J?lj^A=DPV{F}9@{p0m@!&AYorl565Wy3XJXI>U8AJtR3nr0x>Mm1nCIx{U zq!#vqsU39+mS+zKVj*=A3(dTnI^4uJ38d((s%%iH8Am-BN;*i79o0>SdcJ<)G&T}S zV^6S<@~(L+4Lk58^kUY_Hs}>vd5s8sc9>8`*uCE1oknyjA7#qowaT(IX#f_o8<2{p zLHw2tP`i)lOdN%xe$xnN-&{w!B3evK3r-gfN+jr*8e2?qU(U2QdV!oVm>ydp2@9U$ z2bgX_Eu#9+@gCU^a#`R!hAbJck%jjNWDe)faFNN%HRA1%fp)n!dBGnu7;wF(v03wC zR8-ZKD**#lD|6VOhlF9RHq-9&XejyO#b{)nM%;wek8PaDZw^bXqN%L z#LehMpQmx(WgDpzv$NHIEh)1yOwy5>Wk%}f=mXa!!H_TMgvL%`GMItQmyPN$c9m;=N95uLPSW0{ng-Q!v064op855AC0i& z$gb@*c8Aq#MTuyA(LiQ6-#zAAi367jF@}rU4jf5Mwd-{`l z$T+_@RO~{|#l1D2VzB$t_s z4uEeB1G%t!$38crmPGkFD{Vvr?w*fPZEPH-&>(lE{kzZa$R(x+Dgsm)et`qM|T<#&L!7*zoj3Mq7#FH zO{}>vfAp@3dR)5e4>2f};vESU%k2?x(=gah;+R9iAA^ak3m1CSO`og6nqhVCYUfZ~ zBG;=cc4LMxset&|_Uoml2|8UT(O6th1@k zRKK_Y)lULCyCq%oZA8NPqSEZ(;7PsK+IrrSuoAfkX9$rPaO*v z))CN}*q)J!=_Z;c-(>oFWW*&YHcV;rZw- zJ1F@XCU#PL zhJRX=j1{34gz%8LA;52uyG3zWHO!^J&Pgc1h|H)8HZd%WIgbltQeFQ2E{A12uHaF^ z-ERmAIO}m1*lUf`3<62XQp0mf$a@(%+9!0{^B!BdW>(HLqYP-Y)TzJX14w#)f)e2k ztWQ`n?jA@S*n@2#Jt(ini5|;(Yovqxr&lLU2`fWaCod<5!^3sn%W*g-Z#rT~MnhmK zP!2dqYQe$#H3Mckw9bL<5`>#t6+OW>a2(%23xVW6)wh}L(IAnvy}+89Fg@+?K+VHM zex8^itM95i*Cbdvy{d=NOPL;S?sty_5Ld!(Hn-EX&VlTa*w3Quuh2V}LIDGXhik)Y zud1Who(c{(Cx#4`y_5|^G%;WE@|v{`h@Kb8_LBXw_CC^jatd0{_;}bZ&E!grok}oi zA8HLLF!=kO;|eH6)Q^S)aG-LxN4d(Yw-LkY=U8<-2_z9D-y($|Y&R)IQZoixOqkxLJ2CN}vLAl`b&XsE}Q z=_oFI(JO}-|MY(Lus%voF3J5QL9yiW5t*ueb1~43Ne6v6ikIvTyx$_2>LdFk5q#Kz+j-RK-np;ip z?m4kINKI@`&(Qk*Lw8u(FX-|&2;HmC)G>ybg*-EH!nF;Cb?ac>k(Kb0{n?2ldZ zdO%e;PCwp?WQ?_D>}J#Zky}CkDSezN);#i+fDM2`PQ@oE${iO+a2E6skReq7h!0TK z(T+PLZ%9gN)7#@nJCmn6^&iMU2nor`zq~j6{)9ed>J*G<==0ud z?_6Te0eY&)0+Gtz_kA>^i+_53r1C>K$`4B~!W}mi3+)GNL%pLr?sqiAxSx-R(9b`ao!abnss!`&3MbkCYMlU=WWfkVCM|XhRK( z?<3Gjda?EiuI>4%$B`P+jUnI3|8p-DYKSpP%|$oKCo%^Uj@PQpeLlIfTVn0qiBTOCGhXlk22O(M2NUwCk zl?4r8z$Nq8Y-{Aqvq$a-IscIb%2+^6C1U=LWhQSZ`vx%bYqa+U_g5*Z$Zw50#R$}B zp&!eJI>T@un&37Q=bmhU|tAsoc{o zWpRn@H6Wzw>+A;1H!hgr35XJCJAWcVf+7+|K0>+*EAB|Lm~JQ8s^T{JS{S zUZ-Q?e43(wXntrPxnkL%JgUeJq8;_S2Hc^TH>6_ZS02E|RfDB3dazsn5!?2rMxx6r z65Q-<^Xy;im9W$Tnj`-2I=jCer>&^gk~dDK^~f7Q>~0vMj4yh8UPU7H^|==U0jeDG zO0Ni5nsG@z5++((@E+|gWf}3tMOYe{(U3YjTCM_RYpgB0udAwDc|Z5F47+e>L)F+I zP+p{KXCUQ}RYH@d$IeE}I`|R2ziNMmGORy@v<)a@OA3TE#GRb+py!+JR&h`=}QqhSZ13>q6&DoAg=b%?xb zif!_qXR@`_&qvYki6@ba7>q5$TBvjsy54(G!|dY<-F#}C!K_x}iBOGACf@4kLm)L* z5acJ>-%3Vts2}G1j|Y|j&qLiw@c-ZFX)DV*;|AVZS z_aAzd$d7${g-?$ZE^x|izCeHVR5lSQ3I*vSH@kT3J+pv3;fE3S|j|OnNkf7hMgA(*)7~KkyIM;U6_y(8fX^-J1 z%*_(&Iofr8oTok^WxvOL=!~QQEt5rx>$N}vA#t@B&YAU%vKsrjU@FG%5Zk%;2JL`= zd3B3AGz@PPPxstm@PfBt)a~}X){X#huZn0#Dxa8>EeE?5;2N}zPDjT$939>o*4ZvTiV<-T-ev&00Z1j-rR zUL=P7Iss9O8oUkU-D#K>8qt9((eOC7C;{FApL}zK+4|lHwm5r8ZIjoYaU9bXxFqU~ zYmcc+7?Vbx0j!{=v`QEtAb0I~t{S*#Bb>R^Ksy1nUN8F*H+ZbUW1Chk(>fD-6K})u z_S@K$ffYv*==}`s_Ezx{6bK7KO`Sc$Vk=y@eISbq@B_dSq5^FB_T8nL9)iM@)y4m$ z;NjJJr{Z#ofR)|DnZS`7MJKT#Z;9>{sFPMjXo$<899zVj!uzXewS-!kqQj__UCJ&x z$AVAy>MB`|pI4a5(Nmo(9squURMWLY>=1T9+fvW+H&b9>dBZ_zUe;k_iL+5#bPRB# zdf5UUqCz+CFg?iF(4*IEk4$!^N!&OmQ1*?i_^Xzv06)(iFUDan2}tbFl$fSRdF)ZG z)8o^P3Si%1%bKSLgP^H|5?8AYGx9RyE{B=Czaxyj)$~rp48B*aMU^s-$U-!kqui@w zu?^-KLM9K{5CGL^bMot`=pg|8yqgBVuf z5OgDmxui!>xN1Pnr_}mFu0iu<#Uj-_9o#MQ(L$a2dxXFs_o%rVgKBI6 zP@C4LF$lcH@e*mgc5)Z|aGAk@((r_A2Pn0xX-7d9Fjz`~u!P+@4m(xTr8 zKq+EBN3(;sRzb%7YeJb0SOpmibsClh=~|L??*YQuV&Ov}fK(htqvce=6&v5Xr;)=@ zM33Hm`;_{yNA;Jjb9X_Ca(73I1H|v#qrBc61Fj@Q8ern$UFTx(97MM>4rBco+maDQ zdQ-Z%Ysn|1g9^R?&|4hA$`5)c?PeHs#Tnuld{G(!IXeg}cMwb2YazaNesH{2D4E9z zBvN$5kF;+}`m^!=?(!qM%pc_KxdJsP+QQ zdQ2!FI7|GK+9~BngcBw>k@$dteL^1>NQ0vi`3QxGcaH+%_*bK_8`k|6W}k<`Biadh zf6z|oN!5_d{?l3XafjmNEx`5M_l}sC!6Oqik!J^1%}*@pqC;o6xku2Fpoe!Q2eB^A99{*IlEz$EOfiZKw>UICX@g~ z)bZgyVL4`VwMy-l~aD|k|U1Qn*^~STs$tL@(hC=$7 zffKcn{-9W+4b6zS_R9hG<8L^YK^r_Z_L|S$k}8)(M&nP>IHsup9ae2zu%zTCDF)xd z)xe zn~Nz23Q&ZmxQ|9)8wkQ)45NZfxnd#xk&fu4-Uu*w4i`&wU<#gA@P>DAf&SZMmpx+< z2ibwzUXA!EMf?<14SyLXZqxy3B>h818D&CeIm5+v{J5BwdH0;*{9KC+-22jaF={em z#xz!hxp?Wp<^bhg{RqzH5S#@}FfinXBqs<$Pp>tO06&gGhz&BudzU&2D3EN(bCPcZ zcY^y2BC+JQ{J};SHH10$+O}>!{wXJ)g~dFnOFSED0BIuO2X}IY&oFL#C`h4Fvm55Q z!AduuDn}ZzH{wtkaw%!RFe9pQqXAv|lI)TKl+z|2!MaC9 z6DtaEf<@YqQMzxU>q158zFTM`XvqT zw;WN#dWht`o@x!A)ce%WI4R>*3%WDto5O7!grVzHpab3a<$C*D2B_5G;MkXw1!XOVSBHLSJ<3;5eSvM0#T(M@ z+TmcnWHkA#WxcD-!ozyhc)SG@2z52;%dLG#F<}g0p0&M>YqgGJplU^tj#)eG=8hXQ z$_4wsNkqbaf`&wfw>}~v*)kw{r=!Hl&_dmx%K65r3C;999GXw_k`676_kQf`E}(uF zmA@Jnvj5y~#N*$jtoWPkc`Kpjb-ps^QWT|;lo*zN)zlK#s@B#BHL+WN9k8J=eV-9} zVyT%_RSiihV82Pb*dJbKa+VC?Fw}NQnCw?3m=5??VU2_JAp&+nVxLIq@11&63>y8* z_JME*aCJD3d;31klq+-#dSS`#+bca1%cC5|gXuN3GJzcjhc(|2y_CqHD9}4X4YCCO zAoz>KUL}Yv#efXwPBg!tz2VG~X|)`_aIW4(g#g+w>a&oL7e~okjA|1l=9lK!VmcG+gFi61)@}V@tAB{;eM*g5kEM8VvII)flEELF!;x4{o@;F>5{J zLx2ZP@yb>p$cdr)Zdrc+ED}IW)0nSb#wDc}W+Hj>dq1(za7$W)10uIrO^zM$kLT<& zfXK30zMru+5o5_aLiJex5cf^_`WM1>VF>KBYp?e9$F7rHD)t4~r~~%!pdj(XK!&y#x(b z>Z$lRMk$rO79{Ww`V(;RCFl*4+#q1LTR4 z!zVY3_vlj`Es33XdqPx*aPv@h+NE9hWA6?Yc#l_wA4kDv8hH`ObIJ3TCr|k%}4moSv^ZFRb&GyS;BEM1#2Zs$9cv!eQDBy_8vY zMc6fs=Q?FGIm*To9WH&kIny8UWo9NK0xdsfF;Tb;1hv zy_z^_(kVtr6yf{Lv?YTraw35OVSlO1GcvpW;6T_e=|f1G)N6~D`CW@4uhuUdaj?>? zQM^caTQR`8EZ^V@C|EUxU8sBAVvVOtMZ(1OrnCUspRLF~jJ`SHVV6sI*aci6kOyS% ztI-sDaf!5!a3a_Uv(P;~@-+gR<6Q!Q(E_wSBR>!@dc({iE;hMFq`)cj2B zXuuBG12r-`U^kY>As^o*@fcL>5T&VD)K++I57I0K`g(fuldu?xV_3P`vBuugj&ml~ zZEQ*&Bz2EDG?;BB-K55qXeW`*EfBcZEdFVUX;Eg#5G`H9|69ZPd<^GL@zY+cL-blb zJVpZ(H{>>XoEn(tx=(aHMT-r47!xV&DPxrDLi#$cT=QYhN*95w4XM?70H-aCM^`YJ z;hsb+XNd+Rs!YClENrN--3fqD5GRud$?3tiLnIa19_&vgqNI~J-&25?na^M(LQU-% zB~*%n`y|5VgJ7BmJTut;wa{>T4q`W^fPThmk8nyiUiKIJ_wo9<@5~p*5IfwavXyEX zY$U=r5(SBnCK8>zFFH2?yEp_?tzdNFGE#VyPoHtPNtf_vjV;czvgSl8;!_iBUvID|3gy{}aiJ zz*L}4l@3DMX*lgi+ryeZYB@M&$ckA<|C0RvDZtSyGS`j3CeEb0H3<+Wet?le4qA2J zXAeJCJsyqL5$ET=h?`z7nm6eV3P<~Z=-Glmf-wOCi8~#6aJkTg<->_r#KS}-K)Ct} zHgSi#Qv*cH^)l7ZSAGi}90`|Hm;A9xNwZv0_Q)P5R0bfVM2AQ%W2zwjeW|}>LswoY z02KH5gsL`z0Y&Rp*GNWAR0YUw$4X>9d|=BA8?U7;9X=9k=RA*1(iQIUX5Z;fJKn})bWI_Nd#hEO1!#EzY*9dNm znDM!2qG^~*NrX~pCn5mpdFp&Ua4HB25dc5)kV`QNB~>_(9zMKHY}8Gft>YGqJ2Jk+ zU6IfLGGLPka{2@1<=b-+zX z&=;PKOs7!`guoI0*U_WdAdq=dP?&$^xIv53dAGez5bMaEiCOXG&y2&L$-xBfq;@g> zOY<;nBE(ujyF;!WXasmq6YxniPVpEwmy*NaCYu_+;BcrHglc6TH!(@Nczij5qXpT_ z*|T|zwIr^E8KHZ(F%9Bwx0x;@?db1}4k7cTBlw$iMEJU@HNbt*0G*hgqrl5DtX1BY z;e*7C*GuX|MGM2K-ibBL8b?T9mPB7hKeq_UJ;s%lYaFuBv zdJgj0_mvDXXEVlsZPLhphqZbu! zdO6#r5f5e}u0EqAQ#$1{-~uChI1Z-<>DeCDg$?*>k23}4w5rB=PZW4Soui_9U>LnS z6*hH&#eL*(EEO2(5iGjH52A+`+t(C+M5^G43WNl?I1>pcWL;I*sfa=!a=)4})XoBq zzV}rjc%t_YxBym<9mlP+efucPgG6Fg96INyxu$;BngH(x^iWy%Ku17^1#Yh*_S}QA zh(5tIc0R||GzEOm08K!$zn-G&Stcj?lrtk3h+!$UMU=amG6mA8HF(Z7R>CItV+7h5 z02tOR*9;AGTt!Z-1yeuLmZtpSxJ~hH53d`_8l}f!+{uymSCAj154r+aphZN`g0?~w z|Ge~Hffx(@*#rKG9jIUzN^%FfAP^DkaB(BL$jtIPFoJ*Du$CCKyKd!?`jCNsL=yIfE{THbuaf=z$$w>2HUa*YgXwO^kWNY^2j~x^<`boKN=$<%+Q0$ zp?e#3U}1C3XX?V)n2u`*;VMjyhjUw@eiLuj4mV5=01vVbyo7$77yUW#pzdu_)@#v$fc0yJ>aD8O=6hWW=KoLt-TmGVT@Yo+R&_ti>n!W z%LK0`R2T;og|5!it!;TKgKt!Khp2(agePTF^M7M6jv+d=W{$&+*gv7m?&r@V&_TBqP!U zLx+D}vM4P5KsHOpeNIV3^*m{iz6m1^ZJCAu-9eGqCPxpE{)>v z4%3Nmt@PgS*+5iD1Fs#{ z0Mz(tFht)C`<+wP`~~iz(FDqFd`g^3k*oq+m#hU2s@9a@_Y&N+ViJs_T-~IiTj2-< z5-|6A@-;?0UFC7GLhqla58KYS#(KPaiy)}X!VtD>JX%x=stZ_+hAYSnboY6ERKUpM z5?Zr*$dtbuDVw*3Up)QKjU7Y`u_<773Ql_=Mv8J2)uxJe3j3zlfXZp_deGyHB*XSP zkZKnF6mLHu9w;FDHx-~x=*98&|Fhp3L7jqz3cv;&Jo#H2FouW7OO9{|NAYug`6?u- z9yzN!0F}GV9?V$_VE;Uqu;+2Vv`QFwi+iDgXobJ)x+X^u)rvk)D=FzO9*iLbahCAS zqAl}%xa1w-BkI#_=2t^(mJd^=>1O)fw^BOWbtEmRvgebOqqe-H0PP?8gh-a=RIFF& z!kp>V<<}su66>M%tP_YNWjIiWR2TP(St74cfKv=t%yBtQ|KiH~F9sU_N>jQ+ammlg z67DI2y*w~hP!KNiBxtDLu_c3b#XGnx8r54xM)9T_JmnI0Iwt0(bu>L9K4Lk&c0g68 znu72bs4*L`XyIGUJ8|G_98MuL$R5eVDS3Gh7d3ed)yU31ksHWI`I&`Vs>1OX5F8K^ zL9Ljkz!de8gvEkp3*&oO8tl1k25+E2%6?Ro^HrKv6jUl0CGb71>meq$`NPM+d|qUo zr>Gv4q|?#EL_$1>-HdP~m7L`H7MHCXYzpxm z(4VMgq^kmj0$9o!&wFUy+TC|FbDyY@YZ62h z7^b8LAuT|_)DaLMB?RFYN4-8k2Gp9%M>QTkExoc?l2gU%K|ltUj&nAW#-lqyz2Imxid#lsH$*U4g?nU$XiuCJFFPQvkpr z$`aMc^k!cwFPucV^?6Y;J0yd|VY)}Ldg%@N)2&b*+}Bk`|5$&hn6dsec}MK@K~oR` z2!2OL?xgEhC1;cNhFk9GX%J}W{(Mg~SB?JF*iRvQPxrvERY@0KfY|F$X2l9S_Ew5P zKI5in_nT?r^iz-DpTwywO2o>Qdi>l=I${btS%V(`4zQ5SoPpZirMcfm%mehnq9km2 z;^eF`HC6%x6j9JJ5Yfs801Cf>tLEoZVs@Qp9vSnVAceNZ9*$WhJs_4tDn&j973n6| z!z0>KHKj5%Z7`2Y2bi%>w3N^6QrwFI9xIL4iBO=U*AN*;5~4>h$d(>Q2nO!Tl78ti zj^Qu6(1r!YxGWRInfD3O%jvhBakrr8hc8;D&_dN5>>R1@s2>24r>Mj&SKe>m@;08Y zdem@vqvq6~`(A(U1=5>61NUj3dk=MrdsgMLQ3)MU34N}VzapePo8Eux5L4Chnn9ae zgwoq#sB5~d?`-&o1qFqG-Wou?H3|XXbgmTATQBX2NY3W{I|PJ{5V<7{FpmYNxE!o0 ztwE?uX7%I{ruvn_K)CWM6{%a%Mc_0m?!?SKenj6!6iPGm55-n=BHsoGNx+kIr~!#+ zm%y{<9`284Vly$rn=RD8(7U<`a^VpfkH3&63qddH3dAy|@PKOUI9O~ZOeEG-m`G#{ zf4m3^6N2XV!xWAOq_h?;a3=YLe2}Ve?E)tNb`rSWsaZeIk$W^N4oRtN{YKIAEQvjG z^Wf0PfOuTs5F9sbd;0-Q;N6k~p{7J1JsxG=dPwvQY`U4oI8j!cZhX|AK`Brx0HuKX zfW2@$8_HcA90N&_!Rk?iRTWuwv<7=9YKA_m+mUhoc~R+6)$c-rT9>4qkdX~`y|A;w z(ix_p0WR#M%0_@66%BhDx)9%fJx2O1w-xPjc0apw%TOXuwp_su1+<}85VjviOZn8S z2Birz&RUh3jx+0ru-TEe$NywLr;!znMifEZlg>y9Duv4745^jYogr&b;v4f~Aardh zaeJ8mIf^BUfN~1hl|Y$`3t`W43vNJM z&rK%~OJO=NRBJ8BuS|wS6->qKF#~@^#^|G=-1UC~2aH>1qi8;M^CbryKG--N6Ks}| z{!J1sa^3!AkpPVIyzMeQ0J%dn*emMu92aJ_b<5G#o#mzSZ>XOx?2GhD5fMUWhukfp zDRxOY%JccsoUZQv{S4EBu%%@|RA>zf*@%X5WXcck`+TGep|Dv&U(#=c`I@pO(%fSO z`0QGTbso9w({RE8si7^@i*atTvF8JS2-$j+sjf^U8#{JP>-nNbBh&2KS+8Nq7oF14 z`BmpA?o%jj4tVQK(tSdv=9z?r`lp8T;O0E@;4d1NGOtn5ud3it!4ENYC37)d7C#TR z@|q8b4p|`@sNy9#pVj=5gP6pn2F_^sPw8LU@S4zMO(PF_woa=Z19?)RS^?~Rc3SE^ z2L+jiaDCmeMp60HG2-FEgmquy32y{d)OaN_8LANsFe{t7u{K1Af>3v*i+|DJgC;kA z5EFsmp>cSdYt!779Q0BY8tCqK4gH6sxxFv#sLp>hQg_!lw`LL$KsRDO$OoJU4s%M6 zMDppq!QaSMw|?<0+(6GRU>~<eZ1#>?|yzb~;+&V!n`Z6@^9LEPa))pnvCdB)&g zuyaisa_G47)^G=KL81kakKVSQdyzs?JqJ=|m>|eg2iRD?XeU;-Ak7BtitXnEqq0v6 zwN`oMK)~y`g*L?pK$0y(*9Pup&-{y}1$PZWy`LMrCxo?MHHNUsa~92(V$lp2*?#=e zRsF=_;i{oyCp&wP4oVoQ&0t_qOwo3@W7nw^N|+_L5CBp4Grf*Nv>c}(=H?Wo_^swO z!S*#@vt&E*g>YnVN4I)e0{38uM_7xS=f-td(Q@$`n+-ulMkXaT)QIOg=4>dZ;cTq? z@qKw~?DdEjL&t(EzN7NVM&YBO1q5g>Jg*qz)yAAgg3W3*n|^oKz3tl-_I4p63ICFP zV^`2FhBepgTtuUK;YeAL*IU7gRgXu+2cUX*l?xHFu<353F}r!6bq^P6%Trn(nFpBd zaX1496diWlp}GZ#FmG56?{RmG?%fV=RqBYr7rEQy{YNmzN{4GRELBr$js0!?t^vLu zd9Ff<=Mda(qmn2+BvIBfomj*teO9Bz;7&~FS7Nrp{>RnAodUKLDANF*cOSJ}{lfv< zJk}y+M0@k3h)7k>*rv@9JkA5(=mFoTg^4_r!k=`KLZJTlqd;#gF5pwc?{799Og_64 z&F(+022A6z~Z5oI~kNPLv36VvA&UrpQZy&!c$o!paT(T}kl_0Qd_hkE`3;{%J) z#G2aRIkds$E9FB^+A#}M!|xoP>T!5_#Mq-sM-IMQAb^lp#FI{TID1iTX7MZFNU?xK zMma?F0o350$?h)pmgyKsaWtbL6qjcp)r;cFfoy(j_~*0tjotd%t9E|A`1(-Igs{Ds z)Kg-J$P$SmvGL$ZMiv^4LSC?xB&_YlCtne)8WuWs%C{36Pdbfx4kJPVfz)|=%;=%Q zh&|@%VeX=Ze2+cGJ=3wR+r8A3l{1y(Oi^AjZ(#D4WV0ixUAvzhUyp!JOGDWm3I$Pn zXkI@)0FU$NK0;r~fuc#Ya247R@ zeFk`4xk`@@oYhG9=G}+2213A_G3!dphfqRX(DN<5d;b#Re3I?gFOy( zO~L0*!5)Vl_nwBPg}mp03La@W!IfDbCs^!YwP)?2nM)>+uRn5xzUrY~H?WmfV@A_X zi4$CLUKl5c+RmGb!f;Fz_Id_4hA&E( zfa(Wpo%)p9vT7tn5pcVDE0qM_-Z&N+yl*|AugKtHI(os|>QP9f7j|jp&4O4ED4|Yi zKq-WPyo?^<7I~KaE#P&)_fO6$Sl=Wv!aMee(1NK4yGbC?4;iv_vk=I}cJ@)?e!QsB zqAJk=#3pxN;j}v1b(;lIU1`WGhafu%3_4!dc0QA!teph1Bb;KL42TA+3tw>eaG{>T zHv*ZvL)`t7aD2HtSlwd)g##+wL6GinqFw!Zs6|iQ>D0HFp-3;I4|RW%r|QaIF^=6B zK0$kcKzy^Y$G4gcg{MKTNXCEdz*r^*gWo`W~LtVGG?~fU8w+WL*N;imL{S4ODW>*Tar^?3r^g6 znbHg-!=P$DC$EvrbZ6Dx z+~lXm^#q=bjf-U@2nNiPL=Mm;l)BGDhloA(M@W&cOAX~@pI#}dk-WYYW(Yoz4l8Az zV)dubuLaoap?=>sL)IbdI4+m*0W&StqNUim2nOOFfpC3v2>ZByx`ja`+`CkSM>T?M z_2NPo_R2D%R_tb{_&O55c%rI$(8Yql zD0$v{_&DxyKR1dGOygWui_01z%A#2lsgD}ED=+_TgcfD-C-jTPt1$or`h-maMgvmo z_$~19~!XFr6xmWh8^mC=p?BT-%V4kQ|WW|dz^XL8@`U#kJA9_B(yxYp(BXsB$8>^i?Yf) zeMQ|+P~iStAC(%g#CtF&`XS{Es@$-=mqH?Ve}$5=AgnTu$QmG--B8JKqVChLE)C#w zXU z_8dWA#I@z!o^QeZCWK=u4b5bAr`|t*Vx7B%v0pS^)h=pKNsiARuyY7P6nb~mI7g^) zj@Rd}H%pJT1gElndOxc@Tt22RBGmASTXCK; z>OEXJhs*fWRj}sAd)LZnakKqRv`G2l^BF%O#?p!)$HqOqyj;}H)vMEMcF}^M%DVe1 z$bj93c!Wp9=5}5!6crICXNTCwSr*oFSe~VX8j=Xr2q+nGf*Zh433`o*TD55eJDH{* zL9Zfrsq4Cn?C6kp0k62-99GrQwj?m=Aq*t__Hp8~7>qQ#C5CZC_cf&3dovt|{CwVe z_d4}`2K*_S_@#UZ=fDWbcnhIg^}V4TnnM5yKR4D;%#!{hy9a&qy}SrgeGb>`@fViw z@d{G|ZwjhifxLj#rgHqXH4F*0B4A=ROj%shZ8r$v7u=dgE20Ze$M8Y{lID48==on0PMC(-SDP(Ho6fm9{@SBv+Zw^7Iw+5=$FKz43jVtu`%0;}fnSxG) z^EHBE02rbH`;XRN4YBcAv3x^xLj&P90yqh(E}Xxi$Y7brDd8{?lI!OFJ>KLPWAl{s z107-`mgIkT&h>B*N*_~N5c~h@X+tH>-Rr5?sqQWS-7&gWU9?ygdiF` z0$(;Vf(esn==~juCH11fwSZJ%QtYI-hT)?5@KFND$WHwUkvDnIr$&Z~A%!=v_^NbP z>mZcb;;_=msz z=YRV7|Nc2n`t>*d`yc=O$KUd_^lT`Nc?EiNXDwDiVu*z1|W_eUEN>I zh4x!fj6%=YPS)9bz(GJ+lGJ}>;Z}BTJFW<{*8_|k3VBDu6&=>Bt@e5l72iX=CQWPh z%!Tz)EMI%Gt|26y5E8Fo#;`Ou?ZmP|2RZl&lZj>zOeyJsP)7712Imj}CFoI}b9m6f z@lliUT>~20Wa=!ww7-TpC_TWq?;g%IO=prRW|QeFdcZ%V$2zNdD?!f1TZPQ{z6~ea zk6Q*{*1g-n@x~48-bNyJdGzR;A`PraDtX);5!QTI&*Vhbfr?j{2Cz6_p-_n3 zaZ>n#6=o=O4>7Ppq$YV16n&tn4ImuV9)ctHlac~@PXMEXUaI8LOQ0WHN_GYE zZDsd>Ibx5q0ozf|jxO0|fppq&bVWoL&`75Ck+|U?ARz^4 z8nq9{u6CaW;ZZ=fn1@vrE8F$8t-bjc?)_e4T7P|q6bl6>#LN-2sI0oA?RHe4WzKt# z!tdt;N(1Lb{-Re47gNFjP&E}`4g)pc5no1_BR(b=6LLqeJrmQ%WSh+f=gFnXx((o5JvmcLcwH~<{z z0l1^0j<&)8C@-%iMGZcow?`;N50p-JkGc*Dhh8#MTb) z`f3uUrtQ7rKF-v~O`!uycKKldx zS<~HLe$k_cZSfw-Hv4R!X-m)*JzkEDq{HBMa`RDFm!cwaUI@kpZ}-TC}ql zj6i}&Juy30LOs4tel0W5TCs=fQlxk=6{$_nv_%gpnfAa=h#u_N&fT1W<=MS55;P^C z?Dkrjc^vj*%%UZB_A1BEW{84o^JD@_+lSqNOQ;SthThyXJ z!*G01DA?lztWS=`9RD#))o}#d<0ve`(yUd1^v^v!(-Pgtj+LtS7sqMJ zhTxoabT4KpqLjRJ5;RIlMmo722#39D5x6p`-x}@UkVmwGN;*CGtwLKMvl%flWe_o; z2lVTa*Nu^byVj@wyvCKVV-d61a(1#uw zoDn^UGw4wvNYZ1X+=s5tI=QvBhQpCwq?zocWb5v=vf{_sy>-r%Q9X$sRO#rkUoLy> zcX2GC0yW+hoptxMq-&J9rv>MAcx#p7g!2lrkNSubGU;Q9;1jC|t0d%a;1ZMrK_fNH zClG=q7tYKR!dA@Sd$=4(c+)15b|h~$NL6pzaia&EzNAI$G9qM$Xk+dm)%1p z8RVu`HnOVqZVTCYKl`kEm0s%1=O|#8Jq>6Y@SG(x#}6hRqk!HlA`gfst3dfMU<0XK z6c>6OUM;RkJj#`&5M0HyRxx~DVq`Z6H6U==gAqrM9r@UU=di$%QLOen;Qb1D&vZ}J z#JKNUnIpi)Y;i?YztqKsn-36yd=L>y?h%0D&yjrYzxIm!cwVW7jsoSf#Rc`CXsJU4 zQvlhTKb>AJ0Z~Fx>KNCF;zZ2BT{6dEbP|NCMD`qY7Kl?MUyzG`%JIZ2L6^F<4;nMi z45W?RH;7{F=80~>0W~H6_Yx&zVw?`a+)|3VB|9xZanfsxW;JE6jkNvcmhJVDmM7ySaPDCZZ)k+oA2+ld@n zStuNmlr20VGUOQDP1Vnml{2fY6q3nN++g@o96j4>9Kbo6*7Tx2K?h&v2c3rlS9dWb z8d5@F{t!k#C~&IBxR=#KWUP@Ell-V5hP|o?d}4&`*V(r09@il|g?=7!AbRR_sIXl5 zY3VsU#w^p3Iq{sRJ6}9|UCvg=*)*9!so~LyFt5oj=0RBlXxw#n*vMb9=1_nwA(L_I zkPSbfJcUpfP#UBm#WukC!|=2LwM6<}j^_%`MWuQ6el*ni-+SQxJ5g*CIB=f%=U(hh z!8n4VT&8GwG`X(BtqCoSsWKk2mo1jQQe7Mo0)OujzX^N$#Sc!>q`o)~r)G+v=$E|| z<1ZY(K`Xs=$VeRBl*8w?u3(Gz`3b!`{dpYt9T6%+CV z2pHjMh(0O;q4A{qUi55W4H~MZyuEr0^5h-8Q#T$d_lIU#9I5F>8;ljN@A30=Xn6Af^chOPZsG* zS>1DV&CG)AunqlRMAOQ9kI1G_F;ZlK+Hm{gIAEMi_7nsr8RQG)O5D{8GF|T-bmT%_ z6@u(3dw$Uijc-Vda6(MKD~?J?&VH!-Rw96oEJFiM(U59w;ec7urEkP-`DAN=Td{Pz zz$xdL?l{(DoLPclcooL(d^b}wW{LIBN` zx#8VKCXP6HFBNN9x-1SST?+Ru0%}Ub5`|0|NhMu(5ybU3m%S0b4+qDVy~ffj^%A|p z`nT)|Z*K(Yv7@8?j_fZNN5ME#U0GVq(?h>-8o+Zlnh!LXMkLPodXYn|*T(YeBT6>7 zlou|6sIS-~)oKrv{VCvs_9#NVJsfvu$U|d#%Dc|4iMMzk z1kv9pVU;7wr`^)821Iu^(zP2{fNQI)Aj@#L+j5n_;|ZNQRUQ_8hPI$1me^!qw9mMJ zG(cZ{d?G<74~|-zSj2zpO784wa)T(ad=;UZQ9kqr6wWz8dF;WeGTsA(4vK@N2@j4E+Og31 zSRtRfE+epNq)Dfxj9gaCL)}+OupS1xaC%rvw#O4IEmM>o*n`tyjw3;ftcL8@{&Aj^ zuiJSIxAQsU+xr>$ZSec9SUoNvf&27#@LvP2IT??vLD6eSS&2Y8!$P*seXQT;s^Vb- zF>ybp70SEKxI}->7JK_IdMK?pUr*wfjZiNUCaq^g?FUU}0>m&Fk^(|`he{|%a7o`f zMrFPlI|vej@wtcRad(8Q*uH9jRqw}1+C5?(JDT_sCOlLE$h|d1O!#F3!aQS@YhCg7 zu-ABwkq^h&2LT>}z^#c4Sxc2{NP+#?&K7aMo7m|TYz}bktk7l#KZ)oBEAy2kbG`NO zCt(Lq!WIlLSMzolPz#Qb4uJ;~y zs0}t>`Wo=gzaIxvs~tR$?(%jNm=WwP_V`*#aqei30Arx2P5iC732Q3v#kzEx=S)Eh zOkTEEW3U4A(C8vj_O7vyGbJ7DqBRw!Kd7a6UC)K+B_2|W5 z+hKL4ctei@9}Cx%rqW;yH;>&zr*L{0jw0()&u@yL$vWQsO;~g>_V)q++u!+wl;ECE z%zSieF;E%@Z>GZ$Oha1|+}i5a(gLQor~zaT!~E_ChVfb2jvpDkv^} z)#Dfn?brKKQ{OwGOr-)d^8fq_rd=tXuuWoOjv!>ecf((achVx6#dQ3s-IL&b;5rF~ zyuKZRq7W-2N|j~Ag*I1?BZK=zj~TT?FhKJ@lDA%!K!eCHC2>ex>?2v@F=_R!w3E7# z3bUu}jx=-Oe(d2|*}i`6abL~I4c4I@+viTCYxfkLN!KRykGYio#mDTa_Iu-=s@0p~ z{M-vYd0&PJihDSLIQVur;=oV_gb2Fzc#A*4uF^ z$+XVottY~X#QKTExMTH+Ju48Au|oVN9pXV(%zTI+N#lr_$7Uae=34wZx=Dy4zoc}0 zX6kEZJGeE6>dQ5H2(oDk0lXHON9q^7FjsL>j=7)gibp|Jhjx6L4}MFLoZy_q5tcn* zHQ3{tnRfjsm?d?5J5uR95+nt2lI@k?9d(_S14}Nm7bRsSwh54$N>MN^l3@JLJz^1C zmVbz3_FMJkah~%wWoEP*F4#-y>8ahodykb5&s6cz`ifpZ^=e17sWyEOiK?joF7~dF zPe^YCGq$kn_9yLym6)$S{^;F7X?lFfbGhqsyOA4`uBR6fOUKC+eK}xXAU$@Ers|&d z@6`q^?eV;n3B2FU-0SPOu_P3BOkG}4WZ%jpzKG-EsKJ@l7h5yZL#UT zU@prIHQWXy1NRRQ?yzvO@wim0OtTh5Jp){}zp7s5A~2v1rF%NL6`VY=ZDn0e($YB@ zw;DjUxz;|AO54-gjEkg-SL=B1@%vj+qoz3_&CTb%7wL8C(R+in*Ow8Kighot>gsyG z_4s0KeD4I@jsynf1%`XWB$k@{4y&=}-V1>qQVq5J1!b~9qQ;_p4;!sZ5feeAxArJG znmqv$5)}ocGyq`(;@sOKfcK#2r za3co*sCVz4RrXI%Nra#RfK#(pjO+tO=d-;U{@El_iaDG`aZuF0-_j`w1(1Db-Fw0k zWKV^k53wGI+)9-eu~U!QF}zp~kQj(-Z#SuVrBh`i%>p~EK@C$`w?Gb6uZ^-{U+`Rr=CWZb_%;XB&;0BF~fE&tSD{XlG7e zT^6YaX;VGu!6sXssr&7xPR0bwH!7sZd>%|vVTi0WwobmfY8qCOp}e|mZA=YT6A=X^ zJu+ed8M5}_pF6BDr2tTHSwe4}pd;>qhiwm-6-G}WIq_faKqE$!ISNXx41l^KX5EK~ zO_-WBRBA_dSq4*dk8w*E8jE=PA!~yi_Ht50?s4= zmA7xU*O|KISVcJUxt{FT-P>SPLVUAtrf%d&Dd1G8q>)tnr9&9VWwVnbRvwLKU(_R7yk3x;|yq0v@fw7$!3UrFmgIGj<;zZc` zaw1Uj^Z=9LrIGmB4c+`A4Os{)vE6z-0t3ax%=qw2bSBw%x(eUgu~_V0O%u)U8t^gT zC)d5O&5NwPjgp7JcB1u|1Fus3lJo*PA zH^Qa?Uk7|??&tg;!5-G%^N76MwmBm^R9qNH>yrp%GA(i({q1@Lfug{tqM^3P)(&Sz zW!*pbh=)dgb+1VKpR8xk`*D(p4Iis|_j3gj29g$4)AtUD*r2DDNEWw`(eaC3V}+$o z6_c_LGa|rW8bbhz8iwuO&%mEe9R_)tF@L3Ak)?TRA8)Tmv^qW5PA^@Osy#3?;%bW! zC3=wh+dWw1cMlM1=5P!rqHBN6O_GxU3p-ya~HWtR!}o;v*7aezeu;+t^a=^^dj zAvLsv2XSOcWCF@WofyL)V1zu@=KwqbE_LUDDH%pU)HJ0-*>IkGcyEypZ(sG80W^D1 zB$|h){@#lU`MyEeH&Z|e^nc*&*-cS_bvv_9wt@;lKnTJggM)TwXjy}ULaqXN_Cn{j zKrkcL?kcHT>cw>`dWai^oZo8n$UIYJoT@#X&a{QAA7_fy5^=2>y&gizQo{<=9?Y-{ z3mHIzcmihG53zglOzf(;7Pe^iSfK)(u1mna?E%eLHS|#`(uL4Dc6629)XYfeFXSpq z6B(u-U4htbRrzy|B5@Ch^PURtPkQUsa;Q>8V6&WYDdYqDeF6x-A3JX<84lt*vd-M* zw;sPgQ4@Z*?u7Of`NeUHwIr55wfF@pD+`vjk(^)j*x|*=Y<|&}1A~hPkB}|2k^rDS zIWfzKv_DvH-sW!vE0cQ4GQHp4I=m(n7%K}x1{{RXM}ay9o^wfMuW^wi(e_NshExn% z54*mmQ&QaP7WcZ|Dm(nNsqblH-%KZ6?6KH&V@C%vwRDh!qrq^thY!LN8Q4fAsP-Uq zrH9nwd!m@JPt^^$sM*6DNG6ZNTtg4S{=ljCz^RkZq{C!B5yklg;xtji6C*$ILddJ|jMy{fe@dLSR_T_KBm6olVOmkK!uoJ;pI_ zd+QL;r{?qpD)7lpl)vm$pCdMwz#is6&`Nm20v&&x#*E!m^yM_aI6?G>om%pp@dlCe#A&X=k9Q# z!~*Xg=sVaU@&k;69t5Hwcd=($uME!s{BkolwGeR=jtI3zg8He;Y zi!#kzpNOig!g!H9c<7y7Y&K9#pX=h%FL$sXxSKL&{k_-Uoy&onv3spUt~;gRJOkTn z>Zr(y+Iw+cW_(7d``lggm+9U ztunC^uy;hmyCY~ZQIh_m$K9Vn#3N-&)%J_ytivANb5!jvLgbnd@&VyW2H~y8=Y=_< z+RwevE0Ni-ln5a>Y^Jm+ayVr*i32hTpcsPmiXFqA7X@&pM}|?vvNfP3lcGhN&bA`b zh@(jUfLW+$6iz?}@Q_HfNmZzk4WiDQv^-5g*l>ql;en1FejF`HKqH2blbk*~4wOY7 zNJZE^g9u1KqXFLMNAcq1z~xspffThS%vS+0hrC=MKw0{aIpK}6qpAb~p+o375o!4RyI%LF&9Xn9u zz(2^Kbjt;#7&~}OuSJ&y89^+?!R#!)k@Q@dYKI-~{A=oGuCC3gDY! zUfOT5&k53?+13)!-w*cM^d$p9VMQaXcV-ngyzLyftLfO>Btww{qz}GrDn=v`uj@b! zskP7OKn{6#n8KnOq^l$nWWRhXPJ@)S#i9F3gBKCGOI?!vD+q$e+q3txHBQ84iXKuk zRnMB7h%D3T0DOx>knN^N(3)f0CU)1w1c&V zYA;XT=wUy3i;}1c1f`Oq+y!l`>22nAzgoPb8h)d4nzcmi$g-hxi%9O$4YdWmc>i93 zbR7UbR`I*5k#JS`XJ2btq^M4iFjK5&4rg$tcZd3 zlT_-3@)!CQd%)&1@1k%_p?pt$d8vhg3nukBsyHCVv<3v93^#(Ac}`^M_Hi2M-DRnt zUNWRr03CKBvrZ>c9Zuhma4;0F$LRrAi5`Wu5NC9F9z)1bjuZJ)cmI9`*kK6~oD3Jt zbuI^OhUU-kIDt9=U&}?DuXNrb zij2G;HH8ig3E$*^Cr9D&B_|?+&|A!i1eNSp-GJr>$Dlh*jWlTqUIn8GZgcKb&80noruUjk)&dsJ`t1aDlrww zFOQ;F0vUi6Ta|w~db{!9Z{h47?Dqh#X2X??3>9w385FwkjdfTj8H->r@04LtA!1+X z;6L0W1|RiQ30>e1bf=Z35$p+|%lU#c$t4N{t>ivRG?E}g=EHc6Od$;6f$De;1a&q% z-J@{|=g)zbK&Yi@&x{nBiLT|A@}4PH!~J-xnB-HBY^Wp5bEW`C;s#&sAy2kRF^w~c zxS#GSpl7G zM=5|vzeOk3Uw#tabG$E)8vB9;;z#r5(xW&v&;u+KxDZs{G>exzLlXYTdDCmnIDzJN z>|zDtsfEigVXs;3AP9SRXrOsLK801TfT)`u+Bh{5Ef-P8hRd|^-j=5rg5c%2uayi_ zmlWuokGOTOWuY|G5&=^E_Uo_5kH6t7NoETC3z;cQ)!mstEUE0o$SD2k^djR=uZjx* zy`+H`>fzAy6(;B&XCV9NaR3eP9%#te1GEP{oGGnLgBVmNpk4a*b5csg87u17RD9`0 zS|W}E9+2Zm31ki<Ey56#+A zGuoQ~O8_xgoMZRHP*WLT1xUpJ04PhnI$z~Lx@w^?>7FJer7%Z&BH?Vh19V$r?jP@xx@zNFq)_c+Ah^mxD>(j!!pM3E1B zuny8gV?=uRx}-zhc%5l);Os@(fnIDs9mcK4na*_*13xBca{Q7VIgky<)3!wqgqaDy zSN`BxC&HEodmbW;ad2u0okuGNB=no|#%!`Y4s}iUdX=gmo<>hp;b~b4tB%<)4p-@1S1WrSI6M2FqO7QGsv+6@=E4@%mNbb=G;}3U)WDVJSQrGj= zM4}Ww%IEZec7x#@^%Lr!L=f(#mkl4@Jr+kFRr-h<&z>7!Ov>TF8CwX?19`5Q894Ua zT?M#7s>&V=7xvpp@tN_$1ET;XW-uXIIAd$fyvEiTj<7ux%#*6q`~e^ZJTEUU=bh(= zWhFRw40SP)=u}bOVcQ2Vf|{95^SZ+U+eO+67F8_4oAJ3fWd2rFo^pB_=s1L zB>bagPEpf{7!54zv}8x+J4Hy`B1yD&7U^Pkff*tj0MgBiC|43`2P%Mz$~@|igtgY3 zPMl|{Kp^5-f%o5|s--g{e%wEGv=<$J2W=oLo*Y_HlsS!k5MaStgKG)&?H=!Q1}IX= zH&W-y1;c_OqI`5tWi5LQ%-ofwOUH+5CTahOld%xV+z>)a3Ami?qz{7(pcaAd&Seqs zlSuz9C7cK-G1#*D3zxjV9*wnVl)V6YsLRPZM}u=5j}aIiW?N?W;B!#vGdlG1n=)xK z1QK@78b;J@{(G)`; zFYJvVRG5dJYPvJ{(e}#{``L%1z@r!wj|B`RwUDB2!!e?SshnO~le-t*X;@$(twgWj zCt%h`Ai{H3%d#N@0y!@q-ZdFH=HjW95Iy#gF`Rp;7nbG9wDJruqKEe zq=j~mlFMZUa5J5Zm<9el1u7Lg#=a208~7g?ZRNFuN#e5Go;3 z@Ip+2B9dvN!FUh-eu6Lb`;i3d6K4(UIso8QTF9W7_kJ=os|gdq-{%k3UK;>EK)}E4 zXqOaF2_slOjs1BeH24MrZKU`cuWL5!} z1h0eupb94oLT^3nkJ{Pnx}sA$W-VsEr>ci44^=(HKwrRKAr8AoIr>~t+FO=90$|@U zIw04ez}O1hddYzkmn30UiIIG_K|Pqna3d5zn$0DnVjFqr$G%AW1)$0SG4dJKG>-#0 znaE2|7j!~$;(5Y(rw|`8&Z}h7eRtIr6ElEdAO%#ilJKDw&TYbbBoAvL4?hTe8Ji>v z{`ET@Wre5s2f#GRbug9MTab}K?=)(!jB6Hz`WPaVBkxryDGV80O(B+0iM@x9)e@XV z4$clTc`$qQVr6xQ1cWln;QSOW0OK)ObgOERR5b|#iH^h5n5T82Wn2j*FTIHUIoFsKp=FJ+ZiCBObdva2$vTEjn zFhM34geo-G81FIZj~18U{tSTExa^ueYOPE>J?EQKvdrXld&l%0g2M|^u*u6(r%T;v z10_lXWcu7V5z&!}i^E@iB;E=i$T;!{DGUy|0ITlEu~Y#26<|A*p=Ua#`Gu5Z7+EZM zK@*aANBZuRc8Mzlmg8BB^BBEcwJfl(Elh{RkKU(pKy`)WO@(p7nbJ56py|fUG|vZ^ zr5%OzX&oU@0DU^az<>XShI(Za!JF1r=n+H?aX;V$MFM3fCdYltra@H6QJHISgz6o| zJ$98+OO*V2jvy3>0pjj@v_64OVG=;DP;npZ<$Ph;h%ymSbWres^iBEL`i9@kahIlm zC$O;|5~T8YWK-Rrkj0|54UPtc1A+T!hIQPRXckE~pxFy<6s+gP z>dQ3RdsHY(0p2#_&-pW8(^ZfBHUitQv9dhkSouR{jKDzy3f}WjvVyQpzKQs*GV#?w zjj3Iy1XZoW62Lllh-U&U9Z1Qw(zcNuA2}Ea8N&I<>2`=AHNLJ$dJcVhdW6hH7n^Gj z13CTs=4&g(7o8 zl)QyBp=JoXmzynm6d?P^ghBx*au8Vn5UwR_H_J()@%pQj(NDnAk=}frNE`8lbT-qY zLN3+IA!ADi3ha-3)+aiICR7JTdwY~-4+j|z(u)6ygZ}P3ye|Nd(~+^!CBT`kS(^vf z2GWrNSn#YN%y4&P+Cl9>F_)1@*+kzSRZ6hJJ?jbe+mHrVH+LD*1EQ+hnJIDyw=kZK z#H<~M`wmkWqhaO<(9;4VY!N_Obh*ViN z6BG0DzBAc8!H8tS-Z@LakKp`-hYU`ZM1TNLL@CRGN0#jc)|VIG-9T@Lg|}}V`c2H! z_*g#d{1Ik|=s6NbfM%M77>5)2cLnhxkH!X;}o)qixRI07pj`qkldLKq)pu*FDHJPDWSHVOe1{@Z1v9_74;qV#%j zBZ+B`%ASm%k5?6oH-R$$dh>C__*v&qY->;T!33TXpjX5_tJjMj+la6V)JPD6ucA|F`JA}%sVMflGuD_m6c)nkRfe}blXfjii2z+2;uAj!c^5%v93 z7uqw!bLHt_0&?;0f`IO2q%{GgXC$TAmO|&fqo#~e`kQ$Yr7jIoN?)Q&Ln4eLE@~r0 z2%y_b@=nWcBQihUJ09|NNJ;U+EO^4b0;q`zc*%i89nt7-`tcYA#puk&nTJNaL5Gid z=Bv>D+7SmTx<=5SCL)_L4|LrJ87p|X?&_C=v>~|NccA(s$D*MC593%G9w2q2 zs17n3yr~GY*RIS~h*sczB>e!<58H)C-l4(;177qS(J_mW$P>Wcnd&;M$Iy_^h-TsV z?9639ayy0jBAkVwI!ZC`KHfHe)8U3Qg)-WAaUl8K^Sim zqES4oU-Sgh%Wwm!^z#~}VZt@y7u_?{ z-uCx;#T4$~p5P!28?>u*cTebgtRwdOxAi4ZTM0 z>)tkR)nJSm6>)^tFKezVIrG!el?6at=6<^{tFVtd=p4sg#PF+#;bc#XRp&H9t0p6o zg+;l9h-i#HS$~aS;MD;-S7FY>p*#;}AokbR6_qX!0nD$)mC70n=3!cWAC+K940A{E ztK7R#5dMDCqQdZExh&R+N{#mhkxFrjIfY36{p;6JSB5PXQx4sEdL3qpxaEbsMEZ-O zdt}KR&HZB_08=|v5Bp?s6Rxb1yw6H{3i#;?sQ61mGC#m`KyRm|%v+)scqskHOZ{ZC z0;8}XYVly%-ni|FU>@wwEffWov^qeNi0Q)}P^wS$Ix6d)=$XroXH@lcAmP{fl}%yN zVGG*LV9a0?G_JeCm{uzK);z0pc+B1(v>~#~3#7iA9s{=VRK6L9YKOme^;^?C#52=e zjkbOhm&%TFOI;$HQ?qe}+|<$6R}Jy@CZW8sKkUBt!ww3LMDKP>6i~s{s}^ykiJ&ga zF&_w5W;lye03UQkG5X64;uW1=hYJ$}&}Pp`&DFsIqMR;QigDhO;Cm%PSqj9UL5LAi zU&;KUyI8-%pK!#aNRf~uGBP8mrWgt*fGxZ9nnpOIt4VBey{lra=4D%mth^%Fch0xi zgsAX4z+}=DHe}kuCK?gNzgfF499gcCF@u=Wk&h(c3a^4C9sh78La2`WE`j<> ze}EjUy7;x1!))n$iB+Wmg4l1M^XZW6!|bJ5@Yz1DILehR7|ecmzJ3Rk4<2uQ2@RgH zJ-A+dvxm{OQ}_!4h~yGm<3NkG*`||5MM(=+waFPi@y=LogsFwU>Dfq9T4Zk|@zxZm zjG$t(Fg3&8sR+EFz&d4$FkZT%0diBYU~n~O#1(7DqS+dE zFj2O`JE?Gs;(BB&0h`^Whrys3J&ZG2=aM0Jz<$R@0_Zt*%>MfCXas5}HYhd0F3$(?6_4b&9n}4LLFO+Z;dZ8EzMyC)v;t^kN+7FD z300$~MIySMkyKXGLwx+IgG2r><)jZL(Wd2}3<- zD*$cPSiT_2H%V$ryfgU(#r~&y;KJe9zY+aWx#pTN?U!Ddxt$P>Gf*On=$2BP=czZV zj43)@koY5!OK0^LfdH~6GOu`GC909KoFppI`gR+q1twdT>ML3t%v=!Zo%X&ElWjVM zooCu4P-Ty=o3f&Hc=6EA0M&yAlhE*ZbKT3?phW(_scjK-rnRirYF@vRmh6%afQ9X& zz8VwAfk%@4L>#{bH1D){QOoj@&JxeZA-kCMzA`|W*4QC<)` z6G;09<^4+{JZrg+1F8```sZb@L#dwhB(2f9zUYx=fuieYT ztVFTG5w+8Zzx`KXz@0e&9z1*mu-?Tl60s3U%O5k7*%6}X^n#EXNBZUaqeH=df53nh zop`0>pA0L2)slk%en%3g!%E)8iaEm_6?*mkylJZd5L8Xv%`Ql(p4Dq=XGy}S#5dAxAvhtGqlJ}5g zQ|%sjGF+yVHYORk-TD?hRlW&N9X|^un@g?-vJ0;{ei(mqru?nz7%C{%4`w|fL2zDu$*?GO7Iea0^(D&na z=t;Jso;s)L!;m%CBt(u|o!~pDx(U&dQh*DmcE;V7*#Qu`dIWjqFXP@ec7BJ}>6c))-2Pukc23lH=MzVP|)4?gg+`p!r3=B51^ zomGGs`*s{hiy@+{K9{BotDeswIeV_lm)c22&~ESpH+_Va=>c3%l|le*tAv{xW;Kh8 zHF9rsWMENH1t_N0Srkl*y8HXk!9u2p=At!|m%ss*p?x`FFrLJb4+k;99Q(i0!En+*QEB># zfl+@7+0r=E0FrBjLTroN?2PDfoN}-(dN51fHDVvIG9ou#{VKI!azRVS3y1SFfR4Zw4s^U?u%xaMt#n6Y-yN!uk5G~X-lxet zJ3Pf&X3uq|IW<$>Ys>|%ApT*RvQ63rr?TfhqTng@nb|vO&ku*O1xo(iP>T#Y zgmef{1{{^ofc*$J045ilh4(byf@q1b_apux5vu8LGGi~v+chfsA!<^KJ^ z!!D31ds~_em01(szd~(Y<&;q{RJ^WiZteSoR3(2ZQgD_duP#!rg%cm5!2!BX3nx%} zf!1)f&ipxb#l$b@ikC^yT41wS5Z6kzHo>gji3rAc!Zi5x3vBZt6SL}TO!V(|6wvg2 zoGUZ*9RVsl5vN2Zfd0)0d$?14K~4nyffK!>yh&e|p6$dBOmnZZz+)GUA`Vln$cch! zc?|+){``Q)-4&Bg^{ajI8FiMdJa8{67<4VKX4X!3(7jzo)WnK`HWgfVg;~k{R;!3! z5k-fY#kl+U@WS((YI}b##mE4+I&B-aH0BQxKRvCVQ6~^V9d#!8-ft5(SBzLDJI@MA z1PuxvSGHH)r)A>`=EJLbqNM~k3N?#6$dN3iZD`i=q-6sbg2tPDP}^SYR9Arvj9R47 z`?Z#INZ0|@brf?PTv`sJo1Zs zl=HhLhkymBw^s@8&d7RvMx=chp8n^bb4Ghtu!Cr~kGtC9ii8WO z{g7VzJ1}f<6`~tV?xmZ{rCSQdY*$eAa#zFEjU$YS{C>#})iPUYI+Iz9oJk80x_GYZN83CWA7~fwqEd6m%#~zdsPTGB?c3i6%LP4FDe(5+iZ^}978c@E@X9S<KdVSDXD2|W% zQ?NAPpS~yvU>L=QA*J$!60Zw5qA=hf2NT>wC`q76=>vl?y6O(?S52Dh4wByX5s^CT z$-wjqq4hJ0YG|nC!Hr%a5zCElAlvL^MrEVZEPBK+<6lG3ip*MV4o}oAF5YCYc~>D% z8TYLgBQ)2FtHkb!aGyU^;LDjJ=7W0BiY$@(r$n^3+l%>x`k=Mue1xLw z0>)JlAmoX_y(|%QYcCB9aw4!jPjNebjmw--o%poEUw&1I;FTjXIA}5))1u4dhFF5@0mta5*Zmj4ng3WvZiP@nfxl#wgF)yLbIAIDIk4B zr7D*d*=)rDTV!bDsP$i+d_=fzMh0yaWqc+Isw-do&9}t+HH1ZL?X`la|MeViGMloGn^7p?sc~jf6iCJQv=xprc z17Lu(m%;6R2?c&eVNuJh=84Ct>NP7PF(<1e%5c`FlHABxzM=`|t7f>Dy)Sd&Cqy-O zr90gJ+-cR2g4uAM1az89w{MU zE)`MeOt}L+{0I2bRXsyIJN*i0ZMVBFx?avcV6JQ~fU;}lQ4hz__7JqHuibtx`*66# z90_c9IfIFJeWJq-kCp}NxL&gU6(L_fl$wqI`0d|_(ZZVPoDpA_gn6E!w-6EG`sVl` zhi}TWk1y*&kx_U49F>HJdF~VE8p$oks%hG(&2T=@1*LVTCf!Dxs;Lx(^9z1B5KV&$ zBX;hfNa{M(1;>mq8T;f}*nc=yi^0Sn-)tc$6wKpNha)xl(~J566M31u9xK8nPc32^sf_Lq7vUmg_)K8I=&i5-BTUImR2)CK!cn;b+&1*E z2h>#3w<{ZgDDOsd`+W8Ld#ti{6k08bQq&%T$Q>mB6|*5iKeOi3BU|jz-Uu_ML#-nt zoL{sj(9y3mAqGX%sPDt7MbsEiQLd-zOfzJQM^v19Ir-aDdQtp)2K-$lKrY$i{H3=v z<9U(&RL>XMLjRohz29FRWK23+QE{2 zZta*)SrY1MnK>|Xq-URhb^(E^F7LJPcMfJ){RurcTM9ZVi4MBNf`Y;XR=A%U34yxr zoaMgrtY7|%M?V9HeB#U{ur{^~;XFBa0YgV>$qAyl2GxP_D<1GvVY|j>&hi=I_y2Gz z{2qlr1Tz+tNftiETsHb5kNqLl7l^nLp#!F-YVS(%?L!jCF5RI;FHPpLQ)0A8hcy-m zEqY)L{Y2>R4vPRzS=qcZFC*S|jVd|QTn=wkgT|d8A)~C%@!^{do^aeBH~;|THhzd` zGgGPaFdzPEJp5&pej+P=A~K23^5xi-mnH-IBA=`wr#eGZQ6WtOpPJ%*M7+;dgfV!# zeu`qKy_6K7ZB8K#ssJTgj7{!QoPTNEjO*o+JjJ8h7v;5JYX#4v1;f};Req>&C)m2< z$zgnPkFcgO!E65W?L-`la|Qo{4CZrLft5ADFXucm zb%;O|jEFqVw7K~m?WhE4f0OgoH529*bD+!RTp#p^twj(3Q4c}QH1D9?wbq+J%gjGRU-SBbsQu4kBD3|GVWpXv#WIdd7LBT8NP=%`OGTM zPbMk*2ypz&4?;0>y#S2A3_^U0A=h4`cV5eD)5?Jzh;L(7bXPYA`W87W%VI?J_NL|H z`90!&KcQ9LVjs`(!*P%(lVrzx(6e7CYzD?~{1jpR)Z%$_pY2y$^$D?)aadNAXpk*> zCre^ysnVf0JykMS;}bd;h_j*EKR(%eZEN#~h+MoDToQm4ur!~?dDTAQM~^q(8b|Ui z_Tp!~xD>RB!?i?S<%5*(NfDK45j|rE~fwJkFS}3OFm^KEJY*t|T=-kAB7XDCHeY zEl-pmo4Verg&|5y>H0%N3rrQx_6yDbj2b#|@(t*iJSHuqVK50y3stl4{ty(U_YaAX zESh|xW*VHSFFDRLYdRn5B6m8C^v_E;C%0-`HUS8j*o7LvEdjcB~@!pPcS7e=>UMTpYda7Mp`9q%Jz22uE+xwYQ1=Oaq}&>8(u)eoH`4juH+ zKlpUx=@`;H|Ad#16g^07w{-Z5Qq+dm_!6b2zUM>B(T8-V-fm#fmj@9sUUZiGYe(;V@2|VCi=jYTtq(NmVMd$p2D=vZ*6$I2eusEEY=Wy& z-t_x%uKDtIM^`rV$e@X=?I73q9`NJ#_$`Rtq-!-iK}yk9IAf5@!*Lz5Q6(G2cn%KV z+&P-8vSJt0KIVu0sfHLeTF@;(iX4m|YASot128+qji|8&Lq!@;lfwt72J*BP z)tJ0Tt)$@}f=)_jW|W)#fspzzrI=1#_wxsZ`h#2a5%G`JLUQH){^8Wt^2VcyMXSVr zse2Ir#atOtY}$LbIyV$l5RM0P1mu(2Ynrr@ToJYA8$~OpOI5Dp*e<&u2v@|ah_)(X zC4$~mB1q%VseazJ8wq2WP>47e0s065S{8@K=>B`0sx3#({5y0~wBYY2-|?hv!2-S4 zjIZMhQJ zlHHYyJ^f~%^vlt~xu!Tq@8%uR80DEx zXaPz=*?8WMJ7`WHrO31E+4I{uu7W`BB3c~>9Ck@l7g!WgrR{HM;IL!f^EC)MqLKrD zTt`vRD}Lbt4pCxZdG?Tm%qvJv(~W^q3bJm`?ViM|tT^>~uEL$}uYJ7wanGIkAuoJJ z)!-ptBRhzY3OO!+%0owlf-FFGRkxw=U$3Fo6OuO8%4cjCU^k$Tb;{!AVFrG& zCWvt_=>^;#4i@5}P^(R8R{`>f0K3gaZ!a&4C{DP>7KiP1KUa`oMM_IZumh%SG*aDyaeU5gJ#06FXZA z&`G{(;I>LywXZuPUuIw4#RyyzI95NY^&{R03f3*|*%OC()rjCL>J&jlXbn7CG`>pZ zt;0uqN46iwf$D0Tkby+qYR!pI1)$7;yROa1Rb!jg`kL=QrB~_-=omYq#4nbs_XLMombxnLZ)GUMDhqn8wv6E z-yqkX53YRhWl+y*<>LYY{8&HyIgbKV-Z_#p*;&Ks_3MK>t*E`IqNw!i0a~Hds6Dpp zCUnOIFRa+yTm9N+=XrV|rb~I_D-w3c?=ExmpVSy9J=Gd|t#`tNz+WPS^8ga{R;qHE zqqQ$Cd$Jb)4KTw#9I#~nO8cZmS*eUdK{k=m6Z%AiOq^#%LmJUGDeeH~j`{U<-C;n; zt9iG7Z3xaMI`OwxVVX05RyjgW4OJfIA~>JBBuBr0khkVL^ZVn^F>Ndf`~}cKg(y0l zoqPcOr;iBl-hI8dP-sl=VJE!xz1E;DgF@@ulReGin~r6e~c-+MR`?*(r6sq@2UU;j|zv! zaJZRHB^rhd?#t2Va!vxIEU@qWEGo?LJpz5uR|NU{hq9ZLn$}!mNer_tqRu^(Nj{UY zObuhmjn2{>m@3zP1Y{HGF}9+vLGj@uN|m2W+Y37LeMj7?Dt~@JGxEs_wO8I(8{u2j zhsHL;eX}r>x5D|jUi3Ky*kzV?O(k{KQtah>#Mc^vOBk9^PGLTdv-drZg28z}?tlhA%H6%_iGT3{&#IRIp_Aur8XBdwF)C`yXI|y*0iStAC=eQiy?m-CZybi&&Ky`)5SThJfuLk0R~-!*SqpK8ji~nGL)@f}y)ihPOKPpO}h( zho%CKVil_t46wzZ$u+!ar)FFGtvTjfR$rOu@yn5}GD<}90>qr-fN2Bec^>gYj)pPb zpFMy)jjafKl6+C5-=bz& z2(W`di+j=H^k-z8w5u!|c*&2TJ1YX@SpSU3nF5zpui;e^{&t+B_=wC`mHX{MNRbsZ zTPdv=s0ixHGkV_p3;Fj|!DvCP5!JPR*im#iiGn5YxAlFVu*OVkhr_-!bS6wCT0V^D zAQCx|fgau3{EWc4V@ZZraQ%0!qrZr{(Ye^iL7bJY{!~tvg@A3ErB-Fr)9qO(GH#|| z%-i*=M3VW(T00>`s7G0pRcYgFG4#j*6i@XO&x)XjF_LI;%Y$i6L=##fy>iD5i50`cX^|hBmeOZ8l++i4^IDD%9F88S8X@EHUzTl50cKclm zd+J-n3u+3MgVKga?|T%oz=s^_MImyllqh5jJbtP??xvt28>=MyeSU!`Y!sRIUl?v7ID5}~r!)V~S1cW27b zn)>;gj{(32M*N4!yT2yRPb&UDNyq;e9_+yRe*T_l@Bi^1_WyDH_OJiD-~P}4 zfl*-01~&>1fp&Nd&|*f+N+o&MMahZZ9uY;1swpI+Q~MATxTn@zpgM~1T|?9t(5oe= z|FChw?Bc41KkzK|yLSxeUI!d1k z@gv2`KPiTNDuxynOP#>>A3Y9liMU)Nq1NBkURu>&0_jY|7#@T6NKP%()2Yepg%bm7G<5e^S^u4@_5uFhPQXu;Th0b+{_IFfqMHC9eL}96rs1tbL+Erp5N5{c4?0^6K zS@~e6wwKy4~QokcZ#wD>t>4^`v(Pihv$*HClYlgVx3oi|DB^Y`|k&} zu8MzOe}jP*Gx;g#Fr1TMoFBIB59z{R2-x-Aan zL%P?rmgh)Rbgv)Z=KMLsqNx>qbgx}%H0kjx^q{kJV!xv3<>J&NH_op!bv%%0PtVf* zDssbM?mz&UA_4up13@z-Bj8yE&;u1}c8^a5?wn={bwmATM3U{uxByFc7-&}eWG3^l zj}pFz#t}Dc-Rq}_X(7cEybH?rmVA#Q)w{s%(A%@Uat7&6UqCRU)huoQ0oK>-76#Nq z3MMDzTY%?TGgc8AS@7O}KZ>dLUg9RbYSw7r7tB!_VuDGnzeoJ^5Pw9`)M}AGVwV7& zjO-o6o0tm70q1wDz_8Ir67_TRH)1HI_$ObI9*{k2z3hsPh%P!#Pv|uJiAE0)fQC|= zd!4%}H~!Jf5Yh-A(sBj8Vdm}!>;V&S4E-3|T`J|yMtQQo-eD~wjBx5{sm4jYx<^Sj z4dr5ta=dUl1t0#o@SdF(FRd7fAg7kubx$K8WfKvhucw@4cEny3?>ZCB5~5HtZ`sbE zPqLk{A^X5EpXFV9^!i|1M-J>1gAilITPHCfn0MnS`aL}VCA+A}YsFytv)@7u$`{!How>UA;3yG{`HARvs1z{sA17`JSmP4Xr^vb|`fhLl zOtb~CHGM6i6^Dkiq>KeaDSqstdwIJX(Rn9@kj~dJY?nqvZlI(5ehms9X?4b}XX{3WRJV@k`_bRE7avh;r2nN;kom-vt0kw4Wt zCHSAM={#AuqZ+4jDMLJjo&Hlr;qfa);F^(^V`R*52V9Pa;i=z;L7LKtlDT6XV*p?) z{@s*sGB>NFh7S=&<*x_VjVJ|CNORX3S9g@W{@4H*Q323$cFSG zq(t^mt+q4QqZ{qNv(fSIvmi=*m{vFR*47ebs&z?N_0Lnk?2nF_0Op+dmG!tXVmry_ zT3+}o*a4Z{y=v7b!kSV@PcgSU>T)(rzaCf!GsXt2hP~1kaRS448a-_8?hC1Txdxyw z&VV6wRE{}6003A88c?AtuQun56>n!4%6yK#y#`^R#;ATiBXDSS90Dh(&h!y=7J_}K z_y7Qe<=HTYjO@7Hvyu= zSE>e7Gq9%D;IwF0Ia7uL9yN%gZHOz`1qrf?UQx@Jo8)6ZQR zr6Qn2gNLi^aDQsr_(OyQb6f$S1HC!hXVen~VgSIa?f`U8eanFrd$AnH9eBu{wnOxL z+&{oM-$z7NtsdHL2cJ2gAC7Y!>Tx_;0IGF!M;?Db)t)~DIE&=`w(7X>g6|L_-D3W3V4uo(q6L zEKAr)30(MiP_8cWLYI=eY%aYwGij+O0(Xt<2J$VG1ZR6kZ3ESz8NZWL$06RBOb;_F zLyDaK_Y-Gk=)X;+ehtna&A?Au_?`)DtX5#?$+wz6DcCzzhDvpFq`Cot8BmH7NP5m` z1%}tRuc+oV3~Bw%r`k}p_Z~7_Kr25Pyq-!pLBl$NRYtX$&*$?-xX_(wyG$eEp$s|e z6d$tR-0^jIy6614!!5o8?XGbKc4&Yg{4Rsm2t6+10iJJQsly={{d@>^W{(F)Iy4aK zRvxXinunf@7FkMipG+ZX?H!~j&cJ{~L^l4P&xPJf1WqzgUO3C6gIGLvN_ObC4sd@a zYXTObPhq1`aobO@V4u4R1x5gKT#fkyG|fs{BcFZm%*U-)>7g73rtqVSj41-@j_6-x zmZKa=Vb0BbaScfbLZ@19dF?8-Kz$P7zI`wlzvEF)RYJtwlBwAN(?s}nUnMHXx3Wep z^i+KKP?DK%@L=so#JCh6Mq&OmLIwH*z0b-aQEw)qwfXX`sXY*(>Hvo#sVGz)CEJi? zx<)qG+zygo?bhj5Vz+)nAh?HfY)WJsyt|@4`+84s6&nYA&5=y4r3cC5G(5qnc~FmS z-$#bvc!#}dU(msAWTw05m7+1)U(y%zqn$(%h0v{08Td_o=zq(E=@?F!PWRegYPn?D zX9B3f3D7bfz8+0`kkRwkY05@cP7kjD?j3&Mf*aG&>{v7BM?5FLLsD3T{G$qbtgmM( z#C`bhSKcQxdO91vxujdRb zjd&3o0hY(&%LEykkS*>5dZ>utZxNnn+%f4XG4}r*?IE#fX=ObP5n3YL?S;Y$UW- z`5uwsCzXCvK~}Sa&*Pxo;@#h`#aywi+1H42yK7AP8Sxze{ytuR8KnS{mad?lp4cJl z9F0lL5fwm@pj7GXAftSUA)91?@s|N9uAgZ7H$3VEv3xrUi2Nai9et`^C>NECyW^IR z0Oqw?r%&lUb)N50qN#kL_R6Pf+|6Js2K4t41Nt6rs1uUr6s>sd&{}x>CPY-AC-1WhChfLUX4;WHHuY8iY8Aj9WFsVh|jRT5W)NH z7%TyC+(sstqI?H9wPeFN1)%M<55frxH8SS!iJX2mEw1iFN%H7ONKcQ08UqDU)AF8( zh$dbK>U~6*kW!kq`bkNgdPJ9UrY6Y(54@f4jI1cu#550Seau5odOUmE;p2$KqDr=H z)x)L%G;h1cP~QS_lM0*t@yA2I2W>L1mIOVP&HJ5^nfV!U#3&UQ)%~tG--EsZ18oV~ z2xDEkvot3anV4VEvz_z_E*jOn>B>vp=_sCF5OhP%&xrTcOnVSH z-*f(O9G-CWwi}rSWdhuoAO4xT>Xe>^-NoDI{2q0bB@zt}cf&40`^A|^ays5=0 zi|8qR(|j2C3Cm%8+ZDA#Bu@cj3_*inRh-!f3v`VWTOA|gT1F1f*O_F(+Ib`efo+_I zimmbeIqsiq7!Uk2KDRSCQ2aUK{0`6ABfuI~B=&9mUEmOh7WEWOcE!9WyI3cI)uNvF z2uJXTQB5;xS>XCQ;_AASM~rf)sQDR@Y@#ekgrXEjqEBxu)3Ix%KpxLq`zGT_R&+P{2>2>F()3Mu8n6EDB?h(SW5_%k( z3wA^gcqvTrJpzmvjfuVU0jAEGKBHPZCnu%}R|+s@10J5ct7}h zY_Ew+4wK;D{gtCDYKL7Z&R~S!|9aXg^R@k&r^?+yOv`r2gU&on=7e$MrNgokE?TH^jsGLytFjr`+?4Ic6@2wv}4edSRfxAG}MX?ft zLiD?)cEwmRLgnl;(?+k|vEE?{n5&RND-8kC;zjQqS7Wn$~}PM$JLS53%O9 z>^xV>A+V;4nZi{l#SZ^##Qo{3U&MzI%@pKRA6~k^K)KGJ+ayCSiB4~Sm+~g}r6NW} zJSFE3uY-tDw)QgS5HsnKn2(!G=XUS@>g(~iH2 z-jZ8Xa*}tQ_UTf@*#{(#;*D=|910w;4J8kQj#XlFpYRyJYu$&Bb*B@|w}HwApnhL0 zTKaow8VR`c71K1(2$u=iGk4Hyi#Eu3L13a~BCQ7~29d#9tjRl*}*Aakj zB>SIBb$U+|TZ8@59%1pRHqoCX?wNHo8B2^U5ArY)vsBPH54N}tT%gzYBXOTz!4>uO zy|#iPkbR0@5fDD`IJw5`iXCa$R(GaH8*s0G07`sy6-#CsEQ*qnCInL?U8cvi9R`R& z?fB1#JG2YX1Wyq#KcjG$fk(0zDsU=1K>>8~Oe!07Wy*o+ob$TXlCDCMu>Qics~tkB zQ7)}#cxCPfwa_jQF2|gRS&m*ido!6ZxBFf!=m!Ifab;vfppS~fs+$x)@H{h24Sw^K z5RjjQv9?f3Bz=sDUfQlInPTv{su{h}6}=KMGj*MEb`&;^lPe}rIgcpR{WH#@uoH!R z1W|F0^xKpXY%WOa{jQNL!&Ux*=a9dk^RWLt;wk%>uwQf7qX@-78ty<>s0I$%Z0b9;69l8w-ufv3$$%cy_?NH$8HHOk9se#CCBE#W_Yn(>d$R3O5t=^j zI}N^aH?x8^+VJbwRliRG;$2!db_5z+;@#mx-2)1OxIfQ|01ekwW8iZ&cF&YkeynwO z)8z8&G-fe7i1Aatj62v)cD)fF0PF=SlEoci;X5aaji+vGac71=LBxEyn?%{Lqc7zx zM?=Z3z!`WiS1f+LGTbukv(hJz$qO-R2(?R4_uKCgxi#O@wvXei8OULW@jLx0jMDa3 zLG5LI2KWe!JOb_O_5Bt^eQ3hEi-tMt+phYy`~wridDO&E#ZH9!xwiCRpZ*!~=Yqh+ z^B1o14?&`##A8D>h*W_%BmDnReAC&MQ3w?9(w6uzB?Yczdwv!@1`yN|<;B48tS-1} z9!z(CicrtCm!54|5`9Kt$ukQxg=AKXFQ6!|CxCuynUj+Fc0F03F?N!VB6{8A8FtVP zdQizVN*R0%xg;e=hBOm$v5OLmNOtyV(&iX~4QJ5vG0_bvbyVz=-LQ7CH+WZx0z%N>j#5A(NxuayM83V1ud_Qz3BZIPwJ$~PlK zPM!6@t;Ed6-DOGuqVd)1T8~`o*XNrk*u+Fht0xL^U!o%YL5*R_Bcl)VEET2m{ohwMw zgV8O=7=@1J+T=|7B}^L`x0<8qz>>)Voinw?Z9Qmu`As_!Z#F5v2b_0Z6{tC-DfJPg zRxTjH14T+M%vB}8VdJzko9}2gi|x1y2CyH_dz}3#VT%(3dMEgS0b#)Z@wKv(kA36&}Jp0V`u(>vvShrSc;!?X< zloF7}0Om zVqp=a>-s`qJ3?iG?T99}ILsJvkerXth@4<0FYHe)-VevwuNk=DUE^bg=l2srrRBtv zwodUnm$}#nTHY6=--U03R572=sK2f6lVRe{C0+jB!SB8RM=XyS>&N$Ux#A$gqpbGD z*8cny(E~ifi)pkMB>X*U9#L9E91C`eu9$P| z1>sVQP_-Hhlc2j^kci1+qo|hHN%d@ z3Q6*jO;2DHqg&i#k&p)KfrNC&!jb|Y^wLbqnQ<{}7#zFP?Q&g<&0jp!q z)}zoBhjg@8aOC@l0NuhM9E8Fv(eM#8q5-FLx1NtomjkdxzK?GvU+}=}L+Eu03XWmF zF4r0H9lc(5&C_N9n>D7MU_LeOpghO8W1KyzCJ}~QSff6_{V~e##RE#g|0JR}c;~;T zoA>h9AT5MTqeBo?IBswkqPhR6qQn3Z6B!L$p(b4BMA0@z z^fTM57yi&B9pLq&RKoS9(*i^r>*O+(XOnBgGK4gRR?6lwnNPVlZfdUxWqS7s-keI9 zEr}N+9YFV#CVCe)kVkOdloeTGF0f7wE08t=ZD}TN*cXX(ohz){{U)uqMrSlGS$d}# zE&J(AVRj(QbOe9k%xZthLkI|RhScB(^md*gMi5||_Tw{!EL=Ef{c=K5nV&VCdEK!v zrPI-B0bM+JLlu`FnN>@9fwC^b*+j$$zU>Z1{*c{M z5md|^LrmWtc1X;*?ynO)5gMFW0eNAvj34bRD6<`+gAO|nPR|cQP!rrAH{4v|Fvkdt zE`&+uZZ9Sq89ngv`*MRC6k%vrpvlzVcbeHGfHa;{Ugn({_!kY^<^w68!&@db&AJBT z)m@fqyt+4(+Npc_>FkvY1XX9AKt{W>*9)>8XX2)DCi0bBsOj#4QB(7&yTIC_ay|pL z1|I>~dW~Noa~rY8+h@98b5`iOBCuN+)HIY&jsyII)%;rFJRpgy7(2jNirNE9+*Zr& zb!kfgd^SxSUTzIKnA5Q1BIOV3;lL-TiEL3hkvEgK{~tX}5BvEL_&$TL(VlKxQ(*!m zPsMrN0CRI3u#86d#dWylr`-(auAIr=y5IfzvMBJdH51uus9a^g59>I-ELV;bT*<8= zy%DzU57fuF@M%UyrNE(?eUxr)EO{M`cQt(v z^~d+VpXD!!<3sBtoJH*#=L3jaO;NJ zi!+lhTMPC0^q@sOS_EPbLsD+*fL@k`m z_Hz?;znJ{ii#7_|$rykJKOfQqL(s$gY|(&V>7J^1B*HWlHArFlBS?q?2J%tBX?Pu| z9Kq0f&y?*CB$!C!nl{=_UUjAb9(TdacYy~7{dLG4&6)0(MrRiVZ^W7pUNN#Z`?-Uu z4iy39+z3}dDI$c3&NC{S+VrA?$?!N#{zr2BeyO67D}`n3Wc;{l=@>mT@_lWcs{42$ zoDaoB!<{3@%N7Yazlth$G&$l2zDjstZ;CgS+jq`Ya`RREe?Wuk;$PW#`AMp8=o3W> zsCa&nXEl1q!;wXS2t4p`rm%CmBSMKeWAc+TL5a!N(C(IERgj(fl#gFj|3M-G7ui|1 z4ESDSQ@#S=dG$6`u-sYpdf{>dEe3Q|gol*uF96=siX(b_hAGsWB{=*AC8}v1{ilk$q>=(xm;ATSqd%^K=>aT7Ll1W%fX^HU zB=C+{N>c)95(d{uZfbzt^#%*Y@XvLow2m6LHc$oTxkQm0bxcH~CKAPID%;oCoA!PT?_@>K8 z+EJI^f+$*g@V^BOx^1BF5}ODk)Ok4&Ewa&2TLuy45Uq@B@ymG5Uninu8{#(ebrjvi z!ez{y=saG(rxw4bv*nkVObiqWDYFF|(b3p<;%>hf)UZq;x`zRC6$owDfU)GTY2VYd zCkq7)$K(>t*(m#$bt>(@I5&;g#Td9pI-6rJchgtUuKQltj;k%nw!(`w-|F7Kulw z9y3M05i`Y$opId3fT15YU%<>%+?o+Mp7Nd?H3(F0=IJ{2WZJPW(m8!lpF&g7-eknX zO`35peA73ME9r0l-WbU-4gzA?@KKw2K-GJ}@0boN1B$oK6$YS3#(GikljM69gX1Ui*NN`Gb8Jkl4MphF5Zd8>i` zNV*)+F)OJfR@ujyc;p=ihD7fX{A|9ugBx! zOK`dwzo^w0mrRb_18ht#)Mb~VUn8H#x z0yI&s8&QL=bJ0ghRLBR{C0j%d6Uk4^QWO$tMW?0%cSScuE)II&ik!D^@=j+vyE_9z?BV{S53$ej+8kofqaPTqcT@PN=pkK~m| zhrT0_WSkbBIIb#-u`A~xEB7fT>AnOikJyEu5%mE9k2KK!>#XJbTiQ*fa0_3A?LXtu z=!W!Sb)d#QvY#V(WZA$Lfp{56a3#FnM4x8(6v(&t{!yy6j_-i&9f6%UrbyJpJF`_O zN<2F4Pb#AQxT+Plca-^cIhf_&p_eNMNh&tVC}JI99Qzhg*WZ>9I~QCK7 zJF78)GR*`cM|z}d*`h+3gnlwqSV4v0{yKj4N*ecfirc1E zGD3kpdpqRPx#~puYt}eBK_dFcsuvmS znzqW02{#5bp}u%CcjghDHgk29p16wP(79s9bxU#Er!5~w*QD{3co+-Y3W3SOiG&Wi zZf?n|d$jM%bq7Ea$pb5$JZ)WJ@pZb8-7zWY&&op$6P+8{qhu8Vkd!)lcQw8xtLW@% zdo%AK&53f)a;&oQ?3>V#<+z5@;1$*m7HXEgnD%o|)CpKFe*soj>7N5wn`sa3=h!7? zK1I>X(~gQrn)xUn}IHSub+0-Pdh_ky>&y~kKZa=7iE0OXTQqzodgJc=TS7fLF7drD-;qKyB`7c zvtvNiehRV!UE`v`(e98Y2KqkNtB*a>VWxU<)!4TtE;v(lveH z%1$Q{%8|f|rk&XT{lxVzEy<%Ms>(;RS|Fy1gW!G>4vCv2bmYhbw#t*^i2uObz#mUS zu3Fa(GV4lmX=*+3vw1Jotj&~dy;KJEHyBtURbrnje78u??TEOUp+T$g=B%9ShA(ZO8SbT1HQw*zor91wpwkWCt+CfpBBAXN@ZQgsY$+e5fp;z`> z|C??a4?YSRZG^cH4WBAvQ^Z8<3F){?+I%c@XqT{}APCH#jG-z_onS9Bg0hhf4$i8q z3#pCrOYjDKgzw+Un3HO?dE6<}1OZV%sj|tSAYIs0`6hy@{j*?DZ@Ss_IFkA*_6P%| zY_f2@AT%>-@(`TO7qN`7GQtaJ7=D^{X6gLNH@}X8SPJ?Nx!v)1=ez<(%jBnsuIb(?GGp-cUJ`MDI$*N*O|z@SZXl0BPv3~V;LI4>vQqjPt z@@3hPzfMG>p2pC0_7BMQ&T+(xqpD}=^eFq}a9fDyenfrj?j!o{G~eY8vziHAKAddU z!*rFW3eHhSc{`lZ7QeOXcna4Joevb1n0(Q&r)o`5n*k98lNC~p;O#~Q{N_GHNx!{N z3U5Y*cfpHDY4;TGpTcr5nIQ2ayTBbxD}a5GNPzv|eVy^VP&;B*dS}aD1;VAkNlH|K z+`B|w@lRL*bf8bttbEU;O&a*VCvXNZ+-AON;Vv^{FQfMe7FmgJv_>E4q^qz2K4aG> z)SDXnwhAW+I@X6fg|~za8n`jCBj;R^3WCa!{=m|8#A)wa^@QDv|xje7BKUP=PHjd!xs``0rUQhBh4Ko|DIx+*jaB5 zrOzqThFim4anh-|eu9J2@8#LME-(9_k<0T6(rp%N4?;TAzT*|Q!Xj85_%{iMw#PCL zZI7`b0i(!zHg$oBzQ?DIxhug?XRjZr(eQB*S)g}FcxLvxb_B0rB2b2Ey5ZzR?*{EK5Oy1OvvY;@ ztmxwze{S~wPD_RtB}3$~v@8fs%N4Gj^N9TH^RmPlnX-&A$va{{f`%+gFNZ=EfO`>O zJh9mpZ~O$4cTz}`VL)Nb5;dZDMgNKpF8wP+1=4-GU#@8>-rdLB(bxRZGll*+bsuy) z?uc6Lpi4-K{z!VG>|Lvv-j{uu{mvclr9v5-(G2S~dgo9SqeX&^dCSt_pt$i!jm+_s z3uNa$V#YAl3gSmmzB8N!eZ_Egy?UHfMLvSJhJ`Xfd@-Tk=`v7`OeCzpq5u#$_&rtX z&prsm3<}@MSfV38Q8(v#y$#D@xF@V z-Gh-={}~49Svmm8zal_FfRJPaK%P1g60Af}e%FacbaqZ2<&}A)a$5}l9cLa@Q=j@l zK0bs=tt5K6ndF<$S-rwsZs9h&G}ehqQTObT7fAzlrUAQ-x-wI}5wf08_j8=-W)TxJ zxFMLZN;LtS;qIZgS3O}@kbw0H|^@L3+KJ zRE#!IAGgss)3nfr?g|51&F7t9e{sx`xI_6X`3p-_5~Fpc*{X!iIFu{=wWqUg;7&7( zOBaRwb_agG_ju_@9Y?!xMXkg;BKgd4k+tO}HB9knk*yw^ZnpVfTWyQ8;ObR|n1P1e zZO?EODj6}VqvyCTS9}nB-V2q9@Bbt9m!ypa-))pP3o2jgBN60Z)-87;!ymE?O7&UHtTV zEA*`=8v-8Dv)L|6{auk4h#DY~I^t{|N)g=>p^j|ANHjNR>35L#K=0O4=b7Z^g9cOi$WcXmEf{v@lp6R=>*ecAqCWSTV#)cWySPiaMKiI_^1V8sNn9aS&Ep3|9vhG%?F>4> ztpBN=x6m|pN<2%yf0|*C-SJbd!j>S9Bc<>tK~0=g#KL!%G8y@__+aGoJijaf>6-`o zg9=@(mq2Qo5~d5KUSoxCq34&d#uJA;7$`&L2&;5lETi$ZpYP+nDabtv@865Sbq3lV z;Fqk0$9wBeeqgXQV!;)E%FC9rfc?CE_=AII24O7*?7}js&g8}l${67-dH#1H)D}f- zm=uw}H)jb=o;L=#o|G4$zFcjZ*KRLQG!ozA+dNg$zRb?FrEEevqG@jl35)Bn=~0W# zj=+=Ki24Q^WhEk{-JPzzLjGh!j7=GwiN+%;qWWRsAig!~d-ASUt|dpBeLBG1VF`zTS544S)@_arM4j7lxe}0d&xkoFQDVjGcFVj zVPXh)%!R;%0P6{cEFd+gEEKDJ6D>upEnWfARdxN9BGS z^Fv;~V+oDm_hb`9S%Po2BNU=y(2S#E&zs6B!$Qt()FX0eb0LI@pnAH;e}k`YR{Yc$d_&r z_CE6BMLDvm+jt{QR|{V&kHGyO6^~J3J=KQXi*^tCGgU5GB4+}~Ngq-f{Zdy+$+lxr zM)_HK2}*jpT+G7?GINEn5(?gGixrijSU@n0z-%Rs5LyBI-cNk*hyF6-X8N8JfD_@0 zoFF~lS=VtyM5CAZpKe7b9+<~fum~m~L4E(bs$r!Xo=Dz1_s2OHn5CpS(*GmfZqnJBjvCmXff+bm@M5=+sJJ7nCbWI2L& z!@Y%HQ#N@F*h|ZGugSKH_BpOYv(LSd7}c1d)+7`MK=QX7r)3rfD^Sa?KSbdDHSvD` z=AX2m{}&$U4}9J8-yeM3>n>E`G~^>NeYY7&g#I=y&Znv9 z%ABzwJ)^Ag2;)|TZ6U!;MnMw} zgt+31q%aZO=7->kk5{LtH5?iF_oQak%@WG==TM5`gE44{~;8d|ZPbjsJ z)`WK+l0ojfLY|#vaTXts{e!R94i*kCL;5FKfI+Ok3=%H7Mvso>`9MFDm6d=9vQ{GI zA?ytuca2k6+^(8?y$!Dn651_d+O(i4e=eF8kFT87o`HP|ZcO>qdx)8-Pk>HQgDw$PH4Gk6bV z;rn51bYi{PgIZ;&B=G@t0q^890D?6D5P269G2=r6D-|Q!PF} zLqZUcIHCnQJq0<#tvd79lMeXR?YI#U1lHsxE+Z()Omx6I6A{$#ggfx-`_szfH5Znp zgKrmC#8iC$j>ic(0hED>I3*^5+;2wQ{p01Y;4^TdcT_icK|B8e_@*?#!JpXwFR@g9)dnd~!b=U(0$FIuOYJU|I{@D!d zpG`Q6{_-y)%?oU>+0HDc}}U%C?0B1OuM?h5a5<}6=P&Y z+=*JHkxerY!S1t8(AmwgkPuQ2v{)hJRyew|Fppfw_A2Fw8D0mg%1|8IZ%!u;v4_>#&jQNtF)oe@+{1S0o>-&2#p5Pal!?(f(OG(<{F(1>m| z&8}O53UrSGAN7Eb1nvX4RWqH9?}(gk@Jnd~et~XT_NItUoD)K{lQ3b|dUZf+9*M;c(dPM+xZ(1tl{AY`ML+0;40 zS>4NpRJD(Yw3yBshqgo5*T-Sdj#8BaUB(%*Z!kh9!$NwM(aYEmHf?AMTx@2TiOI!F z)ACFc;b72kMuse2TQ_~H*p#_W93@Imgj@XK+%jj1A`dS;?Oh^x>N0&Ery!UIbo4vlg>*3=(AwA^Ts=@n3HImVIH*qW5e0+W z8N+?xg6*Z({7gfbD^SKTu34CuDUou*yEfA(3p&K*z=g8$J<*D;A%ytrNz{FH!7qbd z(TEJw_Fd;@=zFSSvTvOcxvC2k%@y%P1^2j+{a2?xLRyO+@(B5RCTbmZrYAyh*kDB2 zRGRozAUN9TK;pw2ZwmS&h*-~atEcs%I?Uw4hEFJF8FJO+a9_G=wnp+EKA%R!9}&ek zJ>7@i)!=&+{5kPIoxVY?{j@)>|)bd;s2-J~D{hFOk2`C~E2;nlvF*hsIlZW^_u9JXf?`W$(0C0^xj>2-h{g z{D04|&d>~WUO<77dtJKTfGt>}fMlrD{qZ!QzmlFA+fTHtWze^WDx%Ird4`;Rqqr>G zZ@^UtVC{gy;+b}p^7F%gZr@oOg!_@2x=^DQ;C?H)t4;t|D;;}wqFQ>CwsCfyk*P8& zxO>~OqoE^eyDYqfvbsv(e%kHMAJ;1{%ZDwvR(aMOraT+OU{OH(4=bcfTKl?|qQNTu zuTO-&a(L;Yg~E6bKko>#H!d|3+b13~(^1l@dD*Td-Cq2IrVYfdt@EveNJjBtyx$tp z#%PEQv2iGoo$+M|M0xM|5FSKOtK7^xDA4c^lA*t5c|?VyiqFnQPUnOD2qLdtHV2A- zPfPcl;{D&RUg+@myoqe~32t5@oUAMkA+ zOFbNU(e6$M6T3Wx?Mm;6D{3%zQen;{jOQ$08Y0xyc`1fy$n{s%UlZh0fAoDLlM3?O zdjj-;2VJeY{yM>fsT+6fkxxhKE4-Thf%@t>ZVLuyu;gE5D{*I(6^(lu?pKllb<5DZ zxGj6`(rX`R)5fV<5vaz4@XnTC%{A?NxggE?eKUszg;&noPPXZY*Nb}tC^cI+GkSAJ zLVQc47T;}}6P@eTXmvnmVYlxRqB2G)LQhS2hu7~^mJyzc)Qq4ai}p!M;oo3wRs79d-^L_rh*awo+18Dd*^%n^KSw> zokd4pc5fm7@PL zs7>W?f@5}`YxW>LTvs~(JwwD%ayU3_-oO4vA}56~HV(pg;kQw3o+ZC_U2)XXF6S(} zoM-(qOVrzo?Dk5BM(}K$6vAk7(9!{Fo*q96Y7~+^gmJkIY{H;0zX$mBKTHKb4uf$e zGnK*2n57TnSlc4cjU!n_Tov)W{%7x6@$Ewq(6Slm(cdvC)NULqu}v-~SUerOrbP}> zopIHtCgYwr;;4wXU8CyF)V^b$V%7nb-iZC;KdDEsc6noh)_La6~YMignl1S)ujf*2$08op`)?=%2 zSBkP9WqsS{&dJo7z*xFJpAnu2t_+k{Fp=t1e;jA@GJ+&eyMyX+nddjZ?D)ToczL@} zs1U))nZ8AxjtKcl`z=t)WtL020&sRWtEGcRkAjqF_mSRolrHpw`xf!Op)qnp)$DnG zIL>;aV0+5h$&R5`8A!tMQ-noSi%rdawqK3aC$voz?_E)%DYkYYaldWzS#9%_HcEDa z^}d;e<9LejPj{mBLqx9K2nY%5T)o^rj`M1FLZm&%tc*+FiM@68$ae&XvR|sHOI3@z zM?=SZ{_=6LX6ofaNvcWWHjp>!^}z`|e;3$O)YOr_4Dz2id&co&#_{9$+Gvir(+w1W zuW<3i1~v5va~zRzh;EDmMW&P?7k%;zOv{I8+?*+PVHa!StF$;GZH!{R_u>?27*Wk7 z=3%cGfaZ9INTc@+xK+`@A?Q$0^|^(18F5y`eSRe>;Lrj9=Bt@v6etfenQ~RJ z#ol=7)!&(K{tB)tXR@!03}gE3)!*=@+!bLEBQgOg6qn$CByd2*CNBxb~ zQX$Zm(Pjui7aPwED%<@-#GHW;qNBCm>gOX$MbH^V&{l2u6M_w__3I})o^uGX{KcqPu}QYZ-h3-H0%z7yFjc^`}~f^)LLuHST}t zP-sCiS#8haOR~(LS1{^NJjW?_J_G*zPVp{V`#=iU4~Kc~mzE2oW2T&?$3!{HD)|}l z;~x1viVY<$DHC*5{B~bnsmzVDu;p=!qi?$~1bR$rV`99YF;Fyrs$oV(3}~3M6*9S{ z4}a3bc}etWh(V$sx)AT5f>*FhKG31}EAUN{SznjJND426k`1JRD4*s0@$)P9-VFbj}b zH`JGWkL${OASQOj3rLOO6VWLxMt+D;)``S`QLV=nXRrcSl(8#Il-=s>6cl_t+0*S76- zK+3}`(>7WG8bA;cG6OpEr=^<$Zai0CrcOU>q%YwHy*xyxOG$$M|&fN-DY7|AvbBQI*KRa)ZB4J z9GVS9Haoz%V}5;IcbStlLD2tITxL~d?qXG`AGGMY21Dd=;3=-?x7hLc{z0zpcarzV zpW{3O`g~umsmn5mKb9#}+}AOFi10qn+{f8oiJ0$EqV8ER7S^LrebLfs)Eu+FIZ&zQ z@xd*TaW8f+rUsV@AuoGfyg~DRkMbvxNrS2yPi|q*N-g6IT&JEHMQWB56{**FWuIn& zvi4pg5o#5Wr5`=QilD04()S4(eA>^(_eaFM<1Iwn`b91$in5ca6){cFD0aFfgP>j% zeO)rqz_aWZOQTyaUcqVWeT2nvyjp7e7rpGg7Gq1q)z|U%LzEZZ>6YEhy-qg|gGf;I zG43ePi=t;Rq_ViWqr+Cpe!zbn|E~T`Cq0C0F?;?znn(K)un`ZMR}el+#a(9x2?<>; zmElpM#tjL!@%3UjUPN~IbVr@4G4Rm1uAF3dD}$DTe;Ex z{cs$Ztjta2oVGCN1QjA5Id%OUIydBM&(o{D?^&GyVW^c4>GDd(FQl578#S~%Zi~Fc zPHyP4gOI`+_-r9^HJi2?IPoKTyF?FZdfz&v%!YNt2Cw zS`E#w49$6F9fK}44bAU(TTCJb8clbBK9U>2>bpN9@=TSQBc35`jUSE!Nfaz4#(ys6 zXD?w;RQ~RO-BkNHi;_A(Rji_uV$RcI+<4h_LiTh8j0;?DCqyd*@H#sVGJfe7+hywX`dCal8u5s7b=U>I~gHTS*ZMFkQjRQ?0J1g(NqPE(j7 z6Y=%1?#&_KFW&3ce>l!6EhJyoUKrn7ge_Lsp8@UVvr<>Tx8ft{h^{j3fTCko=}f_! zUaR<2?22Ftb+6ujGcUHuHu;=|5k;^p;FT3c2Uq2nE7x|E;j=l~( z04WlIW)Ec)*>s^d9^-+!xe$F{nTwG=^}!EAFNi2wn?x~R|J(XLPgZ~P*ukjpPA3PP zfxVuacbdG8*}!0W+2&^iE*wkR`$KEscRi!Oh`JH7z*=Hw-PelmPij&u2UuMdDag=V zEOJ}1&1u)Kswfe4}C@HaYUdW!3*7CSXCA1SN2xM#TUTC+^<7KS}|gSv%7!O{r|J}b}RNI zSAO;ty!TLtV;{8RJ0cJ5#+ZRIHt&h;qee(F3|OOKED!kZ9nZ5Oc6HY-sS$`l8vQHR z&di-VGGfJAzmFlr2}_!)G92id<|a@J=skvL)=0)T`9+bVB7t0V5fk)()Gc3{gjnHmYHs!O0J?+r zupePVao&57HQwcU1OgFFX4eQ*V+BDX|EJ92Ub6!CgFq7if?r_5P3F z?f)@8{_fv={P+L%=P#B&{@s80;~)Rm$KU0(6I z!v(C>?LVnCU8;?LuB^tB2cY6X?FLKvu)myf_D}|{kbGl&QW#hK9la(+zMATGABbvb zKk`-X3D(lIXC}gcmt&ni6^(=z9Xv@Nh2?Q4H`KpE^HG@XhtO#OLpf%rLN0vNazUJE z1*y#7vdXm+tNfw*UMVO4Up8GHMT{sK3n-(U=^gwV4C^TIz4yJH3P8T+ab6TiJg!5p=#4&*I&`q3DZHb#}m$S~kA{2wH zLF6Tx{{xfLp#)OXi4S`R6WY1zF9iJa2;_%M^!m#V+Y0Vn!f!|k8J7z);HcBto2kS zyIBB#r|SVLdyF?3=SVW(i3K^tA~e?D(_Sk}U+@T^G;vqz(%rxJNXaux$t??=W3)?9 zZpzez^aDaSHvE1ST*&%0d}L@!^?Ccd?=XPE)ZdOJVZ~O}DYf!c)AvIn1@bR{zG=^7oga9hrF?*4478$;SNu2}1Y>AZp87D|9!o;t8TvUuUc><`gk|SF z;aaE}1#SF#pKh0U{#&awQt1H^~3*^mAf?2(X%MT%H zwcp0W7@`ocQ*De+Tuw8FP)JXsvlLo#_4-aiHlBOV0S*E2YX6L?oa6G2x)M+l4L`@7 z0&}2C$lFH1+uPwihPHPp}_B`Y!~z=(kZB2iV|w9N)NADErl~IM{xc z)d0xKTnI9}0fPtH={}srmna$wx?puRNA?%%N_6k_Y&OrcKLAR3!EQVe8Vc=UM%}&2 zDECSaXaTQ8w}dOL)BMRAz#ZEbC^7yz-w+{3-l?aZiQTL0ZY?e#mNGfR_$gM*$%S4)Tw zxpC+3E=nx&`*$3O$1@Q4JIz}D-Vc@o?f}&Y58~I(LENt?`;e@9fYuLodHbE=?=Zu> zH5M1%uq)DwupouRopIV6>q@1DfP^nj2Zp@o*dZ=?>hN^is*`o59~)22bQT)UJY}JSVl9>za0+q$g4@QTQfhub;8#OT ze9rfSxLsTIf;a0Px&Z{pp9e4@z7Od2x2_-?&mlaIT8N_K1CPE51YxyOFhkOEYG4d; z&Z3Tan7OLi-iDn${$9_ALS%d3skA;quQ`0Liq&T7;HTO=Fw<*sFr3g(vf8QApuQT4 zMB!J)incskRa{F}v<|Nz&f>jwIs{e8H=1F`r;ax~_b_jor%i~j*q*9i2V|=vAYNnQ z@N}<(Pso%uyNy1eg*PWfoG9I=rsLMs<@=ihzKY`ql~LXfvKQ2N7YPDQ)L?S<&%dK9 zso5+f-h=Y}JP4nL^89`o;NH=YQA|Vva?=!duMPF%Aq&kR)l@OkX#iP{F|`tHBWlLy zhuDK*zalUa6=ofKj@`V#KJzazdV7_sXhR#jLcs-OsLlwOPtwgn4Gg=qm>anIKC z5d7lZFF+Hz5i?C{6Ofi#M1Wgu#2x-BQRtyR5M#z6_-S$0)TynWXVFR75f~yF5e9N< z&&9ovHoqzH#RY(hHSIwGuKQA`z}oax`^fqh?kN`wTsQ)WHPmu99na_E*H`PC32S1; zLb`W*Rbm^|YpTcruN-ekP#6sK@jqD%x@;hM0A!s z-jHnbnn~goZcD8GbXG7=(zuX)R~KQP<4RcZFb6QYi9mUU2;P4q$iax{D7%|uAoIK< zQ06~Z+D~H1!`xq>jcBF=j0O7{$apre-50B-|{BC)O=8H~*Y?vM|qK_`+q`Z0z zLP(_qzDG2ya#xcc*GCg6zDLs6ICXcyNCBq#8e^!Ldsm~!>tzax-p6!`#l^+x9GQX< z%mi&aV~hDsgmty~`XPRK5cQQLo5_KdAOzbjoezy_ z#YI!h%}Yb!GQ1ze#a7yHY?4e~kr*yY7QRPB*@0dWP30YzQ+bc#7ftb-Q#I>h=S=R6 zZE7m>n~1M4&pWz)8KvY&_@qvy$WTzKwdkB6M4ciuWnHREtRqWD(ZoiJj@es)-w3TA z^!a-rR^QLE)B6M}rUP4A>$J30EUlCKcnhFKqwg^shO>Kkj}kB8`-C?Z*|rx({fxnw zkCF6BVI)l@fTRfNBP>R9e-F5$ZLrPXqYbX#H>X*O#{(~_EFP|Uk1}Tkg{dd3I+>F@ zt|>jB8_|d7%{ZEdMllHyl}5-fL@U3BtFjSE$>|I$J9H0p(ZyqsUF6a*Wf0NQkv*WZ z4~htsqrvug zz8;T9wb%x;+vO!JZ|A_{5Vn$LGsnlcxrXc!pWC+)Ga4kVAO~@NlD8=2FU39Wf%S~I z+sFD&g{u()_SP8co_MG;W#j>i&jGWM_kjC`y3>67bXNs%V$3kD;nGd{=FjYWiY$yM zszlZ8Yg9TGCnyzX3rpjSIrj9^MMXgl^vIT~iO6aP91wixdqg~^jIKcgWQK$9H|HT4 zNvd-Vco-)m>n9@<9@S&72!V+Z}xfk`Jnqxa~HJ`M)9Af+W`NMIUjpK?*m}!yC6M&0YRf-E`Ve9iv1dtHw412l5o;t4d9+ksD zu8W(CO&FdbB%YSdFFMK6fMns(+b`J`nWj}FLGr1`@)ZLP4wLCZ6 z{yYm^Ex&q{3HMaC6Y|I+9|ed(y5gSvIP@%*&_1O#p@Rf2Q8-9|=4D=u>LF+>;8B1V zxX1nPX`AG0#BXmu2dXHwY+EI7=c$!X*X8(&2}ubv{mCdqxVI3%V$SxJj&dA8{i_cH};Q= z7uKkK8hW459R@HqHWLjANs?RoCl1_)dk$Vhh5`V63^ed1s$ol(0o_*{8s#boTZ@aNjD5QXYTZa$D}$_ZBLUmyNC#h*h2`xz*ZdotUubXbN+x5$O9s{_!W)qzK{dr^vU`{A;E zP`?WXFn!B9MzyVoYMDINzDq0o9oWPK9qYRl4N=QgCvABF5 zSb;>A+H{ymuY;4q69$I~;lmFJykdJsRMT--qS2D6prk-M74*w@db22!MFEvY9a^*q z51)>{r^R42v3p2>F$XLY$_4>Za++Xzz z{bWBruJJw1F9KOT2@E<~KS}`wp6p3dFw?A>S4aKR`x5FLm{^E_4sv8zrcR1jp$ z7OgM$s+0NsxLbtd>$w+N?}f(H*09(f8zr(3oH0dD^60O)hY(RM~RPf_r7us>lO zQ8Bg3@rxqO!F8Jb$Bx$q>_;dzYz66?(f?LCg!}y(;r&wlE~EsY3Yf~?&VyaIrAjo( zOMWL|zDJCM{;rJogRXy%!jetA-5$rux;S;S>?2Z~ysQv6&g_F(r3R%?rS88(#BU1` z6Q_$uC+u&|;}GFJV_L1@2riU?b)4kyJpe9k4ddiXWPOjK1xK;GiY?gpTOvQE#beEe z%^am{VPbR`7YWS@+Mly#rfxkyf0IT3VFm9%$@QX9XG1bQ1xbV=xCl4x4saX8rY7v8!ZaMc`1euf(g)$!Cm2mPE#WSxrP)=Atb}1EMJJu(S~$Hp3B$)TV^AqlVOA^E^8v<@L=H17YKsyOsBd z_-(#sJa6ZLSt4x_@T*17afG!H(qaer7U3vp$3FY@rhkoEEgUlUxJ$jC8=i5)&yk6c zP>ozK1$Ej;pnzzN$yE9p@Y@4nvP*%>C{zFYX+XNL2!nLdKLIg6`KSAj8qy*8L+b!* zN?nP*v+4tc5{Ma!F)J|dpvncg%r1sS&Xt8^S??9A#P3e_)fZ1p$<2Qz}CiyP(Q3l{C!?2R#(~^Pssc}Q+=vOB`< zhgV`#3V~IQIuT2LNu#-WUKds_()sa{}LlON8^oi$R*to7*r9F`+e3#F%{3#c2d$u9uZ2x=Y|J+`MGab zOu8hVuISISf`yqT69t}}Gdw;MtxQhQF3L2YIe(_EJrCkXT5acQdw!1^mbD&!qH6@k zHzl6w-b_sLxwi;#%03oGe!J_&*C+)eL3Ab**+`wAs1#vl^sFmfx@ZlW_^TV$N z#)9xnu!c{e1veWkUL?R{xWbeMtbz7UDzUlfGXw+n%m% zJ{0^-fINz=#jmj^!+CUT;U2;^}_AqZ~TJ$Z=zO1 zm0o8N&~r)X*mr++`;7{a%!n@su?=aVbR)J#$?=^7%^Z7QZakjqP8mt|9Z@+DOm7GO zAuhES=$JC?@?}~<`&FPi;ZlIAf|98tfXbf-YCuf1GCJA)>tIhImZhuN`dBNV;cu#d zEE{YF9=6X{7wknlH=tEvSWZ zPbiiwl~(qP$UfC?55p*3LKA#Ccbzi*W`$FV@h1F(j`yoITJWM2>y)u;7g^N5BQ zt3e6gx}siG5NK8XDtzk#y(Ch;(+)E%!=IB2bLgWpjQ2=zU8Q^4wb>rP1O9NLgmnza zsYjtF+@x$jJW<~U5T;CqQ{xO?TSf`8$BS1*$(6kor1({id;oS5H-U{i5MVss>yI>1i6x*BWVBw5%>(Wz*&O9wL2QkljKx&8QT!NeqmYM<2FWyHRI2FD@q%-sl|HWoPlKphjyzxrn$XOq%o8-J zs2a5YV)$n|(o`GR3x=XXd=q8M`=APO&QI;+k;(+TK(|XXojGo|dQXJy7<_ak#ZdZa z!h(<%+4+-Pmb08yC;oN}G3^N=7DZ2RJh600Kt>Q_T)bTqY&hmhAvsTl1ba>ajCOLL z$3Cr6WG3(03eg2R`hN2K%aZ3mfiLLS9H@dosi8VC!bU?ZP{u0!Si<$dOM->f7;L~b z#|Y`=5h1El7dIN@UDIbGq9|$aK-Yi61l?&UE-_6f^k3ifx4a z#?Zb1zYX9jVi8%b#D#x)T0|GQvel@1Dm9unX<{`Lk}lxWS{0|IiiC|!Nf!crP(loq zVI4*GU+2xE0NYEvKwn}W6SC5^K-@d26u&@&^wa6=hZ;=E*Y8)A8Com-IJCa{@Rjkt z5AE-+*1=Lk3veNYw6>o)@Ju5pU!osB5BxCWy9V4H=|E49fRR0lRbLm-Ua(}GBXow50|qNAV0^u<^u+LD7dCwkF?~}>2}G); z3QIidBG>pzv=EcFi%;YWRDr0aVAmJ4EE+x+WlKjeAJLwP*AxJ@aM2x*FX7(4r~t#_fVc}K9^3Gl%1&2FYiZr5v=wH$l)df6(tNUnmoPJ{ zM(HF3Fn_hcCPUnniiJR&5tw)p5t>x?a6mrxs(X{>rde?R=(iqWS48wOYPGs3e!nqa3Rx`lb@xFRmm(nF29fnqmQ`n3GRW@!pLc`0UnoYbb^tvRK!5^A@yfhsr zDJO=HqUbuY=VBF+1ZDD3(uoP|0cU-HW|$OI&>Sh~=U6QPtHEl?TAOM??B=KWkXs#5 zvScJ_Mv|kNGn{XLU8q4`&w$8p!rU@yBsBd_;^X>gYE+_hp@B$d`dG2!_{&MyP3yju z#mF!{_aysQ`3whm5ce*?gA8oI2YTMnY+onwTPSPlG`Y95;Cb21k()e+3qt+`{!Fe6 z8liAy^qEN)#E#PO5$4MWCC5SGc$B)3;`Shg8UQP-wM0BgY&t-#5)r1GnfuKVp5wG43Xp=P9@;1ntftf#wA>k|^=o^9GOk&tn)Hy8^s?AWP| zXyvj)QM$$fhNv-I001>4#ZpH30m3tDS(PUjeS#b)XPMkq}- zqkJU{va>!Oc%q>Do`afGHRHu)i@AV^J%JKIEtG&@E@+i@lYE#9)x2{-y)8DDLeVUo zpA`~Y-Y^)v*!XBxUPuAYg=pT9yRIXyrqWp1OX|TovB~YAYq|?Wj3Ka0s!~#~;o)MZ z0Jg5!(yA>KFEG2M`2Xwg$&Y`-cXNPpud&z0kwM*1mYF}q!GTh;qN4g1DqcG+9BE3$ zr@(3!Ivaa;L?Wj^7M;ziq$y?%?UM7X=x|UQ?4rYa@ly(%fEXo9z@!@`6)Dv6_)9pW zL4dlzGD=SFYS)226<`3t*o5P1|C>Gyc(9U=`o5m7JDaLUr)34YCWIo%v=NGYENWH_ zQ%FpneUJ1nYb1mz62dS-_nYBKjqC*nDSts71xm@kAFE6Gdc#iIWE_xv&uNvWTlV8$ zXDf=v<$Oju%$yd`qqnffO4n4O1%n;Os0#BMFs6GS6sTI*CptCa)Ha{n z9m|j&C*6l(+CZO!&Uy}rQ-DR7fH6Q1tCFit1;Jqko(2O3U@6dfV@t;@kWs$AUkSmi zE-IzE@Pj;}!6S|Vm>_dG3qRiw8kZ3YkYWu$pPvd9qijz;d)jER!(2u{eY_ps^aH-| zHm{053iqeC3XS#ZdF<3rbv@_VgYxk94O71{g8M>dZ{&yoQ}w7(-Ch^cdv~_5c5=JO zbisB}>?#qI0*GkmE792X22%D0^!DMavZ`j>3fetCE{vF$BfV5Z2hWB!K9T%_j1wW! zSV`eZq~8D+rm5tuQA5zDW`&w|Mkn<4xTts}`==vx_DJ!nS`<4sU4e+O(~TA?cYxFY zs2bq^JaRbbr?qzzBu<-Cm`DnHgU82yn=z{_1HPr|sk0-u(D8(P-}9W_NJ-av83&@% zMjSsJ|I-`>Y80dPYtFl;J#*1%`eTtu*rh=M#0p6-6chv$0uEC;sa!#k-eBlI^=Q$Q zlL1)^-$8(h?li6PqQl3VS_7*jflPQa5RZLD;arhK*mpk4rGVVq&nVF--y^ zIGk+Q0{NO zCv?-(gbX!G5qsW1Jxc?Xak$r02S~CaP6-?Q7^GV%4Eu7LNx`$VIa5&525T3sPz)% zFzrgl5psu>r7{-SH=I=|(=Z3ws~LqR>Lr@gP?1B4e2)kcM9>7S3EqMoL#ss; z=BFZ}`X-bu_M7|Q7D>%fhzFQ;I$e8xLL*+ii3-6C?9u+sPPvTMo>RT5Ic%Bf%2XGd z88WLWFfeS%sULug0K3i*IN<*E9DLy-XBp+~81|9^wxyts92C@ZU>ws#SM7bEt+PYd zl+ubtIuk4XT?8xhkK7vyO165|~ZD(wlOOH8Zg5zuTe-I;O}QZil6La_O) z%0Y^nU<StRv16uWb)Mlnw>9hVZw9wEf$Op6>uS&)HI3f8A}*o-kC#bey=(R-HOO zEZC(mdYnuKEB5e6&j=gdYraG_3u-_|YxGV+BfVc({txm{D9wlNKSroRt5||Uc}RR~ zgYn8m1t~ri78Qu#QK(o^1Mq;0A$0q3&(===KT4*oI-QTeafpkk)3IM!`ePnAZMcx_ z?`vJ>t%d19U! zLcfXn6tfx?n7W@AjRU6(c)09!DhQ?eN!FP$XBDB-1ym087RIrBp(KhrF#UoJ$~2o} zD=Q@})eCM6 z=w^wsI2x3@HWNi|pzxCJ+Vh}v_x7=xiOaf#w^8?X9wQ`?B4m=u-}fG5jJUv>^L1Xe zZ`2j3+VugBpc*%0gJH{62^j76MpNVoq^Y8(OF7kuu!Pv0tzR1$Y)&8WK~xDSDyiCu z8r#FGil>wfI<}CjzC$G0Auab%ZY%nmw6ebgOBR^EomOF;8d}adI8!kd?5R}%e~zpi zMsQ_IKggm|2>0i46|NNlZc5QEF`^}ilX8K2h>6OTfG^pk4^Wz};UAl6J&qY-1$&BI zfYk9{?N;UYf1U$a1`F80&g?v`Sw}$Bj+wn+&@#S5UE%l0%Cw*84C$a}APWH3i(rtV?`fCeP!CiYU!hch)5U-DNuAb-j3|^s9Fz9 z+^D*=>bnxm@rp8QRv$W1%^#SG@T9?ucY$xbvdTK6Bg$oa@u#i)34T3fem9r-CI_NGR~}7P1RtsN?G>jD$)SgR4s13e!XPN#paY zaovkbU5^;y-h~2vW5An)(v+R)JQq}VhUJ*B%81i+v247PY+=#icmm)}Pu3*_Q@Kk^ zd;-4RAD(V?PV|*V(#!@ssfUcvaoB>1<}I_0-D^I|13Ff!m-5p`$fU5n%WenQhO-uH z-TSzB4|-l`BrsP=5JOZbL5?djwNHvoE-KmaVeQ&;lYIM|Lv>dnu$?m`5A14cBr<~f zrfTM%JNBwLqGn^J&Z9ZhXg7Yao2-o_HJ(0K^A$zS(Cf{@y;w{DO{$~-{+}nIlsIyK z@Dw}T;TkWtW`#6>UP~%bz;K8v&cyZXGE6l?Hn1QRiaKx_5h3^w4UbggO4Zd8lzxvZ z9bkP>(C(E2$E;cOh$u@aiJ??Kf^>i&9Q#)%8NB?8=jly6Mhno;j+mM*V^^b6;eHnN zaya+g#E7=->>D(CXv|25=SmBsgr$>Lr#pfsqnlk2{j^s^Ezi*IgwMDm?lyAoveM6R zN~SGzKj#RKStWs??1^v*?RtvcBUvX!*#&&}-a}S}Y0E(;Ln!7CA;640Bf^6RIElrl z!fXk0s_W&%A*`^1>K+K%z#@d#%*~34cT@SUmkc! zb^&9DLYcsVE1FfM$7$R}(h5|4D~E!OL>`tjU?_U&U;y@Vlt6MsfFpw|_QR&yd%KsP z1MCX8H6)4>8)GE(b>HeoM-(FC(}p;M;3wuSeGq$0WwfSn$7Cjq*P{YC zXd%+nRcAX+e{{M?Y%4I=EBvtz-nSR@K*NGr5wl9WbcJeDXfCo$^D-lYNzo7Ng2HW$f>engka&=sU=vAyX^w`I8=D%{ex z0yg^N8E}D-ekhrbpfKu}P$Zo?vIXMpkuA`L2sO&^hU1pfi0KW3!9EVVlR_KpPDb{H z?n>{5swjv^2|f}RQo$19r|toPFmxa`?8ky-9rvc;Yzt4MP=i>QZC<2p4fQ7ryz(d4 zfscqFe*AG=&>O}XHLNwl5s5j8aRq_d*Kg-Q7B7H33Ll}CQLn$!a};U!V%FQitlH|E zY-+n@)6fn^#$txMQHe&TE*dE9!c}3TYBB&fsGJ-u^ni;8HNMLBv`@QL4#tFm5-vwh zGOW(Jq+h4%|M(}{pJm(6p=>jL6vP%aNIsil`xI?T5QW4XrPwQi;TUvo$vC%Ol1 zhPDq8x_z5LyhPI2my2E9zbyb2v_h&|DB6h6LAPpHsVZMkc0~j&WcP!MG22n;ZarWr zBwe_ zj%~pZrP+dywCNXJVM}22sdyjLw4cp4b79u(w%4w0Zx408|bg zphh*I;iWZ0_dq-ynWksYQF3P4-}6U4&X&s0l55RVFQ8yu znk_-fB*iRKIb5pPYTNS0q%MG>Nx7*tRtes4k|vw;%BU)E9_nCO*{+=}y>xe5SOy_4 zr1oSxuEy{;L2OZKiA{3DLMD_(En?Qal&mN8^xX07qYdQZ_v#%Ks(JC~y@}<4q(L4# zQcyAWQ=R}nS(Iuid(Waa;eGnA)*W~EbXB*v~&X!dhzUT|3WM61h64#1w8P&8F zmU9A@ZEOS*^naY4uhgGWmAW#~3RJU3IYrdGpn;vw2v-G55giZ!R0Q;+i&N$QiEB6j zoHDMmD!q2|diqKV;;KeRAq&DG%%PBXLai_gCgIZ!wP^1i9$1X&ps*o4g2GNG?P85w zD>RpF8nQDdtDiM`+KM4AwT-ca(q}`A24W}Sle15Dpx^@)ha-xs#eL7Jq1x?RcX?qe zg=)^;O8r^n(cco@^l$&HU*}(WqF*u0jQ{(_AXB&swE-wIm3L8`I!iI$>G_xyL78FF z*-?uM_FI%sNz^&jB6Vw&*pCA)>^#0G=EU{~z~0@zX#kQW7vjF?hGOD~!g9dIS=V2d z>0aF{_->mERm+DV@!71B?paXGbUjGxoe!iJ^?YJEuRJCqfz_xIGP$Ir)AQ6OmS)1u z5}z?bO0m18_7w7HXDs;jHc>OvNPAt#?gAi>qLz4$Wl*~|DXKXjGt{liQFs)aj%+pe zM#P9+>TqL2fF=RBH{9aDI<8BIbrTOnR(iR{mIAkC{0gne|$NBC`kJOj})>(Giq4 z*^?P}_Rxk7P52qpfX_UKjB;IUM-q+`iqA7!jn@T&pPs3{#|7e>Oo;I3$E~0YAm9hZ zClDNyIzfwDiDj~d1J*nY_o*22+3R9`O3S#waGSAomcSlX`TylP>kGYqel+xll~v8L zD6r~|v-|!GC_q_5;Wg1KP1sd2G1Z57GuKSW!W!F)}t=ja}J} zn5dsgp%n#0$0L1y1B-f?nwE1LJ-dlsplrpN*&p8nGP?E3W5{d(V;r&z)@iu_nJyQ& zJv*k(ken|EoJ93e&C^q7tQNdO`FY9g`FbD-YAFb62U7J-R5|jw3wiLNw>XUe))Sc# z9Stz(QursL2~WQc)qF>`zJ&xaw*&ukVap+WHf^G*5^{6m&{TlJ@kbH}R?^-nl z=trru%A&KY$uTA&$rt5iR0IZI&L#zXfl^#oh(U zwT`G@?U`aYzKSB858Z=ZGznPvfex#9fsr!%)Y)Z%o$! z*Yn1jfV}?{t+;o;{QQq(%G zldLJ<3(19e{-`Ly(yfIPZT}JvBxC?mf#?1n5k;WGcEwaWe|sMGcGjcx-HN#+D&BkJ zxN_ik>yp{KpV$)uo9=f=SjyGcBdSZArJoQ95i%k~QTI5`Ma7hcRN%ryT>L-)g6Wxs2kT`!?fPY8t5kuEpg193s;oJs?f6^kBjGLZM;-m!BE7zfTN zJ#ucjMTwTsN&c-K+SCr~!pNiX1Aj4@W>_XnK9)kFK2QJ!E71|~W(-d`PkSOF5_Jki z^tHK?@~luVQ(Q@<@(%g#ZVvN25SqQ(c584WN@e+2x{%CivAT{2T*tLm%nC1yQiDxO zz7e`Mpg4M4s9DgX$d)RQ<;l@5?y#BB@3aQDa8ejE@bLM-vd}Iqq3@^VC$hFpAGsro zqJRAS^rN$EPVH!buF5)GP($4r=L*GIh0N(wyJBc=tz#Y3%%)o3lGcN`v>96ovFUb< z)CaQ<%ayehZKgM1d;a=)U?!*wY{Vd zbShtgEUJMlQa2GGsq(n+<)6U$1ULy1fcA;v$-1o|0#FY2Xo*l_`!YRGXj4u%0Vjfa zR@O@>3cKMMkn++wM?%zVCs-F*PHR?l7sc7-cPe26ng{(bdZtv{X#dxQn${`hf)^U+ zr$Q=K9vfCBAalM(#J;GX&Sx!MyWp=;q$f=hJno7V+bDWKZQT=D3a`M!Q_tAON;^fA zAKGc^_0ye2f;dq)`Vh5G0y%XL=^UOe&J67+5fkC^o&bc+xKapUEVMd9!9%S*BJ!(p z+fV6_sx&oXbUs4Np(14wJq|Zi3mdN|&eMuYAZWfuL}!8KgC(v6OPC0{6Q3Qg=rf_I z2u1Nqc|6XOW`;@88eZuAD2%%%mFR7K6|dx$TS9^rU(S5eyQEE?i1K+Xxm z+Cz70{s+=GYYgvNqQ8GMX{C1oUjm5d8$&i?fT zKR)AjP4-J^GviCdZ%;ErT?buB);CcK90;N;ksbQ)02oTK#RU769lZ`Rs_h1Z5oJ6> zO3rMPdKA(R!E$dHb<0ViT?=N3JP#hExe7gG`YgAet?pM-u{xsmMzA<&YmThwG`{En z2cJ@x2!MhDxY9v%0qtcMthkseR4~EoXXP9fU83-54c2FmD1f`3N9>j&yubyC7~vT$ zdQJH5xP(!JS+TG^=|XY2P#Fu}b-15foC!)#^rNE72dNAZT%NfS#mPJSF5>X*3Qy-c z3B5(ZP7<+GF(!?N-ziBqX%2MfW?*6p?-p|}rAB@}WCQ5fe+`If>Ue$j{Sz@Czn=yW zgY_l|)ii7pRpRHH1Lkcs=XkfTtmAu>TmmxWuw7Hg0Y?C;QM1L{X5W*Hx6e>Tlo!%3 z^|@S~uD$BNJP>=3#rK7p+;zCU@$2vLx(@1s$p<_I;w^9EgTnf$cvYI?;b32EF+J)L zdnNcKGc4G{-aP?g2|!mCMFqC>Xsm`b>GSx80HLzFw+9VarQj61rY@}3&IMF09O8CT zfPQIn+Q$i8m8EZyVbs9v^kmeR3n&sE=Ij)n{*KIHFCGkx3?!YrCjYJd$Dyno( z7zZ zl+3_B&6@UvQFqlHiS!Z(+?4v2E_Z(c;&cUv7WD+pvC16X<1AA;K0A59td!(Ds8BJC ziX~Bb#U%CwG7v0QJ_pptqh3h~3P4jmOQ4P(XsZD@nl{9%jYbaigXt$_V`bf2O72b7 z5M9D-fmf{0;$&oDA{0orsCXx)kHVlJ^)er@zX^PH-@*MmA}+D~gMmCXCl87O2BJnC zZg2fNwnl{_*RAS{SHLu|4fr+{a^ z%$msfDLx1k4FPMZJGK=@KaxF_tfWA#r?$NJtPn6I5ZG_wR~dWy7z3JZsLb3(j0aB6pq2w@3l6pw2Zb(z zBM2uTp3_yu=7UJc<@QYkhBd_n)>QAcN&aP&0_QUX&i81d=Mtjg6_pxRP@2d6L$9P+ z-w&K$cRE>oJ*b>S4JZysIexAOJPD9tCXw_UV$S}W2h$*$nD+^lWQbG+kk-y2R=p!V z1r>BZ$Ne@fPzN*qDlCcTd^T24oRqrG=~yeFC2qv+31nNaaKysBQehnIaall_WVOSD4r@-c!MmC-?luEC?_c$c! z*s89d9Vmg=6diG@Zl4FKeDaLbeub(sB{5;Fvt-e--0wJy4EW-22mFdC-`8(Gr8{Me zbj_lJO{%CopT%jQ# zqf2>K#m^wz;6Q^Zr2?y85BN;jHhy<7l+K>Bu(lKn>rP-t4LATOFg4C0-d#CDG!juH zsDWG|cuMTz-}vi-1HcLb6?(uCp>p0s<^1l32I@~aJ|(sy>-y`bTHI7VsVA)5$D5KI z&Vlgf7`*gH`)X3#?xFwdvn!&ylpLF^%X^tOf!WV#uY`L^TxZyMpqvB8nZ6&-zqf;Q z-pvPKfI`&WyOPCb;`2Zb>|RFZ=S+gHgAFwt)ENYJP#16y4qpLTLBw1Y!ETrbkkC-o zJDo3X*vW{4I-{seyZDsi4je#L6XSr{ z1p-d=tWk}k=I0ahTa2@*4i{F8>`K$Ror-MHv|C1GG#sGut=e#v+4? z!5D)r{pv>=%*V6jqWl=MjFftR?9ffinJU_zuEQ1u!mV{9-rN0Cpqc6od_C~n+jkh+ zY=H7OrtLD(-Ls`?QiuNgvY4 zfV-QeVQCJD3Kigq*ny_wGD^bFM1`I`dru4#QNO*MhqnX&dUnXMrC+PJ{CxBAJvD;0 zdAKQLG?Wx10Yy?!lRt1)kT~~lovm>YAerhZvA65BU{pO(`9@hMSP-Sod4${~F~bot zPlO)m^}wNtu)>i;t&a%&d*e#S+!>J%CIk7HDfpN`$qiskbvhc40b!wNzF8swmfH>} z%)~~pJ!AeOMZVWjyqa2-4$&3ekMC~?Y<)veo9_>|}Nh$+Y>>l#jLg1UyN|L3r;z%Z8UWCycc9cl~! zMo8+@RwJs1G|}?|T8i>e)MN2HER_2^$)@FVFepg3m+(L9tGq6-6p`3DoiZP~fwEuV z7h%MF7ol(tt0he(FTfhd1+Kw>1U43lTd~4jwn27DBXFk>B#Uks(^rituItJ!eGj1x z&BqbuIkfFn&+x8$m#JzQ1RoT#!L!?stWi8gANFdjRqKGX`;m%zkJwL?-WGyP^Ol>9 znkPyu!>O-T6a@}3C$MpyMnciQHDFnLFv1D!&=AgC;9Jw`N912#s}!gi zH_F%UD{QD0+lOYOwFQ$PYxjOjyXf};PYD1Y;Ch)g^T6H;+_6iEZPD=POQ>Z;A+g9B zE&LR5c`fd_TZ+`>l#z&_J4<3!&nLxcVdj34%~3;2VZTRN#XN6<@lq^e`)R9PJEAb6 zhzKz%1RHpTIR`_Ga|jiOEvg*Gp5M{&k1O>chyplXqOa;TFwR8^=>QtcS_G1;1#)2^ zDzoQ&q2aJPWh-N)WPO+Jo2c-e*}T{wy&hl3ecsh?RF5b)Mnp7uQad*pC7g|wfIuQ@ z$_}joIT$E>fqGN60|XH3&+~aDcgnQY{?tv_W!;>E%8+pli3lpfsE){&++AR zL?@H`}^rQ~T(jiyXi94R016zJ5lZqI+3^sC;~;I?eY%-swim4vyb17TG-O0>Z~***FY*)0RO%2{@IKd< zx7Ap0MxDa+a6FTHS;%RY&!fwZM;c&+@oI2*Aya`{==qe_vnmDe5%J&;P4(^Ldp5EV zPvw2eJeyJCs-y|@Xx<|HWqbUx{i?XWMS)^>CQ%XhIXBwkM#m9H(*YEFh=T#tDVTic zg%8O6XtJR6SjKx);6NVTA8kGHqyu@;f>s0`LEb9VM_?Dog<=LCkd}2|BtXz|)G9>_ z)l`o(nI=O5Kpq*2UkjG+5q*UR0fopHF54*T6-0@<1zH9PE@^cXGAkkJ7^Xwz6b{*1 zW7L?@1avc#5eg03@zHA9U+7vS_v3g)BC~Qx@acw!fI5maM4%o_1e=p%lGWk>C|{a% zX&q5LK*`V$%jfxBO-B%h6=I=fpQ%*GNwlcI(4^In<_oAWQ_70#BmPfB)%m)nXvbkB zujG67Fb_5fZ7Qq!9+X3_YYGz)szT~X{Q(ImD*Rj+Wf zI&+te@@5ZY z{qor30*2Z*EHDL#j4ITm^#b;M?lJ>9BmZ0p7S%9{-hM<;pp-Ctpf-TFBo+V&KyITe zH$slV=t^0KJ|Zd#njPtgJ0B56+kHn_H6ThGE{m2c&J&C$l*B}QQ@^fI21$X zb?(K2c(;mt6#&Y)QkIBo^Sy^V2wnR@{>TD}eeEGY{o(+F)vu21!n5wmup(_XiuMIy zK{=x6Cd^P>dBUH<++zQ3l8bF*Ya20Gi^h9IjH#mp_xADCLgl+7enAxjnD$zsQPW zA!L4;juKA8t7wGji9}9pn<_mszS0wqs%WgS_n@g?WQoqMV6^b-2ubHaOkx_|4 z*E{Q5)=g5Ic@Lk9mPZ3I3beJ0>rPjO8v%eBaI_h~x6n~n^V)F^b!z4BB4`8*l`0dm zbr<;dC8{4G>E6fk86G$*#R~;}fr7VC6bfM#Bvg%fkaq0v5m9;kSto&*d?kVO_;ZmenqEh3zkguFKW*%5hK)fQVtcb#qdYx^{PLs6>)IErEf}oOw0t3E(r9yS~%6x0xl&i z!eS*6== zs~01xXXqqW0^m$u)6^N{O3%sJ_jnt59$o7my4Lf^6{aJ<1CnjAbK}agvQXWisXl** zzF2t z$OtuI?}TpeI*LnRMo63Mvwtd@QhQ{m0rmC3fn!9i;L9;;&t`fostds+@^CvV=KykL zHas735}xgKkd+q~8ybr64J&@N2m$C5LV;%A5s(My#iIoVbN@j0@F!Kb9|Lg%2OtoS zfK+krr4doCN68dS`|)R?Ie$z1j}+WLOT+yuPnPhA?LQ~j`agb0gXZJ!{>{gK|8IZ( zV)^6W{f9sP@qc~%-M|0)|N5Ih{pt7r@=rhg{r~>yKmX<*|LLb6|M<`U_oqMp&!7I| z@BZ;m|LI@;_<#O6F8ci^{{P?q^Y4H2r{DkXr+tZE+}!iJgI{3_{}=vy51{>J#{SQW zFZB(3xUm!t`vqWZpO3~aMDh8(dj}KqhZD7@1x@T1L;5X>2r9|i+*1Q@x_K3$*bvH$ z=sQU@0xy^5Ug?03lK37rOwA5MW5fQ?`~XM1C;KK&6pw|+32(oxh@x^D^s1_kL4sx4 z(DMw8vi`Jj0k9)Dw32mqOc$0iSOl?0`!GP+o`iRHGh*);z>cz{_J~ML-=qObi-;p$ z<4yGV6Ee3b@W3MR?#T8I61+sjz>z6~28xLN9mR?>Wk$qK0f?Yr6&FAN3DHx?mei=* zX(2AGxsjJ^SszfVKK1bnBo%2$xkYm}Q0!O`J~;aAmAUn3w(oC;vS-Yw4JdKzKn$^s z;!0g4`+_(IF}#{`?jIz{K-fPPbvfMXF&NQiKp}#C3o|587V`?}6BodfQl=eD!+nB1 z)r58ch&Q?86}uB;^pY1rWIXE)N-~qRKcCF0APff??7q@dgvC&1LhUqWy@}%J#Dr3F zporqvnQR7%ry1w(d$;VT0g8AacmhLGxe?&9K^35+|2jW>I-(TyB?^c!Q4IlmBRbBi zZ1i&6=ic5?R}OeTcL1|6XcD@-=9V#2vQVXWZn`L zLZt)<=EqT)y~g>wXYI}R5L5sY`|C693aJ1r;tdw`Ly3p1{`&}|x%qkb&$sSM^EK$n zR+#rU#;B_5of;5;JK!H=dwA7*v6x1qJASS&2jc#BTF}ZjpP~XeY{QeoKRzzjp?!Nr zx1^Dd?-|?D!BZJ}b?$G7Pj|e^Mga;092|ZuLogIk4?n#TVV4Vih2N#ejI+m7P2Nfh zXwQCM5p0PFCaFnztiLWKYZI|$hCM;Fo(Y2B;%OQkY3-kL1njk)0P;=)_Tx`6_3==M zSZEYYY_#r$prVYRqR@tGnAA12dJy`gQ93}1P@P4qg-K~YI@(M~*fa?`ZxN3hn8`q{ zx6=Mi6cfPS@KPyK5*hj}z**QHdu#mB$@>vDoLCngyaMEkhdMr<=5G-(%oMZUU!ryr zqi@b*&co;$_AmA=G|ePl2fiD<_t=CCDX@Y?;eq%9jDF|MHdOX@?0Mkt^cD!xtP0{D zb4Z&RkWn-_Gai7D7K;vWYiD-x^?3g*RhG$;nQN=^c@!l0f=~Y z6P$}_Jw1XFvt;Lfap_61jQ33ChQf@Cotko}Q?6`U1$-jD1%&`wF*|Ewb!v)W^DW@y zunazD!DM6T_XXC@d?G4^r?PjdSi_jERh#g8c;wFusNIS96=^UUQ6%=%=*PLYf`}HG zh9+%9IN}sTj62|m08NyrZFwO&6d-M`(O^UIP(EJIkzZTNNT5kk7#yxjW9Mnh!Wl1& zW;}}*Al~6$dqaLaFVxP8T!!`yBc&e1P&Ek?UR^6#^hCA9PJ)`Q3v!8OEb0vyXL%k6 zX96g4=$f=>plgCaB?9Dh z2<2f@ZaXAUrEyq&ujL$l2&{V{5}5%ZWnM^YAL`6|(qrb2Jaq)u6Z+sjK;?togkpP7 zd5`b{5Yc*RvJgA?+PCva7c7b0g!gN1hcpGPc>N+DN~}}8)Rbpf9|Ua$bH9TzH!6wQ zZ{Z9I!b0NNUUIEHJZE@6NERKgj8e22rZvg<1Is2z*s39M;N}xi%o}`K(>jBy^q(1A z(`UMEfi(aHJt8znSBEjqj}|BqO0@)kI?J$LHyL}nGsd%*u?z zBdPmaKztvBnipR-NzQyf3ymlm%=ELf_aWAW9v4dE?*ThkyJV-gV@0hqA2FjH!!sB- ziCr7Rm_ioZUJ~>gTU3$)wy61>65mzI0|*nk_j^Jdov8thbt6zbdf?J?Sfy(-$8xUW zbAu-}&n|cxu)ptBeX48mjl3KyIZLsu@h#LMF&z>g-MqQ0pq+PKl#g<(A|XWOz{U`IjgIAK}xgJu{|ikbdlDQGIhF4toICT zaDJ2oRLN6d7hVZ$`3GyRqg^2@ma;NWqO;Ltj@)a!wro~)9Fu6Wot%`Y%vN-;nOkww znhh8=s^_s-%Nha%!w6Put_0_-yZ7Gj$KVmys2|A$P9i7|1BIwauQZoPm}<`v=O^ogis>dDVvd$MlAjxVVLjoe~ z>kG_iKp@kTo_Liakq4)}9tSQ`nJZ-Zn+b^)7~*HxM}LiIAW{wn^8Fyb!@egttP~!= zej*ek_EQI1^U>6MZmDV>EL2S3*2!?Z>i`+|aiOzfxs`kDnh)X8Yzez)|DZU+Svvb!8`VLQH|?m2V=z zwn2+~l=!)W>lZN^l*`_GQ{6sSdd@twsG(=0M9fgk-+4@&#}}Mph+D#AXt5*que^+~ONll+ zpl2cqJdTukbnA5`yPYGv1tJO(MRMl!G=<^}rC9ky1XME-&;@ZJ^HF4^>iJ5MHKF`n z?pH!DmBGI!DFsHDQYZzo$4OS%U<@)M)j}vnJT5bdxiQAuepKgeHz)ZL5m!PJqJH;r zk~52Iiy>}OrB%#M(SfCX6L72#6GC-&L4J>F0kZ?rC^1v5e2QQRP(;knk%`p3x*;b4 zn1Gfs)ht=~Un9H{X6CM#Xt~#KP6Mut)lvtG5jU*gdOEWB?-3!Pz=#Yli@4C2s21pC zO1>t{-teQ@(PQ1p9ds9z(Wz(Ds3;a5%O1D2&G(4=jz)bD>5BdyWC^#CjM5LL`i@D+ z`=8xjVPc2s(ipeQ`85hMlWzr_|CXIe9EMmofL4ausXhohKA0=(NoBH6or^ zR~cdrz1iD0=cyFb&|K_#E}g8bpR9~0R-f3tj*7|sGRaJ*3C=;lsT|xnXN!aG`PnD; z>BrH{*b!qouRi`}2JEM6fV=b^5U!(~!h#xW?rXhyWRwz>AEXSx>bMXL(%9~K5cMJ= zOMOS+`(9XUJj5hFAz7v+K`|3yw6eD!2$+UKF4%luY2Z-qe zDXRtmt5;9c`Kfk9BNeXtm&budGLCe$G6OOz*6X~nIuu>O?e3ilTvxu`^O8++Wt9!_ z9#DXIzf3rCdT%}iM42R0x2V3b(VN%NoL-Z=6K*oc&ky``M*uMT4LSc%Yyd=S7XjLTf^0HUMT_QX=oi0N*kl zaaLI}JOf5;=G9NG!2>%aI>J$G-T%vm@Qiz;jmJ*vYrt=BOHDw?oL$Y5FF};o=|=DE z@!pS)m{F9bWyinhsJ=yfw?Mu{QQwq%@<>YoMAKo&(Gwx**q6>>c8T6D9tk!xiWroY z?O-DECO7~eu47FD7U#4S3e?(t8UUyCIh&Fzwx~%w7q-sUJn=lFBtLYA`)Im#+NMS| zyI91RV*dEIb0IpwHiD=Q&nVD}aI#kns)*L$Y_9|VYB?#z6waDRQS|jG{Tvmg)e6!@ znf=joY}hPl0a#(1hM$uae?Xa04|`$_Q|!u=Y-we(^U$zAjR-1cL|Ao>IXxoAjc5^J z1=a|g4kaCf7lM%4BnX*+59*{?sZ%Ev>um%-Bw4{Q_j{zS62WQ^5xHBMYgG1$2s8|G zeC+_S3Rh>7gSyC!^EBJpiXd|y$Ok@=nc`u}{`LU85)L5M>o1rN8X-2G&)ImIOF&~| zAC&O_YA8c{R^Yr;yYQ`YAR>T3*E5|q8%{pMUV2xgZ$0+zZbyQ?ZI6R!uI+=xmKZi& zbk&}f;A%$10B}83U@^i;g5tk*Eb!xIL+H{Sy=I~;I-p~?Rsh^kPb6xxN=F304ec`; zgxa#(atCA|D1c#?9vCdo6cGhP*RXPbZpko#7TkmOkrC2N(cDto-{&V~9~46K57>C$ zUe>4u5F1fH5Pa1J%}!KpxAUAac7$Ql>K$H5eKShbl?oZJFM_J-v|@z_5E8E-nB}0? zIWGk=#M#l~d_UghHZ0^Q%Y_qu4bqpkvQ68&+owB%0?(PTzW^Rl0|kK^1%11QpdDGO z(+@=iM9e58E_ltyXNlK3`hPfauKC0#%dg+^xu5LE-QwU|HqzSdaXtlz@#|-X`bxZ( z>=$?~^LSo=hQzzmJm$hZk(?Vm$98Th%o#*x6)i?RFe{z{ao6vSUEtEm5eUV8Fb~^0 z<*s{qkBA?PTx73Diyh%;-<>CP-~35Yu?HU@5@zi+8y&rGBGAU-(hKzQ^Eotr5mmK* zO}_qxencOU+%IjA9)G%KR+6ITztE{?a5Di_U)IV1{Cw*BaS(<406}zU_`#WHJ&o8P zHRpQ-2#5ni!@L3L(jQ&0)`^IhcV{}c+dFO^gSE{paW|}C+WGQZyuq(=XdgUO*sM7M zc)B})RsMqs0NTWIDu34NLKg5qnN(r(^;8G2)8k`(sOGis_dwptJcjd-$G|pam_|p6 zt#f@3h-u+V+V*}LwAsWTQrY*}v3;|g)EjcX?-34yb_T&kc)dljQ8Z^A=yLRP!}Hwm zX(6J%LkrP9{hk_k`bZAD2X6i*;mT9R-18p7hx85R9UT-gS!Y_^iKoo|H+^DcE6%WbT5knjS z#XoRw+E4MK?Ae|4ukpF}h}Wb7Z3SqiH~w}Wq!?uDsKrp@KIIatv^Qg^=~99AJCfIV z+$|n=zg_nR0b4#m8>hzw>}%l)3-98U#?&fTcJMIpUKBON0uiZ-p) z*bP8urr|-A!Q$p3yA1TR*Z}J@WMW?eV%W8qWcF@83mN`;7AT+(TNPw7a=`Eb0kD}W zkTHOzL}7RU}u8TVbaim|iYDHU0fA0@XCZxYG{YxzbzIqscAET3f_mu*CO24=~UY z1Sny}=$g}2rfcxn?;-t+i2TCW?eRdeh6cx2=U{-X2PzMKJO%?Anui$`nBYzwA|Bc) z6EGNhFo}ZWwP&8A%xjcG1MvZGGzVR6M~yn7dIgbALYw&CmIsZw-NJ)>m%~b!H{36 zqQ4>TJ!uvZ@OvDyQ(?c`{Ra6&Ed8C|BjQ=L$P}L$5Ad7wj6?pDMw7iiFji}h(7>UB za)_SY)a)yY-WMf&ZY<~wbLg*8|8Q;xjs|sJrfD41YN`d!4` zs=>&kPD)|-oAazgBb_v0?eS7^vY0$jN_vlYTb?7i{ThWBjwA=1nQA17&Eht#4kxaB zSOh!&ojEEgv2{ol7^)NQlP_(d2HJ$O9hP29x}en9%Pt6kz!+#DQccEP)Wl5Q0wzea zMAtWIg^pOn*^m~~hIW8|uIAab4_r0rp6Z=gIw40Amq8&CU4n`VM#J(CXbU{ty;4ZO zr^Q`m&w-aOvG|Fcg0_G~L&=im788)P1eEQh1fg+HYoh%i_U@N|jy_6@l7_Y?ycb_T zdyFSZHM&e-8VNnB==##WM*w}HCx_HRyS=TqC{lxkEk%nox%-=RXXv^UC~i&GV5i6r zhyc(EO`<)p_t!s9SzGg+RPWV~Vp<@cX$b7~Xi_`wu*>GLa>W}2A9Z9I6J|@^!LQ)| zE>KWD{yQr@Lqh)r_ndwPz~2d}Q8o>Icy?y{J?Gv1!-*0hgzjmUZ_@NFTUd~rLUNv} z8X08nY(x->yR|PP3v2KeMbW9-^_d=@c@Qroc079_<*+GHp`d$=P7I|;N(ZRX zvBzM`f#EK$)*p<*NWszwavEYt=-XBuWiJ>53@$PB&kS-_!zzsCaxQ^WrmAm`iG3@b z9wkj?T&9rfz_y>~I%xC8&h}>)q83Y37ZpvE^g^FToCmWkBC6$=q5qBtI&EVP06_JN z0Pg^w`@E1c;9tqr_kIXfMEH?9J@4K0q({|p5(or@97=x*JX827%0Ze1O_Rm6x zq=~`Y)Z@9^4ONmHJXsP@<^eY+c(t(S8LYk|4uIKQSXkaX&L~d9L2A?!xwD-x+hAT^ zkQ&ub$Tc-U+fRNO&eWLO~D8DvaD1H5gD;3M|%YqEqd4i zfl7IZXsDQHUayxCm;(eAsS!KNH?BzYzzGz-(U5iZ1ac!sGh1>L+agSy6TqR$@}@rWC5xIJEOUp%t^eMg2b>HRQPw_G^M3_ZMaZEZxvi<|Y^&fh;VNo*Ag$iTTRK zJ*-$5*=A4*BnsQe&SdcyFFAT=L@M@TX3HY<}uU+{wZf)+bUgAy=jqwzNae^izNOY=7FQ{r;F)C46 zXgT1Z&g}y&Qz|HIf~g|ejN1M-tyIh5(R!(auVp0~94 zp021D3`zW~fAlc<9r4WN9W6ONGIRyre*l!v&$dHxb9M4rh!zjN<(YG5ibai*1#qiQ z%Yk0R*c*tX!PNQH@Yx}y_F45I_88wi2i~B=g+Q}u6c}v^89Fk~PiVRY+i}ZKp+d}L z0ynW+A4dy#r{e;wdra-M1xsQeT%}(`S7&WGv9_pac0Hna%X(G?*%zIT7kDPROojAsvf7+SR_uVi{5FqemXaQ${b$uMzy)`1hP{P8)u80WRe~1UJ zQ=Ah5OqUOz7wm38YQ-z0cxXHrQaAoT#LgxA;d~Q!K?AAG7E@xRl{;^lHw7&0f&ep2 ze>oQ&#Fwzzex4j!2R57c7Hvf3O>FkuX=F`N*KEqufj>D!X573# z3KmMn%pC|ko!CB34PeC7-rSAR5NN^gqM{AI>tQ&y!=@k^9Ctp<2MSuTM=EiU0vXbM zLiedmFn^<-mjg|m#ne3bbd88;A_@}j^H^k`GDZ#1aDD zHvA*V?Vfj@R)A;QkqapNrV&JmgP?BNv?@wOz#2WiG%v1XE(ohe(EC7k1r0eOf=Of7 z{37XL(!>R{*3@fzgz$(-Hrh(n<8J2napbK$I~!20&#r<1P2_ff77f2WyWnogcyGs3 z48MU+=!5S>k3`xM2E7d%j+o23eMEr!B!PeN&G_+e_%ez#dK|uc6x+GqOF0nwrDNFi zJ*@gw3=vBeYZ0Ot69NR;A6eQGS>rA;_V!?gJ&)8o^n@OYVz_WbZyyG#z3HniYLum< zC!FS*ve^O&&|#Qy9^inShh>Ebu%|?%oC1WUk^;A4!6$qO0TCcU{-Rk2aAsntu&h=H zj^b2`VX>tGMS<8Um{TAOw&sw$EaH7)WCXW=9B%(rLId9}`f8_9NYS&Q2~8;!9*fmx z>l_b?K5RM3h(y({wr7HP*I2N4*SZ}F{Bk01rM`F{*?uGU+e{QwNJLTU!=4@Zy^Dq{ zGeB9>06`afew_N0`ce;1ffT9{Hm(L4Ql6g|!ptoR@Qe!~fk{*z!LdDe8ARe zH9|g!b9qEOF3|0&fkWZl+Yi!PTF|kI`-2QXx^113ep8H_U6!Y@x4BQ#U`V%kf>9N0 za(8NawXn)K1gP7b6vz z@EC8)ZGU}4J7MpLj~M&A@H;UYq9wLafvAp2YC^Uh#gGA}VR_8q^=S@6U9Elowvuz$SkR-OOSnH&EQvPV+PQ)2D`z`NyI<;~M!-xc!P*artF*$FPm~7! zeSB;3k*m386b)&FutaFu7=5M*LLFgdsUnU_Z$mUU+U({qzDusGOM?j$!4QR9pM5`N=zaj-wrGG|$ zS9iGTX249Oc>hr($yw2ouCw<%KDR0{Ts1hN8u!yEs6-S^UZP}{D1M@?}oEbF^BifZ1nU64}CmnR6QQmPwE<3J7a!o`D#mgCL-h9b8{=_#* zMdxf)G?v;Dy`0(rOmb>E`w2t!EcP~bhJvKVUc*W1;hLitzvpQ5rFKLWNl%+ZhqQzv zjbEgMI}`?y?bHNRo15&Tir?n2%J{ml%35eZsQ3wCd8x{x`G*{XCW=~5jjo7l1p>Bz zT5%t@_iX2GN0l}ss>O3)YZfv}YF0cNuAB%mOS;1K2rWhG`cQ#85ooXzG1~LG(iD4y zNI4n$jCa{6ke)=SUI8*7*N}aMVd5Rv8N7)jrc|)7hzQ&*K;hR<2_9*aPmqOFCWxII z#8Y1L^#53Uw_WR%^gQnhd=Kl;@qxqIlZQzo2n;w#kdwqmg3(q01CeDoih$gG>UrMc z{8!I0r3C@4hJ@BXzDaU$kX7I2-OQ~1`>miHj`RT{;+MetDb`(+Qsx@q^;yGy;k4JG z+&qlNhDND!QpArRYrFkkLopZ(~Wprnu2;eE`$ zVBEA7jpAEPjm94ESdBwL-8W=!*(EEmB><8@qF?VikwS7s$DWGj>W!}X6;8Ey8Qo38 zTGmr-zt+r|vZ*7MggOOU<2<%V@bcI~-}-|23Uj!TU5OM#CVUf5u@7iGz?A8s)oib^8whl8}9B;gQ}eU&KiZ?1aZz3FIk1vmv1{An); z;`AtxmBhKqvdL8iM8S%vf{EQyJJ5$9^l}_PnQAkEfJh7~LmUMgsa&A0;n&$7%2A<; z(vT?-&IK=PTX zhFF-Vj;#H zsv;XWL66Tcvb~g1*WvXe{aIShgAfkd+;Tmxd3ZpO!j(@i7|{K~@jRhy@puFaFqZ4f zDxvf>stDe*en+6dml_jv9Cx@~iiBWEL<)s`nl^vzhx4dAkfT~x;&mDguz3`s^5%%& zcYw)_)-M1lZS{UG=j4IUgCWH1wF(KFoZ_qawby*i>)D6M^E_@ybm-*qQt#KveEv~I zs#~;6&Um~*j|p{WLdwN{p=7R^Vasy_k(I@|sq6|G-O>0U*<Me zV{|{H&q z!!JH+V60bM_~mW@uw+CH4T%z- z?p2Or2@S6|jfHlvJ~$bM)lm~i)im8x^RA(_mSU#+6d){FWZDB7L(MWS)Nw23aqHZU zbAn-CTBOxDc&wp95`xkSejySE4vRJ31mHe=Z4!qOr7bn#@JmR`6|F6{lPeYfpZ--` z24ELx5zru?Uh!22(CU(ds;dPgeux5+uk;#l&3ychq}KJM)`v8TFKNiA*IW24wcF^> z^aw}wP@BOR-Ul9faGfv$n}spe5CbD*2JORSRjx2ynw_E3NkEA3r8*`xl#;n}7=iZ8 z<-{W^6vAc!`mCir6)N7iAZxfvPt{mxlaoq-`=t>Iagg>IJ$A}yO03LU;x0E zm42pUIK-Fa``DQqX@!bpLR$Dby%u^FKnfoWmS~ucR}Y>B%ghPog|S20zMoUCV0=0- zhzkoIOwzLITQqUQ6kT*=cn{zqfk%?EMICjlG?aEnlITjeBwuy^6piMPwi=?~D$;vL zz450gwm+ii6enUPqF-+WoDL$s&Ln4Le|HhAM1Zt(z3RWtME)iDU`D+eg#n)(wTAlv zVgdNLT2!th>`f&#PNOGm5f<14%v>yT}yRz#|?uq zHNLS%I4!z0D~Ym#z#ep@nHT@*luPkp0R)2}14a;4W%AQ@p9caD3Gn7jBKU>OQbDH@ zR7VZs#xtC0q76YO3u%%s=2Dfm+js=0QDJH5|M++q1p_c6U5*%fR1?N?90m5mAfHIb z&h>R%bG#cGRm)!m;^nzzOeqiDOyTm!RMrS82?s`vpX@0pk8^nE0hoKg!riz&*t2H0 zU2%PgL^zIGyuFy0mr&c9NaXK$a0>0)QXCn~_Jfm!KM5N&@YlO)M;1x>5|&$*2X;Dc z{C$c!#8Z)u<*HTlN0{pEmZy2lD^_@FML0{gM^#d42ht7l>>3l^SU3oI*33?FPl-;? zY8oWiR2WY6_Kv+5=>u|}p)u2TW%$5P!)9Scbk9%R=%Dw@!$$_DeimI_Qqo6t&oKR< zOv&_n^(4!PlKedBxJ*A5`WsALk6xJIQ}q_;GP~XJ6rkpQMYK=k@ue#wI>XLnAD0aJq%Gih6&;Z?i=oI2Vh^3#OSa~H)~a4hNNCfhuI;S)HZ4R= zvgY+j=*bjJlb`n>X9{za{86enX&5sm=qtY4M5v;mos@8tQqUz z;BVAsRN|Q$Y~RZLj8I~;w?7OktE z_n(g+m4N-c?F8xTL~bfOXurGnA&JkB#LZ5j9xz*7SjUt@BTq6G4HF`wJo+f2oCt?r z`xxcudGdg`zTOu)5gAh=*L5e)Iu>pK#ZXi-x5mdt2c4;&k2{Okau-1--u4n^w5-B& zrlZnJ{?JJc?8uIEWDikbB@4bKF;K;Llm-Q191}EI>AGP09$hL7^olgX*|AyxgDf@o z%u>mkqt2(`pQe_IUsb|6%c@E@4~;5b74Z8t{C;xecPC5&1YL`V%N_@B4M+)IF)T6@kV4XT6K5>%o!5%7|^AWAy^)mU&sA2BUSS{~8ILTm8LXn>)`7HaF|ESKSfz233WwV>&2(iL6fj#vXzbT@I{G z6w2jST9ok?Dwi(Ww+w*8Yc*;nXU}gAU*Yh=U6cF4ECV@#z{Celjg0Fh3C*`QQdCz; z2GGrxbDFZdqu0ZKMlTh{R9OKHL|z8$4u&|o0q2U6W}shk$p2tER+lJ+xOuD(d&I?A z^=~l;G(Etuc{H%#PWuAA;1zRmeY+($qCTy6}A-4Xd7)mo0MNPXv5&3$DX2D@*C1Z_-|3?#7^d0!8B{b-$Ka&(``5vg^XX^jIho=*;w!g3%U zID)bUp+MSFR))&*xkWx7hb-t1yQl}Cx}U~&M1^KQGAD{LzLXIrOf!NYEQ)rQ){4yH z>}pw09j(x`fap$J{Y9Xt#i$N@;U`R#y>M-hkp)E(TLDYdMDL@G0lV8#Ogr}uG{%nD zZ&^=7v929xvi@QdsH#lIl`OlV@oqnC=#drxJb>RiQ@x(?tw&@355>Zr*;Ie9#ELXhVHNHWLtACK|%NVg-^UZ=kEg6BoAb=3NbWQ(Zk?;>CxhQ0sZ0y0G?U&0-|Y`2&d zvBmfJj{9ev){~@aq$=2Bc^nJMz24P&#~af1W=AaLB6@qTU}v}eL|Bq6ERD+7nS{xq z2Le*hs3WKjQfh+d6e)#B1eO235#t2hiMMA1i~mdPp5|Fe>g*P>=+^VFnlUO+#0_g)a&BG~;t|?h;ML`13yJ^pgxgyoV-bpC1`9|3Zp(g7(dPUI+zssXc zkszS89%Us+{wYt^ehmKozX}(w<^Wjp@b3_oT>PRI8^yHzwE`%2z$BVP$dQvu!22T{ zo~-pcU>n#=7XHW&F^^Vow3zS<_4sY)v&kAr(G(up*X^&if`YJiYB9sRp0^`_2~|(V z7~yxsdhfgeP|7Ayy(5C*vQOhdkSc)V?*7LPpOn&)2`S3^OJV+w^B|}_bd6`umy_k z->)j5gTuF?pSBdZyUw=%pns25we?4@sr{G+4uDF1<+b_`UG0bdN2|8yYe_K%*& z$%Qtf)u{dGmotNfBJ7CNrcNk994;er0ZC_exB!5Ch4 zUwtHM@T}W@5sIP5Ru9h#a-S>HEOuSL#Yo+AS)RuNja*`d^p`Ck=#i74c`6}@_Al>B z7p)R}eh2l7l?&azaz_s+4%b-|%!(ez#D~5eNxz-koVmPg4%(1kdCrPn7EtVtgMpe=3em=RuK+77q81yo{a zeGtnKkOD_dRN^lw|2V0EQDugxD=~a;G~?K<$2(;-z}KY->yE!4HGwT#Vl`o zmt$CuxQ81QJ*(giu*hZasXaOjqW8oGl1Q}A2+wp>LvECu%k#rwY=e@1Ln#T;BnDrt z7Adda0~W+=AR`?^!}m1bgN9o^(_jEi`zSqwno(b~Md$_lCs!jKdR0Aw_vZr-3pf~1 z(O|aBqVWEe<~nGo^cRbYXlu8XTqUNj<5@Y5MX$i0_m$&A#+(ezw1y%d2)+g?(Vai1 ztbowNszC&ZEh46ayR}lS4P~bj5uh+q)2OezsdI~L~a+hKyyFW6flCy$$)V#ZvpUd1fGm(pNgkXkyesF;6_HY6LFA1 z1A5u>!-o@|7Cf|$94*xCH=Bcp+ruI`Rk$sp@d`ynrtY#xcmEey_`U5c;; zj`15}47OC_sr-CKqB_kV~mLQnYkIRmTv*(dSN5NVdt*CD-j7J+qFIo;URb-3I(w+~#%$ zbJgR)sV%W!b`bHt5`F)2}3OWW7o zm&0UddS2JG85!CZ29vZ=5Vl0@0R4go9xgf2eZx#%h3U4GBTsKf@i6Eygpi)z7Vupp5OlXmHE?O z@imo+qB3sPtxkAO^d>6`aw9M3WHqO)P;FWn1r&`a>UU>*m#<9)B2^DesRgFgoPq+U zygbuzPrF{EQyumPUm#G?eI04%d4No!&7jk;M*tOS<9xujU3=P&^JqbW_i*hc!fpvd z{a<6&asXnyCQr!mMJZ1nscC1dU1CYhI8F-~pb?Fx61_@!t~*F}+ebu-s3*hH8Aj`8 zlqysTdhkQ9w4nkfQZyohIZ@ka^opWDGwf{9!hL}vM4@sqd6h?M=Gc0{&*-s4bFG+1 z_?`&2`RQcyHX@2Xy!k7_L~5WC(SPrrX0hHRpGQ}R7B9LwR}N#zg`h||5iD6mEYB?a zJ1#c+Yvx3O_b5X2s+iQQOiarZz^bn>5%RqA2H+yVeP);^R06vN`#t zCD#&Zvl*JUAIxAQL(abu%6^;ZA5l=h9YO4$A5pOO0AE0$zYeeToGC#*%GF+FKVQ|% zNU?P$(t<8wQRq!Iec`|Q>&hhpfza0negLUE>Ybw{!^*gANA#mNtL4(dL{Krd7gq+k zkY*4x1?BrD5;W`UOi|*ZIlT^Zd>A$Fdd?KHMtaFBa;Z%1`Lcn`A@Q!N(H*`Qr1Nno zH^WWGd%lz8b4U2^n8zymM1M1*#hsI%CaeO--1|rKxbG2%t9si?{_^Ylh`J6+s1y)M zev@I_d?%shKF|zdIBNOC0#nKUZ~vZ^?KRlmzo&eMVM7ZWG}h(?599ga1Ms={ zErHMdk_`NeqPPz3N$XRkY*t2Mj#d|d-s2G(3$&x1fF^Ww4R>K)kHc9c2*;5iydwdS zpE|1H;xSmIXmvyf{QJXzic30a7X`TQyxjYR8TFexJ%@vZqI@|p;XtrVbb=i#j_2A@ zlvdTU`OcdX%&G_X*&D7GIcp-hPn{nCbRuI{o>|US1~h;kFk0i z@P;EKGm;N$*m@^|^2mtZj`m#5F;K$&IuXg51;gEw4L-+Cc^^oZW7_I399_iu;NA2Z z8Jh}VAAT^``o%yCo2fk7ZQ5?AVI4~A!gEw;Y>n(0?vuq5U?{;u&9bYxE*?%9sr_rv zAI#yKAU8^%0+g2*yqLsxv=GAuWa`Z%Ef2EU!gaERk%=7Jk>Xu-W+M>pz0y8BH>hcx zMO0CZM&9IY6x7m8_LibZ9f_|DsC%|pe!esNE1^p}>Z04l1Y<|E7Z7OFQ*43n3(xgr z(E?*xqeq4*ySi#EGx^}+R*=blB?s;Eo$wbXt>>OS8NtX0>rgQ+-d!JAxbqY?+Q_<^ zEqAlEpN;uL#5~!E`PXTmC2F0h=OrCk8nZb@<8IJ}p{KRPngMlo%xmuO3{%GQ&n_VF z)#bnL2Y-{?@ysg2an#0 z%}`|x**mtlb(1EZ#52eH9`HU`QO9V#KR*O9(w4gO&^4#xT`jfQjWqK*tbdu^`w1~s zUGvbXo-x|*sOq`Z9dE)^6EVM_L8Q;)%4^F=KvmACgOT)u+mZNQPbAyCLxUZ$o);Tk z?u>39s#whmS6pBgl9?^=UAT7?q3aqsQ%E}xze zZc=XtO#2RkmH#Q?31(Le&1+xx5ha(R9WKQP|52$>bf{w0F;0d85V1zC;!&NVVLsA7 zBXZMde{@yYn*9II<6N20a3TK6kLx@y7t8Pu0rkM9ule>i8l*o2waF@4nKGHa+hf6_$NEhx)c&*1$6}jujq1q1-xp^hP zq=$U7K0h4imHx!zxxeXFzmjgvBX1F0%FcrBFz0j&!8z)F_uyxxUV7qk$^XB|+K^g{ zT@B-2(EGjc+($iBS6k9an_PFqE^!cfi)NxUzuQqu}7g^=!i_kT5G$%BsuHRp=`%bjYKRYh!ycdI=VYxAkUg# zq~;eHM_@3G5XQ%67T^fMLIcj3>a~nr*gE$22e9cV)wGX;Gf6}Gg_bkvOX>ZGh*}Sz z@V>AiJDl~8s5_4=6ivs1Z;m|ZOz`^t=;tF#56PHYQv8Who_7B;7;{vN zpw2uH%6&*{%IygPt;b;->_)^iz1vH^*PqT3Y)$w9bKL*X=&3$%GOqXvq}RGkS?`%9+GBzx)m~2I{vw*c z?lu1z@s_@ZG_7S%PAf<0s1Pl;z;*C;?mZ>^R+z77rg4x{PHUmweARH#?}ozi&!rL7Jl z^eisx7d!eGTc9DK$}E&rcjh~QSV9$2nvDF5BN(+dbx!0rU*HJmG0jGrlnDk5B2Q3W zk#NMQ-eL&(UTvk%t79s=-|79Ube|nt)%)3@57N$HZvt1Dp{J&8iZI~sdiUXlFl@bI z6SYXMchuBC29heA55S5c0&<)X-)Z(3v3fjz#_x`+5%;uz;b-w3iH{6n<#DCUiKRVb3}Z=t=Bq zId^^KF`vp(&Uk$yOm>Szir{j_2qFZd0O4wv!C?O>Vqb}j?VTJx?{vl}&xCPYOz-VE zBd*#GBQ7fKXf#)Kyn=t+`>0wDAU|127TWDXN`V&&NEIn_p45{=FF=Sbm=-Vi0m+a3 z&2@}>$Uq=l0oyUxfNi|`;had!L|V%ZO&4@ZHIACep)cN<9`2~&sG6v2*fA^reB`q6|EaXL48^;hKxs$3nUKNCOECaX<(CPa!q|mbu$mRt~R&s*q z4uaew!R1eR?1%t>N<<_D;ZeR4-3Y6vcW__FZV`qu5?M{&kjKU13#YXENc@o(uD#f_ zXa&1vE%wyI6xf>?5qmp$1s`{m+A9#Dv}!g=H^aKlRP>bG2EJ#eLRsV`NZC z#E}~8mpW^U?czO4v(7@jp$GT0B(Rdo$^cpEY14tWYE@SQ+T(Iy!FAHmT4V{i%ph&| zLOJIzGcuVn3+q|$+7qc6WC+1L+u241Zte9hyoVUSQPS6_eH=c+xMkJVNW9s=`Wn`L z90iJN*YqCNSbk)6*&&DO1JaSlJ^e_{Ba3wW{Bs)kE3zk8Acl#aKS8NTclNT-oXV2VXw* zpk#bhOmWr^f39(7w^oc8Q|quRfA_&i2MU$c`MYOlg-EmXQM{h)Cimt2VHv09jh?L) zZfxEdG90OS0PtTox%f}98qqNDKJ6aIN@0RY^mf)V<1Vt~Z{}z(hLsOZR`@r551xs7 zx;5G$EtX31t{R}GO;m^OL`cAa&w2dig+xBc47c)ix$Z6dvWkL$Y)LBd?FDTeT9fz) zVJcVfK&;>Zde9lO3F5%%`v-Yrz9(fLe~vxP(Z64k`vLf|G9hDxXxV3kzv9TR_`+O1 zf`r_&PIT+ubDbkyR5mTTgh_fLrRN8?7|dN=zQ@IpqQ}_+8G+9zzmchPT4pH$L?+## z>rotq63%|t<+dRMdca`yAh7mzOvSklocnfJJrF8Q-R0Z#A9~y$5p~3U=s_6Zm6|n* zWc@+wB1)SYz0%U|DrY@Md$$Sv3#d#THXi2dg(Wytoh1~G^(w55{S=(%_CCItI|JS; z!f{SP1cBZM%p)TBnTY650swvc?>gUxW#FN4qq)(EWV*&z73(cT zXZ;YdiiW{C?)ML2GyzR<&FL?MB~5E%lVr7sa*LRAx{vDy^*bjOGlmIDF@?0HSNU;W_& z7^U8U`Nb_skb)(9S1;>K#b>7Cq7WT+QYVfo+`F1Q&{P}kIhf$Qvt)qe?lYH)5(L=@`legwXtd?~+F0=<7YPCx2MyQC}X*Nf`C?QASYVn0N@ ztk#D0;`voRqCodg_3NiCk$#k?;83eEz2e~utz@baz*l0?pZFlH(1xl`Tdc>gaux`E zJ&HQbz7r)3S>UdR;o3lUa&=0Siic)`KXV>ktq5Sdm!1?_8S)h6>pU9C@xuwr*4{dQ z4!e~3bB9nNjpkC(8Wyv?CZp~aN#>q7BFmyg5lrw)RqZKj0nz)tYR$;_qko~>1vfz*e^qh! zJG$l^ZzaD`Ovfu{*0>dup zZm7QjKqyB&jeT^oDFV-uZnRV>l*|5TVz=M55T?FGyeOt%H=yBB)qIbFDx#%!B{k+j z+Uw4x0`~a;EyAF$_KSO<8ebo>;j>rYH~~y`onz86TmqsAsP-bOSB*!?ep0xic8u*F zRi2p--1PbWbg*(-lheD4L}wR}!prrZ4JBB-!9+k^!Km*~2YzuxP$e4ysZahB&b(r+lgL*zH7`A?O+GSM zqT0Smhxfv0M1PFv^=*BTcHeTQMk0iN?ql?FCL_-|{*hqa|D+7|y>qFuYac$iE&un< zrVKO@zG!HD-H8CCzauD`XCj16lrW7w0bTn2BJVt51Na!7$9Jqp%p$ok8-o4`WqojD-yW79U{wgXAs-)DGj+_=bz?25X z@%}An&<+&xZxm0S0MA8R4fRQL(`fWI#~_*(B&tUE71|8}SPk;;68>p8kfZPc3TgW< zo4yD{gf20d1KnT8cZEbSOkqCCZ~Y?aABCI411;>0;&7g)2*_WnCZEjHaDKUP)# zzWzo-gS{yXQHE*f1k$`y=lOF4CRhBtAjZo%{P!Sh=*t=xk(_-PL>_d%*$?Uy1ze2X zsS+n*eFgY`ou40g0`8#Gl<$A_WJlj$dT;hEUV$)xzygDX*^5)G2SMB_lE2@i*Im^Q z@lQZR%=r@NTDv4mjk5eX6QV&+@Q&hj+Oueucm+XYJvy+@D0qoLMHBlPUngoP-M~T} zN9OBL8F0rK9)?tri2b+ePO1sLXr4cb7?8pSiemi-@$H;x2yMf@wnv0*Wm16r91cq5 z$!G*kK>8?Qc+hWz^#B$4GeTetoq@4$b$fr4kElB`T#LEI;O14K?GMUk3M@csSGjBN zC>R!&yX*e-kc26W@IAsaA&G}g#D0DFt$#RCoUZEx;MI6%5iE2VdUl!6DHzo6)wWR2 z=`Y-e8i1tV2yBvm10(!L@4_L9%oi$BV?E~wYy$hQyT(HFyOhTTH+6A&SLzWOcf39z z2g{KL^TS)`@DU)zY#A;vw8OiZ2xZgv`nI%SBti}^W4&SkOl$%o?lt2mBMf95)yyJv zov3<7Z1d$;?-~(gV-Z>fR!`@KnSC7zusiOyJK}N9@hBJG)@Ou^Xo=DEgYM`2={Oac zYM|NOwqV4vcrnRXu4F82zOI>f3@|GN!JkiRL}$pLYQG&P^=Z_8M!Schc|b(d1W$E= z=Oj-v$b+Sfi%0yMxZh6y2XTN-OQmvV!_Ug2i4{$U|s6ASf>_B^WR!M z0$G$6h~@GwQL;CnO%KY54=tC#*IuT?fj#lrsK}z1@bQJVVR~l-*tY_n8Bh!8xQFYw z?6Dc~+!V9BaJ#P@Ur9{#ILd);Hz>V0vv_Y?Aadc)LpPzZTpsP#h#2&LjgX*+Y(p^s zHa-E+ay(C;g7}nH_dN9EKt&a(U??k5ccNy$O7op;H7xY~j>!-UXp&hlWtx3}IwfRF z@)Fa{9wSB^fB2CixiPB4ejBupz6&(g9?{54A5)6t!S#wzcWWL>F;o0eSheHXQ&_*1 zRKNf2YtZP26D&Wuk{Z?ry0fC6kUy8^b-%!H)b55K@X%saQgj_AbCdBzd$#;K0S>T4 zHS;FgQ!_| z*!P>3Fbq@l#pNv4s8nMJLaX_YUXeeYH>x2b$i}o^=|URndb8wiJ5$vBLTmk@}cImLh0o(7hLA$7#j6DRLQ(5*xrGu? zc^7xK7SlsMV58%MlVI?fy>67D8lwb)$h9z!fQC;*>X*G7FuZD+auPr{IRX1C6Hws$ z`aLjJE2h0U4_cZC!_bkYf@!@u8Lm(kprGzNzv#p}ny(@vo%+pJ7}k`1J+9^2puHf2 zy__p-{CIkrJ+KHKXJR`c-o!6$`~lqJ->(wy20D zF`?lWBY}t<%U)A|2zZVyB^%D0s(nN~Gg`xs0#uI|jC2uSx{|`sfn^Lw2Ns5dT@dXZ z_hMXxKYWkKYSlu|Z97F58}M-)nkP1U@ch%Rd8mZXI|KsphX^!e*-y!X=0WiP5CvV1 zdEEZdci#M4WfBVpv~~Q4jv;o}Fndr-?JV-&rl|@sT4TU2Wa>}n_|OL6TiJr<&(C83 za&jk^6O)lf3Pqp&rw2uKCnVci$SC)bNRoDY&I806$TeauqW$l8$i^nOzXr0>bowgo zx+g=!49V!r?S*JR@Tr{k7XsaU%*>xFX67{*X`*ahRCMFZ89TWjHtkqn0v+MsWakI?hgOKo(r(R z;Sl$%pURxs+QE!=PPDV7mY0tKv941nAJ!-0+hkaoEiZBtWihqv`0bCAptk_g@KD!| z{S-DmK%vqM?H>2`kZcLb9(@T5{fS#xLDVr9CPVVW5o|(*(xRR7MRjuD5*mdL37Zpm zYLMHA%ZCzyx;G)+Q86IhWy%=2?#r!=D0ijn*i)iOD#D?Pe)3GSXMMI9A>uAc(5l!0 zhi0#{=kq34l+!w!c14CSrI)#aM+O!^^{L44zCXHZ2c$v-i2WpPS2Kd#`h1gp0|-IT z^g8CJz~TX_w9sOjjy1doos?H|b$XQ7N~g_L*Aqpj6dyc>ixdlIx_9;#sG z@tBSCVvPE}+tg-xzD^#N#u87nZ0XjF~`HJcfE9^>1O5*@);p6xvsAt;x9i$Z0VA`h=p~~ zP`6xhJw>aaTLJ4f>Oqablkx3<2Py{mTZCuow_LxS9zaxXKOCnQm4zsF>$8HdoGL!j zu&v2{0S1%YJ_0VkqX#W)HFMh6gGdNIywb>rQ(FB$_L5 zV8nwwS^FU(LrYH6Skxf(0@V0{8}S+OmvHa*@%oDxy_BDL zDdQm%^J8tIQIonN_{WG`o8c-t@AwYGEJ2g(BO*ohqj>+m!}$7k&`;(gwx zjTG7~hb{Y;#~E>x*b$Wf(adnX!5V$>u8#m1DIs=-7qmgpI0fj+YZhFy6j5kYp`6zN z^w6?l$0y=aNAgkILh)0E@cXY8>ol`3o(Ou<;^q)K8LW0CaOd|#4n7O(V2fZ%NFA=7 z9!7(fz0sA}+h4CnqPoCFn2}PFmh+90IhDvO&0vnF7w!v7O?yS(sN02ih{eXAWCw5~y*}v;&MDE!T_r-LnkE5;p?Kn>+Ao>rq zTyb|RsoF{Z5CK;F;@7#%P|lgYMJa%*nK8e-9@Lk_aJ1?>9YJ6zG;k0*SxN>W5L&%2 zNQKnTi1!^$ix3gtBbxG;$HCeQk?kU6uY?ry`OlVEGH7vwns2Z3d(?^#lDg$6By$0I z5H)gY?lq5apXWdpLA^gN?Q7cV`W}&6cB^j!3{}6#&*PA$T*`j$xY|!@Hco2xBu}-1 z)#Us|5`URrSII-#mv^DP&nf(=l}9|=6|zGZPb=~H_^TPAaw>_iRO10L?;OLX8X&57 z&spKW5{1~ID|GgPNj@8|b(}xJN8w>&MEZl1yBptk_{AP$j3WLJB( zg^v6cd(?F{8xW6dHtwF^1Hg$jM@9@S&(rl0wRd@(Ch?fY@3lFMlle1TEQ3#QCS1&wP41;!IEgrX47JS_4HVG+KDo- zNhOY(fZib|+78CM-ho9yg?q(ZhVlnYR8Vb$^5a|JMiqXfvs0>1IJ| zzdox_!57dLxWrVTHiPsF0jRlbW&u>BnRSJsb1P(*URcf6<%iGvp=vD^6n_5PZW#he zcAT@4Ct|)4{X{%R#Gh)2h&jJbq<{%S#ePS}{({J;1#L~!&0i9Zct377q0mN*HxXsn zmTQqtd19gX9Hq#dX(wsCwEL$hDAK6FQqg$Yuf@7JPEmf;qOCj*cow#^!p9cwNpl`#wDl@O!}zv^ih}6%|0M`6!DvJT5~3BU5&%QW7X7r*8Xwl zJC2yLvHVby*TRWp_~#4MZr|@n(ZgVbSI?h=Ot$mSD$%2mNx(7*S|~@(ujAWp*g;Lv zp=(6ql--QIIT58A!5t$)rGAV>vG(vpyvU3gNalKPXQzTbx(~UbK2nFqdHACM$d>FJj16buujHFibb_x zAhb^5F`$h^!ivgsZ0;G#U-AU`p65AU8U8^od}d%HITHr8&pr{jbd;DY!@l4oT^x}i zr>VN{NOr0Ho{dPhwgchZFZ&DIA*za|EYg@)Fk8gvXw2uRW$e#skhu#~44%2loEwcr-HT>aTwFi*ou04#b#=}) z&xcxR(ZPHCbGxBrWb(^q1cr#Xg+|^jsF(@>vnN7$$G(E{R+&aMM*P~KMjb2Y~By+)HDaXlYt#THM0jlf`1)XSuh&XQDk65SQ(D?`tk2+%^!8jZ=bQZ zN(BE)!uk8pExZfux`8b9 z&xrgu-_o|v<3ODQPu7sf9A1htERH3?RDTcfA?W8JIM}X_Ac{j!wl`ZI(^W5boy&zk zG)n{{ZlrW+e_e?4b8o>pC{*+<;?D;K=k!JAmLGzIL^V-R4^^YWq%!Pbxzh|q#f)7Z zl+qEF_^=`cv1F&d9*PV&s3p>i*3JstaQcpj%EWw*fGev1wW`JI8nn(Hko6KnW3;9{k=MYbpHHr&DtNOFsXrWN;#j-{ z6#(^%P@^J_yP~dccHbjV=I&pRgunPKT9Y+elR~v%fp={&_6_bq=pWEG0M7k-E+9@+ zF97{mC%#8~ttOCGU^X{p*uEbJ*_NB_i{)=bfBe!$j?cDC3+x_}` z6V+i>@>7>rhzwy=3N#BlB52SO!LU++DclJcfX8~yWMcD7>@JPanQxJ>-M`M175j|d z6f}d+u%oVWt{5qPHk(J4OnHnX0Mt7rkh&E}%b3p!;`D-9z8DF-2&?0J! z)+qB?o&$IiFcQ#$J(zmn1x(`5gZ&8LfzDm&!35_#--49L9X%4c0LIolMF5ij&#{eH zIgq@v3Lp^_upb^~+8+TaHjWl2gw9(DzHdhvp#cN!-QRbLxSs%`m>r@O4}fJ{vC9sn z1SZT|D&zG8EUy8|eIgeU4gwh5nwnOiHv$cP9xro^S|&`!A$rL6N|T-x6^HfmmU>J& z*qQFGjL1bs&TNb(S;NWm2uM?sVhU}$66B*YksvuQy&t|s1pTxQw5$r={NXsYSus2S z3!+kUMQIR(BQpgfuv#DdeBjWr)Td?L*u(t8nfzV-Xa0YN-!1Q^ za|mRvhF9ND#04|0e@5g4E9qf_qOyKG&d#|32=6){=@hRIP!+?^ox(2JgNNa{l94^- z{ZV(GeVrcMjtbuGIofL9qyGK59w&?aV#&V%^#s(|jp_A5mpRXp!6#?V0^JAC?Dl)5=MFla;Mv*e_Ie7nuK_g=eYI3f&*P zpI7Q`%>tbU%n%8v<{%O{ zAHU`)7$6h4mP{!{$!2HZCG>RRG$}j;#S!5lsDOYLm=^+CxDB{MRKVv2rCesb8guM5 zVY260pEu+y`$75v8Zm=JM&KfcpgYE60|pF>4bJjt{LDn2SF_(ro&eBpyH1eF&v*fA z6_Eh#ig*G|6(nQE8NUp$0|gN^-VwU0Qh*w5l;8IA0KMh=KW9p_(#OEbY5outKYMXM zZLipo=!!AA@a_3S1WpV?LHT`YXZin(x*`G>^{dK)sJEgEq*Fq0SjeY<5Nd z;TlJBRx=R|UUt<}&h##R)D3uGylpMgpGN_K=$|MFIjF7sZ-0pSd%=Mc@jvP41@`&x zN$2gDYLFGesELZ(MM2i#93K~+asWX@DOK^j9)yD|XJi`E-t5v$wjU<(u;FEdw1kS( zLCrDjgEIasuaW{jZ^By`C%0IQF^Seu4AK~bB#hA*B+*>9yny(EL~s{N(+|VE?9#P8 zDOju0gF6LzaRdY%4Otg6x&q9PncIO!-hX^)yFr=CSO=X@1;w4ALGuGIDcLRNp($j4 zcYM08M0L^6M9Cgh**V9-xRnUBa=a41Ad<&b!WjdKTs{P*efex;=lD~t65eV@osb_N zOmKE(ydcvtPd+GzW4J|AuOShrZWMSj8*Of(iu^zHDf6(>1*m|ML7E@xFi_jYnspEY zRWlz3-6bNhkK<$>q>>Ro*0qf0p;@Oi(tQqr^rH%^JWTZw-&s+KYtn%V2IAP1X!761 zj_o7h&K(0$u>cg=F{`sX35{NJr-_(Xb5xVvOCw(_D zDmJn%<~o2K2@3w0kb&)0PBI-+U1C_F)^8lWEAB;{kT37OJrBN;{X*2c7^Tj_Tt~W(2!sic#6dTM2pu4H#1NSx z{#G?798L{@EkeNkg+$#!Pviy%qf~E=OS!Yy7Pf$mqKig556^YJ(Jml@5}wHxST%q@ zO7H1e&R9VQED)W6ZGxnZ;z`v zFbQyG3>3d`=Qd=ox7##cYt~ENA>%iQ)O3|AM`0^rLEkxz90){lQDWyMLFVmtLG7n+ zIj4>Uz-jH{pd3c)lNq)1B+wj1D&bBx_@1~}S1>D-yd}c<1WAIvM8XfFbnTw>e{{pH z?(h%_D*7Bo$cNtbU_Xq?OJqIsxtSCBMfV}r%c^kJbj!P3QkK0Otc8H7JcnJ`4!e>E ziISeq{j8&2huSdtRlzB0lcI*jnLPJ+GUT*RE0)hiU zukmI?Gmw=HdCC{IG~$W#0J@9^&CqmPzdp92rVJfAXh=k;>fM64UOW3Dza^=`+Lol| zY9(!*5b?TBlV4TBUVW4L^c6?sw4u7=hNMsIrlwC;zT-Gkp?Jk^{)dU$oBI7Iy#%U* zh2&pkVjp~}z+OfvOTUhIsfi&>8pn& zK|%TYBd&h%os0tb;(SOC>;XDC{3G$YJi>wbt|Q|7AR0Ln(FFEwmb0ZFcSdI ztw1%EBSMU*)<|h0m+~n3K!S$w5+F^)D&|qi?K9K4hdE1Xx0WIKzOyRG`fN4GQz$y0 zgjt$5N@sw#C`6<=k|))Wd!s(UcxcmKeb^H7LmpA1hOo04(V_UCgzVMy zc)yT>JrNk+;Yl%Pt|bH+C*m(U9)e6RW0VMH0ub=u&8e-pa|PDyJAm<`hi7!k1zhb& zcdBA}QDDERGja@Q@QDsNvd8__iK&kR#k5sX+OBsgHLnsiA&!n_${)TOfBGxFjr5*? zygywiT(S6u%Ai6u{nWoZQ8;kWp|S<3jy2L1Pz;70I5Y>~!|tM?(<|UOIDOvgy&DkI znL<)?M%XQ~+OWUBSg2_k6=?q5sKWibX^z^2gL&1xlm$aWd2W)79kKI)l1qUr)G;BW zcvtxN;TZe>uP@*AIj9&k`bsR+gg$YsN0RAYfB{LMhdpK>5KiAfC~C2|st7+oAwxS| zsAQ)*3T$0y)v)_{#EwbzX-ro;$~H9iZoS|tm<%Zni!Tlnas(-O2|6W|{#bWBjW92M zC5?zGsAF~9%UTj!dwM$x$gCW6@RNE(n6lrm;W=VW#R&EcBb2GVolOzYM_IqlL)k_U zJqDs2X`|c7tdPIDUaO}PtEb*_0je?YzhvH=2-Zs?-gy;%-(bR!ULK0^9w^!*=jr!p z+qbms%|Y+DMen2Z0dQcnEuctf{q`|e9LYM^>{nKjNN-M*T= z+X`A->Mm2Vs+Gw3A^G`E?-hU%cV{QISJRvBdd-B#K)|5Be7Cj_7M1OCL^GO94m*&@ zCOwqJkM1bLgd(54!&QWEE}&+@myq57 z%Tu0HxyrM>b{h81DpZsl!jycGG!ca>OX?vO#uMXP_%6kDWSS{S?^TvVED)`B=j=(n^(Oy25|&(J!G`)G&IY z)hun)!gz!k0hM`2VA$s6U;UhwpNg~g^R>tW7Kr+Htzs0eV9f~hVd7T;q!DHxEC7Vp zh8^JL)WoSGSHx6tq6CciD8>@%>A%-4!@M|iXZiH(kKL6E6V(>zjG&0Q?~RY@JfJFe zafUn#U|)<2d--)-X@HefX2eBVs3ij-(0DweD!(*di7UgkSOc!e*@B>Q$kpm(xp#oT zM34bOeHHB>wT6p6;-kgvk?e>@TT|w7zl`+S_sckzIq7AZ24rsuDQ$pVDFM~eX^^kfhYhnQ)nUJ!QBtc^Ao*Nl?&8Mr3wp@_PqSfJDL!j*i0VN6q*7 z!_?-lS=Lm{Mo4t`9lJr}Tk6_^-?cEMxrtO^D1bN*NL)IC^A!iD?j%pz`;xVRxAO^m zi3)Uo#|vVnx|skaH5jyuHsA@2J&sCX;KvqP;z?Mfvbfkx*Tj z0fbDDPV=Z^+-gRkdVkK-w134Qf02aEC!0Eo{J;TQh;3YkXB1^AaUBvZu2NmJ0X(##&U1DmhfQ9kyl zx&v)SK{dXmFB1ys7;7M?nWb6jc>{cpc7tmdz?vzsM{OscGu@^s_`&kG* zH@dzD2#;0?4OJ?*P)EoB0vd~$$aO5+7>mw>hD87g-7o0EG{|}LVAfsz&GqSt4z5 zz}r-^zF@@K>Lt;dUnj_3DPJ|joB}LuKVs7=Zs2CuWE_D2-AFmno}lc^tY~KfKiVV zw^1vPrk22ycpHv>d(;PVKo$;M& z6sbU1mcTjksU|Jr7iHAjWfRTJVV~8%cAE}*9D?$#kBLE#4>97r6j2D9-4R1qF^lsc zZ*!AguPlDHx-jSH4P|a*b98cLVQmd%Ze(v_Y7I0tATS_rVrmUMJTFXTZfA68AT~8M zGd4axb98cLVQmcxpj&pu*URs~I$q;LFFifN{T#c(py`TOx+(_>0zWPpxAJZ zL2OQZfnuPj(VYf$1^w;)T-rEk!m{o4GxKTaxRoPunW#V=5EUozH0pr?r}J*!{|&9MufOVm!in8Ov*b0ent z28(&Lc_LrY5QWbPq!eS0nq8qC-3ehj?dwcJ*>_a8=?JzHVcx67*N!_R#0FGVjoOx& zA;vkpA_UxcgW;{Sj=>9hzMSa!s@L}_3jWt#j!{O;sReEZN&hBwJQp4z4?IwsLJ4Y_h9XgaVffb3n z%U^`ug4Z-0(vj=gv!XyQysJG1s zq=9ktjsoJEC?_FQfr{u4Wv?LxgObm+DENe@)pG(!OZW0A|C`x-jG$08dw!jWaW*IF z$NUJ-YO?R;^uBj?{IujVS(RI*Pv@MR6L=?K2T#Q`GVSx0Z zVl)vm7mJb6dmiS`w1mbz0vfyS9HvNTMrF^ciK#+2fFBK18C@a+RC!1ITKX_cSy2iF z(U5bZd+Mx0kp#yp9uMSYcs$r>e-5l2O5cppHRj^L1uN34lz-}>u|2~Sgs=j3o8A=@ z(=RfTFDGV47sBH>kx0A%MMk^yIrRS;G>8#uAfU-bu~EHCafN=_Y!}KXKla*S7^Y36Ln|Zypu?B<#;XjgcOyuVn^H_;rZ4wJGw+zYBZ47~$;ImGbV6 zXKOkAaRu#_7u&1%rQhlk;ynFw@C)$iy z?tz+URwayJC-(?vo_A8ouni7qAyp&MEK3zaQ)WeJq%kV>z1^>#r z6(1DQB}Dko7<};0a0c?vh$QSkTyJfpc8YRndtpQRx~tKJ8&eM`&S|+qSrT)a>uuc0 z_Lnw&K(|<&1u9U7XpBO#{>^X3Nt7xxwAH2}iE)M05XJ^7fG6em_NEU5To*Mb##DR$ z)+4IuJA$!7B2d>Qf=vHC(-qDA2716Cn*IWE=z;La082o$zlbjY1OVujOA!L@*fIgg zLlu2-a-Q}wROO#=VWe>i zD5sIY>-06!&%dnmIa64mBi#Tf?mL&IH?~&ZY9fj@QL`P&_p~j84$~_+RBB+mD8=dz zPIJ{ZgWR2|vxKwa!XpI+0DcTGX#xCV#vg4ugdAz|YDI?o}&lUB97hE>`Cy&^iZ*Nbhh z>iuTsqOtedKaYk2x4SyWpnWaV`j?@=ebj}^<6G9t`E9Z9p3xe*g@+Z&myq=(!@8l{AMc5rMvstw;v>@fG63C5^Pe_`Uu&1C{{p0XKoNde+T zhPqcvo=KHD1^SSNkcWT`8u40QE~Aw`GMpCnQ;6RDLvLUEpo(9+d|YwTq`N_*4Oj&8 z5Wq*YI6yR;G|LZ;<^=EoRX9%G%^|A1_eU&x*OBBaqmq6=+j+9|Y2Ahp%>WXWHAF4w zooKHH@}HKP`TPE8YX@F6D(sP>*rbO*2rLT=LEo?j z0=GqdhdP4(hKk3i-9lQOJ#@`lkI!#>3jYOJe7ORP9ehdjZUi-%ULgMcLaI9=SPhdntFAc>X@ zcL6R(Oz+lb96XWw;*?v85()7~D7psqzATMkf&t~eu{U^y0|QXEJ7H-(U}8FT-!FPg zd%`~W1(IRrFsUd9B7jH87cc@6y{cn@dee=O?%XlCV!*P7;A+6y)5=sZv3e!rsBcOK z*-HgB@@#voiMEy2ib&$@%289mMIfXN-G=e5e!gfMQqr%cU%OE@z8S1s{itxOKe@kZ zPe7wN?Bu1LdpDHp*IhRe1?tfLg8j|!JUD@}bOXp(LX)MY(x zMiya25p?D%NpWN_Ii%`a23^=uf&b32=$sM39G?g_*BEE7L-D`SYQA93{bCEi;bdvKg~q5J z%aV7Y(h?DGi@w4VUqaPCA{*-#t~;WZ0l?<5>vKQplk6X9=c+Hx|B8a47~L9(d+=Yf z7rH;LTlt0KR*xrF>?%8RqTxIauC%@;7)6F}yMg-6%343N$G zS}wI3L?EgyaZ%4ze7qDqZL|a_Fi_Vu73Rjh+@t;Lu>JQt<`n$= zT>(QXav&ifdjcd2QVm9n6)Rnw+Z)Ix^&;Qb@p)MAt!5bR`a<#m=(BSv2f?#nwB2Na z2n=(-a8n#ZKieQG6JFXCVJaLfkIF=X7_)$Xl`482LMac{u8aTuzY5FE&K!fe+s}|8 zap{ZZD~LDw{MQjPxn&B)EqXm13^#4#Lt*F^i_zxV@7%1N3R{J-Aq9=d%P%yL$YwOf9$awXhx6V!lTdZb?PK^ zB=1)j9$^nH1NZ-o5!gBtgB=C9$zDk+U6Mpj1e%j}Bx1^?Mz}RqTLZ~-{}U=q^qwgZ z@Gv32knDvAM_uLtJJBx71r^5pQXE^}ht-e6%E8vdt6$s@W@Gc6 zcU_nBuBD*pHREAaA&gz2U^iT2Rk20;l7Gq%Mp=hoK+gk!q|QXoPdv|ioJ}#Q0xWor3KZaf(JdA z$FFmI^WgF2hXp5Y(`cV#B4Ualdo@bs*)tC%qh=oJaolH^i7`i~3Gs(8yuT*s@8A5B ze)Rvs1O0)~d;ar-vAb}m`l!N;f}cGX!4~`Nrkwe^6O>gG`wxZi{xZmWkR(WgK|?^zK{h3LJ*c}1hzQ5-6C zCa}+;Y9QqNMfTlM`^Quli{jcSrWHh0-DZgBTKtWw_%}G6DSD>NG$ik1@xU>z97BmX zdIsfX)=TfKEG+}gz32kFklG(99!CTg&^lCNf?X+U3*&`mgraIp;I*nDf+U=X>)mks zv1(*;c{6pnk~Kwa8|i3t;8S5TE3;-d#iX@iUf!!7|Hrs-k z@BszTUPB;Xn$HcEh}^(&jL%~35ysiqo<(xcy>n&JgMNl~I}ucW^f%83nw*R*1m1O$ zR`$&6Q1`~tPGzUR2SH{Omhk*G_epoW{sEvnqhAo<1a$k&>ovI=R@b?dlUQV-!R`1< zm3PMvfKng$3_P4Wf4$dD|62P!X!&DF-e>xjk^FSU+r{8aM4-dS51oZW@DrKae$TI- z6Tx6m1E7rPyr?p_1NL`|8`a*QJd9)?B5uFh++U3sgA)xBYoyZ7%#I}#;bE50>U@dP zOIR562#=F?qhPjqj27(}VgE1rHy=)~f)Z zXsQ4=CI2eD(2#9y7?z>N9Iw)pv&k6r%mnS4b&+%QuCqgk8)a6>#EixpEdm;E>lgso z*h1*{K0pGaP*{jQ)nioPXS+b{*J+EOX?)a8J*qh5 zodTrKySy~)VN-tsngwj(H?eW@$gS_65$QsOga7fNRiN#u#b!VAgW33I?B4QB(IeBG zf_KTpWnM0#v%fkkMA@jknYt!Q!+2iy-qx|_Q;NAt${}%DY7Eb!lk6YmU;k{jdH65? zGUmMS23zfn6G}8+g0q4K+miDE%+W)YUhQ@I1F2BDfT`o%AgY7)KQAdQX>)RXVl5t6 zttlSl0E|B>&rwMW~fRJ$Tat}92wQ_jBbIPQwzVkxNLok+XdOK?@w=+T6LQkO>f7k$Etn@pv* zxJ5AqB?<*B1BB?p4-=#MyuN0L)f6M7BkhPWW+DV*GUvjvGAAk-o-;?*{EH|Vg(VV` z`-7{V6SP}_o!P7$!4`L~F-STB0BFF+nNf;UQm{C2BGRD^{yM`;vh~|92 zRt#zD`BPMdIKlNbMePUrwST{Bh2~yV-UJEGfSe8;r~m}2$mpIk4Wh>j)UTnYZD0Mq zWU75cq`g$nD3n)r{cxCxhB>P_!i*zyUo*F*{zF8qqMI=y-UG#a*utVk`vO6TqJwZT zWz~5vV7^9ex5Ad*a3(Ssu_3eI|mHb?U+KJ8!cqzGP_kM}kPT znH_aqIZo;;*~RBf&{THB5`ZL=`_XZ17j%&^0>Q&~qV4r-8kPwZZuW5JUwt7Gof&rE zrP>xcN?^`PKj6CyP)Md>`aZIYGOiy;Vag6|dKG@{sq|YPr+Ot?G683G5Uo>j5_N5^ z=j7n@hMAf!2AB262$GtvRxzBy^FVx$T4C!cUwM_ZkHk1#!kL=tmtuN$id6(%&-PAS z0Ik%|h(Di!h%z)6TIMSIejMjuF_9ZUySs~syw3-urs#*YfF47%#~|)i9s7*pm2k!| zi-f2Yc>cWw2Mn+NXy}9USf1L>c)1%@5FEt$SVbt7vQqE+zZ%1^^ z42YO{(8R*wq*U&D=;LU3@APH7YL0F>-g3&rj4o{X;O?r16H~8@4)?9IiC?3C&%J*% zdHWu5c&E2bX&i zK*2ehgnL=5uM-)N>lm=LA+!fNe|~J(uO*-i?m3{0$aa5!9GIG?ZC0L4ZVr(iYF|H@ z6Mz^^lM3bKK>$ZlnndRgHmo>>YDWQDO-c|bPkMy=nG&<2Xn>n8^5JK=svN5=d$yyq z^enN%%5TU@% z%QHkwZk#T>QKV}$-DXKXNV~ujpbgyVYn|J#A~hSy?c7@Gx>lbW()lqI8Nn0y6qq})}tnlno#>Xb~SOhD>EoI2yj2sA>-RQye-Wb<$U0Ua>dA~_s?p>*npsu*_%=S&35w6( znc961cps~!lMcAPAA+hvIlQJcyEWx&mS+gI6^d3_{PAMMRk-`!gekS}bo(9g+#4j< z|4BU&w+i}08ZyT0xLZvDUSS$j^MQ_*Ocf;3tSBB5LFM~3XLnHBoQyZ4G&Y23IFuVI z;Uw>fU9YS}NXXg8q1+=P&oY1DS6L@Rf|>c6gNS7Vvq|m)Sc8(LhLRnQKUP#`86!?v z>!Ex-LJ}Y=O&8kz8IcpL7?GiwZ4yt^x)Vk6&2KBk2l$UJnTWz%q;+r%SS3Fro?voC z$-GW;A5l8nv=L{U5o)9O57-~^*BB>70cu!pMMxgyCUBW<&u4^Zf-?j66;x!OKOIM( zZRmj{YIkrwEcCouEW!S! zv<r6he+4}REV_~Z+4q5d-)BVbT@MNgCtZLupT~KHJR#K9 zK2E~buOwWUNVV(WPHl@l0w5jqxUHm+JwyF~PBePt=) zw7@3O3jBytxFNYE*yE{6w+di8Z+eDl#`lPF#`>DzzK#3OC{3KW0=p)i!~khb$O0yY zkf~Wu5lm;gyuZm~OR&UA=w=1Fza%!t5teMn1CF7(c%<>_;h@4XjGz-h1UATtpc6>M zDW&UKzgZTdQvk{MvFDfvdb#t5`rZD9u9`61E+{qy)-f<^NSK3UqSQtKJRR3%&T2mU#DAQg$Y;Qx z-!a~_xj$P`fBySnc<)2cwa`f=yCQ%*Xgz#J{J24WL^1b7at}3LG@|M=zPP0@H*R4< zcR$W@5cU`bNgq2rVmm~${LUV}#~z-y7BU>bBgpl)^S$SZf7QOLEM>AWmS(NJNv&v3 zk!52k>?p}mckUb3%y`=~>lo2n`95Y9gLRAXul6|K8AKlu|9IL+uH7GhzBdPYD!Y1% zWX)=)?zzx5slp>4<^T-F5&4l?+N-I*a|mKs(XY2-hEo4?B32@V zLm9D7M13QG`4dr|FL$Nu3fXDc-jV5i9j5ag+R_yU^l_M8m)cGIJC!dBp+@IKplJ!W z+m05w({R zxEpPXm)a4yj}WnI@^y~$#3a262AD1vd@z1@T$kkwF|ipxpdOoI@__Q&j}cmv-RuW< zzJcO37nXO&Hz+Jov|`>S?yPa^+(r6q5}KeNSADOR8W1evD?BO9HW1jBC0o##Npe>Y z#CcrjIj4se`YPyL=&R5F&)U28+Lm1BdB1@FVSfoA8(^#Mmygo1;xIX~1;K$O1VMu# zTQ(CNHbJ!{_}{ypcZ^YMpS@OhM*tyEETP%k3Y;<;yN!qtgyro1l9`ME*4I_UANqyi>jk4uz_J?<2a z6e((9vILiLShy?UKq$|c4U%T$iZ`Hra7xwPci^qzN(**J;l`FYEAZ%SD^kXI47MBW zX~20&&a{C?%~nUGgl*8G(<_10O2&z!hpe1IG-aWLEFDXal6g49n7Z2Y_JL~?O&eHb zVnK6!HzB560JsW-Zoq3!`)g&3Gm78K#y34duv>iI{At#fzmY3Z4^s++Etx_MzZfAK zTVxzPL^+sDC`;KGVd@)8Kd=JPgDEp(mAmIeeD@6!wMhm5VH}1yBKG9ImrKAZ$>deC zprPKWs}n_~0>lgMyl5nbJ(Btgn#vp6@97~(4{T8xT}WL=rBi0f$7z&9QF$ucc|=;> z)0_%*Cg4z^ugsJ+tK)4=*v%V$j7M;=69slR;omP6(pj zQS)eMD`x>CWoK$CW;$*xhBG!IvDyp%(i%9{fruNi>Z-L-&Gl>Ts_Nf<$G69eg zC1Q0*-OASZR5Ja-^e|ncCO0)RH_X!y3_B3+@({1}x^VStl^1``i!e=d&Lo#eAu-+D z1KpBQ=yQ#CZ_hOP%#;lxlphB%Oy_N~_XTVz^1tCrxsRDlx}@pSyPIrHW+8H(V%HB# zQDWj%8OG_BG4(drkOE)BM++z`2NkiCK|UQ_aTfS=TZ<&_?i z@hjVe`!!Y8d@on)OJOb$*1uXR0JPX7Hezoht%-97>NIvK(>OpSn4qe?srdBqfZg1W zk=fIqpSeat2_WsMT15?u z(1#kYhtjf?JrAfI{F@h(K_s4W^W>grSMGn-nz@fuyLp zMuUZQM`NX$g>-GkU0DKLEQeFEmJ4xF_KRHiQ;*WkR|qwsfGb-sHk(3jOh;sHAtxFc z11T4our2|p;j)s%)_9o-Y*1vCBg%%5ko#t~LsAl{LIQ#X;;A9>z>CajFA!B5;s6{>VWQ;#=noCq0C%hQ3G z{T7T%FtJW@M?#U$fRqyBPNbv&i>U57C1#k-jIk(uK;xm;O&{!H$#f_LzCfjbt#C_b2@>FJYte+a<<>IM`cF54c(bE;Zsi%B$^ujuoI0bA%LM=m8eNZC_Ard%p&v}_UB!=?2eC={W(-t zk>;i|n+WaYE+B}`M_C0?*^rn_N)`n_A5(7WtDpV9Y44lC1NCRNlF6lji)=6HqFi7M|N~ioh!RR_L_^PFw`k)~hV< zbz=0-em&bJe_~iBf0MaKrU4}WtS~|J`>DQYx9EAIuRD--sRB$^uxyV!(O111x)m&M zq7p6XgxEe5K4>*5K^e@VUEQ=npG%~cnD-H^66qXPSyCQCnN>5*0eqn~B^$r;OlkP= zc@JPy=dhU`eppy7DE-(AMHL0qS}TX7ndF>T+&ffc0tg;ds8xnR09S!MTeMo-ouIor zf!oD{!8c?_%96>2{8`0>)K*$la!C+XVcxR29!f48;dNAK$>nY7f1kKZg5o>$ZqElg z#|3mK@B>i2rRq<07Qf;?6nDsnhU7^Fk1KBBGW%z+6A)bg{uicvjsyoKXd%-nbcyJO zxEt_;=Hh7Qo7Ig#z}=WeQ!<3&t#5&<&T|DE^VB0COa|QO$^^d{hwQc}(lU*Jh%(ub zJPx2XXxOV>XTUV#AZt7%MLD+*2{vnqXjDqk(vne^aPXG2;Gw*X_$wh5wycTtjl`f?dLIee@aB3NcKNUwEs5_mT;-> z?R%fB5#Ve*3#0{^K9N|K~s0NniiP-+uRp?|%KmcfbAioZ_pC+xH57 zMi1aW^Z(B~cYe$`|5^55onV-2Nu5cR5+v6pCmnW}R>NK?veH%q5)2c}|bQ_xi@r zd7!}AAfJN>Furl1Z970P&jD}>I*^)|IBe6#86>N6nxsa(PSAsL$(Avy^={eZumXI$$h=-t(pEoP&~^M&eCI ziTE^#3jU21sQ`aaRhz2G=roYUfzt&n*z;CN97%O8tR=ZA{!g1W z{Ux;~yl13Fx|-v6aT)mPz>+}zlANX8kb8{9Q^NWsnjY2NkWQze2@J>dJgr z=NVSgJK!vd9CaYcC(4RVCXf*~?Fi12*$i7e56s zbLJ~d(g@jv8RJz0GnQQfii1KQI`4^t09W`c85`&T2p3gu@|$2N+guH${Ce6~cjd#( z4tSeVK76=yuq2sV0GH`rEk?d+nfK_4k;zp*w= zB0~Fxs@G>}S^s z=E`WKW=*H#a{NP6V#0NQMFO&GMD#iAJc$OQh8 z-W;5%;9#Eu;rH&J8nU(~565l;4iP2$%oe5N zXmeg-fgYsOfzU0%ccX{QP|UT#ZSK?t2KU^Zsju6S5&(CpDXIgWeXPXnBTfeg<-#NC z>Hy>6bSSA9XDEqng6vFwoCYuqoeK6S0>p|1xJ`$Q9tE%C4qef}zvduJ8Kv5VWT}1P z&a>=N9sFC^Jot&k&YGO6+0tqFUD-{HbC1ct8CQ`Fg4`9(>hUw5z^C|L&}YMSbu`FF zRWedrB5SMaE&SrWT9O{hT&A^9+{JQLoR;4-j$UteHey+p5Dlx)71aanD9)4>mJEmk zMLkxCWYdf4gG{oNsT)+i5#R)K*g4+u%;5oxaO{Y4KgVGZC`em*#m z+gI+oe`CeQRM0Sg8w5Q9HcgRpSWRXaA`OH;mer!M#iPp1+rnSj7N3Z1P=jdz=+6O3 zkaRoHCAc+%~n^?5}F%2F3k%Cnn*wWy_b4H_0C}X-f1fnvkAwC*=UlLo` zyPj_hl(&@edex_cbRpB#8+263PVjMf?RB!wbhr83DiQ1i?|I??o6KO;Jdm(QeKeyJ zQKnMbYq&=3JC2ej=U}W4mQqSDY zpUmoiQYBT`n7(Z+{aBdIN=FMt(POx7;+oqX(%N|xXPqF_sU}h+A2ZM|O@{;2sm*v}^THfQA*RY|r^O+`d@k+qC>?c1&GU}t+=aS!fc!lhd}H=Ao% z*vB;yE79`8u=)Jel>7IK1NsDUAcd;(@!&>_M4+NB!E9bQ08Ebq%S~umzK0`!9|s}W zkiW0;&wAT(wkd9f+hmp7c`}>WGn;zTd1NHOn=lohY&fwBtA2wX1d!eGwn?X#7f)yt z(8?9eHB-(^4;WX=C~!-?hXoX^>5y2`ooVag-|@@tUB3x=%Y`n6wzVIbPgS38N>3m zt2TAc?Nly$p)|MQ$*5Ws)@L${SzF^QV2g-KjBJG<8OtSrOU*8YVEIX!&rSe*tjRI7 z)MC>Em3VYdM%Eo+Iqd_zBiTz*Fi<|cBQb7^=T9%w5dgzQ#3~l%B-VviYDuc;NPtz0 zT2wqbpwmi)*ouoLly{z~*q|yv;bl1FC&u|qfQ}mhZGl}#m8%*OU0pqr-boB^!p0t0 zJCi0h(6!lhC(Y&bf>NtJ5D+x!zYpud!o!({j3JA~#q{Vxe-#PV~st0viQroKju#|MW01#;Q zLlO>+=MwdfAPpGT&+djb9x{o{8CLVi6H{_IqvPxXNv9!NFtnaats!Nt3C}%Dn|a>n z@by7R+K4+M+a|(bf*~x|g(;(D;MxFmluJCnC{yv~W9g##zv!WPLMnq)PR`@i?)qsQ zgaVQY;t#37k?lG?vsjdDWrrEkJ|T%TbCaz^=f0!xvu#%{#+#fd&O^>r1GQ;sn9`aZ zL{Z7~ERd5`1eU>+Ee~4h`?-f@8vr#VSd1qGmZ89hUOdxePn~URmo)_v5Wo&MSn1~; zUxPp6?fR-ul>$nW4+Z8CW_Nn_C^AxDXD*|NB^U1VcAs4%w;i`b_E(hDo_pvQO*v;! z)Vh?@>MapdATFdI^f9??P+u#d1?_0@jy%=UmP#%v?`LSu?+fqME2v0S0@nl-= zFw(n$JFHeukdeYBwA)NXucU=IdAUE!HrdAE6F)uRX{(3X>z=ib4N10$Y|ullaGN>` zlUjN` zz{1Y-$eetyz^NLMl$wu z54&Zca(LZPAKB%_I20(LGs%jk*o8K&C`hZ5Nqy?En<^w|_L30s+2=zqiDSyfR8WAd zE8fcSAOx{Z-kvlA^7hoc&yR(iuiJqW$#aMJRF%4V`sOfSxV9LNoB!%^U)@AAHTn9(JEd($Wj>MbSh) zjmHpUy&_Lz2RwwuEhfboPVNGGX!bp}QD!sR&6!zFH%;zp1K;`Q*(DV zmg^ZqOcQOmCup@~4fYj@Jko0Z2eU*=3=8!9{~5iK26@C^AW2ndxk z>Gl^LfR9v_ElA*p0sY5bb;({1ATFGvq5{rT6e=a_fHJZ1$)XYuZ55;azP+Ox;Y_G8 zKPZ=W_7<|i-nu63tpWKV%YjMz&{l|IC0o+-!!1#GLQd|40!nS%SpajEUNJA8(k93c z@wu>?CR;%OG?_e_r4Nx*BBZT$iG9EwSWg{(6ZITuq$IP=H}1nI=Wc}N(7y>YAg9se4EI@#(rb<4PF9s5?OgN?0vy%c;KvX3& z7_mvHSGvIf`4}w}0|1?cA>hWc%7~yGvqW<40hKb-uo;}BaHK%>S2_jq(`SF6J?C!P z3~kxTpSgas9te8BBx$8g7rOH++J^iWK;EjeQefMrZ96dhK#Kzqv9N&|F&d{bR|q&t za*`~{o7JAon8Jv0c)y;-i39AI0#_Pwli=SD0vPr$6Bjb>~-q$wBw z7Tim`JFgQk|x>H_q`|E)^IZ1xiWwvl<(!4t#P99Eb5^5C;2M5Kx zICB%#69BJSh>C7R*DbQ>;J|3rz;08~2l*~nE7V6&4C5xHK^7ENMf*DJfHbnt0hz}! zoGPy<%MYRAEG$-X9Yab=#q!D#=~Hx?H#QQZ*tdYNu|it(CzlNCwrHY~A3dKrf@^l~ zB($YBU!tQx<TEYIctuKelRQpSeW$7T-O&W*{+FDwb1bM?q3kHfvQXz;+&a zWppgyC-J!VlE)26tgUgBAVd-Zs-I+U&?i4V7Ny4X5&=Uf%9v^{ypVvmmlSL)s zgpO5k(4~q7F9vg~#xZaQ3NZi^=pk&;mfW~{G~Nz$a)+O(7)AeVm%ZpnFGT>Smtpz1 z9!V@9;7lY$4Hq)dd zD8Cdw=Vk%|ECet$8{=$fek2aWAJnp46dFABu&47O;&~ni+iS5K3y_tiYeCImsc<*T ziynFpn!g7D(s}B|R*Oh_LHGy!rYoLy#oKix#JXN&Bu>OdNeLhjRWhmReeCcXM=_~l zfd#2%{-5!CH8xLC)KnT)+ z?~?5u@x&M4d&+v-h6;1qL!9L&NP2V{~S z)p<3UP+KvfHWqu?lB0D&4}+}&xH~s0Nh^&QmfhWW?s2D$2GI=RhW=t01P+#YE+m3w zjDodSVqlx{1#{ct!2D;JxYl zrRQGKlkCm(BuQkj{~}j}ApTwxsw}YuG{MFEQ=NvH6d6Irb=jk%H=K!1UvmT{L(2a(| zmpzk&E+&a=aGhtN8rhvOOelR&*m{2eedZdlJ8x?e7Z8Bebn6M=5!~ur$$Z(YJvdT9 zA|ZS@h_?utgEfV=fvyMy7Ius6e*v-CZH3Zji5Z!hxAuJSvz_WJ(>8BX^kc%y;BHv} zZPl1oRyhlAM&dXQpdCImZJd4I@Hqb-M6a6lbC{jl8z?&9!cWqHEFSC&Hk)X8F)zgh z*+1l=RQQQIU>ZD))0SaOs9_|bv&n8KF*qt|rKcWGi-@HmKlZ}2SiotF(5uBOO<_qj zy+L7?Wea`+G8H=O_F&esNYk7G4>I7tVs9-Hakr7N*3_%y&y?| z1x=iiuC&ca44{aPM$@@iGR0>YNUX_1TO$b|9Sx5Xq`f2%&dJ7n>R<`+WSjA`US@$R zK8_-rVjcq-eQg_y``s?oGscS^XCpyHll@6=QLT9Dl}zGC!mI>rh}A5Wq)D20>6|X^ zz&RsEHwHQ&+kal?)LmClvhkyu{QQ=iR+t-CU|(Ib>_Xk1aIng98lJmkztBGB5`GTM z6{s)KE}zMV8Lr`bieWbKbl%|e8faCee#%zie9U~QU%Zzv9s?#Tc#|D|Q=LV+TaFui z7~_bFN3vAhwyyk*dQ@TNkiR7RwVS)LcT&F|VnSgNPJwyM++!#&$~^{#H01$g*(eVn zldfnNm^~_xO=-L-5+YECu3A*MkQ6T?S9TgK>}OSM(niIMrpv@+0#|EQz^&6GTQO7I z3)uOj{D2l2LE=r07yp}BT-}v=)`M#g% z>bxd$1}stl>^hN5i92x`%HbLXEsFxC;*5`b7u#yV-sS^2Y+6c@=IG(KCp+3AXI(6P z&hDEhOZ{F5m1Hn>G${zZ`36w!S(+}J=gl-b0(IE>AWpCI8_9E#(oz`&MP^{^UBX-E z2eu|9tTklPX)4I{O3LFUtT(HAfs3FQz*$YqGK^(?MNLjUDvzt+oTLtj{?nn#+ldqD zhn^v3&aGht4LBVjYjRxaV^;bk7#FWTNDp{g%|=i;ii-G( z-$n_9@MLum>JyQO;So7*GQ& zNi1|(G2wJizE24^=?fK>KZQiv=?FCo@(AQMQH&Iy5#0uVaxM!8k-HtJZh>h@k|t`O zMMq7z(5~o0e=dHq{6FX0&&OdA*}c@!pcc=B@o(}XHY6eDUQH9_8}{Vpi6i8Pg?@mi z$-19SR1v(FtsK$Gc5Vm(6yckwGvr9ALCJhFQT8bLJ;YR zxO~1oG-aQeGZslEBNBs3XvtK`ct_O>HLxs~kDV1AmTWwoD!<-@l%MxFb0DD)3zB>(Q%clRsPFN~JosbiF|E)Ipo`1Zb(-X@szugiEHMCRMUCk?j(Z zF@$d3ye=g?Q?G0wYR(-eQ^T#rYyp>`V)7K(G%v_GvjqO^C-Uh}_*J4Qva9W~Hwb3t zH}ZmC>>xl$(;kKBewIY$V`YrYu4Es(v4DgLW?PuPCcz6!vxX}e(o!4N7ZTQYmU=5YK zb92Qr1?8y>x)m2ryw-_e&N#+ohn(N6$kPC$o>&?NlezRD`7(=Mu+p*e1KI;qc;2Y0l^^)g zWcfZ(LWXiRRKU1Y zZS?@4XdG4Sr-VSKZ$r6z#A@CFm<6(Y=l0lkG)eZM0jR$x(mj~X%%8MlWY%EEh=q!J z>{8UD`A;H*(FIW6LDIOo)eH+Otgs%1l%zc{S8>*Nb~vc&P}in7!XxBG5lnj41fElG zr&1@{-6uU)DpvYaDQ^QYnDGO$WLQ;39VF7ZjII0x#m-(08li%>xFftp!@xDDzz^L1 z0J~dmw&lmLVnk7S0RBeh0kIz5S_w2@2a#mX*xWj4qMyIP)>wgLO?PJ=xUSeoHjII5 zZpKJj3_Qy^OkV(DW}16u1Ws$E8eptSldzcDN+DS%i8kzz7OCK1sa~2S&GI0KQxkkh zgm0Orv^mJDYH?ANNLHC$Ea?t#7bw{5{1ASTED&Pj=c|9CR)arRqd=p~D4WK)1G_5Xw^kDXIw2xw(Fs%pfqh2qpU!kW17V5x86U zvX3_z{|RkXbZI}j@ct=d?k2SqBdc20caiO~)twz=uO@40dZX-o1kSK1bs}3W{fYA_ zkYno@tqdn*&uoNj7YgEFlSB3t0#V|m)U5z9i4-AKGOEi`dE$KlwC@L`Qb_>pjQ++h zvG_q<1aExRaH@LhJQ5XcQeR=)DXmoV;}-e7_-^t%r=)P4wC%YT3AARJ(_lnr0@G9J zN<@N9aTTu#R3KzWp!y;NxLlr9HIYgdJ(;o?^+v4N2{93J1zv)u&3q_m^kO2f%`TtV zT(TQ_?{#2F=oE~T9gvd!9In@@y3LN!D81Rij<76ynJqmI4EgLDw(dGo-ctvpxxoP` zT_cKlRU%-&h~#TQIMl84icHY`TpU*~sZhag2WVKK|0)p<%NUv72iCJ1j-5(9DJ@AX zL*_yrmvs`A6RasotXAnZlZppPoz@9XSoaHpasacKF(uIP;KcyRbEyDCN~{JVW+aS$ zswCi4mMH7+B*c&#B6FR)v=GM?2C5^p0#3hE4aSsMV7X(O%AQt_NpgxrqJnsOd8_2MK!k1iuh^xC;PAoaZYoqUk+{nOAGXy9+p zr4H>o0cduUXe0k~K2%P)gzV(6Ww3{;>;_cO0Or8boGeJ)KXD(TjWcdo^-r)+cdq1yI7CCoG)&+NGBv;6nbR^X` zCZTRdyIko*x04lz+X)d0H7DtD0=Z__g81%vM_^hCG7Su0+>zO0N@ez7e&*QLD=|$M z0Vx~V%!cLoo^m}qQ;2{Nkb&|8MXfla2Iwe2W-h885z3(mA`j8U+i~Hh4Yd#v&J$ zj0oo0nZZhvFSaKz$qyrA(Y7mIvW0OnXmceRK8q#My7PZcx|`o9o|TM{WKJ>~s^nk! z?cL;D5NJYa4H;3bdF>m)-KzjM8PSF;M;Hz$zX1Y7tk8=5sfNqs3?z^*O0$3iHHJ&e z^;-C7sX|j}Tk>QaoXnwVc{y-84F}X{uv^gkmI|8ig&}eFB=M`L`-KlnD_f-7h#ozPeYHXtGu* z+#U&A&MBd+p54f>;iQj?8S2zUVH+?_j$Lt6J(Hrp6C5;#Zjn+h2OG5}z|tfZIAD$z zup~rhwj6*CXeC4p#FHJWS@Yb=1S0&tmpcuRIv6#;sOgKVmN#AsIMdgkP(PUzlW_tcgq2J$TVQ-aHfsL zy3o}Hon2b0x7_rN68p?2ZY#4~lnkE5)HG=%xLr-WvYK?Ta(!YUo@bAi8f(5ZW~b&- z^Me8l26xb6YQBG0BDX|-#jUWIfs{E*op@(BGpO7Iv^eYM+EQ5BU~KZlVk&L8%Se4=hc1j%piO@iGRLRpRi{ z%##|sy6V#Ughi?cQBM|7O{C2&XC>x&;X#|`ePPaZ1xkb?MJz@)Egl0>kCvK0TQ z@C%h+&y14{ye<7x-1w-1h`OreD)m{@2{!y7K3%?xe%*^{nqGNjA14lAXTpJd0+KHo zvP1PA2vF*CZ;umDl{meIW40EN1Mi6x0kZ=dFed`PpdQlPT!_-z6*xwCoJV6ijYvhQ zM;`Y)1L=eFmRZ&n9kFG@kbDv+Q?V{OV2C}?qk^rPNaRU+5H)@MqAKqv={}MSMfVZ; zn@=9nHcDS-$^{r_>cfNts1GY;SA@V6c|#U=g6C|3{F0m0!B zI<8_D0v!iAw7T^*taVq9eXA9fE-_-r{}LG(XGJLPHXYRq zg=t^~@o^F&Ke1_qoO3>VxR7OoY7DeGal)8meXkM&aC$tpqxJ?6li~^Wi=ET#IuNm7q|4GGokw{mF z;@OM4CiAWX`ltaY#o~K=-dJXdvq)G9;f0Bw5_>*yGcfP&a>4%=+O**x*v;QqPnnYZ zFx`fwd~`TYFA}IfGOlqy&+J2d7$>lm<;=}Q0HXa3qy0&Bo4=y`a=yu$rV|4#Mz+gAa~!~-l~aFWgT?RRBR^%kp_m|2Sf(gu5;))shn8c-6+a76egt8z$Zb3bpWX5kS)sc-HfIO$E zAE>xVCA)5$K%}NQv}6s-+9+U09KeXuP430j1r3(uhaQXivb3fqdwiKC3W`~auBsUa z;7m?Q%Hq_2z%X@~#MM4-vI~?GEHi~T~0RRFN#S#rY8{FD0ELq0p1EyK8zXZhr z;T)AC)uvRqJ`vafJ*U83L#HUVKW1iBMrKaapBiYmw;{NJah1Ew+^_}Q7ZyFGO*=q5 zHT66V zn%~-#6oe5y7iRJ26X4Yj2ybD6fU>wHoWf0G8cf(#nkcGvgQ%Nyal3aguICb#4o+b+ zizFpUCX&?gRFKgUTId)aJmxd!e-%fT>;Q{8)+>U4m1ti!x7`$dXR)qs--u5dvw#|`) z>ZE+IRM?*Sb8RT0wXJ5hN1A3%4+yd9VJDtu{;D3L&P`ZS`JO;<8o3f=Gd4;tCrjO@ zL0zuaj^!(V4GuvaC?Pa)66_K|PSYV@tzQsGSs2z6>%(iJa8RcZ8pxVB%9ZUR>tHv^ zlHr2*I`rCglnA<%3k8AT4h91Na27Zl#IF&z;2`rTFT@GEwx(M#h!A5d`XvT;0dKDT zD=1@?nie6lZZZp4O!&$5w1tyxMiiBU66MLZ^rl!Yj7k+iFZLSfNuQslrIA#Lii>P~ zl{GrAk+YetlN}P9i#P17Zl8Z)gc9k}LMfMeur#BWdI=iDY-oCVgHWd1IHJVswKP%d zW!#g~naiXKfEBnL6CI-fb3i$Pri7G@Yl`ICvj&h03L;7k$tBjLv|JS;@pRG3~4CYGFnyJjGS+*bHSBbU@4cIWR1tW4j-? zsj+{PD@YQ_&|DphvWO?VLB5{P1~Wqtx4mk!pif(&(3TPNmY}z)OdXP4F=&Ng*BjZ{ zpr22?pCpwDtfHzH3qHM&o76b92~BgR&}9gXAxK3Y+EHr=(P<&8b(2a&NP^5kz9iOc zMFwCVMf&2)RzXs0vYI1QQ`Yv}hvI6F!l`f4d&7RxuuZBIS=B*~OgLlua*Ir{5|POw zBohKfm3L_qeZEyT(Pu7)RdYrVQuzdYoe|g=Zvi>ZtxPU=NzkV~B1?+00$H_nF#d8I zW(lem0_7ys!bi^zSl1%)pN}3!irKMa_nauS7w4}QQe*JDTarPN>{R4BUzKQ+@~hrX zvOp;t5@^amr}O0-bFJGlM%qh4&+dW_zoOn1$Rc}D9+vgOESlA*F%~+yg9G2}143$Hcj?zS4E^M&l57ylbV4|@w`?p5yDE(50B`!I!bCpMv9k3rj1`JT*LaS!I`>SlC3!pvvG-n>Rpd^lQ15vzl-dXM83<@nhb>&qTbCZ%m7N|y*VV(2n$;AE z<4j8GonE{6W%LZDz7Ao)3Z-bWWmSYhv->!)p2lH9Q4dnJIsuXHIZ&f`S6FM4xl|cK z3AiBPnvx-@=@ z=X)V+YC|SB^r+q=;4Y9y_VclVW#*>3n`L1lFd>CBqIE`pKK|;3eCo9+TJZ&9&jZ;O zQ#J~)CnAx~_?E?~UP!bT_-ML5?mczjlfUO|56@3a%K2&~5Fn6Z2oXJE^^h=6JX5-$ zY}FE2NIWhF3#ixGXwfSf*s(fDic%6UAUy`JD25@^lBMVebAZh{2P_Fg8aFaA1(G^lH6AfICZYG-nxe3OY;fP?c@7G1O{?$YODhn9$4UR-RtK^i9KlNNRmlS#U}`9!uX; z#U7LnK2dr`z6IPOXOd8>@LhH$$(ElU0<6TD3`c`V7t-bQN(P{odJ3@|mkcb2Q3v91yrselll`L4Rj#XL+Jkuc`!g_?zc7p)#>H# za?<2&)vlT~2>l!0`Xvr-At6GKpZhL8{VBgiaQ|#G!TqC?1H#5+&&mmfP9&tSuzoMnGI?QbcLuz?4$5PpdY$#-aH2B!VHbvd5p0!9}?x`1hoF*nbj>gEG z08k?{KE=B>VHoGDpLV>fsTU~SL^`%dwzC^iT=dIq6jkYXk&xWY++8#?G*>02hgSE~ z19L?6Fcl1x$f{|0NN=r)(saP>I@vh3ojT8igC4CUl z73ExCOo@3?$4ZY?l01ZbW>U$|pU? z8r4b4Z(_@!p(t=tmQgRdal+?-76jv|HQCB0=gU+v<3o?%cril)4;ZnO^`e)w4usa+ z79i^1hBP#)?P5ZT+Z_jCUa1-|*lYS-!~}+YQnyUH(JW^^Q*u!}BOzO8L_~V9Ar-A4 zu9`y!j$ZOTIianTfg#-xccV-PU`Rcz@>MZdRu6ebOcD@s_;Fm9D7i{~FKIK-BbEC# zAnVrDYY0{g`LnB++-Nk85L~B6l@K*OhP^TZu{C`4M({>aXax@Cq>G|8gBOp;ahe^v z(lhwkHCZhVFY`^C+;z_pCgz@#&wK9Sth=Q|ZI}u@_eu`iVV3lp-^ocsL7P$`YmSSN ztc96zNtV4c(XZzYHdna|Xg`lg0LQ0cKrtxe)JaMWRryGFJ|6%wi%A8L+0ejx(F=4G z^^$3BY*KZ=AAo$+Z1%mW@Hqz#5GIW%c1U*yG(w`L*wsn>I#%dsvLFYXQ*fS#R@8xPD#)y$N|N0t_H5cm=atTJ)k(zLym5s zB82Yi_Oa?jSz5d0`Y;L6MD-3t6B?7y4M1E27Xava6`1s z8tMUmh#o!9a3fftlNcxA6l^+rRBI-B-wIuZLz6ox-(k?{Z0d+$3f_J`NcvN5k6Ng4 zoPE2HcaW|4Ibs^4`?j=%xu}E4t*3!EcbQUy>d1ws(cLhc=BoFWiOyY^Kx|5YOEG%3 zgV5TvL!YM9CuMhJAJ_zRISouxUMlt%s5{1RvU?D?n_a=-MKEJUEg|bvHzY1bGHi=m&=Tz&J77n^<)Wo6ir~qG1}BfXI%gg7yh2 zIm^^$TGV1Vu=7=-SW3ZgkeY@LQpy2IV75TWY=%5`mYy{m-4{Jpv4Y8Yy693E;nzM%><9L~>S zIe)NcRZo#pvShx=N?-_8{XNIJRPt(sGerleFQ{Fh0lXc3dxZ+SSYIfoSA(>GRN>lG zYSRI6Jq%330efW_PM@>H{N}Cmj%r^GiLW(`Oe{bGHXC!Z7c#DR*fo50Ku?)cxg(tf z&*LER-e$dL0zz{kp|5)$AR?oW1KTj0f=9Ry%{?eJ{Mt&~HQrzrOe@g}_;WH_(f~xIzzn9OOmpUPz}rv938oE2f;{XaJGPB(cltCF zs5d2RYwDuZG%m6+uekKb>~1pR!Olmz+Y_A6}y1iWcKXK zUO78Z>_GFqcXtE8r-Ni*6%4=wm4fk7GW>A>Ll}^IGL`f6=i?wsqv$gT)zO9RWNA+) zC?qgJrVABOV5-RY%WcRDUtF+hc2)OCq2>reW0|}I2YQo*Nx>UK2R6ZBmO22_9z^*S z!%1s}gCrjjYbqLMS!^K{Co0qv_hd3ED}F`wO(kVBPltAR__ATiC<5^Su!mwnBi3rt zY{JiW$vyT#-Qw&ZuL}WqsV-5A3@FZ1 zn=%(`NNE*OPf~|70=b7B@R(*3xeEL1*ncK>9L3vIp%5`A;#`}pve{!RK>m2(UwnIP z8+wE4z4&vPMPqO=Wp`@&f(kxQ{3NaoC@YlS&Q@I-YT`jQ-YxQ>;fPSppirH1bYgR= zr0Mcugt(!WF}=sM9=*khruQI-Qx8zc^vL^qFUaH>jSnHFmuZt&YSoAqRO9GS<#D*e zOK}n_S_dVQ2B1GUomX+qR6 zd4~z>Q&giD1b>Y~N>CgW`4MQGZ62RR70d1TlWmHi^yu$SpvValrBaatPZYW8ohe$_ zqZd8oQ38venvzby`o;;uWI$LjR3Rc+h0kmQ#E^zTPDPHEBgWKs#{Aah&My95KXb8& z@$6##+g#P(?Q@4rwG46(JPM&Bia0LHo1zjM5^J?6q@<$JUiE;7%=@-vWqxyaCiF}% z0N2G+Yv7BkvvLd=*O#c6*ti-d3zv~*d}BpmBxmps0v&5sf~uoCg=#G4KQ+714QiO; z*l@*tI5zUPAlhKzhUX~Y`+%bacn@5?)O(sRZ}^0wn&9lxX623!_&u2`aY;Z&&TOgg zbTo4r#3Ke|6+N;rPU-9^c_;)`)&Eo=Egp?02skJ~1vJI8yob3A)G^Lw-r^Az(kqDGCE zk@Wx6V_1)Cd5UGN-jcI*kO1V$$_%HnR66df!Ky|H`cZ zB(v%(^Aa@y7$C+jW3LB7OhjeJz;Vlk(=W1Adpr+$&{n_(O$SYV94{2zjT*3Y#G zUawY~fvD}&kJZK(h|3fmnewC_3#bnBUS(p3jfcvRiVc=h&jioyxrg0%kJO(2d?e$; zlWd>jn>9;op{oK$7N8hI$oYJjT`&-f ziwnrz1zA`)%Xsc(01(+gpc!lLgF!BuQHgL_F}5n3u_n7QQpenQQ zw%cRg?H5Ucpa!>KXtfMggl7PT|q|Wc^HIWq;`W> zT`~x*VW5_wQ9#B>e)>d8;7$ClqunPYap>Q$ZV&aMns2?(O~t)!IKoDltuD1`9guaB zPM}aFWf}9)gAFe|t}qatKmbT}H(Mt%EL%j>WY=jVr<8|O)8w<)gp?y2q#nY9+BU~7 zg%mhr>}z_>#$9KEy{RX=zj0eNP}mryu-`}Z6-@5)ftmf6j|+ddPO>LFoe;DnrOjW# z2oa`Ar6lvFXp-DZXE94&xR$yo=9P6k_pn#WCE%X^9P|fhnX@u(5*MJ4O}bB?a~0Xw z)CdZNZPa27E?36PE# zP&Z({Ks0cBmYb4nP=iw@U(xlijE)eLU)MP#%*s>x|S0I~-R18xeO6VNkVOodt z;4cWtpe<9{h%bi8F3JDF{xbz^_+u}K0Yf6jBIby%DK;t| zMZnjm`S5KeMnt?lzi8W?Dx#P$*Umta6e>BuvXIv|A!mb0g5)-IkW8za+yGsbP=HNJ zpTovS9#%cbNr@|AEEk9~`JTw!V<_nlH$xPF^z4R!~*&pdDOY>>H2 z@|uYgupbj7?tGHTgqux#TRxQ2h`W}N5g@>XuJxXi#Yp2-52R?N41M188fW_0i?>U? zAOgwC#xbmTrYDNjjy4l1$#mYeHs0ljIM|@hpFheHgjkX)Ft~d||MFb9gc6I^U*RJO z*Mzvg%5Tm1Mb%QJqQbGbcj33QvAj;?g)5Ku*w%w}>}dtnm`}Yhkx`TkaYCwe&3tr* z48)3In$7JSw*Z78w@2&eZ5w&Ily!G0>u`rnuD64$&=Puj3F4xK?#y-4NnU1hw4|Yn zobU;L$o?=DF| zlNLNr4!PPN#o_!Z!9Jqg{wVGC-#A#p9=5+H)arlymXgfdpZ&YHzx%I0ezN@b=YRSA z_y73zXMgdlzxnkKKYaJc-+%k7fBN>XfBn1PfBWhCKm3nxzyF7C|LV8D`{6(S@%w-N zgPrvCU;OQNfB5d#KYaJwZ_g>dy14zbSCHQR{BOVe^;ZLO%J2X9?SJ^)mv8^(yFdQs zyWjut?f>!J|FCh}+n@jH_dmSo&42y+|M|!7?7}~P|C`6(ytronqyO^#AHF@m^?&{2 z_rLe+?mssE-~92rfBD^a|N8CUe*drk^#8N#r|8 zw}18f-}>M3@6Nx!{ZHTj{%_7p`@?x}fB8#m{>$(F+xdm>4*cRj{edIPj{M~x|M{Q( zdyMmE|L5<2|LZT`{q}AAyI-WYzc~N-yMO(K=;Z(W_^Fs|%|KtBXP5=7V z-v0XS|N6hv+i&^Hzx~B8S<$~?i;1$++m~Pbl9hp+@rbt1U$^8A|Bu>w#@7B~oi{#>197&zatR zfDOTkQ4xlIdpi54U;KmqT-+wLv*TXHFAJ@LKCJs@>umP~ES|RbA~=m`ej4Pnk1oVn zivy|o=;VtOJ!9F+-rE{UA6a8M8-+JJ;i}0;7O3izUk89azYapNxV$rGZ@X$0vOoLj z(rqS};-{Ns_{iiR-(FsC+$J%#{N^KZ;y(o0sOrrW$KI?_Lc>xT3M00Kgl~ijT%p@86SSkJj^xy-{Q?YaH?SAAZ5(r+q!}7k_d~^k(ZG zKkV^q!O8yQb$#6$FU0yrT4#3uDTBm)T=K1<`Giiz9~^!(@f{E^z1a{&D(Pyh)!TBZ zezgT7uup!r*5li_&zSr`+%mQxZ!fRw)4kxX&`w_xQN|Z55`nJRqpnJvXx#QCM>hHddmHaADv5R+m zW`8A+x_Wzg1)uKxLpskV8ba%sf3|iW{L+N4OUx6qt>{hqGn!W~K3h@2@|mJ!dw%Ff z^|zO2^UFYq@P3eT)cE4fB}X%=J#W?slgek0Y+p2uBl~-W%Y@&dZL5Rzt#`h|DP{N@ zKG_rbH7FOTFA{3w%L{v4@rM(}6*u#H$$~8VQc3T<^ShVX8(Rz^AA=wHA@@7$aqm$R zzAw@CtL9an`@Z(W%v+?XUvKM&F?`l3***iC2YA)Kc(c%ywnD_DLlw(>UU@3BKAJy_ z5rg9#7yZvjceDtR0E0BhBELB%H@VEV$|%(=lD0RZ_W`|Dqj zx`^4c;_?{-?6Tqt*w*@}dE}TW&#is=6XG?f?+)mt`Qpu$NW)*;x&10i-QFmnsGqpB zYBqd6vCsG!38hu3O?Ouialmbh_l{jnpkqbnrvy@wydjCRlJczrGF(zW`6Pk z%u~b0P5VCnoFDVAe^YFhXO6~R-(FtWPmUxnV8 zJ8E1-zmwL7D~Rc0qp<8F^W(mFtEiu9Mf)-4IRDMl!fNX|Q%n`8#NP?Eo3S!|r^n4y z@FTi~7ytI^0)BJv8Mr@hueCiU&Kg&C0;_x{&Um{AdG)vxJ#BxUz1Ntjo12S~z3K5G zJ)Q3J$=p&zx-H#@7=av2W87g^Vfqs<~&(<+caM1xO3k2MkyTn1W@DczR$Ya z+HZ|x-$@~YIGI4ke23Acwe1y?;8=0Bbj*cKR(gD?+e>U zy}v8nCz3}@q^Z=8-xs>pal`UT^IqAB^&%9CNjJZ{xyXyYy}W?9GMR0}5I_&UM01J{ zqUXIS@KBMxN`i4lu9ge(_=pNnz3J^c}LW@{gW4p)tBmtT-(bF_;hbMLdiZb zo8$ZH-XJ=4+ll63W3A+5dwyk=s9!Dg^l3P`*K694PdvgL$lPM`K<30&|}Ja-#925S>qV- z`E3NjVu0@Z&3U(g1tkP~nWmKLtYhxYuL@1x`}d;_&(Hu1^}go~^z@PZ_N;Mw97gT9 zJ$?b&PCh{P0TA%3zH!UV(j~=HTxwinJtn6Q&){=3&*69WepSOA=R3t&LD+sq?RSb>J^Ih#y9vr}5x?AktKLYCx1>E{esq+r zxq7awAgx$@BbNa5S|fUd5q8XSe@MPj!+z`!N{Tov5@7sv08Z-ki&h042Uk zW|0}Az79~@h>-j)-U0?PffK&ak4TwcustB=9$|Ks!sJGERgrzHp zT}Spc10m|iK5NC?jKdJuWT`L?9|sIkV?iLx5gIVD7i0B z7%{-(SCp9Z-|a15|1_{2-qqhac7QzwO{?FZwqh|vB>r)4>dhANQ0`98BUh4@KFy?e zJIm5`Bmf^-NC1;}+~Lt6^@SA!z*_ce=r@D-;nu#7!(vS*P>wJRT&ge1WK2GLbM z4-$)-KK7e~kNOSGHEIb6-S{wsQo|Qww0~~%he3RFggp)2p^Qi5>}4LV(MhqOy#Cpn zd+**X6$)>(dO(p7<#>BJuHP&WruLRA-xX{0kTwE8pM8E?!%m3csRa;h=(|NAV@DZ< zA-vr7Jg%RxkAogN7nJZ)~F5*?K{YkX``ihV(}jhn{9i}#cN{aWQNh{NxYhP`WI z+A(Y&cRNMw^YXg*_)Gv zCrk@IL!2SRKeuVg=kMon{e%<9cf2X>R-{hzunZVUiyRZg1pHlYv%SaYW!m|&6+d4N z>*uR-&yHx|raCGlrl^!7!SiN~BSk_}wma5%CjMwlu_a-EeWPPv1?0K5KE#jDSMd1m zqHr|c6>?^Jw{>nFq0e;p$WS(Re9`${ueM;HV%(cj9Uef?SR<_S!1DP5;->X?MaGo6 zf1nlL_q%az_9f9+BMF#hjz+M9F}cO&qxPG?A9tFl?OkR)T_?ZVY!H)#F9Y zAmi=jPtb2`2^EI2eHzSob$p5Dw(dMiMavqe2n003@bpHDT9vW40;4tCM5X!pb{)~x z!oF7*;8&K@tss$3we7!}%Z%dg^pUfanQ}DpJk^NAt2ZXSY2JJ{6}D5i0dC59sG#=h z3L+7W8tGlbZUJu-wMY7h5~X+b7IM6={`k|&A!Bh}q!>IBl5+N!X!g7uQ-$w9m{udh zJ|jQFv{|EY`bc5`-wU+KjNGy2P2Vv>(0+S)1%4MW4MX5HT}(L8f~*Bp%ztKOIL zQH1u!PccBf19n?=~~cDa6KaLMwW`XZaNZ|UDF;qj2l zO=>0Ij_&)py&v<#=^xjXFHi(Ol6zgrpC&2J`{txHVc!;ZL`KRw$m6q^D26$tAq-68>=?`Px zuCPV?sxEltOyu$RAAW(qsIUaQTp1C5jl-%#_E{@G6ns}qAcp;Ol+UMd8Swa%d}U9#!|d*?Y9wg z6>n^jdz*Mo7GIb5w!nKzw8n4ry|5I?ARY-R!B2OO)ki$;6)~1)2H5a)tMwaqz zu>azX!ys5nw#08(kwtiS!}xovnb4RGCCA^d=!y0C9Uyjn$x0rdz1fMG`V4HE(aGZJ zXjP7rbsrZ~hdXYyx-xHV+c7{jWCPdWZ|yi?e@p%Xh(_WHyeiGfu$Z>Sx}y<$0&n!nlTrZn@e{P$d2F8S*7SEjq9=>1Y4p30 zEgNTzeRRLtb>D;@ct9lu;?KPpB}|DeEW@v8&B(UGU9`d-4`3RC62&hAhjR7P?=;Hp zsl`dgV*pR2$OG3uy84ZoDXFi|&!;gz=I4ID1Ci|g^U{K0eFZc0M`gQO2C!H{3QB3; z0i>4~Q+&K{B@*=dJ}tRAE)a{8d8|ici!Jn{=!BBajK0HB0rV~LW<*hQTznXdFooF| z;JlID7n$yF2A@RS@IZax>bK(aEXIJe=9mlGuo4AS{1$V;i1$5xvr{zgTXvHdcuZV+ zN39Wm8Gj1RqO^{reDZ(;ffUbh{AeDcJ1$F0eDfg!KeZ!M^eXRR&GW|L-BUsmQ;QN` z%$y`QEHDwaHQsmfG4^^1b3Qz$2bIXH45T@841uDy4H2b}A5G{~-_-&0?Qva!-CBWU zVayqc<;v<3e}8t>HzAb0Yi!b%m1KQP-*KFKHRgCElx~qSKXOOS3%vyBBW~>LgqANJ z8rJ4$-PhvJ1%g{N-q~DHV+^qJjZk-q0cK4;>MlEe!SJ08Y~}j+1;Z1~3=rqSx;?_y z8h6kW)mppc z&M{~=?uYgV_s+J+3TR37*Q;J=t#7xjF))7nvCg7(bSFND!rpbHXk0zVprggF5>zb) zsLr|Bk6TzUtc+2IKbm4E@ToUACS78SvHXlcOZO5J5-)=9$C3LO5eRCQvBp3{A!Hgi zh8^DUdjOw1{=Ubqu=#$qrvNwcg;}cMm)cLYIU*H=W87zN?iUdw%Q|b!EjC%D*evto z8%xh7@y>bL2qx<2@p^gW+*UyF7$80V{_!TZ9p^81Mvc5Z9ZNi!SU}sd+?-wMwzU$W zna7SjM_ci23MUT;YFhybeHS#-x81F$FLX`ftNy+{c3O=ZY4nx)x%aIR?X_ty5k2c1 z@g3d)rV&BxXYs9n_U7*3g?&RC`i?to6bp}W6HYf0uDjlGi)Ebj)!E++eaonu*Eal@Emb~KKNm>NL%?yH^Uk=2O&YCJ}p zP!ZklCF&+MO1>B*dzl{3Z~GVXJMJ|ERX*HG_>ZDe+q#2=mYg7#U z+50{U>d8YfNwnXoZ&-jQMe~@EI{|rIY`I^k0EE`qM#R0CxyGnDX8L_0$5>e!c`r6_ zKI+}xHvmmQvcI3@8mg5owiCP@F+eqqAvQEO_NIBy58*o`ZJX!RXytqweOJhf5KBd? z!RS+zT<@@QrSXN$N-NyO8B1%x46xv+B_gc5wQX!@V_NszU@K9^(-dJ5T9edBkL5B7 zAPhTh-YMp@45A>SZ5p)tiS z&grACDj_t3?&|*c(?gC*Eo%6$O3t~XYIrvgX~iAv@4n??Z#=a~k4&PxU2#9SVbtAM zs%jZ=w7a_MZ*a7OR8;F_7D~A*enEP`m2yl(RXOeGI}AC7Q%6YwX=|2>K@#)Q0hI^0 z0oXBII{5M`cYe2vj@?(ymkQ1&vdg`Ks5F8Vz9(BXCY|IgH!f?;Sbx`k{DgTKrGjC9 zj$h`x^92gG`w$V#t((cNC6cYmZC16yb(kgg<+85crurcGR}gt>V5b!^Ng(O64hrO$HJX=6ghC zH^`9!6Gx+0ZH(0(6+0ej_1ZGVf_`1=0{!RxZpw<#KP&g#yiwe5?vABH z%k4TYU)1OSSGDoz8*p5y^L_m)4@)q6KnxgU{AlPVk0uH`; zH+G6Owuiok&wn-F*Phq&#%E^+uK^>N&fZ`KU-!a%!&e1ty`aUrx|Vmm3H{aQXdZ$1 z)>HrNi}|>#o>UrqF)#5&#zXsYyjH*b=Ta=6O*EC_d#K$~8dBN4gxh*o97rN5?!#jx z8#gWAt;%f?T}`1s5Y)XoFC>$xD$=ZVUaT`l#|A_sfWMUQQFFxK9w>=Oe&3vJIUb)C zYHS;cyC1leyW@$hIDnTGjoYm5vx^tRE@aJMc8hTk+pC~jtrdH=;9n_0v3=9$7+)HR zHDk{f?4@z*6c&SRmKBJudc!TjUZa*iY2Q$ z{*1=Us!-3HmzfV5Yl#c+yo}!O5(7M;fW7aygycvr^-CrgchgB@y;pD}TL!p1Nh}?_ zZ@=Gpi-Z7#t;@H&eDWX1H?JKpO_Vp-^ePfbq{1S(#7kcym-YY+R);Y+ z;haWH!}r$E_wvA}|M>%67$OM`{A-YK&g%O}#kCS_^x)K=#@L|3W~h;PjGfGQM+R%t zwm^2WZ1D{F$h(jQ8~!xrOHb%yJ+hn+trchTN+vgguc*b*(SxLETiN*gI)ld`O<-#r zX+5L@B1;;%TV}SKy{0r5Z}N)3n4InRXa+^-99qq&W{NsGV~gP)YJ^8qsZ&x0&#(1*bQa!&^-R}XU){Ort%7N`cNVks`PponY^l|^l ze8s601;zmN#%U2gA~l^N8meCSo40)H5#ewA?_>SpOD}P;QCaCxZs-r?CFZL3sD7pv z{F>##BL(OST_2D7ckHL93D(2_{`%znsNj!1W0kr!?YOCcf#dCA{SVqu>;pT3$sw(j z`z`)PnVGmT)`-3>^=Zw%$sfu_UZC{FI|L@1FDT;1A^db)Y`0i&P!72=AR{mk5hTxd z)s{g~Fi6?e>yvm0`EoLEVifbE9-kUEg?E6FVj|K)|Jj=#S)GgIDR=ocd46kaJWWu)VpoZ0%$}`2h2N+B3vz`a4dT69TPotaEt9$FTiMy{)_90^@XNoX~&(Bz3oO1IwsQhN4=F>40C{A*)FA8QCiq#>@p^*hy5oeof*|pXHLe83%iNDK$hMiMO3jkBrIc6 z;yE-+McV(;YxE(=U%%VNAWB?JJ+eO!uVl#)Ls46y7nUIcd zBMUAw_>NI_NX&2*b{UI#mSHONc0!jdmz3_d%wz+copcvAom9?2ij`)7#Z;IIr1q|_l&V6PDzj+cD}B((LG^@+DNR?#R7pY4QhpKS z#VnkqanNxujryp_CXmr}K-q;#6U^_+Xfr|xIoT448S8zR|i;j!wZ=h?2lD8kz zdU+>!gj2?(yR#W0%wg~1P-T=8hr?koTRFOwk+LHk&aO(UJBhZtQV`gM%3id%w48Yz zWjQ%041q2{LB;f=TO(LBNF(R`0Nrdcy+-Mpk>uUR^ipIpAlMc3wjqfy`SXe*I?{C) zg+YDV@lh6W=xhnP`fK6RA^P_;nn2u;c204~BmtWAB4|=cJ-t(_rr@ovC}QXmyO5M# z5VA|?-CV??1B+0lb%u-6Ru|nDT|x6bx;dYiAlLqTP{ncvl5^7R*#QD!61S zScOFLLIbR#od!5eRU6WGQAvrZ2r+UJX(7-5=vOkp*315 z!{f%p$i-LC9jZ0CfgTiqR!0R0CA>C5Tzf(2!3T`oJoj zsT7_ikA}$#<~uy9q=K3;7HL`8^yVx?4w4VEsM=1{_9#4C%p6J=azJ*M z!(tBd$3>Q4R{d+qNt~-mB6O;G>Wq!ksG<_ zsKb0R46Bq`iqxW^!&E9MaajoyRY~R-lER%-1JX4>t>scCbdGZOk>a9a`sBZmu>rzz zy51ssmMni(=m8T1MIJQL4NxhTsJq3~%Sv{IRFL(VNf~v3K}M1lyaa`nA7tBEblM4rwSyQalR@sMf+wf~m8PRR zOH=ZSGA5zBqHq#bXrPpaNT^>$bc`~H%PD|>&Q94BXrUcM6--PZc^NSsC|*JzL?h6{ zrsFl))Jb#&c?%dFNmR&0tw8rG(^EjkloftoLOW>5oU(#{4&W7i;>*M-R-kG~Rb}Fnis_fpXO|TPyh-(dGK0H= zDp3zA==dbk8&?q0@t^b&a77XAY9V73nf1n^lhH|WF^zo7=w(r`!Ug18Lg#_(@QeyF z#gEQQD?~>j^1!rNQdIC|RxmXeNVP-9tZ~RxV0x^mD$|HluxKHhin-!cFolYc=A|PS zD~jksMWmr9+2KzW<@9LL;jV~3ap-X^EGrI5&|xCn%4u&j;&5ejc)Rnw5eX7CBCQ0TVsEOpvJDZ|2p#TB$JCSwu{zo#l0 z7ABXv|AA9p@FbTMis2V@38$>^vw}rqb9Ouo<%}*Wqv!{xoUWQdDyYJgbI5okCQ+Ww z|Kc!(KFP#UYI&8>{x&pRWlRIqi8Pt=N;|7KrL<3|5PQy{8`!A>LuEm5N_XY$vYGgM z5Yu3e#q4}^l`OMB>Dwnbr8I*ebwqJ7(*TPJ_yw3v`wh@t;B<;EUFZv4fiv?Kq-?V!ePuR@*$Pbnu_Gtw3t=AGwAdcAsnWbF&RG2qPAy?pzSpq|}H@1R0`kk_<~*#Y`qTn>pv)l|qgD+2yoGrH>TZY$myo#ayYwW{!W!wL!V$ z$s1T{7{;M%1F^|?H(Ga5Wq{bseKF_*sEk2$6KWwK5k1M23_vFZjIAIORq0a&x}*aO znOXV(p`57yTuubOS6V4ZVQ* z$QmTH;v(Iw1tD!K$yHAMx&&qCGN6?jmJ$TZnL-E^wD3noOi(Er*1HF(^Dyejg@hPI zRH2-5vZgS-mP=T4O^5Pg+F;PY0abrs(Wy5TWjl2q%9uS?N>zX^XM%i-cV+Ycqyuze zJ~k7GRZ&>x5=zV|q!W@BmP4QB@8n{zOPK?977Z24b`q=+a$r;Vjmfh_$b$jf{CDc_zyTazFV(XNU2@xyV6&qPv!c@2|r338IO{)bNU3B-S z?>|76$RQ`RWT%E{q0F6xmNGFQbQ&Znq>q}&>86BbO!OIpB$2p+J{qIT{h%n(V*2J0 zCNrdn)>veS5{*m9=u=wE1m3fmV^^kF;E(&iYq3_ab_M)0*OVYqQ%(mm?;Ivpkid}o zI=a3!Nnq&dpbu()(ky!A`)BD9MX?e^WegkVlryz2DX;pjbS+XyX9;pE3IsK;aEfN;9>2O-Q z4lu;VWkEx=D&Eob=Ov?Uc-?oRut2f#)zK^I3F&M|p7>UE#Wf zNiHd8Q`N{z5s0LP%=L9hTudLwmQ?I2xnIg=igHl3>PnefwHzkh@u&UT8R25)N=p*y z(<4kdj>+&BdL*g*dUROBrt%Uf-+IXp<6MxKiOwsfFU~06b(%?Mf$W4*{_xi&^!+Jx zS|6E4!h}n)%4p$_x=(*sITPm2E*!Veg^=m&AvWU^L%AqaEIFH5)aVL^aym$)l+7e( zkV=*g4WR2fqVpsA__~y?i}#Zr>{R#TFwwTmB_S0|p@VX!ek~GK&;k7H5<1_2E;&+G zpfm@l-&2J^$eYHf(Dx!H(50O5ApZFEI}3rX?aC=^4tgQ5%IFg4WKayf)pkmimzC~F zf7C|hP7O$AsTZ?ZWlZHvI=F^PccrtN=ra8r6yQT=(3G>7tV}9@kSvl(@0MaF10SW} zFukLLr^!Chxo@muCadO${W`Ey>hp&U(;X)DZb8hXr4%!n>cu4-IxVGm=K_zi@|}*+ z?kElu*p3=4r8g{#)&oEDWsFK>%GZqDcN!5au`=IO5bGCU+Jtplb;m~u+sbJ=mJiw%sA%{(qvY$3m$y7||l+uaxWV|R{F}#XD&koMNUL z9OZd0r7o7Ch)yxxs~>&APBHi&wM>6nDRU~rWNVZ%;E*XhLfyqnQ7t->oJ``N>zS2O zmAH_QDaBb<{I|Y~OPyBlDgHuzt3>(m|LC|I$4=Z^55&^JKWrPQ%|@y>-bGg*FQ5jPahhlbd;fkEf-A(+gaHj!ur~R3C&{-EO z85GY33Y9#XD|;6mv%qdX0i5?ZbX52*@<2j=OW!?-+9I7K#pEkc(f7jLyKD}sgvu%1 zEv|g0W@iiv|0&ozvU&}APX#(jMXdwCI063(Z=t@7Obvmi*oCtQ3H1?0umN_7#k=2d z>VMk2=qE!@y@_G;aS+`D z^tZc7W&!TJd6#IXkBo5VYj!oa^WD4NynBo|=fCao3g#s5W6e$L?TT&30qN{Ag z)UG6*R`B601aw*S>=uE=V3V<=@c%0cRawL#J1xSX(2$7YN&+cqE8Hi3>$&;jJm>$LPfMj@+ougb|n=d2%golV{Axbv9@-YJMl z%eFHpKhXhydw@mn?B4qbF+elBRApg&iwh^-q?nCPzmO1l+6zm7niV80Dp?l;i0uw|F$w&b5Sq`;`$!NS%`ekLvukmxe{aZVgP12~J zd>2X#C@cTbccH_E@`}TS07oZkMPlDt%n8T?mdP7Nq+yCb`^a3J1|=C%#nBge6U+9 z={(Fq}|s2mLf|37~k{R{mI`oA2|X7Il_ zWKT=qC4c_Td+(8*JJ_yn-}5f4#A5b4=Z^jF@fp>5%w-ZMrcHv)33c82nBwf(l&m4) zk~I)PaUJIDJ{Ftc9!WSPrxPA!Gv+tQqjp8(@(6F=`nc%W#+a-sHKXmoT?P^6G)WWF zy7t73`h3ixYKS?FqUBL-!|Nx~Q;Xkm4=q3B9*m9g4<^R>hcLU~SbVbmqqJKuh&wM_ zj=3&=zUk^Qu8-@Q*Pcm_tv>DAv-zx}B0k==H#sUjflcUJGcL1Y;g0zDs|WJxq*K+r zKGFHvn~}EiR}UNi9G~Ex!2RO0i3#q3)QqGi?NxW-KBGM4key9Ti7R7M;^VI$bh5EY z`MK?&)td2Wx|4QQbINIuW`agl(xYx)85JIU@vw!p_JsRcbXZoEb{HjzDP3D^%*0=L zsBgh0dv%+`@ICiBi* zm{WLaI}Gm&>SefJ-HG|+ElHoECF&5Jj5{T#@t~?J<&mDv__V@oz+y^!G%zo!_KaW4 zPkXiEwQ0fel}Ya5q)XX~2b3LKA-&@5obB>9yf-&%vEgB>ZPTe2uG!V?>kefTT7M~@ z#k>h?4GZ*NBSJ&B2!Br>?lq}!pH`3!tGJj?T$}RA8qT`3J-gHpene^~aVLpCM!p%Doo#G!_8Rb_^UOHVK{G$6{)ZS-FdlW4j z6Wxbi-|zl-?V0#++6!YDwwSSysw?f2)Fkb^<7tQZ>{bZoDr``0+GS^6J>!1-YD91_ z?$dU^c*6a3b4+nMJp*gRZC0e*>h{rH=zz1W#@X*)K?Tlc)Yf9w}4Pfq_9bMlX6JnA;wZ4_c&qZsp; zq-p3A?lH;I9)lp|RGdrA2#*s!MN`J7Y{>>q+Wef!lJ=R^u!iI@yQ*<>TveNPYFaT^ zf9oS%AHTSF`n|xH)xVywR{zub1Cua0X_KuF3lF@x*YwB5ud06W`n$UKu?g<})Rf>@ z)Gj{xY;gbI&kP*@6*jJJCwwO5`e<9l+}B6`+lxC@|DLvU_owZ=gE;6K+-H(wUL7y# z(R5;NlZ^0L4O)PTo zW+CRWsPV8#pPtppQUSdr1M3{^g;Y@9GW+QKpB5grlw>2etC@wKtC@(|lAW_%$j$eC zL4-{Dj91x`@rbKauI~NWAh$9b=GVkLEuU^U+CIyMjT+3S)(KTA0@?FpaFn)Di#1dJ^)XB$fS742_E8~?OCQvd#&(S{GA zGlFBY_Zs#Lbp7$~9;;8iM+82fS{v!!7r9yeuZv$bz84!69m@s{3L;|ErUR-D+^-eD z-0AZ{vpyFx$Z?;#^UV{{zBj{?!zZ4Beq`YIKfSsyJWRMO%Dlg~H|CZ$ynIme zo8^afzl%<*n?OJE(?Odx=GO6-hlMZ(pi|uP#++BykaqBntc|s?Up#33V0BD*H0Cx* zUk!_oE_~be;qqP4-sn?pU1CCdGBqtY8XM;wSbp45K6B~xKfipy{WLzK>)f!*&#pXc zE?s@pP@0_TIza?wEtpeQi^1MW!XK_W!;2;^F`5zfkw<_!Bd4V^rJt z=8<^c>I324jR%5Hu?fMEmB;lTguXiQ%lYpb{^-13^Y61?HL-E0UYY>izw)f> z|HvBH71$#evkTS>@Hd!W>baI%uwBZA`>w#=li?m|ZEU)259V(BIO%TxY~9{avTkqL z1M89>gE>h1v^>nCY#~C@^SGaPobdBON4IibJ+Av*d{Wk!^I!Oe2=-k}xV7EU8AW5< zA+1k&6|E_krae1rw%}o{G!1$=zc6q+8yUKj3iR~GeL7J(AgCe&!qd4~MQ0|g;H7-r zz?2jRlj-pE$>h^cx`<3iM;Z~z0R`rVcut1 zue>4a5!Yt@@+RD)Xe9h*UB=&IOM5Mmxjw9{ zPrCqT&8V8APSL68WYio zEjyEP$3E%Bn|5VbJ;Xo zu-CWeWNk@z_o3)?`=0m=?_km?s!D@yk2@6go0IDLwNZKX`mhA{rTEzTaK}e056*wE z{;0DYn^e}le%Qu(b)U=LcqTpa`k~}N^pUFa`R%&jIWFz{``It6{xvzJs3%+|CE+t` zUX6>6#-?>0TfyE-Ihc>NiS|#P->dr7tH)=5w?5wS@%p6bSlp>-06Wo@bf_A$KBI~V zm<{=m)e1ISjRzF%xL?zqaw{5ekESE%)eCYkmU*A(R4&wgH0^8sczv|$*I+x206-5gxrX=hI7YkE|YZbsZpl^5z`ao6Q+n zUDV#ANqc)N*?>Ww@#{oMkB*Ch{Y&^2%}JlSV{6HFEjw>85P*yFOMN%9^U~J1gL@F> zxEAw?YH|?;che=QS$x9(h_$;v^A$wWy!51NPmh50e=J-P9@=u- z3>nZFS)cF}aDp?N)4ap0Q@jHkb_wV_tssx^2VjqwT~UX0OFpdX&cnO25fwiZRCT9) zU{hhNGwv&Q(!Sn47&|WRRkf#s;@Yo6XgBsHhl5!+vm3HfXOs57sBbl!y-Lb@)7F*2I~MreVsMvfz#E2t~{T1 z$?N0O-2KsM{-K0RcrxjeHl{!q;Gw~rb7$(xVQiNkgfl(^Cnx&B|PoF zRsGA9Yw!}{t0@=QN3iF(OVv#H#5J+$=8s-JI`i9?Pio(L1-kpisJY#LoA=4$ckQL| zN$DxzkF_bg`h0X;+W`3J(A@o|4;F9N|3@yQ1)eTHwF(>s_vl5a-IR-WH0kU*7_&E( zJ->VUS1<0i{y8zFX~X<_+45uFC-G_7*_a)44vg>0a65bHyOs~V*U$dj6<*o@I$=2T zyX2UmZF59&B0Ajtadf2fcOh6}!PK%DOj&yvq zKGq6z(S3Mxs%!7cF!1p4jy=<#AOBa^#iPG`8&Y&`g%ur`3%Jmf@W|RY|3Gv?a40&? z+xOy0Yl;8c`uATy;T=u`o{LYzKDSFwM<;j(mmfF&ap`u=f2=>{e){r3N7*$2`ae6T7q$(iLb8rb zK;Dw~OPj#{w-8>yDsI7vgiCNd9T3%IBl5O9=t&|tbQALpTu%9b6Z%YwIM}4MY2ndL zr?_ep_R5BXdj$H}wY_MvVW3~pKGoLG61aN}{%yAON z9Rp06bV+N{K0U}34KbJGbT+J45fPJ#h?vdU1(P`s`T_Qc5bTIPJ7=~L3r6@iiBiMtG)dc-{NasYcfNa3d=@h1QT! zvI;mW^kvJRaEDcshW;hp;xh|(Pybu+M)f~=FI4}dr~k~qtbHd1ydyd7yMF4|ukSXO zYr?T<&Z2}p0cxTOp$^g)oyt+nI5hl{t^i zO!y4as6$j4a{-42UYZN)Wn1%>{xz5E+;jMCVj6H@uxE&Xb&>Xh?U`(5$DQ2c#B+H! z9_hQD4)zX#?p9%-n_;iDrxAy7bpV$Uo+82;aUvjTgf%Pv9=ODI#B9q2bbzH~jlcsW z`2|_W+ZUpSx1j%XL5n#X>b*dO`mUtlJ7BKsV@}EGxJOpE3OIA=>G=<$piA-*l^_?< ztAJ1NFrTCj^Gna+VY4OS*GYgI8M5J?Av|bQW&)}%A_RCoq!WXl)?j{RYkpCy%+70o z2kT_nS-m>tm$xRpGO+6gRR*wP+#xyvHl!8@%o}xxPpwVx_Qf3HlS!nLeFjM!bo|;x z_h%WOvNH|O6A^1~X4YuLL(1;0C8Iev-+K+&m+ZXlLJqKBZr)(TXI0(E&TP#ij%4l0 z1TBUP;8|i8_DaNL0sAOR2NkWc8QB@cjp9C4$C^WM0=PAd$G{gvWbm8plI?Rmpp(U% z;+pj-(TSATYRmb1`V)3#Q*0W>3ha7j_SzKUm2r>a9MT7y4r%Sil=#>t;H&i!!I8wI zvMCePiDO<($LncHW#H@c@4dRm{S^0Uxoe|cdlTc5Q}Izl+uE?c@%cU7sn?G!t*|e| zq(Am?b5RO->C%I9zmH9J?Z=$J0sXSZzeJQg0`_@yT6p5sa0h4QvG7R7(QArL=vtza z^3y9%`3E;fL?Dtp~ zR&=xlY+L)DBw)JLVZr{4#~=b6+z#uy0e9>9(TVPTn_~?hrly9gu82h2JCG< z2pko#&*p^qcw$Q3h?w+L6ASPx9zQnw0?&butE7G$mw^m3j>h<`X$qZ{#l$kXd(&0hS)g}Yx6I3T0 z4aEsp%btXncRUpopUr{J&CdhAnCtyK7wU(p_ zdOGP+wcuW>er>vI-}+R?Ucg}z!as1G-Yf9U#u(U{4DzFe zhHhpfgSYa)g|-0yW#<%K8Du9HOcrG0QeJ}?1B{xB07i>g`*I8L%)CLDoi%FHKBF|{ z)d_ufn#&&z9{tCBuDEmvNb7fIaqS9H49c z7CjDn0|Wc_`e{?q%ZDu`>to^*gxjLaxcmB(z-^OLx>kI~EWxI=Ey%AFpXl2A;%?Qy zu0C%3AUeX^hfV9+Up;Cse|^8XG~?C_Q!YI>G0i)){IL4hi#JdFbM$c=@M=XZ;V}qP z4zM+o+7<#hMSQYzZ*;Wo_ldEye@IL;mS8i8Z)v$%;D#?AHy6Kt*#2?WYu4uiwqD@Y zIy|7};sJSc(kVZe3&EIp6s>E(<1o9fJLB)avN<6*wtTPg59?1_%94}fDr`n}7IP@h z=K#yBjtUQN058nB4B~Cz0AODlqi8>ao`U%}7<0(ard-PN8Q42<;H3oUAi{s)YutbJ z2_EXZM$GnHOaUi|0!L4{wVgz`?<(j=ecEl5rd?KD4md+DqLbrcSu-AzHst2y?b$h< zYI~vQD(F)iU>D#4CRo!}z)QWCw&yH8*|1KU@f&1$z)Rcnwu_ly&j5yS58&#w+oHm~ zIxfs}Lma*X@Sp_`T1**u4-U`8JtkSwW7Veu1J_=T>6%}UNRMG&RmawxLAwp>@cZYc zf$W@CmJ9$6@mtMF-{+H2z}TCB$?%{-k(<@Yw-HtgTP^8;MIUwXs@BJvD>80_DjT@) zHQ~3}qBDxR*bHE8;9x1ghMQmLy^#m|n42}~((W7MnCJ2>JkZmV4w|)bz^oX|2{GUI z#WwP_%^OW|7^lq{bu;MP{g_*JE(c?n3lDyspS^fLJKHmea6sIyZCM+aRpo-#o~T17 zTOQRk#~gzfqR!7BuQ|RLje4&>N(GJj4Tre;^{C*`>a| z6$=1=vTIr}cYoi;v`x7_sXI^jEIPt((tuqT;(nt7zR8L(aL7S*}QbO_BV^))cyxH zBX5oZk6F5P_SbLjpZimMwCi9N<~;Aw@>1}fuO8O1UVMB0&zlpnnx)$EZZ)d^Y28_VR<+ZP#wU%s7@Qwf%RV+UfS7!Zs>07|#um*IwuwDlEocDUPrEKXz z;~%19^3yS|K@4_-a+i{wc6v7}Sm z@!~$06Q9s`B^*`s-!9lL6Ct%Q?(RHC4Yr z`?ePKy8ME<5BR?*;gQ0(3|>pP2QI|j7DXo1GmxF{yOCWOx|ad0l$*EoN+;2v9fTP9ivYO55wu+RKUzzgBTS7PM-phI9EpMM2EqGYTMSfDur)*gn>-y}? zv#z~qmrl6u5w|)n*8FPumY$n(eL0T#fCt!B&Bz9Bjww$&FID}@*?;&~8&9}LUyO;4 zjrN@WZEB(q^vd<&O!&gxm_yZq>`L4rJ@ay;{gbr`Q8f-aV`EC+8JjVR(!Q&Y(t)8b zlfK@e1hO4oEiXE$I1`&Rc17*Ih8K4MOFE6xt=XQ-Talj2-#@nu!ak9wL!!EvyK^sa z&z9(np8xWxuo8aPkdIurN6cP)m9Oy@9E;w*Kus_0tM=Q)DY_n)I zVZcw;U5d7sU`wMeqiWr$>0FM72{kX}mo{X=J%h`WM*fRunzOJ+hJc%VPDFYxWFkF-xdlrv>=AQ5 zVlu8z@@pO&_WwRL**g#q4t}`?_B7@1zm$#ie?`m=-A4HB?V^I8@yna$ziZ(v+~HP5 zy_fF3viIv>O!P?CT|MU9+`!FT#AuG%|AV8bUfviH{}5L5`eX`Fy=Ve z^`zS%*Z^L>GOjoUHvG!gqNOjps8ztXc5O}@yAn>5Fa`K5=2133A8u#+KX3TW`iSg& zey-^uOTKz4{HiwrzD%S{ZXF>VUH}#%ELw zNr$#AZZ~q>-*m9=Nca8Si(Bo-kPkiOMSMwmc5Q}t=*@7)o`rADeHa}Poy>X-vXnyy-!#Xx}0T$>UegSA+foUy4B(>6tHRNI;u)3n5=bZz!q&Bcl{fA}w( z!)84mxO{)zZjvtC6?__nF(HB$z&(~BLF1+Wr5R z^HR<4^O1`WQh_Uv;@%5iM%@FIAl^On_mm~bW>!X^wysxJ><&ibuevM#jw*LzvXtgAP!@Y4R>Q@}{ zy4Mrp%I8n{`!^lh?zzW;&ysGd4xb&om6^K;y4}{B2T3vw9`y3DMFsZSC`?%2U~}*#dti9C%k|ZtxZ{ZyUr> zi~!jFY}lyF_<-yCF5g|95jVe{>^__h41SG;ZnzTuE05saRy@oqR~PChsHf-?N(&qCfMYm4A{!JThpHO zssR_OIT_|2vVLD(litqih>tcd+$uo^^gSceCNWmtWWYdi`NrX?hZ{w$G?Uu?c`3)j4mkHRtQG zB&KAwi7DiF7M@Bu+CItq`PD0<=Zb(caWcNXD`{7cDdjYZlWxFMFh5%h8p-y8oVOKG ziIJ}gclTP-?kl%3*Y(Gk_wuc{L(}%=X-o0aoz_1`Zizqf-Qe!UgSP&d#~^)NR7PE3ga;9?(Mq=F)Pnr1vI}!|9a(;K_P3kk!b+GMDdy91G5EgB+`!jqz-pVo=Tc#_ zA?nq1&p+Yr0llov!tYbS#iK5h9G^AAS~VzuBT4c>4L=EdZEYI#j7QB&2Ck0a3$`nX zkWQTRqxh>{uvvXq@i4-jfX85NQ(lu2^AFxwcUw)%)3WOPT;KH^`kU{)nuR}{jJ*@p z;$CTUHau{1-DOd)+HE?_fB9i10(^bZGL((zl$nrDo(iaW8&2-gOxR$^ESP$hU4j$& zh5nmcFE5M|3%%FzkOARBX)e&y4>tD8T*TIwowW@je{+7ppvy$$T?rq*8gL1Zh-jt6 zywRMQxArGox-QIPRwq4X4XigQib2D?iq<#d-3QjEBvnZi_vADP-V7_MVv{=1*RUqR z-fRUW^;tjfu%owfE8ux5di zS`v0$+vYRX*~FNkGd`|u#(kn%A}nc4xRuRGuT_V17mSZEHf8Ki&h#1qa|jc5?xEO3 z+ebLyNj#w8q4U?2OWm5VOHbn-g9u?U9BfI-C#+6_eM|erXOlkZIUJo$hc4cYdiw|C z-k}?D&*!7Kf9Ol#RKmPZ(@l64Er}Uf9S-X)8R)U%!TxIm*m``HTu+~G&6_M+VHKC~ zi)zz8-ifVLxe1#OOd}1ogM0AxJTB6xg=*Zehm-!LjCGg z+b5}@$&#AwyB-VZ#Tx;WCINN;n=|MW0q*fkK-Po@RqdG&==)&*RU%^POZil7m{)!t z_ests9KxzM;~gI_Px23Av(};XyjFq3Gnv^dV_4`558%Z6fUSmzx!$X}xxw#puvU>T zFB36av9PLZYr)(D;~;}M=-XcGzlr=ATMGv5_M%n|>sGn&h_@Hk%IS2#*1r+Zh*BXf zU?AWd0SoYc+a+SoXvxlNRjFC-+2>Prf66Zme4AVB{~EB|#VxQ^+Y3Vv0P6$SGN_4J zlQE0>60!E9n1fV6+8A{Lp9~o_TMJg;BSsV8EoIy-uSb699N@cL2y_JKp}3b{`DVQR zqm*0Gf%>{?m(|VR?JkW^82Q_<2eUre`IM{mla2ASe|+)a*l*UKwS1bI7T0HdTK>|5 zHn6wiTEcHL!=Gitt~k3oDLAq*+4*VQDXt=dpz|YU+qzd&JO8Mah1vVe7+~U*TQ680 zZe>SqR{qng`)B`@c1WAD0iAe#q_r$D-E|;`V)TLr#oFW3@2x+sElE!c>#{zb2y;u% zt`E1M7_Sea)7-=BQ|iv;2jZ%v1Lk|U?NHjO=H(E_^{P4(Gx7#>CYf|=xG+zbqAnF+ zXGP2QoV?@hoUCm-q84n=X(XAj2UUXd78Xq-u#3rQGX@C2HTu2Pq zL?^*f+{3)lv|^W5Z;Ttearfn0nAf6P9}yhgcqTkac=WP-NUut{rS%&Vt$Wr+n@b4j zGw!wMQ!ax5_p7;>N79fENYAGO@|Nv+y?T4z+#3VVlJM9ppew;X*=)$Cy#;^yu-TRk z!Z^$V|Me@|0FQO#=XL7b(!d=&(su*-XY;d08kjX@1E(l!@roSV$|$6!wE5gW`oF*1^9# z@BK0z5TCD-om^9kMG9=IuTljS!gFeZ?tYN*{)_HMq|{)J+d~^!2!&# zCju5DtiLk}hv*FEQ1LSEo=bWE;OARGTOaV|_LmPk$~GpXbtpa^_-S{_1zg`@(Iq`5 zJz!J7eyZk8-~g+3?!na&(b4406eK?3j^ipx&H-pJT_$8r(z`DZte{Na)Q@Sg~WNtX|s z47g9jOZn7YtDx87)AD-U17j1i^dO&2_!Wq2mLCnk+OjJf zGp^pgtn2dayua^4+#x!(GSbdoe%$c>#(4Xlw9hD88P}d)8`IPgfHAh`dT(q+l--#C z=;45}E$vseC6T{3VC#wdw4x2W{2Xxki@A{5Kp<=xGy#tmos4>U$8lIQTZ>9zZpmOu zz**5{2Jhw<4aS65T8{-(U6@}Z$;~Qd|9wfL0ZiFB|E%%-=g)Mln=tO1GiI&}<;A3;8V_H*p9E}_m^E9lu&FN<=)If{^Y)&PLhlMq9zwlJd%|D)s2u`PGQGD2?k^GB+ySe$E zOF3A#TQBJm-!Eu2|9wes z+@90x(3*|A#HZd&c2uOj$`*8n0(?V*hfF#w)OVej@BfCFAN(#3{Z5DViex}1#Ahv5 zSj+kZ;6dDNHRjMa!1Eh!9XICDaY6qWw!x-gz=fi~!}7tNfqb}UC?B!(Y)1^bY*-^e zalskT@oDdc&$IJ=U*s0eU>}V-3}j%3r1^+mg?!vew+`N+gkR zKNBu#9qq858BI|7i)g)xqR+J+^g550!V*r zcuAj0z2VgJ5%>?v{{G3jX*8j?w_~`Pu zhKGl~CFc6S0N!W<+pZx%-@JInKfZ9M ztt=n54dAmH;r5b7o?lQ)-=eeWMZFfCLt{aM3OM|QY{X&%9C-nB^tlr*a z;No2@d~GTf>br=|noU^5ppFI=t?Sc($9)<>W)AG>jI80+n7S_R8M=XoEWU%*QEO5h-Ubyf$8ScHh>@Wz{+ldH;&?7=FC>)&m{oMVs4WnF{5owy0slipHZ3)>czm#x-(v7E7;WY zDE}foW05E9+V*62boKxmnwVtvRhS=HeYgF`9sTy3zVd`fXri42q-|_%_N< zn3B~jKkWYK%~S4YctB8-baRiObI5Gy(*3l5=qnVjwc(W4VSw>-Fiz=!PJ+(2VUAl# z4(sjfPluG9$oIKAA*+o--!cJ7UBbgZ7V}AJQGP_mXVK+?eHQ`O4&f-qY0f&3h56iC z(t}(ljyY6~f!iGw$S40-;GV>s7I=%iB@+hS0c&o1$!Ph@ORe_%=NeTusN!WqTG`g3 z^>X&P^-BJ^?OJxhb`{o0@77Y!_3an78`(vp6`R#7)~2PWz#jMGq5f;}fJqti_4Z{V zLw7+xUO*Tq2bdi5TeL*T)|&<1l?wC>Btiq%lfkRQsnF%8c%tNL0eKh5fpMN`V>G|uD?eg1*5ynC7@Cvq74tq z&SQulNA%k4ob7VduWXM70P`$dcmVqo_NLj4uwf2#3lUcFqLb39jS>H1UQzOArP0~l7Dfjt7ejhk>N>Jv^yJ*=6o z?RkS1`HkWpJwN6)pgboj;WEnzw_b>QG@ah`s1`I(o`Ya%^MVB3QgHIPl_82WJ!XPXcs#)Gn*v@|*QrUaK+VS9LkA zpZLY{Q^~oQ)2xionD|?MoAoaXwo7jpbg);H-6(Dj_RxuqaXE^^tWGjkXNhyCA71jMx&pB%WoL6r6@TM^#H0#*xfU9id0rkJ~P|JqE~zHC6% znFX7L&ceXfw5J2=Zs3s@k>1^Up*Nv%j)Bct9u^;3dn(-jKbEW)vt&$|QIQD`enTwS zt`Kud{??+T`7Qiq{i4%ZpR_IyIytu-c+Hew1gsr0X^F6+EA3abN1dvscnEB5;LAWVbbT6+ z^nZ~6IUZ7T(;-|D|-t;GYfM$9K|1f6wmZALFl2l}q(K&JqP)8j$VU15zd z86`Kgh$?(cBz^&z#kGb;*-f4>8Z3!agOk)+v5|ey4bXln{=C%X|F|{v};@AlZu-4 zQIz)~sftfYtD+Oa!y99QeQTqAgyXNV_9~ zb$MX*3I7;;W6fU{Rq}j9*_H9j8j>@@6B}c~!D+?Jm`CS(9xd`W)DZBk-vcHtbb%>UbnikEhZtJlZ7 zKa07!hw~A&D38v|pr6@r??nRkKrYgIIXh?6Cj*GH$eT7{e%4V8Z3wsqj0+J)cF5F| zTQHjOkgPcq5TC{U(uNegD+Aa+4HzO3Fln*yz?Wc4u4Lyl!rYu%^!=hv9dj!hHvvnp z*flLLhNZ`n?tzO*Ur+z)gre@{!|w7-NGl=$KO^21n-*2A0VhrS^rDzu+xF_2syYWa zF%LL~@M^lFDviB4r*$|e+Jnw{&thR^k0MSjWreE+Q+jCsbZ7hrBZFE%4R z{pM*~>5GRQrOzL9mTio251{kqWWc1#gw1*~4+Z7pdQiLv|G>tG;Iq*6_TRnymRGhh zB0lo!es@`Pgug#Jrm9(fB=~IUcJuFNuhsm^t6Pn~i;f8npq%v$he^D6r;8mM6CO#s zWOWHx*YPQ3-Re`(0hF(}@k~+q`kwHk=%ljt%~R3A7q_}u8^hv*$TzzQ>;Lt=j-t7n zP4CTr)A7Ob|HILDzcsP#{YJO$?N+xH6$>g>P(TEcUZwY5LP&u0-V;(tLhrqILP7{B zB$UtzJqe`IE&J?q&U^3uL+<4LVIF-DXV$Fs{j`}`YmFkmo~!VQ-{N42R*)qhbf(1TRvLI#&5x5>I&@=}1I-G#{9;;E`PUf&l=L?Q!SCo8!&k?^CiCK<0 zG5>MQ#aZIf9Pfk9wX~vtmU;9a3(V4gR_O?bUOP1Ee1?#GrY5EU_mq4JwBh>#8*#2J zr=HIX*)|m|>wJz``hAW8JyGJ~Y>sCS8bY!`jPN&2U_xv_E;*SYrynawNx!so@c&t1 zqW%jUKY(0go`CyVCeXp>YHI$!*ULr!1~TN|i){S)46X27MNGX=5fZHuOvu3)zt z9O?<+36AI4s0$sJXxEYettX@$Yp_Xyfad&)k{zFG8_@50r^54rW6wnT$<6YSI zMN0mW7Mp08LWG{qQIY>=ThREQbr$i*5~<+t4F=}NIt_lhK+drOeX~D?58dj*`0n-* zg7^BV-{z|zuEx=lum1pi{|}I5&H(;A*C3<5&*74-)8u6RI4NRl3=?vypyUGefUrsl z-~L`=WB%_Jm;3J(;QBN4Jcl+bJyJt_?Mir<#*Z-QM<~wj^7>0j#;A6W7?y=Vkbvsv< zhMvz9huRg`1dE&pw<~yXV62Mm)A*SEVU(X?2;+Y^h6(Q;o>gjd%%z_qB^}Gj zDFB`$kEEqv&7)v<(@^0D1E2$Dami*CHMEYl($A)iA(+1NqkROzEFh9_y!$QT*>sE(h?olry0Ik2TaR zy9yJxKVIZ*kpLP!NlrQ#C&up$q5~}wO8S|ag9JDv#wNo@+a?Jy=5b`e!3f-Uvj_5S zr@z?K(1-bYHc3hQf9pWj2F`6)l9FsPOoU~^F^?1zaiL+C+JXPOH(c^{wkFeMvkmHQ z8^wm~^%Z~I?Ss1S_QE{PqsU-=8_a#H8{-Gyr08IRowL%4@m^|#z1{9b`T*V&cCMv5 z=FVrG1Nwg~r=}Yv_-{ul$G24i{@<%y)&<}h&H)TtWr!d<;05ecfY%s?e=&_Af=*Oq z0E3LY3r$(+cgH)<|DBtF2wZH*eZJD1>vA%MiFa`PIiUL&i*$fT_+{sFeDcu@tH?Np z3tMk2e!bcNey$N@IZu!hu5UEwK0g7p!zLjlo@;O^|8JL6{A&@Dcno;<{RyPMu07|O zzBB)gMT`ir$*|uXbGv@6a;YZ^Y`9%c&$I*eaU`dtIo1)`?uUQg=qPd9=|gz$j*w$7 zW_hHa^W^-aX=KEi7MuKQiIjKY_~v#L@pZ2k`Q>m7mtdEnBhEDp2*CY0#|ml&Q2XIF zF)Gk9j`(62EcIGzDSWM~E_l7)PmA8`W5sQC;=b&5VZQ8jW4>&+BfN|QgpjRHgxBFP zHrP5&jJ5)~;!FkTi;R|Tl~9w+}|o zGW<-2irnhRdSMtr1{y{Q-z*cPMEfK${$RMs!#WQ0u}h2n>@v9jdSB`1>GG62#(|RH z|LC|RM=k9R7PuKlAf8q!BE&k047LHdw#!KIM^l8vgE6@OnUaS-Q&*K)(BfTl~=`#zmh_Q!)?7F`+-_*qHDCtD%Jd^^lG=EF90LfLenCc`o7C z5(9lUM=3bb0J%-e!CU~pzugaZUvJHOvC&-kR^JYNr*DV7+v`R7T7W*YP7q?w6tv7E z2_^Ax1QT#Lg7V+!$bSjw?Yq@_@C(y0F6^g{Pd(PKi_MPt&VVmFR-z+KVwm?%Uy1K# zSCQLdL&j4nBjVaheXgr{7$3gVQ{-kCD*m+GnEPUum-b+mmwA7!0s4A>028#{g%6f< zA&;lIx%cNP!Ou?sow3OYaTYN;#5RGC1lka|Ka34LkPwq+nzF-yELnIyO9dZ_F=1wh z#!aEZcKVAx%vWbUUTMg8+3834&QyZkS6krWKeS~)KhpD!W9UGA7u0Q|so?o~W8U-Q zNnGsDIeL-47Z!CSr5B%TYiq4jY{LN^x4L&wgeaMtoW8Kn93;X#uuLbjpuKx?`;t_{BUS^H^OPY#J$W z+wRDCy4;xLbTD4xx7b?XaWqBF`nAlZ9x2G_rlFz_+dY}j7wQt6M#%xcw;?}X9cCol z8^9+!ZFM0141>6k)kf&6wT2?s-7eIpl}6|rIVbMkI3euTRx8ZcE~RDKCs}!BDKmYy zAN$oX1ov62%XvIkneDvQRpL2B48N`_OMi7dLQ6c7(9?DY2tk|O7~i$V!WX8Vyw|&J z*{+AZ#Xg6FNWZnFf|vG5KJrXm%|DbdGPZhQ?klya_g5;D@2ys)Ijz>DI&JrW-%jyT zofqrjA-lb_D19sRtziJ_VH$#Z><>Ucth7L0X$4siRvYtQAB>~I_WBVa7fK%KSjL1{ zrIZBII5rf(+1K5^63;cjOD@(WI|KMmJs4-C?+lV7t06o!)QXiVeaZy$!4{KdHUTsVZO7jI{S&XJn{Z= zBluMpCdLVr1S!F$7mh?VA|*GE&t zc-tf{>SW5X_Dkt!ReAIsZAI$irTV-VGES@$z{y$X8V=SxPD!*$DM@?%C7w%-xldHQ zv^#2U#$8QC?&D5))E_cdrt3~0I_!9gm2Z}i5_bAvA9gwb&J_XpY|MGC?=Ex)?&r~| zlnyaX3h67|_z;7bntH0Osxpab8M?aMmy@J#zfTZjZ?=_$-ol4H2W|D{M;wTAKdZQ5 zf3CM?xSY)(znv)IfbS{tJ&_~BerO5lzvejDBRMtgOv@u4Yq_M24x+!l9pR~O%YV7j znC-ILQ{;6vO-|F)<$ucYdGOm>V|MVb1p#lP4<0^M9{12Nn)~)(BaT-NZ@y{te@uXbR4bah$J<|@;i zG@`Wo)9j?%ljN9dN`CgUUShIK7c~5qag-V_WrYC01seNdAGcenfPdC^!F=VskXtLw=`Sr~oP6T|En&S4>0uh7#+b$k;igg4 z*ZrXque|~2M}0@m>$#dl=h@2CN9*mBptTNS!2S>+bh{hwwb}@AU2B27nXk=%t`Ow8 zbfTiK4U^;U><^(Mjbo(f<$AEoj3~=V#!k7T7NtF2s7-&e+LZp{KwR=wEQoa;rA9d) z$vJQUgBi=fx#p|WofhkJo&bFCVv-eoYlI$lhnMx~4;d%P^}CXqW9Wx^?ezk>H(2Dg z*_r1KXsz3Rf9$i_%CLL;6NnhQmI|@SNy%nFgO;1&Zo6G*pS7mq_gkHCA0S_PX{)ne z0NFCY;e)n&iar~LiBT)9r61<1!OxbPp>MYa2vG-8MyA7O&4`lj?hPXRR$5?gD@}l> z9><0+HGw}aR2RJ1YAbd>b7)3aiLa8E_j&$0QqpND93--mLX3RxR}F1 z*cU?&%xkx&_``ZfKH$qs!`FJySu#PE+fqxB*TpO;2k;`FcG`1X_PPq*Emh~dU2Q7* ze4*e|&$S|kRmRRYP4FQ5BlytWL72ySTfwWvnw)3zl}S#ERml%dRFvGUA=sz2)>6-v z22#*$CERVf0rqOWx#azN8`ft;P~ejL_30%UGs)+8id(YQh5j=na>9B;eP_%b)~E1-KEW*bk{*%+S`6=+*8vSKE^Oi0JNpV8^DIg zT5Ile6)*nQFy`x(E||w<6)(oQjgauDi;(gbz*53i7wXGaXX!`75bDd`0Md848Rj;C z3HW0G?spaN+V1*Z_-8<8!w$ui^!*`x#K9OY)bUL1hX$AUa~7Ah-kte&x-9AG!8jjg z9A#&1^zYx%it_Xa%JR7T#{QD8hC$@lwbsH{Yt7(STiqp}R@-4d zYwaapPZUCuManDOAH)Q#w&uRjx97T!(c*4b=lNWbag(055aS-VV&a_l0iIa`GL5db z=*2=!_OnT5!kzlUpg$Oi-k0VY3*8zC39oT6pZ)-D=aE%RNIssTq@M!0#Vm#Tt+mJ8 zpQ{MHsS!lo-RLd(+K5eg&M*A-KwXcEGl{uS!yqc?aJcZJp)c?4dV9uuF~#?~zCG*1 ziHZi^>@M|RZ^(PKHvo&;>MM>p7)FKcc7b2-52E}Qo1kymDW9)46h%HgndTq>ZOH_@ zaQsqD%H!3#%x8uHnAeF46JwAdLMloEZsU{OudepN!cBmWUa2d1vC)w0veuC1ETf0r zwn~U`$J1P-d6HFdq~IaV(*oS?2sQ}Fs2}HQa-OKU@wb+%6Yg$zX1%n|5YwcUNspIX z@QH_`O{|@MZu)X_(W{xNtY+l9Tp8R|9b!V5UF(_R;y$7&ZRg z<{&<9fEstZ73%w&1Rws#2tM*!Te1Hi6XbxK#_oHl#%y@nW3PY#E1>Hk`)sI|+<8Ad>xfXowkGKB~}Qy@aklA_NG4dA!x zdQ^~1Sm1dmW`cJI@FCk>#a^aCq`zqd9k>p7GzB-sX|oL;Y#yhk%y8rG?DfDyj3cy+ z?QV3aeHX!~f;w2|+<5kbzRPqWDJ(m9YShV(%)-0&h;zzTDm)0Q*_x z`1oOZ%p+}Oj{7eim-ti1##$!uF^e@B&MS?1u5xbD18Vf^%bg`*kCvL@p~{NuIC*$VEVSO7DO@nM^N_~5mUeAm^+ zjK>;&&O-@3;YKe!^oqV2=3^Kmr)aB-UT$?#qj!5~-+EDgSH_6pH|MLepDffD0KPxh zS>J*5Kc3{m&u6%({Q;Vz`|M7T)66PX zseTv}I3vh@Fel7?yxE5G)>dRbUZ~D~YaF4cEVm%MwmUI?2P4GDgAr_qX$Tv7I7%xz zQ?l?j6{|#8;_n3Hs?dK{D;tkyD0v4!<~^DsB-mxRsN-pT%&CH$YL(J6buFkMg{a`Y zUB;tmYYX0t5QDC1gb}y*1~Xnv2_pWQEl+jY??MNt03GC~y5HFE#bo_l;8QMU>EIJJ zKG7nD1?+aGzFMpZyuR6){q~2JM+Q86{#vWn&K9UQE?9xwS|7RX-fKm7!|zOl>cF?JJ-`NT1qxFqS(#5I56EsV~4! zZ&uoiJaipJpJqD>BeutJSz=D&tNDtIXZt-1nEv{LHa`(FXqMmFf>wCo9{0y2)MC1LWY~h;9)C( z|J~^*a@PtIAB-S_e;*-6-(P6V`(OpI%gqV=vlEvD;E6{+nI#tN4rRI-$H6@rOgDKC6v6uDa^jX>^Q?PNr(HW33h+AzM$^#yMn z&s+=x2*2f)!gm`za3B2;HC9$v~Szmias18Mj1xX!8?NxU&9C(=+E3Y zs|}gX2ScP3qnHN;bo{Maknwy*koA04kooAa4;_9iVSzV#FcAw4u+N8MfGKEXJm}gv&YOdAay-!Pk2Avf`wC9#Lv2-|>qa*z5~!)1Q)M~j z#}bcoI?o{;&onfg&sLPL_m+NDRb&7)l=fg&7wGhx+xVJ@Gj@uf*Ma(y4J=+-s ze{{U30_x?<;Rr3sD(2+N%X8nzSSjz1CfN8RB^$9dT=c;LWM(T+dzMjL#EBRYbTmZ< zY8RidJLFi`2_1N%E<iuno8EYoBNz^{G-ME@IdPU!uOo`Ux~V9w z7e~9@UG6LScB*9*TcpHz!#Fm?Foq7<0J8KHFY3`SHT1zTkjn1{weY^+r# zA{@#ph=(!`9 zIsD;LMXAs6Xcf*T=0c8SjNEe#t#E$?=Dj-%_cKb$avgIQ4iwbfoq=NSgP~I2jrP3v zd&AiuPiL^nb~!csY>tC5$yhl%Bdny|Q9=Gf3;Gk_yT2IwQGQ$Pkaqy)pRIM3`Y&{$ zzBUtL?hcYto%aWL`3GWd!D36v$4P$j14&uzy@k58H`|?HKf@p*_-KlrwKYuqHs4zG z_E^D%ABeG0%Pq-IM=3$KCpmEsx^aPj^x}eUZgo@BC+l#2O^E0lY7yAyOvyn1SfGF} zG?aX+3>&x63wB%YNqgyN>kl2X^k9;dWR>yZ_L(Zq?kFQ|wle401ohiZDLdi$WLeC^ zE{xArF+0p@p`+BhgA)Bf%87rnKY)!t93v+KeGNNTa}dW026RsX@i|bS;*K;z`jN7N zu|Lkv)A!(GfE*pL*9rExkmKTyr1;pKK7`-?5Hj32N{F^ja$u$jYNDnt>Cs|Kskc!~ zO*D$hvByd-{B%}WW>J=v?o436S?6opZF4OnHbohHx3Bb*t|{lawkGS@a%+*Nb&8e^ z=wzO%I_>!sFTu$+MTH$sFmiVL;a`^;Gh9|%^WX1`l4JMAnAw)e3X*Azk-gel`gwl{ zmjvJ}-6kidINmXA_JBXmRHwd_mB-#QNwHB!GaT&UGy}5IiwLwS1!Sv&hcb%sG5T)s z=e_`drtX`u^97pz3^4 zShEb|Bddy-e!fsud#n-B?Fv4^KE;8WMj0u4gS6X)JQQ72=X^t ziC?$ciGlk)oo&+7LB< zrl#nnvI^q4-ieR3PH@mawW2cf1U7oNx6pg7J?rIK7u3@(7vlB?FhPs;87||r_{TC< z<_nbo?7P~G1~OAMVT_A-*8qvRz1mF7|7WSH-toT3JP!5V8-{vqk07HCrNp%Fv&@oH z6)9t>CF9k0H!H<7)yQA(XQyg}1ug)OKO7)L-=CnzKa`cFzgupBN6xnqLij~d_iGEo z?);jo;M(N`fXj=6jAEGIMlZzEAVbAjHFTs^!^IpS7<;B*e^9Rdv}~4>JbvfM2%o*Ky3#0A3}GG<`?u zhY9kx>ncI)-OauNH`6#N*6}UARmv$g4Kb4U`&MhK1SE9 zPj>nNt{0;c3{pg#aT1?nmXiy=FY~!JEd#O8Rrvm38lPYsCng;!>&g#hHT3;4e*Q`u z&JVDPE>kq*Q!SU|rsa}9NT_JH9(cjSE@;8+2`uc9zM2(f=wTBK zQW`MMHGH$YlCmon7V8IDiR<0uptV-&*R4)=l)kkra<+>6NthRYc>tY#Z%#<|F!q#Z z8GAS>iw(4(X%@i)+a{VquB05%YPmjTCNG);DaAlz|O^-#JwZiMl4R>wHDZ z{oRg2ciT8Sd!ws7eXfG!vC+T?SgT`)?zfev+Xk!P`yHaZMNwt63{QDBOUJvJ+JyxN z!@Pq1A+*0`oStEq3#c}A`-n|hU1pUt!OJa0AC!W$$NCOI=H_5CLEl}OyV1-H-R-Uf zZ?prWyn(!sY*X^!E3HV6)rP{Cdp$6ZgF!^7b&QyGEMqt8s>2cCxl2*9Gu{X=h7(-tlr>|Ne}SvNgrd-<@J+9VoaZM>FN5^BDoz zF6Wl+4&%diI!ivB2;TN|e(ysTSGVd-`QKL@B+UrkcBFb)m=@)$$#P?tV z2R>h{?L1qoZ9QG66j`R(#pY>x{^^`hc%$*$Thv=Vs&duZ`LvgOr%P+>Z!3&{Xh_ zmRh7|Yc*YGt8CJ?82L@zkn>8{k^AaUj*YumE*G9JS2Z5U_@%o8)TF=X8+(4vHue6P zuWK<+mc}f##k^YW&3JpD!p5B~mytFH@L}tH+|=nB+LvBh$^CXh!C#_+kcU$>?43y=O4myXnFFxA)LxS_Q&S!`#=<@C#X;^4;!7TmQ3&pX zIOOwwOwpqzXx1HQgx_WNCl5g{?%oDPyncR}l^S_tgjx7f(|`^Ebc1%h%p}-#475=} zhb(j=ycM1%m~pRv_g!Ziih%QDM`4+P7A++jSjjR z`{C_nOoacRl-Pi4q{y$oFV|z^kCaTjb%u?z&6l%xWTb>$DK>Ifj0ymJ(>H^J3t4QX z1!-&Pelj7$XM%%p>m_Eni1MTE!@@pag@y-S_IG^>dh2ut6#DwvWkzDSQzs$$wV0n9 zr0+xL9;sPy+YAqP4B&ZR%*@ymQxa@yF7|YuPqC=!xyxO}9x7qsv$-0a&ukU(Yh`}& zO|akRORxaHOO)8?tC*;-S4oK>f6|kqt^w`766X5kQo8rYYnZsOTQzy{*LDY~NmjLh zVwtI6?uvN@n**4TXd-+})kt0;ACLuKFWBoaOVwQ@(8mvW7f z7I!_`BBlOwXt;Iq~Svc0vf2_Mc^ zSo9-ZMWs;*aLG6|@o<`$aHL^E%}OqEPs}dZ9Hv3!waAEy;-u$g>Ct~x^!&R2ITQ`8Nw{DjZoF-$!|qj}@~<%qbFUOeM}Z=| z+(B=g?}A+J+y=dS_yF|s;a$-6Yrlh@-TDg@=H?2@3-Z3y3QxbIsmO6T)6xp8^Guj- z1Ru08%t@MV7knG3q<(0`<=qrOGp@07)2_gxLobE9zXWR41(f394$AZM0R_Ez4svt4 z1&aCj3gq+d8R+J~eLPc%eYeuViJq?K26WP> zE@g$V>!_5}OF1DSpcJ1kAb(d^(91i2fqWl50)@YH0eybz1iE+g572KQ5a{k7SAlll z1o=F51_9&vo0JW4S?$Ds(GRf_^yAf7bq7CwqKe_wibp?bBVe4HF!0;Vtkf$7!Ty&L zygq@lyg!5TeY`;lA3uOR9y^0x0sZmd_HEGbzx@V!dG{|zUx4`eF~6(pQC>S!m?)D5 zlejhn{j|^wXn>Mgw67Ep<_6eFd0y%YIqwTtj| zq5}20hgN#O77o6NO^Clz5)ll_@$&@5eS8b@aJ~2(C>Fa&u`xWxjlRYa=LK?bmj7Ipl87KQa*Wr zSXt5cH1)(UTO%UKzNEd{qN=SpoEG9X$JyzMM&!FeF6?O)GUE;@EfVOHuuI?E-9R56 zIe~niJp~27dO0BP9aI__c7>P{{?}4BJlr%@8nrqpfDcyUU$-(!?+eg{*Xae2 ztN4ukE0Eaa-vT~(UV3=@F6chcw=eEL0KI+k9Q5tu*DJ)d>^qGZR`%58j?|S0)ByTH%8I{?PK>#Ph>N+D5gY*WeEtOV2{^*t--Euq zc?pc$A3&cx0L6a#c|aPkl*XKpqKZZKu*AYx&fT|{-G1->AicPzkdG{^z`N( zP~eNVAVhfLjb?b>yZL%tn7*G8uOHwf>-t4`;u@x34-5UYvNY$R(EY#efZja-hd_A0dJ2m1_PZSG_326_IPu|X2O;usijlW7M*Oxl zL=0Q$z<<@XQv-)MxR*U_!t)9^q<{7WC3}1Xu)vLX#KyFW5K!p(@munzNzxARL zuI~)uVt=j)YmOG!xWic?eS4~sy05OQ-H?i~qXO(JAuR72Iw9t_IFApPVn2ETed7)C zaCr{8arJl5lbe45*L@6n_v{JC6&TCp&tEUmvor2gqcYzvbs>VaP1#RphqoF0*O8In|RjR!A&Il@DG zu5?gCEvia}agqnIPV+FQ8d1&JLRFo8wpwIR*YY(Zb;UysO$j10(WNLW>w2h<&uvtyj5XWJZl_O2E za>Ukn1$=S1fuQOXCXCh4y&IU==LBepQ+{;v)zskV%fVj$0LGsKwR<0Q=g(W9cP<{F zw6KKVaH%loPFi{V`bbTgNl}I{$r(9E^FrR~YU9M&T6O!$N>%fYii_3^Qobqs%i?;t zv{x-8^2>4v)QOs%a~GGHdAl? zviPa2fflA8Dn}hiYuN`=l@z0#SG+S`mNVOm@znJ)<2NR2sq&8Uq*@}(8J8aWXM(@a zwd}~SKhT+px6&hn{`7kF0`&Q{`<0x8Jg21iq}y5XiB}om;=4jL_Dv6uPUjoR9(O+^iBs+}ELOF=)t=jQzJ?UT=?NDnViUU)R9 zAUg3%jDNsYuUC&j$-zFCv3aR~))3&XLwrWW(qIL8f2tgFq~YQ$^E}a>vZZAgFfiMb z4a|*+deTZCD|)t-bK=0!mgsaY-g&9 zyxz-6()U*6uJ$(+t9zTFnvnsaqOXr8ZfH&#uV@JCX0hCBOCfh!F_;&!%Bu9uktVW5 z(NtqsG&R_jwZfwrKJj>-Pdb?saqKgd+>If2+EN$w+vW(baCf2#wK`N)r0lFsQ*<{M zOn0icd|k77uCeoQuDSnsv99wwkO5ET zm@tEkma3PKqUEiKkMa&i#86XNEG|3!CI=3G(o@}(&?YGNAF8bgY%4GS&{SUSHQB-q zSr@m8Of!AT2ayRN$ac(qAb&o@j}(}6K%tVlRHx*k%%)-Wx0O9E(DO}5JrE$Y@d zH_2(SpPM~9I@T)dX@f{RD&o}r)kSOKW|m>Psp?QAq-+mkVpLUe_jg7hzGq8(;rGo} zoqd6YJXA9BEm|RcYq}h{EUiRN_f=+0wpYZ-TI-Ug+PLA0fd+(fyo;xuXeKU;nMo_7 z)aa!la@fiM_N!GZVj89dxYZu?m;EtJ$gkZx$x(}(6o=kEp5o)}Gv)Ms2_@AiCB*Dc;J=v_09L0NIrbSY=2%xRwm<@OZ==X(6(a}te(5`F*YL^2$ou%fV z%+ZVYr%5RX%CchnOk>O8WS!8a5K#Fbz{@5GU%2vcoCI9%7aC-xu1Z&gUEZY-$eHrYS2u zoGybHrTpT(F@E9EOeOnBQ^hl@1Vo#rw$?OVFFI7#SM1AqC7WV$^p=q2Ny&n>x z^YzN^QJt(3v8rI^u1v8Z`l)i%@w{W*>c)}NrRwHG4GDa*RM~#E+CG1}Qrl%xv5ElC z#XVgFeAg^6wlf^0bw)_qonU0H_ac1`6=g{KTz&h-06tvT27bFSgbJA-B1G$_+1P!x zkiDm=;`9qM->i(1F@PqGpDtFnoT@PK$5ZeyvlJe>Gl~wgOfhm!)qKpcl83xl67r7b zxum_Z62INS0?+l%v}f}T>2Id_kS=?glqADgv5!%V@OR8>H>-H)xpI$z&;q-pN9 zshe8ONi~B@@6<*||=@bnIz$Hb=@eO+bT| zJ7XT()FjC1d~M6YjEFhgjSe_eaj{35%JO4PZT*S5vdpezm+XvU#PqHXFvnqy*i#c=BXUpXri;|5va_j@PDQf)h zdb!}ARSwOrqUEgjpuXsuk=_SV4)SEarpvxm)3K*!7TQ)t^(XrJiJ$uFo_}waSN~Y$ z362&Rhz)VR$B~W3p6Dv8FV<@ZfZF1p0ONTvTZXbJshK+?V2_=_ymu$F zEcDhG1G3(O2|1jgW}M8I;g8nK%g?r|d(YRaT1;9dbgDVkZEX~fI9jgj*^x29W*I8# z`y8$0yUwv@a~aMi$0ly~=LGBx6$PJY8O5e4Hgusm&qZ64a(}%m_l;dmNVQC3GcFdG z#IvQE_Qerg#<(cvsYyY^eAm~FTjbQt!)apjwiKVdJdBP#RMQY=D}uUp$P*r3(=^Bea7{*y?#!*t#fs`1tGlcNn8!Gm(P2!?|06OqXM~D8SFR%E% zTv2;6%fb8v#_sPOA;4ACEUSu|ZJ%L6cEq^Yt$~urUn|wErwet>mT4i*I?2f1??-&L zi3zdaXUa(yxd^9fB}a6U(wukuP%%H}Mde2-KK5{m3k7NwVv>?_EJ`NQt`Rc#rt8>M zsMNB9MV!vmw>doEky=DGE6R$@DrSLU z5*KMv5R)DI({2MllQ$N+0ocwwn-d7nS1KESuJiflIx_gXj#6~7z$n%CmUvm^9NdrP zh9R?>hu@pzK`q*9(V@D!+@$0Yfck9wz9`^0bo5X`NObH^;MlXK4p6d>buZf&gW@`XLF2V$9q1bl$v?ISkC;lDy%R} zA!CdReEONbNd@?+DeDXazBd8>Y?PIRpXjL2^HnKLI%KSc6YHoyBJy&r=Ck$Y*|EWR#xI60v#IC>jy+NLC&+PUvm)-XR>V1;=TT1=t2%$^YbSng z2^;LQ%wn5{S$wwIwqRRq9N1IvQ3ncU;Xis7wh*yt^czrXfRHfq3ZCF z&VA@8mtX3N+Mj@bKAJ=X9}efg+iJ~xx>BG0WV-|EX&5I&4^`y65oY_{wU06LFO-#x zpW1Tr_gNCar`S}p92#I8%Y5@M9SiB;TdS6Z`L$7AbF5>bO>$zgQBF#-YDlRk8gkB& zf|$EEivPAhDk|CQFUz%z)02**jCAuDF=~H+7-EqCe-{&D4TCVR&F<{yMlmwjB4_4q z4^r}MlCt7`2`znd3?H>QNRB$3;1}9uyb`;bgE7gOc@FGdEHn@O0DR5ad^wO6X&Ht| zO7f0`5M`7xfm*JrK3Nd(0MEg)XsWAD=4xuc&oW@&wfLkHRY}l+6dAlH#)R&Tp@R;l zSrA<(F~qE>U>(oZ)*op(C>y}nHpl*5a$?HLATFpL8~Ip2fCFDF*0!JMY8q^_v;vC? zlL+X4v{{ahv;o?31dP*$gj-Tk8gXA)T@-n?)HrBgsA)I^ypzSTr-~F4VVp!q*=9LF z&SRwS4CBL05@x~0LVfplAnTpZRtt~Re7tpvl)f{F46seJi;QAw(hw!-?p7Bf^4}YE zBmeAGj{-i8UPKnzcCq|si@UTA?s%uVXMC=m{A8%6u<0nOj&()_t9;Zaz z-5H|h0$f9wD9?Vas!n-rRnv-ptnhh%ZwRWsua@(yS{{B^RtB}r)wZ22*Y_RBsClcM z1#dPw!LLu0Ku%j?;(pCC5C=oZ(8b#1M^k0-_qY1t2^Wh@yiJLUGK?cb04_|nX+>pL zO$Fn4i6=T)q*ERHll%wZ-BSR6CksO1k5vZ7zQ7@wXP5x*lp&9GJnE12a{kX1e%XnZ zR?*ES#0XXA{qLxjmplWIX)Z6A>8Ao<|&)D zrrJKkLmg|%5NBFe$>A6>=ujyj&$hyS*IIE=KWC~%r_-F`V<|1eE+!@HbVA)MKrLD& z^o-3xbZ7@9>Q;HO$2}{cw`LiuaCZnDxH(klzCHwYU+w{Tc%aDVKZ_#a_7oXBC`@qC z50xhUt*;%tSQeB6xP{pOjBE}=eI=amJC5&xv@Q71a&X{N(*QQ!JW7jSZ7O-cRF~(X ztI2t`P?hm$u`1=BX{gxOETLzq+ffk(k?uG8F$wR?Qg-p-G@oc!bJ53YCggAu6}8cq z_1-8c2|AhM(5wmu8sILDUBiG`mBi%Z8Eo8{787SzA)~g2!G4>)Fux-uk7A!~>N`-C z0ltNpd@)1GIZ@J~RwXIZrlzDHOc7)E#t9K~&0yE9Q9_coy)<#X2j+jIWEC34xbXeK z(oY9~-#e92QZ6*ilCxQs!(a0+bRzCgAS*c5Jvo@f#Q-|&vkzqNZ5cI9*9O#@q>N;g zvk7}LI@k>O+OtJI-KJ&~9m>hsHYKyfB4cEmC8R_^ujA~~>{7=!p(k@yq7z+x$H`Jv zjY&>TF-j@P+kGfM4L9M=(KtTs$6RI2F~CVS4GV6TmzACaS>{;9$De69*aJB}%`ipC zJXA6v7rLst|2o#pU8`#Sxl+zO)iFx!8f27d3LdyO4)--mkijMiCd4cy#@l4H?86CC zoMoDkdkoy0MM*0#h;i|IgUHCESw4MLOb1hPzTR5xC1(FzZ;+fXSGCwRY_wTPOfgQ; z(g8k#0RF87s6W=8gq*x9re^F*n1u$Q?Yd_0XI)M1J3|-h>t<`AmvICWXO_`&H~LWl zW(B42$GWKD`v#YDq9a3W8ho$k<=`-rqmMXRK}91K9q=QrGomP1to{AVPN4u!QxIidTm2q9?vZ7JFYS z5FQ)Yv?BwZ>|VgaPAm-87YhXk=->+L1nkptFX*LhmXK{8Bjjj1*|{!hSBFp8RC}P| zkq#A<5|4sf;he!Id1k2oS_de{C_2_XLC8FuqnB@VgPz%jK(8%5CC}u7#ES}E{2zw; z{8yWkgsjERsyCKS=xgf~xy&YJLYxvd>;Uk$Z;N%cRxvJru0Hee`XD^x1jvUTz*cwU z{;`<|T=Zvm0&mXv+4jphcf_9k3ksEa(e)?N5yZ;o2wm}NtkhmfKEeaxo|Y|OEm zPj^d61=a~v)bMMNi5`kWC zj}$y#8_Enk(%`H2lyt<-3^I8iuzQb;RA!xqC+;udDvvA_%mI+QJPS00wzu;2>KHQl zK*z?N0Q$lvB^O#oQBlrmOuS>FCVX=k9Ig;#JkmGiJ@(GgioF1{>}hKo_YDF*VC$uu zW5{qzd)Wh1ea2<)I5gTmS{1WBRuOz`-~oEJzSbkB)vWecM%u@#qgMt>15N$KPuy~U zU#W(H6JuS+p}#JbskZ5OiGla`|8>sRL^;KXX!jiK)4md&;s)}lQ&jcdH3<)MOXwv> z|La{(tKOZ*W^Iqb!q!HR(Z&HpcrPXE<>mwp=hHX!xYR7n1~7gnpcQu24A`+j*nDJc z8rYo1ByCLNGrek|(6iXsvLU9IEVMvEjV<6{*90zcV*vEVIR%Xa;z{D_Nc9JY1e0-S zpyCe?qKwUL`VZXBL&>nI{zP>$$vD5i-W7rejW)I~R^u>Ie6i==dE4D$P2Bj@ukVeBKy= zN4w|ACB}Yeu)dp==2iDfysG9VuablG%~OgGG%U=Xl9=nBL&mT67d=}LBwb$X2fwj& zR0i1xVPV#O=)3J+z#rO60&OFeA6)aetii^dXEoW+ei=ih+&eKe_IMOLoJ-DvdV&13 zH35%aAAtmKjaR)tT;Q>HGy)b7gK`fS>F854R{;Di);2**w{(GDF0~fjGd34Lv58nk z%YD=Y1-s;ieFV_>Be(?HFd}rLzv`vFt>~`2Hv7sHBkf!lEd1xyUP8HJst&h23=0L; z{;6#W7VVW0D!fu4XUtIwSNg!O&CMl&hw~JSzo*&iNI`&#AODAfm3+?9R`|p<4NE!# z^8K-rTxOj@#cJEi9&C>w!d+ANRObXCWqX{Mv@wJX*&IN8u>0@Vb)ka|jmY<6T5+JE zq4NFqG#4fn6x|k+Vt&=|63%<3A;CM7;E&D;P}u1r6*a=ic#2DV`o|o%F#1Ht_dRPl`zRs;9t?b#&ChB9DHo6Mt9BNeTV2Xp z#?CCQ&^w1uU+XV>wbD}<=n>aM@6I7UTgOY^EY!taGj^oiJvCBsUzY2-_tfY-*Btbd zSAj~k&A@@5_g?n-(1_aZ>sI`>499TrK9W;WCFkqsZJ>(&MC*|dKdX@7od%IwF3U( zlCagQ;F4VuT#|hP`AOeg^w@vaMZ`|NO2~Tpv#zBq&Mv~G=o_+bS~~NeSVzIZbAse6 zv;3653{82rPUh+8BSUSIbB>VV0P?Fp2X2XLVw@6my5A?hn7CZa7%^$FxiY{xLo7O8 z;?TCH@QE9PHSgDYD_@wJ3Lb6tgMwDa&{1QY+*{PNXFu#o*{H2Cbga3f@XlgG-VJ?o z@q@*>+#A~Z+?y-irOyEUTeUWZjbHAE13t*-04&1WnI#n2r=ii7-hu~XsK_(3q}W@& zIb5Z0QNVVJsrd)1-JvHw{*!15S>F>T14bd#N5^CC0vg%S zRQf{OkpIdtUL0<0FL)p#=igoGLS-CjDbS_%v}(Ue^{VyrC;JTWZd&}Hf_+qOcp0+VbDaiSF`+H*A`|I1o6!f81Sidox7oZo$ z|7EOCxoDd#dwsgZr0gwn*n`Z>w{0~cm;Y~D(0iaIfSn@Ho8|iKt15EB*?Drz@9S;3 z_fF)*LPHNMsUYO;4}yxY#{jSJd>R^TeW2o{Mv!!&qw?i%Qg+0(gGCC`r*9K2i>L)w zF*f(xlCbfMj>r0Dpj5jhkjR~B(EqLz0XzZtNG-!U%Pe(>DcO4pdgXd=o z^w$LP<4>bCZ-1PnM_;o}fiwQI&Fd7g5+0lTiX)E=4Ezb;*Cz%J_gD|SkEH6;JUjZF zsWbQKzLo~x)v&P+8Lez{hLGY>F<=K;7SUgG{U0N~^cY}?Llr8^D?{da76_2z)%wvr zGq2gXKq>`vbCna|Yx^8JV^@PMIyNy0?geIzLryE(T@co;i57aE=iGc$9x=cZRwNOdNOX%XQ$%-J`Br0xC%|x$E!L#%o{yyhCq+85KA1JsO zw-}oO_`;o8YQi70goIzWW{8;+?A(X*yn+DF1fzUs0-v z=~L6d#_p1b%iZ~REgiXc0A70Pl8|z?MfhZG1L(n4KlZav#sFJIh=}#UnqV2f?0z#S z<~Lr!$KM7KiC63+r1S}1*>ghXn={I~;)r7vwNl@jd9w`?^aDBR&UZ7YH@`WC3txF= z@CB!UCa_5nnTu`NfyY`l_0%k=_bLEJSCdPR{Pmor_`J;#(5nMA1NCiL*nXsElXlg_ zLZ6C_+gFf^cV{u#TLX}=)t-_UhQ{K1tG$S@33_pmpd|eIYCkN}F-a|QO!5)lX?(JG z0-dtek4#h676rCeeEemeUl_S3<#BhVRFH}vcV5?+^JH@n9QMUXMjmO|M7M$goo_D< znP4Rc_?gS2AQtaTq2leM$W(x@;%p-T$Mjde&2S$KtCXjI}qsZ*Pms_Tf78pePSW(c%VEQfFc*)D{ z@zS7UJ&WO0^Cx(}sqCAXCpRR&wz9oXc0sQ2Y^GK@J>KCgMgfy?Q zb^2hXWAWPxo9mNQg>80c-CXU;x@qpry6;s(Gxrt=6&sU~5YHmM;$XE=^kw~j^^$A- zwZRYdbfia3DL(+>)&2q-JxMFPe{AG)j~6+(V-1(!onwL=Q|Qmu!IA*S5cu_>f?j18 zVLos7BSSsYxU^j*m+XXx`dj)Euk0hs@Z6_3ZLiGBGSAXaFqXX zsDmaQsB1D#40J5Oi{!bM+`xH3X~dDHQ*;cl=`j#z9aFG~#ro{)N_Ofc4ZkcvJBUwY zl)eA07ZG=VcbbDRwv|2`!-oGhMGX5xQM%!vI%LWw!I)n6I*Mb8(kk3v6ePh!%X zGG>iS%7yRAiFteS>d*G+lJ~lnjN8^xRLoanefz0FAaKb578`Xg9C^w z+oNT#w#Py5Y!d*7OCVui%pA_1mVtIk2|2qeD&$x%sNazwKCkqY2FMtRXEnl<3riiP zPc6f^xcaJ;>yXH&KU?~$;sIMHxMbvV`((u%bzSPMX3*Pn+^oQ#WYi>p{|E&>IhUks zt_of22EE%;(5jBLEbM<)xWcbS7TKpF7JBBfse39?u}8@vZcn0Q;s>3T(P!{+kNzdd2>4;CF8%r6dK%h3g--Cv33&%9YQ?dJgY&6zxt@iZ467Jm zuGy-vV?CSs)zsMUQ&6in2J-{m(yI7lEe#2LRvGX=)$UnR@x~A=)Fq+#&(u^;P;+kr zVJtw`Qt@WJwn;#WI!ZdnKOAKFLDf-J4scN`*Al5dfP+5j}pwx~MTD?ud#^OWyVQY(Ph z(ffKf;eXezbu@$z$Z@+`8fI4uf}JUJ?Dh~Mbgds7Vd%z$%n9=Dnff80x2A|`)3mg66IlOw|D5}lcI2n+QA**? zB&lp~o?Pzl6SF2lMp%J(=~?8_+%j^BbEe?ga!>r_#iq}HNf@EO`eeA=Lp>MB@3`!h zL1^$XkT(y2Shla^ppF)>*^Vht(AFsEg=-ofX`95wO1Q;0X9b1#H>YWpTT&i;p#}V8 z1Re8RdwIwYE#(nE(9)j%Yo3#GXB&tS1ki^|)$z~&+!(;6dgmAw2QosoXSyoP)R`GD z%l>pjM2$GNw*XE5?|N;QzkkKvJUU}@5cFzm1pH=Sg)2BUvlvIqY?fa;yX2S*fLUOB z3k-m_pzk+^s^6^-!9MLsuvu;yA!l<6m$1-O5io;J`g3OjmVBrqSF797?<$*O&+9t# z0*3Ihw+E4*ZveR<5zv@;T|Yb(@%ia*YdxhQ|5;~ok94>)rx+Yzn=TIWNC5^|q{93? z%^gyD1rQTn??{M+`)UE*HjRw2^p!m}b(B7k(6g@8=7*f^u8z27AA!VMhQT4sl8`Ib z8GOaQnpU$uTp4WYDSK?}F1ly#%e(ECfbnr{d`cb_SQFY>PjO&-*^6KMA#cyj5u<)lvy%Vv zOxGm+-xjySrleAFT+2=T@14C{7 zk(ou=QxXcifHt>}L&JtqF;|4yFMr)x;L|r}i0K{`rP8CI`_Jh_Yy!5lJy;pE)mQLf zt2_I?XBwH|zlWo4FMqT-PR-wuvJu z4h$_LC#L3+Qxm`TtC`0Ke)9K$aULuIEU6;oIux`Dx4M?KHHq=-lTcF|?y7N} z?*3!)_uym;Y%(CX`uSMPrhPRu^?v~#2f+W>rqJ>0!{FC0sekVmpQ!6VM%u)r{5?Q_ z@2eQq2Xic_Z-xPJjA2ve1SvO0hzU2m^DNBT2qs-7$hz*BMW+Js5Oru`(@!jH<_Qr0 z_O!L!t!YY;N6BY9CCsve1xkfWhDcr;ta!aW1Pk9Bs($OQ-DMkw#?3I&9&C&vGSmwD3%e`4|E!~-Sj&(HRp^nGh7=eVX59dGH z7|jV-YR`FW6@kM3w&04PuOB}|&zuvo+ z0=7qoPC;($blmx zUfURlNA1eFxIHD8u%{su9qMq!&Uru=44@*cv#c72tbxBYOis3p6OwJSguGoP;1@bX z+P7sAZKORh)SAtEmk0K&g29RNM!s18#@{m8*hY)d_ zli%DU)NMci~FM&{`N0@SH=4lbjsbxM;E^{bQgvFV<90uva+c4!K`PRrri6S z{FguT^PirvjTQ(0cZ=Jyqa>DXixDw^z60!*$NFmK^8Z<9F#cY}R&9u~BUT6VgI7nP zvA}pxM@BAZcacl-EpiBl8V>bD#isnnOe6evos2p$6KYn6As;tK2uVO}W$tR4I(9X+ zJfBwBc%Z8l?y9(Cw}emvFcNuBjm`DU!Bgy_>Ja+`A{^lCAZ=sWJ#&-4)?8WOB0uBq zT1Vki`w;Zq)&M$2B1pg6j}7@{wZAIjRL{hJF_Eed^_YAhRwsKD_h zUyM!tCk8&})WBmN`9I4@s@w&*&#t0ZYJ2gYCuyYtrY?Adgj4i@MrENdbi5^T!m1S-ZkN620s z$ET`VAuqOPsNg-VuxVdIuJq1BB3)8YgliU-0*n`XtmTjnv;yY7PQdkP*o0$)pzhef z6Zreq?=Jw1KS9dc8pOwM4r4#f)s{Xk41e%_a^R)!M$oBu*9TCkORbeJmOILyN*Kwv zWc=(W6a0)@J*1R-h8}#H8GxR(QDnrX2o|*|g1q-CY7)P!5K;eZahp#q^#h0Ix=xRl z=D#OQK3Qh*Pc6c}BXiT}zOk;w2gDVpl3WU`Z}2oHB^B($T;x#ZdVZV_GPpZ z>o6)}siW|Yu|4)X?|eRT}Y| z1y{bSh9@ldfr4Dq)B=x~Q@K0O#~&}ZOr2PoCyx!nCdU*u+1y(G;7HE}Gzcuz(vf#> zt+()zzXyV$qw?u;JNkonw612o6CJw1&Al1VK zqJm%cs0WpHv#BihlBorgv^h@7UGA=az1mk3+>1)QGQ&u{?VcqTIcIt3rG7?gFCpWi zya5!xBjc0YDth&yjtDx?V@rKn6kwN_G`EIcBWOs!MR%&$SXrv`7?cIWsqeMo7;wnxF+VLKXgwr3+;ob@I^uL#d%iDd1Fh?la;}$ zupwg7MF~Imum8@6TGV$%pR$*biH4=iMhi_cyc^HCNNqY~gt@`2+t$u0l!6bQA0*YP6sM%ZOP!CN_!(M$|qg_EQ z+XiC%%1BMb))Xd5UY~i_J4*oV=^8uy=U^OSLYls*{6Tlsr=OdO-hV&G&bdk{2>z9n z`uvQsxirEvhs$(J(aCNpCe1ck6}8-(`)Z*s^0JH?a(1<&{mmZmGus3rt``;gM?XCJ z;%YZ8-Z{ywUL63xond9%n6EDmFb^U^6wNj7mj)>XJE{huLqRHb&SEk=3MS@I%b}kx z@dQUkB7RSUg?rU_kXwn%^{A-OqvfXYy(M9bN6Tldj>D7I$IIS(WYwR%3Uu0H7b3PD zk$O+n2#eYtWfz&-pg|&Z%!RJXkPBkM=bIXC+%@YkDEL4_Maw!1gE(bb_D}$9>PRzi3o*0!0wl) z$dxN2s4TC#mc6Utvv##i%-)iq!LtbDHZ>c$qhiC|n$C%>87_HUM9%jtaB168TIqa! z)vG>4`n4f+;w?)@$wL`G{o!I;an!~*GG`F;`HFoOmVKzFp$~O{?p|Wh4@^|5PmQUt zPx$jFA>+_k*S@E#>$OQ~SpR)Y*8&S=lT(VfG}k@;K0-| z?=yC5_RXD||J8;t)HUsDm^BtaFRslpN@nXpPcyvVOJyI92vf?p$Lsxs zn3b;T*M{cY+bTi)86`m{a5C|ez(XPJgkp$B?^$qgjrksc2@Hc^lVz5t0H|M{9D{CvaKj zK3s%l02R4Dj*hpA$=PdzD1SY+bi0gDC~2yEGs!FX;8NB%dKc>&{d*gMI9cjcP>Vf) zzwMd%LXTD`TuWtv_*MJu_}TpsgRTY2d38@ z64PH;Cs1+n+RPg=cG?9?cSXR)Q2BGW7#x12X4dSg1tj$lJQ13F`xi@VWybEpklww} zD?Ko_3?G|$wTDJF)z2O4Gqjw++RE3QqR2~2ZP4^RIkj|W92;jHCMB3VQSWt4HP7Z~ z6*m;D$^hFaInUS%d97{8xv?#(4qX|nc)ikB^42CoB>{{au{~K6y*yeOB&#icwA9MT zQS#BBw}-34kFDjA z#CIjJXk!TU(LPy|x-o@KZAL}k5Yh7 zrRK_@0Yb*z+}Ah1TkWcd*pvD-3?kM#0gqnpfdp-hl{|Myir-kL%HFu=Xr;@8(6l)q zHg1h0VvqDB)UlC(I5LtTr)EmciJ1&{D-p@gIcW43BZqsSsjWRWG<5s@WnV*s@2l_y zj>)QUQ)m8TIXCIr+93Rsc^H>7L{AS;G~_qBg>^jQ`Jkm0RPuf+~3cs0H ztnCrd$64~{D~q)m57q~&-tEaq1$#YAk%x#lpL=DvzI`nJ-CtHTw~ zt&-|6hn!rxqoCtG8a{JlhEnEI^H`omKFcWwd_>KF9a>nxb2{8QOUd)sJp%Mg#i|JK z{UKufQb)~u5jNq^?&{EsE#P-Igk?duTOm<@v|@52O(HzfttLaP5?r2Ro(XfQ+1NcT z8SL`cbPIKp;13iq2ey=h-hVNqxiHLTFN=wP)(Fqj7#>+ViLXn zbAu8@^41hSUDrp59wy~Kc8=3Y`{oXX$H?RQfV_UVL__+tT#8%8r`Tk5BvBnEs;NBb z`f3}l;$V(mxzbVgaH9|Y&eT@*R7uafU=rkCbBv%quZ|H@*q~1rTB~DjdKHXne_aWe z3=wM?D1J6hPr5QlN;q%oEeqThA)V!TZejliDMW+6W z*NVpUD;p!FLAy!@v=bTq*J49a(l=v$-zlKk4^^b1Jq147GmA~%9)*W5byhu>H$tAY zqcg9M;xjJ0C)v=AK44zjk?+i1n6UL;V*EV6>|qlq;%r_1>+hXo$b`klqCjnZ?j4U9 z5x+Z6EZ>#kvyRlHVt@ZTV0@po?L|*C%|(H3iT@f#eV2U}n`s#Zz47_4cj++MM@C%5 zf7j`Be}8R{jDtEA8PSM!0+WAAPza@ge?0UA=jbg5jJP(WnLv0 zzrVmGoh~s6f3I)^U-Wdu5#ak9!$r^4O_{ef?a+{6O3tGJSnO{tMWH`67lxc^D~mY4 z+))v>FQ>yc#!xx)wRr(jcJ>w947C8rWAQEtDr-lI&fAfZN`N>(K3b+z4oo!s?h>1B zRnZaiT_rEfW6;mXYb~;a<+?7{0*Bz#2$&ut%ir^~`gDnmwu)=gJ77_l_7_N)|L3fm z_V=82%a94SN$|V%;qn*z3z(w+S_OT(3IWkI%d6g*=2zP$7`Yq0xTpng{*6gY+!+NU z;hb~0?Cn+`EY{Ro@nW&I@S1y$n7cew{?XK35U|`;^2F4Y`(U#l`rb3eDE@){RxN2)_3t=F@{S2Qn-^9R`xj-nGh zVoIKS7MtpkVzWF_a?P%s43=?|E^G{dqfazkI-n&$o;gH{ONq<#swq`pO?6#+Y9{@5p#57rS7eE)8O0Z*2@$M*>c1mH=06u?|&6Z4H!$uJ;ta-BWVV8$$#j7FNERVPrj6@52D&FS<8JPW^Kn8T-2k`RP|N z?$d9E+Vtz&1K?Ns{_W^pIppj?oW0+*;bWO~HF!O<~AtS&k%6#FG<-a=AV~UTL zxZD%7|GXiKc%4k6XH?T()KlEkV9M@IbWN0MZ%6hztma$ z*g1ht*jKX&E`U$m3TE}bx|X%Kz$JJj=tP$U^kG|+_1M${@W==}>Quua9;mrw+cc$c zPsOF|sf0}HBsFs#(AnGTED*>; z0jk=9m&-lXaSPpuh+0tMr8aQtW9Kxr+SFa~Y(ZH1Sl-!J=A%c1Ql@i}%;b%dcQ_nDHD zaN7gu_7P^z+kSjTphr%HJ7;k@4iO@Hd$=k_QJ?olM2Nql~E{QhEo%p;gP}kQ{=R(&BY&no@Zs;T&ypBzA=PK0KQl0wwRP>=z&Jnm4%;$ zL_PSv8~N#;ONz|e5S6^PNve}~7ieg!oXhbT1g*zwyk`Fy*nhTY%$1=sfFIa`FP3)Y ziKVvlaGAr~U#1fd4EPfF9Q4!fJhs52AV8LeAn}e_0>}l#A{9I9dM_yW=a!irTobl=JTJzqM)BUKp)PUn#v!p zbYjA{hItUnFr#>om~zR~NlLMg^DAv5=+CZcRE%?$kh=xggjGh)mjPNv*A0vGsOVLD zIy&OOK!F3Ay82*|P_n;BEHidjzEaeeKHeC|WS_3q4LPNx{LPV?i1h)~=aqg;oSawk zWEh!nNh{2M>YgJPH{dgFRmQ$NJKqS)wMnSO^4f$8bL^iU;VQ3^Tx=6zldWTjPb6eW9)@{tP-k=!a=iet@M78dO*E_7`gE{U3U(UjH&i3jYB8%O!%McLy!!Ixl2zsrky4NReX@tkgO;1pZXV5mKmV=A73@oqiH>1tn5CmE zz&!=5Lmwi}IaC$4I}82fn68O%Nsuu+5^R!B!L0Euuu&Pug}wXg4_S>{3JJTU|rx+fjJfSuHp9{zexl7?{^j z@_NYAE==0h?Flx*rxeh1t$B~vhbrFM#OTD85!gqsnpo=7vT@rp)I#$hBG^8Ii2L_e z?cmo{ezQ-_f^UwKQx=-??ydp)b5BEp>>HV6;{ZI^KFg_gN?U3jbB%N5tyZPd`oIQ!g7^iUPe-SRCNbrE3%PZ1Cp~zqZ2TUmdB*m@j4)-6=&S?5VM3$9f9p zP{%;I6l{!jmR>PiTlRdZ50>!n?S|>2B`VUT#H4JBU>~jHRqt(MkT*a)iqZFBLr|$< zKV#Bf|G7L|Rk$NVXV0)A&&oM*m$gmB4|N@t?@T>q?{-z_{6hnWu_G4{=7bs7tzvA+ ziCNI(Qeg@mvye#p96HG+B^P*9#ENel!hWZQ&l_Omzsd`{^~0Wt-+Z!GH+W=X`Fnon zZ;8Nfmxqd;==#C$%oCi7F>1xll_65fo{CzzE3FK3jh6-;Lwr z{%k0C@lzix>Q^x}?_wXd;!^O_>)#<0USD2mhlBxo{)I-6dezXJ6`-lhxIKc2IwQyr z`eg|D`QjX>;5H`V`7i68r6I=(Vwt6__?eoUdQ;8+{8ta;*$>?nuYa7z#{NFT&%T%V z^5S;|@2>yo6(g$rXN|rYsEE@AD%>YVYK1wqcKQh-OQk{xnPn!k?IshMBDuVoR|Cu!NPc?Q%a$L8(pXqYc6oVr5;i{hNe zWw_PkQWx;I+cU(1?P+4Zb&8z5K8%ZAAB6?)$|;pc3&L8DoQt>3u*$b5TdA9)^);4$ zTH;bS;{7Bw{pJWR?!w9dBEmMwD)P)RKrX;0-D(cjqp1__DCs1Ji~zEVG3l!VRqxyi za=BN-#vK3}>_A6B>>HSPpN`MiQF5?*avE5{%fCO0P5En%nRCg~Q~tyw!K58(*(B2# zI!@nT9lRqW7j8}C65L8$mQw*wS{Z|eS|#`_pNfNUN{D&>JrajY9F|8%fqmWLcb;wu zhfjdqacE+3wxty4mXrnAoTOyf#FYG#jfN?=mP6k&@cCcNO`WTag0B8TYH3I#r0Q@3|fz_8EKi|jM>qV+*s+)``#bA>Sf<`_BiQV*`= z0wXW}*B(UXJqa`KZU;Ezr(smsPfZo?zVCuX|0?5`1QZGwcICEBIG>M(A1h47PG@u=MHn2dm})rnp?EODhL4`9%;2(OZh)AitU9kaMnfPtzH4Fu5X5(R#&CxX5#vBjYkk7CWF*QS}dp+^Q@!`2uq+NA)vPS2%!7pdUw z8EC|gidwU$6Yy7uP*H%7fB=R@*v9c0oBb6pH4X81t>cv;b}=?-WtyA?FiEyuLe5F6&jfZfmFbC(BT5wO9O~GFJsktLJx40o({VTl1|IK=Nl^c7iA_IVB!j$ia-MyXkmLaT$=r<& zRtwAT?#>hPWgVo?>Sn~dwIOPTu@m`zZ2zi+#@EMR*G`qnXnO1Dc9@ut!87rLIx*L!+C#|8L=UCyd!9-o|YW`FMQnFf8k=@{z8Ze`Mowb_Cg=J>?S%r<}AM` z;_L_|`35`h<;5WS^QCEe z!u?rV*7MCiWbCdKpKl#Q#jFAWO3ug**y^uI+mZ0d+749O#t1ndSij`0F-GZ*mcBS`~Mn7$2>FjSEVoaR=#&D{QLK~44;mP1vD7itKl*oGCD#k$bGOS zLMIw}P$i0n@~Ev5aEO8%b8WM~_?c%8_KHWRJ=<@$UoGxs3!&%_p6C5{I8haf?{z_!TR|-TYUb7c+rn==RFKQo`l{Zqbb&Xz^ z5Iuc!j9#)kU&pdfax1pSh>51IvPZ_Yyc?$WyqjBt=*X2}TJl00{)43(7w(b@Fy{WM z_Y1=GOS;C)t14mIRofIY-lL#Hy(&K2tE;Wok77P+2QZN?RV^?tE&Y3jrh!8vUwEix zV4VtN@(#cpTT)c|tf1hYeS(pOzL7c_nw&yK3HU4iMs3dTT<|!pv(MBlH5t zTr(Hovw|H-O@y&C_wG`A*3FIKiXgX~T;!Z*RayGsA8Y_Lf6?*T0LPbD#e~$A0c6Nh zXZ5R%QB0KoY?V*VCG2VIYBy(Sg$^m7>`@4r4lx6?DP~t{M)|qYURspA4gGF;n31g= zpr_2#!k>sp(EF1l?mllNO_lIc(_t+Wl&x|1x z&imKRG=z&?pQaaXO9Y5jNi$8_jgM@FC!LqFi-P<$*H*?V-xvqW0+xsTHI1u6U9*(Z zwW&^yqNk~{t}6eUO-v|v%|as91_~c8G-O^Lgh&3;1&KN{f{Hu0)D8+VjMDR<84>5+ z-aP+ZQ`x6$I}`suNAKO2W}3bao7qXGB$;F;ebS5Ri7_T7u_Tt*dq>5NNH5YmQWR8B zQ2{}E?_^*^Mi1U*L9vpf%`g6_{TtN-tAHD$BSlE{kKAf_#>%&K+jO3cc$RAjxQTKKn7{WiN8H}ZY@~7jXyh{#N%9+;6hd=m)r4gR) zhImNE(^IsZh~wY=RLtQJ8T#2nu36SMe&|QV{Kmn8&x z`}N)s9ky-bAVLaQqMMg~xwqlnkM!J!$8?1F3p0J?k))z$e^@!WZ~r;X@A)>%?>d~M zk#>E^qE#C_ZCFwft|6qKn$co1F~zZ$GM`=jaaz$(ykoA9T~d^VFAtPIp6@RT@2Y$9 z6Tk4zPfkw4-L2u~N^D-lNp$M-v%c2q_uu?n*7pEUa5&0k?+sDlTOL%=hOIXGKMPd) z;WWL4Q1RvoBoOFX&owc!G5VGg2)Urhzy?oDql=p+nT?Dkoij`Ik_<_nr zEv;zLKq;IVz^C|nF>m#}x;qvw_`#|co8=c*+&7TYE;|?ncSpqV*He9@tVty^Z@9N5 z$J^VIJ*gsRe)7?9hZ8L3f*F@Jt3{l3oU)0gXHEFQ*A=9J^hLe z8+UTwPsgl}@aR)|I%v2X6TNI>)Er^+iiunZ?0wXcpE6R5Z`e8I50`bsk}(anw3kqP z#o7XXw&$V4kFxF-^|1I=6E<_fKrCFa2}vvN_O@lOR5oep=J^y-NKm5^&lq~R^Ez_= z2rnPtQt*>yP4$a)BO-ZL&#dUDm)~t~N;x~$-Vhn;M}3&=s(m~yD|)c!f@E!Znb=t~ zt9C*|DOmHi&^AUm&1-H_`KkwBv}i|XObygWt(xJ39>iRIBF|MP79j&#|pT<~}FVii;FVkGXm!sKxZc6ov zjaaZ~Am%P;X~l~cR>N2i>h+9*lD(`K!pDK1y@IkE6J4bb=LV`H9sJz$fo|}N4JWNu z$1Q(g<$<5=3^UMsBTVe3lMdPUVT%7b4J-Q`s7+n6*2f(E4+{=y$B>*_7#u*w9|W27 z?O_V)AV@(TPLQCVM)6e}PE79J2(cQ_&xBbe{Oz(1{c*#N&YD)#y$mW~Z}aGb;5@wd=J7H-4(h^T$b z&*L23Ke^`P(PnMbie)Q4cUg}}+Hzp?w>&K*UmN84h!7gJX60cPjEwRbIWc8kOU|9r z5Yk5EqmdqSL3(+d4Rk-9i9X{h|P4@j9>Mr2F z*X)2V^^qEn&ibr7N#MOvHg(yA?TbmB*~f;z7P z6X%lFJe*cxVBe4Kftz7*KaIkQwyjMmn-+B8nw3yJqi%x)anYB_y$paU|iSLjn?aYQ17aP6eW+Rqtw8}L*HhbC75VK@M zW~>a+8izH6EG8o33N9<^_@;?jw`J#H*R0ICO&_gk)kA5RcQEQgHg?0hN5J166SQuQ zvhV=6P>*svcRbYkJs%yuKg_~ydztVBGa+ZZ7Zp9PVN}i=xe%|YA>7vrei)EcJ_EG& zvzdYFs3jvjeZxbo_4hR;`FkLVdm{|ox`PA`_SU=z$g=Joj8wk;9H>j(9VMYR{NkQD zyHqqJXMrcw)Y30g?S`+@t;&NSgS_R1XK#5LleazS!hJundfQ8?9G5|!E*UVH2csm& zo)1^DeF;@}gV!luX}F)C@( z-6jYbnUynxwA^_Ey==xnDVsI1VY4<7d(PP@oHB50?IOzi36runXzUlwTR9M5zsi+G7iRDd-j4Y!mXn-PBS9y?gUS~=A%KjM%m=m5e8=3$$%{Qn3#=GHf_bj zLK+#Mt78gu-in7$UbK*kHvrwRVkdzYEkw|wg$~&q74@!!QXVo0;F}(?MBEb5(vC)D4=ERvnV12CQ+Ao0?{)UfBUh{HrQ@ZBtNewP(2{7^N z-ZtK_-pd{)!+ zVcv*MTeH&|wmeMqs;#*&q-l%>ykFv`qc(OpB30eUjUmxNz=IlJ&xVa4*G_=8Ii~?9tB^OMX z)#?$QoEJ2R;LAXrhxEAYMI$b6+s8m{`uUu7AD6H(+(O%*>2z*Qarql#{MOB>wt=l# zkzr?6IIugx;jQ{;$T=GY6x5J%gBpILSxio8X~;Yec)RlL0FSpZ#v+ES)as2PI(B7< zjSIP`Rm(#R#D<^WHKuHg8CTUstoZ=#ZKG6ecv<+p310iRg*MAkUcj1{23vNrP}>2i zbK1acn$i%>*1a9=ZY8UX0ZqA75q0JGj+u)4G%gG7j7p75Za!r~PcIr(uyTA-cB)?`sN^;j zo^7j1zqhDm(0Bdf?(N|g$_~K$N1qALo7gpzDj|4O(OT~Arlfe=Yo3IR$n-@E8MJ6( z)~+~sgb5YDdR*U48P&GKf?8_MqNNo-W0G=a?d|LtGq-VAhD?}N)9Ypp(w2Zhr*QPi zvF`TPtOa!^e#0nbF6(;*AxkH7N{`N7uvEU?A0syG4KdK40DRvX?QyNSSxq4gId@)5 zE8TR_>vw&W+AR;MX35TI2x_Rgvt~AA(JG=%C`f6eqUzh5R$A?vi;Y?La`5XOCTz*U zsGrb~vc~{zzU3AQkMi+91vvEWQ7UrL)m*aTVWQSOqPCE+m1ytaq?o!ySwn;E(0PYM zyyg~*SF9XZPzI0jwpKn^vEWO$d~8xkM=ZDVs%{5*8Y4diTV>nht=$`bE_K<}f?2XL zYnQCV!Z{-%arG#3Xq4aT>Vd~Qr0|$oODA_k-A^CW@oQ#|Y8|-tgjs-?F?R{(%za(f zHg+bfqV(c~j!)Q_5UVx-%-@}9x9rZesn2>jS)V5Z-Ro{4m5-3oMO(n zwO0~y^=VhUJ^D#guf#0tsvKzRN*xeL6J0%>6}A?_v+-VL$+}0{zdhQi-U*2Mb|(0p zOG9+{Mu3Umo)8c2OiI1Gp-#`%1cMebBQoYpn1ZDt4r$!NuJ!cMQ{27O#GtAY&Wg=Me?N~K7YnSD4usRQOo}Jt}URixBjOM*rZi& zGiYlVQ@k6X;daNxgPTFAWl~4TnKM!9w*&2i+d)C+PJm4dS!m!f4L)VoO|IRZqLB{f zSskl>261he&)%78mG6!T+IGf?jk`fQe#P4%_7C(RJ$+idwabheXq9KEC@s$e(mwdC zN=_e^anr4>v=`n^e)3emv~EhR5&8x-LPt+)p^=Y!?G_+X4#qn4I{~qL#m6K2we&ov zlAGfgkYfxzgHX9Znjk^oZrhmD{AFEd>yAg{=S;ksprWJFChpBP@Osj;yxuG! zmJyCCDg1|;S|7RT?vih~dsM6LUj3L>40p==;HIv=3P-=JepIJwHAwsGG;Ok6dyfe- zJm?wpsjdBHnU$q%(N@UW$_!#X=^m*J`JfZTh#bWAeKZRPD?Odv;Lso+`7Ov{|2@SO z9?c5b4vOSHMSG>ALjiJiDyzL+ZF#e1X8DvJ`*FcSDqr{TIV(V4k7~sTOB>^zg-Luo z+QTR`3!ur912o8-u@yUO;bE6t0``7Tr2Ib9ZU%Y`z2QO?9Za@#9_`Vhx1()%fLUS? zVxP*1_?vE-q-NUM$zJz~+L!Gth+odg^+}1Tvjg;!B|z)ih47~V87V#3hs~Z*6N@%| z0?v|)Pn@&1K&SK~#IQ^tyFF1HBG- ziDe%XvET)?nwyUw(Hr_)8k>AbZ|t+nduw|{@+1LI8N(pU9^)IOm+%n5^~%(W-S?_Iy5!{J7Kd2q85gB$XOb^D%D&p46zeyGEau8Em$)$6(HV4$sUAZY&Lo#q zSo^!d4tZDYgr$o)Zk37L@*bF{Plp5sBXz>m zMhaTmXhVba!fBU~y)(h$EVyuGQwBouvP;~tHQH@mA88X$xq0YumxSXrnG{CZ2v;E< zLrS>bJP}R*R*aQB?kB2}R19NYH$(THfRf&UfG9Vq)h#b2WL2hw-kojj5M___^9w?T zHuRdm&pYSoQ;*qtg(kJENvSZx<%)5xR5AwT(FP;sG;N|qXsNY{25a5ot|E_21Jkw{ z%ler1G!Du74pe}9)<|dnj73EADmeLec@IR>V?fAd zZfswV4=ZFFQ(8#cm#qv(nnXC>C}i6+Db4E0$^!BwZ~^(^hh)@Gj~*8si@2WrL)NRR z)3k=(XcgC8>5>e3#iQm1=6xPXnj#h$@`%myHN?sp16m1o!m3odjCQ?BGb|O$hmj0% zD4!{9riH`quvJaVR(_8u!{SHD;uMf$S2Fh^sp%>t~{_E8Z zsyk@vcvMxL=6u8p@DER)mYm2d5uVN|>-qW7GjQ0&D=A@@uV;n5PM{vU@}MRx@(u1d z7Grzb+c!+|nblo$L!I4I4ldHGm$j%>zP66uSprQkozZ}{-;7Hn|Mc`()ro}n*k6ju zlqb{kWk0_CKs|QvaaGu>IKr{$WcINu_o~D0MBu~nLDpXz8ckPe&B`ZAv7$)VAxaw= zU}XCTC~3|C8A{qcisrP>mXZa5#3sD$PGPy|QYA!lr3PZTkeERD$Kyvu$1Y!r3A=wU zJM4KBEbRB|C1EE{+zGpLC-oS(PIdW1D)Z#MN0q13(wncz+xse)M#LTa6Ro}LLtNUp zUQqAR8M@3myG%1U+}$e=vL*7h7CL`21J!K0hsJ2a31rU;6yEs+4DR|f=5^)Cj11zL zl;qlT8A*9}6s_#?C7TdG=Vl>|S_P@2*HXn0PvzoUX5;Eef$;2V<>`b>@()QFlz+T> z-xPM^Sy9+8=U#^056=sGlRym%e+dn{d?!0B^&{#8gQ$F}?9hO`>dr=YUrV-4H9$}( z?L>Z;zk(!OPDe18A{!}z@CuaYLUMlFKVHSc!rmv7en`vW|Kn{EHS9)2RoL&qwJzN) z414kt8TK#=f9&Ba)Q{zrg7ZD?rUIuz4xhD1ICD0cWYp5lGpgD_J@WoyvSc&^gde|J z1RXk)S7$$+kS{&{IF1tb$4yY!?FUt1F^P<0Pu`Nku03uHyB&cDi%#Jmi%J&$@aQf5 z#H$3>DLBxpGJ!O0+S%DQFQ;FHUAtQvc0U>ucJ6xFu~+Z#XRtW^2Yugg z=ZrNpMS#QZJ*^D8a5e4Nt2fXeXt=&RG79CsvlSV&VrM{g-NKwMiKein zeKa2{SbPs$^F<0sap6%+^G}c7VvZ%G3w}t>5d7n14Ei4z@8lnQ{swjI1MpZ>JSFVU za8TIe_~v6DE3GH8YTUmeXu&sK5=)tDup6;BB2}$Ubq!hU`hoVoX%e-4r4YfHih>b* zcWWSq^Z8|+r=Gut9RKrf*|D2XYQyeE)P>!BRvY##w)yz8MCS1~*|OtV)t2)x^4QZB zQLvz|%L}y+YPn+?e&wu%0rn{fi3$Pz8IP`f!EPPRq=@IUVT`5cx#g0(ut_#<{!_8=N|j{%!{zw&uV~rqyLay-11v9tm~b;!`R@~ z%87F}0V1HLWQ?fDnQk>JcW_{!vAuT`%4qX^#L)*Iz%aT$D`1WrZ!&lQ7d8BF?`d{e zb^-sV%rg0}aoOS%&*BI_$9-h{QC*|BO~lI{tJ*oQ#|G#rGkRkBlAQ()S$U`l6Sv+Z zXQ#+z?HQeNJA~CaR*0inBe4|QGZ@kJAg5Y&J~EN~>*Lq;KgK2#jwfbuj;9u~e=n$$ zUM)r_|Ab&ow`mm5yH2hv*C5rwmYhOr$kvqZS2xAGbs~6AyA0GRH8qH3?mDK>om7WZ zU;mKXcKT%!>!&AgaK|1;L4kUxI(GGbL0DuQ?pS0JJM7Any0D7?M?Fgtge8=^e=2UU z+#|D`IVP2(b>80DHa3V(@F<`!hIH%_mCAq-^bD1=`lhNdJsZWfw3(+Bjkfckdc&FY zBGJzmZWV@Ixm^-=|2Z`5dU$2n#cSDNQE!pQ(z1oe3yQ=iTS)5HuHKH4Sp%bd!A`9* zwBufx6}?b-uNlJRXkRhgX9|fOtNAeYOk6q2@Tjm_`e#*R-{l63?q*7P*X8@su%Dyi z;m4BmcqbBz+s~&~w%uzaX(AZ{$H%^YH)hz-37vDY>Ndu>jP1$RzF|uTQLS+bItGGN zmegNHW$WLNDe}h@*1&TFK@nb3*B_o)-gY~!n0Y6wgmnwfu)TpW?XgIvIl4utiqj3s zDxJC>*pN<&m~*ysru3A&SrZ{IsBKOk)-Vc{?Sd3pm!i0<&sf10s-v5k%7<`Jhrvs-jHl zh*?wWn%B!VT*dk@za?ntpy_l*kwoRA@@2!-WZ}qrICJhzHE!fqInr@4yF&i+-Dma3 zPX3V+cI_S*z~# z!V}vw7(7c1j;MJE$EqJy!xT3^7IJ=zO~zfw$miXxg7iG$Ftq7*se>}AH7OR{vOd6n zV@7%#U)n`wcl`2(s9}9eg+Z&tN+o?+R08vMGmLqOEu6@2qIjap>ZLc*GU`tuU~N~> z%>%b#DCG@ovo?muRp%)r`f5vmHz;JIg7$`pHT%Q(@(l+jXI_s_TQD?dcxAX~o19-{ z(5o4P8WpNvrYMqiX-g!XmSPT97mXwLK7?b$_fXjG2RMT46<;7t5O+yqJG%Rl2c()R zw^Bz8nR_Xl9wF(|C7=#xbt-ZeN(9^OLHy|qgFKoe?m=7bXZddWg64e1$`^L8P|ujf|` zNNKP7`gJvu9w)d{YAuxXTdKt}EvQwjNEApkX|xV+7KAYHx*pEEgQ3Wtv86Up_kg-^ zaIm-BZs>sqZ5`OX0IT`i9G(8pMYibcEJrY=qvqN5eN=-<*J>~g(lq_u#VSc>nxd;c zrMIU$UDj`?k}36&o_-BjDm4^!54efFMhkgR*8|pepkGcJ=tXmOK5E{<$1d8r2u~;c z<)W2Rvo^$Itc(aaQ!WwRqi?I%_K*{MxHZ?L0?hr6K6O5=&5+ncP~5AhTk=a0?MYBV zPduU3mOyB=yk~V9KD0;%<2w}EGOOERo*V7z-WaAdt~j6{*FCuUz42D%-ju|>IMOxf zvdBBtntpVjVxX$MTUQ|J8-=plec2?THyH}xYF>r-&%^?Hcup1f1)R!@=~T-wPOF6A zSK-qF?L|=*%DdZ(*5)Gr0I?{j<-x`b9q3_AYqeY1QQ=Wb>Yd8YN>!WitwN;8>=zq~ zdqjqERhJ%O?jEf3_V$1#2PBYXjR>~m67%={t>P^Y9WtXLWX-Edm3tlzd(+EAP8#VY zUR6u6Swc%Ow38BSdc8SLCSO|jED#QPaD zqh`U^Ynu1;>c&k%Y+hL%6@MtE$rURPkmz6zoO3Lrr zyAe?~Wj{tKv!gnIS{3zr8r%AY;bcy4R9y|`ObrNnhKQlu5VrCn2m0h-n^H{*=z6L1 zT1L^fn^3XsB7(mRGtsksRqtHfLs6|>WSaLsT2{bo|CIm&ijo_^88pw_qesC z!81rrH_M3e1_|+vRnABnG6`x!K|XyoNQLfBw}`)l__D)UCgbxcw(M&VU-vmkXxjGS z%V+e>xl_K5_Fe6+d+2dQ-4PA-|T36aw-nQX?> zN?Y}_39DWPY0gY8TyasVmK~VF1#Lsjtg`O;{ty}UeO7GRI{L2NORk=?(JF&#oCB0lj@z%X z&UFzbq6Qu2OQUH!tsD7@xx!z3+e1ehf~ zwS0B}{%)kb_)eg+^8TWZRUR@5;ll&;k^ySNvn`8M{Mp~$e>ln@t{duN7L?$}3u^GQ zRTDg6!-~tC)nT$UnAvPVj;^_fo z{6rt}&7=yKFx-QFJEJ2O&gpSEb2?1&rjuB4lp(!pgnwAjHYLmfYd@oCdO0mezMWE2 z^JWLoX?spg;iscBG#+yG!7!m}+lf7zYg)QLL`SVU81)M}253@=&zMl)if0E=31hPQ z$T1oGZBW^qIBpUl{p$9{5d}AQN<+z7bM(lU?PB4&i`cNBuY5P(pL1u&g2?+c)I$F{ z-rDmmC{lbq%9-*)GZqG`-|je>nEl}v&c2_`U2~wp^Y-S_DZ7{yv`LvB4IgZ4N4=Ue zuxi(>?4~tKGkDR`l)K}`g7&<)vVAWd3S3YAHOK{eomxGwMJ0@MLL;%}(h1cfvu)+-&rDj%D`)snWBs|cSe``ioc=zAI zX2^f1nY@2ZvP7Ro@C~0Q7_@J59qupVE$vI%>UWdvIX46C`L{v?6%n7uo1usEy#DPm z8uH5&1^b^>-oW=MwrIoBR5GtBj-F9u-wLTgFAhiX;J?O+jek!wINN>}J7gx8uDYp> z`xBzU{h1EO{TO;=mUiEXb0TQM*4(MSmS5 z z$bpwowd*8TY*?Tv^UAuHlil@?7gex#TQ+>*L|@&j zk#^9{1s&?+l)NEj-GEA7Q#V9SigPc{b{76Q(^Vcm3p{5*gH5%w(oQc9)+TN{Xwca{ zY}&dGojTuN|8ia16!*!7Oxd#{KZfLW&m7#Ga|%@QsV!4;(f2Vf>tKiqTQ|c#e)2Yh zc5K+}Z4)Ya&qXZv%ORjV_4Yx$ULYE*OxRpN0r^hd~l_%Z4r3Fu)V{?M=!5 z7sNOG-z=R4ta0I{qlK{V7YcX0bl9#F3tBZn;@2(k!?jXIGP1*X-eF&W3vAm=kmWzu>~P*g|-HjULEHjLn?Nfq?{nwx?AGRf=sS4gP(JjLN{ z2UyhYVH$eJjV}E%N~quU5~_DRWcaF`2;P}&5$sLzJCJl+M8uxdW8zPb@^b$4wPgH0r>%>3Nx-pv(BumTHfr^b zskvZX(-^;`D0&oU!Uwz1|&u&cVo()rU zl;N=DVj(tdEacyQ8uqUssc~J^5-Ba~&RZXtiF_9`hJK^cu z#;T|lWo1OLJ?}=SxAF^mxiyKq1oR8T;7hYxpYGZ`>;At z^KxASj@UFn;=T@18V=meRr7rnuMb>^bYRW$mIfL=>{(!Gb3KI-YlD^Xi~VI0TiUAV zuUK9+f zF?9g%5&oKF(ssS1#>K(f1f154@WUohf^)0{fSQ7_i3@= z{|hl%zK_5PmUZCQ^8*#n*GI&qw#idkJOk{^}?NG5e1%c0A~BKep_zQB>h4Ps4{5 zP1)l~arVVYVea`wIqdC3KR#)^v+lu$8u@-v(HOI&Zb)3wG{&x|AW>5i(CzV#iimY} zbH%zNTX~g!lCzK+sa|8I(d|9b@Aw5|rlF7@Yx2U}Av{CB>k_tR*Lbi+=B zd>+DAei=iR@A;7B`yNEWUqKA$a0r<{-(UQ6Qc`&9^DqT7zXhIBro1dg@@M|8A!jUfs44K{LJ8&*$Z}5zDID zms_TWw<`nSC-dD^4?}WzoRW}pp{Mb~$=UAu=xr0RVAFt24=L(jdRW=#Ea=#ii#_Fc zCS~wQ1*!O2ebR$t7D8$G7c04J+fe%^Aj-JlX1_mUC4V^MYbm+CVWgCO8{tqkjo6GW z6DDKVNCj;V;!_W8#Nsto!y7C0{b>#E^@(9#)&*eSUoYz*v0h&K?;G027yz#`*3^}c zhd3#hrX{&gR@CKDGd-nuCOXSs9(u(c>pC%eOoVtn%!5QvNo(F7y5U&|j{5h@TF~>L zq$Iot_Tj7)6932F0XF07(Htvl?K>;?)ulny)3Zx`sh4-nN9R>hDfrxbKiXSzBEF0? zgV#-vSS>#BG$tkd7i@C)PwK|FpO^c=0H7tX1x z-c597UtLvKMXV?xua{K~(L0XDlqDlHSq4u%%Pxre5$NU2ZCg{(OkeT+nO@NSd0?Nn z40W;F)&>B-VR`H3`dAMq{ld63C*uFj6N%p^a5VrwKR?PT473&9@^Q1TeH$kszW{vZ zla{?SQ!{@5I!Yz{Ynsm4wt(YTR5>?S2g+Y9DL_%X=IX>xuDXQ(&XMuX4p0&q6!qI! zUw!I7=i5DBg51_^8#;eMS#Z-ukN>Ts^5H3WOZtN^0RoIs_Vm=41QhedORo9qC06X4 z!7n~}Fhu~r=kGb-$@B938$*KFliiT}KlC?7otjcMy%%FLPn#%ZcNbOIq&X!#dP-LF z*u^cn>15?yaxwD$7#CLDUeaI^+`{tvK5p*iZ3`-6MTz>bs)WDWw>HKDwHdvxsd)+T z?!&Rxf=e3)z~`88nQPjH=m|;cxsjHblO9UquVbvNbHHofkICy^_aWkc@$;*rfSN4} z^?{!R#rf9<@d>B-Rj*FAf}S0>kkfykkkvldG4pSlX!$qh6p-jKap@f=J?(r@oO9hL zD7ZF=Pd?RE6MJHa3yst>3hob*-k%HgRzz;-8k4uQh}3OkbN-qRmcD5M$1NyI?gCma zc266i=KQ&i;O;X)NgaWpl36kMK_et z4;SUNug;8dk}vKWYNG%0;YvQc2^BLk=(}MK==zREM4XWzlDBjXA2#)sF9V|7n?X_j zy~X|-pvUpqe+4+KO)ENQQ4M{)(2q%2S3q9xXiLJE<%RcVdy4Lica}X~HxTRB&5VX2 zan+rf-Xoof%bVz^i||tN&ISbKcl@oz5BzQUH;2WA_qQzg%55_`Z?*??Z@R1W@oZOB z^o*q9;dE!-wN>R&ep=y6zcBaDnQn08ft!wWaf@%*scGi{9Df1y_{(KgW6YwW;ms!C zErLCu$CAdlKbAG{^mP*^Q-w=E$IgHDLr>M46DDlpX&- zhr`X_T|2fg*i{~Br6gW)(o+6#)3VQbXlds@x!?tqvX^JO(Mb~Fr7#M4e%^e^kW()fSR5)tbbMDo6yST)mK@nl9)^khy_8nN70kqlt|n?PIc z-KnnZ%l~hQMA-Ks%O<)@?`)du6Anj(qC-EQ_wN}7^YaKS8}I!_PhLD|&CnRPZK-{~XNRP1+aO7Rr>Xf0a7+*U)ya{LvX=*df84V-7arB^ffrx8 z<3N1;->gG@Xo=l4JFHH%HuC4UeKHJp8K8$qc-PGe^ewh-3qqdFYA|EgR+^D$Z zmX4hBr=TkCtfDFI@2<(vlbUP;Zqq9x{BT^vNF42jC0h-%*+m(;!bQH6~ANsoE| z(-1rFQm~tn&aO*3!>@gNcFTmX{tRf0O&t{Q9`&yR;=CJ^ojF&Ql_ht#Es*#TY3(Cc z!RzD0{PNgcz;lkZ7lbeL0{W+?;KmT^<9R9g1#s=Rzfbmq9(6$9p4Ot0F0S<>6J|tZ zch}@a_jYuk$WV7`L{RkMu9uT|arJ1Xl^mHzEsFe2jr(xpz=^5ewWA95%#ioH#`@R| z4dflLSDC{dc{g3cf_}Pao1aLlSw5{Nwi=A?NLJIhl zp4wMCHfqC)9-B2t$oRbYgO^=&-ObK>?C&Uv zne8i!+|-u6{(BTr@$YFOd_fD3-*yne0483Lfg?|tU@{8CjQ?43M-n8wYhXljX}Ou42#En6PH3irBE!#~zHLL4VJ&Tlaj;^()4@ z*nJPCH8rdcy*Wo>yi*nC&gR^T~^=KJ?SkJ2p%fz>#^Mj%Kk~(5XkC z(R}tX5C=|TDWI)W7YFJhr@AY`=j07fSJlYGEjy!jUXM>*)HTITc2z$a6M^n5siAMT z0bjprgvCurN^UOqRY!gc2s)0u^+G@Nm8T{BqNgSC;*z{Ja@&B(Jap5V7EP!u8!z|r zjJzp%N#C61V&?yD#3i0G;*-t}prTJ{F|UuiC?9`ck~PNfSSTe{TK0vboT5z&scfzf z8M6j>55RZ6@pppnkG5CdT2`SFRso_5$dGTnZI}-ZZhb^RRQ+U5R{Lyi5dMBvK}s8H zuX{4l4SDy))52IY!xPpt(AaG~1-vy#t6l7aM9#^|?o9U--ks@2-M#0Nc^?Q4vk9YiGeH)jD@k#b?jg3KJ@rJN14CZ3s*WM2vL z;?FPlXFuH1)uenKqN04VhF7h%AI{hrB`^OK;C1dgVCmDH`Bx{}axctvgKp3CR6ZK( zu6k|}mfx=Y`1p8T(%o}^4Pi?Uz3BX1YjwXU>yv%pSIg@9VORS$_97Uf(BsUXpF^17ILSuMcH$OJPh>ynO?^!r3t;e(x{&sNN+)Zq^B?Xj-9 z7wcw3)~dWDd{UHoWn7s5=ZvKI`l`0>zpY8Ix8&*>S`3|2acM{wG8x|7VUw_~#S`vg?862W6SpMmiGCE$eIJ z?7X~aUwh8eogsA9{s5#?1T$jN@(1Hg9e%GsD2dcs*U*SqSo!&kf|G5Sj@T(9`A3$-_3I(z+LiUIuJYPb`?|f=8~XxizzcwEWNj;%!J#6EWJF_h+cD;?X+5 zPg^Eb@@F5RVntK^az#}iw`b$xrn_-D6QbICZdTqEeRJIJc1q&e&rUXS)5XP1D6k)G z^rG`yT1?t!K$jl)aK#(ehFA}`=&FNWcFn~qzPfBgrmfi#AD6ZD05@YxzmK-Czj>*R zK}p&3eq`KPBPI31Oh5SPj-wg$-QU9gew68{tczIH;|mWweAcd=UOU|ljdr)zJ-6_y z!@H2#CtX4?poOtz%bL;$3#$D4A!W%c;QhW`Ra8CSGdFyg>MxHpP_r*6fc^w@ZPgbK zDt}&Cuq-rshj zixl*{JK4|9hmkU$oZK|wYYu!Q$m(EuZP}8=%nX(}4l|os7PBnNvdkdKVwPn{s=E6= z^Iyyy<<9fWGxKU*%v$4hmv@$yEafNZx6j#U@9*aa=Mb(xqGo-550UbTYl4&?kZ@{3 zGo-xjarmv}xAj+@V=dQ$N=!yjOD@wtMce>g_?2sdS{za13d2fl#g2)F*s*YNn`6`h z&F$tBW32K&ZBMaVx29=@YvY8Qwl~o8BkZhi`&qd^ZA);au~{Z9X6AO^;2dKX`Q*GNkBSdl=_jVTrb+p(QFv-tiqCdU zpl{AUEBoH|s_IAE^XB6|Ij6xs$tYWVfj+<4Pq+@^5AZ9sFesxGxk0X(dt84)!p;4| z(o@)t9et13EhXnUM+w(mW0YL;UC62JacZeo&aC%HIkm|-bj%N$zS$V0{NW`g^|#8l(yzVKth$XMMv?6~_FU8?q6Bp8W}lK)?UrD3f(m>Q zV9OlmC^^GDA%>Y>&~K~_P;z5B0XnFpmM2UsSWL%7IL2@{*ZUzS9mB2X-4o>etua>V z3v$t)lXHYRCz&PoVayf#0PIXeB|vWh?QMA6_|phA=l3BcyK#GlUbzl@a7ai6i*Haj zXYa!=SRcbv{L19{M_%(95O*Fa7Z%iunaTAejZaWnL;UKirbqC?{RN@mgH6DT8<+&& z46$;#A8~PWng?GP;u98LlXLZb@Y7?QiZ7Du9?jl?o$)J0D3E{iH%3q?5xsyEo$V4O z*DA8Tgq)Iy^8Y+Szw$fBK+W0RIdbj65|3_qgT4gn`uU(tfZQ>5igqmR!i0qh^Q+J~ z>jU+t*IqWBh-!sox158W5|w@L9H$hz4~^!m95aQ~ZY20bHu#Zq4`d zj*@Q{o;IBYda@>@<}^E}sU?!0%I_3+Yf^j)e(PF4@y6Qgmg9yy<=;-QZ~e{kpzOyT z1Enc$<0~6I%-vHo`fji>f#fpOg<#+=d)C-fg-R zlCkQ2)5M&p9%z=2^V;IlH-`|{0&;qldy1U92J*R7Sn{WkiUUus@2=>s`Fi$2?br4( zOzQdw^5Xo9=2M0{HOC`b4s>ITn!h?i&PuMo8PG86?IXb7cj(K$X=<5Y#&33xi<+EM z0?7PZT)OoYG2N@^>+mV>^hEXDqJ!nOwup{gX&-`JT6qaMX`iB2FH8xLuf>gLhWJ(A z_@qLlbCg!%8X@NTCQ14BQC!yM80MN|nwS~Vx6>E;Ddnbzh^yA;xJ=_Cc-G3RmaLct zQ?|cIfkw?_i2iZiw+qi&)5Gdc;kHzW_ox~5@p%@?t-$1M4mYI)X3(YaQ{%5X_KPsUrq_Her+GFIrpzM0e^RnOY}(anK6@y8#3@n z5tESQQE?kLXXr&M6>HdE6Me?*1*_O zcGiBSX)pgepdbJqpjLV1q>6}{P4vwQ$j{lO$NEJLmlsF)%}f34Qu8yyMO|O(PZDm$ zSL*JnlZiPF;r+6>V|Rf^ajJ<$TMAOqFLrU??jn!9J&Dg-0QFDkiQ z*U)R{pF>Wr48T+OR(fB?7P=pLbv*L&80Px&C_a6C2zu7?xbU;hw`G6dc-!!UeS}@= zlC)#ihPgFc6YMh242bO!!VT{hc@ZdB8e( z>qD#()m_Ms>Ykdv`6R+7zeL#LpAtgs{p8FS*z`~DK+bkaI5P$R@Fyo8^z zJ!*E?R*{9$_^iLqi76)-RgN8@_0sWKVh`vX# zGY@Kkj#3epk&41HT1^PlhD|vKW_*iIw+#@B;>Hf1a}1ekA8I+f`L5xor5BAS0t%qp z=7f~6QB0TJg&)`5!=$@sx&cFq(K~uxYfz0ZTz*%3d}FBTj9*45O&A57IMDRgclb2t z46XEFQ7np^Ik<=hU*wuAQsET~$9wduvktN!qUweG2->whY~pegHh~yuu^zvkT(|VD<;=<;?2J#(M*0lxg4Iz{relU$?A9|| zqE;biW0an|HbBjEj?)Xlm~R3)s21p>vLMJuK?56-T;F+X6q~v=j0E+f`E+2a=}bt9 zycUp=3Ia+I+Bv~0@hVvjp*cR@t)Z5?rML|HaMQ`P*X4iT8iZd6NV;%A#a*#;stvjD zhMu=N!YFr0DaBh8xb&6R$g|6@TTW_vD!x#67JcRhx-P00Q#QuvWtL~~6NbA@e;=e5 zeD;Qz`?tA!&F9=>RyPa;#7-ZuUyZ7=vt-ktybNZ!UiE}`Cao_eNV|>^j(#I({>j8#rCN9 zyQl(J9MQCstS^aas@|%BtFM8L#as4v!5^tSlco_ zeVAAGouUhp8B_{MYlE0H%j25AFTRGH+M0k}^r=u8dkX?4s7ue|^WAsWCuv1-Tkj+L z6su|e8SJ#WtN8Ofb>IElIPK=AORri}qt>3sYswD3qOaw9>nn6xPy-mrz^8}we0osN zrv|lrtOMjAx02Bmo9pb@m}FIK1D;LHGco&%Jm$et=Y#m1koR#rDYt%cL|DBvc86%| z7uT(iuuI~z0s?4zQgp7Pb8n?@aDV;IL}WoIbgJpK>ywOPkGx0ZRd(G0`3|;YVPWG7 z)RxT&ME3d!A~$FxK@ux%T^m3btWSwx9^Ji%(UtB8iM2Md129Rz(8c%blJhmO8y!>d ztLww{C%h6=T2#wwh#I)9u4!Tx;P|pI_4}Zk=*vL+6W%X% z-VPZ!tv)5QHaV}Q8!*fI0Q{=;anqIcepG>DoCn#QdEB=qf84hz>B6s#F!C0jBTt+9 z%0HiGW&gWEkoVW9f>;qZb&J<0dhwQiQQ3SyuW)&Qox3^A$k`ku-Bk4y|Lqm>%74@F z%D?gq;BJJJ^s2Q{(#@4&W|>FPh6c6tmhMjJ7Z1^=fAbXk^KZ1>*Z&xw=Tlue5!u#{ zPum>AT=Pn4m7s>DZBHRmy)!MB&5sNJzT97T+OGmRNZZX`9--V?9mb_?j8IB8N2$d@ z4Yl@QkZajN@mI^Qo6iD$2()3#G3%4ol#r6soYZ6?4Ygu@urb~8u<*OZ z=QTgMKwe$w=addHa{ee66@BHOWK{ab2)DNTp;tVk@NCBjI#bqO`8UIZ=I@<=xy-Mz z8QSie?;WG9XXI$g33$6)V-r5j8Q}?j>=b5hRuN;7V?bD13>&wRDmM0CzEl*o6 zuJ%LI{BnGC!ptEh7Q|u~$e$}y#7wW|cDql_LxnZu>X4dTxzZ0g?^dF6HYX9+oYR=h z9Ww`QdkZ_gK90+d&5M{pJ-x*z!xe50G@Y|Ot^UFMsPfzGDMn?)&?VfSVKpkdYre3L z60-tIMtwku%X@F7!;|d-^cX&QP3Xont3GV)eY`xzD4TsjxVG3&$(euAa@jpas*D-L z+@P_$XJL$4VH+W31Px+l(8xut48gD3r_fp3QbKV+%YwvaSuJ~u;`aD_SMLhYB->J4 z{{C`uT+nJmN@|5qj?4B+lWQ+A>U{d1u5I;`mve9EIhI$*vw$~<|8(4$++Awxj(|Ls zSQH6DW+pmp;gI9=LSB6C_5;m5M9PGy>>H1Y()g>r{lVT!TW4&6%kgON6>cr9K42EH z;tQR(cNg#Whvx+J@H`vk1$reuf9Ki9Yu?lyB%hL65>(={{8H>K*Elv4=-Eu?81$l7 zLN1Q$I~dzjq|BuO_!-r`;x9Gb8J}*xXg$3H^1;4^gAV`=5Ymf?UR4jrF)e7+J*Yi4 zNKXAXZP%?oJBCpiUd{cxOEY)qYg66mtw~YC)(Ed^t)FmX71UtIK5$P z)&W}?yGvM`yhGla5H@TMQ!_S($Qg4_nv&OOo!NW=KfnD3dC@V5y0qF)ytXwdgnE>1 z7}pdjXYNt!g}3yqF9&F8f8BUfaWXc`ggSs8vQM+hdpK9D|;>k2jxR zd{J@2InjJIW@NP_=S*B0qvwD+-sD$5c<5EMlK}fN5@@L(Hma_qsh#EDDXmPWU8c=7 zNrY^SF`BHe5f_*GVW(n7M$11o`-XRITyoURgdVI21p5mtYE(-BTAPM&P7_NFk0C$U z22j_7CN4f?;Sl_KT3u+4j`ZosP0}!e1hfp;+9*0bXk@h>7}@w=mT$j^ zntOY_njVf*!YFo5AXBU_YrlI-z5Khk%{ z1oEnHto6e7NYi=Sv)r$AcW(SKHY*@)C)e41$H`rON6T7z(Q+;{3A_GJGo|U*B@TIM zs`aY!QRNSF&s#2fq@4u&L^~Ad*JAr9`ii2v@H5Mk#xs8DZGm6< zxaN{_AzpPmX?=oKr0r|^Q7bO{e0uvtMyeaz0GXqI+$(U6E z-Q5RKl1o% zV@yY_1Kd=0ZCKM&C0`B|Ojyrn`9o-9qgluJB(;2T)K#u{f7&P#4 z&PiP6Onctfvd+v;om24ZyEYMM7kd3$M*g2C+Dd-{n!D8xSl6q-X0Hr1Us!nyJqI}a zN=V0q#w;D}%l-It%Ma6-?s&w=LIZ~59jtXf z-&^W@7%|X+{-;%hbR6j3Vn>%(#YHc@L0+DJQum$ZapgBI39-PZCRYRgA{@-|sUbD7 z)~7;b@7h?{Z3UrZYoO`W;={ThHU|)CUNx=RXJC`SSl}g=#qD+tq0sainG!JdJ=$Hq zKmOj{`P64j&I3rR^=bGZ*9(ZCE{nn%dUf2$M+ejbxKqNdwokIl=bp75*Y-7kt?q96 zR@qVWo$V?7k~euhsVccnI`Ni$l2y7j$ts(DT=R`geDimj?%Y4kfx5EZ-*5&n8FYJE z40DW#s(ljhJdV1uHB|FmV5SblOW8N(7x{lt-7on2;#+81K+(-|PPf6fM(H_j3BTSW z=R>?wYMKAz`h>WfJ{;s^WWvluMhuinuNt2-`>gDXxkm+GB(M99 zS-RT2st$@%Lde^mY`Gd!<4Y4JJ~pgp!$TToll6JiX$e2$^N4}m;L|f398z+wPfjlN z%PB<>ExRQ)%O`mhc*M)R_-5aCWtLpMhsl^wgQvwQlK+jGHS6; z%WX~GQ{$8nvqL&I%%^6<0t!m$j)79YYo-&nBy7n13lh-d%(5L@&y)2Ta)tF>^Eu1& z+7nB!AQwVIa-COLs{g2J|2M6;ERA9O}b>l_OUkzP_-`n1y^0sx|ox0bgY-Ml#Py37Fu75eZ z-ySRp+TU9kq_DEJ#4}!faecJ*=inTllU#p$ag`%ApefD&D} z-VZwy(D9G~jS%IL3!pA3yJl^elC?Cz$X-s)?K~*1aY{N+eoY72qv5wM4WlnDyo8-t zA0pq}p5fK44N^&qclBpoQ`nq79d~A6NyX+jsWE~j! z#J#yTjz`6)U3!O2*&L^2E)KMuoO@9Exo;A8%Rf!XjVkFi2}2tx2Kv}HO)U=0&@1*V zBBo1*rTtv@B*r#C^ zT~fp~$0YI=VB_+{Jdfa3&?~|^X5+y;5415MFCfL-a*Q=!3@S)~y_vA6rkxkj-+#0? zh`+Wu$*N7(Qje5V=a-;vc|bgDkHW63ylgs|(2H673%!q`CNaw`rxt;+*yvXEEOg%s&GvLhP3^pxDXF26S&oT@bMI{&Qq;&H`c-U%Tgip2PB3eI`o7z4buZtq z7NPtqR)cd2ljEAfWJiqbR#1c3AsxNJGegXaXp{G~b+!l9q!Rl$^or$4!QZVfnok8a z0`lHs&%@AySTKIO_AAXZz;|jfDP$Hi!v;{BH3HB z;tPCsXqJU`De0A2d7eO4a4HIr|jx!3@2Kgo4nJ#S1)WHIESm;(!OF>@Fu#Z!7oD+gt=d`fF zKEWz>Oi;3&W9Z9aBelw}qgQWF<8Q6^!_O(jg`XAQL^>Yoce zcR$SaJ&YN;ctFdQxiv!Ema+}EGDN!W979~$8iZa1<0LI2qt@)`JJ}JfhyZG3qjQE< zxHH$z3nb6MjKVH1zp46u@ooK?DRJgs0Fz(xDFx`|X?BgRpOzQa_jbq4-JLN5y)mG| zlbb{?#?zbR|B=Xx&|GZx5P8h2EAbqIrCjD%*}`O$^Wo}Fb; zo7*Jj&W&-(wXX@6r~44s{D$7UF>7~kIJuUUic!Bg){>F1uu1z%Z5=zd2U8w(8-4j5 z^yKOgDmA{;ao??`*9Ua;+JKH)vlIJj%?@w&X`AD~%kP|nvQM)!ib;riR zhk@P-&x=@rSw0Rh4Kg~*L%&~Z>vYJ7mHLOx-+N^|7+_}3!K$$99~&HD*hqqSRiG}< zQR_df3B~HiuzXI{x!>uY)EE7-)%ondyE*)TidG5o3D&Re=K0iOkZ(B6eqC}611=;w zCuT?I+8Az?0JbgXH@W13=B){C<;oB(e{+Ibv@K)TE)SBgFZV;w*~j45lE?D=Ix)$w z5g@z@VXIHoN%d>Gz~@CER=M@b^RE#DALWwsAnRkeblVH)59Wu>Kg>UdpR$iol6tEV z=%~5~VCUGJm=m5Ak&|nHhIGV2pBxMFA|~AfV$dt+)cI7L#wh3ykb@!CrzKx(4wQXk zA1?pKFY9OH^~~V1qk0i_ zXR-H5a<0*}DSAQFETq^auxs||`cnZl62urTb8`gbp9#q6xQW@aKFz6J9>b=CJPwKK zsWm|vJY&y{Ee~t3`RgMs=hr|y1_3`O=QJPhX#U1INiPd0@9UEzu7~A_48WNgs{_O| z`vkqfuM`vG=APcINpc=w?Q;PY3+_|0A)rrc0Ee?cSdUOHf5B3##c2 zQ9}p!z$WB@F;-%GS^LA%8|XQQgbKfhO#j;;Bm3*15>ph`5^4fcLP^5FYB{io7%?NC z;8W9TBFX!=)Sb+&5#(i{K{7llT)uafi;pjeyWX$$zJ9+UzGIi;%X|h7Cb^b^TgyWF z^!Sq1Da4i4DfBfr(CP8nkLyO=1#!iXn3MN5J$M(f^*q{N7V;xnLRCNwI9x_8amxs~ zJ98}j{#xfF)dR?n0WH1dmkm+({wmO0t3pxO%4gWs+}6c0daheZt@_7S@4H_%x?hH8 zIf$@HK-gOq3f^yt`}S5v!W}CUv8}@8tWV*y+-epkVZHr0Zt3m>oFVk7$kp3Ya%Ip! zgC=ZjY|tdY8(!jXPWRNF^=e4iUpCl`pc;BDpu!i%jcn+SiHnNsnN7Q3?EPZt>e{z< zb;ZnL2H>9j<$lzKt#L}WM*!%Qgv8y*`=3#;wRy~M;nt-aC?vi$sJW4Ujr{FgDWaI+Bf?fvld67#( zFWS)wiTl=0(3WD3SI=qosM{%ibzhrXE`o1M>80CJV(!WS`l5Y;aKo!$RD#cyt@gvt zZ4M)^_!N|4z~_zjadOVuASQL`9rScSN-a%Tx;g{Kc9ug*E3r?J^So*v%%=x>0F1M> z30jf$9rDuB5H1aH8UpBgVY02`~Iomq+laE;*w-W)@Ks z^BnBbtBP;!qxC0U)9|ZnBZx}@HMztMcxrWqQ$6>Fm=XoQ8&EQ#YlF?FH-{kS{IZYt zR@TP!eEhDV?|#g{!-9A(2DP;|JloC=Xa$IaMF#A^MsJRq=yeB6JnH^pXHUq`$(esi zOmj6wVQNkrQNTFI}% zWV)s38|!ai=V$Mi{%z?sq#MyV`(oWdbPu|dr&-)du)`*H!9+cvIBF75_pNkPOouCUP1T**9IigGyV!YW zT}7*tJg7Y8QZuMvJaG2r_=FD&fT=BX`0AV5`crNY8~T{GrK@%?XQWuBlAqCPfM>$_M=P1ZE`3C(B!(f5iNzV z_`3F%^+nB3iA51FzR0H}A7Kl%(IG{oI{>?fENmQLk~fJt5pQ#hT;S7iaUa(E1`|ts ze%Q!tam)~NHYUhf+j3fIWS)il)hX!wu)$`#h#|Amk*k>GP^OsM~`%%kmEc?7_i*r19J^(p98P6@fdqu@1pE-C@=;=tTLK6$FQYt1qfLabtQt?7Sr)jfIY4+ zzpg#L{xp(qJ+`-bBVMTr8*~b+I`iDQ@9n07GSYWi8j^1_mUcU^m!D2)Fi( z0_yv<`;x#ckG?jFxVSAPlm>MILVT{fD`D%r?N!q1HGL(=)VB-22V=7(pc0W5`{8G| zCP)Rzb%fnB?1lu$hS?wCN`is`t>9ix!A zYvB>zF9~_?S484%1+#8tfRYpaczjHNvrph}_>;%`wG3!b!^Z8;3x%MD-3gd@Xt##Z z5T5USx?}CRmoRb=F&(KitU+gQ4z{MarS%uuwhCzALET#w_a$rs*50zXbH~Q#xfQtDx#zVf=bqGmzbzpZ?JbFT zvcA$E*T-A);ua?2gN;Z1)yhCd6r@($OGr^aGvfre{M-K)(+hDiBfW9QOm7V9@cEl# z4JSNubVk?!w1k<5jm&c3!2gDbfz{+up|e~w(DVZfqct&?Tmzn32>QPe@NA`XnpEPI zQp?{ja4-=)CfBQII`5V>UpxRc?8CBvxdU=ZSR=xEM62IjV8TO24jSk@*tSv#_v+foE>%0;InF8Fct^aU>Ms1! zK2m=!ASV>LrpQHZ6~ARe(S}%_XlvP=;5Ua<9GGhwpJ{v5a%yD|ehGXR(r;*^Y{}>) zTQW|KM|lVI`|EL!`d)X?(7|v_p>CKTSAXl7p;knVZIpzqP2fnbPb_2Cd5xW-;A{s! zd0(+(5_@y)UBd~$8Rvp3UNeX}P7pALQ^KkYs>NiFgjpF;Giv;DLgCgR;>N}x`pV`A zE-k3!HAl5#l5>olyYv!rYOTNdLP(7-2y1Y;0TsHyC52`=rir;;Z8tw??CEewaM??5 zDu3`v$wfXXtHwP+$okd7qkdSu{nkG#q;Jj8OMVqwl)mE zHus|X$JJ42j$cKr0y?xlZss7TI}5&E8f?zoTjWu8XXz0CNZomDd;XUXo3H$~t>(%n zcaZ7FK&~K!KrZ*|XjLIS6=)YiPE^kZEYCq~PU3DlCAbXtG&$QZV^;3Y3#h*=@n|0w z+1R+5-r!eKOZ{?ob=<&1CX6CtNXc%98#u7AhS?M~un;@5TwGAgY6u(H&0ZB9^bw~v zq7!0+KtuU7yyoR`+;zi~#vgq$YT2%thx3A*9M^ZS;wB+rPjW@rh{+D>Fj+|^0QIyy zVr0WS8a7}SW@$(*K>C#|NLWKGntxJtZ2m#r4}f%dZthuoGD*u=d z^f#y{-La+KXD%HdoviPGwFus6R+E34R_2?=X1OIuz)AR=h(=5a>H2Qeo#R*KU%baB z+qP@6YtqRz+4f}rvdzhpU6UuvNai8r6`_XrO*Is*nUT+A{S%t-6 z*qxlcmpkH-f_#zg{QTABYGMQR(PwE#63};ZFhoqld34oBtT%HdiuCeu{kcDjwoel> zQP=7ogkq;X8sS`-B9(?l!eVnE^L;X$sH%9#?G?~Y%eW2MV8x{%oIIYVzKt$D`j#k) z&aAHy&A9CN!!!X|;m%9Vc?9SP%!vkK^m{5%O!JJjD({y3ZOLrkyj<|ztZhS8j_~)m zp^J#EevXOTDbC#B85XLE>0%HKLi^idr|#lYnLRDsWtN47EgRq>?xwEAi@L-2v#>yM7aPS>YriD{wjOndGIoYckyF?32#jgC+%7UI%BE1XVq?vn_s)=m zziktf5Aa7emvc*aq+A70SfkH9KAQT#!;?pdBV5J4)`5w2PjQ(9T$^4O?EhhiWTm_; z0zq9c{!2BCkp3c5l`aa9A31#+tJxw&p?5Ko9esJm}HE{Hi+ud?>iv^S4J$c42lp@>QD9?60`JyrnJfwuz{fa>0VVxRabB2-xHBG1l$5rEF7@Ad^X#SgWnF%rjV>VZYgyaMuVZb zS_OMjH;&Yz>mclTm#8lPMsSI#1Hbn^i%vaMXg8UVRIpp@Q@R5wWzl0iZlmijOWV@? zZ7-%?YpklJrl!PdyweV^r zsffhMIouKTw?VccaKqsIyI$(hx|_6=?Ht3K8}$u$ncX95uf&IST~@d1Ug2J8;$5S3 z19Q15Gy??bSx>qE5?q6S7*7pVfzhrK$Okv+=87w_0BWAttEn{3GWg0OQ(>Ew3J_CY zM6+TOIX1=>zp4SOnv^h|4}>I*Fe_g}w%yUHuyzS&Ul z+gxB?Qtz$McAqcRCb<;!d8EB=o-?;Rb{m1~vN=clpGWS6>F#_JqT4TqCwDtA3Ii|H zJmx5eSxgodX1iOKK_90UBYteCY!C@@A-e7puPM#VM}lx^_r>#~fE_-G`vW3oXB(T<;kU8H%sE+PI~9ZF0Sbk6 zTKlLx=UkgS92#f6jgT31=!C0tZ0STX5gL8k9j%1l*g3s=| z%A?aqB*D1|^SY<^s8;)teo>lBqqu)M3F(=-7iprbYtj^W&EY0^LL9#owWljzYpM7B zwH>f7bfE_$*}VcpZfW9!Rk6RV`Yus|X_AD46=Fah%a#VF4{n@g{Wog-?2*kwszKj;L9XJLT-?|-=4nJ};~#0_b#OczLkqgh0k)2HuZBQM zGbylk)wLG|RS{OcpR8t~mh8X(WUlxCD9y%}xpJ#U7hbvAZNlbS95!F1Mqy!acu>Kv z$QClm)JxZ6<*N2=9YkFXtZDOED?HoLGu*qugVUQAw!7;_A$)G#=dXXI?WLhr$C+_= zQ)vh!&b#zUO}{)rvWhHbp%A#z<-*}2?dBmy zn6;~duc?asTh4^_CO#Do{sqXd>ju?*o;#%SM{Q%BJ9Fc3Af{CUy{9a&H|QdBFlM8T zrEaU&rWw_^I^BqZRo9R!8}{AnbV-Bf*!WNEN{bVM3Kdbiu*d6ETy+lV{7i*DRzgEX z4Uy6h*aWR$4po2kkm)sE&c7YEf?~d@5t|uU2_Au9B1Ir(mKT-BJq&Sfb&RjCQm02=ahI}1Wyhh zr<%aQRC`fIY4g^?5As=MQRQWY`hJ{Dzi{RbjXZ*s$^AY|zH5UqX$le`G4g92BM2auejAaxCmDs|JxkOU8l9f6)D1HDaXs6567CcQS?utR9WdyHAK zN*`Vbs_S2WI!%RwDyh@Jr@Or}7nnjsP-&rwMt-od~b6=z8agRlh ztj|9Y{^s-h4%oN*rQk!pU!r`yp?2HeZEj@Dafh6!cFVuiRs+U4m6C-{OQ(# zgbb{X6^#MT>9vR`Ze)uY#!p7K8j=vTbN_BX$&@leqyKpzaI3ytLnQbSp4J%0CooX_!s@Gvd7PgILyMI~%)Yz>RO|GvE>#PGw|N9?y6 zLQgxhxcM1tQxg=A4N}IBs#zT#{{WepZrcOZU4AjQL6I`Vsrp$`PP9Gln_0sz?X^%? zNB2wnN9cj*%2-&-q$Er4t-p|v7YgF~18#u0AKpvI)_3EJ8rM5od9nVupcU*?tlLLR zoia2eG0-LxUIv>BIw?xidIj#@M!h0rSt)sS0`Kl^Rg1g*U^x^KR2}a$-DQ@K_kgU0 zpSC1(eEG71Q!zSDV&Vj7Q=N3l!{EjBz-g>5AyYjw{V?ePkH2G=7S-1U(c5}QVHOV{ zGtkzWt6b}7(f7avDfV;3{+jL*?b{ypw+A|M@#2mW5qa6j6;gtxXQqpX`f3SW@nWIQ zvd;=Cet5z*M;MNZbqs;9+h;4G!CjI7ACEuMFzqqty!G4B;-lEpMc>-_2OU-2K<|FH zB(zky3!XMMlht7%a-fk=Nf9T-1czs{qRt#0Qd*1c!lcTJ42e^AHq?75P*O}hRYirAWhqd>ns_dmZbP5gl=ElLkv~il)K%^NilM#3g z4?)8tP#|vRb1-?l+R@V8s3fNLPAF?HqGSuZDtELvS1xEHX1FX-5lx^o87N1AfmNm^ z^~Bsl>@Vc7lZdMMvKhp`zN4Wk`%nc5j-UZBmeXjm3RVhtT^H;w-&D+9`pU}pvD%WL z30HauA=tFB9by+l-NX=RLcN7&QJd)f)6H`qHQ@A_p(Ow%AtDlQL6Z(LesNE2A!kb>H zFi-V`xqJ2-=9=JnO!gc4S2-aiOdg66Bf2N@V8Z)8@?B z#ysNeWe^MWEBtSwD=!r+{t3W_GY{fPlJr=)%LINg0H^A0fP?6>hMDDX0G1>ty_AQK z7UN$QR3s7wK?6%AI=L@rGG-5PmaU)4eOJ5C<87=~Y-y0cgO?-JtNlL2#kT32E(B$f zIrl}Tb(4w0ZE=46M~qQW>xCK{L$EFesCXa3&_vx)fh%wEZvV~XoO?gik>AUR7;Rsy z&e$1BkrEch5WVmuCyS}^cwZcx?1I1g@wmck9*$Ecxz5SqaeaV;3mYtid z#q{JHqB7J&fwJ1$0-g^5Q*-=&bDjn_Vep7W9@lSy%jICw=bp#}{p77RA{CCE?0oor zo1ebzZt0*o?uZGh-f|D~G3`faLhcH_x)a53o!&*lU)NyefoXQ;?l(MJ>9sgR;4-0P z3Ewb9Y0_hzeM}+;&&cu+z#PuWKFXj@Q0(U|P&sN{Z7>*3ljS&%QS8HqD4z7&^B`Sm zTEwD3c4(bmBhOEHH8*ZQ&N1d!Vym$U1T{Lx>D5`%EG*ULqR{NHmYL+AH$M=JyQ*ZK zQtoF{If7A3!MFO)A=CivIRIlUSepQ<3&az;*Ot$mq5%f8h3Q#ZBE9<>b^ zobgxjvCS7oX)EMLbALZ|aer^J)zIAaot|9cZWX8o!vgq>(Uq6rX0`T7!i?hB)IP0zbOIk=Jwo?-*j+c@7^nyK-`i=od$!QsJd>9bi~GX&*pc!+H?KuL zD+*ujmqn}=5Y;})Y$o28+^gSt8zH1l}{#PVb%* zhmlP&qSFC}tl9?o%7+tOwZmMEfMWuhgG&N_Z8@mc7;BX&EhmHeYBq)eRmDjk<<|$W zjD~=PSe;f>*G>|9xyZ#yB%}e!Ec&e$>u;*qPor?Hf~md*5rxZjqHHc`-QP?2zvgmt5}sY%ihzmOYAPp}9l z#qQ_WopZQYcVgOLztHvE9pTfw2}gm2cb;y?-^8x%1Ed==r;$HDdz3SH{MF!V2Lz7z zYoE7v2~77vn?F@H@p99t8~n=yZZJM2;aR0W-ZI?@i@d_cER=`fKzwzSH(v5e^D`6I_u zT4znsm5{)OH0FfWz+P0GS`*)x{RM{wos4NTJ6kw6ViYFMl;UOUvTJ*-hD{4YnD1 zMFMcRT$Exw;cAA{?YV$xZyaM`>nVndht>jEF75Z(;sTnMbMN_|%v zoi)shWpc$-hT&}P`Q{bs_GWGQae+(anZVlVx)KDE#o5$vNw|tGR=kKhouuHo>E_a9 zlt7KO57VyKXI44iX3G-pNdU-qlRc~*6QYUu4#(OL+ryNZ53cV=x$zdI!{^ACTMms) z|7XNhOXv5Oj&|O?`N*K)x9uZ>uNjbY(ug=DXOVM$&c)lwk>}gyaPiq>y`HE~GHaJq z&MgDM{i_Hbb5HfFb#T&+V~J0jk$@B-%lil{)ML*l9{c|1mcZ3#3JIU{^4aL00SoCk zVjcnK{o0j3R1uSL#C&${`=cfeRMMzf|DU&rzkEb%AJlzPK15*AAs#LJV&mN-6g?$j z@zMN)ziP)F+Sy&sQ}=gny6R;>! zT4?p-tS;SK5$0Zp)&xa01cOaj_4Iaqjfo4KzpS#E?O zPE?PTWPrA%=6&B+*W3G~uCej---y*1o>sNN&yI(@nu*V)B`sk<+UNB7XI_o4gVJY@ zKdE5d$Dv^n_2aJTEly%tE79lkq4eR$bhDrvf1AQA9p0^zFD7i7)~voevg$L!-STt% zBX@2di};(=^uNZ#~sCIW!+V@A5Z2pCgAKBQ7s>6~?@(bHW$* zi>72fdr4SyfD0=Qcn4xRr7mB-bF80Zm*V`WTo%}B1B71+j(r~QgG$)=%voX{j!ciL zh2eAvtuOaNbicQ*f@kUmVMAm(1arj@JYL*<@W zlAHxiX^GqT(_|dI_u?vfcQ0SuG}`#@XA_t@@)DYqQ1!Abr=$dIj&&b=uFU{2uX~FZ zPOxTtsNrtz@aC0V;sm^L10{#Ijy$~tbz0M_!t#i2?4vnhQd&T>nx(8YzD{7l^w{DGn&b@@>^TT9DdsS(jud{rtT zk-4?aPLvJ$$#POtTf`Bio+Vc+K?G9f`dd;+6)>au`tq)^NIUEr@x8)OZv=h%?r~od z%#)fI5btazp;iN-dn2W9|94791#;CTL*Cq4wBD(;h|jN3u@`}AMuXeD`y6hMi@Wo- z779Pd_IVkPIhlyQIubeySRGtWadyrSSu-6AS_10mzv(JK27tas;=ItUhwoO>^WW5^ zM4H8aU0aV{fN8+174Nx`m*KOubLTsoZ@BYZHCB=3ka-554l`K2;d_lr*z2T$OqL8` z9{Wu*t%icY-W+ZJ+9J%-d{Dcj8(&@`5-aMlraL8%letBlQFh(94Q9hGqfSR7p$Ae$ zYIsGrwvXDQ&@njeP{`)OLAiAN`5qA&Z048j=j{@s721xB2((onu6mmg;LLWh#;4*a z!&X+QLcA{L%W8P5KT;lONf)04{=lgWRZy0aY+QZo^h!Yf2Q8u8*Z+>lLy+KML6RLE znc_m~;lygHXn1UhzsXvLf#okr2Tp1v2UyFE_w^(?Ax`|x&kpjISS(T+nt+M%UM~#w z`qPr+j|2Bu4lSVRFbiTj72Hj1!i3NnarjB)N5qMC(f10blH$bR$w*Q}aU3bJil3Lh z5PAO#D5OAW3WxD_Q2qNsm)uj4b#-zOsx&kicEI%V!*+qrq4&N1L0_AZZrhT>Wfu`x z4BidOJ@r(kn~-$P^aWb=h14q!4$u$zW{+29`SEd&@U?J{(PwmNeu!VO+}e$U><5MJ z>L!3Lr>q&$9>dUwPOLHJ%T~Q&&af9)nv^{57#Ha3^U~0LuVaZkPD88sr$?*Mk9NbQ zzp6V;NR$+1CY%HoBGQr&nQGfGzJ0cQ)=tfBZZgY&9M?X%K}i zF;C8O)2HTq+Dvp#ncU)^bO)i%mzy~sJ;(9SI6l6#pYa8;|DAfKYti)zdSjq2NqFA< zS}!~(xKl}TH^ewd;`sy6kz*;-!ujP*b;0>PKb+b+tCNC{xu!p7Tod_(70~yi{ifh8 zskf^>-a_8|a@gqDYqU~_nC$S&Lo|DLu=}3)2+CXZ4VtV~EOc3yqtDtN*WNnbTK0(s z^p@b9Co8c=D>c7k`*`m3Xy&H*My_d>IuX4xLFkG<-C+VNEU4~g(R+vmOy@47Cg0VS znI*cu6fD6%@amBFDZ!-cNbhF}>XsHXI7cx>iOv&*-)mY~4)K;$NL6S!7;C5uyy-7? zwemHminGyd`U-!kLc|8%S%(#E4UpHxFR3=@$K>faHa+rYsiSvSdTzVRwXcjJ6a)!5j-if;mX_Q7$Hm z=BxA0f;(0MGGrU!&kMxZTy&?{8aw`?k}QKY<1k^|I&iOVAc-}lNP{upD=aCl7}WH2 zU}BEzRaDDiI-R>6P_0aEay8a%D`;~u5rZcFBk9G=Y8wXktRw5@pO+3=4fWnGPN~pM zGqE3+_)q#kb~6>xhOnzFlu#qOu?X{+02W5`w%TCnr$xaz=IQ~9LW&(1yxBV&hw!A}{Uo>1*U- z22r%@%w@ocUa~IYD-siwuc&x(MM;4Ro(eU01xsJCuL4gurgav_WkUN87T3c&$4;oj zfWsc$TNQ9@P)@o!w;d{yqJ7V9&d<(W(h-YRyk;qz9>k5Fw~x1m7uK8>2H}`>M}CPP z+yzWlpCJ6bqm$ckGR(d6QhMH&bZ56*YQc5Ym)77PPecoDA)~ONjT`MO4m11i&9+p z_!ixNkf}#M6nJk4r^R%&B}fH(t1~T_CQ$VvMM@8dZg@i=e(qAte5Ch@O=i9POaGUJ zc#7?*O{1N+qrVJc6hule%A8)^Ur|%L;Ktcw+?|T#VHB~6g+;2OvAXJx@6tjdxqeQ! z*ZlJCq&H#m@)XSlgt$Z0j1Lt zl~FH8#DS{9e3H^!z+&#h@{0}ABn2MMfQ9J}Fe&+(R~W=1H(UzdgK$8=*-km6np5Op zGA{&Z&Nj#DHJYaI)+mK&LNj%JYR}kwv@z`!8agrWsy6Pw7rYn0g2G9*AVF$D^l0uALsn zvb9w6ZsE(F{bjIr{_rUKhh*M)WH{BR#I?HxS1K6Buc+T)vGD!9KsP=|2a-NFV=zjjc@VeL@Yl;eK4~Qvz8{QH;uI zMJz7`=3B#ZfICYM-ltYnw4Yz`s>29Cs_%521vWFMCHmjE1}d<2mGROiD%Zmdx1xWq zq4VDAkFWawFtyB+%zncpZWP)}0MT0FGzlan)<3nPzo`C457=}vJJs0Jy55;0tJTq& zd^v6Qvt`wZz%(sDI;{z_R2{h4Jt*bQ^d?eK<*|(NQ_1)UHeb34_HGqDvD1J?a13jXX;&6WKU!bTSe1`n?3L@o~ zQ}0cd_9PXAMQSd+S+N5)VL}4FIZRt{?VxMPIkASJLSc{kRj>Z%%^{a}|4waNfbPZWV_TorHnKe}& ze|A{;6sIVLqdi*sc&X$^>(EeX{pdD)%kOzbd}#hjjnTHM%`7w*=Nw>2T?D^7%w3fR zwyT~>qt%!vY?2N+a8{dFRVpM41CyKZ=w!4mCn`uMeUAY+QUjDOt}K$%ZhH!MvGY>@ zj|cCh?Rw%*BzqGR-rVW;8Hl+F){$%!I9_R@o;6}ql2S+pe+r8>$&kK9f#6LsV-_|mkH@+MxsVh4k74-`cVH#bj1hyiR~gN z^R8^=m`}!h7?pH;TYq#JhNe|dy-R55#}}tGCF7{D*+SB0qV~h?Ow!73bebwX3!ca- zT;J0x>{OdwvWHW*H+rjCDnJSl8%9BP)tnwzM2Mc7kJ%&2znrKc1Y`xQUdcGCsw1}3 zithN)8(Uyu-P9yfIv|Sbbwb;fmb=E2f#MOIOID%sw+Rs2 z>Cqg~@D9sGcYt2<9ZVkFCs1Om@5$Vn#ltwRzrFu4$gka%jF6gF_G)%{abML08;X zj)KV&ce~3)g)ahmJfjs&5N9~k)hHnDm1%>`ovEMEaU>+Tyama|i-PtXC>e^P*8rT? zGu=&W`FqjV5>2?V1|tduwh>Iov|mrlh6rzJm`VURF&}HOIa5?|;c=X>1$YR5OgQyz zXdxH}Gc8!JGJjWS3p=n<1}REvh%prWnOhGE18#VVFqVkHPF{5OJ-N>sujDL!M?@mj zU_lfj3e|^hl7H@aEfd*dX5jdd8SjpQi|e|&-YF~An=baLfdSBAJXS;cuO?IP>Jb_I z7mggM7NfKk)AbmBKSQoeT05iBY1-63gyB57h1Nd!7e^V!0i}?~4f}+`i&e@7jh~i- z;nYY$zqgG_F*%sqhWcB^_XQt1JW_n`4!dp``4agKE5!(6-1BzB$V_kcGN_ntP^2bQ zMm20>J%)#j4W}mPI61|I<#g9T)VBZ@yD<>{euIum%gD?RR%7ljB%t~3jhN%{&Bf4B zvBUG=gUZWgkhn?N?xJjBq)Ar$dJdc>8bI-@i_s}xxL7h=&O|JZ~S=ZrAr+vrQ+~P zW7YBYjq)=bqj=Sp#bL62x)IeTz6sfc_Ryj=krispCCKyjZ?Pq_k&mOs=K=rg_Gk92 zrf-zI^;#lU|HMBh@a%%b;p`pPqpwY-4+~)P8Ovr5x3mg)s{_{)gIxAZ&)8$-Q?U2b zw;Tr?>;>Q(`BBG>%J+33*513%<{3cYZ3Pe|x?hABlcXcn&k$esPjMZ}+l2eDe9ecN zZc5yGKAq0-P92vIownPM{njfKv~x2}@;30TpPY}a54Ls-EB7jD_)$0aNnf^hh!i5W zAdx~eomFOn7O;u@(`q3fN7qCfvaAh?$tQxRJ@$La>H#Rcd?Spq)6P=``HKX`N|5O! zk2lrIqB;C6MTr>xa;z9f7Zc)sn4&sxCp(6U5KBy5)$gu8`r6rOS+6q*ts%%ibrJnl zu$}^t4q!B$Tv#m))+q&hYk-n8_AoOt49lP2ZtqbG0HdLRE-t9tM$-cWd43ognWL1@ z5I2bGKXol-yr%gzu4HG-M+#rZF~nK%6hy;;DJ)iJOsmnPpO6n{@$_F=UxMBY5n?{t43*#&h60NPn;H`TlW^g0#77}oF@r1r;SXQn zLK@sla}@IINM#gJy`iz_cFFm_sls%0p(Ut6>`}Lyf|!;Dbt$41 z1_w#}AOK;L-e<|D=?pD5H%jxHhdn|=BEY= z28)N6#&^rtc9@Ayy7(b-Bs2nPiBPbz3`1sL|Yr(qtjq#sR!nL8ZI_mf7Cez(alJD z$E8<)sU)ZWCPrGn0VTVQYxPM3v9Sm}#pPBdmiPvm=oZ12t*0ywo+w7AM;CtOXBLvj zJ(g%r4Du*}I=m;K2scoXk-ev~skSWf0GyH#vbd`+Q7H1Bs5!_8YqT^jV>}cS5 zJ`%HdjV7U94-T97DJd=I01X04megu^1Y9;U+d2%8U6_kRePlzP%Bz{p(TqoS$=727 z?!@a82~{S7-j}Mg-O_C|xAv6zxTQJ4<4?q&94Fivw;$EHl?tHb=6>y*_KwnovP73!Q(I)b94fjgDu%V!c*u2UvV-h-F2GA7so`k!u zc4o#x^LdOdVX9G?+>&%m)eK&+5y(_JvqzjFjisynl}TvgM=T!&)XNacXXN-Y_MO-G zcts?= z68d4)3--ig-%fw}ar!i6b=$pQhUBUDY66|^;FIl>d1v+9+jp+KbRViWAT!CPL{oaw zBULpC=b{Ma8=w545WIyOoS+iqm&h4haF$&VbXC@hZDmAJxvwH>V*jh!$;ev~6xANv zo`Yb5@nmUkp|~0~DPs$t)tdcHLF_CzYH|!jHBocgLrH7{u`?&f77-pRz%37i0Pbr! zeX`f^cE%nD#x53+!1b6tqnxtsE+Cfuyc7af7}Ho<8DVG?I^e|XE?7mTZ+>eUqoeX> zCcS{F0@<>YwJ@EQeF9Ex__FWoqi6E-`Oz}<(&4%zbsP@Gg|1F~l;Ms1G$ZMupJ5!# zDHvb?=o)>*QO+g}VTKJwOSH#Lo%#|LKZE=S4@0&7%d2%qyW=NQL13?Gq(9#`P!zY2 z@l=mN5qK&idSc*u%0zrzAC=sLTJE#=dQEz!Y`LYpm{ z!78GQpw>+DOCChZZHjp+WRfuMsry#%LI0PYaPzqNLUZ|L1>RP8!M8%in7MLMnQ(go zwsSop%hVKxy^!q}_$eJxsdxl^Ij$FJVl7rzJeX>j6`1+jp|OT3z=Y2$x8$l55FTe2-6! z;P17y&|tN$Jm^`pQ9wg0_N9)yGD+B>nrS0A*{!#MC(M+l(@yH^dcrDlLM_-!Hci_TyFHg10N&< zUJ<3qUX}!?JLP%WZJLRuuJk%aJk98zJ(~L83|F&w(GvQq4P(Hh+cxX=$ykQmNZ>XA zR34P)%unc8Z_Lezz{`oRA>*LwTz=4IR)yLL_#R=F2FQ519BZxNqZ}d3dg|lCo`R3? zE#gT5cF@Cz2PP&5nwBb9*BVfedNx~$%R_J-sFLJdC%3CMDg4KwN)WI0CE+G(lD?yn z)*cQNjvW4yQrRDTpa}8CPB#0`=`qI|$oUis@9MTA8o~etjOxo){z_AOJZft8$^k&_ zB{wo=in=J7Iyf1S4zrP6_-IEWcLF8bU#w6^m(`gC=64f)XBJYN%Z7-O#=(t+mp0>G zK4v3M9vlZ<>@l!znW5s!&44kX7JKgR&Feybw~l5B9tz%+`6)a3|Fpl|Bq^)}xB0YV zX*MtcDjbS@rMM5482gsI23e)YvlO_*hb=u1=hYIUVj3oDKXf&-bA#K%>nOUU+(?HJ zZN4<;BL+Q8pFaRAPK4J#3Dfe}(M%g~Y3_2rI<)qaw~9vmq!i}LPxQSOy{z_7_DW;f zcZP~1a$Y&HevJzDj{}ike@)5NY|!Hu$$>h#)bFo%3(I><++|!4oq=;nC-FQ+Aityx zv+T*fY&<~*9>hGCbGOOvl#$piTFb!{IVHVrMN}8XLw&+A(%RlN_$PZ2I4^zGCLU0` zFyBD;Hy!nMcG~Y#&)fwqy?e?~C%)QLumH;%rDkG%@5L)G8(G!d2b+|g2>UGu!BlKh zmv^X+D*{BrLXJaFv~on}Y(UNaz;-O>B?T5xE>dl=?k4fx)Iw@tl%EMUdx}80qE%}) zymhE48cX4|*^)0!#%Qamv7>|cK)nHBt4SIL)bY-b-V%ts7Bm9h%N>+!dLe` z!JlE+oiQD^#FNdT^VH(@umCXEzoXSvgENS&`a6@J1v8I;{%Uhd|3ZiP5{n3$E#CmC*Q%Mtfl=c!6F zgx+{rU{r_z%G^_9d&pI$n_2Ii(o;t}5{BC2MS+xoHk+&X$DySV)0pN3{;?nO$wj+Hm*le3Gvs~o6P9!veaE)ukI1xw;0)(aN!M{f zpa+H7!etn@s<*;~OV3WiyZ71{$;kPb+uG!a+&u0|xWdTPH|cMu2=F~pQQq9U$#S!# zo-;QDCn-&o038=lsg?Ng>l|qn`r{qy;305s(aA}L!tQv>fb9VncM(yrXqS?OMEF$Q zgBnboq7%{XGS6b|h63`d=Vo$PjTZet(De>#@PThpOs_+%0SZ^WQ1cDY8)p5|$JfmE zi^wKw4S#NLuQ#NjI#+Kj$+-zXx0YEMXB_@p<@CXKIS&dc9FM-V>)KB~3y*bWmZL9I z8>A{*&;F`6?DE{b+FrEmePW<2+cr*^o%`ZobE^+nC;Qli02#R>(MdY5_@aaJ z`l+9x=1kr>*tcW4%_aDByd(H@|D|Q(J;LjfK&6n`3C`9kFLhZ1;rM!K9ZQ7R_-GhC z3>lw$S6n_q28lpJDfA!3W7Ri3g`6Atj4i`sSBrle#W%%XWrr!swR5b5n>hzqsYEz2 zT{5Q{4Ms-%c*DQ6E<%Hr$J2A0I~sLr4RcXY=RnYDh>;~aO3`XKCok&TA)A1B}x zV^n6iCPK;3QwBD@>74%HIy9)}>6aJ=$YsQkfCfOUm6yIcu@3`(uFJ#2%%eX1sQ0#T zXQlKI$&Tv8+3N1M z;8{qaU*9F}+n<<9oh5u;I9)+=SqKjsYY@Vc??Ow?v$?P8MW6JOj*fW6#B;3HnE{}i zy?z%>nQID?Hdu8)OP=PaASc&1P;<#b*x?e*K<}$5HCJwGmg7x|=$ra5nwd7ThW6~Z6hNGZ z&WroYU};V<~2polv`hmgbF zOK4ZQ$=r@V^;Ko$?_q&jkD=F`|8GNa=`L48OpVLsl|!Bp0aRAO@>hQDFW7l&GgKx$ zAQwAYjeWKe{*>J5Wm2~11B6`}r*Ndno+|;y)GoVMJgAwzFMaxzRJ=uq4f+Zl<1U#A z)#J`pq~QhAVqsVbq-6j5WZ_$T*aJ^w_TXi&V=~ZQEH`s16)JWG2m24}+t7N72^=p* z?J63Mt+MUm^II;c=4S}Fbjm45b4PAej_OX%-4&H{3l!Up0spPpDP{JgY5IPw?YD}G zL&F*k+k4OUqd$5g4u>KV2KmIi-w{gVdfOeOY@30~4}SS~(VovB&9f8=p3c=G-F3mU z*0TL~->*ILtatwU8K`I*+OczyRVLj>9l}5=bqVooz7-}bALbhMjCVFatx+0q*{S)g zL&1NAh@ww6)l_oe<9_4p(>h$5ro`woB(p*qvk&44BkCZ znK~{KABu_nPl?hYDKqj;h{6;q|J*{3Cm+MS^QQSLq}PQH3`@OAJQQ?LB z^c6Ou`8*hWuLJfHoOwh?+Hf*|Uy9i2Qjkbj%XC9v_}LSTU|80-^YSNhHZr`UdUxkT z+N|gh6*swI`=wRlr=@>K8K-K(UxawkCZcn{Z@KGA!CDsUO1I1qoejP|O|7 zTwI;aP3-@7<@n16h3f}78~Oh(1O-{;%^fUVt;l(}xjA_NfdAWvxH6{WwDQopP0Xew z(vz)O<-+qqp11^e=nOnP?EEuo^bsi*5&LOs^M1}iWl`m{zuYc z1UK1`Q42}0Wl)nJKS38S#InktM_qX@qcnB_y6W+%+0&^`)rDF2%I&Y&o&v}ZF5_5w zZpn|t!|hn5R57qMsMOfs9WS}R3kJc5mjQ3aRwH!n*Wkud-mhvss&tPlnc=^7YsB`? z{>NUYEwgmNrriRSSQ0bqNe@;ur^=?*r{{-}w_v$`$7507sHvQ`Dbp`{%k=9IPvuL7 za4kc~fx6A(p~SkoaDC*{aBXcZNqlGLR(a8}d_G?D@W$^(+8Vlf8t7dkH1<*>G-L%p zy4#X`qn)2BdKw3_9a39u;~>@t44bQ#bag>R_FUpFoSH&&pyR5uvF8;z;r%E>|Kn~9 z01rLc9UQi*siz_7z$$y&t^2)8BusHL>^R8xF-w>s1IJox+4bKa_ql5D!BU~m>P*N0 zGJ;dTNT>=Lis`>5Av^~f9EV%As39yB&R4hJ&90N>SScuB%QUL6ITCBp)t%8tZ3Q%E z2;zfdp&jJWk-(=b3A%w#$M0IWSCtn;66s*YZT57%g^Dzjk8S~{7MUIsoHU$Y8!b0? zwwumWaICv&T^?@?{fstKVLoJo22w-Kla<47AvI&)_wOZ516aya|4=kRqkLaRpgW}` z0NISe_$`G;`?S^0+!}k_pHMd$e^HM{F1)ClplZ^qWuND=HUT{L$Jx`#Fqo>$7+HKB zvQp&MPz6a1N*hB<@Ic()K;f}jpZ3v*A->8wG^kR+*F21nIIN?GDC005s5Qg5;iziThko5 z=|&QYK)T<7i3(c*k_J(rN(FUA&qm+OA^0oIg^{m&#nyj4`GnWMPvBoRU9F72ADkag zc)nhm3AWH71bhmOs&@t}jTLpyrB)i~i;Hwo1hHUYoVCdWEn)pJIyrv0=-g~2N^STq z+=KX7$9gFhrr?92f^}h)~oBB=PC|BI+k`i#I z*Vbm&br>H7JE0E{yIwLs2<9(wy5lvLcS)u|o@h3+0Ra$As&Q~Yp|8&rAFkdjlE4co zHTn*=RhPE}PFCz&g=%yOYTYWB{C#xtuREluUGHV_T-+4M57%BxiH%b|iT9^@Jyc;7`7Gj^Zs{i_dqkhV!dh z<8hjNL%$WVkQ!MDAq74U#+FswYyWD0nMlz}1H{EQ5ldGE>&}(#GIiNN_28(h_%}cp z4u>k}b_q>%C8VDV<~kwDmXTZeFBr%9JZszzRK2q4wLn18N^KBcJb0e|Jqp?NSTfWM z2oaD_XbaRu&$ALrZ{nERKNj*9t#Y+2dP{oxx#<5D+%WdM;0t6fkBe`a@#*ZA)zLZJ zbBUUY)e6QOgt!&jbVv1ISoY%I^FGMY=_tzWru*HR!jxWWOP@x(tg~8=m*mf(~i+&G1dB*`ofV;|JQr<>jjUnEa&L)YNtVfx2;#c5x5kAa#=51Fyt{6g5!20OJ6p+tLthP|3r{94U zIM2nKhZ2pis9gZP$Rk%_*76ibJnY~WBxgg=iu*6i=9lQ4l@chn4;&|fQfhOxI+@Gf z5Z|&-|0&0X((|a2B7==KA;zkWTx;Q!dxAy*(j*(CdKye(&nT!Zm5FeF>IL_>50X}+ zAt+kRKdy=>6_PqFt%$eue9!5BeH>+Mi@-GU)YN4ZrmD06R;xh5GrED6GtawJq%R9O zxC0hx!mnnt`XLrK75Uvvy|7n|f#Uh|MO1F?HLS3tJAg^=Vh2K$1S7fepul&OQ&|NB@f4lplZ8GS)5B)mT2?k(x{zX4O51=jmD*6VolLld;yhwpC|`26-XvssaNzXWlDX^um%&_G)HuFDb-F?&)(__uu{k&9wU{o4Ua4py!l+UkTa| zVL4eEHuJ+b?E$&uH~U_cUO9ba?4s%4z3-6?4P06L}9+jkc^de zcAlmJV(1gKBo~x+6v9g3&kHK$|5?)L_F@_}U9%aNpmcM#<0srJYoYIE`qdmd5qea|v*-FR3dMFMIb5iy#emNi-6LBeg)<>}7=-=A=+S!s`&1^T1i15d#q z3#A~36@p#8g>YAYA!_W}i)z)mu#!vf3noG1aqrdJx%uw9`9%ZxqfU*%*T9EFg{VeO zuW=)8HN1zKQ;2lm$w&I1=7W~21;3@2^cqz3Iw-w=3HORyF}JWbYvUD>t}YYezVDy{Cgs+guJf0*P{tk2+OM(r4+T@2wHx{>N0T z=Pu+%J$Sp40=-!o;}k7CZ#!5Bn|q0flDD^{loS>eqPO(D=|zo-z7LHBwQ|VtS94LL z894GPX!QuSFfc~a&y0XaVnJbD+##sS!aN}%qi%CnT7$c<-a&mgMvy??9XbbReI_jm z*M&v&1{`uQ88$=CvRIJ7zz`O`IfsV22e61ik43eLR8ZO)w*rs*lw2|fergxjuT3dx zGM@fBL}+lBUVAGn_m317Eq4k_UAM`Ig7Bw4lQ@b ztznTNdFO6K*ti=Ow{OpI&h9}b`UmvY&r!9s5R$bc9&v37bZ2v%d2(lxbt)H>v=$;- zCCnFxguH@!n${sugjoIuM^ugaoX`}uKMtRP60hvO@O}TLtT9+=-=un`{>e8+jkoyZ9f1%HxV9# zJmpYvwx%RyX=`6sdag@>x>S%aWo>cD04w9o`!iN?RnBSD>_tpn^DpWTY)^)%c z3mSC==+V8HT2Y8e8UI+83$h+r)5@5vBtEBZrtiO#9(gn7B^mSZkpYn!MFw@YdZ zz|r}zR))D{4fK5`Dgf8#)CqI`o&hS@b1NS)U)_zVWN8=s61_(gxQ9vC7veXA$7rR9 z7xFg!e8QZ#a^p4g$mS^PBoW|NB5u|N`aWbXbnPYX@4rv)*Gf1gH5f2{9>xg|sDy9W=+I_b5~?HO@Z3a*U|2y3z)&ZV?ZSVcb{LHdm9e6V`}xVE35_c+B& zy*ZfA-6aitT`fpE@5$d^_3Fu{M zpQW3O_78zRUZoi*4@{2vjVdZ+w&Wmpp@FX6Y^e7Z61*{vh6YE8Xx9xCGA!MXWK1Dh z7_0wgV?xhOI-dB`{=2SJuxkLg>N@XVlWn*4MJ(9+^^Tw`!E|~}4Ms+Y28VMdW_CWrA0Dg2UA5x17VQCu? z6gOgYK8|WsxtMt{<(IXm0>C^AS6@MYLT?%jG#lnYTd1&{othPuBqoFx(sR-(1ellJ zzmL0w<>?vj+2pj~BJNbRC&tB>w@0|gk3eo?WgXJmtW#PGHN)7O z*Q)4sql`zxNxQZ5b50`4We`(t4Uh0BJL&IB!0%1l#8rq}(Y|AqRIH9kiq=L%7k8$( z=WrLlEM*mzCGDchttsK9#I%vS_>y~OWw_(VBf)DE0e%JH z=aiuyc^eTh>M8I7`kpB25Y`4CTz)_Ju;{&vv*ifn--{HT!@YVr=IJuwkQY#|R+J0t zHNapNV08}ke=`*jRFOUjWP2?i_sd#o-o*kYBV=9hWKmPv-FbS)&N!6|$UAf3S@gXy z@S1i!pkP9d9H4b~cSWP8$2kq2voS6>w=>E+{ePEC1G!)Jgz2PYxbG3QV7^YpWSrea zVe?yf<^25eoKI9!0L>;BRMNdAy?J-3`yp`9V|YExgQn*eAlK0AIALjf4)Rh4vKn|m zMcmoljYlltIWi9Ln!d0ItR2#;sIZiQ`=#wEr@Sc>07Q=fnNT0Z!@`I-UhZZ7I)^1oW20rTFUAi`%U+a5QSnj&B8`G%Z z|JpI1xDN4(s)&HN0rQC(K(p$0>>3^(=)RQ)O#z0}5JAwcu#%Sxi5nAB?31fQ9p7yX zGk;op%{#gBlyxXGBQD9z@Gqv}{}Pj&lk2ZKzF!+^`h4X{?Z=dxS4w##4Y*6*L_&VZ z+67k-yQma*N$V(&qCGPsuSr{VEW~Y);a;Nxv8h@!v(n1=B>minup~1lDoM@o&L^h0 zr*Foy3mc=X!&w{WeA>pnkb%0+IE1C!Q=HSD`=#%N9@l)lIR-!P(DM+d zihdq}hdE>oX&dWwVy5jd?h#Zat)jB{grqp{(hJ{07R`e`<4(vEE@^#gigzwK&N`Nw zVjU;kqN=^967s29S%_+ts8iXVoE8>F224h&hCn>FJKch_(5^TgdFYQOgY~^o#HCp)SvCzu{bh&sh17MY*I9l`%-a zxE`_dE+?jV=gELUlco2GLpI3))8e4VctFL*{lc1rt^H`$&c8ysWQ}z1q`X?e=9IK7 z_mB0SbB` zAGHkTVmb{Hpfii4DGBovr}xl?AZx(5P+{7K^j-M{oq-66o5_Hnn(&M2QFt#B*dGUm z$OiN(EZln!yySW=rsU;g3jWWFYDL;DZAb!3Z`qVBE90U|h`VpdUX}+=EC9z)UL`X%%{z~|IF|?)zclBPHPL%a zb1}th{=z313hPuUSTEu zOw2C*ERIhvqwfbHc3E@Yub1O)Wk)_JXV894dlX_76ldpzSKu=hXW$wzx0-?2jZ(yA zR>x;$m55W>fjJc|=^5UIIKB2@6BMrwGY+kfaE`_ym!z%YvedNbGHI7L{2WlS_TYs> zuGt#tJh(l=K7!0i>k46GR{=64^36_%j0f}))_oTD4BK>Madmoqbx^7NFt zDLJWW!5r!i#HMWC9uu6|dc!)Bnd4r_1ra6N7p}16pYsm>}g+vrvyt zn6Qh$>jhU*HtuEm?;Id7@ARH?+#{)f8<8^$eiaL@LrC8X!@QEZq@7!|I>9)WaY~s# zFPH{*!zv-}6V+1@1qTnyIx}u@^$v{9$}9e{`TLyxOOFLdmY+z^xO$pCTzlF6Lmu=# z4_PJ)`ia<>r!gn5cxOsn9DOZ1Isb}tG&3!!fqG!$Y$Da0nP-I?T{gFZjI_M-FP#!L2*_=xE2&am+G z_PF3e@P6}G?*4|qBth#*r?iglv(=%_eRO{;JnZ~x`BBSf$V|sE#M*W&G0nZSJ*n?Z zOzS&guG$h){BtWqtb?o1Ifpi1agGo+SzXSh>WI_N(ocvkrzXW$pr21AM>~GldfD*# z`t!ywR>23hhTHe8zTsVT4wU@o>eGhr3T`DUGs`Plqrc~6-KX2bP2a%x97uy!XJ^&T zxKqOdO{mJw8o9}7BWG(;Q60J8_C@e!Ea7@-4llln?4vgh$a*^qBba z#%tllm;lxo>O8Riig7qSD~29a)S@;`2S(rH z{N>pM#!udxH6H|S)V!ab;9R6N9s%D+y>ieIg9dRL#7V2FdF>VNczCewZ^3H~AJ|Nl z?~aHr{VH;!=7ThNNBoVT2(j_5{BudvMtV6#DPZ=M5#}M!o$5bF@6~<0Jnl#ypW7 zXPuhATltsO7tEhlUkc7_kBEvAW5SCYL(O09yl(wIG1_)8G2VG7{J3M^l<~@M)}FQ> z+4bl|bS*7CtN(26MbpYb=z75}O`f%w{+n4cS-r3BQq+;_m=NREsbFg1LRPYN!(tpE(UY~+m>x4Xa ziS#MhS*N5vZG+rq*K$z!{)HzEpNDVN{v~pw@ssGCmcK{uwElhh5o`Z8jM>hJ;9PcE zR*`%oDcXF&`Em0F>rmuw-Cx0P{~jD@_{;hb?>ONxNK-a-+ZO%2GjLgIhJ9*%xb?fm z$L;%L_nSZSUMqiZRB`fO0|PZ5ZVmBIEIw@hZ1s8DxBkJJzc}=ze{kq4ejmEl^0$?z zoWtS!{3DaPraw;UYJTtRZulS#J`%41{RiQ}+7D7=yz_)p-W+|@dOA5OZ$s?;{W}xtrp2c$&}QD@@U4cA=JZ9s855uV zm)MQE4?Wi!_C4aAeRp}N<>+pNTe~w_|LxXr^EVsAEl^uEA3M4){eI<1{imo+RF#7~ zpLS_jJ2q9z>TA*2*xmLoS01;2v-qI#Q%Bd8cLD=de_kJA90=WI{y3~J{V_hqzJhz@ z^xtafL|uZ4)Lh4r^%wP@E!-`C&p&wiosHM+2ferIze-MsD@ngjnw%Dwtv_%6V&P%M z2kxtv-gznh@n3E;zx*FVic7zH$}fIr{x0+2>NDQa;9$!~p=-?_F5PE+8yIZ)cv5@e zcdrG<|9$>e>;A;3tRgX{s@)n@mMuNu{;>MI^Ly{r>OVTWtKOZz-~RP;;fY_(YRW!J zy^s|p-^fdso(c}f9&q-D1{?pf{;>9MOSh{&+kzb3{E%}mhd^&co>0|%Z+CTZP*8JJlQ}W)p;lh9JjEhP) zp`NziFb}W2WE|W8wq1JK^i}kJ<7eK%()V0_m;M;LU;bfoigljy>ExL?Rm; zT10O)f4uUfi$P1GTw1#Qt&ZX~%cbJ9QtfgP+*DO8#^G zHShG!jGnPMrD=@2G#olxX6&2`%fOR!#xgmYfh?jF0h7rzhyY z{nD{-)_C?ec1y`0eFL?B8P}eF2Q;}7v1z$muO&tCQOV`yC+%PR25bH_t1tb{oUZ)0 zHcQPP?{%L3pW~{E4;LS^kEbS7bz5&lmlhwlezABLG`9=-bFlh@=!3@3;q&}gW>ns^ zK4z8&9`KK@4ogci_MV>gVOhE7YRzBAL}&hGM0oz!x7&~XJOB6x|84$J-N}vNntijn zntH1xle-8~-9UW`y0i{Dv# zQ1dY|C#k%{JMv%g*PJR6cyoP(dwlV6^B3N0SKjpuT>696c=5N6?y5fzGLHPm)!Kdk zy8euHD*T}FlS%ERU%!zR{c2Qx<=^HERlge*pZ)bq_OX8(GZp=ob?W{9&t@q5W(R6I zc$fRV=W6Ajo!zDH*t$x8Kc&9>A7kQk|2ku+`tu`U@%#5%PW=1YOX-L99PI69qFD9l}HO3edd#thdUJyk^>Ag28ioN$PA}9zbC}I~YO^RKUbDndb`yKB; z@^0@B3^S69&0?+jY3|LOE#TK1&4usvRoPEI9dW%9O{eZN#9Jc3gmQ5F4`D1@l^NfV^I5hP>0&#X zqO=F|b=gk=K6p7!kGeHX3BAL~{PNcXEAjQ2ij-~YEArau0d#Mm&}+RT*B#JWH)CJS zvzf}!d&aS{=shh7;#A<1Y=8zWHo@GsI}tuBjl~}~I$%CPzVyB zivM(Zz8>(s!{|sEJ@KIgA9;0}m2hXFI_KF+E5yw%D-AfAL8VRACBN%L`Cr!67kXGE zm4vl!Vw6gd|8^9g_+%8D^gv0AyCcB`-`p7}4B8pV`(ztKM4#y@MgK0dIXgXB@05%m z*VLSdTZWdjr-p`fm$hc_D?=yDOD)RxQt%)i2g(Y}nU-08s6~Tl}^NWIVbo}IVbQ>1uNpJj2Ly@&;b27Sq*+u z6!+e7LaQo%VWwlls1o6zggcW1>w%SU_7!ya8-@uC(O5P;=sL4I%w5 zbwWKasyWdvt=Ra-o!I2}0G8r6I^o|pI!Zs82I1d#`pds9HbLF`kpX}8!+b9TUfbQ! z1N#c-Z0N3pm}VTrgj>bvkn`WE{nDTle$Sv2SGzLaPcjo-t)pD1d4!R<(TxmTY%cn^ z)>h@D}*>PGm=`Kd2)32uK45#!%E$B0Q@H}{qJnFh-JR$2<)tTch&Y;=`;S#E>+thAN*9V+>FyNpv{ z96$yvx8%GuwB@`OlVfj-a(ymMuoGRIaj{QYkg+aCfM*tfOrx(Ud^uO0^=zCLf2Xb> z=r3x5_l4Pp0=EWi{9APNm%o6I^VlK5CLPF$X-7bAvB`>jSK6ZQ&k93sXn2u#*Lq6) z8c@kExCK8SPSv4fEfRK-X#gIyJ5=z=)SLT$wJrUlg!t{6p)J$nP)!D}ca{3D*5|(2 z>4!#c^cF{3hu}Z9JHc;_190E@#-jI(!ZR7O( zeI=*dHpxS852J#BjQVM|I{WDqJMPw^DE{tdN9HTXG%i(EnfPR}8I!O((n#OxW2Y@P z6~39S%6vA#$$6!yfW96iro7u|M`fR?MV$3sQnaoi?de=YlFL$a=5s@H(TAOGY`}Ci z2_z)r|p6A?}E~h>$IeAHy3Nm z6aUro7?#nJ?;Bmk9$Q^-z@L@4t5~TI_0^Eq{j{`4KR>x#5R>9QtPfyf`$@64TZ+E@ zA;pCKHH?Y4(pv2Q*BBw-hIzEapH&=qpPBRdx?P6J*d8eJS#HU8)z@daEHvi6n5oKo zK1q+iH_c9ZYV3vos45A+-CQ08=r1WnUkiQ%=$6X{;JFO#1@C4=X^;C5->)esK@Zlu z^SlPBkxzGr&}si!5Z3G|2;y7~ zf|xQe0ZwpA zF;O;2S;%Z%=Brjj$c+w6#Dgs{Dt=N30rCy>lTA!aJ5Zw2?Gq*8Tm6Nf78=u@&o^ei zbWYNW76%YNmbyy5ImT%@yJBMUPEX+{%OKdpE-CtI1G4Q>Pf4Iv!olq+>iFA(oUDZg zZY&EexhP$Jer`T-T!ews%FrHd`NA9K}3U*?3l zFSPuWd-}@sht_`RSBHX}G22oeqHien3BBMT;{def%LTw<07oXs2`R=RLJfKx$sy;$2isW5`KyQ z10Yw0{JUJ)us==AwE~%UUyhC6n?Oe%Ok$#sl!O$AjFO>mh6gDH`5*TtI7Dqt{<~pZ z&=n0o{Pxa3`b#-4{N_wW$^&C3B0vr3AUDPRy0HhD`FoB_Je{F{52r8*b{RBayDR0* zyfE!4d9n*cJduL{pYQ2?DyN<_<;MfwaL$yoAX|; z_T!R7&>#0lX=z?ZlN`c9%&p%_uTHzbzi7Wd9=|{_{BKP%BX|KK2+8e z_&O(vX;uk5c)l^uW1}nQvuUW)Z;nL7OIV}EoiU2*);L@$q68}+Z_>GOu($M26da+%K zjxY_DeVY;{-<#&8J(%L9J(}P|zcdaNMW}1?{Q3C-*Vl&$FpC%#x&-*&t@c89EkEJm zaCz{b!?>vXa}Bv34gkCC?7*uX=tKZd9LnJgE`NJ4!_6!vB-ln-g-41C?2(Ft*cm|t zn?}(wP9?Y8JdBOo9W3=(ZpePE7o|M~xGv!L91D4%rsf&Ph_OqZgn&bJjbLBJ!|h7Y zk$dOgwH<+YA5Itka4Jdy%m6Q*%nBH0AonhJP{WrSaRFGYx}*{mgl{T%X}$9mFS_ zC7dEa$KOx!(qByTGGEN_G9K^tBEk-&bnsd?GJLKc`gK=afz~vk{99lFmpe**{+tmc zJTVTH`R^-f`3EZCHN_~k%h8FaIy&lXxw6hVBV-w+C| zdFCNXx}2SGZ>cH!opqED2dw+kG(YaXl9lpETUGFStqUIk)KvD7s)G1yfx|k|vG51e z_4Ow+!iv?NQa`mY9jKwyhco=hTU#v!fY!pi?Xn8=Mn5`S*O>WiYXJP|{5};>FW+~E z$%zgLD^F38^KOEk{BeJbf!S9v$~J}yJ?ubcb^x_!A3=v7O3H%vmucemEcT?4QpQY@S*Q;&$pN>_u(s>}LSC;&^uMk30PgMfiMq9zV zrTXN@YHs*#DLwG+R$rdS{&-o~{x~sxeGnTZfCb&1tj&+yo1zx#8uC93ks}|=dGT)- zTe99Sx2C;u$dEth+97@tZu~R*Bmn~Ot8YKf|GI(|a(}Hm|Kpap#CL0`3h_s0D z3Bb90*ytstA8AB9%Q!V_rLEX&y{p7~f}7|%N{YN~mBGXJ^*q*WFD|b*%I)rAZ^_Ri zEw$J#!^N3KQ9n###E&%~OUpTtkB3Mh4-bHBK3P{3GKhq)TvSz?Y|O94Bz9BeNs>hXMCrst88iE>PGF-Iy!@dzaDKXIu@FSpc(VOG=_!dIgqXl89&%sC) z+9_c}_9v)0CmM2raTw~oJp}VLOPM+6-(9dONjY2n#opGz(r;^RxgU3ivOXP6qmuR% zq^#pv7Sb|7&)yoQCvK1M^5&WmUjX0z-P{NF-Drb+05Jb-rL)w3t`qLpgpIyCfKPET z_H**A5_bN4bIGT1Zqh?3Gv?l0ZR)$t4zRCjpe%S_PRZOD!u`~>6uv)DvSC&UDsr(o z$#sMnbbFi?`>+cg_*V})=*C7DDQ&zK?b}!ub$vYO{Swc!MO9(LsDmLtB zhR?LCn5A1|sGkm9O`CJJdDy9B!nS)$zvvsYUudf{pDnf&es;*oS%6OFszs?UJ4Vgk?t}R)G^9UYZpr(&HA0Bl5!15l<3hYeOwC$uDgA03L?;3`OLHo4$>;YN z*1N%M09!s(1v4k^?tOFwurC@YkKb!q3|S1zuJK zGI~!(g<1iVXBt9;?#WPz2MR>gc7MTl6+hlZObNXuqXYh+Hsj^7nvFW1sp8vJLW*Ou zngP^mxlKk)-0FgT);Hw2&$K}O)`oB~tG$@Wzn3fcyE1I*Mn~C~<(BeK#t}@UU5QPx zDKIIP33P&OhK227XZUdQzTGvCP|3&o=24rrT4+)+p&Mdk&}u)-{hvcTi6ONi;= zZVy1!@aVw3nd)Yff{QhDmws98%zw8zko9J%Bjts@DdpqeOT4O+d4BaGkdGW{T-wQ8 zRn37$K-p7r%l71~BFhLhd1rtewbDxnZN(-&oo2wlItLqw=6+t@S_{r^qYW2m>?TB6 zM@T776}QZ$Wuq5?%nSHaze72z)IP=lJH{E%12qR}pQPv86@*mN2sYF_OpG#$SsAV9 zxSJYb(TB};T)=KWDc(Lz&#{O(g@ztVtWE@dx6y@*HjfK2D}$uC>FUB)sw&9m)ecOI zV~mCPtrakBW2mU@o&xWcw#-*6okgGb6nykfKQd^(F8%o^Iqt~>J>#XC2mZF)ga9&= z2rFineW-^--(GIQ<^8)*Rd;^B$TnK!y)#tgwK-fKxhun^p3Troj@0<{h351(n_cu| zi@brm+Q&%I@bjMoJpQO38+Ct-68C6=nf76^85W^y#s1(HM&7R}2)*<7tdPB@zye%e z9AuV2eb;&*pG_0+ScisE?$EH2yK)x9I?e{4*TKOg2L^ayUx&EJZ+8Upb-5|gWv({n ziBpQpI8uw4E8Vnczytbi_LDOYR2)FB*<}_PKHj3h##xlO1m`Th%&}P8W!5ybm?tX` zrZIBbMt^ybaR?RdoTL_=_dqs{B15+ZVSd~F$jC!AkE*XL@mQ)Wcx`AZ@;3J2a*j0> zG^d(bw%m(|R#&CJv`DF0$GWPj6TMJim7(KzhGE|q+rgewH7V~F+Oq==rzy~1z`yNI zkn?oS;CH7wQSA<(scW4GUyUI9nYphtaCa1yxvypu4rVGy&Piq=fcJs}1*2%Ex5#^n z7k7Jt8FP1n6YDzHkmhcmD2+Ss@pfMK#!+&HLs84JO#*n8Qd14>r5GWOBZ0B;_%J_P-tZ_DL3}1tHM?jz$aUM0M|?4@g`YW zta%)hZBq~m&K9|Brn47oMiuMJp;&VB<+*QlQ zyJ^`34=D-Z)(y*l)LE2&dkh7AY!J~SP2G&d^D^gL{5i?bUFZ|rFi~WeKd$W9kr@6Z#)7;HUp06hdP1517 zSNS;4JxL8=OTsTvH6Xvvx8lPbs=E5kVJ6rF_?;saAE$2wzt#zp?{BvkxI0G~S!w!139Yc0H#wI)XRPA4aE z0a)t@3I4Lby!i0~kMz?%EGXIREBD*zEAn7yx`Cs`x?ZQ2N7gkLd|qgU zdINbO$*JPNmRia^m+K2&?Q}yutpjBt4lyqCK*j*?4)VeJ8b-i+iy(Dxq=B?IB0`&b z7)e@T&U3&IrXDYFIR}fiea2}%aYN3?+m_QZtx9&u{&WTYWSU3VQ?N_7hcKaA9VMUk z#t3mIDh}Ktp6ifxmWyrWq@e_zYOpUkna*71_i?V&>dm3FY(dLPtla{wDx2aSEu21|F<@^GI_ zeaylmd9A=M79uxV@FC`Ia`e8W8oS=bDxRq-bTbcOW38ju*gYi;axl#(-dC2zD}`ZC zq^!_8hMw}kgQ?oebByc&0coif;kDjLO4tEB-p(*4Y;%ARZs;KXoZ=LHoMIJyn6Jl& z>zh#C3$5UfbB&oFmpbx%j+eMqztw@R#S4OIx%16O~fw~HJFd% zJow9DCgynursSRwl6egUT$3Cdc7YQgbD<_P?N3f>%oR#(@E?S<;6Eb2z6Qm(y}j53 z%X-!TPkN#hlzVRV(~~SxCe+YPh%|KJe{2tOG8VeHiE<&yvxkIu)rKj1SPdz<%1BPR zgp3HkSm^uh0y`$+5-&CU3L)C>Qr6cm7b4!j0|mWu0foMM3IYeXU+5^$e7M%ZOtMG> z<-77~#@09=uJ0lKm<6!B&{my2U0o3>rlViUPVi7^3J z@DYB0F4m#q_Ej{DW14|>>M9sp6ZrUT87g910uKOu(@&F>4ViBt2We|4z7u?^&ls!B ztp}I+T#y%e9~%1ga#2{|MgP~Xp!W~%fI{9ryGTt4d(eSRdMn}P1Q~h}Ir~#|m~)zg zJ^=7+l+e<5B*b{<6dQG<;}Y#tl$^!RVox={;Mr_7+GnN;=U17RbOY@B^#U}&_X05{ z>M}CY?=n8&$5l#F51Y1!KG1O?G9WAwyF z3U0;=Lq9t8Zv(gX|LqC}SEOt(5I5epQK0vP@VqOuf}BeQQIVi+Lb>+&u-lWg}S{4V+VAQNza)D&};DIlndEE zet?pFzJvTZu|xM0|Wxy{p&KY?i(N$6|OCu6?o0ge!DL>f%LcG@(P^R}+P@a!B zDE^ZN$n%K{=nb$R4{zTF{rSfqpjUTqp6>;Sn-~4(R2|%FOOA}RXpjjjgGFEFng9(@ z(F%`zie*^Rk_&lK>T`$+cA|A{(6@E00Bg5B6i+xN|Y}9H;`IqHRtlyXr{Ei$K+-Fx{-@iQh zLn+AjKhp3RC-cI}y%`?qNXMfB`J-%CPD)!Fz(=SXa9^5`kcX(~$UiZW(HAqleL$WM zAA&yKzXu9>@d6a_?j5l2cR??1-vPNjdJKAS{W|E<#XmsLfcH!O;t8T>M%~lY;l6JS z3m`^mo5-%N5$;a%(d(m(G-X5ihXFR!wW>V*4n8#k*ptu;Ki%Cx9*-Y@zCCjV1;2U) zdJ0_c*5%8f7r=YGy#EMvb_6$14Jn zLB`Bm9c5?hdkCSEHRbQcoU-?Wbht+?H0Lfl=I0*?Uq4^?2)xI0pe8~9{{Qgc9_R{i zmiK`APWMSJ^RjPd$%sS=#}*eml3sGC=P?f!dF=f;4&I?*!Zyc<@so9>Zx!|UFV%?Rhmf$4OKD%f zfs#Bw0a$hc-3PFI|F0_m4xa-1{s?sc`c2SdVBH^|JO{ma_ypwU`Vy2E68a}DIr_nL z9npWSpPO!yHVbU38qsnOIeNaE8ZpyF{Lw}!egaR5xD@*7E$Guz7m&}(*C1D5fA3$t z4&dSr=q-T7+gGps-}S!$XXyFt4an{3b5KF}kBili#6NoA@z=Hn&@sPP_|^M!4D{{{ zpRy^hBpIh_Yu01})Cdpth7ZlXf{2g)Bi7U7Ld+*GU~jxZp3h%^u3!EW^z_C};C-Kf zK0JF0dJUXq(pSF=l&thSB6!C8h0d~IZDW?}%n&<8TtoI86)}89YXspvB(_^ShWxaX z$bK(^(_f-eN^ccK=3Yl7L+?^@VRr|Z^|A7r-m>|g4$9V8BXdVyNw5Q%&pultvP{=j ztxZ;9lmp!4v35q#U^Vq~H;dsVf+JjO$}vwHv0OJ=F5a~`w)ie2CiezDJ@Ym$Dd9Ff zCB?N9h4h}PW2PGh3Gr(^rCwV@*a)+_lCh%{(G6oYghg>9Svy!)E<_~XsfFh~lhlgh zIz+XR90d6-A-D8FSwhZLNO<=BNCXVYPg!=+Ty|b#&`h%_q;GO^IC|H&mZCL-k^xDVIcRH?x4HZ?|@>y zhFnHu!(6DPShpTFbAzXkn@*$qN3a~QBk%rDum4s)nnBi{CIIS z#k-z{dVwt}d5{;CbU7tB>SC~$KY;NUK<(ZK-MM-T^x?TDC^an(S zv?!Tn76mnXU&rShEjNrEuZY?Xm#Uh!)NF)ifcR6@$BgY}lixJs39l+3MGr_>*>}+y z8MjMQQtyO)3;4_Tg9qrRk1r@T;K#*;;GdTxzWZN*q@`V@mlwNE)RRLEgB5VAtcGEg zR}#$%PVv?#Gkc~5`B~pXi(4D7At~A`5^Hc!7j#<8)p&oOD_Iete<3mwZly&8UG;kN z67==0`=#vo+y{wqiMKQ35-wB0#drA#)VppD-e;--`CZP-dbHG48ho-StUq4n@&SLC zyF0}vn6))k>&j}nyp0i3Lqt5p5uQyLXNx?oB zQMoBstFf@xgIsF(!hjHAlvg14HEgt9#}VwPnwz%)1G70^Pg@(S!!PyHqh?z0K4L!n zLl+VHdbPb3zB$;**&1o0SSM;|Hl>JVRtPXg85e4w7IE!!Z7Q3-y3st%MeQg!r6#oi zzdl(3UsG1Vm&dEIi$e`?Z3ioIwv(H%GS(?v6ZcgaM!PDO1{)Ep{q?Zzep>9&Bo}@J z;LD+{WbP{IV5^J`bBxzlI~3icW_5ktrb2+-R*INg@+!h=4=d5oBg|Rut1q7FX)4kT z_w$v#y<|y!Q{t$wKD3KYcdsdh+-X4~Urtn3rL7M)670&x>OEy+{hq3Zzdy~z9q73D z!&w1qZ@QAbHpoa_=p_AIALbNnk5$2!2dfHI9W|-SuBQCS_U24!eM^kIt}bd^$n_tj z;$Ewp>oR0b^~t?Fu6LiPDsp0=p>%z+nrczd@40BQk8rT8DO_e`KT3fRvZ7tdLB#di2yFQ@5sw1YG&Hpmh!U|m;s4F%M z^Yg3d)^@*k)Mp9Th!qMNE#>0{W+DsradC;)cx+R2MBux~pdgYf7fNDd97N ztd#L)Mo>TCgS+_{H)U4`a=fiFPBX~MT%D-IIA;ay|11iG|5>Q0I?_;)TchYm!!RLY zOU%tP$wd_4OsPv!R<^zyAFwe*j@gg`+EtzPe2^S@Ym}Y%V7`x?H8Ub^ndokX$l8Um zQ+=Ys6-g7_G}&0StL76ohmg_gs@VHm!;o*s3tax$dW(K zZ5SD}DZxgsOL19%S13B0s}$@{SM&E&4M6R(phq*Lyu(>av2hZgY*jIf_of?~cgJh_ zPNjfoQt{B66Fjs@Ud?h&SFvnTa=K-R6tmS&h*%q-#B58OE4F}_VYC{#Jywg`6bqp1 z{j>y|j8}FzC1mc(&u55L)}76@%1?9+z0N5X%Bf+N?oKixW*N75N6anQpRQ!=YpOUl zH4o?1)YMoe>jb;1I-ya)DOr~gqBbP>DANQb%{WF(v`lhg^L^aZ<*q8QZL)V_f2DTN zK1)XfwVGj5Q6P2&Dtdbe?CwyalYT7;MfPbP!J_0-_U39@ZS(cL>zanP&FR*j4RuG) zhO&;eI7EqB8WDgEvKsV`k_|c3GcYG}0?wJ9&;Gw2ncLjT^FD~SDN4Rw%fUMrt7-wx zKktW5Jnu&UdO(H0bfTpco#`1U=Mq0u zzdc6HSnVnI*;O*j_h#$b*7`AF`d0A!wL$m~-5@r~Fv&m}r}&H=O%9kEXcD0~M$Ibb-&=*Rkt7VK~jGFoyp3Wr@IPx z*01%d9@|1~_e2vo$gCiipZA(PnHLBTbvzotGtdo5d8n>6=hI?u!Dq)}1=q1sH@E|= z!#>L;9?T1PP8}O{Fh?!gnN*GYnGg2;AD}{F;7##Mg<|kD8t8FC&@Xg=4|%~evH?z#p;g3xyC-HTEyIy z5fe?~Vjr`l%>Vp*?KU+Bu_&e&ja20M>VI^=4 zm}5#LI9+O%AFnh>EpxRUHeG$kuC}JmsH8zHljKaBnho2ZtsovRurU9-UDb2m?%gt{`nlRbwR`bu2%5=U1pK@)a2~dZuob7 zQ@OWQ#wtJ5Rd?MRs5ssb^_;9$HCwc_B6(Aa+sX*0Y=5z~duxIQwoSkz&t}ObXZrJ=%}lgYflAoy z%MREXEDS!>Qj0BeM$ueT?sIK*^8MA$oOgRu*cAIDD*behhC5!UZkr!Qr;iGvT`fu+ z^2|^>YFCglb|-O3n=(w&;t(QccZytgyu_Ml zFpbhv%_I0os|**tHCXm-XRz4EIgXC{1?a%vdP>p1h6>@?qOj(0hK2kMoZUZLe1NM+ znGQ86Yj2tc*^;1RHu_5<{$3Kb9L?1>*(dpE$2c|1*jM(|DZ$2^O;_OU3IST(LJ04` zr@CzS!lQrd1Qq*gE^1fKE&^&5Vv*sq?J8RNo`z4`nXF}0!Bd_zmnC>xWIVQ2$;a5# zgjAaf6=heUW6iR%P?H1_Xq_aaZ^&qwi@o?Ti=59mUZ@{D@6&EoaNxV+@_bQirq9=gQ^u)r`Iclk)r^L&lV4;64)(_gIIGCMrPLW+J66{Wi zDl94v7O2mLvw0rtTt|16*o5=>1m`omZ7O2go|ce(q@|Xe&DFG=%mC}2!X$4F7JZv* zDs(rG;^R)|*!c5VFef^4!SO7$`20SfSw_k@nXjPzz04Py+{MWZk8%ib;DH8{ zdNhMcKbRrr4$+fbWbE9}_VecgWQ0_U0+C=*{=;jzId|L{SuuhWlj^;VUeJ!PMUsqYPH`gRNTB+_nS`#*&t^z+7g|&7q zm#{j34Rxw1`R6lHjbnt^;~4?_Kr3J!=s2XK`KpdzhMKY88~pmc8CtPZLn}UBZk=<^ zH}vl)xp1qJR`9QZPW@j4mH58~2L1o73tIl)IuSN&T&@Ae_zM0X{{g*c3$p zW^u;5|LE!E=X~qX(vg3!Ra777=?IGgmtQs5bulJ# zeSi?TJH{>8o8Xk}O|g&`1ugd+d#7_vgTDY@bF8ZXvLZR%G)_#~l42vx6EvWft3-!$ zJPzPF=yr{$>TtHY=4^%vJ=0R#|!QjszL9BSr*SC+QG<2kwVWDWo6F*3|85 zS#T%7*Ut0#T?$Z|LWGvs``8kqp-f0Rvuh;RbhvJafo znv`84EDgV}5*0=qFEkA7%~jX$0^Z4fK2t@83^$J>BAqiVAm>rjwuUgF7AY0Rk5W~JL^_6+= zj1ptFM{(hY(;U>VIZ^e|w19D_;bNR>;QVBWu-Uq_C!@s3yIX^#9Dr-EV-;C%)uQCL zjwy2SuO%+$pEX|9*>VNPq2*wpa4>_9C#q?Gja0YpeUG4fVLlIJ}K zptrUOdcpP}B5-}Mzj&wu9y{7pFlJiw2CZWt_0{Krr;aJtB=0B{R+ z0vK5zD*7g6h21%S9!T4a38?@Fx?1{Cakde1+;U^d$A#M5=lbgGXLD8QkLRnB?^yyYYXo8r0s-cw} z&(P2PHTP66VE+cP!g=2l>o__Z&|x1Vki9o2NU8c(pw^^Jyjj7(?o3d?Ho(^&&vPlx zDQe-af{^7@(Ms$S)GV76p8)7}?A|1!^!%C7!`Ujqp}wyDaG|Q&q97%kWyGY-UbwG@ z9e-zk6qEXEwzB#F;3TJp4znqkr6)j^IZ$&k#~K#Os=%b0zlw|_0>5aOr3DQ^_BuJ^Dr{jHbKr=>w^c_l*EExtAhHoH8$%| zPk=Zzm?Vp`Fxaff_di?27yh%xqU=sl^A@^`zn{;K*k7ouJpjD@*{Z1R_j*;^(LA|y zPXmwH7%1|ZZ_oRV#FN^IXQsJd1XN{n+z>F2!;FqT2=PrI0udMsxaZni@` zOnp%A)ef+SQW$$*&5yo0U!Co>HA+gGZH0YZZ7ugUN~jQ%j9qFS=faNwy!|sPsx-<7 z+4Abt_Zz*)lrumN9YC$_D$jWV@I0faD0Nl3`(iWno2IG6laur9k*N!pZ0e=N%<%J{ zPIJ@ls@bXc=Bly*4ahVWEcM(y%E}%GW$-67AyHBzsRd|-ANhV*N2GVE{8w{ z-x|jKTT`4;L5U6{{Bn4s|L&hF15{ge& zX}BXmyBt~uO4nKJZ5TqwAI)(IXF$F%DX6)tgP2h37%ti(Mg(v5mjtVYDev@kS?`^4 zM!pm9EC;&E+QWIF0I2nXts!*qYBS``QgzBh=Wuzrc@P%4GgRbvI?o64Y;~nW$v_yo zi$l!AW#MZ*1@24T`5*1d^SM$rJ!cD|mgDojR7;h0r;9A&`TSq&1R~TbLxtPr@UX)v zY=RxoQR@ioyKNL1WFKdOPyTOqJ)>+-g-_cVf(NY+pu-n>P{EzFw9i{&24P=c*J0Ce zaGSvTtw2`T)3D&D^HudH3w1qPW4O4@F=DDyQ&r`dt!>zpF~Qn~(!hm=5iV%T}1^ibeH?-+bKy-P3O3Cy1vdiRe|1D(ejUGIJko; za;9C5jy81Xd1!@k57)a(zOJ?ue=+yMgN)tf-*!3yJT!ydO#{WgHWeYQw>HBAk?!%= z5GLW(*?et>L(M1Hl$>%WpikRkWcbEFso(Z6?EA5n&)uC7asUp>Jf3A@e=YNb!2iXY z#N_1Fwvx|_jd`yY>hnLEMmTvZUGx|=7yQ&b0A&0DLX4>&6}Z_A`>b!ud#S8UeX(KZH;I|20mbczZwN-#flO^`P`1E^q|gqUa*lM;4@$#I)~=zy&r zl&|^xeq9^ZZ@w1&UB<|FpRXzYzB9&yD};H^Wt7OPGyIslj78A=)}j2HO=ZtYd?KPv@AJqgg)l zbdJycy$JBHmR@?SBNv_MD8)xpq|75AXW52Jd=2&KPj@;pUmhzl2{swG)X<9!Hx6SW zZR7YjixiizGlc)S)r<6>toR;&fKPvQKf4d(({a z(|KX_{;a5KTh4>+YU>+qQJFDHn+}&M+xGQ{JSnA-1x_&G`BzJ8X!^T z5ki8#Ce3xVHT$D+u*6R;jDIvCNVqd!m-XyS#l)V>SJqkOq!bIFujd-LEki_F$Fa%h znE1WK-X3%QBoFywIHUW`vskaVihzi{5M+TaPFwws> zEd0Kbi5jIQKUZ^ezB@(**yT2){{$`VfdrR)M?uedqTy#gTW-wt-0nsuZI04%GW?%k zBqx1;veQq)9_y;AH~X_b=&PddEL11lH;qEPe=V|U2eTF2UUsTa6C&W@|JxCE9_dIW z))DB}mFo1z)6|$7Dr)5Qji$_3XG(JJd9G~;gJgv+=UK+AH2K~R^69Ik#z(^&_ zIKeKk$Y|*YYG(09XYp5^F!{+eJMNAc?SFX?;d5z%8SY|~l%)K3hu=EFiFv==l^=3C z&mx`yxIUY&;GOD$>x{#~RNU}eORbr1hdKswZ-$GvOfn!_Qc{9rngu`7ameRA*Z*B0 z7MucJ;&>X9=A1-lIkcqGQ$zLO!7{(zs-+YFxfx~!{Iyw*P1&2l=bbLGNp>w8VNo(5 zd)lhXjS)8LL?`5(&%--XbCI?&R-Rc*j9u-3x@m;@udHGk#59bFnypNJvQU%!?7SC( zf*1YN0C)_$6cKSU$Kx6mobvS{O6>Ww-$xn_`fQ#C|9gdo`FoX4IbFo&Z;uvzF^yuP z4m51+ngp4qZ#kcH&PUs20_>5RhqKG@2>=#eOweO)PLN`*ZcE9jVs7SJ6+h>LL(D4N z6%*rDnhQT}bQFEIN#LQgbr~gm~i!E#sesnvQ>$MXmc9M#(}u`0Yx2_RH0l%ol)PaDN+89(`kCK zzA@Fcsr1Vq)VLQQDdy`nOMkAHLrTo~1!Mw~0+l-3li`0e=-`8UTDucd@6w zs?dFmocOpR+3V6UF7o4lSJ;*NDpJ-~U(qMy5Y*Qu#iSf+P^tgjW^s-d08b-EN39K_ z)Bm&5Fm|G4k|ce3|d6%j{^ zY-Ui$r@Cip1wj=D9#Al<`f3w@6y&}6=K$=}iCJ>|Z)@E}PmeU@%0I1q@n!YAodkfftZ8?6V6nSgG!+Bvv%%P!6as;sH5umf4X?V;+L*69~C*!O^P8nedB@8vpCoKuz?QMS*aAtoDs#3Y+Owvg&*a z9%5HTD%@4UKDlSgUz=L9uR6!j@n6ji9iNwkLZ1>~vEkac-f2u`P>Rp>O94(*U^843 z$k_kY36l^~!e;77M<=Za+1AH|yQ3newta$_+scXx9`qjb{e6{f#J%Fw_#wwpV zCP1%TlK_XyAkkl}T<)%siSa6k`8!$~RP_s8ea0UxpqIb$avy%Dq@@Do0ZP8 zTU|9DPY@FC{8N~H@7v}2tjB*X(lPF7Omav?EZo=9s*em@d`O2c2 ziyYQhOVdC|O@nO=6+QGTph-tYItqBLO5iy#{~Wn&V;KIbX)W9Zb1y=wWyJu>x zezD$HelIXn6&0L?CTveZlGgjnU#r+LzwXFkMd8ndH%BU-Y!27P0k1&{_Z>P|pg{h% z{jYCMYha!RHMEuA^eUNHzqw~&*V?7u2j(>>tpnC`*%Pk>92-&*%Ky(vV|-o0SA;Zm z>Bn|XCjO_6 zs;F;Ut75*TXFmAnJU9KC1JDQ}=*?MJ(&OJY2Jsm|IkS3SNz4n(K%*^PIrru`A1+I1 zF~9EWYqS2lF6s{Vuh^Z(WN!|Eo^6eSU+ih|#hy2U9tMy^{hiw@y*RLezZ%*Tr_084yW-+P1ZBN3}4ono7u089zra9r1sjKkbFd^aU z5bD!qAQz+n9+O}iKx804J@{#@w<794>ulbk318`zf@54WWf1`dzyJ$0c(|vzN5QBD zG~xNSj8w9x6Ea*gsCZj{#Vr{l_q?bm>SPZr=AwHPl4KhJN3qJI&O2ub)q6U6 z-TFvPq@}m=uDPf5hPA)&s$T(m^L3fW-7^Zg4k^8ON5iN+UKTe0Wf!*W8z|tV)}q^5 zPX77jwz6B^SzOw-f?c!NjeHf-@M*v`p|ACpJ?yA_^5X#Hsy^@_PG_59ZrQTvX$W5%l% z3*Ay`$!BwY&#{qB`Lf8u9cZc5Aq6a9Yoy?bcc$w7pLRi;&rsj&mUGejOQMEDE1SBj zAr=P#Z|dxi_t!P`pK{{rzDb5*evSS(*=aM4gd>Typj){rfHMS1yFLuoCFSU*y zTUtgxTLhx7R=xoEDcA$XxxWmsq?TCVQPZpaIuUzo3LEw(?=0<*N7lBQy9Qz2MSFL} z86?X7vS*pze4GX{!U=`Ma`Zq2^#8ZUZuWJYH18S%v%64b}7X?*ITNx(WV zvGK>t9QvV|MGgUe*)xyMv<#Hq_o-@=w$+q!-z*!tukMi?8hYk{_kvm*13zCKExu)H zOTRb_e|36&2p+RChE427#s9wARUL7-z$NeLx#S%!6VO2l#G^oDS%+#~Xhp@>|F(pC zaB_q$C6HUgd~D>>=hn`n=LId1Mwf9q1S^kEHIMwWaX7PLOhCbF}>3 z-)sDK^AP-_Tnvc|EHn=utu@d3m9+A;VZ=MYr@mYp#iZ`)=!noVm$?_N_im(uozvJP zmjsvSo+0Ph!*!P03a+elxs#K@O}8{N93qw{H@P50*srhl};iU#zWD;XZg@EPTO%nU3-s(3wFKp~Pb#6&ps- z9|lR;m%S2dv2%=^sifzfZ!3TPO-u39Z#kxG{l< z+fniGyBZ#G*FY>iFyYI*^MEfHM8`Pi*mWLdqhNW2l4hGArn%;bg*zG`E=}%A7MP|L$Qml2w_}petPPV&{BjXtYaEy4 zoPtCKRgm`{Y30-11yoMRK!v*%4AkLD<8aVKgY6he6(IvUFKED)`t?l0wuX+`UF5PI za$LG&BK-X*p=zlIRp^`JLVdFEc(9p487|X3hK#WfqN3&EvO9vRsNdFyk%^lV_$1o^ z?73$ao4P)V{Ad{{d$K-R{njf-rvmXvT zhd1ZMS5qMp zcG06ARKnRKfZYNH1_Bs=?f>@L8G*P^b1|DU)O?qM4OQ1e?yrsDvv+~~9j>9iuV*2g z;QZDdqHSsAxJl>5lgTyT?H^z?f{ z(SsANv9iejZt+^THKa<16d4crJ7C@N*VZ z(9GlREbz#o1upTxz@;5)In@7H>BRr8Q_#m&V%_Qp}*5+`nxypwNg6!*Soh1+4 z!?mxr1~Kt6Vb=8lT+|P%1JIbyi!8zy3mJB>h%ExNIxV0k6omkv5dv~XNYBB2F*gq! zFA2Dxm-wv1@N1dLH9G+JxwVWMV;|wu6ush}r5h0=c_v zz;AXGxa>tBmO2QjXNSpAzxWlzvLi!-ctpb4ZLdx1Zw<+hq9E1gvjWX!Zbl!Cnb zlY;C&ddcZGmU;=9Rsed|#!xYv5_sIE1oAqlu1oo1C!zn|;$?I*diXnG z%JB+Y@YyErKeRTF?V0OaLx8S$HIxeA`bN%hv#$=}Q_ml2#LWG0?{+n*a8F4ucaETA zmOD$XnL7&pSR01EcgyJ2Q@r9Es)o`BI~ss>^<0u4@YMkwp)jB&7JBE2`RaDaD|u7Z zt6ht5@W3W)@oC6KZaFb$d!AZ*yu#rD`5o_5vtUjcA!B6#{%&W11$awpwO5JDSskuR zUuvs(v)T`h+nj)9x@TbVTT)!+B(Lc92rc985s(kPYHGPtMk(^BXjMTYm*i1%5RN%| zxmV4G8@uY>+ac0FQl2TB&Oi6TdlF`*y7d@_KEoF4Z~5fl3>Z36$dKOZLI4 zsO>pWthp!of~h0<%F0O9J8gSO^g?^-2mc&0>#sGg$UcKfpYH%Y@yMCTV@vb!v4x5| zT&m6WDGQ%T7|(u?5MTYYIac-YtBogGm*CUQL-<6mhR66~>5%`&(%ApsRXXWU8@_5s zhe%%O2Sxa1XvG03w`ONvKsZ`yoj$g;OdTzWn?2LGG;3Sct;0ng;6dQ;Y@LNS*7{2B zgnJ+?b=Evs>A<`Rj@8$#cVXV?c?Fl}Sg9Ag;IDt>7eDxsoqzA!PUx!>j&WR$W15uV zljE~O26A~&N2xhl6o?NkjYE6O^_`&w4(`~@WqDveRUo%9{<9A5KWH1($Y6QLC zRtm^|EdzF7B7yc7aTOsW8d#UuOuvBv^)IlHd*+6YgJnU(A;8OrmbU4=rKYaErG~aG z;QCt!AYg_JI2bhL)b-JRR(n<1*I9q0xrck{c_-$QX}; zT778YH9C~IGRr6`HE0xw_igq4fdvjWVC3LD^Ry}t5IYBEaZAwD*uJfAtam6G{GPt6$z)vKF-K;%#d=Idcbd~c~5_80>8Vkqb8vO8ceQJTKmyH z0(!mFS#;AmN61^33eYwQvj*Tx%CTA0u&pBGZ>!K*E+sT!O9oE2jg-CeDKQ1ROCr8? zlwD{VBz)M`@wtDmw#xT(v@kP4_7<7gLyNFw&m?L-G>Z6pS}rD}=3({#1`QeN8-i*M z$~(_OZ7V1sU|#CBX9y|5SxT;dj-2nEMkOw`=Uz897T>V;P!if8$-j=`Qcum-7v7p? zW?vh_q@I~2r=PX7mEU(w6Vmi;u&35uY-&&|q`H;Ny4?jX?ZDDJ5?rira;s^T4nX7W zqjfP`)7Vs1L(cW!91*l@YU&K1gYig-nTyR;w|bx-zHcsl{f(THe}P&Y`6D^w@dwmC)0RyE}Pu~1)l!`=&t0(@?` zu6fblHWKW35m&Y~TmCGd#uWSs==7hqrm=rp`NG{fNNixL;`y!;o^)VjV)g;ebj_hN z7Te2jJ0-L-w~T|)x5D0tK?&!|-`_haF8%PExuN>zW-s`WYZ95*hmQSa0P*q6Y7aii zJH>;o4uW6Kva>JGH+sBP5vk)7>>ur=`SCR}8$wi_Q^Hw}dMu1R=| zUq#3R*a$6eD1X+ANjS4I0FD9Jrg|I5HC`#Hzy;jgY9`n^jE&op5KBV9x?h>5)YwPS zxj~(XvttmjcZ@9T?y|5kumI#X9S60oUYC|m5H}4o&bvD2+uFWya=Ndo{44uIL)(V=81Bj9x6&vZ9A|$y+5s~Wy)er2Q zrPtNGq*E$E{?$Hi)gQ4B&i+GO{{9d56ef9d9Pxgs6?D(mQ}bYBsOpJhyzHTWhL8sM z1GIgHSl&bWcrljm`5ZNM-IB zbi!&k{FR(ldUu*ya2=Sd`?$QgU&tA6e>F9NleecZsVlt&*KC7DH?)nZ7lzRo-x&v>ubNt$v0>7l06jG7P4h+bw9c^9mmZbWnd!{OKPfG+J8QDai zl2~}4rBwaLOhf$}xc%40um8Fv;2j!haN9uPgSEkuC+kz#G%@_c`3ZW#-Sq)tyuBOt ze5s}2s#chEZjv1RdkgsaZ^GRB-^{aeu1dKj_qoOIFW9=uB9^;q;_X9#Czj&Uy>qm3 zj})Elk;5}KXFzWNe~S)j#Em=i3`|H%s&Yt+->pmXBb~G0*q|QZMk|XPG*MtKB{Bu@ z0HUwf`kpUVBKTktkrPlMlJ@|8-__?BTrrj(g|cBvQf&ZifP_pIWs zT^ql_ALb-2po3agoq2+iwmHoL`Lra&rVO69zszenv~=nJT&?d5_pb69XpoSVf%S*$ zgw2!kjlGb!*vuC{?P$ad?<}JvT)*8pjCi}&3BJGFQhsfr1$MumQFa0J@!r42NLf#u zljua1DCe@0lX=?KQ+;n^xazTA3Vwg6W7X~Ggk;??BBeI%>JPTIn(Q6@@S&+Yx?FI^IJD!|Awj;XqjD`PbgN>SOJmD%CX$iQLl=igy&)+`kqX`2X6)9iL5rzH3OO z8^fTt?y0(rjcHs)3;N?_38V1tYCk^C)>ZRRFUr0!M^8QjEQ{a86>onVLndEaZmEeF zBxYYPcz)#{tKHQxyNa-fK_)mS5g%82ArV{S<&QnGvKP+j$`^h)y<%mkHd790 z(p7X<#Y?@oHiY)SSoTYI- z@kbUW`N&Kq9GS^=2TR22T{Ef1Z)V~>;eFxDOy-wWPQ7gu5o;SJr0%Fi;y*2Hwqq3Z zc8>Dt{DLU^*7_jy)vl6UysM#B$%SRNmO63od~>u?pB$TG?XP~Z*j{mab)@>SQwEFn zs3;ZNY6c--5U@68sg*t*pB-2bu)QiEMs!U0fsGA3W+0q%)WUGxBf!5@uS$T}A0{O& zch=AnL!D{{*rgY8LV)k{sA@Cq z<3$ex8VbrfOu~F!;WhldCFl&9>A0g6Dkh*oW^V#scyqEYS>0Zh?v{mp0khCE%cgh*Tn^;__V+rHYFH7Hz-4< zZA}xhO#Q@8OyANA+t3RF|@zEUCxFHCLrxT5ZQy z@5>oA_Rh-N8~un^mUid^4Wsb1MObvvGm8GSI!;RGfIggVgT-G7s+q8GT?wBOnP3|% zdo;mFJwHTFK4s~veCUuMvh@Bh8gEZZ~Gi$j2Z@9XfTmVxT$>ZYvo z8>1BwI~rzf7wY5h3yr0zf0`TmKLeiqKua#&RTJ_8bGR(W7~=hM7xb~J339&!lXGc| zkbTZS#i`xs2j-;%^~&0fjb85~CCv*eZ#RQtPSzJa|HeCxN?vFxeQ0bbxE7EilXm7w zRXa*T?xBuc7VdusjPH}Nqx8O^rSzd+7XA#Qq1!!&%dw4tUWCG*cbTwxhh}{Bf7cm| zaDVN9l8ZjjbPflV4eV_Ny*#|{!`4xF!`P60#UReQx}zeLD?2LQgOl!kCoX;S%btOS zvGt;3tZlHztDVp%z?yy$&d-5)1~{PRqU;iAl1z~Jhg%9w`MLtc;35mRv&bgzF0!eA zt%B?-PpFy6zj{Y1 zUvBlo6D)1jPZmTa7yWWl!OC#eTT4&zy_N3r`#Uj6wU9%%+RUN3fbq7izajg&kakY&Q>mKHnEHy!&OyiS( z)d})$YsE!(RJ{C)8gc0#D}97y|Gbdk(=wqB879Fwi%#4I#^9Av!h1+^S4Rk05?;ka zYT>&x#@6!mLu<>VM?ooD9frT#k|5%DHRRF*1F?Kxhb`Kcl8R03;HO)|*uq0&b1x7} z{G$aLbZ4GivN?|V0O-R*`vB>qyczSV9aVB}ng+TS-p_1oEWI&-i#^d@`}$u!h?pNI zNjYZ$EA+>5EBLjpr#7ZI^2WcgA8wu4SK{bm$1uqODYei)hsy{k zaJc~mrEW(>0V{c_r#A+{ACC<@2H+(?0XZ_=r@Tj&&h3BA44Vj z6>R+Ba?`-RwR!w_qeby$Q`q-;g^E1_G|Dwm6XB4+-|Je-94PtZoRkU<*tT{*%&4QT3GXJmYI8Ny&nsVzx0NjlJVOFD&c1d>cfvx{D+^GL|K;{ zgWzY2169w2u(v1LYhzEk$FXVNnYwtrIOmqBG5elZlJg{>EP8gZh%Gx>=JAfL;q!)U z(xH*h3V*M@ZxGP-G<49;9HqcFOD|e%0Y6x5sC_cW0X^-8CZ6o9`ShEp`2EjA$dprh zLBWmfiMp7;9HGcDiOAfVMy777Sdf5LMBg(|KprV9c3zZuUdBl}wcJ&9*E@+x-qUf2 zK7ddBY8GryCt~mFdBlJWlj4(s-Z&(=cP+gDkBlPXJ{!2CeI1YDnxU5LYI)RMt(fJU zqUEduo_nFC`suv5`hlqx@^Ez!9kn^hD%I9k-js9luFNuWA1rrOJ_Pd6J*}wt=}Ipw zQQw1%5rI<9wu3Y7dS_@bOHcJ9y}07;Vl(8``T!(qdm8@!(7;6OhtCYCIf#bZcTK!jj}(`&(NlGArM2Xm zoSE`FJOA+seA4xA#?fzn4l0ltS>5#Vj=!JkoHJuLc0$KFZ2} zIY7vM7*Np=-Z^}}M}qw57=gyC8wy`Yh)L%)g6xY6ZB93&X1S;ExsG8%*0ziTSsBKD zR5ya|*@ua#)0E5$EoE=MpJ!)ZS!k$uyfKVU2HsbNLrN}O>aC5duY7+J5_jvH9@K{$ zJ_RawLsI_SC4;5y=;;`zipLEs3EPg=_$}cxuzzpSS@z*dfFC%*FSZWNu}#!HzxV34Y0oZ^mi4=a~7gPmCbaE|>(+M^ZxO?>a{M@AhWEJNK47?j^+j&aZg!?F2FP z+}3E_`?bNUCmLbiZGA)jL&s2I#O^#Y_t4B2A6Z%^fEelV%u`D|3T9>;mF^-4}*T-BR1V?QQmOk}Nls?($&wc8ZR7D51nDmf_0p8aN>0T)* zFEEQvKGF~>_Ehi;zY>wREyHFU8R)e>1u@?Scv^rp>wNPK9LFRr&)N-ns^q2pK0{4C z39!@U?OEhUfHBi%StXbBjkV8Q5_ZXQYt{X6?5E$vr4iqEg5I38G*{iWcVXXejqoA1 z5oXyCDgCUai=6JB5Y)IvF`s-h=y>lOv0w{W6HX zYS{h)v3zfVRB7(1d8TftxVtfd&HKDsKkQYIi#A8=V%7)IpX>wJL>0gM{s=1htWjL_ zz%QqiH4?J_s7ZKwa=r;(;F8hGRHEe5a!%Y;r7$7F+MD(EK+ho@EQm#}c^VX7{ObHD zJ?-_MHlASL2;@&yc%OUCb4zE@H4Q)ItcIUv9;yi+S8IMopUU3cI2HOc-66w4s;a4Uu*p4qa_N$F;Vttbujos#$V zXLRc09~L?*J_G1ToM5bFs18q%_a@Ggx-l+vSw^Y28Km4=M54I6}yi$;VM$XJRJw?hpT@Ot< zfk}$^c7|MZ&(>ZWQD6S@2U^C>Z~LIne;6mf{}GjP^&g|y_rI+VL!#7O70h2dttG;>^h)lAB-KycRk?5lgo`YFHMaVFUJtkCyl&}n;YGrNMK$+s2U&- zy0MuT9FrVmNF$`1+6wQk4_Ci*Niiw*QTW@Sj#Lpca`28>T8VWC8R?!yCjNU%H1u^< z&=S(I5t|c~bbWK-^)A+B7W;b zBwSkP06jGKz*F3E0b@(fhp$g!Gxip^qN8PD!>*A}cTN)vRN~?X^UdWCOx@rYK@C^% zd8KLiV39`;XlPX)8NJduL$BPFkTP7;jFJs0r+QWld8KW}6@~Pz{q`|L%mg;|m%f@0 z-%S!S&Y4?F9|jfhL?E6k)+QNw;7@OUY(peHJJeCIU#x6~SAkC6)!{0S7OB_+6BF%I zbFj`iM)jPi^6_#%Jo(>_#+k!q8p@}^rf*8%Z=Dm+*RFBM3qX(J7yEHh=#1#^v6;_* zTN#0tY%4K&v+UTDDsJLAV{_SUQ)kU9OK;_?9WAElV2R7zRtbr6arPyr6jy$16*l{{ z*b>hiB-Sm*q`DN8;((S^{pW^wz-tij2bo3B3Zwt{cGn_kIbN$DI<&CEJ->^#B;c1T z!=?941K?NINpAHxt@?_6n4G?=rPb^xYNCA;6%lKbLZqf1bbX#%bVk80KCR-FUtDa3 zJ+KX8->JkE*9Kt8KlgyME>4q6Z!NZf?pQj(51k{>H{Ka+rg;GMPQuK)ZX2vi@X0B8 zu3232)+|12L&_|3D+T!7MUl{}=AqZ6fcKmt!!YX8 z89BH3DmMA?59?hOQAcW0rLDc}k&c&fMJM?5doSeCw>{O*|22b4_<2^4cO&KLnST_& zy7aH06bTKVHTrXjhWxCjAwp7QzGJxTrM0)@hO(jJ1|cu*R2MkqcR3~d+!QwL%p58A z!jc$rdzPGk8b~K+>dT{lq^H09&d^ZyP{z%Fz|4L69XkHrkIm(8f0fZv&a8CRJaA1S z6I@f(sZL4R`$Ihix~CU1cGQ$IKzB=m8g!OxruO~n5IoWoKC?FoaM%L2#yL~>3Gh7# zpR~Oppdyty#=$Wg!=UJ`sk#{dJTX5w&p}!Tu<7HJtjqR(T%4v69W%)Wy)ZOaJXjwr zj_^)b06YRs^Qf?y;W|#At%8O#>x7(8-EMXfp1i$wtykKmeRS2?EIXq#gVb+!R|%5o3# z^%O1Z$|yeZw0#g6J7A*1zitV-K5vOfj)B~9 zU}19|3To|^f(_Z6qGr3K)S}~!#%aHi%h+8K2)7HYhx@Q@s>qGd&<+iHFYH`t(aZ1kFUVQm!W?|Bg zy~vy!GFIXBPH@zBBk1VwnyX)Z(+&Ulqf$_QZ>bBLVD3Y{4S&zWE&FiF+EEj=+*$bu z;G;=#s1#eGvt_EiZv=45@@-S37`&rW!!YU5WY zv6T@8(zknB3Sw77$9QL`g*$R$?$$u*eHkO>)N)5zgnNcoGDs+XX&tW3wGDz| zms_eLH+qp#L0}M@qu`HnUfQKL(CdG#57ILXt@uo18#Kv129KR%Cti`#qED^%W!*bi zfTaXg@DzJ*A;6%OF+mLvZ|Wr!c;@gG00Y4emWZIw%T&bCA_??mnN4xcqB8-8xwbKm zOxsa1;ND4enuHX4T1HL3x!wzlbd10wH-`ub?x_aM)+n#cJte4hOB=`=qWOcz3F3zwD-W@LS8@I;cAAM?o>lS&m-~tWon5~W3 z*3#;BO+vxyFggy15fH%8NY?}*dvl=rsi86Hx^to?$}PpE+Gi-a0F&goWt4)|G4%Vr zMF!@J9a#6Xgo2epc#LZr9`BU`j3C9OhxAnN*9~FkzL`k^)?BG?o?5wQq@vtPB5ZGw z$2~Ihxd$c=ao5DeJLKfzT@#nPzr^Q%u?QRfT;?#27AT;gic;vFBBpwPc(V3jB6Z@b z>pSzrB4sD}ldc8%YHgU7ZSF$7UK<2I*&G8swGV-xo4Ts*Ew)1**oOoqOTD5ZYd0<4 zIms$n3-81&GX$w0oMn_gj>ai`qW%xP{*b0Ej9HW zF1L(^KU>(=lRyqBD$%Q=S8j}xGF=jUqFhjPr3;yGq79Pty^39PbG46^yU>dHFpAGO zr*EizHOH@gt80S4a!5#NE-AfqZ36KgV1Vf5zPd=jcjxPyzzUQH0IyMoJE`vgIB)>`|IAE>;ygi z*9<=P=hZQC7Ng?R<%+nwzfF=e9{Oi+S+sg!s!V!?hp!Dd|s!FrUuOFp_W1 z(Q_Yf_M;MZ6oew@I68h65GWNh@7~rxUFNooPce3)GdD&lMZon-+Ztz9Y)e^S?+h{9 z+6j%Dz{OvtB;5Sx2qylKWdNGB(pU4^uMY3u<1s@f77p-Wn4p2j@+cWdg|Oh(mIRZs z)Qc`xH&(@Mje?`py!eZo17(i_a^UrKq>{~X_y>T8K5j}tZ}v?Ts9TCHTJDCwS!{zm z+Zsp2`{szHAvK3|Z03vp-W2wD=Fmw#C9&vWkxe^ZV)MRQIO0F;{Q55|eBseDk9D}h z<$hijcKx+3?DnfkWnZm)(Z6kirUNyla;+cw+TI0vxY`YiSRbr=y*XO@_Gpm;F%2`a zHpdy|JM;By_Y|+%F-}UcbXVRnw-;WvbQE6M8p6cdN9bw#cETH55B|MRA;ek-ps)4f zth1)3oC{iU<^|U@Dk-35)CRQzPS7N3SRBKCG7e&6eOeJPFRcT+OU;7^W`X#?$i#Wo zsI+Z>Ikpt&tT|!v4fiCoaCMZNvD}S{((%i$t#{YG+#Dn(+576EbmE+g8>5V3kGzEk z@LBP;tS-jfRd9W|BlpV2NOgo?MJe^pv!S*D#2Xg?&0kCc4#4r{PAM_NK8T81?t(qr z7{kVe&sK$WJmRjgUbHz&FYzb@lz>{y@<^GWO)00wFeWHa^wHx~?U+|9Bg{PGAR~QN zgt#vu*WR2Wqn@h8h&R>_eEdoa_N}P__CUd@y)%w2JF6F0-5jA8-{54wJ~56;J{7)h zmSKFt`V6DQArm53Wi50?4fYv>dT7x_my}rLn}fux4VK*2H|CrhLd5>i4T(E3icb7>xdRljG{z{b&5rr?<&{(a zXs-NlaeES(xYU?)(l&-$J{twx zFH55OW0Qav($kTC4W?jQUK9D>O$O`O#uAdt-#&(AJiFtOQ^?2GcKMO5P4ORhgY3Uf zUiXm+S++5je|K9=fOMf!9*&W79&e5#k~c=+(auS9Qb0v0*wbRO{xp#)Tyyv|kAhq4 zka0lfe$of8Tu|pz03LCc0{DCAi=~#LTT)i;Ao3S-P&9LM#+|(RTeUa z>-`Y2w*b~~%`vM1KTxqHVHRvn)AQ{k#3WrGE^0~yz9kb>-gisz8OE0Kd(+gEi%NFh z?S&TPo0WcQhIN>eW9SoRY5Rnk)=^seo{mX4ws1L{QbNY61e3fqP0Dx4ST$}XAMetL zNWkmoEp5^C0^VN~iV4X^CFW0p}l(}I2108Kq7 z7Z<I= zRleAoA{5%B4VZRJ_8%>fkJo@4RkRPRyR|XcE3cf8=aLf(?31YE9e`tkW|2TYf=GvF zJ^#5h=D{Vul0y8?Cco`(8-@7yBBm6`xgX7Kxi=QuiXZ6v3E6pX9)4dH_xOU(KyCc8 zDeONm(@3t_nvdqz)Jt-F%+I?<7S6jMVy#TEN)6q(NT-|)3Tv}#mX;4(`&VH};?p;$Dfyf72C7Tl+T_%B_F84# zBHd6Y%sesLVxR62Y)sN}7eu*0F0Fj*oTz^8ok1n6PI1ZySY@|c>ymz7Y=yp9AHaOD zcGWzx^%mR@&BHSMS`L0y%B`_X(DOW6A#;0yPx7d#Wo`|zU`v5bTN#18a!Zj3AuYDh zt;XfrX0Y)qGlW!$xb)dDDfPmxk_p6~gnE(73g0sktF~3R?9c+G8sKO0>M%0OIf+f&R$w!1 zL*SR@A;epUj9Rn<_~W34iwW1@c86=OYQZ08gz-mk>x`ZYqTt_k*Rp#lENGbh4r z&alhugOsEV2`zhNf}FB2NK0HEV->o!4IHb4p08=czFD7XWvtJ3avf4Wz(ULl+bAJn zZIYN4o|9c8pRoh@Z;t}-r8;VD_^gjtMXd~(cnqh8k2Ov3OYF0(60e2@+t#oV8eZv@ zRUj`pX9-1aDIwRVrdBxRT~ocA`mA57rLhV&*$R4{W12#Fy|MI6GIcXF?PN=IWAsGHkwQj$CaYYl1BecAzIX zsQ0R7MBdjGLAMXcajU}>PZZqr)06m*Cnxbq7yQ8W1hf;nA!Ao<%!+Vp(t3;&$R(~h z0eo|cQ?WKiO*8l4BF(+%w;NNmg2hpG&O$#k&pye91{Q$l0-k(*lAPm~vOz$O0s9P% z_3H`&YE9WdwMg6P+My1Jacq!f8fk;AP6|NACREJU5Fx`iN2~TKIdu!8jI0(y(M3UZ z@~@^Ya*kt~nrrT@yg%QVbbh73;GSDfE89@ileg4Oe2<|)=+<#DP6Y?*n&)9%Y97iV zXI6L=_)O zc~GyWUL07gZ}geCL?E{?!gV}@8U`e!Vr;lun!~147&96mf`Zcu)0Sz{P*GQ=jXsH#; zy|5>aXrVRKv>F>OmmE!+mU!KI*9ZpkR6TQU~RXKL>AF1C$%j1A2j zDmElwYVHj!i`)Gs3c{tKRvQN>A524(cx$-ls*VKuYHu3g>L@xcs22-7s(SX$ zLQ_}3(5tq~I~pBQ5!@pe<5wj7in&JAV_iG?rDcft$syy`Zpm7CZbf&eUoWKl^oUfC zie4V>m!|53MJreZH$NNe>wUV`KDV}W)T!wjG0(J-jgls~p^uwt9T!*YTG>hVURHr$ zB_^*-G7Baf>h3GrF>jVf86_LCdc3Y1AGtI@`eYvm=4XypbGRfFZ|ej!kCsnZnIvUe zCkXKlz{GpCO(M6ZiRYYeV7ljdXrG!*3@vkbZao!Z7$zmoG{PgK0(hiVQV%!wu~Rpv z*`*;P3%#x5)3)aMD36|l{<>a2yk`|PZ|gb8fT5`^VCb4T4xcAB;7Z&wSp0?r9BChg zJYNC4)z+Mds2^hFjnPZat_}e_0dRtCpe}M#f{a<}hd$S|R^A0X#>;@3UK0THd3}mm zZX2UlIA;HUj^4VjjkJ9qrYl`*x9(0$ol>0Q?rsSXAXtbI_du`!K>{QYAt6STC_xgO z;O^FTcW?RrKF|AjhWpPRcMmz?{GT^V!JHAJr1vbVOZk2cq?jI!`kL)5HgV?Ama&|n+Z=K5VnyI@HN_$p;B zcn$b`bEJv6U}(ZkYTB?)RU2Y%hz)m2gzPDeR642Aa(&r$8ow+!r0)i%Dgsg zfcDl?i?^(G_=BmYj;~8?+KYJs8&*1e)mVq#b#xnNRkhVKG7<&5X-U&E zcA2riIg4G9e`TZ-n{qJHT(@hj!>wx>Wf#xvZvgscOOMIgw03rw#dRPiEbd;R?}MNB zG&J1NB;UC=-aWKpZl+Et89C#kx-?riD}GGEE^4UAzTQ@v_-a|sWbBW%_wL#ose1tL zU)&R3RI^H_B`uI~QEP#@ml|j7DE;6XLM1M1Dd1&wEp%TaHpJ> zwybT%&#AlX=M5dKIZZ>AosWu~l`_iaRo#sal|nS!FT|QVT2q#!UHGk`?%GvlAKRtv zs-02Rq%LWTf(|E06$c~Cnj-+;cgA_94Rc+!OGZgwl+*LKO^ot=8x^`^A(yTgm=#VL zEqz|Ys$AA_=u;wc!Z@eo#kP(P-7v8*o7Q^#rll6XGR!QWl95v<0BycwZfUugkALK- zXY7vCP|GGl-kPNry=mdJxrSOvhR*spbq^FVK5J|fuKTJ&a3 zi_hP&vB)k3sld=w{KCPj@;-95@^>d&d$-0KXsf11%!5f;nfiqX31QKm>z1v>`b(D zAIxxvccwekYhxVY%1CqboR*loXskuAj&=0x0DZmlKW)GwuUiS=9Xlp>-$BRiPqq(k zJG-^h3S!!Vnp(c==os8}vb**iEV@fehfK)uar0&hba#eMK3=TrTpweSHtfx;y_r_w z{sg;iZ<18C?_}WDtex#+13Z+aUyj%HXwU<#!Xyc`@sp#wA2BZxGVJ_@L|rTWv$d-^ zW=7CmHY1g_*al@S!@Sn)p=O-FnT?D)p6FEWIogG5wkFD$oRMJ^H>3>@2r(+&AWX>a zj^rY7FZ8vv%vD8K>z<{(ZQELpT2MEYIz^pD`u4t5RTD2k-qe@Uf@ON(^0I%F(#pNJ z%{{^`Gf%Q^?o&?4+Yv@#KSJHpUpOq_myatXt*UN8nXHYUZs4gg_Cd>_O{x=U__{iA zqr6ba5+{+$$*;&D)SE6a(|ZutAFF8@EV6X4iH%*-k_$F1&Gl$7;4r=$zx}#;%J!TDEq!*#~O#R4v$#LK6O&na?eq z)pfBpZJdr(LtW*VkeP1lCdJPWF!ELat!rpOe01>1iOzm(>YS96vu$J7ub7%i3%W+w zjFN-2OIk{1RCPs5#wPTVfdK`)e#+bsGv7AYQLYgWVkI15mcF+u-y-a-)bV(wI-$H) z%Tv`2_2_E5$bAv*$i@%0{vPClu9LoEX~ZttnyHI`hFvl<$~MQCHJgB!Shdw6m#lzR zGdJT$m8yP|OfMW!4)q&^eNY}p7|m`H2Qw-B_xP&rd-zKBlcM;-8-&6de@S~=$*htM zTQN4{mPhJp=1kP$z3FD|#q6uY>2~F|v#xf{-auL!@9c7#+i4b64^A!Q7U~4OkYQmD zbV}P(JE`My%t9XA(l5pt`I2U%kc%IaH)EV?GI(5yjbAgdDchsH$}LC#=>BY%es_{1 zI+$W}Pv@I@j$I7aKkN1UzgLNc`d+eRN5 zWMt2pT3CBib@fXo9B4*G%w09LckYb$>NZE)+Gos7HIpW8y;ZFi4e>`CMC}tOZi6+0 zLstg1WBKm|BuTV{Ih4`MRD=`J-R_lO>N8qNqYsxNN#-(oQ+1u3)KNimmP^%EvoY3h zU9j{^C-i+SYAL^3EE+-xMUxHP+zD6{eb8G-mq%$^v`{@~5NdAk$+7TdaCzGhsK3^b zP*2u<0cYVJ6(*t{z`>2TkRaAoF})*h!_p_+8SN6xX*qPOs6NvmRwwXh7yRD#sID2s9rl0~^25rh8a-TSOdUQc3vNbxJaO0Vb( zkTjT!OxywZs+O3&KU$NwGlnm8sB0mzj=of0tum5JlLqwE^#{uN@)CJhPpVDCh0ke3 z-4p6TuBN}gOx)1%MMUC!Y^kn!QyCNXySN4sH7k@htm?blwnn=|N7Ed@W6)`f^3uQ! z9Wwo3qD?qwRZN+rGM1v7n>D~@he;ZHQ> z(#g%}gi{}@>RM7}2YVPRYHr=MxGu{jqvuZW5uq~!O_i2GS+hyTN1F!w&?;VAlCrBI zc!=MZE$AA`<8sHLOq}Q)p`!M66S+ICovq3s5rlz7S;XgAN$BT^G1a$oQ)=!PXTaXE zvCV-)UA?I`wWw)&xT|Ry&}~a*a`CJZo9q~BL0D8mno6L?cQ)(6jdVp?GhLS6Ow|PA z3C4F7NW~*iA?HeZO6^an>5QLVz6x{mcph>&FN<)UR?U3X%WMueh`1G#+HS`DFo&^b zYGk_9#DWF&}LwA3pi>xjZHEJ9~20Y(cc$*O)^olpHx zkcNMokqCPPO3lAds4RLcVAllF0VAjfQ88Z%W6=Eq~#1-Z9>N`VZy#r90K#mi(aN{H`oG7iJHCfwVn>eXy zrS3Yr4TqEMqD4y^+n`i+i3Y5!F5v>JxqT6WrP#d7s>JtEFwp~e8SiFBBJP*yaMM;#36`dW)4mlS)r4x%yLk6WpX76qn+EL7Q zS0=sXFpJW#6<0$XdyA=7J|)7H?j)q@Ikrl0A6mkH3d-xe3(nS6mpsA_0ceb#GH6MVY&UzMC>+Yv! zxBZ@-$GZk9kX+5o=3VykM_hXBp62EiguV3X3)$_-JBZuwS3bJ=`9goq&uzMihxI*E zbacnh8k%tnmPVpZ+zJ+T4(4?>o6}jf!@(q~$&=ZD1nYjUs8T3wOKmB+u>gWp_7s_xMYYGRggj_}V{Sy(_w!kVtqe?C39A8Ex-8oNDdc9BH6W zD%s^0g{nuRFz{uA_TE0Blf@NoG%}i}lh6e9s~U{VgGjM_M&peiAdsGi!Tv>mBqfoq z#l=8xB}Heri&|L)EBY4vg1HVgBo~o8`?ST(_L+2i<9uit+2N5|BEA}#O!*-?iTa~o zc(vP8&m6bkZuz*q_Q-GxjHJ1F_`uxmyQjLv$D%JYN#c*T<(14NP7 zKx*z8DnAdXa7xaAYxVE=xNYVl32?evb2qL>ZeqZJG!VBpZ5Z8RkH>Tx>cR`HH zevx%o5J0c;+1&}V#;&%x;pSSarnjz{(ea7Nl>0FntRZm1=&R&>!LPoN%%3AuJAaQ) z>-hQg`$D&m-W5MY#4&#gOJQHS_9WTux2t|`k6-4yy$-;*-FgDLoYwgT7z<+qkoo6>sZT1*3bCDiME%e zm8#pBpsp*Q{40Nc_!4yK*$1fGORqAwm!42J&k(}To>8?w2d46WPASpehEpa!HgcR< z{XJHgVNl*MA!{z0*D@hC5iv@{W_UI+#GhHMGeV^tziZ&R8%u zFcvj6@h&wj$0F`1?qv@ov~fm34eir882YGJIdEZ_`)GXhroeu=$gu>w9LmL znG;@x2-*Fpw8r0q$T5rFFZ?udSm>Mn%np%0p9aP*f~^TmJ3M> zdN$G_r6!F^D9L7NUHag_KvhTIIE>k5iN!Gn-@q}7hlTLrr-4aL02fvK@aki#TWVJG zFUcU`@1d#fmpwy?zlO##ZxMH9~T)HOol)9l@Eo8+B z`5j4JLPKR;*F-jsuJguH^`3B&=}lURWpb>$#CnUp{s3TuqM(Xf_)xQu<|4l6^6|lDX8n8DaP=&a-sCOkA+qG+u(B5wZt6G zuXmnjyFGZ3=l1#&%V8T94Wo^COueM_@o7B0U31`iP7NAw{vP-%j7FYG(ufQmt#TE42e;ok-)i(@r zDJG-oa#U`|?f9a$S5;)0H%0LQL_?}uf@Civ2dE}J|Jdz9AUL@q!x~JsTJx4J0ft9s}5LB%ups_^~su<)K z85KPEh@u<0VC<-$QBpJJ)x->^oRDajF|)-T>^OdpD7UA7sF2kn4ItEt-y)b=@3fM> zC&5XaTdxACzdd|ga_Mt8?s8;K=Z#2E-_0EC$Wst?^i5Wc^i6fGHnUec%CPDr)M*7N zWky=+zpBR-ZrYn0o!U;iLNUbQN^G=dzP*IfG8&GkT?j0}jXp0x4c|>I6#n|sv;5K@ zH{;wMzk&eR4E-S>9`?8r+wR-m%8#&$IK?|gGQy>6#haCKu0}D`E|Ds5eSB3>2iKU_ z)NV*(Hfe)#B-vX8R{E|4E_xQ5UH@lD4DL=+X4A9cO5TThraaNmJxm>!t3^v@em~&9 zF{6D|J_ZiReN0&4YgaZFs^ns9H@829My!27fY;t*wM=DJQ!T!r^6sYzN#$3N@U{mv zgn<`uwD>8OAP;V8kYEU5@dRz5C3FKo_H zDW%LonFKB1i*ooq@;q*rHn+Y(5rCugy+vT!U!k$RZ*WAuUo*Qqvc0D}q_ej_W}sVE zY!)j>E=?bG+tNZl8fW5HP3VkyJp=C0_w`uR@_wCqkS$XPkX>DEVfA%AA2Af|`%;YI zO4%7%)?cDPZP!xE#WyR77O&pkalA>TYa5j` zRc>49@Rea4XvIK;?2NOhlco;RKp#7S%jL$mi6`;3To(veXA3~m%^xdB#u5Mj6SD-Cv=gYyZ?V>0)SC&BUw5C)N2mH$sP3{;f-;>p?2loz0 zvj+$J3JfY9%&G6h9ysaF!qL^bVLweM4Hxpn?aHch>k!s~9;8hGz8Qa5Hw<(15gb zA$*o~)Y6TSCg$2GyMD&RVOW%H<#HY+iq}y3xSNf6-PteAq_?S}s)?dkI~dj}l7n>SVa>vLPw$qUTD3L|i`}%~$`2-6#RoH7^YUoV zph+w2lF9@%{i1>5j$TC;r+*yA>aeAfTdXlK09P{#+aE?{F+9?W8$KgwO~GAKKE|ly z62~O?1V=}Xua^4d#j=i&Gd4iVamt(E6ROS{yR5auEbc6{NV(-kaaWO~ttCjr5hV-S zRk=KlszB1Ctkm=lmRb9F;OPNw<*JMW-!rv0AC9%Q?^qa>a}r|8qLf^8V5w(qTWe9% zLySDDq%l{+rN^l{$dP)bk|Q1%tnBOVO_X-BqYVS>T(i6jZjp5(X7qhc%hopPin)cf zVQnTKj5pPvI2#-GEj2k){Jd9AF8Jl@5HWQ|N{XLV))zZ9ZJ3ckM&`7>g)ph>pxG3S zpuzSkcQL!_sjMCKS|@1D7?$-`jVhU$<1#|DOVd%mZ4&AZtWAt#TTSNfaCz{`P<65Ptwzu&2JruZ>f{|5eQ-rDnsi_AP>AH zk~uFJtnQG;Hc4`OS6{4v-<>+C6}N5MM7j+lyMB3yTySDDNMyB1?q&%=n*v=>T)(S)}1ui{%m9WsjHcPK3~f`8OMUoocOX6 zC$W0hhA)^?64GaEogH>PpFLyfV(m?|aJEOhRG()QF)}9rJ{;BXn8V6Wm`c(G zk*gF90-2UEARZzrq+*m_!3K@!NXhF{O)UqG`X-mOG;pjX<-r&?FMM8F@>$EwemvHO ziC&ViF)K=1!TbQ?%V51$wh~DYME;YpS*6UtvsJ#V!t_PsPhRr za(ft;J3D|1o9agePD^l+c3w@;oPw0Spv0vuC@?YGMpEI$4C(bDM8uN3I&uM6`#Djy z&#Vv?G$W;D%n#Hg92ha#M;GtVSSTgOc4G0a5qptqntwRLK(7xo%a;^P@U$48G$q32 z&JUs^C-~*w6MRIFQ%s1ORC7>c(vGT8QA7HSjGD4B%oDB}+FLeFq>3eF(U(O*n){v> znQ=7I$T*v9<$ZN>L}wRsrmV1}rNNS*J!38A(B4>oIL2z&7_NaV8VLC_hIX=3-(72w zHAD0rXukzj9duL2s@~8NAj{h7^gS~cd|<_a4y_Cra6I9evjOOJTFIgu9XZ;W@p?@O zjX!r_iVo~F#P%??a;hKd?dUD?Ii8|7960DTt7?F62GQ{gqKdEsGX{K--*GrXEMC*W zLRK}1h!Y15^KFhs-7&%9?Hy^)7cOQP$jiR$8_`++%u=v>(+tw2nv&&^(z4Fyn*|$c zc=UX4(c38w_@$*G>-k)7@rMHgHtWRBKrX3?5Pehj;{_!=d&@*EotHp8r~C6f96g!O z=lIz#|Lr7H{&%*v>7VI3&e15o;$(_R|GLm=KAmjrSdo`}neIq?=IF?L?iwicI+-NE z&KH{myAyQO=?oS7pY^7JZ!;{;mbN-?QIZ=lCrW+pl7c^_j+#^uv)4vD1WUHg&IMCFVrjUpaz)34 z9@=R&XA>0Mshxy8Heo<#c3i>N70&Qtf5EFA1uhZ5{)~21%rhl1{q>%ToVh8hiQQG> zQkMp>(Tfsf=!~esPfJgKc%ZIF?CNXLYYJS-EVn9fZwQliV#ekkTWT`a28;bo?8FBx z1)qL4F)}~@Yl2n#Z43k2kmbBt5~MuXQWkvN)g$Bf&GfQEYdv9ILoQxX!vFW2(7uTP zI<%lu*Odi6i{ku`Yrq~Y$`Bz-3QXMTD5dh)N-W+tQVO@Uu((BWna^}@`TJ!F{L79W zpFP!I<~P~_ezv4Q$Ib{V;x<+2mxcHP&qzYY{FvDrQ}o@3(7$#_dlqa66-QIF}L;zAA&o9+>b&tFrPh z&h|X_2~LLlo(3ASudDcSY{F&%KPy{TqT;u7*yLR+6}qHCB_E9t%g&u-*p42XwWUHt z9U7`*{^i71{A-?32dr`S_HZNdaIB?e&&q)B8?oSZb!FJ577??qsgBt*p;NZ?h`4n% zEPPQ4{e14A)*Ovs!ADkD`u;F9@%SPOT3a2pXTYR>pKNIUI>TbSdhTE{zfRyPw}(n1wuT_S(-K(thM9>voo?#<$JHV^nW=Bu zb=1*z?ev;Gb4~u~II(=+N-Wv4P!Q_|5@c_>k$o`J+_|E^n1v1fyd;8rToV_6+PIjb zW2lb%*D8~}t11iFQdWmKcu>!lil8eRO3tSv4YOo#h?IVyBWBI@m48_1FMhKmE_uHy zf%&Zr;u9x2%RQ&Mt3Ds9S@7?c2HZCQi+6O;kdfw?TXt5|)rsbmhg*ZlZm3f_5$)zVYO2w+Y#MjlIbbnct@nTC= z?7ye4Njfy+a`p}IxD7>x|DKKpXgGS+x&jgA5|(=cdhz*4YwGif&YY(+{2~tr5AwDL z7J6e%3=i9pA;KL^DR(uvuq%VekUx}|u&d)uX%B6UNjDbcWnm^RB%~h}bLUu3E7?;M zvNmN^VJo7X_fAfNyOkAvXS6B)`eb|V^IbJDcUfAUx&)l_=1^tWo(>teE-Clh9H{VF zl~#tz7)cL!uz)Kr0W{(7v8JXoGZ8Y=ll6A2HS+#+Pw4$aBgFU2hDtdxWAYF5n4F6l z4m+kg?yu?{^WL4++zD*e`ms-q5UdhvCNj9OKd z2CXT<-Wx+jfhSf>&Y1<9|MwWaU4xjJ+e_@$981KzO_7RLk$UVi6Gv)24v!q9h-Y-!-1A0C7;hGF=YVn z5x-B@()X?8s^!7*;HmCxj~z{E=z*P4dpcRi_%=;OoR4FQ&u1Drhf`hpZ?o;H|L0;h zej9~nttue?ivxw9Hr22&fX7Hj6Ey6y3cw6CJoeCpDLOXcz(;0W?nMT$O9BqtR+spU zb!FTiZO?zQAgJ`&R**B-2T{=ryt0pbN=nX}3=_2>1J+ei@?mWd7C6~c`qt72y0_2= z4?Hz8;j1E8@PQtkbZDwc-%=owznkId->oGfXO^m%Wd$O1wy*TVnVDR2JdDd(lK_1x zEby>3MBf-hfBxA>55ID9@qE0M2r>&wd_|@4)!loFC=x~u$p zOTSzemBm~!(J~)7+cICQ3PJDob@2ENEh0+Ol>JHokG^q!k@GG^M2@$m-y3g@zGh)W zUmInmJXjM#KA#RF;${FHp}_W4AQxig4i`zMy;s;z7?l^Sm?`sI@t+&vZ1K0a0w8Rb3HjvmxVdcHwFscj?~9r z(-6aM09yH-OOXHB#!kAqEG`c?93~emi2**4mia6SOFp{#^WSVr^FDna#R8s)P&h3r z^^;SRudl1I87HGu*uI4Ta*0d59c}SXHl*N>JA;KE4h`iYUmf^T3n%sUcu!u)-wrDB z)P_t3YAxr;Qk}3Y&UoeOO@6#7%l&vh3eUE6g8f;A{!fm~n6iJ*G8q4!rqt|ftD`pt z0S;_Sc9&Fryu6{V2w@cX-%9bi|Er-f)91f4RNPgqbhTXRwIK%)@c9TTb5W4{ahjX`{KQU0U(8HD8M)9bxQv5gbmClZ#?u2cI(65GN!`(+ zVzyPKzUza9A4giEZ)tGhzpsig$&)=*-V?2b&qkU-k1P$K`_9f150?NLup~loR!owxT-NCCqy@!HK&$-V%RvQ3QUyuZ2f`H8HF9)uil2LB*#X6)s^% zjfmJ%lzMF|^4}e3A)(7+@WGenD=69hzj~TT2I{`!Y1vgF-iMFH1O^qKK@)! z%H5Du1nOwvS7kW=%l4*}JHWpCuPQ1-tWAkGw&Yd80A43;NQ>T&)W_YM<)(jFlNR{S z@$%iLx(a;Gt?iwgigwln2kCEbstlOvE)6<2BT|ls%fnaY;7?9&o(B&ealN}T?B?GN z7W3>P$4UqNqHFNGH;DdteWgGC{=VkoeN}WEKK<2ChQ>6n(@_FsQ(YM%$46bo#CiON zjq&(JS{3^1ssI9T3MO-Ipfm&$_4K!?UP$o20quTjt4>{$X1$s2jJ{`Ne7Pz@dHp)l z9CvqJg-ub>vmaPnia(xP8|kMeV&R?&;_vE>yC=ta{idhKUsu!OZ!Jrqp?mu3gwfWF zm!~7RqSH}8{|%P zYvM}(RY^s_-f&gi$`CAukBGm{%JTgQ=;h>HeRa-Uf9~tKKJe>BV4rtXWg)w|3IM<1 z8Jn8&5KDdHoyqPrum4*lk-kmgN&$ZUbTOyU(U$Yv){y%6>m(U<3h4P1MKS z-B*)*y&n8zPsSn5agi}Qii(JBWs#4Alm5)f$$YgeC|P>W?Xl$3*5(AbKi`$Wj?dFXTRRj;)`}QH5v0f@T=LL{P*)c#Q}5N!ndbv!}pY zM~!-5q{rPf(^GF*=n1!uOo%Kh$me=*P4t^BBcX78h>&JsB;Qft0F6%$`P0@Ib=61@ z_-#{>ANKEg5^__Ej5#!uisyO@Kg@A+J}hway;l1RV*tz#bhM?voasru|9>lF;-L)% zn(EDW-`12zo{zV1&c~XY{yoR6JsE|k0v;e>RR#51keB)$j#9DTW?Bc1ta$K-sw#9> z3k^RoR3_}|E29BVQ+ft)3=iS=$7msJIbM!3PdB=x1)3-%k=$=r2l4_%9=@jC;;rY9gyF`dTwI z==zQtUvdIyjBN!B@E+xUj`obF(_Lu~R>gTQceRyaqutPVby@yD+nWnQ_5sg1(UIk` z)CcGvUe?o*y4c&@kk7!egKkUyw}#NkYf1umj8pt!tSRrQvkmNF ztxtbE-IeqD6wpKadSv=IJNqfH)-R5X1n{~95i!NddFp6My5($%yKboq`eULc;r0^X zZB~?s=!U{CzqUdGZ!Qm1M4i~M*#OQ*kGEyLHL>DeOmzc3g$MQ9)6*)}l-QI(V$uyk zP5c9U8zf-9x6*$_T<$&D1MxOCfjl|o!B@HPkVnTO^FZ_7dce~%*z|2<1W zEXfgJyTc?1fQfhb5bw)sc<(m^zw52QZ z-_n8u7RAYL=KHeUkF2JW%F6 z(_84VAguVfE=5J{7?{vSB|d&xQ5`znQ}SVg19o4L!h&`IU%x&C51r!XJzMQB@&4*y zcV2kwB>~LO(wKPH(inAjMF{oYRbi6P&GhPJH9AG#lzx9sSRJ#XB&3*XGj9yxqOJ_# zqpuI31Fpz1{y&?ju{T!uRbhKtYMzdsdgo$J(YBThTIfdwZvfr{@STBUU69w~9YxPq zCFsa?fan~2RFJg|6EWOS?&aW=d|2Q^JvRms;qxML!bnHihpFDmFQ=A9=7t6lxgmpv z>?)~{ok2QuxxdnTfe&(@H|>H&N|r_ygrO)Z}iA`+T`ec?lWe_QAVxlecHzaD7@ zznkx`3Yi~7#s~;009M4^+0&wuN1NiVJ6fZz&2dv7IGe(5uL@G%?kGy*&PHfx8^6M@ z6&i8Pz|8ac$I;ZaKMYTt?aF*K)s}u|p%?sOj#u<? z=s&tDqVJ7$Kt08z#3$4J5WiJvdCZcyIOfEJ%Kdhc;cljutt%@-?VPkbE=gs;g0L)j zkq-qp85OCiOSz{YN8gz0$$qnUac@O~j<oOjdh;V-5;l3s1AkulqPeAaMn z(Gw{${n`>A7WelAqxNis0^3$qMvt+hU)kF-e4V_a53`c8h@(*&*`=thoaiq3uxCbO z{xwHJ{l`V2{AYno{A&hNxo?4II{C?uM>`{Lttz3RhNg@FTSwZ*y^)&Y!%=+A=@`87 zpJ_tPS7!}mO#$G5Ap6)3#d zJ2l~Rw>9MKB`G%I*u;db4x*zb+Y6t~_7u4j@~xXAXawXeG4n=p_5bac1ckB zc|(K<+m{ovcNNGqXJ_tP2Pfy*fdP}TtH-6U4#K}I%c_FMxR8(5)}r?_-DRK8txWi` zl9V;ogYaIHHk8f}(lcEH$RL-f)N8yoqDGv0Nxh~^@k`Q{sK#jV7Vq~GVO%0eS5jIxO$ho~E$0VEpy7btF%iYpdgjgDK z9t|@90?e&tpR~;-9zCekKTItUKnr6*tFrtzOOnjj zE^(e8aK1t7qT){nnu>@SL4mi5mU>47^e3Qei%%`+%te06^YQwi2ea+*FJ1iwevAEJ z&pl05^u_F5AwK51wl2r_$V`Tw*=ln3_3-FXPRe6RjsKrRl=z3HrmPp%7RV=kQ_-7( z$d^|JDbXJ17Fg0;SLTaJ_Wx@8(O6^J{pp_k*Sp4=91$bKJ=OEJ8#(#IAKPkt>9LJm zxjtCnJ;zIVG0~QMd$KeC*}Sm)^PIH$OD8(wc3Y|U)om3b_E1aAI@Td#wv;8I3&LE# zO+{hAk*Ol=$c9X@cb5hOTlipGg-$%RRK=f|D$~A=*OY#nBvq{{&~eu00uKc~_4<~Y zP`sw8iCJRX*==dW8Dq#n}8fRCohn5t5Wxg8{{LS9T+|eUr zHs#e{wiT#oM=SX4mJ*%3H;l{O)f2NXX6QKDL627?sMtL%C4Zs6OE6FT+#I3997LMq)*RD~>y!EZJOi#>N#$k;UnKADG1xN$KfbaM!o zv_4qjyFLJUe`=Xj_gd_MV{msw=6_O|(GXADHN60RG0V$jZD=?Nr3sL_Pf?t7w)FeygR& z-jHE@f7=+y`tX+%nfi3O;35YsYJ!^4C70744B+Q3jx^6>KJQVvZt{r;+2CN|Kw~G0XiK)WY6|w zdn^e`;!Z3i_@RkXwr{4E?v2vn7Jl)kZdlk=EiL2ehMJPMI8fz3+M4xjxGw4OOgHEQ zpeM7AEyRLN4LV8NnEuq*n&rK3q!cfS;lWG&g|9|iQlF>@p?4?SQ{2CeqCj7#$u$## z(l9kW>)DnDlf7fCj$M^N!j=R%0f1i!p6M%kYivk*IL^)U2RuwMkC6CcfRXiSQAH}) zx71ee>4~`ueI>psT=WAuHuTQE36Xp>LMT3UkSexpsPyd-bjJA<5p`(CRh*BL5CE6v zyToN-vm!*Ot*1J0MO#zwZH9tdmJ|lg_9j0#8pW0!I2eqh85VncjDlLwlJj>o7v7~Z z;>ZY31b8oLrnlnblC;ckvMc@JmaN2kX`uA8tugk7gOhk?Syb$GVxv{k_Uxm#$ zv5-qQ)#&)iZpfR7w%o@X3T*0v7#VA8%DOwpEqV|1*XP-eV&948;!n%`a^C|jK6_0H z4`1pp{ctu;soK(2g>DZ)eb!|qpLYSRd16MToSBK>eLV$iYzDs?YXyBiHsEvjHSnl2 zJD^$qH`lf*XjNJoa5#*~*ij>+mjFE1(~_MstcMfr*$>wh75>XIsMi#) z;O%Hj)~iz+v1mnsj$M=?BQEkc53RV|bs2EIqH>>o4Laq(Kq=T%(m|VQDtJm*6*S!k z585-b>-J48jc4`-`o9(!jMEWh`nnYI*~JIHSy!VoXVg?^KQsTml#=myUr&Q>%4?Fg zW$@_zAw=xDyeeW)p6fP5a|~^wV{Sb?pf>5Q@j$7Szhkrd13a`S$?+r z#vm;G#DLBIX2Ta;JVUanr{wJ!7^Q%B2^niiy=7}iyty<0{-Q1W?70) zGPi==Esa@s_q8<8ftFTypr(P>#mI18Wze-2u;(=$A@b&$0P^n0N~@URm%N|t%ztup z@qe5Y}2Svm0Wac#%65~RfaB0 zirzar({J1AB7fgAAX5NL$~Lu@y_w>}gSK^;Zz4fw5`6Fe+Kpe>d0qPQEjJ}0RPIU4O0AKJNtr}jqbw<&-N zMo?)d21LS%p(g7X;D*&fF3 zlQAa7&|30pq^%-oN7n#&NM`vrJGuDC0*hY|WxraHg5Dn(s*}HuQHfsxO+PJy1#aqU zlK(Zwqc#h}GXFX~Z=( zDt1~}?tPJ^d@|9}dGWo^o}ngXSBpsj{BGu%4F~@BB(ChpjLVzkmv~P0K|UOfkSf%a zxFz!FX;$jGzF`ub&{R-A&Gu#0{_8l%GYOo;eRbt%x7`R`Yi zh`3ENt!&rQKv-0ebLBM9J!>a2z@@B%tPB!j&jCHWqk)Fc@bf>eYs$lSt;p1~*}Arq zDMqcGTj*_W0=on2T6;21B7L8wG65~tcwk57%!>-&yCjhJ`*s3!f3%jmC`W{E=+KEf zc3k1vG>y0-$3!m;pu;znn0TP)L4Xd*2J}(N5r9WWqXh6pzVn*A(tm9b3h+h2yF)eT z{jnYzaA-ip9h&Inn<{+Lo(Z3K?4-bUtmxDoeO1`{An46ff9j((Ny(=}ZGF{|v6Z>0 zWiuWVAN z(nX(6SJ&yva-G#_DvdHLl_@VFhlBv(z4s;v@4fdBfRHySr7Q^VA-uN)o`@g_?+u{L zs_t|D#l0)C?>R61avts&mvIJDBZLHdv-ezU&ToF-Hfv_q*-z?tjNL^pb(GWm)v}}` zKPcrBHbzj{rYDVmU3t}UYG(>^(I-RXrL8A!_XHORo)$>~G$kidgx4mz>(PIZdnZEDZ>}QpsI_caVgMfsR|rUE5Qzg3U2l zQBX_hNUjMbTYwjA&IlkL)%{1&wf=|64WY;hG)X`s;rdmX^_rM%&Kc;{&C!;V-Z?~e zRLN+KYS=yQS^RCFHB0(+9Ro$)nXhJnpWqVz~%0~>%QU{hZZ=;kok+^x_{F=?EKmVG?(EOI#1c(__cGq zCwphS?TmX8a=|@?Dc*RG%~w6@{C-~2@R<{6U&k!1*78%^cc!PU-CH(@LCh?m2i4Rr-y)`Ld$|3a?ODS&hR1bZ?#|HaA{q&Qcb?HE?Qi_VF^<0-kkVQL zQcUrog$m7#3*cke%rl`|vy7IoW$?-B1g%>C0(X7o9jVa#qU-YB452Qj5wU~X{sHR* zz1B8H$O~#j^q`iFSbGn>>X<>^-d(^|1eA=9n4Zy__tJJ} z2>JG5=o#7licb~&d4Je`(Q`Th@L=D>!Ug~b2&n~luWW$joaMEt9yXmAAzu3@r6l)H z&QV03SMlJUef}dAIY;VL&LdLe5Q_i!82 zcZoYw{MPMJa^BW^VxIA7d*&IfGutnq=XYPjFFHpMm)76mukTFryFF4N$~{9UG(PUR z@P>N((_u>XU$$P?{t(mCyPbd!Ic6EvzGVT@yUgld9q#&md!qZiW3u!7%8S~QuBpze zF)gDjvu2`wf?5dbc)MTz@R4^}OaR&^FKVQ9dZnbQ^_S3dD=#~<-E)-Ytx0yB>K^jD z-5F+mNXbFC=dcB?8RT{Cy{b=5_sYL;zVEu`m-FEE_pB=A-L|v#pO9q{xe(`k56_Z| z%0Hf@=lxD4Df_c>pyG4KB)-@q<@PK+>NzhP=*)_1SP0KNG~4jF;?ogY!5^QYuK(*a zqu^hy&pLnbDg_wbYkbZytMX5(hh1j_G66ZN6;R>^J~<#GH?O^i5wzqJ*s;{ z#@CqJO9Ry>fX*uQEMRJ*1{Mj#A$)fpTIQFbs<+3xFZcmx2xwSH#pC)Ply_^tbc{l? zo#W7a-y*&~t{2ebCJ_g4k1ECe_7k23W`}bUU!{LobHX*&btM3>Gbm+ug;iWk+$6aZ zvGm{FnB%ptP6;|TrUjk4XOJ@+@1Qq53*`0y(9*l}gp!DkgFmoK?q$}}4XPM05Wjqn zoYK8D4J-9ZvBh2)rNyV>B3H+-g+Ud$HEHEBf=WhrRK-RnbQ~<0D~KQ9sdF0Nu{A+! zx4eQ~w7-L#ifL(Gf8QP&P1)GQsDa*nu*T!0K&+c1R(dt{uZTpL^bD?R-&3_n3?Z4PrMEgw7Rq5lp zZ;a2oE_xRFaE>W)H{h=oj&bCb<^Hlio1V6v@h{xr`4^tv-<=V`wnxx6w?=T+-3#mv zuUt&noMMzKhuXhYimE@}9Y$Ub&QmHQs?55`#HzhnY~J322w{Fp&9}eCW^KIc_|7>2 zzY>(QfHo44!Ws$7wLq)gndmw1TgH_I)qLFE0;Ov0P5sHW*X4in%y(qD7wPo@)xC#N z?b9*uvJksFhssHm+gKuF8S$4y*fSdFjTo5htUEj;$&dhgw|wYT3of-8^?wV&}ydDIx-ia`w*Lbe{t{{7Ohg z?~a*z#jEeIH%w2V=lt^w=m$&RKoaC!NQS9)j6rU>7GU{KITy0AB!F1PSQXQPmQ1g? z8%Yfv{*N_2KO`em#5JVWmfU1^k~<8&a{l(<{9auZ)efzlL8nKW+ZX^rY?!_Z+^|Cnq)l{e?R)a>*e%zR4$p z6{Kto^zJgQa%Z^x)XJmgZ?=bF*1pW^pnhU-!YR! z?3MMBTywbM-RZ8YF*&v>spF!`@}tk1{&|~fbA{4K=TOxo#qkZT7OyjwSNgy5z}$WNz>p1=M1T0V*;BWw}|^vE4_me4WlEX#8){d zTF7@N-$AyrZ+L5_N?2w5t_6bg{WtiWzJ=afNo1nDG9=BZAKZ9hMer)-{H3iEH%dpkI zYzPMe0H3#)#c0PeuPbWo9}KA25TAllzden-zBK{698go+(srTv!}h?-tyx-w@-h6> z_6)Zty&}2u_igdR12YW~nC-l_^`h`E8cEsLwzr7lT~&Xd>J_0tI@t2J{S}eq-(CH0 z4(vSfp@~KaOM5CklMNR)$D6(n8o8`Y|Lv7=TbhQ{~poW6h*S+@BO zaweeSzyk_C!n4Hdb}uj+H%3Xf?ZdQ!)y&$?BcevvTra|}=p}j-+#dTV@{;u@$jQz3 z#9O=boaT-9q}$f#xT{Vu?jZ#m5;yV)phh)%fX;G`6Z5>YoJQvqK6mRqB*UXF?oD@{ z@kx=nA$4XRIX!`ScK7ysc<%b!mb2~|bYVcw?OA<;IPaV$+;#yxHa&w}c8#KLA85Jw zv{A_N$Y@RWx9F_xNmBmGaMusUhgBc@rZKtxSzJ+6N^MMPgoGHFW8W;fA}~*_O`8OC z_X6ssdlv8sDLOx>B3Aq4q{g_Oi9G}wBcNnM_LexEA>BZKP}koVlnXHtH5=xbg)X?erc5pMEU=pWbBJ6Ih=<*A$d$F9+J8u@MT~vx;Nz%H#MoORmxEkv^DomHcc+oJ zJhQM8kYhEnp_!?(E42xqi6- z;g>O5T{Eac_dKc~qGk4g8pI5#sI8uPd~rmXd9STc9F!9(9h2QxOixSyYI)IlDyZNQ z(<=jyLRJxP@=nudiXVZ#lZyx;gNPp1fZD9!!FLrrgiFpvy0s6VT4wJt-`{~ zxy+EBfpkl$bzn?-JevMHKCPJRSK!ON^Qc_!JUl0=Mph<3?F^avhrHUs{?#c?^UgG_ z@t3Xs7bzQuwKWPk=NyJyusp55wEeyz@8?w-_1|}UU!>RgT*o4=*7hD!6w+}8#-F+i zHb>h4cj>7}m`OcgOc`F80O6S9wt?SQw(=Hr$+5t0b}k9JcbCPefJRL9Dn)p=j0fGC zCgg97qAst1INlh=-P)d{m2Ql1E4}j)bWGRF0CkwZCnHw@T+VY$k_%l^ye8Kyztu6t zsBlh^3S1M&%V90K-mjuI?9O6yH{U_eNkwIUSQh60^TzA8A3`8T9h1z;Ol$>}+@4I% z#I^Lcgqm3ATSVn=yoX&_d5Jo=HBJON9hJ2`2EP(la$un)Vr5)Gs`Jgk3pU?WpLUIQ zoN-Sfu1EDOxMzu2x;Kw63M!c$Q5~Zz6Yu*rEj3@;-P%GP9^OUlL zQOpTs?!k;hF0Q_=|9a(3%b6Kb{=WiEe#s~0Ay;Rajkb4`;;?$KKW^yni)pBB0U4%f zb-3;9@?gIB5Fj!#S3etqY*!@GpKU{+#2n<9MbSf z30wbzuujDCEOI(B_k&XgAuS?jc7~*+YX35>)G^(8eQl)mT;@Ke5mFE{Qj zkXts#p!pjk-Iw;w9CFgeWo*mIwc0oEto2zo!mI1=4;Xv-L69RMy_g%d_6;Of1;Ra@ zh;5u;)hJ)#F3%3ZuKP8E_hOd*!EmORm5kQ1J<*kyG%*SLc42SA_Hf1{7gAT>cK@*c z9&s&h?|rZ*r?v!C)TV%nTcb#Ul9_uLs}H8Om$>k~C0t?Lz(k~MTynz3!iE9g z3Y!ItfS!v1ng$-#bC8D{!anCBzE1t9^K0)S2Ld!R>tLNP`TG`&AJ!5&yfRRijpUXO z8+?)c38aKof9|)cr!D0_@AUonKlWJMfQ(WH@C5Bw_H%r45x^T(r(c!nVZiQ)8b!>g zQApd9@gTcP+;;a8uXAULUAOk0QnEcoF5g{bHm!~jZ?3-UKIfQ#Ue8?1^Q%Mzzk&zz zF7tbQvOcn3Ap!dfK&-M`GWTC28ZN@U#Oc_az}&FC=>Ep=sPh~16X+?&7%9WI+5nGg zjsWc(Gm2PYy?~hM0UA=_%X~{{fQzUb9uR}xC04Uf#%ha#`2ZN)VR=^h$@Xyd7mm@I zFP!6bKlm1~Mcze5lUL5|_AD|R11dho1LAP)ef>$tWaIbl1?W}xJoKt(9#ari3&@F; z!KazEMmJ`trBMT)0MxQxU(caajkeEK)#)t=XLDy5yL_6eg{b{+O9Z38`q!VFjZrC?iw{<7}rzWWarD zY~}h4?8^EK^72dw9eWUJyxZ*|(nRlBWzKz%h9`CR6xe+ClB z7#Z08jlRdShaKMrl+>=Dw*>wB>ws^q^95lGm*$YOdsZf>MSD_m{oi*6-~PPS|5I4c zf`xTFTzZ|)JKPZsrPl@ggoO^Tw+3e zchbg02X#EG<|pi}*@33BUIhXD^A?j9ly_eb$gpK`EwekJVTC6)W6 zw7Qg0Ks#I+e0H!R8H^b?sGx$;0%D*sAnT{P7y1bvsR-d)X1DqliKYH!YBj*~a`zmy zJfY&__bq*3EJZA@n$_u%i%EX@kZ^BF0Nq`nR_!j}i`Isb7adc$9Pcu%4(wOG{tj|( zdlY`fw@j)4`n=6CNi5tLL0z-I?LHk?AXg<#lD>de%y2GHDjhS#Vy~P7@u>kH0C~1C zMJczugdt;>Y^!EFXbNqzlskzvcX6Qjo!$E_1uK=~RDXbSW14~`leuRGHhH=-or zR$evbT3$5%Ex97##8cMJH=IDbhfGWi&?K*uMgeDgf>`QPurVJthenci zE;p>DcRA@4ix|iTpQ8f!_GGtCfi7EW`1L8cbWuub@ z0X3Y7m1RPua|&0Od8W@JLl-%wJ1&Nmgqnzs4f9HQ-Fx$lTE{Fh$MUk_%gy)gr{a2I zecD25J+Kiw;|5&IhgA+GW#Qm}#zY5Y+-~18wca&HEcGmN+C59G7I6ITq?U*NRiDUM zt*bBF&%0-kg%K^IGi+cZL)t;f-tyyn&V_!Ui2&zY7QkE+jPk8_*z3AS_21cFwtnYd zVl)6=0%{$rX=j1ctb2~Wpnll#HTYfMN2HWm-x8%|YYJcD0{q4;?IQwh*u6Q=s^6X$ zLhWNB^!m5}0df$;Dy?&S0$sJY%;@l{gxH8#(&t!YwX9CyZf?(0D%`(5yG1SwE66o5 zHMt4!p02<$y#Zh?Hlz}fUGqQ>FVpI`#xU1BnR{W2gyOJ@1=*dY)@+aCZieMtNKC~= z0PS&e^;Oe%n{PTVco(2oLJCwKz)g6}#72WU(i}7L32_q}1vJ!c?_$@@xQ5mdSZ205 z#&Ef5EsuP-@n9~X=TJAsVHbB7a8*GS4;MG~OOm#}J62=Xi(#B(?S4DjZ8<(@UjA=4HZ~DRbwB_sFIYN2b zF5oNxUdg(`^ZC@m=~9U9WIkbvhwcBOo%S0yI8Wn!#rl2x_!7M~;Q zFZy?F`9S5ce!L-}bWW)Y=I268K$kzeY&f zS)^9(EV3Fs(z{^ZUrl=C_xppIUYc_Tkz;t$@TF&-TpQI2NlBZK=gjmcUSu|VwS9u1 zzL%SMuh=<_zP0hT^(4?4=Yld$Cx|NHh79HTBM7ZE*k^ty%Q?V5X*fFjK;kw-0fdX9P)?tJ8%4YK6PiFS{c+!gaQ4~{h)E^VbIXabuHja zjnC@O+FrNhM^s#*YgU9X{)j!lJkzpmVF{q$6M0Ff=!hU};X&ZQSVCx$?unPG57Cs}a;G&`$9ySK% zH6X{Lli-=4sovzD06%%D};R0Vc=Qy^Oey540z-HmpSz1XZZp8BGA{ zX-!1Sgm@H8pjqfuAvq83mohrS3VgZwY5fWF!{%>*-suQvfcB7!aDHVUJ){&9Ln=Nc zr~^W(ulokIccUgKLr-QE~!ys+}J zfB5G5g?;ZWR}qsqa8*;k>hqcKbiu{X0lw*onyY{Z>OUFdp_uHe0OFLt!ZSd(K7{9QZg93H3)vx&V z-b=0zZ10m%JCl~){*=A%PI86INUaP!jhRKfsF8z9TG*7RflY{l8XM5zDtDK9E+;{p zxTguXos;OC-C1&5M8zThydu8)?^{fE+<>fg&XQ{E!=${Gx8(ew_U?0!h7AMUtjaH^ z_oS^{$uDa>ZeWSt?4AbtcOH|wJq61OscBsq{uoqduHSW>Umu0uvX5aKH zV>V96-;?%{{W4yUV;WbqG*JG@48Q1K_NK^1ai9aDQc$-hfySA~+)f#IxR{m$_shuj z+oSbowubAz@ytOk$Mq~^+<5OrL^mJ_D7o-3z%t)*W_u7i^}0QlSX3GzKzcgss;Gn1$vENF2;MMgG`@dfVDG? z%{M-&``qxb?i0hKx=%qax2DW}12L13omvrc6XrXQK-@mq-{3Kmb~@ZSMK1Qs`N*h2 zB1&Z9*(SQbcqCr!uLvPco;=#jJ4#PDMFEc-B`97)U0xXi;iyK7uB3d5RC-0*~ zv=V+)*DFr12*tl__dWo5fC>OC4Jw&khigJ%)Xc&LG?W&=RjUIsazo0(L>{iNNNEeD zC!!)WY>jrG-kV3?{;(##7f?`IR!7=SuDz}QerFt-2Ye&}{n?w)4?PO22Zr1rmV#CR zcYk~E4Tx7_=Dm|~TFobhd+-Y}kgEwTr8A_3$WdawYXOt%S*Fz`bxdedM``sfqw+03 zR(@)ER{gbS9(psPqBQzdLY#Yv)4DxLFSorR=9->BPP)gjxt1pl-zz2MU+ad-KlTI8 zoicDS0VT85{=WOX<$2A?h#XduGUDnyOSr1N1yX%L%IpZr#JHg9?wydDi*(N+bInhx z|2ogP_PgEp9al5!4@GoLSU^SX2KuqWHN$Ffj0xJ;ej;Rh=Xx&1^zhqJ9lknl;FF>N zds5cEUZ5KW!bTz8xx|5Oz9&^U#<92DbMVadzRWeNQjZc>o?hiL(kuP>zHu&6spkl ztogF$QO&uXX-u_Waqo^tI>0r(!j!Cx^xXKcHTWi|wXtry%=Zbp)o*PucbJI3r`@uQ`w<7Na#EB4TtBShq4pMl zz8!&`cfRjBwLQ{x3dCgz&@OFJ9TyeRa$uSJkM4P7F__QWKr1#oX8|vsqt&|S*==E2 zA2svNo8@W8x3b=62F`#}qvC`C?Wmj7byM%FyyKB(m?(SZSySv+iOL2D!1SszA z?k>d%?vPU4odPHC&p6-nDSNMVuRU|kj6l*1?A`M0$*)9LM+Iy=S>)pwM2)ysN@AN@>^ zR9v@xj?UX?hGqBkM+oAw#&76lw0@f9KB)4?_vLC)u229@CY!YJZ(*GM6YbAXaQiCH zljTVvL5?2WbD{SGvC}qSBOu7BgPN@fsz-qq>=a*&;>ot5TB};fgw9K$C`AkibimE+ ze<-_X@ou|do>xwH!-;b3SpQ@;J8En%^JDGAZ@`m4%faU3Sje+FR!KrdZV{V7% zHj)NkY(|(DB!mOVs_Cn2%%3iV-KK`#v{pL<&dO>=$yzDeSk_|N!>FNh_nuYW}d2)c1YLhBXj31?eh<-(+> z36RyFi=;^^S^?R+LlL?FqPfs5(zY){8YPnA&k=cDN7{qAH!Rm=Sf- zF@eTp^S%G|#^=Kizv&w&533ezr9 zmEl^P4ZZM)T8!@srM)rsw^XNN8iX8$>O-_tZzZm?Gu$k^)N}dpwB@B|a%a-MTXcIs zlt@p{SEkIYTqa7VBAn_Fn|x+KgG+x;s;N9IW8KmPxr@UBJD%jWKa0NX@re_p^m1ZB zln^z$AH&t#K67?R%pBdYamSJm+XngP@w~q_awDzbwLopjWtrVVD$F-CT!KZ&1DXXTds@)~)->EqX9WyHPTUUp%!vA^>n zmwzED(6_zZ z!M?ZN8lfYtLF{w-@)w(qBxySHgrLG#`)1_$rwd1GeZ~kDltRu=$b^kYEM~;G* zRy9nKcZ5f(3hr(VxVX@5PVCq2jit5^be{B@DR2)&R+n02%s8({6^@14IYc--N;^DD zwWgaw;jjEr_ogT*XP=P0Hqnq3mZ;~`0tZ}5j$FX`j!C>&?Z~q9@%!u=LCFHmg#lpyGN7pV?Zng?R zXb2?Z?vJiHF^+QRwO%vlWikjJOn#5bP`LCu3ZXrFtQMmEYR`cE50T)H&aEkcwwT4L zfI9*&H#2lGgX_a;O=Xv}mbU`8U|Su1o~b1=w{o^QHe_fQ!eQ~B^wJt&>w zCY-OIv5QO2;=HNEg0C}oXnE~UU$x2?+_9)f{Oi9Uv5qNLfqx*CmE9#?88_Jp40m>g zvZbwoV5`eqNlf=F9vc`sSO5NNVDwFfbHn9j?>eMIph;Pfgap$xm3?fpG!KapyPQLG zAAVOg@Ku$2mN+^7>&uNmsXTF#AnK;BMC{TJsb34)eGOEOseM^rWKyn>X~}i-Y4d10 z{O?Glt=3I)=Mk=Xi2h>KWt!bVXWVr?&sY6r2et1$EJEdZoH*a#_V7C_lxlzyD7(RM zNYVZco`m|@F{|V~&d%#$JXp#2+vzL6ol-r&H8{-+-!)=|Z(A>A+7r9r!3$#8@{V`U zIwpALd43yoPequ_1^&t6w#RJ>RCm`xkW01KR&{MWOSNdZMR|)mND3jJg74}St@<0L z3*DtpU*caw9h(?Qp&13|)(&i?2r0BY(&RDXM5Ytz20R6(x~n{mle}x}r1SVGsR>MdTdh4PIKLgh(*>ot-3CGx~4Z_BJz*U49(? za{d_POX}{e-ye8tWuE8@Sx;MB+|=+hl#xtgjIK=Sbfa!{a=^|!1n#vwTRr(LX#)<2 zA0GqGKkFH$S2DG9UBgu0V1oKuvcBiD?=kkE z9Fy)+)sma)xZqx&M#Gpi@f!ZK+58>4YqMezZwoZaviZ)nx=M!f^Y#I0YbKXNeSglH z?%;F7bWh#duvlm3KWG20bFN?yJqn1m{4+xJfn{HDWe1e+8qbp(uV9CUsE@_tgzwSA zD66E*MpHf`m9*G47zH3BIr-%**2=T^-%xvSXD%ovE6Qu~9heuFNYrtRan|dtZjH~_ z5#0+pJ2xfD?ibXD?~7u7&`|>1T**HW4=LtT&oEYercK%+-tFRm1c>@Vb){`r-@J{0 zme%iM*A2(NTN2@)<`$|#5?{>xTmHECdqn31+lXT%^qyl_1D2?%wVmgRnx*VTS?%_4 z4h8)T!4R|13UBP@T_0aG{AK+xwaDfYwmjC=3# zN3hM&{-#(rhXt425H}5)77>5x&J|IcLky{pV&0wIpHp4ys55hqli)bOet8EKyg0TQ z$<`hzq*~6*d>@78_QS8WUCeoZsW|Jc_B1HbWOZDz22J=|y~1S#w2eDUk^2@zDNsuQ zlJjkKlE7`aIgxsBnIGd&I7TDB-{wiG{J5bowAlU&qqvq(fMAeEbhT`!SotcqsI~g9 zfYPa3EE}qekJ+LBIJ@Fw>apq=LmcI(G@fd3@O4j>srYzvsi#WpYL?v~L>up8-1_q_ z|H81$rjLG;W5IVvAglu$x|MVq&&-1~tvN+6%oJO#%LO~;vybsq1s07!9RPWlQ=g0! z({EDJQlURw(NW8_U(Tl=k_Utf5nRi=`}zIzNjKI?TJqLm$BM^3klkjPN?Xrijay!; zUR*S4YJMzkv%PK!toWUXwb=qc-##(JK0gCtK26pespV!8$pq}Z&L`BGE#wlj$%NcN z=W8j&*5iq!JVIXJivQ0);p({h#H<#&`4Rno37%AE?1CGV4FEwT%*mlC2lnevI=Hw5 zvmni<1%O*jo#D#U=Z#faZJs)mk3loRyd-v4IM56IG=X2Fq$Aos1u zOD1!atNwz`*@4MVV$GcpbURrUBMVKMwb|`R4FsI&uPmSGWwLH}_&!K`J+_h%u$pvb z*hUlLb%9>il@$QMvZ7GwT13$nn3!)!wE$ zOPlSEY|jTVzE`%Wx8057ep2*LwKq9X60um9?*ZrphwNgYT5?)0{_~Ha_E@LMczwM1 z82H~`u{d^9>5tt(IR6l*DX}iy(0%Ohz=Mt1Qyel+3QwEIS=8enAD#z4eA91UP9IR# zS5@zSbp$Y+Nw1WiWnqncUR|UJlzw)ue*W`29MN(`*5eBjIjpAVd^3JM22Yx(a^gS> z&^sG;2MD|sCr*8mLjYR~eJp_*Oss$EUOC<|3#Tbdg2VdyHdB($adzzK5)5s^HD%n9 z57T)-1#yh8L`HS+hK1f)nkNQD%Q9MqmXb`Aquf;Xx`R^TTIZL_{Wa5FEp;-^0DcZ`OGRz1!ngLlq4P%lK+%^F#l{k4!M!_5ys33xfqt@hCL=4Siht)*o$2-(3&%B-& zZtZK`03YJ3Z)C_)68=Zn$5kQRTlKb(V;yQv4C5b>l#y40XF&g5-G|by=3HyXuZ8UI z$PEg$!5v)i-EB8j7bO;i?mHaFolWn3FNmMQREW>>Kux4c38Gp6jZND)x3J^X{O^Pu zns3&`Pp)m{7Vx@oqaQaQ{PE< z)HdPlD@%?Biw!9kS!SA``e9$59DmD~PP?>dGFXp;U!!;zvsl$d`W!;m$A5hAO6-$x zO^D$vezv$yFmwl#a9m$^bE=00ww#$O&kToiJJzTwILp_5fqQdEl~Z8K{MFFV2ys9& zxtqV6@$^v@4P=hChMxi>bw39Y5^eUEu<3XitxKe+Wf8aw=$0e7e~yJmL?oV8)+$hE z0(qZe5B$)91o%{T2M31;tn0;;nw?f_P0<-hO2DiMDW?3JyDIPkqU+Xsy0OC03jK}4 zvvg8w!nO75MeuVRvHxfgrUS=aZaD+wcrj>vnhsH#)5aT0Zj23C`{~IgwA6LP#K>>L z)xfRM)0Fd!wg^jZ1^ji}qOvU03IF%Lyc5z95fgiS^T0vfCr~r2g$RCDe}d3bj}13n zf8fh#3P{frAQM2D9Hp~v%5{)MaeThu#+HXz3vGNs=MrwYpB+^l*<0Bs7;uT>_e^#w zi2{yw79roUIo&xnxz++cv5PtCU^t3!xQYaMslJYXq>CY$jz`&ZRJdzaZHH&YU*sj(287)1U#kJtHF7Fl`@R1T|GK3%^K|0W+#6b=R& zODBYRC?3nSbQp_ATfZt-MQbro!=`d`7TU!{`!_J2CXkFeVoKc?q%Qw4v2GpVA=O14 zf%FFCc8jd|EdJ9NZeQ$+RPDu~GMA<+^!*z%+HH~COu}~daY||@gp+0-SL7~fC9Y7H z+ih?!xBPusr_qX+$_kgGH|{#8y8gG=dXUMJieU^(ER!28QZNrvaWpyCwQGv9NwkUo z>^&0tMXEmoN)k5DbqpoG?MZqF;tAelZU;2DIeUjVbs?un-OodD{9KX9mdp;Pmb$c@ zSI)8~u|#_$+Cf1@4Ze@l39hl=uerLlsDK^r_f$F#dOIS**i#wQ{!~Pa%8?OZUuJGY z%<~N%Ig9;~Mk?hjS0`rcd?Ay@Ho*#gq0XY$WE$A-Gp)pEE5|r}YwmtuK~LsiNeGh4 z=_HyOOW9Ya!IcuH=c5Ah>?@676wbg+b6IjxWOn+R!nP3=a9K+_D7EeT#*n>XmOpe5 zSJ8p#V*-XTg)T);QjP#W#C>e0AK!ZX$`PT)g5lpG_)H3}qwE|4Sg$Op>QlszGJBiA z#S5G3s@9;`#qdaopybWoTQ#^(n4&f^#930BrAFh>0Rnf&q(3da?LzD{}F9M-Cc+K#s^v2m=fcy1bmF6?m zvT3m?5ftE%^5hfITU3rxq|X#wLD-xL+(&Uv=1v5tu2KSvt~k*|L9U6i(WS!C72)1~UpUOKVvo1X1FFmeSOwi5@M0@MHJ z&y(4(lE!|vG8~O!eO{V6YQ8H2i9qf>iS5D`nS|%A=*WcVGn$^@*rZ$5Yat6p0B@bS zXQz!nt6n}NU2$D-K3DeUv>RzX!oN$^?%*A3*uf{!bCE*{9xn-ZfBNrD;OzG)+p4VK zoP8e8kLFwK)ZF>&O0(eiOM(9`XW#11`h2C;6y+)u?9||qL<&Whss~U`Nup^%A+_~a z1C03O8}L04dletvH?1L>A2CZ;@8lxxSWm=_g~ZDglQW3pR)Jy~oFT>3Cg>!Q{&uv< zCWB9u%dou{Y!Xuc$TsNl$Rd1-*e(B@G_zCKlCGL`+PPL?JRJ+whq-Wfqm~;@3U6|# z9sFK>5CYQQ?FHHInoiMKjyoAPX~eQU1$V)?gC+cWuSB1>MApuKboI0P0^86;5jy3R zjw@Lt;DzbS!v4zs7Y$S2bQi0GG&6bM_a#&ynxwM>n1`gISGkRZQ^#mE1REc@?W)}V zj+wg9^{Dc+Tf!WMF(k-JS782M1PJt45ds}&FA#kn+rejcsD22QiLnwDx%B9NRebaj zlAIzJn}g)Zz}4vXBO*j`gRr9l%b>NKM$;F!hWsz*0P@|y{4NCO`{Br;paxqjQe-9d zqyn0i!fE3~zve2~FC6UIQ*SFI;E2=xVoyF@PX&^>|5+Cu0>Yk8wKqa(ApITJb>+2l zptzJ#{@b7HKKp_L#B(KFN9k1HUG@OZXo*2rI3_Y;NCReaKPVkU8f$CP(?q4$%yy)r zsC*FrU1!4N|nfk;*b+plDZW6HaNSo9?pl3vhBZnK!T zhm_6%DDkh+@$1-H-%k&+l~v8urIhC+$EM9^$KZI!Uwh6B#tHWy$JkFEF9|EzTa+EH zSukToxH=4Xg&S#V3V38%@6q1~xqtI~gLx6)|8m;F?p!7Xqj-M-%-aI8e>Z?OmRht9 zCPDzvq90I|rMsd_)cw#c9i$diwi6GTK&o`5g~V@I(HlfctXtk7Fxebc@1T$>w68+l z{VcUhCw*&ik#sIgA1+p_CCtz`7EMVlSr! zGqN$ud>{J*I?Mmd^XQxd)xsR5HWg*YFfkoX6+v>bKC~!;y{)vIQzCR zFNee^3AA_+A$p;|o2E;t=n-89r)uKB{$eaC+>dR7|2?JJ+Xm?c5+x}N#12YF5xd(|uz&fR;>#h(h+{Sk zArJRB1$W`E109MW5R>bw!Y5TL`5v(da$*k5F99TWx~h3aTy>gFuna_@9Q z_R4u)n6(pndqDF`+sW@4z8RD5@`lbFuH#=pk}Lf+Vsshgr!=871Mw{>x)|INX(k~| zn`wu!fr`ij!mRt0!(vcRUl@s;h#UnP8rc!|)=oa7O4@_s&J1_^^6MSXE_a-~mw1=C z?H0$fxi87Fx5y&ROt-e-qmspc!eNJYCv@-64s>9Y_>0Z$-WXS3VR(S~C)k@;0!>zZ zgJ%7=Cw>-oJE4 z<>D06J<}JPMj53zswkCkTK9d@T+hl?FxAETFdmus8Vl@L18R-L>RBBWRMs)vk2i;& zWe*R)iB@PNC1(FwS#XEJ=cPN{TPpGbpzJ3WXsHo!#u2JjLvA%wdLS@!<#HCeGe!U) zb`+e~D$W|JpO}40TzoY4SkdeG^|HcFWjaQtQBRZ$)dKlFlY+Nt0joHenDdJ~l188d zsUK7m)L^|CIE2+GOgIgDe(BchX9=Q2+Rv*6*LfjDd%qE?lJWD0xr5maUT`O4H%e;u z8gmr(fAfk(cLB~m+F!48062Oid83Xj+Z6bC7Xy~UN%;H!O_(KSk0V9(0KRx)N+&L} zVoR1W!PPRO*k#G=5dAZHLgkn&r?q)bxL<^_l~~(&<}`N+y5drN5waJ(c5`~a(w2Gu zVNHvil<~{rboxWIz$qXcyc~@G56d_5XGT~oaiwn_5vcXg$5wtO;$2cruoqi`0!JC^ zP{K<)@NS2}4<9`KFF4iUC~q$lih@?ae@b=M8}A*r{T5Kp)yxYh4u6>N;M@V_(46F;&%Esv7-EZ8SeV2dAobS{tiE^(Q2kz)=P`BmS2P>Z$&r zt~PdsBC4h^T?Z_08zM?PADASR!ka8IKk~RO;%y%%Y&3a-|h}ixJ z{eG&A_!Bb`tDBEa4Nk%&MRYAEZ2|`!K(MBS32PlERO;?CDM<%CSr6OCuGE3GSJF_t z?@7oxeHki$|JKxE=OUp_(#pxyqaT2M`>Wu~w96JV9>T+P5hc4KZc_-x(Hh1fIb7Mj z!-+3|`nJYHJ4R0Fcg%YDuL>!u7{QXowO`WNP=UeZfUu-57T+qV1nDC`(014KMI8toP{}!5UJVlzc%% zUxptS`ZOe@<7a{E;iNRls-FiYP-(~ewDeIA0M&!j{mbn1E*1Qa#xUQZZ);fIa$*lj z-b?(uy&7F1_*V~ytZGD$v>x$Sv2FqF0C$I?>kj{}`*??v`}iNkl>iPW9RA<0MF*O? zEPo^Rp8pBjUmAMPLjP_)e~&ymXS;vs+;!|Ng}s`0kH651((YFK>xFhcRgs~E9e!v? zA)ae|!oc73PxO=hd4LQ|+BQ~!5k_pRH9bu&t3n!25Mb(r8R%8o|6Fn6yV3sJDB^EW zyINdPvYce{e#+mSPMhPbO9uPXqc4{mD}*1~;!?j82z9N3SU~e|JR*+C?6AaMC#T}0 zE@h7`^(+2NbB>P|Yyx@3-w?|LfJW82nBqW%nfgTA((?{uKmxJMf9=_+mkW}Sk%q+W zUx>P!n_gL1ihZzxR{3B_Z7c0!C5^&S_e3-*l#J0tc_69%2r|zLfRYoAEg$`t4b93Q?|FT>FeyAii4qP6 zCl8SR6F<~nXb&Us!n}KrpX@%J>(S8nbp<|Kg`q*fD48TW7)f$oo(VPkc+@OM^N0({ zHaS6>!$3u2jmlbXn$!`5EbM~!U83Aq0*53Q__x`jS$JX)Ms}JNjG(OCvO54}Lf}R< z5NpgsizZAXhB|$wtTx%vwoF_+?P;Y?}TW;1Iyua}M&T#2xORz*;bdLRLprTi&4Z;|5 zfQY$Ej;wpB&K#rcy{Vka+-_c3**TO+r>p`-pvL)aw2b%zp^sKISId7Ma_cGSPtQ4K z)D-5((2gP{|=i3DcoCPsz zuP%r=m=U7hbID8EvQ?}usL2$vpjPa;l7bXXbll6TfuLXOy z(_i~;r!F0ieXD&Fr{m}1KbY}hcBLEBHGY9k)5ZAWt*gc?Sx|)+&xoNkf(w! zJxv$Ao441PQV`m70qY2NM@a>`KOam`PJ_5C{Ns97N!e>zXQ`P;J}&%)(wYyrB0{@G z#K>!mYep9FKhB$#uPk`~xaNdrBoWK?BFt#1=~1)6I%*!P*rBm1Vse| z*>EFsu^B*gwC0)7l5Mi{_Ay^=X{_Erv^IeovDKom03Opq{1FKcoaO<7*^P2H2az6P z6I#a^7o2abJfd<_O8mM5i1z_NDZlBm3Yit=lvV{x7B*`Fx9tLoIy-&Y33;Xs*xtm= zmzIJMt;>bvCE;YfwugZ8Y~%n4~$+anInG5Cr;9gdz2H|Ueoubj+ zDw|Y(bqo|Hj#S2MoWvVI(wx|*6-5&num7VNj^@6@J5o>FRy#L;g!n*t0uRNlOLhhC#u-j z4w{?W&aT*(t4WeGAJBuG^utw=OD_i@ROh8)Z&8+dMqBKKA>J%z^UT%fWpnfI<(LSb zp0UZs2p@1FqEa3GqB0%L9EyR2E69^-&A~^+)LVB6Zn2?NWe88p8nakrQu5JfoeMO8# zJW5Ml-!;-Cq=w0tzOg-{i>rT=^sC4TIqeqG>=$;XcVJw#NW~piZZhdS@=YI+p({upy5+q<6Es-QdF(xN)tT)7{5si9oox-s2@163P8cL$ z%k5Vg4z47~KKj1eMGL&3_7m}s&}6?y38axGH=sD!zDfO^_6JlgV=6-ViKK?avm~hM~feguCNp;$IDcUtyXR>}R z+thN5<(9Zze*p3_6$+n@S+^gZ3}n2%05`8o#7wbJ+LuA+%N**9NAW+q3#(mzky%RZ zq$>?sV+p}!6LU)isoye&cD;685W#Qq(0+|0whFCK7=ksSkNWh<>yb=kFL6lyH^4PJ z{a|ljQXP2y-yA=PCthWQ5@*D$rr@Z<46L^*A`oR{q2>MVXT5%4u(iCW$%5gRM8Oj$ zpq6Hc%{dh=#P#pAbmvsbA#9DwVYmeh5$rv7)DSGm^&t_QWKu3l2X8}1>EE&}|NW)N z`;q&p@qJXf6EM#FA3mx66@%y>vf-Vi!C&vrj*$@G4%7c-d4n7l^|Y$ zmXgTx?;QoeN_ky+Wb!ZQR$@+9rguTvg@3j$Su+gg`Xg&=mkY~09Xv3SYtP4?gp-&9 zvIBw#?3N$F89EpFYOVoGsvGb2NVJY%Giq&e0rNv_91r1@DB6(gCt6W9+d|xPaSQ}B zWTc)FtTg3hS5OP3J}IZtv)lX{hDYp5MY_`CX;m6#8=3?7JXl>3$sQa~(&qK|Y_Q#) zBU=RSqwSUZ~>lqUfP#s%#CuCpZx2B-njr7@yBJR;x>g?bh ze8W?+8m}1k6u8br=EB?wN5OeO$&2hcReIO8bQ2J zHmgyd=Goqn862DGKP+uhfqk5bfopTU{cN1uw_j~*DsPBi|SI7&aN42 zgbmzt#W(qfN?QTZiQzi6>C%_k^I;~V(*E^Vc5h=&=Sz|L0{yJ37*jxR(r z#g^cUX-#7!+vpQ$>gJ)>s%v?}xtiyOuNe_ej>1jfh`F1Xcp)_KYE)ypq(1a|X?f8y zi?3LXC-M#SMO;h)mV0n&n53|)&9B^I-=8mGWgmkWn_GOgRp|HadoNv-$W7JF`|f+OqI?)mN#bGJ+rrr!gn9buhVFlU&MO<%2`Qme!n`g}R)piI!q356Q3ZELQt)OJmIUKV(C2(Il`rvs- zmpaon2Uz^NXD_kA^)LHc2@TsUjJ`zzzY;qLj2c{h7sRq7aeocn0dd(7rY=}NuCh@618(e`80jaA(4;_ z<&nY3EdRkQtiiOUeapmGetbgCQ;o@~RH-w{a8vz~Zd^)O?{DOJLiRp8u9OSJwL}v! z6~D#qHC2+4*QcoL5`5}<)_M9$OKpwG0pkZ)7a5q9uQ)BKb3?bU>ppe-jE&|PO_eT^tri#sw!1ZMA@BS#mLLvn?$&C8{#qGFk0D=fLQk=iz}>>-Q% zY1sSsFt2%rs?tmXu!H|v((Fs(u1n@5Ok4I*a_x!di18=0?f%~Wv%Fcg?v0r-bDU%( z`dR5eBe`LwYjD{YpHiLi;b%0%J;`rc(*W=Vi#(=S8r7v*X%p3$&^2dLsE5pQ+I_7rcTAc zgmV=rFYp*z>upD)>`{(2_^m9()5cv}+{l;OjLO<29`6$&>%@IT_lduxHU*B`yvhVo z82+wVXuSAhh6AY^9UV`v8u`rL3b1`xSI`znX={6NU*(&ACWU81uv4fiBv{E8KX=_e zgfG)8$7Wt9nc!=k*LUZz;Wt#a;n$yH(92^KI9yJ*^;G?}QVl2nn2%Gb*M5)JC@{=~ znvQza+6bq{kl=Xz*7e3^w%nKgN!eV2v;%Ja43k~>^gfH+6CO*Hi%;Y!2Yg%0Av@+L zcEf9HDvR6Hufz1$7Zr^qNW(%1bVed(Sx^uWheDiX)jc4ABvj34J3`xCcAFT%eSZLK zZKDm2PpE5TaFlRI%Wpdar(%2Y@>UFr8lJH|``U#5LidouAbxqK{ADt-K92>Xy%{1* z(4ZjW42!`h6y1;9HJvngg-99K-HL6!#u}lI*GNgdfQ?VSfZ^-pfbgH}9PRUrs_d{w z?6_!k@F^eQt=@jH`7|%w>5Df^(%=>XDz?`==y|#{O=-oNa2?6U$e+}AW<@k+km=v} zj*w!|nz-DkBU-(oK9RRZB_^CoZ6FpR_`z3CV;W#8R|8``Fiq>|a6O5YBafSv2CA`B zu#4$@)#?5{W3kY=@z|gh?g|ibuB${$3dHjMdTQ<^L*2CV3y6b6g-*9(Lt_n_iCKg$ zDk8{~A#45TL9`Z%1fI5(cyGF$tfu;UaX0iYXKq_4^lOGa{m{16~ni;)A zs0+#MMJ7an41U`3)PEi0-QxJ@h&m@eAR(h7oHN)`0+^-gTala|x`GlbM9miFd~Be} zWc_2pUvlq<@{a~CT`xg60a5!S&8Ua_p@P^93Xp3RjD`wN$O&4kI z7#T`Vwy}5_6@9$0A!wlLx{@%$tb(tluJn~as;Sq!lD;lVZrF~YLQth^;VM8U}F&V9wOw}Z#RNrLh%wDst^l^vi; zfj{@WAa73Ya>}?4*n^cQmp|q@)S$MO^<5hpl&&dw%6O$$i>hHY1#2RpO*b{9XzyvH zajnu<)zzzDQNUf?md!Kg<3&D6pC^rnt^D=d8zDEXE~{vMRM=X7=0$Ift@$HEs_r{K zHh4L!;9yEh$d2ggfA>(I{1Cq_fc3*AZ!V(CYVHBYS)vvkJ{|k&#Fd2Ia2par=>oJ$ ze#1*D`qs$I`};54`E%&#$&f}&XC&Xv%PYERXNJ*rRc~jRv1#ChK9w_w6>B`%!foO# zMFbI=s~1!oB5a3u=k*tyEH{F^$I_-V&Mdi7csStg#$*e3oV+ToY?41Qc{|fjjQf6 z*_K`tI>WFyd{KFtV~obbWe+{$Esw_vOZ%gZB9N}4CJiAZ238PY=K&WudBYg^si-D%t z%H8f)o0)y0GaQhlNf*IwC`mfmf$*(Wb(C46PT|#4Sa^Dg8bbGLDs{E$mHKX4=f_-xr(P z=|?&m>RP#MlaUOIOTZUq?y{O}mNYez-^AtEHu9uRwuz;3ZlI~3$UZM!l0udR~){O5868O*o&u_|QU7#U-(RzVJ(Co1zy zh9Y*+Ju)(^a`_eky#P_vPMjvqz%T#tJEkiXvL$!@lyKo5&7}&OX2cs1FayZ#4_SX( zMg$T}+&n;Z(2Vqp+}lt%=LWq7hHmQ$rYX7khNs#FOJCP;}RF*2y2=LS1yU_ zsqa-Q1ebR3t6U7ySEyVbffk^x0t9|ae-URof(r?j=T^xY69$5)hgKFW z4JQj+AGGyMvf!QQuSg7Th^>y#%flA$Lx|-?_A!)ZY0;3TZh$G+oKIr4hikg1fqZO? zv07rq(NRR}FHQmdr`{fCv2{SCK}`SO&qHEPTKWq5GP#6v#-v33MTVZoVS~q7igzxU zm7SiFn{7H=C_$%;V&vAsu&*D=mqpo~-_;)Ru!(Uw{xBn_ix9hJ!q!cjwg zlCJ`NA|ai`oUueCIRBbz-Hgp}2`%RI-W*p4$7H4phOND z7&N+w%OvFJaB>4n@^Y@OH&0iI0(3BtTG=>??uB0XT!~KCZ>XiAMU02>GVlBFoS!W8 zaRLk1$Nc)UfTaP8uKsNz@f^dSJ6qTOIB{-8I96)=l6Bd89AD;^khGhksh>a(C*mbm zcikty?9~}G9W08)Zh^0@H8fWIE;|1AeHXsaXO01vlILt$y?Ld)sxBP$92rcE3`R3v zW5j~1^}JYEe_dTzA8R+}zf4w{xXpnFHyF`H>Epi!BMu98@+ba+?fGFqKj424BYd1> ziy=(=MQ;L4fZ43lPJYck8yj73CQphcdfkV{(!>>xTh!7vh&ZR#pFBS^R4f)=;+g0f zQ)D00J(*Aiy&sp8eO`jBcA}i7T(*VB zz5h~eJW+Ah%i+(~g&h`e0UqWd`wZmiD1OY4AX#aEf zon}06l5z=8;B?zsx#on<2D0;sCS{lWgCZ?gRbmSl9{Q4VvWHTm_EwS6%O6~y6@}|4ws*zy0*v}*N(b1g$7$5;FOxu zi(=7@;>&G+uup9bVA{uJPkG%XET^kinO~$#vh+jfKp{piH-=hLgsZ z(EKVwTBgG-$@;z%y|s;vq?%q@{2T=AyVf~u6dqM-FV>ggUp=%ij>qkwHlOJ^P+{j0 zbbsyRWVXKa#v;-U7E`(3>Nlp%Ru1-@w2?Re`;BNZTkJcG3~(&q(T`k7GOVmt>a6kGdW+ysPv*;;OR<<$KTWcH7 z^bjR`5a}3$CH{Q!*(Uq=u^4AfVV$U@l@7H-eok6^03WHrh%(dtNelxyH48s+T`DQ7cnX~zPx?;idr z=Ii~LFaLH4e18EWq@wj+;s&_&+_biWL+VT}y+7dF)@<|K7I!^<8{orcy;lDg11mYeczYox32s?k! z+G>Hi(b!rg91g1~*}pt%P^bQZwWhO|i|M`6K1hXy$o^4!dNV)pace0^%?VKeT$U-z zQXfI^K_K#WB17_IT92 z!MT&{lw0GyaMw^(5S{PTL?_jd_&!J$$}Wl&wS{y2M2CmcS5GtcIyw# z+pK||zlWzm>Ck?+EBp)$C8d_#Hn!_RjD9;~{47@flCs%XfNg`*`mP?>*F#tqZzdv{ zZn{d>RZ0RmWzw21k&De%fPSMKwUxgX_CcKj;PdI%!9tV(^gS|3)Ok)Ax#DJ?Db*W$ zvlOx4WnO~Z`;UV1&q$r)B>y4Xta&yW4iplcMGoXJV*9~aIF5C(8J4&lirzR~a%hd< zdSU=IB>Qka9+2ByA{HE!!V8WJRs&NWG)zr9n#LO)v+X&Po>3E6vw+r-G!)j3Sl-Zt+ ztFD|Y8@$b}zj>kxqjcgU7>6Ncb@ET7utZ;8vLew=@cR#VtjJa<7)XJ;KV2ktu-`0H zfk3=Qp1#Cc26DMv$Ae2@vkBjEwtI(wP2*zY?^Y?3{ zU}8(M@v)&1N;Y3WbqFH3i`&x^;LqZNp7W)LPGK6>!GQK(MM4BxmM&p?q{d}y=xVkJ z@O|YmtD*R#6C;Sc&HV3~u<_D0=1$5jUK@TXa*tZd`1_#8$KQK>>WM?tpCJ2mw@KN#g$ihCq40%k)muGu@Nq2ixPMJ6mIfTU!&H+?65P!?hv&RiBdhz@sER^g>;G z)wHb5apGP5%dBs$Zwe2u55w=PC1k&bkKB(y4yxIu9wq+n_GIPNke-^qGeggIjWW~Y zS_$m~WYIY2b4UYuLdz;~kCT&jM({V>ac>`c(p11e_z zF0g$>$1K_$t~kH=uJjk*7%MlTk+MLyg^+#!@TcD$5?N49%kxZBUG>kP&VPW+zU!!c z61D!}I~1tPdz*vA6gbDcf6sH62?rY+F)>Q~DstNPIQc=uEEW2Yed3T!V!*T^(BqJq zPY9W4h1*KZb)Slo5z(^Cklt|{MAY?hR<>6|$q1Xska?)tdsZGHX5dtWb*vI#31Dv_ z8@U%1S(lE_NyG`{`ktPTtT*IE;aq_74F=_{Ims$q?!#S-SoxGF@H3LTJTsKcEy$!HqmbfJ@+(({ zIoWaG!~}3m)WE~JC&&+iT4G9AOUa69+2zO{)0nfiE$pm)f~-p>X6rlSFy;}1kQCGj z2_6+aXML#h{8C@l`NiJq3)|!Pd&u==zgfbG*rdWg?n6QD$JHbIJ&5?U-Md z-w-Zrk5V75^x!VIM~Qdc?O}Q z`C&-QDD**Hd#Bljn?sa)$o-1j!>shx_vCBtDR%z)5as?-C+^Jl2=zX)HoF7$zxuxF zH|MMJBeSooPOkOfF8Ze^sX;9zb9K1t(sF;*1@|;H)vF?<_|?=**C_SD`T#x2`l9@2 z*AOiwqUKkI)YPoNH1?)<0)263r1HFX40}DSrep+_jJ(Ye^1X#l>~FqFUioTI`LC

)f11%4^CuSG97sn`JBiDocDmE%^l5j#gURB(} z!6JT*c$9yJn(mvXWxzS5d*K*@Isw+NlCk_+g}^n%&hu+{RY486!ZS&J=t9;Wl+^U) ze(a^y0pfKR5JFNU6)C*c)CA3-|k+ymi;cr-h94lBk_a4B6P#ULB)g z3&v)tmvUqFCGp(C8|t;iPS!nbefhza9?Zo!=zScrj1TnFuf*LAYRKtZ{XlfZZCR-isOq#DC9<#R)Tf{|u)HVIz-PO7&}M zjS*%BWZ;yo(dx@vBj|HG)8*&3C#!EFKi?b{l*P=o4Zu@Lz~3lfjSSGg+Y7JozikhY zZ>_z@o?Gm!KDE(Hy}R1YPTm~gXRo|t-nBkKAGSRzKe*mQzOmj|@CaW3?zuD*~ z-|;IsMO$N(Bto0DCxd!O>xB6*!H%F)sEiX|gbxoxQc0lVR8dfpVXUpBy zXORAw>!?1t_`2e#f3oU^f2Q)r_5>+qb4*gbJt3)vxvJbArz9b?RTbXc* zMZmU9AJo5F$;#WFB0gN|L;bcn2p&DmNL%ZpB`>}uT-h9CX8Wc^IM9$J*9hglYnYm} z(TBUd_P+X@v!ndy`OeB;m){UC`i7ZV|FJ>VhR@!fq^G$?h_^Qfko!o9w*yK+)i(U@ zVmIcZx-s{A=hKQ~t{&3$#W$5F7T#1GTX>EBb)yG&&G8g{Z1xrL0`v)LZ9q_9ZYTa` zY(*cLdyP8_?1}cPSjfHed4BMJ_ZTaCWsq`jXA0R{C6`7mVtxRa!4EyWHG;jm4J@!d z3arsxeSWnUd(|_=fF9%)1(d?7AaafKk9!laR}5{1KbW5se(xD2CL24bok zn0;vgcS--O;5)~Q;=`N$*vqaF(mmwfu82j<^5_KB+hd&EZK#i(F=iI>cSPnotIyb< zV~)8 z?JsaouyUaGZ@K#MS2p{w=a=7BowBx-{iJUw`Dv+>eSh%{|KaQl^7)0A#Pf@S9hQ%!4DoRb(60xy(3+oeOYp7wJY~WYukgbmfxbTZI6qpRt9+a-f20sxR5zRGk@@Wqi8Qg}Jt?7t@hiT6kM>bme{duUkXd>s!OPYcA;P?GbwN&L}l? zt+)D;Z&E_=P1g%I$E28rw~V`U9rPQ{7x>e*w(^4uuPaZu`pNgbsf&2k_L6dKOkDo;xVZ3hO<1KjFG-fN}?p;mT4s z>B`z5Eq&&3{!Pe|WqviQbiJQ=a|7|XQT)yMPSkO2L*CcsN5$XIcVf@Y)Ri8yv{l~< zse~|&oZ^*U!WGBMilg?&B|q3&i+*qqlOKdN9F*fV`tHsUr_!%#YT6nVl+SnJL7T}} zZSAFprX{JL4m0lm+4-dSp#D+mnb+j|`xm<_uJ77OMO#B9r#Jdhr`Gx_ptcGRtLsue zU+OIRC7`6|#~{ypv_kxrl3%ghN5AiUjybXP26JluRoO4+tn6)#( z$X52E?6Iz{J1?V#rdWi zr0av+oY~jtW9FvJ&&_q||FYC)?Xxsy?wji9zP|99eb>=mab℞^gYP zijzxS701nO<%d-Aye}-R#Xqg};cqT>Vy_1#C3N=)uY7BWnX=kPx&eK5ap4W-S1Yij zp*ej&^mpc3A2)mcE#uDCP%Uw_M_RPp&A97)QGVFeT=b2px$rw>edf0#(uZGejnK2! zp`JDeaaUJ*uou>VZ5O)APdZ+f9W}IOeXDIu`P%t1=itsb{z23z=6I+173+hfd&u9l zaI~QgFLhR(UPU~z6L%JR<=EO=!Y%I*J#`CebG@5zd8MoBH^;N$gR9^ts+!C%SNq6! zwJaDODiNI+0v^bK3oKzoR(#Orco5{*Ei&Uqmre5ZfY$(^pcpg59;$b4{+K> zAL+KCzUaGQX6i@Xqz4~$5g&Zi%gg%0{)&8Uv736^`~>yW&L}O}HB7$i8Abl~OVybv z+5JydwV7WVTZ#^hh#u?%P0sTxMWl^BW~yt5nYP%8`PJB3_|23g>(gm*&S%Qn!mnRc z-~G!7KlkAL8^TTZD8G1Pke)LC27P?~IcRPT^k-|q4~|!5$KZUv@(yvzSBDiG^DD~r z<$hMCS5;rX+Rx6>Hy8dmOuzT%0osF)pJHzOlXCNi|1*`zl9owt2BV(eJ zj|bVQ9}RIb{yr_u|JM-X{>MFp8-E{`r+$XN^ZkD-rP-&ppr$R)NoV!VdEaU3vi2!! zvOXUdq_{0r8Dol#*K=-ZW*PV|Yn_R@p1ZKxyq`hst?HF^6T?Zk^?eDqJ| z`btC}FD<;mpWW?uldOW} z&Wcluufg};qE0zq6dePvJ-XES;5&8Q&3`QoV3W30ta8^FJ$(Z_$n>Q8=)z0FS?lA< zUuIuapM`vRO4X2m0J7y}goEZfP-m9AX?M--RmU|ArQeyJR2-S@qTXE_;^rcpHB*!J z+2UL51#=tvsQEE4bsy=L@p0)1T|>!%*{7Aqe28bf#GIRy6#b-atT?{-mYHg3KpmWY zMY&{X#h;jz6znrLmmRP_!=Ii}lpJEEocNRB5peH&^4%eR+86!wyMLPzrG9E`Ed0*; zwEXDiP}OC>nv^}=l5zMY>C&IHk5I=~`|B9?4(9#Inv$RT=xJZ~QPcNL@RC36r(OHU z;#<_!#h&658v}%-pti0dZV^isU*{j55M2LgN^<9)j`r+t9IZL~?QNw$I$ltL!An!=>@8b8w+pICv8tl4jCE?zSGvF z?K9M;f8m*87S6sypSHGF{c3(hzpAdQK5BY|K4g1>`Nj5>boRZxJB=u)ez|N9d2 z^M5%zvJbwMWF6^bJ^XgHmy)#HO$Bek{06;o(%N45!;~cXpWVcZ|N9bs^1r7fN&BBt zQ@(mh%{T(RlrsAge}48w)rsYI`16ZhxO1i_=%Zc4%YW&rzVIht?PHGD)n~wGZ?5+< zvzOjcZm;!{Zy;-GAtgCAtR|<~UgaJc7p8r?)+a@;^a%21JBU|IZ56-RpH&=RdV{+# z-$A%As>nJ(OFjCRZf441*8nqJCBO5rw(h|`V@uwFLB_pLb+S8OuD(eP)HTA$bwReUK%O)`Mjz>Ar2YLJCFw8Y{Pa&4*Nh(8bQy zKf#}ykY#@{DSP<&N+;&x@>|?RYkS!t>yy$$v#&6xEzi(rtWg<5}UsUiQOJ8j8;T?}#|<+cw(6uiJ?a_b-9Y7$MW>n@|UIjrrdV^HM%-DZBa? zZt8FUrD-iY+Dc9NiJWx$FK{|vZS_;r-J{HGFXWbup^6LEXGx!HYHxh3l;8c#{u*)mB1mfsW|u|3QArJs53BgeD6sb>*&3uhbdQU1 z8^ipPoe2qUV_Z&Ncu%?t8TEvwvEZ9&aq>S+4Jn_^y~z7~m5^4Heh^_5ytiScO-rZ7li0P+#z!?Q!W(N@>bJ%nf)qt+n6b8LdyJ8@0^S6f=Za?(d(Ah2y|FQfI-`43 zdStok%r9}LoUch&#>9pDhq>AR zvbC1|Ra;le8HWr+AzQ*2q#mV^T6(jY) zaxeYCq@wWqA#TPeqk^-}*THz}!p77-P&_R>7se=Un>nzX#+ccOV00sA`hG($zhHK6>!k zgz(D8!|d~)EOnJ$*cziGcXN`yQq>h4i|EC)uwFpg9H1l_8gurWTZ;~kiPOGd-T&#o zUtn&2WqeY7W3smRglC3F+!`RJtbr!D1}XP9`mr}OEqOmaCEWP<1?A4Z`Cd}Wcx^f4 z8}x~dUS_s?f}FECg1J54g*st;oc+Dwaoz#fIIqI=mT=wt3Uh92m{+*o%gk7OjXJUV zuJrh3f5n*%$hPL!m@8}j6594yi(=uOB;VLd{B>BAyZ;UG?%xb`#Xsx}G9E^CGHy^^ zD_-xHqnCT7Xy+TsHS6=@gO=8uua(k*F9$g(e}7$l<9{7b&}Wx>85ybu)Q__-Soaq? z*!N!JFZ^#W?bgTI`uuNnEhWJHh5H@PahKe~;%fhl7{An2eRgYrQ?Ne3E3&ntkLv3R zzMd54emTO+`Fx0zcfi?3y0_9(Q)OzWUDq{MoYvJA{h(50{!3q%^Tk>h`pniCJ6F?= zyP;ud>IHyoYSXBP$-SsPRQDn~c*%8We!3yq@iYv)t^Z>rkduXPRiM^@f* zGE8mQQ*$qf7uMd>?yS8hU0;1iy0PBFM)@ZB6xS3VqrhC+2f6A-+)~%NGs7%egUq`# zN=?}wA>Vb6QR*+L+n+4H%{eeCzx@xj zEOXz|OTy(T@Ih(jvA-|9Cgz28Qf5HSDfLWKQa6Xtmls}T9x~Kk`N;Vq|42wBVF1Gy zYg$l;7JC@?4XvdIro58AqD=?&=XpZD>yPkF~B z4EGQ_3p6%+YY2CD>21!*g^r?Q#^%f~XJ4RBFTEG#w%{LrG0{|Z!8OjzUhBtSGdwOn zKKrWh*X8cY3-fP^kL#PWzB9h8y!4)z@`bjs^n{~>ban9^<<1&pi|sLf#cU7p)J!GPlwIVP$%`zQK!|<%Wlv0k@NaR zsfV<+Io~aHR9swpUv+bPv^vE%%|QE=w4Aj;%%wi=?T=?)R^9YX^D8$8$#<6DVb4w1 zX8dbLp1p5cp8e&BBYbDj7JCTSmix#L zToY2<%6n?^`n#&LmezuU`iAUpKv})< zy=ABXn9rQY^K}_Fwg+&@RtdWp;PEF~Ui#fhUgli`FXQ@75B$4FN-J3ELWG&>O5W~_ za8bHOkDjm6Vn8V6ip-<-VEx0B^S3Q6?SJoDP8^A_i zv-TrnoD%}f&V&%VGrblM7sWHl~Ldb$z-vGr+52Ae~AM+N-9Q z9sv9QWyTk{6r~~ClA_Ol&oP<*SrZTJ83^F5(ZaV!=IZ7nlSsTj$;H~GghVTVj{wI- zuZ`p4oAKYT%?KgyJi{WCOTq%XCnz}wI#S`*P)Xp%VEIR@l#%0mcVSya%vtX%4cs1p zykBa~eYQE6_0l_uOxjftv-YQ$2-^fLdwqzOxG~JjGd02A0J{5wwHNkrwH5pn$obm~ z9gxqa4%nwgO!UQmT*{5DK6c)=gq1(tRQ7V5lXO+eh`D5{NqxN5UhvV zVk3>sMNd2`R{6FB88O?GSmC^|7HM>MyOJ;N8BO z0{>wm6ddXZdCm!B>{55Z^Tn>z`@XUMFjFDhKcr+bUSB zTgNB6)k4bFC_Q(%3mpyk=%CH^f`CH>D$XrK$E^2MeB2sS+)+}%bKMnRTxuTPsba&d z5_I%(XTjUe-lA6<{e`c$6^Q6vBc)^;5P6nCc<8PSndni#qc-{qKdAZfH%7>z=VUZM zAJk;r+t;v=`vwu;p%#*z$|^b#t5Ca)khtDi{>ogR=Wl2(`Lr~MjalqLNBo-S@^@sI z)YbNiH}lQVms`W=2!{%jVppJ3Y!j#iyMc-6WM#bL%}t|reU;FnW|XXs zAc7bB%0IbuK(5U2I4crDdf4+Tpek7Omt8|ulSRS7EO$ZPEOz8SUhB_#IM<$X*W8%$ z?AIJmbTG}Yng#rkQ-e)AFo~)?IstiC#i`hpGmCA*l;q8RQq)2ZKC}gscx#dld+Qpg zCs_Mg8^b(JTjc_Pw z`3?m>)iR6;wGI)Ywnms4EvUG&I$`nCwKiy6AjVBr72Q{hz^@kD(J{_3Cj6&fz_5=Yqc*w=0~cB|?=N%|zuHysQJZ~;;OW}* zJENqyn-jE*dm3KB`}syV;F-jj5mv?1I&k#)`9^HszcZp*-~A%{XmQ}?VDamu@F0QCU|?CRp5&QkCI&u zXyK2}k>XD~!{E2`jfpo*H8D3`Qf!7-BW5ggQKJD3_-U<=l<85k0lsEc*krhPn*tMO zQ(+TaQ?v@_Y)z+CSKn+^a^aRSQrc=CGq+nFHc8Jh|o9~#9qn*gUSb-+LB1lhN(y^t?EqsUCRhK2VS zxI~wdQ3T|Bp+`Y4-s~w3)biraPcULGPOxMBO!aC0jtNMdug04%?zcus8BRqF)2;;a zRZ2-+ZiBoS!$B)a?8^y)=p2c^{n5DJ>O43#@H4V__F)M>5A?DWXKvP?0 zz}h%6W_64JUhG2@9_T9@_h+lywtYFG5zsA4N`c43qW)Omi}$B_oaGU0ET{C_bxAEa zV{cK|Vgc}Ey%)fG2`t_stBAFZqqFS_eBse7hvm{!Doh=P&$gB5cF89+C7CZ653(dq&tL^lNL4<$9&;5S951w^t zijNPlc5ySUUCiX^I#RHb279o`#|G?5s`2X*ewn%+@pife_uZ+kty>#n6j%Vg<5lyq z=7xd?Mq%>hjkZF6*C;(}se_v~CBy|R)lq^LYUtm#TDWPheo^^Wn;>^uP#HCWCfqiV z(a&wI{QT`fcK+4?{Ig?}oW85z5nbBWA(vXra4M(;vrR=W)V$Q2%Wb@jmHtM|a%W}E zQX}=-MrUQgQZp}QsgWMG*}+bn0meE^gx%|dmR_IX5hEQ#g0i(<=%>}*;y{-cleuT0 zQub$S`@OTZJuW?uWNa#YHPccO2>690mzrHZ*9;AquPeO2*;Nv--Cyy|IfBjf$mj(- z1N;JWH9cshS&+IrTu@75%ruqvnLZOEya+bUL> zdyThGKDn3$OD@v_j3!J^L#Z3WL) zdP`of^<&~{OJc9ImZ$sadDxehUPh5uUL$af2obBzxNp`jQnXuAg<0ujmKsDw&#i-) z*zHkF?5>Im_Ds@C-KvUsmGJvbDKqrKayRsgM_W_r%aKjMAkDSFU$1l!6E*>jw>gCV zzSfTqTW%*rYT3olw9MkC({;Eob0acvrlsJSsUh>(TzlTT{W*^4=Sscu*HZO>MTX6q z?X3ve)(P3}nI_r(LRH89JRP?tfktZUvLBk;avts|kginVsD68b1KH>!CjM=# z@A_$I=>B1>X|j()qGwv7AI^8DKiSqGWA|qnxRpM1=wc5mMOj7q&`m14+=|IRE67j! z9UcAcL|O3XW9hE~K|v4igYI2D3p#)LB*^dVZ=er%E+4}uy#GVa%Jl48FfW5>rFtIkaOjhz~E znjHJ}1U~KSiHNrkKrzoB9d9hpx?K-Tx~URC1J?U!Nj52?WVs6;vD}FZ*%)AF%ye=R zlXI>}{`*_^{)-}+*i|0XMkM16)rG)=AL@m6ptAmFCyg~BJ zQZc(`3f!t9gH7!ffvW1fXB!eysYfs3Ee+8#q%6oyITLzsk`I5O=OaF9*s#}4W%1|e zsozf`qk>Pxym)dP5%&2HLQK$UT==KoW^0jgZZ#F{oTQ^%MlOAQ0vEp_LxyiiU_pRx zinK^s;OS;ku)do7ae_~IH^!`Z-i^(?Bgl)mToU^BRPpyO$3H*t13kHN0rc(B?cYL+V9L&+kZnLn`ss^xRl$fxi#KyaI)MC4u1>Ka;^H&B*U_~`FOjw$Dmys5A zR+Jn67b^P8`MjXlX9R`OH^yj**A$$LyUTs3)L+Y-n*ZAo3@k`l1%Tan(n>D5B!uOi zrWWR$ER2c(g}wF%J-%@fbmzi((9>&ILHDm+1pRsXH_+{KXF;LQAAoX$1CKSAr(Mtq zv+wNdN%>AAwZuGx4mJ%k6P2yJ$e~K&i+WV<8D4SvX?jlT$&#pV$G-dD2V(UOC^^6% zl>6}=DERSR(DN(jK+!KBg5Ev74LWoB6e#>D5X)bl9TVqA{bg!IywTM`9*?t%Z%$Ss zpU$;0qm;F*pmq}R4x_N-PgruwvFwl#Q1ZJEpwAB;fbL&73wnS3I_UfTJD|6IS3sA} z{0=$+0)a06ehL`(8PGew8z5jEPsr%tJM-=656c6z_~lVCQrpIf8xv7pH>2UVS}=$k z4T$pd)XbEV`CmUDi+}wFlo|LIl=m(W6#w!CDB$J|&_iH8uAV;+`t8IC(EW>NeRBcg zBftS_YSEUOhnec5Cyt77{`JVxbHwz7lhDYJW6;nb z5IpP)DB}5D(5qWlKre4x16@4*C+NcOCqWM{p8?$l&L{Ml->KU2@GDb|Mb9kbh_IE> z(svdqCTg)A`ewcZ^Jz>7d(=gOT&^xJID?FjJ6RU?6_ow)6)5)Q6Hvg7%b+KhE`pw4 zz6g4F@dD`5pMQZ)pFI9woO|U5%=??0Cscy`&t4smelRVp+%@osUL%hJ_>YPmIWcXi z9~Z8v$G&MqfUhE>BYs0iL?6!#dH-aRzshRuq&Pn&|oiWN^&lu zVj@o@ynS`-8E}j{KumlC@c-$ROQ6%hT3!O;JMoo22ofH85}W+}>`Z6*cbgm%F+a{L z@2^BZYNkLg^WcSllJmi*(CK+6!7)iEf?m8jcJ2H{&}Cq5?_ItMdUES7DDvf}li1X( z3-t)-eWef|Jl_s~@6=KX96Anid5l{!Ga@Qgch%$!)o_EV0RF&b#vO+zL?5e&jXswC zH3;4D{jgePG>w2j=7|DCW(}W03*RPY{y6U1~?A_>Hj&pDpy0;*FiS zFY0RSD{T$+gSLSg+{q}v(}06tuR`OlBJ=Y8`1(2k6#V=J=;Na&p!=7vfUW@N`5eH^ zXVTuBx6V;2qX zR|(1fLsVXLMuez1i^ei!+2kX#@r}M zh&d7R>N)7cgFB%3fLEXww{8J*{0HdVZ@+;aUGW25K6?T5^tvA?I`HH1nAdMkRu&{& zn{UH}@5m{+>m%66)d6hiTpRk6xs~{3fQh=_MaSF~mV?h1gohnV{`3Kq5EuxGe)Ssk z!tWO7?8#H0n}7ZZ%*S=m+Xv4N8DA7Nz&I90T=Ju{EtndPzE zCN|EgW0bFr5#p7#kVlF-+?y(R=~eLeZzt2l@%KH}j8#X!m%499!^HLut-6Cxg*wxkI`EFA5bQdMu z&`AhsC70fWC54|1efbFV^41N|yL%5ne!%=*{^KtoFD`%{0l9eok3avr|2M!I2HbuK zdVcE;s4y(#cojJDL^mw{&-H#(%+CdWmD@x|?HKsvHF+g*OIuUDBoiQqdB}(SlHAkq z_~;X{0WXflynGGJO&}=X&Rx)7r+x$7I&&5{?oH6s+qXaufVE6|`{@`tEB%5Pmhoh! zqvET+AF-9X1!3Jp*7G(r=~f4U^+XJ#-9x58&J{=G{)J2~xk%0} zzu3>HivCZS>%QD$1)aCjGS- z2KTFmqHi@|IM1oMIKR@^(u?4j+%vfJ%=6f!g!8nN6u%B6B2Zh)NM9bn$1in5Uat>g z!mXN0`ld=uTOO;%&yF;Z^aHg}Aw2m)4J_}rq(&UqF0P4S!%2_uxsWRr2|0g&!!l2O z^bY_%xqcCdjmj^oz&<}DZ9`r7osiWLgdT^N32w#esIYu`unwbX=f{s!kpt_f$h(;0vMYH}NvBf2Mjik9 z`ZJK@cY)Zw47%{gIndKP0ie{-_!Fp6>lzkZ*H4I4_cCI;SfqzdIQ)GsxcCY&EBhiUBjY?ICH2Dh_d&mZeEI?u`R*eq zHYntH!q>=C;U7L91E;0^L4%h1P1KP>mj}48ZCN#aTV6@9D%hp#ql|1rGvbxGn;N$? zUQJZAaTBYtB{xuMF@MB=es?-6JoI;XM#8zY@ZdjQKfDKe`^f)fc6{!Y#JI%snQ;lH zCRgTYSD_SiY9NRf2$_I zucC9Ze~*0g?pS!hYf$d@C{TV>!pZ2*L8o3nya`JB`tCR~H|38iO!uLo@E(2p{&elTh+tv8@HRHqgOdA8sUX zPgGOwDlyZl5TLhYoD#>RnBy?Ds_o{g2J0jTxv5}7EE)lBMahLNskpHD@hZ&hU_DIV z&WxDq;3O=JbqJS6dPU2lo!q&B2KZuMUHL{IHP)--z`Q_yIrWu{9Tlx$TgEDPj@MN= z6+;y zUAQqOg3S+ziq!4ZsjAM#d}UixrnIg(MqXPRH7?|Q9-v?!Xqsv>WQ}#nJv>feuUHf@ z(O(Z)QC3lGDjw3NWR+}6Y3WOyu(vh|KK<|cI*G@?AuW!e!k1*YMBNZOwZEPfG}Krb zIo#5cEop7et|OxT+E{f#eWLc5E*AfdwwW0^-(Oq0JjBo2R^U?YI$D{_Tq$xGYZ~oS z_3b-TjlG`fn)V~W1MW>xODq$llw~O{O3_^LQqe{U8)#s}pt91=Fw4ttb%`6|TX@{h z1J%MWEnM!42Cn$^coQpRQPM21P4=igvvs|@W^t`&zCp6LP&>G57S;LgP1p@g*ruA6 zw=hDA)dT#eXy!($8#&*n`>Tp)C0$ih1Jz~PE^?S*fSEGhL=Wx*bZ{3R{an@Aju>yP zjMEM9G8ZQ*(XJ^0>z`SH@Shp3$g86u)<;nh%R~6^^$|{u79@={i?Ttqqu~?Q1`*L3 zQS9aQA@KYC84mwwrP;h|qCt1mlst!?PhM4Wp|i3|sIsRreY{l|JJDQ|C~skXSM}9Z zs7E{4`msjrtc034H%yG08Ni3m^&vkw^#ZCz&O^<2!9Q$`AVPj^)JWaNHigGj*Xhtz zS6bASpA)6t$&q>q@_!_sBWu-AOKaOG-$w$}rJeW9Ue`3dmI@C3|<0h47fg z)!Q0&g-ZkE-Bhd0Yif}mnCjZSv*H?ymW|v}k~5cv5W#B_O!SHrn+0fv;v-X~z&%;T z-&NNGvCAy+8i;v&Q{>VuB`$ee%_!ZStZ&*Gui?8?0)j=&L#<8lP!@R=(={n#+NGp) z+aNJ!y$>J0)K8AtkT!AGfo*xT3b8R(gIyaDl&tho6YMfx#hzBk*pd5sh*j1enOfur z#`+$YmWg!f7?2$$18kLXN;gM1h3?5px?3k=+ci9_OIKZOQ`QQ0)V0Db1-ooTf{$92 z;G!%O?1Mcal-&RFG0F5<={3Q`xI+1wLb{v-E?Zsmy~v`xn(b;1YQ8<|$yzr!U~>@uWm8JdP*kJhbhYTL9eE`c$RWmonMd{YwA!D- z<=DoGgJ;{LZ@RQNu-91Kv^^=H8am-YI~o?!t*hjEbk((c+DgW*nqIa(jEvosLBsZ^ z1p<#j#P@N`wt@xOQP7GUDteh)56q#Fg|%x4fF4AAwhq&>_Vf*%?kOr_LsI^Ebr2p5 z|Fwy_JA?o(kxZRtgBfKUY_{SVNf8Rtbw5#yR9vua|Kg(r0)bt9suMgOY zp#Fasxx9bpnWS9}DSNRC_QBi;4cwM7p?k)vj@_B6woNUy&^0fp-CM35`?)Oc`kxi9 z_{Th(=QdF)mLz!rZZp2vYZf=`PmAmK%$4H9#p*sFwm5shdL9}WFqfK`u|8A~u->2h zbk9J8uZ~c_i(QD2oiSqio{@oeFLJs2tK#m1MNyMYPc4==raWI5Mpw9JYr57as0H>3 zSj5p3sqDz?tJ%yzxfIBRwchNY&4Hq?dwNQ#O-?U1HRj&YS0!Iw?8te%tHq=^l*sf$ z6BWBZQ`I^>gi0S3MEluPSj5qC&8R~`%-B(4lh$PDq}f4u%#M~+u|LPFIb5xkS*FC* z-OOY^%Q(8|*Bp;$R}#`a8gho^V)OO@e%sNqDm>FwE$*4R z?rj|pXHyc=cE&5dIL09#w?>Krtuk2DKWoBPhl&lK?_}n#chK_=bhXV#W-iUw2hbr! zg>LmDKQHx^255wtSIqULFD#?9RO>JC)Vy06=%-Wr#g4ZRenAN_r@LzM{X0NHH(V^s{oa2)Xp)O_CxmwIEP2NwO)DB!v^9ORCiRSd)`*e1heJJeL@u8vRLRMyZ% zu#}rk6$ybh8IQHC;-l>ve5zfIjB=<^u~u0{s6_()vaQ6Yuga*Ivpu-)HaVZZKT|j0 ztJA)%V8eFCNoh6-GKEu#j{ARub7UrF-ZSiS9w^l&3s~AOg4K?2~ zjtY0Eut~oDv}?e3iu%ImKyGL3Px1H%bCvZ!7de~*Grr)+Oei`uQ6S6RWv`tIChEs* z-GE)oMsJR@iyeBgU`H$F+SF_e5TEr&(>$h+M|V`11YdswU(as4nvk}u$7g%>l(HjJ zb@PD%7`GOkyf#q$-qcv+Zym+O9hz7;UoV&gBdKtIic;#k&u5hpGY+P?)L-*_p+ydj zwyMx+`^ya)Ku^h?la%t!v4XeOiL&o|W@7QdJehE~#1UE)bnx03Ib(mOveBVup@4HC z`JTHtF!0EG00w$==v1!(o$fIZatCQiH)O2bR}SBE0Wy55O#x4^X|U$Ph5lwCcab~MeQ9!*mq2L^2BQfJzoLE`JvT4B_~0|TXecalfk7$zn< zl)hd!RGfDjm1dDbzb*H}zs~kzLf6KSVH;9d__mUm=bdH~+Yig8u7pl6v zOTvc3Mc{K*SmV%h@QeMJP?v_B@9T-WHHMGfHwaiBy@2U4vWeblQTvbO>am}z{JLEO zwbZ4fmhR8Dm|WBKeVZx{Y+FSw{CAl~`QK#<;eVIuwEtTXH2>cUug>jW9x-*#fbhSCdZ>~1$)>>QfE6XS*YCxF%n4k6W;_e7F?@(Py`KjmP zj|?~fPmw8hMRAaIB;)ZvW*XFow@y6`@oR}&CC!ol|a8b6x5uxenPHG z$|&8ElG0X2&=D*B_=ue`PT}qZyKGm>MA#J6Twm@Tni>aw0J>)1$OXJ2Dcv$oNLrU- zBCHcsAeKeqJrj=&Xbze~Cl>8ZRaGAuC?!XFbmE?-EO=W6{kkbZeA^s>2X8BBU~@Y* z#I6$3JX6)RZaov`0`S`9>))loCeQVwf@_iCH<$ZR1&7nst$XIGI+uZz@6aF;0RE4% zE8yWSfLq+aIxR_AWkN{UWwp2{e1E3Cf7eu1w*zP=hp(rK3=w7>hex;!Ou*++($)vj zp*ATs|Ik$1c?5VpuR+XrYdL7A9GAA<4-IlD=|xrvF>!#9cyYC(BK&`rYK8!5K$gG$ z-4qW4It}9lv^cU!M{({rrjJRL|j}S zAm#vAgBjyyJ<^DiA33$8(jRji_TNi9(a}7Y?bNf;8xxFT*Hm@O{%mc}_5?9^zCHi( zQhULpJvHFdW~iuN21>>D0QB2*b<%Y?BkuBQZ+ZOTG!^Yq!y+uB&=3F%lU#ZM!>JQe zJTq*;-ZYu$>re6@0C&AW{_UCg{2%ia#IA{nvrke1++jdHW;XH1BA4@Xj>Fi~lgc`Y zq1Vh^uXbvsBscp5o8}AHwp?G|FB2D)H_l z8|Kk5D)#lXvYioV@Q#{?H#C>OUuZ@}{G1dEyh>)NM@CBDmEhvn+l!w&fLL@&$>}To z@NaE|h;!VefJ;t*Z|xJb!i@p=mz9A+|HXj<|Jg18hx>}&{byRhUz6ht`uXv9mIok- ze=k?}AI|c)K;D+PfE-yFEPgL#e!t**9!TGW{>Cl%;%Dna#@UBSaq|sj&t__J@0hEy zZ<|Ev*QZ6vmuv&2@9k1@rnVIpmLKl_S1%&|sa-}d-BEI|yIL09qosm(#$gdlJ(xcQw$6)xm;~E8Qg@-D);r*U-?jtziJV z1)Fp@Nyy$)lZ%~dT!u?aNZXcUV>U-IVN;C-4_1dUiTYMZ!eUp+XSbSGXqBM8Z}mgo zYy*1FJ3&Z3)KSa!4K$y=<{X*@te=2a@YOxp9!Et3Jp678@ZM_^#8h(&5NlEf&Z?ke zHYdmhc0kwePjkpFEv0BjfzNWOsb!7{N|s%UO91#fc2`M<_?`*fn-U53%(ZQMGomV+ zf|zWT5t7z=U>|j?_zUh)bn1_($|?_lNiH3&+^%3i4gfFX(QwfFIwo>kfljr^F&R5* zD)`VWs`+1E&D;f1^Upah%WI~T?dqTrHhKA%&C&ArRw?wWO^OJy%dl~-2~yV17%tYK zq~v&jb91Ok`4$N(ZnGa6?ly49^Ad6aA^X$0`EG31&&4|F!K|ohS4W52)!1aKoRkLO z5g5>KO+ft7Hl_Ha4GA%QOG+)Y0An{d7Q8iAG55@PuuF$dvZ;!`S{3=9k7jX2e=jk~J6cNKOjqd#U;haAOl6G+(Dp}* z;?|!lqE_!T39_q$#jN%hzn*T(d*+^EbM}lv(cv_QelX3TIP^H!(IS)mbCHS%d~osB z2<-bpN69P81TlSnkeFd?;bhtst<7$2ZKc=1C+};h1x^jEcw0({b4qExS_f#`04CBg zLQLJ4Gm6&QN?urcO9B_$3tp&%v6nUc=(E#R*{|0}iD^?UBJ^mddM*be*UdVPTEBcEA^5|lnroL_HBn06KNTtX7+LOUWC25ah#L#>E^D2i?=Fh z&~*ts)*^*`UFc4GFx?XGw=-TIx;zB=t`(;|+EwC;{|0oAM~_T&jDtT~1~UDZJ7R9? z#ox}YwBA@lu>jk*Wsiz~29i@TGgQ$3~iGx1^{Dnn9$z2>khi;EyqirMbuj_qf zUp2y%$L89sCoVZX-vwwEkFm05Z(1k-V!d#E5cPGj3H)%bD&?wc2pVP`D3912EdF#j z%?JE!Ri#r!hc9=PezOi$ge`R!`p}==@q+bT-xR!EO@0K6*kk2{MtcH zd$m47$Ggq7?RFg-y9SJZ8}JIdIwtIJT2yy1Q`@~hhK*YrBc!@?B9U{drhZMvDA3nK zzRc8@eX@_>W7oP%-fl~vQ2?IAF85b_w2fm^_NQqC?=-ex&xlHO>wGoGd^L^<<-Zom zNY^Ab1JGaJjBVf-TT*nI)4)PGw9JY<6PIzcSXsZL!6dr0sHB|qmq+)RcmmF6bw78#&3P>lk-U*44VS zT$Ec)&G#DESdW&J>5!wMm%H*_=!J1tSGvmHE;g6GvG%}%x4NM3H#>lQXe#ix^p}3L ztMO?)H5o79=`VgCL?_%onyzVgYWR4&iVbxE{IoHG2wUxkeA*Z)|FEy;b9M|uHh@8y z`%?_ek9nRD_*~t>_>jJO9+YE zBgBNwAyV90FDhug8~M@dyIobU!!cIX>z6SvfB^?8wC9?Z`^L9GZ&4y{6)b+3uug(=8Exi=DZT zwq=COjZs|Uat|dt4_>y6c!A<<}#k9=78mUU>9e z@!Bvk*(e6zH}W&D82Kp|Wz=tH`Z4~$8EeAs0C9}ilrRd`d-2iRL!<(Sf{AfX5K`R= zWXvx!6?SN1puJOk#-WMN`8f;Vubu|kH`mN3O%zOJP zbb?*Rfh_l6!nTIc5%zIhoK1>N*c`+~uJ<55PgP|Yd@K0J3CID~^~9%E0( zf*(!^m<}x)u{%kJ9!?9Z+*4xFhMZTvqpxeQYb%*M6ZBk{oRGZI1%AEMp6l;ClVz7fzq>T(M2i#wd=j!0KpKeP@d55!H`i6uMyVe8$u+mW)Fjt%Nc)hFS z&GH~7bdZ~Qo0jzAgh#%4O{HUc{qsvc6x#wdD|_=m%68foNXB`+pDHSjtpG#4?wd3 zxm3740{J-Klyqy5_~rC`M|t2vN5wmHZON0JVOF76Q(tc%XM&g7;qQlW$$v4E--Cpi z?|+k#5^h-f2+@irYZ0!pS25k>>EW@YkgTy&7$ZFGgS$fEu-MTAG0i~XNt?|VWqxn zga=*yzfEC>*GMee9xi#iP?dgtk`i-9O^NtxwK4Plk&2W%-42V(3A%SeSRDKW&?|hm zfeu^kE`DVY#$9SIef6h;6LP~lMMb&Ijl&DWw45y&F7xkMQO!>ipZ)hVwZbtD`L;7w z^55%3fSv$yq>{Nc!78-LsOcUJqja^S^sP~td~=c&cVPte`P2aX-N^|?*o`eoS;~Jl z`7Oijm?!g{`QHwwnS>)CuaBm=yhAgvpYihVYEIa>xt7f5dqz59*TBKqlyvaAl$hY0 zWWu~gHpy3W{offv;USqxW%obm(8>lJX&bWzPH$%c^A-2 z&+X&njLl&}ys^6E;YJrO+^uAmZ4DzsR(s%|lmhUBx{`=L`8l8e>_Ns}v-XpdNBH2E z#MHN^wc`Ac!%14HxgpiB5%T5)CGIXrihld2tuObrQ%cDB0eAw70+~A1nC^dQcmd!K=-o!Z(LzHq)i!Q>{{3rc1?v zITehe4rtULg7lBax+=m?OUV)ES6Z`Q9cszozvua)AM-30BjeF=l`tXb=PaG*)?ss3 z`%B*MO<^;4t11T2pblUy?mW8S6aslS&vJok8c@OoRC z-*Ri3-+W8j1D6h(>X{-IuZ=>2oKu8i?{dxX&(;6dORn_Q2H!U`P);4S$P3`ro}Pml zrRP34oDuL2r?~h-1CQvEvr23dO!!t$!BbmrS>V2gQEnZ^hHrMEzBtG5NxKSc0-z}Z z9g>pwb{U{y)#SX@aa#7~I3s^o$H6!z30ZC19s)RPlfXizs?u+2If+*c0`OB~4BqECXjcMStQY!uTY^b)$?$0o8K6@Y*hI?^^4ovwgbfp7eeWzR zb(g={k`Pmthp4GaexaX|nSNQtNWZ4#75MkyVy})+60R@zKtmTg%0Fv`Y1ifK)CVKP z=}b7;nHk#;q%2ld{k9=!cEAx=f5p> zmPZ4zPP8j2Mb^>cw>oj+?Yfe8=Xh!UrrbQ+$tW~R9F6GrLE+>M?)_^G_rC3 zS>lO)&9Eu%Nm8y;jZ5@QlJlKfE@^WV6Yo;cN{;|9>8tH(mq9)*H0M8TDGfVKhqW1#x>XN~Mc+L;l9xnUlRuKyR-xiJWYZ_nZ*1{8xmN?b>E48x2 zIR@@%7SN*8bU2{n(sn0_*#`hOITgeLV^#LEO%*wBvKbO+m6A&i3?hzo6dS$Vl5u;h zC;O3g0Q_dLA^omx7#-)HVx!jpkFz;k@yRugjtAx`+o`1$1K(rz%pBr>uU(tyNH@T7 zyGA;8*9dSU;1B;i{UI5GXv|RLo z9+z&Dl)Tv(C<(BSA--8g@sZ=ae7^}{?t}F)dg+FOkI>hbJ?qCr{Ml3#bfUf}GsVYnT>t&ZCDW|Lp-Z&v$0saVSbY{+i`+_l!dBri`Amt7SlsW`(u?ED7tp21?mQ6H^@5qEmvyA4voo>ULMxjBZ7-5SHjY)Vj}78xScuEiwmnwUtBvA)Y|ZjySZ>-&%9>iU1o2`Ych z^98`CU=J9lcNWl+lf+D$hF{uTOZv7K5r zd}|z^;80`Y0X%?tjHCjq3KeUSVpDy+R(JJWyi3bLxphE(>scsYudbsR3e+k?M9j1m zJznlb1zRRq73GGrI~Jy`!NN*rn`pucmGIz|b}UoC|7Ype%5?Kj)#TA>l?Z?ER(H z9$3i205+->6>(v?rTERk6qmfG=aP3PnE(z_AT|XedA_Ig?PO)ny?^I?Jvcc+ixS{g zpFTEn=)b0GyM6+j7x*8G1QWg5R~Bej_@4a|kW9^}Zx$Ij+XMLTy-8+;SI&mIrA&xz z5SJhqCfw>L#oThK+1QnSY_d|AcGEV2Nd)i^y+6lc94&HKM*#kLjFr5NF>0PuE8uL8 zv%p?Gwb-si#;^1g2X6MlzOMIHyz|xWvh+ctq|Bs;YXiuXl|g*Igq8VR$4!z` zO`zis)btXMf&pLcFOFU4NPD-~mU{QlL?`W=_`J1#NbqW3_T#mIjHk0r8Bex`OM?Gd zBtU_DYTTPYS?WK3u7lI+V16m0rXbz^A@j>Bq!*baaG!mdo7p)q6Km%eExgXv;7z$~s0) zUGmjgYRL3kXiB@Q=7wFGZAiPf)&+a(k_ixVUGONSDEa1EKP+TdP0af-LoE9-M}q9n zGw^#;0-o0-6zxw}iVw`<+Mo0FqrN_PKj-*@{TVvSVL&IjOoUvUfs|wDLx1lkrQF&c zrsix7kdu`3^lJ?Tfye7|UY$^qBQ9=r7QM8MkyA{4s@c(2Wk+nqb3i%4)%EA0>Wqh!}5~AZG1qfxIvwlm1>HL;jiP@sFlO zVqlE4y(tlYf4a8wU|L-5HM5B>EiThGUJ`5_FAcNHC^=3!w|sqsnxf#A-S5U^KWc}@ z-s~)oJktsdzgS-qa(;*qb8SYPb=N7OWsX;Y0~;|hmsfg8xej?HVPg;%y)_E?WHv{9nsey+^)!kp@zc#aCNWiiogC;juO) zxnQmb9SX$hV~>%`-Jj<3>?(ZvTwB&76({ksoRM;&4gUGubW_?3hlUCnW@o=>Lq%RW z1hkvez(4@whyM3jJ0p-68ZKsijGAFl{C|$#`zy_Cjr*Q6XX=?uO=glzCX+Jhy_%*b zvBlndR}evZ?;R9-2P-PlrAm{gs8j*FCYd>N&Ux1J{vq$p^8;&v3t_YORX*SQ-ut@f zAf2G~`r0JAa1Y4ei5lt$CK@y_1AUpOYib~8d^-w#^n-1v@ktLN_d?2*lb_g!t6%=@ z!FR+p^{GBd@h!8c>=Lu$?iZYj8(#&dYm)xA$?DnC(Y|sRY^tC>U3K=Fdq1kgT>rgX5n|2nb<=!ll0!eApX}y#{SPofWLQP+gB%B zpKVNGvw_%3+cApjLfJTwP;1#tF# zOIQ6xmnczduKwmSr|`nsK-G=VMBC#{F(OUDEx0g_dj93=I4JppjfVN?#DfoQ$O<4< z=friGid{fw>;gGs*Tg`5bcn{^+d0e+b~f!O@m&sl^A5m$Ap@n^GKzUSPp-S>9D*h* znANwZ+OxmcQY(}6jPm;~5x}z2mZv*vRG|& zWqV3kfy&*OgQrDQ*pk&*OrD{)^^Sj$)Us#ci}uZg=Iy1{l&HEXIjV%@0^>y;TbTF* z3zxQU<+66o4D7L;D>$~Zxrx5@`zC<#=kUdw5=`dCBa6RP;-9Ma7wPvaSDGM!RKDIk6`j$0Q9U)cO-8}A*mbE)+gJ#_IDq> zIQZY2tnLpUq4>}x7>rxUiF?9?_ij4pgNHwMOrD+yAD^ruOZh4CgI5|1698}`pbS>n*hBEDaeiUtjdd8VfBq2J;1sqCN2i( z>bMb89yegiBTLv)U0>^CRae8~JttRu=;3xp_4taA3R|?bL~MKSX0U+#j*jYR;D7>? z=N^Z=*jc6l+LG89(V&V}C)#uEy>(Al$3UqYv*7%Y44k$pN9E74D*l)x?+5UrDZw=;#p3VEZet6j%dWwK`%;R1{I8AahBtqDSUjH;o$HXGGa`Bx z<)gD-^k{)%8=cbS2D?_km*3QWW zHBfoUH{J2lHP-edFw>TNWTL^MGHgLqPO8|}(OP2~cve&eO%E-Cl69Rqzc2CgPX%Th zlRi4h#IS+Tx}||-ZmGd3aV;eKa2ebBA1ATxKP!Z`KNCHftu*+SnpB_KL4%;L-3!{7(2W9>0zS0jnhtEJx?Kw#uuEiq%uE5rmT9nkhp_)} zg(Exyc=^cLyRdKX8r-)FdpCjgcZpl?*@qf#*#?{MdnBl`K3MV@IkVu8*gUN&B!RzL z=H{GSqNn}n=qbJKm4IGO;IdCDIAwPa0L>RxB8!}(4Y!=)hFhD7{uV}36QDEb`&Qw| zhYgWt$H5?mm*|j)TEyNp4a+~cx!v1lG6)!J>ncV@)@DF2%{^5Y-4n3nu$tI-e3}oEGK!uk`4Pn4{ z4jugNZEI)WmZ?+V*HCJ=4FdL|MbQ1h#_Nph=c#6D1QDWJrZ_~ab8M@nr5_>%D6!4qz2F{N8tctKDD zO5aqpDP5c6BVoKER;6W`S^9$LNSG(K@!&geG8K+P7qwciW4E;+PU&8d>Pbu=N#Ruyj^l zbPW^Idt0-REXU^CL=BgRL9f3M zRX_Pu#V9>ZtW5eEpLhE!M|WLvT!k)*sS!CbH8MXq4@z~9mfbT6QhwHuo}XA9sJ*u_ zk1Ny)OMY7xlwb6Yv_1!PZlbPv#orzr_`MBPv$+c=H#!DL` zEw_Smu#8c7%D3atHz!wz(V3BX7I;`0J9e=9)_ad#>wlf3hMWR&E(O`C%Qt zCZa?Z#&tC0p@m8Lu)^jZIdGUgGYYzG#5Be9=(4zh)OO?+&F-!6d*T)jb#)e+?VGKC z64!uVZ|e~G%R{jAepuc`Ll-2~Kh3Ci^|jrXBGP^uYJUEc9Q*d1nU!%SFxixJU?w3n zgH=h)%E#Y_rfIDQ#;(q769ab$vgQqMx@kpOz$2ui zwzwvcshd)4)h=N7?gc`#cM4v-ZR9a_%pCfTg@)W);daKCf!tod?yE^%pY14>Am9EHQbI}&ZDhou z70}%)RLX&qMA|hX8$)x6Jc=zmbO`$Qtb);?nv6=^$Bdfj@Sv7hk$lu74s2W!jlEKqghaittwCbx1#Ig&(TwO<%q<)Lfd-k2Rvn6Y59+B-H zDf`_cuDEFE%sxEO&zW(VFt?0a*S#_OR`RcT1 zu;%{CU~`&R0%&46DmS7e)rRHp!mtXGw;^kK3g}z-uAbkyvqVAc8gLDMdF2bAv@|KO z*pjks0=Usd!*5#&;GhQf4$uI+KV5?1k1lM>p$%FT*Fv-Qf%v|&M63zUqcXkY9ZCL4 zVm{!@Wg+18fgW4FIR$ztLBBaYP0GEb1nha1QvKRLMXg_*pcFeN;V%ws0F&GC9Y;1y z>#>vAaR7XF(15O#ca`6sq37KC;1x>3S~el1WI^Lv7CELs)NV;3X`v}lZd8G&kkU&o zjA65Hu1>d?`sUEZt}%47M+{H#%_1@baze?P1fHnJRuIx)s}!Qpr%}DIYkOJH zmDt+|#L2pdj#Lu|{B6(0;m0j}{_+?y-7wJcB4Xe%y)sJ80=f40?uN7<`Wjw*vp!mL zWm^f(+|zed4x`_leslB8KNmVHG7eXmg|P%D8GtxwptU<@2{{`Jw5F&52i;IWN)A?7-AB$r z(_gEC(M0d6h?&&7>!KiIi8^6RxKhhV>oa8j!+-7Q`IN{ar7BUsJum@%wl>gmeWknh z_vLQz^)X7#>83Z=KAXlB+zHIVGqk*-Up0*UpFG2j*VZQ*ZpY*;ua1ng_8lV^Z=8U> zYs)$RrKh*KaK|)Zi{r&jvf3ih<5x~*AM zMqm=3gdhag42X@M{nx`OK0h?eku5cSJL>vJ68KBD1~POjGXMB zZ-3*SZoaSK)m&NWrIzYBh`0Vp@T(&W7PPG;wQk9XEgzQ2h!0jeeoNhv9b9Zp+Sg+$ zchtz@f7>YN|9ScSAFM!p*W;?!Cz_sx=G*hu7f^ZK@Hf9oDdksJ$Iz*s!RDJLUg2pa zIqM`~7C-Roo_#+J%ldt#yZOEtTX>=D!MRUXhZ>Xj)Cmm(OApLJ-?&Fw?{Ci3-VQ5j z9tIZbAI4PVI=7@PUj@X*%~@Fbkqr+&c3@#g4t(ne7qR`livW%3VL1_1+nbLLChNe= z;~m>OhZFv?Zze&=3%G%R1pn%4&@J_&!*B6j!$4)B#*g?b`JMis?c5LIG1J@jLP|@MUzVH<)_2Vi- z;F*G^cqTB}J31c!FDISupK5xhB)t7)nOAt(Ck8#<)8H%j^uz`gx8|~a0QDlOBvnUM z$RgKR<3n3t-5;xyjkf~|@XN54P`9O{VB%&DZGDkgA2qV+@nsG@q6K`!NQE4F=)h|V zG@vAwC+Z#n`lWGI3i$p6E^}p|{fQKn_5CpT#mSzQ$G`IH@1O5&P5rhPS(f6IVme|* zLTf;QE)Or!AW;K@*p;3+TSEwk_W~Q*kq{Cv>9cD z7BObv5Q7>4UMfJQiW;)duJ)lD4^)(9??C+@>toQz&OXo$J*E67C%58Ecnbb@bq1Hq zXnOrqFF5VowvGx;)Rl;8VCf!l&8=BV_Ae5A){oB7`kQ_!EOmm_kW%{c+6fskpzA95Wqqpd{*Ip7HrVmzhvm-d?7tksu@8V|KQ!R0_jH(& zxDr+1pN76#83f(dcC}vbM--i%#uWY>n`gAGj{)=2-|^Trgna26#bqvW>i-ZmC7%#f zJoq#+(~-5@RejSUEc-n!hh^?8;Tm={nBpTNz9!NC4jA8COMmrsb9eR4m?CivLpT&t zqKZ7zO%Hby*SoC9k|PJY@qa!FCDC6yu3^Ft^#c>z8XpgzC{w}7 z_$?#{z5CM*_@Ir3+Og5`dp0`p-)p>~Z4*(d7x)i!a^7&}T!a-0Qfad*8_!#TO1Nc<7-G*K%mVR0Gc&;y@gV zXtBj16}Bv_XJa>%l=^Kw3$t%x;XkabZlxgTv5*b(6S>yy>DjiRFS=Ki+llf=?1 zVo3VGda7T1-d*+ltG@c=AKe3uFZZ<+$odStbct7fP0c9zCAdhc1oBvBQ~@vEQX|T@ zH27K|4iJvq6ykxCjM-UXPyz-DY-yf7giH$Rwp1YHl@(CJw3kq(2T7; zFd{3qnD)+Xv7?=4Xw`^%joLmxDCZ5o;P5wMs4FF@n`-vEvMUmqU! zaRphmtAl0Q2OIB@N?v@2%YXdU<~#xNewl$g(o#XY3QYOt1UOB>D7vZUW&s*3^QRpd zG$*VeHrYBGuSKR2S#deBJf=kD#nq_dxSG(uqb0OxSlK_Vi(B5jH?t^!mS~ErV7XB} zx_sM61pVn04DA_d&;u)ju(3dCT@tiB+tHC44=m)4%_-PxGr!_@4LkSb5ajhK2{Qe+ z^+|9FptVX+<*$Bxd;RPellbzd9|61E(m>us)Q~q}B{*eAj{)YE0s*uGVnc??4ylNh zhi0K*SI59^&bDXlEMaRS^Uzc?FaMkb^7LC{SIx~XME>Q%XIH)+f#;o_hG%~xX-)oI zN67d_M+USo^u@pWL8+%i%_(P>*kw1A-0J(OS5JP@^5)t%n{qtVI|qNWvC#T**MbN8 z)u#ybL*Z;Ef3eNh)E%$8W4fnX}Snpkl^M-WXk%di5+*dy^b4dGoa?_5IP!?SzSJ=8+ZrFrv zca)5#yF;Li69dg}zvorH`bGkK_oInZc5!RAJvpw#RQTth`I`%{>@6*=HE!UM_sxW+ zupFGS#LNFh!N~k^Ww7RIWDb$FZ)9Mj0H4HkH1NKWN8dBCuyFKp-h zWD1)4!OX-R7+Hj%j99g2U=jBWd|F_hROACR_i}gR-6ekG4Qo&9%~dh{`NkZr+90UE zq+*nuTci}CT2kKa3}sE=+k^4NhWJ+?!0kF2mffD>ztJx%vX zaKklsPu1@#>boE4rMJIAXI}Vp2LAM$Z8a=!hFkN1UX${x zYZP9$r6$!|`b#eL+IxneQI`XO%5T^vnKW zVUqXsn9`6GoNO1B-O@9&&c^}WK1C~iG>$2}8P}4a5hc1bEQP)CPlD2P!t#ewZ00X| zPT`s5-iGHfH5H}g*ChH3zuTQ|U zfZtW;m*dOrBW-&Kb;{J|4c1?^3^0X=d_grx+TZt z|6ru#{@@h>+Ighr_6R2B2X@`Vf6ijFf8LyGf3+rVxTEKm{9zK7-tWv%VD@SkJt=_iGZ&{_!KeO`M zo*0-d*VN>aQwDa?d246URV_Q^yo{W5a=EMMa$G@c4lmNc{uyX$qCTci-12-~Tzxk@ zTYYDJtoUw3+VFDQfXLm|Q(6uTTyjK?D~T_{vySzcx_vDqFQ$Q(Y$=d=$7XU{RE;h5 z0GbwH&GzV$kl~*rmAHml?`l{%KgfvLCjfT(b!!p!24KvbMOxKaQ)k<~pp;&<($jE# z2Kn}Te)at?2AZCpaEcoK@D3thZBDXVJ(JWL2`=}Pa}b{!n&mVHrx0(WGI&}M#V;zR1#0?bCo|OVSuoIzxrUoA@V{7-9arKVj=6gC}-PQG3WXXqB!9+xj zuh^JsPxgu7Z@uHl3@y9%`eaAeDGR^iModMh>BJQN)|`I##8MZeET|yWXn9#bsTir} zHQe<3u91Rghb9K*aGB2wE|Ea!%E!M&vp%TKyQh}@7;Y`V9q9woHPoMCjmYOXTD!SHT}6J zoC&B1q@}*nugMM|VH+)d;lUXF0C$ z7b~aoqMcXqySKmc{>Er?dU&eiU0|f+xkXTQ$=lm(S9iP%Pqw}E4AftXEdc8<2Fr*{fL`t>+g^ud?a5I^N7|MGmA$K@weOl3 zaKD^T8l0zA2Bz5!mhP%cGl-X8PQzY)y-0Zbvz}di+9s;G0K~J`mVVfa;g*aOE1k`c ztetg_rlBuSSXgE2HL+!s&YwSNufnp#(y?>l0R2D-MdCQbYqr~YZ8@TSOfIuo*Cb|@1PMJVrWuG z$plB#J-o20i=t-NUm+Gf`WjPk|NBvB`q}0Frkjpo$h(k=L)lcZA-*|e-u^O^cf7(C z?pfI6zyh{R%dfn#B&xk(9cp>Ft!HvRxVt6}Z7gzJPihD&$n^mkxqd^6%L^`0s@CO< z#zlVXV}l4)v1{rX^G-vPXOY?8jyAvkd=8WMv!kc_=C&G=0r+#>+8m{%4 zj*JB4M;DzEQNy$LjHvo!8xeVErNW~+1~QS^x2O%8ph@9Z@|pcS8z! z7MKM+3C^@W1maPeZ4C7sp7-(#Wd4Ki-IJiIEe)b%k)Cow%gp%MBC7erI?(*sIa2?4 z$AG9fv@@w&S}s<_FFYHNqiWx~xT2^5Srt~cri4_8?4X)Z88_e>|61pdN6Z|ym|AhK z{N-=|+;eie->(TIM^1X8=Xb@XwB?a|qWZdZyydZLj@dXvYCPwiz~}B6NXDoOFlmw~Va3b4JeF zA4Xbl{d2hS!Dlj5`ZtT5l8f)|p8TZp@!8L|?)kyOZw0<{QB|SbA{2F*_iwd39tWfc8yX%8rgu1H|2`Z9Tjo zC~JGQDuE=06KD4301jJ5HV0(wZvou{jmrCkaV@UKKhu)DKGF1YbG|(}wuCL+UShyp zHr=A<%G4!{6rn64=#>y-$KXw=E)_| zC2Ct#iz?r@l93<1Ou?a@PK+#}3u8t?T@-k)e-T&dm*FY`3xpEiB>IhS8j`f5B{mX7|yJwvIZe&-^6D!`YcP_*%Ov;{} z!};hENk2M;Lm%yI`jL%E+_rKVJ}D_bkmz?PqbBO4Uvu_XUUE#b>WtlxTY6s8B{ie+ z=Vd|JY42F;-L+Y4hHVUywA|Tt%`?oaaF6shx`)YW>r>ctM}PH2o2c~c`dIsueIvbN z+em2e%|nvcW?&h|E4d&)qQ=Lc4nf|0t>M&O zvkxNE9iy;kiTgavn%6(N`kSAx4AkEO_~^SaSlagrQuav$r|PC%1bXfr#JtgVL!M%a zlfI@DzBmzEL^ZET>TdX_n(nyN`uRl1y{{tkZE34>$ov^X&PA)J`q{pL0NvA*5fK@&d`E>X-W;#K zuAn6UxYA#9KO`eni7}OrToY}@9&uC3N_Ruj`Uvd#HZX{dsg^e?R?gYprYE2I#N<44 z4?5q{3(5>lLsFFVjB|3*%O6)q3$7h5gWqjyA@95+iwC9Fv>1%u%9GcsD) ziz7R`b8`mrCaMFt&c-5bFOypQi*3nU22%T;mCIS3fTsdJ(gZLxEI5lP+!$}XYwpav z5SVR#9+IQ7y)r^Ez$7Ih1)*$p8vbhEMnQb^0(QTMDRYY<$-xCkT0{;of*h5*Ya+J% zxy~Ipa8Pl8%~eO2i1qsxVn;}W1@GHf%wq?ed1z%|_pDU3UxlySvoe_nb~gK?lPmmd zg+V!9CNynp3FV=AY<3v%C)Y3{$;fZGu(O1%&N>&qtUb_|I~oSk zoY;pp&~nE;-27+?`|kWarTm(1kdQ3lAa8aQzWr*fz2*=R-2y)4OEDtlUz3>3 zvwCj%_24|d(m2wd3eUX$*)ZtM)ju8Vu45~k6`W7>4`_Sp>dpT}L(BNZH;F8BP7_O- z((j!x^*0qCE|XgKf!OKlt-ovJ6kZr>e|ln|>F#GTbjmlY)A#~P-P>R5Qm=kLhtIni zlc5R%li;MCMM!FFq3-oO`Q?S~`Ws(rC|SQn0FJ_yz5H$pmz#8CA`$-d&}o0J5Zd?k z$daf6{7%V!dwvF!_N#lM?e!QT_l^Yd_LPj0bxBDszP&Nlk-nqGR0L+=X{$hh(o##V zZH~9+ZztjrGHE!(r!7&K?Uy7 z<|i>-V*ehCx@)DO01bxNHnV794F#s=mR;VIBHr0Y;I+EWhSbfemghQF+L?{yb6O@9D8A|QWl7JqXXEpj~aPOQ$^;aBy z<-a=n%g=2}5GmeCa*nAF^VBnpeic=7k*;yj6BEDSl(nnqw1J<0I=Il08P`$TwhbJ{ zwv{KeO(Wl0#K@GWfd|Y>&-k8QBtCR-_=grMDx&Mi*#ekjQw=Xraw{)}=BVYXQ~11< zp^oQ9cJ1%Jq4q}`Vr-^&wEek}Uvy@Dic%R?b+Z6ItK3qwCp!kqF0AwypIe`7ydTpN zsv}EukY^nFGzdWRM=OT`aC~h*j?MFmJD#r$g72+QBU2M+t9Fen?4Cuy+gK!5h1DEF zT*s${<ajKU{h&2(h_m{>RG1UlU(qg447T-d6jo2(wjr1V0w ze^fK7?Q__njPF+Zo9^4EDdla2$=^LX_v0s``qyW+=3p82&Z0|JPR1{C#KW(s z1%Ldb5A^zkiq(2`d4QhY^!DMW%*w}CwlygHUmiN$KUs5UMO1WJ!_54?v+l;HMq%ZB z{SYVxoO9;{wc!5eA1xHrdlLzAs3$cA=G$J6BVK;TF24DNpz!8DT-_CS=9uLVTV7rK za)F%xbk{_K##QVNp9K0=$ICu53VQJwt?D@Em^%3qz%y?gF6j|5b{EpL70msDL1OxE05A1}K) z*mCvYa+v9dPLclsyXR@ z8x-1m4~>hjeRdn1ckfDAMZmvz^{I|Mz3Tslgo^(KSVPBFSk3x$>D4VArggC6-OXuy z(d~^HXx929D)NJqirco5+x>EQ z&T>!9^?(#o6xZWwtfQa@5fvL6RkIQMdO}O0?>xXaEjtDpBm!tu&jKPNZe=q-#5js3o8e@5`8l^XNh^9VeA_KlYSX$ z%lb}&%sI0%(*6Q?-xDCNJp%l##ydkQcTW=D`Q)TBzm@^=EV3GXa(eB~GMjX4=XLIy ziQu3HnG0x!Houb9646mxb}e+IX91tTrleGDs%SO4mPEZDOyMTL8bKws5zqs5n^J1o z<^s9YJBiISj-sB=^I9$|IQ7>z2uMOZrD@z7%|T^!6-71atjQj z+>u^4nhX9#O1-Qb`9cWF{#nJZd=*|om)nP$ui5)s@3=?X zpRI~~LoH^gt!Sh$A>TbCK|MyA3FVahd*nhyAENNL?$$TI z139YV0I<8YY4GES3R4nPVavR89a%d7$80-z9MdE;7gF%xo9g5nXJZ-y_P-nK-oHHr z?B6y-HIQ@PIC_gO+IlN*n8q-LB~NdD(U5xkbkt1j{J6m#J9Ln6!Nuk`j-KqZDs=KU zdlnihvdp8o=V{gEAyiU8MQ=*P=HMc!6j=Mm2|XpFl>e?~*W3uq5h`r6l*&)b(@)3jc z7|5mdw*#|{4O9Ss2QRRPmpjC(V<}AOm^=0BWY-k30O=3!^V-k^a;9$_=W-{Wy zL59C~5+VCmY{QloRk*uMXax8fzd8YX9+*RBY^jlX9!bk1hXne}uOL?J0Qz`a&qO3@ zaEB5#R}FloXMxkYA!mU$6^s_2j9MR5Q^C6?3grC?o3(A>(tHXID6%Z-iCK8;#Qw>! zkw;xsQECH9Y++!&BQ2&u7svEmqOl8nZ&?U=99HsB8!~#0S4_xUmy!zIv-o$*Vp7J+ zG_5>j5Heg+a;d%-`P3)tq4*XDn0`4MU?FOqX9|iICD!j+S(JdDjkM0PtGtV}s)(Kj-qO>ddRFzhRUj_~7BLkeIi@(OBi02}_;Q~N zSLm1|zVoS=ppZep-*$A3C+h0?Rix(QL~Y+y{!nC}8aSWr993${&>+_^C z+Z4TMd5l`(ouh-|%fSBvn%p;sFAB-&O+b!miJChFJ~gLfO(P^a<$Yv>q`%cNEv8u~ zd%>%7oF+?GNAjiwlNVKz8Y3D;`|=c}pc_+hhSQk!oplgj8dOx(bLZOHKjE2D^{^;vq|x{L!hh(K43o%J{D{dIQ%*1qeVY<%pO!LqmX zlxE9Vdzxh!{C0Plj)`c9^^Vb|2lkP?D+kMU&kyY_SvyuDa?8>^yrv!ynZ(R`=QO4A z$lbGW?CG31aL`FH11vwLZ%dEsk)?YULSx)OtXmlY-|;V?au2LT@U9V45!2zT_Ws|Q z7)(t_fh%{8!IC0cYTLF`IJBwf()}6^I&KkmAJ~Nhe**D4rokj?l@$0jl!kRVuRUVw z9$KDg&(jUny$mjqz(E5Q9x)KX0Tr$`tfhd08a_cQY=3C#Mx;e`T>_tsQRkhZ7OpSQ zs=N#2s?|jnbX6r{t!ev&?gdV>T0~BDE>4U(WaFZBB^wN^SCxN}P#MusAv+ch{a~eY zc-O%d#u8_oydvq2lh++KQo%7RixgU>qdZzVc*Dp*ZCP2=pq>er(W}qdB*>zmo=@6T z5X-j!ZV0N0^_vPp^`?Rbj#@>d5nJzc*di3IYw4|Vt7v3*h2Iym5}-jnvC%S4cx{ys z(p-t2t43VYpWd#S58htYdr#Nop-Uj%v+_i{iC!RP0nahPt#(dxYkUg=SXAHL|K8Ot z^)DiG)~2B;j!{U8ZJ_<}>J&V6+r;ODwF3Iia@SzoJfice`aAt{9we;dquo+=owBpz zwy_WX$SJ|T^(&Zdn~EM*NIf(VGjYjL6Er)lCD$hUrD+GiFV(cNOCKx(LDbkY8Zrz_ z1@wcH4p}eWBJF~hN155K8GfU&ho0#jp_Rq7e7t*(S~e$azpm~>JYAWhRIMunXyXt% z$v%#I>zx7SM@ecvvUB-cMh+=#U}M~KxO~?vCe06s_=us47t(jJ0!u<_Xo&@n>gd?r z6$UG0BDR_*a2c{r$bC5nlH`&KAeK>j_QnFede=gOZyDL7%_Vk6*u;SU=@U%syLh54 z69X1EclE~2gR=LD^TcLURY(C&TbH&Zd8b+*xB+dospMfz5=zN5x$4xa1mFpP6FlSX zNgGmFvV9EnK;Kh;70?)u;yQA39Ei`pd1|d^nq22!M1vD&&)>T!$b%IQe^bXn`;>gZ z&%0Sc{h)YVJ22!?3Q+4B0WqNAkoT>0GT;X_tF!H|*JbE;VKo&L(=(7E6|U4b+m-}q zzqe6M^Q*P#qRZ=Zt1M zxo_{C512aphlu4@1mN_ad@>?v%iJjpE%O=vWfs;cBbRJy+3-z(!NLYUJ8thD-%tX& zYMxOY1pXc|@#t%69%e<}g?21-!Mrj7)G;L%IOWsR%X4C$V?hK90Cw)1$K-FyFhy}A z9TBzg*%2cLzh&l9_q=`Dn481jvhh2k?ykw$s%T+*RXDlhWb;BsDtt{vu6E243mvok zHhDKGlMBi{1!%jPxShw_vM~uhC8cr8L_-HnER1i7(hxAwJGLzRLEA*z%ay5?`(Y!% z-YRm#mVt@gb@F;XY;-9S^#Z~MDm0*FBI5P|t!IJJ?w%!70pnpr%skGHo!hy$(ls8l zb&ag+86B%CeDS89+#FhF6V}y)Qoo|@#X;iyxqU!&=or-P+xzBx3U-@gf|BE!6|`u_ z_$2~Ro4Mp|Anzxx39m1* znp{)DdaIe+DFvH~PONuv zfW4~jVR_}8HuDJljc1zHvbr$9wJ*#`G-G14uBS6^W4aH!B^zJ_=0^oSMIXaGkIdgt z)IHy`5kR{pI&vR~?=kD}Qdq}q_sx4{B#H67%br4+8pS>R2j zi0YIO-&sYCm!e8)b9jk`iWu0~h@JuUYv`@cSz^8oVDp$xC`{DH-?y`AaVrJBxrD0- z>KTZLUex82brRIQ?99bMQJ!h02ePi|?+)v_yMszL)G-2kY3QuG9#mi}V@4LyH%F*Z z^BOMMhua?RJ32?=E1g4I77it_#6|g4jAp-*P`W1TcpFO89J2B|mxf^}ngQ6$RYf0X zd3u~?o8vcm6SfX~wo@+X@X7}TYx1!{Wfv=#SzGywbB>SSa&}K`0Wp8a+oRa=c1=gD zY#Ly*m97PP?bb4%w(BIr<91yAnzoz0q8#n_EsaTohT(bF;%L8oWUy|sYcOk4Fz|M1 zxUW{l#ot{SWmH7;1LJXP-}JU!G`8*J_xVjU*p{7vjXS$%wp{~;9bcb5=A=`7@{XLf zMO2C3#3rsNSk3w|TBdH4^42khdlvzGIACI-HgrswOU6URY{G$EcfTg)>RAk0L}I^* z&-W_uX;c-zGJ6ic2>G)qK*NDYYtzqwn%P`+k0l> z4ng0xokjI2sP(p4Z04$t)EswHi3jV<-jIb(2%Gt=ZFi@5$0q37ULmyYIB3|Qp|{&I zISkj2NwLa7IbyO?oHIq{-m?#k!B(flH1h~2TiHo{VCdtgyT=Dw+|#o{d82tkH2o^3qV%8Bl-7q)-Jp0&H#`;6jn3Po-7u|q z47NBpR;w8wX|>KxbuJ8yx6F2p6sm_8QRW%_jB#2yE+0`cC%DpDF>4}+&`P{Qtb$+f ztENAk!Hi|dg)?>fzM*>G3?Kf%!Djv2%@HPgg~S~qu~E`fr|FecFZE3{8V0+HR^^Nu z_dGgdLqV*G==tm*;MZ1ZcZZ^j{zAbZ+_DbSOXUJcx@(dOUXyjAR~0<8e@VdFbBHEB zdVA!6k0G~~5M>80ZeL=JHVwU9=1E4yf)IU2Ou$~yjr2Eplzps-QPdMqGr<-yz0f#7 z$XcDGRrmqcRSRKv>?6c%#~3=_J54By8U<|s5}&Z9<}FzV7be-ZqX6 zcC0CTseU~dy>8@_*8zsz(9v2V7CJHlXo-N4(Xn9wSWU;rF3&HFEzPRLrg_)?Lwt>@Fu;6@MR_%|tTIAEI?yg49ya3|Y@-dqxCeph^ZrFD5`xCXV z_FUcbQ3sO|G;;_W*4{pcuA8D?7{n}!`)ifsL-iW*V6#&($XHR1h;-s%sD5k$qaB&z zYsLLoik1g+b`c&~1k!05?=s#NJ4IcPttiMC-k$?O7 zlgF!q@f{bN9kFxhYlglar((EM)kDv6E_M+eiY}^YhF0oX60)|PO!meSrpmp5FApqr z_r|P4%E)q8w_C?Ut}OMl4U3Bs*~l_S(rttHa|}fy>iqL=^vJDo!c^K6T~;(ipG(Cf z1}=lE$rmV%+(-SSsoZ|rP`4fMPe^1iv1Ela z&~JnAs51}6ROwrVP|>Us%{1$}2g~#$vruW5tZIy*e#a&%UsRW1&edka&qAxYCp)TG z-%e0_GQ;}O>DY4LxK|;f8YJvuwRjjjJG{^#9?_x4hK*<;OP0wcN*{I7HShX`D{VrS zI+u)_eppvRI$d8v`sH;x;>(-2O8#;GeEO$(Pa3|Zf<}|3IJ&x}{z+&+fiK-zMpncu z*joD{qkgt$G=G>e|E8ZZoixZCdnO%`HcI;j^No^zs8=o-ur1E?%g4rACOADWBm~hN zVLS3VI6dWG6G+%wk9e9BPz`i#E%!}R?rm?gehYPoF$mh7J7f%P-9{H56u#L~q2uB3c&_2R29DoSuaP}=ENhUolMwWJ@kq8Om9YDBc)5|{2<#MihL)Ec8)BAA<7 zppA}BRSxt_`XGP;g{p|E$*xo-C0=Q&nh1Yq!CK%syv$@-zetFb6s0vyJCq0IKp+!5{ zKQ}RrmGnz0huOWaW``!5XUC-&v9LdLN+^1(81Kwgj4`rTMooG7Z4e0}XH?2}(F z6@2>iN&Q#lxyWzWD9#^qy*;^hX+PSjoS0aWspqF=%>&(HGo0S-E2awfN=TfoOeEfN z1JyqNJ03cJnb5It5#2U^rnzzC_o|BCU#iQ8e<&&+`0DEY>Q63SD)=PrE%`GDRCT^z zWNWrBE{KEr{sHF#zs)w+2{X(~c#|_rJjRgc|L5qh-x6Kd_i@}=d#_nJYdY7YqZ7tB zwqtj9fl3QVcZ%KJfgl1Rf;8BTi3zsOtlInad4KEGUjklIcL>&(X5iNrd=XmAZz6~JWcyoz$m zeke$z{#=wvf1DIU_|g4cOKYBu(9zBh7dh#O0 z$v={JH>3e_M$swg2#U@#wdR0 zGlcHK=D{3kYg2+{NRT=?#7)o)^^iKcrpT2oEBTzdnWz%F$~n8B>2VQS@)V9%K8%WF z{>$xS&W$IJgPq>Fq&j`_#W~%5k>_;trn8f?bNmemQuHJuzU&v5k43i<5*g12TKbAM zEzPaR^9{YbrYioNtPZ7@$~(uT>K@6kxvO_zwxWGtr@FLuA&JCLxR5ClS0+dQnMBvP zLm-{cg8hqrNlIefj*CIuPm0cP9&D&6*i_ZiS9RsY(UC!RYp)V2XkJXGSFeO3*)y)G zFyXDpWX^ZdN!)+=g_k(J^vrSk>AsKCJJ$@Sz(}5xs}I)ciF2w`d@SjvfF<-0wo1ST zaT`Y0SDmU74KW9WYF2IMWFfnLE0IvP>5bvexE7N14`VVL|K%5ga|(}Pf0vL_`>&vA zp3_S&sMB5GT94mmJGuK1oj&-|Z+!40{kx#3?m>5pB1y zaQ8reE~kAe2~3~+JqK&Loq<%}ip=cz-Yt~tbng||>CJnnQ*e~vhIA4%m>5Uh` zDIl))hHp&$cOQdFZ~8@+|B45CwWqElVM){0xI9r?W*F-#uPtr)BoK`Fl~x%-a14t} za(@4hzLA3OBU4*{j!$d({+(N)lZSWlcM)-de}|>i-MIZC+3BZSeooKd<~zL$pg7%s z0lMM$g?^h#lSRn-%x%j`sb*!oqh(szR5~K=$5qzr5(FJv5MGZRfhpVYf>X>dP$bJs zC`x%RBfH~fNOa}RxRloKeZNp{JbV`G^wA6ANiou+~4 zyR8;rrCs%Do$ZqB>XxZYYTZUSaLvzg;K7Fhr0+HmJX3^vD2hf%Z6I| zme{k%`$@v2mz&e_fS8ScrYhT89k6h9Ja`B;k7nLFW zDG6bCo`V>C*HI_TSk+XOu8oo7tqNX_Uf2TdsvAmZY_fo=nip~?e2W)~I{yYjn)y8+ zJ#i;Kwe?@`KjWP|0vO+Y4q@N8^C;How|gNsB9p6s2#KQn3d*3qHp;qbE9!ctWS|?? zGcby8>7OB0waw>q>Q~|jf@!ZpyyQt#TJ_Jt@$8$vF^qpda81AQ~Zy_J5K znp^!F1K0UwpmnrZH!{FlRn-w@#M~r{h?A@nm!}U84Pjb(r?7%XeJrhX_&tsyeO8E@ zco~>f18`CCcP<{OPN`Y7KO}<&ehy7-zUdjt{4q3Ea1RC_e8ZymxQSXS{ilaYx%w|{RoZlUhZMJ|spnZ$C5e;bk}#dL3E$n) z18(b36gT(ikY)9{Xe3emA|k!pdO$%et(zcJ!^wb^c z^a$W6&*(a*$b8)oxy4EsPPrycA)07dQ@1rv50fMHgV@g|X+@q$B&XDMn+nSN7NL~x zy<7x;*`pAndH_brZztw7{rK>8w$sx$c~0*>VVz#M7CAk7p6cWqM7)uZTK|1kPV+C- zY_Y$#w>57?E+|-6^Put;n%~%9FLt1J3|&?^;#bzPoXu*rW#TH9LkmdqkJ+$}XHZPv zlVXbWRa`;mlXn5QAAQ5{H)1ksZbs#{Jcuu9biuGC-h#S`*uH)p#Vl{bu4>AWd(%~d z!-a-Ev$B;XmT2l)hh}-@9g`qlg)ET69dP584}Bsq2VL`!{jSLcjc*ci%bZj4%3tFJ zsz7v^I)qp@7EoO;3Y8A`6ltX0IFq!4u&QaPT$FJ$Rus&P*%3yfSt7_5w$#P-bPncr z_Kg-+)QbZcWx@}5fzmq-*83tjsp-B;Aor(dA7D2=htqCG=Ct081ohs_p_*QTxR&=> zWbylwGG%6$*ivd37jYM)tdvDD+tL-tvIQ5oreA#Ls zjAnUVK%97#S~&3ITTj%DU+%>@J$Hcs*o^otARhZ1O>OpVZs>_H3^qXzG;F+eyn(Kh zjkJ$RN1H`rDXq6hUewaA$*XBrCkbkl!8DfS1D-1W2*VA&ip{S4EhL8aFe$U<6%^g= zUMUz!RCi2pr$!Wm>$;vkz<*OLy%-;L6UcdTpxD~Tf? zd}aw(?+Ze8yi7Q^4i;!xs_GY02-(F1D?|CLZWuP zr!jl{YU?^8n>#x~TD$sUhB_orolwfMj`ec)_4VwNDFJ;;OUhVLmEvYpy`6f+NZ+_( zxK1MNC$zOShEa9y?Z*Pxp6wcAh z+N~g3xiNsi*Lf7PG#>(!%kDgSopR&%He?j< z{8mE>nmOc;!q+%cxILZ~9V&3wkT`pIxVJzp@5aumTB%1f<&3M<($YURDw-}%zNEh^SJd8?Fxc4=*W2Bd*wZhE_6TL@?tTfRqeGt4HKb$pjw(6B z(r(B|E7@m3UYfJ2t|hHa)KWLpRRnz--e+T+3*Rx-2(~SCm5bV@QoXDZHPX$A>aK!6 z@2I1^YwZ(f@*Cw*CCot=RH-sEpU@JGWp;-#8&r|Z24#48n>?breK@Rja0E22n;2W2 z>g?Jxb1~Z!*w|e?4RtivAUs-Z*KJrjhqcOqHnF6i+&4G`ZRwI`HT6wlD_V@H?0Q2C z7Qoev!scgDS*5ON(5laPUQKYDxQC({Z)Z-5=m|3|IlfBnmp2>ZjGW0KR?h554Q^WA zN;XRxU^-!Ip>l#KHdb`03L+$7-NfXYl zZLU3@Y-m2vm!g+N%#<}TyXZ(?S+Q>@BQA`V<{3oQxnu48IC%>@QYDi$35SN!y&YYN z;?}xo^-x`|Zln#TmvrD4RlPMEhDPqDuAa4HsAV5b)l{C%R#!Rn{&hl4jz&@&1O42}u{mA`@gz zNf^=Av6jkx?ZEhvp{DfINX|T*Km~7(mV_)x3E|6Q0{FVISFxt=mCh;ZiDpT8woW9- zPz^NXi8_0;dm4t~>&wM{42IB^TdMSBu!lY{*-dXsD>_1|8%4>@ZN0JmJsqiYN@3%^ zad3P`Q&+h$$}TuFH#A<(veXW! z`^F^{~R^?X1A@0YS7$ zQHQY3)|T34d059%b@REkw&!xCOmH?u1zpV2k!Q2al0zfCU|GgUUo^J1m{mP>i|V!t z`*eNNzNxW($56-HHCB>$jC|z!1Ur9C&B}@R< zx-~(`ULPqAUJ)Wc9h*3$uPe=pgX_=Q4IJ32ieEGjFwyyqsrGDtZ$SB;=Qd3VB$^;i{ZI;uO zcB5myrp>A;!)XR9^E88{nbQCtTE@Bs6S7vUT+{{`kxQ%kCCbtv;V4ro77|s`I*@6c zm28`@sXv;jtg(vWfs^$qPbb^+!dJww&q_h|^T|d^^tz;uvMJ*gtPJ75SXy$OXWEM1 zZAi-ttfTdK^H6Et5U<$tK-tlJHrdjDIVE82%8|kALWtYC7~*MD;3N0OX~`>6N{WP& z@picjnfT`<8+)R|C)y{dX@?Wkj1x1fc*nqouByx6+lHp=+{cll*)k^3`?*`nlJLC= zV*HVbS#V@v7ag0qNb4v)*`_W-Urr0CUuSs4%V|Dge}a~~G(-%W?;{2-h-i`KZgSAF zl$E_Iqou7%DKYyRR^jywY1=41Vtu3}aurzn<-ro4r2%5lqL`PlGDJ=|(onKbuD?U0 z=fFTOkP$VfXd+IE zTQHWvs`Nz(H)Us{dtgi5T)(Sj6|c*RzO419Iop+lj1yCJ>BU?__tk9E;KlWvDFZfX zeHa#G*OXC?&DE91lND7v6J*GmnvuV#Zf4J_I?D8tT8OHJN9>MQl zb2`tjI-22=w-f;143pwl2aCgwbQJJ)e#fzi3EdvYhHQ=DBhF@cl&{M??tvB?Z*EC@ zy?Q;vU9N)WxZbRg1R56 zsaa>{Qo_1|2~pK#KVOyMviG!H_=*VOxzLyAI@6i?dbub2?LV^&^naJiYW`X%Z#uEi zi_hi-{HxVg-T7Q~%jO8|%R)=qtC^O}*Vdszud_J@_HwPJ|8Sa5JYVEe|6{8e`np)r zw5Ke|TNC95EDxr>wu-@@FQ@2`Z*wfnpG$(u!^!dr>li2BrsH9b=bMI)ms=-}m+JZs zrYi*78a8Y}N{X73GPAcWE&c1p*49;RC4POP9KAU%KpdNSf}8oeeWgf0#i`ziV9&yEz8 z_(N40Xrzte;y`iSuACIJ zBQEw`Xik5$(w6;fxwF7^1$fT7gc`3dPq?))jEFj%;A2<%s0q7La{O99%4c_^B=kf@ zj5``9###rEo)cAR_XkNazaA(UIbWx%%1=!^-0m1I_Qb#d+f~%mLj@`3NXsgi96-m* z%2Hjf=ZLKI!ejR&@X%9DarBWI60tLy9&j~Hh8!E{xhDoL>QrCFw`;4*_cUdA>j*K% zCd>Cb)FR>>)1|ZnOF7Ld!-sE4u(3y4deN2y^<}m>&w08j!`VKD2yu)Te>v4svw+8t zHW@K~Z=9NZXy78&<;3I@6BBtk%f=q4s9Ae*eAKbJBYrRBgHXYWr`Gmj_h z>+Ob8oI^te+Z5=qT_rxoHdYd2*OF59RrolY0vo<2Mtr`U;gU~GRPc!bo9>uEB%WSp zK`Tq5>}pEtx4Ej?tHp{s>j)-rQ%ZjY-9L%>rhGZ zv5tW{HsP}_XG?M}rm^5%6(;RaN6NUKx$LbyIac3mfryeGuZYErfpBxt^n=E{XelOHg+xM+WT4O2THk5uWwMLBEc1 zaz34m31Ie7R{GI6Gi$jQ<-XbneZMY*xowHCew)Me#OYR)=X?j|^Rc1=_f20#`wC$3 z!8jtsR2y^OToH9^x;Ew6-Y_BAuHqN($g#m&LtwWpVgASC@#3&E&2{c1GIY7GC}6(3 z09Yra*O`G0KQnTQw?<&T*1n`C8-p2d_T*51yNaB2tfS>P)VR1EX|cb3oCjz)KE@`+ zhgk8n~a9VUeQBBO-e znyN{AW~@%SyE=jl)3!rG`miw%PgOjaUBSrOm0-d)2XowJn-ZK2710kZHSu@mnsZ+t zDww$&;*!*L;5F}#qQmUtggBcB<+nRj?6W0Che}G5o_1pcezo=^68@a5skzWGA&Z?^ zA0`_jpDc8SJ~`Gvd@qc|lrtSA|42p2xt`&0pe-lttICOgPV%YWX4#k{GYkH|8NCyv z_%kCpXLS(jvD6LyU=wqob_KieOoLB69EJLB3yZvFTQXi+dyCwGe#t(bWEYQCWIkOU zMEakb*~qVx<*cJ|a z;N|{&*WEE}>eWnDja|ja-;?4ZY(sFLT?xc%UycsFFma1dbqwfQU!ni0mXHXnS;pp2 zam0}lm$2HM?X@#p6t>Y1@;VrS23!~j1)HLRfW_{7mvfVVdt&5bPtAl3hXEC}qksfh z2O-{vYC__PnVNfSq=D8&u+JBB6ePe6Wo~41baG{3Z4G5^WN%_>4K*+zFd%PYY7IO* zFHB`_XLM*FHZ?UfHa2e20OJ*%(Fz&v#_I9*n_5kIbcI=X2$y zUl-W;%P9)Q;^-yX&W&UL~+=&M1GS9@`R=NbWSYY-cJq#`99Ysu+*Qhf3^9WMQw0Tyzh z$HZ(%@u5q-aQ6!x8+JND%h?tIeL7I!YOIRBJ52igy@ntD>)G}Bcmorp>xcOcvQqDz ztLflBC#j%sQ>5$@eR0IL1mw2Rocd^?KK;SQ04`|0pB_EehJ3##CWdbeVuClt#ZlW5 zOvttf?Yr0xelyov=(Q_mBrSGe{JP*@ZVn=2e%10af1hp4e6uwG`sf(P#qTKbQDZgP zF8#RZyO-BF?_zx9RAc(%sfOs=`qJpzmWq_8+XIl#=M(t2MLoc4$ry0ok z9?%C7CHfZyJMp&(LBenS*ubCHdkZ}6N>cg`z^U8ACE;6QLfWp3jyX^eGo-A5-{h=M zcTE-kcb3~zo*n6lNe3!s-f|Byd84N|>c8`)tjk#r{!oXG+Li-+D@=1=?ah8U*9v;E zBP}bo_TwX$J9A!c4CK7t87lZ-s*Jxq#tgd)XyuRA{`}9zx}LR7%<1iN5e2=IXz z>9aNf^RV{izuy(-efnmh0-lIbxG)I!8{sD3u_>q-XBIBjp=W@sLb&%#WBiL9G1%i^ zxX}GrjS9J%p~Lk}sqdyb^Fscd;S$b`gk+%Ba!&Lm2^+!;7i(AY^Ib`<$E5|AZEOYm zR}}icIMGp%|16c3{1^)L_yq^51uFmxN?;@A> z#}pm9BZm4JSWyoZgn(NHe$=hCUNCS?CSs}u?7G+o_Ww4|B>lZwT6Lr%r=RP{nFj!u zPPZgHT^-1Ld$~x(J1lhE#t0$iP$M858LFEurb^5IZ;?y?(?T!V6@x=I`_o)!8{!`R zcdfejWU9JjPt8J~ndn95)1-nUBeCFEPssWK!K0glfu3BrTHI^l#tL3A=qQ0Jn^o&H2e2M6$`xF z3;VP-fbiN9A$$%L#X;Lc5cjn%=m+ZnK2*p|d)SSM_+_OF6>z9vW$nwUiPk}skG?$h zzLFgB%SLyB^Fj~adyt*`92NckhLV}@dOpqp9m)}bGfhbkbrs>a$2k$Vjn#Q?_C~p& zD@!GJf0UYZpr9l zSWnJg6+^$wx267W6GOeWg=qgRQE`BM0u#46ijC>P$KR>Q^8Gi^%gKkTlAPtf+;_{p z;CE}lJ|D=DA&29|0Dj{#cE?a5`pU$Ia~)}3|F_0seVwPl0e=2;J*RM{G3T|hD)sr* z9GiF!@R_j#NT6w61gl&t`UfG5U{5A){zK$_i)QJ`u`QKFzUDFDV z=74>Fo9;u!|FPDhKcB5?I8>1{*M(WHw54IcwHAH&Rac$({(Ods6M#H^oo)vQpBp&v z3j?e0cntFSL{G^9_&xJz0w1$BkonS77xGIN`rUW^7~fwPg(cz5l;m3qF6ixsh#I{r z#0M<)z}>V}Ij=P38IQGsjC*tSg>N<`lt^8D!8>DB`jbN?DQR0sim(asUyjEyVL)vL z>`LH10PlX7ZpeDPCkK4a7%h2cq&Q%{J^sF>I`|hoH|po<@|63)--J&OApQFYVLwgQ z!hC_61zGzb?z7FAFNW!nw`!q&zche7zgKb+@6PuiK1s_mUn%&RFINZA0n^R-&YIGM z2eZv-FN}3r&xh$TzjneyZknpF-ZDYfyJ2?teQR%__ns6Jb1*`PKOALb?nrTo`wB?t z`e5E$K+A=!3=+deXlbvk-Kf}8z)M^E3q99*^FHo~3O)A4&`;XZnA;=7fM2isZM7Hd zx!jiXQizRsG=T8CJzW{|$RURZd^6JW&UDPe=OAH`ryV zGv95c6B@AGUie|DE#tXOcs)NY+h?*q{n>IC#QRiNO4L^6zEtxP9soH08R&7JEfFSo zW3V`IAMh5l-C(zNOz6EW2|jUGK}i5fRKt(Er{kyI*YgwZpJ?$}T#(P5E^_qyJq@GKHp)oT zmnJ`y(*TXn4*AVk9d%2?5BO1kHElJMWL7A}^=vn=U77YyMdll$}{{selIv*pegsR!87| z#}+R2>te&uiGdE@kz+y+m5A^oH9Fx?g^mV14g3P&m~On^FP2u2&ne&^kJOCp>$*KP z(DUsRgxLRO9m+5f%iVcyrrNZ3*Lb)trzQ6D^PlLd5x#q4{GwGc?#pCb){`M>*iVDR zkRNmv=?|7VP@kwP8ulTC@QG!BB7*K5DCn>=Kx6Dnv4HnL`OP$Eyj*BYd%7jedwZxvhgmuh zAIr1+zc<$wgg5}tIo*=wy50-upYE)ert;Va9gxq!wS(?1^n*XPVuNmtkfI;$^b;eO zn?P@O269~NQn0tRE8c6iDdMf6GU}1-I@4-^n8C~O{!L7aczLR!z#M8)*3lR`+%bv@ z*^{8Z0DF~eZq0b5tzDg<{XVtbC^-}to54{4FhZw;Ungoa$e5V zC*7Z|kGrEU5Bg=gKHx6h~YCxV%sNh@exRBpZO>DST zgbI{%67P%gp|_Wtvfu9tVLrP{Si&evH=(cXBG|DF;YuqI4?ztWrSd}5%%E+QB;K+=V>ns)}IsW=R znlqz-aH?VD1KK)%V+iTJ*j4DdI#BFk6BDBj)B?nsj2^!sEeT!dgt<>Qft@$S*q}qe z*V{&Mq4Vu|ueSPN-d8hqt=Haqy&vnRuTFfVua0`OIe_p!lv9!~b^MYI1t~>Ulm29R zpd@Bf#z@hYW!@d7Mg2NTkG?ZR3ix$|;{Uyt8+&)N2NPyja`VRdsSmH`6zwb7pw&KN z@DAWT0N)un*#>zx)l&3&OGJva0Yo>`Lku!BQX(d*P+l`lF!$9Sgy+sMK73`6onUG~ zy3co^zntr<1v_K-$Q=nb z*q0uJ?@y~8Am@d){CB2W@W+)tOvuVGA*P>^0$@elL%WigY^jO6Gt&@td$~RJ>1<8d zgRTD54+m0s+=YopGWHbvH6S8xs|9&Jf6dgiIVNz4OKq9I&o`z&T?3{ck2}{-uGO=@^Fvul1(AAEgG|?uNMBoL~n2Vw0moYKx?;(IOHJDV@0;-(P*xx0GB{$zo9(ro&@{(U(>be&mCGebg>8Jxzr8+ygQ0d z1-x_Sp@NWRSL2g`ItiQUF8J7868%eCarEQK7KEpemH1+z58}5aM#ZcPp)qG#V(!=L z40jzDX_KKt%}r?!ts->5>Hso$tp@>cGBI+jJms;J9esDcGyA>$`n{DgQoOkp@@Bdd z`FVGYkYXFib6sdkemYm5`E0p8_l0c)>2qQP^uB?Gm~Ag~IaHHUFRpXa&86gnFRGU{AZQT{9}=VcIa`LvpvbrEv=FFw`7P= zbxlTqu_evJZX!dEEp+nvBo6)80)u=tONMMq0UYShesyfZCvOfF2W*K5!TSno;;E4X z-y5aGI#lGuJsE&;Vz}3-nhq5&zI#k5?y%AjSt`r|O*_iQqeHiAoG5~h9NlT*l zH9Ux+F7JuHBJ0^~Q^AMze)#8|L425FgpqwHC8W)^=6;xI%6WC9reqwdXz5$SxGx(L zOz>no#KX`~{CsH;;5KBetocs7_qMnSzB0_uvHxJ8DAg)(8sVW@`S|sp^U=0}nIX z4*Jwj481eTjeoe@5Am>1Fu-3Yt1G@<&-4@`y=*di_Nl(M+@UT-EOlW6bPdQ)%37Ff zCo%OGZ9N3g!c@?fB>(-oDD$0FnCAz)-XPl`^wZH;al~SOfw!EO`fw2FPe9i~&-J9t zwVssMQTKfw8*80Gn_AyNK_3YgNddwYVd5-Uij*Yl5l5-p?T(qSr<++IL z|Jx`h{+YHW>y4ov@<~-w^u8eS?XSa}XxB?UHfgyn^UYk{|7!d9$?CKx3!VAz4mISQ z!O{%pRL=)a>}2;}_7!yasgaGg4HtMXcc;9WZcKhK*P8!oWdQYgSzPj^l@#%y5$=6! zUyhGGRx-0r$MG?HGFa&9K(61ev@qaATO4*`B&3)-;K9HaKHZm-63_LR_%kg!?dud7 z{&kLp*^-jt47CNWQhMs0Jp}{0Jw}dSmz9Jp^%r?c7!fao^ys&HN=)+QG>v>Y!>U*s zMn;J^(T}$jsLcOtmR9{`<afF3xG#|{Fw*#fA?*r zx(Dzyb$?8vGf(u0u!Vta=jq<856i=VjvQlT0Co9kS4vB8Ot3+RIu67t#QE=!;bZql zVE$(+Y~;Qi6*AwM@@lC!!+EKzz{N4a%(IPBQ|^wtO))iLsSE9KF~uhyn-~QSEjjZ* ziHF5DLuKHkZ|{UM(FM+Ey*@q>}wl>xSi{`sBJ)dTiWv9N=Q*ZpK2NTd*hV!4d5Pj zl=!&ieq_jW6YSGM8_fGid;R$%yI>s{%Xqn?;N-0h zVf-x(S+6F_lb$bjfZPE+nRTjX7VM6Zl9biyFEtHW-VO~1x-P^8ulE(YSn5+>$QhxJ z=9*KSzgmc(s|7ZBx*r~<;Ag$s8>3_&Xi8$YM3Auc{+s~7F9a|47JblEB|V#J&+`X7 z478h(_-3dy>(iQ?Rp8KkwaH%W#>4}AkJenyjJz1=% z+n?kRSC#DigRyJxf{r-R;1U7eOIqwI_E;Aq{pQ-zpY2It-s?l~&&KN5yE9FR4>ty( zUS~!g#$n=NY-6-UyP8wDHdySn3;6Ht;Q~)fL+m|mdE9SWK_Z~9kWp7tm4cn?yfGC$ z{aDG!*c~hhbjYb0XL>e#S3!!O>wvtUZp?kYBc-OU3JI~snyg35?L}@te|=tRf%;C@ zLO*TvpnQ*%^z3ahE_}TY?tU@F!R(D=Lia}zJ~j#L^C6%$&ve9;3mp^eP;tPTTCmGx z1L*Uqnx5+z!$n<~0nPHixwe>~EipXcc!H8~pdduA19-0DCEYpIF=5yF?wYEsXVcBu z&ur3S{|yPkYrebSgQY&p<=n_D+LV%F*Cd39>-^1Q11;Ak0q%DY<>MG5r5vd_1-mjn zXjj1n&ktaN7J6|(c1>NmLt9^cVXor;y;@p&ZX%@H#E{R{9`Jjcf|R+a;3E12`EFuP z#&d^?hu9qozS2&eFbk%#ls%zskvA4bmW1iIA&`YR9<(~=G3x0P!3u|qJ8b8&J`LrS7f_6qri;pHa&|?iDX-k3% zSn7j*1ZwBgnUPUs*YeTk7MS~FQ{Kzt349XZ)BF#mh@ih`Xow>dF4v|+25m`EzSntM z>p*X5E3%&&%2J;lYf1n=K+3RB;M2}$7&yl?m(ayXel6tZ`7BE~1uH^E@=Rxmm#Gf= zR>?|xs;|m;|8168^3STE{EtO8aaWH^*w)|^{%aG|{jtnu?2Te$76Cpq09qg5bN?d? z8~F|JBu55H_Iwxky>$rY|EIO4=W?;C#cpB~R;9&XR!6Ww8{#7OxwiOwTLUS-Zw}VDv^W8AlrS9D4 zD+AdtmwK|DcZRXyXKHHpS0laP`V7ggij!wom%;(>5;9qzdf!-;cyE0O>|v9m{8#&t zo`Bbh*$~r{bPZrj*d)^*rMBdx(hkQIS@QN3E zU~WsT`7chc|IaLka5YKFI?dVy#V&hxmjGi{N9vN~Z6d4uiJAC*>Bf}3Ox?g*o5>*<}f4n641j3V~Fs@o_r767%KeGKuEn< zDsMcSFD)~-7kcYzz|O$BmYq$pSl^bo0zivZADIa`D}#k@RuRO_VP+s4mNM?z2tIsg zoRoNArWIZ+@R*w;l<4&#QuwZn5)bq|2+%>LeRSj$K4%IQgVEj{mamV-SokWvp+m@wNg=>2+M>hIek*r#J4ZP>W+$-xGl#7S^E&~t9^y<4b^Fn^%V*C4gg(uVHOZKrIeJpPME8u z0rb0w5_hj37yW3qvB2X%N{G={(5@{%LEd zB*Z#Mh@I~%_MfOoyML^w!7ipt8LRzJ@1?fn2U88HPi!M_Z@Us5e5xZRAM2QfHYFo# zL(WLrQCH&jb(|vGD8hebAm3S47I$x=D)yCC2oH6r>Bw($W!!xgHs%1RrTtM-tf4CF ziJ>Cu_NEZ@VS5DWJKqiZaB3vwT+XnGM>-b#SXWwnX5ryi2MgTI)k*iwmFbU+6>;~M zS`%MhXfTQAhBEv@4=zwwlW`w-eV>jeh)H`Ag#W=PDs+1U@_Ac=@;MkMC+w?vMfx^O zP$M$p=L01v`7fY2)IXpzSxvQz4w%eh}|(n zD6VtBe?0=i$ z()ZLH$}l_UrKS=bexl{!*G0vFb8T6_FZY2zY|9~D4jnS=&qXc`;H9ps`KtPjQF6l7 zQcbH(K?luuLO$v%6Cc#)yu2wRhumK2h5BDCRJW~b%Q>35!dDBusK65g;E^mG%Bh({ zIWcocCq@o>2f&~MEfsV*TT#9_N>AGZ{Oq@RTJc}YY}(%|70p+(Jof*dr()$UOL&

{1BVI*Bmb4!4=$6w5rSNv(M6aBSOFF#-8@pko;?2S=s;^9O!->$8!2k;K^ zb&ig{S|k;2$>EV35_t56xCp?XQiU{)WU>1w6yH&xt0fjXW1Bn`7N}8d^d#1xE&cAyfxldw?5HUx23Gau1lyf zi(QD1Q+25i#^{m1Xt*&CFEqHctEno%hP(zlC*-Hh3)%6jLyYLHVQSQt2p^)WPJGmh z@crJvPJMbfgb6;?QZm*@@FCV=TFRlO6a{c;sHr~raSQ6>4;|=tKNzdN{B||ZA?=&^ zge4(1a7$bqU{{gT0S*h?Q^5W0QBeRxws3#L%^2OyH)3n6x!Q zN<1-;vj1M@&@U%hCA$)2h^0R9>1tnrJD?8%ZCLnvp&jads$~{jYqC=VDQ!cPA2inz z|8lu2>+L>(tBXQrvX~n6+XO%1>Cq@P-61Ur-4ho19*)2wcOuDdP2HY2=+YCBBQoO;Q_m2 z#o=FPn5d-z$j1#CCh~HgPdhPF3LR=p{MHc2W2qza%}i_hi#-K3{oGQ?+Z(6nYpb#z z+eUB^$69Lcu@)2a&jJN@y)J+r!(N{W-5jImo-Ne0txBmW)7>Qj%R*w*e0QPGfdZd? zX%VnaCabC!rL?pq2|nz^BA}g2vXE9W%+IDkMeM0c5|52^@Z~hU@XuvI+0}eyjTO)& zdn!!qU#r)BLC!walF}R#m`Jj~R(Uf$L)DQr_r2%BvOU7yi%+vvZr8VoEq84eI+(>SAhzcY)F1G*O2ghM_lNCq~{`5#LQ%4eZKoj4=Vm# zUs|#wMtJK52@hnnuwPA;@%N0?iO*~@T+E@C1D$Ardg`hRyss=w6POnwksUk9B;~#U!76HNztv>qt3PF+6M=@Rl|` z19WI=sGIG_L`+ntyas$$+@T7Sb}_>w0(FSk8;8Xo>55afMv6Wi0h;01!bBNbGu|8P z(_YwyVSzg%uyBVOmvJ@Cr(Mkn*nsv(HZ+1>AF3GO9T_fpx;gdruB6cS7{JaGEvxX% z#KBz6Ro0#_RMoF5xu8`!AG|)wFPQ3pxUCCeUk+6y&@tep_r~$D=TmIl-z$|3*Rym_ zOmsL^(uI)-Xl*ilcMO+o zX#>Am8bW?KnPOp2=a?l&W^&HyECt~(5kMD<{PM3Wg360o7V+3fg{+UDf=(vs(7z`c z*#EB9c3;fY)YuKx%v~ilaaWG;Ug*hwIY9FHX@KzVrwP9Q&+CIAkITtY%D$4Gx-mkE z*p?Hc59A0xhZOp0PXh8>>WqGBst^9{a+-_ZyUy7iWJa$Jk|S2S3q4LpAz%LA3=#By zD@?+Q0_vx2O@A}nRp@o7swlC^%ZdQ~nrItA`D&`-@6EO6yB(=&xksw@#yy1qz9m9` z*%Xxo?5kK{yS@y+A*Ux8>p;#%LCS-@A(YR_I63LubUhcDkbE$P4LeW?kn@9-@Rfe7 z_j(`LWk-teJ<&4(Z^S2_St^;kDr(xc6zb{Fl*FHyc_jxba`#yq*CoZl+oQy|?J(3;ir6u?4Rtx6Hiw8=M@mxe zH!~ss(nQJud{g3lSJAsI879O2kx;d^P#!|?4qs7@z?^m_)^P&o*HQdE5n!o zb0@@oc?bv8TXps6BoB8r$03}VaY+YbB?;daxU7HH%Ip7JEU#J=l|*XmKyG#|mvRYc z#S;q$vpb3jA1{k}rmhITy{mwI`D2L>tP3S~fEs&OUYh(C(A>}?!0XyI=t%2O!6)lL zkq6+zeNRoaqRYARvQ;5AXs#XRaWqbc{b!+~`Ww)5r#eiEO#%toRl&n|^c=|g1Rt^> zVJ439bFZ`Jg1%X3i2rBh@lJL4#48Ii@6v?JwD#w^UCePQr;F7s3&SO`lifKk=aY04 z;NjSRuUB>bxl++`ZlM7BpPY7TVix_mTwZC{u}~}h2%q`(oR@QL>Cg9-xHyNNkO}y& zlD}s;q*FaE+o6L;{;@(s7}-QF`+BC_QDmE$g{j5b~3uD*DbWz$+WV zJU76T743})AUhI%rb7w*lp%e$#aSi=A&zcxkDNeqV~GzWjEA;j;Fs_8LAvf~c&HU&ksILg+}8WkUas{fy*iq}BwkK&NZ;mansyY##C0h){A!`B;*aI> znsW;sd~U=g?@0gu?7jDMV^@|Y_|r~zPnWuSYE)e+qf*MGl#WHJqz^xj(#-h1ys z(4{w0D!pOSlOW-}2k0HV_arl`x@Z53-TSh3W@jdLBj$&VSVjkr;t2%u0q>rB&-u>x zJ>b35er)l19s2t+?6#e9oir1SkaM9ZfnCrd;^cA0i zS{2fA`=D!Twq}UeV>&i0q-Mh+3zV9Sfzp<<&@NFb65!*^+Kg-P;mRzz-tnUQ zsO{&L!=5Ky$76bW7j%wzLnXm^#(4FPSHhOHnF0FZ7_CEfzwNyF5$u%hr?&65CJL&-PepgtPKiXRzc~|i8`Xfqo z#mY={N4^rA<~46l5Q|sez#txV z!av#bLr94(j_ILwe%Ka>Dm^+W7{Lx>8-bz zYdc0hF>e)f0xCv}=LPoE+BoH^`&G|(*1I*Ih9+^v;b}r?Qb}*l8pNa&SmV$%wJI`8 zug}{=Oy3;ll5ZOD2_^PwR7b82sVU8AGaJ7PG)6?rh56@s-7(YPK-4tQA61KS2|X9F zF%3K6n?s!6oJ1A_ZCsbN^9cS0dVO5S?AWpMz#J29MCLHXn`7N4q6;LTy_vA2M#@X* z@7#C4#-CrCWVJxGG%&|$4bPy810Wu@-@(o-{?hqfRxe={oJ04MCJD#CN zTa(?VqdH1+#yN01W*!_!nxwpx3F6SmYnv18M|WKuQqssFhE;5YU&-xSnqans^drN5 z^$%m3(j z+rl!Fw^ML@++w@_WrsK;Nk)q?kp*j2l30)(DW>8Uboc%}4u;cON>YZ?WGD zBTgeKj{bw~CA|l5H#Rs9J!@qk{4*VwVtQ(82+CFU(11^MbJ+WubZKphUX!qhI0*{} zlW__;X(yi@Gqcb>CA|U6NngM)FdQ;U`C$#QGB}GV4$dNrl6rJ?2Gq`&ZD1s592)RW z@LJa=8O_56mems1l(zX80Z8^D5jgFef+WxUHM>qTcNVDJEi&(t}zF z(Wer?S0_nVm)~MeyFnZ;za?B=8)uX+zvfp5XXV(GNyY+oSm;+#YXB}^*%+smY)uGS zwx)&c8xyRm%?Zl&tugfJxRKfz*3p}`r}4!rFX2a(l8Rq1h_C+r^0SWbVjxC0#@W?S zY(=&FJ}76>MrKDwPi_b)Fjtq~Adb7AV2-Z7BLkg|Ikh&5Jd@V)5V3i3by`Dd2+bg` zuRO2)e(P=5555W1`J|bH49t_u{j^JT zR^RrWjv0iMjBDUd+$7-y6ud6z`{0~K%t)x&-7zJlHoQP6-wL?2cbeZ;R;YEfF2HF`%L~#zCHBq3LsZO`*Nee;6u^$Bjn;u~7o+61+7TfuJez9wJtzU)1^F$O;moy!aB zB&4uLfCw%K`$DRIYFHx&?-zks<+efJza|WPly9EbwK9gg=z84yrR84tm-YwnBO9X> zh;MZO9@Uxv+Bs#FaN=eW8R`KV(-A8|^H_k3n2P}rgTZ-DYe>cENP_hM7~JJ}RQ8KBje9n%SLy73{bt_{%CpV^WC0VC@AY~WOVmd}^T*K;# zsz|j_HN8D)ka2fhLLSJmYS%9y8|^*=Wc~RA&vW zo*kQrkuvfLAvLWz0bReP?q{x#B2NPjawVX`l?BaQeA+3I?=BBL-CdR3+?dDLh724G z)JwsyWg$a)eD%^4;>^+%`n(_T^tAb7->6$4uJ{QHbZyh!7YWzk{eoA>PiP5^5jD`^ z3Tm}qK`6~wS@^d&gS2g+ALtBWNJVbi zo+CFz4Yb~@i;az%1bD+Q_{-CSEk6b|B<%ZDHY2L;Js(lwE7C@GZ^pz$rS;6t9LT-* zHo3gukjqmR2?OY!GVe>&@%3@a^}vD<7EuncqR=@-sITFy1P*Z@&(o$qt-fjbDy$?_ z1eBD@kdo1mvx*qI?x9CJZuwBk!ox&0tTqq>%@NfA-8VNt3MeJ0&;qwTq#&1v7wEMB z&nta1^vaA*NGv$|!CXo>K|QBCpq5g?>JhPjUIgEsqt|TD5la_ep-*f~5Q>5ej0W(& z+NGDUqib)GXF>~G+s==z?F1^K7#%y=qY=L+S z8h9vxiQKeJ%u6}MJWx}_e_a#yXRU;m&GD`ua)4(QmL&rTJF_>WrME)s=u>e!Hx~rG zP+#1HmO^kpZ7FsJFsy8PHCD3pB0xE3j#$?xtxRz9xFmVwGw>IYnfF=T*Z$X6E8e>(izQmt5-D~{D z^Q8To@I0#t@Dfn#I4$dQyjIiC*yH-UU0;CD`Zl4Y)raP3ZL1T+vMs=Ge9C?@(1yJ$ zvz*4YSuxx*D#0$j6QMv3f>>pAuZ>}A{0pqEpiYcW*ya5j3Qn7MjBsgfmR9Ba_}eXN zMO;IzOX;aCfcNx77MM){Ywo~COX?oq-JHn;7 znh#6q_$Z(~E_t7}e6#Yr`*?5;ekP{DTmiU=Oxd_tP)Ax*Rv{^E<6?k@x)xOQTuK`l zU6BQL`^G4tIByhCcbD(XM9e(;@;k(d?KwhCR3{*$tpoC`tA9ACq_t^Ast>7$E4~1^ z*%MKTNbZ;LAJ->Hb{9pG?FDA*;wwr?^5gk20d8XgUlfMU_iGuwQ4JecunUEt zhTV*qcxb06Z+##TWjhbJQjAUH`R^BV=&$#$J{{pVX`g6;7)`x9h zY|oG?^BxgTF;er@%6Q+6w2g__bMdGj91LV)fz;>vrK|EK^U7gv-RJ*XN-xBvjP#C- zh29a@;mg*>+7Acj(O2RIz$Gj^Y{JZe1OM9-23BW4g}&yS?Y+2TWAtUMP!D)&1z7(I zpl2Jlrb*SoIcnXmlY>d_lX@)!}h!aW(HtMTqD8;=1JvSGw4fUHK)s` z0Cix7T$Kbm8{jH3G|y^u{@i_RdjfMkX`;2|oJ@Gk$Uy_12isN(;X$30>{ChcTjQLX z)fdDf)j-8(8>4MUBlCnx-xRshuj2QtE{GA{331Qb1iw3`;=p{<_^YlbJx3N_!%u?G zLWT`u%DRGHy{_Oi2b4F#x<4HcsBaHM4Kl{&6spMbpy~6#EVVvq6jQP;v0xMGPpn|K z295oqs9DB`t}EW0#9m&0(S8`{jH6K%uN%Z1Ckiyi)(op5s+N!gGt7pBn$aAdCseGz zMii~SMxR+5#a)OhdEH5^gtRqAF7^DXATKS8+m!N#NREe$^pn}ziCS$cKUEEh-2Be$d0k-Jfg zjK4KUEVn*t{L%HS?P@~DCvQzlP}ZOD$JXAl>jMf2CZNDqCiS?+paNU!`nmh7SwYor z-qDIbbv)`izdlJTpC4)X6!74Rn2rhG7)2C?X7LqK9kC&-CpQK(jHZZI#*JI$@{Ji< z&Bhe*de$wGWtZQ=-gv#>q0|1dDgPx5Fe{|zA`8m{KgFzk z($*}ccKI#*y!CO@S4;1DOTsE*BjBNJX$uEA-CzC(&+G21c{h)eGt;}mqpioZ(z4I) zb)Wfrar2o^ZXz!p0$4$a0W1&eXpJ#F6>t|qNm9=STAqVgo5Wq-n!#Q1Pm`~Q70ia5 zT|j;B;nDWoY;4*>Zx1V}HQ{-7Q`*2oW{o0ZOv!Ff8#u7IhS`}kun-wD7Z=sC+T#Xx zcThzKYs6_u=!DoP;7}nAuiHD0yJUFS@l{Act<70@xFEpfv|h$an}k4nlI!C}%=M@a za}Cl2pq|zxjBHpy!v>m#SrbzWkYOdOE3P3{+8;I^vfpj}66l?-hyiF1wS*AX_A_Hz zDLJMS(xN)Rn=MR4!i2B&E8rJ=K+C2q98AGJazAdm^KN!g(gSZ`2I22!4(pL{w#8mjL=YY~Fetj_Q>tu{1`z2=`m0-b~}NoXXLm~Lcv zYl2p>If^}JzSH=*^Ir4kfZvxTwe*$*z=DJp${}G_a8}TpGKgtO9jAMH5^>S-sN(nb zhvk2Wsp+tcW8_iVE|W%dtgeuXkBb}o1%UT5(?(iLWUlXGTt{sVf*gz)1;m{9<|}~L zw?n3WLDVud6w!0x;RR}Kcpl83mf5{E$*5g@%P0*i#F&i?sSn`4A4AvL`@X=5RBUZEgu8@?Yi z^$WKFp9H##m07tp4s<$kbB27wuf&(2UA8VbH_FGa2L#9T+hRR zI@uo9eEjaEI0W_&sp#EVhio9{=^xIz`K+9K@L|d>5hSfVLe{~hB`sW164cm;iBP>g z-*-9-;>0&ey0$rvE!v)@b|iE>>U+2J=D)78xoHc!esh{q=XpiB;(kHB8a3YhIbh%- z05_`%tC@XyCtvyC-iq|F(r($|W55ym%TIl>;*RiFy@XMZ2eDlgY z8xylg`*R@H;|5}7R7a{PfE?JdOZmGF2|sHU5I1HBB}=b+zFm3I{+0i2-WFF?tqBXEE~IWh6Pj;6vN_&yBmuNQ#?miIn{Ge$D`_o`XH8#w zUp9X4pMe!;Ewmn>C)+@6z2O?go>`w^Gy<(Q5K;5`Y(IB>=l;3=tF(?(pSN&O2`#O6 zV-i*Fc?st19qFoH*-s6t1brKmgwpxJ%HK^1OaIY7K`l)K9gtLlx-||o&MfX)&LSYB zj67smMQvPr+xWxktA;NFGqBTXGY6fv-g=xc4ay^0J~9rlEVKaa58}#5=?Twum&8?A zpp}?StFw%{gtcE51Uct@MJQU3cYNh~(D`*zjl7h#l3NNcAvdZQ5x3`42U$#7W1hVF?5&v{2)OB)1Yq z0X?MdrzMPXVbUa%=G|iHhc($9kO!Cuz|yFe-LtzW7ANf-e8fO&16;K>qM|nC9BlNi zn?uPvXnhGCscH3X@Av*$?6ti`>8*%{*5-ZPad`1XOW}*ul%Fu zr^-Lw8Yn6I2c`wx(H)i%*c3W7Fiy=rpN( z(s2l53 zm>YhzgcQ=rI4P@4kaLMS&~F$QU-X_@AH|gT0e6n;$RJ0_jazfL;=lr}Cj40V%QI(Vs)!iJ9>G^2i z47%9%O&g=4&c$Cy7lJc=CsStRwWNty zo3;okNq{{$XTJ>ShQYX1%-o#kAy(c{sy5!?FZ*VY(D^>-oK<;1OQ_6y`K-Kq;C9L; zWG1YvzO-G)i0K6AkdoCAoTJqHX7D8;4YoXE=aLHE!Jo5>ccwO!jOLJr)|@o4F=-PU z>6^fm*dDc>Hr%T_x;}}k4Qp-<2b6<++f!Vb`*q*Nz15-TQ6nD*^^q-@fG#ni8UeR! zOglhhnhlj{p^7ItSzD9SGnJPKO)_;m%Z9q^g6z$0lawY_yJu5^#_y_^FhE)0;C%7&)t4 z5;m~99j`EFEKiUpY>(T&2kTk~G;hPzql%CSi8+{X9KQ2A( zIK24^c_ykLl_fR!8wCrqH)rP&(qr4 z`)0WvaaBJZy5`OCuGz3aWW2QYNUEgy-Qh6WZVxh-4s+>V%D%t!)$ z0{BSR=0xv_^@+}7o3n^3N!#$fgl*_{P{+k2OhSrJAw+IZu^N(E0Vb}Yw`@=Mo%GKk z&xKU@8)=Yx2`#fds>ELpE#S-6$BC)r2AF?J=L*B-Mht=fj%~|{vuzo#pEv_o4fSp@>)${$vDC%5d zp3#&AdM9NZ91LlN*p+eO)y0>n3#+gCE~k{l%CwHs7zVs$eG>YuFZOa=&4F!=v6~Gy z`@Rg!VoQLo!{^;BT*1L2Mm6a2kdj!RG%;Z@vj7*j+jZyjLj+7$9q$}Zvtb?n~tNz_^2H2P9(fmD~VQ(+%gdHlU4VSm`j zrfp~_b@?d_z`HP>a>kV+FcoVl(7rAs}p@Ex90FwQ3IEjwz08+1^A_a3J$bA zp?XURKfA3!T`73PQlO2*F%zdRX=dXK9)UFH;8M~$LQ}#(XpEZK=zxWX2^#t6po-D# zpTb>QdfIl_cDMFB=kxALAJFq*JstWD3f%P^U!=a(`(@ZDU=|i-Ls^rMkTEbj7(dtY(RMQq%0F%Wq>pIopE4#kzs zwz!H~msGJk(@G&KHb<^qe%A1fUVie|iw~*~#nhyltesCPIQ!*k69>LF)qB$UtnzoE znf9Y;J)yi{VZrh?0U@U2_JpD913;ccz<44G)J5MEh{+k^^?-s}71prY(-tWkXk>QU z#79R|wED#t@S{NmsXSxm<1=Ov#dN>x3*!jp$CYRN#_%)?@Kti-##qx)-&EIyz=8<1 zHOKEv>gD{TVNkL*#i|D!fRlF#1$j50opbZ3fS0kuMi$)t26-wlDd=5#!|zn}xBhYM z1?EZ`;&^ImbI!)WLG>|f6HvA$NW~!q=|;xP!=wy+G^Eu7ljw_^?|O~_jZ&Hb{08VE zYEUJ?uTQY5J%sZ%i`rYopKVO`o(pRzEfGDf8*ndSP)lnJs0laXCR%IC$m#~Vv@2l( zb;<}Y_RqsFrEOf&&f>@mP={qfC8fIH5HR;Vd{$CRY;xXf`+oIt{UPs@nm>n>nDY0_ zvU__g@`q6^7q&G;sRrxVoHH?dQU+#kP=Tu2m~6YSHH9e-X(&xur&L~W58g`v-s1!F z7E%)%vUWBm2smkYruR(7L}^djxahc@hfO&Ij0{-Qn1R(7(U6+*UM?SSMahnrhTL&c zy8nsSO(HUKwsx{Y-DVKQn&RE z=(;&_%?GbomNYZE@-8m1;NcKrdS*vd#qG+Pfv(mNtJdDMf1A?c>)tzL;*4IxSbvAS zusPoKJ#-!0>O0bvwJ~;8NGT&OPqFL0ql62ASxTjUideqq;FA8jDCF_?@@wJ+mAvy$lib=*0hc_THTCoI4w*D%6;jsc2qn(vy{8h;c@m)Y)+ga- zR^IjeusYiPeP>ZNyt633p90z$#OCGgS<;Q9QAA3bd05vA*pCSd zkMNgO@xTW!8{j3q_g}pt(XNL@gmmY|H2T8Y6!E(IB?90DwKlEeVD{{kmVfgy(Eql= zz^Bc~GSAbdKf9l!PDKrTa>O8@uDnK{nU__3Hp)ErPp)6uj;9Q4%==}r{KJY+nliIs z&^-n64nAeqE#LrcR`G)H&EFaNFaJKKz})z-Ea301iev#Lw%YT${hO4H&)?m=tN*Y$ z@H}oHbZCXA{z)Y+{d5ER{gRsAkT7x(V16(9W*AL*r;xGh;SvGIt&f+hg6_wdFtB@|-=IS0*Ei;9l}lrUOY5VQ zlJ|Bt>VIEm5ps4yb5PxKbZxTY_WMvrqk<mk6`Vk%xwO3UxdngDj2DD`nY z9q@iu>yBN*-*HHUX;AmK6y)-VmWN20)SXpZ9BsFRnHk(AxI=IW?oM!bcemi~4#5fT z?!34|7$mp{*CDvO+vfWj`?&k+gYJWVR@GgrV8V#Y)iYq~q+$hfj3g?910FC9AfQ~O zx+|qVMdn`XXTLj*hX-CbxUM!v;t^j9kWFM_X4r4X1+>flvH(R(=gWWql>i1*p1)~8 zwcTuSBYA?7iDrvO%P-c$e>~ge5afE@taSLkXg;LkeUAG+cFhmg-(%YGI3Wy$Pxp#* zV|T(TQ$HqAqp$y4b7U8J7}Ch4X&titIQ7wewkv?C+pWQzotm5ZcHXvZ_bVi8%U-Er zEm#x#V{HsPeIe-QVcdGrKcxN8FiAI`30<*gTr16R;UqCWBM3jFvImp>{Ta9Jdm0=8 zbD4Ph3W?UcilE!x5&{UOqmB2G{s7yh@P<4|N0k_&$SrI?MsnKOp3Z~b@RoY zqr$uy8|vG&ki(l#i0iR%#*g%O6W8Xq-QHR@r;p)YEb;db!Jv=2Q*hFoC0zU|H@}m4 z?@BP`%Oon%ilYZ)B%lc-b)QYl)428j`4)V(+lxU0)HnVHBVr+a#o@HV_6nuWDj{Bd zRew}KURwc=KGDms2qjcX#gzIEi?7J(g9XLQMdA{W(qF?$`GRgmFp)(heAjJ)Xu-Lt zoQ*Qjn8e-7KSfoXw@CRAJ$jH^l@v+%+z)q$KRUnX6LmRV^=Ix_G3)dAzu#7>9ZVSt zxNY~BHVseb@p_*3hqI?E=688T-G8+I?TGn&e)s3qrx;x3Jl?=4#51@19lLbJyl$`a z*U`Vm*$n<*mZMG)jQ-De*kakMvC_l*8WN zN4-LyZx06n3;Du|Uv;b>Jr9OUcja?+-ZhBGW!eB45xU-+ckPSgfIi3+HCa>Hn zW|zdEvPhiSApp?-w{tHx2`r#-du8=f#}Zb zfSGI>vG!0j#4j8k2l@vIDfq@BUU#o#7SqqJ914Bf5*A{Zt%``DQ=HOr8=>38I%F@4 z$Kx+bwf}ribp1NyH65LSX_VcdvbH1p8dnm(nAPgkfM1=FInTbjV{2=mu28!|G@vhC znPK1?|NdT+87XJY>n{@;mFFJCi@=V%rWo?%O)gDihbR@-fzm@%20GhiQirvN` zZPP6jkO?5Y;{VCnBshEac(d3!D2)E?YQtGASd6{hB#Ghe4|M+VWHY=Sr&Vu8=xAc~ zd1eW~&6B_ekY+XUv2s?Ka{QS;ok|f6n$|IYZB`Kal;)d4Wbs$ZAp^iBM(rUs7O0_6 z#jHaEZ-xYDF4M)qfv?T6iy2v}xPR;U#poqJ`9)i$Y-9ss8 zx*jN8cBdZ;XTb*gLUPTLxRTYB;Pv(c;IJz^ zK0YDtuST|Q1O_CMS+!#`SY6klzCP;7+xwbSrKvTGulab3*XQP!*+wxVi3h9!$e-GYT0b?26e}TURe&h z@7Sye@y4{j=ESBdjdk&ORj7E*jUCFYEmw5Q4UPsq49>@vy6PtQ$GU4EK zH}*A$#>r$v$I&q~aQ>i}mz*DT$^V3|I@kO*ukf^RW>H7yEGr5*{`0#b{*Q-6yjEHd ztlw!qm+~uALC>ZZfvJ$Xw!Bq*G68(GY{FD*VFqi|%_>D7qx00>Lif7xX^`{Qf>=$H z88q$yq|_f93nJk3vLK5zlc6BG8;G$q=-QFQS?E7e!znzcJsZJR#=>&WaE{xw z`ZOdZ?cJgb&7f-q%%4k!{l6dLD5oX%RU~Y%TbRqS3wC@wF1~-n8Lg1?1weO1$$U=9 zGr%J2H-kG;f3?Z2GUUPIn#`DqjKSR$33-J~s#^{`wIN26l1U0xOxvPL%B9^JM(xG1 zhy!BG_joM|IlWSSbn%Ibw7*z}kmgbpf$xZeMoy@D-CV1F%kOf3wZ`7drbEmyUBsxY zpshx^Tcu2dnT2g0clJ->NmXV2Y*ZDh;;k$1*tz9CbhuS-4(5TlsZFxWqA=IV4?24n zi<=yNnT!AA{_2Y9z#1zI(VLVp0>yx~K35HeioMoLl)b60@YXUhiO8nGNuFq%qg{Ig zHb^5@f=z;DrmbI7Cl+&}H-0s-bY62wYE8*IDpc?o=oE*ga;fSj6PUVhBAWJ14DX$a z*)Gx-SvaT~F0%3@bI56bPBHj@Wk%_mVzZrP!fe`tlA4ihbkGl!nZFXtxBC`7JkIgF z%5PEzCK?cUCEk6W8|z-V=FMpjQsA3#xCHFVxTxaV=ej{g>7E#R#g>l?i6q{{wzBDs zh*2^@yw0E=CWqqI-Q1Mc?$AYyB`o?+HBSzK7)`5w)4O%2`-a-#0Hfc>CD$mf3+EH- zNR|WyEb-s6j~J*dPEv|b0vo6Ji|K|ZhcT*yh`4_dno7DD%uk5!7nWful3-7e3mqh) zS=zhU3{hL+=-ZzfX?dS^$H+}BL+{o^&!(!1#;to-v}1C%Q7OBg@6iZG79+~KZ<3qt377_~ zPKjFC`(bMHZ^w(%GxA7G#HPlqe+9#uCZsi?Y>n!|J*!dE3EJ%T&r#dj^&uP=hr=JE z)>7I~aeYPBS6}^#``t}U-bryecP{$MhPZr~d&6_?$@NHB?_B+L$fIkgdO0NpUG>tp zm8OE~o%On92JgsLR;KD!!GpwIsPfJ4%KafBLaNrNGat-sjx6RwWZdfrtJnQrcHn1? zqmjj)bnNo!vc_i4!*iO#hdjfYA~fwBUu-$c{^=q~B6BG8{3^O>PK3FM`i0?#PGJAU zksM)16md+(T_1hP=}M(Tm!hA;W^g6)q(`1St@Z}*-x*_V@fwtA${0jcfd4cl#)rEE zSF$H!rVKl2K8GAH0vH$5lAP5hS@{=)^$dFXA{BKX=f!ca`%RoTJtN!Wsxn*F)B&R0h^LF4>#ly%p6dXCBW!F?GTA~I_-D7^d~9S}>CMrIsxn?FmBEhSk& z5r5k{pz8aDc&^i_k}m72?Pcfs59RSXE<@rrd+Y|Z$^ikA$UsJIT(K$6Yo5KtpQ4lE zVeTanVVR4+>Xlb1BxTQ{Z((vY z>u0mnRGOm3Hg^DBH%ko3WDU-@r~S=cyxv|(xEdCIK_kLZPQgeMK6*DI3<`y`jvOf_D)$UhE^pxZldV~bo5K_K-s(MV>DN}UHR5E_59l)ZKZQCjI{E|FG6b0hc`A>UwVq!R z^Tovr;$!}LP%{?&F_g=!oWw`!A%<`=I>Kd`VEmmUV^)>qb;R9$&PxxL&v4qqK&hfF zqfDVfPKn?E-)H14;A=YbgAgs%oYG*glXZ!6B#iFk?nqD+ji<`hPva+@cPryOACU_2 zefIWw&Lqu4)Y_h_cKF{mk^?!WSq2dxT-758meYJjF*} z1l;;4zG8#*&$a2|U}*lMcaZo3hmB$sysGjG(9kM`B8-(Mj(Y(NxC&6A@J2JG0+{mt zFdj&@#Nf2~f}vIrK`no~_yxj`pPra%l(sePCD9$L$2ih%5lZ&$N9mcq{W`K~AEo&P zv&?Rn9EHjz!>*<1`zdpL_ehtyOGU7ey7+%94Occ@Btmw>17_!5-nTiw!eg!J-08rT zrm#Lp^U~~AICDsvcZgbZ1&ILOJLwmEDZs__t0mp@j$UlgX; zUwMH0i@d83;?{sbCaaRxQ=adqugR~5($3|=MW7r#DZ41!XbTLgMp~;Pu29|0OtuCe z5WVWty+Ap834T)e9j2`6xV@aUQ)!>~<4PQzzKU_)1|}!`)d`|$$BE5fQuYGW@<}r4 zNuZeeq^P|feJI2i|F*Pw*7I5X|OB_N~^(-a1as|rfN`s8Zj+JTq|=o&e#%~R1Fx_TcV175jmWbx*%9WL)r{nV;Z zE^@^mHoqbLxk#@Nq%*r%iK==~*GAx}K@Fs|vv9L?(6`^DGu_K&a~|VzV|Z)8xSlyf zhCn-`m#bj4u$mTe(h%FLPYsmst&8Cd@~bL7DX%h{?&hd&pBJg$2k7v%chT;iGa@{B zqmAgm%}9`?xwaN&^S(?<@jmn%g;+s0r;@c(jCR@CjTJ@TTP$kqzc11klraJ4;vF6h z0lQE$z_763SwyEJwi)&X8TMMv8Y+zvd)0t>uxMDzd}odL=lX}pKxQvEQZWTH53R4D zCka%ly+Zecw5RonBIU$-kiss}6YoXm&HY=|D+a{8y8`3ikw~OB?D=PdxRfdxs_KmP z1_k+y8!;4v?_WtvEBVlGan@0L&XgTOCZ!Kw0XXJ4-1+7kZBjd$biy?Upa21$*FL(> z8i1njB%I0F4&Sncln5L*!*6Zp;EyTi$;L;H#&s%dRTQ@N?xNEim¹idJQ)ucaL zRl^v@>r|}F=zS<>5*%ch+PE9R7qG7t>o2GtZciEqOE5H~hi;a|q}D01__jdAdv?q! zOjvszbtL|_=hmlbJ4Xzff3_rS7FcYp)CgLp1d_h$NGxy@b&fdhCVQN(8G9-(xDN;i z?}~J`Pt2^}#3Bx?TatA2oy}3zhz!UyzT-&m_32EMmZYiUEIr*wMD8nYU&8OVJG)*n1QAg@rWVgks zu#4qG7Uhp^UL*)TwW`Cqbd}5+U9C47RtLz5?V$aR!-BXYy&0%romy{NKn=sOTU8&t ziVoX>Ik&^mhu7dcj_HjEVMu}Vl}mmnPDn&*PbWj;(d%P4gUe=gz)HlV27f!n{DE^> z5^J^V4QrjTft_?X(0RMK`O}=`z`Wl~(vNo@nht5P%{U!V&C=2e(J6&!2G+TL2!Z8- z4ort7>0`Hw$mwF`E*00j-z_&X>iso2M~Z&(n@kBm6sbsjq9`%Oevhj64;kclg~fc= z4ZkF3)S#WwAK2Y*+(-rY!Dp7Jht)M$wq)an(Rxt0(A}fL33Q5?kIOX*z-G2$N}Gbv zl2-;ugqiG( zoodE8Yrp(yxHelz4M`ndVCRD|MGW2q8>dyFOlBIX& zH;v&;G-g{X)Wal>nM-B0-l&L8bQKl5*QWMy0L;7YoQU11u67RsWmF-It-%n52{p=T z-lWW<%v$m4mO}!$(i2TRu0XQh9wmV_zKhw^pW>mjW)k$Kc{CBZQ|}U&3RZq_F-ZZ)nqLn2_JvdS!8hmw zCQ?4j6LoWnH}ae&nAEtIzxY#in3y=c1m$(GZTNBk>&=G;$gu^)P-n2hQc@R_8fUxM zm@657V9f+^VeT1#+SpK8`RxvjT@}Fxxj=fbjp5jkuu&3o^w z(PNzAS!v(EZfLIOc|@=J_9XFi280BJOC<5^>_ZCpn!4a^O5THO#Hq)CLfNjN`d8BC z1ww3ds5lf=X3Zk2w3d4XARkqxnMGqENsni}vy{=#IBA!}QVz7?M^l?Rjv#V+Zl(*=cmyk!}g*oQoTx!FAlq%S* z92QUnGed_@F_8)*!2$`^BQ1nYhcUlpTr4z&X`|_3E~9>~oMuy}h40h0Y~#CFNGdpv zSuf{ioB3NYKG?lm>@ZyujU?oyWa*~@1*myqdI>Vl`LR{Nq7LHNR8ZwN)5)?;mKZRX z#$TdN&SPbyV`-hKwx7<;WE8eKx&c~w>?U$e1)JwQL6R`*G&LoQgA40a{kLC-4$YE9^LoYj=Y793Vk)KqTsX%3%&{IS==Y^* zUn?O*uxpd${Qs@hd3NjDnncfc?w9}qL>xhOpFix~&>3F2!}YIj%-=3>oGGD%Lu_^Z zS7lGuL0$$MS#=)1-f8 zs~DnUqOe7DQm^Qshv z%fB50v8P3^zn_k{T_9=Zcc8(f?RcidZVAae0XAtKD2t-3Uq6PHFMjY~tv7+ecdEoK zar58KjR@g?3ir92L{iuXOsXb*%%sq(x`_?0SNQx*-%N}>uyc$RCm6p&9k!=v292*q zTtudp9M{g~U4*J#x*}k=ke8Y@m*3WG0wHVxFdY=oZYF)SMdtt%sAD-L=rOAxHDkfC zA|Ffs`7eS47NaeUfb?D*lE-*bN+(psI-YC~u}SAxRsO2p2-mds>n$suk( zJTbv&$@;VUm0+lT04 z9N@)bSMXXwJtuU-{bgP$lcN+MuI~K+#ftc+QpS(vK@)o43gLscJNcPRTSaY-dP#yo=vEN%rv++E%(@{$Z4pfuKisg_g>s6`Ywo^m2@v&35>>X(Cj`N~*F*JkUW$^G zTmDc0pOa3Rn2O+(p2?$QCEV&?lYVZuLUD?p)JKH4PTad)4YU-#R>$MpW~q&vW%5JX zJUc6*e}^BhVmYCi)JD0N;3W1SN(TStd zS-CoZ3F}-%%|){u@#1{i3@fCwin_eBd#-TT7}(i41(iY$MF?2jh7;L;sM3a?E(+T= zw6?ycg$@gf4@b_@ek{n0Myac}4zzb_5ByQDBScAlf___|#DEc$&QgKv+^=cFRPrYv zG{_&B<30gf#(!jRuxmbwHp&4c%eujUh3O~DN?29+Zp2Z5jqYY-(27V$$16Y0(5r&16sA`GbQbX!Itv1fkhpuc!YHkawKqpW7iKTM zIn{_SvV4l(KnSR{Bof)a#)z(D#gexaP#Z?dk)i>2J9k|#p&!sVwil)qwnz4XwNAj+ zo?a!{*4a2ziZdHRZL>G);(aqKdnbZkk9nK_H&QD|I2-14Qs%ooXn&{|fMyKZ?=(p$ zEShz!s6f+stTDUn9-vNw6lcnX?l*HdJSFU1Y&)WUAuVgOh%>NjvbZJ2C*1)O0kjkF zW?XFynbQ9$v&tc#{n6^y<(nuj#_e#IZ^zY5U%ZqKP_U?V5EWa?DH)8Mv{;~FC>pz@ zk38aTa3;adhiJOUo*~U6w}pf*r>dD$J+F`ZQl|7L$dP^Ejnf0Hw4mqxlI@aGn%94M zKRKahw{wQZ;+sh3Au`V|KDnt@>Wz!>rOIchjgD;PmzMoNCMUg6z$|JOtdHEOSyuB6 zUEalNg0HgBg`cmK5}#PkrEjlu@O#^n^*#P3gip2fTSRu_oWYw&d$&N`mU)(mrnEzp zq*WnR!0=45r+ML5;IM%Hb!*)9aY{?W*Zty#GF%oReq(b}f3jj;i=t7l(gW|8$>62;T8DZgN48T+SP` zOGTzNU?BU*P)Cq|s&-kwDS^RTPz9LKoFy{E(0GDc>vE^?fT~YLdp%in9zFTBJ6K9` zT}^3W0n@yX38jPfh_+d4vNM3bR&xy_l&EA+PvD2;RVn_*YB_~5OhHlc?r_;_rq~an zWLlx51m^TwAgcLY4B6i%IwUl6mx8+da>$ynen1nj#g#CLw0HI=T>8LLqX2UhBN{@F=sC!sf@ zx&gXcBdzAJS5{)uKW7SPoQqWSus(TiGsj+!<)aBStHhMm?u*R*IhMn5RFc=p%bsI%DN5*MVXz z^t)nFw(@Cc_^_Mp6Lqc)TqN_P`Xai_{hB+QZk(Gwq+z4a2EtfgGns&mEQ}zifG1b(CPadKYUAe20m%??zh%T`&Dc z;-WJD$DR(yg5iG}<(TS@uhqY}N?y197A2?m_gFbb0AfUl>ZR#JEekJ~`%6K&WHk!)>`TK(bW)au@ZHdW z#I6uIjg3MJ84ajaN~rFSk%81u>hO=?{G2gVLVoRGPkLcuGk??*i3r<$g8{+fH|NuO#`;av=vo^5JS~cm5XkAOsO- z4zIFZH7L@%d{buIK$_`h7y45j)Qf!5Q0e=(!&Ai42Xj8P_o2E3SeLAHTLDlCK5%X9 zr|{P=G*zrYrr^-RR!)z8J$?`r()e4Uq_=gZS`X>k`SY;&Y}{76F__+Id3zG1qPjhQ zS+Q0yloLbxEHF0I7xcTPzu1fd6$ky>2BLh4lwuUx(Ta9LeAb03)DJZ+b)OVJfo3kd z?9$S;pgae~-A$e6MCYC2ylGxoec+k7Z4_F;0RZqfkcK*Ry!zEhrRu z#Ng8l)o2|>2a8%MqXp>WmT?eU;`GS(C%5?!EU>5vQtf=X_@C8%_Wzz!ch@P^bZT)4 z2;DW@!SH(oTu0{7(MWoS()pP9P3*~cYuk)M7y^WiO2d*b;5V3bdj8j-1ys?zo2+nw z$h2@>5^-Tr=Y$Zj_X0*|y%19wghO7TL4sobRdso!-$%rdUmKdQ*u+#IBBB!fbGK)0 zklg2X=DnH?qO*qrU?0p48U{Dr*}}Nx|E^jo!$I=Ep{v|3y)H95Y|&qIf;J59z2C%O z1L?ZVCUrD!lwqITXc;JmzF!^`PS8vfjT6VE`V*Bc`Jp zR$2?M)MTR8rf|Z@3q&D}4#V2OTrMw~RhPVuP*jHOXAdw4D9*@AbhO4oWu40h(2?Ol}S;W2m4Tp`_B0364-m<+)cHL$li;*S;hhHfpx9zQ zPDUrCxRwlmJ(DldH#$*x_3rxiP=)&sl~-^W0QQo zE(Z`K(|jW$ohBP&fP7+>Bl2UO)khK}wHY8ATs+cl*WiTfk@3eM-pe2Od}F+Dfh14@ z_$>7-Gtbppf5JK}cAJADq=K{=p}GgF2g-eddY|qF8581j2!j^O6k6U;9VAla^fpfC z{4zduS3S|P{z!L5b!j;Uk8!#cfU+Y=s5%jgCPzfj5*%aT!r#bW7Qu&u6C#nfVe-AG zznn~|umhSCk>2>+t_v}h?ii6rka%B^F4GN* zMRr?KxOq|Zi#EuNJA%<3)4cTC0K}vm%7`FINtqew+fD6R^v0iKJ$$h1<)t{#ZY7Lu z!2E80y=PM^62h~lsOqvqNdkrLbW}0h2HZR{61$zDo3xVstI>Jydf4L3;GBl zEXK%Na_0=Dkz4bGr5Dp-&0YAZ-gfHPe>)35QNb=PG~jBX|J2a{i*PzI!^k?&uat+n zHUAK;r-PS7raWmy^Ej?_Xc$Zrr5=G`J-76%EYPGlDd;yp7a$IbZ+FfV%hO6O&v&6& z^-%Z0BIR^1U>>wYMEZhp3rUJsg=q8{u39zy=ZO5rpi$vV?=>@~YX~2ghXXw7Ygz{W zoLm@$s&U*}=jEmy??vEWs+^2GDt_?3UN~lUT4IL)uJ2a4+r)@G^ckz(T5GK1DQ%gu*x|C$oqh(eSUrWzG+#C^Me z@ZikOw!%=GT191?sstSfA|zZhWM-Jr?%3kPx8-EP)ynwHGjez-x}#!)>e*`_cQ9LS z6Q~ehi&LXt+6O6MCQE?n1lR|v=xT4a%tNf>!5wAZrBl2vRYpeF=gZ%r?fc1L3jEQ= z{rd7^b8PeXih;+^Kk_vP8x*!p&#)d#5+mdgvfGjCvU1GwLk0OWm^(>}3)2ZF=Ct=;7xQrgS#?oQ!wH-J=TwPpE zU-?VgL%~-cE)o_DUidwA_Q+FBH8y`foO>@B?RV!m8bl^F6%s5~{`T+UPwCN51+dkj z`_vso~FH!5>dJX-%{JTcY-rx*E z7S(MjP2dPySWTSsAkqs3`D}4%x;?WoMbFId$i1rHGcI?=#xm+8^)7fq?6vcD{rm06 z-@kj6c5_-|$cP#H1$nwbr+an3b*#ey68S~bsxCpqs2ShIW^k1aeu_A%lYYBemZBzy zTgSrOrBw%mY2QEsxQROqKv+~j)d1zC(6+_z$DOPaa_hYtVtTiUKJ_Ck@auY_=pfu( z`ne|hoIfHgbw%1@_?-r+Vr;TYVzwMtq#Wt$!N|(YyIZ1?iXryMGa|-tF@_T$8+dQ_ zg*XJc{+qbexe|t{;PtcSa7fru&Q3onO~uin@G&yn4d?oe1 zyK8>_NonuEAw+{M5C}FErb^8(pg;Ap{F`+iP741mCd|GTCsd{QEY2~qR z|MN9LdPrg;x5l-X6~{%Ef9r0h(Qx_cWFr!)rS=Lr7>_i4EK8}zO??K+ZCNDppegYD^I@1l z<851mWUH3y;1L5u)CE&u1A zuJ`tv^ilp#VVcrrVp9zFzD*^-^{fD{K532O%hN=NOI=~jrXzhdf-XJzTOC?fl#T~J z^J4YR%s@4E?r$pCK8qo%_S?rV@*+Y>wI4A5;_+<^@4Q+sX;C&2#mc63?#Tlh3_&`tQS3%y3mY=|3|X z`Hi0E>q>FO?P$py>w(mOVJmt^L?$h_h_q@S14!qaCG(^i1OmcI!kcGXLh)iLz`02% z^o0_3)N-9sM1e!OjeNlH`iW*v4mV3OK~K{kl<==xn1DjCt7e7;4m3k z{3-bDXpmn@2@S*jhV_dgYIVV%A!?JF1O$eLqUeofConAt4i7bE)&H(xtH5q7^j5b1 zNH&rSewr^KYC|R^RhRig;%+cps4wqXF)056L_5-Ot{*m@2(`pnDJ&#kk_>u?O>PoC zuF@0ZP=KAG`=+!@b*$0<%byR^PlyuDaf+r); zx0AHYAmsc5jYbtw10C$B&+s&QU6CJOpd78jgEsBujd!%3A!q}ZG8W+e{y=>* zl>r9%Ytot3Tl#dATJe`y0oajCPASbKXeV=&nd8|R*|YEHeC$AhIjgUGt6v7?xxExg zens{2D~;b*+hz51MyMndR^gBJ{m9KYfLYpWf)5Kw3=OqJ7}2q?AQ5-yonCK;aHcLj zGl~UIkD2ksa%4DQT(-xDBQXPnQ`N3pEi+b_72I9H>cYUup75n=sex*X$!*S~$b#_b z4>JjULTLL#*0cpalu5)7*~yAw{l)rX$TsS{zT3>8NRGBN42`Ai=sOYDHRZ_n4}kw& zd8#;4_8}Ds3e_;c&B_z=Zr_;T?S4c$Je)}Y$IEb;Y=lARFI?#~W2FcFBQp182nz={ z>Y1+&=Fh^Xb#!B$d|eUO@fDVYEQ90OtI+_;bofSm`ijzlpmH3MgKw`c7z<2h={$QJ z;8uIa<^y44;~jCZcmX4dPclaHWyZEf@T0kcI<`J0yZ{-YZ%d7;@CX47A32ghHGOaJ zG_+tkHq^6+eX^!^@Vi62nPJV{z{P27KPe!98QD4vIWi<}BfGGS54XBx3Wn})wP~oI zOyrsf>~aQU=Y?*GISHM_dIC;^n>X1--}@J;t?H!9gjk9IyxYTJDpdCF@8a)PxuR}% zhv9@+x8`fWa(LZ9G+R6gAu7PGtA3Ue%(AZJ)a`gBuL*c|*pg(UyzD^yM4lYfE<}~@ zR#I1ig3pi@&y<|?i;8QbvVTz2mjm!nQ1ERbvX3v2U3|?;obRp61`Cy zgDt$~5Am#tuSe9m50|vsL|Yag!DP;^iWE(|6vZ&nq%Z)PnYEGu%e&2^47y!hqHVL% zQH3{lP8Y0k(f6W3QU{oX(f6>&zAU_95c*ZpU+AFrB(HHM1SSLLR~0Z`a4N^3T8v0X zB4eK{C?maVGv97^sM}P^Wv#p0}X`3lt{oQRw z*2OyW^ko`4gqPxWtybsPH7xdg7`!xMXhEP0;c9KcUVCaYkuv*8!ngnzhvV*?Q-dNW z8sEWp)XK1w#H<)JYi;<$gHwoqE}Lwb)FdOXc1f9|qVi6g3q^5l6V!p>IZGU#R<99>~!Y^?`PH6bK&wSjuEkWjjpI zp``2?tSd8rNg1pe(6O(v3f6J5uQ)*>IDt}wGFk{mt|2S))d*`&3{E7OF;nXyfE6%C zM-T%e6>zH5g`}9-EAiN!Mt!W5G6tYtJKDpKZR-LC%9DP+k$v-O5x7p$0mC3`!fF#E zJ%74GWR|e*>WOFG(7k!6pRND(D!V zx1|843(Wv^2>sPTEeb%75qa_FyJpPDWR*iUKpkf@fk0W=pP-haKPbORSXgHRuu5mD z#c8}bfvtKkCaX3Tr0Xz5SWrFg&8oB5x;Uv_oXw`a(c;M3-pJNQiZj{^$A6^NC+3M}+vOH&PIPpPoL*@zXnPr~`ZW$(gkGdR zx~1qRq~+kVC|~>Yxd_eFtb=%QFSt{=Lbwk5nRbxTCq=ASWW;AV zQwOwl*_UEK+Eqhku-|Lwmem~HX5^h^D7VamYDKH6VgSPl`9q6g_cP>5&WLGVbEvKn;S|COO?HwxfZBTa+o$YOpA6hy6;(oZ^8A8JT-ZWMaQR`-| ztTcKva)oe@@8>o{w%b|g|Bo!22vBwh61*o|#_P>XaY#??b{a>J5ykF-Iwp=xlUmO$ zEbUAQD+wY&V?$%TD~=B5on!k6vehLqMgVW>^SR+*XQc}nVq6Z)Jm*EAOmdrtfqUK& zGN4vHvF4*Pf>2I!=^@3i)cec*;PAeikAX4pW9p##8sl-CT3kG;US=3evmxw%HzWF* zgg`+jd!}k(-G8ovu(VCeNN9aG4Who*3S@Rc&Ip<0q?pnPT95rf(W+lK(3^lhE_t}G zwPePD!p87zqTu^X2EE|)Zed;1?S%A@nwnk%b~!3u&Wor{#N<@J*=FmufJEqy{+>phU`&3!<1$981eX$LsRhQ zle$W?*m7n=N_rblz5JF=7X|Dm9^paltC4=EIeo*A8Y%YN6?RKNLEpffOH{r(>-++$ zA83$#{{5rOuAQ)vxF7QdFJD8QqA-N#TIg!sNF!SR=`0ry$UK6U zIIbl`M5V7sY4KlvJ$l-0jPRfHYHr^fZ%>}R=Zv50$eW9ZPC8GUQ3}SiYd;9E+62a; zZ|5HLcO&U*i}Xo!TCP0($Lq1ZiHGnic>DLP@Sk2+as5j?%q@R8W;#Sg+*NB{zu;h` zAT0%u8_gR$_3sec&027TvK*KA1O~Y`OT)+BoQ%9TLSU>(THftjVlr5esuRrlI8Rg} z#a!gAjv4Qem$$9bDC;k4+2D#wY8JXrVm>%HiLtwLrgDlzI6^fgAzNjZmV=(ce*M8w zKSSC#j^;ACN&9d6op!&&DmaahE2hLGk^Hkgs;^1<+vUWsE2;pbL)>@jjlg!36bF{R#;Ul_J+YvLB|k08mv;D% zqawLcX9~xBKEuCfEcWc(rKB4%9O^~KGD>}0*+l?eb{u*C_#W5kTMP5Qo4X&Y{2CJ5 z;kA9qQo|w_38HDvptsu*Weg!Ip)2EvWNTvTULBJD-%gXNO`?KiXhYE#VtlCg!sC7c z%TVv{6qInLl;vX5x3uGfikV?ei^Eaw6G%>7zt)R-WNa}R%#}!XM6D zqqJt09%y)Aq#_z+!D=c&y)jUBUjc8vO^flr%}D?BZsLn<8eK@PuM$#7+CTjLEVEvz z)bpQ*R|B0Ucl>wRe@^zkcMi{M>kvi2Q1`j9#}WEtkbX^2JYA7Vt7{bA1Da?)6tySCy7sTG^k$}!_uqdDt9l{5PJn1; zY##*5N9uNqf~n0r4tb_v+p7+B+ zQcB909uYF1doeP~U(Dv!AAo2u9AC${WTKiC(2MUQRFphm(I3+J5CiqjBDU5~`-NPT zl-1sskQ=$Y9AAid`?#Y0n}iW_DgXSw8sjPTxa>N4qe}Rqab%S)^u?$J<)2}VXJ1{N zJGPO*N!Z3Z;djBdfFJ{dAHzb>{;JLFW20;wCKrdtCh1yyDi2V;{Q@ooZXQAdpw}%{ z1-T6Z%5MIg$a~G`aB|B0GnfeoK+umR+RqY>(KsJiGn?R5FBs;xpHm)0V!rQXv93zm zJoL1*0==u|V0O1zHaulO!ee$3{XMmVgY*SlvNGMa6}6)O0Yu()D4YrlqladBPlg>rl5u`>0QGcpkoazYlpJrLAjR(V7u`4*hTXG|!ynow5s%GXrFUmUNndUClsrGt z@X1gut@Y+z*%~Okc`%NAaWH{;ZUcR7pCrc~P7&jGh7gb4a{`Qep@O?VEi75(; zF}Of9GF3lnai<0;PyYS^b^Vim zX52SDgv3+JEvSc^z4#Z#`r?bGx}pnfZRp#E`r=E|yqIr>iLbs`Zi3ynkJHob6YQM5 zaaQVDH}279AL5?2ChJFKdFJWmwvwBDq}QL$b5k!m2I%pJV~muw9^$j5cI_W zofD89ldQsnacbQ5FzzMjvqx)PB{xlgCABq)r$K)w?+i0jS9>XM55~)|+k?WK&3;P6 zQfuKQeNE0!`kL%BvWn!NCxtQJA54-{c0oPukD;Dz4I&@z0JdH0DZF8AE4ZSqPx)C@ z75C#(TiS)g8FX|=$7i?}*u}eJxOdR!T38#4E^Ty|-rj~hvKw^|^vczpUd(IPI63|R z)aGtK=E+u1=^blx?uBjOCvs`>_uIqxh=Vx+a(9MXU{i6iP_}f*u`%m_Cl^GiUn?cK zr!|$CKgmV$-|Fl0FScQ$PJ#No;{=?xH;j9ut;jhuL5=^k9~b>;4>tPKAy&$FmUjH} z^?u?TLqpN8hf}0j+XOzsIR$<8OX=NtQRG+hvg99iwK?Y}dC{kUCTDnLJlx(eHQqK( zO<3VitnCF?!TtQ;8fO%4Pe>SscEXFz5n8fKUQw|r!}>4-z|z_zgCoI zo~cJZ|FS0M?mxGC(QnP|1(&CJai5RT<3Al|Bz?9Z%=~7Y68ZTc=H+J-;`p!8Z_oZs zCQQ9`0BYLUg1e`w$vC4dPdO!%rhGfYN%(St67|nHVdj}mQsUXR;@4kn4boyKdAUCW z=jq(x|VjZTQ!!a@_OPR$BDo6t@8A+taOX*!7jB{0qyC zMVB=dSwE?y8KTC1QE=n^_O>&aH zT9BsvsIE@^$@o6?{LVBDIr0`y>-9=w;TGWC?UHtQ##2j?f)kzWij zUVSCVxc!ewUcv?694%|JyZF|6C-A-AqFYO?Iah(#UfJl5KBFjq_07g8GRiKe722lB ziF?3<^bLqBYi*c&ruXpQms=6{K)$>sugp9TvgH$qgH}3ul zmzR5p5j*3|bcnMSr3qiJ_aYw}8evxq?*UT}<6i6D=U-P>=AB>u0Ke*nJfp4T{+uA^ zS5;NdN>@{G-qMV|y(r1MNQ=AviM9@K?*KkxoSpF9 z2sz?ki@f-+bXD1BOdkrb?2nf|@hEVq3$;m?+Hj9QQPmY)+a4*WSURYYbJDzDhsg;) z4ighk&9Y*@8X-OZ_j+&9v-QE;>wBY^D4(jl^4Q21taoN!n&rIsbYAfG3u{yAPuBXh z)0W2kU#xA2Tl1B9x2DB~*BrAVf=|vxIcM=vyS=ann;o!QYaQ@w=9XN*<)yDp9hi(s zamE#0W6>?If}90d6l0PH8I=zjMx0vzPl|w@po@Hw|^6_y#aCF z+3ZI@|JNid;htlPUt(&*JeU$Z|NE5S>EEY$Z~rw;jr`175Bq(lD*s|}?Dc;kAD;c& z{wO_diktG)80GCJbDXG8Rno*SCm9i+k5ONLx!PHDM+kfRIr`;qpQt79=wr2*wKtA< zywX|to31ANrzuv_7u}dQpDc*de;D9poX&gx+kY<#vVPy_$G)8}%Q&Yj&$_HGPe0Q` zc>OOhT>sNn^4otcb);VC6{KA5rp5fcJw%Aw>?ZRr{FQ(7Q{yh#MyZK%@!QW;<fi{5^}-4%ah zqa*Fn-XtN>(wl$9Hc3ghfox#}c~bu#c6o@B@L3-r>K`-g#IJ0_$j7!BT;zCp+^^Kc zi~n5fp{HsZ(D!FWso%|sV!qw#E_t-si+W^gD!6EB$iKMUUUJ*m47+D)E_vjfm5}$x z1qBFa%w`YlmUA8(wK;}-QC9NgOG3icPmEo#*V}-j4K;b^Ep^#vO?4@!CmFB4JQyRz zIA-{$y$N>S;j93)HzUTc4d9-EjCx&JmG#pCKlTfKW!%>*tr@=@ERs^j%M*UoHxlA^ z2j8<-yZNd5hN6p$6&YtH1=+t$i(nV}sL7X>TX0$4`AWf37cELvpZb%!KJm1nG2^VY zvG}+34&sx=s={9@i=Y3i74{mi3M*e#m3K~Ck#)xWKL1ylFz(-m%8Z}Z+KR8Q^`ag~ zOJ9D*O1%GXeGMw%SR>|a50^YxZiio6ZASw>E4ezyOZiSyRd}I?mHPd=>!<%~h!*?X zQZFH`9|6`Q$87y3m-$(t`5QIWuZX~~+J{ENW1PA>!IvbGdl zQdFk?(2aTU=``cnsiltGTYb!kpLYB4sYkl9YWp-LZ3}o4$UIM-Q^@D;S>(&Tv7)=` zy8O!vwdiM)lDwO{Bdq+j9>R;2Hu$aWUex350nAe~V6<6&(y65ni06C5w4_CT%vb9j zh!HUD6&~pO$7yq@uF)#uz=}-}AzR)2l6oS3y6&UhBcXJ{%_{ z*vAMln)=L(rq+Uc`{TmW)oy0=YB%=bN;mxON@oe+c;s_gRo3mLPUP!$M$)hCl=$ z0(j2ami>#lG2yi7ed76*Hjpu#O`xS6s-kZ)ku z_lBscj#+%#{$$CU)t;j3y7#GPweK^|+h$nB`d-WnLwm{ng9%pl?hrL;y|d{0c3=Lr z{gL9kdm!5yI!m7Jj0i~f=~~HJpCD6LkG(m;OF!L(jrf{aq*e z;r|SgUVpBt$oxrNn+Lc*`?R$M_1H1NM|c+b=#3u4y@OFk*6t`P$J_+FqAAb%agLw< z{Uj^x+i^z5`K3nOyRAWKslJKyLS0pSTV0lOPA*CQMpK^l-A)he?!h!YUDw~zL)^5bTwNXmS-aNb1+K+v@D9-#&DarnE=>z(Xye$1kb!Fz|tpP@oz7ctA zr4{>dXMpr}X8`wNyAStrcaUD>o?{bi^Xw8y$>UQXSG_zome(IHQge1d<~^Ju#@Q$F z5sn!`lxvofd@#;P*EFJ^%}Vorw@(VF^6I=x1EgpFlu6!vx!#+0eoFl2-wILksf{+w zlX>8S!sM%;ZFFKY0%{@Et6=0i7YOnD;d7O>MHklx zDUsUx{PXktq_3vf@4o0DKKQbO{N!6@P10FoW9|iW4=J${_Tt-NR_ZO+jDX@8r>6jo zO+6S#MQrq@-B|0$xvHy4{(iZ&==R0{FTEBW^WAK9!9&{&HFakM{apJ#_u6uM_RY)DFDt7H6(o1hQ# z?6b_={V8U)eU4YU3ZCU^N8y$2fztaxpDr1ii*9IIif$`f3f`;?<1!N%6qrXwO4}JLc|6Q~^Z9aH=_~gF8@@k=f4kX-yuVPE^v$9;_0)nm z_4`Rd)cKA6;=h^}2 zU4GWnf3@NhKtBl>P6a7%tuOt`)(|CrZKFcjG>M4#5rFrJ)9Ghc1Q3LcId3_VBsC-V$loR zbjgz~po<=*l(PkLud$W+#`vD}#PR`qUtgPd37T`+>_I-%Hx~S6=|J4G_R*pzYl?34 z65~HNRKsuV^phiZhe#2d1DNNly@mHT2lByqW?wSarJvsEBPVT*2#SG^UtSQWom&)V zoKuL?f7tE9ympMS^DQ0NH|jdr-Q6J(Ue!rKuEs!aQ{@aMAMWl=Gv^ zH~%m+Wn5aQ%Dp+pPrhJo&b#iOW0h(_P%kfe@$GaOEX6w~Wi2-sTr$)p{V*?l^W_-( z>DQ~>xz`RSkgpFXsA)@m#7HUP+1E2Qc@g#nW|68s@3($>#1B*AxL@^6nU{?pQZF8i zW8bM;3LlLK@$s-tVhJB@b8oOYZNDVIsDN$noI0{I=Xh zOLNI8#oH6iOj9%b)>3=Pok?NB&%?BcFL%b#Zw@tLp0bOS3y-|=wZ5z5olDMy?~jvW zH-~XAHis}TEFeoy2_k;zr@j2n0kZi_ZSl)KQq=cby}0OuNhxu6vW&Dl$t!d!7;v|S zhB#DGv-jnUe4AWA?4m{fpsmfg;hN>*4rYV|*Bl4#0q-BwQ1Wfl@E1EHMUO)U9`9tO zvd^iY7H$j|-1TXyn|$huO826Ov^Pdg*aG?pJT7Wwf*jXGdHubvtn{8^umW!z;TIlG zGP6B0dcnp3?9N(0;^Ed9Hyb*;uro``UhRS3+3732Z)wi?eZ4>Px@!@aXrHEKx|KZa z_9Q!Nb%33)HYm21yXtaI|Jj5GSCqMHX(^i1HBIrEh%=cWX4 zr)*O+*zPDZYpomcNLQbB*4UK$`|2PidVPqMxj#`x-X3CR8kFE^K)S`^?;fcx01-UeSq# zOC!{@H&;&mpDJ|JQ@f(7adTQow04xK+|=Lnjfl6Z55yP3qKI#+3toK{ zRF?7W(?p=<@Ml{iu!ok;!ke3u=$Hc;19c$dV|S-`g*y}ce5ei_GXex);fDu9#gBFe z3-1~m5>BgYqJOlFk?dhGR|u{F;y49w*0bPZMLeXG!rkB^!C5 zuW8?s)i-U;h%lR@^wi}Z)U%C#T$F8wSqyy!^5!u1)oLH&(OM5S!Z|NyYHCZa8EOkI zSR0D(Y;=>d-7*o&HqS*GyD(An6=~NAuBtKrl2=t(vkpAf(uR2`lV+XS>Mnh{JB-UXoaa*<3K7jV!z}{+ zUf`JK6t8y`-&qjHemTjF{(4dn^Rv1>_3Hj)X)N>^Z&2ND4AavOrfYb6GoW9`m?_ql z(rcrXcc0FSqrYD2%Db{XOpb>3;vbCj;M;x7#EouNDq!>IrGD58O>@>AbxX#Ll?hz* z@+h^?+=DIf$jckt`s$V)s7L0doXr_#zC+Dt`Aw2aw^l5)4v}Jn@RvV~)E1^Y&1D}p zfu5{(1FauH$8C-yW40y;S$orz0-s*Ux5=4Ebz8yjJ2Qm1tzlBa;arVqce0AHF(k}2 ze84{hsCagYj`~^7Cts2CDc8nmm@6HKyzkqJ^S&I#!G5q-vLiM-I0?pP^n;}V%rjFD z;_lWs9X#hM;ofvPb!|ifxAwB*&Fz$DrY72>w8Pm5``h2{a^JcwGke~zCI!HsG??J(T(1~g9_6MXTE8VC^%iYCyYzxE;r-H?F z>uY;m`r0m=TufIr7TnZ*fZYLkA<;G`Kp2`(H;i=!zpQt_ZtV0TUmgsRG92Ta{M|lD zzNVVsaOzvOUH zMD{F-DfVf8=~_SG)oN?Wb^9nK)-xwSZ;uJ#O~{l_JJIQ1>ndmwuA$~y*QAuX)?NC* z+E#dfXB40B(N?#*wbe~7b-8qZh6CT5VdT1$63L;QNA{?B#GQ$fS8M%6k4-K4SC+bA zw^n+Iv9++6Q_YC9pXFlG_04W>k!z|(x<6EgU2Y=3-0Gl59gb8Hm)d!7MMcq-t$t$6 z&M+~?KFca}EOOw7v&gvFve!S3@m_sp?L<9wEYy@kJ+l7)arD)1ZKU1bsqMDAb#JM= zmwF47;_mJqcXuPf-Q5Fm7u+EUL`krAcb|Q}*ZYsWljn!I5^|B5d+z(3k7Q=f0SwY? z3-rx$2R?2C(0Cg|$RDfy*pQ`mT)2{1^i)YNdSa@>gcuv){?jdaPYn&}PiNb6-tW$` z1V5MSrN0)d2h1XL#!PQT(3YCdbWb;lcIT@)cIT*=)iH3mvM%$1u`TPtwgeu%Kf~tk z%?RqZrdVa`efapl^z~gowGG{e`kE%&WLe~NOXP#O?$pOy3V8JH3=Ol~hYVimVI)bb z2p_u%rI%V!xo3E}3BMyFzaJ|N_;MumwLj?VgL|O6SI&UWojeZmJM$ap?BUVI@{C*ckc1mD9{AN-A0@#$K`UPB z!iFt%VuIEOm}%3UtavdW|Ee1gz1NDYxKdS6bc&jobQ~5MaIe??ydMF+(+exS)=-pwz9{6=(dRd>g6>{82MT$3=O`s9 z^v@x3{ylXaGzj1g!p;mC78$X?&|cv$tIm14K1L|n(Fiz;L)5ehM%fK99eh{C zg}%^mVV@LC$eX6p*t68+AIISl0VkqfJU$8w`Em*u_4Ons^z&~sweT3XoQ!m+s0gQ? zO&IK{FkBpH@a7;rdpn|7;H- zMBj$|IK?l!-vY@v-Ht}R0Al4lH#_kpDLML&h*yt~UH|iU&^2JYeEKBl{-txEoNupw z8|M^$Kb+w<{Jl))IaSE?`MwN)O>^vvy;%y;ZRA&4xRqFZkq^l^NzTtYo*xkg3VHJ!^yvBp(Czc*Ku@k-0o}WL0rbbo-$1v{ zo&g0vyAR3^@ITUAo^oEz&%C{>A>=yrv(<8#% zh(8UDu(#^EvPYAQq8qA8_>6Y2-(}Fn)4zj`fk2=Ozn=iceH!%M?>Y!r$73RD!R@(r+Nn=L5V^#)k^IdXc^@!Y^KM`GW+1*QAH1LeH;2gSa80eW@gI_Lo~ zA6L$u1O0aF80g-GGrqY1v2r4RQ`SP>tchV^Ry8bcexT^Bp%LH!Ik|9C&Ozz=sPUsh z%=3D9$yt1A+;MPt&=GL(R}eJh8z}7A9nkBWmq9PDUjo9sG8#1NC{F4|&){D7#c$o_87^8*{ugBoLJO z=`|?&z6=}FJ1sWyL18c;KF&(#XtT8ojiW@e>nHr519A2H;&18xnI0$4t3AO zuXJfSc(0yA0{lnCwiutX*pCTS)T7@v!V0dyBg1|}hD9Dt_kRz1b>#}^>7|RHfID|U zp^qK`^L_zz=iGVFv#ZxYm;d||bo}Ts&@JG2iEm$lDCrRw)wSr4D?_}3&52f_T~W>7 zmU0oxqtp~xJ@`pKqu5UXPCbuF4h7~U_(=HkXP_6?E`vVY@&g6ly9c@n?CT%vAB1yk30pAaT|zBNfq+M^Xd7f?nUb33>}`Kb}7YeSCBeShwGSIk^Igdi(N7_^W5f za0%Zpwj+}K#u@of=lcn<`cBL@c{TdAvWEOo*+38Iq?O-pz(B86Au(6rIXR~S-@F0^ zJbMB9^zbq0-lfZ+%fNX)12FUH)yp8iix)v>eme#7JAEGX?e1ex#gDi@8_Tnw=xY(d zOTDC+r9OIsv6q)URz?2QMS=QNmSvt2loy^Bz$(t5($h|azIhFLc>MlYw$M)U~@7xK^|rR7`Xl)Qxjc;IY%;Y+K8 zkh(O%%vJT!VuovIAF7b3>xFSq$AVrz1AV-I8x;HMHR#37o4_2O0-gQsH_*e&exOTd z&V!y@^8-cte>xiV=I!yyytu1#ZK%*~F)4d(1RcIIfDWE*Lw+{4;=c{h5%;>Ns5|`f zf^&JHAx9EFe+0$(`-381zX84Qy9qjT`~>L6AAbPzaSinD{xi_UGiL$3I0p)S_V`G~ z_Ydbr7@4meGD@k($f51ba2Rd_6XQ_R%2&s6vC`VIhmtzX+bU?um4YANkEgu*07`iE z63FH2pi4k5U;6zdkcT&cdA|y}^yeASHDKINZ`=mmxpD*a%yP=)}m&s#@Ha z#XeT5d7_DDlUEDpx(Sh{E>ftr6BpDQ5jq z&VwETxp?l>AOG9`TVM@e-Fg6ecJnqUKP2dARYCl*Zb#BuTRgGX# zT8WhPvl7SKr~w02q}N?^>Kh>h>Q@a$-fTdzo{_ULekIW*7Yd@XPh(Qk&!H3I&QX$* z{5s$;e`PH#b!h+_yVza!W^E7^YEe{DH)KM}(s(s?W~70j8K?#Gp^4{fAUU_jYJ@TE z!kRE9l<*LnU3R%5F6&f5NcxFS&tHKaU%LRr@-@)yi#I^={(&b5$$2+hVd(dodS2>m zCpmIsf|+j-^O5FBK74wBmB)uBU1Ar+{4v2ozL?-qKeRFF&#Um1`=lbo)r7#<;~yTq z0Nn$y)9*ZhZ}(q;f?j_(8u}sNXi7-((SrEglg#pxdqYg<>)AH^ce|pJWSL|ZIHXLN zSIw*1H3({4S|QJ@tY)c)YDxy`8e(}^^zFj*^gq78e}63Gxj!iM-4D?7yU#%v{yYzg zdiVVVG_(9VsSNe3n?e0J&Y=d*_tsFXlKR@UaaMtEvrMtU)+u(y$|%3wG+2jHv~y!e zs)+t|WcVFaQR(HJh=dbKfe}Xo-+Tdb{0ZlM zs-jtCvd$d;&@ zx+Si}StQJowNYB8wi))?*iDXEoUF!6+Su{c=;G^$l&Dj&U*4b02o3%nnih99B{bmF zn+JD6?;buso*A2cIX))-TzX902~u9k1uhi+sEdhtudIiC6mv4J&UTgs?#=M)cIQ}J zKp$prE4f&UrdqHptD=Zoso$${&@0HS%-_S`zCRNB>J2FSM+7K0BJOzPm#-(@Jh%Z$ z2z-AOo}F~63RQl8fJF+K?&m`{#cY^c%|O`oOx}jPsc9V$FsqYwBJS^UubKAtPXTA*M=L3TT|6!n@mWzNO;Ii5v$m)60+=uR=Les)nHMv;2RQV znOVWZEKAvtMHw41H(7<68LWqB+Ua4s4p!X!cn5!Rq*t&s+R2_BXn-#C)s?UJk)yp* z7Q_qWmqSxY+m=!CwnU6_$7EfVL((;BQPkD0N_ePsnUJ<77GM{;>G4b5{H(d&x)Nn~ zW088Ok1OlxA&k{E#*gyrf;%ac=hbBe=bK@$yHk~dl;xoYtXKn0WONGr7N>@pY)NGJLqDC3CE`IkOIr^lM|(eeDypM|Cl{Z5^)77_c>l%A^Of~HXfCu#G$i?OQY==#@R>r6YIOz*hl}M+K$M}1O$NzhpE%2&Iu(eS{*wPR- zbZvx{V-^dEz?za~C+L~RF3i`JK|<8Z1i)QY8Mg-rVP{7f@s~}#j12AYNb^)zOM$43 zAFb>a7S4|~Qq0l@!M1{nTOEW&Dg@D&)`ki`>`t?|2g}Vymw^J_mXmVq8ZL1~$_CGf zD#6m8%GAkLe)LpxO}w~;@k7>ES0NwmU~0x2(KBP@_}O86#Pk3*c(xDz*`eW)&0-E> zt_%8ca|9OjYrST|t#6a;80tFh>gr0fypp&DXoHP$T#Tf))vEjg{krK)e*o~+?I zWjvf&&OxkBaSǘT;4&}|chRO=uw|gE9Wq41;VfThS8=db8JFlXRJYhnbv?`K`qoudOZSSRy?aGg zOP?7eM$8WL@|Hx^hz%K|z+_h%Aw&Xot9DoGr2T&19QYw5?$)#?n1@vBS+N0TFEDYeJQLjnZ8JJn+}8{3$+z@%DUYPA(CbT` zSzmW_f?DG+hbpQo3EGs=u-^G5`L13dbZKaarIE6L32y9pX=T*SZ3!gfa9Pl8o381a zYRn6;NN`|Zt;szTkMGfQ$N}%{Dm;G1s2)c(uFW)y5$7+r|VjO;U}B zQP(0fw#AiXActssMh@B6(`r|T$+C_Y1B#@C3%Y$ZZaq3G%xl0D9L^$hcMbS_ zn+%({qb3#X&amlrIkm#=>jSpTtN-r;oAb{co#0XsG8eiaAB~M*|1A+6?9o?sxTdSx zHk9Oi=Nzxrvs63&b4l3spJlf2aE{4w8%Pz4V>z$fMr@JSC~Vj@3F|$^O5y%ObsrF0 zEDx}r`&t^rDaWU+4duOB>(74T(Ndr*Bcy_bE?Cg^I6l>*ry<=7Z1(PouzPPo&}7w+ zi^Ppd&*q1b74DguuC*z0o^1*ecAz7a9vFQ!n`sEA1Rl5AoB4HPpfJ#*A(dFg)FMM; z_H9j7;-!U-tVb>-D#R0c4MsQ!*pxngH2IU z*#07u;{o_>Tgj-{F;%s=r|Y`6)Etaeic8s^toUZ1Ec>)MQs{3HK_dQM<+s{pOz2!E zJ$tQ#nzN^_Z9XuvDZV~{_6bDrWV)GAX_|+aQ9rhDgyT8`B09N7C9SVGgOGPeN8$(2`^p%GG zniV#C4KZvPLnxZDA?=vt>+3y`$e(&1+pS>1x5bPi zAXW>kB21=TP6oTwT=IsrhAMz0-Ds+a^S6pPj4c@#X;Wa6ZE|>oU5<#hh$@23V+G%~ zq}bFI5jlOP2lK-!=2Ca3>jr#v+P5T3$o3>5#X1HKSnkewx;dKj+OEQ;XN&L=S+0J8ERISBp&D(c-cPDGAp_jO^ET-+cih zY_e4Xjk7Aysapzkl1+t<_Zmp0yEByyK-~3O6kL)^!=)UUSmXl}sccV+PG9Uyxjl$~ zb5hBVc(A7>mAh0N{Q59H-Y)g^x*=n{CPa#P0{nfc9~wB*iwa&Hhli|BKti{q_#Cf^ ziF0d+g>HRiwad^r=AEzV@-Ffl_7{NH8Gemj!@@4~qk^3ZVy>?z>gG5$dRNP1>}YuO z9X%88H3`}em#W8qu5jyIT5^d~O)lA;YcV)Y^?e&M7Gz6C&i`kLLi+C#3HRS6D&_x{ zdCmW~%&D_TX^MGWvW zA8|82U2u(%bN1zxq@Nl#_CSjP@D!e8lN5cmjHEsK+eiWX@YbQBzVt9FbOs8xba zut+fR4mBppqsC^sW$3JpVPyE`Ft2o@mzHH8CC0l&q!il-I&8BK9b}&X-jAW9%>Bi0 zmOC@C#P7qR-N04F5{n)VWaaO)-idpJX(qUE!Iop?e`-aAW zLqON;>e+x-B&3=raS3Y^s4&YE8Hifd9^~WTggH?#F&(|e(+bPlv-#R!^aQc z;xDXpRD}L#v1aI>4dF1L(@+jTivwJjzBB*{nCUF?_n2sC*SxTKT}Cf6j}cP-vnXu; zXGz%p?+U-=Kdampj~(WOvsL!R+)?W0? z4#c8kf|$D85B=VT3p>kBcy-YM@U3l%lD|Fx{kA-i|9oK}@A*s@fWv)-@BcONxT|7p zUOzYX_R>IE{9jAe{rfWN_K8hIE!mc`&@Lqdx}zi) zY)?YM7JJg4S|&;ZJUTkTAtONn?4rBWq+*90ov@>VNAGH2(JlozY-KR-({gw5C%2r5 zb7>oTwiGl#x1baDRk%!#oLJTT*{Uh9knrv3eh#@-Cr3-#cNv2;ugA!zqsX; ze9IW($7X-o+buxvd8crR`)YFOu9o7{*Q|XbkMR@m3ck80Ta$=LfQR320^WOd3ZHCj z0b*@}hOtPfsEsLNo(<5oyCxRVsU#I{ORyPEIl0t6Mar;EVB!G2j&@0@WxjhtJvsr; zW2|lSObe>45`3aXgiBcMfqYUkV$Zurk;#X;%BmdzlbmWwxlKYV+XKAJj)H~URny^H z5@fPjj7r;s;lg!M6e>gEJF-wcf1*qHauSe5m}+yVK#+?@Z$G6ajZ zO%bvddm&$KGF<-Q0wGQlb<475mczZ}e93jbQ96StM5oawHTkG}p9 z?&->!9YEV3EC^eFE(=<{CPJA@4T)OmFM4BY%X#Y7FeA8T;i^Roaazbinb$g2w9=tSE_CqO5dgzj3iv9w0&yEHjZ=Wpq zWFAO=zSI$QLnHitcDXJ4>5iCEWSwGx=X(*~eD^UQYFOxfHIHVO5VJN%kii>+@NYn@ zM6dOgy%kqwKgXx}|2c>Yx$Xfx_}UOUe7U{g#d>GS*u3kjOEp_r zyFWw6`TGBEO+kaVMDP&181loTM8(+w9^D!)|7aV91=uHPd3*osT~DoW$uTJ#gOGsb zeniN0H$1R|obq~Ygom5$;r>(*XVTUEfylVsiqS;?Ob>b|t;SV_?$`7Aotv6{vWp5|OYyg^Y8^ z@!1|Dhr2Y42y?4x=!5wd&HiFlk7tI4+g2bGHYZRKYlHC6)qYrrT~5lI?gD={whW)dLs;<_lWFy>ia;{g)MDHjG>2@(9a;Yolg@zwL-9mH8TT2flV6zMSVWR`c zho-#e=KhjTHaRw>rzY(MH1);rgUGmh2d0{Khk}c>$(Ud#z)$NVu#lDhvd`;7*BZS0-*3#EA4LSFw>vEr(hnYF^os=jAEAOVIAMp77 z*eG)!{M%|*`DeVrN!r4|x;BQME_(JQL5If%o`39{0GxHcE#3{uWIVAvR zh{@TDou&SBb$QQs8Q`>Aal60-;J{l0y@bx<7L@;?kA zKyV8z5`qN?5MUP;7AJTJ?gV#t2`&MG69^F8-Q8t@;O_1OcW3e6-utb4>%DoZ-uYwd zY*){;oHJcJJ$*hMtz5w!h&c-(;+xwILS5*wJJY0rHAGMe4_UBo9B4ouV!B;g%LuRL zmHlDwv$h2wD)F96#5*6~J)|sB8_Y2c{2=+HoyrZP?xi{+2J#d~cP{e!m{~Djq5j8lt<8*i*hfmb$P{#0pd! zO#MD8QTWre2xFC8<$~aNF8@{IRYfrBQ9=cPm$c5#nK-=wK(sXJILUg@C z%C%CHd%rMAAl35Ovk7qer7m+qPRh9u0$G?mIUb0z=fo?vDnuo$;ts&9mN76WA=qt* z5R!&z;2egPF<$fwYwo!e$6}wYH;N4YK1$>QM25I^=GHd)#QwzXD)OO8&qRLS@`*~g-9tSmvAS9l`AqJ6d z&3rwDut#GGejXx6+d7qR+WcH0`S%F`XjYg;tj;7pmHc`f~`R@1;Uns7~x0<|nGLtitdKvY-uA!W;s2n7w) z)D%sxNf40oNW$stg$Za~KzeLgQaHDO@2(x!P&-@96;eK>e{dY4S=6;vIIR5YkbhsF2nYF}xbd~@oUTQ5 z)ftK2_c$E0==W@5vW%SnWW<&fgxfW%t*@RSFXU+1B1H3Welaq|9p3fDQSEgj#-E{AzL?t zm=PBkR`r-$^evU!W;re}0SC5y&iqkVv#a1;JW))w;9&bA2WEkaIJ2Gk%U?{Ln6R|8 z(X*@03r2O|A{T!zn_rlY8LM~R_Rh*8DHR|gCpo?sBI;7sS7WZ9BU$T^MPlen^ zksHi6GkF1MI&a^J-yc6-7bONG5s;i)riDM$exy1c&2X5N2wO>#_=(4Bfns5g8E0U1 zeZ{>)eMtRwpm;6gh+6th6oyOQdj-?HQ-%B)_}5XIlFJ|mn;C-MSFLs)F9GbU$}p$g z*4^3!NaMR#dloNWCTe~Rs3OY>NYH|m(ws8vi?O@zKy3A72;;SQjiV$O}X z!S3`|bT_fSqVG)FHmyd{t8P~RK%GY(OdFQc4#=+XDb9;2J_>(>Gl^j*4! z^iXbr0Bn!D6Ta*T=3dj1E27q=Hy@sil0VY9O`UG~%0|)jEw}!LAKk7yIwwl#9wh`6ZYufjdzYpM|X~t zx%_(h06vE?A+5=gW$y+sGay;S*y-`I3HSUcp`Kz(yBU{Hck*yVAi1_|D%u-^x7y3` zB|1R%nI{#Yy~+9#wWeEh)yu0FYTk;oE{Q#64sKZ<}8G{R;zvG z3uysCEOk4AKPR%~ERpR2rb!p+%$NE!MFW3qV!eE>7llpT-r*j&CdVq9kubaG6ZNbX zF?svM#hgC`#Ns3E3VOzA_E3@T5417Y6Z*_4$PeCJNe{Kb6D;NCbp569eomVSFmgiB zY^D(br?^6;EL!d+!f7;gp^Q^uKL2)m36+_airyYC^2cdR+M8rNI_SgVC$TR*@(MO2 z0-eQKua7L})$2)%)G^dz{s`%wbx4L-+3uQBs{l znAp79Fv_pC+B6Q3QIUEnVRzU^`=5-TKRlG%%C?HgGR$&rF0V#Uj9>3@E{|x1^yVI%!Xc@nkV>^d<;t}kpI}5+n@{^ z;`dE{gYTIJWxUu63{S|mV7aK5#tplvc@by)n!Ir;`m1F%{Y9p6#@dmv$UgZ2kgccG zkXj8fvKpjyHqRXrj=aq?Hxzh5RUjOlhignLwL8^fz` z0|M+Ve$V5DpSV5O++)ZDQ;;ET9v&$ebo@wB-aR0w<_mwd@bYvb^N-?@qT7xUv|h^@#^4GWrQ`niB}Ll68IgZPcfnVt%Tq=hVc zf`SCcMVhENZKOis8(^*CXKP+(+k~2WR5d9^4mg63a^*zX2zXOqE~&lCMk;}}M-_%zA1#GA z6VSAjcHdICHeFa&bwl4Bu(moqG!E`HI9*knSex{iA&x#aob*8LUG$R6Pfm;%pXm3#8ql)Qwmv(MeIxI2{{_1>SAg@BQ4?j zf!AJnT@MD*v&JUJT+MHdu!M-qB*xGL`9`6EylVk(ru|erDtmscAlhO*efE=WmTA2AplP=%|N+CrfSec zgF)XBD?E7b^OPkC_s+sg&{>vn!maXr)I}B&z)STv3(Sn1)^wyczJ-0}-D)NFYoAox zVRw*ry3;IQ!mWW1Y1-zvVYO-y9@T;2TmnB*WTmB^0gj3KRI544I2O#f_)Q{k@z=!^ z9g>6%(*N>cWl4#l+C?dw{A$PFasRwh$XE@pq&zt2=-)`rMX0~KtV|PoG%h;2L$lIZ zV221=CxIlu(CW2J^c}gRq ztT)*Nxeu{qGYranzCnG`mv*Y>jF48t7d%XEm?Id&HhGeCPqm)LV04(4qR~Q=DWrB6 zmbJsHxWZaPyE+e%M{7o5F|BR;pqI;fF~0wCAto`erZ>5>8_|b#t^K0_hr6`2i6HNz z7n#6q_qykD-;D7+cy=mei*D?|s9>#zldDFu2T@N$kLop{_%Ry^*T zrvjLgDjt`86+x=)3LYGmgR zku%1v0(7*2+H;2=Jk81J=}7K^?T5QA@0W8<4~j)9^y0()mlYgNc{Myo$`F&AX;D3e zAZ%uip=h`r%aMSmtf8db&aPRS>lOKo{&`!+oHr2P;vgl}u z9j&-koNlGCD@T()4CzA*Efpso!yuJOe0fAv%)7%0kL7})prf^2ved2he9ewbzfzJ1ZLp>#tTi-Zhjb;t6G*F93R;fX4dNH+ zy#5H$bJ2s);t*ZO3?S4WKlFKZF+I53z3H_^53M^jI5dqHab3;)8zBdX(Lion8n z^?WlOOD|n#-bkG>Uu@+DYlykz&SHk{NK%rb&K%)pmA=V#7^f&~%j;t^cMX^r=zaQp zTsw*6mZ_IL)6;Kmf>y0X^}(ZQDM)4k5w@r%vRtg+&WWmz2yMz^wCGC2p1Z&3naPU0 zDrjh4rOO=?f+Gn+dGq&_pFwUm;(+N+$x2EL#Ioys; z1RvBc=J;F7AD#E6-({aY(3GG&DWo-1F}H5!2yd{%Q5zw{`pr&nvNU^pPAukQkqE;c zPTmkcuM*;)>tW+)?=#(3%|u2bR{UC@nBi=$`K48_Bmaj(8C{DJyI&w=6bX!PE)gN= z60C0@w;oy^hv!O}QN6lviGDmheP=OD$Txcyear$e6|BDW6D)Db{!Moq;?fM%_0ipD z#Nh$6*$JA45H^cD(fLkSVk5ImV!?Y)w3FCiVO%&i)g82V8B!!Qg!8Xf=7rF*e7yOh zT!Zh0%HkB8^lVf|NCfU5ahu19;HJ+u^-`^3TN#;tq4BlhGDgZzlI=y~R6tnclGKBH z4@<|+v1Rc~7UXoZn7iZZ0PlKPgN+&SD1CVPXnz8iMQ2+6+QQIJu_+rc2$$vULyYm>7(OM-v`rFC6@;I=~iBPEBoC-sO9e*e>b`lJE5^gh#K!&BGF7U zqSm}(^!c}|N4^o+re*(##-{w&W3AAYBXr?FiO<7}haxD$VAIt^o6l{vOPX)Z5_5K=OA z!TvRWHzL$EmlBzdg1MBg?ok&ZXrmKmyJmD)T~5Mgu8vtsA=UGwqN&^Ig1nfYaSOGT zc7P=Dx{<(Pc9-A^D$GtwXk;PBtK+rc1;LMf! zw>E3V&p+H=64uRNJyk8KUfa7NkXCbdGXhjPJ0-s$LWr6%+HB+qebQ~hnoP_>&y!D> z9`Y=Pu*W#IA1dTJ2%dFJ)L4FGZ*0nRmzNmuJm}Tc-RQ-I&$EA1vYH5obk;!HaQ+BB zRC3A_pId%YYGezKNIvcmw_@O~Nw1dK@G&jseQp+|eW>+ib}H0YxJd1`wf+`YX53ap)l^pnfvL0M}&VH`Ieb^V=B?5Z z8PmCYnLMJP(`^Fa)5~NyNP}`-PiQ&LJpMuyZP%iQ{n@7EGL!4{UF-+ap#5&tIldKCO9Tbh z#kOegcH}^Q`cBY zpl}+2(yWM=w0nARxG>E?sl_-dg5int)2sf8hVRfN98LyCuU_T;)B{r!;wF2+x@oOz zL8&W)$fS))Y2&vv(F~Th z<^x;u;boO2d6^qmTD&|xg&Q%UZWlAVz8Hn*6aRt=M-(z+n#)kCb`-)rd|SK;XZB-O)P?aR6TTyv<2$T zFH`m2GUKG3eDcZ0^KPTg_2h4zO1Uwyj*WU({JAxkz+_3&iuE2HrlqTWG2}z^iV#&? z`uI+Cg)HdBuP1=-q!05H=7Z^v!PP{@`(xtao7JDf)Gk*}v4D2zi`Q%`@DJv8jtSA0 ziAI^=E5h`b-Ro-SFVa(pHXpbk+A2;yrLC9W-Cn*O|MuXWhe70EUWi53$T3i;4hJp} z?2v9^C?D`*n?-sqD)9GC*`qSGE-pe=MFzQ8qNi@k1IL=(IgjiD2UnBn*1pydCn!ZQ z)iG7>VG|;^Ky-O)*l7w}cHJ&>snmxCZM!gd{bq1gonBQmQe511Ss)cAuC!bEJa;MV6!FFSb;+Yxq&s z-hOrq!Eugg>?jrqu6gUUt1|fdo%8G`u_IE1Req6tH&ce!^`zQHBxh&u10bsz8HaEBg$|G@jd233uiNx+R#B|h-CeG*YC;dnxfA%IFNJxCRCO~`=&tG^Kpf$+F=2=L1E;Q8V z#wRY@hA}j)ByDj@V(<`O9k@SHaHOC-@Cy^%8gN*ZDy~;x$ z6HS-zg>&xH4vaxAa)6}2UhMEN|FERp5`($X)>Pv&yx0dp?!)gcYZ-;}5^AGp70eJ_ zGS;V;FwGoIhJvsC=I`r6k_hS^ zQ##|1N%i}&Nj?l9tu;%@j&MX8Py?EA(;;_#XdhYfpM8F;+?5X39Ng!o77o?0Pwq|e zI;$;A2upo~!yClmZXB0=j6y&(2xuVif9YrG$2*;P@4aG5aq=W`-TlP8{;L1ih1U74 z&X*mH;`R#>K$+_ovouW=k_li28rv= z4&I@8ziPP&B~!AnVD`>4G2(j^V2Up1=ITq*48^FSPXRm@humq-&0*Cu) zJ~mv^JNgfQ$Iy^|QWm5$5f2R3CKk>MknZwxSfuS2M9G_9^`n3te;y(Qpy} zUMf)=@iC=D!zG%j4$)^ZeFMGD>qUo|AY(uCHy?GEi!aqE7T?np-ITOqT+6DrI}^L1 zq4&H*b^}lhorN&T<~Prq19Eq9(VL?L9|qQmW7M@a>sCzb-u0XeIA2@G2mH~+eecH``frX>IElM=H$;>H=T{qvgfSX`7uAS z%^y9w$j-Tdn=QNomKq>pV%FqWXk7aTLpdlx;4vy{p{!~DX{SanI>0$q}->QF|rjnJ|A0sLaAnr znHeMUS$JO=vQgSrdRzEQ(POy3z^8%j(+6xPF!a3kwGM_wJB7(sZMn%k(`tzs?j8b7 z?@Q<5vq}G}^{?LDk0IC^nPFkluNXTh+F*7W1H$SvpX z-VAH5aaxjwzqd+9O zw+k43=2@BW*knbedA<9MOtUEu$t&fM#cfS5UpFycL=e6!=R3N^h={4wcy-j<#C*Cf zekFqQlNWak&Z`{(FG$;rRKEp(naaj*lbv{_i}dbr79sQYf~)MSpl|sq`WVns!g-wl zyb#HqXD5yvk*c6K>5}T%(H4U#F0TSGrlQj#-P&2vj`~J@eH1X^1t{zpC~n|o5%QU< zi5i5I7PJ&TEzBCGs@!bH){Y9B*j*<1tJkjQ7}QHNT#C^Yx=(iMTSYpM9ALLg@PYew zNKj5w(6PRJ>Xz_|M0ix>%}4aq{U(IazTqACxYA@@fNzU%^RN0 zo*CPS>O>|R^!J-NoF%SqL;N3aO1YD&Os8q(>a;~+u`ePYOp}=JPT16{02vu=vN2JM zEb7Pj8Q_+ezIf`mXjn-4_bKc8=>x2rbT&_p{6`Y*(xnb@>AbF1O_Vd z3kDB!Dzf?LDu8>B5`ZL&M9K>z9}K@l4xiop+4vO@jd~!W{JSHztC4)I;>Ne- z$<6P@>P4lbrVWdEH_JP-fP#j)q|0K*)Pw~Qrdn@RMK9Y*3R2KKn)_iEQF{93FRd?1 zRQTIdFlm$J{0s86t^;0KQdmKGR~~sBn5Nj3M^vWm9evId_q^KUyMk}s409Zn-^X!A zNk8>RJ~hvo2oU3;U;Dzmb~|F_u&1C7b=z7R$|~Pan#^RDPhpPP%v*cUD58(6h{I3w zGVfPi3^3}LcT?6VFaL-_b`!H}&lCV`C;OC~Hyd$3=4<9~N~8dM21TfZtz$uQB-^U1 zmc?P*Ods5wJ+-mq$gP=uHV6opmpD5VzXqLgG=H^v0^nqBm_3o(d7(ke&LwCz&gnI` zvptBj(o`^bY)vYTuU5t%Sp($32g%c`?rBX%!CV#KN)2a?cBn4fBd& zb#k#DUZPQshO^&FYg|;ktUkH93d=to*Zbq~IEaS^C+;H%Cdc@uA#pz5y%)5_Q6j5L z{QxULi8a`!dv3LOk^9Jw1zRH~Z{_WwJd6Z=dQRNAIdsMQejyT3PFUlIIu>pkuNJvK z|1j5s?B*r?X&(2n8+j0;$Wj2dcz(>NMM4rYFj_U_pWntdF{~eWHb2^F%vJ9hug^%h zULGO*G23Q<-M_>mVdL2OL}?GeZh1NNtPLRjVIpfgzh21H6c~(bJbZz~TJ+!}((DsgM9R#yVcG8dDX zGP2$4+)Epfh~XD=iAPCVu{kg&d`2KW=>k4TUsp&K@y`&^ zou^gY)C>3j?m=|Bti;5d zt{y35u=>EQHWJuySh|SJve2tRAXs{t^Ji%NhusouG9I5rajO+)T(ZN%%Zsq8K$tC! z6=MeNeqCwBkPINtwO8cDBRj+4{LNTqkT;#Zq@48|B2LjGp8nb(0d&tV$oF9C_l@Q| zMo-g7w@yMoKHqsLp#4CuQAKQm{YgiQEK5_y4_$pkICSX54$-X_+L-;YHm40BQrBdB zIfmUwi^StK{h9HRj~f4&JF#Iab9-}HUu&xgn-ubU6v`6Raz8w&ah5}v=mml7Kd~x` z&P)r5-jTQ&4x!P5Zo^|R6QUwC)(rKV#=PyjH5_Q*6NVPyu zCf`k|QHqyhP6wfKL`9}uhtfNuyKW>A{dyA%f#NXi|A{kJp&-gJw!PN=u$lD&wd?m? zytu~G@^|moc8^>2rClY0foqbc;8VG_1Yo*1`Oc0O)&Qx_53H${&2QPVE}Y$I+A7&> zGg!kRIzlVjK2pb>&N3?()*k(XqlN{cqrMQywML&E*xNRG804mvFFI8=wMi9aP=9e-7E ziG|u~zc`!^hnhc_kzSkA_r9iOXWtqPISxfV&@kHn78J7h5@s|giAv!L3-@9rtG5=< zvral#LFTx{q02>xEXc55l~9L?;3PK~Q2sh(JKL zlZ{@aMR#IiaQXK>uc}FhGY!Y{nW)PZ0VNK<(}k3-v{Xm`ijF#F8_SWfsmP4KGZG2k zyypEBQ-M2hW`?*DI0o|;!MTMQ2cYot@vyHM35&@TnwI#T2iqc&-t~f!uQ+@^Jn&!- zKWqdUBc`bDnU`bG1AMsHef!-iPfY8}6UV-UwQ}I(FAmyhn49kqe;<`KRJ0*iqghFv zuk5IrY6(>f51q4A@(M13zqT*Ts<`$F`#k5PJfA;&8Vm}new?<2lp4VnfLNd3`*=+E zefGEzbbPZu-g0;%KxDl7omp$z*RW?XG_6U!5K@P@T$TlZlC*CN%zur_JRp)@PeLK0 zQ1-)5&DxkH)f%7$KyC)iB!=&q$*`>Pl-~@K^t~XCxIKB3zjgmErq0?w4<%@Sg{8n! z_f0B4_seS^9Wg(w{XZJzFLFz*`|?PSegIs}$9YjZ|i598%<1}+m3KZJN|I7}kV`3o z&1vb8GJEEnMkBz3biZnh?xm`Lb&3>oW0*I1WgWQs=$6#YoshSI-COxfb0$x&xpQ~I zn!e0lcs!xXq+{CpMP=XcP}_nUz^$tFp&LZNvOdZ|0G(ra-wN^#793{q@Y0eQY5T7ju(+;F(#mzGp9zr!~*%jDBgYXO)aw~);~iqG$GzI zR%>OJVO0Z%SA|?BNmwZVrtmvIjfrk@F{NL$I3E6*Ex^LW9l`!tH7P!bG_HPyuwDr8^o#D*R5cL8;Ji7x%K?}M zX;O(XnK*p+Pj_D#2yfQkq5Uo=>6j(pkvKP+{o-i=ND{yh(U6m6)bi*OILv>EcspsY zyrc7Mp{VFE*)7?;wqdmNg$+Lx5BcO-*tBuzuli=5h8aHo6Se1!-cv0bbir8pKWey-QJL~HTmvI?l0@5G9F65ZzQ3v%R(ai z#J4xg4}SNnNQc=A`eQ%B)GM=uOarDWyQqod+x5?X>iOV$74t3H)opkzszzTV2NerT!NUe6qkh3?HIJ9PJ&6I{ zvaaxB+Dq7*lymDB+Bc&upUA85`#V#=gVOU016Zq5;%q0u8y&Cmc6(oNaX*tbQHeL?YNFDzLGc1X*? zLZ*jSt)bW-n(DuNRmo5+OE<@l5HVd5YBO96x?PlI(H|5p*HxwZ(<^E`+gg$pMryS{ z!cVs|y2mml!2sULLQD6+GPW^tbaF7(xBl12*1!S_1fm2|{%fEK0>Vd%l-&P5bN%bA z{U0Wd|1k0Kv5LU2b~3lM5!H7xelNv*vOpAemRmRrn|NoBi4z@x4f&f71O)#r7Xku- z{+0`;as9=|1!3p-n+AmI%>B1q4j|h2>HZA@$_{~aLVV^K{$5pMV+ zEY1cN=Hw6ubFy;_iL=2UK@K(%VK5s|6wEEaCo0A!0udJli~sAf0dk8Afy977E-p4M kHesL;x2UjyHasdhI_Wz&{W~VX)!_njV$sry$%$kAKe0n9P5=M^ literal 0 HcmV?d00001 From 40c674bb570e5434a9876d9dd31df280155e00ff Mon Sep 17 00:00:00 2001 From: Andrew Lamb Date: Sat, 6 Jan 2024 11:20:29 -0500 Subject: [PATCH 564/572] Add Apache attribution to site footer (#8760) --- docs/source/_templates/layout.html | 21 +++++++++++++++++++++ docs/source/conf.py | 6 +++--- 2 files changed, 24 insertions(+), 3 deletions(-) diff --git a/docs/source/_templates/layout.html b/docs/source/_templates/layout.html index a9d0f30bcf8e..9f7880049856 100644 --- a/docs/source/_templates/layout.html +++ b/docs/source/_templates/layout.html @@ -3,3 +3,24 @@ {# Silence the navbar #} {% block docs_navbar %} {% endblock %} + + +{% block footer %} + +

+
+ {% for footer_item in theme_footer_items %} + + {% endfor %} + +
+
+ +{% endblock %} diff --git a/docs/source/conf.py b/docs/source/conf.py index 3fa6c6091d6f..becece330d1a 100644 --- a/docs/source/conf.py +++ b/docs/source/conf.py @@ -33,9 +33,9 @@ # -- Project information ----------------------------------------------------- -project = 'Arrow DataFusion' -copyright = '2023, Apache Software Foundation' -author = 'Arrow DataFusion Authors' +project = 'Apache Arrow DataFusion' +copyright = '2019-2024, Apache Software Foundation' +author = 'Apache Software Foundation' # -- General configuration --------------------------------------------------- From 428973732b14055971604204bf3dcd605120b3dd Mon Sep 17 00:00:00 2001 From: Jeffrey Vo Date: Sun, 7 Jan 2024 06:29:44 +1100 Subject: [PATCH 565/572] ci: speed up win64 test (#8728) --- .github/workflows/rust.yml | 9 ++++----- 1 file changed, 4 insertions(+), 5 deletions(-) diff --git a/.github/workflows/rust.yml b/.github/workflows/rust.yml index 622521a6fbc7..e2faa75e7bc4 100644 --- a/.github/workflows/rust.yml +++ b/.github/workflows/rust.yml @@ -310,11 +310,10 @@ jobs: cd datafusion-cli cargo test --lib --tests --bins --all-features env: - # do not produce debug symbols to keep memory usage down - # use higher optimization level to overcome Windows rust slowness for tpc-ds - # and speed builds: https://github.com/apache/arrow-datafusion/issues/8696 - # Cargo profile docs https://doc.rust-lang.org/cargo/reference/profiles.html?profile-settings#profile-settings - RUSTFLAGS: "-C debuginfo=0 -C opt-level=1 -C target-feature=+crt-static -C incremental=false -C codegen-units=256" + # Minimize producing debug symbols to keep memory usage down + # Set debuginfo=line-tables-only as debuginfo=0 causes immensely slow build + # See for more details: https://github.com/rust-lang/rust/issues/119560 + RUSTFLAGS: "-C debuginfo=line-tables-only" RUST_BACKTRACE: "1" # avoid rust stack overflows on tpc-ds tests RUST_MINSTACK: "3000000" From dd4263f843e093c807d63edf73a571b1ba2669b5 Mon Sep 17 00:00:00 2001 From: comphead Date: Sat, 6 Jan 2024 12:23:40 -0800 Subject: [PATCH 566/572] Add `schema_err!` error macros with optional backtrace (#8620) * Add `schema_err!` error macros with optional backtrace --- .github/workflows/rust.yml | 6 +- datafusion/common/src/column.rs | 17 ++-- datafusion/common/src/dfschema.rs | 35 ++++---- datafusion/common/src/error.rs | 93 +++++++++++++-------- datafusion/core/src/dataframe/mod.rs | 2 +- datafusion/expr/src/logical_plan/builder.rs | 34 ++++---- datafusion/sql/src/planner.rs | 2 +- datafusion/sql/src/statement.rs | 15 ++-- datafusion/sql/tests/sql_integration.rs | 6 +- 9 files changed, 119 insertions(+), 91 deletions(-) diff --git a/.github/workflows/rust.yml b/.github/workflows/rust.yml index e2faa75e7bc4..ae6c1ee56129 100644 --- a/.github/workflows/rust.yml +++ b/.github/workflows/rust.yml @@ -106,7 +106,7 @@ jobs: RUSTFLAGS: "-C debuginfo=0 -C opt-level=0 -C incremental=false -C codegen-units=256" RUST_BACKTRACE: "1" # avoid rust stack overflows on tpc-ds tests - RUST_MINSTACK: "3000000" + RUST_MIN_STACK: "3000000" - name: Verify Working Directory Clean run: git diff --exit-code @@ -316,7 +316,7 @@ jobs: RUSTFLAGS: "-C debuginfo=line-tables-only" RUST_BACKTRACE: "1" # avoid rust stack overflows on tpc-ds tests - RUST_MINSTACK: "3000000" + RUST_MIN_STACK: "3000000" macos: name: cargo test (mac) runs-on: macos-latest @@ -356,7 +356,7 @@ jobs: RUSTFLAGS: "-C debuginfo=0 -C opt-level=0 -C incremental=false -C codegen-units=256" RUST_BACKTRACE: "1" # avoid rust stack overflows on tpc-ds tests - RUST_MINSTACK: "3000000" + RUST_MIN_STACK: "3000000" test-datafusion-pyarrow: name: cargo test pyarrow (amd64) diff --git a/datafusion/common/src/column.rs b/datafusion/common/src/column.rs index 2e729c128e73..f0edc7175948 100644 --- a/datafusion/common/src/column.rs +++ b/datafusion/common/src/column.rs @@ -17,6 +17,7 @@ //! Column +use crate::error::_schema_err; use crate::utils::{parse_identifiers_normalized, quote_identifier}; use crate::{DFSchema, DataFusionError, OwnedTableReference, Result, SchemaError}; use std::collections::HashSet; @@ -211,13 +212,13 @@ impl Column { } } - Err(DataFusionError::SchemaError(SchemaError::FieldNotFound { + _schema_err!(SchemaError::FieldNotFound { field: Box::new(Column::new(self.relation.clone(), self.name)), valid_fields: schemas .iter() .flat_map(|s| s.fields().iter().map(|f| f.qualified_column())) .collect(), - })) + }) } /// Qualify column if not done yet. @@ -299,23 +300,21 @@ impl Column { } // If not due to USING columns then due to ambiguous column name - return Err(DataFusionError::SchemaError( - SchemaError::AmbiguousReference { - field: Column::new_unqualified(self.name), - }, - )); + return _schema_err!(SchemaError::AmbiguousReference { + field: Column::new_unqualified(self.name), + }); } } } - Err(DataFusionError::SchemaError(SchemaError::FieldNotFound { + _schema_err!(SchemaError::FieldNotFound { field: Box::new(self), valid_fields: schemas .iter() .flat_map(|s| s.iter()) .flat_map(|s| s.fields().iter().map(|f| f.qualified_column())) .collect(), - })) + }) } } diff --git a/datafusion/common/src/dfschema.rs b/datafusion/common/src/dfschema.rs index 8772c4c3e1ff..85b97aac037d 100644 --- a/datafusion/common/src/dfschema.rs +++ b/datafusion/common/src/dfschema.rs @@ -26,6 +26,7 @@ use std::sync::Arc; use crate::error::{ unqualified_field_not_found, DataFusionError, Result, SchemaError, _plan_err, + _schema_err, }; use crate::{ field_not_found, Column, FunctionalDependencies, OwnedTableReference, TableReference, @@ -141,11 +142,9 @@ impl DFSchema { if let Some(qualifier) = field.qualifier() { qualified_names.insert((qualifier, field.name())); } else if !unqualified_names.insert(field.name()) { - return Err(DataFusionError::SchemaError( - SchemaError::DuplicateUnqualifiedField { - name: field.name().to_string(), - }, - )); + return _schema_err!(SchemaError::DuplicateUnqualifiedField { + name: field.name().to_string(), + }); } } @@ -159,14 +158,12 @@ impl DFSchema { qualified_names.sort(); for (qualifier, name) in &qualified_names { if unqualified_names.contains(name) { - return Err(DataFusionError::SchemaError( - SchemaError::AmbiguousReference { - field: Column { - relation: Some((*qualifier).clone()), - name: name.to_string(), - }, - }, - )); + return _schema_err!(SchemaError::AmbiguousReference { + field: Column { + relation: Some((*qualifier).clone()), + name: name.to_string(), + } + }); } } Ok(Self { @@ -392,14 +389,12 @@ impl DFSchema { if fields_without_qualifier.len() == 1 { Ok(fields_without_qualifier[0]) } else { - Err(DataFusionError::SchemaError( - SchemaError::AmbiguousReference { - field: Column { - relation: None, - name: name.to_string(), - }, + _schema_err!(SchemaError::AmbiguousReference { + field: Column { + relation: None, + name: name.to_string(), }, - )) + }) } } } diff --git a/datafusion/common/src/error.rs b/datafusion/common/src/error.rs index e58faaa15096..978938809c1b 100644 --- a/datafusion/common/src/error.rs +++ b/datafusion/common/src/error.rs @@ -82,7 +82,9 @@ pub enum DataFusionError { Configuration(String), /// This error happens with schema-related errors, such as schema inference not possible /// and non-unique column names. - SchemaError(SchemaError), + /// 2nd argument is for optional backtrace + /// Boxing the optional backtrace to prevent + SchemaError(SchemaError, Box>), /// Error returned during execution of the query. /// Examples include files not found, errors in parsing certain types. Execution(String), @@ -125,34 +127,6 @@ pub enum SchemaError { }, } -/// Create a "field not found" DataFusion::SchemaError -pub fn field_not_found>( - qualifier: Option, - name: &str, - schema: &DFSchema, -) -> DataFusionError { - DataFusionError::SchemaError(SchemaError::FieldNotFound { - field: Box::new(Column::new(qualifier, name)), - valid_fields: schema - .fields() - .iter() - .map(|f| f.qualified_column()) - .collect(), - }) -} - -/// Convenience wrapper over [`field_not_found`] for when there is no qualifier -pub fn unqualified_field_not_found(name: &str, schema: &DFSchema) -> DataFusionError { - DataFusionError::SchemaError(SchemaError::FieldNotFound { - field: Box::new(Column::new_unqualified(name)), - valid_fields: schema - .fields() - .iter() - .map(|f| f.qualified_column()) - .collect(), - }) -} - impl Display for SchemaError { fn fmt(&self, f: &mut Formatter<'_>) -> std::fmt::Result { match self { @@ -298,7 +272,7 @@ impl Display for DataFusionError { write!(f, "IO error: {desc}") } DataFusionError::SQL(ref desc, ref backtrace) => { - let backtrace = backtrace.clone().unwrap_or("".to_owned()); + let backtrace: String = backtrace.clone().unwrap_or("".to_owned()); write!(f, "SQL error: {desc:?}{backtrace}") } DataFusionError::Configuration(ref desc) => { @@ -314,8 +288,10 @@ impl Display for DataFusionError { DataFusionError::Plan(ref desc) => { write!(f, "Error during planning: {desc}") } - DataFusionError::SchemaError(ref desc) => { - write!(f, "Schema error: {desc}") + DataFusionError::SchemaError(ref desc, ref backtrace) => { + let backtrace: &str = + &backtrace.as_ref().clone().unwrap_or("".to_owned()); + write!(f, "Schema error: {desc}{backtrace}") } DataFusionError::Execution(ref desc) => { write!(f, "Execution error: {desc}") @@ -356,7 +332,7 @@ impl Error for DataFusionError { DataFusionError::Internal(_) => None, DataFusionError::Configuration(_) => None, DataFusionError::Plan(_) => None, - DataFusionError::SchemaError(e) => Some(e), + DataFusionError::SchemaError(e, _) => Some(e), DataFusionError::Execution(_) => None, DataFusionError::ResourcesExhausted(_) => None, DataFusionError::External(e) => Some(e.as_ref()), @@ -556,12 +532,63 @@ macro_rules! arrow_err { }; } +// Exposes a macro to create `DataFusionError::SchemaError` with optional backtrace +#[macro_export] +macro_rules! schema_datafusion_err { + ($ERR:expr) => { + DataFusionError::SchemaError( + $ERR, + Box::new(Some(DataFusionError::get_back_trace())), + ) + }; +} + +// Exposes a macro to create `Err(DataFusionError::SchemaError)` with optional backtrace +#[macro_export] +macro_rules! schema_err { + ($ERR:expr) => { + Err(DataFusionError::SchemaError( + $ERR, + Box::new(Some(DataFusionError::get_back_trace())), + )) + }; +} + // To avoid compiler error when using macro in the same crate: // macros from the current crate cannot be referred to by absolute paths pub use internal_datafusion_err as _internal_datafusion_err; pub use internal_err as _internal_err; pub use not_impl_err as _not_impl_err; pub use plan_err as _plan_err; +pub use schema_err as _schema_err; + +/// Create a "field not found" DataFusion::SchemaError +pub fn field_not_found>( + qualifier: Option, + name: &str, + schema: &DFSchema, +) -> DataFusionError { + schema_datafusion_err!(SchemaError::FieldNotFound { + field: Box::new(Column::new(qualifier, name)), + valid_fields: schema + .fields() + .iter() + .map(|f| f.qualified_column()) + .collect(), + }) +} + +/// Convenience wrapper over [`field_not_found`] for when there is no qualifier +pub fn unqualified_field_not_found(name: &str, schema: &DFSchema) -> DataFusionError { + schema_datafusion_err!(SchemaError::FieldNotFound { + field: Box::new(Column::new_unqualified(name)), + valid_fields: schema + .fields() + .iter() + .map(|f| f.qualified_column()) + .collect(), + }) +} #[cfg(test)] mod test { diff --git a/datafusion/core/src/dataframe/mod.rs b/datafusion/core/src/dataframe/mod.rs index 33e198d6d588..f15f1e9ba6fb 100644 --- a/datafusion/core/src/dataframe/mod.rs +++ b/datafusion/core/src/dataframe/mod.rs @@ -1186,7 +1186,7 @@ impl DataFrame { let field_to_rename = match self.plan.schema().field_from_column(&old_column) { Ok(field) => field, // no-op if field not found - Err(DataFusionError::SchemaError(SchemaError::FieldNotFound { .. })) => { + Err(DataFusionError::SchemaError(SchemaError::FieldNotFound { .. }, _)) => { return Ok(self) } Err(err) => return Err(err), diff --git a/datafusion/expr/src/logical_plan/builder.rs b/datafusion/expr/src/logical_plan/builder.rs index a684f3e97485..847fbbbf61c7 100644 --- a/datafusion/expr/src/logical_plan/builder.rs +++ b/datafusion/expr/src/logical_plan/builder.rs @@ -1845,13 +1845,16 @@ mod tests { .project(vec![col("id"), col("first_name").alias("id")]); match plan { - Err(DataFusionError::SchemaError(SchemaError::AmbiguousReference { - field: - Column { - relation: Some(OwnedTableReference::Bare { table }), - name, - }, - })) => { + Err(DataFusionError::SchemaError( + SchemaError::AmbiguousReference { + field: + Column { + relation: Some(OwnedTableReference::Bare { table }), + name, + }, + }, + _, + )) => { assert_eq!("employee_csv", table); assert_eq!("id", &name); Ok(()) @@ -1872,13 +1875,16 @@ mod tests { .aggregate(vec![col("state")], vec![sum(col("salary")).alias("state")]); match plan { - Err(DataFusionError::SchemaError(SchemaError::AmbiguousReference { - field: - Column { - relation: Some(OwnedTableReference::Bare { table }), - name, - }, - })) => { + Err(DataFusionError::SchemaError( + SchemaError::AmbiguousReference { + field: + Column { + relation: Some(OwnedTableReference::Bare { table }), + name, + }, + }, + _, + )) => { assert_eq!("employee_csv", table); assert_eq!("state", &name); Ok(()) diff --git a/datafusion/sql/src/planner.rs b/datafusion/sql/src/planner.rs index c5c30e3a2253..a04df5589b85 100644 --- a/datafusion/sql/src/planner.rs +++ b/datafusion/sql/src/planner.rs @@ -250,7 +250,7 @@ impl<'a, S: ContextProvider> SqlToRel<'a, S> { // Default expressions are restricted, column references are not allowed let empty_schema = DFSchema::empty(); let error_desc = |e: DataFusionError| match e { - DataFusionError::SchemaError(SchemaError::FieldNotFound { .. }) => { + DataFusionError::SchemaError(SchemaError::FieldNotFound { .. }, _) => { plan_datafusion_err!( "Column reference is not allowed in the DEFAULT expression : {}", e diff --git a/datafusion/sql/src/statement.rs b/datafusion/sql/src/statement.rs index b96553ffbf86..b9fb4c65dc2c 100644 --- a/datafusion/sql/src/statement.rs +++ b/datafusion/sql/src/statement.rs @@ -31,9 +31,10 @@ use arrow_schema::DataType; use datafusion_common::file_options::StatementOptions; use datafusion_common::parsers::CompressionTypeVariant; use datafusion_common::{ - not_impl_err, plan_datafusion_err, plan_err, unqualified_field_not_found, Column, - Constraints, DFField, DFSchema, DFSchemaRef, DataFusionError, OwnedTableReference, - Result, ScalarValue, SchemaReference, TableReference, ToDFSchema, + not_impl_err, plan_datafusion_err, plan_err, schema_err, unqualified_field_not_found, + Column, Constraints, DFField, DFSchema, DFSchemaRef, DataFusionError, + OwnedTableReference, Result, ScalarValue, SchemaError, SchemaReference, + TableReference, ToDFSchema, }; use datafusion_expr::dml::{CopyOptions, CopyTo}; use datafusion_expr::expr_rewriter::normalize_col_with_schemas_and_ambiguity_check; @@ -1138,11 +1139,9 @@ impl<'a, S: ContextProvider> SqlToRel<'a, S> { .index_of_column_by_name(None, &c)? .ok_or_else(|| unqualified_field_not_found(&c, &table_schema))?; if value_indices[column_index].is_some() { - return Err(DataFusionError::SchemaError( - datafusion_common::SchemaError::DuplicateUnqualifiedField { - name: c, - }, - )); + return schema_err!(SchemaError::DuplicateUnqualifiedField { + name: c, + }); } else { value_indices[column_index] = Some(i); } diff --git a/datafusion/sql/tests/sql_integration.rs b/datafusion/sql/tests/sql_integration.rs index 48ba50145308..4de08a7124cf 100644 --- a/datafusion/sql/tests/sql_integration.rs +++ b/datafusion/sql/tests/sql_integration.rs @@ -756,9 +756,11 @@ fn join_with_ambiguous_column() { #[test] fn where_selection_with_ambiguous_column() { let sql = "SELECT * FROM person a, person b WHERE id = id + 1"; - let err = logical_plan(sql).expect_err("query should have failed"); + let err = logical_plan(sql) + .expect_err("query should have failed") + .strip_backtrace(); assert_eq!( - "SchemaError(AmbiguousReference { field: Column { relation: None, name: \"id\" } })", + "\"Schema error: Ambiguous reference to unqualified field id\"", format!("{err:?}") ); } From ff27d9073421d527439e6e338f31fb568227bbb2 Mon Sep 17 00:00:00 2001 From: Andrew Lamb Date: Mon, 8 Jan 2024 09:29:33 -0500 Subject: [PATCH 567/572] Fix regression by reverting Materialize dictionaries in group keys (#8740) * revert eb8aff7becaf5d4a44c723b29445deb958fbe3b4 / Materialize dictionaries in group keys * Update tests * Update tests --- datafusion/core/tests/path_partition.rs | 15 +++- .../src/aggregates/group_values/row.rs | 27 ++++++- .../physical-plan/src/aggregates/mod.rs | 35 +------- .../physical-plan/src/aggregates/row_hash.rs | 4 +- .../sqllogictest/test_files/aggregate.slt | 10 +-- .../sqllogictest/test_files/dictionary.slt | 81 ++++++++++++++++++- 6 files changed, 124 insertions(+), 48 deletions(-) diff --git a/datafusion/core/tests/path_partition.rs b/datafusion/core/tests/path_partition.rs index abe6ab283aff..dd8eb52f67c7 100644 --- a/datafusion/core/tests/path_partition.rs +++ b/datafusion/core/tests/path_partition.rs @@ -168,9 +168,9 @@ async fn parquet_distinct_partition_col() -> Result<()> { assert_eq!(min_limit, resulting_limit); let s = ScalarValue::try_from_array(results[0].column(1), 0)?; - let month = match s { - ScalarValue::Utf8(Some(month)) => month, - s => panic!("Expected month as Utf8 found {s:?}"), + let month = match extract_as_utf(&s) { + Some(month) => month, + s => panic!("Expected month as Dict(_, Utf8) found {s:?}"), }; let sql_on_partition_boundary = format!( @@ -191,6 +191,15 @@ async fn parquet_distinct_partition_col() -> Result<()> { Ok(()) } +fn extract_as_utf(v: &ScalarValue) -> Option { + if let ScalarValue::Dictionary(_, v) = v { + if let ScalarValue::Utf8(v) = v.as_ref() { + return v.clone(); + } + } + None +} + #[tokio::test] async fn csv_filter_with_file_col() -> Result<()> { let ctx = SessionContext::new(); diff --git a/datafusion/physical-plan/src/aggregates/group_values/row.rs b/datafusion/physical-plan/src/aggregates/group_values/row.rs index e7c7a42cf902..10ff9edb8912 100644 --- a/datafusion/physical-plan/src/aggregates/group_values/row.rs +++ b/datafusion/physical-plan/src/aggregates/group_values/row.rs @@ -17,18 +17,22 @@ use crate::aggregates::group_values::GroupValues; use ahash::RandomState; +use arrow::compute::cast; use arrow::record_batch::RecordBatch; use arrow::row::{RowConverter, Rows, SortField}; -use arrow_array::ArrayRef; -use arrow_schema::SchemaRef; +use arrow_array::{Array, ArrayRef}; +use arrow_schema::{DataType, SchemaRef}; use datafusion_common::hash_utils::create_hashes; -use datafusion_common::Result; +use datafusion_common::{DataFusionError, Result}; use datafusion_execution::memory_pool::proxy::{RawTableAllocExt, VecAllocExt}; use datafusion_physical_expr::EmitTo; use hashbrown::raw::RawTable; /// A [`GroupValues`] making use of [`Rows`] pub struct GroupValuesRows { + /// The output schema + schema: SchemaRef, + /// Converter for the group values row_converter: RowConverter, @@ -75,6 +79,7 @@ impl GroupValuesRows { let map = RawTable::with_capacity(0); Ok(Self { + schema, row_converter, map, map_size: 0, @@ -165,7 +170,7 @@ impl GroupValues for GroupValuesRows { .take() .expect("Can not emit from empty rows"); - let output = match emit_to { + let mut output = match emit_to { EmitTo::All => { let output = self.row_converter.convert_rows(&group_values)?; group_values.clear(); @@ -198,6 +203,20 @@ impl GroupValues for GroupValuesRows { } }; + // TODO: Materialize dictionaries in group keys (#7647) + for (field, array) in self.schema.fields.iter().zip(&mut output) { + let expected = field.data_type(); + if let DataType::Dictionary(_, v) = expected { + let actual = array.data_type(); + if v.as_ref() != actual { + return Err(DataFusionError::Internal(format!( + "Converted group rows expected dictionary of {v} got {actual}" + ))); + } + *array = cast(array.as_ref(), expected)?; + } + } + self.group_values = Some(group_values); Ok(output) } diff --git a/datafusion/physical-plan/src/aggregates/mod.rs b/datafusion/physical-plan/src/aggregates/mod.rs index a38044de02e3..0b94dd01cfd4 100644 --- a/datafusion/physical-plan/src/aggregates/mod.rs +++ b/datafusion/physical-plan/src/aggregates/mod.rs @@ -36,7 +36,6 @@ use crate::{ use arrow::array::ArrayRef; use arrow::datatypes::{Field, Schema, SchemaRef}; use arrow::record_batch::RecordBatch; -use arrow_schema::DataType; use datafusion_common::stats::Precision; use datafusion_common::{not_impl_err, plan_err, DataFusionError, Result}; use datafusion_execution::TaskContext; @@ -254,9 +253,6 @@ pub struct AggregateExec { limit: Option, /// Input plan, could be a partial aggregate or the input to the aggregate pub input: Arc, - /// Original aggregation schema, could be different from `schema` before dictionary group - /// keys get materialized - original_schema: SchemaRef, /// Schema after the aggregate is applied schema: SchemaRef, /// Input schema before any aggregation is applied. For partial aggregate this will be the @@ -287,7 +283,7 @@ impl AggregateExec { input: Arc, input_schema: SchemaRef, ) -> Result { - let original_schema = create_schema( + let schema = create_schema( &input.schema(), &group_by.expr, &aggr_expr, @@ -295,11 +291,7 @@ impl AggregateExec { mode, )?; - let schema = Arc::new(materialize_dict_group_keys( - &original_schema, - group_by.expr.len(), - )); - let original_schema = Arc::new(original_schema); + let schema = Arc::new(schema); AggregateExec::try_new_with_schema( mode, group_by, @@ -308,7 +300,6 @@ impl AggregateExec { input, input_schema, schema, - original_schema, ) } @@ -329,7 +320,6 @@ impl AggregateExec { input: Arc, input_schema: SchemaRef, schema: SchemaRef, - original_schema: SchemaRef, ) -> Result { let input_eq_properties = input.equivalence_properties(); // Get GROUP BY expressions: @@ -382,7 +372,6 @@ impl AggregateExec { aggr_expr, filter_expr, input, - original_schema, schema, input_schema, projection_mapping, @@ -693,7 +682,7 @@ impl ExecutionPlan for AggregateExec { children[0].clone(), self.input_schema.clone(), self.schema.clone(), - self.original_schema.clone(), + //self.original_schema.clone(), )?; me.limit = self.limit; Ok(Arc::new(me)) @@ -800,24 +789,6 @@ fn create_schema( Ok(Schema::new(fields)) } -/// returns schema with dictionary group keys materialized as their value types -/// The actual convertion happens in `RowConverter` and we don't do unnecessary -/// conversion back into dictionaries -fn materialize_dict_group_keys(schema: &Schema, group_count: usize) -> Schema { - let fields = schema - .fields - .iter() - .enumerate() - .map(|(i, field)| match field.data_type() { - DataType::Dictionary(_, value_data_type) if i < group_count => { - Field::new(field.name(), *value_data_type.clone(), field.is_nullable()) - } - _ => Field::clone(field), - }) - .collect::>(); - Schema::new(fields) -} - fn group_schema(schema: &Schema, group_count: usize) -> SchemaRef { let group_fields = schema.fields()[0..group_count].to_vec(); Arc::new(Schema::new(group_fields)) diff --git a/datafusion/physical-plan/src/aggregates/row_hash.rs b/datafusion/physical-plan/src/aggregates/row_hash.rs index 89614fd3020c..6a0c02f5caf3 100644 --- a/datafusion/physical-plan/src/aggregates/row_hash.rs +++ b/datafusion/physical-plan/src/aggregates/row_hash.rs @@ -324,9 +324,7 @@ impl GroupedHashAggregateStream { .map(create_group_accumulator) .collect::>()?; - // we need to use original schema so RowConverter in group_values below - // will do the proper coversion of dictionaries into value types - let group_schema = group_schema(&agg.original_schema, agg_group_by.expr.len()); + let group_schema = group_schema(&agg_schema, agg_group_by.expr.len()); let spill_expr = group_schema .fields .into_iter() diff --git a/datafusion/sqllogictest/test_files/aggregate.slt b/datafusion/sqllogictest/test_files/aggregate.slt index 78575c9dffc5..aa512f6e2600 100644 --- a/datafusion/sqllogictest/test_files/aggregate.slt +++ b/datafusion/sqllogictest/test_files/aggregate.slt @@ -2469,11 +2469,11 @@ select max(x_dict) from value_dict group by x_dict % 2 order by max(x_dict); query T select arrow_typeof(x_dict) from value_dict group by x_dict; ---- -Int32 -Int32 -Int32 -Int32 -Int32 +Dictionary(Int64, Int32) +Dictionary(Int64, Int32) +Dictionary(Int64, Int32) +Dictionary(Int64, Int32) +Dictionary(Int64, Int32) statement ok drop table value diff --git a/datafusion/sqllogictest/test_files/dictionary.slt b/datafusion/sqllogictest/test_files/dictionary.slt index b7f375dd6c4f..002aade2528e 100644 --- a/datafusion/sqllogictest/test_files/dictionary.slt +++ b/datafusion/sqllogictest/test_files/dictionary.slt @@ -169,7 +169,7 @@ order by date_bin('30 minutes', time) DESC # Reproducer for https://github.com/apache/arrow-datafusion/issues/8738 # This query should work correctly -query error DataFusion error: External error: Arrow error: Invalid argument error: RowConverter column schema mismatch, expected Utf8 got Dictionary\(Int32, Utf8\) +query P?TT rowsort SELECT "data"."timestamp" as "time", "data"."tag_id", @@ -201,3 +201,82 @@ ORDER BY "time", "data"."tag_id" ; +---- +2023-12-20T00:00:00 1000 f1 32.0 +2023-12-20T00:00:00 1000 f2 foo +2023-12-20T00:10:00 1000 f1 32.0 +2023-12-20T00:10:00 1000 f2 foo +2023-12-20T00:20:00 1000 f1 32.0 +2023-12-20T00:20:00 1000 f2 foo +2023-12-20T00:30:00 1000 f1 32.0 +2023-12-20T00:30:00 1000 f2 foo +2023-12-20T00:40:00 1000 f1 32.0 +2023-12-20T00:40:00 1000 f2 foo +2023-12-20T00:50:00 1000 f1 32.0 +2023-12-20T00:50:00 1000 f2 foo +2023-12-20T01:00:00 1000 f1 32.0 +2023-12-20T01:00:00 1000 f2 foo +2023-12-20T01:10:00 1000 f1 32.0 +2023-12-20T01:10:00 1000 f2 foo +2023-12-20T01:20:00 1000 f1 32.0 +2023-12-20T01:20:00 1000 f2 foo +2023-12-20T01:30:00 1000 f1 32.0 +2023-12-20T01:30:00 1000 f2 foo + + +# deterministic sort (so we can avoid rowsort) +query P?TT +SELECT + "data"."timestamp" as "time", + "data"."tag_id", + "data"."field", + "data"."value" +FROM ( + ( + SELECT "m2"."time" as "timestamp", "m2"."tag_id", 'active_power' as "field", "m2"."f5" as "value" + FROM "m2" + WHERE "m2"."time" >= '2023-12-05T14:46:35+01:00' AND "m2"."time" < '2024-01-03T14:46:35+01:00' + AND "m2"."f5" IS NOT NULL + AND "m2"."type" IN ('active') + AND "m2"."tag_id" IN ('1000') + ) UNION ( + SELECT "m1"."time" as "timestamp", "m1"."tag_id", 'f1' as "field", "m1"."f1" as "value" + FROM "m1" + WHERE "m1"."time" >= '2023-12-05T14:46:35+01:00' AND "m1"."time" < '2024-01-03T14:46:35+01:00' + AND "m1"."f1" IS NOT NULL + AND "m1"."tag_id" IN ('1000') + ) UNION ( + SELECT "m1"."time" as "timestamp", "m1"."tag_id", 'f2' as "field", "m1"."f2" as "value" + FROM "m1" + WHERE "m1"."time" >= '2023-12-05T14:46:35+01:00' AND "m1"."time" < '2024-01-03T14:46:35+01:00' + AND "m1"."f2" IS NOT NULL + AND "m1"."tag_id" IN ('1000') + ) +) as "data" +ORDER BY + "time", + "data"."tag_id", + "data"."field", + "data"."value" +; +---- +2023-12-20T00:00:00 1000 f1 32.0 +2023-12-20T00:00:00 1000 f2 foo +2023-12-20T00:10:00 1000 f1 32.0 +2023-12-20T00:10:00 1000 f2 foo +2023-12-20T00:20:00 1000 f1 32.0 +2023-12-20T00:20:00 1000 f2 foo +2023-12-20T00:30:00 1000 f1 32.0 +2023-12-20T00:30:00 1000 f2 foo +2023-12-20T00:40:00 1000 f1 32.0 +2023-12-20T00:40:00 1000 f2 foo +2023-12-20T00:50:00 1000 f1 32.0 +2023-12-20T00:50:00 1000 f2 foo +2023-12-20T01:00:00 1000 f1 32.0 +2023-12-20T01:00:00 1000 f2 foo +2023-12-20T01:10:00 1000 f1 32.0 +2023-12-20T01:10:00 1000 f2 foo +2023-12-20T01:20:00 1000 f1 32.0 +2023-12-20T01:20:00 1000 f2 foo +2023-12-20T01:30:00 1000 f1 32.0 +2023-12-20T01:30:00 1000 f2 foo From cc4289484c33e478242bf5d2b59f695fdb427ab9 Mon Sep 17 00:00:00 2001 From: Huaijin Date: Mon, 8 Jan 2024 22:43:13 +0800 Subject: [PATCH 568/572] fix: struct field don't push down to TableScan (#8774) * fix: struct don't push down to TableScan * add similar to test and apply comment * remove catch all in outer_columns_helper * minor * fix clippy --------- Co-authored-by: Andrew Lamb --- .../optimizer/src/optimize_projections.rs | 315 ++++++++++++++---- 1 file changed, 253 insertions(+), 62 deletions(-) diff --git a/datafusion/optimizer/src/optimize_projections.rs b/datafusion/optimizer/src/optimize_projections.rs index 891a909a3378..1d4eda0bd23e 100644 --- a/datafusion/optimizer/src/optimize_projections.rs +++ b/datafusion/optimizer/src/optimize_projections.rs @@ -583,11 +583,11 @@ fn rewrite_expr(expr: &Expr, input: &Projection) -> Result> { /// /// # Returns /// -/// If the function can safely infer all outer-referenced columns, returns a -/// `Some(HashSet)` containing these columns. Otherwise, returns `None`. -fn outer_columns(expr: &Expr) -> Option> { +/// returns a `HashSet` containing all outer-referenced columns. +fn outer_columns(expr: &Expr) -> HashSet { let mut columns = HashSet::new(); - outer_columns_helper(expr, &mut columns).then_some(columns) + outer_columns_helper(expr, &mut columns); + columns } /// A recursive subroutine that accumulates outer-referenced columns by the @@ -598,87 +598,104 @@ fn outer_columns(expr: &Expr) -> Option> { /// * `expr` - The expression to analyze for outer-referenced columns. /// * `columns` - A mutable reference to a `HashSet` where detected /// columns are collected. -/// -/// Returns `true` if it can safely collect all outer-referenced columns. -/// Otherwise, returns `false`. -fn outer_columns_helper(expr: &Expr, columns: &mut HashSet) -> bool { +fn outer_columns_helper(expr: &Expr, columns: &mut HashSet) { match expr { Expr::OuterReferenceColumn(_, col) => { columns.insert(col.clone()); - true } Expr::BinaryExpr(binary_expr) => { - outer_columns_helper(&binary_expr.left, columns) - && outer_columns_helper(&binary_expr.right, columns) + outer_columns_helper(&binary_expr.left, columns); + outer_columns_helper(&binary_expr.right, columns); } Expr::ScalarSubquery(subquery) => { let exprs = subquery.outer_ref_columns.iter(); - outer_columns_helper_multi(exprs, columns) + outer_columns_helper_multi(exprs, columns); } Expr::Exists(exists) => { let exprs = exists.subquery.outer_ref_columns.iter(); - outer_columns_helper_multi(exprs, columns) + outer_columns_helper_multi(exprs, columns); } Expr::Alias(alias) => outer_columns_helper(&alias.expr, columns), Expr::InSubquery(insubquery) => { let exprs = insubquery.subquery.outer_ref_columns.iter(); - outer_columns_helper_multi(exprs, columns) + outer_columns_helper_multi(exprs, columns); } - Expr::IsNotNull(expr) | Expr::IsNull(expr) => outer_columns_helper(expr, columns), Expr::Cast(cast) => outer_columns_helper(&cast.expr, columns), Expr::Sort(sort) => outer_columns_helper(&sort.expr, columns), Expr::AggregateFunction(aggregate_fn) => { - outer_columns_helper_multi(aggregate_fn.args.iter(), columns) - && aggregate_fn - .order_by - .as_ref() - .map_or(true, |obs| outer_columns_helper_multi(obs.iter(), columns)) - && aggregate_fn - .filter - .as_ref() - .map_or(true, |filter| outer_columns_helper(filter, columns)) + outer_columns_helper_multi(aggregate_fn.args.iter(), columns); + if let Some(filter) = aggregate_fn.filter.as_ref() { + outer_columns_helper(filter, columns); + } + if let Some(obs) = aggregate_fn.order_by.as_ref() { + outer_columns_helper_multi(obs.iter(), columns); + } } Expr::WindowFunction(window_fn) => { - outer_columns_helper_multi(window_fn.args.iter(), columns) - && outer_columns_helper_multi(window_fn.order_by.iter(), columns) - && outer_columns_helper_multi(window_fn.partition_by.iter(), columns) + outer_columns_helper_multi(window_fn.args.iter(), columns); + outer_columns_helper_multi(window_fn.order_by.iter(), columns); + outer_columns_helper_multi(window_fn.partition_by.iter(), columns); } Expr::GroupingSet(groupingset) => match groupingset { - GroupingSet::GroupingSets(multi_exprs) => multi_exprs - .iter() - .all(|e| outer_columns_helper_multi(e.iter(), columns)), + GroupingSet::GroupingSets(multi_exprs) => { + multi_exprs + .iter() + .for_each(|e| outer_columns_helper_multi(e.iter(), columns)); + } GroupingSet::Cube(exprs) | GroupingSet::Rollup(exprs) => { - outer_columns_helper_multi(exprs.iter(), columns) + outer_columns_helper_multi(exprs.iter(), columns); } }, Expr::ScalarFunction(scalar_fn) => { - outer_columns_helper_multi(scalar_fn.args.iter(), columns) + outer_columns_helper_multi(scalar_fn.args.iter(), columns); } Expr::Like(like) => { - outer_columns_helper(&like.expr, columns) - && outer_columns_helper(&like.pattern, columns) + outer_columns_helper(&like.expr, columns); + outer_columns_helper(&like.pattern, columns); } Expr::InList(in_list) => { - outer_columns_helper(&in_list.expr, columns) - && outer_columns_helper_multi(in_list.list.iter(), columns) + outer_columns_helper(&in_list.expr, columns); + outer_columns_helper_multi(in_list.list.iter(), columns); } Expr::Case(case) => { let when_then_exprs = case .when_then_expr .iter() .flat_map(|(first, second)| [first.as_ref(), second.as_ref()]); - outer_columns_helper_multi(when_then_exprs, columns) - && case - .expr - .as_ref() - .map_or(true, |expr| outer_columns_helper(expr, columns)) - && case - .else_expr - .as_ref() - .map_or(true, |expr| outer_columns_helper(expr, columns)) + outer_columns_helper_multi(when_then_exprs, columns); + if let Some(expr) = case.expr.as_ref() { + outer_columns_helper(expr, columns); + } + if let Some(expr) = case.else_expr.as_ref() { + outer_columns_helper(expr, columns); + } + } + Expr::SimilarTo(similar_to) => { + outer_columns_helper(&similar_to.expr, columns); + outer_columns_helper(&similar_to.pattern, columns); + } + Expr::TryCast(try_cast) => outer_columns_helper(&try_cast.expr, columns), + Expr::GetIndexedField(index) => outer_columns_helper(&index.expr, columns), + Expr::Between(between) => { + outer_columns_helper(&between.expr, columns); + outer_columns_helper(&between.low, columns); + outer_columns_helper(&between.high, columns); } - Expr::Column(_) | Expr::Literal(_) | Expr::Wildcard { .. } => true, - _ => false, + Expr::Not(expr) + | Expr::IsNotFalse(expr) + | Expr::IsFalse(expr) + | Expr::IsTrue(expr) + | Expr::IsNotTrue(expr) + | Expr::IsUnknown(expr) + | Expr::IsNotUnknown(expr) + | Expr::IsNotNull(expr) + | Expr::IsNull(expr) + | Expr::Negative(expr) => outer_columns_helper(expr, columns), + Expr::Column(_) + | Expr::Literal(_) + | Expr::Wildcard { .. } + | Expr::ScalarVariable { .. } + | Expr::Placeholder(_) => (), } } @@ -690,14 +707,11 @@ fn outer_columns_helper(expr: &Expr, columns: &mut HashSet) -> bool { /// * `exprs` - The expressions to analyze for outer-referenced columns. /// * `columns` - A mutable reference to a `HashSet` where detected /// columns are collected. -/// -/// Returns `true` if it can safely collect all outer-referenced columns. -/// Otherwise, returns `false`. fn outer_columns_helper_multi<'a>( - mut exprs: impl Iterator, + exprs: impl Iterator, columns: &mut HashSet, -) -> bool { - exprs.all(|e| outer_columns_helper(e, columns)) +) { + exprs.for_each(|e| outer_columns_helper(e, columns)); } /// Generates the required expressions (columns) that reside at `indices` of @@ -766,13 +780,7 @@ fn indices_referred_by_expr( ) -> Result> { let mut cols = expr.to_columns()?; // Get outer-referenced columns: - if let Some(outer_cols) = outer_columns(expr) { - cols.extend(outer_cols); - } else { - // Expression is not known to contain outer columns or not. Hence, do - // not assume anything and require all the schema indices at the input: - return Ok((0..input_schema.fields().len()).collect()); - } + cols.extend(outer_columns(expr)); Ok(cols .iter() .flat_map(|col| input_schema.index_of_column(col)) @@ -978,8 +986,8 @@ mod tests { use arrow::datatypes::{DataType, Field, Schema}; use datafusion_common::{Result, TableReference}; use datafusion_expr::{ - binary_expr, col, count, lit, logical_plan::builder::LogicalPlanBuilder, - table_scan, Expr, LogicalPlan, Operator, + binary_expr, col, count, lit, logical_plan::builder::LogicalPlanBuilder, not, + table_scan, try_cast, Expr, Like, LogicalPlan, Operator, }; fn assert_optimized_plan_equal(plan: &LogicalPlan, expected: &str) -> Result<()> { @@ -1060,4 +1068,187 @@ mod tests { \n TableScan: ?table? projection=[]"; assert_optimized_plan_equal(&plan, expected) } + + #[test] + fn test_struct_field_push_down() -> Result<()> { + let schema = Arc::new(Schema::new(vec![ + Field::new("a", DataType::Int64, false), + Field::new_struct( + "s", + vec![ + Field::new("x", DataType::Int64, false), + Field::new("y", DataType::Int64, false), + ], + false, + ), + ])); + + let table_scan = table_scan(TableReference::none(), &schema, None)?.build()?; + let plan = LogicalPlanBuilder::from(table_scan) + .project(vec![col("s").field("x")])? + .build()?; + let expected = "Projection: (?table?.s)[x]\ + \n TableScan: ?table? projection=[s]"; + assert_optimized_plan_equal(&plan, expected) + } + + #[test] + fn test_neg_push_down() -> Result<()> { + let table_scan = test_table_scan()?; + let plan = LogicalPlanBuilder::from(table_scan) + .project(vec![-col("a")])? + .build()?; + + let expected = "Projection: (- test.a)\ + \n TableScan: test projection=[a]"; + assert_optimized_plan_equal(&plan, expected) + } + + #[test] + fn test_is_null() -> Result<()> { + let table_scan = test_table_scan()?; + let plan = LogicalPlanBuilder::from(table_scan) + .project(vec![col("a").is_null()])? + .build()?; + + let expected = "Projection: test.a IS NULL\ + \n TableScan: test projection=[a]"; + assert_optimized_plan_equal(&plan, expected) + } + + #[test] + fn test_is_not_null() -> Result<()> { + let table_scan = test_table_scan()?; + let plan = LogicalPlanBuilder::from(table_scan) + .project(vec![col("a").is_not_null()])? + .build()?; + + let expected = "Projection: test.a IS NOT NULL\ + \n TableScan: test projection=[a]"; + assert_optimized_plan_equal(&plan, expected) + } + + #[test] + fn test_is_true() -> Result<()> { + let table_scan = test_table_scan()?; + let plan = LogicalPlanBuilder::from(table_scan) + .project(vec![col("a").is_true()])? + .build()?; + + let expected = "Projection: test.a IS TRUE\ + \n TableScan: test projection=[a]"; + assert_optimized_plan_equal(&plan, expected) + } + + #[test] + fn test_is_not_true() -> Result<()> { + let table_scan = test_table_scan()?; + let plan = LogicalPlanBuilder::from(table_scan) + .project(vec![col("a").is_not_true()])? + .build()?; + + let expected = "Projection: test.a IS NOT TRUE\ + \n TableScan: test projection=[a]"; + assert_optimized_plan_equal(&plan, expected) + } + + #[test] + fn test_is_false() -> Result<()> { + let table_scan = test_table_scan()?; + let plan = LogicalPlanBuilder::from(table_scan) + .project(vec![col("a").is_false()])? + .build()?; + + let expected = "Projection: test.a IS FALSE\ + \n TableScan: test projection=[a]"; + assert_optimized_plan_equal(&plan, expected) + } + + #[test] + fn test_is_not_false() -> Result<()> { + let table_scan = test_table_scan()?; + let plan = LogicalPlanBuilder::from(table_scan) + .project(vec![col("a").is_not_false()])? + .build()?; + + let expected = "Projection: test.a IS NOT FALSE\ + \n TableScan: test projection=[a]"; + assert_optimized_plan_equal(&plan, expected) + } + + #[test] + fn test_is_unknown() -> Result<()> { + let table_scan = test_table_scan()?; + let plan = LogicalPlanBuilder::from(table_scan) + .project(vec![col("a").is_unknown()])? + .build()?; + + let expected = "Projection: test.a IS UNKNOWN\ + \n TableScan: test projection=[a]"; + assert_optimized_plan_equal(&plan, expected) + } + + #[test] + fn test_is_not_unknown() -> Result<()> { + let table_scan = test_table_scan()?; + let plan = LogicalPlanBuilder::from(table_scan) + .project(vec![col("a").is_not_unknown()])? + .build()?; + + let expected = "Projection: test.a IS NOT UNKNOWN\ + \n TableScan: test projection=[a]"; + assert_optimized_plan_equal(&plan, expected) + } + + #[test] + fn test_not() -> Result<()> { + let table_scan = test_table_scan()?; + let plan = LogicalPlanBuilder::from(table_scan) + .project(vec![not(col("a"))])? + .build()?; + + let expected = "Projection: NOT test.a\ + \n TableScan: test projection=[a]"; + assert_optimized_plan_equal(&plan, expected) + } + + #[test] + fn test_try_cast() -> Result<()> { + let table_scan = test_table_scan()?; + let plan = LogicalPlanBuilder::from(table_scan) + .project(vec![try_cast(col("a"), DataType::Float64)])? + .build()?; + + let expected = "Projection: TRY_CAST(test.a AS Float64)\ + \n TableScan: test projection=[a]"; + assert_optimized_plan_equal(&plan, expected) + } + + #[test] + fn test_similar_to() -> Result<()> { + let table_scan = test_table_scan()?; + let expr = Box::new(col("a")); + let pattern = Box::new(lit("[0-9]")); + let similar_to_expr = + Expr::SimilarTo(Like::new(false, expr, pattern, None, false)); + let plan = LogicalPlanBuilder::from(table_scan) + .project(vec![similar_to_expr])? + .build()?; + + let expected = "Projection: test.a SIMILAR TO Utf8(\"[0-9]\")\ + \n TableScan: test projection=[a]"; + assert_optimized_plan_equal(&plan, expected) + } + + #[test] + fn test_between() -> Result<()> { + let table_scan = test_table_scan()?; + let plan = LogicalPlanBuilder::from(table_scan) + .project(vec![col("a").between(lit(1), lit(3))])? + .build()?; + + let expected = "Projection: test.a BETWEEN Int32(1) AND Int32(3)\ + \n TableScan: test projection=[a]"; + assert_optimized_plan_equal(&plan, expected) + } } From 746988a7e5c9f256c3b24ab7e3f30ffd90d542a0 Mon Sep 17 00:00:00 2001 From: Andrew Lamb Date: Mon, 8 Jan 2024 11:19:20 -0500 Subject: [PATCH 569/572] Implement ScalarUDF in terms of ScalarUDFImpl trait (#8713) --- datafusion-examples/examples/advanced_udf.rs | 1 + datafusion/expr/src/expr.rs | 1 + datafusion/expr/src/expr_fn.rs | 10 + datafusion/expr/src/udf.rs | 231 ++++++++++++------ .../optimizer/src/analyzer/type_coercion.rs | 1 + datafusion/physical-expr/src/udf.rs | 2 +- 6 files changed, 171 insertions(+), 75 deletions(-) diff --git a/datafusion-examples/examples/advanced_udf.rs b/datafusion-examples/examples/advanced_udf.rs index 6ebf88a0b671..d530b9abe030 100644 --- a/datafusion-examples/examples/advanced_udf.rs +++ b/datafusion-examples/examples/advanced_udf.rs @@ -40,6 +40,7 @@ use std::sync::Arc; /// the power of the second argument `a^b`. /// /// To do so, we must implement the `ScalarUDFImpl` trait. +#[derive(Debug, Clone)] struct PowUdf { signature: Signature, aliases: Vec, diff --git a/datafusion/expr/src/expr.rs b/datafusion/expr/src/expr.rs index ebf4d3143c12..5617d217eb9f 100644 --- a/datafusion/expr/src/expr.rs +++ b/datafusion/expr/src/expr.rs @@ -1948,6 +1948,7 @@ mod test { ); // UDF + #[derive(Debug)] struct TestScalarUDF { signature: Signature, } diff --git a/datafusion/expr/src/expr_fn.rs b/datafusion/expr/src/expr_fn.rs index 7b3f65248586..0491750d18a9 100644 --- a/datafusion/expr/src/expr_fn.rs +++ b/datafusion/expr/src/expr_fn.rs @@ -984,6 +984,16 @@ pub struct SimpleScalarUDF { fun: ScalarFunctionImplementation, } +impl Debug for SimpleScalarUDF { + fn fmt(&self, f: &mut std::fmt::Formatter) -> std::fmt::Result { + f.debug_struct("ScalarUDF") + .field("name", &self.name) + .field("signature", &self.signature) + .field("fun", &"") + .finish() + } +} + impl SimpleScalarUDF { /// Create a new `SimpleScalarUDF` from a name, input types, return type and /// implementation. Implementing [`ScalarUDFImpl`] allows more flexibility diff --git a/datafusion/expr/src/udf.rs b/datafusion/expr/src/udf.rs index 2ec80a4a9ea1..8b35d5834c61 100644 --- a/datafusion/expr/src/udf.rs +++ b/datafusion/expr/src/udf.rs @@ -35,48 +35,26 @@ use std::sync::Arc; /// functions you supply such name, type signature, return type, and actual /// implementation. /// -/// /// 1. For simple (less performant) use cases, use [`create_udf`] and [`simple_udf.rs`]. /// /// 2. For advanced use cases, use [`ScalarUDFImpl`] and [`advanced_udf.rs`]. /// +/// # API Note +/// +/// This is a separate struct from `ScalarUDFImpl` to maintain backwards +/// compatibility with the older API. +/// /// [`create_udf`]: crate::expr_fn::create_udf /// [`simple_udf.rs`]: https://github.com/apache/arrow-datafusion/blob/main/datafusion-examples/examples/simple_udf.rs /// [`advanced_udf.rs`]: https://github.com/apache/arrow-datafusion/blob/main/datafusion-examples/examples/advanced_udf.rs -#[derive(Clone)] +#[derive(Debug, Clone)] pub struct ScalarUDF { - /// The name of the function - name: String, - /// The signature (the types of arguments that are supported) - signature: Signature, - /// Function that returns the return type given the argument types - return_type: ReturnTypeFunction, - /// actual implementation - /// - /// The fn param is the wrapped function but be aware that the function will - /// be passed with the slice / vec of columnar values (either scalar or array) - /// with the exception of zero param function, where a singular element vec - /// will be passed. In that case the single element is a null array to indicate - /// the batch's row count (so that the generative zero-argument function can know - /// the result array size). - fun: ScalarFunctionImplementation, - /// Optional aliases for the function. This list should NOT include the value of `name` as well - aliases: Vec, -} - -impl Debug for ScalarUDF { - fn fmt(&self, f: &mut Formatter) -> fmt::Result { - f.debug_struct("ScalarUDF") - .field("name", &self.name) - .field("signature", &self.signature) - .field("fun", &"") - .finish() - } + inner: Arc, } impl PartialEq for ScalarUDF { fn eq(&self, other: &Self) -> bool { - self.name == other.name && self.signature == other.signature + self.name() == other.name() && self.signature() == other.signature() } } @@ -84,8 +62,8 @@ impl Eq for ScalarUDF {} impl std::hash::Hash for ScalarUDF { fn hash(&self, state: &mut H) { - self.name.hash(state); - self.signature.hash(state); + self.name().hash(state); + self.signature().hash(state); } } @@ -101,13 +79,12 @@ impl ScalarUDF { return_type: &ReturnTypeFunction, fun: &ScalarFunctionImplementation, ) -> Self { - Self { + Self::new_from_impl(ScalarUdfLegacyWrapper { name: name.to_owned(), signature: signature.clone(), return_type: return_type.clone(), fun: fun.clone(), - aliases: vec![], - } + }) } /// Create a new `ScalarUDF` from a `[ScalarUDFImpl]` trait object @@ -115,37 +92,24 @@ impl ScalarUDF { /// Note this is the same as using the `From` impl (`ScalarUDF::from`) pub fn new_from_impl(fun: F) -> ScalarUDF where - F: ScalarUDFImpl + Send + Sync + 'static, + F: ScalarUDFImpl + 'static, { - // TODO change the internal implementation to use the trait object - let arc_fun = Arc::new(fun); - let captured_self = arc_fun.clone(); - let return_type: ReturnTypeFunction = Arc::new(move |arg_types| { - let return_type = captured_self.return_type(arg_types)?; - Ok(Arc::new(return_type)) - }); - - let captured_self = arc_fun.clone(); - let func: ScalarFunctionImplementation = - Arc::new(move |args| captured_self.invoke(args)); - Self { - name: arc_fun.name().to_string(), - signature: arc_fun.signature().clone(), - return_type: return_type.clone(), - fun: func, - aliases: arc_fun.aliases().to_vec(), + inner: Arc::new(fun), } } - /// Adds additional names that can be used to invoke this function, in addition to `name` - pub fn with_aliases( - mut self, - aliases: impl IntoIterator, - ) -> Self { - self.aliases - .extend(aliases.into_iter().map(|s| s.to_string())); - self + /// Return the underlying [`ScalarUDFImpl`] trait object for this function + pub fn inner(&self) -> Arc { + self.inner.clone() + } + + /// Adds additional names that can be used to invoke this function, in + /// addition to `name` + /// + /// If you implement [`ScalarUDFImpl`] directly you should return aliases directly. + pub fn with_aliases(self, aliases: impl IntoIterator) -> Self { + Self::new_from_impl(AliasedScalarUDFImpl::new(self, aliases)) } /// Returns a [`Expr`] logical expression to call this UDF with specified @@ -159,31 +123,46 @@ impl ScalarUDF { )) } - /// Returns this function's name + /// Returns this function's name. + /// + /// See [`ScalarUDFImpl::name`] for more details. pub fn name(&self) -> &str { - &self.name + self.inner.name() } - /// Returns the aliases for this function. See [`ScalarUDF::with_aliases`] for more details + /// Returns the aliases for this function. + /// + /// See [`ScalarUDF::with_aliases`] for more details pub fn aliases(&self) -> &[String] { - &self.aliases + self.inner.aliases() } - /// Returns this function's [`Signature`] (what input types are accepted) + /// Returns this function's [`Signature`] (what input types are accepted). + /// + /// See [`ScalarUDFImpl::signature`] for more details. pub fn signature(&self) -> &Signature { - &self.signature + self.inner.signature() } - /// The datatype this function returns given the input argument input types + /// The datatype this function returns given the input argument input types. + /// + /// See [`ScalarUDFImpl::return_type`] for more details. pub fn return_type(&self, args: &[DataType]) -> Result { - // Old API returns an Arc of the datatype for some reason - let res = (self.return_type)(args)?; - Ok(res.as_ref().clone()) + self.inner.return_type(args) + } + + /// Invoke the function on `args`, returning the appropriate result. + /// + /// See [`ScalarUDFImpl::invoke`] for more details. + pub fn invoke(&self, args: &[ColumnarValue]) -> Result { + self.inner.invoke(args) } - /// Return an [`Arc`] to the function implementation + /// Returns a `ScalarFunctionImplementation` that can invoke the function + /// during execution pub fn fun(&self) -> ScalarFunctionImplementation { - self.fun.clone() + let captured = self.inner.clone(); + Arc::new(move |args| captured.invoke(args)) } } @@ -213,6 +192,7 @@ where /// # use datafusion_common::{DataFusionError, plan_err, Result}; /// # use datafusion_expr::{col, ColumnarValue, Signature, Volatility}; /// # use datafusion_expr::{ScalarUDFImpl, ScalarUDF}; +/// #[derive(Debug)] /// struct AddOne { /// signature: Signature /// }; @@ -246,7 +226,7 @@ where /// // Call the function `add_one(col)` /// let expr = add_one.call(vec![col("a")]); /// ``` -pub trait ScalarUDFImpl { +pub trait ScalarUDFImpl: Debug + Send + Sync { /// Returns this object as an [`Any`] trait object fn as_any(&self) -> &dyn Any; @@ -292,3 +272,106 @@ pub trait ScalarUDFImpl { &[] } } + +/// ScalarUDF that adds an alias to the underlying function. It is better to +/// implement [`ScalarUDFImpl`], which supports aliases, directly if possible. +#[derive(Debug)] +struct AliasedScalarUDFImpl { + inner: ScalarUDF, + aliases: Vec, +} + +impl AliasedScalarUDFImpl { + pub fn new( + inner: ScalarUDF, + new_aliases: impl IntoIterator, + ) -> Self { + let mut aliases = inner.aliases().to_vec(); + aliases.extend(new_aliases.into_iter().map(|s| s.to_string())); + + Self { inner, aliases } + } +} + +impl ScalarUDFImpl for AliasedScalarUDFImpl { + fn as_any(&self) -> &dyn Any { + self + } + fn name(&self) -> &str { + self.inner.name() + } + + fn signature(&self) -> &Signature { + self.inner.signature() + } + + fn return_type(&self, arg_types: &[DataType]) -> Result { + self.inner.return_type(arg_types) + } + + fn invoke(&self, args: &[ColumnarValue]) -> Result { + self.inner.invoke(args) + } + + fn aliases(&self) -> &[String] { + &self.aliases + } +} + +/// Implementation of [`ScalarUDFImpl`] that wraps the function style pointers +/// of the older API (see +/// for more details) +struct ScalarUdfLegacyWrapper { + /// The name of the function + name: String, + /// The signature (the types of arguments that are supported) + signature: Signature, + /// Function that returns the return type given the argument types + return_type: ReturnTypeFunction, + /// actual implementation + /// + /// The fn param is the wrapped function but be aware that the function will + /// be passed with the slice / vec of columnar values (either scalar or array) + /// with the exception of zero param function, where a singular element vec + /// will be passed. In that case the single element is a null array to indicate + /// the batch's row count (so that the generative zero-argument function can know + /// the result array size). + fun: ScalarFunctionImplementation, +} + +impl Debug for ScalarUdfLegacyWrapper { + fn fmt(&self, f: &mut Formatter) -> fmt::Result { + f.debug_struct("ScalarUDF") + .field("name", &self.name) + .field("signature", &self.signature) + .field("fun", &"") + .finish() + } +} + +impl ScalarUDFImpl for ScalarUdfLegacyWrapper { + fn as_any(&self) -> &dyn Any { + self + } + fn name(&self) -> &str { + &self.name + } + + fn signature(&self) -> &Signature { + &self.signature + } + + fn return_type(&self, arg_types: &[DataType]) -> Result { + // Old API returns an Arc of the datatype for some reason + let res = (self.return_type)(arg_types)?; + Ok(res.as_ref().clone()) + } + + fn invoke(&self, args: &[ColumnarValue]) -> Result { + (self.fun)(args) + } + + fn aliases(&self) -> &[String] { + &[] + } +} diff --git a/datafusion/optimizer/src/analyzer/type_coercion.rs b/datafusion/optimizer/src/analyzer/type_coercion.rs index 4d54dad99670..6f1da5f4e6d9 100644 --- a/datafusion/optimizer/src/analyzer/type_coercion.rs +++ b/datafusion/optimizer/src/analyzer/type_coercion.rs @@ -811,6 +811,7 @@ mod test { static TEST_SIGNATURE: OnceLock = OnceLock::new(); + #[derive(Debug, Clone, Default)] struct TestScalarUDF {} impl ScalarUDFImpl for TestScalarUDF { fn as_any(&self) -> &dyn Any { diff --git a/datafusion/physical-expr/src/udf.rs b/datafusion/physical-expr/src/udf.rs index 0ec1cf3f256b..9daa9eb173dd 100644 --- a/datafusion/physical-expr/src/udf.rs +++ b/datafusion/physical-expr/src/udf.rs @@ -36,7 +36,7 @@ pub fn create_physical_expr( Ok(Arc::new(ScalarFunctionExpr::new( fun.name(), - fun.fun().clone(), + fun.fun(), input_phy_exprs.to_vec(), fun.return_type(&input_exprs_types)?, None, From dd58c5a7b0069e3bade997abad9ec6bed8c8a1c3 Mon Sep 17 00:00:00 2001 From: Alex Huang Date: Tue, 9 Jan 2024 03:34:04 +0800 Subject: [PATCH 570/572] Minor: Fix error messages in array expressions (#8781) * Fix error messages in array expressions * fix fmt --- .../physical-expr/src/array_expressions.rs | 36 +++++++++---------- 1 file changed, 16 insertions(+), 20 deletions(-) diff --git a/datafusion/physical-expr/src/array_expressions.rs b/datafusion/physical-expr/src/array_expressions.rs index cb4ad3ed63fe..9665116b04ab 100644 --- a/datafusion/physical-expr/src/array_expressions.rs +++ b/datafusion/physical-expr/src/array_expressions.rs @@ -529,7 +529,7 @@ fn general_except( pub fn array_except(args: &[ArrayRef]) -> Result { if args.len() != 2 { - return internal_err!("array_except needs two arguments"); + return exec_err!("array_except needs two arguments"); } let array1 = &args[0]; @@ -894,7 +894,7 @@ pub fn gen_range(args: &[ArrayRef]) -> Result { as_int64_array(&args[1])?, Some(as_int64_array(&args[2])?), ), - _ => return internal_err!("gen_range expects 1 to 3 arguments"), + _ => return exec_err!("gen_range expects 1 to 3 arguments"), }; let mut values = vec![]; @@ -948,7 +948,7 @@ pub fn array_sort(args: &[ArrayRef]) -> Result { nulls_first: order_nulls_first(nulls_first)?, }) } - _ => return internal_err!("array_sort expects 1 to 3 arguments"), + _ => return exec_err!("array_sort expects 1 to 3 arguments"), }; let list_array = as_list_array(&args[0])?; @@ -994,7 +994,7 @@ fn order_desc(modifier: &str) -> Result { match modifier.to_uppercase().as_str() { "DESC" => Ok(true), "ASC" => Ok(false), - _ => internal_err!("the second parameter of array_sort expects DESC or ASC"), + _ => exec_err!("the second parameter of array_sort expects DESC or ASC"), } } @@ -1002,7 +1002,7 @@ fn order_nulls_first(modifier: &str) -> Result { match modifier.to_uppercase().as_str() { "NULLS FIRST" => Ok(true), "NULLS LAST" => Ok(false), - _ => internal_err!( + _ => exec_err!( "the third parameter of array_sort expects NULLS FIRST or NULLS LAST" ), } @@ -1208,7 +1208,7 @@ pub fn array_empty(args: &[ArrayRef]) -> Result { match array_type { DataType::List(_) => array_empty_dispatch::(&args[0]), DataType::LargeList(_) => array_empty_dispatch::(&args[0]), - _ => internal_err!("array_empty does not support type '{array_type:?}'."), + _ => exec_err!("array_empty does not support type '{array_type:?}'."), } } @@ -1598,7 +1598,9 @@ fn array_remove_internal( let list_array = array.as_list::(); general_remove::(list_array, element_array, arr_n) } - _ => internal_err!("array_remove_all expects a list array"), + array_type => { + exec_err!("array_remove_all does not support type '{array_type:?}'.") + } } } @@ -2260,10 +2262,7 @@ pub fn array_length(args: &[ArrayRef]) -> Result { match &args[0].data_type() { DataType::List(_) => array_length_dispatch::(args), DataType::LargeList(_) => array_length_dispatch::(args), - _ => internal_err!( - "array_length does not support type '{:?}'", - args[0].data_type() - ), + array_type => exec_err!("array_length does not support type '{array_type:?}'"), } } @@ -2288,11 +2287,8 @@ pub fn array_dims(args: &[ArrayRef]) -> Result { .map(compute_array_dims) .collect::>>()? } - _ => { - return exec_err!( - "array_dims does not support type '{:?}'", - args[0].data_type() - ); + array_type => { + return exec_err!("array_dims does not support type '{array_type:?}'"); } }; @@ -2441,7 +2437,7 @@ pub fn array_has_any(args: &[ArrayRef]) -> Result { DataType::LargeList(_) => { general_array_has_dispatch::(&args[0], &args[1], ComparisonType::Any) } - _ => internal_err!("array_has_any does not support type '{array_type:?}'."), + _ => exec_err!("array_has_any does not support type '{array_type:?}'."), } } @@ -2460,7 +2456,7 @@ pub fn array_has_all(args: &[ArrayRef]) -> Result { DataType::LargeList(_) => { general_array_has_dispatch::(&args[0], &args[1], ComparisonType::All) } - _ => internal_err!("array_has_all does not support type '{array_type:?}'."), + _ => exec_err!("array_has_all does not support type '{array_type:?}'."), } } @@ -2543,7 +2539,7 @@ pub fn string_to_array(args: &[ArrayRef]) -> Result { - return internal_err!( + return exec_err!( "Expect string_to_array function to take two or three parameters" ) } @@ -2611,7 +2607,7 @@ pub fn array_distinct(args: &[ArrayRef]) -> Result { let array = as_large_list_array(&args[0])?; general_array_distinct(array, field) } - _ => internal_err!("array_distinct only support list array"), + array_type => exec_err!("array_distinct does not support type '{array_type:?}'"), } } From 0e53c6d816f3a9d3d27c6ebb6d25b1699e5553e7 Mon Sep 17 00:00:00 2001 From: comphead Date: Mon, 8 Jan 2024 15:15:55 -0800 Subject: [PATCH 571/572] Move tests from `expr.rs` to sqllogictests. Part1 (#8773) * move tests from to sqllogictests part1 * Update datafusion/sqllogictest/test_files/expr.slt Co-authored-by: Andrew Lamb * Update datafusion/sqllogictest/test_files/expr.slt Co-authored-by: Andrew Lamb --------- Co-authored-by: Andrew Lamb --- datafusion/core/tests/sql/expr.rs | 718 ----------- datafusion/sqllogictest/test_files/expr.slt | 1251 +++++++++++++++++++ 2 files changed, 1251 insertions(+), 718 deletions(-) create mode 100644 datafusion/sqllogictest/test_files/expr.slt diff --git a/datafusion/core/tests/sql/expr.rs b/datafusion/core/tests/sql/expr.rs index 8ac0e3e5ef19..e8a3d27c089a 100644 --- a/datafusion/core/tests/sql/expr.rs +++ b/datafusion/core/tests/sql/expr.rs @@ -19,55 +19,6 @@ use datafusion::datasource::empty::EmptyTable; use super::*; -#[tokio::test] -async fn test_boolean_expressions() -> Result<()> { - test_expression!("true", "true"); - test_expression!("false", "false"); - test_expression!("false = false", "true"); - test_expression!("true = false", "false"); - Ok(()) -} - -#[tokio::test] -async fn test_mathematical_expressions_with_null() -> Result<()> { - test_expression!("sqrt(NULL)", "NULL"); - test_expression!("cbrt(NULL)", "NULL"); - test_expression!("sin(NULL)", "NULL"); - test_expression!("cos(NULL)", "NULL"); - test_expression!("tan(NULL)", "NULL"); - test_expression!("asin(NULL)", "NULL"); - test_expression!("acos(NULL)", "NULL"); - test_expression!("atan(NULL)", "NULL"); - test_expression!("sinh(NULL)", "NULL"); - test_expression!("cosh(NULL)", "NULL"); - test_expression!("tanh(NULL)", "NULL"); - test_expression!("asinh(NULL)", "NULL"); - test_expression!("acosh(NULL)", "NULL"); - test_expression!("atanh(NULL)", "NULL"); - test_expression!("floor(NULL)", "NULL"); - test_expression!("ceil(NULL)", "NULL"); - test_expression!("round(NULL)", "NULL"); - test_expression!("trunc(NULL)", "NULL"); - test_expression!("abs(NULL)", "NULL"); - test_expression!("signum(NULL)", "NULL"); - test_expression!("exp(NULL)", "NULL"); - test_expression!("ln(NULL)", "NULL"); - test_expression!("log2(NULL)", "NULL"); - test_expression!("log10(NULL)", "NULL"); - test_expression!("power(NULL, 2)", "NULL"); - test_expression!("power(NULL, NULL)", "NULL"); - test_expression!("power(2, NULL)", "NULL"); - test_expression!("atan2(NULL, NULL)", "NULL"); - test_expression!("atan2(1, NULL)", "NULL"); - test_expression!("atan2(NULL, 1)", "NULL"); - test_expression!("nanvl(NULL, NULL)", "NULL"); - test_expression!("nanvl(1, NULL)", "NULL"); - test_expression!("nanvl(NULL, 1)", "NULL"); - test_expression!("isnan(NULL)", "NULL"); - test_expression!("iszero(NULL)", "NULL"); - Ok(()) -} - #[tokio::test] #[cfg_attr(not(feature = "crypto_expressions"), ignore)] async fn test_encoding_expressions() -> Result<()> { @@ -128,14 +79,6 @@ async fn test_encoding_expressions() -> Result<()> { Ok(()) } -#[should_panic(expected = "Invalid timezone \\\"Foo\\\": 'Foo' is not a valid timezone")] -#[tokio::test] -async fn test_array_cast_invalid_timezone_will_panic() { - let ctx = SessionContext::new(); - let sql = "SELECT arrow_cast('2021-01-02T03:04:00', 'Timestamp(Nanosecond, Some(\"Foo\"))')"; - execute(&ctx, sql).await; -} - #[tokio::test] #[cfg_attr(not(feature = "crypto_expressions"), ignore)] async fn test_crypto_expressions() -> Result<()> { @@ -212,242 +155,6 @@ async fn test_crypto_expressions() -> Result<()> { Ok(()) } -#[tokio::test] -async fn test_array_index() -> Result<()> { - // By default PostgreSQL uses a one-based numbering convention for arrays, that is, an array of n elements starts with array[1] and ends with array[n] - test_expression!("([5,4,3,2,1])[1]", "5"); - test_expression!("([5,4,3,2,1])[2]", "4"); - test_expression!("([5,4,3,2,1])[5]", "1"); - test_expression!("([[1, 2], [2, 3], [3,4]])[1]", "[1, 2]"); - test_expression!("([[1, 2], [2, 3], [3,4]])[3]", "[3, 4]"); - test_expression!("([[1, 2], [2, 3], [3,4]])[1][1]", "1"); - test_expression!("([[1, 2], [2, 3], [3,4]])[2][2]", "3"); - test_expression!("([[1, 2], [2, 3], [3,4]])[3][2]", "4"); - // out of bounds - test_expression!("([5,4,3,2,1])[0]", "NULL"); - test_expression!("([5,4,3,2,1])[6]", "NULL"); - // test_expression!("([5,4,3,2,1])[-1]", "NULL"); - test_expression!("([5,4,3,2,1])[100]", "NULL"); - - Ok(()) -} - -#[tokio::test] -async fn test_array_literals() -> Result<()> { - // Named, just another syntax - test_expression!("ARRAY[1,2,3,4,5]", "[1, 2, 3, 4, 5]"); - // Unnamed variant - test_expression!("[1,2,3,4,5]", "[1, 2, 3, 4, 5]"); - test_expression!("[true, false]", "[true, false]"); - test_expression!("['str1', 'str2']", "[str1, str2]"); - test_expression!("[[1,2], [3,4]]", "[[1, 2], [3, 4]]"); - - // TODO: Not supported in parser, uncomment when it will be available - // test_expression!( - // "[]", - // "[]" - // ); - - Ok(()) -} - -#[tokio::test] -async fn test_struct_literals() -> Result<()> { - test_expression!("STRUCT(1,2,3,4,5)", "{c0: 1, c1: 2, c2: 3, c3: 4, c4: 5}"); - test_expression!("STRUCT(Null)", "{c0: }"); - test_expression!("STRUCT(2)", "{c0: 2}"); - test_expression!("STRUCT('1',Null)", "{c0: 1, c1: }"); - test_expression!("STRUCT(true, false)", "{c0: true, c1: false}"); - test_expression!("STRUCT('str1', 'str2')", "{c0: str1, c1: str2}"); - - Ok(()) -} - -#[tokio::test] -async fn binary_bitwise_shift() -> Result<()> { - test_expression!("2 << 10", "2048"); - test_expression!("2048 >> 10", "2"); - test_expression!("2048 << NULL", "NULL"); - test_expression!("2048 >> NULL", "NULL"); - - Ok(()) -} - -#[tokio::test] -async fn test_interval_expressions() -> Result<()> { - // day nano intervals - test_expression!( - "interval '1'", - "0 years 0 mons 0 days 0 hours 0 mins 1.000000000 secs" - ); - test_expression!( - "interval '1 second'", - "0 years 0 mons 0 days 0 hours 0 mins 1.000000000 secs" - ); - test_expression!( - "interval '500 milliseconds'", - "0 years 0 mons 0 days 0 hours 0 mins 0.500000000 secs" - ); - test_expression!( - "interval '5 second'", - "0 years 0 mons 0 days 0 hours 0 mins 5.000000000 secs" - ); - test_expression!( - "interval '0.5 minute'", - "0 years 0 mons 0 days 0 hours 0 mins 30.000000000 secs" - ); - // https://github.com/apache/arrow-rs/issues/4424 - // test_expression!( - // "interval '.5 minute'", - // "0 years 0 mons 0 days 0 hours 0 mins 30.000000000 secs" - // ); - test_expression!( - "interval '5 minute'", - "0 years 0 mons 0 days 0 hours 5 mins 0.000000000 secs" - ); - test_expression!( - "interval '5 minute 1 second'", - "0 years 0 mons 0 days 0 hours 5 mins 1.000000000 secs" - ); - test_expression!( - "interval '1 hour'", - "0 years 0 mons 0 days 1 hours 0 mins 0.000000000 secs" - ); - test_expression!( - "interval '5 hour'", - "0 years 0 mons 0 days 5 hours 0 mins 0.000000000 secs" - ); - test_expression!( - "interval '1 day'", - "0 years 0 mons 1 days 0 hours 0 mins 0.000000000 secs" - ); - test_expression!( - "interval '1 week'", - "0 years 0 mons 7 days 0 hours 0 mins 0.000000000 secs" - ); - test_expression!( - "interval '2 weeks'", - "0 years 0 mons 14 days 0 hours 0 mins 0.000000000 secs" - ); - test_expression!( - "interval '1 day 1'", - "0 years 0 mons 1 days 0 hours 0 mins 1.000000000 secs" - ); - test_expression!( - "interval '0.5'", - "0 years 0 mons 0 days 0 hours 0 mins 0.500000000 secs" - ); - test_expression!( - "interval '0.5 day 1'", - "0 years 0 mons 0 days 12 hours 0 mins 1.000000000 secs" - ); - test_expression!( - "interval '0.49 day'", - "0 years 0 mons 0 days 11 hours 45 mins 36.000000000 secs" - ); - test_expression!( - "interval '0.499 day'", - "0 years 0 mons 0 days 11 hours 58 mins 33.600000000 secs" - ); - test_expression!( - "interval '0.4999 day'", - "0 years 0 mons 0 days 11 hours 59 mins 51.360000000 secs" - ); - test_expression!( - "interval '0.49999 day'", - "0 years 0 mons 0 days 11 hours 59 mins 59.136000000 secs" - ); - test_expression!( - "interval '0.49999999999 day'", - "0 years 0 mons 0 days 11 hours 59 mins 59.999999136 secs" - ); - test_expression!( - "interval '5 day'", - "0 years 0 mons 5 days 0 hours 0 mins 0.000000000 secs" - ); - // Hour is ignored, this matches PostgreSQL - test_expression!( - "interval '5 day' hour", - "0 years 0 mons 5 days 0 hours 0 mins 0.000000000 secs" - ); - test_expression!( - "interval '5 day 4 hours 3 minutes 2 seconds 100 milliseconds'", - "0 years 0 mons 5 days 4 hours 3 mins 2.100000000 secs" - ); - // month intervals - test_expression!( - "interval '0.5 month'", - "0 years 0 mons 15 days 0 hours 0 mins 0.000000000 secs" - ); - test_expression!( - "interval '0.5' month", - "0 years 0 mons 15 days 0 hours 0 mins 0.000000000 secs" - ); - test_expression!( - "interval '1 month'", - "0 years 1 mons 0 days 0 hours 0 mins 0.000000000 secs" - ); - test_expression!( - "interval '1' MONTH", - "0 years 1 mons 0 days 0 hours 0 mins 0.000000000 secs" - ); - test_expression!( - "interval '5 month'", - "0 years 5 mons 0 days 0 hours 0 mins 0.000000000 secs" - ); - test_expression!( - "interval '13 month'", - "0 years 13 mons 0 days 0 hours 0 mins 0.000000000 secs" - ); - test_expression!( - "interval '0.5 year'", - "0 years 6 mons 0 days 0 hours 0 mins 0.000000000 secs" - ); - test_expression!( - "interval '1 year'", - "0 years 12 mons 0 days 0 hours 0 mins 0.000000000 secs" - ); - test_expression!( - "interval '1 decade'", - "0 years 120 mons 0 days 0 hours 0 mins 0.000000000 secs" - ); - test_expression!( - "interval '2 decades'", - "0 years 240 mons 0 days 0 hours 0 mins 0.000000000 secs" - ); - test_expression!( - "interval '1 century'", - "0 years 1200 mons 0 days 0 hours 0 mins 0.000000000 secs" - ); - test_expression!( - "interval '2 year'", - "0 years 24 mons 0 days 0 hours 0 mins 0.000000000 secs" - ); - test_expression!( - "interval '2' year", - "0 years 24 mons 0 days 0 hours 0 mins 0.000000000 secs" - ); - // complex - test_expression!( - "interval '1 year 1 day'", - "0 years 12 mons 1 days 0 hours 0 mins 0.000000000 secs" - ); - test_expression!( - "interval '1 year 1 day 1 hour'", - "0 years 12 mons 1 days 1 hours 0 mins 0.000000000 secs" - ); - test_expression!( - "interval '1 year 1 day 1 hour 1 minute'", - "0 years 12 mons 1 days 1 hours 1 mins 0.000000000 secs" - ); - test_expression!( - "interval '1 year 1 day 1 hour 1 minute 1 second'", - "0 years 12 mons 1 days 1 hours 1 mins 1.000000000 secs" - ); - - Ok(()) -} - #[cfg(feature = "unicode_expressions")] #[tokio::test] async fn test_substring_expr() -> Result<()> { @@ -458,108 +165,6 @@ async fn test_substring_expr() -> Result<()> { Ok(()) } -/// Test string expressions test split into two batches -/// to prevent stack overflow error -#[tokio::test] -async fn test_string_expressions_batch1() -> Result<()> { - test_expression!("ascii('')", "0"); - test_expression!("ascii('x')", "120"); - test_expression!("ascii(NULL)", "NULL"); - test_expression!("bit_length('')", "0"); - test_expression!("bit_length('chars')", "40"); - test_expression!("bit_length('josé')", "40"); - test_expression!("bit_length(NULL)", "NULL"); - test_expression!("btrim(' xyxtrimyyx ', NULL)", "NULL"); - test_expression!("btrim(' xyxtrimyyx ')", "xyxtrimyyx"); - test_expression!("btrim('\n xyxtrimyyx \n')", "\n xyxtrimyyx \n"); - test_expression!("btrim('xyxtrimyyx', 'xyz')", "trim"); - test_expression!("btrim('\nxyxtrimyyx\n', 'xyz\n')", "trim"); - test_expression!("btrim(NULL, 'xyz')", "NULL"); - test_expression!("chr(CAST(120 AS int))", "x"); - test_expression!("chr(CAST(128175 AS int))", "💯"); - test_expression!("chr(CAST(NULL AS int))", "NULL"); - test_expression!("concat('a','b','c')", "abc"); - test_expression!("concat('abcde', 2, NULL, 22)", "abcde222"); - test_expression!("concat(NULL)", ""); - test_expression!("concat_ws(',', 'abcde', 2, NULL, 22)", "abcde,2,22"); - test_expression!("concat_ws('|','a','b','c')", "a|b|c"); - test_expression!("concat_ws('|',NULL)", ""); - test_expression!("concat_ws(NULL,'a',NULL,'b','c')", "NULL"); - test_expression!("concat_ws('|','a',NULL)", "a"); - test_expression!("concat_ws('|','a',NULL,NULL)", "a"); - test_expression!("initcap('')", ""); - test_expression!("initcap('hi THOMAS')", "Hi Thomas"); - test_expression!("initcap(NULL)", "NULL"); - test_expression!("lower('')", ""); - test_expression!("lower('TOM')", "tom"); - test_expression!("lower(NULL)", "NULL"); - test_expression!("ltrim(' zzzytest ', NULL)", "NULL"); - test_expression!("ltrim(' zzzytest ')", "zzzytest "); - test_expression!("ltrim('zzzytest', 'xyz')", "test"); - test_expression!("ltrim(NULL, 'xyz')", "NULL"); - test_expression!("octet_length('')", "0"); - test_expression!("octet_length('chars')", "5"); - test_expression!("octet_length('josé')", "5"); - test_expression!("octet_length(NULL)", "NULL"); - test_expression!("repeat('Pg', 4)", "PgPgPgPg"); - test_expression!("repeat('Pg', CAST(NULL AS INT))", "NULL"); - test_expression!("repeat(NULL, 4)", "NULL"); - test_expression!("replace('abcdefabcdef', 'cd', 'XX')", "abXXefabXXef"); - test_expression!("replace('abcdefabcdef', 'cd', NULL)", "NULL"); - test_expression!("replace('abcdefabcdef', 'notmatch', 'XX')", "abcdefabcdef"); - test_expression!("replace('abcdefabcdef', NULL, 'XX')", "NULL"); - test_expression!("replace(NULL, 'cd', 'XX')", "NULL"); - test_expression!("rtrim(' testxxzx ')", " testxxzx"); - test_expression!("rtrim(' zzzytest ', NULL)", "NULL"); - test_expression!("rtrim('testxxzx', 'xyz')", "test"); - test_expression!("rtrim(NULL, 'xyz')", "NULL"); - Ok(()) -} - -/// Test string expressions test split into two batches -/// to prevent stack overflow error -#[tokio::test] -async fn test_string_expressions_batch2() -> Result<()> { - test_expression!("split_part('abc~@~def~@~ghi', '~@~', 2)", "def"); - test_expression!("split_part('abc~@~def~@~ghi', '~@~', 20)", ""); - test_expression!("split_part(NULL, '~@~', 20)", "NULL"); - test_expression!("split_part('abc~@~def~@~ghi', NULL, 20)", "NULL"); - test_expression!( - "split_part('abc~@~def~@~ghi', '~@~', CAST(NULL AS INT))", - "NULL" - ); - test_expression!("starts_with('alphabet', 'alph')", "true"); - test_expression!("starts_with('alphabet', 'blph')", "false"); - test_expression!("starts_with(NULL, 'blph')", "NULL"); - test_expression!("starts_with('alphabet', NULL)", "NULL"); - test_expression!("to_hex(2147483647)", "7fffffff"); - test_expression!("to_hex(9223372036854775807)", "7fffffffffffffff"); - test_expression!("to_hex(CAST(NULL AS int))", "NULL"); - test_expression!("trim(' tom ')", "tom"); - test_expression!("trim(LEADING ' tom ')", "tom "); - test_expression!("trim(TRAILING ' tom ')", " tom"); - test_expression!("trim(BOTH ' tom ')", "tom"); - test_expression!("trim(LEADING ' ' FROM ' tom ')", "tom "); - test_expression!("trim(TRAILING ' ' FROM ' tom ')", " tom"); - test_expression!("trim(BOTH ' ' FROM ' tom ')", "tom"); - test_expression!("trim(' ' FROM ' tom ')", "tom"); - test_expression!("trim(LEADING 'x' FROM 'xxxtomxxx')", "tomxxx"); - test_expression!("trim(TRAILING 'x' FROM 'xxxtomxxx')", "xxxtom"); - test_expression!("trim(BOTH 'x' FROM 'xxxtomxx')", "tom"); - test_expression!("trim('x' FROM 'xxxtomxx')", "tom"); - test_expression!("trim(LEADING 'xy' FROM 'xyxabcxyzdefxyx')", "abcxyzdefxyx"); - test_expression!("trim(TRAILING 'xy' FROM 'xyxabcxyzdefxyx')", "xyxabcxyzdef"); - test_expression!("trim(BOTH 'xy' FROM 'xyxabcxyzdefxyx')", "abcxyzdef"); - test_expression!("trim('xy' FROM 'xyxabcxyzdefxyx')", "abcxyzdef"); - test_expression!("trim(' tom')", "tom"); - test_expression!("trim('')", ""); - test_expression!("trim('tom ')", "tom"); - test_expression!("upper('')", ""); - test_expression!("upper('tom')", "TOM"); - test_expression!("upper(NULL)", "NULL"); - Ok(()) -} - #[tokio::test] #[cfg_attr(not(feature = "regex_expressions"), ignore)] async fn test_regex_expressions() -> Result<()> { @@ -593,329 +198,6 @@ async fn test_regex_expressions() -> Result<()> { Ok(()) } -#[tokio::test] -async fn test_cast_expressions() -> Result<()> { - test_expression!("CAST('0' AS INT)", "0"); - test_expression!("CAST(NULL AS INT)", "NULL"); - test_expression!("TRY_CAST('0' AS INT)", "0"); - test_expression!("TRY_CAST('x' AS INT)", "NULL"); - Ok(()) -} - -#[tokio::test] -#[ignore] -// issue: https://github.com/apache/arrow-datafusion/issues/6596 -async fn test_array_cast_expressions() -> Result<()> { - test_expression!("CAST([1,2,3,4] AS INT[])", "[1, 2, 3, 4]"); - test_expression!( - "CAST([1,2,3,4] AS NUMERIC(10,4)[])", - "[1.0000, 2.0000, 3.0000, 4.0000]" - ); - Ok(()) -} - -#[tokio::test] -async fn test_random_expression() -> Result<()> { - let ctx = SessionContext::new(); - let sql = "SELECT random() r1"; - let actual = execute(&ctx, sql).await; - let r1 = actual[0][0].parse::().unwrap(); - assert!(0.0 <= r1); - assert!(r1 < 1.0); - Ok(()) -} - -#[tokio::test] -async fn test_uuid_expression() -> Result<()> { - let ctx = SessionContext::new(); - let sql = "SELECT uuid()"; - let actual = execute(&ctx, sql).await; - let uuid = actual[0][0].parse::().unwrap(); - assert_eq!(uuid.get_version_num(), 4); - Ok(()) -} - -#[tokio::test] -async fn test_extract_date_part() -> Result<()> { - test_expression!("date_part('YEAR', CAST('2000-01-01' AS DATE))", "2000.0"); - test_expression!( - "EXTRACT(year FROM timestamp '2020-09-08T12:00:00+00:00')", - "2020.0" - ); - test_expression!("date_part('QUARTER', CAST('2000-01-01' AS DATE))", "1.0"); - test_expression!( - "EXTRACT(quarter FROM to_timestamp('2020-09-08T12:00:00+00:00'))", - "3.0" - ); - test_expression!("date_part('MONTH', CAST('2000-01-01' AS DATE))", "1.0"); - test_expression!( - "EXTRACT(month FROM to_timestamp('2020-09-08T12:00:00+00:00'))", - "9.0" - ); - test_expression!("date_part('WEEK', CAST('2003-01-01' AS DATE))", "1.0"); - test_expression!( - "EXTRACT(WEEK FROM to_timestamp('2020-09-08T12:00:00+00:00'))", - "37.0" - ); - test_expression!("date_part('DAY', CAST('2000-01-01' AS DATE))", "1.0"); - test_expression!( - "EXTRACT(day FROM to_timestamp('2020-09-08T12:00:00+00:00'))", - "8.0" - ); - test_expression!("date_part('DOY', CAST('2000-01-01' AS DATE))", "1.0"); - test_expression!( - "EXTRACT(doy FROM to_timestamp('2020-09-08T12:00:00+00:00'))", - "252.0" - ); - test_expression!("date_part('DOW', CAST('2000-01-01' AS DATE))", "6.0"); - test_expression!( - "EXTRACT(dow FROM to_timestamp('2020-09-08T12:00:00+00:00'))", - "2.0" - ); - test_expression!("date_part('HOUR', CAST('2000-01-01' AS DATE))", "0.0"); - test_expression!( - "EXTRACT(hour FROM to_timestamp('2020-09-08T12:03:03+00:00'))", - "12.0" - ); - test_expression!( - "EXTRACT(minute FROM to_timestamp('2020-09-08T12:12:00+00:00'))", - "12.0" - ); - test_expression!( - "date_part('minute', to_timestamp('2020-09-08T12:12:00+00:00'))", - "12.0" - ); - test_expression!( - "EXTRACT(second FROM timestamp '2020-09-08T12:00:12.12345678+00:00')", - "12.12345678" - ); - test_expression!( - "EXTRACT(millisecond FROM timestamp '2020-09-08T12:00:12.12345678+00:00')", - "12123.45678" - ); - test_expression!( - "EXTRACT(microsecond FROM timestamp '2020-09-08T12:00:12.12345678+00:00')", - "12123456.78" - ); - test_expression!( - "EXTRACT(nanosecond FROM timestamp '2020-09-08T12:00:12.12345678+00:00')", - "1.212345678e10" - ); - test_expression!( - "date_part('second', timestamp '2020-09-08T12:00:12.12345678+00:00')", - "12.12345678" - ); - test_expression!( - "date_part('millisecond', timestamp '2020-09-08T12:00:12.12345678+00:00')", - "12123.45678" - ); - test_expression!( - "date_part('microsecond', timestamp '2020-09-08T12:00:12.12345678+00:00')", - "12123456.78" - ); - test_expression!( - "date_part('nanosecond', timestamp '2020-09-08T12:00:12.12345678+00:00')", - "1.212345678e10" - ); - - // Keep precision when coercing Utf8 to Timestamp - test_expression!( - "date_part('second', '2020-09-08T12:00:12.12345678+00:00')", - "12.12345678" - ); - test_expression!( - "date_part('millisecond', '2020-09-08T12:00:12.12345678+00:00')", - "12123.45678" - ); - test_expression!( - "date_part('microsecond', '2020-09-08T12:00:12.12345678+00:00')", - "12123456.78" - ); - test_expression!( - "date_part('nanosecond', '2020-09-08T12:00:12.12345678+00:00')", - "1.212345678e10" - ); - - Ok(()) -} - -#[tokio::test] -async fn test_extract_epoch() -> Result<()> { - // timestamp - test_expression!( - "extract(epoch from '1870-01-01T07:29:10.256'::timestamp)", - "-3155646649.744" - ); - test_expression!( - "extract(epoch from '2000-01-01T00:00:00.000'::timestamp)", - "946684800.0" - ); - test_expression!( - "extract(epoch from to_timestamp('2000-01-01T00:00:00+00:00'))", - "946684800.0" - ); - test_expression!("extract(epoch from NULL::timestamp)", "NULL"); - // date - test_expression!( - "extract(epoch from arrow_cast('1970-01-01', 'Date32'))", - "0.0" - ); - test_expression!( - "extract(epoch from arrow_cast('1970-01-02', 'Date32'))", - "86400.0" - ); - test_expression!( - "extract(epoch from arrow_cast('1970-01-11', 'Date32'))", - "864000.0" - ); - test_expression!( - "extract(epoch from arrow_cast('1969-12-31', 'Date32'))", - "-86400.0" - ); - test_expression!( - "extract(epoch from arrow_cast('1970-01-01', 'Date64'))", - "0.0" - ); - test_expression!( - "extract(epoch from arrow_cast('1970-01-02', 'Date64'))", - "86400.0" - ); - test_expression!( - "extract(epoch from arrow_cast('1970-01-11', 'Date64'))", - "864000.0" - ); - test_expression!( - "extract(epoch from arrow_cast('1969-12-31', 'Date64'))", - "-86400.0" - ); - Ok(()) -} - -#[tokio::test] -async fn test_extract_date_part_func() -> Result<()> { - test_expression!( - format!( - "(date_part('{0}', now()) = EXTRACT({0} FROM now()))", - "year" - ), - "true" - ); - test_expression!( - format!( - "(date_part('{0}', now()) = EXTRACT({0} FROM now()))", - "quarter" - ), - "true" - ); - test_expression!( - format!( - "(date_part('{0}', now()) = EXTRACT({0} FROM now()))", - "month" - ), - "true" - ); - test_expression!( - format!( - "(date_part('{0}', now()) = EXTRACT({0} FROM now()))", - "week" - ), - "true" - ); - test_expression!( - format!("(date_part('{0}', now()) = EXTRACT({0} FROM now()))", "day"), - "true" - ); - test_expression!( - format!( - "(date_part('{0}', now()) = EXTRACT({0} FROM now()))", - "hour" - ), - "true" - ); - test_expression!( - format!( - "(date_part('{0}', now()) = EXTRACT({0} FROM now()))", - "minute" - ), - "true" - ); - test_expression!( - format!( - "(date_part('{0}', now()) = EXTRACT({0} FROM now()))", - "second" - ), - "true" - ); - test_expression!( - format!( - "(date_part('{0}', now()) = EXTRACT({0} FROM now()))", - "millisecond" - ), - "true" - ); - test_expression!( - format!( - "(date_part('{0}', now()) = EXTRACT({0} FROM now()))", - "microsecond" - ), - "true" - ); - test_expression!( - format!( - "(date_part('{0}', now()) = EXTRACT({0} FROM now()))", - "nanosecond" - ), - "true" - ); - - Ok(()) -} - -#[tokio::test] -async fn test_in_list_scalar() -> Result<()> { - test_expression!("'a' IN ('a','b')", "true"); - test_expression!("'c' IN ('a','b')", "false"); - test_expression!("'c' NOT IN ('a','b')", "true"); - test_expression!("'a' NOT IN ('a','b')", "false"); - test_expression!("NULL IN ('a','b')", "NULL"); - test_expression!("NULL NOT IN ('a','b')", "NULL"); - test_expression!("'a' IN ('a','b',NULL)", "true"); - test_expression!("'c' IN ('a','b',NULL)", "NULL"); - test_expression!("'a' NOT IN ('a','b',NULL)", "false"); - test_expression!("'c' NOT IN ('a','b',NULL)", "NULL"); - test_expression!("0 IN (0,1,2)", "true"); - test_expression!("3 IN (0,1,2)", "false"); - test_expression!("3 NOT IN (0,1,2)", "true"); - test_expression!("0 NOT IN (0,1,2)", "false"); - test_expression!("NULL IN (0,1,2)", "NULL"); - test_expression!("NULL NOT IN (0,1,2)", "NULL"); - test_expression!("0 IN (0,1,2,NULL)", "true"); - test_expression!("3 IN (0,1,2,NULL)", "NULL"); - test_expression!("0 NOT IN (0,1,2,NULL)", "false"); - test_expression!("3 NOT IN (0,1,2,NULL)", "NULL"); - test_expression!("0.0 IN (0.0,0.1,0.2)", "true"); - test_expression!("0.3 IN (0.0,0.1,0.2)", "false"); - test_expression!("0.3 NOT IN (0.0,0.1,0.2)", "true"); - test_expression!("0.0 NOT IN (0.0,0.1,0.2)", "false"); - test_expression!("NULL IN (0.0,0.1,0.2)", "NULL"); - test_expression!("NULL NOT IN (0.0,0.1,0.2)", "NULL"); - test_expression!("0.0 IN (0.0,0.1,0.2,NULL)", "true"); - test_expression!("0.3 IN (0.0,0.1,0.2,NULL)", "NULL"); - test_expression!("0.0 NOT IN (0.0,0.1,0.2,NULL)", "false"); - test_expression!("0.3 NOT IN (0.0,0.1,0.2,NULL)", "NULL"); - test_expression!("'1' IN ('a','b',1)", "true"); - test_expression!("'2' IN ('a','b',1)", "false"); - test_expression!("'2' NOT IN ('a','b',1)", "true"); - test_expression!("'1' NOT IN ('a','b',1)", "false"); - test_expression!("NULL IN ('a','b',1)", "NULL"); - test_expression!("NULL NOT IN ('a','b',1)", "NULL"); - test_expression!("'1' IN ('a','b',NULL,1)", "true"); - test_expression!("'2' IN ('a','b',NULL,1)", "NULL"); - test_expression!("'1' NOT IN ('a','b',NULL,1)", "false"); - test_expression!("'2' NOT IN ('a','b',NULL,1)", "NULL"); - Ok(()) -} - #[tokio::test] async fn csv_query_nullif_divide_by_0() -> Result<()> { let ctx = SessionContext::new(); diff --git a/datafusion/sqllogictest/test_files/expr.slt b/datafusion/sqllogictest/test_files/expr.slt new file mode 100644 index 000000000000..a2a8d9c6475c --- /dev/null +++ b/datafusion/sqllogictest/test_files/expr.slt @@ -0,0 +1,1251 @@ +# 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. + +# test_boolean_expressions +query BBBB +SELECT true, false, false = false, true = false +---- +true false true false + +# test_mathematical_expressions_with_null +query RRRRRRRRRRRRRRRRRR?RRRRRRRIRRRRRRBB +SELECT + sqrt(NULL), + cbrt(NULL), + sin(NULL), + cos(NULL), + tan(NULL), + asin(NULL), + acos(NULL), + atan(NULL), + sinh(NULL), + cosh(NULL), + tanh(NULL), + asinh(NULL), + acosh(NULL), + atanh(NULL), + floor(NULL), + ceil(NULL), + round(NULL), + trunc(NULL), + abs(NULL), + signum(NULL), + exp(NULL), + ln(NULL), + log2(NULL), + log10(NULL), + power(NULL, 2), + power(NULL, NULL), + power(2, NULL), + atan2(NULL, NULL), + atan2(1, NULL), + atan2(NULL, 1), + nanvl(NULL, NULL), + nanvl(1, NULL), + nanvl(NULL, 1), + isnan(NULL), + iszero(NULL) +---- +NULL NULL NULL NULL NULL NULL NULL NULL NULL NULL NULL NULL NULL NULL NULL NULL NULL NULL NULL NULL NULL NULL NULL NULL NULL NULL NULL NULL NULL NULL NULL NULL NULL NULL NULL + +# test_array_cast_invalid_timezone_will_panic +statement error Parser error: Invalid timezone "Foo": 'Foo' is not a valid timezone +SELECT arrow_cast('2021-01-02T03:04:00', 'Timestamp(Nanosecond, Some("Foo"))') + +# test_array_index +query III??IIIIII +SELECT + ([5,4,3,2,1])[1], + ([5,4,3,2,1])[2], + ([5,4,3,2,1])[5], + ([[1, 2], [2, 3], [3,4]])[1], + ([[1, 2], [2, 3], [3,4]])[3], + ([[1, 2], [2, 3], [3,4]])[1][1], + ([[1, 2], [2, 3], [3,4]])[2][2], + ([[1, 2], [2, 3], [3,4]])[3][2], + -- out of bounds + ([5,4,3,2,1])[0], + ([5,4,3,2,1])[6], + -- ([5,4,3,2,1])[-1], -- TODO: wrong answer + -- ([5,4,3,2,1])[null], -- TODO: not supported + ([5,4,3,2,1])[100] +---- +5 4 1 [1, 2] [3, 4] 1 3 4 NULL NULL NULL + +# test_array_literals +query ????? +SELECT + [1,2,3,4,5], + [true, false], + ['str1', 'str2'], + [[1,2], [3,4]], + [] +---- +[1, 2, 3, 4, 5] [true, false] [str1, str2] [[1, 2], [3, 4]] [] + +# test_struct_literals +query ?????? +SELECT + STRUCT(1,2,3,4,5), + STRUCT(Null), + STRUCT(2), + STRUCT('1',Null), + STRUCT(true, false), + STRUCT('str1', 'str2') +---- +{c0: 1, c1: 2, c2: 3, c3: 4, c4: 5} {c0: } {c0: 2} {c0: 1, c1: } {c0: true, c1: false} {c0: str1, c1: str2} + +# test binary_bitwise_shift +query IIII +SELECT + 2 << 10, + 2048 >> 10, + 2048 << NULL, + 2048 >> NULL +---- +2048 2 NULL NULL + +query ? +SELECT interval '1' +---- +0 years 0 mons 0 days 0 hours 0 mins 1.000000000 secs + +query ? +SELECT interval '1 second' +---- +0 years 0 mons 0 days 0 hours 0 mins 1.000000000 secs + +query ? +SELECT interval '500 milliseconds' +---- +0 years 0 mons 0 days 0 hours 0 mins 0.500000000 secs + +query ? +SELECT interval '5 second' +---- +0 years 0 mons 0 days 0 hours 0 mins 5.000000000 secs + +query ? +SELECT interval '0.5 minute' +---- +0 years 0 mons 0 days 0 hours 0 mins 30.000000000 secs + +query ? +SELECT interval '.5 minute' +---- +0 years 0 mons 0 days 0 hours 0 mins 30.000000000 secs + +query ? +SELECT interval '5 minute' +---- +0 years 0 mons 0 days 0 hours 5 mins 0.000000000 secs + +query ? +SELECT interval '5 minute 1 second' +---- +0 years 0 mons 0 days 0 hours 5 mins 1.000000000 secs + +query ? +SELECT interval '1 hour' +---- +0 years 0 mons 0 days 1 hours 0 mins 0.000000000 secs + +query ? +SELECT interval '5 hour' +---- +0 years 0 mons 0 days 5 hours 0 mins 0.000000000 secs + +query ? +SELECT interval '1 day' +---- +0 years 0 mons 1 days 0 hours 0 mins 0.000000000 secs + +query ? +SELECT interval '1 week' +---- +0 years 0 mons 7 days 0 hours 0 mins 0.000000000 secs + +query ? +SELECT interval '2 weeks' +---- +0 years 0 mons 14 days 0 hours 0 mins 0.000000000 secs + +query ? +SELECT interval '1 day 1' +---- +0 years 0 mons 1 days 0 hours 0 mins 1.000000000 secs + +query ? +SELECT interval '0.5' +---- +0 years 0 mons 0 days 0 hours 0 mins 0.500000000 secs + +query ? +SELECT interval '0.5 day 1' +---- +0 years 0 mons 0 days 12 hours 0 mins 1.000000000 secs + +query ? +SELECT interval '0.49 day' +---- +0 years 0 mons 0 days 11 hours 45 mins 36.000000000 secs + +query ? +SELECT interval '0.499 day' +---- +0 years 0 mons 0 days 11 hours 58 mins 33.600000000 secs + +query ? +SELECT interval '0.4999 day' +---- +0 years 0 mons 0 days 11 hours 59 mins 51.360000000 secs + +query ? +SELECT interval '0.49999 day' +---- +0 years 0 mons 0 days 11 hours 59 mins 59.136000000 secs + +query ? +SELECT interval '0.49999999999 day' +---- +0 years 0 mons 0 days 11 hours 59 mins 59.999999136 secs + +query ? +SELECT interval '5 day' +---- +0 years 0 mons 5 days 0 hours 0 mins 0.000000000 secs + +# Hour is ignored, this matches PostgreSQL +query ? +SELECT interval '5 day' hour +---- +0 years 0 mons 5 days 0 hours 0 mins 0.000000000 secs + +query ? +SELECT interval '5 day 4 hours 3 minutes 2 seconds 100 milliseconds' +---- +0 years 0 mons 5 days 4 hours 3 mins 2.100000000 secs + +query ? +SELECT interval '0.5 month' +---- +0 years 0 mons 15 days 0 hours 0 mins 0.000000000 secs + +query ? +SELECT interval '0.5' month +---- +0 years 0 mons 15 days 0 hours 0 mins 0.000000000 secs + +query ? +SELECT interval '1 month' +---- +0 years 1 mons 0 days 0 hours 0 mins 0.000000000 secs + +query ? +SELECT interval '1' MONTH +---- +0 years 1 mons 0 days 0 hours 0 mins 0.000000000 secs + +query ? +SELECT interval '5 month' +---- +0 years 5 mons 0 days 0 hours 0 mins 0.000000000 secs + +query ? +SELECT interval '13 month' +---- +0 years 13 mons 0 days 0 hours 0 mins 0.000000000 secs + +query ? +SELECT interval '0.5 year' +---- +0 years 6 mons 0 days 0 hours 0 mins 0.000000000 secs + +query ? +SELECT interval '1 year' +---- +0 years 12 mons 0 days 0 hours 0 mins 0.000000000 secs + +query ? +SELECT interval '1 decade' +---- +0 years 120 mons 0 days 0 hours 0 mins 0.000000000 secs + +query ? +SELECT interval '2 decades' +---- +0 years 240 mons 0 days 0 hours 0 mins 0.000000000 secs + +query ? +SELECT interval '1 century' +---- +0 years 1200 mons 0 days 0 hours 0 mins 0.000000000 secs + +query ? +SELECT interval '2 year' +---- +0 years 24 mons 0 days 0 hours 0 mins 0.000000000 secs + +query ? +SELECT interval '1 year 1 day' +---- +0 years 12 mons 1 days 0 hours 0 mins 0.000000000 secs + +query ? +SELECT interval '1 year 1 day 1 hour' +---- +0 years 12 mons 1 days 1 hours 0 mins 0.000000000 secs + +query ? +SELECT interval '1 year 1 day 1 hour 1 minute' +---- +0 years 12 mons 1 days 1 hours 1 mins 0.000000000 secs + +query ? +SELECT interval '1 year 1 day 1 hour 1 minute 1 second' +---- +0 years 12 mons 1 days 1 hours 1 mins 1.000000000 secs + +query I +SELECT ascii('') +---- +0 + +query I +SELECT ascii('x') +---- +120 + +query I +SELECT ascii(NULL) +---- +NULL + +query I +SELECT bit_length('') +---- +0 + +query I +SELECT bit_length('chars') +---- +40 + +query I +SELECT bit_length('josé') +---- +40 + +query ? +SELECT bit_length(NULL) +---- +NULL + +query T +SELECT btrim(' xyxtrimyyx ', NULL) +---- +NULL + +query T +SELECT btrim(' xyxtrimyyx ') +---- +xyxtrimyyx + +query T +SELECT btrim('\n xyxtrimyyx \n') +---- +\n xyxtrimyyx \n + +query T +SELECT btrim('xyxtrimyyx', 'xyz') +---- +trim + +query T +SELECT btrim('\nxyxtrimyyx\n', 'xyz\n') +---- +trim + +query ? +SELECT btrim(NULL, 'xyz') +---- +NULL + +query T +SELECT chr(CAST(120 AS int)) +---- +x + +query T +SELECT chr(CAST(128175 AS int)) +---- +💯 + +query T +SELECT chr(CAST(NULL AS int)) +---- +NULL + +query T +SELECT concat('a','b','c') +---- +abc + +query T +SELECT concat('abcde', 2, NULL, 22) +---- +abcde222 + +query T +SELECT concat(NULL) +---- +(empty) + +query T +SELECT concat_ws(',', 'abcde', 2, NULL, 22) +---- +abcde,2,22 + +query T +SELECT concat_ws('|','a','b','c') +---- +a|b|c + +query T +SELECT concat_ws('|',NULL) +---- +(empty) + +query T +SELECT concat_ws(NULL,'a',NULL,'b','c') +---- +NULL + +query T +SELECT concat_ws('|','a',NULL) +---- +a + +query T +SELECT concat_ws('|','a',NULL,NULL) +---- +a + +query T +SELECT initcap('') +---- +(empty) + +query T +SELECT initcap('hi THOMAS') +---- +Hi Thomas + +query ? +SELECT initcap(NULL) +---- +NULL + +query T +SELECT lower('') +---- +(empty) + +query T +SELECT lower('TOM') +---- +tom + +query ? +SELECT lower(NULL) +---- +NULL + +query T +SELECT ltrim(' zzzytest ', NULL) +---- +NULL + +query T +SELECT ltrim(' zzzytest ') +---- +zzzytest + +query T +SELECT ltrim('zzzytest', 'xyz') +---- +test + +query ? +SELECT ltrim(NULL, 'xyz') +---- +NULL + +query I +SELECT octet_length('') +---- +0 + +query I +SELECT octet_length('chars') +---- +5 + +query I +SELECT octet_length('josé') +---- +5 + +query ? +SELECT octet_length(NULL) +---- +NULL + +query T +SELECT repeat('Pg', 4) +---- +PgPgPgPg + +query T +SELECT repeat('Pg', CAST(NULL AS INT)) +---- +NULL + +query ? +SELECT repeat(NULL, 4) +---- +NULL + +query T +SELECT replace('abcdefabcdef', 'cd', 'XX') +---- +abXXefabXXef + +query T +SELECT replace('abcdefabcdef', 'cd', NULL) +---- +NULL + +query T +SELECT replace('abcdefabcdef', 'notmatch', 'XX') +---- +abcdefabcdef + +query T +SELECT replace('abcdefabcdef', NULL, 'XX') +---- +NULL + +query ? +SELECT replace(NULL, 'cd', 'XX') +---- +NULL + +query T +SELECT rtrim(' testxxzx ') +---- + testxxzx + +query T +SELECT rtrim(' zzzytest ', NULL) +---- +NULL + +query T +SELECT rtrim('testxxzx', 'xyz') +---- +test + +query ? +SELECT rtrim(NULL, 'xyz') +---- +NULL + +query T +SELECT split_part('abc~@~def~@~ghi', '~@~', 2) +---- +def + +query T +SELECT split_part('abc~@~def~@~ghi', '~@~', 20) +---- +(empty) + +query ? +SELECT split_part(NULL, '~@~', 20) +---- +NULL + +query T +SELECT split_part('abc~@~def~@~ghi', NULL, 20) +---- +NULL + +query T +SELECT split_part('abc~@~def~@~ghi', '~@~', CAST(NULL AS INT)) +---- +NULL + +query B +SELECT starts_with('alphabet', 'alph') +---- +true + +query B +SELECT starts_with('alphabet', 'blph') +---- +false + +query B +SELECT starts_with(NULL, 'blph') +---- +NULL + +query B +SELECT starts_with('alphabet', NULL) +---- +NULL + +query T +SELECT to_hex(2147483647) +---- +7fffffff + +query T +SELECT to_hex(9223372036854775807) +---- +7fffffffffffffff + +query T +SELECT to_hex(CAST(NULL AS int)) +---- +NULL + +query T +SELECT trim(' tom ') +---- +tom + +query T +SELECT trim(LEADING ' tom ') +---- +tom + +query T +SELECT trim(TRAILING ' tom ') +---- + tom + +query T +SELECT trim(BOTH ' tom ') +---- +tom + +query T +SELECT trim(LEADING ' ' FROM ' tom ') +---- +tom + +query T +SELECT trim(TRAILING ' ' FROM ' tom ') +---- + tom + +query T +SELECT trim(BOTH ' ' FROM ' tom ') +---- +tom + +query T +SELECT trim(' ' FROM ' tom ') +---- +tom + +query T +SELECT trim(LEADING 'x' FROM 'xxxtomxxx') +---- +tomxxx + +query T +SELECT trim(TRAILING 'x' FROM 'xxxtomxxx') +---- +xxxtom + +query T +SELECT trim(BOTH 'x' FROM 'xxxtomxx') +---- +tom + +query T +SELECT trim('x' FROM 'xxxtomxx') +---- +tom + + +query T +SELECT trim(LEADING 'xy' FROM 'xyxabcxyzdefxyx') +---- +abcxyzdefxyx + +query T +SELECT trim(TRAILING 'xy' FROM 'xyxabcxyzdefxyx') +---- +xyxabcxyzdef + +query T +SELECT trim(BOTH 'xy' FROM 'xyxabcxyzdefxyx') +---- +abcxyzdef + +query T +SELECT trim('xy' FROM 'xyxabcxyzdefxyx') +---- +abcxyzdef + +query T +SELECT trim(' tom') +---- +tom + +query T +SELECT trim('') +---- +(empty) + +query T +SELECT trim('tom ') +---- +tom + +query T +SELECT upper('') +---- +(empty) + +query T +SELECT upper('tom') +---- +TOM + +query ? +SELECT upper(NULL) +---- +NULL + +# TODO issue: https://github.com/apache/arrow-datafusion/issues/6596 +# query ?? +#SELECT +# CAST([1,2,3,4] AS INT[]) as a, +# CAST([1,2,3,4] AS NUMERIC(10,4)[]) as b +#---- +#[1, 2, 3, 4] [1.0000, 2.0000, 3.0000, 4.0000] + +# test_random_expression +query BB +SELECT + random() BETWEEN 0.0 AND 1.0, + random() = random() +---- +true false + +# test_uuid_expression +query II +SELECT octet_length(uuid()), length(uuid()) +---- +36 36 + +# test_cast_expressions +query IIII +SELECT + CAST('0' AS INT) as a, + CAST(NULL AS INT) as b, + TRY_CAST('0' AS INT) as c, + TRY_CAST('x' AS INT) as d +---- +0 NULL 0 NULL + +# test_extract_date_part + +query R +SELECT date_part('YEAR', CAST('2000-01-01' AS DATE)) +---- +2000 + +query R +SELECT EXTRACT(year FROM timestamp '2020-09-08T12:00:00+00:00') +---- +2020 + +query R +SELECT date_part('QUARTER', CAST('2000-01-01' AS DATE)) +---- +1 + +query R +SELECT EXTRACT(quarter FROM to_timestamp('2020-09-08T12:00:00+00:00')) +---- +3 + +query R +SELECT date_part('MONTH', CAST('2000-01-01' AS DATE)) +---- +1 + +query R +SELECT EXTRACT(month FROM to_timestamp('2020-09-08T12:00:00+00:00')) +---- +9 + +query R +SELECT date_part('WEEK', CAST('2003-01-01' AS DATE)) +---- +1 + +query R +SELECT EXTRACT(WEEK FROM to_timestamp('2020-09-08T12:00:00+00:00')) +---- +37 + +query R +SELECT date_part('DAY', CAST('2000-01-01' AS DATE)) +---- +1 + +query R +SELECT EXTRACT(day FROM to_timestamp('2020-09-08T12:00:00+00:00')) +---- +8 + +query R +SELECT date_part('DOY', CAST('2000-01-01' AS DATE)) +---- +1 + +query R +SELECT EXTRACT(doy FROM to_timestamp('2020-09-08T12:00:00+00:00')) +---- +252 + +query R +SELECT date_part('DOW', CAST('2000-01-01' AS DATE)) +---- +6 + +query R +SELECT EXTRACT(dow FROM to_timestamp('2020-09-08T12:00:00+00:00')) +---- +2 + +query R +SELECT date_part('HOUR', CAST('2000-01-01' AS DATE)) +---- +0 + +query R +SELECT EXTRACT(hour FROM to_timestamp('2020-09-08T12:03:03+00:00')) +---- +12 + +query R +SELECT EXTRACT(minute FROM to_timestamp('2020-09-08T12:12:00+00:00')) +---- +12 + +query R +SELECT date_part('minute', to_timestamp('2020-09-08T12:12:00+00:00')) +---- +12 + +query R +SELECT EXTRACT(second FROM timestamp '2020-09-08T12:00:12.12345678+00:00') +---- +12.12345678 + +query R +SELECT EXTRACT(millisecond FROM timestamp '2020-09-08T12:00:12.12345678+00:00') +---- +12123.45678 + +query R +SELECT EXTRACT(microsecond FROM timestamp '2020-09-08T12:00:12.12345678+00:00') +---- +12123456.78 + +query R +SELECT EXTRACT(nanosecond FROM timestamp '2020-09-08T12:00:12.12345678+00:00') +---- +12123456780 + +# Keep precision when coercing Utf8 to Timestamp +query R +SELECT date_part('second', timestamp '2020-09-08T12:00:12.12345678+00:00') +---- +12.12345678 + +query R +SELECT date_part('millisecond', timestamp '2020-09-08T12:00:12.12345678+00:00') +---- +12123.45678 + +query R +SELECT date_part('microsecond', timestamp '2020-09-08T12:00:12.12345678+00:00') +---- +12123456.78 + +query R +SELECT date_part('nanosecond', timestamp '2020-09-08T12:00:12.12345678+00:00') +---- +12123456780 + +query R +SELECT date_part('second', '2020-09-08T12:00:12.12345678+00:00') +---- +12.12345678 + +query R +SELECT date_part('millisecond', '2020-09-08T12:00:12.12345678+00:00') +---- +12123.45678 + +query R +SELECT date_part('microsecond', '2020-09-08T12:00:12.12345678+00:00') +---- +12123456.78 + +query R +SELECT date_part('nanosecond', '2020-09-08T12:00:12.12345678+00:00') +---- +12123456780 + +# test_extract_epoch + +query R +SELECT extract(epoch from '1870-01-01T07:29:10.256'::timestamp) +---- +-3155646649.744 + +query R +SELECT extract(epoch from '2000-01-01T00:00:00.000'::timestamp) +---- +946684800 + +query R +SELECT extract(epoch from to_timestamp('2000-01-01T00:00:00+00:00')) +---- +946684800 + +query R +SELECT extract(epoch from NULL::timestamp) +---- +NULL + +query R +SELECT extract(epoch from arrow_cast('1970-01-01', 'Date32')) +---- +0 + +query R +SELECT extract(epoch from arrow_cast('1970-01-02', 'Date32')) +---- +86400 + +query R +SELECT extract(epoch from arrow_cast('1970-01-11', 'Date32')) +---- +864000 + +query R +SELECT extract(epoch from arrow_cast('1969-12-31', 'Date32')) +---- +-86400 + +query R +SELECT extract(epoch from arrow_cast('1970-01-01', 'Date64')) +---- +0 + +query R +SELECT extract(epoch from arrow_cast('1970-01-02', 'Date64')) +---- +86400 + +query R +SELECT extract(epoch from arrow_cast('1970-01-11', 'Date64')) +---- +864000 + +query R +SELECT extract(epoch from arrow_cast('1969-12-31', 'Date64')) +---- +-86400 + +# test_extract_date_part_func + +query B +SELECT (date_part('year', now()) = EXTRACT(year FROM now())) +---- +true + +query B +SELECT (date_part('quarter', now()) = EXTRACT(quarter FROM now())) +---- +true + +query B +SELECT (date_part('month', now()) = EXTRACT(month FROM now())) +---- +true + +query B +SELECT (date_part('week', now()) = EXTRACT(week FROM now())) +---- +true + +query B +SELECT (date_part('day', now()) = EXTRACT(day FROM now())) +---- +true + +query B +SELECT (date_part('hour', now()) = EXTRACT(hour FROM now())) +---- +true + +query B +SELECT (date_part('minute', now()) = EXTRACT(minute FROM now())) +---- +true + +query B +SELECT (date_part('second', now()) = EXTRACT(second FROM now())) +---- +true + +query B +SELECT (date_part('millisecond', now()) = EXTRACT(millisecond FROM now())) +---- +true + +query B +SELECT (date_part('microsecond', now()) = EXTRACT(microsecond FROM now())) +---- +true + +query B +SELECT (date_part('nanosecond', now()) = EXTRACT(nanosecond FROM now())) +---- +true + +query B +SELECT 'a' IN ('a','b') +---- +true + +query B +SELECT 'c' IN ('a','b') +---- +false + +query B +SELECT 'c' NOT IN ('a','b') +---- +true + +query B +SELECT 'a' NOT IN ('a','b') +---- +false + +query B +SELECT NULL IN ('a','b') +---- +NULL + +query B +SELECT NULL NOT IN ('a','b') +---- +NULL + +query B +SELECT 'a' IN ('a','b',NULL) +---- +true + +query B +SELECT 'c' IN ('a','b',NULL) +---- +NULL + +query B +SELECT 'a' NOT IN ('a','b',NULL) +---- +false + +query B +SELECT 'c' NOT IN ('a','b',NULL) +---- +NULL + +query B +SELECT 0 IN (0,1,2) +---- +true + +query B +SELECT 3 IN (0,1,2) +---- +false + +query B +SELECT 3 NOT IN (0,1,2) +---- +true + +query B +SELECT 0 NOT IN (0,1,2) +---- +false + +query B +SELECT NULL IN (0,1,2) +---- +NULL + +query B +SELECT NULL NOT IN (0,1,2) +---- +NULL + +query B +SELECT 0 IN (0,1,2,NULL) +---- +true + +query B +SELECT 3 IN (0,1,2,NULL) +---- +NULL + +query B +SELECT 0 NOT IN (0,1,2,NULL) +---- +false + +query B +SELECT 3 NOT IN (0,1,2,NULL) +---- +NULL + +query B +SELECT 0.0 IN (0.0,0.1,0.2) +---- +true + +query B +SELECT 0.3 IN (0.0,0.1,0.2) +---- +false + +query B +SELECT 0.3 NOT IN (0.0,0.1,0.2) +---- +true + +query B +SELECT 0.0 NOT IN (0.0,0.1,0.2) +---- +false + +query B +SELECT NULL IN (0.0,0.1,0.2) +---- +NULL + +query B +SELECT NULL NOT IN (0.0,0.1,0.2) +---- +NULL + +query B +SELECT 0.0 IN (0.0,0.1,0.2,NULL) +---- +true + +query B +SELECT 0.3 IN (0.0,0.1,0.2,NULL) +---- +NULL + +query B +SELECT 0.0 NOT IN (0.0,0.1,0.2,NULL) +---- +false + +query B +SELECT 0.3 NOT IN (0.0,0.1,0.2,NULL) +---- +NULL + +query B +SELECT '1' IN ('a','b',1) +---- +true + +query B +SELECT '2' IN ('a','b',1) +---- +false + +query B +SELECT '2' NOT IN ('a','b',1) +---- +true + +query B +SELECT '1' NOT IN ('a','b',1) +---- +false + +query B +SELECT NULL IN ('a','b',1) +---- +NULL + +query B +SELECT NULL NOT IN ('a','b',1) +---- +NULL + +query B +SELECT '1' IN ('a','b',NULL,1) +---- +true + +query B +SELECT '2' IN ('a','b',NULL,1) +---- +NULL + +query B +SELECT '1' NOT IN ('a','b',NULL,1) +---- +false + +query B +SELECT '2' NOT IN ('a','b',NULL,1) +---- +NULL From 128b2c64d74fbfaecbd863322a09433a71e9faa5 Mon Sep 17 00:00:00 2001 From: Matthew Gapp <61894094+matthewgapp@users.noreply.github.com> Date: Wed, 10 Jan 2024 16:45:42 -0800 Subject: [PATCH 572/572] add config flag for recursive ctes --- datafusion/common/src/config.rs | 5 +++++ datafusion/sql/src/query.rs | 13 ++++++++++++- 2 files changed, 17 insertions(+), 1 deletion(-) diff --git a/datafusion/common/src/config.rs b/datafusion/common/src/config.rs index 5b1325ec06ee..cc60f5d1ed07 100644 --- a/datafusion/common/src/config.rs +++ b/datafusion/common/src/config.rs @@ -278,6 +278,11 @@ config_namespace! { /// Hive. Note that this setting does not affect reading partitioned /// tables (e.g. `/table/year=2021/month=01/data.parquet`). pub listing_table_ignore_subdirectory: bool, default = true + + /// Should DataFusion support recursive CTEs + /// Defaults to false since this feature is a work in progress and may not + /// behave as expected + pub enable_recursive_ctes: bool, default = false } } diff --git a/datafusion/sql/src/query.rs b/datafusion/sql/src/query.rs index dd4cab126261..388377e3ee6b 100644 --- a/datafusion/sql/src/query.rs +++ b/datafusion/sql/src/query.rs @@ -54,7 +54,18 @@ impl<'a, S: ContextProvider> SqlToRel<'a, S> { // Process CTEs from top to bottom // do not allow self-references if with.recursive { - return not_impl_err!("Recursive CTEs are not supported"); + if self + .context_provider + .options() + .execution + .enable_recursive_ctes + { + return plan_err!( + "Recursive CTEs are enabled but are not yet supported" + ); + } else { + return not_impl_err!("Recursive CTEs are not supported"); + } } for cte in with.cte_tables {